Compare commits

...

21 Commits

Author SHA1 Message Date
William Banfield
c627a1d762 Revert "rename loadblockextendedcommit -> loadextendedcommit"
This reverts commit 8935da8872.
2022-05-13 15:31:23 -04:00
William Banfield
5dbb706e2c add boolean parameter to vote set 2022-05-13 15:12:06 -04:00
William Banfield
56b5a234bf fix state tests to not check for own extension 2022-05-13 15:11:54 -04:00
William Banfield
c3f64df354 reconstruct last commit works with legacy chains 2022-05-13 14:05:01 -04:00
William Banfield
ec46cc4006 fix buildExtendedCommitInfo logic to allow default 2022-05-13 13:49:00 -04:00
William Banfield
8935da8872 rename loadblockextendedcommit -> loadextendedcommit 2022-05-13 12:53:12 -04:00
William Banfield
ab83d3307d validation shimmed into place for switch to consensus 2022-05-12 18:00:53 -04:00
William Banfield
636cd97712 consolidate and comment vote set tests 2022-05-12 15:52:41 -04:00
William Banfield
a8b85c1999 add internal bool param to internal vote set 2022-05-12 15:02:36 -04:00
William Banfield
eec438ac97 fix require condition to be correct 2022-05-12 14:48:23 -04:00
William Banfield
5e4575695d test comple: still needs comment 2022-05-11 23:04:59 -04:00
William Banfield
b4da26555b remove duplicate logging 2022-05-11 23:00:26 -04:00
William Banfield
d9820182e6 update test to ensure consensus proceeds as expected 2022-05-11 22:52:31 -04:00
William Banfield
667c53dcbc use 3 validators for verify test 2022-05-11 22:22:32 -04:00
William Banfield
c35bcbe320 use 4 validators and only expect 3 verifiy calls 2022-05-11 22:19:46 -04:00
William Banfield
b075117d83 don't validate when constructing extendedcommitsig 2022-05-11 22:17:52 -04:00
William Banfield
3e71e81938 fix test to bypass own verify vote extension 2022-05-11 22:15:57 -04:00
William Banfield
5caea6e01e add logging for verification failed 2022-05-11 22:15:30 -04:00
William Banfield
146c996ec7 fix typo 2022-05-11 22:01:27 -04:00
William Banfield
101d357224 wip: compiles but test fails 2022-05-11 19:27:13 -04:00
William Banfield
5aeee88443 add param to params.go 2022-05-11 16:38:46 -04:00
16 changed files with 537 additions and 110 deletions

View File

