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.

1182 lines
41 KiB

10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
10 years ago
  1. /*
  2. Consensus State Machine Overview:
  3. NewHeight, NewRound, Propose, Prevote, Precommit represent state machine steps. (aka RoundStep).
  4. To "prevote/precommit" something means to broadcast a prevote/precommit vote for something.
  5. During NewHeight/NewRound/Propose/Prevote/Precommit:
  6. * Nodes gossip the proposal block proposed by the designated proposer at round.
  7. * Nodes gossip prevotes/precommits at rounds [0...currentRound+1] (currentRound+1 to allow round-skipping)
  8. * Nodes gossip prevotes for the proposal's POL (proof-of-lock) round if proposed.
  9. * Nodes gossip to late nodes (lagging in height) with precommits of the commit round (aka catchup)
  10. Upon each state transition, the height/round/step is broadcast to neighboring peers.
  11. The set of +2/3 of precommits at the same round for the same block is called a Commit, or Validation.
  12. A block contains the last block's Validation, which includes the Commit precommits.
  13. While all the precommits in the Validation are from the same height & round (ordered by validator index),
  14. some precommits may be <nil> (if the validator's precommit vote didn't reach the proposer in time),
  15. or some precommits may be for different blockhashes for the last block hash (which is fine).
  16. Each unlock/change-of-lock should be justifiable by an POL where +2/3 prevoted for
  17. some block or <nil> at some round.
  18. POL = Proof-of-Lock = +2/3 prevotes for block B (or +2/3 prevotes for <nil>) at (H,R)
  19. lockRound < POLRound <= unlockOrChangeLockRound
  20. Without the POLRound <= unlockOrChangeLockRound condition, an unlock would be possible from a
  21. future condition that hasn't happened yet, so it destroys deterministic accountability.
  22. With lockRound < POLRound <= unlockOrChangeLockRound, blame can be shifted to lower rounds.
  23. * NewRound(height:H,round:R):
  24. * Set up new round. --> goto Propose(H,R)
  25. * NOTE: Not much happens in this step. It exists for clarity.
  26. * Propose(height:H,round:R):
  27. * Upon entering Propose:
  28. * The designated proposer proposes a block at (H,R).
  29. * The Propose step ends:
  30. * After `timeoutPropose` after entering Propose. --> goto Prevote(H,R)
  31. * After receiving proposal block and all POL prevotes. --> goto Prevote(H,R)
  32. * After any +2/3 prevotes received at (H,R+1). --> goto Prevote(H,R+1)
  33. * After any +2/3 precommits received at (H,R+1). --> goto Precommit(H,R+1)
  34. * After +2/3 precommits received for a particular block. --> goto Commit(H)
  35. * Prevote(height:H,round:R):
  36. * Upon entering Prevote, each validator broadcasts its prevote vote.
  37. * If the validator is locked on a block, it prevotes that.
  38. * Else, if the proposed block from Propose(H,R) is good, it prevotes that.
  39. * Else, if the proposal is invalid or wasn't received on time, it prevotes <nil>.
  40. * The Prevote step ends:
  41. * After +2/3 prevotes for a particular block or <nil>. --> goto Precommit(H,R)
  42. * After `timeoutPrevote` after receiving any +2/3 prevotes. --> goto Precommit(H,R)
  43. * After any +2/3 prevotes received at (H,R+1). --> goto Prevote(H,R+1)
  44. * After any +2/3 precommits received at (H,R+1). --> goto Precommit(H,R+1)
  45. * After +2/3 precommits received for a particular block. --> goto Commit(H)
  46. * Precommit(height:H,round:R):
  47. * Upon entering Precommit, each validator broadcasts its precommit vote.
  48. * If the validator had seen +2/3 of prevotes for a particular block from Prevote(H,R),
  49. it locks (changes lock to) that block and precommits that block.
  50. * Else, if the validator had seen +2/3 of prevotes for <nil>, it unlocks and precommits <nil>.
  51. * Else, if +2/3 of prevotes for a particular block or <nil> is not received on time,
  52. it precommits what it's locked on, or <nil>.
  53. * The Precommit step ends:
  54. * After +2/3 precommits for a particular block. --> goto Commit(H)
  55. * After +2/3 precommits for <nil>. --> goto NewRound(H,R+1)
  56. * After `timeoutPrecommit` after receiving any +2/3 precommits. --> goto NewRound(H,R+1)
  57. * After any +2/3 prevotes received at (H,R+1). --> goto Prevote(H,R+1)
  58. * After any +2/3 precommits received at (H,R+1). --> goto Precommit(H,R+1)
  59. * Commit(height:H):
  60. * Set CommitTime = now
  61. * Wait until block is received. --> goto NewHeight(H+1)
  62. * NewHeight(height:H):
  63. * Move Precommits to LastCommit and increment height.
  64. * Set StartTime = CommitTime+timeoutCommit
  65. * Wait until `StartTime` to receive straggler commits. --> goto NewRound(H,0)
  66. * Proof of Safety:
  67. If a good validator commits at round R, it's because it saw +2/3 of precommits at round R.
  68. This implies that (assuming tolerance bounds) +1/3 of honest nodes are still locked at round R+1.
  69. These locked validators will remain locked until they see +2/3 prevote for something
  70. else, but this won't happen because +1/3 are locked and honest.
  71. * Proof of Liveness:
  72. Lemma 1: If +1/3 good nodes are locked on two different blocks, the proposers' POLRound will
  73. eventually cause nodes locked from the earlier round to unlock.
  74. -> `timeoutProposalR` increments with round R, while the block.size && POL prevote size
  75. are fixed, so eventually we'll be able to "fully gossip" the block & POL.
  76. TODO: cap the block.size at something reasonable.
  77. Lemma 2: If a good node is at round R, neighboring good nodes will soon catch up to round R.
  78. Lemma 3: If a node at (H,R) receives +2/3 prevotes for a block (or +2/3 for <nil>) at (H,R+1),
  79. it will enter NewRound(H,R+1).
  80. Lemma 4: Terminal conditions imply the existence of deterministic accountability, for
  81. a synchronous (fixed-duration) protocol extension (judgement).
  82. TODO: define terminal conditions (fork and non-decision).
  83. TODO: define new assumptions for the synchronous judgement period.
  84. +-------------------------------------+
  85. v |(Wait til `CommmitTime+timeoutCommit`)
  86. +-----------+ +-----+-----+
  87. +----------> | Propose +--------------+ | NewHeight |
  88. | +-----------+ | +-----------+
  89. | | ^
  90. |(Else, after timeoutPrecommit) v |
  91. +-----+-----+ +-----------+ |
  92. | Precommit | <------------------------+ Prevote | |
  93. +-----+-----+ +-----------+ |
  94. |(When +2/3 Precommits for block found) |
  95. v |
  96. +--------------------------------------------------------------------+
  97. | Commit |
  98. | |
  99. | * Set CommitTime = now; |
  100. | * Wait for block, then stage/save/commit block; |
  101. +--------------------------------------------------------------------+
  102. */
  103. package consensus
  104. import (
  105. "bytes"
  106. "errors"
  107. "fmt"
  108. "sync"
  109. "sync/atomic"
  110. "time"
  111. "github.com/tendermint/tendermint/account"
  112. "github.com/tendermint/tendermint/binary"
  113. bc "github.com/tendermint/tendermint/blockchain"
  114. . "github.com/tendermint/tendermint/common"
  115. . "github.com/tendermint/tendermint/consensus/types"
  116. "github.com/tendermint/tendermint/events"
  117. mempl "github.com/tendermint/tendermint/mempool"
  118. sm "github.com/tendermint/tendermint/state"
  119. "github.com/tendermint/tendermint/types"
  120. )
  121. var (
  122. timeoutPropose = 3000 * time.Millisecond // Maximum duration of RoundStepPropose
  123. timeoutPrevote0 = 1000 * time.Millisecond // After any +2/3 prevotes received, wait this long for stragglers.
  124. timeoutPrevoteDelta = 0500 * time.Millisecond // timeoutPrevoteN is timeoutPrevote0 + timeoutPrevoteDelta*N
  125. timeoutPrecommit0 = 1000 * time.Millisecond // After any +2/3 precommits received, wait this long for stragglers.
  126. timeoutPrecommitDelta = 0500 * time.Millisecond // timeoutPrecommitN is timeoutPrecommit0 + timeoutPrecommitDelta*N
  127. timeoutCommit = 2000 * time.Millisecond // After +2/3 commits received for committed block, wait this long for stragglers in the next height's RoundStepNewHeight.
  128. )
  129. var (
  130. ErrInvalidProposalSignature = errors.New("Error invalid proposal signature")
  131. ErrInvalidProposalPOLRound = errors.New("Error invalid proposal POL round")
  132. )
  133. //-----------------------------------------------------------------------------
  134. // RoundStepType enum type
  135. type RoundStepType uint8 // These must be numeric, ordered.
  136. const (
  137. RoundStepNewHeight = RoundStepType(0x01) // Wait til CommitTime + timeoutCommit
  138. RoundStepNewRound = RoundStepType(0x02) // Setup new round and go to RoundStepPropose
  139. RoundStepPropose = RoundStepType(0x03) // Did propose, gossip proposal
  140. RoundStepPrevote = RoundStepType(0x04) // Did prevote, gossip prevotes
  141. RoundStepPrevoteWait = RoundStepType(0x05) // Did receive any +2/3 prevotes, start timeout
  142. RoundStepPrecommit = RoundStepType(0x06) // Did precommit, gossip precommits
  143. RoundStepPrecommitWait = RoundStepType(0x07) // Did receive any +2/3 precommits, start timeout
  144. RoundStepCommit = RoundStepType(0x08) // Entered commit state machine
  145. // NOTE: RoundStepNewHeight acts as RoundStepCommitWait.
  146. )
  147. func (rs RoundStepType) String() string {
  148. switch rs {
  149. case RoundStepNewHeight:
  150. return "RoundStepNewHeight"
  151. case RoundStepNewRound:
  152. return "RoundStepNewRound"
  153. case RoundStepPropose:
  154. return "RoundStepPropose"
  155. case RoundStepPrevote:
  156. return "RoundStepPrevote"
  157. case RoundStepPrevoteWait:
  158. return "RoundStepPrevoteWait"
  159. case RoundStepPrecommit:
  160. return "RoundStepPrecommit"
  161. case RoundStepPrecommitWait:
  162. return "RoundStepPrecommitWait"
  163. case RoundStepCommit:
  164. return "RoundStepCommit"
  165. default:
  166. panic(Fmt("Unknown RoundStep %X", rs))
  167. }
  168. }
  169. //-----------------------------------------------------------------------------
  170. // Immutable when returned from ConsensusState.GetRoundState()
  171. type RoundState struct {
  172. Height int // Height we are working on
  173. Round int
  174. Step RoundStepType
  175. StartTime time.Time
  176. CommitTime time.Time // Subjective time when +2/3 precommits for Block at Round were found
  177. Validators *sm.ValidatorSet
  178. Proposal *Proposal
  179. ProposalBlock *types.Block
  180. ProposalBlockParts *types.PartSet
  181. LockedRound int
  182. LockedBlock *types.Block
  183. LockedBlockParts *types.PartSet
  184. Votes *HeightVoteSet
  185. LastCommit *VoteSet // Last precommits at Height-1
  186. LastValidators *sm.ValidatorSet
  187. }
  188. func (rs *RoundState) String() string {
  189. return rs.StringIndented("")
  190. }
  191. func (rs *RoundState) StringIndented(indent string) string {
  192. return fmt.Sprintf(`RoundState{
  193. %s H:%v R:%v S:%v
  194. %s StartTime: %v
  195. %s CommitTime: %v
  196. %s Validators: %v
  197. %s Proposal: %v
  198. %s ProposalBlock: %v %v
  199. %s LockedRound: %v
  200. %s LockedBlock: %v %v
  201. %s Votes: %v
  202. %s LastCommit: %v
  203. %s LastValidators: %v
  204. %s}`,
  205. indent, rs.Height, rs.Round, rs.Step,
  206. indent, rs.StartTime,
  207. indent, rs.CommitTime,
  208. indent, rs.Validators.StringIndented(indent+" "),
  209. indent, rs.Proposal,
  210. indent, rs.ProposalBlockParts.StringShort(), rs.ProposalBlock.StringShort(),
  211. indent, rs.LockedRound,
  212. indent, rs.LockedBlockParts.StringShort(), rs.LockedBlock.StringShort(),
  213. indent, rs.Votes.StringIndented(indent+" "),
  214. indent, rs.LastCommit.StringShort(),
  215. indent, rs.LastValidators.StringIndented(indent+" "),
  216. indent)
  217. }
  218. func (rs *RoundState) StringShort() string {
  219. return fmt.Sprintf(`RoundState{H:%v R:%v S:%v ST:%v}`,
  220. rs.Height, rs.Round, rs.Step, rs.StartTime)
  221. }
  222. //-----------------------------------------------------------------------------
  223. // Tracks consensus state across block heights and rounds.
  224. type ConsensusState struct {
  225. started uint32
  226. stopped uint32
  227. quit chan struct{}
  228. blockStore *bc.BlockStore
  229. mempoolReactor *mempl.MempoolReactor
  230. privValidator *sm.PrivValidator
  231. newStepCh chan *RoundState
  232. mtx sync.Mutex
  233. RoundState
  234. state *sm.State // State until height-1.
  235. stagedBlock *types.Block // Cache last staged block.
  236. stagedState *sm.State // Cache result of staged block.
  237. evsw events.Fireable
  238. evc *events.EventCache // set in stageBlock and passed into state
  239. }
  240. func NewConsensusState(state *sm.State, blockStore *bc.BlockStore, mempoolReactor *mempl.MempoolReactor) *ConsensusState {
  241. cs := &ConsensusState{
  242. quit: make(chan struct{}),
  243. blockStore: blockStore,
  244. mempoolReactor: mempoolReactor,
  245. newStepCh: make(chan *RoundState, 10),
  246. }
  247. cs.updateToState(state, true)
  248. // Don't call scheduleRound0 yet.
  249. // We do that upon Start().
  250. cs.maybeRebond()
  251. cs.reconstructLastCommit(state)
  252. return cs
  253. }
  254. // Reconstruct LastCommit from SeenValidation, which we saved along with the block,
  255. // (which happens even before saving the state)
  256. func (cs *ConsensusState) reconstructLastCommit(state *sm.State) {
  257. if state.LastBlockHeight == 0 {
  258. return
  259. }
  260. lastPrecommits := NewVoteSet(state.LastBlockHeight, 0, types.VoteTypePrecommit, state.LastBondedValidators)
  261. seenValidation := cs.blockStore.LoadSeenValidation(state.LastBlockHeight)
  262. for idx, precommit := range seenValidation.Precommits {
  263. added, _, err := lastPrecommits.AddByIndex(idx, precommit)
  264. if !added || err != nil {
  265. panic(Fmt("Failed to reconstruct LastCommit: %v", err))
  266. }
  267. }
  268. if !lastPrecommits.HasTwoThirdsMajority() {
  269. panic("Failed to reconstruct LastCommit: Does not have +2/3 maj")
  270. }
  271. cs.LastCommit = lastPrecommits
  272. }
  273. func (cs *ConsensusState) GetState() *sm.State {
  274. cs.mtx.Lock()
  275. defer cs.mtx.Unlock()
  276. return cs.state.Copy()
  277. }
  278. func (cs *ConsensusState) GetRoundState() *RoundState {
  279. cs.mtx.Lock()
  280. defer cs.mtx.Unlock()
  281. return cs.getRoundState()
  282. }
  283. func (cs *ConsensusState) getRoundState() *RoundState {
  284. rs := cs.RoundState // copy
  285. return &rs
  286. }
  287. func (cs *ConsensusState) NewStepCh() chan *RoundState {
  288. return cs.newStepCh
  289. }
  290. func (cs *ConsensusState) Start() {
  291. if atomic.CompareAndSwapUint32(&cs.started, 0, 1) {
  292. log.Info("Starting ConsensusState")
  293. cs.scheduleRound0(cs.Height)
  294. }
  295. }
  296. // EnterNewRound(height, 0) at cs.StartTime.
  297. func (cs *ConsensusState) scheduleRound0(height int) {
  298. //log.Debug("scheduleRound0", "now", time.Now(), "startTime", cs.StartTime)
  299. sleepDuration := cs.StartTime.Sub(time.Now())
  300. go func() {
  301. if 0 < sleepDuration {
  302. time.Sleep(sleepDuration)
  303. }
  304. cs.EnterNewRound(height, 0)
  305. }()
  306. }
  307. func (cs *ConsensusState) Stop() {
  308. if atomic.CompareAndSwapUint32(&cs.stopped, 0, 1) {
  309. log.Info("Stopping ConsensusState")
  310. close(cs.quit)
  311. }
  312. }
  313. func (cs *ConsensusState) IsStopped() bool {
  314. return atomic.LoadUint32(&cs.stopped) == 1
  315. }
  316. // Updates ConsensusState and increments height to match that of state.
  317. // The round becomes 0 and cs.Step becomes RoundStepNewHeight.
  318. func (cs *ConsensusState) updateToState(state *sm.State, contiguous bool) {
  319. // SANITY CHECK
  320. if contiguous && 0 < cs.Height && cs.Height != state.LastBlockHeight {
  321. panic(Fmt("updateToState() expected state height of %v but found %v",
  322. cs.Height, state.LastBlockHeight))
  323. }
  324. // END SANITY CHECK
  325. // Reset fields based on state.
  326. validators := state.BondedValidators
  327. height := state.LastBlockHeight + 1 // next desired block height
  328. lastPrecommits := (*VoteSet)(nil)
  329. if contiguous && cs.Votes != nil {
  330. if !cs.Votes.Precommits(cs.Round).HasTwoThirdsMajority() {
  331. panic("updateToState(state, true) called but last Precommit round didn't have +2/3")
  332. }
  333. lastPrecommits = cs.Votes.Precommits(cs.Round)
  334. }
  335. // RoundState fields
  336. cs.Height = height
  337. cs.Round = 0
  338. cs.Step = RoundStepNewHeight
  339. if cs.CommitTime.IsZero() {
  340. // "Now" makes it easier to sync up dev nodes.
  341. // We add timeoutCommit to allow transactions
  342. // to be gathered for the first block.
  343. // And alternative solution that relies on clocks:
  344. // cs.StartTime = state.LastBlockTime.Add(timeoutCommit)
  345. cs.StartTime = time.Now().Add(timeoutCommit)
  346. } else {
  347. cs.StartTime = cs.CommitTime.Add(timeoutCommit)
  348. }
  349. cs.CommitTime = time.Time{}
  350. cs.Validators = validators
  351. cs.Proposal = nil
  352. cs.ProposalBlock = nil
  353. cs.ProposalBlockParts = nil
  354. cs.LockedRound = 0
  355. cs.LockedBlock = nil
  356. cs.LockedBlockParts = nil
  357. cs.Votes = NewHeightVoteSet(height, validators)
  358. cs.LastCommit = lastPrecommits
  359. cs.LastValidators = state.LastBondedValidators
  360. cs.state = state
  361. cs.stagedBlock = nil
  362. cs.stagedState = nil
  363. // Finally, broadcast RoundState
  364. cs.newStepCh <- cs.getRoundState()
  365. }
  366. // If we're unbonded, broadcast RebondTx.
  367. func (cs *ConsensusState) maybeRebond() {
  368. if cs.privValidator == nil || !cs.state.UnbondingValidators.HasAddress(cs.privValidator.Address) {
  369. return
  370. }
  371. rebondTx := &types.RebondTx{
  372. Address: cs.privValidator.Address,
  373. Height: cs.Height,
  374. }
  375. err := cs.privValidator.SignRebondTx(cs.state.ChainID, rebondTx)
  376. if err == nil {
  377. err := cs.mempoolReactor.BroadcastTx(rebondTx)
  378. if err != nil {
  379. log.Error("Failed to broadcast RebondTx",
  380. "height", cs.Height, "round", cs.Round, "tx", rebondTx, "error", err)
  381. } else {
  382. log.Info("Signed and broadcast RebondTx",
  383. "height", cs.Height, "round", cs.Round, "tx", rebondTx)
  384. }
  385. } else {
  386. log.Warn("Error signing RebondTx", "height", cs.Height, "round", cs.Round, "tx", rebondTx, "error", err)
  387. }
  388. }
  389. func (cs *ConsensusState) SetPrivValidator(priv *sm.PrivValidator) {
  390. cs.mtx.Lock()
  391. defer cs.mtx.Unlock()
  392. cs.privValidator = priv
  393. }
  394. //-----------------------------------------------------------------------------
  395. // Enter: +2/3 precommits for nil at (height,round-1)
  396. // Enter: `timeoutPrecommits` after any +2/3 precommits from (height,round-1)
  397. // Enter: `startTime = commitTime+timeoutCommit` from NewHeight(height)
  398. // NOTE: cs.StartTime was already set for height.
  399. func (cs *ConsensusState) EnterNewRound(height int, round int) {
  400. cs.mtx.Lock()
  401. defer cs.mtx.Unlock()
  402. if cs.Height != height || round < cs.Round || (cs.Round == round && cs.Step != RoundStepNewHeight) {
  403. log.Debug(Fmt("EnterNewRound(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
  404. return
  405. }
  406. if now := time.Now(); cs.StartTime.After(now) {
  407. log.Warn("Need to set a buffer and log.Warn() here for sanity.", "startTime", cs.StartTime, "now", now)
  408. }
  409. // Increment validators if necessary
  410. validators := cs.Validators
  411. if cs.Round < round {
  412. validators = validators.Copy()
  413. validators.IncrementAccum(round - cs.Round)
  414. }
  415. // Setup new round
  416. cs.Round = round
  417. cs.Step = RoundStepNewRound
  418. cs.Validators = validators
  419. cs.Proposal = nil
  420. cs.ProposalBlock = nil
  421. cs.ProposalBlockParts = nil
  422. cs.Votes.SetRound(round + 1) // also track next round (round+1) to allow round-skipping
  423. // Immediately go to EnterPropose.
  424. go cs.EnterPropose(height, round)
  425. }
  426. // Enter: from NewRound(height,round).
  427. func (cs *ConsensusState) EnterPropose(height int, round int) {
  428. cs.mtx.Lock()
  429. defer cs.mtx.Unlock()
  430. if cs.Height != height || round < cs.Round || (cs.Round == round && RoundStepPropose <= cs.Step) {
  431. log.Debug(Fmt("EnterPropose(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
  432. return
  433. }
  434. defer func() {
  435. // Done EnterPropose:
  436. cs.Round = round
  437. cs.Step = RoundStepPropose
  438. cs.newStepCh <- cs.getRoundState()
  439. // If we already have the proposal + POL, then goto Prevote
  440. if cs.isProposalComplete() {
  441. go cs.EnterPrevote(height, cs.Round)
  442. }
  443. }()
  444. // This step times out after `timeoutPropose`
  445. go func() {
  446. time.Sleep(timeoutPropose)
  447. cs.EnterPrevote(height, round)
  448. }()
  449. // Nothing more to do if we're not a validator
  450. if cs.privValidator == nil {
  451. return
  452. }
  453. if !bytes.Equal(cs.Validators.Proposer().Address, cs.privValidator.Address) {
  454. log.Debug("EnterPropose: Not our turn to propose", "proposer", cs.Validators.Proposer().Address, "privValidator", cs.privValidator)
  455. } else {
  456. log.Debug("EnterPropose: Our turn to propose", "proposer", cs.Validators.Proposer().Address, "privValidator", cs.privValidator)
  457. cs.decideProposal(height, round)
  458. }
  459. }
  460. // Decides on the next proposal and sets them onto cs.Proposal*
  461. func (cs *ConsensusState) decideProposal(height int, round int) {
  462. var block *types.Block
  463. var blockParts *types.PartSet
  464. // Decide on block
  465. if cs.LockedBlock != nil {
  466. // If we're locked onto a block, just choose that.
  467. block, blockParts = cs.LockedBlock, cs.LockedBlockParts
  468. } else {
  469. // Create a new proposal block from state/txs from the mempool.
  470. block, blockParts = cs.createProposalBlock()
  471. }
  472. // Make proposal
  473. proposal := NewProposal(height, round, blockParts.Header(), cs.Votes.POLRound())
  474. err := cs.privValidator.SignProposal(cs.state.ChainID, proposal)
  475. if err == nil {
  476. log.Info("Signed and set proposal", "height", height, "round", round, "proposal", proposal)
  477. log.Debug(Fmt("Signed and set proposal block: %v", block))
  478. // Set fields
  479. cs.Proposal = proposal
  480. cs.ProposalBlock = block
  481. cs.ProposalBlockParts = blockParts
  482. } else {
  483. log.Warn("EnterPropose: Error signing proposal", "height", height, "round", round, "error", err)
  484. }
  485. }
  486. // Returns true if the proposal block is complete &&
  487. // (if POLRound was proposed, we have +2/3 prevotes from there).
  488. func (cs *ConsensusState) isProposalComplete() bool {
  489. if cs.Proposal == nil || cs.ProposalBlock == nil {
  490. return false
  491. }
  492. if cs.Proposal.POLRound < 0 {
  493. return true
  494. } else {
  495. return cs.Votes.Prevotes(cs.Proposal.POLRound).HasTwoThirdsMajority()
  496. }
  497. }
  498. // Create the next block to propose and return it.
  499. // NOTE: make it side-effect free for clarity.
  500. func (cs *ConsensusState) createProposalBlock() (block *types.Block, blockParts *types.PartSet) {
  501. var validation *types.Validation
  502. if cs.Height == 1 {
  503. // We're creating a proposal for the first block.
  504. // The validation is empty, but not nil.
  505. validation = &types.Validation{}
  506. } else if cs.LastCommit.HasTwoThirdsMajority() {
  507. // Make the validation from LastCommit
  508. validation = cs.LastCommit.MakeValidation()
  509. } else {
  510. // This shouldn't happen.
  511. log.Error("EnterPropose: Cannot propose anything: No validation for the previous block.")
  512. return
  513. }
  514. txs := cs.mempoolReactor.Mempool.GetProposalTxs()
  515. block = &types.Block{
  516. Header: &types.Header{
  517. ChainID: cs.state.ChainID,
  518. Height: cs.Height,
  519. Time: time.Now(),
  520. Fees: 0, // TODO fees
  521. NumTxs: len(txs),
  522. LastBlockHash: cs.state.LastBlockHash,
  523. LastBlockParts: cs.state.LastBlockParts,
  524. StateHash: nil, // Will set afterwards.
  525. },
  526. LastValidation: validation,
  527. Data: &types.Data{
  528. Txs: txs,
  529. },
  530. }
  531. // Set the block.Header.StateHash.
  532. err := cs.state.ComputeBlockStateHash(block)
  533. if err != nil {
  534. log.Error("EnterPropose: Error setting state hash", "error", err)
  535. return
  536. }
  537. blockParts = block.MakePartSet()
  538. return block, blockParts
  539. }
  540. // Enter: `timeoutPropose` after entering Propose.
  541. // Enter: proposal block and POL is ready.
  542. // Enter: any +2/3 prevotes for future round.
  543. // Prevote for LockedBlock if we're locked, or ProposalBlock if valid.
  544. // Otherwise vote nil.
  545. func (cs *ConsensusState) EnterPrevote(height int, round int) {
  546. cs.mtx.Lock()
  547. defer cs.mtx.Unlock()
  548. if cs.Height != height || round < cs.Round || (cs.Round == round && RoundStepPrevote <= cs.Step) {
  549. log.Debug(Fmt("EnterPrevote(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
  550. return
  551. }
  552. // Sign and broadcast vote as necessary
  553. cs.doPrevote(height, round)
  554. // Done EnterPrevote:
  555. cs.Round = round
  556. cs.Step = RoundStepPrevote
  557. cs.newStepCh <- cs.getRoundState()
  558. /* This isn't necessary because addVote() does it for us.
  559. if cs.Votes.Prevotes(round).HasTwoThirdsAny() {
  560. go cs.EnterPrevoteWait(height, round)
  561. }*/
  562. }
  563. func (cs *ConsensusState) doPrevote(height int, round int) {
  564. // If a block is locked, prevote that.
  565. if cs.LockedBlock != nil {
  566. log.Debug("EnterPrevote: Block was locked")
  567. cs.signAddVote(types.VoteTypePrevote, cs.LockedBlock.Hash(), cs.LockedBlockParts.Header())
  568. return
  569. }
  570. // If ProposalBlock is nil, prevote nil.
  571. if cs.ProposalBlock == nil {
  572. log.Warn("EnterPrevote: ProposalBlock is nil")
  573. cs.signAddVote(types.VoteTypePrevote, nil, types.PartSetHeader{})
  574. return
  575. }
  576. // Try staging cs.ProposalBlock
  577. err := cs.stageBlock(cs.ProposalBlock, cs.ProposalBlockParts)
  578. if err != nil {
  579. // ProposalBlock is invalid, prevote nil.
  580. log.Warn("EnterPrevote: ProposalBlock is invalid", "error", err)
  581. cs.signAddVote(types.VoteTypePrevote, nil, types.PartSetHeader{})
  582. return
  583. }
  584. // Prevote cs.ProposalBlock
  585. cs.signAddVote(types.VoteTypePrevote, cs.ProposalBlock.Hash(), cs.ProposalBlockParts.Header())
  586. return
  587. }
  588. // Enter: any +2/3 prevotes at next round.
  589. func (cs *ConsensusState) EnterPrevoteWait(height int, round int) {
  590. cs.mtx.Lock()
  591. defer cs.mtx.Unlock()
  592. if cs.Height != height || round < cs.Round || (cs.Round == round && RoundStepPrevoteWait <= cs.Step) {
  593. log.Debug(Fmt("EnterPrevoteWait(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
  594. return
  595. }
  596. if !cs.Votes.Prevotes(round).HasTwoThirdsAny() {
  597. panic(Fmt("EnterPrevoteWait(%v/%v), but Prevotes does not have any +2/3 votes", height, round))
  598. }
  599. // Done EnterPrevoteWait:
  600. cs.Round = round
  601. cs.Step = RoundStepPrevoteWait
  602. cs.newStepCh <- cs.getRoundState()
  603. // After `timeoutPrevote0+timeoutPrevoteDelta*round`, EnterPrecommit()
  604. go func() {
  605. time.Sleep(timeoutPrevote0 + timeoutPrevote0*time.Duration(round))
  606. cs.EnterPrecommit(height, round)
  607. }()
  608. }
  609. // Enter: +2/3 precomits for block or nil.
  610. // Enter: `timeoutPrevote` after any +2/3 prevotes.
  611. // Enter: any +2/3 precommits for next round.
  612. // Lock & precommit the ProposalBlock if we have enough prevotes for it,
  613. // else, unlock an existing lock and precommit nil if +2/3 of prevotes were nil,
  614. // else, precommit locked block or nil otherwise.
  615. func (cs *ConsensusState) EnterPrecommit(height int, round int) {
  616. cs.mtx.Lock()
  617. defer cs.mtx.Unlock()
  618. if cs.Height != height || round < cs.Round || (cs.Round == round && RoundStepPrecommit <= cs.Step) {
  619. log.Debug(Fmt("EnterPrecommit(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
  620. return
  621. }
  622. defer func() {
  623. // Done EnterPrecommit:
  624. cs.Round = round
  625. cs.Step = RoundStepPrecommit
  626. cs.newStepCh <- cs.getRoundState()
  627. /* This isn't necessary because addVote() does it for us.
  628. if cs.Votes.Precommits(round).HasTwoThirdsAny() {
  629. go cs.EnterPrecommitWait(height, round)
  630. }*/
  631. }()
  632. hash, partsHeader, ok := cs.Votes.Prevotes(round).TwoThirdsMajority()
  633. // If we don't have two thirds of prevotes, just precommit locked block or nil
  634. if !ok {
  635. if cs.LockedBlock != nil {
  636. log.Info("EnterPrecommit: No +2/3 prevotes during EnterPrecommit. Precommitting lock.")
  637. cs.signAddVote(types.VoteTypePrecommit, cs.LockedBlock.Hash(), cs.LockedBlockParts.Header())
  638. } else {
  639. log.Info("EnterPrecommit: No +2/3 prevotes during EnterPrecommit. Precommitting nil.")
  640. cs.signAddVote(types.VoteTypePrecommit, nil, types.PartSetHeader{})
  641. }
  642. return
  643. }
  644. // +2/3 prevoted nil. Unlock and precommit nil.
  645. if len(hash) == 0 {
  646. if cs.LockedBlock == nil {
  647. log.Info("EnterPrecommit: +2/3 prevoted for nil.")
  648. } else {
  649. log.Info("EnterPrecommit: +2/3 prevoted for nil. Unlocking")
  650. cs.LockedRound = 0
  651. cs.LockedBlock = nil
  652. cs.LockedBlockParts = nil
  653. }
  654. cs.signAddVote(types.VoteTypePrecommit, nil, types.PartSetHeader{})
  655. return
  656. }
  657. // At this point, +2/3 prevoted for a particular block.
  658. // If +2/3 prevoted for already locked block, precommit it.
  659. if cs.LockedBlock.HashesTo(hash) {
  660. log.Info("EnterPrecommit: +2/3 prevoted locked block.")
  661. cs.signAddVote(types.VoteTypePrecommit, hash, partsHeader)
  662. return
  663. }
  664. // If +2/3 prevoted for proposal block, stage and precommit it
  665. if cs.ProposalBlock.HashesTo(hash) {
  666. log.Info("EnterPrecommit: +2/3 prevoted proposal block.")
  667. // Validate the block.
  668. if err := cs.stageBlock(cs.ProposalBlock, cs.ProposalBlockParts); err != nil {
  669. panic(Fmt("EnterPrecommit: +2/3 prevoted for an invalid block: %v", err))
  670. }
  671. cs.LockedRound = round
  672. cs.LockedBlock = cs.ProposalBlock
  673. cs.LockedBlockParts = cs.ProposalBlockParts
  674. cs.signAddVote(types.VoteTypePrecommit, hash, partsHeader)
  675. return
  676. }
  677. // Otherwise, we need to fetch the +2/3 prevoted block.
  678. // Unlock and precommit nil.
  679. // The +2/3 prevotes for this round is the POL for our unlock.
  680. if cs.Votes.POLRound() < round {
  681. panic(Fmt("This POLRound shold be %v but got %", round, cs.Votes.POLRound()))
  682. }
  683. cs.LockedRound = 0
  684. cs.LockedBlock = nil
  685. cs.LockedBlockParts = nil
  686. if !cs.ProposalBlockParts.HasHeader(partsHeader) {
  687. cs.ProposalBlock = nil
  688. cs.ProposalBlockParts = types.NewPartSetFromHeader(partsHeader)
  689. }
  690. cs.signAddVote(types.VoteTypePrecommit, nil, types.PartSetHeader{})
  691. return
  692. }
  693. // Enter: any +2/3 precommits for next round.
  694. func (cs *ConsensusState) EnterPrecommitWait(height int, round int) {
  695. cs.mtx.Lock()
  696. defer cs.mtx.Unlock()
  697. if cs.Height != height || round < cs.Round || (cs.Round == round && RoundStepPrecommitWait <= cs.Step) {
  698. log.Debug(Fmt("EnterPrecommitWait(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
  699. return
  700. }
  701. if !cs.Votes.Precommits(round).HasTwoThirdsAny() {
  702. panic(Fmt("EnterPrecommitWait(%v/%v), but Precommits does not have any +2/3 votes", height, round))
  703. }
  704. // Done EnterPrecommitWait:
  705. cs.Round = round
  706. cs.Step = RoundStepPrecommitWait
  707. cs.newStepCh <- cs.getRoundState()
  708. // After `timeoutPrecommit0+timeoutPrecommitDelta*round`, EnterNewRound()
  709. go func() {
  710. time.Sleep(timeoutPrecommit0 + timeoutPrecommitDelta*time.Duration(round))
  711. // If we have +2/3 of precommits for a particular block (or nil),
  712. // we already entered commit (or the next round).
  713. // So just try to transition to the next round,
  714. // which is what we'd do otherwise.
  715. cs.EnterNewRound(height, round+1)
  716. }()
  717. }
  718. // Enter: +2/3 precommits for block
  719. func (cs *ConsensusState) EnterCommit(height int) {
  720. cs.mtx.Lock()
  721. defer cs.mtx.Unlock()
  722. if cs.Height != height || RoundStepCommit <= cs.Step {
  723. log.Debug(Fmt("EnterCommit(%v): Invalid args. Current step: %v/%v/%v", height, cs.Height, cs.Round, cs.Step))
  724. return
  725. }
  726. defer func() {
  727. // Done Entercommit:
  728. // keep ca.Round the same, it points to the right Precommits set.
  729. cs.Step = RoundStepCommit
  730. cs.newStepCh <- cs.getRoundState()
  731. // Maybe finalize immediately.
  732. cs.tryFinalizeCommit(height)
  733. }()
  734. // SANITY CHECK
  735. hash, partsHeader, ok := cs.Votes.Precommits(cs.Round).TwoThirdsMajority()
  736. if !ok {
  737. panic("RunActionCommit() expects +2/3 precommits")
  738. }
  739. // END SANITY CHECK
  740. // The Locked* fields no longer matter.
  741. // Move them over to ProposalBlock if they match the commit hash,
  742. // otherwise they can now be cleared.
  743. if cs.LockedBlock.HashesTo(hash) {
  744. cs.ProposalBlock = cs.LockedBlock
  745. cs.ProposalBlockParts = cs.LockedBlockParts
  746. cs.LockedRound = 0
  747. cs.LockedBlock = nil
  748. cs.LockedBlockParts = nil
  749. } else {
  750. cs.LockedRound = 0
  751. cs.LockedBlock = nil
  752. cs.LockedBlockParts = nil
  753. }
  754. // If we don't have the block being committed, set up to get it.
  755. if !cs.ProposalBlock.HashesTo(hash) {
  756. if !cs.ProposalBlockParts.HasHeader(partsHeader) {
  757. // We're getting the wrong block.
  758. // Set up ProposalBlockParts and keep waiting.
  759. cs.ProposalBlock = nil
  760. cs.ProposalBlockParts = types.NewPartSetFromHeader(partsHeader)
  761. } else {
  762. // We just need to keep waiting.
  763. }
  764. }
  765. }
  766. // If we have the block AND +2/3 commits for it, finalize.
  767. func (cs *ConsensusState) tryFinalizeCommit(height int) {
  768. // SANITY CHECK
  769. if cs.Height != height {
  770. panic(Fmt("tryFinalizeCommit() cs.Height: %v vs height: %v", cs.Height, height))
  771. }
  772. // END SANITY CHECK
  773. hash, _, ok := cs.Votes.Precommits(cs.Round).TwoThirdsMajority()
  774. if !ok || len(hash) == 0 {
  775. return // There was no +2/3 majority, or +2/3 was for <nil>.
  776. }
  777. if !cs.ProposalBlock.HashesTo(hash) {
  778. return // We don't have the commit block.
  779. }
  780. go cs.FinalizeCommit(height)
  781. }
  782. // Increment height and goto RoundStepNewHeight
  783. func (cs *ConsensusState) FinalizeCommit(height int) {
  784. cs.mtx.Lock()
  785. defer cs.mtx.Unlock()
  786. if cs.Height != height || cs.Step != RoundStepCommit {
  787. log.Debug(Fmt("FinalizeCommit(%v): Invalid args. Current step: %v/%v/%v", height, cs.Height, cs.Round, cs.Step))
  788. return
  789. }
  790. hash, header, ok := cs.Votes.Precommits(cs.Round).TwoThirdsMajority()
  791. // SANITY CHECK
  792. if !ok {
  793. panic(Fmt("Cannot FinalizeCommit, commit does not have two thirds majority"))
  794. }
  795. if !cs.ProposalBlockParts.HasHeader(header) {
  796. panic(Fmt("Expected ProposalBlockParts header to be commit header"))
  797. }
  798. if !cs.ProposalBlock.HashesTo(hash) {
  799. panic(Fmt("Cannot FinalizeCommit, ProposalBlock does not hash to commit hash"))
  800. }
  801. if err := cs.stageBlock(cs.ProposalBlock, cs.ProposalBlockParts); err != nil {
  802. panic(Fmt("+2/3 committed an invalid block: %v", err))
  803. }
  804. // END SANITY CHECK
  805. log.Debug(Fmt("Finalizing commit of block: %v", cs.ProposalBlock))
  806. // We have the block, so stage/save/commit-vote.
  807. cs.saveBlock(cs.ProposalBlock, cs.ProposalBlockParts, cs.Votes.Precommits(cs.Round))
  808. // Increment height.
  809. cs.updateToState(cs.stagedState, true)
  810. // cs.StartTime is already set.
  811. // Schedule Round0 to start soon.
  812. go cs.scheduleRound0(height + 1)
  813. // If we're unbonded, broadcast RebondTx.
  814. cs.maybeRebond()
  815. // By here,
  816. // * cs.Height has been increment to height+1
  817. // * cs.Step is now RoundStepNewHeight
  818. // * cs.StartTime is set to when we will start round0.
  819. return
  820. }
  821. //-----------------------------------------------------------------------------
  822. func (cs *ConsensusState) SetProposal(proposal *Proposal) error {
  823. cs.mtx.Lock()
  824. defer cs.mtx.Unlock()
  825. // Already have one
  826. if cs.Proposal != nil {
  827. return nil
  828. }
  829. // Does not apply
  830. if proposal.Height != cs.Height || proposal.Round != cs.Round {
  831. return nil
  832. }
  833. // We don't care about the proposal if we're already in RoundStepCommit.
  834. if RoundStepCommit <= cs.Step {
  835. return nil
  836. }
  837. // Verify POLRound, which must be -1 or between 0 and proposal.Round exclusive.
  838. if proposal.POLRound != -1 &&
  839. (proposal.POLRound < 0 || proposal.Round <= proposal.POLRound) {
  840. return ErrInvalidProposalPOLRound
  841. }
  842. // Verify signature
  843. if !cs.Validators.Proposer().PubKey.VerifyBytes(account.SignBytes(cs.state.ChainID, proposal), proposal.Signature) {
  844. return ErrInvalidProposalSignature
  845. }
  846. cs.Proposal = proposal
  847. cs.ProposalBlockParts = types.NewPartSetFromHeader(proposal.BlockPartsHeader)
  848. return nil
  849. }
  850. // NOTE: block is not necessarily valid.
  851. func (cs *ConsensusState) AddProposalBlockPart(height int, part *types.Part) (added bool, err error) {
  852. cs.mtx.Lock()
  853. defer cs.mtx.Unlock()
  854. // Blocks might be reused, so round mismatch is OK
  855. if cs.Height != height {
  856. return false, nil
  857. }
  858. // We're not expecting a block part.
  859. if cs.ProposalBlockParts == nil {
  860. return false, nil // TODO: bad peer? Return error?
  861. }
  862. added, err = cs.ProposalBlockParts.AddPart(part)
  863. if err != nil {
  864. return added, err
  865. }
  866. if added && cs.ProposalBlockParts.IsComplete() {
  867. // Added and completed!
  868. var n int64
  869. var err error
  870. cs.ProposalBlock = binary.ReadBinary(&types.Block{}, cs.ProposalBlockParts.GetReader(), &n, &err).(*types.Block)
  871. log.Debug("Received complete proposal", "hash", cs.ProposalBlock.Hash())
  872. if cs.Step == RoundStepPropose && cs.isProposalComplete() {
  873. // Move onto the next step
  874. go cs.EnterPrevote(height, cs.Round)
  875. } else if cs.Step == RoundStepCommit {
  876. // If we're waiting on the proposal block...
  877. cs.tryFinalizeCommit(height)
  878. }
  879. return true, err
  880. }
  881. return added, nil
  882. }
  883. func (cs *ConsensusState) AddVote(address []byte, vote *types.Vote, peerKey string) (added bool, index int, err error) {
  884. cs.mtx.Lock()
  885. defer cs.mtx.Unlock()
  886. return cs.addVote(address, vote, peerKey)
  887. }
  888. //-----------------------------------------------------------------------------
  889. func (cs *ConsensusState) addVote(address []byte, vote *types.Vote, peerKey string) (added bool, index int, err error) {
  890. // A precommit for the previous height?
  891. if vote.Height+1 == cs.Height && vote.Type == types.VoteTypePrecommit {
  892. added, index, err = cs.LastCommit.AddByAddress(address, vote)
  893. if added {
  894. log.Debug(Fmt("Added to lastPrecommits: %v", cs.LastCommit.StringShort()))
  895. }
  896. return
  897. }
  898. // A prevote/precommit for this height?
  899. if vote.Height == cs.Height {
  900. height := cs.Height
  901. added, index, err = cs.Votes.AddByAddress(address, vote, peerKey)
  902. if added {
  903. switch vote.Type {
  904. case types.VoteTypePrevote:
  905. prevotes := cs.Votes.Prevotes(vote.Round)
  906. log.Debug(Fmt("Added to prevotes: %v", prevotes.StringShort()))
  907. // First, unlock if prevotes is a valid POL.
  908. // >> lockRound < POLRound <= unlockOrChangeLockRound (see spec)
  909. // NOTE: If (lockRound < POLRound) but !(POLRound <= unlockOrChangeLockRound),
  910. // we'll still EnterNewRound(H,vote.R) and EnterPrecommit(H,vote.R) to process it
  911. // there.
  912. if (cs.LockedBlock != nil) && (cs.LockedRound < vote.Round) && (vote.Round <= cs.Round) {
  913. hash, _, ok := prevotes.TwoThirdsMajority()
  914. if ok && !cs.LockedBlock.HashesTo(hash) {
  915. log.Info("Unlocking because of POL.", "lockedRound", cs.LockedRound, "POLRound", vote.Round)
  916. cs.LockedRound = 0
  917. cs.LockedBlock = nil
  918. cs.LockedBlockParts = nil
  919. }
  920. }
  921. if cs.Round <= vote.Round && prevotes.HasTwoThirdsAny() {
  922. // Round-skip over to PrevoteWait or goto Precommit.
  923. go func() {
  924. if cs.Round < vote.Round {
  925. cs.EnterNewRound(height, vote.Round)
  926. }
  927. if prevotes.HasTwoThirdsMajority() {
  928. cs.EnterPrecommit(height, vote.Round)
  929. } else {
  930. cs.EnterPrevote(height, vote.Round)
  931. cs.EnterPrevoteWait(height, vote.Round)
  932. }
  933. }()
  934. } else if cs.Proposal != nil && 0 <= cs.Proposal.POLRound && cs.Proposal.POLRound == vote.Round {
  935. // If the proposal is now complete, enter prevote of cs.Round.
  936. if cs.isProposalComplete() {
  937. go cs.EnterPrevote(height, cs.Round)
  938. }
  939. }
  940. case types.VoteTypePrecommit:
  941. precommits := cs.Votes.Precommits(vote.Round)
  942. log.Debug(Fmt("Added to precommit: %v", precommits.StringShort()))
  943. if cs.Round <= vote.Round && precommits.HasTwoThirdsAny() {
  944. go func() {
  945. hash, _, ok := precommits.TwoThirdsMajority()
  946. if ok && len(hash) == 0 {
  947. cs.EnterNewRound(height, vote.Round+1)
  948. return
  949. } else if cs.Round < vote.Round {
  950. cs.EnterNewRound(height, vote.Round)
  951. }
  952. if ok {
  953. cs.EnterCommit(height)
  954. } else {
  955. cs.EnterPrecommit(height, vote.Round)
  956. cs.EnterPrecommitWait(height, vote.Round)
  957. }
  958. }()
  959. }
  960. default:
  961. panic(Fmt("Unexpected vote type %X", vote.Type)) // Should not happen.
  962. }
  963. }
  964. return
  965. }
  966. // Height mismatch, bad peer? TODO
  967. return
  968. }
  969. func (cs *ConsensusState) stageBlock(block *types.Block, blockParts *types.PartSet) error {
  970. if block == nil {
  971. panic("Cannot stage nil block")
  972. }
  973. // Already staged?
  974. blockHash := block.Hash()
  975. if cs.stagedBlock != nil && len(blockHash) != 0 && bytes.Equal(cs.stagedBlock.Hash(), blockHash) {
  976. return nil
  977. }
  978. // Create a copy of the state for staging
  979. stateCopy := cs.state.Copy()
  980. // reset the event cache and pass it into the state
  981. cs.evc = events.NewEventCache(cs.evsw)
  982. stateCopy.SetFireable(cs.evc)
  983. // Commit block onto the copied state.
  984. // NOTE: Basic validation is done in state.AppendBlock().
  985. err := sm.ExecBlock(stateCopy, block, blockParts.Header())
  986. if err != nil {
  987. return err
  988. } else {
  989. cs.stagedBlock = block
  990. cs.stagedState = stateCopy
  991. return nil
  992. }
  993. }
  994. func (cs *ConsensusState) signAddVote(type_ byte, hash []byte, header types.PartSetHeader) *types.Vote {
  995. if cs.privValidator == nil || !cs.Validators.HasAddress(cs.privValidator.Address) {
  996. return nil
  997. }
  998. vote := &types.Vote{
  999. Height: cs.Height,
  1000. Round: cs.Round,
  1001. Type: type_,
  1002. BlockHash: hash,
  1003. BlockParts: header,
  1004. }
  1005. err := cs.privValidator.SignVote(cs.state.ChainID, vote)
  1006. if err == nil {
  1007. _, _, err := cs.addVote(cs.privValidator.Address, vote, "")
  1008. log.Info("Signed and added vote", "height", cs.Height, "round", cs.Round, "vote", vote, "error", err)
  1009. return vote
  1010. } else {
  1011. log.Warn("Error signing vote", "height", cs.Height, "round", cs.Round, "vote", vote, "error", err)
  1012. return nil
  1013. }
  1014. }
  1015. // Save Block, save the +2/3 Commits we've seen
  1016. func (cs *ConsensusState) saveBlock(block *types.Block, blockParts *types.PartSet, commits *VoteSet) {
  1017. // The proposal must be valid.
  1018. if err := cs.stageBlock(block, blockParts); err != nil {
  1019. panic(Fmt("saveBlock() an invalid block: %v", err))
  1020. }
  1021. // Save to blockStore.
  1022. if cs.blockStore.Height() < block.Height {
  1023. seenValidation := commits.MakeValidation()
  1024. cs.blockStore.SaveBlock(block, blockParts, seenValidation)
  1025. }
  1026. // Save the state.
  1027. cs.stagedState.Save()
  1028. // Update mempool.
  1029. cs.mempoolReactor.Mempool.ResetForBlockAndState(block, cs.stagedState)
  1030. // Fire off event
  1031. go func(block *types.Block) {
  1032. cs.evsw.FireEvent(types.EventStringNewBlock(), block)
  1033. cs.evc.Flush()
  1034. }(block)
  1035. }
  1036. // implements events.Eventable
  1037. func (cs *ConsensusState) SetFireable(evsw events.Fireable) {
  1038. cs.evsw = evsw
  1039. }