Browse Source

[R4R] Fixed sized and reordered fields for Vote/Proposal/Heartbeat SignBytes (#2598)

* WIP: switching to fixed offsets for SignBytes

* add version field to sign bytes and update order

* more comments on test-cases and add a tc with a chainID

* remove amino:"write_empty" tag

- it doesn't affect if default fixed size fields ((u)int64) are
written or not
- add comment about int->int64 casting

* update CHANGELOG_PENDING

* update documentation

* add back link to issue #1622 in documentation

* remove JSON tags and add (failing test-case)

* fix failing test

* update test-vectors due to added `Type` field

* change Type field from string to byte and add new type alias

- SignedMsgType replaces VoteTypePrevote, VoteTypePrecommit and adds new
ProposalType to separate votes from proposal when signed

- update test-vectors

* fix remains from rebasing

* use SignMessageType instead of byte everywhere

* fixes from review
bucky/node-info-substructs
Ismail Khoffi 6 years ago
committed by Ethan Buchman
parent
commit
8888595b94
28 changed files with 407 additions and 292 deletions
  1. +13
    -7
      CHANGELOG_PENDING.md
  2. +2
    -2
      consensus/byzantine_test.go
  3. +5
    -5
      consensus/common_test.go
  4. +22
    -22
      consensus/reactor.go
  5. +1
    -1
      consensus/replay_test.go
  6. +15
    -15
      consensus/state.go
  7. +98
    -98
      consensus/state_test.go
  8. +9
    -9
      consensus/types/height_vote_set.go
  9. +1
    -1
      consensus/types/height_vote_set_test.go
  10. +3
    -2
      docs/spec/blockchain/blockchain.md
  11. +12
    -9
      docs/spec/blockchain/encoding.md
  12. +1
    -1
      lite/helpers.go
  13. +2
    -2
      privval/priv_validator.go
  14. +3
    -3
      privval/priv_validator_test.go
  15. +2
    -2
      privval/socket_test.go
  16. +2
    -2
      state/execution_test.go
  17. +3
    -3
      types/block.go
  18. +8
    -8
      types/block_test.go
  19. +42
    -38
      types/canonical.go
  20. +1
    -1
      types/evidence_test.go
  21. +27
    -0
      types/signed_msg_type.go
  22. +1
    -1
      types/test_util.go
  23. +2
    -2
      types/validator_set.go
  24. +1
    -1
      types/validator_set_test.go
  25. +10
    -28
      types/vote.go
  26. +6
    -6
      types/vote_set.go
  27. +15
    -15
      types/vote_set_test.go
  28. +100
    -8
      types/vote_test.go

+ 13
- 7
CHANGELOG_PENDING.md View File

@ -24,11 +24,16 @@ BREAKING CHANGES:
* [types] \#2298 Remove `Index` and `Total` fields from `TxProof`.
* [crypto/merkle & lite] \#2298 Various changes to accomodate General Merkle trees
* [crypto/merkle] \#2595 Remove all Hasher objects in favor of byte slices
* [types] \#2598 `VoteTypeXxx` are now
* Blockchain Protocol
* [types] \#2459 `Vote`/`Proposal`/`Heartbeat` use amino encoding instead of JSON in `SignBytes`.
* [types] Update SignBytes for `Vote`/`Proposal`/`Heartbeat`:
* \#2459 Use amino encoding instead of JSON in `SignBytes`.
* \#2598 Reorder fields and use fixed sized encoding.
* \#2598 Change `Type` field fromt `string` to `byte` and use new
`SignedMsgType` to enumerate.
* [types] \#2512 Remove the pubkey field from the validator hash
* [state] \#2587 require block.Time of the fist block to be genesis time
* [state] \#2587 Require block.Time of the fist block to be genesis time
* P2P Protocol
@ -37,9 +42,10 @@ FEATURES:
- [abci] \#2557 Add `Codespace` field to `Response{CheckTx, DeliverTx, Query}`
IMPROVEMENTS:
- [consensus] [\#2169](https://github.com/cosmos/cosmos-sdk/issues/2169) add additional metrics
- [p2p] [\#2169](https://github.com/cosmos/cosmos-sdk/issues/2169) add additional metrics
- [config] \#2232 added ValidateBasic method, which performs basic checks
- Additional Metrics
- [consensus] [\#2169](https://github.com/cosmos/cosmos-sdk/issues/2169)
- [p2p] [\#2169](https://github.com/cosmos/cosmos-sdk/issues/2169)
- [config] \#2232 Added ValidateBasic method, which performs basic checks
- [crypto/ed25519] \#2558 Switch to use latest `golang.org/x/crypto` through our fork at
github.com/tendermint/crypto
- [tools] \#2238 Binary dependencies are now locked to a specific git commit
@ -50,8 +56,8 @@ BUG FIXES:
- [node] \#2434 Make node respond to signal interrupts while sleeping for genesis time
- [consensus] [\#1690](https://github.com/tendermint/tendermint/issues/1690) wait for
timeoutPrecommit before starting next round
- [consensus] [\#1745](https://github.com/tendermint/tendermint/issues/1745) wait for
Proposal or timeoutProposal before entering prevote
- [consensus] [\#1745](https://github.com/tendermint/tendermint/issues/1745) wait for
Proposal or timeoutProposal before entering prevote
- [evidence] \#2515 fix db iter leak (@goolAdapter)
- [common/bit_array] Fixed a bug in the `Or` function
- [common/bit_array] Fixed a bug in the `Sub` function (@james-ray)


+ 2
- 2
consensus/byzantine_test.go View File

@ -226,8 +226,8 @@ func sendProposalAndParts(height int64, round int, cs *ConsensusState, peer p2p.
// votes
cs.mtx.Lock()
prevote, _ := cs.signVote(types.VoteTypePrevote, blockHash, parts.Header())
precommit, _ := cs.signVote(types.VoteTypePrecommit, blockHash, parts.Header())
prevote, _ := cs.signVote(types.PrevoteType, blockHash, parts.Header())
precommit, _ := cs.signVote(types.PrecommitType, blockHash, parts.Header())
cs.mtx.Unlock()
peer.Send(VoteChannel, cdc.MustMarshalBinaryBare(&VoteMessage{prevote}))


+ 5
- 5
consensus/common_test.go View File

@ -71,7 +71,7 @@ func NewValidatorStub(privValidator types.PrivValidator, valIndex int) *validato
}
}
func (vs *validatorStub) signVote(voteType byte, hash []byte, header types.PartSetHeader) (*types.Vote, error) {
func (vs *validatorStub) signVote(voteType types.SignedMsgType, hash []byte, header types.PartSetHeader) (*types.Vote, error) {
vote := &types.Vote{
ValidatorIndex: vs.Index,
ValidatorAddress: vs.PrivValidator.GetAddress(),
@ -86,7 +86,7 @@ func (vs *validatorStub) signVote(voteType byte, hash []byte, header types.PartS
}
// Sign vote for type/hash/header
func signVote(vs *validatorStub, voteType byte, hash []byte, header types.PartSetHeader) *types.Vote {
func signVote(vs *validatorStub, voteType types.SignedMsgType, hash []byte, header types.PartSetHeader) *types.Vote {
v, err := vs.signVote(voteType, hash, header)
if err != nil {
panic(fmt.Errorf("failed to sign vote: %v", err))
@ -94,7 +94,7 @@ func signVote(vs *validatorStub, voteType byte, hash []byte, header types.PartSe
return v
}
func signVotes(voteType byte, hash []byte, header types.PartSetHeader, vss ...*validatorStub) []*types.Vote {
func signVotes(voteType types.SignedMsgType, hash []byte, header types.PartSetHeader, vss ...*validatorStub) []*types.Vote {
votes := make([]*types.Vote, len(vss))
for i, vs := range vss {
votes[i] = signVote(vs, voteType, hash, header)
@ -144,7 +144,7 @@ func addVotes(to *ConsensusState, votes ...*types.Vote) {
}
}
func signAddVotes(to *ConsensusState, voteType byte, hash []byte, header types.PartSetHeader, vss ...*validatorStub) {
func signAddVotes(to *ConsensusState, voteType types.SignedMsgType, hash []byte, header types.PartSetHeader, vss ...*validatorStub) {
votes := signVotes(voteType, hash, header, vss...)
addVotes(to, votes...)
}
@ -463,7 +463,7 @@ func ensureNewUnlock(unlockCh <-chan interface{}, height int64, round int) {
}
func ensureVote(voteCh <-chan interface{}, height int64, round int,
voteType byte) {
voteType types.SignedMsgType) {
select {
case <-time.After(ensureTimeout):
panic("Timeout expired while waiting for NewVote event")


+ 22
- 22
consensus/reactor.go View File

@ -237,9 +237,9 @@ func (conR *ConsensusReactor) Receive(chID byte, src p2p.Peer, msgBytes []byte)
// (and consequently shows which we don't have)
var ourVotes *cmn.BitArray
switch msg.Type {
case types.VoteTypePrevote:
case types.PrevoteType:
ourVotes = votes.Prevotes(msg.Round).BitArrayByBlockID(msg.BlockID)
case types.VoteTypePrecommit:
case types.PrecommitType:
ourVotes = votes.Precommits(msg.Round).BitArrayByBlockID(msg.BlockID)
default:
conR.Logger.Error("Bad VoteSetBitsMessage field Type")
@ -317,9 +317,9 @@ func (conR *ConsensusReactor) Receive(chID byte, src p2p.Peer, msgBytes []byte)
if height == msg.Height {
var ourVotes *cmn.BitArray
switch msg.Type {
case types.VoteTypePrevote:
case types.PrevoteType:
ourVotes = votes.Prevotes(msg.Round).BitArrayByBlockID(msg.BlockID)
case types.VoteTypePrecommit:
case types.PrecommitType:
ourVotes = votes.Precommits(msg.Round).BitArrayByBlockID(msg.BlockID)
default:
conR.Logger.Error("Bad VoteSetBitsMessage field Type")
@ -739,7 +739,7 @@ OUTER_LOOP:
peer.TrySend(StateChannel, cdc.MustMarshalBinaryBare(&VoteSetMaj23Message{
Height: prs.Height,
Round: prs.Round,
Type: types.VoteTypePrevote,
Type: types.PrevoteType,
BlockID: maj23,
}))
time.Sleep(conR.conS.config.PeerQueryMaj23SleepDuration)
@ -756,7 +756,7 @@ OUTER_LOOP:
peer.TrySend(StateChannel, cdc.MustMarshalBinaryBare(&VoteSetMaj23Message{
Height: prs.Height,
Round: prs.Round,
Type: types.VoteTypePrecommit,
Type: types.PrecommitType,
BlockID: maj23,
}))
time.Sleep(conR.conS.config.PeerQueryMaj23SleepDuration)
@ -773,7 +773,7 @@ OUTER_LOOP:
peer.TrySend(StateChannel, cdc.MustMarshalBinaryBare(&VoteSetMaj23Message{
Height: prs.Height,
Round: prs.ProposalPOLRound,
Type: types.VoteTypePrevote,
Type: types.PrevoteType,
BlockID: maj23,
}))
time.Sleep(conR.conS.config.PeerQueryMaj23SleepDuration)
@ -792,7 +792,7 @@ OUTER_LOOP:
peer.TrySend(StateChannel, cdc.MustMarshalBinaryBare(&VoteSetMaj23Message{
Height: prs.Height,
Round: commit.Round(),
Type: types.VoteTypePrecommit,
Type: types.PrecommitType,
BlockID: commit.BlockID,
}))
time.Sleep(conR.conS.config.PeerQueryMaj23SleepDuration)
@ -1022,7 +1022,7 @@ func (ps *PeerState) PickVoteToSend(votes types.VoteSetReader) (vote *types.Vote
return nil, false
}
height, round, type_, size := votes.Height(), votes.Round(), votes.Type(), votes.Size()
height, round, type_, size := votes.Height(), votes.Round(), types.SignedMsgType(votes.Type()), votes.Size()
// Lazily set data using 'votes'.
if votes.IsCommit() {
@ -1041,7 +1041,7 @@ func (ps *PeerState) PickVoteToSend(votes types.VoteSetReader) (vote *types.Vote
return nil, false
}
func (ps *PeerState) getVoteBitArray(height int64, round int, type_ byte) *cmn.BitArray {
func (ps *PeerState) getVoteBitArray(height int64, round int, type_ types.SignedMsgType) *cmn.BitArray {
if !types.IsVoteTypeValid(type_) {
return nil
}
@ -1049,25 +1049,25 @@ func (ps *PeerState) getVoteBitArray(height int64, round int, type_ byte) *cmn.B
if ps.PRS.Height == height {
if ps.PRS.Round == round {
switch type_ {
case types.VoteTypePrevote:
case types.PrevoteType:
return ps.PRS.Prevotes
case types.VoteTypePrecommit:
case types.PrecommitType:
return ps.PRS.Precommits
}
}
if ps.PRS.CatchupCommitRound == round {
switch type_ {
case types.VoteTypePrevote:
case types.PrevoteType:
return nil
case types.VoteTypePrecommit:
case types.PrecommitType:
return ps.PRS.CatchupCommit
}
}
if ps.PRS.ProposalPOLRound == round {
switch type_ {
case types.VoteTypePrevote:
case types.PrevoteType:
return ps.PRS.ProposalPOL
case types.VoteTypePrecommit:
case types.PrecommitType:
return nil
}
}
@ -1076,9 +1076,9 @@ func (ps *PeerState) getVoteBitArray(height int64, round int, type_ byte) *cmn.B
if ps.PRS.Height == height+1 {
if ps.PRS.LastCommitRound == round {
switch type_ {
case types.VoteTypePrevote:
case types.PrevoteType:
return nil
case types.VoteTypePrecommit:
case types.PrecommitType:
return ps.PRS.LastCommit
}
}
@ -1187,7 +1187,7 @@ func (ps *PeerState) SetHasVote(vote *types.Vote) {
ps.setHasVote(vote.Height, vote.Round, vote.Type, vote.ValidatorIndex)
}
func (ps *PeerState) setHasVote(height int64, round int, type_ byte, index int) {
func (ps *PeerState) setHasVote(height int64, round int, type_ types.SignedMsgType, index int) {
logger := ps.logger.With("peerH/R", fmt.Sprintf("%d/%d", ps.PRS.Height, ps.PRS.Round), "H/R", fmt.Sprintf("%d/%d", height, round))
logger.Debug("setHasVote", "type", type_, "index", index)
@ -1453,7 +1453,7 @@ func (m *VoteMessage) String() string {
type HasVoteMessage struct {
Height int64
Round int
Type byte
Type types.SignedMsgType
Index int
}
@ -1468,7 +1468,7 @@ func (m *HasVoteMessage) String() string {
type VoteSetMaj23Message struct {
Height int64
Round int
Type byte
Type types.SignedMsgType
BlockID types.BlockID
}
@ -1483,7 +1483,7 @@ func (m *VoteSetMaj23Message) String() string {
type VoteSetBitsMessage struct {
Height int64
Round int
Type byte
Type types.SignedMsgType
BlockID types.BlockID
Votes *cmn.BitArray
}


+ 1
- 1
consensus/replay_test.go View File

@ -542,7 +542,7 @@ func makeBlockchainFromWAL(wal WAL) ([]*types.Block, []*types.Commit, error) {
return nil, nil, err
}
case *types.Vote:
if p.Type == types.VoteTypePrecommit {
if p.Type == types.PrecommitType {
thisBlockCommit = &types.Commit{
BlockID: p.BlockID,
Precommits: []*types.Vote{p},


+ 15
- 15
consensus/state.go View File

@ -460,7 +460,7 @@ func (cs *ConsensusState) reconstructLastCommit(state sm.State) {
return
}
seenCommit := cs.blockStore.LoadSeenCommit(state.LastBlockHeight)
lastPrecommits := types.NewVoteSet(state.ChainID, state.LastBlockHeight, seenCommit.Round(), types.VoteTypePrecommit, state.LastValidators)
lastPrecommits := types.NewVoteSet(state.ChainID, state.LastBlockHeight, seenCommit.Round(), types.PrecommitType, state.LastValidators)
for _, precommit := range seenCommit.Precommits {
if precommit == nil {
continue
@ -1021,14 +1021,14 @@ func (cs *ConsensusState) defaultDoPrevote(height int64, round int) {
// If a block is locked, prevote that.
if cs.LockedBlock != nil {
logger.Info("enterPrevote: Block was locked")
cs.signAddVote(types.VoteTypePrevote, cs.LockedBlock.Hash(), cs.LockedBlockParts.Header())
cs.signAddVote(types.PrevoteType, cs.LockedBlock.Hash(), cs.LockedBlockParts.Header())
return
}
// If ProposalBlock is nil, prevote nil.
if cs.ProposalBlock == nil {
logger.Info("enterPrevote: ProposalBlock is nil")
cs.signAddVote(types.VoteTypePrevote, nil, types.PartSetHeader{})
cs.signAddVote(types.PrevoteType, nil, types.PartSetHeader{})
return
}
@ -1037,7 +1037,7 @@ func (cs *ConsensusState) defaultDoPrevote(height int64, round int) {
if err != nil {
// ProposalBlock is invalid, prevote nil.
logger.Error("enterPrevote: ProposalBlock is invalid", "err", err)
cs.signAddVote(types.VoteTypePrevote, nil, types.PartSetHeader{})
cs.signAddVote(types.PrevoteType, nil, types.PartSetHeader{})
return
}
@ -1045,7 +1045,7 @@ func (cs *ConsensusState) defaultDoPrevote(height int64, round int) {
// NOTE: the proposal signature is validated when it is received,
// and the proposal block parts are validated as they are received (against the merkle hash in the proposal)
logger.Info("enterPrevote: ProposalBlock is valid")
cs.signAddVote(types.VoteTypePrevote, cs.ProposalBlock.Hash(), cs.ProposalBlockParts.Header())
cs.signAddVote(types.PrevoteType, cs.ProposalBlock.Hash(), cs.ProposalBlockParts.Header())
}
// Enter: any +2/3 prevotes at next round.
@ -1103,7 +1103,7 @@ func (cs *ConsensusState) enterPrecommit(height int64, round int) {
} else {
logger.Info("enterPrecommit: No +2/3 prevotes during enterPrecommit. Precommitting nil.")
}
cs.signAddVote(types.VoteTypePrecommit, nil, types.PartSetHeader{})
cs.signAddVote(types.PrecommitType, nil, types.PartSetHeader{})
return
}
@ -1127,7 +1127,7 @@ func (cs *ConsensusState) enterPrecommit(height int64, round int) {
cs.LockedBlockParts = nil
cs.eventBus.PublishEventUnlock(cs.RoundStateEvent())
}
cs.signAddVote(types.VoteTypePrecommit, nil, types.PartSetHeader{})
cs.signAddVote(types.PrecommitType, nil, types.PartSetHeader{})
return
}
@ -1138,7 +1138,7 @@ func (cs *ConsensusState) enterPrecommit(height int64, round int) {
logger.Info("enterPrecommit: +2/3 prevoted locked block. Relocking")
cs.LockedRound = round
cs.eventBus.PublishEventRelock(cs.RoundStateEvent())
cs.signAddVote(types.VoteTypePrecommit, blockID.Hash, blockID.PartsHeader)
cs.signAddVote(types.PrecommitType, blockID.Hash, blockID.PartsHeader)
return
}
@ -1153,7 +1153,7 @@ func (cs *ConsensusState) enterPrecommit(height int64, round int) {
cs.LockedBlock = cs.ProposalBlock
cs.LockedBlockParts = cs.ProposalBlockParts
cs.eventBus.PublishEventLock(cs.RoundStateEvent())
cs.signAddVote(types.VoteTypePrecommit, blockID.Hash, blockID.PartsHeader)
cs.signAddVote(types.PrecommitType, blockID.Hash, blockID.PartsHeader)
return
}
@ -1169,7 +1169,7 @@ func (cs *ConsensusState) enterPrecommit(height int64, round int) {
cs.ProposalBlockParts = types.NewPartSetFromHeader(blockID.PartsHeader)
}
cs.eventBus.PublishEventUnlock(cs.RoundStateEvent())
cs.signAddVote(types.VoteTypePrecommit, nil, types.PartSetHeader{})
cs.signAddVote(types.PrecommitType, nil, types.PartSetHeader{})
}
// Enter: any +2/3 precommits for next round.
@ -1550,7 +1550,7 @@ func (cs *ConsensusState) addVote(vote *types.Vote, peerID p2p.ID) (added bool,
// A precommit for the previous height?
// These come in while we wait timeoutCommit
if vote.Height+1 == cs.Height {
if !(cs.Step == cstypes.RoundStepNewHeight && vote.Type == types.VoteTypePrecommit) {
if !(cs.Step == cstypes.RoundStepNewHeight && vote.Type == types.PrecommitType) {
// TODO: give the reason ..
// fmt.Errorf("tryAddVote: Wrong height, not a LastCommit straggler commit.")
return added, ErrVoteHeightMismatch
@ -1593,7 +1593,7 @@ func (cs *ConsensusState) addVote(vote *types.Vote, peerID p2p.ID) (added bool,
cs.evsw.FireEvent(types.EventVote, vote)
switch vote.Type {
case types.VoteTypePrevote:
case types.PrevoteType:
prevotes := cs.Votes.Prevotes(vote.Round)
cs.Logger.Info("Added to prevote", "vote", vote, "prevotes", prevotes.StringShort())
@ -1650,7 +1650,7 @@ func (cs *ConsensusState) addVote(vote *types.Vote, peerID p2p.ID) (added bool,
}
}
case types.VoteTypePrecommit:
case types.PrecommitType:
precommits := cs.Votes.Precommits(vote.Round)
cs.Logger.Info("Added to precommit", "vote", vote, "precommits", precommits.StringShort())
@ -1679,7 +1679,7 @@ func (cs *ConsensusState) addVote(vote *types.Vote, peerID p2p.ID) (added bool,
return
}
func (cs *ConsensusState) signVote(type_ byte, hash []byte, header types.PartSetHeader) (*types.Vote, error) {
func (cs *ConsensusState) signVote(type_ types.SignedMsgType, hash []byte, header types.PartSetHeader) (*types.Vote, error) {
addr := cs.privValidator.GetAddress()
valIndex, _ := cs.Validators.GetByAddress(addr)
@ -1714,7 +1714,7 @@ func (cs *ConsensusState) voteTime() time.Time {
}
// sign the vote and publish on internalMsgQueue
func (cs *ConsensusState) signAddVote(type_ byte, hash []byte, header types.PartSetHeader) *types.Vote {
func (cs *ConsensusState) signAddVote(type_ types.SignedMsgType, hash []byte, header types.PartSetHeader) *types.Vote {
// if we don't have a key or we're not in the validator set, do nothing
if cs.privValidator == nil || !cs.Validators.HasAddress(cs.privValidator.GetAddress()) {
return nil


+ 98
- 98
consensus/state_test.go View File

@ -81,7 +81,7 @@ func TestStateProposerSelection0(t *testing.T) {
ensureNewProposal(proposalCh, height, round)
rs := cs1.GetRoundState()
signAddVotes(cs1, types.VoteTypePrecommit, rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header(), vss[1:]...)
signAddVotes(cs1, types.PrecommitType, rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header(), vss[1:]...)
// Wait for new round so next validator is set.
ensureNewRound(newRoundCh, height+1, 0)
@ -116,7 +116,7 @@ func TestStateProposerSelection2(t *testing.T) {
}
rs := cs1.GetRoundState()
signAddVotes(cs1, types.VoteTypePrecommit, nil, rs.ProposalBlockParts.Header(), vss[1:]...)
signAddVotes(cs1, types.PrecommitType, nil, rs.ProposalBlockParts.Header(), vss[1:]...)
ensureNewRound(newRoundCh, height, i+round+1) // wait for the new round event each round
incrementRound(vss[1:]...)
}
@ -214,16 +214,17 @@ func TestStateBadProposal(t *testing.T) {
ensureNewProposal(proposalCh, height, round)
// wait for prevote
ensureVote(voteCh, height, round, types.VoteTypePrevote)
ensureVote(voteCh, height, round, types.PrevoteType)
validatePrevote(t, cs1, round, vss[0], nil)
// add bad prevote from vs2 and wait for it
signAddVotes(cs1, types.VoteTypePrevote, propBlock.Hash(), propBlock.MakePartSet(partSize).Header(), vs2)
ensureVote(voteCh, height, round, types.VoteTypePrevote)
signAddVotes(cs1, types.PrevoteType, propBlock.Hash(), propBlock.MakePartSet(partSize).Header(), vs2)
ensureVote(voteCh, height, round, types.PrevoteType)
// wait for precommit
ensureVote(voteCh, height, round, types.VoteTypePrecommit)
ensureVote(voteCh, height, round, types.PrecommitType)
validatePrecommit(t, cs1, round, 0, vss[0], nil, nil)
signAddVotes(cs1, types.PrecommitType, propBlock.Hash(), propBlock.MakePartSet(partSize).Header(), vs2)
}
//----------------------------------------------------------------------------------------------------
@ -254,10 +255,10 @@ func TestStateFullRound1(t *testing.T) {
ensureNewProposal(propCh, height, round)
propBlockHash := cs.GetRoundState().ProposalBlock.Hash()
ensureVote(voteCh, height, round, types.VoteTypePrevote) // wait for prevote
ensureVote(voteCh, height, round, types.PrevoteType) // wait for prevote
validatePrevote(t, cs, round, vss[0], propBlockHash)
ensureVote(voteCh, height, round, types.VoteTypePrecommit) // wait for precommit
ensureVote(voteCh, height, round, types.PrecommitType) // wait for precommit
// we're going to roll right into new height
ensureNewRound(newRoundCh, height+1, 0)
@ -275,8 +276,8 @@ func TestStateFullRoundNil(t *testing.T) {
cs.enterPrevote(height, round)
cs.startRoutines(4)
ensureVote(voteCh, height, round, types.VoteTypePrevote) // prevote
ensureVote(voteCh, height, round, types.VoteTypePrecommit) // precommit
ensureVote(voteCh, height, round, types.PrevoteType) // prevote
ensureVote(voteCh, height, round, types.PrecommitType) // precommit
// should prevote and precommit nil
validatePrevoteAndPrecommit(t, cs, round, 0, vss[0], nil, nil)
@ -295,25 +296,25 @@ func TestStateFullRound2(t *testing.T) {
// start round and wait for propose and prevote
startTestRound(cs1, height, round)
ensureVote(voteCh, height, round, types.VoteTypePrevote) // prevote
ensureVote(voteCh, height, round, types.PrevoteType) // prevote
// we should be stuck in limbo waiting for more prevotes
rs := cs1.GetRoundState()
propBlockHash, propPartsHeader := rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header()
// prevote arrives from vs2:
signAddVotes(cs1, types.VoteTypePrevote, propBlockHash, propPartsHeader, vs2)
ensureVote(voteCh, height, round, types.VoteTypePrevote) // prevote
signAddVotes(cs1, types.PrevoteType, propBlockHash, propPartsHeader, vs2)
ensureVote(voteCh, height, round, types.PrevoteType) // prevote
ensureVote(voteCh, height, round, types.VoteTypePrecommit) //precommit
ensureVote(voteCh, height, round, types.PrecommitType) //precommit
// the proposed block should now be locked and our precommit added
validatePrecommit(t, cs1, 0, 0, vss[0], propBlockHash, propBlockHash)
// we should be stuck in limbo waiting for more precommits
// precommit arrives from vs2:
signAddVotes(cs1, types.VoteTypePrecommit, propBlockHash, propPartsHeader, vs2)
ensureVote(voteCh, height, round, types.VoteTypePrecommit)
signAddVotes(cs1, types.PrecommitType, propBlockHash, propPartsHeader, vs2)
ensureVote(voteCh, height, round, types.PrecommitType)
// wait to finish commit, propose in next height
ensureNewBlock(newBlockCh, height)
@ -352,14 +353,14 @@ func TestStateLockNoPOL(t *testing.T) {
theBlockHash := roundState.ProposalBlock.Hash()
thePartSetHeader := roundState.ProposalBlockParts.Header()
ensureVote(voteCh, height, round, types.VoteTypePrevote) // prevote
ensureVote(voteCh, height, round, types.PrevoteType) // prevote
// we should now be stuck in limbo forever, waiting for more prevotes
// prevote arrives from vs2:
signAddVotes(cs1, types.VoteTypePrevote, theBlockHash, thePartSetHeader, vs2)
ensureVote(voteCh, height, round, types.VoteTypePrevote) // prevote
signAddVotes(cs1, types.PrevoteType, theBlockHash, thePartSetHeader, vs2)
ensureVote(voteCh, height, round, types.PrevoteType) // prevote
ensureVote(voteCh, height, round, types.VoteTypePrecommit) // precommit
ensureVote(voteCh, height, round, types.PrecommitType) // precommit
// the proposed block should now be locked and our precommit added
validatePrecommit(t, cs1, round, round, vss[0], theBlockHash, theBlockHash)
@ -368,8 +369,8 @@ func TestStateLockNoPOL(t *testing.T) {
hash := make([]byte, len(theBlockHash))
copy(hash, theBlockHash)
hash[0] = byte((hash[0] + 1) % 255)
signAddVotes(cs1, types.VoteTypePrecommit, hash, thePartSetHeader, vs2)
ensureVote(voteCh, height, round, types.VoteTypePrecommit) // precommit
signAddVotes(cs1, types.PrecommitType, hash, thePartSetHeader, vs2)
ensureVote(voteCh, height, round, types.PrecommitType) // precommit
// (note we're entering precommit for a second time this round)
// but with invalid args. then we enterPrecommitWait, and the timeout to new round
@ -396,26 +397,26 @@ func TestStateLockNoPOL(t *testing.T) {
}
// wait to finish prevote
ensureVote(voteCh, height, round, types.VoteTypePrevote)
ensureVote(voteCh, height, round, types.PrevoteType)
// we should have prevoted our locked block
validatePrevote(t, cs1, round, vss[0], rs.LockedBlock.Hash())
// add a conflicting prevote from the other validator
signAddVotes(cs1, types.VoteTypePrevote, hash, rs.LockedBlock.MakePartSet(partSize).Header(), vs2)
ensureVote(voteCh, height, round, types.VoteTypePrevote)
signAddVotes(cs1, types.PrevoteType, hash, rs.LockedBlock.MakePartSet(partSize).Header(), vs2)
ensureVote(voteCh, height, round, types.PrevoteType)
// now we're going to enter prevote again, but with invalid args
// and then prevote wait, which should timeout. then wait for precommit
ensureNewTimeout(timeoutWaitCh, height, round, cs1.config.TimeoutPrevote.Nanoseconds())
ensureVote(voteCh, height, round, types.VoteTypePrecommit) // precommit
ensureVote(voteCh, height, round, types.PrecommitType) // precommit
// the proposed block should still be locked and our precommit added
// we should precommit nil and be locked on the proposal
validatePrecommit(t, cs1, round, 0, vss[0], nil, theBlockHash)
// add conflicting precommit from vs2
signAddVotes(cs1, types.VoteTypePrecommit, hash, rs.LockedBlock.MakePartSet(partSize).Header(), vs2)
ensureVote(voteCh, height, round, types.VoteTypePrecommit)
signAddVotes(cs1, types.PrecommitType, hash, rs.LockedBlock.MakePartSet(partSize).Header(), vs2)
ensureVote(voteCh, height, round, types.PrecommitType)
// (note we're entering precommit for a second time this round, but with invalid args
// then we enterPrecommitWait and timeout into NewRound
@ -438,19 +439,19 @@ func TestStateLockNoPOL(t *testing.T) {
panic(fmt.Sprintf("Expected proposal block to be locked block. Got %v, Expected %v", rs.ProposalBlock, rs.LockedBlock))
}
ensureVote(voteCh, height, round, types.VoteTypePrevote) // prevote
ensureVote(voteCh, height, round, types.PrevoteType) // prevote
validatePrevote(t, cs1, round, vss[0], rs.LockedBlock.Hash())
signAddVotes(cs1, types.VoteTypePrevote, hash, rs.ProposalBlock.MakePartSet(partSize).Header(), vs2)
ensureVote(voteCh, height, round, types.VoteTypePrevote)
signAddVotes(cs1, types.PrevoteType, hash, rs.ProposalBlock.MakePartSet(partSize).Header(), vs2)
ensureVote(voteCh, height, round, types.PrevoteType)
ensureNewTimeout(timeoutWaitCh, height, round, cs1.config.TimeoutPrevote.Nanoseconds())
ensureVote(voteCh, height, round, types.VoteTypePrecommit) // precommit
ensureVote(voteCh, height, round, types.PrecommitType) // precommit
validatePrecommit(t, cs1, round, 0, vss[0], nil, theBlockHash) // precommit nil but be locked on proposal
signAddVotes(cs1, types.VoteTypePrecommit, hash, rs.ProposalBlock.MakePartSet(partSize).Header(), vs2) // NOTE: conflicting precommits at same height
ensureVote(voteCh, height, round, types.VoteTypePrecommit)
signAddVotes(cs1, types.PrecommitType, hash, rs.ProposalBlock.MakePartSet(partSize).Header(), vs2) // NOTE: conflicting precommits at same height
ensureVote(voteCh, height, round, types.PrecommitType)
ensureNewTimeout(timeoutWaitCh, height, round, cs1.config.TimeoutPrecommit.Nanoseconds())
@ -477,20 +478,20 @@ func TestStateLockNoPOL(t *testing.T) {
}
ensureNewProposal(proposalCh, height, round)
ensureVote(voteCh, height, round, types.VoteTypePrevote) // prevote
ensureVote(voteCh, height, round, types.PrevoteType) // prevote
// prevote for locked block (not proposal)
validatePrevote(t, cs1, 3, vss[0], cs1.LockedBlock.Hash())
// prevote for proposed block
signAddVotes(cs1, types.VoteTypePrevote, propBlock.Hash(), propBlock.MakePartSet(partSize).Header(), vs2)
ensureVote(voteCh, height, round, types.VoteTypePrevote)
signAddVotes(cs1, types.PrevoteType, propBlock.Hash(), propBlock.MakePartSet(partSize).Header(), vs2)
ensureVote(voteCh, height, round, types.PrevoteType)
ensureNewTimeout(timeoutWaitCh, height, round, cs1.config.TimeoutPrevote.Nanoseconds())
ensureVote(voteCh, height, round, types.VoteTypePrecommit)
ensureVote(voteCh, height, round, types.PrecommitType)
validatePrecommit(t, cs1, round, 0, vss[0], nil, theBlockHash) // precommit nil but locked on proposal
signAddVotes(cs1, types.VoteTypePrecommit, propBlock.Hash(), propBlock.MakePartSet(partSize).Header(), vs2) // NOTE: conflicting precommits at same height
ensureVote(voteCh, height, round, types.VoteTypePrecommit)
signAddVotes(cs1, types.PrecommitType, propBlock.Hash(), propBlock.MakePartSet(partSize).Header(), vs2) // NOTE: conflicting precommits at same height
ensureVote(voteCh, height, round, types.PrecommitType)
}
// 4 vals, one precommits, other 3 polka at next round, so we unlock and precomit the polka
@ -524,17 +525,17 @@ func TestStateLockPOLRelock(t *testing.T) {
theBlockHash := rs.ProposalBlock.Hash()
theBlockParts := rs.ProposalBlockParts.Header()
ensureVote(voteCh, height, round, types.VoteTypePrevote) // prevote
ensureVote(voteCh, height, round, types.PrevoteType) // prevote
signAddVotes(cs1, types.VoteTypePrevote, theBlockHash, theBlockParts, vs2, vs3, vs4)
signAddVotes(cs1, types.PrevoteType, theBlockHash, theBlockParts, vs2, vs3, vs4)
ensureVote(voteCh, height, round, types.VoteTypePrecommit) // our precommit
ensureVote(voteCh, height, round, types.PrecommitType) // our precommit
// the proposed block should now be locked and our precommit added
validatePrecommit(t, cs1, round, round, vss[0], theBlockHash, theBlockHash)
// add precommits from the rest
signAddVotes(cs1, types.VoteTypePrecommit, nil, types.PartSetHeader{}, vs2, vs4)
signAddVotes(cs1, types.VoteTypePrecommit, theBlockHash, theBlockParts, vs3)
signAddVotes(cs1, types.PrecommitType, nil, types.PartSetHeader{}, vs2, vs4)
signAddVotes(cs1, types.PrecommitType, theBlockHash, theBlockParts, vs3)
// before we timeout to the new round set the new proposal
prop, propBlock := decideProposal(cs1, vs2, vs2.Height, vs2.Round+1)
@ -566,17 +567,17 @@ func TestStateLockPOLRelock(t *testing.T) {
ensureNewProposal(proposalCh, height, round)
// go to prevote, prevote for locked block (not proposal), move on
ensureVote(voteCh, height, round, types.VoteTypePrevote)
ensureVote(voteCh, height, round, types.PrevoteType)
validatePrevote(t, cs1, round, vss[0], theBlockHash)
// now lets add prevotes from everyone else for the new block
signAddVotes(cs1, types.VoteTypePrevote, propBlockHash, propBlockParts.Header(), vs2, vs3, vs4)
signAddVotes(cs1, types.PrevoteType, propBlockHash, propBlockParts.Header(), vs2, vs3, vs4)
ensureVote(voteCh, height, round, types.VoteTypePrecommit)
ensureVote(voteCh, height, round, types.PrecommitType)
// we should have unlocked and locked on the new block
validatePrecommit(t, cs1, round, round, vss[0], propBlockHash, propBlockHash)
signAddVotes(cs1, types.VoteTypePrecommit, propBlockHash, propBlockParts.Header(), vs2, vs3)
signAddVotes(cs1, types.PrecommitType, propBlockHash, propBlockParts.Header(), vs2, vs3)
ensureNewBlockHeader(newBlockCh, height, propBlockHash)
ensureNewRound(newRoundCh, height+1, 0)
@ -613,20 +614,20 @@ func TestStateLockPOLUnlock(t *testing.T) {
theBlockHash := rs.ProposalBlock.Hash()
theBlockParts := rs.ProposalBlockParts.Header()
ensureVote(voteCh, height, round, types.VoteTypePrevote)
ensureVote(voteCh, height, round, types.PrevoteType)
validatePrevote(t, cs1, round, vss[0], theBlockHash)
signAddVotes(cs1, types.VoteTypePrevote, theBlockHash, theBlockParts, vs2, vs3, vs4)
signAddVotes(cs1, types.PrevoteType, theBlockHash, theBlockParts, vs2, vs3, vs4)
ensureVote(voteCh, height, round, types.VoteTypePrecommit)
ensureVote(voteCh, height, round, types.PrecommitType)
// the proposed block should now be locked and our precommit added
validatePrecommit(t, cs1, round, round, vss[0], theBlockHash, theBlockHash)
rs = cs1.GetRoundState()
// add precommits from the rest
signAddVotes(cs1, types.VoteTypePrecommit, nil, types.PartSetHeader{}, vs2, vs4)
signAddVotes(cs1, types.VoteTypePrecommit, theBlockHash, theBlockParts, vs3)
signAddVotes(cs1, types.PrecommitType, nil, types.PartSetHeader{}, vs2, vs4)
signAddVotes(cs1, types.PrecommitType, theBlockHash, theBlockParts, vs3)
// before we time out into new round, set next proposal block
prop, propBlock := decideProposal(cs1, vs2, vs2.Height, vs2.Round+1)
@ -655,21 +656,20 @@ func TestStateLockPOLUnlock(t *testing.T) {
ensureNewProposal(proposalCh, height, round)
// go to prevote, prevote for locked block (not proposal)
ensureVote(voteCh, height, round, types.VoteTypePrevote)
ensureVote(voteCh, height, round, types.PrevoteType)
validatePrevote(t, cs1, round, vss[0], lockedBlockHash)
// now lets add prevotes from everyone else for nil (a polka!)
signAddVotes(cs1, types.VoteTypePrevote, nil, types.PartSetHeader{}, vs2, vs3, vs4)
signAddVotes(cs1, types.PrevoteType, nil, types.PartSetHeader{}, vs2, vs3, vs4)
// the polka makes us unlock and precommit nil
ensureNewUnlock(unlockCh, height, round)
ensureVote(voteCh, height, round, types.VoteTypePrecommit)
ensureVote(voteCh, height, round, types.PrecommitType)
// we should have unlocked and committed nil
// NOTE: since we don't relock on nil, the lock round is 0
validatePrecommit(t, cs1, round, 0, vss[0], nil, nil)
signAddVotes(cs1, types.VoteTypePrecommit, nil, types.PartSetHeader{}, vs2, vs3)
signAddVotes(cs1, types.PrecommitType, nil, types.PartSetHeader{}, vs2, vs3)
ensureNewRound(newRoundCh, height, round+1)
}
@ -698,19 +698,19 @@ func TestStateLockPOLSafety1(t *testing.T) {
rs := cs1.GetRoundState()
propBlock := rs.ProposalBlock
ensureVote(voteCh, height, round, types.VoteTypePrevote)
ensureVote(voteCh, height, round, types.PrevoteType)
validatePrevote(t, cs1, round, vss[0], propBlock.Hash())
// the others sign a polka but we don't see it
prevotes := signVotes(types.VoteTypePrevote, propBlock.Hash(), propBlock.MakePartSet(partSize).Header(), vs2, vs3, vs4)
prevotes := signVotes(types.PrevoteType, propBlock.Hash(), propBlock.MakePartSet(partSize).Header(), vs2, vs3, vs4)
t.Logf("old prop hash %v", fmt.Sprintf("%X", propBlock.Hash()))
// we do see them precommit nil
signAddVotes(cs1, types.VoteTypePrecommit, nil, types.PartSetHeader{}, vs2, vs3, vs4)
signAddVotes(cs1, types.PrecommitType, nil, types.PartSetHeader{}, vs2, vs3, vs4)
// cs1 precommit nil
ensureVote(voteCh, height, round, types.VoteTypePrecommit)
ensureVote(voteCh, height, round, types.PrecommitType)
ensureNewTimeout(timeoutWaitCh, height, round, cs1.config.TimeoutPrecommit.Nanoseconds())
t.Log("### ONTO ROUND 1")
@ -743,17 +743,17 @@ func TestStateLockPOLSafety1(t *testing.T) {
t.Logf("new prop hash %v", fmt.Sprintf("%X", propBlockHash))
// go to prevote, prevote for proposal block
ensureVote(voteCh, height, round, types.VoteTypePrevote)
ensureVote(voteCh, height, round, types.PrevoteType)
validatePrevote(t, cs1, round, vss[0], propBlockHash)
// now we see the others prevote for it, so we should lock on it
signAddVotes(cs1, types.VoteTypePrevote, propBlockHash, propBlockParts.Header(), vs2, vs3, vs4)
signAddVotes(cs1, types.PrevoteType, propBlockHash, propBlockParts.Header(), vs2, vs3, vs4)
ensureVote(voteCh, height, round, types.VoteTypePrecommit)
ensureVote(voteCh, height, round, types.PrecommitType)
// we should have precommitted
validatePrecommit(t, cs1, round, round, vss[0], propBlockHash, propBlockHash)
signAddVotes(cs1, types.VoteTypePrecommit, nil, types.PartSetHeader{}, vs2, vs3, vs4)
signAddVotes(cs1, types.PrecommitType, nil, types.PartSetHeader{}, vs2, vs3, vs4)
ensureNewTimeout(timeoutWaitCh, height, round, cs1.config.TimeoutPrecommit.Nanoseconds())
@ -771,7 +771,7 @@ func TestStateLockPOLSafety1(t *testing.T) {
ensureNewTimeout(timeoutProposeCh, height, round, cs1.config.TimeoutPropose.Nanoseconds())
// finish prevote
ensureVote(voteCh, height, round, types.VoteTypePrevote)
ensureVote(voteCh, height, round, types.PrevoteType)
// we should prevote what we're locked on
validatePrevote(t, cs1, round, vss[0], propBlockHash)
@ -813,7 +813,7 @@ func TestStateLockPOLSafety2(t *testing.T) {
propBlockParts0 := propBlock0.MakePartSet(partSize)
// the others sign a polka but we don't see it
prevotes := signVotes(types.VoteTypePrevote, propBlockHash0, propBlockParts0.Header(), vs2, vs3, vs4)
prevotes := signVotes(types.PrevoteType, propBlockHash0, propBlockParts0.Header(), vs2, vs3, vs4)
// the block for round 1
prop1, propBlock1 := decideProposal(cs1, vs2, vs2.Height, vs2.Round+1)
@ -834,18 +834,18 @@ func TestStateLockPOLSafety2(t *testing.T) {
}
ensureNewProposal(proposalCh, height, round)
ensureVote(voteCh, height, round, types.VoteTypePrevote)
ensureVote(voteCh, height, round, types.PrevoteType)
validatePrevote(t, cs1, round, vss[0], propBlockHash1)
signAddVotes(cs1, types.VoteTypePrevote, propBlockHash1, propBlockParts1.Header(), vs2, vs3, vs4)
signAddVotes(cs1, types.PrevoteType, propBlockHash1, propBlockParts1.Header(), vs2, vs3, vs4)
ensureVote(voteCh, height, round, types.VoteTypePrecommit)
ensureVote(voteCh, height, round, types.PrecommitType)
// the proposed block should now be locked and our precommit added
validatePrecommit(t, cs1, round, round, vss[0], propBlockHash1, propBlockHash1)
// add precommits from the rest
signAddVotes(cs1, types.VoteTypePrecommit, nil, types.PartSetHeader{}, vs2, vs4)
signAddVotes(cs1, types.VoteTypePrecommit, propBlockHash1, propBlockParts1.Header(), vs3)
signAddVotes(cs1, types.PrecommitType, nil, types.PartSetHeader{}, vs2, vs4)
signAddVotes(cs1, types.PrecommitType, propBlockHash1, propBlockParts1.Header(), vs3)
incrementRound(vs2, vs3, vs4)
@ -873,7 +873,7 @@ func TestStateLockPOLSafety2(t *testing.T) {
ensureNewProposal(proposalCh, height, round)
ensureNoNewUnlock(unlockCh)
ensureVote(voteCh, height, round, types.VoteTypePrevote)
ensureVote(voteCh, height, round, types.PrevoteType)
validatePrevote(t, cs1, round, vss[0], propBlockHash1)
}
@ -893,7 +893,7 @@ func TestWaitingTimeoutOnNilPolka(t *testing.T) {
startTestRound(cs1, height, round)
ensureNewRound(newRoundCh, height, round)
signAddVotes(cs1, types.VoteTypePrecommit, nil, types.PartSetHeader{}, vs2, vs3, vs4)
signAddVotes(cs1, types.PrecommitType, nil, types.PartSetHeader{}, vs2, vs3, vs4)
ensureNewTimeout(timeoutWaitCh, height, round, cs1.config.TimeoutPrecommit.Nanoseconds())
ensureNewRound(newRoundCh, height, round+1)
@ -915,10 +915,10 @@ func TestWaitingTimeoutProposeOnNewRound(t *testing.T) {
startTestRound(cs1, height, round)
ensureNewRound(newRoundCh, height, round)
ensureVote(voteCh, height, round, types.VoteTypePrevote)
ensureVote(voteCh, height, round, types.PrevoteType)
incrementRound(vss[1:]...)
signAddVotes(cs1, types.VoteTypePrevote, nil, types.PartSetHeader{}, vs2, vs3, vs4)
signAddVotes(cs1, types.PrevoteType, nil, types.PartSetHeader{}, vs2, vs3, vs4)
round = round + 1 // moving to the next round
ensureNewRound(newRoundCh, height, round)
@ -928,7 +928,7 @@ func TestWaitingTimeoutProposeOnNewRound(t *testing.T) {
ensureNewTimeout(timeoutWaitCh, height, round, cs1.config.TimeoutPropose.Nanoseconds())
ensureVote(voteCh, height, round, types.VoteTypePrevote)
ensureVote(voteCh, height, round, types.PrevoteType)
validatePrevote(t, cs1, round, vss[0], nil)
}
@ -948,15 +948,15 @@ func TestRoundSkipOnNilPolkaFromHigherRound(t *testing.T) {
startTestRound(cs1, height, round)
ensureNewRound(newRoundCh, height, round)
ensureVote(voteCh, height, round, types.VoteTypePrevote)
ensureVote(voteCh, height, round, types.PrevoteType)
incrementRound(vss[1:]...)
signAddVotes(cs1, types.VoteTypePrecommit, nil, types.PartSetHeader{}, vs2, vs3, vs4)
signAddVotes(cs1, types.PrecommitType, nil, types.PartSetHeader{}, vs2, vs3, vs4)
round = round + 1 // moving to the next round
ensureNewRound(newRoundCh, height, round)
ensureVote(voteCh, height, round, types.VoteTypePrecommit)
ensureVote(voteCh, height, round, types.PrecommitType)
validatePrecommit(t, cs1, round, 0, vss[0], nil, nil)
ensureNewTimeout(timeoutWaitCh, height, round, cs1.config.TimeoutPrecommit.Nanoseconds())
@ -982,11 +982,11 @@ func TestWaitTimeoutProposeOnNilPolkaForTheCurrentRound(t *testing.T) {
ensureNewRound(newRoundCh, height, round)
incrementRound(vss[1:]...)
signAddVotes(cs1, types.VoteTypePrevote, nil, types.PartSetHeader{}, vs2, vs3, vs4)
signAddVotes(cs1, types.PrevoteType, nil, types.PartSetHeader{}, vs2, vs3, vs4)
ensureNewTimeout(timeoutProposeCh, height, round, cs1.config.TimeoutPropose.Nanoseconds())
ensureVote(voteCh, height, round, types.VoteTypePrevote)
ensureVote(voteCh, height, round, types.PrevoteType)
validatePrevote(t, cs1, round, vss[0], nil)
}
@ -1017,7 +1017,7 @@ func TestStateSlashingPrevotes(t *testing.T) {
// add one for a different block should cause us to go into prevote wait
hash := rs.ProposalBlock.Hash()
hash[0] = byte(hash[0]+1) % 255
signAddVotes(cs1, types.VoteTypePrevote, hash, rs.ProposalBlockParts.Header(), vs2)
signAddVotes(cs1, types.PrevoteType, hash, rs.ProposalBlockParts.Header(), vs2)
<-timeoutWaitCh
@ -1025,7 +1025,7 @@ func TestStateSlashingPrevotes(t *testing.T) {
// away and ignore more prevotes (and thus fail to slash!)
// add the conflicting vote
signAddVotes(cs1, types.VoteTypePrevote, rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header(), vs2)
signAddVotes(cs1, types.PrevoteType, rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header(), vs2)
// XXX: Check for existence of Dupeout info
}
@ -1047,7 +1047,7 @@ func TestStateSlashingPrecommits(t *testing.T) {
<-voteCh // prevote
// add prevote from vs2
signAddVotes(cs1, types.VoteTypePrevote, rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header(), vs2)
signAddVotes(cs1, types.PrevoteType, rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header(), vs2)
<-voteCh // precommit
@ -1055,13 +1055,13 @@ func TestStateSlashingPrecommits(t *testing.T) {
// add one for a different block should cause us to go into prevote wait
hash := rs.ProposalBlock.Hash()
hash[0] = byte(hash[0]+1) % 255
signAddVotes(cs1, types.VoteTypePrecommit, hash, rs.ProposalBlockParts.Header(), vs2)
signAddVotes(cs1, types.PrecommitType, hash, rs.ProposalBlockParts.Header(), vs2)
// NOTE: we have to send the vote for different block first so we don't just go into precommit round right
// away and ignore more prevotes (and thus fail to slash!)
// add precommit from vs2
signAddVotes(cs1, types.VoteTypePrecommit, rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header(), vs2)
signAddVotes(cs1, types.PrecommitType, rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header(), vs2)
// XXX: Check for existence of Dupeout info
}
@ -1096,19 +1096,19 @@ func TestStateHalt1(t *testing.T) {
propBlock := rs.ProposalBlock
propBlockParts := propBlock.MakePartSet(partSize)
ensureVote(voteCh, height, round, types.VoteTypePrevote)
ensureVote(voteCh, height, round, types.PrevoteType)
signAddVotes(cs1, types.VoteTypePrevote, propBlock.Hash(), propBlockParts.Header(), vs2, vs3, vs4)
signAddVotes(cs1, types.PrevoteType, propBlock.Hash(), propBlockParts.Header(), vs2, vs3, vs4)
ensureVote(voteCh, height, round, types.VoteTypePrecommit)
ensureVote(voteCh, height, round, types.PrecommitType)
// the proposed block should now be locked and our precommit added
validatePrecommit(t, cs1, round, round, vss[0], propBlock.Hash(), propBlock.Hash())
// add precommits from the rest
signAddVotes(cs1, types.VoteTypePrecommit, nil, types.PartSetHeader{}, vs2) // didnt receive proposal
signAddVotes(cs1, types.VoteTypePrecommit, propBlock.Hash(), propBlockParts.Header(), vs3)
signAddVotes(cs1, types.PrecommitType, nil, types.PartSetHeader{}, vs2) // didnt receive proposal
signAddVotes(cs1, types.PrecommitType, propBlock.Hash(), propBlockParts.Header(), vs3)
// we receive this later, but vs3 might receive it earlier and with ours will go to commit!
precommit4 := signVote(vs4, types.VoteTypePrecommit, propBlock.Hash(), propBlockParts.Header())
precommit4 := signVote(vs4, types.PrecommitType, propBlock.Hash(), propBlockParts.Header())
incrementRound(vs2, vs3, vs4)
@ -1127,7 +1127,7 @@ func TestStateHalt1(t *testing.T) {
*/
// go to prevote, prevote for locked block
ensureVote(voteCh, height, round, types.VoteTypePrevote)
ensureVote(voteCh, height, round, types.PrevoteType)
validatePrevote(t, cs1, round, vss[0], rs.LockedBlock.Hash())
// now we receive the precommit from the previous round
@ -1187,7 +1187,7 @@ func TestStateOutputVoteStats(t *testing.T) {
// create dummy peer
peer := p2pdummy.NewPeer()
vote := signVote(vss[1], types.VoteTypePrecommit, []byte("test"), types.PartSetHeader{})
vote := signVote(vss[1], types.PrecommitType, []byte("test"), types.PartSetHeader{})
voteMessage := &VoteMessage{vote}
cs.handleMsg(msgInfo{voteMessage, peer.ID()})
@ -1201,7 +1201,7 @@ func TestStateOutputVoteStats(t *testing.T) {
// sending the vote for the bigger height
incrementHeight(vss[1])
vote = signVote(vss[1], types.VoteTypePrecommit, []byte("test"), types.PartSetHeader{})
vote = signVote(vss[1], types.PrecommitType, []byte("test"), types.PartSetHeader{})
cs.handleMsg(msgInfo{&VoteMessage{vote}, peer.ID()})


+ 9
- 9
consensus/types/height_vote_set.go View File

@ -99,8 +99,8 @@ func (hvs *HeightVoteSet) addRound(round int) {
cmn.PanicSanity("addRound() for an existing round")
}
// log.Debug("addRound(round)", "round", round)
prevotes := types.NewVoteSet(hvs.chainID, hvs.height, round, types.VoteTypePrevote, hvs.valSet)
precommits := types.NewVoteSet(hvs.chainID, hvs.height, round, types.VoteTypePrecommit, hvs.valSet)
prevotes := types.NewVoteSet(hvs.chainID, hvs.height, round, types.PrevoteType, hvs.valSet)
precommits := types.NewVoteSet(hvs.chainID, hvs.height, round, types.PrecommitType, hvs.valSet)
hvs.roundVoteSets[round] = RoundVoteSet{
Prevotes: prevotes,
Precommits: precommits,
@ -134,13 +134,13 @@ func (hvs *HeightVoteSet) AddVote(vote *types.Vote, peerID p2p.ID) (added bool,
func (hvs *HeightVoteSet) Prevotes(round int) *types.VoteSet {
hvs.mtx.Lock()
defer hvs.mtx.Unlock()
return hvs.getVoteSet(round, types.VoteTypePrevote)
return hvs.getVoteSet(round, types.PrevoteType)
}
func (hvs *HeightVoteSet) Precommits(round int) *types.VoteSet {
hvs.mtx.Lock()
defer hvs.mtx.Unlock()
return hvs.getVoteSet(round, types.VoteTypePrecommit)
return hvs.getVoteSet(round, types.PrecommitType)
}
// Last round and blockID that has +2/3 prevotes for a particular block or nil.
@ -149,7 +149,7 @@ func (hvs *HeightVoteSet) POLInfo() (polRound int, polBlockID types.BlockID) {
hvs.mtx.Lock()
defer hvs.mtx.Unlock()
for r := hvs.round; r >= 0; r-- {
rvs := hvs.getVoteSet(r, types.VoteTypePrevote)
rvs := hvs.getVoteSet(r, types.PrevoteType)
polBlockID, ok := rvs.TwoThirdsMajority()
if ok {
return r, polBlockID
@ -158,15 +158,15 @@ func (hvs *HeightVoteSet) POLInfo() (polRound int, polBlockID types.BlockID) {
return -1, types.BlockID{}
}
func (hvs *HeightVoteSet) getVoteSet(round int, type_ byte) *types.VoteSet {
func (hvs *HeightVoteSet) getVoteSet(round int, type_ types.SignedMsgType) *types.VoteSet {
rvs, ok := hvs.roundVoteSets[round]
if !ok {
return nil
}
switch type_ {
case types.VoteTypePrevote:
case types.PrevoteType:
return rvs.Prevotes
case types.VoteTypePrecommit:
case types.PrecommitType:
return rvs.Precommits
default:
cmn.PanicSanity(fmt.Sprintf("Unexpected vote type %X", type_))
@ -178,7 +178,7 @@ func (hvs *HeightVoteSet) getVoteSet(round int, type_ byte) *types.VoteSet {
// NOTE: if there are too many peers, or too much peer churn,
// this can cause memory issues.
// TODO: implement ability to remove peers too
func (hvs *HeightVoteSet) SetPeerMaj23(round int, type_ byte, peerID p2p.ID, blockID types.BlockID) error {
func (hvs *HeightVoteSet) SetPeerMaj23(round int, type_ types.SignedMsgType, peerID p2p.ID, blockID types.BlockID) error {
hvs.mtx.Lock()
defer hvs.mtx.Unlock()
if !types.IsVoteTypeValid(type_) {


+ 1
- 1
consensus/types/height_vote_set_test.go View File

@ -56,7 +56,7 @@ func makeVoteHR(t *testing.T, height int64, round int, privVals []types.PrivVali
Height: height,
Round: round,
Timestamp: tmtime.Now(),
Type: types.VoteTypePrecommit,
Type: types.PrecommitType,
BlockID: types.BlockID{[]byte("fakehash"), types.PartSetHeader{}},
}
chainID := config.ChainID()


+ 3
- 2
docs/spec/blockchain/blockchain.md View File

@ -410,8 +410,9 @@ must be greater than 2/3 of the total voting power of the complete validator set
A vote is a signed message broadcast in the consensus for a particular block at a particular height and round.
When stored in the blockchain or propagated over the network, votes are encoded in Amino.
For signing, votes are represented via `CanonicalVote` and also encoded using amino (protobuf compatible) via
`Vote.SignBytes` which includes the `ChainID`.
For signing, votes are represented via `CanonicalVote` and also encoded using amino (protobuf compatible) via
`Vote.SignBytes` which includes the `ChainID`, and uses a different ordering of
the fields.
We define a method `Verify` that returns `true` if the signature verifies against the pubkey for the `SignBytes`
using the given ChainID:


+ 12
- 9
docs/spec/blockchain/encoding.md View File

@ -300,20 +300,23 @@ Where the `"value"` is the base64 encoding of the raw pubkey bytes, and the
Signed messages (eg. votes, proposals) in the consensus are encoded using Amino.
When signing, the elements of a message are sorted alphabetically by key and prepended with
a `chain_id` and `type` field.
When signing, the elements of a message are re-ordered so the fixed-length fields
are first, making it easy to quickly check the version, height, round, and type.
The `ChainID` is also appended to the end.
We call this encoding the SignBytes. For instance, SignBytes for a vote is the Amino encoding of the following struct:
```go
type CanonicalVote struct {
ChainID string
Type string
BlockID CanonicalBlockID
Height int64
Round int
Timestamp time.Time
Version uint64 `binary:"fixed64"`
Height int64 `binary:"fixed64"`
Round int64 `binary:"fixed64"`
VoteType byte
Timestamp time.Time
BlockID CanonicalBlockID
ChainID string
}
```
NOTE: see [#1622](https://github.com/tendermint/tendermint/issues/1622) for how field ordering will change
The field ordering and the fixed sized encoding for the first three fields is optimized to ease parsing of SignBytes
in HSMs. It creates fixed offsets for relevant fields that need to be read in this context.
See [#1622](https://github.com/tendermint/tendermint/issues/1622) for more details.

+ 1
- 1
lite/helpers.go View File

@ -97,7 +97,7 @@ func makeVote(header *types.Header, valset *types.ValidatorSet, key crypto.PrivK
Height: header.Height,
Round: 1,
Timestamp: tmtime.Now(),
Type: types.VoteTypePrecommit,
Type: types.PrecommitType,
BlockID: types.BlockID{Hash: header.Hash()},
}
// Sign it


+ 2
- 2
privval/priv_validator.go View File

@ -25,9 +25,9 @@ const (
func voteToStep(vote *types.Vote) int8 {
switch vote.Type {
case types.VoteTypePrevote:
case types.PrevoteType:
return stepPrevote
case types.VoteTypePrecommit:
case types.PrecommitType:
return stepPrecommit
default:
cmn.PanicSanity("Unknown vote type")


+ 3
- 3
privval/priv_validator_test.go View File

@ -101,7 +101,7 @@ func TestSignVote(t *testing.T) {
block1 := types.BlockID{[]byte{1, 2, 3}, types.PartSetHeader{}}
block2 := types.BlockID{[]byte{3, 2, 1}, types.PartSetHeader{}}
height, round := int64(10), 1
voteType := types.VoteTypePrevote
voteType := byte(types.PrevoteType)
// sign a vote for first time
vote := newVote(privVal.Address, 0, height, round, voteType, block1)
@ -206,7 +206,7 @@ func TestDifferByTimestamp(t *testing.T) {
// test vote
{
voteType := types.VoteTypePrevote
voteType := byte(types.PrevoteType)
blockID := types.BlockID{[]byte{1, 2, 3}, types.PartSetHeader{}}
vote := newVote(privVal.Address, 0, height, round, voteType, blockID)
err := privVal.SignVote("mychainid", vote)
@ -235,7 +235,7 @@ func newVote(addr types.Address, idx int, height int64, round int, typ byte, blo
ValidatorIndex: idx,
Height: height,
Round: round,
Type: typ,
Type: types.SignedMsgType(typ),
Timestamp: tmtime.Now(),
BlockID: blockID,
}


+ 2
- 2
privval/socket_test.go View File

@ -79,7 +79,7 @@ func TestSocketPVVote(t *testing.T) {
sc, rs = testSetupSocketPair(t, chainID, types.NewMockPV())
ts = time.Now()
vType = types.VoteTypePrecommit
vType = types.PrecommitType
want = &types.Vote{Timestamp: ts, Type: vType}
have = &types.Vote{Timestamp: ts, Type: vType}
)
@ -237,7 +237,7 @@ func TestRemoteSignVoteErrors(t *testing.T) {
sc, rs = testSetupSocketPair(t, chainID, types.NewErroringMockPV())
ts = time.Now()
vType = types.VoteTypePrecommit
vType = types.PrecommitType
vote = &types.Vote{Timestamp: ts, Type: vType}
)
defer sc.Stop()


+ 2
- 2
state/execution_test.go View File

@ -64,7 +64,7 @@ func TestBeginBlockValidators(t *testing.T) {
prevBlockID := types.BlockID{prevHash, prevParts}
now := tmtime.Now()
vote0 := &types.Vote{ValidatorIndex: 0, Timestamp: now, Type: types.VoteTypePrecommit}
vote0 := &types.Vote{ValidatorIndex: 0, Timestamp: now, Type: types.PrecommitType}
vote1 := &types.Vote{ValidatorIndex: 1, Timestamp: now}
testCases := []struct {
@ -135,7 +135,7 @@ func TestBeginBlockByzantineValidators(t *testing.T) {
types.TM2PB.Evidence(ev2, valSet, now)}},
}
vote0 := &types.Vote{ValidatorIndex: 0, Timestamp: now, Type: types.VoteTypePrecommit}
vote0 := &types.Vote{ValidatorIndex: 0, Timestamp: now, Type: types.PrecommitType}
vote1 := &types.Vote{ValidatorIndex: 1, Timestamp: now}
votes := []*types.Vote{vote0, vote1}
lastCommit := &types.Commit{BlockID: prevBlockID, Precommits: votes}


+ 3
- 3
types/block.go View File

@ -388,7 +388,7 @@ func (commit *Commit) FirstPrecommit() *Vote {
}
}
return &Vote{
Type: VoteTypePrecommit,
Type: PrecommitType,
}
}
@ -410,7 +410,7 @@ func (commit *Commit) Round() int {
// Type returns the vote type of the commit, which is always VoteTypePrecommit
func (commit *Commit) Type() byte {
return VoteTypePrecommit
return byte(PrecommitType)
}
// Size returns the number of votes in the commit
@ -462,7 +462,7 @@ func (commit *Commit) ValidateBasic() error {
continue
}
// Ensure that all votes are precommits.
if precommit.Type != VoteTypePrecommit {
if precommit.Type != PrecommitType {
return fmt.Errorf("Invalid commit vote. Expected precommit, got %v",
precommit.Type)
}


+ 8
- 8
types/block_test.go View File

@ -26,7 +26,7 @@ func TestBlockAddEvidence(t *testing.T) {
lastID := makeBlockIDRandom()
h := int64(3)
voteSet, valSet, vals := randVoteSet(h-1, 1, VoteTypePrecommit, 10, 1)
voteSet, valSet, vals := randVoteSet(h-1, 1, PrecommitType, 10, 1)
commit, err := MakeCommit(lastID, h-1, 1, voteSet, vals)
require.NoError(t, err)
@ -46,7 +46,7 @@ func TestBlockValidateBasic(t *testing.T) {
lastID := makeBlockIDRandom()
h := int64(3)
voteSet, valSet, vals := randVoteSet(h-1, 1, VoteTypePrecommit, 10, 1)
voteSet, valSet, vals := randVoteSet(h-1, 1, PrecommitType, 10, 1)
commit, err := MakeCommit(lastID, h-1, 1, voteSet, vals)
require.NoError(t, err)
@ -106,7 +106,7 @@ func TestBlockMakePartSetWithEvidence(t *testing.T) {
lastID := makeBlockIDRandom()
h := int64(3)
voteSet, valSet, vals := randVoteSet(h-1, 1, VoteTypePrecommit, 10, 1)
voteSet, valSet, vals := randVoteSet(h-1, 1, PrecommitType, 10, 1)
commit, err := MakeCommit(lastID, h-1, 1, voteSet, vals)
require.NoError(t, err)
@ -123,7 +123,7 @@ func TestBlockHashesTo(t *testing.T) {
lastID := makeBlockIDRandom()
h := int64(3)
voteSet, valSet, vals := randVoteSet(h-1, 1, VoteTypePrecommit, 10, 1)
voteSet, valSet, vals := randVoteSet(h-1, 1, PrecommitType, 10, 1)
commit, err := MakeCommit(lastID, h-1, 1, voteSet, vals)
require.NoError(t, err)
@ -190,14 +190,14 @@ func TestNilDataHashDoesntCrash(t *testing.T) {
func TestCommit(t *testing.T) {
lastID := makeBlockIDRandom()
h := int64(3)
voteSet, _, vals := randVoteSet(h-1, 1, VoteTypePrecommit, 10, 1)
voteSet, _, vals := randVoteSet(h-1, 1, PrecommitType, 10, 1)
commit, err := MakeCommit(lastID, h-1, 1, voteSet, vals)
require.NoError(t, err)
assert.NotNil(t, commit.FirstPrecommit())
assert.Equal(t, h-1, commit.Height())
assert.Equal(t, 1, commit.Round())
assert.Equal(t, VoteTypePrecommit, commit.Type())
assert.Equal(t, PrecommitType, SignedMsgType(commit.Type()))
if commit.Size() <= 0 {
t.Fatalf("commit %v has a zero or negative size: %d", commit, commit.Size())
}
@ -218,7 +218,7 @@ func TestCommitValidateBasic(t *testing.T) {
{"Random Commit", func(com *Commit) {}, false},
{"Nil precommit", func(com *Commit) { com.Precommits[0] = nil }, false},
{"Incorrect signature", func(com *Commit) { com.Precommits[0].Signature = []byte{0} }, false},
{"Incorrect type", func(com *Commit) { com.Precommits[0].Type = VoteTypePrevote }, true},
{"Incorrect type", func(com *Commit) { com.Precommits[0].Type = PrevoteType }, true},
{"Incorrect height", func(com *Commit) { com.Precommits[0].Height = int64(100) }, true},
{"Incorrect round", func(com *Commit) { com.Precommits[0].Round = 100 }, true},
}
@ -268,7 +268,7 @@ func TestMaxHeaderBytes(t *testing.T) {
func randCommit() *Commit {
lastID := makeBlockIDRandom()
h := int64(3)
voteSet, _, vals := randVoteSet(h-1, 1, VoteTypePrecommit, 10, 1)
voteSet, _, vals := randVoteSet(h-1, 1, PrecommitType, 10, 1)
commit, err := MakeCommit(lastID, h-1, 1, voteSet, vals)
if err != nil {
panic(err)


+ 42
- 38
types/canonical.go View File

@ -13,44 +13,46 @@ import (
const TimeFormat = time.RFC3339Nano
type CanonicalBlockID struct {
Hash cmn.HexBytes `json:"hash,omitempty"`
PartsHeader CanonicalPartSetHeader `json:"parts,omitempty"`
Hash cmn.HexBytes
PartsHeader CanonicalPartSetHeader
}
type CanonicalPartSetHeader struct {
Hash cmn.HexBytes `json:"hash,omitempty"`
Total int `json:"total,omitempty"`
Hash cmn.HexBytes
Total int
}
type CanonicalProposal struct {
ChainID string `json:"@chain_id"`
Type string `json:"@type"`
BlockPartsHeader CanonicalPartSetHeader `json:"block_parts_header"`
Height int64 `json:"height"`
POLBlockID CanonicalBlockID `json:"pol_block_id"`
POLRound int `json:"pol_round"`
Round int `json:"round"`
Timestamp time.Time `json:"timestamp"`
Version uint64 `binary:"fixed64"`
Height int64 `binary:"fixed64"`
Round int64 `binary:"fixed64"`
Type SignedMsgType // type alias for byte
POLRound int64 `binary:"fixed64"`
Timestamp time.Time
BlockPartsHeader CanonicalPartSetHeader
POLBlockID CanonicalBlockID
ChainID string
}
type CanonicalVote struct {
ChainID string `json:"@chain_id"`
Type string `json:"@type"`
BlockID CanonicalBlockID `json:"block_id"`
Height int64 `json:"height"`
Round int `json:"round"`
Timestamp time.Time `json:"timestamp"`
VoteType byte `json:"type"`
Version uint64 `binary:"fixed64"`
Height int64 `binary:"fixed64"`
Round int64 `binary:"fixed64"`
Type SignedMsgType // type alias for byte
Timestamp time.Time
BlockID CanonicalBlockID
ChainID string
}
type CanonicalHeartbeat struct {
ChainID string `json:"@chain_id"`
Type string `json:"@type"`
Height int64 `json:"height"`
Round int `json:"round"`
Sequence int `json:"sequence"`
ValidatorAddress Address `json:"validator_address"`
ValidatorIndex int `json:"validator_index"`
Version uint64 `binary:"fixed64"`
Height int64 `binary:"fixed64"`
Round int `binary:"fixed64"`
Type byte
Sequence int `binary:"fixed64"`
ValidatorAddress Address
ValidatorIndex int
ChainID string
}
//-----------------------------------
@ -72,38 +74,40 @@ func CanonicalizePartSetHeader(psh PartSetHeader) CanonicalPartSetHeader {
func CanonicalizeProposal(chainID string, proposal *Proposal) CanonicalProposal {
return CanonicalProposal{
ChainID: chainID,
Type: "proposal",
BlockPartsHeader: CanonicalizePartSetHeader(proposal.BlockPartsHeader),
Version: 0, // TODO
Height: proposal.Height,
Round: int64(proposal.Round), // cast int->int64 to make amino encode it fixed64 (does not work for int)
Type: ProposalType,
POLRound: int64(proposal.POLRound),
Timestamp: proposal.Timestamp,
BlockPartsHeader: CanonicalizePartSetHeader(proposal.BlockPartsHeader),
POLBlockID: CanonicalizeBlockID(proposal.POLBlockID),
POLRound: proposal.POLRound,
Round: proposal.Round,
ChainID: chainID,
}
}
func CanonicalizeVote(chainID string, vote *Vote) CanonicalVote {
return CanonicalVote{
ChainID: chainID,
Type: "vote",
BlockID: CanonicalizeBlockID(vote.BlockID),
Version: 0, // TODO
Height: vote.Height,
Round: vote.Round,
Round: int64(vote.Round), // cast int->int64 to make amino encode it fixed64 (does not work for int)
Type: vote.Type,
Timestamp: vote.Timestamp,
VoteType: vote.Type,
BlockID: CanonicalizeBlockID(vote.BlockID),
ChainID: chainID,
}
}
func CanonicalizeHeartbeat(chainID string, heartbeat *Heartbeat) CanonicalHeartbeat {
return CanonicalHeartbeat{
ChainID: chainID,
Type: "heartbeat",
Version: 0, // TODO
Height: heartbeat.Height,
Round: heartbeat.Round,
Type: byte(HeartbeatType),
Sequence: heartbeat.Sequence,
ValidatorAddress: heartbeat.ValidatorAddress,
ValidatorIndex: heartbeat.ValidatorIndex,
ChainID: chainID,
}
}


+ 1
- 1
types/evidence_test.go View File

@ -22,7 +22,7 @@ func makeVote(val PrivValidator, chainID string, valIndex int, height int64, rou
ValidatorIndex: valIndex,
Height: height,
Round: round,
Type: byte(step),
Type: SignedMsgType(step),
BlockID: blockID,
}
err := val.SignVote(chainID, v)


+ 27
- 0
types/signed_msg_type.go View File

@ -0,0 +1,27 @@
package types
// SignedMsgType is a type of signed message in the consensus.
type SignedMsgType byte
const (
// Votes
PrevoteType SignedMsgType = 0x01
PrecommitType SignedMsgType = 0x02
// Proposals
ProposalType SignedMsgType = 0x20
// Heartbeat
HeartbeatType SignedMsgType = 0x30
)
func IsVoteTypeValid(type_ SignedMsgType) bool {
switch type_ {
case PrevoteType:
return true
case PrecommitType:
return true
default:
return false
}
}

+ 1
- 1
types/test_util.go View File

@ -16,7 +16,7 @@ func MakeCommit(blockID BlockID, height int64, round int,
ValidatorIndex: i,
Height: height,
Round: round,
Type: VoteTypePrecommit,
Type: PrecommitType,
BlockID: blockID,
Timestamp: tmtime.Now(),
}


+ 2
- 2
types/validator_set.go View File

@ -282,7 +282,7 @@ func (vals *ValidatorSet) VerifyCommit(chainID string, blockID BlockID, height i
if precommit.Round != round {
return fmt.Errorf("Invalid commit -- wrong round: want %v got %v", round, precommit.Round)
}
if precommit.Type != VoteTypePrecommit {
if precommit.Type != PrecommitType {
return fmt.Errorf("Invalid commit -- not precommit @ index %v", idx)
}
_, val := vals.GetByIndex(idx)
@ -361,7 +361,7 @@ func (vals *ValidatorSet) VerifyFutureCommit(newSet *ValidatorSet, chainID strin
if precommit.Round != round {
return cmn.NewError("Invalid commit -- wrong round: %v vs %v", round, precommit.Round)
}
if precommit.Type != VoteTypePrecommit {
if precommit.Type != PrecommitType {
return cmn.NewError("Invalid commit -- not precommit @ index %v", idx)
}
// See if this validator is in oldVals.


+ 1
- 1
types/validator_set_test.go View File

@ -385,7 +385,7 @@ func TestValidatorSetVerifyCommit(t *testing.T) {
Height: height,
Round: 0,
Timestamp: tmtime.Now(),
Type: VoteTypePrecommit,
Type: PrecommitType,
BlockID: blockID,
}
sig, err := privKey.Sign(vote.SignBytes(chainID))


+ 10
- 28
types/vote.go View File

@ -43,37 +43,19 @@ func NewConflictingVoteError(val *Validator, voteA, voteB *Vote) *ErrVoteConflic
}
}
// Types of votes
// TODO Make a new type "VoteType"
const (
VoteTypePrevote = byte(0x01)
VoteTypePrecommit = byte(0x02)
)
func IsVoteTypeValid(type_ byte) bool {
switch type_ {
case VoteTypePrevote:
return true
case VoteTypePrecommit:
return true
default:
return false
}
}
// Address is hex bytes.
type Address = crypto.Address
// Represents a prevote, precommit, or commit vote from validators for consensus.
type Vote struct {
ValidatorAddress Address `json:"validator_address"`
ValidatorIndex int `json:"validator_index"`
Height int64 `json:"height"`
Round int `json:"round"`
Timestamp time.Time `json:"timestamp"`
Type byte `json:"type"`
BlockID BlockID `json:"block_id"` // zero if vote is nil.
Signature []byte `json:"signature"`
ValidatorAddress Address `json:"validator_address"`
ValidatorIndex int `json:"validator_index"`
Height int64 `json:"height"`
Round int `json:"round"`
Timestamp time.Time `json:"timestamp"`
Type SignedMsgType `json:"type"`
BlockID BlockID `json:"block_id"` // zero if vote is nil.
Signature []byte `json:"signature"`
}
func (vote *Vote) SignBytes(chainID string) []byte {
@ -95,9 +77,9 @@ func (vote *Vote) String() string {
}
var typeString string
switch vote.Type {
case VoteTypePrevote:
case PrevoteType:
typeString = "Prevote"
case VoteTypePrecommit:
case PrecommitType:
typeString = "Precommit"
default:
cmn.PanicSanity("Unknown vote type")


+ 6
- 6
types/vote_set.go View File

@ -55,7 +55,7 @@ type VoteSet struct {
chainID string
height int64
round int
type_ byte
type_ SignedMsgType
valSet *ValidatorSet
mtx sync.Mutex
@ -68,7 +68,7 @@ type VoteSet struct {
}
// Constructs a new VoteSet struct used to accumulate votes for given height/round.
func NewVoteSet(chainID string, height int64, round int, type_ byte, valSet *ValidatorSet) *VoteSet {
func NewVoteSet(chainID string, height int64, round int, type_ SignedMsgType, valSet *ValidatorSet) *VoteSet {
if height == 0 {
cmn.PanicSanity("Cannot make VoteSet for height == 0, doesn't make sense.")
}
@ -109,7 +109,7 @@ func (voteSet *VoteSet) Type() byte {
if voteSet == nil {
return 0x00
}
return voteSet.type_
return byte(voteSet.type_)
}
func (voteSet *VoteSet) Size() int {
@ -381,7 +381,7 @@ func (voteSet *VoteSet) IsCommit() bool {
if voteSet == nil {
return false
}
if voteSet.type_ != VoteTypePrecommit {
if voteSet.type_ != PrecommitType {
return false
}
voteSet.mtx.Lock()
@ -529,8 +529,8 @@ func (voteSet *VoteSet) sumTotalFrac() (int64, int64, float64) {
// Commit
func (voteSet *VoteSet) MakeCommit() *Commit {
if voteSet.type_ != VoteTypePrecommit {
cmn.PanicSanity("Cannot MakeCommit() unless VoteSet.Type is VoteTypePrecommit")
if voteSet.type_ != PrecommitType {
cmn.PanicSanity("Cannot MakeCommit() unless VoteSet.Type is PrecommitType")
}
voteSet.mtx.Lock()
defer voteSet.mtx.Unlock()


+ 15
- 15
types/vote_set_test.go View File

@ -11,7 +11,7 @@ import (
)
// NOTE: privValidators are in order
func randVoteSet(height int64, round int, type_ byte, numValidators int, votingPower int64) (*VoteSet, *ValidatorSet, []PrivValidator) {
func randVoteSet(height int64, round int, type_ SignedMsgType, numValidators int, votingPower int64) (*VoteSet, *ValidatorSet, []PrivValidator) {
valSet, privValidators := RandValidatorSet(numValidators, votingPower)
return NewVoteSet("test_chain_id", height, round, type_, valSet), valSet, privValidators
}
@ -41,7 +41,7 @@ func withRound(vote *Vote, round int) *Vote {
// Convenience: Return new vote with different type
func withType(vote *Vote, type_ byte) *Vote {
vote = vote.Copy()
vote.Type = type_
vote.Type = SignedMsgType(type_)
return vote
}
@ -61,7 +61,7 @@ func withBlockPartsHeader(vote *Vote, blockPartsHeader PartSetHeader) *Vote {
func TestAddVote(t *testing.T) {
height, round := int64(1), 0
voteSet, _, privValidators := randVoteSet(height, round, VoteTypePrevote, 10, 1)
voteSet, _, privValidators := randVoteSet(height, round, PrevoteType, 10, 1)
val0 := privValidators[0]
// t.Logf(">> %v", voteSet)
@ -82,7 +82,7 @@ func TestAddVote(t *testing.T) {
ValidatorIndex: 0, // since privValidators are in order
Height: height,
Round: round,
Type: VoteTypePrevote,
Type: PrevoteType,
Timestamp: tmtime.Now(),
BlockID: BlockID{nil, PartSetHeader{}},
}
@ -105,14 +105,14 @@ func TestAddVote(t *testing.T) {
func Test2_3Majority(t *testing.T) {
height, round := int64(1), 0
voteSet, _, privValidators := randVoteSet(height, round, VoteTypePrevote, 10, 1)
voteSet, _, privValidators := randVoteSet(height, round, PrevoteType, 10, 1)
voteProto := &Vote{
ValidatorAddress: nil, // NOTE: must fill in
ValidatorIndex: -1, // NOTE: must fill in
Height: height,
Round: round,
Type: VoteTypePrevote,
Type: PrevoteType,
Timestamp: tmtime.Now(),
BlockID: BlockID{nil, PartSetHeader{}},
}
@ -158,7 +158,7 @@ func Test2_3Majority(t *testing.T) {
func Test2_3MajorityRedux(t *testing.T) {
height, round := int64(1), 0
voteSet, _, privValidators := randVoteSet(height, round, VoteTypePrevote, 100, 1)
voteSet, _, privValidators := randVoteSet(height, round, PrevoteType, 100, 1)
blockHash := crypto.CRandBytes(32)
blockPartsTotal := 123
@ -170,7 +170,7 @@ func Test2_3MajorityRedux(t *testing.T) {
Height: height,
Round: round,
Timestamp: tmtime.Now(),
Type: VoteTypePrevote,
Type: PrevoteType,
BlockID: BlockID{blockHash, blockPartsHeader},
}
@ -257,7 +257,7 @@ func Test2_3MajorityRedux(t *testing.T) {
func TestBadVotes(t *testing.T) {
height, round := int64(1), 0
voteSet, _, privValidators := randVoteSet(height, round, VoteTypePrevote, 10, 1)
voteSet, _, privValidators := randVoteSet(height, round, PrevoteType, 10, 1)
voteProto := &Vote{
ValidatorAddress: nil,
@ -265,7 +265,7 @@ func TestBadVotes(t *testing.T) {
Height: height,
Round: round,
Timestamp: tmtime.Now(),
Type: VoteTypePrevote,
Type: PrevoteType,
BlockID: BlockID{nil, PartSetHeader{}},
}
@ -308,7 +308,7 @@ func TestBadVotes(t *testing.T) {
// val3 votes of another type.
{
vote := withValidator(voteProto, privValidators[3].GetAddress(), 3)
added, err := signAddVote(privValidators[3], withType(vote, VoteTypePrecommit), voteSet)
added, err := signAddVote(privValidators[3], withType(vote, byte(PrecommitType)), voteSet)
if added || err == nil {
t.Errorf("Expected VoteSet.Add to fail, wrong type")
}
@ -317,7 +317,7 @@ func TestBadVotes(t *testing.T) {
func TestConflicts(t *testing.T) {
height, round := int64(1), 0
voteSet, _, privValidators := randVoteSet(height, round, VoteTypePrevote, 4, 1)
voteSet, _, privValidators := randVoteSet(height, round, PrevoteType, 4, 1)
blockHash1 := cmn.RandBytes(32)
blockHash2 := cmn.RandBytes(32)
@ -327,7 +327,7 @@ func TestConflicts(t *testing.T) {
Height: height,
Round: round,
Timestamp: tmtime.Now(),
Type: VoteTypePrevote,
Type: PrevoteType,
BlockID: BlockID{nil, PartSetHeader{}},
}
@ -447,7 +447,7 @@ func TestConflicts(t *testing.T) {
func TestMakeCommit(t *testing.T) {
height, round := int64(1), 0
voteSet, _, privValidators := randVoteSet(height, round, VoteTypePrecommit, 10, 1)
voteSet, _, privValidators := randVoteSet(height, round, PrecommitType, 10, 1)
blockHash, blockPartsHeader := crypto.CRandBytes(32), PartSetHeader{123, crypto.CRandBytes(32)}
voteProto := &Vote{
@ -456,7 +456,7 @@ func TestMakeCommit(t *testing.T) {
Height: height,
Round: round,
Timestamp: tmtime.Now(),
Type: VoteTypePrecommit,
Type: PrecommitType,
BlockID: BlockID{blockHash, blockPartsHeader},
}


+ 100
- 8
types/vote_test.go View File

@ -13,11 +13,11 @@ import (
)
func examplePrevote() *Vote {
return exampleVote(VoteTypePrevote)
return exampleVote(byte(PrevoteType))
}
func examplePrecommit() *Vote {
return exampleVote(VoteTypePrecommit)
return exampleVote(byte(PrecommitType))
}
func exampleVote(t byte) *Vote {
@ -32,7 +32,7 @@ func exampleVote(t byte) *Vote {
Height: 12345,
Round: 2,
Timestamp: stamp,
Type: t,
Type: SignedMsgType(t),
BlockID: BlockID{
Hash: tmhash.Sum([]byte("blockID_hash")),
PartsHeader: PartSetHeader{
@ -53,6 +53,98 @@ func TestVoteSignable(t *testing.T) {
require.Equal(t, expected, signBytes, "Got unexpected sign bytes for Vote.")
}
func TestVoteSignableTestVectors(t *testing.T) {
voteWithVersion := CanonicalizeVote("", &Vote{Height: 1, Round: 1})
voteWithVersion.Version = 123
tests := []struct {
canonicalVote CanonicalVote
want []byte
}{
{
CanonicalizeVote("", &Vote{}),
// NOTE: Height and Round are skipped here. This case needs to be considered while parsing.
[]byte{0xb, 0x2a, 0x9, 0x9, 0x0, 0x9, 0x6e, 0x88, 0xf1, 0xff, 0xff, 0xff},
},
// with proper (fixed size) height and round (PreCommit):
{
CanonicalizeVote("", &Vote{Height: 1, Round: 1, Type: PrecommitType}),
[]byte{
0x1f, // total length
0x11, // (field_number << 3) | wire_type (version is missing)
0x1, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, // height
0x19, // (field_number << 3) | wire_type
0x1, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, // round
0x20, // (field_number << 3) | wire_type
0x2, // PrecommitType
0x2a, // (field_number << 3) | wire_type
// remaining fields (timestamp):
0x9, 0x9, 0x0, 0x9, 0x6e, 0x88, 0xf1, 0xff, 0xff, 0xff},
},
// with proper (fixed size) height and round (PreVote):
{
CanonicalizeVote("", &Vote{Height: 1, Round: 1, Type: PrevoteType}),
[]byte{
0x1f, // total length
0x11, // (field_number << 3) | wire_type (version is missing)
0x1, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, // height
0x19, // (field_number << 3) | wire_type
0x1, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, // round
0x20, // (field_number << 3) | wire_type
0x1, // PrevoteType
0x2a, // (field_number << 3) | wire_type
// remaining fields (timestamp):
0x9, 0x9, 0x0, 0x9, 0x6e, 0x88, 0xf1, 0xff, 0xff, 0xff},
},
// containing version (empty type)
{
voteWithVersion,
[]byte{
0x26, // total length
0x9, // (field_number << 3) | wire_type
0x7b, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, // version (123)
0x11, // (field_number << 3) | wire_type
0x1, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, // height
0x19, // (field_number << 3) | wire_type
0x1, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, // round
// remaining fields (timestamp):
0x2a,
0x9, 0x9, 0x0, 0x9, 0x6e, 0x88, 0xf1, 0xff, 0xff, 0xff},
},
// containing non-empty chain_id:
{
CanonicalizeVote("test_chain_id", &Vote{Height: 1, Round: 1}),
[]byte{
0x2c, // total length
0x11, // (field_number << 3) | wire_type
0x1, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, // height
0x19, // (field_number << 3) | wire_type
0x1, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, // round
// remaining fields:
0x2a, // (field_number << 3) | wire_type
0x9, 0x9, 0x0, 0x9, 0x6e, 0x88, 0xf1, 0xff, 0xff, 0xff, // timestamp
0x3a, // (field_number << 3) | wire_type
0xd, 0x74, 0x65, 0x73, 0x74, 0x5f, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x5f, 0x69, 0x64}, // chainID
},
}
for i, tc := range tests {
got, err := cdc.MarshalBinary(tc.canonicalVote)
require.NoError(t, err)
require.Equal(t, tc.want, got, "test case #%v: got unexpected sign bytes for Vote.", i)
}
}
func TestVoteProposalNotEq(t *testing.T) {
cv := CanonicalizeVote("", &Vote{Height: 1, Round: 1})
p := CanonicalizeProposal("", &Proposal{Height: 1, Round: 1})
vb, err := cdc.MarshalBinary(cv)
require.NoError(t, err)
pb, err := cdc.MarshalBinary(p)
require.NoError(t, err)
require.NotEqual(t, vb, pb)
}
func TestVoteVerifySignature(t *testing.T) {
privVal := NewMockPV()
pubkey := privVal.GetPubKey()
@ -85,12 +177,12 @@ func TestVoteVerifySignature(t *testing.T) {
func TestIsVoteTypeValid(t *testing.T) {
tc := []struct {
name string
in byte
in SignedMsgType
out bool
}{
{"Prevote", VoteTypePrevote, true},
{"Precommit", VoteTypePrecommit, true},
{"InvalidType", byte(3), false},
{"Prevote", PrevoteType, true},
{"Precommit", PrecommitType, true},
{"InvalidType", SignedMsgType(0x3), false},
}
for _, tt := range tc {
@ -128,7 +220,7 @@ func TestMaxVoteBytes(t *testing.T) {
Height: math.MaxInt64,
Round: math.MaxInt64,
Timestamp: tmtime.Now(),
Type: VoteTypePrevote,
Type: PrevoteType,
BlockID: BlockID{
Hash: tmhash.Sum([]byte("blockID_hash")),
PartsHeader: PartSetHeader{


Loading…
Cancel
Save