@@ -32,6 +32,7 @@ import (
"github.com/tendermint/tendermint/internal/test/factory"
"github.com/tendermint/tendermint/libs/log"
tmcons "github.com/tendermint/tendermint/proto/tendermint/consensus"
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
"github.com/tendermint/tendermint/types"
)
@@ -600,6 +601,106 @@ func TestReactorCreatesBlockWhenEmptyBlocksFalse(t *testing.T) {
wg.Wait()
}
func TestSwitchToConsensusVoteExtensions(t *testing.T) {
for _, testCase := range []struct {
name string
storedHeight int64
initialRequiredHeight int64
includeExtensions bool
shouldPanic bool
}{
{
name: "no vote extensions but not required",
initialRequiredHeight: 0,
storedHeight: 2,
includeExtensions: false,
shouldPanic: false,
},
{
name: "no vote extensions but required this height",
initialRequiredHeight: 2,
storedHeight: 2,
includeExtensions: false,
shouldPanic: true,
},
{
name: "no vote extensions and required in future",
initialRequiredHeight: 3,
storedHeight: 2,
includeExtensions: false,
shouldPanic: false,
},
{
name: "no vote extensions and required previous height",
initialRequiredHeight: 1,
storedHeight: 2,
includeExtensions: false,
shouldPanic: true,
},
{
name: "vote extensions and required previous height",
initialRequiredHeight: 1,
storedHeight: 2,
includeExtensions: true,
shouldPanic: false,
},
} {
t.Run(testCase.name, func(t *testing.T) {
ctx, cancel := context.WithTimeout(context.Background(), time.Minute)
defer cancel()
cs, vs := makeState(ctx, t, makeStateArgs{validators: 1})
validator := vs[0]
validator.Height = testCase.storedHeight
cs.state.LastBlockHeight = testCase.storedHeight
cs.state.LastValidators = cs.state.Validators.Copy()
cs.state.ConsensusParams.Vote.ExtensionRequireHeight = testCase.initialRequiredHeight
propBlock, err := cs.createProposalBlock(ctx)
require.NoError(t, err)
// Consensus is preparing to do the next height after the stored height.
cs.Height = testCase.storedHeight + 1
propBlock.Height = testCase.storedHeight
blockParts, err := propBlock.MakePartSet(types.BlockPartSizeBytes)
require.NoError(t, err)
voteSet := types.NewVoteSet(cs.state.ChainID, testCase.storedHeight, 0, tmproto.PrecommitType, cs.state.Validators, false)
signedVote := signVote(ctx, t, validator, tmproto.PrecommitType, cs.state.ChainID, types.BlockID{
Hash: propBlock.Hash(),
PartSetHeader: blockParts.Header(),
})
if !testCase.includeExtensions {
signedVote.Extension = nil
signedVote.ExtensionSignature = nil
}
added, err := voteSet.AddVote(signedVote)
require.NoError(t, err)
require.True(t, added)
cs.blockStore.SaveBlock(propBlock, blockParts, voteSet.MakeExtendedCommit())
reactor := NewReactor(
log.NewNopLogger(),
cs,
nil,
nil,
cs.eventBus,
true,
NopMetrics(),
)
if testCase.shouldPanic {
assert.Panics(t, func() {
reactor.SwitchToConsensus(ctx, cs.state, false)
})
} else {
reactor.SwitchToConsensus(ctx, cs.state, false)
}
})
}
}
func TestReactorRecordsVotesAndBlockParts(t *testing.T) {
ctx, cancel := context.WithTimeout(context.Background(), time.Minute)
defer cancel()

View File

@@ -695,20 +695,48 @@ func (cs *State) sendInternalMessage(ctx context.Context, mi msgInfo) {
// Reconstruct LastCommit from SeenCommit, which we saved along with the block,
// (which happens even before saving the state)
func (cs *State) reconstructLastCommit(state sm.State) {
extCommit := cs.blockStore.LoadBlockExtendedCommit(state.LastBlockHeight)
if extCommit == nil {
panic(fmt.Sprintf(
"failed to reconstruct last commit; commit for height %v not found",
state.LastBlockHeight,
))
requireExtensions := requireVoteExtensions(cs.state.ConsensusParams.Vote.ExtensionRequireHeight, state.LastBlockHeight)
votes, err := cs.votesFromExtendedCommit(state, requireExtensions)
if err == nil {
cs.LastCommit = votes
return
}
if requireExtensions {
panic(fmt.Sprintf("failed to reconstruct last commit; %s", err))
}
votes, err = cs.votesFromSeenCommit(state)
if err != nil {
panic(fmt.Sprintf("failed to reconstruct last commit; %s", err))
}
cs.LastCommit = votes
}
func (cs *State) votesFromExtendedCommit(state sm.State, requireExtensions bool) (*types.VoteSet, error) {
ec := cs.blockStore.LoadBlockExtendedCommit(state.LastBlockHeight)
if ec == nil {
return nil, fmt.Errorf("commit for height %v not found", state.LastBlockHeight)
}
vs := ec.ToVoteSet(state.ChainID, state.LastValidators, requireExtensions)
if !vs.HasTwoThirdsMajority() {
return nil, errors.New("seen commit does not have +2/3 majority")
}
return vs, nil
}
func (cs *State) votesFromSeenCommit(state sm.State) (*types.VoteSet, error) {
commit := cs.blockStore.LoadSeenCommit()
if commit == nil || commit.Height != state.LastBlockHeight {
commit = cs.blockStore.LoadBlockCommit(state.LastBlockHeight)
}
if commit == nil {
return nil, fmt.Errorf("commit for height %v not found", state.LastBlockHeight)
}
lastPrecommits := extCommit.ToVoteSet(state.ChainID, state.LastValidators)
if !lastPrecommits.HasTwoThirdsMajority() {
panic("failed to reconstruct last commit; does not have +2/3 maj")
vs := commit.ToVoteSet(state.ChainID, state.LastValidators)
if !vs.HasTwoThirdsMajority() {
return nil, errors.New("commit does not have +2/3 majority")
}
cs.LastCommit = lastPrecommits
return vs, nil
}
// Updates State and increments height to match that of state.
@@ -810,7 +838,7 @@ func (cs *State) updateToState(state sm.State) {
cs.ValidRound = -1
cs.ValidBlock = nil
cs.ValidBlockParts = nil
cs.Votes = cstypes.NewHeightVoteSet(state.ChainID, height, validators)
cs.Votes = cstypes.NewHeightVoteSet(state.ChainID, height, validators, requireVoteExtensions(state.ConsensusParams.Vote.ExtensionRequireHeight, height))
cs.CommitRound = -1
cs.LastValidators = state.LastValidators
cs.TriggeredTimeoutPrecommit = false
@@ -2308,6 +2336,7 @@ func (cs *State) addVote(
return
}
// late votes still _must_ have extensions.
added, err = cs.LastCommit.AddVote(vote)
if !added {
return
@@ -2339,11 +2368,31 @@ func (cs *State) addVote(
// Verify VoteExtension if precommit and not nil
// https://github.com/tendermint/tendermint/issues/8487
if vote.Type == tmproto.PrecommitType && !vote.BlockID.IsNil() {
err := cs.blockExec.VerifyVoteExtension(ctx, vote)
cs.metrics.MarkVoteExtensionReceived(err == nil)
if err != nil {
return false, err
if vote.Type == tmproto.PrecommitType && !vote.BlockID.IsNil() &&
!bytes.Equal(vote.ValidatorAddress, cs.privValidatorPubKey.Address()) {
// The core fields of the vote message were already validated in the
// consensus reactor when the vote was received.
// Here, we valdiate that the vote extension was included in the vote
// message.
// Chains that are not configured to require vote extensions
// will consider the vote valid even if the extension is absent.
// VerifyVoteExtension will not be called in this case if the extension
// is absent.
err := vote.EnsureExtension()
if err == nil {
_, val := cs.state.Validators.GetByIndex(vote.ValidatorIndex)
err = vote.VerifyWithExtension(cs.state.ChainID, val.PubKey)
}
if err == nil {
err := cs.blockExec.VerifyVoteExtension(ctx, vote)
cs.metrics.MarkVoteExtensionReceived(err == nil)
} else {
if !errors.Is(err, types.ErrVoteExtensionAbsent) {
return false, err
}
if requireVoteExtensions(cs.state.ConsensusParams.Vote.ExtensionRequireHeight, cs.Height) {
return false, err
}
}
}
@@ -2741,6 +2790,13 @@ func (cs *State) calculateProposalTimestampDifferenceMetric() {
}
}
func requireVoteExtensions(requireHeight, currentHeight int64) bool {
if requireHeight == 0 || currentHeight < requireHeight {
return false
}
return true
}
// proposerWaitTime determines how long the proposer should wait to propose its next block.
// If the result is zero, a block can be proposed immediately.
//

View File

@@ -2076,19 +2076,13 @@ func TestExtendVoteCalled(t *testing.T) {
Hash: blockID.Hash,
})
m.AssertCalled(t, "VerifyVoteExtension", ctx, &abci.RequestVerifyVoteExtension{
Hash: blockID.Hash,
ValidatorAddress: addr,
Height: height,
VoteExtension: []byte("extension"),
})
signAddVotes(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), blockID, vss[1:]...)
ensureNewRound(t, newRoundCh, height+1, 0)
m.AssertExpectations(t)
// Only 3 of the vote extensions are seen, as consensus proceeds as soon as the +2/3 threshold
// is observed by the consensus engine.
for _, pv := range vss[:3] {
for _, pv := range vss[1:3] {
pv, err := pv.GetPubKey(ctx)
require.NoError(t, err)
addr := pv.Address()
@@ -2148,13 +2142,6 @@ func TestVerifyVoteExtensionNotCalledOnAbsentPrecommit(t *testing.T) {
Hash: blockID.Hash,
})
m.AssertCalled(t, "VerifyVoteExtension", mock.Anything, &abci.RequestVerifyVoteExtension{
Hash: blockID.Hash,
ValidatorAddress: addr,
Height: height,
VoteExtension: []byte("extension"),
})
m.On("Commit", mock.Anything).Return(&abci.ResponseCommit{}, nil).Maybe()
signAddVotes(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), blockID, vss[2:]...)
ensureNewRound(t, newRoundCh, height+1, 0)
@@ -2266,6 +2253,118 @@ func TestPrepareProposalReceivesVoteExtensions(t *testing.T) {
}
}
// TestVoteExtensionRequiredHeight tests that 'ExtensionRequireHeight' correctly
// enforces that vote extensions be present in consensus for heights greater than
// or equal to the configured value.
func TestVoteExtensionRequiredHeight(t *testing.T) {
for _, testCase := range []struct {
name string
initialRequiredHeight int64
hasExtension bool
expectSuccessfulRound bool
}{
{
name: "extension present but not required",
hasExtension: true,
initialRequiredHeight: 0,
expectSuccessfulRound: true,
},
{
name: "extension absent but not required",
hasExtension: false,
initialRequiredHeight: 0,
expectSuccessfulRound: true,
},
{
name: "extension present and required",
hasExtension: true,
initialRequiredHeight: 1,
expectSuccessfulRound: true,
},
{
name: "extension absent but required",
hasExtension: false,
initialRequiredHeight: 1,
expectSuccessfulRound: false,
},
{
name: "extension absent but required in future height",
hasExtension: false,
initialRequiredHeight: 2,
expectSuccessfulRound: true,
},
} {
t.Run(testCase.name, func(t *testing.T) {
config := configSetup(t)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
numValidators := 3
m := abcimocks.NewApplication(t)
m.On("ProcessProposal", mock.Anything, mock.Anything).Return(&abci.ResponseProcessProposal{
Status: abci.ResponseProcessProposal_ACCEPT,
}, nil)
m.On("PrepareProposal", mock.Anything, mock.Anything).Return(&abci.ResponsePrepareProposal{}, nil)
m.On("ExtendVote", mock.Anything, mock.Anything).Return(&abci.ResponseExtendVote{}, nil)
if testCase.hasExtension {
m.On("VerifyVoteExtension", mock.Anything, mock.Anything).Return(&abci.ResponseVerifyVoteExtension{
Status: abci.ResponseVerifyVoteExtension_ACCEPT,
}, nil).Times(numValidators - 1)
}
m.On("FinalizeBlock", mock.Anything, mock.Anything).Return(&abci.ResponseFinalizeBlock{}, nil).Maybe()
m.On("Commit", mock.Anything).Return(&abci.ResponseCommit{}, nil).Maybe()
cs1, vss := makeState(ctx, t, makeStateArgs{config: config, application: m, validators: numValidators})
cs1.state.ConsensusParams.Vote.ExtensionRequireHeight = testCase.initialRequiredHeight
height, round := cs1.Height, cs1.Round
timeoutCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryTimeoutPropose)
proposalCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryCompleteProposal)
newRoundCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryNewRound)
pv1, err := cs1.privValidator.GetPubKey(ctx)
require.NoError(t, err)
addr := pv1.Address()
voteCh := subscribeToVoter(ctx, t, cs1, addr)
startTestRound(ctx, cs1, cs1.Height, round)
ensureNewRound(t, newRoundCh, height, round)
ensureNewProposal(t, proposalCh, height, round)
rs := cs1.GetRoundState()
blockID := types.BlockID{
Hash: rs.ProposalBlock.Hash(),
PartSetHeader: rs.ProposalBlockParts.Header(),
}
// sign all of the votes
signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), blockID, vss[1:]...)
ensurePrevoteMatch(t, voteCh, height, round, rs.ProposalBlock.Hash())
var ext []byte
if testCase.hasExtension {
ext = []byte("extension")
}
for _, vs := range vss[1:] {
vote, err := vs.signVote(ctx, tmproto.PrecommitType, config.ChainID(), blockID, ext)
if !testCase.hasExtension {
vote.ExtensionSignature = nil
}
require.NoError(t, err)
addVotes(cs1, vote)
}
if testCase.expectSuccessfulRound {
ensurePrecommit(t, voteCh, height, round)
height++
ensureNewRound(t, newRoundCh, height, round)
} else {
ensureNoNewTimeout(t, timeoutCh, cs1.state.ConsensusParams.Timeout.VoteTimeout(round).Nanoseconds())
}
m.AssertExpectations(t)
})
}
}
// 4 vals, 3 Nil Precommits at P0
// What we want:
// P0 waits for timeoutPrecommit before starting next round

