diff --git a/dot/services.go b/dot/services.go index 89ecf571eb..a7a62cb31c 100644 --- a/dot/services.go +++ b/dot/services.go @@ -456,7 +456,7 @@ func (nodeBuilder) createGRANDPAService(config *cfg.Config, st *state.Service, k } func (nodeBuilder) createBlockVerifier(st *state.Service) *babe.VerificationManager { - return babe.NewVerificationManager(st.Block, st.Epoch) + return babe.NewVerificationManager(st.Block, st.Slot, st.Epoch) } func (nodeBuilder) newSyncService(config *cfg.Config, st *state.Service, fg BlockJustificationVerifier, diff --git a/dot/state/initialize.go b/dot/state/initialize.go index 08f7a1eafc..198c8a6d3b 100644 --- a/dot/state/initialize.go +++ b/dot/state/initialize.go @@ -100,6 +100,7 @@ func (s *Service) Initialise(gen *genesis.Genesis, header *types.Header, t *trie s.Block = blockState s.Epoch = epochState s.Grandpa = grandpaState + s.Slot = NewSlotState(db) } else if err = db.Close(); err != nil { return fmt.Errorf("failed to close database: %s", err) } diff --git a/dot/state/service.go b/dot/state/service.go index 97859764fc..674930f922 100644 --- a/dot/state/service.go +++ b/dot/state/service.go @@ -34,6 +34,7 @@ type Service struct { Transaction *TransactionState Epoch *EpochState Grandpa *GrandpaState + Slot *SlotState closeCh chan interface{} PrunerCfg pruner.Config @@ -157,6 +158,7 @@ func (s *Service) Start() (err error) { "created state service with head %s, highest number %d and genesis hash %s", s.Block.BestBlockHash(), num, s.Block.genesisHash.String()) + s.Slot = NewSlotState(s.db) return nil } diff --git a/dot/state/slot.go b/dot/state/slot.go new file mode 100644 index 0000000000..9ffdc0ad1d --- /dev/null +++ b/dot/state/slot.go @@ -0,0 +1,193 @@ +// Copyright 2023 ChainSafe Systems (ON) +// SPDX-License-Identifier: LGPL-3.0-only + +package state + +import ( + "bytes" + "encoding/binary" + "errors" + "fmt" + + "github.com/ChainSafe/chaindb" + "github.com/ChainSafe/gossamer/dot/types" + "github.com/ChainSafe/gossamer/pkg/scale" +) + +const slotTablePrefix = "slot" + +// We keep at least this number of slots in database. +const maxSlotCapacity uint64 = 1000 + +// We prune slots when they reach this number. +const pruningBound = 2 * maxSlotCapacity + +var ( + slotHeaderMapKey = []byte("slot_header_map") + slotHeaderStartKey = []byte("slot_header_start") +) + +type SlotState struct { + db chaindb.Database +} + +func NewSlotState(db *chaindb.BadgerDB) *SlotState { + slotStateDB := chaindb.NewTable(db, slotTablePrefix) + + return &SlotState{ + db: slotStateDB, + } +} + +type headerAndSigner struct { + Header *types.Header `scale:"1"` + Signer types.AuthorityID `scale:"2"` +} + +func (s *SlotState) CheckEquivocation(slotNow, slot uint64, header *types.Header, + signer types.AuthorityID) (*types.BabeEquivocationProof, error) { + // We don't check equivocations for old headers out of our capacity. + // checking slotNow is greater than slot to avoid overflow, same as saturating_sub + if saturatingSub(slotNow, slot) > maxSlotCapacity { + return nil, nil //nolint:nilnil + } + + slotEncoded := make([]byte, 8) + binary.LittleEndian.PutUint64(slotEncoded, slot) + + currentSlotKey := bytes.Join([][]byte{slotHeaderMapKey, slotEncoded[:]}, nil) + encodedHeadersWithSigners, err := s.db.Get(currentSlotKey) + if err != nil && !errors.Is(err, chaindb.ErrKeyNotFound) { + return nil, fmt.Errorf("getting key slot header map key %d: %w", slot, err) + } + + headersWithSigners := make([]headerAndSigner, 0) + if len(encodedHeadersWithSigners) > 0 { + encodedSliceHeadersWithSigners := make([][]byte, 0) + + err = scale.Unmarshal(encodedHeadersWithSigners, &encodedSliceHeadersWithSigners) + if err != nil { + return nil, fmt.Errorf("unmarshaling encoded headers with signers: %w", err) + } + + for _, encodedHeaderAndSigner := range encodedSliceHeadersWithSigners { + // each header and signer instance should have an empty header + // so we will be able to scale decode the whole byte stream with + // the digests correctly in place + decodedHeaderAndSigner := headerAndSigner{ + Header: types.NewEmptyHeader(), + } + + err := scale.Unmarshal(encodedHeaderAndSigner, &decodedHeaderAndSigner) + if err != nil { + return nil, fmt.Errorf("unmarshaling header with signer: %w", err) + } + + headersWithSigners = append(headersWithSigners, decodedHeaderAndSigner) + } + } + + firstSavedSlot := slot + firstSavedSlotEncoded, err := s.db.Get(slotHeaderStartKey) + if err != nil && !errors.Is(err, chaindb.ErrKeyNotFound) { + return nil, fmt.Errorf("getting key slot header start key: %w", err) + } + + if len(firstSavedSlotEncoded) > 0 { + firstSavedSlot = binary.LittleEndian.Uint64(firstSavedSlotEncoded) + } + + if slotNow < firstSavedSlot { + // The code below assumes that slots will be visited sequentially. + return nil, nil //nolint:nilnil + } + + for _, headerAndSigner := range headersWithSigners { + // A proof of equivocation consists of two headers: + // 1) signed by the same voter, + if headerAndSigner.Signer == signer { + // 2) with different hash + if headerAndSigner.Header.Hash() != header.Hash() { + return &types.BabeEquivocationProof{ + Slot: slot, + Offender: signer, + FirstHeader: *headerAndSigner.Header, + SecondHeader: *header, + }, nil + } else { + // We don't need to continue in case of duplicated header, + // since it's already saved and a possible equivocation + // would have been detected before. + return nil, nil //nolint:nilnil + } + } + } + + keysToDelete := make([][]byte, 0) + newFirstSavedSlot := firstSavedSlot + + if slotNow-firstSavedSlot >= pruningBound { + newFirstSavedSlot = saturatingSub(slotNow, maxSlotCapacity) + + for s := firstSavedSlot; s < newFirstSavedSlot; s++ { + slotEncoded := make([]byte, 8) + binary.LittleEndian.PutUint64(slotEncoded, s) + + toDelete := bytes.Join([][]byte{slotHeaderMapKey, slotEncoded[:]}, nil) + keysToDelete = append(keysToDelete, toDelete) + } + } + + headersWithSigners = append(headersWithSigners, headerAndSigner{Header: header, Signer: signer}) + encodedHeaderAndSigner := make([][]byte, len(headersWithSigners)) + + // encode each header and signer and push to a slice of bytes + // that will be scale encoded and stored in the database + for idx, headerAndSigner := range headersWithSigners { + encoded, err := scale.Marshal(headerAndSigner) + if err != nil { + return nil, fmt.Errorf("marshalling header and signer: %w", err) + } + + encodedHeaderAndSigner[idx] = encoded + } + + encodedHeadersWithSigners, err = scale.Marshal(encodedHeaderAndSigner) + if err != nil { + return nil, fmt.Errorf("marshalling: %w", err) + } + + batch := s.db.NewBatch() + err = batch.Put(currentSlotKey, encodedHeadersWithSigners) + if err != nil { + return nil, fmt.Errorf("while batch putting encoded headers with signers: %w", err) + } + + newFirstSavedSlotEncoded := make([]byte, 8) + binary.LittleEndian.PutUint64(newFirstSavedSlotEncoded, newFirstSavedSlot) + err = batch.Put(slotHeaderStartKey, newFirstSavedSlotEncoded) + if err != nil { + return nil, fmt.Errorf("while batch putting encoded new first saved slot: %w", err) + } + + for _, toDelete := range keysToDelete { + err := batch.Del(toDelete) + if err != nil { + return nil, fmt.Errorf("while batch deleting key %s: %w", string(toDelete), err) + } + } + + err = batch.Flush() + if err != nil { + return nil, fmt.Errorf("failed to flush batch operations: %w", err) + } + + return nil, nil //nolint:nilnil +} + +func saturatingSub(a, b uint64) uint64 { + if a > b { + return a - b + } + return 0 +} diff --git a/dot/state/slot_test.go b/dot/state/slot_test.go new file mode 100644 index 0000000000..e7f1eea43e --- /dev/null +++ b/dot/state/slot_test.go @@ -0,0 +1,138 @@ +// Copyright 2023 ChainSafe Systems (ON) +// SPDX-License-Identifier: LGPL-3.0-only + +package state + +import ( + "bytes" + "crypto/rand" + "encoding/binary" + "errors" + "io" + "testing" + + "github.com/ChainSafe/chaindb" + "github.com/ChainSafe/gossamer/dot/types" + "github.com/ChainSafe/gossamer/lib/common" + "github.com/ChainSafe/gossamer/lib/crypto/sr25519" + "github.com/ChainSafe/gossamer/lib/keystore" + "github.com/minio/sha256-simd" + "github.com/stretchr/testify/require" +) + +func createHeader(t *testing.T, n uint) (header *types.Header) { + t.Helper() + + randomBytes := make([]byte, 32) + _, err := io.ReadFull(rand.Reader, randomBytes) + require.NoError(t, err) + + hasher := sha256.New() + _, err = hasher.Write(randomBytes) + require.NoError(t, err) + + header = types.NewEmptyHeader() + header.Number = n + + // so that different headers for the same number get different hashes + header.ParentHash = common.NewHash(hasher.Sum(nil)) + + header.Hash() + return header +} + +func checkSlotToMapKeyExists(t *testing.T, db chaindb.Database, slotNumber uint64) bool { + t.Helper() + + slotEncoded := make([]byte, 8) + binary.LittleEndian.PutUint64(slotEncoded, slotNumber) + + slotToHeaderKey := bytes.Join([][]byte{slotHeaderMapKey, slotEncoded[:]}, nil) + + _, err := db.Get(slotToHeaderKey) + if err != nil { + if errors.Is(err, chaindb.ErrKeyNotFound) { + return false + } + + t.Fatalf("unexpected error while getting key: %s", err) + } + + return true +} + +func Test_checkEquivocation(t *testing.T) { + inMemoryDB, err := chaindb.NewBadgerDB(&chaindb.Config{ + DataDir: t.TempDir(), + InMemory: true, + }) + require.NoError(t, err) + + kr, err := keystore.NewSr25519Keyring() + require.NoError(t, err) + + alicePublicKey := kr.KeyAlice.Public().(*sr25519.PublicKey) + aliceAuthorityID := types.AuthorityID(alicePublicKey.AsBytes()) + + header1 := createHeader(t, 1) // @ slot 2 + header2 := createHeader(t, 2) // @ slot 2 + header3 := createHeader(t, 2) // @ slot 4 + header4 := createHeader(t, 3) // @ slot MAX_SLOT_CAPACITY + 4 + header5 := createHeader(t, 4) // @ slot MAX_SLOT_CAPACITY + 4 + header6 := createHeader(t, 3) // @ slot 4 + + slotState := NewSlotState(inMemoryDB) + + // It's ok to sign same headers. + equivProf, err := slotState.CheckEquivocation(2, 2, header1, aliceAuthorityID) + require.NoError(t, err) + require.Nil(t, equivProf) + + equivProf, err = slotState.CheckEquivocation(3, 2, header1, aliceAuthorityID) + require.NoError(t, err) + require.Nil(t, equivProf) + + // But not two different headers at the same slot. + equivProf, err = slotState.CheckEquivocation(4, 2, header2, aliceAuthorityID) + require.NoError(t, err) + require.NotNil(t, equivProf) + require.Equal(t, &types.BabeEquivocationProof{ + Slot: 2, + Offender: aliceAuthorityID, + FirstHeader: *header1, + SecondHeader: *header2, + }, equivProf) + + // Different slot is ok. + equivProf, err = slotState.CheckEquivocation(5, 4, header3, aliceAuthorityID) + require.NoError(t, err) + require.Nil(t, equivProf) + + // Here we trigger pruning and save header 4. + equivProf, err = slotState.CheckEquivocation( + pruningBound+2, maxSlotCapacity+4, header4, aliceAuthorityID) + require.NoError(t, err) + require.Nil(t, equivProf) + + require.False(t, checkSlotToMapKeyExists(t, slotState.db, 2)) + require.False(t, checkSlotToMapKeyExists(t, slotState.db, 4)) + + // This fails because header 5 is an equivocation of header 4. + equivProf, err = slotState.CheckEquivocation( + pruningBound+3, maxSlotCapacity+4, header5, aliceAuthorityID) + require.NoError(t, err) + require.NotNil(t, equivProf) + + require.Equal(t, &types.BabeEquivocationProof{ + Slot: maxSlotCapacity + 4, + Offender: aliceAuthorityID, + FirstHeader: *header4, + SecondHeader: *header5, + }, equivProf) + + // This is ok because we pruned the corresponding header. Shows that we are pruning. + equivProf, err = slotState.CheckEquivocation( + pruningBound+4, 4, header6, aliceAuthorityID) + require.NoError(t, err) + require.Nil(t, equivProf) +} diff --git a/lib/babe/mock_state_test.go b/lib/babe/mock_state_test.go index 6edfc36199..f97f87dd39 100644 --- a/lib/babe/mock_state_test.go +++ b/lib/babe/mock_state_test.go @@ -1,5 +1,5 @@ // Code generated by MockGen. DO NOT EDIT. -// Source: github.com/ChainSafe/gossamer/lib/babe (interfaces: BlockState,ImportedBlockNotifierManager,StorageState,TransactionState,EpochState,BlockImportHandler) +// Source: github.com/ChainSafe/gossamer/lib/babe (interfaces: BlockState,ImportedBlockNotifierManager,StorageState,TransactionState,EpochState,BlockImportHandler,SlotState) // Package babe is a generated GoMock package. package babe @@ -667,3 +667,41 @@ func (mr *MockBlockImportHandlerMockRecorder) HandleBlockProduced(arg0, arg1 int mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "HandleBlockProduced", reflect.TypeOf((*MockBlockImportHandler)(nil).HandleBlockProduced), arg0, arg1) } + +// MockSlotState is a mock of SlotState interface. +type MockSlotState struct { + ctrl *gomock.Controller + recorder *MockSlotStateMockRecorder +} + +// MockSlotStateMockRecorder is the mock recorder for MockSlotState. +type MockSlotStateMockRecorder struct { + mock *MockSlotState +} + +// NewMockSlotState creates a new mock instance. +func NewMockSlotState(ctrl *gomock.Controller) *MockSlotState { + mock := &MockSlotState{ctrl: ctrl} + mock.recorder = &MockSlotStateMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockSlotState) EXPECT() *MockSlotStateMockRecorder { + return m.recorder +} + +// CheckEquivocation mocks base method. +func (m *MockSlotState) CheckEquivocation(arg0, arg1 uint64, arg2 *types.Header, arg3 types.AuthorityID) (*types.BabeEquivocationProof, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "CheckEquivocation", arg0, arg1, arg2, arg3) + ret0, _ := ret[0].(*types.BabeEquivocationProof) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// CheckEquivocation indicates an expected call of CheckEquivocation. +func (mr *MockSlotStateMockRecorder) CheckEquivocation(arg0, arg1, arg2, arg3 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CheckEquivocation", reflect.TypeOf((*MockSlotState)(nil).CheckEquivocation), arg0, arg1, arg2, arg3) +} diff --git a/lib/babe/mocks_generate_test.go b/lib/babe/mocks_generate_test.go index 1236b19e54..e9e3eacb00 100644 --- a/lib/babe/mocks_generate_test.go +++ b/lib/babe/mocks_generate_test.go @@ -6,4 +6,4 @@ package babe //go:generate mockgen -destination=mock_telemetry_test.go -package $GOPACKAGE . Telemetry //go:generate mockgen -destination=mocks/runtime.go -package mocks github.com/ChainSafe/gossamer/lib/runtime Instance //go:generate mockgen -destination=mocks/core.go -package mocks github.com/ChainSafe/gossamer/dot/core Network,BlockImportDigestHandler -//go:generate mockgen -destination=mock_state_test.go -package $GOPACKAGE . BlockState,ImportedBlockNotifierManager,StorageState,TransactionState,EpochState,BlockImportHandler +//go:generate mockgen -destination=mock_state_test.go -package $GOPACKAGE . BlockState,ImportedBlockNotifierManager,StorageState,TransactionState,EpochState,BlockImportHandler,SlotState diff --git a/lib/babe/state.go b/lib/babe/state.go index 775124dfad..7652d0f2bc 100644 --- a/lib/babe/state.go +++ b/lib/babe/state.go @@ -14,6 +14,11 @@ import ( "github.com/ChainSafe/gossamer/lib/transaction" ) +type SlotState interface { + CheckEquivocation(slotNow, slot uint64, header *types.Header, + signer types.AuthorityID) (*types.BabeEquivocationProof, error) +} + // BlockState interface for block state methods type BlockState interface { BestBlockHash() common.Hash diff --git a/lib/babe/verify.go b/lib/babe/verify.go index 485febe461..c66022e4ca 100644 --- a/lib/babe/verify.go +++ b/lib/babe/verify.go @@ -7,6 +7,7 @@ import ( "errors" "fmt" "sync" + "time" "github.com/ChainSafe/gossamer/dot/types" "github.com/ChainSafe/gossamer/lib/common" @@ -38,6 +39,7 @@ type onDisabledInfo struct { type VerificationManager struct { lock sync.RWMutex blockState BlockState + slotState SlotState epochState EpochState epochInfo map[uint64]*verifierInfo // map of epoch number -> info needed for verification // there may be different OnDisabled digests on different @@ -47,9 +49,10 @@ type VerificationManager struct { } // NewVerificationManager returns a new NewVerificationManager -func NewVerificationManager(blockState BlockState, epochState EpochState) *VerificationManager { +func NewVerificationManager(blockState BlockState, slotState SlotState, epochState EpochState) *VerificationManager { return &VerificationManager{ epochState: epochState, + slotState: slotState, blockState: blockState, epochInfo: make(map[uint64]*verifierInfo), onDisabled: make(map[uint64]map[uint32][]*onDisabledInfo), @@ -182,9 +185,12 @@ func (v *VerificationManager) VerifyBlock(header *types.Header) error { } v.lock.Unlock() + slotDuration, err := v.epochState.GetSlotDuration() + if err != nil { + return fmt.Errorf("getting current slot duration: %w", err) + } - verifier := newVerifier(v.blockState, epoch, info) - + verifier := newVerifier(v.blockState, v.slotState, epoch, info, slotDuration) return verifier.verifyAuthorshipRight(header) } @@ -215,22 +221,27 @@ func (v *VerificationManager) getVerifierInfo(epoch uint64, header *types.Header // verifier is a BABE verifier for a specific authority set, randomness, and threshold type verifier struct { blockState BlockState + slotState SlotState epoch uint64 authorities []types.Authority randomness Randomness threshold *scale.Uint128 secondarySlots bool + slotDuration time.Duration } // newVerifier returns a Verifier for the epoch described by the given descriptor -func newVerifier(blockState BlockState, epoch uint64, info *verifierInfo) *verifier { +func newVerifier(blockState BlockState, slotState SlotState, + epoch uint64, info *verifierInfo, slotDuration time.Duration) *verifier { return &verifier{ blockState: blockState, + slotState: slotState, epoch: epoch, authorities: info.authorities, randomness: info.randomness, threshold: info.threshold, secondarySlots: info.secondarySlots, + slotDuration: slotDuration, } } @@ -341,39 +352,21 @@ func (b *verifier) verifyAuthorshipRight(header *types.Header) error { return nil } -func (b *verifier) submitAndReportEquivocation( - slot uint64, authorityIndex uint32, firstHeader, secondHeader types.Header) error { - - // TODO: Check if it is initial sync - // don't report any equivocations during initial sync - // as they are most likely stale. - // https://github.com/ChainSafe/gossamer/issues/3004 - +func (b *verifier) submitAndReportEquivocation(equivocationProof *types.BabeEquivocationProof) error { bestBlockHash := b.blockState.BestBlockHash() runtimeInstance, err := b.blockState.GetRuntime(bestBlockHash) if err != nil { return fmt.Errorf("getting runtime: %w", err) } - if len(b.authorities) <= int(authorityIndex) { - return ErrAuthIndexOutOfBound - } - - offenderPublicKey := b.authorities[authorityIndex].ToRaw().Key - keyOwnershipProof, err := runtimeInstance.BabeGenerateKeyOwnershipProof(slot, offenderPublicKey) + keyOwnershipProof, err := runtimeInstance.BabeGenerateKeyOwnershipProof( + equivocationProof.Slot, equivocationProof.Offender) if err != nil { return fmt.Errorf("getting key ownership proof from runtime: %w", err) } else if keyOwnershipProof == nil { return errEmptyKeyOwnershipProof } - equivocationProof := &types.BabeEquivocationProof{ - Offender: types.AuthorityID(offenderPublicKey), - Slot: slot, - FirstHeader: firstHeader, - SecondHeader: secondHeader, - } - err = runtimeInstance.BabeSubmitReportEquivocationUnsignedExtrinsic(*equivocationProof, keyOwnershipProof) if err != nil { return fmt.Errorf("submitting equivocation report to runtime: %w", err) @@ -384,50 +377,42 @@ func (b *verifier) submitAndReportEquivocation( // verifyBlockEquivocation checks if the given block's author has occupied the corresponding slot more than once. // It returns true if the block was equivocated. +// TODO: Check if it is initial sync +// don't report any equivocations during initial sync +// as they are most likely stale. +// https://github.com/ChainSafe/gossamer/issues/3004 func (b *verifier) verifyBlockEquivocation(header *types.Header) (bool, error) { - author, err := getAuthorityIndex(header) + authorityIndex, slotNumber, err := getAuthorityIndexAndSlot(header) if err != nil { return false, fmt.Errorf("failed to get authority index: %w", err) } - currentHash := header.Hash() - slot, err := types.GetSlotFromHeader(header) - if err != nil { - return false, fmt.Errorf("failed to get slot from header of block %s: %w", currentHash, err) + if len(b.authorities) <= int(authorityIndex) { + return false, ErrAuthIndexOutOfBound } - blockHashesInSlot, err := b.blockState.GetBlockHashesBySlot(slot) - if err != nil { - return false, fmt.Errorf("failed to get blocks produced in slot: %w", err) + if header.Hash() == b.blockState.GenesisHash() { + return false, nil } - for _, blockHashInSlot := range blockHashesInSlot { - if blockHashInSlot == currentHash { - continue - } - - existingHeader, err := b.blockState.GetHeader(blockHashInSlot) - if err != nil { - return false, fmt.Errorf("failed to get header for block: %w", err) - } - - authorOfExistingHeader, err := getAuthorityIndex(existingHeader) - if err != nil { - return false, fmt.Errorf("failed to get authority index for block %s: %w", blockHashInSlot, err) - } - if authorOfExistingHeader != author { - continue - } + slotNow := getCurrentSlot(b.slotDuration) + signer := types.AuthorityID(b.authorities[authorityIndex].ToRaw().Key) + equivocationProof, err := b.slotState.CheckEquivocation(slotNow, slotNumber, + header, signer) + if err != nil { + return false, fmt.Errorf("checking equivocation: %w", err) + } - err = b.submitAndReportEquivocation(slot, authorOfExistingHeader, *existingHeader, *header) - if err != nil { - return true, fmt.Errorf("submitting and reporting equivocation: %w", err) - } + if equivocationProof == nil { + return false, nil + } - return true, nil + err = b.submitAndReportEquivocation(equivocationProof) + if err != nil { + return false, fmt.Errorf("submiting equivocation: %w", err) } - return false, nil + return true, nil } func (b *verifier) verifyPreRuntimeDigest(digest *types.PreRuntimeDigest) (scale.VaryingDataTypeValue, error) { @@ -536,34 +521,36 @@ func (b *verifier) verifyPrimarySlotWinner(authorityIndex uint32, return pk.VrfVerify(t, vrfOutput, vrfProof) } -func getAuthorityIndex(header *types.Header) (uint32, error) { +func getAuthorityIndexAndSlot(header *types.Header) (authIdx uint32, slot uint64, err error) { if len(header.Digest.Types) == 0 { - return 0, fmt.Errorf("for block hash %s: %w", header.Hash(), errNoDigest) + return 0, 0, fmt.Errorf("for block hash %s: %w", header.Hash(), errNoDigest) } digestValue, err := header.Digest.Types[0].Value() if err != nil { - return 0, fmt.Errorf("getting first digest type value: %w", err) + return 0, 0, fmt.Errorf("getting first digest type value: %w", err) } preDigest, ok := digestValue.(types.PreRuntimeDigest) if !ok { - return 0, fmt.Errorf("first digest item is not pre-runtime digest") + return 0, 0, types.ErrNoFirstPreDigest } babePreDigest, err := types.DecodeBabePreDigest(preDigest.Data) if err != nil { - return 0, fmt.Errorf("cannot decode babe header from pre-digest: %s", err) + return 0, 0, fmt.Errorf("cannot decode babe header from pre-digest: %s", err) } - var authIdx uint32 switch d := babePreDigest.(type) { case types.BabePrimaryPreDigest: authIdx = d.AuthorityIndex + slot = d.SlotNumber case types.BabeSecondaryVRFPreDigest: authIdx = d.AuthorityIndex + slot = d.SlotNumber case types.BabeSecondaryPlainPreDigest: authIdx = d.AuthorityIndex + slot = d.SlotNumber } - return authIdx, nil + return authIdx, slot, nil } diff --git a/lib/babe/verify_integration_test.go b/lib/babe/verify_integration_test.go index 138ab8e021..49731c502f 100644 --- a/lib/babe/verify_integration_test.go +++ b/lib/babe/verify_integration_test.go @@ -27,7 +27,12 @@ import ( func TestVerificationManager_OnDisabled_InvalidIndex(t *testing.T) { genesis, genesisTrie, genesisHeader := newWestendDevGenesisWithTrieAndHeader(t) babeService := createTestService(t, ServiceConfig{}, genesis, genesisTrie, genesisHeader, nil) - vm := NewVerificationManager(babeService.blockState, babeService.epochState) + + db, err := chaindb.NewBadgerDB(&chaindb.Config{DataDir: t.TempDir(), InMemory: true}) + require.NoError(t, err) + + slotState := state.NewSlotState(db) + vm := NewVerificationManager(babeService.blockState, slotState, babeService.epochState) bestBlockHash := babeService.blockState.BestBlockHash() runtime, err := babeService.blockState.GetRuntime(bestBlockHash) @@ -46,7 +51,12 @@ func TestVerificationManager_OnDisabled_InvalidIndex(t *testing.T) { func TestVerificationManager_OnDisabled_NewDigest(t *testing.T) { genesis, genesisTrie, genesisHeader := newWestendDevGenesisWithTrieAndHeader(t) babeService := createTestService(t, ServiceConfig{}, genesis, genesisTrie, genesisHeader, nil) - vm := NewVerificationManager(babeService.blockState, babeService.epochState) + + db, err := chaindb.NewBadgerDB(&chaindb.Config{DataDir: t.TempDir(), InMemory: true}) + require.NoError(t, err) + slotState := state.NewSlotState(db) + + vm := NewVerificationManager(babeService.blockState, slotState, babeService.epochState) bestBlockHash := babeService.blockState.BestBlockHash() runtime, err := babeService.blockState.GetRuntime(bestBlockHash) @@ -92,7 +102,11 @@ func TestVerificationManager_OnDisabled_DuplicateDigest(t *testing.T) { epochData, err := babeService.initiateEpoch(testEpochIndex) require.NoError(t, err) - vm := NewVerificationManager(babeService.blockState, babeService.epochState) + db, err := chaindb.NewBadgerDB(&chaindb.Config{DataDir: t.TempDir(), InMemory: true}) + require.NoError(t, err) + slotState := state.NewSlotState(db) + vm := NewVerificationManager(babeService.blockState, slotState, babeService.epochState) + vm.epochInfo[testEpochIndex] = &verifierInfo{ authorities: epochData.authorities, threshold: epochData.threshold, @@ -121,12 +135,17 @@ func TestVerificationManager_OnDisabled_DuplicateDigest(t *testing.T) { func TestVerificationManager_VerifyBlock_Secondary(t *testing.T) { genesis, genesisTrie, genesisHeader := newWestendDevGenesisWithTrieAndHeader(t) babeService := createTestService(t, ServiceConfig{}, genesis, genesisTrie, genesisHeader, nil) - vm := NewVerificationManager(babeService.blockState, babeService.epochState) + + db, err := chaindb.NewBadgerDB(&chaindb.Config{DataDir: t.TempDir(), InMemory: true}) + require.NoError(t, err) + slotState := state.NewSlotState(db) + + vm := NewVerificationManager(babeService.blockState, slotState, babeService.epochState) secondaryDigest := createSecondaryVRFPreDigest(t, keyring.Alice().(*sr25519.Keypair), 0, uint64(0), uint64(0), Randomness{}) babeDigest := types.NewBabeDigest() - err := babeDigest.Set(secondaryDigest) + err = babeDigest.Set(secondaryDigest) require.NoError(t, err) encodedBabeDigest, err := scale.Marshal(babeDigest) @@ -167,7 +186,12 @@ func TestVerificationManager_VerifyBlock_CurrentEpoch(t *testing.T) { t.Parallel() genesis, genesisTrie, genesisHeader := newWestendDevGenesisWithTrieAndHeader(t) babeService := createTestService(t, ServiceConfig{}, genesis, genesisTrie, genesisHeader, nil) - vm := NewVerificationManager(babeService.blockState, babeService.epochState) + + db, err := chaindb.NewBadgerDB(&chaindb.Config{DataDir: t.TempDir(), InMemory: true}) + require.NoError(t, err) + slotState := state.NewSlotState(db) + + vm := NewVerificationManager(babeService.blockState, slotState, babeService.epochState) bestBlockHash := babeService.blockState.BestBlockHash() runtime, err := babeService.blockState.GetRuntime(bestBlockHash) @@ -200,7 +224,12 @@ func TestVerificationManager_VerifyBlock_FutureEpoch(t *testing.T) { } genesis, genesisTrie, genesisHeader := newWestendDevGenesisWithTrieAndHeader(t) babeService := createTestService(t, ServiceConfig{}, genesis, genesisTrie, genesisHeader, babeConfig) - verificationManager := NewVerificationManager(babeService.blockState, babeService.epochState) + + db, err := chaindb.NewBadgerDB(&chaindb.Config{DataDir: t.TempDir(), InMemory: true}) + require.NoError(t, err) + slotState := state.NewSlotState(db) + + verificationManager := NewVerificationManager(babeService.blockState, slotState, babeService.epochState) bestBlockHash := babeService.blockState.BestBlockHash() runtime, err := babeService.blockState.GetRuntime(bestBlockHash) @@ -245,7 +274,12 @@ func TestVerificationManager_VerifyBlock_MultipleEpochs(t *testing.T) { } genesis, genesisTrie, genesisHeader := newWestendDevGenesisWithTrieAndHeader(t) babeService := createTestService(t, ServiceConfig{}, genesis, genesisTrie, genesisHeader, babeConfig) - verificationManager := NewVerificationManager(babeService.blockState, babeService.epochState) + + db, err := chaindb.NewBadgerDB(&chaindb.Config{DataDir: t.TempDir(), InMemory: true}) + require.NoError(t, err) + slotState := state.NewSlotState(db) + + verificationManager := NewVerificationManager(babeService.blockState, slotState, babeService.epochState) bestBlockHash := babeService.blockState.BestBlockHash() runtime, err := babeService.blockState.GetRuntime(bestBlockHash) @@ -300,7 +334,12 @@ func TestVerificationManager_VerifyBlock_InvalidBlockOverThreshold(t *testing.T) genesis, genesisTrie, genesisHeader := newWestendDevGenesisWithTrieAndHeader(t) babeService := createTestService(t, ServiceConfig{}, genesis, genesisTrie, genesisHeader, babeConfig) - vm := NewVerificationManager(babeService.blockState, babeService.epochState) + + db, err := chaindb.NewBadgerDB(&chaindb.Config{DataDir: t.TempDir(), InMemory: true}) + require.NoError(t, err) + slotState := state.NewSlotState(db) + + vm := NewVerificationManager(babeService.blockState, slotState, babeService.epochState) epochData, err := babeService.initiateEpoch(testEpochIndex) require.NoError(t, err) @@ -343,7 +382,12 @@ func TestVerificationManager_VerifyBlock_InvalidBlockAuthority(t *testing.T) { } genesisBob, genesisTrieBob, genesisHeaderBob := newWestendDevGenesisWithTrieAndHeader(t) babeServiceBob := createTestService(t, ServiceConfig{}, genesisBob, genesisTrieBob, genesisHeaderBob, babeConfig) - verificationManager := NewVerificationManager(babeServiceBob.blockState, babeServiceBob.epochState) + + db, err := chaindb.NewBadgerDB(&chaindb.Config{DataDir: t.TempDir(), InMemory: true}) + require.NoError(t, err) + slotState := state.NewSlotState(db) + + verificationManager := NewVerificationManager(babeServiceBob.blockState, slotState, babeServiceBob.epochState) bestBlockHash := babeService.blockState.BestBlockHash() runtime, err := babeService.blockState.GetRuntime(bestBlockHash) @@ -393,11 +437,15 @@ func TestVerifyPrimarySlotWinner(t *testing.T) { digest, ok := babePreDigest.(types.BabePrimaryPreDigest) require.True(t, ok) - verifier := newVerifier(babeService.blockState, testEpochIndex, &verifierInfo{ + db, err := chaindb.NewBadgerDB(&chaindb.Config{DataDir: t.TempDir(), InMemory: true}) + require.NoError(t, err) + slotState := state.NewSlotState(db) + + verifier := newVerifier(babeService.blockState, slotState, testEpochIndex, &verifierInfo{ authorities: epochData.authorities, threshold: epochData.threshold, randomness: epochData.randomness, - }) + }, time.Second) ok, err = verifier.verifyPrimarySlotWinner(digest.AuthorityIndex, slotNumber, digest.VRFOutput, digest.VRFProof) require.NoError(t, err) @@ -422,11 +470,15 @@ func TestVerifyAuthorshipRight(t *testing.T) { slot := getSlot(t, runtime, time.Now()) block := createTestBlockWithSlot(t, babeService, &genesisHeader, [][]byte{}, testEpochIndex, epochData, slot) - verifier := newVerifier(babeService.blockState, testEpochIndex, &verifierInfo{ + db, err := chaindb.NewBadgerDB(&chaindb.Config{DataDir: t.TempDir(), InMemory: true}) + require.NoError(t, err) + slotState := state.NewSlotState(db) + + verifier := newVerifier(babeService.blockState, slotState, testEpochIndex, &verifierInfo{ authorities: epochData.authorities, threshold: epochData.threshold, randomness: epochData.randomness, - }) + }, time.Second) err = verifier.verifyAuthorshipRight(&block.Header) require.NoError(t, err) @@ -435,26 +487,39 @@ func TestVerifyAuthorshipRight(t *testing.T) { func TestVerifyAuthorshipRight_Equivocation(t *testing.T) { genesis, genesisTrie, genesisHeader := newWestendDevGenesisWithTrieAndHeader(t) babeService := createTestService(t, ServiceConfig{}, genesis, genesisTrie, genesisHeader, nil) - verificationManager := NewVerificationManager(babeService.blockState, babeService.epochState) - epochData, err := babeService.initiateEpoch(testEpochIndex) + db, err := chaindb.NewBadgerDB(&chaindb.Config{DataDir: t.TempDir(), InMemory: true}) require.NoError(t, err) + slotState := state.NewSlotState(db) - bestBlockHash := babeService.blockState.BestBlockHash() - runtime, err := babeService.blockState.GetRuntime(bestBlockHash) + verificationManager := NewVerificationManager(babeService.blockState, slotState, babeService.epochState) + + epochData, err := babeService.initiateEpoch(testEpochIndex) require.NoError(t, err) + // bestBlockHash := babeService.blockState.BestBlockHash() + // runtime, err := babeService.blockState.GetRuntime(bestBlockHash) + // require.NoError(t, err) + // slots are 6 seconds on westend and using time.Now() allows us to create a block at any point in the slot. // So we need to manually set time to produce consistent results. See here: // https://github.com/paritytech/substrate/blob/09de7b41599add51cf27eca8f1bc4c50ed8e9453/frame/timestamp/src/lib.rs#L229 // https://github.com/paritytech/substrate/blob/09de7b41599add51cf27eca8f1bc4c50ed8e9453/frame/timestamp/src/lib.rs#L206 - timestamp := time.Unix(6, 0) - slot := getSlot(t, runtime, timestamp) - block := createTestBlockWithSlot(t, babeService, &genesisHeader, [][]byte{}, testEpochIndex, epochData, slot) + + const slotDuration = 6 * time.Second + slotNumber := getCurrentSlot(slotDuration) + startTime := getSlotStartTime(slotNumber, slotDuration) + slot := NewSlot(startTime, slotDuration, slotNumber) + + if time.Now().After(startTime) { + slot = NewSlot(startTime.Add(6*time.Second), slotDuration, slotNumber+1) + } + + block := createTestBlockWithSlot(t, babeService, &genesisHeader, [][]byte{}, testEpochIndex, epochData, *slot) block.Header.Hash() // create new block for same slot - block2 := createTestBlockWithSlot(t, babeService, &genesisHeader, [][]byte{}, testEpochIndex, epochData, slot) + block2 := createTestBlockWithSlot(t, babeService, &genesisHeader, [][]byte{}, testEpochIndex, epochData, *slot) block2.Header.Hash() err = babeService.blockState.AddBlock(block) @@ -537,7 +602,7 @@ func TestVerifyForkBlocksWithRespectiveEpochData(t *testing.T) { digestHandler.Start() - verificationManager := NewVerificationManager(stateService.Block, epochState) + verificationManager := NewVerificationManager(stateService.Block, stateService.Slot, epochState) /* * lets issue different blocks starting from genesis (a fork) diff --git a/lib/babe/verify_test.go b/lib/babe/verify_test.go index ef0a9a5ba0..7e246d01ae 100644 --- a/lib/babe/verify_test.go +++ b/lib/babe/verify_test.go @@ -7,6 +7,7 @@ import ( "errors" "fmt" "testing" + "time" "github.com/ChainSafe/gossamer/dot/state" "github.com/ChainSafe/gossamer/dot/types" @@ -19,6 +20,8 @@ import ( "github.com/stretchr/testify/require" ) +const testSlotDuration = time.Second + // this is generated by printing key ownership proof while running `test_generate_equivocation_report_blob` // https://github.com/paritytech/substrate/blob/ded44948e2d5a398abcb4e342b0513cb690961bb/frame/grandpa/src/benchmarking.rs#L85 var testKeyOwnershipProof types.OpaqueKeyOwnershipProof = types.OpaqueKeyOwnershipProof([]byte{64, 138, 252, 29, 127, 102, 189, 129, 207, 47, 157, 60, 17, 138, 194, 121, 139, 92, 176, 175, 224, 16, 185, 93, 175, 251, 224, 81, 209, 61, 0, 71}) //nolint:lll @@ -68,7 +71,7 @@ func encodeAndHashHeader(t *testing.T, header *types.Header) common.Hash { return hash } -func newTestVerifier(kp *sr25519.Keypair, blockState BlockState, +func newTestVerifier(kp *sr25519.Keypair, blockState BlockState, slotState SlotState, threshold *scale.Uint128, secSlots bool) *verifier { authority := types.NewAuthority(kp.Public(), uint64(1)) info := &verifierInfo{ @@ -77,7 +80,7 @@ func newTestVerifier(kp *sr25519.Keypair, blockState BlockState, threshold: threshold, secondarySlots: secSlots, } - return newVerifier(blockState, 1, info) + return newVerifier(blockState, slotState, 1, info, testSlotDuration) } func Test_getAuthorityIndex(t *testing.T) { @@ -100,7 +103,7 @@ func Test_getAuthorityIndex(t *testing.T) { // BabePrimaryPreDigest Case babeDigest := types.NewBabeDigest() - err = babeDigest.Set(types.BabePrimaryPreDigest{AuthorityIndex: 21}) + err = babeDigest.Set(types.BabePrimaryPreDigest{AuthorityIndex: 21, SlotNumber: 1}) assert.NoError(t, err) bdEnc, err := scale.Marshal(babeDigest) @@ -117,7 +120,7 @@ func Test_getAuthorityIndex(t *testing.T) { //BabeSecondaryVRFPreDigest Case babeDigest2 := types.NewBabeDigest() - err = babeDigest2.Set(types.BabeSecondaryVRFPreDigest{AuthorityIndex: 21}) + err = babeDigest2.Set(types.BabeSecondaryVRFPreDigest{AuthorityIndex: 21, SlotNumber: 10}) assert.NoError(t, err) bdEnc2, err := scale.Marshal(babeDigest2) @@ -134,7 +137,7 @@ func Test_getAuthorityIndex(t *testing.T) { //BabeSecondaryPlainPreDigest case babeDigest3 := types.NewBabeDigest() - err = babeDigest3.Set(types.BabeSecondaryPlainPreDigest{AuthorityIndex: 21}) + err = babeDigest3.Set(types.BabeSecondaryPlainPreDigest{AuthorityIndex: 21, SlotNumber: 100}) assert.NoError(t, err) bdEnc3, err := scale.Marshal(babeDigest3) @@ -153,10 +156,11 @@ func Test_getAuthorityIndex(t *testing.T) { header *types.Header } tests := []struct { - name string - args args - exp uint32 - expErr error + name string + args args + expAuthIdx uint32 + expSlotNumber uint64 + expErr error }{ { name: "No Digest", @@ -166,7 +170,7 @@ func Test_getAuthorityIndex(t *testing.T) { { name: "First Digest Invalid Type", args: args{headerNoPre}, - expErr: errors.New("first digest item is not pre-runtime digest"), + expErr: errors.New("first digest item is not pre-digest"), }, { name: "Invalid_Preruntime_Digest_Type", @@ -175,30 +179,34 @@ func Test_getAuthorityIndex(t *testing.T) { " index 0: EOF"), }, { - name: "BabePrimaryPreDigest_Type", - args: args{headerPrimary}, - exp: 21, + name: "BabePrimaryPreDigest_Type", + args: args{headerPrimary}, + expAuthIdx: 21, + expSlotNumber: 1, }, { - name: "BabeSecondaryVRFPreDigest_Type", - args: args{headerSecondary}, - exp: 21, + name: "BabeSecondaryVRFPreDigest_Type", + args: args{headerSecondary}, + expAuthIdx: 21, + expSlotNumber: 10, }, { - name: "BabeSecondaryPlainPreDigest_Type", - args: args{headerSecondaryPlain}, - exp: 21, + name: "BabeSecondaryPlainPreDigest_Type", + args: args{headerSecondaryPlain}, + expAuthIdx: 21, + expSlotNumber: 100, }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - res, err := getAuthorityIndex(tt.args.header) + authIdx, slotNumber, err := getAuthorityIndexAndSlot(tt.args.header) if tt.expErr != nil { assert.EqualError(t, err, tt.expErr.Error()) } else { assert.NoError(t, err) } - assert.Equal(t, tt.exp, res) + assert.Equal(t, tt.expAuthIdx, authIdx) + assert.Equal(t, tt.expSlotNumber, slotNumber) }) } } @@ -222,8 +230,9 @@ func Test_verifier_verifyPrimarySlotWinner(t *testing.T) { threshold: scale.MaxUint128, } - v := newVerifier(mockBlockState, 1, vi) - v1 := newVerifier(mockBlockState, 1, vi1) + mockSlotState := NewMockSlotState(nil) + v := newVerifier(mockBlockState, mockSlotState, 1, vi, time.Second) + v1 := newVerifier(mockBlockState, mockSlotState, 1, vi1, time.Second) output, proof, err := kp.VrfSign(makeTranscript(Randomness{}, uint64(1), 1)) assert.NoError(t, err) @@ -311,10 +320,12 @@ func Test_verifier_verifyPreRuntimeDigest(t *testing.T) { threshold: scale.MaxUint128, } - v := newVerifier(mockBlockState, 1, vi) + mockSlotState := NewMockSlotState(nil) + + v := newVerifier(mockBlockState, mockSlotState, 1, vi, testSlotDuration) // Invalid - v2 := newVerifier(mockBlockState, 13, vi) + v2 := newVerifier(mockBlockState, mockSlotState, 13, vi, testSlotDuration) // Above threshold case vi1 := &verifierInfo{ @@ -322,7 +333,7 @@ func Test_verifier_verifyPreRuntimeDigest(t *testing.T) { threshold: &scale.Uint128{}, } - v1 := newVerifier(mockBlockState, 1, vi1) + v1 := newVerifier(mockBlockState, mockSlotState, 1, vi1, testSlotDuration) //BabeSecondaryVRFPreDigest case secVRFDigest := types.BabeSecondaryVRFPreDigest{ @@ -352,8 +363,8 @@ func Test_verifier_verifyPreRuntimeDigest(t *testing.T) { secondarySlots: true, } - vVRFSec := newVerifier(mockBlockState, 1, viVRFSec) - vVRFSec2 := newVerifier(mockBlockState, 1, viVRFSec2) + vVRFSec := newVerifier(mockBlockState, mockSlotState, 1, viVRFSec, testSlotDuration) + vVRFSec2 := newVerifier(mockBlockState, mockSlotState, 1, viVRFSec2, testSlotDuration) //BabeSecondaryPlainPreDigest case secDigest := types.BabeSecondaryPlainPreDigest{AuthorityIndex: 0, SlotNumber: uint64(1)} @@ -372,8 +383,8 @@ func Test_verifier_verifyPreRuntimeDigest(t *testing.T) { secondarySlots: true, } - vSec := newVerifier(mockBlockState, 1, viSec) - vSec2 := newVerifier(mockBlockState, 1, viSec2) + vSec := newVerifier(mockBlockState, mockSlotState, 1, viSec, testSlotDuration) + vSec2 := newVerifier(mockBlockState, mockSlotState, 1, viSec2, testSlotDuration) type args struct { digest *types.PreRuntimeDigest @@ -462,7 +473,6 @@ func Test_verifier_verifyPreRuntimeDigest(t *testing.T) { func Test_verifier_verifyAuthorshipRight(t *testing.T) { ctrl := gomock.NewController(t) mockBlockState := NewMockBlockState(ctrl) - mockBlockStateErr := NewMockBlockState(ctrl) //Generate keys kp, err := sr25519.GenerateKeypair() @@ -529,14 +539,9 @@ func Test_verifier_verifyAuthorshipRight(t *testing.T) { testSecVrfHeader := newTestHeader(t, *types.NewBABEPreRuntimeDigest(encVrfHeader)) testSecVrfHeader.ParentHash = testVrfParentHash - h := common.MustHexToHash("0x01") - h1 := []common.Hash{h} + mockBlockState.EXPECT().GenesisHash().Return(common.Hash([32]byte{})) - mockBlockState.EXPECT().GetHeader(h).Return(types.NewEmptyHeader(), nil) - mockBlockState.EXPECT().GetBlockHashesBySlot(uint64(1)).Return(h1, nil) - - mockBlockStateErr.EXPECT().GetHeader(h).Return(nil, errors.New("get header error")) - mockBlockStateErr.EXPECT().GetBlockHashesBySlot(uint64(1)).Return(h1, nil) + mockSlotStateNoOp := NewMockSlotState(nil) // Case 0: First element not preruntime digest header0 := newTestHeader(t, testInvalidSeal, testInvalidSeal) @@ -551,28 +556,28 @@ func Test_verifier_verifyAuthorshipRight(t *testing.T) { babePrd, err := testBabePrimaryPreDigest.ToPreRuntimeDigest() assert.NoError(t, err) header3 := newTestHeader(t, *babePrd, testInvalidSeal) - babeVerifier := newTestVerifier(kp, mockBlockState, scale.MaxUint128, false) + babeVerifier := newTestVerifier(kp, mockBlockState, mockSlotStateNoOp, scale.MaxUint128, false) // Case 4: Invalid signature - BabePrimaryPreDigest babePrd2, err := testBabePrimaryPreDigest.ToPreRuntimeDigest() assert.NoError(t, err) header4 := newTestHeader(t, *babePrd2) signAndAddSeal(t, kp, header4, []byte{1}) - babeVerifier2 := newTestVerifier(kp, mockBlockState, scale.MaxUint128, false) + babeVerifier2 := newTestVerifier(kp, mockBlockState, mockSlotStateNoOp, scale.MaxUint128, false) // Case 5: Invalid signature - BabeSecondaryPlainPreDigest babeSecPlainPrd, err := testBabeSecondaryPlainPreDigest.ToPreRuntimeDigest() assert.NoError(t, err) header5 := newTestHeader(t, *babeSecPlainPrd) signAndAddSeal(t, kp, header5, []byte{1}) - babeVerifier3 := newTestVerifier(kp, mockBlockState, scale.MaxUint128, true) + babeVerifier3 := newTestVerifier(kp, mockBlockState, mockSlotStateNoOp, scale.MaxUint128, true) // Case 6: Invalid signature - BabeSecondaryVrfPreDigest encSecVrfDigest := newEncodedBabeDigest(t, testBabeSecondaryVRFPreDigest) assert.NoError(t, err) header6 := newTestHeader(t, *types.NewBABEPreRuntimeDigest(encSecVrfDigest)) signAndAddSeal(t, kp, header6, []byte{1}) - babeVerifier4 := newTestVerifier(kp, mockBlockState, scale.MaxUint128, true) + babeVerifier4 := newTestVerifier(kp, mockBlockState, mockSlotStateNoOp, scale.MaxUint128, true) // Case 7: GetAuthorityIndex Err babeParentPrd, err := testBabePrimaryPreDigest.ToPreRuntimeDigest() @@ -588,10 +593,16 @@ func Test_verifier_verifyAuthorshipRight(t *testing.T) { hash := encodeAndHashHeader(t, header7) signAndAddSeal(t, kp, header7, hash[:]) - babeVerifier5 := newTestVerifier(kp, mockBlockState, scale.MaxUint128, false) - //// Case 8: Get header error - babeVerifier6 := newTestVerifier(kp, mockBlockStateErr, scale.MaxUint128, false) + signerPublicKey := kp.Public().(*sr25519.PublicKey) + signerAuthID := types.AuthorityID(signerPublicKey.AsBytes()) + + slotStateMockErr := errors.New("slot state mock error") + mockSlotState := NewMockSlotState(ctrl) + mockSlotState.EXPECT().CheckEquivocation(gomock.Any(), + testBabePrimaryPreDigest.SlotNumber, header7, signerAuthID).Return(nil, slotStateMockErr) + + babeVerifier5 := newTestVerifier(kp, mockBlockState, mockSlotState, scale.MaxUint128, false) tests := []struct { name string @@ -649,19 +660,11 @@ func Test_verifier_verifyAuthorshipRight(t *testing.T) { expErr: ErrBadSignature, }, { - name: "valid digest items, getAuthorityIndex error", + name: "check_equivocation_error", verifier: *babeVerifier5, header: header7, expErr: fmt.Errorf("could not verify block equivocation: "+ - "failed to get authority index for block %s: for block hash %s: %w", - h, types.NewEmptyHeader().Hash(), errNoDigest), - }, - { - name: "get header err", - verifier: *babeVerifier6, - header: header7, - expErr: fmt.Errorf("could not verify block equivocation: " + - "failed to get header for block: get header error"), + "checking equivocation: %w", slotStateMockErr), }, } for _, tt := range tests { @@ -678,182 +681,253 @@ func Test_verifier_verifyAuthorshipRight(t *testing.T) { } } -func Test_verifier_verifyBlockEquivocation(t *testing.T) { - t.Parallel() - +func Test_verifyBlockEquivocation(t *testing.T) { + //t.Parallel() kp, err := sr25519.GenerateKeypair() assert.NoError(t, err) - auth := types.NewAuthority(kp.Public(), uint64(1)) - vi := &verifierInfo{ - authorities: []types.Authority{*auth, *auth}, - threshold: scale.MaxUint128, - } - - // Case 1. could not get authority index from header - verifier1 := newVerifier(NewMockBlockState(gomock.NewController(t)), 1, vi) - testHeader1 := types.NewEmptyHeader() + slotStateMockErr := errors.New("slot state error") + getRuntimeErr := errors.New("mock get runtime error") - // Case 2. could not get slot from header - verifier2 := verifier1 output, proof, err := kp.VrfSign(makeTranscript(Randomness{}, uint64(1), 1)) assert.NoError(t, err) - testDigest := types.BabePrimaryPreDigest{ - AuthorityIndex: 1, + babePrimaryDigest := types.BabePrimaryPreDigest{ + AuthorityIndex: 0, SlotNumber: 1, VRFOutput: output, VRFProof: proof, } - prd, err := testDigest.ToPreRuntimeDigest() - assert.NoError(t, err) - - testHeader2 := newTestHeader(t, *prd) - testHeader2.Number = 0 - - // Case 3. could not get block hashes by slot - testHeader3 := newTestHeader(t, *prd) - testHeader3.Number = 1 - - mockBlockState3 := NewMockBlockState(gomock.NewController(t)) - mockBlockState3.EXPECT().GetBlockHashesBySlot(uint64(1)).Return( - nil, errors.New("test error")) - - verifier3 := newVerifier(mockBlockState3, 1, vi) - - // Case 4. no equivocation on finding the same block - testHeader4 := newTestHeader(t, *prd) - testHeader4.Number = 1 - testHash4 := testHeader4.Hash() - mockBlockState4 := NewMockBlockState(gomock.NewController(t)) - mockBlockState4.EXPECT().GetBlockHashesBySlot(uint64(1)).Return( - []common.Hash{testHash4}, nil) - - verifier4 := newVerifier(mockBlockState4, 1, vi) - - // Case 5. claiming a slot twice results in equivocation - testHeader5 := newTestHeader(t, *prd) - testHeader5.Number = 1 - - output5, proof5, err := kp.VrfSign(makeTranscript(Randomness{}, uint64(1), 2)) - assert.NoError(t, err) - - testDigest5 := types.BabePrimaryPreDigest{ - AuthorityIndex: 1, - SlotNumber: 1, - VRFOutput: output5, - VRFProof: proof5, - } - prd5, err := testDigest5.ToPreRuntimeDigest() + prd, err := babePrimaryDigest.ToPreRuntimeDigest() assert.NoError(t, err) - - existingHeader := newTestHeader(t, *prd5) - mockBlockState5 := NewMockBlockState(gomock.NewController(t)) - mockBlockState5.EXPECT().GetBlockHashesBySlot(uint64(1)).Return( - []common.Hash{existingHeader.Hash()}, nil) - mockBlockState5.EXPECT().GetHeader(existingHeader.Hash()).Return( - existingHeader, nil) - mockBlockState5.EXPECT().BestBlockHash().Return(existingHeader.Hash()) - - verifier5 := newVerifier(mockBlockState5, 1, vi) - - const slot = uint64(1) - const authorityIndex = uint32(1) - offenderPublicKey := verifier5.authorities[authorityIndex].ToRaw().Key - keyOwnershipProof := testKeyOwnershipProof - mockRuntime := mocks.NewMockInstance(gomock.NewController(t)) - testHeader5.Hash() - - equivocationProof := types.BabeEquivocationProof{ - Offender: offenderPublicKey, - Slot: slot, - FirstHeader: *existingHeader, - SecondHeader: *testHeader5, - } - - mockRuntime.EXPECT().BabeGenerateKeyOwnershipProof(slot, offenderPublicKey).Return(keyOwnershipProof, nil) - mockRuntime.EXPECT().BabeSubmitReportEquivocationUnsignedExtrinsic(equivocationProof, keyOwnershipProof).Return(nil) - - mockBlockState5.EXPECT().GetRuntime(existingHeader.Hash()).Return(mockRuntime, nil) - - mockBlockState6 := NewMockBlockState(gomock.NewController(t)) - mockBlockState6.EXPECT().GetBlockHashesBySlot(uint64(1)).Return( - []common.Hash{existingHeader.Hash()}, nil) - mockBlockState6.EXPECT().GetHeader(existingHeader.Hash()).Return( - existingHeader, nil) - mockBlockState6.EXPECT().BestBlockHash().Return(existingHeader.Hash()) - mockBlockState6.EXPECT().GetRuntime(existingHeader.Hash()).Return(nil, errors.New("test error")) - - verifier6 := newVerifier(mockBlockState6, 1, vi) - - testHeader6 := newTestHeader(t, *prd) - testHeader6.Number = 1 - - tests := []struct { - name string - verifier verifier - header *types.Header - equivocated bool - expErr error + defaultHeader := newTestHeader(t, *prd) + + cases := map[string]struct { + header *types.Header + wantErr error + errString string + expected bool + buildVerifier func(t *testing.T) *verifier }{ - { - name: "could not get authority index from header", - verifier: *verifier1, - header: testHeader1, - equivocated: false, - expErr: fmt.Errorf("failed to get authority index: for block hash %s: %w", testHeader1.Hash(), errNoDigest), + "empty_digest_failed_to_get_auth_index": { + header: types.NewEmptyHeader(), + errString: "failed to get authority index: for block hash " + + "0xdcdd89927d8a348e00257e1ecc8617f45edb5118efff3ea2f9961b2ad9b7690a: no digest provided", + wantErr: errNoDigest, + buildVerifier: func(t *testing.T) *verifier { + return &verifier{} + }, }, - { - name: "could not get slot from header", - verifier: *verifier2, - header: testHeader2, - equivocated: false, - expErr: fmt.Errorf("failed to get slot from header of block %s: %w", - testHeader2.Hash(), types.ErrGenesisHeader), + "cannot_get_slot_from_header": { + header: func() *types.Header { + wrongDigest := types.NewGrandpaConsensusDigest() + require.NoError(t, wrongDigest.Set(types.GrandpaForcedChange{})) + + data, err := scale.Marshal(wrongDigest) + require.NoError(t, err) + + consensusDigest := types.ConsensusDigest{ + ConsensusEngineID: types.GrandpaEngineID, + Data: data, + } + + return newTestHeader(t, consensusDigest) + }(), + errString: "failed to get authority index: first digest item is not pre-digest", + wantErr: types.ErrNoFirstPreDigest, + buildVerifier: func(t *testing.T) *verifier { + return &verifier{ + authorities: []types.Authority{}, + } + }, }, - { - name: "could not get block hashes by slot", - verifier: *verifier3, - header: testHeader3, - equivocated: false, - expErr: fmt.Errorf("failed to get blocks produced in slot: test error"), + "auth_index_out_of_bound": { + header: defaultHeader, + errString: "authority index doesn't exist", + wantErr: ErrAuthIndexOutOfBound, + buildVerifier: func(t *testing.T) *verifier { + return &verifier{ + authorities: []types.Authority{}, + } + }, }, - { - name: "no equivocation on finding the same block", - verifier: *verifier4, - header: testHeader4, - equivocated: false, - expErr: nil, + "failed_to_check_equivocation": { + header: defaultHeader, + errString: "checking equivocation: slot state error", + wantErr: slotStateMockErr, + buildVerifier: func(t *testing.T) *verifier { + ctrl := gomock.NewController(t) + + mockBlockState := NewMockBlockState(ctrl) + mockBlockState.EXPECT().GenesisHash().Return(common.Hash([32]byte{})) + + expectedAuthorityId := types.AuthorityID(kp.Public().Encode()) + mockSlotState := NewMockSlotState(ctrl) + mockSlotState. + EXPECT(). + CheckEquivocation(gomock.Any(), uint64(1), + defaultHeader, expectedAuthorityId). + Return(nil, slotStateMockErr) + + return &verifier{ + authorities: []types.Authority{ + { + Key: kp.Public(), + Weight: 1, + }, + }, + blockState: mockBlockState, + slotState: mockSlotState, + slotDuration: 6 * time.Second, + } + }, }, - { - name: "claiming same slot twice results in equivocation", - verifier: *verifier5, - header: testHeader5, - equivocated: true, - expErr: nil, + "nil_equivocation_proof_from_check_equivocation": { + header: defaultHeader, + buildVerifier: func(t *testing.T) *verifier { + ctrl := gomock.NewController(t) + + mockBlockState := NewMockBlockState(ctrl) + mockBlockState.EXPECT().GenesisHash().Return(common.Hash([32]byte{})) + + expectedAuthorityId := types.AuthorityID(kp.Public().Encode()) + mockSlotState := NewMockSlotState(ctrl) + mockSlotState. + EXPECT(). + CheckEquivocation(gomock.Any(), uint64(1), + defaultHeader, expectedAuthorityId). + Return(nil, nil) + + return &verifier{ + authorities: []types.Authority{ + { + Key: kp.Public(), + Weight: 1, + }, + }, + blockState: mockBlockState, + slotState: mockSlotState, + slotDuration: 6 * time.Second, + } + }, }, - { - name: "submitting equivocation fails", - verifier: *verifier6, - header: testHeader6, - equivocated: true, - expErr: errors.New("submitting and reporting equivocation: getting runtime: test error"), + "got_equivocation_proof": { + expected: true, + header: defaultHeader, + buildVerifier: func(t *testing.T) *verifier { + ctrl := gomock.NewController(t) + + secondHeader := types.NewEmptyHeader() + secondHeader.Number = 1 + secondHeader.Hash() + + expectedAuthorityId := types.AuthorityID(kp.Public().Encode()) + + mockedEquivocationProof := &types.BabeEquivocationProof{ + Offender: expectedAuthorityId, + Slot: 1, + FirstHeader: *defaultHeader, + SecondHeader: *secondHeader, + } + + mockSlotState := NewMockSlotState(ctrl) + mockSlotState. + EXPECT(). + CheckEquivocation(gomock.Any(), uint64(1), + defaultHeader, expectedAuthorityId). + Return(mockedEquivocationProof, nil) + + mockBlockState := NewMockBlockState(ctrl) + mockBlockState.EXPECT().GenesisHash().Return(common.Hash([32]byte{})) + mockBlockState.EXPECT().BestBlockHash().Return(defaultHeader.Hash()) + + opaqueOwnershipProof := []byte{0, 1, 2, 3, 4, 5} // random data + + mockRuntimeInstance := mocks.NewMockInstance(ctrl) + mockRuntimeInstance.EXPECT(). + BabeGenerateKeyOwnershipProof(uint64(1), expectedAuthorityId). + Return(types.OpaqueKeyOwnershipProof(opaqueOwnershipProof), nil) + + mockRuntimeInstance.EXPECT(). + BabeSubmitReportEquivocationUnsignedExtrinsic(*mockedEquivocationProof, opaqueOwnershipProof). + Return(nil) + + mockBlockState.EXPECT().GetRuntime(defaultHeader.Hash()).Return(mockRuntimeInstance, nil) + + return &verifier{ + authorities: []types.Authority{ + { + Key: kp.Public(), + Weight: 1, + }, + }, + blockState: mockBlockState, + slotState: mockSlotState, + slotDuration: 6 * time.Second, + } + }, + }, + "failed_to_get_runtime_while_submiting_equivocation": { + header: defaultHeader, + wantErr: getRuntimeErr, + errString: "submiting equivocation: getting runtime: mock get runtime error", + buildVerifier: func(t *testing.T) *verifier { + ctrl := gomock.NewController(t) + + secondHeader := types.NewEmptyHeader() + secondHeader.Number = 1 + secondHeader.Hash() + + expectedAuthorityId := types.AuthorityID(kp.Public().Encode()) + + mockedEquivocationProof := &types.BabeEquivocationProof{ + Offender: expectedAuthorityId, + Slot: 1, + FirstHeader: *defaultHeader, + SecondHeader: *secondHeader, + } + + mockSlotState := NewMockSlotState(ctrl) + mockSlotState. + EXPECT(). + CheckEquivocation(gomock.Any(), uint64(1), + defaultHeader, expectedAuthorityId). + Return(mockedEquivocationProof, nil) + + mockBlockState := NewMockBlockState(ctrl) + mockBlockState.EXPECT().GenesisHash().Return(common.Hash([32]byte{})) + mockBlockState.EXPECT().BestBlockHash().Return(defaultHeader.Hash()) + mockBlockState.EXPECT().GetRuntime(defaultHeader.Hash()).Return(nil, getRuntimeErr) + + return &verifier{ + authorities: []types.Authority{ + { + Key: kp.Public(), + Weight: 1, + }, + }, + blockState: mockBlockState, + slotState: mockSlotState, + slotDuration: 6 * time.Second, + } + }, }, } - for _, tt := range tests { + for tname, tt := range cases { tt := tt - t.Run(tt.name, func(t *testing.T) { - t.Parallel() + t.Run(tname, func(t *testing.T) { + //t.Parallel() - equivocated, err := tt.verifier.verifyBlockEquivocation(tt.header) - assert.Equal(t, equivocated, tt.equivocated) - if tt.expErr != nil { - assert.EqualError(t, err, tt.expErr.Error()) - } else { - assert.NoError(t, err) + verifier := tt.buildVerifier(t) + out, err := verifier.verifyBlockEquivocation(tt.header) + require.ErrorIs(t, err, tt.wantErr) + if tt.errString != "" { + require.EqualError(t, err, tt.errString) } + + require.Equal(t, out, tt.expected) }) } } @@ -872,7 +946,9 @@ func Test_verifier_submitAndReportEquivocation(t *testing.T) { ctrl := gomock.NewController(t) mockBlockState := NewMockBlockState(ctrl) - verifier := newVerifier(mockBlockState, 1, vi) + mockSlotState := NewMockSlotState(nil) + + verifier := newVerifier(mockBlockState, mockSlotState, 1, vi, testSlotDuration) const slot = uint64(1) const authorityIndex = uint32(1) @@ -923,13 +999,13 @@ func Test_verifier_submitAndReportEquivocation(t *testing.T) { mockBlockState.EXPECT().BestBlockHash().Return(firstHash).Times(2) mockBlockState.EXPECT().GetRuntime(firstHash).Return(mockRuntime, nil) - err = verifier.submitAndReportEquivocation(slot, authorityIndex, *firstHeader, *secondHeader) + err = verifier.submitAndReportEquivocation(&equivocationProof) assert.NoError(t, err) // fails on not being able to get a runtime mockBlockState.EXPECT().GetRuntime(firstHash).Return(nil, errors.New("test error")) - err = verifier.submitAndReportEquivocation(slot, authorityIndex, *firstHeader, *secondHeader) + err = verifier.submitAndReportEquivocation(&equivocationProof) assert.EqualError(t, err, "getting runtime: test error") } @@ -937,167 +1013,223 @@ func Test_verifier_submitAndReportEquivocation(t *testing.T) { func Test_verifier_verifyAuthorshipRightEquivocatory(t *testing.T) { ctrl := gomock.NewController(t) - mockBlockStateEquiv1 := NewMockBlockState(ctrl) - mockBlockStateEquiv2 := NewMockBlockState(ctrl) - mockBlockStateEquiv3 := NewMockBlockState(ctrl) - - //Generate keys kp, err := sr25519.GenerateKeypair() assert.NoError(t, err) - output, proof, err := kp.VrfSign(makeTranscript(Randomness{}, uint64(1), 1)) - assert.NoError(t, err) - - output2, proof2, err := kp.VrfSign(makeTranscript(Randomness{}, uint64(1), 2)) - assert.NoError(t, err) - secondDigestExisting := types.BabePrimaryPreDigest{ - AuthorityIndex: 1, - SlotNumber: 1, - VRFOutput: output2, - VRFProof: proof2, - } - prdExisting, err := secondDigestExisting.ToPreRuntimeDigest() - assert.NoError(t, err) - - headerExisting := newTestHeader(t, *prdExisting) - headerExisting.Hash() - hashExisting := encodeAndHashHeader(t, headerExisting) - signAndAddSeal(t, kp, headerExisting, hashExisting[:]) - - testBabeSecondaryPlainPreDigest := types.BabeSecondaryPlainPreDigest{ - AuthorityIndex: 1, - SlotNumber: 1, - } - testBabeSecondaryVRFPreDigest := types.BabeSecondaryVRFPreDigest{ - AuthorityIndex: 1, - SlotNumber: 1, - VrfOutput: output, - VrfProof: proof, - } - - // BabePrimaryPreDigest case - secDigest1 := types.BabePrimaryPreDigest{ - AuthorityIndex: 1, - SlotNumber: 1, - VRFOutput: output, - VRFProof: proof, - } - prd1, err := secDigest1.ToPreRuntimeDigest() - assert.NoError(t, err) - - auth := types.NewAuthority(kp.Public(), uint64(1)) - vi := &verifierInfo{ - authorities: []types.Authority{*auth, *auth}, - threshold: scale.MaxUint128, - } - - verifierEquivocatoryPrimary := newVerifier(mockBlockStateEquiv1, 1, vi) - - headerEquivocatoryPrimary := newTestHeader(t, *prd1) - hashEquivocatoryPrimary := encodeAndHashHeader(t, headerEquivocatoryPrimary) - signAndAddSeal(t, kp, headerEquivocatoryPrimary, hashEquivocatoryPrimary[:]) - headerEquivocatoryPrimary.Hash() - - mockBlockStateEquiv1.EXPECT().GetHeader(hashEquivocatoryPrimary).Return(headerEquivocatoryPrimary, nil) - mockBlockStateEquiv1.EXPECT().GetBlockHashesBySlot(uint64(1)).Return( - []common.Hash{hashEquivocatoryPrimary, hashExisting}, nil) - mockBlockStateEquiv1.EXPECT().BestBlockHash().Return(hashExisting) - - const slot = uint64(1) - const authorityIndex = uint32(1) - offenderPublicKey := types.AuthorityID(verifierEquivocatoryPrimary.authorities[authorityIndex].ToRaw().Key) - keyOwnershipProof := testKeyOwnershipProof - mockRuntime := mocks.NewMockInstance(gomock.NewController(t)) - - mockRuntime.EXPECT().BabeGenerateKeyOwnershipProof(slot, offenderPublicKey).Return(keyOwnershipProof, nil).Times(3) - // equivocationProof changes inside verifyAuthorshipRight, so we can't keep the current value. - mockRuntime.EXPECT().BabeSubmitReportEquivocationUnsignedExtrinsic( - gomock.AssignableToTypeOf(types.BabeEquivocationProof{}), keyOwnershipProof, - ).Return(nil).Times(3) - - mockBlockStateEquiv1.EXPECT().GetRuntime(hashExisting).Return(mockRuntime, nil) + tests := []struct { + name string + setupVerifier func(t *testing.T, header *types.Header) *verifier + setupHeader func(t *testing.T) *types.Header + expErr func(*types.Header) error + }{ + { + name: "equivocate_primary", + setupHeader: func(t *testing.T) *types.Header { + output, proof, err := kp.VrfSign(makeTranscript(Randomness{}, uint64(1), 1)) + assert.NoError(t, err) - // Secondary Plain Test Header - testParentPrd, err := testBabeSecondaryPlainPreDigest.ToPreRuntimeDigest() - assert.NoError(t, err) - testParentHeader := newTestHeader(t, *testParentPrd) + // BabePrimaryPreDigest case + primaryDigest := types.BabePrimaryPreDigest{ + AuthorityIndex: 1, + SlotNumber: 1, + VRFOutput: output, + VRFProof: proof, + } + primaryDigestRuntime, err := primaryDigest.ToPreRuntimeDigest() + assert.NoError(t, err) - testParentHash := encodeAndHashHeader(t, testParentHeader) - testSecondaryPrd, err := testBabeSecondaryPlainPreDigest.ToPreRuntimeDigest() - assert.NoError(t, err) - testSecPlainHeader := newTestHeader(t, *testSecondaryPrd) - testSecPlainHeader.ParentHash = testParentHash + header := newTestHeader(t, *primaryDigestRuntime) + header.Hash() + headerHash := encodeAndHashHeader(t, header) + signAndAddSeal(t, kp, header, headerHash[:]) - babeSecPlainPrd2, err := testBabeSecondaryPlainPreDigest.ToPreRuntimeDigest() - assert.NoError(t, err) - headerEquivocatorySecondaryPlain := newTestHeader(t, *babeSecPlainPrd2) + return header + }, + setupVerifier: func(t *testing.T, header *types.Header) *verifier { + const slot = uint64(1) + offenderPublicKey := types.AuthorityID(kp.Public().Encode()) + + equivocationProof := &types.BabeEquivocationProof{ + Offender: offenderPublicKey, + Slot: slot, + FirstHeader: *header, + SecondHeader: *types.NewEmptyHeader(), + } + + mockSlotState := NewMockSlotState(ctrl) + mockSlotState.EXPECT(). + CheckEquivocation(gomock.Any(), slot, header, offenderPublicKey). + Return(equivocationProof, nil) + + mockBlockState := NewMockBlockState(ctrl) + mockBlockState.EXPECT().GenesisHash().Return(common.Hash([32]byte{})) + mockBlockState.EXPECT().BestBlockHash().Return(header.Hash()) + + mockRuntime := mocks.NewMockInstance(gomock.NewController(t)) + mockRuntime.EXPECT(). + BabeGenerateKeyOwnershipProof(slot, offenderPublicKey). + Return(testKeyOwnershipProof, nil) + + mockRuntime.EXPECT(). + BabeSubmitReportEquivocationUnsignedExtrinsic( + *equivocationProof, testKeyOwnershipProof). + Return(nil) + + mockBlockState.EXPECT().GetRuntime(header.Hash()).Return(mockRuntime, nil) + auth := types.NewAuthority(kp.Public(), uint64(1)) + info := &verifierInfo{ + authorities: []types.Authority{*auth, *auth}, + threshold: scale.MaxUint128, + } + + return newVerifier(mockBlockState, mockSlotState, 1, info, testSlotDuration) + }, + expErr: func(h *types.Header) error { + return fmt.Errorf("%w for block header %s", ErrProducerEquivocated, h.Hash()) + }, + }, + { + name: "equivocate_secondary_plain", + setupHeader: func(t *testing.T) *types.Header { + babeSecondaryPlainPreDigest := types.BabeSecondaryPlainPreDigest{ + AuthorityIndex: 1, + SlotNumber: 1, + } + + babeSecPlainPrd2, err := babeSecondaryPlainPreDigest.ToPreRuntimeDigest() + assert.NoError(t, err) + header := newTestHeader(t, *babeSecPlainPrd2) + header.Hash() - hashEquivocatorySecondaryPlain := encodeAndHashHeader(t, headerEquivocatorySecondaryPlain) - signAndAddSeal(t, kp, headerEquivocatorySecondaryPlain, hashEquivocatorySecondaryPlain[:]) - babeVerifier8 := newTestVerifier(kp, mockBlockStateEquiv2, scale.MaxUint128, true) + hashEquivocatorySecondaryPlain := encodeAndHashHeader(t, header) + signAndAddSeal(t, kp, header, hashEquivocatorySecondaryPlain[:]) - mockBlockStateEquiv2.EXPECT().GetHeader(hashEquivocatorySecondaryPlain).Return(headerEquivocatorySecondaryPlain, nil) - mockBlockStateEquiv2.EXPECT().GetBlockHashesBySlot(uint64(1)).Return( - []common.Hash{hashEquivocatorySecondaryPlain, hashExisting}, nil) - mockBlockStateEquiv2.EXPECT().BestBlockHash().Return(hashExisting) - mockBlockStateEquiv2.EXPECT().GetRuntime(hashExisting).Return(mockRuntime, nil) + return header + }, + setupVerifier: func(t *testing.T, header *types.Header) *verifier { + const slot = uint64(1) + offenderPublicKey := types.AuthorityID(kp.Public().Encode()) + + equivocationProof := &types.BabeEquivocationProof{ + Offender: offenderPublicKey, + Slot: slot, + FirstHeader: *header, + SecondHeader: *types.NewEmptyHeader(), + } + + mockSlotState := NewMockSlotState(ctrl) + mockSlotState.EXPECT(). + CheckEquivocation(gomock.Any(), slot, header, offenderPublicKey). + Return(equivocationProof, nil) + + mockBlockState := NewMockBlockState(ctrl) + mockBlockState.EXPECT().GenesisHash().Return(common.Hash([32]byte{})) + mockBlockState.EXPECT().BestBlockHash().Return(header.Hash()) + + mockRuntime := mocks.NewMockInstance(gomock.NewController(t)) + mockRuntime.EXPECT(). + BabeGenerateKeyOwnershipProof(slot, offenderPublicKey). + Return(testKeyOwnershipProof, nil) + + mockRuntime.EXPECT(). + BabeSubmitReportEquivocationUnsignedExtrinsic( + *equivocationProof, testKeyOwnershipProof). + Return(nil) + + mockBlockState.EXPECT().GetRuntime(header.Hash()).Return(mockRuntime, nil) + auth := types.NewAuthority(kp.Public(), uint64(1)) + info := &verifierInfo{ + authorities: []types.Authority{*auth, *auth}, + threshold: scale.MaxUint128, + secondarySlots: true, + randomness: Randomness{}, + } + + return newVerifier(mockBlockState, mockSlotState, 1, info, testSlotDuration) + }, + expErr: func(h *types.Header) error { + return fmt.Errorf("%w for block header %s", ErrProducerEquivocated, h.Hash()) + }, + }, + { + name: "equivocate_secondary_vrf", + expErr: func(h *types.Header) error { + return fmt.Errorf("%w for block header %s", ErrProducerEquivocated, h.Hash()) + }, + setupHeader: func(t *testing.T) *types.Header { + output, proof, err := kp.VrfSign(makeTranscript(Randomness{}, uint64(1), 1)) + assert.NoError(t, err) - // Secondary Vrf Test Header - encParentVrfDigest := newEncodedBabeDigest(t, testBabeSecondaryVRFPreDigest) - testParentVrfHeader := newTestHeader(t, *types.NewBABEPreRuntimeDigest(encParentVrfDigest)) + babeSecondaryVRFPreDigest := types.BabeSecondaryVRFPreDigest{ + AuthorityIndex: 1, + SlotNumber: 1, + VrfOutput: output, + VrfProof: proof, + } - testVrfParentHash := encodeAndHashHeader(t, testParentVrfHeader) - encVrfHeader := newEncodedBabeDigest(t, testBabeSecondaryVRFPreDigest) - testSecVrfHeader := newTestHeader(t, *types.NewBABEPreRuntimeDigest(encVrfHeader)) - testSecVrfHeader.ParentHash = testVrfParentHash - encVrfDigest := newEncodedBabeDigest(t, testBabeSecondaryVRFPreDigest) - assert.NoError(t, err) - headerEquivocatorySecondaryVRF := newTestHeader(t, *types.NewBABEPreRuntimeDigest(encVrfDigest)) - hashEquivocatorySecondaryVRF := encodeAndHashHeader(t, headerEquivocatorySecondaryVRF) - signAndAddSeal(t, kp, headerEquivocatorySecondaryVRF, hashEquivocatorySecondaryVRF[:]) - babeVerifierEquivocatorySecondaryVRF := newTestVerifier(kp, mockBlockStateEquiv3, scale.MaxUint128, true) + vrfDigest := newEncodedBabeDigest(t, babeSecondaryVRFPreDigest) + header := newTestHeader(t, *types.NewBABEPreRuntimeDigest(vrfDigest)) + header.Hash() - mockBlockStateEquiv3.EXPECT().GetHeader(hashEquivocatorySecondaryVRF).Return(headerEquivocatorySecondaryVRF, nil) - mockBlockStateEquiv3.EXPECT().GetBlockHashesBySlot(uint64(1)).Return( - []common.Hash{hashEquivocatorySecondaryVRF, hashExisting}, nil) - mockBlockStateEquiv3.EXPECT().BestBlockHash().Return(hashExisting) - mockBlockStateEquiv3.EXPECT().GetRuntime(hashExisting).Return(mockRuntime, nil) + hashEquivocatorySecondaryVRF := encodeAndHashHeader(t, header) + signAndAddSeal(t, kp, header, hashEquivocatorySecondaryVRF[:]) - tests := []struct { - name string - verifier verifier - header *types.Header - expErr error - }{ - { - name: "equivocate - primary", - verifier: *verifierEquivocatoryPrimary, - header: headerEquivocatoryPrimary, - expErr: fmt.Errorf("%w for block header %s", ErrProducerEquivocated, headerEquivocatoryPrimary.Hash()), - }, - { - name: "equivocate - secondary plain", - verifier: *babeVerifier8, - header: headerEquivocatorySecondaryPlain, - expErr: fmt.Errorf("%w for block header %s", ErrProducerEquivocated, headerEquivocatorySecondaryPlain.Hash()), - }, - { - name: "equivocate - secondary vrf", - verifier: *babeVerifierEquivocatorySecondaryVRF, - header: headerEquivocatorySecondaryVRF, - expErr: fmt.Errorf("%w for block header %s", ErrProducerEquivocated, headerEquivocatorySecondaryVRF.Hash()), + return header + }, + setupVerifier: func(t *testing.T, header *types.Header) *verifier { + const slot = uint64(1) + offenderPublicKey := types.AuthorityID(kp.Public().Encode()) + + equivocationProof := &types.BabeEquivocationProof{ + Offender: offenderPublicKey, + Slot: slot, + FirstHeader: *header, + SecondHeader: *types.NewEmptyHeader(), + } + + mockSlotState := NewMockSlotState(ctrl) + mockSlotState.EXPECT(). + CheckEquivocation(gomock.Any(), slot, header, offenderPublicKey). + Return(equivocationProof, nil) + + mockBlockState := NewMockBlockState(ctrl) + mockBlockState.EXPECT().GenesisHash().Return(common.Hash([32]byte{})) + mockBlockState.EXPECT().BestBlockHash().Return(header.Hash()) + + mockRuntime := mocks.NewMockInstance(gomock.NewController(t)) + mockRuntime.EXPECT(). + BabeGenerateKeyOwnershipProof(slot, offenderPublicKey). + Return(testKeyOwnershipProof, nil) + + mockRuntime.EXPECT(). + BabeSubmitReportEquivocationUnsignedExtrinsic( + *equivocationProof, testKeyOwnershipProof). + Return(nil) + + mockBlockState.EXPECT().GetRuntime(header.Hash()).Return(mockRuntime, nil) + + auth := types.NewAuthority(kp.Public(), uint64(1)) + info := &verifierInfo{ + authorities: []types.Authority{*auth, *auth}, + threshold: scale.MaxUint128, + secondarySlots: true, + randomness: Randomness{}, + } + + return newVerifier(mockBlockState, mockSlotState, 1, info, testSlotDuration) + }, }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - b := &tt.verifier - err := b.verifyAuthorshipRight(tt.header) + argHeader := tt.setupHeader(t) + verifier := tt.setupVerifier(t, argHeader) + err := verifier.verifyAuthorshipRight(argHeader) if tt.expErr != nil { - assert.EqualError(t, err, tt.expErr.Error()) - } else { - assert.NoError(t, err) + assert.EqualError(t, err, tt.expErr(argHeader).Error()) + return } - + assert.NoError(t, err) }) } } @@ -1234,6 +1366,7 @@ func TestVerificationManager_VerifyBlock(t *testing.T) { mockEpochStateGetVerifierInfoErr.EXPECT().SkipVerify(testBlockHeaderEmpty).Return(false, nil) mockEpochStateVerifyAuthorshipErr.EXPECT().GetEpochForBlock(testBlockHeaderEmpty).Return(uint64(1), nil) + mockEpochStateVerifyAuthorshipErr.EXPECT().GetSlotDuration().Return(6*time.Second, nil) block1Header := types.NewEmptyHeader() block1Header.Number = 1 @@ -1255,14 +1388,16 @@ func TestVerificationManager_VerifyBlock(t *testing.T) { secondarySlots: true, } - vm0 := NewVerificationManager(mockBlockStateCheckFinErr, mockEpochStateEmpty) - vm1 := NewVerificationManager(mockBlockStateNotFinal, mockEpochStateEmpty) - vm2 := NewVerificationManager(mockBlockStateNotFinal2, mockEpochStateSetSlotErr) - vm3 := NewVerificationManager(mockBlockStateNotFinal2, mockEpochStateGetEpochErr) - vm4 := NewVerificationManager(mockBlockStateEmpty, mockEpochStateSkipVerifyErr) - vm5 := NewVerificationManager(mockBlockStateEmpty, mockEpochStateSkipVerifyTrue) - vm6 := NewVerificationManager(mockBlockStateEmpty, mockEpochStateGetVerifierInfoErr) - vm8 := NewVerificationManager(mockBlockStateEmpty, mockEpochStateVerifyAuthorshipErr) + mockSlotState := NewMockSlotState(nil) + + vm0 := NewVerificationManager(mockBlockStateCheckFinErr, mockSlotState, mockEpochStateEmpty) + vm1 := NewVerificationManager(mockBlockStateNotFinal, mockSlotState, mockEpochStateEmpty) + vm2 := NewVerificationManager(mockBlockStateNotFinal2, mockSlotState, mockEpochStateSetSlotErr) + vm3 := NewVerificationManager(mockBlockStateNotFinal2, mockSlotState, mockEpochStateGetEpochErr) + vm4 := NewVerificationManager(mockBlockStateEmpty, mockSlotState, mockEpochStateSkipVerifyErr) + vm5 := NewVerificationManager(mockBlockStateEmpty, mockSlotState, mockEpochStateSkipVerifyTrue) + vm6 := NewVerificationManager(mockBlockStateEmpty, mockSlotState, mockEpochStateGetVerifierInfoErr) + vm8 := NewVerificationManager(mockBlockStateEmpty, mockSlotState, mockEpochStateVerifyAuthorshipErr) vm8.epochInfo[1] = info tests := []struct { @@ -1376,6 +1511,8 @@ func TestVerificationManager_SetOnDisabled(t *testing.T) { mockEpochStateOk3.EXPECT().GetEpochForBlock(testHeader).Return(uint64(2), nil) mockBlockStateOk.EXPECT().IsDescendantOf(gomock.Any(), gomock.Any()).Return(false, nil) + mockSlotState := NewMockSlotState(nil) + authority := types.NewAuthority(kp.Public(), uint64(1)) info := &verifierInfo{ authorities: []types.Authority{*authority, *authority}, @@ -1389,27 +1526,27 @@ func TestVerificationManager_SetOnDisabled(t *testing.T) { }, } - vm0 := NewVerificationManager(mockBlockStateEmpty, mockEpochStateGetEpochErr) - vm1 := NewVerificationManager(mockBlockStateEmpty, mockEpochStateGetEpochDataErr) + vm0 := NewVerificationManager(mockBlockStateEmpty, mockSlotState, mockEpochStateGetEpochErr) + vm1 := NewVerificationManager(mockBlockStateEmpty, mockSlotState, mockEpochStateGetEpochDataErr) vm1.epochInfo[1] = info - vm2 := NewVerificationManager(mockBlockStateEmpty, mockEpochStateIndexLenErr) + vm2 := NewVerificationManager(mockBlockStateEmpty, mockSlotState, mockEpochStateIndexLenErr) vm2.epochInfo[2] = info - vm3 := NewVerificationManager(mockBlockStateEmpty, mockEpochStateSetDisabledProd) + vm3 := NewVerificationManager(mockBlockStateEmpty, mockSlotState, mockEpochStateSetDisabledProd) vm3.epochInfo[2] = info - vm4 := NewVerificationManager(mockBlockStateIsDescendantErr, mockEpochStateOk) + vm4 := NewVerificationManager(mockBlockStateIsDescendantErr, mockSlotState, mockEpochStateOk) vm4.epochInfo[2] = info vm4.onDisabled[2] = map[uint32][]*onDisabledInfo{} vm4.onDisabled[2][0] = disabledInfo - vm5 := NewVerificationManager(mockBlockStateAuthorityDisabled, mockEpochStateOk2) + vm5 := NewVerificationManager(mockBlockStateAuthorityDisabled, mockSlotState, mockEpochStateOk2) vm5.epochInfo[2] = info vm5.onDisabled[2] = map[uint32][]*onDisabledInfo{} vm5.onDisabled[2][0] = disabledInfo - vm6 := NewVerificationManager(mockBlockStateOk, mockEpochStateOk3) + vm6 := NewVerificationManager(mockBlockStateOk, mockSlotState, mockEpochStateOk3) vm6.epochInfo[2] = info vm6.onDisabled[2] = map[uint32][]*onDisabledInfo{} vm6.onDisabled[2][0] = disabledInfo