mirror of
https://github.com/tendermint/tendermint.git
synced 2026-01-03 11:45:18 +00:00
evidence: adr56 form amnesia evidence (#4821)
Creates Amnesia Evidence which is formed from Potential Amnesia Evidence with either a matching proof or after a period of time denoted as the Amnesia Trial Period. This also adds the code necessary so that Amnesia Evidence can be validated and committed on a block
This commit is contained in:
180
evidence/pool.go
180
evidence/pool.go
@@ -21,6 +21,7 @@ const (
|
||||
baseKeyCommitted = byte(0x00)
|
||||
baseKeyPending = byte(0x01)
|
||||
baseKeyPOLC = byte(0x02)
|
||||
baseKeyAwaiting = byte(0x03)
|
||||
)
|
||||
|
||||
// Pool maintains a pool of valid evidence to be broadcasted and committed
|
||||
@@ -43,6 +44,8 @@ type Pool struct {
|
||||
// currently is (ie. [MaxAgeNumBlocks, CurrentHeight])
|
||||
// In simple words, it means it's still bonded -> therefore slashable.
|
||||
valToLastHeight valToLastHeightMap
|
||||
|
||||
nextEvidenceTrialEndedHeight int64
|
||||
}
|
||||
|
||||
// Validator.Address -> Last height it was in validator set
|
||||
@@ -59,22 +62,19 @@ func NewPool(stateDB, evidenceDB dbm.DB, blockStore *store.BlockStore) (*Pool, e
|
||||
}
|
||||
|
||||
pool := &Pool{
|
||||
stateDB: stateDB,
|
||||
blockStore: blockStore,
|
||||
state: state,
|
||||
logger: log.NewNopLogger(),
|
||||
evidenceStore: evidenceDB,
|
||||
evidenceList: clist.New(),
|
||||
valToLastHeight: valToLastHeight,
|
||||
stateDB: stateDB,
|
||||
blockStore: blockStore,
|
||||
state: state,
|
||||
logger: log.NewNopLogger(),
|
||||
evidenceStore: evidenceDB,
|
||||
evidenceList: clist.New(),
|
||||
valToLastHeight: valToLastHeight,
|
||||
nextEvidenceTrialEndedHeight: -1,
|
||||
}
|
||||
|
||||
// if pending evidence already in db, in event of prior failure, then load it back to the evidenceList
|
||||
evList := pool.AllPendingEvidence()
|
||||
for _, ev := range evList {
|
||||
if pool.IsEvidenceExpired(ev) {
|
||||
pool.removePendingEvidence(ev)
|
||||
continue
|
||||
}
|
||||
pool.evidenceList.PushBack(ev)
|
||||
}
|
||||
|
||||
@@ -84,6 +84,7 @@ func NewPool(stateDB, evidenceDB dbm.DB, blockStore *store.BlockStore) (*Pool, e
|
||||
// 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 prioritised 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)
|
||||
@@ -92,6 +93,7 @@ func (evpool *Pool) PendingEvidence(maxNum uint32) []types.Evidence {
|
||||
}
|
||||
|
||||
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)
|
||||
@@ -104,23 +106,24 @@ func (evpool *Pool) AllPendingEvidence() []types.Evidence {
|
||||
func (evpool *Pool) Update(block *types.Block, 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",
|
||||
state.LastBlockHeight,
|
||||
block.Height,
|
||||
),
|
||||
panic(fmt.Sprintf("Failed EvidencePool.Update sanity check: got state.Height=%d with block.Height=%d",
|
||||
state.LastBlockHeight,
|
||||
block.Height,
|
||||
),
|
||||
)
|
||||
}
|
||||
|
||||
// remove evidence from pending and mark committed
|
||||
evpool.MarkEvidenceAsCommitted(block.Height, block.Time, block.Evidence.Evidence)
|
||||
evpool.MarkEvidenceAsCommitted(block.Height, block.Evidence.Evidence)
|
||||
|
||||
// remove expired evidence - this should be done at every height to ensure we don't send expired evidence to peers
|
||||
evpool.removeExpiredPendingEvidence()
|
||||
|
||||
// as it's not vital to remove expired POLCs, we only prune periodically
|
||||
// prune pending, committed and potential evidence and polc's periodically
|
||||
if block.Height%state.ConsensusParams.Evidence.MaxAgeNumBlocks == 0 {
|
||||
evpool.pruneExpiredPOLC()
|
||||
evpool.removeExpiredPendingEvidence()
|
||||
}
|
||||
|
||||
if evpool.nextEvidenceTrialEndedHeight > 0 && block.Height < evpool.nextEvidenceTrialEndedHeight {
|
||||
evpool.upgradePotentialAmnesiaEvidence()
|
||||
}
|
||||
|
||||
// update the state
|
||||
@@ -202,9 +205,74 @@ func (evpool *Pool) AddEvidence(evidence types.Evidence) error {
|
||||
return fmt.Errorf("failed to verify %v: %w", ev, err)
|
||||
}
|
||||
|
||||
// For potential amnesia evidence, if this node is indicted it shall retrieve a polc
|
||||
// to form AmensiaEvidence
|
||||
if pe, ok := ev.(types.PotentialAmnesiaEvidence); ok {
|
||||
var (
|
||||
height = pe.Height()
|
||||
exists = false
|
||||
polc types.ProofOfLockChange
|
||||
)
|
||||
pe.HeightStamp = evpool.State().LastBlockHeight
|
||||
|
||||
// a) first try to find a corresponding polc
|
||||
for round := pe.VoteB.Round; round > pe.VoteA.Round; round-- {
|
||||
polc, err = evpool.RetrievePOLC(height, round)
|
||||
if err != nil {
|
||||
evpool.logger.Error("Failed to retrieve polc for potential amnesia evidence", "err", err, "pae", pe.String())
|
||||
continue
|
||||
}
|
||||
if err == nil && !polc.IsAbsent() {
|
||||
// we should not need to verify it if both the polc and potential amnesia evidence have already
|
||||
// been verified. We replace the potential amnesia evidence.
|
||||
ae := types.MakeAmnesiaEvidence(pe, polc)
|
||||
err := evpool.AddEvidence(ae)
|
||||
if err != nil {
|
||||
evpool.logger.Error("Failed to create amnesia evidence from potential amnesia evidence", "err", err)
|
||||
// revert back to processing potential amnesia evidence
|
||||
exists = false
|
||||
} else {
|
||||
evpool.logger.Info("Formed amnesia evidence from own polc", "amnesiaEvidence", ae)
|
||||
}
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
// b) check if amnesia evidence can be made now or if we need to enact the trial period
|
||||
if !exists && pe.Primed(1, pe.HeightStamp) {
|
||||
err := evpool.AddEvidence(types.MakeAmnesiaEvidence(pe, types.EmptyPOLC()))
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
} else if !exists && evpool.State().LastBlockHeight+evpool.State().ConsensusParams.Evidence.ProofTrialPeriod <
|
||||
pe.Height()+evpool.State().ConsensusParams.Evidence.MaxAgeNumBlocks {
|
||||
// if we can't find a proof of lock change and we know that the trial period will finish before the
|
||||
// evidence has expired, then we commence the trial period by saving it in the awaiting bucket
|
||||
pbe, err := types.EvidenceToProto(pe)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
evBytes, err := pbe.Marshal()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
key := keyAwaiting(pe)
|
||||
err = evpool.evidenceStore.Set(key, evBytes)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
// keep track of when the next pe has finished the trial period
|
||||
if evpool.nextEvidenceTrialEndedHeight == -1 {
|
||||
evpool.nextEvidenceTrialEndedHeight = ev.Height() + evpool.State().ConsensusParams.Evidence.ProofTrialPeriod
|
||||
}
|
||||
}
|
||||
// we don't need to do anymore processing so we can move on to the next piece of evidence
|
||||
continue
|
||||
}
|
||||
|
||||
// 2) Save to store.
|
||||
if err := evpool.addPendingEvidence(ev); err != nil {
|
||||
return fmt.Errorf("database error: %v", err)
|
||||
return fmt.Errorf("database error when adding evidence: %v", err)
|
||||
}
|
||||
|
||||
// 3) Add evidence to clist.
|
||||
@@ -218,7 +286,7 @@ func (evpool *Pool) AddEvidence(evidence types.Evidence) error {
|
||||
|
||||
// MarkEvidenceAsCommitted marks all the evidence as committed and removes it
|
||||
// from the queue.
|
||||
func (evpool *Pool) MarkEvidenceAsCommitted(height int64, lastBlockTime time.Time, evidence []types.Evidence) {
|
||||
func (evpool *Pool) MarkEvidenceAsCommitted(height int64, evidence []types.Evidence) {
|
||||
// make a map of committed evidence to remove from the clist
|
||||
blockEvidenceMap := make(map[string]struct{})
|
||||
for _, ev := range evidence {
|
||||
@@ -291,17 +359,19 @@ func (evpool *Pool) IsPending(evidence types.Evidence) bool {
|
||||
return ok
|
||||
}
|
||||
|
||||
// RetrievePOLC attempts to find a polc at the given height and round, if not there it returns an error
|
||||
// RetrievePOLC attempts to find a polc at the given height and round, if not there than exist returns false, all
|
||||
// database errors are automatically logged
|
||||
func (evpool *Pool) RetrievePOLC(height int64, round int32) (polc types.ProofOfLockChange, err error) {
|
||||
var pbpolc tmproto.ProofOfLockChange
|
||||
key := keyPOLCFromHeightAndRound(height, round)
|
||||
polcBytes, err := evpool.evidenceStore.Get(key)
|
||||
if err != nil {
|
||||
evpool.logger.Error("Unable to retrieve polc", "err", err)
|
||||
return polc, err
|
||||
}
|
||||
|
||||
if polcBytes == nil {
|
||||
return polc, fmt.Errorf("unable to find polc at height %d and round %d", height, round)
|
||||
return polc, fmt.Errorf("nil value in database for key: %s", key)
|
||||
}
|
||||
|
||||
err = proto.Unmarshal(polcBytes, &pbpolc)
|
||||
@@ -366,7 +436,7 @@ func (evpool *Pool) State() sm.State {
|
||||
func (evpool *Pool) addPendingEvidence(evidence types.Evidence) error {
|
||||
evi, err := types.EvidenceToProto(evidence)
|
||||
if err != nil {
|
||||
return err
|
||||
return fmt.Errorf("unable to convert to proto, err: %w", err)
|
||||
}
|
||||
|
||||
evBytes, err := proto.Marshal(evi)
|
||||
@@ -399,13 +469,12 @@ func (evpool *Pool) listEvidence(prefixKey byte, maxNum int64) ([]types.Evidence
|
||||
}
|
||||
defer iter.Close()
|
||||
for ; iter.Valid(); iter.Next() {
|
||||
val := iter.Value()
|
||||
|
||||
if count == maxNum {
|
||||
return evidence, nil
|
||||
}
|
||||
count++
|
||||
|
||||
val := iter.Value()
|
||||
var (
|
||||
ev types.Evidence
|
||||
evpb tmproto.Evidence
|
||||
@@ -511,6 +580,57 @@ func (evpool *Pool) pruneExpiredPOLC() {
|
||||
}
|
||||
}
|
||||
|
||||
// upgrades any potential evidence that has undergone the trial period and is primed to be made into
|
||||
// amnesia evidence
|
||||
func (evpool *Pool) upgradePotentialAmnesiaEvidence() int64 {
|
||||
iter, err := dbm.IteratePrefix(evpool.evidenceStore, []byte{baseKeyAwaiting})
|
||||
if err != nil {
|
||||
evpool.logger.Error("Unable to iterate over POLC's", "err", err)
|
||||
return -1
|
||||
}
|
||||
defer iter.Close()
|
||||
trialPeriod := evpool.State().ConsensusParams.Evidence.ProofTrialPeriod
|
||||
// 1) Iterate through all potential amnesia evidence in order of height
|
||||
for ; iter.Valid(); iter.Next() {
|
||||
paeBytes := iter.Value()
|
||||
// 2) Retrieve the evidence
|
||||
var evpb tmproto.Evidence
|
||||
err := evpb.Unmarshal(paeBytes)
|
||||
if err != nil {
|
||||
evpool.logger.Error("Unable to unmarshal potential amnesia evidence", "err", err)
|
||||
continue
|
||||
}
|
||||
ev, err := types.EvidenceFromProto(&evpb)
|
||||
if err != nil {
|
||||
evpool.logger.Error("coverting to evidence from proto", "err", err)
|
||||
continue
|
||||
}
|
||||
// 3) Check if the trial period has lapsed and amnesia evidence can be formed
|
||||
if pe, ok := ev.(*types.PotentialAmnesiaEvidence); ok {
|
||||
if pe.Primed(trialPeriod, evpool.State().LastBlockHeight) {
|
||||
ae := types.MakeAmnesiaEvidence(*pe, types.EmptyPOLC())
|
||||
err := evpool.AddEvidence(ae)
|
||||
if err != nil {
|
||||
evpool.logger.Error("Unable to add amnesia evidence", "err", err)
|
||||
continue
|
||||
}
|
||||
err = evpool.evidenceStore.Delete(iter.Key())
|
||||
if err != nil {
|
||||
evpool.logger.Error("Unable to delete potential amnesia evidence", "err", err)
|
||||
continue
|
||||
}
|
||||
} else {
|
||||
evpool.logger.Debug("Potential amnesia evidence not ready to be upgraded. Ready at height", "height",
|
||||
pe.HeightStamp+trialPeriod)
|
||||
// once we reach a piece of evidence that isn't ready send back the height with which it will be ready
|
||||
return pe.HeightStamp + trialPeriod
|
||||
}
|
||||
}
|
||||
}
|
||||
// if we have no evidence left to process we want to reset nextEvidenceTrialEndedHeight
|
||||
return -1
|
||||
}
|
||||
|
||||
func evMapKey(ev types.Evidence) string {
|
||||
return string(ev.Hash())
|
||||
}
|
||||
@@ -604,6 +724,10 @@ func keyPending(evidence types.Evidence) []byte {
|
||||
return append([]byte{baseKeyPending}, keySuffix(evidence)...)
|
||||
}
|
||||
|
||||
func keyAwaiting(evidence types.Evidence) []byte {
|
||||
return append([]byte{baseKeyAwaiting}, keySuffix(evidence)...)
|
||||
}
|
||||
|
||||
func keyPOLC(polc types.ProofOfLockChange) []byte {
|
||||
return keyPOLCFromHeightAndRound(polc.Height(), polc.Round())
|
||||
}
|
||||
|
||||
@@ -12,7 +12,8 @@ import (
|
||||
dbm "github.com/tendermint/tm-db"
|
||||
|
||||
"github.com/tendermint/tendermint/crypto"
|
||||
"github.com/tendermint/tendermint/crypto/ed25519"
|
||||
"github.com/tendermint/tendermint/libs/bytes"
|
||||
"github.com/tendermint/tendermint/libs/log"
|
||||
tmrand "github.com/tendermint/tendermint/libs/rand"
|
||||
tmproto "github.com/tendermint/tendermint/proto/types"
|
||||
sm "github.com/tendermint/tendermint/state"
|
||||
@@ -27,6 +28,8 @@ func TestMain(m *testing.M) {
|
||||
os.Exit(code)
|
||||
}
|
||||
|
||||
const evidenceChainID = "test_chain"
|
||||
|
||||
func TestEvidencePool(t *testing.T) {
|
||||
var (
|
||||
valAddr = tmrand.Bytes(crypto.AddressSize)
|
||||
@@ -78,14 +81,13 @@ func TestEvidencePool(t *testing.T) {
|
||||
|
||||
func TestProposingAndCommittingEvidence(t *testing.T) {
|
||||
var (
|
||||
valAddr = tmrand.Bytes(crypto.AddressSize)
|
||||
height = int64(1)
|
||||
lastBlockTime = time.Now()
|
||||
stateDB = initializeValidatorState(valAddr, height)
|
||||
evidenceDB = dbm.NewMemDB()
|
||||
blockStoreDB = dbm.NewMemDB()
|
||||
blockStore = initializeBlockStore(blockStoreDB, sm.LoadState(stateDB), valAddr)
|
||||
evidenceTime = time.Date(2019, 1, 1, 0, 0, 0, 0, time.UTC)
|
||||
valAddr = tmrand.Bytes(crypto.AddressSize)
|
||||
height = int64(1)
|
||||
stateDB = initializeValidatorState(valAddr, height)
|
||||
evidenceDB = dbm.NewMemDB()
|
||||
blockStoreDB = dbm.NewMemDB()
|
||||
blockStore = initializeBlockStore(blockStoreDB, sm.LoadState(stateDB), valAddr)
|
||||
evidenceTime = time.Date(2019, 1, 1, 0, 0, 0, 0, time.UTC)
|
||||
)
|
||||
|
||||
pool, err := NewPool(stateDB, evidenceDB, blockStore)
|
||||
@@ -104,7 +106,7 @@ func TestProposingAndCommittingEvidence(t *testing.T) {
|
||||
assert.Equal(t, proposedEvidence[0], evidence)
|
||||
|
||||
// evidence seen and committed:
|
||||
pool.MarkEvidenceAsCommitted(height, lastBlockTime, proposedEvidence)
|
||||
pool.MarkEvidenceAsCommitted(height, proposedEvidence)
|
||||
assert.True(t, pool.IsCommitted(evidence))
|
||||
assert.False(t, pool.IsPending(evidence))
|
||||
assert.Equal(t, 0, pool.evidenceList.Len())
|
||||
@@ -161,14 +163,10 @@ func TestEvidencePoolUpdate(t *testing.T) {
|
||||
blockStoreDB = dbm.NewMemDB()
|
||||
state = sm.LoadState(stateDB)
|
||||
blockStore = initializeBlockStore(blockStoreDB, state, valAddr)
|
||||
evidenceTime = time.Date(2019, 1, 1, 0, 0, 0, 0, time.UTC)
|
||||
)
|
||||
|
||||
pool, err := NewPool(stateDB, evidenceDB, blockStore)
|
||||
require.NoError(t, err)
|
||||
expiredEvidence := types.NewMockEvidence(1, evidenceTime, valAddr)
|
||||
err = pool.AddEvidence(expiredEvidence)
|
||||
require.NoError(t, err)
|
||||
|
||||
// create new block (no need to save it to blockStore)
|
||||
evidence := types.NewMockEvidence(height, time.Now(), valAddr)
|
||||
@@ -183,8 +181,6 @@ func TestEvidencePoolUpdate(t *testing.T) {
|
||||
assert.True(t, pool.IsCommitted(evidence))
|
||||
// b) Update updates valToLastHeight map
|
||||
assert.Equal(t, height+1, pool.ValidatorLastHeight(valAddr))
|
||||
// c) Expired ecvidence should be removed
|
||||
assert.False(t, pool.IsPending(expiredEvidence))
|
||||
}
|
||||
|
||||
func TestEvidencePoolNewPool(t *testing.T) {
|
||||
@@ -246,9 +242,7 @@ func TestAddingAndPruningPOLC(t *testing.T) {
|
||||
pool.Update(block, state)
|
||||
|
||||
emptyPolc, err = pool.RetrievePOLC(1, 1)
|
||||
if assert.Error(t, err) {
|
||||
assert.Equal(t, "unable to find polc at height 1 and round 1", err.Error())
|
||||
}
|
||||
assert.Error(t, err)
|
||||
assert.Equal(t, types.ProofOfLockChange{}, emptyPolc)
|
||||
|
||||
}
|
||||
@@ -290,18 +284,108 @@ func TestRecoverPendingEvidence(t *testing.T) {
|
||||
assert.True(t, pool.IsPending(goodEvidence))
|
||||
}
|
||||
|
||||
func initializeValidatorState(valAddr []byte, height int64) dbm.DB {
|
||||
stateDB := dbm.NewMemDB()
|
||||
pk := ed25519.GenPrivKey().PubKey()
|
||||
func TestPotentialAmnesiaEvidence(t *testing.T) {
|
||||
var (
|
||||
val = types.NewMockPV()
|
||||
pubKey = val.PrivKey.PubKey()
|
||||
valSet = &types.ValidatorSet{
|
||||
Validators: []*types.Validator{
|
||||
val.ExtractIntoValidator(0),
|
||||
},
|
||||
Proposer: val.ExtractIntoValidator(0),
|
||||
}
|
||||
height = int64(30)
|
||||
stateDB = initializeStateFromValidatorSet(valSet, height)
|
||||
evidenceDB = dbm.NewMemDB()
|
||||
blockStoreDB = dbm.NewMemDB()
|
||||
state = sm.LoadState(stateDB)
|
||||
blockStore = initializeBlockStore(blockStoreDB, state, pubKey.Address())
|
||||
//evidenceTime = time.Date(2019, 1, 1, 0, 0, 0, 0, time.UTC)
|
||||
firstBlockID = types.BlockID{
|
||||
Hash: []byte("aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa"),
|
||||
PartsHeader: types.PartSetHeader{
|
||||
Total: 1,
|
||||
Hash: []byte("aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa"),
|
||||
},
|
||||
}
|
||||
secondBlockID = types.BlockID{
|
||||
Hash: []byte("bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb"),
|
||||
PartsHeader: types.PartSetHeader{
|
||||
Total: 1,
|
||||
Hash: []byte("bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb"),
|
||||
},
|
||||
}
|
||||
evidenceTime = time.Date(2019, 1, 1, 0, 0, 0, 0, time.UTC)
|
||||
)
|
||||
|
||||
// create validator set and state
|
||||
validator := &types.Validator{Address: valAddr, VotingPower: 100, PubKey: pk}
|
||||
valSet := &types.ValidatorSet{
|
||||
Validators: []*types.Validator{validator},
|
||||
Proposer: validator,
|
||||
pool, err := NewPool(stateDB, evidenceDB, blockStore)
|
||||
require.NoError(t, err)
|
||||
|
||||
pool.SetLogger(log.TestingLogger())
|
||||
|
||||
polc := types.NewMockPOLC(25, evidenceTime, pubKey)
|
||||
err = pool.AddPOLC(polc)
|
||||
require.NoError(t, err)
|
||||
|
||||
_, err = pool.RetrievePOLC(25, 1)
|
||||
require.NoError(t, err)
|
||||
|
||||
voteA := makeVote(25, 0, 0, pubKey.Address(), firstBlockID)
|
||||
err = val.SignVote(evidenceChainID, voteA)
|
||||
require.NoError(t, err)
|
||||
voteB := makeVote(25, 1, 0, pubKey.Address(), secondBlockID)
|
||||
err = val.SignVote(evidenceChainID, voteB)
|
||||
require.NoError(t, err)
|
||||
voteC := makeVote(25, 0, 0, pubKey.Address(), firstBlockID)
|
||||
voteC.Timestamp.Add(1 * time.Second)
|
||||
err = val.SignVote(evidenceChainID, voteC)
|
||||
require.NoError(t, err)
|
||||
ev := types.PotentialAmnesiaEvidence{
|
||||
VoteA: voteA,
|
||||
VoteB: voteB,
|
||||
}
|
||||
// we expect the evidence pool to find the polc but log an error as the polc is not valid -> vote was
|
||||
// not from a validator in this set. However, an error isn't thrown because the evidence pool
|
||||
// should still be able to save the regular potential amnesia evidence.
|
||||
err = pool.AddEvidence(ev)
|
||||
assert.NoError(t, err)
|
||||
|
||||
// evidence requires trial period until it is available -> we expect no evidence to be returned
|
||||
assert.Equal(t, 0, len(pool.PendingEvidence(1)))
|
||||
|
||||
nextHeight := pool.nextEvidenceTrialEndedHeight
|
||||
assert.Greater(t, nextHeight, int64(0))
|
||||
|
||||
// evidence is not ready to be upgraded so we return the height we expect the evidence to be.
|
||||
nextHeight = pool.upgradePotentialAmnesiaEvidence()
|
||||
assert.Equal(t, height+pool.state.ConsensusParams.Evidence.ProofTrialPeriod, nextHeight)
|
||||
|
||||
// now evidence is ready to be upgraded to amnesia evidence -> we expect -1 to be the next height as their is
|
||||
// no more pending potential amnesia evidence left
|
||||
pool.state.LastBlockHeight = nextHeight
|
||||
nextHeight = pool.upgradePotentialAmnesiaEvidence()
|
||||
assert.Equal(t, int64(-1), nextHeight)
|
||||
|
||||
assert.Equal(t, 1, len(pool.PendingEvidence(1)))
|
||||
|
||||
// evidence of voting back in the past which is instantly punishable -> amnesia evidence is made directly
|
||||
voteA.Timestamp.Add(1 * time.Second)
|
||||
|
||||
ev2 := types.PotentialAmnesiaEvidence{
|
||||
VoteA: voteB,
|
||||
VoteB: voteC,
|
||||
}
|
||||
err = pool.AddEvidence(ev2)
|
||||
assert.NoError(t, err)
|
||||
|
||||
assert.Equal(t, 2, len(pool.AllPendingEvidence()))
|
||||
|
||||
}
|
||||
|
||||
func initializeStateFromValidatorSet(valSet *types.ValidatorSet, height int64) dbm.DB {
|
||||
stateDB := dbm.NewMemDB()
|
||||
state := sm.State{
|
||||
ChainID: evidenceChainID,
|
||||
LastBlockHeight: height,
|
||||
LastBlockTime: tmtime.Now(),
|
||||
Validators: valSet,
|
||||
@@ -314,8 +398,10 @@ func initializeValidatorState(valAddr []byte, height int64) dbm.DB {
|
||||
MaxGas: -1,
|
||||
},
|
||||
Evidence: tmproto.EvidenceParams{
|
||||
MaxAgeNumBlocks: 20,
|
||||
MaxAgeDuration: 48 * time.Hour,
|
||||
MaxAgeNumBlocks: 20,
|
||||
MaxAgeDuration: 48 * time.Hour,
|
||||
MaxNum: 50,
|
||||
ProofTrialPeriod: 1,
|
||||
},
|
||||
},
|
||||
}
|
||||
@@ -329,6 +415,20 @@ func initializeValidatorState(valAddr []byte, height int64) dbm.DB {
|
||||
return stateDB
|
||||
}
|
||||
|
||||
func initializeValidatorState(valAddr []byte, height int64) dbm.DB {
|
||||
|
||||
pubKey, _ := types.NewMockPV().GetPubKey()
|
||||
validator := &types.Validator{Address: valAddr, VotingPower: 0, PubKey: pubKey}
|
||||
|
||||
// create validator set and state
|
||||
valSet := &types.ValidatorSet{
|
||||
Validators: []*types.Validator{validator},
|
||||
Proposer: validator,
|
||||
}
|
||||
|
||||
return initializeStateFromValidatorSet(valSet, height)
|
||||
}
|
||||
|
||||
// initializeBlockStore creates a block storage and populates it w/ a dummy
|
||||
// block at +height+.
|
||||
func initializeBlockStore(db dbm.DB, state sm.State, valAddr []byte) *store.BlockStore {
|
||||
@@ -358,3 +458,15 @@ func makeCommit(height int64, valAddr []byte) *types.Commit {
|
||||
}}
|
||||
return types.NewCommit(height, 0, types.BlockID{}, commitSigs)
|
||||
}
|
||||
|
||||
func makeVote(height int64, round, index int32, addr bytes.HexBytes, blockID types.BlockID) *types.Vote {
|
||||
return &types.Vote{
|
||||
Type: tmproto.SignedMsgType(2),
|
||||
Height: height,
|
||||
Round: round,
|
||||
BlockID: blockID,
|
||||
Timestamp: time.Now(),
|
||||
ValidatorAddress: addr,
|
||||
ValidatorIndex: index,
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user