Browse Source

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
pull/2736/head
Zarko Milosevic 6 years ago
committed by Ethan Buchman
parent
commit
7a03344480
11 changed files with 286 additions and 76 deletions
  1. +2
    -0
      CHANGELOG_PENDING.md
  2. +1
    -1
      consensus/byzantine_test.go
  3. +5
    -0
      consensus/common_test.go
  4. +50
    -37
      consensus/reactor.go
  5. +31
    -32
      consensus/state.go
  6. +185
    -0
      consensus/state_test.go
  7. +4
    -4
      docs/spec/reactors/consensus/consensus-reactor.md
  8. +1
    -1
      p2p/metrics.go
  9. +4
    -0
      types/event_bus.go
  10. +2
    -0
      types/events.go
  11. +1
    -1
      types/evidence_test.go

+ 2
- 0
CHANGELOG_PENDING.md View File

@ -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


+ 1
- 1
consensus/byzantine_test.go View File

@ -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{}) {


+ 5
- 0
consensus/common_test.go View File

@ -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):


+ 50
- 37
consensus/reactor.go View File

@ -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)
}
//-------------------------------------


+ 31
- 32
consensus/state.go View File

@ -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) {
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 len(blockID.Hash) != 0 && (cs.ValidRound < vote.Round) && (vote.Round == cs.Round) {
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)


+ 185
- 0
consensus/state_test.go View File

@ -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


+ 4
- 4
docs/spec/reactors/consensus/consensus-reactor.md View File

@ -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


+ 1
- 1
p2p/metrics.go View File

@ -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(),


+ 4
- 0
types/event_bus.go View File

@ -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.


+ 2
- 0
types/events.go View File

@ -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)
)


+ 1
- 1
types/evidence_test.go View File

@ -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()


Loading…
Cancel
Save