Browse Source

Proposer-Based Timestamps Merge (#7605)

This pull request merges in the changes for implementing Proposer-based timestamps into `master`. The power was primarily being done in the `wb/proposer-based-timestamps` branch, with changes being merged into that branch during development. This pull request represents an amalgamation of the changes made into that development branch. All of the changes that were placed into that branch have been cleanly rebased on top of the latest `master`. The changes compile and the tests pass insofar as our tests in general pass.

### Note To Reviewers
 These changes have been extensively reviewed during development. There is not much new here. In the interest of making effective use of time, I would recommend against trying to perform a complete audit of the changes presented and instead examine for mistakes that may have occurred during the process of rebasing the changes. I gave the complete change set a first pass for any issues, but additional eyes would be very appreciated. 

In sum, this change set does the following:
closes #6942 
merges in #6849
pull/7699/head
William Banfield 2 years ago
committed by GitHub
parent
commit
0aa3b0b6fc
No known key found for this signature in database GPG Key ID: 4AEE18F83AFDEB23
49 changed files with 2925 additions and 1149 deletions
  1. +6
    -1
      CHANGELOG_PENDING.md
  2. +4
    -0
      config/toml.go
  3. +10
    -6
      internal/blocksync/reactor_test.go
  4. +6
    -4
      internal/consensus/byzantine_test.go
  5. +157
    -148
      internal/consensus/common_test.go
  6. +3
    -2
      internal/consensus/invalid_test.go
  7. +17
    -7
      internal/consensus/mempool_test.go
  8. +489
    -0
      internal/consensus/pbts_test.go
  9. +4
    -3
      internal/consensus/reactor.go
  10. +6
    -7
      internal/consensus/reactor_test.go
  11. +29
    -24
      internal/consensus/replay_test.go
  12. +197
    -131
      internal/consensus/state.go
  13. +1089
    -546
      internal/consensus/state_test.go
  14. +1
    -1
      internal/consensus/types/height_vote_set_test.go
  15. +9
    -8
      internal/consensus/types/round_state.go
  16. +0
    -4
      internal/eventbus/event_bus.go
  17. +0
    -3
      internal/eventbus/event_bus_test.go
  18. +1
    -2
      internal/evidence/pool_test.go
  19. +6
    -8
      internal/evidence/verify_test.go
  20. +2
    -2
      internal/state/helpers_test.go
  21. +7
    -14
      internal/state/rollback_test.go
  22. +1
    -24
      internal/state/state.go
  23. +11
    -12
      internal/state/store_test.go
  24. +0
    -7
      internal/state/validation.go
  25. +0
    -1
      internal/state/validation_test.go
  26. +1
    -1
      internal/statesync/block_queue_test.go
  27. +4
    -4
      internal/statesync/reactor_test.go
  28. +1
    -1
      internal/test/factory/block.go
  29. +20
    -23
      internal/test/factory/genesis.go
  30. +6
    -13
      internal/test/factory/validator.go
  31. +28
    -0
      libs/time/mocks/source.go
  32. +14
    -0
      libs/time/time.go
  33. +4
    -5
      light/client_test.go
  34. +4
    -8
      light/store/db/db_test.go
  35. +2
    -1
      node/node_test.go
  36. +24
    -4
      proto/tendermint/statesync/message_test.go
  37. +353
    -38
      proto/tendermint/types/params.pb.go
  38. +3
    -2
      test/e2e/runner/evidence.go
  39. +1
    -1
      third_party/proto/gogoproto/gogo.proto
  40. +3
    -3
      types/block.go
  41. +1
    -1
      types/canonical.go
  42. +0
    -2
      types/events.go
  43. +9
    -4
      types/genesis_test.go
  44. +45
    -1
      types/params.go
  45. +207
    -54
      types/params_test.go
  46. +27
    -2
      types/proposal.go
  47. +101
    -4
      types/proposal_test.go
  48. +2
    -2
      types/vote.go
  49. +10
    -10
      types/vote_set_test.go

+ 6
- 1
CHANGELOG_PENDING.md View File

@ -49,10 +49,15 @@ Special thanks to external contributors on this release:
- [rpc] [\#7270](https://github.com/tendermint/tendermint/pull/7270) Add `header` and `header_by_hash` RPC Client queries. (@fedekunze)
- [cli] [#7033](https://github.com/tendermint/tendermint/pull/7033) Add a `rollback` command to rollback to the previous tendermint state in the event of non-determinstic app hash or reverting an upgrade.
- [mempool, rpc] \#7041 Add removeTx operation to the RPC layer. (@tychoish)
- [consensus] \#7354 add a new `synchrony` field to the `ConsensusParameter` struct for controlling the parameters of the proposer-based timestamp algorithm. (@williambanfield)
- [consensus] \#7376 Update the proposal logic per the Propose-based timestamps specification so that the proposer will wait for the previous block time to occur before proposing the next block. (@williambanfield)
- [consensus] \#7391 Use the proposed block timestamp as the proposal timestamp. Update the block validation logic to ensure that the proposed block's timestamp matches the timestamp in the proposal message. (@williambanfield)
- [consensus] \#7415 Update proposal validation logic to Prevote nil if a proposal does not meet the conditions for Timelyness per the proposer-based timestamp specification. (@anca)
- [consensus] \#7382 Update block validation to no longer require the block timestamp to be the median of the timestamps of the previous commit. (@anca)
### IMPROVEMENTS
- [internal/protoio] \#7325 Optimized `MarshalDelimited` by inlining the common case and using a `sync.Pool` in the worst case. (@odeke-em)
- [consensus] \#6969 remove logic to 'unlock' a locked block.
- [pubsub] \#7319 Performance improvements for the event query API (@creachadair)
- [node] \#7521 Define concrete type for seed node implementation (@spacech1mp)
- [rpc] \#7612 paginate mempool /unconfirmed_txs rpc endpoint (@spacech1mp)


+ 4
- 0
config/toml.go View File

@ -571,6 +571,10 @@ var testGenesisFmt = `{
"max_gas": "-1",
"time_iota_ms": "10"
},
"synchrony": {
"message_delay": "500000000",
"precision": "10000000"
},
"evidence": {
"max_age_num_blocks": "100000",
"max_age_duration": "172800000000000",


+ 10
- 6
internal/blocksync/reactor_test.go View File

@ -207,7 +207,8 @@ func TestReactor_AbruptDisconnect(t *testing.T) {
require.NoError(t, err)
defer os.RemoveAll(cfg.RootDir)
genDoc, privVals := factory.RandGenesisDoc(ctx, t, cfg, 1, false, 30)
valSet, privVals := factory.ValidatorSet(ctx, t, 1, 30)
genDoc := factory.GenesisDoc(cfg, time.Now(), valSet.Validators, nil)
maxBlockHeight := int64(64)
rts := setup(ctx, t, genDoc, privVals[0], []int64{maxBlockHeight, 0}, 0)
@ -246,7 +247,8 @@ func TestReactor_SyncTime(t *testing.T) {
require.NoError(t, err)
defer os.RemoveAll(cfg.RootDir)
genDoc, privVals := factory.RandGenesisDoc(ctx, t, cfg, 1, false, 30)
valSet, privVals := factory.ValidatorSet(ctx, t, 1, 30)
genDoc := factory.GenesisDoc(cfg, time.Now(), valSet.Validators, nil)
maxBlockHeight := int64(101)
rts := setup(ctx, t, genDoc, privVals[0], []int64{maxBlockHeight, 0}, 0)
@ -271,10 +273,10 @@ func TestReactor_NoBlockResponse(t *testing.T) {
cfg, err := config.ResetTestRoot("block_sync_reactor_test")
require.NoError(t, err)
defer os.RemoveAll(cfg.RootDir)
genDoc, privVals := factory.RandGenesisDoc(ctx, t, cfg, 1, false, 30)
valSet, privVals := factory.ValidatorSet(ctx, t, 1, 30)
genDoc := factory.GenesisDoc(cfg, time.Now(), valSet.Validators, nil)
maxBlockHeight := int64(65)
rts := setup(ctx, t, genDoc, privVals[0], []int64{maxBlockHeight, 0}, 0)
@ -326,7 +328,8 @@ func TestReactor_BadBlockStopsPeer(t *testing.T) {
defer os.RemoveAll(cfg.RootDir)
maxBlockHeight := int64(48)
genDoc, privVals := factory.RandGenesisDoc(ctx, t, cfg, 1, false, 30)
valSet, privVals := factory.ValidatorSet(ctx, t, 1, 30)
genDoc := factory.GenesisDoc(cfg, time.Now(), valSet.Validators, nil)
rts := setup(ctx, t, genDoc, privVals[0], []int64{maxBlockHeight, 0, 0, 0, 0}, 1000)
@ -360,7 +363,8 @@ func TestReactor_BadBlockStopsPeer(t *testing.T) {
//
// XXX: This causes a potential race condition.
// See: https://github.com/tendermint/tendermint/issues/6005
otherGenDoc, otherPrivVals := factory.RandGenesisDoc(ctx, t, cfg, 1, false, 30)
valSet, otherPrivVals := factory.ValidatorSet(ctx, t, 1, 30)
otherGenDoc := factory.GenesisDoc(cfg, time.Now(), valSet.Validators, nil)
newNode := rts.network.MakeNode(ctx, t, p2ptest.NodeOptions{
MaxPeers: uint16(len(rts.nodes) + 1),
MaxConnected: uint16(len(rts.nodes) + 1),


+ 6
- 4
internal/consensus/byzantine_test.go View File

@ -23,6 +23,7 @@ import (
"github.com/tendermint/tendermint/internal/store"
"github.com/tendermint/tendermint/internal/test/factory"
"github.com/tendermint/tendermint/libs/log"
tmtime "github.com/tendermint/tendermint/libs/time"
tmcons "github.com/tendermint/tendermint/proto/tendermint/consensus"
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
"github.com/tendermint/tendermint/types"
@ -46,7 +47,8 @@ func TestByzantinePrevoteEquivocation(t *testing.T) {
tickerFunc := newMockTickerFunc(true)
appFunc := newKVStore
genDoc, privVals := factory.RandGenesisDoc(ctx, t, config, nValidators, false, 30)
valSet, privVals := factory.ValidatorSet(ctx, t, nValidators, 30)
genDoc := factory.GenesisDoc(config, time.Now(), valSet.Validators, nil)
states := make([]*State, nValidators)
for i := 0; i < nValidators; i++ {
@ -215,18 +217,18 @@ func TestByzantinePrevoteEquivocation(t *testing.T) {
// Make proposal
propBlockID := types.BlockID{Hash: block.Hash(), PartSetHeader: blockParts.Header()}
proposal := types.NewProposal(height, round, lazyNodeState.ValidRound, propBlockID)
proposal := types.NewProposal(height, round, lazyNodeState.ValidRound, propBlockID, block.Header.Time)
p := proposal.ToProto()
if err := lazyNodeState.privValidator.SignProposal(ctx, lazyNodeState.state.ChainID, p); err == nil {
proposal.Signature = p.Signature
// send proposal and block parts on internal msg queue
lazyNodeState.sendInternalMessage(ctx, msgInfo{&ProposalMessage{proposal}, ""})
lazyNodeState.sendInternalMessage(ctx, msgInfo{&ProposalMessage{proposal}, "", tmtime.Now()})
for i := 0; i < int(blockParts.Total()); i++ {
part := blockParts.GetPart(i)
lazyNodeState.sendInternalMessage(ctx, msgInfo{&BlockPartMessage{
lazyNodeState.Height, lazyNodeState.Round, part,
}, ""})
}, "", tmtime.Now()})
}
} else if !lazyNodeState.replayMode {
lazyNodeState.logger.Error("enterPropose: Error signing proposal", "height", height, "round", round, "err", err)


+ 157
- 148
internal/consensus/common_test.go View File

@ -3,6 +3,7 @@ package consensus
import (
"bytes"
"context"
"errors"
"fmt"
"io"
"os"
@ -88,6 +89,7 @@ type validatorStub struct {
Index int32 // Validator index. NOTE: we don't assume validator set changes.
Height int64
Round int32
clock tmtime.Source
types.PrivValidator
VotingPower int64
lastVote *types.Vote
@ -100,16 +102,15 @@ func newValidatorStub(privValidator types.PrivValidator, valIndex int32) *valida
Index: valIndex,
PrivValidator: privValidator,
VotingPower: testMinPower,
clock: tmtime.DefaultSource{},
}
}
func (vs *validatorStub) signVote(
ctx context.Context,
cfg *config.Config,
voteType tmproto.SignedMsgType,
hash []byte,
header types.PartSetHeader,
) (*types.Vote, error) {
chainID string,
blockID types.BlockID) (*types.Vote, error) {
pubKey, err := vs.PrivValidator.GetPubKey(ctx)
if err != nil {
@ -121,12 +122,12 @@ func (vs *validatorStub) signVote(
ValidatorAddress: pubKey.Address(),
Height: vs.Height,
Round: vs.Round,
Timestamp: tmtime.Now(),
Timestamp: vs.clock.Now(),
Type: voteType,
BlockID: types.BlockID{Hash: hash, PartSetHeader: header},
BlockID: blockID,
}
v := vote.ToProto()
if err := vs.PrivValidator.SignVote(ctx, cfg.ChainID(), v); err != nil {
if err := vs.PrivValidator.SignVote(ctx, chainID, v); err != nil {
return nil, fmt.Errorf("sign vote failed: %w", err)
}
@ -147,13 +148,11 @@ func signVote(
ctx context.Context,
t *testing.T,
vs *validatorStub,
cfg *config.Config,
voteType tmproto.SignedMsgType,
hash []byte,
header types.PartSetHeader,
) *types.Vote {
chainID string,
blockID types.BlockID) *types.Vote {
v, err := vs.signVote(ctx, cfg, voteType, hash, header)
v, err := vs.signVote(ctx, voteType, chainID, blockID)
require.NoError(t, err, "failed to sign vote")
vs.lastVote = v
@ -164,15 +163,14 @@ func signVote(
func signVotes(
ctx context.Context,
t *testing.T,
cfg *config.Config,
voteType tmproto.SignedMsgType,
hash []byte,
header types.PartSetHeader,
chainID string,
blockID types.BlockID,
vss ...*validatorStub,
) []*types.Vote {
votes := make([]*types.Vote, len(vss))
for i, vs := range vss {
votes[i] = signVote(ctx, t, vs, cfg, voteType, hash, header)
votes[i] = signVote(ctx, t, vs, voteType, chainID, blockID)
}
return votes
}
@ -247,7 +245,7 @@ func decideProposal(
// Make proposal
polRound, propBlockID := validRound, types.BlockID{Hash: block.Hash(), PartSetHeader: blockParts.Header()}
proposal = types.NewProposal(height, round, polRound, propBlockID)
proposal = types.NewProposal(height, round, polRound, propBlockID, block.Header.Time)
p := proposal.ToProto()
require.NoError(t, vs.SignProposal(ctx, chainID, p))
@ -265,14 +263,13 @@ func addVotes(to *State, votes ...*types.Vote) {
func signAddVotes(
ctx context.Context,
t *testing.T,
cfg *config.Config,
to *State,
voteType tmproto.SignedMsgType,
hash []byte,
header types.PartSetHeader,
chainID string,
blockID types.BlockID,
vss ...*validatorStub,
) {
addVotes(to, signVotes(ctx, t, cfg, voteType, hash, header, vss...)...)
addVotes(to, signVotes(ctx, t, voteType, chainID, blockID, vss...)...)
}
func validatePrevote(
@ -378,6 +375,35 @@ func subscribeToVoter(ctx context.Context, t *testing.T, cs *State, addr []byte)
return ch
}
func subscribeToVoterBuffered(ctx context.Context, t *testing.T, cs *State, addr []byte) <-chan tmpubsub.Message {
t.Helper()
votesSub, err := cs.eventBus.SubscribeWithArgs(ctx, tmpubsub.SubscribeArgs{
ClientID: testSubscriber,
Query: types.EventQueryVote,
Limit: 10})
if err != nil {
t.Fatalf("failed to subscribe %s to %v", testSubscriber, types.EventQueryVote)
}
ch := make(chan tmpubsub.Message, 10)
go func() {
for {
msg, err := votesSub.Next(ctx)
if err != nil {
if !errors.Is(err, tmpubsub.ErrTerminated) && !errors.Is(err, context.Canceled) {
t.Errorf("error terminating pubsub %s", err)
}
return
}
vote := msg.Data().(types.EventDataVote)
// we only fire for our own votes
if bytes.Equal(addr, vote.Vote.ValidatorAddress) {
ch <- msg
}
}
}()
return ch
}
//-------------------------------------------------------------------------------
// consensus states
@ -477,17 +503,12 @@ func loadPrivValidator(t *testing.T, cfg *config.Config) *privval.FilePV {
return privValidator
}
func randState(
ctx context.Context,
t *testing.T,
cfg *config.Config,
logger log.Logger,
nValidators int,
) (*State, []*validatorStub) {
func makeState(ctx context.Context, t *testing.T, cfg *config.Config, logger log.Logger, nValidators int) (*State, []*validatorStub) {
t.Helper()
// Get State
state, privVals := randGenesisState(ctx, t, cfg, nValidators, false, 10)
state, privVals := makeGenesisState(ctx, t, cfg, genesisStateArgs{
Validators: nValidators,
})
vss := make([]*validatorStub, nValidators)
@ -504,7 +525,8 @@ func randState(
//-------------------------------------------------------------------------------
func ensureNoNewEvent(t *testing.T, ch <-chan tmpubsub.Message, timeout time.Duration, errorMessage string) {
func ensureNoMessageBeforeTimeout(t *testing.T, ch <-chan tmpubsub.Message, timeout time.Duration,
errorMessage string) {
t.Helper()
select {
case <-time.After(timeout):
@ -516,7 +538,8 @@ func ensureNoNewEvent(t *testing.T, ch <-chan tmpubsub.Message, timeout time.Dur
func ensureNoNewEventOnChannel(t *testing.T, ch <-chan tmpubsub.Message) {
t.Helper()
ensureNoNewEvent(t,
ensureNoMessageBeforeTimeout(
t,
ch,
ensureTimeout,
"We should be stuck waiting, not receiving new event on the channel")
@ -524,137 +547,112 @@ func ensureNoNewEventOnChannel(t *testing.T, ch <-chan tmpubsub.Message) {
func ensureNoNewRoundStep(t *testing.T, stepCh <-chan tmpubsub.Message) {
t.Helper()
ensureNoNewEvent(
ensureNoMessageBeforeTimeout(
t,
stepCh,
ensureTimeout,
"We should be stuck waiting, not receiving NewRoundStep event")
}
func ensureNoNewUnlock(t *testing.T, unlockCh <-chan tmpubsub.Message) {
t.Helper()
ensureNoNewEvent(t,
unlockCh,
ensureTimeout,
"We should be stuck waiting, not receiving Unlock event")
}
func ensureNoNewTimeout(t *testing.T, stepCh <-chan tmpubsub.Message, timeout int64) {
t.Helper()
timeoutDuration := time.Duration(timeout*10) * time.Nanosecond
ensureNoNewEvent(t,
ensureNoMessageBeforeTimeout(
t,
stepCh,
timeoutDuration,
"We should be stuck waiting, not receiving NewTimeout event")
}
func ensureNewEvent(t *testing.T, ch <-chan tmpubsub.Message, height int64, round int32, timeout time.Duration, errorMessage string) {
func ensureNewEvent(t *testing.T, ch <-chan tmpubsub.Message, height int64, round int32, timeout time.Duration) {
t.Helper()
select {
case <-time.After(timeout):
t.Fatal(errorMessage)
case msg := <-ch:
roundStateEvent, ok := msg.Data().(types.EventDataRoundState)
require.True(t, ok,
"expected a EventDataRoundState, got %T. Wrong subscription channel?",
msg.Data())
msg := ensureMessageBeforeTimeout(t, ch, ensureTimeout)
roundStateEvent, ok := msg.Data().(types.EventDataRoundState)
require.True(t, ok,
"expected a EventDataRoundState, got %T. Wrong subscription channel?",
msg.Data())
require.Equal(t, height, roundStateEvent.Height)
require.Equal(t, round, roundStateEvent.Round)
// TODO: We could check also for a step at this point!
}
require.Equal(t, height, roundStateEvent.Height)
require.Equal(t, round, roundStateEvent.Round)
// TODO: We could check also for a step at this point!
}
func ensureNewRound(t *testing.T, roundCh <-chan tmpubsub.Message, height int64, round int32) {
t.Helper()
select {
case <-time.After(ensureTimeout):
t.Fatal("Timeout expired while waiting for NewRound event")
case msg := <-roundCh:
newRoundEvent, ok := msg.Data().(types.EventDataNewRound)
require.True(t, ok, "expected a EventDataNewRound, got %T. Wrong subscription channel?",
msg.Data())
msg := ensureMessageBeforeTimeout(t, roundCh, ensureTimeout)
newRoundEvent, ok := msg.Data().(types.EventDataNewRound)
require.True(t, ok, "expected a EventDataNewRound, got %T. Wrong subscription channel?",
msg.Data())
require.Equal(t, height, newRoundEvent.Height)
require.Equal(t, round, newRoundEvent.Round)
}
require.Equal(t, height, newRoundEvent.Height)
require.Equal(t, round, newRoundEvent.Round)
}
func ensureNewTimeout(t *testing.T, timeoutCh <-chan tmpubsub.Message, height int64, round int32, timeout int64) {
t.Helper()
timeoutDuration := time.Duration(timeout*10) * time.Nanosecond
ensureNewEvent(t, timeoutCh, height, round, timeoutDuration,
"Timeout expired while waiting for NewTimeout event")
ensureNewEvent(t, timeoutCh, height, round, timeoutDuration)
}
func ensureNewProposal(t *testing.T, proposalCh <-chan tmpubsub.Message, height int64, round int32) {
t.Helper()
select {
case <-time.After(ensureTimeout):
t.Fatal("Timeout expired while waiting for NewProposal event")
case msg := <-proposalCh:
proposalEvent, ok := msg.Data().(types.EventDataCompleteProposal)
require.True(t, ok, "expected a EventDataCompleteProposal, got %T. Wrong subscription channel?",
msg.Data())
require.Equal(t, height, proposalEvent.Height)
require.Equal(t, round, proposalEvent.Round)
}
msg := ensureMessageBeforeTimeout(t, proposalCh, ensureTimeout)
proposalEvent, ok := msg.Data().(types.EventDataCompleteProposal)
require.True(t, ok, "expected a EventDataCompleteProposal, got %T. Wrong subscription channel?",
msg.Data())
require.Equal(t, height, proposalEvent.Height)
require.Equal(t, round, proposalEvent.Round)
}
func ensureNewValidBlock(t *testing.T, validBlockCh <-chan tmpubsub.Message, height int64, round int32) {
t.Helper()
ensureNewEvent(t, validBlockCh, height, round, ensureTimeout,
"Timeout expired while waiting for NewValidBlock event")
ensureNewEvent(t, validBlockCh, height, round, ensureTimeout)
}
func ensureNewBlock(t *testing.T, blockCh <-chan tmpubsub.Message, height int64) {
t.Helper()
select {
case <-time.After(ensureTimeout):
t.Fatal("Timeout expired while waiting for NewBlock event")
case msg := <-blockCh:
blockEvent, ok := msg.Data().(types.EventDataNewBlock)
require.True(t, ok, "expected a EventDataNewBlock, got %T. Wrong subscription channel?",
msg.Data())
require.Equal(t, height, blockEvent.Block.Height)
}
msg := ensureMessageBeforeTimeout(t, blockCh, ensureTimeout)
blockEvent, ok := msg.Data().(types.EventDataNewBlock)
require.True(t, ok, "expected a EventDataNewBlock, got %T. Wrong subscription channel?",
msg.Data())
require.Equal(t, height, blockEvent.Block.Height)
}
func ensureNewBlockHeader(t *testing.T, blockCh <-chan tmpubsub.Message, height int64, blockHash tmbytes.HexBytes) {
t.Helper()
select {
case <-time.After(ensureTimeout):
t.Fatal("Timeout expired while waiting for NewBlockHeader event")
case msg := <-blockCh:
blockHeaderEvent, ok := msg.Data().(types.EventDataNewBlockHeader)
require.True(t, ok, "expected a EventDataNewBlockHeader, got %T. Wrong subscription channel?",
msg.Data())
msg := ensureMessageBeforeTimeout(t, blockCh, ensureTimeout)
blockHeaderEvent, ok := msg.Data().(types.EventDataNewBlockHeader)
require.True(t, ok, "expected a EventDataNewBlockHeader, got %T. Wrong subscription channel?",
msg.Data())
require.Equal(t, height, blockHeaderEvent.Header.Height)
require.True(t, bytes.Equal(blockHeaderEvent.Header.Hash(), blockHash))
}
require.Equal(t, height, blockHeaderEvent.Header.Height)
require.True(t, bytes.Equal(blockHeaderEvent.Header.Hash(), blockHash))
}
func ensureLock(t *testing.T, lockCh <-chan tmpubsub.Message, height int64, round int32) {
t.Helper()
ensureNewEvent(t, lockCh, height, round, ensureTimeout)
}
func ensureNewUnlock(t *testing.T, unlockCh <-chan tmpubsub.Message, height int64, round int32) {
func ensureRelock(t *testing.T, relockCh <-chan tmpubsub.Message, height int64, round int32) {
t.Helper()
ensureNewEvent(t, unlockCh, height, round, ensureTimeout,
"Timeout expired while waiting for NewUnlock event")
ensureNewEvent(t, relockCh, height, round, ensureTimeout)
}
func ensureProposal(t *testing.T, proposalCh <-chan tmpubsub.Message, height int64, round int32, propID types.BlockID) {
ensureProposalWithTimeout(t, proposalCh, height, round, &propID, ensureTimeout)
}
func ensureProposalWithTimeout(t *testing.T, proposalCh <-chan tmpubsub.Message, height int64, round int32, propID *types.BlockID, timeout time.Duration) {
t.Helper()
select {
case <-time.After(ensureTimeout):
t.Fatal("Timeout expired while waiting for NewProposal event")
case msg := <-proposalCh:
proposalEvent, ok := msg.Data().(types.EventDataCompleteProposal)
require.True(t, ok, "expected a EventDataCompleteProposal, got %T. Wrong subscription channel?",
msg.Data())
require.Equal(t, height, proposalEvent.Height)
require.Equal(t, round, proposalEvent.Round)
require.True(t, proposalEvent.BlockID.Equals(propID),
msg := ensureMessageBeforeTimeout(t, proposalCh, timeout)
proposalEvent, ok := msg.Data().(types.EventDataCompleteProposal)
require.True(t, ok, "expected a EventDataCompleteProposal, got %T. Wrong subscription channel?",
msg.Data())
require.Equal(t, height, proposalEvent.Height)
require.Equal(t, round, proposalEvent.Round)
if propID != nil {
require.True(t, proposalEvent.BlockID.Equals(*propID),
"Proposed block does not match expected block (%v != %v)", proposalEvent.BlockID, propID)
}
}
@ -703,38 +701,32 @@ func ensureVoteMatch(t *testing.T, voteCh <-chan tmpubsub.Message, height int64,
}
func ensureVote(t *testing.T, voteCh <-chan tmpubsub.Message, height int64, round int32, voteType tmproto.SignedMsgType) {
t.Helper()
select {
case <-time.After(ensureTimeout):
t.Fatal("Timeout expired while waiting for NewVote event")
case msg := <-voteCh:
voteEvent, ok := msg.Data().(types.EventDataVote)
require.True(t, ok, "expected a EventDataVote, got %T. Wrong subscription channel?",
msg.Data())
msg := ensureMessageBeforeTimeout(t, voteCh, ensureTimeout)
voteEvent, ok := msg.Data().(types.EventDataVote)
require.True(t, ok, "expected a EventDataVote, got %T. Wrong subscription channel?",
msg.Data())
vote := voteEvent.Vote
require.Equal(t, height, vote.Height)
require.Equal(t, round, vote.Round)
vote := voteEvent.Vote
require.Equal(t, height, vote.Height)
require.Equal(t, round, vote.Round)
require.Equal(t, voteType, vote.Type)
}
require.Equal(t, voteType, vote.Type)
}
func ensurePrecommitTimeout(t *testing.T, ch <-chan tmpubsub.Message) {
func ensureNewEventOnChannel(t *testing.T, ch <-chan tmpubsub.Message) {
t.Helper()
select {
case <-time.After(ensureTimeout):
t.Fatal("Timeout expired while waiting for the Precommit to Timeout")
case <-ch:
}
ensureMessageBeforeTimeout(t, ch, ensureTimeout)
}
func ensureNewEventOnChannel(t *testing.T, ch <-chan tmpubsub.Message) {
func ensureMessageBeforeTimeout(t *testing.T, ch <-chan tmpubsub.Message, to time.Duration) tmpubsub.Message {
t.Helper()
select {
case <-time.After(ensureTimeout):
t.Fatal("Timeout expired while waiting for new activity on the channel")
case <-ch:
case <-time.After(to):
t.Fatalf("Timeout expired while waiting for message")
case msg := <-ch:
return msg
}
panic("unreachable")
}
//-------------------------------------------------------------------------------
@ -746,7 +738,7 @@ func consensusLogger() log.Logger {
return log.TestingLogger().With("module", "consensus")
}
func randConsensusState(
func makeConsensusState(
ctx context.Context,
t *testing.T,
cfg *config.Config,
@ -756,8 +748,10 @@ func randConsensusState(
appFunc func(t *testing.T, logger log.Logger) abci.Application,
configOpts ...func(*config.Config),
) ([]*State, cleanupFunc) {
t.Helper()
genDoc, privVals := factory.RandGenesisDoc(ctx, t, cfg, nValidators, false, 30)
valSet, privVals := factory.ValidatorSet(ctx, t, nValidators, 30)
genDoc := factory.GenesisDoc(cfg, time.Now(), valSet.Validators, nil)
css := make([]*State, nValidators)
logger := consensusLogger()
@ -817,8 +811,10 @@ func randConsensusNetWithPeers(
) ([]*State, *types.GenesisDoc, *config.Config, cleanupFunc) {
t.Helper()
genDoc, privVals := factory.RandGenesisDoc(ctx, t, cfg, nValidators, false, testMinPower)
valSet, privVals := factory.ValidatorSet(ctx, t, nValidators, testMinPower)
genDoc := factory.GenesisDoc(cfg, time.Now(), valSet.Validators, nil)
css := make([]*State, nPeers)
t.Helper()
logger := consensusLogger()
var peer0Config *config.Config
@ -866,16 +862,29 @@ func randConsensusNetWithPeers(
}
}
func randGenesisState(
ctx context.Context,
t *testing.T,
cfg *config.Config,
numValidators int,
randPower bool,
minPower int64,
) (sm.State, []types.PrivValidator) {
type genesisStateArgs struct {
Validators int
Power int64
Params *types.ConsensusParams
Time time.Time
}
genDoc, privValidators := factory.RandGenesisDoc(ctx, t, cfg, numValidators, randPower, minPower)
func makeGenesisState(ctx context.Context, t *testing.T, cfg *config.Config, args genesisStateArgs) (sm.State, []types.PrivValidator) {
t.Helper()
if args.Power == 0 {
args.Power = 1
}
if args.Validators == 0 {
args.Power = 4
}
valSet, privValidators := factory.ValidatorSet(ctx, t, args.Validators, args.Power)
if args.Params == nil {
args.Params = types.DefaultConsensusParams()
}
if args.Time.IsZero() {
args.Time = time.Now()
}
genDoc := factory.GenesisDoc(cfg, args.Time, valSet.Validators, args.Params)
s0, err := sm.MakeGenesisState(genDoc)
require.NoError(t, err)
return s0, privValidators
@ -884,7 +893,7 @@ func randGenesisState(
func newMockTickerFunc(onlyOnce bool) func() TimeoutTicker {
return func() TimeoutTicker {
return &mockTicker{
c: make(chan timeoutInfo, 10),
c: make(chan timeoutInfo, 100),
onlyOnce: onlyOnce,
}
}


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

@ -11,6 +11,7 @@ import (
"github.com/tendermint/tendermint/internal/p2p"
"github.com/tendermint/tendermint/libs/bytes"
tmrand "github.com/tendermint/tendermint/libs/rand"
tmtime "github.com/tendermint/tendermint/libs/time"
tmcons "github.com/tendermint/tendermint/proto/tendermint/consensus"
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
"github.com/tendermint/tendermint/types"
@ -23,7 +24,7 @@ func TestReactorInvalidPrecommit(t *testing.T) {
config := configSetup(t)
n := 4
states, cleanup := randConsensusState(ctx, t,
states, cleanup := makeConsensusState(ctx, t,
config, n, "consensus_reactor_test",
newMockTickerFunc(true), newKVStore)
t.Cleanup(cleanup)
@ -107,7 +108,7 @@ func invalidDoPrevoteFunc(
ValidatorIndex: valIndex,
Height: cs.Height,
Round: cs.Round,
Timestamp: cs.voteTime(),
Timestamp: tmtime.Now(),
Type: tmproto.PrecommitType,
BlockID: types.BlockID{
Hash: blockHash,


+ 17
- 7
internal/consensus/mempool_test.go View File

@ -40,7 +40,9 @@ func TestMempoolNoProgressUntilTxsAvailable(t *testing.T) {
t.Cleanup(func() { _ = os.RemoveAll(config.RootDir) })
config.Consensus.CreateEmptyBlocks = false
state, privVals := randGenesisState(ctx, t, baseConfig, 1, false, 10)
state, privVals := makeGenesisState(ctx, t, baseConfig, genesisStateArgs{
Validators: 1,
Power: 10})
cs := newStateWithConfig(ctx, t, log.TestingLogger(), config, state, privVals[0], NewCounterApplication())
assertMempool(t, cs.txNotifier).EnableTxsAvailable()
height, round := cs.Height, cs.Round
@ -65,7 +67,9 @@ func TestMempoolProgressAfterCreateEmptyBlocksInterval(t *testing.T) {
t.Cleanup(func() { _ = os.RemoveAll(config.RootDir) })
config.Consensus.CreateEmptyBlocksInterval = ensureTimeout
state, privVals := randGenesisState(ctx, t, baseConfig, 1, false, 10)
state, privVals := makeGenesisState(ctx, t, baseConfig, genesisStateArgs{
Validators: 1,
Power: 10})
cs := newStateWithConfig(ctx, t, log.TestingLogger(), config, state, privVals[0], NewCounterApplication())
assertMempool(t, cs.txNotifier).EnableTxsAvailable()
@ -88,20 +92,22 @@ func TestMempoolProgressInHigherRound(t *testing.T) {
t.Cleanup(func() { _ = os.RemoveAll(config.RootDir) })
config.Consensus.CreateEmptyBlocks = false
state, privVals := randGenesisState(ctx, t, baseConfig, 1, false, 10)
state, privVals := makeGenesisState(ctx, t, baseConfig, genesisStateArgs{
Validators: 1,
Power: 10})
cs := newStateWithConfig(ctx, t, log.TestingLogger(), config, state, privVals[0], NewCounterApplication())
assertMempool(t, cs.txNotifier).EnableTxsAvailable()
height, round := cs.Height, cs.Round
newBlockCh := subscribe(ctx, t, cs.eventBus, types.EventQueryNewBlock)
newRoundCh := subscribe(ctx, t, cs.eventBus, types.EventQueryNewRound)
timeoutCh := subscribe(ctx, t, cs.eventBus, types.EventQueryTimeoutPropose)
cs.setProposal = func(proposal *types.Proposal) error {
cs.setProposal = func(proposal *types.Proposal, recvTime time.Time) error {
if cs.Height == 2 && cs.Round == 0 {
// dont set the proposal in round 0 so we timeout and
// go to next round
return nil
}
return cs.defaultSetProposal(proposal)
return cs.defaultSetProposal(proposal, recvTime)
}
startTestRound(ctx, cs, height, round)
@ -137,7 +143,9 @@ func TestMempoolTxConcurrentWithCommit(t *testing.T) {
config := configSetup(t)
logger := log.TestingLogger()
state, privVals := randGenesisState(ctx, t, config, 1, false, 10)
state, privVals := makeGenesisState(ctx, t, config, genesisStateArgs{
Validators: 1,
Power: 10})
stateStore := sm.NewStore(dbm.NewMemDB())
blockStore := store.NewBlockStore(dbm.NewMemDB())
@ -170,7 +178,9 @@ func TestMempoolRmBadTx(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
state, privVals := randGenesisState(ctx, t, config, 1, false, 10)
state, privVals := makeGenesisState(ctx, t, config, genesisStateArgs{
Validators: 1,
Power: 10})
app := NewCounterApplication()
stateStore := sm.NewStore(dbm.NewMemDB())
blockStore := store.NewBlockStore(dbm.NewMemDB())


+ 489
- 0
internal/consensus/pbts_test.go View File

@ -0,0 +1,489 @@
package consensus
import (
"bytes"
"context"
"testing"
"time"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"github.com/tendermint/tendermint/abci/example/kvstore"
"github.com/tendermint/tendermint/internal/eventbus"
tmpubsub "github.com/tendermint/tendermint/internal/pubsub"
"github.com/tendermint/tendermint/libs/log"
tmtimemocks "github.com/tendermint/tendermint/libs/time/mocks"
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
"github.com/tendermint/tendermint/types"
)
const (
// blockTimeIota is used in the test harness as the time between
// blocks when not otherwise specified.
blockTimeIota = time.Millisecond
)
// pbtsTestHarness constructs a Tendermint network that can be used for testing the
// implementation of the Proposer-Based timestamps algorithm.
// It runs a series of consensus heights and captures timing of votes and events.
type pbtsTestHarness struct {
// configuration options set by the user of the test harness.
pbtsTestConfiguration
// The Tendermint consensus state machine being run during
// a run of the pbtsTestHarness.
observedState *State
// A stub for signing votes and messages using the key
// from the observedState.
observedValidator *validatorStub
// A list of simulated validators that interact with the observedState and are
// fully controlled by the test harness.
otherValidators []*validatorStub
// The mock time source used by all of the validator stubs in the test harness.
// This mock clock allows the test harness to produce votes and blocks with arbitrary
// timestamps.
validatorClock *tmtimemocks.Source
chainID string
// channels for verifying that the observed validator completes certain actions.
ensureProposalCh, roundCh, blockCh, ensureVoteCh <-chan tmpubsub.Message
// channel of events from the observed validator annotated with the timestamp
// the event was received.
eventCh <-chan timestampedEvent
currentHeight int64
currentRound int32
}
type pbtsTestConfiguration struct {
// The timestamp consensus parameters to be used by the state machine under test.
synchronyParams types.SynchronyParams
// The setting to use for the TimeoutPropose configuration parameter.
timeoutPropose time.Duration
// The timestamp of the first block produced by the network.
genesisTime time.Time
// The time at which the proposal at height 2 should be delivered.
height2ProposalDeliverTime time.Time
// The timestamp of the block proposed at height 2.
height2ProposedBlockTime time.Time
// The timestamp of the block proposed at height 4.
// At height 4, the proposed block time and the deliver time are the same so
// that timely-ness does not affect height 4.
height4ProposedBlockTime time.Time
}
func newPBTSTestHarness(ctx context.Context, t *testing.T, tc pbtsTestConfiguration) pbtsTestHarness {
t.Helper()
const validators = 4
cfg := configSetup(t)
clock := new(tmtimemocks.Source)
if tc.height4ProposedBlockTime.IsZero() {
// Set a default height4ProposedBlockTime.
// Use a proposed block time that is greater than the time that the
// block at height 2 was delivered. Height 3 is not relevant for testing
// and always occurs blockTimeIota before height 4. If not otherwise specified,
// height 4 therefore occurs 2*blockTimeIota after height 2.
tc.height4ProposedBlockTime = tc.height2ProposalDeliverTime.Add(2 * blockTimeIota)
}
cfg.Consensus.TimeoutPropose = tc.timeoutPropose
consensusParams := types.DefaultConsensusParams()
consensusParams.Synchrony = tc.synchronyParams
state, privVals := makeGenesisState(ctx, t, cfg, genesisStateArgs{
Params: consensusParams,
Time: tc.genesisTime,
Validators: validators,
})
cs := newState(ctx, t, log.TestingLogger(), state, privVals[0], kvstore.NewApplication())
vss := make([]*validatorStub, validators)
for i := 0; i < validators; i++ {
vss[i] = newValidatorStub(privVals[i], int32(i))
}
incrementHeight(vss[1:]...)
for _, vs := range vss {
vs.clock = clock
}
pubKey, err := vss[0].PrivValidator.GetPubKey(ctx)
require.NoError(t, err)
eventCh := timestampedCollector(ctx, t, cs.eventBus)
return pbtsTestHarness{
pbtsTestConfiguration: tc,
observedValidator: vss[0],
observedState: cs,
otherValidators: vss[1:],
validatorClock: clock,
currentHeight: 1,
chainID: cfg.ChainID(),
roundCh: subscribe(ctx, t, cs.eventBus, types.EventQueryNewRound),
ensureProposalCh: subscribe(ctx, t, cs.eventBus, types.EventQueryCompleteProposal),
blockCh: subscribe(ctx, t, cs.eventBus, types.EventQueryNewBlock),
ensureVoteCh: subscribeToVoterBuffered(ctx, t, cs, pubKey.Address()),
eventCh: eventCh,
}
}
func (p *pbtsTestHarness) observedValidatorProposerHeight(ctx context.Context, t *testing.T, previousBlockTime time.Time) heightResult {
p.validatorClock.On("Now").Return(p.height2ProposedBlockTime).Times(6)
ensureNewRound(t, p.roundCh, p.currentHeight, p.currentRound)
timeout := time.Until(previousBlockTime.Add(ensureTimeout))
ensureProposalWithTimeout(t, p.ensureProposalCh, p.currentHeight, p.currentRound, nil, timeout)
rs := p.observedState.GetRoundState()
bid := types.BlockID{Hash: rs.ProposalBlock.Hash(), PartSetHeader: rs.ProposalBlockParts.Header()}
ensurePrevote(t, p.ensureVoteCh, p.currentHeight, p.currentRound)
signAddVotes(ctx, t, p.observedState, tmproto.PrevoteType, p.chainID, bid, p.otherValidators...)
signAddVotes(ctx, t, p.observedState, tmproto.PrecommitType, p.chainID, bid, p.otherValidators...)
ensurePrecommit(t, p.ensureVoteCh, p.currentHeight, p.currentRound)
ensureNewBlock(t, p.blockCh, p.currentHeight)
vk, err := p.observedValidator.GetPubKey(ctx)
require.NoError(t, err)
res := collectHeightResults(ctx, t, p.eventCh, p.currentHeight, vk.Address())
p.currentHeight++
incrementHeight(p.otherValidators...)
return res
}
func (p *pbtsTestHarness) height2(ctx context.Context, t *testing.T) heightResult {
signer := p.otherValidators[0].PrivValidator
height3BlockTime := p.height2ProposedBlockTime.Add(-blockTimeIota)
return p.nextHeight(ctx, t, signer, p.height2ProposalDeliverTime, p.height2ProposedBlockTime, height3BlockTime)
}
func (p *pbtsTestHarness) intermediateHeights(ctx context.Context, t *testing.T) {
signer := p.otherValidators[1].PrivValidator
blockTimeHeight3 := p.height4ProposedBlockTime.Add(-blockTimeIota)
p.nextHeight(ctx, t, signer, blockTimeHeight3, blockTimeHeight3, p.height4ProposedBlockTime)
signer = p.otherValidators[2].PrivValidator
p.nextHeight(ctx, t, signer, p.height4ProposedBlockTime, p.height4ProposedBlockTime, time.Now())
}
func (p *pbtsTestHarness) height5(ctx context.Context, t *testing.T) heightResult {
return p.observedValidatorProposerHeight(ctx, t, p.height4ProposedBlockTime)
}
func (p *pbtsTestHarness) nextHeight(ctx context.Context, t *testing.T, proposer types.PrivValidator, deliverTime, proposedTime, nextProposedTime time.Time) heightResult {
p.validatorClock.On("Now").Return(nextProposedTime).Times(6)
ensureNewRound(t, p.roundCh, p.currentHeight, p.currentRound)
b, _, err := p.observedState.createProposalBlock()
require.NoError(t, err)
b.Height = p.currentHeight
b.Header.Height = p.currentHeight
b.Header.Time = proposedTime
k, err := proposer.GetPubKey(ctx)
require.NoError(t, err)
b.Header.ProposerAddress = k.Address()
ps, err := b.MakePartSet(types.BlockPartSizeBytes)
require.NoError(t, err)
bid := types.BlockID{Hash: b.Hash(), PartSetHeader: ps.Header()}
prop := types.NewProposal(p.currentHeight, 0, -1, bid, proposedTime)
tp := prop.ToProto()
if err := proposer.SignProposal(ctx, p.observedState.state.ChainID, tp); err != nil {
t.Fatalf("error signing proposal: %s", err)
}
time.Sleep(time.Until(deliverTime))
prop.Signature = tp.Signature
if err := p.observedState.SetProposalAndBlock(ctx, prop, b, ps, "peerID"); err != nil {
t.Fatal(err)
}
ensureProposal(t, p.ensureProposalCh, p.currentHeight, 0, bid)
ensurePrevote(t, p.ensureVoteCh, p.currentHeight, p.currentRound)
signAddVotes(ctx, t, p.observedState, tmproto.PrevoteType, p.chainID, bid, p.otherValidators...)
signAddVotes(ctx, t, p.observedState, tmproto.PrecommitType, p.chainID, bid, p.otherValidators...)
ensurePrecommit(t, p.ensureVoteCh, p.currentHeight, p.currentRound)
vk, err := p.observedValidator.GetPubKey(ctx)
require.NoError(t, err)
res := collectHeightResults(ctx, t, p.eventCh, p.currentHeight, vk.Address())
ensureNewBlock(t, p.blockCh, p.currentHeight)
p.currentHeight++
incrementHeight(p.otherValidators...)
return res
}
func timestampedCollector(ctx context.Context, t *testing.T, eb *eventbus.EventBus) <-chan timestampedEvent {
t.Helper()
// Since eventCh is not read until the end of each height, it must be large
// enough to hold all of the events produced during a single height.
eventCh := make(chan timestampedEvent, 100)
if err := eb.Observe(ctx, func(msg tmpubsub.Message) error {
eventCh <- timestampedEvent{
ts: time.Now(),
m: msg,
}
return nil
}, types.EventQueryVote, types.EventQueryCompleteProposal); err != nil {
t.Fatalf("Failed to observe query %v: %v", types.EventQueryVote, err)
}
return eventCh
}
func collectHeightResults(ctx context.Context, t *testing.T, eventCh <-chan timestampedEvent, height int64, address []byte) heightResult {
t.Helper()
var res heightResult
for event := range eventCh {
switch v := event.m.Data().(type) {
case types.EventDataVote:
if v.Vote.Height > height {
t.Fatalf("received prevote from unexpected height, expected: %d, saw: %d", height, v.Vote.Height)
}
if !bytes.Equal(address, v.Vote.ValidatorAddress) {
continue
}
if v.Vote.Type != tmproto.PrevoteType {
continue
}
res.prevote = v.Vote
res.prevoteIssuedAt = event.ts
case types.EventDataCompleteProposal:
if v.Height > height {
t.Fatalf("received proposal from unexpected height, expected: %d, saw: %d", height, v.Height)
}
res.proposalIssuedAt = event.ts
}
if res.isComplete() {
return res
}
}
t.Fatalf("complete height result never seen for height %d", height)
panic("unreachable")
}
type timestampedEvent struct {
ts time.Time
m tmpubsub.Message
}
func (p *pbtsTestHarness) run(ctx context.Context, t *testing.T) resultSet {
startTestRound(ctx, p.observedState, p.currentHeight, p.currentRound)
r1 := p.observedValidatorProposerHeight(ctx, t, p.genesisTime)
r2 := p.height2(ctx, t)
p.intermediateHeights(ctx, t)
r5 := p.height5(ctx, t)
return resultSet{
genesisHeight: r1,
height2: r2,
height5: r5,
}
}
type resultSet struct {
genesisHeight heightResult
height2 heightResult
height5 heightResult
}
type heightResult struct {
proposalIssuedAt time.Time
prevote *types.Vote
prevoteIssuedAt time.Time
}
func (hr heightResult) isComplete() bool {
return !hr.proposalIssuedAt.IsZero() && !hr.prevoteIssuedAt.IsZero() && hr.prevote != nil
}
// TestProposerWaitsForGenesisTime tests that a proposer will not propose a block
// until after the genesis time has passed. The test sets the genesis time in the
// future and then ensures that the observed validator waits to propose a block.
func TestProposerWaitsForGenesisTime(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
// create a genesis time far (enough) in the future.
initialTime := time.Now().Add(800 * time.Millisecond)
cfg := pbtsTestConfiguration{
synchronyParams: types.SynchronyParams{
Precision: 10 * time.Millisecond,
MessageDelay: 10 * time.Millisecond,
},
timeoutPropose: 10 * time.Millisecond,
genesisTime: initialTime,
height2ProposalDeliverTime: initialTime.Add(10 * time.Millisecond),
height2ProposedBlockTime: initialTime.Add(10 * time.Millisecond),
}
pbtsTest := newPBTSTestHarness(ctx, t, cfg)
results := pbtsTest.run(ctx, t)
// ensure that the proposal was issued after the genesis time.
assert.True(t, results.genesisHeight.proposalIssuedAt.After(cfg.genesisTime))
}
// TestProposerWaitsForPreviousBlock tests that the proposer of a block waits until
// the block time of the previous height has passed to propose the next block.
// The test harness ensures that the observed validator will be the proposer at
// height 1 and height 5. The test sets the block time of height 4 in the future
// and then verifies that the observed validator waits until after the block time
// of height 4 to propose a block at height 5.
func TestProposerWaitsForPreviousBlock(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
initialTime := time.Now().Add(time.Millisecond * 50)
cfg := pbtsTestConfiguration{
synchronyParams: types.SynchronyParams{
Precision: 100 * time.Millisecond,
MessageDelay: 500 * time.Millisecond,
},
timeoutPropose: 50 * time.Millisecond,
genesisTime: initialTime,
height2ProposalDeliverTime: initialTime.Add(150 * time.Millisecond),
height2ProposedBlockTime: initialTime.Add(100 * time.Millisecond),
height4ProposedBlockTime: initialTime.Add(800 * time.Millisecond),
}
pbtsTest := newPBTSTestHarness(ctx, t, cfg)
results := pbtsTest.run(ctx, t)
// the observed validator is the proposer at height 5.
// ensure that the observed validator did not propose a block until after
// the time configured for height 4.
assert.True(t, results.height5.proposalIssuedAt.After(cfg.height4ProposedBlockTime))
// Ensure that the validator issued a prevote for a non-nil block.
assert.NotNil(t, results.height5.prevote.BlockID.Hash)
}
func TestProposerWaitTime(t *testing.T) {
genesisTime, err := time.Parse(time.RFC3339, "2019-03-13T23:00:00Z")
require.NoError(t, err)
testCases := []struct {
name string
previousBlockTime time.Time
localTime time.Time
expectedWait time.Duration
}{
{
name: "block time greater than local time",
previousBlockTime: genesisTime.Add(5 * time.Nanosecond),
localTime: genesisTime.Add(1 * time.Nanosecond),
expectedWait: 4 * time.Nanosecond,
},
{
name: "local time greater than block time",
previousBlockTime: genesisTime.Add(1 * time.Nanosecond),
localTime: genesisTime.Add(5 * time.Nanosecond),
expectedWait: 0,
},
{
name: "both times equal",
previousBlockTime: genesisTime.Add(5 * time.Nanosecond),
localTime: genesisTime.Add(5 * time.Nanosecond),
expectedWait: 0,
},
}
for _, testCase := range testCases {
t.Run(testCase.name, func(t *testing.T) {
mockSource := new(tmtimemocks.Source)
mockSource.On("Now").Return(testCase.localTime)
ti := proposerWaitTime(mockSource, testCase.previousBlockTime)
assert.Equal(t, testCase.expectedWait, ti)
})
}
}
func TestTimelyProposal(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
initialTime := time.Now()
cfg := pbtsTestConfiguration{
synchronyParams: types.SynchronyParams{
Precision: 10 * time.Millisecond,
MessageDelay: 140 * time.Millisecond,
},
timeoutPropose: 40 * time.Millisecond,
genesisTime: initialTime,
height2ProposedBlockTime: initialTime.Add(10 * time.Millisecond),
height2ProposalDeliverTime: initialTime.Add(30 * time.Millisecond),
}
pbtsTest := newPBTSTestHarness(ctx, t, cfg)
results := pbtsTest.run(ctx, t)
require.NotNil(t, results.height2.prevote.BlockID.Hash)
}
func TestTooFarInThePastProposal(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
initialTime := time.Now()
// localtime > proposedBlockTime + MsgDelay + Precision
cfg := pbtsTestConfiguration{
synchronyParams: types.SynchronyParams{
Precision: 1 * time.Millisecond,
MessageDelay: 10 * time.Millisecond,
},
timeoutPropose: 50 * time.Millisecond,
genesisTime: initialTime,
height2ProposedBlockTime: initialTime.Add(10 * time.Millisecond),
height2ProposalDeliverTime: initialTime.Add(21 * time.Millisecond),
}
pbtsTest := newPBTSTestHarness(ctx, t, cfg)
results := pbtsTest.run(ctx, t)
require.Nil(t, results.height2.prevote.BlockID.Hash)
}
func TestTooFarInTheFutureProposal(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
initialTime := time.Now()
// localtime < proposedBlockTime - Precision
cfg := pbtsTestConfiguration{
synchronyParams: types.SynchronyParams{
Precision: 1 * time.Millisecond,
MessageDelay: 10 * time.Millisecond,
},
timeoutPropose: 50 * time.Millisecond,
genesisTime: initialTime,
height2ProposedBlockTime: initialTime.Add(100 * time.Millisecond),
height2ProposalDeliverTime: initialTime.Add(10 * time.Millisecond),
height4ProposedBlockTime: initialTime.Add(150 * time.Millisecond),
}
pbtsTest := newPBTSTestHarness(ctx, t, cfg)
results := pbtsTest.run(ctx, t)
require.Nil(t, results.height2.prevote.BlockID.Hash)
}

+ 4
- 3
internal/consensus/reactor.go View File

@ -16,6 +16,7 @@ import (
tmevents "github.com/tendermint/tendermint/libs/events"
"github.com/tendermint/tendermint/libs/log"
"github.com/tendermint/tendermint/libs/service"
tmtime "github.com/tendermint/tendermint/libs/time"
tmcons "github.com/tendermint/tendermint/proto/tendermint/consensus"
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
"github.com/tendermint/tendermint/types"
@ -1184,7 +1185,7 @@ func (r *Reactor) handleDataMessage(ctx context.Context, envelope *p2p.Envelope,
select {
case <-ctx.Done():
return ctx.Err()
case r.state.peerMsgQueue <- msgInfo{pMsg, envelope.From}:
case r.state.peerMsgQueue <- msgInfo{pMsg, envelope.From, tmtime.Now()}:
}
case *tmcons.ProposalPOL:
ps.ApplyProposalPOLMessage(msgI.(*ProposalPOLMessage))
@ -1194,7 +1195,7 @@ func (r *Reactor) handleDataMessage(ctx context.Context, envelope *p2p.Envelope,
ps.SetHasProposalBlockPart(bpMsg.Height, bpMsg.Round, int(bpMsg.Part.Index))
r.Metrics.BlockParts.With("peer_id", string(envelope.From)).Add(1)
select {
case r.state.peerMsgQueue <- msgInfo{bpMsg, envelope.From}:
case r.state.peerMsgQueue <- msgInfo{bpMsg, envelope.From, tmtime.Now()}:
return nil
case <-ctx.Done():
return ctx.Err()
@ -1238,7 +1239,7 @@ func (r *Reactor) handleVoteMessage(ctx context.Context, envelope *p2p.Envelope,
ps.SetHasVote(vMsg.Vote)
select {
case r.state.peerMsgQueue <- msgInfo{vMsg, envelope.From}:
case r.state.peerMsgQueue <- msgInfo{vMsg, envelope.From, tmtime.Now()}:
return nil
case <-ctx.Done():
return ctx.Err()


+ 6
- 7
internal/consensus/reactor_test.go View File

@ -324,7 +324,7 @@ func TestReactorBasic(t *testing.T) {
cfg := configSetup(t)
n := 4
states, cleanup := randConsensusState(ctx, t,
states, cleanup := makeConsensusState(ctx, t,
cfg, n, "consensus_reactor_test",
newMockTickerFunc(true), newKVStore)
t.Cleanup(cleanup)
@ -381,7 +381,8 @@ func TestReactorWithEvidence(t *testing.T) {
tickerFunc := newMockTickerFunc(true)
appFunc := newKVStore
genDoc, privVals := factory.RandGenesisDoc(ctx, t, cfg, n, false, 30)
valSet, privVals := factory.ValidatorSet(ctx, t, n, 30)
genDoc := factory.GenesisDoc(cfg, time.Now(), valSet.Validators, nil)
states := make([]*State, n)
logger := consensusLogger()
@ -484,8 +485,7 @@ func TestReactorCreatesBlockWhenEmptyBlocksFalse(t *testing.T) {
cfg := configSetup(t)
n := 4
states, cleanup := randConsensusState(
ctx,
states, cleanup := makeConsensusState(ctx,
t,
cfg,
n,
@ -541,7 +541,7 @@ func TestReactorRecordsVotesAndBlockParts(t *testing.T) {
cfg := configSetup(t)
n := 4
states, cleanup := randConsensusState(ctx, t,
states, cleanup := makeConsensusState(ctx, t,
cfg, n, "consensus_reactor_test",
newMockTickerFunc(true), newKVStore)
t.Cleanup(cleanup)
@ -606,8 +606,7 @@ func TestReactorVotingPowerChange(t *testing.T) {
cfg := configSetup(t)
n := 4
states, cleanup := randConsensusState(
ctx,
states, cleanup := makeConsensusState(ctx,
t,
cfg,
n,


+ 29
- 24
internal/consensus/replay_test.go View File

@ -367,8 +367,8 @@ func setupSimulator(ctx context.Context, t *testing.T) *simulatorTestSuite {
ensureNewProposal(t, proposalCh, height, round)
rs := css[0].GetRoundState()
signAddVotes(ctx, t, sim.Config, css[0], tmproto.PrecommitType,
rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header(),
signAddVotes(ctx, t, css[0], tmproto.PrecommitType, sim.Config.ChainID(),
types.BlockID{Hash: rs.ProposalBlock.Hash(), PartSetHeader: rs.ProposalBlockParts.Header()},
vss[1:nVals]...)
ensureNewRound(t, newRoundCh, height+1, 0)
@ -389,7 +389,7 @@ func setupSimulator(ctx context.Context, t *testing.T) *simulatorTestSuite {
require.NoError(t, err)
blockID := types.BlockID{Hash: propBlock.Hash(), PartSetHeader: propBlockParts.Header()}
proposal := types.NewProposal(vss[1].Height, round, -1, blockID)
proposal := types.NewProposal(vss[1].Height, round, -1, blockID, propBlock.Header.Time)
p := proposal.ToProto()
if err := vss[1].SignProposal(ctx, cfg.ChainID(), p); err != nil {
t.Fatal("failed to sign bad proposal", err)
@ -402,8 +402,8 @@ func setupSimulator(ctx context.Context, t *testing.T) *simulatorTestSuite {
}
ensureNewProposal(t, proposalCh, height, round)
rs = css[0].GetRoundState()
signAddVotes(ctx, t, sim.Config, css[0], tmproto.PrecommitType,
rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header(),
signAddVotes(ctx, t, css[0], tmproto.PrecommitType, sim.Config.ChainID(),
types.BlockID{Hash: rs.ProposalBlock.Hash(), PartSetHeader: rs.ProposalBlockParts.Header()},
vss[1:nVals]...)
ensureNewRound(t, newRoundCh, height+1, 0)
@ -423,7 +423,7 @@ func setupSimulator(ctx context.Context, t *testing.T) *simulatorTestSuite {
require.NoError(t, err)
blockID = types.BlockID{Hash: propBlock.Hash(), PartSetHeader: propBlockParts.Header()}
proposal = types.NewProposal(vss[2].Height, round, -1, blockID)
proposal = types.NewProposal(vss[2].Height, round, -1, blockID, propBlock.Header.Time)
p = proposal.ToProto()
if err := vss[2].SignProposal(ctx, cfg.ChainID(), p); err != nil {
t.Fatal("failed to sign bad proposal", err)
@ -436,8 +436,8 @@ func setupSimulator(ctx context.Context, t *testing.T) *simulatorTestSuite {
}
ensureNewProposal(t, proposalCh, height, round)
rs = css[0].GetRoundState()
signAddVotes(ctx, t, sim.Config, css[0], tmproto.PrecommitType,
rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header(),
signAddVotes(ctx, t, css[0], tmproto.PrecommitType, sim.Config.ChainID(),
types.BlockID{Hash: rs.ProposalBlock.Hash(), PartSetHeader: rs.ProposalBlockParts.Header()},
vss[1:nVals]...)
ensureNewRound(t, newRoundCh, height+1, 0)
@ -486,7 +486,7 @@ func setupSimulator(ctx context.Context, t *testing.T) *simulatorTestSuite {
selfIndex := valIndexFn(0)
require.NotEqual(t, -1, selfIndex)
proposal = types.NewProposal(vss[3].Height, round, -1, blockID)
proposal = types.NewProposal(vss[3].Height, round, -1, blockID, propBlock.Header.Time)
p = proposal.ToProto()
if err := vss[3].SignProposal(ctx, cfg.ChainID(), p); err != nil {
t.Fatal("failed to sign bad proposal", err)
@ -508,9 +508,10 @@ func setupSimulator(ctx context.Context, t *testing.T) *simulatorTestSuite {
if i == selfIndex {
continue
}
signAddVotes(ctx, t, sim.Config, css[0],
tmproto.PrecommitType, rs.ProposalBlock.Hash(),
rs.ProposalBlockParts.Header(), newVss[i])
signAddVotes(ctx, t, css[0],
tmproto.PrecommitType, sim.Config.ChainID(),
types.BlockID{Hash: rs.ProposalBlock.Hash(), PartSetHeader: rs.ProposalBlockParts.Header()},
newVss[i])
}
ensureNewRound(t, newRoundCh, height+1, 0)
@ -532,9 +533,10 @@ func setupSimulator(ctx context.Context, t *testing.T) *simulatorTestSuite {
if i == selfIndex {
continue
}
signAddVotes(ctx, t, sim.Config, css[0],
tmproto.PrecommitType, rs.ProposalBlock.Hash(),
rs.ProposalBlockParts.Header(), newVss[i])
signAddVotes(ctx, t, css[0],
tmproto.PrecommitType, sim.Config.ChainID(),
types.BlockID{Hash: rs.ProposalBlock.Hash(), PartSetHeader: rs.ProposalBlockParts.Header()},
newVss[i])
}
ensureNewRound(t, newRoundCh, height+1, 0)
@ -555,8 +557,7 @@ func setupSimulator(ctx context.Context, t *testing.T) *simulatorTestSuite {
selfIndex = valIndexFn(0)
require.NotEqual(t, -1, selfIndex)
proposal = types.NewProposal(vss[1].Height, round, -1, blockID)
proposal = types.NewProposal(vss[1].Height, round, -1, blockID, propBlock.Header.Time)
p = proposal.ToProto()
if err := vss[1].SignProposal(ctx, cfg.ChainID(), p); err != nil {
t.Fatal("failed to sign bad proposal", err)
@ -573,9 +574,10 @@ func setupSimulator(ctx context.Context, t *testing.T) *simulatorTestSuite {
if i == selfIndex {
continue
}
signAddVotes(ctx, t, sim.Config, css[0],
tmproto.PrecommitType, rs.ProposalBlock.Hash(),
rs.ProposalBlockParts.Header(), newVss[i])
signAddVotes(ctx, t, css[0],
tmproto.PrecommitType, sim.Config.ChainID(),
types.BlockID{Hash: rs.ProposalBlock.Hash(), PartSetHeader: rs.ProposalBlockParts.Header()},
newVss[i])
}
ensureNewRound(t, newRoundCh, height+1, 0)
@ -823,14 +825,15 @@ func testHandshakeReplay(
}
// now start the app using the handshake - it should sync
genDoc, _ := sm.MakeGenesisDocFromFile(cfg.GenesisFile())
genDoc, err := sm.MakeGenesisDocFromFile(cfg.GenesisFile())
require.NoError(t, err)
handshaker := NewHandshaker(logger, stateStore, state, store, eventbus.NopEventBus{}, genDoc)
proxyApp := proxy.NewAppConns(clientCreator2, logger, proxy.NopMetrics())
require.NoError(t, proxyApp.Start(ctx), "Error starting proxy app connections")
t.Cleanup(func() { cancel(); proxyApp.Wait() })
err := handshaker.Handshake(ctx, proxyApp)
err = handshaker.Handshake(ctx, proxyApp)
if expectError {
require.Error(t, err)
return
@ -1011,7 +1014,8 @@ func TestHandshakePanicsIfAppReturnsWrongAppHash(t *testing.T) {
require.NoError(t, err)
stateDB, state, store := stateAndStore(t, cfg, pubKey, appVersion)
stateStore := sm.NewStore(stateDB)
genDoc, _ := sm.MakeGenesisDocFromFile(cfg.GenesisFile())
genDoc, err := sm.MakeGenesisDocFromFile(cfg.GenesisFile())
require.NoError(t, err)
state.LastValidators = state.Validators.Copy()
// mode = 0 for committing all the blocks
blocks := sf.MakeBlocks(ctx, t, 3, &state, privVal)
@ -1277,7 +1281,8 @@ func TestHandshakeUpdatesValidators(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
val, _, err := factory.RandValidator(ctx, true, 10)
votePower := 10 + int64(rand.Uint32())
val, _, err := factory.Validator(ctx, votePower)
require.NoError(t, err)
vals := types.NewValidatorSet([]*types.Validator{val})
app := &initChainApp{vals: types.TM2PB.ValidatorUpdates(vals)}


+ 197
- 131
internal/consensus/state.go View File

@ -47,15 +47,17 @@ var msgQueueSize = 1000
// msgs from the reactor which may update the state
type msgInfo struct {
Msg Message
PeerID types.NodeID
Msg Message
PeerID types.NodeID
ReceiveTime time.Time
}
func (msgInfo) TypeTag() string { return "tendermint/wal/MsgInfo" }
type msgInfoJSON struct {
Msg json.RawMessage `json:"msg"`
PeerID types.NodeID `json:"peer_key"`
Msg json.RawMessage `json:"msg"`
PeerID types.NodeID `json:"peer_key"`
ReceiveTime time.Time `json:"receive_time"`
}
func (m msgInfo) MarshalJSON() ([]byte, error) {
@ -63,7 +65,7 @@ func (m msgInfo) MarshalJSON() ([]byte, error) {
if err != nil {
return nil, err
}
return json.Marshal(msgInfoJSON{Msg: msg, PeerID: m.PeerID})
return json.Marshal(msgInfoJSON{Msg: msg, PeerID: m.PeerID, ReceiveTime: m.ReceiveTime})
}
func (m *msgInfo) UnmarshalJSON(data []byte) error {
@ -163,7 +165,7 @@ type State struct {
// some functions can be overwritten for testing
decideProposal func(ctx context.Context, height int64, round int32)
doPrevote func(ctx context.Context, height int64, round int32)
setProposal func(proposal *types.Proposal) error
setProposal func(proposal *types.Proposal, t time.Time) error
// closed when we finish shutting down
done chan struct{}
@ -546,14 +548,14 @@ func (cs *State) AddVote(ctx context.Context, vote *types.Vote, peerID types.Nod
select {
case <-ctx.Done():
return ctx.Err()
case cs.internalMsgQueue <- msgInfo{&VoteMessage{vote}, ""}:
case cs.internalMsgQueue <- msgInfo{&VoteMessage{vote}, "", tmtime.Now()}:
return nil
}
} else {
select {
case <-ctx.Done():
return ctx.Err()
case cs.peerMsgQueue <- msgInfo{&VoteMessage{vote}, peerID}:
case cs.peerMsgQueue <- msgInfo{&VoteMessage{vote}, peerID, tmtime.Now()}:
return nil
}
}
@ -568,14 +570,14 @@ func (cs *State) SetProposal(ctx context.Context, proposal *types.Proposal, peer
select {
case <-ctx.Done():
return ctx.Err()
case cs.internalMsgQueue <- msgInfo{&ProposalMessage{proposal}, ""}:
case cs.internalMsgQueue <- msgInfo{&ProposalMessage{proposal}, "", tmtime.Now()}:
return nil
}
} else {
select {
case <-ctx.Done():
return ctx.Err()
case cs.peerMsgQueue <- msgInfo{&ProposalMessage{proposal}, peerID}:
case cs.peerMsgQueue <- msgInfo{&ProposalMessage{proposal}, peerID, tmtime.Now()}:
return nil
}
}
@ -589,14 +591,14 @@ func (cs *State) AddProposalBlockPart(ctx context.Context, height int64, round i
select {
case <-ctx.Done():
return ctx.Err()
case cs.internalMsgQueue <- msgInfo{&BlockPartMessage{height, round, part}, ""}:
case cs.internalMsgQueue <- msgInfo{&BlockPartMessage{height, round, part}, "", tmtime.Now()}:
return nil
}
} else {
select {
case <-ctx.Done():
return ctx.Err()
case cs.peerMsgQueue <- msgInfo{&BlockPartMessage{height, round, part}, peerID}:
case cs.peerMsgQueue <- msgInfo{&BlockPartMessage{height, round, part}, peerID, tmtime.Now()}:
return nil
}
}
@ -785,6 +787,7 @@ func (cs *State) updateToState(ctx context.Context, state sm.State) {
cs.Validators = validators
cs.Proposal = nil
cs.ProposalReceiveTime = time.Time{}
cs.ProposalBlock = nil
cs.ProposalBlockParts = nil
cs.LockedRound = -1
@ -933,7 +936,7 @@ func (cs *State) handleMsg(ctx context.Context, mi msgInfo) {
case *ProposalMessage:
// will not cause transition.
// once proposal is set, we can receive block parts
err = cs.setProposal(msg.Proposal)
err = cs.setProposal(msg.Proposal, mi.ReceiveTime)
case *BlockPartMessage:
// if the proposal is complete, we'll enterPrevote or tryFinalizeCommit
@ -1125,6 +1128,7 @@ func (cs *State) enterNewRound(ctx context.Context, height int64, round int32) {
} else {
logger.Debug("resetting proposal info")
cs.Proposal = nil
cs.ProposalReceiveTime = time.Time{}
cs.ProposalBlock = nil
cs.ProposalBlockParts = nil
}
@ -1147,9 +1151,10 @@ func (cs *State) enterNewRound(ctx context.Context, height int64, round int32) {
cs.scheduleTimeout(cs.config.CreateEmptyBlocksInterval, height, round,
cstypes.RoundStepNewRound)
}
} else {
cs.enterPropose(ctx, height, round)
return
}
cs.enterPropose(ctx, height, round)
}
// needProofBlock returns true on the first height (so the genesis app hash is signed right away)
@ -1182,6 +1187,16 @@ func (cs *State) enterPropose(ctx context.Context, height int64, round int32) {
return
}
// If this validator is the proposer of this round, and the previous block time is later than
// our local clock time, wait to propose until our local clock time has passed the block time.
if cs.privValidatorPubKey != nil && cs.isProposer(cs.privValidatorPubKey.Address()) {
proposerWaitTime := proposerWaitTime(tmtime.DefaultSource{}, cs.state.LastBlockTime)
if proposerWaitTime > 0 {
cs.scheduleTimeout(proposerWaitTime, height, round, cstypes.RoundStepNewRound)
return
}
}
logger.Debug("entering propose step", "current", fmt.Sprintf("%v/%v/%v", cs.Height, cs.Round, cs.Step))
defer func() {
@ -1202,12 +1217,10 @@ func (cs *State) enterPropose(ctx context.Context, height int64, round int32) {
// Nothing more to do if we're not a validator
if cs.privValidator == nil {
logger.Debug("node is not a validator")
logger.Debug("propose step; not proposing since node is not a validator")
return
}
logger.Debug("node is a validator")
if cs.privValidatorPubKey == nil {
// If this node is a validator & proposer in the current round, it will
// miss the opportunity to create a block.
@ -1215,18 +1228,20 @@ func (cs *State) enterPropose(ctx context.Context, height int64, round int32) {
return
}
address := cs.privValidatorPubKey.Address()
addr := cs.privValidatorPubKey.Address()
// if not a validator, we're done
if !cs.Validators.HasAddress(address) {
logger.Debug("node is not a validator", "addr", address, "vals", cs.Validators)
if !cs.Validators.HasAddress(addr) {
logger.Debug("propose step; not proposing since node is not in the validator set",
"addr", addr,
"vals", cs.Validators)
return
}
if cs.isProposer(address) {
if cs.isProposer(addr) {
logger.Debug(
"propose step; our turn to propose",
"proposer", address,
"proposer", addr,
)
cs.decideProposal(ctx, height, round)
@ -1267,7 +1282,7 @@ func (cs *State) defaultDecideProposal(ctx context.Context, height int64, round
// Make proposal
propBlockID := types.BlockID{Hash: block.Hash(), PartSetHeader: blockParts.Header()}
proposal := types.NewProposal(height, round, cs.ValidRound, propBlockID)
proposal := types.NewProposal(height, round, cs.ValidRound, propBlockID, block.Header.Time)
p := proposal.ToProto()
// wait the max amount we would wait for a proposal
@ -1277,11 +1292,11 @@ func (cs *State) defaultDecideProposal(ctx context.Context, height int64, round
proposal.Signature = p.Signature
// send proposal and block parts on internal msg queue
cs.sendInternalMessage(ctx, msgInfo{&ProposalMessage{proposal}, ""})
cs.sendInternalMessage(ctx, msgInfo{&ProposalMessage{proposal}, "", tmtime.Now()})
for i := 0; i < int(blockParts.Total()); i++ {
part := blockParts.GetPart(i)
cs.sendInternalMessage(ctx, msgInfo{&BlockPartMessage{cs.Height, cs.Round, part}, ""})
cs.sendInternalMessage(ctx, msgInfo{&BlockPartMessage{cs.Height, cs.Round, part}, "", tmtime.Now()})
}
cs.logger.Debug("signed proposal", "height", height, "round", round, "proposal", proposal)
@ -1348,8 +1363,11 @@ func (cs *State) createProposalBlock() (block *types.Block, blockParts *types.Pa
// Enter: `timeoutPropose` after entering Propose.
// Enter: proposal block and POL is ready.
// Prevote for LockedBlock if we're locked, or ProposalBlock if valid.
// Otherwise vote nil.
// If we received a valid proposal within this round and we are not locked on a block,
// we will prevote for block.
// Otherwise, if we receive a valid proposal that matches the block we are
// locked on or matches a block that received a POL in a round later than our
// locked round, prevote for the proposal, otherwise vote nil.
func (cs *State) enterPrevote(ctx context.Context, height int64, round int32) {
logger := cs.logger.With("height", height, "round", round)
@ -1376,19 +1394,47 @@ func (cs *State) enterPrevote(ctx context.Context, height int64, round int32) {
// (so we have more time to try and collect +2/3 prevotes for a single block)
}
func (cs *State) proposalIsTimely() bool {
sp := types.SynchronyParams{
Precision: cs.state.ConsensusParams.Synchrony.Precision,
MessageDelay: cs.state.ConsensusParams.Synchrony.MessageDelay,
}
return cs.Proposal.IsTimely(cs.ProposalReceiveTime, sp, cs.state.InitialHeight)
}
func (cs *State) defaultDoPrevote(ctx context.Context, height int64, round int32) {
logger := cs.logger.With("height", height, "round", round)
// If a block is locked, prevote that.
if cs.LockedBlock != nil {
logger.Debug("prevote step; already locked on a block; prevoting locked block")
cs.signAddVote(ctx, tmproto.PrevoteType, cs.LockedBlock.Hash(), cs.LockedBlockParts.Header())
// Check that a proposed block was not received within this round (and thus executing this from a timeout).
if cs.ProposalBlock == nil {
logger.Debug("prevote step: ProposalBlock is nil; prevoting nil")
cs.signAddVote(ctx, tmproto.PrevoteType, nil, types.PartSetHeader{})
return
}
// If ProposalBlock is nil, prevote nil.
if cs.ProposalBlock == nil {
logger.Debug("prevote step: ProposalBlock is nil")
if cs.Proposal == nil {
logger.Debug("prevote step: did not receive proposal; prevoting nil")
cs.signAddVote(ctx, tmproto.PrevoteType, nil, types.PartSetHeader{})
return
}
if !cs.Proposal.Timestamp.Equal(cs.ProposalBlock.Header.Time) {
logger.Debug("prevote step: proposal timestamp not equal; prevoting nil")
cs.signAddVote(ctx, tmproto.PrevoteType, nil, types.PartSetHeader{})
return
}
if cs.Proposal.POLRound == -1 && cs.LockedRound == -1 && !cs.proposalIsTimely() {
logger.Debug("prevote step: Proposal is not timely; prevoting nil",
"proposed",
tmtime.Canonical(cs.Proposal.Timestamp).Format(time.RFC3339Nano),
"received",
tmtime.Canonical(cs.ProposalReceiveTime).Format(time.RFC3339Nano),
"msg_delay",
cs.state.ConsensusParams.Synchrony.MessageDelay,
"precision",
cs.state.ConsensusParams.Synchrony.Precision)
cs.signAddVote(ctx, tmproto.PrevoteType, nil, types.PartSetHeader{})
return
}
@ -1397,16 +1443,72 @@ func (cs *State) defaultDoPrevote(ctx context.Context, height int64, round int32
err := cs.blockExec.ValidateBlock(cs.state, cs.ProposalBlock)
if err != nil {
// ProposalBlock is invalid, prevote nil.
logger.Error("prevote step: ProposalBlock is invalid", "err", err)
logger.Error("prevote step: ProposalBlock is invalid; prevoting nil", "err", err)
cs.signAddVote(ctx, tmproto.PrevoteType, nil, types.PartSetHeader{})
return
}
// Prevote cs.ProposalBlock
// NOTE: the proposal signature is validated when it is received,
// and the proposal block parts are validated as they are received (against the merkle hash in the proposal)
logger.Debug("prevote step: ProposalBlock is valid")
cs.signAddVote(ctx, tmproto.PrevoteType, cs.ProposalBlock.Hash(), cs.ProposalBlockParts.Header())
/*
22: upon <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(ctx, tmproto.PrevoteType, cs.ProposalBlock.Hash(), cs.ProposalBlockParts.Header())
return
}
if cs.ProposalBlock.HashesTo(cs.LockedBlock.Hash()) {
logger.Debug("prevote step: ProposalBlock is valid and matches our locked block; prevoting the proposal")
cs.signAddVote(ctx, tmproto.PrevoteType, cs.ProposalBlock.Hash(), cs.ProposalBlockParts.Header())
return
}
}
/*
28: upon <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(ctx, tmproto.PrevoteType, cs.ProposalBlock.Hash(), cs.ProposalBlockParts.Header())
return
}
if cs.ProposalBlock.HashesTo(cs.LockedBlock.Hash()) {
logger.Debug("prevote step: ProposalBlock is valid and matches our locked block; prevoting the proposal")
cs.signAddVote(ctx, tmproto.PrevoteType, cs.ProposalBlock.Hash(), cs.ProposalBlockParts.Header())
return
}
}
logger.Debug("prevote step: ProposalBlock is valid but was not our locked block or" +
"did not receive a more recent majority; prevoting nil")
cs.signAddVote(ctx, tmproto.PrevoteType, nil, types.PartSetHeader{})
}
// Enter: any +2/3 prevotes at next round.
@ -1444,7 +1546,6 @@ func (cs *State) enterPrevoteWait(ctx context.Context, height int64, round int32
// Enter: `timeoutPrecommit` after any +2/3 precommits.
// Enter: +2/3 precomits for block or nil.
// Lock & precommit the ProposalBlock if we have enough prevotes for it (a POL in this round)
// else, unlock an existing lock and precommit nil if +2/3 of prevotes were nil,
// else, precommit nil otherwise.
func (cs *State) enterPrecommit(ctx context.Context, height int64, round int32) {
logger := cs.logger.With("height", height, "round", round)
@ -1491,47 +1592,50 @@ func (cs *State) enterPrecommit(ctx context.Context, height int64, round int32)
panic(fmt.Sprintf("this POLRound should be %v but got %v", round, polRound))
}
// +2/3 prevoted nil. Unlock and precommit nil.
if len(blockID.Hash) == 0 {
if cs.LockedBlock == nil {
logger.Debug("precommit step; +2/3 prevoted for nil")
} else {
logger.Debug("precommit step; +2/3 prevoted for nil; unlocking")
cs.LockedRound = -1
cs.LockedBlock = nil
cs.LockedBlockParts = nil
if err := cs.eventBus.PublishEventUnlock(ctx, cs.RoundStateEvent()); err != nil {
logger.Error("failed publishing event unlock", "err", err)
}
}
// +2/3 prevoted nil. Precommit nil.
if blockID.IsNil() {
logger.Debug("precommit step: +2/3 prevoted for nil; precommitting nil")
cs.signAddVote(ctx, tmproto.PrecommitType, nil, types.PartSetHeader{})
return
}
// At this point, +2/3 prevoted for a particular block.
// If we never received a proposal for this block, we must precommit nil
if cs.Proposal == nil || cs.ProposalBlock == nil {
logger.Debug("precommit step; did not receive proposal, precommitting nil")
cs.signAddVote(ctx, tmproto.PrecommitType, nil, types.PartSetHeader{})
return
}
// At this point, +2/3 prevoted for a particular block.
// If the proposal time does not match the block time, precommit nil.
if !cs.Proposal.Timestamp.Equal(cs.ProposalBlock.Header.Time) {
logger.Debug("precommit step: proposal timestamp not equal; precommitting nil")
cs.signAddVote(ctx, tmproto.PrecommitType, nil, types.PartSetHeader{})
return
}
// If we're already locked on that block, precommit it, and update the LockedRound
if cs.LockedBlock.HashesTo(blockID.Hash) {
logger.Debug("precommit step; +2/3 prevoted locked block; relocking")
logger.Debug("precommit step: +2/3 prevoted locked block; relocking")
cs.LockedRound = round
if err := cs.eventBus.PublishEventRelock(ctx, cs.RoundStateEvent()); err != nil {
logger.Error("failed publishing event relock", "err", err)
logger.Error("precommit step: failed publishing event relock", "err", err)
}
cs.signAddVote(ctx, tmproto.PrecommitType, blockID.Hash, blockID.PartSetHeader)
return
}
// If +2/3 prevoted for proposal block, stage and precommit it
// If greater than 2/3 of the voting power on the network prevoted for
// the proposed block, update our locked block to this block and issue a
// precommit vote for it.
if cs.ProposalBlock.HashesTo(blockID.Hash) {
logger.Debug("precommit step; +2/3 prevoted proposal block; locking", "hash", blockID.Hash)
logger.Debug("precommit step: +2/3 prevoted proposal block; locking", "hash", blockID.Hash)
// Validate the block.
if err := cs.blockExec.ValidateBlock(cs.state, cs.ProposalBlock); err != nil {
panic(fmt.Errorf("precommit step; +2/3 prevoted for an invalid block: %w", err))
panic(fmt.Sprintf("precommit step: +2/3 prevoted for an invalid block %v; relocking", err))
}
cs.LockedRound = round
@ -1539,7 +1643,7 @@ func (cs *State) enterPrecommit(ctx context.Context, height int64, round int32)
cs.LockedBlockParts = cs.ProposalBlockParts
if err := cs.eventBus.PublishEventLock(ctx, cs.RoundStateEvent()); err != nil {
logger.Error("failed publishing event lock", "err", err)
logger.Error("precommit step: failed publishing event lock", "err", err)
}
cs.signAddVote(ctx, tmproto.PrecommitType, blockID.Hash, blockID.PartSetHeader)
@ -1547,23 +1651,14 @@ func (cs *State) enterPrecommit(ctx context.Context, height int64, round int32)
}
// There was a polka in this round for a block we don't have.
// Fetch that block, unlock, and precommit nil.
// The +2/3 prevotes for this round is the POL for our unlock.
logger.Debug("precommit step; +2/3 prevotes for a block we do not have; voting nil", "block_id", blockID)
cs.LockedRound = -1
cs.LockedBlock = nil
cs.LockedBlockParts = nil
// Fetch that block, and precommit nil.
logger.Debug("precommit step: +2/3 prevotes for a block we do not have; voting nil", "block_id", blockID)
if !cs.ProposalBlockParts.HasHeader(blockID.PartSetHeader) {
cs.ProposalBlock = nil
cs.ProposalBlockParts = types.NewPartSetFromHeader(blockID.PartSetHeader)
}
if err := cs.eventBus.PublishEventUnlock(ctx, cs.RoundStateEvent()); err != nil {
logger.Error("failed publishing event unlock", "err", err)
}
cs.signAddVote(ctx, tmproto.PrecommitType, nil, types.PartSetHeader{})
}
@ -1671,7 +1766,7 @@ func (cs *State) tryFinalizeCommit(ctx context.Context, height int64) {
}
blockID, ok := cs.Votes.Precommits(cs.CommitRound).TwoThirdsMajority()
if !ok || len(blockID.Hash) == 0 {
if !ok || blockID.IsNil() {
logger.Error("failed attempt to finalize commit; there was no +2/3 majority or +2/3 was for nil")
return
}
@ -1893,10 +1988,10 @@ func (cs *State) RecordMetrics(height int64, block *types.Block) {
//-----------------------------------------------------------------------------
func (cs *State) defaultSetProposal(proposal *types.Proposal) error {
func (cs *State) defaultSetProposal(proposal *types.Proposal, recvTime time.Time) error {
// Already have one
// TODO: possibly catch double proposals
if cs.Proposal != nil {
if cs.Proposal != nil || proposal == nil {
return nil
}
@ -1921,6 +2016,7 @@ func (cs *State) defaultSetProposal(proposal *types.Proposal) error {
proposal.Signature = p.Signature
cs.Proposal = proposal
cs.ProposalReceiveTime = recvTime
// We don't update cs.ProposalBlockParts if it is already set.
// This happens if we're already in cstypes.RoundStepCommit or if there is a valid block in the current round.
// TODO: We can check if Proposal is for a different block as this is a sign of misbehavior!
@ -2000,7 +2096,7 @@ func (cs *State) addProposalBlockPart(
// Update Valid* if we can.
prevotes := cs.Votes.Prevotes(cs.Round)
blockID, hasTwoThirds := prevotes.TwoThirdsMajority()
if hasTwoThirds && !blockID.IsZero() && (cs.ValidRound < cs.Round) {
if hasTwoThirds && !blockID.IsNil() && (cs.ValidRound < cs.Round) {
if cs.ProposalBlock.HashesTo(blockID.Hash) {
cs.logger.Debug(
"updating valid block to new proposal block",
@ -2153,33 +2249,13 @@ func (cs *State) addVote(
prevotes := cs.Votes.Prevotes(vote.Round)
cs.logger.Debug("added vote to prevote", "vote", vote, "prevotes", prevotes.StringShort())
// If +2/3 prevotes for a block or nil for *any* round:
if blockID, ok := prevotes.TwoThirdsMajority(); ok {
// There was a polka!
// If we're locked but this is a recent polka, unlock.
// If it matches our ProposalBlock, update the ValidBlock
// Unlock if `cs.LockedRound < vote.Round <= cs.Round`
// NOTE: If vote.Round > cs.Round, we'll deal with it when we get to vote.Round
if (cs.LockedBlock != nil) &&
(cs.LockedRound < vote.Round) &&
(vote.Round <= cs.Round) &&
!cs.LockedBlock.HashesTo(blockID.Hash) {
cs.logger.Debug("unlocking because of POL", "locked_round", cs.LockedRound, "pol_round", vote.Round)
cs.LockedRound = -1
cs.LockedBlock = nil
cs.LockedBlockParts = nil
if err := cs.eventBus.PublishEventUnlock(ctx, cs.RoundStateEvent()); err != nil {
return added, err
}
}
// Check to see if >2/3 of the voting power on the network voted for any non-nil block.
if blockID, ok := prevotes.TwoThirdsMajority(); ok && !blockID.IsNil() {
// Greater than 2/3 of the voting power on the network voted for some
// non-nil block
// Update Valid* if we can.
// NOTE: our proposal block may be nil or not what received a polka..
if len(blockID.Hash) != 0 && (cs.ValidRound < vote.Round) && (vote.Round == cs.Round) {
if cs.ValidRound < vote.Round && vote.Round == cs.Round {
if cs.ProposalBlock.HashesTo(blockID.Hash) {
cs.logger.Debug("updating valid block because of POL", "valid_round", cs.ValidRound, "pol_round", vote.Round)
cs.ValidRound = vote.Round
@ -2215,7 +2291,7 @@ func (cs *State) addVote(
case cs.Round == vote.Round && cstypes.RoundStepPrevote <= cs.Step: // current round
blockID, ok := prevotes.TwoThirdsMajority()
if ok && (cs.isProposalComplete() || len(blockID.Hash) == 0) {
if ok && (cs.isProposalComplete() || blockID.IsNil()) {
cs.enterPrecommit(ctx, height, vote.Round)
} else if prevotes.HasTwoThirdsAny() {
cs.enterPrevoteWait(ctx, height, vote.Round)
@ -2243,7 +2319,7 @@ func (cs *State) addVote(
cs.enterNewRound(ctx, height, vote.Round)
cs.enterPrecommit(ctx, height, vote.Round)
if len(blockID.Hash) != 0 {
if !blockID.IsNil() {
cs.enterCommit(ctx, height, vote.Round)
if cs.config.SkipTimeoutCommit && precommits.HasAll() {
cs.enterNewRound(ctx, cs.Height, 0)
@ -2288,7 +2364,7 @@ func (cs *State) signVote(
ValidatorIndex: valIdx,
Height: cs.Height,
Round: cs.Round,
Timestamp: cs.voteTime(),
Timestamp: tmtime.Now(),
Type: msgType,
BlockID: types.BlockID{Hash: hash, PartSetHeader: header},
}
@ -2318,30 +2394,6 @@ func (cs *State) signVote(
return vote, err
}
// voteTime ensures monotonicity of the time a validator votes on.
// It ensures that for a prior block with a BFT-timestamp of T,
// any vote from this validator will have time at least time T + 1ms.
// This is needed, as monotonicity of time is a guarantee that BFT time provides.
func (cs *State) voteTime() time.Time {
now := tmtime.Now()
minVoteTime := now
// Minimum time increment between blocks
const timeIota = time.Millisecond
// TODO: We should remove next line in case we don't vote for v in case cs.ProposalBlock == nil,
// even if cs.LockedBlock != nil. See https://docs.tendermint.com/master/spec/.
if cs.LockedBlock != nil {
// See the BFT time spec https://docs.tendermint.com/master/spec/consensus/bft-time.html
minVoteTime = cs.LockedBlock.Time.Add(timeIota)
} else if cs.ProposalBlock != nil {
minVoteTime = cs.ProposalBlock.Time.Add(timeIota)
}
if now.After(minVoteTime) {
return now
}
return minVoteTime
}
// sign the vote and publish on internalMsgQueue
func (cs *State) signAddVote(ctx context.Context, msgType tmproto.SignedMsgType, hash []byte, header types.PartSetHeader) *types.Vote {
if cs.privValidator == nil { // the node does not have a key
@ -2362,7 +2414,7 @@ func (cs *State) signAddVote(ctx context.Context, msgType tmproto.SignedMsgType,
// TODO: pass pubKey to signVote
vote, err := cs.signVote(ctx, msgType, hash, header)
if err == nil {
cs.sendInternalMessage(ctx, msgInfo{&VoteMessage{vote}, ""})
cs.sendInternalMessage(ctx, msgInfo{&VoteMessage{vote}, "", tmtime.Now()})
cs.logger.Debug("signed and pushed vote", "height", cs.Height, "round", cs.Round, "vote", vote)
return vote
}
@ -2509,3 +2561,17 @@ func repairWalFile(src, dst string) error {
return nil
}
// proposerWaitTime determines how long the proposer should wait to propose its next block.
// If the result is zero, a block can be proposed immediately.
//
// Block times must be monotonically increasing, so if the block time of the previous
// block is larger than the proposer's current time, then the proposer will sleep
// until its local clock exceeds the previous block time.
func proposerWaitTime(lt tmtime.Source, bt time.Time) time.Duration {
t := lt.Now()
if bt.After(t) {
return bt.Sub(t)
}
return 0
}

+ 1089
- 546
internal/consensus/state_test.go
File diff suppressed because it is too large
View File


+ 1
- 1
internal/consensus/types/height_vote_set_test.go View File

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


+ 9
- 8
internal/consensus/types/round_state.go View File

@ -71,14 +71,15 @@ type RoundState struct {
StartTime time.Time `json:"start_time"`
// Subjective time when +2/3 precommits for Block at Round were found
CommitTime time.Time `json:"commit_time"`
Validators *types.ValidatorSet `json:"validators"`
Proposal *types.Proposal `json:"proposal"`
ProposalBlock *types.Block `json:"proposal_block"`
ProposalBlockParts *types.PartSet `json:"proposal_block_parts"`
LockedRound int32 `json:"locked_round"`
LockedBlock *types.Block `json:"locked_block"`
LockedBlockParts *types.PartSet `json:"locked_block_parts"`
CommitTime time.Time `json:"commit_time"`
Validators *types.ValidatorSet `json:"validators"`
Proposal *types.Proposal `json:"proposal"`
ProposalReceiveTime time.Time `json:"proposal_receive_time"`
ProposalBlock *types.Block `json:"proposal_block"`
ProposalBlockParts *types.PartSet `json:"proposal_block_parts"`
LockedRound int32 `json:"locked_round"`
LockedBlock *types.Block `json:"locked_block"`
LockedBlockParts *types.PartSet `json:"locked_block_parts"`
// Last known round with POL for non-nil valid block.
ValidRound int32 `json:"valid_round"`


+ 0
- 4
internal/eventbus/event_bus.go View File

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


+ 0
- 3
internal/eventbus/event_bus_test.go View File

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


+ 1
- 2
internal/evidence/pool_test.go View File

@ -41,8 +41,7 @@ func TestEvidencePoolBasic(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
valSet, privVals := factory.RandValidatorSet(ctx, t, 1, 10)
valSet, privVals := factory.ValidatorSet(ctx, t, 1, 10)
blockStore.On("LoadBlockMeta", mock.AnythingOfType("int64")).Return(
&types.BlockMeta{Header: types.Header{Time: defaultEvidenceTime}},
)


+ 6
- 8
internal/evidence/verify_test.go View File

@ -201,7 +201,7 @@ func TestVerifyLightClientAttack_Equivocation(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
conflictingVals, conflictingPrivVals := factory.RandValidatorSet(ctx, t, 5, 10)
conflictingVals, conflictingPrivVals := factory.ValidatorSet(ctx, t, 5, 10)
conflictingHeader := factory.MakeHeader(t, &types.Header{
ChainID: evidenceChainID,
@ -294,12 +294,10 @@ func TestVerifyLightClientAttack_Equivocation(t *testing.T) {
}
func TestVerifyLightClientAttack_Amnesia(t *testing.T) {
var height int64 = 10
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
conflictingVals, conflictingPrivVals := factory.RandValidatorSet(ctx, t, 5, 10)
var height int64 = 10
conflictingVals, conflictingPrivVals := factory.ValidatorSet(ctx, t, 5, 10)
conflictingHeader := factory.MakeHeader(t, &types.Header{
ChainID: evidenceChainID,
@ -496,14 +494,14 @@ func makeLunaticEvidence(
) (ev *types.LightClientAttackEvidence, trusted *types.LightBlock, common *types.LightBlock) {
t.Helper()
commonValSet, commonPrivVals := factory.RandValidatorSet(ctx, t, totalVals, defaultVotingPower)
commonValSet, commonPrivVals := factory.ValidatorSet(ctx, t, totalVals, defaultVotingPower)
require.Greater(t, totalVals, byzVals)
// extract out the subset of byzantine validators in the common validator set
byzValSet, byzPrivVals := commonValSet.Validators[:byzVals], commonPrivVals[:byzVals]
phantomValSet, phantomPrivVals := factory.RandValidatorSet(ctx, t, phantomVals, defaultVotingPower)
phantomValSet, phantomPrivVals := factory.ValidatorSet(ctx, t, phantomVals, defaultVotingPower)
conflictingVals := phantomValSet.Copy()
require.NoError(t, conflictingVals.UpdateWithChangeSet(byzValSet))
@ -558,7 +556,7 @@ func makeLunaticEvidence(
ValidatorSet: commonValSet,
}
trustedBlockID := factory.MakeBlockIDWithHash(trustedHeader.Hash())
trustedVals, privVals := factory.RandValidatorSet(ctx, t, totalVals, defaultVotingPower)
trustedVals, privVals := factory.ValidatorSet(ctx, t, totalVals, defaultVotingPower)
trustedVoteSet := types.NewVoteSet(evidenceChainID, height, 1, tmproto.SignedMsgType(2), trustedVals)
trustedCommit, err := factory.MakeCommit(ctx, trustedBlockID, height, 1, trustedVoteSet, privVals, defaultEvidenceTime)
require.NoError(t, err)


+ 2
- 2
internal/state/helpers_test.go View File

@ -4,6 +4,7 @@ import (
"bytes"
"context"
"fmt"
"math/rand"
"testing"
"time"
@ -258,7 +259,6 @@ func makeRandomStateFromValidatorSet(
InitialHeight: 1,
}
}
func makeRandomStateFromConsensusParams(
ctx context.Context,
t *testing.T,
@ -267,7 +267,7 @@ func makeRandomStateFromConsensusParams(
lastHeightConsensusParamsChanged int64,
) sm.State {
t.Helper()
val, _, err := factory.RandValidator(ctx, true, 10)
val, _, err := factory.Validator(ctx, 10+int64(rand.Uint32()))
require.NoError(t, err)
valSet := types.NewValidatorSet([]*types.Validator{val})
return sm.State{


+ 7
- 14
internal/state/rollback_test.go View File

@ -19,11 +19,9 @@ func TestRollback(t *testing.T) {
height int64 = 100
nextHeight int64 = 101
)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
blockStore := &mocks.BlockStore{}
stateStore := setupStateStore(ctx, t, height)
stateStore := setupStateStore(t, height)
initialState, err := stateStore.Load()
require.NoError(t, err)
@ -83,10 +81,7 @@ func TestRollbackNoState(t *testing.T) {
func TestRollbackNoBlocks(t *testing.T) {
const height = int64(100)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
stateStore := setupStateStore(ctx, t, height)
stateStore := setupStateStore(t, height)
blockStore := &mocks.BlockStore{}
blockStore.On("Height").Return(height)
blockStore.On("LoadBlockMeta", height-1).Return(nil)
@ -98,11 +93,7 @@ func TestRollbackNoBlocks(t *testing.T) {
func TestRollbackDifferentStateHeight(t *testing.T) {
const height = int64(100)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
stateStore := setupStateStore(ctx, t, height)
stateStore := setupStateStore(t, height)
blockStore := &mocks.BlockStore{}
blockStore.On("Height").Return(height + 2)
@ -111,9 +102,11 @@ func TestRollbackDifferentStateHeight(t *testing.T) {
require.Equal(t, err.Error(), "statestore height (100) is not one below or equal to blockstore height (102)")
}
func setupStateStore(ctx context.Context, t *testing.T, height int64) state.Store {
func setupStateStore(t *testing.T, height int64) state.Store {
stateStore := state.NewStore(dbm.NewMemDB())
valSet, _ := factory.RandValidatorSet(ctx, t, 5, 10)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
valSet, _ := factory.ValidatorSet(ctx, t, 5, 10)
params := types.DefaultConsensusParams()
params.Version.AppVersion = 10


+ 1
- 24
internal/state/state.go View File

@ -268,7 +268,7 @@ func (state State) MakeBlock(
if height == state.InitialHeight {
timestamp = state.LastBlockTime // genesis time
} else {
timestamp = MedianTime(commit, state.LastValidators)
timestamp = time.Now()
}
// Fill rest of header with state data.
@ -288,29 +288,6 @@ func (state State) MakeBlock(
return block, bps, nil
}
// MedianTime computes a median time for a given Commit (based on Timestamp field of votes messages) and the
// corresponding validator set. The computed time is always between timestamps of
// the votes sent by honest processes, i.e., a faulty processes can not arbitrarily increase or decrease the
// computed value.
func MedianTime(commit *types.Commit, validators *types.ValidatorSet) time.Time {
weightedTimes := make([]*weightedTime, len(commit.Signatures))
totalVotingPower := int64(0)
for i, commitSig := range commit.Signatures {
if commitSig.Absent() {
continue
}
_, validator := validators.GetByAddress(commitSig.ValidatorAddress)
// If there's no condition, TestValidateBlockCommit panics; not needed normally.
if validator != nil {
totalVotingPower += validator.VotingPower
weightedTimes[i] = newWeightedTime(commitSig.Timestamp, validator.VotingPower)
}
}
return weightedMedian(weightedTimes, totalVotingPower)
}
//------------------------------------------------------------------------
// Genesis


+ 11
- 12
internal/state/store_test.go View File

@ -3,6 +3,7 @@ package state_test
import (
"context"
"fmt"
"math/rand"
"os"
"testing"
@ -27,16 +28,15 @@ const (
)
func TestStoreBootstrap(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
stateDB := dbm.NewMemDB()
stateStore := sm.NewStore(stateDB)
val, _, err := factory.RandValidator(ctx, true, 10)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
val, _, err := factory.Validator(ctx, 10+int64(rand.Uint32()))
require.NoError(t, err)
val2, _, err := factory.RandValidator(ctx, true, 10)
val2, _, err := factory.Validator(ctx, 10+int64(rand.Uint32()))
require.NoError(t, err)
val3, _, err := factory.RandValidator(ctx, true, 10)
val3, _, err := factory.Validator(ctx, 10+int64(rand.Uint32()))
require.NoError(t, err)
vals := types.NewValidatorSet([]*types.Validator{val, val2, val3})
bootstrapState := makeRandomStateFromValidatorSet(vals, 100, 100)
@ -58,16 +58,15 @@ func TestStoreBootstrap(t *testing.T) {
}
func TestStoreLoadValidators(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
stateDB := dbm.NewMemDB()
stateStore := sm.NewStore(stateDB)
val, _, err := factory.RandValidator(ctx, true, 10)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
val, _, err := factory.Validator(ctx, 10+int64(rand.Uint32()))
require.NoError(t, err)
val2, _, err := factory.RandValidator(ctx, true, 10)
val2, _, err := factory.Validator(ctx, 10+int64(rand.Uint32()))
require.NoError(t, err)
val3, _, err := factory.RandValidator(ctx, true, 10)
val3, _, err := factory.Validator(ctx, 10+int64(rand.Uint32()))
require.NoError(t, err)
vals := types.NewValidatorSet([]*types.Validator{val, val2, val3})


+ 0
- 7
internal/state/validation.go View File

@ -114,13 +114,6 @@ func validateBlock(state State, block *types.Block) error {
state.LastBlockTime,
)
}
medianTime := MedianTime(block.LastCommit, state.LastValidators)
if !block.Time.Equal(medianTime) {
return fmt.Errorf("invalid block time. Expected %v, got %v",
medianTime,
block.Time,
)
}
case block.Height == state.InitialHeight:
genesisTime := state.LastBlockTime


+ 0
- 1
internal/state/validation_test.go View File

@ -64,7 +64,6 @@ func TestValidateBlockHeader(t *testing.T) {
{"ChainID wrong", func(block *types.Block) { block.ChainID = "not-the-real-one" }},
{"Height wrong", func(block *types.Block) { block.Height += 10 }},
{"Time wrong", func(block *types.Block) { block.Time = block.Time.Add(-time.Second * 1) }},
{"Time wrong 2", func(block *types.Block) { block.Time = block.Time.Add(time.Second * 1) }},
{"LastBlockID wrong", func(block *types.Block) { block.LastBlockID.PartSetHeader.Total += 10 }},
{"LastCommitHash wrong", func(block *types.Block) { block.LastCommitHash = wrongHash }},


+ 1
- 1
internal/statesync/block_queue_test.go View File

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


+ 4
- 4
internal/statesync/reactor_test.go View File

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


+ 1
- 1
internal/test/factory/block.go View File

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


+ 20
- 23
internal/test/factory/genesis.go View File

@ -1,36 +1,33 @@
package factory
import (
"context"
"sort"
"testing"
"time"
"github.com/stretchr/testify/require"
"github.com/tendermint/tendermint/config"
tmtime "github.com/tendermint/tendermint/libs/time"
cfg "github.com/tendermint/tendermint/config"
"github.com/tendermint/tendermint/types"
)
func RandGenesisDoc(ctx context.Context, t *testing.T, cfg *config.Config, numValidators int, randPower bool, minPower int64) (*types.GenesisDoc, []types.PrivValidator) {
t.Helper()
func GenesisDoc(
config *cfg.Config,
time time.Time,
validators []*types.Validator,
consensusParams *types.ConsensusParams,
) *types.GenesisDoc {
validators := make([]types.GenesisValidator, numValidators)
privValidators := make([]types.PrivValidator, numValidators)
for i := 0; i < numValidators; i++ {
val, privVal, err := RandValidator(ctx, randPower, minPower)
require.NoError(t, err)
validators[i] = types.GenesisValidator{
PubKey: val.PubKey,
Power: val.VotingPower,
genesisValidators := make([]types.GenesisValidator, len(validators))
for i := range validators {
genesisValidators[i] = types.GenesisValidator{
Power: validators[i].VotingPower,
PubKey: validators[i].PubKey,
}
privValidators[i] = privVal
}
sort.Sort(types.PrivValidatorsByAddress(privValidators))
return &types.GenesisDoc{
GenesisTime: tmtime.Now(),
InitialHeight: 1,
ChainID: cfg.ChainID(),
Validators: validators,
}, privValidators
GenesisTime: time,
InitialHeight: 1,
ChainID: config.ChainID(),
Validators: genesisValidators,
ConsensusParams: consensusParams,
}
}

+ 6
- 13
internal/test/factory/validator.go View File

@ -2,8 +2,6 @@ package factory
import (
"context"
"fmt"
"math/rand"
"sort"
"testing"
@ -11,23 +9,18 @@ import (
"github.com/tendermint/tendermint/types"
)
func RandValidator(ctx context.Context, randPower bool, minPower int64) (*types.Validator, types.PrivValidator, error) {
func Validator(ctx context.Context, votingPower int64) (*types.Validator, types.PrivValidator, error) {
privVal := types.NewMockPV()
votePower := minPower
if randPower {
// nolint:gosec // G404: Use of weak random number generator
votePower += int64(rand.Uint32())
}
pubKey, err := privVal.GetPubKey(ctx)
if err != nil {
return nil, nil, fmt.Errorf("could not retrieve public key: %w", err)
return nil, nil, err
}
val := types.NewValidator(pubKey, votePower)
return val, privVal, err
val := types.NewValidator(pubKey, votingPower)
return val, privVal, nil
}
func RandValidatorSet(ctx context.Context, t *testing.T, numValidators int, votingPower int64) (*types.ValidatorSet, []types.PrivValidator) {
func ValidatorSet(ctx context.Context, t *testing.T, numValidators int, votingPower int64) (*types.ValidatorSet, []types.PrivValidator) {
var (
valz = make([]*types.Validator, numValidators)
privValidators = make([]types.PrivValidator, numValidators)
@ -35,7 +28,7 @@ func RandValidatorSet(ctx context.Context, t *testing.T, numValidators int, voti
t.Helper()
for i := 0; i < numValidators; i++ {
val, privValidator, err := RandValidator(ctx, false, votingPower)
val, privValidator, err := Validator(ctx, votingPower)
require.NoError(t, err)
valz[i] = val
privValidators[i] = privValidator


+ 28
- 0
libs/time/mocks/source.go 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
}

+ 14
- 0
libs/time/time.go 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()
}

+ 4
- 5
light/client_test.go View File

@ -37,6 +37,8 @@ func init() {
}
func TestClient(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
var (
keys = genPrivKeys(4)
vals = keys.ToValidators(20, 10)
@ -120,12 +122,9 @@ func TestClient(t *testing.T) {
}
})
t.Run("SequentialVerification", func(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
newKeys := genPrivKeys(4)
newVals := newKeys.ToValidators(10, 1)
differentVals, _ := factory.RandValidatorSet(ctx, t, 10, 100)
differentVals, _ := factory.ValidatorSet(ctx, t, 10, 100)
testCases := []struct {
name string
@ -943,7 +942,7 @@ func TestClient(t *testing.T) {
logger := log.NewTestingLogger(t)
differentVals, _ := factory.RandValidatorSet(ctx, t, 10, 100)
differentVals, _ := factory.ValidatorSet(ctx, t, 10, 100)
mockBadValSetNode := mockNodeFromHeadersAndVals(
map[int64]*types.SignedHeader{
1: h1,


+ 4
- 8
light/store/db/db_test.go View File

@ -19,11 +19,10 @@ import (
)
func TestLast_FirstLightBlockHeight(t *testing.T) {
dbStore := New(dbm.NewMemDB())
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
dbStore := New(dbm.NewMemDB())
// Empty store
height, err := dbStore.LastLightBlockHeight()
require.NoError(t, err)
@ -48,7 +47,6 @@ func TestLast_FirstLightBlockHeight(t *testing.T) {
func Test_SaveLightBlock(t *testing.T) {
dbStore := New(dbm.NewMemDB())
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
@ -81,15 +79,14 @@ func Test_SaveLightBlock(t *testing.T) {
func Test_LightBlockBefore(t *testing.T) {
dbStore := New(dbm.NewMemDB())
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
assert.Panics(t, func() {
_, _ = dbStore.LightBlockBefore(0)
_, _ = dbStore.LightBlockBefore(100)
})
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
err := dbStore.SaveLightBlock(randLightBlock(ctx, t, int64(2)))
require.NoError(t, err)
@ -105,7 +102,6 @@ func Test_LightBlockBefore(t *testing.T) {
func Test_Prune(t *testing.T) {
dbStore := New(dbm.NewMemDB())
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
@ -200,7 +196,7 @@ func Test_Concurrency(t *testing.T) {
func randLightBlock(ctx context.Context, t *testing.T, height int64) *types.LightBlock {
t.Helper()
vals, _ := factory.RandValidatorSet(ctx, t, 2, 1)
vals, _ := factory.ValidatorSet(ctx, t, 2, 1)
return &types.LightBlock{
SignedHeader: &types.SignedHeader{
Header: &types.Header{


+ 2
- 1
node/node_test.go View File

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


+ 24
- 4
proto/tendermint/statesync/message_test.go View File

@ -7,9 +7,9 @@ import (
"github.com/gogo/protobuf/proto"
"github.com/stretchr/testify/require"
"github.com/tendermint/tendermint/crypto/ed25519"
ssproto "github.com/tendermint/tendermint/proto/tendermint/statesync"
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
"github.com/tendermint/tendermint/types"
)
func TestValidateMsg(t *testing.T) {
@ -186,10 +186,30 @@ func TestStateSyncVectors(t *testing.T) {
{
"ParamsResponse",
&ssproto.ParamsResponse{
Height: 9001,
ConsensusParams: types.DefaultConsensusParams().ToProto(),
Height: 9001,
ConsensusParams: tmproto.ConsensusParams{
Block: &tmproto.BlockParams{
MaxBytes: 10,
MaxGas: 20,
},
Evidence: &tmproto.EvidenceParams{
MaxAgeNumBlocks: 10,
MaxAgeDuration: 300,
MaxBytes: 100,
},
Validator: &tmproto.ValidatorParams{
PubKeyTypes: []string{ed25519.KeyType},
},
Version: &tmproto.VersionParams{
AppVersion: 11,
},
Synchrony: &tmproto.SynchronyParams{
MessageDelay: 550,
Precision: 90,
},
},
},
"423408a946122f0a10088080c00a10ffffffffffffffffff01120e08a08d0612040880c60a188080401a090a07656432353531392200",
"423008a946122b0a04080a10141209080a120310ac0218641a090a07656432353531392202080b2a090a0310a6041202105a",
},
}


+ 353
- 38
proto/tendermint/types/params.pb.go View File

@ -34,6 +34,7 @@ type ConsensusParams struct {
Evidence *EvidenceParams `protobuf:"bytes,2,opt,name=evidence,proto3" json:"evidence,omitempty"`
Validator *ValidatorParams `protobuf:"bytes,3,opt,name=validator,proto3" json:"validator,omitempty"`
Version *VersionParams `protobuf:"bytes,4,opt,name=version,proto3" json:"version,omitempty"`
Synchrony *SynchronyParams `protobuf:"bytes,5,opt,name=synchrony,proto3" json:"synchrony,omitempty"`
}
func (m *ConsensusParams) Reset() { *m = ConsensusParams{} }
@ -97,6 +98,13 @@ func (m *ConsensusParams) GetVersion() *VersionParams {
return nil
}
func (m *ConsensusParams) GetSynchrony() *SynchronyParams {
if m != nil {
return m.Synchrony
}
return nil
}
// BlockParams contains limits on the block size.
type BlockParams struct {
// Max block size, in bytes.
@ -373,6 +381,58 @@ func (m *HashedParams) GetBlockMaxGas() int64 {
return 0
}
type SynchronyParams struct {
MessageDelay time.Duration `protobuf:"bytes,1,opt,name=message_delay,json=messageDelay,proto3,stdduration" json:"message_delay"`
Precision time.Duration `protobuf:"bytes,2,opt,name=precision,proto3,stdduration" json:"precision"`
}
func (m *SynchronyParams) Reset() { *m = SynchronyParams{} }
func (m *SynchronyParams) String() string { return proto.CompactTextString(m) }
func (*SynchronyParams) ProtoMessage() {}
func (*SynchronyParams) Descriptor() ([]byte, []int) {
return fileDescriptor_e12598271a686f57, []int{6}
}
func (m *SynchronyParams) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
}
func (m *SynchronyParams) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
if deterministic {
return xxx_messageInfo_SynchronyParams.Marshal(b, m, deterministic)
} else {
b = b[:cap(b)]
n, err := m.MarshalToSizedBuffer(b)
if err != nil {
return nil, err
}
return b[:n], nil
}
}
func (m *SynchronyParams) XXX_Merge(src proto.Message) {
xxx_messageInfo_SynchronyParams.Merge(m, src)
}
func (m *SynchronyParams) XXX_Size() int {
return m.Size()
}
func (m *SynchronyParams) XXX_DiscardUnknown() {
xxx_messageInfo_SynchronyParams.DiscardUnknown(m)
}
var xxx_messageInfo_SynchronyParams proto.InternalMessageInfo
func (m *SynchronyParams) GetMessageDelay() time.Duration {
if m != nil {
return m.MessageDelay
}
return 0
}
func (m *SynchronyParams) GetPrecision() time.Duration {
if m != nil {
return m.Precision
}
return 0
}
func init() {
proto.RegisterType((*ConsensusParams)(nil), "tendermint.types.ConsensusParams")
proto.RegisterType((*BlockParams)(nil), "tendermint.types.BlockParams")
@ -380,44 +440,49 @@ func init() {
proto.RegisterType((*ValidatorParams)(nil), "tendermint.types.ValidatorParams")
proto.RegisterType((*VersionParams)(nil), "tendermint.types.VersionParams")
proto.RegisterType((*HashedParams)(nil), "tendermint.types.HashedParams")
proto.RegisterType((*SynchronyParams)(nil), "tendermint.types.SynchronyParams")
}
func init() { proto.RegisterFile("tendermint/types/params.proto", fileDescriptor_e12598271a686f57) }
var fileDescriptor_e12598271a686f57 = []byte{
// 498 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x6c, 0x93, 0xc1, 0x6a, 0xd4, 0x40,
0x1c, 0xc6, 0x77, 0x9a, 0xda, 0xee, 0xfe, 0xe3, 0x76, 0xcb, 0x20, 0x18, 0x2b, 0xcd, 0xae, 0x39,
0x48, 0x41, 0x48, 0xc4, 0x22, 0x22, 0x08, 0xe2, 0x56, 0xa9, 0x20, 0x15, 0x09, 0xea, 0xa1, 0x97,
0x30, 0xd9, 0x8c, 0x69, 0xe8, 0x4e, 0x66, 0xc8, 0x24, 0xcb, 0xee, 0xcd, 0x47, 0xf0, 0xe8, 0x23,
0xe8, 0x9b, 0xf4, 0xd8, 0xa3, 0x27, 0x95, 0xdd, 0x17, 0x91, 0x4c, 0x32, 0xa6, 0x9b, 0xf6, 0x36,
0x33, 0xdf, 0xef, 0x9b, 0xe1, 0xfb, 0x86, 0x3f, 0xec, 0xe7, 0x34, 0x8d, 0x68, 0xc6, 0x92, 0x34,
0xf7, 0xf2, 0x85, 0xa0, 0xd2, 0x13, 0x24, 0x23, 0x4c, 0xba, 0x22, 0xe3, 0x39, 0xc7, 0xbb, 0x8d,
0xec, 0x2a, 0x79, 0xef, 0x4e, 0xcc, 0x63, 0xae, 0x44, 0xaf, 0x5c, 0x55, 0xdc, 0x9e, 0x1d, 0x73,
0x1e, 0x4f, 0xa9, 0xa7, 0x76, 0x61, 0xf1, 0xc5, 0x8b, 0x8a, 0x8c, 0xe4, 0x09, 0x4f, 0x2b, 0xdd,
0xf9, 0xba, 0x01, 0x83, 0x23, 0x9e, 0x4a, 0x9a, 0xca, 0x42, 0x7e, 0x50, 0x2f, 0xe0, 0x43, 0xb8,
0x15, 0x4e, 0xf9, 0xe4, 0xdc, 0x42, 0x23, 0x74, 0x60, 0x3e, 0xd9, 0x77, 0xdb, 0x6f, 0xb9, 0xe3,
0x52, 0xae, 0x68, 0xbf, 0x62, 0xf1, 0x0b, 0xe8, 0xd2, 0x59, 0x12, 0xd1, 0x74, 0x42, 0xad, 0x0d,
0xe5, 0x1b, 0x5d, 0xf7, 0xbd, 0xa9, 0x89, 0xda, 0xfa, 0xdf, 0x81, 0x5f, 0x42, 0x6f, 0x46, 0xa6,
0x49, 0x44, 0x72, 0x9e, 0x59, 0x86, 0xb2, 0x3f, 0xb8, 0x6e, 0xff, 0xac, 0x91, 0xda, 0xdf, 0x78,
0xf0, 0x73, 0xd8, 0x9e, 0xd1, 0x4c, 0x26, 0x3c, 0xb5, 0x36, 0x95, 0x7d, 0x78, 0x83, 0xbd, 0x02,
0x6a, 0xb3, 0xe6, 0x9d, 0x23, 0x30, 0xaf, 0xe4, 0xc1, 0xf7, 0xa1, 0xc7, 0xc8, 0x3c, 0x08, 0x17,
0x39, 0x95, 0xaa, 0x01, 0xc3, 0xef, 0x32, 0x32, 0x1f, 0x97, 0x7b, 0x7c, 0x17, 0xb6, 0x4b, 0x31,
0x26, 0x52, 0x85, 0x34, 0xfc, 0x2d, 0x46, 0xe6, 0xc7, 0x44, 0x3a, 0x3f, 0x11, 0xec, 0xac, 0xa7,
0xc3, 0x8f, 0x00, 0x97, 0x2c, 0x89, 0x69, 0x90, 0x16, 0x2c, 0x50, 0x35, 0xe9, 0x1b, 0x07, 0x8c,
0xcc, 0x5f, 0xc5, 0xf4, 0x7d, 0xc1, 0xd4, 0xd3, 0x12, 0x9f, 0xc0, 0xae, 0x86, 0xf5, 0x0f, 0xd5,
0x35, 0xde, 0x73, 0xab, 0x2f, 0x74, 0xf5, 0x17, 0xba, 0xaf, 0x6b, 0x60, 0xdc, 0xbd, 0xf8, 0x3d,
0xec, 0x7c, 0xff, 0x33, 0x44, 0xfe, 0x4e, 0x75, 0x9f, 0x56, 0xd6, 0x43, 0x18, 0xeb, 0x21, 0x9c,
0xa7, 0x30, 0x68, 0x35, 0x89, 0x1d, 0xe8, 0x8b, 0x22, 0x0c, 0xce, 0xe9, 0x22, 0x50, 0x5d, 0x59,
0x68, 0x64, 0x1c, 0xf4, 0x7c, 0x53, 0x14, 0xe1, 0x3b, 0xba, 0xf8, 0x58, 0x1e, 0x39, 0x8f, 0xa1,
0xbf, 0xd6, 0x20, 0x1e, 0x82, 0x49, 0x84, 0x08, 0x74, 0xef, 0x65, 0xb2, 0x4d, 0x1f, 0x88, 0x10,
0x35, 0xe6, 0x9c, 0xc2, 0xed, 0xb7, 0x44, 0x9e, 0xd1, 0xa8, 0x36, 0x3c, 0x84, 0x81, 0x6a, 0x21,
0x68, 0x17, 0xdc, 0x57, 0xc7, 0x27, 0xba, 0x65, 0x07, 0xfa, 0x0d, 0xd7, 0x74, 0x6d, 0x6a, 0xea,
0x98, 0xc8, 0xf1, 0xa7, 0x1f, 0x4b, 0x1b, 0x5d, 0x2c, 0x6d, 0x74, 0xb9, 0xb4, 0xd1, 0xdf, 0xa5,
0x8d, 0xbe, 0xad, 0xec, 0xce, 0xe5, 0xca, 0xee, 0xfc, 0x5a, 0xd9, 0x9d, 0xd3, 0x67, 0x71, 0x92,
0x9f, 0x15, 0xa1, 0x3b, 0xe1, 0xcc, 0xbb, 0x3a, 0x48, 0xcd, 0xb2, 0x9a, 0x94, 0xf6, 0x90, 0x85,
0x5b, 0xea, 0xfc, 0xf0, 0x5f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x18, 0x54, 0x4f, 0xe1, 0x7f, 0x03,
0x00, 0x00,
// 561 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x94, 0x4d, 0x6b, 0xd4, 0x40,
0x18, 0xc7, 0x37, 0xdd, 0xbe, 0xec, 0x3e, 0xdb, 0xed, 0x96, 0x41, 0x30, 0x56, 0x9a, 0x5d, 0x73,
0x90, 0x82, 0x90, 0x88, 0x45, 0x44, 0x10, 0xa4, 0xdb, 0x8a, 0x05, 0xa9, 0x48, 0x7c, 0x39, 0xf4,
0x12, 0x26, 0xbb, 0x63, 0x36, 0x74, 0x93, 0x19, 0x32, 0xc9, 0xb2, 0xf9, 0x16, 0x1e, 0x3d, 0x79,
0xd6, 0x8f, 0xe1, 0xad, 0xc7, 0x1e, 0x3d, 0xa9, 0xec, 0x7e, 0x11, 0x99, 0xc9, 0x4c, 0xd3, 0xdd,
0x2a, 0xd8, 0x5b, 0x32, 0xcf, 0xef, 0x97, 0x87, 0xf9, 0x3f, 0x93, 0x81, 0xdd, 0x8c, 0x24, 0x43,
0x92, 0xc6, 0x51, 0x92, 0xb9, 0x59, 0xc1, 0x08, 0x77, 0x19, 0x4e, 0x71, 0xcc, 0x1d, 0x96, 0xd2,
0x8c, 0xa2, 0xed, 0xaa, 0xec, 0xc8, 0xf2, 0xce, 0xad, 0x90, 0x86, 0x54, 0x16, 0x5d, 0xf1, 0x54,
0x72, 0x3b, 0x56, 0x48, 0x69, 0x38, 0x26, 0xae, 0x7c, 0x0b, 0xf2, 0x8f, 0xee, 0x30, 0x4f, 0x71,
0x16, 0xd1, 0xa4, 0xac, 0xdb, 0xdf, 0x57, 0xa0, 0x73, 0x48, 0x13, 0x4e, 0x12, 0x9e, 0xf3, 0x37,
0xb2, 0x03, 0xda, 0x87, 0xb5, 0x60, 0x4c, 0x07, 0x67, 0xa6, 0xd1, 0x33, 0xf6, 0x5a, 0x8f, 0x76,
0x9d, 0xe5, 0x5e, 0x4e, 0x5f, 0x94, 0x4b, 0xda, 0x2b, 0x59, 0xf4, 0x0c, 0x1a, 0x64, 0x12, 0x0d,
0x49, 0x32, 0x20, 0xe6, 0x8a, 0xf4, 0x7a, 0xd7, 0xbd, 0x17, 0x8a, 0x50, 0xea, 0xa5, 0x81, 0x9e,
0x43, 0x73, 0x82, 0xc7, 0xd1, 0x10, 0x67, 0x34, 0x35, 0xeb, 0x52, 0xbf, 0x77, 0x5d, 0xff, 0xa0,
0x11, 0xe5, 0x57, 0x0e, 0x7a, 0x0a, 0x1b, 0x13, 0x92, 0xf2, 0x88, 0x26, 0xe6, 0xaa, 0xd4, 0xbb,
0x7f, 0xd1, 0x4b, 0x40, 0xc9, 0x9a, 0x17, 0xbd, 0x79, 0x91, 0x0c, 0x46, 0x29, 0x4d, 0x0a, 0x73,
0xed, 0x5f, 0xbd, 0xdf, 0x6a, 0x44, 0xf7, 0xbe, 0x74, 0xec, 0x43, 0x68, 0x5d, 0x09, 0x04, 0xdd,
0x85, 0x66, 0x8c, 0xa7, 0x7e, 0x50, 0x64, 0x84, 0xcb, 0x08, 0xeb, 0x5e, 0x23, 0xc6, 0xd3, 0xbe,
0x78, 0x47, 0xb7, 0x61, 0x43, 0x14, 0x43, 0xcc, 0x65, 0x4a, 0x75, 0x6f, 0x3d, 0xc6, 0xd3, 0x97,
0x98, 0xdb, 0xdf, 0x0c, 0xd8, 0x5a, 0x8c, 0x07, 0x3d, 0x00, 0x24, 0x58, 0x1c, 0x12, 0x3f, 0xc9,
0x63, 0x5f, 0xe6, 0xac, 0xbf, 0xd8, 0x89, 0xf1, 0xf4, 0x20, 0x24, 0xaf, 0xf3, 0x58, 0xb6, 0xe6,
0xe8, 0x04, 0xb6, 0x35, 0xac, 0x47, 0xac, 0xe6, 0x70, 0xc7, 0x29, 0xcf, 0x80, 0xa3, 0xcf, 0x80,
0x73, 0xa4, 0x80, 0x7e, 0xe3, 0xfc, 0x67, 0xb7, 0xf6, 0xf9, 0x57, 0xd7, 0xf0, 0xb6, 0xca, 0xef,
0xe9, 0xca, 0xe2, 0x26, 0xea, 0x8b, 0x9b, 0xb0, 0x1f, 0x43, 0x67, 0x69, 0x14, 0xc8, 0x86, 0x36,
0xcb, 0x03, 0xff, 0x8c, 0x14, 0xbe, 0xcc, 0xcb, 0x34, 0x7a, 0xf5, 0xbd, 0xa6, 0xd7, 0x62, 0x79,
0xf0, 0x8a, 0x14, 0xef, 0xc4, 0x92, 0xfd, 0x10, 0xda, 0x0b, 0x23, 0x40, 0x5d, 0x68, 0x61, 0xc6,
0x7c, 0x3d, 0x38, 0xb1, 0xb3, 0x55, 0x0f, 0x30, 0x63, 0x0a, 0xb3, 0x4f, 0x61, 0xf3, 0x18, 0xf3,
0x11, 0x19, 0x2a, 0xe1, 0x3e, 0x74, 0x64, 0x0a, 0xfe, 0x72, 0xc0, 0x6d, 0xb9, 0x7c, 0xa2, 0x53,
0xb6, 0xa1, 0x5d, 0x71, 0x55, 0xd6, 0x2d, 0x4d, 0x89, 0xc0, 0xbf, 0x18, 0xd0, 0x59, 0x1a, 0x2a,
0x3a, 0x86, 0x76, 0x4c, 0x38, 0x97, 0x21, 0x92, 0x31, 0x2e, 0xd4, 0x1f, 0xf0, 0x5f, 0x09, 0x6e,
0x2a, 0xf3, 0x48, 0x88, 0xe8, 0x00, 0x9a, 0x2c, 0x25, 0x83, 0x88, 0xdf, 0x70, 0x0e, 0x95, 0xd5,
0x7f, 0xff, 0x75, 0x66, 0x19, 0xe7, 0x33, 0xcb, 0xb8, 0x98, 0x59, 0xc6, 0xef, 0x99, 0x65, 0x7c,
0x9a, 0x5b, 0xb5, 0x8b, 0xb9, 0x55, 0xfb, 0x31, 0xb7, 0x6a, 0xa7, 0x4f, 0xc2, 0x28, 0x1b, 0xe5,
0x81, 0x33, 0xa0, 0xb1, 0x7b, 0xf5, 0xaa, 0xa8, 0x1e, 0xcb, 0xbb, 0x60, 0xf9, 0x1a, 0x09, 0xd6,
0xe5, 0xfa, 0xfe, 0x9f, 0x00, 0x00, 0x00, 0xff, 0xff, 0xcb, 0x26, 0x8a, 0x0b, 0x61, 0x04, 0x00,
0x00,
}
func (this *ConsensusParams) Equal(that interface{}) bool {
@ -451,6 +516,9 @@ func (this *ConsensusParams) Equal(that interface{}) bool {
if !this.Version.Equal(that1.Version) {
return false
}
if !this.Synchrony.Equal(that1.Synchrony) {
return false
}
return true
}
func (this *BlockParams) Equal(that interface{}) bool {
@ -590,6 +658,33 @@ func (this *HashedParams) Equal(that interface{}) bool {
}
return true
}
func (this *SynchronyParams) Equal(that interface{}) bool {
if that == nil {
return this == nil
}
that1, ok := that.(*SynchronyParams)
if !ok {
that2, ok := that.(SynchronyParams)
if ok {
that1 = &that2
} else {
return false
}
}
if that1 == nil {
return this == nil
} else if this == nil {
return false
}
if this.MessageDelay != that1.MessageDelay {
return false
}
if this.Precision != that1.Precision {
return false
}
return true
}
func (m *ConsensusParams) Marshal() (dAtA []byte, err error) {
size := m.Size()
dAtA = make([]byte, size)
@ -610,6 +705,18 @@ func (m *ConsensusParams) MarshalToSizedBuffer(dAtA []byte) (int, error) {
_ = i
var l int
_ = l
if m.Synchrony != nil {
{
size, err := m.Synchrony.MarshalToSizedBuffer(dAtA[:i])
if err != nil {
return 0, err
}
i -= size
i = encodeVarintParams(dAtA, i, uint64(size))
}
i--
dAtA[i] = 0x2a
}
if m.Version != nil {
{
size, err := m.Version.MarshalToSizedBuffer(dAtA[:i])
@ -719,12 +826,12 @@ func (m *EvidenceParams) MarshalToSizedBuffer(dAtA []byte) (int, error) {
i--
dAtA[i] = 0x18
}
n5, err5 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.MaxAgeDuration, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.MaxAgeDuration):])
if err5 != nil {
return 0, err5
n6, err6 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.MaxAgeDuration, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.MaxAgeDuration):])
if err6 != nil {
return 0, err6
}
i -= n5
i = encodeVarintParams(dAtA, i, uint64(n5))
i -= n6
i = encodeVarintParams(dAtA, i, uint64(n6))
i--
dAtA[i] = 0x12
if m.MaxAgeNumBlocks != 0 {
@ -828,6 +935,45 @@ func (m *HashedParams) MarshalToSizedBuffer(dAtA []byte) (int, error) {
return len(dAtA) - i, nil
}
func (m *SynchronyParams) Marshal() (dAtA []byte, err error) {
size := m.Size()
dAtA = make([]byte, size)
n, err := m.MarshalToSizedBuffer(dAtA[:size])
if err != nil {
return nil, err
}
return dAtA[:n], nil
}
func (m *SynchronyParams) MarshalTo(dAtA []byte) (int, error) {
size := m.Size()
return m.MarshalToSizedBuffer(dAtA[:size])
}
func (m *SynchronyParams) MarshalToSizedBuffer(dAtA []byte) (int, error) {
i := len(dAtA)
_ = i
var l int
_ = l
n7, err7 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.Precision, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.Precision):])
if err7 != nil {
return 0, err7
}
i -= n7
i = encodeVarintParams(dAtA, i, uint64(n7))
i--
dAtA[i] = 0x12
n8, err8 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.MessageDelay, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.MessageDelay):])
if err8 != nil {
return 0, err8
}
i -= n8
i = encodeVarintParams(dAtA, i, uint64(n8))
i--
dAtA[i] = 0xa
return len(dAtA) - i, nil
}
func encodeVarintParams(dAtA []byte, offset int, v uint64) int {
offset -= sovParams(v)
base := offset
@ -861,6 +1007,10 @@ func (m *ConsensusParams) Size() (n int) {
l = m.Version.Size()
n += 1 + l + sovParams(uint64(l))
}
if m.Synchrony != nil {
l = m.Synchrony.Size()
n += 1 + l + sovParams(uint64(l))
}
return n
}
@ -938,6 +1088,19 @@ func (m *HashedParams) Size() (n int) {
return n
}
func (m *SynchronyParams) Size() (n int) {
if m == nil {
return 0
}
var l int
_ = l
l = github_com_gogo_protobuf_types.SizeOfStdDuration(m.MessageDelay)
n += 1 + l + sovParams(uint64(l))
l = github_com_gogo_protobuf_types.SizeOfStdDuration(m.Precision)
n += 1 + l + sovParams(uint64(l))
return n
}
func sovParams(x uint64) (n int) {
return (math_bits.Len64(x|1) + 6) / 7
}
@ -1117,6 +1280,42 @@ func (m *ConsensusParams) Unmarshal(dAtA []byte) error {
return err
}
iNdEx = postIndex
case 5:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field Synchrony", wireType)
}
var msglen int
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowParams
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
msglen |= int(b&0x7F) << shift
if b < 0x80 {
break
}
}
if msglen < 0 {
return ErrInvalidLengthParams
}
postIndex := iNdEx + msglen
if postIndex < 0 {
return ErrInvalidLengthParams
}
if postIndex > l {
return io.ErrUnexpectedEOF
}
if m.Synchrony == nil {
m.Synchrony = &SynchronyParams{}
}
if err := m.Synchrony.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
return err
}
iNdEx = postIndex
default:
iNdEx = preIndex
skippy, err := skipParams(dAtA[iNdEx:])
@ -1586,6 +1785,122 @@ func (m *HashedParams) Unmarshal(dAtA []byte) error {
}
return nil
}
func (m *SynchronyParams) Unmarshal(dAtA []byte) error {
l := len(dAtA)
iNdEx := 0
for iNdEx < l {
preIndex := iNdEx
var wire uint64
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowParams
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
wire |= uint64(b&0x7F) << shift
if b < 0x80 {
break
}
}
fieldNum := int32(wire >> 3)
wireType := int(wire & 0x7)
if wireType == 4 {
return fmt.Errorf("proto: SynchronyParams: wiretype end group for non-group")
}
if fieldNum <= 0 {
return fmt.Errorf("proto: SynchronyParams: illegal tag %d (wire type %d)", fieldNum, wire)
}
switch fieldNum {
case 1:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field MessageDelay", wireType)
}
var msglen int
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowParams
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
msglen |= int(b&0x7F) << shift
if b < 0x80 {
break
}
}
if msglen < 0 {
return ErrInvalidLengthParams
}
postIndex := iNdEx + msglen
if postIndex < 0 {
return ErrInvalidLengthParams
}
if postIndex > l {
return io.ErrUnexpectedEOF
}
if err := github_com_gogo_protobuf_types.StdDurationUnmarshal(&m.MessageDelay, dAtA[iNdEx:postIndex]); err != nil {
return err
}
iNdEx = postIndex
case 2:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field Precision", wireType)
}
var msglen int
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowParams
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
msglen |= int(b&0x7F) << shift
if b < 0x80 {
break
}
}
if msglen < 0 {
return ErrInvalidLengthParams
}
postIndex := iNdEx + msglen
if postIndex < 0 {
return ErrInvalidLengthParams
}
if postIndex > l {
return io.ErrUnexpectedEOF
}
if err := github_com_gogo_protobuf_types.StdDurationUnmarshal(&m.Precision, dAtA[iNdEx:postIndex]); err != nil {
return err
}
iNdEx = postIndex
default:
iNdEx = preIndex
skippy, err := skipParams(dAtA[iNdEx:])
if err != nil {
return err
}
if (skippy < 0) || (iNdEx+skippy) < 0 {
return ErrInvalidLengthParams
}
if (iNdEx + skippy) > l {
return io.ErrUnexpectedEOF
}
iNdEx += skippy
}
}
if iNdEx > l {
return io.ErrUnexpectedEOF
}
return nil
}
func skipParams(dAtA []byte) (n int, err error) {
l := len(dAtA)
iNdEx := 0


+ 3
- 2
test/e2e/runner/evidence.go View File

@ -288,8 +288,9 @@ func makeBlockID(hash []byte, partSetSize uint32, partSetHash []byte) types.Bloc
}
}
func mutateValidatorSet(ctx context.Context, privVals []types.MockPV, vals *types.ValidatorSet) ([]types.PrivValidator, *types.ValidatorSet, error) {
newVal, newPrivVal, err := factory.RandValidator(ctx, false, 10)
func mutateValidatorSet(ctx context.Context, privVals []types.MockPV, vals *types.ValidatorSet,
) ([]types.PrivValidator, *types.ValidatorSet, error) {
newVal, newPrivVal, err := factory.Validator(ctx, 10)
if err != nil {
return nil, nil, err
}


+ 1
- 1
third_party/proto/gogoproto/gogo.proto View File

@ -144,4 +144,4 @@ extend google.protobuf.FieldOptions {
optional bool wktpointer = 65012;
optional string castrepeated = 65013;
}
}

+ 3
- 3
types/block.go 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()
}


+ 1
- 1
types/canonical.go 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{


+ 0
- 2
types/events.go View File

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


+ 9
- 4
types/genesis_test.go View File

@ -56,21 +56,26 @@ func TestGenesisBad(t *testing.T) {
}
}
func TestGenesisGood(t *testing.T) {
func TestBasicGenesisDoc(t *testing.T) {
// test a good one by raw json
genDocBytes := []byte(
`{
"genesis_time": "0001-01-01T00:00:00Z",
"chain_id": "test-chain-QDKdJr",
"initial_height": "1000",
"consensus_params": null,
"validators": [{
"pub_key":{"type":"tendermint/PubKeyEd25519","value":"AT/+aaL1eB0477Mud9JMm8Sh8BIvOYlPGC9KkIUmFaE="},
"power":"10",
"name":""
}],
"app_hash":"",
"app_state":{"account_owner": "Bob"}
"app_state":{"account_owner": "Bob"},
"consensus_params": {
"synchrony": {"precision": "1", "message_delay": "10"},
"validator": {"pub_key_types":["ed25519"]},
"block": {"max_bytes": "100"},
"evidence": {"max_age_num_blocks": "100", "max_age_duration": "10"}
}
}`,
)
_, err := GenesisDocFromJSON(genDocBytes)
@ -97,7 +102,7 @@ func TestGenesisGood(t *testing.T) {
genDocBytes, err = json.Marshal(genDoc)
assert.NoError(t, err, "error marshaling genDoc")
genDoc, err = GenesisDocFromJSON(genDocBytes)
assert.NoError(t, err, "expected no error for valid genDoc json")
require.NoError(t, err, "expected no error for valid genDoc json")
// test with invalid consensus params
genDoc.ConsensusParams.Block.MaxBytes = 0


+ 45
- 1
types/params.go View File

@ -41,6 +41,7 @@ type ConsensusParams struct {
Evidence EvidenceParams `json:"evidence"`
Validator ValidatorParams `json:"validator"`
Version VersionParams `json:"version"`
Synchrony SynchronyParams `json:"synchrony"`
}
// HashedParams is a subset of ConsensusParams.
@ -75,6 +76,15 @@ type VersionParams struct {
AppVersion uint64 `json:"app_version,string"`
}
// SynchronyParams influence the validity of block timestamps.
// For more information on the relationship of the synchrony parameters to
// block validity, see the Proposer-Based Timestamps specification:
// https://github.com/tendermint/spec/blob/master/spec/consensus/proposer-based-timestamp/README.md
type SynchronyParams struct {
Precision time.Duration `json:"precision,string"`
MessageDelay time.Duration `json:"message_delay,string"`
}
// DefaultConsensusParams returns a default ConsensusParams.
func DefaultConsensusParams() *ConsensusParams {
return &ConsensusParams{
@ -82,6 +92,7 @@ func DefaultConsensusParams() *ConsensusParams {
Evidence: DefaultEvidenceParams(),
Validator: DefaultValidatorParams(),
Version: DefaultVersionParams(),
Synchrony: DefaultSynchronyParams(),
}
}
@ -116,6 +127,15 @@ func DefaultVersionParams() VersionParams {
}
}
func DefaultSynchronyParams() SynchronyParams {
// TODO(@wbanfield): Determine experimental values for these defaults
// https://github.com/tendermint/tendermint/issues/7202
return SynchronyParams{
Precision: 500 * time.Millisecond,
MessageDelay: 2 * time.Second,
}
}
func (val *ValidatorParams) IsValidPubkeyType(pubkeyType string) bool {
for i := 0; i < len(val.PubKeyTypes); i++ {
if val.PubKeyTypes[i] == pubkeyType {
@ -148,7 +168,7 @@ func (params ConsensusParams) ValidateConsensusParams() error {
}
if params.Evidence.MaxAgeDuration <= 0 {
return fmt.Errorf("evidence.MaxAgeDuration must be grater than 0 if provided, Got %v",
return fmt.Errorf("evidence.MaxAgeDuration must be greater than 0 if provided, Got %v",
params.Evidence.MaxAgeDuration)
}
@ -162,6 +182,16 @@ func (params ConsensusParams) ValidateConsensusParams() error {
params.Evidence.MaxBytes)
}
if params.Synchrony.MessageDelay <= 0 {
return fmt.Errorf("synchrony.MessageDelay must be greater than 0. Got: %d",
params.Synchrony.MessageDelay)
}
if params.Synchrony.Precision <= 0 {
return fmt.Errorf("synchrony.Precision must be greater than 0. Got: %d",
params.Synchrony.Precision)
}
if len(params.Validator.PubKeyTypes) == 0 {
return errors.New("len(Validator.PubKeyTypes) must be greater than 0")
}
@ -205,6 +235,8 @@ func (params ConsensusParams) HashConsensusParams() []byte {
func (params *ConsensusParams) Equals(params2 *ConsensusParams) bool {
return params.Block == params2.Block &&
params.Evidence == params2.Evidence &&
params.Version == params2.Version &&
params.Synchrony == params2.Synchrony &&
tmstrings.StringSliceEqual(params.Validator.PubKeyTypes, params2.Validator.PubKeyTypes)
}
@ -235,6 +267,10 @@ func (params ConsensusParams) UpdateConsensusParams(params2 *tmproto.ConsensusPa
if params2.Version != nil {
res.Version.AppVersion = params2.Version.AppVersion
}
if params2.Synchrony != nil {
res.Synchrony.Precision = params2.Synchrony.Precision
res.Synchrony.MessageDelay = params2.Synchrony.MessageDelay
}
return res
}
@ -255,6 +291,10 @@ func (params *ConsensusParams) ToProto() tmproto.ConsensusParams {
Version: &tmproto.VersionParams{
AppVersion: params.Version.AppVersion,
},
Synchrony: &tmproto.SynchronyParams{
MessageDelay: params.Synchrony.MessageDelay,
Precision: params.Synchrony.Precision,
},
}
}
@ -275,5 +315,9 @@ func ConsensusParamsFromProto(pbParams tmproto.ConsensusParams) ConsensusParams
Version: VersionParams{
AppVersion: pbParams.Version.AppVersion,
},
Synchrony: SynchronyParams{
MessageDelay: pbParams.Synchrony.MessageDelay,
Precision: pbParams.Synchrony.Precision,
},
}
}

+ 207
- 54
types/params_test.go View File

@ -23,23 +23,140 @@ func TestConsensusParamsValidation(t *testing.T) {
valid bool
}{
// test block params
0: {makeParams(1, 0, 2, 0, valEd25519), true},
1: {makeParams(0, 0, 2, 0, valEd25519), false},
2: {makeParams(47*1024*1024, 0, 2, 0, valEd25519), true},
3: {makeParams(10, 0, 2, 0, valEd25519), true},
4: {makeParams(100*1024*1024, 0, 2, 0, valEd25519), true},
5: {makeParams(101*1024*1024, 0, 2, 0, valEd25519), false},
6: {makeParams(1024*1024*1024, 0, 2, 0, valEd25519), false},
7: {makeParams(1024*1024*1024, 0, -1, 0, valEd25519), false},
{
params: makeParams(makeParamsArgs{
blockBytes: 1,
evidenceAge: 2,
precision: 1,
messageDelay: 1}),
valid: true,
},
{
params: makeParams(makeParamsArgs{
blockBytes: 0,
evidenceAge: 2,
precision: 1,
messageDelay: 1}),
valid: false,
},
{
params: makeParams(makeParamsArgs{
blockBytes: 47 * 1024 * 1024,
evidenceAge: 2,
precision: 1,
messageDelay: 1}),
valid: true,
},
{
params: makeParams(makeParamsArgs{
blockBytes: 10,
evidenceAge: 2,
precision: 1,
messageDelay: 1}),
valid: true,
},
{
params: makeParams(makeParamsArgs{
blockBytes: 100 * 1024 * 1024,
evidenceAge: 2,
precision: 1,
messageDelay: 1}),
valid: true,
},
{
params: makeParams(makeParamsArgs{
blockBytes: 101 * 1024 * 1024,
evidenceAge: 2,
precision: 1,
messageDelay: 1}),
valid: false,
},
{
params: makeParams(makeParamsArgs{
blockBytes: 1024 * 1024 * 1024,
evidenceAge: 2,
precision: 1,
messageDelay: 1}),
valid: false,
},
{
params: makeParams(makeParamsArgs{
blockBytes: 1024 * 1024 * 1024,
evidenceAge: 2,
precision: 1,
messageDelay: 1}),
valid: false,
},
// test evidence params
8: {makeParams(1, 0, 0, 0, valEd25519), false},
9: {makeParams(1, 0, 2, 2, valEd25519), false},
10: {makeParams(1000, 0, 2, 1, valEd25519), true},
11: {makeParams(1, 0, -1, 0, valEd25519), false},
{
params: makeParams(makeParamsArgs{
blockBytes: 1,
evidenceAge: 0,
maxEvidenceBytes: 0,
precision: 1,
messageDelay: 1}),
valid: false,
},
{
params: makeParams(makeParamsArgs{
blockBytes: 1,
evidenceAge: 2,
maxEvidenceBytes: 2,
precision: 1,
messageDelay: 1}),
valid: false,
},
{
params: makeParams(makeParamsArgs{
blockBytes: 1000,
evidenceAge: 2,
maxEvidenceBytes: 1,
precision: 1,
messageDelay: 1}),
valid: true,
},
{
params: makeParams(makeParamsArgs{
blockBytes: 1,
evidenceAge: -1,
maxEvidenceBytes: 0,
precision: 1,
messageDelay: 1}),
valid: false,
},
// test no pubkey type provided
12: {makeParams(1, 0, 2, 0, []string{}), false},
{
params: makeParams(makeParamsArgs{
evidenceAge: 2,
pubkeyTypes: []string{},
precision: 1,
messageDelay: 1}),
valid: false,
},
// test invalid pubkey type provided
13: {makeParams(1, 0, 2, 0, []string{"potatoes make good pubkeys"}), false},
{
params: makeParams(makeParamsArgs{
evidenceAge: 2,
pubkeyTypes: []string{"potatoes make good pubkeys"},
precision: 1,
messageDelay: 1}),
valid: false,
},
// test invalid pubkey type provided
{
params: makeParams(makeParamsArgs{
evidenceAge: 2,
precision: 1,
messageDelay: -1}),
valid: false,
},
{
params: makeParams(makeParamsArgs{
evidenceAge: 2,
precision: -1,
messageDelay: 1}),
valid: false,
},
}
for i, tc := range testCases {
if tc.valid {
@ -50,38 +167,51 @@ func TestConsensusParamsValidation(t *testing.T) {
}
}
func makeParams(
blockBytes, blockGas int64,
evidenceAge int64,
maxEvidenceBytes int64,
pubkeyTypes []string,
) ConsensusParams {
type makeParamsArgs struct {
blockBytes int64
blockGas int64
evidenceAge int64
maxEvidenceBytes int64
pubkeyTypes []string
precision time.Duration
messageDelay time.Duration
}
func makeParams(args makeParamsArgs) ConsensusParams {
if args.pubkeyTypes == nil {
args.pubkeyTypes = valEd25519
}
return ConsensusParams{
Block: BlockParams{
MaxBytes: blockBytes,
MaxGas: blockGas,
MaxBytes: args.blockBytes,
MaxGas: args.blockGas,
},
Evidence: EvidenceParams{
MaxAgeNumBlocks: evidenceAge,
MaxAgeDuration: time.Duration(evidenceAge),
MaxBytes: maxEvidenceBytes,
MaxAgeNumBlocks: args.evidenceAge,
MaxAgeDuration: time.Duration(args.evidenceAge),
MaxBytes: args.maxEvidenceBytes,
},
Validator: ValidatorParams{
PubKeyTypes: pubkeyTypes,
PubKeyTypes: args.pubkeyTypes,
},
Synchrony: SynchronyParams{
Precision: args.precision,
MessageDelay: args.messageDelay,
},
}
}
func TestConsensusParamsHash(t *testing.T) {
params := []ConsensusParams{
makeParams(4, 2, 3, 1, valEd25519),
makeParams(1, 4, 3, 1, valEd25519),
makeParams(1, 2, 4, 1, valEd25519),
makeParams(2, 5, 7, 1, valEd25519),
makeParams(1, 7, 6, 1, valEd25519),
makeParams(9, 5, 4, 1, valEd25519),
makeParams(7, 8, 9, 1, valEd25519),
makeParams(4, 6, 5, 1, valEd25519),
makeParams(makeParamsArgs{blockBytes: 4, blockGas: 2, evidenceAge: 3, maxEvidenceBytes: 1}),
makeParams(makeParamsArgs{blockBytes: 1, blockGas: 4, evidenceAge: 3, maxEvidenceBytes: 1}),
makeParams(makeParamsArgs{blockBytes: 1, blockGas: 2, evidenceAge: 4, maxEvidenceBytes: 1}),
makeParams(makeParamsArgs{blockBytes: 2, blockGas: 5, evidenceAge: 7, maxEvidenceBytes: 1}),
makeParams(makeParamsArgs{blockBytes: 1, blockGas: 7, evidenceAge: 6, maxEvidenceBytes: 1}),
makeParams(makeParamsArgs{blockBytes: 9, blockGas: 5, evidenceAge: 4, maxEvidenceBytes: 1}),
makeParams(makeParamsArgs{blockBytes: 7, blockGas: 8, evidenceAge: 9, maxEvidenceBytes: 1}),
makeParams(makeParamsArgs{blockBytes: 4, blockGas: 6, evidenceAge: 5, maxEvidenceBytes: 1}),
}
hashes := make([][]byte, len(params))
@ -101,20 +231,31 @@ func TestConsensusParamsHash(t *testing.T) {
func TestConsensusParamsUpdate(t *testing.T) {
testCases := []struct {
params ConsensusParams
intialParams ConsensusParams
updates *tmproto.ConsensusParams
updatedParams ConsensusParams
}{
// empty updates
{
makeParams(1, 2, 3, 0, valEd25519),
&tmproto.ConsensusParams{},
makeParams(1, 2, 3, 0, valEd25519),
intialParams: makeParams(makeParamsArgs{blockBytes: 1, blockGas: 2, evidenceAge: 3}),
updates: &tmproto.ConsensusParams{},
updatedParams: makeParams(makeParamsArgs{blockBytes: 1, blockGas: 2, evidenceAge: 3}),
},
{
// update synchrony params
intialParams: makeParams(makeParamsArgs{evidenceAge: 3, precision: time.Second, messageDelay: 3 * time.Second}),
updates: &tmproto.ConsensusParams{
Synchrony: &tmproto.SynchronyParams{
Precision: time.Second * 2,
MessageDelay: time.Second * 4,
},
},
updatedParams: makeParams(makeParamsArgs{evidenceAge: 3, precision: 2 * time.Second, messageDelay: 4 * time.Second}),
},
// fine updates
{
makeParams(1, 2, 3, 0, valEd25519),
&tmproto.ConsensusParams{
intialParams: makeParams(makeParamsArgs{blockBytes: 1, blockGas: 2, evidenceAge: 3}),
updates: &tmproto.ConsensusParams{
Block: &tmproto.BlockParams{
MaxBytes: 100,
MaxGas: 200,
@ -128,11 +269,15 @@ func TestConsensusParamsUpdate(t *testing.T) {
PubKeyTypes: valSecp256k1,
},
},
makeParams(100, 200, 300, 50, valSecp256k1),
updatedParams: makeParams(makeParamsArgs{
blockBytes: 100, blockGas: 200,
evidenceAge: 300,
maxEvidenceBytes: 50,
pubkeyTypes: valSecp256k1}),
},
{
makeParams(1, 2, 3, 0, valEd25519),
&tmproto.ConsensusParams{
intialParams: makeParams(makeParamsArgs{blockBytes: 1, blockGas: 2, evidenceAge: 3}),
updates: &tmproto.ConsensusParams{
Block: &tmproto.BlockParams{
MaxBytes: 100,
MaxGas: 200,
@ -145,17 +290,23 @@ func TestConsensusParamsUpdate(t *testing.T) {
Validator: &tmproto.ValidatorParams{
PubKeyTypes: valSr25519,
},
}, makeParams(100, 200, 300, 50, valSr25519),
},
updatedParams: makeParams(makeParamsArgs{
blockBytes: 100,
blockGas: 200,
evidenceAge: 300,
maxEvidenceBytes: 50,
pubkeyTypes: valSr25519}),
},
}
for _, tc := range testCases {
assert.Equal(t, tc.updatedParams, tc.params.UpdateConsensusParams(tc.updates))
assert.Equal(t, tc.updatedParams, tc.intialParams.UpdateConsensusParams(tc.updates))
}
}
func TestConsensusParamsUpdate_AppVersion(t *testing.T) {
params := makeParams(1, 2, 3, 0, valEd25519)
params := makeParams(makeParamsArgs{blockBytes: 1, blockGas: 2, evidenceAge: 3})
assert.EqualValues(t, 0, params.Version.AppVersion)
@ -167,14 +318,16 @@ func TestConsensusParamsUpdate_AppVersion(t *testing.T) {
func TestProto(t *testing.T) {
params := []ConsensusParams{
makeParams(4, 2, 3, 1, valEd25519),
makeParams(1, 4, 3, 1, valEd25519),
makeParams(1, 2, 4, 1, valEd25519),
makeParams(2, 5, 7, 1, valEd25519),
makeParams(1, 7, 6, 1, valEd25519),
makeParams(9, 5, 4, 1, valEd25519),
makeParams(7, 8, 9, 1, valEd25519),
makeParams(4, 6, 5, 1, valEd25519),
makeParams(makeParamsArgs{blockBytes: 4, blockGas: 2, evidenceAge: 3, maxEvidenceBytes: 1}),
makeParams(makeParamsArgs{blockBytes: 1, blockGas: 4, evidenceAge: 3, maxEvidenceBytes: 1}),
makeParams(makeParamsArgs{blockBytes: 1, blockGas: 2, evidenceAge: 4, maxEvidenceBytes: 1}),
makeParams(makeParamsArgs{blockBytes: 2, blockGas: 5, evidenceAge: 7, maxEvidenceBytes: 1}),
makeParams(makeParamsArgs{blockBytes: 1, blockGas: 7, evidenceAge: 6, maxEvidenceBytes: 1}),
makeParams(makeParamsArgs{blockBytes: 9, blockGas: 5, evidenceAge: 4, maxEvidenceBytes: 1}),
makeParams(makeParamsArgs{blockBytes: 7, blockGas: 8, evidenceAge: 9, maxEvidenceBytes: 1}),
makeParams(makeParamsArgs{blockBytes: 4, blockGas: 6, evidenceAge: 5, maxEvidenceBytes: 1}),
makeParams(makeParamsArgs{precision: time.Second, messageDelay: time.Minute}),
makeParams(makeParamsArgs{precision: time.Nanosecond, messageDelay: time.Millisecond}),
}
for i := range params {


+ 27
- 2
types/proposal.go View File

@ -34,14 +34,14 @@ type Proposal struct {
// NewProposal returns a new Proposal.
// If there is no POLRound, polRound should be -1.
func NewProposal(height int64, round int32, polRound int32, blockID BlockID) *Proposal {
func NewProposal(height int64, round int32, polRound int32, blockID BlockID, ts time.Time) *Proposal {
return &Proposal{
Type: tmproto.ProposalType,
Height: height,
Round: round,
BlockID: blockID,
POLRound: polRound,
Timestamp: tmtime.Now(),
Timestamp: tmtime.Canonical(ts),
}
}
@ -79,6 +79,31 @@ func (p *Proposal) ValidateBasic() error {
return nil
}
// IsTimely validates that the block timestamp is 'timely' according to the proposer-based timestamp algorithm.
// To evaluate if a block is timely, its timestamp is compared to the local time of the validator along with the
// configured Precision and MsgDelay parameters.
// Specifically, a proposed block timestamp is considered timely if it is satisfies the following inequalities:
//
// localtime >= proposedBlockTime - Precision
// localtime <= proposedBlockTime + MsgDelay + Precision
//
// Note: If the proposal is for the `initialHeight` the second inequality is not checked. This is because
// the timestamp in this case is set to the preconfigured genesis time.
// For more information on the meaning of 'timely', see the proposer-based timestamp specification:
// https://github.com/tendermint/spec/tree/master/spec/consensus/proposer-based-timestamp
func (p *Proposal) IsTimely(recvTime time.Time, sp SynchronyParams, initialHeight int64) bool {
// lhs is `proposedBlockTime - Precision` in the first inequality
lhs := p.Timestamp.Add(-sp.Precision)
// rhs is `proposedBlockTime + MsgDelay + Precision` in the second inequality
rhs := p.Timestamp.Add(sp.MessageDelay).Add(sp.Precision)
if recvTime.Before(lhs) || (p.Height != initialHeight && recvTime.After(rhs)) {
return false
}
return true
}
// String returns a string representation of the Proposal.
//
// 1. height


+ 101
- 4
types/proposal_test.go 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"
tmtime "github.com/tendermint/tendermint/libs/time"
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
)
@ -60,7 +61,7 @@ func TestProposalVerifySignature(t *testing.T) {
prop := NewProposal(
4, 2, 2,
BlockID{tmrand.Bytes(tmhash.Size), PartSetHeader{777, tmrand.Bytes(tmhash.Size)}})
BlockID{tmrand.Bytes(tmhash.Size), PartSetHeader{777, tmrand.Bytes(tmhash.Size)}}, tmtime.Now())
p := prop.ToProto()
signBytes := ProposalSignBytes("test_chain_id", p)
@ -172,7 +173,7 @@ func TestProposalValidateBasic(t *testing.T) {
prop := NewProposal(
4, 2, 2,
blockID)
blockID, tmtime.Now())
p := prop.ToProto()
err := privVal.SignProposal(ctx, "test_chain_id", p)
prop.Signature = p.Signature
@ -184,9 +185,9 @@ func TestProposalValidateBasic(t *testing.T) {
}
func TestProposalProtoBuf(t *testing.T) {
proposal := NewProposal(1, 2, 3, makeBlockID([]byte("hash"), 2, []byte("part_set_hash")))
proposal := NewProposal(1, 2, 3, makeBlockID([]byte("hash"), 2, []byte("part_set_hash")), tmtime.Now())
proposal.Signature = []byte("sig")
proposal2 := NewProposal(1, 2, 3, BlockID{})
proposal2 := NewProposal(1, 2, 3, BlockID{}, tmtime.Now())
testCases := []struct {
msg string
@ -210,3 +211,99 @@ func TestProposalProtoBuf(t *testing.T) {
}
}
}
func TestIsTimely(t *testing.T) {
genesisTime, err := time.Parse(time.RFC3339, "2019-03-13T23:00:00Z")
require.NoError(t, err)
testCases := []struct {
name string
genesisHeight int64
proposalHeight int64
proposalTime time.Time
recvTime time.Time
precision time.Duration
msgDelay time.Duration
expectTimely bool
}{
// proposalTime - precision <= localTime <= proposalTime + msgDelay + precision
{
// Checking that the following inequality evaluates to true:
// 0 - 2 <= 1 <= 0 + 1 + 2
name: "basic timely",
genesisHeight: 1,
proposalHeight: 2,
proposalTime: genesisTime,
recvTime: genesisTime.Add(1 * time.Nanosecond),
precision: time.Nanosecond * 2,
msgDelay: time.Nanosecond,
expectTimely: true,
},
{
// Checking that the following inequality evaluates to false:
// 0 - 2 <= 4 <= 0 + 1 + 2
name: "local time too large",
genesisHeight: 1,
proposalHeight: 2,
proposalTime: genesisTime,
recvTime: genesisTime.Add(4 * time.Nanosecond),
precision: time.Nanosecond * 2,
msgDelay: time.Nanosecond,
expectTimely: false,
},
{
// Checking that the following inequality evaluates to false:
// 4 - 2 <= 0 <= 4 + 2 + 1
name: "proposal time too large",
genesisHeight: 1,
proposalHeight: 2,
proposalTime: genesisTime.Add(4 * time.Nanosecond),
recvTime: genesisTime,
precision: time.Nanosecond * 2,
msgDelay: time.Nanosecond,
expectTimely: false,
},
{
// Checking that the following inequality evaluates to true:
// 0 - 2 <= 4
// and the following check is skipped
// 4 <= 0 + 1 + 2
name: "local time too large but proposal is for genesis",
genesisHeight: 1,
proposalHeight: 1,
proposalTime: genesisTime,
recvTime: genesisTime.Add(4 * time.Nanosecond),
precision: time.Nanosecond * 2,
msgDelay: time.Nanosecond,
expectTimely: true,
},
{
// Checking that the following inequality evaluates to false:
// 4 - 2 <= 0
name: "proposal time too large for genesis block proposal",
genesisHeight: 1,
proposalHeight: 1,
proposalTime: genesisTime.Add(4 * time.Nanosecond),
recvTime: genesisTime,
precision: time.Nanosecond * 2,
msgDelay: time.Nanosecond,
expectTimely: false,
},
}
for _, testCase := range testCases {
t.Run(testCase.name, func(t *testing.T) {
p := Proposal{
Height: testCase.proposalHeight,
Timestamp: testCase.proposalTime,
}
sp := SynchronyParams{
Precision: testCase.precision,
MessageDelay: testCase.msgDelay,
}
ti := p.IsTimely(testCase.recvTime, sp, testCase.genesisHeight)
assert.Equal(t, testCase.expectTimely, ti)
})
}
}

+ 2
- 2
types/vote.go 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)
}


+ 10
- 10
types/vote_set_test.go View File

@ -31,7 +31,7 @@ func TestVoteSet_AddVote_Good(t *testing.T) {
assert.Nil(t, voteSet.GetByAddress(val0Addr))
assert.False(t, voteSet.BitArray().GetIndex(0))
blockID, ok := voteSet.TwoThirdsMajority()
assert.False(t, ok || !blockID.IsZero(), "there should be no 2/3 majority")
assert.False(t, ok || !blockID.IsNil(), "there should be no 2/3 majority")
vote := &Vote{
ValidatorAddress: val0Addr,
@ -48,7 +48,7 @@ func TestVoteSet_AddVote_Good(t *testing.T) {
assert.NotNil(t, voteSet.GetByAddress(val0Addr))
assert.True(t, voteSet.BitArray().GetIndex(0))
blockID, ok = voteSet.TwoThirdsMajority()
assert.False(t, ok || !blockID.IsZero(), "there should be no 2/3 majority")
assert.False(t, ok || !blockID.IsNil(), "there should be no 2/3 majority")
}
func TestVoteSet_AddVote_Bad(t *testing.T) {
@ -155,7 +155,7 @@ func TestVoteSet_2_3Majority(t *testing.T) {
require.NoError(t, err)
}
blockID, ok := voteSet.TwoThirdsMajority()
assert.False(t, ok || !blockID.IsZero(), "there should be no 2/3 majority")
assert.False(t, ok || !blockID.IsNil(), "there should be no 2/3 majority")
// 7th validator voted for some blockhash
{
@ -166,7 +166,7 @@ func TestVoteSet_2_3Majority(t *testing.T) {
_, err = signAddVote(ctx, privValidators[6], withBlockHash(vote, tmrand.Bytes(32)), voteSet)
require.NoError(t, err)
blockID, ok = voteSet.TwoThirdsMajority()
assert.False(t, ok || !blockID.IsZero(), "there should be no 2/3 majority")
assert.False(t, ok || !blockID.IsNil(), "there should be no 2/3 majority")
}
// 8th validator voted for nil.
@ -178,7 +178,7 @@ func TestVoteSet_2_3Majority(t *testing.T) {
_, err = signAddVote(ctx, privValidators[7], vote, voteSet)
require.NoError(t, err)
blockID, ok = voteSet.TwoThirdsMajority()
assert.True(t, ok || blockID.IsZero(), "there should be 2/3 majority for nil")
assert.True(t, ok || blockID.IsNil(), "there should be 2/3 majority for nil")
}
}
@ -213,7 +213,7 @@ func TestVoteSet_2_3MajorityRedux(t *testing.T) {
require.NoError(t, err)
}
blockID, ok := voteSet.TwoThirdsMajority()
assert.False(t, ok || !blockID.IsZero(),
assert.False(t, ok || !blockID.IsNil(),
"there should be no 2/3 majority")
// 67th validator voted for nil
@ -225,7 +225,7 @@ func TestVoteSet_2_3MajorityRedux(t *testing.T) {
_, err = signAddVote(ctx, privValidators[66], withBlockHash(vote, nil), voteSet)
require.NoError(t, err)
blockID, ok = voteSet.TwoThirdsMajority()
assert.False(t, ok || !blockID.IsZero(),
assert.False(t, ok || !blockID.IsNil(),
"there should be no 2/3 majority: last vote added was nil")
}
@ -239,7 +239,7 @@ func TestVoteSet_2_3MajorityRedux(t *testing.T) {
_, err = signAddVote(ctx, privValidators[67], withBlockPartSetHeader(vote, blockPartsHeader), voteSet)
require.NoError(t, err)
blockID, ok = voteSet.TwoThirdsMajority()
assert.False(t, ok || !blockID.IsZero(),
assert.False(t, ok || !blockID.IsNil(),
"there should be no 2/3 majority: last vote added had different PartSetHeader Hash")
}
@ -253,7 +253,7 @@ func TestVoteSet_2_3MajorityRedux(t *testing.T) {
_, err = signAddVote(ctx, privValidators[68], withBlockPartSetHeader(vote, blockPartsHeader), voteSet)
require.NoError(t, err)
blockID, ok = voteSet.TwoThirdsMajority()
assert.False(t, ok || !blockID.IsZero(),
assert.False(t, ok || !blockID.IsNil(),
"there should be no 2/3 majority: last vote added had different PartSetHeader Total")
}
@ -266,7 +266,7 @@ func TestVoteSet_2_3MajorityRedux(t *testing.T) {
_, err = signAddVote(ctx, privValidators[69], withBlockHash(vote, tmrand.Bytes(32)), voteSet)
require.NoError(t, err)
blockID, ok = voteSet.TwoThirdsMajority()
assert.False(t, ok || !blockID.IsZero(),
assert.False(t, ok || !blockID.IsNil(),
"there should be no 2/3 majority: last vote added had different BlockHash")
}


Loading…
Cancel
Save