beacon/light: request finality update explicitly when necessary (#29567)
This PR adds an extra mechanism to sync.HeadSync that tries to retrieve the latest finality update from every server each time it sends an optimistic update in a new epoch (unless we already have a validated finality update attested in the same epoch). Note that this is not necessary and does not happen if the new finality update is delivered before the optimistic update. The spec only mandates light_client_finality_update events when a new epoch is finalized. If the chain does not finalize for a while then we might need an explicit request that returns a finality proof that proves the same finality epoch from the latest attested epoch.
This commit is contained in:
parent
b2b0e1da8c
commit
256d4b099c
@ -41,7 +41,7 @@ type beaconBlockSync struct {
|
|||||||
|
|
||||||
type headTracker interface {
|
type headTracker interface {
|
||||||
PrefetchHead() types.HeadInfo
|
PrefetchHead() types.HeadInfo
|
||||||
ValidatedHead() (types.SignedHeader, bool)
|
ValidatedOptimistic() (types.OptimisticUpdate, bool)
|
||||||
ValidatedFinality() (types.FinalityUpdate, bool)
|
ValidatedFinality() (types.FinalityUpdate, bool)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -66,6 +66,7 @@ func (s *beaconBlockSync) Process(requester request.Requester, events []request.
|
|||||||
case request.EvResponse, request.EvFail, request.EvTimeout:
|
case request.EvResponse, request.EvFail, request.EvTimeout:
|
||||||
sid, req, resp := event.RequestInfo()
|
sid, req, resp := event.RequestInfo()
|
||||||
blockRoot := common.Hash(req.(sync.ReqBeaconBlock))
|
blockRoot := common.Hash(req.(sync.ReqBeaconBlock))
|
||||||
|
log.Debug("Beacon block event", "type", event.Type.Name, "hash", blockRoot)
|
||||||
if resp != nil {
|
if resp != nil {
|
||||||
s.recentBlocks.Add(blockRoot, resp.(*types.BeaconBlock))
|
s.recentBlocks.Add(blockRoot, resp.(*types.BeaconBlock))
|
||||||
}
|
}
|
||||||
@ -80,8 +81,8 @@ func (s *beaconBlockSync) Process(requester request.Requester, events []request.
|
|||||||
}
|
}
|
||||||
s.updateEventFeed()
|
s.updateEventFeed()
|
||||||
// request validated head block if unavailable and not yet requested
|
// request validated head block if unavailable and not yet requested
|
||||||
if vh, ok := s.headTracker.ValidatedHead(); ok {
|
if vh, ok := s.headTracker.ValidatedOptimistic(); ok {
|
||||||
s.tryRequestBlock(requester, vh.Header.Hash(), false)
|
s.tryRequestBlock(requester, vh.Attested.Hash(), false)
|
||||||
}
|
}
|
||||||
// request prefetch head if the given server has announced it
|
// request prefetch head if the given server has announced it
|
||||||
if prefetchHead := s.headTracker.PrefetchHead().BlockRoot; prefetchHead != (common.Hash{}) {
|
if prefetchHead := s.headTracker.PrefetchHead().BlockRoot; prefetchHead != (common.Hash{}) {
|
||||||
@ -114,12 +115,12 @@ func blockHeadInfo(block *types.BeaconBlock) types.HeadInfo {
|
|||||||
}
|
}
|
||||||
|
|
||||||
func (s *beaconBlockSync) updateEventFeed() {
|
func (s *beaconBlockSync) updateEventFeed() {
|
||||||
head, ok := s.headTracker.ValidatedHead()
|
optimistic, ok := s.headTracker.ValidatedOptimistic()
|
||||||
if !ok {
|
if !ok {
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
validatedHead := head.Header.Hash()
|
validatedHead := optimistic.Attested.Hash()
|
||||||
headBlock, ok := s.recentBlocks.Get(validatedHead)
|
headBlock, ok := s.recentBlocks.Get(validatedHead)
|
||||||
if !ok {
|
if !ok {
|
||||||
return
|
return
|
||||||
@ -127,7 +128,7 @@ func (s *beaconBlockSync) updateEventFeed() {
|
|||||||
|
|
||||||
var finalizedHash common.Hash
|
var finalizedHash common.Hash
|
||||||
if finality, ok := s.headTracker.ValidatedFinality(); ok {
|
if finality, ok := s.headTracker.ValidatedFinality(); ok {
|
||||||
he := head.Header.Epoch()
|
he := optimistic.Attested.Epoch()
|
||||||
fe := finality.Attested.Header.Epoch()
|
fe := finality.Attested.Header.Epoch()
|
||||||
switch {
|
switch {
|
||||||
case he == fe:
|
case he == fe:
|
||||||
@ -135,10 +136,9 @@ func (s *beaconBlockSync) updateEventFeed() {
|
|||||||
case he < fe:
|
case he < fe:
|
||||||
return
|
return
|
||||||
case he == fe+1:
|
case he == fe+1:
|
||||||
parent, ok := s.recentBlocks.Get(head.Header.ParentRoot)
|
parent, ok := s.recentBlocks.Get(optimistic.Attested.ParentRoot)
|
||||||
if !ok || parent.Slot()/params.EpochLength == fe {
|
if !ok || parent.Slot()/params.EpochLength == fe {
|
||||||
return // head is at first slot of next epoch, wait for finality update
|
return // head is at first slot of next epoch, wait for finality update
|
||||||
//TODO: try to fetch finality update directly if subscription does not deliver
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -156,7 +156,7 @@ func (s *beaconBlockSync) updateEventFeed() {
|
|||||||
return
|
return
|
||||||
}
|
}
|
||||||
s.chainHeadFeed.Send(types.ChainHeadEvent{
|
s.chainHeadFeed.Send(types.ChainHeadEvent{
|
||||||
BeaconHead: head.Header,
|
BeaconHead: optimistic.Attested.Header,
|
||||||
Block: execBlock,
|
Block: execBlock,
|
||||||
Finalized: finalizedHash,
|
Finalized: finalizedHash,
|
||||||
})
|
})
|
||||||
|
@ -140,8 +140,12 @@ func (h *testHeadTracker) PrefetchHead() types.HeadInfo {
|
|||||||
return h.prefetch
|
return h.prefetch
|
||||||
}
|
}
|
||||||
|
|
||||||
func (h *testHeadTracker) ValidatedHead() (types.SignedHeader, bool) {
|
func (h *testHeadTracker) ValidatedOptimistic() (types.OptimisticUpdate, bool) {
|
||||||
return h.validated, h.validated.Header != (types.Header{})
|
return types.OptimisticUpdate{
|
||||||
|
Attested: types.HeaderWithExecProof{Header: h.validated.Header},
|
||||||
|
Signature: h.validated.Signature,
|
||||||
|
SignatureSlot: h.validated.SignatureSlot,
|
||||||
|
}, h.validated.Header != (types.Header{})
|
||||||
}
|
}
|
||||||
|
|
||||||
// TODO add test case for finality
|
// TODO add test case for finality
|
||||||
|
@ -62,6 +62,7 @@ func (ec *engineClient) updateLoop(headCh <-chan types.ChainHeadEvent) {
|
|||||||
for {
|
for {
|
||||||
select {
|
select {
|
||||||
case <-ec.rootCtx.Done():
|
case <-ec.rootCtx.Done():
|
||||||
|
log.Debug("Stopping engine API update loop")
|
||||||
return
|
return
|
||||||
|
|
||||||
case event := <-headCh:
|
case event := <-headCh:
|
||||||
@ -73,12 +74,14 @@ func (ec *engineClient) updateLoop(headCh <-chan types.ChainHeadEvent) {
|
|||||||
fork := ec.config.ForkAtEpoch(event.BeaconHead.Epoch())
|
fork := ec.config.ForkAtEpoch(event.BeaconHead.Epoch())
|
||||||
forkName := strings.ToLower(fork.Name)
|
forkName := strings.ToLower(fork.Name)
|
||||||
|
|
||||||
|
log.Debug("Calling NewPayload", "number", event.Block.NumberU64(), "hash", event.Block.Hash())
|
||||||
if status, err := ec.callNewPayload(forkName, event); err == nil {
|
if status, err := ec.callNewPayload(forkName, event); err == nil {
|
||||||
log.Info("Successful NewPayload", "number", event.Block.NumberU64(), "hash", event.Block.Hash(), "status", status)
|
log.Info("Successful NewPayload", "number", event.Block.NumberU64(), "hash", event.Block.Hash(), "status", status)
|
||||||
} else {
|
} else {
|
||||||
log.Error("Failed NewPayload", "number", event.Block.NumberU64(), "hash", event.Block.Hash(), "error", err)
|
log.Error("Failed NewPayload", "number", event.Block.NumberU64(), "hash", event.Block.Hash(), "error", err)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
log.Debug("Calling ForkchoiceUpdated", "head", event.Block.Hash())
|
||||||
if status, err := ec.callForkchoiceUpdated(forkName, event); err == nil {
|
if status, err := ec.callForkchoiceUpdated(forkName, event); err == nil {
|
||||||
log.Info("Successful ForkchoiceUpdated", "head", event.Block.Hash(), "status", status)
|
log.Info("Successful ForkchoiceUpdated", "head", event.Block.Hash(), "status", status)
|
||||||
} else {
|
} else {
|
||||||
|
@ -46,13 +46,13 @@ func (s *ApiServer) Subscribe(eventCallback func(event request.Event)) {
|
|||||||
log.Debug("New head received", "slot", slot, "blockRoot", blockRoot)
|
log.Debug("New head received", "slot", slot, "blockRoot", blockRoot)
|
||||||
eventCallback(request.Event{Type: sync.EvNewHead, Data: types.HeadInfo{Slot: slot, BlockRoot: blockRoot}})
|
eventCallback(request.Event{Type: sync.EvNewHead, Data: types.HeadInfo{Slot: slot, BlockRoot: blockRoot}})
|
||||||
},
|
},
|
||||||
OnSignedHead: func(head types.SignedHeader) {
|
OnOptimistic: func(update types.OptimisticUpdate) {
|
||||||
log.Debug("New signed head received", "slot", head.Header.Slot, "blockRoot", head.Header.Hash(), "signerCount", head.Signature.SignerCount())
|
log.Debug("New optimistic update received", "slot", update.Attested.Slot, "blockRoot", update.Attested.Hash(), "signerCount", update.Signature.SignerCount())
|
||||||
eventCallback(request.Event{Type: sync.EvNewSignedHead, Data: head})
|
eventCallback(request.Event{Type: sync.EvNewOptimisticUpdate, Data: update})
|
||||||
},
|
},
|
||||||
OnFinality: func(head types.FinalityUpdate) {
|
OnFinality: func(update types.FinalityUpdate) {
|
||||||
log.Debug("New finality update received", "slot", head.Attested.Slot, "blockRoot", head.Attested.Hash(), "signerCount", head.Signature.SignerCount())
|
log.Debug("New finality update received", "slot", update.Attested.Slot, "blockRoot", update.Attested.Hash(), "signerCount", update.Signature.SignerCount())
|
||||||
eventCallback(request.Event{Type: sync.EvNewFinalityUpdate, Data: head})
|
eventCallback(request.Event{Type: sync.EvNewFinalityUpdate, Data: update})
|
||||||
},
|
},
|
||||||
OnError: func(err error) {
|
OnError: func(err error) {
|
||||||
log.Warn("Head event stream error", "err", err)
|
log.Warn("Head event stream error", "err", err)
|
||||||
@ -83,6 +83,9 @@ func (s *ApiServer) SendRequest(id request.ID, req request.Request) {
|
|||||||
case sync.ReqBeaconBlock:
|
case sync.ReqBeaconBlock:
|
||||||
log.Debug("Beacon API: requesting block", "reqid", id, "hash", common.Hash(data))
|
log.Debug("Beacon API: requesting block", "reqid", id, "hash", common.Hash(data))
|
||||||
resp, err = s.api.GetBeaconBlock(common.Hash(data))
|
resp, err = s.api.GetBeaconBlock(common.Hash(data))
|
||||||
|
case sync.ReqFinality:
|
||||||
|
log.Debug("Beacon API: requesting finality update")
|
||||||
|
resp, err = s.api.GetFinalityUpdate()
|
||||||
default:
|
default:
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -90,6 +93,7 @@ func (s *ApiServer) SendRequest(id request.ID, req request.Request) {
|
|||||||
log.Warn("Beacon API request failed", "type", reflect.TypeOf(req), "reqid", id, "err", err)
|
log.Warn("Beacon API request failed", "type", reflect.TypeOf(req), "reqid", id, "err", err)
|
||||||
s.eventCallback(request.Event{Type: request.EvFail, Data: request.RequestResponse{ID: id, Request: req}})
|
s.eventCallback(request.Event{Type: request.EvFail, Data: request.RequestResponse{ID: id, Request: req}})
|
||||||
} else {
|
} else {
|
||||||
|
log.Debug("Beacon API request answered", "type", reflect.TypeOf(req), "reqid", id)
|
||||||
s.eventCallback(request.Event{Type: request.EvResponse, Data: request.RequestResponse{ID: id, Request: req, Response: resp}})
|
s.eventCallback(request.Event{Type: request.EvResponse, Data: request.RequestResponse{ID: id, Request: req, Response: resp}})
|
||||||
}
|
}
|
||||||
}()
|
}()
|
||||||
|
@ -32,6 +32,7 @@ import (
|
|||||||
"github.com/ethereum/go-ethereum/beacon/types"
|
"github.com/ethereum/go-ethereum/beacon/types"
|
||||||
"github.com/ethereum/go-ethereum/common"
|
"github.com/ethereum/go-ethereum/common"
|
||||||
"github.com/ethereum/go-ethereum/common/hexutil"
|
"github.com/ethereum/go-ethereum/common/hexutil"
|
||||||
|
"github.com/ethereum/go-ethereum/log"
|
||||||
)
|
)
|
||||||
|
|
||||||
var (
|
var (
|
||||||
@ -184,46 +185,56 @@ func (api *BeaconLightApi) GetBestUpdatesAndCommittees(firstPeriod, count uint64
|
|||||||
return updates, committees, nil
|
return updates, committees, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// GetOptimisticHeadUpdate fetches a signed header based on the latest available
|
// GetOptimisticUpdate fetches the latest available optimistic update.
|
||||||
// optimistic update. Note that the signature should be verified by the caller
|
// Note that the signature should be verified by the caller as its validity
|
||||||
// as its validity depends on the update chain.
|
// depends on the update chain.
|
||||||
//
|
//
|
||||||
// See data structure definition here:
|
// See data structure definition here:
|
||||||
// https://github.com/ethereum/consensus-specs/blob/dev/specs/altair/light-client/sync-protocol.md#lightclientoptimisticupdate
|
// https://github.com/ethereum/consensus-specs/blob/dev/specs/altair/light-client/sync-protocol.md#lightclientoptimisticupdate
|
||||||
func (api *BeaconLightApi) GetOptimisticHeadUpdate() (types.SignedHeader, error) {
|
func (api *BeaconLightApi) GetOptimisticUpdate() (types.OptimisticUpdate, error) {
|
||||||
resp, err := api.httpGet("/eth/v1/beacon/light_client/optimistic_update")
|
resp, err := api.httpGet("/eth/v1/beacon/light_client/optimistic_update")
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return types.SignedHeader{}, err
|
return types.OptimisticUpdate{}, err
|
||||||
}
|
}
|
||||||
return decodeOptimisticHeadUpdate(resp)
|
return decodeOptimisticUpdate(resp)
|
||||||
}
|
}
|
||||||
|
|
||||||
func decodeOptimisticHeadUpdate(enc []byte) (types.SignedHeader, error) {
|
func decodeOptimisticUpdate(enc []byte) (types.OptimisticUpdate, error) {
|
||||||
var data struct {
|
var data struct {
|
||||||
|
Version string
|
||||||
Data struct {
|
Data struct {
|
||||||
Header jsonBeaconHeader `json:"attested_header"`
|
Attested jsonHeaderWithExecProof `json:"attested_header"`
|
||||||
Aggregate types.SyncAggregate `json:"sync_aggregate"`
|
Aggregate types.SyncAggregate `json:"sync_aggregate"`
|
||||||
SignatureSlot common.Decimal `json:"signature_slot"`
|
SignatureSlot common.Decimal `json:"signature_slot"`
|
||||||
} `json:"data"`
|
} `json:"data"`
|
||||||
}
|
}
|
||||||
if err := json.Unmarshal(enc, &data); err != nil {
|
if err := json.Unmarshal(enc, &data); err != nil {
|
||||||
return types.SignedHeader{}, err
|
return types.OptimisticUpdate{}, err
|
||||||
}
|
}
|
||||||
if data.Data.Header.Beacon.StateRoot == (common.Hash{}) {
|
// Decode the execution payload headers.
|
||||||
|
attestedExecHeader, err := types.ExecutionHeaderFromJSON(data.Version, data.Data.Attested.Execution)
|
||||||
|
if err != nil {
|
||||||
|
return types.OptimisticUpdate{}, fmt.Errorf("invalid attested header: %v", err)
|
||||||
|
}
|
||||||
|
if data.Data.Attested.Beacon.StateRoot == (common.Hash{}) {
|
||||||
// workaround for different event encoding format in Lodestar
|
// workaround for different event encoding format in Lodestar
|
||||||
if err := json.Unmarshal(enc, &data.Data); err != nil {
|
if err := json.Unmarshal(enc, &data.Data); err != nil {
|
||||||
return types.SignedHeader{}, err
|
return types.OptimisticUpdate{}, err
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if len(data.Data.Aggregate.Signers) != params.SyncCommitteeBitmaskSize {
|
if len(data.Data.Aggregate.Signers) != params.SyncCommitteeBitmaskSize {
|
||||||
return types.SignedHeader{}, errors.New("invalid sync_committee_bits length")
|
return types.OptimisticUpdate{}, errors.New("invalid sync_committee_bits length")
|
||||||
}
|
}
|
||||||
if len(data.Data.Aggregate.Signature) != params.BLSSignatureSize {
|
if len(data.Data.Aggregate.Signature) != params.BLSSignatureSize {
|
||||||
return types.SignedHeader{}, errors.New("invalid sync_committee_signature length")
|
return types.OptimisticUpdate{}, errors.New("invalid sync_committee_signature length")
|
||||||
}
|
}
|
||||||
return types.SignedHeader{
|
return types.OptimisticUpdate{
|
||||||
Header: data.Data.Header.Beacon,
|
Attested: types.HeaderWithExecProof{
|
||||||
|
Header: data.Data.Attested.Beacon,
|
||||||
|
PayloadHeader: attestedExecHeader,
|
||||||
|
PayloadBranch: data.Data.Attested.ExecutionBranch,
|
||||||
|
},
|
||||||
Signature: data.Data.Aggregate,
|
Signature: data.Data.Aggregate,
|
||||||
SignatureSlot: uint64(data.Data.SignatureSlot),
|
SignatureSlot: uint64(data.Data.SignatureSlot),
|
||||||
}, nil
|
}, nil
|
||||||
@ -411,7 +422,7 @@ func decodeHeadEvent(enc []byte) (uint64, common.Hash, error) {
|
|||||||
|
|
||||||
type HeadEventListener struct {
|
type HeadEventListener struct {
|
||||||
OnNewHead func(slot uint64, blockRoot common.Hash)
|
OnNewHead func(slot uint64, blockRoot common.Hash)
|
||||||
OnSignedHead func(head types.SignedHeader)
|
OnOptimistic func(head types.OptimisticUpdate)
|
||||||
OnFinality func(head types.FinalityUpdate)
|
OnFinality func(head types.FinalityUpdate)
|
||||||
OnError func(err error)
|
OnError func(err error)
|
||||||
}
|
}
|
||||||
@ -449,21 +460,35 @@ func (api *BeaconLightApi) StartHeadListener(listener HeadEventListener) func()
|
|||||||
defer wg.Done()
|
defer wg.Done()
|
||||||
|
|
||||||
// Request initial data.
|
// Request initial data.
|
||||||
|
log.Trace("Requesting initial head header")
|
||||||
if head, _, _, err := api.GetHeader(common.Hash{}); err == nil {
|
if head, _, _, err := api.GetHeader(common.Hash{}); err == nil {
|
||||||
|
log.Trace("Retrieved initial head header", "slot", head.Slot, "hash", head.Hash())
|
||||||
listener.OnNewHead(head.Slot, head.Hash())
|
listener.OnNewHead(head.Slot, head.Hash())
|
||||||
|
} else {
|
||||||
|
log.Debug("Failed to retrieve initial head header", "error", err)
|
||||||
}
|
}
|
||||||
if signedHead, err := api.GetOptimisticHeadUpdate(); err == nil {
|
log.Trace("Requesting initial optimistic update")
|
||||||
listener.OnSignedHead(signedHead)
|
if optimisticUpdate, err := api.GetOptimisticUpdate(); err == nil {
|
||||||
|
log.Trace("Retrieved initial optimistic update", "slot", optimisticUpdate.Attested.Slot, "hash", optimisticUpdate.Attested.Hash())
|
||||||
|
listener.OnOptimistic(optimisticUpdate)
|
||||||
|
} else {
|
||||||
|
log.Debug("Failed to retrieve initial optimistic update", "error", err)
|
||||||
}
|
}
|
||||||
|
log.Trace("Requesting initial finality update")
|
||||||
if finalityUpdate, err := api.GetFinalityUpdate(); err == nil {
|
if finalityUpdate, err := api.GetFinalityUpdate(); err == nil {
|
||||||
|
log.Trace("Retrieved initial finality update", "slot", finalityUpdate.Finalized.Slot, "hash", finalityUpdate.Finalized.Hash())
|
||||||
listener.OnFinality(finalityUpdate)
|
listener.OnFinality(finalityUpdate)
|
||||||
|
} else {
|
||||||
|
log.Debug("Failed to retrieve initial finality update", "error", err)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
log.Trace("Starting event stream processing loop")
|
||||||
// Receive the stream.
|
// Receive the stream.
|
||||||
var stream *eventsource.Stream
|
var stream *eventsource.Stream
|
||||||
select {
|
select {
|
||||||
case stream = <-streamCh:
|
case stream = <-streamCh:
|
||||||
case <-ctx.Done():
|
case <-ctx.Done():
|
||||||
|
log.Trace("Stopping event stream processing loop")
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -474,8 +499,10 @@ func (api *BeaconLightApi) StartHeadListener(listener HeadEventListener) func()
|
|||||||
|
|
||||||
case event, ok := <-stream.Events:
|
case event, ok := <-stream.Events:
|
||||||
if !ok {
|
if !ok {
|
||||||
|
log.Trace("Event stream closed")
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
log.Trace("New event received from event stream", "type", event.Event())
|
||||||
switch event.Event() {
|
switch event.Event() {
|
||||||
case "head":
|
case "head":
|
||||||
slot, blockRoot, err := decodeHeadEvent([]byte(event.Data()))
|
slot, blockRoot, err := decodeHeadEvent([]byte(event.Data()))
|
||||||
@ -485,9 +512,9 @@ func (api *BeaconLightApi) StartHeadListener(listener HeadEventListener) func()
|
|||||||
listener.OnError(fmt.Errorf("error decoding head event: %v", err))
|
listener.OnError(fmt.Errorf("error decoding head event: %v", err))
|
||||||
}
|
}
|
||||||
case "light_client_optimistic_update":
|
case "light_client_optimistic_update":
|
||||||
signedHead, err := decodeOptimisticHeadUpdate([]byte(event.Data()))
|
optimisticUpdate, err := decodeOptimisticUpdate([]byte(event.Data()))
|
||||||
if err == nil {
|
if err == nil {
|
||||||
listener.OnSignedHead(signedHead)
|
listener.OnOptimistic(optimisticUpdate)
|
||||||
} else {
|
} else {
|
||||||
listener.OnError(fmt.Errorf("error decoding optimistic update event: %v", err))
|
listener.OnError(fmt.Errorf("error decoding optimistic update event: %v", err))
|
||||||
}
|
}
|
||||||
@ -521,7 +548,8 @@ func (api *BeaconLightApi) StartHeadListener(listener HeadEventListener) func()
|
|||||||
// established. It can only return nil when the context is canceled.
|
// established. It can only return nil when the context is canceled.
|
||||||
func (api *BeaconLightApi) startEventStream(ctx context.Context, listener *HeadEventListener) *eventsource.Stream {
|
func (api *BeaconLightApi) startEventStream(ctx context.Context, listener *HeadEventListener) *eventsource.Stream {
|
||||||
for retry := true; retry; retry = ctxSleep(ctx, 5*time.Second) {
|
for retry := true; retry; retry = ctxSleep(ctx, 5*time.Second) {
|
||||||
path := "/eth/v1/events?topics=head&topics=light_client_optimistic_update&topics=light_client_finality_update"
|
path := "/eth/v1/events?topics=head&topics=light_client_finality_update&topics=light_client_optimistic_update"
|
||||||
|
log.Trace("Sending event subscription request")
|
||||||
req, err := http.NewRequestWithContext(ctx, "GET", api.url+path, nil)
|
req, err := http.NewRequestWithContext(ctx, "GET", api.url+path, nil)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
listener.OnError(fmt.Errorf("error creating event subscription request: %v", err))
|
listener.OnError(fmt.Errorf("error creating event subscription request: %v", err))
|
||||||
@ -535,6 +563,7 @@ func (api *BeaconLightApi) startEventStream(ctx context.Context, listener *HeadE
|
|||||||
listener.OnError(fmt.Errorf("error creating event subscription: %v", err))
|
listener.OnError(fmt.Errorf("error creating event subscription: %v", err))
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
|
log.Trace("Successfully created event stream")
|
||||||
return stream
|
return stream
|
||||||
}
|
}
|
||||||
return nil
|
return nil
|
||||||
|
@ -32,8 +32,8 @@ type HeadTracker struct {
|
|||||||
lock sync.RWMutex
|
lock sync.RWMutex
|
||||||
committeeChain *CommitteeChain
|
committeeChain *CommitteeChain
|
||||||
minSignerCount int
|
minSignerCount int
|
||||||
signedHead types.SignedHeader
|
optimisticUpdate types.OptimisticUpdate
|
||||||
hasSignedHead bool
|
hasOptimisticUpdate bool
|
||||||
finalityUpdate types.FinalityUpdate
|
finalityUpdate types.FinalityUpdate
|
||||||
hasFinalityUpdate bool
|
hasFinalityUpdate bool
|
||||||
prefetchHead types.HeadInfo
|
prefetchHead types.HeadInfo
|
||||||
@ -48,15 +48,15 @@ func NewHeadTracker(committeeChain *CommitteeChain, minSignerCount int) *HeadTra
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// ValidatedHead returns the latest validated head.
|
// ValidatedOptimistic returns the latest validated optimistic update.
|
||||||
func (h *HeadTracker) ValidatedHead() (types.SignedHeader, bool) {
|
func (h *HeadTracker) ValidatedOptimistic() (types.OptimisticUpdate, bool) {
|
||||||
h.lock.RLock()
|
h.lock.RLock()
|
||||||
defer h.lock.RUnlock()
|
defer h.lock.RUnlock()
|
||||||
|
|
||||||
return h.signedHead, h.hasSignedHead
|
return h.optimisticUpdate, h.hasOptimisticUpdate
|
||||||
}
|
}
|
||||||
|
|
||||||
// ValidatedFinality returns the latest validated finality.
|
// ValidatedFinality returns the latest validated finality update.
|
||||||
func (h *HeadTracker) ValidatedFinality() (types.FinalityUpdate, bool) {
|
func (h *HeadTracker) ValidatedFinality() (types.FinalityUpdate, bool) {
|
||||||
h.lock.RLock()
|
h.lock.RLock()
|
||||||
defer h.lock.RUnlock()
|
defer h.lock.RUnlock()
|
||||||
@ -64,26 +64,36 @@ func (h *HeadTracker) ValidatedFinality() (types.FinalityUpdate, bool) {
|
|||||||
return h.finalityUpdate, h.hasFinalityUpdate
|
return h.finalityUpdate, h.hasFinalityUpdate
|
||||||
}
|
}
|
||||||
|
|
||||||
// ValidateHead validates the given signed head. If the head is successfully validated
|
// ValidateOptimistic validates the given optimistic update. If the update is
|
||||||
// and it is better than the old validated head (higher slot or same slot and more
|
// successfully validated and it is better than the old validated update (higher
|
||||||
// signers) then ValidatedHead is updated. The boolean return flag signals if
|
// slot or same slot and more signers) then ValidatedOptimistic is updated.
|
||||||
// ValidatedHead has been changed.
|
// The boolean return flag signals if ValidatedOptimistic has been changed.
|
||||||
func (h *HeadTracker) ValidateHead(head types.SignedHeader) (bool, error) {
|
func (h *HeadTracker) ValidateOptimistic(update types.OptimisticUpdate) (bool, error) {
|
||||||
h.lock.Lock()
|
h.lock.Lock()
|
||||||
defer h.lock.Unlock()
|
defer h.lock.Unlock()
|
||||||
|
|
||||||
replace, err := h.validate(head, h.signedHead)
|
if err := update.Validate(); err != nil {
|
||||||
|
return false, err
|
||||||
|
}
|
||||||
|
replace, err := h.validate(update.SignedHeader(), h.optimisticUpdate.SignedHeader())
|
||||||
if replace {
|
if replace {
|
||||||
h.signedHead, h.hasSignedHead = head, true
|
h.optimisticUpdate, h.hasOptimisticUpdate = update, true
|
||||||
h.changeCounter++
|
h.changeCounter++
|
||||||
}
|
}
|
||||||
return replace, err
|
return replace, err
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// ValidateFinality validates the given finality update. If the update is
|
||||||
|
// successfully validated and it is better than the old validated update (higher
|
||||||
|
// slot or same slot and more signers) then ValidatedFinality is updated.
|
||||||
|
// The boolean return flag signals if ValidatedFinality has been changed.
|
||||||
func (h *HeadTracker) ValidateFinality(update types.FinalityUpdate) (bool, error) {
|
func (h *HeadTracker) ValidateFinality(update types.FinalityUpdate) (bool, error) {
|
||||||
h.lock.Lock()
|
h.lock.Lock()
|
||||||
defer h.lock.Unlock()
|
defer h.lock.Unlock()
|
||||||
|
|
||||||
|
if err := update.Validate(); err != nil {
|
||||||
|
return false, err
|
||||||
|
}
|
||||||
replace, err := h.validate(update.SignedHeader(), h.finalityUpdate.SignedHeader())
|
replace, err := h.validate(update.SignedHeader(), h.finalityUpdate.SignedHeader())
|
||||||
if replace {
|
if replace {
|
||||||
h.finalityUpdate, h.hasFinalityUpdate = update, true
|
h.finalityUpdate, h.hasFinalityUpdate = update, true
|
||||||
@ -142,6 +152,7 @@ func (h *HeadTracker) SetPrefetchHead(head types.HeadInfo) {
|
|||||||
h.changeCounter++
|
h.changeCounter++
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// ChangeCounter implements request.targetData
|
||||||
func (h *HeadTracker) ChangeCounter() uint64 {
|
func (h *HeadTracker) ChangeCounter() uint64 {
|
||||||
h.lock.RLock()
|
h.lock.RLock()
|
||||||
defer h.lock.RUnlock()
|
defer h.lock.RUnlock()
|
||||||
|
@ -19,11 +19,13 @@ package sync
|
|||||||
import (
|
import (
|
||||||
"github.com/ethereum/go-ethereum/beacon/light/request"
|
"github.com/ethereum/go-ethereum/beacon/light/request"
|
||||||
"github.com/ethereum/go-ethereum/beacon/types"
|
"github.com/ethereum/go-ethereum/beacon/types"
|
||||||
|
"github.com/ethereum/go-ethereum/log"
|
||||||
)
|
)
|
||||||
|
|
||||||
type headTracker interface {
|
type headTracker interface {
|
||||||
ValidateHead(head types.SignedHeader) (bool, error)
|
ValidateOptimistic(update types.OptimisticUpdate) (bool, error)
|
||||||
ValidateFinality(head types.FinalityUpdate) (bool, error)
|
ValidateFinality(head types.FinalityUpdate) (bool, error)
|
||||||
|
ValidatedFinality() (types.FinalityUpdate, bool)
|
||||||
SetPrefetchHead(head types.HeadInfo)
|
SetPrefetchHead(head types.HeadInfo)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -37,9 +39,10 @@ type HeadSync struct {
|
|||||||
chain committeeChain
|
chain committeeChain
|
||||||
nextSyncPeriod uint64
|
nextSyncPeriod uint64
|
||||||
chainInit bool
|
chainInit bool
|
||||||
unvalidatedHeads map[request.Server]types.SignedHeader
|
unvalidatedOptimistic map[request.Server]types.OptimisticUpdate
|
||||||
unvalidatedFinality map[request.Server]types.FinalityUpdate
|
unvalidatedFinality map[request.Server]types.FinalityUpdate
|
||||||
serverHeads map[request.Server]types.HeadInfo
|
serverHeads map[request.Server]types.HeadInfo
|
||||||
|
reqFinalityEpoch map[request.Server]uint64 // next epoch to request finality update
|
||||||
headServerCount map[types.HeadInfo]headServerCount
|
headServerCount map[types.HeadInfo]headServerCount
|
||||||
headCounter uint64
|
headCounter uint64
|
||||||
prefetchHead types.HeadInfo
|
prefetchHead types.HeadInfo
|
||||||
@ -59,73 +62,96 @@ func NewHeadSync(headTracker headTracker, chain committeeChain) *HeadSync {
|
|||||||
s := &HeadSync{
|
s := &HeadSync{
|
||||||
headTracker: headTracker,
|
headTracker: headTracker,
|
||||||
chain: chain,
|
chain: chain,
|
||||||
unvalidatedHeads: make(map[request.Server]types.SignedHeader),
|
unvalidatedOptimistic: make(map[request.Server]types.OptimisticUpdate),
|
||||||
unvalidatedFinality: make(map[request.Server]types.FinalityUpdate),
|
unvalidatedFinality: make(map[request.Server]types.FinalityUpdate),
|
||||||
serverHeads: make(map[request.Server]types.HeadInfo),
|
serverHeads: make(map[request.Server]types.HeadInfo),
|
||||||
headServerCount: make(map[types.HeadInfo]headServerCount),
|
headServerCount: make(map[types.HeadInfo]headServerCount),
|
||||||
|
reqFinalityEpoch: make(map[request.Server]uint64),
|
||||||
}
|
}
|
||||||
return s
|
return s
|
||||||
}
|
}
|
||||||
|
|
||||||
// Process implements request.Module.
|
// Process implements request.Module.
|
||||||
func (s *HeadSync) Process(requester request.Requester, events []request.Event) {
|
func (s *HeadSync) Process(requester request.Requester, events []request.Event) {
|
||||||
|
nextPeriod, chainInit := s.chain.NextSyncPeriod()
|
||||||
|
if nextPeriod != s.nextSyncPeriod || chainInit != s.chainInit {
|
||||||
|
s.nextSyncPeriod, s.chainInit = nextPeriod, chainInit
|
||||||
|
s.processUnvalidatedUpdates()
|
||||||
|
}
|
||||||
|
|
||||||
for _, event := range events {
|
for _, event := range events {
|
||||||
switch event.Type {
|
switch event.Type {
|
||||||
case EvNewHead:
|
case EvNewHead:
|
||||||
s.setServerHead(event.Server, event.Data.(types.HeadInfo))
|
s.setServerHead(event.Server, event.Data.(types.HeadInfo))
|
||||||
case EvNewSignedHead:
|
case EvNewOptimisticUpdate:
|
||||||
s.newSignedHead(event.Server, event.Data.(types.SignedHeader))
|
update := event.Data.(types.OptimisticUpdate)
|
||||||
|
s.newOptimisticUpdate(event.Server, update)
|
||||||
|
epoch := update.Attested.Epoch()
|
||||||
|
if epoch < s.reqFinalityEpoch[event.Server] {
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
if finality, ok := s.headTracker.ValidatedFinality(); ok && finality.Attested.Header.Epoch() >= epoch {
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
requester.Send(event.Server, ReqFinality{})
|
||||||
|
s.reqFinalityEpoch[event.Server] = epoch + 1
|
||||||
case EvNewFinalityUpdate:
|
case EvNewFinalityUpdate:
|
||||||
s.newFinalityUpdate(event.Server, event.Data.(types.FinalityUpdate))
|
s.newFinalityUpdate(event.Server, event.Data.(types.FinalityUpdate))
|
||||||
|
case request.EvResponse:
|
||||||
|
_, _, resp := event.RequestInfo()
|
||||||
|
s.newFinalityUpdate(event.Server, resp.(types.FinalityUpdate))
|
||||||
case request.EvUnregistered:
|
case request.EvUnregistered:
|
||||||
s.setServerHead(event.Server, types.HeadInfo{})
|
s.setServerHead(event.Server, types.HeadInfo{})
|
||||||
delete(s.serverHeads, event.Server)
|
delete(s.serverHeads, event.Server)
|
||||||
delete(s.unvalidatedHeads, event.Server)
|
delete(s.unvalidatedOptimistic, event.Server)
|
||||||
|
delete(s.unvalidatedFinality, event.Server)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
nextPeriod, chainInit := s.chain.NextSyncPeriod()
|
|
||||||
if nextPeriod != s.nextSyncPeriod || chainInit != s.chainInit {
|
|
||||||
s.nextSyncPeriod, s.chainInit = nextPeriod, chainInit
|
|
||||||
s.processUnvalidated()
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// newSignedHead handles received signed head; either validates it if the chain
|
// newOptimisticUpdate handles received optimistic update; either validates it if
|
||||||
// is properly synced or stores it for further validation.
|
// the chain is properly synced or stores it for further validation.
|
||||||
func (s *HeadSync) newSignedHead(server request.Server, signedHead types.SignedHeader) {
|
func (s *HeadSync) newOptimisticUpdate(server request.Server, optimisticUpdate types.OptimisticUpdate) {
|
||||||
if !s.chainInit || types.SyncPeriod(signedHead.SignatureSlot) > s.nextSyncPeriod {
|
if !s.chainInit || types.SyncPeriod(optimisticUpdate.SignatureSlot) > s.nextSyncPeriod {
|
||||||
s.unvalidatedHeads[server] = signedHead
|
s.unvalidatedOptimistic[server] = optimisticUpdate
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
s.headTracker.ValidateHead(signedHead)
|
if _, err := s.headTracker.ValidateOptimistic(optimisticUpdate); err != nil {
|
||||||
|
log.Debug("Error validating optimistic update", "error", err)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// newFinalityUpdate handles received finality update; either validates it if the chain
|
// newFinalityUpdate handles received finality update; either validates it if
|
||||||
// is properly synced or stores it for further validation.
|
// the chain is properly synced or stores it for further validation.
|
||||||
func (s *HeadSync) newFinalityUpdate(server request.Server, finalityUpdate types.FinalityUpdate) {
|
func (s *HeadSync) newFinalityUpdate(server request.Server, finalityUpdate types.FinalityUpdate) {
|
||||||
if !s.chainInit || types.SyncPeriod(finalityUpdate.SignatureSlot) > s.nextSyncPeriod {
|
if !s.chainInit || types.SyncPeriod(finalityUpdate.SignatureSlot) > s.nextSyncPeriod {
|
||||||
s.unvalidatedFinality[server] = finalityUpdate
|
s.unvalidatedFinality[server] = finalityUpdate
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
s.headTracker.ValidateFinality(finalityUpdate)
|
if _, err := s.headTracker.ValidateFinality(finalityUpdate); err != nil {
|
||||||
|
log.Debug("Error validating finality update", "error", err)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// processUnvalidated iterates the list of unvalidated heads and validates
|
// processUnvalidatedUpdates iterates the list of unvalidated updates and validates
|
||||||
// those which can be validated.
|
// those which can be validated.
|
||||||
func (s *HeadSync) processUnvalidated() {
|
func (s *HeadSync) processUnvalidatedUpdates() {
|
||||||
if !s.chainInit {
|
if !s.chainInit {
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
for server, signedHead := range s.unvalidatedHeads {
|
for server, optimisticUpdate := range s.unvalidatedOptimistic {
|
||||||
if types.SyncPeriod(signedHead.SignatureSlot) <= s.nextSyncPeriod {
|
if types.SyncPeriod(optimisticUpdate.SignatureSlot) <= s.nextSyncPeriod {
|
||||||
s.headTracker.ValidateHead(signedHead)
|
if _, err := s.headTracker.ValidateOptimistic(optimisticUpdate); err != nil {
|
||||||
delete(s.unvalidatedHeads, server)
|
log.Debug("Error validating deferred optimistic update", "error", err)
|
||||||
|
}
|
||||||
|
delete(s.unvalidatedOptimistic, server)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
for server, finalityUpdate := range s.unvalidatedFinality {
|
for server, finalityUpdate := range s.unvalidatedFinality {
|
||||||
if types.SyncPeriod(finalityUpdate.SignatureSlot) <= s.nextSyncPeriod {
|
if types.SyncPeriod(finalityUpdate.SignatureSlot) <= s.nextSyncPeriod {
|
||||||
s.headTracker.ValidateFinality(finalityUpdate)
|
if _, err := s.headTracker.ValidateFinality(finalityUpdate); err != nil {
|
||||||
|
log.Debug("Error validating deferred finality update", "error", err)
|
||||||
|
}
|
||||||
delete(s.unvalidatedFinality, server)
|
delete(s.unvalidatedFinality, server)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -19,6 +19,7 @@ package sync
|
|||||||
import (
|
import (
|
||||||
"testing"
|
"testing"
|
||||||
|
|
||||||
|
"github.com/ethereum/go-ethereum/beacon/light/request"
|
||||||
"github.com/ethereum/go-ethereum/beacon/types"
|
"github.com/ethereum/go-ethereum/beacon/types"
|
||||||
"github.com/ethereum/go-ethereum/common"
|
"github.com/ethereum/go-ethereum/common"
|
||||||
)
|
)
|
||||||
@ -28,6 +29,7 @@ var (
|
|||||||
testServer2 = testServer("testServer2")
|
testServer2 = testServer("testServer2")
|
||||||
testServer3 = testServer("testServer3")
|
testServer3 = testServer("testServer3")
|
||||||
testServer4 = testServer("testServer4")
|
testServer4 = testServer("testServer4")
|
||||||
|
testServer5 = testServer("testServer5")
|
||||||
|
|
||||||
testHead0 = types.HeadInfo{}
|
testHead0 = types.HeadInfo{}
|
||||||
testHead1 = types.HeadInfo{Slot: 123, BlockRoot: common.Hash{1}}
|
testHead1 = types.HeadInfo{Slot: 123, BlockRoot: common.Hash{1}}
|
||||||
@ -35,13 +37,21 @@ var (
|
|||||||
testHead3 = types.HeadInfo{Slot: 124, BlockRoot: common.Hash{3}}
|
testHead3 = types.HeadInfo{Slot: 124, BlockRoot: common.Hash{3}}
|
||||||
testHead4 = types.HeadInfo{Slot: 125, BlockRoot: common.Hash{4}}
|
testHead4 = types.HeadInfo{Slot: 125, BlockRoot: common.Hash{4}}
|
||||||
|
|
||||||
testSHead1 = types.SignedHeader{SignatureSlot: 0x0124, Header: types.Header{Slot: 0x0123, StateRoot: common.Hash{1}}}
|
testOptUpdate1 = types.OptimisticUpdate{SignatureSlot: 0x0124, Attested: types.HeaderWithExecProof{Header: types.Header{Slot: 0x0123, StateRoot: common.Hash{1}}}}
|
||||||
testSHead2 = types.SignedHeader{SignatureSlot: 0x2010, Header: types.Header{Slot: 0x200e, StateRoot: common.Hash{2}}}
|
testOptUpdate2 = types.OptimisticUpdate{SignatureSlot: 0x2010, Attested: types.HeaderWithExecProof{Header: types.Header{Slot: 0x200e, StateRoot: common.Hash{2}}}}
|
||||||
// testSHead3 is at the end of period 1 but signed in period 2
|
// testOptUpdate3 is at the end of period 1 but signed in period 2
|
||||||
testSHead3 = types.SignedHeader{SignatureSlot: 0x4000, Header: types.Header{Slot: 0x3fff, StateRoot: common.Hash{3}}}
|
testOptUpdate3 = types.OptimisticUpdate{SignatureSlot: 0x4000, Attested: types.HeaderWithExecProof{Header: types.Header{Slot: 0x3fff, StateRoot: common.Hash{3}}}}
|
||||||
testSHead4 = types.SignedHeader{SignatureSlot: 0x6444, Header: types.Header{Slot: 0x6443, StateRoot: common.Hash{4}}}
|
testOptUpdate4 = types.OptimisticUpdate{SignatureSlot: 0x6444, Attested: types.HeaderWithExecProof{Header: types.Header{Slot: 0x6443, StateRoot: common.Hash{4}}}}
|
||||||
)
|
)
|
||||||
|
|
||||||
|
func finality(opt types.OptimisticUpdate) types.FinalityUpdate {
|
||||||
|
return types.FinalityUpdate{
|
||||||
|
SignatureSlot: opt.SignatureSlot,
|
||||||
|
Attested: opt.Attested,
|
||||||
|
Finalized: types.HeaderWithExecProof{Header: types.Header{Slot: (opt.Attested.Header.Slot - 64) & uint64(0xffffffffffffffe0)}},
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
type testServer string
|
type testServer string
|
||||||
|
|
||||||
func (t testServer) Name() string {
|
func (t testServer) Name() string {
|
||||||
@ -57,50 +67,66 @@ func TestValidatedHead(t *testing.T) {
|
|||||||
ht.ExpValidated(t, 0, nil)
|
ht.ExpValidated(t, 0, nil)
|
||||||
|
|
||||||
ts.AddServer(testServer1, 1)
|
ts.AddServer(testServer1, 1)
|
||||||
ts.ServerEvent(EvNewSignedHead, testServer1, testSHead1)
|
ts.ServerEvent(EvNewOptimisticUpdate, testServer1, testOptUpdate1)
|
||||||
ts.Run(1)
|
ts.Run(1, testServer1, ReqFinality{})
|
||||||
// announced head should be queued because of uninitialized chain
|
// announced head should be queued because of uninitialized chain
|
||||||
ht.ExpValidated(t, 1, nil)
|
ht.ExpValidated(t, 1, nil)
|
||||||
|
|
||||||
chain.SetNextSyncPeriod(0) // initialize chain
|
chain.SetNextSyncPeriod(0) // initialize chain
|
||||||
ts.Run(2)
|
ts.Run(2)
|
||||||
// expect previously queued head to be validated
|
// expect previously queued head to be validated
|
||||||
ht.ExpValidated(t, 2, []types.SignedHeader{testSHead1})
|
ht.ExpValidated(t, 2, []types.OptimisticUpdate{testOptUpdate1})
|
||||||
|
|
||||||
chain.SetNextSyncPeriod(1)
|
chain.SetNextSyncPeriod(1)
|
||||||
ts.ServerEvent(EvNewSignedHead, testServer1, testSHead2)
|
ts.ServerEvent(EvNewFinalityUpdate, testServer1, finality(testOptUpdate2))
|
||||||
|
ts.ServerEvent(EvNewOptimisticUpdate, testServer1, testOptUpdate2)
|
||||||
ts.AddServer(testServer2, 1)
|
ts.AddServer(testServer2, 1)
|
||||||
ts.ServerEvent(EvNewSignedHead, testServer2, testSHead2)
|
ts.ServerEvent(EvNewOptimisticUpdate, testServer2, testOptUpdate2)
|
||||||
ts.Run(3)
|
ts.Run(3)
|
||||||
// expect both head announcements to be validated instantly
|
// expect both head announcements to be validated instantly
|
||||||
ht.ExpValidated(t, 3, []types.SignedHeader{testSHead2, testSHead2})
|
ht.ExpValidated(t, 3, []types.OptimisticUpdate{testOptUpdate2, testOptUpdate2})
|
||||||
|
|
||||||
ts.ServerEvent(EvNewSignedHead, testServer1, testSHead3)
|
ts.ServerEvent(EvNewOptimisticUpdate, testServer1, testOptUpdate3)
|
||||||
ts.AddServer(testServer3, 1)
|
ts.AddServer(testServer3, 1)
|
||||||
ts.ServerEvent(EvNewSignedHead, testServer3, testSHead4)
|
ts.ServerEvent(EvNewOptimisticUpdate, testServer3, testOptUpdate4)
|
||||||
ts.Run(4)
|
// finality should be requested from both servers
|
||||||
// future period announced heads should be queued
|
ts.Run(4, testServer1, ReqFinality{}, testServer3, ReqFinality{})
|
||||||
|
// future period annonced heads should be queued
|
||||||
ht.ExpValidated(t, 4, nil)
|
ht.ExpValidated(t, 4, nil)
|
||||||
|
|
||||||
chain.SetNextSyncPeriod(2)
|
chain.SetNextSyncPeriod(2)
|
||||||
ts.Run(5)
|
ts.Run(5)
|
||||||
// testSHead3 can be validated now but not testSHead4
|
// testOptUpdate3 can be validated now but not testOptUpdate4
|
||||||
ht.ExpValidated(t, 5, []types.SignedHeader{testSHead3})
|
ht.ExpValidated(t, 5, []types.OptimisticUpdate{testOptUpdate3})
|
||||||
|
|
||||||
|
ts.AddServer(testServer4, 1)
|
||||||
|
ts.ServerEvent(EvNewOptimisticUpdate, testServer4, testOptUpdate3)
|
||||||
|
// new server joined with recent optimistic update but still no finality; should be requested
|
||||||
|
ts.Run(6, testServer4, ReqFinality{})
|
||||||
|
ht.ExpValidated(t, 6, []types.OptimisticUpdate{testOptUpdate3})
|
||||||
|
|
||||||
|
ts.AddServer(testServer5, 1)
|
||||||
|
ts.RequestEvent(request.EvResponse, ts.Request(6, 1), finality(testOptUpdate3))
|
||||||
|
ts.ServerEvent(EvNewOptimisticUpdate, testServer5, testOptUpdate3)
|
||||||
|
// finality update request answered; new server should not be requested
|
||||||
|
ts.Run(7)
|
||||||
|
ht.ExpValidated(t, 7, []types.OptimisticUpdate{testOptUpdate3})
|
||||||
|
|
||||||
// server 3 disconnected without proving period 3, its announced head should be dropped
|
// server 3 disconnected without proving period 3, its announced head should be dropped
|
||||||
ts.RemoveServer(testServer3)
|
ts.RemoveServer(testServer3)
|
||||||
ts.Run(6)
|
ts.Run(8)
|
||||||
ht.ExpValidated(t, 6, nil)
|
ht.ExpValidated(t, 8, nil)
|
||||||
|
|
||||||
chain.SetNextSyncPeriod(3)
|
chain.SetNextSyncPeriod(3)
|
||||||
ts.Run(7)
|
ts.Run(9)
|
||||||
// testSHead4 could be validated now but it's not queued by any registered server
|
// testOptUpdate4 could be validated now but it's not queued by any registered server
|
||||||
ht.ExpValidated(t, 7, nil)
|
ht.ExpValidated(t, 9, nil)
|
||||||
|
|
||||||
ts.ServerEvent(EvNewSignedHead, testServer2, testSHead4)
|
ts.ServerEvent(EvNewFinalityUpdate, testServer2, finality(testOptUpdate4))
|
||||||
ts.Run(8)
|
ts.ServerEvent(EvNewOptimisticUpdate, testServer2, testOptUpdate4)
|
||||||
// now testSHead4 should be validated
|
ts.Run(10)
|
||||||
ht.ExpValidated(t, 8, []types.SignedHeader{testSHead4})
|
// now testOptUpdate4 should be validated
|
||||||
|
ht.ExpValidated(t, 10, []types.OptimisticUpdate{testOptUpdate4})
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestPrefetchHead(t *testing.T) {
|
func TestPrefetchHead(t *testing.T) {
|
||||||
|
@ -212,32 +212,37 @@ func (tc *TestCommitteeChain) ExpNextSyncPeriod(t *testing.T, expNsp uint64) {
|
|||||||
|
|
||||||
type TestHeadTracker struct {
|
type TestHeadTracker struct {
|
||||||
phead types.HeadInfo
|
phead types.HeadInfo
|
||||||
validated []types.SignedHeader
|
validated []types.OptimisticUpdate
|
||||||
|
finality types.FinalityUpdate
|
||||||
}
|
}
|
||||||
|
|
||||||
func (ht *TestHeadTracker) ValidateHead(head types.SignedHeader) (bool, error) {
|
func (ht *TestHeadTracker) ValidateOptimistic(update types.OptimisticUpdate) (bool, error) {
|
||||||
ht.validated = append(ht.validated, head)
|
ht.validated = append(ht.validated, update)
|
||||||
return true, nil
|
return true, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// TODO add test case for finality
|
func (ht *TestHeadTracker) ValidateFinality(update types.FinalityUpdate) (bool, error) {
|
||||||
func (ht *TestHeadTracker) ValidateFinality(head types.FinalityUpdate) (bool, error) {
|
ht.finality = update
|
||||||
return true, nil
|
return true, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (ht *TestHeadTracker) ExpValidated(t *testing.T, tci int, expHeads []types.SignedHeader) {
|
func (ht *TestHeadTracker) ValidatedFinality() (types.FinalityUpdate, bool) {
|
||||||
|
return ht.finality, ht.finality.Attested.Header != (types.Header{})
|
||||||
|
}
|
||||||
|
|
||||||
|
func (ht *TestHeadTracker) ExpValidated(t *testing.T, tci int, expHeads []types.OptimisticUpdate) {
|
||||||
for i, expHead := range expHeads {
|
for i, expHead := range expHeads {
|
||||||
if i >= len(ht.validated) {
|
if i >= len(ht.validated) {
|
||||||
t.Errorf("Missing validated head in test case #%d index #%d (expected {slot %d blockRoot %x}, got none)", tci, i, expHead.Header.Slot, expHead.Header.Hash())
|
t.Errorf("Missing validated head in test case #%d index #%d (expected {slot %d blockRoot %x}, got none)", tci, i, expHead.Attested.Header.Slot, expHead.Attested.Header.Hash())
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
if ht.validated[i] != expHead {
|
if !reflect.DeepEqual(ht.validated[i], expHead) {
|
||||||
vhead := ht.validated[i].Header
|
vhead := ht.validated[i].Attested.Header
|
||||||
t.Errorf("Wrong validated head in test case #%d index #%d (expected {slot %d blockRoot %x}, got {slot %d blockRoot %x})", tci, i, expHead.Header.Slot, expHead.Header.Hash(), vhead.Slot, vhead.Hash())
|
t.Errorf("Wrong validated head in test case #%d index #%d (expected {slot %d blockRoot %x}, got {slot %d blockRoot %x})", tci, i, expHead.Attested.Header.Slot, expHead.Attested.Header.Hash(), vhead.Slot, vhead.Hash())
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
for i := len(expHeads); i < len(ht.validated); i++ {
|
for i := len(expHeads); i < len(ht.validated); i++ {
|
||||||
vhead := ht.validated[i].Header
|
vhead := ht.validated[i].Attested.Header
|
||||||
t.Errorf("Unexpected validated head in test case #%d index #%d (expected none, got {slot %d blockRoot %x})", tci, i, vhead.Slot, vhead.Hash())
|
t.Errorf("Unexpected validated head in test case #%d index #%d (expected none, got {slot %d blockRoot %x})", tci, i, vhead.Slot, vhead.Hash())
|
||||||
}
|
}
|
||||||
ht.validated = nil
|
ht.validated = nil
|
||||||
|
@ -24,7 +24,7 @@ import (
|
|||||||
|
|
||||||
var (
|
var (
|
||||||
EvNewHead = &request.EventType{Name: "newHead"} // data: types.HeadInfo
|
EvNewHead = &request.EventType{Name: "newHead"} // data: types.HeadInfo
|
||||||
EvNewSignedHead = &request.EventType{Name: "newSignedHead"} // data: types.SignedHeader
|
EvNewOptimisticUpdate = &request.EventType{Name: "newOptimisticUpdate"} // data: types.OptimisticUpdate
|
||||||
EvNewFinalityUpdate = &request.EventType{Name: "newFinalityUpdate"} // data: types.FinalityUpdate
|
EvNewFinalityUpdate = &request.EventType{Name: "newFinalityUpdate"} // data: types.FinalityUpdate
|
||||||
)
|
)
|
||||||
|
|
||||||
@ -43,4 +43,5 @@ type (
|
|||||||
}
|
}
|
||||||
ReqCheckpointData common.Hash
|
ReqCheckpointData common.Hash
|
||||||
ReqBeaconBlock common.Hash
|
ReqBeaconBlock common.Hash
|
||||||
|
ReqFinality struct{}
|
||||||
)
|
)
|
||||||
|
@ -84,6 +84,7 @@ func (s *CheckpointInit) Process(requester request.Requester, events []request.E
|
|||||||
if s.initialized {
|
if s.initialized {
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
for _, event := range events {
|
for _, event := range events {
|
||||||
switch event.Type {
|
switch event.Type {
|
||||||
case request.EvResponse, request.EvFail, request.EvTimeout:
|
case request.EvResponse, request.EvFail, request.EvTimeout:
|
||||||
@ -132,10 +133,12 @@ func (s *CheckpointInit) Process(requester request.Requester, events []request.E
|
|||||||
newState.state = ssPrintStatus
|
newState.state = ssPrintStatus
|
||||||
s.serverState[sid.Server] = newState
|
s.serverState[sid.Server] = newState
|
||||||
}
|
}
|
||||||
|
|
||||||
case request.EvUnregistered:
|
case request.EvUnregistered:
|
||||||
delete(s.serverState, event.Server)
|
delete(s.serverState, event.Server)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// start a request if possible
|
// start a request if possible
|
||||||
for _, server := range requester.CanSendTo() {
|
for _, server := range requester.CanSendTo() {
|
||||||
switch s.serverState[server].state {
|
switch s.serverState[server].state {
|
||||||
@ -156,6 +159,7 @@ func (s *CheckpointInit) Process(requester request.Requester, events []request.E
|
|||||||
s.serverState[server] = newState
|
s.serverState[server] = newState
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// print log message if necessary
|
// print log message if necessary
|
||||||
for server, state := range s.serverState {
|
for server, state := range s.serverState {
|
||||||
if state.state != ssPrintStatus {
|
if state.state != ssPrintStatus {
|
||||||
@ -316,9 +320,9 @@ func (s *ForwardUpdateSync) Process(requester request.Requester, events []reques
|
|||||||
if !queued {
|
if !queued {
|
||||||
s.unlockRange(sid, req)
|
s.unlockRange(sid, req)
|
||||||
}
|
}
|
||||||
case EvNewSignedHead:
|
case EvNewOptimisticUpdate:
|
||||||
signedHead := event.Data.(types.SignedHeader)
|
update := event.Data.(types.OptimisticUpdate)
|
||||||
s.nextSyncPeriod[event.Server] = types.SyncPeriod(signedHead.SignatureSlot + 256)
|
s.nextSyncPeriod[event.Server] = types.SyncPeriod(update.SignatureSlot + 256)
|
||||||
case request.EvUnregistered:
|
case request.EvUnregistered:
|
||||||
delete(s.nextSyncPeriod, event.Server)
|
delete(s.nextSyncPeriod, event.Server)
|
||||||
}
|
}
|
||||||
|
@ -68,9 +68,9 @@ func TestUpdateSyncParallel(t *testing.T) {
|
|||||||
ts := NewTestScheduler(t, updateSync)
|
ts := NewTestScheduler(t, updateSync)
|
||||||
// add 2 servers, head at period 100; allow 3-3 parallel requests for each
|
// add 2 servers, head at period 100; allow 3-3 parallel requests for each
|
||||||
ts.AddServer(testServer1, 3)
|
ts.AddServer(testServer1, 3)
|
||||||
ts.ServerEvent(EvNewSignedHead, testServer1, types.SignedHeader{SignatureSlot: 0x2000*100 + 0x1000})
|
ts.ServerEvent(EvNewOptimisticUpdate, testServer1, types.OptimisticUpdate{SignatureSlot: 0x2000*100 + 0x1000})
|
||||||
ts.AddServer(testServer2, 3)
|
ts.AddServer(testServer2, 3)
|
||||||
ts.ServerEvent(EvNewSignedHead, testServer2, types.SignedHeader{SignatureSlot: 0x2000*100 + 0x1000})
|
ts.ServerEvent(EvNewOptimisticUpdate, testServer2, types.OptimisticUpdate{SignatureSlot: 0x2000*100 + 0x1000})
|
||||||
|
|
||||||
// expect 6 requests to be sent
|
// expect 6 requests to be sent
|
||||||
ts.Run(1,
|
ts.Run(1,
|
||||||
@ -150,11 +150,11 @@ func TestUpdateSyncDifferentHeads(t *testing.T) {
|
|||||||
ts := NewTestScheduler(t, updateSync)
|
ts := NewTestScheduler(t, updateSync)
|
||||||
// add 3 servers with different announced head periods
|
// add 3 servers with different announced head periods
|
||||||
ts.AddServer(testServer1, 1)
|
ts.AddServer(testServer1, 1)
|
||||||
ts.ServerEvent(EvNewSignedHead, testServer1, types.SignedHeader{SignatureSlot: 0x2000*15 + 0x1000})
|
ts.ServerEvent(EvNewOptimisticUpdate, testServer1, types.OptimisticUpdate{SignatureSlot: 0x2000*15 + 0x1000})
|
||||||
ts.AddServer(testServer2, 1)
|
ts.AddServer(testServer2, 1)
|
||||||
ts.ServerEvent(EvNewSignedHead, testServer2, types.SignedHeader{SignatureSlot: 0x2000*16 + 0x1000})
|
ts.ServerEvent(EvNewOptimisticUpdate, testServer2, types.OptimisticUpdate{SignatureSlot: 0x2000*16 + 0x1000})
|
||||||
ts.AddServer(testServer3, 1)
|
ts.AddServer(testServer3, 1)
|
||||||
ts.ServerEvent(EvNewSignedHead, testServer3, types.SignedHeader{SignatureSlot: 0x2000*17 + 0x1000})
|
ts.ServerEvent(EvNewOptimisticUpdate, testServer3, types.OptimisticUpdate{SignatureSlot: 0x2000*17 + 0x1000})
|
||||||
|
|
||||||
// expect request to the best announced head
|
// expect request to the best announced head
|
||||||
ts.Run(1, testServer3, ReqUpdates{FirstPeriod: 10, Count: 7})
|
ts.Run(1, testServer3, ReqUpdates{FirstPeriod: 10, Count: 7})
|
||||||
@ -190,7 +190,7 @@ func TestUpdateSyncDifferentHeads(t *testing.T) {
|
|||||||
|
|
||||||
// a new server is registered with announced head period 17
|
// a new server is registered with announced head period 17
|
||||||
ts.AddServer(testServer4, 1)
|
ts.AddServer(testServer4, 1)
|
||||||
ts.ServerEvent(EvNewSignedHead, testServer4, types.SignedHeader{SignatureSlot: 0x2000*17 + 0x1000})
|
ts.ServerEvent(EvNewOptimisticUpdate, testServer4, types.OptimisticUpdate{SignatureSlot: 0x2000*17 + 0x1000})
|
||||||
// expect request to sync one more period
|
// expect request to sync one more period
|
||||||
ts.Run(7, testServer4, ReqUpdates{FirstPeriod: 16, Count: 1})
|
ts.Run(7, testServer4, ReqUpdates{FirstPeriod: 16, Count: 1})
|
||||||
|
|
||||||
|
@ -66,9 +66,8 @@ func convertPayload[T payloadType](payload T, parentRoot *zrntcommon.Root) (*typ
|
|||||||
block := types.NewBlockWithHeader(&header)
|
block := types.NewBlockWithHeader(&header)
|
||||||
block = block.WithBody(transactions, nil)
|
block = block.WithBody(transactions, nil)
|
||||||
block = block.WithWithdrawals(withdrawals)
|
block = block.WithWithdrawals(withdrawals)
|
||||||
hash := block.Hash()
|
if hash := block.Hash(); hash != expectedHash {
|
||||||
if hash != expectedHash {
|
return nil, fmt.Errorf("Sanity check failed, payload hash does not match (expected %x, got %x)", expectedHash, hash)
|
||||||
return block, fmt.Errorf("Sanity check failed, payload hash does not match (expected %x, got %x)", expectedHash, hash)
|
|
||||||
}
|
}
|
||||||
return block, nil
|
return block, nil
|
||||||
}
|
}
|
||||||
|
@ -23,7 +23,7 @@ import (
|
|||||||
"github.com/ethereum/go-ethereum/beacon/merkle"
|
"github.com/ethereum/go-ethereum/beacon/merkle"
|
||||||
"github.com/ethereum/go-ethereum/beacon/params"
|
"github.com/ethereum/go-ethereum/beacon/params"
|
||||||
"github.com/ethereum/go-ethereum/common"
|
"github.com/ethereum/go-ethereum/common"
|
||||||
"github.com/ethereum/go-ethereum/core/types"
|
ctypes "github.com/ethereum/go-ethereum/core/types"
|
||||||
)
|
)
|
||||||
|
|
||||||
// HeadInfo represents an unvalidated new head announcement.
|
// HeadInfo represents an unvalidated new head announcement.
|
||||||
@ -142,17 +142,57 @@ func (u UpdateScore) BetterThan(w UpdateScore) bool {
|
|||||||
return u.SignerCount > w.SignerCount
|
return u.SignerCount > w.SignerCount
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// HeaderWithExecProof contains a beacon header and proves the belonging execution
|
||||||
|
// payload header with a Merkle proof.
|
||||||
type HeaderWithExecProof struct {
|
type HeaderWithExecProof struct {
|
||||||
Header
|
Header
|
||||||
PayloadHeader *ExecutionHeader
|
PayloadHeader *ExecutionHeader
|
||||||
PayloadBranch merkle.Values
|
PayloadBranch merkle.Values
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Validate verifies the Merkle proof of the execution payload header.
|
||||||
func (h *HeaderWithExecProof) Validate() error {
|
func (h *HeaderWithExecProof) Validate() error {
|
||||||
payloadRoot := h.PayloadHeader.PayloadRoot()
|
return merkle.VerifyProof(h.BodyRoot, params.BodyIndexExecPayload, h.PayloadBranch, h.PayloadHeader.PayloadRoot())
|
||||||
return merkle.VerifyProof(h.BodyRoot, params.BodyIndexExecPayload, h.PayloadBranch, payloadRoot)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// OptimisticUpdate proves sync committee commitment on the attested beacon header.
|
||||||
|
// It also proves the belonging execution payload header with a Merkle proof.
|
||||||
|
//
|
||||||
|
// See data structure definition here:
|
||||||
|
// https://github.com/ethereum/consensus-specs/blob/dev/specs/altair/light-client/sync-protocol.md#lightclientoptimisticupdate
|
||||||
|
type OptimisticUpdate struct {
|
||||||
|
Attested HeaderWithExecProof
|
||||||
|
// Sync committee BLS signature aggregate
|
||||||
|
Signature SyncAggregate
|
||||||
|
// Slot in which the signature has been created (newer than Header.Slot,
|
||||||
|
// determines the signing sync committee)
|
||||||
|
SignatureSlot uint64
|
||||||
|
}
|
||||||
|
|
||||||
|
// SignedHeader returns the signed attested header of the update.
|
||||||
|
func (u *OptimisticUpdate) SignedHeader() SignedHeader {
|
||||||
|
return SignedHeader{
|
||||||
|
Header: u.Attested.Header,
|
||||||
|
Signature: u.Signature,
|
||||||
|
SignatureSlot: u.SignatureSlot,
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Validate verifies the Merkle proof proving the execution payload header.
|
||||||
|
// Note that the sync committee signature of the attested header should be
|
||||||
|
// verified separately by a synced committee chain.
|
||||||
|
func (u *OptimisticUpdate) Validate() error {
|
||||||
|
return u.Attested.Validate()
|
||||||
|
}
|
||||||
|
|
||||||
|
// FinalityUpdate proves a finalized beacon header by a sync committee commitment
|
||||||
|
// on an attested beacon header, referring to the latest finalized header with a
|
||||||
|
// Merkle proof.
|
||||||
|
// It also proves the execution payload header belonging to both the attested and
|
||||||
|
// the finalized beacon header with Merkle proofs.
|
||||||
|
//
|
||||||
|
// See data structure definition here:
|
||||||
|
// https://github.com/ethereum/consensus-specs/blob/dev/specs/altair/light-client/sync-protocol.md#lightclientfinalityupdate
|
||||||
type FinalityUpdate struct {
|
type FinalityUpdate struct {
|
||||||
Attested, Finalized HeaderWithExecProof
|
Attested, Finalized HeaderWithExecProof
|
||||||
FinalityBranch merkle.Values
|
FinalityBranch merkle.Values
|
||||||
@ -163,6 +203,7 @@ type FinalityUpdate struct {
|
|||||||
SignatureSlot uint64
|
SignatureSlot uint64
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// SignedHeader returns the signed attested header of the update.
|
||||||
func (u *FinalityUpdate) SignedHeader() SignedHeader {
|
func (u *FinalityUpdate) SignedHeader() SignedHeader {
|
||||||
return SignedHeader{
|
return SignedHeader{
|
||||||
Header: u.Attested.Header,
|
Header: u.Attested.Header,
|
||||||
@ -171,6 +212,10 @@ func (u *FinalityUpdate) SignedHeader() SignedHeader {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Validate verifies the Merkle proofs proving the finalized beacon header and
|
||||||
|
// the execution payload headers belonging to the attested and finalized headers.
|
||||||
|
// Note that the sync committee signature of the attested header should be
|
||||||
|
// verified separately by a synced committee chain.
|
||||||
func (u *FinalityUpdate) Validate() error {
|
func (u *FinalityUpdate) Validate() error {
|
||||||
if err := u.Attested.Validate(); err != nil {
|
if err := u.Attested.Validate(); err != nil {
|
||||||
return err
|
return err
|
||||||
@ -186,6 +231,6 @@ func (u *FinalityUpdate) Validate() error {
|
|||||||
// finalized execution block.
|
// finalized execution block.
|
||||||
type ChainHeadEvent struct {
|
type ChainHeadEvent struct {
|
||||||
BeaconHead Header
|
BeaconHead Header
|
||||||
Block *types.Block
|
Block *ctypes.Block
|
||||||
Finalized common.Hash
|
Finalized common.Hash
|
||||||
}
|
}
|
||||||
|
Loading…
Reference in New Issue
Block a user