Browse Source

types: fix verify commit light / trusting bug (#6414)

pull/6448/head
Callum Waters 4 years ago
committed by GitHub
parent
commit
9e2af66a0a
No known key found for this signature in database GPG Key ID: 4AEE18F83AFDEB23
5 changed files with 627 additions and 590 deletions
  1. +23
    -9
      evidence/verify_test.go
  2. +337
    -18
      types/validation.go
  3. +262
    -0
      types/validation_test.go
  4. +5
    -358
      types/validator_set.go
  5. +0
    -205
      types/validator_set_test.go

+ 23
- 9
evidence/verify_test.go View File

@ -193,6 +193,7 @@ func TestVerifyLightClientAttack_Equivocation(t *testing.T) {
conflictingVals, conflictingPrivVals := factory.RandValidatorSet(5, 10)
conflictingHeader, err := factory.MakeHeader(&types.Header{
ChainID: evidenceChainID,
Height: 10,
Time: defaultEvidenceTime,
ValidatorsHash: conflictingVals.Hash(),
@ -200,6 +201,7 @@ func TestVerifyLightClientAttack_Equivocation(t *testing.T) {
require.NoError(t, err)
trustedHeader, _ := factory.MakeHeader(&types.Header{
ChainID: evidenceChainID,
Height: 10,
Time: defaultEvidenceTime,
ValidatorsHash: conflictingHeader.ValidatorsHash,
@ -286,6 +288,7 @@ func TestVerifyLightClientAttack_Amnesia(t *testing.T) {
conflictingVals, conflictingPrivVals := factory.RandValidatorSet(5, 10)
conflictingHeader, err := factory.MakeHeader(&types.Header{
ChainID: evidenceChainID,
Height: height,
Time: defaultEvidenceTime,
ValidatorsHash: conflictingVals.Hash(),
@ -293,6 +296,7 @@ func TestVerifyLightClientAttack_Amnesia(t *testing.T) {
require.NoError(t, err)
trustedHeader, _ := factory.MakeHeader(&types.Header{
ChainID: evidenceChainID,
Height: height,
Time: defaultEvidenceTime,
ValidatorsHash: conflictingHeader.ValidatorsHash,
@ -485,16 +489,26 @@ func makeLunaticEvidence(
conflictingPrivVals = orderPrivValsByValSet(t, conflictingVals, conflictingPrivVals)
commonHeader := factory.MakeRandomHeader()
commonHeader.Height = commonHeight
commonHeader.Time = commonTime
trustedHeader := factory.MakeRandomHeader()
trustedHeader.Height = height
commonHeader, err := factory.MakeHeader(&types.Header{
ChainID: evidenceChainID,
Height: commonHeight,
Time: commonTime,
})
require.NoError(t, err)
trustedHeader, err := factory.MakeHeader(&types.Header{
ChainID: evidenceChainID,
Height: height,
Time: defaultEvidenceTime,
})
require.NoError(t, err)
conflictingHeader := factory.MakeRandomHeader()
conflictingHeader.Height = height
conflictingHeader.Time = attackTime
conflictingHeader.ValidatorsHash = conflictingVals.Hash()
conflictingHeader, err := factory.MakeHeader(&types.Header{
ChainID: evidenceChainID,
Height: height,
Time: attackTime,
ValidatorsHash: conflictingVals.Hash(),
})
require.NoError(t, err)
blockID := factory.MakeBlockIDWithHash(conflictingHeader.Hash())
voteSet := types.NewVoteSet(evidenceChainID, height, 1, tmproto.SignedMsgType(2), conflictingVals)


+ 337
- 18
types/validation.go View File

@ -1,30 +1,141 @@
package types
import (
"errors"
"fmt"
"time"
"github.com/tendermint/tendermint/crypto/batch"
"github.com/tendermint/tendermint/crypto/tmhash"
tmtime "github.com/tendermint/tendermint/types/time"
tmmath "github.com/tendermint/tendermint/libs/math"
)
// ValidateTime does a basic time validation ensuring time does not drift too
// much: +/- one year.
// TODO: reduce this to eg 1 day
// NOTE: DO NOT USE in ValidateBasic methods in this package. This function
// can only be used for real time validation, like on proposals and votes
// in the consensus. If consensus is stuck, and rounds increase for more than a day,
// having only a 1-day band here could break things...
// Can't use for validating blocks because we may be syncing years worth of history.
func ValidateTime(t time.Time) error {
var (
now = tmtime.Now()
oneYear = 8766 * time.Hour
)
if t.Before(now.Add(-oneYear)) || t.After(now.Add(oneYear)) {
return fmt.Errorf("time drifted too much. Expected: -1 < %v < 1 year", now)
// VerifyCommit verifies +2/3 of the set had signed the given commit.
//
// It checks all the signatures! While it's safe to exit as soon as we have
// 2/3+ signatures, doing so would impact incentivization logic in the ABCI
// application that depends on the LastCommitInfo sent in BeginBlock, which
// includes which validators signed. For instance, Gaia incentivizes proposers
// with a bonus for including more than +2/3 of the signatures.
func VerifyCommit(chainID string, vals *ValidatorSet, blockID BlockID,
height int64, commit *Commit) error {
// run a basic validation of the arguments
if err := verifyBasicValsAndCommit(vals, commit, height, blockID); err != nil {
return err
}
return nil
// calculate voting power needed. Note that total voting power is capped to
// 1/8th of max int64 so this operation should never overflow
votingPowerNeeded := vals.TotalVotingPower() * 2 / 3
// ignore all absent signatures
ignore := func(c CommitSig) bool { return c.Absent() }
// only count the signatures that are for the block
count := func(c CommitSig) bool { return c.ForBlock() }
// attempt to batch verify
cacheSignBytes, success, err := tryVerifyCommitBatch(
chainID, vals, commit, votingPowerNeeded, ignore, count, true, true)
if err != nil {
return err
}
if success {
return nil
}
// if verification failed or is not supported then fallback to single verification
return verifyCommitSingle(chainID, vals, commit, votingPowerNeeded,
cacheSignBytes, ignore, count, true, true)
}
// LIGHT CLIENT VERIFICATION METHODS
// VerifyCommitLight verifies +2/3 of the set had signed the given commit.
//
// This method is primarily used by the light client and does not check all the
// signatures.
func VerifyCommitLight(chainID string, vals *ValidatorSet, blockID BlockID,
height int64, commit *Commit) error {
// run a basic validation of the arguments
if err := verifyBasicValsAndCommit(vals, commit, height, blockID); err != nil {
return err
}
// calculate voting power needed
votingPowerNeeded := vals.TotalVotingPower() * 2 / 3
// ignore all commit signatures that are not for the block
ignore := func(c CommitSig) bool { return !c.ForBlock() }
// count all the remaining signatures
count := func(c CommitSig) bool { return true }
// attempt to batch verify
cacheSignBytes, success, err := tryVerifyCommitBatch(
chainID, vals, commit, votingPowerNeeded, ignore, count, false, true)
if err != nil {
return err
}
if success {
return nil
}
// if verification failed or is not supported then fallback to single verification
return verifyCommitSingle(chainID, vals, commit, votingPowerNeeded,
cacheSignBytes, ignore, count, false, true)
}
// VerifyCommitLightTrusting verifies that trustLevel of the validator set signed
// this commit.
//
// NOTE the given validators do not necessarily correspond to the validator set
// for this commit, but there may be some intersection.
//
// This method is primarily used by the light client and does not check all the
// signatures.
func VerifyCommitLightTrusting(chainID string, vals *ValidatorSet, commit *Commit, trustLevel tmmath.Fraction) error {
// sanity checks
if vals == nil {
return errors.New("nil validator set")
}
if trustLevel.Denominator == 0 {
return errors.New("trustLevel has zero Denominator")
}
if commit == nil {
return errors.New("nil commit")
}
// safely calculate voting power needed.
totalVotingPowerMulByNumerator, overflow := safeMul(vals.TotalVotingPower(), int64(trustLevel.Numerator))
if overflow {
return errors.New("int64 overflow while calculating voting power needed. please provide smaller trustLevel numerator")
}
votingPowerNeeded := totalVotingPowerMulByNumerator / int64(trustLevel.Denominator)
// ignore all commit signatures that are not for the block
ignore := func(c CommitSig) bool { return !c.ForBlock() }
// count all the remaining signatures
count := func(c CommitSig) bool { return true }
// attempt to batch verify commit. As the validator set doesn't necessarily
// correspond with the validator set that signed the block we need to look
// up by address rather than index.
cacheSignBytes, success, err := tryVerifyCommitBatch(
chainID, vals, commit, votingPowerNeeded, ignore, count, false, false)
if err != nil {
return err
}
if success {
return nil
}
// attempt with single verification
return verifyCommitSingle(chainID, vals, commit, votingPowerNeeded,
cacheSignBytes, ignore, count, false, false)
}
// ValidateHash returns an error if the hash is not empty, but its
@ -38,3 +149,211 @@ func ValidateHash(h []byte) error {
}
return nil
}
// Batch verification
// tryVerifyCommitBatch attempts to batch verify. If it is not supported or
// verification fails it returns false. If there is an error in the signatures
// or the way that they are counted an error is returned. A cache of all the
// commits in byte form is returned in case it needs to be used again for single
// verification
func tryVerifyCommitBatch(
chainID string,
vals *ValidatorSet,
commit *Commit,
votingPowerNeeded int64,
ignoreSig func(CommitSig) bool,
countSig func(CommitSig) bool,
countAllSignatures bool,
lookUpByIndex bool,
) (map[string][]byte, bool, error) {
var (
val *Validator
valIdx int32
seenVals = make(map[int32]int, len(commit.Signatures))
talliedVotingPower int64 = 0
// we keep a cache of the signed bytes to make it quicker to verify
// individually if we need to
cacheSignBytes = make(map[string][]byte, len(commit.Signatures))
)
// attempt to create a batch verifier
bv, ok := batch.CreateBatchVerifier(vals.GetProposer().PubKey)
// check if batch verification is supported
if !ok || len(commit.Signatures) < 2 {
return cacheSignBytes, false, nil
}
for idx, commitSig := range commit.Signatures {
// skip over signatures that should be ignored
if ignoreSig(commitSig) {
continue
}
// If the vals and commit have a 1-to-1 correspondance we can retrieve
// them by index else we need to retrieve them by address
if lookUpByIndex {
val = vals.Validators[idx]
} else {
valIdx, val = vals.GetByAddress(commitSig.ValidatorAddress)
// if the signature doesn't belong to anyone in the validator set
// then we just skip over it
if val == nil {
continue
}
// because we are getting validators by address we need to make sure
// that the same validator doesn't commit twice
if firstIndex, ok := seenVals[valIdx]; ok {
secondIndex := idx
return cacheSignBytes, false, fmt.Errorf("double vote from %v (%d and %d)", val, firstIndex, secondIndex)
}
seenVals[valIdx] = idx
}
// Validate signature.
voteSignBytes := commit.VoteSignBytes(chainID, int32(idx))
// cache the signBytes in case batch verification fails
cacheSignBytes[string(val.PubKey.Bytes())] = voteSignBytes
// add the key, sig and message to the verifier
if err := bv.Add(val.PubKey, voteSignBytes, commitSig.Signature); err != nil {
return cacheSignBytes, false, err
}
// If this signature counts then add the voting power of the validator
// to the tally
if countSig(commitSig) {
talliedVotingPower += val.VotingPower
}
// if we don't need to verify all signatures and already have sufficient
// voting power we can break from batching and verify all the signatures
if !countAllSignatures && talliedVotingPower > votingPowerNeeded {
break
}
}
// ensure that we have batched together enough signatures to exceed the
// voting power needed else there is no need to even verify
if got, needed := talliedVotingPower, votingPowerNeeded; got <= needed {
return cacheSignBytes, false, ErrNotEnoughVotingPowerSigned{Got: got, Needed: needed}
}
// attempt to verify the batch. If this fails, fall back to single
// verification
if bv.Verify() {
// success
return cacheSignBytes, true, nil
}
// verification failed
return cacheSignBytes, false, nil
}
// Single Verification
// verifyCommitSingle single verifies commits.
// If a key does not support batch verification, or batch verification fails this will be used
// This method is used to check all the signatures included in a commit.
// It is used in consensus for validating a block LastCommit.
// CONTRACT: both commit and validator set should have passed validate basic
func verifyCommitSingle(
chainID string,
vals *ValidatorSet,
commit *Commit,
votingPowerNeeded int64,
cachedVals map[string][]byte,
ignoreSig func(CommitSig) bool,
countSig func(CommitSig) bool,
countAllSignatures bool,
lookUpByIndex bool,
) error {
var (
val *Validator
valIdx int32
seenVals = make(map[int32]int, len(commit.Signatures))
talliedVotingPower int64 = 0
voteSignBytes []byte
)
for idx, commitSig := range commit.Signatures {
if ignoreSig(commitSig) {
continue
}
// If the vals and commit have a 1-to-1 correspondance we can retrieve
// them by index else we need to retrieve them by address
if lookUpByIndex {
val = vals.Validators[idx]
} else {
valIdx, val = vals.GetByAddress(commitSig.ValidatorAddress)
// if the signature doesn't belong to anyone in the validator set
// then we just skip over it
if val == nil {
continue
}
// because we are getting validators by address we need to make sure
// that the same validator doesn't commit twice
if firstIndex, ok := seenVals[valIdx]; ok {
secondIndex := idx
return fmt.Errorf("double vote from %v (%d and %d)", val, firstIndex, secondIndex)
}
seenVals[valIdx] = idx
}
// Check if we have the validator in the cache
if cachedVote, ok := cachedVals[string(val.PubKey.Bytes())]; !ok {
voteSignBytes = commit.VoteSignBytes(chainID, int32(idx))
} else {
voteSignBytes = cachedVote
}
if !val.PubKey.VerifySignature(voteSignBytes, commitSig.Signature) {
return fmt.Errorf("wrong signature (#%d): %X", idx, commitSig.Signature)
}
// If this signature counts then add the voting power of the validator
// to the tally
if countSig(commitSig) {
talliedVotingPower += val.VotingPower
}
// check if we have enough signatures and can thus exit early
if !countAllSignatures && talliedVotingPower > votingPowerNeeded {
return nil
}
}
if got, needed := talliedVotingPower, votingPowerNeeded; got <= needed {
return ErrNotEnoughVotingPowerSigned{Got: got, Needed: needed}
}
return nil
}
func verifyBasicValsAndCommit(vals *ValidatorSet, commit *Commit, height int64, blockID BlockID) error {
if vals == nil {
return errors.New("nil validator set")
}
if commit == nil {
return errors.New("nil commit")
}
if vals.Size() != len(commit.Signatures) {
return NewErrInvalidCommitSignatures(vals.Size(), len(commit.Signatures))
}
// Validate Height and BlockID.
if height != commit.Height {
return NewErrInvalidCommitHeight(height, commit.Height)
}
if !blockID.Equals(commit.BlockID) {
return fmt.Errorf("invalid commit -- wrong block ID: want %v, got %v",
blockID, commit.BlockID)
}
return nil
}

+ 262
- 0
types/validation_test.go View File

@ -0,0 +1,262 @@
package types
import (
"context"
"testing"
"time"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
tmmath "github.com/tendermint/tendermint/libs/math"
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
)
// Check VerifyCommit, VerifyCommitLight and VerifyCommitLightTrusting basic
// verification.
func TestValidatorSet_VerifyCommit_All(t *testing.T) {
var (
round = int32(0)
height = int64(100)
blockID = makeBlockID([]byte("blockhash"), 1000, []byte("partshash"))
chainID = "Lalande21185"
trustLevel = tmmath.Fraction{Numerator: 2, Denominator: 3}
)
testCases := []struct {
description string
// vote chainID
chainID string
// vote blockID
blockID BlockID
valSize int
// height of the commit
height int64
// votes
blockVotes int
nilVotes int
absentVotes int
expErr bool
}{
{"good (batch verification)", chainID, blockID, 3, height, 3, 0, 0, false},
{"good (single verification)", chainID, blockID, 1, height, 1, 0, 0, false},
{"wrong signature (#0)", "EpsilonEridani", blockID, 2, height, 2, 0, 0, true},
{"wrong block ID", chainID, makeBlockIDRandom(), 2, height, 2, 0, 0, true},
{"wrong height", chainID, blockID, 1, height - 1, 1, 0, 0, true},
{"wrong set size: 4 vs 3", chainID, blockID, 4, height, 3, 0, 0, true},
{"wrong set size: 1 vs 2", chainID, blockID, 1, height, 2, 0, 0, true},
{"insufficient voting power: got 30, needed more than 66", chainID, blockID, 10, height, 3, 2, 5, true},
{"insufficient voting power: got 0, needed more than 6", chainID, blockID, 1, height, 0, 0, 1, true},
{"insufficient voting power: got 60, needed more than 60", chainID, blockID, 9, height, 6, 3, 0, true},
}
for _, tc := range testCases {
tc := tc
t.Run(tc.description, func(t *testing.T) {
_, valSet, vals := randVoteSet(tc.height, round, tmproto.PrecommitType, tc.valSize, 10)
totalVotes := tc.blockVotes + tc.absentVotes + tc.nilVotes
sigs := make([]CommitSig, totalVotes)
vi := 0
// add absent sigs first
for i := 0; i < tc.absentVotes; i++ {
sigs[vi] = NewCommitSigAbsent()
vi++
}
for i := 0; i < tc.blockVotes+tc.nilVotes; i++ {
pubKey, err := vals[vi%len(vals)].GetPubKey(context.Background())
require.NoError(t, err)
vote := &Vote{
ValidatorAddress: pubKey.Address(),
ValidatorIndex: int32(vi),
Height: tc.height,
Round: round,
Type: tmproto.PrecommitType,
BlockID: tc.blockID,
Timestamp: time.Now(),
}
if i >= tc.blockVotes {
vote.BlockID = BlockID{}
}
v := vote.ToProto()
require.NoError(t, vals[vi%len(vals)].SignVote(context.Background(), tc.chainID, v))
vote.Signature = v.Signature
sigs[vi] = vote.CommitSig()
vi++
}
commit := NewCommit(tc.height, round, tc.blockID, sigs)
err := valSet.VerifyCommit(chainID, blockID, height, commit)
if tc.expErr {
if assert.Error(t, err, "VerifyCommit") {
assert.Contains(t, err.Error(), tc.description, "VerifyCommit")
}
} else {
assert.NoError(t, err, "VerifyCommit")
}
err = valSet.VerifyCommitLight(chainID, blockID, height, commit)
if tc.expErr {
if assert.Error(t, err, "VerifyCommitLight") {
assert.Contains(t, err.Error(), tc.description, "VerifyCommitLight")
}
} else {
assert.NoError(t, err, "VerifyCommitLight")
}
// only a subsection of the tests apply to VerifyCommitLightTrusting
if totalVotes != tc.valSize || !tc.blockID.Equals(blockID) || tc.height != height {
tc.expErr = false
}
err = valSet.VerifyCommitLightTrusting(chainID, commit, trustLevel)
if tc.expErr {
if assert.Error(t, err, "VerifyCommitLightTrusting") {
assert.Contains(t, err.Error(), tc.description, "VerifyCommitLightTrusting")
}
} else {
assert.NoError(t, err, "VerifyCommitLightTrusting")
}
})
}
}
func TestValidatorSet_VerifyCommit_CheckAllSignatures(t *testing.T) {
var (
chainID = "test_chain_id"
h = int64(3)
blockID = makeBlockIDRandom()
)
voteSet, valSet, vals := randVoteSet(h, 0, tmproto.PrecommitType, 4, 10)
commit, err := makeCommit(blockID, h, 0, voteSet, vals, time.Now())
require.NoError(t, err)
require.NoError(t, valSet.VerifyCommit(chainID, blockID, h, commit))
// malleate 4th signature
vote := voteSet.GetByIndex(3)
v := vote.ToProto()
err = vals[3].SignVote(context.Background(), "CentaurusA", v)
require.NoError(t, err)
vote.Signature = v.Signature
commit.Signatures[3] = vote.CommitSig()
err = valSet.VerifyCommit(chainID, blockID, h, commit)
if assert.Error(t, err) {
assert.Contains(t, err.Error(), "wrong signature (#3)")
}
}
func TestValidatorSet_VerifyCommitLight_ReturnsAsSoonAsMajorityOfVotingPowerSigned(t *testing.T) {
var (
chainID = "test_chain_id"
h = int64(3)
blockID = makeBlockIDRandom()
)
voteSet, valSet, vals := randVoteSet(h, 0, tmproto.PrecommitType, 4, 10)
commit, err := makeCommit(blockID, h, 0, voteSet, vals, time.Now())
require.NoError(t, err)
require.NoError(t, valSet.VerifyCommit(chainID, blockID, h, commit))
// malleate 4th signature (3 signatures are enough for 2/3+)
vote := voteSet.GetByIndex(3)
v := vote.ToProto()
err = vals[3].SignVote(context.Background(), "CentaurusA", v)
require.NoError(t, err)
vote.Signature = v.Signature
commit.Signatures[3] = vote.CommitSig()
err = valSet.VerifyCommitLight(chainID, blockID, h, commit)
assert.NoError(t, err)
}
func TestValidatorSet_VerifyCommitLightTrusting_ReturnsAsSoonAsTrustLevelOfVotingPowerSigned(t *testing.T) {
var (
chainID = "test_chain_id"
h = int64(3)
blockID = makeBlockIDRandom()
)
voteSet, valSet, vals := randVoteSet(h, 0, tmproto.PrecommitType, 4, 10)
commit, err := makeCommit(blockID, h, 0, voteSet, vals, time.Now())
require.NoError(t, err)
require.NoError(t, valSet.VerifyCommit(chainID, blockID, h, commit))
// malleate 3rd signature (2 signatures are enough for 1/3+ trust level)
vote := voteSet.GetByIndex(2)
v := vote.ToProto()
err = vals[2].SignVote(context.Background(), "CentaurusA", v)
require.NoError(t, err)
vote.Signature = v.Signature
commit.Signatures[2] = vote.CommitSig()
err = valSet.VerifyCommitLightTrusting(chainID, commit, tmmath.Fraction{Numerator: 1, Denominator: 3})
assert.NoError(t, err)
}
func TestValidatorSet_VerifyCommitLightTrusting(t *testing.T) {
var (
blockID = makeBlockIDRandom()
voteSet, originalValset, vals = randVoteSet(1, 1, tmproto.PrecommitType, 6, 1)
commit, err = makeCommit(blockID, 1, 1, voteSet, vals, time.Now())
newValSet, _ = randValidatorPrivValSet(2, 1)
)
require.NoError(t, err)
testCases := []struct {
valSet *ValidatorSet
err bool
}{
// good
0: {
valSet: originalValset,
err: false,
},
// bad - no overlap between validator sets
1: {
valSet: newValSet,
err: true,
},
// good - first two are different but the rest of the same -> >1/3
2: {
valSet: NewValidatorSet(append(newValSet.Validators, originalValset.Validators...)),
err: false,
},
}
for _, tc := range testCases {
err = tc.valSet.VerifyCommitLightTrusting("test_chain_id", commit,
tmmath.Fraction{Numerator: 1, Denominator: 3})
if tc.err {
assert.Error(t, err)
} else {
assert.NoError(t, err)
}
}
}
func TestValidatorSet_VerifyCommitLightTrustingErrorsOnOverflow(t *testing.T) {
var (
blockID = makeBlockIDRandom()
voteSet, valSet, vals = randVoteSet(1, 1, tmproto.PrecommitType, 1, MaxTotalVotingPower)
commit, err = makeCommit(blockID, 1, 1, voteSet, vals, time.Now())
)
require.NoError(t, err)
err = valSet.VerifyCommitLightTrusting("test_chain_id", commit,
tmmath.Fraction{Numerator: 25, Denominator: 55})
if assert.Error(t, err) {
assert.Contains(t, err.Error(), "int64 overflow")
}
}

+ 5
- 358
types/validator_set.go View File

@ -9,7 +9,6 @@ import (
"sort"
"strings"
"github.com/tendermint/tendermint/crypto/batch"
"github.com/tendermint/tendermint/crypto/merkle"
tmmath "github.com/tendermint/tendermint/libs/math"
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
@ -653,253 +652,25 @@ func (vals *ValidatorSet) UpdateWithChangeSet(changes []*Validator) error {
return vals.updateWithChangeSet(changes, true)
}
// VerifyCommit verifies +2/3 of the set had signed the given commit.
//
// It checks all the signatures! While it's safe to exit as soon as we have
// 2/3+ signatures, doing so would impact incentivization logic in the ABCI
// application that depends on the LastCommitInfo sent in BeginBlock, which
// includes which validators signed. For instance, Gaia incentivizes proposers
// with a bonus for including more than +2/3 of the signatures.
// VerifyCommit verifies +2/3 of the set had signed the given commit and all
// other signatures are valid
func (vals *ValidatorSet) VerifyCommit(chainID string, blockID BlockID,
height int64, commit *Commit) error {
if commit == nil {
return errors.New("nil commit")
}
if vals.Size() != len(commit.Signatures) {
return NewErrInvalidCommitSignatures(vals.Size(), len(commit.Signatures))
}
// Validate Height and BlockID.
if height != commit.Height {
return NewErrInvalidCommitHeight(height, commit.Height)
}
if !blockID.Equals(commit.BlockID) {
return fmt.Errorf("invalid commit -- wrong block ID: want %v, got %v",
blockID, commit.BlockID)
}
votingPowerNeeded := vals.TotalVotingPower() * 2 / 3
var (
talliedVotingPower int64 = 0
err error
cacheSignBytes = make(map[string][]byte, len(commit.Signatures))
)
bv, ok := batch.CreateBatchVerifier(vals.GetProposer().PubKey)
if ok && len(commit.Signatures) > 1 {
for idx, commitSig := range commit.Signatures {
if commitSig.Absent() {
continue // OK, some signatures can be absent.
}
// The vals and commit have a 1-to-1 correspondance.
// This means we don't need the validator address or to do any lookup.
val := vals.Validators[idx]
// Validate signature.
voteSignBytes := commit.VoteSignBytes(chainID, int32(idx))
// cache the signBytes in case batch verification fails
cacheSignBytes[string(val.PubKey.Bytes())] = voteSignBytes
// add the key, sig and message to the verifier
if err := bv.Add(val.PubKey, voteSignBytes, commitSig.Signature); err != nil {
return err
}
// Good!
if commitSig.ForBlock() {
talliedVotingPower += val.VotingPower
}
}
if !bv.Verify() {
talliedVotingPower, err = verifyCommitSingle(chainID, vals, commit, cacheSignBytes)
if err != nil {
return err
}
}
} else {
talliedVotingPower, err = verifyCommitSingle(chainID, vals, commit, cacheSignBytes)
if err != nil {
return err
}
}
if got, needed := talliedVotingPower, votingPowerNeeded; got <= needed {
return ErrNotEnoughVotingPowerSigned{Got: got, Needed: needed}
}
return nil
return VerifyCommit(chainID, vals, blockID, height, commit)
}
// LIGHT CLIENT VERIFICATION METHODS
// VerifyCommitLight verifies +2/3 of the set had signed the given commit.
//
// This method is primarily used by the light client and does not check all the
// signatures.
func (vals *ValidatorSet) VerifyCommitLight(chainID string, blockID BlockID,
height int64, commit *Commit) error {
if commit == nil {
return errors.New("nil commit")
}
if vals.Size() != len(commit.Signatures) {
return NewErrInvalidCommitSignatures(vals.Size(), len(commit.Signatures))
}
// Validate Height and BlockID.
if height != commit.Height {
return NewErrInvalidCommitHeight(height, commit.Height)
}
if !blockID.Equals(commit.BlockID) {
return fmt.Errorf("invalid commit -- wrong block ID: want %v, got %v",
blockID, commit.BlockID)
}
talliedVotingPower := int64(0)
votingPowerNeeded := vals.TotalVotingPower() * 2 / 3
cacheSignBytes := make(map[string][]byte, len(commit.Signatures))
var err error
// need to check if batch verification is supported
// if batch is supported and the there are more than x key(s) run batch, otherwise run single.
// if batch verification fails reset tally votes to 0 and single verify until we have 2/3+
// check if the key supports batch verification
bv, ok := batch.CreateBatchVerifier(vals.GetProposer().PubKey)
if ok && len(commit.Signatures) > 1 {
for idx, commitSig := range commit.Signatures {
// No need to verify absent or nil votes.
if !commitSig.ForBlock() {
continue
}
// The vals and commit have a 1-to-1 correspondance.
// This means we don't need the validator address or to do any lookup.
val := vals.Validators[idx]
voteSignBytes := commit.VoteSignBytes(chainID, int32(idx))
cacheSignBytes[string(val.PubKey.Bytes())] = voteSignBytes
// add the key, sig and message to the verifier
if err := bv.Add(val.PubKey, voteSignBytes, commitSig.Signature); err != nil {
return err
}
talliedVotingPower += val.VotingPower
// return as soon as +2/3 of the signatures are verified
if talliedVotingPower > votingPowerNeeded {
return nil
}
}
if !bv.Verify() {
// reset talliedVotingPower to verify enough signatures to meet the 2/3+ threshold
talliedVotingPower, err = verifyCommitLightSingle(
chainID, vals, commit, votingPowerNeeded, cacheSignBytes)
if err != nil {
return err
} else if talliedVotingPower > votingPowerNeeded {
return nil
}
}
} else {
talliedVotingPower, err = verifyCommitLightSingle(
chainID, vals, commit, votingPowerNeeded, cacheSignBytes)
if err != nil {
return err
} else if talliedVotingPower > votingPowerNeeded {
return nil
}
}
return ErrNotEnoughVotingPowerSigned{Got: talliedVotingPower, Needed: votingPowerNeeded}
return VerifyCommitLight(chainID, vals, blockID, height, commit)
}
// VerifyCommitLightTrusting verifies that trustLevel of the validator set signed
// this commit.
//
// NOTE the given validators do not necessarily correspond to the validator set
// for this commit, but there may be some intersection.
//
// This method is primarily used by the light client and does not check all the
// signatures.
func (vals *ValidatorSet) VerifyCommitLightTrusting(chainID string, commit *Commit, trustLevel tmmath.Fraction) error {
// sanity checks
if trustLevel.Denominator == 0 {
return errors.New("trustLevel has zero Denominator")
}
if commit == nil {
return errors.New("nil commit")
}
var (
talliedVotingPower int64
seenVals = make(map[int32]int, len(commit.Signatures)) // validator index -> commit index
err error
cacheSignBytes = make(map[string][]byte, len(commit.Signatures))
)
// Safely calculate voting power needed.
totalVotingPowerMulByNumerator, overflow := safeMul(vals.TotalVotingPower(), int64(trustLevel.Numerator))
if overflow {
return errors.New("int64 overflow while calculating voting power needed. please provide smaller trustLevel numerator")
}
votingPowerNeeded := totalVotingPowerMulByNumerator / int64(trustLevel.Denominator)
bv, ok := batch.CreateBatchVerifier(vals.GetProposer().PubKey)
if ok && len(commit.Signatures) > 1 {
for idx, commitSig := range commit.Signatures {
// No need to verify absent or nil votes.
if !commitSig.ForBlock() {
continue
}
// We don't know the validators that committed this block, so we have to
// check for each vote if its validator is already known.
valIdx, val := vals.GetByAddress(commitSig.ValidatorAddress)
if val != nil {
// check for double vote of validator on the same commit
if firstIndex, ok := seenVals[valIdx]; ok {
secondIndex := idx
return fmt.Errorf("double vote from %v (%d and %d)", val, firstIndex, secondIndex)
}
seenVals[valIdx] = idx
// Validate signature.
voteSignBytes := commit.VoteSignBytes(chainID, int32(idx))
// cache the signed bytes in case we fail verification
cacheSignBytes[string(val.PubKey.Bytes())] = voteSignBytes
// if batch verification is supported add the key, sig and message to the verifier
if err := bv.Add(val.PubKey, voteSignBytes, commitSig.Signature); err != nil {
return err
}
talliedVotingPower += val.VotingPower
if talliedVotingPower > votingPowerNeeded {
return nil
}
}
}
if !bv.Verify() {
talliedVotingPower, err = verifyCommitLightTrustingSingle(
chainID, vals, commit, votingPowerNeeded, cacheSignBytes)
if err != nil {
return err
} else if talliedVotingPower > votingPowerNeeded {
return nil
}
}
} else {
talliedVotingPower, err = verifyCommitLightTrustingSingle(
chainID, vals, commit, votingPowerNeeded, cacheSignBytes)
if err != nil {
return err
} else if talliedVotingPower > votingPowerNeeded {
return nil
}
}
return ErrNotEnoughVotingPowerSigned{Got: talliedVotingPower, Needed: votingPowerNeeded}
return VerifyCommitLightTrusting(chainID, vals, commit, trustLevel)
}
// findPreviousProposer reverses the compare proposer priority function to find the validator
@ -1160,127 +931,3 @@ func safeMul(a, b int64) (int64, bool) {
return a * b, false
}
// verifyCommitLightTrustingSingle single verifies commits
// If a key does not support batch verification, or batch verification fails this will be used
// This method is used for light clients, it only checks 2/3+ of the signatures
func verifyCommitLightTrustingSingle(
chainID string, vals *ValidatorSet, commit *Commit, votingPowerNeeded int64,
cachedVals map[string][]byte) (int64, error) {
var (
seenVals = make(map[int32]int, len(commit.Signatures))
talliedVotingPower int64 = 0
)
for idx, commitSig := range commit.Signatures {
// No need to verify absent or nil votes.
if !commitSig.ForBlock() {
continue
}
var voteSignBytes []byte
// We don't know the validators that committed this block, so we have to
// check for each vote if its validator is already known.
valIdx, val := vals.GetByAddress(commitSig.ValidatorAddress)
if val != nil {
// check for double vote of validator on the same commit
if firstIndex, ok := seenVals[valIdx]; ok {
secondIndex := idx
return 0, fmt.Errorf("double vote from %v (%d and %d)", val, firstIndex, secondIndex)
}
seenVals[valIdx] = idx
// Validate signature.
// voteSignBytes := commit.VoteSignBytes(chainID, int32(idx))
if val, ok := cachedVals[string(val.PubKey.Bytes())]; !ok {
voteSignBytes = commit.VoteSignBytes(chainID, int32(idx))
} else {
voteSignBytes = val
}
if !val.PubKey.VerifySignature(voteSignBytes, commitSig.Signature) {
return 0, fmt.Errorf("wrong signature (#%d): %X", idx, commitSig.Signature)
}
talliedVotingPower += val.VotingPower
if talliedVotingPower > votingPowerNeeded {
return talliedVotingPower, nil
}
}
}
return talliedVotingPower, nil
}
// verifyCommitLightSingle single verifies commits.
// If a key does not support batch verification, or batch verification fails this will be used
// This method is used for light client and block sync verification, it will only check 2/3+ signatures
func verifyCommitLightSingle(
chainID string, vals *ValidatorSet, commit *Commit, votingPowerNeeded int64,
cachedVals map[string][]byte) (int64, error) {
var talliedVotingPower int64 = 0
for idx, commitSig := range commit.Signatures {
// No need to verify absent or nil votes.
if !commitSig.ForBlock() {
continue
}
// The vals and commit have a 1-to-1 correspondance.
// This means we don't need the validator address or to do any lookup.
var voteSignBytes []byte
val := vals.Validators[idx]
// Check if we have the validator in the cache
if val, ok := cachedVals[string(val.PubKey.Bytes())]; !ok {
voteSignBytes = commit.VoteSignBytes(chainID, int32(idx))
} else {
voteSignBytes = val
}
// Validate signature.
if !val.PubKey.VerifySignature(voteSignBytes, commitSig.Signature) {
return 0, fmt.Errorf("wrong signature (#%d): %X", idx, commitSig.Signature)
}
talliedVotingPower += val.VotingPower
// return as soon as +2/3 of the signatures are verified
if talliedVotingPower > votingPowerNeeded {
return talliedVotingPower, nil
}
}
return talliedVotingPower, nil
}
// verifyCommitSingle single verifies commits.
// If a key does not support batch verification, or batch verification fails this will be used
// This method is used to check all the signatures included in a commit.
// It is used in consensus for validating a block LastCommit.
func verifyCommitSingle(chainID string, vals *ValidatorSet, commit *Commit,
cachedVals map[string][]byte) (int64, error) {
var talliedVotingPower int64 = 0
for idx, commitSig := range commit.Signatures {
if commitSig.Absent() {
continue // OK, some signatures can be absent.
}
var voteSignBytes []byte
val := vals.Validators[idx]
// Check if we have the validator in the cache
if val, ok := cachedVals[string(val.PubKey.Bytes())]; !ok {
voteSignBytes = commit.VoteSignBytes(chainID, int32(idx))
} else {
voteSignBytes = val
}
if !val.PubKey.VerifySignature(voteSignBytes, commitSig.Signature) {
return talliedVotingPower, fmt.Errorf("wrong signature (#%d): %X", idx, commitSig.Signature)
}
// Good!
if commitSig.ForBlock() {
talliedVotingPower += val.VotingPower
}
}
return talliedVotingPower, nil
}

+ 0
- 205
types/validator_set_test.go View File

@ -679,155 +679,6 @@ func TestSafeSubClip(t *testing.T) {
//-------------------------------------------------------------------
// Check VerifyCommit, VerifyCommitLight and VerifyCommitLightTrusting basic
// verification.
func TestValidatorSet_VerifyCommit_All(t *testing.T) {
var (
privKey = ed25519.GenPrivKey()
pubKey = privKey.PubKey()
v1 = NewValidator(pubKey, 1000)
vset = NewValidatorSet([]*Validator{v1})
chainID = "Lalande21185"
)
vote := examplePrecommit()
vote.ValidatorAddress = pubKey.Address()
v := vote.ToProto()
sig, err := privKey.Sign(VoteSignBytes(chainID, v))
require.NoError(t, err)
vote.Signature = sig
commit := NewCommit(vote.Height, vote.Round, vote.BlockID, []CommitSig{vote.CommitSig()})
vote2 := *vote
sig2, err := privKey.Sign(VoteSignBytes("EpsilonEridani", v))
require.NoError(t, err)
vote2.Signature = sig2
testCases := []struct {
description string
chainID string
blockID BlockID
height int64
commit *Commit
expErr bool
}{
{"good", chainID, vote.BlockID, vote.Height, commit, false},
{"wrong signature (#0)", "EpsilonEridani", vote.BlockID, vote.Height, commit, true},
{"wrong block ID", chainID, makeBlockIDRandom(), vote.Height, commit, true},
{"wrong height", chainID, vote.BlockID, vote.Height - 1, commit, true},
{"wrong set size: 1 vs 0", chainID, vote.BlockID, vote.Height,
NewCommit(vote.Height, vote.Round, vote.BlockID, []CommitSig{}), true},
{"wrong set size: 1 vs 2", chainID, vote.BlockID, vote.Height,
NewCommit(vote.Height, vote.Round, vote.BlockID,
[]CommitSig{vote.CommitSig(), {BlockIDFlag: BlockIDFlagAbsent}}), true},
{"insufficient voting power: got 0, needed more than 666", chainID, vote.BlockID, vote.Height,
NewCommit(vote.Height, vote.Round, vote.BlockID, []CommitSig{{BlockIDFlag: BlockIDFlagAbsent}}), true},
{"wrong signature (#0)", chainID, vote.BlockID, vote.Height,
NewCommit(vote.Height, vote.Round, vote.BlockID, []CommitSig{vote2.CommitSig()}), true},
}
for _, tc := range testCases {
tc := tc
t.Run(tc.description, func(t *testing.T) {
err := vset.VerifyCommit(tc.chainID, tc.blockID, tc.height, tc.commit)
if tc.expErr {
if assert.Error(t, err, "VerifyCommit") {
assert.Contains(t, err.Error(), tc.description, "VerifyCommit")
}
} else {
assert.NoError(t, err, "VerifyCommit")
}
err = vset.VerifyCommitLight(tc.chainID, tc.blockID, tc.height, tc.commit)
if tc.expErr {
if assert.Error(t, err, "VerifyCommitLight") {
assert.Contains(t, err.Error(), tc.description, "VerifyCommitLight")
}
} else {
assert.NoError(t, err, "VerifyCommitLight")
}
})
}
}
func TestValidatorSet_VerifyCommit_CheckAllSignatures(t *testing.T) {
var (
chainID = "test_chain_id"
h = int64(3)
blockID = makeBlockIDRandom()
)
voteSet, valSet, vals := randVoteSet(h, 0, tmproto.PrecommitType, 4, 10)
commit, err := makeCommit(blockID, h, 0, voteSet, vals, time.Now())
require.NoError(t, err)
// malleate 4th signature
vote := voteSet.GetByIndex(3)
v := vote.ToProto()
err = vals[3].SignVote(context.Background(), "CentaurusA", v)
require.NoError(t, err)
vote.Signature = v.Signature
commit.Signatures[3] = vote.CommitSig()
err = valSet.VerifyCommit(chainID, blockID, h, commit)
if assert.Error(t, err) {
assert.Contains(t, err.Error(), "wrong signature (#3)")
}
}
func TestValidatorSet_VerifyCommitLight_ReturnsAsSoonAsMajorityOfVotingPowerSigned(t *testing.T) {
var (
chainID = "test_chain_id"
h = int64(3)
blockID = makeBlockIDRandom()
)
voteSet, valSet, vals := randVoteSet(h, 0, tmproto.PrecommitType, 4, 10)
commit, err := makeCommit(blockID, h, 0, voteSet, vals, time.Now())
require.NoError(t, err)
// malleate 4th signature (3 signatures are enough for 2/3+)
vote := voteSet.GetByIndex(3)
v := vote.ToProto()
err = vals[3].SignVote(context.Background(), "CentaurusA", v)
require.NoError(t, err)
vote.Signature = v.Signature
commit.Signatures[3] = vote.CommitSig()
err = valSet.VerifyCommitLight(chainID, blockID, h, commit)
assert.NoError(t, err)
}
func TestValidatorSet_VerifyCommitLightTrusting_ReturnsAsSoonAsTrustLevelOfVotingPowerSigned(t *testing.T) {
var (
chainID = "test_chain_id"
h = int64(3)
blockID = makeBlockIDRandom()
)
voteSet, valSet, vals := randVoteSet(h, 0, tmproto.PrecommitType, 4, 10)
commit, err := makeCommit(blockID, h, 0, voteSet, vals, time.Now())
require.NoError(t, err)
// malleate 3rd signature (2 signatures are enough for 1/3+ trust level)
vote := voteSet.GetByIndex(2)
v := vote.ToProto()
err = vals[2].SignVote(context.Background(), "CentaurusA", v)
require.NoError(t, err)
vote.Signature = v.Signature
commit.Signatures[2] = vote.CommitSig()
err = valSet.VerifyCommitLightTrusting(chainID, commit, tmmath.Fraction{Numerator: 1, Denominator: 3})
assert.NoError(t, err)
}
func TestEmptySet(t *testing.T) {
var valList []*Validator
@ -1531,62 +1382,6 @@ func TestValSetUpdateOverflowRelated(t *testing.T) {
}
}
func TestValidatorSet_VerifyCommitLightTrusting(t *testing.T) {
var (
blockID = makeBlockIDRandom()
voteSet, originalValset, vals = randVoteSet(1, 1, tmproto.PrecommitType, 6, 1)
commit, err = makeCommit(blockID, 1, 1, voteSet, vals, time.Now())
newValSet, _ = randValidatorPrivValSet(2, 1)
)
require.NoError(t, err)
testCases := []struct {
valSet *ValidatorSet
err bool
}{
// good
0: {
valSet: originalValset,
err: false,
},
// bad - no overlap between validator sets
1: {
valSet: newValSet,
err: true,
},
// good - first two are different but the rest of the same -> >1/3
2: {
valSet: NewValidatorSet(append(newValSet.Validators, originalValset.Validators...)),
err: false,
},
}
for _, tc := range testCases {
err = tc.valSet.VerifyCommitLightTrusting("test_chain_id", commit,
tmmath.Fraction{Numerator: 1, Denominator: 3})
if tc.err {
assert.Error(t, err)
} else {
assert.NoError(t, err)
}
}
}
func TestValidatorSet_VerifyCommitLightTrustingErrorsOnOverflow(t *testing.T) {
var (
blockID = makeBlockIDRandom()
voteSet, valSet, vals = randVoteSet(1, 1, tmproto.PrecommitType, 1, MaxTotalVotingPower)
commit, err = makeCommit(blockID, 1, 1, voteSet, vals, time.Now())
)
require.NoError(t, err)
err = valSet.VerifyCommitLightTrusting("test_chain_id", commit,
tmmath.Fraction{Numerator: 25, Denominator: 55})
if assert.Error(t, err) {
assert.Contains(t, err.Error(), "int64 overflow")
}
}
func TestSafeMul(t *testing.T) {
testCases := []struct {
a int64


Loading…
Cancel
Save