Browse Source

evidence: modularise evidence by moving verification function into evidence package (#5234)

pull/5275/head
Callum Waters 4 years ago
committed by GitHub
parent
commit
b7f6e47a42
No known key found for this signature in database GPG Key ID: 4AEE18F83AFDEB23
19 changed files with 546 additions and 717 deletions
  1. +1
    -1
      consensus/byzantine_test.go
  2. +1
    -1
      consensus/common_test.go
  3. +1
    -12
      consensus/reactor_test.go
  4. +1
    -0
      consensus/replay_stubs.go
  5. +0
    -1
      evidence/mocks/block_store.go
  6. +42
    -35
      evidence/pool.go
  7. +144
    -121
      evidence/pool_test.go
  8. +5
    -5
      evidence/reactor_test.go
  9. +28
    -0
      evidence/services.go
  10. +96
    -0
      evidence/verify.go
  11. +180
    -0
      evidence/verify_test.go
  12. +1
    -1
      light/store/store.go
  13. +1
    -1
      node/node.go
  14. +1
    -1
      node/node_test.go
  15. +14
    -44
      state/mocks/evidence_pool.go
  16. +3
    -7
      state/services.go
  17. +7
    -104
      state/validation.go
  18. +1
    -364
      state/validation_test.go
  19. +19
    -19
      types/evidence.go

+ 1
- 1
consensus/byzantine_test.go View File

@ -70,7 +70,7 @@ func TestByzantinePrevoteEquivocation(t *testing.T) {
// Make a full instance of the evidence pool
evidenceDB := dbm.NewMemDB()
evpool, err := evidence.NewPool(stateDB, evidenceDB, blockStore)
evpool, err := evidence.NewPool(evidenceDB, evidence.NewEvidenceStateStore(stateDB), blockStore)
require.NoError(t, err)
evpool.SetLogger(logger.With("module", "evidence"))


+ 1
- 1
consensus/common_test.go View File

@ -452,7 +452,7 @@ func randStateWithEvpool(t *testing.T, nValidators int) (*State, []*validatorStu
}
stateDB := dbm.NewMemDB()
sm.SaveState(stateDB, state)
evpool, err := evidence.NewPool(stateDB, evidenceDB, blockStore)
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)


+ 1
- 12
consensus/reactor_test.go View File

@ -224,19 +224,8 @@ func (m *mockEvidencePool) Update(block *types.Block, state sm.State) {
}
m.height++
}
func (m *mockEvidencePool) IsCommitted(types.Evidence) bool { return false }
func (m *mockEvidencePool) IsPending(evidence types.Evidence) bool {
if m.height > 0 {
for _, e := range m.ev {
if e.Equal(evidence) {
return true
}
}
}
return false
}
func (m *mockEvidencePool) Verify(types.Evidence) error { return nil }
func (m *mockEvidencePool) AddPOLC(*types.ProofOfLockChange) error { return nil }
func (m *mockEvidencePool) Header(int64) *types.Header { return &types.Header{Time: defaultTestTime} }
//------------------------------------


+ 1
- 0
consensus/replay_stubs.go View File

@ -54,6 +54,7 @@ var _ sm.EvidencePool = emptyEvidencePool{}
func (emptyEvidencePool) PendingEvidence(uint32) []types.Evidence { return nil }
func (emptyEvidencePool) AddEvidence(types.Evidence) error { return nil }
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 }


+ 0
- 1
evidence/mocks/block_store.go View File

@ -4,7 +4,6 @@ package mocks
import (
mock "github.com/stretchr/testify/mock"
types "github.com/tendermint/tendermint/types"
)


+ 42
- 35
evidence/pool.go View File

