diff --git a/blockchain/v0/reactor_test.go b/blockchain/v0/reactor_test.go index a31c9a141..bddbb3f3b 100644 --- a/blockchain/v0/reactor_test.go +++ b/blockchain/v0/reactor_test.go @@ -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 diff --git a/blockchain/v1/reactor_test.go b/blockchain/v1/reactor_test.go index 0e6f21222..a82b2211c 100644 --- a/blockchain/v1/reactor_test.go +++ b/blockchain/v1/reactor_test.go @@ -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 diff --git a/blockchain/v2/reactor_test.go b/blockchain/v2/reactor_test.go index 3cc3c4426..aaa5c979d 100644 --- a/blockchain/v2/reactor_test.go +++ b/blockchain/v2/reactor_test.go @@ -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 diff --git a/consensus/common_test.go b/consensus/common_test.go index f8055f01c..e98c66837 100644 --- a/consensus/common_test.go +++ b/consensus/common_test.go @@ -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 diff --git a/consensus/reactor_test.go b/consensus/reactor_test.go index aee222bbd..3c3eef7ed 100644 --- a/consensus/reactor_test.go +++ b/consensus/reactor_test.go @@ -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 +} //------------------------------------ diff --git a/consensus/replay.go b/consensus/replay.go index 3d9d6614b..0245a5341 100644 --- a/consensus/replay.go +++ b/consensus/replay.go @@ -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 diff --git a/consensus/replay_file.go b/consensus/replay_file.go index b8b8c51da..e27584ba0 100644 --- a/consensus/replay_file.go +++ b/consensus/replay_file.go @@ -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, diff --git a/consensus/replay_test.go b/consensus/replay_test.go index f886cdeeb..7fd4f9e6d 100644 --- a/consensus/replay_test.go +++ b/consensus/replay_test.go @@ -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 ) diff --git a/consensus/wal_generator.go b/consensus/wal_generator.go index 422c3f73b..f14d11db3 100644 --- a/consensus/wal_generator.go +++ b/consensus/wal_generator.go @@ -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) diff --git a/evidence/mock/pool_mock.go b/evidence/mock/pool_mock.go new file mode 100644 index 000000000..9ba15807f --- /dev/null +++ b/evidence/mock/pool_mock.go @@ -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}) +} diff --git a/state/execution_test.go b/state/execution_test.go index 3af90f8a0..d8443100f 100644 --- a/state/execution_test.go +++ b/state/execution_test.go @@ -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) diff --git a/state/helpers_test.go b/state/helpers_test.go index a85e35748..2805af63c 100644 --- a/state/helpers_test.go +++ b/state/helpers_test.go @@ -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) diff --git a/state/services.go b/state/services.go index a30956bdc..df6ce8f8f 100644 --- a/state/services.go +++ b/state/services.go @@ -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 } diff --git a/state/validation.go b/state/validation.go index 18608ab2c..c3231c270 100644 --- a/state/validation.go +++ b/state/validation.go @@ -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 diff --git a/state/validation_test.go b/state/validation_test.go index 10ff4c8ea..8cc7f44a7 100644 --- a/state/validation_test.go +++ b/state/validation_test.go @@ -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) +//}