Browse Source

Merge remote-tracking branch 'origin/consensus_refactor' into consensus_refactor_jae

pull/169/head
Jae Kwon 9 years ago
parent
commit
c95b89e98c
12 changed files with 654 additions and 758 deletions
  1. +2
    -2
      blockchain/reactor.go
  2. +50
    -27
      consensus/common_test.go
  3. +49
    -40
      consensus/reactor.go
  4. +136
    -150
      consensus/state.go
  5. +362
    -509
      consensus/state_test.go
  6. +11
    -1
      events/events.go
  7. +2
    -2
      mempool/reactor.go
  8. +3
    -3
      node/node.go
  9. +16
    -3
      rpc/server/handlers.go
  10. +2
    -3
      state/state.go
  11. +6
    -2
      types/block.go
  12. +15
    -16
      types/events.go

+ 2
- 2
blockchain/reactor.go View File

@ -52,7 +52,7 @@ type BlockchainReactor struct {
timeoutsCh chan string
lastBlock *types.Block
evsw events.Fireable
evsw *events.EventSwitch
}
func NewBlockchainReactor(state *sm.State, proxyAppCtx proxy.AppContext, store *BlockStore, sync bool) *BlockchainReactor {
@ -263,7 +263,7 @@ func (bcR *BlockchainReactor) BroadcastStatusRequest() error {
}
// implements events.Eventable
func (bcR *BlockchainReactor) SetFireable(evsw events.Fireable) {
func (bcR *BlockchainReactor) SetEventSwitch(evsw *events.EventSwitch) {
bcR.evsw = evsw
}


+ 50
- 27
consensus/common_test.go View File

@ -78,6 +78,7 @@ func decideProposal(cs1 *ConsensusState, cs2 *validatorStub, height, round int)
//-------------------------------------------------------------------------------
// utils
/*
func nilRound(t *testing.T, cs1 *ConsensusState, vss ...*validatorStub) {
cs1.mtx.Lock()
height, round := cs1.Height, cs1.Round
@ -93,6 +94,7 @@ func nilRound(t *testing.T, cs1 *ConsensusState, vss ...*validatorStub) {
waitFor(t, cs1, height, round+1, RoundStepNewRound)
}
*/
// NOTE: this switches the propser as far as `perspectiveOf` is concerned,
// but for simplicity we return a block it generated.
@ -172,6 +174,17 @@ func signAddVoteToFrom(voteType byte, to *ConsensusState, from *validatorStub, h
return vote
}
func ensureNoNewStep(stepCh chan interface{}) {
timeout := time.NewTicker(ensureTimeout * time.Second)
select {
case <-timeout.C:
break
case <-stepCh:
panic("We should be stuck waiting for more votes, not moving to the next step")
}
}
/*
func ensureNoNewStep(t *testing.T, cs *ConsensusState) {
timeout := time.NewTicker(ensureTimeout * time.Second)
select {
@ -202,6 +215,19 @@ func waitFor(t *testing.T, cs *ConsensusState, height int, round int, step Round
}
}
}
*/
func incrementHeight(vss ...*validatorStub) {
for _, vs := range vss {
vs.Height += 1
}
}
func incrementRound(vss ...*validatorStub) {
for _, vs := range vss {
vs.Round += 1
}
}
func validatePrevote(t *testing.T, cs *ConsensusState, round int, privVal *validatorStub, blockHash []byte) {
prevotes := cs.Votes.Prevotes(round)
@ -220,15 +246,14 @@ func validatePrevote(t *testing.T, cs *ConsensusState, round int, privVal *valid
}
}
func incrementHeight(vss ...*validatorStub) {
for _, vs := range vss {
vs.Height += 1
func validateLastPrecommit(t *testing.T, cs *ConsensusState, privVal *validatorStub, blockHash []byte) {
votes := cs.LastCommit
var vote *types.Vote
if vote = votes.GetByAddress(privVal.Address); vote == nil {
panic("Failed to find precommit from validator")
}
}
func incrementRound(vss ...*validatorStub) {
for _, vs := range vss {
vs.Round += 1
if !bytes.Equal(vote.BlockHash, blockHash) {
panic(fmt.Sprintf("Expected precommit to be for %X, got %X", blockHash, vote.BlockHash))
}
}
@ -298,17 +323,9 @@ func simpleConsensusState(nValidators int) (*ConsensusState, []*validatorStub) {
cs := NewConsensusState(state, proxyAppCtxCon, blockStore, mempool)
cs.SetPrivValidator(privVals[0])
// from the updateToState in NewConsensusState
<-cs.NewStepCh()
evsw := events.NewEventSwitch()
cs.SetFireable(evsw)
evsw.OnStart()
go func() {
for {
<-cs.NewStepCh()
}
}()
cs.SetEventSwitch(evsw)
evsw.Start()
// start the transition routines
// cs.startRoutines()
@ -322,14 +339,20 @@ func simpleConsensusState(nValidators int) (*ConsensusState, []*validatorStub) {
return cs, vss
}
func subscribeToEvent(cs *ConsensusState, eventID string) chan interface{} {
evsw := cs.evsw.(*events.EventSwitch)
// listen for new round
ch := make(chan interface{}, 10)
evsw.AddListenerForEvent("tester", eventID, func(data types.EventData) {
ch <- data
})
return ch
func subscribeToVoter(cs *ConsensusState, addr []byte) chan interface{} {
voteCh0 := cs.evsw.SubscribeToEvent("tester", types.EventStringVote(), 0)
voteCh := make(chan interface{})
go func() {
for {
v := <-voteCh0
vote := v.(*types.EventDataVote)
// we only fire for our own votes
if bytes.Equal(addr, vote.Address) {
voteCh <- v
}
}
}()
return voteCh
}
func randGenesisState(numValidators int, randPower bool, minPower int64) (*sm.State, []*types.PrivValidator) {
@ -361,6 +384,6 @@ func randGenesisDoc(numValidators int, randPower bool, minPower int64) (*types.G
}
func startTestRound(cs *ConsensusState, height, round int) {
cs.EnterNewRound(height, round)
cs.enterNewRound(height, round)
cs.startRoutines(0)
}

+ 49
- 40
consensus/reactor.go View File

@ -34,7 +34,7 @@ type ConsensusReactor struct {
blockStore *bc.BlockStore
conS *ConsensusState
fastSync bool
evsw events.Fireable
evsw *events.EventSwitch
}
func NewConsensusReactor(consensusState *ConsensusState, blockStore *bc.BlockStore, fastSync bool) *ConsensusReactor {
@ -50,13 +50,17 @@ func NewConsensusReactor(consensusState *ConsensusState, blockStore *bc.BlockSto
func (conR *ConsensusReactor) OnStart() error {
log.Notice("ConsensusReactor ", "fastSync", conR.fastSync)
conR.BaseReactor.OnStart()
// callbacks for broadcasting new steps and votes to peers
// upon their respective events (ie. uses evsw)
conR.registerEventCallbacks()
if !conR.fastSync {
_, err := conR.conS.Start()
if err != nil {
return err
}
}
go conR.broadcastNewRoundStepRoutine()
return nil
}
@ -134,7 +138,7 @@ func (conR *ConsensusReactor) RemovePeer(peer *p2p.Peer, reason interface{}) {
// NOTE: We process these messages even when we're fast_syncing.
// Messages affect either a peer state or the consensus state.
// Peer state updates can happen in parallel, but processing of
// proposals, block parts, and votes are ordered.
// proposals, block parts, and votes are ordered by the receiveRoutine
func (conR *ConsensusReactor) Receive(chID byte, peer *p2p.Peer, msgBytes []byte) {
if !conR.IsRunning() {
log.Debug("Receive", "channel", chID, "peer", peer, "bytes", msgBytes)
@ -213,6 +217,45 @@ func (conR *ConsensusReactor) Receive(chID byte, peer *p2p.Peer, msgBytes []byte
}
}
// Sets our private validator account for signing votes.
func (conR *ConsensusReactor) SetPrivValidator(priv *types.PrivValidator) {
conR.conS.SetPrivValidator(priv)
}
// implements events.Eventable
func (conR *ConsensusReactor) SetEventSwitch(evsw *events.EventSwitch) {
conR.evsw = evsw
conR.conS.SetEventSwitch(evsw)
}
//--------------------------------------
// Listens for new steps and votes,
// broadcasting the result to peers
func (conR *ConsensusReactor) registerEventCallbacks() {
conR.evsw.AddListenerForEvent("conR", types.EventStringNewRoundStep(), func(data types.EventData) {
rs := data.(*types.EventDataRoundState).RoundState().(*RoundState)
conR.broadcastNewRoundStep(rs)
})
conR.evsw.AddListenerForEvent("conR", types.EventStringVote(), func(data types.EventData) {
edv := data.(*types.EventDataVote)
conR.broadcastHasVoteMessage(edv.Vote, edv.Index)
})
}
func (conR *ConsensusReactor) broadcastNewRoundStep(rs *RoundState) {
nrsMsg, csMsg := makeRoundStepMessages(rs)
if nrsMsg != nil {
conR.Switch.Broadcast(StateChannel, nrsMsg)
}
if csMsg != nil {
conR.Switch.Broadcast(StateChannel, csMsg)
}
}
// Broadcasts HasVoteMessage to peers that care.
func (conR *ConsensusReactor) broadcastHasVoteMessage(vote *types.Vote, index int) {
msg := &HasVoteMessage{
@ -239,28 +282,16 @@ func (conR *ConsensusReactor) broadcastHasVoteMessage(vote *types.Vote, index in
*/
}
// Sets our private validator account for signing votes.
func (conR *ConsensusReactor) SetPrivValidator(priv *types.PrivValidator) {
conR.conS.SetPrivValidator(priv)
}
// implements events.Eventable
func (conR *ConsensusReactor) SetFireable(evsw events.Fireable) {
conR.evsw = evsw
conR.conS.SetFireable(evsw)
}
//--------------------------------------
func makeRoundStepMessages(rs *RoundState) (nrsMsg *NewRoundStepMessage, csMsg *CommitStepMessage) {
step := RoundStepType(rs.Step)
nrsMsg = &NewRoundStepMessage{
Height: rs.Height,
Round: rs.Round,
Step: rs.Step,
Step: step,
SecondsSinceStartTime: int(time.Now().Sub(rs.StartTime).Seconds()),
LastCommitRound: rs.LastCommit.Round(),
}
if rs.Step == RoundStepCommit {
if step == RoundStepCommit {
csMsg = &CommitStepMessage{
Height: rs.Height,
BlockPartsHeader: rs.ProposalBlockParts.Header(),
@ -270,28 +301,6 @@ func makeRoundStepMessages(rs *RoundState) (nrsMsg *NewRoundStepMessage, csMsg *
return
}
// Listens for changes to the ConsensusState.Step by pulling
// on conR.conS.NewStepCh().
func (conR *ConsensusReactor) broadcastNewRoundStepRoutine() {
for {
// Get RoundState with new Step or quit.
var rs *RoundState
select {
case rs = <-conR.conS.NewStepCh():
case <-conR.Quit:
return
}
nrsMsg, csMsg := makeRoundStepMessages(rs)
if nrsMsg != nil {
conR.Switch.Broadcast(StateChannel, nrsMsg)
}
if csMsg != nil {
conR.Switch.Broadcast(StateChannel, csMsg)
}
}
}
func (conR *ConsensusReactor) sendNewRoundStepMessage(peer *p2p.Peer) {
rs := conR.conS.GetRoundState()
nrsMsg, csMsg := makeRoundStepMessages(rs)


+ 136
- 150
consensus/state.go View File

@ -98,19 +98,13 @@ type RoundState struct {
}
func (rs *RoundState) RoundStateEvent() *types.EventDataRoundState {
return &types.EventDataRoundState{
CurrentTime: time.Now(),
Height: rs.Height,
Round: rs.Round,
Step: rs.Step.String(),
StartTime: rs.StartTime,
CommitTime: rs.CommitTime,
Proposal: rs.Proposal,
ProposalBlock: rs.ProposalBlock,
LockedRound: rs.LockedRound,
LockedBlock: rs.LockedBlock,
POLRound: rs.Votes.POLRound(),
edrs := &types.EventDataRoundState{
Height: rs.Height,
Round: rs.Round,
Step: rs.Step.String(),
}
edrs.SetRoundState(rs)
return edrs
}
func (rs *RoundState) String() string {
@ -183,7 +177,6 @@ type ConsensusState struct {
blockStore *bc.BlockStore
mempool *mempl.Mempool
privValidator *types.PrivValidator
newStepCh chan *RoundState
mtx sync.Mutex
RoundState
@ -197,7 +190,7 @@ type ConsensusState struct {
tickChan chan timeoutInfo // start the timeoutTicker in the timeoutRoutine
tockChan chan timeoutInfo // timeouts are relayed on tockChan to the receiveRoutine
evsw events.Fireable
evsw *events.EventSwitch
evc *events.EventCache // set in stageBlock and passed into state
nSteps int // used for testing to limit the number of transitions the state makes
@ -208,7 +201,6 @@ func NewConsensusState(state *sm.State, proxyAppCtx proxy.AppContext, blockStore
proxyAppCtx: proxyAppCtx,
blockStore: blockStore,
mempool: mempool,
newStepCh: make(chan *RoundState, 10),
peerMsgQueue: make(chan msgInfo, msgQueueSize),
internalMsgQueue: make(chan msgInfo, msgQueueSize),
timeoutTicker: new(time.Ticker),
@ -227,7 +219,7 @@ func NewConsensusState(state *sm.State, proxyAppCtx proxy.AppContext, blockStore
// Public interface
// implements events.Eventable
func (cs *ConsensusState) SetFireable(evsw events.Fireable) {
func (cs *ConsensusState) SetEventSwitch(evsw *events.EventSwitch) {
cs.evsw = evsw
}
@ -258,35 +250,33 @@ func (cs *ConsensusState) SetPrivValidator(priv *types.PrivValidator) {
cs.privValidator = priv
}
func (cs *ConsensusState) NewStepCh() chan *RoundState {
return cs.newStepCh
}
func (cs *ConsensusState) OnStart() error {
cs.BaseService.OnStart()
// first we start the round (no go routines)
// first we schedule the round (no go routines)
// then we start the timeout and receive routines.
// buffered channels means scheduleRound0 will finish. Once it does,
// all further access to the RoundState is through the receiveRoutine
// tickChan is buffered so scheduleRound0 will finish.
// Then all further access to the RoundState is through the receiveRoutine
cs.scheduleRound0(cs.Height)
cs.startRoutines(0) // start timeout and receive
cs.startRoutines(0)
return nil
}
// timeoutRoutine: receive requests for timeouts on tickChan and fire timeouts on tockChan
// receiveRoutine: serializes processing of proposoals, block parts, votes; coordinates state transitions
func (cs *ConsensusState) startRoutines(maxSteps int) {
go cs.timeoutRoutine() // receive requests for timeouts on tickChan and fire timeouts on tockChan
go cs.receiveRoutine(maxSteps) // serializes processing of proposoals, block parts, votes, and coordinates state transitions
go cs.timeoutRoutine()
go cs.receiveRoutine(maxSteps)
}
func (cs *ConsensusState) OnStop() {
cs.QuitService.OnStop()
}
/*
The following three functions can be used to send messages into the consensus state
which may cause a state transition
*/
//------------------------------------------------------------
// Public interface for passing messages into the consensus state,
// possibly causing a state transition
// TODO: should these return anything or let callers just use events?
// May block on send if queue is full.
func (cs *ConsensusState) AddVote(valIndex int, vote *types.Vote, peerKey string) (added bool, address []byte, err error) {
@ -330,12 +320,12 @@ func (cs *ConsensusState) SetProposalAndBlock(proposal *types.Proposal, block *t
cs.SetProposal(proposal, peerKey)
for i := 0; i < parts.Total(); i++ {
part := parts.GetPart(i)
cs.AddProposalBlockPart(cs.Height, cs.Round, part, peerKey)
cs.AddProposalBlockPart(proposal.Height, proposal.Round, part, peerKey)
}
return nil // TODO errors
}
//----------------------------------------------
//------------------------------------------------------------
// internal functions for managing the state
func (cs *ConsensusState) updateHeight(height int) {
@ -347,11 +337,11 @@ func (cs *ConsensusState) updateRoundStep(round int, step RoundStepType) {
cs.Step = step
}
// EnterNewRound(height, 0) at cs.StartTime.
// enterNewRound(height, 0) at cs.StartTime.
func (cs *ConsensusState) scheduleRound0(height int) {
//log.Info("scheduleRound0", "now", time.Now(), "startTime", cs.StartTime)
sleepDuration := cs.StartTime.Sub(time.Now())
cs.scheduleTimeout(sleepDuration, height, 0, 1)
cs.scheduleTimeout(sleepDuration, height, 0, RoundStepNewHeight)
}
// Attempt to schedule a timeout by sending timeoutInfo on the tickChan.
@ -363,11 +353,14 @@ func (cs *ConsensusState) scheduleTimeout(duration time.Duration, height, round
// send a msg into the receiveRoutine regarding our own proposal, block part, or vote
func (cs *ConsensusState) sendInternalMessage(mi msgInfo) {
timeout := time.After(10 * time.Millisecond)
select {
case cs.internalMsgQueue <- mi:
case <-timeout:
log.Debug("Timed out trying to send an internal messge. Launching go-routine")
default:
// NOTE: using the go-routine means our votes can
// be processed out of order.
// TODO: use CList here for strict determinism and
// attempt push to internalMsgQueue in receiveRoutine
log.Debug("Internal msg queue is full. Using a go-routine")
go func() { cs.internalMsgQueue <- mi }()
}
}
@ -464,7 +457,10 @@ func (cs *ConsensusState) updateToState(state *sm.State) {
func (cs *ConsensusState) newStep() {
cs.nSteps += 1
cs.newStepCh <- cs.getRoundState()
// newStep is called by updateToStep in NewConsensusState before the evsw is set!
if cs.evsw != nil {
cs.evsw.FireEvent(types.EventStringNewRoundStep(), cs.RoundStateEvent())
}
}
//-----------------------------------------
@ -527,7 +523,7 @@ func (cs *ConsensusState) stopTimer() {
// receiveRoutine handles messages which may cause state transitions.
// it's argument (n) is the number of messages to process before exiting - use 0 to run forever
// It keeps the RoundState and is the only thing that updates it.
// Updates happen on timeouts, complete proposals, and 2/3 majorities
// Updates (state transitions) happen on timeouts, complete proposals, and 2/3 majorities
func (cs *ConsensusState) receiveRoutine(maxSteps int) {
for {
if maxSteps > 0 {
@ -571,25 +567,23 @@ func (cs *ConsensusState) handleMsg(mi msgInfo, rs RoundState) {
// once proposal is set, we can receive block parts
err = cs.setProposal(msg.Proposal)
case *BlockPartMessage:
// if the proposal is complete, we'll EnterPrevote or tryFinalizeCommit
// if we're the only validator, the EnterPrevote may take us through to the next round
// if the proposal is complete, we'll enterPrevote or tryFinalizeCommit
// if we're the only validator, the enterPrevote may take us through to the next round
_, err = cs.addProposalBlockPart(msg.Height, msg.Part)
case *VoteMessage:
// attempt to add the vote and dupeout the validator if its a duplicate signature
// if the vote gives us a 2/3-any or 2/3-one, we transition
added, err := cs.tryAddVote(msg.ValidatorIndex, msg.Vote, peerKey)
err := cs.tryAddVote(msg.ValidatorIndex, msg.Vote, peerKey)
if err == ErrAddingVote {
// TODO: punish peer
}
if added {
// If rs.Height == vote.Height && rs.Round < vote.Round,
// the peer is sending us CatchupCommit precommits.
// We could make note of this and help filter in broadcastHasVoteMessage().
// NOTE: the vote is broadcast to peers by the reactor listening
// for vote events
// XXX TODO: do this
// conR.broadcastHasVoteMessage(msg.Vote, msg.ValidatorIndex)
}
// TODO: If rs.Height == vote.Height && rs.Round < vote.Round,
// the peer is sending us CatchupCommit precommits.
// We could make note of this and help filter in broadcastHasVoteMessage().
default:
log.Warn("Unknown msg type", reflect.TypeOf(msg))
}
@ -601,17 +595,6 @@ func (cs *ConsensusState) handleMsg(mi msgInfo, rs RoundState) {
func (cs *ConsensusState) handleTimeout(ti timeoutInfo, rs RoundState) {
log.Debug("Received tock", "timeout", ti.duration, "height", ti.height, "round", ti.round, "step", ti.step)
// if this is a timeout for the new height
if ti.height == rs.Height+1 && ti.round == 0 && ti.step == 1 {
cs.mtx.Lock()
// Increment height.
cs.updateToState(cs.stagedState)
// event fired from EnterNewRound after some updates
cs.EnterNewRound(ti.height, 0)
cs.mtx.Unlock()
return
}
// timeouts must be for current height, round, step
if ti.height != rs.Height || ti.round < rs.Round || (ti.round == rs.Round && ti.step < rs.Step) {
log.Debug("Ignoring tock because we're ahead", "height", rs.Height, "round", rs.Round, "step", rs.Step)
@ -623,15 +606,19 @@ func (cs *ConsensusState) handleTimeout(ti timeoutInfo, rs RoundState) {
defer cs.mtx.Unlock()
switch ti.step {
case RoundStepNewHeight:
// NewRound event fired from enterNewRound.
// Do we want a timeout event too?
cs.enterNewRound(ti.height, 0)
case RoundStepPropose:
cs.evsw.FireEvent(types.EventStringTimeoutPropose(), cs.RoundStateEvent())
cs.EnterPrevote(ti.height, ti.round)
cs.enterPrevote(ti.height, ti.round)
case RoundStepPrevoteWait:
cs.evsw.FireEvent(types.EventStringTimeoutWait(), cs.RoundStateEvent())
cs.EnterPrecommit(ti.height, ti.round)
cs.enterPrecommit(ti.height, ti.round)
case RoundStepPrecommitWait:
cs.evsw.FireEvent(types.EventStringTimeoutWait(), cs.RoundStateEvent())
cs.EnterNewRound(ti.height, ti.round+1)
cs.enterNewRound(ti.height, ti.round+1)
default:
panic(Fmt("Invalid timeout step: %v", ti.step))
}
@ -640,17 +627,15 @@ func (cs *ConsensusState) handleTimeout(ti timeoutInfo, rs RoundState) {
//-----------------------------------------------------------------------------
// State functions
// Many of these functions are capitalized but are not really meant to be used
// by external code as it will cause race conditions with running timeout/receiveRoutine.
// Use AddVote, SetProposal, AddProposalBlockPart instead
// Used internally by handleTimeout and handleMsg to make state transitions
// Enter: +2/3 precommits for nil at (height,round-1)
// Enter: `timeoutPrecommits` after any +2/3 precommits from (height,round-1)
// Enter: `startTime = commitTime+timeoutCommit` from NewHeight(height)
// NOTE: cs.StartTime was already set for height.
func (cs *ConsensusState) EnterNewRound(height int, round int) {
func (cs *ConsensusState) enterNewRound(height int, round int) {
if cs.Height != height || round < cs.Round || (cs.Round == round && cs.Step != RoundStepNewHeight) {
log.Debug(Fmt("EnterNewRound(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
log.Debug(Fmt("enterNewRound(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
return
}
@ -659,7 +644,7 @@ func (cs *ConsensusState) EnterNewRound(height int, round int) {
}
// cs.stopTimer()
log.Notice(Fmt("EnterNewRound(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
log.Notice(Fmt("enterNewRound(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
// Increment validators if necessary
validators := cs.Validators
@ -686,25 +671,32 @@ func (cs *ConsensusState) EnterNewRound(height int, round int) {
cs.evsw.FireEvent(types.EventStringNewRound(), cs.RoundStateEvent())
// Immediately go to EnterPropose.
cs.EnterPropose(height, round)
// Immediately go to enterPropose.
cs.enterPropose(height, round)
}
// Enter: from NewRound(height,round).
func (cs *ConsensusState) EnterPropose(height int, round int) {
func (cs *ConsensusState) enterPropose(height int, round int) {
// cs.mtx.Lock()
// cs.mtx.Unlock()
if cs.Height != height || round < cs.Round || (cs.Round == round && RoundStepPropose <= cs.Step) {
log.Debug(Fmt("EnterPropose(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
log.Debug(Fmt("enterPropose(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
return
}
log.Info(Fmt("EnterPropose(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
log.Info(Fmt("enterPropose(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
defer func() {
// Done EnterPropose:
// Done enterPropose:
cs.updateRoundStep(round, RoundStepPropose)
cs.newStep()
// If we have the whole proposal + POL, then goto Prevote now.
// else, we'll enterPrevote when the rest of the proposal is received (in AddProposalBlockPart),
// or else after timeoutPropose
if cs.isProposalComplete() {
cs.enterPrevote(height, cs.Round)
}
}()
// This step times out after `timeoutPropose`
@ -716,18 +708,12 @@ func (cs *ConsensusState) EnterPropose(height int, round int) {
}
if !bytes.Equal(cs.Validators.Proposer().Address, cs.privValidator.Address) {
log.Info("EnterPropose: Not our turn to propose", "proposer", cs.Validators.Proposer().Address, "privValidator", cs.privValidator)
log.Info("enterPropose: Not our turn to propose", "proposer", cs.Validators.Proposer().Address, "privValidator", cs.privValidator)
} else {
log.Info("EnterPropose: Our turn to propose", "proposer", cs.Validators.Proposer().Address, "privValidator", cs.privValidator)
log.Info("enterPropose: Our turn to propose", "proposer", cs.Validators.Proposer().Address, "privValidator", cs.privValidator)
cs.decideProposal(height, round)
}
// If we have the whole proposal + POL, then goto Prevote now.
// else, we'll EnterPrevote when the rest of the proposal is received (in AddProposalBlockPart),
// or else after timeoutPropose
if cs.isProposalComplete() {
cs.EnterPrevote(height, cs.Round)
}
}
func (cs *ConsensusState) decideProposal(height, round int) {
@ -766,7 +752,7 @@ func (cs *ConsensusState) decideProposal(height, round int) {
log.Notice("Signed and sent proposal", "height", height, "round", round, "proposal", proposal)
log.Debug(Fmt("Signed and sent proposal block: %v", block))
} else {
log.Warn("EnterPropose: Error signing proposal", "height", height, "round", round, "error", err)
log.Warn("enterPropose: Error signing proposal", "height", height, "round", round, "error", err)
}
}
@ -801,7 +787,7 @@ func (cs *ConsensusState) createProposalBlock() (block *types.Block, blockParts
validation = cs.LastCommit.MakeValidation()
} else {
// This shouldn't happen.
log.Error("EnterPropose: Cannot propose anything: No validation for the previous block.")
log.Error("enterPropose: Cannot propose anything: No validation for the previous block.")
return
}
@ -840,16 +826,16 @@ func (cs *ConsensusState) createProposalBlock() (block *types.Block, blockParts
// Enter: any +2/3 prevotes for future round.
// Prevote for LockedBlock if we're locked, or ProposalBlock if valid.
// Otherwise vote nil.
func (cs *ConsensusState) EnterPrevote(height int, round int) {
func (cs *ConsensusState) enterPrevote(height int, round int) {
//cs.mtx.Lock()
//defer cs.mtx.Unlock()
if cs.Height != height || round < cs.Round || (cs.Round == round && RoundStepPrevote <= cs.Step) {
log.Debug(Fmt("EnterPrevote(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
log.Debug(Fmt("enterPrevote(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
return
}
defer func() {
// Done EnterPrevote:
// Done enterPrevote:
cs.updateRoundStep(round, RoundStepPrevote)
cs.newStep()
}()
@ -864,7 +850,7 @@ func (cs *ConsensusState) EnterPrevote(height int, round int) {
// cs.stopTimer()
log.Info(Fmt("EnterPrevote(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
log.Info(Fmt("enterPrevote(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
// Sign and broadcast vote as necessary
cs.doPrevote(height, round)
@ -876,14 +862,14 @@ func (cs *ConsensusState) EnterPrevote(height int, round int) {
func (cs *ConsensusState) doPrevote(height int, round int) {
// If a block is locked, prevote that.
if cs.LockedBlock != nil {
log.Info("EnterPrevote: Block was locked")
log.Info("enterPrevote: Block was locked")
cs.signAddVote(types.VoteTypePrevote, cs.LockedBlock.Hash(), cs.LockedBlockParts.Header())
return
}
// If ProposalBlock is nil, prevote nil.
if cs.ProposalBlock == nil {
log.Warn("EnterPrevote: ProposalBlock is nil")
log.Warn("enterPrevote: ProposalBlock is nil")
cs.signAddVote(types.VoteTypePrevote, nil, types.PartSetHeader{})
return
}
@ -892,7 +878,7 @@ func (cs *ConsensusState) doPrevote(height int, round int) {
err := cs.stageBlock(cs.ProposalBlock, cs.ProposalBlockParts)
if err != nil {
// ProposalBlock is invalid, prevote nil.
log.Warn("EnterPrevote: ProposalBlock is invalid", "error", err)
log.Warn("enterPrevote: ProposalBlock is invalid", "error", err)
cs.signAddVote(types.VoteTypePrevote, nil, types.PartSetHeader{})
return
}
@ -905,25 +891,25 @@ func (cs *ConsensusState) doPrevote(height int, round int) {
}
// Enter: any +2/3 prevotes at next round.
func (cs *ConsensusState) EnterPrevoteWait(height int, round int) {
func (cs *ConsensusState) enterPrevoteWait(height int, round int) {
//cs.mtx.Lock()
//defer cs.mtx.Unlock()
if cs.Height != height || round < cs.Round || (cs.Round == round && RoundStepPrevoteWait <= cs.Step) {
log.Debug(Fmt("EnterPrevoteWait(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
log.Debug(Fmt("enterPrevoteWait(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
return
}
if !cs.Votes.Prevotes(round).HasTwoThirdsAny() {
PanicSanity(Fmt("EnterPrevoteWait(%v/%v), but Prevotes does not have any +2/3 votes", height, round))
PanicSanity(Fmt("enterPrevoteWait(%v/%v), but Prevotes does not have any +2/3 votes", height, round))
}
log.Info(Fmt("EnterPrevoteWait(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
log.Info(Fmt("enterPrevoteWait(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
defer func() {
// Done EnterPrevoteWait:
// Done enterPrevoteWait:
cs.updateRoundStep(round, RoundStepPrevoteWait)
cs.newStep()
}()
// After `timeoutPrevote0+timeoutPrevoteDelta*round`, EnterPrecommit()
// After `timeoutPrevote0+timeoutPrevoteDelta*round`, enterPrecommit()
cs.scheduleTimeout(timeoutPrevote0+timeoutPrevoteDelta*time.Duration(round), height, round, RoundStepPrevoteWait)
}
@ -933,20 +919,20 @@ func (cs *ConsensusState) EnterPrevoteWait(height int, round int) {
// Lock & precommit the ProposalBlock if we have enough prevotes for it (a POL in this round)
// else, unlock an existing lock and precommit nil if +2/3 of prevotes were nil,
// else, precommit nil otherwise.
func (cs *ConsensusState) EnterPrecommit(height int, round int) {
func (cs *ConsensusState) enterPrecommit(height int, round int) {
//cs.mtx.Lock()
// defer cs.mtx.Unlock()
if cs.Height != height || round < cs.Round || (cs.Round == round && RoundStepPrecommit <= cs.Step) {
log.Debug(Fmt("EnterPrecommit(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
log.Debug(Fmt("enterPrecommit(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
return
}
// cs.stopTimer()
log.Info(Fmt("EnterPrecommit(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
log.Info(Fmt("enterPrecommit(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
defer func() {
// Done EnterPrecommit:
// Done enterPrecommit:
cs.updateRoundStep(round, RoundStepPrecommit)
cs.newStep()
}()
@ -956,9 +942,9 @@ func (cs *ConsensusState) EnterPrecommit(height int, round int) {
// If we don't have a polka, we must precommit nil
if !ok {
if cs.LockedBlock != nil {
log.Info("EnterPrecommit: No +2/3 prevotes during EnterPrecommit while we're locked. Precommitting nil")
log.Info("enterPrecommit: No +2/3 prevotes during enterPrecommit while we're locked. Precommitting nil")
} else {
log.Info("EnterPrecommit: No +2/3 prevotes during EnterPrecommit. Precommitting nil.")
log.Info("enterPrecommit: No +2/3 prevotes during enterPrecommit. Precommitting nil.")
}
cs.signAddVote(types.VoteTypePrecommit, nil, types.PartSetHeader{})
return
@ -975,9 +961,9 @@ func (cs *ConsensusState) EnterPrecommit(height int, round int) {
// +2/3 prevoted nil. Unlock and precommit nil.
if len(hash) == 0 {
if cs.LockedBlock == nil {
log.Info("EnterPrecommit: +2/3 prevoted for nil.")
log.Info("enterPrecommit: +2/3 prevoted for nil.")
} else {
log.Info("EnterPrecommit: +2/3 prevoted for nil. Unlocking")
log.Info("enterPrecommit: +2/3 prevoted for nil. Unlocking")
cs.LockedRound = 0
cs.LockedBlock = nil
cs.LockedBlockParts = nil
@ -991,7 +977,7 @@ func (cs *ConsensusState) EnterPrecommit(height int, round int) {
// If we're already locked on that block, precommit it, and update the LockedRound
if cs.LockedBlock.HashesTo(hash) {
log.Info("EnterPrecommit: +2/3 prevoted locked block. Relocking")
log.Info("enterPrecommit: +2/3 prevoted locked block. Relocking")
cs.LockedRound = round
cs.evsw.FireEvent(types.EventStringRelock(), cs.RoundStateEvent())
cs.signAddVote(types.VoteTypePrecommit, hash, partsHeader)
@ -1000,10 +986,10 @@ func (cs *ConsensusState) EnterPrecommit(height int, round int) {
// If +2/3 prevoted for proposal block, stage and precommit it
if cs.ProposalBlock.HashesTo(hash) {
log.Info("EnterPrecommit: +2/3 prevoted proposal block. Locking", "hash", hash)
log.Info("enterPrecommit: +2/3 prevoted proposal block. Locking", "hash", hash)
// Validate the block.
if err := cs.stageBlock(cs.ProposalBlock, cs.ProposalBlockParts); err != nil {
PanicConsensus(Fmt("EnterPrecommit: +2/3 prevoted for an invalid block: %v", err))
PanicConsensus(Fmt("enterPrecommit: +2/3 prevoted for an invalid block: %v", err))
}
cs.LockedRound = round
cs.LockedBlock = cs.ProposalBlock
@ -1030,41 +1016,41 @@ func (cs *ConsensusState) EnterPrecommit(height int, round int) {
}
// Enter: any +2/3 precommits for next round.
func (cs *ConsensusState) EnterPrecommitWait(height int, round int) {
func (cs *ConsensusState) enterPrecommitWait(height int, round int) {
//cs.mtx.Lock()
//defer cs.mtx.Unlock()
if cs.Height != height || round < cs.Round || (cs.Round == round && RoundStepPrecommitWait <= cs.Step) {
log.Debug(Fmt("EnterPrecommitWait(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
log.Debug(Fmt("enterPrecommitWait(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
return
}
if !cs.Votes.Precommits(round).HasTwoThirdsAny() {
PanicSanity(Fmt("EnterPrecommitWait(%v/%v), but Precommits does not have any +2/3 votes", height, round))
PanicSanity(Fmt("enterPrecommitWait(%v/%v), but Precommits does not have any +2/3 votes", height, round))
}
log.Info(Fmt("EnterPrecommitWait(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
log.Info(Fmt("enterPrecommitWait(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
defer func() {
// Done EnterPrecommitWait:
// Done enterPrecommitWait:
cs.updateRoundStep(round, RoundStepPrecommitWait)
cs.newStep()
}()
// After `timeoutPrecommit0+timeoutPrecommitDelta*round`, EnterNewRound()
// After `timeoutPrecommit0+timeoutPrecommitDelta*round`, enterNewRound()
cs.scheduleTimeout(timeoutPrecommit0+timeoutPrecommitDelta*time.Duration(round), height, round, RoundStepPrecommitWait)
}
// Enter: +2/3 precommits for block
func (cs *ConsensusState) EnterCommit(height int, commitRound int) {
func (cs *ConsensusState) enterCommit(height int, commitRound int) {
//cs.mtx.Lock()
//defer cs.mtx.Unlock()
if cs.Height != height || RoundStepCommit <= cs.Step {
log.Debug(Fmt("EnterCommit(%v/%v): Invalid args. Current step: %v/%v/%v", height, commitRound, cs.Height, cs.Round, cs.Step))
log.Debug(Fmt("enterCommit(%v/%v): Invalid args. Current step: %v/%v/%v", height, commitRound, cs.Height, cs.Round, cs.Step))
return
}
log.Info(Fmt("EnterCommit(%v/%v). Current: %v/%v/%v", height, commitRound, cs.Height, cs.Round, cs.Step))
log.Info(Fmt("enterCommit(%v/%v). Current: %v/%v/%v", height, commitRound, cs.Height, cs.Round, cs.Step))
defer func() {
// Done Entercommit:
// Done enterCommit:
// keep ca.Round the same, it points to the right Precommits set.
cs.updateRoundStep(cs.Round, RoundStepCommit)
cs.CommitRound = commitRound
@ -1116,29 +1102,29 @@ func (cs *ConsensusState) tryFinalizeCommit(height int) {
return
}
// go
cs.FinalizeCommit(height)
cs.finalizeCommit(height)
}
// Increment height and goto RoundStepNewHeight
func (cs *ConsensusState) FinalizeCommit(height int) {
func (cs *ConsensusState) finalizeCommit(height int) {
//cs.mtx.Lock()
//defer cs.mtx.Unlock()
if cs.Height != height || cs.Step != RoundStepCommit {
log.Debug(Fmt("FinalizeCommit(%v): Invalid args. Current step: %v/%v/%v", height, cs.Height, cs.Round, cs.Step))
log.Debug(Fmt("finalizeCommit(%v): Invalid args. Current step: %v/%v/%v", height, cs.Height, cs.Round, cs.Step))
return
}
hash, header, ok := cs.Votes.Precommits(cs.CommitRound).TwoThirdsMajority()
if !ok {
PanicSanity(Fmt("Cannot FinalizeCommit, commit does not have two thirds majority"))
PanicSanity(Fmt("Cannot finalizeCommit, commit does not have two thirds majority"))
}
if !cs.ProposalBlockParts.HasHeader(header) {
PanicSanity(Fmt("Expected ProposalBlockParts header to be commit header"))
}
if !cs.ProposalBlock.HashesTo(hash) {
PanicSanity(Fmt("Cannot FinalizeCommit, ProposalBlock does not hash to commit hash"))
PanicSanity(Fmt("Cannot finalizeCommit, ProposalBlock does not hash to commit hash"))
}
if err := cs.stageBlock(cs.ProposalBlock, cs.ProposalBlockParts); err != nil {
PanicConsensus(Fmt("+2/3 committed an invalid block: %v", err))
@ -1148,7 +1134,8 @@ func (cs *ConsensusState) FinalizeCommit(height int) {
// We have the block, so stage/save/commit-vote.
cs.saveBlock(cs.ProposalBlock, cs.ProposalBlockParts, cs.Votes.Precommits(cs.CommitRound))
// call updateToState from handleTimeout
// NewHeightStep!
cs.updateToState(cs.stagedState)
// cs.StartTime is already set.
// Schedule Round0 to start soon.
@ -1199,7 +1186,7 @@ func (cs *ConsensusState) setProposal(proposal *types.Proposal) error {
}
// NOTE: block is not necessarily valid.
// This can trigger us to go into EnterPrevote asynchronously (before we timeout of propose) or to attempt to commit
// This can trigger us to go into enterPrevote asynchronously (before we timeout of propose) or to attempt to commit
func (cs *ConsensusState) addProposalBlockPart(height int, part *types.Part) (added bool, err error) {
//cs.mtx.Lock()
//defer cs.mtx.Unlock()
@ -1223,10 +1210,10 @@ func (cs *ConsensusState) addProposalBlockPart(height int, part *types.Part) (ad
var n int
var err error
cs.ProposalBlock = wire.ReadBinary(&types.Block{}, cs.ProposalBlockParts.GetReader(), types.MaxBlockSize, &n, &err).(*types.Block)
log.Info("Received complete proposal", "hash", cs.ProposalBlock.Hash())
log.Info("Received complete proposal", "hash", cs.ProposalBlock.Hash(), "round", cs.Proposal.Round)
if cs.Step == RoundStepPropose && cs.isProposalComplete() {
// Move onto the next step
cs.EnterPrevote(height, cs.Round)
cs.enterPrevote(height, cs.Round)
} else if cs.Step == RoundStepCommit {
// If we're waiting on the proposal block...
cs.tryFinalizeCommit(height)
@ -1237,14 +1224,14 @@ func (cs *ConsensusState) addProposalBlockPart(height int, part *types.Part) (ad
}
// Attempt to add the vote. if its a duplicate signature, dupeout the validator
func (cs *ConsensusState) tryAddVote(valIndex int, vote *types.Vote, peerKey string) (bool, error) {
added, _, err := cs.addVote(valIndex, vote, peerKey)
func (cs *ConsensusState) tryAddVote(valIndex int, vote *types.Vote, peerKey string) error {
_, _, err := cs.addVote(valIndex, vote, peerKey)
if err != nil {
// If the vote height is off, we'll just ignore it,
// But if it's a conflicting sig, broadcast evidence tx for slashing.
// If it's otherwise invalid, punish peer.
if err == ErrVoteHeightMismatch {
return added, err
return err
} else if _, ok := err.(*types.ErrVoteConflictingSignature); ok {
log.Warn("Found conflicting vote. Publish evidence")
/* TODO
@ -1255,14 +1242,14 @@ func (cs *ConsensusState) tryAddVote(valIndex int, vote *types.Vote, peerKey str
}
cs.mempool.BroadcastTx(evidenceTx) // shouldn't need to check returned err
*/
return added, err
return err
} else {
// Probably an invalid signature. Bad peer.
log.Warn("Error attempting to add vote", "error", err)
return added, ErrAddingVote
return ErrAddingVote
}
}
return added, nil
return nil
}
//-----------------------------------------------------------------------------
@ -1300,7 +1287,7 @@ func (cs *ConsensusState) addVote(valIndex int, vote *types.Vote, peerKey string
// First, unlock if prevotes is a valid POL.
// >> lockRound < POLRound <= unlockOrChangeLockRound (see spec)
// NOTE: If (lockRound < POLRound) but !(POLRound <= unlockOrChangeLockRound),
// we'll still EnterNewRound(H,vote.R) and EnterPrecommit(H,vote.R) to process it
// we'll still enterNewRound(H,vote.R) and enterPrecommit(H,vote.R) to process it
// there.
if (cs.LockedBlock != nil) && (cs.LockedRound < vote.Round) && (vote.Round <= cs.Round) {
hash, _, ok := prevotes.TwoThirdsMajority()
@ -1314,17 +1301,17 @@ func (cs *ConsensusState) addVote(valIndex int, vote *types.Vote, peerKey string
}
if cs.Round <= vote.Round && prevotes.HasTwoThirdsAny() {
// Round-skip over to PrevoteWait or goto Precommit.
cs.EnterNewRound(height, vote.Round) // if the vote is ahead of us
cs.enterNewRound(height, vote.Round) // if the vote is ahead of us
if prevotes.HasTwoThirdsMajority() {
cs.EnterPrecommit(height, vote.Round)
cs.enterPrecommit(height, vote.Round)
} else {
cs.EnterPrevote(height, vote.Round) // if the vote is ahead of us
cs.EnterPrevoteWait(height, vote.Round)
cs.enterPrevote(height, vote.Round) // if the vote is ahead of us
cs.enterPrevoteWait(height, vote.Round)
}
} else if cs.Proposal != nil && 0 <= cs.Proposal.POLRound && cs.Proposal.POLRound == vote.Round {
// If the proposal is now complete, enter prevote of cs.Round.
if cs.isProposalComplete() {
cs.EnterPrevote(height, cs.Round)
cs.enterPrevote(height, cs.Round)
}
}
case types.VoteTypePrecommit:
@ -1333,16 +1320,16 @@ func (cs *ConsensusState) addVote(valIndex int, vote *types.Vote, peerKey string
hash, _, ok := precommits.TwoThirdsMajority()
if ok {
if len(hash) == 0 {
cs.EnterNewRound(height, vote.Round+1)
cs.enterNewRound(height, vote.Round+1)
} else {
cs.EnterNewRound(height, vote.Round)
cs.EnterPrecommit(height, vote.Round)
cs.EnterCommit(height, vote.Round)
cs.enterNewRound(height, vote.Round)
cs.enterPrecommit(height, vote.Round)
cs.enterCommit(height, vote.Round)
}
} else if cs.Round <= vote.Round && precommits.HasTwoThirdsAny() {
cs.EnterNewRound(height, vote.Round)
cs.EnterPrecommit(height, vote.Round)
cs.EnterPrecommitWait(height, vote.Round)
cs.enterNewRound(height, vote.Round)
cs.enterPrecommit(height, vote.Round)
cs.enterPrecommitWait(height, vote.Round)
//}()
}
default:
@ -1376,7 +1363,7 @@ func (cs *ConsensusState) stageBlock(block *types.Block, blockParts *types.PartS
// Create a copy of the state for staging
stateCopy := cs.state.Copy()
stateCopy.SetFireable(cs.evc)
stateCopy.SetEventCache(cs.evc)
// Run the block on the State:
// + update validator sets
@ -1412,9 +1399,8 @@ func (cs *ConsensusState) signAddVote(type_ byte, hash []byte, header types.Part
}
vote, err := cs.signVote(type_, hash, header)
if err == nil {
// NOTE: store our index in the cs so we don't have to do this every time
// TODO: store our index in the cs so we don't have to do this every time
valIndex, _ := cs.Validators.GetByAddress(cs.privValidator.Address)
// _, _, err := cs.addVote(valIndex, vote, "")
cs.sendInternalMessage(msgInfo{&VoteMessage{valIndex, vote}, ""})
log.Notice("Signed and pushed vote", "height", cs.Height, "round", cs.Round, "vote", vote, "error", err)
return vote


+ 362
- 509
consensus/state_test.go
File diff suppressed because it is too large
View File


+ 11
- 1
events/events.go View File

@ -10,7 +10,7 @@ import (
// reactors and other modules should export
// this interface to become eventable
type Eventable interface {
SetFireable(Fireable)
SetEventSwitch(evsw *EventSwitch)
}
// an event switch or cache implements fireable
@ -123,6 +123,16 @@ func (evsw *EventSwitch) FireEvent(event string, data types.EventData) {
eventCell.FireEvent(data)
}
func (evsw *EventSwitch) SubscribeToEvent(receiver, eventID string, chanCap int) chan interface{} {
// listen for new round
ch := make(chan interface{}, chanCap)
evsw.AddListenerForEvent(receiver, eventID, func(data types.EventData) {
// NOTE: in production, evsw callbacks should be nonblocking.
ch <- data
})
return ch
}
//-----------------------------------------------------------------------------
// eventCell handles keeping track of listener callbacks for a given event.


+ 2
- 2
mempool/reactor.go View File

@ -25,7 +25,7 @@ const (
type MempoolReactor struct {
p2p.BaseReactor
Mempool *Mempool // TODO: un-expose
evsw events.Fireable
evsw *events.EventSwitch
}
func NewMempoolReactor(mempool *Mempool) *MempoolReactor {
@ -135,7 +135,7 @@ func (memR *MempoolReactor) broadcastTxRoutine(peer Peer) {
}
// implements events.Eventable
func (memR *MempoolReactor) SetFireable(evsw events.Fireable) {
func (memR *MempoolReactor) SetEventSwitch(evsw *events.EventSwitch) {
memR.evsw = evsw
}


+ 3
- 3
node/node.go View File

@ -94,7 +94,7 @@ func NewNode() *Node {
// add the event switch to all services
// they should all satisfy events.Eventable
SetFireable(eventSwitch, bcReactor, mempoolReactor, consensusReactor)
SetEventSwitch(eventSwitch, bcReactor, mempoolReactor, consensusReactor)
// run the profile server
profileHost := config.GetString("prof_laddr")
@ -133,9 +133,9 @@ func (n *Node) Stop() {
}
// Add the event switch to reactors, mempool, etc.
func SetFireable(evsw *events.EventSwitch, eventables ...events.Eventable) {
func SetEventSwitch(evsw *events.EventSwitch, eventables ...events.Eventable) {
for _, e := range eventables {
e.SetFireable(evsw)
e.SetEventSwitch(evsw)
}
}


+ 16
- 3
rpc/server/handlers.go View File

@ -206,7 +206,7 @@ func _jsonStringToArg(ty reflect.Type, arg string) (reflect.Value, error) {
// rpc.websocket
const (
writeChanCapacity = 20
writeChanCapacity = 1000
wsWriteTimeoutSeconds = 30 // each write times out after this
wsReadTimeoutSeconds = 30 // connection times out if we haven't received *anything* in this long, not even pings.
wsPingTickerSeconds = 10 // send a ping every PingTickerSeconds.
@ -289,7 +289,7 @@ func (wsc *WSConnection) readTimeoutRoutine() {
}
}
// Block trying to write to writeChan until service stops.
// Blocking write to writeChan until service stops.
func (wsc *WSConnection) writeRPCResponse(resp RPCResponse) {
select {
case <-wsc.Quit:
@ -298,6 +298,18 @@ func (wsc *WSConnection) writeRPCResponse(resp RPCResponse) {
}
}
// Nonblocking write.
func (wsc *WSConnection) tryWriteRPCResponse(resp RPCResponse) bool {
select {
case <-wsc.Quit:
return false
case wsc.writeChan <- resp:
return true
default:
return false
}
}
// Read from the socket and subscribe to or unsubscribe from events
func (wsc *WSConnection) readRoutine() {
// Do not close writeChan, to allow writeRPCResponse() to fail.
@ -340,8 +352,9 @@ func (wsc *WSConnection) readRoutine() {
} else {
log.Notice("Subscribe to event", "id", wsc.id, "event", event)
wsc.evsw.AddListenerForEvent(wsc.id, event, func(msg types.EventData) {
// NOTE: EventSwitch callbacks must be nonblocking
// NOTE: RPCResponses of subscribed events have id suffix "#event"
wsc.writeRPCResponse(NewRPCResponse(request.ID+"#event", ctypes.ResultEvent{event, msg}, ""))
wsc.tryWriteRPCResponse(NewRPCResponse(request.ID+"#event", ctypes.ResultEvent{event, msg}, ""))
})
continue
}


+ 2
- 3
state/state.go View File

@ -33,7 +33,7 @@ type State struct {
LastValidators *types.ValidatorSet
LastAppHash []byte
evc events.Fireable // typically an events.EventCache
evc *events.EventCache
}
func LoadState(db dbm.DB) *State {
@ -81,8 +81,7 @@ func (s *State) Save() {
s.db.Set(stateKey, buf.Bytes())
}
// Implements events.Eventable. Typically uses events.EventCache
func (s *State) SetFireable(evc events.Fireable) {
func (s *State) SetEventCache(evc *events.EventCache) {
s.mtx.Lock()
defer s.mtx.Unlock()


+ 6
- 2
types/block.go View File

@ -62,8 +62,12 @@ func (b *Block) ValidateBasic(chainID string, lastBlockHeight int, lastBlockHash
}
func (b *Block) FillHeader() {
b.LastValidationHash = b.LastValidation.Hash()
b.DataHash = b.Data.Hash()
if b.LastValidationHash == nil {
b.LastValidationHash = b.LastValidation.Hash()
}
if b.DataHash == nil {
b.DataHash = b.Data.Hash()
}
}
// Computes and returns the block hash.


+ 15
- 16
types/events.go View File

@ -1,8 +1,6 @@
package types
import (
"time"
"github.com/tendermint/go-wire"
)
@ -17,6 +15,7 @@ func EventStringFork() string { return "Fork" }
func EventStringNewBlock() string { return "NewBlock" }
func EventStringNewRound() string { return "NewRound" }
func EventStringNewRoundStep() string { return "NewRoundStep" }
func EventStringTimeoutPropose() string { return "TimeoutPropose" }
func EventStringCompleteProposal() string { return "CompleteProposal" }
func EventStringPolka() string { return "Polka" }
@ -72,21 +71,21 @@ type EventDataApp struct {
Data []byte `json:"bytes"`
}
// We fire the most recent round state that led to the event
// (ie. NewRound will have the previous rounds state)
type EventDataRoundState struct {
CurrentTime time.Time `json:"current_time"`
Height int `json:"height"`
Round int `json:"round"`
Step string `json:"step"`
StartTime time.Time `json:"start_time"`
CommitTime time.Time `json:"commit_time"`
Proposal *Proposal `json:"proposal"`
ProposalBlock *Block `json:"proposal_block"`
LockedRound int `json:"locked_round"`
LockedBlock *Block `json:"locked_block"`
POLRound int `json:"pol_round"`
Height int `json:"height"`
Round int `json:"round"`
Step string `json:"step"`
// private, not exposed to websockets
rs interface{}
}
func (edrs *EventDataRoundState) RoundState() interface{} {
return edrs.rs
}
func (edrs *EventDataRoundState) SetRoundState(rs interface{}) {
edrs.rs = rs
}
type EventDataVote struct {


Loading…
Cancel
Save