Browse Source

evidence: create proof of lock change and implement it in evidence store (#4746)

creates a proof of lock change which is an array of votes that proves the validator was permitted to change locked block and vote again. This proof is stored in the evidence pool and is used as a part of amnesia evidence
pull/4825/head
Callum Waters 4 years ago
committed by GitHub
parent
commit
7cb3d90e01
No known key found for this signature in database GPG Key ID: 4AEE18F83AFDEB23
10 changed files with 571 additions and 35 deletions
  1. +5
    -4
      consensus/common_test.go
  2. +1
    -0
      consensus/reactor_test.go
  3. +1
    -0
      consensus/replay_stubs.go
  4. +28
    -2
      consensus/state.go
  5. +126
    -0
      consensus/state_test.go
  6. +115
    -26
      evidence/pool.go
  7. +57
    -3
      evidence/pool_test.go
  8. +150
    -0
      types/evidence.go
  9. +57
    -0
      types/evidence_test.go
  10. +31
    -0
      types/vote_set_test.go

+ 5
- 4
consensus/common_test.go View File

@ -48,9 +48,11 @@ const (
type cleanupFunc func()
// genesis, chain_id, priv_val
var config *cfg.Config // NOTE: must be reset for each _test.go file
var consensusReplayConfig *cfg.Config
var ensureTimeout = time.Millisecond * 100
var (
config *cfg.Config // NOTE: must be reset for each _test.go file
consensusReplayConfig *cfg.Config
ensureTimeout = time.Millisecond * 100
)
func ensureDir(dir string, mode os.FileMode) {
if err := tmos.EnsureDir(dir, mode); err != nil {
@ -300,7 +302,6 @@ func validatePrecommit(
lockedBlockHash))
}
}
}
func validatePrevoteAndPrecommit(


+ 1
- 0
consensus/reactor_test.go View File

@ -235,6 +235,7 @@ func (m *mockEvidencePool) IsPending(evidence types.Evidence) bool {
}
return false
}
func (m *mockEvidencePool) AddPOLC(types.ProofOfLockChange) error { return nil }
//------------------------------------


+ 1
- 0
consensus/replay_stubs.go View File

@ -55,6 +55,7 @@ func (emptyEvidencePool) AddEvidence(types.Evidence) error { return nil }
func (emptyEvidencePool) Update(*types.Block, sm.State) {}
func (emptyEvidencePool) IsCommitted(types.Evidence) bool { return false }
func (emptyEvidencePool) IsPending(types.Evidence) bool { return false }
func (emptyEvidencePool) AddPOLC(types.ProofOfLockChange) error { return nil }
//-----------------------------------------------------------------------------
// mockProxyApp uses ABCIResponses to give the right results.


+ 28
- 2
consensus/state.go View File

