You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.

500 lines
17 KiB

  1. package consensus
  2. import (
  3. "bytes"
  4. "context"
  5. "testing"
  6. "time"
  7. "github.com/stretchr/testify/assert"
  8. "github.com/stretchr/testify/require"
  9. "github.com/tendermint/tendermint/abci/example/kvstore"
  10. "github.com/tendermint/tendermint/internal/eventbus"
  11. tmpubsub "github.com/tendermint/tendermint/internal/pubsub"
  12. "github.com/tendermint/tendermint/libs/log"
  13. tmtimemocks "github.com/tendermint/tendermint/libs/time/mocks"
  14. tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
  15. "github.com/tendermint/tendermint/types"
  16. )
  17. const (
  18. // blockTimeIota is used in the test harness as the time between
  19. // blocks when not otherwise specified.
  20. blockTimeIota = time.Millisecond
  21. )
  22. // pbtsTestHarness constructs a Tendermint network that can be used for testing the
  23. // implementation of the Proposer-Based timestamps algorithm.
  24. // It runs a series of consensus heights and captures timing of votes and events.
  25. type pbtsTestHarness struct {
  26. // configuration options set by the user of the test harness.
  27. pbtsTestConfiguration
  28. // The timestamp of the first block produced by the network.
  29. firstBlockTime time.Time
  30. // The Tendermint consensus state machine being run during
  31. // a run of the pbtsTestHarness.
  32. observedState *State
  33. // A stub for signing votes and messages using the key
  34. // from the observedState.
  35. observedValidator *validatorStub
  36. // A list of simulated validators that interact with the observedState and are
  37. // fully controlled by the test harness.
  38. otherValidators []*validatorStub
  39. // The mock time source used by all of the validator stubs in the test harness.
  40. // This mock clock allows the test harness to produce votes and blocks with arbitrary
  41. // timestamps.
  42. validatorClock *tmtimemocks.Source
  43. chainID string
  44. // channels for verifying that the observed validator completes certain actions.
  45. ensureProposalCh, roundCh, blockCh, ensureVoteCh <-chan tmpubsub.Message
  46. // channel of events from the observed validator annotated with the timestamp
  47. // the event was received.
  48. eventCh <-chan timestampedEvent
  49. currentHeight int64
  50. currentRound int32
  51. }
  52. type pbtsTestConfiguration struct {
  53. // The timestamp consensus parameters to be used by the state machine under test.
  54. synchronyParams types.SynchronyParams
  55. // The setting to use for the TimeoutPropose configuration parameter.
  56. timeoutPropose time.Duration
  57. // The genesis time
  58. genesisTime time.Time
  59. // The times offset from height 1 block time of the block proposed at height 2.
  60. height2ProposedBlockOffset time.Duration
  61. // The time offset from height 1 block time at which the proposal at height 2 should be delivered.
  62. height2ProposalTimeDeliveryOffset time.Duration
  63. // The time offset from height 1 block time of the block proposed at height 4.
  64. // At height 4, the proposed block and the deliver offsets are the same so
  65. // that timely-ness does not affect height 4.
  66. height4ProposedBlockOffset time.Duration
  67. }
  68. func newPBTSTestHarness(ctx context.Context, t *testing.T, tc pbtsTestConfiguration) pbtsTestHarness {
  69. t.Helper()
  70. const validators = 4
  71. cfg := configSetup(t)
  72. clock := new(tmtimemocks.Source)
  73. if tc.genesisTime.IsZero() {
  74. tc.genesisTime = time.Now()
  75. }
  76. if tc.height4ProposedBlockOffset == 0 {
  77. // Set a default height4ProposedBlockOffset.
  78. // Use a proposed block time that is greater than the time that the
  79. // block at height 2 was delivered. Height 3 is not relevant for testing
  80. // and always occurs blockTimeIota before height 4. If not otherwise specified,
  81. // height 4 therefore occurs 2*blockTimeIota after height 2.
  82. tc.height4ProposedBlockOffset = tc.height2ProposalTimeDeliveryOffset + 2*blockTimeIota
  83. }
  84. cfg.Consensus.TimeoutPropose = tc.timeoutPropose
  85. consensusParams := types.DefaultConsensusParams()
  86. consensusParams.Synchrony = tc.synchronyParams
  87. state, privVals := makeGenesisState(ctx, t, cfg, genesisStateArgs{
  88. Params: consensusParams,
  89. Time: tc.genesisTime,
  90. Validators: validators,
  91. })
  92. cs := newState(ctx, t, log.NewNopLogger(), state, privVals[0], kvstore.NewApplication())
  93. vss := make([]*validatorStub, validators)
  94. for i := 0; i < validators; i++ {
  95. vss[i] = newValidatorStub(privVals[i], int32(i))
  96. }
  97. incrementHeight(vss[1:]...)
  98. for _, vs := range vss {
  99. vs.clock = clock
  100. }
  101. pubKey, err := vss[0].PrivValidator.GetPubKey(ctx)
  102. require.NoError(t, err)
  103. eventCh := timestampedCollector(ctx, t, cs.eventBus)
  104. return pbtsTestHarness{
  105. pbtsTestConfiguration: tc,
  106. observedValidator: vss[0],
  107. observedState: cs,
  108. otherValidators: vss[1:],
  109. validatorClock: clock,
  110. currentHeight: 1,
  111. chainID: cfg.ChainID(),
  112. roundCh: subscribe(ctx, t, cs.eventBus, types.EventQueryNewRound),
  113. ensureProposalCh: subscribe(ctx, t, cs.eventBus, types.EventQueryCompleteProposal),
  114. blockCh: subscribe(ctx, t, cs.eventBus, types.EventQueryNewBlock),
  115. ensureVoteCh: subscribeToVoterBuffered(ctx, t, cs, pubKey.Address()),
  116. eventCh: eventCh,
  117. }
  118. }
  119. func (p *pbtsTestHarness) observedValidatorProposerHeight(ctx context.Context, t *testing.T, previousBlockTime time.Time) (heightResult, time.Time) {
  120. p.validatorClock.On("Now").Return(p.genesisTime.Add(p.height2ProposedBlockOffset)).Times(6)
  121. ensureNewRound(t, p.roundCh, p.currentHeight, p.currentRound)
  122. timeout := time.Until(previousBlockTime.Add(ensureTimeout))
  123. ensureProposalWithTimeout(t, p.ensureProposalCh, p.currentHeight, p.currentRound, nil, timeout)
  124. rs := p.observedState.GetRoundState()
  125. bid := types.BlockID{Hash: rs.ProposalBlock.Hash(), PartSetHeader: rs.ProposalBlockParts.Header()}
  126. ensurePrevote(t, p.ensureVoteCh, p.currentHeight, p.currentRound)
  127. signAddVotes(ctx, t, p.observedState, tmproto.PrevoteType, p.chainID, bid, p.otherValidators...)
  128. signAddVotes(ctx, t, p.observedState, tmproto.PrecommitType, p.chainID, bid, p.otherValidators...)
  129. ensurePrecommit(t, p.ensureVoteCh, p.currentHeight, p.currentRound)
  130. ensureNewBlock(t, p.blockCh, p.currentHeight)
  131. vk, err := p.observedValidator.GetPubKey(ctx)
  132. require.NoError(t, err)
  133. res := collectHeightResults(ctx, t, p.eventCh, p.currentHeight, vk.Address())
  134. p.currentHeight++
  135. incrementHeight(p.otherValidators...)
  136. return res, rs.ProposalBlock.Time
  137. }
  138. func (p *pbtsTestHarness) height2(ctx context.Context, t *testing.T) heightResult {
  139. signer := p.otherValidators[0].PrivValidator
  140. return p.nextHeight(ctx, t, signer,
  141. p.firstBlockTime.Add(p.height2ProposalTimeDeliveryOffset),
  142. p.firstBlockTime.Add(p.height2ProposedBlockOffset),
  143. p.firstBlockTime.Add(p.height2ProposedBlockOffset+10*blockTimeIota))
  144. }
  145. func (p *pbtsTestHarness) intermediateHeights(ctx context.Context, t *testing.T) {
  146. signer := p.otherValidators[1].PrivValidator
  147. p.nextHeight(ctx, t, signer,
  148. p.firstBlockTime.Add(p.height2ProposedBlockOffset+10*blockTimeIota),
  149. p.firstBlockTime.Add(p.height2ProposedBlockOffset+10*blockTimeIota),
  150. p.firstBlockTime.Add(p.height4ProposedBlockOffset))
  151. signer = p.otherValidators[2].PrivValidator
  152. p.nextHeight(ctx, t, signer,
  153. p.firstBlockTime.Add(p.height4ProposedBlockOffset),
  154. p.firstBlockTime.Add(p.height4ProposedBlockOffset),
  155. time.Now())
  156. }
  157. func (p *pbtsTestHarness) height5(ctx context.Context, t *testing.T) (heightResult, time.Time) {
  158. return p.observedValidatorProposerHeight(ctx, t, p.firstBlockTime.Add(p.height4ProposedBlockOffset))
  159. }
  160. func (p *pbtsTestHarness) nextHeight(ctx context.Context, t *testing.T, proposer types.PrivValidator, deliverTime, proposedTime, nextProposedTime time.Time) heightResult {
  161. p.validatorClock.On("Now").Return(nextProposedTime).Times(6)
  162. ensureNewRound(t, p.roundCh, p.currentHeight, p.currentRound)
  163. b, err := p.observedState.createProposalBlock(ctx)
  164. require.NoError(t, err)
  165. b.Height = p.currentHeight
  166. b.Header.Height = p.currentHeight
  167. b.Header.Time = proposedTime
  168. k, err := proposer.GetPubKey(ctx)
  169. require.NoError(t, err)
  170. b.Header.ProposerAddress = k.Address()
  171. ps, err := b.MakePartSet(types.BlockPartSizeBytes)
  172. require.NoError(t, err)
  173. bid := types.BlockID{Hash: b.Hash(), PartSetHeader: ps.Header()}
  174. prop := types.NewProposal(p.currentHeight, 0, -1, bid, proposedTime)
  175. tp := prop.ToProto()
  176. if err := proposer.SignProposal(ctx, p.observedState.state.ChainID, tp); err != nil {
  177. t.Fatalf("error signing proposal: %s", err)
  178. }
  179. time.Sleep(time.Until(deliverTime))
  180. prop.Signature = tp.Signature
  181. if err := p.observedState.SetProposalAndBlock(ctx, prop, b, ps, "peerID"); err != nil {
  182. t.Fatal(err)
  183. }
  184. ensureProposal(t, p.ensureProposalCh, p.currentHeight, 0, bid)
  185. ensurePrevote(t, p.ensureVoteCh, p.currentHeight, p.currentRound)
  186. signAddVotes(ctx, t, p.observedState, tmproto.PrevoteType, p.chainID, bid, p.otherValidators...)
  187. signAddVotes(ctx, t, p.observedState, tmproto.PrecommitType, p.chainID, bid, p.otherValidators...)
  188. ensurePrecommit(t, p.ensureVoteCh, p.currentHeight, p.currentRound)
  189. vk, err := p.observedValidator.GetPubKey(ctx)
  190. require.NoError(t, err)
  191. res := collectHeightResults(ctx, t, p.eventCh, p.currentHeight, vk.Address())
  192. ensureNewBlock(t, p.blockCh, p.currentHeight)
  193. p.currentHeight++
  194. incrementHeight(p.otherValidators...)
  195. return res
  196. }
  197. func timestampedCollector(ctx context.Context, t *testing.T, eb *eventbus.EventBus) <-chan timestampedEvent {
  198. t.Helper()
  199. // Since eventCh is not read until the end of each height, it must be large
  200. // enough to hold all of the events produced during a single height.
  201. eventCh := make(chan timestampedEvent, 100)
  202. if err := eb.Observe(ctx, func(msg tmpubsub.Message) error {
  203. eventCh <- timestampedEvent{
  204. ts: time.Now(),
  205. m: msg,
  206. }
  207. return nil
  208. }, types.EventQueryVote, types.EventQueryCompleteProposal); err != nil {
  209. t.Fatalf("Failed to observe query %v: %v", types.EventQueryVote, err)
  210. }
  211. return eventCh
  212. }
  213. func collectHeightResults(ctx context.Context, t *testing.T, eventCh <-chan timestampedEvent, height int64, address []byte) heightResult {
  214. t.Helper()
  215. var res heightResult
  216. for event := range eventCh {
  217. switch v := event.m.Data().(type) {
  218. case types.EventDataVote:
  219. if v.Vote.Height > height {
  220. t.Fatalf("received prevote from unexpected height, expected: %d, saw: %d", height, v.Vote.Height)
  221. }
  222. if !bytes.Equal(address, v.Vote.ValidatorAddress) {
  223. continue
  224. }
  225. if v.Vote.Type != tmproto.PrevoteType {
  226. continue
  227. }
  228. res.prevote = v.Vote
  229. res.prevoteIssuedAt = event.ts
  230. case types.EventDataCompleteProposal:
  231. if v.Height > height {
  232. t.Fatalf("received proposal from unexpected height, expected: %d, saw: %d", height, v.Height)
  233. }
  234. res.proposalIssuedAt = event.ts
  235. }
  236. if res.isComplete() {
  237. return res
  238. }
  239. }
  240. t.Fatalf("complete height result never seen for height %d", height)
  241. panic("unreachable")
  242. }
  243. type timestampedEvent struct {
  244. ts time.Time
  245. m tmpubsub.Message
  246. }
  247. func (p *pbtsTestHarness) run(ctx context.Context, t *testing.T) resultSet {
  248. startTestRound(ctx, p.observedState, p.currentHeight, p.currentRound)
  249. r1, proposalBlockTime := p.observedValidatorProposerHeight(ctx, t, p.genesisTime)
  250. p.firstBlockTime = proposalBlockTime
  251. r2 := p.height2(ctx, t)
  252. p.intermediateHeights(ctx, t)
  253. r5, _ := p.height5(ctx, t)
  254. return resultSet{
  255. genesisHeight: r1,
  256. height2: r2,
  257. height5: r5,
  258. }
  259. }
  260. type resultSet struct {
  261. genesisHeight heightResult
  262. height2 heightResult
  263. height5 heightResult
  264. }
  265. type heightResult struct {
  266. proposalIssuedAt time.Time
  267. prevote *types.Vote
  268. prevoteIssuedAt time.Time
  269. }
  270. func (hr heightResult) isComplete() bool {
  271. return !hr.proposalIssuedAt.IsZero() && !hr.prevoteIssuedAt.IsZero() && hr.prevote != nil
  272. }
  273. // TestProposerWaitsForGenesisTime tests that a proposer will not propose a block
  274. // until after the genesis time has passed. The test sets the genesis time in the
  275. // future and then ensures that the observed validator waits to propose a block.
  276. func TestProposerWaitsForGenesisTime(t *testing.T) {
  277. ctx, cancel := context.WithCancel(context.Background())
  278. defer cancel()
  279. // create a genesis time far (enough) in the future.
  280. initialTime := time.Now().Add(800 * time.Millisecond)
  281. cfg := pbtsTestConfiguration{
  282. synchronyParams: types.SynchronyParams{
  283. Precision: 10 * time.Millisecond,
  284. MessageDelay: 10 * time.Millisecond,
  285. },
  286. timeoutPropose: 10 * time.Millisecond,
  287. genesisTime: initialTime,
  288. height2ProposalTimeDeliveryOffset: 10 * time.Millisecond,
  289. height2ProposedBlockOffset: 10 * time.Millisecond,
  290. height4ProposedBlockOffset: 30 * time.Millisecond,
  291. }
  292. pbtsTest := newPBTSTestHarness(ctx, t, cfg)
  293. results := pbtsTest.run(ctx, t)
  294. // ensure that the proposal was issued after the genesis time.
  295. assert.True(t, results.genesisHeight.proposalIssuedAt.After(cfg.genesisTime))
  296. }
  297. // TestProposerWaitsForPreviousBlock tests that the proposer of a block waits until
  298. // the block time of the previous height has passed to propose the next block.
  299. // The test harness ensures that the observed validator will be the proposer at
  300. // height 1 and height 5. The test sets the block time of height 4 in the future
  301. // and then verifies that the observed validator waits until after the block time
  302. // of height 4 to propose a block at height 5.
  303. func TestProposerWaitsForPreviousBlock(t *testing.T) {
  304. ctx, cancel := context.WithCancel(context.Background())
  305. defer cancel()
  306. initialTime := time.Now().Add(time.Millisecond * 50)
  307. cfg := pbtsTestConfiguration{
  308. synchronyParams: types.SynchronyParams{
  309. Precision: 100 * time.Millisecond,
  310. MessageDelay: 500 * time.Millisecond,
  311. },
  312. timeoutPropose: 50 * time.Millisecond,
  313. genesisTime: initialTime,
  314. height2ProposalTimeDeliveryOffset: 150 * time.Millisecond,
  315. height2ProposedBlockOffset: 100 * time.Millisecond,
  316. height4ProposedBlockOffset: 800 * time.Millisecond,
  317. }
  318. pbtsTest := newPBTSTestHarness(ctx, t, cfg)
  319. results := pbtsTest.run(ctx, t)
  320. // the observed validator is the proposer at height 5.
  321. // ensure that the observed validator did not propose a block until after
  322. // the time configured for height 4.
  323. assert.True(t, results.height5.proposalIssuedAt.After(pbtsTest.firstBlockTime.Add(cfg.height4ProposedBlockOffset)))
  324. // Ensure that the validator issued a prevote for a non-nil block.
  325. assert.NotNil(t, results.height5.prevote.BlockID.Hash)
  326. }
  327. func TestProposerWaitTime(t *testing.T) {
  328. genesisTime, err := time.Parse(time.RFC3339, "2019-03-13T23:00:00Z")
  329. require.NoError(t, err)
  330. testCases := []struct {
  331. name string
  332. previousBlockTime time.Time
  333. localTime time.Time
  334. expectedWait time.Duration
  335. }{
  336. {
  337. name: "block time greater than local time",
  338. previousBlockTime: genesisTime.Add(5 * time.Nanosecond),
  339. localTime: genesisTime.Add(1 * time.Nanosecond),
  340. expectedWait: 4 * time.Nanosecond,
  341. },
  342. {
  343. name: "local time greater than block time",
  344. previousBlockTime: genesisTime.Add(1 * time.Nanosecond),
  345. localTime: genesisTime.Add(5 * time.Nanosecond),
  346. expectedWait: 0,
  347. },
  348. {
  349. name: "both times equal",
  350. previousBlockTime: genesisTime.Add(5 * time.Nanosecond),
  351. localTime: genesisTime.Add(5 * time.Nanosecond),
  352. expectedWait: 0,
  353. },
  354. }
  355. for _, testCase := range testCases {
  356. t.Run(testCase.name, func(t *testing.T) {
  357. mockSource := new(tmtimemocks.Source)
  358. mockSource.On("Now").Return(testCase.localTime)
  359. ti := proposerWaitTime(mockSource, testCase.previousBlockTime)
  360. assert.Equal(t, testCase.expectedWait, ti)
  361. })
  362. }
  363. }
  364. func TestTimelyProposal(t *testing.T) {
  365. ctx, cancel := context.WithCancel(context.Background())
  366. defer cancel()
  367. initialTime := time.Now()
  368. cfg := pbtsTestConfiguration{
  369. synchronyParams: types.SynchronyParams{
  370. Precision: 10 * time.Millisecond,
  371. MessageDelay: 140 * time.Millisecond,
  372. },
  373. timeoutPropose: 40 * time.Millisecond,
  374. genesisTime: initialTime,
  375. height2ProposedBlockOffset: 15 * time.Millisecond,
  376. height2ProposalTimeDeliveryOffset: 30 * time.Millisecond,
  377. }
  378. pbtsTest := newPBTSTestHarness(ctx, t, cfg)
  379. results := pbtsTest.run(ctx, t)
  380. require.NotNil(t, results.height2.prevote.BlockID.Hash)
  381. }
  382. func TestTooFarInThePastProposal(t *testing.T) {
  383. ctx, cancel := context.WithCancel(context.Background())
  384. defer cancel()
  385. // localtime > proposedBlockTime + MsgDelay + Precision
  386. cfg := pbtsTestConfiguration{
  387. synchronyParams: types.SynchronyParams{
  388. Precision: 1 * time.Millisecond,
  389. MessageDelay: 10 * time.Millisecond,
  390. },
  391. timeoutPropose: 50 * time.Millisecond,
  392. height2ProposedBlockOffset: 15 * time.Millisecond,
  393. height2ProposalTimeDeliveryOffset: 27 * time.Millisecond,
  394. }
  395. pbtsTest := newPBTSTestHarness(ctx, t, cfg)
  396. results := pbtsTest.run(ctx, t)
  397. require.Nil(t, results.height2.prevote.BlockID.Hash)
  398. }
  399. func TestTooFarInTheFutureProposal(t *testing.T) {
  400. ctx, cancel := context.WithCancel(context.Background())
  401. defer cancel()
  402. // localtime < proposedBlockTime - Precision
  403. cfg := pbtsTestConfiguration{
  404. synchronyParams: types.SynchronyParams{
  405. Precision: 1 * time.Millisecond,
  406. MessageDelay: 10 * time.Millisecond,
  407. },
  408. timeoutPropose: 50 * time.Millisecond,
  409. height2ProposedBlockOffset: 100 * time.Millisecond,
  410. height2ProposalTimeDeliveryOffset: 10 * time.Millisecond,
  411. height4ProposedBlockOffset: 150 * time.Millisecond,
  412. }
  413. pbtsTest := newPBTSTestHarness(ctx, t, cfg)
  414. results := pbtsTest.run(ctx, t)
  415. require.Nil(t, results.height2.prevote.BlockID.Hash)
  416. }