@ -1,6 +1,7 @@
package evidence
import (
"errors"
"fmt"
"sync"
"time"
@ -31,7 +32,7 @@ type Pool struct {
evidenceList *clist.CList // concurrent linked-list of evidence
// needed to load validators to verify evidence
stateDB dbm.DB
stateDB StateStore
// needed to load headers to verify evidence
blockStore BlockStore
@ -45,11 +46,11 @@ type Pool struct {
nextEvidenceTrialEndedHeight int64
}
// Creates a new pool. If using an existing evidence store, it will add all pending evidence
// to the concurrent list.
func NewPool(stateDB, evidenceDB dbm.DB, blockStore BlockStore) (*Pool, error) {
// NewPool creates an evidence pool. If using an existing evidence store,
// it will add all pending evidence to the concurrent list.
func NewPool(evidenceDB dbm.DB, stateDB StateStore, blockStore BlockStore) (*Pool, error) {
var (
state = sm.LoadState(stateDB)
state = stateDB.LoadState()
)
pool := &Pool{
@ -145,14 +146,11 @@ func (evpool *Pool) AddPOLC(polc *types.ProofOfLockChange) error {
// evidence is composite (ConflictingHeadersEvidence), it will be broken up
// into smaller pieces.
func (evpool *Pool) AddEvidence(evidence types.Evidence) error {
var (
state = evpool.State()
evList = []types.Evidence{evidence}
)
var evList = []types.Evidence{evidence}
evpool.logger.Debug("Attempting to add evidence", "ev", evidence)
valSet, err := sm.LoadValidators(evpool.stateDB, evidence.Height())
valSet, err := evpool.stateDB.LoadValidators(evidence.Height())
if err != nil {
return fmt.Errorf("can't load validators at height #%d: %w", evidence.Height(), err)
}
@ -177,36 +175,14 @@ func (evpool *Pool) AddEvidence(evidence types.Evidence) error {
if evpool.Has(ev) {
// if it is an amnesia evidence we have but POLC is not absent then
// we should still process it
// we should still process it else we loop to the next piece of evidence
if ae, ok := ev.(*types.AmnesiaEvidence); !ok || ae.Polc.IsAbsent() {
continue
}
}
// A header needs to be fetched. For lunatic evidence this is so we can verify
// that some of the fields are different to the ones we have. For all evidence it
// it so we can verify that the time of the evidence is correct
var header *types.Header
// if the evidence is from the current height - this means the evidence is fresh from the consensus
// and we won't have it in the block store. We thus check that the time isn't before the previous block
if ev.Height() == evpool.State().LastBlockHeight+1 {
if ev.Time().Before(evpool.State().LastBlockTime) {
return fmt.Errorf("evidence is from an earlier time than the previous block: %v < %v",
ev.Time(),
evpool.State().LastBlockTime)
}
header = &types.Header{Time: ev.Time()}
} else { // if the evidence is from a prior height
header = evpool.Header(ev.Height())
if header == nil {
return fmt.Errorf("don't have header at height #%d", ev.Height())
}
}
// 1) Verify against state.
if err := sm.VerifyEvidence(evpool.stateDB, state, ev, header); err != nil {
evpool.logger.Debug("Inbound evidence is invalid", "evidence", ev, "err", err)
if err := evpool.verify(ev); err != nil {
return types.NewErrEvidenceInvalid(ev, err)
}
@ -256,6 +232,37 @@ func (evpool *Pool) AddEvidence(evidence types.Evidence) error {
return nil
}
// Verify verifies the evidence against the node's (or evidence pool's) state. More specifically, to validate
// evidence against state is to validate it against the nodes own header and validator set for that height. This ensures
// as well as meeting the evidence's own validation rules, that the evidence hasn't expired, that the validator is still
// bonded and that the evidence can be committed to the chain.
func (evpool *Pool) Verify(evidence types.Evidence) error {
if evpool.IsCommitted(evidence) {
return errors.New("evidence was already committed")
}
// We have already verified this piece of evidence - no need to do it again
if evpool.IsPending(evidence) {
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)
}
func (evpool *Pool) verify(evidence types.Evidence) error {
return VerifyEvidence(evidence, evpool.State(), evpool.stateDB, evpool.blockStore)
}
// MarkEvidenceAsCommitted marks all the evidence as committed and removes it
// from the queue.
func (evpool *Pool) MarkEvidenceAsCommitted(height int64, evidence []types.Evidence) {
@ -543,7 +550,7 @@ func (evpool *Pool) pruneExpiredPOLC() {
evpool.logger.Error("Unable to transition POLC from protobuf", "err", err)
continue
}
if !evpool.IsExpired(proof.Height()-1, proof.Time()) {
if !evpool.IsExpired(proof.Height(), proof.Time()) {
return
}
err = evpool.evidenceStore.Delete(iter.Key())


+ 144
- 121
evidence/pool_test.go View File

@ -21,7 +21,6 @@ import (
sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/store"
"github.com/tendermint/tendermint/types"
tmtime "github.com/tendermint/tendermint/types/time"
)
func TestMain(m *testing.M) {
@ -32,33 +31,27 @@ func TestMain(m *testing.M) {
const evidenceChainID = "test_chain"
func TestEvidencePool(t *testing.T) {
var (
val = types.NewMockPV()
height = int64(52)
stateDB = initializeValidatorState(val, height)
evidenceDB = dbm.NewMemDB()
blockStore = &mocks.BlockStore{}
evidenceTime = time.Date(2019, 1, 1, 0, 0, 0, 0, time.UTC)
var defaultEvidenceTime = time.Date(2019, 1, 1, 0, 0, 0, 0, time.UTC)
goodEvidence = types.NewMockDuplicateVoteEvidenceWithValidator(height, evidenceTime, val, evidenceChainID)
badEvidence = types.NewMockDuplicateVoteEvidenceWithValidator(1, evidenceTime, val, evidenceChainID)
func TestEvidencePoolBasic(t *testing.T) {
var (
val = types.NewMockPV()
height = int64(1)
stateStore = initializeValidatorState(val, height)
evidenceDB = dbm.NewMemDB()
blockStore = &mocks.BlockStore{}
)
blockStore.On("LoadBlockMeta", mock.AnythingOfType("int64")).Return(
&types.BlockMeta{Header: types.Header{Time: evidenceTime}},
&types.BlockMeta{Header: types.Header{Time: defaultEvidenceTime}},
)
pool, err := NewPool(stateDB, evidenceDB, blockStore)
pool, err := NewPool(evidenceDB, stateStore, blockStore)
require.NoError(t, err)
// bad evidence
err = pool.AddEvidence(badEvidence)
if assert.Error(t, err) {
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.IsEvidenceExpired(badEvidence))
// evidence not seen yet:
evidence := types.NewMockDuplicateVoteEvidenceWithValidator(height, defaultEvidenceTime, val, evidenceChainID)
assert.False(t, pool.IsCommitted(evidence))
// good evidence
evAdded := make(chan struct{})
@ -67,8 +60,8 @@ func TestEvidencePool(t *testing.T) {
close(evAdded)
}()
err = pool.AddEvidence(goodEvidence)
require.NoError(t, err)
// evidence seen but not yet committed:
assert.NoError(t, pool.AddEvidence(evidence))
select {
case <-evAdded:
@ -78,63 +71,47 @@ func TestEvidencePool(t *testing.T) {
assert.Equal(t, 1, pool.evidenceList.Len())
// if we send it again, it shouldnt add and return an error
err = pool.AddEvidence(goodEvidence)
assert.NoError(t, err)
assert.Equal(t, 1, pool.evidenceList.Len())
}
func TestProposingAndCommittingEvidence(t *testing.T) {
var (
val = types.NewMockPV()
height = int64(1)
stateDB = initializeValidatorState(val, height)
evidenceDB = dbm.NewMemDB()
blockStore = &mocks.BlockStore{}
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}},
)
pool, err := NewPool(stateDB, evidenceDB, blockStore)
require.NoError(t, err)
// evidence not seen yet:
evidence := types.NewMockDuplicateVoteEvidenceWithValidator(height, evidenceTime, val, evidenceChainID)
assert.False(t, pool.IsCommitted(evidence))
// evidence seen but not yet committed:
assert.NoError(t, pool.AddEvidence(evidence))
assert.False(t, pool.IsCommitted(evidence))
assert.True(t, pool.IsPending(evidence))
// test evidence is proposed
proposedEvidence := pool.AllPendingEvidence()
assert.Equal(t, proposedEvidence[0], evidence)
proposedEvidence = pool.PendingEvidence(1)
assert.Equal(t, proposedEvidence[0], evidence)
// evidence seen and committed:
pool.MarkEvidenceAsCommitted(height, proposedEvidence)
assert.True(t, pool.IsCommitted(evidence))
assert.False(t, pool.IsPending(evidence))
assert.Equal(t, 0, pool.evidenceList.Len())
// evidence should
// no evidence should be pending
proposedEvidence = pool.PendingEvidence(1)
assert.Empty(t, proposedEvidence)
}
func TestAddEvidence(t *testing.T) {
// Tests inbound evidence for the right time and height
func TestAddExpiredEvidence(t *testing.T) {
var (
val = types.NewMockPV()
valAddr = val.PrivKey.PubKey().Address()
height = int64(30)
stateDB = initializeValidatorState(val, height)
evidenceDB = dbm.NewMemDB()
blockStoreDB = dbm.NewMemDB()
blockStore = initializeBlockStore(blockStoreDB, sm.LoadState(stateDB), valAddr)
evidenceTime = time.Date(2019, 1, 1, 0, 0, 0, 0, time.UTC)
val = types.NewMockPV()
height = int64(30)
stateStore = initializeValidatorState(val, height)
evidenceDB = dbm.NewMemDB()
blockStore = &mocks.BlockStore{}
expiredEvidenceTime = time.Date(2018, 1, 1, 0, 0, 0, 0, time.UTC)
expiredHeight = int64(2)
)
pool, err := NewPool(stateDB, evidenceDB, blockStore)
blockStore.On("LoadBlockMeta", mock.AnythingOfType("int64")).Return(func(h int64) *types.BlockMeta {
if h == height || h == expiredHeight {
return &types.BlockMeta{Header: types.Header{Time: defaultEvidenceTime}}
}
return &types.BlockMeta{Header: types.Header{Time: expiredEvidenceTime}}
})
pool, err := NewPool(evidenceDB, stateStore, blockStore)
require.NoError(t, err)
testCases := []struct {
@ -143,44 +120,37 @@ func TestAddEvidence(t *testing.T) {
expErr bool
evDescription string
}{
{height, time.Now(), false, "valid evidence"},
{height, evidenceTime, false, "valid evidence (despite old time)"},
{int64(1), time.Now(), false, "valid evidence (despite old height)"},
{int64(1), evidenceTime, true,
{height, defaultEvidenceTime, false, "valid evidence"},
{expiredHeight, defaultEvidenceTime, false, "valid evidence (despite old height)"},
{height - 1, expiredEvidenceTime, false, "valid evidence (despite old time)"},
{expiredHeight - 1, expiredEvidenceTime, true,
"evidence from height 1 (created at: 2019-01-01 00:00:00 +0000 UTC) is too old"},
}
for _, tc := range testCases {
tc := tc
t.Run(tc.evDescription, func(t *testing.T) {
ev := types.NewMockDuplicateVoteEvidence(tc.evHeight, tc.evTime, evidenceChainID)
ev := types.NewMockDuplicateVoteEvidenceWithValidator(tc.evHeight, tc.evTime, val, evidenceChainID)
err := pool.AddEvidence(ev)
if tc.expErr {
assert.Error(t, err)
t.Log(err)
} else {
assert.NoError(t, err)
}
})
}
}
func TestEvidencePoolUpdate(t *testing.T) {
var (
val = types.NewMockPV()
valAddr = val.PrivKey.PubKey().Address()
height = int64(21)
stateDB = initializeValidatorState(val, height)
evidenceDB = dbm.NewMemDB()
blockStoreDB = dbm.NewMemDB()
state = sm.LoadState(stateDB)
blockStore = initializeBlockStore(blockStoreDB, state, valAddr)
)
height := int64(21)
pool, err := NewPool(stateDB, evidenceDB, blockStore)
require.NoError(t, err)
pool, val := defaultTestPool(height)
state := pool.State()
// create new block (no need to save it to blockStore)
evidence := types.NewMockDuplicateVoteEvidence(height, time.Now(), evidenceChainID)
lastCommit := makeCommit(height, valAddr)
lastCommit := makeCommit(height, val.PrivKey.PubKey().Address())
block := types.MakeBlock(height+1, []types.Tx{}, lastCommit, []types.Evidence{evidence})
// update state (partially)
state.LastBlockHeight = height + 1
@ -193,27 +163,33 @@ func TestEvidencePoolUpdate(t *testing.T) {
func TestAddingAndPruningPOLC(t *testing.T) {
var (
val = types.NewMockPV()
valAddr = val.PrivKey.PubKey().Address()
stateDB = initializeValidatorState(val, 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)
firstBlockID = types.BlockID{
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, evidenceTime)
voteA := makeVote(1, 1, 0, val.PrivKey.PubKey().Address(), firstBlockID, expiredEvidenceTime)
vA := voteA.ToProto()
err := val.SignVote(evidenceChainID, vA)
err = val.SignVote(evidenceChainID, vA)
require.NoError(t, err)
voteA.Signature = vA.Signature
@ -223,9 +199,6 @@ func TestAddingAndPruningPOLC(t *testing.T) {
PubKey: pubKey,
}
pool, err := NewPool(stateDB, evidenceDB, blockStore)
require.NoError(t, err)
err = pool.AddPOLC(polc)
assert.NoError(t, err)
@ -239,7 +212,7 @@ func TestAddingAndPruningPOLC(t *testing.T) {
assert.NoError(t, err)
assert.Nil(t, emptyPolc)
lastCommit := makeCommit(height-1, valAddr)
lastCommit := makeCommit(height-1, val.PrivKey.PubKey().Address())
block := types.MakeBlock(height, []types.Tx{}, lastCommit, []types.Evidence{})
// update state (partially)
state.LastBlockHeight = height
@ -254,19 +227,55 @@ func TestAddingAndPruningPOLC(t *testing.T) {
}
func TestVerifyEvidenceCommittedEvidenceFails(t *testing.T) {
height := int64(1)
pool, _ := defaultTestPool(height)
committedEvidence := types.NewMockDuplicateVoteEvidence(height, time.Now(), evidenceChainID)
pool.MarkEvidenceAsCommitted(height, []types.Evidence{committedEvidence})
err := pool.Verify(committedEvidence)
if assert.Error(t, err) {
assert.Equal(t, "evidence was already committed", err.Error())
}
}
func TestVeriyEvidencePendingEvidencePasses(t *testing.T) {
var (
val = types.NewMockPV()
height = int64(1)
stateStore = initializeValidatorState(val, height)
blockStore = &mocks.BlockStore{}
)
blockStore.On("LoadBlockMeta", mock.AnythingOfType("int64")).Return(
&types.BlockMeta{Header: types.Header{Time: defaultEvidenceTime}},
)
pool, err := NewPool(dbm.NewMemDB(), stateStore, blockStore)
require.NoError(t, err)
evidence := types.NewMockDuplicateVoteEvidenceWithValidator(height, defaultEvidenceTime, val, evidenceChainID)
err = pool.AddEvidence(evidence)
require.NoError(t, err)
err = pool.Verify(evidence)
assert.NoError(t, err)
}
func TestRecoverPendingEvidence(t *testing.T) {
var (
val = types.NewMockPV()
valAddr = val.PrivKey.PubKey().Address()
height = int64(30)
stateDB = initializeValidatorState(val, 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)
goodEvidence = types.NewMockDuplicateVoteEvidenceWithValidator(height, time.Now(), val, evidenceChainID)
expiredEvidence = types.NewMockDuplicateVoteEvidenceWithValidator(int64(1), evidenceTime, val, evidenceChainID)
val = types.NewMockPV()
valAddr = val.PrivKey.PubKey().Address()
height = int64(30)
stateStore = initializeValidatorState(val, height)
evidenceDB = dbm.NewMemDB()
blockStoreDB = dbm.NewMemDB()
state = stateStore.LoadState()
blockStore = initializeBlockStore(blockStoreDB, state, valAddr)
expiredEvidenceTime = time.Date(2018, 1, 1, 0, 0, 0, 0, time.UTC)
goodEvidence = types.NewMockDuplicateVoteEvidenceWithValidator(height,
defaultEvidenceTime, val, evidenceChainID)
expiredEvidence = types.NewMockDuplicateVoteEvidenceWithValidator(int64(1),
expiredEvidenceTime, val, evidenceChainID)
)
// load good evidence
@ -286,15 +295,16 @@ func TestRecoverPendingEvidence(t *testing.T) {
require.NoError(t, err)
_ = evidenceDB.Set(expiredKey, expiredEvidenceBytes)
pool, err := NewPool(stateDB, evidenceDB, blockStore)
pool, err := NewPool(evidenceDB, stateStore, blockStore)
require.NoError(t, err)
assert.Equal(t, 1, pool.evidenceList.Len())
assert.True(t, pool.IsPending(goodEvidence))
assert.False(t, pool.Has(expiredEvidence))
}
// Comprehensive set of test cases relating to the adding, upgrading and overall
// processing of PotentialAmnesiaEvidence and AmnesiaEvidence
func TestAddingPotentialAmnesiaEvidence(t *testing.T) {
func TestAmnesiaEvidence(t *testing.T) {
var (
val = types.NewMockPV()
val2 = types.NewMockPV()
@ -308,23 +318,23 @@ func TestAddingPotentialAmnesiaEvidence(t *testing.T) {
Proposer: val.ExtractIntoValidator(1),
}
height = int64(30)
stateDB = initializeStateFromValidatorSet(valSet, height)
stateStore = initializeStateFromValidatorSet(valSet, height)
evidenceDB = dbm.NewMemDB()
state = sm.LoadState(stateDB)
state = stateStore.LoadState()
blockStore = &mocks.BlockStore{}
//evidenceTime = time.Date(2019, 1, 1, 0, 0, 0, 0, time.UTC)
firstBlockID = types.BlockID{
Hash: []byte("aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa"),
Hash: tmrand.Bytes(tmhash.Size),
PartSetHeader: types.PartSetHeader{
Total: 1,
Hash: []byte("aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa"),
Hash: tmrand.Bytes(tmhash.Size),
},
}
secondBlockID = types.BlockID{
Hash: []byte("bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb"),
Hash: tmrand.Bytes(tmhash.Size),
PartSetHeader: types.PartSetHeader{
Total: 1,
Hash: []byte("bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb"),
Hash: tmrand.Bytes(tmhash.Size),
},
}
evidenceTime = time.Date(2019, 1, 1, 0, 0, 0, 0, time.UTC)
@ -335,7 +345,7 @@ func TestAddingPotentialAmnesiaEvidence(t *testing.T) {
)
// TEST SETUP
pool, err := NewPool(stateDB, evidenceDB, blockStore)
pool, err := NewPool(evidenceDB, stateStore, blockStore)
require.NoError(t, err)
pool.SetLogger(log.TestingLogger())
@ -487,13 +497,13 @@ func TestAddingPotentialAmnesiaEvidence(t *testing.T) {
}
func initializeStateFromValidatorSet(valSet *types.ValidatorSet, height int64) dbm.DB {
func initializeStateFromValidatorSet(valSet *types.ValidatorSet, height int64) StateStore {
stateDB := dbm.NewMemDB()
state := sm.State{
ChainID: evidenceChainID,
InitialHeight: 1,
LastBlockHeight: height,
LastBlockTime: tmtime.Now(),
LastBlockTime: defaultEvidenceTime,
Validators: valSet,
NextValidators: valSet.CopyIncrementProposerPriority(1),
LastValidators: valSet,
@ -518,10 +528,10 @@ func initializeStateFromValidatorSet(valSet *types.ValidatorSet, height int64) d
sm.SaveState(stateDB, state)
}
return stateDB
return &stateStore{db: stateDB}
}
func initializeValidatorState(privVal types.PrivValidator, height int64) dbm.DB {
func initializeValidatorState(privVal types.PrivValidator, height int64) StateStore {
pubKey, _ := privVal.GetPubKey()
validator := &types.Validator{Address: pubKey.Address(), VotingPower: 0, PubKey: pubKey}
@ -577,3 +587,16 @@ func makeVote(height int64, round, index int32, addr bytes.HexBytes,
ValidatorIndex: index,
}
}
func defaultTestPool(height int64) (*Pool, types.MockPV) {
val := types.NewMockPV()
valAddress := val.PrivKey.PubKey().Address()
evidenceDB := dbm.NewMemDB()
stateStore := initializeValidatorState(val, height)
blockStore := initializeBlockStore(dbm.NewMemDB(), stateStore.LoadState(), valAddress)
pool, err := NewPool(evidenceDB, stateStore, blockStore)
if err != nil {
panic("test evidence pool could not be created")
}
return pool, val
}

+ 5
- 5
evidence/reactor_test.go View File

@ -40,8 +40,8 @@ func evidenceLogger() log.Logger {
}
// connect N evidence reactors through N switches
func makeAndConnectReactors(config *cfg.Config, stateDBs []dbm.DB) []*Reactor {
N := len(stateDBs)
func makeAndConnectReactors(config *cfg.Config, stateStores []StateStore) []*Reactor {
N := len(stateStores)
reactors := make([]*Reactor, N)
logger := evidenceLogger()
@ -53,7 +53,7 @@ func makeAndConnectReactors(config *cfg.Config, stateDBs []dbm.DB) []*Reactor {
blockStore.On("LoadBlockMeta", mock.AnythingOfType("int64")).Return(
&types.BlockMeta{Header: types.Header{Time: evidenceTime}},
)
pool, err := NewPool(stateDBs[i], evidenceDB, blockStore)
pool, err := NewPool(evidenceDB, stateStores[i], blockStore)
if err != nil {
panic(err)
}
@ -144,7 +144,7 @@ func TestReactorBroadcastEvidence(t *testing.T) {
N := 7
// create statedb for everyone
stateDBs := make([]dbm.DB, N)
stateDBs := make([]StateStore, N)
val := types.NewMockPV()
// we need validators saved for heights at least as high as we have evidence for
height := int64(numEvidence) + 10
@ -189,7 +189,7 @@ func TestReactorSelectiveBroadcast(t *testing.T) {
stateDB2 := initializeValidatorState(val, height2)
// make reactors from statedb
reactors := makeAndConnectReactors(config, []dbm.DB{stateDB1, stateDB2})
reactors := makeAndConnectReactors(config, []StateStore{stateDB1, stateDB2})
// set the peer height on each reactor
for _, r := range reactors {


+ 28
- 0
evidence/services.go View File

@ -1,6 +1,9 @@
package evidence
import (
dbm "github.com/tendermint/tm-db"
"github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/types"
)
@ -9,3 +12,28 @@ import (
type BlockStore interface {
LoadBlockMeta(height int64) *types.BlockMeta
}
type StateStore interface {
LoadValidators(height int64) (*types.ValidatorSet, error)
LoadState() state.State
}
type stateStore struct {
db dbm.DB
}
var _ StateStore = &stateStore{}
// This is a temporary measure until stateDB becomes a store
// TODO: deprecate once state has a store
func NewEvidenceStateStore(db dbm.DB) StateStore {
return &stateStore{db}
}
func (s *stateStore) LoadValidators(height int64) (*types.ValidatorSet, error) {
return state.LoadValidators(s.db, height)
}
func (s *stateStore) LoadState() state.State {
return state.LoadState(s.db)
}

+ 96
- 0
evidence/verify.go View File

@ -0,0 +1,96 @@
package evidence
import (
"fmt"
sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/types"
)
// VerifyEvidence verifies the evidence fully by checking:
// - it is sufficiently recent (MaxAge)
// - it is from a key who was a validator at the given height
// - it is internally consistent
// - it was properly signed by the alleged equivocator
func VerifyEvidence(evidence types.Evidence, state sm.State, stateDB StateStore, blockStore BlockStore) error {
var (
height = state.LastBlockHeight
evidenceParams = state.ConsensusParams.Evidence
ageDuration = state.LastBlockTime.Sub(evidence.Time())
ageNumBlocks = height - evidence.Height()
header *types.Header
)
// if the evidence is from the current height - this means the evidence is fresh from the consensus
// and we won't have it in the block store. We thus check that the time isn't before the previous block
if evidence.Height() == height+1 {
if evidence.Time().Before(state.LastBlockTime) {
return fmt.Errorf("evidence is from an earlier time than the previous block: %v < %v",
evidence.Time(),
state.LastBlockTime)
}
} else {
// try to retrieve header from blockstore
blockMeta := blockStore.LoadBlockMeta(evidence.Height())
header = &blockMeta.Header
if header == nil {
return fmt.Errorf("don't have header at height #%d", evidence.Height())
}
if header.Time != evidence.Time() {
return fmt.Errorf("evidence time (%v) is different to the time of the header we have for the same height (%v)",
evidence.Time(),
header.Time,
)
}
}
if ageDuration > evidenceParams.MaxAgeDuration && ageNumBlocks > evidenceParams.MaxAgeNumBlocks {
return fmt.Errorf(
"evidence from height %d (created at: %v) is too old; min height is %d and evidence can not be older than %v",
evidence.Height(),
evidence.Time(),
height-evidenceParams.MaxAgeNumBlocks,
state.LastBlockTime.Add(evidenceParams.MaxAgeDuration),
)
}
// If in the case of lunatic validator evidence we need our committed header again to verify the evidence
if ev, ok := evidence.(*types.LunaticValidatorEvidence); ok {
if err := ev.VerifyHeader(header); err != nil {
return err
}
}
valset, err := stateDB.LoadValidators(evidence.Height())
if err != nil {
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
// For all other types, expect evidence.Address to be a validator at height
// evidence.Height.
_, val = valset.GetByAddress(addr)
if val == nil {
return fmt.Errorf("address %X was not a validator at height %d", addr, evidence.Height())
}
if err := evidence.Verify(state.ChainID, val.PubKey); err != nil {
return err
}
return nil
}

+ 180
- 0
evidence/verify_test.go View File

@ -0,0 +1,180 @@
package evidence
import (
"fmt"
"testing"
"time"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
"github.com/tendermint/tendermint/crypto"
"github.com/tendermint/tendermint/crypto/tmhash"
"github.com/tendermint/tendermint/evidence/mocks"
tmrand "github.com/tendermint/tendermint/libs/rand"
"github.com/tendermint/tendermint/types"
)
func TestVerifyEvidenceWrongAddress(t *testing.T) {
var height int64 = 4
val := types.NewMockPV()
stateStore := initializeValidatorState(val, height)
state := stateStore.LoadState()
blockStore := &mocks.BlockStore{}
blockStore.On("LoadBlockMeta", mock.AnythingOfType("int64")).Return(
&types.BlockMeta{Header: types.Header{Time: defaultEvidenceTime}},
)
evidence := types.NewMockDuplicateVoteEvidence(1, defaultEvidenceTime, evidenceChainID)
err := VerifyEvidence(evidence, state, stateStore, blockStore)
errMsg := fmt.Sprintf("address %X was not a validator at height 1", evidence.Address())
if assert.Error(t, err) {
assert.Equal(t, err.Error(), errMsg)
}
}
func TestVerifyEvidenceExpiredEvidence(t *testing.T) {
var height int64 = 4
val := types.NewMockPV()
stateStore := initializeValidatorState(val, height)
state := stateStore.LoadState()
state.ConsensusParams.Evidence.MaxAgeNumBlocks = 1
expiredEvidenceTime := time.Date(2018, 1, 1, 0, 0, 0, 0, time.UTC)
blockStore := &mocks.BlockStore{}
blockStore.On("LoadBlockMeta", mock.AnythingOfType("int64")).Return(
&types.BlockMeta{Header: types.Header{Time: expiredEvidenceTime}},
)
expiredEv := types.NewMockDuplicateVoteEvidenceWithValidator(1, expiredEvidenceTime, val, evidenceChainID)
err := VerifyEvidence(expiredEv, state, stateStore, blockStore)
errMsg := "evidence from height 1 (created at: 2018-01-01 00:00:00 +0000 UTC) is too old"
if assert.Error(t, err) {
assert.Equal(t, err.Error()[:len(errMsg)], errMsg)
}
}
func TestVerifyEvidenceInvalidTime(t *testing.T) {
height := int64(4)
val := types.NewMockPV()
stateStore := initializeValidatorState(val, height)
state := stateStore.LoadState()
blockStore := &mocks.BlockStore{}
blockStore.On("LoadBlockMeta", mock.AnythingOfType("int64")).Return(
&types.BlockMeta{Header: types.Header{Time: defaultEvidenceTime}},
)
differentTime := time.Date(2019, 2, 1, 0, 0, 0, 0, time.UTC)
ev := types.NewMockDuplicateVoteEvidenceWithValidator(height, differentTime, val, evidenceChainID)
err := VerifyEvidence(ev, state, stateStore, blockStore)
errMsg := "evidence time (2019-02-01 00:00:00 +0000 UTC) is different to the time" +
" of the header we have for the same height (2019-01-01 00:00:00 +0000 UTC)"
if assert.Error(t, err) {
assert.Equal(t, errMsg, err.Error())
}
}
func TestVerifyEvidenceWithLunaticValidatorEvidence(t *testing.T) {
var height int64 = 4
val := types.NewMockPV()
stateStore := initializeValidatorState(val, height)
blockID := types.BlockID{
Hash: tmrand.Bytes(tmhash.Size),
PartSetHeader: types.PartSetHeader{
Total: 1,
Hash: tmrand.Bytes(tmhash.Size),
},
}
h := &types.Header{
ChainID: evidenceChainID,
Height: 3,
Time: defaultEvidenceTime,
LastBlockID: blockID,
LastCommitHash: tmhash.Sum([]byte("last_commit_hash")),
DataHash: tmhash.Sum([]byte("data_hash")),
ValidatorsHash: tmhash.Sum([]byte("validators_hash")),
NextValidatorsHash: tmhash.Sum([]byte("next_validators_hash")),
ConsensusHash: tmhash.Sum([]byte("consensus_hash")),
AppHash: tmhash.Sum([]byte("app_hash")),
LastResultsHash: tmhash.Sum([]byte("last_results_hash")),
EvidenceHash: tmhash.Sum([]byte("evidence_hash")),
ProposerAddress: crypto.AddressHash([]byte("proposer_address")),
}
blockStore := &mocks.BlockStore{}
blockStore.On("LoadBlockMeta", mock.AnythingOfType("int64")).Return(
&types.BlockMeta{Header: *h},
)
validH1 := *h
validH1.ValidatorsHash = tmhash.Sum([]byte("different_validators_hash"))
validH2 := validH1
validH2.Time = time.Date(2020, 1, 1, 0, 0, 0, 0, time.UTC)
badH1 := validH1
badH1.ChainID = "different_chain_id"
badH2 := *h
badH2.DataHash = tmhash.Sum([]byte("different_data_hash"))
testCases := []struct {
Header *types.Header
ExpErr bool
ErrMsg string
}{
{
h,
true,
"ValidatorsHash matches committed hash",
},
{
&validH1,
false,
"",
},
{
&validH2,
false,
"",
},
{
&badH1,
true,
"chainID do not match: test_chain vs different_chain_id",
},
{
&badH2,
true,
"ValidatorsHash matches committed hash", // it doesn't recognise that the data hashes are different
},
}
for idx, tc := range testCases {
ev := types.NewLunaticValidatorEvidence(tc.Header,
makeValidVoteForHeader(tc.Header, val), "ValidatorsHash", defaultEvidenceTime)
err := VerifyEvidence(ev, stateStore.LoadState(), stateStore, blockStore)
if tc.ExpErr {
if assert.Error(t, err, fmt.Sprintf("expected an error for case: %d", idx)) {
assert.Equal(t, tc.ErrMsg, err.Error(), fmt.Sprintf("case: %d", idx))
}
} else {
assert.NoError(t, err, fmt.Sprintf("did not expect an error for case: %d", idx))
}
}
}
func makeValidVoteForHeader(header *types.Header, val types.MockPV) *types.Vote {
vote := makeVote(header.Height, 1, 0, val.PrivKey.PubKey().Address(), types.BlockID{
Hash: header.Hash(),
PartSetHeader: types.PartSetHeader{
Total: 100,
Hash: crypto.CRandBytes(tmhash.Size),
},
}, defaultEvidenceTime)
v := vote.ToProto()
err := val.SignVote(evidenceChainID, v)
if err != nil {
panic("verify_test: failed to sign vote for header")
}
vote.Signature = v.Signature
return vote
}

+ 1
- 1
light/store/store.go View File

@ -2,7 +2,7 @@ package store
import "github.com/tendermint/tendermint/types"
// Store is anything that can persistenly store headers.
// Store is anything that can persistently store headers.
type Store interface {
// SaveSignedHeaderAndValidatorSet saves a SignedHeader (h: sh.Height) and a
// ValidatorSet (h: sh.Height).


+ 1
- 1
node/node.go View File

@ -341,7 +341,7 @@ func createEvidenceReactor(config *cfg.Config, dbProvider DBProvider,
return nil, nil, err
}
evidenceLogger := logger.With("module", "evidence")
evidencePool, err := evidence.NewPool(stateDB, evidenceDB, blockStore)
evidencePool, err := evidence.NewPool(evidenceDB, evidence.NewEvidenceStateStore(stateDB), blockStore)
if err != nil {
return nil, nil, err
}


+ 1
- 1
node/node_test.go View File

@ -252,7 +252,7 @@ func TestCreateProposalBlock(t *testing.T) {
// Make EvidencePool
evidenceDB := dbm.NewMemDB()
blockStore := store.NewBlockStore(dbm.NewMemDB())
evidencePool, err := evidence.NewPool(stateDB, evidenceDB, blockStore)
evidencePool, err := evidence.NewPool(evidenceDB, evidence.NewEvidenceStateStore(stateDB), blockStore)
require.NoError(t, err)
evidencePool.SetLogger(logger)


+ 14
- 44
state/mocks/evidence_pool.go View File

@ -1,4 +1,4 @@
// Code generated by mockery v1.0.0. DO NOT EDIT.
// Code generated by mockery v2.1.0. DO NOT EDIT.
package mocks
@ -28,67 +28,37 @@ func (_m *EvidencePool) AddEvidence(_a0 types.Evidence) error {
return r0
}
// Header provides a mock function with given fields: _a0
func (_m *EvidencePool) Header(_a0 int64) *types.Header {
// PendingEvidence provides a mock function with given fields: _a0
func (_m *EvidencePool) PendingEvidence(_a0 uint32) []types.Evidence {
ret := _m.Called(_a0)
var r0 *types.Header
if rf, ok := ret.Get(0).(func(int64) *types.Header); ok {
var r0 []types.Evidence
if rf, ok := ret.Get(0).(func(uint32) []types.Evidence); ok {
r0 = rf(_a0)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*types.Header)
r0 = ret.Get(0).([]types.Evidence)
}
}
return r0
}
// IsCommitted provides a mock function with given fields: _a0
func (_m *EvidencePool) IsCommitted(_a0 types.Evidence) bool {
ret := _m.Called(_a0)
var r0 bool
if rf, ok := ret.Get(0).(func(types.Evidence) bool); ok {
r0 = rf(_a0)
} else {
r0 = ret.Get(0).(bool)
}
return r0
}
// IsPending provides a mock function with given fields: _a0
func (_m *EvidencePool) IsPending(_a0 types.Evidence) bool {
ret := _m.Called(_a0)
var r0 bool
if rf, ok := ret.Get(0).(func(types.Evidence) bool); ok {
r0 = rf(_a0)
} else {
r0 = ret.Get(0).(bool)
}
return r0
// Update provides a mock function with given fields: _a0, _a1
func (_m *EvidencePool) Update(_a0 *types.Block, _a1 state.State) {
_m.Called(_a0, _a1)
}
// PendingEvidence provides a mock function with given fields: _a0
func (_m *EvidencePool) PendingEvidence(_a0 uint32) []types.Evidence {
// Verify provides a mock function with given fields: _a0
func (_m *EvidencePool) Verify(_a0 types.Evidence) error {
ret := _m.Called(_a0)
var r0 []types.Evidence
if rf, ok := ret.Get(0).(func(uint32) []types.Evidence); ok {
var r0 error
if rf, ok := ret.Get(0).(func(types.Evidence) error); ok {
r0 = rf(_a0)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).([]types.Evidence)
}
r0 = ret.Error(0)
}
return r0
}
// Update provides a mock function with given fields: _a0, _a1
func (_m *EvidencePool) Update(_a0 *types.Block, _a1 state.State) {
_m.Called(_a0, _a1)
}

+ 3
- 7
state/services.go View File

@ -35,7 +35,7 @@ type BlockStore interface {
//-----------------------------------------------------------------------------
// evidence pool
//go:generate mockery -case underscore -name EvidencePool
//go:generate mockery --case underscore --name EvidencePool
// EvidencePool defines the EvidencePool interface used by the ConsensusState.
// Get/Set/Commit
@ -43,9 +43,7 @@ type EvidencePool interface {
PendingEvidence(uint32) []types.Evidence
AddEvidence(types.Evidence) error
Update(*types.Block, State)
IsCommitted(types.Evidence) bool
IsPending(types.Evidence) bool
Header(int64) *types.Header
Verify(types.Evidence) error
}
// MockEvidencePool is an empty implementation of EvidencePool, useful for testing.
@ -54,6 +52,4 @@ type MockEvidencePool struct{}
func (me MockEvidencePool) PendingEvidence(uint32) []types.Evidence { return nil }
func (me MockEvidencePool) AddEvidence(types.Evidence) error { return nil }
func (me MockEvidencePool) Update(*types.Block, State) {}
func (me MockEvidencePool) IsCommitted(types.Evidence) bool { return false }
func (me MockEvidencePool) IsPending(types.Evidence) bool { return false }
func (me MockEvidencePool) Header(int64) *types.Header { return nil }
func (me MockEvidencePool) Verify(types.Evidence) error { return nil }

+ 7
- 104
state/validation.go View File

@ -144,50 +144,23 @@ func validateBlock(evidencePool EvidencePool, stateDB dbm.DB, state State, block
block.Height, state.InitialHeight)
}
// Limit the amount of evidence
numEvidence := len(block.Evidence.Evidence)
// MaxNumEvidence is capped at uint16, so conversion is always safe.
if maxEvidence := int(state.ConsensusParams.Evidence.MaxNum); numEvidence > maxEvidence {
return types.NewErrEvidenceOverflow(maxEvidence, numEvidence)
// Check evidence doesn't exceed the limit. MaxNumEvidence is capped at uint16, so conversion is always safe.
if max, got := int(state.ConsensusParams.Evidence.MaxNum), len(block.Evidence.Evidence); got > max {
return types.NewErrEvidenceOverflow(max, got)
}
// Validate all evidence.
for idx, ev := range block.Evidence.Evidence {
// check that no evidence has been submitted more than once
// Check that no evidence has been submitted more than once
for i := idx + 1; i < len(block.Evidence.Evidence); i++ {
if ev.Equal(block.Evidence.Evidence[i]) {
return types.NewErrEvidenceInvalid(ev, errors.New("evidence was submitted twice"))
}
}
if evidencePool != nil {
if evidencePool.IsCommitted(ev) {
return types.NewErrEvidenceInvalid(ev, errors.New("evidence was already committed"))
}
if evidencePool.IsPending(ev) {
continue
}
}
// 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 := ev.(*types.AmnesiaEvidence); ok && ae.Polc.IsAbsent() && ae.PotentialAmnesiaEvidence.VoteA.Round <
ae.PotentialAmnesiaEvidence.VoteB.Round {
if err := evidencePool.AddEvidence(ae.PotentialAmnesiaEvidence); err != nil {
return types.NewErrEvidenceInvalid(ev,
fmt.Errorf("unknown amnesia evidence, trying to add to evidence pool, err: %w", err))
}
return types.NewErrEvidenceInvalid(ev, errors.New("amnesia evidence is new and hasn't undergone trial period yet"))
}
// A header needs to be fetched. For lunatic evidence this is so we can verify
// that some of the fields are different to the ones we have. For all evidence it
// it so we can verify that the time of the evidence is correct
header := evidencePool.Header(ev.Height())
if header == nil {
return fmt.Errorf("don't have block meta at height #%d", ev.Height())
}
if err := VerifyEvidence(stateDB, state, ev, header); err != nil {
// Verify evidence using the evidence pool
err := evidencePool.Verify(ev)
if err != nil {
return types.NewErrEvidenceInvalid(ev, err)
}
@ -195,73 +168,3 @@ func validateBlock(evidencePool EvidencePool, stateDB dbm.DB, state State, block
return nil
}
// VerifyEvidence verifies the evidence fully by checking:
// - it is sufficiently recent (MaxAge)
// - it is from a key who was a validator at the given height
// - it is internally consistent
// - it was properly signed by the alleged equivocator
func VerifyEvidence(stateDB dbm.DB, state State, evidence types.Evidence, committedHeader *types.Header) error {
var (
height = state.LastBlockHeight
evidenceParams = state.ConsensusParams.Evidence
ageDuration = state.LastBlockTime.Sub(evidence.Time())
ageNumBlocks = height - evidence.Height()
)
if committedHeader.Time != evidence.Time() {
return fmt.Errorf("evidence time (%v) is different to the time of the header we have for the same height (%v)",
evidence.Time(),
committedHeader.Time,
)
}
if ageDuration > evidenceParams.MaxAgeDuration && ageNumBlocks > evidenceParams.MaxAgeNumBlocks {
return fmt.Errorf(
"evidence from height %d (created at: %v) is too old; min height is %d and evidence can not be older than %v",
evidence.Height(),
evidence.Time(),
height-evidenceParams.MaxAgeNumBlocks,
state.LastBlockTime.Add(evidenceParams.MaxAgeDuration),
)
}
if ev, ok := evidence.(*types.LunaticValidatorEvidence); ok {
if err := ev.VerifyHeader(committedHeader); err != nil {
return err
}
}
valset, err := LoadValidators(stateDB, evidence.Height())
if err != nil {
// TODO: if err is just that we cant find it cuz we pruned, ignore.
// TODO: if its actually bad evidence, punish peer
return err
}
addr := evidence.Address()
var val *types.Validator
if ae, ok := evidence.(*types.AmnesiaEvidence); ok {
// check the validator set against the polc to make sure that a majority of valid votes was reached
if !ae.Polc.IsAbsent() {
err = ae.Polc.ValidateVotes(valset, state.ChainID)
if err != nil {
return fmt.Errorf("amnesia evidence contains invalid polc, err: %w", err)
}
}
}
// For all other types, expect evidence.Address to be a validator at height
// evidence.Height.
_, val = valset.GetByAddress(addr)
if val == nil {
return fmt.Errorf("address %X was not a validator at height %d", addr, evidence.Height())
}
if err := evidence.Verify(state.ChainID, val.PubKey); err != nil {
return err
}
return nil
}

+ 1
- 364
state/validation_test.go View File

@ -8,11 +8,6 @@ import (
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/require"
"github.com/tendermint/tendermint/crypto"
"github.com/tendermint/tendermint/libs/bytes"
tmrand "github.com/tendermint/tendermint/libs/rand"
"github.com/tendermint/tendermint/proto/tendermint/version"
"github.com/tendermint/tendermint/crypto/ed25519"
"github.com/tendermint/tendermint/crypto/tmhash"
"github.com/tendermint/tendermint/libs/log"
@ -220,14 +215,7 @@ func TestValidateBlockEvidence(t *testing.T) {
defaultEvidenceTime := time.Date(2019, 1, 1, 0, 0, 0, 0, time.UTC)
evpool := &mocks.EvidencePool{}
evpool.On("IsPending", mock.AnythingOfType("*types.DuplicateVoteEvidence")).Return(false)
evpool.On("IsCommitted", mock.AnythingOfType("*types.DuplicateVoteEvidence")).Return(false)
evpool.On("Header", mock.AnythingOfType("int64")).Return(func(height int64) *types.Header {
return &types.Header{
Time: defaultEvidenceTime,
Height: height,
}
})
evpool.On("Verify", mock.AnythingOfType("*types.DuplicateVoteEvidence")).Return(nil)
evpool.On("Update", mock.AnythingOfType("*types.Block"), mock.AnythingOfType("state.State")).Return()
state.ConsensusParams.Evidence.MaxNum = 3
@ -289,73 +277,6 @@ func TestValidateBlockEvidence(t *testing.T) {
}
}
func TestValidateFailBlockOnCommittedEvidence(t *testing.T) {
var height int64 = 1
state, stateDB, privVals := makeState(2, int(height))
_, val := state.Validators.GetByIndex(0)
_, val2 := state.Validators.GetByIndex(1)
ev := types.NewMockDuplicateVoteEvidenceWithValidator(height, defaultTestTime,
privVals[val.Address.String()], chainID)
ev2 := types.NewMockDuplicateVoteEvidenceWithValidator(height, defaultTestTime,
privVals[val2.Address.String()], chainID)
header := &types.Header{Time: defaultTestTime}
evpool := &mocks.EvidencePool{}
evpool.On("IsPending", ev).Return(false)
evpool.On("IsPending", ev2).Return(false)
evpool.On("IsCommitted", ev).Return(false)
evpool.On("IsCommitted", ev2).Return(true)
evpool.On("Header", height).Return(header)
blockExec := sm.NewBlockExecutor(
stateDB, log.TestingLogger(),
nil,
nil,
evpool)
// A block with a couple pieces of evidence passes.
block := makeBlock(state, height)
block.Evidence.Evidence = []types.Evidence{ev, ev2}
block.EvidenceHash = block.Evidence.Hash()
err := blockExec.ValidateBlock(state, block)
assert.Error(t, err)
assert.IsType(t, err, &types.ErrEvidenceInvalid{})
}
func TestValidateAlreadyPendingEvidence(t *testing.T) {
var height int64 = 1
state, stateDB, privVals := makeState(2, int(height))
_, val := state.Validators.GetByIndex(0)
_, val2 := state.Validators.GetByIndex(1)
ev := types.NewMockDuplicateVoteEvidenceWithValidator(height, defaultTestTime,
privVals[val.Address.String()], chainID)
ev2 := types.NewMockDuplicateVoteEvidenceWithValidator(height, defaultTestTime,
privVals[val2.Address.String()], chainID)
header := &types.Header{Time: defaultTestTime}
evpool := &mocks.EvidencePool{}
evpool.On("IsPending", ev).Return(false)
evpool.On("IsPending", ev2).Return(true)
evpool.On("IsCommitted", ev).Return(false)
evpool.On("IsCommitted", ev2).Return(false)
evpool.On("Header", height).Return(header)
blockExec := sm.NewBlockExecutor(
stateDB, log.TestingLogger(),
nil,
nil,
evpool)
// A block with a couple pieces of evidence passes.
block := makeBlock(state, height)
// add one evidence seen before and one evidence that hasn't
block.Evidence.Evidence = []types.Evidence{ev, ev2}
block.EvidenceHash = block.Evidence.Hash()
err := blockExec.ValidateBlock(state, block)
assert.NoError(t, err)
}
func TestValidateDuplicateEvidenceShouldFail(t *testing.T) {
var height int64 = 1
state, stateDB, privVals := makeState(2, int(height))
@ -379,287 +300,3 @@ func TestValidateDuplicateEvidenceShouldFail(t *testing.T) {
assert.Error(t, err)
}
var (
blockID = types.BlockID{
Hash: tmrand.Bytes(tmhash.Size),
PartSetHeader: types.PartSetHeader{
Total: 1,
Hash: tmrand.Bytes(tmhash.Size),
},
}
differentBlockID = types.BlockID{
Hash: tmrand.Bytes(tmhash.Size),
PartSetHeader: types.PartSetHeader{
Total: 1,
Hash: tmrand.Bytes(tmhash.Size),
},
}
)
func TestValidateUnseenAmnesiaEvidence(t *testing.T) {
var height int64 = 1
state, stateDB, vals := makeState(1, int(height))
addr, val := state.Validators.GetByIndex(0)
voteA := makeVote(height, 1, 0, addr, blockID)
vA := voteA.ToProto()
err := vals[val.Address.String()].SignVote(chainID, vA)
voteA.Signature = vA.Signature
require.NoError(t, err)
voteB := makeVote(height, 2, 0, addr, differentBlockID)
vB := voteB.ToProto()
err = vals[val.Address.String()].SignVote(chainID, vB)
voteB.Signature = vB.Signature
require.NoError(t, err)
pe := &types.PotentialAmnesiaEvidence{
VoteA: voteA,
VoteB: voteB,
}
ae := &types.AmnesiaEvidence{
PotentialAmnesiaEvidence: pe,
Polc: types.NewEmptyPOLC(),
}
evpool := &mocks.EvidencePool{}
evpool.On("IsPending", ae).Return(false)
evpool.On("IsCommitted", ae).Return(false)
evpool.On("AddEvidence", ae).Return(nil)
evpool.On("AddEvidence", pe).Return(nil)
blockExec := sm.NewBlockExecutor(
stateDB, log.TestingLogger(),
nil,
nil,
evpool)
// A block with a couple pieces of evidence passes.
block := makeBlock(state, height)
block.Evidence.Evidence = []types.Evidence{ae}
block.EvidenceHash = block.Evidence.Hash()
err = blockExec.ValidateBlock(state, block)
// if we don't have this evidence and it is has an empty polc then we expect to
// start our own trial period first
errMsg := "Invalid evidence: amnesia evidence is new and hasn't undergone trial period yet."
if assert.Error(t, err) {
assert.Equal(t, errMsg, err.Error()[:len(errMsg)])
}
}
// Amnesia Evidence can be directly approved without needing to undergo the trial period
func TestValidatePrimedAmnesiaEvidence(t *testing.T) {
var height int64 = 1
state, stateDB, vals := makeState(1, int(height))
addr, val := state.Validators.GetByIndex(0)
voteA := makeVote(height, 1, 0, addr, blockID)
voteA.Timestamp = defaultTestTime.Add(1 * time.Minute)
vA := voteA.ToProto()
err := vals[val.Address.String()].SignVote(chainID, vA)
require.NoError(t, err)
voteA.Signature = vA.Signature
voteB := makeVote(height, 2, 0, addr, differentBlockID)
voteB.Timestamp = defaultTestTime
vB := voteB.ToProto()
err = vals[val.Address.String()].SignVote(chainID, vB)
voteB.Signature = vB.Signature
require.NoError(t, err)
pe := types.NewPotentialAmnesiaEvidence(voteB, voteA, defaultTestTime)
ae := &types.AmnesiaEvidence{
PotentialAmnesiaEvidence: pe,
Polc: types.NewEmptyPOLC(),
}
header := &types.Header{Time: defaultTestTime}
evpool := &mocks.EvidencePool{}
evpool.On("IsPending", ae).Return(false)
evpool.On("IsCommitted", ae).Return(false)
evpool.On("AddEvidence", ae).Return(nil)
evpool.On("AddEvidence", pe).Return(nil)
evpool.On("Header", height).Return(header)
blockExec := sm.NewBlockExecutor(
stateDB, log.TestingLogger(),
nil,
nil,
evpool)
// A block with a couple pieces of evidence passes.
block := makeBlock(state, height)
block.Evidence.Evidence = []types.Evidence{ae}
block.EvidenceHash = block.Evidence.Hash()
err = blockExec.ValidateBlock(state, block)
// No error because this type of amnesia evidence is punishable
// without the need of a trial period
assert.NoError(t, err)
}
func TestVerifyEvidenceWrongAddress(t *testing.T) {
var height int64 = 1
state, stateDB, _ := makeState(1, int(height))
ev := types.NewMockDuplicateVoteEvidence(height, defaultTestTime, chainID)
header := &types.Header{Time: defaultTestTime}
evpool := &mocks.EvidencePool{}
evpool.On("IsPending", ev).Return(false)
evpool.On("IsCommitted", ev).Return(false)
evpool.On("Header", height).Return(header)
blockExec := sm.NewBlockExecutor(
stateDB, log.TestingLogger(),
nil,
nil,
evpool,
)
// A block with a couple pieces of evidence passes.
block := makeBlock(state, height)
block.Evidence.Evidence = []types.Evidence{ev}
block.EvidenceHash = block.Evidence.Hash()
err := blockExec.ValidateBlock(state, block)
errMsg := "Invalid evidence: address "
if assert.Error(t, err) {
assert.Equal(t, err.Error()[:len(errMsg)], errMsg)
}
}
func TestVerifyEvidenceExpiredEvidence(t *testing.T) {
var height int64 = 4
state, stateDB, _ := makeState(1, int(height))
state.ConsensusParams.Evidence.MaxAgeNumBlocks = 1
ev := types.NewMockDuplicateVoteEvidence(1, defaultTestTime, chainID)
err := sm.VerifyEvidence(stateDB, state, ev, &types.Header{Time: defaultTestTime})
errMsg := "evidence from height 1 (created at: 2019-01-01 00:00:00 +0000 UTC) is too old"
if assert.Error(t, err) {
assert.Equal(t, err.Error()[:len(errMsg)], errMsg)
}
}
func TestVerifyEvidenceInvalidTime(t *testing.T) {
height := 4
state, stateDB, _ := makeState(1, height)
differentTime := time.Date(2019, 2, 1, 0, 0, 0, 0, time.UTC)
ev := types.NewMockDuplicateVoteEvidence(int64(height), differentTime, chainID)
err := sm.VerifyEvidence(stateDB, state, ev, &types.Header{Time: defaultTestTime})
errMsg := "evidence time (2019-02-01 00:00:00 +0000 UTC) is different to the time" +
" of the header we have for the same height (2019-01-01 00:00:00 +0000 UTC)"
if assert.Error(t, err) {
assert.Equal(t, errMsg, err.Error())
}
}
func TestVerifyEvidenceWithAmnesiaEvidence(t *testing.T) {
var height int64 = 1
state, stateDB, vals := makeState(4, int(height))
addr, val := state.Validators.GetByIndex(0)
addr2, val2 := state.Validators.GetByIndex(1)
voteA := makeVote(height, 1, 0, addr, types.BlockID{})
vA := voteA.ToProto()
err := vals[val.Address.String()].SignVote(chainID, vA)
voteA.Signature = vA.Signature
require.NoError(t, err)
voteB := makeVote(height, 2, 0, addr, blockID)
vB := voteB.ToProto()
err = vals[val.Address.String()].SignVote(chainID, vB)
voteB.Signature = vB.Signature
require.NoError(t, err)
pae := types.NewPotentialAmnesiaEvidence(voteA, voteB, defaultTestTime)
voteC := makeVote(height, 2, 1, addr2, blockID)
vC := voteC.ToProto()
err = vals[val2.Address.String()].SignVote(chainID, vC)
voteC.Signature = vC.Signature
require.NoError(t, err)
//var ae types.Evidence
badAe := &types.AmnesiaEvidence{
PotentialAmnesiaEvidence: pae,
Polc: &types.ProofOfLockChange{
Votes: []*types.Vote{voteC},
PubKey: val.PubKey,
},
}
err = sm.VerifyEvidence(stateDB, state, badAe, &types.Header{Time: defaultTestTime})
if assert.Error(t, err) {
assert.Equal(t, err.Error(), "amnesia evidence contains invalid polc, err: "+
"invalid commit -- insufficient voting power: got 1000, needed more than 2667")
}
addr3, val3 := state.Validators.GetByIndex(2)
voteD := makeVote(height, 2, 2, addr3, blockID)
vD := voteD.ToProto()
err = vals[val3.Address.String()].SignVote(chainID, vD)
require.NoError(t, err)
voteD.Signature = vD.Signature
addr4, val4 := state.Validators.GetByIndex(3)
voteE := makeVote(height, 2, 3, addr4, blockID)
vE := voteE.ToProto()
err = vals[val4.Address.String()].SignVote(chainID, vE)
voteE.Signature = vE.Signature
require.NoError(t, err)
goodAe := &types.AmnesiaEvidence{
PotentialAmnesiaEvidence: pae,
Polc: &types.ProofOfLockChange{
Votes: []*types.Vote{voteC, voteD, voteE},
PubKey: val.PubKey,
},
}
err = sm.VerifyEvidence(stateDB, state, goodAe, &types.Header{Time: defaultTestTime})
assert.NoError(t, err)
goodAe = &types.AmnesiaEvidence{
PotentialAmnesiaEvidence: pae,
Polc: types.NewEmptyPOLC(),
}
err = sm.VerifyEvidence(stateDB, state, goodAe, &types.Header{Time: defaultTestTime})
assert.NoError(t, err)
}
func TestVerifyEvidenceWithLunaticValidatorEvidence(t *testing.T) {
state, stateDB, vals := makeState(4, 4)
state.ConsensusParams.Evidence.MaxAgeNumBlocks = 1
addr, val := state.Validators.GetByIndex(0)
h := &types.Header{
Version: version.Consensus{Block: 1, App: 2},
ChainID: chainID,
Height: 3,
Time: defaultTestTime,
LastBlockID: blockID,
LastCommitHash: tmhash.Sum([]byte("last_commit_hash")),
DataHash: tmhash.Sum([]byte("data_hash")),
ValidatorsHash: tmhash.Sum([]byte("validators_hash")),
NextValidatorsHash: tmhash.Sum([]byte("next_validators_hash")),
ConsensusHash: tmhash.Sum([]byte("consensus_hash")),
AppHash: tmhash.Sum([]byte("app_hash")),
LastResultsHash: tmhash.Sum([]byte("last_results_hash")),
EvidenceHash: tmhash.Sum([]byte("evidence_hash")),
ProposerAddress: crypto.AddressHash([]byte("proposer_address")),
}
vote := makeVote(3, 1, 0, addr, types.BlockID{
Hash: h.Hash(),
PartSetHeader: types.PartSetHeader{
Total: 100,
Hash: crypto.CRandBytes(tmhash.Size),
},
})
v := vote.ToProto()
err := vals[val.Address.String()].SignVote(chainID, v)
vote.Signature = v.Signature
require.NoError(t, err)
ev := types.NewLunaticValidatorEvidence(h, vote, "ConsensusHash", defaultTestTime)
err = ev.ValidateBasic()
require.NoError(t, err)
err = sm.VerifyEvidence(stateDB, state, ev, h)
if assert.Error(t, err) {
assert.Equal(t, "ConsensusHash matches committed hash", err.Error())
}
}
func makeVote(height int64, round, index int32, addr bytes.HexBytes, blockID types.BlockID) *types.Vote {
return &types.Vote{
Type: tmproto.SignedMsgType(2),
Height: height,
Round: round,
BlockID: blockID,
Timestamp: time.Now(),
ValidatorAddress: addr,
ValidatorIndex: index,
}
}

+ 19
- 19
types/evidence.go View File

@ -17,6 +17,25 @@ import (
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
)
// Evidence represents any provable malicious activity by a validator.
type Evidence interface {
Height() int64 // height of the equivocation
Time() time.Time // time of the equivocation
Address() []byte // address of the equivocating validator
Bytes() []byte // bytes which comprise the evidence
Hash() []byte // hash of the evidence
Verify(chainID string, pubKey crypto.PubKey) error // verify the evidence
Equal(Evidence) bool // check equality of evidence
ValidateBasic() error
String() string
}
type CompositeEvidence interface {
VerifyComposite(committedHeader *Header, valSet *ValidatorSet) error
Split(committedHeader *Header, valSet *ValidatorSet) []Evidence
}
const (
// MaxEvidenceBytes is a maximum size of any evidence (including amino overhead).
MaxEvidenceBytes int64 = 444
@ -64,25 +83,6 @@ func (err *ErrEvidenceOverflow) Error() string {
//-------------------------------------------
// Evidence represents any provable malicious activity by a validator.
type Evidence interface {
Height() int64 // height of the equivocation
Time() time.Time // time of the equivocation
Address() []byte // address of the equivocating validator
Bytes() []byte // bytes which comprise the evidence
Hash() []byte // hash of the evidence
Verify(chainID string, pubKey crypto.PubKey) error // verify the evidence
Equal(Evidence) bool // check equality of evidence
ValidateBasic() error
String() string
}
type CompositeEvidence interface {
VerifyComposite(committedHeader *Header, valSet *ValidatorSet) error
Split(committedHeader *Header, valSet *ValidatorSet) []Evidence
}
func EvidenceToProto(evidence Evidence) (*tmproto.Evidence, error) {
if evidence == nil {
return nil, errors.New("nil evidence")


Loading…
Cancel
Save