mirror of
https://github.com/fluencelabs/tendermint
synced 2025-06-23 17:51:39 +00:00
Introduce EventValidBlock for informing peers about wanted block (#2652)
* Introduce EventValidBlock for informing peer about wanted block * Merge with develop * Add isCommit flag to NewValidBlock message - Add test for the case of +2/3 Precommit from the previous round
This commit is contained in:
committed by
Ethan Buchman
parent
a530352f61
commit
7a03344480
@ -102,6 +102,8 @@ Friendly reminder, we have a [bug bounty program](https://hackerone.com/tendermi
|
||||
timeoutPrecommit before starting next round
|
||||
- [consensus] [\#1745](https://github.com/tendermint/tendermint/issues/1745) Wait for
|
||||
Proposal or timeoutProposal before entering prevote
|
||||
- [consensus] [\#2583](https://github.com/tendermint/tendermint/issues/2583) ensure valid
|
||||
block property with faulty proposer
|
||||
- [consensus] [\#2642](https://github.com/tendermint/tendermint/issues/2642) Only propose ValidBlock, not LockedBlock
|
||||
- [consensus] [\#2642](https://github.com/tendermint/tendermint/issues/2642) Initialized ValidRound and LockedRound to -1
|
||||
- [consensus] [\#1637](https://github.com/tendermint/tendermint/issues/1637) Limit the amount of evidence that can be included in a
|
||||
|
@ -263,7 +263,7 @@ func (br *ByzantineReactor) AddPeer(peer p2p.Peer) {
|
||||
// Send our state to peer.
|
||||
// If we're fast_syncing, broadcast a RoundStepMessage later upon SwitchToConsensus().
|
||||
if !br.reactor.fastSync {
|
||||
br.reactor.sendNewRoundStepMessages(peer)
|
||||
br.reactor.sendNewRoundStepMessage(peer)
|
||||
}
|
||||
}
|
||||
func (br *ByzantineReactor) RemovePeer(peer p2p.Peer, reason interface{}) {
|
||||
|
@ -420,6 +420,11 @@ func ensureNewProposal(proposalCh <-chan interface{}, height int64, round int) {
|
||||
"Timeout expired while waiting for NewProposal event")
|
||||
}
|
||||
|
||||
func ensureNewValidBlock(validBlockCh <-chan interface{}, height int64, round int) {
|
||||
ensureNewEvent(validBlockCh, height, round, ensureTimeout,
|
||||
"Timeout expired while waiting for NewValidBlock event")
|
||||
}
|
||||
|
||||
func ensureNewBlock(blockCh <-chan interface{}, height int64) {
|
||||
select {
|
||||
case <-time.After(ensureTimeout):
|
||||
|
@ -174,7 +174,7 @@ func (conR *ConsensusReactor) AddPeer(peer p2p.Peer) {
|
||||
// Send our state to peer.
|
||||
// If we're fast_syncing, broadcast a RoundStepMessage later upon SwitchToConsensus().
|
||||
if !conR.FastSync() {
|
||||
conR.sendNewRoundStepMessages(peer)
|
||||
conR.sendNewRoundStepMessage(peer)
|
||||
}
|
||||
}
|
||||
|
||||
@ -215,8 +215,8 @@ func (conR *ConsensusReactor) Receive(chID byte, src p2p.Peer, msgBytes []byte)
|
||||
switch msg := msg.(type) {
|
||||
case *NewRoundStepMessage:
|
||||
ps.ApplyNewRoundStepMessage(msg)
|
||||
case *CommitStepMessage:
|
||||
ps.ApplyCommitStepMessage(msg)
|
||||
case *NewValidBlockMessage:
|
||||
ps.ApplyNewValidBlockMessage(msg)
|
||||
case *HasVoteMessage:
|
||||
ps.ApplyHasVoteMessage(msg)
|
||||
case *VoteSetMaj23Message:
|
||||
@ -365,7 +365,12 @@ func (conR *ConsensusReactor) subscribeToBroadcastEvents() {
|
||||
const subscriber = "consensus-reactor"
|
||||
conR.conS.evsw.AddListenerForEvent(subscriber, types.EventNewRoundStep,
|
||||
func(data tmevents.EventData) {
|
||||
conR.broadcastNewRoundStepMessages(data.(*cstypes.RoundState))
|
||||
conR.broadcastNewRoundStepMessage(data.(*cstypes.RoundState))
|
||||
})
|
||||
|
||||
conR.conS.evsw.AddListenerForEvent(subscriber, types.EventValidBlock,
|
||||
func(data tmevents.EventData) {
|
||||
conR.broadcastNewValidBlockMessage(data.(*cstypes.RoundState))
|
||||
})
|
||||
|
||||
conR.conS.evsw.AddListenerForEvent(subscriber, types.EventVote,
|
||||
@ -391,14 +396,20 @@ func (conR *ConsensusReactor) broadcastProposalHeartbeatMessage(hb *types.Heartb
|
||||
conR.Switch.Broadcast(StateChannel, cdc.MustMarshalBinaryBare(msg))
|
||||
}
|
||||
|
||||
func (conR *ConsensusReactor) broadcastNewRoundStepMessages(rs *cstypes.RoundState) {
|
||||
nrsMsg, csMsg := makeRoundStepMessages(rs)
|
||||
if nrsMsg != nil {
|
||||
conR.Switch.Broadcast(StateChannel, cdc.MustMarshalBinaryBare(nrsMsg))
|
||||
}
|
||||
if csMsg != nil {
|
||||
conR.Switch.Broadcast(StateChannel, cdc.MustMarshalBinaryBare(csMsg))
|
||||
func (conR *ConsensusReactor) broadcastNewRoundStepMessage(rs *cstypes.RoundState) {
|
||||
nrsMsg := makeRoundStepMessage(rs)
|
||||
conR.Switch.Broadcast(StateChannel, cdc.MustMarshalBinaryBare(nrsMsg))
|
||||
}
|
||||
|
||||
func (conR *ConsensusReactor) broadcastNewValidBlockMessage(rs *cstypes.RoundState) {
|
||||
csMsg := &NewValidBlockMessage{
|
||||
Height: rs.Height,
|
||||
Round: rs.Round,
|
||||
BlockPartsHeader: rs.ProposalBlockParts.Header(),
|
||||
BlockParts: rs.ProposalBlockParts.BitArray(),
|
||||
IsCommit: rs.Step == cstypes.RoundStepCommit,
|
||||
}
|
||||
conR.Switch.Broadcast(StateChannel, cdc.MustMarshalBinaryBare(csMsg))
|
||||
}
|
||||
|
||||
// Broadcasts HasVoteMessage to peers that care.
|
||||
@ -427,33 +438,21 @@ func (conR *ConsensusReactor) broadcastHasVoteMessage(vote *types.Vote) {
|
||||
*/
|
||||
}
|
||||
|
||||
func makeRoundStepMessages(rs *cstypes.RoundState) (nrsMsg *NewRoundStepMessage, csMsg *CommitStepMessage) {
|
||||
func makeRoundStepMessage(rs *cstypes.RoundState) (nrsMsg *NewRoundStepMessage) {
|
||||
nrsMsg = &NewRoundStepMessage{
|
||||
Height: rs.Height,
|
||||
Round: rs.Round,
|
||||
Step: rs.Step,
|
||||
Height: rs.Height,
|
||||
Round: rs.Round,
|
||||
Step: rs.Step,
|
||||
SecondsSinceStartTime: int(time.Since(rs.StartTime).Seconds()),
|
||||
LastCommitRound: rs.LastCommit.Round(),
|
||||
}
|
||||
if rs.Step == cstypes.RoundStepCommit {
|
||||
csMsg = &CommitStepMessage{
|
||||
Height: rs.Height,
|
||||
BlockPartsHeader: rs.ProposalBlockParts.Header(),
|
||||
BlockParts: rs.ProposalBlockParts.BitArray(),
|
||||
}
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
func (conR *ConsensusReactor) sendNewRoundStepMessages(peer p2p.Peer) {
|
||||
func (conR *ConsensusReactor) sendNewRoundStepMessage(peer p2p.Peer) {
|
||||
rs := conR.conS.GetRoundState()
|
||||
nrsMsg, csMsg := makeRoundStepMessages(rs)
|
||||
if nrsMsg != nil {
|
||||
peer.Send(StateChannel, cdc.MustMarshalBinaryBare(nrsMsg))
|
||||
}
|
||||
if csMsg != nil {
|
||||
peer.Send(StateChannel, cdc.MustMarshalBinaryBare(csMsg))
|
||||
}
|
||||
nrsMsg := makeRoundStepMessage(rs)
|
||||
peer.Send(StateChannel, cdc.MustMarshalBinaryBare(nrsMsg))
|
||||
}
|
||||
|
||||
func (conR *ConsensusReactor) gossipDataRoutine(peer p2p.Peer, ps *PeerState) {
|
||||
@ -524,6 +523,7 @@ OUTER_LOOP:
|
||||
msg := &ProposalMessage{Proposal: rs.Proposal}
|
||||
logger.Debug("Sending proposal", "height", prs.Height, "round", prs.Round)
|
||||
if peer.Send(DataChannel, cdc.MustMarshalBinaryBare(msg)) {
|
||||
// NOTE[ZM]: A peer might have received different proposal msg so this Proposal msg will be rejected!
|
||||
ps.SetHasProposal(rs.Proposal)
|
||||
}
|
||||
}
|
||||
@ -964,11 +964,18 @@ func (ps *PeerState) SetHasProposal(proposal *types.Proposal) {
|
||||
if ps.PRS.Height != proposal.Height || ps.PRS.Round != proposal.Round {
|
||||
return
|
||||
}
|
||||
|
||||
if ps.PRS.Proposal {
|
||||
return
|
||||
}
|
||||
|
||||
ps.PRS.Proposal = true
|
||||
|
||||
// ps.PRS.ProposalBlockParts is set due to NewValidBlockMessage
|
||||
if ps.PRS.ProposalBlockParts != nil {
|
||||
return
|
||||
}
|
||||
|
||||
ps.PRS.ProposalBlockPartsHeader = proposal.BlockPartsHeader
|
||||
ps.PRS.ProposalBlockParts = cmn.NewBitArray(proposal.BlockPartsHeader.Total)
|
||||
ps.PRS.ProposalPOLRound = proposal.POLRound
|
||||
@ -1211,7 +1218,6 @@ func (ps *PeerState) ApplyNewRoundStepMessage(msg *NewRoundStepMessage) {
|
||||
// Just remember these values.
|
||||
psHeight := ps.PRS.Height
|
||||
psRound := ps.PRS.Round
|
||||
//psStep := ps.PRS.Step
|
||||
psCatchupCommitRound := ps.PRS.CatchupCommitRound
|
||||
psCatchupCommit := ps.PRS.CatchupCommit
|
||||
|
||||
@ -1252,8 +1258,8 @@ func (ps *PeerState) ApplyNewRoundStepMessage(msg *NewRoundStepMessage) {
|
||||
}
|
||||
}
|
||||
|
||||
// ApplyCommitStepMessage updates the peer state for the new commit.
|
||||
func (ps *PeerState) ApplyCommitStepMessage(msg *CommitStepMessage) {
|
||||
// ApplyNewValidBlockMessage updates the peer state for the new valid block.
|
||||
func (ps *PeerState) ApplyNewValidBlockMessage(msg *NewValidBlockMessage) {
|
||||
ps.mtx.Lock()
|
||||
defer ps.mtx.Unlock()
|
||||
|
||||
@ -1261,6 +1267,10 @@ func (ps *PeerState) ApplyCommitStepMessage(msg *CommitStepMessage) {
|
||||
return
|
||||
}
|
||||
|
||||
if ps.PRS.Round != msg.Round && !msg.IsCommit {
|
||||
return
|
||||
}
|
||||
|
||||
ps.PRS.ProposalBlockPartsHeader = msg.BlockPartsHeader
|
||||
ps.PRS.ProposalBlockParts = msg.BlockParts
|
||||
}
|
||||
@ -1344,7 +1354,7 @@ type ConsensusMessage interface{}
|
||||
func RegisterConsensusMessages(cdc *amino.Codec) {
|
||||
cdc.RegisterInterface((*ConsensusMessage)(nil), nil)
|
||||
cdc.RegisterConcrete(&NewRoundStepMessage{}, "tendermint/NewRoundStepMessage", nil)
|
||||
cdc.RegisterConcrete(&CommitStepMessage{}, "tendermint/CommitStep", nil)
|
||||
cdc.RegisterConcrete(&NewValidBlockMessage{}, "tendermint/NewValidBlockMessage", nil)
|
||||
cdc.RegisterConcrete(&ProposalMessage{}, "tendermint/Proposal", nil)
|
||||
cdc.RegisterConcrete(&ProposalPOLMessage{}, "tendermint/ProposalPOL", nil)
|
||||
cdc.RegisterConcrete(&BlockPartMessage{}, "tendermint/BlockPart", nil)
|
||||
@ -1384,15 +1394,18 @@ func (m *NewRoundStepMessage) String() string {
|
||||
//-------------------------------------
|
||||
|
||||
// CommitStepMessage is sent when a block is committed.
|
||||
type CommitStepMessage struct {
|
||||
type NewValidBlockMessage struct {
|
||||
Height int64
|
||||
Round int
|
||||
BlockPartsHeader types.PartSetHeader
|
||||
BlockParts *cmn.BitArray
|
||||
IsCommit bool
|
||||
}
|
||||
|
||||
// String returns a string representation.
|
||||
func (m *CommitStepMessage) String() string {
|
||||
return fmt.Sprintf("[CommitStep H:%v BP:%v BA:%v]", m.Height, m.BlockPartsHeader, m.BlockParts)
|
||||
func (m *NewValidBlockMessage) String() string {
|
||||
return fmt.Sprintf("[ValidBlockMessage H:%v R:%v BP:%v BA:%v IsCommit:%v]",
|
||||
m.Height, m.Round, m.BlockPartsHeader, m.BlockParts, m.IsCommit)
|
||||
}
|
||||
|
||||
//-------------------------------------
|
||||
|
@ -904,13 +904,6 @@ func (cs *ConsensusState) defaultDecideProposal(height int64, round int) {
|
||||
polRound, polBlockID := cs.Votes.POLInfo()
|
||||
proposal := types.NewProposal(height, round, blockParts.Header(), polRound, polBlockID)
|
||||
if err := cs.privValidator.SignProposal(cs.state.ChainID, proposal); err == nil {
|
||||
// Set fields
|
||||
/* fields set by setProposal and addBlockPart
|
||||
cs.Proposal = proposal
|
||||
cs.ProposalBlock = block
|
||||
cs.ProposalBlockParts = blockParts
|
||||
*/
|
||||
|
||||
// send proposal and block parts on internal msg queue
|
||||
cs.sendInternalMessage(msgInfo{&ProposalMessage{proposal}, ""})
|
||||
for i := 0; i < blockParts.Total(); i++ {
|
||||
@ -994,14 +987,6 @@ func (cs *ConsensusState) enterPrevote(height int64, round int) {
|
||||
cs.newStep()
|
||||
}()
|
||||
|
||||
// fire event for how we got here
|
||||
if cs.isProposalComplete() {
|
||||
cs.eventBus.PublishEventCompleteProposal(cs.RoundStateEvent())
|
||||
} else {
|
||||
// we received +2/3 prevotes for a future round
|
||||
// TODO: catchup event?
|
||||
}
|
||||
|
||||
cs.Logger.Info(fmt.Sprintf("enterPrevote(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
|
||||
|
||||
// Sign and broadcast vote as necessary
|
||||
@ -1240,6 +1225,8 @@ func (cs *ConsensusState) enterCommit(height int64, commitRound int) {
|
||||
// Set up ProposalBlockParts and keep waiting.
|
||||
cs.ProposalBlock = nil
|
||||
cs.ProposalBlockParts = types.NewPartSetFromHeader(blockID.PartsHeader)
|
||||
cs.eventBus.PublishEventValidBlock(cs.RoundStateEvent())
|
||||
cs.evsw.FireEvent(types.EventValidBlock, &cs.RoundState)
|
||||
} else {
|
||||
// We just need to keep waiting.
|
||||
}
|
||||
@ -1420,11 +1407,6 @@ func (cs *ConsensusState) defaultSetProposal(proposal *types.Proposal) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
// We don't care about the proposal if we're already in cstypes.RoundStepCommit.
|
||||
if cstypes.RoundStepCommit <= cs.Step {
|
||||
return nil
|
||||
}
|
||||
|
||||
// Verify POLRound, which must be -1 or between 0 and proposal.Round exclusive.
|
||||
if proposal.POLRound != -1 &&
|
||||
(proposal.POLRound < 0 || proposal.Round <= proposal.POLRound) {
|
||||
@ -1437,7 +1419,12 @@ func (cs *ConsensusState) defaultSetProposal(proposal *types.Proposal) error {
|
||||
}
|
||||
|
||||
cs.Proposal = proposal
|
||||
cs.ProposalBlockParts = types.NewPartSetFromHeader(proposal.BlockPartsHeader)
|
||||
// We don't update cs.ProposalBlockParts if it is already set.
|
||||
// This happens if we're already in cstypes.RoundStepCommit or if there is a valid block in the current round.
|
||||
// TODO: We can check if Proposal is for a different block as this is a sign of misbehavior!
|
||||
if cs.ProposalBlockParts == nil {
|
||||
cs.ProposalBlockParts = types.NewPartSetFromHeader(proposal.BlockPartsHeader)
|
||||
}
|
||||
cs.Logger.Info("Received proposal", "proposal", proposal)
|
||||
return nil
|
||||
}
|
||||
@ -1478,6 +1465,7 @@ func (cs *ConsensusState) addProposalBlockPart(msg *BlockPartMessage, peerID p2p
|
||||
}
|
||||
// NOTE: it's possible to receive complete proposal blocks for future rounds without having the proposal
|
||||
cs.Logger.Info("Received complete proposal block", "height", cs.ProposalBlock.Height, "hash", cs.ProposalBlock.Hash())
|
||||
cs.eventBus.PublishEventCompleteProposal(cs.RoundStateEvent())
|
||||
|
||||
// Update Valid* if we can.
|
||||
prevotes := cs.Votes.Prevotes(cs.Round)
|
||||
@ -1616,16 +1604,26 @@ func (cs *ConsensusState) addVote(vote *types.Vote, peerID p2p.ID) (added bool,
|
||||
|
||||
// Update Valid* if we can.
|
||||
// NOTE: our proposal block may be nil or not what received a polka..
|
||||
// TODO: we may want to still update the ValidBlock and obtain it via gossipping
|
||||
if len(blockID.Hash) != 0 &&
|
||||
(cs.ValidRound < vote.Round) &&
|
||||
(vote.Round <= cs.Round) &&
|
||||
cs.ProposalBlock.HashesTo(blockID.Hash) {
|
||||
if len(blockID.Hash) != 0 && (cs.ValidRound < vote.Round) && (vote.Round == cs.Round) {
|
||||
|
||||
cs.Logger.Info("Updating ValidBlock because of POL.", "validRound", cs.ValidRound, "POLRound", vote.Round)
|
||||
cs.ValidRound = vote.Round
|
||||
cs.ValidBlock = cs.ProposalBlock
|
||||
cs.ValidBlockParts = cs.ProposalBlockParts
|
||||
if cs.ProposalBlock.HashesTo(blockID.Hash) {
|
||||
cs.Logger.Info(
|
||||
"Updating ValidBlock because of POL.", "validRound", cs.ValidRound, "POLRound", vote.Round)
|
||||
cs.ValidRound = vote.Round
|
||||
cs.ValidBlock = cs.ProposalBlock
|
||||
cs.ValidBlockParts = cs.ProposalBlockParts
|
||||
} else {
|
||||
cs.Logger.Info(
|
||||
"Valid block we don't know about. Set ProposalBlock=nil",
|
||||
"proposal", cs.ProposalBlock.Hash(), "blockId", blockID.Hash)
|
||||
// We're getting the wrong block.
|
||||
cs.ProposalBlock = nil
|
||||
}
|
||||
if !cs.ProposalBlockParts.HasHeader(blockID.PartsHeader) {
|
||||
cs.ProposalBlockParts = types.NewPartSetFromHeader(blockID.PartsHeader)
|
||||
}
|
||||
cs.evsw.FireEvent(types.EventValidBlock, &cs.RoundState)
|
||||
cs.eventBus.PublishEventValidBlock(cs.RoundStateEvent())
|
||||
}
|
||||
}
|
||||
|
||||
@ -1634,7 +1632,8 @@ func (cs *ConsensusState) addVote(vote *types.Vote, peerID p2p.ID) (added bool,
|
||||
// Round-skip if there is any 2/3+ of votes ahead of us
|
||||
cs.enterNewRound(height, vote.Round)
|
||||
} else if cs.Round == vote.Round && cstypes.RoundStepPrevote <= cs.Step { // current round
|
||||
if prevotes.HasTwoThirdsMajority() {
|
||||
blockID, ok := prevotes.TwoThirdsMajority()
|
||||
if ok && (cs.isProposalComplete() || len(blockID.Hash) == 0) {
|
||||
cs.enterPrecommit(height, vote.Round)
|
||||
} else if prevotes.HasTwoThirdsAny() {
|
||||
cs.enterPrevoteWait(height, vote.Round)
|
||||
|
@ -966,6 +966,117 @@ func TestProposeValidBlock(t *testing.T) {
|
||||
assert.True(t, bytes.Equal(rs.ProposalBlock.Hash(), rs.ValidBlock.Hash()))
|
||||
}
|
||||
|
||||
// What we want:
|
||||
// P0 miss to lock B but set valid block to B after receiving delayed prevote.
|
||||
func TestSetValidBlockOnDelayedPrevote(t *testing.T) {
|
||||
cs1, vss := randConsensusState(4)
|
||||
vs2, vs3, vs4 := vss[1], vss[2], vss[3]
|
||||
height, round := cs1.Height, cs1.Round
|
||||
|
||||
partSize := types.BlockPartSizeBytes
|
||||
|
||||
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
|
||||
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)
|
||||
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
|
||||
validBlockCh := subscribe(cs1.eventBus, types.EventQueryValidBlock)
|
||||
voteCh := subscribeToVoter(cs1, cs1.privValidator.GetAddress())
|
||||
|
||||
// start round and wait for propose and prevote
|
||||
startTestRound(cs1, cs1.Height, round)
|
||||
ensureNewRound(newRoundCh, height, round)
|
||||
|
||||
ensureNewProposal(proposalCh, height, round)
|
||||
rs := cs1.GetRoundState()
|
||||
propBlock := rs.ProposalBlock
|
||||
propBlockHash := propBlock.Hash()
|
||||
propBlockParts := propBlock.MakePartSet(partSize)
|
||||
|
||||
ensurePrevote(voteCh, height, round)
|
||||
validatePrevote(t, cs1, round, vss[0], propBlockHash)
|
||||
|
||||
// vs2 send prevote for propBlock
|
||||
signAddVotes(cs1, types.PrevoteType, propBlockHash, propBlockParts.Header(), vs2)
|
||||
|
||||
// vs3 send prevote nil
|
||||
signAddVotes(cs1, types.PrevoteType, nil, types.PartSetHeader{}, vs3)
|
||||
|
||||
ensureNewTimeout(timeoutWaitCh, height, round, cs1.config.TimeoutPrevote.Nanoseconds())
|
||||
|
||||
ensurePrecommit(voteCh, height, round)
|
||||
// we should have precommitted
|
||||
validatePrecommit(t, cs1, round, -1, vss[0], nil, nil)
|
||||
|
||||
rs = cs1.GetRoundState()
|
||||
|
||||
assert.True(t, rs.ValidBlock == nil)
|
||||
assert.True(t, rs.ValidBlockParts == nil)
|
||||
assert.True(t, rs.ValidRound == -1)
|
||||
|
||||
// vs2 send (delayed) prevote for propBlock
|
||||
signAddVotes(cs1, types.PrevoteType, propBlockHash, propBlockParts.Header(), vs4)
|
||||
|
||||
ensureNewValidBlock(validBlockCh, height, round)
|
||||
|
||||
rs = cs1.GetRoundState()
|
||||
|
||||
assert.True(t, bytes.Equal(rs.ValidBlock.Hash(), propBlockHash))
|
||||
assert.True(t, rs.ValidBlockParts.Header().Equals(propBlockParts.Header()))
|
||||
assert.True(t, rs.ValidRound == round)
|
||||
}
|
||||
|
||||
// What we want:
|
||||
// P0 miss to lock B as Proposal Block is missing, but set valid block to B after
|
||||
// receiving delayed Block Proposal.
|
||||
func TestSetValidBlockOnDelayedProposal(t *testing.T) {
|
||||
cs1, vss := randConsensusState(4)
|
||||
vs2, vs3, vs4 := vss[1], vss[2], vss[3]
|
||||
height, round := cs1.Height, cs1.Round
|
||||
|
||||
partSize := types.BlockPartSizeBytes
|
||||
|
||||
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)
|
||||
timeoutProposeCh := subscribe(cs1.eventBus, types.EventQueryTimeoutPropose)
|
||||
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
|
||||
validBlockCh := subscribe(cs1.eventBus, types.EventQueryValidBlock)
|
||||
voteCh := subscribeToVoter(cs1, cs1.privValidator.GetAddress())
|
||||
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
|
||||
|
||||
round = round + 1 // move to round in which P0 is not proposer
|
||||
incrementRound(vs2, vs3, vs4)
|
||||
|
||||
startTestRound(cs1, cs1.Height, round)
|
||||
ensureNewRound(newRoundCh, height, round)
|
||||
|
||||
ensureNewTimeout(timeoutProposeCh, height, round, cs1.config.TimeoutPropose.Nanoseconds())
|
||||
|
||||
ensurePrevote(voteCh, height, round)
|
||||
validatePrevote(t, cs1, round, vss[0], nil)
|
||||
|
||||
prop, propBlock := decideProposal(cs1, vs2, vs2.Height, vs2.Round+1)
|
||||
propBlockHash := propBlock.Hash()
|
||||
propBlockParts := propBlock.MakePartSet(partSize)
|
||||
|
||||
// vs2, vs3 and vs4 send prevote for propBlock
|
||||
signAddVotes(cs1, types.PrevoteType, propBlockHash, propBlockParts.Header(), vs2, vs3, vs4)
|
||||
ensureNewValidBlock(validBlockCh, height, round)
|
||||
|
||||
ensureNewTimeout(timeoutWaitCh, height, round, cs1.config.TimeoutPrevote.Nanoseconds())
|
||||
|
||||
ensurePrecommit(voteCh, height, round)
|
||||
validatePrecommit(t, cs1, round, -1, vss[0], nil, nil)
|
||||
|
||||
if err := cs1.SetProposalAndBlock(prop, propBlock, propBlockParts, "some peer"); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
ensureNewProposal(proposalCh, height, round)
|
||||
rs := cs1.GetRoundState()
|
||||
|
||||
assert.True(t, bytes.Equal(rs.ValidBlock.Hash(), propBlockHash))
|
||||
assert.True(t, rs.ValidBlockParts.Header().Equals(propBlockParts.Header()))
|
||||
assert.True(t, rs.ValidRound == round)
|
||||
}
|
||||
|
||||
// 4 vals, 3 Nil Precommits at P0
|
||||
// What we want:
|
||||
// P0 waits for timeoutPrecommit before starting next round
|
||||
@ -1078,6 +1189,80 @@ func TestWaitTimeoutProposeOnNilPolkaForTheCurrentRound(t *testing.T) {
|
||||
validatePrevote(t, cs1, round, vss[0], nil)
|
||||
}
|
||||
|
||||
// What we want:
|
||||
// P0 emit NewValidBlock event upon receiving 2/3+ Precommit for B but hasn't received block B yet
|
||||
func TestEmitNewValidBlockEventOnCommitWithoutBlock(t *testing.T) {
|
||||
cs1, vss := randConsensusState(4)
|
||||
vs2, vs3, vs4 := vss[1], vss[2], vss[3]
|
||||
height, round := cs1.Height, 1
|
||||
|
||||
incrementRound(vs2, vs3, vs4)
|
||||
|
||||
partSize := types.BlockPartSizeBytes
|
||||
|
||||
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
|
||||
validBlockCh := subscribe(cs1.eventBus, types.EventQueryValidBlock)
|
||||
|
||||
_, propBlock := decideProposal(cs1, vs2, vs2.Height, vs2.Round)
|
||||
propBlockHash := propBlock.Hash()
|
||||
propBlockParts := propBlock.MakePartSet(partSize)
|
||||
|
||||
// start round in which PO is not proposer
|
||||
startTestRound(cs1, height, round)
|
||||
ensureNewRound(newRoundCh, height, round)
|
||||
|
||||
// vs2, vs3 and vs4 send precommit for propBlock
|
||||
signAddVotes(cs1, types.PrecommitType, propBlockHash, propBlockParts.Header(), vs2, vs3, vs4)
|
||||
ensureNewValidBlock(validBlockCh, height, round)
|
||||
|
||||
rs := cs1.GetRoundState()
|
||||
assert.True(t, rs.Step == cstypes.RoundStepCommit)
|
||||
assert.True(t, rs.ProposalBlock == nil)
|
||||
assert.True(t, rs.ProposalBlockParts.Header().Equals(propBlockParts.Header()))
|
||||
|
||||
}
|
||||
|
||||
// What we want:
|
||||
// P0 receives 2/3+ Precommit for B for round 0, while being in round 1. It emits NewValidBlock event.
|
||||
// After receiving block, it executes block and moves to the next height.
|
||||
func TestCommitFromPreviousRound(t *testing.T) {
|
||||
cs1, vss := randConsensusState(4)
|
||||
vs2, vs3, vs4 := vss[1], vss[2], vss[3]
|
||||
height, round := cs1.Height, 1
|
||||
|
||||
partSize := types.BlockPartSizeBytes
|
||||
|
||||
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
|
||||
validBlockCh := subscribe(cs1.eventBus, types.EventQueryValidBlock)
|
||||
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
|
||||
|
||||
prop, propBlock := decideProposal(cs1, vs2, vs2.Height, vs2.Round)
|
||||
propBlockHash := propBlock.Hash()
|
||||
propBlockParts := propBlock.MakePartSet(partSize)
|
||||
|
||||
// start round in which PO is not proposer
|
||||
startTestRound(cs1, height, round)
|
||||
ensureNewRound(newRoundCh, height, round)
|
||||
|
||||
// vs2, vs3 and vs4 send precommit for propBlock for the previous round
|
||||
signAddVotes(cs1, types.PrecommitType, propBlockHash, propBlockParts.Header(), vs2, vs3, vs4)
|
||||
|
||||
ensureNewValidBlock(validBlockCh, height, round)
|
||||
|
||||
rs := cs1.GetRoundState()
|
||||
assert.True(t, rs.Step == cstypes.RoundStepCommit)
|
||||
assert.True(t, rs.CommitRound == vs2.Round)
|
||||
assert.True(t, rs.ProposalBlock == nil)
|
||||
assert.True(t, rs.ProposalBlockParts.Header().Equals(propBlockParts.Header()))
|
||||
|
||||
if err := cs1.SetProposalAndBlock(prop, propBlock, propBlockParts, "some peer"); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
ensureNewProposal(proposalCh, height, round)
|
||||
ensureNewRound(newRoundCh, height+1, 0)
|
||||
}
|
||||
|
||||
//------------------------------------------------------------------------------------------
|
||||
// SlashingSuite
|
||||
// TODO: Slashing
|
||||
|
@ -129,11 +129,11 @@ handleMessage(msg):
|
||||
Reset prs.CatchupCommitRound and prs.CatchupCommit
|
||||
```
|
||||
|
||||
### CommitStepMessage handler
|
||||
### NewValidBlockMessage handler
|
||||
|
||||
```
|
||||
handleMessage(msg):
|
||||
if prs.Height == msg.Height then
|
||||
if prs.Height == msg.Height && prs.Round == msg.Round then
|
||||
prs.ProposalBlockPartsHeader = msg.BlockPartsHeader
|
||||
prs.ProposalBlockParts = msg.BlockParts
|
||||
```
|
||||
@ -161,8 +161,8 @@ handleMessage(msg):
|
||||
handleMessage(msg):
|
||||
if prs.Height != msg.Height || prs.Round != msg.Round || prs.Proposal then return
|
||||
prs.Proposal = true
|
||||
prs.ProposalBlockPartsHeader = msg.BlockPartsHeader
|
||||
prs.ProposalBlockParts = empty set
|
||||
if prs.ProposalBlockParts == empty set then // otherwise it is set in NewValidBlockMessage handler
|
||||
prs.ProposalBlockPartsHeader = msg.BlockPartsHeader
|
||||
prs.ProposalPOLRound = msg.POLRound
|
||||
prs.ProposalPOL = nil
|
||||
Send msg through internal peerMsgQueue to ConsensusState service
|
||||
|
@ -62,7 +62,7 @@ func PrometheusMetrics(namespace string) *Metrics {
|
||||
// NopMetrics returns no-op Metrics.
|
||||
func NopMetrics() *Metrics {
|
||||
return &Metrics{
|
||||
Peers: discard.NewGauge(),
|
||||
Peers: discard.NewGauge(),
|
||||
PeerReceiveBytesTotal: discard.NewCounter(),
|
||||
PeerSendBytesTotal: discard.NewCounter(),
|
||||
PeerPendingSendBytes: discard.NewGauge(),
|
||||
|
@ -83,6 +83,10 @@ func (b *EventBus) PublishEventVote(data EventDataVote) error {
|
||||
return b.Publish(EventVote, data)
|
||||
}
|
||||
|
||||
func (b *EventBus) PublishEventValidBlock(data EventDataRoundState) error {
|
||||
return b.Publish(EventValidBlock, data)
|
||||
}
|
||||
|
||||
// PublishEventTx publishes tx event with tags from Result. Note it will add
|
||||
// predefined tags (EventTypeKey, TxHashKey). Existing tags with the same names
|
||||
// will be overwritten.
|
||||
|
@ -23,6 +23,7 @@ const (
|
||||
EventTimeoutWait = "TimeoutWait"
|
||||
EventTx = "Tx"
|
||||
EventUnlock = "Unlock"
|
||||
EventValidBlock = "ValidBlock"
|
||||
EventValidatorSetUpdates = "ValidatorSetUpdates"
|
||||
EventVote = "Vote"
|
||||
)
|
||||
@ -119,6 +120,7 @@ var (
|
||||
EventQueryTx = QueryForEvent(EventTx)
|
||||
EventQueryUnlock = QueryForEvent(EventUnlock)
|
||||
EventQueryValidatorSetUpdates = QueryForEvent(EventValidatorSetUpdates)
|
||||
EventQueryValidBlock = QueryForEvent(EventValidBlock)
|
||||
EventQueryVote = QueryForEvent(EventVote)
|
||||
)
|
||||
|
||||
|
@ -61,7 +61,7 @@ func TestEvidence(t *testing.T) {
|
||||
{vote1, makeVote(val, chainID, 0, 10, 3, 1, blockID2), false}, // wrong round
|
||||
{vote1, makeVote(val, chainID, 0, 10, 2, 2, blockID2), false}, // wrong step
|
||||
{vote1, makeVote(val2, chainID, 0, 10, 2, 1, blockID), false}, // wrong validator
|
||||
{vote1, badVote, false}, // signed by wrong key
|
||||
{vote1, badVote, false}, // signed by wrong key
|
||||
}
|
||||
|
||||
pubKey := val.GetPubKey()
|
||||
|
Reference in New Issue
Block a user