View File

@@ -38,9 +38,10 @@ We let each peer provide us with up to 2 unexpected "catchup" rounds.
One for their LastCommit round, and another for the official commit round.
*/
type HeightVoteSet struct {
chainID string
height int64
valSet *types.ValidatorSet
chainID string
height int64
valSet *types.ValidatorSet
requireExtensions bool
mtx sync.Mutex
round int32 // max tracked round
@@ -48,9 +49,10 @@ type HeightVoteSet struct {
peerCatchupRounds map[types.NodeID][]int32 // keys: peer.ID; values: at most 2 rounds
}
func NewHeightVoteSet(chainID string, height int64, valSet *types.ValidatorSet) *HeightVoteSet {
func NewHeightVoteSet(chainID string, height int64, valSet *types.ValidatorSet, requireExtensions bool) *HeightVoteSet {
hvs := &HeightVoteSet{
chainID: chainID,
chainID: chainID,
requireExtensions: requireExtensions,
}
hvs.Reset(height, valSet)
return hvs
@@ -107,8 +109,8 @@ func (hvs *HeightVoteSet) addRound(round int32) {
panic("addRound() for an existing round")
}
// log.Debug("addRound(round)", "round", round)
prevotes := types.NewVoteSet(hvs.chainID, hvs.height, round, tmproto.PrevoteType, hvs.valSet)
precommits := types.NewVoteSet(hvs.chainID, hvs.height, round, tmproto.PrecommitType, hvs.valSet)
prevotes := types.NewVoteSet(hvs.chainID, hvs.height, round, tmproto.PrevoteType, hvs.valSet, hvs.requireExtensions)
precommits := types.NewVoteSet(hvs.chainID, hvs.height, round, tmproto.PrecommitType, hvs.valSet, hvs.requireExtensions)
hvs.roundVoteSets[round] = RoundVoteSet{
Prevotes: prevotes,
Precommits: precommits,

View File

@@ -27,7 +27,7 @@ func TestPeerCatchupRounds(t *testing.T) {
valSet, privVals := factory.ValidatorSet(ctx, t, 10, 1)
chainID := cfg.ChainID()
hvs := NewHeightVoteSet(chainID, 1, valSet)
hvs := NewHeightVoteSet(chainID, 1, valSet, false)
vote999_0 := makeVoteHR(ctx, t, 1, 0, 999, privVals, chainID)
added, err := hvs.AddVote(vote999_0, "peer1")

View File

@@ -233,7 +233,7 @@ func TestVerifyLightClientAttack_Equivocation(t *testing.T) {
// we are simulating a duplicate vote attack where all the validators in the conflictingVals set
// except the last validator vote twice
blockID := factory.MakeBlockIDWithHash(conflictingHeader.Hash())
voteSet := types.NewVoteSet(evidenceChainID, 10, 1, tmproto.SignedMsgType(2), conflictingVals)
voteSet := types.NewVoteSet(evidenceChainID, 10, 1, tmproto.SignedMsgType(2), conflictingVals, false)
extCommit, err := factory.MakeExtendedCommit(ctx, blockID, 10, 1, voteSet, conflictingPrivVals[:4], defaultEvidenceTime)
require.NoError(t, err)
commit := extCommit.StripExtensions()
@@ -253,7 +253,7 @@ func TestVerifyLightClientAttack_Equivocation(t *testing.T) {
}
trustedBlockID := makeBlockID(trustedHeader.Hash(), 1000, []byte("partshash"))
trustedVoteSet := types.NewVoteSet(evidenceChainID, 10, 1, tmproto.SignedMsgType(2), conflictingVals)
trustedVoteSet := types.NewVoteSet(evidenceChainID, 10, 1, tmproto.SignedMsgType(2), conflictingVals, false)
trustedExtCommit, err := factory.MakeExtendedCommit(ctx, trustedBlockID, 10, 1,
trustedVoteSet, conflictingPrivVals, defaultEvidenceTime)
require.NoError(t, err)
@@ -336,7 +336,7 @@ func TestVerifyLightClientAttack_Amnesia(t *testing.T) {
// we are simulating an amnesia attack where all the validators in the conflictingVals set
// except the last validator vote twice. However this time the commits are of different rounds.
blockID := makeBlockID(conflictingHeader.Hash(), 1000, []byte("partshash"))
voteSet := types.NewVoteSet(evidenceChainID, height, 0, tmproto.SignedMsgType(2), conflictingVals)
voteSet := types.NewVoteSet(evidenceChainID, height, 0, tmproto.SignedMsgType(2), conflictingVals, false)
extCommit, err := factory.MakeExtendedCommit(ctx, blockID, height, 0, voteSet, conflictingPrivVals, defaultEvidenceTime)
require.NoError(t, err)
commit := extCommit.StripExtensions()
@@ -356,7 +356,7 @@ func TestVerifyLightClientAttack_Amnesia(t *testing.T) {
}
trustedBlockID := makeBlockID(trustedHeader.Hash(), 1000, []byte("partshash"))
trustedVoteSet := types.NewVoteSet(evidenceChainID, height, 1, tmproto.SignedMsgType(2), conflictingVals)
trustedVoteSet := types.NewVoteSet(evidenceChainID, height, 1, tmproto.SignedMsgType(2), conflictingVals, false)
trustedExtCommit, err := factory.MakeExtendedCommit(ctx, trustedBlockID, height, 1,
trustedVoteSet, conflictingPrivVals, defaultEvidenceTime)
require.NoError(t, err)
@@ -553,7 +553,7 @@ func makeLunaticEvidence(
})
blockID := factory.MakeBlockIDWithHash(conflictingHeader.Hash())
voteSet := types.NewVoteSet(evidenceChainID, height, 1, tmproto.SignedMsgType(2), conflictingVals)
voteSet := types.NewVoteSet(evidenceChainID, height, 1, tmproto.SignedMsgType(2), conflictingVals, false)
extCommit, err := factory.MakeExtendedCommit(ctx, blockID, height, 1, voteSet, conflictingPrivVals, defaultEvidenceTime)
require.NoError(t, err)
commit := extCommit.StripExtensions()
@@ -582,7 +582,7 @@ func makeLunaticEvidence(
}
trustedBlockID := factory.MakeBlockIDWithHash(trustedHeader.Hash())
trustedVals, privVals := factory.ValidatorSet(ctx, t, totalVals, defaultVotingPower)
trustedVoteSet := types.NewVoteSet(evidenceChainID, height, 1, tmproto.SignedMsgType(2), trustedVals)
trustedVoteSet := types.NewVoteSet(evidenceChainID, height, 1, tmproto.SignedMsgType(2), trustedVals, false)
trustedExtCommit, err := factory.MakeExtendedCommit(ctx, trustedBlockID, height, 1, trustedVoteSet, privVals, defaultEvidenceTime)
require.NoError(t, err)
trustedCommit := trustedExtCommit.StripExtensions()

View File

@@ -3,6 +3,7 @@ package state
import (
"bytes"
"context"
"errors"
"fmt"
"time"
@@ -102,13 +103,12 @@ func (blockExec *BlockExecutor) CreateProposalBlock(
txs := blockExec.mempool.ReapMaxBytesMaxGas(maxDataBytes, maxGas)
commit := lastExtCommit.StripExtensions()
block := state.MakeBlock(height, txs, commit, evidence, proposerAddr)
rpp, err := blockExec.appClient.PrepareProposal(
ctx,
&abci.RequestPrepareProposal{
MaxTxBytes: maxDataBytes,
Txs: block.Txs.ToSliceOfBytes(),
LocalLastCommit: buildExtendedCommitInfo(lastExtCommit, blockExec.store, state.InitialHeight),
LocalLastCommit: buildExtendedCommitInfo(lastExtCommit, blockExec.store, state.InitialHeight, state.ConsensusParams.Vote.ExtensionRequireHeight),
ByzantineValidators: block.Evidence.ToABCI(),
Height: block.Height,
Time: block.Time,
@@ -321,7 +321,7 @@ func (blockExec *BlockExecutor) VerifyVoteExtension(ctx context.Context, vote *t
}
if !resp.IsOK() {
return types.ErrVoteInvalidExtension
return errors.New("invalid vote extension")
}
return nil
@@ -428,7 +428,7 @@ func buildLastCommitInfo(block *types.Block, store Store, initialHeight int64) a
// data, it returns an empty record.
//
// Assumes that the commit signatures are sorted according to validator index.
func buildExtendedCommitInfo(ec *types.ExtendedCommit, store Store, initialHeight int64) abci.ExtendedCommitInfo {
func buildExtendedCommitInfo(ec *types.ExtendedCommit, store Store, initialHeight, extensionRequireHeight int64) abci.ExtendedCommitInfo {
if ec.Height < initialHeight {
// There are no extended commits for heights below the initial height.
return abci.ExtendedCommitInfo{}
@@ -469,6 +469,11 @@ func buildExtendedCommitInfo(ec *types.ExtendedCommit, store Store, initialHeigh
if ecs.BlockIDFlag == types.BlockIDFlagCommit {
// We only care about vote extensions if a validator has voted to
// commit.
if ecs.Extension == nil && ecs.ExtensionSignature == nil &&
extensionRequireHeight != 0 && ec.Height >= extensionRequireHeight {
// TODO: this error is akward, make it better
panic(fmt.Errorf("commit received with missing vote extension data"))
}
ext = ecs.Extension
}

View File

@@ -855,7 +855,7 @@ func mockLB(ctx context.Context, t *testing.T, height int64, time time.Time, las
header.NextValidatorsHash = nextVals.Hash()
header.ConsensusHash = types.DefaultConsensusParams().HashConsensusParams()
lastBlockID = factory.MakeBlockIDWithHash(header.Hash())
voteSet := types.NewVoteSet(factory.DefaultTestChainID, height, 0, tmproto.PrecommitType, currentVals)
voteSet := types.NewVoteSet(factory.DefaultTestChainID, height, 0, tmproto.PrecommitType, currentVals, false)
extCommit, err := factory.MakeExtendedCommit(ctx, lastBlockID, height, 0, voteSet, currentPrivVals, time)
require.NoError(t, err)
return nextVals, nextPrivVals, &types.LightBlock{

View File

@@ -526,7 +526,7 @@ func TestMaxProposalBlockSize(t *testing.T) {
}
state.ChainID = maxChainID
voteSet := types.NewVoteSet(state.ChainID, math.MaxInt64-1, math.MaxInt32, tmproto.PrecommitType, state.Validators)
voteSet := types.NewVoteSet(state.ChainID, math.MaxInt64-1, math.MaxInt32, tmproto.PrecommitType, state.Validators, false)
// add maximum amount of signatures to a single commit
for i := 0; i < types.MaxVotesCount; i++ {

View File

@@ -165,7 +165,7 @@ func generateLightClientAttackEvidence(
// create a commit for the forged header
blockID := makeBlockID(header.Hash(), 1000, []byte("partshash"))
voteSet := types.NewVoteSet(chainID, forgedHeight, 0, tmproto.SignedMsgType(2), conflictingVals)
voteSet := types.NewVoteSet(chainID, forgedHeight, 0, tmproto.SignedMsgType(2), conflictingVals, false)
commit, err := factory.MakeExtendedCommit(ctx, blockID, forgedHeight, 0, voteSet, pv, forgedTime)
if err != nil {

View File

@@ -757,9 +757,6 @@ func (ecs ExtendedCommitSig) ValidateBasic() error {
if len(ecs.Extension) > MaxVoteExtensionSize {
return fmt.Errorf("vote extension is too big (max: %d)", MaxVoteExtensionSize)
}
if len(ecs.ExtensionSignature) == 0 {
return errors.New("vote extension signature is missing")
}
if len(ecs.ExtensionSignature) > MaxSignatureSize {
return fmt.Errorf("vote extension signature is too big (max: %d)", MaxSignatureSize)
}
@@ -777,6 +774,13 @@ func (ecs ExtendedCommitSig) ValidateBasic() error {
return nil
}
func (ecs ExtendedCommitSig) ValidateExtension() error {
if len(ecs.ExtensionSignature) == 0 {
return errors.New("vote extension signature is missing")
}
return nil
}
// ToProto converts the ExtendedCommitSig to its Protobuf representation.
func (ecs *ExtendedCommitSig) ToProto() *tmproto.ExtendedCommitSig {
if ecs == nil {
@@ -1016,14 +1020,14 @@ func (ec *ExtendedCommit) Clone() *ExtendedCommit {
// ToVoteSet constructs a VoteSet from the Commit and validator set.
// Panics if signatures from the commit can't be added to the voteset.
// Inverse of VoteSet.MakeExtendedCommit().
func (ec *ExtendedCommit) ToVoteSet(chainID string, vals *ValidatorSet) *VoteSet {
voteSet := NewVoteSet(chainID, ec.Height, ec.Round, tmproto.PrecommitType, vals)
func (ec *ExtendedCommit) ToVoteSet(chainID string, vals *ValidatorSet, requireExtensions bool) *VoteSet {
voteSet := NewVoteSet(chainID, ec.Height, ec.Round, tmproto.PrecommitType, vals, requireExtensions)
for idx, ecs := range ec.ExtendedSignatures {
if ecs.BlockIDFlag == BlockIDFlagAbsent {
continue // OK, some precommits can be missing.
}
vote := ec.GetExtendedVote(int32(idx))
if err := vote.ValidateWithExtension(); err != nil {
if err := vote.ValidateBasic(); err != nil {
panic(fmt.Errorf("failed to validate vote reconstructed from LastCommit: %w", err))
}
added, err := voteSet.AddVote(vote)
@@ -1034,6 +1038,43 @@ func (ec *ExtendedCommit) ToVoteSet(chainID string, vals *ValidatorSet) *VoteSet
return voteSet
}
// ToVoteSet constructs a VoteSet from the Commit and validator set.
// Panics if signatures from the commit can't be added to the voteset.
// Inverse of VoteSet.MakeCommit().
func (c *Commit) ToVoteSet(chainID string, vals *ValidatorSet) *VoteSet {
voteSet := NewVoteSet(chainID, c.Height, c.Round, tmproto.PrecommitType, vals, false)
for idx, cs := range c.Signatures {
if cs.BlockIDFlag == BlockIDFlagAbsent {
continue // OK, some precommits can be missing.
}
vote := c.GetVote(int32(idx))
if err := vote.ValidateBasic(); err != nil {
panic(fmt.Errorf("failed to validate vote reconstructed from commit: %w", err))
}
added, err := voteSet.AddVote(vote)
if !added || err != nil {
panic(fmt.Errorf("failed to reconstruct vote set from commit: %w", err))
}
}
return voteSet
}
// TODO Comment
// this should probably also verify the signature
// probably want to change to just verify when present.
func (ec *ExtendedCommit) EnsureExtensions() error {
for idx, ecs := range ec.ExtendedSignatures {
if ecs.BlockIDFlag == BlockIDFlagAbsent {
continue
}
vote := ec.GetExtendedVote(int32(idx))
if err := vote.EnsureExtension(); err != nil {
return err
}
}
return nil
}
// StripExtensions converts an ExtendedCommit to a Commit by removing all vote
// extension-related fields.
func (ec *ExtendedCommit) StripExtensions() *Commit {

View File

@@ -556,33 +556,127 @@ func TestBlockMaxDataBytesNoEvidence(t *testing.T) {
}
}
// TestVoteSetToExtendedCommit tests that the extended commit produced from a
// vote set contains the same vote information as the vote set. The test ensures
// that the MakeExtendedCommit method behaves as expected, whether vote extensions
// are present in the original votes or not.
func TestVoteSetToExtendedCommit(t *testing.T) {
for _, testCase := range []struct {
name string
includeExtension bool
}{
{
name: "no extensions",
includeExtension: false,
},
{
name: "with extensions",
includeExtension: true,
},
} {
t.Run(testCase.name, func(t *testing.T) {
blockID := makeBlockIDRandom()
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
voteSet, _, vals := randVoteSet(ctx, t, 3, 1, tmproto.PrecommitType, 10, 1)
for i := 0; i < len(vals); i++ {
pubKey, err := vals[i].GetPubKey(ctx)
require.NoError(t, err)
vote := &Vote{
ValidatorAddress: pubKey.Address(),
ValidatorIndex: int32(i),
Height: 3,
Round: 1,
Type: tmproto.PrecommitType,
BlockID: blockID,
Timestamp: time.Now(),
}
v := vote.ToProto()
err = vals[i].SignVote(ctx, voteSet.ChainID(), v)
require.NoError(t, err)
vote.Signature = v.Signature
if testCase.includeExtension {
vote.ExtensionSignature = v.ExtensionSignature
}
added, err := voteSet.AddVote(vote)
require.NoError(t, err)
require.True(t, added)
}
ec := voteSet.MakeExtendedCommit()
for i := int32(0); int(i) < len(vals); i++ {
vote1 := voteSet.GetByIndex(i)
vote2 := ec.GetExtendedVote(i)
vote1bz, err := vote1.ToProto().Marshal()
require.NoError(t, err)
vote2bz, err := vote2.ToProto().Marshal()
require.NoError(t, err)
assert.Equal(t, vote1bz, vote2bz)
}
})
}
}
// TestExtendedCommitToVoteSet tests that the vote set produced from an extended commit
// contains the same vote information as the extended commit. The test ensures
// that the ToVoteSet method behaves as expected, whether vote extensions
// are present in the original votes or not.
func TestExtendedCommitToVoteSet(t *testing.T) {
lastID := makeBlockIDRandom()
h := int64(3)
for _, testCase := range []struct {
name string
includeExtension bool
}{
{
name: "no extensions",
includeExtension: false,
},
{
name: "with extensions",
includeExtension: true,
},
} {
t.Run(testCase.name, func(t *testing.T) {
lastID := makeBlockIDRandom()
h := int64(3)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
voteSet, valSet, vals := randVoteSet(ctx, t, h-1, 1, tmproto.PrecommitType, 10, 1)
extCommit, err := makeExtCommit(ctx, lastID, h-1, 1, voteSet, vals, time.Now())
assert.NoError(t, err)
voteSet, valSet, vals := randVoteSet(ctx, t, h-1, 1, tmproto.PrecommitType, 10, 1)
extCommit, err := makeExtCommit(ctx, lastID, h-1, 1, voteSet, vals, time.Now())
assert.NoError(t, err)
chainID := voteSet.ChainID()
voteSet2 := extCommit.ToVoteSet(chainID, valSet)
if !testCase.includeExtension {
for i := 0; i < len(vals); i++ {
v := voteSet.GetByIndex(int32(i))
v.Extension = nil
v.ExtensionSignature = nil
extCommit.ExtendedSignatures[i].Extension = nil
extCommit.ExtendedSignatures[i].ExtensionSignature = nil
}
}
for i := int32(0); int(i) < len(vals); i++ {
vote1 := voteSet.GetByIndex(i)
vote2 := voteSet2.GetByIndex(i)
vote3 := extCommit.GetExtendedVote(i)
chainID := voteSet.ChainID()
voteSet2 := extCommit.ToVoteSet(chainID, valSet, true)
vote1bz, err := vote1.ToProto().Marshal()
require.NoError(t, err)
vote2bz, err := vote2.ToProto().Marshal()
require.NoError(t, err)
vote3bz, err := vote3.ToProto().Marshal()
require.NoError(t, err)
assert.Equal(t, vote1bz, vote2bz)
assert.Equal(t, vote1bz, vote3bz)
for i := int32(0); int(i) < len(vals); i++ {
vote1 := voteSet.GetByIndex(i)
vote2 := voteSet2.GetByIndex(i)
vote3 := extCommit.GetExtendedVote(i)
vote1bz, err := vote1.ToProto().Marshal()
require.NoError(t, err)
vote2bz, err := vote2.ToProto().Marshal()
require.NoError(t, err)
vote3bz, err := vote3.ToProto().Marshal()
require.NoError(t, err)
assert.Equal(t, vote1bz, vote2bz)
assert.Equal(t, vote1bz, vote3bz)
}
})
}
}

View File

@@ -43,6 +43,7 @@ type ConsensusParams struct {
Version VersionParams `json:"version"`
Synchrony SynchronyParams `json:"synchrony"`
Timeout TimeoutParams `json:"timeout"`
Vote VoteParams `json:"vote"`
}
// HashedParams is a subset of ConsensusParams.
@@ -96,6 +97,11 @@ type TimeoutParams struct {
BypassCommitTimeout bool `json:"bypass_commit_timeout"`
}
// VoteParams configure validity rules of the votes within Tendermint consensus.
type VoteParams struct {
ExtensionRequireHeight int64 `json:"extension_require_height"`
}
// DefaultConsensusParams returns a default ConsensusParams.
func DefaultConsensusParams() *ConsensusParams {
return &ConsensusParams{
@@ -105,6 +111,7 @@ func DefaultConsensusParams() *ConsensusParams {
Version: DefaultVersionParams(),
Synchrony: DefaultSynchronyParams(),
Timeout: DefaultTimeoutParams(),
Vote: DefaultVoteParams(),
}
}
@@ -176,6 +183,13 @@ func DefaultTimeoutParams() TimeoutParams {
}
}
func DefaultVoteParams() VoteParams {
return VoteParams{
// When set to 0, vote extensions are not required.
ExtensionRequireHeight: 0,
}
}
// TimeoutParamsOrDefaults returns the SynchronyParams, filling in any zero values
// with the Tendermint defined default values.
func (t TimeoutParams) TimeoutParamsOrDefaults() TimeoutParams {

View File

@@ -27,7 +27,7 @@ var (
ErrVoteInvalidBlockHash = errors.New("invalid block hash")
ErrVoteNonDeterministicSignature = errors.New("non-deterministic signature")
ErrVoteNil = errors.New("nil vote")
ErrVoteInvalidExtension = errors.New("invalid vote extension")
ErrVoteExtensionAbsent = errors.New("vote extension absent")
)
type ErrVoteConflictingVotes struct {
@@ -121,10 +121,6 @@ func (vote *Vote) ExtendedCommitSig() ExtendedCommitSig {
}
cs := vote.CommitSig()
if vote.BlockID.IsComplete() && len(vote.ExtensionSignature) == 0 {
panic(fmt.Sprintf("Invalid vote %v - BlockID is complete but missing vote extension signature", vote))
}
return ExtendedCommitSig{
CommitSig: cs,
Extension: vote.Extension,
@@ -317,16 +313,21 @@ func (vote *Vote) ValidateWithExtension() error {
return err
}
if err := vote.EnsureExtension(); err != nil {
return err
}
return nil
}
//
func (vote *Vote) EnsureExtension() error {
// We should always see vote extension signatures in non-nil precommits
if vote.Type == tmproto.PrecommitType && !vote.BlockID.IsNil() {
if len(vote.ExtensionSignature) == 0 {
return errors.New("vote extension signature is missing")
}
if len(vote.ExtensionSignature) > MaxSignatureSize {
return fmt.Errorf("vote extension signature is too big (max: %d)", MaxSignatureSize)
return ErrVoteExtensionAbsent
}
}
return nil
}

View File

@@ -53,11 +53,12 @@ const (
NOTE: Assumes that the sum total of voting power does not exceed MaxUInt64.
*/
type VoteSet struct {
chainID string
height int64
round int32
signedMsgType tmproto.SignedMsgType
valSet *ValidatorSet
chainID string
height int64
round int32
signedMsgType tmproto.SignedMsgType
valSet *ValidatorSet
requireExtensions bool
mtx sync.Mutex
votesBitArray *bits.BitArray
@@ -70,22 +71,23 @@ type VoteSet struct {
// Constructs a new VoteSet struct used to accumulate votes for given height/round.
func NewVoteSet(chainID string, height int64, round int32,
signedMsgType tmproto.SignedMsgType, valSet *ValidatorSet) *VoteSet {
signedMsgType tmproto.SignedMsgType, valSet *ValidatorSet, requireExtensions bool) *VoteSet {
if height == 0 {
panic("Cannot make VoteSet for height == 0, doesn't make sense.")
}
return &VoteSet{
chainID: chainID,
height: height,
round: round,
signedMsgType: signedMsgType,
valSet: valSet,
votesBitArray: bits.NewBitArray(valSet.Size()),
votes: make([]*Vote, valSet.Size()),
sum: 0,
maj23: nil,
votesByBlock: make(map[string]*blockVotes, valSet.Size()),
peerMaj23s: make(map[string]BlockID),
chainID: chainID,
height: height,
round: round,
signedMsgType: signedMsgType,
valSet: valSet,
requireExtensions: requireExtensions,
votesBitArray: bits.NewBitArray(valSet.Size()),
votes: make([]*Vote, valSet.Size()),
sum: 0,
maj23: nil,
votesByBlock: make(map[string]*blockVotes, valSet.Size()),
peerMaj23s: make(map[string]BlockID),
}
}
@@ -194,8 +196,20 @@ func (voteSet *VoteSet) addVote(vote *Vote) (added bool, err error) {
}
// Check signature.
if err := vote.VerifyWithExtension(voteSet.chainID, val.PubKey); err != nil {
return false, fmt.Errorf("failed to verify vote with ChainID %s and PubKey %s: %w", voteSet.chainID, val.PubKey, err)
if voteSet.requireExtensions {
if err := vote.VerifyWithExtension(voteSet.chainID, val.PubKey); err != nil {
return false, fmt.Errorf("failed to verify vote with ChainID %s and PubKey %s: %w", voteSet.chainID, val.PubKey, err)
}
} else {
if len(vote.ExtensionSignature) != 0 {
if err := vote.VerifyWithExtension(voteSet.chainID, val.PubKey); err != nil {
return false, fmt.Errorf("failed to verify vote with ChainID %s and PubKey %s: %w", voteSet.chainID, val.PubKey, err)
}
} else {
if err := vote.Verify(voteSet.chainID, val.PubKey); err != nil {
return false, fmt.Errorf("failed to verify vote with ChainID %s and PubKey %s: %w", voteSet.chainID, val.PubKey, err)
}
}
}
// Add vote and get conflicting vote if any.

View File

@@ -510,7 +510,7 @@ func randVoteSet(
) (*VoteSet, *ValidatorSet, []PrivValidator) {
t.Helper()
valSet, privValidators := randValidatorPrivValSet(ctx, t, numValidators, votingPower)
return NewVoteSet("test_chain_id", height, round, signedMsgType, valSet), valSet, privValidators
return NewVoteSet("test_chain_id", height, round, signedMsgType, valSet, false), valSet, privValidators
}
func deterministicVoteSet(
@@ -523,7 +523,7 @@ func deterministicVoteSet(
) (*VoteSet, *ValidatorSet, []PrivValidator) {
t.Helper()
valSet, privValidators := deterministicValidatorSet(ctx, t)
return NewVoteSet("test_chain_id", height, round, signedMsgType, valSet), valSet, privValidators
return NewVoteSet("test_chain_id", height, round, signedMsgType, valSet, false), valSet, privValidators
}
func randValidatorPrivValSet(ctx context.Context, t testing.TB, numValidators int, votingPower int64) (*ValidatorSet, []PrivValidator) {