tendermint/consensus/reactor.go

952 lines
27 KiB
Go
Raw Normal View History

2014-08-10 16:35:08 -07:00
package consensus
import (
"bytes"
"errors"
"fmt"
"reflect"
2014-08-10 16:35:08 -07:00
"sync"
"sync/atomic"
"time"
2015-04-01 17:30:16 -07:00
"github.com/tendermint/tendermint/binary"
bc "github.com/tendermint/tendermint/blockchain"
. "github.com/tendermint/tendermint/common"
. "github.com/tendermint/tendermint/consensus/types"
"github.com/tendermint/tendermint/events"
2015-04-01 17:30:16 -07:00
"github.com/tendermint/tendermint/p2p"
sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/types"
2014-08-10 16:35:08 -07:00
)
const (
StateChannel = byte(0x20)
DataChannel = byte(0x21)
VoteChannel = byte(0x22)
2014-09-14 15:37:32 -07:00
PeerStateKey = "ConsensusReactor.peerState"
2014-08-10 16:35:08 -07:00
peerGossipSleepDuration = 100 * time.Millisecond // Time to sleep if there's nothing to send.
2014-08-10 16:35:08 -07:00
)
//-----------------------------------------------------------------------------
// The reactor's underlying ConsensusState may change state at any time.
// We atomically copy the RoundState struct before using it.
2014-09-14 15:37:32 -07:00
type ConsensusReactor struct {
sw *p2p.Switch
running uint32
2014-10-30 03:32:09 -07:00
quit chan struct{}
2014-08-10 16:35:08 -07:00
blockStore *bc.BlockStore
conS *ConsensusState
// if fast sync is running we don't really do anything
sync bool
2015-04-15 23:40:27 -07:00
evsw events.Fireable
2014-08-10 16:35:08 -07:00
}
func NewConsensusReactor(consensusState *ConsensusState, blockStore *bc.BlockStore, sync bool) *ConsensusReactor {
2014-09-14 15:37:32 -07:00
conR := &ConsensusReactor{
quit: make(chan struct{}),
2015-06-19 15:30:10 -07:00
blockStore: blockStore,
2015-01-11 14:27:46 -08:00
conS: consensusState,
sync: sync,
2014-08-10 16:35:08 -07:00
}
2014-09-14 15:37:32 -07:00
return conR
2014-08-10 16:35:08 -07:00
}
2014-10-22 17:20:44 -07:00
// Implements Reactor
func (conR *ConsensusReactor) Start(sw *p2p.Switch) {
if atomic.CompareAndSwapUint32(&conR.running, 0, 1) {
2014-09-14 15:37:32 -07:00
log.Info("Starting ConsensusReactor")
2014-10-22 17:20:44 -07:00
conR.sw = sw
if !conR.sync {
conR.conS.Start()
}
2014-10-30 03:32:09 -07:00
go conR.broadcastNewRoundStepRoutine()
2014-09-14 15:37:32 -07:00
}
}
2014-10-22 17:20:44 -07:00
// Implements Reactor
2014-09-14 15:37:32 -07:00
func (conR *ConsensusReactor) Stop() {
if atomic.CompareAndSwapUint32(&conR.running, 1, 0) {
2014-09-14 15:37:32 -07:00
log.Info("Stopping ConsensusReactor")
2014-10-30 03:32:09 -07:00
conR.conS.Stop()
2014-09-14 15:37:32 -07:00
close(conR.quit)
2014-08-10 16:35:08 -07:00
}
}
func (conR *ConsensusReactor) IsRunning() bool {
return atomic.LoadUint32(&conR.running) == 1
2014-10-30 03:32:09 -07:00
}
2014-09-14 15:37:32 -07:00
// Implements Reactor
func (conR *ConsensusReactor) GetChannels() []*p2p.ChannelDescriptor {
// TODO optimize
return []*p2p.ChannelDescriptor{
&p2p.ChannelDescriptor{
2015-05-05 17:03:11 -07:00
Id: StateChannel,
Priority: 5,
SendQueueCapacity: 100,
2014-09-14 15:37:32 -07:00
},
&p2p.ChannelDescriptor{
2015-05-05 17:03:11 -07:00
Id: DataChannel,
Priority: 5,
SendQueueCapacity: 2,
2014-09-14 15:37:32 -07:00
},
&p2p.ChannelDescriptor{
2015-05-05 17:03:11 -07:00
Id: VoteChannel,
Priority: 5,
SendQueueCapacity: 40,
2014-09-14 15:37:32 -07:00
},
2014-08-10 16:35:08 -07:00
}
}
2014-09-14 15:37:32 -07:00
// Implements Reactor
func (conR *ConsensusReactor) AddPeer(peer *p2p.Peer) {
if !conR.IsRunning() {
return
}
2014-09-14 15:37:32 -07:00
// Create peerState for peer
peerState := NewPeerState(peer)
peer.Data.Set(PeerStateKey, peerState)
2014-09-14 15:37:32 -07:00
// Begin gossip routines for this peer.
go conR.gossipDataRoutine(peer, peerState)
go conR.gossipVotesRoutine(peer, peerState)
// Send our state to peer.
2015-06-19 15:30:10 -07:00
conR.sendNewRoundStepMessage(peer)
2014-08-10 16:35:08 -07:00
}
2014-09-14 15:37:32 -07:00
// Implements Reactor
func (conR *ConsensusReactor) RemovePeer(peer *p2p.Peer, reason interface{}) {
if !conR.IsRunning() {
return
}
//peer.Data.Get(PeerStateKey).(*PeerState).Disconnect()
2014-08-10 16:35:08 -07:00
}
2014-09-14 15:37:32 -07:00
// Implements Reactor
func (conR *ConsensusReactor) Receive(chId byte, peer *p2p.Peer, msgBytes []byte) {
if conR.sync || !conR.IsRunning() {
return
}
2014-08-10 16:35:08 -07:00
2014-09-14 15:37:32 -07:00
// Get round state
rs := conR.conS.GetRoundState()
ps := peer.Data.Get(PeerStateKey).(*PeerState)
_, msg_, err := DecodeMessage(msgBytes)
2014-12-29 15:14:54 -08:00
if err != nil {
2014-12-29 18:09:06 -08:00
log.Warn("Error decoding message", "channel", chId, "peer", peer, "msg", msg_, "error", err, "bytes", msgBytes)
2014-12-29 15:14:54 -08:00
return
}
2015-04-20 18:51:20 -07:00
log.Debug("Receive", "channel", chId, "peer", peer, "msg", msg_) //, "bytes", msgBytes)
2014-10-25 14:27:53 -07:00
2014-09-14 15:37:32 -07:00
switch chId {
case StateChannel:
2015-03-22 03:29:28 -07:00
switch msg := msg_.(type) {
2014-09-14 15:37:32 -07:00
case *NewRoundStepMessage:
ps.ApplyNewRoundStepMessage(msg, rs)
case *CommitStepMessage:
ps.ApplyCommitStepMessage(msg)
case *HasVoteMessage:
ps.ApplyHasVoteMessage(msg)
2014-09-14 15:37:32 -07:00
default:
log.Warn(Fmt("Unknown message type %v", reflect.TypeOf(msg)))
2014-09-14 15:37:32 -07:00
}
2014-08-10 16:35:08 -07:00
case DataChannel:
2015-03-22 03:29:28 -07:00
switch msg := msg_.(type) {
case *ProposalMessage:
ps.SetHasProposal(msg.Proposal)
err = conR.conS.SetProposal(msg.Proposal)
2015-06-22 19:04:31 -07:00
case *ProposalPOLMessage:
ps.ApplyProposalPOLMessage(msg)
case *BlockPartMessage:
ps.SetHasProposalBlockPart(msg.Height, msg.Round, msg.Part.Proof.Index)
_, err = conR.conS.AddProposalBlockPart(msg.Height, msg.Round, msg.Part)
2014-09-14 15:37:32 -07:00
default:
log.Warn(Fmt("Unknown message type %v", reflect.TypeOf(msg)))
2014-09-14 15:37:32 -07:00
}
case VoteChannel:
2015-03-22 03:29:28 -07:00
switch msg := msg_.(type) {
case *VoteMessage:
2015-03-22 03:29:28 -07:00
vote := msg.Vote
if rs.Height != vote.Height {
if rs.Height == vote.Height+1 {
2015-06-19 15:30:10 -07:00
if rs.Step == RoundStepNewHeight && vote.Type == types.VoteTypePrecommit {
goto VOTE_PASS // *ducks*
}
}
return // Wrong height. Not necessarily a bad peer.
}
VOTE_PASS:
2015-06-19 15:30:10 -07:00
address, _ := rs.Validators.GetByIndex(msg.ValidatorIndex)
2015-06-24 14:04:40 -07:00
added, index, err := conR.conS.AddVote(address, vote, peer.Key)
2014-09-14 15:37:32 -07:00
if err != nil {
// If conflicting sig, broadcast evidence tx for slashing. Else punish peer.
if errDupe, ok := err.(*types.ErrVoteConflictingSignature); ok {
log.Warn("Found conflicting vote. Publish evidence")
evidenceTx := &types.DupeoutTx{
Address: address,
VoteA: *errDupe.VoteA,
VoteB: *errDupe.VoteB,
}
conR.conS.mempoolReactor.BroadcastTx(evidenceTx) // shouldn't need to check returned err
} else {
// Probably an invalid signature. Bad peer.
log.Warn("Error attempting to add vote", "error", err)
// TODO: punish peer
}
2014-09-14 15:37:32 -07:00
}
2015-06-22 19:04:31 -07:00
ps.EnsureVoteBitArrays(rs.Height, rs.Validators.Size(), nil)
ps.EnsureVoteBitArrays(rs.Height-1, rs.LastCommit.Size(), nil)
ps.SetHasVote(vote, index)
2014-09-14 15:37:32 -07:00
if added {
2015-06-22 19:04:31 -07:00
// If rs.Height == vote.Height && rs.Round < vote.Round,
// the peer is sending us CatchupCommit precommits.
// We could make note of this and help filter in broadcastHasVoteMessage().
conR.broadcastHasVoteMessage(vote, index)
2014-08-10 16:35:08 -07:00
}
default:
log.Warn(Fmt("Unknown message type %v", reflect.TypeOf(msg)))
2014-08-10 16:35:08 -07:00
}
2014-09-14 15:37:32 -07:00
default:
log.Warn(Fmt("Unknown channel %X", chId))
2014-08-10 16:35:08 -07:00
}
2014-09-14 15:37:32 -07:00
if err != nil {
2014-12-29 18:39:19 -08:00
log.Warn("Error in Receive()", "error", err)
2014-09-14 15:37:32 -07:00
}
2014-08-10 16:35:08 -07:00
}
2015-06-22 19:04:31 -07:00
// Broadcasts HasVoteMessage to peers that care.
func (conR *ConsensusReactor) broadcastHasVoteMessage(vote *types.Vote, index uint) {
msg := &HasVoteMessage{
Height: vote.Height,
Round: vote.Round,
Type: vote.Type,
Index: index,
}
conR.sw.Broadcast(StateChannel, msg)
/*
// TODO: Make this broadcast more selective.
for _, peer := range conR.sw.Peers().List() {
ps := peer.Data.Get(PeerStateKey).(*PeerState)
prs := ps.GetRoundState()
if prs.Height == vote.Height {
// TODO: Also filter on round?
peer.TrySend(StateChannel, msg)
} else {
// Height doesn't match
// TODO: check a field, maybe CatchupCommitRound?
// TODO: But that requires changing the struct field comment.
}
}
*/
}
2014-10-22 17:20:44 -07:00
// Sets our private validator account for signing votes.
func (conR *ConsensusReactor) SetPrivValidator(priv *sm.PrivValidator) {
2014-10-22 17:20:44 -07:00
conR.conS.SetPrivValidator(priv)
}
// Switch from the fast sync to the consensus:
// reset the state, turn off fast sync, start the consensus-state-machine
func (conR *ConsensusReactor) SwitchToConsensus(state *sm.State) {
conR.conS.updateToState(state, false)
conR.sync = false
conR.conS.Start()
}
// implements events.Eventable
2015-04-15 23:40:27 -07:00
func (conR *ConsensusReactor) SetFireable(evsw events.Fireable) {
conR.evsw = evsw
2015-04-15 23:40:27 -07:00
conR.conS.SetFireable(evsw)
}
2014-10-21 23:30:18 -07:00
//--------------------------------------
func makeRoundStepMessages(rs *RoundState) (nrsMsg *NewRoundStepMessage, csMsg *CommitStepMessage) {
nrsMsg = &NewRoundStepMessage{
Height: rs.Height,
Round: rs.Round,
Step: rs.Step,
2015-06-22 19:04:31 -07:00
SecondsSinceStartTime: uint(time.Now().Sub(rs.StartTime).Seconds()),
2015-06-19 15:30:10 -07:00
LastCommitRound: rs.LastCommit.Round(),
}
if rs.Step == RoundStepCommit {
csMsg = &CommitStepMessage{
2015-06-22 19:04:31 -07:00
Height: rs.Height,
BlockPartsHeader: rs.ProposalBlockParts.Header(),
BlockParts: rs.ProposalBlockParts.BitArray(),
}
}
return
}
2014-10-30 03:32:09 -07:00
// Listens for changes to the ConsensusState.Step by pulling
// on conR.conS.NewStepCh().
func (conR *ConsensusReactor) broadcastNewRoundStepRoutine() {
for {
2014-10-30 03:32:09 -07:00
// Get RoundState with new Step or quit.
var rs *RoundState
select {
case rs = <-conR.conS.NewStepCh():
case <-conR.quit:
return
}
nrsMsg, csMsg := makeRoundStepMessages(rs)
if nrsMsg != nil {
conR.sw.Broadcast(StateChannel, nrsMsg)
}
if csMsg != nil {
conR.sw.Broadcast(StateChannel, csMsg)
}
}
2014-10-21 23:30:18 -07:00
}
2015-06-19 15:30:10 -07:00
func (conR *ConsensusReactor) sendNewRoundStepMessage(peer *p2p.Peer) {
rs := conR.conS.GetRoundState()
nrsMsg, csMsg := makeRoundStepMessages(rs)
if nrsMsg != nil {
peer.Send(StateChannel, nrsMsg)
}
if csMsg != nil {
2015-04-20 18:51:20 -07:00
peer.Send(StateChannel, csMsg)
}
}
2014-09-14 15:37:32 -07:00
func (conR *ConsensusReactor) gossipDataRoutine(peer *p2p.Peer, ps *PeerState) {
2014-08-10 16:35:08 -07:00
OUTER_LOOP:
for {
2014-09-14 15:37:32 -07:00
// Manage disconnects from self or peer.
if !peer.IsRunning() || !conR.IsRunning() {
2014-12-29 18:09:06 -08:00
log.Info(Fmt("Stopping gossipDataRoutine for %v.", peer))
2014-09-14 15:37:32 -07:00
return
2014-08-10 16:35:08 -07:00
}
2014-09-14 15:37:32 -07:00
rs := conR.conS.GetRoundState()
prs := ps.GetRoundState()
2014-08-10 16:35:08 -07:00
2014-10-26 13:26:27 -07:00
// Send proposal Block parts?
2015-06-22 19:04:31 -07:00
if rs.ProposalBlockParts.HasHeader(prs.ProposalBlockPartsHeader) {
//log.Debug("ProposalBlockParts matched", "blockParts", prs.ProposalBlockParts)
2015-06-22 19:04:31 -07:00
if index, ok := rs.ProposalBlockParts.BitArray().Sub(prs.ProposalBlockParts.Copy()).PickRandom(); ok {
part := rs.ProposalBlockParts.GetPart(index)
2015-06-22 19:04:31 -07:00
msg := &BlockPartMessage{
Height: rs.Height,
Round: rs.Round,
Part: part,
}
peer.Send(DataChannel, msg)
ps.SetHasProposalBlockPart(rs.Height, rs.Round, index)
continue OUTER_LOOP
}
}
// If the peer is on a previous height, help catch up.
2015-06-25 12:52:16 -07:00
if (0 < prs.Height) && (prs.Height < rs.Height) {
2015-06-22 19:04:31 -07:00
//log.Debug("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)
2015-06-22 19:04:31 -07:00
if !blockMeta.PartsHeader.Equals(prs.ProposalBlockPartsHeader) {
log.Debug("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)
if part == nil {
log.Warn("Could not load part", "index", index,
2015-06-22 19:04:31 -07:00
"peerHeight", prs.Height, "blockPartsHeader", blockMeta.PartsHeader, "peerBlockPartsHeader", prs.ProposalBlockPartsHeader)
time.Sleep(peerGossipSleepDuration)
continue OUTER_LOOP
}
// Send the part
2015-06-22 19:04:31 -07:00
msg := &BlockPartMessage{
Height: prs.Height,
Round: prs.Round,
Part: part,
}
peer.Send(DataChannel, msg)
ps.SetHasProposalBlockPart(prs.Height, prs.Round, index)
continue OUTER_LOOP
} else {
//log.Debug("No parts to send in catch-up, sleeping")
time.Sleep(peerGossipSleepDuration)
continue OUTER_LOOP
}
}
// If height and round don't match, sleep.
2014-09-14 15:37:32 -07:00
if rs.Height != prs.Height || rs.Round != prs.Round {
//log.Debug("Peer Height|Round mismatch, sleeping", "peerHeight", prs.Height, "peerRound", prs.Round, "peer", peer)
2014-09-14 15:37:32 -07:00
time.Sleep(peerGossipSleepDuration)
2014-08-10 16:35:08 -07:00
continue OUTER_LOOP
}
2015-06-22 19:04:31 -07:00
// By here, height and round match.
// Send Proposal && ProposalPOL BitArray?
2014-09-14 15:37:32 -07:00
if rs.Proposal != nil && !prs.Proposal {
2015-06-22 19:04:31 -07:00
// Proposal
{
msg := &ProposalMessage{Proposal: rs.Proposal}
peer.Send(DataChannel, msg)
ps.SetHasProposal(rs.Proposal)
}
// ProposalPOL.
// Must be in the same channel, sequential.
// That is, peer must receive ProposalMessage first.
if 0 <= rs.Proposal.POLRound {
msg := &ProposalPOLMessage{
Height: rs.Height,
ProposalPOLRound: uint(rs.Proposal.POLRound),
ProposalPOL: rs.Votes.Prevotes(uint(rs.Proposal.POLRound)).BitArray(),
}
peer.Send(DataChannel, msg)
}
2014-09-14 15:37:32 -07:00
continue OUTER_LOOP
2014-08-10 16:35:08 -07:00
}
2014-09-14 15:37:32 -07:00
// Nothing to do. Sleep.
time.Sleep(peerGossipSleepDuration)
continue OUTER_LOOP
2014-08-10 16:35:08 -07:00
}
}
2014-09-14 15:37:32 -07:00
func (conR *ConsensusReactor) gossipVotesRoutine(peer *p2p.Peer, ps *PeerState) {
2015-05-04 22:21:07 -07:00
// Simple hack to throttle logs upon sleep.
var sleeping = 0
2014-09-14 15:37:32 -07:00
OUTER_LOOP:
for {
// Manage disconnects from self or peer.
if !peer.IsRunning() || !conR.IsRunning() {
2014-12-29 18:09:06 -08:00
log.Info(Fmt("Stopping gossipVotesRoutine for %v.", peer))
2014-09-14 15:37:32 -07:00
return
2014-08-10 16:35:08 -07:00
}
2014-09-14 15:37:32 -07:00
rs := conR.conS.GetRoundState()
prs := ps.GetRoundState()
2015-05-04 22:21:07 -07:00
switch sleeping {
case 1: // First sleep
sleeping = 2
case 2: // No more sleep
sleeping = 0
}
2015-06-19 15:30:10 -07:00
// prsVoteSet: a pointer to a VoteSet field of prs.
2015-05-07 17:35:58 -07:00
// Returns true when useful work was done.
2015-06-19 15:30:10 -07:00
trySendVote := func(voteSet *VoteSet, prsVoteSet **BitArray) (sent bool) {
2015-05-07 17:35:58 -07:00
if voteSet == nil {
return false
2015-06-19 15:30:10 -07:00
}
if *prsVoteSet == nil {
ps.EnsureVoteBitArrays(voteSet.Height(), voteSet.Size(), prs)
// We could return true here (useful work was done)
// or, we can continue since prsVoteSet is no longer nil.
if *prsVoteSet == nil {
panic("prsVoteSet should not be nil after ps.EnsureVoteBitArrays")
}
2015-05-07 17:35:58 -07:00
}
2014-10-30 03:32:09 -07:00
// TODO: give priority to our vote.
2015-06-19 15:30:10 -07:00
if index, ok := voteSet.BitArray().Sub((*prsVoteSet).Copy()).PickRandom(); ok {
2014-10-30 03:32:09 -07:00
vote := voteSet.GetByIndex(index)
msg := &VoteMessage{index, vote}
peer.Send(VoteChannel, msg)
ps.SetHasVote(vote, index)
2014-10-30 03:32:09 -07:00
return true
}
return false
}
2015-06-19 15:30:10 -07:00
// prsVoteSet: a pointer to a VoteSet field of prs.
2015-05-07 17:35:58 -07:00
// Returns true when useful work was done.
2015-06-19 15:30:10 -07:00
trySendPrecommitFromValidation := func(validation *types.Validation, prsVoteSet **BitArray) (sent bool) {
2015-05-07 17:35:58 -07:00
if validation == nil {
return false
2015-06-19 15:30:10 -07:00
} else if *prsVoteSet == nil {
ps.EnsureVoteBitArrays(validation.Height(), uint(len(validation.Precommits)), prs)
// We could return true here (useful work was done)
// or, we can continue since prsVoteSet is no longer nil.
if *prsVoteSet == nil {
panic("prsVoteSet should not be nil after ps.EnsureVoteBitArrays")
}
2015-06-19 15:30:10 -07:00
}
if index, ok := validation.BitArray().Sub((*prsVoteSet).Copy()).PickRandom(); ok {
precommit := validation.Precommits[index]
log.Debug("Picked precommit to send", "index", index, "precommit", precommit)
msg := &VoteMessage{index, precommit}
peer.Send(VoteChannel, msg)
2015-06-19 15:30:10 -07:00
ps.SetHasVote(precommit, index)
return true
}
return false
}
2015-06-19 15:30:10 -07:00
// If height matches, then send LastCommit, Prevotes, Precommits.
if rs.Height == prs.Height {
2015-06-19 15:30:10 -07:00
// If there are lastCommits to send...
if prs.Step == RoundStepNewHeight {
2015-06-22 19:04:31 -07:00
if trySendVote(rs.LastCommit, &prs.LastCommit) {
2015-05-07 17:35:58 -07:00
continue OUTER_LOOP
}
}
// If there are prevotes to send...
if rs.Round == prs.Round && prs.Step <= RoundStepPrevote {
2015-06-22 19:04:31 -07:00
if trySendVote(rs.Votes.Prevotes(rs.Round), &prs.Prevotes) {
continue OUTER_LOOP
}
}
// If there are precommits to send...
if rs.Round == prs.Round && prs.Step <= RoundStepPrecommit {
2015-06-22 19:04:31 -07:00
if trySendVote(rs.Votes.Precommits(rs.Round), &prs.Precommits) {
continue OUTER_LOOP
}
}
2015-06-22 19:04:31 -07:00
// If there are POLPrevotes to send...
if 0 <= prs.ProposalPOLRound {
if polPrevotes := rs.Votes.Prevotes(uint(prs.ProposalPOLRound)); polPrevotes != nil {
if trySendVote(polPrevotes, &prs.ProposalPOL) {
continue OUTER_LOOP
}
}
}
2014-09-14 15:37:32 -07:00
}
2015-06-19 15:30:10 -07:00
// Special catchup logic.
// If peer is lagging by height 1, send LastCommit.
if prs.Height != 0 && prs.Height == rs.Height-1 {
if prs.Round == rs.LastCommit.Round() {
2015-06-22 19:04:31 -07:00
if trySendVote(rs.LastCommit, &prs.Precommits) {
2015-06-19 15:30:10 -07:00
continue OUTER_LOOP
}
2015-06-19 15:30:10 -07:00
} else {
ps.SetCatchupCommitRound(prs.Height, rs.LastCommit.Round())
2015-06-22 19:04:31 -07:00
if trySendVote(rs.LastCommit, &prs.CatchupCommit) {
continue OUTER_LOOP
}
}
2014-09-14 15:37:32 -07:00
}
2014-08-10 16:35:08 -07:00
2015-06-19 15:30:10 -07:00
// Catchup logic
// If peer is lagging by more than 1, send Validation.
if prs.Height != 0 && prs.Height <= rs.Height-2 {
// Load the block validation for prs.Height,
// which contains precommit signatures for prs.Height.
validation := conR.blockStore.LoadBlockValidation(prs.Height)
log.Debug("Loaded BlockValidation for catch-up", "height", prs.Height, "validation", validation)
2015-06-22 19:04:31 -07:00
// Peer's CatchupCommitRound should be -1 or equal to the validation's precommit rounds.
2015-06-19 15:30:10 -07:00
// If not, warn.
2015-06-22 19:04:31 -07:00
if prs.CatchupCommitRound == -1 {
ps.SetCatchupCommitRound(prs.Height, validation.Round())
2015-06-19 15:30:10 -07:00
continue OUTER_LOOP // Get prs := ps.GetRoundState() again.
2015-06-22 19:04:31 -07:00
} else if prs.CatchupCommitRound != int(validation.Round()) {
log.Warn("Peer's CatchupCommitRound during catchup not equal to commit round",
"height", prs.Height, "validation", validation, "prs.CatchupCommitRound", prs.CatchupCommitRound)
} else if trySendPrecommitFromValidation(validation, &prs.CatchupCommit) {
2015-06-19 15:30:10 -07:00
continue OUTER_LOOP
}
}
2015-05-04 22:21:07 -07:00
if sleeping == 0 {
// We sent nothing. Sleep...
sleeping = 1
2015-05-05 18:03:00 -07:00
log.Debug("No votes to send, sleeping", "peer", peer,
2015-06-22 19:04:31 -07:00
"localPV", rs.Votes.Prevotes(rs.Round).BitArray(), "peerPV", prs.Prevotes,
"localPC", rs.Votes.Precommits(rs.Round).BitArray(), "peerPC", prs.Precommits)
2015-05-04 22:21:07 -07:00
} else if sleeping == 2 {
// Continued sleep...
sleeping = 1
}
2014-09-14 15:37:32 -07:00
time.Sleep(peerGossipSleepDuration)
continue OUTER_LOOP
2014-08-10 16:35:08 -07:00
}
2014-09-14 15:37:32 -07:00
}
2014-08-10 16:35:08 -07:00
//-----------------------------------------------------------------------------
2014-09-14 15:37:32 -07:00
// Read only when returned by PeerState.GetRoundState().
type PeerRoundState struct {
2015-06-22 19:04:31 -07:00
Height uint // Height peer is at
Round uint // Round peer is at
Step RoundStepType // Step peer is at
StartTime time.Time // Estimated start of round 0 at this height
Proposal bool // True if peer has proposal for this round
ProposalBlockPartsHeader types.PartSetHeader //
ProposalBlockParts *BitArray //
ProposalPOLRound int // -1 if none
ProposalPOL *BitArray // nil until ProposalPOLMessage received.
Prevotes *BitArray // All votes peer has for this round
Precommits *BitArray // All precommits peer has for this round
LastCommitRound uint // Round of commit for last height.
LastCommit *BitArray // All commit precommits of commit for last height.
CatchupCommitRound int // Round that we believe commit round is.
CatchupCommit *BitArray // All commit precommits peer has for this height
2014-09-14 15:37:32 -07:00
}
//-----------------------------------------------------------------------------
2014-08-10 16:35:08 -07:00
var (
ErrPeerStateHeightRegression = errors.New("Error peer state height regression")
ErrPeerStateInvalidStartTime = errors.New("Error peer state invalid startTime")
)
type PeerState struct {
Key string
2014-09-14 15:37:32 -07:00
mtx sync.Mutex
PeerRoundState
2014-08-10 16:35:08 -07:00
}
func NewPeerState(peer *p2p.Peer) *PeerState {
return &PeerState{Key: peer.Key}
2014-08-10 16:35:08 -07:00
}
2014-09-14 15:37:32 -07:00
// Returns an atomic snapshot of the PeerRoundState.
// There's no point in mutating it since it won't change PeerState.
func (ps *PeerState) GetRoundState() *PeerRoundState {
ps.mtx.Lock()
defer ps.mtx.Unlock()
2014-09-14 15:37:32 -07:00
prs := ps.PeerRoundState // copy
return &prs
}
func (ps *PeerState) SetHasProposal(proposal *Proposal) {
ps.mtx.Lock()
defer ps.mtx.Unlock()
if ps.Height != proposal.Height || ps.Round != proposal.Round {
return
}
if ps.Proposal {
return
}
ps.Proposal = true
2015-06-22 19:04:31 -07:00
ps.ProposalBlockPartsHeader = proposal.BlockPartsHeader
ps.ProposalBlockParts = NewBitArray(uint(proposal.BlockPartsHeader.Total))
ps.ProposalPOLRound = proposal.POLRound
ps.ProposalPOL = nil // Nil until ProposalPOLMessage received.
2014-09-14 15:37:32 -07:00
}
func (ps *PeerState) SetHasProposalBlockPart(height uint, round uint, index uint) {
2014-08-10 16:35:08 -07:00
ps.mtx.Lock()
defer ps.mtx.Unlock()
2014-09-14 15:37:32 -07:00
if ps.Height != height || ps.Round != round {
return
2014-08-10 16:35:08 -07:00
}
2015-06-22 19:04:31 -07:00
ps.ProposalBlockParts.SetIndex(uint(index), true)
}
2015-06-19 15:30:10 -07:00
// prs: If given, will also update this PeerRoundState copy.
func (ps *PeerState) EnsureVoteBitArrays(height uint, numValidators uint, prs *PeerRoundState) {
ps.mtx.Lock()
defer ps.mtx.Unlock()
2015-05-07 17:35:58 -07:00
if ps.Height == height {
if ps.Prevotes == nil {
ps.Prevotes = NewBitArray(numValidators)
}
if ps.Precommits == nil {
ps.Precommits = NewBitArray(numValidators)
}
2015-06-19 15:30:10 -07:00
if ps.CatchupCommit == nil {
ps.CatchupCommit = NewBitArray(numValidators)
2015-05-07 17:35:58 -07:00
}
2015-06-22 19:04:31 -07:00
if ps.ProposalPOL == nil {
ps.ProposalPOL = NewBitArray(numValidators)
}
2015-05-07 17:35:58 -07:00
} else if ps.Height == height+1 {
2015-06-19 15:30:10 -07:00
if ps.LastCommit == nil {
ps.LastCommit = NewBitArray(numValidators)
2015-05-07 17:35:58 -07:00
}
}
2015-06-19 15:30:10 -07:00
// Also, update prs if given.
if prs != nil {
prs.Prevotes = ps.Prevotes
prs.Precommits = ps.Precommits
prs.LastCommit = ps.LastCommit
prs.CatchupCommit = ps.CatchupCommit
2015-06-22 19:04:31 -07:00
prs.ProposalPOL = ps.ProposalPOL
2015-06-19 15:30:10 -07:00
}
2014-08-10 16:35:08 -07:00
}
func (ps *PeerState) SetHasVote(vote *types.Vote, index uint) {
2014-08-10 16:35:08 -07:00
ps.mtx.Lock()
defer ps.mtx.Unlock()
ps.setHasVote(vote.Height, vote.Round, vote.Type, index)
}
func (ps *PeerState) setHasVote(height uint, round uint, type_ byte, index uint) {
2015-06-19 15:30:10 -07:00
if ps.Height == height+1 && ps.LastCommitRound == round && type_ == types.VoteTypePrecommit {
// Special case for LastCommit.
ps.LastCommit.SetIndex(index, true)
log.Debug("setHasVote", "LastCommit", ps.LastCommit, "index", index)
return
} else if ps.Height != height {
// Does not apply.
return
}
2015-06-22 19:04:31 -07:00
// By here, ps.Height is height.
switch type_ {
case types.VoteTypePrevote:
2015-06-22 19:04:31 -07:00
if ps.ProposalPOLRound == int(round) {
ps.ProposalPOL.SetIndex(index, true)
}
ps.Prevotes.SetIndex(index, true)
log.Debug("SetHasVote", "peer", ps.Key, "prevotes", ps.Prevotes, "index", index)
case types.VoteTypePrecommit:
2015-06-22 19:04:31 -07:00
if ps.CatchupCommitRound == int(round) {
ps.CatchupCommit.SetIndex(index, true)
2015-06-19 15:30:10 -07:00
}
ps.Precommits.SetIndex(index, true)
log.Debug("SetHasVote", "peer", ps.Key, "precommits", ps.Precommits, "index", index)
default:
panic("Invalid vote type")
2014-08-10 16:35:08 -07:00
}
}
2015-06-19 15:30:10 -07:00
func (ps *PeerState) SetCatchupCommitRound(height, round uint) {
ps.mtx.Lock()
defer ps.mtx.Unlock()
2015-06-19 15:30:10 -07:00
if ps.Height != height {
return
}
2015-06-22 19:04:31 -07:00
if ps.CatchupCommitRound != -1 && ps.CatchupCommitRound != int(round) {
2015-06-19 15:30:10 -07:00
log.Warn("Conflicting CatchupCommitRound",
"height", height,
"orig", ps.CatchupCommitRound,
"new", round,
)
// TODO think harder
}
2015-06-22 19:04:31 -07:00
ps.CatchupCommitRound = int(round)
2015-06-19 15:30:10 -07:00
ps.CatchupCommit = nil
}
func (ps *PeerState) ApplyNewRoundStepMessage(msg *NewRoundStepMessage, rs *RoundState) {
2014-08-10 16:35:08 -07:00
ps.mtx.Lock()
defer ps.mtx.Unlock()
2014-09-14 15:37:32 -07:00
// Just remember these values.
psHeight := ps.Height
psRound := ps.Round
//psStep := ps.Step
2015-06-19 15:30:10 -07:00
psCatchupCommitRound := ps.CatchupCommitRound
2015-06-22 19:04:31 -07:00
psCatchupCommit := ps.CatchupCommit
2014-09-14 15:37:32 -07:00
startTime := time.Now().Add(-1 * time.Duration(msg.SecondsSinceStartTime) * time.Second)
ps.Height = msg.Height
ps.Round = msg.Round
ps.Step = msg.Step
ps.StartTime = startTime
if psHeight != msg.Height || psRound != msg.Round {
2014-10-25 14:27:53 -07:00
ps.Proposal = false
2015-06-22 19:04:31 -07:00
ps.ProposalBlockPartsHeader = types.PartSetHeader{}
ps.ProposalBlockParts = nil
ps.ProposalPOLRound = -1
ps.ProposalPOL = nil
// We'll update the BitArray capacity later.
ps.Prevotes = nil
ps.Precommits = nil
2014-10-25 14:27:53 -07:00
}
2015-06-22 19:04:31 -07:00
if psHeight == msg.Height && psRound != msg.Round && int(msg.Round) == psCatchupCommitRound {
2015-06-19 15:30:10 -07:00
// Peer caught up to CatchupCommitRound.
ps.Precommits = psCatchupCommit
}
if psHeight != msg.Height {
2015-06-19 15:30:10 -07:00
// Shift Precommits to LastCommit.
if psHeight+1 == msg.Height && psRound == msg.LastCommitRound {
ps.LastCommitRound = msg.LastCommitRound
ps.LastCommit = ps.Precommits
} else {
2015-06-19 15:30:10 -07:00
ps.LastCommitRound = msg.LastCommitRound
ps.LastCommit = nil
}
// We'll update the BitArray capacity later.
2015-06-19 15:30:10 -07:00
ps.CatchupCommitRound = -1
ps.CatchupCommit = nil
2014-09-14 15:37:32 -07:00
}
2014-08-10 16:35:08 -07:00
}
func (ps *PeerState) ApplyCommitStepMessage(msg *CommitStepMessage) {
ps.mtx.Lock()
defer ps.mtx.Unlock()
if ps.Height != msg.Height {
return
}
2015-06-22 19:04:31 -07:00
ps.ProposalBlockPartsHeader = msg.BlockPartsHeader
ps.ProposalBlockParts = msg.BlockParts
}
func (ps *PeerState) ApplyHasVoteMessage(msg *HasVoteMessage) {
ps.mtx.Lock()
defer ps.mtx.Unlock()
2015-06-19 15:30:10 -07:00
if ps.Height != msg.Height {
return
}
ps.setHasVote(msg.Height, msg.Round, msg.Type, msg.Index)
}
2015-06-22 19:04:31 -07:00
func (ps *PeerState) ApplyProposalPOLMessage(msg *ProposalPOLMessage) {
ps.mtx.Lock()
defer ps.mtx.Unlock()
if ps.Height != msg.Height {
return
}
if ps.ProposalPOLRound != int(msg.ProposalPOLRound) {
return
}
// TODO: Merge onto existing ps.ProposalPOL?
// We might have sent some prevotes in the meantime.
ps.ProposalPOL = msg.ProposalPOL
}
2014-08-10 16:35:08 -07:00
//-----------------------------------------------------------------------------
// Messages
const (
2014-09-14 15:37:32 -07:00
msgTypeNewRoundStep = byte(0x01)
msgTypeCommitStep = byte(0x02)
msgTypeProposal = byte(0x11)
2015-06-22 19:04:31 -07:00
msgTypeProposalPOL = byte(0x12)
msgTypeBlockPart = byte(0x13) // both block & POL
msgTypeVote = byte(0x14)
msgTypeHasVote = byte(0x15)
2014-08-10 16:35:08 -07:00
)
2015-04-14 15:57:16 -07:00
type ConsensusMessage interface{}
var _ = binary.RegisterInterface(
struct{ ConsensusMessage }{},
binary.ConcreteType{&NewRoundStepMessage{}, msgTypeNewRoundStep},
binary.ConcreteType{&CommitStepMessage{}, msgTypeCommitStep},
binary.ConcreteType{&ProposalMessage{}, msgTypeProposal},
2015-06-22 19:04:31 -07:00
binary.ConcreteType{&ProposalPOLMessage{}, msgTypeProposalPOL},
binary.ConcreteType{&BlockPartMessage{}, msgTypeBlockPart},
binary.ConcreteType{&VoteMessage{}, msgTypeVote},
binary.ConcreteType{&HasVoteMessage{}, msgTypeHasVote},
2015-04-14 15:57:16 -07:00
)
2014-08-10 16:35:08 -07:00
// TODO: check for unnecessary extra bytes at the end.
2015-04-14 15:57:16 -07:00
func DecodeMessage(bz []byte) (msgType byte, msg ConsensusMessage, err error) {
msgType = bz[0]
2015-04-14 15:57:16 -07:00
n := new(int64)
r := bytes.NewReader(bz)
msg = binary.ReadBinary(struct{ ConsensusMessage }{}, r, n, &err).(struct{ ConsensusMessage }).ConsensusMessage
return
2014-08-10 16:35:08 -07:00
}
//-------------------------------------
2015-06-19 15:30:10 -07:00
// For every height/round/step transition
2014-09-14 15:37:32 -07:00
type NewRoundStepMessage struct {
Height uint
Round uint
Step RoundStepType
SecondsSinceStartTime uint
2015-06-19 15:30:10 -07:00
LastCommitRound uint
2014-08-10 16:35:08 -07:00
}
2014-09-14 15:37:32 -07:00
func (m *NewRoundStepMessage) String() string {
2015-06-19 15:30:10 -07:00
return fmt.Sprintf("[NewRoundStep H:%v R:%v S:%v LCR:%v]",
m.Height, m.Round, m.Step, m.LastCommitRound)
2014-08-10 16:35:08 -07:00
}
//-------------------------------------
type CommitStepMessage struct {
2015-06-22 19:04:31 -07:00
Height uint
BlockPartsHeader types.PartSetHeader
BlockParts *BitArray
}
func (m *CommitStepMessage) String() string {
2015-06-22 19:04:31 -07:00
return fmt.Sprintf("[CommitStep H:%v BP:%v BA:%v]", m.Height, m.BlockPartsHeader, m.BlockParts)
}
//-------------------------------------
type ProposalMessage struct {
Proposal *Proposal
}
func (m *ProposalMessage) String() string {
return fmt.Sprintf("[Proposal %v]", m.Proposal)
}
//-------------------------------------
2015-06-22 19:04:31 -07:00
type ProposalPOLMessage struct {
Height uint
ProposalPOLRound uint
ProposalPOL *BitArray
}
func (m *ProposalPOLMessage) String() string {
return fmt.Sprintf("[ProposalPOL H:%v POLR:%v POL:%v]", m.Height, m.ProposalPOLRound, m.ProposalPOL)
}
2014-09-14 15:37:32 -07:00
2015-06-22 19:04:31 -07:00
//-------------------------------------
type BlockPartMessage struct {
Height uint
Round uint
Part *types.Part
2014-08-10 16:35:08 -07:00
}
2015-06-22 19:04:31 -07:00
func (m *BlockPartMessage) String() string {
return fmt.Sprintf("[BlockPart H:%v R:%v T:%X P:%v]", m.Height, m.Round, m.Part)
2014-08-10 16:35:08 -07:00
}
//-------------------------------------
type VoteMessage struct {
ValidatorIndex uint
Vote *types.Vote
2014-08-10 16:35:08 -07:00
}
func (m *VoteMessage) String() string {
2015-05-06 00:47:20 -07:00
return fmt.Sprintf("[Vote VI:%v V:%v VI:%v]", m.ValidatorIndex, m.Vote, m.ValidatorIndex)
2014-08-10 16:35:08 -07:00
}
//-------------------------------------
type HasVoteMessage struct {
Height uint
Round uint
Type byte
Index uint
}
func (m *HasVoteMessage) String() string {
2015-05-06 00:47:20 -07:00
return fmt.Sprintf("[HasVote VI:%v V:{%v/%02d/%v} VI:%v]", m.Index, m.Height, m.Round, m.Type, m.Index)
}