Browse Source

evidence: remove amnesia & POLC (#5319)

## Description

remove unneeded types 

![](https://media1.giphy.com/media/fSAyceY3BCgtiQGnJs/giphy.gif)

ref #5288
pull/5323/head
Marko 4 years ago
committed by GitHub
parent
commit
710a97d850
No known key found for this signature in database GPG Key ID: 4AEE18F83AFDEB23
24 changed files with 273 additions and 3246 deletions
  1. +2
    -0
      CHANGELOG_PENDING.md
  2. +0
    -44
      consensus/common_test.go
  3. +1
    -2
      consensus/reactor_test.go
  4. +0
    -1
      consensus/replay_stubs.go
  5. +0
    -28
      consensus/state.go
  6. +0
    -232
      consensus/state_test.go
  7. +0
    -4
      docs/tendermint-core/using-tendermint.md
  8. +1
    -7
      evidence/doc.go
  9. +10
    -306
      evidence/pool.go
  10. +3
    -282
      evidence/pool_test.go
  11. +0
    -10
      evidence/verify.go
  12. +212
    -1199
      proto/tendermint/types/evidence.pb.go
  13. +1
    -22
      proto/tendermint/types/evidence.proto
  14. +36
    -78
      proto/tendermint/types/params.pb.go
  15. +0
    -5
      proto/tendermint/types/params.proto
  16. +1
    -2
      tools/tm-signer-harness/internal/test_harness_test.go
  17. +0
    -6
      types/block.go
  18. +0
    -3
      types/block_test.go
  19. +0
    -650
      types/evidence.go
  20. +0
    -304
      types/evidence_test.go
  21. +3
    -14
      types/params.go
  22. +3
    -4
      types/params_test.go
  23. +0
    -3
      types/protobuf.go
  24. +0
    -40
      types/vote_set_test.go

+ 2
- 0
CHANGELOG_PENDING.md View File

@ -17,6 +17,8 @@ Friendly reminder, we have a [bug bounty program](https://hackerone.com/tendermi
- Go API
- [evidence] \#5317 Remove ConflictingHeaders evidence type & CompositeEvidence Interface. (@marbar3778)
- [evidence] \#5318 Remove LunaticValidator evidence type. (@marbar3778)
- [evidence] \#5319 Remove Amnesia & potentialAmnesia evidence types and removed POLC. (@marbar3778)
- [params] \#5319 Remove `ProofofTrialPeriod` from evidence params (@marbar3778)
- [crypto/secp256k1] \#5280 `secp256k1` has been removed from the Tendermint repo. (@marbar3778)
- Blockchain Protocol


+ 0
- 44
consensus/common_test.go View File

@ -25,7 +25,6 @@ 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"
@ -430,49 +429,6 @@ func randState(nValidators int) (*State, []*validatorStub) {
return cs, vss
}
func randStateWithEvpool(t *testing.T, 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(tmsync.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()
sm.SaveState(stateDB, state)
evpool, err := evidence.NewPool(evidenceDB, evidence.NewEvidenceStateStore(stateDB), blockStore)
require.NoError(t, err)
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,


+ 1
- 2
consensus/reactor_test.go View File

@ -224,8 +224,7 @@ func (m *mockEvidencePool) Update(block *types.Block, state sm.State) {
}
m.height++
}
func (m *mockEvidencePool) Verify(types.Evidence) error { return nil }
func (m *mockEvidencePool) AddPOLC(*types.ProofOfLockChange) error { return nil }
func (m *mockEvidencePool) Verify(types.Evidence) error { return nil }
//------------------------------------


+ 0
- 1
consensus/replay_stubs.go View File

@ -57,7 +57,6 @@ func (emptyEvidencePool) Update(*types.Block, sm.State) {}
func (emptyEvidencePool) Verify(types.Evidence) error { return nil }
func (emptyEvidencePool) IsCommitted(types.Evidence) bool { return false }
func (emptyEvidencePool) IsPending(types.Evidence) bool { return false }
func (emptyEvidencePool) AddPOLC(*types.ProofOfLockChange) error { return nil }
func (emptyEvidencePool) Header(int64) *types.Header { return nil }
//-----------------------------------------------------------------------------


+ 0
- 28
consensus/state.go View File

@ -74,7 +74,6 @@ 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.
@ -1328,30 +1327,6 @@ func (cs *State) enterPrecommit(height int64, round int32) {
cs.signAddVote(tmproto.PrecommitType, nil, types.PartSetHeader{})
}
func (cs *State) savePOLC(round int32, blockID types.BlockID) {
// polc must be for rounds greater than 0
if round == 0 {
return
}
if cs.privValidatorPubKey == nil {
// This may result in this validator being slashed later during an amnesia
// trial.
cs.Logger.Error(fmt.Sprintf("savePOLC: %v", errPubKeyIsNotSet))
return
}
polc, err := types.NewPOLCFromVoteSet(cs.Votes.Prevotes(round), cs.privValidatorPubKey, 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 int32) {
logger := cs.Logger.With("height", height, "round", round)
@ -1944,9 +1919,6 @@ 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())
}


+ 0
- 232
consensus/state_test.go View File

@ -618,238 +618,6 @@ func TestStateLockPOLRelockThenChangeLock(t *testing.T) {
ensureNewRound(newRoundCh, height+1, 0)
}
// 4 vals, one precommits, other 3 polka at next round, so we unlock and precomit the polka
func TestStateLockPOLUnlock(t *testing.T) {
cs1, vss, evpool := randStateWithEvpool(t, 4)
vs2, vs3, vs4 := vss[1], vss[2], vss[3]
height, round := cs1.Height, cs1.Round
partSize := types.BlockPartSizeBytes
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
unlockCh := subscribe(cs1.eventBus, types.EventQueryUnlock)
pv1, err := cs1.privValidator.GetPubKey()
require.NoError(t, err)
addr := pv1.Address()
voteCh := subscribeToVoter(cs1, addr)
// everything done from perspective of cs1
/*
Round1 (cs1, B) // B B B B // B nil B nil
eg. didn't see the 2/3 prevotes
*/
// start round and wait for propose and prevote
startTestRound(cs1, height, round)
ensureNewRound(newRoundCh, height, round)
ensureNewProposal(proposalCh, height, round)
rs := cs1.GetRoundState()
theBlockHash := rs.ProposalBlock.Hash()
theBlockParts := rs.ProposalBlockParts.Header()
ensurePrevote(voteCh, height, round)
validatePrevote(t, cs1, round, vss[0], theBlockHash)
signAddVotes(cs1, tmproto.PrevoteType, theBlockHash, theBlockParts, vs2, vs3, vs4)
ensurePrecommit(voteCh, height, round)
// the proposed block should now be locked and our precommit added
validatePrecommit(t, cs1, round, round, vss[0], theBlockHash, theBlockHash)
// add precommits from the rest
signAddVotes(cs1, tmproto.PrecommitType, nil, types.PartSetHeader{}, vs2, vs4)
signAddVotes(cs1, tmproto.PrecommitType, theBlockHash, theBlockParts, vs3)
// before we time out into new round, set next proposal block
prop, propBlock := decideProposal(cs1, vs2, vs2.Height, vs2.Round+1)
propBlockParts := propBlock.MakePartSet(partSize)
// timeout to new round
ensureNewTimeout(timeoutWaitCh, height, round, cs1.config.Precommit(round).Nanoseconds())
rs = cs1.GetRoundState()
lockedBlockHash := rs.LockedBlock.Hash()
incrementRound(vs2, vs3, vs4)
round++ // moving to the next round
ensureNewRound(newRoundCh, height, round)
t.Log("#### ONTO ROUND 1")
/*
Round2 (vs2, C) // B nil nil nil // nil nil nil _
cs1 unlocks!
*/
//XXX: this isnt guaranteed to get there before the timeoutPropose ...
if err := cs1.SetProposalAndBlock(prop, propBlock, propBlockParts, "some peer"); err != nil {
t.Fatal(err)
}
ensureNewProposal(proposalCh, height, round)
// go to prevote, prevote for locked block (not proposal)
ensurePrevote(voteCh, height, round)
validatePrevote(t, cs1, round, vss[0], lockedBlockHash)
// now lets add prevotes from everyone else for nil (a polka!)
signAddVotes(cs1, tmproto.PrevoteType, nil, types.PartSetHeader{}, vs2, vs3, vs4)
// the polka makes us unlock and precommit nil
ensureNewUnlock(unlockCh, height, round)
ensurePrecommit(voteCh, height, round)
// we should have unlocked and committed nil
// NOTE: since we don't relock on nil, the lock round is -1
validatePrecommit(t, cs1, round, -1, vss[0], nil, nil)
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.NotNil(t, polc)
assert.False(t, polc.IsAbsent())
// but not for round 0
polc, err = evpool.RetrievePOLC(height, round-1)
assert.NoError(t, err)
assert.Nil(t, polc)
}
// 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, evpool := randStateWithEvpool(t, 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, tmproto.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, tmproto.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, tmproto.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, tmproto.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)
// polc should be in the evpool for round 1
polc, err := evpool.RetrievePOLC(height, round)
assert.NoError(t, err)
assert.NotNil(t, polc)
assert.False(t, polc.IsAbsent())
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, tmproto.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


+ 0
- 4
docs/tendermint-core/using-tendermint.md View File

@ -65,9 +65,6 @@ definition](https://github.com/tendermint/tendermint/blob/master/types/genesis.g
- `max_num`: This sets the maximum number of evidence that can be committed
in a single block. and should fall comfortably under the max block
bytes when we consider the size of each evidence.
- `proof_trial_period`: 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.
- `validator`
- `pub_key_types`: Public key types validators can use.
- `version`
@ -104,7 +101,6 @@ definition](https://github.com/tendermint/tendermint/blob/master/types/genesis.g
"max_age_num_blocks": "100000",
"max_age_duration": "172800000000000",
"max_num": 50,
"proof_trial_period": "5000000"
},
"validator": {
"pub_key_types": [


+ 1
- 7
evidence/doc.go View File

@ -21,18 +21,12 @@ uncommitted evidence at intervals of 60 seconds (set by the by broadcastEvidence
It uses a concurrent list to store the evidence and before sending verifies that each evidence is still valid in the
sense that it has not exceeded the max evidence age and height (see types/params.go#EvidenceParams).
Three are four buckets that evidence can be stored in: Pending, Committed, Awaiting and POLC's.
There are two buckets that evidence can be stored in: Pending & Committed.
1. Pending is awaiting to be committed (evidence is usually broadcasted then)
2. Committed is for those already on the block and is to ensure that evidence isn't submitted twice
3. AwaitingTrial primarily refers to PotentialAmnesiaEvidence which must wait for a trial period before
being ready to be submitted (see docs/architecture/adr-056)
4. POLC's store all the ProofOfLockChanges that the node has done as part of consensus. To change lock is to vote
for a different block in a later round. The consensus module calls `AddPOLC()` to add to this bucket.
All evidence is proto encoded to disk.
Proposing


+ 10
- 306
evidence/pool.go View File

@ -18,10 +18,8 @@ import (
)
const (
baseKeyCommitted = byte(0x00)
baseKeyPending = byte(0x01)
baseKeyPOLC = byte(0x02)
baseKeyAwaitingTrial = byte(0x03)
baseKeyCommitted = byte(0x00)
baseKeyPending = byte(0x01)
)
// Pool maintains a pool of valid evidence to be broadcasted and committed
@ -39,11 +37,6 @@ type Pool struct {
mtx sync.Mutex
// latest state
state sm.State
// This is the closest height where at one or more of the current trial periods
// will have ended and we will need to then upgrade the evidence to amnesia evidence.
// It is set to -1 when we don't have any evidence on trial.
nextEvidenceTrialEndedHeight int64
}
// NewPool creates an evidence pool. If using an existing evidence store,
@ -54,13 +47,12 @@ func NewPool(evidenceDB dbm.DB, stateDB StateStore, blockStore BlockStore) (*Poo
)
pool := &Pool{
stateDB: stateDB,
blockStore: blockStore,
state: state,
logger: log.NewNopLogger(),
evidenceStore: evidenceDB,
evidenceList: clist.New(),
nextEvidenceTrialEndedHeight: -1,
stateDB: stateDB,
blockStore: blockStore,
state: state,
logger: log.NewNopLogger(),
evidenceStore: evidenceDB,
evidenceList: clist.New(),
}
// if pending evidence already in db, in event of prior failure, then load it back to the evidenceList
@ -94,7 +86,6 @@ func (evpool *Pool) AllPendingEvidence() []types.Evidence {
}
// Update uses the latest block & state to update any evidence that has been committed, to prune all expired evidence
// and to check if any trial period of potential amnesia evidence has finished.
func (evpool *Pool) Update(block *types.Block, state sm.State) {
// sanity check
if state.LastBlockHeight != block.Height {
@ -114,32 +105,11 @@ func (evpool *Pool) Update(block *types.Block, state sm.State) {
// prune pending, committed and potential evidence and polc's periodically
if block.Height%state.ConsensusParams.Evidence.MaxAgeNumBlocks == 0 {
evpool.logger.Debug("Pruning expired evidence")
evpool.pruneExpiredPOLC()
// NOTE: As this is periodic, this implies that there may be some pending evidence in the
// db that have already expired. However, expired evidence will also be removed whenever
// PendingEvidence() is called ensuring that no expired evidence is proposed.
evpool.removeExpiredPendingEvidence()
}
if evpool.nextEvidenceTrialEndedHeight > 0 && block.Height > evpool.nextEvidenceTrialEndedHeight {
evpool.logger.Debug("Upgrading all potential amnesia evidence that have served the trial period")
evpool.nextEvidenceTrialEndedHeight = evpool.upgradePotentialAmnesiaEvidence()
}
}
// 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)
pbplc, err := polc.ToProto()
if err != nil {
return err
}
polcBytes, err := proto.Marshal(pbplc)
if err != nil {
return fmt.Errorf("addPOLC: unable to marshal ProofOfLockChange: %w", err)
}
return evpool.evidenceStore.Set(key, polcBytes)
}
// AddEvidence checks the evidence is valid and adds it to the pool.
@ -147,11 +117,7 @@ func (evpool *Pool) AddEvidence(ev types.Evidence) error {
evpool.logger.Debug("Attempting to add evidence", "ev", ev)
if evpool.Has(ev) {
// if it is an amnesia evidence we have but POLC is not absent then
// we should still process it else we loop to the next piece of evidence
if ae, ok := ev.(*types.AmnesiaEvidence); !ok || ae.Polc.IsAbsent() {
return nil
}
return nil
}
// 1) Verify against state.
@ -159,37 +125,6 @@ func (evpool *Pool) AddEvidence(ev types.Evidence) error {
return types.NewErrEvidenceInvalid(ev, err)
}
// For potential amnesia evidence, if this node is indicted it shall retrieve a polc
// to form AmensiaEvidence else start the trial period for the piece of evidence
if pe, ok := ev.(*types.PotentialAmnesiaEvidence); ok {
if err := evpool.handleInboundPotentialAmnesiaEvidence(pe); err != nil {
return err
}
return nil
} else if ae, ok := ev.(*types.AmnesiaEvidence); ok {
// we have received an new amnesia evidence that we have never seen before so we must extract out the
// potential amnesia evidence part and run our own trial
if ae.Polc.IsAbsent() && ae.PotentialAmnesiaEvidence.VoteA.Round <
ae.PotentialAmnesiaEvidence.VoteB.Round {
if err := evpool.handleInboundPotentialAmnesiaEvidence(ae.PotentialAmnesiaEvidence); err != nil {
return fmt.Errorf("failed to handle amnesia evidence, err: %w", err)
}
return nil
}
// we are going to add this amnesia evidence as it's already punishable.
// We also check if we already have an amnesia evidence or potential
// amnesia evidence that addesses the same case that we will need to remove
aeWithoutPolc := types.NewAmnesiaEvidence(ae.PotentialAmnesiaEvidence, types.NewEmptyPOLC())
if evpool.IsPending(aeWithoutPolc) {
evpool.removePendingEvidence(aeWithoutPolc)
} else if evpool.IsOnTrial(ae.PotentialAmnesiaEvidence) {
key := keyAwaitingTrial(ae.PotentialAmnesiaEvidence)
if err := evpool.evidenceStore.Delete(key); err != nil {
evpool.logger.Error("Failed to remove potential amnesia evidence from database", "err", err)
}
}
}
// 2) Save to store.
if err := evpool.addPendingEvidence(ev); err != nil {
return fmt.Errorf("database error when adding evidence: %v", err)
@ -216,17 +151,6 @@ func (evpool *Pool) Verify(evidence types.Evidence) error {
return nil
}
// if we don't already have amnesia evidence we need to add it to start our own trial period unless
// a) a valid polc has already been attached
// b) the accused node voted back on an earlier round
if ae, ok := evidence.(*types.AmnesiaEvidence); ok && ae.Polc.IsAbsent() && ae.PotentialAmnesiaEvidence.VoteA.Round <
ae.PotentialAmnesiaEvidence.VoteB.Round {
if err := evpool.AddEvidence(ae.PotentialAmnesiaEvidence); err != nil {
return fmt.Errorf("unknown amnesia evidence, trying to add to evidence pool, err: %w", err)
}
return errors.New("amnesia evidence is new and hasn't undergone trial period yet")
}
return evpool.verify(evidence)
}
@ -269,7 +193,7 @@ func (evpool *Pool) MarkEvidenceAsCommitted(height int64, evidence []types.Evide
// Has checks whether the evidence exists either pending or already committed
func (evpool *Pool) Has(evidence types.Evidence) bool {
return evpool.IsPending(evidence) || evpool.IsCommitted(evidence) || evpool.IsOnTrial(evidence)
return evpool.IsPending(evidence) || evpool.IsCommitted(evidence)
}
// IsEvidenceExpired checks whether evidence is past the maximum age where it can be used
@ -309,51 +233,6 @@ func (evpool *Pool) IsPending(evidence types.Evidence) bool {
return ok
}
// IsOnTrial checks whether a piece of evidence is in the awaiting bucket.
// Only Potential Amnesia Evidence is stored here.
func (evpool *Pool) IsOnTrial(evidence types.Evidence) bool {
pe, ok := evidence.(*types.PotentialAmnesiaEvidence)
if !ok {
return false
}
key := keyAwaitingTrial(pe)
ok, err := evpool.evidenceStore.Has(key)
if err != nil {
evpool.logger.Error("Unable to find evidence on trial", "err", err)
}
return ok
}
// 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) (*types.ProofOfLockChange, 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 nil, err
}
// polc doesn't exist
if polcBytes == nil {
return nil, nil
}
err = proto.Unmarshal(polcBytes, &pbpolc)
if err != nil {
return nil, err
}
polc, err := types.ProofOfLockChangeFromProto(&pbpolc)
if err != nil {
return nil, err
}
return polc, err
}
// EvidenceFront goes to the first evidence in the clist
func (evpool *Pool) EvidenceFront() *clist.CElement {
return evpool.evidenceList.Front()
@ -488,175 +367,12 @@ func (evpool *Pool) removeEvidenceFromList(
}
}
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 (
pbproof tmproto.ProofOfLockChange
)
err := proto.Unmarshal(proofBytes, &pbproof)
if err != nil {
evpool.logger.Error("Unable to unmarshal POLC", "err", err)
continue
}
proof, err := types.ProofOfLockChangeFromProto(&pbproof)
if err != nil {
evpool.logger.Error("Unable to transition POLC from protobuf", "err", err)
continue
}
if !evpool.IsExpired(proof.Height(), 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 (evpool *Pool) updateState(state sm.State) {
evpool.mtx.Lock()
defer evpool.mtx.Unlock()
evpool.state = state
}
// 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{baseKeyAwaitingTrial})
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
currentHeight := evpool.State().LastBlockHeight
// 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("Converting from proto to evidence", "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, currentHeight) {
ae := types.NewAmnesiaEvidence(pe, types.NewEmptyPOLC())
err := evpool.addPendingEvidence(ae)
if err != nil {
evpool.logger.Error("Unable to add amnesia evidence", "err", err)
continue
}
evpool.logger.Info("Upgraded to amnesia evidence", "amnesiaEvidence", ae)
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 is not ready to be upgraded. Ready at", "height",
pe.HeightStamp+trialPeriod, "currentHeight", currentHeight)
// 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 (evpool *Pool) handleInboundPotentialAmnesiaEvidence(pe *types.PotentialAmnesiaEvidence) error {
var (
height = pe.Height()
exists = false
polc *types.ProofOfLockChange
err error
)
evpool.logger.Debug("Received Potential Amnesia Evidence", "pe", pe)
// 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 polc != nil && !polc.IsAbsent() {
evpool.logger.Debug("Found polc for potential amnesia evidence", "polc", polc)
// 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.NewAmnesiaEvidence(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
}
}
// stamp height that the evidence was received
pe.HeightStamp = evpool.State().LastBlockHeight
// 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) {
evpool.logger.Debug("PotentialAmnesiaEvidence can be instantly upgraded")
err := evpool.AddEvidence(types.NewAmnesiaEvidence(pe, types.NewEmptyPOLC()))
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 := keyAwaitingTrial(pe)
err = evpool.evidenceStore.Set(key, evBytes)
if err != nil {
return err
}
evpool.logger.Debug("Valid potential amnesia evidence has been added. Starting trial period",
"ev", pe)
// keep track of when the next pe has finished the trial period
if evpool.nextEvidenceTrialEndedHeight == -1 {
evpool.nextEvidenceTrialEndedHeight = pe.Height() + evpool.State().ConsensusParams.Evidence.ProofTrialPeriod
}
// add to the broadcast list so it can continue to be gossiped
evpool.evidenceList.PushBack(pe)
}
return nil
}
func evMapKey(ev types.Evidence) string {
return string(ev.Hash())
}
@ -674,18 +390,6 @@ func keyPending(evidence types.Evidence) []byte {
return append([]byte{baseKeyPending}, keySuffix(evidence)...)
}
func keyAwaitingTrial(evidence types.Evidence) []byte {
return append([]byte{baseKeyAwaitingTrial}, keySuffix(evidence)...)
}
func keyPOLC(polc *types.ProofOfLockChange) []byte {
return keyPOLCFromHeightAndRound(polc.Height(), polc.Round())
}
func keyPOLCFromHeightAndRound(height int64, round int32) []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()))
}

+ 3
- 282
evidence/pool_test.go View File

@ -12,11 +12,7 @@ import (
dbm "github.com/tendermint/tm-db"
"github.com/tendermint/tendermint/crypto/tmhash"
"github.com/tendermint/tendermint/evidence/mocks"
"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/tendermint/types"
sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/store"
@ -161,72 +157,6 @@ func TestEvidencePoolUpdate(t *testing.T) {
assert.True(t, pool.IsCommitted(evidence))
}
func TestAddingAndPruningPOLC(t *testing.T) {
var (
val = types.NewMockPV()
expiredHeight = int64(1)
firstBlockID = types.BlockID{
Hash: tmrand.Bytes(tmhash.Size),
PartSetHeader: types.PartSetHeader{
Total: 1,
Hash: tmrand.Bytes(tmhash.Size),
},
}
stateStore = initializeValidatorState(val, expiredHeight)
blockStore = &mocks.BlockStore{}
expiredEvidenceTime = time.Date(2018, 1, 1, 0, 0, 0, 0, time.UTC)
)
pool, err := NewPool(dbm.NewMemDB(), stateStore, blockStore)
require.NoError(t, err)
pool.SetLogger(log.TestingLogger())
state := pool.State()
height := state.ConsensusParams.Evidence.MaxAgeNumBlocks * 2
blockStore.On("LoadBlockMeta", mock.AnythingOfType("int64")).Return(
&types.BlockMeta{Header: types.Header{Time: expiredEvidenceTime}},
)
voteA := makeVote(1, 1, 0, val.PrivKey.PubKey().Address(), firstBlockID, expiredEvidenceTime)
vA := voteA.ToProto()
err = val.SignVote(evidenceChainID, vA)
require.NoError(t, err)
voteA.Signature = vA.Signature
pubKey, _ := types.NewMockPV().GetPubKey()
polc := &types.ProofOfLockChange{
Votes: []*types.Vote{voteA},
PubKey: pubKey,
}
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 because it doesn't exist
emptyPolc, err := pool.RetrievePOLC(2, 1)
assert.NoError(t, err)
assert.Nil(t, emptyPolc)
lastCommit := makeCommit(height-1, val.PrivKey.PubKey().Address())
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)
assert.NoError(t, err)
assert.Nil(t, emptyPolc)
}
func TestVerifyEvidenceCommittedEvidenceFails(t *testing.T) {
height := int64(1)
pool, _ := defaultTestPool(height)
@ -302,201 +232,6 @@ func TestRecoverPendingEvidence(t *testing.T) {
assert.False(t, pool.Has(expiredEvidence))
}
// Comprehensive set of test cases relating to the adding, upgrading and overall
// processing of PotentialAmnesiaEvidence and AmnesiaEvidence
func TestAmnesiaEvidence(t *testing.T) {
var (
val = types.NewMockPV()
val2 = types.NewMockPV()
pubKey = val.PrivKey.PubKey()
pubKey2 = val2.PrivKey.PubKey()
valSet = &types.ValidatorSet{
Validators: []*types.Validator{
val.ExtractIntoValidator(1),
val2.ExtractIntoValidator(3),
},
Proposer: val.ExtractIntoValidator(1),
}
height = int64(30)
stateStore = initializeStateFromValidatorSet(valSet, height)
evidenceDB = dbm.NewMemDB()
state = stateStore.LoadState()
blockStore = &mocks.BlockStore{}
//evidenceTime = time.Date(2019, 1, 1, 0, 0, 0, 0, time.UTC)
firstBlockID = types.BlockID{
Hash: tmrand.Bytes(tmhash.Size),
PartSetHeader: types.PartSetHeader{
Total: 1,
Hash: tmrand.Bytes(tmhash.Size),
},
}
secondBlockID = types.BlockID{
Hash: tmrand.Bytes(tmhash.Size),
PartSetHeader: types.PartSetHeader{
Total: 1,
Hash: tmrand.Bytes(tmhash.Size),
},
}
evidenceTime = time.Date(2019, 1, 1, 0, 0, 0, 0, time.UTC)
)
blockStore.On("LoadBlockMeta", mock.AnythingOfType("int64")).Return(
&types.BlockMeta{Header: types.Header{Time: evidenceTime}},
)
// TEST SETUP
pool, err := NewPool(evidenceDB, stateStore, blockStore)
require.NoError(t, err)
pool.SetLogger(log.TestingLogger())
voteA := makeVote(height, 0, 0, pubKey.Address(), firstBlockID, evidenceTime)
vA := voteA.ToProto()
err = val.SignVote(evidenceChainID, vA)
voteA.Signature = vA.Signature
require.NoError(t, err)
voteB := makeVote(height, 1, 0, pubKey.Address(), secondBlockID, evidenceTime.Add(3*time.Second))
vB := voteB.ToProto()
err = val.SignVote(evidenceChainID, vB)
voteB.Signature = vB.Signature
require.NoError(t, err)
voteC := makeVote(height, 2, 0, pubKey.Address(), firstBlockID, evidenceTime.Add(2*time.Second))
vC := voteC.ToProto()
err = val.SignVote(evidenceChainID, vC)
voteC.Signature = vC.Signature
require.NoError(t, err)
ev := &types.PotentialAmnesiaEvidence{
VoteA: voteA,
VoteB: voteB,
Timestamp: evidenceTime,
}
polc := &types.ProofOfLockChange{
Votes: []*types.Vote{voteB},
PubKey: pubKey2,
}
err = pool.AddPOLC(polc)
require.NoError(t, err)
polc, err = pool.RetrievePOLC(height, 1)
require.NoError(t, err)
require.NotEmpty(t, polc)
secondValVote := makeVote(height, 1, 0, pubKey2.Address(), secondBlockID, evidenceTime.Add(1*time.Second))
vv2 := secondValVote.ToProto()
err = val2.SignVote(evidenceChainID, vv2)
require.NoError(t, err)
secondValVote.Signature = vv2.Signature
validPolc := &types.ProofOfLockChange{
Votes: []*types.Vote{secondValVote},
PubKey: pubKey,
}
// CASE A
pool.logger.Info("CASE A")
// 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)))
assert.True(t, pool.IsOnTrial(ev))
nextHeight := pool.nextEvidenceTrialEndedHeight
assert.Greater(t, nextHeight, int64(0))
// CASE B
pool.logger.Info("CASE B")
// 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)
// CASE C
pool.logger.Info("CASE C")
// 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
lastCommit := makeCommit(height+1, pubKey.Address())
block := types.MakeBlock(height+2, []types.Tx{}, lastCommit, []types.Evidence{})
state.LastBlockHeight = height + 2
pool.Update(block, state)
assert.Equal(t, int64(-1), pool.nextEvidenceTrialEndedHeight)
assert.Equal(t, 1, len(pool.PendingEvidence(1)))
// CASE D
pool.logger.Info("CASE D")
// evidence of voting back in the past which is instantly punishable -> amnesia evidence is made directly
ev2 := &types.PotentialAmnesiaEvidence{
VoteA: voteC,
VoteB: voteB,
Timestamp: evidenceTime,
}
err = pool.AddEvidence(ev2)
assert.NoError(t, err)
expectedAe := &types.AmnesiaEvidence{
PotentialAmnesiaEvidence: ev2,
Polc: types.NewEmptyPOLC(),
}
assert.True(t, pool.IsPending(expectedAe))
assert.Equal(t, 2, len(pool.AllPendingEvidence()))
// CASE E
pool.logger.Info("CASE E")
// test for receiving amnesia evidence
ae := types.NewAmnesiaEvidence(ev, types.NewEmptyPOLC())
// we need to run the trial period ourselves so amnesia evidence should not be added, instead
// we should extract out the potential amnesia evidence and trying to add that before realising
// that we already have it -> no error
err = pool.AddEvidence(ae)
assert.NoError(t, err)
assert.Equal(t, 2, len(pool.AllPendingEvidence()))
voteD := makeVote(height, 2, 0, pubKey.Address(), firstBlockID, evidenceTime.Add(4*time.Second))
vD := voteD.ToProto()
err = val.SignVote(evidenceChainID, vD)
require.NoError(t, err)
voteD.Signature = vD.Signature
// CASE F
pool.logger.Info("CASE F")
// a new amnesia evidence is seen. It has an empty polc so we should extract the potential amnesia evidence
// and start our own trial
newPe := &types.PotentialAmnesiaEvidence{
VoteA: voteB,
VoteB: voteD,
Timestamp: evidenceTime,
}
newAe := &types.AmnesiaEvidence{
PotentialAmnesiaEvidence: newPe,
Polc: types.NewEmptyPOLC(),
}
err = pool.AddEvidence(newAe)
assert.NoError(t, err)
assert.Equal(t, 2, len(pool.AllPendingEvidence()))
assert.True(t, pool.IsOnTrial(newPe))
// CASE G
pool.logger.Info("CASE G")
// Finally, we receive an amnesia evidence containing a valid polc for an earlier potential amnesia evidence
// that we have already upgraded to. We should ad this new amnesia evidence in replace of the prior
// amnesia evidence with an empty polc that we have
aeWithPolc := &types.AmnesiaEvidence{
PotentialAmnesiaEvidence: ev,
Polc: validPolc,
}
err = pool.AddEvidence(aeWithPolc)
assert.NoError(t, err)
assert.True(t, pool.IsPending(aeWithPolc))
assert.Equal(t, 2, len(pool.AllPendingEvidence()))
t.Log(pool.AllPendingEvidence())
}
func initializeStateFromValidatorSet(valSet *types.ValidatorSet, height int64) StateStore {
stateDB := dbm.NewMemDB()
state := sm.State{
@ -514,10 +249,9 @@ func initializeStateFromValidatorSet(valSet *types.ValidatorSet, height int64) S
MaxGas: -1,
},
Evidence: tmproto.EvidenceParams{
MaxAgeNumBlocks: 20,
MaxAgeDuration: 48 * time.Hour,
MaxNum: 50,
ProofTrialPeriod: 1,
MaxAgeNumBlocks: 20,
MaxAgeDuration: 48 * time.Hour,
MaxNum: 50,
},
},
}
@ -575,19 +309,6 @@ 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, time time.Time) *types.Vote {
return &types.Vote{
Type: tmproto.SignedMsgType(2),
Height: height,
Round: round,
BlockID: blockID,
Timestamp: time,
ValidatorAddress: addr,
ValidatorIndex: index,
}
}
func defaultTestPool(height int64) (*Pool, types.MockPV) {
val := types.NewMockPV()
valAddress := val.PrivKey.PubKey().Address()


+ 0
- 10
evidence/verify.go View File

@ -61,16 +61,6 @@ func VerifyEvidence(evidence types.Evidence, state sm.State, stateDB StateStore,
return err
}
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)
}
}
}
addr := evidence.Address()
var val *types.Validator


