tendermint/consensus/reactor.go

780 lines
20 KiB
Go
Raw Normal View History

2014-08-10 16:35:08 -07:00
package consensus
import (
"bytes"
"errors"
"fmt"
"io"
"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
2014-09-14 15:37:32 -07:00
peerGossipSleepDuration = 50 * time.Millisecond // Time to sleep if there's nothing to send.
2014-08-10 16:35:08 -07:00
)
//-----------------------------------------------------------------------------
2014-09-14 15:37:32 -07:00
type ConsensusReactor struct {
sw *p2p.Switch
started uint32
stopped uint32
2014-10-30 03:32:09 -07:00
quit chan struct{}
2014-08-10 16:35:08 -07:00
2014-10-30 03:32:09 -07:00
conS *ConsensusState
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-10-30 03:32:09 -07:00
conS: conS,
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
2014-10-30 03:32:09 -07:00
conR.conS.Start()
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.stopped, 0, 1) {
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
}
}
2014-10-30 03:32:09 -07:00
func (conR *ConsensusReactor) IsStopped() bool {
return atomic.LoadUint32(&conR.stopped) == 1
}
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)
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)
ps.ApplyNewRoundStepMessage(msg, rs)
2014-08-10 16:35:08 -07:00
case *CommitMessage:
msg := msg_.(*CommitMessage)
ps.ApplyCommitMessage(msg)
case *HasVoteMessage:
msg := msg_.(*HasVoteMessage)
ps.ApplyHasVoteMessage(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)
2014-09-14 15:37:32 -07:00
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)
added, index, err := conR.conS.AddVote(vote)
2014-09-14 15:37:32 -07:00
if err != nil {
log.Warning("Error attempting to add vote: %v", err)
}
// Initialize Prevotes/Precommits/Commits if needed
ps.EnsureVoteBitArrays(rs.Height, rs.Validators.Size())
ps.SetHasVote(vote, index)
2014-09-14 15:37:32 -07:00
if added {
msg := &HasVoteMessage{
Height: vote.Height,
Round: vote.Round,
Type: vote.Type,
Index: index,
2014-08-10 16:35:08 -07:00
}
conR.sw.Broadcast(StateCh, msg)
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)
}
2014-10-21 23:30:18 -07:00
//--------------------------------------
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
}
2014-10-30 03:32:09 -07:00
// Get seconds since beginning of height.
// Due to the condition documented, this is safe.
timeElapsed := rs.StartTime.Sub(time.Now())
2014-10-30 03:32:09 -07:00
// Broadcast NewRoundStepMessage
{
msg := &NewRoundStepMessage{
Height: rs.Height,
Round: rs.Round,
Step: rs.Step,
SecondsSinceStartTime: uint32(timeElapsed.Seconds()),
}
conR.sw.Broadcast(StateCh, msg)
}
// If the step is commit, then also broadcast a CommitMessage.
if rs.Step == RoundStepCommit {
msg := &CommitMessage{
Height: rs.Height,
BlockParts: rs.ProposalBlockParts.Header(),
BlockBitArray: rs.ProposalBlockParts.BitArray(),
}
conR.sw.Broadcast(StateCh, 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
2014-10-26 13:26:27 -07:00
// Send proposal Block parts?
// NOTE: if we or peer is at RoundStepCommit*, the round
// won't necessarily match, but that's OK.
2014-10-30 03:32:09 -07:00
if rs.ProposalBlockParts.Header().Equals(prs.ProposalBlockParts) {
2014-10-26 13:26:27 -07:00
if index, ok := rs.ProposalBlockParts.BitArray().Sub(
prs.ProposalBlockBitArray).PickRandom(); ok {
msg := &PartMessage{
Height: rs.Height,
Round: rs.Round,
Type: partTypeProposalBlock,
2014-10-26 13:26:27 -07:00
Part: rs.ProposalBlockParts.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.Proposal)
2014-09-14 15:37:32 -07:00
continue OUTER_LOOP
2014-08-10 16:35:08 -07:00
}
2014-10-26 13:26:27 -07:00
// Send proposal POL parts?
2014-10-30 03:32:09 -07:00
if rs.ProposalPOLParts.Header().Equals(prs.ProposalPOLParts) {
2014-10-26 13:26:27 -07:00
if index, ok := rs.ProposalPOLParts.BitArray().Sub(
2014-10-25 14:27:53 -07:00
prs.ProposalPOLBitArray).PickRandom(); ok {
msg := &PartMessage{
Height: rs.Height,
Round: rs.Round,
Type: partTypeProposalPOL,
2014-10-26 13:26:27 -07:00
Part: rs.ProposalPOLParts.GetPart(uint16(index)),
2014-10-25 14:27:53 -07:00
}
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()
2014-10-30 03:32:09 -07:00
trySendVote := func(voteSet *VoteSet, peerVoteSet BitArray) (sent bool) {
// TODO: give priority to our vote.
index, ok := voteSet.BitArray().Sub(peerVoteSet).PickRandom()
if ok {
vote := voteSet.GetByIndex(index)
// NOTE: vote may be a commit.
msg := p2p.TypedMessage{msgTypeVote, vote}
peer.Send(VoteCh, msg)
ps.SetHasVote(vote, index)
2014-10-30 03:32:09 -07:00
return true
}
return false
}
// If height matches, then send LastCommits, Prevotes, Precommits, or Commits.
if rs.Height == prs.Height {
// If there are lastcommits to send...
if prs.Round == 0 && prs.Step == RoundStepNewHeight {
if prs.LastCommits.Size() == rs.LastCommits.Size() {
if trySendVote(rs.LastCommits, prs.LastCommits) {
continue OUTER_LOOP
}
}
}
// Initialize Prevotes/Precommits/Commits if needed
ps.EnsureVoteBitArrays(rs.Height, rs.Validators.Size())
// If there are prevotes to send...
if rs.Round == prs.Round && prs.Step <= RoundStepPrevote {
if trySendVote(rs.Prevotes, prs.Prevotes) {
continue OUTER_LOOP
}
}
// If there are precommits to send...
if rs.Round == prs.Round && prs.Step <= RoundStepPrecommit {
if trySendVote(rs.Precommits, prs.Precommits) {
continue OUTER_LOOP
}
}
// If there are any commits to send...
if trySendVote(rs.Commits, prs.Commits) {
2014-10-30 03:32:09 -07:00
continue OUTER_LOOP
2014-09-14 15:37:32 -07:00
}
}
// If peer is lagging by height 1, match our LastCommits to peer's Commits.
if rs.Height == prs.Height+1 {
// Initialize Commits if needed
ps.EnsureVoteBitArrays(rs.Height-1, rs.LastCommits.Size())
// If there are lastcommits to send...
if trySendVote(rs.LastCommits, prs.Commits) {
2014-10-30 03:32:09 -07:00
continue OUTER_LOOP
2014-08-10 16:35:08 -07:00
}
2014-08-10 16:35:08 -07:00
}
// If peer is lagging by more than 1, load and send Validation and send Commits.
if rs.Height >= prs.Height+2 {
// Load the block header and validation for prs.Height+1,
// which contains commit signatures for prs.Height.
header, validation := conR.conS.LoadHeaderValidation(prs.Height + 1)
size := uint(len(validation.Commits))
// Initialize Commits if needed
ps.EnsureVoteBitArrays(prs.Height, size)
index, ok := validation.BitArray().Sub(prs.Commits).PickRandom()
if ok {
rsig := validation.Commits[index]
// Reconstruct vote.
vote := &Vote{
Height: prs.Height,
Round: rsig.Round,
Type: VoteTypeCommit,
BlockHash: header.LastBlockHash,
BlockParts: header.LastBlockParts,
Signature: rsig.Signature,
}
msg := p2p.TypedMessage{msgTypeVote, vote}
peer.Send(VoteCh, msg)
ps.SetHasVote(vote, 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
// 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 {
2014-10-30 03:32:09 -07:00
Height uint32 // Height peer is at
Round uint16 // Round peer is at
Step RoundStep // 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
ProposalBlockParts PartSetHeader //
ProposalBlockBitArray BitArray // True bit -> has part
ProposalPOLParts PartSetHeader //
ProposalPOLBitArray BitArray // True bit -> has part
Prevotes BitArray // All votes peer has for this round
Precommits BitArray // All precommits peer has for this round
Commits BitArray // All commits peer has for this height
LastCommits BitArray // All commits peer has for last 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 {
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
}
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
2014-10-30 03:32:09 -07:00
ps.ProposalBlockParts = proposal.BlockParts
ps.ProposalBlockBitArray = NewBitArray(uint(proposal.BlockParts.Total))
ps.ProposalPOLParts = proposal.POLParts
ps.ProposalPOLBitArray = NewBitArray(uint(proposal.POLParts.Total))
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 {
return
2014-08-10 16:35:08 -07:00
}
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()
if ps.Height != height || ps.Round != round {
return
}
ps.ProposalPOLBitArray.SetIndex(uint(index), true)
}
func (ps *PeerState) EnsureVoteBitArrays(height uint32, numValidators uint) {
ps.mtx.Lock()
defer ps.mtx.Unlock()
if ps.Height != height {
return
}
if ps.Prevotes.IsZero() {
ps.Prevotes = NewBitArray(numValidators)
}
if ps.Precommits.IsZero() {
ps.Precommits = NewBitArray(numValidators)
}
if ps.Commits.IsZero() {
ps.Commits = NewBitArray(numValidators)
}
2014-08-10 16:35:08 -07:00
}
func (ps *PeerState) SetHasVote(vote *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 uint32, round uint16, type_ byte, index uint) {
if ps.Height == height+1 && type_ == VoteTypeCommit {
// Special case for LastCommits.
ps.LastCommits.SetIndex(index, true)
return
} else if ps.Height != height {
// Does not apply.
return
}
switch type_ {
case VoteTypePrevote:
ps.Prevotes.SetIndex(index, true)
case VoteTypePrecommit:
ps.Precommits.SetIndex(index, true)
case VoteTypeCommit:
if round < ps.Round {
2014-10-30 03:32:09 -07:00
ps.Prevotes.SetIndex(index, true)
ps.Precommits.SetIndex(index, true)
}
ps.Commits.SetIndex(index, true)
default:
panic("Invalid vote type")
2014-08-10 16:35:08 -07:00
}
}
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
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
2014-10-30 03:32:09 -07:00
ps.ProposalBlockParts = PartSetHeader{}
2014-10-25 14:27:53 -07:00
ps.ProposalBlockBitArray = BitArray{}
2014-10-30 03:32:09 -07:00
ps.ProposalPOLParts = PartSetHeader{}
2014-10-25 14:27:53 -07:00
ps.ProposalPOLBitArray = BitArray{}
// We'll update the BitArray capacity later.
ps.Prevotes = BitArray{}
ps.Precommits = BitArray{}
2014-10-25 14:27:53 -07:00
}
if psHeight != msg.Height {
// Shift Commits to LastCommits
if psHeight+1 == msg.Height {
ps.LastCommits = ps.Commits
} else {
ps.LastCommits = BitArray{}
}
// We'll update the BitArray capacity later.
ps.Commits = BitArray{}
2014-09-14 15:37:32 -07:00
}
2014-08-10 16:35:08 -07:00
}
func (ps *PeerState) ApplyCommitMessage(msg *CommitMessage) {
ps.mtx.Lock()
defer ps.mtx.Unlock()
if ps.Height != msg.Height {
return
}
ps.ProposalBlockParts = msg.BlockParts
ps.ProposalBlockBitArray = msg.BlockBitArray
}
func (ps *PeerState) ApplyHasVoteMessage(msg *HasVoteMessage) {
ps.mtx.Lock()
defer ps.mtx.Unlock()
// Special case for LastCommits
if ps.Height == msg.Height+1 && msg.Type == VoteTypeCommit {
ps.LastCommits.SetIndex(msg.Index, true)
return
} else if ps.Height != msg.Height {
return
}
ps.setHasVote(msg.Height, msg.Round, msg.Type, msg.Index)
}
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)
msgTypeCommit = byte(0x02)
2014-09-14 15:37:32 -07:00
// Messages of data
msgTypeProposal = byte(0x11)
msgTypePart = byte(0x12) // both block & POL
msgTypeVote = byte(0x13)
msgTypeHasVote = byte(0x14)
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 msgTypeCommit:
msg = readCommitMessage(r, n, err)
2014-09-14 15:37:32 -07:00
// 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)
case msgTypeHasVote:
msg = readHasVoteMessage(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-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{
Height: ReadUInt32(r, n, err),
Round: ReadUInt16(r, n, err),
Step: RoundStep(ReadUInt8(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-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
}
//-------------------------------------
type CommitMessage struct {
Height uint32
BlockParts PartSetHeader
BlockBitArray BitArray
}
func readCommitMessage(r io.Reader, n *int64, err *error) *CommitMessage {
return &CommitMessage{
Height: ReadUInt32(r, n, err),
BlockParts: ReadPartSetHeader(r, n, err),
BlockBitArray: ReadBitArray(r, n, err),
}
}
func (m *CommitMessage) WriteTo(w io.Writer) (n int64, err error) {
WriteByte(w, msgTypeCommit, &n, &err)
WriteUInt32(w, m.Height, &n, &err)
WriteBinary(w, m.BlockParts, &n, &err)
WriteBinary(w, m.BlockBitArray, &n, &err)
return
}
func (m *CommitMessage) String() string {
return fmt.Sprintf("[Commit %v/%v/%v]", m.Height, m.BlockParts, m.BlockBitArray)
}
//-------------------------------------
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
}
//-------------------------------------
type HasVoteMessage struct {
Height uint32
Round uint16
Type byte
Index uint
}
func readHasVoteMessage(r io.Reader, n *int64, err *error) *HasVoteMessage {
return &HasVoteMessage{
Height: ReadUInt32(r, n, err),
Round: ReadUInt16(r, n, err),
Type: ReadByte(r, n, err),
Index: ReadUVarInt(r, n, err),
}
}
func (m *HasVoteMessage) WriteTo(w io.Writer) (n int64, err error) {
WriteByte(w, msgTypeHasVote, &n, &err)
WriteUInt32(w, m.Height, &n, &err)
WriteUInt16(w, m.Round, &n, &err)
WriteByte(w, m.Type, &n, &err)
WriteUVarInt(w, m.Index, &n, &err)
return
}
func (m *HasVoteMessage) String() string {
return fmt.Sprintf("[HasVoteMessage H:%v R:%v T:%X]", m.Height, m.Round, m.Type)
}