mirror of
https://github.com/tendermint/tendermint.git
synced 2025-12-23 06:15:19 +00:00
evidence: structs can independently form abci evidence (#5610)
This commit is contained in:
@@ -33,3 +33,4 @@ Friendly reminder, we have a [bug bounty program](https://hackerone.com/tendermi
|
||||
- [block] \#5567 Fix MaxCommitSigBytes (@cmwaters)
|
||||
- [evidence] \#5574 Fix bug where node sends committed evidence to peer (@cmwaters)
|
||||
- [privval] \#5583 Make `Vote`, `Proposal` & `PubKey` non-nullable in Responses (@marbar3778)
|
||||
- [evidence] \#5610 Make it possible for abci evidence to be formed from tm evidence (@cmwaters)
|
||||
|
||||
@@ -162,12 +162,12 @@ func TestByzantinePrevoteEquivocation(t *testing.T) {
|
||||
defer stopConsensusNet(log.TestingLogger(), reactors, eventBuses)
|
||||
|
||||
// Evidence should be submitted and committed at the third height but
|
||||
// we will check the first five just in case
|
||||
// we will check the first six just in case
|
||||
evidenceFromEachValidator := make([]types.Evidence, nValidators)
|
||||
|
||||
wg := new(sync.WaitGroup)
|
||||
wg.Add(4)
|
||||
for height := 1; height < 5; height++ {
|
||||
for height := 1; height < 6; height++ {
|
||||
for i := 0; i < nValidators; i++ {
|
||||
go func(j int) {
|
||||
msg := <-blocksSubs[j].Out()
|
||||
|
||||
@@ -172,9 +172,7 @@ func TestReactorWithEvidence(t *testing.T) {
|
||||
evpool.On("CheckEvidence", mock.AnythingOfType("types.EvidenceList")).Return(nil)
|
||||
evpool.On("PendingEvidence", mock.AnythingOfType("int64")).Return([]types.Evidence{
|
||||
ev}, int64(len(ev.Bytes())))
|
||||
evpool.On("Update", mock.AnythingOfType("state.State")).Return()
|
||||
evpool.On("ABCIEvidence", mock.AnythingOfType("int64"), mock.AnythingOfType("[]types.Evidence")).Return(
|
||||
[]abci.Evidence{})
|
||||
evpool.On("Update", mock.AnythingOfType("state.State"), mock.AnythingOfType("types.EvidenceList")).Return()
|
||||
|
||||
evpool2 := sm.EmptyEvidencePool{}
|
||||
|
||||
|
||||
@@ -73,9 +73,9 @@ type txNotifier interface {
|
||||
|
||||
// interface to the evidence pool
|
||||
type evidencePool interface {
|
||||
// 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
|
||||
// Adds consensus based evidence to the evidence pool. This function differs to
|
||||
// AddEvidence by bypassing verification and adding it immediately to the pool
|
||||
AddEvidenceFromConsensus(types.Evidence) error
|
||||
}
|
||||
|
||||
// State handles execution of the consensus algorithm.
|
||||
@@ -1871,13 +1871,14 @@ func (cs *State) tryAddVote(vote *types.Vote, peerID p2p.ID) (bool, error) {
|
||||
} else {
|
||||
timestamp = sm.MedianTime(cs.LastCommit.MakeCommit(), cs.LastValidators)
|
||||
}
|
||||
evidence := types.NewDuplicateVoteEvidence(voteErr.VoteA, voteErr.VoteB)
|
||||
evidenceErr := cs.evpool.AddEvidenceFromConsensus(evidence, timestamp, cs.Validators)
|
||||
|
||||
// form duplicate vote evidence from the conflicting votes and send it across to the
|
||||
// evidence pool
|
||||
ev := types.NewDuplicateVoteEvidence(voteErr.VoteA, voteErr.VoteB, timestamp, cs.Validators)
|
||||
evidenceErr := cs.evpool.AddEvidenceFromConsensus(ev)
|
||||
if evidenceErr != nil {
|
||||
cs.Logger.Error("Failed to add evidence to the evidence pool", "err", evidenceErr)
|
||||
} else {
|
||||
cs.Logger.Debug("Added evidence to the evidence pool", "evidence", evidence)
|
||||
cs.Logger.Debug("Added evidence to the evidence pool", "ev", ev)
|
||||
}
|
||||
return added, err
|
||||
} else if err == types.ErrVoteNonDeterministicSignature {
|
||||
|
||||
342
evidence/pool.go
342
evidence/pool.go
@@ -4,7 +4,7 @@ import (
|
||||
"bytes"
|
||||
"errors"
|
||||
"fmt"
|
||||
"reflect"
|
||||
"sort"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
@@ -13,10 +13,8 @@ import (
|
||||
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"
|
||||
@@ -94,7 +92,7 @@ func (evpool *Pool) PendingEvidence(maxBytes int64) ([]types.Evidence, int64) {
|
||||
}
|
||||
|
||||
// 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) {
|
||||
func (evpool *Pool) Update(state sm.State, ev types.EvidenceList) {
|
||||
// sanity check
|
||||
if state.LastBlockHeight <= evpool.state.LastBlockHeight {
|
||||
panic(fmt.Sprintf(
|
||||
@@ -109,6 +107,8 @@ func (evpool *Pool) Update(state sm.State) {
|
||||
// update the state
|
||||
evpool.updateState(state)
|
||||
|
||||
evpool.markEvidenceAsCommitted(ev)
|
||||
|
||||
// prune pending evidence when it has expired. This also updates when the next evidence will expire
|
||||
if evpool.Size() > 0 && state.LastBlockHeight > evpool.pruningHeight &&
|
||||
state.LastBlockTime.After(evpool.pruningTime) {
|
||||
@@ -135,13 +135,13 @@ func (evpool *Pool) AddEvidence(ev types.Evidence) error {
|
||||
}
|
||||
|
||||
// 1) Verify against state.
|
||||
evInfo, err := evpool.verify(ev)
|
||||
err := evpool.verify(ev)
|
||||
if err != nil {
|
||||
return types.NewErrInvalidEvidence(ev, err)
|
||||
}
|
||||
|
||||
// 2) Save to store.
|
||||
if err := evpool.addPendingEvidence(evInfo); err != nil {
|
||||
if err := evpool.addPendingEvidence(ev); err != nil {
|
||||
return fmt.Errorf("can't add evidence to pending list: %w", err)
|
||||
}
|
||||
|
||||
@@ -153,13 +153,9 @@ func (evpool *Pool) AddEvidence(ev types.Evidence) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
// 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
|
||||
)
|
||||
// AddEvidenceFromConsensus should be exposed only to the consensus reactor so it can add evidence
|
||||
// to the pool directly without the need for verification.
|
||||
func (evpool *Pool) AddEvidenceFromConsensus(ev types.Evidence) error {
|
||||
|
||||
// we already have this evidence, log this but don't return an error.
|
||||
if evpool.isPending(ev) {
|
||||
@@ -167,23 +163,7 @@ func (evpool *Pool) AddEvidenceFromConsensus(ev types.Evidence, time time.Time,
|
||||
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 {
|
||||
if err := evpool.addPendingEvidence(ev); err != nil {
|
||||
return fmt.Errorf("can't add evidence to pending list: %w", err)
|
||||
}
|
||||
// add evidence to be gossiped with peers
|
||||
@@ -210,15 +190,15 @@ func (evpool *Pool) CheckEvidence(evList types.EvidenceList) error {
|
||||
return &types.ErrInvalidEvidence{Evidence: ev, Reason: errors.New("evidence was already committed")}
|
||||
}
|
||||
|
||||
evInfo, err := evpool.verify(ev)
|
||||
err := evpool.verify(ev)
|
||||
if err != nil {
|
||||
return &types.ErrInvalidEvidence{Evidence: ev, Reason: err}
|
||||
}
|
||||
|
||||
if err := evpool.addPendingEvidence(evInfo); err != nil {
|
||||
if err := evpool.addPendingEvidence(ev); 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)
|
||||
evpool.logger.Error("Can't add evidence to pending list", "err", err, "ev", ev)
|
||||
}
|
||||
|
||||
evpool.logger.Info("Verified new evidence of byzantine behavior", "evidence", ev)
|
||||
@@ -236,85 +216,6 @@ func (evpool *Pool) CheckEvidence(evList types.EvidenceList) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
// 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{}, 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)
|
||||
|
||||
h := gogotypes.Int64Value{Value: height}
|
||||
evBytes, err := proto.Marshal(&h)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
if err := evpool.evidenceStore.Set(key, evBytes); err != nil {
|
||||
evpool.logger.Error("Unable to add committed evidence", "err", err)
|
||||
}
|
||||
}
|
||||
|
||||
// remove committed evidence from the clist
|
||||
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()
|
||||
@@ -330,6 +231,7 @@ func (evpool *Pool) SetLogger(l log.Logger) {
|
||||
evpool.logger = l
|
||||
}
|
||||
|
||||
// Size returns the number of evidence in the pool.
|
||||
func (evpool *Pool) Size() uint32 {
|
||||
return atomic.LoadUint32(&evpool.evidenceSize)
|
||||
}
|
||||
@@ -343,106 +245,59 @@ func (evpool *Pool) State() sm.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
|
||||
ByteSize 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,
|
||||
ByteSize: int64(proto.Evidence.Size()),
|
||||
}, 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 {
|
||||
key := keyPending(ev)
|
||||
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)
|
||||
evpool.logger.Error("Failed to load light client attack evidence", "err", err, "key(height/hash)", key)
|
||||
return false
|
||||
}
|
||||
evInfo, err := bytesToInfo(evBytes)
|
||||
var trustedPb tmproto.LightClientAttackEvidence
|
||||
err = trustedPb.Unmarshal(evBytes)
|
||||
if err != nil {
|
||||
evpool.logger.Error("Failed to convert evidence from proto", "err", err, "evidence", lcae)
|
||||
evpool.logger.Error("Failed to convert light client attack evidence from bytes",
|
||||
"err", err, "key(height/hash)", key)
|
||||
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
|
||||
}
|
||||
trustedEv, err := types.LightClientAttackEvidenceFromProto(&trustedPb)
|
||||
if err != nil {
|
||||
evpool.logger.Error("Failed to convert light client attack evidence from protobuf",
|
||||
"err", err, "key(height/hash)", key)
|
||||
return false
|
||||
}
|
||||
// ensure that all the byzantine validators that the evidence pool has match the byzantine validators
|
||||
// in this evidence
|
||||
if trustedEv.ByzantineValidators == nil && lcae.ByzantineValidators != nil {
|
||||
return false
|
||||
}
|
||||
|
||||
if len(trustedEv.ByzantineValidators) != len(lcae.ByzantineValidators) {
|
||||
return false
|
||||
}
|
||||
|
||||
byzValsCopy := make([]*types.Validator, len(lcae.ByzantineValidators))
|
||||
for i, v := range lcae.ByzantineValidators {
|
||||
byzValsCopy[i] = v.Copy()
|
||||
}
|
||||
|
||||
// ensure that both validator arrays are in the same order
|
||||
sort.Sort(types.ValidatorsByVotingPower(byzValsCopy))
|
||||
|
||||
for idx, val := range trustedEv.ByzantineValidators {
|
||||
if !bytes.Equal(byzValsCopy[idx].Address, val.Address) {
|
||||
return false
|
||||
}
|
||||
if byzValsCopy[idx].VotingPower != val.VotingPower {
|
||||
return false
|
||||
}
|
||||
// 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
|
||||
}
|
||||
|
||||
@@ -482,8 +337,8 @@ func (evpool *Pool) isPending(evidence types.Evidence) bool {
|
||||
return ok
|
||||
}
|
||||
|
||||
func (evpool *Pool) addPendingEvidence(evInfo *info) error {
|
||||
evpb, err := evInfo.ToProto()
|
||||
func (evpool *Pool) addPendingEvidence(ev types.Evidence) error {
|
||||
evpb, err := types.EvidenceToProto(ev)
|
||||
if err != nil {
|
||||
return fmt.Errorf("unable to convert to proto, err: %w", err)
|
||||
}
|
||||
@@ -493,7 +348,7 @@ func (evpool *Pool) addPendingEvidence(evInfo *info) error {
|
||||
return fmt.Errorf("unable to marshal evidence: %w", err)
|
||||
}
|
||||
|
||||
key := keyPending(evInfo.Evidence)
|
||||
key := keyPending(ev)
|
||||
|
||||
err = evpool.evidenceStore.Set(key, evBytes)
|
||||
if err != nil {
|
||||
@@ -513,31 +368,80 @@ func (evpool *Pool) removePendingEvidence(evidence types.Evidence) {
|
||||
}
|
||||
}
|
||||
|
||||
// markEvidenceAsCommitted processes all the evidence in the block, marking it as
|
||||
// committed and removing it from the pending database.
|
||||
func (evpool *Pool) markEvidenceAsCommitted(evidence types.EvidenceList) {
|
||||
blockEvidenceMap := make(map[string]struct{}, len(evidence))
|
||||
for _, ev := range evidence {
|
||||
if evpool.isPending(ev) {
|
||||
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)
|
||||
|
||||
h := gogotypes.Int64Value{Value: ev.Height()}
|
||||
evBytes, err := proto.Marshal(&h)
|
||||
if err != nil {
|
||||
evpool.logger.Error("failed to marshal committed evidence", "err", err, "key(height/hash)", key)
|
||||
continue
|
||||
}
|
||||
|
||||
if err := evpool.evidenceStore.Set(key, evBytes); err != nil {
|
||||
evpool.logger.Error("Unable to save committed evidence", "err", err, "key(height/hash)", key)
|
||||
}
|
||||
}
|
||||
|
||||
// remove committed evidence from the clist
|
||||
if len(blockEvidenceMap) != 0 {
|
||||
evpool.removeEvidenceFromList(blockEvidenceMap)
|
||||
}
|
||||
}
|
||||
|
||||
// listEvidence retrieves lists evidence from oldest to newest within maxBytes.
|
||||
// If maxBytes is -1, there's no cap on the size of returned evidence.
|
||||
func (evpool *Pool) listEvidence(prefixKey byte, maxBytes int64) ([]types.Evidence, int64, error) {
|
||||
var totalSize int64
|
||||
var evidence []types.Evidence
|
||||
var (
|
||||
evSize int64
|
||||
totalSize int64
|
||||
evidence []types.Evidence
|
||||
evList tmproto.EvidenceList // used for calculating the bytes size
|
||||
)
|
||||
|
||||
iter, err := dbm.IteratePrefix(evpool.evidenceStore, []byte{prefixKey})
|
||||
if err != nil {
|
||||
return nil, totalSize, fmt.Errorf("database error: %v", err)
|
||||
}
|
||||
defer iter.Close()
|
||||
for ; iter.Valid(); iter.Next() {
|
||||
evInfo, err := bytesToInfo(iter.Value())
|
||||
var evpb tmproto.Evidence
|
||||
err := evpb.Unmarshal(iter.Value())
|
||||
if err != nil {
|
||||
return evidence, totalSize, err
|
||||
}
|
||||
evList.Evidence = append(evList.Evidence, evpb)
|
||||
evSize = int64(evList.Size())
|
||||
if maxBytes != -1 && evSize > maxBytes {
|
||||
if err := iter.Error(); err != nil {
|
||||
return evidence, totalSize, err
|
||||
}
|
||||
return evidence, totalSize, nil
|
||||
}
|
||||
|
||||
ev, err := types.EvidenceFromProto(&evpb)
|
||||
if err != nil {
|
||||
return nil, totalSize, err
|
||||
}
|
||||
|
||||
totalSize += evInfo.ByteSize
|
||||
|
||||
if maxBytes != -1 && totalSize > maxBytes {
|
||||
return evidence, totalSize - evInfo.ByteSize, nil
|
||||
}
|
||||
|
||||
evidence = append(evidence, evInfo.Evidence)
|
||||
totalSize = evSize
|
||||
evidence = append(evidence, ev)
|
||||
}
|
||||
|
||||
if err := iter.Error(); err != nil {
|
||||
return evidence, totalSize, err
|
||||
}
|
||||
return evidence, totalSize, nil
|
||||
}
|
||||
|
||||
@@ -550,22 +454,22 @@ func (evpool *Pool) removeExpiredPendingEvidence() (int64, time.Time) {
|
||||
defer iter.Close()
|
||||
blockEvidenceMap := make(map[string]struct{})
|
||||
for ; iter.Valid(); iter.Next() {
|
||||
evInfo, err := bytesToInfo(iter.Value())
|
||||
ev, err := bytesToEv(iter.Value())
|
||||
if err != nil {
|
||||
evpool.logger.Error("Error in transition evidence from protobuf", "err", err)
|
||||
continue
|
||||
}
|
||||
if !evpool.isExpired(evInfo.Evidence.Height(), evInfo.Time) {
|
||||
if !evpool.isExpired(ev.Height(), ev.Time()) {
|
||||
if len(blockEvidenceMap) != 0 {
|
||||
evpool.removeEvidenceFromList(blockEvidenceMap)
|
||||
}
|
||||
|
||||
// return the height and 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)
|
||||
return ev.Height() + evpool.State().ConsensusParams.Evidence.MaxAgeNumBlocks + 1,
|
||||
ev.Time().Add(evpool.State().ConsensusParams.Evidence.MaxAgeDuration).Add(time.Second)
|
||||
}
|
||||
evpool.removePendingEvidence(evInfo.Evidence)
|
||||
blockEvidenceMap[evMapKey(evInfo.Evidence)] = struct{}{}
|
||||
evpool.removePendingEvidence(ev)
|
||||
blockEvidenceMap[evMapKey(ev)] = struct{}{}
|
||||
}
|
||||
// We either have no pending evidence or all evidence has expired
|
||||
if len(blockEvidenceMap) != 0 {
|
||||
@@ -593,14 +497,14 @@ func (evpool *Pool) updateState(state sm.State) {
|
||||
evpool.state = state
|
||||
}
|
||||
|
||||
func bytesToInfo(evBytes []byte) (info, error) {
|
||||
var evpb evproto.Info
|
||||
func bytesToEv(evBytes []byte) (types.Evidence, error) {
|
||||
var evpb tmproto.Evidence
|
||||
err := evpb.Unmarshal(evBytes)
|
||||
if err != nil {
|
||||
return info{}, err
|
||||
return &types.DuplicateVoteEvidence{}, err
|
||||
}
|
||||
|
||||
return infoFromProto(&evpb)
|
||||
return types.EvidenceFromProto(&evpb)
|
||||
}
|
||||
|
||||
func evMapKey(ev types.Evidence) string {
|
||||
|
||||
@@ -11,7 +11,6 @@ import (
|
||||
|
||||
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"
|
||||
@@ -45,7 +44,7 @@ func TestEvidencePoolBasic(t *testing.T) {
|
||||
blockStore = &mocks.BlockStore{}
|
||||
)
|
||||
|
||||
valSet, privVals := types.RandValidatorSet(3, 10)
|
||||
valSet, privVals := types.RandValidatorSet(1, 10)
|
||||
|
||||
blockStore.On("LoadBlockMeta", mock.AnythingOfType("int64")).Return(
|
||||
&types.BlockMeta{Header: types.Header{Time: defaultEvidenceTime}},
|
||||
@@ -83,9 +82,10 @@ func TestEvidencePoolBasic(t *testing.T) {
|
||||
next := pool.EvidenceFront()
|
||||
assert.Equal(t, ev, next.Value.(types.Evidence))
|
||||
|
||||
evs, size = pool.PendingEvidence(defaultEvidenceMaxBytes)
|
||||
const evidenceBytes int64 = 372
|
||||
evs, size = pool.PendingEvidence(evidenceBytes)
|
||||
assert.Equal(t, 1, len(evs))
|
||||
assert.Equal(t, int64(357), size) // check that the size of the single evidence in bytes is correct
|
||||
assert.Equal(t, evidenceBytes, size) // check that the size of the single evidence in bytes is correct
|
||||
|
||||
// shouldn't be able to add evidence twice
|
||||
assert.NoError(t, pool.AddEvidence(ev))
|
||||
@@ -108,7 +108,7 @@ 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.Add(time.Duration(height) * time.Minute)}}
|
||||
return &types.BlockMeta{Header: types.Header{Time: defaultEvidenceTime}}
|
||||
}
|
||||
return &types.BlockMeta{Header: types.Header{Time: expiredEvidenceTime}}
|
||||
})
|
||||
@@ -127,6 +127,7 @@ func TestAddExpiredEvidence(t *testing.T) {
|
||||
{height - 1, expiredEvidenceTime, false, "valid evidence (despite old time)"},
|
||||
{expiredHeight - 1, expiredEvidenceTime, true,
|
||||
"evidence from height 1 (created at: 2019-01-01 00:00:00 +0000 UTC) is too old"},
|
||||
{height, defaultEvidenceTime.Add(1 * time.Minute), true, "evidence time and block time is different"},
|
||||
}
|
||||
|
||||
for _, tc := range testCases {
|
||||
@@ -147,15 +148,13 @@ func TestAddEvidenceFromConsensus(t *testing.T) {
|
||||
var height int64 = 10
|
||||
pool, val := defaultTestPool(height)
|
||||
ev := types.NewMockDuplicateVoteEvidenceWithValidator(height, defaultEvidenceTime, val, evidenceChainID)
|
||||
valSet := types.NewValidatorSet([]*types.Validator{val.ExtractIntoValidator(2)})
|
||||
err := pool.AddEvidenceFromConsensus(ev, defaultEvidenceTime, valSet)
|
||||
err := pool.AddEvidenceFromConsensus(ev)
|
||||
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)}))
|
||||
err = pool.AddEvidenceFromConsensus(ev)
|
||||
assert.NoError(t, err)
|
||||
evs, _ := pool.PendingEvidence(defaultEvidenceMaxBytes)
|
||||
assert.Equal(t, 1, len(evs))
|
||||
@@ -167,11 +166,12 @@ func TestEvidencePoolUpdate(t *testing.T) {
|
||||
state := pool.State()
|
||||
|
||||
// create new block (no need to save it to blockStore)
|
||||
prunedEv := types.NewMockDuplicateVoteEvidenceWithValidator(1, defaultEvidenceTime,
|
||||
prunedEv := types.NewMockDuplicateVoteEvidenceWithValidator(1, defaultEvidenceTime.Add(1*time.Minute),
|
||||
val, evidenceChainID)
|
||||
err := pool.AddEvidence(prunedEv)
|
||||
require.NoError(t, err)
|
||||
ev := types.NewMockDuplicateVoteEvidenceWithValidator(height, defaultEvidenceTime, val, evidenceChainID)
|
||||
ev := types.NewMockDuplicateVoteEvidenceWithValidator(height, defaultEvidenceTime.Add(21*time.Minute),
|
||||
val, evidenceChainID)
|
||||
lastCommit := makeCommit(height, val.PrivKey.PubKey().Address())
|
||||
block := types.MakeBlock(height+1, []types.Tx{}, lastCommit, []types.Evidence{ev})
|
||||
// update state (partially)
|
||||
@@ -180,22 +180,7 @@ func TestEvidencePoolUpdate(t *testing.T) {
|
||||
err = pool.CheckEvidence(types.EvidenceList{ev})
|
||||
require.NoError(t, err)
|
||||
|
||||
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)
|
||||
evList, _ := pool.PendingEvidence(defaultEvidenceMaxBytes)
|
||||
assert.Equal(t, 1, len(evList))
|
||||
|
||||
pool.Update(state)
|
||||
|
||||
pool.Update(state, block.Evidence.Evidence)
|
||||
// a) Update marks evidence as committed so pending evidence should be empty
|
||||
evList, evSize := pool.PendingEvidence(defaultEvidenceMaxBytes)
|
||||
assert.Empty(t, evList)
|
||||
@@ -206,14 +191,13 @@ func TestEvidencePoolUpdate(t *testing.T) {
|
||||
if assert.Error(t, err) {
|
||||
assert.Equal(t, "evidence was already committed", err.(*types.ErrInvalidEvidence).Reason.Error())
|
||||
}
|
||||
|
||||
assert.Empty(t, pool.ABCIEvidence(height, []types.Evidence{}))
|
||||
}
|
||||
|
||||
func TestVerifyPendingEvidencePasses(t *testing.T) {
|
||||
var height int64 = 1
|
||||
pool, val := defaultTestPool(height)
|
||||
ev := types.NewMockDuplicateVoteEvidenceWithValidator(height, defaultEvidenceTime, val, evidenceChainID)
|
||||
ev := types.NewMockDuplicateVoteEvidenceWithValidator(height, defaultEvidenceTime.Add(1*time.Minute),
|
||||
val, evidenceChainID)
|
||||
err := pool.AddEvidence(ev)
|
||||
require.NoError(t, err)
|
||||
|
||||
@@ -224,20 +208,27 @@ func TestVerifyPendingEvidencePasses(t *testing.T) {
|
||||
func TestVerifyDuplicatedEvidenceFails(t *testing.T) {
|
||||
var height int64 = 1
|
||||
pool, val := defaultTestPool(height)
|
||||
ev := types.NewMockDuplicateVoteEvidenceWithValidator(height, defaultEvidenceTime, val, evidenceChainID)
|
||||
ev := types.NewMockDuplicateVoteEvidenceWithValidator(height, defaultEvidenceTime.Add(1*time.Minute),
|
||||
val, evidenceChainID)
|
||||
err := pool.CheckEvidence(types.EvidenceList{ev, ev})
|
||||
if assert.Error(t, err) {
|
||||
assert.Equal(t, "duplicate evidence", err.(*types.ErrInvalidEvidence).Reason.Error())
|
||||
}
|
||||
}
|
||||
|
||||
// check that
|
||||
// check that valid light client evidence is correctly validated and stored in
|
||||
// evidence pool
|
||||
func TestCheckEvidenceWithLightClientAttack(t *testing.T) {
|
||||
nValidators := 5
|
||||
conflictingVals, conflictingPrivVals := types.RandValidatorSet(nValidators, 10)
|
||||
trustedHeader := makeHeaderRandom(10)
|
||||
var (
|
||||
nValidators = 5
|
||||
validatorPower int64 = 10
|
||||
height int64 = 10
|
||||
)
|
||||
conflictingVals, conflictingPrivVals := types.RandValidatorSet(nValidators, validatorPower)
|
||||
trustedHeader := makeHeaderRandom(height)
|
||||
trustedHeader.Time = defaultEvidenceTime
|
||||
|
||||
conflictingHeader := makeHeaderRandom(10)
|
||||
conflictingHeader := makeHeaderRandom(height)
|
||||
conflictingHeader.ValidatorsHash = conflictingVals.Hash()
|
||||
|
||||
trustedHeader.ValidatorsHash = conflictingHeader.ValidatorsHash
|
||||
@@ -249,8 +240,8 @@ func TestCheckEvidenceWithLightClientAttack(t *testing.T) {
|
||||
// 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)
|
||||
voteSet := types.NewVoteSet(evidenceChainID, height, 1, tmproto.SignedMsgType(2), conflictingVals)
|
||||
commit, err := types.MakeCommit(blockID, height, 1, voteSet, conflictingPrivVals, defaultEvidenceTime)
|
||||
require.NoError(t, err)
|
||||
ev := &types.LightClientAttackEvidence{
|
||||
ConflictingBlock: &types.LightBlock{
|
||||
@@ -260,12 +251,16 @@ func TestCheckEvidenceWithLightClientAttack(t *testing.T) {
|
||||
},
|
||||
ValidatorSet: conflictingVals,
|
||||
},
|
||||
CommonHeight: 10,
|
||||
CommonHeight: 10,
|
||||
TotalVotingPower: int64(nValidators) * validatorPower,
|
||||
ByzantineValidators: conflictingVals.Validators,
|
||||
Timestamp: defaultEvidenceTime,
|
||||
}
|
||||
|
||||
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)
|
||||
trustedVoteSet := types.NewVoteSet(evidenceChainID, height, 1, tmproto.SignedMsgType(2), conflictingVals)
|
||||
trustedCommit, err := types.MakeCommit(trustedBlockID, height, 1, trustedVoteSet, conflictingPrivVals,
|
||||
defaultEvidenceTime)
|
||||
require.NoError(t, err)
|
||||
|
||||
state := sm.State{
|
||||
@@ -274,11 +269,11 @@ func TestCheckEvidenceWithLightClientAttack(t *testing.T) {
|
||||
ConsensusParams: *types.DefaultConsensusParams(),
|
||||
}
|
||||
stateStore := &smmocks.Store{}
|
||||
stateStore.On("LoadValidators", int64(10)).Return(conflictingVals, nil)
|
||||
stateStore.On("LoadValidators", height).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)
|
||||
blockStore.On("LoadBlockMeta", height).Return(&types.BlockMeta{Header: *trustedHeader})
|
||||
blockStore.On("LoadBlockCommit", height).Return(trustedCommit)
|
||||
|
||||
pool, err := evidence.NewPool(dbm.NewMemDB(), stateStore, blockStore)
|
||||
require.NoError(t, err)
|
||||
@@ -291,17 +286,14 @@ func TestCheckEvidenceWithLightClientAttack(t *testing.T) {
|
||||
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
|
||||
// hence this should fail
|
||||
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)
|
||||
}
|
||||
|
||||
// Tests that restarting the evidence pool after a potential failure will recover the
|
||||
// pending evidence and continue to gossip it
|
||||
func TestRecoverPendingEvidence(t *testing.T) {
|
||||
height := int64(10)
|
||||
val := types.NewMockPV()
|
||||
@@ -316,9 +308,9 @@ func TestRecoverPendingEvidence(t *testing.T) {
|
||||
require.NoError(t, err)
|
||||
pool.SetLogger(log.TestingLogger())
|
||||
goodEvidence := types.NewMockDuplicateVoteEvidenceWithValidator(height,
|
||||
defaultEvidenceTime, val, evidenceChainID)
|
||||
defaultEvidenceTime.Add(10*time.Minute), val, evidenceChainID)
|
||||
expiredEvidence := types.NewMockDuplicateVoteEvidenceWithValidator(int64(1),
|
||||
defaultEvidenceTime, val, evidenceChainID)
|
||||
defaultEvidenceTime.Add(1*time.Minute), val, evidenceChainID)
|
||||
err = pool.AddEvidence(goodEvidence)
|
||||
require.NoError(t, err)
|
||||
err = pool.AddEvidence(expiredEvidence)
|
||||
|
||||
@@ -7,7 +7,6 @@ import (
|
||||
clist "github.com/tendermint/tendermint/libs/clist"
|
||||
"github.com/tendermint/tendermint/libs/log"
|
||||
"github.com/tendermint/tendermint/p2p"
|
||||
ep "github.com/tendermint/tendermint/proto/tendermint/evidence"
|
||||
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
)
|
||||
@@ -128,7 +127,7 @@ func (evR *Reactor) broadcastEvidenceRoutine(peer p2p.Peer) {
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
evR.Logger.Debug("Gossiping evidence to peer", "ev", ev, "peer", peer.ID())
|
||||
success := peer.Send(EvidenceChannel, msgBytes)
|
||||
if !success {
|
||||
time.Sleep(peerRetryMessageIntervalMS * time.Millisecond)
|
||||
@@ -210,16 +209,15 @@ type PeerState interface {
|
||||
// encodemsg takes a array of evidence
|
||||
// returns the byte encoding of the List Message
|
||||
func encodeMsg(evis []types.Evidence) ([]byte, error) {
|
||||
evi := make([]*tmproto.Evidence, len(evis))
|
||||
evi := make([]tmproto.Evidence, len(evis))
|
||||
for i := 0; i < len(evis); i++ {
|
||||
ev, err := types.EvidenceToProto(evis[i])
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
evi[i] = ev
|
||||
evi[i] = *ev
|
||||
}
|
||||
|
||||
epl := ep.List{
|
||||
epl := tmproto.EvidenceList{
|
||||
Evidence: evi,
|
||||
}
|
||||
|
||||
@@ -229,14 +227,14 @@ func encodeMsg(evis []types.Evidence) ([]byte, error) {
|
||||
// decodemsg takes an array of bytes
|
||||
// returns an array of evidence
|
||||
func decodeMsg(bz []byte) (evis []types.Evidence, err error) {
|
||||
lm := ep.List{}
|
||||
lm := tmproto.EvidenceList{}
|
||||
if err := lm.Unmarshal(bz); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
evis = make([]types.Evidence, len(lm.Evidence))
|
||||
for i := 0; i < len(lm.Evidence); i++ {
|
||||
ev, err := types.EvidenceFromProto(lm.Evidence[i])
|
||||
ev, err := types.EvidenceFromProto(&lm.Evidence[i])
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
@@ -21,7 +21,6 @@ import (
|
||||
"github.com/tendermint/tendermint/evidence/mocks"
|
||||
"github.com/tendermint/tendermint/libs/log"
|
||||
"github.com/tendermint/tendermint/p2p"
|
||||
ep "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"
|
||||
@@ -119,15 +118,17 @@ func TestReactorsGossipNoCommittedEvidence(t *testing.T) {
|
||||
var height int64 = 10
|
||||
|
||||
// DB1 is ahead of DB2
|
||||
stateDB1 := initializeValidatorState(val, height)
|
||||
stateDB1 := initializeValidatorState(val, height-1)
|
||||
stateDB2 := initializeValidatorState(val, height-2)
|
||||
state, err := stateDB1.Load()
|
||||
require.NoError(t, err)
|
||||
state.LastBlockHeight++
|
||||
|
||||
// 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))
|
||||
pools[0].Update(state, evList)
|
||||
require.EqualValues(t, uint32(0), pools[0].Size())
|
||||
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
@@ -150,7 +151,7 @@ func TestReactorsGossipNoCommittedEvidence(t *testing.T) {
|
||||
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)
|
||||
time.Date(2019, 1, 1, 0, 0, 0, 0, time.UTC), val, state.ChainID)
|
||||
err := pools[0].AddEvidence(ev)
|
||||
require.NoError(t, err)
|
||||
evList[i] = ev
|
||||
@@ -160,18 +161,19 @@ func TestReactorsGossipNoCommittedEvidence(t *testing.T) {
|
||||
time.Sleep(300 * time.Millisecond)
|
||||
|
||||
// the second pool should only have received the first evidence because it is behind
|
||||
peerEv, _ := pools[1].PendingEvidence(1000)
|
||||
peerEv, _ := pools[1].PendingEvidence(10000)
|
||||
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]})
|
||||
state.LastBlockHeight++
|
||||
pools[0].Update(state, []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})
|
||||
pools[1].Update(state, types.EvidenceList{})
|
||||
peer = reactors[0].Switch.Peers().List()[0]
|
||||
ps = peerState{height}
|
||||
peer.Set(types.PeerStateKey, ps)
|
||||
@@ -180,7 +182,7 @@ func TestReactorsGossipNoCommittedEvidence(t *testing.T) {
|
||||
time.Sleep(300 * time.Millisecond)
|
||||
|
||||
peerEv, _ = pools[1].PendingEvidence(1000)
|
||||
assert.EqualValues(t, evList[0:1], peerEv)
|
||||
assert.EqualValues(t, []types.Evidence{evList[0], evList[1]}, peerEv)
|
||||
}
|
||||
|
||||
// evidenceLogger is a TestingLogger which uses a different
|
||||
@@ -331,27 +333,39 @@ 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))
|
||||
val := &types.Validator{
|
||||
Address: crypto.AddressHash([]byte("validator_address")),
|
||||
VotingPower: 10,
|
||||
}
|
||||
|
||||
valSet := types.NewValidatorSet([]*types.Validator{val})
|
||||
|
||||
dupl := types.NewDuplicateVoteEvidence(
|
||||
exampleVote(1),
|
||||
exampleVote(2),
|
||||
defaultEvidenceTime,
|
||||
valSet,
|
||||
)
|
||||
|
||||
testCases := []struct {
|
||||
testName string
|
||||
evidenceList []types.Evidence
|
||||
expBytes string
|
||||
}{
|
||||
{"DuplicateVoteEvidence", []types.Evidence{dupl}, "0af9010af6010a79080210031802224a0a208b01023386c371778ecb6368573e539afc3cc860ec3a2f614e54fe5652f4fc80122608c0843d122072db3d959635dff1bb567bedaa70573392c5159666a3f8caf11e413aac52207a2a0b08b1d381d20510809dca6f32146af1f4111082efb388211bc72c55bcd61e9ac3d538d5bb031279080110031802224a0a208b01023386c371778ecb6368573e539afc3cc860ec3a2f614e54fe5652f4fc80122608c0843d122072db3d959635dff1bb567bedaa70573392c5159666a3f8caf11e413aac52207a2a0b08b1d381d20510809dca6f32146af1f4111082efb388211bc72c55bcd61e9ac3d538d5bb03"},
|
||||
{"DuplicateVoteEvidence", []types.Evidence{dupl}, "0a85020a82020a79080210031802224a0a208b01023386c371778ecb6368573e539afc3cc860ec3a2f614e54fe5652f4fc80122608c0843d122072db3d959635dff1bb567bedaa70573392c5159666a3f8caf11e413aac52207a2a0b08b1d381d20510809dca6f32146af1f4111082efb388211bc72c55bcd61e9ac3d538d5bb031279080110031802224a0a208b01023386c371778ecb6368573e539afc3cc860ec3a2f614e54fe5652f4fc80122608c0843d122072db3d959635dff1bb567bedaa70573392c5159666a3f8caf11e413aac52207a2a0b08b1d381d20510809dca6f32146af1f4111082efb388211bc72c55bcd61e9ac3d538d5bb03180a200a2a060880dbaae105"},
|
||||
}
|
||||
|
||||
for _, tc := range testCases {
|
||||
tc := tc
|
||||
|
||||
evi := make([]*tmproto.Evidence, len(tc.evidenceList))
|
||||
evi := make([]tmproto.Evidence, len(tc.evidenceList))
|
||||
for i := 0; i < len(tc.evidenceList); i++ {
|
||||
ev, err := types.EvidenceToProto(tc.evidenceList[i])
|
||||
require.NoError(t, err, tc.testName)
|
||||
evi[i] = ev
|
||||
evi[i] = *ev
|
||||
}
|
||||
|
||||
epl := ep.List{
|
||||
epl := tmproto.EvidenceList{
|
||||
Evidence: evi,
|
||||
}
|
||||
|
||||
|
||||
@@ -4,6 +4,7 @@ import (
|
||||
"bytes"
|
||||
"errors"
|
||||
"fmt"
|
||||
"sort"
|
||||
"time"
|
||||
|
||||
"github.com/tendermint/tendermint/light"
|
||||
@@ -16,7 +17,7 @@ import (
|
||||
// - it is from a key who was a validator at the given height
|
||||
// - 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) {
|
||||
func (evpool *Pool) verify(evidence types.Evidence) error {
|
||||
var (
|
||||
state = evpool.State()
|
||||
height = state.LastBlockHeight
|
||||
@@ -27,14 +28,18 @@ func (evpool *Pool) verify(evidence types.Evidence) (*info, error) {
|
||||
// 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())
|
||||
return fmt.Errorf("don't have header #%d", evidence.Height())
|
||||
}
|
||||
evTime := blockMeta.Header.Time
|
||||
if evidence.Time() != evTime {
|
||||
return fmt.Errorf("evidence has a different time to the block it is associated with (%v != %v)",
|
||||
evidence.Time(), evTime)
|
||||
}
|
||||
ageDuration := state.LastBlockTime.Sub(evTime)
|
||||
|
||||
// check that the evidence hasn't expired
|
||||
if ageDuration > evidenceParams.MaxAgeDuration && ageNumBlocks > evidenceParams.MaxAgeNumBlocks {
|
||||
return nil, fmt.Errorf(
|
||||
return 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(),
|
||||
evTime,
|
||||
@@ -48,62 +53,66 @@ func (evpool *Pool) verify(evidence types.Evidence) (*info, error) {
|
||||
case *types.DuplicateVoteEvidence:
|
||||
valSet, err := evpool.stateDB.LoadValidators(evidence.Height())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
return 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
|
||||
return VerifyDuplicateVote(ev, state.ChainID, valSet)
|
||||
|
||||
case *types.LightClientAttackEvidence:
|
||||
commonHeader, err := getSignedHeader(evpool.blockStore, evidence.Height())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
return err
|
||||
}
|
||||
commonVals, err := evpool.stateDB.LoadValidators(evidence.Height())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
return 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
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
err = VerifyLightClientAttack(ev, commonHeader, trustedHeader, commonVals, state.LastBlockTime,
|
||||
state.ConsensusParams.Evidence.MaxAgeDuration)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
return 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()
|
||||
validators := ev.GetByzantineValidators(commonVals, trustedHeader)
|
||||
// ensure this matches the validators that are listed in the evidence. They should be ordered based on power.
|
||||
if validators == nil && ev.ByzantineValidators != nil {
|
||||
return fmt.Errorf("expected nil validators from an amnesia light client attack but got %d",
|
||||
len(ev.ByzantineValidators))
|
||||
}
|
||||
|
||||
return &info{
|
||||
Evidence: evidence,
|
||||
Time: evTime,
|
||||
Validators: validators,
|
||||
TotalVotingPower: totalVotingPower,
|
||||
}, nil
|
||||
if exp, got := len(validators), len(ev.ByzantineValidators); exp != got {
|
||||
return fmt.Errorf("expected %d byzantine validators from evidence but got %d",
|
||||
exp, got)
|
||||
}
|
||||
|
||||
// ensure that both validator arrays are in the same order
|
||||
sort.Sort(types.ValidatorsByVotingPower(ev.ByzantineValidators))
|
||||
|
||||
for idx, val := range validators {
|
||||
if !bytes.Equal(ev.ByzantineValidators[idx].Address, val.Address) {
|
||||
return fmt.Errorf("evidence contained a different byzantine validator address to the one we were expecting."+
|
||||
"Expected %v, got %v", val.Address, ev.ByzantineValidators[idx].Address)
|
||||
}
|
||||
if ev.ByzantineValidators[idx].VotingPower != val.VotingPower {
|
||||
return fmt.Errorf("evidence contained a byzantine validator with a different power to the one we were expecting."+
|
||||
"Expected %d, got %d", val.VotingPower, ev.ByzantineValidators[idx].VotingPower)
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
default:
|
||||
return nil, fmt.Errorf("unrecognized evidence type: %T", evidence)
|
||||
return fmt.Errorf("unrecognized evidence type: %T", evidence)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
// VerifyLightClientAttack verifies LightClientAttackEvidence against the state of the full node. This involves
|
||||
@@ -134,8 +143,13 @@ func VerifyLightClientAttack(e *types.LightClientAttackEvidence, commonHeader, t
|
||||
}
|
||||
}
|
||||
|
||||
if evTotal, valsTotal := e.TotalVotingPower, commonVals.TotalVotingPower(); evTotal != valsTotal {
|
||||
return fmt.Errorf("total voting power from the evidence and our validator set does not match (%d != %d)",
|
||||
evTotal, valsTotal)
|
||||
}
|
||||
|
||||
if bytes.Equal(trustedHeader.Hash(), e.ConflictingBlock.Hash()) {
|
||||
return fmt.Errorf("trusted header hash matches the evidence conflicting header hash: %X",
|
||||
return fmt.Errorf("trusted header hash matches the evidence's conflicting header hash: %X",
|
||||
trustedHeader.Hash())
|
||||
}
|
||||
|
||||
@@ -186,6 +200,17 @@ func VerifyDuplicateVote(e *types.DuplicateVoteEvidence, chainID string, valSet
|
||||
return fmt.Errorf("address (%X) doesn't match pubkey (%v - %X)",
|
||||
addr, pubKey, pubKey.Address())
|
||||
}
|
||||
|
||||
// validator voting power and total voting power must match
|
||||
if val.VotingPower != e.ValidatorPower {
|
||||
return fmt.Errorf("validator power from evidence and our validator set does not match (%d != %d)",
|
||||
e.ValidatorPower, val.VotingPower)
|
||||
}
|
||||
if valSet.TotalVotingPower() != e.TotalVotingPower {
|
||||
return fmt.Errorf("total voting power from the evidence and our validator set does not match (%d != %d)",
|
||||
e.TotalVotingPower, valSet.TotalVotingPower())
|
||||
}
|
||||
|
||||
va := e.VoteA.ToProto()
|
||||
vb := e.VoteB.ToProto()
|
||||
// Signatures must be valid
|
||||
@@ -214,55 +239,6 @@ func getSignedHeader(blockStore BlockStore, height int64) (*types.SignedHeader,
|
||||
}, 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.
|
||||
@@ -274,11 +250,3 @@ func isInvalidHeader(trusted, conflicting *types.Header) bool {
|
||||
!bytes.Equal(trusted.AppHash, conflicting.AppHash) ||
|
||||
!bytes.Equal(trusted.LastResultsHash, conflicting.LastResultsHash)
|
||||
}
|
||||
|
||||
type lightClientAttackType int
|
||||
|
||||
const (
|
||||
lunaticType lightClientAttackType = iota + 1
|
||||
equivocationType
|
||||
amnesiaType
|
||||
)
|
||||
|
||||
@@ -9,7 +9,6 @@ import (
|
||||
|
||||
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"
|
||||
@@ -33,14 +32,14 @@ func TestVerifyLightClientAttack_Lunatic(t *testing.T) {
|
||||
conflictingPrivVals := append(commonPrivVals, newPrivVal)
|
||||
|
||||
commonHeader := makeHeaderRandom(4)
|
||||
commonHeader.Time = defaultEvidenceTime.Add(-1 * time.Hour)
|
||||
commonHeader.Time = defaultEvidenceTime
|
||||
trustedHeader := makeHeaderRandom(10)
|
||||
|
||||
conflictingHeader := makeHeaderRandom(10)
|
||||
conflictingHeader.Time = defaultEvidenceTime.Add(1 * time.Hour)
|
||||
conflictingHeader.ValidatorsHash = conflictingVals.Hash()
|
||||
|
||||
// we are simulating a duplicate vote attack where all the validators in the conflictingVals set
|
||||
// vote twice
|
||||
// we are simulating a lunatic light client attack
|
||||
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)
|
||||
@@ -53,7 +52,10 @@ func TestVerifyLightClientAttack_Lunatic(t *testing.T) {
|
||||
},
|
||||
ValidatorSet: conflictingVals,
|
||||
},
|
||||
CommonHeight: 4,
|
||||
CommonHeight: 4,
|
||||
TotalVotingPower: 20,
|
||||
ByzantineValidators: commonVals.Validators,
|
||||
Timestamp: defaultEvidenceTime,
|
||||
}
|
||||
|
||||
commonSignedHeader := &types.SignedHeader{
|
||||
@@ -72,16 +74,23 @@ func TestVerifyLightClientAttack_Lunatic(t *testing.T) {
|
||||
|
||||
// good pass -> no error
|
||||
err = evidence.VerifyLightClientAttack(ev, commonSignedHeader, trustedSignedHeader, commonVals,
|
||||
defaultEvidenceTime.Add(1*time.Minute), 2*time.Hour)
|
||||
defaultEvidenceTime.Add(2*time.Hour), 3*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)
|
||||
defaultEvidenceTime.Add(2*time.Hour), 3*time.Hour)
|
||||
assert.Error(t, err)
|
||||
|
||||
// evidence with different total validator power should fail
|
||||
ev.TotalVotingPower = 1
|
||||
err = evidence.VerifyLightClientAttack(ev, commonSignedHeader, trustedSignedHeader, commonVals,
|
||||
defaultEvidenceTime.Add(2*time.Hour), 3*time.Hour)
|
||||
assert.Error(t, err)
|
||||
ev.TotalVotingPower = 20
|
||||
|
||||
state := sm.State{
|
||||
LastBlockTime: defaultEvidenceTime.Add(1 * time.Minute),
|
||||
LastBlockTime: defaultEvidenceTime.Add(2 * time.Hour),
|
||||
LastBlockHeight: 11,
|
||||
ConsensusParams: *types.DefaultConsensusParams(),
|
||||
}
|
||||
@@ -105,27 +114,18 @@ func TestVerifyLightClientAttack_Lunatic(t *testing.T) {
|
||||
pendingEvs, _ := pool.PendingEvidence(state.ConsensusParams.Evidence.MaxBytes)
|
||||
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})
|
||||
// if we submit evidence only against a single byzantine validator when we see there are more validators then this
|
||||
// should return an error
|
||||
ev.ByzantineValidators = []*types.Validator{commonVals.Validators[0]}
|
||||
err = pool.CheckEvidence(evList)
|
||||
assert.Error(t, err)
|
||||
ev.ByzantineValidators = commonVals.Validators // restore evidence
|
||||
|
||||
abciEv := pool.ABCIEvidence(block.Height, block.Evidence.Evidence)
|
||||
expectedAbciEv := make([]abci.Evidence, len(commonVals.Validators))
|
||||
// If evidence is submitted with an altered timestamp it should return an error
|
||||
ev.Timestamp = defaultEvidenceTime.Add(1 * time.Minute)
|
||||
err = pool.CheckEvidence(evList)
|
||||
assert.Error(t, err)
|
||||
|
||||
// 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 TestVerifyLightClientAttack_Equivocation(t *testing.T) {
|
||||
@@ -155,7 +155,10 @@ func TestVerifyLightClientAttack_Equivocation(t *testing.T) {
|
||||
},
|
||||
ValidatorSet: conflictingVals,
|
||||
},
|
||||
CommonHeight: 10,
|
||||
CommonHeight: 10,
|
||||
ByzantineValidators: conflictingVals.Validators[:4],
|
||||
TotalVotingPower: 50,
|
||||
Timestamp: defaultEvidenceTime,
|
||||
}
|
||||
|
||||
trustedBlockID := makeBlockID(trustedHeader.Hash(), 1000, []byte("partshash"))
|
||||
@@ -168,12 +171,12 @@ func TestVerifyLightClientAttack_Equivocation(t *testing.T) {
|
||||
}
|
||||
|
||||
// good pass -> no error
|
||||
err = evidence.VerifyLightClientAttack(ev, trustedSignedHeader, trustedSignedHeader, nil,
|
||||
err = evidence.VerifyLightClientAttack(ev, trustedSignedHeader, trustedSignedHeader, conflictingVals,
|
||||
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,
|
||||
err = evidence.VerifyLightClientAttack(ev, trustedSignedHeader, ev.ConflictingBlock.SignedHeader, conflictingVals,
|
||||
defaultEvidenceTime.Add(1*time.Minute), 2*time.Hour)
|
||||
assert.Error(t, err)
|
||||
|
||||
@@ -208,31 +211,6 @@ func TestVerifyLightClientAttack_Equivocation(t *testing.T) {
|
||||
|
||||
pendingEvs, _ := pool.PendingEvidence(state.ConsensusParams.Evidence.MaxBytes)
|
||||
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
|
||||
}
|
||||
|
||||
assert.Equal(t, expectedAbciEv, abciEv)
|
||||
}
|
||||
|
||||
func TestVerifyLightClientAttack_Amnesia(t *testing.T) {
|
||||
@@ -261,7 +239,10 @@ func TestVerifyLightClientAttack_Amnesia(t *testing.T) {
|
||||
},
|
||||
ValidatorSet: conflictingVals,
|
||||
},
|
||||
CommonHeight: 10,
|
||||
CommonHeight: 10,
|
||||
ByzantineValidators: nil, // with amnesia evidence no validators are submitted as abci evidence
|
||||
TotalVotingPower: 50,
|
||||
Timestamp: defaultEvidenceTime,
|
||||
}
|
||||
|
||||
trustedBlockID := makeBlockID(trustedHeader.Hash(), 1000, []byte("partshash"))
|
||||
@@ -274,12 +255,12 @@ func TestVerifyLightClientAttack_Amnesia(t *testing.T) {
|
||||
}
|
||||
|
||||
// good pass -> no error
|
||||
err = evidence.VerifyLightClientAttack(ev, trustedSignedHeader, trustedSignedHeader, nil,
|
||||
err = evidence.VerifyLightClientAttack(ev, trustedSignedHeader, trustedSignedHeader, conflictingVals,
|
||||
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,
|
||||
err = evidence.VerifyLightClientAttack(ev, trustedSignedHeader, ev.ConflictingBlock.SignedHeader, conflictingVals,
|
||||
defaultEvidenceTime.Add(1*time.Minute), 2*time.Hour)
|
||||
assert.Error(t, err)
|
||||
|
||||
@@ -305,19 +286,6 @@ func TestVerifyLightClientAttack_Amnesia(t *testing.T) {
|
||||
|
||||
pendingEvs, _ := pool.PendingEvidence(state.ConsensusParams.Evidence.MaxBytes)
|
||||
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)
|
||||
}
|
||||
|
||||
type voteData struct {
|
||||
@@ -368,8 +336,11 @@ func TestVerifyDuplicateVoteEvidence(t *testing.T) {
|
||||
require.NoError(t, err)
|
||||
for _, c := range cases {
|
||||
ev := &types.DuplicateVoteEvidence{
|
||||
VoteA: c.vote1,
|
||||
VoteB: c.vote2,
|
||||
VoteA: c.vote1,
|
||||
VoteB: c.vote2,
|
||||
ValidatorPower: 1,
|
||||
TotalVotingPower: 1,
|
||||
Timestamp: defaultEvidenceTime,
|
||||
}
|
||||
if c.valid {
|
||||
assert.Nil(t, evidence.VerifyDuplicateVote(ev, chainID, valSet), "evidence should be valid")
|
||||
@@ -378,7 +349,14 @@ func TestVerifyDuplicateVoteEvidence(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
// create good evidence and correct validator power
|
||||
goodEv := types.NewMockDuplicateVoteEvidenceWithValidator(10, defaultEvidenceTime, val, chainID)
|
||||
goodEv.ValidatorPower = 1
|
||||
goodEv.TotalVotingPower = 1
|
||||
badEv := types.NewMockDuplicateVoteEvidenceWithValidator(10, defaultEvidenceTime, val, chainID)
|
||||
badTimeEv := types.NewMockDuplicateVoteEvidenceWithValidator(10, defaultEvidenceTime.Add(1*time.Minute), val, chainID)
|
||||
badTimeEv.ValidatorPower = 1
|
||||
badTimeEv.TotalVotingPower = 1
|
||||
state := sm.State{
|
||||
ChainID: chainID,
|
||||
LastBlockTime: defaultEvidenceTime.Add(1 * time.Minute),
|
||||
@@ -397,6 +375,16 @@ func TestVerifyDuplicateVoteEvidence(t *testing.T) {
|
||||
evList := types.EvidenceList{goodEv}
|
||||
err = pool.CheckEvidence(evList)
|
||||
assert.NoError(t, err)
|
||||
|
||||
// evidence with a different validator power should fail
|
||||
evList = types.EvidenceList{badEv}
|
||||
err = pool.CheckEvidence(evList)
|
||||
assert.Error(t, err)
|
||||
|
||||
// evidence with a different timestamp should fail
|
||||
evList = types.EvidenceList{badTimeEv}
|
||||
err = pool.CheckEvidence(evList)
|
||||
assert.Error(t, err)
|
||||
}
|
||||
|
||||
func makeVote(
|
||||
|
||||
@@ -78,24 +78,13 @@ func (c *Client) detectDivergence(ctx context.Context, primaryTrace []*types.Lig
|
||||
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,
|
||||
primaryEv := newLightClientAttackEvidence(primaryBlock, witnessTrace[len(witnessTrace)-1], witnessTrace[0])
|
||||
c.logger.Error("Attempted attack detected. Sending evidence againt primary by witness", "ev", primaryEv,
|
||||
"primary", c.primary, "witness", supportingWitness)
|
||||
c.sendEvidence(ctx, ev, supportingWitness)
|
||||
c.sendEvidence(ctx, primaryEv, 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
|
||||
@@ -111,23 +100,12 @@ func (c *Client) detectDivergence(ctx context.Context, primaryTrace []*types.Lig
|
||||
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,
|
||||
witnessEv := newLightClientAttackEvidence(witnessBlock, primaryTrace[len(primaryTrace)-1], primaryTrace[0])
|
||||
c.logger.Error("Sending evidence against witness by primary", "ev", witnessEv,
|
||||
"primary", c.primary, "witness", supportingWitness)
|
||||
c.sendEvidence(ctx, ev, c.primary)
|
||||
c.sendEvidence(ctx, witnessEv, c.primary)
|
||||
// We return the error and don't process anymore witnesses
|
||||
return e
|
||||
|
||||
@@ -245,14 +223,22 @@ func (c *Client) examineConflictingHeaderAgainstTrace(
|
||||
|
||||
}
|
||||
|
||||
// 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)
|
||||
// newLightClientAttackEvidence determines the type of attack and then forms the evidence filling out
|
||||
// all the fields such that it is ready to be sent to a full node.
|
||||
func newLightClientAttackEvidence(conflicted, trusted, common *types.LightBlock) *types.LightClientAttackEvidence {
|
||||
ev := &types.LightClientAttackEvidence{ConflictingBlock: conflicted}
|
||||
// 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.
|
||||
if ev.ConflictingHeaderIsInvalid(trusted.Header) {
|
||||
ev.CommonHeight = common.Height
|
||||
ev.Timestamp = common.Time
|
||||
ev.TotalVotingPower = common.ValidatorSet.TotalVotingPower()
|
||||
} else {
|
||||
ev.CommonHeight = trusted.Height
|
||||
ev.Timestamp = trusted.Time
|
||||
ev.TotalVotingPower = trusted.ValidatorSet.TotalVotingPower()
|
||||
}
|
||||
ev.ByzantineValidators = ev.GetByzantineValidators(common.ValidatorSet, trusted.SignedHeader)
|
||||
return ev
|
||||
}
|
||||
|
||||
@@ -265,10 +265,15 @@ func TestCreateProposalBlock(t *testing.T) {
|
||||
for currentBytes <= maxEvidenceBytes {
|
||||
ev := types.NewMockDuplicateVoteEvidenceWithValidator(height, time.Now(), privVals[0], "test-chain")
|
||||
currentBytes += int64(len(ev.Bytes()))
|
||||
err := evidencePool.AddEvidenceFromConsensus(ev, time.Now(), state.Validators)
|
||||
err := evidencePool.AddEvidenceFromConsensus(ev)
|
||||
require.NoError(t, err)
|
||||
}
|
||||
|
||||
evList, size := evidencePool.PendingEvidence(state.ConsensusParams.Evidence.MaxBytes)
|
||||
require.Less(t, size, state.ConsensusParams.Evidence.MaxBytes+1)
|
||||
evData := &types.EvidenceData{Evidence: evList}
|
||||
require.EqualValues(t, size, evData.ByteSize())
|
||||
|
||||
// fill the mempool with more txs
|
||||
// than can fit in a block
|
||||
txLength := 100
|
||||
|
||||
@@ -1,668 +0,0 @@
|
||||
// Code generated by protoc-gen-gogo. DO NOT EDIT.
|
||||
// source: tendermint/evidence/types.proto
|
||||
|
||||
package evidence
|
||||
|
||||
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.
|
||||
// A compilation error at this line likely means your copy of the
|
||||
// proto package needs to be updated.
|
||||
const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package
|
||||
|
||||
type List struct {
|
||||
Evidence []*types.Evidence `protobuf:"bytes,1,rep,name=evidence,proto3" json:"evidence,omitempty"`
|
||||
}
|
||||
|
||||
func (m *List) Reset() { *m = List{} }
|
||||
func (m *List) String() string { return proto.CompactTextString(m) }
|
||||
func (*List) ProtoMessage() {}
|
||||
func (*List) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_5e804d1c041a0e47, []int{0}
|
||||
}
|
||||
func (m *List) XXX_Unmarshal(b []byte) error {
|
||||
return m.Unmarshal(b)
|
||||
}
|
||||
func (m *List) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
if deterministic {
|
||||
return xxx_messageInfo_List.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 *List) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_List.Merge(m, src)
|
||||
}
|
||||
func (m *List) XXX_Size() int {
|
||||
return m.Size()
|
||||
}
|
||||
func (m *List) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_List.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_List proto.InternalMessageInfo
|
||||
|
||||
func (m *List) GetEvidence() []*types.Evidence {
|
||||
if m != nil {
|
||||
return m.Evidence
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
type Info struct {
|
||||
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{} }
|
||||
func (m *Info) String() string { return proto.CompactTextString(m) }
|
||||
func (*Info) ProtoMessage() {}
|
||||
func (*Info) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_5e804d1c041a0e47, []int{1}
|
||||
}
|
||||
func (m *Info) XXX_Unmarshal(b []byte) error {
|
||||
return m.Unmarshal(b)
|
||||
}
|
||||
func (m *Info) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
if deterministic {
|
||||
return xxx_messageInfo_Info.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 *Info) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_Info.Merge(m, src)
|
||||
}
|
||||
func (m *Info) XXX_Size() int {
|
||||
return m.Size()
|
||||
}
|
||||
func (m *Info) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_Info.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_Info proto.InternalMessageInfo
|
||||
|
||||
func (m *Info) GetEvidence() types.Evidence {
|
||||
if m != nil {
|
||||
return m.Evidence
|
||||
}
|
||||
return types.Evidence{}
|
||||
}
|
||||
|
||||
func (m *Info) GetTime() time.Time {
|
||||
if m != nil {
|
||||
return m.Time
|
||||
}
|
||||
return time.Time{}
|
||||
}
|
||||
|
||||
func (m *Info) GetValidators() []*types.Validator {
|
||||
if m != nil {
|
||||
return m.Validators
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *Info) GetTotalVotingPower() int64 {
|
||||
if m != nil {
|
||||
return m.TotalVotingPower
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func init() {
|
||||
proto.RegisterType((*List)(nil), "tendermint.evidence.List")
|
||||
proto.RegisterType((*Info)(nil), "tendermint.evidence.Info")
|
||||
}
|
||||
|
||||
func init() { proto.RegisterFile("tendermint/evidence/types.proto", fileDescriptor_5e804d1c041a0e47) }
|
||||
|
||||
var fileDescriptor_5e804d1c041a0e47 = []byte{
|
||||
// 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,
|
||||
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) {
|
||||
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 *List) MarshalTo(dAtA []byte) (int, error) {
|
||||
size := m.Size()
|
||||
return m.MarshalToSizedBuffer(dAtA[:size])
|
||||
}
|
||||
|
||||
func (m *List) MarshalToSizedBuffer(dAtA []byte) (int, error) {
|
||||
i := len(dAtA)
|
||||
_ = i
|
||||
var l int
|
||||
_ = l
|
||||
if len(m.Evidence) > 0 {
|
||||
for iNdEx := len(m.Evidence) - 1; iNdEx >= 0; iNdEx-- {
|
||||
{
|
||||
size, err := m.Evidence[iNdEx].MarshalToSizedBuffer(dAtA[:i])
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
i -= size
|
||||
i = encodeVarintTypes(dAtA, i, uint64(size))
|
||||
}
|
||||
i--
|
||||
dAtA[i] = 0xa
|
||||
}
|
||||
}
|
||||
return len(dAtA) - i, nil
|
||||
}
|
||||
|
||||
func (m *Info) 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 *Info) MarshalTo(dAtA []byte) (int, error) {
|
||||
size := m.Size()
|
||||
return m.MarshalToSizedBuffer(dAtA[:size])
|
||||
}
|
||||
|
||||
func (m *Info) MarshalToSizedBuffer(dAtA []byte) (int, error) {
|
||||
i := len(dAtA)
|
||||
_ = 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 {
|
||||
return 0, err
|
||||
}
|
||||
i -= size
|
||||
i = encodeVarintTypes(dAtA, i, uint64(size))
|
||||
}
|
||||
i--
|
||||
dAtA[i] = 0xa
|
||||
return len(dAtA) - i, nil
|
||||
}
|
||||
|
||||
func encodeVarintTypes(dAtA []byte, offset int, v uint64) int {
|
||||
offset -= sovTypes(v)
|
||||
base := offset
|
||||
for v >= 1<<7 {
|
||||
dAtA[offset] = uint8(v&0x7f | 0x80)
|
||||
v >>= 7
|
||||
offset++
|
||||
}
|
||||
dAtA[offset] = uint8(v)
|
||||
return base
|
||||
}
|
||||
func (m *List) Size() (n int) {
|
||||
if m == nil {
|
||||
return 0
|
||||
}
|
||||
var l int
|
||||
_ = l
|
||||
if len(m.Evidence) > 0 {
|
||||
for _, e := range m.Evidence {
|
||||
l = e.Size()
|
||||
n += 1 + l + sovTypes(uint64(l))
|
||||
}
|
||||
}
|
||||
return n
|
||||
}
|
||||
|
||||
func (m *Info) Size() (n int) {
|
||||
if m == nil {
|
||||
return 0
|
||||
}
|
||||
var l int
|
||||
_ = l
|
||||
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
|
||||
}
|
||||
|
||||
func sovTypes(x uint64) (n int) {
|
||||
return (math_bits.Len64(x|1) + 6) / 7
|
||||
}
|
||||
func sozTypes(x uint64) (n int) {
|
||||
return sovTypes(uint64((x << 1) ^ uint64((int64(x) >> 63))))
|
||||
}
|
||||
func (m *List) 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 ErrIntOverflowTypes
|
||||
}
|
||||
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: List: wiretype end group for non-group")
|
||||
}
|
||||
if fieldNum <= 0 {
|
||||
return fmt.Errorf("proto: List: illegal tag %d (wire type %d)", fieldNum, wire)
|
||||
}
|
||||
switch fieldNum {
|
||||
case 1:
|
||||
if wireType != 2 {
|
||||
return fmt.Errorf("proto: wrong wireType = %d for field Evidence", wireType)
|
||||
}
|
||||
var msglen int
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if shift >= 64 {
|
||||
return ErrIntOverflowTypes
|
||||
}
|
||||
if iNdEx >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := dAtA[iNdEx]
|
||||
iNdEx++
|
||||
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
|
||||
}
|
||||
m.Evidence = append(m.Evidence, &types.Evidence{})
|
||||
if err := m.Evidence[len(m.Evidence)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
|
||||
return err
|
||||
}
|
||||
iNdEx = postIndex
|
||||
default:
|
||||
iNdEx = preIndex
|
||||
skippy, err := skipTypes(dAtA[iNdEx:])
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if skippy < 0 {
|
||||
return ErrInvalidLengthTypes
|
||||
}
|
||||
if (iNdEx + skippy) < 0 {
|
||||
return ErrInvalidLengthTypes
|
||||
}
|
||||
if (iNdEx + skippy) > l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
iNdEx += skippy
|
||||
}
|
||||
}
|
||||
|
||||
if iNdEx > l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
return nil
|
||||
}
|
||||
func (m *Info) 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 ErrIntOverflowTypes
|
||||
}
|
||||
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: Info: wiretype end group for non-group")
|
||||
}
|
||||
if fieldNum <= 0 {
|
||||
return fmt.Errorf("proto: Info: illegal tag %d (wire type %d)", fieldNum, wire)
|
||||
}
|
||||
switch fieldNum {
|
||||
case 1:
|
||||
if wireType != 2 {
|
||||
return fmt.Errorf("proto: wrong wireType = %d for field Evidence", wireType)
|
||||
}
|
||||
var msglen int
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if shift >= 64 {
|
||||
return ErrIntOverflowTypes
|
||||
}
|
||||
if iNdEx >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := dAtA[iNdEx]
|
||||
iNdEx++
|
||||
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 := m.Evidence.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
|
||||
return err
|
||||
}
|
||||
iNdEx = postIndex
|
||||
case 2:
|
||||
if wireType != 2 {
|
||||
return fmt.Errorf("proto: wrong wireType = %d for field Time", wireType)
|
||||
}
|
||||
var msglen int
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if shift >= 64 {
|
||||
return ErrIntOverflowTypes
|
||||
}
|
||||
if iNdEx >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := dAtA[iNdEx]
|
||||
iNdEx++
|
||||
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 Validators", wireType)
|
||||
}
|
||||
var msglen int
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if shift >= 64 {
|
||||
return ErrIntOverflowTypes
|
||||
}
|
||||
if iNdEx >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := dAtA[iNdEx]
|
||||
iNdEx++
|
||||
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
|
||||
}
|
||||
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:])
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if skippy < 0 {
|
||||
return ErrInvalidLengthTypes
|
||||
}
|
||||
if (iNdEx + skippy) < 0 {
|
||||
return ErrInvalidLengthTypes
|
||||
}
|
||||
if (iNdEx + skippy) > l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
iNdEx += skippy
|
||||
}
|
||||
}
|
||||
|
||||
if iNdEx > l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
return nil
|
||||
}
|
||||
func skipTypes(dAtA []byte) (n int, err error) {
|
||||
l := len(dAtA)
|
||||
iNdEx := 0
|
||||
depth := 0
|
||||
for iNdEx < l {
|
||||
var wire uint64
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if shift >= 64 {
|
||||
return 0, ErrIntOverflowTypes
|
||||
}
|
||||
if iNdEx >= l {
|
||||
return 0, io.ErrUnexpectedEOF
|
||||
}
|
||||
b := dAtA[iNdEx]
|
||||
iNdEx++
|
||||
wire |= (uint64(b) & 0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
wireType := int(wire & 0x7)
|
||||
switch wireType {
|
||||
case 0:
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if shift >= 64 {
|
||||
return 0, ErrIntOverflowTypes
|
||||
}
|
||||
if iNdEx >= l {
|
||||
return 0, io.ErrUnexpectedEOF
|
||||
}
|
||||
iNdEx++
|
||||
if dAtA[iNdEx-1] < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
case 1:
|
||||
iNdEx += 8
|
||||
case 2:
|
||||
var length int
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if shift >= 64 {
|
||||
return 0, ErrIntOverflowTypes
|
||||
}
|
||||
if iNdEx >= l {
|
||||
return 0, io.ErrUnexpectedEOF
|
||||
}
|
||||
b := dAtA[iNdEx]
|
||||
iNdEx++
|
||||
length |= (int(b) & 0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
if length < 0 {
|
||||
return 0, ErrInvalidLengthTypes
|
||||
}
|
||||
iNdEx += length
|
||||
case 3:
|
||||
depth++
|
||||
case 4:
|
||||
if depth == 0 {
|
||||
return 0, ErrUnexpectedEndOfGroupTypes
|
||||
}
|
||||
depth--
|
||||
case 5:
|
||||
iNdEx += 4
|
||||
default:
|
||||
return 0, fmt.Errorf("proto: illegal wireType %d", wireType)
|
||||
}
|
||||
if iNdEx < 0 {
|
||||
return 0, ErrInvalidLengthTypes
|
||||
}
|
||||
if depth == 0 {
|
||||
return iNdEx, nil
|
||||
}
|
||||
}
|
||||
return 0, io.ErrUnexpectedEOF
|
||||
}
|
||||
|
||||
var (
|
||||
ErrInvalidLengthTypes = fmt.Errorf("proto: negative length found during unmarshaling")
|
||||
ErrIntOverflowTypes = fmt.Errorf("proto: integer overflow")
|
||||
ErrUnexpectedEndOfGroupTypes = fmt.Errorf("proto: unexpected end of group")
|
||||
)
|
||||
@@ -1,20 +0,0 @@
|
||||
syntax = "proto3";
|
||||
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 {
|
||||
repeated tendermint.types.Evidence evidence = 1;
|
||||
}
|
||||
|
||||
message Info {
|
||||
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;
|
||||
}
|
||||
@@ -26,7 +26,7 @@ const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package
|
||||
type Block struct {
|
||||
Header Header `protobuf:"bytes,1,opt,name=header,proto3" json:"header"`
|
||||
Data Data `protobuf:"bytes,2,opt,name=data,proto3" json:"data"`
|
||||
Evidence EvidenceData `protobuf:"bytes,3,opt,name=evidence,proto3" json:"evidence"`
|
||||
Evidence EvidenceList `protobuf:"bytes,3,opt,name=evidence,proto3" json:"evidence"`
|
||||
LastCommit *Commit `protobuf:"bytes,4,opt,name=last_commit,json=lastCommit,proto3" json:"last_commit,omitempty"`
|
||||
}
|
||||
|
||||
@@ -77,11 +77,11 @@ func (m *Block) GetData() Data {
|
||||
return Data{}
|
||||
}
|
||||
|
||||
func (m *Block) GetEvidence() EvidenceData {
|
||||
func (m *Block) GetEvidence() EvidenceList {
|
||||
if m != nil {
|
||||
return m.Evidence
|
||||
}
|
||||
return EvidenceData{}
|
||||
return EvidenceList{}
|
||||
}
|
||||
|
||||
func (m *Block) GetLastCommit() *Commit {
|
||||
@@ -98,7 +98,7 @@ func init() {
|
||||
func init() { proto.RegisterFile("tendermint/types/block.proto", fileDescriptor_70840e82f4357ab1) }
|
||||
|
||||
var fileDescriptor_70840e82f4357ab1 = []byte{
|
||||
// 262 bytes of a gzipped FileDescriptorProto
|
||||
// 266 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x92, 0x29, 0x49, 0xcd, 0x4b,
|
||||
0x49, 0x2d, 0xca, 0xcd, 0xcc, 0x2b, 0xd1, 0x2f, 0xa9, 0x2c, 0x48, 0x2d, 0xd6, 0x4f, 0xca, 0xc9,
|
||||
0x4f, 0xce, 0xd6, 0x2b, 0x28, 0xca, 0x2f, 0xc9, 0x17, 0x12, 0x40, 0xc8, 0xea, 0x81, 0x65, 0xa5,
|
||||
@@ -109,13 +109,13 @@ var fileDescriptor_70840e82f4357ab1 = []byte{
|
||||
0xf2, 0x4e, 0x2c, 0x27, 0xee, 0xc9, 0x33, 0x04, 0x41, 0x55, 0x0b, 0x19, 0x70, 0xb1, 0xa4, 0x24,
|
||||
0x96, 0x24, 0x4a, 0x30, 0x81, 0x75, 0x89, 0x61, 0xea, 0x72, 0x49, 0x2c, 0x49, 0x84, 0xea, 0x01,
|
||||
0xab, 0x14, 0x72, 0xe0, 0xe2, 0x80, 0xb9, 0x42, 0x82, 0x19, 0xac, 0x4b, 0x0e, 0x53, 0x97, 0x2b,
|
||||
0x54, 0x05, 0x92, 0x6e, 0xb8, 0x2e, 0x21, 0x4b, 0x2e, 0xee, 0x9c, 0xc4, 0xe2, 0x92, 0xf8, 0xe4,
|
||||
0xfc, 0xdc, 0xdc, 0xcc, 0x12, 0x09, 0x16, 0x5c, 0x0e, 0x76, 0x06, 0xcb, 0x07, 0x71, 0x81, 0x14,
|
||||
0x43, 0xd8, 0x4e, 0x81, 0x27, 0x1e, 0xc9, 0x31, 0x5e, 0x78, 0x24, 0xc7, 0xf8, 0xe0, 0x91, 0x1c,
|
||||
0xe3, 0x84, 0xc7, 0x72, 0x0c, 0x17, 0x1e, 0xcb, 0x31, 0xdc, 0x78, 0x2c, 0xc7, 0x10, 0x65, 0x9e,
|
||||
0x9e, 0x59, 0x92, 0x51, 0x9a, 0xa4, 0x97, 0x9c, 0x9f, 0xab, 0x8f, 0x1c, 0x6c, 0x08, 0x26, 0x24,
|
||||
0xf0, 0xd1, 0x83, 0x34, 0x89, 0x0d, 0x2c, 0x6e, 0x0c, 0x08, 0x00, 0x00, 0xff, 0xff, 0x7a, 0x4b,
|
||||
0x9b, 0x9a, 0xd1, 0x01, 0x00, 0x00,
|
||||
0x54, 0x85, 0x4f, 0x66, 0x71, 0x09, 0x54, 0x37, 0x5c, 0x97, 0x90, 0x25, 0x17, 0x77, 0x4e, 0x62,
|
||||
0x71, 0x49, 0x7c, 0x72, 0x7e, 0x6e, 0x6e, 0x66, 0x89, 0x04, 0x0b, 0x2e, 0x07, 0x3b, 0x83, 0xe5,
|
||||
0x83, 0xb8, 0x40, 0x8a, 0x21, 0x6c, 0xa7, 0xc0, 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, 0x32, 0x4f, 0xcf, 0x2c, 0xc9, 0x28, 0x4d, 0xd2, 0x4b, 0xce, 0xcf, 0xd5, 0x47, 0x0e,
|
||||
0x36, 0x04, 0x13, 0x12, 0xf8, 0xe8, 0x41, 0x9a, 0xc4, 0x06, 0x16, 0x37, 0x06, 0x04, 0x00, 0x00,
|
||||
0xff, 0xff, 0x79, 0x8c, 0xb5, 0x43, 0xd1, 0x01, 0x00, 0x00,
|
||||
}
|
||||
|
||||
func (m *Block) Marshal() (dAtA []byte, err error) {
|
||||
|
||||
@@ -10,6 +10,6 @@ import "tendermint/types/evidence.proto";
|
||||
message Block {
|
||||
Header header = 1 [(gogoproto.nullable) = false];
|
||||
Data data = 2 [(gogoproto.nullable) = false];
|
||||
tendermint.types.EvidenceData evidence = 3 [(gogoproto.nullable) = false];
|
||||
tendermint.types.EvidenceList evidence = 3 [(gogoproto.nullable) = false];
|
||||
Commit last_commit = 4;
|
||||
}
|
||||
|
||||
File diff suppressed because it is too large
Load Diff
@@ -4,19 +4,9 @@ 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";
|
||||
|
||||
// DuplicateVoteEvidence contains evidence a validator signed two conflicting
|
||||
// votes.
|
||||
message DuplicateVoteEvidence {
|
||||
Vote vote_a = 1;
|
||||
Vote vote_b = 2;
|
||||
}
|
||||
|
||||
message LightClientAttackEvidence {
|
||||
LightBlock conflicting_block = 1;
|
||||
int64 common_height = 2;
|
||||
}
|
||||
import "tendermint/types/validator.proto";
|
||||
|
||||
message Evidence {
|
||||
oneof sum {
|
||||
@@ -25,7 +15,24 @@ message Evidence {
|
||||
}
|
||||
}
|
||||
|
||||
// EvidenceData contains any evidence of malicious wrong-doing by validators
|
||||
message EvidenceData {
|
||||
// DuplicateVoteEvidence contains evidence of a validator signed two conflicting votes.
|
||||
message DuplicateVoteEvidence {
|
||||
tendermint.types.Vote vote_a = 1;
|
||||
tendermint.types.Vote vote_b = 2;
|
||||
int64 total_voting_power = 3;
|
||||
int64 validator_power = 4;
|
||||
google.protobuf.Timestamp timestamp = 5 [(gogoproto.nullable) = false, (gogoproto.stdtime) = true];
|
||||
}
|
||||
|
||||
// LightClientAttackEvidence contains evidence of a set of validators attempting to mislead a light client.
|
||||
message LightClientAttackEvidence {
|
||||
tendermint.types.LightBlock conflicting_block = 1;
|
||||
int64 common_height = 2;
|
||||
repeated tendermint.types.Validator byzantine_validators = 3;
|
||||
int64 total_voting_power = 4;
|
||||
google.protobuf.Timestamp timestamp = 5 [(gogoproto.nullable) = false, (gogoproto.stdtime) = true];
|
||||
}
|
||||
|
||||
message EvidenceList {
|
||||
repeated Evidence evidence = 1 [(gogoproto.nullable) = false];
|
||||
}
|
||||
|
||||
@@ -42,7 +42,10 @@ 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)
|
||||
validator := types.NewValidator(val.Key.PubKey, 10)
|
||||
valSet := types.NewValidatorSet([]*types.Validator{validator})
|
||||
|
||||
return types.NewDuplicateVoteEvidence(vote, vote2, defaultTestTime, valSet)
|
||||
}
|
||||
|
||||
func makeEvidences(
|
||||
|
||||
@@ -115,7 +115,11 @@ func (blockExec *BlockExecutor) CreateProposalBlock(
|
||||
// Validation does not mutate state, but does require historical information from the stateDB,
|
||||
// ie. to verify evidence from a validator at an old height.
|
||||
func (blockExec *BlockExecutor) ValidateBlock(state State, block *types.Block) error {
|
||||
return validateBlock(blockExec.evpool, state, block)
|
||||
err := validateBlock(state, block)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return blockExec.evpool.CheckEvidence(block.Evidence.Evidence)
|
||||
}
|
||||
|
||||
// ApplyBlock validates the block against the state, executes it against the app,
|
||||
@@ -128,16 +132,13 @@ func (blockExec *BlockExecutor) ApplyBlock(
|
||||
state State, blockID types.BlockID, block *types.Block,
|
||||
) (State, int64, error) {
|
||||
|
||||
if err := blockExec.ValidateBlock(state, block); err != nil {
|
||||
if err := validateBlock(state, block); err != nil {
|
||||
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, byzVals)
|
||||
blockExec.store, state.InitialHeight)
|
||||
endTime := time.Now().UnixNano()
|
||||
blockExec.metrics.BlockProcessingTime.Observe(float64(endTime-startTime) / 1000000)
|
||||
if err != nil {
|
||||
@@ -180,7 +181,7 @@ func (blockExec *BlockExecutor) ApplyBlock(
|
||||
}
|
||||
|
||||
// Update evpool with the latest state.
|
||||
blockExec.evpool.Update(state)
|
||||
blockExec.evpool.Update(state, block.Evidence.Evidence)
|
||||
|
||||
fail.Fail() // XXX
|
||||
|
||||
@@ -262,7 +263,6 @@ func execBlockOnProxyApp(
|
||||
block *types.Block,
|
||||
store Store,
|
||||
initialHeight int64,
|
||||
byzVals []abci.Evidence,
|
||||
) (*tmstate.ABCIResponses, error) {
|
||||
var validTxs, invalidTxs = 0, 0
|
||||
|
||||
@@ -292,6 +292,11 @@ func execBlockOnProxyApp(
|
||||
|
||||
commitInfo := getBeginBlockValidatorInfo(block, store, initialHeight)
|
||||
|
||||
byzVals := make([]abci.Evidence, 0)
|
||||
for _, evidence := range block.Evidence.Evidence {
|
||||
byzVals = append(byzVals, evidence.ABCI()...)
|
||||
}
|
||||
|
||||
// Begin block
|
||||
var err error
|
||||
pbh := block.Header.ToProto()
|
||||
@@ -526,7 +531,7 @@ func ExecCommitBlock(
|
||||
store Store,
|
||||
initialHeight int64,
|
||||
) ([]byte, error) {
|
||||
_, err := execBlockOnProxyApp(logger, appConnConsensus, block, store, initialHeight, []abci.Evidence{})
|
||||
_, err := execBlockOnProxyApp(logger, appConnConsensus, block, store, initialHeight)
|
||||
if err != nil {
|
||||
logger.Error("Error executing block on proxy app", "height", block.Height, "err", err)
|
||||
return nil, err
|
||||
|
||||
@@ -10,16 +10,20 @@ import (
|
||||
"github.com/stretchr/testify/require"
|
||||
|
||||
abci "github.com/tendermint/tendermint/abci/types"
|
||||
"github.com/tendermint/tendermint/crypto"
|
||||
"github.com/tendermint/tendermint/crypto/ed25519"
|
||||
cryptoenc "github.com/tendermint/tendermint/crypto/encoding"
|
||||
"github.com/tendermint/tendermint/crypto/tmhash"
|
||||
"github.com/tendermint/tendermint/libs/log"
|
||||
mmock "github.com/tendermint/tendermint/mempool/mock"
|
||||
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
|
||||
tmversion "github.com/tendermint/tendermint/proto/tendermint/version"
|
||||
"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"
|
||||
"github.com/tendermint/tendermint/version"
|
||||
)
|
||||
|
||||
var (
|
||||
@@ -125,10 +129,52 @@ func TestBeginBlockByzantineValidators(t *testing.T) {
|
||||
require.Nil(t, err)
|
||||
defer proxyApp.Stop() //nolint:errcheck // ignore for tests
|
||||
|
||||
state, stateDB, _ := makeState(1, 1)
|
||||
state, stateDB, privVals := makeState(1, 1)
|
||||
stateStore := sm.NewStore(stateDB)
|
||||
|
||||
defaultEvidenceTime := time.Date(2019, 1, 1, 0, 0, 0, 0, time.UTC)
|
||||
privVal := privVals[state.Validators.Validators[0].Address.String()]
|
||||
blockID := makeBlockID([]byte("headerhash"), 1000, []byte("partshash"))
|
||||
header := &types.Header{
|
||||
Version: tmversion.Consensus{Block: version.BlockProtocol, App: 1},
|
||||
ChainID: state.ChainID,
|
||||
Height: 10,
|
||||
Time: defaultEvidenceTime,
|
||||
LastBlockID: blockID,
|
||||
LastCommitHash: crypto.CRandBytes(tmhash.Size),
|
||||
DataHash: crypto.CRandBytes(tmhash.Size),
|
||||
ValidatorsHash: state.Validators.Hash(),
|
||||
NextValidatorsHash: state.Validators.Hash(),
|
||||
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),
|
||||
}
|
||||
|
||||
// we don't need to worry about validating the evidence as long as they pass validate basic
|
||||
dve := types.NewMockDuplicateVoteEvidenceWithValidator(3, defaultEvidenceTime, privVal, state.ChainID)
|
||||
dve.ValidatorPower = 1000
|
||||
lcae := &types.LightClientAttackEvidence{
|
||||
ConflictingBlock: &types.LightBlock{
|
||||
SignedHeader: &types.SignedHeader{
|
||||
Header: header,
|
||||
Commit: types.NewCommit(10, 0, makeBlockID(header.Hash(), 100, []byte("partshash")), []types.CommitSig{{
|
||||
BlockIDFlag: types.BlockIDFlagNil,
|
||||
ValidatorAddress: crypto.AddressHash([]byte("validator_address")),
|
||||
Timestamp: defaultEvidenceTime,
|
||||
Signature: crypto.CRandBytes(types.MaxSignatureSize),
|
||||
}}),
|
||||
},
|
||||
ValidatorSet: state.Validators,
|
||||
},
|
||||
CommonHeight: 8,
|
||||
ByzantineValidators: []*types.Validator{state.Validators.Validators[0]},
|
||||
TotalVotingPower: 12,
|
||||
Timestamp: defaultEvidenceTime,
|
||||
}
|
||||
|
||||
ev := []types.Evidence{dve, lcae}
|
||||
|
||||
abciEv := []abci.Evidence{
|
||||
{
|
||||
@@ -136,7 +182,7 @@ func TestBeginBlockByzantineValidators(t *testing.T) {
|
||||
Height: 3,
|
||||
Time: defaultEvidenceTime,
|
||||
Validator: types.TM2PB.Validator(state.Validators.Validators[0]),
|
||||
TotalVotingPower: 33,
|
||||
TotalVotingPower: 10,
|
||||
},
|
||||
{
|
||||
Type: abci.EvidenceType_LIGHT_CLIENT_ATTACK,
|
||||
@@ -148,15 +194,17 @@ func TestBeginBlockByzantineValidators(t *testing.T) {
|
||||
}
|
||||
|
||||
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("PendingEvidence", mock.AnythingOfType("int64")).Return(ev, int64(100))
|
||||
evpool.On("Update", mock.AnythingOfType("state.State"), mock.AnythingOfType("types.EvidenceList")).Return()
|
||||
evpool.On("CheckEvidence", mock.AnythingOfType("types.EvidenceList")).Return(nil)
|
||||
|
||||
blockExec := sm.NewBlockExecutor(stateStore, log.TestingLogger(), proxyApp.Consensus(),
|
||||
mmock.Mempool{}, evpool)
|
||||
|
||||
block := makeBlock(state, 1)
|
||||
blockID := types.BlockID{Hash: block.Hash(), PartSetHeader: block.MakePartSet(testPartSize).Header()}
|
||||
block.Evidence = types.EvidenceData{Evidence: ev}
|
||||
block.Header.EvidenceHash = block.Evidence.Hash()
|
||||
blockID = types.BlockID{Hash: block.Hash(), PartSetHeader: block.MakePartSet(testPartSize).Header()}
|
||||
|
||||
state, retainHeight, err := blockExec.ApplyBlock(state, blockID, block)
|
||||
require.Nil(t, err)
|
||||
@@ -400,3 +448,19 @@ func TestEndBlockValidatorUpdatesResultingInEmptySet(t *testing.T) {
|
||||
assert.NotNil(t, err)
|
||||
assert.NotEmpty(t, state.NextValidators.Validators)
|
||||
}
|
||||
|
||||
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,
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
@@ -4,8 +4,6 @@ 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"
|
||||
@@ -16,22 +14,6 @@ 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)
|
||||
@@ -60,13 +42,13 @@ func (_m *EvidencePool) CheckEvidence(_a0 types.EvidenceList) error {
|
||||
return r0
|
||||
}
|
||||
|
||||
// PendingEvidence provides a mock function with given fields: _a0
|
||||
func (_m *EvidencePool) PendingEvidence(_a0 int64) ([]types.Evidence, int64) {
|
||||
ret := _m.Called(_a0)
|
||||
// PendingEvidence provides a mock function with given fields: maxBytes
|
||||
func (_m *EvidencePool) PendingEvidence(maxBytes int64) ([]types.Evidence, int64) {
|
||||
ret := _m.Called(maxBytes)
|
||||
|
||||
var r0 []types.Evidence
|
||||
if rf, ok := ret.Get(0).(func(int64) []types.Evidence); ok {
|
||||
r0 = rf(_a0)
|
||||
r0 = rf(maxBytes)
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).([]types.Evidence)
|
||||
@@ -75,7 +57,7 @@ func (_m *EvidencePool) PendingEvidence(_a0 int64) ([]types.Evidence, int64) {
|
||||
|
||||
var r1 int64
|
||||
if rf, ok := ret.Get(1).(func(int64) int64); ok {
|
||||
r1 = rf(_a0)
|
||||
r1 = rf(maxBytes)
|
||||
} else {
|
||||
r1 = ret.Get(1).(int64)
|
||||
}
|
||||
@@ -83,7 +65,7 @@ func (_m *EvidencePool) PendingEvidence(_a0 int64) ([]types.Evidence, int64) {
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// Update provides a mock function with given fields: _a0
|
||||
func (_m *EvidencePool) Update(_a0 state.State) {
|
||||
_m.Called(_a0)
|
||||
// Update provides a mock function with given fields: _a0, _a1
|
||||
func (_m *EvidencePool) Update(_a0 state.State, _a1 types.EvidenceList) {
|
||||
_m.Called(_a0, _a1)
|
||||
}
|
||||
|
||||
@@ -1,9 +1,6 @@
|
||||
package state
|
||||
|
||||
import (
|
||||
"time"
|
||||
|
||||
abci "github.com/tendermint/tendermint/abci/types"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
)
|
||||
|
||||
@@ -45,9 +42,8 @@ type BlockStore interface {
|
||||
type EvidencePool interface {
|
||||
PendingEvidence(maxBytes int64) (ev []types.Evidence, size int64)
|
||||
AddEvidence(types.Evidence) error
|
||||
Update(State)
|
||||
Update(State, types.EvidenceList)
|
||||
CheckEvidence(types.EvidenceList) error
|
||||
ABCIEvidence(int64, []types.Evidence) []abci.Evidence
|
||||
}
|
||||
|
||||
// EmptyEvidencePool is an empty implementation of EvidencePool, useful for testing. It also complies
|
||||
@@ -58,11 +54,8 @@ func (EmptyEvidencePool) PendingEvidence(maxBytes int64) (ev []types.Evidence, s
|
||||
return nil, 0
|
||||
}
|
||||
func (EmptyEvidencePool) AddEvidence(types.Evidence) error { return nil }
|
||||
func (EmptyEvidencePool) Update(State) {}
|
||||
func (EmptyEvidencePool) Update(State, types.EvidenceList) {}
|
||||
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 {
|
||||
func (EmptyEvidencePool) AddEvidenceFromConsensus(evidence types.Evidence) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
@@ -12,7 +12,7 @@ import (
|
||||
//-----------------------------------------------------
|
||||
// Validate block
|
||||
|
||||
func validateBlock(evidencePool EvidencePool, state State, block *types.Block) error {
|
||||
func validateBlock(state State, block *types.Block) error {
|
||||
// Validate internal consistency.
|
||||
if err := block.ValidateBasic(); err != nil {
|
||||
return err
|
||||
@@ -147,6 +147,5 @@ func validateBlock(evidencePool EvidencePool, state State, block *types.Block) e
|
||||
return types.NewErrEvidenceOverflow(max, got)
|
||||
}
|
||||
|
||||
// Validate all evidence.
|
||||
return evidencePool.CheckEvidence(block.Evidence.Evidence)
|
||||
return nil
|
||||
}
|
||||
|
||||
@@ -218,7 +218,7 @@ func TestValidateBlockEvidence(t *testing.T) {
|
||||
|
||||
evpool := &mocks.EvidencePool{}
|
||||
evpool.On("CheckEvidence", mock.AnythingOfType("types.EvidenceList")).Return(nil)
|
||||
evpool.On("Update", mock.AnythingOfType("state.State")).Return()
|
||||
evpool.On("Update", mock.AnythingOfType("state.State"), mock.AnythingOfType("types.EvidenceList")).Return()
|
||||
evpool.On("ABCIEvidence", mock.AnythingOfType("int64"), mock.AnythingOfType("[]types.Evidence")).Return(
|
||||
[]abci.Evidence{})
|
||||
|
||||
|
||||
@@ -41,11 +41,8 @@ var (
|
||||
"restart": 0.1,
|
||||
}
|
||||
nodeMisbehaviors = weightedChoice{
|
||||
// FIXME Disabled due to:
|
||||
// https://github.com/tendermint/tendermint/issues/5554
|
||||
// https://github.com/tendermint/tendermint/issues/5560
|
||||
// misbehaviorOption{"double-prevote"}: 1,
|
||||
misbehaviorOption{}: 9,
|
||||
misbehaviorOption{"double-prevote"}: 1,
|
||||
misbehaviorOption{}: 9,
|
||||
}
|
||||
)
|
||||
|
||||
|
||||
@@ -36,10 +36,7 @@ seeds = ["seed01"]
|
||||
seeds = ["seed01"]
|
||||
snapshot_interval = 5
|
||||
perturb = ["disconnect"]
|
||||
# FIXME Evidence handling causes panics and halts
|
||||
# https://github.com/tendermint/tendermint/issues/5554
|
||||
# https://github.com/tendermint/tendermint/issues/5560
|
||||
#misbehaviors = { 1012 = "double-prevote", 1018 = "double-prevote" }
|
||||
misbehaviors = { 1018 = "double-prevote" }
|
||||
|
||||
[node.validator02]
|
||||
seeds = ["seed02"]
|
||||
@@ -62,7 +59,6 @@ perturb = ["kill"]
|
||||
persistent_peers = ["validator01"]
|
||||
database = "rocksdb"
|
||||
abci_protocol = "builtin"
|
||||
retain_blocks = 1
|
||||
perturb = ["pause"]
|
||||
|
||||
[node.validator05]
|
||||
@@ -81,6 +77,7 @@ mode = "full"
|
||||
# https://github.com/tendermint/tendermint/issues/5444
|
||||
fast_sync = "v2"
|
||||
persistent_peers = ["validator01", "validator02", "validator03", "validator04", "validator05"]
|
||||
retain_blocks = 1
|
||||
perturb = ["restart"]
|
||||
|
||||
[node.full02]
|
||||
|
||||
@@ -10,11 +10,12 @@ import (
|
||||
"github.com/tendermint/tendermint/types"
|
||||
)
|
||||
|
||||
// assert that all nodes that have blocks during the height (or height + 1) of a misbehavior has evidence
|
||||
// assert that all nodes that have blocks at the height of a misbehavior has evidence
|
||||
// for that misbehavior
|
||||
func TestEvidence_Misbehavior(t *testing.T) {
|
||||
blocks := fetchBlockChain(t)
|
||||
testNode(t, func(t *testing.T, node e2e.Node) {
|
||||
seenEvidence := make(map[int64]struct{})
|
||||
for _, block := range blocks {
|
||||
// Find any evidence blaming this node in this block
|
||||
var nodeEvidence types.Evidence
|
||||
@@ -28,16 +29,14 @@ func TestEvidence_Misbehavior(t *testing.T) {
|
||||
t.Fatalf("unexpected evidence type %T", evidence)
|
||||
}
|
||||
}
|
||||
|
||||
// Check that evidence was as expected (evidence is submitted in following height)
|
||||
misbehavior, ok := node.Misbehaviors[block.Height-1]
|
||||
if !ok {
|
||||
require.Nil(t, nodeEvidence, "found unexpected evidence %v in height %v",
|
||||
nodeEvidence, block.Height)
|
||||
continue
|
||||
if nodeEvidence == nil {
|
||||
continue // no evidence for the node at this height
|
||||
}
|
||||
require.NotNil(t, nodeEvidence, "no evidence found for misbehavior %v in height %v",
|
||||
misbehavior, block.Height)
|
||||
|
||||
// Check that evidence was as expected
|
||||
misbehavior, ok := node.Misbehaviors[nodeEvidence.Height()]
|
||||
require.True(t, ok, "found unexpected evidence %v in height %v",
|
||||
nodeEvidence, block.Height)
|
||||
|
||||
switch misbehavior {
|
||||
case "double-prevote":
|
||||
@@ -45,6 +44,14 @@ func TestEvidence_Misbehavior(t *testing.T) {
|
||||
default:
|
||||
t.Fatalf("unknown misbehavior %v", misbehavior)
|
||||
}
|
||||
|
||||
seenEvidence[nodeEvidence.Height()] = struct{}{}
|
||||
}
|
||||
// see if there is any evidence that we were expecting but didn't see
|
||||
for height, misbehavior := range node.Misbehaviors {
|
||||
_, ok := seenEvidence[height]
|
||||
require.True(t, ok, "expected evidence for %v misbehavior at height %v by node but was never found",
|
||||
misbehavior, height)
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
@@ -467,7 +467,7 @@ type txNotifier interface {
|
||||
type evidencePool interface {
|
||||
// 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
|
||||
AddEvidenceFromConsensus(evidence types.Evidence) error
|
||||
}
|
||||
|
||||
//----------------------------------------
|
||||
@@ -1773,8 +1773,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.AddEvidenceFromConsensus(
|
||||
types.NewDuplicateVoteEvidence(voteErr.VoteA, voteErr.VoteB), timestamp, cs.Validators)
|
||||
ev := types.NewDuplicateVoteEvidence(voteErr.VoteA, voteErr.VoteB, timestamp, cs.Validators)
|
||||
evidenceErr := cs.evpool.AddEvidenceFromConsensus(ev)
|
||||
if evidenceErr != nil {
|
||||
cs.Logger.Error("Failed to add evidence to the evidence pool", "err", evidenceErr)
|
||||
}
|
||||
|
||||
@@ -1118,12 +1118,12 @@ func (data *EvidenceData) StringIndented(indent string) string {
|
||||
}
|
||||
|
||||
// ToProto converts EvidenceData to protobuf
|
||||
func (data *EvidenceData) ToProto() (*tmproto.EvidenceData, error) {
|
||||
func (data *EvidenceData) ToProto() (*tmproto.EvidenceList, error) {
|
||||
if data == nil {
|
||||
return nil, errors.New("nil evidence data")
|
||||
}
|
||||
|
||||
evi := new(tmproto.EvidenceData)
|
||||
evi := new(tmproto.EvidenceList)
|
||||
eviBzs := make([]tmproto.Evidence, len(data.Evidence))
|
||||
for i := range data.Evidence {
|
||||
protoEvi, err := EvidenceToProto(data.Evidence[i])
|
||||
@@ -1138,7 +1138,7 @@ func (data *EvidenceData) ToProto() (*tmproto.EvidenceData, error) {
|
||||
}
|
||||
|
||||
// FromProto sets a protobuf EvidenceData to the given pointer.
|
||||
func (data *EvidenceData) FromProto(eviData *tmproto.EvidenceData) error {
|
||||
func (data *EvidenceData) FromProto(eviData *tmproto.EvidenceList) error {
|
||||
if eviData == nil {
|
||||
return errors.New("nil evidenceData")
|
||||
}
|
||||
|
||||
@@ -698,13 +698,8 @@ func TestDataProtoBuf(t *testing.T) {
|
||||
|
||||
// TestEvidenceDataProtoBuf ensures parity in converting to and from proto.
|
||||
func TestEvidenceDataProtoBuf(t *testing.T) {
|
||||
val := NewMockPV()
|
||||
blockID := makeBlockID(tmhash.Sum([]byte("blockhash")), math.MaxInt32, tmhash.Sum([]byte("partshash")))
|
||||
blockID2 := makeBlockID(tmhash.Sum([]byte("blockhash2")), math.MaxInt32, tmhash.Sum([]byte("partshash")))
|
||||
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)
|
||||
ev := NewMockDuplicateVoteEvidence(math.MaxInt64, time.Now(), chainID)
|
||||
data := &EvidenceData{Evidence: EvidenceList{ev}}
|
||||
_ = data.ByteSize()
|
||||
testCases := []struct {
|
||||
|
||||
@@ -5,9 +5,11 @@ import (
|
||||
"encoding/binary"
|
||||
"errors"
|
||||
"fmt"
|
||||
"sort"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
abci "github.com/tendermint/tendermint/abci/types"
|
||||
"github.com/tendermint/tendermint/crypto/merkle"
|
||||
"github.com/tendermint/tendermint/crypto/tmhash"
|
||||
tmjson "github.com/tendermint/tendermint/libs/json"
|
||||
@@ -18,31 +20,42 @@ 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 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
|
||||
ABCI() []abci.Evidence // forms individual evidence to be sent to the application
|
||||
Bytes() []byte // bytes which comprise the evidence
|
||||
Hash() []byte // hash of the evidence
|
||||
Height() int64 // height of the infraction
|
||||
String() string // string format of the evidence
|
||||
Time() time.Time // time of the infraction
|
||||
ValidateBasic() error // basic consistency check
|
||||
}
|
||||
|
||||
//--------------------------------------------------------------------------------------
|
||||
|
||||
// DuplicateVoteEvidence contains evidence a validator signed two conflicting
|
||||
// votes.
|
||||
// DuplicateVoteEvidence contains evidence of a single validator signing two conflicting votes.
|
||||
type DuplicateVoteEvidence struct {
|
||||
VoteA *Vote `json:"vote_a"`
|
||||
VoteB *Vote `json:"vote_b"`
|
||||
|
||||
// abci specific information
|
||||
TotalVotingPower int64
|
||||
ValidatorPower int64
|
||||
Timestamp time.Time
|
||||
}
|
||||
|
||||
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) *DuplicateVoteEvidence {
|
||||
func NewDuplicateVoteEvidence(vote1, vote2 *Vote, blockTime time.Time, valSet *ValidatorSet) *DuplicateVoteEvidence {
|
||||
var voteA, voteB *Vote
|
||||
if vote1 == nil || vote2 == nil {
|
||||
if vote1 == nil || vote2 == nil || valSet == nil {
|
||||
return nil
|
||||
}
|
||||
idx, val := valSet.GetByAddress(vote1.ValidatorAddress)
|
||||
if idx == -1 {
|
||||
return nil
|
||||
}
|
||||
|
||||
if strings.Compare(vote1.BlockID.Key(), vote2.BlockID.Key()) == -1 {
|
||||
voteA = vote1
|
||||
voteB = vote2
|
||||
@@ -51,19 +64,26 @@ func NewDuplicateVoteEvidence(vote1, vote2 *Vote) *DuplicateVoteEvidence {
|
||||
voteB = vote1
|
||||
}
|
||||
return &DuplicateVoteEvidence{
|
||||
VoteA: voteA,
|
||||
VoteB: voteB,
|
||||
VoteA: voteA,
|
||||
VoteB: voteB,
|
||||
TotalVotingPower: valSet.TotalVotingPower(),
|
||||
ValidatorPower: val.VotingPower,
|
||||
Timestamp: blockTime,
|
||||
}
|
||||
}
|
||||
|
||||
// String returns a string representation of the evidence.
|
||||
func (dve *DuplicateVoteEvidence) String() string {
|
||||
return fmt.Sprintf("DuplicateVoteEvidence{VoteA: %v, VoteB: %v}", dve.VoteA, dve.VoteB)
|
||||
}
|
||||
|
||||
// Height returns the height this evidence refers to.
|
||||
func (dve *DuplicateVoteEvidence) Height() int64 {
|
||||
return dve.VoteA.Height
|
||||
// ABCI returns the application relevant representation of the evidence
|
||||
func (dve *DuplicateVoteEvidence) ABCI() []abci.Evidence {
|
||||
return []abci.Evidence{{
|
||||
Type: abci.EvidenceType_DUPLICATE_VOTE,
|
||||
Validator: abci.Validator{
|
||||
Address: dve.VoteA.ValidatorAddress,
|
||||
Power: dve.ValidatorPower,
|
||||
},
|
||||
Height: dve.VoteA.Height,
|
||||
Time: dve.Timestamp,
|
||||
TotalVotingPower: dve.TotalVotingPower,
|
||||
}}
|
||||
}
|
||||
|
||||
// Bytes returns the proto-encoded evidence as a byte array.
|
||||
@@ -82,6 +102,21 @@ func (dve *DuplicateVoteEvidence) Hash() []byte {
|
||||
return tmhash.Sum(dve.Bytes())
|
||||
}
|
||||
|
||||
// Height returns the height of the infraction
|
||||
func (dve *DuplicateVoteEvidence) Height() int64 {
|
||||
return dve.VoteA.Height
|
||||
}
|
||||
|
||||
// String returns a string representation of the evidence.
|
||||
func (dve *DuplicateVoteEvidence) String() string {
|
||||
return fmt.Sprintf("DuplicateVoteEvidence{VoteA: %v, VoteB: %v}", dve.VoteA, dve.VoteB)
|
||||
}
|
||||
|
||||
// Time returns the time of the infraction
|
||||
func (dve *DuplicateVoteEvidence) Time() time.Time {
|
||||
return dve.Timestamp
|
||||
}
|
||||
|
||||
// ValidateBasic performs basic validation.
|
||||
func (dve *DuplicateVoteEvidence) ValidateBasic() error {
|
||||
if dve == nil {
|
||||
@@ -109,8 +144,11 @@ func (dve *DuplicateVoteEvidence) ToProto() *tmproto.DuplicateVoteEvidence {
|
||||
voteB := dve.VoteB.ToProto()
|
||||
voteA := dve.VoteA.ToProto()
|
||||
tp := tmproto.DuplicateVoteEvidence{
|
||||
VoteA: voteA,
|
||||
VoteB: voteB,
|
||||
VoteA: voteA,
|
||||
VoteB: voteB,
|
||||
TotalVotingPower: dve.TotalVotingPower,
|
||||
ValidatorPower: dve.ValidatorPower,
|
||||
Timestamp: dve.Timestamp,
|
||||
}
|
||||
return &tp
|
||||
}
|
||||
@@ -131,7 +169,13 @@ func DuplicateVoteEvidenceFromProto(pb *tmproto.DuplicateVoteEvidence) (*Duplica
|
||||
return nil, err
|
||||
}
|
||||
|
||||
dve := NewDuplicateVoteEvidence(vA, vB)
|
||||
dve := &DuplicateVoteEvidence{
|
||||
VoteA: vA,
|
||||
VoteB: vB,
|
||||
TotalVotingPower: pb.TotalVotingPower,
|
||||
ValidatorPower: pb.ValidatorPower,
|
||||
Timestamp: pb.Timestamp,
|
||||
}
|
||||
|
||||
return dve, dve.ValidateBasic()
|
||||
}
|
||||
@@ -146,15 +190,28 @@ func DuplicateVoteEvidenceFromProto(pb *tmproto.DuplicateVoteEvidence) (*Duplica
|
||||
type LightClientAttackEvidence struct {
|
||||
ConflictingBlock *LightBlock
|
||||
CommonHeight int64
|
||||
|
||||
// abci specific information
|
||||
ByzantineValidators []*Validator // validators in the validator set that misbehaved in creating the conflicting block
|
||||
TotalVotingPower int64 // total voting power of the validator set at the common height
|
||||
Timestamp time.Time // timestamp of the block at the common height
|
||||
}
|
||||
|
||||
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
|
||||
// ABCI forms an array of abci evidence for each byzantine validator
|
||||
func (l *LightClientAttackEvidence) ABCI() []abci.Evidence {
|
||||
abciEv := make([]abci.Evidence, len(l.ByzantineValidators))
|
||||
for idx, val := range l.ByzantineValidators {
|
||||
abciEv[idx] = abci.Evidence{
|
||||
Type: abci.EvidenceType_LIGHT_CLIENT_ATTACK,
|
||||
Validator: TM2PB.Validator(val),
|
||||
Height: l.Height(),
|
||||
Time: l.Timestamp,
|
||||
TotalVotingPower: l.TotalVotingPower,
|
||||
}
|
||||
}
|
||||
return abciEv
|
||||
}
|
||||
|
||||
// Bytes returns the proto-encoded evidence as a byte array
|
||||
@@ -170,10 +227,75 @@ func (l *LightClientAttackEvidence) Bytes() []byte {
|
||||
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.
|
||||
// GetByzantineValidators finds out what style of attack LightClientAttackEvidence was and then works out who
|
||||
// the malicious validators were and returns them. This is used both for forming the ByzantineValidators
|
||||
// field and for validating that it is correct. Validators are ordered based on validator power
|
||||
func (l *LightClientAttackEvidence) GetByzantineValidators(commonVals *ValidatorSet,
|
||||
trusted *SignedHeader) []*Validator {
|
||||
var validators []*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 l.ConflictingHeaderIsInvalid(trusted.Header) {
|
||||
for _, commitSig := range l.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)
|
||||
}
|
||||
sort.Sort(ValidatorsByVotingPower(validators))
|
||||
return validators
|
||||
} else if trusted.Commit.Round == l.ConflictingBlock.Commit.Round {
|
||||
// This is an equivocation attack as both commits are in the same round. We then find the validators
|
||||
// from the conflicting light block validator set that voted in both headers.
|
||||
// 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(l.ConflictingBlock.Commit.Signatures); i++ {
|
||||
sigA := l.ConflictingBlock.Commit.Signatures[i]
|
||||
if sigA.Absent() {
|
||||
continue
|
||||
}
|
||||
|
||||
sigB := trusted.Commit.Signatures[i]
|
||||
if sigB.Absent() {
|
||||
continue
|
||||
}
|
||||
|
||||
_, val := l.ConflictingBlock.ValidatorSet.GetByAddress(sigA.ValidatorAddress)
|
||||
validators = append(validators, val)
|
||||
}
|
||||
sort.Sort(ValidatorsByVotingPower(validators))
|
||||
return validators
|
||||
}
|
||||
// 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
|
||||
}
|
||||
|
||||
// ConflictingHeaderIsInvalid 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 (l *LightClientAttackEvidence) ConflictingHeaderIsInvalid(trustedHeader *Header) bool {
|
||||
return !bytes.Equal(trustedHeader.ValidatorsHash, l.ConflictingBlock.ValidatorsHash) ||
|
||||
!bytes.Equal(trustedHeader.NextValidatorsHash, l.ConflictingBlock.NextValidatorsHash) ||
|
||||
!bytes.Equal(trustedHeader.ConsensusHash, l.ConflictingBlock.ConsensusHash) ||
|
||||
!bytes.Equal(trustedHeader.AppHash, l.ConflictingBlock.AppHash) ||
|
||||
!bytes.Equal(trustedHeader.LastResultsHash, l.ConflictingBlock.LastResultsHash)
|
||||
|
||||
}
|
||||
|
||||
// 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 (captures the most byzantine validators) but anything greater than 1/3 is sufficient.
|
||||
func (l *LightClientAttackEvidence) Hash() []byte {
|
||||
buf := make([]byte, binary.MaxVarintLen64)
|
||||
n := binary.PutVarint(buf, l.CommonHeight)
|
||||
@@ -183,6 +305,24 @@ func (l *LightClientAttackEvidence) Hash() []byte {
|
||||
return tmhash.Sum(bz)
|
||||
}
|
||||
|
||||
// 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
|
||||
}
|
||||
|
||||
// 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)
|
||||
}
|
||||
|
||||
// Time returns the time of the common block where the infraction leveraged off.
|
||||
func (l *LightClientAttackEvidence) Time() time.Time {
|
||||
return l.Timestamp
|
||||
}
|
||||
|
||||
// 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 {
|
||||
@@ -213,12 +353,6 @@ func (l *LightClientAttackEvidence) ValidateBasic() error {
|
||||
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()
|
||||
@@ -226,29 +360,53 @@ func (l *LightClientAttackEvidence) ToProto() (*tmproto.LightClientAttackEvidenc
|
||||
return nil, err
|
||||
}
|
||||
|
||||
byzVals := make([]*tmproto.Validator, len(l.ByzantineValidators))
|
||||
for idx, val := range l.ByzantineValidators {
|
||||
valpb, err := val.ToProto()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
byzVals[idx] = valpb
|
||||
}
|
||||
|
||||
return &tmproto.LightClientAttackEvidence{
|
||||
ConflictingBlock: conflictingBlock,
|
||||
CommonHeight: l.CommonHeight,
|
||||
ConflictingBlock: conflictingBlock,
|
||||
CommonHeight: l.CommonHeight,
|
||||
ByzantineValidators: byzVals,
|
||||
TotalVotingPower: l.TotalVotingPower,
|
||||
Timestamp: l.Timestamp,
|
||||
}, nil
|
||||
}
|
||||
|
||||
// LightClientAttackEvidenceFromProto decodes protobuf
|
||||
func LightClientAttackEvidenceFromProto(l *tmproto.LightClientAttackEvidence) (*LightClientAttackEvidence, error) {
|
||||
if l == nil {
|
||||
func LightClientAttackEvidenceFromProto(lpb *tmproto.LightClientAttackEvidence) (*LightClientAttackEvidence, error) {
|
||||
if lpb == nil {
|
||||
return nil, errors.New("empty light client attack evidence")
|
||||
}
|
||||
|
||||
conflictingBlock, err := LightBlockFromProto(l.ConflictingBlock)
|
||||
conflictingBlock, err := LightBlockFromProto(lpb.ConflictingBlock)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
le := &LightClientAttackEvidence{
|
||||
ConflictingBlock: conflictingBlock,
|
||||
CommonHeight: l.CommonHeight,
|
||||
byzVals := make([]*Validator, len(lpb.ByzantineValidators))
|
||||
for idx, valpb := range lpb.ByzantineValidators {
|
||||
val, err := ValidatorFromProto(valpb)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
byzVals[idx] = val
|
||||
}
|
||||
|
||||
return le, le.ValidateBasic()
|
||||
l := &LightClientAttackEvidence{
|
||||
ConflictingBlock: conflictingBlock,
|
||||
CommonHeight: lpb.CommonHeight,
|
||||
ByzantineValidators: byzVals,
|
||||
TotalVotingPower: lpb.TotalVotingPower,
|
||||
Timestamp: lpb.Timestamp,
|
||||
}
|
||||
|
||||
return l, l.ValidateBasic()
|
||||
}
|
||||
|
||||
//------------------------------------------------------------------------------------------
|
||||
@@ -386,9 +544,11 @@ func NewMockDuplicateVoteEvidence(height int64, time time.Time, chainID string)
|
||||
return NewMockDuplicateVoteEvidenceWithValidator(height, time, val, chainID)
|
||||
}
|
||||
|
||||
// assumes voting power to be 10 and validator to be the only one in the set
|
||||
func NewMockDuplicateVoteEvidenceWithValidator(height int64, time time.Time,
|
||||
pv PrivValidator, chainID string) *DuplicateVoteEvidence {
|
||||
pubKey, _ := pv.GetPubKey()
|
||||
val := NewValidator(pubKey, 10)
|
||||
voteA := makeMockVote(height, 0, 0, pubKey.Address(), randBlockID(), time)
|
||||
vA := voteA.ToProto()
|
||||
_ = pv.SignVote(chainID, vA)
|
||||
@@ -397,7 +557,7 @@ func NewMockDuplicateVoteEvidenceWithValidator(height int64, time time.Time,
|
||||
vB := voteB.ToProto()
|
||||
_ = pv.SignVote(chainID, vB)
|
||||
voteB.Signature = vB.Signature
|
||||
return NewDuplicateVoteEvidence(voteA, voteB)
|
||||
return NewDuplicateVoteEvidence(voteA, voteB, time, NewValidatorSet([]*Validator{val}))
|
||||
}
|
||||
|
||||
func makeMockVote(height int64, round, index int32, addr Address,
|
||||
|
||||
@@ -33,8 +33,11 @@ func randomDuplicateVoteEvidence(t *testing.T) *DuplicateVoteEvidence {
|
||||
blockID2 := makeBlockID([]byte("blockhash2"), 1000, []byte("partshash"))
|
||||
const chainID = "mychain"
|
||||
return &DuplicateVoteEvidence{
|
||||
VoteA: makeVote(t, val, chainID, 0, 10, 2, 1, blockID, defaultVoteTime),
|
||||
VoteB: makeVote(t, val, chainID, 0, 10, 2, 1, blockID2, defaultVoteTime.Add(1*time.Minute)),
|
||||
VoteA: makeVote(t, val, chainID, 0, 10, 2, 1, blockID, defaultVoteTime),
|
||||
VoteB: makeVote(t, val, chainID, 0, 10, 2, 1, blockID2, defaultVoteTime.Add(1*time.Minute)),
|
||||
TotalVotingPower: 30,
|
||||
ValidatorPower: 10,
|
||||
Timestamp: defaultVoteTime,
|
||||
}
|
||||
}
|
||||
|
||||
@@ -78,7 +81,8 @@ 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)
|
||||
valSet := NewValidatorSet([]*Validator{val.ExtractIntoValidator(10)})
|
||||
ev := NewDuplicateVoteEvidence(vote1, vote2, defaultVoteTime, valSet)
|
||||
tc.malleateEvidence(ev)
|
||||
assert.Equal(t, tc.expectErr, ev.ValidateBasic() != nil, "Validate Basic had an unexpected result")
|
||||
})
|
||||
|
||||
@@ -1,9 +1,6 @@
|
||||
package types
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"reflect"
|
||||
|
||||
abci "github.com/tendermint/tendermint/abci/types"
|
||||
"github.com/tendermint/tendermint/crypto"
|
||||
"github.com/tendermint/tendermint/crypto/ed25519"
|
||||
@@ -111,29 +108,6 @@ func (tm2pb) ConsensusParams(params *tmproto.ConsensusParams) *abci.ConsensusPar
|
||||
}
|
||||
}
|
||||
|
||||
// ABCI Evidence includes information from the past that's not included in the evidence itself
|
||||
// so Evidence types stays compact.
|
||||
// XXX: panics on nil or unknown pubkey type
|
||||
func (tm2pb) Evidence(ev Evidence, valSet *ValidatorSet) abci.Evidence {
|
||||
|
||||
// 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,
|
||||
Height: ev.Height(),
|
||||
TotalVotingPower: valSet.TotalVotingPower(),
|
||||
}
|
||||
}
|
||||
|
||||
// XXX: panics on nil or unknown pubkey type
|
||||
func (tm2pb) NewValidatorUpdate(pubkey crypto.PubKey, power int64) abci.ValidatorUpdate {
|
||||
pubkeyABCI, err := cryptoenc.PubKeyToProto(pubkey)
|
||||
|
||||
@@ -60,26 +60,6 @@ func TestABCIConsensusParams(t *testing.T) {
|
||||
assert.Equal(t, *cp, cp2)
|
||||
}
|
||||
|
||||
func TestABCIEvidence(t *testing.T) {
|
||||
val := NewMockPV()
|
||||
blockID := makeBlockID([]byte("blockhash"), 1000, []byte("partshash"))
|
||||
blockID2 := makeBlockID([]byte("blockhash2"), 1000, []byte("partshash"))
|
||||
const chainID = "mychain"
|
||||
pubKey, err := val.GetPubKey()
|
||||
require.NoError(t, err)
|
||||
ev := &DuplicateVoteEvidence{
|
||||
VoteA: makeVote(t, val, chainID, 0, 10, 2, 1, blockID, defaultVoteTime),
|
||||
VoteB: makeVote(t, val, chainID, 0, 10, 2, 1, blockID2, defaultVoteTime),
|
||||
}
|
||||
abciEv := TM2PB.Evidence(
|
||||
ev,
|
||||
NewValidatorSet([]*Validator{NewValidator(pubKey, 10)}),
|
||||
)
|
||||
|
||||
assert.Equal(t, abci.EvidenceType_DUPLICATE_VOTE, abciEv.Type)
|
||||
assert.Equal(t, ev.Height(), abciEv.GetHeight())
|
||||
}
|
||||
|
||||
type pubKeyEddie struct{}
|
||||
|
||||
func (pubKeyEddie) Address() Address { return []byte{} }
|
||||
|
||||
Reference in New Issue
Block a user