From c72272cab9ea5ccf2af4e7df11a5c2e265e42024 Mon Sep 17 00:00:00 2001 From: Teddy Ding Date: Tue, 5 Sep 2023 21:52:50 -0400 Subject: [PATCH] [Cherry-pick #1175]: remove logic to unlock block on +2/3 prevote for nil (#19) * cp #1175: remove logic to unlock block on +2/3 prevote for nil * fix lint * fix test --- consensus/common_test.go | 13 +- consensus/state.go | 68 ++----- consensus/state_test.go | 398 +++++++++++++++++++++++--------------- internal/test/block.go | 2 +- spec/consensus/signing.md | 10 +- types/block.go | 6 +- types/canonical.go | 2 +- types/event_bus.go | 8 - types/event_bus_test.go | 6 +- types/events.go | 2 - types/vote.go | 4 +- types/vote_set_test.go | 20 +- 12 files changed, 280 insertions(+), 259 deletions(-) diff --git a/consensus/common_test.go b/consensus/common_test.go index 02235cb2c..49a9935c5 100644 --- a/consensus/common_test.go +++ b/consensus/common_test.go @@ -510,13 +510,6 @@ func ensureNoNewRoundStep(stepCh <-chan cmtpubsub.Message) { "We should be stuck waiting, not receiving NewRoundStep event") } -func ensureNoNewUnlock(unlockCh <-chan cmtpubsub.Message) { - ensureNoNewEvent( - unlockCh, - ensureTimeout, - "We should be stuck waiting, not receiving Unlock event") -} - func ensureNoNewTimeout(stepCh <-chan cmtpubsub.Message, timeout int64) { timeoutDuration := time.Duration(timeout*10) * time.Nanosecond ensureNoNewEvent( @@ -629,9 +622,9 @@ func ensureNewBlockHeader(blockCh <-chan cmtpubsub.Message, height int64, blockH } } -func ensureNewUnlock(unlockCh <-chan cmtpubsub.Message, height int64, round int32) { - ensureNewEvent(unlockCh, height, round, ensureTimeout, - "Timeout expired while waiting for NewUnlock event") +func ensureLock(lockCh <-chan cmtpubsub.Message, height int64, round int32) { + ensureNewEvent(lockCh, height, round, ensureTimeout, + "Timeout expired while waiting for LockValue event") } func ensureProposal(proposalCh <-chan cmtpubsub.Message, height int64, round int32, propID types.BlockID) { diff --git a/consensus/state.go b/consensus/state.go index 405788aa7..27b30d150 100644 --- a/consensus/state.go +++ b/consensus/state.go @@ -1417,7 +1417,6 @@ func (cs *State) enterPrevoteWait(height int64, round int32) { // Enter: `timeoutPrecommit` after any +2/3 precommits. // Enter: +2/3 precomits for block or nil. // Lock & precommit the ProposalBlock if we have enough prevotes for it (a POL in this round) -// else, unlock an existing lock and precommit nil if +2/3 of prevotes were nil, // else, precommit nil otherwise. func (cs *State) enterPrecommit(height int64, round int32) { logger := cs.Logger.With("height", height, "round", round) @@ -1464,21 +1463,9 @@ func (cs *State) enterPrecommit(height int64, round int32) { panic(fmt.Sprintf("this POLRound should be %v but got %v", round, polRound)) } - // +2/3 prevoted nil. Unlock and precommit nil. - if len(blockID.Hash) == 0 { - if cs.LockedBlock == nil { - logger.Debug("precommit step; +2/3 prevoted for nil") - } else { - logger.Debug("precommit step; +2/3 prevoted for nil; unlocking") - cs.LockedRound = -1 - cs.LockedBlock = nil - cs.LockedBlockParts = nil - - if err := cs.eventBus.PublishEventUnlock(cs.RoundStateEvent()); err != nil { - logger.Error("failed publishing event unlock", "err", err) - } - } - + // +2/3 prevoted nil. Precommit nil. + if blockID.IsNil() { + logger.Debug("precommit step; +2/3 prevoted for nil") cs.signAddVote(cmtproto.PrecommitType, nil, types.PartSetHeader{}) return } @@ -1498,7 +1485,9 @@ func (cs *State) enterPrecommit(height int64, round int32) { return } - // If +2/3 prevoted for proposal block, stage and precommit it + // If greater than 2/3 of the voting power on the network prevoted for + // the proposed block, update our locked block to this block and issue a + // precommit vote for it. if cs.ProposalBlock.HashesTo(blockID.Hash) { logger.Debug("precommit step; +2/3 prevoted proposal block; locking", "hash", blockID.Hash) @@ -1520,23 +1509,14 @@ func (cs *State) enterPrecommit(height int64, round int32) { } // There was a polka in this round for a block we don't have. - // Fetch that block, unlock, and precommit nil. - // The +2/3 prevotes for this round is the POL for our unlock. + // Fetch that block, and precommit nil. logger.Debug("precommit step; +2/3 prevotes for a block we do not have; voting nil", "block_id", blockID) - cs.LockedRound = -1 - cs.LockedBlock = nil - cs.LockedBlockParts = nil - if !cs.ProposalBlockParts.HasHeader(blockID.PartSetHeader) { cs.ProposalBlock = nil cs.ProposalBlockParts = types.NewPartSetFromHeader(blockID.PartSetHeader) } - if err := cs.eventBus.PublishEventUnlock(cs.RoundStateEvent()); err != nil { - logger.Error("failed publishing event unlock", "err", err) - } - cs.signAddVote(cmtproto.PrecommitType, nil, types.PartSetHeader{}) } @@ -1644,7 +1624,7 @@ func (cs *State) tryFinalizeCommit(height int64) { } blockID, ok := cs.Votes.Precommits(cs.CommitRound).TwoThirdsMajority() - if !ok || len(blockID.Hash) == 0 { + if !ok || blockID.IsNil() { logger.Error("failed attempt to finalize commit; there was no +2/3 majority or +2/3 was for nil") return } @@ -2019,7 +1999,7 @@ func (cs *State) handleCompleteProposal(blockHeight int64) { // Update Valid* if we can. prevotes := cs.Votes.Prevotes(cs.Round) blockID, hasTwoThirds := prevotes.TwoThirdsMajority() - if hasTwoThirds && !blockID.IsZero() && (cs.ValidRound < cs.Round) { + if hasTwoThirds && !blockID.IsNil() && (cs.ValidRound < cs.Round) { if cs.ProposalBlock.HashesTo(blockID.Hash) { cs.Logger.Debug( "updating valid block to new proposal block", @@ -2172,33 +2152,13 @@ func (cs *State) addVote(vote *types.Vote, peerID p2p.ID) (added bool, err error prevotes := cs.Votes.Prevotes(vote.Round) cs.Logger.Debug("added vote to prevote", "vote", vote, "prevotes", prevotes.StringShort()) - // If +2/3 prevotes for a block or nil for *any* round: - if blockID, ok := prevotes.TwoThirdsMajority(); ok { - // There was a polka! - // If we're locked but this is a recent polka, unlock. - // If it matches our ProposalBlock, update the ValidBlock - - // Unlock if `cs.LockedRound < vote.Round <= cs.Round` - // NOTE: If vote.Round > cs.Round, we'll deal with it when we get to vote.Round - if (cs.LockedBlock != nil) && - (cs.LockedRound < vote.Round) && - (vote.Round <= cs.Round) && - !cs.LockedBlock.HashesTo(blockID.Hash) { - - cs.Logger.Debug("unlocking because of POL", "locked_round", cs.LockedRound, "pol_round", vote.Round) - - cs.LockedRound = -1 - cs.LockedBlock = nil - cs.LockedBlockParts = nil - - if err := cs.eventBus.PublishEventUnlock(cs.RoundStateEvent()); err != nil { - return added, err - } - } + // Check to see if >2/3 of the voting power on the network voted for any non-nil block. + if blockID, ok := prevotes.TwoThirdsMajority(); ok && !blockID.IsNil() { + // Greater than 2/3 of the voting power on the network voted for some + // non-nil block // Update Valid* if we can. - // NOTE: our proposal block may be nil or not what received a polka.. - if len(blockID.Hash) != 0 && (cs.ValidRound < vote.Round) && (vote.Round == cs.Round) { + if cs.ValidRound < vote.Round && vote.Round == cs.Round { if cs.ProposalBlock.HashesTo(blockID.Hash) { cs.Logger.Debug("updating valid block because of POL", "valid_round", cs.ValidRound, "pol_round", vote.Round) cs.ValidRound = vote.Round diff --git a/consensus/state_test.go b/consensus/state_test.go index 298dd9077..f061c862a 100644 --- a/consensus/state_test.go +++ b/consensus/state_test.go @@ -40,11 +40,17 @@ x * TestFullRound1 - 1 val, full successful round x * TestFullRoundNil - 1 val, full round of nil x * TestFullRound2 - 2 vals, both required for full round LockSuite -x * TestLockNoPOL - 2 vals, 4 rounds. one val locked, precommits nil every round except first. -x * TestLockPOLRelock - 4 vals, one precommits, other 3 polka at next round, so we unlock and precomit the polka -x * TestLockPOLUnlock - 4 vals, one precommits, other 3 polka nil at next round, so we unlock and precomit nil -x * TestLockPOLSafety1 - 4 vals. We shouldn't change lock based on polka at earlier round -x * TestLockPOLSafety2 - 4 vals. After unlocking, we shouldn't relock based on polka at earlier round +x * TestStateLockNoPOL - 2 vals, 4 rounds. one val locked, precommits nil every round except first. +x * TestStateLockPOLUpdateLock - 4 vals, one precommits, +other 3 polka at next round, so we unlock and precomit the polka +x * TestStateLockPOLRelock - 4 vals, polka in round 1 and polka in round 2. +Ensure validator updates locked round. +x * TestStateLockPOLDoesNotUnlock 4 vals, one precommits, other 3 polka nil at +next round, so we precommit nil but maintain lock +x * TestStateLockMissingProposalWhenPOLSeenDoesNotUpdateLock - 4 vals, 1 misses proposal but sees POL. +x * TestStateLockMissingProposalWhenPOLSeenDoesNotUnlock - 4 vals, 1 misses proposal but sees POL. +x * TestStateLockPOLSafety1 - 4 vals. We shouldn't change lock based on polka at earlier round +x * TestStateLockPOLSafety2 - 4 vals. After unlocking, we shouldn't relock based on polka at earlier round * TestNetworkLock - once +1/3 precommits, network should be locked * TestNetworkLockPOL - once +1/3 precommits, the block with more recent polka is committed SlashingSuite @@ -128,8 +134,7 @@ func TestStateProposerSelection2(t *testing.T) { prop.Address)) } - rs := cs1.GetRoundState() - signAddVotes(cs1, cmtproto.PrecommitType, nil, rs.ProposalBlockParts.Header(), vss[1:]...) + signAddVotes(cs1, cmtproto.PrecommitType, nil, types.PartSetHeader{}, vss[1:]...) ensureNewRound(newRoundCh, height, i+round+1) // wait for the new round event each round incrementRound(vss[1:]...) } @@ -635,11 +640,15 @@ func TestStateLockNoPOL(t *testing.T) { ensurePrecommit(voteCh, height, round) } -// 4 vals in two rounds, -// in round one: v1 precommits, other 3 only prevote so the block isn't committed -// in round two: v1 prevotes the same block that the node is locked on -// the others prevote a new block hence v1 changes lock and precommits the new block with the others -func TestStateLockPOLRelock(t *testing.T) { +// TestStateLockPOLUpdateLock tests that a validator maintains updates its locked +// block if the following conditions are met within a round: +// 1. The validator received a valid proposal for the block +// 2. The validator received prevotes representing greater than 2/3 of the voting +// power on the network for the block. +func TestStateLockPOLUpdateLock(t *testing.T) { + _, cancel := context.WithCancel(context.Background()) + defer cancel() + cs1, vss := randState(4) vs2, vs3, vs4 := vss[1], vss[2], vss[3] height, round := cs1.Height, cs1.Round @@ -652,18 +661,18 @@ func TestStateLockPOLRelock(t *testing.T) { require.NoError(t, err) addr := pv1.Address() voteCh := subscribeToVoter(cs1, addr) + lockCh := subscribe(cs1.eventBus, types.EventQueryLock) newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound) - newBlockCh := subscribe(cs1.eventBus, types.EventQueryNewBlockHeader) - - // everything done from perspective of cs1 /* - Round1 (cs1, B) // B B B B// B nil B nil - - eg. vs2 and vs4 didn't see the 2/3 prevotes + Round 0: + cs1 creates a proposal for block B. + Send a prevote for B from each of the validators to cs1. + Send a precommit for nil from all of the validators to cs1. + This ensures that cs1 will lock on B in this round but not precommit it. */ + t.Log("### Starting Round 0") - // start round and wait for propose and prevote startTestRound(cs1, height, round) ensureNewRound(newRoundCh, height, round) @@ -672,94 +681,106 @@ func TestStateLockPOLRelock(t *testing.T) { theBlockHash := rs.ProposalBlock.Hash() theBlockParts := rs.ProposalBlockParts.Header() - ensurePrevote(voteCh, height, round) // prevote + ensurePrevote(voteCh, height, round) signAddVotes(cs1, cmtproto.PrevoteType, theBlockHash, theBlockParts, vs2, vs3, vs4) + // check that the validator generates a Lock event. + ensureLock(lockCh, height, round) - ensurePrecommit(voteCh, height, round) // our precommit - // the proposed block should now be locked and our precommit added + // the proposed block should now be locked and our precommit added. + ensurePrecommit(voteCh, height, round) validatePrecommit(t, cs1, round, round, vss[0], theBlockHash, theBlockHash) - // add precommits from the rest + // add precommits from the rest of the validators. signAddVotes(cs1, cmtproto.PrecommitType, nil, types.PartSetHeader{}, vs2, vs3, vs4) - // before we timeout to the new round set the new proposal - cs2 := newState(cs1.state, vs2, kvstore.NewApplication()) - prop, propBlock := decideProposal(t, cs2, vs2, vs2.Height, vs2.Round+1) - if prop == nil || propBlock == nil { - t.Fatal("Failed to create proposal block with vs2") - } - propBlockParts, err := propBlock.MakePartSet(partSize) - require.NoError(t, err) - - propBlockHash := propBlock.Hash() - require.NotEqual(t, propBlockHash, theBlockHash) + // timeout to new round. + ensureNewTimeout(timeoutWaitCh, height, round, cs1.config.Precommit(round).Nanoseconds()) + /* + Round 1: + Create a block, D and send a proposal for it to cs1 + Send a prevote for D from each of the validators to cs1. + Send a precommit for nil from all of the validtors to cs1. + Check that cs1 is now locked on the new block, D and no longer on the old block. + */ + t.Log("### Starting Round 1") incrementRound(vs2, vs3, vs4) + round++ - // timeout to new round - ensureNewTimeout(timeoutWaitCh, height, round, cs1.config.Precommit(round).Nanoseconds()) - - round++ // moving to the next round - //XXX: this isnt guaranteed to get there before the timeoutPropose ... - if err := cs1.SetProposalAndBlock(prop, propBlock, propBlockParts, "some peer"); err != nil { + // Generate a new proposal block. + cs2 := newState(cs1.state, vs2, kvstore.NewApplication()) + propR1, propBlockR1 := decideProposal(t, cs2, vs2, vs2.Height, vs2.Round) + propBlockR1Parts, err := propBlockR1.MakePartSet(partSize) + require.NoError(t, err) + propBlockR1Hash := propBlockR1.Hash() + require.NotEqual(t, propBlockR1Hash, theBlockHash) + if err := cs1.SetProposalAndBlock(propR1, propBlockR1, propBlockR1Parts, "some peer"); err != nil { t.Fatal(err) } ensureNewRound(newRoundCh, height, round) - t.Log("### ONTO ROUND 1") - - /* - Round2 (vs2, C) // B C C C // C C C _) - cs1 changes lock! - */ - - // now we're on a new round and not the proposer - // but we should receive the proposal + // ensure that the validator receives the proposal. ensureNewProposal(proposalCh, height, round) - // go to prevote, node should prevote for locked block (not the new proposal) - this is relocking + // Prevote our locked block. + // TODO: Ensure we prevote for the proposal if it is valid and from a round greater than + // the valid round: https://github.com/tendermint/tendermint/issues/6850. ensurePrevote(voteCh, height, round) validatePrevote(t, cs1, round, vss[0], theBlockHash) - // now lets add prevotes from everyone else for the new block - signAddVotes(cs1, cmtproto.PrevoteType, propBlockHash, propBlockParts.Header(), vs2, vs3, vs4) + // Add prevotes from the remainder of the validators for the new locked block. + signAddVotes(cs1, cmtproto.PrevoteType, propBlockR1Hash, propBlockR1Parts.Header(), vs2, vs3, vs4) - ensurePrecommit(voteCh, height, round) - // we should have unlocked and locked on the new block, sending a precommit for this new block - validatePrecommit(t, cs1, round, round, vss[0], propBlockHash, propBlockHash) + // Check that we lock on a new block. + ensureLock(lockCh, height, round) - // more prevote creating a majority on the new block and this is then committed - signAddVotes(cs1, cmtproto.PrecommitType, propBlockHash, propBlockParts.Header(), vs2, vs3) - ensureNewBlockHeader(newBlockCh, height, propBlockHash) + ensurePrecommit(voteCh, height, round) - ensureNewRound(newRoundCh, height+1, 0) + // We should now be locked on the new block and prevote it since we saw a sufficient amount + // prevote for the block. + validatePrecommit(t, cs1, round, round, vss[0], propBlockR1Hash, propBlockR1Hash) } -// 4 vals, one precommits, other 3 polka at next round, so we unlock and precomit the polka -func TestStateLockPOLUnlock(t *testing.T) { +// TestStateLockPOLDoesNotUnlock tests that a validator maintains its locked block +// despite receiving +2/3 nil prevotes and nil precommits from other validators. +// Tendermint used to 'unlock' its locked block when greater than 2/3 prevotes +// for a nil block were seen. This behavior has been removed and this test ensures +// that it has been completely removed. +func TestStateLockPOLDoesNotUnlock(t *testing.T) { + _, cancel := context.WithCancel(context.Background()) + defer cancel() + + /* + All of the assertions in this test occur on the `cs1` validator. + The test sends signed votes from the other validators to cs1 and + cs1's state is then examined to verify that it now matches the expected + state. + */ + cs1, vss := randState(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) - unlockCh := subscribe(cs1.eventBus, types.EventQueryUnlock) + lockCh := subscribe(cs1.eventBus, types.EventQueryLock) pv1, err := cs1.privValidator.GetPubKey() require.NoError(t, err) addr := pv1.Address() voteCh := subscribeToVoter(cs1, addr) - // everything done from perspective of cs1 - /* - Round1 (cs1, B) // B B B B // B nil B nil - eg. didn't see the 2/3 prevotes + Round 0: + Create a block, B + Send a prevote for B from each of the validators to `cs1`. + Send a precommit for B from one of the validtors to `cs1`. + + This ensures that cs1 will lock on B in this round. */ + t.Log("#### ONTO ROUND 0") // start round and wait for propose and prevote startTestRound(cs1, height, round) @@ -775,63 +796,101 @@ func TestStateLockPOLUnlock(t *testing.T) { signAddVotes(cs1, cmtproto.PrevoteType, theBlockHash, theBlockParts, vs2, vs3, vs4) + // the validator should have locked a block in this round. + ensureLock(lockCh, height, round) + ensurePrecommit(voteCh, height, round) - // the proposed block should now be locked and our precommit added + // the proposed block should now be locked and our should be for this locked block. + validatePrecommit(t, cs1, round, round, vss[0], theBlockHash, theBlockHash) - // add precommits from the rest + // Add precommits from the other validators. + // We only issue 1/2 Precommits for the block in this round. + // This ensures that the validator being tested does not commit the block. + // We do not want the validator to commit the block because we want the test + // test to proceeds to the next consensus round. signAddVotes(cs1, cmtproto.PrecommitType, nil, types.PartSetHeader{}, vs2, vs4) signAddVotes(cs1, cmtproto.PrecommitType, theBlockHash, theBlockParts, vs3) - // before we time out into new round, set next proposal block - prop, propBlock := decideProposal(t, cs1, vs2, vs2.Height, vs2.Round+1) - propBlockParts, err := propBlock.MakePartSet(partSize) - require.NoError(t, err) - // timeout to new round ensureNewTimeout(timeoutWaitCh, height, round, cs1.config.Precommit(round).Nanoseconds()) - rs = cs1.GetRoundState() - lockedBlockHash := rs.LockedBlock.Hash() + /* + Round 1: + Send a prevote for nil from >2/3 of the validators to `cs1`. + Check that cs1 maintains its lock on B but precommits nil. + Send a precommit for nil from >2/3 of the validators to `cs1`. + */ + t.Log("#### ONTO ROUND 1") + round++ incrementRound(vs2, vs3, vs4) - round++ // moving to the next round + prop, propBlock := decideProposal(t, cs1, vs2, vs2.Height, vs2.Round) + propBlockParts, err := propBlock.MakePartSet(types.BlockPartSizeBytes) + require.NoError(t, err) + if err := cs1.SetProposalAndBlock(prop, propBlock, propBlockParts, ""); err != nil { + t.Fatal(err) + } ensureNewRound(newRoundCh, height, round) - t.Log("#### ONTO ROUND 1") + + ensureNewProposal(proposalCh, height, round) + + // prevote for the locked block. We do not currently prevote for the + // proposal. + // TODO: do not prevote the locked block if it does not match the proposal. + // (https://github.com/tendermint/tendermint/issues/6850) + ensurePrevote(voteCh, height, round) + validatePrevote(t, cs1, round, vss[0], theBlockHash) + // add >2/3 prevotes for nil from all other validators + signAddVotes(cs1, cmtproto.PrevoteType, nil, types.PartSetHeader{}, vs2, vs3, vs4) + + ensurePrecommit(voteCh, height, round) + + // verify that we haven't update our locked block since the first round + validatePrecommit(t, cs1, round, 0, vss[0], nil, theBlockHash) + + signAddVotes(cs1, cmtproto.PrecommitType, nil, types.PartSetHeader{}, vs2, vs3, vs4) + ensureNewTimeout(timeoutWaitCh, height, round, cs1.config.Precommit(round).Nanoseconds()) + /* - Round2 (vs2, C) // B nil nil nil // nil nil nil _ - cs1 unlocks! + Round 2: + The validator cs1 saw >2/3 precommits for nil in the previous round. + Send the validator >2/3 prevotes for nil and ensure that it did not + unlock its block at the end of the previous round. */ - //XXX: this isnt guaranteed to get there before the timeoutPropose ... - if err := cs1.SetProposalAndBlock(prop, propBlock, propBlockParts, "some peer"); err != nil { + t.Log("#### ONTO ROUND 2") + round++ + incrementRound(vs2, vs3, vs4) + prop, propBlock = decideProposal(t, cs1, vs3, vs3.Height, vs3.Round) + propBlockParts, err = propBlock.MakePartSet(types.BlockPartSizeBytes) + require.NoError(t, err) + if err := cs1.SetProposalAndBlock(prop, propBlock, propBlockParts, ""); err != nil { t.Fatal(err) } + ensureNewRound(newRoundCh, height, round) + ensureNewProposal(proposalCh, height, round) - // go to prevote, prevote for locked block (not proposal) ensurePrevote(voteCh, height, round) - validatePrevote(t, cs1, round, vss[0], lockedBlockHash) - // now lets add prevotes from everyone else for nil (a polka!) + validatePrevote(t, cs1, round, vss[0], theBlockHash) + signAddVotes(cs1, cmtproto.PrevoteType, nil, types.PartSetHeader{}, vs2, vs3, vs4) - // the polka makes us unlock and precommit nil - ensureNewUnlock(unlockCh, height, round) ensurePrecommit(voteCh, height, round) - // we should have unlocked and committed nil - // NOTE: since we don't relock on nil, the lock round is -1 - validatePrecommit(t, cs1, round, -1, vss[0], nil, nil) + // verify that we haven't update our locked block since the first round + validatePrecommit(t, cs1, round, 0, vss[0], nil, theBlockHash) - signAddVotes(cs1, cmtproto.PrecommitType, nil, types.PartSetHeader{}, vs2, vs3) - ensureNewRound(newRoundCh, height, round+1) } -// 4 vals, v1 locks on proposed block in the first round but the other validators only prevote -// In the second round, v1 misses the proposal but sees a majority prevote an unknown block so -// v1 should unlock and precommit nil. In the third round another block is proposed, all vals -// prevote and now v1 can lock onto the third block and precommit that -func TestStateLockPOLUnlockOnUnknownBlock(t *testing.T) { +// TestStateLockMissingProposalWhenPOLSeenDoesNotUnlock tests that observing +// a two thirds majority for a block does not cause a validator to upate its lock on the +// new block if a proposal was not seen for that block. +func TestStateLockMissingProposalWhenPOLSeenDoesNotUpdateLock(t *testing.T) { + _, cancel := context.WithCancel(context.Background()) + defer cancel() + cs1, vss := randState(4) vs2, vs3, vs4 := vss[1], vss[2], vss[3] height, round := cs1.Height, cs1.Round @@ -845,13 +904,15 @@ func TestStateLockPOLUnlockOnUnknownBlock(t *testing.T) { addr := pv1.Address() voteCh := subscribeToVoter(cs1, addr) newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound) - // everything done from perspective of cs1 - /* - Round0 (cs1, A) // A A A A// A nil nil nil - */ + Round 0: + cs1 creates a proposal for block B. + Send a prevote for B from each of the validators to cs1. + Send a precommit for nil from all of the validators to cs1. - // start round and wait for propose and prevote + This ensures that cs1 will lock on B in this round but not precommit it. + */ + t.Log("### Starting Round 0") startTestRound(cs1, height, round) ensureNewRound(newRoundCh, height, round) @@ -871,89 +932,112 @@ func TestStateLockPOLUnlockOnUnknownBlock(t *testing.T) { // add precommits from the rest signAddVotes(cs1, cmtproto.PrecommitType, nil, types.PartSetHeader{}, vs2, vs3, vs4) - // before we timeout to the new round set the new proposal + // timeout to new round + ensureNewTimeout(timeoutWaitCh, height, round, cs1.config.Precommit(round).Nanoseconds()) + + /* + Round 1: + Create a new block, D but do not send it to cs1. + Send a prevote for D from each of the validators to cs1. + + Check that cs1 does not update its locked block to this missed block D. + */ + t.Log("### Starting Round 1") + incrementRound(vs2, vs3, vs4) + round++ cs2 := newState(cs1.state, vs2, kvstore.NewApplication()) - prop, propBlock := decideProposal(t, cs2, vs2, vs2.Height, vs2.Round+1) + prop, propBlock := decideProposal(t, cs2, vs2, vs2.Height, vs2.Round) if prop == nil || propBlock == nil { t.Fatal("Failed to create proposal block with vs2") } secondBlockParts, err := propBlock.MakePartSet(partSize) require.NoError(t, err) - secondBlockHash := propBlock.Hash() require.NotEqual(t, secondBlockHash, firstBlockHash) - incrementRound(vs2, vs3, vs4) + ensureNewRound(newRoundCh, height, round) - // timeout to new round - ensureNewTimeout(timeoutWaitCh, height, round, cs1.config.Precommit(round).Nanoseconds()) + // go to prevote, node should prevote for locked block (not the new proposal) - this is relocking + ensurePrevote(voteCh, height, round) + validatePrevote(t, cs1, round, vss[0], firstBlockHash) - round++ // moving to the next round + // now lets add prevotes from everyone else for the new block + signAddVotes(cs1, cmtproto.PrevoteType, secondBlockHash, secondBlockParts.Header(), vs2, vs3, vs4) - ensureNewRound(newRoundCh, height, round) - t.Log("### ONTO ROUND 1") + ensurePrecommit(voteCh, height, round) + validatePrecommit(t, cs1, round, 0, vss[0], nil, firstBlockHash) +} +// TestStateLockDoesNotLockOnOldProposal tests that observing +// a two thirds majority for a block does not cause a validator to lock on the +// block if a proposal was not seen for that block in the current round, but +// was seen in a previous round. +func TestStateLockDoesNotLockOnOldProposal(t *testing.T) { + cs1, vss := randState(4) + vs2, vs3, vs4 := vss[1], vss[2], vss[3] + height, round := cs1.Height, cs1.Round + + timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait) + proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal) + pv1, err := cs1.privValidator.GetPubKey() + require.NoError(t, err) + addr := pv1.Address() + voteCh := subscribeToVoter(cs1, addr) + newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound) /* - Round1 (vs2, B) // A B B B // nil nil nil nil) + Round 0: + cs1 creates a proposal for block B. + Send a prevote for nil from each of the validators to cs1. + Send a precommit for nil from all of the validators to cs1. + + This ensures that cs1 will not lock on B. */ + t.Log("### Starting Round 0") + startTestRound(cs1, height, round) - // now we're on a new round but v1 misses the proposal + ensureNewRound(newRoundCh, height, round) + ensureNewProposal(proposalCh, height, round) + rs := cs1.GetRoundState() + firstBlockHash := rs.ProposalBlock.Hash() + firstBlockParts := rs.ProposalBlockParts.Header() - // go to prevote, node should prevote for locked block (not the new proposal) - this is relocking ensurePrevote(voteCh, height, round) - validatePrevote(t, cs1, round, vss[0], firstBlockHash) - // now lets add prevotes from everyone else for the new block - signAddVotes(cs1, cmtproto.PrevoteType, secondBlockHash, secondBlockParts.Header(), vs2, vs3, vs4) + signAddVotes(cs1, cmtproto.PrevoteType, nil, types.PartSetHeader{}, vs2, vs3, vs4) + // The proposed block should not have been locked. ensurePrecommit(voteCh, height, round) - // we should have unlocked and locked on the new block, sending a precommit for this new block validatePrecommit(t, cs1, round, -1, vss[0], nil, nil) - if err := cs1.SetProposalAndBlock(prop, propBlock, secondBlockParts, "some peer"); err != nil { - t.Fatal(err) - } - - // more prevote creating a majority on the new block and this is then committed signAddVotes(cs1, cmtproto.PrecommitType, nil, types.PartSetHeader{}, vs2, vs3, vs4) - // before we timeout to the new round set the new proposal - cs3 := newState(cs1.state, vs3, kvstore.NewApplication()) - prop, propBlock = decideProposal(t, cs3, vs3, vs3.Height, vs3.Round+1) - if prop == nil || propBlock == nil { - t.Fatal("Failed to create proposal block with vs2") - } - thirdPropBlockParts, err := propBlock.MakePartSet(partSize) - require.NoError(t, err) - thirdPropBlockHash := propBlock.Hash() - require.NotEqual(t, secondBlockHash, thirdPropBlockHash) - incrementRound(vs2, vs3, vs4) // timeout to new round ensureNewTimeout(timeoutWaitCh, height, round, cs1.config.Precommit(round).Nanoseconds()) - round++ // moving to the next round - ensureNewRound(newRoundCh, height, round) - t.Log("### ONTO ROUND 2") - /* - Round2 (vs3, C) // C C C C // C nil nil nil) - */ + Round 1: + No proposal new proposal is created. + Send a prevote for B, the block from round 0, from each of the validators to cs1. + Send a precommit for nil from all of the validators to cs1. - if err := cs1.SetProposalAndBlock(prop, propBlock, thirdPropBlockParts, "some peer"); err != nil { - t.Fatal(err) - } + cs1 saw a POL for the block it saw in round 0. We ensure that it does not + lock on this block, since it did not see a proposal for it in this round. + */ + t.Log("### Starting Round 1") + round++ + ensureNewRound(newRoundCh, height, round) ensurePrevote(voteCh, height, round) - // we are no longer locked to the first block so we should be able to prevote - validatePrevote(t, cs1, round, vss[0], thirdPropBlockHash) + validatePrevote(t, cs1, round, vss[0], nil) - signAddVotes(cs1, cmtproto.PrevoteType, thirdPropBlockHash, thirdPropBlockParts.Header(), vs2, vs3, vs4) + // All validators prevote for the old block. + signAddVotes(cs1, cmtproto.PrevoteType, firstBlockHash, firstBlockParts, vs2, vs3, vs4) + // Make sure that cs1 did not lock on the block since it did not receive a proposal for it. ensurePrecommit(voteCh, height, round) - // we have a majority, now vs1 can change lock to the third block - validatePrecommit(t, cs1, round, round, vss[0], thirdPropBlockHash, thirdPropBlockHash) + validatePrecommit(t, cs1, round, -1, vss[0], nil, nil) } // 4 vals @@ -1093,7 +1177,6 @@ func TestStateLockPOLSafety2(t *testing.T) { proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal) timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait) newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound) - unlockCh := subscribe(cs1.eventBus, types.EventQueryUnlock) pv1, err := cs1.privValidator.GetPubKey() require.NoError(t, err) addr := pv1.Address() @@ -1171,14 +1254,13 @@ func TestStateLockPOLSafety2(t *testing.T) { */ ensureNewProposal(proposalCh, height, round) - ensureNoNewUnlock(unlockCh) ensurePrevote(voteCh, height, round) validatePrevote(t, cs1, round, vss[0], propBlockHash1) } // 4 vals. -// polka P0 at R0 for B0. We lock B0 on P0 at R0. P0 unlocks value at R1. +// polka P0 at R0 for B0. We lock B0 on P0 at R0. // What we want: // P0 proposes B0 at R3. @@ -1193,7 +1275,6 @@ func TestProposeValidBlock(t *testing.T) { timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait) timeoutProposeCh := subscribe(cs1.eventBus, types.EventQueryTimeoutPropose) newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound) - unlockCh := subscribe(cs1.eventBus, types.EventQueryUnlock) pv1, err := cs1.privValidator.GetPubKey() require.NoError(t, err) addr := pv1.Address() @@ -1234,16 +1315,17 @@ func TestProposeValidBlock(t *testing.T) { // timeout of propose ensureNewTimeout(timeoutProposeCh, height, round, cs1.config.Propose(round).Nanoseconds()) + // We did not see a valid proposal within this round, so prevote our locked block. + // TODO: prevote nil ensurePrevote(voteCh, height, round) validatePrevote(t, cs1, round, vss[0], propBlockHash) - signAddVotes(cs1, cmtproto.PrevoteType, nil, types.PartSetHeader{}, vs2, vs3, vs4) - - ensureNewUnlock(unlockCh, height, round) + signAddVotes(cs1, cmtproto.PrecommitType, nil, types.PartSetHeader{}, vs2, vs3, vs4) ensurePrecommit(voteCh, height, round) - // we should have precommitted - validatePrecommit(t, cs1, round, -1, vss[0], nil, nil) + // we should have precommitted nil during this round because we received + // >2/3 precommits for nil from the other validators. + validatePrecommit(t, cs1, round, 0, vss[0], nil, propBlockHash) incrementRound(vs2, vs3, vs4) incrementRound(vs2, vs3, vs4) diff --git a/internal/test/block.go b/internal/test/block.go index d79f08100..137e4b826 100644 --- a/internal/test/block.go +++ b/internal/test/block.go @@ -53,7 +53,7 @@ func MakeHeader(t *testing.T, h *types.Header) *types.Header { if h.Height == 0 { h.Height = 1 } - if h.LastBlockID.IsZero() { + if h.LastBlockID.IsNil() { h.LastBlockID = MakeBlockID() } if h.ChainID == "" { diff --git a/spec/consensus/signing.md b/spec/consensus/signing.md index 38afe3502..8ad040ac6 100644 --- a/spec/consensus/signing.md +++ b/spec/consensus/signing.md @@ -62,9 +62,9 @@ type PartSetHeader struct { ``` To be included in a valid vote or proposal, BlockID must either represent a `nil` block, or a complete one. -We introduce two methods, `BlockID.IsZero()` and `BlockID.IsComplete()` for these cases, respectively. +We introduce two methods, `BlockID.IsNil()` and `BlockID.IsComplete()` for these cases, respectively. -`BlockID.IsZero()` returns true for BlockID `b` if each of the following +`BlockID.IsNil()` returns true for BlockID `b` if each of the following are true: ```go @@ -133,7 +133,7 @@ A vote is valid if each of the following lines evaluates to true for vote `v`: v.Type == 0x1 || v.Type == 0x2 v.Height > 0 v.Round >= 0 -v.BlockID.IsZero() || v.BlockID.IsComplete() +v.BlockID.IsNil() || v.BlockID.IsComplete() ``` In other words, a vote is valid for signing if it contains the type of a Prevote @@ -207,8 +207,8 @@ In other words, a vote should only be signed if it's: This means that once a validator signs a prevote for a given height and round, the only other message it can sign for that height and round is a precommit. And once a validator signs a precommit for a given height and round, it must not sign any other message for that same height and round. -Note this includes votes for `nil`, ie. where `BlockID.IsZero()` is true. If a -signer has already signed a vote where `BlockID.IsZero()` is true, it cannot +Note this includes votes for `nil`, ie. where `BlockID.IsNil()` is true. If a +signer has already signed a vote where `BlockID.IsNil()` is true, it cannot sign another vote with the same type for the same height and round where `BlockID.IsComplete()` is true. Thus only a single vote of a particular type (ie. 0x01 or 0x02) can be signed for the same height and round. diff --git a/types/block.go b/types/block.go index c93acf85c..108cbc70f 100644 --- a/types/block.go +++ b/types/block.go @@ -874,7 +874,7 @@ func (commit *Commit) ValidateBasic() error { } if commit.Height >= 1 { - if commit.BlockID.IsZero() { + if commit.BlockID.IsNil() { return errors.New("commit cannot be for nil block") } @@ -1192,8 +1192,8 @@ func (blockID BlockID) ValidateBasic() error { return nil } -// IsZero returns true if this is the BlockID of a nil block. -func (blockID BlockID) IsZero() bool { +// IsNil returns true if this is the BlockID of a nil block. +func (blockID BlockID) IsNil() bool { return len(blockID.Hash) == 0 && blockID.PartSetHeader.IsZero() } diff --git a/types/canonical.go b/types/canonical.go index e64ed1ba7..8857c71c6 100644 --- a/types/canonical.go +++ b/types/canonical.go @@ -21,7 +21,7 @@ func CanonicalizeBlockID(bid cmtproto.BlockID) *cmtproto.CanonicalBlockID { panic(err) } var cbid *cmtproto.CanonicalBlockID - if rbid == nil || rbid.IsZero() { + if rbid == nil || rbid.IsNil() { cbid = nil } else { cbid = &cmtproto.CanonicalBlockID{ diff --git a/types/event_bus.go b/types/event_bus.go index cd42ec05d..8fefe1da4 100644 --- a/types/event_bus.go +++ b/types/event_bus.go @@ -211,10 +211,6 @@ func (b *EventBus) PublishEventPolka(data EventDataRoundState) error { return b.Publish(EventPolka, data) } -func (b *EventBus) PublishEventUnlock(data EventDataRoundState) error { - return b.Publish(EventUnlock, data) -} - func (b *EventBus) PublishEventRelock(data EventDataRoundState) error { return b.Publish(EventRelock, data) } @@ -291,10 +287,6 @@ func (NopEventBus) PublishEventPolka(data EventDataRoundState) error { return nil } -func (NopEventBus) PublishEventUnlock(data EventDataRoundState) error { - return nil -} - func (NopEventBus) PublishEventRelock(data EventDataRoundState) error { return nil } diff --git a/types/event_bus_test.go b/types/event_bus_test.go index dd32662a6..7d8da3ff2 100644 --- a/types/event_bus_test.go +++ b/types/event_bus_test.go @@ -323,7 +323,7 @@ func TestEventBusPublish(t *testing.T) { } }) - const numEventsExpected = 14 + const numEventsExpected = 13 sub, err := eventBus.Subscribe(context.Background(), "test", cmtquery.Empty{}, numEventsExpected) require.NoError(t, err) @@ -360,8 +360,6 @@ func TestEventBusPublish(t *testing.T) { require.NoError(t, err) err = eventBus.PublishEventPolka(EventDataRoundState{}) require.NoError(t, err) - err = eventBus.PublishEventUnlock(EventDataRoundState{}) - require.NoError(t, err) err = eventBus.PublishEventRelock(EventDataRoundState{}) require.NoError(t, err) err = eventBus.PublishEventLock(EventDataRoundState{}) @@ -469,7 +467,6 @@ var events = []string{ EventTimeoutPropose, EventCompleteProposal, EventPolka, - EventUnlock, EventLock, EventRelock, EventTimeoutWait, @@ -488,7 +485,6 @@ var queries = []cmtpubsub.Query{ EventQueryTimeoutPropose, EventQueryCompleteProposal, EventQueryPolka, - EventQueryUnlock, EventQueryLock, EventQueryRelock, EventQueryTimeoutWait, diff --git a/types/events.go b/types/events.go index e1b89dcb7..27335b249 100644 --- a/types/events.go +++ b/types/events.go @@ -33,7 +33,6 @@ const ( EventRelock = "Relock" EventTimeoutPropose = "TimeoutPropose" EventTimeoutWait = "TimeoutWait" - EventUnlock = "Unlock" EventValidBlock = "ValidBlock" EventVote = "Vote" ) @@ -155,7 +154,6 @@ var ( EventQueryTimeoutPropose = QueryForEvent(EventTimeoutPropose) EventQueryTimeoutWait = QueryForEvent(EventTimeoutWait) EventQueryTx = QueryForEvent(EventTx) - EventQueryUnlock = QueryForEvent(EventUnlock) EventQueryValidatorSetUpdates = QueryForEvent(EventValidatorSetUpdates) EventQueryValidBlock = QueryForEvent(EventValidBlock) EventQueryVote = QueryForEvent(EventVote) diff --git a/types/vote.go b/types/vote.go index 1b27fe63f..995bb1c62 100644 --- a/types/vote.go +++ b/types/vote.go @@ -68,7 +68,7 @@ func (vote *Vote) CommitSig() CommitSig { switch { case vote.BlockID.IsComplete(): blockIDFlag = BlockIDFlagCommit - case vote.BlockID.IsZero(): + case vote.BlockID.IsNil(): blockIDFlag = BlockIDFlagNil default: panic(fmt.Sprintf("Invalid vote %v - expected BlockID to be either empty or complete", vote)) @@ -177,7 +177,7 @@ func (vote *Vote) ValidateBasic() error { // BlockID.ValidateBasic would not err if we for instance have an empty hash but a // non-empty PartsSetHeader: - if !vote.BlockID.IsZero() && !vote.BlockID.IsComplete() { + if !vote.BlockID.IsNil() && !vote.BlockID.IsComplete() { return fmt.Errorf("blockID must be either empty or complete, got: %v", vote.BlockID) } diff --git a/types/vote_set_test.go b/types/vote_set_test.go index 7c3c049d3..edb47d228 100644 --- a/types/vote_set_test.go +++ b/types/vote_set_test.go @@ -25,7 +25,7 @@ func TestVoteSet_AddVote_Good(t *testing.T) { assert.Nil(t, voteSet.GetByAddress(val0Addr)) assert.False(t, voteSet.BitArray().GetIndex(0)) blockID, ok := voteSet.TwoThirdsMajority() - assert.False(t, ok || !blockID.IsZero(), "there should be no 2/3 majority") + assert.False(t, ok || !blockID.IsNil(), "there should be no 2/3 majority") vote := &Vote{ ValidatorAddress: val0Addr, @@ -42,7 +42,7 @@ func TestVoteSet_AddVote_Good(t *testing.T) { assert.NotNil(t, voteSet.GetByAddress(val0Addr)) assert.True(t, voteSet.BitArray().GetIndex(0)) blockID, ok = voteSet.TwoThirdsMajority() - assert.False(t, ok || !blockID.IsZero(), "there should be no 2/3 majority") + assert.False(t, ok || !blockID.IsNil(), "there should be no 2/3 majority") } func TestVoteSet_AddVote_Bad(t *testing.T) { @@ -143,7 +143,7 @@ func TestVoteSet_2_3Majority(t *testing.T) { require.NoError(t, err) } blockID, ok := voteSet.TwoThirdsMajority() - assert.False(t, ok || !blockID.IsZero(), "there should be no 2/3 majority") + assert.False(t, ok || !blockID.IsNil(), "there should be no 2/3 majority") // 7th validator voted for some blockhash { @@ -154,7 +154,7 @@ func TestVoteSet_2_3Majority(t *testing.T) { _, err = signAddVote(privValidators[6], withBlockHash(vote, cmtrand.Bytes(32)), voteSet) require.NoError(t, err) blockID, ok = voteSet.TwoThirdsMajority() - assert.False(t, ok || !blockID.IsZero(), "there should be no 2/3 majority") + assert.False(t, ok || !blockID.IsNil(), "there should be no 2/3 majority") } // 8th validator voted for nil. @@ -166,7 +166,7 @@ func TestVoteSet_2_3Majority(t *testing.T) { _, err = signAddVote(privValidators[7], vote, voteSet) require.NoError(t, err) blockID, ok = voteSet.TwoThirdsMajority() - assert.True(t, ok || blockID.IsZero(), "there should be 2/3 majority for nil") + assert.True(t, ok || blockID.IsNil(), "there should be 2/3 majority for nil") } } @@ -198,7 +198,7 @@ func TestVoteSet_2_3MajorityRedux(t *testing.T) { require.NoError(t, err) } blockID, ok := voteSet.TwoThirdsMajority() - assert.False(t, ok || !blockID.IsZero(), + assert.False(t, ok || !blockID.IsNil(), "there should be no 2/3 majority") // 67th validator voted for nil @@ -210,7 +210,7 @@ func TestVoteSet_2_3MajorityRedux(t *testing.T) { _, err = signAddVote(privValidators[66], withBlockHash(vote, nil), voteSet) require.NoError(t, err) blockID, ok = voteSet.TwoThirdsMajority() - assert.False(t, ok || !blockID.IsZero(), + assert.False(t, ok || !blockID.IsNil(), "there should be no 2/3 majority: last vote added was nil") } @@ -224,7 +224,7 @@ func TestVoteSet_2_3MajorityRedux(t *testing.T) { _, err = signAddVote(privValidators[67], withBlockPartSetHeader(vote, blockPartsHeader), voteSet) require.NoError(t, err) blockID, ok = voteSet.TwoThirdsMajority() - assert.False(t, ok || !blockID.IsZero(), + assert.False(t, ok || !blockID.IsNil(), "there should be no 2/3 majority: last vote added had different PartSetHeader Hash") } @@ -238,7 +238,7 @@ func TestVoteSet_2_3MajorityRedux(t *testing.T) { _, err = signAddVote(privValidators[68], withBlockPartSetHeader(vote, blockPartsHeader), voteSet) require.NoError(t, err) blockID, ok = voteSet.TwoThirdsMajority() - assert.False(t, ok || !blockID.IsZero(), + assert.False(t, ok || !blockID.IsNil(), "there should be no 2/3 majority: last vote added had different PartSetHeader Total") } @@ -251,7 +251,7 @@ func TestVoteSet_2_3MajorityRedux(t *testing.T) { _, err = signAddVote(privValidators[69], withBlockHash(vote, cmtrand.Bytes(32)), voteSet) require.NoError(t, err) blockID, ok = voteSet.TwoThirdsMajority() - assert.False(t, ok || !blockID.IsZero(), + assert.False(t, ok || !blockID.IsNil(), "there should be no 2/3 majority: last vote added had different BlockHash") }