diff --git a/.github/workflows/copyright.yml b/.github/workflows/copyright.yml index 021bed83eeb..ebc7245cbc0 100644 --- a/.github/workflows/copyright.yml +++ b/.github/workflows/copyright.yml @@ -1,8 +1,6 @@ name: Copyright check on: pull_request: - branches: - - development # Commented paths to avoid skipping required workflow # See https://github.community/t/feature-request-conditional-required-checks/16761 # paths: diff --git a/.github/workflows/mocks.yml b/.github/workflows/mocks.yml index 7e9519e52ed..965323b0a0e 100644 --- a/.github/workflows/mocks.yml +++ b/.github/workflows/mocks.yml @@ -1,8 +1,6 @@ name: Mocks check on: pull_request: - branches: - - development # Commented paths to avoid skipping required workflow # See https://github.community/t/feature-request-conditional-required-checks/16761 # paths: diff --git a/client/consensus/grandpa/authorities.go b/client/consensus/grandpa/authorities.go new file mode 100644 index 00000000000..37092c55bc2 --- /dev/null +++ b/client/consensus/grandpa/authorities.go @@ -0,0 +1,899 @@ +// Copyright 2023 ChainSafe Systems (ON) +// SPDX-License-Identifier: LGPL-3.0-only + +package grandpa + +import ( + "errors" + "fmt" + "sync" + + "github.com/ChainSafe/gossamer/pkg/scale" + "golang.org/x/exp/constraints" + "golang.org/x/exp/slices" +) + +var ( + errInvalidAuthoritySet = errors.New("invalid authority set, either empty or with" + + " an authority weight set to 0") + errDuplicateAuthoritySetChanges = errors.New("duplicate authority set hashNumber") + errMultiplePendingForcedAuthoritySetChanges = errors.New("multiple pending forced authority set " + + "changes are not allowed") + errForcedAuthoritySetChangeDependencyUnsatisfied = errors.New("a pending forced authority set hashNumber " + + "could not be applied since it must be applied after the pending standard hashNumber") + errForkTree = errors.New("invalid operation in the pending hashNumber tree") + errInvalidAuthorityList = errors.New("invalid authority list") +) + +// SharedAuthoritySet A shared authority set +type SharedAuthoritySet[H comparable, N constraints.Unsigned] struct { + mtx sync.Mutex + inner AuthoritySet[H, N] +} + +// IsDescendentOf is the function definition to determine if target is a descendant of base +type IsDescendentOf[H comparable] func(base, target H) (bool, error) + +// setIDNumber represents the set id and block number of an authority set hashNumber +type setIDNumber[N constraints.Unsigned] struct { + setID uint64 + blockNumber N +} + +// generic representation of hash and number tuple +type hashNumber[H, N any] struct { + hash H + number N +} + +// medianAuthoritySet represents the median and new set when a forced hashNumber has occurred +type medianAuthoritySet[H comparable, N constraints.Unsigned] struct { + median N + set AuthoritySet[H, N] +} + +// Current Get the current set id and a reference to the current authority set. +func (sas *SharedAuthoritySet[H, N]) Current() (uint64, *[]Authority) { + sas.mtx.Lock() + defer sas.mtx.Unlock() + return sas.inner.current() +} + +func (sas *SharedAuthoritySet[H, N]) revert() { //nolint //skipcq: SCC-U1000 + sas.mtx.Lock() + defer sas.mtx.Unlock() + sas.inner.revert() +} + +func (sas *SharedAuthoritySet[H, N]) nextChange(bestHash H, //nolint //skipcq: SCC-U1000 + isDescendentOf IsDescendentOf[H]) (*hashNumber[H, N], error) { + sas.mtx.Lock() + defer sas.mtx.Unlock() + return sas.inner.nextChange(bestHash, isDescendentOf) +} + +func (sas *SharedAuthoritySet[H, N]) addStandardChange(pending PendingChange[H, N], //nolint //skipcq: SCC-U1000 + isDescendentOf IsDescendentOf[H]) error { + sas.mtx.Lock() + defer sas.mtx.Unlock() + return sas.inner.addStandardChange(pending, isDescendentOf) +} + +func (sas *SharedAuthoritySet[H, N]) addForcedChange(pending PendingChange[H, N], //nolint //skipcq: SCC-U1000 + isDescendentOf IsDescendentOf[H]) error { + sas.mtx.Lock() + defer sas.mtx.Unlock() + return sas.inner.addForcedChange(pending, isDescendentOf) +} + +func (sas *SharedAuthoritySet[H, N]) addPendingChange(pending PendingChange[H, N], //nolint //skipcq: SCC-U1000 + isDescendentOf IsDescendentOf[H]) error { + sas.mtx.Lock() + defer sas.mtx.Unlock() + return sas.inner.addPendingChange(pending, isDescendentOf) +} + +// PendingChanges Inspect pending changes. Standard pending changes are iterated first, +// and the changes in the roots are traversed in pre-order, afterwards all +// forced changes are iterated. +func (sas *SharedAuthoritySet[H, N]) PendingChanges() []PendingChange[H, N] { + sas.mtx.Lock() + defer sas.mtx.Unlock() + return sas.inner.pendingChanges() +} + +// currentLimit Get the earliest limit-block number, if any. If there are pending changes across +// different forks, this method will return the earliest effective number (across the +// different branches) that is higher or equal to the given min number. +// +// Only standard changes are taken into account for the current +// limit, since any existing forced change should preclude the voter from voting. +func (sas *SharedAuthoritySet[H, N]) currentLimit(min N) (limit *N) { //nolint //skipcq: SCC-U1000 + sas.mtx.Lock() + defer sas.mtx.Unlock() + return sas.inner.currentLimit(min) +} + +func (sas *SharedAuthoritySet[H, N]) applyForcedChanges(bestHash H, //nolint //skipcq: SCC-U1000 + bestNumber N, + isDescendentOf IsDescendentOf[H], + telemetry *Telemetry) (newSet *medianAuthoritySet[H, N], err error) { + sas.mtx.Lock() + defer sas.mtx.Unlock() + return sas.inner.applyForcedChanges(bestHash, bestNumber, isDescendentOf, telemetry) +} + +// applyStandardChanges Apply or prune any pending transitions based on a finality trigger. This +// method ensures that if there are multiple changes in the same branch, +// finalising this block won't finalise past multiple transitions (i.e. +// transitions must be finalised in-order). The given function +// `is_descendent_of` should return `true` if the second hash (target) is a +// descendent of the first hash (base). +// +// When the set has changed, the return value will be a status type where newSetBlockInfo +// is the canonical block where the set last changed (i.e. the given +// hash and number). +func (sas *SharedAuthoritySet[H, N]) applyStandardChanges(finalisedHash H, //nolint //skipcq: SCC-U1000 + finalisedNumber N, + isDescendentOf IsDescendentOf[H], + telemetry *Telemetry) (status[H, N], error) { + sas.mtx.Lock() + defer sas.mtx.Unlock() + return sas.inner.applyStandardChanges(finalisedHash, finalisedNumber, isDescendentOf, telemetry) +} + +// EnactsStandardChange Check whether the given finalised block number enacts any standard +// authority set change (without triggering it), ensuring that if there are +// multiple changes in the same branch, finalising this block won't +// finalise past multiple transitions (i.e. transitions must be finalised +// in-order). Returns *true if the block being finalised enacts a +// change that can be immediately applied, *false if the block being +// finalised enacts a change but it cannot be applied yet since there are +// other dependent changes, and nil if no change is enacted. The given +// function `is_descendent_of` should return `true` if the second hash +// (target) is a descendent of the first hash (base). +func (sas *SharedAuthoritySet[H, N]) EnactsStandardChange(finalisedHash H, + finalisedNumber N, + isDescendentOf IsDescendentOf[H]) (*bool, error) { + sas.mtx.Lock() + defer sas.mtx.Unlock() + return sas.inner.EnactsStandardChange(finalisedHash, finalisedNumber, isDescendentOf) +} + +// status of the set after changes were applied. +type status[H comparable, N constraints.Unsigned] struct { + // Whether internal changes were made. + changed bool + // Not nil when underlying authority set has changed, containing the + // block where that set changed. + newSetBlock *hashNumber[H, N] +} + +// AuthoritySet A set of authorities. +type AuthoritySet[H comparable, N constraints.Unsigned] struct { + // The current active authorities. + currentAuthorities []Authority + // The current set id. + setID uint64 + // Tree of pending standard changes across forks. Standard changes are + // enacted on finality and must be enacted (i.e. finalised) in-order across + // a given branch + pendingStandardChanges ChangeTree[H, N] + // Pending forced changes across different forks (at most one per fork). + // Forced changes are enacted on block depth (not finality), for this + // reason only one forced hashNumber should exist per fork. When trying to + // apply forced changes we keep track of any pending standard changes that + // they may depend on, this is done by making sure that any pending hashNumber + // that is an ancestor of the forced changed and its effective block number + // is lower than the last finalised block (as signalled in the forced + // hashNumber) must be applied beforehand. + pendingForcedChanges []PendingChange[H, N] + // Track at which blocks the set id changed. This is useful when we need to prove finality for + // a given block since we can figure out what set the block belongs to and when the set + // started/ended. + authoritySetChanges AuthoritySetChanges[N] +} + +// invalidAuthorityList authority sets must be non-empty and all weights must be greater than 0 +func invalidAuthorityList(authorities []Authority) bool { //skipcq: RVV-B0001 + if len(authorities) == 0 { + return true + } + + for _, authority := range authorities { + if authority.Weight == 0 { + return true + } + } + return false +} + +// NewGenesisAuthoritySet Get a genesis set with given authorities. +func NewGenesisAuthoritySet[H comparable, N constraints.Unsigned]( + initial []Authority) (authSet *AuthoritySet[H, N], err error) { + return NewAuthoritySet[H, N]( + initial, + 0, + NewChangeTree[H, N](), + make([]PendingChange[H, N], 0), + make(AuthoritySetChanges[N], 0), + ) +} + +// NewAuthoritySet creates a new AuthoritySet +func NewAuthoritySet[H comparable, N constraints.Unsigned](authorities []Authority, + setId uint64, + pendingStandardChanges ChangeTree[H, N], + pendingForcedChanges []PendingChange[H, N], + authoritySetChanges AuthoritySetChanges[N], +) (authSet *AuthoritySet[H, N], err error) { + if invalidAuthorityList(authorities) { + return nil, errInvalidAuthorityList + } + + return &AuthoritySet[H, N]{ + currentAuthorities: authorities, + setID: setId, + pendingStandardChanges: pendingStandardChanges, + pendingForcedChanges: pendingForcedChanges, + authoritySetChanges: authoritySetChanges, + }, nil +} + +// current Get the current set id and a reference to the current authority set. +func (authSet *AuthoritySet[H, N]) current() (uint64, *[]Authority) { //skipcq: RVV-B0001 + return authSet.setID, &authSet.currentAuthorities +} + +// Revert to a specified block given its `hash` and `number`. +// This removes all the authority set changes that were announced after +// the revert point. +// Revert point is identified by `number` and `hash`. +func (authSet *AuthoritySet[H, N]) revert() { //nolint //skipcq: SCC-U1000 //skipcq: RVV-B0001 + panic("AuthoritySet.revert not implemented yet") +} + +// Returns the block hash and height at which the next pending hashNumber in +// the given chain (i.e. it includes `best_hash`) was signalled, nil if +// there are no pending changes for the given chain. +func (authSet *AuthoritySet[H, N]) nextChange(bestHash H, //skipcq: RVV-B0001 + isDescendentOf IsDescendentOf[H]) (*hashNumber[H, N], error) { + var forced *hashNumber[H, N] + for _, c := range authSet.pendingForcedChanges { + isDesc, err := isDescendentOf(c.canonHash, bestHash) + if err != nil { + return nil, err + } + if !isDesc { + continue + } + forced = &hashNumber[H, N]{ + hash: c.canonHash, + number: c.canonHeight, + } + break + } + + var standard *hashNumber[H, N] + for _, changeNode := range authSet.pendingStandardChanges.Roots() { + c := changeNode.change + isDesc, err := isDescendentOf(c.canonHash, bestHash) + if err != nil { + return nil, err + } + if !isDesc { + continue + } + standard = &hashNumber[H, N]{ + hash: c.canonHash, + number: c.canonHeight, + } + break + } + + switch { + case standard != nil && forced != nil: + if forced.number < standard.number { + return forced, nil + } + return standard, nil + case forced != nil: + return forced, nil + case standard != nil: + return standard, nil + default: + return nil, nil + } +} + +func (authSet *AuthoritySet[H, N]) addStandardChange(pending PendingChange[H, N], //skipcq: RVV-B0001 + isDescendentOf IsDescendentOf[H]) error { + hash := pending.canonHash + number := pending.canonHeight + + logger.Debugf( + "inserting potential standard set hashNumber signalled at block %d (delayed by %d blocks).", + number, pending.delay, + ) + + _, err := authSet.pendingStandardChanges.Import(hash, number, pending, isDescendentOf) + if err != nil { + return err + } + + logger.Debugf( + "There are now %d alternatives for the next pending standard hashNumber (roots), "+ + "and a total of %d pending standard changes (across all forks)", + len(authSet.pendingStandardChanges.Roots()), len(authSet.pendingStandardChanges.PendingChanges()), + ) + + return nil +} + +func (pc PendingChange[H, N]) GreaterThan(other PendingChange[H, N]) bool { + effectiveNumberGreaterThan := pc.EffectiveNumber() > other.EffectiveNumber() + cannonHeighGreaterThan := pc.EffectiveNumber() == other.EffectiveNumber() && + pc.canonHeight > other.canonHeight + + return effectiveNumberGreaterThan || cannonHeighGreaterThan +} + +func (pc PendingChange[H, N]) LessThan(other PendingChange[H, N]) bool { + effectiveNumberLessThan := pc.EffectiveNumber() < other.EffectiveNumber() + cannonHeighLessThan := pc.EffectiveNumber() == other.EffectiveNumber() && + pc.canonHeight < other.canonHeight + + return effectiveNumberLessThan || cannonHeighLessThan +} + +func (authSet *AuthoritySet[H, N]) addForcedChange(pending PendingChange[H, N], //skipcq: RVV-B0001 + isDescendentOf IsDescendentOf[H]) error { + for _, change := range authSet.pendingForcedChanges { + if change.canonHash == pending.canonHash { + return errDuplicateAuthoritySetChanges + } + + isDescendent, err := isDescendentOf(change.canonHash, pending.canonHash) + if err != nil { + return fmt.Errorf("addForcedChange: checking isDescendentOf: %w", err) + } + + if isDescendent { + return errMultiplePendingForcedAuthoritySetChanges + } + } + + // Changes are inserted in ascending order + idx, _ := slices.BinarySearchFunc( + authSet.pendingForcedChanges, + pending, + func(change, toInsert PendingChange[H, N]) int { + switch { + case toInsert.LessThan(change): + return 1 + case toInsert.GreaterThan(change): + return -1 + default: + return 0 + } + }, + ) + + logger.Debugf( + "inserting potential forced set hashNumber at block number %d (delayed by %d blocks).", + pending.canonHeight, pending.delay, + ) + + // Insert hashNumber at index + if len(authSet.pendingForcedChanges) == idx { + authSet.pendingForcedChanges = append(authSet.pendingForcedChanges, pending) + } else if len(authSet.pendingForcedChanges) > idx { + authSet.pendingForcedChanges = append( + authSet.pendingForcedChanges[:idx+1], authSet.pendingForcedChanges[idx:]...) + authSet.pendingForcedChanges[idx] = pending + } else { + panic("invalid insertion into pending forced changes") + } + + logger.Debugf( + "there are now %d pending forced changes", + len(authSet.pendingForcedChanges), + ) + + return nil +} + +// addPendingChange Note an upcoming pending transition. Multiple pending standard changes +// on the same branch can be added as long as they don't overlap. Forced +// changes are restricted to one per fork. This method assumes that changes +// on the same branch will be added in-order. The given function +// `is_descendent_of` should return `true` if the second hash (target) is a +// descendent of the first hash (base). +func (authSet *AuthoritySet[H, N]) addPendingChange(pending PendingChange[H, N], //skipcq: RVV-B0001 + isDescendentOf IsDescendentOf[H]) error { + if invalidAuthorityList(pending.nextAuthorities) { + return errInvalidAuthoritySet + } + + switch pending.delayKind.value.(type) { + case Finalized: + return authSet.addStandardChange(pending, isDescendentOf) + case Best[N]: + return authSet.addForcedChange(pending, isDescendentOf) + default: + panic("delayKind is invalid type") + } +} + +// pendingChanges Inspect pending changes. Standard pending changes are iterated first, +// and the changes in the roots are traversed in pre-order, afterwards all +// forced changes are iterated. +func (authSet *AuthoritySet[H, N]) pendingChanges() []PendingChange[H, N] { //skipcq: RVV-B0001 + // get everything from standard hashNumber roots + changes := authSet.pendingStandardChanges.PendingChanges() + + // append forced changes + changes = append(changes, authSet.pendingForcedChanges...) + + return changes +} + +// currentLimit Get the earliest limit-block number, if any. If there are pending changes across +// different forks, this method will return the earliest effective number (across the +// different branches) that is higher or equal to the given min number. +// +// Only standard changes are taken into account for the current +// limit, since any existing forced hashNumber should preclude the voter from voting. +func (authSet *AuthoritySet[H, N]) currentLimit(min N) (limit *N) { //skipcq: RVV-B0001 + roots := authSet.pendingStandardChanges.Roots() + for i := 0; i < len(roots); i++ { + effectiveNumber := roots[i].change.EffectiveNumber() + if effectiveNumber >= min { + if limit == nil { + limit = &effectiveNumber + } else if effectiveNumber < *limit { + limit = &effectiveNumber + } + } + } + return limit +} + +// ApplyForcedChanges Apply or prune any pending transitions based on a best-block trigger. +// +// Returns a pointer to the median and new_set when a forced hashNumber has occurred. The +// median represents the median last finalised block at the time the hashNumber +// was signalled, and it should be used as the canon block when starting the +// new grandpa voter. Only alters the internal state in this case. +// +// These transitions are always forced and do not lead to justifications +// which light clients can follow. +// +// Forced changes can only be applied after all pending standard changes +// that it depends on have been applied. If any pending standard hashNumber +// exists that is an ancestor of a given forced changed and which effective +// block number is lower than the last finalised block (as defined by the +// forced hashNumber), then the forced hashNumber cannot be applied. An error will +// be returned in that case which will prevent block import. +func (authSet *AuthoritySet[H, N]) applyForcedChanges(bestHash H, //skipcq: RVV-B0001 + bestNumber N, + isDescendentOf IsDescendentOf[H], + _ Telemetry) (newSet *medianAuthoritySet[H, N], err error) { + + for _, change := range authSet.pendingForcedChanges { + effectiveNumber := change.EffectiveNumber() + if effectiveNumber != bestNumber { + continue + } + // check if the given best block is in the same branch as + // the block that signalled the hashNumber. + isDesc, err := isDescendentOf(change.canonHash, bestHash) + // Avoid case where err is returned because canonHash == bestHash + if change.canonHash != bestHash && err != nil { + return nil, err + } + if change.canonHash == bestHash || isDesc { + switch delayKindType := change.delayKind.value.(type) { + case Best[N]: + medianLastFinalized := delayKindType.medianLastFinalized + roots := authSet.pendingStandardChanges.Roots() + for _, standardChangeNode := range roots { + standardChange := standardChangeNode.change + + isDescStandard, err := isDescendentOf(standardChange.canonHash, change.canonHash) + if err != nil { + return nil, err + } + if standardChange.EffectiveNumber() <= medianLastFinalized && isDescStandard { + logger.Infof( + "Not applying authority set hashNumber forced at block %d, due to "+ + "pending standard hashNumber at block %d", + change.canonHeight, standardChange.EffectiveNumber()) + return nil, errForcedAuthoritySetChangeDependencyUnsatisfied + } + } + + // apply this hashNumber: make the set canonical + logger.Infof("👴 Applying authority set hashNumber forced at block #%d", change.canonHeight) + + // TODO telemetry + + authSetChanges := authSet.authoritySetChanges + authSetChanges.append(authSet.setID, medianLastFinalized) + newSet = &medianAuthoritySet[H, N]{ + medianLastFinalized, + AuthoritySet[H, N]{ + currentAuthorities: change.nextAuthorities, + setID: authSet.setID + 1, + pendingStandardChanges: NewChangeTree[H, N](), // new set, new changes + pendingForcedChanges: []PendingChange[H, N]{}, + authoritySetChanges: authSetChanges, + }, + } + return newSet, nil + default: + panic("pending_forced_changes only contains forced changes; forced changes have delay kind Best") + } + } + } + + return newSet, nil +} + +// applyStandardChanges Apply or prune any pending transitions based on a finality trigger. This +// method ensures that if there are multiple changes in the same branch, +// finalising this block won't finalise past multiple transitions (i.e. +// transitions must be finalised in-order). The given function +// `is_descendent_of` should return `true` if the second hash (target) is a +// descendent of the first hash (base). +// +// When the set has changed, the return value will be a status type where newSetBlock +// is the canonical block where the set last changed (i.e. the given +// hash and number). +func (authSet *AuthoritySet[H, N]) applyStandardChanges( //skipcq: RVV-B0001 + finalisedHash H, + finalisedNumber N, + isDescendentOf IsDescendentOf[H], + _ Telemetry) (status[H, N], error) { + // TODO telemetry here is just a place holder, replace with real + + status := status[H, N]{} + finalisationResult, err := authSet.pendingStandardChanges.FinalizeWithDescendentIf(&finalisedHash, + finalisedNumber, + isDescendentOf, + func(change *PendingChange[H, N]) bool { + return change.EffectiveNumber() <= finalisedNumber + }) + if err != nil { + return status, err + } + + finalisationResultVal, err := finalisationResult.Value() + if err != nil { + return status, err + } + switch val := finalisationResultVal.(type) { + case unchanged: + return status, nil + case changed[H, N]: + // Changed Case + status.changed = true + + // Flush pending forced changes to re add + pendingForcedChanges := authSet.pendingForcedChanges + authSet.pendingForcedChanges = []PendingChange[H, N]{} + + // we will keep all forced changes for any later blocks and that are a + // descendent of the finalised block (i.e. they are part of this branch). + for _, forcedChange := range pendingForcedChanges { + isDesc, err := isDescendentOf(finalisedHash, forcedChange.canonHash) + if err != nil { + return status, err + } + if forcedChange.EffectiveNumber() > finalisedNumber && isDesc { + authSet.pendingForcedChanges = append(authSet.pendingForcedChanges, forcedChange) + } + } + + if val.value != nil { + logger.Infof("👴 Applying authority set hashNumber forced at block #%d", *val.value) + + // TODO add telemetry + + // Store the set_id together with the last block_number for the set + authSet.authoritySetChanges.append(authSet.setID, finalisedNumber) + authSet.currentAuthorities = val.value.nextAuthorities + authSet.setID++ + + status.newSetBlock = &hashNumber[H, N]{ + hash: finalisedHash, + number: finalisedNumber, + } + } + + return status, nil + default: + panic("invalid type for FinalizationResult") + } +} + +// EnactsStandardChange Check whether the given finalised block number enacts any standard +// authority set hashNumber (without triggering it), ensuring that if there are +// multiple changes in the same branch, finalising this block won't +// finalise past multiple transitions (i.e. transitions must be finalised +// in-order). Returns *true if the block being finalised enacts a +// hashNumber that can be immediately applied, *false if the block being +// finalised enacts a hashNumber but it cannot be applied yet since there are +// other dependent changes, and nil if no hashNumber is enacted. The given +// function `is_descendent_of` should return `true` if the second hash +// (target) is a descendent of the first hash (base). +func (authSet *AuthoritySet[H, N]) EnactsStandardChange( //skipcq: RVV-B0001 + finalisedHash H, finalisedNumber N, isDescendentOf IsDescendentOf[H]) (*bool, error) { + applied, err := authSet.pendingStandardChanges.FinalizesAnyWithDescendentIf(&finalisedHash, + finalisedNumber, + isDescendentOf, + func(change *PendingChange[H, N]) bool { + return change.EffectiveNumber() == finalisedNumber + }) + if err != nil { + return nil, fmt.Errorf("%w: %v", errForkTree, err) + } + return applied, nil +} + +// delayKinds Kinds of delays for pending changes. +type delayKinds[N constraints.Unsigned] interface { + Finalized | Best[N] +} + +// delayKind struct to represent delayedKinds +type delayKind struct { + value interface{} +} + +func newDelayKind[N constraints.Unsigned, T delayKinds[N]](val T) delayKind { + return delayKind{ + value: val, + } +} + +// Finalized Depth in finalised chain. +type Finalized struct{} + +// Best Depth in best chain. The median last finalised block is calculated at the time the +// hashNumber was signalled. +type Best[N constraints.Unsigned] struct { + medianLastFinalized N +} + +// PendingChange A pending hashNumber to the authority set. +// +// This will be applied when the announcing block is at some depth within +// the finalised or unfinalised chain. +type PendingChange[H comparable, N constraints.Unsigned] struct { + // The new authorities and weights to apply. + nextAuthorities []Authority + // How deep in the chain the announcing block must be + // before the hashNumber is applied. + delay N + // The announcing block's height. + canonHeight N + // The announcing block's hash. + canonHash H + // The delay kind. + delayKind delayKind +} + +// EffectiveNumber Returns the effective number this hashNumber will be applied at. +func (pc *PendingChange[H, N]) EffectiveNumber() N { + return pc.canonHeight + pc.delay +} + +// AuthoritySetChanges Tracks historical authority set changes. We store the block numbers for the last block +// of each authority set, once they have been finalised. These blocks are guaranteed to +// have a justification unless they were triggered by a forced hashNumber. +type AuthoritySetChanges[N constraints.Unsigned] []setIDNumber[N] + +// append an setIDNumber to AuthoritySetChanges +func (asc *AuthoritySetChanges[N]) append(setID uint64, blockNumber N) { + *asc = append(*asc, setIDNumber[N]{ + setID: setID, + blockNumber: blockNumber, + }) +} + +type authoritySetChangeID scale.VaryingDataType + +// Set will set a VaryingDataTypeValue using the underlying VaryingDataType +func (asc *authoritySetChangeID) Set(val scale.VaryingDataTypeValue) (err error) { + vdt := scale.VaryingDataType(*asc) + err = vdt.Set(val) + if err != nil { + return + } + *asc = authoritySetChangeID(vdt) + return +} + +// Value will return value from underying VaryingDataType +func (asc *authoritySetChangeID) Value() (val scale.VaryingDataTypeValue, err error) { + vdt := scale.VaryingDataType(*asc) + return vdt.Value() +} + +func newAuthoritySetChangeID[N constraints.Unsigned]() authoritySetChangeID { + vdt := scale.MustNewVaryingDataType(latest{}, set[N]{}, unknown{}) + return authoritySetChangeID(vdt) +} + +type latest struct{} + +func (latest) Index() uint { + return 0 +} + +type set[N constraints.Unsigned] struct { + inner setIDNumber[N] +} + +func (set[N]) Index() uint { + return 1 +} + +type unknown struct{} + +func (unknown) Index() uint { + return 2 +} + +// Three states that can be returned: Latest, Set (tuple), Unknown +func (asc *AuthoritySetChanges[N]) getSetID(blockNumber N) (authSetChangeID authoritySetChangeID, err error) { + if asc == nil { + return authSetChangeID, fmt.Errorf("getSetID: authSetChanges is nil") + } + authSetChangeID = newAuthoritySetChangeID[N]() + authSet := *asc + last := authSet[len(authSet)-1] + if last.blockNumber < blockNumber { + err = authSetChangeID.Set(latest{}) + if err != nil { + return authSetChangeID, err + } + return authSetChangeID, nil + } + + idx, _ := slices.BinarySearchFunc( + authSet, + blockNumber, + func(a setIDNumber[N], b N) int { + switch { + case a.blockNumber == b: + return 0 + case a.blockNumber > b: + return 1 + case a.blockNumber < b: + return -1 + default: + panic("invalid return in binary search") + } + }, + ) + if idx < len(authSet) { + authChange := authSet[idx] + + // if this is the first index but not the first set id then we are missing data. + if idx == 0 && authChange.setID != 0 { + err = authSetChangeID.Set(unknown{}) + if err != nil { + return authSetChangeID, err + } + return authSetChangeID, nil + } + err = authSetChangeID.Set(set[N]{ + authChange, + }) + if err != nil { + return authSetChangeID, err + } + return authSetChangeID, nil + } + + err = authSetChangeID.Set(unknown{}) + if err != nil { + return authSetChangeID, err + } + return authSetChangeID, nil +} + +func (asc *AuthoritySetChanges[N]) insert(blockNumber N) { + var idx int + if asc == nil { + panic("authority set changes must be initialised") + } else { + idx, _ = slices.BinarySearchFunc( + *asc, + blockNumber, + func(a setIDNumber[N], b N) int { + switch { + case a.blockNumber == b: + return 0 + case a.blockNumber > b: + return 1 + case a.blockNumber < b: + return -1 + default: + panic("invalid return in binary search") + } + }, + ) + } + + set := *asc + + var setId uint64 + if idx == 0 { + setId = 0 + } else { + setId = set[idx-1].setID + 1 + } + + if idx != len(set) && set[idx].setID == setId { + panic("inserting authority set hashNumber") + } + + change := setIDNumber[N]{ + setID: setId, + blockNumber: blockNumber, + } + + // Insert hashNumber at index + if len(set) <= idx { + set = append(set, change) + } else { + set = append(set[:idx+1], set[idx:]...) + set[idx] = change + } + *asc = set +} + +// IterFrom This logic is used in warp sync proof +func (asc *AuthoritySetChanges[N]) IterFrom(blockNumber N) *AuthoritySetChanges[N] { + if asc == nil { + return nil + } + authSet := *asc + + idx, found := slices.BinarySearchFunc( + *asc, + blockNumber, + func(a setIDNumber[N], b N) int { + switch { + case a.blockNumber == b: + return 0 + case a.blockNumber > b: + return 1 + case a.blockNumber < b: + return -1 + default: + panic("invalid return in binary search") + } + }, + ) + if found { + // if there was a hashNumber at the given block number then we should start on the next + // index since we want to exclude the current block number + idx += 1 + } + + if idx < len(*asc) { + authChange := authSet[idx] + + // if this is the first index but not the first set id then we are missing data. + if idx == 0 && authChange.setID != 0 { + return nil + } + } + + iterChanges := authSet[idx:] + return &iterChanges +} diff --git a/client/consensus/grandpa/authorities_test.go b/client/consensus/grandpa/authorities_test.go new file mode 100644 index 00000000000..ec06c581dcc --- /dev/null +++ b/client/consensus/grandpa/authorities_test.go @@ -0,0 +1,1428 @@ +// Copyright 2023 ChainSafe Systems (ON) +// SPDX-License-Identifier: LGPL-3.0-only +package grandpa + +import ( + "fmt" + "strings" + "testing" + + "github.com/ChainSafe/gossamer/lib/crypto/ed25519" + "github.com/stretchr/testify/require" +) + +const ( + hashA = "hash_a" + hashB = "hash_b" + hashC = "hash_c" + hashD = "hash_d" +) + +func staticIsDescendentOf[H comparable](value bool) IsDescendentOf[H] { + return func(H, H) (bool, error) { return value, nil } +} + +func isDescendentof[H comparable](f IsDescendentOf[H]) IsDescendentOf[H] { + return func(h1, h2 H) (bool, error) { return f(h1, h2) } +} + +func TestDelayKind(t *testing.T) { + finalisedKind := Finalized{} + delayKind := newDelayKind[uint](finalisedKind) + _, isFinalizedType := delayKind.value.(Finalized) + require.True(t, isFinalizedType) + + medLastFinalized := uint(3) + bestKind := Best[uint]{medianLastFinalized: medLastFinalized} + delayKind = newDelayKind[uint](bestKind) + best, isBestType := delayKind.value.(Best[uint]) + require.True(t, isBestType) + require.Equal(t, medLastFinalized, best.medianLastFinalized) +} + +func TestCurrentLimitFiltersMin(t *testing.T) { + var currentAuthorities []Authority + kp, err := ed25519.GenerateKeypair() + require.NoError(t, err) + currentAuthorities = append(currentAuthorities, Authority{ + Key: kp.Public(), + Weight: 1, + }) + + finalisedKind := Finalized{} + delayKind := newDelayKind[uint](finalisedKind) + + pendingChange1 := PendingChange[string, uint]{ + nextAuthorities: currentAuthorities, + delay: 0, + canonHeight: 1, + canonHash: "a", + delayKind: delayKind, + } + + pendingChange2 := PendingChange[string, uint]{ + nextAuthorities: currentAuthorities, + delay: 0, + canonHeight: 2, + canonHash: "b", + delayKind: delayKind, + } + + authorities := AuthoritySet[string, uint]{ + currentAuthorities: currentAuthorities, + setID: 0, + pendingStandardChanges: NewChangeTree[string, uint](), + pendingForcedChanges: []PendingChange[string, uint]{}, + authoritySetChanges: AuthoritySetChanges[uint]{}, + } + + err = authorities.addPendingChange(pendingChange1, staticIsDescendentOf[string](false)) + require.NoError(t, err) + + err = authorities.addPendingChange(pendingChange2, staticIsDescendentOf[string](false)) + require.NoError(t, err) + + require.Equal(t, uint(1), *authorities.currentLimit(0)) + require.Equal(t, uint(1), *authorities.currentLimit(1)) + require.Equal(t, uint(2), *authorities.currentLimit(2)) + require.Nil(t, authorities.currentLimit(3)) +} + +func TestChangesIteratedInPreOrder(t *testing.T) { + var currentAuthorities []Authority + kp, err := ed25519.GenerateKeypair() + require.NoError(t, err) + currentAuthorities = append(currentAuthorities, Authority{ + Key: kp.Public(), + Weight: 1, + }) + + finalisedKind := Finalized{} + delayKindFinalized := newDelayKind[uint](finalisedKind) + + bestKind := Best[uint]{} + delayKindBest := newDelayKind[uint](bestKind) + + authorities := AuthoritySet[string, uint]{ + currentAuthorities: currentAuthorities, + setID: 0, + pendingStandardChanges: NewChangeTree[string, uint](), + pendingForcedChanges: []PendingChange[string, uint]{}, + authoritySetChanges: AuthoritySetChanges[uint]{}, + } + + changeA := PendingChange[string, uint]{ + nextAuthorities: currentAuthorities, + delay: 10, + canonHeight: 5, + canonHash: hashA, + delayKind: delayKindFinalized, + } + + changeB := PendingChange[string, uint]{ + nextAuthorities: currentAuthorities, + delay: 0, + canonHeight: 5, + canonHash: hashB, + delayKind: delayKindFinalized, + } + + changeC := PendingChange[string, uint]{ + nextAuthorities: currentAuthorities, + delay: 5, + canonHeight: 10, + canonHash: hashC, + delayKind: delayKindFinalized, + } + + err = authorities.addPendingChange(changeA, staticIsDescendentOf[string](false)) + require.NoError(t, err) + + err = authorities.addPendingChange(changeB, staticIsDescendentOf[string](false)) + require.NoError(t, err) + + err = authorities.addPendingChange(changeC, isDescendentof(func(h1, h2 string) (bool, error) { + if h1 == hashA && h2 == hashC { + return true, nil + } else if h1 == hashB && h2 == hashC { + return false, nil + } else { + panic("unreachable") + } + })) + require.NoError(t, err) + + changeD := PendingChange[string, uint]{ + nextAuthorities: currentAuthorities, + delay: 2, + canonHeight: 1, + canonHash: hashD, + delayKind: delayKindBest, + } + + changeE := PendingChange[string, uint]{ + nextAuthorities: currentAuthorities, + delay: 2, + canonHeight: 0, + canonHash: "hash_e", + delayKind: delayKindBest, + } + + err = authorities.addPendingChange(changeD, staticIsDescendentOf[string](false)) + require.NoError(t, err) + + err = authorities.addPendingChange(changeE, staticIsDescendentOf[string](false)) + require.NoError(t, err) + + expectedChanges := []PendingChange[string, uint]{ + changeA, changeC, changeB, changeE, changeD, + } + pendingChanges := authorities.pendingChanges() + require.Equal(t, expectedChanges, pendingChanges) +} + +func TestApplyChange(t *testing.T) { + authorities := AuthoritySet[string, uint]{ + currentAuthorities: []Authority{}, + setID: 0, + pendingStandardChanges: NewChangeTree[string, uint](), + pendingForcedChanges: []PendingChange[string, uint]{}, + authoritySetChanges: AuthoritySetChanges[uint]{}, + } + + var setA []Authority + kpA, err := ed25519.GenerateKeypair() + require.NoError(t, err) + setA = append(setA, Authority{ + Key: kpA.Public(), + Weight: 5, + }) + + var setB []Authority + kpB, err := ed25519.GenerateKeypair() + require.NoError(t, err) + setB = append(setB, Authority{ + Key: kpB.Public(), + Weight: 5, + }) + + finalisedKind := Finalized{} + delayKindFinalized := newDelayKind[uint](finalisedKind) + + changeA := PendingChange[string, uint]{ + nextAuthorities: setA, + delay: 10, + canonHeight: 5, + canonHash: hashA, + delayKind: delayKindFinalized, + } + + changeB := PendingChange[string, uint]{ + nextAuthorities: setB, + delay: 10, + canonHeight: 5, + canonHash: hashB, + delayKind: delayKindFinalized, + } + + err = authorities.addPendingChange(changeA, staticIsDescendentOf[string](true)) + require.NoError(t, err) + + err = authorities.addPendingChange(changeB, staticIsDescendentOf[string](true)) + require.NoError(t, err) + + expectedChanges := []PendingChange[string, uint]{ + changeA, changeB, + } + pendingChanges := authorities.pendingChanges() + require.Equal(t, expectedChanges, pendingChanges) + + // finalising hashC won't enact the hashNumber signalled at hashA but it will prune out + // hashB + status, err := authorities.applyStandardChanges( + hashC, + 11, + isDescendentof(func(h1 string, h2 string) (bool, error) { + if h1 == hashA && h2 == hashC { + return true, nil + } else if h1 == hashB && h2 == hashC { + return false, nil + } else { + panic("unreachable") + } + }), + nil, + ) + + require.NoError(t, err) + require.True(t, status.changed) + require.Nil(t, status.newSetBlock) + + expectedChanges = []PendingChange[string, uint]{ + changeA, + } + pendingChanges = authorities.pendingChanges() + require.Equal(t, expectedChanges, pendingChanges) + require.True(t, len(authorities.authoritySetChanges) == 0) + + status, err = authorities.applyStandardChanges( + hashD, + 15, + isDescendentof(func(h1 string, h2 string) (bool, error) { + if h1 == hashA && h2 == hashD { + return true, nil + } else { + panic("unreachable") + } + }), + nil, + ) + require.NoError(t, err) + + expectedBlockInfo := &hashNumber[string, uint]{ + hash: hashD, + number: 15, + } + + require.True(t, status.changed) + require.Equal(t, status.newSetBlock, expectedBlockInfo) + require.Equal(t, authorities.currentAuthorities, setA) + require.Equal(t, authorities.setID, uint64(1)) + + pendingChanges = authorities.pendingChanges() + require.Equal(t, 0, len(pendingChanges)) + expChange := setIDNumber[uint]{ + setID: 0, + blockNumber: 15, + } + require.Equal(t, authorities.authoritySetChanges, AuthoritySetChanges[uint]{expChange}) +} + +func TestDisallowMultipleChangesBeingFinalizedAtOnce(t *testing.T) { + authorities := AuthoritySet[string, uint]{ + currentAuthorities: []Authority{}, + setID: 0, + pendingStandardChanges: NewChangeTree[string, uint](), + pendingForcedChanges: []PendingChange[string, uint]{}, + authoritySetChanges: AuthoritySetChanges[uint]{}, + } + + var setA []Authority + kpA, err := ed25519.GenerateKeypair() + require.NoError(t, err) + setA = append(setA, Authority{ + Key: kpA.Public(), + Weight: 5, + }) + + var setC []Authority + kpC, err := ed25519.GenerateKeypair() + require.NoError(t, err) + setC = append(setC, Authority{ + Key: kpC.Public(), + Weight: 5, + }) + + finalisedKind := Finalized{} + delayKindFinalized := newDelayKind[uint](finalisedKind) + + changeA := PendingChange[string, uint]{ + nextAuthorities: setA, + delay: 10, + canonHeight: 5, + canonHash: hashA, + delayKind: delayKindFinalized, + } + + changeC := PendingChange[string, uint]{ + nextAuthorities: setC, + delay: 10, + canonHeight: 30, + canonHash: hashC, + delayKind: delayKindFinalized, + } + + err = authorities.addPendingChange(changeA, staticIsDescendentOf[string](true)) + require.NoError(t, err) + + err = authorities.addPendingChange(changeC, staticIsDescendentOf[string](true)) + require.NoError(t, err) + + isDescOf := isDescendentof(func(h1 string, h2 string) (bool, error) { + if h1 == hashA && h2 == hashB || + h1 == hashA && h2 == hashC || + h1 == hashA && h2 == hashD || + h1 == hashC && h2 == hashD || + h1 == hashB && h2 == hashC { + return true, nil + } else if h1 == hashC && h2 == hashB { + return false, nil + } else { + panic("unreachable") + } + }) + + // trying to finalise past `change_c` without finalising `change_a` first + _, err = authorities.applyStandardChanges( + hashD, + 40, + isDescOf, + nil, + ) + + require.ErrorIs(t, err, errUnfinalisedAncestor) + require.Equal(t, AuthoritySetChanges[uint]{}, authorities.authoritySetChanges) + + status, err := authorities.applyStandardChanges( + hashB, + 15, + isDescOf, + nil, + ) + require.NoError(t, err) + require.True(t, status.changed) + + expectedBlockInfo := &hashNumber[string, uint]{ + hash: hashB, + number: 15, + } + expAuthSetChange := AuthoritySetChanges[uint]{setIDNumber[uint]{ + setID: 0, + blockNumber: 15, + }} + require.Equal(t, expectedBlockInfo, status.newSetBlock) + require.Equal(t, setA, authorities.currentAuthorities) + require.Equal(t, uint64(1), authorities.setID) + require.Equal(t, expAuthSetChange, authorities.authoritySetChanges) + + status, err = authorities.applyStandardChanges( + hashD, + 40, + isDescOf, + nil, + ) + require.NoError(t, err) + require.True(t, status.changed) + + expectedBlockInfo = &hashNumber[string, uint]{ + hash: hashD, + number: 40, + } + expAuthSetChange = AuthoritySetChanges[uint]{ + setIDNumber[uint]{ + setID: 0, + blockNumber: 15, + }, + setIDNumber[uint]{ + setID: 1, + blockNumber: 40, + }, + } + + require.Equal(t, expectedBlockInfo, status.newSetBlock) + require.Equal(t, setC, authorities.currentAuthorities) + require.Equal(t, uint64(2), authorities.setID) + require.Equal(t, expAuthSetChange, authorities.authoritySetChanges) +} + +func TestEnactsStandardChangeWorks(t *testing.T) { + authorities := AuthoritySet[string, uint]{ + currentAuthorities: []Authority{}, + setID: 0, + pendingStandardChanges: NewChangeTree[string, uint](), + pendingForcedChanges: []PendingChange[string, uint]{}, + authoritySetChanges: AuthoritySetChanges[uint]{}, + } + + var setA []Authority + kpA, err := ed25519.GenerateKeypair() + require.NoError(t, err) + setA = append(setA, Authority{ + Key: kpA.Public(), + Weight: 5, + }) + + finalisedKind := Finalized{} + delayKindFinalized := newDelayKind[uint](finalisedKind) + + changeA := PendingChange[string, uint]{ + nextAuthorities: setA, + delay: 10, + canonHeight: 5, + canonHash: hashA, + delayKind: delayKindFinalized, + } + + changeB := PendingChange[string, uint]{ + nextAuthorities: setA, + delay: 10, + canonHeight: 20, + canonHash: hashB, + delayKind: delayKindFinalized, + } + + err = authorities.addPendingChange(changeA, staticIsDescendentOf[string](false)) + require.NoError(t, err) + + err = authorities.addPendingChange(changeB, staticIsDescendentOf[string](true)) + require.NoError(t, err) + + isDescOf := isDescendentof(func(h1 string, h2 string) (bool, error) { + if h1 == hashA && h2 == hashD || + h1 == hashA && h2 == "hash_e" || + h1 == hashB && h2 == hashD || + h1 == hashB && h2 == "hash_e" { + return true, nil + } else if h1 == hashA && h2 == hashC || + h1 == hashB && h2 == hashC { + return false, nil + } else { + panic("unreachable") + } + }) + + // hashC won't finalise the existing hashNumber since it isn't a descendent + res, err := authorities.EnactsStandardChange(hashC, 15, isDescOf) + require.NoError(t, err) + require.Nil(t, res) + + // hashD at depth 14 won't work either + res, err = authorities.EnactsStandardChange(hashD, 14, isDescOf) + require.NoError(t, err) + require.Nil(t, res) + + // but it should work at depth 15 (hashNumber height + depth) + res, err = authorities.EnactsStandardChange(hashD, 15, isDescOf) + require.NoError(t, err) + require.Equal(t, true, *res) + + // finalising "hash_e" at depth 20 will trigger hashNumber at hashB, but + // it can't be applied yet since hashA must be applied first + res, err = authorities.EnactsStandardChange("hash_e", 30, isDescOf) + require.NoError(t, err) + require.Equal(t, false, *res) +} + +func TestForceChanges(t *testing.T) { + authorities := AuthoritySet[string, uint]{ + currentAuthorities: []Authority{}, + setID: 0, + pendingStandardChanges: NewChangeTree[string, uint](), + pendingForcedChanges: []PendingChange[string, uint]{}, + authoritySetChanges: AuthoritySetChanges[uint]{}, + } + + var setA []Authority + kpA, err := ed25519.GenerateKeypair() + require.NoError(t, err) + setA = append(setA, Authority{ + Key: kpA.Public(), + Weight: 5, + }) + + var setB []Authority + kpB, err := ed25519.GenerateKeypair() + require.NoError(t, err) + setB = append(setB, Authority{ + Key: kpB.Public(), + Weight: 5, + }) + + finalisedKindA := Best[uint]{42} + delayKindFinalizedA := newDelayKind[uint](finalisedKindA) + + finalisedKindB := Best[uint]{0} + delayKindFinalizedB := newDelayKind[uint](finalisedKindB) + + changeA := PendingChange[string, uint]{ + nextAuthorities: setA, + delay: 10, + canonHeight: 5, + canonHash: hashA, + delayKind: delayKindFinalizedA, + } + + changeB := PendingChange[string, uint]{ + nextAuthorities: setB, + delay: 10, + canonHeight: 5, + canonHash: hashB, + delayKind: delayKindFinalizedB, + } + + err = authorities.addPendingChange(changeA, staticIsDescendentOf[string](false)) + require.NoError(t, err) + + err = authorities.addPendingChange(changeB, staticIsDescendentOf[string](false)) + require.NoError(t, err) + + // no duplicates are allowed + err = authorities.addPendingChange(changeB, staticIsDescendentOf[string](false)) + require.ErrorIs(t, err, errDuplicateAuthoritySetChanges) + + res, err := authorities.EnactsStandardChange(hashC, 1, staticIsDescendentOf[string](true)) + require.NoError(t, err) + require.Nil(t, res) + + changeC := PendingChange[string, uint]{ + nextAuthorities: setA, + delay: 3, + canonHeight: 8, + canonHash: "hash_a8", + delayKind: delayKindFinalizedB, + } + + isDescOfA := isDescendentof(func(h1 string, _ string) (bool, error) { + return strings.HasPrefix(h1, hashA), nil + }) + + err = authorities.addPendingChange(changeC, isDescOfA) + require.ErrorIs(t, err, errMultiplePendingForcedAuthoritySetChanges) + + // let's try and apply the forced changes. + // too early and there's no forced changes to apply + resForced, err := authorities.applyForcedChanges("hash_a10", 10, staticIsDescendentOf[string](true), nil) + require.NoError(t, err) + require.Nil(t, resForced) + + // too late + resForced, err = authorities.applyForcedChanges("hash_a16", 16, isDescOfA, nil) + require.NoError(t, err) + require.Nil(t, resForced) + + // on time -- chooses the right hashNumber for this fork + exp := medianAuthoritySet[string, uint]{ + median: 42, + set: AuthoritySet[string, uint]{ + currentAuthorities: setA, + setID: 1, + pendingStandardChanges: NewChangeTree[string, uint](), + pendingForcedChanges: []PendingChange[string, uint]{}, + authoritySetChanges: AuthoritySetChanges[uint]{ + setIDNumber[uint]{ + setID: 0, + blockNumber: 42, + }, + }, + }, + } + resForced, err = authorities.applyForcedChanges("hash_a15", 15, isDescOfA, nil) + require.NoError(t, err) + require.NotNil(t, resForced) + require.Equal(t, exp, *resForced) +} + +func TestForceChangesWithNoDelay(t *testing.T) { + // NOTE: this is a regression test + authorities := AuthoritySet[string, uint]{ + currentAuthorities: []Authority{}, + setID: 0, + pendingStandardChanges: NewChangeTree[string, uint](), + pendingForcedChanges: []PendingChange[string, uint]{}, + authoritySetChanges: AuthoritySetChanges[uint]{}, + } + + var setA []Authority + kpA, err := ed25519.GenerateKeypair() + require.NoError(t, err) + setA = append(setA, Authority{ + Key: kpA.Public(), + Weight: 5, + }) + + finalisedKind := Best[uint]{0} + delayKindFinalized := newDelayKind[uint](finalisedKind) + + // we create a forced hashNumber with no delay + changeA := PendingChange[string, uint]{ + nextAuthorities: setA, + delay: 0, + canonHeight: 5, + canonHash: hashA, + delayKind: delayKindFinalized, + } + + // and import it + err = authorities.addPendingChange(changeA, staticIsDescendentOf[string](false)) + require.NoError(t, err) + + // it should be enacted at the same block that signalled it + resForced, err := authorities.applyForcedChanges(hashA, 5, staticIsDescendentOf[string](false), nil) + require.NoError(t, err) + require.NotNil(t, resForced) +} + +func TestForceChangesBlockedByStandardChanges(t *testing.T) { + authorities := AuthoritySet[string, uint]{ + currentAuthorities: []Authority{}, + setID: 0, + pendingStandardChanges: NewChangeTree[string, uint](), + pendingForcedChanges: []PendingChange[string, uint]{}, + authoritySetChanges: AuthoritySetChanges[uint]{}, + } + + var setA []Authority + kpA, err := ed25519.GenerateKeypair() + require.NoError(t, err) + setA = append(setA, Authority{ + Key: kpA.Public(), + Weight: 5, + }) + + finalisedKind := Finalized{} + delayKindFinalized := newDelayKind[uint](finalisedKind) + + // effective at #15 + changeA := PendingChange[string, uint]{ + nextAuthorities: setA, + delay: 5, + canonHeight: 10, + canonHash: hashA, + delayKind: delayKindFinalized, + } + + // effective #20 + changeB := PendingChange[string, uint]{ + nextAuthorities: setA, + delay: 0, + canonHeight: 20, + canonHash: hashB, + delayKind: delayKindFinalized, + } + + // effective at #35 + changeC := PendingChange[string, uint]{ + nextAuthorities: setA, + delay: 5, + canonHeight: 30, + canonHash: hashC, + delayKind: delayKindFinalized, + } + + // add some pending standard changes all on the same fork + err = authorities.addPendingChange(changeA, staticIsDescendentOf[string](true)) + require.NoError(t, err) + + err = authorities.addPendingChange(changeB, staticIsDescendentOf[string](true)) + require.NoError(t, err) + + err = authorities.addPendingChange(changeC, staticIsDescendentOf[string](true)) + require.NoError(t, err) + + finalisedKind2 := Best[uint]{31} + delayKindFinalized2 := newDelayKind[uint](finalisedKind2) + + // effective at #45 + changeD := PendingChange[string, uint]{ + nextAuthorities: setA, + delay: 5, + canonHeight: 40, + canonHash: hashD, + delayKind: delayKindFinalized2, + } + + err = authorities.addPendingChange(changeD, staticIsDescendentOf[string](true)) + require.NoError(t, err) + + // the forced hashNumber cannot be applied since the pending changes it depends on + // have not been applied yet. + _, err = authorities.applyForcedChanges("hash_d45", 45, staticIsDescendentOf[string](true), nil) + require.ErrorIs(t, err, errForcedAuthoritySetChangeDependencyUnsatisfied) + require.Equal(t, 0, len(authorities.authoritySetChanges)) + + // we apply the first pending standard hashNumber at #15 + expChanges := AuthoritySetChanges[uint]{ + setIDNumber[uint]{ + setID: 0, + blockNumber: 15, + }, + } + _, err = authorities.applyStandardChanges("hash_a15", 15, staticIsDescendentOf[string](true), nil) + require.NoError(t, err) + require.Equal(t, expChanges, authorities.authoritySetChanges) + + // but the forced hashNumber still depends on the next standard hashNumber + _, err = authorities.applyForcedChanges("hash_d45", 45, staticIsDescendentOf[string](true), nil) + require.ErrorIs(t, err, errForcedAuthoritySetChangeDependencyUnsatisfied) + require.Equal(t, expChanges, authorities.authoritySetChanges) + + // we apply the pending standard hashNumber at #20 + expChanges = append(expChanges, setIDNumber[uint]{ + setID: 1, + blockNumber: 20, + }) + _, err = authorities.applyStandardChanges(hashB, 20, staticIsDescendentOf[string](true), nil) + require.NoError(t, err) + require.Equal(t, expChanges, authorities.authoritySetChanges) + + // afterwards the forced hashNumber at #45 can already be applied since it signals + // that finality stalled at #31, and the next pending standard hashNumber is effective + // at #35. subsequent forced changes on the same branch must be kept + expChanges = append(expChanges, setIDNumber[uint]{ + setID: 2, + blockNumber: 31, + }) + exp := medianAuthoritySet[string, uint]{ + median: 31, + set: AuthoritySet[string, uint]{ + currentAuthorities: setA, + setID: 3, + pendingStandardChanges: NewChangeTree[string, uint](), + pendingForcedChanges: []PendingChange[string, uint]{}, + authoritySetChanges: expChanges, + }, + } + resForced, err := authorities.applyForcedChanges(hashD, 45, staticIsDescendentOf[string](true), nil) + require.NoError(t, err) + require.NotNil(t, resForced) + require.Equal(t, exp, *resForced) +} + +func TestNextChangeWorks(t *testing.T) { + var currentAuthorities []Authority + kpA, err := ed25519.GenerateKeypair() + require.NoError(t, err) + currentAuthorities = append(currentAuthorities, Authority{ + Key: kpA.Public(), + Weight: 1, + }) + + authorities := AuthoritySet[string, uint]{ + currentAuthorities: currentAuthorities, + setID: 0, + pendingStandardChanges: NewChangeTree[string, uint](), + pendingForcedChanges: []PendingChange[string, uint]{}, + authoritySetChanges: AuthoritySetChanges[uint]{}, + } + + finalisedKind := Finalized{} + delayKindFinalized := newDelayKind[uint](finalisedKind) + + // We have three pending changes with 2 possible roots that are enacted + // immediately on finality (i.e. standard changes). + changeA0 := PendingChange[string, uint]{ + nextAuthorities: currentAuthorities, + delay: 0, + canonHeight: 5, + canonHash: "hash_a0", + delayKind: delayKindFinalized, + } + + changeA1 := PendingChange[string, uint]{ + nextAuthorities: currentAuthorities, + delay: 0, + canonHeight: 10, + canonHash: "hash_a1", + delayKind: delayKindFinalized, + } + + changeB := PendingChange[string, uint]{ + nextAuthorities: currentAuthorities, + delay: 0, + canonHeight: 4, + canonHash: hashB, + delayKind: delayKindFinalized, + } + + // A0 (#5) <- A10 (#8) <- A1 (#10) <- best_a + // B (#4) <- best_b + isDescOf := isDescendentof(func(h1 string, h2 string) (bool, error) { + if h1 == "hash_a0" && h2 == "hash_a1" || + h1 == "hash_a0" && h2 == hashB || + h1 == "hash_a1" && h2 == "best_a" || + h1 == "hash_a10" && h2 == "best_a" || + h1 == hashB && h2 == "best_b" { + return true, nil + } else { + return false, nil + } + }) + + // add the three pending changes + err = authorities.addPendingChange(changeB, isDescOf) + require.NoError(t, err) + + err = authorities.addPendingChange(changeA0, isDescOf) + require.NoError(t, err) + + err = authorities.addPendingChange(changeA1, isDescOf) + require.NoError(t, err) + + // the earliest hashNumber at block `best_a` should be the hashNumber at A0 (#5) + expChange := &hashNumber[string, uint]{ + hash: "hash_a0", + number: 5, + } + c, err := authorities.nextChange(hashB, isDescOf) + require.NoError(t, err) + require.Equal(t, expChange, c) + + // the earliest hashNumber at block `best_b` should be the hashNumber at B (#4) + expChange = &hashNumber[string, uint]{ + hash: hashB, + number: 4, + } + c, err = authorities.nextChange("best_b", isDescOf) + require.NoError(t, err) + require.Equal(t, expChange, c) + + // we apply the hashNumber at A0 which should prune it and the fork at B + _, err = authorities.applyStandardChanges("hash_a0", 5, isDescOf, nil) + require.NoError(t, err) + + // the next hashNumber is now at A1 (#10) + expChange = &hashNumber[string, uint]{ + hash: "hash_a1", + number: 10, + } + c, err = authorities.nextChange("best_a", isDescOf) + require.NoError(t, err) + require.Equal(t, expChange, c) + + // there's no longer any pending hashNumber at `best_b` fork + c, err = authorities.nextChange("best_b", isDescOf) + require.NoError(t, err) + require.Nil(t, c) + + // we a forced hashNumber at A10 (#8) + finalisedKind2 := Best[uint]{0} + delayKindFinalized2 := newDelayKind[uint](finalisedKind2) + changeA10 := PendingChange[string, uint]{ + nextAuthorities: currentAuthorities, + delay: 0, + canonHeight: 8, + canonHash: "hash_a10", + delayKind: delayKindFinalized2, + } + + err = authorities.addPendingChange(changeA10, staticIsDescendentOf[string](false)) + require.NoError(t, err) + + // it should take precedence over the hashNumber at A1 (#10) + expChange = &hashNumber[string, uint]{ + hash: "hash_a10", + number: 8, + } + c, err = authorities.nextChange("best_a", isDescOf) + require.NoError(t, err) + require.Equal(t, expChange, c) +} + +func TestMaintainsAuthorityListInvariants(t *testing.T) { + // empty authority lists are invalid + _, err := NewGenesisAuthoritySet[string, uint]([]Authority{}) + require.NotNil(t, err) + _, err = NewAuthoritySet[string, uint]([]Authority{}, 0, NewChangeTree[string, uint](), nil, nil) + require.NotNil(t, err) + + kpA, err := ed25519.GenerateKeypair() + require.NoError(t, err) + + kpB, err := ed25519.GenerateKeypair() + require.NoError(t, err) + + invalidAuthoritiesWeight := []Authority{ + { + Key: kpA.Public(), + Weight: 5, + }, + { + Key: kpB.Public(), + Weight: 0, + }, + } + + // authority weight of zero is invalid + _, err = NewGenesisAuthoritySet[string, uint](invalidAuthoritiesWeight) + require.NotNil(t, err) + _, err = NewAuthoritySet[string, uint](invalidAuthoritiesWeight, 0, NewChangeTree[string, uint](), nil, nil) + require.NotNil(t, err) + + authoritySet, err := NewGenesisAuthoritySet[string, uint]([]Authority{{ + Key: kpA.Public(), + Weight: 5, + }}) + require.NoError(t, err) + + finalisedKind := Finalized{} + delayKindFinalized := newDelayKind[uint](finalisedKind) + invalidChangeEmptyAuthorities := PendingChange[string, uint]{ + nextAuthorities: nil, + delay: 10, + canonHeight: 5, + canonHash: "", + delayKind: delayKindFinalized, + } + + // pending hashNumber contains an empty authority set + err = authoritySet.addPendingChange(invalidChangeEmptyAuthorities, staticIsDescendentOf[string](false)) + require.ErrorIs(t, err, errInvalidAuthoritySet) + + delayKind := Best[uint]{0} + delayKindBest := newDelayKind[uint](delayKind) + + invalidChangeAuthoritiesWeight := PendingChange[string, uint]{ + nextAuthorities: invalidAuthoritiesWeight, + delay: 10, + canonHeight: 5, + canonHash: "", + delayKind: delayKindBest, + } + + // pending hashNumber contains an authority set + // where one authority has weight of 0 + err = authoritySet.addPendingChange(invalidChangeAuthoritiesWeight, staticIsDescendentOf[string](false)) + require.ErrorIs(t, err, errInvalidAuthoritySet) +} + +func TestCleanUpStaleForcedChangesWhenApplyingStandardChange(t *testing.T) { + var currentAuthorities []Authority + kpA, err := ed25519.GenerateKeypair() + require.NoError(t, err) + currentAuthorities = append(currentAuthorities, Authority{ + Key: kpA.Public(), + Weight: 1, + }) + + authorities := AuthoritySet[string, uint]{ + currentAuthorities: currentAuthorities, + setID: 0, + pendingStandardChanges: NewChangeTree[string, uint](), + pendingForcedChanges: []PendingChange[string, uint]{}, + authoritySetChanges: AuthoritySetChanges[uint]{}, + } + + // Create the following pending changes tree: + // + // [#C3] + // / + // /- (#C2) + // / + // (#A) - (#B) - [#C1] + // \ + // (#C0) - [#D] + // + // () - Standard hashNumber + // [] - Forced hashNumber + isDescOf := isDescendentof(func(h1 string, h2 string) (bool, error) { + hashes := []string{ + "B", + "C0", + "C1", + "C2", + "C3", + "D", + } + + if h1 == "B" && h2 == "B" { + return false, nil + } else if h1 == "A" || h1 == "B" { + for _, val := range hashes { + if val == h2 { + return true, nil + } + } + return false, nil + } else if h1 == "C0" && h2 == "D" { + return true, nil + } + return false, nil + }) + + addPendingChangeFunction := func(canonHeight uint, canonHash string, forced bool) { + var change PendingChange[string, uint] + if forced { + delayKind := Best[uint]{0} + delayKindBest := newDelayKind[uint](delayKind) + change = PendingChange[string, uint]{ + nextAuthorities: currentAuthorities, + delay: 0, + canonHeight: canonHeight, + canonHash: canonHash, + delayKind: delayKindBest, + } + } else { + delayKind := Finalized{} + delayKindFinalized := newDelayKind[uint](delayKind) + change = PendingChange[string, uint]{ + nextAuthorities: currentAuthorities, + delay: 0, + canonHeight: canonHeight, + canonHash: canonHash, + delayKind: delayKindFinalized, + } + } + + err := authorities.addPendingChange(change, isDescOf) + require.NoError(t, err) + } + + addPendingChangeFunction(5, "A", false) + addPendingChangeFunction(10, "B", false) + addPendingChangeFunction(15, "C0", false) + addPendingChangeFunction(15, "C1", true) + addPendingChangeFunction(15, "C2", false) + addPendingChangeFunction(15, "C3", true) + addPendingChangeFunction(20, "D", true) + + // applying the standard hashNumber at A should not prune anything + // other then the hashNumber that was applied + _, err = authorities.applyStandardChanges("A", 5, isDescOf, nil) + require.NoError(t, err) + require.Equal(t, 6, len(authorities.pendingChanges())) + + // same for B + _, err = authorities.applyStandardChanges("B", 10, isDescOf, nil) + require.NoError(t, err) + require.Equal(t, 5, len(authorities.pendingChanges())) + + // finalising C2 should clear all forced changes + _, err = authorities.applyStandardChanges("C2", 15, isDescOf, nil) + require.NoError(t, err) + require.Equal(t, 0, len(authorities.pendingForcedChanges)) +} + +func TestCleanUpStaleForcedChangesWhenApplyingStandardChangeAlternateCase(t *testing.T) { + var currentAuthorities []Authority + kpA, err := ed25519.GenerateKeypair() + require.NoError(t, err) + currentAuthorities = append(currentAuthorities, Authority{ + Key: kpA.Public(), + Weight: 1, + }) + + authorities := AuthoritySet[string, uint]{ + currentAuthorities: currentAuthorities, + setID: 0, + pendingStandardChanges: NewChangeTree[string, uint](), + pendingForcedChanges: []PendingChange[string, uint]{}, + authoritySetChanges: AuthoritySetChanges[uint]{}, + } + + // Create the following pending changes tree: + // + // [#C3] + // / + // /- (#C2) + // / + // (#A) - (#B) - [#C1] + // \ + // (#C0) - [#D] + // + // () - Standard hashNumber + // [] - Forced hashNumber + isDescOf := isDescendentof(func(h1 string, h2 string) (bool, error) { + hashes := []string{ + "B", + "C0", + "C1", + "C2", + "C3", + "D", + } + + if h1 == "B" && h2 == "B" { + return false, nil + } else if h1 == "A" || h1 == "B" { + for _, val := range hashes { + if val == h2 { + return true, nil + } + } + return false, nil + } else if h1 == "C0" && h2 == "D" { + return true, nil + } + return false, nil + }) + + addPendingChangeFunction := func(canonHeight uint, canonHash string, forced bool) { + var change PendingChange[string, uint] + if forced { + delayKind := Best[uint]{0} + delayKindBest := newDelayKind[uint](delayKind) + change = PendingChange[string, uint]{ + nextAuthorities: currentAuthorities, + delay: 0, + canonHeight: canonHeight, + canonHash: canonHash, + delayKind: delayKindBest, + } + } else { + delayKind := Finalized{} + delayKindFinalized := newDelayKind[uint](delayKind) + change = PendingChange[string, uint]{ + nextAuthorities: currentAuthorities, + delay: 0, + canonHeight: canonHeight, + canonHash: canonHash, + delayKind: delayKindFinalized, + } + } + + err := authorities.addPendingChange(change, isDescOf) + require.NoError(t, err) + } + + addPendingChangeFunction(5, "A", false) + addPendingChangeFunction(10, "B", false) + addPendingChangeFunction(15, "C0", false) + addPendingChangeFunction(15, "C1", true) + addPendingChangeFunction(15, "C2", false) + addPendingChangeFunction(15, "C3", true) + addPendingChangeFunction(20, "D", true) + + // applying the standard hashNumber at A should not prune anything + // other then the hashNumber that was applied + _, err = authorities.applyStandardChanges("A", 5, isDescOf, nil) + require.NoError(t, err) + require.Equal(t, 6, len(authorities.pendingChanges())) + + // same for B + _, err = authorities.applyStandardChanges("B", 10, isDescOf, nil) + require.NoError(t, err) + require.Equal(t, 5, len(authorities.pendingChanges())) + + // finalising C0 should clear all forced changes but D + _, err = authorities.applyStandardChanges("C0", 15, isDescOf, nil) + require.NoError(t, err) + require.Equal(t, 1, len(authorities.pendingForcedChanges)) + require.Equal(t, "D", authorities.pendingForcedChanges[0].canonHash) +} + +func assertExpectedSet(t *testing.T, authSetID authoritySetChangeID, expected setIDNumber[uint]) { + t.Helper() + authSetVal, err := authSetID.Value() + require.NoError(t, err) + switch val := authSetVal.(type) { + case set[uint]: + require.Equal(t, expected, val.inner) + default: + err = fmt.Errorf("invalid authSetID type") + } + require.NoError(t, err) +} + +func assertUnknown(t *testing.T, authSetID authoritySetChangeID) { + t.Helper() + authSetVal, err := authSetID.Value() + require.NoError(t, err) + isUnknown := false + switch authSetVal.(type) { + case unknown: + isUnknown = true + } + require.True(t, isUnknown) +} + +func assertLatest(t *testing.T, authSetID authoritySetChangeID) { + t.Helper() + authSetVal, err := authSetID.Value() + require.NoError(t, err) + isLatest := false + switch authSetVal.(type) { + case latest: + isLatest = true + } + require.True(t, isLatest) +} + +func TestAuthoritySetChangesInsert(t *testing.T) { + authoritySetChanges := AuthoritySetChanges[uint]{} + authoritySetChanges.append(0, 41) + authoritySetChanges.append(1, 81) + authoritySetChanges.append(4, 121) + + authoritySetChanges.insert(101) + + expChange := setIDNumber[uint]{ + setID: 2, + blockNumber: 101, + } + + authSetID, err := authoritySetChanges.getSetID(100) + require.NoError(t, err) + assertExpectedSet(t, authSetID, expChange) + + authSetID, err = authoritySetChanges.getSetID(101) + require.NoError(t, err) + assertExpectedSet(t, authSetID, expChange) +} + +func TestAuthoritySetChangesForCompleteData(t *testing.T) { + authoritySetChanges := AuthoritySetChanges[uint]{} + authoritySetChanges.append(0, 41) + authoritySetChanges.append(1, 81) + authoritySetChanges.append(2, 121) + + expChange0 := setIDNumber[uint]{ + setID: 0, + blockNumber: 41, + } + + expChange1 := setIDNumber[uint]{ + setID: 1, + blockNumber: 81, + } + + authSetID, err := authoritySetChanges.getSetID(20) + require.NoError(t, err) + assertExpectedSet(t, authSetID, expChange0) + + authSetID, err = authoritySetChanges.getSetID(40) + require.NoError(t, err) + assertExpectedSet(t, authSetID, expChange0) + + authSetID, err = authoritySetChanges.getSetID(41) + require.NoError(t, err) + assertExpectedSet(t, authSetID, expChange0) + + authSetID, err = authoritySetChanges.getSetID(42) + require.NoError(t, err) + assertExpectedSet(t, authSetID, expChange1) + + authSetID, err = authoritySetChanges.getSetID(141) + require.NoError(t, err) + assertLatest(t, authSetID) +} + +func TestAuthoritySetChangesForIncompleteData(t *testing.T) { + authoritySetChanges := AuthoritySetChanges[uint]{} + authoritySetChanges.append(2, 41) + authoritySetChanges.append(3, 81) + authoritySetChanges.append(4, 121) + + expChange := setIDNumber[uint]{ + setID: 3, + blockNumber: 81, + } + + authSetID, err := authoritySetChanges.getSetID(20) + require.NoError(t, err) + assertUnknown(t, authSetID) + + authSetID, err = authoritySetChanges.getSetID(40) + require.NoError(t, err) + assertUnknown(t, authSetID) + + authSetID, err = authoritySetChanges.getSetID(41) + require.NoError(t, err) + assertUnknown(t, authSetID) + + authSetID, err = authoritySetChanges.getSetID(42) + require.NoError(t, err) + assertExpectedSet(t, authSetID, expChange) + + authSetID, err = authoritySetChanges.getSetID(141) + require.NoError(t, err) + assertLatest(t, authSetID) +} + +func TestIterFromWorks(t *testing.T) { + authoritySetChanges := AuthoritySetChanges[uint]{} + authoritySetChanges.append(1, 41) + authoritySetChanges.append(2, 81) + + // we are missing the data for the first set, therefore we should return `None` + iterSet := authoritySetChanges.IterFrom(40) + require.Nil(t, iterSet) + + // after adding the data for the first set the same query should work + authoritySetChanges = AuthoritySetChanges[uint]{} + authoritySetChanges.append(0, 21) + authoritySetChanges.append(1, 41) + authoritySetChanges.append(2, 81) + authoritySetChanges.append(3, 121) + + expectedChanges := &AuthoritySetChanges[uint]{ + setIDNumber[uint]{ + setID: 1, + blockNumber: 41, + }, + setIDNumber[uint]{ + setID: 2, + blockNumber: 81, + }, + setIDNumber[uint]{ + setID: 3, + blockNumber: 121, + }, + } + + iterSet = authoritySetChanges.IterFrom(40) + require.Equal(t, expectedChanges, iterSet) + + expectedChanges = &AuthoritySetChanges[uint]{ + setIDNumber[uint]{ + setID: 2, + blockNumber: 81, + }, + setIDNumber[uint]{ + setID: 3, + blockNumber: 121, + }, + } + + iterSet = authoritySetChanges.IterFrom(41) + require.Equal(t, expectedChanges, iterSet) + + iterSet = authoritySetChanges.IterFrom(121) + require.Equal(t, 0, len(*iterSet)) + + iterSet = authoritySetChanges.IterFrom(200) + require.Equal(t, 0, len(*iterSet)) +} + +func TestAuthoritySet_InvalidAuthorityList(t *testing.T) { + type args struct { + authorities []Authority + } + tests := []struct { + name string + args args + exp bool + }{ + { + name: "nilAuthorities", + args: args{ + authorities: nil, + }, + exp: true, + }, + { + name: "emptyAuthorities", + args: args{ + authorities: []Authority{}, + }, + exp: true, + }, + { + name: "invalidAuthoritiesWeight", + args: args{ + authorities: []Authority{ + { + Weight: 0, + }, + }, + }, + exp: true, + }, + { + name: "validAuthorityList", + args: args{ + authorities: []Authority{ + { + Weight: 1, + }, + }, + }, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + if got := invalidAuthorityList(tt.args.authorities); got != tt.exp { + t.Errorf("invalidAuthorityList() = %v, want %v", got, tt.exp) + } + }) + } +} diff --git a/client/consensus/grandpa/change_tree.go b/client/consensus/grandpa/change_tree.go new file mode 100644 index 00000000000..cd9ffaa7365 --- /dev/null +++ b/client/consensus/grandpa/change_tree.go @@ -0,0 +1,464 @@ +// Copyright 2023 ChainSafe Systems (ON) +// SPDX-License-Identifier: LGPL-3.0-only + +package grandpa + +import ( + "errors" + "fmt" + + "github.com/ChainSafe/gossamer/pkg/scale" + "golang.org/x/exp/constraints" +) + +/* + The grandpa ChangeTree is a structure built to track pending changes across forks for the Grandpa Protocol. + This structure is intended to represent an acyclic directed graph where the children are + placed in descending order and number, you can ensure node ancestry using the `isDescendantOfFunc`. +*/ + +var ( + errDuplicateHashes = errors.New("duplicated hashes") + errUnfinalisedAncestor = errors.New("finalised descendent of Tree node without finalising its " + + "ancestor(s) first") + errRevert = errors.New("tried to import or finalise node that is an ancestor of " + + "a previously finalised node") +) + +// ChangeTree keeps track of changes across forks +type ChangeTree[H comparable, N constraints.Unsigned] struct { + roots []*pendingChangeNode[H, N] + bestFinalizedNumber *N +} + +// NewChangeTree create an empty ChangeTree +func NewChangeTree[H comparable, N constraints.Unsigned]() ChangeTree[H, N] { + return ChangeTree[H, N]{} +} + +// pendingChangeNode Represents a node in the ChangeTree +type pendingChangeNode[H comparable, N constraints.Unsigned] struct { + change *PendingChange[H, N] + children []*pendingChangeNode[H, N] +} + +// Roots returns the roots of each fork in the ChangeTree +// This is the equivalent of the slice in the outermost layer of the roots +func (ct *ChangeTree[H, N]) Roots() []*pendingChangeNode[H, N] { //skipcq: RVV-B0011 + return ct.roots +} + +// Import a new node into the roots. +// +// The given function `is_descendent_of` should return `true` if the second +// hash (target) is a descendent of the first hash (base). +// +// This method assumes that children in the same branch are imported in order. +// +// Returns `true` if the imported node is a root. +// WARNING: some users of this method (i.e. consensus epoch changes roots) currently silently +// rely on a **post-order DFS** traversal. If we are using instead a top-down traversal method +// then the `is_descendent_of` closure, when used after a warp-sync, may end up querying the +// backend for a block (the one corresponding to the root) that is not present and thus will +// return a wrong result. +func (ct *ChangeTree[H, N]) Import(hash H, + number N, + change PendingChange[H, N], + isDescendentOf IsDescendentOf[H]) (bool, error) { + for _, root := range ct.roots { + imported, err := root.importNode(hash, number, change, isDescendentOf) + if err != nil { + return false, err + } + + if imported { + logger.Debugf("changes on header %s (%d) imported successfully", + hash, number) + return false, nil + } + } + + pendingChangeNode := &pendingChangeNode[H, N]{ + change: &change, + } + + ct.roots = append(ct.roots, pendingChangeNode) + return true, nil +} + +// PendingChanges does a preorder traversal of the ChangeTree to get all pending changes +func (ct *ChangeTree[H, N]) PendingChanges() []PendingChange[H, N] { + if len(ct.roots) == 0 { + return nil + } + + changes := make([]PendingChange[H, N], 0, len(ct.roots)) + + for i := 0; i < len(ct.roots); i++ { + getPreOrder(&changes, ct.roots[i]) + } + + return changes +} + +// getPreOrderChangeNodes does a preorder traversal of the ChangeTree to get all pending changes +func (ct *ChangeTree[H, N]) getPreOrderChangeNodes() []*pendingChangeNode[H, N] { + if len(ct.roots) == 0 { + return nil + } + + changes := &[]*pendingChangeNode[H, N]{} + + for i := 0; i < len(ct.roots); i++ { + getPreOrderChangeNodes(changes, ct.roots[i]) + } + + return *changes +} + +// FinalizesAnyWithDescendentIf Checks if any node in the tree is finalised by either finalising the +// node itself or a node's descendent that's not in the tree, guaranteeing +// that the node being finalised isn't a descendent of (or equal to) any of +// the node's children. Returns *true if the node being finalised is +// a root, *false if the node being finalised is not a root, and +// nil if no node in the tree is finalised. The given `Predicate` is +// checked on the prospective finalised root and must pass for finalisation +// to occur. The given function `is_descendent_of` should return `true` if +// the second hash (target) is a descendent of the first hash (base). func(T) bool +func (ct *ChangeTree[H, N]) FinalizesAnyWithDescendentIf(hash *H, + number N, + isDescendentOf IsDescendentOf[H], + predicate func(*PendingChange[H, N]) bool) (*bool, error) { + if ct.bestFinalizedNumber != nil { + if number <= *ct.bestFinalizedNumber { + return nil, errRevert + } + } + + roots := ct.Roots() + + nodes := ct.getPreOrderChangeNodes() + + // check if the given hash is equal or a descendent of any node in the + // tree, if we find a valid node that passes the Predicate then we must + // ensure that we're not finalising past any of its child nodes. + for i := 0; i < len(nodes); i++ { + root := nodes[i] + isDesc, err := isDescendentOf(root.change.canonHash, *hash) + if err != nil { + return nil, err + } + + if predicate(root.change) && (root.change.canonHash == *hash || isDesc) { + children := root.children + for _, child := range children { + isChildDescOf, err := isDescendentOf(child.change.canonHash, *hash) + if err != nil { + return nil, err + } + + if child.change.canonHeight <= number && (child.change.canonHash == *hash || isChildDescOf) { + return nil, errUnfinalisedAncestor + } + } + + isEqual := false + for _, val := range roots { + if val.change.canonHash == root.change.canonHash { + isEqual = true + return &isEqual, nil + } + } + return &isEqual, nil + } + } + + return nil, nil +} + +// FinalizationResult Result of finalising a node (that could be a part of the roots or not). +type FinalizationResult scale.VaryingDataType + +// Set will set a VaryingDataTypeValue using the underlying VaryingDataType +func (fr *FinalizationResult) Set(val scale.VaryingDataTypeValue) (err error) { + vdt := scale.VaryingDataType(*fr) + err = vdt.Set(val) + if err != nil { + return + } + *fr = FinalizationResult(vdt) + return +} + +// Value will return value from underying VaryingDataType +func (fr *FinalizationResult) Value() (val scale.VaryingDataTypeValue, err error) { + vdt := scale.VaryingDataType(*fr) + return vdt.Value() +} + +func newFinalizationResult[H comparable, N constraints.Unsigned]() FinalizationResult { + vdt, err := scale.NewVaryingDataType(changed[H, N]{}, unchanged{}) + if err != nil { + panic(err) + } + return FinalizationResult(vdt) +} + +type changed[H comparable, N constraints.Unsigned] struct { + value *PendingChange[H, N] +} + +func (changed[H, N]) Index() uint { + return 0 +} + +type unchanged struct{} + +func (unchanged) Index() uint { + return 1 +} + +// FinalizeWithDescendentIf Finalize a root in the roots by either finalising the node itself or a +// node's descendent that's not in the roots, guaranteeing that the node +// being finalised isn't a descendent of (or equal to) any of the root's +// children. The given `Predicate` is checked on the prospective finalised +// root and must pass for finalisation to occur. The given function +// `is_descendent_of` should return `true` if the second hash (target) is a +// descendent of the first hash (base). +func (ct *ChangeTree[H, N]) FinalizeWithDescendentIf(hash *H, //skipcq: GO-R1005 + number N, + isDescendentOf IsDescendentOf[H], + predicate func(*PendingChange[H, N]) bool) (result FinalizationResult, err error) { + if ct.bestFinalizedNumber != nil { + if number <= *ct.bestFinalizedNumber { + return result, errRevert + } + } + + roots := ct.Roots() + + // check if the given hash is equal or a descendent of any root, if we + // find a valid root that passes the Predicate then we must ensure that + // we're not finalising past any children node. + var position *N + for i, root := range roots { + isDesc, err := isDescendentOf(root.change.canonHash, *hash) + if err != nil { + return result, err + } + + if predicate(root.change) && (root.change.canonHash == *hash || isDesc) { + for _, child := range root.children { + isDesc, err := isDescendentOf(child.change.canonHash, *hash) + if err != nil { + return result, err + } + if child.change.canonHeight <= number && (child.change.canonHash == *hash || isDesc) { + return result, errUnfinalisedAncestor + } + } + uintI := N(i) + position = &uintI + break + } + } + + var nodeData *PendingChange[H, N] + if position != nil { + node := ct.swapRemove(ct.Roots(), *position) + ct.roots = node.children + ct.bestFinalizedNumber = &node.change.canonHeight + nodeData = node.change + } + + // Retain only roots that are descendents of the finalised block (this + // happens if the node has been properly finalised) or that are + // ancestors (or equal) to the finalised block (in this case the node + // wasn't finalised earlier presumably because the Predicate didn't + // pass). + didChange := false + roots = ct.Roots() + + ct.roots = []*pendingChangeNode[H, N]{} + for _, root := range roots { + retain := false + if root.change.canonHeight > number { + isDescA, err := isDescendentOf(*hash, root.change.canonHash) + if err != nil { + return result, err + } + + if isDescA { + retain = true + } + } else if root.change.canonHeight == number && root.change.canonHash == *hash { + retain = true + } else { + isDescB, err := isDescendentOf(root.change.canonHash, *hash) + if err != nil { + return result, err + } + + if isDescB { + retain = true + } + } + if retain { + ct.roots = append(ct.roots, root) + } else { + didChange = true + } + + ct.bestFinalizedNumber = &number + } + + result = newFinalizationResult[H, N]() + + if nodeData != nil { + err = result.Set(changed[H, N]{ + value: nodeData, + }) + if err != nil { + return result, err + } + return result, nil + } else { + if didChange { + err = result.Set(changed[H, N]{}) + if err != nil { + return result, err + } + return result, nil + } else { + err = result.Set(unchanged{}) + if err != nil { + return result, err + } + return result, nil + } + } +} + +func (pcn *pendingChangeNode[H, N]) importNode(hash H, + number N, + change PendingChange[H, N], + isDescendentOf IsDescendentOf[H]) (bool, error) { + announcingHash := pcn.change.canonHash + if hash == announcingHash { + return false, fmt.Errorf("%w: %v", errDuplicateHashes, hash) + } + + isDescendant, err := isDescendentOf(announcingHash, hash) + if err != nil { + return false, fmt.Errorf("cannot check ancestry: %w", err) + } + + if !isDescendant { + return false, nil + } + + if number <= pcn.change.canonHeight { + return false, nil + } + + for _, childrenNodes := range pcn.children { + imported, err := childrenNodes.importNode(hash, number, change, isDescendentOf) + if err != nil { + return false, err + } + + if imported { + return true, nil + } + } + childrenNode := &pendingChangeNode[H, N]{ + change: &change, + } + pcn.children = append(pcn.children, childrenNode) + return true, nil +} + +func getPreOrder[H comparable, N constraints.Unsigned](changes *[]PendingChange[H, N], //skipcq: RVV-A0006 + changeNode *pendingChangeNode[H, N]) { + if changeNode == nil { + return + } + + if changes != nil { + tempChanges := *changes + tempChanges = append(tempChanges, *changeNode.change) + *changes = tempChanges + } else { + change := []PendingChange[H, N]{*changeNode.change} + changes = &change + } + + for i := 0; i < len(changeNode.children); i++ { + getPreOrder(changes, changeNode.children[i]) + } +} + +func getPreOrderChangeNodes[H comparable, N constraints.Unsigned]( //skipcq: RVV-A0006 //skipcq: RVV-B0001 + changes *[]*pendingChangeNode[H, N], + changeNode *pendingChangeNode[H, N]) { + if changeNode == nil { + return + } + + if changes != nil { + tempChanges := *changes + tempChanges = append(tempChanges, changeNode) + *changes = tempChanges + } else { + change := []*pendingChangeNode[H, N]{changeNode} + changes = &change + } + + for i := 0; i < len(changeNode.children); i++ { + getPreOrderChangeNodes(changes, changeNode.children[i]) + } +} + +// Removes an element from the vector and returns it. +// +// The removed element is replaced by the last element of the vector. +// +// This does not preserve ordering, but is *O*(1). +// +// Panics if `index` is out of bounds. +func (ct *ChangeTree[H, N]) swapRemove(roots []*pendingChangeNode[H, N], index N) pendingChangeNode[H, N] { + if index >= N(len(roots)) { + panic("swap_remove index out of bounds") + } + + val := pendingChangeNode[H, N]{} + if roots[index] != nil { + val = *roots[index] + } else { + panic("nil pending hashNumber node") + } + + lastElem := roots[len(roots)-1] + + newRoots := roots[:len(roots)-1] + // This should be the case where last elem was removed + if index == N(len(newRoots)) { + ct.roots = newRoots + return val + } + newRoots[index] = lastElem + ct.roots = newRoots + return val +} + +// Remove from the tree some nodes (and their subtrees) using a `filter` predicate. +// +// The `filter` is called over tree nodes and returns a filter action: +// - `Remove` if the node and its subtree should be removed; +// - `KeepNode` if we should maintain the node and keep processing the tree. +// - `KeepTree` if we should maintain the node and its entire subtree. +// +// An iterator over all the pruned nodes is returned. +func (_ *ChangeTree[H, N]) drainFilter() { //nolint //skipcq: SCC-U1000 //skipcq: RVV-B0013 + // TODO implement +} diff --git a/client/consensus/grandpa/change_tree_test.go b/client/consensus/grandpa/change_tree_test.go new file mode 100644 index 00000000000..02433821aff --- /dev/null +++ b/client/consensus/grandpa/change_tree_test.go @@ -0,0 +1,85 @@ +// Copyright 2023 ChainSafe Systems (ON) +// SPDX-License-Identifier: LGPL-3.0-only + +package grandpa + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestSwapRemove(t *testing.T) { + change1 := &PendingChange[string, uint]{ + canonHash: "a", + } + + change2 := &PendingChange[string, uint]{ + canonHash: "b", + } + + change3 := &PendingChange[string, uint]{ + canonHash: "b", + } + + pendingChangeNode1 := &pendingChangeNode[string, uint]{ + change: change1, + } + + pendingChangeNode2 := &pendingChangeNode[string, uint]{ + change: change2, + } + + pendingChangeNode3 := &pendingChangeNode[string, uint]{ + change: change3, + } + + changeNodes1 := []*pendingChangeNode[string, uint]{ + pendingChangeNode1, + pendingChangeNode2, + } + + changeNodes2 := []*pendingChangeNode[string, uint]{ + pendingChangeNode1, + pendingChangeNode2, + pendingChangeNode3, + } + type args struct { + ct ChangeTree[string, uint] + index uint + } + tests := []struct { + name string + args args + exp pendingChangeNode[string, uint] + }{ + { + name: "TwoElemSliceDeletingLastElement", + args: args{ + ct: ChangeTree[string, uint]{ + roots: changeNodes1, + }, + index: 1, + }, + exp: *pendingChangeNode2, + }, + { + name: "ThreeElemSliceDeletingFirstElement", + args: args{ + ct: ChangeTree[string, uint]{ + roots: changeNodes2, + }, + index: 0, + }, + exp: *pendingChangeNode1, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + oldLength := len(tt.args.ct.Roots()) + removedVal := tt.args.ct.swapRemove(tt.args.ct.Roots(), tt.args.index) + require.Equal(t, tt.exp, removedVal) + require.Equal(t, oldLength-1, len(tt.args.ct.Roots())) + }) + } +} diff --git a/client/consensus/grandpa/interfaces.go b/client/consensus/grandpa/interfaces.go new file mode 100644 index 00000000000..8bf2e01fdf2 --- /dev/null +++ b/client/consensus/grandpa/interfaces.go @@ -0,0 +1,7 @@ +// Copyright 2023 ChainSafe Systems (ON) +// SPDX-License-Identifier: LGPL-3.0-only + +package grandpa + +// Telemetry TODO issue #3474 +type Telemetry interface{} diff --git a/client/consensus/grandpa/lib.go b/client/consensus/grandpa/lib.go new file mode 100644 index 00000000000..0b232348dcc --- /dev/null +++ b/client/consensus/grandpa/lib.go @@ -0,0 +1,17 @@ +// Copyright 2023 ChainSafe Systems (ON) +// SPDX-License-Identifier: LGPL-3.0-only + +package grandpa + +import ( + "github.com/ChainSafe/gossamer/internal/log" + "github.com/ChainSafe/gossamer/lib/crypto" +) + +var logger = log.NewFromGlobal(log.AddContext("consensus", "grandpa")) + +// Authority represents a grandpa authority +type Authority struct { + Key crypto.PublicKey + Weight uint64 +}