diff --git a/CHANGELOG_PENDING.md b/CHANGELOG_PENDING.md index 8b130aa00..47f26f2c2 100644 --- a/CHANGELOG_PENDING.md +++ b/CHANGELOG_PENDING.md @@ -49,10 +49,15 @@ Special thanks to external contributors on this release: - [rpc] [\#7270](https://github.com/tendermint/tendermint/pull/7270) Add `header` and `header_by_hash` RPC Client queries. (@fedekunze) - [cli] [#7033](https://github.com/tendermint/tendermint/pull/7033) Add a `rollback` command to rollback to the previous tendermint state in the event of non-determinstic app hash or reverting an upgrade. - [mempool, rpc] \#7041 Add removeTx operation to the RPC layer. (@tychoish) +- [consensus] \#7354 add a new `synchrony` field to the `ConsensusParameter` struct for controlling the parameters of the proposer-based timestamp algorithm. (@williambanfield) +- [consensus] \#7376 Update the proposal logic per the Propose-based timestamps specification so that the proposer will wait for the previous block time to occur before proposing the next block. (@williambanfield) +- [consensus] \#7391 Use the proposed block timestamp as the proposal timestamp. Update the block validation logic to ensure that the proposed block's timestamp matches the timestamp in the proposal message. (@williambanfield) +- [consensus] \#7415 Update proposal validation logic to Prevote nil if a proposal does not meet the conditions for Timelyness per the proposer-based timestamp specification. (@anca) +- [consensus] \#7382 Update block validation to no longer require the block timestamp to be the median of the timestamps of the previous commit. (@anca) ### IMPROVEMENTS - [internal/protoio] \#7325 Optimized `MarshalDelimited` by inlining the common case and using a `sync.Pool` in the worst case. (@odeke-em) - +- [consensus] \#6969 remove logic to 'unlock' a locked block. - [pubsub] \#7319 Performance improvements for the event query API (@creachadair) - [node] \#7521 Define concrete type for seed node implementation (@spacech1mp) - [rpc] \#7612 paginate mempool /unconfirmed_txs rpc endpoint (@spacech1mp) diff --git a/config/toml.go b/config/toml.go index 530d67e18..665300727 100644 --- a/config/toml.go +++ b/config/toml.go @@ -571,6 +571,10 @@ var testGenesisFmt = `{ "max_gas": "-1", "time_iota_ms": "10" }, + "synchrony": { + "message_delay": "500000000", + "precision": "10000000" + }, "evidence": { "max_age_num_blocks": "100000", "max_age_duration": "172800000000000", diff --git a/internal/blocksync/reactor_test.go b/internal/blocksync/reactor_test.go index 73d5a3bc7..25814a2ea 100644 --- a/internal/blocksync/reactor_test.go +++ b/internal/blocksync/reactor_test.go @@ -207,7 +207,8 @@ func TestReactor_AbruptDisconnect(t *testing.T) { require.NoError(t, err) defer os.RemoveAll(cfg.RootDir) - genDoc, privVals := factory.RandGenesisDoc(ctx, t, cfg, 1, false, 30) + valSet, privVals := factory.ValidatorSet(ctx, t, 1, 30) + genDoc := factory.GenesisDoc(cfg, time.Now(), valSet.Validators, nil) maxBlockHeight := int64(64) rts := setup(ctx, t, genDoc, privVals[0], []int64{maxBlockHeight, 0}, 0) @@ -246,7 +247,8 @@ func TestReactor_SyncTime(t *testing.T) { require.NoError(t, err) defer os.RemoveAll(cfg.RootDir) - genDoc, privVals := factory.RandGenesisDoc(ctx, t, cfg, 1, false, 30) + valSet, privVals := factory.ValidatorSet(ctx, t, 1, 30) + genDoc := factory.GenesisDoc(cfg, time.Now(), valSet.Validators, nil) maxBlockHeight := int64(101) rts := setup(ctx, t, genDoc, privVals[0], []int64{maxBlockHeight, 0}, 0) @@ -271,10 +273,10 @@ func TestReactor_NoBlockResponse(t *testing.T) { cfg, err := config.ResetTestRoot("block_sync_reactor_test") require.NoError(t, err) - defer os.RemoveAll(cfg.RootDir) - genDoc, privVals := factory.RandGenesisDoc(ctx, t, cfg, 1, false, 30) + valSet, privVals := factory.ValidatorSet(ctx, t, 1, 30) + genDoc := factory.GenesisDoc(cfg, time.Now(), valSet.Validators, nil) maxBlockHeight := int64(65) rts := setup(ctx, t, genDoc, privVals[0], []int64{maxBlockHeight, 0}, 0) @@ -326,7 +328,8 @@ func TestReactor_BadBlockStopsPeer(t *testing.T) { defer os.RemoveAll(cfg.RootDir) maxBlockHeight := int64(48) - genDoc, privVals := factory.RandGenesisDoc(ctx, t, cfg, 1, false, 30) + valSet, privVals := factory.ValidatorSet(ctx, t, 1, 30) + genDoc := factory.GenesisDoc(cfg, time.Now(), valSet.Validators, nil) rts := setup(ctx, t, genDoc, privVals[0], []int64{maxBlockHeight, 0, 0, 0, 0}, 1000) @@ -360,7 +363,8 @@ func TestReactor_BadBlockStopsPeer(t *testing.T) { // // XXX: This causes a potential race condition. // See: https://github.com/tendermint/tendermint/issues/6005 - otherGenDoc, otherPrivVals := factory.RandGenesisDoc(ctx, t, cfg, 1, false, 30) + valSet, otherPrivVals := factory.ValidatorSet(ctx, t, 1, 30) + otherGenDoc := factory.GenesisDoc(cfg, time.Now(), valSet.Validators, nil) newNode := rts.network.MakeNode(ctx, t, p2ptest.NodeOptions{ MaxPeers: uint16(len(rts.nodes) + 1), MaxConnected: uint16(len(rts.nodes) + 1), diff --git a/internal/consensus/byzantine_test.go b/internal/consensus/byzantine_test.go index 0e3013d95..217fca1af 100644 --- a/internal/consensus/byzantine_test.go +++ b/internal/consensus/byzantine_test.go @@ -23,6 +23,7 @@ import ( "github.com/tendermint/tendermint/internal/store" "github.com/tendermint/tendermint/internal/test/factory" "github.com/tendermint/tendermint/libs/log" + tmtime "github.com/tendermint/tendermint/libs/time" tmcons "github.com/tendermint/tendermint/proto/tendermint/consensus" tmproto "github.com/tendermint/tendermint/proto/tendermint/types" "github.com/tendermint/tendermint/types" @@ -46,7 +47,8 @@ func TestByzantinePrevoteEquivocation(t *testing.T) { tickerFunc := newMockTickerFunc(true) appFunc := newKVStore - genDoc, privVals := factory.RandGenesisDoc(ctx, t, config, nValidators, false, 30) + valSet, privVals := factory.ValidatorSet(ctx, t, nValidators, 30) + genDoc := factory.GenesisDoc(config, time.Now(), valSet.Validators, nil) states := make([]*State, nValidators) for i := 0; i < nValidators; i++ { @@ -215,18 +217,18 @@ func TestByzantinePrevoteEquivocation(t *testing.T) { // Make proposal propBlockID := types.BlockID{Hash: block.Hash(), PartSetHeader: blockParts.Header()} - proposal := types.NewProposal(height, round, lazyNodeState.ValidRound, propBlockID) + proposal := types.NewProposal(height, round, lazyNodeState.ValidRound, propBlockID, block.Header.Time) p := proposal.ToProto() if err := lazyNodeState.privValidator.SignProposal(ctx, lazyNodeState.state.ChainID, p); err == nil { proposal.Signature = p.Signature // send proposal and block parts on internal msg queue - lazyNodeState.sendInternalMessage(ctx, msgInfo{&ProposalMessage{proposal}, ""}) + lazyNodeState.sendInternalMessage(ctx, msgInfo{&ProposalMessage{proposal}, "", tmtime.Now()}) for i := 0; i < int(blockParts.Total()); i++ { part := blockParts.GetPart(i) lazyNodeState.sendInternalMessage(ctx, msgInfo{&BlockPartMessage{ lazyNodeState.Height, lazyNodeState.Round, part, - }, ""}) + }, "", tmtime.Now()}) } } else if !lazyNodeState.replayMode { lazyNodeState.logger.Error("enterPropose: Error signing proposal", "height", height, "round", round, "err", err) diff --git a/internal/consensus/common_test.go b/internal/consensus/common_test.go index e21193434..b7b5e2a8c 100644 --- a/internal/consensus/common_test.go +++ b/internal/consensus/common_test.go @@ -3,6 +3,7 @@ package consensus import ( "bytes" "context" + "errors" "fmt" "io" "os" @@ -88,6 +89,7 @@ type validatorStub struct { Index int32 // Validator index. NOTE: we don't assume validator set changes. Height int64 Round int32 + clock tmtime.Source types.PrivValidator VotingPower int64 lastVote *types.Vote @@ -100,16 +102,15 @@ func newValidatorStub(privValidator types.PrivValidator, valIndex int32) *valida Index: valIndex, PrivValidator: privValidator, VotingPower: testMinPower, + clock: tmtime.DefaultSource{}, } } func (vs *validatorStub) signVote( ctx context.Context, - cfg *config.Config, voteType tmproto.SignedMsgType, - hash []byte, - header types.PartSetHeader, -) (*types.Vote, error) { + chainID string, + blockID types.BlockID) (*types.Vote, error) { pubKey, err := vs.PrivValidator.GetPubKey(ctx) if err != nil { @@ -121,12 +122,12 @@ func (vs *validatorStub) signVote( ValidatorAddress: pubKey.Address(), Height: vs.Height, Round: vs.Round, - Timestamp: tmtime.Now(), + Timestamp: vs.clock.Now(), Type: voteType, - BlockID: types.BlockID{Hash: hash, PartSetHeader: header}, + BlockID: blockID, } v := vote.ToProto() - if err := vs.PrivValidator.SignVote(ctx, cfg.ChainID(), v); err != nil { + if err := vs.PrivValidator.SignVote(ctx, chainID, v); err != nil { return nil, fmt.Errorf("sign vote failed: %w", err) } @@ -147,13 +148,11 @@ func signVote( ctx context.Context, t *testing.T, vs *validatorStub, - cfg *config.Config, voteType tmproto.SignedMsgType, - hash []byte, - header types.PartSetHeader, -) *types.Vote { + chainID string, + blockID types.BlockID) *types.Vote { - v, err := vs.signVote(ctx, cfg, voteType, hash, header) + v, err := vs.signVote(ctx, voteType, chainID, blockID) require.NoError(t, err, "failed to sign vote") vs.lastVote = v @@ -164,15 +163,14 @@ func signVote( func signVotes( ctx context.Context, t *testing.T, - cfg *config.Config, voteType tmproto.SignedMsgType, - hash []byte, - header types.PartSetHeader, + chainID string, + blockID types.BlockID, vss ...*validatorStub, ) []*types.Vote { votes := make([]*types.Vote, len(vss)) for i, vs := range vss { - votes[i] = signVote(ctx, t, vs, cfg, voteType, hash, header) + votes[i] = signVote(ctx, t, vs, voteType, chainID, blockID) } return votes } @@ -247,7 +245,7 @@ func decideProposal( // Make proposal polRound, propBlockID := validRound, types.BlockID{Hash: block.Hash(), PartSetHeader: blockParts.Header()} - proposal = types.NewProposal(height, round, polRound, propBlockID) + proposal = types.NewProposal(height, round, polRound, propBlockID, block.Header.Time) p := proposal.ToProto() require.NoError(t, vs.SignProposal(ctx, chainID, p)) @@ -265,14 +263,13 @@ func addVotes(to *State, votes ...*types.Vote) { func signAddVotes( ctx context.Context, t *testing.T, - cfg *config.Config, to *State, voteType tmproto.SignedMsgType, - hash []byte, - header types.PartSetHeader, + chainID string, + blockID types.BlockID, vss ...*validatorStub, ) { - addVotes(to, signVotes(ctx, t, cfg, voteType, hash, header, vss...)...) + addVotes(to, signVotes(ctx, t, voteType, chainID, blockID, vss...)...) } func validatePrevote( @@ -378,6 +375,35 @@ func subscribeToVoter(ctx context.Context, t *testing.T, cs *State, addr []byte) return ch } +func subscribeToVoterBuffered(ctx context.Context, t *testing.T, cs *State, addr []byte) <-chan tmpubsub.Message { + t.Helper() + votesSub, err := cs.eventBus.SubscribeWithArgs(ctx, tmpubsub.SubscribeArgs{ + ClientID: testSubscriber, + Query: types.EventQueryVote, + Limit: 10}) + if err != nil { + t.Fatalf("failed to subscribe %s to %v", testSubscriber, types.EventQueryVote) + } + ch := make(chan tmpubsub.Message, 10) + go func() { + for { + msg, err := votesSub.Next(ctx) + if err != nil { + if !errors.Is(err, tmpubsub.ErrTerminated) && !errors.Is(err, context.Canceled) { + t.Errorf("error terminating pubsub %s", err) + } + return + } + vote := msg.Data().(types.EventDataVote) + // we only fire for our own votes + if bytes.Equal(addr, vote.Vote.ValidatorAddress) { + ch <- msg + } + } + }() + return ch +} + //------------------------------------------------------------------------------- // consensus states @@ -477,17 +503,12 @@ func loadPrivValidator(t *testing.T, cfg *config.Config) *privval.FilePV { return privValidator } -func randState( - ctx context.Context, - t *testing.T, - cfg *config.Config, - logger log.Logger, - nValidators int, -) (*State, []*validatorStub) { +func makeState(ctx context.Context, t *testing.T, cfg *config.Config, logger log.Logger, nValidators int) (*State, []*validatorStub) { t.Helper() - // Get State - state, privVals := randGenesisState(ctx, t, cfg, nValidators, false, 10) + state, privVals := makeGenesisState(ctx, t, cfg, genesisStateArgs{ + Validators: nValidators, + }) vss := make([]*validatorStub, nValidators) @@ -504,7 +525,8 @@ func randState( //------------------------------------------------------------------------------- -func ensureNoNewEvent(t *testing.T, ch <-chan tmpubsub.Message, timeout time.Duration, errorMessage string) { +func ensureNoMessageBeforeTimeout(t *testing.T, ch <-chan tmpubsub.Message, timeout time.Duration, + errorMessage string) { t.Helper() select { case <-time.After(timeout): @@ -516,7 +538,8 @@ func ensureNoNewEvent(t *testing.T, ch <-chan tmpubsub.Message, timeout time.Dur func ensureNoNewEventOnChannel(t *testing.T, ch <-chan tmpubsub.Message) { t.Helper() - ensureNoNewEvent(t, + ensureNoMessageBeforeTimeout( + t, ch, ensureTimeout, "We should be stuck waiting, not receiving new event on the channel") @@ -524,137 +547,112 @@ func ensureNoNewEventOnChannel(t *testing.T, ch <-chan tmpubsub.Message) { func ensureNoNewRoundStep(t *testing.T, stepCh <-chan tmpubsub.Message) { t.Helper() - ensureNoNewEvent( + ensureNoMessageBeforeTimeout( t, stepCh, ensureTimeout, "We should be stuck waiting, not receiving NewRoundStep event") } -func ensureNoNewUnlock(t *testing.T, unlockCh <-chan tmpubsub.Message) { - t.Helper() - ensureNoNewEvent(t, - unlockCh, - ensureTimeout, - "We should be stuck waiting, not receiving Unlock event") -} - func ensureNoNewTimeout(t *testing.T, stepCh <-chan tmpubsub.Message, timeout int64) { t.Helper() timeoutDuration := time.Duration(timeout*10) * time.Nanosecond - ensureNoNewEvent(t, + ensureNoMessageBeforeTimeout( + t, stepCh, timeoutDuration, "We should be stuck waiting, not receiving NewTimeout event") } -func ensureNewEvent(t *testing.T, ch <-chan tmpubsub.Message, height int64, round int32, timeout time.Duration, errorMessage string) { +func ensureNewEvent(t *testing.T, ch <-chan tmpubsub.Message, height int64, round int32, timeout time.Duration) { t.Helper() - select { - case <-time.After(timeout): - t.Fatal(errorMessage) - case msg := <-ch: - roundStateEvent, ok := msg.Data().(types.EventDataRoundState) - require.True(t, ok, - "expected a EventDataRoundState, got %T. Wrong subscription channel?", - msg.Data()) + msg := ensureMessageBeforeTimeout(t, ch, ensureTimeout) + roundStateEvent, ok := msg.Data().(types.EventDataRoundState) + require.True(t, ok, + "expected a EventDataRoundState, got %T. Wrong subscription channel?", + msg.Data()) - require.Equal(t, height, roundStateEvent.Height) - require.Equal(t, round, roundStateEvent.Round) - // TODO: We could check also for a step at this point! - } + require.Equal(t, height, roundStateEvent.Height) + require.Equal(t, round, roundStateEvent.Round) + // TODO: We could check also for a step at this point! } func ensureNewRound(t *testing.T, roundCh <-chan tmpubsub.Message, height int64, round int32) { t.Helper() - select { - case <-time.After(ensureTimeout): - t.Fatal("Timeout expired while waiting for NewRound event") - case msg := <-roundCh: - newRoundEvent, ok := msg.Data().(types.EventDataNewRound) - require.True(t, ok, "expected a EventDataNewRound, got %T. Wrong subscription channel?", - msg.Data()) + msg := ensureMessageBeforeTimeout(t, roundCh, ensureTimeout) + newRoundEvent, ok := msg.Data().(types.EventDataNewRound) + require.True(t, ok, "expected a EventDataNewRound, got %T. Wrong subscription channel?", + msg.Data()) - require.Equal(t, height, newRoundEvent.Height) - require.Equal(t, round, newRoundEvent.Round) - } + require.Equal(t, height, newRoundEvent.Height) + require.Equal(t, round, newRoundEvent.Round) } func ensureNewTimeout(t *testing.T, timeoutCh <-chan tmpubsub.Message, height int64, round int32, timeout int64) { t.Helper() timeoutDuration := time.Duration(timeout*10) * time.Nanosecond - ensureNewEvent(t, timeoutCh, height, round, timeoutDuration, - "Timeout expired while waiting for NewTimeout event") + ensureNewEvent(t, timeoutCh, height, round, timeoutDuration) } func ensureNewProposal(t *testing.T, proposalCh <-chan tmpubsub.Message, height int64, round int32) { t.Helper() - select { - case <-time.After(ensureTimeout): - t.Fatal("Timeout expired while waiting for NewProposal event") - case msg := <-proposalCh: - proposalEvent, ok := msg.Data().(types.EventDataCompleteProposal) - require.True(t, ok, "expected a EventDataCompleteProposal, got %T. Wrong subscription channel?", - msg.Data()) - - require.Equal(t, height, proposalEvent.Height) - require.Equal(t, round, proposalEvent.Round) - } + msg := ensureMessageBeforeTimeout(t, proposalCh, ensureTimeout) + proposalEvent, ok := msg.Data().(types.EventDataCompleteProposal) + require.True(t, ok, "expected a EventDataCompleteProposal, got %T. Wrong subscription channel?", + msg.Data()) + require.Equal(t, height, proposalEvent.Height) + require.Equal(t, round, proposalEvent.Round) } func ensureNewValidBlock(t *testing.T, validBlockCh <-chan tmpubsub.Message, height int64, round int32) { t.Helper() - ensureNewEvent(t, validBlockCh, height, round, ensureTimeout, - "Timeout expired while waiting for NewValidBlock event") + ensureNewEvent(t, validBlockCh, height, round, ensureTimeout) } func ensureNewBlock(t *testing.T, blockCh <-chan tmpubsub.Message, height int64) { t.Helper() - - select { - case <-time.After(ensureTimeout): - t.Fatal("Timeout expired while waiting for NewBlock event") - case msg := <-blockCh: - blockEvent, ok := msg.Data().(types.EventDataNewBlock) - require.True(t, ok, "expected a EventDataNewBlock, got %T. Wrong subscription channel?", - msg.Data()) - require.Equal(t, height, blockEvent.Block.Height) - } + msg := ensureMessageBeforeTimeout(t, blockCh, ensureTimeout) + blockEvent, ok := msg.Data().(types.EventDataNewBlock) + require.True(t, ok, "expected a EventDataNewBlock, got %T. Wrong subscription channel?", + msg.Data()) + require.Equal(t, height, blockEvent.Block.Height) } func ensureNewBlockHeader(t *testing.T, blockCh <-chan tmpubsub.Message, height int64, blockHash tmbytes.HexBytes) { t.Helper() - select { - case <-time.After(ensureTimeout): - t.Fatal("Timeout expired while waiting for NewBlockHeader event") - case msg := <-blockCh: - blockHeaderEvent, ok := msg.Data().(types.EventDataNewBlockHeader) - require.True(t, ok, "expected a EventDataNewBlockHeader, got %T. Wrong subscription channel?", - msg.Data()) + msg := ensureMessageBeforeTimeout(t, blockCh, ensureTimeout) + blockHeaderEvent, ok := msg.Data().(types.EventDataNewBlockHeader) + require.True(t, ok, "expected a EventDataNewBlockHeader, got %T. Wrong subscription channel?", + msg.Data()) - require.Equal(t, height, blockHeaderEvent.Header.Height) - require.True(t, bytes.Equal(blockHeaderEvent.Header.Hash(), blockHash)) - } + require.Equal(t, height, blockHeaderEvent.Header.Height) + require.True(t, bytes.Equal(blockHeaderEvent.Header.Hash(), blockHash)) +} + +func ensureLock(t *testing.T, lockCh <-chan tmpubsub.Message, height int64, round int32) { + t.Helper() + ensureNewEvent(t, lockCh, height, round, ensureTimeout) } -func ensureNewUnlock(t *testing.T, unlockCh <-chan tmpubsub.Message, height int64, round int32) { +func ensureRelock(t *testing.T, relockCh <-chan tmpubsub.Message, height int64, round int32) { t.Helper() - ensureNewEvent(t, unlockCh, height, round, ensureTimeout, - "Timeout expired while waiting for NewUnlock event") + ensureNewEvent(t, relockCh, height, round, ensureTimeout) } func ensureProposal(t *testing.T, proposalCh <-chan tmpubsub.Message, height int64, round int32, propID types.BlockID) { + ensureProposalWithTimeout(t, proposalCh, height, round, &propID, ensureTimeout) +} + +func ensureProposalWithTimeout(t *testing.T, proposalCh <-chan tmpubsub.Message, height int64, round int32, propID *types.BlockID, timeout time.Duration) { t.Helper() - select { - case <-time.After(ensureTimeout): - t.Fatal("Timeout expired while waiting for NewProposal event") - case msg := <-proposalCh: - proposalEvent, ok := msg.Data().(types.EventDataCompleteProposal) - require.True(t, ok, "expected a EventDataCompleteProposal, got %T. Wrong subscription channel?", - msg.Data()) - require.Equal(t, height, proposalEvent.Height) - require.Equal(t, round, proposalEvent.Round) - require.True(t, proposalEvent.BlockID.Equals(propID), + msg := ensureMessageBeforeTimeout(t, proposalCh, timeout) + proposalEvent, ok := msg.Data().(types.EventDataCompleteProposal) + require.True(t, ok, "expected a EventDataCompleteProposal, got %T. Wrong subscription channel?", + msg.Data()) + require.Equal(t, height, proposalEvent.Height) + require.Equal(t, round, proposalEvent.Round) + if propID != nil { + require.True(t, proposalEvent.BlockID.Equals(*propID), "Proposed block does not match expected block (%v != %v)", proposalEvent.BlockID, propID) } } @@ -703,38 +701,32 @@ func ensureVoteMatch(t *testing.T, voteCh <-chan tmpubsub.Message, height int64, } func ensureVote(t *testing.T, voteCh <-chan tmpubsub.Message, height int64, round int32, voteType tmproto.SignedMsgType) { t.Helper() - select { - case <-time.After(ensureTimeout): - t.Fatal("Timeout expired while waiting for NewVote event") - case msg := <-voteCh: - voteEvent, ok := msg.Data().(types.EventDataVote) - require.True(t, ok, "expected a EventDataVote, got %T. Wrong subscription channel?", - msg.Data()) + msg := ensureMessageBeforeTimeout(t, voteCh, ensureTimeout) + voteEvent, ok := msg.Data().(types.EventDataVote) + require.True(t, ok, "expected a EventDataVote, got %T. Wrong subscription channel?", + msg.Data()) - vote := voteEvent.Vote - require.Equal(t, height, vote.Height) - require.Equal(t, round, vote.Round) + vote := voteEvent.Vote + require.Equal(t, height, vote.Height) + require.Equal(t, round, vote.Round) - require.Equal(t, voteType, vote.Type) - } + require.Equal(t, voteType, vote.Type) } -func ensurePrecommitTimeout(t *testing.T, ch <-chan tmpubsub.Message) { +func ensureNewEventOnChannel(t *testing.T, ch <-chan tmpubsub.Message) { t.Helper() - select { - case <-time.After(ensureTimeout): - t.Fatal("Timeout expired while waiting for the Precommit to Timeout") - case <-ch: - } + ensureMessageBeforeTimeout(t, ch, ensureTimeout) } -func ensureNewEventOnChannel(t *testing.T, ch <-chan tmpubsub.Message) { +func ensureMessageBeforeTimeout(t *testing.T, ch <-chan tmpubsub.Message, to time.Duration) tmpubsub.Message { t.Helper() select { - case <-time.After(ensureTimeout): - t.Fatal("Timeout expired while waiting for new activity on the channel") - case <-ch: + case <-time.After(to): + t.Fatalf("Timeout expired while waiting for message") + case msg := <-ch: + return msg } + panic("unreachable") } //------------------------------------------------------------------------------- @@ -746,7 +738,7 @@ func consensusLogger() log.Logger { return log.TestingLogger().With("module", "consensus") } -func randConsensusState( +func makeConsensusState( ctx context.Context, t *testing.T, cfg *config.Config, @@ -756,8 +748,10 @@ func randConsensusState( appFunc func(t *testing.T, logger log.Logger) abci.Application, configOpts ...func(*config.Config), ) ([]*State, cleanupFunc) { + t.Helper() - genDoc, privVals := factory.RandGenesisDoc(ctx, t, cfg, nValidators, false, 30) + valSet, privVals := factory.ValidatorSet(ctx, t, nValidators, 30) + genDoc := factory.GenesisDoc(cfg, time.Now(), valSet.Validators, nil) css := make([]*State, nValidators) logger := consensusLogger() @@ -817,8 +811,10 @@ func randConsensusNetWithPeers( ) ([]*State, *types.GenesisDoc, *config.Config, cleanupFunc) { t.Helper() - genDoc, privVals := factory.RandGenesisDoc(ctx, t, cfg, nValidators, false, testMinPower) + valSet, privVals := factory.ValidatorSet(ctx, t, nValidators, testMinPower) + genDoc := factory.GenesisDoc(cfg, time.Now(), valSet.Validators, nil) css := make([]*State, nPeers) + t.Helper() logger := consensusLogger() var peer0Config *config.Config @@ -866,16 +862,29 @@ func randConsensusNetWithPeers( } } -func randGenesisState( - ctx context.Context, - t *testing.T, - cfg *config.Config, - numValidators int, - randPower bool, - minPower int64, -) (sm.State, []types.PrivValidator) { +type genesisStateArgs struct { + Validators int + Power int64 + Params *types.ConsensusParams + Time time.Time +} - genDoc, privValidators := factory.RandGenesisDoc(ctx, t, cfg, numValidators, randPower, minPower) +func makeGenesisState(ctx context.Context, t *testing.T, cfg *config.Config, args genesisStateArgs) (sm.State, []types.PrivValidator) { + t.Helper() + if args.Power == 0 { + args.Power = 1 + } + if args.Validators == 0 { + args.Power = 4 + } + valSet, privValidators := factory.ValidatorSet(ctx, t, args.Validators, args.Power) + if args.Params == nil { + args.Params = types.DefaultConsensusParams() + } + if args.Time.IsZero() { + args.Time = time.Now() + } + genDoc := factory.GenesisDoc(cfg, args.Time, valSet.Validators, args.Params) s0, err := sm.MakeGenesisState(genDoc) require.NoError(t, err) return s0, privValidators @@ -884,7 +893,7 @@ func randGenesisState( func newMockTickerFunc(onlyOnce bool) func() TimeoutTicker { return func() TimeoutTicker { return &mockTicker{ - c: make(chan timeoutInfo, 10), + c: make(chan timeoutInfo, 100), onlyOnce: onlyOnce, } } diff --git a/internal/consensus/invalid_test.go b/internal/consensus/invalid_test.go index e06e51a61..541b1bbc4 100644 --- a/internal/consensus/invalid_test.go +++ b/internal/consensus/invalid_test.go @@ -11,6 +11,7 @@ import ( "github.com/tendermint/tendermint/internal/p2p" "github.com/tendermint/tendermint/libs/bytes" tmrand "github.com/tendermint/tendermint/libs/rand" + tmtime "github.com/tendermint/tendermint/libs/time" tmcons "github.com/tendermint/tendermint/proto/tendermint/consensus" tmproto "github.com/tendermint/tendermint/proto/tendermint/types" "github.com/tendermint/tendermint/types" @@ -23,7 +24,7 @@ func TestReactorInvalidPrecommit(t *testing.T) { config := configSetup(t) n := 4 - states, cleanup := randConsensusState(ctx, t, + states, cleanup := makeConsensusState(ctx, t, config, n, "consensus_reactor_test", newMockTickerFunc(true), newKVStore) t.Cleanup(cleanup) @@ -107,7 +108,7 @@ func invalidDoPrevoteFunc( ValidatorIndex: valIndex, Height: cs.Height, Round: cs.Round, - Timestamp: cs.voteTime(), + Timestamp: tmtime.Now(), Type: tmproto.PrecommitType, BlockID: types.BlockID{ Hash: blockHash, diff --git a/internal/consensus/mempool_test.go b/internal/consensus/mempool_test.go index 52df2c1f3..78f064600 100644 --- a/internal/consensus/mempool_test.go +++ b/internal/consensus/mempool_test.go @@ -40,7 +40,9 @@ func TestMempoolNoProgressUntilTxsAvailable(t *testing.T) { t.Cleanup(func() { _ = os.RemoveAll(config.RootDir) }) config.Consensus.CreateEmptyBlocks = false - state, privVals := randGenesisState(ctx, t, baseConfig, 1, false, 10) + state, privVals := makeGenesisState(ctx, t, baseConfig, genesisStateArgs{ + Validators: 1, + Power: 10}) cs := newStateWithConfig(ctx, t, log.TestingLogger(), config, state, privVals[0], NewCounterApplication()) assertMempool(t, cs.txNotifier).EnableTxsAvailable() height, round := cs.Height, cs.Round @@ -65,7 +67,9 @@ func TestMempoolProgressAfterCreateEmptyBlocksInterval(t *testing.T) { t.Cleanup(func() { _ = os.RemoveAll(config.RootDir) }) config.Consensus.CreateEmptyBlocksInterval = ensureTimeout - state, privVals := randGenesisState(ctx, t, baseConfig, 1, false, 10) + state, privVals := makeGenesisState(ctx, t, baseConfig, genesisStateArgs{ + Validators: 1, + Power: 10}) cs := newStateWithConfig(ctx, t, log.TestingLogger(), config, state, privVals[0], NewCounterApplication()) assertMempool(t, cs.txNotifier).EnableTxsAvailable() @@ -88,20 +92,22 @@ func TestMempoolProgressInHigherRound(t *testing.T) { t.Cleanup(func() { _ = os.RemoveAll(config.RootDir) }) config.Consensus.CreateEmptyBlocks = false - state, privVals := randGenesisState(ctx, t, baseConfig, 1, false, 10) + state, privVals := makeGenesisState(ctx, t, baseConfig, genesisStateArgs{ + Validators: 1, + Power: 10}) cs := newStateWithConfig(ctx, t, log.TestingLogger(), config, state, privVals[0], NewCounterApplication()) assertMempool(t, cs.txNotifier).EnableTxsAvailable() height, round := cs.Height, cs.Round newBlockCh := subscribe(ctx, t, cs.eventBus, types.EventQueryNewBlock) newRoundCh := subscribe(ctx, t, cs.eventBus, types.EventQueryNewRound) timeoutCh := subscribe(ctx, t, cs.eventBus, types.EventQueryTimeoutPropose) - cs.setProposal = func(proposal *types.Proposal) error { + cs.setProposal = func(proposal *types.Proposal, recvTime time.Time) error { if cs.Height == 2 && cs.Round == 0 { // dont set the proposal in round 0 so we timeout and // go to next round return nil } - return cs.defaultSetProposal(proposal) + return cs.defaultSetProposal(proposal, recvTime) } startTestRound(ctx, cs, height, round) @@ -137,7 +143,9 @@ func TestMempoolTxConcurrentWithCommit(t *testing.T) { config := configSetup(t) logger := log.TestingLogger() - state, privVals := randGenesisState(ctx, t, config, 1, false, 10) + state, privVals := makeGenesisState(ctx, t, config, genesisStateArgs{ + Validators: 1, + Power: 10}) stateStore := sm.NewStore(dbm.NewMemDB()) blockStore := store.NewBlockStore(dbm.NewMemDB()) @@ -170,7 +178,9 @@ func TestMempoolRmBadTx(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() - state, privVals := randGenesisState(ctx, t, config, 1, false, 10) + state, privVals := makeGenesisState(ctx, t, config, genesisStateArgs{ + Validators: 1, + Power: 10}) app := NewCounterApplication() stateStore := sm.NewStore(dbm.NewMemDB()) blockStore := store.NewBlockStore(dbm.NewMemDB()) diff --git a/internal/consensus/pbts_test.go b/internal/consensus/pbts_test.go new file mode 100644 index 000000000..b8de658ea --- /dev/null +++ b/internal/consensus/pbts_test.go @@ -0,0 +1,489 @@ +package consensus + +import ( + "bytes" + "context" + "testing" + "time" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "github.com/tendermint/tendermint/abci/example/kvstore" + "github.com/tendermint/tendermint/internal/eventbus" + tmpubsub "github.com/tendermint/tendermint/internal/pubsub" + "github.com/tendermint/tendermint/libs/log" + tmtimemocks "github.com/tendermint/tendermint/libs/time/mocks" + tmproto "github.com/tendermint/tendermint/proto/tendermint/types" + "github.com/tendermint/tendermint/types" +) + +const ( + // blockTimeIota is used in the test harness as the time between + // blocks when not otherwise specified. + blockTimeIota = time.Millisecond +) + +// pbtsTestHarness constructs a Tendermint network that can be used for testing the +// implementation of the Proposer-Based timestamps algorithm. +// It runs a series of consensus heights and captures timing of votes and events. +type pbtsTestHarness struct { + // configuration options set by the user of the test harness. + pbtsTestConfiguration + + // The Tendermint consensus state machine being run during + // a run of the pbtsTestHarness. + observedState *State + + // A stub for signing votes and messages using the key + // from the observedState. + observedValidator *validatorStub + + // A list of simulated validators that interact with the observedState and are + // fully controlled by the test harness. + otherValidators []*validatorStub + + // The mock time source used by all of the validator stubs in the test harness. + // This mock clock allows the test harness to produce votes and blocks with arbitrary + // timestamps. + validatorClock *tmtimemocks.Source + + chainID string + + // channels for verifying that the observed validator completes certain actions. + ensureProposalCh, roundCh, blockCh, ensureVoteCh <-chan tmpubsub.Message + + // channel of events from the observed validator annotated with the timestamp + // the event was received. + eventCh <-chan timestampedEvent + + currentHeight int64 + currentRound int32 +} + +type pbtsTestConfiguration struct { + // The timestamp consensus parameters to be used by the state machine under test. + synchronyParams types.SynchronyParams + + // The setting to use for the TimeoutPropose configuration parameter. + timeoutPropose time.Duration + + // The timestamp of the first block produced by the network. + genesisTime time.Time + + // The time at which the proposal at height 2 should be delivered. + height2ProposalDeliverTime time.Time + + // The timestamp of the block proposed at height 2. + height2ProposedBlockTime time.Time + + // The timestamp of the block proposed at height 4. + // At height 4, the proposed block time and the deliver time are the same so + // that timely-ness does not affect height 4. + height4ProposedBlockTime time.Time +} + +func newPBTSTestHarness(ctx context.Context, t *testing.T, tc pbtsTestConfiguration) pbtsTestHarness { + t.Helper() + const validators = 4 + cfg := configSetup(t) + clock := new(tmtimemocks.Source) + if tc.height4ProposedBlockTime.IsZero() { + + // Set a default height4ProposedBlockTime. + // Use a proposed block time that is greater than the time that the + // block at height 2 was delivered. Height 3 is not relevant for testing + // and always occurs blockTimeIota before height 4. If not otherwise specified, + // height 4 therefore occurs 2*blockTimeIota after height 2. + tc.height4ProposedBlockTime = tc.height2ProposalDeliverTime.Add(2 * blockTimeIota) + } + cfg.Consensus.TimeoutPropose = tc.timeoutPropose + consensusParams := types.DefaultConsensusParams() + consensusParams.Synchrony = tc.synchronyParams + + state, privVals := makeGenesisState(ctx, t, cfg, genesisStateArgs{ + Params: consensusParams, + Time: tc.genesisTime, + Validators: validators, + }) + cs := newState(ctx, t, log.TestingLogger(), state, privVals[0], kvstore.NewApplication()) + vss := make([]*validatorStub, validators) + for i := 0; i < validators; i++ { + vss[i] = newValidatorStub(privVals[i], int32(i)) + } + incrementHeight(vss[1:]...) + + for _, vs := range vss { + vs.clock = clock + } + pubKey, err := vss[0].PrivValidator.GetPubKey(ctx) + require.NoError(t, err) + + eventCh := timestampedCollector(ctx, t, cs.eventBus) + + return pbtsTestHarness{ + pbtsTestConfiguration: tc, + observedValidator: vss[0], + observedState: cs, + otherValidators: vss[1:], + validatorClock: clock, + currentHeight: 1, + chainID: cfg.ChainID(), + roundCh: subscribe(ctx, t, cs.eventBus, types.EventQueryNewRound), + ensureProposalCh: subscribe(ctx, t, cs.eventBus, types.EventQueryCompleteProposal), + blockCh: subscribe(ctx, t, cs.eventBus, types.EventQueryNewBlock), + ensureVoteCh: subscribeToVoterBuffered(ctx, t, cs, pubKey.Address()), + eventCh: eventCh, + } +} + +func (p *pbtsTestHarness) observedValidatorProposerHeight(ctx context.Context, t *testing.T, previousBlockTime time.Time) heightResult { + p.validatorClock.On("Now").Return(p.height2ProposedBlockTime).Times(6) + + ensureNewRound(t, p.roundCh, p.currentHeight, p.currentRound) + + timeout := time.Until(previousBlockTime.Add(ensureTimeout)) + ensureProposalWithTimeout(t, p.ensureProposalCh, p.currentHeight, p.currentRound, nil, timeout) + + rs := p.observedState.GetRoundState() + bid := types.BlockID{Hash: rs.ProposalBlock.Hash(), PartSetHeader: rs.ProposalBlockParts.Header()} + ensurePrevote(t, p.ensureVoteCh, p.currentHeight, p.currentRound) + signAddVotes(ctx, t, p.observedState, tmproto.PrevoteType, p.chainID, bid, p.otherValidators...) + + signAddVotes(ctx, t, p.observedState, tmproto.PrecommitType, p.chainID, bid, p.otherValidators...) + ensurePrecommit(t, p.ensureVoteCh, p.currentHeight, p.currentRound) + + ensureNewBlock(t, p.blockCh, p.currentHeight) + + vk, err := p.observedValidator.GetPubKey(ctx) + require.NoError(t, err) + res := collectHeightResults(ctx, t, p.eventCh, p.currentHeight, vk.Address()) + + p.currentHeight++ + incrementHeight(p.otherValidators...) + return res +} + +func (p *pbtsTestHarness) height2(ctx context.Context, t *testing.T) heightResult { + signer := p.otherValidators[0].PrivValidator + height3BlockTime := p.height2ProposedBlockTime.Add(-blockTimeIota) + return p.nextHeight(ctx, t, signer, p.height2ProposalDeliverTime, p.height2ProposedBlockTime, height3BlockTime) +} + +func (p *pbtsTestHarness) intermediateHeights(ctx context.Context, t *testing.T) { + signer := p.otherValidators[1].PrivValidator + blockTimeHeight3 := p.height4ProposedBlockTime.Add(-blockTimeIota) + p.nextHeight(ctx, t, signer, blockTimeHeight3, blockTimeHeight3, p.height4ProposedBlockTime) + + signer = p.otherValidators[2].PrivValidator + p.nextHeight(ctx, t, signer, p.height4ProposedBlockTime, p.height4ProposedBlockTime, time.Now()) +} + +func (p *pbtsTestHarness) height5(ctx context.Context, t *testing.T) heightResult { + return p.observedValidatorProposerHeight(ctx, t, p.height4ProposedBlockTime) +} + +func (p *pbtsTestHarness) nextHeight(ctx context.Context, t *testing.T, proposer types.PrivValidator, deliverTime, proposedTime, nextProposedTime time.Time) heightResult { + p.validatorClock.On("Now").Return(nextProposedTime).Times(6) + + ensureNewRound(t, p.roundCh, p.currentHeight, p.currentRound) + + b, _, err := p.observedState.createProposalBlock() + require.NoError(t, err) + b.Height = p.currentHeight + b.Header.Height = p.currentHeight + b.Header.Time = proposedTime + + k, err := proposer.GetPubKey(ctx) + require.NoError(t, err) + b.Header.ProposerAddress = k.Address() + ps, err := b.MakePartSet(types.BlockPartSizeBytes) + require.NoError(t, err) + bid := types.BlockID{Hash: b.Hash(), PartSetHeader: ps.Header()} + prop := types.NewProposal(p.currentHeight, 0, -1, bid, proposedTime) + tp := prop.ToProto() + + if err := proposer.SignProposal(ctx, p.observedState.state.ChainID, tp); err != nil { + t.Fatalf("error signing proposal: %s", err) + } + + time.Sleep(time.Until(deliverTime)) + prop.Signature = tp.Signature + if err := p.observedState.SetProposalAndBlock(ctx, prop, b, ps, "peerID"); err != nil { + t.Fatal(err) + } + ensureProposal(t, p.ensureProposalCh, p.currentHeight, 0, bid) + + ensurePrevote(t, p.ensureVoteCh, p.currentHeight, p.currentRound) + signAddVotes(ctx, t, p.observedState, tmproto.PrevoteType, p.chainID, bid, p.otherValidators...) + + signAddVotes(ctx, t, p.observedState, tmproto.PrecommitType, p.chainID, bid, p.otherValidators...) + ensurePrecommit(t, p.ensureVoteCh, p.currentHeight, p.currentRound) + + vk, err := p.observedValidator.GetPubKey(ctx) + require.NoError(t, err) + res := collectHeightResults(ctx, t, p.eventCh, p.currentHeight, vk.Address()) + ensureNewBlock(t, p.blockCh, p.currentHeight) + + p.currentHeight++ + incrementHeight(p.otherValidators...) + return res +} + +func timestampedCollector(ctx context.Context, t *testing.T, eb *eventbus.EventBus) <-chan timestampedEvent { + t.Helper() + + // Since eventCh is not read until the end of each height, it must be large + // enough to hold all of the events produced during a single height. + eventCh := make(chan timestampedEvent, 100) + + if err := eb.Observe(ctx, func(msg tmpubsub.Message) error { + eventCh <- timestampedEvent{ + ts: time.Now(), + m: msg, + } + return nil + }, types.EventQueryVote, types.EventQueryCompleteProposal); err != nil { + t.Fatalf("Failed to observe query %v: %v", types.EventQueryVote, err) + } + return eventCh +} + +func collectHeightResults(ctx context.Context, t *testing.T, eventCh <-chan timestampedEvent, height int64, address []byte) heightResult { + t.Helper() + var res heightResult + for event := range eventCh { + switch v := event.m.Data().(type) { + case types.EventDataVote: + if v.Vote.Height > height { + t.Fatalf("received prevote from unexpected height, expected: %d, saw: %d", height, v.Vote.Height) + } + if !bytes.Equal(address, v.Vote.ValidatorAddress) { + continue + } + if v.Vote.Type != tmproto.PrevoteType { + continue + } + res.prevote = v.Vote + res.prevoteIssuedAt = event.ts + + case types.EventDataCompleteProposal: + if v.Height > height { + t.Fatalf("received proposal from unexpected height, expected: %d, saw: %d", height, v.Height) + } + res.proposalIssuedAt = event.ts + } + if res.isComplete() { + return res + } + } + t.Fatalf("complete height result never seen for height %d", height) + + panic("unreachable") +} + +type timestampedEvent struct { + ts time.Time + m tmpubsub.Message +} + +func (p *pbtsTestHarness) run(ctx context.Context, t *testing.T) resultSet { + startTestRound(ctx, p.observedState, p.currentHeight, p.currentRound) + + r1 := p.observedValidatorProposerHeight(ctx, t, p.genesisTime) + r2 := p.height2(ctx, t) + p.intermediateHeights(ctx, t) + r5 := p.height5(ctx, t) + return resultSet{ + genesisHeight: r1, + height2: r2, + height5: r5, + } +} + +type resultSet struct { + genesisHeight heightResult + height2 heightResult + height5 heightResult +} + +type heightResult struct { + proposalIssuedAt time.Time + prevote *types.Vote + prevoteIssuedAt time.Time +} + +func (hr heightResult) isComplete() bool { + return !hr.proposalIssuedAt.IsZero() && !hr.prevoteIssuedAt.IsZero() && hr.prevote != nil +} + +// TestProposerWaitsForGenesisTime tests that a proposer will not propose a block +// until after the genesis time has passed. The test sets the genesis time in the +// future and then ensures that the observed validator waits to propose a block. +func TestProposerWaitsForGenesisTime(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + // create a genesis time far (enough) in the future. + initialTime := time.Now().Add(800 * time.Millisecond) + cfg := pbtsTestConfiguration{ + synchronyParams: types.SynchronyParams{ + Precision: 10 * time.Millisecond, + MessageDelay: 10 * time.Millisecond, + }, + timeoutPropose: 10 * time.Millisecond, + genesisTime: initialTime, + height2ProposalDeliverTime: initialTime.Add(10 * time.Millisecond), + height2ProposedBlockTime: initialTime.Add(10 * time.Millisecond), + } + + pbtsTest := newPBTSTestHarness(ctx, t, cfg) + results := pbtsTest.run(ctx, t) + + // ensure that the proposal was issued after the genesis time. + assert.True(t, results.genesisHeight.proposalIssuedAt.After(cfg.genesisTime)) +} + +// TestProposerWaitsForPreviousBlock tests that the proposer of a block waits until +// the block time of the previous height has passed to propose the next block. +// The test harness ensures that the observed validator will be the proposer at +// height 1 and height 5. The test sets the block time of height 4 in the future +// and then verifies that the observed validator waits until after the block time +// of height 4 to propose a block at height 5. +func TestProposerWaitsForPreviousBlock(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + initialTime := time.Now().Add(time.Millisecond * 50) + cfg := pbtsTestConfiguration{ + synchronyParams: types.SynchronyParams{ + Precision: 100 * time.Millisecond, + MessageDelay: 500 * time.Millisecond, + }, + timeoutPropose: 50 * time.Millisecond, + genesisTime: initialTime, + height2ProposalDeliverTime: initialTime.Add(150 * time.Millisecond), + height2ProposedBlockTime: initialTime.Add(100 * time.Millisecond), + height4ProposedBlockTime: initialTime.Add(800 * time.Millisecond), + } + + pbtsTest := newPBTSTestHarness(ctx, t, cfg) + results := pbtsTest.run(ctx, t) + + // the observed validator is the proposer at height 5. + // ensure that the observed validator did not propose a block until after + // the time configured for height 4. + assert.True(t, results.height5.proposalIssuedAt.After(cfg.height4ProposedBlockTime)) + + // Ensure that the validator issued a prevote for a non-nil block. + assert.NotNil(t, results.height5.prevote.BlockID.Hash) +} + +func TestProposerWaitTime(t *testing.T) { + genesisTime, err := time.Parse(time.RFC3339, "2019-03-13T23:00:00Z") + require.NoError(t, err) + testCases := []struct { + name string + previousBlockTime time.Time + localTime time.Time + expectedWait time.Duration + }{ + { + name: "block time greater than local time", + previousBlockTime: genesisTime.Add(5 * time.Nanosecond), + localTime: genesisTime.Add(1 * time.Nanosecond), + expectedWait: 4 * time.Nanosecond, + }, + { + name: "local time greater than block time", + previousBlockTime: genesisTime.Add(1 * time.Nanosecond), + localTime: genesisTime.Add(5 * time.Nanosecond), + expectedWait: 0, + }, + { + name: "both times equal", + previousBlockTime: genesisTime.Add(5 * time.Nanosecond), + localTime: genesisTime.Add(5 * time.Nanosecond), + expectedWait: 0, + }, + } + for _, testCase := range testCases { + t.Run(testCase.name, func(t *testing.T) { + mockSource := new(tmtimemocks.Source) + mockSource.On("Now").Return(testCase.localTime) + + ti := proposerWaitTime(mockSource, testCase.previousBlockTime) + assert.Equal(t, testCase.expectedWait, ti) + }) + } +} + +func TestTimelyProposal(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + initialTime := time.Now() + + cfg := pbtsTestConfiguration{ + synchronyParams: types.SynchronyParams{ + Precision: 10 * time.Millisecond, + MessageDelay: 140 * time.Millisecond, + }, + timeoutPropose: 40 * time.Millisecond, + genesisTime: initialTime, + height2ProposedBlockTime: initialTime.Add(10 * time.Millisecond), + height2ProposalDeliverTime: initialTime.Add(30 * time.Millisecond), + } + + pbtsTest := newPBTSTestHarness(ctx, t, cfg) + results := pbtsTest.run(ctx, t) + require.NotNil(t, results.height2.prevote.BlockID.Hash) +} + +func TestTooFarInThePastProposal(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + initialTime := time.Now() + + // localtime > proposedBlockTime + MsgDelay + Precision + cfg := pbtsTestConfiguration{ + synchronyParams: types.SynchronyParams{ + Precision: 1 * time.Millisecond, + MessageDelay: 10 * time.Millisecond, + }, + timeoutPropose: 50 * time.Millisecond, + genesisTime: initialTime, + height2ProposedBlockTime: initialTime.Add(10 * time.Millisecond), + height2ProposalDeliverTime: initialTime.Add(21 * time.Millisecond), + } + + pbtsTest := newPBTSTestHarness(ctx, t, cfg) + results := pbtsTest.run(ctx, t) + + require.Nil(t, results.height2.prevote.BlockID.Hash) +} + +func TestTooFarInTheFutureProposal(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + initialTime := time.Now() + + // localtime < proposedBlockTime - Precision + cfg := pbtsTestConfiguration{ + synchronyParams: types.SynchronyParams{ + Precision: 1 * time.Millisecond, + MessageDelay: 10 * time.Millisecond, + }, + timeoutPropose: 50 * time.Millisecond, + genesisTime: initialTime, + height2ProposedBlockTime: initialTime.Add(100 * time.Millisecond), + height2ProposalDeliverTime: initialTime.Add(10 * time.Millisecond), + height4ProposedBlockTime: initialTime.Add(150 * time.Millisecond), + } + + pbtsTest := newPBTSTestHarness(ctx, t, cfg) + results := pbtsTest.run(ctx, t) + + require.Nil(t, results.height2.prevote.BlockID.Hash) +} diff --git a/internal/consensus/reactor.go b/internal/consensus/reactor.go index 127b175bc..7f4dc4426 100644 --- a/internal/consensus/reactor.go +++ b/internal/consensus/reactor.go @@ -16,6 +16,7 @@ import ( tmevents "github.com/tendermint/tendermint/libs/events" "github.com/tendermint/tendermint/libs/log" "github.com/tendermint/tendermint/libs/service" + tmtime "github.com/tendermint/tendermint/libs/time" tmcons "github.com/tendermint/tendermint/proto/tendermint/consensus" tmproto "github.com/tendermint/tendermint/proto/tendermint/types" "github.com/tendermint/tendermint/types" @@ -1184,7 +1185,7 @@ func (r *Reactor) handleDataMessage(ctx context.Context, envelope *p2p.Envelope, select { case <-ctx.Done(): return ctx.Err() - case r.state.peerMsgQueue <- msgInfo{pMsg, envelope.From}: + case r.state.peerMsgQueue <- msgInfo{pMsg, envelope.From, tmtime.Now()}: } case *tmcons.ProposalPOL: ps.ApplyProposalPOLMessage(msgI.(*ProposalPOLMessage)) @@ -1194,7 +1195,7 @@ func (r *Reactor) handleDataMessage(ctx context.Context, envelope *p2p.Envelope, ps.SetHasProposalBlockPart(bpMsg.Height, bpMsg.Round, int(bpMsg.Part.Index)) r.Metrics.BlockParts.With("peer_id", string(envelope.From)).Add(1) select { - case r.state.peerMsgQueue <- msgInfo{bpMsg, envelope.From}: + case r.state.peerMsgQueue <- msgInfo{bpMsg, envelope.From, tmtime.Now()}: return nil case <-ctx.Done(): return ctx.Err() @@ -1238,7 +1239,7 @@ func (r *Reactor) handleVoteMessage(ctx context.Context, envelope *p2p.Envelope, ps.SetHasVote(vMsg.Vote) select { - case r.state.peerMsgQueue <- msgInfo{vMsg, envelope.From}: + case r.state.peerMsgQueue <- msgInfo{vMsg, envelope.From, tmtime.Now()}: return nil case <-ctx.Done(): return ctx.Err() diff --git a/internal/consensus/reactor_test.go b/internal/consensus/reactor_test.go index b489ca8fa..c0a6445aa 100644 --- a/internal/consensus/reactor_test.go +++ b/internal/consensus/reactor_test.go @@ -324,7 +324,7 @@ func TestReactorBasic(t *testing.T) { cfg := configSetup(t) n := 4 - states, cleanup := randConsensusState(ctx, t, + states, cleanup := makeConsensusState(ctx, t, cfg, n, "consensus_reactor_test", newMockTickerFunc(true), newKVStore) t.Cleanup(cleanup) @@ -381,7 +381,8 @@ func TestReactorWithEvidence(t *testing.T) { tickerFunc := newMockTickerFunc(true) appFunc := newKVStore - genDoc, privVals := factory.RandGenesisDoc(ctx, t, cfg, n, false, 30) + valSet, privVals := factory.ValidatorSet(ctx, t, n, 30) + genDoc := factory.GenesisDoc(cfg, time.Now(), valSet.Validators, nil) states := make([]*State, n) logger := consensusLogger() @@ -484,8 +485,7 @@ func TestReactorCreatesBlockWhenEmptyBlocksFalse(t *testing.T) { cfg := configSetup(t) n := 4 - states, cleanup := randConsensusState( - ctx, + states, cleanup := makeConsensusState(ctx, t, cfg, n, @@ -541,7 +541,7 @@ func TestReactorRecordsVotesAndBlockParts(t *testing.T) { cfg := configSetup(t) n := 4 - states, cleanup := randConsensusState(ctx, t, + states, cleanup := makeConsensusState(ctx, t, cfg, n, "consensus_reactor_test", newMockTickerFunc(true), newKVStore) t.Cleanup(cleanup) @@ -606,8 +606,7 @@ func TestReactorVotingPowerChange(t *testing.T) { cfg := configSetup(t) n := 4 - states, cleanup := randConsensusState( - ctx, + states, cleanup := makeConsensusState(ctx, t, cfg, n, diff --git a/internal/consensus/replay_test.go b/internal/consensus/replay_test.go index 9f8d31714..566aa8bff 100644 --- a/internal/consensus/replay_test.go +++ b/internal/consensus/replay_test.go @@ -367,8 +367,8 @@ func setupSimulator(ctx context.Context, t *testing.T) *simulatorTestSuite { ensureNewProposal(t, proposalCh, height, round) rs := css[0].GetRoundState() - signAddVotes(ctx, t, sim.Config, css[0], tmproto.PrecommitType, - rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header(), + signAddVotes(ctx, t, css[0], tmproto.PrecommitType, sim.Config.ChainID(), + types.BlockID{Hash: rs.ProposalBlock.Hash(), PartSetHeader: rs.ProposalBlockParts.Header()}, vss[1:nVals]...) ensureNewRound(t, newRoundCh, height+1, 0) @@ -389,7 +389,7 @@ func setupSimulator(ctx context.Context, t *testing.T) *simulatorTestSuite { require.NoError(t, err) blockID := types.BlockID{Hash: propBlock.Hash(), PartSetHeader: propBlockParts.Header()} - proposal := types.NewProposal(vss[1].Height, round, -1, blockID) + proposal := types.NewProposal(vss[1].Height, round, -1, blockID, propBlock.Header.Time) p := proposal.ToProto() if err := vss[1].SignProposal(ctx, cfg.ChainID(), p); err != nil { t.Fatal("failed to sign bad proposal", err) @@ -402,8 +402,8 @@ func setupSimulator(ctx context.Context, t *testing.T) *simulatorTestSuite { } ensureNewProposal(t, proposalCh, height, round) rs = css[0].GetRoundState() - signAddVotes(ctx, t, sim.Config, css[0], tmproto.PrecommitType, - rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header(), + signAddVotes(ctx, t, css[0], tmproto.PrecommitType, sim.Config.ChainID(), + types.BlockID{Hash: rs.ProposalBlock.Hash(), PartSetHeader: rs.ProposalBlockParts.Header()}, vss[1:nVals]...) ensureNewRound(t, newRoundCh, height+1, 0) @@ -423,7 +423,7 @@ func setupSimulator(ctx context.Context, t *testing.T) *simulatorTestSuite { require.NoError(t, err) blockID = types.BlockID{Hash: propBlock.Hash(), PartSetHeader: propBlockParts.Header()} - proposal = types.NewProposal(vss[2].Height, round, -1, blockID) + proposal = types.NewProposal(vss[2].Height, round, -1, blockID, propBlock.Header.Time) p = proposal.ToProto() if err := vss[2].SignProposal(ctx, cfg.ChainID(), p); err != nil { t.Fatal("failed to sign bad proposal", err) @@ -436,8 +436,8 @@ func setupSimulator(ctx context.Context, t *testing.T) *simulatorTestSuite { } ensureNewProposal(t, proposalCh, height, round) rs = css[0].GetRoundState() - signAddVotes(ctx, t, sim.Config, css[0], tmproto.PrecommitType, - rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header(), + signAddVotes(ctx, t, css[0], tmproto.PrecommitType, sim.Config.ChainID(), + types.BlockID{Hash: rs.ProposalBlock.Hash(), PartSetHeader: rs.ProposalBlockParts.Header()}, vss[1:nVals]...) ensureNewRound(t, newRoundCh, height+1, 0) @@ -486,7 +486,7 @@ func setupSimulator(ctx context.Context, t *testing.T) *simulatorTestSuite { selfIndex := valIndexFn(0) require.NotEqual(t, -1, selfIndex) - proposal = types.NewProposal(vss[3].Height, round, -1, blockID) + proposal = types.NewProposal(vss[3].Height, round, -1, blockID, propBlock.Header.Time) p = proposal.ToProto() if err := vss[3].SignProposal(ctx, cfg.ChainID(), p); err != nil { t.Fatal("failed to sign bad proposal", err) @@ -508,9 +508,10 @@ func setupSimulator(ctx context.Context, t *testing.T) *simulatorTestSuite { if i == selfIndex { continue } - signAddVotes(ctx, t, sim.Config, css[0], - tmproto.PrecommitType, rs.ProposalBlock.Hash(), - rs.ProposalBlockParts.Header(), newVss[i]) + signAddVotes(ctx, t, css[0], + tmproto.PrecommitType, sim.Config.ChainID(), + types.BlockID{Hash: rs.ProposalBlock.Hash(), PartSetHeader: rs.ProposalBlockParts.Header()}, + newVss[i]) } ensureNewRound(t, newRoundCh, height+1, 0) @@ -532,9 +533,10 @@ func setupSimulator(ctx context.Context, t *testing.T) *simulatorTestSuite { if i == selfIndex { continue } - signAddVotes(ctx, t, sim.Config, css[0], - tmproto.PrecommitType, rs.ProposalBlock.Hash(), - rs.ProposalBlockParts.Header(), newVss[i]) + signAddVotes(ctx, t, css[0], + tmproto.PrecommitType, sim.Config.ChainID(), + types.BlockID{Hash: rs.ProposalBlock.Hash(), PartSetHeader: rs.ProposalBlockParts.Header()}, + newVss[i]) } ensureNewRound(t, newRoundCh, height+1, 0) @@ -555,8 +557,7 @@ func setupSimulator(ctx context.Context, t *testing.T) *simulatorTestSuite { selfIndex = valIndexFn(0) require.NotEqual(t, -1, selfIndex) - - proposal = types.NewProposal(vss[1].Height, round, -1, blockID) + proposal = types.NewProposal(vss[1].Height, round, -1, blockID, propBlock.Header.Time) p = proposal.ToProto() if err := vss[1].SignProposal(ctx, cfg.ChainID(), p); err != nil { t.Fatal("failed to sign bad proposal", err) @@ -573,9 +574,10 @@ func setupSimulator(ctx context.Context, t *testing.T) *simulatorTestSuite { if i == selfIndex { continue } - signAddVotes(ctx, t, sim.Config, css[0], - tmproto.PrecommitType, rs.ProposalBlock.Hash(), - rs.ProposalBlockParts.Header(), newVss[i]) + signAddVotes(ctx, t, css[0], + tmproto.PrecommitType, sim.Config.ChainID(), + types.BlockID{Hash: rs.ProposalBlock.Hash(), PartSetHeader: rs.ProposalBlockParts.Header()}, + newVss[i]) } ensureNewRound(t, newRoundCh, height+1, 0) @@ -823,14 +825,15 @@ func testHandshakeReplay( } // now start the app using the handshake - it should sync - genDoc, _ := sm.MakeGenesisDocFromFile(cfg.GenesisFile()) + genDoc, err := sm.MakeGenesisDocFromFile(cfg.GenesisFile()) + require.NoError(t, err) handshaker := NewHandshaker(logger, stateStore, state, store, eventbus.NopEventBus{}, genDoc) proxyApp := proxy.NewAppConns(clientCreator2, logger, proxy.NopMetrics()) require.NoError(t, proxyApp.Start(ctx), "Error starting proxy app connections") t.Cleanup(func() { cancel(); proxyApp.Wait() }) - err := handshaker.Handshake(ctx, proxyApp) + err = handshaker.Handshake(ctx, proxyApp) if expectError { require.Error(t, err) return @@ -1011,7 +1014,8 @@ func TestHandshakePanicsIfAppReturnsWrongAppHash(t *testing.T) { require.NoError(t, err) stateDB, state, store := stateAndStore(t, cfg, pubKey, appVersion) stateStore := sm.NewStore(stateDB) - genDoc, _ := sm.MakeGenesisDocFromFile(cfg.GenesisFile()) + genDoc, err := sm.MakeGenesisDocFromFile(cfg.GenesisFile()) + require.NoError(t, err) state.LastValidators = state.Validators.Copy() // mode = 0 for committing all the blocks blocks := sf.MakeBlocks(ctx, t, 3, &state, privVal) @@ -1277,7 +1281,8 @@ func TestHandshakeUpdatesValidators(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() - val, _, err := factory.RandValidator(ctx, true, 10) + votePower := 10 + int64(rand.Uint32()) + val, _, err := factory.Validator(ctx, votePower) require.NoError(t, err) vals := types.NewValidatorSet([]*types.Validator{val}) app := &initChainApp{vals: types.TM2PB.ValidatorUpdates(vals)} diff --git a/internal/consensus/state.go b/internal/consensus/state.go index f06a658e1..f4b4edf8e 100644 --- a/internal/consensus/state.go +++ b/internal/consensus/state.go @@ -47,15 +47,17 @@ var msgQueueSize = 1000 // msgs from the reactor which may update the state type msgInfo struct { - Msg Message - PeerID types.NodeID + Msg Message + PeerID types.NodeID + ReceiveTime time.Time } func (msgInfo) TypeTag() string { return "tendermint/wal/MsgInfo" } type msgInfoJSON struct { - Msg json.RawMessage `json:"msg"` - PeerID types.NodeID `json:"peer_key"` + Msg json.RawMessage `json:"msg"` + PeerID types.NodeID `json:"peer_key"` + ReceiveTime time.Time `json:"receive_time"` } func (m msgInfo) MarshalJSON() ([]byte, error) { @@ -63,7 +65,7 @@ func (m msgInfo) MarshalJSON() ([]byte, error) { if err != nil { return nil, err } - return json.Marshal(msgInfoJSON{Msg: msg, PeerID: m.PeerID}) + return json.Marshal(msgInfoJSON{Msg: msg, PeerID: m.PeerID, ReceiveTime: m.ReceiveTime}) } func (m *msgInfo) UnmarshalJSON(data []byte) error { @@ -163,7 +165,7 @@ type State struct { // some functions can be overwritten for testing decideProposal func(ctx context.Context, height int64, round int32) doPrevote func(ctx context.Context, height int64, round int32) - setProposal func(proposal *types.Proposal) error + setProposal func(proposal *types.Proposal, t time.Time) error // closed when we finish shutting down done chan struct{} @@ -546,14 +548,14 @@ func (cs *State) AddVote(ctx context.Context, vote *types.Vote, peerID types.Nod select { case <-ctx.Done(): return ctx.Err() - case cs.internalMsgQueue <- msgInfo{&VoteMessage{vote}, ""}: + case cs.internalMsgQueue <- msgInfo{&VoteMessage{vote}, "", tmtime.Now()}: return nil } } else { select { case <-ctx.Done(): return ctx.Err() - case cs.peerMsgQueue <- msgInfo{&VoteMessage{vote}, peerID}: + case cs.peerMsgQueue <- msgInfo{&VoteMessage{vote}, peerID, tmtime.Now()}: return nil } } @@ -568,14 +570,14 @@ func (cs *State) SetProposal(ctx context.Context, proposal *types.Proposal, peer select { case <-ctx.Done(): return ctx.Err() - case cs.internalMsgQueue <- msgInfo{&ProposalMessage{proposal}, ""}: + case cs.internalMsgQueue <- msgInfo{&ProposalMessage{proposal}, "", tmtime.Now()}: return nil } } else { select { case <-ctx.Done(): return ctx.Err() - case cs.peerMsgQueue <- msgInfo{&ProposalMessage{proposal}, peerID}: + case cs.peerMsgQueue <- msgInfo{&ProposalMessage{proposal}, peerID, tmtime.Now()}: return nil } } @@ -589,14 +591,14 @@ func (cs *State) AddProposalBlockPart(ctx context.Context, height int64, round i select { case <-ctx.Done(): return ctx.Err() - case cs.internalMsgQueue <- msgInfo{&BlockPartMessage{height, round, part}, ""}: + case cs.internalMsgQueue <- msgInfo{&BlockPartMessage{height, round, part}, "", tmtime.Now()}: return nil } } else { select { case <-ctx.Done(): return ctx.Err() - case cs.peerMsgQueue <- msgInfo{&BlockPartMessage{height, round, part}, peerID}: + case cs.peerMsgQueue <- msgInfo{&BlockPartMessage{height, round, part}, peerID, tmtime.Now()}: return nil } } @@ -785,6 +787,7 @@ func (cs *State) updateToState(ctx context.Context, state sm.State) { cs.Validators = validators cs.Proposal = nil + cs.ProposalReceiveTime = time.Time{} cs.ProposalBlock = nil cs.ProposalBlockParts = nil cs.LockedRound = -1 @@ -933,7 +936,7 @@ func (cs *State) handleMsg(ctx context.Context, mi msgInfo) { case *ProposalMessage: // will not cause transition. // once proposal is set, we can receive block parts - err = cs.setProposal(msg.Proposal) + err = cs.setProposal(msg.Proposal, mi.ReceiveTime) case *BlockPartMessage: // if the proposal is complete, we'll enterPrevote or tryFinalizeCommit @@ -1125,6 +1128,7 @@ func (cs *State) enterNewRound(ctx context.Context, height int64, round int32) { } else { logger.Debug("resetting proposal info") cs.Proposal = nil + cs.ProposalReceiveTime = time.Time{} cs.ProposalBlock = nil cs.ProposalBlockParts = nil } @@ -1147,9 +1151,10 @@ func (cs *State) enterNewRound(ctx context.Context, height int64, round int32) { cs.scheduleTimeout(cs.config.CreateEmptyBlocksInterval, height, round, cstypes.RoundStepNewRound) } - } else { - cs.enterPropose(ctx, height, round) + return } + + cs.enterPropose(ctx, height, round) } // needProofBlock returns true on the first height (so the genesis app hash is signed right away) @@ -1182,6 +1187,16 @@ func (cs *State) enterPropose(ctx context.Context, height int64, round int32) { return } + // If this validator is the proposer of this round, and the previous block time is later than + // our local clock time, wait to propose until our local clock time has passed the block time. + if cs.privValidatorPubKey != nil && cs.isProposer(cs.privValidatorPubKey.Address()) { + proposerWaitTime := proposerWaitTime(tmtime.DefaultSource{}, cs.state.LastBlockTime) + if proposerWaitTime > 0 { + cs.scheduleTimeout(proposerWaitTime, height, round, cstypes.RoundStepNewRound) + return + } + } + logger.Debug("entering propose step", "current", fmt.Sprintf("%v/%v/%v", cs.Height, cs.Round, cs.Step)) defer func() { @@ -1202,12 +1217,10 @@ func (cs *State) enterPropose(ctx context.Context, height int64, round int32) { // Nothing more to do if we're not a validator if cs.privValidator == nil { - logger.Debug("node is not a validator") + logger.Debug("propose step; not proposing since node is not a validator") return } - logger.Debug("node is a validator") - if cs.privValidatorPubKey == nil { // If this node is a validator & proposer in the current round, it will // miss the opportunity to create a block. @@ -1215,18 +1228,20 @@ func (cs *State) enterPropose(ctx context.Context, height int64, round int32) { return } - address := cs.privValidatorPubKey.Address() + addr := cs.privValidatorPubKey.Address() // if not a validator, we're done - if !cs.Validators.HasAddress(address) { - logger.Debug("node is not a validator", "addr", address, "vals", cs.Validators) + if !cs.Validators.HasAddress(addr) { + logger.Debug("propose step; not proposing since node is not in the validator set", + "addr", addr, + "vals", cs.Validators) return } - if cs.isProposer(address) { + if cs.isProposer(addr) { logger.Debug( "propose step; our turn to propose", - "proposer", address, + "proposer", addr, ) cs.decideProposal(ctx, height, round) @@ -1267,7 +1282,7 @@ func (cs *State) defaultDecideProposal(ctx context.Context, height int64, round // Make proposal propBlockID := types.BlockID{Hash: block.Hash(), PartSetHeader: blockParts.Header()} - proposal := types.NewProposal(height, round, cs.ValidRound, propBlockID) + proposal := types.NewProposal(height, round, cs.ValidRound, propBlockID, block.Header.Time) p := proposal.ToProto() // wait the max amount we would wait for a proposal @@ -1277,11 +1292,11 @@ func (cs *State) defaultDecideProposal(ctx context.Context, height int64, round proposal.Signature = p.Signature // send proposal and block parts on internal msg queue - cs.sendInternalMessage(ctx, msgInfo{&ProposalMessage{proposal}, ""}) + cs.sendInternalMessage(ctx, msgInfo{&ProposalMessage{proposal}, "", tmtime.Now()}) for i := 0; i < int(blockParts.Total()); i++ { part := blockParts.GetPart(i) - cs.sendInternalMessage(ctx, msgInfo{&BlockPartMessage{cs.Height, cs.Round, part}, ""}) + cs.sendInternalMessage(ctx, msgInfo{&BlockPartMessage{cs.Height, cs.Round, part}, "", tmtime.Now()}) } cs.logger.Debug("signed proposal", "height", height, "round", round, "proposal", proposal) @@ -1348,8 +1363,11 @@ func (cs *State) createProposalBlock() (block *types.Block, blockParts *types.Pa // Enter: `timeoutPropose` after entering Propose. // Enter: proposal block and POL is ready. -// Prevote for LockedBlock if we're locked, or ProposalBlock if valid. -// Otherwise vote nil. +// If we received a valid proposal within this round and we are not locked on a block, +// we will prevote for block. +// Otherwise, if we receive a valid proposal that matches the block we are +// locked on or matches a block that received a POL in a round later than our +// locked round, prevote for the proposal, otherwise vote nil. func (cs *State) enterPrevote(ctx context.Context, height int64, round int32) { logger := cs.logger.With("height", height, "round", round) @@ -1376,19 +1394,47 @@ func (cs *State) enterPrevote(ctx context.Context, height int64, round int32) { // (so we have more time to try and collect +2/3 prevotes for a single block) } +func (cs *State) proposalIsTimely() bool { + sp := types.SynchronyParams{ + Precision: cs.state.ConsensusParams.Synchrony.Precision, + MessageDelay: cs.state.ConsensusParams.Synchrony.MessageDelay, + } + + return cs.Proposal.IsTimely(cs.ProposalReceiveTime, sp, cs.state.InitialHeight) +} + func (cs *State) defaultDoPrevote(ctx context.Context, height int64, round int32) { logger := cs.logger.With("height", height, "round", round) - // If a block is locked, prevote that. - if cs.LockedBlock != nil { - logger.Debug("prevote step; already locked on a block; prevoting locked block") - cs.signAddVote(ctx, tmproto.PrevoteType, cs.LockedBlock.Hash(), cs.LockedBlockParts.Header()) + // Check that a proposed block was not received within this round (and thus executing this from a timeout). + if cs.ProposalBlock == nil { + logger.Debug("prevote step: ProposalBlock is nil; prevoting nil") + cs.signAddVote(ctx, tmproto.PrevoteType, nil, types.PartSetHeader{}) return } - // If ProposalBlock is nil, prevote nil. - if cs.ProposalBlock == nil { - logger.Debug("prevote step: ProposalBlock is nil") + if cs.Proposal == nil { + logger.Debug("prevote step: did not receive proposal; prevoting nil") + cs.signAddVote(ctx, tmproto.PrevoteType, nil, types.PartSetHeader{}) + return + } + + if !cs.Proposal.Timestamp.Equal(cs.ProposalBlock.Header.Time) { + logger.Debug("prevote step: proposal timestamp not equal; prevoting nil") + cs.signAddVote(ctx, tmproto.PrevoteType, nil, types.PartSetHeader{}) + return + } + + if cs.Proposal.POLRound == -1 && cs.LockedRound == -1 && !cs.proposalIsTimely() { + logger.Debug("prevote step: Proposal is not timely; prevoting nil", + "proposed", + tmtime.Canonical(cs.Proposal.Timestamp).Format(time.RFC3339Nano), + "received", + tmtime.Canonical(cs.ProposalReceiveTime).Format(time.RFC3339Nano), + "msg_delay", + cs.state.ConsensusParams.Synchrony.MessageDelay, + "precision", + cs.state.ConsensusParams.Synchrony.Precision) cs.signAddVote(ctx, tmproto.PrevoteType, nil, types.PartSetHeader{}) return } @@ -1397,16 +1443,72 @@ func (cs *State) defaultDoPrevote(ctx context.Context, height int64, round int32 err := cs.blockExec.ValidateBlock(cs.state, cs.ProposalBlock) if err != nil { // ProposalBlock is invalid, prevote nil. - logger.Error("prevote step: ProposalBlock is invalid", "err", err) + logger.Error("prevote step: ProposalBlock is invalid; prevoting nil", "err", err) cs.signAddVote(ctx, tmproto.PrevoteType, nil, types.PartSetHeader{}) return } - // Prevote cs.ProposalBlock - // NOTE: the proposal signature is validated when it is received, - // and the proposal block parts are validated as they are received (against the merkle hash in the proposal) - logger.Debug("prevote step: ProposalBlock is valid") - cs.signAddVote(ctx, tmproto.PrevoteType, cs.ProposalBlock.Hash(), cs.ProposalBlockParts.Header()) + /* + 22: upon from proposer(h_p, round_p) while step_p = propose do + 23: if valid(v) && (lockedRound_p = −1 || lockedValue_p = v) then + 24: broadcast + + Here, cs.Proposal.POLRound corresponds to the -1 in the above algorithm rule. + This means that the proposer is producing a new proposal that has not previously + seen a 2/3 majority by the network. + + If we have already locked on a different value that is different from the proposed value, + we prevote nil since we are locked on a different value. Otherwise, if we're not locked on a block + or the proposal matches our locked block, we prevote the proposal. + */ + if cs.Proposal.POLRound == -1 { + if cs.LockedRound == -1 { + logger.Debug("prevote step: ProposalBlock is valid and there is no locked block; prevoting the proposal") + cs.signAddVote(ctx, tmproto.PrevoteType, cs.ProposalBlock.Hash(), cs.ProposalBlockParts.Header()) + return + } + if cs.ProposalBlock.HashesTo(cs.LockedBlock.Hash()) { + logger.Debug("prevote step: ProposalBlock is valid and matches our locked block; prevoting the proposal") + cs.signAddVote(ctx, tmproto.PrevoteType, cs.ProposalBlock.Hash(), cs.ProposalBlockParts.Header()) + return + } + } + + /* + 28: upon from proposer(h_p, round_p) AND 2f + 1 while + step_p = propose && (v_r ≥ 0 && v_r < round_p) do + 29: if valid(v) && (lockedRound_p ≤ v_r || lockedValue_p = v) then + 30: broadcast + + This rule is a bit confusing but breaks down as follows: + + If we see a proposal in the current round for value 'v' that lists its valid round as 'v_r' + AND this validator saw a 2/3 majority of the voting power prevote 'v' in round 'v_r', then we will + issue a prevote for 'v' in this round if 'v' is valid and either matches our locked value OR + 'v_r' is a round greater than or equal to our current locked round. + + 'v_r' can be a round greater than to our current locked round if a 2/3 majority of + the network prevoted a value in round 'v_r' but we did not lock on it, possibly because we + missed the proposal in round 'v_r'. + */ + blockID, ok := cs.Votes.Prevotes(cs.Proposal.POLRound).TwoThirdsMajority() + if ok && cs.ProposalBlock.HashesTo(blockID.Hash) && cs.Proposal.POLRound >= 0 && cs.Proposal.POLRound < cs.Round { + if cs.LockedRound <= cs.Proposal.POLRound { + logger.Debug("prevote step: ProposalBlock is valid and received a 2/3" + + "majority in a round later than the locked round; prevoting the proposal") + cs.signAddVote(ctx, tmproto.PrevoteType, cs.ProposalBlock.Hash(), cs.ProposalBlockParts.Header()) + return + } + if cs.ProposalBlock.HashesTo(cs.LockedBlock.Hash()) { + logger.Debug("prevote step: ProposalBlock is valid and matches our locked block; prevoting the proposal") + cs.signAddVote(ctx, tmproto.PrevoteType, cs.ProposalBlock.Hash(), cs.ProposalBlockParts.Header()) + return + } + } + + logger.Debug("prevote step: ProposalBlock is valid but was not our locked block or" + + "did not receive a more recent majority; prevoting nil") + cs.signAddVote(ctx, tmproto.PrevoteType, nil, types.PartSetHeader{}) } // Enter: any +2/3 prevotes at next round. @@ -1444,7 +1546,6 @@ func (cs *State) enterPrevoteWait(ctx context.Context, 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(ctx context.Context, height int64, round int32) { logger := cs.logger.With("height", height, "round", round) @@ -1491,47 +1592,50 @@ func (cs *State) enterPrecommit(ctx context.Context, 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(ctx, 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; precommitting nil") + cs.signAddVote(ctx, tmproto.PrecommitType, nil, types.PartSetHeader{}) + return + } + // At this point, +2/3 prevoted for a particular block. + // If we never received a proposal for this block, we must precommit nil + if cs.Proposal == nil || cs.ProposalBlock == nil { + logger.Debug("precommit step; did not receive proposal, precommitting nil") cs.signAddVote(ctx, tmproto.PrecommitType, nil, types.PartSetHeader{}) return } - // At this point, +2/3 prevoted for a particular block. + // If the proposal time does not match the block time, precommit nil. + if !cs.Proposal.Timestamp.Equal(cs.ProposalBlock.Header.Time) { + logger.Debug("precommit step: proposal timestamp not equal; precommitting nil") + cs.signAddVote(ctx, tmproto.PrecommitType, nil, types.PartSetHeader{}) + return + } // If we're already locked on that block, precommit it, and update the LockedRound if cs.LockedBlock.HashesTo(blockID.Hash) { - logger.Debug("precommit step; +2/3 prevoted locked block; relocking") + logger.Debug("precommit step: +2/3 prevoted locked block; relocking") cs.LockedRound = round if err := cs.eventBus.PublishEventRelock(ctx, cs.RoundStateEvent()); err != nil { - logger.Error("failed publishing event relock", "err", err) + logger.Error("precommit step: failed publishing event relock", "err", err) } cs.signAddVote(ctx, tmproto.PrecommitType, blockID.Hash, blockID.PartSetHeader) 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) + logger.Debug("precommit step: +2/3 prevoted proposal block; locking", "hash", blockID.Hash) // Validate the block. if err := cs.blockExec.ValidateBlock(cs.state, cs.ProposalBlock); err != nil { - panic(fmt.Errorf("precommit step; +2/3 prevoted for an invalid block: %w", err)) + panic(fmt.Sprintf("precommit step: +2/3 prevoted for an invalid block %v; relocking", err)) } cs.LockedRound = round @@ -1539,7 +1643,7 @@ func (cs *State) enterPrecommit(ctx context.Context, height int64, round int32) cs.LockedBlockParts = cs.ProposalBlockParts if err := cs.eventBus.PublishEventLock(ctx, cs.RoundStateEvent()); err != nil { - logger.Error("failed publishing event lock", "err", err) + logger.Error("precommit step: failed publishing event lock", "err", err) } cs.signAddVote(ctx, tmproto.PrecommitType, blockID.Hash, blockID.PartSetHeader) @@ -1547,23 +1651,14 @@ func (cs *State) enterPrecommit(ctx context.Context, 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. - 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 + // 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) if !cs.ProposalBlockParts.HasHeader(blockID.PartSetHeader) { cs.ProposalBlock = nil cs.ProposalBlockParts = types.NewPartSetFromHeader(blockID.PartSetHeader) } - if err := cs.eventBus.PublishEventUnlock(ctx, cs.RoundStateEvent()); err != nil { - logger.Error("failed publishing event unlock", "err", err) - } - cs.signAddVote(ctx, tmproto.PrecommitType, nil, types.PartSetHeader{}) } @@ -1671,7 +1766,7 @@ func (cs *State) tryFinalizeCommit(ctx context.Context, 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 } @@ -1893,10 +1988,10 @@ func (cs *State) RecordMetrics(height int64, block *types.Block) { //----------------------------------------------------------------------------- -func (cs *State) defaultSetProposal(proposal *types.Proposal) error { +func (cs *State) defaultSetProposal(proposal *types.Proposal, recvTime time.Time) error { // Already have one // TODO: possibly catch double proposals - if cs.Proposal != nil { + if cs.Proposal != nil || proposal == nil { return nil } @@ -1921,6 +2016,7 @@ func (cs *State) defaultSetProposal(proposal *types.Proposal) error { proposal.Signature = p.Signature cs.Proposal = proposal + cs.ProposalReceiveTime = recvTime // We don't update cs.ProposalBlockParts if it is already set. // This happens if we're already in cstypes.RoundStepCommit or if there is a valid block in the current round. // TODO: We can check if Proposal is for a different block as this is a sign of misbehavior! @@ -2000,7 +2096,7 @@ func (cs *State) addProposalBlockPart( // 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", @@ -2153,33 +2249,13 @@ func (cs *State) addVote( 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(ctx, 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 @@ -2215,7 +2291,7 @@ func (cs *State) addVote( 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(ctx, height, vote.Round) } else if prevotes.HasTwoThirdsAny() { cs.enterPrevoteWait(ctx, height, vote.Round) @@ -2243,7 +2319,7 @@ func (cs *State) addVote( cs.enterNewRound(ctx, height, vote.Round) cs.enterPrecommit(ctx, height, vote.Round) - if len(blockID.Hash) != 0 { + if !blockID.IsNil() { cs.enterCommit(ctx, height, vote.Round) if cs.config.SkipTimeoutCommit && precommits.HasAll() { cs.enterNewRound(ctx, cs.Height, 0) @@ -2288,7 +2364,7 @@ func (cs *State) signVote( ValidatorIndex: valIdx, Height: cs.Height, Round: cs.Round, - Timestamp: cs.voteTime(), + Timestamp: tmtime.Now(), Type: msgType, BlockID: types.BlockID{Hash: hash, PartSetHeader: header}, } @@ -2318,30 +2394,6 @@ func (cs *State) signVote( return vote, err } -// voteTime ensures monotonicity of the time a validator votes on. -// It ensures that for a prior block with a BFT-timestamp of T, -// any vote from this validator will have time at least time T + 1ms. -// This is needed, as monotonicity of time is a guarantee that BFT time provides. -func (cs *State) voteTime() time.Time { - now := tmtime.Now() - minVoteTime := now - // Minimum time increment between blocks - const timeIota = time.Millisecond - // TODO: We should remove next line in case we don't vote for v in case cs.ProposalBlock == nil, - // even if cs.LockedBlock != nil. See https://docs.tendermint.com/master/spec/. - if cs.LockedBlock != nil { - // See the BFT time spec https://docs.tendermint.com/master/spec/consensus/bft-time.html - minVoteTime = cs.LockedBlock.Time.Add(timeIota) - } else if cs.ProposalBlock != nil { - minVoteTime = cs.ProposalBlock.Time.Add(timeIota) - } - - if now.After(minVoteTime) { - return now - } - return minVoteTime -} - // sign the vote and publish on internalMsgQueue func (cs *State) signAddVote(ctx context.Context, msgType tmproto.SignedMsgType, hash []byte, header types.PartSetHeader) *types.Vote { if cs.privValidator == nil { // the node does not have a key @@ -2362,7 +2414,7 @@ func (cs *State) signAddVote(ctx context.Context, msgType tmproto.SignedMsgType, // TODO: pass pubKey to signVote vote, err := cs.signVote(ctx, msgType, hash, header) if err == nil { - cs.sendInternalMessage(ctx, msgInfo{&VoteMessage{vote}, ""}) + cs.sendInternalMessage(ctx, msgInfo{&VoteMessage{vote}, "", tmtime.Now()}) cs.logger.Debug("signed and pushed vote", "height", cs.Height, "round", cs.Round, "vote", vote) return vote } @@ -2509,3 +2561,17 @@ func repairWalFile(src, dst string) error { return nil } + +// proposerWaitTime determines how long the proposer should wait to propose its next block. +// If the result is zero, a block can be proposed immediately. +// +// Block times must be monotonically increasing, so if the block time of the previous +// block is larger than the proposer's current time, then the proposer will sleep +// until its local clock exceeds the previous block time. +func proposerWaitTime(lt tmtime.Source, bt time.Time) time.Duration { + t := lt.Now() + if bt.After(t) { + return bt.Sub(t) + } + return 0 +} diff --git a/internal/consensus/state_test.go b/internal/consensus/state_test.go index c1b7eaf9c..10adbccaa 100644 --- a/internal/consensus/state_test.go +++ b/internal/consensus/state_test.go @@ -18,6 +18,7 @@ import ( tmquery "github.com/tendermint/tendermint/internal/pubsub/query" "github.com/tendermint/tendermint/libs/log" tmrand "github.com/tendermint/tendermint/libs/rand" + tmtime "github.com/tendermint/tendermint/libs/time" tmproto "github.com/tendermint/tendermint/proto/tendermint/types" "github.com/tendermint/tendermint/types" ) @@ -36,16 +37,23 @@ 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 * TestStateLock_NoPOL - 2 vals, 4 rounds. one val locked, precommits nil every round except first. +x * TestStateLock_POLUpdateLock - 4 vals, one precommits, +other 3 polka at next round, so we unlock and precomit the polka +x * TestStateLock_POLRelock - 4 vals, polka in round 1 and polka in round 2. +Ensure validator updates locked round. +x_*_TestStateLock_POLDoesNotUnlock 4 vals, one precommits, other 3 polka nil at +next round, so we precommit nil but maintain lock +x * TestStateLock_MissingProposalWhenPOLSeenDoesNotUpdateLock - 4 vals, 1 misses proposal but sees POL. +x * TestStateLock_MissingProposalWhenPOLSeenDoesNotUnlock - 4 vals, 1 misses proposal but sees POL. +x * TestStateLock_POLSafety1 - 4 vals. We shouldn't change lock based on polka at earlier round +x * TestStateLock_POLSafety2 - 4 vals. After unlocking, we shouldn't relock based on polka at earlier round +x_*_TestState_PrevotePOLFromPreviousRound 4 vals, prevote a proposal if a POL was seen for it in a previous round. * TestNetworkLock - once +1/3 precommits, network should be locked * TestNetworkLockPOL - once +1/3 precommits, the block with more recent polka is committed SlashingSuite -x * TestSlashingPrevotes - a validator prevoting twice in a round gets slashed -x * TestSlashingPrecommits - a validator precomitting twice in a round gets slashed +x * TestStateSlashing_Prevotes - a validator prevoting twice in a round gets slashed +x * TestStateSlashing_Precommits - a validator precomitting twice in a round gets slashed CatchupSuite * TestCatchup - if we might be behind and we've seen any 2/3 prevotes, round skip to new round, precommit, or prevote HaltSuite @@ -60,9 +68,9 @@ func TestStateProposerSelection0(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() config := configSetup(t) + logger := log.NewNopLogger() - cs1, vss := randState(ctx, t, config, log.TestingLogger(), 4) - + cs1, vss := makeState(ctx, t, config, logger, 4) height, round := cs1.Height, cs1.Round newRoundCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryNewRound) @@ -78,24 +86,16 @@ func TestStateProposerSelection0(t *testing.T) { pv, err := cs1.privValidator.GetPubKey(ctx) require.NoError(t, err) address := pv.Address() - if !bytes.Equal(prop.Address, address) { - t.Fatalf("expected proposer to be validator %d. Got %X", 0, prop.Address) - } + require.Truef(t, bytes.Equal(prop.Address, address), "expected proposer to be validator %d. Got %X", 0, prop.Address) // Wait for complete proposal. ensureNewProposal(t, proposalCh, height, round) rs := cs1.GetRoundState() - signAddVotes( - ctx, - t, - config, - cs1, - tmproto.PrecommitType, - rs.ProposalBlock.Hash(), - rs.ProposalBlockParts.Header(), - vss[1:]..., - ) + signAddVotes(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), types.BlockID{ + Hash: rs.ProposalBlock.Hash(), + PartSetHeader: rs.ProposalBlockParts.Header(), + }, vss[1:]...) // Wait for new round so next validator is set. ensureNewRound(t, newRoundCh, height+1, 0) @@ -110,12 +110,11 @@ func TestStateProposerSelection0(t *testing.T) { // Now let's do it all again, but starting from round 2 instead of 0 func TestStateProposerSelection2(t *testing.T) { config := configSetup(t) + logger := log.NewNopLogger() ctx, cancel := context.WithCancel(context.Background()) defer cancel() - - cs1, vss := randState(ctx, t, config, log.TestingLogger(), 4) // test needs more work for more than 3 validators - + cs1, vss := makeState(ctx, t, config, logger, 4) // test needs more work for more than 3 validators height := cs1.Height newRoundCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryNewRound) @@ -140,8 +139,7 @@ func TestStateProposerSelection2(t *testing.T) { int(i+2)%len(vss), prop.Address) - rs := cs1.GetRoundState() - signAddVotes(ctx, t, config, cs1, tmproto.PrecommitType, nil, rs.ProposalBlockParts.Header(), vss[1:]...) + signAddVotes(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), types.BlockID{}, vss[1:]...) ensureNewRound(t, newRoundCh, height, i+round+1) // wait for the new round event each round incrementRound(vss[1:]...) } @@ -151,11 +149,11 @@ func TestStateProposerSelection2(t *testing.T) { // a non-validator should timeout into the prevote round func TestStateEnterProposeNoPrivValidator(t *testing.T) { config := configSetup(t) + logger := log.NewNopLogger() ctx, cancel := context.WithCancel(context.Background()) defer cancel() - cs, _ := randState(ctx, t, config, log.TestingLogger(), 1) - + cs, _ := makeState(ctx, t, config, logger, 1) cs.SetPrivValidator(ctx, nil) height, round := cs.Height, cs.Round @@ -175,10 +173,11 @@ func TestStateEnterProposeNoPrivValidator(t *testing.T) { // a validator should not timeout of the prevote round (TODO: unless the block is really big!) func TestStateEnterProposeYesPrivValidator(t *testing.T) { config := configSetup(t) + logger := log.NewNopLogger() ctx, cancel := context.WithCancel(context.Background()) defer cancel() - cs, _ := randState(ctx, t, config, log.TestingLogger(), 1) + cs, _ := makeState(ctx, t, config, logger, 1) height, round := cs.Height, cs.Round // Listen for propose timeout event @@ -209,10 +208,11 @@ func TestStateEnterProposeYesPrivValidator(t *testing.T) { func TestStateBadProposal(t *testing.T) { config := configSetup(t) + logger := log.NewNopLogger() ctx, cancel := context.WithCancel(context.Background()) defer cancel() - cs1, vss := randState(ctx, t, config, log.TestingLogger(), 2) + cs1, vss := makeState(ctx, t, config, logger, 2) height, round := cs1.Height, cs1.Round vs2 := vss[1] @@ -238,18 +238,16 @@ func TestStateBadProposal(t *testing.T) { propBlockParts, err := propBlock.MakePartSet(partSize) require.NoError(t, err) blockID := types.BlockID{Hash: propBlock.Hash(), PartSetHeader: propBlockParts.Header()} - proposal := types.NewProposal(vs2.Height, round, -1, blockID) + proposal := types.NewProposal(vs2.Height, round, -1, blockID, propBlock.Header.Time) p := proposal.ToProto() - if err := vs2.SignProposal(ctx, config.ChainID(), p); err != nil { - t.Fatal("failed to sign bad proposal", err) - } + err = vs2.SignProposal(ctx, config.ChainID(), p) + require.NoError(t, err) proposal.Signature = p.Signature // set the proposal block - if err := cs1.SetProposalAndBlock(ctx, proposal, propBlock, propBlockParts, "some peer"); err != nil { - t.Fatal(err) - } + err = cs1.SetProposalAndBlock(ctx, proposal, propBlock, propBlockParts, "some peer") + require.NoError(t, err) // start the machine startTestRound(ctx, cs1, height, round) @@ -261,27 +259,22 @@ func TestStateBadProposal(t *testing.T) { ensurePrevoteMatch(t, voteCh, height, round, nil) // add bad prevote from vs2 and wait for it - bps, err := propBlock.MakePartSet(partSize) - require.NoError(t, err) - - signAddVotes(ctx, t, config, cs1, tmproto.PrevoteType, propBlock.Hash(), bps.Header(), vs2) + signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), blockID, vs2) ensurePrevote(t, voteCh, height, round) // wait for precommit ensurePrecommit(t, voteCh, height, round) validatePrecommit(ctx, t, cs1, round, -1, vss[0], nil, nil) - - bps2, err := propBlock.MakePartSet(partSize) - require.NoError(t, err) - signAddVotes(ctx, t, config, cs1, tmproto.PrecommitType, propBlock.Hash(), bps2.Header(), vs2) + signAddVotes(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), blockID, vs2) } func TestStateOversizedBlock(t *testing.T) { config := configSetup(t) + logger := log.NewNopLogger() ctx, cancel := context.WithCancel(context.Background()) defer cancel() - cs1, vss := randState(ctx, t, config, log.TestingLogger(), 2) + cs1, vss := makeState(ctx, t, config, logger, 2) cs1.state.ConsensusParams.Block.MaxBytes = 2000 height, round := cs1.Height, cs1.Round vs2 := vss[1] @@ -303,11 +296,10 @@ func TestStateOversizedBlock(t *testing.T) { propBlockParts, err := propBlock.MakePartSet(partSize) require.NoError(t, err) blockID := types.BlockID{Hash: propBlock.Hash(), PartSetHeader: propBlockParts.Header()} - proposal := types.NewProposal(height, round, -1, blockID) + proposal := types.NewProposal(height, round, -1, blockID, propBlock.Header.Time) p := proposal.ToProto() - if err := vs2.SignProposal(ctx, config.ChainID(), p); err != nil { - t.Fatal("failed to sign bad proposal", err) - } + err = vs2.SignProposal(ctx, config.ChainID(), p) + require.NoError(t, err) proposal.Signature = p.Signature totalBytes := 0 @@ -316,9 +308,8 @@ func TestStateOversizedBlock(t *testing.T) { totalBytes += len(part.Bytes) } - if err := cs1.SetProposalAndBlock(ctx, proposal, propBlock, propBlockParts, "some peer"); err != nil { - t.Fatal(err) - } + err = cs1.SetProposalAndBlock(ctx, proposal, propBlock, propBlockParts, "some peer") + require.NoError(t, err) // start the machine startTestRound(ctx, cs1, height, round) @@ -332,18 +323,11 @@ func TestStateOversizedBlock(t *testing.T) { // and then should send nil prevote and precommit regardless of whether other validators prevote and // precommit on it ensurePrevoteMatch(t, voteCh, height, round, nil) - - bps, err := propBlock.MakePartSet(partSize) - require.NoError(t, err) - - signAddVotes(ctx, t, config, cs1, tmproto.PrevoteType, propBlock.Hash(), bps.Header(), vs2) + signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), blockID, vs2) ensurePrevote(t, voteCh, height, round) ensurePrecommit(t, voteCh, height, round) validatePrecommit(ctx, t, cs1, round, -1, vss[0], nil, nil) - - bps2, err := propBlock.MakePartSet(partSize) - require.NoError(t, err) - signAddVotes(ctx, t, config, cs1, tmproto.PrecommitType, propBlock.Hash(), bps2.Header(), vs2) + signAddVotes(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), blockID, vs2) } //---------------------------------------------------------------------------------------------------- @@ -352,11 +336,11 @@ func TestStateOversizedBlock(t *testing.T) { // propose, prevote, and precommit a block func TestStateFullRound1(t *testing.T) { config := configSetup(t) - logger := log.TestingLogger() + logger := log.NewNopLogger() ctx, cancel := context.WithCancel(context.Background()) defer cancel() - cs, vss := randState(ctx, t, config, logger, 1) + cs, vss := makeState(ctx, t, config, logger, 1) height, round := cs.Height, cs.Round pv, err := cs.privValidator.GetPubKey(ctx) @@ -386,10 +370,11 @@ func TestStateFullRound1(t *testing.T) { // nil is proposed, so prevote and precommit nil func TestStateFullRoundNil(t *testing.T) { config := configSetup(t) + logger := log.NewNopLogger() ctx, cancel := context.WithCancel(context.Background()) defer cancel() - cs, _ := randState(ctx, t, config, log.TestingLogger(), 1) + cs, _ := makeState(ctx, t, config, logger, 1) height, round := cs.Height, cs.Round voteCh := subscribe(ctx, t, cs.eventBus, types.EventQueryVote) @@ -405,10 +390,11 @@ func TestStateFullRoundNil(t *testing.T) { // where the first validator has to wait for votes from the second func TestStateFullRound2(t *testing.T) { config := configSetup(t) + logger := log.NewNopLogger() ctx, cancel := context.WithCancel(context.Background()) defer cancel() - cs1, vss := randState(ctx, t, config, log.TestingLogger(), 2) + cs1, vss := makeState(ctx, t, config, logger, 2) vs2 := vss[1] height, round := cs1.Height, cs1.Round @@ -422,20 +408,20 @@ func TestStateFullRound2(t *testing.T) { // we should be stuck in limbo waiting for more prevotes rs := cs1.GetRoundState() - propBlockHash, propPartSetHeader := rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header() + blockID := types.BlockID{Hash: rs.ProposalBlock.Hash(), PartSetHeader: rs.ProposalBlockParts.Header()} // prevote arrives from vs2: - signAddVotes(ctx, t, config, cs1, tmproto.PrevoteType, propBlockHash, propPartSetHeader, vs2) + signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), blockID, vs2) ensurePrevote(t, voteCh, height, round) // prevote ensurePrecommit(t, voteCh, height, round) // precommit // the proposed block should now be locked and our precommit added - validatePrecommit(ctx, t, cs1, 0, 0, vss[0], propBlockHash, propBlockHash) + validatePrecommit(ctx, t, cs1, 0, 0, vss[0], blockID.Hash, blockID.Hash) // we should be stuck in limbo waiting for more precommits // precommit arrives from vs2: - signAddVotes(ctx, t, config, cs1, tmproto.PrecommitType, propBlockHash, propPartSetHeader, vs2) + signAddVotes(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), blockID, vs2) ensurePrecommit(t, voteCh, height, round) // wait to finish commit, propose in next height @@ -447,12 +433,13 @@ func TestStateFullRound2(t *testing.T) { // two validators, 4 rounds. // two vals take turns proposing. val1 locks on first one, precommits nil on everything else -func TestStateLockNoPOL(t *testing.T) { +func TestStateLock_NoPOL(t *testing.T) { config := configSetup(t) + logger := log.NewNopLogger() ctx, cancel := context.WithCancel(context.Background()) defer cancel() - cs1, vss := randState(ctx, t, config, log.TestingLogger(), 2) + cs1, vss := makeState(ctx, t, config, logger, 2) vs2 := vss[1] height, round := cs1.Height, cs1.Round @@ -476,26 +463,32 @@ func TestStateLockNoPOL(t *testing.T) { ensureNewProposal(t, proposalCh, height, round) roundState := cs1.GetRoundState() - theBlockHash := roundState.ProposalBlock.Hash() - thePartSetHeader := roundState.ProposalBlockParts.Header() + initialBlockID := types.BlockID{ + Hash: roundState.ProposalBlock.Hash(), + PartSetHeader: roundState.ProposalBlockParts.Header(), + } ensurePrevote(t, voteCh, height, round) // prevote // we should now be stuck in limbo forever, waiting for more prevotes // prevote arrives from vs2: - signAddVotes(ctx, t, config, cs1, tmproto.PrevoteType, theBlockHash, thePartSetHeader, vs2) + signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), initialBlockID, vs2) ensurePrevote(t, voteCh, height, round) // prevote + validatePrevote(ctx, t, cs1, round, vss[0], initialBlockID.Hash) - ensurePrecommit(t, voteCh, height, round) // precommit // the proposed block should now be locked and our precommit added - validatePrecommit(ctx, t, cs1, round, round, vss[0], theBlockHash, theBlockHash) + ensurePrecommit(t, voteCh, height, round) + validatePrecommit(ctx, t, cs1, round, round, vss[0], initialBlockID.Hash, initialBlockID.Hash) // we should now be stuck in limbo forever, waiting for more precommits // lets add one for a different block - hash := make([]byte, len(theBlockHash)) - copy(hash, theBlockHash) + hash := make([]byte, len(initialBlockID.Hash)) + copy(hash, initialBlockID.Hash) hash[0] = (hash[0] + 1) % 255 - signAddVotes(ctx, t, config, cs1, tmproto.PrecommitType, hash, thePartSetHeader, vs2) + signAddVotes(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), types.BlockID{ + Hash: hash, + PartSetHeader: initialBlockID.PartSetHeader, + }, vs2) ensurePrecommit(t, voteCh, height, round) // precommit // (note we're entering precommit for a second time this round) @@ -520,31 +513,27 @@ func TestStateLockNoPOL(t *testing.T) { require.Nil(t, rs.ProposalBlock, "Expected proposal block to be nil") - // wait to finish prevote + // we should have prevoted nil since we did not see a proposal in the round. ensurePrevote(t, voteCh, height, round) - // we should have prevoted our locked block - validatePrevote(ctx, t, cs1, round, vss[0], rs.LockedBlock.Hash()) + validatePrevote(ctx, t, cs1, round, vss[0], nil) // add a conflicting prevote from the other validator - bps, err := rs.LockedBlock.MakePartSet(partSize) + partSet, err := rs.LockedBlock.MakePartSet(partSize) require.NoError(t, err) - - signAddVotes(ctx, t, config, cs1, tmproto.PrevoteType, hash, bps.Header(), vs2) + conflictingBlockID := types.BlockID{Hash: hash, PartSetHeader: partSet.Header()} + signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), conflictingBlockID, vs2) ensurePrevote(t, voteCh, height, round) // now we're going to enter prevote again, but with invalid args // and then prevote wait, which should timeout. then wait for precommit ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.config.Prevote(round).Nanoseconds()) - - ensurePrecommit(t, voteCh, height, round) // precommit - // the proposed block should still be locked and our precommit added - // we should precommit nil and be locked on the proposal - validatePrecommit(ctx, t, cs1, round, 0, vss[0], nil, theBlockHash) + // the proposed block should still be locked block. + // we should precommit nil and be locked on the proposal. + ensurePrecommit(t, voteCh, height, round) + validatePrecommit(ctx, t, cs1, round, 0, vss[0], nil, initialBlockID.Hash) // add conflicting precommit from vs2 - bps2, err := rs.LockedBlock.MakePartSet(partSize) - require.NoError(t, err) - signAddVotes(ctx, t, config, cs1, tmproto.PrecommitType, hash, bps2.Header(), vs2) + signAddVotes(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), conflictingBlockID, vs2) ensurePrecommit(t, voteCh, height, round) // (note we're entering precommit for a second time this round, but with invalid args @@ -571,39 +560,37 @@ func TestStateLockNoPOL(t *testing.T) { ensurePrevote(t, voteCh, height, round) // prevote validatePrevote(ctx, t, cs1, round, vss[0], rs.LockedBlock.Hash()) - - bps0, err := rs.ProposalBlock.MakePartSet(partSize) + partSet, err = rs.ProposalBlock.MakePartSet(partSize) require.NoError(t, err) - signAddVotes(ctx, t, config, cs1, tmproto.PrevoteType, hash, bps0.Header(), vs2) + newBlockID := types.BlockID{Hash: hash, PartSetHeader: partSet.Header()} + signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), newBlockID, vs2) ensurePrevote(t, voteCh, height, round) ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.config.Prevote(round).Nanoseconds()) ensurePrecommit(t, voteCh, height, round) // precommit - validatePrecommit(ctx, t, cs1, round, 0, vss[0], nil, theBlockHash) // precommit nil but be locked on proposal + validatePrecommit(ctx, t, cs1, round, 0, vss[0], nil, initialBlockID.Hash) // precommit nil but be locked on proposal - bps1, err := rs.ProposalBlock.MakePartSet(partSize) - require.NoError(t, err) signAddVotes( ctx, t, - config, cs1, tmproto.PrecommitType, - hash, - bps1.Header(), + config.ChainID(), + newBlockID, vs2) // NOTE: conflicting precommits at same height ensurePrecommit(t, voteCh, height, round) ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.config.Precommit(round).Nanoseconds()) - // needed so generated block is different than locked block - cs2, _ := randState(ctx, t, config, log.TestingLogger(), 2) - require.NoError(t, err) + cs2, _ := makeState(ctx, t, config, logger, 2) // needed so generated block is different than locked block // before we time out into new round, set next proposal block 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") + require.NotNil(t, propBlock, "Failed to create proposal block with vs2") + require.NotNil(t, prop, "Failed to create proposal block with vs2") + propBlockID := types.BlockID{ + Hash: propBlock.Hash(), + PartSetHeader: partSet.Header(), } incrementRound(vs2) @@ -619,52 +606,47 @@ func TestStateLockNoPOL(t *testing.T) { // so set the proposal block bps3, err := propBlock.MakePartSet(partSize) require.NoError(t, err) - if err := cs1.SetProposalAndBlock(ctx, prop, propBlock, bps3, ""); err != nil { - t.Fatal(err) - } + err = cs1.SetProposalAndBlock(ctx, prop, propBlock, bps3, "") + require.NoError(t, err) ensureNewProposal(t, proposalCh, height, round) - ensurePrevote(t, voteCh, height, round) // prevote - // prevote for locked block (not proposal) - validatePrevote(ctx, t, cs1, 3, vss[0], cs1.LockedBlock.Hash()) - // prevote for proposed block - bps4, err := propBlock.MakePartSet(partSize) - require.NoError(t, err) + // prevote for nil since we did not see a proposal for our locked block in the round. + ensurePrevote(t, voteCh, height, round) + validatePrevote(ctx, t, cs1, 3, vss[0], nil) - signAddVotes(ctx, t, config, cs1, tmproto.PrevoteType, propBlock.Hash(), bps4.Header(), vs2) + // prevote for proposed block + signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), propBlockID, vs2) ensurePrevote(t, voteCh, height, round) ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.config.Prevote(round).Nanoseconds()) ensurePrecommit(t, voteCh, height, round) - validatePrecommit(ctx, t, cs1, round, 0, vss[0], nil, theBlockHash) // precommit nil but locked on proposal + validatePrecommit(ctx, t, cs1, round, 0, vss[0], nil, initialBlockID.Hash) // precommit nil but locked on proposal - bps5, err := propBlock.MakePartSet(partSize) - require.NoError(t, err) signAddVotes( ctx, t, - config, cs1, tmproto.PrecommitType, - propBlock.Hash(), - bps5.Header(), + config.ChainID(), + propBlockID, vs2) // NOTE: conflicting precommits at same height ensurePrecommit(t, 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) { +// TestStateLock_POLUpdateLock tests that a validator 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 TestStateLock_POLUpdateLock(t *testing.T) { config := configSetup(t) - logger := log.TestingLogger() + logger := log.NewNopLogger() ctx, cancel := context.WithCancel(context.Background()) defer cancel() - cs1, vss := randState(ctx, t, config, logger, 4) + cs1, vss := makeState(ctx, t, config, logger, 4) vs2, vs3, vs4 := vss[1], vss[2], vss[3] height, round := cs1.Height, cs1.Round @@ -676,16 +658,18 @@ func TestStateLockPOLRelock(t *testing.T) { require.NoError(t, err) addr := pv1.Address() voteCh := subscribeToVoter(ctx, t, cs1, addr) + lockCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryLock) newRoundCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryNewRound) - newBlockCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryNewBlockHeader) - - // everything done from perspective of cs1 /* - Round1 (cs1, B) // B B B B// B nil B 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. - eg. vs2 and vs4 didn't see the 2/3 prevotes + 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(ctx, cs1, height, round) @@ -693,101 +677,398 @@ func TestStateLockPOLRelock(t *testing.T) { ensureNewRound(t, newRoundCh, height, round) ensureNewProposal(t, proposalCh, height, round) rs := cs1.GetRoundState() - theBlockHash := rs.ProposalBlock.Hash() - theBlockParts := rs.ProposalBlockParts.Header() + initialBlockID := types.BlockID{ + Hash: rs.ProposalBlock.Hash(), + PartSetHeader: rs.ProposalBlockParts.Header(), + } - ensurePrevote(t, voteCh, height, round) // prevote + ensurePrevote(t, voteCh, height, round) - signAddVotes(ctx, t, config, cs1, tmproto.PrevoteType, theBlockHash, theBlockParts, vs2, vs3, vs4) + signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), initialBlockID, vs2, vs3, vs4) - ensurePrecommit(t, voteCh, height, round) // our precommit - // the proposed block should now be locked and our precommit added - validatePrecommit(ctx, t, cs1, round, round, vss[0], theBlockHash, theBlockHash) + // check that the validator generates a Lock event. + ensureLock(t, lockCh, height, round) - // add precommits from the rest - signAddVotes(ctx, t, config, cs1, tmproto.PrecommitType, nil, types.PartSetHeader{}, vs2, vs3, vs4) + // the proposed block should now be locked and our precommit added. + ensurePrecommit(t, voteCh, height, round) + validatePrecommit(ctx, t, cs1, round, round, vss[0], initialBlockID.Hash, initialBlockID.Hash) - // before we timeout to the new round set the new proposal - cs2 := newState(ctx, t, logger, cs1.state, vs2, kvstore.NewApplication()) + // add precommits from the rest of the validators. + signAddVotes(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), types.BlockID{}, vs2, vs3, vs4) - 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") + // timeout to new round. + ensureNewTimeout(t, 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(ctx, t, logger, cs1.state, vs2, kvstore.NewApplication()) + require.NoError(t, err) + propR1, propBlockR1 := decideProposal(ctx, t, cs2, vs2, vs2.Height, vs2.Round) + propBlockR1Parts, err := propBlockR1.MakePartSet(partSize) + require.NoError(t, err) + propBlockR1Hash := propBlockR1.Hash() + r1BlockID := types.BlockID{ + Hash: propBlockR1Hash, + PartSetHeader: propBlockR1Parts.Header(), } - propBlockParts, err := propBlock.MakePartSet(partSize) + require.NotEqual(t, propBlockR1Hash, initialBlockID.Hash) + err = cs1.SetProposalAndBlock(ctx, propR1, propBlockR1, propBlockR1Parts, "some peer") require.NoError(t, err) - propBlockHash := propBlock.Hash() - require.NotEqual(t, propBlockHash, theBlockHash) + ensureNewRound(t, newRoundCh, height, round) - incrementRound(vs2, vs3, vs4) + // ensure that the validator receives the proposal. + ensureNewProposal(t, proposalCh, height, round) - // timeout to new round + // Prevote our nil since the proposal does not match our locked block. + ensurePrevoteMatch(t, voteCh, height, round, nil) + + // Add prevotes from the remainder of the validators for the new locked block. + signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), r1BlockID, vs2, vs3, vs4) + + // Check that we lock on a new block. + ensureLock(t, lockCh, height, round) + + ensurePrecommit(t, 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(ctx, t, cs1, round, round, vss[0], propBlockR1Hash, propBlockR1Hash) +} + +// TestStateLock_POLRelock 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 TestStateLock_POLRelock(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + config := configSetup(t) + logger := log.NewNopLogger() + + cs1, vss := makeState(ctx, t, config, logger, 4) + vs2, vs3, vs4 := vss[1], vss[2], vss[3] + height, round := cs1.Height, cs1.Round + + timeoutWaitCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryTimeoutWait) + proposalCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryCompleteProposal) + pv1, err := cs1.privValidator.GetPubKey(ctx) + require.NoError(t, err) + addr := pv1.Address() + voteCh := subscribeToVoter(ctx, t, cs1, addr) + lockCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryLock) + relockCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryRelock) + newRoundCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryNewRound) + + /* + 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") + + startTestRound(ctx, cs1, height, round) + + ensureNewRound(t, newRoundCh, height, round) + ensureNewProposal(t, proposalCh, height, round) + rs := cs1.GetRoundState() + theBlock := rs.ProposalBlock + theBlockParts := rs.ProposalBlockParts + blockID := types.BlockID{ + Hash: rs.ProposalBlock.Hash(), + PartSetHeader: rs.ProposalBlockParts.Header(), + } + + ensurePrevote(t, voteCh, height, round) + + signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), blockID, vs2, vs3, vs4) + + // check that the validator generates a Lock event. + ensureLock(t, lockCh, height, round) + + // the proposed block should now be locked and our precommit added. + ensurePrecommit(t, voteCh, height, round) + validatePrecommit(ctx, t, cs1, round, round, vss[0], blockID.Hash, blockID.Hash) + + // add precommits from the rest of the validators. + signAddVotes(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), types.BlockID{}, vs2, vs3, vs4) + + // timeout to new round. ensureNewTimeout(t, 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(ctx, prop, propBlock, propBlockParts, "some peer"); err != nil { - t.Fatal(err) + /* + 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++ + propR1 := types.NewProposal(height, round, cs1.ValidRound, blockID, theBlock.Header.Time) + p := propR1.ToProto() + err = vs2.SignProposal(ctx, cs1.state.ChainID, p) + require.NoError(t, err) + propR1.Signature = p.Signature + err = cs1.SetProposalAndBlock(ctx, propR1, theBlock, theBlockParts, "") + require.NoError(t, err) + + ensureNewRound(t, newRoundCh, height, round) + + // ensure that the validator receives the proposal. + ensureNewProposal(t, proposalCh, height, round) + + // Prevote our locked block since it matches the propsal seen in this round. + ensurePrevote(t, voteCh, height, round) + validatePrevote(ctx, t, cs1, round, vss[0], blockID.Hash) + + // Add prevotes from the remainder of the validators for the locked block. + signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), blockID, vs2, vs3, vs4) + + // Check that we relock. + ensureRelock(t, relockCh, height, round) + + ensurePrecommit(t, voteCh, height, round) + + // We should now be locked on the same block but with an updated locked round. + validatePrecommit(ctx, t, cs1, round, round, vss[0], blockID.Hash, blockID.Hash) +} + +// TestStateLock_PrevoteNilWhenLockedAndMissProposal tests that a validator prevotes nil +// if it is locked on a block and misses the proposal in a round. +func TestStateLock_PrevoteNilWhenLockedAndMissProposal(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + config := configSetup(t) + logger := log.NewNopLogger() + + cs1, vss := makeState(ctx, t, config, logger, 4) + vs2, vs3, vs4 := vss[1], vss[2], vss[3] + height, round := cs1.Height, cs1.Round + + timeoutWaitCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryTimeoutWait) + proposalCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryCompleteProposal) + pv1, err := cs1.privValidator.GetPubKey(context.Background()) + require.NoError(t, err) + addr := pv1.Address() + voteCh := subscribeToVoter(ctx, t, cs1, addr) + lockCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryLock) + newRoundCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryNewRound) + + /* + 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") + + startTestRound(ctx, cs1, height, round) + + ensureNewRound(t, newRoundCh, height, round) + ensureNewProposal(t, proposalCh, height, round) + rs := cs1.GetRoundState() + blockID := types.BlockID{ + Hash: rs.ProposalBlock.Hash(), + PartSetHeader: rs.ProposalBlockParts.Header(), } + ensurePrevote(t, voteCh, height, round) + + signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), blockID, vs2, vs3, vs4) + + // check that the validator generates a Lock event. + ensureLock(t, lockCh, height, round) + + // the proposed block should now be locked and our precommit added. + ensurePrecommit(t, voteCh, height, round) + validatePrecommit(ctx, t, cs1, round, round, vss[0], blockID.Hash, blockID.Hash) + + // add precommits from the rest of the validators. + signAddVotes(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), types.BlockID{}, vs2, vs3, vs4) + + // timeout to new round. + ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.config.Precommit(round).Nanoseconds()) + + /* + Round 1: + Send a prevote for nil from each of the validators to cs1. + Send a precommit for nil from all of the validtors to cs1. + + Check that cs1 prevotes nil instead of its locked block, but ensure + that it maintains its locked block. + */ + t.Log("### Starting Round 1") + incrementRound(vs2, vs3, vs4) + round++ + ensureNewRound(t, newRoundCh, height, round) - t.Log("### ONTO ROUND 1") + + // Prevote our nil. + ensurePrevote(t, voteCh, height, round) + validatePrevote(ctx, t, cs1, round, vss[0], nil) + + // Add prevotes from the remainder of the validators nil. + signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), types.BlockID{}, vs2, vs3, vs4) + ensurePrecommit(t, voteCh, height, round) + // We should now be locked on the same block but with an updated locked round. + validatePrecommit(ctx, t, cs1, round, 0, vss[0], nil, blockID.Hash) +} + +// TestStateLock_PrevoteNilWhenLockedAndMissProposal tests that a validator prevotes nil +// if it is locked on a block and misses the proposal in a round. +func TestStateLock_PrevoteNilWhenLockedAndDifferentProposal(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + logger := log.NewNopLogger() + config := configSetup(t) + /* + 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 := makeState(ctx, t, config, logger, 4) + vs2, vs3, vs4 := vss[1], vss[2], vss[3] + height, round := cs1.Height, cs1.Round + + timeoutWaitCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryTimeoutWait) + proposalCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryCompleteProposal) + pv1, err := cs1.privValidator.GetPubKey(context.Background()) + require.NoError(t, err) + addr := pv1.Address() + voteCh := subscribeToVoter(ctx, t, cs1, addr) + lockCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryLock) + newRoundCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryNewRound) /* - Round2 (vs2, C) // B C C C // C C C _) + 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. - cs1 changes lock! + This ensures that cs1 will lock on B in this round but not precommit it. */ + t.Log("### Starting Round 0") + startTestRound(ctx, cs1, height, round) - // now we're on a new round and not the proposer - // but we should receive the proposal + ensureNewRound(t, newRoundCh, height, round) ensureNewProposal(t, proposalCh, height, round) + rs := cs1.GetRoundState() + blockID := types.BlockID{ + Hash: rs.ProposalBlock.Hash(), + PartSetHeader: rs.ProposalBlockParts.Header(), + } - // go to prevote, node should prevote for locked block (not the new proposal) - this is relocking - ensurePrevoteMatch(t, voteCh, height, round, theBlockHash) + ensurePrevote(t, voteCh, height, round) - // now lets add prevotes from everyone else for the new block - signAddVotes(ctx, t, config, cs1, tmproto.PrevoteType, propBlockHash, propBlockParts.Header(), vs2, vs3, vs4) + signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), blockID, vs2, vs3, vs4) + + // check that the validator generates a Lock event. + ensureLock(t, lockCh, height, round) + // the proposed block should now be locked and our precommit added. ensurePrecommit(t, voteCh, height, round) - // we should have unlocked and locked on the new block, sending a precommit for this new block - validatePrecommit(ctx, t, cs1, round, round, vss[0], propBlockHash, propBlockHash) + validatePrecommit(ctx, t, cs1, round, round, vss[0], blockID.Hash, blockID.Hash) - // more prevote creating a majority on the new block and this is then committed - signAddVotes(ctx, t, config, cs1, tmproto.PrecommitType, propBlockHash, propBlockParts.Header(), vs2, vs3) - ensureNewBlockHeader(t, newBlockCh, height, propBlockHash) + // add precommits from the rest of the validators. + signAddVotes(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), types.BlockID{}, vs2, vs3, vs4) - ensureNewRound(t, newRoundCh, height+1, 0) + // timeout to new round. + ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.config.Precommit(round).Nanoseconds()) + + /* + Round 1: + Create a proposal for a new block. + Send a prevote for nil from each of the validators to cs1. + Send a precommit for nil from all of the validtors to cs1. + + Check that cs1 prevotes nil instead of its locked block, but ensure + that it maintains its locked block. + */ + t.Log("### Starting Round 1") + incrementRound(vs2, vs3, vs4) + round++ + cs2 := newState(ctx, t, logger, cs1.state, vs2, kvstore.NewApplication()) + propR1, propBlockR1 := decideProposal(ctx, t, cs2, vs2, vs2.Height, vs2.Round) + propBlockR1Parts, err := propBlockR1.MakePartSet(types.BlockPartSizeBytes) + require.NoError(t, err) + propBlockR1Hash := propBlockR1.Hash() + require.NotEqual(t, propBlockR1Hash, blockID.Hash) + err = cs1.SetProposalAndBlock(ctx, propR1, propBlockR1, propBlockR1Parts, "some peer") + require.NoError(t, err) + + ensureNewRound(t, newRoundCh, height, round) + ensureNewProposal(t, proposalCh, height, round) + + // Prevote our nil. + ensurePrevote(t, voteCh, height, round) + validatePrevote(ctx, t, cs1, round, vss[0], nil) + + // Add prevotes from the remainder of the validators for nil. + signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), types.BlockID{}, vs2, vs3, vs4) + + // We should now be locked on the same block but prevote nil. + ensurePrecommit(t, voteCh, height, round) + validatePrecommit(ctx, t, cs1, round, 0, vss[0], nil, blockID.Hash) } -// 4 vals, one precommits, other 3 polka at next round, so we unlock and precomit the polka -func TestStateLockPOLUnlock(t *testing.T) { +// TestStateLock_POLDoesNotUnlock 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 TestStateLock_POLDoesNotUnlock(t *testing.T) { config := configSetup(t) + logger := log.NewNopLogger() 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(ctx, t, config, log.TestingLogger(), 4) + cs1, vss := makeState(ctx, t, config, logger, 4) vs2, vs3, vs4 := vss[1], vss[2], vss[3] height, round := cs1.Height, cs1.Round - partSize := types.BlockPartSizeBytes - proposalCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryCompleteProposal) timeoutWaitCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryTimeoutWait) newRoundCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryNewRound) - unlockCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryUnlock) - pv1, err := cs1.privValidator.GetPubKey(ctx) + lockCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryLock) + pv1, err := cs1.privValidator.GetPubKey(context.Background()) require.NoError(t, err) addr := pv1.Address() voteCh := subscribeToVoter(ctx, t, 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(ctx, cs1, height, round) @@ -795,76 +1076,112 @@ func TestStateLockPOLUnlock(t *testing.T) { ensureNewProposal(t, proposalCh, height, round) rs := cs1.GetRoundState() - theBlockHash := rs.ProposalBlock.Hash() - theBlockParts := rs.ProposalBlockParts.Header() + blockID := types.BlockID{ + Hash: rs.ProposalBlock.Hash(), + PartSetHeader: rs.ProposalBlockParts.Header(), + } + + ensurePrevoteMatch(t, voteCh, height, round, blockID.Hash) - ensurePrevoteMatch(t, voteCh, height, round, theBlockHash) + signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), blockID, vs2, vs3, vs4) - signAddVotes(ctx, t, config, cs1, tmproto.PrevoteType, theBlockHash, theBlockParts, vs2, vs3, vs4) + // the validator should have locked a block in this round. + ensureLock(t, lockCh, height, round) ensurePrecommit(t, voteCh, height, round) - // the proposed block should now be locked and our precommit added - validatePrecommit(ctx, t, cs1, round, round, vss[0], theBlockHash, theBlockHash) + // the proposed block should now be locked and our should be for this locked block. - // add precommits from the rest - signAddVotes(ctx, t, config, cs1, tmproto.PrecommitType, nil, types.PartSetHeader{}, vs2, vs4) - signAddVotes(ctx, t, config, cs1, tmproto.PrecommitType, theBlockHash, theBlockParts, vs3) + validatePrecommit(ctx, t, cs1, round, round, vss[0], blockID.Hash, blockID.Hash) - // 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) + // 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(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), types.BlockID{}, vs2, vs4) + signAddVotes(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), blockID, vs3) // timeout to new round ensureNewTimeout(t, 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 + cs2 := newState(ctx, t, logger, cs1.state, vs2, kvstore.NewApplication()) + prop, propBlock := decideProposal(ctx, t, cs2, vs2, vs2.Height, vs2.Round) + propBlockParts, err := propBlock.MakePartSet(types.BlockPartSizeBytes) + require.NoError(t, err) + require.NotEqual(t, propBlock.Hash(), blockID.Hash) + err = cs1.SetProposalAndBlock(ctx, prop, propBlock, propBlockParts, "") + require.NoError(t, err) ensureNewRound(t, newRoundCh, height, round) - t.Log("#### ONTO ROUND 1") + + ensureNewProposal(t, proposalCh, height, round) + + // Prevote for nil since the proposed block does not match our locked block. + ensurePrevoteMatch(t, voteCh, height, round, nil) + + // add >2/3 prevotes for nil from all other validators + signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), types.BlockID{}, vs2, vs3, vs4) + + ensurePrecommit(t, voteCh, height, round) + + // verify that we haven't update our locked block since the first round + validatePrecommit(ctx, t, cs1, round, 0, vss[0], nil, blockID.Hash) + + signAddVotes(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), types.BlockID{}, vs2, vs3, vs4) + ensureNewTimeout(t, 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(ctx, prop, propBlock, propBlockParts, "some peer"); err != nil { - t.Fatal(err) - } + t.Log("#### ONTO ROUND 2") + round++ + incrementRound(vs2, vs3, vs4) + cs3 := newState(ctx, t, logger, cs1.state, vs2, kvstore.NewApplication()) + prop, propBlock = decideProposal(ctx, t, cs3, vs3, vs3.Height, vs3.Round) + propBlockParts, err = propBlock.MakePartSet(types.BlockPartSizeBytes) + require.NoError(t, err) + err = cs1.SetProposalAndBlock(ctx, prop, propBlock, propBlockParts, "") + require.NoError(t, err) + + ensureNewRound(t, newRoundCh, height, round) ensureNewProposal(t, proposalCh, height, round) - // go to prevote, prevote for locked block (not proposal) - ensurePrevoteMatch(t, voteCh, height, round, lockedBlockHash) - // now lets add prevotes from everyone else for nil (a polka!) - signAddVotes(ctx, t, config, cs1, tmproto.PrevoteType, nil, types.PartSetHeader{}, vs2, vs3, vs4) + // Prevote for nil since the proposal does not match our locked block. + ensurePrevote(t, voteCh, height, round) + validatePrevote(ctx, t, cs1, round, vss[0], nil) + + signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), types.BlockID{}, vs2, vs3, vs4) - // the polka makes us unlock and precommit nil - ensureNewUnlock(t, unlockCh, height, round) ensurePrecommit(t, voteCh, height, round) - // we should have unlocked and committed nil - // NOTE: since we don't relock on nil, the lock round is -1 - validatePrecommit(ctx, t, cs1, round, -1, vss[0], nil, nil) + // verify that we haven't update our locked block since the first round + validatePrecommit(ctx, t, cs1, round, 0, vss[0], nil, blockID.Hash) - signAddVotes(ctx, t, config, cs1, tmproto.PrecommitType, nil, types.PartSetHeader{}, vs2, vs3) - ensureNewRound(t, 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) { +// TestStateLock_MissingProposalWhenPOLSeenDoesNotUnlock 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 TestStateLock_MissingProposalWhenPOLSeenDoesNotUpdateLock(t *testing.T) { config := configSetup(t) - logger := log.TestingLogger() + logger := log.NewNopLogger() ctx, cancel := context.WithCancel(context.Background()) defer cancel() - cs1, vss := randState(ctx, t, config, logger, 4) - + cs1, vss := makeState(ctx, t, config, logger, 4) vs2, vs3, vs4 := vss[1], vss[2], vss[3] height, round := cs1.Height, cs1.Round @@ -877,127 +1194,163 @@ func TestStateLockPOLUnlockOnUnknownBlock(t *testing.T) { addr := pv1.Address() voteCh := subscribeToVoter(ctx, t, cs1, addr) newRoundCh := subscribe(ctx, t, 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(ctx, cs1, height, round) ensureNewRound(t, newRoundCh, height, round) ensureNewProposal(t, proposalCh, height, round) rs := cs1.GetRoundState() - firstBlockHash := rs.ProposalBlock.Hash() - firstBlockParts := rs.ProposalBlockParts.Header() + firstBlockID := types.BlockID{ + Hash: rs.ProposalBlock.Hash(), + PartSetHeader: rs.ProposalBlockParts.Header(), + } ensurePrevote(t, voteCh, height, round) // prevote - signAddVotes(ctx, t, config, cs1, tmproto.PrevoteType, firstBlockHash, firstBlockParts, vs2, vs3, vs4) + signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), firstBlockID, vs2, vs3, vs4) ensurePrecommit(t, voteCh, height, round) // our precommit // the proposed block should now be locked and our precommit added - validatePrecommit(ctx, t, cs1, round, round, vss[0], firstBlockHash, firstBlockHash) + validatePrecommit(ctx, t, cs1, round, round, vss[0], firstBlockID.Hash, firstBlockID.Hash) // add precommits from the rest - signAddVotes(ctx, t, config, cs1, tmproto.PrecommitType, nil, types.PartSetHeader{}, vs2, vs3, vs4) + signAddVotes(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), types.BlockID{}, vs2, vs3, vs4) + + // timeout to new round + ensureNewTimeout(t, 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. - // before we timeout to the new round set the new proposal + 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(ctx, t, logger, cs1.state, vs2, kvstore.NewApplication()) - 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") - } - secondBlockParts, err := propBlock.MakePartSet(partSize) require.NoError(t, err) + prop, propBlock := decideProposal(ctx, t, cs2, vs2, vs2.Height, vs2.Round) + require.NotNil(t, propBlock, "Failed to create proposal block with vs2") + require.NotNil(t, prop, "Failed to create proposal block with vs2") + partSet, err := propBlock.MakePartSet(partSize) + require.NoError(t, err) + secondBlockID := types.BlockID{ + Hash: propBlock.Hash(), + PartSetHeader: partSet.Header(), + } + require.NotEqual(t, secondBlockID.Hash, firstBlockID.Hash) - secondBlockHash := propBlock.Hash() - require.NotEqual(t, secondBlockHash, firstBlockHash) + ensureNewRound(t, newRoundCh, height, round) - incrementRound(vs2, vs3, vs4) + // prevote for nil since the proposal was not seen. + ensurePrevoteMatch(t, voteCh, height, round, nil) - // timeout to new round - ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.config.Precommit(round).Nanoseconds()) + // now lets add prevotes from everyone else for the new block + signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), secondBlockID, vs2, vs3, vs4) - round++ // moving to the next round + ensurePrecommit(t, voteCh, height, round) + validatePrecommit(ctx, t, cs1, round, 0, vss[0], nil, firstBlockID.Hash) +} - ensureNewRound(t, newRoundCh, height, round) - t.Log("### ONTO ROUND 1") +// TestStateLock_DoesNotLockOnOldProposal 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 TestStateLock_DoesNotLockOnOldProposal(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + config := configSetup(t) + logger := log.NewNopLogger() + cs1, vss := makeState(ctx, t, config, logger, 4) + vs2, vs3, vs4 := vss[1], vss[2], vss[3] + height, round := cs1.Height, cs1.Round + + timeoutWaitCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryTimeoutWait) + proposalCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryCompleteProposal) + pv1, err := cs1.privValidator.GetPubKey(context.Background()) + require.NoError(t, err) + addr := pv1.Address() + voteCh := subscribeToVoter(ctx, t, cs1, addr) + newRoundCh := subscribe(ctx, t, 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(ctx, cs1, height, round) - // now we're on a new round but v1 misses the proposal + ensureNewRound(t, newRoundCh, height, round) + ensureNewProposal(t, proposalCh, height, round) + rs := cs1.GetRoundState() + firstBlockID := types.BlockID{ + Hash: rs.ProposalBlock.Hash(), + PartSetHeader: rs.ProposalBlockParts.Header(), + } - // go to prevote, node should prevote for locked block (not the new proposal) - this is relocking - ensurePrevoteMatch(t, voteCh, height, round, firstBlockHash) + ensurePrevote(t, voteCh, height, round) - // now lets add prevotes from everyone else for the new block - signAddVotes(ctx, t, config, cs1, tmproto.PrevoteType, secondBlockHash, secondBlockParts.Header(), vs2, vs3, vs4) + signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), types.BlockID{}, vs2, vs3, vs4) + // The proposed block should not have been locked. ensurePrecommit(t, voteCh, height, round) - // we should have unlocked and locked on the new block, sending a precommit for this new block validatePrecommit(ctx, t, cs1, round, -1, vss[0], nil, nil) - if err := cs1.SetProposalAndBlock(ctx, 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(ctx, t, config, cs1, tmproto.PrecommitType, nil, types.PartSetHeader{}, vs2, vs3, vs4) - - // before we timeout to the new round set the new proposal - cs3 := newState(ctx, t, logger, cs1.state, vs3, kvstore.NewApplication()) - require.NoError(t, err) - 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) + signAddVotes(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), types.BlockID{}, vs2, vs3, vs4) incrementRound(vs2, vs3, vs4) // timeout to new round ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.config.Precommit(round).Nanoseconds()) - round++ // moving to the next round - ensureNewRound(t, 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. + 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. */ - - if err := cs1.SetProposalAndBlock(ctx, prop, propBlock, thirdPropBlockParts, "some peer"); err != nil { - t.Fatal(err) - } + t.Log("### Starting Round 1") + round++ + ensureNewRound(t, newRoundCh, height, round) ensurePrevote(t, voteCh, height, round) - // we are no longer locked to the first block so we should be able to prevote - validatePrevote(ctx, t, cs1, round, vss[0], thirdPropBlockHash) + validatePrevote(ctx, t, cs1, round, vss[0], nil) // All validators prevote for the old block. - signAddVotes(ctx, t, config, cs1, tmproto.PrevoteType, thirdPropBlockHash, thirdPropBlockParts.Header(), vs2, vs3, vs4) + // All validators prevote for the old block. + signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), firstBlockID, vs2, vs3, vs4) + // Make sure that cs1 did not lock on the block since it did not receive a proposal for it. ensurePrecommit(t, voteCh, height, round) - // we have a majority, now vs1 can change lock to the third block - validatePrecommit(ctx, t, cs1, round, round, vss[0], thirdPropBlockHash, thirdPropBlockHash) + validatePrecommit(ctx, t, cs1, round, -1, vss[0], nil, nil) } // 4 vals // a polka at round 1 but we miss it // then a polka at round 2 that we lock on // then we see the polka from round 1 but shouldn't unlock -func TestStateLockPOLSafety1(t *testing.T) { +func TestStateLock_POLSafety1(t *testing.T) { config := configSetup(t) + logger := log.NewNopLogger() ctx, cancel := context.WithCancel(context.Background()) defer cancel() - cs1, vss := randState(ctx, t, config, log.TestingLogger(), 4) + cs1, vss := makeState(ctx, t, config, logger, 4) vs2, vs3, vs4 := vss[1], vss[2], vss[3] height, round := cs1.Height, cs1.Round @@ -1021,40 +1374,38 @@ func TestStateLockPOLSafety1(t *testing.T) { propBlock := rs.ProposalBlock ensurePrevoteMatch(t, voteCh, height, round, propBlock.Hash()) - - // the others sign a polka but we don't see it - bps, err := propBlock.MakePartSet(partSize) + partSet, err := propBlock.MakePartSet(partSize) require.NoError(t, err) - - prevotes := signVotes(ctx, t, config, tmproto.PrevoteType, - propBlock.Hash(), bps.Header(), + blockID := types.BlockID{Hash: propBlock.Hash(), PartSetHeader: partSet.Header()} + // the others sign a polka but we don't see it + prevotes := signVotes(ctx, t, tmproto.PrevoteType, config.ChainID(), + blockID, vs2, vs3, vs4) - t.Logf("old prop hash %v", fmt.Sprintf("%X", propBlock.Hash())) - // we do see them precommit nil - signAddVotes(ctx, t, config, cs1, tmproto.PrecommitType, nil, types.PartSetHeader{}, vs2, vs3, vs4) + signAddVotes(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), types.BlockID{}, vs2, vs3, vs4) // cs1 precommit nil ensurePrecommit(t, voteCh, height, round) ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.config.Precommit(round).Nanoseconds()) t.Log("### ONTO ROUND 1") - - prop, propBlock := decideProposal(ctx, t, cs1, vs2, vs2.Height, vs2.Round+1) - propBlockHash := propBlock.Hash() + incrementRound(vs2, vs3, vs4) + round++ // moving to the next round + cs2 := newState(ctx, t, logger, cs1.state, vs2, kvstore.NewApplication()) + prop, propBlock := decideProposal(ctx, t, cs2, vs2, vs2.Height, vs2.Round) propBlockParts, err := propBlock.MakePartSet(partSize) require.NoError(t, err) + r2BlockID := types.BlockID{ + Hash: propBlock.Hash(), + PartSetHeader: propBlockParts.Header(), + } - incrementRound(vs2, vs3, vs4) - - round++ // moving to the next round ensureNewRound(t, newRoundCh, height, round) //XXX: this isnt guaranteed to get there before the timeoutPropose ... - if err := cs1.SetProposalAndBlock(ctx, prop, propBlock, propBlockParts, "some peer"); err != nil { - t.Fatal(err) - } + err = cs1.SetProposalAndBlock(ctx, prop, propBlock, propBlockParts, "some peer") + require.NoError(t, err) /*Round2 // we timeout and prevote our lock // a polka happened but we didn't see it! @@ -1066,19 +1417,19 @@ func TestStateLockPOLSafety1(t *testing.T) { require.Nil(t, rs.LockedBlock, "we should not be locked!") - t.Logf("new prop hash %v", fmt.Sprintf("%X", propBlockHash)) + t.Logf("new prop hash %v", fmt.Sprintf("%X", propBlock.Hash())) // go to prevote, prevote for proposal block - ensurePrevoteMatch(t, voteCh, height, round, propBlockHash) + ensurePrevoteMatch(t, voteCh, height, round, r2BlockID.Hash) // now we see the others prevote for it, so we should lock on it - signAddVotes(ctx, t, config, cs1, tmproto.PrevoteType, propBlockHash, propBlockParts.Header(), vs2, vs3, vs4) + signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), r2BlockID, vs2, vs3, vs4) ensurePrecommit(t, voteCh, height, round) // we should have precommitted - validatePrecommit(ctx, t, cs1, round, round, vss[0], propBlockHash, propBlockHash) + validatePrecommit(ctx, t, cs1, round, round, vss[0], r2BlockID.Hash, r2BlockID.Hash) - signAddVotes(ctx, t, config, cs1, tmproto.PrecommitType, nil, types.PartSetHeader{}, vs2, vs3, vs4) + signAddVotes(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), types.BlockID{}, vs2, vs3, vs4) ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.config.Precommit(round).Nanoseconds()) @@ -1096,8 +1447,7 @@ func TestStateLockPOLSafety1(t *testing.T) { ensureNewTimeout(t, timeoutProposeCh, height, round, cs1.config.Propose(round).Nanoseconds()) // finish prevote - // we should prevote what we're locked on - ensurePrevoteMatch(t, voteCh, height, round, propBlockHash) + ensurePrevoteMatch(t, voteCh, height, round, nil) newStepCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryNewRoundStep) @@ -1105,8 +1455,6 @@ func TestStateLockPOLSafety1(t *testing.T) { // add prevotes from the earlier round addVotes(cs1, prevotes...) - t.Log("Done adding prevotes!") - ensureNoNewRoundStep(t, newStepCh) } @@ -1117,12 +1465,13 @@ func TestStateLockPOLSafety1(t *testing.T) { // What we want: // dont see P0, lock on P1 at R1, dont unlock using P0 at R2 -func TestStateLockPOLSafety2(t *testing.T) { +func TestStateLock_POLSafety2(t *testing.T) { config := configSetup(t) + logger := log.NewNopLogger() ctx, cancel := context.WithCancel(context.Background()) defer cancel() - cs1, vss := randState(ctx, t, config, log.TestingLogger(), 4) + cs1, vss := makeState(ctx, t, config, logger, 4) vs2, vs3, vs4 := vss[1], vss[2], vss[3] height, round := cs1.Height, cs1.Round @@ -1131,7 +1480,6 @@ func TestStateLockPOLSafety2(t *testing.T) { proposalCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryCompleteProposal) timeoutWaitCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryTimeoutWait) newRoundCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryNewRound) - unlockCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryUnlock) pv1, err := cs1.privValidator.GetPubKey(ctx) require.NoError(t, err) addr := pv1.Address() @@ -1146,13 +1494,13 @@ func TestStateLockPOLSafety2(t *testing.T) { propBlockID0 := types.BlockID{Hash: propBlockHash0, PartSetHeader: propBlockParts0.Header()} // the others sign a polka but we don't see it - prevotes := signVotes(ctx, t, config, tmproto.PrevoteType, propBlockHash0, propBlockParts0.Header(), vs2, vs3, vs4) + prevotes := signVotes(ctx, t, tmproto.PrevoteType, config.ChainID(), propBlockID0, vs2, vs3, vs4) // the block for round 1 prop1, propBlock1 := decideProposal(ctx, t, cs1, vs2, vs2.Height, vs2.Round+1) - propBlockHash1 := propBlock1.Hash() propBlockParts1, err := propBlock1.MakePartSet(partSize) require.NoError(t, err) + propBlockID1 := types.BlockID{Hash: propBlock1.Hash(), PartSetHeader: propBlockParts1.Header()} incrementRound(vs2, vs3, vs4) @@ -1162,22 +1510,21 @@ func TestStateLockPOLSafety2(t *testing.T) { startTestRound(ctx, cs1, height, round) ensureNewRound(t, newRoundCh, height, round) - if err := cs1.SetProposalAndBlock(ctx, prop1, propBlock1, propBlockParts1, "some peer"); err != nil { - t.Fatal(err) - } + err = cs1.SetProposalAndBlock(ctx, prop1, propBlock1, propBlockParts1, "some peer") + require.NoError(t, err) ensureNewProposal(t, proposalCh, height, round) - ensurePrevoteMatch(t, voteCh, height, round, propBlockHash1) + ensurePrevoteMatch(t, voteCh, height, round, propBlockID1.Hash) - signAddVotes(ctx, t, config, cs1, tmproto.PrevoteType, propBlockHash1, propBlockParts1.Header(), vs2, vs3, vs4) + signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), propBlockID1, vs2, vs3, vs4) ensurePrecommit(t, voteCh, height, round) // the proposed block should now be locked and our precommit added - validatePrecommit(ctx, t, cs1, round, round, vss[0], propBlockHash1, propBlockHash1) + validatePrecommit(ctx, t, cs1, round, round, vss[0], propBlockID1.Hash, propBlockID1.Hash) // add precommits from the rest - signAddVotes(ctx, t, config, cs1, tmproto.PrecommitType, nil, types.PartSetHeader{}, vs2, vs4) - signAddVotes(ctx, t, config, cs1, tmproto.PrecommitType, propBlockHash1, propBlockParts1.Header(), vs3) + signAddVotes(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), types.BlockID{}, vs2, vs4) + signAddVotes(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), propBlockID1, vs3) incrementRound(vs2, vs3, vs4) @@ -1186,17 +1533,15 @@ func TestStateLockPOLSafety2(t *testing.T) { round++ // moving to the next round // in round 2 we see the polkad block from round 0 - newProp := types.NewProposal(height, round, 0, propBlockID0) + newProp := types.NewProposal(height, round, 0, propBlockID0, propBlock0.Header.Time) p := newProp.ToProto() - if err := vs3.SignProposal(ctx, config.ChainID(), p); err != nil { - t.Fatal(err) - } + err = vs3.SignProposal(ctx, config.ChainID(), p) + require.NoError(t, err) newProp.Signature = p.Signature - if err := cs1.SetProposalAndBlock(ctx, newProp, propBlock0, propBlockParts0, "some peer"); err != nil { - t.Fatal(err) - } + err = cs1.SetProposalAndBlock(ctx, newProp, propBlock0, propBlockParts0, "some peer") + require.NoError(t, err) // Add the pol votes addVotes(cs1, prevotes...) @@ -1208,22 +1553,164 @@ func TestStateLockPOLSafety2(t *testing.T) { */ ensureNewProposal(t, proposalCh, height, round) - ensureNoNewUnlock(t, unlockCh) - ensurePrevoteMatch(t, voteCh, height, round, propBlockHash1) + ensurePrevote(t, voteCh, height, round) + validatePrevote(ctx, t, cs1, round, vss[0], propBlockID1.Hash) + +} + +// TestState_PrevotePOLFromPreviousRound tests that a validator will prevote +// for a block if it is locked on a different block but saw a POL for the block +// it is not locked on in a previous round. +func TestState_PrevotePOLFromPreviousRound(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + config := configSetup(t) + logger := log.NewNopLogger() + + cs1, vss := makeState(ctx, t, config, logger, 4) + vs2, vs3, vs4 := vss[1], vss[2], vss[3] + height, round := cs1.Height, cs1.Round + + partSize := types.BlockPartSizeBytes + + timeoutWaitCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryTimeoutWait) + proposalCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryCompleteProposal) + pv1, err := cs1.privValidator.GetPubKey(context.Background()) + require.NoError(t, err) + addr := pv1.Address() + voteCh := subscribeToVoter(ctx, t, cs1, addr) + lockCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryLock) + newRoundCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryNewRound) + + /* + 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") + + startTestRound(ctx, cs1, height, round) + + ensureNewRound(t, newRoundCh, height, round) + ensureNewProposal(t, proposalCh, height, round) + rs := cs1.GetRoundState() + r0BlockID := types.BlockID{ + Hash: rs.ProposalBlock.Hash(), + PartSetHeader: rs.ProposalBlockParts.Header(), + } + + ensurePrevote(t, voteCh, height, round) + + signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), r0BlockID, vs2, vs3, vs4) + + // check that the validator generates a Lock event. + ensureLock(t, lockCh, height, round) + + // the proposed block should now be locked and our precommit added. + ensurePrecommit(t, voteCh, height, round) + validatePrecommit(ctx, t, cs1, round, round, vss[0], r0BlockID.Hash, r0BlockID.Hash) + + // add precommits from the rest of the validators. + signAddVotes(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), types.BlockID{}, vs2, vs3, vs4) + + // timeout to new round. + ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.config.Precommit(round).Nanoseconds()) + + /* + Round 1: + Create a block, D but do not send a proposal for it to cs1. + Send a prevote for D from each of the validators to cs1 so that cs1 sees a POL. + Send a precommit for nil from all of the validtors to cs1. + + cs1 has now seen greater than 2/3 of the voting power prevote D in this round + but cs1 did not see the proposal for D in this round so it will not prevote or precommit it. + */ + t.Log("### Starting Round 1") + incrementRound(vs2, vs3, vs4) + round++ + // Generate a new proposal block. + cs2 := newState(ctx, t, logger, cs1.state, vs2, kvstore.NewApplication()) + cs2.ValidRound = 1 + propR1, propBlockR1 := decideProposal(ctx, t, cs2, vs2, vs2.Height, round) + t.Log(propR1.POLRound) + propBlockR1Parts, err := propBlockR1.MakePartSet(partSize) + require.NoError(t, err) + r1BlockID := types.BlockID{ + Hash: propBlockR1.Hash(), + PartSetHeader: propBlockR1Parts.Header(), + } + require.NotEqual(t, r1BlockID.Hash, r0BlockID.Hash) + + ensureNewRound(t, newRoundCh, height, round) + + signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), r1BlockID, vs2, vs3, vs4) + + ensurePrevote(t, voteCh, height, round) + validatePrevote(ctx, t, cs1, round, vss[0], nil) + + signAddVotes(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), types.BlockID{}, vs2, vs3, vs4) + + ensurePrecommit(t, voteCh, height, round) + + // timeout to new round. + ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.config.Precommit(round).Nanoseconds()) + + /* + Create a new proposal for D, the same block from Round 1. + cs1 already saw greater than 2/3 of the voting power on the network vote for + D in a previous round, so it should prevote D once it receives a proposal for it. + + cs1 does not need to receive prevotes from other validators before the proposal + in this round. It will still prevote the block. + + Send cs1 prevotes for nil and check that it still prevotes its locked block + and not the block that it prevoted. + */ + t.Log("### Starting Round 2") + incrementRound(vs2, vs3, vs4) + round++ + propR2 := types.NewProposal(height, round, 1, r1BlockID, propBlockR1.Header.Time) + p := propR2.ToProto() + err = vs3.SignProposal(ctx, cs1.state.ChainID, p) + require.NoError(t, err) + propR2.Signature = p.Signature + + // cs1 receives a proposal for D, the block that received a POL in round 1. + err = cs1.SetProposalAndBlock(ctx, propR2, propBlockR1, propBlockR1Parts, "") + require.NoError(t, err) + + ensureNewRound(t, newRoundCh, height, round) + + ensureNewProposal(t, proposalCh, height, round) + + // We should now prevote this block, despite being locked on the block from + // round 0. + ensurePrevote(t, voteCh, height, round) + validatePrevote(ctx, t, cs1, round, vss[0], r1BlockID.Hash) + + signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), types.BlockID{}, vs2, vs3, vs4) + // cs1 did not receive a POL within this round, so it should remain locked + // on the block from round 0. + ensurePrecommit(t, voteCh, height, round) + validatePrecommit(ctx, t, cs1, round, 0, vss[0], nil, r0BlockID.Hash) } // 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. func TestProposeValidBlock(t *testing.T) { - config := configSetup(t) + cfg := configSetup(t) + logger := log.NewNopLogger() ctx, cancel := context.WithCancel(context.Background()) defer cancel() - cs1, vss := randState(ctx, t, config, log.TestingLogger(), 4) + cs1, vss := makeState(ctx, t, cfg, logger, 4) vs2, vs3, vs4 := vss[1], vss[2], vss[3] height, round := cs1.Height, cs1.Round @@ -1233,7 +1720,6 @@ func TestProposeValidBlock(t *testing.T) { timeoutWaitCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryTimeoutWait) timeoutProposeCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryTimeoutPropose) newRoundCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryNewRound) - unlockCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryUnlock) pv1, err := cs1.privValidator.GetPubKey(ctx) require.NoError(t, err) addr := pv1.Address() @@ -1246,22 +1732,24 @@ func TestProposeValidBlock(t *testing.T) { ensureNewProposal(t, proposalCh, height, round) rs := cs1.GetRoundState() propBlock := rs.ProposalBlock - propBlockHash := propBlock.Hash() + partSet, err := propBlock.MakePartSet(partSize) + require.NoError(t, err) + blockID := types.BlockID{ + Hash: propBlock.Hash(), + PartSetHeader: partSet.Header(), + } - ensurePrevoteMatch(t, voteCh, height, round, propBlockHash) + ensurePrevoteMatch(t, voteCh, height, round, blockID.Hash) // the others sign a polka - bps, err := propBlock.MakePartSet(partSize) - require.NoError(t, err) - signAddVotes(ctx, t, config, cs1, tmproto.PrevoteType, - propBlockHash, bps.Header(), vs2, - vs3, vs4) + signAddVotes(ctx, t, cs1, tmproto.PrevoteType, cfg.ChainID(), blockID, vs2, vs3, vs4) ensurePrecommit(t, voteCh, height, round) - // we should have precommitted - validatePrecommit(ctx, t, cs1, round, round, vss[0], propBlockHash, propBlockHash) + // we should have precommitted the proposed block in this round. + + validatePrecommit(ctx, t, cs1, round, round, vss[0], blockID.Hash, blockID.Hash) - signAddVotes(ctx, t, config, cs1, tmproto.PrecommitType, nil, types.PartSetHeader{}, vs2, vs3, vs4) + signAddVotes(ctx, t, cs1, tmproto.PrecommitType, cfg.ChainID(), types.BlockID{}, vs2, vs3, vs4) ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.config.Precommit(round).Nanoseconds()) @@ -1269,28 +1757,27 @@ func TestProposeValidBlock(t *testing.T) { round++ // moving to the next round ensureNewRound(t, newRoundCh, height, round) - - t.Log("### ONTO ROUND 2") + t.Log("### ONTO ROUND 1") // timeout of propose ensureNewTimeout(t, timeoutProposeCh, height, round, cs1.config.Propose(round).Nanoseconds()) - ensurePrevoteMatch(t, voteCh, height, round, propBlockHash) - - signAddVotes(ctx, t, config, cs1, tmproto.PrevoteType, nil, types.PartSetHeader{}, vs2, vs3, vs4) + // We did not see a valid proposal within this round, so prevote nil. + ensurePrevoteMatch(t, voteCh, height, round, nil) - ensureNewUnlock(t, unlockCh, height, round) + signAddVotes(ctx, t, cs1, tmproto.PrecommitType, cfg.ChainID(), types.BlockID{}, vs2, vs3, vs4) ensurePrecommit(t, voteCh, height, round) - // we should have precommitted - validatePrecommit(ctx, 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(ctx, t, cs1, round, 0, vss[0], nil, blockID.Hash) incrementRound(vs2, vs3, vs4) incrementRound(vs2, vs3, vs4) - signAddVotes(ctx, t, config, cs1, tmproto.PrecommitType, nil, types.PartSetHeader{}, vs2, vs3, vs4) + signAddVotes(ctx, t, cs1, tmproto.PrecommitType, cfg.ChainID(), types.BlockID{}, vs2, vs3, vs4) - round += 2 // moving to the next round + round += 2 // increment by multiple rounds ensureNewRound(t, newRoundCh, height, round) t.Log("### ONTO ROUND 3") @@ -1301,12 +1788,10 @@ func TestProposeValidBlock(t *testing.T) { ensureNewRound(t, newRoundCh, height, round) - t.Log("### ONTO ROUND 4") - ensureNewProposal(t, proposalCh, height, round) rs = cs1.GetRoundState() - assert.True(t, bytes.Equal(rs.ProposalBlock.Hash(), propBlockHash)) + assert.True(t, bytes.Equal(rs.ProposalBlock.Hash(), blockID.Hash)) assert.True(t, bytes.Equal(rs.ProposalBlock.Hash(), rs.ValidBlock.Hash())) assert.True(t, rs.Proposal.POLRound == rs.ValidRound) assert.True(t, bytes.Equal(rs.Proposal.BlockID.Hash, rs.ValidBlock.Hash())) @@ -1316,10 +1801,11 @@ func TestProposeValidBlock(t *testing.T) { // P0 miss to lock B but set valid block to B after receiving delayed prevote. func TestSetValidBlockOnDelayedPrevote(t *testing.T) { config := configSetup(t) + logger := log.NewNopLogger() ctx, cancel := context.WithCancel(context.Background()) defer cancel() - cs1, vss := randState(ctx, t, config, log.TestingLogger(), 4) + cs1, vss := makeState(ctx, t, config, logger, 4) vs2, vs3, vs4 := vss[1], vss[2], vss[3] height, round := cs1.Height, cs1.Round @@ -1341,17 +1827,20 @@ func TestSetValidBlockOnDelayedPrevote(t *testing.T) { ensureNewProposal(t, proposalCh, height, round) rs := cs1.GetRoundState() propBlock := rs.ProposalBlock - propBlockHash := propBlock.Hash() - propBlockParts, err := propBlock.MakePartSet(partSize) + partSet, err := propBlock.MakePartSet(partSize) require.NoError(t, err) + blockID := types.BlockID{ + Hash: propBlock.Hash(), + PartSetHeader: partSet.Header(), + } - ensurePrevoteMatch(t, voteCh, height, round, propBlockHash) + ensurePrevoteMatch(t, voteCh, height, round, blockID.Hash) // vs2 send prevote for propBlock - signAddVotes(ctx, t, config, cs1, tmproto.PrevoteType, propBlockHash, propBlockParts.Header(), vs2) + signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), blockID, vs2) // vs3 send prevote nil - signAddVotes(ctx, t, config, cs1, tmproto.PrevoteType, nil, types.PartSetHeader{}, vs3) + signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), types.BlockID{}, vs3) ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.config.Prevote(round).Nanoseconds()) @@ -1366,14 +1855,14 @@ func TestSetValidBlockOnDelayedPrevote(t *testing.T) { assert.True(t, rs.ValidRound == -1) // vs2 send (delayed) prevote for propBlock - signAddVotes(ctx, t, config, cs1, tmproto.PrevoteType, propBlockHash, propBlockParts.Header(), vs4) + signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), blockID, vs4) ensureNewValidBlock(t, validBlockCh, height, round) rs = cs1.GetRoundState() - assert.True(t, bytes.Equal(rs.ValidBlock.Hash(), propBlockHash)) - assert.True(t, rs.ValidBlockParts.Header().Equals(propBlockParts.Header())) + assert.True(t, bytes.Equal(rs.ValidBlock.Hash(), blockID.Hash)) + assert.True(t, rs.ValidBlockParts.Header().Equals(blockID.PartSetHeader)) assert.True(t, rs.ValidRound == round) } @@ -1382,10 +1871,11 @@ func TestSetValidBlockOnDelayedPrevote(t *testing.T) { // receiving delayed Block Proposal. func TestSetValidBlockOnDelayedProposal(t *testing.T) { config := configSetup(t) + logger := log.NewNopLogger() ctx, cancel := context.WithCancel(context.Background()) defer cancel() - cs1, vss := randState(ctx, t, config, log.TestingLogger(), 4) + cs1, vss := makeState(ctx, t, config, logger, 4) vs2, vs3, vs4 := vss[1], vss[2], vss[3] height, round := cs1.Height, cs1.Round @@ -1412,12 +1902,15 @@ func TestSetValidBlockOnDelayedProposal(t *testing.T) { ensurePrevoteMatch(t, voteCh, height, round, nil) prop, propBlock := decideProposal(ctx, t, cs1, vs2, vs2.Height, vs2.Round+1) - propBlockHash := propBlock.Hash() - propBlockParts, err := propBlock.MakePartSet(partSize) + partSet, err := propBlock.MakePartSet(partSize) require.NoError(t, err) + blockID := types.BlockID{ + Hash: propBlock.Hash(), + PartSetHeader: partSet.Header(), + } // vs2, vs3 and vs4 send prevote for propBlock - signAddVotes(ctx, t, config, cs1, tmproto.PrevoteType, propBlockHash, propBlockParts.Header(), vs2, vs3, vs4) + signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), blockID, vs2, vs3, vs4) ensureNewValidBlock(t, validBlockCh, height, round) ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.config.Prevote(round).Nanoseconds()) @@ -1425,15 +1918,16 @@ func TestSetValidBlockOnDelayedProposal(t *testing.T) { ensurePrecommit(t, voteCh, height, round) validatePrecommit(ctx, t, cs1, round, -1, vss[0], nil, nil) - if err := cs1.SetProposalAndBlock(ctx, prop, propBlock, propBlockParts, "some peer"); err != nil { - t.Fatal(err) - } + partSet, err = propBlock.MakePartSet(partSize) + require.NoError(t, err) + err = cs1.SetProposalAndBlock(ctx, prop, propBlock, partSet, "some peer") + require.NoError(t, err) ensureNewProposal(t, proposalCh, height, round) rs := cs1.GetRoundState() - assert.True(t, bytes.Equal(rs.ValidBlock.Hash(), propBlockHash)) - assert.True(t, rs.ValidBlockParts.Header().Equals(propBlockParts.Header())) + assert.True(t, bytes.Equal(rs.ValidBlock.Hash(), blockID.Hash)) + assert.True(t, rs.ValidBlockParts.Header().Equals(blockID.PartSetHeader)) assert.True(t, rs.ValidRound == round) } @@ -1441,12 +1935,12 @@ func TestSetValidBlockOnDelayedProposal(t *testing.T) { // What we want: // P0 waits for timeoutPrecommit before starting next round func TestWaitingTimeoutOnNilPolka(t *testing.T) { - config := configSetup(t) - ctx, cancel := context.WithCancel(context.Background()) defer cancel() + config := configSetup(t) + logger := log.NewNopLogger() - cs1, vss := randState(ctx, t, config, log.TestingLogger(), 4) + cs1, vss := makeState(ctx, t, config, logger, 4) vs2, vs3, vs4 := vss[1], vss[2], vss[3] height, round := cs1.Height, cs1.Round @@ -1457,7 +1951,7 @@ func TestWaitingTimeoutOnNilPolka(t *testing.T) { startTestRound(ctx, cs1, height, round) ensureNewRound(t, newRoundCh, height, round) - signAddVotes(ctx, t, config, cs1, tmproto.PrecommitType, nil, types.PartSetHeader{}, vs2, vs3, vs4) + signAddVotes(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), types.BlockID{}, vs2, vs3, vs4) ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.config.Precommit(round).Nanoseconds()) ensureNewRound(t, newRoundCh, height, round+1) @@ -1468,10 +1962,11 @@ func TestWaitingTimeoutOnNilPolka(t *testing.T) { // P0 waits for timeoutPropose in the next round before entering prevote func TestWaitingTimeoutProposeOnNewRound(t *testing.T) { config := configSetup(t) + logger := log.NewNopLogger() ctx, cancel := context.WithCancel(context.Background()) defer cancel() - cs1, vss := randState(ctx, t, config, log.TestingLogger(), 4) + cs1, vss := makeState(ctx, t, config, logger, 4) vs2, vs3, vs4 := vss[1], vss[2], vss[3] height, round := cs1.Height, cs1.Round @@ -1489,7 +1984,7 @@ func TestWaitingTimeoutProposeOnNewRound(t *testing.T) { ensurePrevote(t, voteCh, height, round) incrementRound(vss[1:]...) - signAddVotes(ctx, t, config, cs1, tmproto.PrevoteType, nil, types.PartSetHeader{}, vs2, vs3, vs4) + signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), types.BlockID{}, vs2, vs3, vs4) round++ // moving to the next round ensureNewRound(t, newRoundCh, height, round) @@ -1507,10 +2002,11 @@ func TestWaitingTimeoutProposeOnNewRound(t *testing.T) { // P0 jump to higher round, precommit and start precommit wait func TestRoundSkipOnNilPolkaFromHigherRound(t *testing.T) { config := configSetup(t) + logger := log.NewNopLogger() ctx, cancel := context.WithCancel(context.Background()) defer cancel() - cs1, vss := randState(ctx, t, config, log.TestingLogger(), 4) + cs1, vss := makeState(ctx, t, config, logger, 4) vs2, vs3, vs4 := vss[1], vss[2], vss[3] height, round := cs1.Height, cs1.Round @@ -1528,7 +2024,7 @@ func TestRoundSkipOnNilPolkaFromHigherRound(t *testing.T) { ensurePrevote(t, voteCh, height, round) incrementRound(vss[1:]...) - signAddVotes(ctx, t, config, cs1, tmproto.PrecommitType, nil, types.PartSetHeader{}, vs2, vs3, vs4) + signAddVotes(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), types.BlockID{}, vs2, vs3, vs4) round++ // moving to the next round ensureNewRound(t, newRoundCh, height, round) @@ -1547,11 +2043,11 @@ func TestRoundSkipOnNilPolkaFromHigherRound(t *testing.T) { // P0 wait for timeoutPropose to expire before sending prevote. func TestWaitTimeoutProposeOnNilPolkaForTheCurrentRound(t *testing.T) { config := configSetup(t) + logger := log.NewNopLogger() ctx, cancel := context.WithCancel(context.Background()) defer cancel() - cs1, vss := randState(ctx, t, config, log.TestingLogger(), 4) - + cs1, vss := makeState(ctx, t, config, logger, 4) vs2, vs3, vs4 := vss[1], vss[2], vss[3] height, round := cs1.Height, int32(1) @@ -1567,7 +2063,7 @@ func TestWaitTimeoutProposeOnNilPolkaForTheCurrentRound(t *testing.T) { ensureNewRound(t, newRoundCh, height, round) incrementRound(vss[1:]...) - signAddVotes(ctx, t, config, cs1, tmproto.PrevoteType, nil, types.PartSetHeader{}, vs2, vs3, vs4) + signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), types.BlockID{}, vs2, vs3, vs4) ensureNewTimeout(t, timeoutProposeCh, height, round, cs1.config.Propose(round).Nanoseconds()) @@ -1578,10 +2074,11 @@ func TestWaitTimeoutProposeOnNilPolkaForTheCurrentRound(t *testing.T) { // P0 emit NewValidBlock event upon receiving 2/3+ Precommit for B but hasn't received block B yet func TestEmitNewValidBlockEventOnCommitWithoutBlock(t *testing.T) { config := configSetup(t) + logger := log.NewNopLogger() ctx, cancel := context.WithCancel(context.Background()) defer cancel() - cs1, vss := randState(ctx, t, config, log.TestingLogger(), 4) + cs1, vss := makeState(ctx, t, config, logger, 4) vs2, vs3, vs4 := vss[1], vss[2], vss[3] height, round := cs1.Height, int32(1) @@ -1593,22 +2090,25 @@ func TestEmitNewValidBlockEventOnCommitWithoutBlock(t *testing.T) { validBlockCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryValidBlock) _, propBlock := decideProposal(ctx, t, cs1, vs2, vs2.Height, vs2.Round) - propBlockHash := propBlock.Hash() - propBlockParts, err := propBlock.MakePartSet(partSize) + partSet, err := propBlock.MakePartSet(partSize) require.NoError(t, err) + blockID := types.BlockID{ + Hash: propBlock.Hash(), + PartSetHeader: partSet.Header(), + } // start round in which PO is not proposer startTestRound(ctx, cs1, height, round) ensureNewRound(t, newRoundCh, height, round) // vs2, vs3 and vs4 send precommit for propBlock - signAddVotes(ctx, t, config, cs1, tmproto.PrecommitType, propBlockHash, propBlockParts.Header(), vs2, vs3, vs4) + signAddVotes(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), blockID, vs2, vs3, vs4) ensureNewValidBlock(t, validBlockCh, height, round) rs := cs1.GetRoundState() assert.True(t, rs.Step == cstypes.RoundStepCommit) assert.True(t, rs.ProposalBlock == nil) - assert.True(t, rs.ProposalBlockParts.Header().Equals(propBlockParts.Header())) + assert.True(t, rs.ProposalBlockParts.Header().Equals(blockID.PartSetHeader)) } @@ -1617,10 +2117,11 @@ func TestEmitNewValidBlockEventOnCommitWithoutBlock(t *testing.T) { // After receiving block, it executes block and moves to the next height. func TestCommitFromPreviousRound(t *testing.T) { config := configSetup(t) + logger := log.NewNopLogger() ctx, cancel := context.WithCancel(context.Background()) defer cancel() - cs1, vss := randState(ctx, t, config, log.TestingLogger(), 4) + cs1, vss := makeState(ctx, t, config, logger, 4) vs2, vs3, vs4 := vss[1], vss[2], vss[3] height, round := cs1.Height, int32(1) @@ -1631,16 +2132,19 @@ func TestCommitFromPreviousRound(t *testing.T) { proposalCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryCompleteProposal) prop, propBlock := decideProposal(ctx, t, cs1, vs2, vs2.Height, vs2.Round) - propBlockHash := propBlock.Hash() - propBlockParts, err := propBlock.MakePartSet(partSize) + partSet, err := propBlock.MakePartSet(partSize) require.NoError(t, err) + blockID := types.BlockID{ + Hash: propBlock.Hash(), + PartSetHeader: partSet.Header(), + } // start round in which PO is not proposer startTestRound(ctx, cs1, height, round) ensureNewRound(t, newRoundCh, height, round) // vs2, vs3 and vs4 send precommit for propBlock for the previous round - signAddVotes(ctx, t, config, cs1, tmproto.PrecommitType, propBlockHash, propBlockParts.Header(), vs2, vs3, vs4) + signAddVotes(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), blockID, vs2, vs3, vs4) ensureNewValidBlock(t, validBlockCh, height, round) @@ -1648,11 +2152,11 @@ func TestCommitFromPreviousRound(t *testing.T) { assert.True(t, rs.Step == cstypes.RoundStepCommit) assert.True(t, rs.CommitRound == vs2.Round) assert.True(t, rs.ProposalBlock == nil) - assert.True(t, rs.ProposalBlockParts.Header().Equals(propBlockParts.Header())) - - if err := cs1.SetProposalAndBlock(ctx, prop, propBlock, propBlockParts, "some peer"); err != nil { - t.Fatal(err) - } + assert.True(t, rs.ProposalBlockParts.Header().Equals(blockID.PartSetHeader)) + partSet, err = propBlock.MakePartSet(partSize) + require.NoError(t, err) + err = cs1.SetProposalAndBlock(ctx, prop, propBlock, partSet, "some peer") + require.NoError(t, err) ensureNewProposal(t, proposalCh, height, round) ensureNewRound(t, newRoundCh, height+1, 0) @@ -1675,11 +2179,12 @@ func (n *fakeTxNotifier) Notify() { // start of the next round func TestStartNextHeightCorrectlyAfterTimeout(t *testing.T) { config := configSetup(t) + logger := log.NewNopLogger() ctx, cancel := context.WithCancel(context.Background()) defer cancel() config.Consensus.SkipTimeoutCommit = false - cs1, vss := randState(ctx, t, config, log.TestingLogger(), 4) + cs1, vss := makeState(ctx, t, config, logger, 4) cs1.txNotifier = &fakeTxNotifier{ch: make(chan struct{})} vs2, vs3, vs4 := vss[1], vss[2], vss[3] @@ -1702,30 +2207,32 @@ func TestStartNextHeightCorrectlyAfterTimeout(t *testing.T) { ensureNewProposal(t, proposalCh, height, round) rs := cs1.GetRoundState() - theBlockHash := rs.ProposalBlock.Hash() - theBlockParts := rs.ProposalBlockParts.Header() + blockID := types.BlockID{ + Hash: rs.ProposalBlock.Hash(), + PartSetHeader: rs.ProposalBlockParts.Header(), + } - ensurePrevoteMatch(t, voteCh, height, round, theBlockHash) + ensurePrevoteMatch(t, voteCh, height, round, blockID.Hash) - signAddVotes(ctx, t, config, cs1, tmproto.PrevoteType, theBlockHash, theBlockParts, vs2, vs3, vs4) + signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), blockID, vs2, vs3, vs4) ensurePrecommit(t, voteCh, height, round) // the proposed block should now be locked and our precommit added - validatePrecommit(ctx, t, cs1, round, round, vss[0], theBlockHash, theBlockHash) + validatePrecommit(ctx, t, cs1, round, round, vss[0], blockID.Hash, blockID.Hash) // add precommits - signAddVotes(ctx, t, config, cs1, tmproto.PrecommitType, nil, types.PartSetHeader{}, vs2) - signAddVotes(ctx, t, config, cs1, tmproto.PrecommitType, theBlockHash, theBlockParts, vs3) + signAddVotes(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), types.BlockID{}, vs2) + signAddVotes(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), blockID, vs3) // wait till timeout occurs - ensurePrecommitTimeout(t, precommitTimeoutCh) + ensureNewTimeout(t, precommitTimeoutCh, height, round, cs1.config.TimeoutPrecommit.Nanoseconds()) ensureNewRound(t, newRoundCh, height, round+1) // majority is now reached - signAddVotes(ctx, t, config, cs1, tmproto.PrecommitType, theBlockHash, theBlockParts, vs4) + signAddVotes(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), blockID, vs4) - ensureNewBlockHeader(t, newBlockHeader, height, theBlockHash) + ensureNewBlockHeader(t, newBlockHeader, height, blockID.Hash) cs1.txNotifier.(*fakeTxNotifier).Notify() @@ -1739,11 +2246,12 @@ func TestStartNextHeightCorrectlyAfterTimeout(t *testing.T) { func TestResetTimeoutPrecommitUponNewHeight(t *testing.T) { config := configSetup(t) + logger := log.NewNopLogger() ctx, cancel := context.WithCancel(context.Background()) defer cancel() config.Consensus.SkipTimeoutCommit = false - cs1, vss := randState(ctx, t, config, log.TestingLogger(), 4) + cs1, vss := makeState(ctx, t, config, logger, 4) vs2, vs3, vs4 := vss[1], vss[2], vss[3] height, round := cs1.Height, cs1.Round @@ -1765,30 +2273,31 @@ func TestResetTimeoutPrecommitUponNewHeight(t *testing.T) { ensureNewProposal(t, proposalCh, height, round) rs := cs1.GetRoundState() - theBlockHash := rs.ProposalBlock.Hash() - theBlockParts := rs.ProposalBlockParts.Header() + blockID := types.BlockID{ + Hash: rs.ProposalBlock.Hash(), + PartSetHeader: rs.ProposalBlockParts.Header(), + } - ensurePrevoteMatch(t, voteCh, height, round, theBlockHash) + ensurePrevoteMatch(t, voteCh, height, round, blockID.Hash) - signAddVotes(ctx, t, config, cs1, tmproto.PrevoteType, theBlockHash, theBlockParts, vs2, vs3, vs4) + signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), blockID, vs2, vs3, vs4) ensurePrecommit(t, voteCh, height, round) - validatePrecommit(ctx, t, cs1, round, round, vss[0], theBlockHash, theBlockHash) + validatePrecommit(ctx, t, cs1, round, round, vss[0], blockID.Hash, blockID.Hash) // add precommits - signAddVotes(ctx, t, config, cs1, tmproto.PrecommitType, nil, types.PartSetHeader{}, vs2) - signAddVotes(ctx, t, config, cs1, tmproto.PrecommitType, theBlockHash, theBlockParts, vs3) - signAddVotes(ctx, t, config, cs1, tmproto.PrecommitType, theBlockHash, theBlockParts, vs4) + signAddVotes(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), types.BlockID{}, vs2) + signAddVotes(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), blockID, vs3) + signAddVotes(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), blockID, vs4) - ensureNewBlockHeader(t, newBlockHeader, height, theBlockHash) + ensureNewBlockHeader(t, newBlockHeader, height, blockID.Hash) prop, propBlock := decideProposal(ctx, t, cs1, vs2, height+1, 0) propBlockParts, err := propBlock.MakePartSet(partSize) require.NoError(t, err) - if err := cs1.SetProposalAndBlock(ctx, prop, propBlock, propBlockParts, "some peer"); err != nil { - t.Fatal(err) - } + err = cs1.SetProposalAndBlock(ctx, prop, propBlock, propBlockParts, "some peer") + require.NoError(t, err) ensureNewProposal(t, proposalCh, height+1, 0) rs = cs1.GetRoundState() @@ -1798,83 +2307,6 @@ func TestResetTimeoutPrecommitUponNewHeight(t *testing.T) { "triggeredTimeoutPrecommit should be false at the beginning of each height") } -//------------------------------------------------------------------------------------------ -// SlashingSuite -// TODO: Slashing - -/* -func TestStateSlashingPrevotes(t *testing.T) { - cs1, vss := randState(2) - vs2 := vss[1] - - - proposalCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryCompleteProposal) - timeoutWaitCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryTimeoutWait) - newRoundCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryNewRound) - voteCh := subscribeToVoter(ctx, t, cs1, cs1.privValidator.GetAddress()) - - // start round and wait for propose and prevote - startTestRound(cs1, cs1.Height, 0) - <-newRoundCh - re := <-proposalCh - <-voteCh // prevote - - rs := re.(types.EventDataRoundState).RoundState.(*cstypes.RoundState) - - // we should now be stuck in limbo forever, waiting for more prevotes - // add one for a different block should cause us to go into prevote wait - hash := rs.ProposalBlock.Hash() - hash[0] = byte(hash[0]+1) % 255 - signAddVotes(cs1, tmproto.PrevoteType, hash, rs.ProposalBlockParts.Header(), vs2) - - <-timeoutWaitCh - - // NOTE: we have to send the vote for different block first so we don't just go into precommit round right - // away and ignore more prevotes (and thus fail to slash!) - - // add the conflicting vote - signAddVotes(cs1, tmproto.PrevoteType, rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header(), vs2) - - // XXX: Check for existence of Dupeout info -} - -func TestStateSlashingPrecommits(t *testing.T) { - cs1, vss := randState(2) - vs2 := vss[1] - - - proposalCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryCompleteProposal) - timeoutWaitCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryTimeoutWait) - newRoundCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryNewRound) - voteCh := subscribeToVoter(ctx, t, cs1, cs1.privValidator.GetAddress()) - - // start round and wait for propose and prevote - startTestRound(cs1, cs1.Height, 0) - <-newRoundCh - re := <-proposalCh - <-voteCh // prevote - - // add prevote from vs2 - signAddVotes(cs1, tmproto.PrevoteType, rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header(), vs2) - - <-voteCh // precommit - - // we should now be stuck in limbo forever, waiting for more prevotes - // add one for a different block should cause us to go into prevote wait - hash := rs.ProposalBlock.Hash() - hash[0] = byte(hash[0]+1) % 255 - signAddVotes(cs1, tmproto.PrecommitType, hash, rs.ProposalBlockParts.Header(), vs2) - - // NOTE: we have to send the vote for different block first so we don't just go into precommit round right - // away and ignore more prevotes (and thus fail to slash!) - - // add precommit from vs2 - signAddVotes(cs1, tmproto.PrecommitType, rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header(), vs2) - - // XXX: Check for existence of Dupeout info -} -*/ - //------------------------------------------------------------------------------------------ // CatchupSuite @@ -1884,11 +2316,12 @@ func TestStateSlashingPrecommits(t *testing.T) { // 4 vals. // we receive a final precommit after going into next round, but others might have gone to commit already! func TestStateHalt1(t *testing.T) { - config := configSetup(t) + cfg := configSetup(t) + logger := log.NewNopLogger() ctx, cancel := context.WithCancel(context.Background()) defer cancel() - cs1, vss := randState(ctx, t, config, log.TestingLogger(), 4) + cs1, vss := makeState(ctx, t, cfg, logger, 4) vs2, vs3, vs4 := vss[1], vss[2], vss[3] height, round := cs1.Height, cs1.Round partSize := types.BlockPartSizeBytes @@ -1909,22 +2342,26 @@ func TestStateHalt1(t *testing.T) { ensureNewProposal(t, proposalCh, height, round) rs := cs1.GetRoundState() propBlock := rs.ProposalBlock - propBlockParts, err := propBlock.MakePartSet(partSize) + partSet, err := propBlock.MakePartSet(partSize) require.NoError(t, err) + blockID := types.BlockID{ + Hash: propBlock.Hash(), + PartSetHeader: partSet.Header(), + } ensurePrevote(t, voteCh, height, round) - signAddVotes(ctx, t, config, cs1, tmproto.PrevoteType, propBlock.Hash(), propBlockParts.Header(), vs2, vs3, vs4) + signAddVotes(ctx, t, cs1, tmproto.PrevoteType, cfg.ChainID(), blockID, vs2, vs3, vs4) ensurePrecommit(t, voteCh, height, round) // the proposed block should now be locked and our precommit added validatePrecommit(ctx, t, cs1, round, round, vss[0], propBlock.Hash(), propBlock.Hash()) // add precommits from the rest - signAddVotes(ctx, t, config, cs1, tmproto.PrecommitType, nil, types.PartSetHeader{}, vs2) // didnt receive proposal - signAddVotes(ctx, t, config, cs1, tmproto.PrecommitType, propBlock.Hash(), propBlockParts.Header(), vs3) + signAddVotes(ctx, t, cs1, tmproto.PrecommitType, cfg.ChainID(), types.BlockID{}, vs2) // didnt receive proposal + signAddVotes(ctx, t, cs1, tmproto.PrecommitType, cfg.ChainID(), blockID, vs3) // we receive this later, but vs3 might receive it earlier and with ours will go to commit! - precommit4 := signVote(ctx, t, vs4, config, tmproto.PrecommitType, propBlock.Hash(), propBlockParts.Header()) + precommit4 := signVote(ctx, t, vs4, tmproto.PrecommitType, cfg.ChainID(), blockID) incrementRound(vs2, vs3, vs4) @@ -1934,15 +2371,14 @@ func TestStateHalt1(t *testing.T) { round++ // moving to the next round ensureNewRound(t, newRoundCh, height, round) - rs = cs1.GetRoundState() t.Log("### ONTO ROUND 1") /*Round2 - // we timeout and prevote our lock + // we timeout and prevote // a polka happened but we didn't see it! */ - // go to prevote, prevote for locked block + // prevote for nil since we did not receive a proposal in this round. ensurePrevoteMatch(t, voteCh, height, round, rs.LockedBlock.Hash()) // now we receive the precommit from the previous round @@ -1956,11 +2392,12 @@ func TestStateHalt1(t *testing.T) { func TestStateOutputsBlockPartsStats(t *testing.T) { config := configSetup(t) + logger := log.NewNopLogger() ctx, cancel := context.WithCancel(context.Background()) defer cancel() // create dummy peer - cs, _ := randState(ctx, t, config, log.TestingLogger(), 1) + cs, _ := makeState(ctx, t, config, logger, 1) peerID, err := types.NewNodeID("AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA") require.NoError(t, err) @@ -1973,26 +2410,26 @@ func TestStateOutputsBlockPartsStats(t *testing.T) { } cs.ProposalBlockParts = types.NewPartSetFromHeader(parts.Header()) - cs.handleMsg(ctx, msgInfo{msg, peerID}) + cs.handleMsg(ctx, msgInfo{msg, peerID, tmtime.Now()}) statsMessage := <-cs.statsMsgQueue require.Equal(t, msg, statsMessage.Msg, "") require.Equal(t, peerID, statsMessage.PeerID, "") // sending the same part from different peer - cs.handleMsg(ctx, msgInfo{msg, "peer2"}) + cs.handleMsg(ctx, msgInfo{msg, "peer2", tmtime.Now()}) // sending the part with the same height, but different round msg.Round = 1 - cs.handleMsg(ctx, msgInfo{msg, peerID}) + cs.handleMsg(ctx, msgInfo{msg, peerID, tmtime.Now()}) // sending the part from the smaller height msg.Height = 0 - cs.handleMsg(ctx, msgInfo{msg, peerID}) + cs.handleMsg(ctx, msgInfo{msg, peerID, tmtime.Now()}) // sending the part from the bigger height msg.Height = 3 - cs.handleMsg(ctx, msgInfo{msg, peerID}) + cs.handleMsg(ctx, msgInfo{msg, peerID, tmtime.Now()}) select { case <-cs.statsMsgQueue: @@ -2004,33 +2441,37 @@ func TestStateOutputsBlockPartsStats(t *testing.T) { func TestStateOutputVoteStats(t *testing.T) { config := configSetup(t) + logger := log.NewNopLogger() ctx, cancel := context.WithCancel(context.Background()) defer cancel() - cs, vss := randState(ctx, t, config, log.TestingLogger(), 2) + cs, vss := makeState(ctx, t, config, logger, 2) // create dummy peer peerID, err := types.NewNodeID("AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA") require.NoError(t, err) randBytes := tmrand.Bytes(tmhash.Size) + blockID := types.BlockID{ + Hash: randBytes, + } - vote := signVote(ctx, t, vss[1], config, tmproto.PrecommitType, randBytes, types.PartSetHeader{}) + vote := signVote(ctx, t, vss[1], tmproto.PrecommitType, config.ChainID(), blockID) voteMessage := &VoteMessage{vote} - cs.handleMsg(ctx, msgInfo{voteMessage, peerID}) + cs.handleMsg(ctx, msgInfo{voteMessage, peerID, tmtime.Now()}) statsMessage := <-cs.statsMsgQueue require.Equal(t, voteMessage, statsMessage.Msg, "") require.Equal(t, peerID, statsMessage.PeerID, "") // sending the same part from different peer - cs.handleMsg(ctx, msgInfo{&VoteMessage{vote}, "peer2"}) + cs.handleMsg(ctx, msgInfo{&VoteMessage{vote}, "peer2", tmtime.Now()}) // sending the vote for the bigger height incrementHeight(vss[1]) - vote = signVote(ctx, t, vss[1], config, tmproto.PrecommitType, randBytes, types.PartSetHeader{}) + vote = signVote(ctx, t, vss[1], tmproto.PrecommitType, config.ChainID(), blockID) - cs.handleMsg(ctx, msgInfo{&VoteMessage{vote}, peerID}) + cs.handleMsg(ctx, msgInfo{&VoteMessage{vote}, peerID, tmtime.Now()}) select { case <-cs.statsMsgQueue: @@ -2042,10 +2483,11 @@ func TestStateOutputVoteStats(t *testing.T) { func TestSignSameVoteTwice(t *testing.T) { config := configSetup(t) + logger := log.NewNopLogger() ctx, cancel := context.WithCancel(context.Background()) defer cancel() - _, vss := randState(ctx, t, config, log.TestingLogger(), 2) + _, vss := makeState(ctx, t, config, logger, 2) randBytes := tmrand.Bytes(tmhash.Size) @@ -2053,25 +2495,128 @@ func TestSignSameVoteTwice(t *testing.T) { ctx, t, vss[1], - config, tmproto.PrecommitType, - randBytes, - types.PartSetHeader{Total: 10, Hash: randBytes}, - ) + config.ChainID(), + types.BlockID{ + Hash: randBytes, + PartSetHeader: types.PartSetHeader{Total: 10, Hash: randBytes}, + }, + ) vote2 := signVote( ctx, t, vss[1], - config, tmproto.PrecommitType, - randBytes, - types.PartSetHeader{Total: 10, Hash: randBytes}, + config.ChainID(), + + types.BlockID{ + Hash: randBytes, + PartSetHeader: types.PartSetHeader{Total: 10, Hash: randBytes}, + }, ) require.Equal(t, vote, vote2) } +// TestStateTimestamp_ProposalNotMatch tests that a validator does not prevote a +// proposed block if the timestamp in the block does not matche the timestamp in the +// corresponding proposal message. +func TestStateTimestamp_ProposalNotMatch(t *testing.T) { + config := configSetup(t) + logger := log.NewNopLogger() + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + cs1, vss := makeState(ctx, t, config, logger, 4) + height, round := cs1.Height, cs1.Round + vs2, vs3, vs4 := vss[1], vss[2], vss[3] + + proposalCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryCompleteProposal) + pv1, err := cs1.privValidator.GetPubKey(ctx) + require.NoError(t, err) + addr := pv1.Address() + voteCh := subscribeToVoter(ctx, t, cs1, addr) + + propBlock, _, err := cs1.createProposalBlock() + require.NoError(t, err) + round++ + incrementRound(vss[1:]...) + + propBlockParts, err := propBlock.MakePartSet(types.BlockPartSizeBytes) + require.NoError(t, err) + blockID := types.BlockID{Hash: propBlock.Hash(), PartSetHeader: propBlockParts.Header()} + + // Create a proposal with a timestamp that does not match the timestamp of the block. + proposal := types.NewProposal(vs2.Height, round, -1, blockID, propBlock.Header.Time.Add(time.Millisecond)) + p := proposal.ToProto() + err = vs2.SignProposal(ctx, config.ChainID(), p) + require.NoError(t, err) + proposal.Signature = p.Signature + require.NoError(t, cs1.SetProposalAndBlock(ctx, proposal, propBlock, propBlockParts, "some peer")) + + startTestRound(ctx, cs1, height, round) + ensureProposal(t, proposalCh, height, round, blockID) + + signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), blockID, vs2, vs3, vs4) + + // ensure that the validator prevotes nil. + ensurePrevote(t, voteCh, height, round) + validatePrevote(ctx, t, cs1, round, vss[0], nil) + + ensurePrecommit(t, voteCh, height, round) + validatePrecommit(ctx, t, cs1, round, -1, vss[0], nil, nil) +} + +// TestStateTimestamp_ProposalMatch tests that a validator prevotes a +// proposed block if the timestamp in the block matches the timestamp in the +// corresponding proposal message. +func TestStateTimestamp_ProposalMatch(t *testing.T) { + config := configSetup(t) + logger := log.NewNopLogger() + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + cs1, vss := makeState(ctx, t, config, logger, 4) + height, round := cs1.Height, cs1.Round + vs2, vs3, vs4 := vss[1], vss[2], vss[3] + + proposalCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryCompleteProposal) + pv1, err := cs1.privValidator.GetPubKey(ctx) + require.NoError(t, err) + addr := pv1.Address() + voteCh := subscribeToVoter(ctx, t, cs1, addr) + + propBlock, _, err := cs1.createProposalBlock() + require.NoError(t, err) + round++ + incrementRound(vss[1:]...) + + propBlockParts, err := propBlock.MakePartSet(types.BlockPartSizeBytes) + require.NoError(t, err) + blockID := types.BlockID{Hash: propBlock.Hash(), PartSetHeader: propBlockParts.Header()} + + // Create a proposal with a timestamp that matches the timestamp of the block. + proposal := types.NewProposal(vs2.Height, round, -1, blockID, propBlock.Header.Time) + p := proposal.ToProto() + err = vs2.SignProposal(ctx, config.ChainID(), p) + require.NoError(t, err) + proposal.Signature = p.Signature + require.NoError(t, cs1.SetProposalAndBlock(ctx, proposal, propBlock, propBlockParts, "some peer")) + + startTestRound(ctx, cs1, height, round) + ensureProposal(t, proposalCh, height, round, blockID) + + signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), blockID, vs2, vs3, vs4) + + // ensure that the validator prevotes the block. + ensurePrevote(t, voteCh, height, round) + validatePrevote(ctx, t, cs1, round, vss[0], propBlock.Hash()) + + ensurePrecommit(t, voteCh, height, round) + validatePrecommit(ctx, t, cs1, round, 1, vss[0], propBlock.Hash(), propBlock.Hash()) +} + // subscribe subscribes test client to the given query and returns a channel with cap = 1. func subscribe( ctx context.Context, @@ -2084,9 +2629,7 @@ func subscribe( ClientID: testSubscriber, Query: q, }) - if err != nil { - t.Fatalf("Failed to subscribe %q to %v: %v", testSubscriber, q, err) - } + require.NoErrorf(t, err, "Failed to subscribe %q to %v: %v", testSubscriber, q, err) ch := make(chan tmpubsub.Message) go func() { for { diff --git a/internal/consensus/types/height_vote_set_test.go b/internal/consensus/types/height_vote_set_test.go index cc656ffb5..3ebfcf2ee 100644 --- a/internal/consensus/types/height_vote_set_test.go +++ b/internal/consensus/types/height_vote_set_test.go @@ -33,7 +33,7 @@ func TestPeerCatchupRounds(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() - valSet, privVals := factory.RandValidatorSet(ctx, t, 10, 1) + valSet, privVals := factory.ValidatorSet(ctx, t, 10, 1) hvs := NewHeightVoteSet(cfg.ChainID(), 1, valSet) diff --git a/internal/consensus/types/round_state.go b/internal/consensus/types/round_state.go index 15f1777b4..566ca04d0 100644 --- a/internal/consensus/types/round_state.go +++ b/internal/consensus/types/round_state.go @@ -71,14 +71,15 @@ type RoundState struct { StartTime time.Time `json:"start_time"` // Subjective time when +2/3 precommits for Block at Round were found - CommitTime time.Time `json:"commit_time"` - Validators *types.ValidatorSet `json:"validators"` - Proposal *types.Proposal `json:"proposal"` - ProposalBlock *types.Block `json:"proposal_block"` - ProposalBlockParts *types.PartSet `json:"proposal_block_parts"` - LockedRound int32 `json:"locked_round"` - LockedBlock *types.Block `json:"locked_block"` - LockedBlockParts *types.PartSet `json:"locked_block_parts"` + CommitTime time.Time `json:"commit_time"` + Validators *types.ValidatorSet `json:"validators"` + Proposal *types.Proposal `json:"proposal"` + ProposalReceiveTime time.Time `json:"proposal_receive_time"` + ProposalBlock *types.Block `json:"proposal_block"` + ProposalBlockParts *types.PartSet `json:"proposal_block_parts"` + LockedRound int32 `json:"locked_round"` + LockedBlock *types.Block `json:"locked_block"` + LockedBlockParts *types.PartSet `json:"locked_block_parts"` // Last known round with POL for non-nil valid block. ValidRound int32 `json:"valid_round"` diff --git a/internal/eventbus/event_bus.go b/internal/eventbus/event_bus.go index add10da0a..3d5f7d82f 100644 --- a/internal/eventbus/event_bus.go +++ b/internal/eventbus/event_bus.go @@ -186,10 +186,6 @@ func (b *EventBus) PublishEventPolka(ctx context.Context, data types.EventDataRo return b.Publish(ctx, types.EventPolkaValue, data) } -func (b *EventBus) PublishEventUnlock(ctx context.Context, data types.EventDataRoundState) error { - return b.Publish(ctx, types.EventUnlockValue, data) -} - func (b *EventBus) PublishEventRelock(ctx context.Context, data types.EventDataRoundState) error { return b.Publish(ctx, types.EventRelockValue, data) } diff --git a/internal/eventbus/event_bus_test.go b/internal/eventbus/event_bus_test.go index dd711564e..8ecd1f08a 100644 --- a/internal/eventbus/event_bus_test.go +++ b/internal/eventbus/event_bus_test.go @@ -385,7 +385,6 @@ func TestEventBusPublish(t *testing.T) { require.NoError(t, eventBus.PublishEventNewRound(ctx, types.EventDataNewRound{})) require.NoError(t, eventBus.PublishEventCompleteProposal(ctx, types.EventDataCompleteProposal{})) require.NoError(t, eventBus.PublishEventPolka(ctx, types.EventDataRoundState{})) - require.NoError(t, eventBus.PublishEventUnlock(ctx, types.EventDataRoundState{})) require.NoError(t, eventBus.PublishEventRelock(ctx, types.EventDataRoundState{})) require.NoError(t, eventBus.PublishEventLock(ctx, types.EventDataRoundState{})) require.NoError(t, eventBus.PublishEventValidatorSetUpdates(ctx, types.EventDataValidatorSetUpdates{})) @@ -487,7 +486,6 @@ var events = []string{ types.EventTimeoutProposeValue, types.EventCompleteProposalValue, types.EventPolkaValue, - types.EventUnlockValue, types.EventLockValue, types.EventRelockValue, types.EventTimeoutWaitValue, @@ -508,7 +506,6 @@ var queries = []*tmquery.Query{ types.EventQueryTimeoutPropose, types.EventQueryCompleteProposal, types.EventQueryPolka, - types.EventQueryUnlock, types.EventQueryLock, types.EventQueryRelock, types.EventQueryTimeoutWait, diff --git a/internal/evidence/pool_test.go b/internal/evidence/pool_test.go index 1c994724c..36431d49d 100644 --- a/internal/evidence/pool_test.go +++ b/internal/evidence/pool_test.go @@ -41,8 +41,7 @@ func TestEvidencePoolBasic(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() - valSet, privVals := factory.RandValidatorSet(ctx, t, 1, 10) - + valSet, privVals := factory.ValidatorSet(ctx, t, 1, 10) blockStore.On("LoadBlockMeta", mock.AnythingOfType("int64")).Return( &types.BlockMeta{Header: types.Header{Time: defaultEvidenceTime}}, ) diff --git a/internal/evidence/verify_test.go b/internal/evidence/verify_test.go index 723364363..bcc008aee 100644 --- a/internal/evidence/verify_test.go +++ b/internal/evidence/verify_test.go @@ -201,7 +201,7 @@ func TestVerifyLightClientAttack_Equivocation(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() - conflictingVals, conflictingPrivVals := factory.RandValidatorSet(ctx, t, 5, 10) + conflictingVals, conflictingPrivVals := factory.ValidatorSet(ctx, t, 5, 10) conflictingHeader := factory.MakeHeader(t, &types.Header{ ChainID: evidenceChainID, @@ -294,12 +294,10 @@ func TestVerifyLightClientAttack_Equivocation(t *testing.T) { } func TestVerifyLightClientAttack_Amnesia(t *testing.T) { - var height int64 = 10 - ctx, cancel := context.WithCancel(context.Background()) defer cancel() - - conflictingVals, conflictingPrivVals := factory.RandValidatorSet(ctx, t, 5, 10) + var height int64 = 10 + conflictingVals, conflictingPrivVals := factory.ValidatorSet(ctx, t, 5, 10) conflictingHeader := factory.MakeHeader(t, &types.Header{ ChainID: evidenceChainID, @@ -496,14 +494,14 @@ func makeLunaticEvidence( ) (ev *types.LightClientAttackEvidence, trusted *types.LightBlock, common *types.LightBlock) { t.Helper() - commonValSet, commonPrivVals := factory.RandValidatorSet(ctx, t, totalVals, defaultVotingPower) + commonValSet, commonPrivVals := factory.ValidatorSet(ctx, t, totalVals, defaultVotingPower) require.Greater(t, totalVals, byzVals) // extract out the subset of byzantine validators in the common validator set byzValSet, byzPrivVals := commonValSet.Validators[:byzVals], commonPrivVals[:byzVals] - phantomValSet, phantomPrivVals := factory.RandValidatorSet(ctx, t, phantomVals, defaultVotingPower) + phantomValSet, phantomPrivVals := factory.ValidatorSet(ctx, t, phantomVals, defaultVotingPower) conflictingVals := phantomValSet.Copy() require.NoError(t, conflictingVals.UpdateWithChangeSet(byzValSet)) @@ -558,7 +556,7 @@ func makeLunaticEvidence( ValidatorSet: commonValSet, } trustedBlockID := factory.MakeBlockIDWithHash(trustedHeader.Hash()) - trustedVals, privVals := factory.RandValidatorSet(ctx, t, totalVals, defaultVotingPower) + trustedVals, privVals := factory.ValidatorSet(ctx, t, totalVals, defaultVotingPower) trustedVoteSet := types.NewVoteSet(evidenceChainID, height, 1, tmproto.SignedMsgType(2), trustedVals) trustedCommit, err := factory.MakeCommit(ctx, trustedBlockID, height, 1, trustedVoteSet, privVals, defaultEvidenceTime) require.NoError(t, err) diff --git a/internal/state/helpers_test.go b/internal/state/helpers_test.go index c28bdc104..99bb2720d 100644 --- a/internal/state/helpers_test.go +++ b/internal/state/helpers_test.go @@ -4,6 +4,7 @@ import ( "bytes" "context" "fmt" + "math/rand" "testing" "time" @@ -258,7 +259,6 @@ func makeRandomStateFromValidatorSet( InitialHeight: 1, } } - func makeRandomStateFromConsensusParams( ctx context.Context, t *testing.T, @@ -267,7 +267,7 @@ func makeRandomStateFromConsensusParams( lastHeightConsensusParamsChanged int64, ) sm.State { t.Helper() - val, _, err := factory.RandValidator(ctx, true, 10) + val, _, err := factory.Validator(ctx, 10+int64(rand.Uint32())) require.NoError(t, err) valSet := types.NewValidatorSet([]*types.Validator{val}) return sm.State{ diff --git a/internal/state/rollback_test.go b/internal/state/rollback_test.go index 17d8e2be8..d74889ed0 100644 --- a/internal/state/rollback_test.go +++ b/internal/state/rollback_test.go @@ -19,11 +19,9 @@ func TestRollback(t *testing.T) { height int64 = 100 nextHeight int64 = 101 ) - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() blockStore := &mocks.BlockStore{} - stateStore := setupStateStore(ctx, t, height) + stateStore := setupStateStore(t, height) initialState, err := stateStore.Load() require.NoError(t, err) @@ -83,10 +81,7 @@ func TestRollbackNoState(t *testing.T) { func TestRollbackNoBlocks(t *testing.T) { const height = int64(100) - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - stateStore := setupStateStore(ctx, t, height) + stateStore := setupStateStore(t, height) blockStore := &mocks.BlockStore{} blockStore.On("Height").Return(height) blockStore.On("LoadBlockMeta", height-1).Return(nil) @@ -98,11 +93,7 @@ func TestRollbackNoBlocks(t *testing.T) { func TestRollbackDifferentStateHeight(t *testing.T) { const height = int64(100) - - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - stateStore := setupStateStore(ctx, t, height) + stateStore := setupStateStore(t, height) blockStore := &mocks.BlockStore{} blockStore.On("Height").Return(height + 2) @@ -111,9 +102,11 @@ func TestRollbackDifferentStateHeight(t *testing.T) { require.Equal(t, err.Error(), "statestore height (100) is not one below or equal to blockstore height (102)") } -func setupStateStore(ctx context.Context, t *testing.T, height int64) state.Store { +func setupStateStore(t *testing.T, height int64) state.Store { stateStore := state.NewStore(dbm.NewMemDB()) - valSet, _ := factory.RandValidatorSet(ctx, t, 5, 10) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + valSet, _ := factory.ValidatorSet(ctx, t, 5, 10) params := types.DefaultConsensusParams() params.Version.AppVersion = 10 diff --git a/internal/state/state.go b/internal/state/state.go index 8b69e6f0c..876c87fe8 100644 --- a/internal/state/state.go +++ b/internal/state/state.go @@ -268,7 +268,7 @@ func (state State) MakeBlock( if height == state.InitialHeight { timestamp = state.LastBlockTime // genesis time } else { - timestamp = MedianTime(commit, state.LastValidators) + timestamp = time.Now() } // Fill rest of header with state data. @@ -288,29 +288,6 @@ func (state State) MakeBlock( return block, bps, nil } -// MedianTime computes a median time for a given Commit (based on Timestamp field of votes messages) and the -// corresponding validator set. The computed time is always between timestamps of -// the votes sent by honest processes, i.e., a faulty processes can not arbitrarily increase or decrease the -// computed value. -func MedianTime(commit *types.Commit, validators *types.ValidatorSet) time.Time { - weightedTimes := make([]*weightedTime, len(commit.Signatures)) - totalVotingPower := int64(0) - - for i, commitSig := range commit.Signatures { - if commitSig.Absent() { - continue - } - _, validator := validators.GetByAddress(commitSig.ValidatorAddress) - // If there's no condition, TestValidateBlockCommit panics; not needed normally. - if validator != nil { - totalVotingPower += validator.VotingPower - weightedTimes[i] = newWeightedTime(commitSig.Timestamp, validator.VotingPower) - } - } - - return weightedMedian(weightedTimes, totalVotingPower) -} - //------------------------------------------------------------------------ // Genesis diff --git a/internal/state/store_test.go b/internal/state/store_test.go index 594c25c76..d7e599610 100644 --- a/internal/state/store_test.go +++ b/internal/state/store_test.go @@ -3,6 +3,7 @@ package state_test import ( "context" "fmt" + "math/rand" "os" "testing" @@ -27,16 +28,15 @@ const ( ) func TestStoreBootstrap(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - stateDB := dbm.NewMemDB() stateStore := sm.NewStore(stateDB) - val, _, err := factory.RandValidator(ctx, true, 10) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + val, _, err := factory.Validator(ctx, 10+int64(rand.Uint32())) require.NoError(t, err) - val2, _, err := factory.RandValidator(ctx, true, 10) + val2, _, err := factory.Validator(ctx, 10+int64(rand.Uint32())) require.NoError(t, err) - val3, _, err := factory.RandValidator(ctx, true, 10) + val3, _, err := factory.Validator(ctx, 10+int64(rand.Uint32())) require.NoError(t, err) vals := types.NewValidatorSet([]*types.Validator{val, val2, val3}) bootstrapState := makeRandomStateFromValidatorSet(vals, 100, 100) @@ -58,16 +58,15 @@ func TestStoreBootstrap(t *testing.T) { } func TestStoreLoadValidators(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - stateDB := dbm.NewMemDB() stateStore := sm.NewStore(stateDB) - val, _, err := factory.RandValidator(ctx, true, 10) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + val, _, err := factory.Validator(ctx, 10+int64(rand.Uint32())) require.NoError(t, err) - val2, _, err := factory.RandValidator(ctx, true, 10) + val2, _, err := factory.Validator(ctx, 10+int64(rand.Uint32())) require.NoError(t, err) - val3, _, err := factory.RandValidator(ctx, true, 10) + val3, _, err := factory.Validator(ctx, 10+int64(rand.Uint32())) require.NoError(t, err) vals := types.NewValidatorSet([]*types.Validator{val, val2, val3}) diff --git a/internal/state/validation.go b/internal/state/validation.go index fbd285f8a..8bec8f119 100644 --- a/internal/state/validation.go +++ b/internal/state/validation.go @@ -114,13 +114,6 @@ func validateBlock(state State, block *types.Block) error { state.LastBlockTime, ) } - medianTime := MedianTime(block.LastCommit, state.LastValidators) - if !block.Time.Equal(medianTime) { - return fmt.Errorf("invalid block time. Expected %v, got %v", - medianTime, - block.Time, - ) - } case block.Height == state.InitialHeight: genesisTime := state.LastBlockTime diff --git a/internal/state/validation_test.go b/internal/state/validation_test.go index 411740c37..3fb723a27 100644 --- a/internal/state/validation_test.go +++ b/internal/state/validation_test.go @@ -64,7 +64,6 @@ func TestValidateBlockHeader(t *testing.T) { {"ChainID wrong", func(block *types.Block) { block.ChainID = "not-the-real-one" }}, {"Height wrong", func(block *types.Block) { block.Height += 10 }}, {"Time wrong", func(block *types.Block) { block.Time = block.Time.Add(-time.Second * 1) }}, - {"Time wrong 2", func(block *types.Block) { block.Time = block.Time.Add(time.Second * 1) }}, {"LastBlockID wrong", func(block *types.Block) { block.LastBlockID.PartSetHeader.Total += 10 }}, {"LastCommitHash wrong", func(block *types.Block) { block.LastCommitHash = wrongHash }}, diff --git a/internal/statesync/block_queue_test.go b/internal/statesync/block_queue_test.go index 2ce2b8a68..364a7f5b2 100644 --- a/internal/statesync/block_queue_test.go +++ b/internal/statesync/block_queue_test.go @@ -294,7 +294,7 @@ loop: func mockLBResp(ctx context.Context, t *testing.T, peer types.NodeID, height int64, time time.Time) lightBlockResponse { t.Helper() - vals, pv := factory.RandValidatorSet(ctx, t, 3, 10) + vals, pv := factory.ValidatorSet(ctx, t, 3, 10) _, _, lb := mockLB(ctx, t, height, time, factory.MakeBlockID(), vals, pv) return lightBlockResponse{ block: lb, diff --git a/internal/statesync/reactor_test.go b/internal/statesync/reactor_test.go index ffe5de096..c1ca87b2c 100644 --- a/internal/statesync/reactor_test.go +++ b/internal/statesync/reactor_test.go @@ -442,7 +442,7 @@ func TestReactor_LightBlockResponse(t *testing.T) { h := factory.MakeHeader(t, &types.Header{}) h.Height = height blockID := factory.MakeBlockIDWithHash(h.Hash()) - vals, pv := factory.RandValidatorSet(ctx, t, 1, 10) + vals, pv := factory.ValidatorSet(ctx, t, 1, 10) vote, err := factory.MakeVote(ctx, pv[0], h.ChainID, 0, h.Height, 0, 2, blockID, factory.DefaultTestTime) require.NoError(t, err) @@ -733,7 +733,7 @@ func handleLightBlockRequests( } else { switch errorCount % 3 { case 0: // send a different block - vals, pv := factory.RandValidatorSet(ctx, t, 3, 10) + vals, pv := factory.ValidatorSet(ctx, t, 3, 10) _, _, lb := mockLB(ctx, t, int64(msg.Height), factory.DefaultTestTime, factory.MakeBlockID(), vals, pv) differntLB, err := lb.ToProto() require.NoError(t, err) @@ -802,7 +802,7 @@ func buildLightBlockChain(ctx context.Context, t *testing.T, fromHeight, toHeigh chain := make(map[int64]*types.LightBlock, toHeight-fromHeight) lastBlockID := factory.MakeBlockID() blockTime := startTime.Add(time.Duration(fromHeight-toHeight) * time.Minute) - vals, pv := factory.RandValidatorSet(ctx, t, 3, 10) + vals, pv := factory.ValidatorSet(ctx, t, 3, 10) for height := fromHeight; height < toHeight; height++ { vals, pv, chain[height] = mockLB(ctx, t, height, blockTime, lastBlockID, vals, pv) lastBlockID = factory.MakeBlockIDWithHash(chain[height].Header.Hash()) @@ -822,7 +822,7 @@ func mockLB(ctx context.Context, t *testing.T, height int64, time time.Time, las }) header.Version.App = testAppVersion - nextVals, nextPrivVals := factory.RandValidatorSet(ctx, t, 3, 10) + nextVals, nextPrivVals := factory.ValidatorSet(ctx, t, 3, 10) header.ValidatorsHash = currentVals.Hash() header.NextValidatorsHash = nextVals.Hash() header.ConsensusHash = types.DefaultConsensusParams().HashConsensusParams() diff --git a/internal/test/factory/block.go b/internal/test/factory/block.go index 654572ddf..3fd34cdc5 100644 --- a/internal/test/factory/block.go +++ b/internal/test/factory/block.go @@ -51,7 +51,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/internal/test/factory/genesis.go b/internal/test/factory/genesis.go index e03a7ea90..c49f9fce8 100644 --- a/internal/test/factory/genesis.go +++ b/internal/test/factory/genesis.go @@ -1,36 +1,33 @@ package factory import ( - "context" - "sort" - "testing" + "time" - "github.com/stretchr/testify/require" - "github.com/tendermint/tendermint/config" - tmtime "github.com/tendermint/tendermint/libs/time" + cfg "github.com/tendermint/tendermint/config" "github.com/tendermint/tendermint/types" ) -func RandGenesisDoc(ctx context.Context, t *testing.T, cfg *config.Config, numValidators int, randPower bool, minPower int64) (*types.GenesisDoc, []types.PrivValidator) { - t.Helper() +func GenesisDoc( + config *cfg.Config, + time time.Time, + validators []*types.Validator, + consensusParams *types.ConsensusParams, +) *types.GenesisDoc { - validators := make([]types.GenesisValidator, numValidators) - privValidators := make([]types.PrivValidator, numValidators) - for i := 0; i < numValidators; i++ { - val, privVal, err := RandValidator(ctx, randPower, minPower) - require.NoError(t, err) - validators[i] = types.GenesisValidator{ - PubKey: val.PubKey, - Power: val.VotingPower, + genesisValidators := make([]types.GenesisValidator, len(validators)) + + for i := range validators { + genesisValidators[i] = types.GenesisValidator{ + Power: validators[i].VotingPower, + PubKey: validators[i].PubKey, } - privValidators[i] = privVal } - sort.Sort(types.PrivValidatorsByAddress(privValidators)) return &types.GenesisDoc{ - GenesisTime: tmtime.Now(), - InitialHeight: 1, - ChainID: cfg.ChainID(), - Validators: validators, - }, privValidators + GenesisTime: time, + InitialHeight: 1, + ChainID: config.ChainID(), + Validators: genesisValidators, + ConsensusParams: consensusParams, + } } diff --git a/internal/test/factory/validator.go b/internal/test/factory/validator.go index abdb263f6..383ba7536 100644 --- a/internal/test/factory/validator.go +++ b/internal/test/factory/validator.go @@ -2,8 +2,6 @@ package factory import ( "context" - "fmt" - "math/rand" "sort" "testing" @@ -11,23 +9,18 @@ import ( "github.com/tendermint/tendermint/types" ) -func RandValidator(ctx context.Context, randPower bool, minPower int64) (*types.Validator, types.PrivValidator, error) { +func Validator(ctx context.Context, votingPower int64) (*types.Validator, types.PrivValidator, error) { privVal := types.NewMockPV() - votePower := minPower - if randPower { - // nolint:gosec // G404: Use of weak random number generator - votePower += int64(rand.Uint32()) - } pubKey, err := privVal.GetPubKey(ctx) if err != nil { - return nil, nil, fmt.Errorf("could not retrieve public key: %w", err) + return nil, nil, err } - val := types.NewValidator(pubKey, votePower) - return val, privVal, err + val := types.NewValidator(pubKey, votingPower) + return val, privVal, nil } -func RandValidatorSet(ctx context.Context, t *testing.T, numValidators int, votingPower int64) (*types.ValidatorSet, []types.PrivValidator) { +func ValidatorSet(ctx context.Context, t *testing.T, numValidators int, votingPower int64) (*types.ValidatorSet, []types.PrivValidator) { var ( valz = make([]*types.Validator, numValidators) privValidators = make([]types.PrivValidator, numValidators) @@ -35,7 +28,7 @@ func RandValidatorSet(ctx context.Context, t *testing.T, numValidators int, voti t.Helper() for i := 0; i < numValidators; i++ { - val, privValidator, err := RandValidator(ctx, false, votingPower) + val, privValidator, err := Validator(ctx, votingPower) require.NoError(t, err) valz[i] = val privValidators[i] = privValidator diff --git a/libs/time/mocks/source.go b/libs/time/mocks/source.go new file mode 100644 index 000000000..a8e49b314 --- /dev/null +++ b/libs/time/mocks/source.go @@ -0,0 +1,28 @@ +// Code generated by mockery. DO NOT EDIT. + +package mocks + +import ( + time "time" + + mock "github.com/stretchr/testify/mock" +) + +// Source is an autogenerated mock type for the Source type +type Source struct { + mock.Mock +} + +// Now provides a mock function with given fields: +func (_m *Source) Now() time.Time { + ret := _m.Called() + + var r0 time.Time + if rf, ok := ret.Get(0).(func() time.Time); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(time.Time) + } + + return r0 +} diff --git a/libs/time/time.go b/libs/time/time.go index 786f9bbb4..7ab45d8f1 100644 --- a/libs/time/time.go +++ b/libs/time/time.go @@ -15,3 +15,17 @@ func Now() time.Time { func Canonical(t time.Time) time.Time { return t.Round(0).UTC() } + +//go:generate ../../scripts/mockery_generate.sh Source + +// Source is an interface that defines a way to fetch the current time. +type Source interface { + Now() time.Time +} + +// DefaultSource implements the Source interface using the system clock provided by the standard library. +type DefaultSource struct{} + +func (DefaultSource) Now() time.Time { + return Now() +} diff --git a/light/client_test.go b/light/client_test.go index 09abe6d9e..bc585f7b8 100644 --- a/light/client_test.go +++ b/light/client_test.go @@ -37,6 +37,8 @@ func init() { } func TestClient(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() var ( keys = genPrivKeys(4) vals = keys.ToValidators(20, 10) @@ -120,12 +122,9 @@ func TestClient(t *testing.T) { } }) t.Run("SequentialVerification", func(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - newKeys := genPrivKeys(4) newVals := newKeys.ToValidators(10, 1) - differentVals, _ := factory.RandValidatorSet(ctx, t, 10, 100) + differentVals, _ := factory.ValidatorSet(ctx, t, 10, 100) testCases := []struct { name string @@ -943,7 +942,7 @@ func TestClient(t *testing.T) { logger := log.NewTestingLogger(t) - differentVals, _ := factory.RandValidatorSet(ctx, t, 10, 100) + differentVals, _ := factory.ValidatorSet(ctx, t, 10, 100) mockBadValSetNode := mockNodeFromHeadersAndVals( map[int64]*types.SignedHeader{ 1: h1, diff --git a/light/store/db/db_test.go b/light/store/db/db_test.go index b758ea026..7069eb11d 100644 --- a/light/store/db/db_test.go +++ b/light/store/db/db_test.go @@ -19,11 +19,10 @@ import ( ) func TestLast_FirstLightBlockHeight(t *testing.T) { + dbStore := New(dbm.NewMemDB()) ctx, cancel := context.WithCancel(context.Background()) defer cancel() - dbStore := New(dbm.NewMemDB()) - // Empty store height, err := dbStore.LastLightBlockHeight() require.NoError(t, err) @@ -48,7 +47,6 @@ func TestLast_FirstLightBlockHeight(t *testing.T) { func Test_SaveLightBlock(t *testing.T) { dbStore := New(dbm.NewMemDB()) - ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -81,15 +79,14 @@ func Test_SaveLightBlock(t *testing.T) { func Test_LightBlockBefore(t *testing.T) { dbStore := New(dbm.NewMemDB()) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() assert.Panics(t, func() { _, _ = dbStore.LightBlockBefore(0) _, _ = dbStore.LightBlockBefore(100) }) - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - err := dbStore.SaveLightBlock(randLightBlock(ctx, t, int64(2))) require.NoError(t, err) @@ -105,7 +102,6 @@ func Test_LightBlockBefore(t *testing.T) { func Test_Prune(t *testing.T) { dbStore := New(dbm.NewMemDB()) - ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -200,7 +196,7 @@ func Test_Concurrency(t *testing.T) { func randLightBlock(ctx context.Context, t *testing.T, height int64) *types.LightBlock { t.Helper() - vals, _ := factory.RandValidatorSet(ctx, t, 2, 1) + vals, _ := factory.ValidatorSet(ctx, t, 2, 1) return &types.LightBlock{ SignedHeader: &types.SignedHeader{ Header: &types.Header{ diff --git a/node/node_test.go b/node/node_test.go index c563906b3..99ea4aa50 100644 --- a/node/node_test.go +++ b/node/node_test.go @@ -728,7 +728,8 @@ func loadStatefromGenesis(ctx context.Context, t *testing.T) sm.State { require.NoError(t, err) require.True(t, loadedState.IsEmpty()) - genDoc, _ := factory.RandGenesisDoc(ctx, t, cfg, 0, false, 10) + valSet, _ := factory.ValidatorSet(ctx, t, 0, 10) + genDoc := factory.GenesisDoc(cfg, time.Now(), valSet.Validators, nil) state, err := loadStateFromDBOrGenesisDocProvider( stateStore, diff --git a/proto/tendermint/statesync/message_test.go b/proto/tendermint/statesync/message_test.go index cccd25766..a0b241615 100644 --- a/proto/tendermint/statesync/message_test.go +++ b/proto/tendermint/statesync/message_test.go @@ -7,9 +7,9 @@ import ( "github.com/gogo/protobuf/proto" "github.com/stretchr/testify/require" + "github.com/tendermint/tendermint/crypto/ed25519" ssproto "github.com/tendermint/tendermint/proto/tendermint/statesync" tmproto "github.com/tendermint/tendermint/proto/tendermint/types" - "github.com/tendermint/tendermint/types" ) func TestValidateMsg(t *testing.T) { @@ -186,10 +186,30 @@ func TestStateSyncVectors(t *testing.T) { { "ParamsResponse", &ssproto.ParamsResponse{ - Height: 9001, - ConsensusParams: types.DefaultConsensusParams().ToProto(), + Height: 9001, + ConsensusParams: tmproto.ConsensusParams{ + Block: &tmproto.BlockParams{ + MaxBytes: 10, + MaxGas: 20, + }, + Evidence: &tmproto.EvidenceParams{ + MaxAgeNumBlocks: 10, + MaxAgeDuration: 300, + MaxBytes: 100, + }, + Validator: &tmproto.ValidatorParams{ + PubKeyTypes: []string{ed25519.KeyType}, + }, + Version: &tmproto.VersionParams{ + AppVersion: 11, + }, + Synchrony: &tmproto.SynchronyParams{ + MessageDelay: 550, + Precision: 90, + }, + }, }, - "423408a946122f0a10088080c00a10ffffffffffffffffff01120e08a08d0612040880c60a188080401a090a07656432353531392200", + "423008a946122b0a04080a10141209080a120310ac0218641a090a07656432353531392202080b2a090a0310a6041202105a", }, } diff --git a/proto/tendermint/types/params.pb.go b/proto/tendermint/types/params.pb.go index 3bdf4cb6f..32689932c 100644 --- a/proto/tendermint/types/params.pb.go +++ b/proto/tendermint/types/params.pb.go @@ -34,6 +34,7 @@ type ConsensusParams struct { Evidence *EvidenceParams `protobuf:"bytes,2,opt,name=evidence,proto3" json:"evidence,omitempty"` Validator *ValidatorParams `protobuf:"bytes,3,opt,name=validator,proto3" json:"validator,omitempty"` Version *VersionParams `protobuf:"bytes,4,opt,name=version,proto3" json:"version,omitempty"` + Synchrony *SynchronyParams `protobuf:"bytes,5,opt,name=synchrony,proto3" json:"synchrony,omitempty"` } func (m *ConsensusParams) Reset() { *m = ConsensusParams{} } @@ -97,6 +98,13 @@ func (m *ConsensusParams) GetVersion() *VersionParams { return nil } +func (m *ConsensusParams) GetSynchrony() *SynchronyParams { + if m != nil { + return m.Synchrony + } + return nil +} + // BlockParams contains limits on the block size. type BlockParams struct { // Max block size, in bytes. @@ -373,6 +381,58 @@ func (m *HashedParams) GetBlockMaxGas() int64 { return 0 } +type SynchronyParams struct { + MessageDelay time.Duration `protobuf:"bytes,1,opt,name=message_delay,json=messageDelay,proto3,stdduration" json:"message_delay"` + Precision time.Duration `protobuf:"bytes,2,opt,name=precision,proto3,stdduration" json:"precision"` +} + +func (m *SynchronyParams) Reset() { *m = SynchronyParams{} } +func (m *SynchronyParams) String() string { return proto.CompactTextString(m) } +func (*SynchronyParams) ProtoMessage() {} +func (*SynchronyParams) Descriptor() ([]byte, []int) { + return fileDescriptor_e12598271a686f57, []int{6} +} +func (m *SynchronyParams) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SynchronyParams) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_SynchronyParams.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *SynchronyParams) XXX_Merge(src proto.Message) { + xxx_messageInfo_SynchronyParams.Merge(m, src) +} +func (m *SynchronyParams) XXX_Size() int { + return m.Size() +} +func (m *SynchronyParams) XXX_DiscardUnknown() { + xxx_messageInfo_SynchronyParams.DiscardUnknown(m) +} + +var xxx_messageInfo_SynchronyParams proto.InternalMessageInfo + +func (m *SynchronyParams) GetMessageDelay() time.Duration { + if m != nil { + return m.MessageDelay + } + return 0 +} + +func (m *SynchronyParams) GetPrecision() time.Duration { + if m != nil { + return m.Precision + } + return 0 +} + func init() { proto.RegisterType((*ConsensusParams)(nil), "tendermint.types.ConsensusParams") proto.RegisterType((*BlockParams)(nil), "tendermint.types.BlockParams") @@ -380,44 +440,49 @@ func init() { proto.RegisterType((*ValidatorParams)(nil), "tendermint.types.ValidatorParams") proto.RegisterType((*VersionParams)(nil), "tendermint.types.VersionParams") proto.RegisterType((*HashedParams)(nil), "tendermint.types.HashedParams") + proto.RegisterType((*SynchronyParams)(nil), "tendermint.types.SynchronyParams") } func init() { proto.RegisterFile("tendermint/types/params.proto", fileDescriptor_e12598271a686f57) } var fileDescriptor_e12598271a686f57 = []byte{ - // 498 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x6c, 0x93, 0xc1, 0x6a, 0xd4, 0x40, - 0x1c, 0xc6, 0x77, 0x9a, 0xda, 0xee, 0xfe, 0xe3, 0x76, 0xcb, 0x20, 0x18, 0x2b, 0xcd, 0xae, 0x39, - 0x48, 0x41, 0x48, 0xc4, 0x22, 0x22, 0x08, 0xe2, 0x56, 0xa9, 0x20, 0x15, 0x09, 0xea, 0xa1, 0x97, - 0x30, 0xd9, 0x8c, 0x69, 0xe8, 0x4e, 0x66, 0xc8, 0x24, 0xcb, 0xee, 0xcd, 0x47, 0xf0, 0xe8, 0x23, - 0xe8, 0x9b, 0xf4, 0xd8, 0xa3, 0x27, 0x95, 0xdd, 0x17, 0x91, 0x4c, 0x32, 0xa6, 0x9b, 0xf6, 0x36, - 0x33, 0xdf, 0xef, 0x9b, 0xe1, 0xfb, 0x86, 0x3f, 0xec, 0xe7, 0x34, 0x8d, 0x68, 0xc6, 0x92, 0x34, - 0xf7, 0xf2, 0x85, 0xa0, 0xd2, 0x13, 0x24, 0x23, 0x4c, 0xba, 0x22, 0xe3, 0x39, 0xc7, 0xbb, 0x8d, - 0xec, 0x2a, 0x79, 0xef, 0x4e, 0xcc, 0x63, 0xae, 0x44, 0xaf, 0x5c, 0x55, 0xdc, 0x9e, 0x1d, 0x73, - 0x1e, 0x4f, 0xa9, 0xa7, 0x76, 0x61, 0xf1, 0xc5, 0x8b, 0x8a, 0x8c, 0xe4, 0x09, 0x4f, 0x2b, 0xdd, - 0xf9, 0xba, 0x01, 0x83, 0x23, 0x9e, 0x4a, 0x9a, 0xca, 0x42, 0x7e, 0x50, 0x2f, 0xe0, 0x43, 0xb8, - 0x15, 0x4e, 0xf9, 0xe4, 0xdc, 0x42, 0x23, 0x74, 0x60, 0x3e, 0xd9, 0x77, 0xdb, 0x6f, 0xb9, 0xe3, - 0x52, 0xae, 0x68, 0xbf, 0x62, 0xf1, 0x0b, 0xe8, 0xd2, 0x59, 0x12, 0xd1, 0x74, 0x42, 0xad, 0x0d, - 0xe5, 0x1b, 0x5d, 0xf7, 0xbd, 0xa9, 0x89, 0xda, 0xfa, 0xdf, 0x81, 0x5f, 0x42, 0x6f, 0x46, 0xa6, - 0x49, 0x44, 0x72, 0x9e, 0x59, 0x86, 0xb2, 0x3f, 0xb8, 0x6e, 0xff, 0xac, 0x91, 0xda, 0xdf, 0x78, - 0xf0, 0x73, 0xd8, 0x9e, 0xd1, 0x4c, 0x26, 0x3c, 0xb5, 0x36, 0x95, 0x7d, 0x78, 0x83, 0xbd, 0x02, - 0x6a, 0xb3, 0xe6, 0x9d, 0x23, 0x30, 0xaf, 0xe4, 0xc1, 0xf7, 0xa1, 0xc7, 0xc8, 0x3c, 0x08, 0x17, - 0x39, 0x95, 0xaa, 0x01, 0xc3, 0xef, 0x32, 0x32, 0x1f, 0x97, 0x7b, 0x7c, 0x17, 0xb6, 0x4b, 0x31, - 0x26, 0x52, 0x85, 0x34, 0xfc, 0x2d, 0x46, 0xe6, 0xc7, 0x44, 0x3a, 0x3f, 0x11, 0xec, 0xac, 0xa7, - 0xc3, 0x8f, 0x00, 0x97, 0x2c, 0x89, 0x69, 0x90, 0x16, 0x2c, 0x50, 0x35, 0xe9, 0x1b, 0x07, 0x8c, - 0xcc, 0x5f, 0xc5, 0xf4, 0x7d, 0xc1, 0xd4, 0xd3, 0x12, 0x9f, 0xc0, 0xae, 0x86, 0xf5, 0x0f, 0xd5, - 0x35, 0xde, 0x73, 0xab, 0x2f, 0x74, 0xf5, 0x17, 0xba, 0xaf, 0x6b, 0x60, 0xdc, 0xbd, 0xf8, 0x3d, - 0xec, 0x7c, 0xff, 0x33, 0x44, 0xfe, 0x4e, 0x75, 0x9f, 0x56, 0xd6, 0x43, 0x18, 0xeb, 0x21, 0x9c, - 0xa7, 0x30, 0x68, 0x35, 0x89, 0x1d, 0xe8, 0x8b, 0x22, 0x0c, 0xce, 0xe9, 0x22, 0x50, 0x5d, 0x59, - 0x68, 0x64, 0x1c, 0xf4, 0x7c, 0x53, 0x14, 0xe1, 0x3b, 0xba, 0xf8, 0x58, 0x1e, 0x39, 0x8f, 0xa1, - 0xbf, 0xd6, 0x20, 0x1e, 0x82, 0x49, 0x84, 0x08, 0x74, 0xef, 0x65, 0xb2, 0x4d, 0x1f, 0x88, 0x10, - 0x35, 0xe6, 0x9c, 0xc2, 0xed, 0xb7, 0x44, 0x9e, 0xd1, 0xa8, 0x36, 0x3c, 0x84, 0x81, 0x6a, 0x21, - 0x68, 0x17, 0xdc, 0x57, 0xc7, 0x27, 0xba, 0x65, 0x07, 0xfa, 0x0d, 0xd7, 0x74, 0x6d, 0x6a, 0xea, - 0x98, 0xc8, 0xf1, 0xa7, 0x1f, 0x4b, 0x1b, 0x5d, 0x2c, 0x6d, 0x74, 0xb9, 0xb4, 0xd1, 0xdf, 0xa5, - 0x8d, 0xbe, 0xad, 0xec, 0xce, 0xe5, 0xca, 0xee, 0xfc, 0x5a, 0xd9, 0x9d, 0xd3, 0x67, 0x71, 0x92, - 0x9f, 0x15, 0xa1, 0x3b, 0xe1, 0xcc, 0xbb, 0x3a, 0x48, 0xcd, 0xb2, 0x9a, 0x94, 0xf6, 0x90, 0x85, - 0x5b, 0xea, 0xfc, 0xf0, 0x5f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x18, 0x54, 0x4f, 0xe1, 0x7f, 0x03, - 0x00, 0x00, + // 561 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x94, 0x4d, 0x6b, 0xd4, 0x40, + 0x18, 0xc7, 0x37, 0xdd, 0xbe, 0xec, 0x3e, 0xdb, 0xed, 0x96, 0x41, 0x30, 0x56, 0x9a, 0x5d, 0x73, + 0x90, 0x82, 0x90, 0x88, 0x45, 0x44, 0x10, 0xa4, 0xdb, 0x8a, 0x05, 0xa9, 0x48, 0x7c, 0x39, 0xf4, + 0x12, 0x26, 0xbb, 0x63, 0x36, 0x74, 0x93, 0x19, 0x32, 0xc9, 0xb2, 0xf9, 0x16, 0x1e, 0x3d, 0x79, + 0xd6, 0x8f, 0xe1, 0xad, 0xc7, 0x1e, 0x3d, 0xa9, 0xec, 0x7e, 0x11, 0x99, 0xc9, 0x4c, 0xd3, 0xdd, + 0x2a, 0xd8, 0x5b, 0x32, 0xcf, 0xef, 0x97, 0x87, 0xf9, 0x3f, 0x93, 0x81, 0xdd, 0x8c, 0x24, 0x43, + 0x92, 0xc6, 0x51, 0x92, 0xb9, 0x59, 0xc1, 0x08, 0x77, 0x19, 0x4e, 0x71, 0xcc, 0x1d, 0x96, 0xd2, + 0x8c, 0xa2, 0xed, 0xaa, 0xec, 0xc8, 0xf2, 0xce, 0xad, 0x90, 0x86, 0x54, 0x16, 0x5d, 0xf1, 0x54, + 0x72, 0x3b, 0x56, 0x48, 0x69, 0x38, 0x26, 0xae, 0x7c, 0x0b, 0xf2, 0x8f, 0xee, 0x30, 0x4f, 0x71, + 0x16, 0xd1, 0xa4, 0xac, 0xdb, 0xdf, 0x57, 0xa0, 0x73, 0x48, 0x13, 0x4e, 0x12, 0x9e, 0xf3, 0x37, + 0xb2, 0x03, 0xda, 0x87, 0xb5, 0x60, 0x4c, 0x07, 0x67, 0xa6, 0xd1, 0x33, 0xf6, 0x5a, 0x8f, 0x76, + 0x9d, 0xe5, 0x5e, 0x4e, 0x5f, 0x94, 0x4b, 0xda, 0x2b, 0x59, 0xf4, 0x0c, 0x1a, 0x64, 0x12, 0x0d, + 0x49, 0x32, 0x20, 0xe6, 0x8a, 0xf4, 0x7a, 0xd7, 0xbd, 0x17, 0x8a, 0x50, 0xea, 0xa5, 0x81, 0x9e, + 0x43, 0x73, 0x82, 0xc7, 0xd1, 0x10, 0x67, 0x34, 0x35, 0xeb, 0x52, 0xbf, 0x77, 0x5d, 0xff, 0xa0, + 0x11, 0xe5, 0x57, 0x0e, 0x7a, 0x0a, 0x1b, 0x13, 0x92, 0xf2, 0x88, 0x26, 0xe6, 0xaa, 0xd4, 0xbb, + 0x7f, 0xd1, 0x4b, 0x40, 0xc9, 0x9a, 0x17, 0xbd, 0x79, 0x91, 0x0c, 0x46, 0x29, 0x4d, 0x0a, 0x73, + 0xed, 0x5f, 0xbd, 0xdf, 0x6a, 0x44, 0xf7, 0xbe, 0x74, 0xec, 0x43, 0x68, 0x5d, 0x09, 0x04, 0xdd, + 0x85, 0x66, 0x8c, 0xa7, 0x7e, 0x50, 0x64, 0x84, 0xcb, 0x08, 0xeb, 0x5e, 0x23, 0xc6, 0xd3, 0xbe, + 0x78, 0x47, 0xb7, 0x61, 0x43, 0x14, 0x43, 0xcc, 0x65, 0x4a, 0x75, 0x6f, 0x3d, 0xc6, 0xd3, 0x97, + 0x98, 0xdb, 0xdf, 0x0c, 0xd8, 0x5a, 0x8c, 0x07, 0x3d, 0x00, 0x24, 0x58, 0x1c, 0x12, 0x3f, 0xc9, + 0x63, 0x5f, 0xe6, 0xac, 0xbf, 0xd8, 0x89, 0xf1, 0xf4, 0x20, 0x24, 0xaf, 0xf3, 0x58, 0xb6, 0xe6, + 0xe8, 0x04, 0xb6, 0x35, 0xac, 0x47, 0xac, 0xe6, 0x70, 0xc7, 0x29, 0xcf, 0x80, 0xa3, 0xcf, 0x80, + 0x73, 0xa4, 0x80, 0x7e, 0xe3, 0xfc, 0x67, 0xb7, 0xf6, 0xf9, 0x57, 0xd7, 0xf0, 0xb6, 0xca, 0xef, + 0xe9, 0xca, 0xe2, 0x26, 0xea, 0x8b, 0x9b, 0xb0, 0x1f, 0x43, 0x67, 0x69, 0x14, 0xc8, 0x86, 0x36, + 0xcb, 0x03, 0xff, 0x8c, 0x14, 0xbe, 0xcc, 0xcb, 0x34, 0x7a, 0xf5, 0xbd, 0xa6, 0xd7, 0x62, 0x79, + 0xf0, 0x8a, 0x14, 0xef, 0xc4, 0x92, 0xfd, 0x10, 0xda, 0x0b, 0x23, 0x40, 0x5d, 0x68, 0x61, 0xc6, + 0x7c, 0x3d, 0x38, 0xb1, 0xb3, 0x55, 0x0f, 0x30, 0x63, 0x0a, 0xb3, 0x4f, 0x61, 0xf3, 0x18, 0xf3, + 0x11, 0x19, 0x2a, 0xe1, 0x3e, 0x74, 0x64, 0x0a, 0xfe, 0x72, 0xc0, 0x6d, 0xb9, 0x7c, 0xa2, 0x53, + 0xb6, 0xa1, 0x5d, 0x71, 0x55, 0xd6, 0x2d, 0x4d, 0x89, 0xc0, 0xbf, 0x18, 0xd0, 0x59, 0x1a, 0x2a, + 0x3a, 0x86, 0x76, 0x4c, 0x38, 0x97, 0x21, 0x92, 0x31, 0x2e, 0xd4, 0x1f, 0xf0, 0x5f, 0x09, 0x6e, + 0x2a, 0xf3, 0x48, 0x88, 0xe8, 0x00, 0x9a, 0x2c, 0x25, 0x83, 0x88, 0xdf, 0x70, 0x0e, 0x95, 0xd5, + 0x7f, 0xff, 0x75, 0x66, 0x19, 0xe7, 0x33, 0xcb, 0xb8, 0x98, 0x59, 0xc6, 0xef, 0x99, 0x65, 0x7c, + 0x9a, 0x5b, 0xb5, 0x8b, 0xb9, 0x55, 0xfb, 0x31, 0xb7, 0x6a, 0xa7, 0x4f, 0xc2, 0x28, 0x1b, 0xe5, + 0x81, 0x33, 0xa0, 0xb1, 0x7b, 0xf5, 0xaa, 0xa8, 0x1e, 0xcb, 0xbb, 0x60, 0xf9, 0x1a, 0x09, 0xd6, + 0xe5, 0xfa, 0xfe, 0x9f, 0x00, 0x00, 0x00, 0xff, 0xff, 0xcb, 0x26, 0x8a, 0x0b, 0x61, 0x04, 0x00, + 0x00, } func (this *ConsensusParams) Equal(that interface{}) bool { @@ -451,6 +516,9 @@ func (this *ConsensusParams) Equal(that interface{}) bool { if !this.Version.Equal(that1.Version) { return false } + if !this.Synchrony.Equal(that1.Synchrony) { + return false + } return true } func (this *BlockParams) Equal(that interface{}) bool { @@ -590,6 +658,33 @@ func (this *HashedParams) Equal(that interface{}) bool { } return true } +func (this *SynchronyParams) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*SynchronyParams) + if !ok { + that2, ok := that.(SynchronyParams) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.MessageDelay != that1.MessageDelay { + return false + } + if this.Precision != that1.Precision { + return false + } + return true +} func (m *ConsensusParams) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -610,6 +705,18 @@ func (m *ConsensusParams) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.Synchrony != nil { + { + size, err := m.Synchrony.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintParams(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x2a + } if m.Version != nil { { size, err := m.Version.MarshalToSizedBuffer(dAtA[:i]) @@ -719,12 +826,12 @@ func (m *EvidenceParams) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x18 } - n5, err5 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.MaxAgeDuration, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.MaxAgeDuration):]) - if err5 != nil { - return 0, err5 + n6, err6 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.MaxAgeDuration, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.MaxAgeDuration):]) + if err6 != nil { + return 0, err6 } - i -= n5 - i = encodeVarintParams(dAtA, i, uint64(n5)) + i -= n6 + i = encodeVarintParams(dAtA, i, uint64(n6)) i-- dAtA[i] = 0x12 if m.MaxAgeNumBlocks != 0 { @@ -828,6 +935,45 @@ func (m *HashedParams) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *SynchronyParams) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SynchronyParams) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SynchronyParams) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + n7, err7 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.Precision, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.Precision):]) + if err7 != nil { + return 0, err7 + } + i -= n7 + i = encodeVarintParams(dAtA, i, uint64(n7)) + i-- + dAtA[i] = 0x12 + n8, err8 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.MessageDelay, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.MessageDelay):]) + if err8 != nil { + return 0, err8 + } + i -= n8 + i = encodeVarintParams(dAtA, i, uint64(n8)) + i-- + dAtA[i] = 0xa + return len(dAtA) - i, nil +} + func encodeVarintParams(dAtA []byte, offset int, v uint64) int { offset -= sovParams(v) base := offset @@ -861,6 +1007,10 @@ func (m *ConsensusParams) Size() (n int) { l = m.Version.Size() n += 1 + l + sovParams(uint64(l)) } + if m.Synchrony != nil { + l = m.Synchrony.Size() + n += 1 + l + sovParams(uint64(l)) + } return n } @@ -938,6 +1088,19 @@ func (m *HashedParams) Size() (n int) { return n } +func (m *SynchronyParams) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = github_com_gogo_protobuf_types.SizeOfStdDuration(m.MessageDelay) + n += 1 + l + sovParams(uint64(l)) + l = github_com_gogo_protobuf_types.SizeOfStdDuration(m.Precision) + n += 1 + l + sovParams(uint64(l)) + return n +} + func sovParams(x uint64) (n int) { return (math_bits.Len64(x|1) + 6) / 7 } @@ -1117,6 +1280,42 @@ func (m *ConsensusParams) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Synchrony", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowParams + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthParams + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthParams + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Synchrony == nil { + m.Synchrony = &SynchronyParams{} + } + if err := m.Synchrony.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipParams(dAtA[iNdEx:]) @@ -1586,6 +1785,122 @@ func (m *HashedParams) Unmarshal(dAtA []byte) error { } return nil } +func (m *SynchronyParams) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowParams + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SynchronyParams: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SynchronyParams: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MessageDelay", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowParams + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthParams + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthParams + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := github_com_gogo_protobuf_types.StdDurationUnmarshal(&m.MessageDelay, dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Precision", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowParams + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthParams + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthParams + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := github_com_gogo_protobuf_types.StdDurationUnmarshal(&m.Precision, dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipParams(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthParams + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func skipParams(dAtA []byte) (n int, err error) { l := len(dAtA) iNdEx := 0 diff --git a/test/e2e/runner/evidence.go b/test/e2e/runner/evidence.go index 8f8408ff1..06a76a79d 100644 --- a/test/e2e/runner/evidence.go +++ b/test/e2e/runner/evidence.go @@ -288,8 +288,9 @@ func makeBlockID(hash []byte, partSetSize uint32, partSetHash []byte) types.Bloc } } -func mutateValidatorSet(ctx context.Context, privVals []types.MockPV, vals *types.ValidatorSet) ([]types.PrivValidator, *types.ValidatorSet, error) { - newVal, newPrivVal, err := factory.RandValidator(ctx, false, 10) +func mutateValidatorSet(ctx context.Context, privVals []types.MockPV, vals *types.ValidatorSet, +) ([]types.PrivValidator, *types.ValidatorSet, error) { + newVal, newPrivVal, err := factory.Validator(ctx, 10) if err != nil { return nil, nil, err } diff --git a/third_party/proto/gogoproto/gogo.proto b/third_party/proto/gogoproto/gogo.proto index 27960ecfb..31c516cd0 100644 --- a/third_party/proto/gogoproto/gogo.proto +++ b/third_party/proto/gogoproto/gogo.proto @@ -144,4 +144,4 @@ extend google.protobuf.FieldOptions { optional bool wktpointer = 65012; optional string castrepeated = 65013; -} \ No newline at end of file +} diff --git a/types/block.go b/types/block.go index 241c360de..d674bd6b4 100644 --- a/types/block.go +++ b/types/block.go @@ -883,7 +883,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") } @@ -1204,8 +1204,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 01c3d851d..fc07ab477 100644 --- a/types/canonical.go +++ b/types/canonical.go @@ -21,7 +21,7 @@ func CanonicalizeBlockID(bid tmproto.BlockID) *tmproto.CanonicalBlockID { panic(err) } var cbid *tmproto.CanonicalBlockID - if rbid == nil || rbid.IsZero() { + if rbid == nil || rbid.IsNil() { cbid = nil } else { cbid = &tmproto.CanonicalBlockID{ diff --git a/types/events.go b/types/events.go index 6edf30420..263176083 100644 --- a/types/events.go +++ b/types/events.go @@ -38,7 +38,6 @@ const ( EventStateSyncStatusValue = "StateSyncStatus" EventTimeoutProposeValue = "TimeoutPropose" EventTimeoutWaitValue = "TimeoutWait" - EventUnlockValue = "Unlock" EventValidBlockValue = "ValidBlock" EventVoteValue = "Vote" ) @@ -259,7 +258,6 @@ var ( EventQueryTimeoutPropose = QueryForEvent(EventTimeoutProposeValue) EventQueryTimeoutWait = QueryForEvent(EventTimeoutWaitValue) EventQueryTx = QueryForEvent(EventTxValue) - EventQueryUnlock = QueryForEvent(EventUnlockValue) EventQueryValidatorSetUpdates = QueryForEvent(EventValidatorSetUpdatesValue) EventQueryValidBlock = QueryForEvent(EventValidBlockValue) EventQueryVote = QueryForEvent(EventVoteValue) diff --git a/types/genesis_test.go b/types/genesis_test.go index a82ce46ea..1045b7065 100644 --- a/types/genesis_test.go +++ b/types/genesis_test.go @@ -56,21 +56,26 @@ func TestGenesisBad(t *testing.T) { } } -func TestGenesisGood(t *testing.T) { +func TestBasicGenesisDoc(t *testing.T) { // test a good one by raw json genDocBytes := []byte( `{ "genesis_time": "0001-01-01T00:00:00Z", "chain_id": "test-chain-QDKdJr", "initial_height": "1000", - "consensus_params": null, "validators": [{ "pub_key":{"type":"tendermint/PubKeyEd25519","value":"AT/+aaL1eB0477Mud9JMm8Sh8BIvOYlPGC9KkIUmFaE="}, "power":"10", "name":"" }], "app_hash":"", - "app_state":{"account_owner": "Bob"} + "app_state":{"account_owner": "Bob"}, + "consensus_params": { + "synchrony": {"precision": "1", "message_delay": "10"}, + "validator": {"pub_key_types":["ed25519"]}, + "block": {"max_bytes": "100"}, + "evidence": {"max_age_num_blocks": "100", "max_age_duration": "10"} + } }`, ) _, err := GenesisDocFromJSON(genDocBytes) @@ -97,7 +102,7 @@ func TestGenesisGood(t *testing.T) { genDocBytes, err = json.Marshal(genDoc) assert.NoError(t, err, "error marshaling genDoc") genDoc, err = GenesisDocFromJSON(genDocBytes) - assert.NoError(t, err, "expected no error for valid genDoc json") + require.NoError(t, err, "expected no error for valid genDoc json") // test with invalid consensus params genDoc.ConsensusParams.Block.MaxBytes = 0 diff --git a/types/params.go b/types/params.go index 5ba065edf..3540805cc 100644 --- a/types/params.go +++ b/types/params.go @@ -41,6 +41,7 @@ type ConsensusParams struct { Evidence EvidenceParams `json:"evidence"` Validator ValidatorParams `json:"validator"` Version VersionParams `json:"version"` + Synchrony SynchronyParams `json:"synchrony"` } // HashedParams is a subset of ConsensusParams. @@ -75,6 +76,15 @@ type VersionParams struct { AppVersion uint64 `json:"app_version,string"` } +// SynchronyParams influence the validity of block timestamps. +// For more information on the relationship of the synchrony parameters to +// block validity, see the Proposer-Based Timestamps specification: +// https://github.com/tendermint/spec/blob/master/spec/consensus/proposer-based-timestamp/README.md +type SynchronyParams struct { + Precision time.Duration `json:"precision,string"` + MessageDelay time.Duration `json:"message_delay,string"` +} + // DefaultConsensusParams returns a default ConsensusParams. func DefaultConsensusParams() *ConsensusParams { return &ConsensusParams{ @@ -82,6 +92,7 @@ func DefaultConsensusParams() *ConsensusParams { Evidence: DefaultEvidenceParams(), Validator: DefaultValidatorParams(), Version: DefaultVersionParams(), + Synchrony: DefaultSynchronyParams(), } } @@ -116,6 +127,15 @@ func DefaultVersionParams() VersionParams { } } +func DefaultSynchronyParams() SynchronyParams { + // TODO(@wbanfield): Determine experimental values for these defaults + // https://github.com/tendermint/tendermint/issues/7202 + return SynchronyParams{ + Precision: 500 * time.Millisecond, + MessageDelay: 2 * time.Second, + } +} + func (val *ValidatorParams) IsValidPubkeyType(pubkeyType string) bool { for i := 0; i < len(val.PubKeyTypes); i++ { if val.PubKeyTypes[i] == pubkeyType { @@ -148,7 +168,7 @@ func (params ConsensusParams) ValidateConsensusParams() error { } if params.Evidence.MaxAgeDuration <= 0 { - return fmt.Errorf("evidence.MaxAgeDuration must be grater than 0 if provided, Got %v", + return fmt.Errorf("evidence.MaxAgeDuration must be greater than 0 if provided, Got %v", params.Evidence.MaxAgeDuration) } @@ -162,6 +182,16 @@ func (params ConsensusParams) ValidateConsensusParams() error { params.Evidence.MaxBytes) } + if params.Synchrony.MessageDelay <= 0 { + return fmt.Errorf("synchrony.MessageDelay must be greater than 0. Got: %d", + params.Synchrony.MessageDelay) + } + + if params.Synchrony.Precision <= 0 { + return fmt.Errorf("synchrony.Precision must be greater than 0. Got: %d", + params.Synchrony.Precision) + } + if len(params.Validator.PubKeyTypes) == 0 { return errors.New("len(Validator.PubKeyTypes) must be greater than 0") } @@ -205,6 +235,8 @@ func (params ConsensusParams) HashConsensusParams() []byte { func (params *ConsensusParams) Equals(params2 *ConsensusParams) bool { return params.Block == params2.Block && params.Evidence == params2.Evidence && + params.Version == params2.Version && + params.Synchrony == params2.Synchrony && tmstrings.StringSliceEqual(params.Validator.PubKeyTypes, params2.Validator.PubKeyTypes) } @@ -235,6 +267,10 @@ func (params ConsensusParams) UpdateConsensusParams(params2 *tmproto.ConsensusPa if params2.Version != nil { res.Version.AppVersion = params2.Version.AppVersion } + if params2.Synchrony != nil { + res.Synchrony.Precision = params2.Synchrony.Precision + res.Synchrony.MessageDelay = params2.Synchrony.MessageDelay + } return res } @@ -255,6 +291,10 @@ func (params *ConsensusParams) ToProto() tmproto.ConsensusParams { Version: &tmproto.VersionParams{ AppVersion: params.Version.AppVersion, }, + Synchrony: &tmproto.SynchronyParams{ + MessageDelay: params.Synchrony.MessageDelay, + Precision: params.Synchrony.Precision, + }, } } @@ -275,5 +315,9 @@ func ConsensusParamsFromProto(pbParams tmproto.ConsensusParams) ConsensusParams Version: VersionParams{ AppVersion: pbParams.Version.AppVersion, }, + Synchrony: SynchronyParams{ + MessageDelay: pbParams.Synchrony.MessageDelay, + Precision: pbParams.Synchrony.Precision, + }, } } diff --git a/types/params_test.go b/types/params_test.go index 8b69a81fc..0aaf1e2b9 100644 --- a/types/params_test.go +++ b/types/params_test.go @@ -23,23 +23,140 @@ func TestConsensusParamsValidation(t *testing.T) { valid bool }{ // test block params - 0: {makeParams(1, 0, 2, 0, valEd25519), true}, - 1: {makeParams(0, 0, 2, 0, valEd25519), false}, - 2: {makeParams(47*1024*1024, 0, 2, 0, valEd25519), true}, - 3: {makeParams(10, 0, 2, 0, valEd25519), true}, - 4: {makeParams(100*1024*1024, 0, 2, 0, valEd25519), true}, - 5: {makeParams(101*1024*1024, 0, 2, 0, valEd25519), false}, - 6: {makeParams(1024*1024*1024, 0, 2, 0, valEd25519), false}, - 7: {makeParams(1024*1024*1024, 0, -1, 0, valEd25519), false}, + { + params: makeParams(makeParamsArgs{ + blockBytes: 1, + evidenceAge: 2, + precision: 1, + messageDelay: 1}), + valid: true, + }, + { + params: makeParams(makeParamsArgs{ + blockBytes: 0, + evidenceAge: 2, + precision: 1, + messageDelay: 1}), + valid: false, + }, + { + params: makeParams(makeParamsArgs{ + blockBytes: 47 * 1024 * 1024, + evidenceAge: 2, + precision: 1, + messageDelay: 1}), + valid: true, + }, + { + params: makeParams(makeParamsArgs{ + blockBytes: 10, + evidenceAge: 2, + precision: 1, + messageDelay: 1}), + valid: true, + }, + { + params: makeParams(makeParamsArgs{ + blockBytes: 100 * 1024 * 1024, + evidenceAge: 2, + precision: 1, + messageDelay: 1}), + valid: true, + }, + { + params: makeParams(makeParamsArgs{ + blockBytes: 101 * 1024 * 1024, + evidenceAge: 2, + precision: 1, + messageDelay: 1}), + valid: false, + }, + { + params: makeParams(makeParamsArgs{ + blockBytes: 1024 * 1024 * 1024, + evidenceAge: 2, + precision: 1, + messageDelay: 1}), + valid: false, + }, + { + params: makeParams(makeParamsArgs{ + blockBytes: 1024 * 1024 * 1024, + evidenceAge: 2, + precision: 1, + messageDelay: 1}), + valid: false, + }, // test evidence params - 8: {makeParams(1, 0, 0, 0, valEd25519), false}, - 9: {makeParams(1, 0, 2, 2, valEd25519), false}, - 10: {makeParams(1000, 0, 2, 1, valEd25519), true}, - 11: {makeParams(1, 0, -1, 0, valEd25519), false}, + { + params: makeParams(makeParamsArgs{ + blockBytes: 1, + evidenceAge: 0, + maxEvidenceBytes: 0, + precision: 1, + messageDelay: 1}), + valid: false, + }, + { + params: makeParams(makeParamsArgs{ + blockBytes: 1, + evidenceAge: 2, + maxEvidenceBytes: 2, + precision: 1, + messageDelay: 1}), + valid: false, + }, + { + params: makeParams(makeParamsArgs{ + blockBytes: 1000, + evidenceAge: 2, + maxEvidenceBytes: 1, + precision: 1, + messageDelay: 1}), + valid: true, + }, + { + params: makeParams(makeParamsArgs{ + blockBytes: 1, + evidenceAge: -1, + maxEvidenceBytes: 0, + precision: 1, + messageDelay: 1}), + valid: false, + }, // test no pubkey type provided - 12: {makeParams(1, 0, 2, 0, []string{}), false}, + { + params: makeParams(makeParamsArgs{ + evidenceAge: 2, + pubkeyTypes: []string{}, + precision: 1, + messageDelay: 1}), + valid: false, + }, // test invalid pubkey type provided - 13: {makeParams(1, 0, 2, 0, []string{"potatoes make good pubkeys"}), false}, + { + params: makeParams(makeParamsArgs{ + evidenceAge: 2, + pubkeyTypes: []string{"potatoes make good pubkeys"}, + precision: 1, + messageDelay: 1}), + valid: false, + }, + // test invalid pubkey type provided + { + params: makeParams(makeParamsArgs{ + evidenceAge: 2, + precision: 1, + messageDelay: -1}), + valid: false, + }, + { + params: makeParams(makeParamsArgs{ + evidenceAge: 2, + precision: -1, + messageDelay: 1}), + valid: false, + }, } for i, tc := range testCases { if tc.valid { @@ -50,38 +167,51 @@ func TestConsensusParamsValidation(t *testing.T) { } } -func makeParams( - blockBytes, blockGas int64, - evidenceAge int64, - maxEvidenceBytes int64, - pubkeyTypes []string, -) ConsensusParams { +type makeParamsArgs struct { + blockBytes int64 + blockGas int64 + evidenceAge int64 + maxEvidenceBytes int64 + pubkeyTypes []string + precision time.Duration + messageDelay time.Duration +} + +func makeParams(args makeParamsArgs) ConsensusParams { + if args.pubkeyTypes == nil { + args.pubkeyTypes = valEd25519 + } return ConsensusParams{ Block: BlockParams{ - MaxBytes: blockBytes, - MaxGas: blockGas, + MaxBytes: args.blockBytes, + MaxGas: args.blockGas, }, Evidence: EvidenceParams{ - MaxAgeNumBlocks: evidenceAge, - MaxAgeDuration: time.Duration(evidenceAge), - MaxBytes: maxEvidenceBytes, + MaxAgeNumBlocks: args.evidenceAge, + MaxAgeDuration: time.Duration(args.evidenceAge), + MaxBytes: args.maxEvidenceBytes, }, Validator: ValidatorParams{ - PubKeyTypes: pubkeyTypes, + PubKeyTypes: args.pubkeyTypes, + }, + Synchrony: SynchronyParams{ + Precision: args.precision, + MessageDelay: args.messageDelay, }, } + } func TestConsensusParamsHash(t *testing.T) { params := []ConsensusParams{ - makeParams(4, 2, 3, 1, valEd25519), - makeParams(1, 4, 3, 1, valEd25519), - makeParams(1, 2, 4, 1, valEd25519), - makeParams(2, 5, 7, 1, valEd25519), - makeParams(1, 7, 6, 1, valEd25519), - makeParams(9, 5, 4, 1, valEd25519), - makeParams(7, 8, 9, 1, valEd25519), - makeParams(4, 6, 5, 1, valEd25519), + makeParams(makeParamsArgs{blockBytes: 4, blockGas: 2, evidenceAge: 3, maxEvidenceBytes: 1}), + makeParams(makeParamsArgs{blockBytes: 1, blockGas: 4, evidenceAge: 3, maxEvidenceBytes: 1}), + makeParams(makeParamsArgs{blockBytes: 1, blockGas: 2, evidenceAge: 4, maxEvidenceBytes: 1}), + makeParams(makeParamsArgs{blockBytes: 2, blockGas: 5, evidenceAge: 7, maxEvidenceBytes: 1}), + makeParams(makeParamsArgs{blockBytes: 1, blockGas: 7, evidenceAge: 6, maxEvidenceBytes: 1}), + makeParams(makeParamsArgs{blockBytes: 9, blockGas: 5, evidenceAge: 4, maxEvidenceBytes: 1}), + makeParams(makeParamsArgs{blockBytes: 7, blockGas: 8, evidenceAge: 9, maxEvidenceBytes: 1}), + makeParams(makeParamsArgs{blockBytes: 4, blockGas: 6, evidenceAge: 5, maxEvidenceBytes: 1}), } hashes := make([][]byte, len(params)) @@ -101,20 +231,31 @@ func TestConsensusParamsHash(t *testing.T) { func TestConsensusParamsUpdate(t *testing.T) { testCases := []struct { - params ConsensusParams + intialParams ConsensusParams updates *tmproto.ConsensusParams updatedParams ConsensusParams }{ // empty updates { - makeParams(1, 2, 3, 0, valEd25519), - &tmproto.ConsensusParams{}, - makeParams(1, 2, 3, 0, valEd25519), + intialParams: makeParams(makeParamsArgs{blockBytes: 1, blockGas: 2, evidenceAge: 3}), + updates: &tmproto.ConsensusParams{}, + updatedParams: makeParams(makeParamsArgs{blockBytes: 1, blockGas: 2, evidenceAge: 3}), + }, + { + // update synchrony params + intialParams: makeParams(makeParamsArgs{evidenceAge: 3, precision: time.Second, messageDelay: 3 * time.Second}), + updates: &tmproto.ConsensusParams{ + Synchrony: &tmproto.SynchronyParams{ + Precision: time.Second * 2, + MessageDelay: time.Second * 4, + }, + }, + updatedParams: makeParams(makeParamsArgs{evidenceAge: 3, precision: 2 * time.Second, messageDelay: 4 * time.Second}), }, // fine updates { - makeParams(1, 2, 3, 0, valEd25519), - &tmproto.ConsensusParams{ + intialParams: makeParams(makeParamsArgs{blockBytes: 1, blockGas: 2, evidenceAge: 3}), + updates: &tmproto.ConsensusParams{ Block: &tmproto.BlockParams{ MaxBytes: 100, MaxGas: 200, @@ -128,11 +269,15 @@ func TestConsensusParamsUpdate(t *testing.T) { PubKeyTypes: valSecp256k1, }, }, - makeParams(100, 200, 300, 50, valSecp256k1), + updatedParams: makeParams(makeParamsArgs{ + blockBytes: 100, blockGas: 200, + evidenceAge: 300, + maxEvidenceBytes: 50, + pubkeyTypes: valSecp256k1}), }, { - makeParams(1, 2, 3, 0, valEd25519), - &tmproto.ConsensusParams{ + intialParams: makeParams(makeParamsArgs{blockBytes: 1, blockGas: 2, evidenceAge: 3}), + updates: &tmproto.ConsensusParams{ Block: &tmproto.BlockParams{ MaxBytes: 100, MaxGas: 200, @@ -145,17 +290,23 @@ func TestConsensusParamsUpdate(t *testing.T) { Validator: &tmproto.ValidatorParams{ PubKeyTypes: valSr25519, }, - }, makeParams(100, 200, 300, 50, valSr25519), + }, + updatedParams: makeParams(makeParamsArgs{ + blockBytes: 100, + blockGas: 200, + evidenceAge: 300, + maxEvidenceBytes: 50, + pubkeyTypes: valSr25519}), }, } for _, tc := range testCases { - assert.Equal(t, tc.updatedParams, tc.params.UpdateConsensusParams(tc.updates)) + assert.Equal(t, tc.updatedParams, tc.intialParams.UpdateConsensusParams(tc.updates)) } } func TestConsensusParamsUpdate_AppVersion(t *testing.T) { - params := makeParams(1, 2, 3, 0, valEd25519) + params := makeParams(makeParamsArgs{blockBytes: 1, blockGas: 2, evidenceAge: 3}) assert.EqualValues(t, 0, params.Version.AppVersion) @@ -167,14 +318,16 @@ func TestConsensusParamsUpdate_AppVersion(t *testing.T) { func TestProto(t *testing.T) { params := []ConsensusParams{ - makeParams(4, 2, 3, 1, valEd25519), - makeParams(1, 4, 3, 1, valEd25519), - makeParams(1, 2, 4, 1, valEd25519), - makeParams(2, 5, 7, 1, valEd25519), - makeParams(1, 7, 6, 1, valEd25519), - makeParams(9, 5, 4, 1, valEd25519), - makeParams(7, 8, 9, 1, valEd25519), - makeParams(4, 6, 5, 1, valEd25519), + makeParams(makeParamsArgs{blockBytes: 4, blockGas: 2, evidenceAge: 3, maxEvidenceBytes: 1}), + makeParams(makeParamsArgs{blockBytes: 1, blockGas: 4, evidenceAge: 3, maxEvidenceBytes: 1}), + makeParams(makeParamsArgs{blockBytes: 1, blockGas: 2, evidenceAge: 4, maxEvidenceBytes: 1}), + makeParams(makeParamsArgs{blockBytes: 2, blockGas: 5, evidenceAge: 7, maxEvidenceBytes: 1}), + makeParams(makeParamsArgs{blockBytes: 1, blockGas: 7, evidenceAge: 6, maxEvidenceBytes: 1}), + makeParams(makeParamsArgs{blockBytes: 9, blockGas: 5, evidenceAge: 4, maxEvidenceBytes: 1}), + makeParams(makeParamsArgs{blockBytes: 7, blockGas: 8, evidenceAge: 9, maxEvidenceBytes: 1}), + makeParams(makeParamsArgs{blockBytes: 4, blockGas: 6, evidenceAge: 5, maxEvidenceBytes: 1}), + makeParams(makeParamsArgs{precision: time.Second, messageDelay: time.Minute}), + makeParams(makeParamsArgs{precision: time.Nanosecond, messageDelay: time.Millisecond}), } for i := range params { diff --git a/types/proposal.go b/types/proposal.go index 3f06738fb..92d6acb09 100644 --- a/types/proposal.go +++ b/types/proposal.go @@ -34,14 +34,14 @@ type Proposal struct { // NewProposal returns a new Proposal. // If there is no POLRound, polRound should be -1. -func NewProposal(height int64, round int32, polRound int32, blockID BlockID) *Proposal { +func NewProposal(height int64, round int32, polRound int32, blockID BlockID, ts time.Time) *Proposal { return &Proposal{ Type: tmproto.ProposalType, Height: height, Round: round, BlockID: blockID, POLRound: polRound, - Timestamp: tmtime.Now(), + Timestamp: tmtime.Canonical(ts), } } @@ -79,6 +79,31 @@ func (p *Proposal) ValidateBasic() error { return nil } +// IsTimely validates that the block timestamp is 'timely' according to the proposer-based timestamp algorithm. +// To evaluate if a block is timely, its timestamp is compared to the local time of the validator along with the +// configured Precision and MsgDelay parameters. +// Specifically, a proposed block timestamp is considered timely if it is satisfies the following inequalities: +// +// localtime >= proposedBlockTime - Precision +// localtime <= proposedBlockTime + MsgDelay + Precision +// +// Note: If the proposal is for the `initialHeight` the second inequality is not checked. This is because +// the timestamp in this case is set to the preconfigured genesis time. +// For more information on the meaning of 'timely', see the proposer-based timestamp specification: +// https://github.com/tendermint/spec/tree/master/spec/consensus/proposer-based-timestamp +func (p *Proposal) IsTimely(recvTime time.Time, sp SynchronyParams, initialHeight int64) bool { + // lhs is `proposedBlockTime - Precision` in the first inequality + lhs := p.Timestamp.Add(-sp.Precision) + // rhs is `proposedBlockTime + MsgDelay + Precision` in the second inequality + rhs := p.Timestamp.Add(sp.MessageDelay).Add(sp.Precision) + + if recvTime.Before(lhs) || (p.Height != initialHeight && recvTime.After(rhs)) { + return false + } + + return true +} + // String returns a string representation of the Proposal. // // 1. height diff --git a/types/proposal_test.go b/types/proposal_test.go index 472c66af5..8e5aa39dc 100644 --- a/types/proposal_test.go +++ b/types/proposal_test.go @@ -13,6 +13,7 @@ import ( "github.com/tendermint/tendermint/crypto/tmhash" "github.com/tendermint/tendermint/internal/libs/protoio" tmrand "github.com/tendermint/tendermint/libs/rand" + tmtime "github.com/tendermint/tendermint/libs/time" tmproto "github.com/tendermint/tendermint/proto/tendermint/types" ) @@ -60,7 +61,7 @@ func TestProposalVerifySignature(t *testing.T) { prop := NewProposal( 4, 2, 2, - BlockID{tmrand.Bytes(tmhash.Size), PartSetHeader{777, tmrand.Bytes(tmhash.Size)}}) + BlockID{tmrand.Bytes(tmhash.Size), PartSetHeader{777, tmrand.Bytes(tmhash.Size)}}, tmtime.Now()) p := prop.ToProto() signBytes := ProposalSignBytes("test_chain_id", p) @@ -172,7 +173,7 @@ func TestProposalValidateBasic(t *testing.T) { prop := NewProposal( 4, 2, 2, - blockID) + blockID, tmtime.Now()) p := prop.ToProto() err := privVal.SignProposal(ctx, "test_chain_id", p) prop.Signature = p.Signature @@ -184,9 +185,9 @@ func TestProposalValidateBasic(t *testing.T) { } func TestProposalProtoBuf(t *testing.T) { - proposal := NewProposal(1, 2, 3, makeBlockID([]byte("hash"), 2, []byte("part_set_hash"))) + proposal := NewProposal(1, 2, 3, makeBlockID([]byte("hash"), 2, []byte("part_set_hash")), tmtime.Now()) proposal.Signature = []byte("sig") - proposal2 := NewProposal(1, 2, 3, BlockID{}) + proposal2 := NewProposal(1, 2, 3, BlockID{}, tmtime.Now()) testCases := []struct { msg string @@ -210,3 +211,99 @@ func TestProposalProtoBuf(t *testing.T) { } } } + +func TestIsTimely(t *testing.T) { + genesisTime, err := time.Parse(time.RFC3339, "2019-03-13T23:00:00Z") + require.NoError(t, err) + testCases := []struct { + name string + genesisHeight int64 + proposalHeight int64 + proposalTime time.Time + recvTime time.Time + precision time.Duration + msgDelay time.Duration + expectTimely bool + }{ + // proposalTime - precision <= localTime <= proposalTime + msgDelay + precision + { + // Checking that the following inequality evaluates to true: + // 0 - 2 <= 1 <= 0 + 1 + 2 + name: "basic timely", + genesisHeight: 1, + proposalHeight: 2, + proposalTime: genesisTime, + recvTime: genesisTime.Add(1 * time.Nanosecond), + precision: time.Nanosecond * 2, + msgDelay: time.Nanosecond, + expectTimely: true, + }, + { + // Checking that the following inequality evaluates to false: + // 0 - 2 <= 4 <= 0 + 1 + 2 + name: "local time too large", + genesisHeight: 1, + proposalHeight: 2, + proposalTime: genesisTime, + recvTime: genesisTime.Add(4 * time.Nanosecond), + precision: time.Nanosecond * 2, + msgDelay: time.Nanosecond, + expectTimely: false, + }, + { + // Checking that the following inequality evaluates to false: + // 4 - 2 <= 0 <= 4 + 2 + 1 + name: "proposal time too large", + genesisHeight: 1, + proposalHeight: 2, + proposalTime: genesisTime.Add(4 * time.Nanosecond), + recvTime: genesisTime, + precision: time.Nanosecond * 2, + msgDelay: time.Nanosecond, + expectTimely: false, + }, + { + // Checking that the following inequality evaluates to true: + // 0 - 2 <= 4 + // and the following check is skipped + // 4 <= 0 + 1 + 2 + name: "local time too large but proposal is for genesis", + genesisHeight: 1, + proposalHeight: 1, + proposalTime: genesisTime, + recvTime: genesisTime.Add(4 * time.Nanosecond), + precision: time.Nanosecond * 2, + msgDelay: time.Nanosecond, + expectTimely: true, + }, + { + // Checking that the following inequality evaluates to false: + // 4 - 2 <= 0 + name: "proposal time too large for genesis block proposal", + genesisHeight: 1, + proposalHeight: 1, + proposalTime: genesisTime.Add(4 * time.Nanosecond), + recvTime: genesisTime, + precision: time.Nanosecond * 2, + msgDelay: time.Nanosecond, + expectTimely: false, + }, + } + + for _, testCase := range testCases { + t.Run(testCase.name, func(t *testing.T) { + p := Proposal{ + Height: testCase.proposalHeight, + Timestamp: testCase.proposalTime, + } + + sp := SynchronyParams{ + Precision: testCase.precision, + MessageDelay: testCase.msgDelay, + } + + ti := p.IsTimely(testCase.recvTime, sp, testCase.genesisHeight) + assert.Equal(t, testCase.expectTimely, ti) + }) + } +} diff --git a/types/vote.go b/types/vote.go index ceae65e48..e5aa32ada 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 a3ef9e802..8baa74172 100644 --- a/types/vote_set_test.go +++ b/types/vote_set_test.go @@ -31,7 +31,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, @@ -48,7 +48,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) { @@ -155,7 +155,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 { @@ -166,7 +166,7 @@ func TestVoteSet_2_3Majority(t *testing.T) { _, err = signAddVote(ctx, privValidators[6], withBlockHash(vote, tmrand.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. @@ -178,7 +178,7 @@ func TestVoteSet_2_3Majority(t *testing.T) { _, err = signAddVote(ctx, 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") } } @@ -213,7 +213,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 @@ -225,7 +225,7 @@ func TestVoteSet_2_3MajorityRedux(t *testing.T) { _, err = signAddVote(ctx, 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") } @@ -239,7 +239,7 @@ func TestVoteSet_2_3MajorityRedux(t *testing.T) { _, err = signAddVote(ctx, 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") } @@ -253,7 +253,7 @@ func TestVoteSet_2_3MajorityRedux(t *testing.T) { _, err = signAddVote(ctx, 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") } @@ -266,7 +266,7 @@ func TestVoteSet_2_3MajorityRedux(t *testing.T) { _, err = signAddVote(ctx, privValidators[69], withBlockHash(vote, tmrand.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") }