+ 212
- 1199
proto/tendermint/types/evidence.pb.go
File diff suppressed because it is too large
View File


+ 1
- 22
proto/tendermint/types/evidence.proto View File

@ -18,25 +18,9 @@ message DuplicateVoteEvidence {
[(gogoproto.nullable) = false, (gogoproto.stdtime) = true];
}
message PotentialAmnesiaEvidence {
Vote vote_a = 1;
Vote vote_b = 2;
int64 height_stamp = 3;
google.protobuf.Timestamp timestamp = 4
[(gogoproto.nullable) = false, (gogoproto.stdtime) = true];
}
message AmnesiaEvidence {
PotentialAmnesiaEvidence potential_amnesia_evidence = 1;
ProofOfLockChange polc = 2;
}
message Evidence {
oneof sum {
DuplicateVoteEvidence duplicate_vote_evidence = 1;
PotentialAmnesiaEvidence potential_amnesia_evidence = 2;
AmnesiaEvidence amnesia_evidence = 3;
DuplicateVoteEvidence duplicate_vote_evidence = 1;
}
}
@ -45,8 +29,3 @@ message EvidenceData {
repeated Evidence evidence = 1 [(gogoproto.nullable) = false];
bytes hash = 2;
}
message ProofOfLockChange {
repeated Vote votes = 1;
tendermint.crypto.PublicKey pub_key = 2;
}

