Browse Source

evidence: introduction of LightClientAttackEvidence and refactor of evidence lifecycle (#5361)

evidence: modify evidence types (#5342)

light: detect light client attacks (#5344)

evidence: refactor evidence pool (#5345)

abci: application evidence prepared by evidence pool (#5354)
pull/5362/head
Callum Waters 4 years ago
committed by GitHub
parent
commit
ed002cea7e
No known key found for this signature in database GPG Key ID: 4AEE18F83AFDEB23
57 changed files with 3167 additions and 1168 deletions
  1. +2
    -0
      CHANGELOG_PENDING.md
  2. +1
    -1
      blockchain/v0/reactor_test.go
  3. +1
    -1
      blockchain/v1/reactor_test.go
  4. +2
    -2
      blockchain/v2/reactor_test.go
  5. +39
    -14
      consensus/byzantine_test.go
  6. +1
    -1
      consensus/common_test.go
  7. +24
    -50
      consensus/reactor_test.go
  8. +1
    -1
      consensus/replay.go
  9. +1
    -1
      consensus/replay_file.go
  10. +0
    -15
      consensus/replay_stubs.go
  11. +1
    -1
      consensus/replay_test.go
  12. +16
    -6
      consensus/state.go
  13. +1
    -1
      consensus/wal_generator.go
  14. +1
    -1
      docs/architecture/adr-059-evidence-composition-and-lifecycle.md
  15. +17
    -1
      evidence/mocks/block_store.go
  16. +52
    -0
      evidence/mocks/state_store.go
  17. +331
    -129
      evidence/pool.go
  18. +211
    -96
      evidence/pool_test.go
  19. +4
    -11
      evidence/reactor.go
  20. +29
    -25
      evidence/reactor_test.go
  21. +1
    -0
      evidence/services.go
  22. +254
    -44
      evidence/verify.go
  23. +431
    -51
      evidence/verify_test.go
  24. +46
    -118
      light/client.go
  25. +1
    -1
      light/client_benchmark_test.go
  26. +30
    -6
      light/client_test.go
  27. +247
    -0
      light/detector.go
  28. +210
    -0
      light/detector_test.go
  29. +22
    -16
      light/errors.go
  30. +29
    -15
      light/helpers_test.go
  31. +8
    -14
      light/verifier.go
  32. +3
    -3
      light/verifier_test.go
  33. +1
    -1
      node/node_test.go
  34. +1
    -1
      p2p/mocks/peer.go
  35. +139
    -58
      proto/tendermint/evidence/types.pb.go
  36. +6
    -3
      proto/tendermint/evidence/types.proto
  37. +295
    -51
      proto/tendermint/types/evidence.pb.go
  38. +6
    -5
      proto/tendermint/types/evidence.proto
  39. +1
    -1
      proxy/mocks/app_conn_consensus.go
  40. +1
    -1
      proxy/mocks/app_conn_mempool.go
  41. +1
    -1
      proxy/mocks/app_conn_query.go
  42. +1
    -1
      proxy/mocks/app_conn_snapshot.go
  43. +4
    -6
      rpc/client/evidence_test.go
  44. +11
    -19
      state/execution.go
  45. +40
    -48
      state/execution_test.go
  46. +36
    -18
      state/mocks/evidence_pool.go
  47. +205
    -0
      state/mocks/store.go
  48. +20
    -10
      state/services.go
  49. +6
    -0
      state/store.go
  50. +2
    -18
      state/validation.go
  51. +7
    -31
      state/validation_test.go
  52. +1
    -1
      statesync/mocks/state_provider.go
  53. +1
    -1
      types/block_test.go
  54. +225
    -189
      types/evidence.go
  55. +138
    -69
      types/evidence_test.go
  56. +2
    -8
      types/protobuf.go
  57. +0
    -2
      types/protobuf_test.go

+ 2
- 0
CHANGELOG_PENDING.md View File

@ -20,6 +20,7 @@ Friendly reminder, we have a [bug bounty program](https://hackerone.com/tendermi
- [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)
- [evidence] [\#5361](https://github.com/tendermint/tendermint/pull/5361) Add LightClientAttackEvidence and change evidence interface (@cmwaters)
- [params] \#5319 Remove `ProofofTrialPeriod` from evidence params (@marbar3778)
- [crypto/secp256k1] \#5280 `secp256k1` has been removed from the Tendermint repo. (@marbar3778)
- [state] \#5348 Define an Interface for the state store. (@marbar3778)
@ -31,6 +32,7 @@ Friendly reminder, we have a [bug bounty program](https://hackerone.com/tendermi
- [privval] \#5239 Add `chainID` to requests from client. (@marbar3778)
- [config] Add `--consensus.double_sign_check_height` flag and `DoubleSignCheckHeight` config variable. See [ADR-51](https://github.com/tendermint/tendermint/blob/master/docs/architecture/adr-051-double-signing-risk-reduction.md)
- [light] [\#5298](https://github.com/tendermint/tendermint/pull/5298) Morph validator set and signed header into light block (@cmwaters)
- [evidence] [\#5361](https://github.com/tendermint/tendermint/pull/5361) Add LightClientAttackEvidence and refactor evidence lifecycle (@cmwaters)
## IMPROVEMENTS


+ 1
- 1
blockchain/v0/reactor_test.go View File

@ -85,7 +85,7 @@ func newBlockchainReactor(
db := dbm.NewMemDB()
stateStore = sm.NewStore(db)
blockExec := sm.NewBlockExecutor(stateStore, log.TestingLogger(), proxyApp.Consensus(),
mock.Mempool{}, sm.MockEvidencePool{})
mock.Mempool{}, sm.EmptyEvidencePool{})
if err = stateStore.Save(state); err != nil {
panic(err)
}


+ 1
- 1
blockchain/v1/reactor_test.go View File

@ -117,7 +117,7 @@ func newBlockchainReactor(
db := dbm.NewMemDB()
stateStore = sm.NewStore(db)
blockExec := sm.NewBlockExecutor(stateStore, log.TestingLogger(), proxyApp.Consensus(),
mock.Mempool{}, sm.MockEvidencePool{})
mock.Mempool{}, sm.EmptyEvidencePool{})
if err = stateStore.Save(state); err != nil {
panic(err)
}


+ 2
- 2
blockchain/v2/reactor_test.go View File

@ -157,7 +157,7 @@ func newTestReactor(p testReactorParams) *BlockchainReactor {
}
db := dbm.NewMemDB()
stateStore := sm.NewStore(db)
appl = sm.NewBlockExecutor(stateStore, p.logger, proxyApp.Consensus(), mock.Mempool{}, sm.MockEvidencePool{})
appl = sm.NewBlockExecutor(stateStore, p.logger, proxyApp.Consensus(), mock.Mempool{}, sm.EmptyEvidencePool{})
if err = stateStore.Save(state); err != nil {
panic(err)
}
@ -510,7 +510,7 @@ func newReactorStore(
db := dbm.NewMemDB()
stateStore = sm.NewStore(db)
blockExec := sm.NewBlockExecutor(stateStore, log.TestingLogger(), proxyApp.Consensus(),
mock.Mempool{}, sm.MockEvidencePool{})
mock.Mempool{}, sm.EmptyEvidencePool{})
if err = stateStore.Save(state); err != nil {
panic(err)
}


+ 39
- 14
consensus/byzantine_test.go View File

@ -35,6 +35,7 @@ import (
func TestByzantinePrevoteEquivocation(t *testing.T) {
const nValidators = 4
const byzantineNode = 0
const prevoteHeight = int64(2)
testName := "consensus_byzantine_test"
tickerFunc := newMockTickerFunc(true)
appFunc := newCounter
@ -129,7 +130,7 @@ func TestByzantinePrevoteEquivocation(t *testing.T) {
// alter prevote so that the byzantine node double votes when height is 2
bcs.doPrevote = func(height int64, round int32) {
// allow first height to happen normally so that byzantine validator is no longer proposer
if height == 2 {
if height == prevoteHeight {
bcs.Logger.Info("Sending two votes")
prevote1, err := bcs.signVote(tmproto.PrevoteType, bcs.ProposalBlock.Hash(), bcs.ProposalBlockParts.Header())
require.NoError(t, err)
@ -162,22 +163,46 @@ func TestByzantinePrevoteEquivocation(t *testing.T) {
// Evidence should be submitted and committed at the third height but
// we will check the first five just in case
var evidence types.Evidence
for i := 0; i < 5; i++ {
msg := <-blocksSubs[nValidators-1].Out()
block := msg.Data().(types.EventDataNewBlock).Block
if len(block.Evidence.Evidence) > 0 {
evidence = block.Evidence.Evidence[0]
break
evidenceFromEachValidator := make([]types.Evidence, nValidators)
wg := new(sync.WaitGroup)
wg.Add(4)
for height := 1; height < 5; height++ {
for i := 0; i < nValidators; i++ {
go func(j int) {
msg := <-blocksSubs[j].Out()
block := msg.Data().(types.EventDataNewBlock).Block
if len(block.Evidence.Evidence) != 0 {
evidenceFromEachValidator[j] = block.Evidence.Evidence[0]
wg.Done()
}
}(i)
}
}
if assert.NotNil(t, evidence) {
ev, ok := evidence.(*types.DuplicateVoteEvidence)
assert.True(t, ok)
pubkey, _ := bcs.privValidator.GetPubKey()
assert.Equal(t, []byte(pubkey.Address()), ev.Address())
done := make(chan struct{})
go func() {
wg.Wait()
close(done)
}()
pubkey, _ := bcs.privValidator.GetPubKey()
select {
case <-done:
for idx, ev := range evidenceFromEachValidator {
if assert.NotNil(t, ev, idx) {
ev, ok := ev.(*types.DuplicateVoteEvidence)
assert.True(t, ok)
assert.Equal(t, pubkey.Address(), ev.VoteA.ValidatorAddress)
assert.Equal(t, prevoteHeight, ev.Height())
}
}
case <-time.After(10 * time.Second):
for i, reactor := range reactors {
t.Logf("Consensus Reactor %d\n%v", i, reactor)
}
t.Fatalf("Timed out waiting for all validators to commit first block")
}
}


+ 1
- 1
consensus/common_test.go View File

@ -386,7 +386,7 @@ func newStateWithConfigAndBlockStore(
mempool.EnableTxsAvailable()
}
evpool := emptyEvidencePool{}
evpool := sm.EmptyEvidencePool{}
// Make State
stateDB := blockDB


+ 24
- 50
consensus/reactor_test.go View File

@ -12,6 +12,7 @@ import (
"time"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/require"
dbm "github.com/tendermint/tm-db"
@ -29,9 +30,10 @@ import (
tmsync "github.com/tendermint/tendermint/libs/sync"
mempl "github.com/tendermint/tendermint/mempool"
"github.com/tendermint/tendermint/p2p"
"github.com/tendermint/tendermint/p2p/mock"
p2pmock "github.com/tendermint/tendermint/p2p/mock"
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
sm "github.com/tendermint/tendermint/state"
statemocks "github.com/tendermint/tendermint/state/mocks"
"github.com/tendermint/tendermint/store"
"github.com/tendermint/tendermint/types"
)
@ -165,11 +167,20 @@ func TestReactorWithEvidence(t *testing.T) {
// mock the evidence pool
// everyone includes evidence of another double signing
vIdx := (i + 1) % nValidators
evpool := newMockEvidencePool(privVals[vIdx])
evpool := &statemocks.EvidencePool{}
evpool.On("CheckEvidence", mock.AnythingOfType("types.EvidenceList")).Return(nil)
evpool.On("PendingEvidence", mock.AnythingOfType("uint32")).Return([]types.Evidence{
types.NewMockDuplicateVoteEvidenceWithValidator(1, defaultTestTime, privVals[vIdx], config.ChainID()),
})
evpool.On("Update", mock.AnythingOfType("state.State")).Return()
evpool.On("ABCIEvidence", mock.AnythingOfType("int64"), mock.AnythingOfType("[]types.Evidence")).Return(
[]abci.Evidence{})
evpool2 := sm.EmptyEvidencePool{}
// Make State
blockExec := sm.NewBlockExecutor(stateStore, log.TestingLogger(), proxyAppConnCon, mempool, evpool)
cs := NewState(thisConfig.Consensus, state, blockExec, blockStore, mempool, evpool)
cs := NewState(thisConfig.Consensus, state, blockExec, blockStore, mempool, evpool2)
cs.SetLogger(log.TestingLogger().With("module", "consensus"))
cs.SetPrivValidator(pv)
@ -188,52 +199,15 @@ func TestReactorWithEvidence(t *testing.T) {
reactors, blocksSubs, eventBuses := startConsensusNet(t, css, nValidators)
defer stopConsensusNet(log.TestingLogger(), reactors, eventBuses)
// wait till everyone makes the first new block with no evidence
timeoutWaitGroup(t, nValidators, func(j int) {
msg := <-blocksSubs[j].Out()
block := msg.Data().(types.EventDataNewBlock).Block
assert.True(t, len(block.Evidence.Evidence) == 0)
}, css)
// second block should have evidence
timeoutWaitGroup(t, nValidators, func(j int) {
msg := <-blocksSubs[j].Out()
block := msg.Data().(types.EventDataNewBlock).Block
assert.True(t, len(block.Evidence.Evidence) > 0)
}, css)
}
// mock evidence pool returns no evidence for block 1,
// and returnes one piece for all higher blocks. The one piece
// is for a given validator at block 1.
type mockEvidencePool struct {
height int
ev []types.Evidence
}
func newMockEvidencePool(val types.PrivValidator) *mockEvidencePool {
return &mockEvidencePool{
ev: []types.Evidence{types.NewMockDuplicateVoteEvidenceWithValidator(1, defaultTestTime, val, config.ChainID())},
}
}
// NOTE: maxBytes is ignored
func (m *mockEvidencePool) PendingEvidence(maxBytes uint32) []types.Evidence {
if m.height > 0 {
return m.ev
}
return nil
}
func (m *mockEvidencePool) AddEvidence(types.Evidence) error { return nil }
func (m *mockEvidencePool) Update(block *types.Block, state sm.State) {
if m.height > 0 {
if len(block.Evidence.Evidence) == 0 {
panic("block has no evidence")
}
// we expect for each validator that is the proposer to propose one piece of evidence.
for i := 0; i < nValidators; i++ {
timeoutWaitGroup(t, nValidators, func(j int) {
msg := <-blocksSubs[j].Out()
block := msg.Data().(types.EventDataNewBlock).Block
assert.Len(t, block.Evidence.Evidence, 1)
}, css)
}
m.height++
}
func (m *mockEvidencePool) Verify(types.Evidence) error { return nil }
//------------------------------------
@ -268,7 +242,7 @@ func TestReactorReceiveDoesNotPanicIfAddPeerHasntBeenCalledYet(t *testing.T) {
var (
reactor = reactors[0]
peer = mock.NewPeer(nil)
peer = p2pmock.NewPeer(nil)
msg = MustEncode(&HasVoteMessage{Height: 1,
Round: 1, Index: 1, Type: tmproto.PrevoteType})
)
@ -291,7 +265,7 @@ func TestReactorReceivePanicsIfInitPeerHasntBeenCalledYet(t *testing.T) {
var (
reactor = reactors[0]
peer = mock.NewPeer(nil)
peer = p2pmock.NewPeer(nil)
msg = MustEncode(&HasVoteMessage{Height: 1,
Round: 1, Index: 1, Type: tmproto.PrevoteType})
)
@ -667,7 +641,7 @@ func timeoutWaitGroup(t *testing.T, n int, f func(int), css []*State) {
// we're running many nodes in-process, possibly in in a virtual machine,
// and spewing debug messages - making a block could take a while,
timeout := time.Second * 300
timeout := time.Second * 120
select {
case <-done:


+ 1
- 1
consensus/replay.go View File

@ -496,7 +496,7 @@ func (h *Handshaker) replayBlock(state sm.State, height int64, proxyApp proxy.Ap
// Use stubs for both mempool and evidence pool since no transactions nor
// evidence are needed here - block already exists.
blockExec := sm.NewBlockExecutor(h.stateStore, h.logger, proxyApp, emptyMempool{}, emptyEvidencePool{})
blockExec := sm.NewBlockExecutor(h.stateStore, h.logger, proxyApp, emptyMempool{}, sm.EmptyEvidencePool{})
blockExec.SetEventBus(h.eventBus)
var err error


+ 1
- 1
consensus/replay_file.go View File

@ -327,7 +327,7 @@ func newConsensusStateForReplay(config cfg.BaseConfig, csConfig *cfg.ConsensusCo
tmos.Exit(fmt.Sprintf("Error on handshake: %v", err))
}
mempool, evpool := emptyMempool{}, emptyEvidencePool{}
mempool, evpool := emptyMempool{}, sm.EmptyEvidencePool{}
blockExec := sm.NewBlockExecutor(stateStore, log.TestingLogger(), proxyApp.Consensus(), mempool, evpool)
consensusState := NewState(csConfig, state.Copy(), blockExec,


+ 0
- 15
consensus/replay_stubs.go View File

@ -6,7 +6,6 @@ import (
mempl "github.com/tendermint/tendermint/mempool"
tmstate "github.com/tendermint/tendermint/proto/tendermint/state"
"github.com/tendermint/tendermint/proxy"
sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/types"
)
@ -45,20 +44,6 @@ func (emptyMempool) TxsWaitChan() <-chan struct{} { return nil }
func (emptyMempool) InitWAL() error { return nil }
func (emptyMempool) CloseWAL() {}
//-----------------------------------------------------------------------------
type emptyEvidencePool struct{}
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) Header(int64) *types.Header { return nil }
//-----------------------------------------------------------------------------
// mockProxyApp uses ABCIResponses to give the right results.
//


+ 1
- 1
consensus/replay_test.go View File

@ -305,7 +305,7 @@ const (
var (
mempool = emptyMempool{}
evpool = emptyEvidencePool{}
evpool = sm.EmptyEvidencePool{}
sim testSim
)


+ 16
- 6
consensus/state.go View File

@ -73,7 +73,9 @@ type txNotifier interface {
// interface to the evidence pool
type evidencePool interface {
AddEvidence(types.Evidence) error
// Adds consensus based evidence to the evidence pool where time is the time
// of the block where the offense occurred and the validator set is the current one.
AddEvidenceFromConsensus(types.Evidence, time.Time, *types.ValidatorSet) error
}
// State handles execution of the consensus algorithm.
@ -1681,13 +1683,20 @@ func (cs *State) recordMetrics(height int64, block *types.Block) {
cs.metrics.MissingValidators.Set(float64(missingValidators))
cs.metrics.MissingValidatorsPower.Set(float64(missingValidatorsPower))
cs.metrics.ByzantineValidators.Set(float64(len(block.Evidence.Evidence)))
byzantineValidatorsPower := int64(0)
// NOTE: byzantine validators power and count is only for consensus evidence i.e. duplicate vote
var (
byzantineValidatorsPower = int64(0)
byzantineValidatorsCount = int64(0)
)
for _, ev := range block.Evidence.Evidence {
if _, val := cs.Validators.GetByAddress(ev.Address()); val != nil {
byzantineValidatorsPower += val.VotingPower
if dve, ok := ev.(*types.DuplicateVoteEvidence); ok {
if _, val := cs.Validators.GetByAddress(dve.VoteA.ValidatorAddress); val != nil {
byzantineValidatorsCount++
byzantineValidatorsPower += val.VotingPower
}
}
}
cs.metrics.ByzantineValidators.Set(float64(byzantineValidatorsCount))
cs.metrics.ByzantineValidatorsPower.Set(float64(byzantineValidatorsPower))
if height > 1 {
@ -1855,7 +1864,8 @@ func (cs *State) tryAddVote(vote *types.Vote, peerID p2p.ID) (bool, error) {
} else {
timestamp = sm.MedianTime(cs.LastCommit.MakeCommit(), cs.LastValidators)
}
evidenceErr := cs.evpool.AddEvidence(types.NewDuplicateVoteEvidence(voteErr.VoteA, voteErr.VoteB, timestamp))
evidenceErr := cs.evpool.AddEvidenceFromConsensus(
types.NewDuplicateVoteEvidence(voteErr.VoteA, voteErr.VoteB), timestamp, cs.Validators)
if evidenceErr != nil {
cs.Logger.Error("Failed to add evidence to the evidence pool", "err", evidenceErr)
}


+ 1
- 1
consensus/wal_generator.go View File

@ -82,7 +82,7 @@ func WALGenerateNBlocks(t *testing.T, wr io.Writer, numBlocks int) (err error) {
}
})
mempool := emptyMempool{}
evpool := emptyEvidencePool{}
evpool := sm.EmptyEvidencePool{}
blockExec := sm.NewBlockExecutor(stateStore, log.TestingLogger(), proxyApp.Consensus(), mempool, evpool)
consensusState := NewState(config.Consensus, state.Copy(), blockExec, blockStore, mempool, evpool)
consensusState.SetLogger(logger)


+ 1
- 1
docs/architecture/adr-059-evidence-composition-and-lifecycle.md View File

@ -269,7 +269,7 @@ Evidence is first detected and created in the light client and consensus reactor
## Status
Accepted
Implemented
## Consequences


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

@ -1,4 +1,4 @@
// Code generated by mockery v2.2.1. DO NOT EDIT.
// Code generated by mockery v2.1.0. DO NOT EDIT.
package mocks
@ -12,6 +12,22 @@ type BlockStore struct {
mock.Mock
}
// LoadBlockCommit provides a mock function with given fields: height
func (_m *BlockStore) LoadBlockCommit(height int64) *types.Commit {
ret := _m.Called(height)
var r0 *types.Commit
if rf, ok := ret.Get(0).(func(int64) *types.Commit); ok {
r0 = rf(height)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*types.Commit)
}
}
return r0
}
// LoadBlockMeta provides a mock function with given fields: height
func (_m *BlockStore) LoadBlockMeta(height int64) *types.BlockMeta {
ret := _m.Called(height)


+ 52
- 0
evidence/mocks/state_store.go View File

@ -0,0 +1,52 @@
// Code generated by mockery v2.1.0. DO NOT EDIT.
package mocks
import (
mock "github.com/stretchr/testify/mock"
state "github.com/tendermint/tendermint/state"
types "github.com/tendermint/tendermint/types"
)
// StateStore is an autogenerated mock type for the StateStore type
type StateStore struct {
mock.Mock
}
// LoadState provides a mock function with given fields:
func (_m *StateStore) LoadState() state.State {
ret := _m.Called()
var r0 state.State
if rf, ok := ret.Get(0).(func() state.State); ok {
r0 = rf()
} else {
r0 = ret.Get(0).(state.State)
}
return r0
}
// LoadValidators provides a mock function with given fields: height
func (_m *StateStore) LoadValidators(height int64) (*types.ValidatorSet, error) {
ret := _m.Called(height)
var r0 *types.ValidatorSet
if rf, ok := ret.Get(0).(func(int64) *types.ValidatorSet); ok {
r0 = rf(height)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*types.ValidatorSet)
}
}
var r1 error
if rf, ok := ret.Get(1).(func(int64) error); ok {
r1 = rf(height)
} else {
r1 = ret.Error(1)
}
return r0, r1
}

+ 331
- 129
evidence/pool.go View File

@ -1,17 +1,22 @@
package evidence
import (
"bytes"
"errors"
"fmt"
"reflect"
"sync"
"sync/atomic"
"time"
"github.com/gogo/protobuf/proto"
gogotypes "github.com/gogo/protobuf/types"
dbm "github.com/tendermint/tm-db"
abci "github.com/tendermint/tendermint/abci/types"
clist "github.com/tendermint/tendermint/libs/clist"
"github.com/tendermint/tendermint/libs/log"
evproto "github.com/tendermint/tendermint/proto/tendermint/evidence"
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/types"
@ -28,15 +33,19 @@ type Pool struct {
evidenceStore dbm.DB
evidenceList *clist.CList // concurrent linked-list of evidence
evidenceSize uint32 // amount of pending evidence
// needed to load validators to verify evidence
stateDB sm.Store
// needed to load headers to verify evidence
// needed to load headers and commits to verify evidence
blockStore BlockStore
mtx sync.Mutex
// latest state
state sm.State
pruningHeight int64
pruningTime time.Time
}
// NewPool creates an evidence pool. If using an existing evidence store,
@ -55,10 +64,19 @@ func NewPool(evidenceDB dbm.DB, stateDB sm.Store, blockStore BlockStore) (*Pool,
logger: log.NewNopLogger(),
evidenceStore: evidenceDB,
evidenceList: clist.New(),
evidenceSize: 0,
pruningHeight: state.LastBlockHeight,
pruningTime: state.LastBlockTime,
}
// if pending evidence already in db, in event of prior failure, then load it back to the evidenceList
evList := pool.AllPendingEvidence()
// if pending evidence already in db, in event of prior failure, then check for expiration,
// update the size and load it back to the evidenceList
pool.removeExpiredPendingEvidence()
evList, err := pool.listEvidence(baseKeyPending, -1)
if err != nil {
return nil, err
}
atomic.AddUint32(&pool.evidenceSize, uint32(len(evList)))
for _, ev := range evList {
pool.evidenceList.PushBack(ev)
}
@ -67,9 +85,7 @@ func NewPool(evidenceDB dbm.DB, stateDB sm.Store, blockStore BlockStore) (*Pool,
}
// PendingEvidence is used primarily as part of block proposal and returns up to maxNum of uncommitted evidence.
// If maxNum is -1, all evidence is returned. Pending evidence is prioritized based on time.
func (evpool *Pool) PendingEvidence(maxNum uint32) []types.Evidence {
evpool.removeExpiredPendingEvidence()
evidence, err := evpool.listEvidence(baseKeyPending, int64(maxNum))
if err != nil {
evpool.logger.Error("Unable to retrieve pending evidence", "err", err)
@ -77,40 +93,26 @@ func (evpool *Pool) PendingEvidence(maxNum uint32) []types.Evidence {
return evidence
}
// AllPendingEvidence returns all evidence ready to be proposed and committed.
func (evpool *Pool) AllPendingEvidence() []types.Evidence {
evpool.removeExpiredPendingEvidence()
evidence, err := evpool.listEvidence(baseKeyPending, -1)
if err != nil {
evpool.logger.Error("Unable to retrieve pending evidence", "err", err)
}
return evidence
}
// Update uses the latest block & state to update any evidence that has been committed, to prune all expired evidence
func (evpool *Pool) Update(block *types.Block, state sm.State) {
// Update pulls the latest state to be used for expiration and evidence params and then prunes all expired evidence
func (evpool *Pool) Update(state sm.State) {
// sanity check
if state.LastBlockHeight != block.Height {
panic(fmt.Sprintf("Failed EvidencePool.Update sanity check: got state.Height=%d with block.Height=%d",
if state.LastBlockHeight <= evpool.state.LastBlockHeight {
panic(fmt.Sprintf(
"Failed EvidencePool.Update new state height is less than or equal to previous state height: %d <= %d",
state.LastBlockHeight,
block.Height,
),
)
evpool.state.LastBlockHeight,
))
}
evpool.logger.Info("Updating evidence pool", "last_block_height", state.LastBlockHeight,
"last_block_time", state.LastBlockTime)
// update the state
evpool.updateState(state)
// remove evidence from pending and mark committed
evpool.MarkEvidenceAsCommitted(block.Height, block.Evidence.Evidence)
// prune pending, committed and potential evidence and polc's periodically
if block.Height%state.ConsensusParams.Evidence.MaxAgeNumBlocks == 0 {
evpool.logger.Debug("Pruning expired evidence")
// 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()
// prune pending evidence when it has expired. This also updates when the next evidence will expire
if atomic.LoadUint32(&evpool.evidenceSize) > 0 && state.LastBlockHeight > evpool.pruningHeight &&
state.LastBlockTime.After(evpool.pruningTime) {
evpool.pruningHeight, evpool.pruningTime = evpool.removeExpiredPendingEvidence()
}
}
@ -118,18 +120,20 @@ func (evpool *Pool) Update(block *types.Block, state sm.State) {
func (evpool *Pool) AddEvidence(ev types.Evidence) error {
evpool.logger.Debug("Attempting to add evidence", "ev", ev)
if evpool.Has(ev) {
return nil
// We have already verified this piece of evidence - no need to do it again
if evpool.isPending(ev) {
return errors.New("evidence already verified and added")
}
// 1) Verify against state.
if err := evpool.verify(ev); err != nil {
return types.NewErrEvidenceInvalid(ev, err)
evInfo, err := evpool.verify(ev)
if err != nil {
return types.NewErrInvalidEvidence(ev, err)
}
// 2) Save to store.
if err := evpool.addPendingEvidence(ev); err != nil {
return fmt.Errorf("database error when adding evidence: %v", err)
if err := evpool.addPendingEvidence(evInfo); err != nil {
return fmt.Errorf("can't add evidence to pending list: %w", err)
}
// 3) Add evidence to clist.
@ -140,32 +144,137 @@ func (evpool *Pool) AddEvidence(ev 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")
// AddEvidenceFromConsensus should be exposed only to the consensus so it can add evidence to the pool
// directly without the need for verification.
func (evpool *Pool) AddEvidenceFromConsensus(ev types.Evidence, time time.Time, valSet *types.ValidatorSet) error {
var (
vals []*types.Validator
totalPower int64
)
if evpool.isPending(ev) {
return errors.New("evidence already verified and added") // we already have this evidence
}
// We have already verified this piece of evidence - no need to do it again
if evpool.IsPending(evidence) {
return nil
switch ev := ev.(type) {
case *types.DuplicateVoteEvidence:
_, val := valSet.GetByAddress(ev.VoteA.ValidatorAddress)
vals = append(vals, val)
totalPower = valSet.TotalVotingPower()
default:
return fmt.Errorf("unrecognized evidence type: %T", ev)
}
evInfo := &info{
Evidence: ev,
Time: time,
Validators: vals,
TotalVotingPower: totalPower,
}
if err := evpool.addPendingEvidence(evInfo); err != nil {
return fmt.Errorf("can't add evidence to pending list: %w", err)
}
return evpool.verify(evidence)
evpool.evidenceList.PushBack(ev)
evpool.logger.Info("Verified new evidence of byzantine behavior", "evidence", ev)
return nil
}
func (evpool *Pool) verify(evidence types.Evidence) error {
return VerifyEvidence(evidence, evpool.State(), evpool.stateDB, evpool.blockStore)
// CheckEvidence takes an array of evidence from a block and verifies all the evidence there.
// If it has already verified the evidence then it jumps to the next one. It ensures that no
// evidence has already been committed or is being proposed twice. It also adds any
// evidence that it doesn't currently have so that it can quickly form ABCI Evidence later.
func (evpool *Pool) CheckEvidence(evList types.EvidenceList) error {
hashes := make([][]byte, len(evList))
for idx, ev := range evList {
ok := evpool.fastCheck(ev)
if !ok {
evInfo, err := evpool.verify(ev)
if err != nil {
return &types.ErrInvalidEvidence{Evidence: ev, Reason: err}
}
if err := evpool.addPendingEvidence(evInfo); err != nil {
evpool.logger.Error("Can't add evidence to pending list", "err", err, "evInfo", evInfo)
}
evpool.logger.Info("Verified new evidence of byzantine behavior", "evidence", ev)
}
// check for duplicate evidence. We cache hashes so we don't have to work them out again.
hashes[idx] = ev.Hash()
for i := idx - 1; i >= 0; i-- {
if bytes.Equal(hashes[i], hashes[idx]) {
return &types.ErrInvalidEvidence{Evidence: ev, Reason: errors.New("duplicate evidence")}
}
}
}
return nil
}
// MarkEvidenceAsCommitted marks all the evidence as committed and removes it
// from the queue.
func (evpool *Pool) MarkEvidenceAsCommitted(height int64, evidence []types.Evidence) {
// ABCIEvidence processes all the evidence in the block, marking it as committed and removing it
// from the pending database. It then forms the individual abci evidence that will be passed back to
// the application.
func (evpool *Pool) ABCIEvidence(height int64, evidence []types.Evidence) []abci.Evidence {
// make a map of committed evidence to remove from the clist
blockEvidenceMap := make(map[string]struct{})
blockEvidenceMap := make(map[string]struct{}, len(evidence))
abciEvidence := make([]abci.Evidence, 0)
for _, ev := range evidence {
// get entire evidence info from pending list
infoBytes, err := evpool.evidenceStore.Get(keyPending(ev))
if err != nil {
evpool.logger.Error("Unable to retrieve evidence to pass to ABCI. "+
"Evidence pool should have seen this evidence before",
"evidence", ev, "err", err)
continue
}
var infoProto evproto.Info
err = infoProto.Unmarshal(infoBytes)
if err != nil {
evpool.logger.Error("Decoding evidence info failed", "err", err, "height", ev.Height(), "hash", ev.Hash())
continue
}
evInfo, err := infoFromProto(&infoProto)
if err != nil {
evpool.logger.Error("Converting evidence info from proto failed", "err", err, "height", ev.Height(),
"hash", ev.Hash())
continue
}
var evType abci.EvidenceType
switch ev.(type) {
case *types.DuplicateVoteEvidence:
evType = abci.EvidenceType_DUPLICATE_VOTE
case *types.LightClientAttackEvidence:
evType = abci.EvidenceType_LIGHT_CLIENT_ATTACK
default:
evpool.logger.Error("Unknown evidence type", "T", reflect.TypeOf(ev))
continue
}
for _, val := range evInfo.Validators {
abciEv := abci.Evidence{
Type: evType,
Validator: types.TM2PB.Validator(val),
Height: ev.Height(),
Time: evInfo.Time,
TotalVotingPower: evInfo.TotalVotingPower,
}
abciEvidence = append(abciEvidence, abciEv)
evpool.logger.Info("Created ABCI evidence", "ev", abciEv)
}
// we can now remove the evidence from the pending list and the clist that we use for gossiping
evpool.removePendingEvidence(ev)
blockEvidenceMap[evMapKey(ev)] = struct{}{}
// Add evidence to the committed list
// As the evidence is stored in the block store we only need to record the height that it was saved at.
key := keyCommitted(ev)
@ -177,13 +286,6 @@ func (evpool *Pool) MarkEvidenceAsCommitted(height int64, evidence []types.Evide
if err := evpool.evidenceStore.Set(key, evBytes); err != nil {
evpool.logger.Error("Unable to add committed evidence", "err", err)
// if we can't move evidence to committed then don't remove the evidence from pending
continue
}
// if pending, remove from that bucket, remember not all evidence has been seen before
if evpool.IsPending(ev) {
evpool.removePendingEvidence(ev)
blockEvidenceMap[evMapKey(ev)] = struct{}{}
}
}
@ -191,21 +293,142 @@ func (evpool *Pool) MarkEvidenceAsCommitted(height int64, evidence []types.Evide
if len(blockEvidenceMap) != 0 {
evpool.removeEvidenceFromList(blockEvidenceMap)
}
return abciEvidence
}
// EvidenceFront goes to the first evidence in the clist
func (evpool *Pool) EvidenceFront() *clist.CElement {
return evpool.evidenceList.Front()
}
// EvidenceWaitChan is a channel that closes once the first evidence in the list is there. i.e Front is not nil
func (evpool *Pool) EvidenceWaitChan() <-chan struct{} {
return evpool.evidenceList.WaitChan()
}
// 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)
// SetLogger sets the Logger.
func (evpool *Pool) SetLogger(l log.Logger) {
evpool.logger = l
}
// IsEvidenceExpired checks whether evidence is past the maximum age where it can be used
func (evpool *Pool) IsEvidenceExpired(evidence types.Evidence) bool {
return evpool.IsExpired(evidence.Height(), evidence.Time())
// State returns the current state of the evpool.
func (evpool *Pool) State() sm.State {
evpool.mtx.Lock()
defer evpool.mtx.Unlock()
return evpool.state
}
//--------------------------------------------------------------------------
// Info is a wrapper around the evidence that the evidence pool receives with extensive
// information of what validators were malicious, the time of the attack and the total voting power
// This is saved as a form of cache so that the evidence pool can easily produce the ABCI Evidence
// needed to be sent to the application.
type info struct {
Evidence types.Evidence
Time time.Time
Validators []*types.Validator
TotalVotingPower int64
}
// ToProto encodes into protobuf
func (ei info) ToProto() (*evproto.Info, error) {
evpb, err := types.EvidenceToProto(ei.Evidence)
if err != nil {
return nil, err
}
valsProto := make([]*tmproto.Validator, len(ei.Validators))
for i := 0; i < len(ei.Validators); i++ {
valp, err := ei.Validators[i].ToProto()
if err != nil {
return nil, err
}
valsProto[i] = valp
}
return &evproto.Info{
Evidence: *evpb,
Time: ei.Time,
Validators: valsProto,
TotalVotingPower: ei.TotalVotingPower,
}, nil
}
// InfoFromProto decodes from protobuf into Info
func infoFromProto(proto *evproto.Info) (info, error) {
if proto == nil {
return info{}, errors.New("nil evidence info")
}
ev, err := types.EvidenceFromProto(&proto.Evidence)
if err != nil {
return info{}, err
}
vals := make([]*types.Validator, len(proto.Validators))
for i := 0; i < len(proto.Validators); i++ {
val, err := types.ValidatorFromProto(proto.Validators[i])
if err != nil {
return info{}, err
}
vals[i] = val
}
return info{
Evidence: ev,
Time: proto.Time,
Validators: vals,
TotalVotingPower: proto.TotalVotingPower,
}, nil
}
//--------------------------------------------------------------------------
// fastCheck leverages the fact that the evidence pool may have already verified the evidence to see if it can
// quickly conclude that the evidence is already valid.
func (evpool *Pool) fastCheck(ev types.Evidence) bool {
key := keyPending(ev)
if lcae, ok := ev.(*types.LightClientAttackEvidence); ok {
evBytes, err := evpool.evidenceStore.Get(key)
if evBytes == nil { // the evidence is not in the nodes pending list
return false
}
if err != nil {
evpool.logger.Error("Failed to load evidence", "err", err, "evidence", lcae)
return false
}
evInfo, err := bytesToInfo(evBytes)
if err != nil {
evpool.logger.Error("Failed to convert evidence from proto", "err", err, "evidence", lcae)
return false
}
// ensure that all the validators that the evidence pool have found to be malicious
// are present in the list of commit signatures in the conflicting block
OUTER:
for _, sig := range lcae.ConflictingBlock.Commit.Signatures {
for _, val := range evInfo.Validators {
if bytes.Equal(val.Address, sig.ValidatorAddress) {
continue OUTER
}
}
// a validator we know is malicious is not included in the commit
evpool.logger.Info("Fast check failed: a validator we know is malicious is not " +
"in the commit sigs. Reverting to full verification")
return false
}
return true
}
// for all other evidence the evidence pool just checks if it is already in the pending db
return evpool.isPending(ev)
}
// IsExpired checks whether evidence or a polc is expired by checking whether a height and time is older
// than set by the evidence consensus parameters
func (evpool *Pool) IsExpired(height int64, time time.Time) bool {
func (evpool *Pool) isExpired(height int64, time time.Time) bool {
var (
params = evpool.State().ConsensusParams.Evidence
ageDuration = evpool.State().LastBlockTime.Sub(time)
@ -216,7 +439,7 @@ func (evpool *Pool) IsExpired(height int64, time time.Time) bool {
}
// IsCommitted returns true if we have already seen this exact evidence and it is already marked as committed.
func (evpool *Pool) IsCommitted(evidence types.Evidence) bool {
func (evpool *Pool) isCommitted(evidence types.Evidence) bool {
key := keyCommitted(evidence)
ok, err := evpool.evidenceStore.Has(key)
if err != nil {
@ -226,7 +449,7 @@ func (evpool *Pool) IsCommitted(evidence types.Evidence) bool {
}
// IsPending checks whether the evidence is already pending. DB errors are passed to the logger.
func (evpool *Pool) IsPending(evidence types.Evidence) bool {
func (evpool *Pool) isPending(evidence types.Evidence) bool {
key := keyPending(evidence)
ok, err := evpool.evidenceStore.Has(key)
if err != nil {
@ -235,42 +458,25 @@ func (evpool *Pool) IsPending(evidence types.Evidence) bool {
return ok
}
// EvidenceFront goes to the first evidence in the clist
func (evpool *Pool) EvidenceFront() *clist.CElement {
return evpool.evidenceList.Front()
}
// EvidenceWaitChan is a channel that closes once the first evidence in the list is there. i.e Front is not nil
func (evpool *Pool) EvidenceWaitChan() <-chan struct{} {
return evpool.evidenceList.WaitChan()
}
// SetLogger sets the Logger.
func (evpool *Pool) SetLogger(l log.Logger) {
evpool.logger = l
}
// State returns the current state of the evpool.
func (evpool *Pool) State() sm.State {
evpool.mtx.Lock()
defer evpool.mtx.Unlock()
return evpool.state
}
func (evpool *Pool) addPendingEvidence(evidence types.Evidence) error {
evi, err := types.EvidenceToProto(evidence)
func (evpool *Pool) addPendingEvidence(evInfo *info) error {
evpb, err := evInfo.ToProto()
if err != nil {
return fmt.Errorf("unable to convert to proto, err: %w", err)
}
evBytes, err := proto.Marshal(evi)
evBytes, err := evpb.Marshal()
if err != nil {
return fmt.Errorf("unable to marshal evidence: %w", err)
}
key := keyPending(evidence)
key := keyPending(evInfo.Evidence)
return evpool.evidenceStore.Set(key, evBytes)
err = evpool.evidenceStore.Set(key, evBytes)
if err != nil {
return fmt.Errorf("can't persist evidence: %w", err)
}
atomic.AddUint32(&evpool.evidenceSize, 1)
return nil
}
func (evpool *Pool) removePendingEvidence(evidence types.Evidence) {
@ -278,6 +484,7 @@ func (evpool *Pool) removePendingEvidence(evidence types.Evidence) {
if err := evpool.evidenceStore.Delete(key); err != nil {
evpool.logger.Error("Unable to delete pending evidence", "err", err)
} else {
atomic.AddUint32(&evpool.evidenceSize, ^uint32(0))
evpool.logger.Info("Deleted pending evidence", "evidence", evidence)
}
}
@ -298,62 +505,47 @@ func (evpool *Pool) listEvidence(prefixKey byte, maxNum int64) ([]types.Evidence
}
count++
val := iter.Value()
var (
ev types.Evidence
evpb tmproto.Evidence
)
err := proto.Unmarshal(val, &evpb)
if err != nil {
return nil, err
}
ev, err = types.EvidenceFromProto(&evpb)
evInfo, err := bytesToInfo(iter.Value())
if err != nil {
return nil, err
}
evidence = append(evidence, ev)
evidence = append(evidence, evInfo.Evidence)
}
return evidence, nil
}
func (evpool *Pool) removeExpiredPendingEvidence() {
func (evpool *Pool) removeExpiredPendingEvidence() (int64, time.Time) {
iter, err := dbm.IteratePrefix(evpool.evidenceStore, []byte{baseKeyPending})
if err != nil {
evpool.logger.Error("Unable to iterate over pending evidence", "err", err)
return
return evpool.State().LastBlockHeight, evpool.State().LastBlockTime
}
defer iter.Close()
blockEvidenceMap := make(map[string]struct{})
for ; iter.Valid(); iter.Next() {
evBytes := iter.Value()
var (
ev types.Evidence
evpb tmproto.Evidence
)
err := proto.Unmarshal(evBytes, &evpb)
if err != nil {
evpool.logger.Error("Unable to unmarshal Evidence", "err", err)
continue
}
ev, err = types.EvidenceFromProto(&evpb)
evInfo, err := bytesToInfo(iter.Value())
if err != nil {
evpool.logger.Error("Error in transition evidence from protobuf", "err", err)
continue
}
if !evpool.IsExpired(ev.Height()-1, ev.Time()) {
if !evpool.isExpired(evInfo.Evidence.Height(), evInfo.Time) {
if len(blockEvidenceMap) != 0 {
evpool.removeEvidenceFromList(blockEvidenceMap)
}
return
// return the time with which this evidence will have expired so we know when to prune next
return evInfo.Evidence.Height() + evpool.State().ConsensusParams.Evidence.MaxAgeNumBlocks + 1,
evInfo.Time.Add(evpool.State().ConsensusParams.Evidence.MaxAgeDuration).Add(time.Second)
}
evpool.removePendingEvidence(ev)
blockEvidenceMap[evMapKey(ev)] = struct{}{}
evpool.removePendingEvidence(evInfo.Evidence)
blockEvidenceMap[evMapKey(evInfo.Evidence)] = struct{}{}
}
// We either have no pending evidence or all evidence has expired
if len(blockEvidenceMap) != 0 {
evpool.removeEvidenceFromList(blockEvidenceMap)
}
return evpool.State().LastBlockHeight, evpool.State().LastBlockTime
}
func (evpool *Pool) removeEvidenceFromList(
@ -375,6 +567,16 @@ func (evpool *Pool) updateState(state sm.State) {
evpool.state = state
}
func bytesToInfo(evBytes []byte) (info, error) {
var evpb evproto.Info
err := evpb.Unmarshal(evBytes)
if err != nil {
return info{}, err
}
return infoFromProto(&evpb)
}
func evMapKey(ev types.Evidence) string {
return string(ev.Hash())
}


+ 211
- 96
evidence/pool_test.go View File

@ -1,22 +1,27 @@
package evidence
package evidence_test
import (
"os"
"testing"
"time"
"github.com/gogo/protobuf/proto"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/require"
dbm "github.com/tendermint/tm-db"
abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/evidence"
"github.com/tendermint/tendermint/evidence/mocks"
"github.com/tendermint/tendermint/libs/log"
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
tmversion "github.com/tendermint/tendermint/proto/tendermint/version"
sm "github.com/tendermint/tendermint/state"
smmocks "github.com/tendermint/tendermint/state/mocks"
"github.com/tendermint/tendermint/store"
"github.com/tendermint/tendermint/types"
"github.com/tendermint/tendermint/version"
)
func TestMain(m *testing.M) {
@ -31,23 +36,28 @@ var defaultEvidenceTime = time.Date(2019, 1, 1, 0, 0, 0, 0, time.UTC)
func TestEvidencePoolBasic(t *testing.T) {
var (
val = types.NewMockPV()
height = int64(1)
stateStore = initializeValidatorState(val, height)
stateStore = &smmocks.Store{}
evidenceDB = dbm.NewMemDB()
blockStore = &mocks.BlockStore{}
)
valSet, privVals := types.RandValidatorSet(3, 10)
blockStore.On("LoadBlockMeta", mock.AnythingOfType("int64")).Return(
&types.BlockMeta{Header: types.Header{Time: defaultEvidenceTime}},
)
stateStore.On("LoadValidators", mock.AnythingOfType("int64")).Return(valSet, nil)
stateStore.On("Load").Return(createState(height+1, valSet), nil)
pool, err := NewPool(evidenceDB, stateStore, blockStore)
pool, err := evidence.NewPool(evidenceDB, stateStore, blockStore)
require.NoError(t, err)
// evidence not seen yet:
evidence := types.NewMockDuplicateVoteEvidenceWithValidator(height, defaultEvidenceTime, val, evidenceChainID)
assert.False(t, pool.IsCommitted(evidence))
evs := pool.PendingEvidence(10)
assert.Equal(t, 0, len(evs))
ev := types.NewMockDuplicateVoteEvidenceWithValidator(height, defaultEvidenceTime, privVals[0], evidenceChainID)
// good evidence
evAdded := make(chan struct{})
@ -57,7 +67,7 @@ func TestEvidencePoolBasic(t *testing.T) {
}()
// evidence seen but not yet committed:
assert.NoError(t, pool.AddEvidence(evidence))
assert.NoError(t, pool.AddEvidence(ev))
select {
case <-evAdded:
@ -65,27 +75,16 @@ func TestEvidencePoolBasic(t *testing.T) {
t.Fatal("evidence was not added to list after 5s")
}
assert.Equal(t, 1, pool.evidenceList.Len())
assert.False(t, pool.IsCommitted(evidence))
assert.True(t, pool.IsPending(evidence))
next := pool.EvidenceFront()
assert.Equal(t, ev, next.Value.(types.Evidence))
// test evidence is proposed
proposedEvidence := pool.AllPendingEvidence()
assert.Equal(t, proposedEvidence[0], evidence)
evs = pool.PendingEvidence(10)
assert.Equal(t, 1, len(evs))
proposedEvidence = pool.PendingEvidence(1)
assert.Equal(t, proposedEvidence[0], evidence)
// shouldn't be able to add evidence twice
assert.Error(t, pool.AddEvidence(ev))
assert.Equal(t, 1, len(pool.PendingEvidence(10)))
// 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())
// no evidence should be pending
proposedEvidence = pool.PendingEvidence(1)
assert.Empty(t, proposedEvidence)
}
// Tests inbound evidence for the right time and height
@ -102,12 +101,12 @@ func TestAddExpiredEvidence(t *testing.T) {
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: defaultEvidenceTime.Add(time.Duration(height) * time.Minute)}}
}
return &types.BlockMeta{Header: types.Header{Time: expiredEvidenceTime}}
})
pool, err := NewPool(evidenceDB, stateStore, blockStore)
pool, err := evidence.NewPool(evidenceDB, stateStore, blockStore)
require.NoError(t, err)
testCases := []struct {
@ -137,100 +136,204 @@ func TestAddExpiredEvidence(t *testing.T) {
}
}
func TestAddEvidenceFromConsensus(t *testing.T) {
var height int64 = 10
pool, val := defaultTestPool(height)
ev := types.NewMockDuplicateVoteEvidenceWithValidator(height, defaultEvidenceTime, val, evidenceChainID)
err := pool.AddEvidenceFromConsensus(ev, defaultEvidenceTime,
types.NewValidatorSet([]*types.Validator{val.ExtractIntoValidator(2)}))
assert.NoError(t, err)
next := pool.EvidenceFront()
assert.Equal(t, ev, next.Value.(types.Evidence))
// shouldn't be able to submit the same evidence twice
err = pool.AddEvidenceFromConsensus(ev, defaultEvidenceTime.Add(-1*time.Second),
types.NewValidatorSet([]*types.Validator{val.ExtractIntoValidator(3)}))
if assert.Error(t, err) {
assert.Equal(t, "evidence already verified and added", err.Error())
}
}
func TestEvidencePoolUpdate(t *testing.T) {
height := int64(21)
pool, val := defaultTestPool(height)
state := pool.State()
// create new block (no need to save it to blockStore)
evidence := types.NewMockDuplicateVoteEvidence(height, time.Now(), evidenceChainID)
prunedEv := types.NewMockDuplicateVoteEvidenceWithValidator(1, defaultEvidenceTime,
val, evidenceChainID)
err := pool.AddEvidence(prunedEv)
require.NoError(t, err)
ev := types.NewMockDuplicateVoteEvidenceWithValidator(height, defaultEvidenceTime, val, evidenceChainID)
lastCommit := makeCommit(height, val.PrivKey.PubKey().Address())
block := types.MakeBlock(height+1, []types.Tx{}, lastCommit, []types.Evidence{evidence})
block := types.MakeBlock(height+1, []types.Tx{}, lastCommit, []types.Evidence{ev})
// update state (partially)
state.LastBlockHeight = height + 1
state.LastBlockTime = defaultEvidenceTime.Add(22 * time.Minute)
err = pool.CheckEvidence(types.EvidenceList{ev})
require.NoError(t, err)
pool.Update(block, state)
byzVals := pool.ABCIEvidence(block.Height, block.Evidence.Evidence)
expectedByzVals := []abci.Evidence{
{
Type: abci.EvidenceType_DUPLICATE_VOTE,
Validator: types.TM2PB.Validator(val.ExtractIntoValidator(10)),
Height: height,
Time: defaultEvidenceTime.Add(time.Duration(height) * time.Minute),
TotalVotingPower: 10,
},
}
assert.Equal(t, expectedByzVals, byzVals)
assert.Equal(t, 1, len(pool.PendingEvidence(10)))
// a) Update marks evidence as committed
assert.True(t, pool.IsCommitted(evidence))
}
pool.Update(state)
func TestVerifyEvidenceCommittedEvidenceFails(t *testing.T) {
height := int64(1)
pool, _ := defaultTestPool(height)
committedEvidence := types.NewMockDuplicateVoteEvidence(height, time.Now(), evidenceChainID)
pool.MarkEvidenceAsCommitted(height, []types.Evidence{committedEvidence})
// a) Update marks evidence as committed so pending evidence should be empty
assert.Empty(t, pool.PendingEvidence(10))
err := pool.Verify(committedEvidence)
// b) If we try to check this evidence again it should fail because it has already been committed
err = pool.CheckEvidence(types.EvidenceList{ev})
if assert.Error(t, err) {
assert.Equal(t, "evidence was already committed", err.Error())
assert.Equal(t, "evidence was already committed", err.(*types.ErrInvalidEvidence).Reason.Error())
}
assert.Empty(t, pool.ABCIEvidence(height, []types.Evidence{}))
}
func TestVeriyEvidencePendingEvidencePasses(t *testing.T) {
var (
val = types.NewMockPV()
height = int64(1)
stateStore = initializeValidatorState(val, height)
blockStore = &mocks.BlockStore{}
)
func TestVerifyPendingEvidencePasses(t *testing.T) {
var height int64 = 1
pool, val := defaultTestPool(height)
ev := types.NewMockDuplicateVoteEvidenceWithValidator(height, defaultEvidenceTime, val, evidenceChainID)
err := pool.AddEvidence(ev)
require.NoError(t, err)
blockStore.On("LoadBlockMeta", mock.AnythingOfType("int64")).Return(
&types.BlockMeta{Header: types.Header{Time: defaultEvidenceTime}},
)
err = pool.CheckEvidence(types.EvidenceList{ev})
assert.NoError(t, err)
}
func TestVerifyDuplicatedEvidenceFails(t *testing.T) {
var height int64 = 1
pool, val := defaultTestPool(height)
ev := types.NewMockDuplicateVoteEvidenceWithValidator(height, defaultEvidenceTime, val, evidenceChainID)
err := pool.CheckEvidence(types.EvidenceList{ev, ev})
if assert.Error(t, err) {
assert.Equal(t, "duplicate evidence", err.(*types.ErrInvalidEvidence).Reason.Error())
}
}
pool, err := NewPool(dbm.NewMemDB(), stateStore, blockStore)
// check that
func TestCheckEvidenceWithLightClientAttack(t *testing.T) {
nValidators := 5
conflictingVals, conflictingPrivVals := types.RandValidatorSet(nValidators, 10)
trustedHeader := makeHeaderRandom(10)
conflictingHeader := makeHeaderRandom(10)
conflictingHeader.ValidatorsHash = conflictingVals.Hash()
trustedHeader.ValidatorsHash = conflictingHeader.ValidatorsHash
trustedHeader.NextValidatorsHash = conflictingHeader.NextValidatorsHash
trustedHeader.ConsensusHash = conflictingHeader.ConsensusHash
trustedHeader.AppHash = conflictingHeader.AppHash
trustedHeader.LastResultsHash = conflictingHeader.LastResultsHash
// for simplicity we are simulating a duplicate vote attack where all the validators in the
// conflictingVals set voted twice
blockID := makeBlockID(conflictingHeader.Hash(), 1000, []byte("partshash"))
voteSet := types.NewVoteSet(evidenceChainID, 10, 1, tmproto.SignedMsgType(2), conflictingVals)
commit, err := types.MakeCommit(blockID, 10, 1, voteSet, conflictingPrivVals, defaultEvidenceTime)
require.NoError(t, err)
evidence := types.NewMockDuplicateVoteEvidenceWithValidator(height, defaultEvidenceTime, val, evidenceChainID)
err = pool.AddEvidence(evidence)
ev := &types.LightClientAttackEvidence{
ConflictingBlock: &types.LightBlock{
SignedHeader: &types.SignedHeader{
Header: conflictingHeader,
Commit: commit,
},
ValidatorSet: conflictingVals,
},
CommonHeight: 10,
}
trustedBlockID := makeBlockID(trustedHeader.Hash(), 1000, []byte("partshash"))
trustedVoteSet := types.NewVoteSet(evidenceChainID, 10, 1, tmproto.SignedMsgType(2), conflictingVals)
trustedCommit, err := types.MakeCommit(trustedBlockID, 10, 1, trustedVoteSet, conflictingPrivVals, defaultEvidenceTime)
require.NoError(t, err)
err = pool.Verify(evidence)
state := sm.State{
LastBlockTime: defaultEvidenceTime.Add(1 * time.Minute),
LastBlockHeight: 11,
ConsensusParams: *types.DefaultConsensusParams(),
}
stateStore := &smmocks.Store{}
stateStore.On("LoadValidators", int64(10)).Return(conflictingVals, nil)
stateStore.On("Load").Return(state, nil)
blockStore := &mocks.BlockStore{}
blockStore.On("LoadBlockMeta", int64(10)).Return(&types.BlockMeta{Header: *trustedHeader})
blockStore.On("LoadBlockCommit", int64(10)).Return(trustedCommit)
pool, err := evidence.NewPool(dbm.NewMemDB(), stateStore, blockStore)
require.NoError(t, err)
pool.SetLogger(log.TestingLogger())
err = pool.AddEvidence(ev)
assert.NoError(t, err)
err = pool.CheckEvidence(types.EvidenceList{ev})
assert.NoError(t, err)
// take away the last signature -> there are less validators then what we have detected,
// hence we move to full verification where the evidence should still pass
commit.Signatures = append(commit.Signatures[:nValidators-1], types.NewCommitSigAbsent())
err = pool.CheckEvidence(types.EvidenceList{ev})
assert.NoError(t, err)
// take away the last two signatures -> should fail due to insufficient power
commit.Signatures = append(commit.Signatures[:nValidators-2], types.NewCommitSigAbsent(), types.NewCommitSigAbsent())
err = pool.CheckEvidence(types.EvidenceList{ev})
assert.Error(t, err)
}
func TestRecoverPendingEvidence(t *testing.T) {
var (
val = types.NewMockPV()
valAddr = val.PrivKey.PubKey().Address()
height = int64(30)
stateStore = initializeValidatorState(val, height)
evidenceDB = dbm.NewMemDB()
blockStoreDB = dbm.NewMemDB()
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)
)
height := int64(10)
expiredEvidenceTime := time.Date(2018, 1, 1, 0, 0, 0, 0, time.UTC)
val := types.NewMockPV()
valAddress := val.PrivKey.PubKey().Address()
evidenceDB := dbm.NewMemDB()
stateStore := initializeValidatorState(val, height)
state, err := stateStore.Load()
require.NoError(t, err)
blockStore := initializeBlockStore(blockStoreDB, state, valAddr)
// load good evidence
goodKey := keyPending(goodEvidence)
evi, err := types.EvidenceToProto(goodEvidence)
require.NoError(t, err)
goodEvidenceBytes, err := proto.Marshal(evi)
blockStore := initializeBlockStore(dbm.NewMemDB(), state, valAddress)
pool, err := evidence.NewPool(evidenceDB, stateStore, blockStore)
require.NoError(t, err)
_ = evidenceDB.Set(goodKey, goodEvidenceBytes)
// load expired evidence
expiredKey := keyPending(expiredEvidence)
eevi, err := types.EvidenceToProto(expiredEvidence)
pool.SetLogger(log.TestingLogger())
goodEvidence := types.NewMockDuplicateVoteEvidenceWithValidator(height,
defaultEvidenceTime, val, evidenceChainID)
expiredEvidence := types.NewMockDuplicateVoteEvidenceWithValidator(int64(1),
expiredEvidenceTime, val, evidenceChainID)
err = pool.AddEvidence(goodEvidence)
require.NoError(t, err)
expiredEvidenceBytes, err := proto.Marshal(eevi)
err = pool.AddEvidence(expiredEvidence)
require.NoError(t, err)
newStateStore := &smmocks.Store{}
newStateStore.On("Load").Return(sm.State{
LastBlockTime: defaultEvidenceTime.Add(49 * time.Hour),
LastBlockHeight: height + 12,
ConsensusParams: tmproto.ConsensusParams{
Block: tmproto.BlockParams{
MaxBytes: 22020096,
MaxGas: -1,
},
Evidence: tmproto.EvidenceParams{
MaxAgeNumBlocks: 20,
MaxAgeDuration: 1 * time.Hour,
MaxNum: 50,
},
},
}, nil)
newPool, err := evidence.NewPool(evidenceDB, newStateStore, blockStore)
assert.NoError(t, err)
assert.Equal(t, 1, len(newPool.PendingEvidence(10)))
next := newPool.EvidenceFront()
assert.Equal(t, goodEvidence, next.Value.(types.Evidence))
_ = evidenceDB.Set(expiredKey, expiredEvidenceBytes)
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))
}
func initializeStateFromValidatorSet(valSet *types.ValidatorSet, height int64) sm.Store {
@ -252,7 +355,7 @@ func initializeStateFromValidatorSet(valSet *types.ValidatorSet, height int64) s
},
Evidence: tmproto.EvidenceParams{
MaxAgeNumBlocks: 20,
MaxAgeDuration: 48 * time.Hour,
MaxAgeDuration: 20 * time.Minute,
MaxNum: 50,
},
},
@ -272,7 +375,7 @@ func initializeStateFromValidatorSet(valSet *types.ValidatorSet, height int64) s
func initializeValidatorState(privVal types.PrivValidator, height int64) sm.Store {
pubKey, _ := privVal.GetPubKey()
validator := &types.Validator{Address: pubKey.Address(), VotingPower: 0, PubKey: pubKey}
validator := &types.Validator{Address: pubKey.Address(), VotingPower: 10, PubKey: pubKey}
// create validator set and state
valSet := &types.ValidatorSet{
@ -292,7 +395,8 @@ func initializeBlockStore(db dbm.DB, state sm.State, valAddr []byte) *store.Bloc
lastCommit := makeCommit(i-1, valAddr)
block, _ := state.MakeBlock(i, []types.Tx{}, lastCommit, nil,
state.Validators.GetProposer().Address)
block.Header.Time = defaultEvidenceTime.Add(time.Duration(i) * time.Minute)
block.Header.Version = tmversion.Consensus{Block: version.BlockProtocol, App: 1}
const parts = 1
partSet := block.MakePartSet(parts)
@ -307,22 +411,33 @@ func makeCommit(height int64, valAddr []byte) *types.Commit {
commitSigs := []types.CommitSig{{
BlockIDFlag: types.BlockIDFlagCommit,
ValidatorAddress: valAddr,
Timestamp: time.Now(),
Timestamp: defaultEvidenceTime,
Signature: []byte("Signature"),
}}
return types.NewCommit(height, 0, types.BlockID{}, commitSigs)
}
func defaultTestPool(height int64) (*Pool, types.MockPV) {
func defaultTestPool(height int64) (*evidence.Pool, types.MockPV) {
val := types.NewMockPV()
valAddress := val.PrivKey.PubKey().Address()
evidenceDB := dbm.NewMemDB()
stateStore := initializeValidatorState(val, height)
state, _ := stateStore.Load()
blockStore := initializeBlockStore(dbm.NewMemDB(), state, valAddress)
pool, err := NewPool(evidenceDB, stateStore, blockStore)
pool, err := evidence.NewPool(evidenceDB, stateStore, blockStore)
if err != nil {
panic("test evidence pool could not be created")
}
pool.SetLogger(log.TestingLogger())
return pool, val
}
func createState(height int64, valSet *types.ValidatorSet) sm.State {
return sm.State{
ChainID: evidenceChainID,
LastBlockHeight: height,
LastBlockTime: defaultEvidenceTime,
Validators: valSet,
ConsensusParams: *types.DefaultConsensusParams(),
}
}

+ 4
- 11
evidence/reactor.go View File

@ -73,7 +73,7 @@ func (evR *Reactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) {
for _, ev := range evis {
err := evR.evpool.AddEvidence(ev)
switch err.(type) {
case *types.ErrEvidenceInvalid:
case *types.ErrInvalidEvidence:
evR.Logger.Error(err.Error())
// punish peer
evR.Switch.StopPeerForError(src, err)
@ -171,21 +171,15 @@ func (evR Reactor) checkSendEvidenceMessage(
// NOTE: We only send evidence to peers where
// peerHeight - maxAge < evidenceHeight < peerHeight
// and
// lastBlockTime - maxDuration < evidenceTime
var (
peerHeight = peerState.GetHeight()
params = evR.evpool.State().ConsensusParams.Evidence
ageDuration = evR.evpool.State().LastBlockTime.Sub(ev.Time())
peerHeight = peerState.GetHeight()
params = evR.evpool.State().ConsensusParams.Evidence
ageNumBlocks = peerHeight - evHeight
)
if peerHeight < evHeight { // peer is behind. sleep while he catches up
return nil, true
} else if ageNumBlocks > params.MaxAgeNumBlocks &&
ageDuration > params.MaxAgeDuration { // evidence is too old, skip
} else if ageNumBlocks > params.MaxAgeNumBlocks { // evidence is too old relative to the peer, skip
// NOTE: if evidence is too old for an honest peer, then we're behind and
// either it already got committed or it never will!
@ -194,7 +188,6 @@ func (evR Reactor) checkSendEvidenceMessage(
"evHeight", evHeight,
"maxAgeNumBlocks", params.MaxAgeNumBlocks,
"lastBlockTime", evR.evpool.State().LastBlockTime,
"evTime", ev.Time(),
"maxAgeDuration", params.MaxAgeDuration,
"peer", peer,
)


+ 29
- 25
evidence/reactor_test.go View File

@ -1,4 +1,4 @@
package evidence
package evidence_test
import (
"encoding/hex"
@ -17,6 +17,7 @@ import (
cfg "github.com/tendermint/tendermint/config"
"github.com/tendermint/tendermint/crypto"
"github.com/tendermint/tendermint/crypto/tmhash"
"github.com/tendermint/tendermint/evidence"
"github.com/tendermint/tendermint/evidence/mocks"
"github.com/tendermint/tendermint/libs/log"
"github.com/tendermint/tendermint/p2p"
@ -40,10 +41,12 @@ func evidenceLogger() log.Logger {
}
// connect N evidence reactors through N switches
func makeAndConnectReactors(config *cfg.Config, stateStores []sm.Store) []*Reactor {
func makeAndConnectReactorsAndPools(config *cfg.Config, stateStores []sm.Store) ([]*evidence.Reactor,
[]*evidence.Pool) {
N := len(stateStores)
reactors := make([]*Reactor, N)
reactors := make([]*evidence.Reactor, N)
pools := make([]*evidence.Pool, N)
logger := evidenceLogger()
evidenceTime := time.Date(2019, 1, 1, 0, 0, 0, 0, time.UTC)
@ -53,11 +56,12 @@ func makeAndConnectReactors(config *cfg.Config, stateStores []sm.Store) []*React
blockStore.On("LoadBlockMeta", mock.AnythingOfType("int64")).Return(
&types.BlockMeta{Header: types.Header{Time: evidenceTime}},
)
pool, err := NewPool(evidenceDB, stateStores[i], blockStore)
pool, err := evidence.NewPool(evidenceDB, stateStores[i], blockStore)
if err != nil {
panic(err)
}
reactors[i] = NewReactor(pool)
pools[i] = pool
reactors[i] = evidence.NewReactor(pool)
reactors[i].SetLogger(logger.With("validator", i))
}
@ -67,16 +71,16 @@ func makeAndConnectReactors(config *cfg.Config, stateStores []sm.Store) []*React
}, p2p.Connect2Switches)
return reactors
return reactors, pools
}
// wait for all evidence on all reactors
func waitForEvidence(t *testing.T, evs types.EvidenceList, reactors []*Reactor) {
func waitForEvidence(t *testing.T, evs types.EvidenceList, pools []*evidence.Pool) {
// wait for the evidence in all evpools
wg := new(sync.WaitGroup)
for i := 0; i < len(reactors); i++ {
for i := 0; i < len(pools); i++ {
wg.Add(1)
go _waitForEvidence(t, wg, evs, i, reactors)
go _waitForEvidence(t, wg, evs, i, pools)
}
done := make(chan struct{})
@ -98,15 +102,15 @@ func _waitForEvidence(
t *testing.T,
wg *sync.WaitGroup,
evs types.EvidenceList,
reactorIdx int,
reactors []*Reactor,
poolIdx int,
pools []*evidence.Pool,
) {
evpool := reactors[reactorIdx].evpool
for len(evpool.AllPendingEvidence()) != len(evs) {
evpool := pools[poolIdx]
for len(evpool.PendingEvidence(uint32(len(evs)))) != len(evs) {
time.Sleep(time.Millisecond * 100)
}
reapedEv := evpool.AllPendingEvidence()
reapedEv := evpool.PendingEvidence(uint32(len(evs)))
// put the reaped evidence in a map so we can quickly check we got everything
evMap := make(map[string]types.Evidence)
for _, e := range reapedEv {
@ -115,14 +119,14 @@ func _waitForEvidence(
for i, expectedEv := range evs {
gotEv := evMap[string(expectedEv.Hash())]
assert.Equal(t, expectedEv, gotEv,
fmt.Sprintf("evidence at index %d on reactor %d don't match: %v vs %v",
i, reactorIdx, expectedEv, gotEv))
fmt.Sprintf("evidence at index %d on pool %d don't match: %v vs %v",
i, poolIdx, expectedEv, gotEv))
}
wg.Done()
}
func sendEvidence(t *testing.T, evpool *Pool, val types.PrivValidator, n int) types.EvidenceList {
func sendEvidence(t *testing.T, evpool *evidence.Pool, val types.PrivValidator, n int) types.EvidenceList {
evList := make([]types.Evidence, n)
for i := 0; i < n; i++ {
ev := types.NewMockDuplicateVoteEvidenceWithValidator(int64(i+1),
@ -153,7 +157,7 @@ func TestReactorBroadcastEvidence(t *testing.T) {
}
// make reactors from statedb
reactors := makeAndConnectReactors(config, stateDBs)
reactors, pools := makeAndConnectReactorsAndPools(config, stateDBs)
// set the peer height on each reactor
for _, r := range reactors {
@ -165,8 +169,8 @@ func TestReactorBroadcastEvidence(t *testing.T) {
// send a bunch of valid evidence to the first reactor's evpool
// and wait for them all to be received in the others
evList := sendEvidence(t, reactors[0].evpool, val, numEvidence)
waitForEvidence(t, evList, reactors)
evList := sendEvidence(t, pools[0], val, numEvidence)
waitForEvidence(t, evList, pools)
}
type peerState struct {
@ -189,7 +193,7 @@ func TestReactorSelectiveBroadcast(t *testing.T) {
stateDB2 := initializeValidatorState(val, height2)
// make reactors from statedb
reactors := makeAndConnectReactors(config, []sm.Store{stateDB1, stateDB2})
reactors, pools := makeAndConnectReactorsAndPools(config, []sm.Store{stateDB1, stateDB2})
// set the peer height on each reactor
for _, r := range reactors {
@ -205,10 +209,10 @@ func TestReactorSelectiveBroadcast(t *testing.T) {
peer.Set(types.PeerStateKey, ps)
// send a bunch of valid evidence to the first reactor's evpool
evList := sendEvidence(t, reactors[0].evpool, val, numEvidence)
evList := sendEvidence(t, pools[0], val, numEvidence)
// only ones less than the peers height should make it through
waitForEvidence(t, evList[:numEvidence/2], reactors[1:2])
waitForEvidence(t, evList[:numEvidence/2], pools[1:2])
// peers should still be connected
peers := reactors[1].Switch.Peers().List()
@ -241,14 +245,14 @@ func exampleVote(t byte) *types.Vote {
// nolint:lll //ignore line length for tests
func TestEvidenceVectors(t *testing.T) {
dupl := types.NewDuplicateVoteEvidence(exampleVote(1), exampleVote(2), time.Date(2019, 10, 13, 16, 14, 44, 0, time.UTC))
dupl := types.NewDuplicateVoteEvidence(exampleVote(1), exampleVote(2))
testCases := []struct {
testName string
evidenceList []types.Evidence
expBytes string
}{
{"DuplicateVoteEvidence", []types.Evidence{dupl}, "0a81020afe010a79080210031802224a0a208b01023386c371778ecb6368573e539afc3cc860ec3a2f614e54fe5652f4fc80122608c0843d122072db3d959635dff1bb567bedaa70573392c5159666a3f8caf11e413aac52207a2a0b08b1d381d20510809dca6f32146af1f4111082efb388211bc72c55bcd61e9ac3d538d5bb031279080110031802224a0a208b01023386c371778ecb6368573e539afc3cc860ec3a2f614e54fe5652f4fc80122608c0843d122072db3d959635dff1bb567bedaa70573392c5159666a3f8caf11e413aac52207a2a0b08b1d381d20510809dca6f32146af1f4111082efb388211bc72c55bcd61e9ac3d538d5bb031a0608f49a8ded05"},
{"DuplicateVoteEvidence", []types.Evidence{dupl}, "0af9010af6010a79080210031802224a0a208b01023386c371778ecb6368573e539afc3cc860ec3a2f614e54fe5652f4fc80122608c0843d122072db3d959635dff1bb567bedaa70573392c5159666a3f8caf11e413aac52207a2a0b08b1d381d20510809dca6f32146af1f4111082efb388211bc72c55bcd61e9ac3d538d5bb031279080110031802224a0a208b01023386c371778ecb6368573e539afc3cc860ec3a2f614e54fe5652f4fc80122608c0843d122072db3d959635dff1bb567bedaa70573392c5159666a3f8caf11e413aac52207a2a0b08b1d381d20510809dca6f32146af1f4111082efb388211bc72c55bcd61e9ac3d538d5bb03"},
}
for _, tc := range testCases {


+ 1
- 0
evidence/services.go View File

@ -8,4 +8,5 @@ import (
type BlockStore interface {
LoadBlockMeta(height int64) *types.BlockMeta
LoadBlockCommit(height int64) *types.Commit
}

+ 254
- 44
evidence/verify.go View File

@ -1,79 +1,289 @@
package evidence
import (
"bytes"
"errors"
"fmt"
"time"
sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/light"
"github.com/tendermint/tendermint/types"
)
// VerifyEvidence verifies the evidence fully by checking:
// verify verifies the evidence fully by checking:
// - It has not already been committed
// - 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 sm.Store, blockStore BlockStore) error {
// - it is internally consistent with state
// - it was properly signed by the alleged equivocator and meets the individual evidence verification requirements
func (evpool *Pool) verify(evidence types.Evidence) (*info, error) {
var (
state = evpool.State()
height = state.LastBlockHeight
evidenceParams = state.ConsensusParams.Evidence
ageDuration = state.LastBlockTime.Sub(evidence.Time())
ageNumBlocks = height - evidence.Height()
header *types.Header
ageNumBlocks = height - evidence.Height()
)
// 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,
)
}
// check that the evidence isn't already committed
if evpool.isCommitted(evidence) {
return nil, errors.New("evidence was already committed")
}
// verify the time of the evidence
blockMeta := evpool.blockStore.LoadBlockMeta(evidence.Height())
if blockMeta == nil {
return nil, fmt.Errorf("don't have header at height #%d", evidence.Height())
}
evTime := blockMeta.Header.Time
ageDuration := state.LastBlockTime.Sub(evTime)
// check that the evidence hasn't expired
if ageDuration > evidenceParams.MaxAgeDuration && ageNumBlocks > evidenceParams.MaxAgeNumBlocks {
return fmt.Errorf(
return nil, 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(),
evTime,
height-evidenceParams.MaxAgeNumBlocks,
state.LastBlockTime.Add(evidenceParams.MaxAgeDuration),
)
}
valset, err := stateDB.LoadValidators(evidence.Height())
if err != nil {
return err
// apply the evidence-specific verification logic
switch ev := evidence.(type) {
case *types.DuplicateVoteEvidence:
valSet, err := evpool.stateDB.LoadValidators(evidence.Height())
if err != nil {
return nil, err
}
err = VerifyDuplicateVote(ev, state.ChainID, valSet)
if err != nil {
return nil, fmt.Errorf("verifying duplicate vote evidence: %w", err)
}
_, val := valSet.GetByAddress(ev.VoteA.ValidatorAddress)
return &info{
Evidence: evidence,
Time: evTime,
Validators: []*types.Validator{val}, // just a single validator for duplicate vote evidence
TotalVotingPower: valSet.TotalVotingPower(),
}, nil
case *types.LightClientAttackEvidence:
commonHeader, err := getSignedHeader(evpool.blockStore, evidence.Height())
if err != nil {
return nil, err
}
commonVals, err := evpool.stateDB.LoadValidators(evidence.Height())
if err != nil {
return nil, err
}
trustedHeader := commonHeader
// in the case of lunatic the trusted header is different to the common header
if evidence.Height() != ev.ConflictingBlock.Height {
trustedHeader, err = getSignedHeader(evpool.blockStore, ev.ConflictingBlock.Height)
if err != nil {
return nil, err
}
}
err = VerifyLightClientAttack(ev, commonHeader, trustedHeader, commonVals, state.LastBlockTime,
state.ConsensusParams.Evidence.MaxAgeDuration)
if err != nil {
return nil, err
}
// find out what type of attack this was and thus extract the malicious validators. Note in the case of an
// Amnesia attack we don't have any malicious validators.
validators, attackType := getMaliciousValidators(ev, commonVals, trustedHeader)
totalVotingPower := ev.ConflictingBlock.ValidatorSet.TotalVotingPower()
if attackType == lunaticType {
totalVotingPower = commonVals.TotalVotingPower()
}
return &info{
Evidence: evidence,
Time: evTime,
Validators: validators,
TotalVotingPower: totalVotingPower,
}, nil
default:
return nil, fmt.Errorf("unrecognized evidence type: %T", evidence)
}
}
addr := evidence.Address()
var val *types.Validator
// VerifyLightClientAttack verifies LightClientAttackEvidence against the state of the full node. This involves
// the following checks:
// - the common header from the full node has at least 1/3 voting power which is also present in
// the conflicting header's commit
// - the nodes trusted header at the same height as the conflicting header has a different hash
func VerifyLightClientAttack(e *types.LightClientAttackEvidence, commonHeader, trustedHeader *types.SignedHeader,
commonVals *types.ValidatorSet, now time.Time, trustPeriod time.Duration) error {
// In the case of lunatic attack we need to perform a single verification jump between the
// common header and the conflicting one
if commonHeader.Height != trustedHeader.Height {
err := light.Verify(commonHeader, commonVals, e.ConflictingBlock.SignedHeader, e.ConflictingBlock.ValidatorSet,
trustPeriod, now, 0*time.Second, light.DefaultTrustLevel)
if err != nil {
return fmt.Errorf("skipping verification from common to conflicting header failed: %w", err)
}
} else {
// in the case of equivocation and amnesia we expect some header hashes to be correctly derived
if isInvalidHeader(trustedHeader.Header, e.ConflictingBlock.Header) {
return errors.New("common height is the same as conflicting block height so expected the conflicting" +
" block to be correctly derived yet it wasn't")
}
// ensure that 2/3 of the validator set did vote for this block
if err := e.ConflictingBlock.ValidatorSet.VerifyCommitLight(trustedHeader.ChainID, e.ConflictingBlock.Commit.BlockID,
e.ConflictingBlock.Height, e.ConflictingBlock.Commit); err != nil {
return fmt.Errorf("invalid commit from conflicting block: %w", err)
}
}
// For all other types, expect evidence.Address to be a validator at height
// evidence.Height.
_, val = valset.GetByAddress(addr)
if bytes.Equal(trustedHeader.Hash(), e.ConflictingBlock.Hash()) {
return fmt.Errorf("trusted header hash matches the evidence conflicting header hash: %X",
trustedHeader.Hash())
}
return nil
}
// VerifyDuplicateVote verifies DuplicateVoteEvidence against the state of full node. This involves the
// following checks:
// - the validator is in the validator set at the height of the evidence
// - the height, round, type and validator address of the votes must be the same
// - the block ID's must be different
// - The signatures must both be valid
func VerifyDuplicateVote(e *types.DuplicateVoteEvidence, chainID string, valSet *types.ValidatorSet) error {
_, val := valSet.GetByAddress(e.VoteA.ValidatorAddress)
if val == nil {
return fmt.Errorf("address %X was not a validator at height %d", addr, evidence.Height())
return fmt.Errorf("address %X was not a validator at height %d", e.VoteA.ValidatorAddress, e.Height())
}
pubKey := val.PubKey
// H/R/S must be the same
if e.VoteA.Height != e.VoteB.Height ||
e.VoteA.Round != e.VoteB.Round ||
e.VoteA.Type != e.VoteB.Type {
return fmt.Errorf("h/r/s does not match: %d/%d/%v vs %d/%d/%v",
e.VoteA.Height, e.VoteA.Round, e.VoteA.Type,
e.VoteB.Height, e.VoteB.Round, e.VoteB.Type)
}
// 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 err := evidence.Verify(state.ChainID, val.PubKey); err != nil {
return err
// 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,
)
}
// 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(types.VoteSignBytes(chainID, va), e.VoteA.Signature) {
return fmt.Errorf("verifying VoteA: %w", types.ErrVoteInvalidSignature)
}
if !pubKey.VerifySignature(types.VoteSignBytes(chainID, vb), e.VoteB.Signature) {
return fmt.Errorf("verifying VoteB: %w", types.ErrVoteInvalidSignature)
}
return nil
}
func getSignedHeader(blockStore BlockStore, height int64) (*types.SignedHeader, error) {
blockMeta := blockStore.LoadBlockMeta(height)
if blockMeta == nil {
return nil, fmt.Errorf("don't have header at height #%d", height)
}
commit := blockStore.LoadBlockCommit(height)
if commit == nil {
return nil, fmt.Errorf("don't have commit at height #%d", height)
}
return &types.SignedHeader{
Header: &blockMeta.Header,
Commit: commit,
}, nil
}
// getMaliciousValidators finds out what style of attack LightClientAttackEvidence was and then works out who
// the malicious validators were and returns them.
func getMaliciousValidators(evidence *types.LightClientAttackEvidence, commonVals *types.ValidatorSet,
trusted *types.SignedHeader) ([]*types.Validator, lightClientAttackType) {
var validators []*types.Validator
// First check if the header is invalid. This means that it is a lunatic attack and therefore we take the
// validators who are in the commonVals and voted for the lunatic header
if isInvalidHeader(trusted.Header, evidence.ConflictingBlock.Header) {
for _, commitSig := range evidence.ConflictingBlock.Commit.Signatures {
if !commitSig.ForBlock() {
continue
}
_, val := commonVals.GetByAddress(commitSig.ValidatorAddress)
if val == nil {
// validator wasn't in the common validator set
continue
}
validators = append(validators, val)
}
return validators, lunaticType
// Next, check to see if it is an equivocation attack and both commits are in the same round. If this is the
// case then we take the validators from the conflicting light block validator set that voted in both headers.
} else if trusted.Commit.Round == evidence.ConflictingBlock.Commit.Round {
// validator hashes are the same therefore the indexing order of validators are the same and thus we
// only need a single loop to find the validators that voted twice.
for i := 0; i < len(evidence.ConflictingBlock.Commit.Signatures); i++ {
sigA := evidence.ConflictingBlock.Commit.Signatures[i]
if sigA.Absent() {
continue
}
sigB := trusted.Commit.Signatures[i]
if sigB.Absent() {
continue
}
_, val := evidence.ConflictingBlock.ValidatorSet.GetByAddress(sigA.ValidatorAddress)
validators = append(validators, val)
}
return validators, equivocationType
}
// if the rounds are different then this is an amnesia attack. Unfortunately, given the nature of the attack,
// we aren't able yet to deduce which are malicious validators and which are not hence we return an
// empty validator set.
return validators, amnesiaType
}
// isInvalidHeader takes a trusted header and matches it againt a conflicting header
// to determine whether the conflicting header was the product of a valid state transition
// or not. If it is then all the deterministic fields of the header should be the same.
// If not, it is an invalid header and constitutes a lunatic attack.
func isInvalidHeader(trusted, conflicting *types.Header) bool {
return !bytes.Equal(trusted.ValidatorsHash, conflicting.ValidatorsHash) ||
!bytes.Equal(trusted.NextValidatorsHash, conflicting.NextValidatorsHash) ||
!bytes.Equal(trusted.ConsensusHash, conflicting.ConsensusHash) ||
!bytes.Equal(trusted.AppHash, conflicting.AppHash) ||
!bytes.Equal(trusted.LastResultsHash, conflicting.LastResultsHash)
}
type lightClientAttackType int
const (
lunaticType lightClientAttackType = iota + 1
equivocationType
amnesiaType
)

+ 431
- 51
evidence/verify_test.go View File

@ -1,79 +1,459 @@
package evidence
package evidence_test
import (
"fmt"
"testing"
"time"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/require"
dbm "github.com/tendermint/tm-db"
abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/crypto"
"github.com/tendermint/tendermint/crypto/tmhash"
"github.com/tendermint/tendermint/evidence"
"github.com/tendermint/tendermint/evidence/mocks"
"github.com/tendermint/tendermint/libs/log"
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
tmversion "github.com/tendermint/tendermint/proto/tendermint/version"
sm "github.com/tendermint/tendermint/state"
smmocks "github.com/tendermint/tendermint/state/mocks"
"github.com/tendermint/tendermint/types"
"github.com/tendermint/tendermint/version"
)
func TestVerifyEvidenceWrongAddress(t *testing.T) {
var height int64 = 4
val := types.NewMockPV()
stateStore := initializeValidatorState(val, height)
state, err := stateStore.Load()
if err != nil {
t.Error(err)
func TestVerifyLightClientAttack_Lunatic(t *testing.T) {
commonVals, commonPrivVals := types.RandValidatorSet(2, 10)
newVal, newPrivVal := types.RandValidator(false, 9)
conflictingVals, err := types.ValidatorSetFromExistingValidators(append(commonVals.Validators, newVal))
require.NoError(t, err)
conflictingPrivVals := append(commonPrivVals, newPrivVal)
commonHeader := makeHeaderRandom(4)
commonHeader.Time = defaultEvidenceTime.Add(-1 * time.Hour)
trustedHeader := makeHeaderRandom(10)
conflictingHeader := makeHeaderRandom(10)
conflictingHeader.ValidatorsHash = conflictingVals.Hash()
// we are simulating a duplicate vote attack where all the validators in the conflictingVals set
// vote twice
blockID := makeBlockID(conflictingHeader.Hash(), 1000, []byte("partshash"))
voteSet := types.NewVoteSet(evidenceChainID, 10, 1, tmproto.SignedMsgType(2), conflictingVals)
commit, err := types.MakeCommit(blockID, 10, 1, voteSet, conflictingPrivVals, defaultEvidenceTime)
require.NoError(t, err)
ev := &types.LightClientAttackEvidence{
ConflictingBlock: &types.LightBlock{
SignedHeader: &types.SignedHeader{
Header: conflictingHeader,
Commit: commit,
},
ValidatorSet: conflictingVals,
},
CommonHeight: 4,
}
commonSignedHeader := &types.SignedHeader{
Header: commonHeader,
Commit: &types.Commit{},
}
trustedBlockID := makeBlockID(trustedHeader.Hash(), 1000, []byte("partshash"))
vals, privVals := types.RandValidatorSet(3, 8)
trustedVoteSet := types.NewVoteSet(evidenceChainID, 10, 1, tmproto.SignedMsgType(2), vals)
trustedCommit, err := types.MakeCommit(trustedBlockID, 10, 1, trustedVoteSet, privVals, defaultEvidenceTime)
require.NoError(t, err)
trustedSignedHeader := &types.SignedHeader{
Header: trustedHeader,
Commit: trustedCommit,
}
// good pass -> no error
err = evidence.VerifyLightClientAttack(ev, commonSignedHeader, trustedSignedHeader, commonVals,
defaultEvidenceTime.Add(1*time.Minute), 2*time.Hour)
assert.NoError(t, err)
// trusted and conflicting hashes are the same -> an error should be returned
err = evidence.VerifyLightClientAttack(ev, commonSignedHeader, ev.ConflictingBlock.SignedHeader, commonVals,
defaultEvidenceTime.Add(1*time.Minute), 2*time.Hour)
assert.Error(t, err)
state := sm.State{
LastBlockTime: defaultEvidenceTime.Add(1 * time.Minute),
LastBlockHeight: 11,
ConsensusParams: *types.DefaultConsensusParams(),
}
stateStore := &smmocks.Store{}
stateStore.On("LoadValidators", int64(4)).Return(commonVals, nil)
stateStore.On("Load").Return(state, nil)
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)
blockStore.On("LoadBlockMeta", int64(4)).Return(&types.BlockMeta{Header: *commonHeader})
blockStore.On("LoadBlockMeta", int64(10)).Return(&types.BlockMeta{Header: *trustedHeader})
blockStore.On("LoadBlockCommit", int64(4)).Return(commit)
blockStore.On("LoadBlockCommit", int64(10)).Return(trustedCommit)
pool, err := evidence.NewPool(dbm.NewMemDB(), stateStore, blockStore)
require.NoError(t, err)
pool.SetLogger(log.TestingLogger())
evList := types.EvidenceList{ev}
err = pool.CheckEvidence(evList)
assert.NoError(t, err)
pendingEvs := pool.PendingEvidence(2)
assert.Equal(t, 1, len(pendingEvs))
pubKey, err := newPrivVal.GetPubKey()
require.NoError(t, err)
lastCommit := makeCommit(state.LastBlockHeight, pubKey.Address())
block := types.MakeBlock(state.LastBlockHeight, []types.Tx{}, lastCommit, []types.Evidence{ev})
abciEv := pool.ABCIEvidence(block.Height, block.Evidence.Evidence)
expectedAbciEv := make([]abci.Evidence, len(commonVals.Validators))
// we expect evidence to be made for all validators in the common validator set
for idx, val := range commonVals.Validators {
ev := abci.Evidence{
Type: abci.EvidenceType_LIGHT_CLIENT_ATTACK,
Validator: types.TM2PB.Validator(val),
Height: commonHeader.Height,
Time: commonHeader.Time,
TotalVotingPower: commonVals.TotalVotingPower(),
}
expectedAbciEv[idx] = ev
}
assert.Equal(t, expectedAbciEv, abciEv)
}
func TestVerifyEvidenceExpiredEvidence(t *testing.T) {
var height int64 = 4
val := types.NewMockPV()
stateStore := initializeValidatorState(val, height)
state, err := stateStore.Load()
if err != nil {
t.Error(err)
func TestVerifyLightClientAttack_Equivocation(t *testing.T) {
conflictingVals, conflictingPrivVals := types.RandValidatorSet(5, 10)
trustedHeader := makeHeaderRandom(10)
conflictingHeader := makeHeaderRandom(10)
conflictingHeader.ValidatorsHash = conflictingVals.Hash()
trustedHeader.ValidatorsHash = conflictingHeader.ValidatorsHash
trustedHeader.NextValidatorsHash = conflictingHeader.NextValidatorsHash
trustedHeader.ConsensusHash = conflictingHeader.ConsensusHash
trustedHeader.AppHash = conflictingHeader.AppHash
trustedHeader.LastResultsHash = conflictingHeader.LastResultsHash
// we are simulating a duplicate vote attack where all the validators in the conflictingVals set
// except the last validator vote twice
blockID := makeBlockID(conflictingHeader.Hash(), 1000, []byte("partshash"))
voteSet := types.NewVoteSet(evidenceChainID, 10, 1, tmproto.SignedMsgType(2), conflictingVals)
commit, err := types.MakeCommit(blockID, 10, 1, voteSet, conflictingPrivVals[:4], defaultEvidenceTime)
require.NoError(t, err)
ev := &types.LightClientAttackEvidence{
ConflictingBlock: &types.LightBlock{
SignedHeader: &types.SignedHeader{
Header: conflictingHeader,
Commit: commit,
},
ValidatorSet: conflictingVals,
},
CommonHeight: 10,
}
state.ConsensusParams.Evidence.MaxAgeNumBlocks = 1
expiredEvidenceTime := time.Date(2018, 1, 1, 0, 0, 0, 0, time.UTC)
trustedBlockID := makeBlockID(trustedHeader.Hash(), 1000, []byte("partshash"))
trustedVoteSet := types.NewVoteSet(evidenceChainID, 10, 1, tmproto.SignedMsgType(2), conflictingVals)
trustedCommit, err := types.MakeCommit(trustedBlockID, 10, 1, trustedVoteSet, conflictingPrivVals, defaultEvidenceTime)
require.NoError(t, err)
trustedSignedHeader := &types.SignedHeader{
Header: trustedHeader,
Commit: trustedCommit,
}
// good pass -> no error
err = evidence.VerifyLightClientAttack(ev, trustedSignedHeader, trustedSignedHeader, nil,
defaultEvidenceTime.Add(1*time.Minute), 2*time.Hour)
assert.NoError(t, err)
// trusted and conflicting hashes are the same -> an error should be returned
err = evidence.VerifyLightClientAttack(ev, trustedSignedHeader, ev.ConflictingBlock.SignedHeader, nil,
defaultEvidenceTime.Add(1*time.Minute), 2*time.Hour)
assert.Error(t, err)
// conflicting header has different next validators hash which should have been correctly derived from
// the previous round
ev.ConflictingBlock.Header.NextValidatorsHash = crypto.CRandBytes(tmhash.Size)
err = evidence.VerifyLightClientAttack(ev, trustedSignedHeader, trustedSignedHeader, nil,
defaultEvidenceTime.Add(1*time.Minute), 2*time.Hour)
assert.Error(t, err)
// revert next validators hash
ev.ConflictingBlock.Header.NextValidatorsHash = trustedHeader.NextValidatorsHash
state := sm.State{
LastBlockTime: defaultEvidenceTime.Add(1 * time.Minute),
LastBlockHeight: 11,
ConsensusParams: *types.DefaultConsensusParams(),
}
stateStore := &smmocks.Store{}
stateStore.On("LoadValidators", int64(10)).Return(conflictingVals, nil)
stateStore.On("Load").Return(state, nil)
blockStore := &mocks.BlockStore{}
blockStore.On("LoadBlockMeta", mock.AnythingOfType("int64")).Return(
&types.BlockMeta{Header: types.Header{Time: expiredEvidenceTime}},
)
blockStore.On("LoadBlockMeta", int64(10)).Return(&types.BlockMeta{Header: *trustedHeader})
blockStore.On("LoadBlockCommit", int64(10)).Return(trustedCommit)
pool, err := evidence.NewPool(dbm.NewMemDB(), stateStore, blockStore)
require.NoError(t, err)
pool.SetLogger(log.TestingLogger())
evList := types.EvidenceList{ev}
err = pool.CheckEvidence(evList)
assert.NoError(t, err)
pendingEvs := pool.PendingEvidence(2)
assert.Equal(t, 1, len(pendingEvs))
pubKey, err := conflictingPrivVals[0].GetPubKey()
require.NoError(t, err)
lastCommit := makeCommit(state.LastBlockHeight, pubKey.Address())
block := types.MakeBlock(state.LastBlockHeight, []types.Tx{}, lastCommit, []types.Evidence{ev})
abciEv := pool.ABCIEvidence(block.Height, block.Evidence.Evidence)
expectedAbciEv := make([]abci.Evidence, len(conflictingVals.Validators)-1)
// we expect evidence to be made for all validators except the last one
for idx, val := range conflictingVals.Validators {
if idx == 4 { // skip the last validator
continue
}
ev := abci.Evidence{
Type: abci.EvidenceType_LIGHT_CLIENT_ATTACK,
Validator: types.TM2PB.Validator(val),
Height: ev.ConflictingBlock.Height,
Time: ev.ConflictingBlock.Time,
TotalVotingPower: ev.ConflictingBlock.ValidatorSet.TotalVotingPower(),
}
expectedAbciEv[idx] = ev
}
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)
assert.Equal(t, expectedAbciEv, abciEv)
}
func TestVerifyLightClientAttack_Amnesia(t *testing.T) {
conflictingVals, conflictingPrivVals := types.RandValidatorSet(5, 10)
conflictingHeader := makeHeaderRandom(10)
conflictingHeader.ValidatorsHash = conflictingVals.Hash()
trustedHeader := makeHeaderRandom(10)
trustedHeader.ValidatorsHash = conflictingHeader.ValidatorsHash
trustedHeader.NextValidatorsHash = conflictingHeader.NextValidatorsHash
trustedHeader.AppHash = conflictingHeader.AppHash
trustedHeader.ConsensusHash = conflictingHeader.ConsensusHash
trustedHeader.LastResultsHash = conflictingHeader.LastResultsHash
// we are simulating an amnesia attack where all the validators in the conflictingVals set
// except the last validator vote twice. However this time the commits are of different rounds.
blockID := makeBlockID(conflictingHeader.Hash(), 1000, []byte("partshash"))
voteSet := types.NewVoteSet(evidenceChainID, 10, 0, tmproto.SignedMsgType(2), conflictingVals)
commit, err := types.MakeCommit(blockID, 10, 0, voteSet, conflictingPrivVals, defaultEvidenceTime)
require.NoError(t, err)
ev := &types.LightClientAttackEvidence{
ConflictingBlock: &types.LightBlock{
SignedHeader: &types.SignedHeader{
Header: conflictingHeader,
Commit: commit,
},
ValidatorSet: conflictingVals,
},
CommonHeight: 10,
}
trustedBlockID := makeBlockID(trustedHeader.Hash(), 1000, []byte("partshash"))
trustedVoteSet := types.NewVoteSet(evidenceChainID, 10, 1, tmproto.SignedMsgType(2), conflictingVals)
trustedCommit, err := types.MakeCommit(trustedBlockID, 10, 1, trustedVoteSet, conflictingPrivVals, defaultEvidenceTime)
require.NoError(t, err)
trustedSignedHeader := &types.SignedHeader{
Header: trustedHeader,
Commit: trustedCommit,
}
// good pass -> no error
err = evidence.VerifyLightClientAttack(ev, trustedSignedHeader, trustedSignedHeader, nil,
defaultEvidenceTime.Add(1*time.Minute), 2*time.Hour)
assert.NoError(t, err)
// trusted and conflicting hashes are the same -> an error should be returned
err = evidence.VerifyLightClientAttack(ev, trustedSignedHeader, ev.ConflictingBlock.SignedHeader, nil,
defaultEvidenceTime.Add(1*time.Minute), 2*time.Hour)
assert.Error(t, err)
state := sm.State{
LastBlockTime: defaultEvidenceTime.Add(1 * time.Minute),
LastBlockHeight: 11,
ConsensusParams: *types.DefaultConsensusParams(),
}
stateStore := &smmocks.Store{}
stateStore.On("LoadValidators", int64(10)).Return(conflictingVals, nil)
stateStore.On("Load").Return(state, nil)
blockStore := &mocks.BlockStore{}
blockStore.On("LoadBlockMeta", int64(10)).Return(&types.BlockMeta{Header: *trustedHeader})
blockStore.On("LoadBlockCommit", int64(10)).Return(trustedCommit)
pool, err := evidence.NewPool(dbm.NewMemDB(), stateStore, blockStore)
require.NoError(t, err)
pool.SetLogger(log.TestingLogger())
evList := types.EvidenceList{ev}
err = pool.CheckEvidence(evList)
assert.NoError(t, err)
pendingEvs := pool.PendingEvidence(2)
assert.Equal(t, 1, len(pendingEvs))
pubKey, err := conflictingPrivVals[0].GetPubKey()
require.NoError(t, err)
lastCommit := makeCommit(state.LastBlockHeight, pubKey.Address())
block := types.MakeBlock(state.LastBlockHeight, []types.Tx{}, lastCommit, []types.Evidence{ev})
abciEv := pool.ABCIEvidence(block.Height, block.Evidence.Evidence)
// as we are unable to find out which subset of validators in the commit were malicious, no information
// is sent to the application. We expect the array to be empty
emptyEvidenceBlock := types.MakeBlock(state.LastBlockHeight, []types.Tx{}, lastCommit, []types.Evidence{})
expectedAbciEv := pool.ABCIEvidence(emptyEvidenceBlock.Height, emptyEvidenceBlock.Evidence.Evidence)
assert.Equal(t, expectedAbciEv, abciEv)
}
func TestVerifyEvidenceInvalidTime(t *testing.T) {
height := int64(4)
type voteData struct {
vote1 *types.Vote
vote2 *types.Vote
valid bool
}
func TestVerifyDuplicateVoteEvidence(t *testing.T) {
val := types.NewMockPV()
stateStore := initializeValidatorState(val, height)
state, err := stateStore.Load()
if err != nil {
t.Error(err)
val2 := types.NewMockPV()
valSet := types.NewValidatorSet([]*types.Validator{val.ExtractIntoValidator(1)})
blockID := makeBlockID([]byte("blockhash"), 1000, []byte("partshash"))
blockID2 := makeBlockID([]byte("blockhash2"), 1000, []byte("partshash"))
blockID3 := makeBlockID([]byte("blockhash"), 10000, []byte("partshash"))
blockID4 := makeBlockID([]byte("blockhash"), 10000, []byte("partshash2"))
const chainID = "mychain"
vote1 := makeVote(t, val, chainID, 0, 10, 2, 1, blockID, defaultEvidenceTime)
v1 := vote1.ToProto()
err := val.SignVote(chainID, v1)
require.NoError(t, err)
badVote := makeVote(t, val, chainID, 0, 10, 2, 1, blockID, defaultEvidenceTime)
bv := badVote.ToProto()
err = val2.SignVote(chainID, bv)
require.NoError(t, err)
vote1.Signature = v1.Signature
badVote.Signature = bv.Signature
cases := []voteData{
{vote1, makeVote(t, val, chainID, 0, 10, 2, 1, blockID2, defaultEvidenceTime), true}, // different block ids
{vote1, makeVote(t, val, chainID, 0, 10, 2, 1, blockID3, defaultEvidenceTime), true},
{vote1, makeVote(t, val, chainID, 0, 10, 2, 1, blockID4, defaultEvidenceTime), true},
{vote1, makeVote(t, val, chainID, 0, 10, 2, 1, blockID, defaultEvidenceTime), false}, // wrong block id
{vote1, makeVote(t, val, "mychain2", 0, 10, 2, 1, blockID2, defaultEvidenceTime), false}, // wrong chain id
{vote1, makeVote(t, val, chainID, 0, 11, 2, 1, blockID2, defaultEvidenceTime), false}, // wrong height
{vote1, makeVote(t, val, chainID, 0, 10, 3, 1, blockID2, defaultEvidenceTime), false}, // wrong round
{vote1, makeVote(t, val, chainID, 0, 10, 2, 2, blockID2, defaultEvidenceTime), false}, // wrong step
{vote1, makeVote(t, val2, chainID, 0, 10, 2, 1, blockID2, defaultEvidenceTime), false}, // wrong validator
// a different vote time doesn't matter
{vote1, makeVote(t, val, chainID, 0, 10, 2, 1, blockID2, time.Date(2020, 1, 1, 0, 0, 0, 0, time.UTC)), true},
{vote1, badVote, false}, // signed by wrong key
}
require.NoError(t, err)
for _, c := range cases {
ev := &types.DuplicateVoteEvidence{
VoteA: c.vote1,
VoteB: c.vote2,
}
if c.valid {
assert.Nil(t, evidence.VerifyDuplicateVote(ev, chainID, valSet), "evidence should be valid")
} else {
assert.NotNil(t, evidence.VerifyDuplicateVote(ev, chainID, valSet), "evidence should be invalid")
}
}
goodEv := types.NewMockDuplicateVoteEvidenceWithValidator(10, defaultEvidenceTime, val, chainID)
state := sm.State{
ChainID: chainID,
LastBlockTime: defaultEvidenceTime.Add(1 * time.Minute),
LastBlockHeight: 11,
ConsensusParams: *types.DefaultConsensusParams(),
}
stateStore := &smmocks.Store{}
stateStore.On("LoadValidators", int64(10)).Return(valSet, nil)
stateStore.On("Load").Return(state, nil)
blockStore := &mocks.BlockStore{}
blockStore.On("LoadBlockMeta", mock.AnythingOfType("int64")).Return(
&types.BlockMeta{Header: types.Header{Time: defaultEvidenceTime}},
)
blockStore.On("LoadBlockMeta", int64(10)).Return(&types.BlockMeta{Header: types.Header{Time: defaultEvidenceTime}})
pool, err := evidence.NewPool(dbm.NewMemDB(), stateStore, blockStore)
require.NoError(t, err)
evList := types.EvidenceList{goodEv}
err = pool.CheckEvidence(evList)
assert.NoError(t, err)
}
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 makeVote(
t *testing.T, val types.PrivValidator, chainID string, valIndex int32, height int64,
round int32, step int, blockID types.BlockID, time time.Time) *types.Vote {
pubKey, err := val.GetPubKey()
require.NoError(t, err)
v := &types.Vote{
ValidatorAddress: pubKey.Address(),
ValidatorIndex: valIndex,
Height: height,
Round: round,
Type: tmproto.SignedMsgType(step),
BlockID: blockID,
Timestamp: time,
}
vpb := v.ToProto()
err = val.SignVote(chainID, vpb)
if err != nil {
panic(err)
}
v.Signature = vpb.Signature
return v
}
func makeHeaderRandom(height int64) *types.Header {
return &types.Header{
Version: tmversion.Consensus{Block: version.BlockProtocol, App: 1},
ChainID: evidenceChainID,
Height: height,
Time: defaultEvidenceTime,
LastBlockID: makeBlockID([]byte("headerhash"), 1000, []byte("partshash")),
LastCommitHash: crypto.CRandBytes(tmhash.Size),
DataHash: crypto.CRandBytes(tmhash.Size),
ValidatorsHash: crypto.CRandBytes(tmhash.Size),
NextValidatorsHash: crypto.CRandBytes(tmhash.Size),
ConsensusHash: crypto.CRandBytes(tmhash.Size),
AppHash: crypto.CRandBytes(tmhash.Size),
LastResultsHash: crypto.CRandBytes(tmhash.Size),
EvidenceHash: crypto.CRandBytes(tmhash.Size),
ProposerAddress: crypto.CRandBytes(crypto.AddressSize),
}
}
func makeBlockID(hash []byte, partSetSize uint32, partSetHash []byte) types.BlockID {
var (
h = make([]byte, tmhash.Size)
psH = make([]byte, tmhash.Size)
)
copy(h, hash)
copy(psH, partSetHash)
return types.BlockID{
Hash: h,
PartSetHeader: types.PartSetHeader{
Total: partSetSize,
Hash: psH,
},
}
}

+ 46
- 118
light/client.go View File

@ -402,6 +402,37 @@ func (c *Client) compareWithLatestHeight(height int64) (int64, error) {
return height, nil
}
// Update attempts to advance the state by downloading the latest light
// block and verifying it. It returns a new light block on a successful
// update. Otherwise, it returns nil (plus an error, if any).
func (c *Client) Update(now time.Time) (*types.LightBlock, error) {
lastTrustedHeight, err := c.LastTrustedHeight()
if err != nil {
return nil, fmt.Errorf("can't get last trusted height: %w", err)
}
if lastTrustedHeight == -1 {
// no light blocks yet => wait
return nil, nil
}
latestBlock, err := c.lightBlockFromPrimary(0)
if err != nil {
return nil, err
}
if latestBlock.Height > lastTrustedHeight {
err = c.verifyLightBlock(latestBlock, now)
if err != nil {
return nil, err
}
c.logger.Info("Advanced to new state", "height", latestBlock.Height, "hash", hash2str(latestBlock.Hash()))
return latestBlock, nil
}
return nil, nil
}
// VerifyLightBlockAtHeight fetches the light block at the given height
// and calls verifyLightBlock. It returns the block immediately if it exists in
// the trustedStore (no verification is needed).
@ -579,7 +610,7 @@ func (c *Client) verifySequential(
"newHeight", interimBlock.Height,
"newHash", hash2str(interimBlock.Hash()))
err = VerifyAdjacent(c.chainID, verifiedBlock.SignedHeader, interimBlock.SignedHeader, interimBlock.ValidatorSet,
err = VerifyAdjacent(verifiedBlock.SignedHeader, interimBlock.SignedHeader, interimBlock.ValidatorSet,
c.trustingPeriod, now, c.maxClockDrift)
if err != nil {
err := ErrVerificationFailed{From: verifiedBlock.Height, To: interimBlock.Height, Reason: err}
@ -644,13 +675,14 @@ func (c *Client) verifySkipping(
source provider.Provider,
trustedBlock *types.LightBlock,
newLightBlock *types.LightBlock,
now time.Time) error {
now time.Time) ([]*types.LightBlock, error) {
var (
blockCache = []*types.LightBlock{newLightBlock}
depth = 0
verifiedBlock = trustedBlock
trace = []*types.LightBlock{trustedBlock}
)
for {
@ -660,13 +692,14 @@ func (c *Client) verifySkipping(
"newHeight", blockCache[depth].Height,
"newHash", hash2str(blockCache[depth].Hash()))
err := Verify(c.chainID, verifiedBlock.SignedHeader, verifiedBlock.ValidatorSet, blockCache[depth].SignedHeader,
err := Verify(verifiedBlock.SignedHeader, verifiedBlock.ValidatorSet, blockCache[depth].SignedHeader,
blockCache[depth].ValidatorSet, c.trustingPeriod, now, c.maxClockDrift, c.trustLevel)
switch err.(type) {
case nil:
// Have we verified the last header
if depth == 0 {
return nil
trace = append(trace, newLightBlock)
return trace, nil
}
// If not, update the lower bound to the previous upper bound
verifiedBlock = blockCache[depth]
@ -674,35 +707,36 @@ func (c *Client) verifySkipping(
blockCache = blockCache[:depth]
// Reset the cache depth so that we start from the upper bound again
depth = 0
// add verifiedBlock to the trace
trace = append(trace, verifiedBlock)
case ErrNewValSetCantBeTrusted:
// do add another header to the end of the cache
if depth == len(blockCache)-1 {
pivotHeight := verifiedBlock.Height + (blockCache[depth].Height-verifiedBlock.
Height)*verifySkippingNumerator/verifySkippingDenominator
interimBlock, err := source.LightBlock(pivotHeight)
if err != nil {
return ErrVerificationFailed{From: verifiedBlock.Height, To: pivotHeight, Reason: err}
interimBlock, providerErr := source.LightBlock(pivotHeight)
if providerErr != nil {
return nil, ErrVerificationFailed{From: verifiedBlock.Height, To: pivotHeight, Reason: providerErr}
}
blockCache = append(blockCache, interimBlock)
}
depth++
default:
return ErrVerificationFailed{From: verifiedBlock.Height, To: blockCache[depth].Height, Reason: err}
return nil, ErrVerificationFailed{From: verifiedBlock.Height, To: blockCache[depth].Height, Reason: err}
}
}
}
// verifySkippingAgainstPrimary does verifySkipping plus it compares new header with
// witnesses and replaces primary if it does not respond after
// MaxRetryAttempts.
// witnesses and replaces primary if it sends the light client an invalid header
func (c *Client) verifySkippingAgainstPrimary(
trustedBlock *types.LightBlock,
newLightBlock *types.LightBlock,
now time.Time) error {
err := c.verifySkipping(c.primary, trustedBlock, newLightBlock, now)
trace, err := c.verifySkipping(c.primary, trustedBlock, newLightBlock, now)
switch errors.Unwrap(err).(type) {
case ErrInvalidHeader:
@ -746,7 +780,7 @@ func (c *Client) verifySkippingAgainstPrimary(
//
// CORRECTNESS ASSUMPTION: there's at least 1 correct full node
// (primary or one of the witnesses).
if cmpErr := c.compareNewHeaderWithWitnesses(newLightBlock, now); cmpErr != nil {
if cmpErr := c.detectDivergence(trace, now); cmpErr != nil {
return cmpErr
}
default:
@ -894,81 +928,6 @@ func (c *Client) backwards(
return nil
}
// compare header with all witnesses provided.
func (c *Client) compareNewHeaderWithWitnesses(l *types.LightBlock, now time.Time) error {
c.providerMutex.Lock()
defer c.providerMutex.Unlock()
// 1. Make sure AT LEAST ONE witness returns the same header.
var headerMatched bool
if len(c.witnesses) == 0 {
return errNoWitnesses{}
}
// launch one goroutine per witness
errc := make(chan error, len(c.witnesses))
for i, witness := range c.witnesses {
go c.compareNewHeaderWithWitness(errc, l, witness, i, now)
}
witnessesToRemove := make([]int, 0)
// handle errors as they come
for i := 0; i < cap(errc); i++ {
err := <-errc
switch e := err.(type) {
case nil: // at least one header matched
headerMatched = true
case errBadWitness:
c.logger.Info("Requested light block from bad witness", "witness", c.witnesses[e.WitnessIndex], "err", err)
// if witness sent us invalid header / vals, remove it
if e.Code == invalidLightBlock {
c.logger.Info("Witness sent us invalid header / vals -> removing it", "witness", c.witnesses[e.WitnessIndex])
witnessesToRemove = append(witnessesToRemove, e.WitnessIndex)
}
default:
c.logger.Info("Requested light block from witness but got error", "error", err)
}
}
for _, idx := range witnessesToRemove {
c.removeWitness(idx)
}
if headerMatched {
return nil
}
return errors.New("awaiting response from all witnesses exceeded dropout time")
}
func (c *Client) compareNewHeaderWithWitness(errc chan error, l *types.LightBlock,
witness provider.Provider, witnessIndex int, now time.Time) {
altBlock, err := witness.LightBlock(l.Height)
if err != nil {
if _, ok := err.(provider.ErrBadLightBlock); ok {
errc <- errBadWitness{Reason: err, Code: invalidLightBlock, WitnessIndex: witnessIndex}
} else {
errc <- err
}
// if not a bad light block, then the witness has either not responded or
// doesn't have the block -> we ignore
return
}
if !bytes.Equal(l.Hash(), altBlock.Hash()) {
if bsErr := c.verifySkipping(witness, c.latestTrustedBlock, altBlock, now); bsErr != nil {
errc <- errBadWitness{Reason: bsErr, Code: invalidLightBlock, WitnessIndex: witnessIndex}
return
}
}
errc <- nil
}
// NOTE: requires a providerMutex locked.
func (c *Client) removeWitness(idx int) {
switch len(c.witnesses) {
@ -982,37 +941,6 @@ func (c *Client) removeWitness(idx int) {
}
}
// Update attempts to advance the state by downloading the latest light
// block and verifying it. It returns a new light block on a successful
// update. Otherwise, it returns nil (plus an error, if any).
func (c *Client) Update(now time.Time) (*types.LightBlock, error) {
lastTrustedHeight, err := c.LastTrustedHeight()
if err != nil {
return nil, fmt.Errorf("can't get last trusted height: %w", err)
}
if lastTrustedHeight == -1 {
// no light blocks yet => wait
return nil, nil
}
latestBlock, err := c.lightBlockFromPrimary(0)
if err != nil {
return nil, err
}
if latestBlock.Height > lastTrustedHeight {
err = c.verifyLightBlock(latestBlock, now)
if err != nil {
return nil, err
}
c.logger.Info("Advanced to new state", "height", latestBlock.Height, "hash", hash2str(latestBlock.Hash()))
return latestBlock, nil
}
return nil, nil
}
// replaceProvider takes the first alternative provider and promotes it as the
// primary provider.
func (c *Client) replacePrimaryProvider() error {


+ 1
- 1
light/client_benchmark_test.go View File

@ -21,7 +21,7 @@ import (
//
// Remember that none of these benchmarks account for network latency.
var (
benchmarkFullNode = mockp.New(GenMockNode(chainID, 1000, 100, 1, bTime))
benchmarkFullNode = mockp.New(genMockNode(chainID, 1000, 100, 1, bTime))
genesisBlock, _ = benchmarkFullNode.LightBlock(1)
)


+ 30
- 6
light/client_test.go View File

@ -61,7 +61,7 @@ var (
valSet,
)
deadNode = mockp.NewDeadMock(chainID)
largeFullNode = mockp.New(GenMockNode(chainID, 10, 3, 0, bTime))
largeFullNode = mockp.New(genMockNode(chainID, 10, 3, 0, bTime))
)
func TestValidateTrustOptions(t *testing.T) {
@ -118,6 +118,7 @@ func TestMock(t *testing.T) {
func TestClient_SequentialVerification(t *testing.T) {
newKeys := genPrivKeys(4)
newVals := newKeys.ToValidators(10, 1)
differentVals, _ := types.RandValidatorSet(10, 100)
testCases := []struct {
name string
@ -146,6 +147,26 @@ func TestClient_SequentialVerification(t *testing.T) {
true,
false,
},
{
"bad: no first signed header",
map[int64]*types.SignedHeader{},
map[int64]*types.ValidatorSet{
1: differentVals,
},
true,
true,
},
{
"bad: different first validator set",
map[int64]*types.SignedHeader{
1: h1,
},
map[int64]*types.ValidatorSet{
1: differentVals,
},
true,
true,
},
{
"bad: 1/3 signed interim header",
map[int64]*types.SignedHeader{
@ -356,15 +377,15 @@ func TestClient_SkippingVerification(t *testing.T) {
// start from a large light block to make sure that the pivot height doesn't select a height outside
// the appropriate range
func TestClientLargeBisectionVerification(t *testing.T) {
veryLargeFullNode := mockp.New(GenMockNode(chainID, 100, 3, 1, bTime))
l1, err := veryLargeFullNode.LightBlock(90)
veryLargeFullNode := mockp.New(genMockNode(chainID, 100, 3, 0, bTime))
trustedLightBlock, err := veryLargeFullNode.LightBlock(5)
require.NoError(t, err)
c, err := light.NewClient(
chainID,
light.TrustOptions{
Period: 4 * time.Hour,
Height: l1.Height,
Hash: l1.Hash(),
Height: trustedLightBlock.Height,
Hash: trustedLightBlock.Hash(),
},
veryLargeFullNode,
[]provider.Provider{veryLargeFullNode},
@ -896,6 +917,9 @@ func TestClientRemovesWitnessIfItSendsUsIncorrectHeader(t *testing.T) {
},
)
lb1, _ := badProvider1.LightBlock(2)
require.NotEqual(t, lb1.Hash(), l1.Hash())
c, err := light.NewClient(
chainID,
trustOptions,
@ -919,7 +943,7 @@ func TestClientRemovesWitnessIfItSendsUsIncorrectHeader(t *testing.T) {
// remaining witnesses don't have light block -> error
_, err = c.VerifyLightBlockAtHeight(3, bTime.Add(2*time.Hour))
if assert.Error(t, err) {
assert.Equal(t, "awaiting response from all witnesses exceeded dropout time", err.Error())
assert.Equal(t, light.ErrFailedHeaderCrossReferencing, err)
}
// witness does not have a light block -> left in the list
assert.EqualValues(t, 1, len(c.Witnesses()))


+ 247
- 0
light/detector.go View File

@ -0,0 +1,247 @@
package light
import (
"bytes"
"errors"
"fmt"
"time"
"github.com/tendermint/tendermint/light/provider"
"github.com/tendermint/tendermint/types"
)
// The detector component of the light client detect and handles attacks on the light client.
// More info here:
// tendermint/docs/architecture/adr-047-handling-evidence-from-light-client.md
// detectDivergence is a second wall of defense for the light client and is used
// only in the case of skipping verification which employs the trust level mechanism.
//
// It takes the target verified header and compares it with the headers of a set of
// witness providers that the light client is connected to. If a conflicting header
// is returned it verifies and examines the conflicting header against the verified
// trace that was produced from the primary. If successful it produces two sets of evidence
// and sends them to the opposite provider before halting.
//
// If there are no conflictinge headers, the light client deems the verified target header
// trusted and saves it to the trusted store.
func (c *Client) detectDivergence(primaryTrace []*types.LightBlock, now time.Time) error {
if primaryTrace == nil || len(primaryTrace) < 2 {
return errors.New("nil or single block primary trace")
}
var (
headerMatched bool
lastVerifiedHeader = primaryTrace[len(primaryTrace)-1].SignedHeader
witnessesToRemove = make([]int, 0)
)
c.logger.Info("Running detector against trace", "endBlockHeight", lastVerifiedHeader.Height,
"endBlockHash", lastVerifiedHeader.Hash, "length", len(primaryTrace))
c.providerMutex.Lock()
defer c.providerMutex.Unlock()
if len(c.witnesses) == 0 {
return errNoWitnesses{}
}
// launch one goroutine per witness to retrieve the light block of the target height
// and compare it with the header from the primary
errc := make(chan error, len(c.witnesses))
for i, witness := range c.witnesses {
go c.compareNewHeaderWithWitness(errc, lastVerifiedHeader, witness, i)
}
// handle errors from the header comparisons as they come in
for i := 0; i < cap(errc); i++ {
err := <-errc
switch e := err.(type) {
case nil: // at least one header matched
headerMatched = true
case errConflictingHeaders:
// We have conflicting headers. This could possibly imply an attack on the light client.
// First we need to verify the witness's header using the same skipping verification and then we
// need to find the point that the headers diverge and examine this for any evidence of an attack.
//
// We combine these actions together, verifying the witnesses headers and outputting the trace
// which captures the bifurcation point and if successful provides the information to create
supportingWitness := c.witnesses[e.WitnessIndex]
witnessTrace, primaryBlock, err := c.examineConflictingHeaderAgainstTrace(primaryTrace, e.Block.SignedHeader,
supportingWitness, now)
if err != nil {
c.logger.Info("Error validating witness's divergent header", "witness", supportingWitness, "err", err)
witnessesToRemove = append(witnessesToRemove, e.WitnessIndex)
continue
}
// if this is an equivocation or amnesia attack, i.e. the validator sets are the same, then we
// return the height of the conflicting block else if it is a lunatic attack and the validator sets
// are not the same then we send the height of the common header.
commonHeight := primaryBlock.Height
if isInvalidHeader(witnessTrace[len(witnessTrace)-1].Header, primaryBlock.Header) {
// height of the common header
commonHeight = witnessTrace[0].Height
}
// We are suspecting that the primary is faulty, hence we hold the witness as the source of truth
// and generate evidence against the primary that we can send to the witness
ev := &types.LightClientAttackEvidence{
ConflictingBlock: primaryBlock,
CommonHeight: commonHeight, // the first block in the bisection is common to both providers
}
c.logger.Error("Attack detected. Sending evidence againt primary by witness", "ev", ev,
"primary", c.primary, "witness", supportingWitness)
c.sendEvidence(ev, supportingWitness)
// This may not be valid because the witness itself is at fault. So now we reverse it, examining the
// trace provided by the witness and holding the primary as the source of truth. Note: primary may not
// respond but this is okay as we will halt anyway.
primaryTrace, witnessBlock, err := c.examineConflictingHeaderAgainstTrace(witnessTrace, primaryBlock.SignedHeader,
c.primary, now)
if err != nil {
c.logger.Info("Error validating primary's divergent header", "primary", c.primary, "err", err)
continue
}
// if this is an equivocation or amnesia attack, i.e. the validator sets are the same, then we
// return the height of the conflicting block else if it is a lunatic attack and the validator sets
// are not the same then we send the height of the common header.
commonHeight = primaryBlock.Height
if isInvalidHeader(primaryTrace[len(primaryTrace)-1].Header, witnessBlock.Header) {
// height of the common header
commonHeight = primaryTrace[0].Height
}
// We now use the primary trace to create evidence against the witness and send it to the primary
ev = &types.LightClientAttackEvidence{
ConflictingBlock: witnessBlock,
CommonHeight: commonHeight, // the first block in the bisection is common to both providers
}
c.logger.Error("Sending evidence against witness by primary", "ev", ev,
"primary", c.primary, "witness", supportingWitness)
c.sendEvidence(ev, c.primary)
// We return the error and don't process anymore witnesses
return e
case errBadWitness:
c.logger.Info("Witness returned an error during header comparison", "witness", c.witnesses[e.WitnessIndex],
"err", err)
// if witness sent us an invalid header, then remove it. If it didn't respond or couldn't find the block, then we
// ignore it and move on to the next witness
if _, ok := e.Reason.(provider.ErrBadLightBlock); ok {
c.logger.Info("Witness sent us invalid header / vals -> removing it", "witness", c.witnesses[e.WitnessIndex])
witnessesToRemove = append(witnessesToRemove, e.WitnessIndex)
}
}
}
for _, idx := range witnessesToRemove {
c.removeWitness(idx)
}
// 1. If we had at least one witness that returned the same header then we
// conclude that we can trust the header
if headerMatched {
return nil
}
// 2. ELse all witnesses have either not responded, don't have the block or sent invalid blocks.
return ErrFailedHeaderCrossReferencing
}
// compareNewHeaderWithWitness takes the verified header from the primary and compares it with a
// header from a specified witness. The function can return one of three errors:
//
// 1: errConflictingHeaders -> there may have been an attack on this light client
// 2: errBadWitness -> the witness has either not responded, doesn't have the header or has given us an invalid one
// Note: In the case of an invalid header we remove the witness
// 3: nil -> the hashes of the two headers match
func (c *Client) compareNewHeaderWithWitness(errc chan error, h *types.SignedHeader,
witness provider.Provider, witnessIndex int) {
lightBlock, err := witness.LightBlock(h.Height)
if err != nil {
errc <- errBadWitness{Reason: err, WitnessIndex: witnessIndex}
return
}
if !bytes.Equal(h.Hash(), lightBlock.Hash()) {
errc <- errConflictingHeaders{Block: lightBlock, WitnessIndex: witnessIndex}
}
c.logger.Info("Matching header received by witness", "height", h.Height, "witness", witnessIndex)
errc <- nil
}
// sendEvidence sends evidence to a provider on a best effort basis.
func (c *Client) sendEvidence(ev *types.LightClientAttackEvidence, receiver provider.Provider) {
err := receiver.ReportEvidence(ev)
if err != nil {
c.logger.Error("Failed to report evidence to provider", "ev", ev, "provider", receiver)
}
}
// examineConflictingHeaderAgainstTrace takes a trace from one provider and a divergent header that
// it has received from another and preforms verifySkipping at the heights of each of the intermediate
// headers in the trace until it reaches the divergentHeader. 1 of 2 things can happen.
//
// 1. The light client verifies a header that is different to the intermediate header in the trace. This
// is the bifurcation point and the light client can create evidence from it
// 2. The source stops responding, doesn't have the block or sends an invalid header in which case we
// return the error and remove the witness
func (c *Client) examineConflictingHeaderAgainstTrace(
trace []*types.LightBlock,
divergentHeader *types.SignedHeader,
source provider.Provider, now time.Time) ([]*types.LightBlock, *types.LightBlock, error) {
var previouslyVerifiedBlock *types.LightBlock
for idx, traceBlock := range trace {
// The first block in the trace MUST be the same to the light block that the source produces
// else we cannot continue with verification.
sourceBlock, err := source.LightBlock(traceBlock.Height)
if err != nil {
return nil, nil, err
}
if idx == 0 {
if shash, thash := sourceBlock.Hash(), traceBlock.Hash(); !bytes.Equal(shash, thash) {
return nil, nil, fmt.Errorf("trusted block is different to the source's first block (%X = %X)",
thash, shash)
}
previouslyVerifiedBlock = sourceBlock
continue
}
// we check that the source provider can verify a block at the same height of the
// intermediate height
trace, err := c.verifySkipping(source, previouslyVerifiedBlock, sourceBlock, now)
if err != nil {
return nil, nil, fmt.Errorf("verifySkipping of conflicting header failed: %w", err)
}
// check if the headers verified by the source has diverged from the trace
if shash, thash := sourceBlock.Hash(), traceBlock.Hash(); !bytes.Equal(shash, thash) {
// Bifurcation point found!
return trace, traceBlock, nil
}
// headers are still the same. update the previouslyVerifiedBlock
previouslyVerifiedBlock = sourceBlock
}
// We have reached the end of the trace without observing a divergence. The last header is thus different
// from the divergent header that the source originally sent us, then we return an error.
return nil, nil, fmt.Errorf("source provided different header to the original header it provided (%X != %X)",
previouslyVerifiedBlock.Hash(), divergentHeader.Hash())
}
// isInvalidHeader takes a trusted header and matches it againt a conflicting header
// to determine whether the conflicting header was the product of a valid state transition
// or not. If it is then all the deterministic fields of the header should be the same.
// If not, it is an invalid header and constitutes a lunatic attack.
func isInvalidHeader(trusted, conflicting *types.Header) bool {
return !bytes.Equal(trusted.ValidatorsHash, conflicting.ValidatorsHash) ||
!bytes.Equal(trusted.NextValidatorsHash, conflicting.NextValidatorsHash) ||
!bytes.Equal(trusted.ConsensusHash, conflicting.ConsensusHash) ||
!bytes.Equal(trusted.AppHash, conflicting.AppHash) ||
!bytes.Equal(trusted.LastResultsHash, conflicting.LastResultsHash)
}

+ 210
- 0
light/detector_test.go View File

@ -0,0 +1,210 @@
package light_test
import (
"testing"
"time"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
dbm "github.com/tendermint/tm-db"
"github.com/tendermint/tendermint/libs/log"
"github.com/tendermint/tendermint/light"
"github.com/tendermint/tendermint/light/provider"
mockp "github.com/tendermint/tendermint/light/provider/mock"
dbs "github.com/tendermint/tendermint/light/store/db"
"github.com/tendermint/tendermint/types"
)
func TestLightClientAttackEvidence_Lunatic(t *testing.T) {
// primary performs a lunatic attack
var (
latestHeight = int64(10)
valSize = 5
divergenceHeight = int64(6)
primaryHeaders = make(map[int64]*types.SignedHeader, latestHeight)
primaryValidators = make(map[int64]*types.ValidatorSet, latestHeight)
)
witnessHeaders, witnessValidators, chainKeys := genMockNodeWithKeys(chainID, latestHeight, valSize, 2, bTime)
witness := mockp.New(chainID, witnessHeaders, witnessValidators)
forgedKeys := chainKeys[divergenceHeight-1].ChangeKeys(3) // we change 3 out of the 5 validators (still 2/5 remain)
forgedVals := forgedKeys.ToValidators(2, 0)
for height := int64(1); height <= latestHeight; height++ {
if height < divergenceHeight {
primaryHeaders[height] = witnessHeaders[height]
primaryValidators[height] = witnessValidators[height]
continue
}
primaryHeaders[height] = forgedKeys.GenSignedHeader(chainID, height, bTime.Add(time.Duration(height)*time.Minute),
nil, forgedVals, forgedVals, hash("app_hash"), hash("cons_hash"), hash("results_hash"), 0, len(forgedKeys))
primaryValidators[height] = forgedVals
}
primary := mockp.New(chainID, primaryHeaders, primaryValidators)
c, err := light.NewClient(
chainID,
light.TrustOptions{
Period: 4 * time.Hour,
Height: 1,
Hash: primaryHeaders[1].Hash(),
},
primary,
[]provider.Provider{witness},
dbs.New(dbm.NewMemDB(), chainID),
light.Logger(log.TestingLogger()),
light.MaxRetryAttempts(1),
)
require.NoError(t, err)
// Check verification returns an error.
_, err = c.VerifyLightBlockAtHeight(10, bTime.Add(1*time.Hour))
if assert.Error(t, err) {
assert.Contains(t, err.Error(), "does not match primary")
}
// Check evidence was sent to both full nodes.
evAgainstPrimary := &types.LightClientAttackEvidence{
// after the divergence height the valset doesn't change so we expect the evidence to be for height 10
ConflictingBlock: &types.LightBlock{
SignedHeader: primaryHeaders[10],
ValidatorSet: primaryValidators[10],
},
CommonHeight: 4,
}
assert.True(t, witness.HasEvidence(evAgainstPrimary))
evAgainstWitness := &types.LightClientAttackEvidence{
// when forming evidence against witness we learn that the canonical chain continued to change validator sets
// hence the conflicting block is at 7
ConflictingBlock: &types.LightBlock{
SignedHeader: witnessHeaders[7],
ValidatorSet: witnessValidators[7],
},
CommonHeight: 4,
}
assert.True(t, primary.HasEvidence(evAgainstWitness))
}
func TestLightClientAttackEvidence_Equivocation(t *testing.T) {
// primary performs an equivocation attack
var (
latestHeight = int64(10)
valSize = 5
divergenceHeight = int64(6)
primaryHeaders = make(map[int64]*types.SignedHeader, latestHeight)
primaryValidators = make(map[int64]*types.ValidatorSet, latestHeight)
)
// validators don't change in this network (however we still use a map just for convenience)
witnessHeaders, witnessValidators, chainKeys := genMockNodeWithKeys(chainID, latestHeight+2, valSize, 2, bTime)
witness := mockp.New(chainID, witnessHeaders, witnessValidators)
for height := int64(1); height <= latestHeight; height++ {
if height < divergenceHeight {
primaryHeaders[height] = witnessHeaders[height]
primaryValidators[height] = witnessValidators[height]
continue
}
// we don't have a network partition so we will make 4/5 (greater than 2/3) malicious and vote again for
// a different block (which we do by adding txs)
primaryHeaders[height] = chainKeys[height].GenSignedHeader(chainID, height,
bTime.Add(time.Duration(height)*time.Minute), []types.Tx{[]byte("abcd")},
witnessValidators[height], witnessValidators[height+1], hash("app_hash"),
hash("cons_hash"), hash("results_hash"), 0, len(chainKeys[height])-1)
primaryValidators[height] = witnessValidators[height]
}
primary := mockp.New(chainID, primaryHeaders, primaryValidators)
c, err := light.NewClient(
chainID,
light.TrustOptions{
Period: 4 * time.Hour,
Height: 1,
Hash: primaryHeaders[1].Hash(),
},
primary,
[]provider.Provider{witness},
dbs.New(dbm.NewMemDB(), chainID),
light.Logger(log.TestingLogger()),
light.MaxRetryAttempts(1),
)
require.NoError(t, err)
// Check verification returns an error.
_, err = c.VerifyLightBlockAtHeight(10, bTime.Add(1*time.Hour))
if assert.Error(t, err) {
assert.Contains(t, err.Error(), "does not match primary")
}
// Check evidence was sent to both full nodes.
// Common height should be set to the height of the divergent header in the instance
// of an equivocation attack and the validator sets are the same as what the witness has
evAgainstPrimary := &types.LightClientAttackEvidence{
ConflictingBlock: &types.LightBlock{
SignedHeader: primaryHeaders[divergenceHeight],
ValidatorSet: primaryValidators[divergenceHeight],
},
CommonHeight: divergenceHeight,
}
assert.True(t, witness.HasEvidence(evAgainstPrimary))
evAgainstWitness := &types.LightClientAttackEvidence{
ConflictingBlock: &types.LightBlock{
SignedHeader: witnessHeaders[divergenceHeight],
ValidatorSet: witnessValidators[divergenceHeight],
},
CommonHeight: divergenceHeight,
}
assert.True(t, primary.HasEvidence(evAgainstWitness))
}
func TestClientDivergentTraces(t *testing.T) {
primary := mockp.New(genMockNode(chainID, 10, 5, 2, bTime))
firstBlock, err := primary.LightBlock(1)
require.NoError(t, err)
witness := mockp.New(genMockNode(chainID, 10, 5, 2, bTime))
c, err := light.NewClient(
chainID,
light.TrustOptions{
Height: 1,
Hash: firstBlock.Hash(),
Period: 4 * time.Hour,
},
primary,
[]provider.Provider{witness},
dbs.New(dbm.NewMemDB(), chainID),
light.Logger(log.TestingLogger()),
light.MaxRetryAttempts(1),
)
require.NoError(t, err)
// 1. Different nodes therefore a divergent header is produced but the
// light client can't verify it because it has a different trusted header.
_, err = c.VerifyLightBlockAtHeight(10, bTime.Add(1*time.Hour))
assert.Error(t, err)
assert.Equal(t, 0, len(c.Witnesses()))
// 2. Two out of three nodes don't respond but the third has a header that matches
// verification should be successful and all the witnesses should remain
c, err = light.NewClient(
chainID,
light.TrustOptions{
Height: 1,
Hash: firstBlock.Hash(),
Period: 4 * time.Hour,
},
primary,
[]provider.Provider{deadNode, deadNode, primary},
dbs.New(dbm.NewMemDB(), chainID),
light.Logger(log.TestingLogger()),
light.MaxRetryAttempts(1),
)
require.NoError(t, err)
_, err = c.VerifyLightBlockAtHeight(10, bTime.Add(1*time.Hour))
assert.NoError(t, err)
assert.Equal(t, 3, len(c.Witnesses()))
}

+ 22
- 16
light/errors.go View File

@ -1,6 +1,7 @@
package light
import (
"errors"
"fmt"
"time"
@ -39,6 +40,12 @@ func (e ErrInvalidHeader) Error() string {
return fmt.Sprintf("invalid header: %v", e.Reason)
}
// ErrFailedHeaderCrossReferencing is returned when the detector was not able to cross reference the header
// with any of the connected witnesses.
var ErrFailedHeaderCrossReferencing = errors.New("all witnesses have either not responded, don't have the " +
" blocks or sent invalid blocks. You should look to change your witnesses" +
" or review the light client's logs for more information")
// ErrVerificationFailed means either sequential or skipping verification has
// failed to verify from header #1 to header #2 due to some reason.
type ErrVerificationFailed struct {
@ -58,6 +65,20 @@ func (e ErrVerificationFailed) Error() string {
e.From, e.To, e.Reason)
}
// ----------------------------- INTERNAL ERRORS ---------------------------------
// ErrConflictingHeaders is thrown when two conflicting headers are discovered.
type errConflictingHeaders struct {
Block *types.LightBlock
WitnessIndex int
}
func (e errConflictingHeaders) Error() string {
return fmt.Sprintf(
"header hash (%X) from witness (%d) does not match primary",
e.Block.Hash(), e.WitnessIndex)
}
// errNoWitnesses means that there are not enough witnesses connected to
// continue running the light client.
type errNoWitnesses struct{}
@ -66,28 +87,13 @@ func (e errNoWitnesses) Error() string {
return "no witnesses connected. please reset light client"
}
type badWitnessCode int
const (
noResponse badWitnessCode = iota + 1
invalidLightBlock
)
// errBadWitness is returned when the witness either does not respond or
// responds with an invalid header.
type errBadWitness struct {
Reason error
Code badWitnessCode
WitnessIndex int
}
func (e errBadWitness) Error() string {
switch e.Code {
case noResponse:
return fmt.Sprintf("failed to get a header/vals from witness: %v", e.Reason)
case invalidLightBlock:
return fmt.Sprintf("witness sent us an invalid light block: %v", e.Reason)
default:
return fmt.Sprintf("unknown code: %d", e.Code)
}
return fmt.Sprintf("Witness %d returned error: %s", e.WitnessIndex, e.Reason.Error())
}

+ 29
- 15
light/helpers_test.go View File

@ -73,15 +73,12 @@ func (pkz privKeys) ToValidators(init, inc int64) *types.ValidatorSet {
}
// signHeader properly signs the header with all keys from first to last exclusive.
func (pkz privKeys) signHeader(header *types.Header, first, last int) *types.Commit {
func (pkz privKeys) signHeader(header *types.Header, valSet *types.ValidatorSet, first, last int) *types.Commit {
commitSigs := make([]types.CommitSig, len(pkz))
for i := 0; i < len(pkz); i++ {
commitSigs[i] = types.NewCommitSigAbsent()
}
// We need this list to keep the ordering.
vset := pkz.ToValidators(1, 1)
blockID := types.BlockID{
Hash: header.Hash(),
PartSetHeader: types.PartSetHeader{Total: 1, Hash: crypto.CRandBytes(32)},
@ -89,7 +86,7 @@ func (pkz privKeys) signHeader(header *types.Header, first, last int) *types.Com
// Fill in the votes we want.
for i := first; i < last && i < len(pkz); i++ {
vote := makeVote(header, vset, pkz[i], blockID)
vote := makeVote(header, valSet, pkz[i], blockID)
commitSigs[vote.ValidatorIndex] = vote.CommitSig()
}
@ -151,7 +148,7 @@ func (pkz privKeys) GenSignedHeader(chainID string, height int64, bTime time.Tim
header := genHeader(chainID, height, bTime, txs, valset, nextValset, appHash, consHash, resHash)
return &types.SignedHeader{
Header: header,
Commit: pkz.signHeader(header, first, last),
Commit: pkz.signHeader(header, valset, first, last),
}
}
@ -164,7 +161,7 @@ func (pkz privKeys) GenSignedHeaderLastBlockID(chainID string, height int64, bTi
header.LastBlockID = lastBlockID
return &types.SignedHeader{
Header: header,
Commit: pkz.signHeader(header, first, last),
Commit: pkz.signHeader(header, valset, first, last),
}
}
@ -176,19 +173,20 @@ func (pkz privKeys) ChangeKeys(delta int) privKeys {
// Generates the header and validator set to create a full entire mock node with blocks to height (
// blockSize) and with variation in validator sets. BlockIntervals are in per minute.
// NOTE: Expected to have a large validator set size ~ 100 validators.
func GenMockNode(
func genMockNodeWithKeys(
chainID string,
blockSize int64,
valSize int,
valVariation float32,
bTime time.Time) (
string,
map[int64]*types.SignedHeader,
map[int64]*types.ValidatorSet) {
map[int64]*types.ValidatorSet,
map[int64]privKeys) {
var (
headers = make(map[int64]*types.SignedHeader, blockSize)
valset = make(map[int64]*types.ValidatorSet, blockSize)
valset = make(map[int64]*types.ValidatorSet, blockSize+1)
keymap = make(map[int64]privKeys, blockSize+1)
keys = genPrivKeys(valSize)
totalVariation = valVariation
valVariationInt int
@ -198,14 +196,16 @@ func GenMockNode(
valVariationInt = int(totalVariation)
totalVariation = -float32(valVariationInt)
newKeys = keys.ChangeKeys(valVariationInt)
keymap[1] = keys
keymap[2] = newKeys
// genesis header and vals
lastHeader := keys.GenSignedHeader(chainID, 1, bTime.Add(1*time.Minute), nil,
keys.ToValidators(2, 2), newKeys.ToValidators(2, 2), hash("app_hash"), hash("cons_hash"),
keys.ToValidators(2, 0), newKeys.ToValidators(2, 0), hash("app_hash"), hash("cons_hash"),
hash("results_hash"), 0, len(keys))
currentHeader := lastHeader
headers[1] = currentHeader
valset[1] = keys.ToValidators(2, 2)
valset[1] = keys.ToValidators(2, 0)
keys = newKeys
for height := int64(2); height <= blockSize; height++ {
@ -215,14 +215,28 @@ func GenMockNode(
newKeys = keys.ChangeKeys(valVariationInt)
currentHeader = keys.GenSignedHeaderLastBlockID(chainID, height, bTime.Add(time.Duration(height)*time.Minute),
nil,
keys.ToValidators(2, 2), newKeys.ToValidators(2, 2), hash("app_hash"), hash("cons_hash"),
keys.ToValidators(2, 0), newKeys.ToValidators(2, 0), hash("app_hash"), hash("cons_hash"),
hash("results_hash"), 0, len(keys), types.BlockID{Hash: lastHeader.Hash()})
headers[height] = currentHeader
valset[height] = keys.ToValidators(2, 2)
valset[height] = keys.ToValidators(2, 0)
lastHeader = currentHeader
keys = newKeys
keymap[height+1] = keys
}
return headers, valset, keymap
}
func genMockNode(
chainID string,
blockSize int64,
valSize int,
valVariation float32,
bTime time.Time) (
string,
map[int64]*types.SignedHeader,
map[int64]*types.ValidatorSet) {
headers, valset, _ := genMockNodeWithKeys(chainID, blockSize, valSize, valVariation, bTime)
return chainID, headers, valset
}


+ 8
- 14
light/verifier.go View File

@ -30,7 +30,6 @@ var (
// maxClockDrift defines how much untrustedHeader.Time can drift into the
// future.
func VerifyNonAdjacent(
chainID string,
trustedHeader *types.SignedHeader, // height=X
trustedVals *types.ValidatorSet, // height=X or height=X+1
untrustedHeader *types.SignedHeader, // height=Y
@ -49,7 +48,6 @@ func VerifyNonAdjacent(
}
if err := verifyNewHeaderAndVals(
chainID,
untrustedHeader, untrustedVals,
trustedHeader,
now, maxClockDrift); err != nil {
@ -57,7 +55,7 @@ func VerifyNonAdjacent(
}
// Ensure that +`trustLevel` (default 1/3) or more of last trusted validators signed correctly.
err := trustedVals.VerifyCommitLightTrusting(chainID, untrustedHeader.Commit, trustLevel)
err := trustedVals.VerifyCommitLightTrusting(trustedHeader.ChainID, untrustedHeader.Commit, trustLevel)
if err != nil {
switch e := err.(type) {
case types.ErrNotEnoughVotingPowerSigned:
@ -72,8 +70,8 @@ func VerifyNonAdjacent(
// NOTE: this should always be the last check because untrustedVals can be
// intentionally made very large to DOS the light client. not the case for
// VerifyAdjacent, where validator set is known in advance.
if err := untrustedVals.VerifyCommitLight(chainID, untrustedHeader.Commit.BlockID, untrustedHeader.Height,
untrustedHeader.Commit); err != nil {
if err := untrustedVals.VerifyCommitLight(trustedHeader.ChainID, untrustedHeader.Commit.BlockID,
untrustedHeader.Height, untrustedHeader.Commit); err != nil {
return ErrInvalidHeader{err}
}
@ -93,7 +91,6 @@ func VerifyNonAdjacent(
// maxClockDrift defines how much untrustedHeader.Time can drift into the
// future.
func VerifyAdjacent(
chainID string,
trustedHeader *types.SignedHeader, // height=X
untrustedHeader *types.SignedHeader, // height=X+1
untrustedVals *types.ValidatorSet, // height=X+1
@ -110,7 +107,6 @@ func VerifyAdjacent(
}
if err := verifyNewHeaderAndVals(
chainID,
untrustedHeader, untrustedVals,
trustedHeader,
now, maxClockDrift); err != nil {
@ -127,8 +123,8 @@ func VerifyAdjacent(
}
// Ensure that +2/3 of new validators signed correctly.
if err := untrustedVals.VerifyCommitLight(chainID, untrustedHeader.Commit.BlockID, untrustedHeader.Height,
untrustedHeader.Commit); err != nil {
if err := untrustedVals.VerifyCommitLight(trustedHeader.ChainID, untrustedHeader.Commit.BlockID,
untrustedHeader.Height, untrustedHeader.Commit); err != nil {
return ErrInvalidHeader{err}
}
@ -137,7 +133,6 @@ func VerifyAdjacent(
// Verify combines both VerifyAdjacent and VerifyNonAdjacent functions.
func Verify(
chainID string,
trustedHeader *types.SignedHeader, // height=X
trustedVals *types.ValidatorSet, // height=X or height=X+1
untrustedHeader *types.SignedHeader, // height=Y
@ -148,22 +143,21 @@ func Verify(
trustLevel tmmath.Fraction) error {
if untrustedHeader.Height != trustedHeader.Height+1 {
return VerifyNonAdjacent(chainID, trustedHeader, trustedVals, untrustedHeader, untrustedVals,
return VerifyNonAdjacent(trustedHeader, trustedVals, untrustedHeader, untrustedVals,
trustingPeriod, now, maxClockDrift, trustLevel)
}
return VerifyAdjacent(chainID, trustedHeader, untrustedHeader, untrustedVals, trustingPeriod, now, maxClockDrift)
return VerifyAdjacent(trustedHeader, untrustedHeader, untrustedVals, trustingPeriod, now, maxClockDrift)
}
func verifyNewHeaderAndVals(
chainID string,
untrustedHeader *types.SignedHeader,
untrustedVals *types.ValidatorSet,
trustedHeader *types.SignedHeader,
now time.Time,
maxClockDrift time.Duration) error {
if err := untrustedHeader.ValidateBasic(chainID); err != nil {
if err := untrustedHeader.ValidateBasic(trustedHeader.ChainID); err != nil {
return fmt.Errorf("untrustedHeader.ValidateBasic failed: %w", err)
}


+ 3
- 3
light/verifier_test.go View File

@ -155,7 +155,7 @@ func TestVerifyAdjacentHeaders(t *testing.T) {
for i, tc := range testCases {
tc := tc
t.Run(fmt.Sprintf("#%d", i), func(t *testing.T) {
err := light.VerifyAdjacent(chainID, header, tc.newHeader, tc.newVals, tc.trustingPeriod, tc.now, maxClockDrift)
err := light.VerifyAdjacent(header, tc.newHeader, tc.newVals, tc.trustingPeriod, tc.now, maxClockDrift)
switch {
case tc.expErr != nil && assert.Error(t, err):
assert.Equal(t, tc.expErr, err)
@ -269,7 +269,7 @@ func TestVerifyNonAdjacentHeaders(t *testing.T) {
for i, tc := range testCases {
tc := tc
t.Run(fmt.Sprintf("#%d", i), func(t *testing.T) {
err := light.VerifyNonAdjacent(chainID, header, vals, tc.newHeader, tc.newVals, tc.trustingPeriod,
err := light.VerifyNonAdjacent(header, vals, tc.newHeader, tc.newVals, tc.trustingPeriod,
tc.now, maxClockDrift,
light.DefaultTrustLevel)
@ -300,7 +300,7 @@ func TestVerifyReturnsErrorIfTrustLevelIsInvalid(t *testing.T) {
hash("app_hash"), hash("cons_hash"), hash("results_hash"), 0, len(keys))
)
err := light.Verify(chainID, header, vals, header, vals, 2*time.Hour, time.Now(), maxClockDrift,
err := light.Verify(header, vals, header, vals, 2*time.Hour, time.Now(), maxClockDrift,
tmmath.Fraction{Numerator: 2, Denominator: 1})
assert.Error(t, err)
}


+ 1
- 1
node/node_test.go View File

@ -262,7 +262,7 @@ func TestCreateProposalBlock(t *testing.T) {
// than can fit in a block
for i := 0; i <= maxEvidence; i++ {
ev := types.NewMockDuplicateVoteEvidenceWithValidator(height, time.Now(), privVals[0], "test-chain")
err := evidencePool.AddEvidence(ev)
err := evidencePool.AddEvidenceFromConsensus(ev, time.Now(), state.Validators)
require.NoError(t, err)
}


+ 1
- 1
p2p/mocks/peer.go View File

@ -1,4 +1,4 @@
// Code generated by mockery v2.2.1. DO NOT EDIT.
// Code generated by mockery v2.1.0. DO NOT EDIT.
package mocks


+ 139
- 58
proto/tendermint/evidence/types.pb.go View File

@ -7,16 +7,20 @@ import (
fmt "fmt"
_ "github.com/gogo/protobuf/gogoproto"
proto "github.com/gogo/protobuf/proto"
_ "github.com/gogo/protobuf/types"
github_com_gogo_protobuf_types "github.com/gogo/protobuf/types"
types "github.com/tendermint/tendermint/proto/tendermint/types"
io "io"
math "math"
math_bits "math/bits"
time "time"
)
// Reference imports to suppress errors if they are not otherwise used.
var _ = proto.Marshal
var _ = fmt.Errorf
var _ = math.Inf
var _ = time.Kitchen
// This is a compile-time assertion to ensure that this generated file
// is compatible with the proto package it is being compiled against.
@ -69,9 +73,10 @@ func (m *List) GetEvidence() []*types.Evidence {
}
type Info struct {
Committed bool `protobuf:"varint,1,opt,name=committed,proto3" json:"committed,omitempty"`
Priority int64 `protobuf:"varint,2,opt,name=priority,proto3" json:"priority,omitempty"`
Evidence types.Evidence `protobuf:"bytes,3,opt,name=evidence,proto3" json:"evidence"`
Evidence types.Evidence `protobuf:"bytes,1,opt,name=evidence,proto3" json:"evidence"`
Time time.Time `protobuf:"bytes,2,opt,name=time,proto3,stdtime" json:"time"`
Validators []*types.Validator `protobuf:"bytes,3,rep,name=validators,proto3" json:"validators,omitempty"`
TotalVotingPower int64 `protobuf:"varint,4,opt,name=total_voting_power,json=totalVotingPower,proto3" json:"total_voting_power,omitempty"`
}
func (m *Info) Reset() { *m = Info{} }
@ -107,25 +112,32 @@ func (m *Info) XXX_DiscardUnknown() {
var xxx_messageInfo_Info proto.InternalMessageInfo
func (m *Info) GetCommitted() bool {
func (m *Info) GetEvidence() types.Evidence {
if m != nil {
return m.Committed
return m.Evidence
}
return false
return types.Evidence{}
}
func (m *Info) GetPriority() int64 {
func (m *Info) GetTime() time.Time {
if m != nil {
return m.Priority
return m.Time
}
return 0
return time.Time{}
}
func (m *Info) GetEvidence() types.Evidence {
func (m *Info) GetValidators() []*types.Validator {
if m != nil {
return m.Evidence
return m.Validators
}
return types.Evidence{}
return nil
}
func (m *Info) GetTotalVotingPower() int64 {
if m != nil {
return m.TotalVotingPower
}
return 0
}
func init() {
@ -136,23 +148,28 @@ func init() {
func init() { proto.RegisterFile("tendermint/evidence/types.proto", fileDescriptor_5e804d1c041a0e47) }
var fileDescriptor_5e804d1c041a0e47 = []byte{
// 243 bytes of a gzipped FileDescriptorProto
// 329 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x92, 0x2f, 0x49, 0xcd, 0x4b,
0x49, 0x2d, 0xca, 0xcd, 0xcc, 0x2b, 0xd1, 0x4f, 0x2d, 0xcb, 0x4c, 0x49, 0xcd, 0x4b, 0x4e, 0xd5,
0x2f, 0xa9, 0x2c, 0x48, 0x2d, 0xd6, 0x2b, 0x28, 0xca, 0x2f, 0xc9, 0x17, 0x12, 0x46, 0x28, 0xd0,
0x83, 0x29, 0x90, 0x12, 0x49, 0xcf, 0x4f, 0xcf, 0x07, 0xcb, 0xeb, 0x83, 0x58, 0x10, 0xa5, 0x52,
0xc8, 0x66, 0x81, 0x8d, 0x80, 0x9b, 0x08, 0x51, 0xa0, 0x64, 0xc7, 0xc5, 0xe2, 0x93, 0x59, 0x5c,
0x22, 0x64, 0xc6, 0xc5, 0x01, 0x93, 0x91, 0x60, 0x54, 0x60, 0xd6, 0xe0, 0x36, 0x92, 0xd2, 0x43,
0xb2, 0x06, 0x62, 0xbd, 0x2b, 0x54, 0x45, 0x10, 0x5c, 0xad, 0x52, 0x1d, 0x17, 0x8b, 0x67, 0x5e,
0x5a, 0xbe, 0x90, 0x0c, 0x17, 0x67, 0x72, 0x7e, 0x6e, 0x6e, 0x66, 0x49, 0x49, 0x6a, 0x8a, 0x04,
0xa3, 0x02, 0xa3, 0x06, 0x47, 0x10, 0x42, 0x40, 0x48, 0x8a, 0x8b, 0xa3, 0xa0, 0x28, 0x33, 0xbf,
0x28, 0xb3, 0xa4, 0x52, 0x82, 0x49, 0x81, 0x51, 0x83, 0x39, 0x08, 0xce, 0x17, 0xb2, 0x41, 0xb2,
0x99, 0x59, 0x81, 0x11, 0xbf, 0xcd, 0x4e, 0x2c, 0x27, 0xee, 0xc9, 0x33, 0x20, 0xec, 0x77, 0x0a,
0x39, 0xf1, 0x48, 0x8e, 0xf1, 0xc2, 0x23, 0x39, 0xc6, 0x07, 0x8f, 0xe4, 0x18, 0x27, 0x3c, 0x96,
0x63, 0xb8, 0xf0, 0x58, 0x8e, 0xe1, 0xc6, 0x63, 0x39, 0x86, 0x28, 0xab, 0xf4, 0xcc, 0x92, 0x8c,
0xd2, 0x24, 0xbd, 0xe4, 0xfc, 0x5c, 0x7d, 0xe4, 0x50, 0x40, 0x30, 0x21, 0xa1, 0x85, 0x25, 0xb4,
0x93, 0xd8, 0xc0, 0x52, 0xc6, 0x80, 0x00, 0x00, 0x00, 0xff, 0xff, 0xba, 0x95, 0x45, 0x68, 0x8b,
0x01, 0x00, 0x00,
0xf2, 0xe9, 0xf9, 0xf9, 0xe9, 0x39, 0xa9, 0xfa, 0x60, 0x5e, 0x52, 0x69, 0x9a, 0x7e, 0x49, 0x66,
0x6e, 0x6a, 0x71, 0x49, 0x62, 0x6e, 0x01, 0x54, 0x81, 0x02, 0x92, 0x65, 0x60, 0x3b, 0xf4, 0xcb,
0x12, 0x73, 0x32, 0x53, 0x12, 0x4b, 0xf2, 0x8b, 0x60, 0x46, 0x60, 0xa8, 0x80, 0xd9, 0x09, 0x51,
0xa0, 0x64, 0xc7, 0xc5, 0xe2, 0x93, 0x59, 0x5c, 0x22, 0x64, 0xc6, 0xc5, 0x01, 0x93, 0x91, 0x60,
0x54, 0x60, 0xd6, 0xe0, 0x36, 0x92, 0xd2, 0x43, 0x72, 0x29, 0xc4, 0x07, 0xae, 0x50, 0x15, 0x41,
0x70, 0xb5, 0x4a, 0x2f, 0x19, 0xb9, 0x58, 0x3c, 0xf3, 0xd2, 0xf2, 0x85, 0x6c, 0x50, 0x0c, 0x60,
0xc4, 0x6f, 0x80, 0x13, 0xcb, 0x89, 0x7b, 0xf2, 0x0c, 0x08, 0x63, 0x84, 0x2c, 0xb8, 0x58, 0x40,
0x9e, 0x93, 0x60, 0x82, 0xea, 0x84, 0xf8, 0x5c, 0x0f, 0xe6, 0x73, 0xbd, 0x10, 0x98, 0xcf, 0x9d,
0x38, 0x40, 0x3a, 0x27, 0xdc, 0x97, 0x67, 0x0c, 0x02, 0xeb, 0x10, 0xb2, 0xe6, 0xe2, 0x82, 0x7b,
0xba, 0x58, 0x82, 0x19, 0xec, 0x74, 0x69, 0x4c, 0x9b, 0xc3, 0x60, 0x6a, 0x82, 0x90, 0x94, 0x0b,
0xe9, 0x70, 0x09, 0x95, 0xe4, 0x97, 0x24, 0xe6, 0xc4, 0x97, 0xe5, 0x97, 0x64, 0xe6, 0xa5, 0xc7,
0x17, 0xe4, 0x97, 0xa7, 0x16, 0x49, 0xb0, 0x28, 0x30, 0x6a, 0x30, 0x07, 0x09, 0x80, 0x65, 0xc2,
0xc0, 0x12, 0x01, 0x20, 0x71, 0xa7, 0x90, 0x13, 0x8f, 0xe4, 0x18, 0x2f, 0x3c, 0x92, 0x63, 0x7c,
0xf0, 0x48, 0x8e, 0x71, 0xc2, 0x63, 0x39, 0x86, 0x0b, 0x8f, 0xe5, 0x18, 0x6e, 0x3c, 0x96, 0x63,
0x88, 0xb2, 0x4a, 0xcf, 0x2c, 0xc9, 0x28, 0x4d, 0xd2, 0x4b, 0xce, 0xcf, 0xd5, 0x47, 0x0e, 0x71,
0x04, 0x13, 0x12, 0xb9, 0x58, 0x12, 0x47, 0x12, 0x1b, 0x58, 0xca, 0x18, 0x10, 0x00, 0x00, 0xff,
0xff, 0x89, 0xbc, 0x3a, 0x32, 0x3a, 0x02, 0x00, 0x00,
}
func (m *List) Marshal() (dAtA []byte, err error) {
@ -212,6 +229,33 @@ func (m *Info) MarshalToSizedBuffer(dAtA []byte) (int, error) {
_ = i
var l int
_ = l
if m.TotalVotingPower != 0 {
i = encodeVarintTypes(dAtA, i, uint64(m.TotalVotingPower))
i--
dAtA[i] = 0x20
}
if len(m.Validators) > 0 {
for iNdEx := len(m.Validators) - 1; iNdEx >= 0; iNdEx-- {
{
size, err := m.Validators[iNdEx].MarshalToSizedBuffer(dAtA[:i])
if err != nil {
return 0, err
}
i -= size
i = encodeVarintTypes(dAtA, i, uint64(size))
}
i--
dAtA[i] = 0x1a
}
}
n1, err1 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Time, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Time):])
if err1 != nil {
return 0, err1
}
i -= n1
i = encodeVarintTypes(dAtA, i, uint64(n1))
i--
dAtA[i] = 0x12
{
size, err := m.Evidence.MarshalToSizedBuffer(dAtA[:i])
if err != nil {
@ -221,22 +265,7 @@ func (m *Info) MarshalToSizedBuffer(dAtA []byte) (int, error) {
i = encodeVarintTypes(dAtA, i, uint64(size))
}
i--
dAtA[i] = 0x1a
if m.Priority != 0 {
i = encodeVarintTypes(dAtA, i, uint64(m.Priority))
i--
dAtA[i] = 0x10
}
if m.Committed {
i--
if m.Committed {
dAtA[i] = 1
} else {
dAtA[i] = 0
}
i--
dAtA[i] = 0x8
}
dAtA[i] = 0xa
return len(dAtA) - i, nil
}
@ -272,14 +301,19 @@ func (m *Info) Size() (n int) {
}
var l int
_ = l
if m.Committed {
n += 2
}
if m.Priority != 0 {
n += 1 + sovTypes(uint64(m.Priority))
}
l = m.Evidence.Size()
n += 1 + l + sovTypes(uint64(l))
l = github_com_gogo_protobuf_types.SizeOfStdTime(m.Time)
n += 1 + l + sovTypes(uint64(l))
if len(m.Validators) > 0 {
for _, e := range m.Validators {
l = e.Size()
n += 1 + l + sovTypes(uint64(l))
}
}
if m.TotalVotingPower != 0 {
n += 1 + sovTypes(uint64(m.TotalVotingPower))
}
return n
}
@ -406,10 +440,10 @@ func (m *Info) Unmarshal(dAtA []byte) error {
}
switch fieldNum {
case 1:
if wireType != 0 {
return fmt.Errorf("proto: wrong wireType = %d for field Committed", wireType)
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field Evidence", wireType)
}
var v int
var msglen int
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowTypes
@ -419,17 +453,30 @@ func (m *Info) Unmarshal(dAtA []byte) error {
}
b := dAtA[iNdEx]
iNdEx++
v |= int(b&0x7F) << shift
msglen |= int(b&0x7F) << shift
if b < 0x80 {
break
}
}
m.Committed = bool(v != 0)
if msglen < 0 {
return ErrInvalidLengthTypes
}
postIndex := iNdEx + msglen
if postIndex < 0 {
return ErrInvalidLengthTypes
}
if postIndex > l {
return io.ErrUnexpectedEOF
}
if err := m.Evidence.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
return err
}
iNdEx = postIndex
case 2:
if wireType != 0 {
return fmt.Errorf("proto: wrong wireType = %d for field Priority", wireType)
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field Time", wireType)
}
m.Priority = 0
var msglen int
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowTypes
@ -439,14 +486,28 @@ func (m *Info) Unmarshal(dAtA []byte) error {
}
b := dAtA[iNdEx]
iNdEx++
m.Priority |= int64(b&0x7F) << shift
msglen |= int(b&0x7F) << shift
if b < 0x80 {
break
}
}
if msglen < 0 {
return ErrInvalidLengthTypes
}
postIndex := iNdEx + msglen
if postIndex < 0 {
return ErrInvalidLengthTypes
}
if postIndex > l {
return io.ErrUnexpectedEOF
}
if err := github_com_gogo_protobuf_types.StdTimeUnmarshal(&m.Time, dAtA[iNdEx:postIndex]); err != nil {
return err
}
iNdEx = postIndex
case 3:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field Evidence", wireType)
return fmt.Errorf("proto: wrong wireType = %d for field Validators", wireType)
}
var msglen int
for shift := uint(0); ; shift += 7 {
@ -473,10 +534,30 @@ func (m *Info) Unmarshal(dAtA []byte) error {
if postIndex > l {
return io.ErrUnexpectedEOF
}
if err := m.Evidence.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
m.Validators = append(m.Validators, &types.Validator{})
if err := m.Validators[len(m.Validators)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
return err
}
iNdEx = postIndex
case 4:
if wireType != 0 {
return fmt.Errorf("proto: wrong wireType = %d for field TotalVotingPower", wireType)
}
m.TotalVotingPower = 0
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowTypes
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
m.TotalVotingPower |= int64(b&0x7F) << shift
if b < 0x80 {
break
}
}
default:
iNdEx = preIndex
skippy, err := skipTypes(dAtA[iNdEx:])


+ 6
- 3
proto/tendermint/evidence/types.proto View File

@ -4,6 +4,8 @@ package tendermint.evidence;
option go_package = "github.com/tendermint/tendermint/proto/tendermint/evidence";
import "gogoproto/gogo.proto";
import "google/protobuf/timestamp.proto";
import "tendermint/types/validator.proto";
import "tendermint/types/evidence.proto";
message List {
@ -11,7 +13,8 @@ message List {
}
message Info {
bool committed = 1;
int64 priority = 2;
tendermint.types.Evidence evidence = 3 [(gogoproto.nullable) = false];
tendermint.types.Evidence evidence = 1 [(gogoproto.nullable) = false];
google.protobuf.Timestamp time = 2 [(gogoproto.nullable) = false, (gogoproto.stdtime) = true];
repeated tendermint.types.Validator validators = 3;
int64 total_voting_power = 4;
}

+ 295
- 51
proto/tendermint/types/evidence.pb.go View File

@ -7,20 +7,15 @@ import (
fmt "fmt"
_ "github.com/gogo/protobuf/gogoproto"
proto "github.com/gogo/protobuf/proto"
_ "github.com/gogo/protobuf/types"
github_com_gogo_protobuf_types "github.com/gogo/protobuf/types"
_ "github.com/tendermint/tendermint/proto/tendermint/crypto"
io "io"
math "math"
math_bits "math/bits"
time "time"
)
// Reference imports to suppress errors if they are not otherwise used.
var _ = proto.Marshal
var _ = fmt.Errorf
var _ = math.Inf
var _ = time.Kitchen
// This is a compile-time assertion to ensure that this generated file
// is compatible with the proto package it is being compiled against.
@ -31,9 +26,8 @@ const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package
// DuplicateVoteEvidence contains evidence a validator signed two conflicting
// votes.
type DuplicateVoteEvidence struct {
VoteA *Vote `protobuf:"bytes,1,opt,name=vote_a,json=voteA,proto3" json:"vote_a,omitempty"`
VoteB *Vote `protobuf:"bytes,2,opt,name=vote_b,json=voteB,proto3" json:"vote_b,omitempty"`
Timestamp time.Time `protobuf:"bytes,3,opt,name=timestamp,proto3,stdtime" json:"timestamp"`
VoteA *Vote `protobuf:"bytes,1,opt,name=vote_a,json=voteA,proto3" json:"vote_a,omitempty"`
VoteB *Vote `protobuf:"bytes,2,opt,name=vote_b,json=voteB,proto3" json:"vote_b,omitempty"`
}
func (m *DuplicateVoteEvidence) Reset() { *m = DuplicateVoteEvidence{} }
@ -83,16 +77,62 @@ func (m *DuplicateVoteEvidence) GetVoteB() *Vote {
return nil
}
func (m *DuplicateVoteEvidence) GetTimestamp() time.Time {
type LightClientAttackEvidence struct {
ConflictingBlock *LightBlock `protobuf:"bytes,1,opt,name=conflicting_block,json=conflictingBlock,proto3" json:"conflicting_block,omitempty"`
CommonHeight int64 `protobuf:"varint,2,opt,name=common_height,json=commonHeight,proto3" json:"common_height,omitempty"`
}
func (m *LightClientAttackEvidence) Reset() { *m = LightClientAttackEvidence{} }
func (m *LightClientAttackEvidence) String() string { return proto.CompactTextString(m) }
func (*LightClientAttackEvidence) ProtoMessage() {}
func (*LightClientAttackEvidence) Descriptor() ([]byte, []int) {
return fileDescriptor_6825fabc78e0a168, []int{1}
}
func (m *LightClientAttackEvidence) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
}
func (m *LightClientAttackEvidence) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
if deterministic {
return xxx_messageInfo_LightClientAttackEvidence.Marshal(b, m, deterministic)
} else {
b = b[:cap(b)]
n, err := m.MarshalToSizedBuffer(b)
if err != nil {
return nil, err
}
return b[:n], nil
}
}
func (m *LightClientAttackEvidence) XXX_Merge(src proto.Message) {
xxx_messageInfo_LightClientAttackEvidence.Merge(m, src)
}
func (m *LightClientAttackEvidence) XXX_Size() int {
return m.Size()
}
func (m *LightClientAttackEvidence) XXX_DiscardUnknown() {
xxx_messageInfo_LightClientAttackEvidence.DiscardUnknown(m)
}
var xxx_messageInfo_LightClientAttackEvidence proto.InternalMessageInfo
func (m *LightClientAttackEvidence) GetConflictingBlock() *LightBlock {
if m != nil {
return m.Timestamp
return m.ConflictingBlock
}
return time.Time{}
return nil
}
func (m *LightClientAttackEvidence) GetCommonHeight() int64 {
if m != nil {
return m.CommonHeight
}
return 0
}
type Evidence struct {
// Types that are valid to be assigned to Sum:
// *Evidence_DuplicateVoteEvidence
// *Evidence_LightClientAttackEvidence
Sum isEvidence_Sum `protobuf_oneof:"sum"`
}
@ -100,7 +140,7 @@ func (m *Evidence) Reset() { *m = Evidence{} }
func (m *Evidence) String() string { return proto.CompactTextString(m) }
func (*Evidence) ProtoMessage() {}
func (*Evidence) Descriptor() ([]byte, []int) {
return fileDescriptor_6825fabc78e0a168, []int{1}
return fileDescriptor_6825fabc78e0a168, []int{2}
}
func (m *Evidence) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@ -138,8 +178,12 @@ type isEvidence_Sum interface {
type Evidence_DuplicateVoteEvidence struct {
DuplicateVoteEvidence *DuplicateVoteEvidence `protobuf:"bytes,1,opt,name=duplicate_vote_evidence,json=duplicateVoteEvidence,proto3,oneof" json:"duplicate_vote_evidence,omitempty"`
}
type Evidence_LightClientAttackEvidence struct {
LightClientAttackEvidence *LightClientAttackEvidence `protobuf:"bytes,2,opt,name=light_client_attack_evidence,json=lightClientAttackEvidence,proto3,oneof" json:"light_client_attack_evidence,omitempty"`
}
func (*Evidence_DuplicateVoteEvidence) isEvidence_Sum() {}
func (*Evidence_DuplicateVoteEvidence) isEvidence_Sum() {}
func (*Evidence_LightClientAttackEvidence) isEvidence_Sum() {}
func (m *Evidence) GetSum() isEvidence_Sum {
if m != nil {
@ -155,10 +199,18 @@ func (m *Evidence) GetDuplicateVoteEvidence() *DuplicateVoteEvidence {
return nil
}
func (m *Evidence) GetLightClientAttackEvidence() *LightClientAttackEvidence {
if x, ok := m.GetSum().(*Evidence_LightClientAttackEvidence); ok {
return x.LightClientAttackEvidence
}
return nil
}
// XXX_OneofWrappers is for the internal use of the proto package.
func (*Evidence) XXX_OneofWrappers() []interface{} {
return []interface{}{
(*Evidence_DuplicateVoteEvidence)(nil),
(*Evidence_LightClientAttackEvidence)(nil),
}
}
@ -172,7 +224,7 @@ func (m *EvidenceData) Reset() { *m = EvidenceData{} }
func (m *EvidenceData) String() string { return proto.CompactTextString(m) }
func (*EvidenceData) ProtoMessage() {}
func (*EvidenceData) Descriptor() ([]byte, []int) {
return fileDescriptor_6825fabc78e0a168, []int{2}
return fileDescriptor_6825fabc78e0a168, []int{3}
}
func (m *EvidenceData) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@ -217,6 +269,7 @@ func (m *EvidenceData) GetHash() []byte {
func init() {
proto.RegisterType((*DuplicateVoteEvidence)(nil), "tendermint.types.DuplicateVoteEvidence")
proto.RegisterType((*LightClientAttackEvidence)(nil), "tendermint.types.LightClientAttackEvidence")
proto.RegisterType((*Evidence)(nil), "tendermint.types.Evidence")
proto.RegisterType((*EvidenceData)(nil), "tendermint.types.EvidenceData")
}
@ -224,30 +277,32 @@ func init() {
func init() { proto.RegisterFile("tendermint/types/evidence.proto", fileDescriptor_6825fabc78e0a168) }
var fileDescriptor_6825fabc78e0a168 = []byte{
// 355 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x84, 0x92, 0xcf, 0x4e, 0xf2, 0x40,
0x14, 0xc5, 0x3b, 0x1f, 0x7f, 0xc2, 0x37, 0xb2, 0x30, 0x8d, 0x28, 0x21, 0xa6, 0x35, 0x6c, 0x74,
0xe3, 0x4c, 0xa2, 0x0b, 0x37, 0x6e, 0x6c, 0x30, 0x71, 0x6b, 0x63, 0x5c, 0xb8, 0xc1, 0x69, 0x7b,
0x2d, 0x8d, 0x94, 0x69, 0xe8, 0x2d, 0x09, 0x6f, 0xc1, 0xeb, 0xf8, 0x06, 0x2c, 0x59, 0xba, 0x52,
0x03, 0x2f, 0x62, 0x3a, 0xb4, 0x85, 0x40, 0x13, 0x37, 0xcd, 0xed, 0x9c, 0xdf, 0xed, 0x3d, 0x67,
0x7a, 0xa9, 0x89, 0x30, 0xf2, 0x60, 0x1c, 0x06, 0x23, 0xe4, 0x38, 0x8d, 0x20, 0xe6, 0x30, 0x09,
0x3c, 0x18, 0xb9, 0xc0, 0xa2, 0xb1, 0x44, 0xa9, 0x1f, 0x6e, 0x00, 0xa6, 0x80, 0xce, 0x91, 0x2f,
0x7d, 0xa9, 0x44, 0x9e, 0x56, 0x6b, 0xae, 0x63, 0xfa, 0x52, 0xfa, 0x43, 0xe0, 0xea, 0xcd, 0x49,
0xde, 0x38, 0x06, 0x21, 0xc4, 0x28, 0xc2, 0x28, 0x03, 0x4e, 0xf7, 0x26, 0xa9, 0x67, 0x89, 0xea,
0x8e, 0xa7, 0x11, 0x4a, 0xfe, 0x0e, 0xd3, 0x4c, 0xed, 0x7e, 0x10, 0xda, 0xea, 0x25, 0xd1, 0x30,
0x70, 0x05, 0xc2, 0xb3, 0x44, 0xb8, 0xcf, 0x4c, 0xea, 0x97, 0xb4, 0x3e, 0x91, 0x08, 0x7d, 0xd1,
0x26, 0x67, 0xe4, 0xe2, 0xe0, 0xea, 0x98, 0xed, 0xfa, 0x65, 0x29, 0x6f, 0xd7, 0x52, 0xea, 0xae,
0xc0, 0x9d, 0xf6, 0xbf, 0xbf, 0x71, 0x4b, 0xb7, 0xe8, 0xff, 0x22, 0x46, 0xbb, 0xa2, 0x3a, 0x3a,
0x6c, 0x1d, 0x94, 0xe5, 0x41, 0xd9, 0x53, 0x4e, 0x58, 0x8d, 0xf9, 0x97, 0xa9, 0xcd, 0xbe, 0x4d,
0x62, 0x6f, 0xda, 0xba, 0x48, 0x1b, 0x85, 0x5b, 0x41, 0x4f, 0xbc, 0x3c, 0x46, 0x5f, 0x19, 0xc9,
0x6f, 0x3b, 0xb3, 0x7f, 0xbe, 0xef, 0xa7, 0x34, 0xf7, 0x83, 0x66, 0xb7, 0xbc, 0x32, 0xc1, 0xaa,
0xd1, 0x4a, 0x9c, 0x84, 0xdd, 0x57, 0xda, 0xcc, 0x8f, 0x7a, 0x02, 0x85, 0x7e, 0x4b, 0x1b, 0x5b,
0xa3, 0x2a, 0x2a, 0xc8, 0xde, 0xa8, 0xe2, 0x23, 0xd5, 0x34, 0x88, 0x5d, 0x74, 0xe8, 0x3a, 0xad,
0x0e, 0x44, 0x3c, 0x50, 0x97, 0xd6, 0xb4, 0x55, 0x6d, 0x3d, 0xce, 0x97, 0x06, 0x59, 0x2c, 0x0d,
0xf2, 0xb3, 0x34, 0xc8, 0x6c, 0x65, 0x68, 0x8b, 0x95, 0xa1, 0x7d, 0xae, 0x0c, 0xed, 0xe5, 0xc6,
0x0f, 0x70, 0x90, 0x38, 0xcc, 0x95, 0x21, 0xdf, 0xfe, 0xe9, 0x9b, 0x72, 0xbd, 0x3d, 0xbb, 0x0b,
0xe1, 0xd4, 0xd5, 0xf9, 0xf5, 0x6f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x0d, 0x48, 0x97, 0xaf, 0x95,
0x02, 0x00, 0x00,
// 398 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x84, 0x52, 0xcd, 0x6a, 0xea, 0x40,
0x14, 0x4e, 0xae, 0x3f, 0xc8, 0xe8, 0x05, 0x6f, 0xb8, 0xb6, 0x2a, 0x12, 0x4b, 0xba, 0xa8, 0x50,
0x9a, 0x80, 0x5d, 0x74, 0xd3, 0x8d, 0xa9, 0x05, 0x0b, 0xdd, 0x34, 0x8b, 0x2e, 0xba, 0x49, 0x27,
0x93, 0x69, 0x32, 0x98, 0xcc, 0x88, 0x4e, 0x84, 0x3e, 0x43, 0x37, 0x7d, 0x2c, 0x97, 0x2e, 0xbb,
0x2a, 0x45, 0xfb, 0x20, 0x25, 0x13, 0x8d, 0x62, 0x94, 0x6e, 0x86, 0xc3, 0x39, 0xdf, 0x9c, 0xef,
0x87, 0x03, 0xda, 0x1c, 0x53, 0x17, 0x8f, 0x43, 0x42, 0xb9, 0xc1, 0x5f, 0x47, 0x78, 0x62, 0xe0,
0x29, 0x71, 0x31, 0x45, 0x58, 0x1f, 0x8d, 0x19, 0x67, 0x4a, 0x75, 0x03, 0xd0, 0x05, 0xa0, 0xf9,
0xdf, 0x63, 0x1e, 0x13, 0x43, 0x23, 0xae, 0x12, 0x5c, 0xb3, 0x95, 0x59, 0x24, 0xde, 0x64, 0xaa,
0x45, 0xa0, 0xd6, 0x8f, 0x46, 0x01, 0x41, 0x90, 0xe3, 0x47, 0xc6, 0xf1, 0xed, 0x8a, 0x44, 0xb9,
0x00, 0xc5, 0x29, 0xe3, 0xd8, 0x86, 0x75, 0xf9, 0x44, 0xee, 0x94, 0xbb, 0x47, 0xfa, 0x2e, 0x9f,
0x1e, 0xe3, 0xad, 0x42, 0x8c, 0xea, 0xa5, 0x70, 0xa7, 0xfe, 0xe7, 0x77, 0xb8, 0xa9, 0xbd, 0xc9,
0xa0, 0x71, 0x4f, 0x3c, 0x9f, 0xdf, 0x04, 0x04, 0x53, 0xde, 0xe3, 0x1c, 0xa2, 0x61, 0xca, 0x7d,
0x07, 0xfe, 0x21, 0x46, 0x5f, 0x02, 0x82, 0x38, 0xa1, 0x9e, 0xed, 0x04, 0x0c, 0x0d, 0x57, 0x32,
0x5a, 0xd9, 0xbd, 0x62, 0x8f, 0x19, 0x63, 0xac, 0xea, 0xd6, 0x37, 0xd1, 0x51, 0x4e, 0xc1, 0x5f,
0xc4, 0xc2, 0x90, 0x51, 0xdb, 0xc7, 0x31, 0x4e, 0xc8, 0xcb, 0x59, 0x95, 0xa4, 0x39, 0x10, 0x3d,
0xed, 0x5b, 0x06, 0xa5, 0x94, 0x1c, 0x82, 0x63, 0x77, 0x9d, 0x88, 0x2d, 0x3c, 0xad, 0x83, 0x5f,
0x49, 0x38, 0xcb, 0x4a, 0xd8, 0x1b, 0xe1, 0x40, 0xb2, 0x6a, 0xee, 0xde, 0x6c, 0x29, 0x68, 0x05,
0x31, 0xb1, 0x8d, 0x84, 0x7b, 0x1b, 0x0a, 0xfb, 0x1b, 0x9e, 0x24, 0xc2, 0xf3, 0x03, 0x56, 0xf7,
0x45, 0x36, 0x90, 0xac, 0x46, 0x70, 0x68, 0x68, 0x16, 0x40, 0x6e, 0x12, 0x85, 0xda, 0x33, 0xa8,
0xac, 0x5b, 0x7d, 0xc8, 0xa1, 0x72, 0x0d, 0x4a, 0x5b, 0xd6, 0x72, 0x9d, 0x72, 0xb7, 0x99, 0xa5,
0x4c, 0x97, 0xe4, 0x67, 0x9f, 0x6d, 0xc9, 0x4a, 0x7f, 0x28, 0x0a, 0xc8, 0xfb, 0x70, 0xe2, 0x0b,
0xb1, 0x15, 0x4b, 0xd4, 0xe6, 0xc3, 0x6c, 0xa1, 0xca, 0xf3, 0x85, 0x2a, 0x7f, 0x2d, 0x54, 0xf9,
0x7d, 0xa9, 0x4a, 0xf3, 0xa5, 0x2a, 0x7d, 0x2c, 0x55, 0xe9, 0xe9, 0xca, 0x23, 0xdc, 0x8f, 0x1c,
0x1d, 0xb1, 0xd0, 0xd8, 0x3e, 0xc8, 0x4d, 0x99, 0x1c, 0xee, 0xee, 0xb1, 0x3a, 0x45, 0xd1, 0xbf,
0xfc, 0x09, 0x00, 0x00, 0xff, 0xff, 0xd1, 0x75, 0x06, 0x32, 0x10, 0x03, 0x00, 0x00,
}
func (m *DuplicateVoteEvidence) Marshal() (dAtA []byte, err error) {
@ -270,14 +325,6 @@ func (m *DuplicateVoteEvidence) MarshalToSizedBuffer(dAtA []byte) (int, error) {
_ = i
var l int
_ = l
n1, err1 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Timestamp, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Timestamp):])
if err1 != nil {
return 0, err1
}
i -= n1
i = encodeVarintEvidence(dAtA, i, uint64(n1))
i--
dAtA[i] = 0x1a
if m.VoteB != nil {
{
size, err := m.VoteB.MarshalToSizedBuffer(dAtA[:i])
@ -305,6 +352,46 @@ func (m *DuplicateVoteEvidence) MarshalToSizedBuffer(dAtA []byte) (int, error) {
return len(dAtA) - i, nil
}
func (m *LightClientAttackEvidence) Marshal() (dAtA []byte, err error) {
size := m.Size()
dAtA = make([]byte, size)
n, err := m.MarshalToSizedBuffer(dAtA[:size])
if err != nil {
return nil, err
}
return dAtA[:n], nil
}
func (m *LightClientAttackEvidence) MarshalTo(dAtA []byte) (int, error) {
size := m.Size()
return m.MarshalToSizedBuffer(dAtA[:size])
}
func (m *LightClientAttackEvidence) MarshalToSizedBuffer(dAtA []byte) (int, error) {
i := len(dAtA)
_ = i
var l int
_ = l
if m.CommonHeight != 0 {
i = encodeVarintEvidence(dAtA, i, uint64(m.CommonHeight))
i--
dAtA[i] = 0x10
}
if m.ConflictingBlock != nil {
{
size, err := m.ConflictingBlock.MarshalToSizedBuffer(dAtA[:i])
if err != nil {
return 0, err
}
i -= size
i = encodeVarintEvidence(dAtA, i, uint64(size))
}
i--
dAtA[i] = 0xa
}
return len(dAtA) - i, nil
}
func (m *Evidence) Marshal() (dAtA []byte, err error) {
size := m.Size()
dAtA = make([]byte, size)
@ -358,6 +445,27 @@ func (m *Evidence_DuplicateVoteEvidence) MarshalToSizedBuffer(dAtA []byte) (int,
}
return len(dAtA) - i, nil
}
func (m *Evidence_LightClientAttackEvidence) MarshalTo(dAtA []byte) (int, error) {
size := m.Size()
return m.MarshalToSizedBuffer(dAtA[:size])
}
func (m *Evidence_LightClientAttackEvidence) MarshalToSizedBuffer(dAtA []byte) (int, error) {
i := len(dAtA)
if m.LightClientAttackEvidence != nil {
{
size, err := m.LightClientAttackEvidence.MarshalToSizedBuffer(dAtA[:i])
if err != nil {
return 0, err
}
i -= size
i = encodeVarintEvidence(dAtA, i, uint64(size))
}
i--
dAtA[i] = 0x12
}
return len(dAtA) - i, nil
}
func (m *EvidenceData) Marshal() (dAtA []byte, err error) {
size := m.Size()
dAtA = make([]byte, size)
@ -427,8 +535,22 @@ func (m *DuplicateVoteEvidence) Size() (n int) {
l = m.VoteB.Size()
n += 1 + l + sovEvidence(uint64(l))
}
l = github_com_gogo_protobuf_types.SizeOfStdTime(m.Timestamp)
n += 1 + l + sovEvidence(uint64(l))
return n
}
func (m *LightClientAttackEvidence) Size() (n int) {
if m == nil {
return 0
}
var l int
_ = l
if m.ConflictingBlock != nil {
l = m.ConflictingBlock.Size()
n += 1 + l + sovEvidence(uint64(l))
}
if m.CommonHeight != 0 {
n += 1 + sovEvidence(uint64(m.CommonHeight))
}
return n
}
@ -456,6 +578,18 @@ func (m *Evidence_DuplicateVoteEvidence) Size() (n int) {
}
return n
}
func (m *Evidence_LightClientAttackEvidence) Size() (n int) {
if m == nil {
return 0
}
var l int
_ = l
if m.LightClientAttackEvidence != nil {
l = m.LightClientAttackEvidence.Size()
n += 1 + l + sovEvidence(uint64(l))
}
return n
}
func (m *EvidenceData) Size() (n int) {
if m == nil {
return 0
@ -582,9 +716,62 @@ func (m *DuplicateVoteEvidence) Unmarshal(dAtA []byte) error {
return err
}
iNdEx = postIndex
case 3:
default:
iNdEx = preIndex
skippy, err := skipEvidence(dAtA[iNdEx:])
if err != nil {
return err
}
if skippy < 0 {
return ErrInvalidLengthEvidence
}
if (iNdEx + skippy) < 0 {
return ErrInvalidLengthEvidence
}
if (iNdEx + skippy) > l {
return io.ErrUnexpectedEOF
}
iNdEx += skippy
}
}
if iNdEx > l {
return io.ErrUnexpectedEOF
}
return nil
}
func (m *LightClientAttackEvidence) Unmarshal(dAtA []byte) error {
l := len(dAtA)
iNdEx := 0
for iNdEx < l {
preIndex := iNdEx
var wire uint64
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowEvidence
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
wire |= uint64(b&0x7F) << shift
if b < 0x80 {
break
}
}
fieldNum := int32(wire >> 3)
wireType := int(wire & 0x7)
if wireType == 4 {
return fmt.Errorf("proto: LightClientAttackEvidence: wiretype end group for non-group")
}
if fieldNum <= 0 {
return fmt.Errorf("proto: LightClientAttackEvidence: illegal tag %d (wire type %d)", fieldNum, wire)
}
switch fieldNum {
case 1:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field Timestamp", wireType)
return fmt.Errorf("proto: wrong wireType = %d for field ConflictingBlock", wireType)
}
var msglen int
for shift := uint(0); ; shift += 7 {
@ -611,10 +798,32 @@ func (m *DuplicateVoteEvidence) Unmarshal(dAtA []byte) error {
if postIndex > l {
return io.ErrUnexpectedEOF
}
if err := github_com_gogo_protobuf_types.StdTimeUnmarshal(&m.Timestamp, dAtA[iNdEx:postIndex]); err != nil {
if m.ConflictingBlock == nil {
m.ConflictingBlock = &LightBlock{}
}
if err := m.ConflictingBlock.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
return err
}
iNdEx = postIndex
case 2:
if wireType != 0 {
return fmt.Errorf("proto: wrong wireType = %d for field CommonHeight", wireType)
}
m.CommonHeight = 0
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowEvidence
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
m.CommonHeight |= int64(b&0x7F) << shift
if b < 0x80 {
break
}
}
default:
iNdEx = preIndex
skippy, err := skipEvidence(dAtA[iNdEx:])
@ -703,6 +912,41 @@ func (m *Evidence) Unmarshal(dAtA []byte) error {
}
m.Sum = &Evidence_DuplicateVoteEvidence{v}
iNdEx = postIndex
case 2:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field LightClientAttackEvidence", wireType)
}
var msglen int
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowEvidence
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
msglen |= int(b&0x7F) << shift
if b < 0x80 {
break
}
}
if msglen < 0 {
return ErrInvalidLengthEvidence
}
postIndex := iNdEx + msglen
if postIndex < 0 {
return ErrInvalidLengthEvidence
}
if postIndex > l {
return io.ErrUnexpectedEOF
}
v := &LightClientAttackEvidence{}
if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
return err
}
m.Sum = &Evidence_LightClientAttackEvidence{v}
iNdEx = postIndex
default:
iNdEx = preIndex
skippy, err := skipEvidence(dAtA[iNdEx:])


+ 6
- 5
proto/tendermint/types/evidence.proto View File

@ -4,23 +4,24 @@ package tendermint.types;
option go_package = "github.com/tendermint/tendermint/proto/tendermint/types";
import "gogoproto/gogo.proto";
import "google/protobuf/timestamp.proto";
import "tendermint/types/types.proto";
import "tendermint/crypto/keys.proto";
// DuplicateVoteEvidence contains evidence a validator signed two conflicting
// votes.
message DuplicateVoteEvidence {
Vote vote_a = 1;
Vote vote_b = 2;
}
google.protobuf.Timestamp timestamp = 3
[(gogoproto.nullable) = false, (gogoproto.stdtime) = true];
message LightClientAttackEvidence {
LightBlock conflicting_block = 1;
int64 common_height = 2;
}
message Evidence {
oneof sum {
DuplicateVoteEvidence duplicate_vote_evidence = 1;
DuplicateVoteEvidence duplicate_vote_evidence = 1;
LightClientAttackEvidence light_client_attack_evidence = 2;
}
}


+ 1
- 1
proxy/mocks/app_conn_consensus.go View File

@ -1,4 +1,4 @@
// Code generated by mockery v2.2.1. DO NOT EDIT.
// Code generated by mockery v2.1.0. DO NOT EDIT.
package mocks


+ 1
- 1
proxy/mocks/app_conn_mempool.go View File

@ -1,4 +1,4 @@
// Code generated by mockery v2.2.1. DO NOT EDIT.
// Code generated by mockery v2.1.0. DO NOT EDIT.
package mocks


+ 1
- 1
proxy/mocks/app_conn_query.go View File

@ -1,4 +1,4 @@
// Code generated by mockery v2.2.1. DO NOT EDIT.
// Code generated by mockery v2.1.0. DO NOT EDIT.
package mocks


+ 1
- 1
proxy/mocks/app_conn_snapshot.go View File

@ -1,4 +1,4 @@
// Code generated by mockery v2.2.1. DO NOT EDIT.
// Code generated by mockery v2.1.0. DO NOT EDIT.
package mocks


+ 4
- 6
rpc/client/evidence_test.go View File

@ -12,6 +12,7 @@ import (
"github.com/tendermint/tendermint/crypto/ed25519"
cryptoenc "github.com/tendermint/tendermint/crypto/encoding"
"github.com/tendermint/tendermint/crypto/tmhash"
tmrand "github.com/tendermint/tendermint/libs/rand"
"github.com/tendermint/tendermint/privval"
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
"github.com/tendermint/tendermint/rpc/client"
@ -40,7 +41,7 @@ func newEvidence(t *testing.T, val *privval.FilePV,
vote2.Signature, err = val.Key.PrivKey.Sign(types.VoteSignBytes(chainID, v2))
require.NoError(t, err)
return types.NewDuplicateVoteEvidence(vote, vote2, defaultTestTime)
return types.NewDuplicateVoteEvidence(vote, vote2)
}
func makeEvidences(
@ -56,7 +57,7 @@ func makeEvidences(
Type: tmproto.PrevoteType,
Timestamp: defaultTestTime,
BlockID: types.BlockID{
Hash: tmhash.Sum([]byte("blockhash")),
Hash: tmhash.Sum(tmrand.Bytes(tmhash.Size)),
PartSetHeader: types.PartSetHeader{
Total: 1000,
Hash: tmhash.Sum([]byte("partset")),
@ -114,13 +115,10 @@ func TestBroadcastEvidence_DuplicateVoteEvidence(t *testing.T) {
pv = privval.LoadOrGenFilePV(config.PrivValidatorKeyFile(), config.PrivValidatorStateFile())
)
correct, fakes := makeEvidences(t, pv, chainID)
for i, c := range GetClients() {
correct, fakes := makeEvidences(t, pv, chainID)
t.Logf("client %d", i)
t.Log(correct.Time())
result, err := c.BroadcastEvidence(correct)
require.NoError(t, err, "BroadcastEvidence(%s) failed", correct)
assert.Equal(t, correct.Hash(), result.Hash, "expected result hash to match evidence hash")


+ 11
- 19
state/execution.go View File

@ -131,9 +131,12 @@ func (blockExec *BlockExecutor) ApplyBlock(
return state, 0, ErrInvalidBlock(err)
}
// Update evpool with the block and state and get any byzantine validators for that block
byzVals := blockExec.evpool.ABCIEvidence(block.Height, block.Evidence.Evidence)
startTime := time.Now().UnixNano()
abciResponses, err := execBlockOnProxyApp(blockExec.logger, blockExec.proxyApp, block,
blockExec.store, state.InitialHeight)
blockExec.store, state.InitialHeight, byzVals)
endTime := time.Now().UnixNano()
blockExec.metrics.BlockProcessingTime.Observe(float64(endTime-startTime) / 1000000)
if err != nil {
@ -175,8 +178,8 @@ func (blockExec *BlockExecutor) ApplyBlock(
return state, 0, fmt.Errorf("commit failed for application: %v", err)
}
// Update evpool with the block and state.
blockExec.evpool.Update(block, state)
// Update evpool with the latest state.
blockExec.evpool.Update(state)
fail.Fail() // XXX
@ -258,6 +261,7 @@ func execBlockOnProxyApp(
block *types.Block,
store Store,
initialHeight int64,
byzVals []abci.Evidence,
) (*tmstate.ABCIResponses, error) {
var validTxs, invalidTxs = 0, 0
@ -285,7 +289,7 @@ func execBlockOnProxyApp(
}
proxyAppConn.SetResponseCallback(proxyCb)
commitInfo, byzVals := getBeginBlockValidatorInfo(block, store, initialHeight)
commitInfo := getBeginBlockValidatorInfo(block, store, initialHeight)
// Begin block
var err error
@ -325,7 +329,7 @@ func execBlockOnProxyApp(
}
func getBeginBlockValidatorInfo(block *types.Block, store Store,
initialHeight int64) (abci.LastCommitInfo, []abci.Evidence) {
initialHeight int64) abci.LastCommitInfo {
voteInfos := make([]abci.VoteInfo, block.LastCommit.Size())
// Initial block -> LastCommitInfo.Votes are empty.
// Remember that the first LastCommit is intentionally empty, so it makes
@ -356,22 +360,10 @@ func getBeginBlockValidatorInfo(block *types.Block, store Store,
}
}
byzVals := make([]abci.Evidence, len(block.Evidence.Evidence))
for i, ev := range block.Evidence.Evidence {
// We need the validator set. We already did this in validateBlock.
// TODO: Should we instead cache the valset in the evidence itself and add
// `SetValidatorSet()` and `ToABCI` methods ?
valset, err := store.LoadValidators(ev.Height())
if err != nil {
panic(err)
}
byzVals[i] = types.TM2PB.Evidence(ev, valset)
}
return abci.LastCommitInfo{
Round: block.LastCommit.Round,
Votes: voteInfos,
}, byzVals
}
}
func validateValidatorUpdates(abciUpdates []abci.ValidatorUpdate,
@ -533,7 +525,7 @@ func ExecCommitBlock(
store Store,
initialHeight int64,
) ([]byte, error) {
_, err := execBlockOnProxyApp(logger, appConnConsensus, block, store, initialHeight)
_, err := execBlockOnProxyApp(logger, appConnConsensus, block, store, initialHeight, []abci.Evidence{})
if err != nil {
logger.Error("Error executing block on proxy app", "height", block.Height, "err", err)
return nil, err


+ 40
- 48
state/execution_test.go View File

@ -6,16 +6,18 @@ import (
"time"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/require"
abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/crypto/ed25519"
cryptoenc "github.com/tendermint/tendermint/crypto/encoding"
"github.com/tendermint/tendermint/libs/log"
"github.com/tendermint/tendermint/mempool/mock"
mmock "github.com/tendermint/tendermint/mempool/mock"
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
"github.com/tendermint/tendermint/proxy"
sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/state/mocks"
"github.com/tendermint/tendermint/types"
tmtime "github.com/tendermint/tendermint/types/time"
)
@ -38,7 +40,7 @@ func TestApplyBlock(t *testing.T) {
stateStore := sm.NewStore(stateDB)
blockExec := sm.NewBlockExecutor(stateStore, log.TestingLogger(), proxyApp.Consensus(),
mock.Mempool{}, sm.MockEvidencePool{})
mmock.Mempool{}, sm.EmptyEvidencePool{})
block := makeBlock(state, 1)
blockID := types.BlockID{Hash: block.Hash(), PartSetHeader: block.MakePartSet(testPartSize).Header()}
@ -123,55 +125,45 @@ func TestBeginBlockByzantineValidators(t *testing.T) {
require.Nil(t, err)
defer proxyApp.Stop() //nolint:errcheck // ignore for tests
state, stateDB, privVals := makeState(2, 12)
state, stateDB, _ := makeState(1, 1)
stateStore := sm.NewStore(stateDB)
prevHash := state.LastBlockID.Hash
prevParts := types.PartSetHeader{}
prevBlockID := types.BlockID{Hash: prevHash, PartSetHeader: prevParts}
height1, val1 := int64(8), state.Validators.Validators[0].Address
height2, val2 := int64(3), state.Validators.Validators[1].Address
ev1 := types.NewMockDuplicateVoteEvidenceWithValidator(height1, time.Now(), privVals[val1.String()], chainID)
ev2 := types.NewMockDuplicateVoteEvidenceWithValidator(height2, time.Now(), privVals[val2.String()], chainID)
defaultEvidenceTime := time.Date(2019, 1, 1, 0, 0, 0, 0, time.UTC)
now := tmtime.Now()
valSet := state.Validators
testCases := []struct {
desc string
evidence []types.Evidence
expectedByzantineValidators []abci.Evidence
}{
{"none byzantine", []types.Evidence{}, []abci.Evidence{}},
{"one byzantine", []types.Evidence{ev1}, []abci.Evidence{types.TM2PB.Evidence(ev1, valSet)}},
{"multiple byzantine", []types.Evidence{ev1, ev2}, []abci.Evidence{
types.TM2PB.Evidence(ev1, valSet),
types.TM2PB.Evidence(ev2, valSet)}},
abciEv := []abci.Evidence{
{
Type: abci.EvidenceType_DUPLICATE_VOTE,
Height: 3,
Time: defaultEvidenceTime,
Validator: types.TM2PB.Validator(state.Validators.Validators[0]),
TotalVotingPower: 33,
},
{
Type: abci.EvidenceType_LIGHT_CLIENT_ATTACK,
Height: 8,
Time: defaultEvidenceTime,
Validator: types.TM2PB.Validator(state.Validators.Validators[0]),
TotalVotingPower: 12,
},
}
var (
commitSig0 = types.NewCommitSigForBlock(
[]byte("Signature1"),
state.Validators.Validators[0].Address,
now)
commitSig1 = types.NewCommitSigForBlock(
[]byte("Signature2"),
state.Validators.Validators[1].Address,
now)
)
commitSigs := []types.CommitSig{commitSig0, commitSig1}
lastCommit := types.NewCommit(9, 0, prevBlockID, commitSigs)
for _, tc := range testCases {
evpool := &mocks.EvidencePool{}
evpool.On("ABCIEvidence", mock.AnythingOfType("int64"), mock.AnythingOfType("[]types.Evidence")).Return(abciEv)
evpool.On("Update", mock.AnythingOfType("state.State")).Return()
evpool.On("CheckEvidence", mock.AnythingOfType("types.EvidenceList")).Return(nil)
block, _ := state.MakeBlock(10, makeTxs(2), lastCommit, nil, state.Validators.GetProposer().Address)
block.Time = now
block.Evidence.Evidence = tc.evidence
_, err = sm.ExecCommitBlock(proxyApp.Consensus(), block, log.TestingLogger(), stateStore, 1)
require.Nil(t, err, tc.desc)
blockExec := sm.NewBlockExecutor(stateStore, log.TestingLogger(), proxyApp.Consensus(),
mmock.Mempool{}, evpool)
// -> app must receive an index of the byzantine validator
assert.Equal(t, tc.expectedByzantineValidators, app.ByzantineValidators, tc.desc)
}
block := makeBlock(state, 1)
blockID := types.BlockID{Hash: block.Hash(), PartSetHeader: block.MakePartSet(testPartSize).Header()}
state, retainHeight, err := blockExec.ApplyBlock(state, blockID, block)
require.Nil(t, err)
assert.EqualValues(t, retainHeight, 1)
// TODO check state and mempool
assert.Equal(t, abciEv, app.ByzantineValidators)
}
func TestValidateValidatorUpdates(t *testing.T) {
@ -320,8 +312,8 @@ func TestEndBlockValidatorUpdates(t *testing.T) {
stateStore,
log.TestingLogger(),
proxyApp.Consensus(),
mock.Mempool{},
sm.MockEvidencePool{},
mmock.Mempool{},
sm.EmptyEvidencePool{},
)
eventBus := types.NewEventBus()
@ -390,8 +382,8 @@ func TestEndBlockValidatorUpdatesResultingInEmptySet(t *testing.T) {
stateStore,
log.TestingLogger(),
proxyApp.Consensus(),
mock.Mempool{},
sm.MockEvidencePool{},
mmock.Mempool{},
sm.EmptyEvidencePool{},
)
block := makeBlock(state, 1)


+ 36
- 18
state/mocks/evidence_pool.go View File

@ -1,9 +1,11 @@
// Code generated by mockery v2.2.1. DO NOT EDIT.
// Code generated by mockery v2.1.0. DO NOT EDIT.
package mocks
import (
mock "github.com/stretchr/testify/mock"
abcitypes "github.com/tendermint/tendermint/abci/types"
state "github.com/tendermint/tendermint/state"
types "github.com/tendermint/tendermint/types"
@ -14,6 +16,22 @@ type EvidencePool struct {
mock.Mock
}
// ABCIEvidence provides a mock function with given fields: _a0, _a1
func (_m *EvidencePool) ABCIEvidence(_a0 int64, _a1 []types.Evidence) []abcitypes.Evidence {
ret := _m.Called(_a0, _a1)
var r0 []abcitypes.Evidence
if rf, ok := ret.Get(0).(func(int64, []types.Evidence) []abcitypes.Evidence); ok {
r0 = rf(_a0, _a1)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).([]abcitypes.Evidence)
}
}
return r0
}
// AddEvidence provides a mock function with given fields: _a0
func (_m *EvidencePool) AddEvidence(_a0 types.Evidence) error {
ret := _m.Called(_a0)
@ -28,37 +46,37 @@ func (_m *EvidencePool) AddEvidence(_a0 types.Evidence) error {
return r0
}
// PendingEvidence provides a mock function with given fields: _a0
func (_m *EvidencePool) PendingEvidence(_a0 uint32) []types.Evidence {
// CheckEvidence provides a mock function with given fields: _a0
func (_m *EvidencePool) CheckEvidence(_a0 types.EvidenceList) 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.EvidenceList) 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)
}
// Verify provides a mock function with given fields: _a0
func (_m *EvidencePool) Verify(_a0 types.Evidence) error {
// PendingEvidence provides a mock function with given fields: _a0
func (_m *EvidencePool) PendingEvidence(_a0 uint32) []types.Evidence {
ret := _m.Called(_a0)
var r0 error
if rf, ok := ret.Get(0).(func(types.Evidence) error); ok {
var r0 []types.Evidence
if rf, ok := ret.Get(0).(func(uint32) []types.Evidence); ok {
r0 = rf(_a0)
} else {
r0 = ret.Error(0)
if ret.Get(0) != nil {
r0 = ret.Get(0).([]types.Evidence)
}
}
return r0
}
// Update provides a mock function with given fields: _a0
func (_m *EvidencePool) Update(_a0 state.State) {
_m.Called(_a0)
}

+ 205
- 0
state/mocks/store.go View File

@ -0,0 +1,205 @@
// Code generated by mockery v2.1.0. DO NOT EDIT.
package mocks
import (
mock "github.com/stretchr/testify/mock"
state "github.com/tendermint/tendermint/state"
tendermintstate "github.com/tendermint/tendermint/proto/tendermint/state"
tenderminttypes "github.com/tendermint/tendermint/types"
types "github.com/tendermint/tendermint/proto/tendermint/types"
)
// Store is an autogenerated mock type for the Store type
type Store struct {
mock.Mock
}
// Bootstrap provides a mock function with given fields: _a0
func (_m *Store) Bootstrap(_a0 state.State) error {
ret := _m.Called(_a0)
var r0 error
if rf, ok := ret.Get(0).(func(state.State) error); ok {
r0 = rf(_a0)
} else {
r0 = ret.Error(0)
}
return r0
}
// Load provides a mock function with given fields:
func (_m *Store) Load() (state.State, error) {
ret := _m.Called()
var r0 state.State
if rf, ok := ret.Get(0).(func() state.State); ok {
r0 = rf()
} else {
r0 = ret.Get(0).(state.State)
}
var r1 error
if rf, ok := ret.Get(1).(func() error); ok {
r1 = rf()
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// LoadABCIResponses provides a mock function with given fields: _a0
func (_m *Store) LoadABCIResponses(_a0 int64) (*tendermintstate.ABCIResponses, error) {
ret := _m.Called(_a0)
var r0 *tendermintstate.ABCIResponses
if rf, ok := ret.Get(0).(func(int64) *tendermintstate.ABCIResponses); ok {
r0 = rf(_a0)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*tendermintstate.ABCIResponses)
}
}
var r1 error
if rf, ok := ret.Get(1).(func(int64) error); ok {
r1 = rf(_a0)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// LoadConsensusParams provides a mock function with given fields: _a0
func (_m *Store) LoadConsensusParams(_a0 int64) (types.ConsensusParams, error) {
ret := _m.Called(_a0)
var r0 types.ConsensusParams
if rf, ok := ret.Get(0).(func(int64) types.ConsensusParams); ok {
r0 = rf(_a0)
} else {
r0 = ret.Get(0).(types.ConsensusParams)
}
var r1 error
if rf, ok := ret.Get(1).(func(int64) error); ok {
r1 = rf(_a0)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// LoadFromDBOrGenesisDoc provides a mock function with given fields: _a0
func (_m *Store) LoadFromDBOrGenesisDoc(_a0 *tenderminttypes.GenesisDoc) (state.State, error) {
ret := _m.Called(_a0)
var r0 state.State
if rf, ok := ret.Get(0).(func(*tenderminttypes.GenesisDoc) state.State); ok {
r0 = rf(_a0)
} else {
r0 = ret.Get(0).(state.State)
}
var r1 error
if rf, ok := ret.Get(1).(func(*tenderminttypes.GenesisDoc) error); ok {
r1 = rf(_a0)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// LoadFromDBOrGenesisFile provides a mock function with given fields: _a0
func (_m *Store) LoadFromDBOrGenesisFile(_a0 string) (state.State, error) {
ret := _m.Called(_a0)
var r0 state.State
if rf, ok := ret.Get(0).(func(string) state.State); ok {
r0 = rf(_a0)
} else {
r0 = ret.Get(0).(state.State)
}
var r1 error
if rf, ok := ret.Get(1).(func(string) error); ok {
r1 = rf(_a0)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// LoadValidators provides a mock function with given fields: _a0
func (_m *Store) LoadValidators(_a0 int64) (*tenderminttypes.ValidatorSet, error) {
ret := _m.Called(_a0)
var r0 *tenderminttypes.ValidatorSet
if rf, ok := ret.Get(0).(func(int64) *tenderminttypes.ValidatorSet); ok {
r0 = rf(_a0)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*tenderminttypes.ValidatorSet)
}
}
var r1 error
if rf, ok := ret.Get(1).(func(int64) error); ok {
r1 = rf(_a0)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// PruneStates provides a mock function with given fields: _a0, _a1
func (_m *Store) PruneStates(_a0 int64, _a1 int64) error {
ret := _m.Called(_a0, _a1)
var r0 error
if rf, ok := ret.Get(0).(func(int64, int64) error); ok {
r0 = rf(_a0, _a1)
} else {
r0 = ret.Error(0)
}
return r0
}
// Save provides a mock function with given fields: _a0
func (_m *Store) Save(_a0 state.State) error {
ret := _m.Called(_a0)
var r0 error
if rf, ok := ret.Get(0).(func(state.State) error); ok {
r0 = rf(_a0)
} else {
r0 = ret.Error(0)
}
return r0
}
// SaveABCIResponses provides a mock function with given fields: _a0, _a1
func (_m *Store) SaveABCIResponses(_a0 int64, _a1 *tendermintstate.ABCIResponses) error {
ret := _m.Called(_a0, _a1)
var r0 error
if rf, ok := ret.Get(0).(func(int64, *tendermintstate.ABCIResponses) error); ok {
r0 = rf(_a0, _a1)
} else {
r0 = ret.Error(0)
}
return r0
}

+ 20
- 10
state/services.go View File

@ -1,6 +1,9 @@
package state
import (
"time"
abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/types"
)
@ -37,19 +40,26 @@ type BlockStore interface {
//go:generate mockery --case underscore --name EvidencePool
// EvidencePool defines the EvidencePool interface used by the ConsensusState.
// Get/Set/Commit
// EvidencePool defines the EvidencePool interface used by State.
type EvidencePool interface {
PendingEvidence(uint32) []types.Evidence
AddEvidence(types.Evidence) error
Update(*types.Block, State)
Verify(types.Evidence) error
Update(State)
CheckEvidence(types.EvidenceList) error
ABCIEvidence(int64, []types.Evidence) []abci.Evidence
}
// MockEvidencePool is an empty implementation of EvidencePool, useful for testing.
type MockEvidencePool struct{}
// EmptyEvidencePool is an empty implementation of EvidencePool, useful for testing. It also complies
// to the consensus evidence pool interface
type EmptyEvidencePool 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) Verify(types.Evidence) error { return nil }
func (EmptyEvidencePool) PendingEvidence(uint32) []types.Evidence { return nil }
func (EmptyEvidencePool) AddEvidence(types.Evidence) error { return nil }
func (EmptyEvidencePool) Update(State) {}
func (EmptyEvidencePool) CheckEvidence(evList types.EvidenceList) error { return nil }
func (EmptyEvidencePool) ABCIEvidence(int64, []types.Evidence) []abci.Evidence {
return []abci.Evidence{}
}
func (EmptyEvidencePool) AddEvidenceFromConsensus(types.Evidence, time.Time, *types.ValidatorSet) error {
return nil
}

+ 6
- 0
state/store.go View File

@ -39,6 +39,12 @@ func calcABCIResponsesKey(height int64) []byte {
//----------------------
//go:generate mockery --case underscore --name Store
// Store defines the state store interface
//
// It is used to retrieve current state and save and load ABCI responses,
// validators and consensus parameters
type Store interface {
// LoadFromDBOrGenesisFile loads the most recent state.
// If the chain is new it will use the genesis file from the provided genesis file path as the current state.


+ 2
- 18
state/validation.go View File

@ -96,7 +96,7 @@ func validateBlock(evidencePool EvidencePool, state State, block *types.Block) e
}
}
// NOTE: We can't actually verify it's the right proposer because we dont
// NOTE: We can't actually verify it's the right proposer because we don't
// know what round the block was first proposed. So just check that it's
// a legit address and a known validator.
if len(block.ProposerAddress) != crypto.AddressSize {
@ -148,21 +148,5 @@ func validateBlock(evidencePool EvidencePool, state State, block *types.Block) e
}
// Validate all evidence.
for idx, ev := range block.Evidence.Evidence {
// 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"))
}
}
// Verify evidence using the evidence pool
err := evidencePool.Verify(ev)
if err != nil {
return types.NewErrEvidenceInvalid(ev, err)
}
}
return nil
return evidencePool.CheckEvidence(block.Evidence.Evidence)
}

+ 7
- 31
state/validation_test.go View File

@ -8,6 +8,7 @@ import (
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/require"
abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/crypto/ed25519"
"github.com/tendermint/tendermint/crypto/tmhash"
"github.com/tendermint/tendermint/libs/log"
@ -21,8 +22,6 @@ import (
const validationTestsStopHeight int64 = 10
var defaultTestTime = time.Date(2019, 1, 1, 0, 0, 0, 0, time.UTC)
func TestValidateBlockHeader(t *testing.T) {
proxyApp := newTestApp()
require.NoError(t, proxyApp.Start())
@ -35,7 +34,7 @@ func TestValidateBlockHeader(t *testing.T) {
log.TestingLogger(),
proxyApp.Consensus(),
memmock.Mempool{},
sm.MockEvidencePool{},
sm.EmptyEvidencePool{},
)
lastCommit := types.NewCommit(0, 0, types.BlockID{}, nil)
@ -106,7 +105,7 @@ func TestValidateBlockCommit(t *testing.T) {
log.TestingLogger(),
proxyApp.Consensus(),
memmock.Mempool{},
sm.MockEvidencePool{},
sm.EmptyEvidencePool{},
)
lastCommit := types.NewCommit(0, 0, types.BlockID{}, nil)
wrongSigsCommit := types.NewCommit(1, 0, types.BlockID{}, nil)
@ -218,8 +217,10 @@ func TestValidateBlockEvidence(t *testing.T) {
defaultEvidenceTime := time.Date(2019, 1, 1, 0, 0, 0, 0, time.UTC)
evpool := &mocks.EvidencePool{}
evpool.On("Verify", mock.AnythingOfType("*types.DuplicateVoteEvidence")).Return(nil)
evpool.On("Update", mock.AnythingOfType("*types.Block"), mock.AnythingOfType("state.State")).Return()
evpool.On("CheckEvidence", mock.AnythingOfType("types.EvidenceList")).Return(nil)
evpool.On("Update", mock.AnythingOfType("state.State")).Return()
evpool.On("ABCIEvidence", mock.AnythingOfType("int64"), mock.AnythingOfType("[]types.Evidence")).Return(
[]abci.Evidence{})
state.ConsensusParams.Evidence.MaxNum = 3
blockExec := sm.NewBlockExecutor(
@ -279,28 +280,3 @@ func TestValidateBlockEvidence(t *testing.T) {
require.NoError(t, err, "height %d", height)
}
}
func TestValidateDuplicateEvidenceShouldFail(t *testing.T) {
var height int64 = 1
state, stateDB, privVals := makeState(2, int(height))
stateStore := sm.NewStore(stateDB)
_, 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)
blockExec := sm.NewBlockExecutor(
stateStore, log.TestingLogger(),
nil,
nil,
sm.MockEvidencePool{})
// A block with a couple pieces of evidence passes.
block := makeBlock(state, height)
block.Evidence.Evidence = []types.Evidence{ev, ev2, ev2}
block.EvidenceHash = block.Evidence.Hash()
err := blockExec.ValidateBlock(state, block)
assert.Error(t, err)
}

+ 1
- 1
statesync/mocks/state_provider.go View File

@ -1,4 +1,4 @@
// Code generated by mockery v2.2.1. DO NOT EDIT.
// Code generated by mockery v2.1.0. DO NOT EDIT.
package mocks


+ 1
- 1
types/block_test.go View File

@ -660,7 +660,7 @@ func TestEvidenceDataProtoBuf(t *testing.T) {
const chainID = "mychain"
v := makeVote(t, val, chainID, math.MaxInt32, math.MaxInt64, 1, 0x01, blockID, time.Now())
v2 := makeVote(t, val, chainID, math.MaxInt32, math.MaxInt64, 2, 0x01, blockID2, time.Now())
ev := NewDuplicateVoteEvidence(v2, v, v2.Timestamp)
ev := NewDuplicateVoteEvidence(v2, v)
data := &EvidenceData{Evidence: EvidenceList{ev}}
_ = data.Hash()
testCases := []struct {


+ 225
- 189
types/evidence.go View File

@ -2,12 +2,12 @@ package types
import (
"bytes"
"encoding/binary"
"errors"
"fmt"
"strings"
"time"
"github.com/tendermint/tendermint/crypto"
"github.com/tendermint/tendermint/crypto/merkle"
"github.com/tendermint/tendermint/crypto/tmhash"
tmjson "github.com/tendermint/tendermint/libs/json"
@ -16,17 +16,13 @@ import (
)
// Evidence represents any provable malicious activity by a validator.
// Verification logic for each evidence is part of the evidence module.
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
Height() int64 // height of the infraction
Bytes() []byte // bytes which comprise the evidence
Hash() []byte // hash of the evidence
ValidateBasic() error // basic consistency check
String() string // string format of the evidence
}
const (
@ -34,93 +30,20 @@ const (
MaxEvidenceBytes int64 = 444
)
// ErrEvidenceInvalid wraps a piece of evidence and the error denoting how or why it is invalid.
type ErrEvidenceInvalid struct {
Evidence Evidence
ErrorValue error
}
// NewErrEvidenceInvalid returns a new EvidenceInvalid with the given err.
func NewErrEvidenceInvalid(ev Evidence, err error) *ErrEvidenceInvalid {
return &ErrEvidenceInvalid{ev, err}
}
// Error returns a string representation of the error.
func (err *ErrEvidenceInvalid) Error() string {
return fmt.Sprintf("Invalid evidence: %v. Evidence: %v", err.ErrorValue, err.Evidence)
}
// ErrEvidenceOverflow is for when there is too much evidence in a block.
type ErrEvidenceOverflow struct {
MaxNum int
GotNum int
}
// NewErrEvidenceOverflow returns a new ErrEvidenceOverflow where got > max.
func NewErrEvidenceOverflow(max, got int) *ErrEvidenceOverflow {
return &ErrEvidenceOverflow{max, got}
}
// Error returns a string representation of the error.
func (err *ErrEvidenceOverflow) Error() string {
return fmt.Sprintf("Too much evidence: Max %d, got %d", err.MaxNum, err.GotNum)
}
//-------------------------------------------
func EvidenceToProto(evidence Evidence) (*tmproto.Evidence, error) {
if evidence == nil {
return nil, errors.New("nil evidence")
}
switch evi := evidence.(type) {
case *DuplicateVoteEvidence:
pbevi := evi.ToProto()
tp := &tmproto.Evidence{
Sum: &tmproto.Evidence_DuplicateVoteEvidence{
DuplicateVoteEvidence: pbevi,
},
}
return tp, nil
default:
return nil, fmt.Errorf("toproto: evidence is not recognized: %T", evi)
}
}
func EvidenceFromProto(evidence *tmproto.Evidence) (Evidence, error) {
if evidence == nil {
return nil, errors.New("nil evidence")
}
switch evi := evidence.Sum.(type) {
case *tmproto.Evidence_DuplicateVoteEvidence:
return DuplicateVoteEvidenceFromProto(evi.DuplicateVoteEvidence)
default:
return nil, errors.New("evidence is not recognized")
}
}
func init() {
tmjson.RegisterType(&DuplicateVoteEvidence{}, "tendermint/DuplicateVoteEvidence")
}
//-------------------------------------------
//--------------------------------------------------------------------------------------
// DuplicateVoteEvidence contains evidence a validator signed two conflicting
// votes.
type DuplicateVoteEvidence struct {
VoteA *Vote `json:"vote_a"`
VoteB *Vote `json:"vote_b"`
Timestamp time.Time `json:"timestamp"`
}
var _ Evidence = &DuplicateVoteEvidence{}
// NewDuplicateVoteEvidence creates DuplicateVoteEvidence with right ordering given
// two conflicting votes. If one of the votes is nil, evidence returned is nil as well
func NewDuplicateVoteEvidence(vote1, vote2 *Vote, time time.Time) *DuplicateVoteEvidence {
func NewDuplicateVoteEvidence(vote1, vote2 *Vote) *DuplicateVoteEvidence {
var voteA, voteB *Vote
if vote1 == nil || vote2 == nil {
return nil
@ -135,14 +58,12 @@ func NewDuplicateVoteEvidence(vote1, vote2 *Vote, time time.Time) *DuplicateVote
return &DuplicateVoteEvidence{
VoteA: voteA,
VoteB: voteB,
Timestamp: time,
}
}
// String returns a string representation of the evidence.
func (dve *DuplicateVoteEvidence) String() string {
return fmt.Sprintf("DuplicateVoteEvidence{VoteA: %v, VoteB: %v, Time: %v}", dve.VoteA, dve.VoteB, dve.Timestamp)
return fmt.Sprintf("DuplicateVoteEvidence{VoteA: %v, VoteB: %v}", dve.VoteA, dve.VoteB)
}
// Height returns the height this evidence refers to.
@ -150,17 +71,7 @@ func (dve *DuplicateVoteEvidence) Height() int64 {
return dve.VoteA.Height
}
// Time returns time of the latest vote.
func (dve *DuplicateVoteEvidence) Time() time.Time {
return dve.Timestamp
}
// Address returns the address of the validator.
func (dve *DuplicateVoteEvidence) Address() []byte {
return dve.VoteA.ValidatorAddress
}
// Hash returns the hash of the evidence.
// Bytes returns the proto-encoded evidence as a byte array.
func (dve *DuplicateVoteEvidence) Bytes() []byte {
pbe := dve.ToProto()
bz, err := pbe.Marshal()
@ -173,88 +84,7 @@ func (dve *DuplicateVoteEvidence) Bytes() []byte {
// Hash returns the hash of the evidence.
func (dve *DuplicateVoteEvidence) Hash() []byte {
pbe := dve.ToProto()
bz, err := pbe.Marshal()
if err != nil {
panic(err)
}
return tmhash.Sum(bz)
}
// Verify returns an error if the two votes aren't conflicting.
//
// To be conflicting, they must be from the same validator, for the same H/R/S,
// but for different blocks.
func (dve *DuplicateVoteEvidence) Verify(chainID string, pubKey crypto.PubKey) error {
// H/R/S must be the same
if dve.VoteA.Height != dve.VoteB.Height ||
dve.VoteA.Round != dve.VoteB.Round ||
dve.VoteA.Type != dve.VoteB.Type {
return fmt.Errorf("h/r/s does not match: %d/%d/%v vs %d/%d/%v",
dve.VoteA.Height, dve.VoteA.Round, dve.VoteA.Type,
dve.VoteB.Height, dve.VoteB.Round, dve.VoteB.Type)
}
// Address must be the same
if !bytes.Equal(dve.VoteA.ValidatorAddress, dve.VoteB.ValidatorAddress) {
return fmt.Errorf("validator addresses do not match: %X vs %X",
dve.VoteA.ValidatorAddress,
dve.VoteB.ValidatorAddress,
)
}
// BlockIDs must be different
if dve.VoteA.BlockID.Equals(dve.VoteB.BlockID) {
return fmt.Errorf(
"block IDs are the same (%v) - not a real duplicate vote",
dve.VoteA.BlockID,
)
}
// pubkey must match address (this should already be true, sanity check)
addr := dve.VoteA.ValidatorAddress
if !bytes.Equal(pubKey.Address(), addr) {
return fmt.Errorf("address (%X) doesn't match pubkey (%v - %X)",
addr, pubKey, pubKey.Address())
}
va := dve.VoteA.ToProto()
vb := dve.VoteB.ToProto()
// Signatures must be valid
if !pubKey.VerifySignature(VoteSignBytes(chainID, va), dve.VoteA.Signature) {
return fmt.Errorf("verifying VoteA: %w", ErrVoteInvalidSignature)
}
if !pubKey.VerifySignature(VoteSignBytes(chainID, vb), dve.VoteB.Signature) {
return fmt.Errorf("verifying VoteB: %w", ErrVoteInvalidSignature)
}
return nil
}
// Equal checks if two pieces of evidence are equal.
func (dve *DuplicateVoteEvidence) Equal(ev Evidence) bool {
if _, ok := ev.(*DuplicateVoteEvidence); !ok {
return false
}
pbdev := dve.ToProto()
bz, err := pbdev.Marshal()
if err != nil {
panic(err)
}
var evbz []byte
if ev, ok := ev.(*DuplicateVoteEvidence); ok {
evpb := ev.ToProto()
evbz, err = evpb.Marshal()
if err != nil {
panic(err)
}
}
// just check their hashes
dveHash := tmhash.Sum(bz)
evHash := tmhash.Sum(evbz)
return bytes.Equal(dveHash, evHash)
return tmhash.Sum(dve.Bytes())
}
// ValidateBasic performs basic validation.
@ -279,17 +109,18 @@ func (dve *DuplicateVoteEvidence) ValidateBasic() error {
return nil
}
// ToProto encodes DuplicateVoteEvidence to protobuf
func (dve *DuplicateVoteEvidence) ToProto() *tmproto.DuplicateVoteEvidence {
voteB := dve.VoteB.ToProto()
voteA := dve.VoteA.ToProto()
tp := tmproto.DuplicateVoteEvidence{
VoteA: voteA,
VoteB: voteB,
Timestamp: dve.Timestamp,
VoteA: voteA,
VoteB: voteB,
}
return &tp
}
// DuplicateVoteEvidenceFromProto decodes protobuf into DuplicateVoteEvidence
func DuplicateVoteEvidenceFromProto(pb *tmproto.DuplicateVoteEvidence) (*DuplicateVoteEvidence, error) {
if pb == nil {
return nil, errors.New("nil duplicate vote evidence")
@ -305,12 +136,127 @@ func DuplicateVoteEvidenceFromProto(pb *tmproto.DuplicateVoteEvidence) (*Duplica
return nil, err
}
dve := NewDuplicateVoteEvidence(vA, vB, pb.Timestamp)
dve := NewDuplicateVoteEvidence(vA, vB)
return dve, dve.ValidateBasic()
}
//--------------------------------------------------
//------------------------------------ LIGHT EVIDENCE --------------------------------------
// LightClientAttackEvidence is a generalized evidence that captures all forms of known attacks on
// a light client such that a full node can verify, propose and commit the evidence on-chain for
// punishment of the malicious validators. There are three forms of attacks: Lunatic, Equivocation
// and Amnesia. These attacks are exhaustive. You can find a more detailed overview of this at
// tendermint/docs/architecture/adr-047-handling-evidence-from-light-client.md
type LightClientAttackEvidence struct {
ConflictingBlock *LightBlock
CommonHeight int64
}
var _ Evidence = &LightClientAttackEvidence{}
// Height returns the last height at which the primary provider and witness provider had the same header.
// We use this as the height of the infraction rather than the actual conflicting header because we know
// that the malicious validators were bonded at this height which is important for evidence expiry
func (l *LightClientAttackEvidence) Height() int64 {
return l.CommonHeight
}
// Bytes returns the proto-encoded evidence as a byte array
func (l *LightClientAttackEvidence) Bytes() []byte {
pbe, err := l.ToProto()
if err != nil {
panic(err)
}
bz, err := pbe.Marshal()
if err != nil {
panic(err)
}
return bz
}
// Hash returns the hash of the header and the commonHeight. This is designed to cause hash collisions with evidence
// that have the same conflicting header and common height but different permutations of validator commit signatures.
// The reason for this is that we don't want to allow several permutations of the same evidence to be committed on
// chain. Ideally we commit the header with the most commit signatures but anything greater than 1/3 is sufficient.
func (l *LightClientAttackEvidence) Hash() []byte {
buf := make([]byte, binary.MaxVarintLen64)
n := binary.PutVarint(buf, l.CommonHeight)
bz := make([]byte, tmhash.Size+n)
copy(bz[:tmhash.Size-1], l.ConflictingBlock.Hash().Bytes())
copy(bz[tmhash.Size:], buf)
return tmhash.Sum(bz)
}
// ValidateBasic performs basic validation such that the evidence is consistent and can now be used for verification.
func (l *LightClientAttackEvidence) ValidateBasic() error {
if l.ConflictingBlock == nil {
return errors.New("conflicting block is nil")
}
// this check needs to be done before we can run validate basic
if l.ConflictingBlock.Header == nil {
return errors.New("conflicting block missing header")
}
if err := l.ConflictingBlock.ValidateBasic(l.ConflictingBlock.ChainID); err != nil {
return fmt.Errorf("invalid conflicting light block: %w", err)
}
if l.CommonHeight <= 0 {
return errors.New("negative or zero common height")
}
// check that common height isn't ahead of the height of the conflicting block. It
// is possible that they are the same height if the light node witnesses either an
// amnesia or a equivocation attack.
if l.CommonHeight > l.ConflictingBlock.Height {
return fmt.Errorf("common height is ahead of the conflicting block height (%d > %d)",
l.CommonHeight, l.ConflictingBlock.Height)
}
return nil
}
// String returns a string representation of LightClientAttackEvidence
func (l *LightClientAttackEvidence) String() string {
return fmt.Sprintf("LightClientAttackEvidence{ConflictingBlock: %v, CommonHeight: %d}",
l.ConflictingBlock.String(), l.CommonHeight)
}
// ToProto encodes LightClientAttackEvidence to protobuf
func (l *LightClientAttackEvidence) ToProto() (*tmproto.LightClientAttackEvidence, error) {
conflictingBlock, err := l.ConflictingBlock.ToProto()
if err != nil {
return nil, err
}
return &tmproto.LightClientAttackEvidence{
ConflictingBlock: conflictingBlock,
CommonHeight: l.CommonHeight,
}, nil
}
// LightClientAttackEvidenceFromProto decodes protobuf
func LightClientAttackEvidenceFromProto(l *tmproto.LightClientAttackEvidence) (*LightClientAttackEvidence, error) {
if l == nil {
return nil, errors.New("empty light client attack evidence")
}
conflictingBlock, err := LightBlockFromProto(l.ConflictingBlock)
if err != nil {
return nil, err
}
le := &LightClientAttackEvidence{
ConflictingBlock: conflictingBlock,
CommonHeight: l.CommonHeight,
}
return le, le.ValidateBasic()
}
//------------------------------------------------------------------------------------------
// EvidenceList is a list of Evidence. Evidences is not a word.
type EvidenceList []Evidence
@ -338,13 +284,103 @@ func (evl EvidenceList) String() string {
// Has returns true if the evidence is in the EvidenceList.
func (evl EvidenceList) Has(evidence Evidence) bool {
for _, ev := range evl {
if ev.Equal(evidence) {
if bytes.Equal(evidence.Hash(), ev.Hash()) {
return true
}
}
return false
}
//------------------------------------------ PROTO --------------------------------------
// EvidenceToProto is a generalized function for encoding evidence that conforms to the
// evidence interface to protobuf
func EvidenceToProto(evidence Evidence) (*tmproto.Evidence, error) {
if evidence == nil {
return nil, errors.New("nil evidence")
}
switch evi := evidence.(type) {
case *DuplicateVoteEvidence:
pbev := evi.ToProto()
return &tmproto.Evidence{
Sum: &tmproto.Evidence_DuplicateVoteEvidence{
DuplicateVoteEvidence: pbev,
},
}, nil
case *LightClientAttackEvidence:
pbev, err := evi.ToProto()
if err != nil {
return nil, err
}
return &tmproto.Evidence{
Sum: &tmproto.Evidence_LightClientAttackEvidence{
LightClientAttackEvidence: pbev,
},
}, nil
default:
return nil, fmt.Errorf("toproto: evidence is not recognized: %T", evi)
}
}
// EvidenceFromProto is a generalized function for decoding protobuf into the
// evidence interface
func EvidenceFromProto(evidence *tmproto.Evidence) (Evidence, error) {
if evidence == nil {
return nil, errors.New("nil evidence")
}
switch evi := evidence.Sum.(type) {
case *tmproto.Evidence_DuplicateVoteEvidence:
return DuplicateVoteEvidenceFromProto(evi.DuplicateVoteEvidence)
case *tmproto.Evidence_LightClientAttackEvidence:
return LightClientAttackEvidenceFromProto(evi.LightClientAttackEvidence)
default:
return nil, errors.New("evidence is not recognized")
}
}
func init() {
tmjson.RegisterType(&DuplicateVoteEvidence{}, "tendermint/DuplicateVoteEvidence")
tmjson.RegisterType(&LightClientAttackEvidence{}, "tendermint/LightClientAttackEvidence")
}
//-------------------------------------------- ERRORS --------------------------------------
// ErrInvalidEvidence wraps a piece of evidence and the error denoting how or why it is invalid.
type ErrInvalidEvidence struct {
Evidence Evidence
Reason error
}
// NewErrInvalidEvidence returns a new EvidenceInvalid with the given err.
func NewErrInvalidEvidence(ev Evidence, err error) *ErrInvalidEvidence {
return &ErrInvalidEvidence{ev, err}
}
// Error returns a string representation of the error.
func (err *ErrInvalidEvidence) Error() string {
return fmt.Sprintf("Invalid evidence: %v. Evidence: %v", err.Reason, err.Evidence)
}
// ErrEvidenceOverflow is for when there is too much evidence in a block.
type ErrEvidenceOverflow struct {
MaxNum int
GotNum int
}
// NewErrEvidenceOverflow returns a new ErrEvidenceOverflow where got > max.
func NewErrEvidenceOverflow(max, got int) *ErrEvidenceOverflow {
return &ErrEvidenceOverflow{max, got}
}
// Error returns a string representation of the error.
func (err *ErrEvidenceOverflow) Error() string {
return fmt.Sprintf("Too much evidence: Max %d, got %d", err.MaxNum, err.GotNum)
}
//-------------------------------------------- MOCKING --------------------------------------
// unstable - use only for testing
@ -366,7 +402,7 @@ func NewMockDuplicateVoteEvidenceWithValidator(height int64, time time.Time,
vB := voteB.ToProto()
_ = pv.SignVote(chainID, vB)
voteB.Signature = vB.Signature
return NewDuplicateVoteEvidence(voteA, voteB, time)
return NewDuplicateVoteEvidence(voteA, voteB)
}
func makeMockVote(height int64, round, index int32, addr Address,


+ 138
- 69
types/evidence_test.go View File

@ -12,78 +12,14 @@ import (
"github.com/tendermint/tendermint/crypto/tmhash"
tmrand "github.com/tendermint/tendermint/libs/rand"
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
tmversion "github.com/tendermint/tendermint/proto/tendermint/version"
"github.com/tendermint/tendermint/version"
)
type voteData struct {
vote1 *Vote
vote2 *Vote
valid bool
}
var defaultVoteTime = time.Date(2019, 1, 1, 0, 0, 0, 0, time.UTC)
func TestDuplicateVoteEvidence(t *testing.T) {
val := NewMockPV()
val2 := NewMockPV()
blockID := makeBlockID([]byte("blockhash"), 1000, []byte("partshash"))
blockID2 := makeBlockID([]byte("blockhash2"), 1000, []byte("partshash"))
blockID3 := makeBlockID([]byte("blockhash"), 10000, []byte("partshash"))
blockID4 := makeBlockID([]byte("blockhash"), 10000, []byte("partshash2"))
const chainID = "mychain"
vote1 := makeVote(t, val, chainID, 0, 10, 2, 1, blockID, defaultVoteTime)
v1 := vote1.ToProto()
err := val.SignVote(chainID, v1)
require.NoError(t, err)
badVote := makeVote(t, val, chainID, 0, 10, 2, 1, blockID, defaultVoteTime)
bv := badVote.ToProto()
err = val2.SignVote(chainID, bv)
require.NoError(t, err)
vote1.Signature = v1.Signature
badVote.Signature = bv.Signature
cases := []voteData{
{vote1, makeVote(t, val, chainID, 0, 10, 2, 1, blockID2, defaultVoteTime), true}, // different block ids
{vote1, makeVote(t, val, chainID, 0, 10, 2, 1, blockID3, defaultVoteTime), true},
{vote1, makeVote(t, val, chainID, 0, 10, 2, 1, blockID4, defaultVoteTime), true},
{vote1, makeVote(t, val, chainID, 0, 10, 2, 1, blockID, defaultVoteTime), false}, // wrong block id
{vote1, makeVote(t, val, "mychain2", 0, 10, 2, 1, blockID2, defaultVoteTime), false}, // wrong chain id
{vote1, makeVote(t, val, chainID, 0, 11, 2, 1, blockID2, defaultVoteTime), false}, // wrong height
{vote1, makeVote(t, val, chainID, 0, 10, 3, 1, blockID2, defaultVoteTime), false}, // wrong round
{vote1, makeVote(t, val, chainID, 0, 10, 2, 2, blockID2, defaultVoteTime), false}, // wrong step
{vote1, makeVote(t, val2, chainID, 0, 10, 2, 1, blockID, defaultVoteTime), false}, // wrong validator
{vote1, makeVote(t, val2, chainID, 0, 10, 2, 1, blockID, time.Date(2020, 1, 1, 0, 0, 0, 0, time.UTC)), false},
{vote1, badVote, false}, // signed by wrong key
}
pubKey, err := val.GetPubKey()
require.NoError(t, err)
for _, c := range cases {
ev := &DuplicateVoteEvidence{
VoteA: c.vote1,
VoteB: c.vote2,
Timestamp: defaultVoteTime,
}
if c.valid {
assert.Nil(t, ev.Verify(chainID, pubKey), "evidence should be valid")
} else {
assert.NotNil(t, ev.Verify(chainID, pubKey), "evidence should be invalid")
}
}
ev := randomDuplicatedVoteEvidence(t)
assert.True(t, ev.Equal(ev))
assert.False(t, ev.Equal(&DuplicateVoteEvidence{}))
}
func TestEvidenceList(t *testing.T) {
ev := randomDuplicatedVoteEvidence(t)
ev := randomDuplicateVoteEvidence(t)
evl := EvidenceList([]Evidence{ev})
assert.NotNil(t, evl.Hash())
@ -122,7 +58,7 @@ func TestMaxEvidenceBytes(t *testing.T) {
}
func randomDuplicatedVoteEvidence(t *testing.T) *DuplicateVoteEvidence {
func randomDuplicateVoteEvidence(t *testing.T) *DuplicateVoteEvidence {
val := NewMockPV()
blockID := makeBlockID([]byte("blockhash"), 1000, []byte("partshash"))
blockID2 := makeBlockID([]byte("blockhash2"), 1000, []byte("partshash"))
@ -133,6 +69,14 @@ func randomDuplicatedVoteEvidence(t *testing.T) *DuplicateVoteEvidence {
}
}
func TestDuplicateVoteEvidence(t *testing.T) {
const height = int64(13)
ev := NewMockDuplicateVoteEvidence(height, time.Now(), "mock-chain-id")
assert.Equal(t, ev.Hash(), tmhash.Sum(ev.Bytes()))
assert.NotNil(t, ev.String())
assert.Equal(t, ev.Height(), height)
}
func TestDuplicateVoteEvidenceValidation(t *testing.T) {
val := NewMockPV()
blockID := makeBlockID(tmhash.Sum([]byte("blockhash")), math.MaxInt32, tmhash.Sum([]byte("partshash")))
@ -165,13 +109,137 @@ func TestDuplicateVoteEvidenceValidation(t *testing.T) {
t.Run(tc.testName, func(t *testing.T) {
vote1 := makeVote(t, val, chainID, math.MaxInt32, math.MaxInt64, math.MaxInt32, 0x02, blockID, defaultVoteTime)
vote2 := makeVote(t, val, chainID, math.MaxInt32, math.MaxInt64, math.MaxInt32, 0x02, blockID2, defaultVoteTime)
ev := NewDuplicateVoteEvidence(vote1, vote2, vote1.Timestamp)
ev := NewDuplicateVoteEvidence(vote1, vote2)
tc.malleateEvidence(ev)
assert.Equal(t, tc.expectErr, ev.ValidateBasic() != nil, "Validate Basic had an unexpected result")
})
}
}
func TestLightClientAttackEvidence(t *testing.T) {
height := int64(5)
voteSet, valSet, privVals := randVoteSet(height, 1, tmproto.PrecommitType, 10, 1)
header := makeHeaderRandom()
header.Height = height
blockID := makeBlockID(tmhash.Sum([]byte("blockhash")), math.MaxInt32, tmhash.Sum([]byte("partshash")))
commit, err := MakeCommit(blockID, height, 1, voteSet, privVals, defaultVoteTime)
require.NoError(t, err)
lcae := &LightClientAttackEvidence{
ConflictingBlock: &LightBlock{
SignedHeader: &SignedHeader{
Header: header,
Commit: commit,
},
ValidatorSet: valSet,
},
CommonHeight: height - 1,
}
assert.NotNil(t, lcae.String())
assert.NotNil(t, lcae.Hash())
// only 7 validators sign
differentCommit, err := MakeCommit(blockID, height, 1, voteSet, privVals[:7], defaultVoteTime)
require.NoError(t, err)
differentEv := &LightClientAttackEvidence{
ConflictingBlock: &LightBlock{
SignedHeader: &SignedHeader{
Header: header,
Commit: differentCommit,
},
ValidatorSet: valSet,
},
CommonHeight: height - 1,
}
assert.Equal(t, lcae.Hash(), differentEv.Hash())
// different header hash
differentHeader := makeHeaderRandom()
differentEv = &LightClientAttackEvidence{
ConflictingBlock: &LightBlock{
SignedHeader: &SignedHeader{
Header: differentHeader,
Commit: differentCommit,
},
ValidatorSet: valSet,
},
CommonHeight: height - 1,
}
assert.NotEqual(t, lcae.Hash(), differentEv.Hash())
// different common height should produce a different header
differentEv = &LightClientAttackEvidence{
ConflictingBlock: &LightBlock{
SignedHeader: &SignedHeader{
Header: header,
Commit: differentCommit,
},
ValidatorSet: valSet,
},
CommonHeight: height - 2,
}
assert.NotEqual(t, lcae.Hash(), differentEv.Hash())
assert.Equal(t, lcae.Height(), int64(4)) // Height should be the common Height
assert.NotNil(t, lcae.Bytes())
}
func TestLightClientAttackEvidenceValidation(t *testing.T) {
height := int64(5)
voteSet, valSet, privVals := randVoteSet(height, 1, tmproto.PrecommitType, 10, 1)
header := makeHeaderRandom()
header.Height = height
header.ValidatorsHash = valSet.Hash()
blockID := makeBlockID(header.Hash(), math.MaxInt32, tmhash.Sum([]byte("partshash")))
commit, err := MakeCommit(blockID, height, 1, voteSet, privVals, time.Now())
require.NoError(t, err)
lcae := &LightClientAttackEvidence{
ConflictingBlock: &LightBlock{
SignedHeader: &SignedHeader{
Header: header,
Commit: commit,
},
ValidatorSet: valSet,
},
CommonHeight: height - 1,
}
assert.NoError(t, lcae.ValidateBasic())
testCases := []struct {
testName string
malleateEvidence func(*LightClientAttackEvidence)
expectErr bool
}{
{"Good DuplicateVoteEvidence", func(ev *LightClientAttackEvidence) {}, false},
{"Negative height", func(ev *LightClientAttackEvidence) { ev.CommonHeight = -10 }, true},
{"Height is greater than divergent block", func(ev *LightClientAttackEvidence) {
ev.CommonHeight = height + 1
}, true},
{"Nil conflicting header", func(ev *LightClientAttackEvidence) { ev.ConflictingBlock.Header = nil }, true},
{"Nil conflicting blocl", func(ev *LightClientAttackEvidence) { ev.ConflictingBlock = nil }, true},
{"Nil validator set", func(ev *LightClientAttackEvidence) {
ev.ConflictingBlock.ValidatorSet = &ValidatorSet{}
}, true},
}
for _, tc := range testCases {
tc := tc
t.Run(tc.testName, func(t *testing.T) {
lcae := &LightClientAttackEvidence{
ConflictingBlock: &LightBlock{
SignedHeader: &SignedHeader{
Header: header,
Commit: commit,
},
ValidatorSet: valSet,
},
CommonHeight: height - 1,
}
tc.malleateEvidence(lcae)
if tc.expectErr {
assert.Error(t, lcae.ValidateBasic(), tc.testName)
} else {
assert.NoError(t, lcae.ValidateBasic(), tc.testName)
}
})
}
}
func TestMockEvidenceValidateBasic(t *testing.T) {
goodEvidence := NewMockDuplicateVoteEvidence(int64(1), time.Now(), "mock-chain-id")
assert.Nil(t, goodEvidence.ValidateBasic())
@ -203,6 +271,7 @@ func makeVote(
func makeHeaderRandom() *Header {
return &Header{
Version: tmversion.Consensus{Block: version.BlockProtocol, App: 1},
ChainID: tmrand.Str(12),
Height: int64(tmrand.Uint16()) + 1,
Time: time.Now(),


+ 2
- 8
types/protobuf.go View File

@ -112,27 +112,21 @@ func (tm2pb) ConsensusParams(params *tmproto.ConsensusParams) *abci.ConsensusPar
// so Evidence types stays compact.
// XXX: panics on nil or unknown pubkey type
func (tm2pb) Evidence(ev Evidence, valSet *ValidatorSet) abci.Evidence {
addr := ev.Address()
_, val := valSet.GetByAddress(addr)
if val == nil {
// should already have checked this
panic(fmt.Sprintf("validator in evidence is not in val set, val addr: %v", addr))
}
// set type
var evType abci.EvidenceType
switch ev.(type) {
case *DuplicateVoteEvidence:
evType = abci.EvidenceType_DUPLICATE_VOTE
case *LightClientAttackEvidence:
evType = abci.EvidenceType_LIGHT_CLIENT_ATTACK
default:
panic(fmt.Sprintf("unknown evidence type: %v %v", ev, reflect.TypeOf(ev)))
}
return abci.Evidence{
Type: evType,
Validator: TM2PB.Validator(val),
Height: ev.Height(),
Time: ev.Time(),
TotalVotingPower: valSet.TotalVotingPower(),
}
}


+ 0
- 2
types/protobuf_test.go View File

@ -77,8 +77,6 @@ func TestABCIEvidence(t *testing.T) {
)
assert.Equal(t, abci.EvidenceType_DUPLICATE_VOTE, abciEv.Type)
assert.Equal(t, ev.Time(), abciEv.GetTime())
assert.Equal(t, ev.Address(), abciEv.Validator.GetAddress())
assert.Equal(t, ev.Height(), abciEv.GetHeight())
}


Loading…
Cancel
Save