beacon/light: add CommitteeChain (#27766)
This change implements CommitteeChain which is a key component of the beacon light client. It is a passive data structure that can validate, hold and update a chain of beacon light sync committees and updates, starting from a checkpoint that proves the starting committee through a beacon block hash, header and corresponding state. Once synced to the current sync period, CommitteeChain can also validate signed beacon headers.
This commit is contained in:
parent
1048e2d6a3
commit
fff843cfaf
125
beacon/light/canonical.go
Normal file
125
beacon/light/canonical.go
Normal file
@ -0,0 +1,125 @@
|
||||
// Copyright 2023 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 light
|
||||
|
||||
import (
|
||||
"encoding/binary"
|
||||
"fmt"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common/lru"
|
||||
"github.com/ethereum/go-ethereum/ethdb"
|
||||
"github.com/ethereum/go-ethereum/log"
|
||||
"github.com/ethereum/go-ethereum/rlp"
|
||||
)
|
||||
|
||||
// canonicalStore stores instances of the given type in a database and caches
|
||||
// them in memory, associated with a continuous range of period numbers.
|
||||
// Note: canonicalStore is not thread safe and it is the caller's responsibility
|
||||
// to avoid concurrent access.
|
||||
type canonicalStore[T any] struct {
|
||||
keyPrefix []byte
|
||||
periods periodRange
|
||||
cache *lru.Cache[uint64, T]
|
||||
}
|
||||
|
||||
// newCanonicalStore creates a new canonicalStore and loads all keys associated
|
||||
// with the keyPrefix in order to determine the ranges available in the database.
|
||||
func newCanonicalStore[T any](db ethdb.Iteratee, keyPrefix []byte) (*canonicalStore[T], error) {
|
||||
cs := &canonicalStore[T]{
|
||||
keyPrefix: keyPrefix,
|
||||
cache: lru.NewCache[uint64, T](100),
|
||||
}
|
||||
var (
|
||||
iter = db.NewIterator(keyPrefix, nil)
|
||||
kl = len(keyPrefix)
|
||||
first = true
|
||||
)
|
||||
defer iter.Release()
|
||||
|
||||
for iter.Next() {
|
||||
if len(iter.Key()) != kl+8 {
|
||||
log.Warn("Invalid key length in the canonical chain database", "key", fmt.Sprintf("%#x", iter.Key()))
|
||||
continue
|
||||
}
|
||||
period := binary.BigEndian.Uint64(iter.Key()[kl : kl+8])
|
||||
if first {
|
||||
cs.periods.Start = period
|
||||
} else if cs.periods.End != period {
|
||||
return nil, fmt.Errorf("gap in the canonical chain database between periods %d and %d", cs.periods.End, period-1)
|
||||
}
|
||||
first = false
|
||||
cs.periods.End = period + 1
|
||||
}
|
||||
return cs, nil
|
||||
}
|
||||
|
||||
// databaseKey returns the database key belonging to the given period.
|
||||
func (cs *canonicalStore[T]) databaseKey(period uint64) []byte {
|
||||
return binary.BigEndian.AppendUint64(append([]byte{}, cs.keyPrefix...), period)
|
||||
}
|
||||
|
||||
// add adds the given item to the database. It also ensures that the range remains
|
||||
// continuous. Can be used either with a batch or database backend.
|
||||
func (cs *canonicalStore[T]) add(backend ethdb.KeyValueWriter, period uint64, value T) error {
|
||||
if !cs.periods.canExpand(period) {
|
||||
return fmt.Errorf("period expansion is not allowed, first: %d, next: %d, period: %d", cs.periods.Start, cs.periods.End, period)
|
||||
}
|
||||
enc, err := rlp.EncodeToBytes(value)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if err := backend.Put(cs.databaseKey(period), enc); err != nil {
|
||||
return err
|
||||
}
|
||||
cs.cache.Add(period, value)
|
||||
cs.periods.expand(period)
|
||||
return nil
|
||||
}
|
||||
|
||||
// deleteFrom removes items starting from the given period.
|
||||
func (cs *canonicalStore[T]) deleteFrom(db ethdb.KeyValueWriter, fromPeriod uint64) (deleted periodRange) {
|
||||
keepRange, deleteRange := cs.periods.split(fromPeriod)
|
||||
deleteRange.each(func(period uint64) {
|
||||
db.Delete(cs.databaseKey(period))
|
||||
cs.cache.Remove(period)
|
||||
})
|
||||
cs.periods = keepRange
|
||||
return deleteRange
|
||||
}
|
||||
|
||||
// get returns the item at the given period or the null value of the given type
|
||||
// if no item is present.
|
||||
func (cs *canonicalStore[T]) get(backend ethdb.KeyValueReader, period uint64) (T, bool) {
|
||||
var null, value T
|
||||
if !cs.periods.contains(period) {
|
||||
return null, false
|
||||
}
|
||||
if value, ok := cs.cache.Get(period); ok {
|
||||
return value, true
|
||||
}
|
||||
enc, err := backend.Get(cs.databaseKey(period))
|
||||
if err != nil {
|
||||
log.Error("Canonical store value not found", "period", period, "start", cs.periods.Start, "end", cs.periods.End)
|
||||
return null, false
|
||||
}
|
||||
if err := rlp.DecodeBytes(enc, &value); err != nil {
|
||||
log.Error("Error decoding canonical store value", "error", err)
|
||||
return null, false
|
||||
}
|
||||
cs.cache.Add(period, value)
|
||||
return value, true
|
||||
}
|
514
beacon/light/committee_chain.go
Normal file
514
beacon/light/committee_chain.go
Normal file
@ -0,0 +1,514 @@
|
||||
// Copyright 2023 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 light
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"math"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/ethereum/go-ethereum/beacon/params"
|
||||
"github.com/ethereum/go-ethereum/beacon/types"
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/common/lru"
|
||||
"github.com/ethereum/go-ethereum/common/mclock"
|
||||
"github.com/ethereum/go-ethereum/core/rawdb"
|
||||
"github.com/ethereum/go-ethereum/ethdb"
|
||||
"github.com/ethereum/go-ethereum/log"
|
||||
)
|
||||
|
||||
var (
|
||||
ErrNeedCommittee = errors.New("sync committee required")
|
||||
ErrInvalidUpdate = errors.New("invalid committee update")
|
||||
ErrInvalidPeriod = errors.New("invalid update period")
|
||||
ErrWrongCommitteeRoot = errors.New("wrong committee root")
|
||||
ErrCannotReorg = errors.New("can not reorg committee chain")
|
||||
)
|
||||
|
||||
// CommitteeChain is a passive data structure that can validate, hold and update
|
||||
// a chain of beacon light sync committees and updates. It requires at least one
|
||||
// externally set fixed committee root at the beginning of the chain which can
|
||||
// be set either based on a BootstrapData or a trusted source (a local beacon
|
||||
// full node). This makes the structure useful for both light client and light
|
||||
// server setups.
|
||||
//
|
||||
// It always maintains the following consistency constraints:
|
||||
// - a committee can only be present if its root hash matches an existing fixed
|
||||
// root or if it is proven by an update at the previous period
|
||||
// - an update can only be present if a committee is present at the same period
|
||||
// and the update signature is valid and has enough participants.
|
||||
// The committee at the next period (proven by the update) should also be
|
||||
// present (note that this means they can only be added together if neither
|
||||
// is present yet). If a fixed root is present at the next period then the
|
||||
// update can only be present if it proves the same committee root.
|
||||
//
|
||||
// Once synced to the current sync period, CommitteeChain can also validate
|
||||
// signed beacon headers.
|
||||
type CommitteeChain struct {
|
||||
// chainmu guards against concurrent access to the canonicalStore structures
|
||||
// (updates, committees, fixedCommitteeRoots) and ensures that they stay consistent
|
||||
// with each other and with committeeCache.
|
||||
chainmu sync.RWMutex
|
||||
db ethdb.KeyValueStore
|
||||
updates *canonicalStore[*types.LightClientUpdate]
|
||||
committees *canonicalStore[*types.SerializedSyncCommittee]
|
||||
fixedCommitteeRoots *canonicalStore[common.Hash]
|
||||
committeeCache *lru.Cache[uint64, syncCommittee] // cache deserialized committees
|
||||
|
||||
clock mclock.Clock // monotonic clock (simulated clock in tests)
|
||||
unixNano func() int64 // system clock (simulated clock in tests)
|
||||
sigVerifier committeeSigVerifier // BLS sig verifier (dummy verifier in tests)
|
||||
|
||||
config *types.ChainConfig
|
||||
signerThreshold int
|
||||
minimumUpdateScore types.UpdateScore
|
||||
enforceTime bool // enforceTime specifies whether the age of a signed header should be checked
|
||||
}
|
||||
|
||||
// NewCommitteeChain creates a new CommitteeChain.
|
||||
func NewCommitteeChain(db ethdb.KeyValueStore, config *types.ChainConfig, signerThreshold int, enforceTime bool) *CommitteeChain {
|
||||
return newCommitteeChain(db, config, signerThreshold, enforceTime, blsVerifier{}, &mclock.System{}, func() int64 { return time.Now().UnixNano() })
|
||||
}
|
||||
|
||||
// newCommitteeChain creates a new CommitteeChain with the option of replacing the
|
||||
// clock source and signature verification for testing purposes.
|
||||
func newCommitteeChain(db ethdb.KeyValueStore, config *types.ChainConfig, signerThreshold int, enforceTime bool, sigVerifier committeeSigVerifier, clock mclock.Clock, unixNano func() int64) *CommitteeChain {
|
||||
s := &CommitteeChain{
|
||||
committeeCache: lru.NewCache[uint64, syncCommittee](10),
|
||||
db: db,
|
||||
sigVerifier: sigVerifier,
|
||||
clock: clock,
|
||||
unixNano: unixNano,
|
||||
config: config,
|
||||
signerThreshold: signerThreshold,
|
||||
enforceTime: enforceTime,
|
||||
minimumUpdateScore: types.UpdateScore{
|
||||
SignerCount: uint32(signerThreshold),
|
||||
SubPeriodIndex: params.SyncPeriodLength / 16,
|
||||
},
|
||||
}
|
||||
|
||||
var err1, err2, err3 error
|
||||
if s.fixedCommitteeRoots, err1 = newCanonicalStore[common.Hash](db, rawdb.FixedCommitteeRootKey); err1 != nil {
|
||||
log.Error("Error creating fixed committee root store", "error", err1)
|
||||
}
|
||||
if s.committees, err2 = newCanonicalStore[*types.SerializedSyncCommittee](db, rawdb.SyncCommitteeKey); err2 != nil {
|
||||
log.Error("Error creating committee store", "error", err2)
|
||||
}
|
||||
if s.updates, err3 = newCanonicalStore[*types.LightClientUpdate](db, rawdb.BestUpdateKey); err3 != nil {
|
||||
log.Error("Error creating update store", "error", err3)
|
||||
}
|
||||
if err1 != nil || err2 != nil || err3 != nil || !s.checkConstraints() {
|
||||
log.Info("Resetting invalid committee chain")
|
||||
s.Reset()
|
||||
}
|
||||
// roll back invalid updates (might be necessary if forks have been changed since last time)
|
||||
for !s.updates.periods.isEmpty() {
|
||||
update, ok := s.updates.get(s.db, s.updates.periods.End-1)
|
||||
if !ok {
|
||||
log.Error("Sync committee update missing", "period", s.updates.periods.End-1)
|
||||
s.Reset()
|
||||
break
|
||||
}
|
||||
if valid, err := s.verifyUpdate(update); err != nil {
|
||||
log.Error("Error validating update", "period", s.updates.periods.End-1, "error", err)
|
||||
} else if valid {
|
||||
break
|
||||
}
|
||||
if err := s.rollback(s.updates.periods.End); err != nil {
|
||||
log.Error("Error writing batch into chain database", "error", err)
|
||||
}
|
||||
}
|
||||
if !s.committees.periods.isEmpty() {
|
||||
log.Trace("Sync committee chain loaded", "first period", s.committees.periods.Start, "last period", s.committees.periods.End-1)
|
||||
}
|
||||
return s
|
||||
}
|
||||
|
||||
// checkConstraints checks committee chain validity constraints
|
||||
func (s *CommitteeChain) checkConstraints() bool {
|
||||
isNotInFixedCommitteeRootRange := func(r periodRange) bool {
|
||||
return s.fixedCommitteeRoots.periods.isEmpty() ||
|
||||
r.Start < s.fixedCommitteeRoots.periods.Start ||
|
||||
r.Start >= s.fixedCommitteeRoots.periods.End
|
||||
}
|
||||
|
||||
valid := true
|
||||
if !s.updates.periods.isEmpty() {
|
||||
if isNotInFixedCommitteeRootRange(s.updates.periods) {
|
||||
log.Error("Start update is not in the fixed roots range")
|
||||
valid = false
|
||||
}
|
||||
if s.committees.periods.Start > s.updates.periods.Start || s.committees.periods.End <= s.updates.periods.End {
|
||||
log.Error("Missing committees in update range")
|
||||
valid = false
|
||||
}
|
||||
}
|
||||
if !s.committees.periods.isEmpty() {
|
||||
if isNotInFixedCommitteeRootRange(s.committees.periods) {
|
||||
log.Error("Start committee is not in the fixed roots range")
|
||||
valid = false
|
||||
}
|
||||
if s.committees.periods.End > s.fixedCommitteeRoots.periods.End && s.committees.periods.End > s.updates.periods.End+1 {
|
||||
log.Error("Last committee is neither in the fixed roots range nor proven by updates")
|
||||
valid = false
|
||||
}
|
||||
}
|
||||
return valid
|
||||
}
|
||||
|
||||
// Reset resets the committee chain.
|
||||
func (s *CommitteeChain) Reset() {
|
||||
s.chainmu.Lock()
|
||||
defer s.chainmu.Unlock()
|
||||
|
||||
if err := s.rollback(0); err != nil {
|
||||
log.Error("Error writing batch into chain database", "error", err)
|
||||
}
|
||||
}
|
||||
|
||||
// CheckpointInit initializes a CommitteeChain based on the checkpoint.
|
||||
// Note: if the chain is already initialized and the committees proven by the
|
||||
// checkpoint do match the existing chain then the chain is retained and the
|
||||
// new checkpoint becomes fixed.
|
||||
func (s *CommitteeChain) CheckpointInit(bootstrap *types.BootstrapData) error {
|
||||
s.chainmu.Lock()
|
||||
defer s.chainmu.Unlock()
|
||||
|
||||
if err := bootstrap.Validate(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
period := bootstrap.Header.SyncPeriod()
|
||||
if err := s.deleteFixedCommitteeRootsFrom(period + 2); err != nil {
|
||||
s.Reset()
|
||||
return err
|
||||
}
|
||||
if s.addFixedCommitteeRoot(period, bootstrap.CommitteeRoot) != nil {
|
||||
s.Reset()
|
||||
if err := s.addFixedCommitteeRoot(period, bootstrap.CommitteeRoot); err != nil {
|
||||
s.Reset()
|
||||
return err
|
||||
}
|
||||
}
|
||||
if err := s.addFixedCommitteeRoot(period+1, common.Hash(bootstrap.CommitteeBranch[0])); err != nil {
|
||||
s.Reset()
|
||||
return err
|
||||
}
|
||||
if err := s.addCommittee(period, bootstrap.Committee); err != nil {
|
||||
s.Reset()
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// addFixedCommitteeRoot sets a fixed committee root at the given period.
|
||||
// Note that the period where the first committee is added has to have a fixed
|
||||
// root which can either come from a BootstrapData or a trusted source.
|
||||
func (s *CommitteeChain) addFixedCommitteeRoot(period uint64, root common.Hash) error {
|
||||
if root == (common.Hash{}) {
|
||||
return ErrWrongCommitteeRoot
|
||||
}
|
||||
|
||||
batch := s.db.NewBatch()
|
||||
oldRoot := s.getCommitteeRoot(period)
|
||||
if !s.fixedCommitteeRoots.periods.canExpand(period) {
|
||||
// Note: the fixed committee root range should always be continuous and
|
||||
// therefore the expected syncing method is to forward sync and optionally
|
||||
// backward sync periods one by one, starting from a checkpoint. The only
|
||||
// case when a root that is not adjacent to the already fixed ones can be
|
||||
// fixed is when the same root has already been proven by an update chain.
|
||||
// In this case the all roots in between can and should be fixed.
|
||||
// This scenario makes sense when a new trusted checkpoint is added to an
|
||||
// existing chain, ensuring that it will not be rolled back (might be
|
||||
// important in case of low signer participation rate).
|
||||
if root != oldRoot {
|
||||
return ErrInvalidPeriod
|
||||
}
|
||||
// if the old root exists and matches the new one then it is guaranteed
|
||||
// that the given period is after the existing fixed range and the roots
|
||||
// in between can also be fixed.
|
||||
for p := s.fixedCommitteeRoots.periods.End; p < period; p++ {
|
||||
if err := s.fixedCommitteeRoots.add(batch, p, s.getCommitteeRoot(p)); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
}
|
||||
if oldRoot != (common.Hash{}) && (oldRoot != root) {
|
||||
// existing old root was different, we have to reorg the chain
|
||||
if err := s.rollback(period); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
if err := s.fixedCommitteeRoots.add(batch, period, root); err != nil {
|
||||
return err
|
||||
}
|
||||
if err := batch.Write(); err != nil {
|
||||
log.Error("Error writing batch into chain database", "error", err)
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// deleteFixedCommitteeRootsFrom deletes fixed roots starting from the given period.
|
||||
// It also maintains chain consistency, meaning that it also deletes updates and
|
||||
// committees if they are no longer supported by a valid update chain.
|
||||
func (s *CommitteeChain) deleteFixedCommitteeRootsFrom(period uint64) error {
|
||||
if period >= s.fixedCommitteeRoots.periods.End {
|
||||
return nil
|
||||
}
|
||||
batch := s.db.NewBatch()
|
||||
s.fixedCommitteeRoots.deleteFrom(batch, period)
|
||||
if s.updates.periods.isEmpty() || period <= s.updates.periods.Start {
|
||||
// Note: the first period of the update chain should always be fixed so if
|
||||
// the fixed root at the first update is removed then the entire update chain
|
||||
// and the proven committees have to be removed. Earlier committees in the
|
||||
// remaining fixed root range can stay.
|
||||
s.updates.deleteFrom(batch, period)
|
||||
s.deleteCommitteesFrom(batch, period)
|
||||
} else {
|
||||
// The update chain stays intact, some previously fixed committee roots might
|
||||
// get unfixed but are still proven by the update chain. If there were
|
||||
// committees present after the range proven by updates, those should be
|
||||
// removed if the belonging fixed roots are also removed.
|
||||
fromPeriod := s.updates.periods.End + 1 // not proven by updates
|
||||
if period > fromPeriod {
|
||||
fromPeriod = period // also not justified by fixed roots
|
||||
}
|
||||
s.deleteCommitteesFrom(batch, fromPeriod)
|
||||
}
|
||||
if err := batch.Write(); err != nil {
|
||||
log.Error("Error writing batch into chain database", "error", err)
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// deleteCommitteesFrom deletes committees starting from the given period.
|
||||
func (s *CommitteeChain) deleteCommitteesFrom(batch ethdb.Batch, period uint64) {
|
||||
deleted := s.committees.deleteFrom(batch, period)
|
||||
for period := deleted.Start; period < deleted.End; period++ {
|
||||
s.committeeCache.Remove(period)
|
||||
}
|
||||
}
|
||||
|
||||
// addCommittee adds a committee at the given period if possible.
|
||||
func (s *CommitteeChain) addCommittee(period uint64, committee *types.SerializedSyncCommittee) error {
|
||||
if !s.committees.periods.canExpand(period) {
|
||||
return ErrInvalidPeriod
|
||||
}
|
||||
root := s.getCommitteeRoot(period)
|
||||
if root == (common.Hash{}) {
|
||||
return ErrInvalidPeriod
|
||||
}
|
||||
if root != committee.Root() {
|
||||
return ErrWrongCommitteeRoot
|
||||
}
|
||||
if !s.committees.periods.contains(period) {
|
||||
if err := s.committees.add(s.db, period, committee); err != nil {
|
||||
return err
|
||||
}
|
||||
s.committeeCache.Remove(period)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// InsertUpdate adds a new update if possible.
|
||||
func (s *CommitteeChain) InsertUpdate(update *types.LightClientUpdate, nextCommittee *types.SerializedSyncCommittee) error {
|
||||
s.chainmu.Lock()
|
||||
defer s.chainmu.Unlock()
|
||||
|
||||
period := update.AttestedHeader.Header.SyncPeriod()
|
||||
if !s.updates.periods.canExpand(period) || !s.committees.periods.contains(period) {
|
||||
return ErrInvalidPeriod
|
||||
}
|
||||
if s.minimumUpdateScore.BetterThan(update.Score()) {
|
||||
return ErrInvalidUpdate
|
||||
}
|
||||
oldRoot := s.getCommitteeRoot(period + 1)
|
||||
reorg := oldRoot != (common.Hash{}) && oldRoot != update.NextSyncCommitteeRoot
|
||||
if oldUpdate, ok := s.updates.get(s.db, period); ok && !update.Score().BetterThan(oldUpdate.Score()) {
|
||||
// a better or equal update already exists; no changes, only fail if new one tried to reorg
|
||||
if reorg {
|
||||
return ErrCannotReorg
|
||||
}
|
||||
return nil
|
||||
}
|
||||
if s.fixedCommitteeRoots.periods.contains(period+1) && reorg {
|
||||
return ErrCannotReorg
|
||||
}
|
||||
if ok, err := s.verifyUpdate(update); err != nil {
|
||||
return err
|
||||
} else if !ok {
|
||||
return ErrInvalidUpdate
|
||||
}
|
||||
addCommittee := !s.committees.periods.contains(period+1) || reorg
|
||||
if addCommittee {
|
||||
if nextCommittee == nil {
|
||||
return ErrNeedCommittee
|
||||
}
|
||||
if nextCommittee.Root() != update.NextSyncCommitteeRoot {
|
||||
return ErrWrongCommitteeRoot
|
||||
}
|
||||
}
|
||||
if reorg {
|
||||
if err := s.rollback(period + 1); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
batch := s.db.NewBatch()
|
||||
if addCommittee {
|
||||
if err := s.committees.add(batch, period+1, nextCommittee); err != nil {
|
||||
return err
|
||||
}
|
||||
s.committeeCache.Remove(period + 1)
|
||||
}
|
||||
if err := s.updates.add(batch, period, update); err != nil {
|
||||
return err
|
||||
}
|
||||
if err := batch.Write(); err != nil {
|
||||
log.Error("Error writing batch into chain database", "error", err)
|
||||
return err
|
||||
}
|
||||
log.Info("Inserted new committee update", "period", period, "next committee root", update.NextSyncCommitteeRoot)
|
||||
return nil
|
||||
}
|
||||
|
||||
// NextSyncPeriod returns the next period where an update can be added and also
|
||||
// whether the chain is initialized at all.
|
||||
func (s *CommitteeChain) NextSyncPeriod() (uint64, bool) {
|
||||
s.chainmu.RLock()
|
||||
defer s.chainmu.RUnlock()
|
||||
|
||||
if s.committees.periods.isEmpty() {
|
||||
return 0, false
|
||||
}
|
||||
if !s.updates.periods.isEmpty() {
|
||||
return s.updates.periods.End, true
|
||||
}
|
||||
return s.committees.periods.End - 1, true
|
||||
}
|
||||
|
||||
// rollback removes all committees and fixed roots from the given period and updates
|
||||
// starting from the previous period.
|
||||
func (s *CommitteeChain) rollback(period uint64) error {
|
||||
max := s.updates.periods.End + 1
|
||||
if s.committees.periods.End > max {
|
||||
max = s.committees.periods.End
|
||||
}
|
||||
if s.fixedCommitteeRoots.periods.End > max {
|
||||
max = s.fixedCommitteeRoots.periods.End
|
||||
}
|
||||
for max > period {
|
||||
max--
|
||||
batch := s.db.NewBatch()
|
||||
s.deleteCommitteesFrom(batch, max)
|
||||
s.fixedCommitteeRoots.deleteFrom(batch, max)
|
||||
if max > 0 {
|
||||
s.updates.deleteFrom(batch, max-1)
|
||||
}
|
||||
if err := batch.Write(); err != nil {
|
||||
log.Error("Error writing batch into chain database", "error", err)
|
||||
return err
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// getCommitteeRoot returns the committee root at the given period, either fixed,
|
||||
// proven by a previous update or both. It returns an empty hash if the committee
|
||||
// root is unknown.
|
||||
func (s *CommitteeChain) getCommitteeRoot(period uint64) common.Hash {
|
||||
if root, ok := s.fixedCommitteeRoots.get(s.db, period); ok || period == 0 {
|
||||
return root
|
||||
}
|
||||
if update, ok := s.updates.get(s.db, period-1); ok {
|
||||
return update.NextSyncCommitteeRoot
|
||||
}
|
||||
return common.Hash{}
|
||||
}
|
||||
|
||||
// getSyncCommittee returns the deserialized sync committee at the given period.
|
||||
func (s *CommitteeChain) getSyncCommittee(period uint64) (syncCommittee, error) {
|
||||
if c, ok := s.committeeCache.Get(period); ok {
|
||||
return c, nil
|
||||
}
|
||||
if sc, ok := s.committees.get(s.db, period); ok {
|
||||
c, err := s.sigVerifier.deserializeSyncCommittee(sc)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("Sync committee #%d deserialization error: %v", period, err)
|
||||
}
|
||||
s.committeeCache.Add(period, c)
|
||||
return c, nil
|
||||
}
|
||||
return nil, fmt.Errorf("Missing serialized sync committee #%d", period)
|
||||
}
|
||||
|
||||
// VerifySignedHeader returns true if the given signed header has a valid signature
|
||||
// according to the local committee chain. The caller should ensure that the
|
||||
// committees advertised by the same source where the signed header came from are
|
||||
// synced before verifying the signature.
|
||||
// The age of the header is also returned (the time elapsed since the beginning
|
||||
// of the given slot, according to the local system clock). If enforceTime is
|
||||
// true then negative age (future) headers are rejected.
|
||||
func (s *CommitteeChain) VerifySignedHeader(head types.SignedHeader) (bool, time.Duration, error) {
|
||||
s.chainmu.RLock()
|
||||
defer s.chainmu.RUnlock()
|
||||
|
||||
return s.verifySignedHeader(head)
|
||||
}
|
||||
|
||||
func (s *CommitteeChain) verifySignedHeader(head types.SignedHeader) (bool, time.Duration, error) {
|
||||
var age time.Duration
|
||||
now := s.unixNano()
|
||||
if head.Header.Slot < (uint64(now-math.MinInt64)/uint64(time.Second)-s.config.GenesisTime)/12 {
|
||||
age = time.Duration(now - int64(time.Second)*int64(s.config.GenesisTime+head.Header.Slot*12))
|
||||
} else {
|
||||
age = time.Duration(math.MinInt64)
|
||||
}
|
||||
if s.enforceTime && age < 0 {
|
||||
return false, age, nil
|
||||
}
|
||||
committee, err := s.getSyncCommittee(types.SyncPeriod(head.SignatureSlot))
|
||||
if err != nil {
|
||||
return false, 0, err
|
||||
}
|
||||
if committee == nil {
|
||||
return false, age, nil
|
||||
}
|
||||
if signingRoot, err := s.config.Forks.SigningRoot(head.Header); err == nil {
|
||||
return s.sigVerifier.verifySignature(committee, signingRoot, &head.Signature), age, nil
|
||||
}
|
||||
return false, age, nil
|
||||
}
|
||||
|
||||
// verifyUpdate checks whether the header signature is correct and the update
|
||||
// fits into the specified constraints (assumes that the update has been
|
||||
// successfully validated previously)
|
||||
func (s *CommitteeChain) verifyUpdate(update *types.LightClientUpdate) (bool, error) {
|
||||
// Note: SignatureSlot determines the sync period of the committee used for signature
|
||||
// verification. Though in reality SignatureSlot is always bigger than update.Header.Slot,
|
||||
// setting them as equal here enforces the rule that they have to be in the same sync
|
||||
// period in order for the light client update proof to be meaningful.
|
||||
ok, age, err := s.verifySignedHeader(update.AttestedHeader)
|
||||
if age < 0 {
|
||||
log.Warn("Future committee update received", "age", age)
|
||||
}
|
||||
return ok, err
|
||||
}
|
356
beacon/light/committee_chain_test.go
Normal file
356
beacon/light/committee_chain_test.go
Normal file
@ -0,0 +1,356 @@
|
||||
// Copyright 2022 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 light
|
||||
|
||||
import (
|
||||
"crypto/rand"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/ethereum/go-ethereum/beacon/params"
|
||||
"github.com/ethereum/go-ethereum/beacon/types"
|
||||
"github.com/ethereum/go-ethereum/common/mclock"
|
||||
"github.com/ethereum/go-ethereum/ethdb/memorydb"
|
||||
)
|
||||
|
||||
var (
|
||||
testGenesis = newTestGenesis()
|
||||
testGenesis2 = newTestGenesis()
|
||||
|
||||
tfBase = newTestForks(testGenesis, types.Forks{
|
||||
&types.Fork{Epoch: 0, Version: []byte{0}},
|
||||
})
|
||||
tfAlternative = newTestForks(testGenesis, types.Forks{
|
||||
&types.Fork{Epoch: 0, Version: []byte{0}},
|
||||
&types.Fork{Epoch: 0x700, Version: []byte{1}},
|
||||
})
|
||||
tfAnotherGenesis = newTestForks(testGenesis2, types.Forks{
|
||||
&types.Fork{Epoch: 0, Version: []byte{0}},
|
||||
})
|
||||
|
||||
tcBase = newTestCommitteeChain(nil, tfBase, true, 0, 10, 400, false)
|
||||
tcBaseWithInvalidUpdates = newTestCommitteeChain(tcBase, tfBase, false, 5, 10, 200, false) // signer count too low
|
||||
tcBaseWithBetterUpdates = newTestCommitteeChain(tcBase, tfBase, false, 5, 10, 440, false)
|
||||
tcReorgWithWorseUpdates = newTestCommitteeChain(tcBase, tfBase, true, 5, 10, 400, false)
|
||||
tcReorgWithWorseUpdates2 = newTestCommitteeChain(tcBase, tfBase, true, 5, 10, 380, false)
|
||||
tcReorgWithBetterUpdates = newTestCommitteeChain(tcBase, tfBase, true, 5, 10, 420, false)
|
||||
tcReorgWithFinalizedUpdates = newTestCommitteeChain(tcBase, tfBase, true, 5, 10, 400, true)
|
||||
tcFork = newTestCommitteeChain(tcBase, tfAlternative, true, 7, 10, 400, false)
|
||||
tcAnotherGenesis = newTestCommitteeChain(nil, tfAnotherGenesis, true, 0, 10, 400, false)
|
||||
)
|
||||
|
||||
func TestCommitteeChainFixedCommitteeRoots(t *testing.T) {
|
||||
for _, reload := range []bool{false, true} {
|
||||
c := newCommitteeChainTest(t, tfBase, 300, true)
|
||||
c.setClockPeriod(7)
|
||||
c.addFixedCommitteeRoot(tcBase, 4, nil)
|
||||
c.addFixedCommitteeRoot(tcBase, 5, nil)
|
||||
c.addFixedCommitteeRoot(tcBase, 6, nil)
|
||||
c.addFixedCommitteeRoot(tcBase, 8, ErrInvalidPeriod) // range has to be continuous
|
||||
c.addFixedCommitteeRoot(tcBase, 3, nil)
|
||||
c.addFixedCommitteeRoot(tcBase, 2, nil)
|
||||
if reload {
|
||||
c.reloadChain()
|
||||
}
|
||||
c.addCommittee(tcBase, 4, nil)
|
||||
c.addCommittee(tcBase, 6, ErrInvalidPeriod) // range has to be continuous
|
||||
c.addCommittee(tcBase, 5, nil)
|
||||
c.addCommittee(tcBase, 6, nil)
|
||||
c.addCommittee(tcAnotherGenesis, 3, ErrWrongCommitteeRoot)
|
||||
c.addCommittee(tcBase, 3, nil)
|
||||
if reload {
|
||||
c.reloadChain()
|
||||
}
|
||||
c.verifyRange(tcBase, 3, 6)
|
||||
}
|
||||
}
|
||||
|
||||
func TestCommitteeChainCheckpointSync(t *testing.T) {
|
||||
for _, enforceTime := range []bool{false, true} {
|
||||
for _, reload := range []bool{false, true} {
|
||||
c := newCommitteeChainTest(t, tfBase, 300, enforceTime)
|
||||
if enforceTime {
|
||||
c.setClockPeriod(6)
|
||||
}
|
||||
c.insertUpdate(tcBase, 3, true, ErrInvalidPeriod)
|
||||
c.addFixedCommitteeRoot(tcBase, 3, nil)
|
||||
c.addFixedCommitteeRoot(tcBase, 4, nil)
|
||||
c.insertUpdate(tcBase, 4, true, ErrInvalidPeriod) // still no committee
|
||||
c.addCommittee(tcBase, 3, nil)
|
||||
c.addCommittee(tcBase, 4, nil)
|
||||
if reload {
|
||||
c.reloadChain()
|
||||
}
|
||||
c.verifyRange(tcBase, 3, 4)
|
||||
c.insertUpdate(tcBase, 3, false, nil) // update can be added without committee here
|
||||
c.insertUpdate(tcBase, 4, false, ErrNeedCommittee) // but not here as committee 5 is not there yet
|
||||
c.insertUpdate(tcBase, 4, true, nil)
|
||||
c.verifyRange(tcBase, 3, 5)
|
||||
c.insertUpdate(tcBaseWithInvalidUpdates, 5, true, ErrInvalidUpdate) // signer count too low
|
||||
c.insertUpdate(tcBase, 5, true, nil)
|
||||
if reload {
|
||||
c.reloadChain()
|
||||
}
|
||||
if enforceTime {
|
||||
c.insertUpdate(tcBase, 6, true, ErrInvalidUpdate) // future update rejected
|
||||
c.setClockPeriod(7)
|
||||
}
|
||||
c.insertUpdate(tcBase, 6, true, nil) // when the time comes it's accepted
|
||||
if reload {
|
||||
c.reloadChain()
|
||||
}
|
||||
if enforceTime {
|
||||
c.verifyRange(tcBase, 3, 6) // committee 7 is there but still in the future
|
||||
c.setClockPeriod(8)
|
||||
}
|
||||
c.verifyRange(tcBase, 3, 7) // now period 7 can also be verified
|
||||
// try reverse syncing an update
|
||||
c.insertUpdate(tcBase, 2, false, ErrInvalidPeriod) // fixed committee is needed first
|
||||
c.addFixedCommitteeRoot(tcBase, 2, nil)
|
||||
c.addCommittee(tcBase, 2, nil)
|
||||
c.insertUpdate(tcBase, 2, false, nil)
|
||||
c.verifyRange(tcBase, 2, 7)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestCommitteeChainReorg(t *testing.T) {
|
||||
for _, reload := range []bool{false, true} {
|
||||
for _, addBetterUpdates := range []bool{false, true} {
|
||||
c := newCommitteeChainTest(t, tfBase, 300, true)
|
||||
c.setClockPeriod(11)
|
||||
c.addFixedCommitteeRoot(tcBase, 3, nil)
|
||||
c.addFixedCommitteeRoot(tcBase, 4, nil)
|
||||
c.addCommittee(tcBase, 3, nil)
|
||||
for period := uint64(3); period < 10; period++ {
|
||||
c.insertUpdate(tcBase, period, true, nil)
|
||||
}
|
||||
if reload {
|
||||
c.reloadChain()
|
||||
}
|
||||
c.verifyRange(tcBase, 3, 10)
|
||||
c.insertUpdate(tcReorgWithWorseUpdates, 5, true, ErrCannotReorg)
|
||||
c.insertUpdate(tcReorgWithWorseUpdates2, 5, true, ErrCannotReorg)
|
||||
if addBetterUpdates {
|
||||
// add better updates for the base chain and expect first reorg to fail
|
||||
// (only add updates as committees should be the same)
|
||||
for period := uint64(5); period < 10; period++ {
|
||||
c.insertUpdate(tcBaseWithBetterUpdates, period, false, nil)
|
||||
}
|
||||
if reload {
|
||||
c.reloadChain()
|
||||
}
|
||||
c.verifyRange(tcBase, 3, 10) // still on the same chain
|
||||
c.insertUpdate(tcReorgWithBetterUpdates, 5, true, ErrCannotReorg)
|
||||
} else {
|
||||
// reorg with better updates
|
||||
c.insertUpdate(tcReorgWithBetterUpdates, 5, false, ErrNeedCommittee)
|
||||
c.verifyRange(tcBase, 3, 10) // no success yet, still on the base chain
|
||||
c.verifyRange(tcReorgWithBetterUpdates, 3, 5)
|
||||
c.insertUpdate(tcReorgWithBetterUpdates, 5, true, nil)
|
||||
// successful reorg, base chain should only match before the reorg period
|
||||
if reload {
|
||||
c.reloadChain()
|
||||
}
|
||||
c.verifyRange(tcBase, 3, 5)
|
||||
c.verifyRange(tcReorgWithBetterUpdates, 3, 6)
|
||||
for period := uint64(6); period < 10; period++ {
|
||||
c.insertUpdate(tcReorgWithBetterUpdates, period, true, nil)
|
||||
}
|
||||
c.verifyRange(tcReorgWithBetterUpdates, 3, 10)
|
||||
}
|
||||
// reorg with finalized updates; should succeed even if base chain updates
|
||||
// have been improved because a finalized update beats everything else
|
||||
c.insertUpdate(tcReorgWithFinalizedUpdates, 5, false, ErrNeedCommittee)
|
||||
c.insertUpdate(tcReorgWithFinalizedUpdates, 5, true, nil)
|
||||
if reload {
|
||||
c.reloadChain()
|
||||
}
|
||||
c.verifyRange(tcReorgWithFinalizedUpdates, 3, 6)
|
||||
for period := uint64(6); period < 10; period++ {
|
||||
c.insertUpdate(tcReorgWithFinalizedUpdates, period, true, nil)
|
||||
}
|
||||
c.verifyRange(tcReorgWithFinalizedUpdates, 3, 10)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestCommitteeChainFork(t *testing.T) {
|
||||
c := newCommitteeChainTest(t, tfAlternative, 300, true)
|
||||
c.setClockPeriod(11)
|
||||
// trying to sync a chain on an alternative fork with the base chain data
|
||||
c.addFixedCommitteeRoot(tcBase, 0, nil)
|
||||
c.addFixedCommitteeRoot(tcBase, 1, nil)
|
||||
c.addCommittee(tcBase, 0, nil)
|
||||
// shared section should sync without errors
|
||||
for period := uint64(0); period < 7; period++ {
|
||||
c.insertUpdate(tcBase, period, true, nil)
|
||||
}
|
||||
c.insertUpdate(tcBase, 7, true, ErrInvalidUpdate) // wrong fork
|
||||
// committee root #7 is still the same but signatures are already signed with
|
||||
// a different fork id so period 7 should only verify on the alternative fork
|
||||
c.verifyRange(tcBase, 0, 6)
|
||||
c.verifyRange(tcFork, 0, 7)
|
||||
for period := uint64(7); period < 10; period++ {
|
||||
c.insertUpdate(tcFork, period, true, nil)
|
||||
}
|
||||
c.verifyRange(tcFork, 0, 10)
|
||||
// reload the chain while switching to the base fork
|
||||
c.config = tfBase
|
||||
c.reloadChain()
|
||||
// updates 7..9 should be rolled back now
|
||||
c.verifyRange(tcFork, 0, 6) // again, period 7 only verifies on the right fork
|
||||
c.verifyRange(tcBase, 0, 7)
|
||||
c.insertUpdate(tcFork, 7, true, ErrInvalidUpdate) // wrong fork
|
||||
for period := uint64(7); period < 10; period++ {
|
||||
c.insertUpdate(tcBase, period, true, nil)
|
||||
}
|
||||
c.verifyRange(tcBase, 0, 10)
|
||||
}
|
||||
|
||||
type committeeChainTest struct {
|
||||
t *testing.T
|
||||
db *memorydb.Database
|
||||
clock *mclock.Simulated
|
||||
config types.ChainConfig
|
||||
signerThreshold int
|
||||
enforceTime bool
|
||||
chain *CommitteeChain
|
||||
}
|
||||
|
||||
func newCommitteeChainTest(t *testing.T, config types.ChainConfig, signerThreshold int, enforceTime bool) *committeeChainTest {
|
||||
c := &committeeChainTest{
|
||||
t: t,
|
||||
db: memorydb.New(),
|
||||
clock: &mclock.Simulated{},
|
||||
config: config,
|
||||
signerThreshold: signerThreshold,
|
||||
enforceTime: enforceTime,
|
||||
}
|
||||
c.chain = newCommitteeChain(c.db, &config, signerThreshold, enforceTime, dummyVerifier{}, c.clock, func() int64 { return int64(c.clock.Now()) })
|
||||
return c
|
||||
}
|
||||
|
||||
func (c *committeeChainTest) reloadChain() {
|
||||
c.chain = newCommitteeChain(c.db, &c.config, c.signerThreshold, c.enforceTime, dummyVerifier{}, c.clock, func() int64 { return int64(c.clock.Now()) })
|
||||
}
|
||||
|
||||
func (c *committeeChainTest) setClockPeriod(period float64) {
|
||||
target := mclock.AbsTime(period * float64(time.Second*12*params.SyncPeriodLength))
|
||||
wait := time.Duration(target - c.clock.Now())
|
||||
if wait < 0 {
|
||||
c.t.Fatalf("Invalid setClockPeriod")
|
||||
}
|
||||
c.clock.Run(wait)
|
||||
}
|
||||
|
||||
func (c *committeeChainTest) addFixedCommitteeRoot(tc *testCommitteeChain, period uint64, expErr error) {
|
||||
if err := c.chain.addFixedCommitteeRoot(period, tc.periods[period].committee.Root()); err != expErr {
|
||||
c.t.Errorf("Incorrect error output from addFixedCommitteeRoot at period %d (expected %v, got %v)", period, expErr, err)
|
||||
}
|
||||
}
|
||||
|
||||
func (c *committeeChainTest) addCommittee(tc *testCommitteeChain, period uint64, expErr error) {
|
||||
if err := c.chain.addCommittee(period, tc.periods[period].committee); err != expErr {
|
||||
c.t.Errorf("Incorrect error output from addCommittee at period %d (expected %v, got %v)", period, expErr, err)
|
||||
}
|
||||
}
|
||||
|
||||
func (c *committeeChainTest) insertUpdate(tc *testCommitteeChain, period uint64, addCommittee bool, expErr error) {
|
||||
var committee *types.SerializedSyncCommittee
|
||||
if addCommittee {
|
||||
committee = tc.periods[period+1].committee
|
||||
}
|
||||
if err := c.chain.InsertUpdate(tc.periods[period].update, committee); err != expErr {
|
||||
c.t.Errorf("Incorrect error output from InsertUpdate at period %d (expected %v, got %v)", period, expErr, err)
|
||||
}
|
||||
}
|
||||
|
||||
func (c *committeeChainTest) verifySignedHeader(tc *testCommitteeChain, period float64, expOk bool) {
|
||||
slot := uint64(period * float64(params.SyncPeriodLength))
|
||||
signedHead := GenerateTestSignedHeader(types.Header{Slot: slot}, &tc.config, tc.periods[types.SyncPeriod(slot)].committee, slot+1, 400)
|
||||
if ok, _, _ := c.chain.VerifySignedHeader(signedHead); ok != expOk {
|
||||
c.t.Errorf("Incorrect output from VerifySignedHeader at period %f (expected %v, got %v)", period, expOk, ok)
|
||||
}
|
||||
}
|
||||
|
||||
func (c *committeeChainTest) verifyRange(tc *testCommitteeChain, begin, end uint64) {
|
||||
if begin > 0 {
|
||||
c.verifySignedHeader(tc, float64(begin)-0.5, false)
|
||||
}
|
||||
for period := begin; period <= end; period++ {
|
||||
c.verifySignedHeader(tc, float64(period)+0.5, true)
|
||||
}
|
||||
c.verifySignedHeader(tc, float64(end)+1.5, false)
|
||||
}
|
||||
|
||||
func newTestGenesis() types.ChainConfig {
|
||||
var config types.ChainConfig
|
||||
rand.Read(config.GenesisValidatorsRoot[:])
|
||||
return config
|
||||
}
|
||||
|
||||
func newTestForks(config types.ChainConfig, forks types.Forks) types.ChainConfig {
|
||||
for _, fork := range forks {
|
||||
config.AddFork(fork.Name, fork.Epoch, fork.Version)
|
||||
}
|
||||
return config
|
||||
}
|
||||
|
||||
func newTestCommitteeChain(parent *testCommitteeChain, config types.ChainConfig, newCommittees bool, begin, end int, signerCount int, finalizedHeader bool) *testCommitteeChain {
|
||||
tc := &testCommitteeChain{
|
||||
config: config,
|
||||
}
|
||||
if parent != nil {
|
||||
tc.periods = make([]testPeriod, len(parent.periods))
|
||||
copy(tc.periods, parent.periods)
|
||||
}
|
||||
if newCommittees {
|
||||
if begin == 0 {
|
||||
tc.fillCommittees(begin, end+1)
|
||||
} else {
|
||||
tc.fillCommittees(begin+1, end+1)
|
||||
}
|
||||
}
|
||||
tc.fillUpdates(begin, end, signerCount, finalizedHeader)
|
||||
return tc
|
||||
}
|
||||
|
||||
type testPeriod struct {
|
||||
committee *types.SerializedSyncCommittee
|
||||
update *types.LightClientUpdate
|
||||
}
|
||||
|
||||
type testCommitteeChain struct {
|
||||
periods []testPeriod
|
||||
config types.ChainConfig
|
||||
}
|
||||
|
||||
func (tc *testCommitteeChain) fillCommittees(begin, end int) {
|
||||
if len(tc.periods) <= end {
|
||||
tc.periods = append(tc.periods, make([]testPeriod, end+1-len(tc.periods))...)
|
||||
}
|
||||
for i := begin; i <= end; i++ {
|
||||
tc.periods[i].committee = GenerateTestCommittee()
|
||||
}
|
||||
}
|
||||
|
||||
func (tc *testCommitteeChain) fillUpdates(begin, end int, signerCount int, finalizedHeader bool) {
|
||||
for i := begin; i <= end; i++ {
|
||||
tc.periods[i].update = GenerateTestUpdate(&tc.config, uint64(i), tc.periods[i].committee, tc.periods[i+1].committee, signerCount, finalizedHeader)
|
||||
}
|
||||
}
|
78
beacon/light/range.go
Normal file
78
beacon/light/range.go
Normal file
@ -0,0 +1,78 @@
|
||||
// Copyright 2023 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 light
|
||||
|
||||
// periodRange represents a (possibly zero-length) range of integers (sync periods).
|
||||
type periodRange struct {
|
||||
Start, End uint64
|
||||
}
|
||||
|
||||
// isEmpty returns true if the length of the range is zero.
|
||||
func (a periodRange) isEmpty() bool {
|
||||
return a.End == a.Start
|
||||
}
|
||||
|
||||
// contains returns true if the range includes the given period.
|
||||
func (a periodRange) contains(period uint64) bool {
|
||||
return period >= a.Start && period < a.End
|
||||
}
|
||||
|
||||
// canExpand returns true if the range includes or can be expanded with the given
|
||||
// period (either the range is empty or the given period is inside, right before or
|
||||
// right after the range).
|
||||
func (a periodRange) canExpand(period uint64) bool {
|
||||
return a.isEmpty() || (period+1 >= a.Start && period <= a.End)
|
||||
}
|
||||
|
||||
// expand expands the range with the given period.
|
||||
// This method assumes that canExpand returned true: otherwise this is a no-op.
|
||||
func (a *periodRange) expand(period uint64) {
|
||||
if a.isEmpty() {
|
||||
a.Start, a.End = period, period+1
|
||||
return
|
||||
}
|
||||
if a.Start == period+1 {
|
||||
a.Start--
|
||||
}
|
||||
if a.End == period {
|
||||
a.End++
|
||||
}
|
||||
}
|
||||
|
||||
// split splits the range into two ranges. The 'fromPeriod' will be the first
|
||||
// element in the second range (if present).
|
||||
// The original range is unchanged by this operation
|
||||
func (a *periodRange) split(fromPeriod uint64) (periodRange, periodRange) {
|
||||
if fromPeriod <= a.Start {
|
||||
// First range empty, everything in second range,
|
||||
return periodRange{}, *a
|
||||
}
|
||||
if fromPeriod >= a.End {
|
||||
// Second range empty, everything in first range,
|
||||
return *a, periodRange{}
|
||||
}
|
||||
x := periodRange{a.Start, fromPeriod}
|
||||
y := periodRange{fromPeriod, a.End}
|
||||
return x, y
|
||||
}
|
||||
|
||||
// each invokes the supplied function fn once per period in range
|
||||
func (a *periodRange) each(fn func(uint64)) {
|
||||
for p := a.Start; p < a.End; p++ {
|
||||
fn(p)
|
||||
}
|
||||
}
|
152
beacon/light/test_helpers.go
Normal file
152
beacon/light/test_helpers.go
Normal file
@ -0,0 +1,152 @@
|
||||
// Copyright 2023 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 light
|
||||
|
||||
import (
|
||||
"crypto/rand"
|
||||
"crypto/sha256"
|
||||
mrand "math/rand"
|
||||
|
||||
"github.com/ethereum/go-ethereum/beacon/merkle"
|
||||
"github.com/ethereum/go-ethereum/beacon/params"
|
||||
"github.com/ethereum/go-ethereum/beacon/types"
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
)
|
||||
|
||||
func GenerateTestCommittee() *types.SerializedSyncCommittee {
|
||||
s := new(types.SerializedSyncCommittee)
|
||||
rand.Read(s[:32])
|
||||
return s
|
||||
}
|
||||
|
||||
func GenerateTestUpdate(config *types.ChainConfig, period uint64, committee, nextCommittee *types.SerializedSyncCommittee, signerCount int, finalizedHeader bool) *types.LightClientUpdate {
|
||||
update := new(types.LightClientUpdate)
|
||||
update.NextSyncCommitteeRoot = nextCommittee.Root()
|
||||
var attestedHeader types.Header
|
||||
if finalizedHeader {
|
||||
update.FinalizedHeader = new(types.Header)
|
||||
*update.FinalizedHeader, update.NextSyncCommitteeBranch = makeTestHeaderWithMerkleProof(types.SyncPeriodStart(period)+100, params.StateIndexNextSyncCommittee, merkle.Value(update.NextSyncCommitteeRoot))
|
||||
attestedHeader, update.FinalityBranch = makeTestHeaderWithMerkleProof(types.SyncPeriodStart(period)+200, params.StateIndexFinalBlock, merkle.Value(update.FinalizedHeader.Hash()))
|
||||
} else {
|
||||
attestedHeader, update.NextSyncCommitteeBranch = makeTestHeaderWithMerkleProof(types.SyncPeriodStart(period)+2000, params.StateIndexNextSyncCommittee, merkle.Value(update.NextSyncCommitteeRoot))
|
||||
}
|
||||
update.AttestedHeader = GenerateTestSignedHeader(attestedHeader, config, committee, attestedHeader.Slot+1, signerCount)
|
||||
return update
|
||||
}
|
||||
|
||||
func GenerateTestSignedHeader(header types.Header, config *types.ChainConfig, committee *types.SerializedSyncCommittee, signatureSlot uint64, signerCount int) types.SignedHeader {
|
||||
bitmask := makeBitmask(signerCount)
|
||||
signingRoot, _ := config.Forks.SigningRoot(header)
|
||||
c, _ := dummyVerifier{}.deserializeSyncCommittee(committee)
|
||||
return types.SignedHeader{
|
||||
Header: header,
|
||||
Signature: types.SyncAggregate{
|
||||
Signers: bitmask,
|
||||
Signature: makeDummySignature(c.(dummySyncCommittee), signingRoot, bitmask),
|
||||
},
|
||||
SignatureSlot: signatureSlot,
|
||||
}
|
||||
}
|
||||
|
||||
func GenerateTestCheckpoint(period uint64, committee *types.SerializedSyncCommittee) *types.BootstrapData {
|
||||
header, branch := makeTestHeaderWithMerkleProof(types.SyncPeriodStart(period)+200, params.StateIndexSyncCommittee, merkle.Value(committee.Root()))
|
||||
return &types.BootstrapData{
|
||||
Header: header,
|
||||
Committee: committee,
|
||||
CommitteeRoot: committee.Root(),
|
||||
CommitteeBranch: branch,
|
||||
}
|
||||
}
|
||||
|
||||
func makeBitmask(signerCount int) (bitmask [params.SyncCommitteeBitmaskSize]byte) {
|
||||
for i := 0; i < params.SyncCommitteeSize; i++ {
|
||||
if mrand.Intn(params.SyncCommitteeSize-i) < signerCount {
|
||||
bitmask[i/8] += byte(1) << (i & 7)
|
||||
signerCount--
|
||||
}
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
func makeTestHeaderWithMerkleProof(slot, index uint64, value merkle.Value) (types.Header, merkle.Values) {
|
||||
var branch merkle.Values
|
||||
hasher := sha256.New()
|
||||
for index > 1 {
|
||||
var proofHash merkle.Value
|
||||
rand.Read(proofHash[:])
|
||||
hasher.Reset()
|
||||
if index&1 == 0 {
|
||||
hasher.Write(value[:])
|
||||
hasher.Write(proofHash[:])
|
||||
} else {
|
||||
hasher.Write(proofHash[:])
|
||||
hasher.Write(value[:])
|
||||
}
|
||||
hasher.Sum(value[:0])
|
||||
index >>= 1
|
||||
branch = append(branch, proofHash)
|
||||
}
|
||||
return types.Header{Slot: slot, StateRoot: common.Hash(value)}, branch
|
||||
}
|
||||
|
||||
// syncCommittee holds either a blsSyncCommittee or a fake dummySyncCommittee used for testing
|
||||
type syncCommittee interface{}
|
||||
|
||||
// committeeSigVerifier verifies sync committee signatures (either proper BLS
|
||||
// signatures or fake signatures used for testing)
|
||||
type committeeSigVerifier interface {
|
||||
deserializeSyncCommittee(s *types.SerializedSyncCommittee) (syncCommittee, error)
|
||||
verifySignature(committee syncCommittee, signedRoot common.Hash, aggregate *types.SyncAggregate) bool
|
||||
}
|
||||
|
||||
// blsVerifier implements committeeSigVerifier
|
||||
type blsVerifier struct{}
|
||||
|
||||
// deserializeSyncCommittee implements committeeSigVerifier
|
||||
func (blsVerifier) deserializeSyncCommittee(s *types.SerializedSyncCommittee) (syncCommittee, error) {
|
||||
return s.Deserialize()
|
||||
}
|
||||
|
||||
// verifySignature implements committeeSigVerifier
|
||||
func (blsVerifier) verifySignature(committee syncCommittee, signingRoot common.Hash, aggregate *types.SyncAggregate) bool {
|
||||
return committee.(*types.SyncCommittee).VerifySignature(signingRoot, aggregate)
|
||||
}
|
||||
|
||||
type dummySyncCommittee [32]byte
|
||||
|
||||
// dummyVerifier implements committeeSigVerifier
|
||||
type dummyVerifier struct{}
|
||||
|
||||
// deserializeSyncCommittee implements committeeSigVerifier
|
||||
func (dummyVerifier) deserializeSyncCommittee(s *types.SerializedSyncCommittee) (syncCommittee, error) {
|
||||
var sc dummySyncCommittee
|
||||
copy(sc[:], s[:32])
|
||||
return sc, nil
|
||||
}
|
||||
|
||||
// verifySignature implements committeeSigVerifier
|
||||
func (dummyVerifier) verifySignature(committee syncCommittee, signingRoot common.Hash, aggregate *types.SyncAggregate) bool {
|
||||
return aggregate.Signature == makeDummySignature(committee.(dummySyncCommittee), signingRoot, aggregate.Signers)
|
||||
}
|
||||
|
||||
func makeDummySignature(committee dummySyncCommittee, signingRoot common.Hash, bitmask [params.SyncCommitteeBitmaskSize]byte) (sig [params.BLSSignatureSize]byte) {
|
||||
for i, b := range committee[:] {
|
||||
sig[i] = b ^ signingRoot[i]
|
||||
}
|
||||
copy(sig[32:], bitmask[:])
|
||||
return
|
||||
}
|
@ -25,6 +25,24 @@ import (
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
)
|
||||
|
||||
// BootstrapData contains a sync committee where light sync can be started,
|
||||
// together with a proof through a beacon header and corresponding state.
|
||||
// Note: BootstrapData is fetched from a server based on a known checkpoint hash.
|
||||
type BootstrapData struct {
|
||||
Header Header
|
||||
CommitteeRoot common.Hash
|
||||
Committee *SerializedSyncCommittee `rlp:"-"`
|
||||
CommitteeBranch merkle.Values
|
||||
}
|
||||
|
||||
// Validate verifies the proof included in BootstrapData.
|
||||
func (c *BootstrapData) Validate() error {
|
||||
if c.CommitteeRoot != c.Committee.Root() {
|
||||
return errors.New("wrong committee root")
|
||||
}
|
||||
return merkle.VerifyProof(c.Header.StateRoot, params.StateIndexSyncCommittee, c.CommitteeBranch, merkle.Value(c.CommitteeRoot))
|
||||
}
|
||||
|
||||
// LightClientUpdate is a proof of the next sync committee root based on a header
|
||||
// signed by the sync committee of the given period. Optionally, the update can
|
||||
// prove quasi-finality by the signed header referring to a previous, finalized
|
@ -132,6 +132,10 @@ var (
|
||||
|
||||
CliqueSnapshotPrefix = []byte("clique-")
|
||||
|
||||
BestUpdateKey = []byte("update-") // bigEndian64(syncPeriod) -> RLP(types.LightClientUpdate) (nextCommittee only referenced by root hash)
|
||||
FixedCommitteeRootKey = []byte("fixedRoot-") // bigEndian64(syncPeriod) -> committee root hash
|
||||
SyncCommitteeKey = []byte("committee-") // bigEndian64(syncPeriod) -> serialized committee
|
||||
|
||||
preimageCounter = metrics.NewRegisteredCounter("db/preimage/total", nil)
|
||||
preimageHitCounter = metrics.NewRegisteredCounter("db/preimage/hits", nil)
|
||||
)
|
||||
|
Loading…
Reference in New Issue
Block a user