tendermint/consensus/reactor.go

889 lines
25 KiB
Go
Raw Normal View History

2014-08-10 16:35:08 -07:00
package consensus
import (
"bytes"
"errors"
"fmt"
"io"
"math"
"sync"
"sync/atomic"
"time"
. "github.com/tendermint/tendermint/binary"
. "github.com/tendermint/tendermint/blocks"
. "github.com/tendermint/tendermint/common"
"github.com/tendermint/tendermint/mempool"
2014-08-10 16:35:08 -07:00
"github.com/tendermint/tendermint/p2p"
"github.com/tendermint/tendermint/state"
2014-08-10 16:35:08 -07:00
)
const (
2014-09-14 15:37:32 -07:00
StateCh = byte(0x20)
DataCh = byte(0x21)
VoteCh = byte(0x22)
peerStateKey = "ConsensusReactor.peerState"
2014-08-10 16:35:08 -07:00
voteTypeNil = byte(0x00)
voteTypeBlock = byte(0x01)
roundDuration0 = 60 * time.Second // The first round is 60 seconds long.
roundDurationDelta = 15 * time.Second // Each successive round lasts 15 seconds longer.
roundDeadlinePrevote = float64(1.0 / 3.0) // When the prevote is due.
2014-08-10 16:35:08 -07:00
roundDeadlinePrecommit = float64(2.0 / 3.0) // When the precommit vote is due.
finalizeDuration = roundDuration0 / 3 // The time to wait between commitTime and startTime of next consensus rounds.
2014-09-14 15:37:32 -07:00
peerGossipSleepDuration = 50 * time.Millisecond // Time to sleep if there's nothing to send.
hasVotesThreshold = 50 // After this many new votes we'll send a HasVotesMessage.
2014-08-10 16:35:08 -07:00
)
//-----------------------------------------------------------------------------
// total duration of given round
func calcRoundDuration(round uint16) time.Duration {
return roundDuration0 + roundDurationDelta*time.Duration(round)
}
// startTime is when round zero started.
func calcRoundStartTime(round uint16, startTime time.Time) time.Time {
return startTime.Add(roundDuration0*time.Duration(round) +
roundDurationDelta*(time.Duration((int64(round)*int64(round)-int64(round))/2)))
}
2014-09-11 10:55:32 -07:00
// calculates the current round given startTime of round zero.
// NOTE: round is zero if startTime is in the future.
2014-08-10 16:35:08 -07:00
func calcRound(startTime time.Time) uint16 {
now := time.Now()
if now.Before(startTime) {
return 0
2014-08-10 16:35:08 -07:00
}
// Start + D_0 * R + D_delta * (R^2 - R)/2 <= Now; find largest integer R.
// D_delta * R^2 + (2D_0 - D_delta) * R + 2(Start - Now) <= 0.
2014-08-10 16:35:08 -07:00
// AR^2 + BR + C <= 0; A = D_delta, B = (2_D0 - D_delta), C = 2(Start - Now).
// R = Floor((-B + Sqrt(B^2 - 4AC))/2A)
A := float64(roundDurationDelta)
B := 2.0*float64(roundDuration0) - float64(roundDurationDelta)
C := 2.0 * float64(startTime.Sub(now))
R := math.Floor((-B + math.Sqrt(B*B-4.0*A*C)) / (2 * A))
2014-08-10 16:35:08 -07:00
if math.IsNaN(R) {
panic("Could not calc round, should not happen")
}
if R > math.MaxInt16 {
Panicf("Could not calc round, round overflow: %v", R)
}
if R < 0 {
return 0
}
return uint16(R)
}
// convenience
// NOTE: elapsedRatio can be negative if startTime is in the future.
func calcRoundInfo(startTime time.Time) (round uint16, roundStartTime time.Time, roundDuration time.Duration,
roundElapsed time.Duration, elapsedRatio float64) {
round = calcRound(startTime)
roundStartTime = calcRoundStartTime(round, startTime)
roundDuration = calcRoundDuration(round)
roundElapsed = time.Now().Sub(roundStartTime)
elapsedRatio = float64(roundElapsed) / float64(roundDuration)
return
}
2014-08-10 16:35:08 -07:00
//-----------------------------------------------------------------------------
2014-10-21 23:30:18 -07:00
type RoundAction struct {
Height uint32 // The block height for which consensus is reaching for.
Round uint16 // The round number at given height.
Action RoundActionType // Action to perform.
}
2014-10-22 17:20:44 -07:00
//-----------------------------------------------------------------------------
2014-09-14 15:37:32 -07:00
type ConsensusReactor struct {
sw *p2p.Switch
quit chan struct{}
started uint32
stopped uint32
2014-08-10 16:35:08 -07:00
2014-09-11 10:55:32 -07:00
conS *ConsensusState
doActionCh chan RoundAction
2014-08-10 16:35:08 -07:00
}
2014-10-22 17:20:44 -07:00
func NewConsensusReactor(blockStore *BlockStore, mempool *mempool.Mempool, state *state.State) *ConsensusReactor {
2014-09-14 15:37:32 -07:00
conS := NewConsensusState(state, blockStore, mempool)
conR := &ConsensusReactor{
quit: make(chan struct{}),
2014-09-11 10:55:32 -07:00
conS: conS,
doActionCh: make(chan RoundAction, 1),
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) {
2014-09-14 15:37:32 -07:00
if atomic.CompareAndSwapUint32(&conR.started, 0, 1) {
log.Info("Starting ConsensusReactor")
2014-10-22 17:20:44 -07:00
conR.sw = sw
go conR.stepTransitionRoutine()
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.stopped, 0, 1) {
log.Info("Stopping ConsensusReactor")
close(conR.quit)
2014-08-10 16:35:08 -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{
2014-10-04 19:16:49 -07:00
Id: StateCh,
Priority: 5,
2014-09-14 15:37:32 -07:00
},
&p2p.ChannelDescriptor{
2014-10-04 19:16:49 -07:00
Id: DataCh,
Priority: 5,
2014-09-14 15:37:32 -07:00
},
&p2p.ChannelDescriptor{
2014-10-04 19:16:49 -07:00
Id: VoteCh,
Priority: 5,
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) {
// Create peerState for peer
peerState := NewPeerState(peer)
peer.Data.Set(peerStateKey, peerState)
// Begin gossip routines for this peer.
go conR.gossipDataRoutine(peer, peerState)
go conR.gossipVotesRoutine(peer, peerState)
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{}) {
//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) {
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_ := decodeMessage(msgBytes)
voteAddCounter := 0
var err error = nil
2014-08-10 16:35:08 -07:00
2014-10-25 14:27:53 -07:00
log.Debug("[%X][%v] Receive: %v", chId, peer, msg_)
2014-09-14 15:37:32 -07:00
switch chId {
case StateCh:
2014-08-10 16:35:08 -07:00
switch msg_.(type) {
2014-09-14 15:37:32 -07:00
case *NewRoundStepMessage:
msg := msg_.(*NewRoundStepMessage)
err = ps.ApplyNewRoundStepMessage(msg)
2014-08-10 16:35:08 -07:00
2014-09-14 15:37:32 -07:00
case *HasVotesMessage:
msg := msg_.(*HasVotesMessage)
err = ps.ApplyHasVotesMessage(msg)
2014-08-10 16:35:08 -07:00
2014-09-14 15:37:32 -07:00
default:
// Ignore unknown message
}
2014-08-10 16:35:08 -07:00
2014-09-14 15:37:32 -07:00
case DataCh:
switch msg_.(type) {
case *Proposal:
proposal := msg_.(*Proposal)
ps.SetHasProposal(proposal.Height, proposal.Round)
err = conR.conS.SetProposal(proposal)
case *PartMessage:
msg := msg_.(*PartMessage)
if msg.Type == partTypeProposalBlock {
ps.SetHasProposalBlockPart(msg.Height, msg.Round, msg.Part.Index)
_, err = conR.conS.AddProposalBlockPart(msg.Height, msg.Round, msg.Part)
} else if msg.Type == partTypeProposalPOL {
ps.SetHasProposalPOLPart(msg.Height, msg.Round, msg.Part.Index)
_, err = conR.conS.AddProposalPOLPart(msg.Height, msg.Round, msg.Part)
} else {
// Ignore unknown part type
}
2014-09-14 15:37:32 -07:00
default:
// Ignore unknown message
}
2014-09-14 15:37:32 -07:00
case VoteCh:
switch msg_.(type) {
case *Vote:
vote := msg_.(*Vote)
// We can't deal with votes from another height,
// as they have a different validator set.
if vote.Height != rs.Height || vote.Height != ps.Height {
return
}
2014-10-11 21:27:58 -07:00
index, val := rs.Validators.GetById(vote.SignerId)
if val == nil {
2014-09-14 15:37:32 -07:00
log.Warning("Peer gave us an invalid vote.")
return
}
ps.SetHasVote(rs.Height, rs.Round, vote.Type, uint32(index))
added, err := conR.conS.AddVote(vote)
if err != nil {
log.Warning("Error attempting to add vote: %v", err)
}
if added {
// Maybe send HasVotesMessage
2014-10-21 23:30:18 -07:00
// TODO optimize. It would be better to just acks for each vote!
2014-09-14 15:37:32 -07:00
voteAddCounter++
if voteAddCounter%hasVotesThreshold == 0 {
msg := &HasVotesMessage{
Height: rs.Height,
Round: rs.Round,
Prevotes: rs.Prevotes.BitArray(),
2014-09-14 15:37:32 -07:00
Precommits: rs.Precommits.BitArray(),
Commits: rs.Commits.BitArray(),
2014-08-10 16:35:08 -07:00
}
2014-09-14 15:37:32 -07:00
conR.sw.Broadcast(StateCh, msg)
2014-08-10 16:35:08 -07:00
}
// Maybe run RoundActionCommitWait.
if vote.Type == VoteTypeCommit &&
rs.Commits.HasTwoThirdsMajority() &&
rs.Step < RoundStepCommit {
// NOTE: Do not call RunAction*() methods here directly.
conR.doActionCh <- RoundAction{rs.Height, rs.Round, RoundActionCommitWait}
}
2014-08-10 16:35:08 -07:00
}
default:
// Ignore unknown message
}
2014-09-14 15:37:32 -07:00
default:
// Ignore unknown channel
2014-08-10 16:35:08 -07:00
}
2014-09-14 15:37:32 -07:00
if err != nil {
log.Warning("Error in Receive(): %v", err)
}
2014-08-10 16:35:08 -07:00
}
2014-10-22 17:20:44 -07:00
// Sets our private validator account for signing votes.
func (conR *ConsensusReactor) SetPrivValidator(priv *PrivValidator) {
conR.conS.SetPrivValidator(priv)
}
func (conR *ConsensusReactor) IsStopped() bool {
return atomic.LoadUint32(&conR.stopped) == 1
}
2014-10-21 23:30:18 -07:00
//--------------------------------------
// Source of all round state transitions (and votes).
func (conR *ConsensusReactor) stepTransitionRoutine() {
2014-10-21 23:30:18 -07:00
// Schedule the next action by pushing a RoundAction{} to conR.doActionCh
// when it is due.
scheduleNextAction := func() {
rs := conR.conS.GetRoundState()
2014-10-24 18:21:30 -07:00
round, roundStartTime, roundDuration, _, elapsedRatio := calcRoundInfo(rs.StartTime)
log.Debug("Called scheduleNextAction. round:%v roundStartTime:%v elapsedRatio:%v",
round, roundStartTime, elapsedRatio)
2014-10-21 23:30:18 -07:00
go func() {
switch rs.Step {
case RoundStepStart:
// It's a new RoundState.
if elapsedRatio < 0 {
// startTime is in the future.
time.Sleep(time.Duration((-1.0 * elapsedRatio) * float64(roundDuration)))
2014-10-21 23:30:18 -07:00
}
conR.doActionCh <- RoundAction{rs.Height, rs.Round, RoundActionPropose}
case RoundStepPropose:
// Wake up when it's time to vote.
time.Sleep(time.Duration((roundDeadlinePrevote - elapsedRatio) * float64(roundDuration)))
2014-10-21 23:30:18 -07:00
conR.doActionCh <- RoundAction{rs.Height, rs.Round, RoundActionPrevote}
case RoundStepPrevote:
// Wake up when it's time to precommit.
time.Sleep(time.Duration((roundDeadlinePrecommit - elapsedRatio) * float64(roundDuration)))
2014-10-21 23:30:18 -07:00
conR.doActionCh <- RoundAction{rs.Height, rs.Round, RoundActionPrecommit}
case RoundStepPrecommit:
// Wake up when the round is over.
time.Sleep(time.Duration((1.0 - elapsedRatio) * float64(roundDuration)))
2014-10-21 23:30:18 -07:00
conR.doActionCh <- RoundAction{rs.Height, rs.Round, RoundActionNextRound}
case RoundStepCommit:
panic("Should not happen: RoundStepCommit waits until +2/3 commits.")
case RoundStepCommitWait:
// Wake up when it's time to finalize commit.
if rs.CommitTime.IsZero() {
panic("RoundStepCommitWait requires rs.CommitTime")
}
time.Sleep(rs.CommitTime.Sub(time.Now()) + finalizeDuration)
conR.doActionCh <- RoundAction{rs.Height, rs.Round, RoundActionFinalize}
default:
panic("Should not happen")
}
2014-10-21 23:30:18 -07:00
}()
}
scheduleNextAction()
// NOTE: All ConsensusState.RunAction*() calls must come from here.
// Since only one routine calls them, it is safe to assume that
// the RoundState Height/Round/Step won't change concurrently.
// However, other fields like Proposal could change, due to gossip.
ACTION_LOOP:
for {
roundAction := <-conR.doActionCh
height := roundAction.Height
round := roundAction.Round
action := roundAction.Action
rs := conR.conS.GetRoundState()
log.Info("Running round action A:%X %v", action, rs.Description())
2014-10-21 01:18:46 -07:00
broadcastNewRoundStep := func(step RoundStep) {
// Broadcast NewRoundStepMessage
msg := &NewRoundStepMessage{
2014-10-25 14:27:53 -07:00
Height: height,
Round: round,
Step: step,
NumValidators: uint32(rs.Validators.Size()),
SecondsSinceStartTime: uint32(rs.RoundElapsed().Seconds()),
}
conR.sw.Broadcast(StateCh, msg)
}
// Continue if action is not relevant
if height != rs.Height {
continue
}
// If action >= RoundActionCommit, the round doesn't matter.
if action < RoundActionCommit && round != rs.Round {
continue
}
// Run action
switch action {
case RoundActionPropose:
if rs.Step != RoundStepStart {
continue ACTION_LOOP
}
2014-10-21 01:18:46 -07:00
conR.conS.RunActionPropose(rs.Height, rs.Round)
broadcastNewRoundStep(RoundStepPropose)
scheduleNextAction()
continue ACTION_LOOP
case RoundActionPrevote:
if rs.Step >= RoundStepPrevote {
continue ACTION_LOOP
}
vote := conR.conS.RunActionPrevote(rs.Height, rs.Round)
2014-10-21 01:18:46 -07:00
broadcastNewRoundStep(RoundStepPrevote)
if vote != nil {
conR.broadcastVote(vote)
}
scheduleNextAction()
continue ACTION_LOOP
case RoundActionPrecommit:
if rs.Step >= RoundStepPrecommit {
continue ACTION_LOOP
}
vote := conR.conS.RunActionPrecommit(rs.Height, rs.Round)
2014-10-21 01:18:46 -07:00
broadcastNewRoundStep(RoundStepPrecommit)
if vote != nil {
conR.broadcastVote(vote)
2014-10-21 01:18:46 -07:00
}
scheduleNextAction()
continue ACTION_LOOP
case RoundActionNextRound:
if rs.Step >= RoundStepCommit {
continue ACTION_LOOP
}
conR.conS.SetupRound(rs.Round + 1)
scheduleNextAction()
continue ACTION_LOOP
case RoundActionCommit:
if rs.Step >= RoundStepCommit {
continue ACTION_LOOP
}
// NOTE: Duplicated in RoundActionCommitWait.
vote := conR.conS.RunActionCommit(rs.Height, rs.Round)
broadcastNewRoundStep(RoundStepCommit)
if vote != nil {
conR.broadcastVote(vote)
}
// do not schedule next action.
continue ACTION_LOOP
case RoundActionCommitWait:
if rs.Step >= RoundStepCommitWait {
continue ACTION_LOOP
}
2014-10-21 23:30:18 -07:00
// Commit first we haven't already.
if rs.Step < RoundStepCommit {
// NOTE: Duplicated in RoundActionCommit.
vote := conR.conS.RunActionCommit(rs.Height, rs.Round)
broadcastNewRoundStep(RoundStepCommit)
if vote != nil {
conR.broadcastVote(vote)
}
}
2014-10-21 23:30:18 -07:00
// Wait for more commit votes.
conR.conS.RunActionCommitWait(rs.Height, rs.Round)
scheduleNextAction()
continue ACTION_LOOP
case RoundActionFinalize:
if rs.Step != RoundStepCommitWait {
panic("This shouldn't happen")
}
conR.conS.RunActionFinalize(rs.Height, rs.Round)
// Height has been incremented, step is now RoundStepStart.
scheduleNextAction()
continue ACTION_LOOP
default:
panic("Unknown action")
}
// For clarity, ensure that all switch cases call "continue"
panic("Should not happen.")
}
}
func (conR *ConsensusReactor) broadcastVote(vote *Vote) {
msg := p2p.TypedMessage{msgTypeVote, vote}
conR.sw.Broadcast(VoteCh, msg)
2014-10-21 23:30:18 -07:00
}
//--------------------------------------
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.IsStopped() || conR.IsStopped() {
log.Info("Stopping gossipDataRoutine for %v.", peer)
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
// If ProposalBlockHash matches, send parts?
// NOTE: if we or peer is at RoundStepCommit*, the round
// won't necessarily match, but that's OK.
if rs.ProposalBlock.HashesTo(prs.ProposalBlockHash) {
if index, ok := rs.ProposalBlockPartSet.BitArray().Sub(
prs.ProposalBlockBitArray).PickRandom(); ok {
msg := &PartMessage{
Height: rs.Height,
Round: rs.Round,
Type: partTypeProposalBlock,
Part: rs.ProposalBlockPartSet.GetPart(uint16(index)),
}
peer.Send(DataCh, msg)
ps.SetHasProposalBlockPart(rs.Height, rs.Round, uint16(index))
continue OUTER_LOOP
}
}
2014-09-14 15:37:32 -07:00
// If height and round doesn't match, sleep.
if rs.Height != prs.Height || rs.Round != prs.Round {
time.Sleep(peerGossipSleepDuration)
2014-08-10 16:35:08 -07:00
continue OUTER_LOOP
}
2014-09-14 15:37:32 -07:00
// Send proposal?
if rs.Proposal != nil && !prs.Proposal {
msg := p2p.TypedMessage{msgTypeProposal, rs.Proposal}
peer.Send(DataCh, msg)
ps.SetHasProposal(rs.Height, rs.Round)
continue OUTER_LOOP
2014-08-10 16:35:08 -07:00
}
2014-09-14 15:37:32 -07:00
// Send proposal POL part?
2014-10-25 14:27:53 -07:00
if rs.ProposalPOLPartSet != nil {
if index, ok := rs.ProposalPOLPartSet.BitArray().Sub(
prs.ProposalPOLBitArray).PickRandom(); ok {
msg := &PartMessage{
Height: rs.Height,
Round: rs.Round,
Type: partTypeProposalPOL,
Part: rs.ProposalPOLPartSet.GetPart(uint16(index)),
}
peer.Send(DataCh, msg)
ps.SetHasProposalPOLPart(rs.Height, rs.Round, uint16(index))
continue OUTER_LOOP
2014-09-14 15:37:32 -07:00
}
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) {
OUTER_LOOP:
for {
// Manage disconnects from self or peer.
if peer.IsStopped() || conR.IsStopped() {
log.Info("Stopping gossipVotesRoutine for %v.", peer)
return
2014-08-10 16:35:08 -07:00
}
2014-09-14 15:37:32 -07:00
rs := conR.conS.GetRoundState()
prs := ps.GetRoundState()
// If height doesn't match, sleep.
2014-09-14 15:37:32 -07:00
if rs.Height != prs.Height {
time.Sleep(peerGossipSleepDuration)
continue OUTER_LOOP
2014-08-10 16:35:08 -07:00
}
2014-09-14 15:37:32 -07:00
// If there are prevotes to send...
2014-10-21 01:18:46 -07:00
if prs.Step <= RoundStepPrevote {
index, ok := rs.Prevotes.BitArray().Sub(prs.Prevotes).PickRandom()
2014-09-14 15:37:32 -07:00
if ok {
2014-10-11 21:27:58 -07:00
valId, val := rs.Validators.GetByIndex(uint32(index))
if val != nil {
vote := rs.Prevotes.Get(valId)
2014-09-14 15:37:32 -07:00
msg := p2p.TypedMessage{msgTypeVote, vote}
peer.Send(VoteCh, msg)
ps.SetHasVote(rs.Height, rs.Round, VoteTypePrevote, uint32(index))
if vote.Type == VoteTypeCommit {
ps.SetHasVote(rs.Height, rs.Round, VoteTypePrecommit, uint32(index))
ps.SetHasVote(rs.Height, rs.Round, VoteTypeCommit, uint32(index))
}
2014-09-14 15:37:32 -07:00
continue OUTER_LOOP
} else {
log.Error("index is not a valid validator index")
2014-08-10 16:35:08 -07:00
}
2014-09-14 15:37:32 -07:00
}
}
// If there are precommits to send...
if prs.Step <= RoundStepPrecommit {
index, ok := rs.Precommits.BitArray().Sub(prs.Precommits).PickRandom()
if ok {
2014-10-11 21:27:58 -07:00
valId, val := rs.Validators.GetByIndex(uint32(index))
if val != nil {
vote := rs.Precommits.Get(valId)
2014-09-14 15:37:32 -07:00
msg := p2p.TypedMessage{msgTypeVote, vote}
peer.Send(VoteCh, msg)
ps.SetHasVote(rs.Height, rs.Round, VoteTypePrecommit, uint32(index))
if vote.Type == VoteTypeCommit {
ps.SetHasVote(rs.Height, rs.Round, VoteTypeCommit, uint32(index))
}
2014-09-14 15:37:32 -07:00
continue OUTER_LOOP
} else {
log.Error("index is not a valid validator index")
2014-08-10 16:35:08 -07:00
}
}
}
2014-09-14 15:37:32 -07:00
// If there are any commits to send...
index, ok := rs.Commits.BitArray().Sub(prs.Commits).PickRandom()
if ok {
2014-10-11 21:27:58 -07:00
valId, val := rs.Validators.GetByIndex(uint32(index))
if val != nil {
vote := rs.Commits.Get(valId)
2014-09-14 15:37:32 -07:00
msg := p2p.TypedMessage{msgTypeVote, vote}
peer.Send(VoteCh, msg)
ps.SetHasVote(rs.Height, rs.Round, VoteTypeCommit, uint32(index))
continue OUTER_LOOP
} else {
log.Error("index is not a valid validator index")
}
}
2014-08-10 16:35:08 -07:00
2014-09-14 15:37:32 -07:00
// We sent nothing. Sleep...
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 {
Height uint32 // Height peer is at
Round uint16 // Round peer is at
Step RoundStep // Step peer is at
2014-09-14 15:37:32 -07:00
StartTime time.Time // Estimated start of round 0 at this height
Proposal bool // True if peer has proposal for this round
ProposalBlockHash []byte // Block parts merkle root
ProposalBlockBitArray BitArray // Block parts bitarray
ProposalPOLHash []byte // POL parts merkle root
ProposalPOLBitArray BitArray // POL parts bitarray
Prevotes BitArray // All votes peer has for this round
2014-09-14 15:37:32 -07:00
Precommits BitArray // All precommits peer has for this round
Commits BitArray // All commits peer has for this height
}
//-----------------------------------------------------------------------------
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 {
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 {
2014-09-14 15:37:32 -07:00
return &PeerState{}
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
}
2014-09-14 15:37:32 -07:00
func (ps *PeerState) SetHasProposal(height uint32, round uint16) {
ps.mtx.Lock()
defer ps.mtx.Unlock()
2014-09-14 15:37:32 -07:00
if ps.Height == height && ps.Round == round {
ps.Proposal = true
}
2014-09-14 15:37:32 -07:00
}
func (ps *PeerState) SetHasProposalBlockPart(height uint32, round uint16, index uint16) {
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 {
ps.ProposalBlockBitArray.SetIndex(uint(index), true)
2014-08-10 16:35:08 -07:00
}
}
2014-09-14 15:37:32 -07:00
func (ps *PeerState) SetHasProposalPOLPart(height uint32, round uint16, index uint16) {
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 {
ps.ProposalPOLBitArray.SetIndex(uint(index), true)
}
2014-08-10 16:35:08 -07:00
}
2014-09-14 15:37:32 -07:00
func (ps *PeerState) SetHasVote(height uint32, round uint16, type_ uint8, index uint32) {
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 || type_ == VoteTypeCommit) {
switch type_ {
case VoteTypePrevote:
ps.Prevotes.SetIndex(uint(index), true)
2014-09-14 15:37:32 -07:00
case VoteTypePrecommit:
ps.Precommits.SetIndex(uint(index), true)
case VoteTypeCommit:
ps.Commits.SetIndex(uint(index), true)
default:
panic("Invalid vote type")
}
2014-08-10 16:35:08 -07:00
}
}
2014-09-14 15:37:32 -07:00
func (ps *PeerState) ApplyNewRoundStepMessage(msg *NewRoundStepMessage) error {
2014-08-10 16:35:08 -07:00
ps.mtx.Lock()
defer ps.mtx.Unlock()
2014-09-14 15:37:32 -07:00
// Set step state
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
// Reset the rest
2014-10-25 14:27:53 -07:00
if ps.Round != msg.Round {
ps.Proposal = false
ps.ProposalBlockHash = nil
ps.ProposalBlockBitArray = BitArray{}
ps.ProposalPOLHash = nil
ps.ProposalPOLBitArray = BitArray{}
ps.Prevotes = NewBitArray(uint(msg.NumValidators))
ps.Precommits = NewBitArray(uint(msg.NumValidators))
}
2014-09-14 15:37:32 -07:00
if ps.Height != msg.Height {
2014-10-25 14:27:53 -07:00
ps.Commits = NewBitArray(uint(msg.NumValidators))
2014-09-14 15:37:32 -07:00
}
2014-08-10 16:35:08 -07:00
return nil
}
2014-09-14 15:37:32 -07:00
func (ps *PeerState) ApplyHasVotesMessage(msg *HasVotesMessage) error {
ps.mtx.Lock()
defer ps.mtx.Unlock()
2014-09-14 15:37:32 -07:00
if ps.Height == msg.Height {
ps.Commits = ps.Commits.Or(msg.Commits)
if ps.Round == msg.Round {
ps.Prevotes = ps.Prevotes.Or(msg.Prevotes)
2014-09-14 15:37:32 -07:00
ps.Precommits = ps.Precommits.Or(msg.Precommits)
} else {
ps.Prevotes = msg.Prevotes
2014-09-14 15:37:32 -07:00
ps.Precommits = msg.Precommits
}
}
2014-09-14 15:37:32 -07:00
return nil
}
2014-08-10 16:35:08 -07:00
//-----------------------------------------------------------------------------
// Messages
const (
2014-09-14 15:37:32 -07:00
msgTypeUnknown = byte(0x00)
// Messages for communicating state changes
msgTypeNewRoundStep = byte(0x01)
msgTypeHasVotes = byte(0x02)
// Messages of data
msgTypeProposal = byte(0x11)
msgTypePart = byte(0x12) // both block & POL
msgTypeVote = byte(0x13)
2014-08-10 16:35:08 -07:00
)
// TODO: check for unnecessary extra bytes at the end.
func decodeMessage(bz []byte) (msgType byte, msg interface{}) {
n, err := new(int64), new(error)
2014-08-10 16:35:08 -07:00
// log.Debug("decoding msg bytes: %X", bz)
msgType = bz[0]
2014-09-14 15:37:32 -07:00
r := bytes.NewReader(bz[1:])
switch msgType {
2014-09-14 15:37:32 -07:00
// Messages for communicating state changes
case msgTypeNewRoundStep:
msg = readNewRoundStepMessage(r, n, err)
case msgTypeHasVotes:
msg = readHasVotesMessage(r, n, err)
// Messages of data
case msgTypeProposal:
msg = ReadProposal(r, n, err)
case msgTypePart:
msg = readPartMessage(r, n, err)
2014-08-10 16:35:08 -07:00
case msgTypeVote:
2014-09-14 15:37:32 -07:00
msg = ReadVote(r, n, err)
2014-08-10 16:35:08 -07:00
default:
msg = nil
2014-08-10 16:35:08 -07:00
}
return
2014-08-10 16:35:08 -07:00
}
//-------------------------------------
2014-09-14 15:37:32 -07:00
type NewRoundStepMessage struct {
Height uint32
Round uint16
Step RoundStep
2014-10-25 14:27:53 -07:00
NumValidators uint32
2014-09-14 15:37:32 -07:00
SecondsSinceStartTime uint32
2014-08-10 16:35:08 -07:00
}
2014-09-14 15:37:32 -07:00
func readNewRoundStepMessage(r io.Reader, n *int64, err *error) *NewRoundStepMessage {
return &NewRoundStepMessage{
2014-10-25 14:27:53 -07:00
Height: ReadUInt32(r, n, err),
Round: ReadUInt16(r, n, err),
Step: RoundStep(ReadUInt8(r, n, err)),
NumValidators: ReadUInt32(r, n, err),
2014-09-14 15:37:32 -07:00
SecondsSinceStartTime: ReadUInt32(r, n, err),
2014-08-10 16:35:08 -07:00
}
}
2014-09-14 15:37:32 -07:00
func (m *NewRoundStepMessage) WriteTo(w io.Writer) (n int64, err error) {
WriteByte(w, msgTypeNewRoundStep, &n, &err)
WriteUInt32(w, m.Height, &n, &err)
WriteUInt16(w, m.Round, &n, &err)
WriteUInt8(w, uint8(m.Step), &n, &err)
2014-10-25 14:27:53 -07:00
WriteUInt32(w, m.NumValidators, &n, &err)
2014-09-14 15:37:32 -07:00
WriteUInt32(w, m.SecondsSinceStartTime, &n, &err)
2014-08-10 16:35:08 -07:00
return
}
2014-09-14 15:37:32 -07:00
func (m *NewRoundStepMessage) String() string {
return fmt.Sprintf("[NewRoundStep %v/%v/%X]", m.Height, m.Round, m.Step)
2014-08-10 16:35:08 -07:00
}
//-------------------------------------
2014-09-14 15:37:32 -07:00
type HasVotesMessage struct {
Height uint32
Round uint16
Prevotes BitArray
2014-09-14 15:37:32 -07:00
Precommits BitArray
Commits BitArray
2014-08-10 16:35:08 -07:00
}
2014-09-14 15:37:32 -07:00
func readHasVotesMessage(r io.Reader, n *int64, err *error) *HasVotesMessage {
return &HasVotesMessage{
Height: ReadUInt32(r, n, err),
Round: ReadUInt16(r, n, err),
Prevotes: ReadBitArray(r, n, err),
2014-09-14 15:37:32 -07:00
Precommits: ReadBitArray(r, n, err),
Commits: ReadBitArray(r, n, err),
2014-08-10 16:35:08 -07:00
}
}
2014-09-14 15:37:32 -07:00
func (m *HasVotesMessage) WriteTo(w io.Writer) (n int64, err error) {
WriteByte(w, msgTypeHasVotes, &n, &err)
WriteUInt32(w, m.Height, &n, &err)
2014-09-14 15:37:32 -07:00
WriteUInt16(w, m.Round, &n, &err)
WriteBinary(w, m.Prevotes, &n, &err)
2014-09-14 15:37:32 -07:00
WriteBinary(w, m.Precommits, &n, &err)
WriteBinary(w, m.Commits, &n, &err)
2014-08-10 16:35:08 -07:00
return
}
2014-09-14 15:37:32 -07:00
func (m *HasVotesMessage) String() string {
return fmt.Sprintf("[HasVotesMessage H:%v R:%v]", m.Height, m.Round)
2014-08-10 16:35:08 -07:00
}
//-------------------------------------
2014-09-14 15:37:32 -07:00
const (
partTypeProposalBlock = byte(0x01)
partTypeProposalPOL = byte(0x02)
)
type PartMessage struct {
Height uint32
Round uint16
Type byte
Part *Part
2014-08-10 16:35:08 -07:00
}
2014-09-14 15:37:32 -07:00
func readPartMessage(r io.Reader, n *int64, err *error) *PartMessage {
return &PartMessage{
Height: ReadUInt32(r, n, err),
Round: ReadUInt16(r, n, err),
Type: ReadByte(r, n, err),
Part: ReadPart(r, n, err),
2014-08-10 16:35:08 -07:00
}
}
2014-09-14 15:37:32 -07:00
func (m *PartMessage) WriteTo(w io.Writer) (n int64, err error) {
WriteByte(w, msgTypePart, &n, &err)
WriteUInt32(w, m.Height, &n, &err)
WriteUInt16(w, m.Round, &n, &err)
WriteByte(w, m.Type, &n, &err)
WriteBinary(w, m.Part, &n, &err)
2014-08-10 16:35:08 -07:00
return
}
2014-09-14 15:37:32 -07:00
func (m *PartMessage) String() string {
return fmt.Sprintf("[PartMessage H:%v R:%v T:%X]", m.Height, m.Round, m.Type)
2014-08-10 16:35:08 -07:00
}