Browse Source

Fixed tests

pull/96/head
Jae Kwon 9 years ago
parent
commit
9e1794eaea
9 changed files with 102 additions and 103 deletions
  1. +8
    -6
      consensus/height_vote_set.go
  2. +1
    -1
      consensus/reactor.go
  3. +62
    -45
      consensus/state.go
  4. +4
    -6
      consensus/state_test.go
  5. +7
    -6
      consensus/types/proposal_test.go
  6. +6
    -24
      consensus/vote_set.go
  7. +1
    -4
      rpc/test/helpers.go
  8. +7
    -8
      state/state_test.go
  9. +6
    -3
      types/block.go

+ 8
- 6
consensus/height_vote_set.go View File

@ -42,7 +42,8 @@ func NewHeightVoteSet(height uint, valSet *sm.ValidatorSet) *HeightVoteSet {
roundVoteSets: make(map[uint]RoundVoteSet),
peerFastForward: make(map[string]uint),
}
hvs.SetRound(0)
hvs.addRound(0)
hvs.round = 0
return hvs
}
@ -85,14 +86,15 @@ func (hvs *HeightVoteSet) addRound(round uint) {
}
// Duplicate votes return added=false, err=nil.
func (hvs *HeightVoteSet) AddByAddress(address []byte, vote *types.Vote, peer string) (added bool, index uint, err error) {
// By convention, peerKey is "" if origin is self.
func (hvs *HeightVoteSet) AddByAddress(address []byte, vote *types.Vote, peerKey string) (added bool, index uint, err error) {
hvs.mtx.Lock()
defer hvs.mtx.Unlock()
voteSet := hvs.getVoteSet(vote.Round, vote.Type)
if voteSet == nil {
if _, ok := hvs.peerFastForward[peer]; !ok {
if _, ok := hvs.peerFastForward[peerKey]; !ok {
hvs.addRound(vote.Round)
hvs.peerFastForward[peer] = vote.Round
hvs.peerFastForward[peerKey] = vote.Round
} else {
// Peer has sent a vote that does not match our round,
// for more than one round. Bad peer!
@ -122,8 +124,8 @@ func (hvs *HeightVoteSet) Precommits(round uint) *VoteSet {
func (hvs *HeightVoteSet) POLRound() int {
hvs.mtx.Lock()
defer hvs.mtx.Unlock()
for r := hvs.round; r >= 0; r-- {
if hvs.getVoteSet(r, types.VoteTypePrevote).HasTwoThirdsMajority() {
for r := int(hvs.round); r >= 0; r-- {
if hvs.getVoteSet(uint(r), types.VoteTypePrevote).HasTwoThirdsMajority() {
return int(r)
}
}


+ 1
- 1
consensus/reactor.go View File

@ -187,7 +187,7 @@ func (conR *ConsensusReactor) Receive(chId byte, peer *p2p.Peer, msgBytes []byte
}
VOTE_PASS:
address, _ := rs.Validators.GetByIndex(msg.ValidatorIndex)
added, index, err := conR.conS.AddVote(address, vote)
added, index, err := conR.conS.AddVote(address, vote, peer.Key)
if err != nil {
// If conflicting sig, broadcast evidence tx for slashing. Else punish peer.
if errDupe, ok := err.(*types.ErrVoteConflictingSignature); ok {


+ 62
- 45
consensus/state.go View File

@ -252,6 +252,8 @@ func NewConsensusState(state *sm.State, blockStore *bc.BlockStore, mempoolReacto
newStepCh: make(chan *RoundState, 10),
}
cs.updateToState(state, true)
// Don't call scheduleRound0 yet.
// We do that upon Start().
cs.maybeRebond()
cs.reconstructLastCommit(state)
return cs
@ -268,7 +270,7 @@ func (cs *ConsensusState) reconstructLastCommit(state *sm.State) {
for idx, precommit := range seenValidation.Precommits {
precommitVote := &types.Vote{
Height: state.LastBlockHeight,
Round: seenValidation.Round,
Round: seenValidation.Round(),
Type: types.VoteTypePrecommit,
BlockHash: state.LastBlockHash,
BlockParts: state.LastBlockParts,
@ -309,11 +311,12 @@ func (cs *ConsensusState) NewStepCh() chan *RoundState {
func (cs *ConsensusState) Start() {
if atomic.CompareAndSwapUint32(&cs.started, 0, 1) {
log.Info("Starting ConsensusState")
cs.scheduleRound0()
cs.scheduleRound0(cs.Height)
}
}
func (cs *ConsensusState) scheduleRound0(height uint) {
log.Debug("scheduleRound0", "now", time.Now(), "startTime", cs.StartTime)
sleepDuration := cs.StartTime.Sub(time.Now())
go func() {
if sleepDuration > 0 {
@ -423,7 +426,7 @@ func (cs *ConsensusState) SetPrivValidator(priv *sm.PrivValidator) {
func (cs *ConsensusState) EnterNewRound(height uint, round uint) {
cs.mtx.Lock()
defer cs.mtx.Unlock()
if cs.Height != height || cs.Round >= round {
if cs.Height != height || round < cs.Round || (cs.Round == round && cs.Step != RoundStepNewHeight) {
log.Debug(Fmt("EnterNewRound(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
return
}
@ -432,8 +435,9 @@ func (cs *ConsensusState) EnterNewRound(height uint, round uint) {
}
// Increment validators if necessary
validators := cs.Validators
if cs.Round < round {
validators := cs.Validators.Copy()
validators = validators.Copy()
validators.IncrementAccum(round - cs.Round)
}
@ -517,18 +521,23 @@ func (cs *ConsensusState) EnterPropose(height uint, round uint) {
} else {
log.Warn("EnterPropose: Error signing proposal", "height", cs.Height, "round", cs.Round, "error", err)
}
}
func (cs *ConsensusState) isProposalComplete() bool {
if cs.Proposal == nil || cs.ProposalBlock == nil {
return false
}
return cs.Votes.Prevote(cs.Proposal.POLRound).HasTwoThirdsMajority()
if cs.Proposal.POLRound < 0 {
return true
} else {
return cs.Votes.Prevotes(uint(cs.Proposal.POLRound)).HasTwoThirdsMajority()
}
}
// Create the next block to propose and return it.
// NOTE: make it side-effect free for clarity.
func (cs *ConsensusState) createProposalBlock() (*types.Block, *types.PartSet) {
func (cs *ConsensusState) createProposalBlock() (block *types.Block, blockParts *types.PartSet) {
var validation *types.Validation
if cs.Height == 1 {
// We're creating a proposal for the first block.
@ -554,7 +563,7 @@ func (cs *ConsensusState) createProposalBlock() (*types.Block, *types.PartSet) {
LastBlockParts: cs.state.LastBlockParts,
StateHash: nil, // Will set afterwards.
},
Validation: validation,
LastValidation: validation,
Data: &types.Data{
Txs: txs,
},
@ -645,9 +654,9 @@ func (cs *ConsensusState) EnterPrevoteWait(height uint, round uint) {
cs.Step = RoundStepPrevoteWait
cs.newStepCh <- cs.getRoundState()
// After `timeoutPrevote`, EnterPrecommit()
// After `timeoutPrevote0+timeoutPrevoteDelta*round`, EnterPrecommit()
go func() {
time.Sleep(timeoutPrevote)
time.Sleep(timeoutPrevote0 + timeoutPrevote0*time.Duration(round))
cs.EnterPrecommit(height, round)
}()
}
@ -734,7 +743,7 @@ func (cs *ConsensusState) EnterPrecommit(height uint, round uint) {
cs.ProposalBlock = nil
cs.ProposalBlockParts = types.NewPartSetFromHeader(partsHeader)
}
cs.signAddVote(types.VoteTypePrecommit, nil, PartSetHeader{})
cs.signAddVote(types.VoteTypePrecommit, nil, types.PartSetHeader{})
return
}
@ -755,9 +764,9 @@ func (cs *ConsensusState) EnterPrecommitWait(height uint, round uint) {
cs.Step = RoundStepPrecommitWait
cs.newStepCh <- cs.getRoundState()
// After `timeoutPrecommit`, EnterNewRound()
// After `timeoutPrecommit0+timeoutPrecommitDelta*round`, EnterNewRound()
go func() {
time.Sleep(timeoutPrecommit)
time.Sleep(timeoutPrecommit0 + timeoutPrecommitDelta*time.Duration(round))
// If we have +2/3 of precommits for a particular block (or nil),
// we already entered commit (or the next round).
// So just try to transition to the next round,
@ -782,11 +791,11 @@ func (cs *ConsensusState) EnterCommit(height uint) {
cs.newStepCh <- cs.getRoundState()
// Maybe finalize immediately.
cs.TryFinalizeCommit(height)
cs.tryFinalizeCommit(height)
}()
// SANITY CHECK
hash, partsHeader, ok := cs.Precommits.TwoThirdsMajority()
hash, partsHeader, ok := cs.Votes.Precommits(cs.Round).TwoThirdsMajority()
if !ok {
panic("RunActionCommit() expects +2/3 precommits")
}
@ -815,17 +824,25 @@ func (cs *ConsensusState) EnterCommit(height uint) {
} else {
// We just need to keep waiting.
}
} else {
// We have the block, so sign a Commit-vote.
cs.commitVoteBlock(cs.ProposalBlock, cs.ProposalBlockParts)
}
}
// If we have the block AND +2/3 commits for it, finalize.
func (cs *ConsensusState) TryFinalizeCommit(height uint) {
if cs.ProposalBlock.HashesTo(hash) && cs.Commits.HasTwoThirdsMajority() {
go cs.FinalizeCommit(height)
func (cs *ConsensusState) tryFinalizeCommit(height uint) {
// SANITY CHECK
if cs.Height != height {
panic(Fmt("tryFinalizeCommit() cs.Height: %v vs height: %v", cs.Height, height))
}
// END SANITY CHECK
hash, _, ok := cs.Votes.Precommits(cs.Round).TwoThirdsMajority()
if !ok || len(hash) == 0 {
return
}
if !cs.ProposalBlock.HashesTo(hash) {
return
}
go cs.FinalizeCommit(height)
}
// Increment height and goto RoundStepNewHeight
@ -838,7 +855,7 @@ func (cs *ConsensusState) FinalizeCommit(height uint) {
return
}
hash, header, ok := cs.Commits.TwoThirdsMajority()
hash, header, ok := cs.Votes.Precommits(cs.Round).TwoThirdsMajority()
// SANITY CHECK
if !ok {
@ -857,26 +874,19 @@ func (cs *ConsensusState) FinalizeCommit(height uint) {
log.Debug(Fmt("Finalizing commit of block: %v", cs.ProposalBlock))
// We have the block, so stage/save/commit-vote.
cs.saveBlock(cs.ProposalBlock, cs.ProposalBlockParts, cs.Commits)
cs.commitVoteBlock(cs.ProposalBlock, cs.ProposalBlockParts, cs.Commits)
// Fire off event
go func() {
cs.evsw.FireEvent(types.EventStringNewBlock(), cs.ProposalBlock)
cs.evc.Flush()
}(cs.ProposalBlock)
cs.saveBlock(cs.ProposalBlock, cs.ProposalBlockParts, cs.Votes.Precommits(cs.Round))
// Increment height.
cs.updateToState(cs.stagedState, true)
// cs.StartTime is already set.
// Schedule Round0 to start soon.
go cs.scheduleRound0(height + 1)
// If we're unbonded, broadcast RebondTx.
cs.maybeRebond()
// By here,
// * cs.Height has been increment to height+1
// * cs.Step is now RoundStepNewHeight
// * cs.StartTime is set to when we should start round0.
// Start round 0 when cs.StartTime.
go cs.scheduleRound0(height)
// * cs.StartTime is set to when we will start round0.
return
}
@ -903,7 +913,7 @@ func (cs *ConsensusState) SetProposal(proposal *Proposal) error {
// Verify POLRound, which must be -1 or between 0 and proposal.Round exclusive.
if proposal.POLRound != -1 &&
(proposal.POLRound < 0 || proposal.Round <= proposal.POLRound) {
(proposal.POLRound < 0 || proposal.Round <= uint(proposal.POLRound)) {
return ErrInvalidProposalPOLRound
}
@ -913,7 +923,7 @@ func (cs *ConsensusState) SetProposal(proposal *Proposal) error {
}
cs.Proposal = proposal
cs.ProposalBlockParts = types.NewPartSetFromHeader(proposal.BlockParts)
cs.ProposalBlockParts = types.NewPartSetFromHeader(proposal.BlockPartsHeader)
return nil
}
@ -944,23 +954,23 @@ func (cs *ConsensusState) AddProposalBlockPart(height uint, round uint, part *ty
if cs.Step == RoundStepPropose && cs.isProposalComplete() {
go cs.EnterPrevote(height, round)
} else if cs.Step == RoundStepCommit {
cs.TryFinalizeCommit(height)
cs.tryFinalizeCommit(height)
}
return true, err
}
return true, nil
}
func (cs *ConsensusState) AddVote(address []byte, vote *types.Vote) (added bool, index uint, err error) {
func (cs *ConsensusState) AddVote(address []byte, vote *types.Vote, peerKey string) (added bool, index uint, err error) {
cs.mtx.Lock()
defer cs.mtx.Unlock()
return cs.addVote(address, vote)
return cs.addVote(address, vote, peerKey)
}
//-----------------------------------------------------------------------------
func (cs *ConsensusState) addVote(address []byte, vote *types.Vote) (added bool, index uint, err error) {
func (cs *ConsensusState) addVote(address []byte, vote *types.Vote, peerKey string) (added bool, index uint, err error) {
// A precommit for the previous height?
if vote.Height+1 == cs.Height && vote.Type == types.VoteTypePrecommit {
added, index, err = cs.LastCommit.AddByAddress(address, vote)
@ -972,7 +982,8 @@ func (cs *ConsensusState) addVote(address []byte, vote *types.Vote) (added bool,
// A prevote/precommit for this height?
if vote.Height == cs.Height {
added, index, err = cs.Votes.AddByAddress(address, vote)
height := cs.Height
added, index, err = cs.Votes.AddByAddress(address, vote, peerKey)
if added {
switch vote.Type {
case types.VoteTypePrevote:
@ -995,7 +1006,7 @@ func (cs *ConsensusState) addVote(address []byte, vote *types.Vote) (added bool,
cs.EnterPrevoteWait(height, cs.Round)
}()
}
} else if cs.Proposal != nil && cs.Proposal.POLRound == vote.Round {
} else if cs.Proposal != nil && cs.Proposal.POLRound >= 0 && uint(cs.Proposal.POLRound) == vote.Round {
if cs.isProposalComplete() {
go cs.EnterPrevote(height, cs.Round)
}
@ -1017,7 +1028,7 @@ func (cs *ConsensusState) addVote(address []byte, vote *types.Vote) (added bool,
// If hash is block, goto Commit
go func() {
cs.EnterNewRound(height, vote.Round)
cs.EnterCommit(height, vote.Round)
cs.EnterCommit(height)
}()
}
} else if cs.Votes.Precommits(vote.Round).HasTwoThirdsAny() {
@ -1035,7 +1046,7 @@ func (cs *ConsensusState) addVote(address []byte, vote *types.Vote) (added bool,
go cs.EnterNewRound(height, cs.Round+1)
} else {
// If hash is block, goto Commit
go cs.EnterCommit(height, cs.Round)
go cs.EnterCommit(height)
}
} else if cs.Votes.Precommits(cs.Round).HasTwoThirdsAny() {
// Goto PrecommitWait
@ -1098,8 +1109,8 @@ func (cs *ConsensusState) signAddVote(type_ byte, hash []byte, header types.Part
}
err := cs.privValidator.SignVote(cs.state.ChainID, vote)
if err == nil {
log.Info("Signed and added vote", "height", cs.Height, "round", cs.Round, "vote", vote)
cs.addVote(cs.privValidator.Address, vote)
_, _, err := cs.addVote(cs.privValidator.Address, vote, "")
log.Info("Signed and added vote", "height", cs.Height, "round", cs.Round, "vote", vote, "error", err)
return vote
} else {
log.Warn("Error signing vote", "height", cs.Height, "round", cs.Round, "vote", vote, "error", err)
@ -1127,6 +1138,12 @@ func (cs *ConsensusState) saveBlock(block *types.Block, blockParts *types.PartSe
// Update mempool.
cs.mempoolReactor.Mempool.ResetForBlockAndState(block, cs.stagedState)
// Fire off event
go func(block *types.Block) {
cs.evsw.FireEvent(types.EventStringNewBlock(), block)
cs.evc.Flush()
}(block)
}
// implements events.Eventable


+ 4
- 6
consensus/state_test.go View File

@ -1,28 +1,26 @@
package consensus
import (
"bytes"
"testing"
_ "github.com/tendermint/tendermint/config/tendermint_test"
"github.com/tendermint/tendermint/types"
)
func TestRunActionProposeNoPrivValidator(t *testing.T) {
func TestEnterProposeNoPrivValidator(t *testing.T) {
cs, _ := randConsensusState()
cs.RunActionPropose(1, 0)
cs.EnterPropose(1, 0)
rs := cs.GetRoundState()
if rs.Proposal != nil {
t.Error("Expected to make no proposal, since no privValidator")
}
}
func TestRunActionPropose(t *testing.T) {
func TestEnterPropose(t *testing.T) {
cs, privValidators := randConsensusState()
val0 := privValidators[0]
cs.SetPrivValidator(val0)
cs.RunActionPropose(1, 0)
cs.EnterPropose(1, 0)
rs := cs.GetRoundState()
// Check that Proposal, ProposalBlock, ProposalBlockParts are set.


+ 7
- 6
consensus/types/proposal_test.go View File

@ -11,15 +11,16 @@ import (
func TestProposalSignable(t *testing.T) {
proposal := &Proposal{
Height: 12345,
Round: 23456,
BlockParts: types.PartSetHeader{111, []byte("blockparts")},
POLParts: types.PartSetHeader{222, []byte("polparts")},
Signature: nil,
Height: 12345,
Round: 23456,
BlockPartsHeader: types.PartSetHeader{111, []byte("blockparts")},
POLRound: -1,
Signature: nil,
}
signBytes := account.SignBytes(config.GetString("chain_id"), proposal)
signStr := string(signBytes)
expected := Fmt(`{"chain_id":"%s","proposal":{"block_parts":{"hash":"626C6F636B7061727473","total":111},"height":12345,"pol_parts":{"hash":"706F6C7061727473","total":222},"round":23456}}`,
expected := Fmt(`{"chain_id":"%s","proposal":{"block_parts_header":{"hash":"626C6F636B7061727473","total":111},"height":12345,"pol_round":-1,"round":23456}}`,
config.GetString("chain_id"))
if signStr != expected {
t.Errorf("Got unexpected sign string for SendTx. Expected:\n%v\nGot:\n%v", expected, signStr)


+ 6
- 24
consensus/vote_set.go View File

@ -199,7 +199,7 @@ func (voteSet *VoteSet) HasTwoThirdsAny() bool {
}
voteSet.mtx.Lock()
defer voteSet.mtx.Unlock()
return voteSet.totalBlockHashVotes > voteSet.valSet.TotalVotingPower()*2/3
return voteSet.totalVotes > voteSet.valSet.TotalVotingPower()*2/3
}
// Returns either a blockhash (or nil) that received +2/3 majority.
@ -215,6 +215,9 @@ func (voteSet *VoteSet) TwoThirdsMajority() (hash []byte, parts types.PartSetHea
}
func (voteSet *VoteSet) String() string {
if voteSet == nil {
return "nil-VoteSet"
}
return voteSet.StringIndented("")
}
@ -260,7 +263,7 @@ func (voteSet *VoteSet) MakeValidation() *types.Validation {
if len(voteSet.maj23Hash) == 0 {
panic("Cannot MakeValidation() unless a blockhash has +2/3")
}
precommits := make([]types.Precommit, voteSet.valSet.Size())
precommits := make([]*types.Vote, voteSet.valSet.Size())
voteSet.valSet.Iterate(func(valIndex uint, val *sm.Validator) bool {
vote := voteSet.votes[valIndex]
if vote == nil {
@ -272,31 +275,10 @@ func (voteSet *VoteSet) MakeValidation() *types.Validation {
if !vote.BlockParts.Equals(voteSet.maj23Parts) {
return false
}
precommits[valIndex] = types.Precommit{val.Address, vote.Signature}
precommits[valIndex] = vote
return false
})
return &types.Validation{
Round: voteSet.round,
Precommits: precommits,
}
}
// XXX
func VoteSetFromValidation(validation *types.Validation) *VoteSet {
lastPrecommits := NewVoteSet(state.LastBlockHeight, 0, types.VoteTypePrecommit, state.LastBondedValidators)
seenValidation := cs.blockStore.LoadSeenValidation(state.LastBlockHeight)
for idx, precommit := range seenValidation.Precommits {
precommitVote := &types.Vote{
Height: state.LastBlockHeight,
Round: seenValidation.Round,
Type: types.VoteTypePrecommit,
BlockHash: state.LastBlockHash,
BlockParts: state.LastBlockParts,
Signature: precommit.Signature,
}
added, _, err := lastPrecommits.AddByIndex(uint(idx), precommitVote)
if !added || err != nil {
panic(Fmt("Failed to reconstruct LastPrecommits: %v", err))
}
}
}

+ 1
- 4
rpc/test/helpers.go View File

@ -4,11 +4,9 @@ import (
"bytes"
"strconv"
"testing"
"time"
"github.com/tendermint/tendermint/account"
. "github.com/tendermint/tendermint/common"
"github.com/tendermint/tendermint/consensus"
nm "github.com/tendermint/tendermint/node"
"github.com/tendermint/tendermint/p2p"
ctypes "github.com/tendermint/tendermint/rpc/core/types"
@ -79,8 +77,7 @@ func init() {
priv.SetFile(config.GetString("priv_validator_file"))
priv.Save()
consensus.RoundDuration0 = 2 * time.Second
consensus.RoundDurationDelta = 1 * time.Second
// TODO: change consensus/state.go timeouts to be shorter
// start a node
ready := make(chan struct{})


+ 7
- 8
state/state_test.go View File

@ -72,6 +72,9 @@ func TestCopyState(t *testing.T) {
}
func makeBlock(t *testing.T, state *State, validation *types.Validation, txs []types.Tx) *types.Block {
if validation == nil {
validation = &types.Validation{}
}
block := &types.Block{
Header: &types.Header{
ChainID: state.ChainID,
@ -83,7 +86,7 @@ func makeBlock(t *testing.T, state *State, validation *types.Validation, txs []t
LastBlockParts: state.LastBlockParts,
StateHash: nil,
},
Validation: validation,
LastValidation: validation,
Data: &types.Data{
Txs: txs,
},
@ -628,13 +631,9 @@ func TestAddValidator(t *testing.T) {
privValidators[0].SignVote(s0.ChainID, precommit0)
block1 := makeBlock(t, s0,
types.Validation{
Round: 0,
Precommits: []types.Precommit{
types.Precommit{
Address: privValidators[0].Address,
Signature: precommit0.Signature,
},
&types.Validation{
Precommits: []*types.Vote{
precommit0,
},
}, nil,
)


+ 6
- 3
types/block.go View File

@ -208,6 +208,10 @@ func (v *Validation) ValidateBasic() error {
}
height, round := v.Height(), v.Round()
for _, precommit := range v.Precommits {
// It's OK for precommits to be missing.
if precommit == nil {
continue
}
// Ensure that all votes are precommits
if precommit.Type != VoteTypePrecommit {
return fmt.Errorf("Invalid validation vote. Expected precommit, got %v",
@ -229,10 +233,9 @@ func (v *Validation) ValidateBasic() error {
func (v *Validation) Hash() []byte {
if v.hash == nil {
bs := make([]interface{}, 1+len(v.Precommits))
bs[0] = v.Round
bs := make([]interface{}, len(v.Precommits))
for i, precommit := range v.Precommits {
bs[1+i] = precommit
bs[i] = precommit
}
v.hash = merkle.SimpleHashFromBinaries(bs)
}


Loading…
Cancel
Save