RoundAction // FinalizeCommit // LastCommits

This commit is contained in:
Jae Kwon 2014-10-20 19:02:10 -07:00
parent 9ea504030e
commit f5d0794756
14 changed files with 382 additions and 243 deletions

View File

@ -85,19 +85,6 @@ func (b *Block) HashesTo(hash []byte) bool {
return bytes.Equal(b.Hash(), hash)
}
// Makes an empty next block.
func (b *Block) MakeNextBlock() *Block {
return &Block{
Header: Header{
Network: b.Header.Network,
Height: b.Header.Height + 1,
Time: time.Now(),
LastBlockHash: b.Hash(),
StateHash: nil,
},
}
}
func (b *Block) String() string {
return b.StringWithIndent("")
}

View File

@ -40,14 +40,14 @@ func randBlock() *Block {
VoteA: Vote{
Height: RandUInt32Exp(),
Round: RandUInt16Exp(),
Type: VoteTypeBare,
Type: VoteTypePrevote,
BlockHash: RandBytes(32),
Signature: randSig(),
},
VoteB: Vote{
Height: RandUInt32Exp(),
Round: RandUInt16Exp(),
Type: VoteTypeBare,
Type: VoteTypePrevote,
BlockHash: RandBytes(32),
Signature: randSig(),
},

View File

@ -9,7 +9,7 @@ import (
)
const (
VoteTypeBare = byte(0x00)
VoteTypePrevote = byte(0x00)
VoteTypePrecommit = byte(0x01)
VoteTypeCommit = byte(0x02)
)
@ -22,7 +22,7 @@ var (
ErrVoteConflictingSignature = errors.New("Conflicting round vote signature")
)
// Represents a bare, precommit, or commit vote for proposals.
// Represents a prevote, precommit, or commit vote for proposals.
type Vote struct {
Height uint32
Round uint16
@ -64,12 +64,12 @@ func (v *Vote) String() string {
blockHash = make([]byte, 6) // for printing
}
switch v.Type {
case VoteTypeBare:
case VoteTypePrevote:
return fmt.Sprintf("Vote{%v/%v:%X:%v}", v.Height, v.Round, blockHash, v.SignerId)
case VoteTypePrecommit:
return fmt.Sprintf("Precommit{%v/%v:%X:%v}", v.Height, v.Round, blockHash, v.SignerId)
case VoteTypeCommit:
return fmt.Sprintf("Commit{%v/%v:%X:%v}", v.Height, v.Round, v.BlockHash[:6], v.SignerId)
return fmt.Sprintf("Commit{%v/%v:%X:%v}", v.Height, v.Round, blockHash, v.SignerId)
default:
panic("Unknown vote type")
}

View File

@ -30,10 +30,10 @@ const (
roundDuration0 = 60 * time.Second // The first round is 60 seconds long.
roundDurationDelta = 15 * time.Second // Each successive round lasts 15 seconds longer.
roundDeadlineBare = float64(1.0 / 3.0) // When the bare vote is due.
roundDeadlinePrevote = float64(1.0 / 3.0) // When the prevote is due.
roundDeadlinePrecommit = float64(2.0 / 3.0) // When the precommit vote is due.
newBlockWaitDuration = roundDuration0 / 3 // The time to wait between commitTime and startTime of next consensus rounds.
finalizeDuration = roundDuration0 / 3 // The time to wait between commitTime and startTime of next consensus rounds.
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.
)
@ -122,7 +122,7 @@ func (conR *ConsensusReactor) SetPrivValidator(priv *PrivValidator) {
func (conR *ConsensusReactor) Start() {
if atomic.CompareAndSwapUint32(&conR.started, 0, 1) {
log.Info("Starting ConsensusReactor")
go conR.proposeAndVoteRoutine()
go conR.stepTransitionRoutine()
}
}
@ -247,7 +247,7 @@ func (conR *ConsensusReactor) Receive(chId byte, peer *p2p.Peer, msgBytes []byte
msg := &HasVotesMessage{
Height: rs.Height,
Round: rs.Round,
Votes: rs.Votes.BitArray(),
Prevotes: rs.Prevotes.BitArray(),
Precommits: rs.Precommits.BitArray(),
Commits: rs.Commits.BitArray(),
}
@ -267,6 +267,113 @@ func (conR *ConsensusReactor) Receive(chId byte, peer *p2p.Peer, msgBytes []byte
}
}
//-------------------------------------
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.
}
// Source of all round state transitions (and votes).
// It can be preemptively woken up via a message to
// doActionCh.
func (conR *ConsensusReactor) stepTransitionRoutine() {
scheduleNextAction := func() {
// Figure out which height/round/step we're at,
// then schedule an action for when it is due.
rs := conR.conS.GetRoundState()
_, _, roundDuration, _, elapsedRatio := calcRoundInfo(rs.StartTime)
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) * roundDuration)
}
conR.doActionCh <- RoundAction{rs.Height, rs.Round, RoundActionPropose}
case RoundStepPropose:
// Wake up when it's time to vote.
time.Sleep(time.Duration(roundDeadlinePrevote-elapsedRatio) * roundDuration)
conR.doActionCh <- RoundAction{rs.Height, rs.Round, RoundActionVote}
case RoundStepVote:
// Wake up when it's time to precommit.
time.Sleep(time.Duration(roundDeadlinePrecommit-elapsedRatio) * roundDuration)
conR.doActionCh <- RoundAction{rs.Height, rs.Round, RoundActionPrecommit}
case RoundStepPrecommit:
// Wake up when the round is over.
time.Sleep(time.Duration(1.0-elapsedRatio) * roundDuration)
conR.doActionCh <- RoundAction{rs.Height, rs.Round, RoundActionCommit}
case RoundStepCommit:
// Wake up when it's time to finalize commit.
time.Sleep(time.Duration(1.0-elapsedRatio)*roundDuration + finalizeDuration)
conR.doActionCh <- RoundAction{rs.Height, rs.Round, RoundActionFinalize}
default:
panic("Should not happen")
}
}
scheduleNextAction()
for {
roundAction := <-conR.doActionCh
height := roundAction.Height
round := roundAction.Round
action := roundAction.Action
rs := conR.conS.GetRoundState()
setStepAndBroadcast := func(step RoundStep) {
conR.conS.SetStep(step)
// Broadcast NewRoundStepMessage
msg := &NewRoundStepMessage{
Height: height,
Round: round,
Step: step,
SecondsSinceStartTime: uint32(rs.RoundElapsed().Seconds()),
}
conR.sw.Broadcast(StateCh, msg)
}
if height != rs.Height || round != rs.Round {
// Action is not relevant
// This may happen if an external routine
// pushes an action to conR.doActionCh.
return
}
// Run step
if action == RoundActionPropose && rs.Step == RoundStepStart {
conR.runStepPropose(rs)
setStepAndBroadcast(RoundStepPropose)
} else if action == RoundActionVote && rs.Step <= RoundStepPropose {
conR.runStepPrevote(rs)
setStepAndBroadcast(RoundStepVote)
} else if action == RoundActionPrecommit && rs.Step <= RoundStepVote {
conR.runStepPrecommit(rs)
setStepAndBroadcast(RoundStepPrecommit)
} else if action == RoundActionCommit && rs.Step <= RoundStepPrecommit {
committed := conR.runStepCommit(rs)
if committed {
setStepAndBroadcast(RoundStepCommit)
} else {
// runStepCommit() already set the round to the next round,
// so the step is already RoundStepStart (same height).
}
} else if action == RoundActionFinalize && rs.Step == RoundStepCommit {
conR.runStepFinalize(rs)
// Height has been incremented, step is now RoundStepStart.
} else {
// This shouldn't happen now, but if an external source pushes
// to conR.doActionCh, we might just want to continue here.
panic("Shouldn't happen")
}
scheduleNextAction()
}
}
//-------------------------------------
func (conR *ConsensusReactor) gossipDataRoutine(peer *p2p.Peer, ps *PeerState) {
OUTER_LOOP:
@ -327,6 +434,8 @@ OUTER_LOOP:
}
}
//-------------------------------------
func (conR *ConsensusReactor) gossipVotesRoutine(peer *p2p.Peer, ps *PeerState) {
OUTER_LOOP:
for {
@ -338,22 +447,22 @@ OUTER_LOOP:
rs := conR.conS.GetRoundState()
prs := ps.GetRoundState()
// If height doens't match, sleep.
// If height doesn't match, sleep.
if rs.Height != prs.Height {
time.Sleep(peerGossipSleepDuration)
continue OUTER_LOOP
}
// If there are bare votes to send...
// If there are prevotes to send...
if prs.Step <= RoundStepVote {
index, ok := rs.Votes.BitArray().Sub(prs.Votes).PickRandom()
index, ok := rs.Prevotes.BitArray().Sub(prs.Prevotes).PickRandom()
if ok {
valId, val := rs.Validators.GetByIndex(uint32(index))
if val != nil {
vote := rs.Votes.GetVote(valId)
vote := rs.Prevotes.Get(valId)
msg := p2p.TypedMessage{msgTypeVote, vote}
peer.Send(VoteCh, msg)
ps.SetHasVote(rs.Height, rs.Round, VoteTypeBare, uint32(index))
ps.SetHasVote(rs.Height, rs.Round, VoteTypePrevote, uint32(index))
continue OUTER_LOOP
} else {
log.Error("index is not a valid validator index")
@ -367,7 +476,7 @@ OUTER_LOOP:
if ok {
valId, val := rs.Validators.GetByIndex(uint32(index))
if val != nil {
vote := rs.Precommits.GetVote(valId)
vote := rs.Precommits.Get(valId)
msg := p2p.TypedMessage{msgTypeVote, vote}
peer.Send(VoteCh, msg)
ps.SetHasVote(rs.Height, rs.Round, VoteTypePrecommit, uint32(index))
@ -383,7 +492,7 @@ OUTER_LOOP:
if ok {
valId, val := rs.Validators.GetByIndex(uint32(index))
if val != nil {
vote := rs.Commits.GetVote(valId)
vote := rs.Commits.Get(valId)
msg := p2p.TypedMessage{msgTypeVote, vote}
peer.Send(VoteCh, msg)
ps.SetHasVote(rs.Height, rs.Round, VoteTypeCommit, uint32(index))
@ -415,68 +524,62 @@ func (conR *ConsensusReactor) runStepPropose(rs *RoundState) {
conR.conS.MakeProposal()
}
func (conR *ConsensusReactor) runStepVote(rs *RoundState) {
func (conR *ConsensusReactor) runStepPrevote(rs *RoundState) {
// If we have a locked block, we must vote for that.
// NOTE: a locked block is already valid.
if rs.LockedBlock != nil {
conR.signAndBroadcastVote(rs, &Vote{
Height: rs.Height,
Round: rs.Round,
Type: VoteTypeBare,
Type: VoteTypePrevote,
BlockHash: rs.LockedBlock.Hash(),
})
}
// Try staging proposed block.
// If Block is nil, an error is returned.
err := conR.conS.stageBlock(rs.ProposalBlock)
if err != nil {
// Vote nil
// Prevote nil
conR.signAndBroadcastVote(rs, &Vote{
Height: rs.Height,
Round: rs.Round,
Type: VoteTypeBare,
Type: VoteTypePrevote,
BlockHash: nil,
})
} else {
// Vote for block
// Prevote block
conR.signAndBroadcastVote(rs, &Vote{
Height: rs.Height,
Round: rs.Round,
Type: VoteTypeBare,
Type: VoteTypePrevote,
BlockHash: rs.ProposalBlock.Hash(),
})
}
}
func (conR *ConsensusReactor) runStepPrecommit(rs *RoundState) {
// If we see a 2/3 majority of votes for a block, lock.
hash := conR.conS.LockOrUnlock(rs.Height, rs.Round)
if len(hash) > 0 {
// Precommit
// Precommit block
conR.signAndBroadcastVote(rs, &Vote{
Height: rs.Height,
Round: rs.Round,
Type: VoteTypePrecommit,
BlockHash: hash,
})
}
}
func (conR *ConsensusReactor) runStepCommit(rs *RoundState) bool {
// If we see a 2/3 majority of precommits for a block, commit.
block := conR.conS.Commit(rs.Height, rs.Round)
// If we see a 2/3 majority of votes for a block, lock.
block := conR.conS.TryCommit(rs.Height, rs.Round)
if block == nil {
// Couldn't commit, try next round.
conR.conS.SetupRound(rs.Round + 1)
return false
} else {
// Commit block.
conR.signAndBroadcastVote(rs, &Vote{
Height: rs.Height,
Round: rs.Round,
@ -487,108 +590,9 @@ func (conR *ConsensusReactor) runStepCommit(rs *RoundState) bool {
}
}
//-------------------------------------
type RoundAction struct {
Height uint32 // The block height for which consensus is reaching for.
Round uint16 // The round number at given height.
XnToStep uint8 // Transition to this step. Action depends on this value.
RoundElapsed time.Duration // Duration since round start.
}
// Source of all round state transitions and votes.
// It can be preemptively woken up via a message to
// doActionCh.
func (conR *ConsensusReactor) proposeAndVoteRoutine() {
// Figure out when to wake up next (in the absence of other events)
setAlarm := func() {
if len(conR.doActionCh) > 0 {
return // Already going to wake up later.
}
// Figure out which height/round/step we're at,
// then schedule an action for when it is due.
rs := conR.conS.GetRoundState()
_, _, roundDuration, roundElapsed, elapsedRatio := calcRoundInfo(rs.StartTime)
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) * roundDuration)
}
conR.doActionCh <- RoundAction{rs.Height, rs.Round, RoundStepPropose, roundElapsed}
case RoundStepPropose:
// Wake up when it's time to vote.
time.Sleep(time.Duration(roundDeadlineBare-elapsedRatio) * roundDuration)
conR.doActionCh <- RoundAction{rs.Height, rs.Round, RoundStepVote, roundElapsed}
case RoundStepVote:
// Wake up when it's time to precommit.
time.Sleep(time.Duration(roundDeadlinePrecommit-elapsedRatio) * roundDuration)
conR.doActionCh <- RoundAction{rs.Height, rs.Round, RoundStepPrecommit, roundElapsed}
case RoundStepPrecommit:
// Wake up when the round is over.
time.Sleep(time.Duration(1.0-elapsedRatio) * roundDuration)
conR.doActionCh <- RoundAction{rs.Height, rs.Round, RoundStepCommit, roundElapsed}
case RoundStepCommit:
// This shouldn't happen.
// Before setAlarm() got called,
// logic should have created a new RoundState for the next round.
panic("Should not happen")
}
}
for {
func() {
roundAction := <-conR.doActionCh
// Always set the alarm after any processing below.
defer setAlarm()
height := roundAction.Height
round := roundAction.Round
step := roundAction.XnToStep
roundElapsed := roundAction.RoundElapsed
rs := conR.conS.GetRoundState()
if height != rs.Height || round != rs.Round {
return // Action is not relevant
}
// Run step
if step == RoundStepPropose && rs.Step == RoundStepStart {
conR.runStepPropose(rs)
} else if step == RoundStepVote && rs.Step <= RoundStepPropose {
conR.runStepVote(rs)
} else if step == RoundStepPrecommit && rs.Step <= RoundStepVote {
conR.runStepPrecommit(rs)
} else if step == RoundStepCommit && rs.Step <= RoundStepPrecommit {
didCommit := conR.runStepCommit(rs)
if didCommit {
// We already set up ConsensusState for the next height
// (it happens in the call to conR.runStepCommit).
} else {
// Prepare a new RoundState for the next state.
conR.conS.SetupRound(rs.Round + 1)
return // setAlarm() takes care of the rest.
}
} else {
return // Action is not relevant.
}
// Transition to new step.
conR.conS.SetStep(step)
// Broadcast NewRoundStepMessage.
msg := &NewRoundStepMessage{
Height: height,
Round: round,
Step: step,
SecondsSinceStartTime: uint32(roundElapsed.Seconds()),
}
conR.sw.Broadcast(StateCh, msg)
}()
}
func (conR *ConsensusReactor) runStepFinalize(rs *RoundState) {
// This actually updates the height and sets up round 0.
conR.conS.FinalizeCommit()
}
//-----------------------------------------------------------------------------
@ -597,14 +601,14 @@ func (conR *ConsensusReactor) proposeAndVoteRoutine() {
type PeerRoundState struct {
Height uint32 // Height peer is at
Round uint16 // Round peer is at
Step uint8 // Step 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
ProposalBlockHash []byte // Block parts merkle root
ProposalBlockBitArray BitArray // Block parts bitarray
ProposalPOLHash []byte // POL parts merkle root
ProposalPOLBitArray BitArray // POL parts bitarray
Votes BitArray // All votes peer has for this round
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
}
@ -665,8 +669,8 @@ func (ps *PeerState) SetHasVote(height uint32, round uint16, type_ uint8, index
defer ps.mtx.Unlock()
if ps.Height == height && (ps.Round == round || type_ == VoteTypeCommit) {
switch type_ {
case VoteTypeBare:
ps.Votes.SetIndex(uint(index), true)
case VoteTypePrevote:
ps.Prevotes.SetIndex(uint(index), true)
case VoteTypePrecommit:
ps.Precommits.SetIndex(uint(index), true)
case VoteTypeCommit:
@ -694,7 +698,7 @@ func (ps *PeerState) ApplyNewRoundStepMessage(msg *NewRoundStepMessage) error {
ps.ProposalBlockBitArray = nil
ps.ProposalPOLHash = nil
ps.ProposalPOLBitArray = nil
ps.Votes = nil
ps.Prevotes = nil
ps.Precommits = nil
if ps.Height != msg.Height {
ps.Commits = nil
@ -708,10 +712,10 @@ func (ps *PeerState) ApplyHasVotesMessage(msg *HasVotesMessage) error {
if ps.Height == msg.Height {
ps.Commits = ps.Commits.Or(msg.Commits)
if ps.Round == msg.Round {
ps.Votes = ps.Votes.Or(msg.Votes)
ps.Prevotes = ps.Prevotes.Or(msg.Prevotes)
ps.Precommits = ps.Precommits.Or(msg.Precommits)
} else {
ps.Votes = msg.Votes
ps.Prevotes = msg.Prevotes
ps.Precommits = msg.Precommits
}
}
@ -762,7 +766,7 @@ func decodeMessage(bz []byte) (msgType byte, msg interface{}) {
type NewRoundStepMessage struct {
Height uint32
Round uint16
Step uint8
Step RoundStep
SecondsSinceStartTime uint32
}
@ -770,7 +774,7 @@ func readNewRoundStepMessage(r io.Reader, n *int64, err *error) *NewRoundStepMes
return &NewRoundStepMessage{
Height: ReadUInt32(r, n, err),
Round: ReadUInt16(r, n, err),
Step: ReadUInt8(r, n, err),
Step: RoundStep(ReadUInt8(r, n, err)),
SecondsSinceStartTime: ReadUInt32(r, n, err),
}
}
@ -779,7 +783,7 @@ 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, m.Step, &n, &err)
WriteUInt8(w, uint8(m.Step), &n, &err)
WriteUInt32(w, m.SecondsSinceStartTime, &n, &err)
return
}
@ -793,7 +797,7 @@ func (m *NewRoundStepMessage) String() string {
type HasVotesMessage struct {
Height uint32
Round uint16
Votes BitArray
Prevotes BitArray
Precommits BitArray
Commits BitArray
}
@ -802,7 +806,7 @@ func readHasVotesMessage(r io.Reader, n *int64, err *error) *HasVotesMessage {
return &HasVotesMessage{
Height: ReadUInt32(r, n, err),
Round: ReadUInt16(r, n, err),
Votes: ReadBitArray(r, n, err),
Prevotes: ReadBitArray(r, n, err),
Precommits: ReadBitArray(r, n, err),
Commits: ReadBitArray(r, n, err),
}
@ -812,7 +816,7 @@ func (m *HasVotesMessage) WriteTo(w io.Writer) (n int64, err error) {
WriteByte(w, msgTypeHasVotes, &n, &err)
WriteUInt32(w, m.Height, &n, &err)
WriteUInt16(w, m.Round, &n, &err)
WriteBinary(w, m.Votes, &n, &err)
WriteBinary(w, m.Prevotes, &n, &err)
WriteBinary(w, m.Precommits, &n, &err)
WriteBinary(w, m.Commits, &n, &err)
return

View File

@ -11,7 +11,7 @@ import (
)
// Proof of lock.
// +2/3 of validators' (bare) votes for a given blockhash (or nil)
// +2/3 of validators' prevotes for a given blockhash (or nil)
type POL struct {
Height uint32
Round uint16
@ -47,7 +47,7 @@ func (pol *POL) Verify(vset *state.ValidatorSet) error {
talliedVotingPower := uint64(0)
voteDoc := BinaryBytes(&Vote{Height: pol.Height, Round: pol.Round,
Type: VoteTypeBare, BlockHash: pol.BlockHash})
Type: VoteTypePrevote, BlockHash: pol.BlockHash})
seenValidators := map[uint64]struct{}{}
for _, sig := range pol.Votes {

View File

@ -19,7 +19,7 @@ func TestVerifyVotes(t *testing.T) {
Height: 0, Round: 0, BlockHash: blockHash,
}
vote := &Vote{
Height: 0, Round: 0, Type: VoteTypeBare, BlockHash: blockHash,
Height: 0, Round: 0, Type: VoteTypePrevote, BlockHash: blockHash,
}
for i := 0; i < 6; i++ {
privAccounts[i].Sign(vote)
@ -50,7 +50,7 @@ func TestVerifyInvalidVote(t *testing.T) {
Height: 0, Round: 0, BlockHash: blockHash,
}
vote := &Vote{
Height: 0, Round: 0, Type: VoteTypeBare, BlockHash: blockHash,
Height: 0, Round: 0, Type: VoteTypePrevote, BlockHash: blockHash,
}
for i := 0; i < 7; i++ {
privAccounts[i].Sign(vote)
@ -168,7 +168,7 @@ func TestReadWrite(t *testing.T) {
Height: 0, Round: 0, BlockHash: blockHash,
}
vote := &Vote{
Height: 0, Round: 0, Type: VoteTypeBare, BlockHash: blockHash,
Height: 0, Round: 0, Type: VoteTypePrevote, BlockHash: blockHash,
}
for i := 0; i < 7; i++ {
privAccounts[i].Sign(vote)

View File

@ -10,7 +10,11 @@ import (
type PrivValidator struct {
state.PrivAccount
db *db_.LevelDB
db db_.DB
}
func NewPrivValidator(priv *state.PrivAccount, db db_.DB) *PrivValidator {
return &PrivValidator{*priv, db}
}
// Double signing results in a panic.

View File

@ -9,16 +9,33 @@ import (
. "github.com/tendermint/tendermint/binary"
. "github.com/tendermint/tendermint/blocks"
. "github.com/tendermint/tendermint/common"
. "github.com/tendermint/tendermint/config"
"github.com/tendermint/tendermint/mempool"
"github.com/tendermint/tendermint/state"
)
type RoundStep uint8
type RoundActionType uint8
const (
RoundStepStart = uint8(0x00) // Round started.
RoundStepPropose = uint8(0x01) // Did propose, broadcasting proposal.
RoundStepVote = uint8(0x02) // Did vote, broadcasting votes.
RoundStepPrecommit = uint8(0x03) // Did precommit, broadcasting precommits.
RoundStepCommit = uint8(0x04) // We committed at this round -- do not progress to the next round.
RoundStepStart = RoundStep(0x00) // Round started.
RoundStepPropose = RoundStep(0x01) // Did propose, gossip proposal.
RoundStepVote = RoundStep(0x02) // Did vote, gossip votes.
RoundStepPrecommit = RoundStep(0x03) // Did precommit, gossip precommits.
RoundStepCommit = RoundStep(0x04) // Did commit, gossip commits.
// If a block could not be committed at a given round,
// we progress to the next round, skipping RoundStepCommit.
//
// If a block was committed, we goto RoundStepCommit,
// then wait "finalizeDuration" to gather more commit votes,
// then we progress to the next height at round 0.
RoundActionPropose = RoundActionType(0x00) // Goto RoundStepPropose
RoundActionVote = RoundActionType(0x01) // Goto RoundStepVote
RoundActionPrecommit = RoundActionType(0x02) // Goto RoundStepPrecommit
RoundActionCommit = RoundActionType(0x03) // Goto RoundStepCommit or RoundStepStart next round
RoundActionFinalize = RoundActionType(0x04) // Goto RoundStepStart next height
)
var (
@ -31,7 +48,7 @@ var (
type RoundState struct {
Height uint32 // Height we are working on
Round uint16
Step uint8
Step RoundStep
StartTime time.Time
Validators *state.ValidatorSet
Proposal *Proposal
@ -41,12 +58,17 @@ type RoundState struct {
ProposalPOLPartSet *PartSet
LockedBlock *Block
LockedPOL *POL
Votes *VoteSet
Prevotes *VoteSet
Precommits *VoteSet
Commits *VoteSet
LastCommits *VoteSet
PrivValidator *PrivValidator
}
func (rs *RoundState) RoundElapsed() time.Duration {
return rs.StartTime.Sub(time.Now())
}
func (rs *RoundState) String() string {
return rs.StringWithIndent("")
}
@ -61,9 +83,10 @@ func (rs *RoundState) StringWithIndent(indent string) string {
%s ProposalPOL: %v %v
%s LockedBlock: %v
%s LockedPOL: %v
%s Votes: %v
%s Precommits: %v
%s Commits: %v
%s Prevotes: %v
%s Precommits: %v
%s Commits: %v
%s LastCommits: %v
%s}`,
indent, rs.Height, rs.Round, rs.Step,
indent, rs.StartTime,
@ -73,9 +96,10 @@ func (rs *RoundState) StringWithIndent(indent string) string {
indent, rs.ProposalPOLPartSet.Description(), rs.ProposalPOL.Description(),
indent, rs.LockedBlock.Description(),
indent, rs.LockedPOL.Description(),
indent, rs.Votes.StringWithIndent(indent+" "),
indent, rs.Prevotes.StringWithIndent(indent+" "),
indent, rs.Precommits.StringWithIndent(indent+" "),
indent, rs.Commits.StringWithIndent(indent+" "),
indent, rs.LastCommits.StringWithIndent(indent+" "),
indent)
}
@ -122,7 +146,7 @@ func (cs *ConsensusState) updateToState(state *state.State) {
cs.Height = height
cs.Round = 0
cs.Step = RoundStepStart
cs.StartTime = state.CommitTime.Add(newBlockWaitDuration)
cs.StartTime = state.CommitTime.Add(finalizeDuration)
cs.Validators = validators
cs.Proposal = nil
cs.ProposalBlock = nil
@ -131,8 +155,9 @@ func (cs *ConsensusState) updateToState(state *state.State) {
cs.ProposalPOLPartSet = nil
cs.LockedBlock = nil
cs.LockedPOL = nil
cs.Votes = NewVoteSet(height, 0, VoteTypeBare, validators)
cs.Prevotes = NewVoteSet(height, 0, VoteTypePrevote, validators)
cs.Precommits = NewVoteSet(height, 0, VoteTypePrecommit, validators)
cs.LastCommits = cs.Commits
cs.Commits = NewVoteSet(height, 0, VoteTypeCommit, validators)
cs.state = state
@ -171,13 +196,13 @@ func (cs *ConsensusState) setupRound(round uint16) {
cs.ProposalBlockPartSet = nil
cs.ProposalPOL = nil
cs.ProposalPOLPartSet = nil
cs.Votes = NewVoteSet(cs.Height, round, VoteTypeBare, validators)
cs.Votes.AddFromCommits(cs.Commits)
cs.Prevotes = NewVoteSet(cs.Height, round, VoteTypePrevote, validators)
cs.Prevotes.AddFromCommits(cs.Commits)
cs.Precommits = NewVoteSet(cs.Height, round, VoteTypePrecommit, validators)
cs.Precommits.AddFromCommits(cs.Commits)
}
func (cs *ConsensusState) SetStep(step byte) {
func (cs *ConsensusState) SetStep(step RoundStep) {
cs.mtx.Lock()
defer cs.mtx.Unlock()
if cs.Step < step {
@ -237,8 +262,26 @@ func (cs *ConsensusState) MakeProposal() {
block = cs.LockedBlock
pol = cs.LockedPOL
} else {
// TODO: make use of state returned from MakeProposalBlock()
block, _ = cs.mempool.MakeProposalBlock()
// We need to create a proposal.
// If we don't have enough commits from the last height,
// we can't do anything.
if !cs.LastCommits.HasTwoThirdsMajority() {
return
}
txs, state := cs.mempool.GetProposalTxs() // TODO: cache state
block = &Block{
Header: Header{
Network: Config.Network,
Height: cs.Height,
Time: time.Now(),
LastBlockHash: cs.state.BlockHash,
StateHash: state.Hash(),
},
Validation: cs.LastCommits.MakeValidation(),
Data: Data{
Txs: txs,
},
}
pol = cs.LockedPOL // If exists, is a PoUnlock.
}
@ -317,15 +360,15 @@ func (cs *ConsensusState) AddProposalPOLPart(height uint32, round uint16, part *
func (cs *ConsensusState) AddVote(vote *Vote) (added bool, err error) {
switch vote.Type {
case VoteTypeBare:
// Votes checks for height+round match.
return cs.Votes.Add(vote)
case VoteTypePrevote:
// Prevotes checks for height+round match.
return cs.Prevotes.Add(vote)
case VoteTypePrecommit:
// Precommits checks for height+round match.
return cs.Precommits.Add(vote)
case VoteTypeCommit:
// Commits checks for height match.
cs.Votes.Add(vote)
cs.Prevotes.Add(vote)
cs.Precommits.Add(vote)
return cs.Commits.Add(vote)
default:
@ -344,10 +387,10 @@ func (cs *ConsensusState) LockOrUnlock(height uint32, round uint16) []byte {
return nil
}
if hash, _, ok := cs.Votes.TwoThirdsMajority(); ok {
if hash, _, ok := cs.Prevotes.TwoThirdsMajority(); ok {
// Remember this POL. (hash may be nil)
cs.LockedPOL = cs.Votes.MakePOL()
cs.LockedPOL = cs.Prevotes.MakePOL()
if len(hash) == 0 {
// +2/3 voted nil. Just unlock.
@ -378,7 +421,12 @@ func (cs *ConsensusState) LockOrUnlock(height uint32, round uint16) []byte {
}
}
func (cs *ConsensusState) Commit(height uint32, round uint16) *Block {
// Commits a block if we have enough precommits (and we have the block).
// If successful, saves the block and state and resets mempool,
// and returns the committed block.
// Commit is not finalized until FinalizeCommit() is called.
// This allows us to stay at this height and gather more commit votes.
func (cs *ConsensusState) TryCommit(height uint32, round uint16) *Block {
cs.mtx.Lock()
defer cs.mtx.Unlock()
@ -414,13 +462,11 @@ func (cs *ConsensusState) Commit(height uint32, round uint16) *Block {
// Save to blockStore
cs.blockStore.SaveBlock(block)
// What was staged becomes committed.
state := cs.stagedState
state.Save(commitTime)
cs.updateToState(state)
// Save the state
cs.stagedState.Save(commitTime)
// Update mempool.
cs.mempool.ResetForBlockAndState(block, state)
cs.mempool.ResetForBlockAndState(block, cs.stagedState)
return block
}
@ -428,6 +474,13 @@ func (cs *ConsensusState) Commit(height uint32, round uint16) *Block {
return nil
}
// After TryCommit(), if successful, must call this in order to
// update the RoundState.
func (cs *ConsensusState) FinalizeCommit() {
// What was staged becomes committed.
cs.updateToState(cs.stagedState)
}
func (cs *ConsensusState) stageBlock(block *Block) error {
// Already staged?

View File

@ -45,16 +45,79 @@ func randGenesisState(numAccounts int, numValidators int) (*state.State, []*stat
func makeConsensusState() (*ConsensusState, []*state.PrivAccount) {
state, privAccounts := randGenesisState(20, 10)
blockStore := NewBlockStore(db_.NewMemDB())
mempool := mempool.NewMempool(nil, state)
mempool := mempool.NewMempool(state)
cs := NewConsensusState(state, blockStore, mempool)
return cs, privAccounts
}
func TestUnit(t *testing.T) {
//-----------------------------------------------------------------------------
func TestSetupRound(t *testing.T) {
cs, privAccounts := makeConsensusState()
// Add a vote, precommit, and commit by val0.
voteTypes := []byte{VoteTypePrevote, VoteTypePrecommit, VoteTypeCommit}
for _, voteType := range voteTypes {
vote := &Vote{Height: 0, Round: 0, Type: voteType} // nil vote
privAccounts[0].Sign(vote)
cs.AddVote(vote)
}
// Ensure that vote appears in RoundState.
rs0 := cs.GetRoundState()
if vote := rs0.Prevotes.Get(0); vote == nil || vote.Type != VoteTypePrevote {
t.Errorf("Expected to find prevote %v, not there", vote)
}
if vote := rs0.Precommits.Get(0); vote == nil || vote.Type != VoteTypePrecommit {
t.Errorf("Expected to find precommit %v, not there", vote)
}
if vote := rs0.Commits.Get(0); vote == nil || vote.Type != VoteTypeCommit {
t.Errorf("Expected to find commit %v, not there", vote)
}
// Setup round 1 (next round)
cs.SetupRound(1)
// Now the commit should be copied over to prevotes and precommits.
rs1 := cs.GetRoundState()
if vote := rs1.Prevotes.Get(0); vote == nil || vote.Type != VoteTypeCommit {
t.Errorf("Expected to find commit %v, not there", vote)
}
if vote := rs1.Precommits.Get(0); vote == nil || vote.Type != VoteTypeCommit {
t.Errorf("Expected to find commit %v, not there", vote)
}
if vote := rs1.Commits.Get(0); vote == nil || vote.Type != VoteTypeCommit {
t.Errorf("Expected to find commit %v, not there", vote)
}
// Setup round 1 (should fail)
{
defer func() {
if e := recover(); e == nil {
t.Errorf("Expected to panic, round did not increment")
}
}()
cs.SetupRound(1)
}
}
func TestMakeProposalNoPrivValidator(t *testing.T) {
cs, _ := makeConsensusState()
cs.MakeProposal()
rs := cs.GetRoundState()
t.Log(rs)
if false {
t.Log(privAccounts)
if rs.Proposal != nil {
t.Error("Expected to make no proposal, since no privValidator")
}
}
func TestMakeProposalEmptyMempool(t *testing.T) {
cs, privAccounts := makeConsensusState()
priv := NewPrivValidator(privAccounts[0], db_.NewMemDB())
cs.SetPrivValidator(priv)
cs.MakeProposal()
rs := cs.GetRoundState()
t.Log(rs.Proposal)
}

View File

@ -25,5 +25,5 @@ func makeVoteSet(height uint32, round uint16, numValidators int, votingPower uin
privAccounts[i] = privAccount
}
valSet := state.NewValidatorSet(vals)
return NewVoteSet(height, round, VoteTypeBare, valSet), valSet, privAccounts
return NewVoteSet(height, round, VoteTypePrevote, valSet), valSet, privAccounts
}

View File

@ -14,7 +14,7 @@ import (
// VoteSet helps collect signatures from validators at each height+round
// for a predefined vote type.
// Note that there three kinds of votes: (bare) votes, precommits, and commits.
// 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.
// TODO: test majority calculations etc.
type VoteSet struct {
@ -51,7 +51,7 @@ func NewVoteSet(height uint32, round uint16, type_ byte, vset *state.ValidatorSe
// True if added, false if not.
// Returns ErrVote[UnexpectedPhase|InvalidAccount|InvalidSignature|InvalidBlockHash|ConflictingSignature]
// NOTE: vote should be mutated after adding.
// NOTE: vote should not be mutated after adding.
func (vs *VoteSet) Add(vote *Vote) (bool, error) {
vs.mtx.Lock()
defer vs.mtx.Unlock()
@ -126,7 +126,7 @@ func (vs *VoteSet) BitArray() BitArray {
return vs.votesBitArray.Copy()
}
func (vs *VoteSet) GetVote(id uint64) *Vote {
func (vs *VoteSet) Get(id uint64) *Vote {
vs.mtx.Lock()
defer vs.mtx.Unlock()
return vs.votes[id]
@ -142,6 +142,15 @@ func (vs *VoteSet) AllVotes() []*Vote {
return votes
}
func (vs *VoteSet) HasTwoThirdsMajority() bool {
if vs == nil {
return false
}
vs.mtx.Lock()
defer vs.mtx.Unlock()
return !vs.twoThirdsCommitTime.IsZero()
}
// Returns either a blockhash (or nil) that received +2/3 majority.
// If there exists no such majority, returns (nil, false).
func (vs *VoteSet) TwoThirdsMajority() (hash []byte, commitTime time.Time, ok bool) {
@ -154,6 +163,9 @@ func (vs *VoteSet) TwoThirdsMajority() (hash []byte, commitTime time.Time, ok bo
}
func (vs *VoteSet) MakePOL() *POL {
if vs.type_ != VoteTypePrevote {
panic("Cannot MakePOL() unless VoteSet.Type is VoteTypePrevote")
}
vs.mtx.Lock()
defer vs.mtx.Unlock()
if vs.twoThirdsCommitTime.IsZero() {
@ -167,7 +179,7 @@ func (vs *VoteSet) MakePOL() *POL {
}
for _, vote := range vs.votes {
if bytes.Equal(vote.BlockHash, majHash) {
if vote.Type == VoteTypeBare {
if vote.Type == VoteTypePrevote {
pol.Votes = append(pol.Votes, vote.Signature)
} else if vote.Type == VoteTypeCommit {
pol.Commits = append(pol.Votes, vote.Signature)
@ -180,6 +192,27 @@ func (vs *VoteSet) MakePOL() *POL {
return pol
}
func (vs *VoteSet) MakeValidation() Validation {
if vs.type_ != VoteTypeCommit {
panic("Cannot MakeValidation() unless VoteSet.Type is VoteTypePrevote")
}
vs.mtx.Lock()
defer vs.mtx.Unlock()
if len(vs.twoThirdsMajority) == 0 {
panic("Cannot MakeValidation() unless a blockhash has +2/3")
}
sigs := []Signature{}
for _, vote := range vs.votes {
if !bytes.Equal(vote.BlockHash, vs.twoThirdsMajority) {
continue
}
sigs = append(sigs, vote.Signature)
}
return Validation{
Signatures: sigs,
}
}
func (vs *VoteSet) String() string {
return vs.StringWithIndent("")
}
@ -189,8 +222,10 @@ func (vs *VoteSet) StringWithIndent(indent string) string {
defer vs.mtx.Unlock()
voteStrings := make([]string, len(vs.votes))
counter := 0
for _, vote := range vs.votes {
voteStrings[vote.SignerId] = vote.String()
voteStrings[counter] = vote.String()
counter++
}
return fmt.Sprintf(`VoteSet{
%s H:%v R:%v T:%v

View File

@ -14,8 +14,8 @@ func TestAddVote(t *testing.T) {
// t.Logf(">> %v", voteSet)
if voteSet.GetVote(0) != nil {
t.Errorf("Expected GetVote(0) to be nil")
if voteSet.Get(0) != nil {
t.Errorf("Expected Get(0) to be nil")
}
if voteSet.BitArray().GetIndex(0) {
t.Errorf("Expected BitArray.GetIndex(0) to be false")
@ -25,12 +25,12 @@ func TestAddVote(t *testing.T) {
t.Errorf("There should be no 2/3 majority")
}
vote := &Vote{Height: 0, Round: 0, Type: VoteTypeBare, BlockHash: nil}
vote := &Vote{Height: 0, Round: 0, Type: VoteTypePrevote, BlockHash: nil}
privAccounts[0].Sign(vote)
voteSet.Add(vote)
if voteSet.GetVote(0) == nil {
t.Errorf("Expected GetVote(0) to be present")
if voteSet.Get(0) == nil {
t.Errorf("Expected Get(0) to be present")
}
if !voteSet.BitArray().GetIndex(0) {
t.Errorf("Expected BitArray.GetIndex(0) to be true")
@ -45,7 +45,7 @@ func Test2_3Majority(t *testing.T) {
voteSet, _, privAccounts := makeVoteSet(0, 0, 10, 1)
// 6 out of 10 voted for nil.
vote := &Vote{Height: 0, Round: 0, Type: VoteTypeBare, BlockHash: nil}
vote := &Vote{Height: 0, Round: 0, Type: VoteTypePrevote, BlockHash: nil}
for i := 0; i < 6; i++ {
privAccounts[i].Sign(vote)
voteSet.Add(vote)
@ -79,7 +79,7 @@ func TestBadVotes(t *testing.T) {
voteSet, _, privAccounts := makeVoteSet(1, 0, 10, 1)
// val0 votes for nil.
vote := &Vote{Height: 1, Round: 0, Type: VoteTypeBare, BlockHash: nil}
vote := &Vote{Height: 1, Round: 0, Type: VoteTypePrevote, BlockHash: nil}
privAccounts[0].Sign(vote)
added, err := voteSet.Add(vote)
if !added || err != nil {
@ -87,7 +87,7 @@ func TestBadVotes(t *testing.T) {
}
// val0 votes again for some block.
vote = &Vote{Height: 1, Round: 0, Type: VoteTypeBare, BlockHash: CRandBytes(32)}
vote = &Vote{Height: 1, Round: 0, Type: VoteTypePrevote, BlockHash: CRandBytes(32)}
privAccounts[0].Sign(vote)
added, err = voteSet.Add(vote)
if added || err == nil {
@ -95,7 +95,7 @@ func TestBadVotes(t *testing.T) {
}
// val1 votes on another height
vote = &Vote{Height: 0, Round: 0, Type: VoteTypeBare, BlockHash: nil}
vote = &Vote{Height: 0, Round: 0, Type: VoteTypePrevote, BlockHash: nil}
privAccounts[1].Sign(vote)
added, err = voteSet.Add(vote)
if added {
@ -103,7 +103,7 @@ func TestBadVotes(t *testing.T) {
}
// val2 votes on another round
vote = &Vote{Height: 1, Round: 1, Type: VoteTypeBare, BlockHash: nil}
vote = &Vote{Height: 1, Round: 1, Type: VoteTypePrevote, BlockHash: nil}
privAccounts[2].Sign(vote)
added, err = voteSet.Add(vote)
if added {
@ -119,11 +119,11 @@ func TestBadVotes(t *testing.T) {
}
}
func TestAddCommitsToBareVotes(t *testing.T) {
func TestAddCommitsToPrevoteVotes(t *testing.T) {
voteSet, _, privAccounts := makeVoteSet(1, 5, 10, 1)
// val0, val1, val2, val3, val4, val5 vote for nil.
vote := &Vote{Height: 1, Round: 5, Type: VoteTypeBare, BlockHash: nil}
vote := &Vote{Height: 1, Round: 5, Type: VoteTypePrevote, BlockHash: nil}
for i := 0; i < 6; i++ {
privAccounts[i].Sign(vote)
voteSet.Add(vote)

View File

@ -17,16 +17,14 @@ import (
)
type Mempool struct {
mtx sync.Mutex
lastBlock *Block
state *state.State
txs []Tx
mtx sync.Mutex
state *state.State
txs []Tx
}
func NewMempool(lastBlock *Block, state *state.State) *Mempool {
func NewMempool(state *state.State) *Mempool {
return &Mempool{
lastBlock: lastBlock,
state: state,
state: state,
}
}
@ -43,14 +41,10 @@ func (mem *Mempool) AddTx(tx Tx) (err error) {
}
}
// Returns a new block from the current state and associated transactions.
// The block's Validation is empty, and some parts of the header too.
func (mem *Mempool) MakeProposalBlock() (*Block, *state.State) {
func (mem *Mempool) GetProposalTxs() ([]Tx, *state.State) {
mem.mtx.Lock()
defer mem.mtx.Unlock()
nextBlock := mem.lastBlock.MakeNextBlock()
nextBlock.Data.Txs = mem.txs
return nextBlock, mem.state
return mem.txs, mem.state
}
// "block" is the new block being committed.
@ -60,7 +54,6 @@ func (mem *Mempool) MakeProposalBlock() (*Block, *state.State) {
func (mem *Mempool) ResetForBlockAndState(block *Block, state *state.State) {
mem.mtx.Lock()
defer mem.mtx.Unlock()
mem.lastBlock = block
mem.state = state.Copy()
// First, create a lookup map of txns in new block.

View File

@ -46,9 +46,9 @@ type State struct {
Height uint32 // Last known block height
BlockHash []byte // Last known block hash
CommitTime time.Time
accountDetails merkle.Tree // Shouldn't be accessed directly.
BondedValidators *ValidatorSet
UnbondingValidators *ValidatorSet
accountDetails merkle.Tree // Shouldn't be accessed directly.
}
func GenesisState(db db_.DB, genesisTime time.Time, accDets []*AccountDetail) *State {
@ -78,9 +78,9 @@ func GenesisState(db db_.DB, genesisTime time.Time, accDets []*AccountDetail) *S
Height: 0,
BlockHash: nil,
CommitTime: genesisTime,
accountDetails: accountDetails,
BondedValidators: NewValidatorSet(validators),
UnbondingValidators: NewValidatorSet(nil),
accountDetails: accountDetails,
}
}
@ -96,11 +96,11 @@ func LoadState(db db_.DB) *State {
s.Height = ReadUInt32(reader, &n, &err)
s.CommitTime = ReadTime(reader, &n, &err)
s.BlockHash = ReadByteSlice(reader, &n, &err)
s.BondedValidators = ReadValidatorSet(reader, &n, &err)
s.UnbondingValidators = ReadValidatorSet(reader, &n, &err)
accountDetailsHash := ReadByteSlice(reader, &n, &err)
s.accountDetails = merkle.NewIAVLTree(BasicCodec, AccountDetailCodec, defaultAccountDetailsCacheCapacity, db)
s.accountDetails.Load(accountDetailsHash)
s.BondedValidators = ReadValidatorSet(reader, &n, &err)
s.UnbondingValidators = ReadValidatorSet(reader, &n, &err)
if err != nil {
panic(err)
}
@ -121,9 +121,9 @@ func (s *State) Save(commitTime time.Time) {
WriteUInt32(&buf, s.Height, &n, &err)
WriteTime(&buf, commitTime, &n, &err)
WriteByteSlice(&buf, s.BlockHash, &n, &err)
WriteByteSlice(&buf, s.accountDetails.Hash(), &n, &err)
WriteBinary(&buf, s.BondedValidators, &n, &err)
WriteBinary(&buf, s.UnbondingValidators, &n, &err)
WriteByteSlice(&buf, s.accountDetails.Hash(), &n, &err)
if err != nil {
panic(err)
}
@ -136,9 +136,9 @@ func (s *State) Copy() *State {
Height: s.Height,
CommitTime: s.CommitTime,
BlockHash: s.BlockHash,
accountDetails: s.accountDetails.Copy(),
BondedValidators: s.BondedValidators.Copy(),
UnbondingValidators: s.UnbondingValidators.Copy(),
accountDetails: s.accountDetails.Copy(),
}
}
@ -421,9 +421,9 @@ func (s *State) SetAccountDetail(accDet *AccountDetail) (updated bool) {
// excluding Height, BlockHash, and CommitTime.
func (s *State) Hash() []byte {
hashables := []merkle.Hashable{
s.accountDetails,
s.BondedValidators,
s.UnbondingValidators,
s.accountDetails,
}
return merkle.HashFromHashables(hashables)
}