Browse Source

Merge pull request #314 from tendermint/bft_fix_rebase

Bft fix rebase
pull/317/head
Ethan Buchman 8 years ago
committed by GitHub
parent
commit
c05b2c5c59
34 changed files with 1836 additions and 746 deletions
  1. +9
    -4
      blockchain/reactor.go
  2. +2
    -1
      config/tendermint/config.go
  3. +1
    -0
      config/tendermint_test/config.go
  4. +1
    -1
      consensus/common.go
  5. +56
    -161
      consensus/common_test.go
  6. +35
    -11
      consensus/height_vote_set.go
  7. +19
    -10
      consensus/height_vote_set_test.go
  8. +331
    -94
      consensus/reactor.go
  9. +351
    -0
      consensus/reactor_test.go
  10. +1
    -1
      consensus/replay.go
  11. +17
    -13
      consensus/replay_test.go
  12. +85
    -68
      consensus/state.go
  13. +144
    -111
      consensus/state_test.go
  14. +12
    -1
      consensus/test_data/README.md
  15. +9
    -8
      consensus/test_data/empty_block.cswal
  16. +9
    -8
      consensus/test_data/small_block1.cswal
  17. +13
    -10
      consensus/test_data/small_block2.cswal
  18. +18
    -2
      consensus/wal.go
  19. +2
    -2
      node/node.go
  20. +19
    -0
      scripts/txs/random.sh
  21. +3
    -4
      state/execution.go
  22. +3
    -6
      state/state.go
  23. +67
    -33
      types/block.go
  24. +1
    -3
      types/events.go
  25. +2
    -6
      types/part_set.go
  26. +8
    -4
      types/part_set_test.go
  27. +5
    -0
      types/priv_validator.go
  28. +9
    -5
      types/proposal.go
  29. +2
    -2
      types/proposal_test.go
  30. +3
    -6
      types/validator_set.go
  31. +35
    -17
      types/vote.go
  32. +286
    -105
      types/vote_set.go
  33. +248
    -49
      types/vote_set_test.go
  34. +30
    -0
      types/vote_test.go

+ 9
- 4
blockchain/reactor.go View File

