Browse Source

Added CommitTimeWait step, the state machine model changed a bit.

pull/9/head
Jae Kwon 10 years ago
parent
commit
89418ee7af
7 changed files with 244 additions and 93 deletions
  1. +130
    -35
      consensus/consensus.go
  2. +65
    -27
      consensus/state.go
  3. +35
    -12
      consensus/state_test.go
  4. +1
    -0
      consensus/vote_set.go
  5. +10
    -12
      state/state.go
  6. +2
    -7
      state/state_test.go
  7. +1
    -0
      state/validator_set.go

+ 130
- 35
consensus/consensus.go View File

@ -253,6 +253,13 @@ func (conR *ConsensusReactor) Receive(chId byte, peer *p2p.Peer, msgBytes []byte
}
conR.sw.Broadcast(StateCh, msg)
}
// Maybe run RoundActionCommitWait.
if vote.Type == VoteTypeCommit &&
rs.Commits.HasTwoThirdsMajority() &&
rs.Step < RoundStepCommit {
// NOTE: Do not call RunAction*() methods here directly.
conR.doActionCh <- RoundAction{rs.Height, rs.Round, RoundActionCommitWait}
}
}
default:
@ -304,10 +311,15 @@ func (conR *ConsensusReactor) stepTransitionRoutine() {
case RoundStepPrecommit:
// Wake up when the round is over.
time.Sleep(time.Duration(1.0-elapsedRatio) * roundDuration)
conR.doActionCh <- RoundAction{rs.Height, rs.Round, RoundActionCommit}
conR.doActionCh <- RoundAction{rs.Height, rs.Round, RoundActionNextRound}
case RoundStepCommit:
panic("Should not happen: RoundStepCommit waits until +2/3 commits.")
case RoundStepCommitWait:
// Wake up when it's time to finalize commit.
time.Sleep(time.Duration(1.0-elapsedRatio)*roundDuration + finalizeDuration)
if rs.CommitTime.IsZero() {
panic("RoundStepCommitWait requires rs.CommitTime")
}
time.Sleep(rs.CommitTime.Sub(time.Now()) + finalizeDuration)
conR.doActionCh <- RoundAction{rs.Height, rs.Round, RoundActionFinalize}
default:
panic("Should not happen")
@ -316,6 +328,11 @@ func (conR *ConsensusReactor) stepTransitionRoutine() {
scheduleNextAction()
// NOTE: All ConsensusState.RunAction*() calls must come from here.
// Since only one routine calls them, it is safe to assume that
// the RoundState Height/Round/Step won't change concurrently.
// However, other fields like Proposal could change, due to gossip.
ACTION_LOOP:
for {
roundAction := <-conR.doActionCh
@ -334,18 +351,30 @@ func (conR *ConsensusReactor) stepTransitionRoutine() {
conR.sw.Broadcast(StateCh, msg)
}
if height != rs.Height || round != rs.Round {
// Action is not relevant
// This may happen if an external routine
// pushes an action to conR.doActionCh.
return
// Continue if action is not relevant
if height != rs.Height {
continue
}
// If action >= RoundActionCommit, the round doesn't matter.
if action < RoundActionCommit && round != rs.Round {
continue
}
// Run step
if action == RoundActionPropose && rs.Step == RoundStepStart {
// Run action
switch action {
case RoundActionPropose:
if rs.Step != RoundStepStart {
continue ACTION_LOOP
}
conR.conS.RunActionPropose(rs.Height, rs.Round)
broadcastNewRoundStep(RoundStepPropose)
} else if action == RoundActionPrevote && rs.Step <= RoundStepPropose {
scheduleNextAction()
continue ACTION_LOOP
case RoundActionPrevote:
if rs.Step >= RoundStepPrevote {
continue ACTION_LOOP
}
hash := conR.conS.RunActionPrevote(rs.Height, rs.Round)
broadcastNewRoundStep(RoundStepPrevote)
conR.signAndBroadcastVote(rs, &Vote{
@ -354,7 +383,13 @@ func (conR *ConsensusReactor) stepTransitionRoutine() {
Type: VoteTypePrevote,
BlockHash: hash,
})
} else if action == RoundActionPrecommit && rs.Step <= RoundStepPrevote {
scheduleNextAction()
continue ACTION_LOOP
case RoundActionPrecommit:
if rs.Step >= RoundStepPrecommit {
continue ACTION_LOOP
}
hash := conR.conS.RunActionPrecommit(rs.Height, rs.Round)
broadcastNewRoundStep(RoundStepPrecommit)
if len(hash) > 0 {
@ -365,8 +400,23 @@ func (conR *ConsensusReactor) stepTransitionRoutine() {
BlockHash: hash,
})
}
} else if action == RoundActionCommit && rs.Step <= RoundStepPrecommit {
hash := conR.conS.RunActionCommit(rs.Height, rs.Round)
scheduleNextAction()
continue ACTION_LOOP
case RoundActionNextRound:
if rs.Step >= RoundStepCommit {
continue ACTION_LOOP
}
conR.conS.SetupRound(rs.Round + 1)
scheduleNextAction()
continue ACTION_LOOP
case RoundActionCommit:
if rs.Step >= RoundStepCommit {
continue ACTION_LOOP
}
// NOTE: Duplicated in RoundActionCommitWait.
hash := conR.conS.RunActionCommit(rs.Height)
if len(hash) > 0 {
broadcastNewRoundStep(RoundStepCommit)
conR.signAndBroadcastVote(rs, &Vote{
@ -376,17 +426,51 @@ func (conR *ConsensusReactor) stepTransitionRoutine() {
BlockHash: hash,
})
} else {
conR.conS.SetupRound(rs.Round + 1)
panic("This shouldn't happen")
}
// do not schedule next action.
continue ACTION_LOOP
case RoundActionCommitWait:
if rs.Step >= RoundStepCommitWait {
continue ACTION_LOOP
}
// First we must commit.
if rs.Step < RoundStepCommit {
// NOTE: Duplicated in RoundActionCommit.
hash := conR.conS.RunActionCommit(rs.Height)
if len(hash) > 0 {
broadcastNewRoundStep(RoundStepCommit)
conR.signAndBroadcastVote(rs, &Vote{
Height: rs.Height,
Round: rs.Round,
Type: VoteTypeCommit,
BlockHash: hash,
})
} else {
panic("This shouldn't happen")
}
}
// Now wait for more commit votes.
conR.conS.RunActionCommitWait(rs.Height)
scheduleNextAction()
continue ACTION_LOOP
case RoundActionFinalize:
if rs.Step != RoundStepCommitWait {
panic("This shouldn't happen")
}
} else if action == RoundActionFinalize && rs.Step == RoundStepCommit {
conR.conS.RunActionFinalize(rs.Height, rs.Round)
conR.conS.RunActionFinalize(rs.Height)
// Height has been incremented, step is now RoundStepStart.
} else {
// This shouldn't happen now, but if an external source pushes
// to conR.doActionCh, we might just want to continue here.
panic("Shouldn't happen")
scheduleNextAction()
continue ACTION_LOOP
default:
panic("Unknown action")
}
scheduleNextAction()
// For clarity, ensure that all switch cases call "continue"
panic("Should not happen.")
}
}
@ -404,6 +488,24 @@ OUTER_LOOP:
rs := conR.conS.GetRoundState()
prs := ps.GetRoundState()
// If ProposalBlockHash matches, send parts?
// NOTE: if we or peer is at RoundStepCommit*, the round
// won't necessarily match, but that's OK.
if rs.ProposalBlock.HashesTo(prs.ProposalBlockHash) {
if index, ok := rs.ProposalBlockPartSet.BitArray().Sub(
prs.ProposalBlockBitArray).PickRandom(); ok {
msg := &PartMessage{
Height: rs.Height,
Round: rs.Round,
Type: partTypeProposalBlock,
Part: rs.ProposalBlockPartSet.GetPart(uint16(index)),
}
peer.Send(DataCh, msg)
ps.SetHasProposalBlockPart(rs.Height, rs.Round, uint16(index))
continue OUTER_LOOP
}
}
// If height and round doesn't match, sleep.
if rs.Height != prs.Height || rs.Round != prs.Round {
time.Sleep(peerGossipSleepDuration)
@ -418,20 +520,6 @@ OUTER_LOOP:
continue OUTER_LOOP
}
// Send proposal block part?
if index, ok := rs.ProposalBlockPartSet.BitArray().Sub(
prs.ProposalBlockBitArray).PickRandom(); ok {
msg := &PartMessage{
Height: rs.Height,
Round: rs.Round,
Type: partTypeProposalBlock,
Part: rs.ProposalBlockPartSet.GetPart(uint16(index)),
}
peer.Send(DataCh, msg)
ps.SetHasProposalBlockPart(rs.Height, rs.Round, uint16(index))
continue OUTER_LOOP
}
// Send proposal POL part?
if index, ok := rs.ProposalPOLPartSet.BitArray().Sub(
prs.ProposalPOLBitArray).PickRandom(); ok {
@ -481,6 +569,10 @@ OUTER_LOOP:
msg := p2p.TypedMessage{msgTypeVote, vote}
peer.Send(VoteCh, msg)
ps.SetHasVote(rs.Height, rs.Round, VoteTypePrevote, uint32(index))
if vote.Type == VoteTypeCommit {
ps.SetHasVote(rs.Height, rs.Round, VoteTypePrecommit, uint32(index))
ps.SetHasVote(rs.Height, rs.Round, VoteTypeCommit, uint32(index))
}
continue OUTER_LOOP
} else {
log.Error("index is not a valid validator index")
@ -498,6 +590,9 @@ OUTER_LOOP:
msg := p2p.TypedMessage{msgTypeVote, vote}
peer.Send(VoteCh, msg)
ps.SetHasVote(rs.Height, rs.Round, VoteTypePrecommit, uint32(index))
if vote.Type == VoteTypeCommit {
ps.SetHasVote(rs.Height, rs.Round, VoteTypeCommit, uint32(index))
}
continue OUTER_LOOP
} else {
log.Error("index is not a valid validator index")


+ 65
- 27
consensus/state.go View File

@ -18,11 +18,12 @@ type RoundStep uint8
type RoundActionType uint8
const (
RoundStepStart = RoundStep(0x00) // Round started.
RoundStepPropose = RoundStep(0x01) // Did propose, gossip proposal.
RoundStepPrevote = RoundStep(0x02) // Did prevote, gossip prevotes.
RoundStepPrecommit = RoundStep(0x03) // Did precommit, gossip precommits.
RoundStepCommit = RoundStep(0x04) // Did commit, gossip commits.
RoundStepStart = RoundStep(0x00) // Round started.
RoundStepPropose = RoundStep(0x01) // Did propose, gossip proposal.
RoundStepPrevote = RoundStep(0x02) // Did prevote, gossip prevotes.
RoundStepPrecommit = RoundStep(0x03) // Did precommit, gossip precommits.
RoundStepCommit = RoundStep(0x04) // Did commit, gossip commits.
RoundStepCommitWait = RoundStep(0x05) // Found +2/3 commits, wait more.
// If a block could not be committed at a given round,
// we progress to the next round, skipping RoundStepCommit.
@ -30,12 +31,15 @@ const (
// If a block was committed, we goto RoundStepCommit,
// then wait "finalizeDuration" to gather more commits,
// then we progress to the next height at round 0.
RoundActionPropose = RoundActionType(0x00) // Goto RoundStepPropose
RoundActionPrevote = RoundActionType(0x01) // Goto RoundStepPrevote
RoundActionPrecommit = RoundActionType(0x02) // Goto RoundStepPrecommit
RoundActionCommit = RoundActionType(0x03) // Goto RoundStepCommit or RoundStepStart next round
RoundActionFinalize = RoundActionType(0x04) // Goto RoundStepStart next height
// TODO: document how RoundStepCommit transcends all rounds.
RoundActionPropose = RoundActionType(0x00) // Goto RoundStepPropose
RoundActionPrevote = RoundActionType(0x01) // Goto RoundStepPrevote
RoundActionPrecommit = RoundActionType(0x02) // Goto RoundStepPrecommit
RoundActionNextRound = RoundActionType(0x04) // Goto next round RoundStepStart
RoundActionCommit = RoundActionType(0x05) // Goto RoundStepCommit or RoundStepStart next round
RoundActionCommitWait = RoundActionType(0x06) // Goto RoundStepCommitWait
RoundActionFinalize = RoundActionType(0x07) // Goto RoundStepStart next height
)
var (
@ -50,6 +54,7 @@ type RoundState struct {
Round uint16
Step RoundStep
StartTime time.Time
CommitTime time.Time // Time when +2/3 commits were found
Validators *state.ValidatorSet
Proposal *Proposal
ProposalBlock *Block
@ -57,7 +62,8 @@ type RoundState struct {
ProposalPOL *POL
ProposalPOLPartSet *PartSet
LockedBlock *Block
LockedPOL *POL
LockedBlockPartSet *PartSet
LockedPOL *POL // Rarely needed, so no LockedPOLPartSet.
Prevotes *VoteSet
Precommits *VoteSet
Commits *VoteSet
@ -77,11 +83,12 @@ func (rs *RoundState) StringWithIndent(indent string) string {
return fmt.Sprintf(`RoundState{
%s H:%v R:%v S:%v
%s StartTime: %v
%s CommitTime: %v
%s Validators: %v
%s Proposal: %v
%s ProposalBlock: %v %v
%s ProposalPOL: %v %v
%s LockedBlock: %v
%s LockedBlock: %v %v
%s LockedPOL: %v
%s Prevotes: %v
%s Precommits: %v
@ -90,11 +97,12 @@ func (rs *RoundState) StringWithIndent(indent string) string {
%s}`,
indent, rs.Height, rs.Round, rs.Step,
indent, rs.StartTime,
indent, rs.CommitTime,
indent, rs.Validators.StringWithIndent(indent+" "),
indent, rs.Proposal,
indent, rs.ProposalBlockPartSet.Description(), rs.ProposalBlock.Description(),
indent, rs.ProposalPOLPartSet.Description(), rs.ProposalPOL.Description(),
indent, rs.LockedBlock.Description(),
indent, rs.LockedBlockPartSet.Description(), rs.LockedBlock.Description(),
indent, rs.LockedPOL.Description(),
indent, rs.Prevotes.StringWithIndent(indent+" "),
indent, rs.Precommits.StringWithIndent(indent+" "),
@ -146,7 +154,12 @@ func (cs *ConsensusState) updateToState(state *state.State) {
cs.Height = height
cs.Round = 0
cs.Step = RoundStepStart
cs.StartTime = state.CommitTime.Add(finalizeDuration)
if cs.CommitTime.IsZero() {
cs.StartTime = state.BlockTime.Add(finalizeDuration)
} else {
cs.StartTime = cs.CommitTime.Add(finalizeDuration)
}
cs.CommitTime = time.Time{}
cs.Validators = validators
cs.Proposal = nil
cs.ProposalBlock = nil
@ -154,6 +167,7 @@ func (cs *ConsensusState) updateToState(state *state.State) {
cs.ProposalPOL = nil
cs.ProposalPOLPartSet = nil
cs.LockedBlock = nil
cs.LockedBlockPartSet = nil
cs.LockedPOL = nil
cs.Prevotes = NewVoteSet(height, 0, VoteTypePrevote, validators)
cs.Precommits = NewVoteSet(height, 0, VoteTypePrecommit, validators)
@ -254,6 +268,7 @@ func (cs *ConsensusState) RunActionPropose(height uint32, round uint16) {
if cs.LockedBlock != nil {
// If we're locked onto a block, just choose that.
block = cs.LockedBlock
blockPartSet = cs.LockedBlockPartSet
pol = cs.LockedPOL
} else {
var validation Validation
@ -284,14 +299,12 @@ func (cs *ConsensusState) RunActionPropose(height uint32, round uint16) {
Txs: txs,
},
}
blockPartSet = NewPartSetFromData(BinaryBytes(block))
pol = cs.LockedPOL // If exists, is a PoUnlock.
}
blockPartSet = NewPartSetFromData(BinaryBytes(block))
if pol != nil {
polPartSet = NewPartSetFromData(BinaryBytes(pol))
} else {
}
// Make proposal
@ -423,6 +436,7 @@ func (cs *ConsensusState) RunActionPrecommit(height uint32, round uint16) []byte
if len(hash) == 0 {
// +2/3 prevoted nil. Just unlock.
cs.LockedBlock = nil
cs.LockedBlockPartSet = nil
return nil
} else if cs.ProposalBlock.HashesTo(hash) {
// +2/3 prevoted for proposal block
@ -433,15 +447,16 @@ func (cs *ConsensusState) RunActionPrecommit(height uint32, round uint16) []byte
return nil
}
cs.LockedBlock = cs.ProposalBlock
cs.LockedBlockPartSet = cs.ProposalBlockPartSet
return hash
} else if cs.LockedBlock.HashesTo(hash) {
// +2/3 prevoted for already locked block
// cs.LockedBlock = cs.LockedBlock
return hash
} else {
// We don't have the block that hashes to hash.
// Unlock if we're locked.
cs.LockedBlock = nil
cs.LockedBlockPartSet = nil
return nil
}
} else {
@ -454,15 +469,15 @@ func (cs *ConsensusState) RunActionPrecommit(height uint32, round uint16) []byte
// and returns the committed block.
// Commit is not finalized until FinalizeCommit() is called.
// This allows us to stay at this height and gather more commits.
func (cs *ConsensusState) RunActionCommit(height uint32, round uint16) []byte {
func (cs *ConsensusState) RunActionCommit(height uint32) []byte {
cs.mtx.Lock()
defer cs.mtx.Unlock()
if cs.Height != height || cs.Round != round {
if cs.Height != height {
return nil
}
cs.Step = RoundStepCommit
if hash, commitTime, ok := cs.Precommits.TwoThirdsMajority(); ok {
if hash, _, ok := cs.Precommits.TwoThirdsMajority(); ok {
// There are some strange cases that shouldn't happen
// (unless voters are duplicitous).
@ -473,10 +488,13 @@ func (cs *ConsensusState) RunActionCommit(height uint32, round uint16) []byte {
// TODO: Identify these strange cases.
var block *Block
var blockPartSet *PartSet
if cs.LockedBlock.HashesTo(hash) {
block = cs.LockedBlock
blockPartSet = cs.LockedBlockPartSet
} else if cs.ProposalBlock.HashesTo(hash) {
block = cs.ProposalBlock
blockPartSet = cs.ProposalBlockPartSet
} else {
return nil
}
@ -487,11 +505,17 @@ func (cs *ConsensusState) RunActionCommit(height uint32, round uint16) []byte {
return nil
}
// Keep block in cs.Proposal*
if !cs.ProposalBlock.HashesTo(hash) {
cs.ProposalBlock = block
cs.ProposalBlockPartSet = blockPartSet
}
// Save to blockStore
cs.blockStore.SaveBlock(block)
// Save the state
cs.stagedState.Save(commitTime)
cs.stagedState.Save()
// Update mempool.
cs.mempool.ResetForBlockAndState(block, cs.stagedState)
@ -502,12 +526,26 @@ func (cs *ConsensusState) RunActionCommit(height uint32, round uint16) []byte {
return nil
}
// After TryCommit(), if successful, must call this in order to
// update the RoundState.
func (cs *ConsensusState) RunActionFinalize(height uint32, round uint16) {
func (cs *ConsensusState) RunActionCommitWait(height uint32) {
cs.mtx.Lock()
defer cs.mtx.Unlock()
if cs.Height != height || cs.Round != round {
if cs.Height != height {
return
}
cs.Step = RoundStepCommitWait
if _, commitTime, ok := cs.Commits.TwoThirdsMajority(); ok {
// Remember the commitTime.
cs.CommitTime = commitTime
} else {
panic("RunActionCommitWait() expects +2/3 commits")
}
}
func (cs *ConsensusState) RunActionFinalize(height uint32) {
cs.mtx.Lock()
defer cs.mtx.Unlock()
if cs.Height != height {
return
}


+ 35
- 12
consensus/state_test.go View File

@ -18,7 +18,7 @@ func randAccountDetail(id uint64, status byte) (*state.AccountDetail, *state.Pri
return &state.AccountDetail{
Account: account,
Sequence: RandUInt(),
Balance: RandUInt64() + 1000, // At least 1000.
Balance: 1000,
Status: status,
}, privAccount
}
@ -38,7 +38,7 @@ func randGenesisState(numAccounts int, numValidators int) (*state.State, []*stat
}
}
s0 := state.GenesisState(db, time.Now(), accountDetails)
s0.Save(time.Now())
s0.Save()
return s0, privAccounts
}
@ -145,7 +145,7 @@ func checkRoundState(t *testing.T, cs *ConsensusState,
}
}
func TestRunActionPrecommit(t *testing.T) {
func TestRunActionPrecommitCommitFinalize(t *testing.T) {
cs, privAccounts := makeConsensusState()
priv := NewPrivValidator(privAccounts[0], db_.NewMemDB())
cs.SetPrivValidator(priv)
@ -175,7 +175,7 @@ func TestRunActionPrecommit(t *testing.T) {
for i := 0; i < 7; i++ {
vote := &Vote{
Height: 1,
Round: uint16(i),
Round: 0,
Type: VoteTypePrevote,
BlockHash: cs.ProposalBlock.Hash(),
}
@ -191,15 +191,11 @@ func TestRunActionPrecommit(t *testing.T) {
checkRoundState(t, cs, 1, 0, RoundStepPrecommit)
// Test RunActionCommit failures:
blockHash = cs.RunActionCommit(1, 1)
if blockHash != nil {
t.Errorf("RunActionCommit should fail for wrong round")
}
blockHash = cs.RunActionCommit(2, 0)
blockHash = cs.RunActionCommit(2)
if blockHash != nil {
t.Errorf("RunActionCommit should fail for wrong height")
}
blockHash = cs.RunActionCommit(1, 0)
blockHash = cs.RunActionCommit(1)
if blockHash != nil {
t.Errorf("RunActionCommit should fail, not enough commits")
}
@ -208,7 +204,7 @@ func TestRunActionPrecommit(t *testing.T) {
for i := 0; i < 7; i++ {
vote := &Vote{
Height: 1,
Round: uint16(i),
Round: 0,
Type: VoteTypePrecommit,
BlockHash: cs.ProposalBlock.Hash(),
}
@ -217,10 +213,37 @@ func TestRunActionPrecommit(t *testing.T) {
}
// Test RunActionCommit success:
blockHash = cs.RunActionCommit(1, 0)
blockHash = cs.RunActionCommit(1)
if len(blockHash) == 0 {
t.Errorf("RunActionCommit should have succeeded")
}
checkRoundState(t, cs, 1, 0, RoundStepCommit)
// cs.CommitTime should still be zero
if !cs.CommitTime.IsZero() {
t.Errorf("Expected CommitTime to yet be zero")
}
// Add at least +2/3 commits.
for i := 0; i < 7; i++ {
vote := &Vote{
Height: 1,
Round: uint16(i), // Doesn't matter what round
Type: VoteTypeCommit,
BlockHash: cs.ProposalBlock.Hash(),
}
privAccounts[i].Sign(vote)
cs.AddVote(vote)
}
// Test RunActionCommitWait:
cs.RunActionCommitWait(1)
if cs.CommitTime.IsZero() {
t.Errorf("Expected CommitTime to have been set")
}
checkRoundState(t, cs, 1, 0, RoundStepCommitWait)
// Test RunActionFinalize:
cs.RunActionFinalize(1)
checkRoundState(t, cs, 2, 0, RoundStepStart)
}

+ 1
- 0
consensus/vote_set.go View File

@ -17,6 +17,7 @@ import (
// Note that there three kinds of votes: prevotes, precommits, and commits.
// A commit of prior rounds can be added added in lieu of votes/precommits.
// TODO: test majority calculations etc.
// NOTE: assumes that the sum total of voting power does not exceed MaxUInt64.
type VoteSet struct {
height uint32
round uint16


+ 10
- 12
state/state.go View File

@ -43,9 +43,9 @@ func (txErr InvalidTxError) Error() string {
// NOTE: not goroutine-safe.
type State struct {
DB db_.DB
Height uint32 // Last known block height
BlockHash []byte // Last known block hash
CommitTime time.Time
Height uint32 // Last known block height
BlockHash []byte // Last known block hash
BlockTime time.Time // LastKnown block time
BondedValidators *ValidatorSet
UnbondingValidators *ValidatorSet
accountDetails merkle.Tree // Shouldn't be accessed directly.
@ -77,7 +77,7 @@ func GenesisState(db db_.DB, genesisTime time.Time, accDets []*AccountDetail) *S
DB: db,
Height: 0,
BlockHash: nil,
CommitTime: genesisTime,
BlockTime: genesisTime,
BondedValidators: NewValidatorSet(validators),
UnbondingValidators: NewValidatorSet(nil),
accountDetails: accountDetails,
@ -94,8 +94,8 @@ func LoadState(db db_.DB) *State {
var n int64
var err error
s.Height = ReadUInt32(reader, &n, &err)
s.CommitTime = ReadTime(reader, &n, &err)
s.BlockHash = ReadByteSlice(reader, &n, &err)
s.BlockTime = ReadTime(reader, &n, &err)
s.BondedValidators = ReadValidatorSet(reader, &n, &err)
s.UnbondingValidators = ReadValidatorSet(reader, &n, &err)
accountDetailsHash := ReadByteSlice(reader, &n, &err)
@ -110,17 +110,14 @@ func LoadState(db db_.DB) *State {
}
// Save this state into the db.
// For convenience, the commitTime (required by ConsensusAgent)
// is saved here.
func (s *State) Save(commitTime time.Time) {
s.CommitTime = commitTime
func (s *State) Save() {
s.accountDetails.Save()
var buf bytes.Buffer
var n int64
var err error
WriteUInt32(&buf, s.Height, &n, &err)
WriteTime(&buf, commitTime, &n, &err)
WriteByteSlice(&buf, s.BlockHash, &n, &err)
WriteTime(&buf, s.BlockTime, &n, &err)
WriteBinary(&buf, s.BondedValidators, &n, &err)
WriteBinary(&buf, s.UnbondingValidators, &n, &err)
WriteByteSlice(&buf, s.accountDetails.Hash(), &n, &err)
@ -134,8 +131,8 @@ func (s *State) Copy() *State {
return &State{
DB: s.DB,
Height: s.Height,
CommitTime: s.CommitTime,
BlockHash: s.BlockHash,
BlockTime: s.BlockTime,
BondedValidators: s.BondedValidators.Copy(),
UnbondingValidators: s.UnbondingValidators.Copy(),
accountDetails: s.accountDetails.Copy(),
@ -397,6 +394,7 @@ func (s *State) AppendBlock(b *Block, checkStateHash bool) error {
s.Height = b.Height
s.BlockHash = b.Hash()
s.BlockTime = b.Time
return nil
}
@ -418,7 +416,7 @@ func (s *State) SetAccountDetail(accDet *AccountDetail) (updated bool) {
}
// Returns a hash that represents the state data,
// excluding Height, BlockHash, and CommitTime.
// excluding Height, BlockHash.
func (s *State) Hash() []byte {
hashables := []merkle.Hashable{
s.BondedValidators,


+ 2
- 7
state/state_test.go View File

@ -38,7 +38,7 @@ func randGenesisState(numAccounts int, numValidators int) (*State, []*PrivAccoun
}
}
s0 := GenesisState(db, time.Now(), accountDetails)
s0.Save(time.Now())
s0.Save()
return s0, privAccounts
}
@ -110,8 +110,7 @@ func TestGenesisSaveLoad(t *testing.T) {
}
// Save s0
commitTime := time.Now()
s0.Save(commitTime)
s0.Save()
// Sanity check s0
//s0.DB.(*db_.MemDB).Print()
@ -125,10 +124,6 @@ func TestGenesisSaveLoad(t *testing.T) {
// Load s1
s1 := LoadState(s0.DB)
// Compare CommitTime
if !s0.CommitTime.Equal(s1.CommitTime) {
t.Error("CommitTime was not the same", s0.CommitTime, s1.CommitTime)
}
// Compare height & blockHash
if s0.Height != s1.Height {
t.Error("Height mismatch")


+ 1
- 0
state/validator_set.go View File

@ -10,6 +10,7 @@ import (
)
// Not goroutine-safe.
// TODO: consider validator Accum overflow?
type ValidatorSet struct {
validators merkle.Tree
proposer *Validator // Whoever has the highest Accum.


Loading…
Cancel
Save