Browse Source

evidence: structs can independently form abci evidence (#5610)

pull/5618/head
Callum Waters 4 years ago
parent
commit
9d354c842e
35 changed files with 1270 additions and 1655 deletions
  1. +1
    -0
      CHANGELOG_PENDING.md
  2. +2
    -2
      consensus/byzantine_test.go
  3. +1
    -3
      consensus/reactor_test.go
  4. +8
    -7
      consensus/state.go
  5. +122
    -218
      evidence/pool.go
  6. +44
    -52
      evidence/pool_test.go
  7. +6
    -8
      evidence/reactor.go
  8. +28
    -14
      evidence/reactor_test.go
  9. +58
    -90
      evidence/verify.go
  10. +62
    -74
      evidence/verify_test.go
  11. +24
    -38
      light/detector.go
  12. +6
    -1
      node/node_test.go
  13. +0
    -668
      proto/tendermint/evidence/types.pb.go
  14. +0
    -20
      proto/tendermint/evidence/types.proto
  15. +11
    -11
      proto/tendermint/types/block.pb.go
  16. +1
    -1
      proto/tendermint/types/block.proto
  17. +532
    -251
      proto/tendermint/types/evidence.pb.go
  18. +21
    -14
      proto/tendermint/types/evidence.proto
  19. +4
    -1
      rpc/client/evidence_test.go
  20. +14
    -9
      state/execution.go
  21. +69
    -5
      state/execution_test.go
  22. +8
    -26
      state/mocks/evidence_pool.go
  23. +3
    -10
      state/services.go
  24. +2
    -3
      state/validation.go
  25. +1
    -1
      state/validation_test.go
  26. +2
    -5
      test/e2e/generator/generate.go
  27. +2
    -5
      test/e2e/networks/ci.toml
  28. +17
    -10
      test/e2e/tests/evidence_test.go
  29. +3
    -3
      test/maverick/consensus/state.go
  30. +3
    -3
      types/block.go
  31. +1
    -6
      types/block_test.go
  32. +207
    -47
      types/evidence.go
  33. +7
    -3
      types/evidence_test.go
  34. +0
    -26
      types/protobuf.go
  35. +0
    -20
      types/protobuf_test.go

+ 1
- 0
CHANGELOG_PENDING.md View File

@ -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)

+ 2
- 2
consensus/byzantine_test.go View File

@ -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()


+ 1
- 3
consensus/reactor_test.go View File

@ -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{}


+ 8
- 7
consensus/state.go View File

@ -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 {


+ 122
- 218
evidence/pool.go View File

@ -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
}
}
// 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")
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
}
}
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 nil, totalSize, err
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
}
totalSize += evInfo.ByteSize
if maxBytes != -1 && totalSize > maxBytes {
return evidence, totalSize - evInfo.ByteSize, nil
ev, err := types.EvidenceFromProto(&evpb)
if err != nil {
return nil, totalSize, err
}
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 {


+ 44
- 52
evidence/pool_test.go View File

@ -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)


+ 6
- 8
evidence/reactor.go View File

@ -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
}


+ 28
- 14
evidence/reactor_test.go View File

@ -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,
}


+ 58
- 90
evidence/verify.go View File

@ -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))
}
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 &info{
Evidence: evidence,
Time: evTime,
Validators: validators,
TotalVotingPower: totalVotingPower,
}, nil
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
)

+ 62
- 74
evidence/verify_test.go View File

@ -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})
abciEv := pool.ABCIEvidence(block.Height, block.Evidence.Evidence)
expectedAbciEv := make([]abci.Evidence, len(commonVals.Validators))
// 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
}
// 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
// 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)
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(


+ 24
- 38
light/detector.go View File

@ -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
}

+ 6
- 1
node/node_test.go View File

@ -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


+ 0
- 668
proto/tendermint/evidence/types.pb.go View File

@ -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")
)

+ 0
- 20
proto/tendermint/evidence/types.proto View File

@ -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;
}

+ 11
- 11
proto/tendermint/types/block.pb.go View File

@ -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) {


+ 1
- 1
proto/tendermint/types/block.proto View File

@ -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;
}

+ 532
- 251
proto/tendermint/types/evidence.pb.go
File diff suppressed because it is too large
View File


+ 21
- 14
proto/tendermint/types/evidence.proto View File

@ -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];
}

+ 4
- 1
rpc/client/evidence_test.go View File

@ -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(


+ 14
- 9
state/execution.go View File

@ -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


+ 69
- 5
state/execution_test.go View File

@ -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,
},
}
}

+ 8
- 26
state/mocks/evidence_pool.go View File

@ -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)
}

+ 3
- 10
state/services.go View File

@ -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
}

+ 2
- 3
state/validation.go View File

@ -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
}

+ 1
- 1
state/validation_test.go View File

@ -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{})


+ 2
- 5
test/e2e/generator/generate.go View File

@ -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,
}
)


+ 2
- 5
test/e2e/networks/ci.toml View File

@ -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]


+ 17
- 10
test/e2e/tests/evidence_test.go View File

@ -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)
}
})
}

+ 3
- 3
test/maverick/consensus/state.go View File

@ -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)
}


+ 3
- 3
types/block.go View File

@ -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")
}


+ 1
- 6
types/block_test.go View File

@ -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 {


+ 207
- 47
types/evidence.go View File

@ -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
}
l := &LightClientAttackEvidence{
ConflictingBlock: conflictingBlock,
CommonHeight: lpb.CommonHeight,
ByzantineValidators: byzVals,
TotalVotingPower: lpb.TotalVotingPower,
Timestamp: lpb.Timestamp,
}
return le, le.ValidateBasic()
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,


+ 7
- 3
types/evidence_test.go View File

@ -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")
})


+ 0
- 26
types/protobuf.go View File

@ -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)


+ 0
- 20
types/protobuf_test.go View File

@ -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{} }


Loading…
Cancel
Save