@ -8,6 +8,7 @@ import (
"time"
. "github.com/tendermint/go-common"
cfg "github.com/tendermint/go-config"
"github.com/tendermint/go-p2p"
"github.com/tendermint/go-wire"
"github.com/tendermint/tendermint/proxy"
@ -41,7 +42,7 @@ type consensusReactor interface {
type BlockchainReactor struct {
p2p.BaseReactor
sw *p2p.Switch
config cfg.Config
state *sm.State
proxyAppConn proxy.AppConnConsensus // same as consensus.proxyAppConn
store *BlockStore
@ -54,7 +55,7 @@ type BlockchainReactor struct {
evsw types.EventSwitch
}
func NewBlockchainReactor(state *sm.State, proxyAppConn proxy.AppConnConsensus, store *BlockStore, fastSync bool) *BlockchainReactor {
func NewBlockchainReactor(config cfg.Config, state *sm.State, proxyAppConn proxy.AppConnConsensus, store *BlockStore, fastSync bool) *BlockchainReactor {
if state.LastBlockHeight == store.Height()-1 {
store.height -= 1 // XXX HACK, make this better
}
@ -69,6 +70,7 @@ func NewBlockchainReactor(state *sm.State, proxyAppConn proxy.AppConnConsensus,
timeoutsCh,
)
bcR := &BlockchainReactor{
config: config,
state: state,
proxyAppConn: proxyAppConn,
store: store,
@ -219,11 +221,14 @@ FOR_LOOP:
// We need both to sync the first block.
break SYNC_LOOP
}
firstParts := first.MakePartSet()
firstParts := first.MakePartSet(bcR.config.GetInt("block_part_size"))
firstPartsHeader := firstParts.Header()
// Finally, verify the first block using the second's commit
// NOTE: we can probably make this more efficient, but note that calling
// first.Hash() doesn't verify the tx contents, so MakePartSet() is
// currently necessary.
err := bcR.state.Validators.VerifyCommit(
bcR.state.ChainID, first.Hash(), firstPartsHeader, first.Height, second.LastCommit)
bcR.state.ChainID, types.BlockID{first.Hash(), firstPartsHeader}, first.Height, second.LastCommit)
if err != nil {
log.Info("error in validation", "error", err)
bcR.pool.RedoRequest(first.Height)


+ 2
- 1
config/tendermint/config.go View File

@ -73,7 +73,8 @@ func GetConfig(rootDir string) cfg.Config {
mapConfig.SetDefault("cs_wal_light", false)
mapConfig.SetDefault("filter_peers", false)
mapConfig.SetDefault("block_size", 10000)
mapConfig.SetDefault("block_size", 10000) // max number of txs
mapConfig.SetDefault("block_part_size", 65536) // part size 64K
mapConfig.SetDefault("disable_data_hash", false)
mapConfig.SetDefault("timeout_propose", 3000)
mapConfig.SetDefault("timeout_propose_delta", 500)


+ 1
- 0
config/tendermint_test/config.go View File

@ -87,6 +87,7 @@ func ResetConfig(localPath string) cfg.Config {
mapConfig.SetDefault("filter_peers", false)
mapConfig.SetDefault("block_size", 10000)
mapConfig.SetDefault("block_part_size", 65536) // part size 64K
mapConfig.SetDefault("disable_data_hash", false)
mapConfig.SetDefault("timeout_propose", 2000)
mapConfig.SetDefault("timeout_propose_delta", 500)


+ 1
- 1
consensus/common.go View File

@ -6,7 +6,7 @@ import (
// NOTE: this is blocking
func subscribeToEvent(evsw types.EventSwitch, receiver, eventID string, chanCap int) chan interface{} {
// listen for new round
// listen for event
ch := make(chan interface{}, chanCap)
types.AddListenerForEvent(evsw, receiver, eventID, func(data types.TMEventData) {
ch <- data


+ 56
- 161
consensus/common_test.go View File

@ -8,9 +8,11 @@ import (
"testing"
"time"
. "github.com/tendermint/go-common"
cfg "github.com/tendermint/go-config"
dbm "github.com/tendermint/go-db"
bc "github.com/tendermint/tendermint/blockchain"
"github.com/tendermint/tendermint/config/tendermint_test"
mempl "github.com/tendermint/tendermint/mempool"
sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/types"
@ -25,30 +27,36 @@ var config cfg.Config // NOTE: must be reset for each _test.go file
var ensureTimeout = time.Duration(2)
type validatorStub struct {
Index int // Validator index. NOTE: we don't assume validator set changes.
Height int
Round int
*types.PrivValidator
}
func NewValidatorStub(privValidator *types.PrivValidator) *validatorStub {
func NewValidatorStub(privValidator *types.PrivValidator, valIndex int) *validatorStub {
return &validatorStub{
Index: valIndex,
PrivValidator: privValidator,
}
}
func (vs *validatorStub) signVote(voteType byte, hash []byte, header types.PartSetHeader) (*types.Vote, error) {
vote := &types.Vote{
ValidatorIndex: vs.Index,
ValidatorAddress: vs.PrivValidator.Address,
Height: vs.Height,
Round: vs.Round,
Type: voteType,
BlockHash: hash,
BlockPartsHeader: header,
BlockID: types.BlockID{hash, header},
}
err := vs.PrivValidator.SignVote(config.GetString("chain_id"), vote)
return vote, err
}
// convenienve function for testing
//-------------------------------------------------------------------------------
// Convenience functions
// Sign vote for type/hash/header
func signVote(vs *validatorStub, voteType byte, hash []byte, header types.PartSetHeader) *types.Vote {
v, err := vs.signVote(voteType, hash, header)
if err != nil {
@ -57,102 +65,29 @@ func signVote(vs *validatorStub, voteType byte, hash []byte, header types.PartSe
return v
}
// create proposal block from cs1 but sign it with vs
func decideProposal(cs1 *ConsensusState, cs2 *validatorStub, height, round int) (proposal *types.Proposal, block *types.Block) {
// Create proposal block from cs1 but sign it with vs
func decideProposal(cs1 *ConsensusState, vs *validatorStub, height, round int) (proposal *types.Proposal, block *types.Block) {
block, blockParts := cs1.createProposalBlock()
if block == nil { // on error
panic("error creating proposal block")
}
// Make proposal
proposal = types.NewProposal(height, round, blockParts.Header(), cs1.Votes.POLRound())
if err := cs2.SignProposal(config.GetString("chain_id"), proposal); err != nil {
polRound, polBlockID := cs1.Votes.POLInfo()
proposal = types.NewProposal(height, round, blockParts.Header(), polRound, polBlockID)
if err := vs.SignProposal(config.GetString("chain_id"), proposal); err != nil {
panic(err)
}
return
}
//-------------------------------------------------------------------------------
// utils
/*
func nilRound(t *testing.T, cs1 *ConsensusState, vss ...*validatorStub) {
cs1.mtx.Lock()
height, round := cs1.Height, cs1.Round
cs1.mtx.Unlock()
waitFor(t, cs1, height, round, RoundStepPrevote)
signAddVoteToFromMany(types.VoteTypePrevote, cs1, nil, cs1.ProposalBlockParts.Header(), vss...)
waitFor(t, cs1, height, round, RoundStepPrecommit)
signAddVoteToFromMany(types.VoteTypePrecommit, cs1, nil, cs1.ProposalBlockParts.Header(), vss...)
waitFor(t, cs1, height, round+1, RoundStepNewRound)
}
*/
// NOTE: this switches the propser as far as `perspectiveOf` is concerned,
// but for simplicity we return a block it generated.
func changeProposer(t *testing.T, perspectiveOf *ConsensusState, newProposer *validatorStub) *types.Block {
_, v1 := perspectiveOf.Validators.GetByAddress(perspectiveOf.privValidator.Address)
v1.Accum, v1.VotingPower = 0, 0
if updated := perspectiveOf.Validators.Update(v1); !updated {
panic("failed to update validator")
}
_, v2 := perspectiveOf.Validators.GetByAddress(newProposer.Address)
v2.Accum, v2.VotingPower = 100, 100
if updated := perspectiveOf.Validators.Update(v2); !updated {
panic("failed to update validator")
}
// make the proposal
propBlock, _ := perspectiveOf.createProposalBlock()
if propBlock == nil {
panic("Failed to create proposal block with cs2")
}
return propBlock
}
func fixVotingPower(t *testing.T, cs1 *ConsensusState, addr2 []byte) {
_, v1 := cs1.Validators.GetByAddress(cs1.privValidator.Address)
_, v2 := cs1.Validators.GetByAddress(addr2)
v1.Accum, v1.VotingPower = v2.Accum, v2.VotingPower
if updated := cs1.Validators.Update(v1); !updated {
panic("failed to update validator")
}
}
func addVoteToFromMany(to *ConsensusState, votes []*types.Vote, froms ...*validatorStub) {
if len(votes) != len(froms) {
panic("len(votes) and len(froms) must match")
func addVotes(to *ConsensusState, votes ...*types.Vote) {
for _, vote := range votes {
to.peerMsgQueue <- msgInfo{Msg: &VoteMessage{vote}}
}
for i, from := range froms {
addVoteToFrom(to, from, votes[i])
}
}
func addVoteToFrom(to *ConsensusState, from *validatorStub, vote *types.Vote) {
to.mtx.Lock() // NOTE: wont need this when the vote comes with the index!
valIndex, _ := to.Validators.GetByAddress(from.PrivValidator.Address)
to.mtx.Unlock()
to.peerMsgQueue <- msgInfo{Msg: &VoteMessage{valIndex, vote}}
// added, err := to.TryAddVote(valIndex, vote, "")
/*
if _, ok := err.(*types.ErrVoteConflictingSignature); ok {
// let it fly
} else if !added {
fmt.Println("to, from, vote:", to.Height, from.Height, vote.Height)
panic(fmt.Sprintln("Failed to add vote. Err:", err))
} else if err != nil {
panic(fmt.Sprintln("Failed to add vote:", err))
}*/
}
func signVoteMany(voteType byte, hash []byte, header types.PartSetHeader, vss ...*validatorStub) []*types.Vote {
func signVotes(voteType byte, 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)
@ -160,34 +95,9 @@ func signVoteMany(voteType byte, hash []byte, header types.PartSetHeader, vss ..
return votes
}
// add vote to one cs from another
// if voteCh is not nil, read all votes
func signAddVoteToFromMany(voteType byte, to *ConsensusState, hash []byte, header types.PartSetHeader, voteCh chan interface{}, froms ...*validatorStub) {
var wg chan struct{} // when done reading all votes
if voteCh != nil {
wg = readVotes(voteCh, len(froms))
}
for _, from := range froms {
vote := signVote(from, voteType, hash, header)
addVoteToFrom(to, from, vote)
}
if voteCh != nil {
<-wg
}
}
func signAddVoteToFrom(voteType byte, to *ConsensusState, from *validatorStub, hash []byte, header types.PartSetHeader, voteCh chan interface{}) *types.Vote {
var wg chan struct{} // when done reading all votes
if voteCh != nil {
wg = readVotes(voteCh, 1)
}
vote := signVote(from, voteType, hash, header)
addVoteToFrom(to, from, vote)
if voteCh != nil {
<-wg
}
return vote
func signAddVotes(to *ConsensusState, voteType byte, hash []byte, header types.PartSetHeader, vss ...*validatorStub) {
votes := signVotes(voteType, hash, header, vss...)
addVotes(to, votes...)
}
func ensureNoNewStep(stepCh chan interface{}) {
@ -200,39 +110,6 @@ func ensureNoNewStep(stepCh chan interface{}) {
}
}
/*
func ensureNoNewStep(t *testing.T, cs *ConsensusState) {
timeout := time.NewTicker(ensureTimeout * time.Second)
select {
case <-timeout.C:
break
case <-cs.NewStepCh():
panic("We should be stuck waiting for more votes, not moving to the next step")
}
}
func ensureNewStep(t *testing.T, cs *ConsensusState) *RoundState {
timeout := time.NewTicker(ensureTimeout * time.Second)
select {
case <-timeout.C:
panic("We should have gone to the next step, not be stuck waiting")
case rs := <-cs.NewStepCh():
return rs
}
}
func waitFor(t *testing.T, cs *ConsensusState, height int, round int, step RoundStepType) {
for {
rs := ensureNewStep(t, cs)
if CompareHRS(rs.Height, rs.Round, rs.Step, height, round, step) < 0 {
continue
} else {
break
}
}
}
*/
func incrementHeight(vss ...*validatorStub) {
for _, vs := range vss {
vs.Height += 1
@ -252,12 +129,12 @@ func validatePrevote(t *testing.T, cs *ConsensusState, round int, privVal *valid
panic("Failed to find prevote from validator")
}
if blockHash == nil {
if vote.BlockHash != nil {
panic(fmt.Sprintf("Expected prevote to be for nil, got %X", vote.BlockHash))
if vote.BlockID.Hash != nil {
panic(fmt.Sprintf("Expected prevote to be for nil, got %X", vote.BlockID.Hash))
}
} else {
if !bytes.Equal(vote.BlockHash, blockHash) {
panic(fmt.Sprintf("Expected prevote to be for %X, got %X", blockHash, vote.BlockHash))
if !bytes.Equal(vote.BlockID.Hash, blockHash) {
panic(fmt.Sprintf("Expected prevote to be for %X, got %X", blockHash, vote.BlockID.Hash))
}
}
}
@ -268,8 +145,8 @@ func validateLastPrecommit(t *testing.T, cs *ConsensusState, privVal *validatorS
if vote = votes.GetByAddress(privVal.Address); vote == nil {
panic("Failed to find precommit from validator")
}
if !bytes.Equal(vote.BlockHash, blockHash) {
panic(fmt.Sprintf("Expected precommit to be for %X, got %X", blockHash, vote.BlockHash))
if !bytes.Equal(vote.BlockID.Hash, blockHash) {
panic(fmt.Sprintf("Expected precommit to be for %X, got %X", blockHash, vote.BlockID.Hash))
}
}
@ -281,11 +158,11 @@ func validatePrecommit(t *testing.T, cs *ConsensusState, thisRound, lockRound in
}
if votedBlockHash == nil {
if vote.BlockHash != nil {
if vote.BlockID.Hash != nil {
panic("Expected precommit to be for nil")
}
} else {
if !bytes.Equal(vote.BlockHash, votedBlockHash) {
if !bytes.Equal(vote.BlockID.Hash, votedBlockHash) {
panic("Expected precommit to be for proposal block")
}
}
@ -331,7 +208,7 @@ func fixedConsensusStateDummy() *ConsensusState {
return cs
}
func newConsensusState(state *sm.State, pv *types.PrivValidator, app tmsp.Application) *ConsensusState {
func newConsensusStateWithConfig(thisConfig cfg.Config, state *sm.State, pv *types.PrivValidator, app tmsp.Application) *ConsensusState {
// Get BlockStore
blockDB := dbm.NewMemDB()
blockStore := bc.NewBlockStore(blockDB)
@ -342,10 +219,10 @@ func newConsensusState(state *sm.State, pv *types.PrivValidator, app tmsp.Applic
proxyAppConnCon := tmspcli.NewLocalClient(mtx, app)
// Make Mempool
mempool := mempl.NewMempool(config, proxyAppConnMem)
mempool := mempl.NewMempool(thisConfig, proxyAppConnMem)
// Make ConsensusReactor
cs := NewConsensusState(config, state, proxyAppConnCon, blockStore, mempool)
cs := NewConsensusState(thisConfig, state, proxyAppConnCon, blockStore, mempool)
cs.SetPrivValidator(pv)
evsw := types.NewEventSwitch()
@ -354,6 +231,10 @@ func newConsensusState(state *sm.State, pv *types.PrivValidator, app tmsp.Applic
return cs
}
func newConsensusState(state *sm.State, pv *types.PrivValidator, app tmsp.Application) *ConsensusState {
return newConsensusStateWithConfig(config, state, pv, app)
}
func randConsensusState(nValidators int) (*ConsensusState, []*validatorStub) {
// Get State
state, privVals := randGenesisState(nValidators, false, 10)
@ -363,7 +244,7 @@ func randConsensusState(nValidators int) (*ConsensusState, []*validatorStub) {
cs := newConsensusState(state, privVals[0], counter.NewCounterApplication(true))
for i := 0; i < nValidators; i++ {
vss[i] = NewValidatorStub(privVals[i])
vss[i] = NewValidatorStub(privVals[i], i)
}
// since cs1 starts at 1
incrementHeight(vss[1:]...)
@ -371,6 +252,20 @@ func randConsensusState(nValidators int) (*ConsensusState, []*validatorStub) {
return cs, vss
}
func randConsensusNet(nValidators int) []*ConsensusState {
genDoc, privVals := randGenesisDoc(nValidators, false, 10)
css := make([]*ConsensusState, nValidators)
for i := 0; i < nValidators; i++ {
db := dbm.NewMemDB() // each state needs its own db
state := sm.MakeGenesisState(db, genDoc)
state.Save()
thisConfig := tendermint_test.ResetConfig(Fmt("consensus_reactor_test_%d", i))
EnsureDir(thisConfig.GetString("cs_wal_dir"), 0700) // dir for wal
css[i] = newConsensusStateWithConfig(thisConfig, state, privVals[i], counter.NewCounterApplication(true))
}
return css
}
func subscribeToVoter(cs *ConsensusState, addr []byte) chan interface{} {
voteCh0 := subscribeToEvent(cs.evsw, "tester", types.EventStringVote(), 1)
voteCh := make(chan interface{})
@ -379,7 +274,7 @@ func subscribeToVoter(cs *ConsensusState, addr []byte) chan interface{} {
v := <-voteCh0
vote := v.(types.EventDataVote)
// we only fire for our own votes
if bytes.Equal(addr, vote.Address) {
if bytes.Equal(addr, vote.Vote.ValidatorAddress) {
voteCh <- v
}
}
@ -399,8 +294,8 @@ func readVotes(ch chan interface{}, reads int) chan struct{} {
}
func randGenesisState(numValidators int, randPower bool, minPower int64) (*sm.State, []*types.PrivValidator) {
db := dbm.NewMemDB()
genDoc, privValidators := randGenesisDoc(numValidators, randPower, minPower)
db := dbm.NewMemDB()
s0 := sm.MakeGenesisState(db, genDoc)
s0.Save()
return s0, privValidators


+ 35
- 11
consensus/height_vote_set.go View File

@ -24,6 +24,8 @@ but which round is not known in advance, so when a peer
provides a precommit for a round greater than mtx.round,
we create a new entry in roundVoteSets but also remember the
peer to prevent abuse.
We let each peer provide us with up to 2 unexpected "catchup" rounds.
One for their LastCommit round, and another for the official commit round.
*/
type HeightVoteSet struct {
chainID string
@ -33,7 +35,7 @@ type HeightVoteSet struct {
mtx sync.Mutex
round int // max tracked round
roundVoteSets map[int]RoundVoteSet // keys: [0...round]
peerCatchupRounds map[string]int // keys: peer.Key; values: round
peerCatchupRounds map[string][]int // keys: peer.Key; values: at most 2 rounds
}
func NewHeightVoteSet(chainID string, height int, valSet *types.ValidatorSet) *HeightVoteSet {
@ -51,7 +53,7 @@ func (hvs *HeightVoteSet) Reset(height int, valSet *types.ValidatorSet) {
hvs.height = height
hvs.valSet = valSet
hvs.roundVoteSets = make(map[int]RoundVoteSet)
hvs.peerCatchupRounds = make(map[string]int)
hvs.peerCatchupRounds = make(map[string][]int)
hvs.addRound(0)
hvs.round = 0
@ -100,15 +102,18 @@ func (hvs *HeightVoteSet) addRound(round int) {
// Duplicate votes return added=false, err=nil.
// By convention, peerKey is "" if origin is self.
func (hvs *HeightVoteSet) AddByIndex(valIndex int, vote *types.Vote, peerKey string) (added bool, address []byte, err error) {
func (hvs *HeightVoteSet) AddVote(vote *types.Vote, peerKey string) (added bool, err error) {
hvs.mtx.Lock()
defer hvs.mtx.Unlock()
if !types.IsVoteTypeValid(vote.Type) {
return
}
voteSet := hvs.getVoteSet(vote.Round, vote.Type)
if voteSet == nil {
if _, ok := hvs.peerCatchupRounds[peerKey]; !ok {
if rndz := hvs.peerCatchupRounds[peerKey]; len(rndz) < 2 {
hvs.addRound(vote.Round)
voteSet = hvs.getVoteSet(vote.Round, vote.Type)
hvs.peerCatchupRounds[peerKey] = vote.Round
hvs.peerCatchupRounds[peerKey] = append(rndz, vote.Round)
} else {
// Peer has sent a vote that does not match our round,
// for more than one round. Bad peer!
@ -117,7 +122,7 @@ func (hvs *HeightVoteSet) AddByIndex(valIndex int, vote *types.Vote, peerKey str
return
}
}
added, address, err = voteSet.AddByIndex(valIndex, vote)
added, err = voteSet.AddVote(vote)
return
}
@ -133,17 +138,19 @@ func (hvs *HeightVoteSet) Precommits(round int) *types.VoteSet {
return hvs.getVoteSet(round, types.VoteTypePrecommit)
}
// Last round that has +2/3 prevotes for a particular block or nil.
// Last round and blockID that has +2/3 prevotes for a particular block or nil.
// Returns -1 if no such round exists.
func (hvs *HeightVoteSet) POLRound() int {
func (hvs *HeightVoteSet) POLInfo() (polRound int, polBlockID types.BlockID) {
hvs.mtx.Lock()
defer hvs.mtx.Unlock()
for r := hvs.round; r >= 0; r-- {
if hvs.getVoteSet(r, types.VoteTypePrevote).HasTwoThirdsMajority() {
return r
rvs := hvs.getVoteSet(r, types.VoteTypePrevote)
polBlockID, ok := rvs.TwoThirdsMajority()
if ok {
return r, polBlockID
}
}
return -1
return -1, types.BlockID{}
}
func (hvs *HeightVoteSet) getVoteSet(round int, type_ byte) *types.VoteSet {
@ -194,3 +201,20 @@ func (hvs *HeightVoteSet) StringIndented(indent string) string {
indent, strings.Join(vsStrings, "\n"+indent+" "),
indent)
}
// If a peer claims that it has 2/3 majority for given blockKey, call this.
// 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 string, blockID types.BlockID) {
hvs.mtx.Lock()
defer hvs.mtx.Unlock()
if !types.IsVoteTypeValid(type_) {
return
}
voteSet := hvs.getVoteSet(round, type_)
if voteSet == nil {
return
}
voteSet.SetPeerMaj23(peerID, blockID)
}

+ 19
- 10
consensus/height_vote_set_test.go View File

@ -17,31 +17,40 @@ func TestPeerCatchupRounds(t *testing.T) {
hvs := NewHeightVoteSet(config.GetString("chain_id"), 1, valSet)
vote999_0 := makeVoteHR(t, 1, 999, privVals[0])
added, _, err := hvs.AddByIndex(0, vote999_0, "peer1")
vote999_0 := makeVoteHR(t, 1, 999, privVals, 0)
added, err := hvs.AddVote(vote999_0, "peer1")
if !added || err != nil {
t.Error("Expected to successfully add vote from peer", added, err)
}
vote1000_0 := makeVoteHR(t, 1, 1000, privVals[0])
added, _, err = hvs.AddByIndex(0, vote1000_0, "peer1")
vote1000_0 := makeVoteHR(t, 1, 1000, privVals, 0)
added, err = hvs.AddVote(vote1000_0, "peer1")
if !added || err != nil {
t.Error("Expected to successfully add vote from peer", added, err)
}
vote1001_0 := makeVoteHR(t, 1, 1001, privVals, 0)
added, err = hvs.AddVote(vote1001_0, "peer1")
if added {
t.Error("Expected to *not* add vote from peer, too many catchup rounds.")
}
added, _, err = hvs.AddByIndex(0, vote1000_0, "peer2")
added, err = hvs.AddVote(vote1001_0, "peer2")
if !added || err != nil {
t.Error("Expected to successfully add vote from another peer")
}
}
func makeVoteHR(t *testing.T, height, round int, privVal *types.PrivValidator) *types.Vote {
func makeVoteHR(t *testing.T, height, round int, privVals []*types.PrivValidator, valIndex int) *types.Vote {
privVal := privVals[valIndex]
vote := &types.Vote{
Height: height,
Round: round,
Type: types.VoteTypePrecommit,
BlockHash: []byte("fakehash"),
ValidatorAddress: privVal.Address,
ValidatorIndex: valIndex,
Height: height,
Round: round,
Type: types.VoteTypePrecommit,
BlockID: types.BlockID{[]byte("fakehash"), types.PartSetHeader{}},
}
chainID := config.GetString("chain_id")
err := privVal.SignVote(chainID, vote)


+ 331
- 94
consensus/reactor.go View File

@ -11,18 +11,19 @@ import (
. "github.com/tendermint/go-common"
"github.com/tendermint/go-p2p"
"github.com/tendermint/go-wire"
bc "github.com/tendermint/tendermint/blockchain"
sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/types"
)
const (
StateChannel = byte(0x20)
DataChannel = byte(0x21)
VoteChannel = byte(0x22)
peerGossipSleepDuration = 100 * time.Millisecond // Time to sleep if there's nothing to send.
maxConsensusMessageSize = 1048576 // 1MB; NOTE: keep in sync with types.PartSet sizes.
StateChannel = byte(0x20)
DataChannel = byte(0x21)
VoteChannel = byte(0x22)
VoteSetBitsChannel = byte(0x23)
peerGossipSleepDuration = 100 * time.Millisecond // Time to sleep if there's nothing to send.
peerQueryMaj23SleepDuration = 2 * time.Second // Time to sleep after each VoteSetMaj23Message sent
maxConsensusMessageSize = 1048576 // 1MB; NOTE: keep in sync with types.PartSet sizes.
)
//-----------------------------------------------------------------------------
@ -30,17 +31,15 @@ const (
type ConsensusReactor struct {
p2p.BaseReactor // BaseService + p2p.Switch
blockStore *bc.BlockStore
conS *ConsensusState
fastSync bool
evsw types.EventSwitch
conS *ConsensusState
fastSync bool
evsw types.EventSwitch
}
func NewConsensusReactor(consensusState *ConsensusState, blockStore *bc.BlockStore, fastSync bool) *ConsensusReactor {
func NewConsensusReactor(consensusState *ConsensusState, fastSync bool) *ConsensusReactor {
conR := &ConsensusReactor{
blockStore: blockStore,
conS: consensusState,
fastSync: fastSync,
conS: consensusState,
fastSync: fastSync,
}
conR.BaseReactor = *p2p.NewBaseReactor(log, "ConsensusReactor", conR)
return conR
@ -101,6 +100,12 @@ func (conR *ConsensusReactor) GetChannels() []*p2p.ChannelDescriptor {
SendQueueCapacity: 100,
RecvBufferCapacity: 100 * 100,
},
&p2p.ChannelDescriptor{
ID: VoteSetBitsChannel,
Priority: 1,
SendQueueCapacity: 2,
RecvBufferCapacity: 1024,
},
}
}
@ -114,9 +119,10 @@ func (conR *ConsensusReactor) AddPeer(peer *p2p.Peer) {
peerState := NewPeerState(peer)
peer.Data.Set(types.PeerStateKey, peerState)
// Begin gossip routines for this peer.
// Begin routines for this peer.
go conR.gossipDataRoutine(peer, peerState)
go conR.gossipVotesRoutine(peer, peerState)
go conR.queryMaj23Routine(peer, peerState)
// Send our state to peer.
// If we're fast_syncing, broadcast a RoundStepMessage later upon SwitchToConsensus().
@ -166,6 +172,36 @@ func (conR *ConsensusReactor) Receive(chID byte, src *p2p.Peer, msgBytes []byte)
ps.ApplyCommitStepMessage(msg)
case *HasVoteMessage:
ps.ApplyHasVoteMessage(msg)
case *VoteSetMaj23Message:
cs := conR.conS
cs.mtx.Lock()
height, votes := cs.Height, cs.Votes
cs.mtx.Unlock()
if height != msg.Height {
return
}
// Peer claims to have a maj23 for some BlockID at H,R,S,
votes.SetPeerMaj23(msg.Round, msg.Type, ps.Peer.Key, msg.BlockID)
// Respond with a VoteSetBitsMessage showing which votes we have.
// (and consequently shows which we don't have)
var ourVotes *BitArray
switch msg.Type {
case types.VoteTypePrevote:
ourVotes = votes.Prevotes(msg.Round).BitArrayByBlockID(msg.BlockID)
case types.VoteTypePrecommit:
ourVotes = votes.Precommits(msg.Round).BitArrayByBlockID(msg.BlockID)
default:
log.Warn("Bad VoteSetBitsMessage field Type")
return
}
src.TrySend(VoteSetBitsChannel, struct{ ConsensusMessage }{&VoteSetBitsMessage{
Height: msg.Height,
Round: msg.Round,
Type: msg.Type,
BlockID: msg.BlockID,
Votes: ourVotes,
}})
default:
log.Warn(Fmt("Unknown message type %v", reflect.TypeOf(msg)))
}
@ -201,7 +237,7 @@ func (conR *ConsensusReactor) Receive(chID byte, src *p2p.Peer, msgBytes []byte)
cs.mtx.Unlock()
ps.EnsureVoteBitArrays(height, valSize)
ps.EnsureVoteBitArrays(height-1, lastCommitSize)
ps.SetHasVote(msg.Vote, msg.ValidatorIndex)
ps.SetHasVote(msg.Vote)
conR.conS.peerMsgQueue <- msgInfo{msg, src.Key}
@ -209,6 +245,39 @@ func (conR *ConsensusReactor) Receive(chID byte, src *p2p.Peer, msgBytes []byte)
// don't punish (leave room for soft upgrades)
log.Warn(Fmt("Unknown message type %v", reflect.TypeOf(msg)))
}
case VoteSetBitsChannel:
if conR.fastSync {
log.Warn("Ignoring message received during fastSync", "msg", msg)
return
}
switch msg := msg.(type) {
case *VoteSetBitsMessage:
cs := conR.conS
cs.mtx.Lock()
height, votes := cs.Height, cs.Votes
cs.mtx.Unlock()
if height == msg.Height {
var ourVotes *BitArray
switch msg.Type {
case types.VoteTypePrevote:
ourVotes = votes.Prevotes(msg.Round).BitArrayByBlockID(msg.BlockID)
case types.VoteTypePrecommit:
ourVotes = votes.Precommits(msg.Round).BitArrayByBlockID(msg.BlockID)
default:
log.Warn("Bad VoteSetBitsMessage field Type")
return
}
ps.ApplyVoteSetBitsMessage(msg, ourVotes)
} else {
ps.ApplyVoteSetBitsMessage(msg, nil)
}
default:
// don't punish (leave room for soft upgrades)
log.Warn(Fmt("Unknown message type %v", reflect.TypeOf(msg)))
}
default:
log.Warn(Fmt("Unknown chId %X", chID))
}
@ -219,7 +288,7 @@ func (conR *ConsensusReactor) Receive(chID byte, src *p2p.Peer, msgBytes []byte)
}
// Sets our private validator account for signing votes.
func (conR *ConsensusReactor) SetPrivValidator(priv *types.PrivValidator) {
func (conR *ConsensusReactor) SetPrivValidator(priv PrivValidator) {
conR.conS.SetPrivValidator(priv)
}
@ -242,7 +311,7 @@ func (conR *ConsensusReactor) registerEventCallbacks() {
types.AddListenerForEvent(conR.evsw, "conR", types.EventStringVote(), func(data types.TMEventData) {
edv := data.(types.EventDataVote)
conR.broadcastHasVoteMessage(edv.Vote, edv.Index)
conR.broadcastHasVoteMessage(edv.Vote)
})
}
@ -258,12 +327,12 @@ func (conR *ConsensusReactor) broadcastNewRoundStep(rs *RoundState) {
}
// Broadcasts HasVoteMessage to peers that care.
func (conR *ConsensusReactor) broadcastHasVoteMessage(vote *types.Vote, index int) {
func (conR *ConsensusReactor) broadcastHasVoteMessage(vote *types.Vote) {
msg := &HasVoteMessage{
Height: vote.Height,
Round: vote.Round,
Type: vote.Type,
Index: index,
Index: vote.ValidatorIndex,
}
conR.Switch.Broadcast(StateChannel, struct{ ConsensusMessage }{msg})
/*
@ -346,15 +415,19 @@ OUTER_LOOP:
//log.Info("Data catchup", "height", rs.Height, "peerHeight", prs.Height, "peerProposalBlockParts", prs.ProposalBlockParts)
if index, ok := prs.ProposalBlockParts.Not().PickRandom(); ok {
// Ensure that the peer's PartSetHeader is correct
blockMeta := conR.blockStore.LoadBlockMeta(prs.Height)
if !blockMeta.PartsHeader.Equals(prs.ProposalBlockPartsHeader) {
blockMeta := conR.conS.blockStore.LoadBlockMeta(prs.Height)
if blockMeta == nil {
log.Warn("Failed to load block meta", "peer height", prs.Height, "our height", rs.Height, "blockstore height", conR.conS.blockStore.Height(), "pv", conR.conS.privValidator)
time.Sleep(peerGossipSleepDuration)
continue OUTER_LOOP
} else if !blockMeta.PartsHeader.Equals(prs.ProposalBlockPartsHeader) {
log.Info("Peer ProposalBlockPartsHeader mismatch, sleeping",
"peerHeight", prs.Height, "blockPartsHeader", blockMeta.PartsHeader, "peerBlockPartsHeader", prs.ProposalBlockPartsHeader)
time.Sleep(peerGossipSleepDuration)
continue OUTER_LOOP
}
// Load the part
part := conR.blockStore.LoadBlockPart(prs.Height, index)
part := conR.conS.blockStore.LoadBlockPart(prs.Height, index)
if part == nil {
log.Warn("Could not load part", "index", index,
"peerHeight", prs.Height, "blockPartsHeader", blockMeta.PartsHeader, "peerBlockPartsHeader", prs.ProposalBlockPartsHeader)
@ -391,13 +464,13 @@ OUTER_LOOP:
// Send Proposal && ProposalPOL BitArray?
if rs.Proposal != nil && !prs.Proposal {
// Proposal
// Proposal: share the proposal metadata with peer.
{
msg := &ProposalMessage{Proposal: rs.Proposal}
peer.Send(DataChannel, struct{ ConsensusMessage }{msg})
ps.SetHasProposal(rs.Proposal)
}
// ProposalPOL.
// ProposalPOL: lets peer know which POL votes we have so far.
// Peer must receive ProposalMessage first.
// rs.Proposal was validated, so rs.Proposal.POLRound <= rs.Round,
// so we definitely have rs.Votes.Prevotes(rs.Proposal.POLRound).
@ -492,8 +565,8 @@ OUTER_LOOP:
if prs.Height != 0 && rs.Height >= prs.Height+2 {
// Load the block commit for prs.Height,
// which contains precommit signatures for prs.Height.
commit := conR.blockStore.LoadBlockCommit(prs.Height)
log.Debug("Loaded BlockCommit for catch-up", "height", prs.Height, "commit", commit)
commit := conR.conS.blockStore.LoadBlockCommit(prs.Height)
log.Info("Loaded BlockCommit for catch-up", "height", prs.Height, "commit", commit)
if ps.PickSendVote(commit) {
log.Debug("Picked Catchup commit to send")
continue OUTER_LOOP
@ -516,6 +589,112 @@ OUTER_LOOP:
}
}
func (conR *ConsensusReactor) queryMaj23Routine(peer *p2p.Peer, ps *PeerState) {
log := log.New("peer", peer)
OUTER_LOOP:
for {
// Manage disconnects from self or peer.
if !peer.IsRunning() || !conR.IsRunning() {
log.Notice(Fmt("Stopping queryMaj23Routine for %v.", peer))
return
}
// Maybe send Height/Round/Prevotes
{
rs := conR.conS.GetRoundState()
prs := ps.GetRoundState()
if rs.Height == prs.Height {
if maj23, ok := rs.Votes.Prevotes(prs.Round).TwoThirdsMajority(); ok {
peer.TrySend(StateChannel, struct{ ConsensusMessage }{&VoteSetMaj23Message{
Height: prs.Height,
Round: prs.Round,
Type: types.VoteTypePrevote,
BlockID: maj23,
}})
time.Sleep(peerQueryMaj23SleepDuration)
}
}
}
// Maybe send Height/Round/Precommits
{
rs := conR.conS.GetRoundState()
prs := ps.GetRoundState()
if rs.Height == prs.Height {
if maj23, ok := rs.Votes.Precommits(prs.Round).TwoThirdsMajority(); ok {
peer.TrySend(StateChannel, struct{ ConsensusMessage }{&VoteSetMaj23Message{
Height: prs.Height,
Round: prs.Round,
Type: types.VoteTypePrecommit,
BlockID: maj23,
}})
time.Sleep(peerQueryMaj23SleepDuration)
}
}
}
// Maybe send Height/Round/ProposalPOL
{
rs := conR.conS.GetRoundState()
prs := ps.GetRoundState()
if rs.Height == prs.Height && prs.ProposalPOLRound >= 0 {
if maj23, ok := rs.Votes.Prevotes(prs.ProposalPOLRound).TwoThirdsMajority(); ok {
peer.TrySend(StateChannel, struct{ ConsensusMessage }{&VoteSetMaj23Message{
Height: prs.Height,
Round: prs.ProposalPOLRound,
Type: types.VoteTypePrevote,
BlockID: maj23,
}})
time.Sleep(peerQueryMaj23SleepDuration)
}
}
}
// Little point sending LastCommitRound/LastCommit,
// These are fleeting and non-blocking.
// Maybe send Height/CatchupCommitRound/CatchupCommit.
{
prs := ps.GetRoundState()
if prs.CatchupCommitRound != -1 && 0 < prs.Height && prs.Height <= conR.conS.blockStore.Height() {
var commit *types.Commit
if prs.Height == conR.conS.blockStore.Height() {
commit = conR.conS.blockStore.LoadSeenCommit(prs.Height)
} else {
commit = conR.conS.blockStore.LoadBlockCommit(prs.Height)
}
peer.TrySend(StateChannel, struct{ ConsensusMessage }{&VoteSetMaj23Message{
Height: prs.Height,
Round: commit.Round(),
Type: types.VoteTypePrecommit,
BlockID: commit.BlockID,
}})
time.Sleep(peerQueryMaj23SleepDuration)
}
}
time.Sleep(peerQueryMaj23SleepDuration)
continue OUTER_LOOP
}
}
func (conR *ConsensusReactor) String() string {
return conR.StringIndented("")
}
func (conR *ConsensusReactor) StringIndented(indent string) string {
s := "ConsensusReactor{\n"
s += indent + " " + conR.conS.StringIndented(indent+" ") + "\n"
for _, peer := range conR.Switch.Peers().List() {
ps := peer.Data.Get(types.PeerStateKey).(*PeerState)
s += indent + " " + ps.StringIndented(indent+" ") + "\n"
}
s += indent + "}"
return s
}
//-----------------------------------------------------------------------------
// Read only when returned by PeerState.GetRoundState().
@ -537,6 +716,30 @@ type PeerRoundState struct {
CatchupCommit *BitArray // All commit precommits peer has for this height & CatchupCommitRound
}
func (prs PeerRoundState) String() string {
return prs.StringIndented("")
}
func (prs PeerRoundState) StringIndented(indent string) string {
return fmt.Sprintf(`PeerRoundState{
%s %v/%v/%v @%v
%s Proposal %v -> %v
%s POL %v (round %v)
%s Prevotes %v
%s Precommits %v
%s LastCommit %v (round %v)
%s Catchup %v (round %v)
%s}`,
indent, prs.Height, prs.Round, prs.Step, prs.StartTime,
indent, prs.ProposalBlockPartsHeader, prs.ProposalBlockParts,
indent, prs.ProposalPOL, prs.ProposalPOLRound,
indent, prs.Prevotes,
indent, prs.Precommits,
indent, prs.LastCommit, prs.LastCommitRound,
indent, prs.CatchupCommit, prs.CatchupCommitRound,
indent)
}
//-----------------------------------------------------------------------------
var (
@ -613,8 +816,8 @@ func (ps *PeerState) SetHasProposalBlockPart(height int, round int, index int) {
// Convenience function to send vote to peer.
// Returns true if vote was sent.
func (ps *PeerState) PickSendVote(votes types.VoteSetReader) (ok bool) {
if index, vote, ok := ps.PickVoteToSend(votes); ok {
msg := &VoteMessage{index, vote}
if vote, ok := ps.PickVoteToSend(votes); ok {
msg := &VoteMessage{vote}
ps.Peer.Send(VoteChannel, struct{ ConsensusMessage }{msg})
return true
}
@ -622,12 +825,12 @@ func (ps *PeerState) PickSendVote(votes types.VoteSetReader) (ok bool) {
}
// votes: Must be the correct Size() for the Height().
func (ps *PeerState) PickVoteToSend(votes types.VoteSetReader) (index int, vote *types.Vote, ok bool) {
func (ps *PeerState) PickVoteToSend(votes types.VoteSetReader) (vote *types.Vote, ok bool) {
ps.mtx.Lock()
defer ps.mtx.Unlock()
if votes.Size() == 0 {
return 0, nil, false
return nil, false
}
height, round, type_, size := votes.Height(), votes.Round(), votes.Type(), votes.Size()
@ -640,16 +843,20 @@ func (ps *PeerState) PickVoteToSend(votes types.VoteSetReader) (index int, vote
psVotes := ps.getVoteBitArray(height, round, type_)
if psVotes == nil {
return 0, nil, false // Not something worth sending
return nil, false // Not something worth sending
}
if index, ok := votes.BitArray().Sub(psVotes).PickRandom(); ok {
ps.setHasVote(height, round, type_, index)
return index, votes.GetByIndex(index), true
return votes.GetByIndex(index), true
}
return 0, nil, false
return nil, false
}
func (ps *PeerState) getVoteBitArray(height, round int, type_ byte) *BitArray {
if !types.IsVoteTypeValid(type_) {
PanicSanity("Invalid vote type")
}
if ps.Height == height {
if ps.Round == round {
switch type_ {
@ -657,8 +864,6 @@ func (ps *PeerState) getVoteBitArray(height, round int, type_ byte) *BitArray {
return ps.Prevotes
case types.VoteTypePrecommit:
return ps.Precommits
default:
PanicSanity(Fmt("Unexpected vote type %X", type_))
}
}
if ps.CatchupCommitRound == round {
@ -667,8 +872,14 @@ func (ps *PeerState) getVoteBitArray(height, round int, type_ byte) *BitArray {
return nil
case types.VoteTypePrecommit:
return ps.CatchupCommit
default:
PanicSanity(Fmt("Unexpected vote type %X", type_))
}
}
if ps.ProposalPOLRound == round {
switch type_ {
case types.VoteTypePrevote:
return ps.ProposalPOL
case types.VoteTypePrecommit:
return nil
}
}
return nil
@ -680,8 +891,6 @@ func (ps *PeerState) getVoteBitArray(height, round int, type_ byte) *BitArray {
return nil
case types.VoteTypePrecommit:
return ps.LastCommit
default:
PanicSanity(Fmt("Unexpected vote type %X", type_))
}
}
return nil
@ -741,56 +950,19 @@ func (ps *PeerState) ensureVoteBitArrays(height int, numValidators int) {
}
}
func (ps *PeerState) SetHasVote(vote *types.Vote, index int) {
func (ps *PeerState) SetHasVote(vote *types.Vote) {
ps.mtx.Lock()
defer ps.mtx.Unlock()
ps.setHasVote(vote.Height, vote.Round, vote.Type, index)
ps.setHasVote(vote.Height, vote.Round, vote.Type, vote.ValidatorIndex)
}
func (ps *PeerState) setHasVote(height int, round int, type_ byte, index int) {
log := log.New("peer", ps.Peer, "peerRound", ps.Round, "height", height, "round", round)
if type_ != types.VoteTypePrevote && type_ != types.VoteTypePrecommit {
PanicSanity("Invalid vote type")
}
log.Info("setHasVote(LastCommit)", "lastCommit", ps.LastCommit, "index", index)
if ps.Height == height {
if ps.Round == round {
switch type_ {
case types.VoteTypePrevote:
ps.Prevotes.SetIndex(index, true)
log.Debug("SetHasVote(round-match)", "prevotes", ps.Prevotes, "index", index)
case types.VoteTypePrecommit:
ps.Precommits.SetIndex(index, true)
log.Debug("SetHasVote(round-match)", "precommits", ps.Precommits, "index", index)
}
} else if ps.CatchupCommitRound == round {
switch type_ {
case types.VoteTypePrevote:
case types.VoteTypePrecommit:
ps.CatchupCommit.SetIndex(index, true)
log.Debug("SetHasVote(CatchupCommit)", "precommits", ps.Precommits, "index", index)
}
} else if ps.ProposalPOLRound == round {
switch type_ {
case types.VoteTypePrevote:
ps.ProposalPOL.SetIndex(index, true)
log.Debug("SetHasVote(ProposalPOL)", "prevotes", ps.Prevotes, "index", index)
case types.VoteTypePrecommit:
}
}
} else if ps.Height == height+1 {
if ps.LastCommitRound == round {
switch type_ {
case types.VoteTypePrevote:
case types.VoteTypePrecommit:
ps.LastCommit.SetIndex(index, true)
log.Debug("setHasVote(LastCommit)", "lastCommit", ps.LastCommit, "index", index)
}
}
} else {
// Does not apply.
}
// NOTE: some may be nil BitArrays -> no side effects.
ps.getVoteBitArray(height, round, type_).SetIndex(index, true)
}
func (ps *PeerState) ApplyNewRoundStepMessage(msg *NewRoundStepMessage) {
@ -858,31 +1030,66 @@ func (ps *PeerState) ApplyCommitStepMessage(msg *CommitStepMessage) {
ps.ProposalBlockParts = msg.BlockParts
}
func (ps *PeerState) ApplyHasVoteMessage(msg *HasVoteMessage) {
func (ps *PeerState) ApplyProposalPOLMessage(msg *ProposalPOLMessage) {
ps.mtx.Lock()
defer ps.mtx.Unlock()
if ps.Height != msg.Height {
return
}
if ps.ProposalPOLRound != msg.ProposalPOLRound {
return
}
ps.setHasVote(msg.Height, msg.Round, msg.Type, msg.Index)
// TODO: Merge onto existing ps.ProposalPOL?
// We might have sent some prevotes in the meantime.
ps.ProposalPOL = msg.ProposalPOL
}
func (ps *PeerState) ApplyProposalPOLMessage(msg *ProposalPOLMessage) {
func (ps *PeerState) ApplyHasVoteMessage(msg *HasVoteMessage) {
ps.mtx.Lock()
defer ps.mtx.Unlock()
if ps.Height != msg.Height {
return
}
if ps.ProposalPOLRound != msg.ProposalPOLRound {
return
ps.setHasVote(msg.Height, msg.Round, msg.Type, msg.Index)
}
// The peer has responded with a bitarray of votes that it has
// of the corresponding BlockID.
// ourVotes: BitArray of votes we have for msg.BlockID
// NOTE: if ourVotes is nil (e.g. msg.Height < rs.Height),
// we conservatively overwrite ps's votes w/ msg.Votes.
func (ps *PeerState) ApplyVoteSetBitsMessage(msg *VoteSetBitsMessage, ourVotes *BitArray) {
ps.mtx.Lock()
defer ps.mtx.Unlock()
votes := ps.getVoteBitArray(msg.Height, msg.Round, msg.Type)
if votes != nil {
if ourVotes == nil {
votes.Update(msg.Votes)
} else {
otherVotes := votes.Sub(ourVotes)
hasVotes := otherVotes.Or(msg.Votes)
votes.Update(hasVotes)
}
}
}
// TODO: Merge onto existing ps.ProposalPOL?
// We might have sent some prevotes in the meantime.
ps.ProposalPOL = msg.ProposalPOL
func (ps *PeerState) String() string {
return ps.StringIndented("")
}
func (ps *PeerState) StringIndented(indent string) string {
return fmt.Sprintf(`PeerState{
%s Key %v
%s PRS %v
%s}`,
indent, ps.Peer.Key,
indent, ps.PeerRoundState.StringIndented(indent+" "),
indent)
}
//-----------------------------------------------------------------------------
@ -896,6 +1103,8 @@ const (
msgTypeBlockPart = byte(0x13) // both block & POL
msgTypeVote = byte(0x14)
msgTypeHasVote = byte(0x15)
msgTypeVoteSetMaj23 = byte(0x16)
msgTypeVoteSetBits = byte(0x17)
)
type ConsensusMessage interface{}
@ -909,6 +1118,8 @@ var _ = wire.RegisterInterface(
wire.ConcreteType{&BlockPartMessage{}, msgTypeBlockPart},
wire.ConcreteType{&VoteMessage{}, msgTypeVote},
wire.ConcreteType{&HasVoteMessage{}, msgTypeHasVote},
wire.ConcreteType{&VoteSetMaj23Message{}, msgTypeVoteSetMaj23},
wire.ConcreteType{&VoteSetBitsMessage{}, msgTypeVoteSetBits},
)
// TODO: check for unnecessary extra bytes at the end.
@ -985,12 +1196,11 @@ func (m *BlockPartMessage) String() string {
//-------------------------------------
type VoteMessage struct {
ValidatorIndex int
Vote *types.Vote
Vote *types.Vote
}
func (m *VoteMessage) String() string {
return fmt.Sprintf("[Vote VI:%v V:%v VI:%v]", m.ValidatorIndex, m.Vote, m.ValidatorIndex)
return fmt.Sprintf("[Vote %v]", m.Vote)
}
//-------------------------------------
@ -1005,3 +1215,30 @@ type HasVoteMessage struct {
func (m *HasVoteMessage) String() string {
return fmt.Sprintf("[HasVote VI:%v V:{%v/%02d/%v} VI:%v]", m.Index, m.Height, m.Round, m.Type, m.Index)
}
//-------------------------------------
type VoteSetMaj23Message struct {
Height int
Round int
Type byte
BlockID types.BlockID
}
func (m *VoteSetMaj23Message) String() string {
return fmt.Sprintf("[VSM23 %v/%02d/%v %v]", m.Height, m.Round, m.Type, m.BlockID)
}
//-------------------------------------
type VoteSetBitsMessage struct {
Height int
Round int
Type byte
BlockID types.BlockID
Votes *BitArray
}
func (m *VoteSetBitsMessage) String() string {
return fmt.Sprintf("[VSB %v/%02d/%v %v %v]", m.Height, m.Round, m.Type, m.BlockID, m.Votes)
}

+ 351
- 0
consensus/reactor_test.go View File

@ -0,0 +1,351 @@
package consensus
import (
"bytes"
"sync"
"testing"
"time"
"github.com/tendermint/tendermint/config/tendermint_test"
. "github.com/tendermint/go-common"
cfg "github.com/tendermint/go-config"
"github.com/tendermint/go-crypto"
"github.com/tendermint/go-events"
"github.com/tendermint/go-logger"
"github.com/tendermint/go-p2p"
"github.com/tendermint/tendermint/types"
)
func init() {
config = tendermint_test.ResetConfig("consensus_reactor_test")
}
func resetConfigTimeouts() {
logger.SetLogLevel("notice")
//config.Set("log_level", "notice")
config.Set("timeout_propose", 2000)
// config.Set("timeout_propose_delta", 500)
// config.Set("timeout_prevote", 1000)
// config.Set("timeout_prevote_delta", 500)
// config.Set("timeout_precommit", 1000)
// config.Set("timeout_precommit_delta", 500)
// config.Set("timeout_commit", 1000)
}
func TestReactor(t *testing.T) {
resetConfigTimeouts()
N := 4
css := randConsensusNet(N)
reactors := make([]*ConsensusReactor, N)
eventChans := make([]chan interface{}, N)
for i := 0; i < N; i++ {
reactors[i] = NewConsensusReactor(css[i], false)
reactors[i].SetPrivValidator(css[i].privValidator)
eventSwitch := events.NewEventSwitch()
_, err := eventSwitch.Start()
if err != nil {
t.Fatalf("Failed to start switch: %v", err)
}
reactors[i].SetEventSwitch(eventSwitch)
eventChans[i] = subscribeToEvent(eventSwitch, "tester", types.EventStringNewBlock(), 1)
}
p2p.MakeConnectedSwitches(N, func(i int, s *p2p.Switch) *p2p.Switch {
s.AddReactor("CONSENSUS", reactors[i])
return s
}, p2p.Connect2Switches)
// wait till everyone makes the first new block
wg := new(sync.WaitGroup)
wg.Add(N)
for i := 0; i < N; i++ {
go func(j int) {
<-eventChans[j]
wg.Done()
}(i)
}
// Make wait into a channel
done := make(chan struct{})
go func() {
wg.Wait()
close(done)
}()
tick := time.NewTicker(time.Second * 3)
select {
case <-done:
case <-tick.C:
t.Fatalf("Timed out waiting for all validators to commit first block")
}
}
// 4 validators. 1 is byzantine. The other three are partitioned into A (1 val) and B (2 vals).
// byzantine validator sends conflicting proposals into A and B,
// and prevotes/precommits on both of them.
// B sees a commit, A doesn't.
// Byzantine validator refuses to prevote.
// Heal partition and ensure A sees the commit
func TestByzantine(t *testing.T) {
resetConfigTimeouts()
N := 4
css := randConsensusNet(N)
switches := make([]*p2p.Switch, N)
for i := 0; i < N; i++ {
switches[i] = p2p.NewSwitch(cfg.NewMapConfig(nil))
}
reactors := make([]p2p.Reactor, N)
eventChans := make([]chan interface{}, N)
for i := 0; i < N; i++ {
var privVal PrivValidator
privVal = css[i].privValidator
if i == 0 {
privVal = NewByzantinePrivValidator(privVal.(*types.PrivValidator))
// make byzantine
css[i].decideProposal = func(j int) func(int, int) {
return func(height, round int) {
byzantineDecideProposalFunc(height, round, css[j], switches[j])
}
}(i)
css[i].doPrevote = func(height, round int) {}
}
eventSwitch := events.NewEventSwitch()
_, err := eventSwitch.Start()
if err != nil {
t.Fatalf("Failed to start switch: %v", err)
}
eventChans[i] = subscribeToEvent(eventSwitch, "tester", types.EventStringNewBlock(), 1)
conR := NewConsensusReactor(css[i], false)
conR.SetPrivValidator(privVal)
conR.SetEventSwitch(eventSwitch)
var conRI p2p.Reactor
conRI = conR
if i == 0 {
conRI = NewByzantineReactor(conR)
}
reactors[i] = conRI
}
p2p.MakeConnectedSwitches(N, func(i int, s *p2p.Switch) *p2p.Switch {
// ignore new switch s, we already made ours
switches[i].AddReactor("CONSENSUS", reactors[i])
return switches[i]
}, func(sws []*p2p.Switch, i, j int) {
// the network starts partitioned with globally active adversary
if i != 0 {
return
}
p2p.Connect2Switches(sws, i, j)
})
// byz proposer sends one block to peers[0]
// and the other block to peers[1] and peers[2].
// note peers and switches order don't match.
peers := switches[0].Peers().List()
ind0 := getSwitchIndex(switches, peers[0])
ind1 := getSwitchIndex(switches, peers[1])
ind2 := getSwitchIndex(switches, peers[2])
// connect the 2 peers in the larger partition
p2p.Connect2Switches(switches, ind1, ind2)
// wait for someone in the big partition to make a block
select {
case <-eventChans[ind2]:
}
log.Notice("A block has been committed. Healing partition")
// connect the partitions
p2p.Connect2Switches(switches, ind0, ind1)
p2p.Connect2Switches(switches, ind0, ind2)
// wait till everyone makes the first new block
// (one of them already has)
wg := new(sync.WaitGroup)
wg.Add(2)
for i := 1; i < N-1; i++ {
go func(j int) {
<-eventChans[j]
wg.Done()
}(i)
}
done := make(chan struct{})
go func() {
wg.Wait()
close(done)
}()
tick := time.NewTicker(time.Second * 10)
select {
case <-done:
case <-tick.C:
for i, reactor := range reactors {
t.Log(Fmt("Consensus Reactor %v", i))
t.Log(Fmt("%v", reactor))
}
t.Fatalf("Timed out waiting for all validators to commit first block")
}
}
func getSwitchIndex(switches []*p2p.Switch, peer *p2p.Peer) int {
for i, s := range switches {
if bytes.Equal(peer.NodeInfo.PubKey.Address(), s.NodeInfo().PubKey.Address()) {
return i
}
}
panic("didnt find peer in switches")
return -1
}
//-------------------------------
// byzantine consensus functions
func byzantineDecideProposalFunc(height, round int, cs *ConsensusState, sw *p2p.Switch) {
// byzantine user should create two proposals and try to split the vote.
// Avoid sending on internalMsgQueue and running consensus state.
// Create a new proposal block from state/txs from the mempool.
block1, blockParts1 := cs.createProposalBlock()
polRound, polBlockID := cs.Votes.POLInfo()
proposal1 := types.NewProposal(height, round, blockParts1.Header(), polRound, polBlockID)
cs.privValidator.SignProposal(cs.state.ChainID, proposal1) // byzantine doesnt err
// Create a new proposal block from state/txs from the mempool.
block2, blockParts2 := cs.createProposalBlock()
polRound, polBlockID = cs.Votes.POLInfo()
proposal2 := types.NewProposal(height, round, blockParts2.Header(), polRound, polBlockID)
cs.privValidator.SignProposal(cs.state.ChainID, proposal2) // byzantine doesnt err
block1Hash := block1.Hash()
block2Hash := block2.Hash()
// broadcast conflicting proposals/block parts to peers
peers := sw.Peers().List()
log.Notice("Byzantine: broadcasting conflicting proposals", "peers", len(peers))
for i, peer := range peers {
if i < len(peers)/2 {
go sendProposalAndParts(height, round, cs, peer, proposal1, block1Hash, blockParts1)
} else {
go sendProposalAndParts(height, round, cs, peer, proposal2, block2Hash, blockParts2)
}
}
}
func sendProposalAndParts(height, round int, cs *ConsensusState, peer *p2p.Peer, proposal *types.Proposal, blockHash []byte, parts *types.PartSet) {
// proposal
msg := &ProposalMessage{Proposal: proposal}
peer.Send(DataChannel, struct{ ConsensusMessage }{msg})
// parts
for i := 0; i < parts.Total(); i++ {
part := parts.GetPart(i)
msg := &BlockPartMessage{
Height: height, // This tells peer that this part applies to us.
Round: round, // This tells peer that this part applies to us.
Part: part,
}
peer.Send(DataChannel, struct{ ConsensusMessage }{msg})
}
// votes
cs.mtx.Lock()
prevote, _ := cs.signVote(types.VoteTypePrevote, blockHash, parts.Header())
precommit, _ := cs.signVote(types.VoteTypePrecommit, blockHash, parts.Header())
cs.mtx.Unlock()
peer.Send(VoteChannel, struct{ ConsensusMessage }{&VoteMessage{prevote}})
peer.Send(VoteChannel, struct{ ConsensusMessage }{&VoteMessage{precommit}})
}
//----------------------------------------
// byzantine consensus reactor
type ByzantineReactor struct {
Service
reactor *ConsensusReactor
}
func NewByzantineReactor(conR *ConsensusReactor) *ByzantineReactor {
return &ByzantineReactor{
Service: conR,
reactor: conR,
}
}
func (br *ByzantineReactor) SetSwitch(s *p2p.Switch) { br.reactor.SetSwitch(s) }
func (br *ByzantineReactor) GetChannels() []*p2p.ChannelDescriptor { return br.reactor.GetChannels() }
func (br *ByzantineReactor) AddPeer(peer *p2p.Peer) {
if !br.reactor.IsRunning() {
return
}
// Create peerState for peer
peerState := NewPeerState(peer)
peer.Data.Set(types.PeerStateKey, peerState)
// Send our state to peer.
// If we're fast_syncing, broadcast a RoundStepMessage later upon SwitchToConsensus().
if !br.reactor.fastSync {
br.reactor.sendNewRoundStepMessage(peer)
}
}
func (br *ByzantineReactor) RemovePeer(peer *p2p.Peer, reason interface{}) {
br.reactor.RemovePeer(peer, reason)
}
func (br *ByzantineReactor) Receive(chID byte, peer *p2p.Peer, msgBytes []byte) {
br.reactor.Receive(chID, peer, msgBytes)
}
//----------------------------------------
// byzantine privValidator
type ByzantinePrivValidator struct {
Address []byte `json:"address"`
types.Signer `json:"-"`
mtx sync.Mutex
}
// Return a priv validator that will sign anything
func NewByzantinePrivValidator(pv *types.PrivValidator) *ByzantinePrivValidator {
return &ByzantinePrivValidator{
Address: pv.Address,
Signer: pv.Signer,
}
}
func (privVal *ByzantinePrivValidator) GetAddress() []byte {
return privVal.Address
}
func (privVal *ByzantinePrivValidator) SignVote(chainID string, vote *types.Vote) error {
privVal.mtx.Lock()
defer privVal.mtx.Unlock()
// Sign
vote.Signature = privVal.Sign(types.SignBytes(chainID, vote)).(crypto.SignatureEd25519)
return nil
}
func (privVal *ByzantinePrivValidator) SignProposal(chainID string, proposal *types.Proposal) error {
privVal.mtx.Lock()
defer privVal.mtx.Unlock()
// Sign
proposal.Signature = privVal.Sign(types.SignBytes(chainID, proposal)).(crypto.SignatureEd25519)
return nil
}
func (privVal *ByzantinePrivValidator) String() string {
return Fmt("PrivValidator{%X}", privVal.Address)
}

+ 1
- 1
consensus/replay.go View File

@ -68,7 +68,7 @@ func (cs *ConsensusState) readReplayMessage(msgBytes []byte, newStepCh chan inte
case *VoteMessage:
v := msg.Vote
log.Notice("Replay: Vote", "height", v.Height, "round", v.Round, "type", v.Type,
"hash", v.BlockHash, "header", v.BlockPartsHeader, "peer", peerKey)
"blockID", v.BlockID, "peer", peerKey)
}
cs.handleMsg(m, cs.RoundState)


+ 17
- 13
consensus/replay_test.go View File

@ -21,9 +21,9 @@ var baseStepChanges = []int{3, 6, 8}
// test recovery from each line in each testCase
var testCases = []*testCase{
newTestCase("empty_block", baseStepChanges), // empty block (has 1 block part)
newTestCase("small_block1", baseStepChanges), // small block with txs in 1 block part
newTestCase("small_block2", []int{3, 8, 10}), // small block with txs across 3 smaller block parts
newTestCase("empty_block", baseStepChanges), // empty block (has 1 block part)
newTestCase("small_block1", baseStepChanges), // small block with txs in 1 block part
newTestCase("small_block2", []int{3, 10, 12}), // small block with txs across 5 smaller block parts
}
type testCase struct {
@ -110,9 +110,13 @@ func runReplayTest(t *testing.T, cs *ConsensusState, walDir string, newBlockCh c
cs.Wait()
}
func toPV(pv PrivValidator) *types.PrivValidator {
return pv.(*types.PrivValidator)
}
func setupReplayTest(thisCase *testCase, nLines int, crashAfter bool) (*ConsensusState, chan interface{}, string, string) {
fmt.Println("-------------------------------------")
log.Notice(Fmt("Starting replay test of %d lines of WAL (crash before write)", nLines))
log.Notice(Fmt("Starting replay test of %d lines of WAL. Crash after = %v", nLines, crashAfter))
lineStep := nLines
if crashAfter {
@ -128,10 +132,10 @@ func setupReplayTest(thisCase *testCase, nLines int, crashAfter bool) (*Consensu
cs := fixedConsensusStateDummy()
// set the last step according to when we crashed vs the wal
cs.privValidator.LastHeight = 1 // first block
cs.privValidator.LastStep = thisCase.stepMap[lineStep]
toPV(cs.privValidator).LastHeight = 1 // first block
toPV(cs.privValidator).LastStep = thisCase.stepMap[lineStep]
fmt.Println("LAST STEP", cs.privValidator.LastStep)
log.Warn("setupReplayTest", "LastStep", toPV(cs.privValidator).LastStep)
newBlockCh := subscribeToEvent(cs.evsw, "tester", types.EventStringNewBlock(), 1)
@ -168,8 +172,8 @@ func TestReplayCrashBeforeWritePropose(t *testing.T) {
if err != nil {
t.Fatalf("Error reading json data: %v", err)
}
cs.privValidator.LastSignBytes = types.SignBytes(cs.state.ChainID, proposal.Proposal)
cs.privValidator.LastSignature = proposal.Proposal.Signature
toPV(cs.privValidator).LastSignBytes = types.SignBytes(cs.state.ChainID, proposal.Proposal)
toPV(cs.privValidator).LastSignature = proposal.Proposal.Signature
runReplayTest(t, cs, walDir, newBlockCh, thisCase, lineNum)
}
}
@ -187,8 +191,8 @@ func TestReplayCrashBeforeWritePrevote(t *testing.T) {
if err != nil {
t.Fatalf("Error reading json data: %v", err)
}
cs.privValidator.LastSignBytes = types.SignBytes(cs.state.ChainID, vote.Vote)
cs.privValidator.LastSignature = vote.Vote.Signature
toPV(cs.privValidator).LastSignBytes = types.SignBytes(cs.state.ChainID, vote.Vote)
toPV(cs.privValidator).LastSignature = vote.Vote.Signature
})
runReplayTest(t, cs, walDir, newBlockCh, thisCase, lineNum)
}
@ -207,8 +211,8 @@ func TestReplayCrashBeforeWritePrecommit(t *testing.T) {
if err != nil {
t.Fatalf("Error reading json data: %v", err)
}
cs.privValidator.LastSignBytes = types.SignBytes(cs.state.ChainID, vote.Vote)
cs.privValidator.LastSignature = vote.Vote.Signature
toPV(cs.privValidator).LastSignBytes = types.SignBytes(cs.state.ChainID, vote.Vote)
toPV(cs.privValidator).LastSignature = vote.Vote.Signature
})
runReplayTest(t, cs, walDir, newBlockCh, thisCase, lineNum)
}


+ 85
- 68
consensus/state.go View File

@ -209,6 +209,12 @@ func (ti *timeoutInfo) String() string {
return fmt.Sprintf("%v ; %d/%d %v", ti.Duration, ti.Height, ti.Round, ti.Step)
}
type PrivValidator interface {
GetAddress() []byte
SignVote(chainID string, vote *types.Vote) error
SignProposal(chainID string, proposal *types.Proposal) error
}
// Tracks consensus state across block heights and rounds.
type ConsensusState struct {
BaseService
@ -217,7 +223,7 @@ type ConsensusState struct {
proxyAppConn proxy.AppConnConsensus
blockStore *bc.BlockStore
mempool *mempl.Mempool
privValidator *types.PrivValidator
privValidator PrivValidator
mtx sync.Mutex
RoundState
@ -236,6 +242,11 @@ type ConsensusState struct {
replayMode bool // so we don't log signing errors during replay
nSteps int // used for testing to limit the number of transitions the state makes
// allow certain function to be overwritten for testing
decideProposal func(height, round int)
doPrevote func(height, round int)
setProposal func(proposal *types.Proposal) error
}
func NewConsensusState(config cfg.Config, state *sm.State, proxyAppConn proxy.AppConnConsensus, blockStore *bc.BlockStore, mempool *mempl.Mempool) *ConsensusState {
@ -251,6 +262,11 @@ func NewConsensusState(config cfg.Config, state *sm.State, proxyAppConn proxy.Ap
tockChan: make(chan timeoutInfo, tickTockBufferSize),
timeoutParams: InitTimeoutParamsFromConfig(config),
}
// set function defaults (may be overwritten before calling Start)
cs.decideProposal = cs.defaultDecideProposal
cs.doPrevote = cs.defaultDoPrevote
cs.setProposal = cs.defaultSetProposal
cs.updateToState(state)
// Don't call scheduleRound0 yet.
// We do that upon Start().
@ -295,7 +311,7 @@ func (cs *ConsensusState) GetValidators() (int, []*types.Validator) {
return cs.state.LastBlockHeight, cs.state.Validators.Copy().Validators
}
func (cs *ConsensusState) SetPrivValidator(priv *types.PrivValidator) {
func (cs *ConsensusState) SetPrivValidator(priv PrivValidator) {
cs.mtx.Lock()
defer cs.mtx.Unlock()
cs.privValidator = priv
@ -374,15 +390,15 @@ func (cs *ConsensusState) OpenWAL(walDir string) (err error) {
// TODO: should these return anything or let callers just use events?
// May block on send if queue is full.
func (cs *ConsensusState) AddVote(valIndex int, vote *types.Vote, peerKey string) (added bool, address []byte, err error) {
func (cs *ConsensusState) AddVote(vote *types.Vote, peerKey string) (added bool, err error) {
if peerKey == "" {
cs.internalMsgQueue <- msgInfo{&VoteMessage{valIndex, vote}, ""}
cs.internalMsgQueue <- msgInfo{&VoteMessage{vote}, ""}
} else {
cs.peerMsgQueue <- msgInfo{&VoteMessage{valIndex, vote}, peerKey}
cs.peerMsgQueue <- msgInfo{&VoteMessage{vote}, peerKey}
}
// TODO: wait for event?!
return false, nil, nil
return false, nil
}
// May block on send if queue is full.
@ -472,11 +488,11 @@ func (cs *ConsensusState) reconstructLastCommit(state *sm.State) {
}
seenCommit := cs.blockStore.LoadSeenCommit(state.LastBlockHeight)
lastPrecommits := types.NewVoteSet(cs.config.GetString("chain_id"), state.LastBlockHeight, seenCommit.Round(), types.VoteTypePrecommit, state.LastValidators)
for idx, precommit := range seenCommit.Precommits {
for _, precommit := range seenCommit.Precommits {
if precommit == nil {
continue
}
added, _, err := lastPrecommits.AddByIndex(idx, precommit)
added, err := lastPrecommits.AddVote(precommit)
if !added || err != nil {
PanicCrisis(Fmt("Failed to reconstruct LastCommit: %v", err))
}
@ -694,7 +710,7 @@ func (cs *ConsensusState) handleMsg(mi msgInfo, rs RoundState) {
case *VoteMessage:
// attempt to add the vote and dupeout the validator if its a duplicate signature
// if the vote gives us a 2/3-any or 2/3-one, we transition
err := cs.tryAddVote(msg.ValidatorIndex, msg.Vote, peerKey)
err := cs.tryAddVote(msg.Vote, peerKey)
if err == ErrAddingVote {
// TODO: punish peer
}
@ -825,16 +841,16 @@ func (cs *ConsensusState) enterPropose(height int, round int) {
return
}
if !bytes.Equal(cs.Validators.Proposer().Address, cs.privValidator.Address) {
if !bytes.Equal(cs.Validators.Proposer().Address, cs.privValidator.GetAddress()) {
log.Info("enterPropose: Not our turn to propose", "proposer", cs.Validators.Proposer().Address, "privValidator", cs.privValidator)
} else {
log.Info("enterPropose: Our turn to propose", "proposer", cs.Validators.Proposer().Address, "privValidator", cs.privValidator)
cs.decideProposal(height, round)
}
}
}
func (cs *ConsensusState) decideProposal(height, round int) {
func (cs *ConsensusState) defaultDecideProposal(height, round int) {
var block *types.Block
var blockParts *types.PartSet
@ -851,7 +867,8 @@ func (cs *ConsensusState) decideProposal(height, round int) {
}
// Make proposal
proposal := types.NewProposal(height, round, blockParts.Header(), cs.Votes.POLRound())
polRound, polBlockID := cs.Votes.POLInfo()
proposal := types.NewProposal(height, round, blockParts.Header(), polRound, polBlockID)
err := cs.privValidator.SignProposal(cs.state.ChainID, proposal)
if err == nil {
// Set fields
@ -874,7 +891,6 @@ func (cs *ConsensusState) decideProposal(height, round int) {
log.Warn("enterPropose: Error signing proposal", "height", height, "round", round, "error", err)
}
}
}
// Returns true if the proposal block is complete &&
@ -920,8 +936,7 @@ func (cs *ConsensusState) createProposalBlock() (block *types.Block, blockParts
Height: cs.Height,
Time: time.Now(),
NumTxs: len(txs),
LastBlockHash: cs.state.LastBlockHash,
LastBlockParts: cs.state.LastBlockParts,
LastBlockID: cs.state.LastBlockID,
ValidatorsHash: cs.state.Validators.Hash(),
AppHash: cs.state.AppHash, // state merkle root of txs from the previous block.
},
@ -931,7 +946,7 @@ func (cs *ConsensusState) createProposalBlock() (block *types.Block, blockParts
},
}
block.FillHeader()
blockParts = block.MakePartSet()
blockParts = block.MakePartSet(cs.config.GetInt("block_part_size"))
return block, blockParts
}
@ -972,10 +987,10 @@ func (cs *ConsensusState) enterPrevote(height int, round int) {
// (so we have more time to try and collect +2/3 prevotes for a single block)
}
func (cs *ConsensusState) doPrevote(height int, round int) {
func (cs *ConsensusState) defaultDoPrevote(height int, round int) {
// If a block is locked, prevote that.
if cs.LockedBlock != nil {
log.Info("enterPrevote: Block was locked")
log.Notice("enterPrevote: Block was locked")
cs.signAddVote(types.VoteTypePrevote, cs.LockedBlock.Hash(), cs.LockedBlockParts.Header())
return
}
@ -1046,14 +1061,14 @@ func (cs *ConsensusState) enterPrecommit(height int, round int) {
cs.newStep()
}()
hash, partsHeader, ok := cs.Votes.Prevotes(round).TwoThirdsMajority()
blockID, ok := cs.Votes.Prevotes(round).TwoThirdsMajority()
// If we don't have a polka, we must precommit nil
if !ok {
if cs.LockedBlock != nil {
log.Info("enterPrecommit: No +2/3 prevotes during enterPrecommit while we're locked. Precommitting nil")
log.Notice("enterPrecommit: No +2/3 prevotes during enterPrecommit while we're locked. Precommitting nil")
} else {
log.Info("enterPrecommit: No +2/3 prevotes during enterPrecommit. Precommitting nil.")
log.Notice("enterPrecommit: No +2/3 prevotes during enterPrecommit. Precommitting nil.")
}
cs.signAddVote(types.VoteTypePrecommit, nil, types.PartSetHeader{})
return
@ -1063,12 +1078,13 @@ func (cs *ConsensusState) enterPrecommit(height int, round int) {
types.FireEventPolka(cs.evsw, cs.RoundStateEvent())
// the latest POLRound should be this round
if cs.Votes.POLRound() < round {
PanicSanity(Fmt("This POLRound should be %v but got %", round, cs.Votes.POLRound()))
polRound, _ := cs.Votes.POLInfo()
if polRound < round {
PanicSanity(Fmt("This POLRound should be %v but got %", round, polRound))
}
// +2/3 prevoted nil. Unlock and precommit nil.
if len(hash) == 0 {
if len(blockID.Hash) == 0 {
if cs.LockedBlock == nil {
log.Notice("enterPrecommit: +2/3 prevoted for nil.")
} else {
@ -1085,17 +1101,17 @@ func (cs *ConsensusState) enterPrecommit(height int, round int) {
// At this point, +2/3 prevoted for a particular block.
// If we're already locked on that block, precommit it, and update the LockedRound
if cs.LockedBlock.HashesTo(hash) {
if cs.LockedBlock.HashesTo(blockID.Hash) {
log.Notice("enterPrecommit: +2/3 prevoted locked block. Relocking")
cs.LockedRound = round
types.FireEventRelock(cs.evsw, cs.RoundStateEvent())
cs.signAddVote(types.VoteTypePrecommit, hash, partsHeader)
cs.signAddVote(types.VoteTypePrecommit, blockID.Hash, blockID.PartsHeader)
return
}
// If +2/3 prevoted for proposal block, stage and precommit it
if cs.ProposalBlock.HashesTo(hash) {
log.Notice("enterPrecommit: +2/3 prevoted proposal block. Locking", "hash", hash)
if cs.ProposalBlock.HashesTo(blockID.Hash) {
log.Notice("enterPrecommit: +2/3 prevoted proposal block. Locking", "hash", blockID.Hash)
// Validate the block.
if err := cs.state.ValidateBlock(cs.ProposalBlock); err != nil {
PanicConsensus(Fmt("enterPrecommit: +2/3 prevoted for an invalid block: %v", err))
@ -1104,7 +1120,7 @@ func (cs *ConsensusState) enterPrecommit(height int, round int) {
cs.LockedBlock = cs.ProposalBlock
cs.LockedBlockParts = cs.ProposalBlockParts
types.FireEventLock(cs.evsw, cs.RoundStateEvent())
cs.signAddVote(types.VoteTypePrecommit, hash, partsHeader)
cs.signAddVote(types.VoteTypePrecommit, blockID.Hash, blockID.PartsHeader)
return
}
@ -1115,9 +1131,9 @@ func (cs *ConsensusState) enterPrecommit(height int, round int) {
cs.LockedRound = 0
cs.LockedBlock = nil
cs.LockedBlockParts = nil
if !cs.ProposalBlockParts.HasHeader(partsHeader) {
if !cs.ProposalBlockParts.HasHeader(blockID.PartsHeader) {
cs.ProposalBlock = nil
cs.ProposalBlockParts = types.NewPartSetFromHeader(partsHeader)
cs.ProposalBlockParts = types.NewPartSetFromHeader(blockID.PartsHeader)
}
types.FireEventUnlock(cs.evsw, cs.RoundStateEvent())
cs.signAddVote(types.VoteTypePrecommit, nil, types.PartSetHeader{})
@ -1165,7 +1181,7 @@ func (cs *ConsensusState) enterCommit(height int, commitRound int) {
cs.tryFinalizeCommit(height)
}()
hash, partsHeader, ok := cs.Votes.Precommits(commitRound).TwoThirdsMajority()
blockID, ok := cs.Votes.Precommits(commitRound).TwoThirdsMajority()
if !ok {
PanicSanity("RunActionCommit() expects +2/3 precommits")
}
@ -1173,18 +1189,18 @@ func (cs *ConsensusState) enterCommit(height int, commitRound int) {
// The Locked* fields no longer matter.
// Move them over to ProposalBlock if they match the commit hash,
// otherwise they'll be cleared in updateToState.
if cs.LockedBlock.HashesTo(hash) {
if cs.LockedBlock.HashesTo(blockID.Hash) {
cs.ProposalBlock = cs.LockedBlock
cs.ProposalBlockParts = cs.LockedBlockParts
}
// If we don't have the block being committed, set up to get it.
if !cs.ProposalBlock.HashesTo(hash) {
if !cs.ProposalBlockParts.HasHeader(partsHeader) {
if !cs.ProposalBlock.HashesTo(blockID.Hash) {
if !cs.ProposalBlockParts.HasHeader(blockID.PartsHeader) {
// We're getting the wrong block.
// Set up ProposalBlockParts and keep waiting.
cs.ProposalBlock = nil
cs.ProposalBlockParts = types.NewPartSetFromHeader(partsHeader)
cs.ProposalBlockParts = types.NewPartSetFromHeader(blockID.PartsHeader)
} else {
// We just need to keep waiting.
}
@ -1197,12 +1213,12 @@ func (cs *ConsensusState) tryFinalizeCommit(height int) {
PanicSanity(Fmt("tryFinalizeCommit() cs.Height: %v vs height: %v", cs.Height, height))
}
hash, _, ok := cs.Votes.Precommits(cs.CommitRound).TwoThirdsMajority()
if !ok || len(hash) == 0 {
blockID, ok := cs.Votes.Precommits(cs.CommitRound).TwoThirdsMajority()
if !ok || len(blockID.Hash) == 0 {
log.Warn("Attempt to finalize failed. There was no +2/3 majority, or +2/3 was for <nil>.")
return
}
if !cs.ProposalBlock.HashesTo(hash) {
if !cs.ProposalBlock.HashesTo(blockID.Hash) {
// TODO: this happens every time if we're not a validator (ugly logs)
// TODO: ^^ wait, why does it matter that we're a validator?
log.Warn("Attempt to finalize failed. We don't have the commit block.")
@ -1219,16 +1235,16 @@ func (cs *ConsensusState) finalizeCommit(height int) {
return
}
hash, header, ok := cs.Votes.Precommits(cs.CommitRound).TwoThirdsMajority()
blockID, ok := cs.Votes.Precommits(cs.CommitRound).TwoThirdsMajority()
block, blockParts := cs.ProposalBlock, cs.ProposalBlockParts
if !ok {
PanicSanity(Fmt("Cannot finalizeCommit, commit does not have two thirds majority"))
}
if !blockParts.HasHeader(header) {
if !blockParts.HasHeader(blockID.PartsHeader) {
PanicSanity(Fmt("Expected ProposalBlockParts header to be commit header"))
}
if !block.HashesTo(hash) {
if !block.HashesTo(blockID.Hash) {
PanicSanity(Fmt("Cannot finalizeCommit, ProposalBlock does not hash to commit hash"))
}
if err := cs.state.ValidateBlock(block); err != nil {
@ -1321,8 +1337,9 @@ func (cs *ConsensusState) commitStateUpdateMempool(s *sm.State, block *types.Blo
//-----------------------------------------------------------------------------
func (cs *ConsensusState) setProposal(proposal *types.Proposal) error {
func (cs *ConsensusState) defaultSetProposal(proposal *types.Proposal) error {
// Already have one
// TODO: possibly catch double proposals
if cs.Proposal != nil {
return nil
}
@ -1390,15 +1407,15 @@ func (cs *ConsensusState) addProposalBlockPart(height int, part *types.Part, ver
}
// Attempt to add the vote. if its a duplicate signature, dupeout the validator
func (cs *ConsensusState) tryAddVote(valIndex int, vote *types.Vote, peerKey string) error {
_, _, err := cs.addVote(valIndex, vote, peerKey)
func (cs *ConsensusState) tryAddVote(vote *types.Vote, peerKey string) error {
_, err := cs.addVote(vote, peerKey)
if err != nil {
// If the vote height is off, we'll just ignore it,
// But if it's a conflicting sig, broadcast evidence tx for slashing.
// If it's otherwise invalid, punish peer.
if err == ErrVoteHeightMismatch {
return err
} else if _, ok := err.(*types.ErrVoteConflictingSignature); ok {
} else if _, ok := err.(*types.ErrVoteConflictingVotes); ok {
if peerKey == "" {
log.Warn("Found conflicting vote from ourselves. Did you unsafe_reset a validator?", "height", vote.Height, "round", vote.Round, "type", vote.Type)
return err
@ -1424,7 +1441,7 @@ func (cs *ConsensusState) tryAddVote(valIndex int, vote *types.Vote, peerKey str
//-----------------------------------------------------------------------------
func (cs *ConsensusState) addVote(valIndex int, vote *types.Vote, peerKey string) (added bool, address []byte, err error) {
func (cs *ConsensusState) addVote(vote *types.Vote, peerKey string) (added bool, err error) {
log.Debug("addVote", "voteHeight", vote.Height, "voteType", vote.Type, "csHeight", cs.Height)
// A precommit for the previous height?
@ -1432,13 +1449,12 @@ func (cs *ConsensusState) addVote(valIndex int, vote *types.Vote, peerKey string
if !(cs.Step == RoundStepNewHeight && vote.Type == types.VoteTypePrecommit) {
// TODO: give the reason ..
// fmt.Errorf("tryAddVote: Wrong height, not a LastCommit straggler commit.")
return added, nil, ErrVoteHeightMismatch
return added, ErrVoteHeightMismatch
}
added, address, err = cs.LastCommit.AddByIndex(valIndex, vote)
added, err = cs.LastCommit.AddVote(vote)
if added {
log.Info(Fmt("Added to lastPrecommits: %v", cs.LastCommit.StringShort()))
types.FireEventVote(cs.evsw, types.EventDataVote{valIndex, address, vote})
types.FireEventVote(cs.evsw, types.EventDataVote{vote})
}
return
}
@ -1446,9 +1462,9 @@ func (cs *ConsensusState) addVote(valIndex int, vote *types.Vote, peerKey string
// A prevote/precommit for this height?
if vote.Height == cs.Height {
height := cs.Height
added, address, err = cs.Votes.AddByIndex(valIndex, vote, peerKey)
added, err = cs.Votes.AddVote(vote, peerKey)
if added {
types.FireEventVote(cs.evsw, types.EventDataVote{valIndex, address, vote})
types.FireEventVote(cs.evsw, types.EventDataVote{vote})
switch vote.Type {
case types.VoteTypePrevote:
@ -1460,8 +1476,8 @@ func (cs *ConsensusState) addVote(valIndex int, vote *types.Vote, peerKey string
// we'll still enterNewRound(H,vote.R) and enterPrecommit(H,vote.R) to process it
// there.
if (cs.LockedBlock != nil) && (cs.LockedRound < vote.Round) && (vote.Round <= cs.Round) {
hash, _, ok := prevotes.TwoThirdsMajority()
if ok && !cs.LockedBlock.HashesTo(hash) {
blockID, ok := prevotes.TwoThirdsMajority()
if ok && !cs.LockedBlock.HashesTo(blockID.Hash) {
log.Notice("Unlocking because of POL.", "lockedRound", cs.LockedRound, "POLRound", vote.Round)
cs.LockedRound = 0
cs.LockedBlock = nil
@ -1487,9 +1503,9 @@ func (cs *ConsensusState) addVote(valIndex int, vote *types.Vote, peerKey string
case types.VoteTypePrecommit:
precommits := cs.Votes.Precommits(vote.Round)
log.Info("Added to precommit", "vote", vote, "precommits", precommits.StringShort())
hash, _, ok := precommits.TwoThirdsMajority()
blockID, ok := precommits.TwoThirdsMajority()
if ok {
if len(hash) == 0 {
if len(blockID.Hash) == 0 {
cs.enterNewRound(height, vote.Round+1)
} else {
cs.enterNewRound(height, vote.Round)
@ -1518,12 +1534,16 @@ func (cs *ConsensusState) addVote(valIndex int, vote *types.Vote, peerKey string
}
func (cs *ConsensusState) signVote(type_ byte, hash []byte, header types.PartSetHeader) (*types.Vote, error) {
// TODO: store our index in the cs so we don't have to do this every time
addr := cs.privValidator.GetAddress()
valIndex, _ := cs.Validators.GetByAddress(addr)
vote := &types.Vote{
ValidatorAddress: addr,
ValidatorIndex: valIndex,
Height: cs.Height,
Round: cs.Round,
Type: type_,
BlockHash: hash,
BlockPartsHeader: header,
BlockID: types.BlockID{hash, header},
}
err := cs.privValidator.SignVote(cs.state.ChainID, vote)
return vote, err
@ -1531,21 +1551,18 @@ func (cs *ConsensusState) signVote(type_ byte, hash []byte, header types.PartSet
// sign the vote and publish on internalMsgQueue
func (cs *ConsensusState) signAddVote(type_ byte, hash []byte, header types.PartSetHeader) *types.Vote {
if cs.privValidator == nil || !cs.Validators.HasAddress(cs.privValidator.Address) {
if cs.privValidator == nil || !cs.Validators.HasAddress(cs.privValidator.GetAddress()) {
return nil
}
vote, err := cs.signVote(type_, hash, header)
if err == nil {
// TODO: store our index in the cs so we don't have to do this every time
valIndex, _ := cs.Validators.GetByAddress(cs.privValidator.Address)
cs.sendInternalMessage(msgInfo{&VoteMessage{valIndex, vote}, ""})
cs.sendInternalMessage(msgInfo{&VoteMessage{vote}, ""})
log.Info("Signed and pushed vote", "height", cs.Height, "round", cs.Round, "vote", vote, "error", err)
return vote
} else {
if !cs.replayMode {
log.Warn("Error signing vote", "height", cs.Height, "round", cs.Round, "vote", vote, "error", err)
}
//if !cs.replayMode {
log.Warn("Error signing vote", "height", cs.Height, "round", cs.Round, "vote", vote, "error", err)
//}
return nil
}
}


+ 144
- 111
consensus/state_test.go View File

@ -30,7 +30,7 @@ x * TestBadProposal - 2 vals, bad proposal (bad block state hash), should prevot
FullRoundSuite
x * TestFullRound1 - 1 val, full successful round
x * TestFullRoundNil - 1 val, full round of nil
x * TestFullRound2 - 2 vals, both required for fuill round
x * TestFullRound2 - 2 vals, both required for full round
LockSuite
x * TestLockNoPOL - 2 vals, 4 rounds. one val locked, precommits nil every round except first.
x * TestLockPOLRelock - 4 vals, one precommits, other 3 polka at next round, so we unlock and precomit the polka
@ -66,15 +66,15 @@ func TestProposerSelection0(t *testing.T) {
// lets commit a block and ensure proposer for the next height is correct
prop := cs1.GetRoundState().Validators.Proposer()
if !bytes.Equal(prop.Address, cs1.privValidator.Address) {
panic(Fmt("expected proposer to be validator %d. Got %X", 0, prop.Address))
if !bytes.Equal(prop.Address, cs1.privValidator.GetAddress()) {
t.Fatalf("expected proposer to be validator %d. Got %X", 0, prop.Address)
}
// wait for complete proposal
<-proposalCh
rs := cs1.GetRoundState()
signAddVoteToFromMany(types.VoteTypePrecommit, cs1, rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header(), nil, vss[1:]...)
signAddVotes(cs1, types.VoteTypePrecommit, rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header(), vss[1:]...)
// wait for new round so next validator is set
<-newRoundCh
@ -106,7 +106,7 @@ func TestProposerSelection2(t *testing.T) {
}
rs := cs1.GetRoundState()
signAddVoteToFromMany(types.VoteTypePrecommit, cs1, nil, rs.ProposalBlockParts.Header(), nil, vss[1:]...)
signAddVotes(cs1, types.VoteTypePrecommit, nil, rs.ProposalBlockParts.Header(), vss[1:]...)
<-newRoundCh // wait for the new round event each round
incrementRound(vss[1:]...)
@ -179,12 +179,14 @@ func TestEnterProposeYesPrivValidator(t *testing.T) {
func TestBadProposal(t *testing.T) {
cs1, vss := randConsensusState(2)
height, round := cs1.Height, cs1.Round
cs2 := vss[1]
vs2 := vss[1]
partSize := config.GetInt("block_part_size")
proposalCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringCompleteProposal(), 1)
voteCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringVote(), 1)
propBlock, _ := cs1.createProposalBlock() //changeProposer(t, cs1, cs2)
propBlock, _ := cs1.createProposalBlock() //changeProposer(t, cs1, vs2)
// make the second validator the proposer by incrementing round
round = round + 1
@ -197,10 +199,10 @@ func TestBadProposal(t *testing.T) {
}
stateHash[0] = byte((stateHash[0] + 1) % 255)
propBlock.AppHash = stateHash
propBlockParts := propBlock.MakePartSet()
proposal := types.NewProposal(cs2.Height, round, propBlockParts.Header(), -1)
if err := cs2.SignProposal(config.GetString("chain_id"), proposal); err != nil {
panic("failed to sign bad proposal: " + err.Error())
propBlockParts := propBlock.MakePartSet(partSize)
proposal := types.NewProposal(vs2.Height, round, propBlockParts.Header(), -1, types.BlockID{})
if err := vs2.SignProposal(config.GetString("chain_id"), proposal); err != nil {
t.Fatal("failed to sign bad proposal", err)
}
// set the proposal block
@ -217,14 +219,15 @@ func TestBadProposal(t *testing.T) {
validatePrevote(t, cs1, round, vss[0], nil)
// add bad prevote from cs2 and wait for it
signAddVoteToFrom(types.VoteTypePrevote, cs1, cs2, propBlock.Hash(), propBlock.MakePartSet().Header(), voteCh)
// add bad prevote from vs2 and wait for it
signAddVotes(cs1, types.VoteTypePrevote, propBlock.Hash(), propBlock.MakePartSet(partSize).Header(), vs2)
<-voteCh
// wait for precommit
<-voteCh
validatePrecommit(t, cs1, round, 0, vss[0], nil, nil)
signAddVoteToFrom(types.VoteTypePrecommit, cs1, cs2, propBlock.Hash(), propBlock.MakePartSet().Header(), voteCh)
signAddVotes(cs1, types.VoteTypePrecommit, propBlock.Hash(), propBlock.MakePartSet(partSize).Header(), vs2)
}
//----------------------------------------------------------------------------------------------------
@ -281,7 +284,7 @@ func TestFullRoundNil(t *testing.T) {
// where the first validator has to wait for votes from the second
func TestFullRound2(t *testing.T) {
cs1, vss := randConsensusState(2)
cs2 := vss[1]
vs2 := vss[1]
height, round := cs1.Height, cs1.Round
voteCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringVote(), 1)
@ -296,8 +299,9 @@ func TestFullRound2(t *testing.T) {
rs := cs1.GetRoundState()
propBlockHash, propPartsHeader := rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header()
// prevote arrives from cs2:
signAddVoteToFrom(types.VoteTypePrevote, cs1, cs2, propBlockHash, propPartsHeader, voteCh)
// prevote arrives from vs2:
signAddVotes(cs1, types.VoteTypePrevote, propBlockHash, propPartsHeader, vs2)
<-voteCh
<-voteCh //precommit
@ -306,8 +310,9 @@ func TestFullRound2(t *testing.T) {
// we should be stuck in limbo waiting for more precommits
// precommit arrives from cs2:
signAddVoteToFrom(types.VoteTypePrecommit, cs1, cs2, propBlockHash, propPartsHeader, voteCh)
// precommit arrives from vs2:
signAddVotes(cs1, types.VoteTypePrecommit, propBlockHash, propPartsHeader, vs2)
<-voteCh
// wait to finish commit, propose in next height
<-newBlockCh
@ -320,9 +325,11 @@ func TestFullRound2(t *testing.T) {
// two vals take turns proposing. val1 locks on first one, precommits nil on everything else
func TestLockNoPOL(t *testing.T) {
cs1, vss := randConsensusState(2)
cs2 := vss[1]
vs2 := vss[1]
height := cs1.Height
partSize := config.GetInt("block_part_size")
timeoutProposeCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringTimeoutPropose(), 1)
timeoutWaitCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringTimeoutWait(), 1)
voteCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringVote(), 1)
@ -344,8 +351,9 @@ func TestLockNoPOL(t *testing.T) {
<-voteCh // prevote
// we should now be stuck in limbo forever, waiting for more prevotes
// prevote arrives from cs2:
signAddVoteToFrom(types.VoteTypePrevote, cs1, cs2, rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header(), voteCh)
// prevote arrives from vs2:
signAddVotes(cs1, types.VoteTypePrevote, cs1.ProposalBlock.Hash(), cs1.ProposalBlockParts.Header(), vs2)
<-voteCh // prevote
<-voteCh // precommit
@ -358,7 +366,8 @@ func TestLockNoPOL(t *testing.T) {
hash := make([]byte, len(theBlockHash))
copy(hash, theBlockHash)
hash[0] = byte((hash[0] + 1) % 255)
signAddVoteToFrom(types.VoteTypePrecommit, cs1, cs2, hash, rs.ProposalBlock.MakePartSet().Header(), voteCh)
signAddVotes(cs1, types.VoteTypePrecommit, hash, rs.ProposalBlock.MakePartSet(partSize).Header(), vs2)
<-voteCh // 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
@ -372,7 +381,7 @@ func TestLockNoPOL(t *testing.T) {
Round2 (cs1, B) // B B2
*/
incrementRound(cs2)
incrementRound(vs2)
// now we're on a new round and not the proposer, so wait for timeout
re = <-timeoutProposeCh
@ -389,7 +398,8 @@ func TestLockNoPOL(t *testing.T) {
validatePrevote(t, cs1, 1, vss[0], rs.LockedBlock.Hash())
// add a conflicting prevote from the other validator
signAddVoteToFrom(types.VoteTypePrevote, cs1, cs2, hash, rs.ProposalBlock.MakePartSet().Header(), voteCh)
signAddVotes(cs1, types.VoteTypePrevote, hash, rs.ProposalBlock.MakePartSet(partSize).Header(), vs2)
<-voteCh
// now we're going to enter prevote again, but with invalid args
// and then prevote wait, which should timeout. then wait for precommit
@ -401,9 +411,10 @@ func TestLockNoPOL(t *testing.T) {
// we should precommit nil and be locked on the proposal
validatePrecommit(t, cs1, 1, 0, vss[0], nil, theBlockHash)
// add conflicting precommit from cs2
// add conflicting precommit from vs2
// NOTE: in practice we should never get to a point where there are precommits for different blocks at the same round
signAddVoteToFrom(types.VoteTypePrecommit, cs1, cs2, hash, rs.ProposalBlock.MakePartSet().Header(), voteCh)
signAddVotes(cs1, types.VoteTypePrecommit, hash, rs.ProposalBlock.MakePartSet(partSize).Header(), vs2)
<-voteCh
// (note we're entering precommit for a second time this round, but with invalid args
// then we enterPrecommitWait and timeout into NewRound
@ -412,10 +423,10 @@ func TestLockNoPOL(t *testing.T) {
<-newRoundCh
log.Notice("#### ONTO ROUND 2")
/*
Round3 (cs2, _) // B, B2
Round3 (vs2, _) // B, B2
*/
incrementRound(cs2)
incrementRound(vs2)
re = <-proposalCh
rs = re.(types.EventDataRoundState).RoundState.(*RoundState)
@ -429,33 +440,36 @@ func TestLockNoPOL(t *testing.T) {
validatePrevote(t, cs1, 2, vss[0], rs.LockedBlock.Hash())
signAddVoteToFrom(types.VoteTypePrevote, cs1, cs2, hash, rs.ProposalBlock.MakePartSet().Header(), voteCh)
signAddVotes(cs1, types.VoteTypePrevote, hash, rs.ProposalBlock.MakePartSet(partSize).Header(), vs2)
<-voteCh
<-timeoutWaitCh // prevote wait
<-voteCh // precommit
validatePrecommit(t, cs1, 2, 0, vss[0], nil, theBlockHash) // precommit nil but be locked on proposal
signAddVoteToFrom(types.VoteTypePrecommit, cs1, cs2, hash, rs.ProposalBlock.MakePartSet().Header(), voteCh) // NOTE: conflicting precommits at same height
validatePrecommit(t, cs1, 2, 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
<-voteCh
<-timeoutWaitCh
// before we time out into new round, set next proposal block
prop, propBlock := decideProposal(cs1, cs2, cs2.Height, cs2.Round+1)
prop, propBlock := decideProposal(cs1, vs2, vs2.Height, vs2.Round+1)
if prop == nil || propBlock == nil {
panic("Failed to create proposal block with cs2")
t.Fatal("Failed to create proposal block with vs2")
}
incrementRound(cs2)
incrementRound(vs2)
<-newRoundCh
log.Notice("#### ONTO ROUND 3")
/*
Round4 (cs2, C) // B C // B C
Round4 (vs2, C) // B C // B C
*/
// now we're on a new round and not the proposer
// so set the proposal block
cs1.SetProposalAndBlock(prop, propBlock, propBlock.MakePartSet(), "")
cs1.SetProposalAndBlock(prop, propBlock, propBlock.MakePartSet(partSize), "")
<-proposalCh
<-voteCh // prevote
@ -463,19 +477,24 @@ func TestLockNoPOL(t *testing.T) {
// prevote for locked block (not proposal)
validatePrevote(t, cs1, 0, vss[0], cs1.LockedBlock.Hash())
signAddVoteToFrom(types.VoteTypePrevote, cs1, cs2, propBlock.Hash(), propBlock.MakePartSet().Header(), voteCh)
signAddVotes(cs1, types.VoteTypePrevote, propBlock.Hash(), propBlock.MakePartSet(partSize).Header(), vs2)
<-voteCh
<-timeoutWaitCh
<-voteCh
validatePrecommit(t, cs1, 2, 0, vss[0], nil, theBlockHash) // precommit nil but locked on proposal
signAddVoteToFrom(types.VoteTypePrecommit, cs1, cs2, propBlock.Hash(), propBlock.MakePartSet().Header(), voteCh) // NOTE: conflicting precommits at same height
validatePrecommit(t, cs1, 2, 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
<-voteCh
}
// 4 vals, one precommits, other 3 polka at next round, so we unlock and precomit the polka
func TestLockPOLRelock(t *testing.T) {
cs1, vss := randConsensusState(4)
cs2, cs3, cs4 := vss[1], vss[2], vss[3]
vs2, vs3, vs4 := vss[1], vss[2], vss[3]
partSize := config.GetInt("block_part_size")
timeoutProposeCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringTimeoutPropose(), 1)
timeoutWaitCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringTimeoutWait(), 1)
@ -484,14 +503,14 @@ func TestLockPOLRelock(t *testing.T) {
newRoundCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringNewRound(), 1)
newBlockCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringNewBlockHeader(), 1)
log.Debug("cs2 last round", "lr", cs2.PrivValidator.LastRound)
log.Debug("vs2 last round", "lr", vs2.PrivValidator.LastRound)
// everything done from perspective of cs1
/*
Round1 (cs1, B) // B B B B// B nil B nil
eg. cs2 and cs4 didn't see the 2/3 prevotes
eg. vs2 and vs4 didn't see the 2/3 prevotes
*/
// start round and wait for propose and prevote
@ -501,26 +520,27 @@ func TestLockPOLRelock(t *testing.T) {
re := <-proposalCh
rs := re.(types.EventDataRoundState).RoundState.(*RoundState)
theBlockHash := rs.ProposalBlock.Hash()
theBlockPartsHeader := rs.ProposalBlockParts.Header()
<-voteCh // prevote
signAddVoteToFromMany(types.VoteTypePrevote, cs1, theBlockHash, theBlockPartsHeader, voteCh, cs2, cs3, cs4)
signAddVotes(cs1, types.VoteTypePrevote, cs1.ProposalBlock.Hash(), cs1.ProposalBlockParts.Header(), vs2, vs3, vs4)
_, _, _ = <-voteCh, <-voteCh, <-voteCh // prevotes
<-voteCh // our precommit
// the proposed block should now be locked and our precommit added
validatePrecommit(t, cs1, 0, 0, vss[0], theBlockHash, theBlockHash)
// add precommits from the rest
signAddVoteToFromMany(types.VoteTypePrecommit, cs1, nil, types.PartSetHeader{}, voteCh, cs2, cs4)
signAddVoteToFrom(types.VoteTypePrecommit, cs1, cs3, theBlockHash, theBlockPartsHeader, voteCh)
signAddVotes(cs1, types.VoteTypePrecommit, nil, types.PartSetHeader{}, vs2, vs4)
signAddVotes(cs1, types.VoteTypePrecommit, cs1.ProposalBlock.Hash(), cs1.ProposalBlockParts.Header(), vs3)
_, _, _ = <-voteCh, <-voteCh, <-voteCh // precommits
// before we timeout to the new round set the new proposal
prop, propBlock := decideProposal(cs1, cs2, cs2.Height, cs2.Round+1)
propBlockParts := propBlock.MakePartSet()
prop, propBlock := decideProposal(cs1, vs2, vs2.Height, vs2.Round+1)
propBlockParts := propBlock.MakePartSet(partSize)
propBlockHash := propBlock.Hash()
incrementRound(cs2, cs3, cs4)
incrementRound(vs2, vs3, vs4)
// timeout to new round
<-timeoutWaitCh
@ -532,7 +552,7 @@ func TestLockPOLRelock(t *testing.T) {
log.Notice("### ONTO ROUND 1")
/*
Round2 (cs2, C) // B C C C // C C C _)
Round2 (vs2, C) // B C C C // C C C _)
cs1 changes lock!
*/
@ -550,7 +570,8 @@ func TestLockPOLRelock(t *testing.T) {
validatePrevote(t, cs1, 0, vss[0], theBlockHash)
// now lets add prevotes from everyone else for the new block
signAddVoteToFromMany(types.VoteTypePrevote, cs1, propBlockHash, propBlockParts.Header(), voteCh, cs2, cs3, cs4)
signAddVotes(cs1, types.VoteTypePrevote, propBlockHash, propBlockParts.Header(), vs2, vs3, vs4)
_, _, _ = <-voteCh, <-voteCh, <-voteCh // prevotes
// now either we go to PrevoteWait or Precommit
select {
@ -564,7 +585,8 @@ func TestLockPOLRelock(t *testing.T) {
// we should have unlocked and locked on the new block
validatePrecommit(t, cs1, 1, 1, vss[0], propBlockHash, propBlockHash)
signAddVoteToFromMany(types.VoteTypePrecommit, cs1, propBlockHash, propBlockParts.Header(), voteCh, cs2, cs3)
signAddVotes(cs1, types.VoteTypePrecommit, propBlockHash, propBlockParts.Header(), vs2, vs3)
_, _ = <-voteCh, <-voteCh
be := <-newBlockCh
b := be.(types.EventDataNewBlockHeader)
@ -582,14 +604,16 @@ func TestLockPOLRelock(t *testing.T) {
// 4 vals, one precommits, other 3 polka at next round, so we unlock and precomit the polka
func TestLockPOLUnlock(t *testing.T) {
cs1, vss := randConsensusState(4)
cs2, cs3, cs4 := vss[1], vss[2], vss[3]
vs2, vs3, vs4 := vss[1], vss[2], vss[3]
partSize := config.GetInt("block_part_size")
proposalCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringCompleteProposal(), 1)
timeoutProposeCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringTimeoutPropose(), 1)
timeoutWaitCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringTimeoutWait(), 1)
newRoundCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringNewRound(), 1)
unlockCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringUnlock(), 1)
voteCh := subscribeToVoter(cs1, cs1.privValidator.Address)
voteCh := subscribeToVoter(cs1, cs1.privValidator.GetAddress())
// everything done from perspective of cs1
@ -608,7 +632,7 @@ func TestLockPOLUnlock(t *testing.T) {
<-voteCh // prevote
signAddVoteToFromMany(types.VoteTypePrevote, cs1, rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header(), nil, cs2, cs3, cs4)
signAddVotes(cs1, types.VoteTypePrevote, cs1.ProposalBlock.Hash(), cs1.ProposalBlockParts.Header(), vs2, vs3, vs4)
<-voteCh //precommit
@ -618,14 +642,14 @@ func TestLockPOLUnlock(t *testing.T) {
rs = cs1.GetRoundState()
// add precommits from the rest
signAddVoteToFromMany(types.VoteTypePrecommit, cs1, nil, types.PartSetHeader{}, nil, cs2, cs4)
signAddVoteToFrom(types.VoteTypePrecommit, cs1, cs3, rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header(), nil)
signAddVotes(cs1, types.VoteTypePrecommit, nil, types.PartSetHeader{}, vs2, vs4)
signAddVotes(cs1, types.VoteTypePrecommit, cs1.ProposalBlock.Hash(), cs1.ProposalBlockParts.Header(), vs3)
// before we time out into new round, set next proposal block
prop, propBlock := decideProposal(cs1, cs2, cs2.Height, cs2.Round+1)
propBlockParts := propBlock.MakePartSet()
prop, propBlock := decideProposal(cs1, vs2, vs2.Height, vs2.Round+1)
propBlockParts := propBlock.MakePartSet(partSize)
incrementRound(cs2, cs3, cs4)
incrementRound(vs2, vs3, vs4)
// timeout to new round
re = <-timeoutWaitCh
@ -638,7 +662,7 @@ func TestLockPOLUnlock(t *testing.T) {
<-newRoundCh
log.Notice("#### ONTO ROUND 1")
/*
Round2 (cs2, C) // B nil nil nil // nil nil nil _
Round2 (vs2, C) // B nil nil nil // nil nil nil _
cs1 unlocks!
*/
@ -655,7 +679,7 @@ func TestLockPOLUnlock(t *testing.T) {
<-voteCh
validatePrevote(t, cs1, 0, vss[0], lockedBlockHash)
// now lets add prevotes from everyone else for nil (a polka!)
signAddVoteToFromMany(types.VoteTypePrevote, cs1, nil, types.PartSetHeader{}, nil, cs2, cs3, cs4)
signAddVotes(cs1, types.VoteTypePrevote, nil, types.PartSetHeader{}, vs2, vs3, vs4)
// the polka makes us unlock and precommit nil
<-unlockCh
@ -665,7 +689,7 @@ func TestLockPOLUnlock(t *testing.T) {
// NOTE: since we don't relock on nil, the lock round is 0
validatePrecommit(t, cs1, 1, 0, vss[0], nil, nil)
signAddVoteToFromMany(types.VoteTypePrecommit, cs1, nil, types.PartSetHeader{}, nil, cs2, cs3)
signAddVotes(cs1, types.VoteTypePrecommit, nil, types.PartSetHeader{}, vs2, vs3)
<-newRoundCh
}
@ -675,13 +699,15 @@ func TestLockPOLUnlock(t *testing.T) {
// then we see the polka from round 1 but shouldn't unlock
func TestLockPOLSafety1(t *testing.T) {
cs1, vss := randConsensusState(4)
cs2, cs3, cs4 := vss[1], vss[2], vss[3]
vs2, vs3, vs4 := vss[1], vss[2], vss[3]
partSize := config.GetInt("block_part_size")
proposalCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringCompleteProposal(), 1)
timeoutProposeCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringTimeoutPropose(), 1)
timeoutWaitCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringTimeoutWait(), 1)
newRoundCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringNewRound(), 1)
voteCh := subscribeToVoter(cs1, cs1.privValidator.Address)
voteCh := subscribeToVoter(cs1, cs1.privValidator.GetAddress())
// start round and wait for propose and prevote
startTestRound(cs1, cs1.Height, 0)
@ -695,7 +721,7 @@ func TestLockPOLSafety1(t *testing.T) {
validatePrevote(t, cs1, 0, vss[0], propBlock.Hash())
// the others sign a polka but we don't see it
prevotes := signVoteMany(types.VoteTypePrevote, propBlock.Hash(), propBlock.MakePartSet().Header(), cs2, cs3, cs4)
prevotes := signVotes(types.VoteTypePrevote, propBlock.Hash(), propBlock.MakePartSet(partSize).Header(), vs2, vs3, vs4)
// before we time out into new round, set next proposer
// and next proposal block
@ -709,13 +735,13 @@ func TestLockPOLSafety1(t *testing.T) {
log.Warn("old prop", "hash", fmt.Sprintf("%X", propBlock.Hash()))
// we do see them precommit nil
signAddVoteToFromMany(types.VoteTypePrecommit, cs1, nil, types.PartSetHeader{}, nil, cs2, cs3, cs4)
signAddVotes(cs1, types.VoteTypePrecommit, nil, types.PartSetHeader{}, vs2, vs3, vs4)
prop, propBlock := decideProposal(cs1, cs2, cs2.Height, cs2.Round+1)
prop, propBlock := decideProposal(cs1, vs2, vs2.Height, vs2.Round+1)
propBlockHash := propBlock.Hash()
propBlockParts := propBlock.MakePartSet()
propBlockParts := propBlock.MakePartSet(partSize)
incrementRound(cs2, cs3, cs4)
incrementRound(vs2, vs3, vs4)
//XXX: this isnt gauranteed to get there before the timeoutPropose ...
cs1.SetProposalAndBlock(prop, propBlock, propBlockParts, "some peer")
@ -746,18 +772,18 @@ func TestLockPOLSafety1(t *testing.T) {
validatePrevote(t, cs1, 1, vss[0], propBlockHash)
// now we see the others prevote for it, so we should lock on it
signAddVoteToFromMany(types.VoteTypePrevote, cs1, propBlockHash, propBlockParts.Header(), nil, cs2, cs3, cs4)
signAddVotes(cs1, types.VoteTypePrevote, propBlockHash, propBlockParts.Header(), vs2, vs3, vs4)
<-voteCh // precommit
// we should have precommitted
validatePrecommit(t, cs1, 1, 1, vss[0], propBlockHash, propBlockHash)
signAddVoteToFromMany(types.VoteTypePrecommit, cs1, nil, types.PartSetHeader{}, nil, cs2, cs3)
signAddVotes(cs1, types.VoteTypePrecommit, nil, types.PartSetHeader{}, vs2, vs3)
<-timeoutWaitCh
incrementRound(cs2, cs3, cs4)
incrementRound(vs2, vs3, vs4)
<-newRoundCh
@ -778,7 +804,7 @@ func TestLockPOLSafety1(t *testing.T) {
newStepCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringNewRoundStep(), 1)
// add prevotes from the earlier round
addVoteToFromMany(cs1, prevotes, cs2, cs3, cs4)
addVotes(cs1, prevotes...)
log.Warn("Done adding prevotes!")
@ -794,30 +820,33 @@ func TestLockPOLSafety1(t *testing.T) {
// dont see P0, lock on P1 at R1, dont unlock using P0 at R2
func TestLockPOLSafety2(t *testing.T) {
cs1, vss := randConsensusState(4)
cs2, cs3, cs4 := vss[1], vss[2], vss[3]
vs2, vs3, vs4 := vss[1], vss[2], vss[3]
partSize := config.GetInt("block_part_size")
proposalCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringCompleteProposal(), 1)
timeoutProposeCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringTimeoutPropose(), 1)
timeoutWaitCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringTimeoutWait(), 1)
newRoundCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringNewRound(), 1)
unlockCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringUnlock(), 1)
voteCh := subscribeToVoter(cs1, cs1.privValidator.Address)
voteCh := subscribeToVoter(cs1, cs1.privValidator.GetAddress())
// the block for R0: gets polkad but we miss it
// (even though we signed it, shhh)
_, propBlock0 := decideProposal(cs1, vss[0], cs1.Height, cs1.Round)
propBlockHash0 := propBlock0.Hash()
propBlockParts0 := propBlock0.MakePartSet()
propBlockParts0 := propBlock0.MakePartSet(partSize)
// the others sign a polka but we don't see it
prevotes := signVoteMany(types.VoteTypePrevote, propBlockHash0, propBlockParts0.Header(), cs2, cs3, cs4)
prevotes := signVotes(types.VoteTypePrevote, propBlockHash0, propBlockParts0.Header(), vs2, vs3, vs4)
// the block for round 1
prop1, propBlock1 := decideProposal(cs1, cs2, cs2.Height, cs2.Round+1)
prop1, propBlock1 := decideProposal(cs1, vs2, vs2.Height, vs2.Round+1)
propBlockHash1 := propBlock1.Hash()
propBlockParts1 := propBlock1.MakePartSet()
propBlockParts1 := propBlock1.MakePartSet(partSize)
propBlockID1 := types.BlockID{propBlockHash1, propBlockParts1.Header()}
incrementRound(cs2, cs3, cs4)
incrementRound(vs2, vs3, vs4)
cs1.updateRoundStep(0, RoundStepPrecommitWait)
@ -832,28 +861,30 @@ func TestLockPOLSafety2(t *testing.T) {
<-voteCh // prevote
signAddVoteToFromMany(types.VoteTypePrevote, cs1, propBlockHash1, propBlockParts1.Header(), nil, cs2, cs3, cs4)
signAddVotes(cs1, types.VoteTypePrevote, propBlockHash1, propBlockParts1.Header(), vs2, vs3, vs4)
<-voteCh // precommit
// the proposed block should now be locked and our precommit added
validatePrecommit(t, cs1, 1, 1, vss[0], propBlockHash1, propBlockHash1)
// add precommits from the rest
signAddVoteToFromMany(types.VoteTypePrecommit, cs1, nil, types.PartSetHeader{}, nil, cs2, cs4)
signAddVoteToFrom(types.VoteTypePrecommit, cs1, cs3, propBlockHash1, propBlockParts1.Header(), nil)
signAddVotes(cs1, types.VoteTypePrecommit, nil, types.PartSetHeader{}, vs2, vs4)
signAddVotes(cs1, types.VoteTypePrecommit, propBlockHash1, propBlockParts1.Header(), vs3)
incrementRound(cs2, cs3, cs4)
incrementRound(vs2, vs3, vs4)
// timeout of precommit wait to new round
<-timeoutWaitCh
// in round 2 we see the polkad block from round 0
newProp := types.NewProposal(height, 2, propBlockParts0.Header(), 0)
if err := cs3.SignProposal(config.GetString("chain_id"), newProp); err != nil {
panic(err)
newProp := types.NewProposal(height, 2, propBlockParts0.Header(), 0, propBlockID1)
if err := vs3.SignProposal(config.GetString("chain_id"), newProp); err != nil {
t.Fatal(err)
}
cs1.SetProposalAndBlock(newProp, propBlock0, propBlockParts0, "some peer")
addVoteToFromMany(cs1, prevotes, cs2, cs3, cs4) // add the pol votes
// Add the pol votes
addVotes(cs1, prevotes...)
<-newRoundCh
log.Notice("### ONTO Round 2")
@ -884,13 +915,13 @@ func TestLockPOLSafety2(t *testing.T) {
/*
func TestSlashingPrevotes(t *testing.T) {
cs1, vss := randConsensusState(2)
cs2 := vss[1]
vs2 := vss[1]
proposalCh := subscribeToEvent(cs1.evsw,"tester",types.EventStringCompleteProposal() , 1)
timeoutWaitCh := subscribeToEvent(cs1.evsw,"tester",types.EventStringTimeoutWait() , 1)
newRoundCh := subscribeToEvent(cs1.evsw,"tester",types.EventStringNewRound() , 1)
voteCh := subscribeToVoter(cs1, cs1.privValidator.Address)
voteCh := subscribeToVoter(cs1, cs1.privValidator.GetAddress())
// start round and wait for propose and prevote
startTestRound(cs1, cs1.Height, 0)
@ -904,7 +935,7 @@ func TestSlashingPrevotes(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
signAddVoteToFrom(types.VoteTypePrevote, cs1, cs2, hash, rs.ProposalBlockParts.Header(), nil)
signAddVotes(cs1, types.VoteTypePrevote, hash, rs.ProposalBlockParts.Header(), vs2)
<-timeoutWaitCh
@ -912,20 +943,20 @@ func TestSlashingPrevotes(t *testing.T) {
// away and ignore more prevotes (and thus fail to slash!)
// add the conflicting vote
signAddVoteToFrom(types.VoteTypePrevote, cs1, cs2, rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header(),nil)
signAddVotes(cs1, types.VoteTypePrevote, rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header(), vs2)
// XXX: Check for existence of Dupeout info
}
func TestSlashingPrecommits(t *testing.T) {
cs1, vss := randConsensusState(2)
cs2 := vss[1]
vs2 := vss[1]
proposalCh := subscribeToEvent(cs1.evsw,"tester",types.EventStringCompleteProposal() , 1)
timeoutWaitCh := subscribeToEvent(cs1.evsw,"tester",types.EventStringTimeoutWait() , 1)
newRoundCh := subscribeToEvent(cs1.evsw,"tester",types.EventStringNewRound() , 1)
voteCh := subscribeToVoter(cs1, cs1.privValidator.Address)
voteCh := subscribeToVoter(cs1, cs1.privValidator.GetAddress())
// start round and wait for propose and prevote
startTestRound(cs1, cs1.Height, 0)
@ -933,8 +964,8 @@ func TestSlashingPrecommits(t *testing.T) {
re := <-proposalCh
<-voteCh // prevote
// add prevote from cs2
signAddVoteToFrom(types.VoteTypePrevote, cs1, cs2, rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header(), nil)
// add prevote from vs2
signAddVotes(cs1, types.VoteTypePrevote, rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header(), vs2)
<-voteCh // precommit
@ -942,13 +973,13 @@ func TestSlashingPrecommits(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
signAddVoteToFrom(types.VoteTypePrecommit, cs1, cs2, hash, rs.ProposalBlockParts.Header(),nil)
signAddVotes(cs1, types.VoteTypePrecommit, 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 cs2
signAddVoteToFrom(types.VoteTypePrecommit, cs1, cs2, rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header(),nil)
// add precommit from vs2
signAddVotes(cs1, types.VoteTypePrecommit, rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header(), vs2)
// XXX: Check for existence of Dupeout info
}
@ -964,13 +995,15 @@ func TestSlashingPrecommits(t *testing.T) {
// we receive a final precommit after going into next round, but others might have gone to commit already!
func TestHalt1(t *testing.T) {
cs1, vss := randConsensusState(4)
cs2, cs3, cs4 := vss[1], vss[2], vss[3]
vs2, vs3, vs4 := vss[1], vss[2], vss[3]
partSize := config.GetInt("block_part_size")
proposalCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringCompleteProposal(), 1)
timeoutWaitCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringTimeoutWait(), 1)
newRoundCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringNewRound(), 1)
newBlockCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringNewBlock(), 1)
voteCh := subscribeToVoter(cs1, cs1.privValidator.Address)
voteCh := subscribeToVoter(cs1, cs1.privValidator.GetAddress())
// start round and wait for propose and prevote
startTestRound(cs1, cs1.Height, 0)
@ -978,23 +1011,23 @@ func TestHalt1(t *testing.T) {
re := <-proposalCh
rs := re.(types.EventDataRoundState).RoundState.(*RoundState)
propBlock := rs.ProposalBlock
propBlockParts := propBlock.MakePartSet()
propBlockParts := propBlock.MakePartSet(partSize)
<-voteCh // prevote
signAddVoteToFromMany(types.VoteTypePrevote, cs1, propBlock.Hash(), propBlockParts.Header(), nil, cs3, cs4)
signAddVotes(cs1, types.VoteTypePrevote, propBlock.Hash(), propBlockParts.Header(), vs3, vs4)
<-voteCh // precommit
// the proposed block should now be locked and our precommit added
validatePrecommit(t, cs1, 0, 0, vss[0], propBlock.Hash(), propBlock.Hash())
// add precommits from the rest
signAddVoteToFrom(types.VoteTypePrecommit, cs1, cs2, nil, types.PartSetHeader{}, nil) // didnt receive proposal
signAddVoteToFrom(types.VoteTypePrecommit, cs1, cs3, propBlock.Hash(), propBlockParts.Header(), nil)
// we receive this later, but cs3 might receive it earlier and with ours will go to commit!
precommit4 := signVote(cs4, types.VoteTypePrecommit, propBlock.Hash(), propBlockParts.Header())
signAddVotes(cs1, types.VoteTypePrecommit, nil, types.PartSetHeader{}, vs2) // didnt receive proposal
signAddVotes(cs1, types.VoteTypePrecommit, 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())
incrementRound(cs2, cs3, cs4)
incrementRound(vs2, vs3, vs4)
// timeout to new round
<-timeoutWaitCh
@ -1012,7 +1045,7 @@ func TestHalt1(t *testing.T) {
validatePrevote(t, cs1, 0, vss[0], rs.LockedBlock.Hash())
// now we receive the precommit from the previous round
addVoteToFrom(cs1, cs4, precommit4)
addVotes(cs1, precommit4)
// receiving that precommit should take us straight to commit
<-newBlockCh


+ 12
- 1
consensus/test_data/README.md View File

@ -1,7 +1,18 @@
# Generating test data
TODO: automate this process.
The easiest way to generate this data is to copy `~/.tendermint_test/somedir/*` to `~/.tendermint`
and to run a local node.
and to run a local node. Note the tests expect a wal for block 1.
For `empty_block.cswal`, run the node and don't send any transactions.
For `small_block1.cswal` and `small_block2.cswal`,
use the `scripts/txs/random.sh 1000 36657` to start sending transactions before starting the node.
`small_block1.cswal` uses the default large block part size, so the block should have one big part.
For `small_block2.cswal`, set `block_part_size = 512` in the config.toml.
Make sure to adjust the stepChanges in the testCases if the number of messages changes
If you need to change the signatures, you can use a script as follows:
The privBytes comes from `config/tendermint_test/...`:


+ 9
- 8
consensus/test_data/empty_block.cswal View File

@ -1,9 +1,10 @@
#HEIGHT: 1
{"time":"2016-04-03T11:23:54.387Z","msg":[3,{"duration":972835254,"height":1,"round":0,"step":1}]}
{"time":"2016-04-03T11:23:54.388Z","msg":[1,{"height":1,"round":0,"step":"RoundStepPropose"}]}
{"time":"2016-04-03T11:23:54.388Z","msg":[2,{"msg":[17,{"Proposal":{"height":1,"round":0,"block_parts_header":{"total":1,"hash":"3BA1E90CB868DA6B4FD7F3589826EC461E9EB4EF"},"pol_round":-1,"signature":"3A2ECD5023B21EC144EC16CFF1B992A4321317B83EEDD8969FDFEA6EB7BF4389F38DDA3E7BB109D63A07491C16277A197B241CF1F05F5E485C59882ECACD9E07"}}],"peer_key":""}]}
{"time":"2016-04-03T11:23:54.389Z","msg":[2,{"msg":[19,{"Height":1,"Round":0,"Part":{"index":0,"bytes":"0101010F74656E6465726D696E745F7465737401011441D59F4B718AC00000000000000114C4B01D3810579550997AC5641E759E20D99B51C10001000100","proof":{"aunts":[]}}}],"peer_key":""}]}
{"time":"2016-04-03T11:23:54.390Z","msg":[1,{"height":1,"round":0,"step":"RoundStepPrevote"}]}
{"time":"2016-04-03T11:23:54.390Z","msg":[2,{"msg":[20,{"ValidatorIndex":0,"Vote":{"height":1,"round":0,"type":1,"block_hash":"4291966B8A9DFBA00AEC7C700F2718E61DF4331D","block_parts_header":{"total":1,"hash":"3BA1E90CB868DA6B4FD7F3589826EC461E9EB4EF"},"signature":"47D2A75A4E2F15DB1F0D1B656AC0637AF9AADDFEB6A156874F6553C73895E5D5DC948DBAEF15E61276C5342D0E638DFCB77C971CD282096EA8735A564A90F008"}}],"peer_key":""}]}
{"time":"2016-04-03T11:23:54.392Z","msg":[1,{"height":1,"round":0,"step":"RoundStepPrecommit"}]}
{"time":"2016-04-03T11:23:54.392Z","msg":[2,{"msg":[20,{"ValidatorIndex":0,"Vote":{"height":1,"round":0,"type":2,"block_hash":"4291966B8A9DFBA00AEC7C700F2718E61DF4331D","block_parts_header":{"total":1,"hash":"3BA1E90CB868DA6B4FD7F3589826EC461E9EB4EF"},"signature":"39147DA595F08B73CF8C899967C8403B5872FD9042FFA4E239159E0B6C5D9665C9CA81D766EACA2AE658872F94C2FCD1E34BF51859CD5B274DA8512BACE4B50D"}}],"peer_key":""}]}
{"time":"2016-11-16T03:26:47.819Z","msg":[3,{"duration":966969466,"height":1,"round":0,"step":1}]}
{"time":"2016-11-16T03:26:47.822Z","msg":[1,{"height":1,"round":0,"step":"RoundStepPropose"}]}
{"time":"2016-11-16T03:26:47.822Z","msg":[2,{"msg":[17,{"Proposal":{"height":1,"round":0,"block_parts_header":{"total":1,"hash":"2AF632C1AFEE1FC06B297A5E5D45171FE1C79B24"},"pol_round":-1,"pol_block_id":{"hash":"","parts":{"total":0,"hash":""}},"signature":"C5C9290F723CFF1E9EB1D7A493FF0CAAEE2E3F1865EA4014BEEECC4682CDEF24C73C07A725A273B20340A1F49C4E055CAC4B349281727096235E3C7F72DA3C00"}}],"peer_key":""}]}
{"time":"2016-11-16T03:26:47.823Z","msg":[2,{"msg":[19,{"Height":1,"Round":0,"Part":{"index":0,"bytes":"0101010F74656E6465726D696E745F7465737401011487695300231B000000000000000114C4B01D3810579550997AC5641E759E20D99B51C10001000100000000","proof":{"aunts":[]}}}],"peer_key":""}]}
{"time":"2016-11-16T03:26:47.825Z","msg":[1,{"height":1,"round":0,"step":"RoundStepPrevote"}]}
{"time":"2016-11-16T03:26:47.825Z","msg":[2,{"msg":[20,{"Vote":{"validator_address":"D028C9981F7A87F3093672BF0D5B0E2A1B3ED456","validator_index":0,"height":1,"round":0,"type":1,"block_id":{"hash":"B6CD5337C4585262B851F75F0532EA4E5B12D92C","parts":{"total":1,"hash":"2AF632C1AFEE1FC06B297A5E5D45171FE1C79B24"}},"signature":"7D52D42FBEA806740547AEFBCA5BF30AB758C320307E7C48C39A9DC3C027EE87F10808B532F7EF168BC170649A8330C194FE9E45F84C0F74DAC508D380CCA808"}}],"peer_key":""}]}
{"time":"2016-11-16T03:26:47.825Z","msg":[1,{"height":1,"round":0,"step":"RoundStepPrecommit"}]}
{"time":"2016-11-16T03:26:47.825Z","msg":[2,{"msg":[20,{"Vote":{"validator_address":"D028C9981F7A87F3093672BF0D5B0E2A1B3ED456","validator_index":0,"height":1,"round":0,"type":2,"block_id":{"hash":"B6CD5337C4585262B851F75F0532EA4E5B12D92C","parts":{"total":1,"hash":"2AF632C1AFEE1FC06B297A5E5D45171FE1C79B24"}},"signature":"450DFB9AA3CDE45B5081A95C488DB7785EE0A6875898662ADDB289FB0A3D7BA5C69E6369272A5F285EBE1B87A4D55C23F1D45EA924E3562FE0C569E44C499504"}}],"peer_key":""}]}
{"time":"2016-11-16T03:26:47.826Z","msg":[1,{"height":1,"round":0,"step":"RoundStepCommit"}]}

+ 9
- 8
consensus/test_data/small_block1.cswal View File

@ -1,9 +1,10 @@
#HEIGHT: 1
{"time":"2016-10-11T15:29:08.113Z","msg":[3,{"duration":0,"height":1,"round":0,"step":1}]}
{"time":"2016-10-11T15:29:08.115Z","msg":[1,{"height":1,"round":0,"step":"RoundStepPropose"}]}
{"time":"2016-10-11T15:29:08.115Z","msg":[2,{"msg":[17,{"Proposal":{"height":1,"round":0,"block_parts_header":{"total":1,"hash":"A2C0B5D384DFF2692FF679D00CEAE93A55DCDD1A"},"pol_round":-1,"signature":"116961B715FB54C09983209F7F3BFD95C7DA2E0D7AB9CFE9F0750F2402E2AEB715CFD55FB2C5DB88F485391D426A48705E0474AB94328463290D086D88BAD704"}}],"peer_key":""}]}
{"time":"2016-10-11T15:29:08.116Z","msg":[2,{"msg":[19,{"Height":1,"Round":0,"Part":{"index":0,"bytes":"0101010F74656E6465726D696E745F746573740101147C83D983CBE6400185000000000114CA4CC7A87B85A9FB7DBFEF8A342B66DF2B03CFB30114C4B01D3810579550997AC5641E759E20D99B51C100010185010F616263643234353D64636261323435010F616263643234363D64636261323436010F616263643234373D64636261323437010F616263643234383D64636261323438010F616263643234393D64636261323439010F616263643235303D64636261323530010F616263643235313D64636261323531010F616263643235323D64636261323532010F616263643235333D64636261323533010F616263643235343D64636261323534010F616263643235353D64636261323535010F616263643235363D64636261323536010F616263643235373D64636261323537010F616263643235383D64636261323538010F616263643235393D64636261323539010F616263643236303D64636261323630010F616263643236313D64636261323631010F616263643236323D64636261323632010F616263643236333D64636261323633010F616263643236343D64636261323634010F616263643236353D64636261323635010F616263643236363D64636261323636010F616263643236373D64636261323637010F616263643236383D64636261323638010F616263643236393D64636261323639010F616263643237303D64636261323730010F616263643237313D64636261323731010F616263643237323D64636261323732010F616263643237333D64636261323733010F616263643237343D64636261323734010F616263643237353D64636261323735010F616263643237363D64636261323736010F616263643237373D64636261323737010F616263643237383D64636261323738010F616263643237393D64636261323739010F616263643238303D64636261323830010F616263643238313D64636261323831010F616263643238323D64636261323832010F616263643238333D64636261323833010F616263643238343D64636261323834010F616263643238353D64636261323835010F616263643238363D64636261323836010F616263643238373D64636261323837010F616263643238383D64636261323838010F616263643238393D64636261323839010F616263643239303D64636261323930010F616263643239313D64636261323931010F616263643239323D64636261323932010F616263643239333D64636261323933010F616263643239343D64636261323934010F616263643239353D64636261323935010F616263643239363D64636261323936010F616263643239373D64636261323937010F616263643239383D64636261323938010F616263643239393D64636261323939010F616263643330303D64636261333030010F616263643330313D64636261333031010F616263643330323D64636261333032010F616263643330333D64636261333033010F616263643330343D64636261333034010F616263643330353D64636261333035010F616263643330363D64636261333036010F616263643330373D64636261333037010F616263643330383D64636261333038010F616263643330393D64636261333039010F616263643331303D64636261333130010F616263643331313D64636261333131010F616263643331323D64636261333132010F616263643331333D64636261333133010F616263643331343D64636261333134010F616263643331353D64636261333135010F616263643331363D64636261333136010F616263643331373D64636261333137010F616263643331383D64636261333138010F616263643331393D64636261333139010F616263643332303D64636261333230010F616263643332313D64636261333231010F616263643332323D64636261333232010F616263643332333D64636261333233010F616263643332343D64636261333234010F616263643332353D64636261333235010F616263643332363D64636261333236010F616263643332373D64636261333237010F616263643332383D64636261333238010F616263643332393D64636261333239010F616263643333303D64636261333330010F616263643333313D64636261333331010F616263643333323D64636261333332010F616263643333333D64636261333333010F616263643333343D64636261333334010F616263643333353D64636261333335010F616263643333363D64636261333336010F616263643333373D64636261333337010F616263643333383D64636261333338010F616263643333393D64636261333339010F616263643334303D64636261333430010F616263643334313D64636261333431010F616263643334323D64636261333432010F616263643334333D64636261333433010F616263643334343D64636261333434010F616263643334353D64636261333435010F616263643334363D64636261333436010F616263643334373D64636261333437010F616263643334383D64636261333438010F616263643334393D64636261333439010F616263643335303D64636261333530010F616263643335313D64636261333531010F616263643335323D64636261333532010F616263643335333D64636261333533010F616263643335343D64636261333534010F616263643335353D64636261333535010F616263643335363D64636261333536010F616263643335373D64636261333537010F616263643335383D64636261333538010F616263643335393D64636261333539010F616263643336303D64636261333630010F616263643336313D64636261333631010F616263643336323D64636261333632010F616263643336333D64636261333633010F616263643336343D64636261333634010F616263643336353D64636261333635010F616263643336363D64636261333636010F616263643336373D64636261333637010F616263643336383D64636261333638010F616263643336393D64636261333639010F616263643337303D64636261333730010F616263643337313D64636261333731010F616263643337323D64636261333732010F616263643337333D64636261333733010F616263643337343D64636261333734010F616263643337353D64636261333735010F616263643337363D64636261333736010F616263643337373D646362613337370100","proof":{"aunts":[]}}}],"peer_key":""}]}
{"time":"2016-10-11T15:29:08.117Z","msg":[1,{"height":1,"round":0,"step":"RoundStepPrevote"}]}
{"time":"2016-10-11T15:29:08.117Z","msg":[2,{"msg":[20,{"ValidatorIndex":0,"Vote":{"height":1,"round":0,"type":1,"block_hash":"FB2F51D0C6D25AD8D4ED9C33DF145E358D414A79","block_parts_header":{"total":1,"hash":"A2C0B5D384DFF2692FF679D00CEAE93A55DCDD1A"},"signature":"9BA7F5DEF2CE51CDF078DE42E3BB74D6DB6BC84767F212A88D34B3393E5915A4DC0E6C478E1C955E099617800722582E4D90AB1AC293EE5C19BC1FCC04C3CA05"}}],"peer_key":""}]}
{"time":"2016-10-11T15:29:08.118Z","msg":[1,{"height":1,"round":0,"step":"RoundStepPrecommit"}]}
{"time":"2016-10-11T15:29:08.118Z","msg":[2,{"msg":[20,{"ValidatorIndex":0,"Vote":{"height":1,"round":0,"type":2,"block_hash":"FB2F51D0C6D25AD8D4ED9C33DF145E358D414A79","block_parts_header":{"total":1,"hash":"A2C0B5D384DFF2692FF679D00CEAE93A55DCDD1A"},"signature":"9DA197CC1D7D0463FF211FB55EA12B3B0647B319E0011308C7AC3FB36E66688B4AC92EA51BD7B055814F9E4E6AB97B1AD0891EDAC42B47877100770FF467BF0A"}}],"peer_key":""}]}
{"time":"2016-11-16T05:39:30.505Z","msg":[3,{"duration":895582278,"height":1,"round":0,"step":1}]}
{"time":"2016-11-16T05:39:30.506Z","msg":[1,{"height":1,"round":0,"step":"RoundStepPropose"}]}
{"time":"2016-11-16T05:39:30.506Z","msg":[2,{"msg":[17,{"Proposal":{"height":1,"round":0,"block_parts_header":{"total":1,"hash":"F653FD4F72369E9B8097C8F20EFBCF4689305C66"},"pol_round":-1,"pol_block_id":{"hash":"","parts":{"total":0,"hash":""}},"signature":"BD598FE65B16B0A8B5DC76A8BF239C7BBB0797799811048FD8178419CF0BAC49CD9B2D9E10C6A5179E837E698BF22F089B722500076C14AC628159AD24C66C05"}}],"peer_key":""}]}
{"time":"2016-11-16T05:39:30.506Z","msg":[2,{"msg":[19,{"Height":1,"Round":0,"Part":{"index":0,"bytes":"0101010F74656E6465726D696E745F74657374010114877090F525E44001870000000001143717D3C8D4F27BDF7BEF4D79CD32D275E420B1D90114C4B01D3810579550997AC5641E759E20D99B51C100010187010F616263643337393D64636261333739010F616263643338303D64636261333830010F616263643338313D64636261333831010F616263643338323D64636261333832010F616263643338333D64636261333833010F616263643338343D64636261333834010F616263643338353D64636261333835010F616263643338363D64636261333836010F616263643338373D64636261333837010F616263643338383D64636261333838010F616263643338393D64636261333839010F616263643339303D64636261333930010F616263643339313D64636261333931010F616263643339323D64636261333932010F616263643339333D64636261333933010F616263643339343D64636261333934010F616263643339353D64636261333935010F616263643339363D64636261333936010F616263643339373D64636261333937010F616263643339383D64636261333938010F616263643339393D64636261333939010F616263643430303D64636261343030010F616263643430313D64636261343031010F616263643430323D64636261343032010F616263643430333D64636261343033010F616263643430343D64636261343034010F616263643430353D64636261343035010F616263643430363D64636261343036010F616263643430373D64636261343037010F616263643430383D64636261343038010F616263643430393D64636261343039010F616263643431303D64636261343130010F616263643431313D64636261343131010F616263643431323D64636261343132010F616263643431333D64636261343133010F616263643431343D64636261343134010F616263643431353D64636261343135010F616263643431363D64636261343136010F616263643431373D64636261343137010F616263643431383D64636261343138010F616263643431393D64636261343139010F616263643432303D64636261343230010F616263643432313D64636261343231010F616263643432323D64636261343232010F616263643432333D64636261343233010F616263643432343D64636261343234010F616263643432353D64636261343235010F616263643432363D64636261343236010F616263643432373D64636261343237010F616263643432383D64636261343238010F616263643432393D64636261343239010F616263643433303D64636261343330010F616263643433313D64636261343331010F616263643433323D64636261343332010F616263643433333D64636261343333010F616263643433343D64636261343334010F616263643433353D64636261343335010F616263643433363D64636261343336010F616263643433373D64636261343337010F616263643433383D64636261343338010F616263643433393D64636261343339010F616263643434303D64636261343430010F616263643434313D64636261343431010F616263643434323D64636261343432010F616263643434333D64636261343433010F616263643434343D64636261343434010F616263643434353D64636261343435010F616263643434363D64636261343436010F616263643434373D64636261343437010F616263643434383D64636261343438010F616263643434393D64636261343439010F616263643435303D64636261343530010F616263643435313D64636261343531010F616263643435323D64636261343532010F616263643435333D64636261343533010F616263643435343D64636261343534010F616263643435353D64636261343535010F616263643435363D64636261343536010F616263643435373D64636261343537010F616263643435383D64636261343538010F616263643435393D64636261343539010F616263643436303D64636261343630010F616263643436313D64636261343631010F616263643436323D64636261343632010F616263643436333D64636261343633010F616263643436343D64636261343634010F616263643436353D64636261343635010F616263643436363D64636261343636010F616263643436373D64636261343637010F616263643436383D64636261343638010F616263643436393D64636261343639010F616263643437303D64636261343730010F616263643437313D64636261343731010F616263643437323D64636261343732010F616263643437333D64636261343733010F616263643437343D64636261343734010F616263643437353D64636261343735010F616263643437363D64636261343736010F616263643437373D64636261343737010F616263643437383D64636261343738010F616263643437393D64636261343739010F616263643438303D64636261343830010F616263643438313D64636261343831010F616263643438323D64636261343832010F616263643438333D64636261343833010F616263643438343D64636261343834010F616263643438353D64636261343835010F616263643438363D64636261343836010F616263643438373D64636261343837010F616263643438383D64636261343838010F616263643438393D64636261343839010F616263643439303D64636261343930010F616263643439313D64636261343931010F616263643439323D64636261343932010F616263643439333D64636261343933010F616263643439343D64636261343934010F616263643439353D64636261343935010F616263643439363D64636261343936010F616263643439373D64636261343937010F616263643439383D64636261343938010F616263643439393D64636261343939010F616263643530303D64636261353030010F616263643530313D64636261353031010F616263643530323D64636261353032010F616263643530333D64636261353033010F616263643530343D64636261353034010F616263643530353D64636261353035010F616263643530363D64636261353036010F616263643530373D64636261353037010F616263643530383D64636261353038010F616263643530393D64636261353039010F616263643531303D64636261353130010F616263643531313D64636261353131010F616263643531323D64636261353132010F616263643531333D646362613531330100000000","proof":{"aunts":[]}}}],"peer_key":""}]}
{"time":"2016-11-16T05:39:30.507Z","msg":[1,{"height":1,"round":0,"step":"RoundStepPrevote"}]}
{"time":"2016-11-16T05:39:30.507Z","msg":[2,{"msg":[20,{"Vote":{"validator_address":"D028C9981F7A87F3093672BF0D5B0E2A1B3ED456","validator_index":0,"height":1,"round":0,"type":1,"block_id":{"hash":"CA01C60FAC15A5234318A0EB241DA209EB5360B9","parts":{"total":1,"hash":"F653FD4F72369E9B8097C8F20EFBCF4689305C66"}},"signature":"4CB04B638E6AF950F797A5388266826626B15EC214C47DAAC5A34C32E444A1CFB2DC187D83A61CBEBCCBA7DD5963D88CACF80998F99336809995B8701836E700"}}],"peer_key":""}]}
{"time":"2016-11-16T05:39:30.508Z","msg":[1,{"height":1,"round":0,"step":"RoundStepPrecommit"}]}
{"time":"2016-11-16T05:39:30.508Z","msg":[2,{"msg":[20,{"Vote":{"validator_address":"D028C9981F7A87F3093672BF0D5B0E2A1B3ED456","validator_index":0,"height":1,"round":0,"type":2,"block_id":{"hash":"CA01C60FAC15A5234318A0EB241DA209EB5360B9","parts":{"total":1,"hash":"F653FD4F72369E9B8097C8F20EFBCF4689305C66"}},"signature":"19516390079992FF1C9F7C370D66137FFD246A31183B10E20611E40D375B597D83718617D3951F39768DA3E588986DE95A65EE12F2577A94065D5489CD8A8F07"}}],"peer_key":""}]}
{"time":"2016-11-16T05:39:30.508Z","msg":[1,{"height":1,"round":0,"step":"RoundStepCommit"}]}

+ 13
- 10
consensus/test_data/small_block2.cswal View File

@ -1,11 +1,14 @@
#HEIGHT: 1
{"time":"2016-10-11T16:21:23.438Z","msg":[3,{"duration":0,"height":1,"round":0,"step":1}]}
{"time":"2016-10-11T16:21:23.440Z","msg":[1,{"height":1,"round":0,"step":"RoundStepPropose"}]}
{"time":"2016-10-11T16:21:23.440Z","msg":[2,{"msg":[17,{"Proposal":{"height":1,"round":0,"block_parts_header":{"total":3,"hash":"88BC082C86DED0A5E2BBC3677B610D155FEDBCEA"},"pol_round":-1,"signature":"8F74F7032E50DFBC17E8B42DD15FD54858B45EEB1B8DAF6432AFBBB1333AC1E850290DE82DF613A10430EB723023527498D45C106FD2946FEF03A9C8B301020B"}}],"peer_key":""}]}
{"time":"2016-10-11T16:21:23.440Z","msg":[2,{"msg":[19,{"Height":1,"Round":0,"Part":{"index":0,"bytes":"0101010F74656E6465726D696E745F746573740101147C86B383BAB78001A60000000001148A3835062BB5E79BE490FAB65168D69BD716AD530114C4B01D3810579550997AC5641E759E20D99B51C1000101A6010F616263643139363D64636261313936010F616263643139373D64636261313937010F616263643139383D64636261313938010F616263643139393D64636261313939010F616263643230303D64636261323030010F616263643230313D64636261323031010F616263643230323D64636261323032010F616263643230333D64636261323033010F616263643230343D64636261323034010F616263643230353D64636261323035010F616263643230363D64636261323036010F616263643230373D64636261323037010F616263643230383D64636261323038010F616263643230393D64636261323039010F616263643231303D64636261323130010F616263643231313D64636261323131010F616263643231323D64636261323132010F616263643231333D64636261323133010F616263643231343D64636261323134010F616263643231353D64636261323135010F616263643231363D64636261323136010F616263643231373D64636261323137010F616263643231383D64636261323138010F616263643231393D64636261323139010F616263643232303D64636261323230010F616263643232313D64636261323231010F616263643232323D64636261323232010F616263643232333D64636261323233010F616263643232343D64636261323234010F616263643232353D64636261323235010F616263643232363D64636261323236010F616263643232373D64636261323237010F616263643232383D64636261323238010F616263643232393D64636261323239010F616263643233303D64636261323330010F616263643233313D64636261323331010F616263643233323D64636261323332010F616263643233333D64636261323333010F616263643233343D64636261323334010F616263643233353D64636261323335010F616263643233363D64636261323336010F616263643233373D64636261323337010F616263643233383D64636261323338010F616263643233393D64636261323339010F616263643234303D64636261323430010F616263643234313D64636261323431010F616263643234323D64636261323432010F616263643234333D64636261323433010F616263643234343D64636261323434010F616263643234353D64636261323435010F616263643234363D64636261323436010F616263643234373D64636261323437010F616263643234383D64636261323438010F616263643234393D64636261323439010F616263643235303D64636261323530010F61626364","proof":{"aunts":["22516491F7E1B5ADD8F12B309E9E8F6F04C034AB","C65A9589F377F2B6CF44B9BAFEBB535DF3C3A4FB"]}}}],"peer_key":""}]}
{"time":"2016-10-11T16:21:23.441Z","msg":[2,{"msg":[19,{"Height":1,"Round":0,"Part":{"index":1,"bytes":"3235313D64636261323531010F616263643235323D64636261323532010F616263643235333D64636261323533010F616263643235343D64636261323534010F616263643235353D64636261323535010F616263643235363D64636261323536010F616263643235373D64636261323537010F616263643235383D64636261323538010F616263643235393D64636261323539010F616263643236303D64636261323630010F616263643236313D64636261323631010F616263643236323D64636261323632010F616263643236333D64636261323633010F616263643236343D64636261323634010F616263643236353D64636261323635010F616263643236363D64636261323636010F616263643236373D64636261323637010F616263643236383D64636261323638010F616263643236393D64636261323639010F616263643237303D64636261323730010F616263643237313D64636261323731010F616263643237323D64636261323732010F616263643237333D64636261323733010F616263643237343D64636261323734010F616263643237353D64636261323735010F616263643237363D64636261323736010F616263643237373D64636261323737010F616263643237383D64636261323738010F616263643237393D64636261323739010F616263643238303D64636261323830010F616263643238313D64636261323831010F616263643238323D64636261323832010F616263643238333D64636261323833010F616263643238343D64636261323834010F616263643238353D64636261323835010F616263643238363D64636261323836010F616263643238373D64636261323837010F616263643238383D64636261323838010F616263643238393D64636261323839010F616263643239303D64636261323930010F616263643239313D64636261323931010F616263643239323D64636261323932010F616263643239333D64636261323933010F616263643239343D64636261323934010F616263643239353D64636261323935010F616263643239363D64636261323936010F616263643239373D64636261323937010F616263643239383D64636261323938010F616263643239393D64636261323939010F616263643330303D64636261333030010F616263643330313D64636261333031010F616263643330323D64636261333032010F616263643330333D64636261333033010F616263643330343D64636261333034010F616263643330353D64636261333035010F616263643330363D64636261333036010F616263643330373D64636261333037010F616263643330383D64636261333038010F616263643330393D64636261333039010F616263643331303D64636261333130010F616263643331313D","proof":{"aunts":["F730990451BAB63C3CF6AC8E6ED4F52259CA5F53","C65A9589F377F2B6CF44B9BAFEBB535DF3C3A4FB"]}}}],"peer_key":""}]}
{"time":"2016-10-11T16:21:23.441Z","msg":[2,{"msg":[19,{"Height":1,"Round":0,"Part":{"index":2,"bytes":"64636261333131010F616263643331323D64636261333132010F616263643331333D64636261333133010F616263643331343D64636261333134010F616263643331353D64636261333135010F616263643331363D64636261333136010F616263643331373D64636261333137010F616263643331383D64636261333138010F616263643331393D64636261333139010F616263643332303D64636261333230010F616263643332313D64636261333231010F616263643332323D64636261333232010F616263643332333D64636261333233010F616263643332343D64636261333234010F616263643332353D64636261333235010F616263643332363D64636261333236010F616263643332373D64636261333237010F616263643332383D64636261333238010F616263643332393D64636261333239010F616263643333303D64636261333330010F616263643333313D64636261333331010F616263643333323D64636261333332010F616263643333333D64636261333333010F616263643333343D64636261333334010F616263643333353D64636261333335010F616263643333363D64636261333336010F616263643333373D64636261333337010F616263643333383D64636261333338010F616263643333393D64636261333339010F616263643334303D64636261333430010F616263643334313D64636261333431010F616263643334323D64636261333432010F616263643334333D64636261333433010F616263643334343D64636261333434010F616263643334353D64636261333435010F616263643334363D64636261333436010F616263643334373D64636261333437010F616263643334383D64636261333438010F616263643334393D64636261333439010F616263643335303D64636261333530010F616263643335313D64636261333531010F616263643335323D64636261333532010F616263643335333D64636261333533010F616263643335343D64636261333534010F616263643335353D64636261333535010F616263643335363D64636261333536010F616263643335373D64636261333537010F616263643335383D64636261333538010F616263643335393D64636261333539010F616263643336303D64636261333630010F616263643336313D646362613336310100","proof":{"aunts":["56EF782EE04E0359D0B38271FD22B312A546FC3A"]}}}],"peer_key":""}]}
{"time":"2016-10-11T16:21:23.447Z","msg":[1,{"height":1,"round":0,"step":"RoundStepPrevote"}]}
{"time":"2016-10-11T16:21:23.447Z","msg":[2,{"msg":[20,{"ValidatorIndex":0,"Vote":{"height":1,"round":0,"type":1,"block_hash":"AAE0ECF64D818A61F6E3D6D11E60F343C3FC8800","block_parts_header":{"total":3,"hash":"88BC082C86DED0A5E2BBC3677B610D155FEDBCEA"},"signature":"0870A9C3FF59DE0F5574B77F030BD160C1E2966AECE815E7C97CFA8BC4A6B01D7A10D91416B1AA02D49EFF7F08A239048CD9CD93E7AE4F80871FBFFF7DBFC50C"}}],"peer_key":""}]}
{"time":"2016-10-11T16:21:23.448Z","msg":[1,{"height":1,"round":0,"step":"RoundStepPrecommit"}]}
{"time":"2016-10-11T16:21:23.448Z","msg":[2,{"msg":[20,{"ValidatorIndex":0,"Vote":{"height":1,"round":0,"type":2,"block_hash":"AAE0ECF64D818A61F6E3D6D11E60F343C3FC8800","block_parts_header":{"total":3,"hash":"88BC082C86DED0A5E2BBC3677B610D155FEDBCEA"},"signature":"0CEEA8A987D88D0A0870C0076DB8D1B57D3B051D017745B46C4710BBE6DF0F9AE8D5A95B49E4158A1A8C8C6475B8A8E91275303B9C10A5C0C18F40EBB0DA0905"}}],"peer_key":""}]}
{"time":"2016-11-16T05:41:35.271Z","msg":[3,{"duration":953420437,"height":1,"round":0,"step":1}]}
{"time":"2016-11-16T05:41:35.272Z","msg":[1,{"height":1,"round":0,"step":"RoundStepPropose"}]}
{"time":"2016-11-16T05:41:35.272Z","msg":[2,{"msg":[17,{"Proposal":{"height":1,"round":0,"block_parts_header":{"total":5,"hash":"2BDB8E53CD09D48FF4CB53E538EE4824D1F66AF6"},"pol_round":-1,"pol_block_id":{"hash":"","parts":{"total":0,"hash":""}},"signature":"96B4688C79E9DEB99BCFD86D663FB206FEF8793A29815474B45090D3D7D2955EC9E19A0750BCE9F84C2331F566E3D8073DFA2ECB387C266E502CA3BED12D4407"}}],"peer_key":""}]}
{"time":"2016-11-16T05:41:35.272Z","msg":[2,{"msg":[19,{"Height":1,"Round":0,"Part":{"index":0,"bytes":"0101010F74656E6465726D696E745F746573740101148770AE01C7F7C0018000000000011417A1EBF69CBCDF3A2E4FCC009EBA8F1BE1CCFBBF0114C4B01D3810579550997AC5641E759E20D99B51C100010180010F616263643338353D64636261333835010F616263643338363D64636261333836010F616263643338373D64636261333837010F616263643338383D64636261333838010F616263643338393D64636261333839010F616263643339303D64636261333930010F616263643339313D64636261333931010F616263643339323D64636261333932010F616263643339333D64636261333933010F616263643339343D64636261333934010F616263643339353D64636261333935010F616263643339363D64636261333936010F616263643339373D64636261333937010F616263643339383D64636261333938010F616263643339393D64636261333939010F616263643430303D64636261343030010F616263643430313D64636261343031010F616263643430323D64636261343032010F616263643430333D64636261343033010F616263643430343D64636261343034010F616263643430353D64636261343035010F616263643430363D64636261343036010F616263643430373D64636261343037010F616263643430383D64636261343038010F616263643430393D64636261343039010F6162","proof":{"aunts":["1D502BD66FA41A3C310CFEAA95D61959A6A59D62","E942609B2117FE09937DAED58E1202B21C0462F6","B06CAA4A19EEE5627EDC7E420A8A4C7D5C14BCD5"]}}}],"peer_key":""}]}
{"time":"2016-11-16T05:41:35.272Z","msg":[2,{"msg":[19,{"Height":1,"Round":0,"Part":{"index":1,"bytes":"63643431303D64636261343130010F616263643431313D64636261343131010F616263643431323D64636261343132010F616263643431333D64636261343133010F616263643431343D64636261343134010F616263643431353D64636261343135010F616263643431363D64636261343136010F616263643431373D64636261343137010F616263643431383D64636261343138010F616263643431393D64636261343139010F616263643432303D64636261343230010F616263643432313D64636261343231010F616263643432323D64636261343232010F616263643432333D64636261343233010F616263643432343D64636261343234010F616263643432353D64636261343235010F616263643432363D64636261343236010F616263643432373D64636261343237010F616263643432383D64636261343238010F616263643432393D64636261343239010F616263643433303D64636261343330010F616263643433313D64636261343331010F616263643433323D64636261343332010F616263643433333D64636261343333010F616263643433343D64636261343334010F616263643433353D64636261343335010F616263643433363D64636261343336010F616263643433373D64636261343337010F616263643433383D64636261343338010F616263643433393D64636261343339010F61626364","proof":{"aunts":["FB7ABE2193609A655B1471562C7BF6826CB73E69","E942609B2117FE09937DAED58E1202B21C0462F6","B06CAA4A19EEE5627EDC7E420A8A4C7D5C14BCD5"]}}}],"peer_key":""}]}
{"time":"2016-11-16T05:41:35.272Z","msg":[2,{"msg":[19,{"Height":1,"Round":0,"Part":{"index":2,"bytes":"3434303D64636261343430010F616263643434313D64636261343431010F616263643434323D64636261343432010F616263643434333D64636261343433010F616263643434343D64636261343434010F616263643434353D64636261343435010F616263643434363D64636261343436010F616263643434373D64636261343437010F616263643434383D64636261343438010F616263643434393D64636261343439010F616263643435303D64636261343530010F616263643435313D64636261343531010F616263643435323D64636261343532010F616263643435333D64636261343533010F616263643435343D64636261343534010F616263643435353D64636261343535010F616263643435363D64636261343536010F616263643435373D64636261343537010F616263643435383D64636261343538010F616263643435393D64636261343539010F616263643436303D64636261343630010F616263643436313D64636261343631010F616263643436323D64636261343632010F616263643436333D64636261343633010F616263643436343D64636261343634010F616263643436353D64636261343635010F616263643436363D64636261343636010F616263643436373D64636261343637010F616263643436383D64636261343638010F616263643436393D64636261343639010F616263643437","proof":{"aunts":["35F636CBC2D6A2B2BC897AFBC42DFFB47C899527","B06CAA4A19EEE5627EDC7E420A8A4C7D5C14BCD5"]}}}],"peer_key":""}]}
{"time":"2016-11-16T05:41:35.272Z","msg":[2,{"msg":[19,{"Height":1,"Round":0,"Part":{"index":3,"bytes":"303D64636261343730010F616263643437313D64636261343731010F616263643437323D64636261343732010F616263643437333D64636261343733010F616263643437343D64636261343734010F616263643437353D64636261343735010F616263643437363D64636261343736010F616263643437373D64636261343737010F616263643437383D64636261343738010F616263643437393D64636261343739010F616263643438303D64636261343830010F616263643438313D64636261343831010F616263643438323D64636261343832010F616263643438333D64636261343833010F616263643438343D64636261343834010F616263643438353D64636261343835010F616263643438363D64636261343836010F616263643438373D64636261343837010F616263643438383D64636261343838010F616263643438393D64636261343839010F616263643439303D64636261343930010F616263643439313D64636261343931010F616263643439323D64636261343932010F616263643439333D64636261343933010F616263643439343D64636261343934010F616263643439353D64636261343935010F616263643439363D64636261343936010F616263643439373D64636261343937010F616263643439383D64636261343938010F616263643439393D64636261343939010F616263643530303D","proof":{"aunts":["206BCAA7D04AA535ECCE9D394B6D72C27E94FFB0","E9C0EE8407389C3CBF707425A577513095F38DE2"]}}}],"peer_key":""}]}
{"time":"2016-11-16T05:41:35.272Z","msg":[2,{"msg":[19,{"Height":1,"Round":0,"Part":{"index":4,"bytes":"64636261353030010F616263643530313D64636261353031010F616263643530323D64636261353032010F616263643530333D64636261353033010F616263643530343D64636261353034010F616263643530353D64636261353035010F616263643530363D64636261353036010F616263643530373D64636261353037010F616263643530383D64636261353038010F616263643530393D64636261353039010F616263643531303D64636261353130010F616263643531313D64636261353131010F616263643531323D646362613531320100000000","proof":{"aunts":["8AA5C39BFC1E062439B4FC6B03812909E4D9D85C","E9C0EE8407389C3CBF707425A577513095F38DE2"]}}}],"peer_key":""}]}
{"time":"2016-11-16T05:41:35.273Z","msg":[1,{"height":1,"round":0,"step":"RoundStepPrevote"}]}
{"time":"2016-11-16T05:41:35.273Z","msg":[2,{"msg":[20,{"Vote":{"validator_address":"D028C9981F7A87F3093672BF0D5B0E2A1B3ED456","validator_index":0,"height":1,"round":0,"type":1,"block_id":{"hash":"06C99E8836006B39E00C84828A87920E0354CC95","parts":{"total":5,"hash":"2BDB8E53CD09D48FF4CB53E538EE4824D1F66AF6"}},"signature":"7FD916E1CC78500981F05A936E9099A7500408F2A786DFB1ACAADA7754E014CF53B87216A8AE1DF9A876C51CD5F3EF57CA04E1EEFC21085EF57FA9F7BE32230F"}}],"peer_key":""}]}
{"time":"2016-11-16T05:41:35.303Z","msg":[1,{"height":1,"round":0,"step":"RoundStepPrecommit"}]}
{"time":"2016-11-16T05:41:35.303Z","msg":[2,{"msg":[20,{"Vote":{"validator_address":"D028C9981F7A87F3093672BF0D5B0E2A1B3ED456","validator_index":0,"height":1,"round":0,"type":2,"block_id":{"hash":"06C99E8836006B39E00C84828A87920E0354CC95","parts":{"total":5,"hash":"2BDB8E53CD09D48FF4CB53E538EE4824D1F66AF6"}},"signature":"BB4EC9F1FD4DA983C1756499DA94947E8CA8C1DD882F84C369C672FB8542C5722311787B9E774F4F1776CC4B46A25D369F7FCB8D5D34BB73E85C2CF94852DF0B"}}],"peer_key":""}]}
{"time":"2016-11-16T05:41:35.303Z","msg":[1,{"height":1,"round":0,"step":"RoundStepCommit"}]}

+ 18
- 2
consensus/wal.go View File

@ -54,7 +54,19 @@ func NewWAL(walDir string, light bool) (*WAL, error) {
light: light,
}
wal.BaseService = *NewBaseService(log, "WAL", wal)
return wal, nil
_, err = wal.Start()
return wal, err
}
func (wal *WAL) OnStart() error {
wal.BaseService.OnStart()
size, err := wal.group.Head.Size()
if err != nil {
return err
} else if size == 0 {
wal.writeHeight(1)
}
return nil
}
func (wal *WAL) OnStop() {
@ -80,7 +92,7 @@ func (wal *WAL) Save(wmsg WALMessage) {
// Write #HEIGHT: XYZ if new height
if edrs, ok := wmsg.(types.EventDataRoundState); ok {
if edrs.Step == RoundStepNewHeight.String() {
wal.group.WriteLine(Fmt("#HEIGHT: %v", edrs.Height))
wal.writeHeight(edrs.Height)
}
}
// Write the wal message
@ -90,3 +102,7 @@ func (wal *WAL) Save(wmsg WALMessage) {
PanicQ(Fmt("Error writing msg to consensus wal. Error: %v \n\nMessage: %v", err, wmsg))
}
}
func (wal *WAL) writeHeight(height int) {
wal.group.WriteLine(Fmt("#HEIGHT: %v", height))
}

+ 2
- 2
node/node.go View File

@ -94,7 +94,7 @@ func NewNode(config cfg.Config, privValidator *types.PrivValidator, clientCreato
}
// Make BlockchainReactor
bcReactor := bc.NewBlockchainReactor(state.Copy(), proxyApp.Consensus(), blockStore, fastSync)
bcReactor := bc.NewBlockchainReactor(config, state.Copy(), proxyApp.Consensus(), blockStore, fastSync)
// Make MempoolReactor
mempool := mempl.NewMempool(config, proxyApp.Mempool())
@ -102,7 +102,7 @@ func NewNode(config cfg.Config, privValidator *types.PrivValidator, clientCreato
// Make ConsensusReactor
consensusState := consensus.NewConsensusState(config, state.Copy(), proxyApp.Consensus(), blockStore, mempool)
consensusReactor := consensus.NewConsensusReactor(consensusState, blockStore, fastSync)
consensusReactor := consensus.NewConsensusReactor(consensusState, fastSync)
if privValidator != nil {
consensusReactor.SetPrivValidator(privValidator)
}


+ 19
- 0
scripts/txs/random.sh View File

@ -0,0 +1,19 @@
#! /bin/bash
set -u
function toHex() {
echo -n $1 | hexdump -ve '1/1 "%.2X"'
}
N=$1
PORT=$2
for i in `seq 1 $N`; do
# store key value pair
KEY="abcd$i"
VALUE="dcba$i"
echo "$KEY:$VALUE"
curl 127.0.0.1:$PORT/broadcast_tx_sync?tx=\"$(toHex $KEY=$VALUE)\"
done

+ 3
- 4
state/execution.go View File

@ -43,8 +43,7 @@ func (s *State) ExecBlock(eventCache types.Fireable, proxyAppConn proxy.AppConnC
// All good!
nextValSet.IncrementAccum(1)
s.LastBlockHeight = block.Height
s.LastBlockHash = block.Hash()
s.LastBlockParts = blockPartsHeader
s.LastBlockID = types.BlockID{block.Hash(), blockPartsHeader}
s.LastBlockTime = block.Time
s.Validators = nextValSet
s.LastValidators = valSet
@ -119,7 +118,7 @@ func (s *State) execBlockOnProxyApp(eventCache types.Fireable, proxyAppConn prox
func (s *State) validateBlock(block *types.Block) error {
// Basic block validation.
err := block.ValidateBasic(s.ChainID, s.LastBlockHeight, s.LastBlockHash, s.LastBlockParts, s.LastBlockTime, s.AppHash)
err := block.ValidateBasic(s.ChainID, s.LastBlockHeight, s.LastBlockID, s.LastBlockTime, s.AppHash)
if err != nil {
return err
}
@ -135,7 +134,7 @@ func (s *State) validateBlock(block *types.Block) error {
s.LastValidators.Size(), len(block.LastCommit.Precommits))
}
err := s.LastValidators.VerifyCommit(
s.ChainID, s.LastBlockHash, s.LastBlockParts, block.Height-1, block.LastCommit)
s.ChainID, s.LastBlockID, block.Height-1, block.LastCommit)
if err != nil {
return err
}


+ 3
- 6
state/state.go View File

@ -25,8 +25,7 @@ type State struct {
GenesisDoc *types.GenesisDoc
ChainID string
LastBlockHeight int // Genesis state has this set to 0. So, Block(H=0) does not exist.
LastBlockHash []byte
LastBlockParts types.PartSetHeader
LastBlockID types.BlockID
LastBlockTime time.Time
Validators *types.ValidatorSet
LastValidators *types.ValidatorSet
@ -56,8 +55,7 @@ func (s *State) Copy() *State {
GenesisDoc: s.GenesisDoc,
ChainID: s.ChainID,
LastBlockHeight: s.LastBlockHeight,
LastBlockHash: s.LastBlockHash,
LastBlockParts: s.LastBlockParts,
LastBlockID: s.LastBlockID,
LastBlockTime: s.LastBlockTime,
Validators: s.Validators.Copy(),
LastValidators: s.LastValidators.Copy(),
@ -117,8 +115,7 @@ func MakeGenesisState(db dbm.DB, genDoc *types.GenesisDoc) *State {
GenesisDoc: genDoc,
ChainID: genDoc.ChainID,
LastBlockHeight: 0,
LastBlockHash: nil,
LastBlockParts: types.PartSetHeader{},
LastBlockID: types.BlockID{},
LastBlockTime: genDoc.GenesisTime,
Validators: types.NewValidatorSet(validators),
LastValidators: types.NewValidatorSet(nil),


+ 67
- 33
types/block.go View File

@ -4,6 +4,7 @@ import (
"bytes"
"errors"
"fmt"
"io"
"strings"
"time"
@ -21,8 +22,8 @@ type Block struct {
}
// Basic validation that doesn't involve state data.
func (b *Block) ValidateBasic(chainID string, lastBlockHeight int, lastBlockHash []byte,
lastBlockParts PartSetHeader, lastBlockTime time.Time, appHash []byte) error {
func (b *Block) ValidateBasic(chainID string, lastBlockHeight int, lastBlockID BlockID,
lastBlockTime time.Time, appHash []byte) error {
if b.ChainID != chainID {
return errors.New(Fmt("Wrong Block.Header.ChainID. Expected %v, got %v", chainID, b.ChainID))
}
@ -39,11 +40,8 @@ func (b *Block) ValidateBasic(chainID string, lastBlockHeight int, lastBlockHash
if b.NumTxs != len(b.Data.Txs) {
return errors.New(Fmt("Wrong Block.Header.NumTxs. Expected %v, got %v", len(b.Data.Txs), b.NumTxs))
}
if !bytes.Equal(b.LastBlockHash, lastBlockHash) {
return errors.New(Fmt("Wrong Block.Header.LastBlockHash. Expected %X, got %X", lastBlockHash, b.LastBlockHash))
}
if !b.LastBlockParts.Equals(lastBlockParts) {
return errors.New(Fmt("Wrong Block.Header.LastBlockParts. Expected %v, got %v", lastBlockParts, b.LastBlockParts))
if !b.LastBlockID.Equals(lastBlockID) {
return errors.New(Fmt("Wrong Block.Header.LastBlockID. Expected %v, got %v", lastBlockID, b.LastBlockID))
}
if !bytes.Equal(b.LastCommitHash, b.LastCommit.Hash()) {
return errors.New(Fmt("Wrong Block.Header.LastCommitHash. Expected %X, got %X", b.LastCommitHash, b.LastCommit.Hash()))
@ -83,8 +81,8 @@ func (b *Block) Hash() []byte {
return b.Header.Hash()
}
func (b *Block) MakePartSet() *PartSet {
return NewPartSetFromData(wire.BinaryBytes(b))
func (b *Block) MakePartSet(partSize int) *PartSet {
return NewPartSetFromData(wire.BinaryBytes(b), partSize)
}
// Convenience.
@ -130,16 +128,15 @@ func (b *Block) StringShort() string {
//-----------------------------------------------------------------------------
type Header struct {
ChainID string `json:"chain_id"`
Height int `json:"height"`
Time time.Time `json:"time"`
NumTxs int `json:"num_txs"`
LastBlockHash []byte `json:"last_block_hash"`
LastBlockParts PartSetHeader `json:"last_block_parts"`
LastCommitHash []byte `json:"last_commit_hash"`
DataHash []byte `json:"data_hash"`
ValidatorsHash []byte `json:"validators_hash"`
AppHash []byte `json:"app_hash"` // state merkle root of txs from the previous block
ChainID string `json:"chain_id"`
Height int `json:"height"`
Time time.Time `json:"time"`
NumTxs int `json:"num_txs"`
LastBlockID BlockID `json:"last_block_id"`
LastCommitHash []byte `json:"last_commit_hash"`
DataHash []byte `json:"data_hash"`
ValidatorsHash []byte `json:"validators_hash"`
AppHash []byte `json:"app_hash"` // state merkle root of txs from the previous block
}
// NOTE: hash is nil if required fields are missing.
@ -148,16 +145,15 @@ func (h *Header) Hash() []byte {
return nil
}
return merkle.SimpleHashFromMap(map[string]interface{}{
"ChainID": h.ChainID,
"Height": h.Height,
"Time": h.Time,
"NumTxs": h.NumTxs,
"LastBlock": h.LastBlockHash,
"LastBlockParts": h.LastBlockParts,
"LastCommit": h.LastCommitHash,
"Data": h.DataHash,
"Validators": h.ValidatorsHash,
"App": h.AppHash,
"ChainID": h.ChainID,
"Height": h.Height,
"Time": h.Time,
"NumTxs": h.NumTxs,
"LastBlockID": h.LastBlockID,
"LastCommit": h.LastCommitHash,
"Data": h.DataHash,
"Validators": h.ValidatorsHash,
"App": h.AppHash,
})
}
@ -170,8 +166,7 @@ func (h *Header) StringIndented(indent string) string {
%s Height: %v
%s Time: %v
%s NumTxs: %v
%s LastBlock: %X
%s LastBlockParts: %v
%s LastBlockID: %v
%s LastCommit: %X
%s Data: %X
%s Validators: %X
@ -181,8 +176,7 @@ func (h *Header) StringIndented(indent string) string {
indent, h.Height,
indent, h.Time,
indent, h.NumTxs,
indent, h.LastBlockHash,
indent, h.LastBlockParts,
indent, h.LastBlockID,
indent, h.LastCommitHash,
indent, h.DataHash,
indent, h.ValidatorsHash,
@ -197,6 +191,7 @@ type Commit struct {
// NOTE: The Precommits are in order of address to preserve the bonded ValidatorSet order.
// Any peer with a block can gossip precommits by index with a peer without recalculating the
// active ValidatorSet.
BlockID BlockID `json:"blockID"`
Precommits []*Vote `json:"precommits"`
// Volatile
@ -268,6 +263,9 @@ func (commit *Commit) IsCommit() bool {
}
func (commit *Commit) ValidateBasic() error {
if commit.BlockID.IsZero() {
return errors.New("Commit cannot be for nil block")
}
if len(commit.Precommits) == 0 {
return errors.New("No precommits in commit")
}
@ -316,8 +314,10 @@ func (commit *Commit) StringIndented(indent string) string {
precommitStrings[i] = precommit.String()
}
return fmt.Sprintf(`Commit{
%s BlockID: %v
%s Precommits: %v
%s}#%X`,
indent, commit.BlockID,
indent, strings.Join(precommitStrings, "\n"+indent+" "),
indent, commit.hash)
}
@ -360,3 +360,37 @@ func (data *Data) StringIndented(indent string) string {
indent, strings.Join(txStrings, "\n"+indent+" "),
indent, data.hash)
}
//--------------------------------------------------------------------------------
type BlockID struct {
Hash []byte `json:"hash"`
PartsHeader PartSetHeader `json:"parts"`
}
func (blockID BlockID) IsZero() bool {
return len(blockID.Hash) == 0 && blockID.PartsHeader.IsZero()
}
func (blockID BlockID) Equals(other BlockID) bool {
return bytes.Equal(blockID.Hash, other.Hash) &&
blockID.PartsHeader.Equals(other.PartsHeader)
}
func (blockID BlockID) Key() string {
return string(blockID.Hash) + string(wire.BinaryBytes(blockID.PartsHeader))
}
func (blockID BlockID) WriteSignBytes(w io.Writer, n *int, err *error) {
if blockID.IsZero() {
wire.WriteTo([]byte("null"), w, n, err)
} else {
wire.WriteTo([]byte(Fmt(`{"hash":"%X","parts":`, blockID.Hash)), w, n, err)
blockID.PartsHeader.WriteSignBytes(w, n, err)
wire.WriteTo([]byte("}"), w, n, err)
}
}
func (blockID BlockID) String() string {
return fmt.Sprintf(`%X:%v`, blockID.Hash, blockID.PartsHeader)
}

+ 1
- 3
types/events.go View File

@ -91,9 +91,7 @@ type EventDataRoundState struct {
}
type EventDataVote struct {
Index int
Address []byte
Vote *Vote
Vote *Vote
}
func (_ EventDataNewBlock) AssertIsTMEventData() {}


+ 2
- 6
types/part_set.go View File

@ -14,10 +14,6 @@ import (
"github.com/tendermint/go-wire"
)
const (
partSize = 65536 // 64KB ... 4096 // 4KB
)
var (
ErrPartSetUnexpectedIndex = errors.New("Error part set unexpected index")
ErrPartSetInvalidProof = errors.New("Error part set invalid proof")
@ -66,7 +62,7 @@ type PartSetHeader struct {
}
func (psh PartSetHeader) String() string {
return fmt.Sprintf("PartSet{T:%v %X}", psh.Total, Fingerprint(psh.Hash))
return fmt.Sprintf("%v:%X", psh.Total, Fingerprint(psh.Hash))
}
func (psh PartSetHeader) IsZero() bool {
@ -95,7 +91,7 @@ type PartSet struct {
// Returns an immutable, full PartSet from the data bytes.
// The data bytes are split into "partSize" chunks, and merkle tree computed.
func NewPartSetFromData(data []byte) *PartSet {
func NewPartSetFromData(data []byte, partSize int) *PartSet {
// divide data into 4kb parts.
total := (len(data) + partSize - 1) / partSize
parts := make([]*Part, total)


+ 8
- 4
types/part_set_test.go View File

@ -8,12 +8,16 @@ import (
. "github.com/tendermint/go-common"
)
const (
testPartSize = 65536 // 64KB ... 4096 // 4KB
)
func TestBasicPartSet(t *testing.T) {
// Construct random data of size partSize * 100
data := RandBytes(partSize * 100)
data := RandBytes(testPartSize * 100)
partSet := NewPartSetFromData(data)
partSet := NewPartSetFromData(data, testPartSize)
if len(partSet.Hash()) == 0 {
t.Error("Expected to get hash")
}
@ -61,8 +65,8 @@ func TestBasicPartSet(t *testing.T) {
func TestWrongProof(t *testing.T) {
// Construct random data of size partSize * 100
data := RandBytes(partSize * 100)
partSet := NewPartSetFromData(data)
data := RandBytes(testPartSize * 100)
partSet := NewPartSetFromData(data, testPartSize)
// Test adding a part with wrong data.
partSet2 := NewPartSetFromHeader(partSet.Header())


+ 5
- 0
types/priv_validator.go View File

@ -163,6 +163,10 @@ func (privVal *PrivValidator) Reset() {
privVal.Save()
}
func (privVal *PrivValidator) GetAddress() []byte {
return privVal.Address
}
func (privVal *PrivValidator) SignVote(chainID string, vote *Vote) error {
privVal.mtx.Lock()
defer privVal.mtx.Unlock()
@ -231,6 +235,7 @@ func (privVal *PrivValidator) signBytesHRS(height, round int, step int8, signByt
privVal.save()
return signature, nil
}
func (privVal *PrivValidator) String() string {


+ 9
- 5
types/proposal.go View File

@ -19,29 +19,33 @@ type Proposal struct {
Height int `json:"height"`
Round int `json:"round"`
BlockPartsHeader PartSetHeader `json:"block_parts_header"`
POLRound int `json:"pol_round"` // -1 if null.
POLRound int `json:"pol_round"` // -1 if null.
POLBlockID BlockID `json:"pol_block_id"` // zero if null.
Signature crypto.SignatureEd25519 `json:"signature"`
}
// polRound: -1 if no polRound.
func NewProposal(height int, round int, blockPartsHeader PartSetHeader, polRound int) *Proposal {
func NewProposal(height int, round int, blockPartsHeader PartSetHeader, polRound int, polBlockID BlockID) *Proposal {
return &Proposal{
Height: height,
Round: round,
BlockPartsHeader: blockPartsHeader,
POLRound: polRound,
POLBlockID: polBlockID,
}
}
func (p *Proposal) String() string {
return fmt.Sprintf("Proposal{%v/%v %v %v %v}", p.Height, p.Round,
p.BlockPartsHeader, p.POLRound, p.Signature)
return fmt.Sprintf("Proposal{%v/%v %v (%v,%v) %v}", p.Height, p.Round,
p.BlockPartsHeader, p.POLRound, p.POLBlockID, p.Signature)
}
func (p *Proposal) WriteSignBytes(chainID string, w io.Writer, n *int, err *error) {
wire.WriteTo([]byte(Fmt(`{"chain_id":"%s"`, chainID)), w, n, err)
wire.WriteTo([]byte(`,"proposal":{"block_parts_header":`), w, n, err)
p.BlockPartsHeader.WriteSignBytes(w, n, err)
wire.WriteTo([]byte(Fmt(`,"height":%v,"pol_round":%v`, p.Height, p.POLRound)), w, n, err)
wire.WriteTo([]byte(Fmt(`,"height":%v,"pol_block_id":`, p.Height)), w, n, err)
p.POLBlockID.WriteSignBytes(w, n, err)
wire.WriteTo([]byte(Fmt(`,"pol_round":%v`, p.POLRound)), w, n, err)
wire.WriteTo([]byte(Fmt(`,"round":%v}}`, p.Round)), w, n, err)
}

+ 2
- 2
types/proposal_test.go View File

@ -14,8 +14,8 @@ func TestProposalSignable(t *testing.T) {
signBytes := SignBytes("test_chain_id", proposal)
signStr := string(signBytes)
expected := `{"chain_id":"test_chain_id","proposal":{"block_parts_header":{"hash":"626C6F636B7061727473","total":111},"height":12345,"pol_round":-1,"round":23456}}`
expected := `{"chain_id":"test_chain_id","proposal":{"block_parts_header":{"hash":"626C6F636B7061727473","total":111},"height":12345,"pol_block_id":null,"pol_round":-1,"round":23456}}`
if signStr != expected {
t.Errorf("Got unexpected sign string for SendTx. Expected:\n%v\nGot:\n%v", expected, signStr)
t.Errorf("Got unexpected sign string for Proposal. Expected:\n%v\nGot:\n%v", expected, signStr)
}
}

+ 3
- 6
types/validator_set.go View File

@ -206,8 +206,7 @@ func (valSet *ValidatorSet) Iterate(fn func(index int, val *Validator) bool) {
}
// Verify that +2/3 of the set had signed the given signBytes
func (valSet *ValidatorSet) VerifyCommit(chainID string,
hash []byte, parts PartSetHeader, height int, commit *Commit) error {
func (valSet *ValidatorSet) VerifyCommit(chainID string, blockID BlockID, height int, commit *Commit) error {
if valSet.Size() != len(commit.Precommits) {
return fmt.Errorf("Invalid commit -- wrong set size: %v vs %v", valSet.Size(), len(commit.Precommits))
}
@ -238,10 +237,7 @@ func (valSet *ValidatorSet) VerifyCommit(chainID string,
if !val.PubKey.VerifyBytes(precommitSignBytes, precommit.Signature) {
return fmt.Errorf("Invalid commit -- invalid signature: %v", precommit)
}
if !bytes.Equal(precommit.BlockHash, hash) {
continue // Not an error, but doesn't count
}
if !parts.Equals(precommit.BlockPartsHeader) {
if !blockID.Equals(precommit.BlockID) {
continue // Not an error, but doesn't count
}
// Good precommit!
@ -313,6 +309,7 @@ func (ac accumComparable) Less(o interface{}) bool {
//----------------------------------------
// For testing
// NOTE: PrivValidator are in order.
func RandValidatorSet(numValidators int, votingPower int64) (*ValidatorSet, []*PrivValidator) {
vals := make([]*Validator, numValidators)
privValidators := make([]*PrivValidator, numValidators)


+ 35
- 17
types/vote.go View File

@ -11,40 +11,55 @@ import (
)
var (
ErrVoteUnexpectedStep = errors.New("Unexpected step")
ErrVoteInvalidAccount = errors.New("Invalid round vote account")
ErrVoteInvalidSignature = errors.New("Invalid round vote signature")
ErrVoteInvalidBlockHash = errors.New("Invalid block hash")
ErrVoteUnexpectedStep = errors.New("Unexpected step")
ErrVoteInvalidValidatorIndex = errors.New("Invalid round vote validator index")
ErrVoteInvalidValidatorAddress = errors.New("Invalid round vote validator address")
ErrVoteInvalidSignature = errors.New("Invalid round vote signature")
ErrVoteInvalidBlockHash = errors.New("Invalid block hash")
)
type ErrVoteConflictingSignature struct {
type ErrVoteConflictingVotes struct {
VoteA *Vote
VoteB *Vote
}
func (err *ErrVoteConflictingSignature) Error() string {
return "Conflicting round vote signature"
func (err *ErrVoteConflictingVotes) Error() string {
return "Conflicting votes"
}
// 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
}
}
// Represents a prevote, precommit, or commit vote from validators for consensus.
type Vote struct {
ValidatorAddress []byte `json:"validator_address"`
ValidatorIndex int `json:"validator_index"`
Height int `json:"height"`
Round int `json:"round"`
Type byte `json:"type"`
BlockHash []byte `json:"block_hash"` // empty if vote is nil.
BlockPartsHeader PartSetHeader `json:"block_parts_header"` // zero if vote is nil.
BlockID BlockID `json:"block_id"` // zero if vote is nil.
Signature crypto.SignatureEd25519 `json:"signature"`
}
// Types of votes
const (
VoteTypePrevote = byte(0x01)
VoteTypePrecommit = byte(0x02)
)
func (vote *Vote) WriteSignBytes(chainID string, w io.Writer, n *int, err *error) {
wire.WriteTo([]byte(Fmt(`{"chain_id":"%s"`, chainID)), w, n, err)
wire.WriteTo([]byte(Fmt(`,"vote":{"block_hash":"%X","block_parts_header":%v`, vote.BlockHash, vote.BlockPartsHeader)), w, n, err)
wire.WriteTo([]byte(`,"vote":{"block_id":`), w, n, err)
vote.BlockID.WriteSignBytes(w, n, err)
wire.WriteTo([]byte(Fmt(`,"height":%v,"round":%v,"type":%v}}`, vote.Height, vote.Round, vote.Type)), w, n, err)
}
@ -67,5 +82,8 @@ func (vote *Vote) String() string {
PanicSanity("Unknown vote type")
}
return fmt.Sprintf("Vote{%v/%02d/%v(%v) %X#%v %v}", vote.Height, vote.Round, vote.Type, typeString, Fingerprint(vote.BlockHash), vote.BlockPartsHeader, vote.Signature)
return fmt.Sprintf("Vote{%v:%X %v/%02d/%v(%v) %X %v}",
vote.ValidatorIndex, Fingerprint(vote.ValidatorAddress),
vote.Height, vote.Round, vote.Type, typeString,
Fingerprint(vote.BlockID.Hash), vote.Signature)
}

+ 286
- 105
types/vote_set.go View File

@ -7,29 +7,56 @@ import (
"sync"
. "github.com/tendermint/go-common"
"github.com/tendermint/go-wire"
)
// VoteSet helps collect signatures from validators at each height+round
// for a predefined vote type.
// Note that there three kinds of votes: prevotes, precommits, and commits.
// A commit of prior rounds can be added added in lieu of votes/precommits.
// NOTE: Assumes that the sum total of voting power does not exceed MaxUInt64.
/*
VoteSet helps collect signatures from validators at each height+round for a
predefined vote type.
We need VoteSet to be able to keep track of conflicting votes when validators
double-sign. Yet, we can't keep track of *all* the votes seen, as that could
be a DoS attack vector.
There are two storage areas for votes.
1. voteSet.votes
2. voteSet.votesByBlock
`.votes` is the "canonical" list of votes. It always has at least one vote,
if a vote from a validator had been seen at all. Usually it keeps track of
the first vote seen, but when a 2/3 majority is found, votes for that get
priority and are copied over from `.votesByBlock`.
`.votesByBlock` keeps track of a list of votes for a particular block. There
are two ways a &blockVotes{} gets created in `.votesByBlock`.
1. the first vote seen by a validator was for the particular block.
2. a peer claims to have seen 2/3 majority for the particular block.
Since the first vote from a validator will always get added in `.votesByBlock`
, all votes in `.votes` will have a corresponding entry in `.votesByBlock`.
When a &blockVotes{} in `.votesByBlock` reaches a 2/3 majority quorum, its
votes are copied into `.votes`.
All this is memory bounded because conflicting votes only get added if a peer
told us to track that block, each peer only gets to tell us 1 such block, and,
there's only a limited number of peers.
NOTE: Assumes that the sum total of voting power does not exceed MaxUInt64.
*/
type VoteSet struct {
chainID string
height int
round int
type_ byte
mtx sync.Mutex
valSet *ValidatorSet
votes []*Vote // validator index -> vote
votesBitArray *BitArray // validator index -> has vote?
votesByBlock map[string]int64 // string(blockHash)+string(blockParts) -> vote sum.
totalVotes int64
maj23Hash []byte
maj23PartsHeader PartSetHeader
maj23Exists bool
mtx sync.Mutex
valSet *ValidatorSet
votesBitArray *BitArray
votes []*Vote // Primary votes to share
sum int64 // Sum of voting power for seen votes, discounting conflicts
maj23 *BlockID // First 2/3 majority seen
votesByBlock map[string]*blockVotes // string(blockHash|blockParts) -> blockVotes
peerMaj23s map[string]BlockID // Maj23 for each peer
}
// Constructs a new VoteSet struct used to accumulate votes for given height/round.
@ -43,10 +70,12 @@ func NewVoteSet(chainID string, height int, round int, type_ byte, valSet *Valid
round: round,
type_: type_,
valSet: valSet,
votes: make([]*Vote, valSet.Size()),
votesBitArray: NewBitArray(valSet.Size()),
votesByBlock: make(map[string]int64),
totalVotes: 0,
votes: make([]*Vote, valSet.Size()),
sum: 0,
maj23: nil,
votesByBlock: make(map[string]*blockVotes, valSet.Size()),
peerMaj23s: make(map[string]BlockID),
}
}
@ -86,94 +115,197 @@ func (voteSet *VoteSet) Size() int {
}
}
// Returns added=true, index if vote was added
// Otherwise returns err=ErrVote[UnexpectedStep|InvalidAccount|InvalidSignature|InvalidBlockHash|ConflictingSignature]
// Returns added=true if vote is valid and new.
// Otherwise returns err=ErrVote[
// UnexpectedStep | InvalidIndex | InvalidAddress |
// InvalidSignature | InvalidBlockHash | ConflictingVotes ]
// Duplicate votes return added=false, err=nil.
// Conflicting votes return added=*, err=ErrVoteConflictingVotes.
// NOTE: vote should not be mutated after adding.
func (voteSet *VoteSet) AddByIndex(valIndex int, vote *Vote) (added bool, address []byte, err error) {
func (voteSet *VoteSet) AddVote(vote *Vote) (added bool, err error) {
voteSet.mtx.Lock()
defer voteSet.mtx.Unlock()
return voteSet.addByIndex(valIndex, vote)
return voteSet.addVote(vote)
}
// Returns added=true, index if vote was added
// Otherwise returns err=ErrVote[UnexpectedStep|InvalidAccount|InvalidSignature|InvalidBlockHash|ConflictingSignature]
// Duplicate votes return added=false, err=nil.
// NOTE: vote should not be mutated after adding.
func (voteSet *VoteSet) AddByAddress(address []byte, vote *Vote) (added bool, index int, err error) {
voteSet.mtx.Lock()
defer voteSet.mtx.Unlock()
// NOTE: Validates as much as possible before attempting to verify the signature.
func (voteSet *VoteSet) addVote(vote *Vote) (added bool, err error) {
valIndex := vote.ValidatorIndex
valAddr := vote.ValidatorAddress
blockKey := vote.BlockID.Key()
// Ensure that signer is a validator.
valIndex, val := voteSet.valSet.GetByAddress(address)
if val == nil {
return false, 0, ErrVoteInvalidAccount
// Ensure that validator index was set
if valIndex < 0 || len(valAddr) == 0 {
panic("Validator index or address was not set in vote.")
}
return voteSet.addVote(val, valIndex, vote)
}
// Make sure the step matches.
if (vote.Height != voteSet.height) ||
(vote.Round != voteSet.round) ||
(vote.Type != voteSet.type_) {
return false, ErrVoteUnexpectedStep
}
func (voteSet *VoteSet) addByIndex(valIndex int, vote *Vote) (added bool, address []byte, err error) {
// Ensure that signer is a validator.
address, val := voteSet.valSet.GetByIndex(valIndex)
lookupAddr, val := voteSet.valSet.GetByIndex(valIndex)
if val == nil {
return false, nil, ErrVoteInvalidAccount
return false, ErrVoteInvalidValidatorIndex
}
added, _, err = voteSet.addVote(val, valIndex, vote)
return
}
func (voteSet *VoteSet) addVote(val *Validator, valIndex int, vote *Vote) (bool, int, error) {
// Make sure the step matches. (or that vote is commit && round < voteSet.round)
if (vote.Height != voteSet.height) ||
(vote.Round != voteSet.round) ||
(vote.Type != voteSet.type_) {
return false, 0, ErrVoteUnexpectedStep
// Ensure that the signer has the right address
if !bytes.Equal(valAddr, lookupAddr) {
return false, ErrVoteInvalidValidatorAddress
}
// If vote already exists, return false.
if existingVote := voteSet.votes[valIndex]; existingVote != nil {
if bytes.Equal(existingVote.BlockHash, vote.BlockHash) {
return false, valIndex, nil
// If we already know of this vote, return false.
if existing, ok := voteSet.getVote(valIndex, blockKey); ok {
if existing.Signature.Equals(vote.Signature) {
return false, nil // duplicate
} else {
// Check signature.
if !val.PubKey.VerifyBytes(SignBytes(voteSet.chainID, vote), vote.Signature) {
// Bad signature.
return false, 0, ErrVoteInvalidSignature
}
return false, valIndex, &ErrVoteConflictingSignature{
VoteA: existingVote,
VoteB: vote,
}
return false, ErrVoteInvalidSignature // NOTE: assumes deterministic signatures
}
}
// Check signature.
if !val.PubKey.VerifyBytes(SignBytes(voteSet.chainID, vote), vote.Signature) {
// Bad signature.
return false, 0, ErrVoteInvalidSignature
return false, ErrVoteInvalidSignature
}
// Add vote.
voteSet.votes[valIndex] = vote
voteSet.votesBitArray.SetIndex(valIndex, true)
blockKey := string(vote.BlockHash) + string(wire.BinaryBytes(vote.BlockPartsHeader))
totalBlockHashVotes := voteSet.votesByBlock[blockKey] + val.VotingPower
voteSet.votesByBlock[blockKey] = totalBlockHashVotes
voteSet.totalVotes += val.VotingPower
// Add vote and get conflicting vote if any
added, conflicting := voteSet.addVerifiedVote(vote, blockKey, val.VotingPower)
if conflicting != nil {
return added, &ErrVoteConflictingVotes{
VoteA: conflicting,
VoteB: vote,
}
} else {
if !added {
PanicSanity("Expected to add non-conflicting vote")
}
return added, nil
}
}
// If we just nudged it up to two thirds majority, add it.
if totalBlockHashVotes > voteSet.valSet.TotalVotingPower()*2/3 &&
(totalBlockHashVotes-val.VotingPower) <= voteSet.valSet.TotalVotingPower()*2/3 {
voteSet.maj23Hash = vote.BlockHash
voteSet.maj23PartsHeader = vote.BlockPartsHeader
voteSet.maj23Exists = true
// Returns (vote, true) if vote exists for valIndex and blockKey
func (voteSet *VoteSet) getVote(valIndex int, blockKey string) (vote *Vote, ok bool) {
if existing := voteSet.votes[valIndex]; existing != nil && existing.BlockID.Key() == blockKey {
return existing, true
}
if existing := voteSet.votesByBlock[blockKey].getByIndex(valIndex); existing != nil {
return existing, true
}
return nil, false
}
// Assumes signature is valid.
// If conflicting vote exists, returns it.
func (voteSet *VoteSet) addVerifiedVote(vote *Vote, blockKey string, votingPower int64) (added bool, conflicting *Vote) {
valIndex := vote.ValidatorIndex
// Already exists in voteSet.votes?
if existing := voteSet.votes[valIndex]; existing != nil {
if existing.BlockID.Equals(vote.BlockID) {
PanicSanity("addVerifiedVote does not expect duplicate votes")
} else {
conflicting = existing
}
// Replace vote if blockKey matches voteSet.maj23.
if voteSet.maj23 != nil && voteSet.maj23.Key() == blockKey {
voteSet.votes[valIndex] = vote
voteSet.votesBitArray.SetIndex(valIndex, true)
}
// Otherwise don't add it to voteSet.votes
} else {
// Add to voteSet.votes and incr .sum
voteSet.votes[valIndex] = vote
voteSet.votesBitArray.SetIndex(valIndex, true)
voteSet.sum += votingPower
}
votesByBlock, ok := voteSet.votesByBlock[blockKey]
if ok {
if conflicting != nil && !votesByBlock.peerMaj23 {
// There's a conflict and no peer claims that this block is special.
return false, conflicting
}
// We'll add the vote in a bit.
} else {
// .votesByBlock doesn't exist...
if conflicting != nil {
// ... and there's a conflicting vote.
// We're not even tracking this blockKey, so just forget it.
return false, conflicting
} else {
// ... and there's no conflicting vote.
// Start tracking this blockKey
votesByBlock = newBlockVotes(false, voteSet.valSet.Size())
voteSet.votesByBlock[blockKey] = votesByBlock
// We'll add the vote in a bit.
}
}
return true, valIndex, nil
// Before adding to votesByBlock, see if we'll exceed quorum
origSum := votesByBlock.sum
quorum := voteSet.valSet.TotalVotingPower()*2/3 + 1
// Add vote to votesByBlock
votesByBlock.addVerifiedVote(vote, votingPower)
// If we just crossed the quorum threshold and have 2/3 majority...
if origSum < quorum && quorum <= votesByBlock.sum {
// Only consider the first quorum reached
if voteSet.maj23 == nil {
maj23BlockID := vote.BlockID
voteSet.maj23 = &maj23BlockID
// And also copy votes over to voteSet.votes
for i, vote := range votesByBlock.votes {
if vote != nil {
voteSet.votes[i] = vote
}
}
}
}
return true, conflicting
}
// If a peer claims that it has 2/3 majority for given blockKey, call this.
// 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 (voteSet *VoteSet) SetPeerMaj23(peerID string, blockID BlockID) {
voteSet.mtx.Lock()
defer voteSet.mtx.Unlock()
blockKey := blockID.Key()
// Make sure peer hasn't already told us something.
if existing, ok := voteSet.peerMaj23s[peerID]; ok {
if existing.Equals(blockID) {
return // Nothing to do
} else {
return // TODO bad peer!
}
}
voteSet.peerMaj23s[peerID] = blockID
// Create .votesByBlock entry if needed.
votesByBlock, ok := voteSet.votesByBlock[blockKey]
if ok {
if votesByBlock.peerMaj23 {
return // Nothing to do
} else {
votesByBlock.peerMaj23 = true
// No need to copy votes, already there.
}
} else {
votesByBlock = newBlockVotes(true, voteSet.valSet.Size())
voteSet.votesByBlock[blockKey] = votesByBlock
// No need to copy votes, no votes to copy over.
}
}
func (voteSet *VoteSet) BitArray() *BitArray {
@ -185,6 +317,20 @@ func (voteSet *VoteSet) BitArray() *BitArray {
return voteSet.votesBitArray.Copy()
}
func (voteSet *VoteSet) BitArrayByBlockID(blockID BlockID) *BitArray {
if voteSet == nil {
return nil
}
voteSet.mtx.Lock()
defer voteSet.mtx.Unlock()
votesByBlock, ok := voteSet.votesByBlock[blockID.Key()]
if ok {
return votesByBlock.bitArray
}
return nil
}
// NOTE: if validator has conflicting votes, returns "canonical" vote
func (voteSet *VoteSet) GetByIndex(valIndex int) *Vote {
voteSet.mtx.Lock()
defer voteSet.mtx.Unlock()
@ -207,7 +353,7 @@ func (voteSet *VoteSet) HasTwoThirdsMajority() bool {
}
voteSet.mtx.Lock()
defer voteSet.mtx.Unlock()
return voteSet.maj23Exists
return voteSet.maj23 != nil
}
func (voteSet *VoteSet) IsCommit() bool {
@ -219,7 +365,7 @@ func (voteSet *VoteSet) IsCommit() bool {
}
voteSet.mtx.Lock()
defer voteSet.mtx.Unlock()
return len(voteSet.maj23Hash) > 0
return voteSet.maj23 != nil
}
func (voteSet *VoteSet) HasTwoThirdsAny() bool {
@ -228,18 +374,18 @@ func (voteSet *VoteSet) HasTwoThirdsAny() bool {
}
voteSet.mtx.Lock()
defer voteSet.mtx.Unlock()
return voteSet.totalVotes > voteSet.valSet.TotalVotingPower()*2/3
return voteSet.sum > voteSet.valSet.TotalVotingPower()*2/3
}
// Returns either a blockhash (or nil) that received +2/3 majority.
// If there exists no such majority, returns (nil, false).
func (voteSet *VoteSet) TwoThirdsMajority() (hash []byte, parts PartSetHeader, ok bool) {
// If there exists no such majority, returns (nil, PartSetHeader{}, false).
func (voteSet *VoteSet) TwoThirdsMajority() (blockID BlockID, ok bool) {
voteSet.mtx.Lock()
defer voteSet.mtx.Unlock()
if voteSet.maj23Exists {
return voteSet.maj23Hash, voteSet.maj23PartsHeader, true
if voteSet.maj23 != nil {
return *voteSet.maj23, true
} else {
return nil, PartSetHeader{}, false
return BlockID{}, false
}
}
@ -263,10 +409,12 @@ func (voteSet *VoteSet) StringIndented(indent string) string {
%s H:%v R:%v T:%v
%s %v
%s %v
%s %v
%s}`,
indent, voteSet.height, voteSet.round, voteSet.type_,
indent, strings.Join(voteStrings, "\n"+indent+" "),
indent, voteSet.votesBitArray,
indent, voteSet.peerMaj23s,
indent)
}
@ -276,8 +424,8 @@ func (voteSet *VoteSet) StringShort() string {
}
voteSet.mtx.Lock()
defer voteSet.mtx.Unlock()
return fmt.Sprintf(`VoteSet{H:%v R:%v T:%v +2/3:%v %v}`,
voteSet.height, voteSet.round, voteSet.type_, voteSet.maj23Exists, voteSet.votesBitArray)
return fmt.Sprintf(`VoteSet{H:%v R:%v T:%v +2/3:%v %v %v}`,
voteSet.height, voteSet.round, voteSet.type_, voteSet.maj23, voteSet.votesBitArray, voteSet.peerMaj23s)
}
//--------------------------------------------------------------------------------
@ -289,30 +437,63 @@ func (voteSet *VoteSet) MakeCommit() *Commit {
}
voteSet.mtx.Lock()
defer voteSet.mtx.Unlock()
if len(voteSet.maj23Hash) == 0 {
// Make sure we have a 2/3 majority
if voteSet.maj23 == nil {
PanicSanity("Cannot MakeCommit() unless a blockhash has +2/3")
}
precommits := make([]*Vote, voteSet.valSet.Size())
voteSet.valSet.Iterate(func(valIndex int, val *Validator) bool {
vote := voteSet.votes[valIndex]
if vote == nil {
return false
}
if !bytes.Equal(vote.BlockHash, voteSet.maj23Hash) {
return false
}
if !vote.BlockPartsHeader.Equals(voteSet.maj23PartsHeader) {
return false
}
precommits[valIndex] = vote
return false
})
// For every validator, get the precommit
votesCopy := make([]*Vote, len(voteSet.votes))
copy(votesCopy, voteSet.votes)
return &Commit{
Precommits: precommits,
BlockID: *voteSet.maj23,
Precommits: votesCopy,
}
}
//----------------------------------------
//--------------------------------------------------------------------------------
/*
Votes for a particular block
There are two ways a *blockVotes gets created for a blockKey.
1. first (non-conflicting) vote of a validator w/ blockKey (peerMaj23=false)
2. A peer claims to have a 2/3 majority w/ blockKey (peerMaj23=true)
*/
type blockVotes struct {
peerMaj23 bool // peer claims to have maj23
bitArray *BitArray // valIndex -> hasVote?
votes []*Vote // valIndex -> *Vote
sum int64 // vote sum
}
func newBlockVotes(peerMaj23 bool, numValidators int) *blockVotes {
return &blockVotes{
peerMaj23: peerMaj23,
bitArray: NewBitArray(numValidators),
votes: make([]*Vote, numValidators),
sum: 0,
}
}
func (vs *blockVotes) addVerifiedVote(vote *Vote, votingPower int64) {
valIndex := vote.ValidatorIndex
if existing := vs.votes[valIndex]; existing == nil {
vs.bitArray.SetIndex(valIndex, true)
vs.votes[valIndex] = vote
vs.sum += votingPower
}
}
func (vs *blockVotes) getByIndex(index int) *Vote {
if vs == nil {
return nil
}
return vs.votes[index]
}
//--------------------------------------------------------------------------------
// Common interface between *consensus.VoteSet and types.Commit
type VoteSetReader interface {
Height() int


+ 248
- 49
types/vote_set_test.go View File

@ -10,12 +10,21 @@ import (
"testing"
)
// Move it out?
// NOTE: privValidators are in order
// TODO: Move it out?
func randVoteSet(height int, round int, type_ byte, numValidators int, votingPower int64) (*VoteSet, *ValidatorSet, []*PrivValidator) {
valSet, privValidators := RandValidatorSet(numValidators, votingPower)
return NewVoteSet("test_chain_id", height, round, type_, valSet), valSet, privValidators
}
// Convenience: Return new vote with different validator address/index
func withValidator(vote *Vote, addr []byte, idx int) *Vote {
vote = vote.Copy()
vote.ValidatorAddress = addr
vote.ValidatorIndex = idx
return vote
}
// Convenience: Return new vote with different height
func withHeight(vote *Vote, height int) *Vote {
vote = vote.Copy()
@ -40,20 +49,20 @@ func withType(vote *Vote, type_ byte) *Vote {
// Convenience: Return new vote with different blockHash
func withBlockHash(vote *Vote, blockHash []byte) *Vote {
vote = vote.Copy()
vote.BlockHash = blockHash
vote.BlockID.Hash = blockHash
return vote
}
// Convenience: Return new vote with different blockParts
func withBlockPartsHeader(vote *Vote, blockPartsHeader PartSetHeader) *Vote {
vote = vote.Copy()
vote.BlockPartsHeader = blockPartsHeader
vote.BlockID.PartsHeader = blockPartsHeader
return vote
}
func signAddVote(privVal *PrivValidator, vote *Vote, voteSet *VoteSet) (bool, error) {
vote.Signature = privVal.Sign(SignBytes(voteSet.ChainID(), vote)).(crypto.SignatureEd25519)
added, _, err := voteSet.AddByAddress(privVal.Address, vote)
added, err := voteSet.AddVote(vote)
return added, err
}
@ -70,12 +79,19 @@ func TestAddVote(t *testing.T) {
if voteSet.BitArray().GetIndex(0) {
t.Errorf("Expected BitArray.GetIndex(0) to be false")
}
hash, header, ok := voteSet.TwoThirdsMajority()
if hash != nil || !header.IsZero() || ok {
blockID, ok := voteSet.TwoThirdsMajority()
if ok || !blockID.IsZero() {
t.Errorf("There should be no 2/3 majority")
}
vote := &Vote{Height: height, Round: round, Type: VoteTypePrevote, BlockHash: nil}
vote := &Vote{
ValidatorAddress: val0.Address,
ValidatorIndex: 0, // since privValidators are in order
Height: height,
Round: round,
Type: VoteTypePrevote,
BlockID: BlockID{nil, PartSetHeader{}},
}
signAddVote(val0, vote, voteSet)
if voteSet.GetByAddress(val0.Address) == nil {
@ -84,8 +100,8 @@ func TestAddVote(t *testing.T) {
if !voteSet.BitArray().GetIndex(0) {
t.Errorf("Expected BitArray.GetIndex(0) to be true")
}
hash, header, ok = voteSet.TwoThirdsMajority()
if hash != nil || !header.IsZero() || ok {
blockID, ok = voteSet.TwoThirdsMajority()
if ok || !blockID.IsZero() {
t.Errorf("There should be no 2/3 majority")
}
}
@ -94,31 +110,40 @@ func Test2_3Majority(t *testing.T) {
height, round := 1, 0
voteSet, _, privValidators := randVoteSet(height, round, VoteTypePrevote, 10, 1)
vote := &Vote{Height: height, Round: round, Type: VoteTypePrevote, BlockHash: nil}
voteProto := &Vote{
ValidatorAddress: nil, // NOTE: must fill in
ValidatorIndex: -1, // NOTE: must fill in
Height: height,
Round: round,
Type: VoteTypePrevote,
BlockID: BlockID{nil, PartSetHeader{}},
}
// 6 out of 10 voted for nil.
for i := 0; i < 6; i++ {
vote := withValidator(voteProto, privValidators[i].Address, i)
signAddVote(privValidators[i], vote, voteSet)
}
hash, header, ok := voteSet.TwoThirdsMajority()
if hash != nil || !header.IsZero() || ok {
blockID, ok := voteSet.TwoThirdsMajority()
if ok || !blockID.IsZero() {
t.Errorf("There should be no 2/3 majority")
}
// 7th validator voted for some blockhash
{
vote := withValidator(voteProto, privValidators[6].Address, 6)
signAddVote(privValidators[6], withBlockHash(vote, RandBytes(32)), voteSet)
hash, header, ok = voteSet.TwoThirdsMajority()
if hash != nil || !header.IsZero() || ok {
blockID, ok = voteSet.TwoThirdsMajority()
if ok || !blockID.IsZero() {
t.Errorf("There should be no 2/3 majority")
}
}
// 8th validator voted for nil.
{
vote := withValidator(voteProto, privValidators[7].Address, 7)
signAddVote(privValidators[7], vote, voteSet)
hash, header, ok = voteSet.TwoThirdsMajority()
if hash != nil || !header.IsZero() || !ok {
blockID, ok = voteSet.TwoThirdsMajority()
if !ok || !blockID.IsZero() {
t.Errorf("There should be 2/3 majority for nil")
}
}
@ -132,60 +157,73 @@ func Test2_3MajorityRedux(t *testing.T) {
blockPartsTotal := 123
blockPartsHeader := PartSetHeader{blockPartsTotal, crypto.CRandBytes(32)}
vote := &Vote{Height: height, Round: round, Type: VoteTypePrevote, BlockHash: blockHash, BlockPartsHeader: blockPartsHeader}
voteProto := &Vote{
ValidatorAddress: nil, // NOTE: must fill in
ValidatorIndex: -1, // NOTE: must fill in
Height: height,
Round: round,
Type: VoteTypePrevote,
BlockID: BlockID{blockHash, blockPartsHeader},
}
// 66 out of 100 voted for nil.
for i := 0; i < 66; i++ {
vote := withValidator(voteProto, privValidators[i].Address, i)
signAddVote(privValidators[i], vote, voteSet)
}
hash, header, ok := voteSet.TwoThirdsMajority()
if hash != nil || !header.IsZero() || ok {
blockID, ok := voteSet.TwoThirdsMajority()
if ok || !blockID.IsZero() {
t.Errorf("There should be no 2/3 majority")
}
// 67th validator voted for nil
{
vote := withValidator(voteProto, privValidators[66].Address, 66)
signAddVote(privValidators[66], withBlockHash(vote, nil), voteSet)
hash, header, ok = voteSet.TwoThirdsMajority()
if hash != nil || !header.IsZero() || ok {
blockID, ok = voteSet.TwoThirdsMajority()
if ok || !blockID.IsZero() {
t.Errorf("There should be no 2/3 majority: last vote added was nil")
}
}
// 68th validator voted for a different BlockParts PartSetHeader
{
vote := withValidator(voteProto, privValidators[67].Address, 67)
blockPartsHeader := PartSetHeader{blockPartsTotal, crypto.CRandBytes(32)}
signAddVote(privValidators[67], withBlockPartsHeader(vote, blockPartsHeader), voteSet)
hash, header, ok = voteSet.TwoThirdsMajority()
if hash != nil || !header.IsZero() || ok {
blockID, ok = voteSet.TwoThirdsMajority()
if ok || !blockID.IsZero() {
t.Errorf("There should be no 2/3 majority: last vote added had different PartSetHeader Hash")
}
}
// 69th validator voted for different BlockParts Total
{
vote := withValidator(voteProto, privValidators[68].Address, 68)
blockPartsHeader := PartSetHeader{blockPartsTotal + 1, blockPartsHeader.Hash}
signAddVote(privValidators[68], withBlockPartsHeader(vote, blockPartsHeader), voteSet)
hash, header, ok = voteSet.TwoThirdsMajority()
if hash != nil || !header.IsZero() || ok {
blockID, ok = voteSet.TwoThirdsMajority()
if ok || !blockID.IsZero() {
t.Errorf("There should be no 2/3 majority: last vote added had different PartSetHeader Total")
}
}
// 70th validator voted for different BlockHash
{
vote := withValidator(voteProto, privValidators[69].Address, 69)
signAddVote(privValidators[69], withBlockHash(vote, RandBytes(32)), voteSet)
hash, header, ok = voteSet.TwoThirdsMajority()
if hash != nil || !header.IsZero() || ok {
blockID, ok = voteSet.TwoThirdsMajority()
if ok || !blockID.IsZero() {
t.Errorf("There should be no 2/3 majority: last vote added had different BlockHash")
}
}
// 71st validator voted for the right BlockHash & BlockPartsHeader
{
vote := withValidator(voteProto, privValidators[70].Address, 70)
signAddVote(privValidators[70], vote, voteSet)
hash, header, ok = voteSet.TwoThirdsMajority()
if !bytes.Equal(hash, blockHash) || !header.Equals(blockPartsHeader) || !ok {
blockID, ok = voteSet.TwoThirdsMajority()
if !ok || !blockID.Equals(BlockID{blockHash, blockPartsHeader}) {
t.Errorf("There should be 2/3 majority")
}
}
@ -195,36 +233,188 @@ func TestBadVotes(t *testing.T) {
height, round := 1, 0
voteSet, _, privValidators := randVoteSet(height, round, VoteTypePrevote, 10, 1)
voteProto := &Vote{
ValidatorAddress: nil,
ValidatorIndex: -1,
Height: height,
Round: round,
Type: VoteTypePrevote,
BlockID: BlockID{nil, PartSetHeader{}},
}
// val0 votes for nil.
vote := &Vote{Height: height, Round: round, Type: VoteTypePrevote, BlockHash: nil}
added, err := signAddVote(privValidators[0], vote, voteSet)
if !added || err != nil {
t.Errorf("Expected VoteSet.Add to succeed")
{
vote := withValidator(voteProto, privValidators[0].Address, 0)
added, err := signAddVote(privValidators[0], vote, voteSet)
if !added || err != nil {
t.Errorf("Expected VoteSet.Add to succeed")
}
}
// val0 votes again for some block.
added, err = signAddVote(privValidators[0], withBlockHash(vote, RandBytes(32)), voteSet)
if added || err == nil {
t.Errorf("Expected VoteSet.Add to fail, dupeout.")
{
vote := withValidator(voteProto, privValidators[0].Address, 0)
added, err := signAddVote(privValidators[0], withBlockHash(vote, RandBytes(32)), voteSet)
if added || err == nil {
t.Errorf("Expected VoteSet.Add to fail, conflicting vote.")
}
}
// val1 votes on another height
added, err = signAddVote(privValidators[1], withHeight(vote, height+1), voteSet)
if added {
t.Errorf("Expected VoteSet.Add to fail, wrong height")
{
vote := withValidator(voteProto, privValidators[1].Address, 1)
added, err := signAddVote(privValidators[1], withHeight(vote, height+1), voteSet)
if added || err == nil {
t.Errorf("Expected VoteSet.Add to fail, wrong height")
}
}
// val2 votes on another round
added, err = signAddVote(privValidators[2], withRound(vote, round+1), voteSet)
if added {
t.Errorf("Expected VoteSet.Add to fail, wrong round")
{
vote := withValidator(voteProto, privValidators[2].Address, 2)
added, err := signAddVote(privValidators[2], withRound(vote, round+1), voteSet)
if added || err == nil {
t.Errorf("Expected VoteSet.Add to fail, wrong round")
}
}
// val3 votes of another type.
added, err = signAddVote(privValidators[3], withType(vote, VoteTypePrecommit), voteSet)
if added {
t.Errorf("Expected VoteSet.Add to fail, wrong type")
{
vote := withValidator(voteProto, privValidators[3].Address, 3)
added, err := signAddVote(privValidators[3], withType(vote, VoteTypePrecommit), voteSet)
if added || err == nil {
t.Errorf("Expected VoteSet.Add to fail, wrong type")
}
}
}
func TestConflicts(t *testing.T) {
height, round := 1, 0
voteSet, _, privValidators := randVoteSet(height, round, VoteTypePrevote, 4, 1)
blockHash1 := RandBytes(32)
blockHash2 := RandBytes(32)
voteProto := &Vote{
ValidatorAddress: nil,
ValidatorIndex: -1,
Height: height,
Round: round,
Type: VoteTypePrevote,
BlockID: BlockID{nil, PartSetHeader{}},
}
// val0 votes for nil.
{
vote := withValidator(voteProto, privValidators[0].Address, 0)
added, err := signAddVote(privValidators[0], vote, voteSet)
if !added || err != nil {
t.Errorf("Expected VoteSet.Add to succeed")
}
}
// val0 votes again for blockHash1.
{
vote := withValidator(voteProto, privValidators[0].Address, 0)
added, err := signAddVote(privValidators[0], withBlockHash(vote, blockHash1), voteSet)
if added {
t.Errorf("Expected VoteSet.Add to fail, conflicting vote.")
}
if err == nil {
t.Errorf("Expected VoteSet.Add to return error, conflicting vote.")
}
}
// start tracking blockHash1
voteSet.SetPeerMaj23("peerA", BlockID{blockHash1, PartSetHeader{}})
// val0 votes again for blockHash1.
{
vote := withValidator(voteProto, privValidators[0].Address, 0)
added, err := signAddVote(privValidators[0], withBlockHash(vote, blockHash1), voteSet)
if !added {
t.Errorf("Expected VoteSet.Add to succeed, called SetPeerMaj23().")
}
if err == nil {
t.Errorf("Expected VoteSet.Add to return error, conflicting vote.")
}
}
// attempt tracking blockHash2, should fail because already set for peerA.
voteSet.SetPeerMaj23("peerA", BlockID{blockHash2, PartSetHeader{}})
// val0 votes again for blockHash1.
{
vote := withValidator(voteProto, privValidators[0].Address, 0)
added, err := signAddVote(privValidators[0], withBlockHash(vote, blockHash2), voteSet)
if added {
t.Errorf("Expected VoteSet.Add to fail, duplicate SetPeerMaj23() from peerA")
}
if err == nil {
t.Errorf("Expected VoteSet.Add to return error, conflicting vote.")
}
}
// val1 votes for blockHash1.
{
vote := withValidator(voteProto, privValidators[1].Address, 1)
added, err := signAddVote(privValidators[1], withBlockHash(vote, blockHash1), voteSet)
if !added || err != nil {
t.Errorf("Expected VoteSet.Add to succeed")
}
}
// check
if voteSet.HasTwoThirdsMajority() {
t.Errorf("We shouldn't have 2/3 majority yet")
}
if voteSet.HasTwoThirdsAny() {
t.Errorf("We shouldn't have 2/3 if any votes yet")
}
// val2 votes for blockHash2.
{
vote := withValidator(voteProto, privValidators[2].Address, 2)
added, err := signAddVote(privValidators[2], withBlockHash(vote, blockHash2), voteSet)
if !added || err != nil {
t.Errorf("Expected VoteSet.Add to succeed")
}
}
// check
if voteSet.HasTwoThirdsMajority() {
t.Errorf("We shouldn't have 2/3 majority yet")
}
if !voteSet.HasTwoThirdsAny() {
t.Errorf("We should have 2/3 if any votes")
}
// now attempt tracking blockHash1
voteSet.SetPeerMaj23("peerB", BlockID{blockHash1, PartSetHeader{}})
// val2 votes for blockHash1.
{
vote := withValidator(voteProto, privValidators[2].Address, 2)
added, err := signAddVote(privValidators[2], withBlockHash(vote, blockHash1), voteSet)
if !added {
t.Errorf("Expected VoteSet.Add to succeed")
}
if err == nil {
t.Errorf("Expected VoteSet.Add to return error, conflicting vote")
}
}
// check
if !voteSet.HasTwoThirdsMajority() {
t.Errorf("We should have 2/3 majority for blockHash1")
}
blockIDMaj23, _ := voteSet.TwoThirdsMajority()
if !bytes.Equal(blockIDMaj23.Hash, blockHash1) {
t.Errorf("Got the wrong 2/3 majority blockhash")
}
if !voteSet.HasTwoThirdsAny() {
t.Errorf("We should have 2/3 if any votes")
}
}
func TestMakeCommit(t *testing.T) {
@ -232,11 +422,18 @@ func TestMakeCommit(t *testing.T) {
voteSet, _, privValidators := randVoteSet(height, round, VoteTypePrecommit, 10, 1)
blockHash, blockPartsHeader := crypto.CRandBytes(32), PartSetHeader{123, crypto.CRandBytes(32)}
vote := &Vote{Height: height, Round: round, Type: VoteTypePrecommit,
BlockHash: blockHash, BlockPartsHeader: blockPartsHeader}
voteProto := &Vote{
ValidatorAddress: nil,
ValidatorIndex: -1,
Height: height,
Round: round,
Type: VoteTypePrecommit,
BlockID: BlockID{blockHash, blockPartsHeader},
}
// 6 out of 10 voted for some block.
for i := 0; i < 6; i++ {
vote := withValidator(voteProto, privValidators[i].Address, i)
signAddVote(privValidators[i], vote, voteSet)
}
@ -245,13 +442,15 @@ func TestMakeCommit(t *testing.T) {
// 7th voted for some other block.
{
vote := withBlockHash(vote, RandBytes(32))
vote := withValidator(voteProto, privValidators[6].Address, 6)
vote = withBlockHash(vote, RandBytes(32))
vote = withBlockPartsHeader(vote, PartSetHeader{123, RandBytes(32)})
signAddVote(privValidators[6], vote, voteSet)
}
// The 8th voted like everyone else.
{
vote := withValidator(voteProto, privValidators[7].Address, 7)
signAddVote(privValidators[7], vote, voteSet)
}


+ 30
- 0
types/vote_test.go View File

@ -0,0 +1,30 @@
package types
import (
"testing"
)
func TestVoteSignable(t *testing.T) {
vote := &Vote{
ValidatorAddress: []byte("addr"),
ValidatorIndex: 56789,
Height: 12345,
Round: 23456,
Type: byte(2),
BlockID: BlockID{
Hash: []byte("hash"),
PartsHeader: PartSetHeader{
Total: 1000000,
Hash: []byte("parts_hash"),
},
},
}
signBytes := SignBytes("test_chain_id", vote)
signStr := string(signBytes)
expected := `{"chain_id":"test_chain_id","vote":{"block_id":{"hash":"68617368","parts":{"hash":"70617274735F68617368","total":1000000}},"height":12345,"round":23456,"type":2}}`
if signStr != expected {
// NOTE: when this fails, you probably want to fix up consensus/replay_test too
t.Errorf("Got unexpected sign string for Vote. Expected:\n%v\nGot:\n%v", expected, signStr)
}
}

Loading…
Cancel
Save