diff --git a/consensus/common_test.go b/consensus/common_test.go index 95effc131dd..7e103bea900 100644 --- a/consensus/common_test.go +++ b/consensus/common_test.go @@ -521,13 +521,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( @@ -640,9 +633,14 @@ 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 ensureRelock(relockCh <-chan cmtpubsub.Message, height int64, round int32) { + ensureNewEvent(relockCh, height, round, ensureTimeout, + "Timeout expired while waiting for RelockValue 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 9d4eaf393f1..d0958061481 100644 --- a/consensus/state.go +++ b/consensus/state.go @@ -1435,7 +1435,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) @@ -1482,21 +1481,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{}, nil) return } @@ -1516,7 +1503,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) @@ -1538,23 +1527,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{}, nil) } @@ -1662,7 +1642,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 } @@ -2020,7 +2000,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", @@ -2173,7 +2153,7 @@ func (cs *State) addVote(vote *types.Vote, peerID p2p.ID) (added bool, err error } // Verify VoteExtension if precommit and not nil // https://github.com/tendermint/tendermint/issues/8487 - if vote.Type == cmtproto.PrecommitType && !vote.BlockID.IsZero() && + if vote.Type == cmtproto.PrecommitType && !vote.BlockID.IsNil() && !bytes.Equal(vote.ValidatorAddress, myAddr) { // Skip the VerifyVoteExtension call if the vote was issued by this validator. // The core fields of the vote message were already validated in the @@ -2230,33 +2210,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 @@ -2292,7 +2252,7 @@ func (cs *State) addVote(vote *types.Vote, peerID p2p.ID) (added bool, err error case cs.Round == vote.Round && cstypes.RoundStepPrevote <= cs.Step: // current round blockID, ok := prevotes.TwoThirdsMajority() - if ok && (cs.isProposalComplete() || len(blockID.Hash) == 0) { + if ok && (cs.isProposalComplete() || blockID.IsNil()) { cs.enterPrecommit(height, vote.Round) } else if prevotes.HasTwoThirdsAny() { cs.enterPrevoteWait(height, vote.Round) @@ -2320,7 +2280,7 @@ func (cs *State) addVote(vote *types.Vote, peerID p2p.ID) (added bool, err error cs.enterNewRound(height, vote.Round) cs.enterPrecommit(height, vote.Round) - if len(blockID.Hash) != 0 { + if !blockID.IsNil() { cs.enterCommit(height, vote.Round) if cs.config.SkipTimeoutCommit && precommits.HasAll() { cs.enterNewRound(cs.Height, 0) @@ -2371,7 +2331,7 @@ func (cs *State) signVote( } extEnabled := cs.state.ConsensusParams.ABCI.VoteExtensionsEnabled(vote.Height) - if msgType == cmtproto.PrecommitType && !vote.BlockID.IsZero() { + if msgType == cmtproto.PrecommitType && !vote.BlockID.IsNil() { // if the signedMessage type is for a non-nil precommit, add // VoteExtension if extEnabled { @@ -2443,7 +2403,7 @@ func (cs *State) signAddVote( } hasExt := len(vote.ExtensionSignature) > 0 extEnabled := cs.state.ConsensusParams.ABCI.VoteExtensionsEnabled(vote.Height) - if vote.Type == cmtproto.PrecommitType && !vote.BlockID.IsZero() && hasExt != extEnabled { + if vote.Type == cmtproto.PrecommitType && !vote.BlockID.IsNil() && hasExt != extEnabled { panic(fmt.Errorf("vote extension absence/presence does not match extensions enabled %t!=%t, height %d, type %v", hasExt, extEnabled, vote.Height, vote.Type)) } diff --git a/consensus/state_test.go b/consensus/state_test.go index 65eac76e6d3..0f374159fe7 100644 --- a/consensus/state_test.go +++ b/consensus/state_test.go @@ -42,11 +42,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 @@ -130,8 +136,7 @@ func TestStateProposerSelection2(t *testing.T) { prop.Address)) } - rs := cs1.GetRoundState() - signAddVotes(cs1, cmtproto.PrecommitType, nil, rs.ProposalBlockParts.Header(), true, vss[1:]...) + signAddVotes(cs1, cmtproto.PrecommitType, nil, types.PartSetHeader{}, true, vss[1:]...) ensureNewRound(newRoundCh, height, i+round+1) // wait for the new round event each round incrementRound(vss[1:]...) } @@ -641,11 +646,12 @@ 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) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -661,18 +667,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) @@ -681,97 +687,203 @@ 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, false, 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{}, true, 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 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++ + + // Generate a new proposal block. cs2 := newState(cs1.state, vs2, kvstore.NewInMemoryApplication()) - prop, propBlock := decideProposal(ctx, t, cs2, vs2, vs2.Height, vs2.Round+1) - if prop == nil || propBlock == nil { - t.Fatal("Failed to create proposal block with vs2") + propR1, propBlockR1 := decideProposal(ctx, 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) } - propBlockParts, err := propBlock.MakePartSet(partSize) + + ensureNewRound(newRoundCh, height, round) + + // ensure that the validator receives the proposal. + ensureNewProposal(proposalCh, height, round) + + // 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) + + // Add prevotes from the remainder of the validators for the new locked block. + signAddVotes(cs1, cmtproto.PrevoteType, propBlockR1Hash, propBlockR1Parts.Header(), false, vs2, vs3, vs4) + + // Check that we lock on a new block. + ensureLock(lockCh, height, round) + + ensurePrecommit(voteCh, height, round) + + // 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) +} + +// TestStateLockPOLRelock tests that a validator updates its locked round if +// it receives votes representing over 2/3 of the voting power on the network +// for a block that it is already locked in. +func TestStateLockPOLRelock(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) + lockCh := subscribe(cs1.eventBus, types.EventQueryLock) + relockCh := subscribe(cs1.eventBus, types.EventQueryRelock) + newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound) - propBlockHash := propBlock.Hash() - require.NotEqual(t, propBlockHash, theBlockHash) + /* + 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") - incrementRound(vs2, vs3, vs4) + startTestRound(cs1, height, round) - // timeout to new round + ensureNewRound(newRoundCh, height, round) + ensureNewProposal(proposalCh, height, round) + rs := cs1.GetRoundState() + theBlock := rs.ProposalBlock + theBlockHash := rs.ProposalBlock.Hash() + theBlockParts := rs.ProposalBlockParts + + ensurePrevote(voteCh, height, round) + + signAddVotes(cs1, cmtproto.PrevoteType, theBlockHash, theBlockParts.Header(), false, vs2, vs3, vs4) + + // check that the validator generates a Lock event. + ensureLock(lockCh, height, round) + + // 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 of the validators. + signAddVotes(cs1, cmtproto.PrecommitType, nil, types.PartSetHeader{}, true, vs2, vs3, vs4) + + // 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 { + /* + Round 1: + Create a proposal for block B, the same block from round 1. + Send a prevote for B from each of the validators to cs1. + Send a precommit for nil from all of the validtors to cs1. + Check that cs1 updates its 'locked round' value to the current round. + */ + t.Log("### Starting Round 1") + incrementRound(vs2, vs3, vs4) + round++ + propBlockID := types.BlockID{Hash: theBlockHash, PartSetHeader: theBlockParts.Header()} + propR1 := types.NewProposal(height, round, cs1.ValidRound, propBlockID) + p := propR1.ToProto() + if err := vs2.SignProposal(cs1.state.ChainID, p); err != nil { + t.Fatalf("error signing proposal: %s", err) + } + propR1.Signature = p.Signature + if err := cs1.SetProposalAndBlock(propR1, theBlock, theBlockParts, ""); 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(), false, vs2, vs3, vs4) + // Add prevotes from the remainder of the validators for the locked block. + signAddVotes(cs1, cmtproto.PrevoteType, theBlockHash, theBlockParts.Header(), false, 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 relock. + ensureRelock(relockCh, height, round) - // more prevote creating a majority on the new block and this is then committed - signAddVotes(cs1, cmtproto.PrecommitType, propBlockHash, propBlockParts.Header(), true, vs2, vs3) - ensureNewBlockHeader(newBlockCh, height, propBlockHash) + ensurePrecommit(voteCh, height, round) - ensureNewRound(newRoundCh, height+1, 0) + // We should now be locked on the same block but with an updated locked round. + validatePrecommit(t, cs1, round, round, vss[0], theBlockHash, theBlockHash) } -// 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) { ctx, 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) @@ -787,63 +899,98 @@ func TestStateLockPOLUnlock(t *testing.T) { signAddVotes(cs1, cmtproto.PrevoteType, theBlockHash, theBlockParts, false, 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{}, true, vs2, vs4) signAddVotes(cs1, cmtproto.PrecommitType, theBlockHash, theBlockParts, true, vs3) - // before we time out into new round, set next proposal block - prop, propBlock := decideProposal(ctx, 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(ctx, 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{}, false, 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{}, true, 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(ctx, 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{}, false, 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{}, true, 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) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -860,13 +1007,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) @@ -886,89 +1035,112 @@ func TestStateLockPOLUnlockOnUnknownBlock(t *testing.T) { // add precommits from the rest signAddVotes(cs1, cmtproto.PrecommitType, nil, types.PartSetHeader{}, true, 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.NewInMemoryApplication()) - prop, propBlock := decideProposal(ctx, t, cs2, vs2, vs2.Height, vs2.Round+1) + prop, propBlock := decideProposal(ctx, 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(), false, 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(), false, vs2, vs3, vs4) + signAddVotes(cs1, cmtproto.PrevoteType, nil, types.PartSetHeader{}, false, 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{}, true, vs2, vs3, vs4) - // before we timeout to the new round set the new proposal - cs3 := newState(cs1.state, vs3, kvstore.NewInMemoryApplication()) - prop, propBlock = decideProposal(ctx, 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(), false, vs2, vs3, vs4) + // All validators prevote for the old block. + signAddVotes(cs1, cmtproto.PrevoteType, firstBlockHash, firstBlockParts, false, 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 @@ -1114,7 +1286,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() @@ -1192,13 +1363,12 @@ 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. @@ -1213,7 +1383,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() @@ -1254,16 +1423,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{}, false, vs2, vs3, vs4) - - ensureNewUnlock(unlockCh, height, round) + signAddVotes(cs1, cmtproto.PrecommitType, nil, types.PartSetHeader{}, true, 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 5889ac3517d..31587cf1712 100644 --- a/internal/test/block.go +++ b/internal/test/block.go @@ -52,7 +52,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 68547eea25e..9c6ebad3268 100644 --- a/spec/consensus/signing.md +++ b/spec/consensus/signing.md @@ -63,9 +63,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 @@ -134,7 +134,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 @@ -208,8 +208,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 82d0fa4d989..b7343f18bcb 100644 --- a/types/block.go +++ b/types/block.go @@ -901,7 +901,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") } @@ -1207,7 +1207,7 @@ func (ec *ExtendedCommit) ValidateBasic() error { } if ec.Height >= 1 { - if ec.BlockID.IsZero() { + if ec.BlockID.IsNil() { return errors.New("extended commit cannot be for nil block") } @@ -1479,8 +1479,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 64f4bac2b04..9ba64ab0a60 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 0abb87aa51f..4e1bf406739 100644 --- a/types/event_bus.go +++ b/types/event_bus.go @@ -212,10 +212,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) } @@ -296,10 +292,6 @@ func (NopEventBus) PublishEventPolka(EventDataRoundState) error { return nil } -func (NopEventBus) PublishEventUnlock(EventDataRoundState) error { - return nil -} - func (NopEventBus) PublishEventRelock(EventDataRoundState) error { return nil } diff --git a/types/event_bus_test.go b/types/event_bus_test.go index 3058a80d402..8b04f9fd24a 100644 --- a/types/event_bus_test.go +++ b/types/event_bus_test.go @@ -351,7 +351,7 @@ func TestEventBusPublish(t *testing.T) { } }) - const numEventsExpected = 15 + const numEventsExpected = 14 sub, err := eventBus.Subscribe(context.Background(), "test", cmtquery.All, numEventsExpected) require.NoError(t, err) @@ -390,8 +390,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{}) @@ -500,7 +498,6 @@ var events = []string{ EventTimeoutPropose, EventCompleteProposal, EventPolka, - EventUnlock, EventLock, EventRelock, EventTimeoutWait, @@ -520,7 +517,6 @@ var queries = []cmtpubsub.Query{ EventQueryTimeoutPropose, EventQueryCompleteProposal, EventQueryPolka, - EventQueryUnlock, EventQueryLock, EventQueryRelock, EventQueryTimeoutWait, diff --git a/types/events.go b/types/events.go index 3dbf9a78db6..0b385185d39 100644 --- a/types/events.go +++ b/types/events.go @@ -34,7 +34,6 @@ const ( EventRelock = "Relock" EventTimeoutPropose = "TimeoutPropose" EventTimeoutWait = "TimeoutWait" - EventUnlock = "Unlock" EventValidBlock = "ValidBlock" EventVote = "Vote" ) @@ -159,7 +158,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 660b538d178..0ec96eb2cfb 100644 --- a/types/vote.go +++ b/types/vote.go @@ -99,7 +99,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)) @@ -252,7 +252,7 @@ func (vote *Vote) VerifyVoteAndExtension(chainID string, pubKey crypto.PubKey) e // VerifyExtension checks whether the vote extension signature corresponds to the // given chain ID and public key. func (vote *Vote) VerifyExtension(chainID string, pubKey crypto.PubKey) error { - if vote.Type != cmtproto.PrecommitType || vote.BlockID.IsZero() { + if vote.Type != cmtproto.PrecommitType || vote.BlockID.IsNil() { return nil } v := vote.ToProto() @@ -287,7 +287,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) } @@ -311,11 +311,11 @@ func (vote *Vote) ValidateBasic() error { // We should only ever see vote extensions in non-nil precommits, otherwise // this is a violation of the specification. // https://github.com/tendermint/tendermint/issues/8487 - if vote.Type != cmtproto.PrecommitType || vote.BlockID.IsZero() { + if vote.Type != cmtproto.PrecommitType || vote.BlockID.IsNil() { if len(vote.Extension) > 0 { return fmt.Errorf( "unexpected vote extension; vote type %d, isNil %t", - vote.Type, vote.BlockID.IsZero(), + vote.Type, vote.BlockID.IsNil(), ) } if len(vote.ExtensionSignature) > 0 { @@ -323,7 +323,7 @@ func (vote *Vote) ValidateBasic() error { } } - if vote.Type == cmtproto.PrecommitType && !vote.BlockID.IsZero() { + if vote.Type == cmtproto.PrecommitType && !vote.BlockID.IsNil() { // It's possible that this vote has vote extensions but // they could also be disabled and thus not present thus // we can't do all checks @@ -350,7 +350,7 @@ func (vote *Vote) EnsureExtension() error { if vote.Type != cmtproto.PrecommitType { return nil } - if vote.BlockID.IsZero() { + if vote.BlockID.IsNil() { return nil } if len(vote.ExtensionSignature) > 0 { @@ -415,7 +415,7 @@ func SignAndCheckVote( return false, fmt.Errorf("only Precommit votes may have extensions enabled; vote type: %d", vote.Type) } - isNil := vote.BlockID.IsZero() + isNil := vote.BlockID.IsNil() extSignature := (len(v.ExtensionSignature) > 0) if extSignature == (!isPrecommit || isNil) { // Non-recoverable because the vote is malformed diff --git a/types/vote_set_test.go b/types/vote_set_test.go index bfd7ac49248..7887d228b0d 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) { @@ -144,7 +144,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 { @@ -155,7 +155,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. @@ -167,7 +167,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") } } @@ -199,7 +199,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 @@ -211,7 +211,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") } @@ -225,7 +225,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") } @@ -239,7 +239,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") } @@ -252,7 +252,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") }