+ 36
- 78
proto/tendermint/types/params.pb.go View File

@ -184,10 +184,6 @@ 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{} }
@ -244,13 +240,6 @@ 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 {
@ -409,43 +398,42 @@ func init() {
func init() { proto.RegisterFile("tendermint/types/params.proto", fileDescriptor_e12598271a686f57) }
var fileDescriptor_e12598271a686f57 = []byte{
// 572 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x6c, 0x53, 0xcd, 0x6e, 0xd3, 0x4c,
0x14, 0x8d, 0xeb, 0x7e, 0x6d, 0x7a, 0xd3, 0x34, 0xd1, 0xe8, 0x93, 0x08, 0x45, 0xb5, 0x83, 0x17,
0xa8, 0x12, 0xc8, 0x96, 0x60, 0x81, 0xe8, 0xa6, 0xc2, 0x50, 0x15, 0x84, 0x52, 0x55, 0x56, 0x61,
0xd1, 0x8d, 0x35, 0x8e, 0xa7, 0xae, 0xd5, 0x8c, 0x67, 0xe4, 0x19, 0x47, 0xc9, 0x5b, 0xb0, 0xec,
0xb2, 0x4b, 0x1e, 0x81, 0x47, 0xe8, 0xb2, 0x12, 0x1b, 0x56, 0x80, 0x92, 0x0d, 0x8f, 0x81, 0x3c,
0xce, 0x90, 0x9f, 0xb2, 0xb3, 0xef, 0xf9, 0x99, 0xb9, 0xe7, 0x68, 0x60, 0x4f, 0x92, 0x2c, 0x26,
0x39, 0x4d, 0x33, 0xe9, 0xc9, 0x31, 0x27, 0xc2, 0xe3, 0x38, 0xc7, 0x54, 0xb8, 0x3c, 0x67, 0x92,
0xa1, 0xf6, 0x1c, 0x76, 0x15, 0xbc, 0xfb, 0x7f, 0xc2, 0x12, 0xa6, 0x40, 0xaf, 0xfc, 0xaa, 0x78,
0xbb, 0x56, 0xc2, 0x58, 0x32, 0x20, 0x9e, 0xfa, 0x8b, 0x8a, 0x0b, 0x2f, 0x2e, 0x72, 0x2c, 0x53,
0x96, 0x55, 0xb8, 0x73, 0xbd, 0x06, 0xad, 0x37, 0x2c, 0x13, 0x24, 0x13, 0x85, 0x38, 0x55, 0x27,
0xa0, 0x57, 0xf0, 0x5f, 0x34, 0x60, 0xfd, 0xab, 0x8e, 0xd1, 0x35, 0xf6, 0x1b, 0xcf, 0xf7, 0xdc,
0xd5, 0xb3, 0x5c, 0xbf, 0x84, 0x2b, 0xb6, 0xbf, 0x7e, 0xfb, 0xc3, 0xae, 0x05, 0x95, 0x02, 0xf9,
0x50, 0x27, 0xc3, 0x34, 0x26, 0x59, 0x9f, 0x74, 0xd6, 0x94, 0xba, 0x7b, 0x5f, 0x7d, 0x34, 0x63,
0x2c, 0x19, 0xfc, 0xd5, 0xa1, 0x23, 0xd8, 0x1a, 0xe2, 0x41, 0x1a, 0x63, 0xc9, 0xf2, 0x8e, 0xa9,
0x4c, 0x1e, 0xdf, 0x37, 0xf9, 0xa4, 0x29, 0x4b, 0x2e, 0x73, 0x25, 0x3a, 0x84, 0xcd, 0x21, 0xc9,
0x45, 0xca, 0xb2, 0xce, 0xba, 0x32, 0xb1, 0xff, 0x61, 0x52, 0x11, 0x96, 0x2c, 0xb4, 0xca, 0x21,
0xd0, 0x58, 0xd8, 0x13, 0x3d, 0x82, 0x2d, 0x8a, 0x47, 0x61, 0x34, 0x96, 0x44, 0xa8, 0x64, 0xcc,
0xa0, 0x4e, 0xf1, 0xc8, 0x2f, 0xff, 0xd1, 0x03, 0xd8, 0x2c, 0xc1, 0x04, 0x0b, 0xb5, 0xb6, 0x19,
0x6c, 0x50, 0x3c, 0x3a, 0xc6, 0x02, 0x75, 0x61, 0x5b, 0xa6, 0x94, 0x84, 0x29, 0x93, 0x38, 0xa4,
0x42, 0xed, 0x63, 0x06, 0x50, 0xce, 0xde, 0x33, 0x89, 0x7b, 0xc2, 0xf9, 0x66, 0xc0, 0xce, 0x72,
0x22, 0xe8, 0x29, 0xa0, 0xd2, 0x0d, 0x27, 0x24, 0xcc, 0x0a, 0x1a, 0xaa, 0x68, 0xf5, 0x99, 0x2d,
0x8a, 0x47, 0xaf, 0x13, 0x72, 0x52, 0x50, 0x75, 0x39, 0x81, 0x7a, 0xd0, 0xd6, 0x64, 0xdd, 0xed,
0x2c, 0xfa, 0x87, 0x6e, 0x55, 0xbe, 0xab, 0xcb, 0x77, 0xdf, 0xce, 0x08, 0x7e, 0xbd, 0x5c, 0xf5,
0xfa, 0xa7, 0x6d, 0x04, 0x3b, 0x95, 0x9f, 0x46, 0xf4, 0x26, 0x59, 0x41, 0xd5, 0x5d, 0x9b, 0x6a,
0x93, 0x93, 0x82, 0xa2, 0x67, 0x80, 0x78, 0xce, 0xd8, 0x45, 0x28, 0xf3, 0x14, 0x0f, 0x42, 0x4e,
0xf2, 0x94, 0xc5, 0x2a, 0x5a, 0x33, 0x68, 0x2b, 0xe4, 0xac, 0x04, 0x4e, 0xd5, 0xdc, 0x39, 0x84,
0xd6, 0x4a, 0x43, 0xc8, 0x81, 0x26, 0x2f, 0xa2, 0xf0, 0x8a, 0x8c, 0x43, 0x95, 0x7e, 0xc7, 0xe8,
0x9a, 0xfb, 0x5b, 0x41, 0x83, 0x17, 0xd1, 0x07, 0x32, 0x3e, 0x2b, 0x47, 0x07, 0xf5, 0xaf, 0x37,
0xb6, 0xf1, 0xfb, 0xc6, 0x36, 0x9c, 0x03, 0x68, 0x2e, 0xb5, 0x83, 0x6c, 0x68, 0x60, 0xce, 0x43,
0xdd, 0x69, 0x99, 0xc6, 0x7a, 0x00, 0x98, 0xf3, 0x19, 0x6d, 0x41, 0x7b, 0x0e, 0xdb, 0xef, 0xb0,
0xb8, 0x24, 0xf1, 0x4c, 0xfa, 0x04, 0x5a, 0x2a, 0xc3, 0x70, 0xb5, 0xc0, 0xa6, 0x1a, 0xf7, 0x74,
0x8b, 0x0e, 0x34, 0xe7, 0xbc, 0x79, 0x97, 0x0d, 0xcd, 0x3a, 0xc6, 0xc2, 0xff, 0xf8, 0x65, 0x62,
0x19, 0xb7, 0x13, 0xcb, 0xb8, 0x9b, 0x58, 0xc6, 0xaf, 0x89, 0x65, 0x7c, 0x9e, 0x5a, 0xb5, 0xbb,
0xa9, 0x55, 0xfb, 0x3e, 0xb5, 0x6a, 0xe7, 0x2f, 0x93, 0x54, 0x5e, 0x16, 0x91, 0xdb, 0x67, 0xd4,
0x5b, 0x7c, 0xc0, 0xf3, 0xcf, 0xea, 0x85, 0xae, 0x3e, 0xee, 0x68, 0x43, 0xcd, 0x5f, 0xfc, 0x09,
0x00, 0x00, 0xff, 0xff, 0xf4, 0xdd, 0x09, 0xef, 0xf7, 0x03, 0x00, 0x00,
// 545 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x6c, 0x53, 0x4f, 0x6f, 0xd3, 0x30,
0x1c, 0xad, 0xd7, 0xb2, 0xb5, 0xbf, 0xae, 0xeb, 0x64, 0x21, 0x51, 0x86, 0x96, 0x94, 0x1c, 0xd0,
0x24, 0xa4, 0x44, 0x82, 0x03, 0x62, 0x97, 0x89, 0xc0, 0x34, 0x10, 0xea, 0x84, 0x22, 0xe0, 0xb0,
0x4b, 0xe4, 0xb4, 0x26, 0x8b, 0x56, 0xc7, 0x51, 0x6c, 0x57, 0xed, 0xb7, 0xe0, 0xb8, 0xe3, 0x2e,
0x48, 0x7c, 0x04, 0x3e, 0xc2, 0x8e, 0x3b, 0x72, 0x02, 0xd4, 0x5e, 0xf8, 0x18, 0x28, 0x4e, 0x4d,
0xff, 0x6c, 0xb7, 0xe4, 0xf7, 0x7b, 0xef, 0xd9, 0xef, 0x3d, 0x19, 0xf6, 0x25, 0x4d, 0x07, 0x34,
0x67, 0x49, 0x2a, 0x3d, 0x39, 0xc9, 0xa8, 0xf0, 0x32, 0x92, 0x13, 0x26, 0xdc, 0x2c, 0xe7, 0x92,
0xe3, 0xdd, 0xc5, 0xda, 0xd5, 0xeb, 0xbd, 0xfb, 0x31, 0x8f, 0xb9, 0x5e, 0x7a, 0xc5, 0x57, 0x89,
0xdb, 0xb3, 0x62, 0xce, 0xe3, 0x21, 0xf5, 0xf4, 0x5f, 0xa4, 0xbe, 0x78, 0x03, 0x95, 0x13, 0x99,
0xf0, 0xb4, 0xdc, 0x3b, 0x97, 0x1b, 0xd0, 0x7e, 0xcd, 0x53, 0x41, 0x53, 0xa1, 0xc4, 0x07, 0x7d,
0x02, 0x7e, 0x09, 0xf7, 0xa2, 0x21, 0xef, 0x5f, 0x74, 0x50, 0x17, 0x1d, 0x34, 0x9f, 0xed, 0xbb,
0xeb, 0x67, 0xb9, 0x7e, 0xb1, 0x2e, 0xd1, 0x7e, 0xed, 0xfa, 0x97, 0x5d, 0x09, 0x4a, 0x06, 0xf6,
0xa1, 0x4e, 0x47, 0xc9, 0x80, 0xa6, 0x7d, 0xda, 0xd9, 0xd0, 0xec, 0xee, 0x6d, 0xf6, 0xf1, 0x1c,
0xb1, 0x22, 0xf0, 0x9f, 0x87, 0x8f, 0xa1, 0x31, 0x22, 0xc3, 0x64, 0x40, 0x24, 0xcf, 0x3b, 0x55,
0x2d, 0xf2, 0xf8, 0xb6, 0xc8, 0x67, 0x03, 0x59, 0x51, 0x59, 0x30, 0xf1, 0x11, 0x6c, 0x8d, 0x68,
0x2e, 0x12, 0x9e, 0x76, 0x6a, 0x5a, 0xc4, 0xbe, 0x43, 0xa4, 0x04, 0xac, 0x48, 0x18, 0x96, 0x43,
0xa1, 0xb9, 0xe4, 0x13, 0x3f, 0x82, 0x06, 0x23, 0xe3, 0x30, 0x9a, 0x48, 0x2a, 0x74, 0x32, 0xd5,
0xa0, 0xce, 0xc8, 0xd8, 0x2f, 0xfe, 0xf1, 0x03, 0xd8, 0x2a, 0x96, 0x31, 0x11, 0xda, 0x76, 0x35,
0xd8, 0x64, 0x64, 0x7c, 0x42, 0x04, 0xee, 0xc2, 0xb6, 0x4c, 0x18, 0x0d, 0x13, 0x2e, 0x49, 0xc8,
0x84, 0xf6, 0x53, 0x0d, 0xa0, 0x98, 0xbd, 0xe3, 0x92, 0xf4, 0x84, 0xf3, 0x0d, 0xc1, 0xce, 0x6a,
0x22, 0xf8, 0x29, 0xe0, 0x42, 0x8d, 0xc4, 0x34, 0x4c, 0x15, 0x0b, 0x75, 0xb4, 0xe6, 0xcc, 0x36,
0x23, 0xe3, 0x57, 0x31, 0x3d, 0x55, 0x4c, 0x5f, 0x4e, 0xe0, 0x1e, 0xec, 0x1a, 0xb0, 0xe9, 0x76,
0x1e, 0xfd, 0x43, 0xb7, 0x2c, 0xdf, 0x35, 0xe5, 0xbb, 0x6f, 0xe6, 0x00, 0xbf, 0x5e, 0x58, 0xbd,
0xfc, 0x6d, 0xa3, 0x60, 0xa7, 0xd4, 0x33, 0x1b, 0xe3, 0x24, 0x55, 0x4c, 0xdf, 0xb5, 0xa5, 0x9d,
0x9c, 0x2a, 0xe6, 0x1c, 0x41, 0x7b, 0x2d, 0x73, 0xec, 0x40, 0x2b, 0x53, 0x51, 0x78, 0x41, 0x27,
0xa1, 0xce, 0xb3, 0x83, 0xba, 0xd5, 0x83, 0x46, 0xd0, 0xcc, 0x54, 0xf4, 0x9e, 0x4e, 0x3e, 0x16,
0xa3, 0xc3, 0xfa, 0x8f, 0x2b, 0x1b, 0xfd, 0xbd, 0xb2, 0x91, 0x73, 0x08, 0xad, 0x95, 0xbc, 0xb1,
0x0d, 0x4d, 0x92, 0x65, 0xa1, 0x69, 0xa9, 0xf0, 0x57, 0x0b, 0x80, 0x64, 0xd9, 0x1c, 0xb6, 0xc4,
0x3d, 0x83, 0xed, 0xb7, 0x44, 0x9c, 0xd3, 0xc1, 0x9c, 0xfa, 0x04, 0xda, 0x3a, 0x95, 0x70, 0xbd,
0x92, 0x96, 0x1e, 0xf7, 0x4c, 0x2f, 0x0e, 0xb4, 0x16, 0xb8, 0x45, 0x3b, 0x4d, 0x83, 0x3a, 0x21,
0xc2, 0xff, 0xf4, 0x7d, 0x6a, 0xa1, 0xeb, 0xa9, 0x85, 0x6e, 0xa6, 0x16, 0xfa, 0x33, 0xb5, 0xd0,
0xd7, 0x99, 0x55, 0xb9, 0x99, 0x59, 0x95, 0x9f, 0x33, 0xab, 0x72, 0xf6, 0x22, 0x4e, 0xe4, 0xb9,
0x8a, 0xdc, 0x3e, 0x67, 0xde, 0xf2, 0x93, 0x5c, 0x7c, 0x96, 0x6f, 0x6e, 0xfd, 0xb9, 0x46, 0x9b,
0x7a, 0xfe, 0xfc, 0x5f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x53, 0x0a, 0x6c, 0x0d, 0xc9, 0x03, 0x00,
0x00,
}
func (this *ConsensusParams) Equal(that interface{}) bool {
@ -539,9 +527,6 @@ 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 {
@ -745,11 +730,6 @@ 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--
@ -1016,9 +996,6 @@ 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
}
@ -1465,25 +1442,6 @@ 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:])


+ 0
- 5
proto/tendermint/types/params.proto View File

@ -53,11 +53,6 @@ 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.


+ 1
- 2
tools/tm-signer-harness/internal/test_harness_test.go View File

@ -48,8 +48,7 @@ const (
"evidence": {
"max_age_num_blocks": "100000",
"max_age_duration": "172800000000000",
"max_num": 50,
"proof_trial_period": "50000"
"max_num": 50
},
"validator": {
"pub_key_types": [


+ 0
- 6
types/block.go View File

@ -87,12 +87,6 @@ func (b *Block) ValidateBasic() error {
// NOTE: b.Evidence.Evidence may be nil, but we're just looping.
for i, ev := range b.Evidence.Evidence {
if _, ok := ev.(*PotentialAmnesiaEvidence); ok {
// PotentialAmnesiaEvidence does not contribute to anything on its own, so
// reject it as well.
return fmt.Errorf("found PotentialAmnesiaEvidence (#%d)", i)
}
if err := ev.ValidateBasic(); err != nil {
return fmt.Errorf("invalid evidence (#%d): %v", i, err)
}


+ 0
- 3
types/block_test.go View File

@ -86,9 +86,6 @@ func TestBlockValidateBasic(t *testing.T) {
{"Tampered EvidenceHash", func(blk *Block) {
blk.EvidenceHash = []byte("something else")
}, true},
{"PotentialAmnesiaEvidence", func(blk *Block) {
blk.Evidence = EvidenceData{Evidence: []Evidence{&PotentialAmnesiaEvidence{}}}
}, true},
}
for i, tc := range testCases {
tc := tc


+ 0
- 650
types/evidence.go View File

@ -8,7 +8,6 @@ import (
"time"
"github.com/tendermint/tendermint/crypto"
cryptoenc "github.com/tendermint/tendermint/crypto/encoding"
"github.com/tendermint/tendermint/crypto/merkle"
"github.com/tendermint/tendermint/crypto/tmhash"
tmjson "github.com/tendermint/tendermint/libs/json"
@ -84,27 +83,6 @@ func EvidenceToProto(evidence Evidence) (*tmproto.Evidence, error) {
}
return tp, nil
case *PotentialAmnesiaEvidence:
pbevi := evi.ToProto()
tp := &tmproto.Evidence{
Sum: &tmproto.Evidence_PotentialAmnesiaEvidence{
PotentialAmnesiaEvidence: pbevi,
},
}
return tp, nil
case *AmnesiaEvidence:
aepb := evi.ToProto()
tp := &tmproto.Evidence{
Sum: &tmproto.Evidence_AmnesiaEvidence{
AmnesiaEvidence: aepb,
},
}
return tp, nil
default:
return nil, fmt.Errorf("toproto: evidence is not recognized: %T", evi)
}
@ -118,10 +96,6 @@ func EvidenceFromProto(evidence *tmproto.Evidence) (Evidence, error) {
switch evi := evidence.Sum.(type) {
case *tmproto.Evidence_DuplicateVoteEvidence:
return DuplicateVoteEvidenceFromProto(evi.DuplicateVoteEvidence)
case *tmproto.Evidence_PotentialAmnesiaEvidence:
return PotentialAmnesiaEvidenceFromProto(evi.PotentialAmnesiaEvidence)
case *tmproto.Evidence_AmnesiaEvidence:
return AmnesiaEvidenceFromProto(evi.AmnesiaEvidence)
default:
return nil, errors.New("evidence is not recognized")
}
@ -129,8 +103,6 @@ func EvidenceFromProto(evidence *tmproto.Evidence) (Evidence, error) {
func init() {
tmjson.RegisterType(&DuplicateVoteEvidence{}, "tendermint/DuplicateVoteEvidence")
tmjson.RegisterType(&PotentialAmnesiaEvidence{}, "tendermint/PotentialAmnesiaEvidence")
tmjson.RegisterType(&AmnesiaEvidence{}, "tendermint/AmnesiaEvidence")
}
//-------------------------------------------
@ -338,609 +310,6 @@ func DuplicateVoteEvidenceFromProto(pb *tmproto.DuplicateVoteEvidence) (*Duplica
return dve, dve.ValidateBasic()
}
//-------------------------------------------
// PotentialAmnesiaEvidence is constructed when a validator votes on two different blocks at different rounds
// in the same height. PotentialAmnesiaEvidence can then evolve into AmnesiaEvidence 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. Votes are ordered based on their timestamp
type PotentialAmnesiaEvidence struct {
VoteA *Vote `json:"vote_a"`
VoteB *Vote `json:"vote_b"`
HeightStamp int64
Timestamp time.Time `json:"timestamp"`
}
var _ Evidence = &PotentialAmnesiaEvidence{}
// NewPotentialAmnesiaEvidence creates a new instance of the evidence and orders the votes correctly
func NewPotentialAmnesiaEvidence(voteA, voteB *Vote, time time.Time) *PotentialAmnesiaEvidence {
if voteA == nil || voteB == nil {
return nil
}
if voteA.Timestamp.Before(voteB.Timestamp) {
return &PotentialAmnesiaEvidence{VoteA: voteA, VoteB: voteB, Timestamp: time}
}
return &PotentialAmnesiaEvidence{VoteA: voteB, VoteB: voteA, Timestamp: time}
}
func (e *PotentialAmnesiaEvidence) Height() int64 {
return e.VoteA.Height
}
func (e *PotentialAmnesiaEvidence) Time() time.Time {
return e.Timestamp
}
func (e *PotentialAmnesiaEvidence) Address() []byte {
return e.VoteA.ValidatorAddress
}
// NOTE: Heightstamp must not be included in hash
func (e *PotentialAmnesiaEvidence) Hash() []byte {
v1, err := e.VoteA.ToProto().Marshal()
if err != nil {
panic(fmt.Errorf("trying to hash potential amnesia evidence, err: %w", err))
}
v2, err := e.VoteB.ToProto().Marshal()
if err != nil {
panic(fmt.Errorf("trying to hash potential amnesia evidence, err: %w", err))
}
return tmhash.Sum(append(v1, v2...))
}
func (e *PotentialAmnesiaEvidence) Bytes() []byte {
pbe := e.ToProto()
bz, err := pbe.Marshal()
if err != nil {
panic(err)
}
return bz
}
func (e *PotentialAmnesiaEvidence) Verify(chainID string, pubKey crypto.PubKey) error {
// pubkey must match address (this should already be true, sanity check)
addr := e.VoteA.ValidatorAddress
if !bytes.Equal(pubKey.Address(), addr) {
return fmt.Errorf("address (%X) doesn't match pubkey (%v - %X)",
addr, pubKey, pubKey.Address())
}
va := e.VoteA.ToProto()
vb := e.VoteB.ToProto()
// Signatures must be valid
if !pubKey.VerifySignature(VoteSignBytes(chainID, va), e.VoteA.Signature) {
return fmt.Errorf("verifying VoteA: %w", ErrVoteInvalidSignature)
}
if !pubKey.VerifySignature(VoteSignBytes(chainID, vb), e.VoteB.Signature) {
return fmt.Errorf("verifying VoteB: %w", ErrVoteInvalidSignature)
}
return nil
}
func (e *PotentialAmnesiaEvidence) Equal(ev Evidence) bool {
if e2, ok := ev.(*PotentialAmnesiaEvidence); ok {
return e.Height() == e2.Height() && e.VoteA.Round == e2.VoteA.Round && e.VoteB.Round == e2.VoteB.Round &&
bytes.Equal(e.Address(), e2.Address())
}
return false
}
func (e *PotentialAmnesiaEvidence) ValidateBasic() error {
if e == nil {
return errors.New("empty potential amnesia evidence")
}
if e.VoteA == nil || e.VoteB == nil {
return fmt.Errorf("one or both of the votes are empty %v, %v", e.VoteA, e.VoteB)
}
if err := e.VoteA.ValidateBasic(); err != nil {
return fmt.Errorf("invalid VoteA: %v", err)
}
if err := e.VoteB.ValidateBasic(); err != nil {
return fmt.Errorf("invalid VoteB: %v", err)
}
// H/S must be the same
if e.VoteA.Height != e.VoteB.Height {
return fmt.Errorf("heights do not match: %d vs %d",
e.VoteA.Height, e.VoteB.Height)
}
if e.VoteA.Round == e.VoteB.Round {
return fmt.Errorf("votes must be for different rounds: %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
if !bytes.Equal(e.VoteA.ValidatorAddress, e.VoteB.ValidatorAddress) {
return fmt.Errorf("validator addresses do not match: %X vs %X",
e.VoteA.ValidatorAddress,
e.VoteB.ValidatorAddress,
)
}
if e.VoteA.BlockID.IsZero() {
return errors.New("first vote is for a nil block - voter hasn't locked on a block")
}
// BlockIDs must be different
if e.VoteA.BlockID.Equals(e.VoteB.BlockID) {
return fmt.Errorf(
"block IDs are the same (%v) - not a real duplicate vote",
e.VoteA.BlockID,
)
}
return nil
}
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
}
func (e *PotentialAmnesiaEvidence) ToProto() *tmproto.PotentialAmnesiaEvidence {
voteB := e.VoteB.ToProto()
voteA := e.VoteA.ToProto()
tp := &tmproto.PotentialAmnesiaEvidence{
VoteA: voteA,
VoteB: voteB,
HeightStamp: e.HeightStamp,
Timestamp: e.Timestamp,
}
return tp
}
// ------------------
// 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 NewPOLCFromVoteSet(voteSet *VoteSet, pubKey crypto.PubKey, blockID BlockID) (*ProofOfLockChange, error) {
polc := newPOLCFromVoteSet(voteSet, pubKey, blockID)
return polc, polc.ValidateBasic()
}
func newPOLCFromVoteSet(voteSet *VoteSet, pubKey crypto.PubKey, blockID BlockID) *ProofOfLockChange {
if voteSet == nil {
return nil
}
var votes []*Vote
valSetSize := voteSet.Size()
for valIdx := int32(0); int(valIdx) < valSetSize; valIdx++ {
vote := voteSet.GetByIndex(valIdx)
if vote != nil && vote.BlockID.Equals(blockID) {
votes = append(votes, vote)
}
}
return NewPOLC(votes, pubKey)
}
// NewPOLC creates a POLC
func NewPOLC(votes []*Vote, pubKey crypto.PubKey) *ProofOfLockChange {
return &ProofOfLockChange{
Votes: votes,
PubKey: pubKey,
}
}
// 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 NewEmptyPOLC() *ProofOfLockChange {
return &ProofOfLockChange{
nil,
nil,
}
}
func (e *ProofOfLockChange) Height() int64 {
return e.Votes[0].Height
}
// Time returns time of the latest 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() int32 {
return e.Votes[0].Round
}
func (e *ProofOfLockChange) Address() []byte {
return e.PubKey.Address()
}
func (e *ProofOfLockChange) BlockID() BlockID {
return e.Votes[0].BlockID
}
// 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 _, vote := range e.Votes {
exists := false
for _, validator := range valSet.Validators {
if bytes.Equal(validator.Address, vote.ValidatorAddress) {
exists = true
v := vote.ToProto()
if !validator.PubKey.VerifySignature(VoteSignBytes(chainID, v), 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())
}
}
if talliedVotingPower <= votingPowerNeeded {
return ErrNotEnoughVotingPowerSigned{
Got: talliedVotingPower,
Needed: votingPowerNeeded + 1,
}
}
return nil
}
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 == nil {
return errors.New("empty proof of lock change")
}
// first check if the polc is absent / empty
if e.IsAbsent() {
return nil
}
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 || len(e.Votes) == 0 {
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 vote == nil {
return fmt.Errorf("nil vote at index: %d", idx)
}
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.PubKey.Address().Bytes())
}
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 {
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) {
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 plc, errors.New("invalid proof of lock change (no votes), did you forget to validate?")
}
for i, v := range e.Votes {
pb := v.ToProto()
if pb != nil {
vpb[i] = pb
}
}
pk, err := cryptoenc.PubKeyToProto(e.PubKey)
if err != nil {
return plc, fmt.Errorf("invalid proof of lock change (err: %w), did you forget to validate?", err)
}
plc.PubKey = &pk
plc.Votes = vpb
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, and Hash functions are all inherited by the PotentialAmnesiaEvidence struct
var _ Evidence = &AmnesiaEvidence{}
func NewAmnesiaEvidence(pe *PotentialAmnesiaEvidence, proof *ProofOfLockChange) *AmnesiaEvidence {
return &AmnesiaEvidence{
pe,
proof,
}
}
// Note: Amnesia evidence with or without a polc are considered the same
func (e *AmnesiaEvidence) Equal(ev Evidence) bool {
if e2, ok := ev.(*AmnesiaEvidence); ok {
return e.PotentialAmnesiaEvidence.Equal(e2.PotentialAmnesiaEvidence)
}
return false
}
func (e *AmnesiaEvidence) Bytes() []byte {
pbe := e.ToProto()
bz, err := pbe.Marshal()
if err != nil {
panic(fmt.Errorf("converting amnesia evidence to bytes, err: %w", err))
}
return bz
}
func (e *AmnesiaEvidence) ValidateBasic() error {
if e == nil {
return errors.New("empty amnesia evidence")
}
if e.Polc == nil || e.PotentialAmnesiaEvidence == nil {
return errors.New("amnesia evidence is missing either the polc or the potential amnesia evidence")
}
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 AmnesiaEvidence 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 (e *AmnesiaEvidence) ToProto() *tmproto.AmnesiaEvidence {
paepb := e.PotentialAmnesiaEvidence.ToProto()
polc, err := e.Polc.ToProto()
if err != nil {
polc, _ = NewEmptyPOLC().ToProto()
}
return &tmproto.AmnesiaEvidence{
PotentialAmnesiaEvidence: paepb,
Polc: 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)
if err != nil {
return nil, err
}
vpb[i] = vi
}
if pb.PubKey == nil {
return nil, errors.New("proofOfLockChange: is not absent but has nil PubKey")
}
pk, err := cryptoenc.PubKeyFromProto(*pb.PubKey)
if err != nil {
return nil, err
}
plc.PubKey = pk
plc.Votes = vpb
return plc, plc.ValidateBasic()
}
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(),
Timestamp: pb.Timestamp,
}
return &tp, tp.ValidateBasic()
}
func AmnesiaEvidenceFromProto(pb *tmproto.AmnesiaEvidence) (*AmnesiaEvidence, error) {
if pb == nil {
return nil, errors.New("nil amnesia evidence")
}
pae, err := PotentialAmnesiaEvidenceFromProto(pb.PotentialAmnesiaEvidence)
if err != nil {
return nil, fmt.Errorf("decoding to amnesia evidence, err: %w", err)
}
polc, err := ProofOfLockChangeFromProto(pb.Polc)
if err != nil {
return nil, fmt.Errorf("decoding to amnesia evidence, err: %w", err)
}
tp := &AmnesiaEvidence{
PotentialAmnesiaEvidence: pae,
Polc: polc,
}
return tp, tp.ValidateBasic()
}
//--------------------------------------------------
// EvidenceList is a list of Evidence. Evidences is not a word.
@ -1022,22 +391,3 @@ func randBlockID() BlockID {
},
}
}
// 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: tmproto.PrecommitType, Height: height, Round: 1, BlockID: BlockID{},
Timestamp: time, ValidatorAddress: pKey.Address(), ValidatorIndex: 1, Signature: []byte{}}
v := vote.ToProto()
if err := voteVal.SignVote("mock-chain-id", v); err != nil {
panic(err)
}
vote.Signature = v.Signature
return ProofOfLockChange{
Votes: []*Vote{&vote},
PubKey: pubKey,
}
}

