Compare commits

...

7 Commits

Author SHA1 Message Date
William Banfield
6be5efaaa9 consensus: add calculation for proposal step waits from pbts (#7290)
* initial proposerWaitsUntil implementation

* switch to duration for easier use with timeout scheduling

* add proposal step waiting time with tests

* minor aesthetic change to IsTimely

* minor language fix

* Update internal/consensus/state.go

Co-authored-by: M. J. Fromberger <fromberger@interchain.io>

* reword comment

* change accuracy to precision

* move tests to separate pbts test file

Co-authored-by: M. J. Fromberger <fromberger@interchain.io>
2021-11-24 18:47:34 -05:00
William Banfield
24c40b5b7b consensus: refactor the fake validator to take a clock source (#7300) 2021-11-24 18:46:46 -05:00
William Banfield
d4e712e4f1 factory: simplify validator and genesis factory functions (#7305) 2021-11-24 18:27:30 -05:00
William Banfield
15d6aefaf9 state: add an 'IsTimely' method to implement the 'timely' check for proposer-based timestamps (#7170)
* state: add an IsTimely function to implement the check for timely in proposer-based timestamps

* move time checks into block.go and add time source mechanism

* timestamp params comment

* add todo related to pbts spec and timestamp params

* remove old istimely

* switch to using built in before function

* lint++

* wip

* move into proposal and create a default set of params

* defer using default cons params for now
2021-11-24 17:59:25 -05:00
William Banfield
0c3401c5a4 consensus: update state to prevote nil when proposal block does not match locked block. (#6986)
* add failing test

* tweak comments in failing test

* failing test comment

* initial attempt at removing prevote locked block logic

* comment out broken function

* undo reset on prevotes

* fixing TestProposeValidBlock test

* update test for completed POL update

* comment updates

* further unlock testing

* update comments

* Update internal/consensus/state.go

* spacing nit

* comment cleanup

* nil check in addVote

* update unlock description

* update precommit on relock comment

* add ensure new timeout back

* rename IsZero to IsNil and replace uses of block len check with helper

* add testing.T to new assertions

* begin removing unlock condition

* fix TestStateProposerSelection2 to precommit for nil correctly

* remove erroneous sleep

* update TestStatePOL comment

* update relock test to be more clear

* add _ into test names

* rename slashing

* udpate no relock function to be cleaner

* do not relock on old proposal test cleanup

* con state name update

* remove all references to unlock

* update test comments to include new

* add relock test

* add ensureRelock to common_test

* remove all event unlock

* remove unlock checks

* no lint add space

* lint ++

* add test for nil prevote on different proposal

* fix prevote nil condition

* fix defaultDoPrevote

* state_test.go fixes to accomodate prevoting for nil

* add failing test for POL from previous round case

* update prevote logic to prevote POL from previous round

* state.go comment fixes

* update validatePrevotes to correctly look for nil

* update new test name and comment

* update POLFromPreviousRound test

* fixes post merge

* fix spacing

* make the linter happy

* change prevote log message

* update prevote nil debug line

* update enterPrevote comment

* lint

* Update internal/consensus/state.go

Co-authored-by: Dev Ojha <ValarDragon@users.noreply.github.com>

* Update internal/consensus/state.go

Co-authored-by: Dev Ojha <ValarDragon@users.noreply.github.com>

* add english description of alg rules

* Update internal/consensus/state.go

Co-authored-by: Dev Ojha <ValarDragon@users.noreply.github.com>

* comment fixes from review

* fix comment

* fix comment

Co-authored-by: Dev Ojha <ValarDragon@users.noreply.github.com>
2021-11-24 17:58:24 -05:00
William Banfield
d42addfe9c consensus: remove logic to unlock block on 2/3 prevote for nil (#6954) 2021-11-24 17:33:12 -05:00
William Banfield
765e1e313c consensus: remove panics from test helper functions (#6969) 2021-11-24 16:51:08 -05:00
37 changed files with 1863 additions and 955 deletions

View File

@@ -200,7 +200,8 @@ func TestReactor_AbruptDisconnect(t *testing.T) {
require.NoError(t, err)
defer os.RemoveAll(cfg.RootDir)
genDoc, privVals := factory.RandGenesisDoc(cfg, 1, false, 30)
valSet, privVals := factory.ValidatorSet(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)
@@ -239,7 +240,8 @@ func TestReactor_SyncTime(t *testing.T) {
require.NoError(t, err)
defer os.RemoveAll(cfg.RootDir)
genDoc, privVals := factory.RandGenesisDoc(cfg, 1, false, 30)
valSet, privVals := factory.ValidatorSet(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)
@@ -264,10 +266,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(cfg, 1, false, 30)
valSet, privVals := factory.ValidatorSet(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)
@@ -319,7 +321,8 @@ func TestReactor_BadBlockStopsPeer(t *testing.T) {
defer os.RemoveAll(cfg.RootDir)
maxBlockHeight := int64(48)
genDoc, privVals := factory.RandGenesisDoc(cfg, 1, false, 30)
valSet, privVals := factory.ValidatorSet(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)
@@ -353,7 +356,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(cfg, 1, false, 30)
valSet, otherPrivVals := factory.ValidatorSet(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),

View File

@@ -7,6 +7,7 @@ import (
"path"
"sync"
"testing"
"time"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
@@ -42,7 +43,8 @@ func TestByzantinePrevoteEquivocation(t *testing.T) {
tickerFunc := newMockTickerFunc(true)
appFunc := newKVStore
genDoc, privVals := factory.RandGenesisDoc(config, nValidators, false, 30)
valSet, privVals := factory.ValidatorSet(nValidators, 30)
genDoc := factory.GenesisDoc(config, time.Now(), valSet.Validators, nil)
states := make([]*State, nValidators)
for i := 0; i < nValidators; i++ {
@@ -59,7 +61,7 @@ func TestByzantinePrevoteEquivocation(t *testing.T) {
defer os.RemoveAll(thisConfig.RootDir)
ensureDir(path.Dir(thisConfig.Consensus.WalFile()), 0700) // dir for wal
ensureDir(t, path.Dir(thisConfig.Consensus.WalFile()), 0700) // dir for wal
app := appFunc()
vals := types.TM2PB.ValidatorUpdates(state.Validators)
app.InitChain(abci.RequestInitChain{Validators: vals})

View File

@@ -73,9 +73,10 @@ func configSetup(t *testing.T) *config.Config {
return cfg
}
func ensureDir(dir string, mode os.FileMode) {
func ensureDir(t *testing.T, dir string, mode os.FileMode) {
t.Helper()
if err := tmos.EnsureDir(dir, mode); err != nil {
panic(err)
t.Fatalf("error opening directory: %s", err)
}
}
@@ -90,6 +91,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
@@ -102,16 +104,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 {
@@ -123,12 +124,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)
}
@@ -148,13 +149,11 @@ func (vs *validatorStub) signVote(
func signVote(
ctx context.Context,
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)
if err != nil {
panic(fmt.Errorf("failed to sign vote: %v", err))
}
@@ -166,14 +165,13 @@ func signVote(
func signVotes(
ctx context.Context,
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, vs, cfg, voteType, hash, header)
votes[i] = signVote(ctx, vs, voteType, chainID, blockID)
}
return votes
}
@@ -231,18 +229,20 @@ func startTestRound(ctx context.Context, cs *State, height int64, round int32) {
// Create proposal block from cs1 but sign it with vs.
func decideProposal(
ctx context.Context,
t *testing.T,
cs1 *State,
vs *validatorStub,
height int64,
round int32,
) (proposal *types.Proposal, block *types.Block) {
t.Helper()
cs1.mtx.Lock()
block, blockParts := cs1.createProposalBlock()
validRound := cs1.ValidRound
chainID := cs1.state.ChainID
cs1.mtx.Unlock()
if block == nil {
panic("Failed to createProposalBlock. Did you forget to add commit for previous block?")
t.Fatal("Failed to createProposalBlock. Did you forget to add commit for previous block?")
}
// Make proposal
@@ -250,7 +250,7 @@ func decideProposal(
proposal = types.NewProposal(height, round, polRound, propBlockID)
p := proposal.ToProto()
if err := vs.SignProposal(ctx, chainID, p); err != nil {
panic(err)
t.Fatalf("error signing proposal: %s", err)
}
proposal.Signature = p.Signature
@@ -266,54 +266,48 @@ func addVotes(to *State, votes ...*types.Vote) {
func signAddVotes(
ctx context.Context,
cfg *config.Config,
to *State,
voteType tmproto.SignedMsgType,
hash []byte,
header types.PartSetHeader,
chainID string,
blockID types.BlockID,
vss ...*validatorStub,
) {
addVotes(to, signVotes(ctx, cfg, voteType, hash, header, vss...)...)
addVotes(to, signVotes(ctx, voteType, chainID, blockID, vss...)...)
}
func validatePrevote(
ctx context.Context,
t *testing.T,
cs *State,
round int32,
privVal *validatorStub,
blockHash []byte,
) {
func validatePrevote(ctx context.Context, t *testing.T, cs *State, round int32, privVal *validatorStub, blockHash []byte) {
t.Helper()
prevotes := cs.Votes.Prevotes(round)
pubKey, err := privVal.GetPubKey(ctx)
require.NoError(t, err)
address := pubKey.Address()
var vote *types.Vote
if vote = prevotes.GetByAddress(address); vote == nil {
panic("Failed to find prevote from validator")
t.Fatalf("Failed to find prevote from validator")
}
if blockHash == nil {
if vote.BlockID.Hash != nil {
panic(fmt.Sprintf("Expected prevote to be for nil, got %X", vote.BlockID.Hash))
t.Fatalf("Expected prevote to be for nil, got %X", vote.BlockID.Hash)
}
} else {
if !bytes.Equal(vote.BlockID.Hash, blockHash) {
panic(fmt.Sprintf("Expected prevote to be for %X, got %X", blockHash, vote.BlockID.Hash))
t.Fatalf("Expected prevote to be for %X, got %X", blockHash, vote.BlockID.Hash)
}
}
}
func validateLastPrecommit(ctx context.Context, t *testing.T, cs *State, privVal *validatorStub, blockHash []byte) {
t.Helper()
votes := cs.LastCommit
pv, err := privVal.GetPubKey(ctx)
require.NoError(t, err)
address := pv.Address()
var vote *types.Vote
if vote = votes.GetByAddress(address); vote == nil {
panic("Failed to find precommit from validator")
t.Fatalf("Failed to find precommit from validator")
}
if !bytes.Equal(vote.BlockID.Hash, blockHash) {
panic(fmt.Sprintf("Expected precommit to be for %X, got %X", blockHash, vote.BlockID.Hash))
t.Fatalf("Expected precommit to be for %X, got %X", blockHash, vote.BlockID.Hash)
}
}
@@ -327,41 +321,42 @@ func validatePrecommit(
votedBlockHash,
lockedBlockHash []byte,
) {
t.Helper()
precommits := cs.Votes.Precommits(thisRound)
pv, err := privVal.GetPubKey(ctx)
require.NoError(t, err)
address := pv.Address()
var vote *types.Vote
if vote = precommits.GetByAddress(address); vote == nil {
panic("Failed to find precommit from validator")
t.Fatalf("Failed to find precommit from validator")
}
if votedBlockHash == nil {
if vote.BlockID.Hash != nil {
panic("Expected precommit to be for nil")
t.Fatalf("Expected precommit to be for nil")
}
} else {
if !bytes.Equal(vote.BlockID.Hash, votedBlockHash) {
panic("Expected precommit to be for proposal block")
t.Fatalf("Expected precommit to be for proposal block")
}
}
if lockedBlockHash == nil {
if cs.LockedRound != lockRound || cs.LockedBlock != nil {
panic(fmt.Sprintf(
t.Fatalf(
"Expected to be locked on nil at round %d. Got locked at round %d with block %v",
lockRound,
cs.LockedRound,
cs.LockedBlock))
cs.LockedBlock)
}
} else {
if cs.LockedRound != lockRound || !bytes.Equal(cs.LockedBlock.Hash(), lockedBlockHash) {
panic(fmt.Sprintf(
t.Fatalf(
"Expected block to be locked on round %d, got %d. Got locked block %X, expected %X",
lockRound,
cs.LockedRound,
cs.LockedBlock.Hash(),
lockedBlockHash))
lockedBlockHash)
}
}
}
@@ -376,6 +371,7 @@ func validatePrevoteAndPrecommit(
votedBlockHash,
lockedBlockHash []byte,
) {
t.Helper()
// verify the prevote
validatePrevote(ctx, t, cs, thisRound, privVal, votedBlockHash)
// verify precommit
@@ -479,26 +475,24 @@ func newStateWithConfigAndBlockStore(
return cs
}
func loadPrivValidator(cfg *config.Config) *privval.FilePV {
privValidatorKeyFile := cfg.PrivValidator.KeyFile()
ensureDir(filepath.Dir(privValidatorKeyFile), 0700)
privValidatorStateFile := cfg.PrivValidator.StateFile()
func loadPrivValidator(t *testing.T, config *config.Config) *privval.FilePV {
t.Helper()
privValidatorKeyFile := config.PrivValidator.KeyFile()
ensureDir(t, filepath.Dir(privValidatorKeyFile), 0700)
privValidatorStateFile := config.PrivValidator.StateFile()
privValidator, err := privval.LoadOrGenFilePV(privValidatorKeyFile, privValidatorStateFile)
if err != nil {
panic(err)
t.Fatalf("error generating validator file: %s", err)
}
privValidator.Reset()
return privValidator
}
func randState(
ctx context.Context,
cfg *config.Config,
logger log.Logger,
nValidators int,
) (*State, []*validatorStub, error) {
func makeState(ctx context.Context, cfg *config.Config, logger log.Logger, nValidators int) (*State, []*validatorStub, error) {
// Get State
state, privVals := randGenesisState(cfg, nValidators, false, 10)
state, privVals := makeGenesisState(cfg, genesisStateArgs{
Validators: nValidators,
})
vss := make([]*validatorStub, nValidators)
@@ -518,220 +512,238 @@ func randState(
//-------------------------------------------------------------------------------
func ensureNoNewEvent(ch <-chan tmpubsub.Message, timeout time.Duration,
func ensureNoNewEvent(t *testing.T, ch <-chan tmpubsub.Message, timeout time.Duration,
errorMessage string) {
t.Helper()
select {
case <-time.After(timeout):
break
case <-ch:
panic(errorMessage)
t.Fatalf("unexpected event: %s", errorMessage)
}
}
func ensureNoNewEventOnChannel(ch <-chan tmpubsub.Message) {
func ensureNoNewEventOnChannel(t *testing.T, ch <-chan tmpubsub.Message) {
t.Helper()
ensureNoNewEvent(
t,
ch,
ensureTimeout,
"We should be stuck waiting, not receiving new event on the channel")
}
func ensureNoNewRoundStep(stepCh <-chan tmpubsub.Message) {
func ensureNoNewRoundStep(t *testing.T, stepCh <-chan tmpubsub.Message) {
t.Helper()
ensureNoNewEvent(
t,
stepCh,
ensureTimeout,
"We should be stuck waiting, not receiving NewRoundStep event")
}
func ensureNoNewUnlock(unlockCh <-chan tmpubsub.Message) {
ensureNoNewEvent(
unlockCh,
ensureTimeout,
"We should be stuck waiting, not receiving Unlock event")
}
func ensureNoNewTimeout(stepCh <-chan tmpubsub.Message, timeout int64) {
func ensureNoNewTimeout(t *testing.T, stepCh <-chan tmpubsub.Message, timeout int64) {
t.Helper()
timeoutDuration := time.Duration(timeout*10) * time.Nanosecond
ensureNoNewEvent(
t,
stepCh,
timeoutDuration,
"We should be stuck waiting, not receiving NewTimeout event")
}
func ensureNewEvent(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, errorMessage string) { // nolint: lll
t.Helper()
select {
case <-time.After(timeout):
panic(errorMessage)
t.Fatalf("timed out waiting for new event: %s", errorMessage)
case msg := <-ch:
roundStateEvent, ok := msg.Data().(types.EventDataRoundState)
if !ok {
panic(fmt.Sprintf("expected a EventDataRoundState, got %T. Wrong subscription channel?",
msg.Data()))
t.Fatalf("expected a EventDataRoundState, got %T. Wrong subscription channel?", msg.Data())
}
if roundStateEvent.Height != height {
panic(fmt.Sprintf("expected height %v, got %v", height, roundStateEvent.Height))
t.Fatalf("expected height %v, got %v", height, roundStateEvent.Height)
}
if roundStateEvent.Round != round {
panic(fmt.Sprintf("expected round %v, got %v", round, roundStateEvent.Round))
t.Fatalf("expected round %v, got %v", round, roundStateEvent.Round)
}
// TODO: We could check also for a step at this point!
}
}
func ensureNewRound(roundCh <-chan tmpubsub.Message, height int64, round int32) {
func ensureNewRound(t *testing.T, roundCh <-chan tmpubsub.Message, height int64, round int32) {
t.Helper()
select {
case <-time.After(ensureTimeout):
panic("Timeout expired while waiting for NewRound event")
t.Fatal("Timeout expired while waiting for NewRound event")
case msg := <-roundCh:
newRoundEvent, ok := msg.Data().(types.EventDataNewRound)
if !ok {
panic(fmt.Sprintf("expected a EventDataNewRound, got %T. Wrong subscription channel?",
msg.Data()))
t.Fatalf("expected a EventDataNewRound, got %T. Wrong subscription channel?", msg.Data())
}
if newRoundEvent.Height != height {
panic(fmt.Sprintf("expected height %v, got %v", height, newRoundEvent.Height))
t.Fatalf("expected height %v, got %v", height, newRoundEvent.Height)
}
if newRoundEvent.Round != round {
panic(fmt.Sprintf("expected round %v, got %v", round, newRoundEvent.Round))
t.Fatalf("expected round %v, got %v", round, newRoundEvent.Round)
}
}
}
func ensureNewTimeout(timeoutCh <-chan tmpubsub.Message, height int64, round int32, timeout int64) {
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(timeoutCh, height, round, timeoutDuration,
ensureNewEvent(t, timeoutCh, height, round, timeoutDuration,
"Timeout expired while waiting for NewTimeout event")
}
func ensureNewProposal(proposalCh <-chan tmpubsub.Message, height int64, round int32) {
func ensureNewProposal(t *testing.T, proposalCh <-chan tmpubsub.Message, height int64, round int32) {
t.Helper()
select {
case <-time.After(ensureTimeout):
panic("Timeout expired while waiting for NewProposal event")
t.Fatalf("Timeout expired while waiting for NewProposal event")
case msg := <-proposalCh:
proposalEvent, ok := msg.Data().(types.EventDataCompleteProposal)
if !ok {
panic(fmt.Sprintf("expected a EventDataCompleteProposal, got %T. Wrong subscription channel?",
msg.Data()))
t.Fatalf("expected a EventDataCompleteProposal, got %T. Wrong subscription channel?",
msg.Data())
}
if proposalEvent.Height != height {
panic(fmt.Sprintf("expected height %v, got %v", height, proposalEvent.Height))
t.Fatalf("expected height %v, got %v", height, proposalEvent.Height)
}
if proposalEvent.Round != round {
panic(fmt.Sprintf("expected round %v, got %v", round, proposalEvent.Round))
t.Fatalf("expected round %v, got %v", round, proposalEvent.Round)
}
}
}
func ensureNewValidBlock(validBlockCh <-chan tmpubsub.Message, height int64, round int32) {
ensureNewEvent(validBlockCh, height, round, ensureTimeout,
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")
}
func ensureNewBlock(blockCh <-chan tmpubsub.Message, height int64) {
func ensureNewBlock(t *testing.T, blockCh <-chan tmpubsub.Message, height int64) {
t.Helper()
select {
case <-time.After(ensureTimeout):
panic("Timeout expired while waiting for NewBlock event")
t.Fatalf("Timeout expired while waiting for NewBlock event")
case msg := <-blockCh:
blockEvent, ok := msg.Data().(types.EventDataNewBlock)
if !ok {
panic(fmt.Sprintf("expected a EventDataNewBlock, got %T. Wrong subscription channel?",
msg.Data()))
t.Fatalf("expected a EventDataNewBlock, got %T. Wrong subscription channel?",
msg.Data())
}
if blockEvent.Block.Height != height {
panic(fmt.Sprintf("expected height %v, got %v", height, blockEvent.Block.Height))
t.Fatalf("expected height %v, got %v", height, blockEvent.Block.Height)
}
}
}
func ensureNewBlockHeader(blockCh <-chan tmpubsub.Message, height int64, blockHash tmbytes.HexBytes) {
func ensureNewBlockHeader(t *testing.T, blockCh <-chan tmpubsub.Message, height int64, blockHash tmbytes.HexBytes) {
t.Helper()
select {
case <-time.After(ensureTimeout):
panic("Timeout expired while waiting for NewBlockHeader event")
t.Fatalf("Timeout expired while waiting for NewBlockHeader event")
case msg := <-blockCh:
blockHeaderEvent, ok := msg.Data().(types.EventDataNewBlockHeader)
if !ok {
panic(fmt.Sprintf("expected a EventDataNewBlockHeader, got %T. Wrong subscription channel?",
msg.Data()))
t.Fatalf("expected a EventDataNewBlockHeader, got %T. Wrong subscription channel?",
msg.Data())
}
if blockHeaderEvent.Header.Height != height {
panic(fmt.Sprintf("expected height %v, got %v", height, blockHeaderEvent.Header.Height))
t.Fatalf("expected height %v, got %v", height, blockHeaderEvent.Header.Height)
}
if !bytes.Equal(blockHeaderEvent.Header.Hash(), blockHash) {
panic(fmt.Sprintf("expected header %X, got %X", blockHash, blockHeaderEvent.Header.Hash()))
t.Fatalf("expected header %X, got %X", blockHash, blockHeaderEvent.Header.Hash())
}
}
}
func ensureNewUnlock(unlockCh <-chan tmpubsub.Message, height int64, round int32) {
ensureNewEvent(unlockCh, height, round, ensureTimeout,
"Timeout expired while waiting for NewUnlock event")
func ensureLock(t *testing.T, lockCh <-chan tmpubsub.Message, height int64, round int32) {
t.Helper()
ensureNewEvent(t, lockCh, height, round, ensureTimeout,
"Timeout expired while waiting for LockValue event")
}
func ensureProposal(proposalCh <-chan tmpubsub.Message, height int64, round int32, propID types.BlockID) {
func ensureRelock(t *testing.T, relockCh <-chan tmpubsub.Message, height int64, round int32) {
t.Helper()
ensureNewEvent(t, relockCh, height, round, ensureTimeout,
"Timeout expired while waiting for RelockValue event")
}
func ensureProposal(t *testing.T, proposalCh <-chan tmpubsub.Message, height int64, round int32, propID types.BlockID) {
t.Helper()
select {
case <-time.After(ensureTimeout):
panic("Timeout expired while waiting for NewProposal event")
t.Fatalf("Timeout expired while waiting for NewProposal event")
case msg := <-proposalCh:
proposalEvent, ok := msg.Data().(types.EventDataCompleteProposal)
if !ok {
panic(fmt.Sprintf("expected a EventDataCompleteProposal, got %T. Wrong subscription channel?",
msg.Data()))
t.Fatalf("expected a EventDataCompleteProposal, got %T. Wrong subscription channel?",
msg.Data())
}
if proposalEvent.Height != height {
panic(fmt.Sprintf("expected height %v, got %v", height, proposalEvent.Height))
t.Fatalf("expected height %v, got %v", height, proposalEvent.Height)
}
if proposalEvent.Round != round {
panic(fmt.Sprintf("expected round %v, got %v", round, proposalEvent.Round))
t.Fatalf("expected round %v, got %v", round, proposalEvent.Round)
}
if !proposalEvent.BlockID.Equals(propID) {
panic(fmt.Sprintf("Proposed block does not match expected block (%v != %v)", proposalEvent.BlockID, propID))
t.Fatalf("Proposed block does not match expected block (%v != %v)", proposalEvent.BlockID, propID)
}
}
}
func ensurePrecommit(voteCh <-chan tmpubsub.Message, height int64, round int32) {
ensureVote(voteCh, height, round, tmproto.PrecommitType)
func ensurePrecommit(t *testing.T, voteCh <-chan tmpubsub.Message, height int64, round int32) {
t.Helper()
ensureVote(t, voteCh, height, round, tmproto.PrecommitType)
}
func ensurePrevote(voteCh <-chan tmpubsub.Message, height int64, round int32) {
ensureVote(voteCh, height, round, tmproto.PrevoteType)
func ensurePrevote(t *testing.T, voteCh <-chan tmpubsub.Message, height int64, round int32) {
t.Helper()
ensureVote(t, voteCh, height, round, tmproto.PrevoteType)
}
func ensureVote(voteCh <-chan tmpubsub.Message, height int64, round int32,
func ensureVote(t *testing.T, voteCh <-chan tmpubsub.Message, height int64, round int32,
voteType tmproto.SignedMsgType) {
t.Helper()
select {
case <-time.After(ensureTimeout):
panic("Timeout expired while waiting for NewVote event")
t.Fatalf("Timeout expired while waiting for NewVote event")
case msg := <-voteCh:
voteEvent, ok := msg.Data().(types.EventDataVote)
if !ok {
panic(fmt.Sprintf("expected a EventDataVote, got %T. Wrong subscription channel?",
msg.Data()))
t.Fatalf("expected a EventDataVote, got %T. Wrong subscription channel?",
msg.Data())
}
vote := voteEvent.Vote
if vote.Height != height {
panic(fmt.Sprintf("expected height %v, got %v", height, vote.Height))
t.Fatalf("expected height %v, got %v", height, vote.Height)
}
if vote.Round != round {
panic(fmt.Sprintf("expected round %v, got %v", round, vote.Round))
t.Fatalf("expected round %v, got %v", round, vote.Round)
}
if vote.Type != voteType {
panic(fmt.Sprintf("expected type %v, got %v", voteType, vote.Type))
t.Fatalf("expected type %v, got %v", voteType, vote.Type)
}
}
}
func ensurePrecommitTimeout(ch <-chan tmpubsub.Message) {
func ensurePrecommitTimeout(t *testing.T, ch <-chan tmpubsub.Message) {
t.Helper()
select {
case <-time.After(ensureTimeout):
panic("Timeout expired while waiting for the Precommit to Timeout")
t.Fatalf("Timeout expired while waiting for the Precommit to Timeout")
case <-ch:
}
}
func ensureNewEventOnChannel(ch <-chan tmpubsub.Message) {
func ensureNewEventOnChannel(t *testing.T, ch <-chan tmpubsub.Message) {
t.Helper()
select {
case <-time.After(ensureTimeout):
panic("Timeout expired while waiting for new activity on the channel")
t.Fatalf("Timeout expired while waiting for new activity on the channel")
case <-ch:
}
}
@@ -745,7 +757,7 @@ func consensusLogger() log.Logger {
return log.TestingLogger().With("module", "consensus")
}
func randConsensusState(
func makeConsensusState(
ctx context.Context,
t *testing.T,
cfg *config.Config,
@@ -755,8 +767,10 @@ func randConsensusState(
appFunc func() abci.Application,
configOpts ...func(*config.Config),
) ([]*State, cleanupFunc) {
t.Helper()
genDoc, privVals := factory.RandGenesisDoc(cfg, nValidators, false, 30)
valSet, privVals := factory.ValidatorSet(nValidators, 30)
genDoc := factory.GenesisDoc(cfg, time.Now(), valSet.Validators, nil)
css := make([]*State, nValidators)
logger := consensusLogger()
@@ -777,7 +791,7 @@ func randConsensusState(
opt(thisConfig)
}
ensureDir(filepath.Dir(thisConfig.Consensus.WalFile()), 0700) // dir for wal
ensureDir(t, filepath.Dir(thisConfig.Consensus.WalFile()), 0700) // dir for wal
app := appFunc()
@@ -806,6 +820,7 @@ func randConsensusState(
// nPeers = nValidators + nNotValidator
func randConsensusNetWithPeers(
ctx context.Context,
t *testing.T,
cfg *config.Config,
nValidators,
nPeers int,
@@ -813,8 +828,10 @@ func randConsensusNetWithPeers(
tickerFunc func() TimeoutTicker,
appFunc func(string) abci.Application,
) ([]*State, *types.GenesisDoc, *config.Config, cleanupFunc) {
genDoc, privVals := factory.RandGenesisDoc(cfg, nValidators, false, testMinPower)
valSet, privVals := factory.ValidatorSet(nValidators, testMinPower)
genDoc := factory.GenesisDoc(cfg, time.Now(), valSet.Validators, nil)
css := make([]*State, nPeers)
t.Helper()
logger := consensusLogger()
var peer0Config *config.Config
@@ -827,7 +844,7 @@ func randConsensusNetWithPeers(
}
configRootDirs = append(configRootDirs, thisConfig.RootDir)
ensureDir(filepath.Dir(thisConfig.Consensus.WalFile()), 0700) // dir for wal
ensureDir(t, filepath.Dir(thisConfig.Consensus.WalFile()), 0700) // dir for wal
if i == 0 {
peer0Config = thisConfig
}
@@ -837,16 +854,16 @@ func randConsensusNetWithPeers(
} else {
tempKeyFile, err := os.CreateTemp("", "priv_validator_key_")
if err != nil {
panic(err)
t.Fatalf("error creating temp file for validator key: %s", err)
}
tempStateFile, err := os.CreateTemp("", "priv_validator_state_")
if err != nil {
panic(err)
t.Fatalf("error loading validator state: %s", err)
}
privVal, err = privval.GenFilePV(tempKeyFile.Name(), tempStateFile.Name(), "")
if err != nil {
panic(err)
t.Fatalf("error generating validator key: %s", err)
}
}
@@ -869,13 +886,28 @@ func randConsensusNetWithPeers(
}
}
func randGenesisState(
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(cfg, numValidators, randPower, minPower)
func makeGenesisState(cfg *config.Config, args genesisStateArgs) (sm.State, []types.PrivValidator) {
if args.Power == 0 {
args.Power = 1
}
if args.Validators == 0 {
args.Power = 4
}
valSet, privValidators := factory.ValidatorSet(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, _ := sm.MakeGenesisState(genDoc)
return s0, privValidators
}

View File

@@ -23,7 +23,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)

View File

@@ -37,19 +37,21 @@ func TestMempoolNoProgressUntilTxsAvailable(t *testing.T) {
t.Cleanup(func() { _ = os.RemoveAll(config.RootDir) })
config.Consensus.CreateEmptyBlocks = false
state, privVals := randGenesisState(baseConfig, 1, false, 10)
state, privVals := makeGenesisState(baseConfig, genesisStateArgs{
Validators: 1,
Power: 10})
cs := newStateWithConfig(ctx, log.TestingLogger(), config, state, privVals[0], NewCounterApplication())
assertMempool(cs.txNotifier).EnableTxsAvailable()
height, round := cs.Height, cs.Round
newBlockCh := subscribe(ctx, t, cs.eventBus, types.EventQueryNewBlock)
startTestRound(ctx, cs, height, round)
ensureNewEventOnChannel(newBlockCh) // first block gets committed
ensureNoNewEventOnChannel(newBlockCh)
ensureNewEventOnChannel(t, newBlockCh) // first block gets committed
ensureNoNewEventOnChannel(t, newBlockCh)
deliverTxsRange(ctx, cs, 0, 1)
ensureNewEventOnChannel(newBlockCh) // commit txs
ensureNewEventOnChannel(newBlockCh) // commit updated app hash
ensureNoNewEventOnChannel(newBlockCh)
ensureNewEventOnChannel(t, newBlockCh) // commit txs
ensureNewEventOnChannel(t, newBlockCh) // commit updated app hash
ensureNoNewEventOnChannel(t, newBlockCh)
}
func TestMempoolProgressAfterCreateEmptyBlocksInterval(t *testing.T) {
@@ -62,7 +64,9 @@ func TestMempoolProgressAfterCreateEmptyBlocksInterval(t *testing.T) {
t.Cleanup(func() { _ = os.RemoveAll(config.RootDir) })
config.Consensus.CreateEmptyBlocksInterval = ensureTimeout
state, privVals := randGenesisState(baseConfig, 1, false, 10)
state, privVals := makeGenesisState(baseConfig, genesisStateArgs{
Validators: 1,
Power: 10})
cs := newStateWithConfig(ctx, log.TestingLogger(), config, state, privVals[0], NewCounterApplication())
assertMempool(cs.txNotifier).EnableTxsAvailable()
@@ -70,9 +74,9 @@ func TestMempoolProgressAfterCreateEmptyBlocksInterval(t *testing.T) {
newBlockCh := subscribe(ctx, t, cs.eventBus, types.EventQueryNewBlock)
startTestRound(ctx, cs, cs.Height, cs.Round)
ensureNewEventOnChannel(newBlockCh) // first block gets committed
ensureNoNewEventOnChannel(newBlockCh) // then we dont make a block ...
ensureNewEventOnChannel(newBlockCh) // until the CreateEmptyBlocksInterval has passed
ensureNewEventOnChannel(t, newBlockCh) // first block gets committed
ensureNoNewEventOnChannel(t, newBlockCh) // then we dont make a block ...
ensureNewEventOnChannel(t, newBlockCh) // until the CreateEmptyBlocksInterval has passed
}
func TestMempoolProgressInHigherRound(t *testing.T) {
@@ -85,7 +89,9 @@ func TestMempoolProgressInHigherRound(t *testing.T) {
t.Cleanup(func() { _ = os.RemoveAll(config.RootDir) })
config.Consensus.CreateEmptyBlocks = false
state, privVals := randGenesisState(baseConfig, 1, false, 10)
state, privVals := makeGenesisState(baseConfig, genesisStateArgs{
Validators: 1,
Power: 10})
cs := newStateWithConfig(ctx, log.TestingLogger(), config, state, privVals[0], NewCounterApplication())
assertMempool(cs.txNotifier).EnableTxsAvailable()
height, round := cs.Height, cs.Round
@@ -103,19 +109,19 @@ func TestMempoolProgressInHigherRound(t *testing.T) {
}
startTestRound(ctx, cs, height, round)
ensureNewRound(newRoundCh, height, round) // first round at first height
ensureNewEventOnChannel(newBlockCh) // first block gets committed
ensureNewRound(t, newRoundCh, height, round) // first round at first height
ensureNewEventOnChannel(t, newBlockCh) // first block gets committed
height++ // moving to the next height
round = 0
ensureNewRound(newRoundCh, height, round) // first round at next height
deliverTxsRange(ctx, cs, 0, 1) // we deliver txs, but dont set a proposal so we get the next round
ensureNewTimeout(timeoutCh, height, round, cs.config.TimeoutPropose.Nanoseconds())
ensureNewRound(t, newRoundCh, height, round) // first round at next height
deliverTxsRange(ctx, cs, 0, 1) // we deliver txs, but dont set a proposal so we get the next round
ensureNewTimeout(t, timeoutCh, height, round, cs.config.TimeoutPropose.Nanoseconds())
round++ // moving to the next round
ensureNewRound(newRoundCh, height, round) // wait for the next round
ensureNewEventOnChannel(newBlockCh) // now we can commit the block
round++ // moving to the next round
ensureNewRound(t, newRoundCh, height, round) // wait for the next round
ensureNewEventOnChannel(t, newBlockCh) // now we can commit the block
}
func deliverTxsRange(ctx context.Context, cs *State, start, end int) {
@@ -136,7 +142,9 @@ func TestMempoolTxConcurrentWithCommit(t *testing.T) {
config := configSetup(t)
logger := log.TestingLogger()
state, privVals := randGenesisState(config, 1, false, 10)
state, privVals := makeGenesisState(config, genesisStateArgs{
Validators: 1,
Power: 10})
stateStore := sm.NewStore(dbm.NewMemDB())
blockStore := store.NewBlockStore(dbm.NewMemDB())
@@ -168,7 +176,9 @@ func TestMempoolRmBadTx(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
state, privVals := randGenesisState(config, 1, false, 10)
state, privVals := makeGenesisState(config, genesisStateArgs{
Validators: 1,
Power: 10})
app := NewCounterApplication()
stateStore := sm.NewStore(dbm.NewMemDB())
blockStore := store.NewBlockStore(dbm.NewMemDB())

View File

@@ -0,0 +1,115 @@
package consensus
import (
"testing"
"time"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
tmtimemocks "github.com/tendermint/tendermint/libs/time/mocks"
"github.com/tendermint/tendermint/types"
)
func TestProposerWaitTime(t *testing.T) {
genesisTime, err := time.Parse(time.RFC3339, "2019-03-13T23:00:00Z")
require.NoError(t, err)
testCases := []struct {
name string
blockTime time.Time
localTime time.Time
expectedWait time.Duration
}{
{
name: "block time greater than local time",
blockTime: genesisTime.Add(5 * time.Nanosecond),
localTime: genesisTime.Add(1 * time.Nanosecond),
expectedWait: 4 * time.Nanosecond,
},
{
name: "local time greater than block time",
blockTime: genesisTime.Add(1 * time.Nanosecond),
localTime: genesisTime.Add(5 * time.Nanosecond),
expectedWait: 0,
},
{
name: "both times equal",
blockTime: 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) {
b := types.Block{
Header: types.Header{
Time: testCase.blockTime,
},
}
mockSource := new(tmtimemocks.Source)
mockSource.On("Now").Return(testCase.localTime)
ti := proposerWaitTime(mockSource, b.Header)
assert.Equal(t, testCase.expectedWait, ti)
})
}
}
func TestProposalTimeout(t *testing.T) {
genesisTime, err := time.Parse(time.RFC3339, "2019-03-13T23:00:00Z")
require.NoError(t, err)
testCases := []struct {
name string
localTime time.Time
previousBlockTime time.Time
precision time.Duration
msgDelay time.Duration
expectedDuration time.Duration
}{
{
name: "MsgDelay + Precision has not quite elapsed",
localTime: genesisTime.Add(525 * time.Millisecond),
previousBlockTime: genesisTime.Add(6 * time.Millisecond),
precision: time.Millisecond * 20,
msgDelay: time.Millisecond * 500,
expectedDuration: 1 * time.Millisecond,
},
{
name: "MsgDelay + Precision equals current time",
localTime: genesisTime.Add(525 * time.Millisecond),
previousBlockTime: genesisTime.Add(5 * time.Millisecond),
precision: time.Millisecond * 20,
msgDelay: time.Millisecond * 500,
expectedDuration: 0,
},
{
name: "MsgDelay + Precision has elapsed",
localTime: genesisTime.Add(725 * time.Millisecond),
previousBlockTime: genesisTime.Add(5 * time.Millisecond),
precision: time.Millisecond * 20,
msgDelay: time.Millisecond * 500,
expectedDuration: 0,
},
}
for _, testCase := range testCases {
t.Run(testCase.name, func(t *testing.T) {
b := types.Block{
Header: types.Header{
Time: testCase.previousBlockTime,
},
}
mockSource := new(tmtimemocks.Source)
mockSource.On("Now").Return(testCase.localTime)
tp := types.TimestampParams{
Precision: testCase.precision,
MsgDelay: testCase.msgDelay,
}
ti := proposalStepWaitingTime(mockSource, b.Header, tp)
assert.Equal(t, testCase.expectedDuration, ti)
})
}
}

View File

@@ -311,7 +311,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)
@@ -368,7 +368,8 @@ func TestReactorWithEvidence(t *testing.T) {
tickerFunc := newMockTickerFunc(true)
appFunc := newKVStore
genDoc, privVals := factory.RandGenesisDoc(cfg, n, false, 30)
valSet, privVals := factory.ValidatorSet(n, 30)
genDoc := factory.GenesisDoc(cfg, time.Now(), valSet.Validators, nil)
states := make([]*State, n)
logger := consensusLogger()
@@ -382,7 +383,7 @@ func TestReactorWithEvidence(t *testing.T) {
defer os.RemoveAll(thisConfig.RootDir)
ensureDir(path.Dir(thisConfig.Consensus.WalFile()), 0700) // dir for wal
ensureDir(t, path.Dir(thisConfig.Consensus.WalFile()), 0700) // dir for wal
app := appFunc()
vals := types.TM2PB.ValidatorUpdates(state.Validators)
app.InitChain(abci.RequestInitChain{Validators: vals})
@@ -470,8 +471,7 @@ func TestReactorCreatesBlockWhenEmptyBlocksFalse(t *testing.T) {
cfg := configSetup(t)
n := 4
states, cleanup := randConsensusState(
ctx,
states, cleanup := makeConsensusState(ctx,
t,
cfg,
n,
@@ -527,7 +527,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)
@@ -592,8 +592,7 @@ func TestReactorVotingPowerChange(t *testing.T) {
cfg := configSetup(t)
n := 4
states, cleanup := randConsensusState(
ctx,
states, cleanup := makeConsensusState(ctx,
t,
cfg,
n,
@@ -703,6 +702,7 @@ func TestReactorValidatorSetChanges(t *testing.T) {
nVals := 4
states, _, _, cleanup := randConsensusNetWithPeers(
ctx,
t,
cfg,
nVals,
nPeers,

View File

@@ -61,7 +61,7 @@ func startNewStateAndWaitForBlock(ctx context.Context, t *testing.T, consensusRe
logger := log.TestingLogger()
state, err := sm.MakeGenesisStateFromFile(consensusReplayConfig.GenesisFile())
require.NoError(t, err)
privValidator := loadPrivValidator(consensusReplayConfig)
privValidator := loadPrivValidator(t, consensusReplayConfig)
blockStore := store.NewBlockStore(dbm.NewMemDB())
cs := newStateWithConfigAndBlockStore(
ctx,
@@ -166,7 +166,7 @@ LOOP:
blockStore := store.NewBlockStore(blockDB)
state, err := sm.MakeGenesisStateFromFile(consensusReplayConfig.GenesisFile())
require.NoError(t, err)
privValidator := loadPrivValidator(consensusReplayConfig)
privValidator := loadPrivValidator(t, consensusReplayConfig)
cs := newStateWithConfigAndBlockStore(
ctx,
logger,
@@ -335,6 +335,7 @@ func setupSimulator(ctx context.Context, t *testing.T) *simulatorTestSuite {
css, genDoc, cfg, cleanup := randConsensusNetWithPeers(
ctx,
t,
cfg,
nVals,
nPeers,
@@ -359,15 +360,15 @@ func setupSimulator(ctx context.Context, t *testing.T) *simulatorTestSuite {
// start the machine
startTestRound(ctx, css[0], height, round)
incrementHeight(vss...)
ensureNewRound(newRoundCh, height, 0)
ensureNewProposal(proposalCh, height, round)
ensureNewRound(t, newRoundCh, height, 0)
ensureNewProposal(t, proposalCh, height, round)
rs := css[0].GetRoundState()
signAddVotes(ctx, sim.Config, css[0], tmproto.PrecommitType,
rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header(),
signAddVotes(ctx, css[0], tmproto.PrecommitType, sim.Config.ChainID(),
types.BlockID{Hash: rs.ProposalBlock.Hash(), PartSetHeader: rs.ProposalBlockParts.Header()},
vss[1:nVals]...)
ensureNewRound(newRoundCh, height+1, 0)
ensureNewRound(t, newRoundCh, height+1, 0)
// HEIGHT 2
height++
@@ -394,12 +395,12 @@ func setupSimulator(ctx context.Context, t *testing.T) *simulatorTestSuite {
if err := css[0].SetProposalAndBlock(proposal, propBlock, propBlockParts, "some peer"); err != nil {
t.Fatal(err)
}
ensureNewProposal(proposalCh, height, round)
ensureNewProposal(t, proposalCh, height, round)
rs = css[0].GetRoundState()
signAddVotes(ctx, sim.Config, css[0], tmproto.PrecommitType,
rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header(),
signAddVotes(ctx, css[0], tmproto.PrecommitType, sim.Config.ChainID(),
types.BlockID{Hash: rs.ProposalBlock.Hash(), PartSetHeader: rs.ProposalBlockParts.Header()},
vss[1:nVals]...)
ensureNewRound(newRoundCh, height+1, 0)
ensureNewRound(t, newRoundCh, height+1, 0)
// HEIGHT 3
height++
@@ -426,12 +427,12 @@ func setupSimulator(ctx context.Context, t *testing.T) *simulatorTestSuite {
if err := css[0].SetProposalAndBlock(proposal, propBlock, propBlockParts, "some peer"); err != nil {
t.Fatal(err)
}
ensureNewProposal(proposalCh, height, round)
ensureNewProposal(t, proposalCh, height, round)
rs = css[0].GetRoundState()
signAddVotes(ctx, sim.Config, css[0], tmproto.PrecommitType,
rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header(),
signAddVotes(ctx, css[0], tmproto.PrecommitType, sim.Config.ChainID(),
types.BlockID{Hash: rs.ProposalBlock.Hash(), PartSetHeader: rs.ProposalBlockParts.Header()},
vss[1:nVals]...)
ensureNewRound(newRoundCh, height+1, 0)
ensureNewRound(t, newRoundCh, height+1, 0)
// HEIGHT 4
height++
@@ -485,7 +486,7 @@ func setupSimulator(ctx context.Context, t *testing.T) *simulatorTestSuite {
if err := css[0].SetProposalAndBlock(proposal, propBlock, propBlockParts, "some peer"); err != nil {
t.Fatal(err)
}
ensureNewProposal(proposalCh, height, round)
ensureNewProposal(t, proposalCh, height, round)
removeValidatorTx2 := kvstore.MakeValSetChangeTx(newVal2ABCI, 0)
err = assertMempool(css[0].txNotifier).CheckTx(ctx, removeValidatorTx2, nil, mempool.TxInfo{})
@@ -496,12 +497,13 @@ func setupSimulator(ctx context.Context, t *testing.T) *simulatorTestSuite {
if i == selfIndex {
continue
}
signAddVotes(ctx, sim.Config, css[0],
tmproto.PrecommitType, rs.ProposalBlock.Hash(),
rs.ProposalBlockParts.Header(), newVss[i])
signAddVotes(ctx, css[0],
tmproto.PrecommitType, sim.Config.ChainID(),
types.BlockID{Hash: rs.ProposalBlock.Hash(), PartSetHeader: rs.ProposalBlockParts.Header()},
newVss[i])
}
ensureNewRound(newRoundCh, height+1, 0)
ensureNewRound(t, newRoundCh, height+1, 0)
// HEIGHT 5
height++
@@ -511,17 +513,18 @@ func setupSimulator(ctx context.Context, t *testing.T) *simulatorTestSuite {
newVss[newVssIdx].VotingPower = 25
sort.Sort(ValidatorStubsByPower(newVss))
selfIndex = valIndexFn(0)
ensureNewProposal(proposalCh, height, round)
ensureNewProposal(t, proposalCh, height, round)
rs = css[0].GetRoundState()
for i := 0; i < nVals+1; i++ {
if i == selfIndex {
continue
}
signAddVotes(ctx, sim.Config, css[0],
tmproto.PrecommitType, rs.ProposalBlock.Hash(),
rs.ProposalBlockParts.Header(), newVss[i])
signAddVotes(ctx, css[0],
tmproto.PrecommitType, sim.Config.ChainID(),
types.BlockID{Hash: rs.ProposalBlock.Hash(), PartSetHeader: rs.ProposalBlockParts.Header()},
newVss[i])
}
ensureNewRound(newRoundCh, height+1, 0)
ensureNewRound(t, newRoundCh, height+1, 0)
// HEIGHT 6
height++
@@ -548,17 +551,18 @@ func setupSimulator(ctx context.Context, t *testing.T) *simulatorTestSuite {
if err := css[0].SetProposalAndBlock(proposal, propBlock, propBlockParts, "some peer"); err != nil {
t.Fatal(err)
}
ensureNewProposal(proposalCh, height, round)
ensureNewProposal(t, proposalCh, height, round)
rs = css[0].GetRoundState()
for i := 0; i < nVals+3; i++ {
if i == selfIndex {
continue
}
signAddVotes(ctx, sim.Config, css[0],
tmproto.PrecommitType, rs.ProposalBlock.Hash(),
rs.ProposalBlockParts.Header(), newVss[i])
signAddVotes(ctx, css[0],
tmproto.PrecommitType, sim.Config.ChainID(),
types.BlockID{Hash: rs.ProposalBlock.Hash(), PartSetHeader: rs.ProposalBlockParts.Header()},
newVss[i])
}
ensureNewRound(newRoundCh, height+1, 0)
ensureNewRound(t, newRoundCh, height+1, 0)
sim.Chain = make([]*types.Block, 0)
sim.Commits = make([]*types.Commit, 0)
@@ -1273,7 +1277,8 @@ func TestHandshakeUpdatesValidators(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
val, _ := factory.RandValidator(true, 10)
votePower := 10 + int64(rand.Uint32())
val, _ := factory.Validator(votePower)
vals := types.NewValidatorSet([]*types.Validator{val})
app := &initChainApp{vals: types.TM2PB.ValidatorUpdates(vals)}
clientCreator := abciclient.NewLocalCreator(app)

View File

@@ -1269,8 +1269,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(height int64, round int32) {
logger := cs.Logger.With("height", height, "round", round)
@@ -1300,14 +1303,7 @@ func (cs *State) enterPrevote(height int64, round int32) {
func (cs *State) defaultDoPrevote(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(tmproto.PrevoteType, cs.LockedBlock.Hash(), cs.LockedBlockParts.Header())
return
}
// If ProposalBlock is nil, prevote nil.
// We did not receive a proposal within this round. (and thus executing this from a timeout)
if cs.ProposalBlock == nil {
logger.Debug("prevote step: ProposalBlock is nil")
cs.signAddVote(tmproto.PrevoteType, nil, types.PartSetHeader{})
@@ -1323,11 +1319,67 @@ func (cs *State) defaultDoPrevote(height int64, round int32) {
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(tmproto.PrevoteType, cs.ProposalBlock.Hash(), cs.ProposalBlockParts.Header())
/*
22: upon <PROPOSAL, h_p, round_p, v, 1> from proposer(h_p, round_p) while step_p = propose do
23: if valid(v) && (lockedRound_p = 1 || lockedValue_p = v) then
24: broadcast <PREVOTE, h_p, round_p, id(v)>
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(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(tmproto.PrevoteType, cs.ProposalBlock.Hash(), cs.ProposalBlockParts.Header())
return
}
}
/*
28: upon <PROPOSAL, h_p, round_p, v, v_r> from proposer(h_p, round_p) AND 2f + 1 <PREVOTE, h_p, v_r, id(v)> 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 <PREVOTE, h_p, round_p, id(v)>
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(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(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(tmproto.PrevoteType, nil, types.PartSetHeader{})
}
// Enter: any +2/3 prevotes at next round.
@@ -1365,7 +1417,6 @@ func (cs *State) enterPrevoteWait(height int64, round int32) {
// Enter: `timeoutPrecommit` after any +2/3 precommits.
// Enter: +2/3 precomits for block or nil.
// Lock & precommit the ProposalBlock if we have enough prevotes for it (a POL in this round)
// else, unlock an existing lock and precommit nil if +2/3 of prevotes were nil,
// else, precommit nil otherwise.
func (cs *State) enterPrecommit(height int64, round int32) {
logger := cs.Logger.With("height", height, "round", round)
@@ -1412,21 +1463,9 @@ func (cs *State) enterPrecommit(height int64, round int32) {
panic(fmt.Sprintf("this POLRound should be %v but got %v", round, polRound))
}
// +2/3 prevoted nil. Unlock and precommit nil.
if len(blockID.Hash) == 0 {
if cs.LockedBlock == nil {
logger.Debug("precommit step; +2/3 prevoted for nil")
} else {
logger.Debug("precommit step; +2/3 prevoted for nil; unlocking")
cs.LockedRound = -1
cs.LockedBlock = nil
cs.LockedBlockParts = nil
if err := cs.eventBus.PublishEventUnlock(cs.RoundStateEvent()); err != nil {
logger.Error("failed publishing event unlock", "err", err)
}
}
// +2/3 prevoted nil. Precommit nil.
if blockID.IsNil() {
logger.Debug("precommit step; +2/3 prevoted for nil")
cs.signAddVote(tmproto.PrecommitType, nil, types.PartSetHeader{})
return
}
@@ -1446,7 +1485,9 @@ func (cs *State) enterPrecommit(height int64, round int32) {
return
}
// If +2/3 prevoted for proposal block, stage and precommit it
// If greater than 2/3 of the voting power on the network prevoted for
// the proposed block, update our locked block to this block and issue a
// precommit vote for it.
if cs.ProposalBlock.HashesTo(blockID.Hash) {
logger.Debug("precommit step; +2/3 prevoted proposal block; locking", "hash", blockID.Hash)
@@ -1468,23 +1509,14 @@ func (cs *State) enterPrecommit(height int64, round int32) {
}
// There was a polka in this round for a block we don't have.
// Fetch that block, unlock, and precommit nil.
// The +2/3 prevotes for this round is the POL for our unlock.
// Fetch that block, and precommit nil.
logger.Debug("precommit step; +2/3 prevotes for a block we do not have; voting nil", "block_id", blockID)
cs.LockedRound = -1
cs.LockedBlock = nil
cs.LockedBlockParts = nil
if !cs.ProposalBlockParts.HasHeader(blockID.PartSetHeader) {
cs.ProposalBlock = nil
cs.ProposalBlockParts = types.NewPartSetFromHeader(blockID.PartSetHeader)
}
if err := cs.eventBus.PublishEventUnlock(cs.RoundStateEvent()); err != nil {
logger.Error("failed publishing event unlock", "err", err)
}
cs.signAddVote(tmproto.PrecommitType, nil, types.PartSetHeader{})
}
@@ -1592,7 +1624,7 @@ func (cs *State) tryFinalizeCommit(height int64) {
}
blockID, ok := cs.Votes.Precommits(cs.CommitRound).TwoThirdsMajority()
if !ok || len(blockID.Hash) == 0 {
if !ok || blockID.IsNil() {
logger.Error("failed attempt to finalize commit; there was no +2/3 majority or +2/3 was for nil")
return
}
@@ -1925,7 +1957,7 @@ func (cs *State) addProposalBlockPart(msg *BlockPartMessage, peerID types.NodeID
// 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",
@@ -2074,33 +2106,13 @@ func (cs *State) addVote(vote *types.Vote, peerID types.NodeID) (added bool, err
prevotes := cs.Votes.Prevotes(vote.Round)
cs.Logger.Debug("added vote to prevote", "vote", vote, "prevotes", prevotes.StringShort())
// If +2/3 prevotes for a block or nil for *any* round:
if blockID, ok := prevotes.TwoThirdsMajority(); ok {
// There was a polka!
// If we're locked but this is a recent polka, unlock.
// If it matches our ProposalBlock, update the ValidBlock
// Unlock if `cs.LockedRound < vote.Round <= cs.Round`
// NOTE: If vote.Round > cs.Round, we'll deal with it when we get to vote.Round
if (cs.LockedBlock != nil) &&
(cs.LockedRound < vote.Round) &&
(vote.Round <= cs.Round) &&
!cs.LockedBlock.HashesTo(blockID.Hash) {
cs.Logger.Debug("unlocking because of POL", "locked_round", cs.LockedRound, "pol_round", vote.Round)
cs.LockedRound = -1
cs.LockedBlock = nil
cs.LockedBlockParts = nil
if err := cs.eventBus.PublishEventUnlock(cs.RoundStateEvent()); err != nil {
return added, err
}
}
// Check to see if >2/3 of the voting power on the network voted for any non-nil block.
if blockID, ok := prevotes.TwoThirdsMajority(); ok && !blockID.IsNil() {
// Greater than 2/3 of the voting power on the network voted for some
// non-nil block
// Update Valid* if we can.
// NOTE: our proposal block may be nil or not what received a polka..
if len(blockID.Hash) != 0 && (cs.ValidRound < vote.Round) && (vote.Round == cs.Round) {
if cs.ValidRound < vote.Round && vote.Round == cs.Round {
if cs.ProposalBlock.HashesTo(blockID.Hash) {
cs.Logger.Debug("updating valid block because of POL", "valid_round", cs.ValidRound, "pol_round", vote.Round)
cs.ValidRound = vote.Round
@@ -2136,7 +2148,7 @@ func (cs *State) addVote(vote *types.Vote, peerID types.NodeID) (added bool, err
case cs.Round == vote.Round && cstypes.RoundStepPrevote <= cs.Step: // current round
blockID, ok := prevotes.TwoThirdsMajority()
if ok && (cs.isProposalComplete() || len(blockID.Hash) == 0) {
if ok && (cs.isProposalComplete() || blockID.IsNil()) {
cs.enterPrecommit(height, vote.Round)
} else if prevotes.HasTwoThirdsAny() {
cs.enterPrevoteWait(height, vote.Round)
@@ -2164,7 +2176,7 @@ func (cs *State) addVote(vote *types.Vote, peerID types.NodeID) (added bool, err
cs.enterNewRound(height, vote.Round)
cs.enterPrecommit(height, vote.Round)
if len(blockID.Hash) != 0 {
if !blockID.IsNil() {
cs.enterCommit(height, vote.Round)
if cs.config.SkipTimeoutCommit && precommits.HasAll() {
cs.enterNewRound(cs.Height, 0)
@@ -2404,3 +2416,39 @@ 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, h types.Header) time.Duration {
t := lt.Now()
if h.Time.After(t) {
return h.Time.Sub(t)
}
return 0
}
// proposalStepWaitingTime is used along with the `timeout-propose` configuration
// parameter to determines how long a validator will wait for a block to be sent from a proposer.
// proposalStepWaitingTime ensures that the validator waits long enough for the proposer to
// deliver a block with a monotically increasing timestamp.
//
// To ensure that the validator waits long enough, it must wait until the previous
// block's timestamp. It also must account for the difference between its own clock and
// the proposer's clock, i.e. the 'Precision', and the amount of time for the message to be transmitted,
// i.e. the MsgDelay.
//
// The result of proposalStepWaitingTime is compared with the configured `timeout-propose` duration,
// and the validator waits for whichever duration is larger before advancing to the next step
// and prevoting nil.
func proposalStepWaitingTime(lt tmtime.Source, h types.Header, tp types.TimestampParams) time.Duration {
t := lt.Now()
wt := h.Time.Add(tp.Precision).Add(tp.MsgDelay)
if t.After(wt) {
return 0
}
return wt.Sub(t)
}

File diff suppressed because it is too large Load Diff

View File

@@ -32,7 +32,7 @@ func TestPeerCatchupRounds(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
valSet, privVals := factory.RandValidatorSet(10, 1)
valSet, privVals := factory.ValidatorSet(10, 1)
hvs := NewHeightVoteSet(cfg.ChainID(), 1, valSet)

View File

@@ -202,10 +202,6 @@ func (b *EventBus) PublishEventPolka(data types.EventDataRoundState) error {
return b.Publish(types.EventPolkaValue, data)
}
func (b *EventBus) PublishEventUnlock(data types.EventDataRoundState) error {
return b.Publish(types.EventUnlockValue, data)
}
func (b *EventBus) PublishEventRelock(data types.EventDataRoundState) error {
return b.Publish(types.EventRelockValue, data)
}

View File

@@ -382,7 +382,6 @@ func TestEventBusPublish(t *testing.T) {
require.NoError(t, eventBus.PublishEventNewRound(types.EventDataNewRound{}))
require.NoError(t, eventBus.PublishEventCompleteProposal(types.EventDataCompleteProposal{}))
require.NoError(t, eventBus.PublishEventPolka(types.EventDataRoundState{}))
require.NoError(t, eventBus.PublishEventUnlock(types.EventDataRoundState{}))
require.NoError(t, eventBus.PublishEventRelock(types.EventDataRoundState{}))
require.NoError(t, eventBus.PublishEventLock(types.EventDataRoundState{}))
require.NoError(t, eventBus.PublishEventValidatorSetUpdates(types.EventDataValidatorSetUpdates{}))
@@ -488,7 +487,6 @@ var events = []string{
types.EventTimeoutProposeValue,
types.EventCompleteProposalValue,
types.EventPolkaValue,
types.EventUnlockValue,
types.EventLockValue,
types.EventRelockValue,
types.EventTimeoutWaitValue,
@@ -509,7 +507,6 @@ var queries = []tmpubsub.Query{
types.EventQueryTimeoutPropose,
types.EventQueryCompleteProposal,
types.EventQueryPolka,
types.EventQueryUnlock,
types.EventQueryLock,
types.EventQueryRelock,
types.EventQueryTimeoutWait,

View File

@@ -38,7 +38,7 @@ func TestEvidencePoolBasic(t *testing.T) {
blockStore = &mocks.BlockStore{}
)
valSet, privVals := factory.RandValidatorSet(1, 10)
valSet, privVals := factory.ValidatorSet(1, 10)
blockStore.On("LoadBlockMeta", mock.AnythingOfType("int64")).Return(
&types.BlockMeta{Header: types.Header{Time: defaultEvidenceTime}},

View File

@@ -190,7 +190,7 @@ func TestVerify_ForwardLunaticAttack(t *testing.T) {
}
func TestVerifyLightClientAttack_Equivocation(t *testing.T) {
conflictingVals, conflictingPrivVals := factory.RandValidatorSet(5, 10)
conflictingVals, conflictingPrivVals := factory.ValidatorSet(5, 10)
conflictingHeader, err := factory.MakeHeader(&types.Header{
ChainID: evidenceChainID,
@@ -285,7 +285,7 @@ func TestVerifyLightClientAttack_Equivocation(t *testing.T) {
func TestVerifyLightClientAttack_Amnesia(t *testing.T) {
var height int64 = 10
conflictingVals, conflictingPrivVals := factory.RandValidatorSet(5, 10)
conflictingVals, conflictingPrivVals := factory.ValidatorSet(5, 10)
conflictingHeader, err := factory.MakeHeader(&types.Header{
ChainID: evidenceChainID,
@@ -474,14 +474,14 @@ func makeLunaticEvidence(
totalVals, byzVals, phantomVals int,
commonTime, attackTime time.Time,
) (ev *types.LightClientAttackEvidence, trusted *types.LightBlock, common *types.LightBlock) {
commonValSet, commonPrivVals := factory.RandValidatorSet(totalVals, defaultVotingPower)
commonValSet, commonPrivVals := factory.ValidatorSet(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(phantomVals, defaultVotingPower)
phantomValSet, phantomPrivVals := factory.ValidatorSet(phantomVals, defaultVotingPower)
conflictingVals := phantomValSet.Copy()
require.NoError(t, conflictingVals.UpdateWithChangeSet(byzValSet))
@@ -537,7 +537,7 @@ func makeLunaticEvidence(
ValidatorSet: commonValSet,
}
trustedBlockID := factory.MakeBlockIDWithHash(trustedHeader.Hash())
trustedVals, privVals := factory.RandValidatorSet(totalVals, defaultVotingPower)
trustedVals, privVals := factory.ValidatorSet(totalVals, defaultVotingPower)
trustedVoteSet := types.NewVoteSet(evidenceChainID, height, 1, tmproto.SignedMsgType(2), trustedVals)
trustedCommit, err := factory.MakeCommit(trustedBlockID, height, 1, trustedVoteSet, privVals, defaultEvidenceTime)
require.NoError(t, err)

View File

@@ -3,6 +3,7 @@ package state_test
import (
"bytes"
"fmt"
"math/rand"
"time"
dbm "github.com/tendermint/tm-db"
@@ -246,7 +247,7 @@ func makeRandomStateFromValidatorSet(
func makeRandomStateFromConsensusParams(consensusParams *types.ConsensusParams,
height, lastHeightConsensusParamsChanged int64) sm.State {
val, _ := factory.RandValidator(true, 10)
val, _ := factory.Validator(10 + int64(rand.Uint32()))
valSet := types.NewValidatorSet([]*types.Validator{val})
return sm.State{
LastBlockHeight: height - 1,

View File

@@ -102,7 +102,7 @@ func TestRollbackDifferentStateHeight(t *testing.T) {
func setupStateStore(t *testing.T, height int64) state.Store {
stateStore := state.NewStore(dbm.NewMemDB())
valSet, _ := factory.RandValidatorSet(5, 10)
valSet, _ := factory.ValidatorSet(5, 10)
params := types.DefaultConsensusParams()
params.Version.AppVersion = 10

View File

@@ -2,6 +2,7 @@ package state_test
import (
"fmt"
"math/rand"
"os"
"testing"
@@ -28,9 +29,9 @@ const (
func TestStoreBootstrap(t *testing.T) {
stateDB := dbm.NewMemDB()
stateStore := sm.NewStore(stateDB)
val, _ := factory.RandValidator(true, 10)
val2, _ := factory.RandValidator(true, 10)
val3, _ := factory.RandValidator(true, 10)
val, _ := factory.Validator(10 + int64(rand.Uint32()))
val2, _ := factory.Validator(10 + int64(rand.Uint32()))
val3, _ := factory.Validator(10 + int64(rand.Uint32()))
vals := types.NewValidatorSet([]*types.Validator{val, val2, val3})
bootstrapState := makeRandomStateFromValidatorSet(vals, 100, 100)
err := stateStore.Bootstrap(bootstrapState)
@@ -54,9 +55,9 @@ func TestStoreBootstrap(t *testing.T) {
func TestStoreLoadValidators(t *testing.T) {
stateDB := dbm.NewMemDB()
stateStore := sm.NewStore(stateDB)
val, _ := factory.RandValidator(true, 10)
val2, _ := factory.RandValidator(true, 10)
val3, _ := factory.RandValidator(true, 10)
val, _ := factory.Validator(10 + int64(rand.Uint32()))
val2, _ := factory.Validator(10 + int64(rand.Uint32()))
val3, _ := factory.Validator(10 + int64(rand.Uint32()))
vals := types.NewValidatorSet([]*types.Validator{val, val2, val3})
// 1) LoadValidators loads validators using a height where they were last changed

View File

@@ -274,7 +274,7 @@ loop:
}
func mockLBResp(t *testing.T, peer types.NodeID, height int64, time time.Time) lightBlockResponse {
vals, pv := factory.RandValidatorSet(3, 10)
vals, pv := factory.ValidatorSet(3, 10)
_, _, lb := mockLB(t, height, time, factory.MakeBlockID(), vals, pv)
return lightBlockResponse{
block: lb,

View File

@@ -423,7 +423,7 @@ func TestReactor_LightBlockResponse(t *testing.T) {
h := factory.MakeRandomHeader()
h.Height = height
blockID := factory.MakeBlockIDWithHash(h.Hash())
vals, pv := factory.RandValidatorSet(1, 10)
vals, pv := factory.ValidatorSet(1, 10)
vote, err := factory.MakeVote(pv[0], h.ChainID, 0, h.Height, 0, 2,
blockID, factory.DefaultTestTime)
require.NoError(t, err)
@@ -714,7 +714,7 @@ func handleLightBlockRequests(
} else {
switch errorCount % 3 {
case 0: // send a different block
vals, pv := factory.RandValidatorSet(3, 10)
vals, pv := factory.ValidatorSet(3, 10)
_, _, lb := mockLB(t, int64(msg.Height), factory.DefaultTestTime, factory.MakeBlockID(), vals, pv)
differntLB, err := lb.ToProto()
require.NoError(t, err)
@@ -782,7 +782,7 @@ func buildLightBlockChain(t *testing.T, fromHeight, toHeight int64, startTime ti
chain := make(map[int64]*types.LightBlock, toHeight-fromHeight)
lastBlockID := factory.MakeBlockID()
blockTime := startTime.Add(time.Duration(fromHeight-toHeight) * time.Minute)
vals, pv := factory.RandValidatorSet(3, 10)
vals, pv := factory.ValidatorSet(3, 10)
for height := fromHeight; height < toHeight; height++ {
vals, pv, chain[height] = mockLB(t, height, blockTime, lastBlockID, vals, pv)
lastBlockID = factory.MakeBlockIDWithHash(chain[height].Header.Hash())
@@ -800,7 +800,7 @@ func mockLB(t *testing.T, height int64, time time.Time, lastBlockID types.BlockI
Time: time,
})
require.NoError(t, err)
nextVals, nextPrivVals := factory.RandValidatorSet(3, 10)
nextVals, nextPrivVals := factory.ValidatorSet(3, 10)
header.ValidatorsHash = currentVals.Hash()
header.NextValidatorsHash = nextVals.Hash()
header.ConsensusHash = types.DefaultConsensusParams().HashConsensusParams()

View File

@@ -55,7 +55,7 @@ func MakeHeader(h *types.Header) (*types.Header, error) {
if h.Height == 0 {
h.Height = 1
}
if h.LastBlockID.IsZero() {
if h.LastBlockID.IsNil() {
h.LastBlockID = MakeBlockID()
}
if h.ChainID == "" {

View File

@@ -1,35 +1,33 @@
package factory
import (
"sort"
"time"
"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(
cfg *config.Config,
numValidators int,
randPower bool,
minPower int64) (*types.GenesisDoc, []types.PrivValidator) {
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 := RandValidator(randPower, minPower)
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,
}
}

View File

@@ -3,35 +3,29 @@ package factory
import (
"context"
"fmt"
"math/rand"
"sort"
"github.com/tendermint/tendermint/types"
)
func RandValidator(randPower bool, minPower int64) (*types.Validator, types.PrivValidator) {
func Validator(votingPower int64) (*types.Validator, types.PrivValidator) {
privVal := types.NewMockPV()
votePower := minPower
if randPower {
// nolint:gosec // G404: Use of weak random number generator
votePower += int64(rand.Uint32())
}
pubKey, err := privVal.GetPubKey(context.Background())
if err != nil {
panic(fmt.Errorf("could not retrieve pubkey %w", err))
}
val := types.NewValidator(pubKey, votePower)
val := types.NewValidator(pubKey, votingPower)
return val, privVal
}
func RandValidatorSet(numValidators int, votingPower int64) (*types.ValidatorSet, []types.PrivValidator) {
func ValidatorSet(numValidators int, votingPower int64) (*types.ValidatorSet, []types.PrivValidator) {
var (
valz = make([]*types.Validator, numValidators)
privValidators = make([]types.PrivValidator, numValidators)
)
for i := 0; i < numValidators; i++ {
val, privValidator := RandValidator(false, votingPower)
val, privValidator := Validator(votingPower)
valz[i] = val
privValidators[i] = privValidator
}

28
libs/time/mocks/source.go Normal file
View File

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

View File

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

View File

@@ -112,7 +112,7 @@ func TestValidateTrustOptions(t *testing.T) {
func TestClient_SequentialVerification(t *testing.T) {
newKeys := genPrivKeys(4)
newVals := newKeys.ToValidators(10, 1)
differentVals, _ := factory.RandValidatorSet(10, 100)
differentVals, _ := factory.ValidatorSet(10, 100)
testCases := []struct {
name string
@@ -865,7 +865,7 @@ func TestClientRemovesWitnessIfItSendsUsIncorrectHeader(t *testing.T) {
}
func TestClient_TrustedValidatorSet(t *testing.T) {
differentVals, _ := factory.RandValidatorSet(10, 100)
differentVals, _ := factory.ValidatorSet(10, 100)
mockBadValSetNode := mockNodeFromHeadersAndVals(
map[int64]*types.SignedHeader{
1: h1,

View File

@@ -183,7 +183,7 @@ func Test_Concurrency(t *testing.T) {
}
func randLightBlock(height int64) *types.LightBlock {
vals, _ := factory.RandValidatorSet(2, 1)
vals, _ := factory.ValidatorSet(2, 1)
return &types.LightBlock{
SignedHeader: &types.SignedHeader{
Header: &types.Header{

View File

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

View File

@@ -286,7 +286,7 @@ func makeBlockID(hash []byte, partSetSize uint32, partSetHash []byte) types.Bloc
func mutateValidatorSet(privVals []types.MockPV, vals *types.ValidatorSet,
) ([]types.PrivValidator, *types.ValidatorSet, error) {
newVal, newPrivVal := factory.RandValidator(false, 10)
newVal, newPrivVal := factory.Validator(10)
var newVals *types.ValidatorSet
if vals.Size() > 2 {

View File

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

View File

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

View File

@@ -38,7 +38,6 @@ const (
EventStateSyncStatusValue = "StateSyncStatus"
EventTimeoutProposeValue = "TimeoutPropose"
EventTimeoutWaitValue = "TimeoutWait"
EventUnlockValue = "Unlock"
EventValidBlockValue = "ValidBlock"
EventVoteValue = "Vote"
)
@@ -223,7 +222,6 @@ var (
EventQueryTimeoutPropose = QueryForEvent(EventTimeoutProposeValue)
EventQueryTimeoutWait = QueryForEvent(EventTimeoutWaitValue)
EventQueryTx = QueryForEvent(EventTxValue)
EventQueryUnlock = QueryForEvent(EventUnlockValue)
EventQueryValidatorSetUpdates = QueryForEvent(EventValidatorSetUpdatesValue)
EventQueryValidBlock = QueryForEvent(EventValidBlockValue)
EventQueryVote = QueryForEvent(EventVoteValue)

View File

@@ -41,6 +41,7 @@ type ConsensusParams struct {
Evidence EvidenceParams `json:"evidence"`
Validator ValidatorParams `json:"validator"`
Version VersionParams `json:"version"`
Timestamp TimestampParams `json:"timestamp"`
}
// HashedParams is a subset of ConsensusParams.
@@ -75,6 +76,14 @@ type VersionParams struct {
AppVersion uint64 `json:"app_version"`
}
// TimestampParams influence the validity of block timestamps.
// TODO (@wbanfield): add link to proposer-based timestamp spec when completed.
type TimestampParams struct {
Precision time.Duration `json:"precision"`
Accuracy time.Duration `json:"accuracy"`
MsgDelay time.Duration `json:"msg_delay"`
}
// DefaultConsensusParams returns a default ConsensusParams.
func DefaultConsensusParams() *ConsensusParams {
return &ConsensusParams{
@@ -116,6 +125,16 @@ func DefaultVersionParams() VersionParams {
}
}
func DefaultTimestampParams() TimestampParams {
// TODO(@wbanfield): Determine experimental values for these defaults
// https://github.com/tendermint/tendermint/issues/7202
return TimestampParams{
Precision: 2 * time.Second,
Accuracy: 500 * time.Millisecond,
MsgDelay: 3 * time.Second,
}
}
func (val *ValidatorParams) IsValidPubkeyType(pubkeyType string) bool {
for i := 0; i < len(val.PubKeyTypes); i++ {
if val.PubKeyTypes[i] == pubkeyType {

View File

@@ -79,6 +79,25 @@ 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:
//
// proposedBlockTime > validatorLocaltime - Precision && proposedBlockTime < validatorLocalTime + Precision + MsgDelay.
//
// 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(clock tmtime.Source, tp TimestampParams) bool {
lt := clock.Now()
lhs := lt.Add(-tp.Precision)
rhs := lt.Add(tp.Precision).Add(tp.MsgDelay)
if lhs.Before(p.Timestamp) && rhs.After(p.Timestamp) {
return true
}
return false
}
// String returns a string representation of the Proposal.
//
// 1. height

View File

@@ -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"
tmtimemocks "github.com/tendermint/tendermint/libs/time/mocks"
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
)
@@ -191,3 +192,65 @@ 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
proposalTime time.Time
localTime time.Time
precision time.Duration
msgDelay time.Duration
expectTimely bool
}{
{
// Checking that the following inequality evaluates to true:
// 1 - 2 < 0 < 1 + 2 + 1
name: "basic timely",
proposalTime: genesisTime,
localTime: genesisTime.Add(1 * time.Nanosecond),
precision: time.Nanosecond * 2,
msgDelay: time.Nanosecond,
expectTimely: true,
},
{
// Checking that the following inequality evaluates to false:
// 3 - 2 < 0 < 3 + 2 + 1
name: "local time too large",
proposalTime: genesisTime,
localTime: genesisTime.Add(3 * time.Nanosecond),
precision: time.Nanosecond * 2,
msgDelay: time.Nanosecond,
expectTimely: false,
},
{
// Checking that the following inequality evaluates to false:
// 0 - 2 < 2 < 2 + 1
name: "proposal time too large",
proposalTime: genesisTime.Add(4 * time.Nanosecond),
localTime: genesisTime,
precision: time.Nanosecond * 2,
msgDelay: time.Nanosecond,
expectTimely: false,
},
}
for _, testCase := range testCases {
t.Run(testCase.name, func(t *testing.T) {
p := Proposal{
Timestamp: testCase.proposalTime,
}
tp := TimestampParams{
Precision: testCase.precision,
MsgDelay: testCase.msgDelay,
}
mockSource := new(tmtimemocks.Source)
mockSource.On("Now").Return(testCase.localTime)
ti := p.IsTimely(mockSource, tp)
assert.Equal(t, testCase.expectTimely, ti)
})
}
}

View File

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

View File

@@ -27,7 +27,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,
@@ -44,7 +44,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) {
@@ -145,7 +145,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
{
@@ -156,7 +156,7 @@ func TestVoteSet_2_3Majority(t *testing.T) {
_, err = signAddVote(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.
@@ -168,7 +168,7 @@ func TestVoteSet_2_3Majority(t *testing.T) {
_, err = signAddVote(privValidators[7], vote, voteSet)
require.NoError(t, err)
blockID, ok = voteSet.TwoThirdsMajority()
assert.True(t, ok || blockID.IsZero(), "there should be 2/3 majority for nil")
assert.True(t, ok || blockID.IsNil(), "there should be 2/3 majority for nil")
}
}
@@ -200,7 +200,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
@@ -212,7 +212,7 @@ func TestVoteSet_2_3MajorityRedux(t *testing.T) {
_, err = signAddVote(privValidators[66], withBlockHash(vote, nil), voteSet)
require.NoError(t, err)
blockID, ok = voteSet.TwoThirdsMajority()
assert.False(t, ok || !blockID.IsZero(),
assert.False(t, ok || !blockID.IsNil(),
"there should be no 2/3 majority: last vote added was nil")
}
@@ -226,7 +226,7 @@ func TestVoteSet_2_3MajorityRedux(t *testing.T) {
_, err = signAddVote(privValidators[67], withBlockPartSetHeader(vote, blockPartsHeader), voteSet)
require.NoError(t, err)
blockID, ok = voteSet.TwoThirdsMajority()
assert.False(t, ok || !blockID.IsZero(),
assert.False(t, ok || !blockID.IsNil(),
"there should be no 2/3 majority: last vote added had different PartSetHeader Hash")
}
@@ -240,7 +240,7 @@ func TestVoteSet_2_3MajorityRedux(t *testing.T) {
_, err = signAddVote(privValidators[68], withBlockPartSetHeader(vote, blockPartsHeader), voteSet)
require.NoError(t, err)
blockID, ok = voteSet.TwoThirdsMajority()
assert.False(t, ok || !blockID.IsZero(),
assert.False(t, ok || !blockID.IsNil(),
"there should be no 2/3 majority: last vote added had different PartSetHeader Total")
}
@@ -253,7 +253,7 @@ func TestVoteSet_2_3MajorityRedux(t *testing.T) {
_, err = signAddVote(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")
}