mirror of
https://github.com/tendermint/tendermint.git
synced 2025-12-23 06:15:19 +00:00
evidence: don't send committed evidence and ignore inbound evidence that is already committed (#5574)
This commit is contained in:
@@ -83,7 +83,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.
|
||||
func (evpool *Pool) PendingEvidence(maxBytes int64) ([]types.Evidence, int64) {
|
||||
if atomic.LoadUint32(&evpool.evidenceSize) == 0 {
|
||||
if evpool.Size() == 0 {
|
||||
return []types.Evidence{}, 0
|
||||
}
|
||||
evidence, size, err := evpool.listEvidence(baseKeyPending, maxBytes)
|
||||
@@ -110,7 +110,7 @@ func (evpool *Pool) Update(state sm.State) {
|
||||
evpool.updateState(state)
|
||||
|
||||
// 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 &&
|
||||
if evpool.Size() > 0 && state.LastBlockHeight > evpool.pruningHeight &&
|
||||
state.LastBlockTime.After(evpool.pruningTime) {
|
||||
evpool.pruningHeight, evpool.pruningTime = evpool.removeExpiredPendingEvidence()
|
||||
}
|
||||
@@ -126,6 +126,14 @@ func (evpool *Pool) AddEvidence(ev types.Evidence) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
// check that the evidence isn't already committed
|
||||
if evpool.isCommitted(ev) {
|
||||
// this can happen if the peer that sent us the evidence is behind so we shouldn't
|
||||
// punish the peer.
|
||||
evpool.logger.Debug("Evidence was already committed, ignoring this one", "ev", ev)
|
||||
return nil
|
||||
}
|
||||
|
||||
// 1) Verify against state.
|
||||
evInfo, err := evpool.verify(ev)
|
||||
if err != nil {
|
||||
@@ -197,12 +205,19 @@ func (evpool *Pool) CheckEvidence(evList types.EvidenceList) error {
|
||||
ok := evpool.fastCheck(ev)
|
||||
|
||||
if !ok {
|
||||
// check that the evidence isn't already committed
|
||||
if evpool.isCommitted(ev) {
|
||||
return &types.ErrInvalidEvidence{Evidence: ev, Reason: errors.New("evidence was already committed")}
|
||||
}
|
||||
|
||||
evInfo, err := evpool.verify(ev)
|
||||
if err != nil {
|
||||
return &types.ErrInvalidEvidence{Evidence: ev, Reason: err}
|
||||
}
|
||||
|
||||
if err := evpool.addPendingEvidence(evInfo); err != nil {
|
||||
// Something went wrong with adding the evidence but we already know it is valid
|
||||
// hence we log an error and continue
|
||||
evpool.logger.Error("Can't add evidence to pending list", "err", err, "evInfo", evInfo)
|
||||
}
|
||||
|
||||
@@ -315,6 +330,10 @@ func (evpool *Pool) SetLogger(l log.Logger) {
|
||||
evpool.logger = l
|
||||
}
|
||||
|
||||
func (evpool *Pool) Size() uint32 {
|
||||
return atomic.LoadUint32(&evpool.evidenceSize)
|
||||
}
|
||||
|
||||
// State returns the current state of the evpool.
|
||||
func (evpool *Pool) State() sm.State {
|
||||
evpool.mtx.Lock()
|
||||
|
||||
@@ -17,8 +17,13 @@ const (
|
||||
|
||||
maxMsgSize = 1048576 // 1MB TODO make it configurable
|
||||
|
||||
broadcastEvidenceIntervalS = 60 // broadcast uncommitted evidence this often
|
||||
peerCatchupSleepIntervalMS = 100 // If peer is behind, sleep this amount
|
||||
// broadcast all uncommitted evidence this often. This sets when the reactor
|
||||
// goes back to the start of the list and begins sending the evidence again.
|
||||
// Most evidence should be committed in the very next block that is why we wait
|
||||
// just over the block production rate before sending evidence again.
|
||||
broadcastEvidenceIntervalS = 10
|
||||
// If a message fails wait this much before sending it again
|
||||
peerRetryMessageIntervalMS = 100
|
||||
)
|
||||
|
||||
// Reactor handles evpool evidence broadcasting amongst peers.
|
||||
@@ -117,7 +122,7 @@ func (evR *Reactor) broadcastEvidenceRoutine(peer p2p.Peer) {
|
||||
}
|
||||
|
||||
ev := next.Value.(types.Evidence)
|
||||
evis, retry := evR.checkSendEvidenceMessage(peer, ev)
|
||||
evis := evR.prepareEvidenceMessage(peer, ev)
|
||||
if len(evis) > 0 {
|
||||
msgBytes, err := encodeMsg(evis)
|
||||
if err != nil {
|
||||
@@ -125,12 +130,10 @@ func (evR *Reactor) broadcastEvidenceRoutine(peer p2p.Peer) {
|
||||
}
|
||||
|
||||
success := peer.Send(EvidenceChannel, msgBytes)
|
||||
retry = !success
|
||||
}
|
||||
|
||||
if retry {
|
||||
time.Sleep(peerCatchupSleepIntervalMS * time.Millisecond)
|
||||
continue
|
||||
if !success {
|
||||
time.Sleep(peerRetryMessageIntervalMS * time.Millisecond)
|
||||
continue
|
||||
}
|
||||
}
|
||||
|
||||
afterCh := time.After(time.Second * broadcastEvidenceIntervalS)
|
||||
@@ -150,12 +153,12 @@ func (evR *Reactor) broadcastEvidenceRoutine(peer p2p.Peer) {
|
||||
}
|
||||
}
|
||||
|
||||
// Returns the message to send the peer, or nil if the evidence is invalid for the peer.
|
||||
// If message is nil, return true if we should sleep and try again.
|
||||
func (evR Reactor) checkSendEvidenceMessage(
|
||||
// Returns the message to send to the peer, or nil if the evidence is invalid for the peer.
|
||||
// If message is nil, we should sleep and try again.
|
||||
func (evR Reactor) prepareEvidenceMessage(
|
||||
peer p2p.Peer,
|
||||
ev types.Evidence,
|
||||
) (evis []types.Evidence, retry bool) {
|
||||
) (evis []types.Evidence) {
|
||||
|
||||
// make sure the peer is up to date
|
||||
evHeight := ev.Height()
|
||||
@@ -166,7 +169,7 @@ func (evR Reactor) checkSendEvidenceMessage(
|
||||
// different every time due to us using a map. Sometimes other reactors
|
||||
// will be initialized before the consensus reactor. We should wait a few
|
||||
// milliseconds and retry.
|
||||
return nil, true
|
||||
return nil
|
||||
}
|
||||
|
||||
// NOTE: We only send evidence to peers where
|
||||
@@ -178,7 +181,7 @@ func (evR Reactor) checkSendEvidenceMessage(
|
||||
)
|
||||
|
||||
if peerHeight <= evHeight { // peer is behind. sleep while he catches up
|
||||
return nil, true
|
||||
return nil
|
||||
} 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
|
||||
@@ -192,11 +195,11 @@ func (evR Reactor) checkSendEvidenceMessage(
|
||||
"peer", peer,
|
||||
)
|
||||
|
||||
return nil, false
|
||||
return nil
|
||||
}
|
||||
|
||||
// send evidence
|
||||
return []types.Evidence{ev}, false
|
||||
return []types.Evidence{ev}
|
||||
}
|
||||
|
||||
// PeerState describes the state of a peer.
|
||||
|
||||
@@ -27,6 +27,162 @@ import (
|
||||
"github.com/tendermint/tendermint/types"
|
||||
)
|
||||
|
||||
var (
|
||||
numEvidence = 10
|
||||
timeout = 120 * time.Second // ridiculously high because CircleCI is slow
|
||||
)
|
||||
|
||||
// We have N evidence reactors connected to one another. The first reactor
|
||||
// receives a number of evidence at varying heights. We test that all
|
||||
// other reactors receive the evidence and add it to their own respective
|
||||
// evidence pools.
|
||||
func TestReactorBroadcastEvidence(t *testing.T) {
|
||||
config := cfg.TestConfig()
|
||||
N := 7
|
||||
|
||||
// create statedb for everyone
|
||||
stateDBs := make([]sm.Store, N)
|
||||
val := types.NewMockPV()
|
||||
// we need validators saved for heights at least as high as we have evidence for
|
||||
height := int64(numEvidence) + 10
|
||||
for i := 0; i < N; i++ {
|
||||
stateDBs[i] = initializeValidatorState(val, height)
|
||||
}
|
||||
|
||||
// make reactors from statedb
|
||||
reactors, pools := makeAndConnectReactorsAndPools(config, stateDBs)
|
||||
|
||||
// set the peer height on each reactor
|
||||
for _, r := range reactors {
|
||||
for _, peer := range r.Switch.Peers().List() {
|
||||
ps := peerState{height}
|
||||
peer.Set(types.PeerStateKey, ps)
|
||||
}
|
||||
}
|
||||
|
||||
// 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, pools[0], val, numEvidence)
|
||||
waitForEvidence(t, evList, pools)
|
||||
}
|
||||
|
||||
// We have two evidence reactors connected to one another but are at different heights.
|
||||
// Reactor 1 which is ahead receives a number of evidence. It should only send the evidence
|
||||
// that is below the height of the peer to that peer.
|
||||
func TestReactorSelectiveBroadcast(t *testing.T) {
|
||||
config := cfg.TestConfig()
|
||||
|
||||
val := types.NewMockPV()
|
||||
height1 := int64(numEvidence) + 10
|
||||
height2 := int64(numEvidence) / 2
|
||||
|
||||
// DB1 is ahead of DB2
|
||||
stateDB1 := initializeValidatorState(val, height1)
|
||||
stateDB2 := initializeValidatorState(val, height2)
|
||||
|
||||
// make reactors from statedb
|
||||
reactors, pools := makeAndConnectReactorsAndPools(config, []sm.Store{stateDB1, stateDB2})
|
||||
|
||||
// set the peer height on each reactor
|
||||
for _, r := range reactors {
|
||||
for _, peer := range r.Switch.Peers().List() {
|
||||
ps := peerState{height1}
|
||||
peer.Set(types.PeerStateKey, ps)
|
||||
}
|
||||
}
|
||||
|
||||
// update the first reactor peer's height to be very small
|
||||
peer := reactors[0].Switch.Peers().List()[0]
|
||||
ps := peerState{height2}
|
||||
peer.Set(types.PeerStateKey, ps)
|
||||
|
||||
// send a bunch of valid evidence to the first reactor's evpool
|
||||
evList := sendEvidence(t, pools[0], val, numEvidence)
|
||||
|
||||
// only ones less than the peers height should make it through
|
||||
waitForEvidence(t, evList[:numEvidence/2-1], []*evidence.Pool{pools[1]})
|
||||
|
||||
// peers should still be connected
|
||||
peers := reactors[1].Switch.Peers().List()
|
||||
assert.Equal(t, 1, len(peers))
|
||||
}
|
||||
|
||||
// This tests aims to ensure that reactors don't send evidence that they have committed or that ar
|
||||
// not ready for the peer through three scenarios.
|
||||
// First, committed evidence to a newly connected peer
|
||||
// Second, evidence to a peer that is behind
|
||||
// Third, evidence that was pending and became committed just before the peer caught up
|
||||
func TestReactorsGossipNoCommittedEvidence(t *testing.T) {
|
||||
config := cfg.TestConfig()
|
||||
|
||||
val := types.NewMockPV()
|
||||
var height int64 = 10
|
||||
|
||||
// DB1 is ahead of DB2
|
||||
stateDB1 := initializeValidatorState(val, height)
|
||||
stateDB2 := initializeValidatorState(val, height-2)
|
||||
|
||||
// make reactors from statedb
|
||||
reactors, pools := makeAndConnectReactorsAndPools(config, []sm.Store{stateDB1, stateDB2})
|
||||
|
||||
evList := sendEvidence(t, pools[0], val, 2)
|
||||
abciEvs := pools[0].ABCIEvidence(height, evList)
|
||||
require.EqualValues(t, 2, len(abciEvs))
|
||||
require.EqualValues(t, uint32(0), pools[0].Size())
|
||||
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
|
||||
peer := reactors[0].Switch.Peers().List()[0]
|
||||
ps := peerState{height - 2}
|
||||
peer.Set(types.PeerStateKey, ps)
|
||||
|
||||
peer = reactors[1].Switch.Peers().List()[0]
|
||||
ps = peerState{height}
|
||||
peer.Set(types.PeerStateKey, ps)
|
||||
|
||||
// wait to see that no evidence comes through
|
||||
time.Sleep(300 * time.Millisecond)
|
||||
|
||||
// the second pool should not have received any evidence because it has already been committed
|
||||
assert.Equal(t, uint32(0), pools[1].Size(), "second reactor should not have received evidence")
|
||||
|
||||
// the first reactor receives three more evidence
|
||||
evList = make([]types.Evidence, 3)
|
||||
for i := 0; i < 3; i++ {
|
||||
ev := types.NewMockDuplicateVoteEvidenceWithValidator(height-3+int64(i),
|
||||
time.Date(2019, 1, 1, 0, 0, 0, 0, time.UTC), val, evidenceChainID)
|
||||
err := pools[0].AddEvidence(ev)
|
||||
require.NoError(t, err)
|
||||
evList[i] = ev
|
||||
}
|
||||
|
||||
// wait to see that only one evidence is sent
|
||||
time.Sleep(300 * time.Millisecond)
|
||||
|
||||
// the second pool should only have received the first evidence because it is behind
|
||||
peerEv, _ := pools[1].PendingEvidence(1000)
|
||||
assert.EqualValues(t, []types.Evidence{evList[0]}, peerEv)
|
||||
|
||||
// the last evidence is committed and the second reactor catches up in state to the first
|
||||
// reactor. We therefore expect that the second reactor only receives one more evidence, the
|
||||
// one that is still pending and not the evidence that has already been committed.
|
||||
_ = pools[0].ABCIEvidence(height, []types.Evidence{evList[2]})
|
||||
// the first reactor should have the two remaining pending evidence
|
||||
require.EqualValues(t, uint32(2), pools[0].Size())
|
||||
|
||||
// now update the state of the second reactor
|
||||
pools[1].Update(sm.State{LastBlockHeight: height})
|
||||
peer = reactors[0].Switch.Peers().List()[0]
|
||||
ps = peerState{height}
|
||||
peer.Set(types.PeerStateKey, ps)
|
||||
|
||||
// wait to see that only two evidence is sent
|
||||
time.Sleep(300 * time.Millisecond)
|
||||
|
||||
peerEv, _ = pools[1].PendingEvidence(1000)
|
||||
assert.EqualValues(t, evList[0:1], peerEv)
|
||||
}
|
||||
|
||||
// evidenceLogger is a TestingLogger which uses a different
|
||||
// color for each validator ("validator" key must exist).
|
||||
func evidenceLogger() log.Logger {
|
||||
@@ -141,41 +297,6 @@ func sendEvidence(t *testing.T, evpool *evidence.Pool, val types.PrivValidator,
|
||||
return evList
|
||||
}
|
||||
|
||||
var (
|
||||
numEvidence = 10
|
||||
timeout = 120 * time.Second // ridiculously high because CircleCI is slow
|
||||
)
|
||||
|
||||
func TestReactorBroadcastEvidence(t *testing.T) {
|
||||
config := cfg.TestConfig()
|
||||
N := 7
|
||||
|
||||
// create statedb for everyone
|
||||
stateDBs := make([]sm.Store, N)
|
||||
val := types.NewMockPV()
|
||||
// we need validators saved for heights at least as high as we have evidence for
|
||||
height := int64(numEvidence) + 10
|
||||
for i := 0; i < N; i++ {
|
||||
stateDBs[i] = initializeValidatorState(val, height)
|
||||
}
|
||||
|
||||
// make reactors from statedb
|
||||
reactors, pools := makeAndConnectReactorsAndPools(config, stateDBs)
|
||||
|
||||
// set the peer height on each reactor
|
||||
for _, r := range reactors {
|
||||
for _, peer := range r.Switch.Peers().List() {
|
||||
ps := peerState{height}
|
||||
peer.Set(types.PeerStateKey, ps)
|
||||
}
|
||||
}
|
||||
|
||||
// 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, pools[0], val, numEvidence)
|
||||
waitForEvidence(t, evList, pools)
|
||||
}
|
||||
|
||||
type peerState struct {
|
||||
height int64
|
||||
}
|
||||
@@ -184,44 +305,6 @@ func (ps peerState) GetHeight() int64 {
|
||||
return ps.height
|
||||
}
|
||||
|
||||
func TestReactorSelectiveBroadcast(t *testing.T) {
|
||||
config := cfg.TestConfig()
|
||||
|
||||
val := types.NewMockPV()
|
||||
height1 := int64(numEvidence) + 10
|
||||
height2 := int64(numEvidence) / 2
|
||||
|
||||
// DB1 is ahead of DB2
|
||||
stateDB1 := initializeValidatorState(val, height1)
|
||||
stateDB2 := initializeValidatorState(val, height2)
|
||||
|
||||
// make reactors from statedb
|
||||
reactors, pools := makeAndConnectReactorsAndPools(config, []sm.Store{stateDB1, stateDB2})
|
||||
|
||||
// set the peer height on each reactor
|
||||
for _, r := range reactors {
|
||||
for _, peer := range r.Switch.Peers().List() {
|
||||
ps := peerState{height1}
|
||||
peer.Set(types.PeerStateKey, ps)
|
||||
}
|
||||
}
|
||||
|
||||
// update the first reactor peer's height to be very small
|
||||
peer := reactors[0].Switch.Peers().List()[0]
|
||||
ps := peerState{height2}
|
||||
peer.Set(types.PeerStateKey, ps)
|
||||
|
||||
// send a bunch of valid evidence to the first reactor's evpool
|
||||
evList := sendEvidence(t, pools[0], val, numEvidence)
|
||||
|
||||
// only ones less than the peers height should make it through
|
||||
waitForEvidence(t, evList[:numEvidence/2-1], pools[1:2])
|
||||
|
||||
// peers should still be connected
|
||||
peers := reactors[1].Switch.Peers().List()
|
||||
assert.Equal(t, 1, len(peers))
|
||||
}
|
||||
|
||||
func exampleVote(t byte) *types.Vote {
|
||||
var stamp, err = time.Parse(types.TimeFormat, "2017-12-25T03:00:01.234Z")
|
||||
if err != nil {
|
||||
|
||||
@@ -24,11 +24,6 @@ func (evpool *Pool) verify(evidence types.Evidence) (*info, error) {
|
||||
ageNumBlocks = height - evidence.Height()
|
||||
)
|
||||
|
||||
// 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 {
|
||||
|
||||
Reference in New Issue
Block a user