+ 0
- 304
types/evidence_test.go View File

@ -9,7 +9,6 @@ import (
"github.com/stretchr/testify/require"
"github.com/tendermint/tendermint/crypto"
"github.com/tendermint/tendermint/crypto/ed25519"
"github.com/tendermint/tendermint/crypto/tmhash"
tmrand "github.com/tendermint/tendermint/libs/rand"
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
@ -178,256 +177,6 @@ func TestMockEvidenceValidateBasic(t *testing.T) {
assert.Nil(t, goodEvidence.ValidateBasic())
}
func TestPotentialAmnesiaEvidence(t *testing.T) {
const (
chainID = "TestPotentialAmnesiaEvidence"
height int64 = 37
)
var (
val = NewMockPV()
val2 = NewMockPV()
blockID = makeBlockID(tmhash.Sum([]byte("blockhash")), math.MaxInt32, tmhash.Sum([]byte("partshash")))
blockID2 = makeBlockID(tmhash.Sum([]byte("blockhash2")), math.MaxInt32, tmhash.Sum([]byte("partshash")))
vote1 = makeVote(t, val, chainID, 0, height, 0, 2, blockID, defaultVoteTime)
vote2 = makeVote(t, val, chainID, 0, height, 1, 2, blockID2, defaultVoteTime.Add(1*time.Second))
vote3 = makeVote(t, val, chainID, 0, height, 2, 2, blockID, defaultVoteTime)
)
ev := NewPotentialAmnesiaEvidence(vote1, vote2, vote1.Timestamp)
assert.Equal(t, height, ev.Height())
assert.Equal(t, vote1.Timestamp, ev.Time())
assert.EqualValues(t, vote1.ValidatorAddress, ev.Address())
assert.NotEmpty(t, ev.Hash())
assert.NotEmpty(t, ev.Bytes())
pubKey, err := val.GetPubKey()
require.NoError(t, err)
assert.NoError(t, ev.Verify(chainID, pubKey))
assert.Error(t, ev.Verify("other", pubKey))
privKey2 := ed25519.GenPrivKey()
pubKey2 := privKey2.PubKey()
assert.Error(t, ev.Verify("other", pubKey2))
assert.True(t, ev.Equal(ev))
assert.NoError(t, ev.ValidateBasic())
assert.NotEmpty(t, ev.String())
ev2 := &PotentialAmnesiaEvidence{
VoteA: vote1,
VoteB: vote2,
HeightStamp: 5,
}
assert.True(t, ev.Equal(ev2))
assert.Equal(t, ev.Hash(), ev2.Hash())
ev3 := NewPotentialAmnesiaEvidence(vote2, vote1, vote1.Timestamp)
assert.True(t, ev3.Equal(ev))
ev4 := &PotentialAmnesiaEvidence{
VoteA: vote3,
VoteB: vote2,
}
assert.NoError(t, ev4.ValidateBasic())
assert.NotEqual(t, ev.Hash(), ev4.Hash())
assert.False(t, ev.Equal(ev4))
// bad evidence
badEv := []*PotentialAmnesiaEvidence{
// first vote is for a later time than the second vote
{
VoteA: vote2,
VoteB: vote1,
},
// votes are for the same round
{
VoteA: vote1,
VoteB: makeVote(t, val, chainID, 0, height, 0, 2, blockID2, defaultVoteTime.Add(1*time.Second)),
},
// first vote was for a nil block - not locked
{
VoteA: makeVote(t, val, chainID, 0, height, 0, 2, BlockID{}, defaultVoteTime.Add(1*time.Second)),
VoteB: vote2,
},
// second vote is from a different validator
{
VoteA: vote1,
VoteB: makeVote(t, val2, chainID, 0, height, 1, 2, blockID2, defaultVoteTime.Add(1*time.Second)),
},
}
for _, ev := range badEv {
assert.Error(t, ev.ValidateBasic())
}
}
func TestProofOfLockChange(t *testing.T) {
const (
chainID = "test_chain_id"
height int64 = 37
)
// 1: valid POLC - nothing should fail
voteSet, valSet, privValidators, blockID := buildVoteSet(height, 1, 3, 7, 0, tmproto.PrecommitType)
pubKey, err := privValidators[7].GetPubKey()
require.NoError(t, err)
polc, err := NewPOLCFromVoteSet(voteSet, pubKey, blockID)
assert.NoError(t, err)
assert.Equal(t, height, polc.Height())
assert.NoError(t, polc.ValidateBasic())
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
pubKey, err = privValidators[0].GetPubKey()
require.NoError(t, err)
polc2 := newPOLCFromVoteSet(voteSet, pubKey, blockID)
badPOLCs = append(badPOLCs, polc2)
// 3: one vote was from a different round
voteSet, _, privValidators, blockID = buildVoteSet(height, 1, 3, 7, 0, tmproto.PrecommitType)
pubKey, err = privValidators[7].GetPubKey()
require.NoError(t, err)
polc = newPOLCFromVoteSet(voteSet, pubKey, blockID)
badVote := makeVote(t, privValidators[8], chainID, 8, height, 2, 2, blockID, defaultVoteTime)
polc.Votes = append(polc.Votes, badVote)
badPOLCs = append(badPOLCs, polc)
// 4: one vote was from a different height
polc = newPOLCFromVoteSet(voteSet, pubKey, blockID)
badVote = makeVote(t, privValidators[8], chainID, 8, height+1, 1, 2, blockID, defaultVoteTime)
polc.Votes = append(polc.Votes, badVote)
badPOLCs = append(badPOLCs, polc)
// 5: one vote was from a different vote type
polc = newPOLCFromVoteSet(voteSet, pubKey, blockID)
badVote = makeVote(t, privValidators[8], chainID, 8, height, 1, 1, blockID, defaultVoteTime)
polc.Votes = append(polc.Votes, badVote)
badPOLCs = append(badPOLCs, polc)
// 5: one of the votes was for a nil block
polc = newPOLCFromVoteSet(voteSet, pubKey, blockID)
badVote = makeVote(t, privValidators[8], chainID, 8, height, 1, 2, BlockID{}, defaultVoteTime)
polc.Votes = append(polc.Votes, badVote)
badPOLCs = append(badPOLCs, polc)
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)
}
}
}
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 = newPOLCFromVoteSet(voteSet, pubKey, blockID)
)
require.False(t, polc.IsAbsent())
pe := &PotentialAmnesiaEvidence{
VoteA: vote1,
VoteB: vote2,
}
emptyAmnesiaEvidence := NewAmnesiaEvidence(pe, NewEmptyPOLC())
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 := NewAmnesiaEvidence(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.Equal(t, completeAmnesiaEvidence.Hash(), emptyAmnesiaEvidence.Hash())
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 := NewAmnesiaEvidence(pe2, NewEmptyPOLC())
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 = newPOLCFromVoteSet(voteSet, pubKey, blockID)
badAE = append(badAE, NewAmnesiaEvidence(pe, polc))
// 2) Polc is of a later round
voteSet, _, _ = buildVoteSetForBlock(height, 2, 2, 7, 0, tmproto.PrecommitType, blockID)
polc = newPOLCFromVoteSet(voteSet, pubKey, blockID)
badAE = append(badAE, NewAmnesiaEvidence(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 = newPOLCFromVoteSet(voteSet, pubKey2, blockID)
badAE = append(badAE, NewAmnesiaEvidence(pe, polc))
// 4) Polc has a different block ID
voteSet, _, _, blockID = buildVoteSet(height, 1, 2, 7, 0, tmproto.PrecommitType)
polc = newPOLCFromVoteSet(voteSet, pubKey, blockID)
badAE = append(badAE, NewAmnesiaEvidence(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 {
@ -506,17 +255,6 @@ func TestEvidenceProto(t *testing.T) {
{"DuplicateVoteEvidence nil voteB", &DuplicateVoteEvidence{VoteA: v, VoteB: nil}, false, true},
{"DuplicateVoteEvidence nil voteA", &DuplicateVoteEvidence{VoteA: nil, VoteB: v}, false, true},
{"DuplicateVoteEvidence success", &DuplicateVoteEvidence{VoteA: v2, VoteB: v}, false, false},
{"PotentialAmnesiaEvidence empty fail", &PotentialAmnesiaEvidence{}, false, true},
{"PotentialAmnesiaEvidence nil VoteB", &PotentialAmnesiaEvidence{VoteA: v, VoteB: nil}, false, true},
{"PotentialAmnesiaEvidence nil VoteA", &PotentialAmnesiaEvidence{VoteA: nil, VoteB: v2}, false, true},
{"PotentialAmnesiaEvidence success", &PotentialAmnesiaEvidence{VoteA: v2, VoteB: v}, false, false},
{"AmnesiaEvidence nil ProofOfLockChange", &AmnesiaEvidence{PotentialAmnesiaEvidence: &PotentialAmnesiaEvidence{},
Polc: NewEmptyPOLC()}, false, true},
{"AmnesiaEvidence nil Polc",
&AmnesiaEvidence{PotentialAmnesiaEvidence: &PotentialAmnesiaEvidence{VoteA: v2, VoteB: v},
Polc: &ProofOfLockChange{}}, false, false},
{"AmnesiaEvidence success", &AmnesiaEvidence{PotentialAmnesiaEvidence: &PotentialAmnesiaEvidence{VoteA: v2, VoteB: v},
Polc: NewEmptyPOLC()}, false, false},
}
for _, tt := range tests {
tt := tt
@ -537,45 +275,3 @@ func TestEvidenceProto(t *testing.T) {
})
}
}
func TestProofOfLockChangeProtoBuf(t *testing.T) {
// -------- Votes --------
val := NewMockPV()
val2 := NewMockPV()
val3 := NewMockPV()
blockID := makeBlockID(tmhash.Sum([]byte("blockhash")), math.MaxInt32, tmhash.Sum([]byte("partshash")))
const chainID = "mychain"
v := makeVote(t, val, chainID, math.MaxInt32, math.MaxInt64, 1, 0x01, blockID, defaultVoteTime)
v2 := makeVote(t, val2, chainID, math.MaxInt32, math.MaxInt64, 1, 0x01, blockID, defaultVoteTime)
testCases := []struct {
msg string
polc *ProofOfLockChange
toProtoErr bool
fromProtoErr bool
}{
{"failure, empty key", &ProofOfLockChange{Votes: []*Vote{v, v2}, PubKey: nil}, true, false},
{"failure, empty votes", &ProofOfLockChange{PubKey: val3.PrivKey.PubKey()}, true, false},
{"success empty ProofOfLockChange", NewEmptyPOLC(), 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.toProtoErr {
assert.Error(t, err, tc.msg)
} else {
assert.NoError(t, err, tc.msg)
}
c, err := ProofOfLockChangeFromProto(pbpolc)
if !tc.fromProtoErr {
assert.NoError(t, err, tc.msg)
if !tc.toProtoErr {
assert.Equal(t, tc.polc, c, tc.msg)
}
} else {
assert.Error(t, err, tc.msg)
}
}
}

+ 3
- 14
types/params.go View File

@ -46,10 +46,9 @@ func DefaultBlockParams() tmproto.BlockParams {
// 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,
ProofTrialPeriod: 50000, // half MaxAgeNumBlocks
MaxAgeNumBlocks: 100000, // 27.8 hrs at 1block/s
MaxAgeDuration: 48 * time.Hour,
MaxNum: 50,
}
}
@ -118,16 +117,6 @@ 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")
}


+ 3
- 4
types/params_test.go View File

@ -64,10 +64,9 @@ func makeParams(
TimeIotaMs: blockTimeIotaMs,
},
Evidence: tmproto.EvidenceParams{
MaxAgeNumBlocks: evidenceAge,
MaxAgeDuration: time.Duration(evidenceAge),
MaxNum: maxEvidence,
ProofTrialPeriod: 1,
MaxAgeNumBlocks: evidenceAge,
MaxAgeDuration: time.Duration(evidenceAge),
MaxNum: maxEvidence,
},
Validator: tmproto.ValidatorParams{
PubKeyTypes: pubkeyTypes,


+ 0
- 3
types/protobuf.go View File

@ -16,7 +16,6 @@ import (
const (
ABCIEvidenceTypeDuplicateVote = "duplicate/vote"
ABCIEvidenceTypeAmnesia = "amnesia"
)
const (
@ -129,8 +128,6 @@ func (tm2pb) Evidence(ev Evidence, valSet *ValidatorSet) abci.Evidence {
switch ev.(type) {
case *DuplicateVoteEvidence:
evType = ABCIEvidenceTypeDuplicateVote
case *AmnesiaEvidence:
evType = ABCIEvidenceTypeAmnesia
default:
panic(fmt.Sprintf("unknown evidence type: %v %v", ev, reflect.TypeOf(ev)))
}


+ 0
- 40
types/vote_set_test.go View File

@ -471,46 +471,6 @@ func TestVoteSet_MakeCommit(t *testing.T) {
}
}
func buildVoteSet(
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, blockID BlockID) (*VoteSet, *ValidatorSet, []PrivValidator) {
valSize := nonVotes + nilVotes + nonNilVotes
voteSet, valSet, privValidators := randVoteSet(height, round, voteType, valSize, 1)
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, int32(i))
_, _ = signAddVote(privValidators[i], vote, voteSet)
}
for i := nonNilVotes; i < nonNilVotes+nilVotes; i++ {
pubKey, _ := privValidators[i].GetPubKey()
addr := pubKey.Address()
vote := withValidator(voteProto, addr, int32(i))
_, _ = signAddVote(privValidators[i], withBlockHash(vote, nil), voteSet)
}
return voteSet, valSet, privValidators
}
// NOTE: privValidators are in order
func randVoteSet(
height int64,


Loading…
Cancel
Save