Browse Source

consensus: update state machine to use the new consensus params (#8181)

master
William Banfield 2 years ago
committed by GitHub
parent
commit
e4ae922c33
No known key found for this signature in database GPG Key ID: 4AEE18F83AFDEB23
13 changed files with 127 additions and 110 deletions
  1. +0
    -27
      config/config.go
  2. +6
    -6
      config/toml.go
  3. +5
    -5
      internal/blocksync/reactor_test.go
  4. +1
    -1
      internal/consensus/byzantine_test.go
  5. +3
    -2
      internal/consensus/common_test.go
  6. +13
    -6
      internal/consensus/mempool_test.go
  7. +3
    -2
      internal/consensus/pbts_test.go
  8. +1
    -1
      internal/consensus/reactor_test.go
  9. +12
    -20
      internal/consensus/state.go
  10. +39
    -39
      internal/consensus/state_test.go
  11. +22
    -0
      internal/test/factory/params.go
  12. +1
    -1
      node/node_test.go
  13. +21
    -0
      types/params.go

+ 0
- 27
config/config.go View File

@ -1030,33 +1030,6 @@ func (cfg *ConsensusConfig) WaitForTxs() bool {
return !cfg.CreateEmptyBlocks || cfg.CreateEmptyBlocksInterval > 0
}
// Propose returns the amount of time to wait for a proposal
func (cfg *ConsensusConfig) Propose(round int32) time.Duration {
return time.Duration(
cfg.TimeoutPropose.Nanoseconds()+cfg.TimeoutProposeDelta.Nanoseconds()*int64(round),
) * time.Nanosecond
}
// Prevote returns the amount of time to wait for straggler votes after receiving any +2/3 prevotes
func (cfg *ConsensusConfig) Prevote(round int32) time.Duration {
return time.Duration(
cfg.TimeoutPrevote.Nanoseconds()+cfg.TimeoutPrevoteDelta.Nanoseconds()*int64(round),
) * time.Nanosecond
}
// Precommit returns the amount of time to wait for straggler votes after receiving any +2/3 precommits
func (cfg *ConsensusConfig) Precommit(round int32) time.Duration {
return time.Duration(
cfg.TimeoutPrecommit.Nanoseconds()+cfg.TimeoutPrecommitDelta.Nanoseconds()*int64(round),
) * time.Nanosecond
}
// Commit returns the amount of time to wait for straggler votes after receiving +2/3 precommits
// for a single block (ie. a commit).
func (cfg *ConsensusConfig) Commit(t time.Time) time.Time {
return t.Add(cfg.TimeoutCommit)
}
// WalFile returns the full path to the write-ahead log file
func (cfg *ConsensusConfig) WalFile() string {
if cfg.walFile != "" {


+ 6
- 6
config/toml.go View File

@ -603,12 +603,12 @@ var testGenesisFmt = `{
"precision": "10000000"
},
"timeout": {
"propose": "30000000000",
"propose_delta": "50000000",
"vote": "30000000000",
"vote_delta": "50000000",
"commit": "10000000000",
"bypass_commit_timeout": false
"propose": "30000000",
"propose_delta": "50000",
"vote": "30000000",
"vote_delta": "50000",
"commit": "10000000",
"bypass_timeout_commit": true
},
"evidence": {
"max_age_num_blocks": "100000",


+ 5
- 5
internal/blocksync/reactor_test.go View File

@ -226,7 +226,7 @@ func TestReactor_AbruptDisconnect(t *testing.T) {
defer os.RemoveAll(cfg.RootDir)
valSet, privVals := factory.ValidatorSet(ctx, t, 1, 30)
genDoc := factory.GenesisDoc(cfg, time.Now(), valSet.Validators, nil)
genDoc := factory.GenesisDoc(cfg, time.Now(), valSet.Validators, factory.ConsensusParams())
maxBlockHeight := int64(64)
rts := setup(ctx, t, genDoc, privVals[0], []int64{maxBlockHeight, 0}, 0)
@ -266,7 +266,7 @@ func TestReactor_SyncTime(t *testing.T) {
defer os.RemoveAll(cfg.RootDir)
valSet, privVals := factory.ValidatorSet(ctx, t, 1, 30)
genDoc := factory.GenesisDoc(cfg, time.Now(), valSet.Validators, nil)
genDoc := factory.GenesisDoc(cfg, time.Now(), valSet.Validators, factory.ConsensusParams())
maxBlockHeight := int64(101)
rts := setup(ctx, t, genDoc, privVals[0], []int64{maxBlockHeight, 0}, 0)
@ -294,7 +294,7 @@ func TestReactor_NoBlockResponse(t *testing.T) {
defer os.RemoveAll(cfg.RootDir)
valSet, privVals := factory.ValidatorSet(ctx, t, 1, 30)
genDoc := factory.GenesisDoc(cfg, time.Now(), valSet.Validators, nil)
genDoc := factory.GenesisDoc(cfg, time.Now(), valSet.Validators, factory.ConsensusParams())
maxBlockHeight := int64(65)
rts := setup(ctx, t, genDoc, privVals[0], []int64{maxBlockHeight, 0}, 0)
@ -347,7 +347,7 @@ func TestReactor_BadBlockStopsPeer(t *testing.T) {
maxBlockHeight := int64(48)
valSet, privVals := factory.ValidatorSet(ctx, t, 1, 30)
genDoc := factory.GenesisDoc(cfg, time.Now(), valSet.Validators, nil)
genDoc := factory.GenesisDoc(cfg, time.Now(), valSet.Validators, factory.ConsensusParams())
rts := setup(ctx, t, genDoc, privVals[0], []int64{maxBlockHeight, 0, 0, 0, 0}, 1000)
@ -382,7 +382,7 @@ func TestReactor_BadBlockStopsPeer(t *testing.T) {
// XXX: This causes a potential race condition.
// See: https://github.com/tendermint/tendermint/issues/6005
valSet, otherPrivVals := factory.ValidatorSet(ctx, t, 1, 30)
otherGenDoc := factory.GenesisDoc(cfg, time.Now(), valSet.Validators, nil)
otherGenDoc := factory.GenesisDoc(cfg, time.Now(), valSet.Validators, factory.ConsensusParams())
newNode := rts.network.MakeNode(ctx, t, p2ptest.NodeOptions{
MaxPeers: uint16(len(rts.nodes) + 1),
MaxConnected: uint16(len(rts.nodes) + 1),


+ 1
- 1
internal/consensus/byzantine_test.go View File

@ -48,7 +48,7 @@ func TestByzantinePrevoteEquivocation(t *testing.T) {
tickerFunc := newMockTickerFunc(true)
valSet, privVals := factory.ValidatorSet(ctx, t, nValidators, 30)
genDoc := factory.GenesisDoc(config, time.Now(), valSet.Validators, nil)
genDoc := factory.GenesisDoc(config, time.Now(), valSet.Validators, factory.ConsensusParams())
states := make([]*State, nValidators)
for i := 0; i < nValidators; i++ {


+ 3
- 2
internal/consensus/common_test.go View File

@ -548,6 +548,7 @@ func makeState(ctx context.Context, t *testing.T, args makeStateArgs) (*State, [
}
state, privVals := makeGenesisState(ctx, t, args.config, genesisStateArgs{
Params: factory.ConsensusParams(),
Validators: validators,
})
@ -793,7 +794,7 @@ func makeConsensusState(
tempDir := t.TempDir()
valSet, privVals := factory.ValidatorSet(ctx, t, nValidators, 30)
genDoc := factory.GenesisDoc(cfg, time.Now(), valSet.Validators, nil)
genDoc := factory.GenesisDoc(cfg, time.Now(), valSet.Validators, factory.ConsensusParams())
css := make([]*State, nValidators)
logger := consensusLogger()
@ -852,7 +853,7 @@ func randConsensusNetWithPeers(
t.Helper()
valSet, privVals := factory.ValidatorSet(ctx, t, nValidators, testMinPower)
genDoc := factory.GenesisDoc(cfg, time.Now(), valSet.Validators, nil)
genDoc := factory.GenesisDoc(cfg, time.Now(), valSet.Validators, factory.ConsensusParams())
css := make([]*State, nPeers)
t.Helper()
logger := consensusLogger()


+ 13
- 6
internal/consensus/mempool_test.go View File

@ -17,6 +17,7 @@ import (
"github.com/tendermint/tendermint/internal/mempool"
sm "github.com/tendermint/tendermint/internal/state"
"github.com/tendermint/tendermint/internal/store"
"github.com/tendermint/tendermint/internal/test/factory"
"github.com/tendermint/tendermint/libs/log"
"github.com/tendermint/tendermint/types"
)
@ -42,7 +43,8 @@ func TestMempoolNoProgressUntilTxsAvailable(t *testing.T) {
config.Consensus.CreateEmptyBlocks = false
state, privVals := makeGenesisState(ctx, t, baseConfig, genesisStateArgs{
Validators: 1,
Power: 10})
Power: 10,
Params: factory.ConsensusParams()})
cs := newStateWithConfig(ctx, t, log.NewNopLogger(), config, state, privVals[0], NewCounterApplication())
assertMempool(t, cs.txNotifier).EnableTxsAvailable()
height, round := cs.Height, cs.Round
@ -69,7 +71,8 @@ func TestMempoolProgressAfterCreateEmptyBlocksInterval(t *testing.T) {
config.Consensus.CreateEmptyBlocksInterval = ensureTimeout
state, privVals := makeGenesisState(ctx, t, baseConfig, genesisStateArgs{
Validators: 1,
Power: 10})
Power: 10,
Params: factory.ConsensusParams()})
cs := newStateWithConfig(ctx, t, log.NewNopLogger(), config, state, privVals[0], NewCounterApplication())
assertMempool(t, cs.txNotifier).EnableTxsAvailable()
@ -94,7 +97,8 @@ func TestMempoolProgressInHigherRound(t *testing.T) {
config.Consensus.CreateEmptyBlocks = false
state, privVals := makeGenesisState(ctx, t, baseConfig, genesisStateArgs{
Validators: 1,
Power: 10})
Power: 10,
Params: factory.ConsensusParams()})
cs := newStateWithConfig(ctx, t, log.NewNopLogger(), config, state, privVals[0], NewCounterApplication())
assertMempool(t, cs.txNotifier).EnableTxsAvailable()
height, round := cs.Height, cs.Round
@ -119,7 +123,7 @@ func TestMempoolProgressInHigherRound(t *testing.T) {
ensureNewRound(t, newRoundCh, height, round) // first round at next height
checkTxsRange(ctx, t, cs, 0, 1) // we deliver txs, but don't set a proposal so we get the next round
ensureNewTimeout(t, timeoutCh, height, round, cs.config.TimeoutPropose.Nanoseconds())
ensureNewTimeout(t, timeoutCh, height, round, cs.state.ConsensusParams.Timeout.ProposeTimeout(round).Nanoseconds())
round++ // moving to the next round
ensureNewRound(t, newRoundCh, height, round) // wait for the next round
@ -145,7 +149,9 @@ func TestMempoolTxConcurrentWithCommit(t *testing.T) {
logger := log.NewNopLogger()
state, privVals := makeGenesisState(ctx, t, config, genesisStateArgs{
Validators: 1,
Power: 10})
Power: 10,
Params: factory.ConsensusParams(),
})
stateStore := sm.NewStore(dbm.NewMemDB())
blockStore := store.NewBlockStore(dbm.NewMemDB())
@ -180,7 +186,8 @@ func TestMempoolRmBadTx(t *testing.T) {
state, privVals := makeGenesisState(ctx, t, config, genesisStateArgs{
Validators: 1,
Power: 10})
Power: 10,
Params: factory.ConsensusParams()})
app := NewCounterApplication()
stateStore := sm.NewStore(dbm.NewMemDB())
blockStore := store.NewBlockStore(dbm.NewMemDB())


+ 3
- 2
internal/consensus/pbts_test.go View File

@ -12,6 +12,7 @@ import (
"github.com/tendermint/tendermint/abci/example/kvstore"
"github.com/tendermint/tendermint/internal/eventbus"
tmpubsub "github.com/tendermint/tendermint/internal/pubsub"
"github.com/tendermint/tendermint/internal/test/factory"
"github.com/tendermint/tendermint/libs/log"
tmtimemocks "github.com/tendermint/tendermint/libs/time/mocks"
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
@ -105,8 +106,8 @@ func newPBTSTestHarness(ctx context.Context, t *testing.T, tc pbtsTestConfigurat
// height 4 therefore occurs 2*blockTimeIota after height 2.
tc.height4ProposedBlockOffset = tc.height2ProposalTimeDeliveryOffset + 2*blockTimeIota
}
cfg.Consensus.TimeoutPropose = tc.timeoutPropose
consensusParams := types.DefaultConsensusParams()
consensusParams := factory.ConsensusParams()
consensusParams.Timeout.Propose = tc.timeoutPropose
consensusParams.Synchrony = tc.synchronyParams
state, privVals := makeGenesisState(ctx, t, cfg, genesisStateArgs{


+ 1
- 1
internal/consensus/reactor_test.go View File

@ -450,7 +450,7 @@ func TestReactorWithEvidence(t *testing.T) {
tickerFunc := newMockTickerFunc(true)
valSet, privVals := factory.ValidatorSet(ctx, t, n, 30)
genDoc := factory.GenesisDoc(cfg, time.Now(), valSet.Validators, nil)
genDoc := factory.GenesisDoc(cfg, time.Now(), valSet.Validators, factory.ConsensusParams())
states := make([]*State, n)
logger := consensusLogger()


+ 12
- 20
internal/consensus/state.go View File

@ -506,8 +506,8 @@ func (cs *State) OnStop() {
if cs.GetRoundState().Step == cstypes.RoundStepCommit {
select {
case <-cs.getOnStopCh():
case <-time.After(cs.config.TimeoutCommit):
cs.logger.Error("OnStop: timeout waiting for commit to finish", "time", cs.config.TimeoutCommit)
case <-time.After(cs.state.ConsensusParams.Timeout.Commit):
cs.logger.Error("OnStop: timeout waiting for commit to finish", "time", cs.state.ConsensusParams.Timeout.Commit)
}
}
@ -787,9 +787,9 @@ func (cs *State) updateToState(ctx context.Context, state sm.State) {
// to be gathered for the first block.
// And alternative solution that relies on clocks:
// cs.StartTime = state.LastBlockTime.Add(timeoutCommit)
cs.StartTime = cs.config.Commit(tmtime.Now())
cs.StartTime = cs.state.ConsensusParams.Timeout.CommitTime(tmtime.Now())
} else {
cs.StartTime = cs.config.Commit(cs.CommitTime)
cs.StartTime = cs.state.ConsensusParams.Timeout.CommitTime(cs.CommitTime)
}
cs.Validators = validators
@ -1262,7 +1262,7 @@ func (cs *State) enterPropose(ctx context.Context, height int64, round int32) {
}()
// If we don't get the proposal and all block parts quick enough, enterPrevote
cs.scheduleTimeout(cs.config.Propose(round), height, round, cstypes.RoundStepPropose)
cs.scheduleTimeout(cs.state.ConsensusParams.Timeout.ProposeTimeout(round), height, round, cstypes.RoundStepPropose)
// Nothing more to do if we're not a validator
if cs.privValidator == nil {
@ -1343,7 +1343,7 @@ func (cs *State) defaultDecideProposal(ctx context.Context, height int64, round
p := proposal.ToProto()
// wait the max amount we would wait for a proposal
ctxto, cancel := context.WithTimeout(ctx, cs.config.TimeoutPropose)
ctxto, cancel := context.WithTimeout(ctx, cs.state.ConsensusParams.Timeout.Propose)
defer cancel()
if err := cs.privValidator.SignProposal(ctxto, cs.state.ChainID, p); err == nil {
proposal.Signature = p.Signature
@ -1620,7 +1620,7 @@ func (cs *State) enterPrevoteWait(ctx context.Context, height int64, round int32
}()
// Wait for some more prevotes; enterPrecommit
cs.scheduleTimeout(cs.config.Prevote(round), height, round, cstypes.RoundStepPrevoteWait)
cs.scheduleTimeout(cs.state.ConsensusParams.Timeout.VoteTimeout(round), height, round, cstypes.RoundStepPrevoteWait)
}
// Enter: `timeoutPrevote` after any +2/3 prevotes.
@ -1773,7 +1773,7 @@ func (cs *State) enterPrecommitWait(ctx context.Context, height int64, round int
}()
// wait for some more precommits; enterNewRound
cs.scheduleTimeout(cs.config.Precommit(round), height, round, cstypes.RoundStepPrecommitWait)
cs.scheduleTimeout(cs.state.ConsensusParams.Timeout.VoteTimeout(round), height, round, cstypes.RoundStepPrecommitWait)
}
// Enter: +2/3 precommits for block
@ -2309,7 +2309,7 @@ func (cs *State) addVote(
cs.evsw.FireEvent(ctx, types.EventVoteValue, vote)
// if we can skip timeoutCommit and have all the votes now,
if cs.config.SkipTimeoutCommit && cs.LastCommit.HasAll() {
if cs.state.ConsensusParams.Timeout.BypassCommitTimeout && cs.LastCommit.HasAll() {
// go straight to new round (skip timeout commit)
// cs.scheduleTimeout(time.Duration(0), cs.Height, 0, cstypes.RoundStepNewHeight)
cs.enterNewRound(ctx, cs.Height, 0)
@ -2422,7 +2422,7 @@ func (cs *State) addVote(
if !blockID.IsNil() {
cs.enterCommit(ctx, height, vote.Round)
if cs.config.SkipTimeoutCommit && precommits.HasAll() {
if cs.state.ConsensusParams.Timeout.BypassCommitTimeout && precommits.HasAll() {
cs.enterNewRound(ctx, cs.Height, 0)
}
} else {
@ -2472,19 +2472,16 @@ func (cs *State) signVote(
// If the signedMessageType is for precommit,
// use our local precommit Timeout as the max wait time for getting a singed commit. The same goes for prevote.
var timeout time.Duration
timeout := cs.state.ConsensusParams.Timeout.VoteTimeout(cs.Round)
switch msgType {
case tmproto.PrecommitType:
timeout = cs.config.TimeoutPrecommit
// if the signedMessage type is for a precommit, add VoteExtension
ext, err := cs.blockExec.ExtendVote(ctx, vote)
if err != nil {
return nil, err
}
vote.VoteExtension = ext
case tmproto.PrevoteType:
timeout = cs.config.TimeoutPrevote
default:
timeout = time.Second
}
@ -2543,12 +2540,7 @@ func (cs *State) updatePrivValidatorPubKey(rctx context.Context) error {
return nil
}
var timeout time.Duration
if cs.config.TimeoutPrecommit > cs.config.TimeoutPrevote {
timeout = cs.config.TimeoutPrecommit
} else {
timeout = cs.config.TimeoutPrevote
}
timeout := cs.state.ConsensusParams.Timeout.VoteTimeout(cs.Round)
// no GetPubKey retry beyond the proposal/voting in RetrySignerClient
if cs.Step >= cstypes.RoundStepPrecommit && cs.privValidatorType == types.RetrySignerClient {


+ 39
- 39
internal/consensus/state_test.go View File

@ -163,7 +163,7 @@ func TestStateEnterProposeNoPrivValidator(t *testing.T) {
startTestRound(ctx, cs, height, round)
// if we're not a validator, EnterPropose should timeout
ensureNewTimeout(t, timeoutCh, height, round, cs.config.TimeoutPropose.Nanoseconds())
ensureNewTimeout(t, timeoutCh, height, round, cs.state.ConsensusParams.Timeout.ProposeTimeout(round).Nanoseconds())
if cs.GetRoundState().Proposal != nil {
t.Error("Expected to make no proposal, since no privValidator")
@ -202,7 +202,7 @@ func TestStateEnterProposeYesPrivValidator(t *testing.T) {
}
// if we're a validator, enterPropose should not timeout
ensureNoNewTimeout(t, timeoutCh, cs.config.TimeoutPropose.Nanoseconds())
ensureNoNewTimeout(t, timeoutCh, cs.state.ConsensusParams.Timeout.ProposeTimeout(round).Nanoseconds())
}
func TestStateBadProposal(t *testing.T) {
@ -313,7 +313,7 @@ func TestStateOversizedBlock(t *testing.T) {
// c1 should log an error with the block part message as it exceeds the consensus params. The
// block is not added to cs.ProposalBlock so the node timeouts.
ensureNewTimeout(t, timeoutProposeCh, height, round, cs1.config.Propose(round).Nanoseconds())
ensureNewTimeout(t, timeoutProposeCh, height, round, cs1.state.ConsensusParams.Timeout.ProposeTimeout(round).Nanoseconds())
// and then should send nil prevote and precommit regardless of whether other validators prevote and
// precommit on it
@ -481,7 +481,7 @@ func TestStateLock_NoPOL(t *testing.T) {
// (note we're entering precommit for a second time this round)
// but with invalid args. then we enterPrecommitWait, and the timeout to new round
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.config.Precommit(round).Nanoseconds())
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.state.ConsensusParams.Timeout.VoteTimeout(round).Nanoseconds())
///
@ -494,7 +494,7 @@ func TestStateLock_NoPOL(t *testing.T) {
incrementRound(vs2)
// now we're on a new round and not the proposer, so wait for timeout
ensureNewTimeout(t, timeoutProposeCh, height, round, cs1.config.Propose(round).Nanoseconds())
ensureNewTimeout(t, timeoutProposeCh, height, round, cs1.state.ConsensusParams.Timeout.ProposeTimeout(round).Nanoseconds())
rs := cs1.GetRoundState()
@ -513,7 +513,7 @@ func TestStateLock_NoPOL(t *testing.T) {
// 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())
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.state.ConsensusParams.Timeout.VoteTimeout(round).Nanoseconds())
// the proposed block should still be locked block.
// we should precommit nil and be locked on the proposal.
ensurePrecommit(t, voteCh, height, round)
@ -525,7 +525,7 @@ func TestStateLock_NoPOL(t *testing.T) {
// (note we're entering precommit for a second time this round, but with invalid args
// then we enterPrecommitWait and timeout into NewRound
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.config.Precommit(round).Nanoseconds())
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.state.ConsensusParams.Timeout.VoteTimeout(round).Nanoseconds())
round++ // entering new round
ensureNewRound(t, newRoundCh, height, round)
@ -552,7 +552,7 @@ func TestStateLock_NoPOL(t *testing.T) {
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())
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.state.ConsensusParams.Timeout.VoteTimeout(round).Nanoseconds())
ensurePrecommit(t, voteCh, height, round) // precommit
validatePrecommit(ctx, t, cs1, round, 0, vss[0], nil, initialBlockID.Hash) // precommit nil but be locked on proposal
@ -567,7 +567,7 @@ func TestStateLock_NoPOL(t *testing.T) {
vs2) // NOTE: conflicting precommits at same height
ensurePrecommit(t, voteCh, height, round)
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.config.Precommit(round).Nanoseconds())
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.state.ConsensusParams.Timeout.VoteTimeout(round).Nanoseconds())
// cs1 is locked on a block at this point, so we must generate a new consensus
// state to force a new proposal block to be generated.
@ -606,7 +606,7 @@ func TestStateLock_NoPOL(t *testing.T) {
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())
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.state.ConsensusParams.Timeout.VoteTimeout(round).Nanoseconds())
ensurePrecommit(t, voteCh, height, round)
validatePrecommit(ctx, t, cs1, round, 0, vss[0], nil, initialBlockID.Hash) // precommit nil but locked on proposal
@ -683,7 +683,7 @@ func TestStateLock_POLUpdateLock(t *testing.T) {
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())
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.state.ConsensusParams.Timeout.VoteTimeout(round).Nanoseconds())
/*
Round 1:
@ -789,7 +789,7 @@ func TestStateLock_POLRelock(t *testing.T) {
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())
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.state.ConsensusParams.Timeout.VoteTimeout(round).Nanoseconds())
/*
Round 1:
@ -884,7 +884,7 @@ func TestStateLock_PrevoteNilWhenLockedAndMissProposal(t *testing.T) {
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())
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.state.ConsensusParams.Timeout.VoteTimeout(round).Nanoseconds())
/*
Round 1:
@ -970,7 +970,7 @@ func TestStateLock_PrevoteNilWhenLockedAndDifferentProposal(t *testing.T) {
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())
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.state.ConsensusParams.Timeout.VoteTimeout(round).Nanoseconds())
/*
Round 1:
@ -1078,7 +1078,7 @@ func TestStateLock_POLDoesNotUnlock(t *testing.T) {
signAddVotes(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), blockID, vs3)
// timeout to new round
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.config.Precommit(round).Nanoseconds())
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.state.ConsensusParams.Timeout.VoteTimeout(round).Nanoseconds())
/*
Round 1:
@ -1112,7 +1112,7 @@ func TestStateLock_POLDoesNotUnlock(t *testing.T) {
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())
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.state.ConsensusParams.Timeout.VoteTimeout(round).Nanoseconds())
/*
Round 2:
@ -1198,7 +1198,7 @@ func TestStateLock_MissingProposalWhenPOLSeenDoesNotUpdateLock(t *testing.T) {
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())
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.state.ConsensusParams.Timeout.VoteTimeout(round).Nanoseconds())
/*
Round 1:
@ -1285,7 +1285,7 @@ func TestStateLock_DoesNotLockOnOldProposal(t *testing.T) {
incrementRound(vs2, vs3, vs4)
// timeout to new round
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.config.Precommit(round).Nanoseconds())
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.state.ConsensusParams.Timeout.VoteTimeout(round).Nanoseconds())
/*
Round 1:
@ -1356,7 +1356,7 @@ func TestStateLock_POLSafety1(t *testing.T) {
// cs1 precommit nil
ensurePrecommit(t, voteCh, height, round)
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.config.Precommit(round).Nanoseconds())
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.state.ConsensusParams.Timeout.VoteTimeout(round).Nanoseconds())
incrementRound(vs2, vs3, vs4)
round++ // moving to the next round
@ -1397,7 +1397,7 @@ func TestStateLock_POLSafety1(t *testing.T) {
signAddVotes(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), types.BlockID{}, vs2, vs3, vs4)
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.config.Precommit(round).Nanoseconds())
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.state.ConsensusParams.Timeout.VoteTimeout(round).Nanoseconds())
incrementRound(vs2, vs3, vs4)
round++ // moving to the next round
@ -1409,7 +1409,7 @@ func TestStateLock_POLSafety1(t *testing.T) {
*/
// timeout of propose
ensureNewTimeout(t, timeoutProposeCh, height, round, cs1.config.Propose(round).Nanoseconds())
ensureNewTimeout(t, timeoutProposeCh, height, round, cs1.state.ConsensusParams.Timeout.ProposeTimeout(round).Nanoseconds())
// finish prevote
ensurePrevoteMatch(t, voteCh, height, round, nil)
@ -1493,7 +1493,7 @@ func TestStateLock_POLSafety2(t *testing.T) {
incrementRound(vs2, vs3, vs4)
// timeout of precommit wait to new round
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.config.Precommit(round).Nanoseconds())
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.state.ConsensusParams.Timeout.VoteTimeout(round).Nanoseconds())
round++ // moving to the next round
// in round 2 we see the polkad block from round 0
@ -1580,7 +1580,7 @@ func TestState_PrevotePOLFromPreviousRound(t *testing.T) {
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())
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.state.ConsensusParams.Timeout.VoteTimeout(round).Nanoseconds())
/*
Round 1:
@ -1621,7 +1621,7 @@ func TestState_PrevotePOLFromPreviousRound(t *testing.T) {
ensurePrecommit(t, voteCh, height, round)
// timeout to new round.
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.config.Precommit(round).Nanoseconds())
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.state.ConsensusParams.Timeout.VoteTimeout(round).Nanoseconds())
/*
Create a new proposal for D, the same block from Round 1.
@ -1714,7 +1714,7 @@ func TestProposeValidBlock(t *testing.T) {
signAddVotes(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), types.BlockID{}, vs2, vs3, vs4)
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.config.Precommit(round).Nanoseconds())
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.state.ConsensusParams.Timeout.VoteTimeout(round).Nanoseconds())
incrementRound(vs2, vs3, vs4)
round++ // moving to the next round
@ -1722,7 +1722,7 @@ func TestProposeValidBlock(t *testing.T) {
ensureNewRound(t, newRoundCh, height, round)
// timeout of propose
ensureNewTimeout(t, timeoutProposeCh, height, round, cs1.config.Propose(round).Nanoseconds())
ensureNewTimeout(t, timeoutProposeCh, height, round, cs1.state.ConsensusParams.Timeout.ProposeTimeout(round).Nanoseconds())
// We did not see a valid proposal within this round, so prevote nil.
ensurePrevoteMatch(t, voteCh, height, round, nil)
@ -1743,7 +1743,7 @@ func TestProposeValidBlock(t *testing.T) {
ensureNewRound(t, newRoundCh, height, round)
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.config.Precommit(round).Nanoseconds())
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.state.ConsensusParams.Timeout.VoteTimeout(round).Nanoseconds())
round++ // moving to the next round
@ -1802,7 +1802,7 @@ func TestSetValidBlockOnDelayedPrevote(t *testing.T) {
// vs3 send prevote nil
signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), types.BlockID{}, vs3)
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.config.Prevote(round).Nanoseconds())
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.state.ConsensusParams.Timeout.VoteTimeout(round).Nanoseconds())
ensurePrecommit(t, voteCh, height, round)
// we should have precommitted
@ -1856,7 +1856,7 @@ func TestSetValidBlockOnDelayedProposal(t *testing.T) {
startTestRound(ctx, cs1, cs1.Height, round)
ensureNewRound(t, newRoundCh, height, round)
ensureNewTimeout(t, timeoutProposeCh, height, round, cs1.config.Propose(round).Nanoseconds())
ensureNewTimeout(t, timeoutProposeCh, height, round, cs1.state.ConsensusParams.Timeout.ProposeTimeout(round).Nanoseconds())
ensurePrevoteMatch(t, voteCh, height, round, nil)
@ -1872,7 +1872,7 @@ func TestSetValidBlockOnDelayedProposal(t *testing.T) {
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())
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.state.ConsensusParams.Timeout.VoteTimeout(round).Nanoseconds())
ensurePrecommit(t, voteCh, height, round)
validatePrecommit(ctx, t, cs1, round, -1, vss[0], nil, nil)
@ -2036,7 +2036,7 @@ func TestWaitingTimeoutOnNilPolka(t *testing.T) {
signAddVotes(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), types.BlockID{}, vs2, vs3, vs4)
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.config.Precommit(round).Nanoseconds())
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.state.ConsensusParams.Timeout.VoteTimeout(round).Nanoseconds())
ensureNewRound(t, newRoundCh, height, round+1)
}
@ -2074,7 +2074,7 @@ func TestWaitingTimeoutProposeOnNewRound(t *testing.T) {
rs := cs1.GetRoundState()
assert.True(t, rs.Step == cstypes.RoundStepPropose) // P0 does not prevote before timeoutPropose expires
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.config.Propose(round).Nanoseconds())
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.state.ConsensusParams.Timeout.ProposeTimeout(round).Nanoseconds())
ensurePrevoteMatch(t, voteCh, height, round, nil)
}
@ -2113,7 +2113,7 @@ func TestRoundSkipOnNilPolkaFromHigherRound(t *testing.T) {
ensurePrecommit(t, voteCh, height, round)
validatePrecommit(ctx, t, cs1, round, -1, vss[0], nil, nil)
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.config.Precommit(round).Nanoseconds())
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.state.ConsensusParams.Timeout.VoteTimeout(round).Nanoseconds())
round++ // moving to the next round
ensureNewRound(t, newRoundCh, height, round)
@ -2145,7 +2145,7 @@ func TestWaitTimeoutProposeOnNilPolkaForTheCurrentRound(t *testing.T) {
incrementRound(vss[1:]...)
signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), types.BlockID{}, vs2, vs3, vs4)
ensureNewTimeout(t, timeoutProposeCh, height, round, cs1.config.Propose(round).Nanoseconds())
ensureNewTimeout(t, timeoutProposeCh, height, round, cs1.state.ConsensusParams.Timeout.ProposeTimeout(round).Nanoseconds())
ensurePrevoteMatch(t, voteCh, height, round, nil)
}
@ -2260,8 +2260,8 @@ func TestStartNextHeightCorrectlyAfterTimeout(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
config.Consensus.SkipTimeoutCommit = false
cs1, vss := makeState(ctx, t, makeStateArgs{config: config})
cs1.state.ConsensusParams.Timeout.BypassCommitTimeout = false
cs1.txNotifier = &fakeTxNotifier{ch: make(chan struct{})}
vs2, vs3, vs4 := vss[1], vss[2], vss[3]
@ -2302,7 +2302,7 @@ func TestStartNextHeightCorrectlyAfterTimeout(t *testing.T) {
signAddVotes(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), blockID, vs3)
// wait till timeout occurs
ensureNewTimeout(t, precommitTimeoutCh, height, round, cs1.config.TimeoutPrecommit.Nanoseconds())
ensureNewTimeout(t, precommitTimeoutCh, height, round, cs1.state.ConsensusParams.Timeout.VoteTimeout(round).Nanoseconds())
ensureNewRound(t, newRoundCh, height, round+1)
@ -2313,7 +2313,7 @@ func TestStartNextHeightCorrectlyAfterTimeout(t *testing.T) {
cs1.txNotifier.(*fakeTxNotifier).Notify()
ensureNewTimeout(t, timeoutProposeCh, height+1, round, cs1.config.Propose(round).Nanoseconds())
ensureNewTimeout(t, timeoutProposeCh, height+1, round, cs1.state.ConsensusParams.Timeout.ProposeTimeout(round).Nanoseconds())
rs = cs1.GetRoundState()
assert.False(
t,
@ -2326,8 +2326,8 @@ func TestResetTimeoutPrecommitUponNewHeight(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
config.Consensus.SkipTimeoutCommit = false
cs1, vss := makeState(ctx, t, makeStateArgs{config: config})
cs1.state.ConsensusParams.Timeout.BypassCommitTimeout = false
vs2, vs3, vs4 := vss[1], vss[2], vss[3]
height, round := cs1.Height, cs1.Round
@ -2441,7 +2441,7 @@ func TestStateHalt1(t *testing.T) {
incrementRound(vs2, vs3, vs4)
// timeout to new round
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.config.Precommit(round).Nanoseconds())
ensureNewTimeout(t, timeoutWaitCh, height, round, cs1.state.ConsensusParams.Timeout.VoteTimeout(round).Nanoseconds())
round++ // moving to the next round


+ 22
- 0
internal/test/factory/params.go View File

@ -0,0 +1,22 @@
package factory
import (
"time"
"github.com/tendermint/tendermint/types"
)
// ConsensusParams returns a default set of ConsensusParams that are suitable
// for use in testing
func ConsensusParams() *types.ConsensusParams {
c := types.DefaultConsensusParams()
c.Timeout = types.TimeoutParams{
Commit: 10 * time.Millisecond,
Propose: 40 * time.Millisecond,
ProposeDelta: 1 * time.Millisecond,
Vote: 10 * time.Millisecond,
VoteDelta: 1 * time.Millisecond,
BypassCommitTimeout: true,
}
return c
}

+ 1
- 1
node/node_test.go View File

@ -751,7 +751,7 @@ func loadStatefromGenesis(ctx context.Context, t *testing.T) sm.State {
require.True(t, loadedState.IsEmpty())
valSet, _ := factory.ValidatorSet(ctx, t, 0, 10)
genDoc := factory.GenesisDoc(cfg, time.Now(), valSet.Validators, nil)
genDoc := factory.GenesisDoc(cfg, time.Now(), valSet.Validators, factory.ConsensusParams())
state, err := loadStateFromDBOrGenesisDocProvider(
stateStore,


+ 21
- 0
types/params.go View File

@ -161,6 +161,27 @@ func DefaultTimeoutParams() TimeoutParams {
}
}
// ProposeTimeout returns the amount of time to wait for a proposal.
func (t TimeoutParams) ProposeTimeout(round int32) time.Duration {
return time.Duration(
t.Propose.Nanoseconds()+t.ProposeDelta.Nanoseconds()*int64(round),
) * time.Nanosecond
}
// VoteTimeout returns the amount of time to wait for remaining votes after receiving any +2/3 votes.
func (t TimeoutParams) VoteTimeout(round int32) time.Duration {
return time.Duration(
t.Vote.Nanoseconds()+t.VoteDelta.Nanoseconds()*int64(round),
) * time.Nanosecond
}
// CommitTime accepts ti, the time at which the consensus engine received +2/3
// precommits for a block and returns the point in time at which the consensus
// engine should begin consensus on the next block.
func (t TimeoutParams) CommitTime(ti time.Time) time.Time {
return ti.Add(t.Commit)
}
func (val *ValidatorParams) IsValidPubkeyType(pubkeyType string) bool {
for i := 0; i < len(val.PubKeyTypes); i++ {
if val.PubKeyTypes[i] == pubkeyType {


Loading…
Cancel
Save