Browse Source

evidence: check evidence is pending before validating evidence

checks evidence store first to potentially save the work of having to verify again

fixes #4728
pull/4774/head
Callum Waters 4 years ago
committed by GitHub
parent
commit
fd71c6844f
No known key found for this signature in database GPG Key ID: 4AEE18F83AFDEB23
15 changed files with 226 additions and 37 deletions
  1. +2
    -1
      blockchain/v0/reactor_test.go
  2. +2
    -1
      blockchain/v1/reactor_test.go
  3. +3
    -2
      blockchain/v2/reactor_test.go
  4. +2
    -1
      consensus/common_test.go
  5. +10
    -0
      consensus/reactor_test.go
  6. +2
    -1
      consensus/replay.go
  7. +2
    -1
      consensus/replay_file.go
  8. +2
    -1
      consensus/replay_test.go
  9. +2
    -1
      consensus/wal_generator.go
  10. +124
    -0
      evidence/mock/pool_mock.go
  11. +4
    -3
      state/execution_test.go
  12. +0
    -8
      state/helpers_test.go
  13. +1
    -8
      state/services.go
  14. +8
    -3
      state/validation.go
  15. +62
    -6
      state/validation_test.go

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

@ -13,6 +13,7 @@ import (
abci "github.com/tendermint/tendermint/abci/types"
cfg "github.com/tendermint/tendermint/config"
evmock "github.com/tendermint/tendermint/evidence/mock"
"github.com/tendermint/tendermint/libs/log"
"github.com/tendermint/tendermint/mock"
"github.com/tendermint/tendermint/p2p"
@ -82,7 +83,7 @@ func newBlockchainReactor(
fastSync := true
db := dbm.NewMemDB()
blockExec := sm.NewBlockExecutor(db, log.TestingLogger(), proxyApp.Consensus(),
mock.Mempool{}, sm.MockEvidencePool{})
mock.Mempool{}, evmock.NewDefaultEvidencePool())
sm.SaveState(db, state)
// let's add some blocks in


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

@ -16,6 +16,7 @@ import (
abci "github.com/tendermint/tendermint/abci/types"
cfg "github.com/tendermint/tendermint/config"
evmock "github.com/tendermint/tendermint/evidence/mock"
"github.com/tendermint/tendermint/libs/log"
"github.com/tendermint/tendermint/mock"
"github.com/tendermint/tendermint/p2p"
@ -112,7 +113,7 @@ func newBlockchainReactor(
fastSync := true
db := dbm.NewMemDB()
blockExec := sm.NewBlockExecutor(db, log.TestingLogger(), proxyApp.Consensus(),
mock.Mempool{}, sm.MockEvidencePool{})
mock.Mempool{}, evmock.NewDefaultEvidencePool())
sm.SaveState(db, state)
// let's add some blocks in


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

@ -15,6 +15,7 @@ import (
abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/behaviour"
cfg "github.com/tendermint/tendermint/config"
evmock "github.com/tendermint/tendermint/evidence/mock"
"github.com/tendermint/tendermint/libs/log"
"github.com/tendermint/tendermint/libs/service"
"github.com/tendermint/tendermint/mock"
@ -152,7 +153,7 @@ func newTestReactor(p testReactorParams) *BlockchainReactor {
panic(errors.Wrap(err, "error start app"))
}
db := dbm.NewMemDB()
appl = sm.NewBlockExecutor(db, p.logger, proxyApp.Consensus(), mock.Mempool{}, sm.MockEvidencePool{})
appl = sm.NewBlockExecutor(db, p.logger, proxyApp.Consensus(), mock.Mempool{}, evmock.NewDefaultEvidencePool())
sm.SaveState(db, state)
}
@ -494,7 +495,7 @@ func newReactorStore(
db := dbm.NewMemDB()
blockExec := sm.NewBlockExecutor(db, log.TestingLogger(), proxyApp.Consensus(),
mock.Mempool{}, sm.MockEvidencePool{})
mock.Mempool{}, evmock.NewDefaultEvidencePool())
sm.SaveState(db, state)
// add blocks in


+ 2
- 1
consensus/common_test.go View File

@ -26,6 +26,7 @@ import (
abci "github.com/tendermint/tendermint/abci/types"
cfg "github.com/tendermint/tendermint/config"
cstypes "github.com/tendermint/tendermint/consensus/types"
evmock "github.com/tendermint/tendermint/evidence/mock"
tmbytes "github.com/tendermint/tendermint/libs/bytes"
"github.com/tendermint/tendermint/libs/log"
tmos "github.com/tendermint/tendermint/libs/os"
@ -373,7 +374,7 @@ func newStateWithConfigAndBlockStore(
}
// mock the evidence pool
evpool := sm.MockEvidencePool{}
evpool := evmock.NewDefaultEvidencePool()
// Make State
stateDB := blockDB


+ 10
- 0
consensus/reactor_test.go View File

@ -225,6 +225,16 @@ func (m *mockEvidencePool) Update(block *types.Block, state sm.State) {
m.height++
}
func (m *mockEvidencePool) IsCommitted(types.Evidence) bool { return false }
func (m *mockEvidencePool) IsPending(evidence types.Evidence) bool {
if m.height > 0 {
for _, e := range m.ev {
if e.Equal(evidence) {
return true
}
}
}
return false
}
//------------------------------------


+ 2
- 1
consensus/replay.go View File

@ -15,6 +15,7 @@ import (
//auto "github.com/tendermint/tendermint/libs/autofile"
dbm "github.com/tendermint/tm-db"
evmock "github.com/tendermint/tendermint/evidence/mock"
"github.com/tendermint/tendermint/libs/log"
"github.com/tendermint/tendermint/mock"
"github.com/tendermint/tendermint/proxy"
@ -473,7 +474,7 @@ func (h *Handshaker) replayBlock(state sm.State, height int64, proxyApp proxy.Ap
block := h.store.LoadBlock(height)
meta := h.store.LoadBlockMeta(height)
blockExec := sm.NewBlockExecutor(h.stateDB, h.logger, proxyApp, mock.Mempool{}, sm.MockEvidencePool{})
blockExec := sm.NewBlockExecutor(h.stateDB, h.logger, proxyApp, mock.Mempool{}, evmock.NewDefaultEvidencePool())
blockExec.SetEventBus(h.eventBus)
var err error


+ 2
- 1
consensus/replay_file.go View File

@ -13,6 +13,7 @@ import (
dbm "github.com/tendermint/tm-db"
cfg "github.com/tendermint/tendermint/config"
evmock "github.com/tendermint/tendermint/evidence/mock"
"github.com/tendermint/tendermint/libs/log"
tmos "github.com/tendermint/tendermint/libs/os"
"github.com/tendermint/tendermint/mock"
@ -311,7 +312,7 @@ func newConsensusStateForReplay(config cfg.BaseConfig, csConfig *cfg.ConsensusCo
tmos.Exit(fmt.Sprintf("Error on handshake: %v", err))
}
mempool, evpool := mock.Mempool{}, sm.MockEvidencePool{}
mempool, evpool := mock.Mempool{}, evmock.NewDefaultEvidencePool()
blockExec := sm.NewBlockExecutor(stateDB, log.TestingLogger(), proxyApp.Consensus(), mempool, evpool)
consensusState := NewState(csConfig, state.Copy(), blockExec,


+ 2
- 1
consensus/replay_test.go View File

@ -23,6 +23,7 @@ import (
abci "github.com/tendermint/tendermint/abci/types"
cfg "github.com/tendermint/tendermint/config"
"github.com/tendermint/tendermint/crypto"
evmock "github.com/tendermint/tendermint/evidence/mock"
"github.com/tendermint/tendermint/libs/log"
tmrand "github.com/tendermint/tendermint/libs/rand"
mempl "github.com/tendermint/tendermint/mempool"
@ -296,7 +297,7 @@ const (
var (
mempool = mock.Mempool{}
evpool = sm.MockEvidencePool{}
evpool = evmock.NewDefaultEvidencePool()
sim testSim
)


+ 2
- 1
consensus/wal_generator.go View File

@ -15,6 +15,7 @@ import (
"github.com/tendermint/tendermint/abci/example/kvstore"
cfg "github.com/tendermint/tendermint/config"
evmock "github.com/tendermint/tendermint/evidence/mock"
"github.com/tendermint/tendermint/libs/log"
tmrand "github.com/tendermint/tendermint/libs/rand"
"github.com/tendermint/tendermint/mock"
@ -73,7 +74,7 @@ func WALGenerateNBlocks(t *testing.T, wr io.Writer, numBlocks int) (err error) {
}
defer eventBus.Stop()
mempool := mock.Mempool{}
evpool := sm.MockEvidencePool{}
evpool := evmock.NewDefaultEvidencePool()
blockExec := sm.NewBlockExecutor(stateDB, log.TestingLogger(), proxyApp.Consensus(), mempool, evpool)
consensusState := NewState(config.Consensus, state.Copy(), blockExec, blockStore, mempool, evpool)
consensusState.SetLogger(logger)


+ 124
- 0
evidence/mock/pool_mock.go View File

@ -0,0 +1,124 @@
package mock
import (
"time"
sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/types"
)
// A quick and easy in-memory mock implementation of the evidence pool configured with both the standard exposed
// functions and extra helper function to set up, operate and ultimately imitate the evidence pool in a testing
// environment. Note: this mock does no validation. Do not test with large amounts of evidence
type EvidencePool struct {
PendingEvidenceList []types.Evidence
CommittedEvidenceList []types.Evidence
ExpirationAgeTime time.Duration
ExpirationAgeBlock int64
BlockHeight int64
BlockTime time.Time
}
// ------------------------ INSTANTIATION METHODS --------------------------------------
func NewEvidencePool(height, expiryHeight int64, currentTime time.Time, expiryTime time.Duration) *EvidencePool {
return &EvidencePool{
[]types.Evidence{},
[]types.Evidence{},
expiryTime,
expiryHeight,
height,
currentTime,
}
}
func NewDefaultEvidencePool() *EvidencePool {
return NewEvidencePool(1, 1, time.Now(), time.Millisecond)
}
// ----------------------- EVIDENCE POOL PUBLIC METHODS --------------------------------
func (p *EvidencePool) PendingEvidence(maxNum int64) []types.Evidence {
if maxNum == -1 || maxNum >= int64(len(p.PendingEvidenceList)) {
return p.PendingEvidenceList
}
return p.PendingEvidenceList[:maxNum]
}
func (p *EvidencePool) AddEvidence(evidence types.Evidence) error {
p.PendingEvidenceList = append(p.PendingEvidenceList, evidence)
return nil
}
func (p *EvidencePool) Update(block *types.Block, state sm.State) {
p.BlockHeight = block.Height
p.BlockTime = block.Time
p.MarkEvidenceAsCommitted(block.Height, block.Time, block.Evidence.Evidence)
p.RemoveExpiredEvidence()
}
func (p *EvidencePool) MarkEvidenceAsCommitted(height int64, lastBlockTime time.Time, evidence []types.Evidence) {
for _, ev := range evidence {
if p.IsPending(ev) {
p.RemovePendingEvidence(ev)
}
p.CommittedEvidenceList = append(p.CommittedEvidenceList, ev)
}
}
func (p *EvidencePool) IsPending(evidence types.Evidence) bool {
for _, ev := range p.PendingEvidenceList {
if ev.Equal(evidence) {
return true
}
}
return false
}
func (p *EvidencePool) IsCommitted(evidence types.Evidence) bool {
for _, ev := range p.CommittedEvidenceList {
if ev.Equal(evidence) {
return true
}
}
return false
}
func (p *EvidencePool) IsExpired(evidence types.Evidence) bool {
return evidence.Height()+p.ExpirationAgeBlock < p.BlockHeight &&
evidence.Time().Add(p.ExpirationAgeTime).Before(p.BlockTime)
}
// ------------------------------- HELPER METHODS --------------------------------------
func (p *EvidencePool) RemovePendingEvidence(evidence types.Evidence) {
for idx, ev := range p.PendingEvidenceList {
if ev.Equal(evidence) {
p.PendingEvidenceList[idx] = p.PendingEvidenceList[len(p.PendingEvidenceList)-1]
p.PendingEvidenceList = p.PendingEvidenceList[:len(p.PendingEvidenceList)-1]
return
}
}
}
func (p *EvidencePool) RemoveExpiredEvidence() {
for _, evidence := range p.PendingEvidenceList {
if p.IsExpired(evidence) {
p.RemovePendingEvidence(evidence)
}
}
}
func (p *EvidencePool) AddMockEvidence(height int64, address []byte) types.Evidence {
mock := types.MockEvidence{
EvidenceHeight: height,
EvidenceTime: p.BlockTime,
EvidenceAddress: address,
}
_ = p.AddEvidence(mock)
return mock
}
func (p *EvidencePool) CommitEvidence(evidence types.Evidence) {
p.MarkEvidenceAsCommitted(evidence.Height(), evidence.Time(), []types.Evidence{evidence})
}

+ 4
- 3
state/execution_test.go View File

@ -12,6 +12,7 @@ import (
abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/crypto/ed25519"
"github.com/tendermint/tendermint/crypto/secp256k1"
evmock "github.com/tendermint/tendermint/evidence/mock"
"github.com/tendermint/tendermint/libs/log"
"github.com/tendermint/tendermint/mock"
"github.com/tendermint/tendermint/proxy"
@ -38,7 +39,7 @@ func TestApplyBlock(t *testing.T) {
state, stateDB, _ := makeState(1, 1)
blockExec := sm.NewBlockExecutor(stateDB, log.TestingLogger(), proxyApp.Consensus(),
mock.Mempool{}, sm.MockEvidencePool{})
mock.Mempool{}, evmock.NewDefaultEvidencePool())
block := makeBlock(state, 1)
blockID := types.BlockID{Hash: block.Hash(), PartsHeader: block.MakePartSet(testPartSize).Header()}
@ -334,7 +335,7 @@ func TestEndBlockValidatorUpdates(t *testing.T) {
log.TestingLogger(),
proxyApp.Consensus(),
mock.Mempool{},
sm.MockEvidencePool{},
evmock.NewDefaultEvidencePool(),
)
eventBus := types.NewEventBus()
@ -401,7 +402,7 @@ func TestEndBlockValidatorUpdatesResultingInEmptySet(t *testing.T) {
log.TestingLogger(),
proxyApp.Consensus(),
mock.Mempool{},
sm.MockEvidencePool{},
evmock.NewDefaultEvidencePool(),
)
block := makeBlock(state, 1)


+ 0
- 8
state/helpers_test.go View File

@ -22,14 +22,6 @@ type paramsChangeTestCase struct {
params types.ConsensusParams
}
// always returns true if asked if any evidence was already committed.
type mockEvPoolAlwaysCommitted struct{}
func (m mockEvPoolAlwaysCommitted) PendingEvidence(int64) []types.Evidence { return nil }
func (m mockEvPoolAlwaysCommitted) AddEvidence(types.Evidence) error { return nil }
func (m mockEvPoolAlwaysCommitted) Update(*types.Block, sm.State) {}
func (m mockEvPoolAlwaysCommitted) IsCommitted(types.Evidence) bool { return true }
func newTestApp() proxy.AppConns {
app := &testApp{}
cc := proxy.NewLocalClientCreator(app)


+ 1
- 8
state/services.go View File

@ -43,12 +43,5 @@ type EvidencePool interface {
Update(*types.Block, State)
// IsCommitted indicates if this evidence was already marked committed in another block.
IsCommitted(types.Evidence) bool
IsPending(types.Evidence) bool
}
// MockEvidencePool is an empty implementation of EvidencePool, useful for testing.
type MockEvidencePool struct{}
func (m MockEvidencePool) PendingEvidence(int64) []types.Evidence { return nil }
func (m MockEvidencePool) AddEvidence(types.Evidence) error { return nil }
func (m MockEvidencePool) Update(*types.Block, State) {}
func (m MockEvidencePool) IsCommitted(types.Evidence) bool { return false }

+ 8
- 3
state/validation.go View File

@ -132,12 +132,17 @@ func validateBlock(evidencePool EvidencePool, stateDB dbm.DB, state State, block
// Validate all evidence.
for _, ev := range block.Evidence.Evidence {
if evidencePool != nil {
if evidencePool.IsCommitted(ev) {
return types.NewErrEvidenceInvalid(ev, errors.New("evidence was already committed"))
}
if evidencePool.IsPending(ev) {
continue
}
}
if err := VerifyEvidence(stateDB, state, ev, &block.Header); err != nil {
return types.NewErrEvidenceInvalid(ev, err)
}
if evidencePool != nil && evidencePool.IsCommitted(ev) {
return types.NewErrEvidenceInvalid(ev, errors.New("evidence was already committed"))
}
}
// NOTE: We can't actually verify it's the right proposer because we dont


+ 62
- 6
state/validation_test.go View File

@ -8,6 +8,7 @@ import (
"github.com/tendermint/tendermint/crypto/ed25519"
"github.com/tendermint/tendermint/crypto/tmhash"
evmock "github.com/tendermint/tendermint/evidence/mock"
"github.com/tendermint/tendermint/libs/log"
"github.com/tendermint/tendermint/mock"
sm "github.com/tendermint/tendermint/state"
@ -28,7 +29,7 @@ func TestValidateBlockHeader(t *testing.T) {
log.TestingLogger(),
proxyApp.Consensus(),
mock.Mempool{},
sm.MockEvidencePool{},
evmock.NewDefaultEvidencePool(),
)
lastCommit := types.NewCommit(0, 0, types.BlockID{}, nil)
@ -98,7 +99,7 @@ func TestValidateBlockCommit(t *testing.T) {
log.TestingLogger(),
proxyApp.Consensus(),
mock.Mempool{},
sm.MockEvidencePool{},
evmock.NewDefaultEvidencePool(),
)
lastCommit := types.NewCommit(0, 0, types.BlockID{}, nil)
wrongSigsCommit := types.NewCommit(1, 0, types.BlockID{}, nil)
@ -205,7 +206,7 @@ func TestValidateBlockEvidence(t *testing.T) {
log.TestingLogger(),
proxyApp.Consensus(),
mock.Mempool{},
sm.MockEvidencePool{},
evmock.NewDefaultEvidencePool(),
)
lastCommit := types.NewCommit(0, 0, types.BlockID{}, nil)
@ -260,15 +261,70 @@ func TestValidateFailBlockOnCommittedEvidence(t *testing.T) {
var height int64 = 1
state, stateDB, _ := makeState(1, int(height))
blockExec := sm.NewBlockExecutor(stateDB, log.TestingLogger(), nil, nil, mockEvPoolAlwaysCommitted{})
evpool := evmock.NewDefaultEvidencePool()
blockExec := sm.NewBlockExecutor(
stateDB, log.TestingLogger(),
nil,
nil,
evpool)
// A block with a couple pieces of evidence passes.
block := makeBlock(state, height)
addr, _ := state.Validators.GetByIndex(0)
alreadyCommittedEvidence := types.NewMockEvidence(height, time.Now(), addr)
block.Evidence.Evidence = []types.Evidence{alreadyCommittedEvidence}
evpool.CommitEvidence(evpool.AddMockEvidence(height, addr))
block.Evidence.Evidence = evpool.CommittedEvidenceList
block.EvidenceHash = block.Evidence.Hash()
err := blockExec.ValidateBlock(state, block)
require.Error(t, err)
require.IsType(t, err, &types.ErrEvidenceInvalid{})
}
func TestValidateAlreadyPendingEvidence(t *testing.T) {
var height int64 = 1
state, stateDB, _ := makeState(2, int(height))
evpool := evmock.NewDefaultEvidencePool()
blockExec := sm.NewBlockExecutor(
stateDB, log.TestingLogger(),
nil,
nil,
evpool)
// A block with a couple pieces of evidence passes.
block := makeBlock(state, height)
addr, _ := state.Validators.GetByIndex(0)
addr2, _ := state.Validators.GetByIndex(0)
// add pending evidence
pendingEv := evpool.AddMockEvidence(height, addr)
// add evidence that hasn't seen before
ev := types.NewMockEvidence(height, time.Now(), addr2)
block.Evidence.Evidence = []types.Evidence{pendingEv, ev}
block.EvidenceHash = block.Evidence.Hash()
err := blockExec.ValidateBlock(state, block)
require.NoError(t, err)
}
// TODO: prevent committing duplicate votes
//func TestValidateDuplicateEvidenceShouldFail(t *testing.T) {
// var height int64 = 1
// var evidence []types.Evidence
// state, stateDB, _ := makeState(1, int(height))
//
// evpool := evmock.NewDefaultEvidencePool()
// blockExec := sm.NewBlockExecutor(
// stateDB, log.TestingLogger(),
// nil,
// nil,
// evpool)
// // A block with a couple pieces of evidence passes.
// block := makeBlock(state, height)
// addr, _ := state.Validators.GetByIndex(0)
// for i := 0; i < 2; i++ {
// evidence = append(evidence, types.NewMockEvidence(height, time.Now(), addr))
// }
// block.Evidence.Evidence = evidence
// block.EvidenceHash = block.Evidence.Hash()
// err := blockExec.ValidateBlock(state, block)
//
// require.Error(t, err)
//}

Loading…
Cancel
Save