@ -66,6 +66,7 @@ type txNotifier interface {
// interface to the evidence pool
type evidencePool interface {
AddEvidence(types.Evidence) error
AddPOLC(types.ProofOfLockChange) error
}
// State handles execution of the consensus algorithm.
@ -1244,7 +1245,7 @@ func (cs *State) enterPrecommit(height int64, round int) {
// There was a polka in this round for a block we don't have.
// Fetch that block, unlock, and precommit nil.
// The +2/3 prevotes for this round is the POL for our unlock.
// TODO: In the future save the POL prevotes for justification.
logger.Info("enterPrecommit: +2/3 prevotes for a block we don't have. Voting nil", "blockID", blockID)
cs.LockedRound = -1
cs.LockedBlock = nil
cs.LockedBlockParts = nil
@ -1256,6 +1257,29 @@ func (cs *State) enterPrecommit(height int64, round int) {
cs.signAddVote(types.PrecommitType, nil, types.PartSetHeader{})
}
func (cs *State) savePOLC(round int, blockID types.BlockID) {
// polc must be for rounds greater than 0
if round == 0 {
return
}
pubKey, err := cs.privValidator.GetPubKey()
if err != nil {
cs.Logger.Error("Error on retrieval of pubkey", "err", err)
return
}
polc, err := types.MakePOLCFromVoteSet(cs.Votes.Prevotes(round), pubKey, blockID)
if err != nil {
cs.Logger.Error("Error on forming POLC", "err", err)
return
}
err = cs.evpool.AddPOLC(polc)
if err != nil {
cs.Logger.Error("Error on saving POLC", "err", err)
return
}
cs.Logger.Info("Saved POLC to evidence pool", "round", round, "height", polc.Height())
}
// Enter: any +2/3 precommits for next round.
func (cs *State) enterPrecommitWait(height int64, round int) {
logger := cs.Logger.With("height", height, "round", round)
@ -1281,7 +1305,6 @@ func (cs *State) enterPrecommitWait(height int64, round int) {
// Wait for some more precommits; enterNewRound
cs.scheduleTimeout(cs.config.Precommit(round), height, round, cstypes.RoundStepPrecommitWait)
}
// Enter: +2/3 precommits for block
@ -1831,6 +1854,9 @@ func (cs *State) addVote(
cs.LockedRound = -1
cs.LockedBlock = nil
cs.LockedBlockParts = nil
// If this is not the first round and we have already locked onto something then we are
// changing the locked block so save POLC prevotes in evidence db in case of future justification
cs.savePOLC(vote.Round, blockID)
cs.eventBus.PublishEventUnlock(cs.RoundStateEvent())
}


+ 126
- 0
consensus/state_test.go View File

@ -704,6 +704,132 @@ func TestStateLockPOLUnlock(t *testing.T) {
ensureNewRound(newRoundCh, height, round+1)
}
// 4 vals, v1 locks on proposed block in the first round but the other validators only prevote
// In the second round, v1 misses the proposal but sees a majority prevote an unknown block so
// 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)
vs2, vs3, vs4 := vss[1], vss[2], vss[3]
height, round := cs1.Height, cs1.Round
partSize := types.BlockPartSizeBytes
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
pv1, err := cs1.privValidator.GetPubKey()
require.NoError(t, err)
addr := pv1.Address()
voteCh := subscribeToVoter(cs1, addr)
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
// everything done from perspective of cs1
/*
Round0 (cs1, A) // A A A A// A nil nil nil
*/
// start round and wait for propose and prevote
startTestRound(cs1, height, round)
ensureNewRound(newRoundCh, height, round)
ensureNewProposal(proposalCh, height, round)
rs := cs1.GetRoundState()
firstBlockHash := rs.ProposalBlock.Hash()
firstBlockParts := rs.ProposalBlockParts.Header()
ensurePrevote(voteCh, height, round) // prevote
signAddVotes(cs1, types.PrevoteType, firstBlockHash, firstBlockParts, vs2, vs3, vs4)
ensurePrecommit(voteCh, height, round) // our precommit
// the proposed block should now be locked and our precommit added
validatePrecommit(t, cs1, round, round, vss[0], firstBlockHash, firstBlockHash)
// add precommits from the rest
signAddVotes(cs1, types.PrecommitType, nil, types.PartSetHeader{}, vs2, vs3, vs4)
// before we timeout to the new round set the new proposal
cs2 := newState(cs1.state, vs2, counter.NewApplication(true))
prop, propBlock := decideProposal(cs2, vs2, vs2.Height, vs2.Round+1)
if prop == nil || propBlock == nil {
t.Fatal("Failed to create proposal block with vs2")
}
secondBlockParts := propBlock.MakePartSet(partSize)
secondBlockHash := propBlock.Hash()
require.NotEqual(t, secondBlockHash, firstBlockHash)
incrementRound(vs2, vs3, vs4)
// timeout to new round
ensureNewTimeout(timeoutWaitCh, height, round, cs1.config.Precommit(round).Nanoseconds())
round++ // moving to the next round
ensureNewRound(newRoundCh, height, round)
t.Log("### ONTO ROUND 1")
/*
Round1 (vs2, B) // A B B B // nil nil nil nil)
*/
// now we're on a new round but v1 misses the proposal
// go to prevote, node should prevote for locked block (not the new proposal) - this is relocking
ensurePrevote(voteCh, height, round)
validatePrevote(t, cs1, round, vss[0], firstBlockHash)
// now lets add prevotes from everyone else for the new block
signAddVotes(cs1, types.PrevoteType, secondBlockHash, secondBlockParts.Header(), vs2, vs3, vs4)
ensurePrecommit(voteCh, height, round)
// we should have unlocked and locked on the new block, sending a precommit for this new block
validatePrecommit(t, cs1, round, -1, vss[0], nil, nil)
if err := cs1.SetProposalAndBlock(prop, propBlock, secondBlockParts, "some peer"); err != nil {
t.Fatal(err)
}
// more prevote creating a majority on the new block and this is then committed
signAddVotes(cs1, types.PrecommitType, nil, types.PartSetHeader{}, vs2, vs3, vs4)
// before we timeout to the new round set the new proposal
cs3 := newState(cs1.state, vs3, counter.NewApplication(true))
prop, propBlock = decideProposal(cs3, vs3, vs3.Height, vs3.Round+1)
if prop == nil || propBlock == nil {
t.Fatal("Failed to create proposal block with vs2")
}
thirdPropBlockParts := propBlock.MakePartSet(partSize)
thirdPropBlockHash := propBlock.Hash()
require.NotEqual(t, secondBlockHash, thirdPropBlockHash)
incrementRound(vs2, vs3, vs4)
// timeout to new round
ensureNewTimeout(timeoutWaitCh, height, round, cs1.config.Precommit(round).Nanoseconds())
round++ // moving to the next round
ensureNewRound(newRoundCh, height, round)
t.Log("### ONTO ROUND 2")
/*
Round2 (vs3, C) // C C C C // C nil nil nil)
*/
if err := cs1.SetProposalAndBlock(prop, propBlock, thirdPropBlockParts, "some peer"); err != nil {
t.Fatal(err)
}
ensurePrevote(voteCh, height, round)
// we are no longer locked to the first block so we should be able to prevote
validatePrevote(t, cs1, round, vss[0], thirdPropBlockHash)
signAddVotes(cs1, types.PrevoteType, thirdPropBlockHash, thirdPropBlockParts.Header(), vs2, vs3, vs4)
ensurePrecommit(voteCh, height, round)
// we have a majority, now vs1 can change lock to the third block
validatePrecommit(t, cs1, round, round, vss[0], thirdPropBlockHash, thirdPropBlockHash)
}
// 4 vals
// a polka at round 1 but we miss it
// then a polka at round 2 that we lock on


+ 115
- 26
evidence/pool.go View File

@ -15,8 +15,9 @@ import (
)
const (
baseKeyCommitted = byte(0x00) // committed evidence
baseKeyPending = byte(0x01) // pending evidence
baseKeyCommitted = byte(0x00)
baseKeyPending = byte(0x01)
baseKeyPOLC = byte(0x02)
)
// Pool maintains a pool of valid evidence to be broadcasted and committed
@ -65,12 +66,9 @@ func NewPool(stateDB, evidenceDB dbm.DB, blockStore *store.BlockStore) (*Pool, e
}
// if pending evidence already in db, in event of prior failure, then load it back to the evidenceList
evList, err := pool.listEvidence(baseKeyPending, -1)
if err != nil {
return nil, err
}
evList := pool.AllPendingEvidence()
for _, ev := range evList {
if pool.IsExpired(ev) {
if pool.IsEvidenceExpired(ev) {
pool.removePendingEvidence(ev)
continue
}
@ -114,6 +112,14 @@ func (evpool *Pool) Update(block *types.Block, state sm.State) {
// remove evidence from pending and mark committed
evpool.MarkEvidenceAsCommitted(block.Height, block.Time, 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
if block.Height%state.ConsensusParams.Evidence.MaxAgeNumBlocks == 0 {
evpool.pruneExpiredPOLC()
}
// update the state
evpool.mtx.Lock()
defer evpool.mtx.Unlock()
@ -121,6 +127,14 @@ func (evpool *Pool) Update(block *types.Block, state sm.State) {
evpool.updateValToLastHeight(block.Height, state)
}
// AddPOLC adds a proof of lock change to the evidence database
// that may be needed in the future to verify votes
func (evpool *Pool) AddPOLC(polc types.ProofOfLockChange) error {
key := keyPOLC(polc)
polcBytes := cdc.MustMarshalBinaryBare(polc)
return evpool.evidenceStore.Set(key, polcBytes)
}
// AddEvidence checks the evidence is valid and adds it to the pool. If
// evidence is composite (ConflictingHeadersEvidence), it will be broken up
// into smaller pieces.
@ -216,8 +230,7 @@ func (evpool *Pool) MarkEvidenceAsCommitted(height int64, lastBlockTime time.Tim
// remove committed evidence from the clist
if len(blockEvidenceMap) != 0 {
evidenceParams := evpool.State().ConsensusParams.Evidence
evpool.removeEvidenceFromList(height, lastBlockTime, evidenceParams, blockEvidenceMap)
evpool.removeEvidenceFromList(blockEvidenceMap)
}
}
@ -226,12 +239,18 @@ func (evpool *Pool) Has(evidence types.Evidence) bool {
return evpool.IsPending(evidence) || evpool.IsCommitted(evidence)
}
// IsExpired checks whether evidence is past the maximum age where it can be used
func (evpool *Pool) IsExpired(evidence types.Evidence) bool {
// IsEvidenceExpired checks whether evidence is past the maximum age where it can be used
func (evpool *Pool) IsEvidenceExpired(evidence types.Evidence) bool {
return evpool.IsExpired(evidence.Height(), evidence.Time())
}
// IsExpired checks whether evidence or a polc is expired by checking whether a height and time is older
// than set by the evidence consensus parameters
func (evpool *Pool) IsExpired(height int64, time time.Time) bool {
var (
params = evpool.State().ConsensusParams.Evidence
ageDuration = evpool.State().LastBlockTime.Sub(evidence.Time())
ageNumBlocks = evpool.State().LastBlockHeight - evidence.Height()
ageDuration = evpool.State().LastBlockTime.Sub(time)
ageNumBlocks = evpool.State().LastBlockHeight - height
)
return ageNumBlocks > params.MaxAgeNumBlocks &&
ageDuration > params.MaxAgeDuration
@ -257,6 +276,21 @@ 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
func (evpool *Pool) RetrievePOLC(height int64, round int) (types.ProofOfLockChange, error) {
var polc types.ProofOfLockChange
key := keyPOLCFromHeightAndRound(height, round)
polcBytes, err := evpool.evidenceStore.Get(key)
if err != nil {
return polc, err
}
if polcBytes == nil {
return polc, fmt.Errorf("unable to find polc at height %d and round %d", height, round)
}
err = cdc.UnmarshalBinaryBare(polcBytes, &polc)
return polc, err
}
// EvidenceFront goes to the first evidence in the clist
func (evpool *Pool) EvidenceFront() *clist.CElement {
return evpool.evidenceList.Front()
@ -304,6 +338,8 @@ func (evpool *Pool) removePendingEvidence(evidence types.Evidence) {
key := keyPending(evidence)
if err := evpool.evidenceStore.Delete(key); err != nil {
evpool.logger.Error("Unable to delete pending evidence", "err", err)
} else {
evpool.logger.Info("Deleted pending evidence", "evidence", evidence)
}
}
@ -335,29 +371,74 @@ func (evpool *Pool) listEvidence(prefixKey byte, maxNum int64) ([]types.Evidence
return evidence, nil
}
func (evpool *Pool) removeExpiredPendingEvidence() {
iter, err := dbm.IteratePrefix(evpool.evidenceStore, []byte{baseKeyPending})
if err != nil {
evpool.logger.Error("Unable to iterate over pending evidence", "err", err)
return
}
defer iter.Close()
blockEvidenceMap := make(map[string]struct{})
for ; iter.Valid(); iter.Next() {
evBytes := iter.Value()
var ev types.Evidence
err := cdc.UnmarshalBinaryBare(evBytes, &ev)
if err != nil {
evpool.logger.Error("Unable to unmarshal POLC", "err", err)
continue
}
if !evpool.IsExpired(ev.Height()-1, ev.Time()) {
if len(blockEvidenceMap) != 0 {
evpool.removeEvidenceFromList(blockEvidenceMap)
}
return
}
evpool.removePendingEvidence(ev)
blockEvidenceMap[evMapKey(ev)] = struct{}{}
}
}
func (evpool *Pool) removeEvidenceFromList(
height int64,
lastBlockTime time.Time,
params types.EvidenceParams,
blockEvidenceMap map[string]struct{}) {
for e := evpool.evidenceList.Front(); e != nil; e = e.Next() {
var (
ev = e.Value.(types.Evidence)
ageDuration = lastBlockTime.Sub(ev.Time())
ageNumBlocks = height - ev.Height()
)
// Remove the evidence if it's already in a block or if it's now too old.
if _, ok := blockEvidenceMap[evMapKey(ev)]; ok ||
(ageDuration > params.MaxAgeDuration && ageNumBlocks > params.MaxAgeNumBlocks) {
// remove from clist
// Remove from clist
ev := e.Value.(types.Evidence)
if _, ok := blockEvidenceMap[evMapKey(ev)]; ok {
evpool.evidenceList.Remove(e)
e.DetachPrev()
}
}
}
func (evpool *Pool) pruneExpiredPOLC() {
evpool.logger.Debug("Pruning expired POLC's")
iter, err := dbm.IteratePrefix(evpool.evidenceStore, []byte{baseKeyPOLC})
if err != nil {
evpool.logger.Error("Unable to iterate over POLC's", "err", err)
return
}
defer iter.Close()
for ; iter.Valid(); iter.Next() {
proofBytes := iter.Value()
var proof types.ProofOfLockChange
err := cdc.UnmarshalBinaryBare(proofBytes, &proof)
if err != nil {
evpool.logger.Error("Unable to unmarshal POLC", "err", err)
continue
}
if !evpool.IsExpired(proof.Height()-1, proof.Time()) {
return
}
err = evpool.evidenceStore.Delete(iter.Key())
if err != nil {
evpool.logger.Error("Unable to delete expired POLC", "err", err)
continue
}
evpool.logger.Info("Deleted expired POLC", "polc", proof)
}
}
func evMapKey(ev types.Evidence) string {
return string(ev.Hash())
}
@ -451,6 +532,14 @@ func keyPending(evidence types.Evidence) []byte {
return append([]byte{baseKeyPending}, keySuffix(evidence)...)
}
func keyPOLC(polc types.ProofOfLockChange) []byte {
return keyPOLCFromHeightAndRound(polc.Height(), polc.Round())
}
func keyPOLCFromHeightAndRound(height int64, round int) []byte {
return append([]byte{baseKeyPOLC}, []byte(fmt.Sprintf("%s/%s", bE(height), bE(int64(round))))...)
}
func keySuffix(evidence types.Evidence) []byte {
return []byte(fmt.Sprintf("%s/%X", bE(evidence.Height()), evidence.Hash()))
}


+ 57
- 3
evidence/pool_test.go View File

@ -46,7 +46,7 @@ func TestEvidencePool(t *testing.T) {
assert.Contains(t, err.Error(), "is too old; min height is 32 and evidence can not be older than")
}
assert.False(t, pool.IsPending(badEvidence))
assert.True(t, pool.IsExpired(badEvidence))
assert.True(t, pool.IsEvidenceExpired(badEvidence))
// good evidence
evAdded := make(chan struct{})
@ -108,7 +108,7 @@ func TestProposingAndCommittingEvidence(t *testing.T) {
// evidence should
}
func TestEvidencePoolAddEvidence(t *testing.T) {
func TestAddEvidence(t *testing.T) {
var (
valAddr = []byte("val1")
height = int64(30)
@ -151,16 +151,20 @@ func TestEvidencePoolAddEvidence(t *testing.T) {
func TestEvidencePoolUpdate(t *testing.T) {
var (
valAddr = []byte("validator_address")
height = int64(1)
height = int64(21)
stateDB = initializeValidatorState(valAddr, height)
evidenceDB = dbm.NewMemDB()
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)
@ -175,6 +179,8 @@ 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) {
@ -195,6 +201,54 @@ func TestEvidencePoolNewPool(t *testing.T) {
assert.EqualValues(t, 0, pool.ValidatorLastHeight([]byte("non-existent-validator")))
}
func TestAddingAndPruningPOLC(t *testing.T) {
var (
valAddr = []byte("validator_address")
stateDB = initializeValidatorState(valAddr, 1)
evidenceDB = dbm.NewMemDB()
blockStoreDB = dbm.NewMemDB()
state = sm.LoadState(stateDB)
blockStore = initializeBlockStore(blockStoreDB, state, valAddr)
height = state.ConsensusParams.Evidence.MaxAgeNumBlocks * 2
evidenceTime = time.Date(2019, 1, 1, 0, 0, 0, 0, time.UTC)
)
pubKey, _ := types.NewMockPV().GetPubKey()
polc := types.NewMockPOLC(1, evidenceTime, pubKey)
pool, err := NewPool(stateDB, evidenceDB, blockStore)
require.NoError(t, err)
err = pool.AddPOLC(polc)
assert.NoError(t, err)
// should be able to retrieve polc
newPolc, err := pool.RetrievePOLC(1, 1)
assert.NoError(t, err)
assert.True(t, polc.Equal(newPolc))
// should not be able to retrieve
emptyPolc, err := pool.RetrievePOLC(2, 1)
assert.Error(t, err)
assert.Equal(t, types.ProofOfLockChange{}, emptyPolc)
lastCommit := makeCommit(height-1, valAddr)
block := types.MakeBlock(height, []types.Tx{}, lastCommit, []types.Evidence{})
// update state (partially)
state.LastBlockHeight = height
pool.state.LastBlockHeight = height
// update should prune the polc
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.Equal(t, types.ProofOfLockChange{}, emptyPolc)
}
func TestRecoverPendingEvidence(t *testing.T) {
var (
valAddr = []byte("val1")


+ 150
- 0
types/evidence.go View File

@ -900,6 +900,143 @@ func (e PotentialAmnesiaEvidence) String() string {
return fmt.Sprintf("PotentialAmnesiaEvidence{VoteA: %v, VoteB: %v}", e.VoteA, e.VoteB)
}
// 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.
type ProofOfLockChange struct {
Votes []Vote `json:"votes"`
PubKey crypto.PubKey `json:"pubkey"`
}
// MakePOLCFromVoteSet can be used when a majority of prevotes or precommits for a block is seen
// that the node has itself not yet voted for in order to process the vote set into a proof of lock change
func MakePOLCFromVoteSet(voteSet *VoteSet, pubKey crypto.PubKey, blockID BlockID) (ProofOfLockChange, error) {
polc := makePOLCFromVoteSet(voteSet, pubKey, blockID)
return polc, polc.ValidateBasic()
}
func makePOLCFromVoteSet(voteSet *VoteSet, pubKey crypto.PubKey, blockID BlockID) ProofOfLockChange {
var votes []Vote
valSetSize := voteSet.Size()
for valIdx := 0; valIdx < valSetSize; valIdx++ {
vote := voteSet.GetByIndex(valIdx)
if vote != nil && vote.BlockID.Equals(blockID) {
votes = append(votes, *vote)
}
}
return ProofOfLockChange{
Votes: votes,
PubKey: pubKey,
}
}
func (e ProofOfLockChange) Height() int64 {
return e.Votes[0].Height
}
// returns the time of the last vote
func (e ProofOfLockChange) Time() time.Time {
latest := e.Votes[0].Timestamp
for _, vote := range e.Votes {
if vote.Timestamp.After(latest) {
latest = vote.Timestamp
}
}
return latest
}
func (e ProofOfLockChange) Round() int {
return e.Votes[0].Round
}
func (e ProofOfLockChange) Address() []byte {
return e.PubKey.Address()
}
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 {
talliedVotingPower := int64(0)
votingPowerNeeded := valSet.TotalVotingPower() * 2 / 3
for _, validator := range valSet.Validators {
for _, vote := range e.Votes {
if bytes.Equal(validator.Address, vote.ValidatorAddress) {
talliedVotingPower += validator.VotingPower
if talliedVotingPower > votingPowerNeeded {
return true
}
}
}
}
return false
}
func (e ProofOfLockChange) Equal(e2 ProofOfLockChange) bool {
return bytes.Equal(e.Address(), e2.Address()) && e.Height() == e2.Height() &&
e.Round() == e2.Round()
}
func (e ProofOfLockChange) ValidateBasic() error {
if e.PubKey == nil {
return errors.New("missing public key")
}
// validate basic doesn't count the number of votes and their voting power, this is to be done by VerifyEvidence
if e.Votes == nil {
return errors.New("missing votes")
}
// height, round and vote type must be the same for all votes
height := e.Height()
round := e.Round()
if round == 0 {
return errors.New("can't have a polc for the first round")
}
voteType := e.Votes[0].Type
for idx, vote := range e.Votes {
if err := vote.ValidateBasic(); err != nil {
return fmt.Errorf("invalid vote#%d: %w", idx, err)
}
if vote.Height != height {
return fmt.Errorf("invalid height for vote#%d: %d instead of %d", idx, vote.Height, height)
}
if vote.Round != round {
return fmt.Errorf("invalid round for vote#%d: %d instead of %d", idx, vote.Round, round)
}
if vote.Type != voteType {
return fmt.Errorf("invalid vote type for vote#%d: %d instead of %d", idx, vote.Type, voteType)
}
if !vote.BlockID.Equals(e.BlockID()) {
return fmt.Errorf("vote must be for the same block id: %v instead of %v", e.BlockID(), vote.BlockID)
}
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)
}
for i := idx + 1; i < len(e.Votes); i++ {
if bytes.Equal(vote.ValidatorAddress.Bytes(), e.Votes[i].ValidatorAddress.Bytes()) {
return fmt.Errorf("duplicate votes: %v", vote)
}
}
}
return nil
}
func (e ProofOfLockChange) String() string {
return fmt.Sprintf("ProofOfLockChange {Address: %X, Height: %d, Round: %d", e.Address(), e.Height(),
e.Votes[0].Round)
}
//-----------------------------------------------------------------
// UNSTABLE
@ -962,3 +1099,16 @@ func (e MockEvidence) ValidateBasic() error { return nil }
func (e MockEvidence) String() string {
return fmt.Sprintf("Evidence: %d/%s/%s", e.EvidenceHeight, e.Time(), e.EvidenceAddress)
}
// mock polc - fails validate basic, not stable
func NewMockPOLC(height int64, time time.Time, pubKey crypto.PubKey) ProofOfLockChange {
voteVal := NewMockPV()
pKey, _ := voteVal.GetPubKey()
vote := Vote{Type: PrecommitType, Height: height, Round: 1, BlockID: BlockID{},
Timestamp: time, ValidatorAddress: pKey.Address(), ValidatorIndex: 1, Signature: []byte{}}
_ = voteVal.SignVote("mock-chain-id", &vote)
return ProofOfLockChange{
Votes: []Vote{vote},
PubKey: pubKey,
}
}

+ 57
- 0
types/evidence_test.go View File

@ -386,6 +386,63 @@ func TestPotentialAmnesiaEvidence(t *testing.T) {
assert.NotEmpty(t, ev.String())
}
func TestProofOfLockChange(t *testing.T) {
const (
chainID = "TestProofOfLockChange"
height int64 = 37
)
// 1: valid POLC - nothing should fail
voteSet, valSet, privValidators, blockID := buildVoteSet(height, 1, 3, 7, 0, PrecommitType)
pubKey, err := privValidators[7].GetPubKey()
require.NoError(t, err)
polc := makePOLCFromVoteSet(voteSet, pubKey, blockID)
assert.Equal(t, height, polc.Height())
assert.NoError(t, polc.ValidateBasic())
assert.True(t, polc.MajorityOfVotes(valSet))
assert.NotEmpty(t, polc.String())
// test validate basic on a set of bad cases
var badPOLCs []ProofOfLockChange
// 2: node has already voted in next round
pubKey, err = privValidators[0].GetPubKey()
require.NoError(t, err)
polc2 := makePOLCFromVoteSet(voteSet, pubKey, blockID)
badPOLCs = append(badPOLCs, polc2)
// 3: one vote was from a different round
voteSet, _, privValidators, blockID = buildVoteSet(height, 1, 3, 7, 0, PrecommitType)
pubKey, err = privValidators[7].GetPubKey()
require.NoError(t, err)
polc = makePOLCFromVoteSet(voteSet, pubKey, blockID)
badVote := makeVote(t, privValidators[8], chainID, 8, height, 2, 2, blockID)
polc.Votes = append(polc.Votes, *badVote)
badPOLCs = append(badPOLCs, polc)
// 4: one vote was from a different height
polc = makePOLCFromVoteSet(voteSet, pubKey, blockID)
badVote = makeVote(t, privValidators[8], chainID, 8, height+1, 1, 2, blockID)
polc.Votes = append(polc.Votes, *badVote)
badPOLCs = append(badPOLCs, polc)
// 5: one vote was from a different vote type
polc = makePOLCFromVoteSet(voteSet, pubKey, blockID)
badVote = makeVote(t, privValidators[8], chainID, 8, height, 1, 1, blockID)
polc.Votes = append(polc.Votes, *badVote)
badPOLCs = append(badPOLCs, polc)
// 5: one of the votes was for a nil block
polc = makePOLCFromVoteSet(voteSet, pubKey, blockID)
badVote = makeVote(t, privValidators[8], chainID, 8, height, 1, 2, BlockID{})
polc.Votes = append(polc.Votes, *badVote)
badPOLCs = append(badPOLCs, polc)
for idx, polc := range badPOLCs {
err := polc.ValidateBasic()
assert.Error(t, err)
if err == nil {
t.Errorf("test no. %d failed", idx+2)
}
}
}
func makeHeaderRandom() *Header {
return &Header{
ChainID: tmrand.Str(12),


+ 31
- 0
types/vote_set_test.go View File

@ -595,3 +595,34 @@ func TestMakeCommit(t *testing.T) {
t.Errorf("error in Commit.ValidateBasic(): %v", err)
}
}
func buildVoteSet(
height int64,
round, nonVotes, nonNilVotes, nilVotes int,
voteType SignedMsgType) (*VoteSet, *ValidatorSet, []PrivValidator, BlockID) {
valSize := nonVotes + nilVotes + nonNilVotes
voteSet, valSet, privValidators := randVoteSet(height, round, voteType, valSize, 1)
blockID := makeBlockIDRandom()
voteProto := &Vote{
ValidatorAddress: nil,
ValidatorIndex: -1,
Height: height,
Round: round,
Type: voteType,
Timestamp: tmtime.Now(),
BlockID: blockID,
}
for i := 0; i < nonNilVotes; i++ {
pubKey, _ := privValidators[i].GetPubKey()
addr := pubKey.Address()
vote := withValidator(voteProto, addr, i)
_, _ = signAddVote(privValidators[i], vote, voteSet)
}
for i := nonNilVotes; i < nonNilVotes+nilVotes; i++ {
pubKey, _ := privValidators[i].GetPubKey()
addr := pubKey.Address()
vote := withValidator(voteProto, addr, i)
_, _ = signAddVote(privValidators[i], withBlockHash(vote, nil), voteSet)
}
return voteSet, valSet, privValidators, blockID
}

Loading…
Cancel
Save