diff --git a/CHANGELOG_PENDING.md b/CHANGELOG_PENDING.md index 32cff9bda..0d5dba43f 100644 --- a/CHANGELOG_PENDING.md +++ b/CHANGELOG_PENDING.md @@ -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) diff --git a/consensus/byzantine_test.go b/consensus/byzantine_test.go index 4d6660210..fa945c310 100644 --- a/consensus/byzantine_test.go +++ b/consensus/byzantine_test.go @@ -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() diff --git a/consensus/reactor_test.go b/consensus/reactor_test.go index 192622e43..f23ec727d 100644 --- a/consensus/reactor_test.go +++ b/consensus/reactor_test.go @@ -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{} diff --git a/consensus/state.go b/consensus/state.go index 939752734..a046840e2 100644 --- a/consensus/state.go +++ b/consensus/state.go @@ -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 { diff --git a/evidence/pool.go b/evidence/pool.go index 580f8c9e8..77dbf1a39 100644 --- a/evidence/pool.go +++ b/evidence/pool.go @@ -4,7 +4,7 @@ import ( "bytes" "errors" "fmt" - "reflect" + "sort" "sync" "sync/atomic" "time" @@ -13,10 +13,8 @@ import ( gogotypes "github.com/gogo/protobuf/types" dbm "github.com/tendermint/tm-db" - abci "github.com/tendermint/tendermint/abci/types" clist "github.com/tendermint/tendermint/libs/clist" "github.com/tendermint/tendermint/libs/log" - evproto "github.com/tendermint/tendermint/proto/tendermint/evidence" tmproto "github.com/tendermint/tendermint/proto/tendermint/types" sm "github.com/tendermint/tendermint/state" "github.com/tendermint/tendermint/types" @@ -94,7 +92,7 @@ func (evpool *Pool) PendingEvidence(maxBytes int64) ([]types.Evidence, int64) { } // Update pulls the latest state to be used for expiration and evidence params and then prunes all expired evidence -func (evpool *Pool) Update(state sm.State) { +func (evpool *Pool) Update(state sm.State, ev types.EvidenceList) { // sanity check if state.LastBlockHeight <= evpool.state.LastBlockHeight { panic(fmt.Sprintf( @@ -109,6 +107,8 @@ func (evpool *Pool) Update(state sm.State) { // update the state evpool.updateState(state) + evpool.markEvidenceAsCommitted(ev) + // prune pending evidence when it has expired. This also updates when the next evidence will expire if evpool.Size() > 0 && state.LastBlockHeight > evpool.pruningHeight && state.LastBlockTime.After(evpool.pruningTime) { @@ -135,13 +135,13 @@ func (evpool *Pool) AddEvidence(ev types.Evidence) error { } // 1) Verify against state. - evInfo, err := evpool.verify(ev) + err := evpool.verify(ev) if err != nil { return types.NewErrInvalidEvidence(ev, err) } // 2) Save to store. - if err := evpool.addPendingEvidence(evInfo); err != nil { + if err := evpool.addPendingEvidence(ev); err != nil { return fmt.Errorf("can't add evidence to pending list: %w", err) } @@ -153,13 +153,9 @@ func (evpool *Pool) AddEvidence(ev types.Evidence) error { return nil } -// AddEvidenceFromConsensus should be exposed only to the consensus so it can add evidence to the pool -// directly without the need for verification. -func (evpool *Pool) AddEvidenceFromConsensus(ev types.Evidence, time time.Time, valSet *types.ValidatorSet) error { - var ( - vals []*types.Validator - totalPower int64 - ) +// AddEvidenceFromConsensus should be exposed only to the consensus reactor so it can add evidence +// to the pool directly without the need for verification. +func (evpool *Pool) AddEvidenceFromConsensus(ev types.Evidence) error { // we already have this evidence, log this but don't return an error. if evpool.isPending(ev) { @@ -167,23 +163,7 @@ func (evpool *Pool) AddEvidenceFromConsensus(ev types.Evidence, time time.Time, return nil } - switch ev := ev.(type) { - case *types.DuplicateVoteEvidence: - _, val := valSet.GetByAddress(ev.VoteA.ValidatorAddress) - vals = append(vals, val) - totalPower = valSet.TotalVotingPower() - default: - return fmt.Errorf("unrecognized evidence type: %T", ev) - } - - evInfo := &info{ - Evidence: ev, - Time: time, - Validators: vals, - TotalVotingPower: totalPower, - } - - if err := evpool.addPendingEvidence(evInfo); err != nil { + if err := evpool.addPendingEvidence(ev); err != nil { return fmt.Errorf("can't add evidence to pending list: %w", err) } // add evidence to be gossiped with peers @@ -210,15 +190,15 @@ func (evpool *Pool) CheckEvidence(evList types.EvidenceList) error { return &types.ErrInvalidEvidence{Evidence: ev, Reason: errors.New("evidence was already committed")} } - evInfo, err := evpool.verify(ev) + err := evpool.verify(ev) if err != nil { return &types.ErrInvalidEvidence{Evidence: ev, Reason: err} } - if err := evpool.addPendingEvidence(evInfo); err != nil { + if err := evpool.addPendingEvidence(ev); err != nil { // Something went wrong with adding the evidence but we already know it is valid // hence we log an error and continue - evpool.logger.Error("Can't add evidence to pending list", "err", err, "evInfo", evInfo) + evpool.logger.Error("Can't add evidence to pending list", "err", err, "ev", ev) } evpool.logger.Info("Verified new evidence of byzantine behavior", "evidence", ev) @@ -236,85 +216,6 @@ func (evpool *Pool) CheckEvidence(evList types.EvidenceList) error { return nil } -// ABCIEvidence processes all the evidence in the block, marking it as committed and removing it -// from the pending database. It then forms the individual abci evidence that will be passed back to -// the application. -func (evpool *Pool) ABCIEvidence(height int64, evidence []types.Evidence) []abci.Evidence { - // make a map of committed evidence to remove from the clist - blockEvidenceMap := make(map[string]struct{}, len(evidence)) - abciEvidence := make([]abci.Evidence, 0) - for _, ev := range evidence { - - // get entire evidence info from pending list - infoBytes, err := evpool.evidenceStore.Get(keyPending(ev)) - if err != nil { - evpool.logger.Error("Unable to retrieve evidence to pass to ABCI. "+ - "Evidence pool should have seen this evidence before", - "evidence", ev, "err", err) - continue - } - var infoProto evproto.Info - err = infoProto.Unmarshal(infoBytes) - if err != nil { - evpool.logger.Error("Decoding evidence info failed", "err", err, "height", ev.Height(), "hash", ev.Hash()) - continue - } - evInfo, err := infoFromProto(&infoProto) - if err != nil { - evpool.logger.Error("Converting evidence info from proto failed", "err", err, "height", ev.Height(), - "hash", ev.Hash()) - continue - } - - var evType abci.EvidenceType - switch ev.(type) { - case *types.DuplicateVoteEvidence: - evType = abci.EvidenceType_DUPLICATE_VOTE - case *types.LightClientAttackEvidence: - evType = abci.EvidenceType_LIGHT_CLIENT_ATTACK - default: - evpool.logger.Error("Unknown evidence type", "T", reflect.TypeOf(ev)) - continue - } - for _, val := range evInfo.Validators { - abciEv := abci.Evidence{ - Type: evType, - Validator: types.TM2PB.Validator(val), - Height: ev.Height(), - Time: evInfo.Time, - TotalVotingPower: evInfo.TotalVotingPower, - } - abciEvidence = append(abciEvidence, abciEv) - evpool.logger.Info("Created ABCI evidence", "ev", abciEv) - } - - // we can now remove the evidence from the pending list and the clist that we use for gossiping - evpool.removePendingEvidence(ev) - blockEvidenceMap[evMapKey(ev)] = struct{}{} - - // Add evidence to the committed list - // As the evidence is stored in the block store we only need to record the height that it was saved at. - key := keyCommitted(ev) - - h := gogotypes.Int64Value{Value: height} - evBytes, err := proto.Marshal(&h) - if err != nil { - panic(err) - } - - if err := evpool.evidenceStore.Set(key, evBytes); err != nil { - evpool.logger.Error("Unable to add committed evidence", "err", err) - } - } - - // remove committed evidence from the clist - if len(blockEvidenceMap) != 0 { - evpool.removeEvidenceFromList(blockEvidenceMap) - } - - return abciEvidence -} - // EvidenceFront goes to the first evidence in the clist func (evpool *Pool) EvidenceFront() *clist.CElement { return evpool.evidenceList.Front() @@ -330,6 +231,7 @@ func (evpool *Pool) SetLogger(l log.Logger) { evpool.logger = l } +// Size returns the number of evidence in the pool. func (evpool *Pool) Size() uint32 { return atomic.LoadUint32(&evpool.evidenceSize) } @@ -343,106 +245,59 @@ func (evpool *Pool) State() sm.State { //-------------------------------------------------------------------------- -// Info is a wrapper around the evidence that the evidence pool receives with extensive -// information of what validators were malicious, the time of the attack and the total voting power -// This is saved as a form of cache so that the evidence pool can easily produce the ABCI Evidence -// needed to be sent to the application. -type info struct { - Evidence types.Evidence - Time time.Time - Validators []*types.Validator - TotalVotingPower int64 - ByteSize int64 -} - -// ToProto encodes into protobuf -func (ei info) ToProto() (*evproto.Info, error) { - evpb, err := types.EvidenceToProto(ei.Evidence) - if err != nil { - return nil, err - } - - valsProto := make([]*tmproto.Validator, len(ei.Validators)) - for i := 0; i < len(ei.Validators); i++ { - valp, err := ei.Validators[i].ToProto() - if err != nil { - return nil, err - } - valsProto[i] = valp - } - - return &evproto.Info{ - Evidence: *evpb, - Time: ei.Time, - Validators: valsProto, - TotalVotingPower: ei.TotalVotingPower, - }, nil -} - -// InfoFromProto decodes from protobuf into Info -func infoFromProto(proto *evproto.Info) (info, error) { - if proto == nil { - return info{}, errors.New("nil evidence info") - } - - ev, err := types.EvidenceFromProto(&proto.Evidence) - if err != nil { - return info{}, err - } - - vals := make([]*types.Validator, len(proto.Validators)) - for i := 0; i < len(proto.Validators); i++ { - val, err := types.ValidatorFromProto(proto.Validators[i]) - if err != nil { - return info{}, err - } - vals[i] = val - } - - return info{ - Evidence: ev, - Time: proto.Time, - Validators: vals, - TotalVotingPower: proto.TotalVotingPower, - ByteSize: int64(proto.Evidence.Size()), - }, nil - -} - -//-------------------------------------------------------------------------- - // fastCheck leverages the fact that the evidence pool may have already verified the evidence to see if it can // quickly conclude that the evidence is already valid. func (evpool *Pool) fastCheck(ev types.Evidence) bool { - key := keyPending(ev) if lcae, ok := ev.(*types.LightClientAttackEvidence); ok { + key := keyPending(ev) evBytes, err := evpool.evidenceStore.Get(key) if evBytes == nil { // the evidence is not in the nodes pending list return false } if err != nil { - evpool.logger.Error("Failed to load evidence", "err", err, "evidence", lcae) + evpool.logger.Error("Failed to load light client attack evidence", "err", err, "key(height/hash)", key) return false } - evInfo, err := bytesToInfo(evBytes) + var trustedPb tmproto.LightClientAttackEvidence + err = trustedPb.Unmarshal(evBytes) if err != nil { - evpool.logger.Error("Failed to convert evidence from proto", "err", err, "evidence", lcae) + evpool.logger.Error("Failed to convert light client attack evidence from bytes", + "err", err, "key(height/hash)", key) return false } - // ensure that all the validators that the evidence pool have found to be malicious - // are present in the list of commit signatures in the conflicting block - OUTER: - for _, sig := range lcae.ConflictingBlock.Commit.Signatures { - for _, val := range evInfo.Validators { - if bytes.Equal(val.Address, sig.ValidatorAddress) { - continue OUTER - } - } - // 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 { diff --git a/evidence/pool_test.go b/evidence/pool_test.go index 9dc657221..046f4efc5 100644 --- a/evidence/pool_test.go +++ b/evidence/pool_test.go @@ -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) diff --git a/evidence/reactor.go b/evidence/reactor.go index e9003ded1..421e4bc18 100644 --- a/evidence/reactor.go +++ b/evidence/reactor.go @@ -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 } diff --git a/evidence/reactor_test.go b/evidence/reactor_test.go index 03a250ed6..170b45348 100644 --- a/evidence/reactor_test.go +++ b/evidence/reactor_test.go @@ -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, } diff --git a/evidence/verify.go b/evidence/verify.go index 53717d4be..0721ade9a 100644 --- a/evidence/verify.go +++ b/evidence/verify.go @@ -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 -) diff --git a/evidence/verify_test.go b/evidence/verify_test.go index e344cd496..0e72582b2 100644 --- a/evidence/verify_test.go +++ b/evidence/verify_test.go @@ -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( diff --git a/light/detector.go b/light/detector.go index 43f7a3f23..612f186d3 100644 --- a/light/detector.go +++ b/light/detector.go @@ -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 } diff --git a/node/node_test.go b/node/node_test.go index f25e6243d..e94da4da5 100644 --- a/node/node_test.go +++ b/node/node_test.go @@ -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 diff --git a/proto/tendermint/evidence/types.pb.go b/proto/tendermint/evidence/types.pb.go deleted file mode 100644 index 396b2ee4a..000000000 --- a/proto/tendermint/evidence/types.pb.go +++ /dev/null @@ -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") -) diff --git a/proto/tendermint/evidence/types.proto b/proto/tendermint/evidence/types.proto deleted file mode 100644 index 64148beec..000000000 --- a/proto/tendermint/evidence/types.proto +++ /dev/null @@ -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; -} diff --git a/proto/tendermint/types/block.pb.go b/proto/tendermint/types/block.pb.go index 206da91aa..aacb90fab 100644 --- a/proto/tendermint/types/block.pb.go +++ b/proto/tendermint/types/block.pb.go @@ -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) { diff --git a/proto/tendermint/types/block.proto b/proto/tendermint/types/block.proto index 18984c06a..84e9bb15d 100644 --- a/proto/tendermint/types/block.proto +++ b/proto/tendermint/types/block.proto @@ -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; } diff --git a/proto/tendermint/types/evidence.pb.go b/proto/tendermint/types/evidence.pb.go index c1ed2f09f..3d9e8f2c5 100644 --- a/proto/tendermint/types/evidence.pb.go +++ b/proto/tendermint/types/evidence.pb.go @@ -7,15 +7,19 @@ 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" 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. @@ -23,18 +27,105 @@ var _ = math.Inf // proto package needs to be updated. const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package -// DuplicateVoteEvidence contains evidence a validator signed two conflicting -// votes. +type Evidence struct { + // Types that are valid to be assigned to Sum: + // *Evidence_DuplicateVoteEvidence + // *Evidence_LightClientAttackEvidence + Sum isEvidence_Sum `protobuf_oneof:"sum"` +} + +func (m *Evidence) Reset() { *m = Evidence{} } +func (m *Evidence) String() string { return proto.CompactTextString(m) } +func (*Evidence) ProtoMessage() {} +func (*Evidence) Descriptor() ([]byte, []int) { + return fileDescriptor_6825fabc78e0a168, []int{0} +} +func (m *Evidence) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Evidence) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Evidence.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 *Evidence) XXX_Merge(src proto.Message) { + xxx_messageInfo_Evidence.Merge(m, src) +} +func (m *Evidence) XXX_Size() int { + return m.Size() +} +func (m *Evidence) XXX_DiscardUnknown() { + xxx_messageInfo_Evidence.DiscardUnknown(m) +} + +var xxx_messageInfo_Evidence proto.InternalMessageInfo + +type isEvidence_Sum interface { + isEvidence_Sum() + MarshalTo([]byte) (int, error) + Size() int +} + +type Evidence_DuplicateVoteEvidence struct { + DuplicateVoteEvidence *DuplicateVoteEvidence `protobuf:"bytes,1,opt,name=duplicate_vote_evidence,json=duplicateVoteEvidence,proto3,oneof" json:"duplicate_vote_evidence,omitempty"` +} +type Evidence_LightClientAttackEvidence struct { + LightClientAttackEvidence *LightClientAttackEvidence `protobuf:"bytes,2,opt,name=light_client_attack_evidence,json=lightClientAttackEvidence,proto3,oneof" json:"light_client_attack_evidence,omitempty"` +} + +func (*Evidence_DuplicateVoteEvidence) isEvidence_Sum() {} +func (*Evidence_LightClientAttackEvidence) isEvidence_Sum() {} + +func (m *Evidence) GetSum() isEvidence_Sum { + if m != nil { + return m.Sum + } + return nil +} + +func (m *Evidence) GetDuplicateVoteEvidence() *DuplicateVoteEvidence { + if x, ok := m.GetSum().(*Evidence_DuplicateVoteEvidence); ok { + return x.DuplicateVoteEvidence + } + return nil +} + +func (m *Evidence) GetLightClientAttackEvidence() *LightClientAttackEvidence { + if x, ok := m.GetSum().(*Evidence_LightClientAttackEvidence); ok { + return x.LightClientAttackEvidence + } + return nil +} + +// XXX_OneofWrappers is for the internal use of the proto package. +func (*Evidence) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*Evidence_DuplicateVoteEvidence)(nil), + (*Evidence_LightClientAttackEvidence)(nil), + } +} + +// DuplicateVoteEvidence contains evidence of a validator signed two conflicting votes. type DuplicateVoteEvidence struct { - VoteA *Vote `protobuf:"bytes,1,opt,name=vote_a,json=voteA,proto3" json:"vote_a,omitempty"` - VoteB *Vote `protobuf:"bytes,2,opt,name=vote_b,json=voteB,proto3" json:"vote_b,omitempty"` + VoteA *Vote `protobuf:"bytes,1,opt,name=vote_a,json=voteA,proto3" json:"vote_a,omitempty"` + VoteB *Vote `protobuf:"bytes,2,opt,name=vote_b,json=voteB,proto3" json:"vote_b,omitempty"` + TotalVotingPower int64 `protobuf:"varint,3,opt,name=total_voting_power,json=totalVotingPower,proto3" json:"total_voting_power,omitempty"` + ValidatorPower int64 `protobuf:"varint,4,opt,name=validator_power,json=validatorPower,proto3" json:"validator_power,omitempty"` + Timestamp time.Time `protobuf:"bytes,5,opt,name=timestamp,proto3,stdtime" json:"timestamp"` } func (m *DuplicateVoteEvidence) Reset() { *m = DuplicateVoteEvidence{} } func (m *DuplicateVoteEvidence) String() string { return proto.CompactTextString(m) } func (*DuplicateVoteEvidence) ProtoMessage() {} func (*DuplicateVoteEvidence) Descriptor() ([]byte, []int) { - return fileDescriptor_6825fabc78e0a168, []int{0} + return fileDescriptor_6825fabc78e0a168, []int{1} } func (m *DuplicateVoteEvidence) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -77,16 +168,41 @@ func (m *DuplicateVoteEvidence) GetVoteB() *Vote { return nil } +func (m *DuplicateVoteEvidence) GetTotalVotingPower() int64 { + if m != nil { + return m.TotalVotingPower + } + return 0 +} + +func (m *DuplicateVoteEvidence) GetValidatorPower() int64 { + if m != nil { + return m.ValidatorPower + } + return 0 +} + +func (m *DuplicateVoteEvidence) GetTimestamp() time.Time { + if m != nil { + return m.Timestamp + } + return time.Time{} +} + +// LightClientAttackEvidence contains evidence of a set of validators attempting to mislead a light client. type LightClientAttackEvidence struct { - ConflictingBlock *LightBlock `protobuf:"bytes,1,opt,name=conflicting_block,json=conflictingBlock,proto3" json:"conflicting_block,omitempty"` - CommonHeight int64 `protobuf:"varint,2,opt,name=common_height,json=commonHeight,proto3" json:"common_height,omitempty"` + ConflictingBlock *LightBlock `protobuf:"bytes,1,opt,name=conflicting_block,json=conflictingBlock,proto3" json:"conflicting_block,omitempty"` + CommonHeight int64 `protobuf:"varint,2,opt,name=common_height,json=commonHeight,proto3" json:"common_height,omitempty"` + ByzantineValidators []*Validator `protobuf:"bytes,3,rep,name=byzantine_validators,json=byzantineValidators,proto3" json:"byzantine_validators,omitempty"` + TotalVotingPower int64 `protobuf:"varint,4,opt,name=total_voting_power,json=totalVotingPower,proto3" json:"total_voting_power,omitempty"` + Timestamp time.Time `protobuf:"bytes,5,opt,name=timestamp,proto3,stdtime" json:"timestamp"` } func (m *LightClientAttackEvidence) Reset() { *m = LightClientAttackEvidence{} } func (m *LightClientAttackEvidence) String() string { return proto.CompactTextString(m) } func (*LightClientAttackEvidence) ProtoMessage() {} func (*LightClientAttackEvidence) Descriptor() ([]byte, []int) { - return fileDescriptor_6825fabc78e0a168, []int{1} + return fileDescriptor_6825fabc78e0a168, []int{2} } func (m *LightClientAttackEvidence) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -129,108 +245,43 @@ func (m *LightClientAttackEvidence) GetCommonHeight() int64 { return 0 } -type Evidence struct { - // Types that are valid to be assigned to Sum: - // *Evidence_DuplicateVoteEvidence - // *Evidence_LightClientAttackEvidence - Sum isEvidence_Sum `protobuf_oneof:"sum"` -} - -func (m *Evidence) Reset() { *m = Evidence{} } -func (m *Evidence) String() string { return proto.CompactTextString(m) } -func (*Evidence) ProtoMessage() {} -func (*Evidence) Descriptor() ([]byte, []int) { - return fileDescriptor_6825fabc78e0a168, []int{2} -} -func (m *Evidence) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *Evidence) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - if deterministic { - return xxx_messageInfo_Evidence.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 *Evidence) XXX_Merge(src proto.Message) { - xxx_messageInfo_Evidence.Merge(m, src) -} -func (m *Evidence) XXX_Size() int { - return m.Size() -} -func (m *Evidence) XXX_DiscardUnknown() { - xxx_messageInfo_Evidence.DiscardUnknown(m) -} - -var xxx_messageInfo_Evidence proto.InternalMessageInfo - -type isEvidence_Sum interface { - isEvidence_Sum() - MarshalTo([]byte) (int, error) - Size() int -} - -type Evidence_DuplicateVoteEvidence struct { - DuplicateVoteEvidence *DuplicateVoteEvidence `protobuf:"bytes,1,opt,name=duplicate_vote_evidence,json=duplicateVoteEvidence,proto3,oneof" json:"duplicate_vote_evidence,omitempty"` -} -type Evidence_LightClientAttackEvidence struct { - LightClientAttackEvidence *LightClientAttackEvidence `protobuf:"bytes,2,opt,name=light_client_attack_evidence,json=lightClientAttackEvidence,proto3,oneof" json:"light_client_attack_evidence,omitempty"` -} - -func (*Evidence_DuplicateVoteEvidence) isEvidence_Sum() {} -func (*Evidence_LightClientAttackEvidence) isEvidence_Sum() {} - -func (m *Evidence) GetSum() isEvidence_Sum { +func (m *LightClientAttackEvidence) GetByzantineValidators() []*Validator { if m != nil { - return m.Sum - } - return nil -} - -func (m *Evidence) GetDuplicateVoteEvidence() *DuplicateVoteEvidence { - if x, ok := m.GetSum().(*Evidence_DuplicateVoteEvidence); ok { - return x.DuplicateVoteEvidence + return m.ByzantineValidators } return nil } -func (m *Evidence) GetLightClientAttackEvidence() *LightClientAttackEvidence { - if x, ok := m.GetSum().(*Evidence_LightClientAttackEvidence); ok { - return x.LightClientAttackEvidence +func (m *LightClientAttackEvidence) GetTotalVotingPower() int64 { + if m != nil { + return m.TotalVotingPower } - return nil + return 0 } -// XXX_OneofWrappers is for the internal use of the proto package. -func (*Evidence) XXX_OneofWrappers() []interface{} { - return []interface{}{ - (*Evidence_DuplicateVoteEvidence)(nil), - (*Evidence_LightClientAttackEvidence)(nil), +func (m *LightClientAttackEvidence) GetTimestamp() time.Time { + if m != nil { + return m.Timestamp } + return time.Time{} } -// EvidenceData contains any evidence of malicious wrong-doing by validators -type EvidenceData struct { +type EvidenceList struct { Evidence []Evidence `protobuf:"bytes,1,rep,name=evidence,proto3" json:"evidence"` } -func (m *EvidenceData) Reset() { *m = EvidenceData{} } -func (m *EvidenceData) String() string { return proto.CompactTextString(m) } -func (*EvidenceData) ProtoMessage() {} -func (*EvidenceData) Descriptor() ([]byte, []int) { +func (m *EvidenceList) Reset() { *m = EvidenceList{} } +func (m *EvidenceList) String() string { return proto.CompactTextString(m) } +func (*EvidenceList) ProtoMessage() {} +func (*EvidenceList) Descriptor() ([]byte, []int) { return fileDescriptor_6825fabc78e0a168, []int{3} } -func (m *EvidenceData) XXX_Unmarshal(b []byte) error { +func (m *EvidenceList) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) } -func (m *EvidenceData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { +func (m *EvidenceList) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { if deterministic { - return xxx_messageInfo_EvidenceData.Marshal(b, m, deterministic) + return xxx_messageInfo_EvidenceList.Marshal(b, m, deterministic) } else { b = b[:cap(b)] n, err := m.MarshalToSizedBuffer(b) @@ -240,19 +291,19 @@ func (m *EvidenceData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) return b[:n], nil } } -func (m *EvidenceData) XXX_Merge(src proto.Message) { - xxx_messageInfo_EvidenceData.Merge(m, src) +func (m *EvidenceList) XXX_Merge(src proto.Message) { + xxx_messageInfo_EvidenceList.Merge(m, src) } -func (m *EvidenceData) XXX_Size() int { +func (m *EvidenceList) XXX_Size() int { return m.Size() } -func (m *EvidenceData) XXX_DiscardUnknown() { - xxx_messageInfo_EvidenceData.DiscardUnknown(m) +func (m *EvidenceList) XXX_DiscardUnknown() { + xxx_messageInfo_EvidenceList.DiscardUnknown(m) } -var xxx_messageInfo_EvidenceData proto.InternalMessageInfo +var xxx_messageInfo_EvidenceList proto.InternalMessageInfo -func (m *EvidenceData) GetEvidence() []Evidence { +func (m *EvidenceList) GetEvidence() []Evidence { if m != nil { return m.Evidence } @@ -260,44 +311,53 @@ func (m *EvidenceData) GetEvidence() []Evidence { } func init() { + proto.RegisterType((*Evidence)(nil), "tendermint.types.Evidence") proto.RegisterType((*DuplicateVoteEvidence)(nil), "tendermint.types.DuplicateVoteEvidence") proto.RegisterType((*LightClientAttackEvidence)(nil), "tendermint.types.LightClientAttackEvidence") - proto.RegisterType((*Evidence)(nil), "tendermint.types.Evidence") - proto.RegisterType((*EvidenceData)(nil), "tendermint.types.EvidenceData") + proto.RegisterType((*EvidenceList)(nil), "tendermint.types.EvidenceList") } func init() { proto.RegisterFile("tendermint/types/evidence.proto", fileDescriptor_6825fabc78e0a168) } var fileDescriptor_6825fabc78e0a168 = []byte{ - // 388 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x84, 0x52, 0xc1, 0x6a, 0xea, 0x40, - 0x14, 0x4d, 0x9e, 0x4f, 0x91, 0xd1, 0x07, 0xbe, 0xf0, 0x7c, 0x55, 0x91, 0x58, 0xd2, 0x45, 0x85, - 0xd2, 0x04, 0xec, 0xa2, 0x9b, 0x6e, 0x4c, 0x2d, 0x58, 0x70, 0xd3, 0x2c, 0xba, 0xe8, 0x26, 0x24, - 0x93, 0x69, 0x1c, 0x4c, 0x66, 0x44, 0x6f, 0x84, 0x7e, 0x43, 0x37, 0xfd, 0x2c, 0x97, 0x2e, 0xbb, - 0x2a, 0x45, 0xfb, 0x21, 0x25, 0x13, 0x8d, 0x62, 0x94, 0x6e, 0x86, 0xe1, 0xdc, 0x73, 0xef, 0xb9, - 0xe7, 0x70, 0x51, 0x0b, 0x08, 0xf3, 0xc8, 0x24, 0xa4, 0x0c, 0x0c, 0x78, 0x19, 0x93, 0xa9, 0x41, - 0x66, 0xd4, 0x23, 0x0c, 0x13, 0x7d, 0x3c, 0xe1, 0xc0, 0x95, 0xca, 0x96, 0xa0, 0x0b, 0x42, 0xe3, - 0x9f, 0xcf, 0x7d, 0x2e, 0x8a, 0x46, 0xfc, 0x4b, 0x78, 0x8d, 0x66, 0x66, 0x90, 0x78, 0x93, 0xaa, - 0x16, 0xa1, 0x6a, 0x2f, 0x1a, 0x07, 0x14, 0x3b, 0x40, 0x1e, 0x39, 0x90, 0xbb, 0xb5, 0x88, 0x72, - 0x89, 0x0a, 0x33, 0x0e, 0xc4, 0x76, 0x6a, 0xf2, 0xa9, 0xdc, 0x2e, 0x75, 0xfe, 0xeb, 0xfb, 0x7a, - 0x7a, 0xcc, 0xb7, 0xf2, 0x31, 0xab, 0x9b, 0xd2, 0xdd, 0xda, 0xaf, 0x9f, 0xe9, 0xa6, 0xf6, 0x2a, - 0xa3, 0xfa, 0x80, 0xfa, 0x43, 0xb8, 0x0d, 0x28, 0x61, 0xd0, 0x05, 0x70, 0xf0, 0x28, 0xd5, 0xbe, - 0x47, 0x7f, 0x31, 0x67, 0xcf, 0x01, 0xc5, 0x40, 0x99, 0x6f, 0xbb, 0x01, 0xc7, 0xa3, 0xf5, 0x1a, - 0xcd, 0xec, 0x5c, 0x31, 0xc7, 0x8c, 0x39, 0x56, 0x65, 0xa7, 0x4d, 0x20, 0xca, 0x19, 0xfa, 0x83, - 0x79, 0x18, 0x72, 0x66, 0x0f, 0x49, 0xcc, 0x13, 0xeb, 0xe5, 0xac, 0x72, 0x02, 0xf6, 0x05, 0xa6, - 0x7d, 0xc9, 0xa8, 0x98, 0x8a, 0x3b, 0xe8, 0xc4, 0xdb, 0x24, 0x62, 0x0b, 0x4f, 0x9b, 0xe0, 0xd7, - 0x2b, 0x9c, 0x67, 0x57, 0x38, 0x18, 0x61, 0x5f, 0xb2, 0xaa, 0xde, 0xc1, 0x6c, 0x19, 0x6a, 0x06, - 0xb1, 0xb0, 0x8d, 0x85, 0x7b, 0xdb, 0x11, 0xf6, 0xb7, 0x3a, 0x49, 0x84, 0x17, 0x47, 0xac, 0x1e, - 0x8a, 0xac, 0x2f, 0x59, 0xf5, 0xe0, 0x58, 0xd1, 0xcc, 0xa3, 0xdc, 0x34, 0x0a, 0xb5, 0x01, 0x2a, - 0x6f, 0xa0, 0x9e, 0x03, 0x8e, 0x72, 0x83, 0x8a, 0x3b, 0xd6, 0x72, 0xed, 0x52, 0xa7, 0x91, 0x95, - 0x4c, 0x87, 0xfc, 0x9e, 0x7f, 0xb4, 0x24, 0x2b, 0xed, 0x30, 0x1f, 0xe6, 0x4b, 0x55, 0x5e, 0x2c, - 0x55, 0xf9, 0x73, 0xa9, 0xca, 0x6f, 0x2b, 0x55, 0x5a, 0xac, 0x54, 0xe9, 0x7d, 0xa5, 0x4a, 0x4f, - 0xd7, 0x3e, 0x85, 0x61, 0xe4, 0xea, 0x98, 0x87, 0xc6, 0xee, 0xf1, 0x6d, 0xbf, 0xc9, 0x91, 0xee, - 0x1f, 0xa6, 0x5b, 0x10, 0xf8, 0xd5, 0x77, 0x00, 0x00, 0x00, 0xff, 0xff, 0xc0, 0x7f, 0xe9, 0x7a, - 0xfc, 0x02, 0x00, 0x00, + // 532 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x94, 0xcf, 0x6e, 0xd3, 0x40, + 0x10, 0xc6, 0xed, 0x3a, 0xa9, 0xc2, 0xb6, 0x40, 0x58, 0x5a, 0x48, 0x43, 0xe4, 0x44, 0xe1, 0xd0, + 0x48, 0x80, 0x2d, 0x95, 0x03, 0x17, 0x2e, 0x35, 0x20, 0x15, 0x29, 0x42, 0x60, 0xa1, 0x1e, 0xb8, + 0x58, 0x6b, 0x7b, 0xeb, 0xac, 0x6a, 0xef, 0x5a, 0xf1, 0x24, 0xa8, 0x3c, 0x45, 0x1e, 0xab, 0x17, + 0xa4, 0x1e, 0x39, 0x01, 0x4a, 0x78, 0x10, 0xe4, 0xf5, 0x9f, 0x44, 0x75, 0xcc, 0x89, 0x4b, 0xe4, + 0xcc, 0xfc, 0xbe, 0x9d, 0x99, 0xcf, 0xb3, 0x46, 0x7d, 0xa0, 0xdc, 0xa7, 0xd3, 0x88, 0x71, 0x30, + 0xe1, 0x2a, 0xa6, 0x89, 0x49, 0xe7, 0xcc, 0xa7, 0xdc, 0xa3, 0x46, 0x3c, 0x15, 0x20, 0x70, 0x7b, + 0x0d, 0x18, 0x12, 0xe8, 0x1e, 0x04, 0x22, 0x10, 0x32, 0x69, 0xa6, 0x4f, 0x19, 0xd7, 0xed, 0x07, + 0x42, 0x04, 0x21, 0x35, 0xe5, 0x3f, 0x77, 0x76, 0x61, 0x02, 0x8b, 0x68, 0x02, 0x24, 0x8a, 0x73, + 0xa0, 0x57, 0xa9, 0x24, 0x7f, 0xf3, 0xec, 0xa0, 0x92, 0x9d, 0x93, 0x90, 0xf9, 0x04, 0xc4, 0x34, + 0x23, 0x86, 0x7f, 0x54, 0xd4, 0x7a, 0x97, 0xf7, 0x86, 0x09, 0x7a, 0xec, 0xcf, 0xe2, 0x90, 0x79, + 0x04, 0xa8, 0x33, 0x17, 0x40, 0x9d, 0xa2, 0xed, 0x8e, 0x3a, 0x50, 0x47, 0x7b, 0x27, 0xc7, 0xc6, + 0xed, 0xbe, 0x8d, 0xb7, 0x85, 0xe0, 0x5c, 0x00, 0x2d, 0x4e, 0x3a, 0x53, 0xec, 0x43, 0x7f, 0x5b, + 0x02, 0x73, 0xd4, 0x0b, 0x59, 0x30, 0x01, 0xc7, 0x0b, 0x19, 0xe5, 0xe0, 0x10, 0x00, 0xe2, 0x5d, + 0xae, 0xeb, 0xec, 0xc8, 0x3a, 0xcf, 0xaa, 0x75, 0xc6, 0xa9, 0xea, 0x8d, 0x14, 0x9d, 0x4a, 0xcd, + 0x46, 0xad, 0xa3, 0xb0, 0x2e, 0x69, 0x35, 0x91, 0x96, 0xcc, 0xa2, 0xe1, 0x62, 0x07, 0x1d, 0x6e, + 0xed, 0x14, 0xbf, 0x40, 0xbb, 0x72, 0x52, 0x92, 0x8f, 0xf8, 0xa8, 0x5a, 0x3a, 0xe5, 0xed, 0x66, + 0x4a, 0x9d, 0x96, 0xb8, 0x9b, 0x77, 0xfa, 0x4f, 0xdc, 0xc2, 0xcf, 0x11, 0x06, 0x01, 0x24, 0x4c, + 0xdd, 0x64, 0x3c, 0x70, 0x62, 0xf1, 0x95, 0x4e, 0x3b, 0xda, 0x40, 0x1d, 0x69, 0x76, 0x5b, 0x66, + 0xce, 0x65, 0xe2, 0x63, 0x1a, 0xc7, 0xc7, 0xe8, 0x7e, 0xf9, 0x7e, 0x72, 0xb4, 0x21, 0xd1, 0x7b, + 0x65, 0x38, 0x03, 0x2d, 0x74, 0xa7, 0x5c, 0x84, 0x4e, 0x53, 0x36, 0xd2, 0x35, 0xb2, 0x55, 0x31, + 0x8a, 0x55, 0x31, 0x3e, 0x17, 0x84, 0xd5, 0xba, 0xfe, 0xd9, 0x57, 0x16, 0xbf, 0xfa, 0xaa, 0xbd, + 0x96, 0x0d, 0xbf, 0xef, 0xa0, 0xa3, 0x5a, 0x53, 0xf1, 0x7b, 0xf4, 0xc0, 0x13, 0xfc, 0x22, 0x64, + 0x9e, 0xec, 0xdb, 0x0d, 0x85, 0x77, 0x99, 0x3b, 0xd4, 0xab, 0x79, 0x39, 0x56, 0xca, 0xd8, 0xed, + 0x0d, 0x99, 0x8c, 0xe0, 0xa7, 0xe8, 0xae, 0x27, 0xa2, 0x48, 0x70, 0x67, 0x42, 0x53, 0x4e, 0x3a, + 0xa7, 0xd9, 0xfb, 0x59, 0xf0, 0x4c, 0xc6, 0xf0, 0x07, 0x74, 0xe0, 0x5e, 0x7d, 0x23, 0x1c, 0x18, + 0xa7, 0x4e, 0x39, 0x6d, 0xd2, 0xd1, 0x06, 0xda, 0x68, 0xef, 0xe4, 0xc9, 0x16, 0x97, 0x0b, 0xc6, + 0x7e, 0x58, 0x0a, 0xcb, 0x58, 0x52, 0x63, 0x7c, 0xa3, 0xc6, 0xf8, 0xff, 0xe1, 0xe7, 0x18, 0xed, + 0x17, 0xee, 0x8d, 0x59, 0x02, 0xf8, 0x35, 0x6a, 0x6d, 0xdc, 0x1e, 0x4d, 0x1e, 0x59, 0x99, 0xa2, + 0xdc, 0xd3, 0x46, 0x7a, 0xa4, 0x5d, 0x2a, 0xac, 0x4f, 0xd7, 0x4b, 0x5d, 0xbd, 0x59, 0xea, 0xea, + 0xef, 0xa5, 0xae, 0x2e, 0x56, 0xba, 0x72, 0xb3, 0xd2, 0x95, 0x1f, 0x2b, 0x5d, 0xf9, 0xf2, 0x2a, + 0x60, 0x30, 0x99, 0xb9, 0x86, 0x27, 0x22, 0x73, 0xf3, 0x7a, 0xaf, 0x1f, 0xb3, 0xaf, 0xc8, 0xed, + 0xab, 0xef, 0xee, 0xca, 0xf8, 0xcb, 0xbf, 0x01, 0x00, 0x00, 0xff, 0xff, 0xa6, 0x21, 0x16, 0x68, + 0x9d, 0x04, 0x00, 0x00, } -func (m *DuplicateVoteEvidence) Marshal() (dAtA []byte, err error) { +func (m *Evidence) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalToSizedBuffer(dAtA[:size]) @@ -307,19 +367,38 @@ func (m *DuplicateVoteEvidence) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *DuplicateVoteEvidence) MarshalTo(dAtA []byte) (int, error) { +func (m *Evidence) MarshalTo(dAtA []byte) (int, error) { size := m.Size() return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *DuplicateVoteEvidence) MarshalToSizedBuffer(dAtA []byte) (int, error) { +func (m *Evidence) MarshalToSizedBuffer(dAtA []byte) (int, error) { i := len(dAtA) _ = i var l int _ = l - if m.VoteB != nil { + if m.Sum != nil { { - size, err := m.VoteB.MarshalToSizedBuffer(dAtA[:i]) + size := m.Sum.Size() + i -= size + if _, err := m.Sum.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + } + } + return len(dAtA) - i, nil +} + +func (m *Evidence_DuplicateVoteEvidence) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Evidence_DuplicateVoteEvidence) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + if m.DuplicateVoteEvidence != nil { + { + size, err := m.DuplicateVoteEvidence.MarshalToSizedBuffer(dAtA[:i]) if err != nil { return 0, err } @@ -327,11 +406,20 @@ func (m *DuplicateVoteEvidence) MarshalToSizedBuffer(dAtA []byte) (int, error) { i = encodeVarintEvidence(dAtA, i, uint64(size)) } i-- - dAtA[i] = 0x12 + dAtA[i] = 0xa } - if m.VoteA != nil { + return len(dAtA) - i, nil +} +func (m *Evidence_LightClientAttackEvidence) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Evidence_LightClientAttackEvidence) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + if m.LightClientAttackEvidence != nil { { - size, err := m.VoteA.MarshalToSizedBuffer(dAtA[:i]) + size, err := m.LightClientAttackEvidence.MarshalToSizedBuffer(dAtA[:i]) if err != nil { return 0, err } @@ -339,12 +427,11 @@ func (m *DuplicateVoteEvidence) MarshalToSizedBuffer(dAtA []byte) (int, error) { i = encodeVarintEvidence(dAtA, i, uint64(size)) } i-- - dAtA[i] = 0xa + dAtA[i] = 0x12 } return len(dAtA) - i, nil } - -func (m *LightClientAttackEvidence) Marshal() (dAtA []byte, err error) { +func (m *DuplicateVoteEvidence) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalToSizedBuffer(dAtA[:size]) @@ -354,24 +441,49 @@ func (m *LightClientAttackEvidence) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *LightClientAttackEvidence) MarshalTo(dAtA []byte) (int, error) { +func (m *DuplicateVoteEvidence) MarshalTo(dAtA []byte) (int, error) { size := m.Size() return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *LightClientAttackEvidence) MarshalToSizedBuffer(dAtA []byte) (int, error) { +func (m *DuplicateVoteEvidence) MarshalToSizedBuffer(dAtA []byte) (int, error) { i := len(dAtA) _ = i var l int _ = l - if m.CommonHeight != 0 { - i = encodeVarintEvidence(dAtA, i, uint64(m.CommonHeight)) + n3, err3 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Timestamp, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Timestamp):]) + if err3 != nil { + return 0, err3 + } + i -= n3 + i = encodeVarintEvidence(dAtA, i, uint64(n3)) + i-- + dAtA[i] = 0x2a + if m.ValidatorPower != 0 { + i = encodeVarintEvidence(dAtA, i, uint64(m.ValidatorPower)) i-- - dAtA[i] = 0x10 + dAtA[i] = 0x20 } - if m.ConflictingBlock != nil { + if m.TotalVotingPower != 0 { + i = encodeVarintEvidence(dAtA, i, uint64(m.TotalVotingPower)) + i-- + dAtA[i] = 0x18 + } + if m.VoteB != nil { { - size, err := m.ConflictingBlock.MarshalToSizedBuffer(dAtA[:i]) + size, err := m.VoteB.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintEvidence(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + if m.VoteA != nil { + { + size, err := m.VoteA.MarshalToSizedBuffer(dAtA[:i]) if err != nil { return 0, err } @@ -384,7 +496,7 @@ func (m *LightClientAttackEvidence) MarshalToSizedBuffer(dAtA []byte) (int, erro return len(dAtA) - i, nil } -func (m *Evidence) Marshal() (dAtA []byte, err error) { +func (m *LightClientAttackEvidence) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalToSizedBuffer(dAtA[:size]) @@ -394,59 +506,51 @@ func (m *Evidence) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *Evidence) MarshalTo(dAtA []byte) (int, error) { +func (m *LightClientAttackEvidence) MarshalTo(dAtA []byte) (int, error) { size := m.Size() return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *Evidence) MarshalToSizedBuffer(dAtA []byte) (int, error) { +func (m *LightClientAttackEvidence) MarshalToSizedBuffer(dAtA []byte) (int, error) { i := len(dAtA) _ = i var l int _ = l - if m.Sum != nil { - { - size := m.Sum.Size() - i -= size - if _, err := m.Sum.MarshalTo(dAtA[i:]); err != nil { - return 0, err - } - } + n6, err6 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Timestamp, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Timestamp):]) + if err6 != nil { + return 0, err6 } - return len(dAtA) - i, nil -} - -func (m *Evidence_DuplicateVoteEvidence) MarshalTo(dAtA []byte) (int, error) { - size := m.Size() - return m.MarshalToSizedBuffer(dAtA[:size]) -} - -func (m *Evidence_DuplicateVoteEvidence) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - if m.DuplicateVoteEvidence != nil { - { - size, err := m.DuplicateVoteEvidence.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err + i -= n6 + i = encodeVarintEvidence(dAtA, i, uint64(n6)) + i-- + dAtA[i] = 0x2a + if m.TotalVotingPower != 0 { + i = encodeVarintEvidence(dAtA, i, uint64(m.TotalVotingPower)) + i-- + dAtA[i] = 0x20 + } + if len(m.ByzantineValidators) > 0 { + for iNdEx := len(m.ByzantineValidators) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.ByzantineValidators[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintEvidence(dAtA, i, uint64(size)) } - i -= size - i = encodeVarintEvidence(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a } + } + if m.CommonHeight != 0 { + i = encodeVarintEvidence(dAtA, i, uint64(m.CommonHeight)) i-- - dAtA[i] = 0xa + dAtA[i] = 0x10 } - return len(dAtA) - i, nil -} -func (m *Evidence_LightClientAttackEvidence) MarshalTo(dAtA []byte) (int, error) { - size := m.Size() - return m.MarshalToSizedBuffer(dAtA[:size]) -} - -func (m *Evidence_LightClientAttackEvidence) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - if m.LightClientAttackEvidence != nil { + if m.ConflictingBlock != nil { { - size, err := m.LightClientAttackEvidence.MarshalToSizedBuffer(dAtA[:i]) + size, err := m.ConflictingBlock.MarshalToSizedBuffer(dAtA[:i]) if err != nil { return 0, err } @@ -454,11 +558,12 @@ func (m *Evidence_LightClientAttackEvidence) MarshalToSizedBuffer(dAtA []byte) ( i = encodeVarintEvidence(dAtA, i, uint64(size)) } i-- - dAtA[i] = 0x12 + dAtA[i] = 0xa } return len(dAtA) - i, nil } -func (m *EvidenceData) Marshal() (dAtA []byte, err error) { + +func (m *EvidenceList) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalToSizedBuffer(dAtA[:size]) @@ -468,12 +573,12 @@ func (m *EvidenceData) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *EvidenceData) MarshalTo(dAtA []byte) (int, error) { +func (m *EvidenceList) MarshalTo(dAtA []byte) (int, error) { size := m.Size() return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *EvidenceData) MarshalToSizedBuffer(dAtA []byte) (int, error) { +func (m *EvidenceList) MarshalToSizedBuffer(dAtA []byte) (int, error) { i := len(dAtA) _ = i var l int @@ -506,76 +611,95 @@ func encodeVarintEvidence(dAtA []byte, offset int, v uint64) int { dAtA[offset] = uint8(v) return base } -func (m *DuplicateVoteEvidence) Size() (n int) { +func (m *Evidence) Size() (n int) { if m == nil { return 0 } var l int _ = l - if m.VoteA != nil { - l = m.VoteA.Size() - n += 1 + l + sovEvidence(uint64(l)) - } - if m.VoteB != nil { - l = m.VoteB.Size() - n += 1 + l + sovEvidence(uint64(l)) + if m.Sum != nil { + n += m.Sum.Size() } return n } -func (m *LightClientAttackEvidence) Size() (n int) { +func (m *Evidence_DuplicateVoteEvidence) Size() (n int) { if m == nil { return 0 } var l int _ = l - if m.ConflictingBlock != nil { - l = m.ConflictingBlock.Size() + if m.DuplicateVoteEvidence != nil { + l = m.DuplicateVoteEvidence.Size() n += 1 + l + sovEvidence(uint64(l)) } - if m.CommonHeight != 0 { - n += 1 + sovEvidence(uint64(m.CommonHeight)) - } return n } - -func (m *Evidence) Size() (n int) { +func (m *Evidence_LightClientAttackEvidence) Size() (n int) { if m == nil { return 0 } var l int _ = l - if m.Sum != nil { - n += m.Sum.Size() + if m.LightClientAttackEvidence != nil { + l = m.LightClientAttackEvidence.Size() + n += 1 + l + sovEvidence(uint64(l)) } return n } - -func (m *Evidence_DuplicateVoteEvidence) Size() (n int) { +func (m *DuplicateVoteEvidence) Size() (n int) { if m == nil { return 0 } var l int _ = l - if m.DuplicateVoteEvidence != nil { - l = m.DuplicateVoteEvidence.Size() + if m.VoteA != nil { + l = m.VoteA.Size() + n += 1 + l + sovEvidence(uint64(l)) + } + if m.VoteB != nil { + l = m.VoteB.Size() n += 1 + l + sovEvidence(uint64(l)) } + if m.TotalVotingPower != 0 { + n += 1 + sovEvidence(uint64(m.TotalVotingPower)) + } + if m.ValidatorPower != 0 { + n += 1 + sovEvidence(uint64(m.ValidatorPower)) + } + l = github_com_gogo_protobuf_types.SizeOfStdTime(m.Timestamp) + n += 1 + l + sovEvidence(uint64(l)) return n } -func (m *Evidence_LightClientAttackEvidence) Size() (n int) { + +func (m *LightClientAttackEvidence) Size() (n int) { if m == nil { return 0 } var l int _ = l - if m.LightClientAttackEvidence != nil { - l = m.LightClientAttackEvidence.Size() + if m.ConflictingBlock != nil { + l = m.ConflictingBlock.Size() n += 1 + l + sovEvidence(uint64(l)) } + if m.CommonHeight != 0 { + n += 1 + sovEvidence(uint64(m.CommonHeight)) + } + if len(m.ByzantineValidators) > 0 { + for _, e := range m.ByzantineValidators { + l = e.Size() + n += 1 + l + sovEvidence(uint64(l)) + } + } + if m.TotalVotingPower != 0 { + n += 1 + sovEvidence(uint64(m.TotalVotingPower)) + } + l = github_com_gogo_protobuf_types.SizeOfStdTime(m.Timestamp) + n += 1 + l + sovEvidence(uint64(l)) return n } -func (m *EvidenceData) Size() (n int) { + +func (m *EvidenceList) Size() (n int) { if m == nil { return 0 } @@ -596,7 +720,7 @@ func sovEvidence(x uint64) (n int) { func sozEvidence(x uint64) (n int) { return sovEvidence(uint64((x << 1) ^ uint64((int64(x) >> 63)))) } -func (m *DuplicateVoteEvidence) Unmarshal(dAtA []byte) error { +func (m *Evidence) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -619,15 +743,15 @@ func (m *DuplicateVoteEvidence) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: DuplicateVoteEvidence: wiretype end group for non-group") + return fmt.Errorf("proto: Evidence: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: DuplicateVoteEvidence: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: Evidence: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field VoteA", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field DuplicateVoteEvidence", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -654,16 +778,15 @@ func (m *DuplicateVoteEvidence) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.VoteA == nil { - m.VoteA = &Vote{} - } - if err := m.VoteA.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + v := &DuplicateVoteEvidence{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } + m.Sum = &Evidence_DuplicateVoteEvidence{v} iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field VoteB", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field LightClientAttackEvidence", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -690,12 +813,11 @@ func (m *DuplicateVoteEvidence) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.VoteB == nil { - m.VoteB = &Vote{} - } - if err := m.VoteB.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + v := &LightClientAttackEvidence{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } + m.Sum = &Evidence_LightClientAttackEvidence{v} iNdEx = postIndex default: iNdEx = preIndex @@ -721,7 +843,7 @@ func (m *DuplicateVoteEvidence) Unmarshal(dAtA []byte) error { } return nil } -func (m *LightClientAttackEvidence) Unmarshal(dAtA []byte) error { +func (m *DuplicateVoteEvidence) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -744,15 +866,15 @@ func (m *LightClientAttackEvidence) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: LightClientAttackEvidence: wiretype end group for non-group") + return fmt.Errorf("proto: DuplicateVoteEvidence: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: LightClientAttackEvidence: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: DuplicateVoteEvidence: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ConflictingBlock", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field VoteA", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -779,18 +901,54 @@ func (m *LightClientAttackEvidence) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.ConflictingBlock == nil { - m.ConflictingBlock = &LightBlock{} + if m.VoteA == nil { + m.VoteA = &Vote{} } - if err := m.ConflictingBlock.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.VoteA.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field VoteB", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEvidence + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthEvidence + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthEvidence + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.VoteB == nil { + m.VoteB = &Vote{} + } + if err := m.VoteB.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field CommonHeight", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field TotalVotingPower", wireType) } - m.CommonHeight = 0 + m.TotalVotingPower = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowEvidence @@ -800,11 +958,63 @@ func (m *LightClientAttackEvidence) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.CommonHeight |= int64(b&0x7F) << shift + m.TotalVotingPower |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ValidatorPower", wireType) + } + m.ValidatorPower = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEvidence + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ValidatorPower |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Timestamp", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEvidence + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } + if msglen < 0 { + return ErrInvalidLengthEvidence + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthEvidence + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := github_com_gogo_protobuf_types.StdTimeUnmarshal(&m.Timestamp, dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipEvidence(dAtA[iNdEx:]) @@ -829,7 +1039,7 @@ func (m *LightClientAttackEvidence) Unmarshal(dAtA []byte) error { } return nil } -func (m *Evidence) Unmarshal(dAtA []byte) error { +func (m *LightClientAttackEvidence) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -852,15 +1062,15 @@ func (m *Evidence) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: Evidence: wiretype end group for non-group") + return fmt.Errorf("proto: LightClientAttackEvidence: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: Evidence: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: LightClientAttackEvidence: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field DuplicateVoteEvidence", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ConflictingBlock", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -887,15 +1097,35 @@ func (m *Evidence) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - v := &DuplicateVoteEvidence{} - if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if m.ConflictingBlock == nil { + m.ConflictingBlock = &LightBlock{} + } + if err := m.ConflictingBlock.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } - m.Sum = &Evidence_DuplicateVoteEvidence{v} iNdEx = postIndex case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field CommonHeight", wireType) + } + m.CommonHeight = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEvidence + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.CommonHeight |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field LightClientAttackEvidence", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ByzantineValidators", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -922,11 +1152,62 @@ func (m *Evidence) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - v := &LightClientAttackEvidence{} - if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + m.ByzantineValidators = append(m.ByzantineValidators, &Validator{}) + if err := m.ByzantineValidators[len(m.ByzantineValidators)-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 ErrIntOverflowEvidence + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.TotalVotingPower |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Timestamp", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEvidence + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthEvidence + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthEvidence + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := github_com_gogo_protobuf_types.StdTimeUnmarshal(&m.Timestamp, dAtA[iNdEx:postIndex]); err != nil { return err } - m.Sum = &Evidence_LightClientAttackEvidence{v} iNdEx = postIndex default: iNdEx = preIndex @@ -952,7 +1233,7 @@ func (m *Evidence) Unmarshal(dAtA []byte) error { } return nil } -func (m *EvidenceData) Unmarshal(dAtA []byte) error { +func (m *EvidenceList) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -975,10 +1256,10 @@ func (m *EvidenceData) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: EvidenceData: wiretype end group for non-group") + return fmt.Errorf("proto: EvidenceList: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: EvidenceData: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: EvidenceList: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: diff --git a/proto/tendermint/types/evidence.proto b/proto/tendermint/types/evidence.proto index 5f7d860bf..3b234571b 100644 --- a/proto/tendermint/types/evidence.proto +++ b/proto/tendermint/types/evidence.proto @@ -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]; } diff --git a/rpc/client/evidence_test.go b/rpc/client/evidence_test.go index 351c51bc9..527b8a9b5 100644 --- a/rpc/client/evidence_test.go +++ b/rpc/client/evidence_test.go @@ -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( diff --git a/state/execution.go b/state/execution.go index 009fdea22..241c15e93 100644 --- a/state/execution.go +++ b/state/execution.go @@ -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 diff --git a/state/execution_test.go b/state/execution_test.go index c1ce7ebd3..3d7fa93ab 100644 --- a/state/execution_test.go +++ b/state/execution_test.go @@ -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, + }, + } +} diff --git a/state/mocks/evidence_pool.go b/state/mocks/evidence_pool.go index bfd82e596..7292991ca 100644 --- a/state/mocks/evidence_pool.go +++ b/state/mocks/evidence_pool.go @@ -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) } diff --git a/state/services.go b/state/services.go index e2f12b237..5213f8fdd 100644 --- a/state/services.go +++ b/state/services.go @@ -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 } diff --git a/state/validation.go b/state/validation.go index 779edf273..0ebe37bf9 100644 --- a/state/validation.go +++ b/state/validation.go @@ -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 } diff --git a/state/validation_test.go b/state/validation_test.go index 8fa5f89b3..2b556b268 100644 --- a/state/validation_test.go +++ b/state/validation_test.go @@ -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{}) diff --git a/test/e2e/generator/generate.go b/test/e2e/generator/generate.go index 37779248e..a7da44596 100644 --- a/test/e2e/generator/generate.go +++ b/test/e2e/generator/generate.go @@ -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, } ) diff --git a/test/e2e/networks/ci.toml b/test/e2e/networks/ci.toml index 67cd6fbb6..e2de60a75 100644 --- a/test/e2e/networks/ci.toml +++ b/test/e2e/networks/ci.toml @@ -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] diff --git a/test/e2e/tests/evidence_test.go b/test/e2e/tests/evidence_test.go index b98224940..8abb361c8 100644 --- a/test/e2e/tests/evidence_test.go +++ b/test/e2e/tests/evidence_test.go @@ -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) } }) } diff --git a/test/maverick/consensus/state.go b/test/maverick/consensus/state.go index b12d21edf..a1385bd1e 100644 --- a/test/maverick/consensus/state.go +++ b/test/maverick/consensus/state.go @@ -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) } diff --git a/types/block.go b/types/block.go index 0bdb2be67..71cbe6f82 100644 --- a/types/block.go +++ b/types/block.go @@ -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") } diff --git a/types/block_test.go b/types/block_test.go index 2bdeac7f0..2355cb0f1 100644 --- a/types/block_test.go +++ b/types/block_test.go @@ -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 { diff --git a/types/evidence.go b/types/evidence.go index 5b450f9e1..8007763b7 100644 --- a/types/evidence.go +++ b/types/evidence.go @@ -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, diff --git a/types/evidence_test.go b/types/evidence_test.go index 4982a7d90..2e61f6a9c 100644 --- a/types/evidence_test.go +++ b/types/evidence_test.go @@ -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") }) diff --git a/types/protobuf.go b/types/protobuf.go index 1e633338b..1ee094a9f 100644 --- a/types/protobuf.go +++ b/types/protobuf.go @@ -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) diff --git a/types/protobuf_test.go b/types/protobuf_test.go index 6809591ed..c617751e4 100644 --- a/types/protobuf_test.go +++ b/types/protobuf_test.go @@ -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{} }