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.

1334 lines
48 KiB

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