From 5697e144a72cb5f3d5183d19e5d97d6e4c9156c1 Mon Sep 17 00:00:00 2001 From: Callum Waters Date: Wed, 10 Jun 2020 13:53:55 +0200 Subject: [PATCH] evidence: adr56 form amnesia evidence (#4821) Creates Amnesia Evidence which is formed from Potential Amnesia Evidence with either a matching proof or after a period of time denoted as the Amnesia Trial Period. This also adds the code necessary so that Amnesia Evidence can be validated and committed on a block --- consensus/common_test.go | 42 ++ consensus/state_test.go | 19 +- evidence/pool.go | 180 +++++- evidence/pool_test.go | 170 +++++- proto/types/evidence.pb.go | 512 +++++++++++++++--- proto/types/evidence.proto | 12 +- proto/types/params.pb.go | 109 ++-- proto/types/params.proto | 5 + state/store_test.go | 4 +- state/validation.go | 31 +- state/validation_test.go | 283 +++++++++- .../internal/test_harness_test.go | 3 +- types/evidence.go | 315 +++++++++-- types/evidence_test.go | 166 ++++-- types/params.go | 19 +- types/params_test.go | 31 +- types/priv_validator.go | 9 + types/validator_set.go | 3 + types/vote_set_test.go | 17 +- 19 files changed, 1656 insertions(+), 274 deletions(-) diff --git a/consensus/common_test.go b/consensus/common_test.go index 3f9378acb..54ee847ea 100644 --- a/consensus/common_test.go +++ b/consensus/common_test.go @@ -25,6 +25,7 @@ import ( abci "github.com/tendermint/tendermint/abci/types" cfg "github.com/tendermint/tendermint/config" cstypes "github.com/tendermint/tendermint/consensus/types" + "github.com/tendermint/tendermint/evidence" tmbytes "github.com/tendermint/tendermint/libs/bytes" "github.com/tendermint/tendermint/libs/log" tmos "github.com/tendermint/tendermint/libs/os" @@ -422,6 +423,47 @@ func randState(nValidators int) (*State, []*validatorStub) { return cs, vss } +func randStateWithEvpool(nValidators int) (*State, []*validatorStub, *evidence.Pool) { + state, privVals := randGenesisState(nValidators, false, 10) + + vss := make([]*validatorStub, nValidators) + + app := counter.NewApplication(true) + config := cfg.ResetTestRoot("consensus_state_test") + + blockStore := store.NewBlockStore(dbm.NewMemDB()) + evidenceDB := dbm.NewMemDB() + + mtx := new(sync.Mutex) + proxyAppConnMem := abcicli.NewLocalClient(mtx, app) + proxyAppConnCon := abcicli.NewLocalClient(mtx, app) + + mempool := mempl.NewCListMempool(config.Mempool, proxyAppConnMem, 0) + mempool.SetLogger(log.TestingLogger().With("module", "mempool")) + if config.Consensus.WaitForTxs() { + mempool.EnableTxsAvailable() + } + stateDB := dbm.NewMemDB() + evpool, _ := evidence.NewPool(stateDB, evidenceDB, blockStore) + blockExec := sm.NewBlockExecutor(stateDB, log.TestingLogger(), proxyAppConnCon, mempool, evpool) + cs := NewState(config.Consensus, state, blockExec, blockStore, mempool, evpool) + cs.SetLogger(log.TestingLogger().With("module", "consensus")) + cs.SetPrivValidator(privVals[0]) + + eventBus := types.NewEventBus() + eventBus.SetLogger(log.TestingLogger().With("module", "events")) + eventBus.Start() + cs.SetEventBus(eventBus) + + for i := 0; i < nValidators; i++ { + vss[i] = newValidatorStub(privVals[i], int32(i)) + } + // since cs1 starts at 1 + incrementHeight(vss[1:]...) + + return cs, vss, evpool +} + //------------------------------------------------------------------------------- func ensureNoNewEvent(ch <-chan tmpubsub.Message, timeout time.Duration, diff --git a/consensus/state_test.go b/consensus/state_test.go index 739060e81..52d235190 100644 --- a/consensus/state_test.go +++ b/consensus/state_test.go @@ -616,7 +616,7 @@ func TestStateLockPOLRelockThenChangeLock(t *testing.T) { // 4 vals, one precommits, other 3 polka at next round, so we unlock and precomit the polka func TestStateLockPOLUnlock(t *testing.T) { - cs1, vss := randState(4) + cs1, vss, evpool := randStateWithEvpool(4) vs2, vs3, vs4 := vss[1], vss[2], vss[3] height, round := cs1.Height, cs1.Round @@ -703,6 +703,16 @@ func TestStateLockPOLUnlock(t *testing.T) { signAddVotes(cs1, tmproto.PrecommitType, nil, types.PartSetHeader{}, vs2, vs3) ensureNewRound(newRoundCh, height, round+1) + // polc should be in the evpool for round 1 + polc, err := evpool.RetrievePOLC(height, round) + assert.NoError(t, err) + assert.False(t, polc.IsAbsent()) + t.Log(polc.Address()) + // but not for round 0 + polc, err = evpool.RetrievePOLC(height, round-1) + assert.Error(t, err) + assert.True(t, polc.IsAbsent()) + } // 4 vals, v1 locks on proposed block in the first round but the other validators only prevote @@ -710,7 +720,7 @@ func TestStateLockPOLUnlock(t *testing.T) { // v1 should unlock and precommit nil. In the third round another block is proposed, all vals // prevote and now v1 can lock onto the third block and precommit that func TestStateLockPOLUnlockOnUnknownBlock(t *testing.T) { - cs1, vss := randState(4) + cs1, vss, evpool := randStateWithEvpool(4) vs2, vs3, vs4 := vss[1], vss[2], vss[3] height, round := cs1.Height, cs1.Round @@ -803,6 +813,11 @@ func TestStateLockPOLUnlockOnUnknownBlock(t *testing.T) { thirdPropBlockHash := propBlock.Hash() require.NotEqual(t, secondBlockHash, thirdPropBlockHash) + // polc should be in the evpool for round 1 + polc, err := evpool.RetrievePOLC(height, round) + assert.NoError(t, err) + assert.False(t, polc.IsAbsent()) + incrementRound(vs2, vs3, vs4) // timeout to new round diff --git a/evidence/pool.go b/evidence/pool.go index 35c8a0b05..4e2a4f53d 100644 --- a/evidence/pool.go +++ b/evidence/pool.go @@ -21,6 +21,7 @@ const ( baseKeyCommitted = byte(0x00) baseKeyPending = byte(0x01) baseKeyPOLC = byte(0x02) + baseKeyAwaiting = byte(0x03) ) // Pool maintains a pool of valid evidence to be broadcasted and committed @@ -43,6 +44,8 @@ type Pool struct { // currently is (ie. [MaxAgeNumBlocks, CurrentHeight]) // In simple words, it means it's still bonded -> therefore slashable. valToLastHeight valToLastHeightMap + + nextEvidenceTrialEndedHeight int64 } // Validator.Address -> Last height it was in validator set @@ -59,22 +62,19 @@ func NewPool(stateDB, evidenceDB dbm.DB, blockStore *store.BlockStore) (*Pool, e } pool := &Pool{ - stateDB: stateDB, - blockStore: blockStore, - state: state, - logger: log.NewNopLogger(), - evidenceStore: evidenceDB, - evidenceList: clist.New(), - valToLastHeight: valToLastHeight, + stateDB: stateDB, + blockStore: blockStore, + state: state, + logger: log.NewNopLogger(), + evidenceStore: evidenceDB, + evidenceList: clist.New(), + valToLastHeight: valToLastHeight, + nextEvidenceTrialEndedHeight: -1, } // if pending evidence already in db, in event of prior failure, then load it back to the evidenceList evList := pool.AllPendingEvidence() for _, ev := range evList { - if pool.IsEvidenceExpired(ev) { - pool.removePendingEvidence(ev) - continue - } pool.evidenceList.PushBack(ev) } @@ -84,6 +84,7 @@ func NewPool(stateDB, evidenceDB dbm.DB, blockStore *store.BlockStore) (*Pool, e // PendingEvidence is used primarily as part of block proposal and returns up to maxNum of uncommitted evidence. // If maxNum is -1, all evidence is returned. Pending evidence is prioritised based on time. func (evpool *Pool) PendingEvidence(maxNum uint32) []types.Evidence { + evpool.removeExpiredPendingEvidence() evidence, err := evpool.listEvidence(baseKeyPending, int64(maxNum)) if err != nil { evpool.logger.Error("Unable to retrieve pending evidence", "err", err) @@ -92,6 +93,7 @@ func (evpool *Pool) PendingEvidence(maxNum uint32) []types.Evidence { } func (evpool *Pool) AllPendingEvidence() []types.Evidence { + evpool.removeExpiredPendingEvidence() evidence, err := evpool.listEvidence(baseKeyPending, -1) if err != nil { evpool.logger.Error("Unable to retrieve pending evidence", "err", err) @@ -104,23 +106,24 @@ func (evpool *Pool) AllPendingEvidence() []types.Evidence { func (evpool *Pool) Update(block *types.Block, state sm.State) { // sanity check if state.LastBlockHeight != block.Height { - panic( - fmt.Sprintf("Failed EvidencePool.Update sanity check: got state.Height=%d with block.Height=%d", - state.LastBlockHeight, - block.Height, - ), + panic(fmt.Sprintf("Failed EvidencePool.Update sanity check: got state.Height=%d with block.Height=%d", + state.LastBlockHeight, + block.Height, + ), ) } // remove evidence from pending and mark committed - evpool.MarkEvidenceAsCommitted(block.Height, block.Time, block.Evidence.Evidence) + evpool.MarkEvidenceAsCommitted(block.Height, block.Evidence.Evidence) - // remove expired evidence - this should be done at every height to ensure we don't send expired evidence to peers - evpool.removeExpiredPendingEvidence() - - // as it's not vital to remove expired POLCs, we only prune periodically + // prune pending, committed and potential evidence and polc's periodically if block.Height%state.ConsensusParams.Evidence.MaxAgeNumBlocks == 0 { evpool.pruneExpiredPOLC() + evpool.removeExpiredPendingEvidence() + } + + if evpool.nextEvidenceTrialEndedHeight > 0 && block.Height < evpool.nextEvidenceTrialEndedHeight { + evpool.upgradePotentialAmnesiaEvidence() } // update the state @@ -202,9 +205,74 @@ func (evpool *Pool) AddEvidence(evidence types.Evidence) error { return fmt.Errorf("failed to verify %v: %w", ev, err) } + // For potential amnesia evidence, if this node is indicted it shall retrieve a polc + // to form AmensiaEvidence + if pe, ok := ev.(types.PotentialAmnesiaEvidence); ok { + var ( + height = pe.Height() + exists = false + polc types.ProofOfLockChange + ) + pe.HeightStamp = evpool.State().LastBlockHeight + + // a) first try to find a corresponding polc + for round := pe.VoteB.Round; round > pe.VoteA.Round; round-- { + polc, err = evpool.RetrievePOLC(height, round) + if err != nil { + evpool.logger.Error("Failed to retrieve polc for potential amnesia evidence", "err", err, "pae", pe.String()) + continue + } + if err == nil && !polc.IsAbsent() { + // we should not need to verify it if both the polc and potential amnesia evidence have already + // been verified. We replace the potential amnesia evidence. + ae := types.MakeAmnesiaEvidence(pe, polc) + err := evpool.AddEvidence(ae) + if err != nil { + evpool.logger.Error("Failed to create amnesia evidence from potential amnesia evidence", "err", err) + // revert back to processing potential amnesia evidence + exists = false + } else { + evpool.logger.Info("Formed amnesia evidence from own polc", "amnesiaEvidence", ae) + } + break + } + } + + // b) check if amnesia evidence can be made now or if we need to enact the trial period + if !exists && pe.Primed(1, pe.HeightStamp) { + err := evpool.AddEvidence(types.MakeAmnesiaEvidence(pe, types.EmptyPOLC())) + if err != nil { + return err + } + } else if !exists && evpool.State().LastBlockHeight+evpool.State().ConsensusParams.Evidence.ProofTrialPeriod < + pe.Height()+evpool.State().ConsensusParams.Evidence.MaxAgeNumBlocks { + // if we can't find a proof of lock change and we know that the trial period will finish before the + // evidence has expired, then we commence the trial period by saving it in the awaiting bucket + pbe, err := types.EvidenceToProto(pe) + if err != nil { + return err + } + evBytes, err := pbe.Marshal() + if err != nil { + return err + } + key := keyAwaiting(pe) + err = evpool.evidenceStore.Set(key, evBytes) + if err != nil { + return err + } + // keep track of when the next pe has finished the trial period + if evpool.nextEvidenceTrialEndedHeight == -1 { + evpool.nextEvidenceTrialEndedHeight = ev.Height() + evpool.State().ConsensusParams.Evidence.ProofTrialPeriod + } + } + // we don't need to do anymore processing so we can move on to the next piece of evidence + continue + } + // 2) Save to store. if err := evpool.addPendingEvidence(ev); err != nil { - return fmt.Errorf("database error: %v", err) + return fmt.Errorf("database error when adding evidence: %v", err) } // 3) Add evidence to clist. @@ -218,7 +286,7 @@ func (evpool *Pool) AddEvidence(evidence types.Evidence) error { // MarkEvidenceAsCommitted marks all the evidence as committed and removes it // from the queue. -func (evpool *Pool) MarkEvidenceAsCommitted(height int64, lastBlockTime time.Time, evidence []types.Evidence) { +func (evpool *Pool) MarkEvidenceAsCommitted(height int64, evidence []types.Evidence) { // make a map of committed evidence to remove from the clist blockEvidenceMap := make(map[string]struct{}) for _, ev := range evidence { @@ -291,17 +359,19 @@ func (evpool *Pool) IsPending(evidence types.Evidence) bool { return ok } -// RetrievePOLC attempts to find a polc at the given height and round, if not there it returns an error +// RetrievePOLC attempts to find a polc at the given height and round, if not there than exist returns false, all +// database errors are automatically logged func (evpool *Pool) RetrievePOLC(height int64, round int32) (polc types.ProofOfLockChange, err error) { var pbpolc tmproto.ProofOfLockChange key := keyPOLCFromHeightAndRound(height, round) polcBytes, err := evpool.evidenceStore.Get(key) if err != nil { + evpool.logger.Error("Unable to retrieve polc", "err", err) return polc, err } if polcBytes == nil { - return polc, fmt.Errorf("unable to find polc at height %d and round %d", height, round) + return polc, fmt.Errorf("nil value in database for key: %s", key) } err = proto.Unmarshal(polcBytes, &pbpolc) @@ -366,7 +436,7 @@ func (evpool *Pool) State() sm.State { func (evpool *Pool) addPendingEvidence(evidence types.Evidence) error { evi, err := types.EvidenceToProto(evidence) if err != nil { - return err + return fmt.Errorf("unable to convert to proto, err: %w", err) } evBytes, err := proto.Marshal(evi) @@ -399,13 +469,12 @@ func (evpool *Pool) listEvidence(prefixKey byte, maxNum int64) ([]types.Evidence } defer iter.Close() for ; iter.Valid(); iter.Next() { - val := iter.Value() - if count == maxNum { return evidence, nil } count++ + val := iter.Value() var ( ev types.Evidence evpb tmproto.Evidence @@ -511,6 +580,57 @@ func (evpool *Pool) pruneExpiredPOLC() { } } +// upgrades any potential evidence that has undergone the trial period and is primed to be made into +// amnesia evidence +func (evpool *Pool) upgradePotentialAmnesiaEvidence() int64 { + iter, err := dbm.IteratePrefix(evpool.evidenceStore, []byte{baseKeyAwaiting}) + if err != nil { + evpool.logger.Error("Unable to iterate over POLC's", "err", err) + return -1 + } + defer iter.Close() + trialPeriod := evpool.State().ConsensusParams.Evidence.ProofTrialPeriod + // 1) Iterate through all potential amnesia evidence in order of height + for ; iter.Valid(); iter.Next() { + paeBytes := iter.Value() + // 2) Retrieve the evidence + var evpb tmproto.Evidence + err := evpb.Unmarshal(paeBytes) + if err != nil { + evpool.logger.Error("Unable to unmarshal potential amnesia evidence", "err", err) + continue + } + ev, err := types.EvidenceFromProto(&evpb) + if err != nil { + evpool.logger.Error("coverting to evidence from proto", "err", err) + continue + } + // 3) Check if the trial period has lapsed and amnesia evidence can be formed + if pe, ok := ev.(*types.PotentialAmnesiaEvidence); ok { + if pe.Primed(trialPeriod, evpool.State().LastBlockHeight) { + ae := types.MakeAmnesiaEvidence(*pe, types.EmptyPOLC()) + err := evpool.AddEvidence(ae) + if err != nil { + evpool.logger.Error("Unable to add amnesia evidence", "err", err) + continue + } + err = evpool.evidenceStore.Delete(iter.Key()) + if err != nil { + evpool.logger.Error("Unable to delete potential amnesia evidence", "err", err) + continue + } + } else { + evpool.logger.Debug("Potential amnesia evidence not ready to be upgraded. Ready at height", "height", + pe.HeightStamp+trialPeriod) + // once we reach a piece of evidence that isn't ready send back the height with which it will be ready + return pe.HeightStamp + trialPeriod + } + } + } + // if we have no evidence left to process we want to reset nextEvidenceTrialEndedHeight + return -1 +} + func evMapKey(ev types.Evidence) string { return string(ev.Hash()) } @@ -604,6 +724,10 @@ func keyPending(evidence types.Evidence) []byte { return append([]byte{baseKeyPending}, keySuffix(evidence)...) } +func keyAwaiting(evidence types.Evidence) []byte { + return append([]byte{baseKeyAwaiting}, keySuffix(evidence)...) +} + func keyPOLC(polc types.ProofOfLockChange) []byte { return keyPOLCFromHeightAndRound(polc.Height(), polc.Round()) } diff --git a/evidence/pool_test.go b/evidence/pool_test.go index 5aa56e28d..666b4e8e8 100644 --- a/evidence/pool_test.go +++ b/evidence/pool_test.go @@ -12,7 +12,8 @@ import ( dbm "github.com/tendermint/tm-db" "github.com/tendermint/tendermint/crypto" - "github.com/tendermint/tendermint/crypto/ed25519" + "github.com/tendermint/tendermint/libs/bytes" + "github.com/tendermint/tendermint/libs/log" tmrand "github.com/tendermint/tendermint/libs/rand" tmproto "github.com/tendermint/tendermint/proto/types" sm "github.com/tendermint/tendermint/state" @@ -27,6 +28,8 @@ func TestMain(m *testing.M) { os.Exit(code) } +const evidenceChainID = "test_chain" + func TestEvidencePool(t *testing.T) { var ( valAddr = tmrand.Bytes(crypto.AddressSize) @@ -78,14 +81,13 @@ func TestEvidencePool(t *testing.T) { func TestProposingAndCommittingEvidence(t *testing.T) { var ( - valAddr = tmrand.Bytes(crypto.AddressSize) - height = int64(1) - lastBlockTime = time.Now() - stateDB = initializeValidatorState(valAddr, height) - evidenceDB = dbm.NewMemDB() - blockStoreDB = dbm.NewMemDB() - blockStore = initializeBlockStore(blockStoreDB, sm.LoadState(stateDB), valAddr) - evidenceTime = time.Date(2019, 1, 1, 0, 0, 0, 0, time.UTC) + valAddr = tmrand.Bytes(crypto.AddressSize) + height = int64(1) + stateDB = initializeValidatorState(valAddr, height) + evidenceDB = dbm.NewMemDB() + blockStoreDB = dbm.NewMemDB() + blockStore = initializeBlockStore(blockStoreDB, sm.LoadState(stateDB), valAddr) + evidenceTime = time.Date(2019, 1, 1, 0, 0, 0, 0, time.UTC) ) pool, err := NewPool(stateDB, evidenceDB, blockStore) @@ -104,7 +106,7 @@ func TestProposingAndCommittingEvidence(t *testing.T) { assert.Equal(t, proposedEvidence[0], evidence) // evidence seen and committed: - pool.MarkEvidenceAsCommitted(height, lastBlockTime, proposedEvidence) + pool.MarkEvidenceAsCommitted(height, proposedEvidence) assert.True(t, pool.IsCommitted(evidence)) assert.False(t, pool.IsPending(evidence)) assert.Equal(t, 0, pool.evidenceList.Len()) @@ -161,14 +163,10 @@ func TestEvidencePoolUpdate(t *testing.T) { blockStoreDB = dbm.NewMemDB() state = sm.LoadState(stateDB) blockStore = initializeBlockStore(blockStoreDB, state, valAddr) - evidenceTime = time.Date(2019, 1, 1, 0, 0, 0, 0, time.UTC) ) pool, err := NewPool(stateDB, evidenceDB, blockStore) require.NoError(t, err) - expiredEvidence := types.NewMockEvidence(1, evidenceTime, valAddr) - err = pool.AddEvidence(expiredEvidence) - require.NoError(t, err) // create new block (no need to save it to blockStore) evidence := types.NewMockEvidence(height, time.Now(), valAddr) @@ -183,8 +181,6 @@ func TestEvidencePoolUpdate(t *testing.T) { assert.True(t, pool.IsCommitted(evidence)) // b) Update updates valToLastHeight map assert.Equal(t, height+1, pool.ValidatorLastHeight(valAddr)) - // c) Expired ecvidence should be removed - assert.False(t, pool.IsPending(expiredEvidence)) } func TestEvidencePoolNewPool(t *testing.T) { @@ -246,9 +242,7 @@ func TestAddingAndPruningPOLC(t *testing.T) { pool.Update(block, state) emptyPolc, err = pool.RetrievePOLC(1, 1) - if assert.Error(t, err) { - assert.Equal(t, "unable to find polc at height 1 and round 1", err.Error()) - } + assert.Error(t, err) assert.Equal(t, types.ProofOfLockChange{}, emptyPolc) } @@ -290,18 +284,108 @@ func TestRecoverPendingEvidence(t *testing.T) { assert.True(t, pool.IsPending(goodEvidence)) } -func initializeValidatorState(valAddr []byte, height int64) dbm.DB { - stateDB := dbm.NewMemDB() - pk := ed25519.GenPrivKey().PubKey() +func TestPotentialAmnesiaEvidence(t *testing.T) { + var ( + val = types.NewMockPV() + pubKey = val.PrivKey.PubKey() + valSet = &types.ValidatorSet{ + Validators: []*types.Validator{ + val.ExtractIntoValidator(0), + }, + Proposer: val.ExtractIntoValidator(0), + } + height = int64(30) + stateDB = initializeStateFromValidatorSet(valSet, height) + evidenceDB = dbm.NewMemDB() + blockStoreDB = dbm.NewMemDB() + state = sm.LoadState(stateDB) + blockStore = initializeBlockStore(blockStoreDB, state, pubKey.Address()) + //evidenceTime = time.Date(2019, 1, 1, 0, 0, 0, 0, time.UTC) + firstBlockID = types.BlockID{ + Hash: []byte("aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa"), + PartsHeader: types.PartSetHeader{ + Total: 1, + Hash: []byte("aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa"), + }, + } + secondBlockID = types.BlockID{ + Hash: []byte("bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb"), + PartsHeader: types.PartSetHeader{ + Total: 1, + Hash: []byte("bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb"), + }, + } + evidenceTime = time.Date(2019, 1, 1, 0, 0, 0, 0, time.UTC) + ) - // create validator set and state - validator := &types.Validator{Address: valAddr, VotingPower: 100, PubKey: pk} - valSet := &types.ValidatorSet{ - Validators: []*types.Validator{validator}, - Proposer: validator, + pool, err := NewPool(stateDB, evidenceDB, blockStore) + require.NoError(t, err) + + pool.SetLogger(log.TestingLogger()) + + polc := types.NewMockPOLC(25, evidenceTime, pubKey) + err = pool.AddPOLC(polc) + require.NoError(t, err) + + _, err = pool.RetrievePOLC(25, 1) + require.NoError(t, err) + + voteA := makeVote(25, 0, 0, pubKey.Address(), firstBlockID) + err = val.SignVote(evidenceChainID, voteA) + require.NoError(t, err) + voteB := makeVote(25, 1, 0, pubKey.Address(), secondBlockID) + err = val.SignVote(evidenceChainID, voteB) + require.NoError(t, err) + voteC := makeVote(25, 0, 0, pubKey.Address(), firstBlockID) + voteC.Timestamp.Add(1 * time.Second) + err = val.SignVote(evidenceChainID, voteC) + require.NoError(t, err) + ev := types.PotentialAmnesiaEvidence{ + VoteA: voteA, + VoteB: voteB, + } + // we expect the evidence pool to find the polc but log an error as the polc is not valid -> vote was + // not from a validator in this set. However, an error isn't thrown because the evidence pool + // should still be able to save the regular potential amnesia evidence. + err = pool.AddEvidence(ev) + assert.NoError(t, err) + + // evidence requires trial period until it is available -> we expect no evidence to be returned + assert.Equal(t, 0, len(pool.PendingEvidence(1))) + + nextHeight := pool.nextEvidenceTrialEndedHeight + assert.Greater(t, nextHeight, int64(0)) + + // evidence is not ready to be upgraded so we return the height we expect the evidence to be. + nextHeight = pool.upgradePotentialAmnesiaEvidence() + assert.Equal(t, height+pool.state.ConsensusParams.Evidence.ProofTrialPeriod, nextHeight) + + // now evidence is ready to be upgraded to amnesia evidence -> we expect -1 to be the next height as their is + // no more pending potential amnesia evidence left + pool.state.LastBlockHeight = nextHeight + nextHeight = pool.upgradePotentialAmnesiaEvidence() + assert.Equal(t, int64(-1), nextHeight) + + assert.Equal(t, 1, len(pool.PendingEvidence(1))) + + // evidence of voting back in the past which is instantly punishable -> amnesia evidence is made directly + voteA.Timestamp.Add(1 * time.Second) + + ev2 := types.PotentialAmnesiaEvidence{ + VoteA: voteB, + VoteB: voteC, } + err = pool.AddEvidence(ev2) + assert.NoError(t, err) + + assert.Equal(t, 2, len(pool.AllPendingEvidence())) + +} +func initializeStateFromValidatorSet(valSet *types.ValidatorSet, height int64) dbm.DB { + stateDB := dbm.NewMemDB() state := sm.State{ + ChainID: evidenceChainID, LastBlockHeight: height, LastBlockTime: tmtime.Now(), Validators: valSet, @@ -314,8 +398,10 @@ func initializeValidatorState(valAddr []byte, height int64) dbm.DB { MaxGas: -1, }, Evidence: tmproto.EvidenceParams{ - MaxAgeNumBlocks: 20, - MaxAgeDuration: 48 * time.Hour, + MaxAgeNumBlocks: 20, + MaxAgeDuration: 48 * time.Hour, + MaxNum: 50, + ProofTrialPeriod: 1, }, }, } @@ -329,6 +415,20 @@ func initializeValidatorState(valAddr []byte, height int64) dbm.DB { return stateDB } +func initializeValidatorState(valAddr []byte, height int64) dbm.DB { + + pubKey, _ := types.NewMockPV().GetPubKey() + validator := &types.Validator{Address: valAddr, VotingPower: 0, PubKey: pubKey} + + // create validator set and state + valSet := &types.ValidatorSet{ + Validators: []*types.Validator{validator}, + Proposer: validator, + } + + return initializeStateFromValidatorSet(valSet, height) +} + // initializeBlockStore creates a block storage and populates it w/ a dummy // block at +height+. func initializeBlockStore(db dbm.DB, state sm.State, valAddr []byte) *store.BlockStore { @@ -358,3 +458,15 @@ func makeCommit(height int64, valAddr []byte) *types.Commit { }} return types.NewCommit(height, 0, types.BlockID{}, commitSigs) } + +func makeVote(height int64, round, index int32, addr bytes.HexBytes, blockID types.BlockID) *types.Vote { + return &types.Vote{ + Type: tmproto.SignedMsgType(2), + Height: height, + Round: round, + BlockID: blockID, + Timestamp: time.Now(), + ValidatorAddress: addr, + ValidatorIndex: index, + } +} diff --git a/proto/types/evidence.pb.go b/proto/types/evidence.pb.go index d129c04c9..391075d90 100644 --- a/proto/types/evidence.pb.go +++ b/proto/types/evidence.pb.go @@ -83,8 +83,9 @@ func (m *DuplicateVoteEvidence) GetVoteB() *Vote { } type PotentialAmnesiaEvidence struct { - VoteA *Vote `protobuf:"bytes,1,opt,name=vote_a,json=voteA,proto3" json:"vote_a,omitempty"` - VoteB *Vote `protobuf:"bytes,2,opt,name=vote_b,json=voteB,proto3" json:"vote_b,omitempty"` + VoteA *Vote `protobuf:"bytes,1,opt,name=vote_a,json=voteA,proto3" json:"vote_a,omitempty"` + VoteB *Vote `protobuf:"bytes,2,opt,name=vote_b,json=voteB,proto3" json:"vote_b,omitempty"` + HeightStamp int64 `protobuf:"varint,3,opt,name=height_stamp,json=heightStamp,proto3" json:"height_stamp,omitempty"` } func (m *PotentialAmnesiaEvidence) Reset() { *m = PotentialAmnesiaEvidence{} } @@ -134,6 +135,65 @@ func (m *PotentialAmnesiaEvidence) GetVoteB() *Vote { return nil } +func (m *PotentialAmnesiaEvidence) GetHeightStamp() int64 { + if m != nil { + return m.HeightStamp + } + return 0 +} + +type AmnesiaEvidence struct { + PotentialAmnesiaEvidence *PotentialAmnesiaEvidence `protobuf:"bytes,1,opt,name=potential_amnesia_evidence,json=potentialAmnesiaEvidence,proto3" json:"potential_amnesia_evidence,omitempty"` + Polc *ProofOfLockChange `protobuf:"bytes,2,opt,name=polc,proto3" json:"polc,omitempty"` +} + +func (m *AmnesiaEvidence) Reset() { *m = AmnesiaEvidence{} } +func (m *AmnesiaEvidence) String() string { return proto.CompactTextString(m) } +func (*AmnesiaEvidence) ProtoMessage() {} +func (*AmnesiaEvidence) Descriptor() ([]byte, []int) { + return fileDescriptor_86495eef24aeacc0, []int{2} +} +func (m *AmnesiaEvidence) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *AmnesiaEvidence) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_AmnesiaEvidence.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *AmnesiaEvidence) XXX_Merge(src proto.Message) { + xxx_messageInfo_AmnesiaEvidence.Merge(m, src) +} +func (m *AmnesiaEvidence) XXX_Size() int { + return m.Size() +} +func (m *AmnesiaEvidence) XXX_DiscardUnknown() { + xxx_messageInfo_AmnesiaEvidence.DiscardUnknown(m) +} + +var xxx_messageInfo_AmnesiaEvidence proto.InternalMessageInfo + +func (m *AmnesiaEvidence) GetPotentialAmnesiaEvidence() *PotentialAmnesiaEvidence { + if m != nil { + return m.PotentialAmnesiaEvidence + } + return nil +} + +func (m *AmnesiaEvidence) GetPolc() *ProofOfLockChange { + if m != nil { + return m.Polc + } + return nil +} + // MockEvidence is used for testing pruposes type MockEvidence struct { EvidenceHeight int64 `protobuf:"varint,1,opt,name=evidence_height,json=evidenceHeight,proto3" json:"evidence_height,omitempty"` @@ -145,7 +205,7 @@ func (m *MockEvidence) Reset() { *m = MockEvidence{} } func (m *MockEvidence) String() string { return proto.CompactTextString(m) } func (*MockEvidence) ProtoMessage() {} func (*MockEvidence) Descriptor() ([]byte, []int) { - return fileDescriptor_86495eef24aeacc0, []int{2} + return fileDescriptor_86495eef24aeacc0, []int{3} } func (m *MockEvidence) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -206,7 +266,7 @@ func (m *MockRandomEvidence) Reset() { *m = MockRandomEvidence{} } func (m *MockRandomEvidence) String() string { return proto.CompactTextString(m) } func (*MockRandomEvidence) ProtoMessage() {} func (*MockRandomEvidence) Descriptor() ([]byte, []int) { - return fileDescriptor_86495eef24aeacc0, []int{3} + return fileDescriptor_86495eef24aeacc0, []int{4} } func (m *MockRandomEvidence) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -272,7 +332,7 @@ func (m *ConflictingHeadersEvidence) Reset() { *m = ConflictingHeadersEv func (m *ConflictingHeadersEvidence) String() string { return proto.CompactTextString(m) } func (*ConflictingHeadersEvidence) ProtoMessage() {} func (*ConflictingHeadersEvidence) Descriptor() ([]byte, []int) { - return fileDescriptor_86495eef24aeacc0, []int{4} + return fileDescriptor_86495eef24aeacc0, []int{5} } func (m *ConflictingHeadersEvidence) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -325,7 +385,7 @@ func (m *LunaticValidatorEvidence) Reset() { *m = LunaticValidatorEviden func (m *LunaticValidatorEvidence) String() string { return proto.CompactTextString(m) } func (*LunaticValidatorEvidence) ProtoMessage() {} func (*LunaticValidatorEvidence) Descriptor() ([]byte, []int) { - return fileDescriptor_86495eef24aeacc0, []int{5} + return fileDescriptor_86495eef24aeacc0, []int{6} } func (m *LunaticValidatorEvidence) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -381,6 +441,7 @@ type Evidence struct { // *Evidence_ConflictingHeadersEvidence // *Evidence_LunaticValidatorEvidence // *Evidence_PotentialAmnesiaEvidence + // *Evidence_AmnesiaEvidence // *Evidence_MockEvidence // *Evidence_MockRandomEvidence Sum isEvidence_Sum `protobuf_oneof:"sum"` @@ -390,7 +451,7 @@ func (m *Evidence) Reset() { *m = Evidence{} } func (m *Evidence) String() string { return proto.CompactTextString(m) } func (*Evidence) ProtoMessage() {} func (*Evidence) Descriptor() ([]byte, []int) { - return fileDescriptor_86495eef24aeacc0, []int{6} + return fileDescriptor_86495eef24aeacc0, []int{7} } func (m *Evidence) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -437,17 +498,21 @@ type Evidence_LunaticValidatorEvidence struct { type Evidence_PotentialAmnesiaEvidence struct { PotentialAmnesiaEvidence *PotentialAmnesiaEvidence `protobuf:"bytes,4,opt,name=potential_amnesia_evidence,json=potentialAmnesiaEvidence,proto3,oneof" json:"potential_amnesia_evidence,omitempty"` } +type Evidence_AmnesiaEvidence struct { + AmnesiaEvidence *AmnesiaEvidence `protobuf:"bytes,5,opt,name=amnesia_evidence,json=amnesiaEvidence,proto3,oneof" json:"amnesia_evidence,omitempty"` +} type Evidence_MockEvidence struct { - MockEvidence *MockEvidence `protobuf:"bytes,5,opt,name=mock_evidence,json=mockEvidence,proto3,oneof" json:"mock_evidence,omitempty"` + MockEvidence *MockEvidence `protobuf:"bytes,6,opt,name=mock_evidence,json=mockEvidence,proto3,oneof" json:"mock_evidence,omitempty"` } type Evidence_MockRandomEvidence struct { - MockRandomEvidence *MockRandomEvidence `protobuf:"bytes,6,opt,name=mock_random_evidence,json=mockRandomEvidence,proto3,oneof" json:"mock_random_evidence,omitempty"` + MockRandomEvidence *MockRandomEvidence `protobuf:"bytes,7,opt,name=mock_random_evidence,json=mockRandomEvidence,proto3,oneof" json:"mock_random_evidence,omitempty"` } func (*Evidence_DuplicateVoteEvidence) isEvidence_Sum() {} func (*Evidence_ConflictingHeadersEvidence) isEvidence_Sum() {} func (*Evidence_LunaticValidatorEvidence) isEvidence_Sum() {} func (*Evidence_PotentialAmnesiaEvidence) isEvidence_Sum() {} +func (*Evidence_AmnesiaEvidence) isEvidence_Sum() {} func (*Evidence_MockEvidence) isEvidence_Sum() {} func (*Evidence_MockRandomEvidence) isEvidence_Sum() {} @@ -486,6 +551,13 @@ func (m *Evidence) GetPotentialAmnesiaEvidence() *PotentialAmnesiaEvidence { return nil } +func (m *Evidence) GetAmnesiaEvidence() *AmnesiaEvidence { + if x, ok := m.GetSum().(*Evidence_AmnesiaEvidence); ok { + return x.AmnesiaEvidence + } + return nil +} + func (m *Evidence) GetMockEvidence() *MockEvidence { if x, ok := m.GetSum().(*Evidence_MockEvidence); ok { return x.MockEvidence @@ -507,6 +579,7 @@ func (*Evidence) XXX_OneofWrappers() []interface{} { (*Evidence_ConflictingHeadersEvidence)(nil), (*Evidence_LunaticValidatorEvidence)(nil), (*Evidence_PotentialAmnesiaEvidence)(nil), + (*Evidence_AmnesiaEvidence)(nil), (*Evidence_MockEvidence)(nil), (*Evidence_MockRandomEvidence)(nil), } @@ -522,7 +595,7 @@ func (m *EvidenceData) Reset() { *m = EvidenceData{} } func (m *EvidenceData) String() string { return proto.CompactTextString(m) } func (*EvidenceData) ProtoMessage() {} func (*EvidenceData) Descriptor() ([]byte, []int) { - return fileDescriptor_86495eef24aeacc0, []int{7} + return fileDescriptor_86495eef24aeacc0, []int{8} } func (m *EvidenceData) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -574,7 +647,7 @@ func (m *ProofOfLockChange) Reset() { *m = ProofOfLockChange{} } func (m *ProofOfLockChange) String() string { return proto.CompactTextString(m) } func (*ProofOfLockChange) ProtoMessage() {} func (*ProofOfLockChange) Descriptor() ([]byte, []int) { - return fileDescriptor_86495eef24aeacc0, []int{8} + return fileDescriptor_86495eef24aeacc0, []int{9} } func (m *ProofOfLockChange) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -620,6 +693,7 @@ func (m *ProofOfLockChange) GetPubKey() *keys.PublicKey { func init() { proto.RegisterType((*DuplicateVoteEvidence)(nil), "tendermint.proto.types.DuplicateVoteEvidence") proto.RegisterType((*PotentialAmnesiaEvidence)(nil), "tendermint.proto.types.PotentialAmnesiaEvidence") + proto.RegisterType((*AmnesiaEvidence)(nil), "tendermint.proto.types.AmnesiaEvidence") proto.RegisterType((*MockEvidence)(nil), "tendermint.proto.types.MockEvidence") proto.RegisterType((*MockRandomEvidence)(nil), "tendermint.proto.types.MockRandomEvidence") proto.RegisterType((*ConflictingHeadersEvidence)(nil), "tendermint.proto.types.ConflictingHeadersEvidence") @@ -632,57 +706,61 @@ func init() { func init() { proto.RegisterFile("proto/types/evidence.proto", fileDescriptor_86495eef24aeacc0) } var fileDescriptor_86495eef24aeacc0 = []byte{ - // 786 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x55, 0xcf, 0x4f, 0xdc, 0x46, - 0x14, 0xb6, 0xd9, 0x1f, 0x85, 0xc7, 0xd2, 0x1f, 0x16, 0x94, 0x95, 0x05, 0x0b, 0xb2, 0xaa, 0x42, - 0xab, 0xd6, 0x0b, 0x4b, 0xd5, 0x73, 0x59, 0x28, 0xda, 0x0a, 0xaa, 0x22, 0x37, 0xe2, 0x90, 0x43, - 0xac, 0xb1, 0x3d, 0x6b, 0x8f, 0xd6, 0xf6, 0x58, 0xf6, 0x78, 0x25, 0x1f, 0xa3, 0xe4, 0x90, 0xdc, - 0xf8, 0x47, 0x72, 0xcd, 0xdf, 0xc0, 0x11, 0xe5, 0x94, 0x53, 0x12, 0xc1, 0x3f, 0x12, 0x79, 0xfc, - 0x6b, 0x11, 0x18, 0xed, 0x2d, 0xca, 0x65, 0x35, 0xfb, 0xe6, 0x7d, 0xef, 0xfb, 0xc6, 0xef, 0xcd, - 0x37, 0x20, 0x07, 0x21, 0x65, 0xb4, 0xcf, 0x92, 0x00, 0x47, 0x7d, 0x3c, 0x25, 0x16, 0xf6, 0x4d, - 0xac, 0xf2, 0xa0, 0xf4, 0x23, 0xc3, 0xbe, 0x85, 0x43, 0x8f, 0xf8, 0x2c, 0x8b, 0xa8, 0x3c, 0x4d, - 0xfe, 0x99, 0x39, 0x24, 0xb4, 0xf4, 0x00, 0x85, 0x2c, 0xe9, 0x67, 0x78, 0x9b, 0xda, 0xb4, 0x5a, - 0x65, 0xd9, 0xf2, 0xfa, 0x6c, 0x6d, 0xfe, 0x9b, 0x6f, 0x6c, 0xd9, 0x94, 0xda, 0x2e, 0xce, 0xb0, - 0x46, 0x3c, 0xee, 0x33, 0xe2, 0xe1, 0x88, 0x21, 0x2f, 0xc8, 0x13, 0x36, 0x33, 0xa4, 0x19, 0x26, - 0x01, 0xa3, 0xfd, 0x09, 0x4e, 0xee, 0xe0, 0x95, 0xe7, 0x22, 0xac, 0x1d, 0xc7, 0x81, 0x4b, 0x4c, - 0xc4, 0xf0, 0x05, 0x65, 0xf8, 0xef, 0x5c, 0xb8, 0x74, 0x00, 0xed, 0x29, 0x65, 0x58, 0x47, 0x5d, - 0x71, 0x5b, 0xdc, 0x5d, 0x1e, 0x6c, 0xa8, 0x0f, 0x9f, 0x41, 0x4d, 0x51, 0x5a, 0x2b, 0xcd, 0x3d, - 0x2c, 0x41, 0x46, 0x77, 0x61, 0x5e, 0xd0, 0x50, 0x79, 0x29, 0x42, 0xf7, 0x9c, 0x32, 0xec, 0x33, - 0x82, 0xdc, 0x43, 0xcf, 0xc7, 0x11, 0x41, 0x5f, 0x40, 0xc6, 0x1b, 0x11, 0x3a, 0xff, 0x52, 0x73, - 0x52, 0x52, 0xef, 0xc0, 0x77, 0x45, 0x1b, 0x75, 0x07, 0x13, 0xdb, 0x61, 0x5c, 0x43, 0x43, 0xfb, - 0xb6, 0x08, 0x8f, 0x78, 0x54, 0xfa, 0x07, 0x56, 0xca, 0xc4, 0xf4, 0xfb, 0xe7, 0xac, 0xb2, 0x9a, - 0x35, 0x47, 0x2d, 0x9a, 0xa3, 0x3e, 0x29, 0x9a, 0x33, 0x5c, 0xbc, 0xfa, 0xb0, 0x25, 0x5c, 0x7e, - 0xdc, 0x12, 0xb5, 0x4e, 0x01, 0x4d, 0x37, 0xa5, 0x5f, 0xe0, 0xfb, 0xb2, 0x14, 0xb2, 0xac, 0x10, - 0x47, 0x51, 0xb7, 0xb1, 0x2d, 0xee, 0x76, 0xb4, 0x52, 0xcb, 0x61, 0x16, 0x56, 0xde, 0x89, 0x20, - 0xa5, 0x7a, 0x35, 0xe4, 0x5b, 0xd4, 0xfb, 0x4a, 0x54, 0x4b, 0x9b, 0x00, 0x21, 0xf2, 0x2d, 0xdd, - 0x48, 0x18, 0x8e, 0xba, 0x4d, 0x9e, 0xb4, 0x94, 0x46, 0x86, 0x69, 0x40, 0x79, 0x25, 0x82, 0x7c, - 0x44, 0xfd, 0xb1, 0x4b, 0x4c, 0x46, 0x7c, 0x7b, 0x84, 0x91, 0x85, 0xc3, 0xa8, 0x3c, 0xdc, 0x1f, - 0xb0, 0xe0, 0xec, 0xe7, 0x93, 0xf0, 0x53, 0x5d, 0x53, 0xff, 0x27, 0xb6, 0x8f, 0xad, 0x0c, 0xaa, - 0x2d, 0x38, 0xfb, 0x1c, 0x35, 0xc8, 0x8f, 0x37, 0x2f, 0x6a, 0xa0, 0xbc, 0x15, 0xa1, 0x7b, 0x16, - 0xfb, 0x88, 0x11, 0xf3, 0x02, 0xb9, 0xc4, 0x42, 0x8c, 0x86, 0xa5, 0x90, 0x3f, 0xa1, 0xed, 0xf0, - 0xd4, 0x5c, 0x4c, 0xaf, 0xae, 0x6c, 0x5e, 0x30, 0xcf, 0x96, 0xf6, 0xa0, 0x99, 0x4e, 0xdb, 0x5c, - 0x73, 0xc9, 0x33, 0xa5, 0x3d, 0x58, 0x25, 0xfe, 0x34, 0x15, 0xa0, 0x67, 0x35, 0xf4, 0x31, 0xc1, - 0xae, 0xc5, 0xbf, 0xef, 0x92, 0x26, 0xe5, 0x7b, 0x19, 0xcd, 0x49, 0xba, 0xa3, 0xbc, 0x68, 0xc1, - 0x62, 0x29, 0xd4, 0x86, 0x75, 0xab, 0xb8, 0xdf, 0x3a, 0xbf, 0x14, 0x45, 0x47, 0x72, 0xe5, 0xbf, - 0xd7, 0x69, 0x78, 0xd0, 0x16, 0x46, 0x82, 0xb6, 0x66, 0x3d, 0xe8, 0x17, 0x53, 0xd8, 0x30, 0xab, - 0xc6, 0xe5, 0x5a, 0xa3, 0x8a, 0x2d, 0x3b, 0xf1, 0xa0, 0x8e, 0xad, 0xbe, 0xe9, 0x23, 0x41, 0x93, - 0xcd, 0xfa, 0x91, 0x08, 0x40, 0x76, 0xb3, 0x2e, 0xe9, 0xd3, 0xa2, 0x4d, 0x15, 0x6b, 0x83, 0xb3, - 0xee, 0xd5, 0xb1, 0xd6, 0xf5, 0x77, 0x24, 0x68, 0x5d, 0xb7, 0xae, 0xf7, 0x01, 0xc8, 0x41, 0x61, - 0x57, 0x3a, 0xca, 0xfc, 0xaa, 0x62, 0x6c, 0x3e, 0xce, 0x58, 0x67, 0x74, 0x29, 0x63, 0x50, 0x67, - 0x82, 0xa7, 0xb0, 0xe2, 0x51, 0x73, 0x52, 0x91, 0xb4, 0x1e, 0x9f, 0xe5, 0x59, 0x1b, 0x1b, 0x09, - 0x5a, 0xc7, 0x9b, 0xb5, 0xb5, 0x67, 0xb0, 0xca, 0x8b, 0x85, 0xdc, 0x37, 0xaa, 0x9a, 0x6d, 0x5e, - 0xf3, 0xd7, 0xc7, 0x6a, 0xde, 0xb5, 0x9a, 0x91, 0xa0, 0x49, 0xde, 0xbd, 0xe8, 0xb0, 0x05, 0x8d, - 0x28, 0xf6, 0x94, 0x31, 0x74, 0x8a, 0xd0, 0x31, 0x62, 0x48, 0x1a, 0xc2, 0xe2, 0xcc, 0xe4, 0x35, - 0x76, 0x97, 0x07, 0xdb, 0x75, 0x54, 0x65, 0xa9, 0x66, 0xea, 0x37, 0x5a, 0x89, 0x93, 0x24, 0x68, - 0x3a, 0x28, 0x72, 0xf8, 0x2c, 0x75, 0x34, 0xbe, 0x56, 0x5e, 0x8b, 0xf0, 0xc3, 0x79, 0x48, 0xe9, - 0xf8, 0xbf, 0xf1, 0x19, 0x35, 0x27, 0x47, 0x0e, 0xf2, 0x6d, 0x2c, 0x0d, 0x80, 0xbb, 0x7a, 0x94, - 0x53, 0xcd, 0xf1, 0x00, 0x44, 0xd2, 0x5f, 0xf0, 0x4d, 0x10, 0x1b, 0xfa, 0x04, 0x27, 0xf9, 0xb0, - 0xee, 0xdc, 0x47, 0x65, 0xef, 0xa8, 0x9a, 0xbe, 0xa3, 0xea, 0x79, 0x6c, 0xb8, 0xc4, 0x3c, 0xc5, - 0x89, 0xd6, 0x0e, 0x62, 0xe3, 0x14, 0x27, 0xc3, 0x93, 0xab, 0x9b, 0x9e, 0x78, 0x7d, 0xd3, 0x13, - 0x3f, 0xdd, 0xf4, 0xc4, 0xcb, 0xdb, 0x9e, 0x70, 0x7d, 0xdb, 0x13, 0xde, 0xdf, 0xf6, 0x84, 0xa7, - 0xbf, 0xd9, 0x84, 0x39, 0xb1, 0xa1, 0x9a, 0xd4, 0xeb, 0x57, 0x45, 0x67, 0x97, 0x33, 0x2f, 0xbc, - 0xd1, 0xe6, 0x7f, 0x0e, 0x3e, 0x07, 0x00, 0x00, 0xff, 0xff, 0x6a, 0x4c, 0xe5, 0x0b, 0x53, 0x08, - 0x00, 0x00, + // 858 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x56, 0xcf, 0x6f, 0xdc, 0x44, + 0x14, 0xb6, 0xb3, 0x9b, 0x6d, 0xfa, 0xb2, 0x25, 0x65, 0xd4, 0x52, 0xcb, 0x6a, 0x37, 0xc1, 0x42, + 0x34, 0x45, 0xe0, 0x4d, 0xb7, 0x88, 0x1b, 0x12, 0xd9, 0x96, 0x6a, 0x51, 0x8a, 0xa8, 0xa6, 0x55, + 0x0f, 0x1c, 0xb0, 0xc6, 0xf6, 0xac, 0x3d, 0x5a, 0xdb, 0x63, 0xd9, 0xe3, 0x95, 0x7c, 0xe4, 0x06, + 0xb7, 0xfe, 0x17, 0x9c, 0xb8, 0x72, 0xe6, 0xd8, 0x63, 0xc5, 0x09, 0x2e, 0x80, 0x92, 0x7f, 0x04, + 0x79, 0xc6, 0xf6, 0x6e, 0x7e, 0x38, 0x8a, 0x38, 0x20, 0x71, 0x59, 0x79, 0xdf, 0xbc, 0xef, 0x7b, + 0xdf, 0x9b, 0xf7, 0xc3, 0x06, 0x33, 0xcd, 0xb8, 0xe0, 0x63, 0x51, 0xa6, 0x34, 0x1f, 0xd3, 0x25, + 0xf3, 0x69, 0xe2, 0x51, 0x5b, 0x1a, 0xd1, 0x7b, 0x82, 0x26, 0x3e, 0xcd, 0x62, 0x96, 0x08, 0x65, + 0xb1, 0xa5, 0x9b, 0xf9, 0xa1, 0x08, 0x59, 0xe6, 0x3b, 0x29, 0xc9, 0x44, 0x39, 0x56, 0xf8, 0x80, + 0x07, 0x7c, 0xf5, 0xa4, 0xbc, 0xcd, 0x3b, 0xeb, 0xdc, 0xf2, 0xb7, 0x3e, 0xd8, 0x0d, 0x38, 0x0f, + 0x22, 0xaa, 0xb0, 0x6e, 0x31, 0x1f, 0x0b, 0x16, 0xd3, 0x5c, 0x90, 0x38, 0xad, 0x1d, 0xee, 0x29, + 0xa4, 0x97, 0x95, 0xa9, 0xe0, 0xe3, 0x05, 0x2d, 0x4f, 0xe1, 0xad, 0xef, 0x75, 0xb8, 0xfd, 0xa4, + 0x48, 0x23, 0xe6, 0x11, 0x41, 0x5f, 0x71, 0x41, 0xbf, 0xac, 0x85, 0xa3, 0x47, 0x30, 0x58, 0x72, + 0x41, 0x1d, 0x62, 0xe8, 0x7b, 0xfa, 0xfe, 0xf6, 0xe4, 0xae, 0x7d, 0x71, 0x0e, 0x76, 0x85, 0xc2, + 0x9b, 0x95, 0xef, 0x61, 0x0b, 0x72, 0x8d, 0x8d, 0xab, 0x82, 0xa6, 0xd6, 0x4f, 0x3a, 0x18, 0xcf, + 0xb9, 0xa0, 0x89, 0x60, 0x24, 0x3a, 0x8c, 0x13, 0x9a, 0x33, 0xf2, 0xdf, 0xcb, 0x40, 0xef, 0xc3, + 0x30, 0xa4, 0x2c, 0x08, 0x85, 0x23, 0xef, 0xcf, 0xe8, 0xed, 0xe9, 0xfb, 0x3d, 0xbc, 0xad, 0x6c, + 0x2f, 0x2a, 0x93, 0xf5, 0xab, 0x0e, 0x3b, 0x67, 0x05, 0x26, 0x60, 0xa6, 0x8d, 0x78, 0x87, 0xa8, + 0x43, 0xa7, 0x29, 0x7f, 0x2d, 0xfa, 0xa0, 0x2b, 0x7e, 0x57, 0xda, 0xd8, 0x48, 0xbb, 0x2e, 0xe4, + 0x73, 0xe8, 0xa7, 0x3c, 0xf2, 0xea, 0xcc, 0x1e, 0x74, 0x32, 0x67, 0x9c, 0xcf, 0xbf, 0x99, 0x3f, + 0xe3, 0xde, 0xe2, 0x71, 0x48, 0x92, 0x80, 0x62, 0x09, 0xb3, 0x7e, 0xd6, 0x61, 0xf8, 0x35, 0xf7, + 0x16, 0x2d, 0xdf, 0x7d, 0xd8, 0x69, 0xd4, 0x3a, 0x2a, 0x57, 0x29, 0xba, 0x87, 0xdf, 0x69, 0xcc, + 0x33, 0x69, 0x45, 0x5f, 0xc1, 0x8d, 0xd6, 0xb1, 0xea, 0xb2, 0x5a, 0x81, 0x69, 0xab, 0x16, 0xb4, + 0x9b, 0x16, 0xb4, 0x5f, 0x36, 0x2d, 0x38, 0xdd, 0x7a, 0xf3, 0xe7, 0xae, 0xf6, 0xfa, 0xaf, 0x5d, + 0x1d, 0x0f, 0x1b, 0x68, 0x75, 0x88, 0x1e, 0xc0, 0xcd, 0x96, 0x8a, 0xf8, 0x7e, 0x46, 0xf3, 0x5c, + 0x5e, 0xf7, 0x10, 0xb7, 0x5a, 0x0e, 0x95, 0xd9, 0xfa, 0x4d, 0x07, 0x54, 0xe9, 0xc5, 0x24, 0xf1, + 0x79, 0xfc, 0x3f, 0x51, 0x8d, 0xee, 0x01, 0x64, 0x24, 0xf1, 0x1d, 0xb7, 0x14, 0x34, 0x37, 0xfa, + 0xd2, 0xe9, 0x7a, 0x65, 0x99, 0x56, 0x06, 0xeb, 0x07, 0x1d, 0xcc, 0xc7, 0x3c, 0x99, 0x47, 0xcc, + 0x13, 0x2c, 0x09, 0x66, 0x94, 0xf8, 0x34, 0xcb, 0xdb, 0xe4, 0x3e, 0x85, 0x8d, 0xf0, 0x61, 0xdd, + 0x3a, 0x1f, 0x74, 0x15, 0xf8, 0x05, 0x0b, 0x12, 0xea, 0x2b, 0x28, 0xde, 0x08, 0x1f, 0x4a, 0xd4, + 0xa4, 0x4e, 0xef, 0xaa, 0xa8, 0x89, 0xf5, 0x8b, 0x0e, 0xc6, 0xb3, 0x22, 0x21, 0x82, 0x79, 0xaf, + 0x48, 0xc4, 0x7c, 0x22, 0x78, 0xd6, 0x0a, 0xf9, 0x0c, 0x06, 0xa1, 0x74, 0xad, 0xc5, 0x8c, 0xba, + 0x68, 0x6b, 0xc2, 0xda, 0x1b, 0x1d, 0x40, 0xbf, 0x9a, 0xa9, 0x2b, 0x4d, 0x9f, 0xf4, 0x44, 0x07, + 0x70, 0x8b, 0x25, 0xcb, 0x4a, 0x80, 0xa3, 0x38, 0x9c, 0x39, 0xa3, 0x91, 0x2f, 0xef, 0xf7, 0x3a, + 0x46, 0xf5, 0x99, 0x0a, 0xf3, 0xb4, 0x3a, 0xb1, 0xfe, 0xd8, 0x84, 0xad, 0x56, 0x68, 0x00, 0x77, + 0xfc, 0x66, 0x8b, 0x39, 0x72, 0xf4, 0xcf, 0x4c, 0xe0, 0x27, 0x5d, 0x1a, 0x2e, 0x5c, 0x7e, 0x33, + 0x0d, 0xdf, 0xf6, 0x2f, 0xdc, 0x8a, 0x4b, 0xb8, 0xeb, 0xad, 0x0a, 0x57, 0x6b, 0xcd, 0x57, 0xd1, + 0x54, 0xc6, 0x93, 0xae, 0x68, 0xdd, 0x45, 0x9f, 0x69, 0xd8, 0xf4, 0xba, 0x5b, 0x22, 0x05, 0x33, + 0x52, 0x55, 0x72, 0x96, 0x4d, 0x99, 0x56, 0x51, 0x7b, 0x97, 0x6f, 0x99, 0xae, 0xfa, 0xce, 0x34, + 0x6c, 0x44, 0x5d, 0xb5, 0x4f, 0x2f, 0xdd, 0x6b, 0xfd, 0x7f, 0xb7, 0xd7, 0xaa, 0x88, 0x9d, 0x9b, + 0xed, 0x25, 0xdc, 0x3c, 0x17, 0x67, 0x53, 0xc6, 0xb9, 0xdf, 0x15, 0xe7, 0x3c, 0xfd, 0x0e, 0x39, + 0xc3, 0x7a, 0x04, 0x37, 0x62, 0xee, 0x2d, 0x56, 0x94, 0x83, 0xcb, 0x27, 0x64, 0x7d, 0x39, 0xce, + 0x34, 0x3c, 0x8c, 0xd7, 0x97, 0xe5, 0x77, 0x70, 0x4b, 0x92, 0x65, 0x72, 0x1b, 0xad, 0x38, 0xaf, + 0x49, 0xce, 0x8f, 0x2e, 0xe3, 0x3c, 0xbd, 0xc0, 0x66, 0x1a, 0x46, 0xf1, 0x39, 0xeb, 0x74, 0x13, + 0x7a, 0x79, 0x11, 0x5b, 0x73, 0x18, 0x36, 0xa6, 0x27, 0x44, 0x10, 0x34, 0x85, 0xad, 0xb5, 0x7e, + 0xee, 0xed, 0x6f, 0x4f, 0xf6, 0xba, 0x42, 0xb5, 0x54, 0xfd, 0x6a, 0x8b, 0xe1, 0x16, 0x87, 0x10, + 0xf4, 0x43, 0x92, 0x87, 0xb2, 0x43, 0x87, 0x58, 0x3e, 0x5b, 0x3f, 0xea, 0xf0, 0xee, 0xb9, 0x17, + 0x05, 0x9a, 0x80, 0x7c, 0x23, 0xe6, 0x75, 0xa8, 0x2b, 0xbc, 0x3c, 0x73, 0xf4, 0x05, 0x5c, 0x4b, + 0x0b, 0xd7, 0x59, 0xd0, 0xb2, 0x1e, 0x81, 0x0b, 0x4a, 0xa6, 0xbe, 0x41, 0xec, 0xea, 0x1b, 0xc4, + 0x7e, 0x5e, 0xb8, 0x11, 0xf3, 0x8e, 0x68, 0x89, 0x07, 0x69, 0xe1, 0x1e, 0xd1, 0x72, 0xfa, 0xf4, + 0xcd, 0xf1, 0x48, 0x7f, 0x7b, 0x3c, 0xd2, 0xff, 0x3e, 0x1e, 0xe9, 0xaf, 0x4f, 0x46, 0xda, 0xdb, + 0x93, 0x91, 0xf6, 0xfb, 0xc9, 0x48, 0xfb, 0xf6, 0xe3, 0x80, 0x89, 0xb0, 0x70, 0x6d, 0x8f, 0xc7, + 0xe3, 0x15, 0xe9, 0xfa, 0xe3, 0xda, 0xd7, 0x91, 0x3b, 0x90, 0x7f, 0x1e, 0xfd, 0x13, 0x00, 0x00, + 0xff, 0xff, 0xa6, 0x62, 0x87, 0x20, 0x8f, 0x09, 0x00, 0x00, } func (m *DuplicateVoteEvidence) Marshal() (dAtA []byte, err error) { @@ -752,6 +830,11 @@ func (m *PotentialAmnesiaEvidence) MarshalToSizedBuffer(dAtA []byte) (int, error _ = i var l int _ = l + if m.HeightStamp != 0 { + i = encodeVarintEvidence(dAtA, i, uint64(m.HeightStamp)) + i-- + dAtA[i] = 0x18 + } if m.VoteB != nil { { size, err := m.VoteB.MarshalToSizedBuffer(dAtA[:i]) @@ -779,6 +862,53 @@ func (m *PotentialAmnesiaEvidence) MarshalToSizedBuffer(dAtA []byte) (int, error return len(dAtA) - i, nil } +func (m *AmnesiaEvidence) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *AmnesiaEvidence) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *AmnesiaEvidence) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Polc != nil { + { + size, err := m.Polc.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintEvidence(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + if m.PotentialAmnesiaEvidence != nil { + { + size, err := m.PotentialAmnesiaEvidence.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintEvidence(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + func (m *MockEvidence) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -806,12 +936,12 @@ func (m *MockEvidence) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x1a } - n5, err5 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.EvidenceTime, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.EvidenceTime):]) - if err5 != nil { - return 0, err5 + n7, err7 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.EvidenceTime, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.EvidenceTime):]) + if err7 != nil { + return 0, err7 } - i -= n5 - i = encodeVarintEvidence(dAtA, i, uint64(n5)) + i -= n7 + i = encodeVarintEvidence(dAtA, i, uint64(n7)) i-- dAtA[i] = 0x12 if m.EvidenceHeight != 0 { @@ -856,12 +986,12 @@ func (m *MockRandomEvidence) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x1a } - n6, err6 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.EvidenceTime, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.EvidenceTime):]) - if err6 != nil { - return 0, err6 + n8, err8 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.EvidenceTime, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.EvidenceTime):]) + if err8 != nil { + return 0, err8 } - i -= n6 - i = encodeVarintEvidence(dAtA, i, uint64(n6)) + i -= n8 + i = encodeVarintEvidence(dAtA, i, uint64(n8)) i-- dAtA[i] = 0x12 if m.EvidenceHeight != 0 { @@ -1089,6 +1219,27 @@ func (m *Evidence_PotentialAmnesiaEvidence) MarshalToSizedBuffer(dAtA []byte) (i } return len(dAtA) - i, nil } +func (m *Evidence_AmnesiaEvidence) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Evidence_AmnesiaEvidence) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + if m.AmnesiaEvidence != nil { + { + size, err := m.AmnesiaEvidence.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintEvidence(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x2a + } + return len(dAtA) - i, nil +} func (m *Evidence_MockEvidence) MarshalTo(dAtA []byte) (int, error) { size := m.Size() return m.MarshalToSizedBuffer(dAtA[:size]) @@ -1106,7 +1257,7 @@ func (m *Evidence_MockEvidence) MarshalToSizedBuffer(dAtA []byte) (int, error) { i = encodeVarintEvidence(dAtA, i, uint64(size)) } i-- - dAtA[i] = 0x2a + dAtA[i] = 0x32 } return len(dAtA) - i, nil } @@ -1127,7 +1278,7 @@ func (m *Evidence_MockRandomEvidence) MarshalToSizedBuffer(dAtA []byte) (int, er i = encodeVarintEvidence(dAtA, i, uint64(size)) } i-- - dAtA[i] = 0x32 + dAtA[i] = 0x3a } return len(dAtA) - i, nil } @@ -1266,6 +1417,26 @@ func (m *PotentialAmnesiaEvidence) Size() (n int) { l = m.VoteB.Size() n += 1 + l + sovEvidence(uint64(l)) } + if m.HeightStamp != 0 { + n += 1 + sovEvidence(uint64(m.HeightStamp)) + } + return n +} + +func (m *AmnesiaEvidence) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.PotentialAmnesiaEvidence != nil { + l = m.PotentialAmnesiaEvidence.Size() + n += 1 + l + sovEvidence(uint64(l)) + } + if m.Polc != nil { + l = m.Polc.Size() + n += 1 + l + sovEvidence(uint64(l)) + } return n } @@ -1407,6 +1578,18 @@ func (m *Evidence_PotentialAmnesiaEvidence) Size() (n int) { } return n } +func (m *Evidence_AmnesiaEvidence) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.AmnesiaEvidence != nil { + l = m.AmnesiaEvidence.Size() + n += 1 + l + sovEvidence(uint64(l)) + } + return n +} func (m *Evidence_MockEvidence) Size() (n int) { if m == nil { return 0 @@ -1701,6 +1884,150 @@ func (m *PotentialAmnesiaEvidence) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field HeightStamp", wireType) + } + m.HeightStamp = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEvidence + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.HeightStamp |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipEvidence(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthEvidence + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthEvidence + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *AmnesiaEvidence) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEvidence + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: AmnesiaEvidence: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: AmnesiaEvidence: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PotentialAmnesiaEvidence", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEvidence + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthEvidence + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthEvidence + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.PotentialAmnesiaEvidence == nil { + m.PotentialAmnesiaEvidence = &PotentialAmnesiaEvidence{} + } + if err := m.PotentialAmnesiaEvidence.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Polc", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEvidence + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthEvidence + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthEvidence + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Polc == nil { + m.Polc = &ProofOfLockChange{} + } + if err := m.Polc.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipEvidence(dAtA[iNdEx:]) @@ -2489,6 +2816,41 @@ func (m *Evidence) Unmarshal(dAtA []byte) error { m.Sum = &Evidence_PotentialAmnesiaEvidence{v} iNdEx = postIndex case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AmnesiaEvidence", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEvidence + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthEvidence + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthEvidence + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &AmnesiaEvidence{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Sum = &Evidence_AmnesiaEvidence{v} + iNdEx = postIndex + case 6: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field MockEvidence", wireType) } @@ -2523,7 +2885,7 @@ func (m *Evidence) Unmarshal(dAtA []byte) error { } m.Sum = &Evidence_MockEvidence{v} iNdEx = postIndex - case 6: + case 7: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field MockRandomEvidence", wireType) } diff --git a/proto/types/evidence.proto b/proto/types/evidence.proto index 894382f7f..749e010ca 100644 --- a/proto/types/evidence.proto +++ b/proto/types/evidence.proto @@ -18,6 +18,13 @@ message DuplicateVoteEvidence { message PotentialAmnesiaEvidence { Vote vote_a = 1; Vote vote_b = 2; + + int64 height_stamp = 3; +} + +message AmnesiaEvidence { + PotentialAmnesiaEvidence potential_amnesia_evidence = 1; + ProofOfLockChange polc = 2; } // MockEvidence is used for testing pruposes @@ -52,9 +59,10 @@ message Evidence { ConflictingHeadersEvidence conflicting_headers_evidence = 2; LunaticValidatorEvidence lunatic_validator_evidence = 3; PotentialAmnesiaEvidence potential_amnesia_evidence = 4; + AmnesiaEvidence amnesia_evidence = 5; - MockEvidence mock_evidence = 5; - MockRandomEvidence mock_random_evidence = 6; + MockEvidence mock_evidence = 6; + MockRandomEvidence mock_random_evidence = 7; } } diff --git a/proto/types/params.pb.go b/proto/types/params.pb.go index 492070955..ef930e025 100644 --- a/proto/types/params.pb.go +++ b/proto/types/params.pb.go @@ -174,6 +174,10 @@ type EvidenceParams struct { // each evidence (See MaxEvidenceBytes). The maximum number is MaxEvidencePerBlock. // Default is 50 MaxNum uint32 `protobuf:"varint,3,opt,name=max_num,json=maxNum,proto3" json:"max_num,omitempty"` + // Proof trial period dictates the time given for nodes accused of amnesia evidence, incorrectly + // voting twice in two different rounds to respond with their respective proofs. + // Default is half the max age in blocks: 50,000 + ProofTrialPeriod int64 `protobuf:"varint,4,opt,name=proof_trial_period,json=proofTrialPeriod,proto3" json:"proof_trial_period,omitempty"` } func (m *EvidenceParams) Reset() { *m = EvidenceParams{} } @@ -230,6 +234,13 @@ func (m *EvidenceParams) GetMaxNum() uint32 { return 0 } +func (m *EvidenceParams) GetProofTrialPeriod() int64 { + if m != nil { + return m.ProofTrialPeriod + } + return 0 +} + // ValidatorParams restrict the public key types validators can use. // NOTE: uses ABCI pubkey naming, not Amino names. type ValidatorParams struct { @@ -348,39 +359,41 @@ func init() { proto.RegisterFile("proto/types/params.proto", fileDescriptor_95a9 func init() { golang_proto.RegisterFile("proto/types/params.proto", fileDescriptor_95a9f934fa6f056c) } var fileDescriptor_95a9f934fa6f056c = []byte{ - // 512 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x74, 0x53, 0x31, 0x6f, 0xd3, 0x40, - 0x14, 0xce, 0x61, 0x28, 0xe9, 0x4b, 0xd3, 0xa0, 0x1b, 0xc0, 0x14, 0xc9, 0x89, 0x8c, 0x14, 0x2a, - 0x81, 0x6c, 0x09, 0x36, 0x96, 0x0a, 0x03, 0x6a, 0x51, 0x95, 0x08, 0x59, 0x88, 0xa1, 0x8b, 0x75, - 0x8e, 0x0f, 0xc7, 0x6a, 0xce, 0x67, 0xf9, 0xee, 0xaa, 0xf8, 0x5f, 0x30, 0x32, 0x76, 0x41, 0xe2, - 0x27, 0x30, 0x32, 0x76, 0xec, 0xc8, 0x04, 0x28, 0x59, 0xe0, 0x5f, 0x20, 0x9f, 0x63, 0x9c, 0x54, - 0x74, 0xbb, 0x7b, 0xef, 0xfb, 0xbe, 0x7b, 0xdf, 0xf7, 0x6c, 0x30, 0xb3, 0x9c, 0x4b, 0xee, 0xca, - 0x22, 0xa3, 0xc2, 0xcd, 0x48, 0x4e, 0x98, 0x70, 0x74, 0x09, 0xdf, 0x95, 0x34, 0x8d, 0x68, 0xce, - 0x92, 0x54, 0x56, 0x15, 0x47, 0x83, 0xf6, 0x86, 0x72, 0x9a, 0xe4, 0x51, 0x90, 0x91, 0x5c, 0x16, - 0x6e, 0xc5, 0x8e, 0x79, 0xcc, 0x9b, 0x53, 0x85, 0xde, 0xb3, 0x62, 0xce, 0xe3, 0x19, 0xad, 0x20, - 0xa1, 0xfa, 0xe0, 0x46, 0x2a, 0x27, 0x32, 0xe1, 0x69, 0xd5, 0xb7, 0xff, 0x20, 0xe8, 0xbd, 0xe4, - 0xa9, 0xa0, 0xa9, 0x50, 0xe2, 0xad, 0x7e, 0x19, 0x1f, 0xc0, 0xad, 0x70, 0xc6, 0x27, 0xa7, 0x26, - 0x1a, 0xa0, 0xfd, 0xce, 0xd3, 0x87, 0xce, 0xff, 0x67, 0x70, 0xbc, 0x12, 0x54, 0x71, 0xbc, 0x9b, - 0x17, 0x3f, 0xfa, 0x2d, 0xbf, 0xe2, 0xe1, 0x23, 0x68, 0xd3, 0xb3, 0x24, 0xa2, 0xe9, 0x84, 0x9a, - 0x37, 0xb4, 0xc6, 0xf0, 0x3a, 0x8d, 0xd7, 0x2b, 0xdc, 0x86, 0xcc, 0x3f, 0x36, 0x3e, 0x86, 0xed, - 0x33, 0x32, 0x4b, 0x22, 0x22, 0x79, 0x6e, 0x1a, 0x5a, 0xea, 0xd1, 0x75, 0x52, 0xef, 0x6b, 0xe0, - 0x86, 0x56, 0xc3, 0xb7, 0x29, 0x74, 0xd6, 0x46, 0xc6, 0x0f, 0x60, 0x9b, 0x91, 0x79, 0x10, 0x16, - 0x92, 0x0a, 0x6d, 0xd5, 0xf0, 0xdb, 0x8c, 0xcc, 0xbd, 0xf2, 0x8e, 0xef, 0xc1, 0xed, 0xb2, 0x19, - 0x13, 0xa1, 0x1d, 0x18, 0xfe, 0x16, 0x23, 0xf3, 0x43, 0x22, 0xf0, 0x00, 0x76, 0x64, 0xc2, 0x68, - 0x90, 0x70, 0x49, 0x02, 0x26, 0xf4, 0x50, 0x86, 0x0f, 0x65, 0xed, 0x0d, 0x97, 0x64, 0x24, 0xec, - 0xcf, 0x08, 0x76, 0x37, 0x6d, 0xe1, 0xc7, 0x80, 0x4b, 0x35, 0x12, 0xd3, 0x20, 0x55, 0x2c, 0xd0, - 0x29, 0xd5, 0x6f, 0xf6, 0x18, 0x99, 0xbf, 0x88, 0xe9, 0x58, 0x31, 0x3d, 0x9c, 0xc0, 0x23, 0xb8, - 0x53, 0x83, 0xeb, 0x65, 0xad, 0x52, 0xbc, 0xef, 0x54, 0xdb, 0x74, 0xea, 0x6d, 0x3a, 0xaf, 0x56, - 0x00, 0xaf, 0x5d, 0x9a, 0xfd, 0xf4, 0xb3, 0x8f, 0xfc, 0xdd, 0x4a, 0xaf, 0xee, 0xd4, 0x4e, 0x52, - 0xc5, 0xf4, 0xac, 0x5d, 0xed, 0x64, 0xac, 0x98, 0x7d, 0x00, 0xbd, 0x2b, 0x91, 0x61, 0x1b, 0xba, - 0x99, 0x0a, 0x83, 0x53, 0x5a, 0x04, 0x3a, 0x53, 0x13, 0x0d, 0x8c, 0xfd, 0x6d, 0xbf, 0x93, 0xa9, - 0xf0, 0x98, 0x16, 0xef, 0xca, 0xd2, 0xf3, 0xf6, 0xd7, 0xf3, 0x3e, 0xfa, 0x7d, 0xde, 0x47, 0xf6, - 0x09, 0xec, 0x1c, 0x11, 0x31, 0xa5, 0xd1, 0x8a, 0x3d, 0x84, 0x9e, 0x76, 0x16, 0x5c, 0x8d, 0xb5, - 0xab, 0xcb, 0xa3, 0x3a, 0x5b, 0x1b, 0xba, 0x0d, 0xae, 0x49, 0xb8, 0x53, 0xa3, 0x0e, 0x89, 0xf0, - 0xc6, 0x5f, 0x16, 0x16, 0xba, 0x58, 0x58, 0xe8, 0x72, 0x61, 0xa1, 0x5f, 0x0b, 0x0b, 0x7d, 0x5c, - 0x5a, 0xad, 0x6f, 0x4b, 0x0b, 0x5d, 0x2e, 0xad, 0xd6, 0xf7, 0xa5, 0xd5, 0x3a, 0x79, 0x12, 0x27, - 0x72, 0xaa, 0x42, 0x67, 0xc2, 0x99, 0xdb, 0x7c, 0x11, 0xeb, 0xc7, 0xb5, 0x9f, 0x2a, 0xdc, 0xd2, - 0x97, 0x67, 0x7f, 0x03, 0x00, 0x00, 0xff, 0xff, 0x66, 0x2c, 0xc5, 0x1a, 0x6a, 0x03, 0x00, 0x00, + // 540 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x74, 0x53, 0xbf, 0x6f, 0xd3, 0x40, + 0x18, 0xcd, 0x91, 0x52, 0x92, 0x4b, 0xd3, 0x54, 0x37, 0x80, 0x29, 0x92, 0x13, 0x19, 0x29, 0x54, + 0xa2, 0x72, 0x24, 0xd8, 0x58, 0x2a, 0x02, 0xa8, 0x45, 0x55, 0xa2, 0xca, 0xaa, 0x18, 0xba, 0x9c, + 0xce, 0xf1, 0xd5, 0x39, 0x35, 0xe7, 0xb3, 0xee, 0x47, 0x95, 0xfc, 0x17, 0x8c, 0x8c, 0x1d, 0xf9, + 0x13, 0x18, 0x19, 0x3b, 0x56, 0x62, 0x61, 0x02, 0x94, 0x2c, 0xf0, 0x5f, 0x20, 0x9f, 0x63, 0x9c, + 0x54, 0x74, 0xbb, 0xfb, 0xbe, 0xf7, 0xbd, 0x7b, 0xef, 0x7b, 0x36, 0x74, 0x52, 0x29, 0xb4, 0xe8, + 0xe9, 0x59, 0x4a, 0x55, 0x2f, 0x25, 0x92, 0x70, 0xe5, 0xdb, 0x12, 0x7a, 0xa8, 0x69, 0x12, 0x51, + 0xc9, 0x59, 0xa2, 0xf3, 0x8a, 0x6f, 0x41, 0xbb, 0x5d, 0x3d, 0x66, 0x32, 0xc2, 0x29, 0x91, 0x7a, + 0xd6, 0xcb, 0xa7, 0x63, 0x11, 0x8b, 0xf2, 0x94, 0xa3, 0x77, 0xdd, 0x58, 0x88, 0x78, 0x42, 0x73, + 0x48, 0x68, 0xce, 0x7b, 0x91, 0x91, 0x44, 0x33, 0x91, 0xe4, 0x7d, 0xef, 0x0f, 0x80, 0xad, 0x37, + 0x22, 0x51, 0x34, 0x51, 0x46, 0x9d, 0xd8, 0x97, 0xd1, 0x01, 0xbc, 0x1f, 0x4e, 0xc4, 0xe8, 0xc2, + 0x01, 0x1d, 0xb0, 0xd7, 0x78, 0xf1, 0xd4, 0xff, 0xbf, 0x06, 0xbf, 0x9f, 0x81, 0xf2, 0x99, 0xfe, + 0xc6, 0xf5, 0x8f, 0x76, 0x25, 0xc8, 0xe7, 0xd0, 0x11, 0xac, 0xd1, 0x4b, 0x16, 0xd1, 0x64, 0x44, + 0x9d, 0x7b, 0x96, 0xa3, 0x7b, 0x17, 0xc7, 0xbb, 0x25, 0x6e, 0x8d, 0xe6, 0xdf, 0x34, 0x3a, 0x86, + 0xf5, 0x4b, 0x32, 0x61, 0x11, 0xd1, 0x42, 0x3a, 0x55, 0x4b, 0xf5, 0xec, 0x2e, 0xaa, 0x0f, 0x05, + 0x70, 0x8d, 0xab, 0x9c, 0xf7, 0x28, 0x6c, 0xac, 0x48, 0x46, 0x4f, 0x60, 0x9d, 0x93, 0x29, 0x0e, + 0x67, 0x9a, 0x2a, 0x6b, 0xb5, 0x1a, 0xd4, 0x38, 0x99, 0xf6, 0xb3, 0x3b, 0x7a, 0x04, 0x1f, 0x64, + 0xcd, 0x98, 0x28, 0xeb, 0xa0, 0x1a, 0x6c, 0x72, 0x32, 0x3d, 0x24, 0x0a, 0x75, 0xe0, 0x96, 0x66, + 0x9c, 0x62, 0x26, 0x34, 0xc1, 0x5c, 0x59, 0x51, 0xd5, 0x00, 0x66, 0xb5, 0xf7, 0x42, 0x93, 0x81, + 0xf2, 0xbe, 0x01, 0xb8, 0xbd, 0x6e, 0x0b, 0x3d, 0x87, 0x28, 0x63, 0x23, 0x31, 0xc5, 0x89, 0xe1, + 0xd8, 0x6e, 0xa9, 0x78, 0xb3, 0xc5, 0xc9, 0xf4, 0x75, 0x4c, 0x87, 0x86, 0x5b, 0x71, 0x0a, 0x0d, + 0xe0, 0x4e, 0x01, 0x2e, 0xc2, 0x5a, 0x6e, 0xf1, 0xb1, 0x9f, 0xa7, 0xe9, 0x17, 0x69, 0xfa, 0x6f, + 0x97, 0x80, 0x7e, 0x2d, 0x33, 0xfb, 0xe9, 0x67, 0x1b, 0x04, 0xdb, 0x39, 0x5f, 0xd1, 0x29, 0x9c, + 0x24, 0x86, 0x5b, 0xad, 0x4d, 0xeb, 0x64, 0x68, 0x38, 0xda, 0x87, 0x28, 0x95, 0x42, 0x9c, 0x63, + 0x2d, 0x19, 0x99, 0xe0, 0x94, 0x4a, 0x26, 0x22, 0x67, 0xc3, 0x8a, 0xda, 0xb1, 0x9d, 0xd3, 0xac, + 0x71, 0x62, 0xeb, 0xde, 0x01, 0x6c, 0xdd, 0x5a, 0x30, 0xf2, 0x60, 0x33, 0x35, 0x21, 0xbe, 0xa0, + 0x33, 0x6c, 0x13, 0x70, 0x40, 0xa7, 0xba, 0x57, 0x0f, 0x1a, 0xa9, 0x09, 0x8f, 0xe9, 0xec, 0x34, + 0x2b, 0xbd, 0xaa, 0x7d, 0xb9, 0x6a, 0x83, 0xdf, 0x57, 0x6d, 0xe0, 0x9d, 0xc1, 0xad, 0x23, 0xa2, + 0xc6, 0x34, 0x5a, 0x4e, 0x77, 0x61, 0xcb, 0xee, 0x01, 0xdf, 0x0e, 0xa1, 0x69, 0xcb, 0x83, 0x22, + 0x09, 0x0f, 0x36, 0x4b, 0x5c, 0x99, 0x47, 0xa3, 0x40, 0x1d, 0x12, 0xd5, 0x1f, 0x7e, 0x9e, 0xbb, + 0xe0, 0x7a, 0xee, 0x82, 0x9b, 0xb9, 0x0b, 0x7e, 0xcd, 0x5d, 0xf0, 0x71, 0xe1, 0x56, 0xbe, 0x2e, + 0x5c, 0x70, 0xb3, 0x70, 0x2b, 0xdf, 0x17, 0x6e, 0xe5, 0x6c, 0x3f, 0x66, 0x7a, 0x6c, 0x42, 0x7f, + 0x24, 0x78, 0xaf, 0xfc, 0x7e, 0x56, 0x8f, 0x2b, 0xbf, 0x60, 0xb8, 0x69, 0x2f, 0x2f, 0xff, 0x06, + 0x00, 0x00, 0xff, 0xff, 0x5d, 0x39, 0x60, 0xf5, 0x98, 0x03, 0x00, 0x00, } func (this *ConsensusParams) Equal(that interface{}) bool { @@ -471,6 +484,9 @@ func (this *EvidenceParams) Equal(that interface{}) bool { if this.MaxNum != that1.MaxNum { return false } + if this.ProofTrialPeriod != that1.ProofTrialPeriod { + return false + } return true } func (this *ValidatorParams) Equal(that interface{}) bool { @@ -640,6 +656,11 @@ func (m *EvidenceParams) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.ProofTrialPeriod != 0 { + i = encodeVarintParams(dAtA, i, uint64(m.ProofTrialPeriod)) + i-- + dAtA[i] = 0x20 + } if m.MaxNum != 0 { i = encodeVarintParams(dAtA, i, uint64(m.MaxNum)) i-- @@ -868,6 +889,9 @@ func (m *EvidenceParams) Size() (n int) { if m.MaxNum != 0 { n += 1 + sovParams(uint64(m.MaxNum)) } + if m.ProofTrialPeriod != 0 { + n += 1 + sovParams(uint64(m.ProofTrialPeriod)) + } return n } @@ -1269,6 +1293,25 @@ func (m *EvidenceParams) Unmarshal(dAtA []byte) error { break } } + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ProofTrialPeriod", wireType) + } + m.ProofTrialPeriod = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowParams + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ProofTrialPeriod |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipParams(dAtA[iNdEx:]) diff --git a/proto/types/params.proto b/proto/types/params.proto index bc8eeb9ab..9fc5406bc 100644 --- a/proto/types/params.proto +++ b/proto/types/params.proto @@ -52,6 +52,11 @@ message EvidenceParams { // each evidence (See MaxEvidenceBytes). The maximum number is MaxEvidencePerBlock. // Default is 50 uint32 max_num = 3; + + // Proof trial period dictates the time given for nodes accused of amnesia evidence, incorrectly + // voting twice in two different rounds to respond with their respective proofs. + // Default is half the max age in blocks: 50,000 + int64 proof_trial_period = 4; } // ValidatorParams restrict the public key types validators can use. diff --git a/state/store_test.go b/state/store_test.go index 04b952373..2d8e0067b 100644 --- a/state/store_test.go +++ b/state/store_test.go @@ -12,7 +12,9 @@ import ( abci "github.com/tendermint/tendermint/abci/types" cfg "github.com/tendermint/tendermint/config" + "github.com/tendermint/tendermint/crypto" "github.com/tendermint/tendermint/crypto/ed25519" + tmrand "github.com/tendermint/tendermint/libs/rand" tmstate "github.com/tendermint/tendermint/proto/state" tmproto "github.com/tendermint/tendermint/proto/types" sm "github.com/tendermint/tendermint/state" @@ -99,7 +101,7 @@ func TestPruneStates(t *testing.T) { // Generate a bunch of state data. Validators change for heights ending with 3, and // parameters when ending with 5. - validator := &types.Validator{Address: []byte{1, 2, 3}, VotingPower: 100, PubKey: pk} + validator := &types.Validator{Address: tmrand.Bytes(crypto.AddressSize), VotingPower: 100, PubKey: pk} validatorSet := &types.ValidatorSet{ Validators: []*types.Validator{validator}, Proposer: validator, diff --git a/state/validation.go b/state/validation.go index 018b8d295..609f66b50 100644 --- a/state/validation.go +++ b/state/validation.go @@ -142,6 +142,16 @@ func validateBlock(evidencePool EvidencePool, stateDB dbm.DB, state State, block continue } } + // if we don't already have amnesia evidence we need to add it to start our own timer unless + // a) a valid polc has already been attached + // b) the accused node voted back on an earlier round + if ae, ok := ev.(types.AmnesiaEvidence); ok && ae.Polc.IsAbsent() && ae.PotentialAmnesiaEvidence.VoteA.Round < + ae.PotentialAmnesiaEvidence.VoteB.Round { + if err := evidencePool.AddEvidence(ae); err != nil { + return types.NewErrEvidenceInvalid(ev, + fmt.Errorf("unknown amnesia evidence, trying to add to evidence pool, err: %w", err)) + } + } var header *types.Header if _, ok := ev.(*types.LunaticValidatorEvidence); ok { @@ -154,6 +164,7 @@ func validateBlock(evidencePool EvidencePool, stateDB dbm.DB, state State, block if err := VerifyEvidence(stateDB, state, ev, header); err != nil { return types.NewErrEvidenceInvalid(ev, err) } + } // NOTE: We can't actually verify it's the right proposer because we dont @@ -197,7 +208,7 @@ func VerifyEvidence(stateDB dbm.DB, state State, evidence types.Evidence, commit state.LastBlockTime.Add(evidenceParams.MaxAgeDuration), ) } - if ev, ok := evidence.(*types.LunaticValidatorEvidence); ok { + if ev, ok := evidence.(types.LunaticValidatorEvidence); ok { if err := ev.VerifyHeader(committedHeader); err != nil { return err } @@ -216,7 +227,9 @@ func VerifyEvidence(stateDB dbm.DB, state State, evidence types.Evidence, commit // For PhantomValidatorEvidence, check evidence.Address was not part of the // validator set at height evidence.Height, but was a validator before OR // after. - if phve, ok := evidence.(*types.PhantomValidatorEvidence); ok { + if phve, ok := evidence.(types.PhantomValidatorEvidence); ok { + // confirm that it hasn't been forged + _, val = valset.GetByAddress(addr) if val != nil { return fmt.Errorf("address %X was a validator at height %d", addr, evidence.Height()) @@ -224,9 +237,9 @@ func VerifyEvidence(stateDB dbm.DB, state State, evidence types.Evidence, commit // check if last height validator was in the validator set is within // MaxAgeNumBlocks. - if ageNumBlocks > 0 && phve.LastHeightValidatorWasInSet <= ageNumBlocks { + if height-phve.LastHeightValidatorWasInSet > evidenceParams.MaxAgeNumBlocks { return fmt.Errorf("last time validator was in the set at height %d, min: %d", - phve.LastHeightValidatorWasInSet, ageNumBlocks+1) + phve.LastHeightValidatorWasInSet, height-phve.LastHeightValidatorWasInSet) } valset, err := LoadValidators(stateDB, phve.LastHeightValidatorWasInSet) @@ -240,6 +253,16 @@ func VerifyEvidence(stateDB dbm.DB, state State, evidence types.Evidence, commit return fmt.Errorf("phantom validator %X not found", addr) } } else { + if ae, ok := evidence.(types.AmnesiaEvidence); ok { + // check the validator set against the polc to make sure that a majority of valid votes was reached + if !ae.Polc.IsAbsent() { + err = ae.Polc.ValidateVotes(valset, state.ChainID) + if err != nil { + return fmt.Errorf("amnesia evidence contains invalid polc, err: %w", err) + } + } + } + // For all other types, expect evidence.Address to be a validator at height // evidence.Height. _, val = valset.GetByAddress(addr) diff --git a/state/validation_test.go b/state/validation_test.go index ad2c63599..89f6e311f 100644 --- a/state/validation_test.go +++ b/state/validation_test.go @@ -1,16 +1,22 @@ package state_test import ( + "fmt" "testing" "time" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" + "github.com/tendermint/tendermint/crypto" + "github.com/tendermint/tendermint/libs/bytes" + "github.com/tendermint/tendermint/proto/version" "github.com/tendermint/tendermint/crypto/ed25519" "github.com/tendermint/tendermint/crypto/tmhash" "github.com/tendermint/tendermint/libs/log" memmock "github.com/tendermint/tendermint/mempool/mock" + protostate "github.com/tendermint/tendermint/proto/state" tmproto "github.com/tendermint/tendermint/proto/types" sm "github.com/tendermint/tendermint/state" "github.com/tendermint/tendermint/state/mocks" @@ -285,8 +291,8 @@ func TestValidateFailBlockOnCommittedEvidence(t *testing.T) { block.EvidenceHash = block.Evidence.Hash() err := blockExec.ValidateBlock(state, block) - require.Error(t, err) - require.IsType(t, err, &types.ErrEvidenceInvalid{}) + assert.Error(t, err) + assert.IsType(t, err, &types.ErrEvidenceInvalid{}) } func TestValidateAlreadyPendingEvidence(t *testing.T) { @@ -314,7 +320,7 @@ func TestValidateAlreadyPendingEvidence(t *testing.T) { block.EvidenceHash = block.Evidence.Hash() err := blockExec.ValidateBlock(state, block) - require.NoError(t, err) + assert.NoError(t, err) } func TestValidateDuplicateEvidenceShouldFail(t *testing.T) { @@ -336,5 +342,274 @@ func TestValidateDuplicateEvidenceShouldFail(t *testing.T) { block.EvidenceHash = block.Evidence.Hash() err := blockExec.ValidateBlock(state, block) - require.Error(t, err) + assert.Error(t, err) +} + +var blockID = types.BlockID{ + Hash: []byte("aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa"), + PartsHeader: types.PartSetHeader{ + Total: 1, + Hash: []byte("aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa"), + }, +} + +func TestValidateAmnesiaEvidence(t *testing.T) { + var height int64 = 1 + state, stateDB, vals := makeState(1, int(height)) + addr, val := state.Validators.GetByIndex(0) + voteA := makeVote(height, 1, 0, addr, blockID) + err := vals[val.Address.String()].SignVote(chainID, voteA) + require.NoError(t, err) + voteB := makeVote(height, 2, 0, addr, types.BlockID{}) + err = vals[val.Address.String()].SignVote(chainID, voteB) + require.NoError(t, err) + ae := types.AmnesiaEvidence{ + PotentialAmnesiaEvidence: types.PotentialAmnesiaEvidence{ + VoteA: voteA, + VoteB: voteB, + }, + Polc: types.EmptyPOLC(), + } + + evpool := &mocks.EvidencePool{} + evpool.On("IsPending", ae).Return(false) + evpool.On("IsCommitted", ae).Return(false) + evpool.On("AddEvidence", ae).Return(fmt.Errorf("test error")) + + blockExec := sm.NewBlockExecutor( + stateDB, log.TestingLogger(), + nil, + nil, + evpool) + // A block with a couple pieces of evidence passes. + block := makeBlock(state, height) + block.Evidence.Evidence = []types.Evidence{ae} + block.EvidenceHash = block.Evidence.Hash() + err = blockExec.ValidateBlock(state, block) + + errMsg := "Invalid evidence: unknown amnesia evidence, trying to add to evidence pool, err: test error" + if assert.Error(t, err) { + assert.Equal(t, err.Error()[:len(errMsg)], errMsg) + } +} + +func TestVerifyEvidenceWrongAddress(t *testing.T) { + var height int64 = 1 + state, stateDB, _ := makeState(1, int(height)) + randomAddr := []byte("wrong address") + ev := types.NewMockEvidence(height, defaultTestTime, randomAddr) + + blockExec := sm.NewBlockExecutor( + stateDB, log.TestingLogger(), + nil, + nil, + sm.MockEvidencePool{}) + // A block with a couple pieces of evidence passes. + block := makeBlock(state, height) + block.Evidence.Evidence = []types.Evidence{ev} + block.EvidenceHash = block.Evidence.Hash() + err := blockExec.ValidateBlock(state, block) + errMsg := "Invalid evidence: address 77726F6E672061646472657373 was not a validator at height 1" + if assert.Error(t, err) { + assert.Equal(t, err.Error()[:len(errMsg)], errMsg) + } +} + +func TestVerifyEvidenceExpiredEvidence(t *testing.T) { + var height int64 = 4 + state, stateDB, _ := makeState(1, int(height)) + state.ConsensusParams.Evidence.MaxAgeNumBlocks = 1 + addr, _ := state.Validators.GetByIndex(0) + ev := types.NewMockEvidence(1, defaultTestTime, addr) + err := sm.VerifyEvidence(stateDB, state, ev, nil) + errMsg := "evidence from height 1 (created at: 2019-01-01 00:00:00 +0000 UTC) is too old" + if assert.Error(t, err) { + assert.Equal(t, err.Error()[:len(errMsg)], errMsg) + } +} + +func TestVerifyEvidenceWithAmnesiaEvidence(t *testing.T) { + var height int64 = 1 + state, stateDB, vals := makeState(4, int(height)) + addr, val := state.Validators.GetByIndex(0) + addr2, val2 := state.Validators.GetByIndex(1) + voteA := makeVote(height, 1, 0, addr, types.BlockID{}) + err := vals[val.Address.String()].SignVote(chainID, voteA) + require.NoError(t, err) + voteB := makeVote(height, 2, 0, addr, blockID) + err = vals[val.Address.String()].SignVote(chainID, voteB) + require.NoError(t, err) + voteC := makeVote(height, 2, 1, addr2, blockID) + err = vals[val2.Address.String()].SignVote(chainID, voteC) + require.NoError(t, err) + //var ae types.Evidence + badAe := types.AmnesiaEvidence{ + PotentialAmnesiaEvidence: types.PotentialAmnesiaEvidence{ + VoteA: voteA, + VoteB: voteB, + }, + Polc: types.ProofOfLockChange{ + Votes: []types.Vote{*voteC}, + PubKey: val.PubKey, + }, + } + err = sm.VerifyEvidence(stateDB, state, badAe, nil) + if assert.Error(t, err) { + assert.Equal(t, err.Error(), "amnesia evidence contains invalid polc, err: "+ + "invalid commit -- insufficient voting power: got 1000, needed more than 2667") + } + addr3, val3 := state.Validators.GetByIndex(2) + voteD := makeVote(height, 2, 2, addr3, blockID) + err = vals[val3.Address.String()].SignVote(chainID, voteD) + require.NoError(t, err) + addr4, val4 := state.Validators.GetByIndex(3) + voteE := makeVote(height, 2, 3, addr4, blockID) + err = vals[val4.Address.String()].SignVote(chainID, voteE) + require.NoError(t, err) + + goodAe := types.AmnesiaEvidence{ + PotentialAmnesiaEvidence: types.PotentialAmnesiaEvidence{ + VoteA: voteA, + VoteB: voteB, + }, + Polc: types.ProofOfLockChange{ + Votes: []types.Vote{*voteC, *voteD, *voteE}, + PubKey: val.PubKey, + }, + } + err = sm.VerifyEvidence(stateDB, state, goodAe, nil) + assert.NoError(t, err) + + goodAe = types.AmnesiaEvidence{ + PotentialAmnesiaEvidence: types.PotentialAmnesiaEvidence{ + VoteA: voteA, + VoteB: voteB, + }, + Polc: types.EmptyPOLC(), + } + err = sm.VerifyEvidence(stateDB, state, goodAe, nil) + assert.NoError(t, err) + +} + +func TestVerifyEvidenceWithLunaticValidatorEvidence(t *testing.T) { + state, stateDB, vals := makeState(4, 4) + state.ConsensusParams.Evidence.MaxAgeNumBlocks = 1 + addr, val := state.Validators.GetByIndex(0) + h := &types.Header{ + Version: version.Consensus{Block: 1, App: 2}, + ChainID: chainID, + Height: 3, + Time: defaultTestTime, + LastBlockID: blockID, + LastCommitHash: tmhash.Sum([]byte("last_commit_hash")), + DataHash: tmhash.Sum([]byte("data_hash")), + ValidatorsHash: tmhash.Sum([]byte("validators_hash")), + NextValidatorsHash: tmhash.Sum([]byte("next_validators_hash")), + ConsensusHash: tmhash.Sum([]byte("consensus_hash")), + AppHash: tmhash.Sum([]byte("app_hash")), + LastResultsHash: tmhash.Sum([]byte("last_results_hash")), + EvidenceHash: tmhash.Sum([]byte("evidence_hash")), + ProposerAddress: crypto.AddressHash([]byte("proposer_address")), + } + vote := makeVote(3, 1, 0, addr, blockID) + err := vals[val.Address.String()].SignVote(chainID, vote) + require.NoError(t, err) + ev := types.LunaticValidatorEvidence{ + Header: h, + Vote: vote, + InvalidHeaderField: "ConsensusHash", + } + err = ev.ValidateBasic() + require.NoError(t, err) + err = sm.VerifyEvidence(stateDB, state, ev, h) + if assert.Error(t, err) { + assert.Equal(t, "ConsensusHash matches committed hash", err.Error()) + } +} + +func TestVerifyEvidenceWithPhantomValidatorEvidence(t *testing.T) { + state, stateDB, vals := makeState(4, 4) + state.ConsensusParams.Evidence.MaxAgeNumBlocks = 1 + addr, val := state.Validators.GetByIndex(0) + vote := makeVote(3, 1, 0, addr, blockID) + err := vals[val.Address.String()].SignVote(chainID, vote) + require.NoError(t, err) + ev := types.PhantomValidatorEvidence{ + Vote: vote, + LastHeightValidatorWasInSet: 1, + } + err = ev.ValidateBasic() + require.NoError(t, err) + err = sm.VerifyEvidence(stateDB, state, ev, nil) + if assert.Error(t, err) { + assert.Equal(t, "address 576585A00DD4D58318255611D8AAC60E8E77CB32 was a validator at height 3", err.Error()) + } + + privVal := types.NewMockPV() + pubKey, _ := privVal.GetPubKey() + vote2 := makeVote(3, 1, 0, pubKey.Address(), blockID) + err = privVal.SignVote(chainID, vote2) + require.NoError(t, err) + ev = types.PhantomValidatorEvidence{ + Vote: vote2, + LastHeightValidatorWasInSet: 1, + } + err = ev.ValidateBasic() + assert.NoError(t, err) + err = sm.VerifyEvidence(stateDB, state, ev, nil) + if assert.Error(t, err) { + assert.Equal(t, "last time validator was in the set at height 1, min: 2", err.Error()) + } + + ev = types.PhantomValidatorEvidence{ + Vote: vote2, + LastHeightValidatorWasInSet: 2, + } + err = ev.ValidateBasic() + assert.NoError(t, err) + err = sm.VerifyEvidence(stateDB, state, ev, nil) + errMsg := "phantom validator" + if assert.Error(t, err) { + assert.Equal(t, errMsg, err.Error()[:len(errMsg)]) + } + + vals2, err := sm.LoadValidators(stateDB, 2) + require.NoError(t, err) + vals2.Validators = append(vals2.Validators, types.NewValidator(pubKey, 1000)) + valKey := []byte("validatorsKey:2") + protoVals, err := vals2.ToProto() + require.NoError(t, err) + valInfo := &protostate.ValidatorsInfo{ + LastHeightChanged: 2, + ValidatorSet: protoVals, + } + + bz, err := valInfo.Marshal() + require.NoError(t, err) + + stateDB.Set(valKey, bz) + ev = types.PhantomValidatorEvidence{ + Vote: vote2, + LastHeightValidatorWasInSet: 2, + } + err = ev.ValidateBasic() + assert.NoError(t, err) + err = sm.VerifyEvidence(stateDB, state, ev, nil) + if !assert.NoError(t, err) { + t.Log(err) + } + +} + +func makeVote(height int64, round, index int32, addr bytes.HexBytes, blockID types.BlockID) *types.Vote { + return &types.Vote{ + Type: tmproto.SignedMsgType(2), + Height: height, + Round: round, + BlockID: blockID, + Timestamp: time.Now(), + ValidatorAddress: addr, + ValidatorIndex: index, + } } diff --git a/tools/tm-signer-harness/internal/test_harness_test.go b/tools/tm-signer-harness/internal/test_harness_test.go index 82db9949e..28d0d61c4 100644 --- a/tools/tm-signer-harness/internal/test_harness_test.go +++ b/tools/tm-signer-harness/internal/test_harness_test.go @@ -48,7 +48,8 @@ const ( "evidence": { "max_age_num_blocks": "100000", "max_age_duration": "172800000000000", - "max_num_evidence": 50 + "max_num": 50, + "proof_trial_period": "50000" }, "validator": { "pub_key_types": [ diff --git a/types/evidence.go b/types/evidence.go index 39b16064d..6c59bf93a 100644 --- a/types/evidence.go +++ b/types/evidence.go @@ -165,8 +165,9 @@ func EvidenceToProto(evidence Evidence) (*tmproto.Evidence, error) { tp := &tmproto.Evidence{ Sum: &tmproto.Evidence_PotentialAmnesiaEvidence{ PotentialAmnesiaEvidence: &tmproto.PotentialAmnesiaEvidence{ - VoteA: voteA, - VoteB: voteB, + VoteA: voteA, + VoteB: voteB, + HeightStamp: evi.HeightStamp, }, }, } @@ -179,13 +180,20 @@ func EvidenceToProto(evidence Evidence) (*tmproto.Evidence, error) { tp := &tmproto.Evidence{ Sum: &tmproto.Evidence_PotentialAmnesiaEvidence{ PotentialAmnesiaEvidence: &tmproto.PotentialAmnesiaEvidence{ - VoteA: voteA, - VoteB: voteB, + VoteA: voteA, + VoteB: voteB, + HeightStamp: evi.HeightStamp, }, }, } - return tp, nil + + case AmnesiaEvidence: + return AmnesiaEvidenceToProto(evi) + + case *AmnesiaEvidence: + return AmnesiaEvidenceToProto(*evi) + case MockEvidence: if err := evi.ValidateBasic(); err != nil { return nil, err @@ -284,21 +292,24 @@ func EvidenceFromProto(evidence *tmproto.Evidence) (Evidence, error) { return &tp, tp.ValidateBasic() case *tmproto.Evidence_PotentialAmnesiaEvidence: - voteA, err := VoteFromProto(evi.PotentialAmnesiaEvidence.GetVoteA()) + return PotentialAmnesiaEvidenceFromProto(evi.PotentialAmnesiaEvidence) + + case *tmproto.Evidence_AmnesiaEvidence: + pae, err := PotentialAmnesiaEvidenceFromProto(evi.AmnesiaEvidence.PotentialAmnesiaEvidence) if err != nil { return nil, err } - - voteB, err := VoteFromProto(evi.PotentialAmnesiaEvidence.GetVoteB()) + polc, err := ProofOfLockChangeFromProto(evi.AmnesiaEvidence.Polc) if err != nil { return nil, err } - tp := PotentialAmnesiaEvidence{ - VoteA: voteA, - VoteB: voteB, + + tp := AmnesiaEvidence{ + PotentialAmnesiaEvidence: *pae, + Polc: *polc, } - return &tp, tp.ValidateBasic() + return tp, tp.ValidateBasic() case *tmproto.Evidence_MockEvidence: me := MockEvidence{ EvidenceHeight: evi.MockEvidence.GetEvidenceHeight(), @@ -328,6 +339,7 @@ func RegisterEvidences(cdc *amino.Codec) { cdc.RegisterConcrete(&PhantomValidatorEvidence{}, "tendermint/PhantomValidatorEvidence", nil) cdc.RegisterConcrete(&LunaticValidatorEvidence{}, "tendermint/LunaticValidatorEvidence", nil) cdc.RegisterConcrete(&PotentialAmnesiaEvidence{}, "tendermint/PotentialAmnesiaEvidence", nil) + cdc.RegisterConcrete(&AmnesiaEvidence{}, "tendermint/AmnesiaEvidence", nil) } func init() { @@ -336,6 +348,7 @@ func init() { tmjson.RegisterType(&PhantomValidatorEvidence{}, "tendermint/PhantomValidatorEvidence") tmjson.RegisterType(&LunaticValidatorEvidence{}, "tendermint/LunaticValidatorEvidence") tmjson.RegisterType(&PotentialAmnesiaEvidence{}, "tendermint/PotentialAmnesiaEvidence") + tmjson.RegisterType(&AmnesiaEvidence{}, "tendermint/AmnesiaEvidence") } func RegisterMockEvidences(cdc *amino.Codec) { @@ -654,10 +667,26 @@ OUTER_LOOP: } else { // if H1.Round != H2.Round we need to run full detection procedure => not // immediately slashable. - evList = append(evList, &PotentialAmnesiaEvidence{ - VoteA: ev.H1.Commit.GetVote(int32(i)), - VoteB: ev.H2.Commit.GetVote(int32(j)), - }) + firstVote := ev.H1.Commit.GetVote(int32(i)) + secondVote := ev.H2.Commit.GetVote(int32(j)) + var newEv PotentialAmnesiaEvidence + if firstVote.Timestamp.Before(secondVote.Timestamp) { + newEv = PotentialAmnesiaEvidence{ + VoteA: firstVote, + VoteB: secondVote, + } + } else { + newEv = PotentialAmnesiaEvidence{ + VoteA: secondVote, + VoteB: firstVote, + } + } + // has the validator incorrectly voted for a previous round + if newEv.VoteA.Round > newEv.VoteB.Round { + evList = append(evList, MakeAmnesiaEvidence(newEv, EmptyPOLC())) + } else { + evList = append(evList, newEv) + } } i++ @@ -1001,9 +1030,16 @@ func (e LunaticValidatorEvidence) VerifyHeader(committedHeader *Header) error { //------------------------------------------- +// PotentialAmnesiaEvidence is constructed when a validator votes on two different blocks at different rounds +// in the same height. PotentialAmnesiaEvidence can then evolve into AmensiaEvidence if the indicted validator +// is incapable of providing the proof of lock change that validates voting twice in the allotted trial period. +// Heightstamp is used for each node to keep a track of how much time has passed so as to know when the trial period +// is finished and is set when the node first receives the evidence. type PotentialAmnesiaEvidence struct { VoteA *Vote `json:"vote_a"` VoteB *Vote `json:"vote_b"` + + HeightStamp int64 } var _ Evidence = PotentialAmnesiaEvidence{} @@ -1013,10 +1049,7 @@ func (e PotentialAmnesiaEvidence) Height() int64 { } func (e PotentialAmnesiaEvidence) Time() time.Time { - if e.VoteA.Timestamp.Before(e.VoteB.Timestamp) { - return e.VoteA.Timestamp - } - return e.VoteB.Timestamp + return e.VoteA.Timestamp } func (e PotentialAmnesiaEvidence) Address() []byte { @@ -1053,9 +1086,11 @@ func (e PotentialAmnesiaEvidence) Verify(chainID string, pubKey crypto.PubKey) e func (e PotentialAmnesiaEvidence) Equal(ev Evidence) bool { switch e2 := ev.(type) { case PotentialAmnesiaEvidence: - return bytes.Equal(e.Hash(), e2.Hash()) + return e.Height() == e2.Height() && e.VoteA.Round == e2.VoteA.Round && e.VoteB.Round == e2.VoteB.Round && + bytes.Equal(e.Address(), e2.Address()) case *PotentialAmnesiaEvidence: - return bytes.Equal(e.Hash(), e2.Hash()) + return e.Height() == e2.Height() && e.VoteA.Round == e2.VoteA.Round && e.VoteB.Round == e2.VoteB.Round && + bytes.Equal(e.Address(), e2.Address()) default: return false } @@ -1071,10 +1106,6 @@ func (e PotentialAmnesiaEvidence) ValidateBasic() error { if err := e.VoteB.ValidateBasic(); err != nil { return fmt.Errorf("invalid VoteB: %v", err) } - // Enforce Votes are lexicographically sorted on blockID - if strings.Compare(e.VoteA.BlockID.Key(), e.VoteB.BlockID.Key()) >= 0 { - return errors.New("amnesia votes in invalid order") - } // H/S must be the same if e.VoteA.Height != e.VoteB.Height || @@ -1083,9 +1114,10 @@ func (e PotentialAmnesiaEvidence) ValidateBasic() error { e.VoteA.Height, e.VoteA.Type, e.VoteB.Height, e.VoteB.Type) } - // R must be different - if e.VoteA.Round == e.VoteB.Round { - return fmt.Errorf("expected votes from different rounds, got %d", e.VoteA.Round) + // Enforce that vote A came before vote B + if e.VoteA.Timestamp.After(e.VoteB.Timestamp) { + return fmt.Errorf("vote A should have a timestamp before vote B, but got %s > %s", + e.VoteA.Timestamp, e.VoteB.Timestamp) } // Address must be the same @@ -1121,6 +1153,21 @@ func (e PotentialAmnesiaEvidence) String() string { return fmt.Sprintf("PotentialAmnesiaEvidence{VoteA: %v, VoteB: %v}", e.VoteA, e.VoteB) } +// Primed finds whether the PotentialAmnesiaEvidence is ready to be upgraded to Amnesia Evidence. It is decided if +// either the prosecuted node voted in the past or if the allocated trial period has expired without a proof of lock +// change having been provided. +func (e PotentialAmnesiaEvidence) Primed(trialPeriod, currentHeight int64) bool { + // voted in the past can be instantly punishable + if e.VoteA.Round > e.VoteB.Round { + return true + } + // has the trial period expired + if e.HeightStamp > 0 { + return e.HeightStamp+trialPeriod <= currentHeight + } + return false +} + // ProofOfLockChange (POLC) proves that a node followed the consensus protocol and voted for a precommit in two // different rounds because the node received a majority of votes for a different block in the latter round. In cases of // amnesia evidence, a suspected node will need ProofOfLockChange to prove that the node did not break protocol. @@ -1151,6 +1198,15 @@ func makePOLCFromVoteSet(voteSet *VoteSet, pubKey crypto.PubKey, blockID BlockID } } +// EmptyPOLC returns an empty polc. This is used when no polc has been provided in the allocated trial period time +// and the node now needs to move to upgrading to AmnesiaEvidence and hence uses an empty polc +func EmptyPOLC() ProofOfLockChange { + return ProofOfLockChange{ + nil, + nil, + } +} + func (e ProofOfLockChange) Height() int64 { return e.Votes[0].Height } @@ -1178,23 +1234,38 @@ func (e ProofOfLockChange) BlockID() BlockID { return e.Votes[0].BlockID } -// In order for a ProofOfLockChange to be valid, a validator must have received +2/3 majority of votes -// MajorityOfVotes checks that there were sufficient votes in order to change locks -func (e ProofOfLockChange) MajorityOfVotes(valSet *ValidatorSet) bool { +// ValidateVotes checks the polc against the validator set of that height. The function makes sure that the polc +// contains a majority of votes and that each +func (e ProofOfLockChange) ValidateVotes(valSet *ValidatorSet, chainID string) error { + if e.IsAbsent() { + return errors.New("polc is empty") + } talliedVotingPower := int64(0) votingPowerNeeded := valSet.TotalVotingPower() * 2 / 3 - for _, validator := range valSet.Validators { - for _, vote := range e.Votes { + for _, vote := range e.Votes { + exists := false + for _, validator := range valSet.Validators { if bytes.Equal(validator.Address, vote.ValidatorAddress) { - talliedVotingPower += validator.VotingPower - - if talliedVotingPower > votingPowerNeeded { - return true + exists = true + if !validator.PubKey.VerifyBytes(vote.SignBytes(chainID), vote.Signature) { + return fmt.Errorf("cannot verify vote (from validator: %d) against signature: %v", + vote.ValidatorIndex, vote.Signature) } + + talliedVotingPower += validator.VotingPower } } + if !exists { + return fmt.Errorf("vote was not from a validator in this set: %v", vote.String()) + } } - return false + if talliedVotingPower <= votingPowerNeeded { + return ErrNotEnoughVotingPowerSigned{ + Got: talliedVotingPower, + Needed: votingPowerNeeded + 1, + } + } + return nil } func (e ProofOfLockChange) Equal(e2 ProofOfLockChange) bool { @@ -1203,6 +1274,11 @@ func (e ProofOfLockChange) Equal(e2 ProofOfLockChange) bool { } func (e ProofOfLockChange) ValidateBasic() error { + // first check if the polc is absent / empty + if e.IsAbsent() { + return nil + } + if e.PubKey == nil { return errors.New("missing public key") } @@ -1240,7 +1316,7 @@ func (e ProofOfLockChange) ValidateBasic() error { if bytes.Equal(vote.ValidatorAddress.Bytes(), e.PubKey.Address().Bytes()) { return fmt.Errorf("vote validator address cannot be the same as the public key address: %X all votes %v", - vote.ValidatorAddress.Bytes(), e.Votes) + vote.ValidatorAddress.Bytes(), e.PubKey.Address().Bytes()) } for i := idx + 1; i < len(e.Votes); i++ { @@ -1254,10 +1330,18 @@ func (e ProofOfLockChange) ValidateBasic() error { } func (e ProofOfLockChange) String() string { + if e.IsAbsent() { + return "Empty ProofOfLockChange" + } return fmt.Sprintf("ProofOfLockChange {Address: %X, Height: %d, Round: %d", e.Address(), e.Height(), e.Votes[0].Round) } +// IsAbsent checks if the polc is empty +func (e ProofOfLockChange) IsAbsent() bool { + return e.Votes == nil && e.PubKey == nil +} + func (e *ProofOfLockChange) ToProto() (*tmproto.ProofOfLockChange, error) { if e == nil { return nil, errors.New("nil proof of lock change") @@ -1265,6 +1349,14 @@ func (e *ProofOfLockChange) ToProto() (*tmproto.ProofOfLockChange, error) { plc := new(tmproto.ProofOfLockChange) vpb := make([]*tmproto.Vote, len(e.Votes)) + // if absent create empty proto polc + if e.IsAbsent() { + return plc, nil + } + + if e.Votes == nil { + return nil, errors.New("polc is not absent but has no votes") + } for i, v := range e.Votes { pb := v.ToProto() if pb != nil { @@ -1282,12 +1374,105 @@ func (e *ProofOfLockChange) ToProto() (*tmproto.ProofOfLockChange, error) { return plc, nil } +// AmnesiaEvidence is the progression of PotentialAmnesiaEvidence and is used to prove an infringement of the +// Tendermint consensus when a validator incorrectly sends a vote in a later round without correctly changing the lock +type AmnesiaEvidence struct { + PotentialAmnesiaEvidence + Polc ProofOfLockChange +} + +// Height, Time, Address and Verify functions are all inherited by the PotentialAmnesiaEvidence struct +var _ Evidence = &AmnesiaEvidence{} +var _ Evidence = AmnesiaEvidence{} + +func MakeAmnesiaEvidence(pe PotentialAmnesiaEvidence, proof ProofOfLockChange) AmnesiaEvidence { + return AmnesiaEvidence{ + pe, + proof, + } +} + +func (e AmnesiaEvidence) Equal(ev Evidence) bool { + e2, ok := ev.(AmnesiaEvidence) + if !ok { + return false + } + return e.PotentialAmnesiaEvidence.Equal(e2.PotentialAmnesiaEvidence) +} + +func (e AmnesiaEvidence) ValidateBasic() error { + if err := e.PotentialAmnesiaEvidence.ValidateBasic(); err != nil { + return fmt.Errorf("invalid potential amnesia evidence: %w", err) + } + if !e.Polc.IsAbsent() { + if err := e.Polc.ValidateBasic(); err != nil { + return fmt.Errorf("invalid proof of lock change: %w", err) + } + + if !bytes.Equal(e.PotentialAmnesiaEvidence.Address(), e.Polc.Address()) { + return fmt.Errorf("validator addresses do not match (%X - %X)", e.PotentialAmnesiaEvidence.Address(), + e.Polc.Address()) + } + + if e.PotentialAmnesiaEvidence.Height() != e.Polc.Height() { + return fmt.Errorf("heights do not match (%d - %d)", e.PotentialAmnesiaEvidence.Height(), + e.Polc.Height()) + } + + if e.Polc.Round() <= e.VoteA.Round || e.Polc.Round() > e.VoteB.Round { + return fmt.Errorf("polc must be between %d and %d (inclusive)", e.VoteA.Round+1, e.VoteB.Round) + } + + if !e.Polc.BlockID().Equals(e.PotentialAmnesiaEvidence.VoteB.BlockID) && !e.Polc.BlockID().IsZero() { + return fmt.Errorf("polc must be either for a nil block or for the same block as the second vote: %v != %v", + e.Polc.BlockID(), e.PotentialAmnesiaEvidence.VoteB.BlockID) + } + + if e.Polc.Time().After(e.PotentialAmnesiaEvidence.VoteB.Timestamp) { + return fmt.Errorf("validator voted again before receiving a majority of votes for the new block: %v is after %v", + e.Polc.Time(), e.PotentialAmnesiaEvidence.VoteB.Timestamp) + } + } + return nil +} + +// ViolatedConsensus assess on the basis of the AmensiaEvidence whether the validator has violated the +// Tendermint consensus. Evidence must be validated first (see ValidateBasic). +// We are only interested in proving that the latter of the votes in terms of time was correctly done. +func (e AmnesiaEvidence) ViolatedConsensus() (bool, string) { + // a validator having voted cannot go back and vote on an earlier round + if e.PotentialAmnesiaEvidence.VoteA.Round > e.PotentialAmnesiaEvidence.VoteB.Round { + return true, "validator went back and voted on a previous round" + } + + // if empty, then no proof was provided to defend the validators actions + if e.Polc.IsAbsent() { + return true, "no proof of lock was provided" + } + + return false, "" +} + +func (e AmnesiaEvidence) String() string { + return fmt.Sprintf("AmnesiaEvidence{ %v, polc: %v }", e.PotentialAmnesiaEvidence, e.Polc) +} + func ProofOfLockChangeFromProto(pb *tmproto.ProofOfLockChange) (*ProofOfLockChange, error) { if pb == nil { return nil, errors.New("nil proof of lock change") } plc := new(ProofOfLockChange) + + // check if it is an empty polc + if pb.PubKey == nil && pb.Votes == nil { + return plc, nil + } + + if pb.Votes == nil { + return nil, errors.New("proofOfLockChange: is not absent but has no votes") + } + vpb := make([]Vote, len(pb.Votes)) for i, v := range pb.Votes { vi, err := VoteFromProto(v) @@ -1298,7 +1483,7 @@ func ProofOfLockChangeFromProto(pb *tmproto.ProofOfLockChange) (*ProofOfLockChan } if pb.PubKey == nil { - return nil, errors.New("proofOfLockChange: nil PubKey") + return nil, errors.New("proofOfLockChange: is not abest but has nil PubKey") } pk, err := cryptoenc.PubKeyFromProto(*pb.PubKey) if err != nil { @@ -1311,6 +1496,54 @@ func ProofOfLockChangeFromProto(pb *tmproto.ProofOfLockChange) (*ProofOfLockChan return plc, nil } +func PotentialAmnesiaEvidenceFromProto(pb *tmproto.PotentialAmnesiaEvidence) (*PotentialAmnesiaEvidence, error) { + voteA, err := VoteFromProto(pb.GetVoteA()) + if err != nil { + return nil, err + } + + voteB, err := VoteFromProto(pb.GetVoteB()) + if err != nil { + return nil, err + } + tp := PotentialAmnesiaEvidence{ + VoteA: voteA, + VoteB: voteB, + HeightStamp: pb.GetHeightStamp(), + } + + return &tp, tp.ValidateBasic() +} + +func AmnesiaEvidenceToProto(evi AmnesiaEvidence) (*tmproto.Evidence, error) { + ev, err := EvidenceToProto(evi.PotentialAmnesiaEvidence) + if err != nil { + return nil, err + } + + paepb := ev.GetPotentialAmnesiaEvidence() + if paepb == nil { + return nil, errors.New("provided evidence is not potential amnesia evidence") + } + + polc, err := evi.Polc.ToProto() + if err != nil { + return nil, err + } + + tp := &tmproto.Evidence{ + Sum: &tmproto.Evidence_AmnesiaEvidence{ + AmnesiaEvidence: &tmproto.AmnesiaEvidence{ + PotentialAmnesiaEvidence: paepb, + Polc: polc, + }, + }, + } + + return tp, nil + +} + //----------------------------------------------------------------- // UNSTABLE diff --git a/types/evidence_test.go b/types/evidence_test.go index 6f4488346..d26919b59 100644 --- a/types/evidence_test.go +++ b/types/evidence_test.go @@ -22,27 +22,6 @@ type voteData struct { var defaultVoteTime = time.Date(2019, 1, 1, 0, 0, 0, 0, time.UTC) -func makeVote( - t *testing.T, val PrivValidator, chainID string, valIndex int32, height int64, round int32, step int, blockID BlockID, - time time.Time) *Vote { - pubKey, err := val.GetPubKey() - require.NoError(t, err) - v := &Vote{ - ValidatorAddress: pubKey.Address(), - ValidatorIndex: valIndex, - Height: height, - Round: round, - Type: tmproto.SignedMsgType(step), - Timestamp: time, - BlockID: blockID, - } - err = val.SignVote(chainID, v) - if err != nil { - panic(err) - } - return v -} - func TestEvidence(t *testing.T) { val := NewMockPV() val2 := NewMockPV() @@ -368,8 +347,8 @@ func TestPotentialAmnesiaEvidence(t *testing.T) { ) ev := &PotentialAmnesiaEvidence{ - VoteA: vote2, - VoteB: vote1, + VoteA: vote1, + VoteB: vote2, } assert.Equal(t, height, ev.Height()) @@ -391,7 +370,7 @@ func TestPotentialAmnesiaEvidence(t *testing.T) { func TestProofOfLockChange(t *testing.T) { const ( - chainID = "TestProofOfLockChange" + chainID = "test_chain_id" height int64 = 37 ) // 1: valid POLC - nothing should fail @@ -402,9 +381,21 @@ func TestProofOfLockChange(t *testing.T) { assert.Equal(t, height, polc.Height()) assert.NoError(t, polc.ValidateBasic()) - assert.True(t, polc.MajorityOfVotes(valSet)) + assert.NoError(t, polc.ValidateVotes(valSet, chainID)) assert.NotEmpty(t, polc.String()) + // tamper with one of the votes + polc.Votes[0].Timestamp = time.Now().Add(1 * time.Second) + err = polc.ValidateVotes(valSet, chainID) + t.Log(err) + assert.Error(t, err) + + // remove a vote such that majority wasn't reached + polc.Votes = polc.Votes[1:] + err = polc.ValidateVotes(valSet, chainID) + t.Log(err) + assert.Error(t, err) + // test validate basic on a set of bad cases var badPOLCs []ProofOfLockChange // 2: node has already voted in next round @@ -438,6 +429,7 @@ func TestProofOfLockChange(t *testing.T) { for idx, polc := range badPOLCs { err := polc.ValidateBasic() + t.Logf("case: %d: %v", idx+2, err) assert.Error(t, err) if err == nil { t.Errorf("test no. %d failed", idx+2) @@ -446,6 +438,117 @@ func TestProofOfLockChange(t *testing.T) { } +func TestAmnesiaEvidence(t *testing.T) { + const ( + chainID = "test_chain_id" + height int64 = 37 + ) + + voteSet, valSet, privValidators, blockID := buildVoteSet(height, 1, 2, 7, 0, tmproto.PrecommitType) + + var ( + val = privValidators[7] + pubKey, _ = val.GetPubKey() + blockID2 = makeBlockID(tmhash.Sum([]byte("blockhash2")), math.MaxInt32, tmhash.Sum([]byte("partshash"))) + vote1 = makeVote(t, val, chainID, 7, height, 0, 2, blockID2, time.Now()) + vote2 = makeVote(t, val, chainID, 7, height, 1, 2, blockID, + time.Now().Add(time.Second)) + vote3 = makeVote(t, val, chainID, 7, height, 2, 2, blockID2, time.Now()) + polc = makePOLCFromVoteSet(voteSet, pubKey, blockID) + ) + + require.False(t, polc.IsAbsent()) + + pe := PotentialAmnesiaEvidence{ + VoteA: vote1, + VoteB: vote2, + } + + emptyAmnesiaEvidence := MakeAmnesiaEvidence(pe, EmptyPOLC()) + + assert.NoError(t, emptyAmnesiaEvidence.ValidateBasic()) + violated, reason := emptyAmnesiaEvidence.ViolatedConsensus() + if assert.True(t, violated) { + assert.Equal(t, reason, "no proof of lock was provided") + } + assert.NoError(t, emptyAmnesiaEvidence.Verify(chainID, pubKey)) + + completeAmnesiaEvidence := MakeAmnesiaEvidence(pe, polc) + + assert.NoError(t, completeAmnesiaEvidence.ValidateBasic()) + violated, reason = completeAmnesiaEvidence.ViolatedConsensus() + if !assert.False(t, violated) { + t.Log(reason) + } + assert.NoError(t, completeAmnesiaEvidence.Verify(chainID, pubKey)) + assert.NoError(t, completeAmnesiaEvidence.Polc.ValidateVotes(valSet, chainID)) + + assert.True(t, completeAmnesiaEvidence.Equal(emptyAmnesiaEvidence)) + assert.NotEmpty(t, completeAmnesiaEvidence.Hash()) + assert.NotEmpty(t, completeAmnesiaEvidence.Bytes()) + + pe2 := PotentialAmnesiaEvidence{ + VoteA: vote3, + VoteB: vote2, + } + + // validator has incorrectly voted for a previous round after voting for a later round + ae := MakeAmnesiaEvidence(pe2, EmptyPOLC()) + assert.NoError(t, ae.ValidateBasic()) + violated, reason = ae.ViolatedConsensus() + if assert.True(t, violated) { + assert.Equal(t, reason, "validator went back and voted on a previous round") + } + + var badAE []AmnesiaEvidence + // 1) Polc is at an incorrect height + voteSet, _, _ = buildVoteSetForBlock(height+1, 1, 2, 7, 0, tmproto.PrecommitType, blockID) + polc = makePOLCFromVoteSet(voteSet, pubKey, blockID) + badAE = append(badAE, MakeAmnesiaEvidence(pe, polc)) + // 2) Polc is of a later round + voteSet, _, _ = buildVoteSetForBlock(height, 2, 2, 7, 0, tmproto.PrecommitType, blockID) + polc = makePOLCFromVoteSet(voteSet, pubKey, blockID) + badAE = append(badAE, MakeAmnesiaEvidence(pe, polc)) + // 3) Polc has a different public key + voteSet, _, privValidators = buildVoteSetForBlock(height, 1, 2, 7, 0, tmproto.PrecommitType, blockID) + pubKey2, _ := privValidators[7].GetPubKey() + polc = makePOLCFromVoteSet(voteSet, pubKey2, blockID) + badAE = append(badAE, MakeAmnesiaEvidence(pe, polc)) + // 4) Polc has a different block ID + voteSet, _, _, blockID = buildVoteSet(height, 1, 2, 7, 0, tmproto.PrecommitType) + polc = makePOLCFromVoteSet(voteSet, pubKey, blockID) + badAE = append(badAE, MakeAmnesiaEvidence(pe, polc)) + + for idx, ae := range badAE { + t.Log(ae.ValidateBasic()) + if !assert.Error(t, ae.ValidateBasic()) { + t.Errorf("test no. %d failed", idx+1) + } + } + +} + +func makeVote( + t *testing.T, val PrivValidator, chainID string, valIndex int32, height int64, round int32, step int, blockID BlockID, + time time.Time) *Vote { + pubKey, err := val.GetPubKey() + require.NoError(t, err) + v := &Vote{ + ValidatorAddress: pubKey.Address(), + ValidatorIndex: valIndex, + Height: height, + Round: round, + Type: tmproto.SignedMsgType(step), + BlockID: blockID, + Timestamp: time, + } + err = val.SignVote(chainID, v) + if err != nil { + panic(err) + } + return v +} + func makeHeaderRandom() *Header { return &Header{ ChainID: tmrand.Str(12), @@ -591,24 +694,25 @@ func TestProofOfLockChangeProtoBuf(t *testing.T) { expErr2 bool }{ {"failure, empty key", ProofOfLockChange{Votes: []Vote{*v, *v2}}, true, true}, - {"failure empty ProofOfLockChange", ProofOfLockChange{}, true, true}, + {"failure, empty votes", ProofOfLockChange{PubKey: val3.PrivKey.PubKey()}, true, true}, + {"success empty ProofOfLockChange", EmptyPOLC(), false, false}, {"success", ProofOfLockChange{Votes: []Vote{*v, *v2}, PubKey: val3.PrivKey.PubKey()}, false, false}, } for _, tc := range testCases { tc := tc pbpolc, err := tc.polc.ToProto() if tc.expErr { - require.Error(t, err) + assert.Error(t, err, tc.msg) } else { - require.NoError(t, err) + assert.NoError(t, err, tc.msg) } c, err := ProofOfLockChangeFromProto(pbpolc) if !tc.expErr2 { - require.NoError(t, err, tc.msg) - require.Equal(t, &tc.polc, c, tc.msg) + assert.NoError(t, err, tc.msg) + assert.Equal(t, &tc.polc, c, tc.msg) } else { - require.Error(t, err, tc.msg) + assert.Error(t, err, tc.msg) } } } diff --git a/types/params.go b/types/params.go index e384435a4..022fd9561 100644 --- a/types/params.go +++ b/types/params.go @@ -42,12 +42,13 @@ func DefaultBlockParams() tmproto.BlockParams { } } -// DefaultEvidenceParams Params returns a default EvidenceParams. +// DefaultEvidenceParams returns a default EvidenceParams. func DefaultEvidenceParams() tmproto.EvidenceParams { return tmproto.EvidenceParams{ - MaxAgeNumBlocks: 100000, // 27.8 hrs at 1block/s - MaxAgeDuration: 48 * time.Hour, - MaxNum: 50, + MaxAgeNumBlocks: 100000, // 27.8 hrs at 1block/s + MaxAgeDuration: 48 * time.Hour, + MaxNum: 50, + ProofTrialPeriod: 50000, // half MaxAgeNumBlocks } } @@ -110,6 +111,16 @@ func ValidateConsensusParams(params tmproto.ConsensusParams) error { int64(params.Evidence.MaxNum)*MaxEvidenceBytes, params.Block.MaxBytes) } + if params.Evidence.ProofTrialPeriod <= 0 { + return fmt.Errorf("evidenceParams.ProofTrialPeriod must be grater than 0 if provided, Got %v", + params.Evidence.ProofTrialPeriod) + } + + if params.Evidence.ProofTrialPeriod >= params.Evidence.MaxAgeNumBlocks { + return fmt.Errorf("evidenceParams.ProofTrialPeriod must be smaller than evidenceParams.MaxAgeNumBlocks, %d > %d", + params.Evidence.ProofTrialPeriod, params.Evidence.MaxAgeDuration) + } + if len(params.Validator.PubKeyTypes) == 0 { return errors.New("len(Validator.PubKeyTypes) must be greater than 0") } diff --git a/types/params_test.go b/types/params_test.go index c2e14b191..943ca8d55 100644 --- a/types/params_test.go +++ b/types/params_test.go @@ -23,24 +23,24 @@ func TestConsensusParamsValidation(t *testing.T) { valid bool }{ // test block params - 0: {makeParams(1, 0, 10, 1, 0, valEd25519), true}, - 1: {makeParams(0, 0, 10, 1, 0, valEd25519), false}, - 2: {makeParams(47*1024*1024, 0, 10, 1, 0, valEd25519), true}, - 3: {makeParams(10, 0, 10, 1, 0, valEd25519), true}, - 4: {makeParams(100*1024*1024, 0, 10, 1, 0, valEd25519), true}, - 5: {makeParams(101*1024*1024, 0, 10, 1, 0, valEd25519), false}, - 6: {makeParams(1024*1024*1024, 0, 10, 1, 0, valEd25519), false}, + 0: {makeParams(1, 0, 10, 2, 0, valEd25519), true}, + 1: {makeParams(0, 0, 10, 2, 0, valEd25519), false}, + 2: {makeParams(47*1024*1024, 0, 10, 2, 0, valEd25519), true}, + 3: {makeParams(10, 0, 10, 2, 0, valEd25519), true}, + 4: {makeParams(100*1024*1024, 0, 10, 2, 0, valEd25519), true}, + 5: {makeParams(101*1024*1024, 0, 10, 2, 0, valEd25519), false}, + 6: {makeParams(1024*1024*1024, 0, 10, 2, 0, valEd25519), false}, 7: {makeParams(1024*1024*1024, 0, 10, -1, 0, valEd25519), false}, - 8: {makeParams(1, 0, -10, 1, 0, valEd25519), false}, + 8: {makeParams(1, 0, -10, 2, 0, valEd25519), false}, // test evidence params 9: {makeParams(1, 0, 10, 0, 0, valEd25519), false}, - 10: {makeParams(1, 0, 10, 1, 1, valEd25519), false}, - 11: {makeParams(1000, 0, 10, 1, 1, valEd25519), true}, + 10: {makeParams(1, 0, 10, 2, 1, valEd25519), false}, + 11: {makeParams(1000, 0, 10, 2, 1, valEd25519), true}, 12: {makeParams(1, 0, 10, -1, 0, valEd25519), false}, // test no pubkey type provided - 13: {makeParams(1, 0, 10, 1, 0, []string{}), false}, + 13: {makeParams(1, 0, 10, 2, 0, []string{}), false}, // test invalid pubkey type provided - 14: {makeParams(1, 0, 10, 1, 0, []string{"potatoes make good pubkeys"}), false}, + 14: {makeParams(1, 0, 10, 2, 0, []string{"potatoes make good pubkeys"}), false}, } for i, tc := range testCases { if tc.valid { @@ -65,9 +65,10 @@ func makeParams( TimeIotaMs: blockTimeIotaMs, }, Evidence: tmproto.EvidenceParams{ - MaxAgeNumBlocks: evidenceAge, - MaxAgeDuration: time.Duration(evidenceAge), - MaxNum: maxEvidence, + MaxAgeNumBlocks: evidenceAge, + MaxAgeDuration: time.Duration(evidenceAge), + MaxNum: maxEvidence, + ProofTrialPeriod: 1, }, Validator: tmproto.ValidatorParams{ PubKeyTypes: pubkeyTypes, diff --git a/types/priv_validator.go b/types/priv_validator.go index 3d8e72bba..de1407150 100644 --- a/types/priv_validator.go +++ b/types/priv_validator.go @@ -100,6 +100,15 @@ func (pv MockPV) SignProposal(chainID string, proposal *Proposal) error { return nil } +func (pv MockPV) ExtractIntoValidator(votingPower int64) *Validator { + pubKey, _ := pv.GetPubKey() + return &Validator{ + Address: pubKey.Address(), + PubKey: pubKey, + VotingPower: votingPower, + } +} + // String returns a string representation of the MockPV. func (pv MockPV) String() string { mpv, _ := pv.GetPubKey() // mockPV will never return an error, ignored here diff --git a/types/validator_set.go b/types/validator_set.go index cf04df6dd..1f56faaac 100644 --- a/types/validator_set.go +++ b/types/validator_set.go @@ -856,6 +856,9 @@ func (vals *ValidatorSet) ToProto() (*tmproto.ValidatorSet, error) { if vals == nil { return nil, errors.New("nil validator set") // validator set should never be nil } + if err := vals.ValidateBasic(); err != nil { + return nil, fmt.Errorf("validator set failed basic: %w", err) + } vp := new(tmproto.ValidatorSet) valsProto := make([]*tmproto.Validator, len(vals.Validators)) for i := 0; i < len(vals.Validators); i++ { diff --git a/types/vote_set_test.go b/types/vote_set_test.go index d87300bdc..63b61e81b 100644 --- a/types/vote_set_test.go +++ b/types/vote_set_test.go @@ -598,12 +598,21 @@ func TestMakeCommit(t *testing.T) { } func buildVoteSet( - height int64, + height int64, round int32, nonVotes, nonNilVotes, nilVotes int, + voteType tmproto.SignedMsgType) (voteSet *VoteSet, valSet *ValidatorSet, + privValidators []PrivValidator, blockID BlockID) { + + blockID = makeBlockIDRandom() + voteSet, valSet, privValidators = buildVoteSetForBlock(height, round, nonVotes, nonNilVotes, nilVotes, voteType, + blockID) + return +} + +func buildVoteSetForBlock(height int64, round int32, nonVotes, nonNilVotes, nilVotes int, - voteType tmproto.SignedMsgType) (*VoteSet, *ValidatorSet, []PrivValidator, BlockID) { + voteType tmproto.SignedMsgType, blockID BlockID) (*VoteSet, *ValidatorSet, []PrivValidator) { valSize := nonVotes + nilVotes + nonNilVotes voteSet, valSet, privValidators := randVoteSet(height, round, voteType, valSize, 1) - blockID := makeBlockIDRandom() voteProto := &Vote{ ValidatorAddress: nil, ValidatorIndex: -1, @@ -625,5 +634,5 @@ func buildVoteSet( vote := withValidator(voteProto, addr, int32(i)) _, _ = signAddVote(privValidators[i], withBlockHash(vote, nil), voteSet) } - return voteSet, valSet, privValidators, blockID + return voteSet, valSet, privValidators }