Compare commits

...

3 Commits

Author SHA1 Message Date
Callum Waters
7296f8a149 Merge branch 'master' into callum/evidence-hashes 2021-04-12 18:26:28 +02:00
Callum Waters
0dc607972c add more evidence testing 2021-04-12 18:20:02 +02:00
Callum Waters
6c095dcdb8 modify evidence hashes 2021-04-09 23:25:30 +02:00
9 changed files with 222 additions and 86 deletions

View File

@@ -21,12 +21,6 @@ import (
"github.com/tendermint/tendermint/types"
)
const (
// prefixes are unique across all tm db's
prefixCommitted = int64(9)
prefixPending = int64(10)
)
// Pool maintains a pool of valid evidence to be broadcasted and committed
type Pool struct {
logger log.Logger
@@ -152,6 +146,11 @@ func (evpool *Pool) AddEvidence(ev types.Evidence) error {
return nil
}
if evpool.checkForSimilarLightClientAttackEvidence(ev) {
evpool.logger.Debug("similar light client evidence already pending; ignoring", "evidence", ev)
return nil
}
// check that the evidence isn't already committed
if evpool.isCommitted(ev) {
// This can happen if the peer that sent us the evidence is behind so we
@@ -203,7 +202,7 @@ func (evpool *Pool) CheckEvidence(evList types.EvidenceList) error {
hashes := make([][]byte, len(evList))
for idx, ev := range evList {
ok := evpool.fastCheck(ev)
ok := evpool.isPending(ev)
if !ok {
// check that the evidence isn't already committed
@@ -382,6 +381,15 @@ func (evpool *Pool) addPendingEvidence(ev types.Evidence) error {
return fmt.Errorf("failed to persist evidence: %w", err)
}
// if it is light client attack evidence, adds a secondary key to avoid
// submission of multiple variants
if lcae, ok := ev.(*types.LightClientAttackEvidence); ok {
if err := evpool.evidenceStore.Set(keyLightEvidence(lcae), keyPending(ev)); err != nil {
return fmt.Errorf("failed to persist secondary key for light evidence: %w",
err)
}
}
atomic.AddUint32(&evpool.evidenceSize, 1)
return nil
}
@@ -396,7 +404,12 @@ func (evpool *Pool) markEvidenceAsCommitted(evidence types.EvidenceList, height
for _, ev := range evidence {
if evpool.isPending(ev) {
if err := batch.Delete(keyPending(ev)); err != nil {
evpool.logger.Error("failed to batch pending evidence", "err", err)
evpool.logger.Error("failed to batch delete pending evidence", "err", err)
}
if lcae, ok := ev.(*types.LightClientAttackEvidence); ok {
if err := batch.Delete(keyLightEvidence(lcae)); err != nil {
evpool.logger.Error("failed to batch delete pending evidence", "err", err)
}
}
blockEvidenceMap[evMapKey(ev)] = struct{}{}
}
@@ -546,10 +559,18 @@ func (evpool *Pool) batchExpiredPendingEvidence(batch dbm.Batch) (int64, time.Ti
// else add to the batch
if err := batch.Delete(iter.Key()); err != nil {
evpool.logger.Error("failed to batch evidence", "err", err, "ev", ev)
evpool.logger.Error("failed to batch delete evidence", "err", err, "ev", ev)
continue
}
// if it is light client attack evidence then delete the secondary index
if lcae, ok := ev.(*types.LightClientAttackEvidence); ok {
if err := batch.Delete(keyLightEvidence(lcae)); err != nil {
evpool.logger.Error("failed to batch delete evidence", "err", err, "ev", ev)
continue
}
}
// and add to the map to remove the evidence from the clist
blockEvidenceMap[evMapKey(ev)] = struct{}{}
}
@@ -651,6 +672,17 @@ func (evpool *Pool) processConsensusBuffer(state sm.State) {
evpool.consensusBuffer = make([]duplicateVoteSet, 0)
}
func (evpool *Pool) checkForSimilarLightClientAttackEvidence(ev types.Evidence) bool {
if lcae, ok := ev.(*types.LightClientAttackEvidence); ok {
ok, err := evpool.evidenceStore.Has(keyLightEvidence(lcae))
if err != nil {
evpool.logger.Error("failed to find pending evidence", "err", err)
}
return ok
}
return false
}
type duplicateVoteSet struct {
VoteA *types.Vote
VoteB *types.Vote
@@ -670,6 +702,24 @@ func evMapKey(ev types.Evidence) string {
return string(ev.Hash())
}
// ########################### KEYS ##############################
const (
// prefixes are unique across all tm db's
prefixCommitted = int64(9)
prefixPending = int64(10)
// It is very easy to manipulate LightClientAttackEvidence to
// form multiple valid versions of that evidence that all have
// different hashes i.e. change the common height or remove a
// commit. This is a potential DOS vector as this evidence is
// relatively large and a malicious node could freely fill
// blocks with it. To prevent this nodes won't verify a new
// LightClientAttackEvidence that has the same conflicting
// header
prefixLightEvidence = int64(13)
)
func prefixToBytes(prefix int64) []byte {
key, err := orderedcode.Append(nil, prefix)
if err != nil {
@@ -680,7 +730,15 @@ func prefixToBytes(prefix int64) []byte {
func keyCommitted(evidence types.Evidence) []byte {
var height int64 = evidence.Height()
key, err := orderedcode.Append(nil, prefixCommitted, height, string(evidence.Hash()))
var hash string
// if it is light client attack evidence we add the header hash key to avoid
// submission of multiple variants
if lcae, ok := evidence.(*types.LightClientAttackEvidence); ok {
hash = string(lcae.ConflictingBlock.Header.Hash())
} else {
hash = string(evidence.Hash())
}
key, err := orderedcode.Append(nil, prefixCommitted, height, hash)
if err != nil {
panic(err)
}
@@ -695,3 +753,12 @@ func keyPending(evidence types.Evidence) []byte {
}
return key
}
func keyLightEvidence(evidence *types.LightClientAttackEvidence) []byte {
key, err := orderedcode.Append(nil, prefixLightEvidence,
string(evidence.ConflictingBlock.Header.Hash()))
if err != nil {
panic(err)
}
return key
}

View File

@@ -338,8 +338,30 @@ func TestCheckEvidenceWithLightClientAttack(t *testing.T) {
// Take away the last signature -> there are less validators then what we have detected,
// hence this should fail.
commit.Signatures = append(commit.Signatures[:nValidators-1], types.NewCommitSigAbsent())
require.Error(t, pool.CheckEvidence(types.EvidenceList{ev}))
newSigs := append(commit.Signatures[:nValidators-2], types.NewCommitSigAbsent(), types.NewCommitSigAbsent())
newCommit := types.NewCommit(commit.Height, commit.Round, commit.BlockID, newSigs)
differentEv := &types.LightClientAttackEvidence{
ConflictingBlock: &types.LightBlock{
SignedHeader: &types.SignedHeader{
Header: conflictingHeader,
Commit: newCommit,
},
ValidatorSet: conflictingVals,
},
CommonHeight: 10,
TotalVotingPower: int64(nValidators) * validatorPower,
ByzantineValidators: conflictingVals.Validators,
Timestamp: defaultEvidenceTime,
}
require.Error(t, pool.CheckEvidence(types.EvidenceList{differentEv}))
state.LastBlockHeight++
pool.Update(state, types.EvidenceList{ev})
require.NoError(t, pool.AddEvidence(differentEv))
require.NoError(t, pool.AddEvidence(ev))
pendingEv, _ := pool.PendingEvidence(state.ConsensusParams.Evidence.MaxBytes)
require.Empty(t, pendingEv)
}
// Tests that restarting the evidence pool after a potential failure will recover the

View File

@@ -1,3 +1,5 @@
evidence = 5
[node.validator01]
[node.validator02]
[node.validator03]

View File

@@ -20,10 +20,7 @@ import (
)
// 1 in 11 evidence is light client evidence, the rest is duplicate vote
// FIXME: Setting to 11 disables light client attack evidence since nodes
// don't follow a minimum retention height invariant. When we fix this we
// should use a ratio of 4.
const lightClientEvidenceRatio = 11
const lightClientEvidenceRatio = 4
// InjectEvidence takes a running testnet and generates an amount of valid
// evidence and broadcasts it to a random node through the rpc endpoint `/broadcast_evidence`.
@@ -74,7 +71,7 @@ func InjectEvidence(testnet *e2e.Testnet, amount int) error {
duplicateVoteTime := status.SyncInfo.LatestBlockTime
var ev types.Evidence
for i := 0; i < amount; i++ {
for i := 1; i <= amount; i++ {
if i%lightClientEvidenceRatio == 0 {
ev, err = generateLightClientAttackEvidence(
privVals, lightEvidenceCommonHeight, valSet, testnet.Name, blockRes.Block.Time,

View File

@@ -38,7 +38,10 @@ func TestBlock_Header(t *testing.T) {
resp, err := client.Block(ctx, &block.Header.Height)
require.NoError(t, err)
require.Equal(t, block, resp.Block,
"block mismatch for height %v", block.Header.Height)
"block mismatch for height %d", block.Header.Height)
require.NoError(t, resp.Block.ValidateBasic(),
"block at height %d is invalid", block.Header.Height)
}
})
}

View File

@@ -292,18 +292,20 @@ func (l *LightClientAttackEvidence) ConflictingHeaderIsInvalid(trustedHeader *He
}
// 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.
// Hash returns the SHA256 hash of the header, commit, common height, total
// voting power, byzantine validators and timestamp
func (l *LightClientAttackEvidence) Hash() []byte {
buf := make([]byte, binary.MaxVarintLen64)
n := binary.PutVarint(buf, l.CommonHeight)
bz := make([]byte, tmhash.Size+n)
copy(bz[:tmhash.Size-1], l.ConflictingBlock.Hash().Bytes())
copy(bz[tmhash.Size:], buf)
return tmhash.Sum(bz)
buf := new(bytes.Buffer)
buf.Write(l.ConflictingBlock.Header.Hash())
buf.Write(l.ConflictingBlock.Commit.Hash())
_ = binary.Write(buf, binary.LittleEndian, l.CommonHeight)
_ = binary.Write(buf, binary.LittleEndian, l.TotalVotingPower)
for _, val := range l.ByzantineValidators {
_, _ = buf.Write(val.Bytes())
}
timeBytes, _ := l.Timestamp.MarshalBinary()
buf.Write(timeBytes)
return tmhash.Sum(buf.Bytes())
}
// Height returns the last height at which the primary provider and witness provider had the same header.
@@ -315,8 +317,14 @@ func (l *LightClientAttackEvidence) Height() int64 {
// 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)
return fmt.Sprintf(`LightClientAttackEvidence{
ConflictingBlock: %v,
CommonHeight: %d,
ByzatineValidators: %v,
TotalVotingPower: %d,
Timestamp: %v}#%X`,
l.ConflictingBlock.String(), l.CommonHeight, l.ByzantineValidators,
l.TotalVotingPower, l.Timestamp, l.Hash())
}
// Time returns the time of the common block where the infraction leveraged off.
@@ -335,20 +343,24 @@ func (l *LightClientAttackEvidence) ValidateBasic() error {
return errors.New("conflicting block missing header")
}
if err := l.ConflictingBlock.ValidateBasic(l.ConflictingBlock.ChainID); err != nil {
return fmt.Errorf("invalid conflicting light block: %w", err)
if l.TotalVotingPower <= 0 {
return errors.New("negative or zero total voting power")
}
if l.CommonHeight <= 0 {
return errors.New("negative or zero common height")
}
// check that common height isn't ahead of the height of the conflicting block. It
// is possible that they are the same height if the light node witnesses either an
// amnesia or a equivocation attack.
if l.CommonHeight > l.ConflictingBlock.Height {
return fmt.Errorf("common height is ahead of the conflicting block height (%d > %d)",
l.CommonHeight, l.ConflictingBlock.Height)
l.CommonHeight, l.ConflictingBlock.Height)
}
if err := l.ConflictingBlock.ValidateBasic(l.ConflictingBlock.ChainID); err != nil {
return fmt.Errorf("invalid conflicting light block: %w", err)
}
return nil
@@ -422,7 +434,7 @@ func (evl EvidenceList) Hash() []byte {
// the Evidence size is capped.
evidenceBzs := make([][]byte, len(evl))
for i := 0; i < len(evl); i++ {
evidenceBzs[i] = evl[i].Bytes()
evidenceBzs[i] = evl[i].Hash()
}
return merkle.HashFromByteSlices(evidenceBzs)
}

View File

@@ -89,9 +89,11 @@ func TestDuplicateVoteEvidenceValidation(t *testing.T) {
}
}
func TestLightClientAttackEvidence(t *testing.T) {
func TestLightClientAttackEvidenceBasic(t *testing.T) {
height := int64(5)
voteSet, valSet, privVals := randVoteSet(height, 1, tmproto.PrecommitType, 10, 1)
commonHeight := height - 1
nValidators := 10
voteSet, valSet, privVals := randVoteSet(height, 1, tmproto.PrecommitType, nValidators, 1)
header := makeHeaderRandom()
header.Height = height
blockID := makeBlockID(tmhash.Sum([]byte("blockhash")), math.MaxInt32, tmhash.Sum([]byte("partshash")))
@@ -105,56 +107,64 @@ func TestLightClientAttackEvidence(t *testing.T) {
},
ValidatorSet: valSet,
},
CommonHeight: height - 1,
CommonHeight: commonHeight,
TotalVotingPower: valSet.TotalVotingPower(),
Timestamp: header.Time,
ByzantineValidators: valSet.Validators[:nValidators/2],
}
assert.NotNil(t, lcae.String())
assert.NotNil(t, lcae.Hash())
// only 7 validators sign
differentCommit, err := MakeCommit(blockID, height, 1, voteSet, privVals[:7], defaultVoteTime)
require.NoError(t, err)
differentEv := &LightClientAttackEvidence{
ConflictingBlock: &LightBlock{
SignedHeader: &SignedHeader{
Header: header,
Commit: differentCommit,
},
ValidatorSet: valSet,
},
CommonHeight: height - 1,
}
assert.Equal(t, lcae.Hash(), differentEv.Hash())
// different header hash
differentHeader := makeHeaderRandom()
differentEv = &LightClientAttackEvidence{
ConflictingBlock: &LightBlock{
SignedHeader: &SignedHeader{
Header: differentHeader,
Commit: differentCommit,
},
ValidatorSet: valSet,
},
CommonHeight: height - 1,
}
assert.NotEqual(t, lcae.Hash(), differentEv.Hash())
// different common height should produce a different header
differentEv = &LightClientAttackEvidence{
ConflictingBlock: &LightBlock{
SignedHeader: &SignedHeader{
Header: header,
Commit: differentCommit,
},
ValidatorSet: valSet,
},
CommonHeight: height - 2,
}
assert.NotEqual(t, lcae.Hash(), differentEv.Hash())
assert.Equal(t, lcae.Height(), int64(4)) // Height should be the common Height
assert.Equal(t, lcae.Height(), commonHeight) // Height should be the common Height
assert.NotNil(t, lcae.Bytes())
// maleate evidence to test hash uniqueness
testCases := []struct {
testName string
malleateEvidence func(*LightClientAttackEvidence)
}{
{"Different header", func(ev *LightClientAttackEvidence) { ev.ConflictingBlock.Header = makeHeaderRandom() }},
{"Different commit", func(ev *LightClientAttackEvidence) {
newSigs := append(commit.Signatures[:len(commit.Signatures)-1], NewCommitSigAbsent())
newCommit := NewCommit(height, 0, blockID, newSigs)
require.NotEqual(t, commit.Hash(), newCommit.Hash())
ev.ConflictingBlock.Commit = newCommit
}},
{"Different common height", func(ev *LightClientAttackEvidence) {
ev.CommonHeight = height + 1
}},
{"Different total voting power", func(ev *LightClientAttackEvidence) { ev.TotalVotingPower *= 2 }},
{"Different timestamp", func(ev *LightClientAttackEvidence) { ev.Timestamp = header.Time.Add(1 * time.Hour) }},
{"Different byzantine validators", func(ev *LightClientAttackEvidence) {
ev.ByzantineValidators = []*Validator{}
}},
}
for _, tc := range testCases {
lcae := &LightClientAttackEvidence{
ConflictingBlock: &LightBlock{
SignedHeader: &SignedHeader{
Header: header,
Commit: commit,
},
ValidatorSet: valSet,
},
CommonHeight: commonHeight,
TotalVotingPower: valSet.TotalVotingPower(),
Timestamp: header.Time,
ByzantineValidators: valSet.Validators[:nValidators/2],
}
hash := lcae.Hash()
t.Log(hash)
tc.malleateEvidence(lcae)
assert.NotEqual(t, hash, lcae.Hash(), tc.testName)
}
}
func TestLightClientAttackEvidenceValidation(t *testing.T) {
height := int64(5)
voteSet, valSet, privVals := randVoteSet(height, 1, tmproto.PrecommitType, 10, 1)
commonHeight := height - 1
nValidators := 10
voteSet, valSet, privVals := randVoteSet(height, 1, tmproto.PrecommitType, nValidators, 1)
header := makeHeaderRandom()
header.Height = height
header.ValidatorsHash = valSet.Hash()
@@ -169,7 +179,10 @@ func TestLightClientAttackEvidenceValidation(t *testing.T) {
},
ValidatorSet: valSet,
},
CommonHeight: height - 1,
CommonHeight: commonHeight,
TotalVotingPower: valSet.TotalVotingPower(),
Timestamp: header.Time,
ByzantineValidators: valSet.Validators[:nValidators/2],
}
assert.NoError(t, lcae.ValidateBasic())
@@ -178,16 +191,22 @@ func TestLightClientAttackEvidenceValidation(t *testing.T) {
malleateEvidence func(*LightClientAttackEvidence)
expectErr bool
}{
{"Good DuplicateVoteEvidence", func(ev *LightClientAttackEvidence) {}, false},
{"Good LightClientAttackEvidence", func(ev *LightClientAttackEvidence) {}, false},
{"Negative height", func(ev *LightClientAttackEvidence) { ev.CommonHeight = -10 }, true},
{"Height is greater than divergent block", func(ev *LightClientAttackEvidence) {
ev.CommonHeight = height + 1
}, true},
{"Height is equal to the divergent block", func(ev *LightClientAttackEvidence) {
ev.CommonHeight = height
}, false},
{"Nil conflicting header", func(ev *LightClientAttackEvidence) { ev.ConflictingBlock.Header = nil }, true},
{"Nil conflicting blocl", func(ev *LightClientAttackEvidence) { ev.ConflictingBlock = nil }, true},
{"Nil validator set", func(ev *LightClientAttackEvidence) {
ev.ConflictingBlock.ValidatorSet = &ValidatorSet{}
}, true},
{"Negative total voting power", func(ev *LightClientAttackEvidence) {
ev.TotalVotingPower = -1
}, true},
}
for _, tc := range testCases {
tc := tc
@@ -200,7 +219,10 @@ func TestLightClientAttackEvidenceValidation(t *testing.T) {
},
ValidatorSet: valSet,
},
CommonHeight: height - 1,
CommonHeight: commonHeight,
TotalVotingPower: valSet.TotalVotingPower(),
Timestamp: header.Time,
ByzantineValidators: valSet.Validators[:nValidators/2],
}
tc.malleateEvidence(lcae)
if tc.expectErr {

View File

@@ -5,6 +5,7 @@ import (
"errors"
"fmt"
"github.com/tendermint/tendermint/crypto/tmhash"
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
)
@@ -149,13 +150,21 @@ func (sh SignedHeader) ValidateBasic(chainID string) error {
if sh.Commit.Height != sh.Height {
return fmt.Errorf("header and commit height mismatch: %d vs %d", sh.Height, sh.Commit.Height)
}
if hhash, chash := sh.Hash(), sh.Commit.BlockID.Hash; !bytes.Equal(hhash, chash) {
if hhash, chash := sh.Header.Hash(), sh.Commit.BlockID.Hash; !bytes.Equal(hhash, chash) {
return fmt.Errorf("commit signs block %X, header is block %X", chash, hhash)
}
return nil
}
// Hash returns the SHA256 hash of both the header and the commit that signed that header
func (sh SignedHeader) Hash() []byte {
bz := make([]byte, tmhash.Size*2)
copy(bz[:tmhash.Size-1], sh.Header.Hash())
copy(bz[tmhash.Size:], sh.Commit.Hash())
return tmhash.Sum(bz)
}
// String returns a string representation of SignedHeader.
func (sh SignedHeader) String() string {
return sh.StringIndented("")

View File

@@ -152,11 +152,13 @@ func TestSignedHeaderValidateBasic(t *testing.T) {
Header: tc.shHeader,
Commit: tc.shCommit,
}
assert.Equal(
err := sh.ValidateBasic(validSignedHeader.Header.ChainID)
assert.Equalf(
t,
tc.expectErr,
sh.ValidateBasic(validSignedHeader.Header.ChainID) != nil,
err != nil,
"Validate Basic had an unexpected result",
err,
)
})
}