mirror of
https://github.com/fluencelabs/tendermint
synced 2025-06-13 21:31:23 +00:00
Added LastBlockParts to state, and also validates.
This commit is contained in:
@ -14,12 +14,6 @@ import (
|
||||
"github.com/tendermint/tendermint/merkle"
|
||||
)
|
||||
|
||||
var (
|
||||
ErrBlockInvalidNetwork = errors.New("Error block invalid network")
|
||||
ErrBlockInvalidBlockHeight = errors.New("Error block invalid height")
|
||||
ErrBlockInvalidLastBlockHash = errors.New("Error block invalid last blockhash")
|
||||
)
|
||||
|
||||
type Block struct {
|
||||
Header
|
||||
Validation
|
||||
@ -45,17 +39,23 @@ func (b *Block) WriteTo(w io.Writer) (n int64, err error) {
|
||||
}
|
||||
|
||||
// Basic validation that doesn't involve state data.
|
||||
func (b *Block) ValidateBasic(lastBlockHeight uint32, lastBlockHash []byte) error {
|
||||
if b.Header.Network != Config.Network {
|
||||
return ErrBlockInvalidNetwork
|
||||
func (b *Block) ValidateBasic(lastBlockHeight uint32, lastBlockHash []byte,
|
||||
lastBlockParts PartSetHeader, lastBlockTime time.Time) error {
|
||||
if b.Network != Config.Network {
|
||||
return errors.New("Invalid block network")
|
||||
}
|
||||
if b.Header.Height != lastBlockHeight+1 {
|
||||
return ErrBlockInvalidBlockHeight
|
||||
if b.Height != lastBlockHeight+1 {
|
||||
return errors.New("Invalid block height")
|
||||
}
|
||||
if !bytes.Equal(b.Header.LastBlockHash, lastBlockHash) {
|
||||
return ErrBlockInvalidLastBlockHash
|
||||
if !bytes.Equal(b.LastBlockHash, lastBlockHash) {
|
||||
return errors.New("Invalid block hash")
|
||||
}
|
||||
if !b.LastBlockParts.Equals(lastBlockParts) {
|
||||
return errors.New("Invalid block parts header")
|
||||
}
|
||||
if !b.Time.After(lastBlockTime) {
|
||||
return errors.New("Invalid block time")
|
||||
}
|
||||
// XXX We need to validate LastBlockParts too.
|
||||
// XXX more validation
|
||||
return nil
|
||||
}
|
||||
|
@ -126,6 +126,7 @@ type PartSet struct {
|
||||
}
|
||||
|
||||
// Returns an immutable, full PartSet.
|
||||
// TODO Name is confusing, Data/Header clash with Block.Data/Header
|
||||
func NewPartSetFromData(data []byte) *PartSet {
|
||||
// divide data into 4kb parts.
|
||||
total := (len(data) + partSize - 1) / partSize
|
||||
@ -156,6 +157,7 @@ func NewPartSetFromData(data []byte) *PartSet {
|
||||
}
|
||||
|
||||
// Returns an empty PartSet ready to be populated.
|
||||
// TODO Name is confusing, Data/Header clash with Block.Data/Header
|
||||
func NewPartSetFromHeader(header PartSetHeader) *PartSet {
|
||||
return &PartSet{
|
||||
total: header.Total,
|
||||
|
@ -305,6 +305,7 @@ OUTER_LOOP:
|
||||
}
|
||||
}
|
||||
|
||||
// XXX Need to also send commits for LastComits.
|
||||
func (conR *ConsensusReactor) gossipVotesRoutine(peer *p2p.Peer, ps *PeerState) {
|
||||
OUTER_LOOP:
|
||||
for {
|
||||
|
@ -374,19 +374,19 @@ ACTION_LOOP:
|
||||
// Otherwise the round is 0 and cs.Step becomes RoundStepNewHeight.
|
||||
func (cs *ConsensusState) updateToState(state *state.State) {
|
||||
// Sanity check state.
|
||||
if cs.Height > 0 && cs.Height != state.Height {
|
||||
if cs.Height > 0 && cs.Height != state.LastBlockHeight {
|
||||
Panicf("updateToState() expected state height of %v but found %v",
|
||||
cs.Height, state.Height)
|
||||
cs.Height, state.LastBlockHeight)
|
||||
}
|
||||
|
||||
// Reset fields based on state.
|
||||
validators := state.BondedValidators
|
||||
height := state.Height + 1 // next desired block height
|
||||
height := state.LastBlockHeight + 1 // next desired block height
|
||||
cs.Height = height
|
||||
cs.Round = 0
|
||||
cs.Step = RoundStepNewHeight
|
||||
if cs.CommitTime.IsZero() {
|
||||
cs.StartTime = state.BlockTime.Add(newHeightDelta)
|
||||
cs.StartTime = state.LastBlockTime.Add(newHeightDelta)
|
||||
} else {
|
||||
cs.StartTime = cs.CommitTime.Add(newHeightDelta)
|
||||
}
|
||||
@ -509,11 +509,13 @@ func (cs *ConsensusState) RunActionPropose(height uint32, round uint16) {
|
||||
txs, state := cs.mempool.GetProposalTxs() // TODO: cache state
|
||||
block = &Block{
|
||||
Header: Header{
|
||||
Network: Config.Network,
|
||||
Height: cs.Height,
|
||||
Time: time.Now(),
|
||||
LastBlockHash: cs.state.BlockHash,
|
||||
StateHash: state.Hash(),
|
||||
Network: Config.Network,
|
||||
Height: cs.Height,
|
||||
Time: time.Now(),
|
||||
Fees: 0, // TODO fees
|
||||
LastBlockHash: cs.state.LastBlockHash,
|
||||
LastBlockParts: cs.state.LastBlockParts,
|
||||
StateHash: state.Hash(),
|
||||
},
|
||||
Validation: validation,
|
||||
Data: Data{
|
||||
@ -564,7 +566,7 @@ func (cs *ConsensusState) RunActionPrevote(height uint32, round uint16) {
|
||||
}
|
||||
|
||||
// Try staging cs.ProposalBlock
|
||||
err := cs.stageBlock(cs.ProposalBlock)
|
||||
err := cs.stageBlock(cs.ProposalBlock, cs.ProposalBlockParts)
|
||||
if err != nil {
|
||||
// ProposalBlock is invalid, prevote nil.
|
||||
cs.signAddVote(VoteTypePrevote, nil, PartSetHeader{})
|
||||
@ -613,7 +615,7 @@ func (cs *ConsensusState) RunActionPrecommit(height uint32, round uint16) {
|
||||
// If +2/3 prevoted for cs.ProposalBlock, lock it and precommit it.
|
||||
if cs.ProposalBlock.HashesTo(hash) {
|
||||
// Validate the block.
|
||||
if err := cs.stageBlock(cs.ProposalBlock); err != nil {
|
||||
if err := cs.stageBlock(cs.ProposalBlock, cs.ProposalBlockParts); err != nil {
|
||||
// Prevent zombies.
|
||||
log.Warning("+2/3 prevoted for an invalid block: %v", err)
|
||||
return
|
||||
@ -631,6 +633,7 @@ func (cs *ConsensusState) RunActionPrecommit(height uint32, round uint16) {
|
||||
return
|
||||
}
|
||||
|
||||
// XXX Need to send new message to peers to get the right parts.
|
||||
// Enter commit step. See the diagram for details.
|
||||
func (cs *ConsensusState) RunActionCommit(height uint32) {
|
||||
cs.mtx.Lock()
|
||||
@ -653,6 +656,7 @@ func (cs *ConsensusState) RunActionCommit(height uint32) {
|
||||
|
||||
// Clear the Locked* fields and use cs.Proposed*
|
||||
if cs.LockedBlock.HashesTo(hash) {
|
||||
// XXX maybe just use CommitBlock* instead. Proposals need to be signed...
|
||||
cs.ProposalBlock = cs.LockedBlock
|
||||
cs.ProposalBlockParts = cs.LockedBlockParts
|
||||
cs.LockedBlock = nil
|
||||
@ -707,7 +711,7 @@ func (cs *ConsensusState) TryFinalizeCommit(height uint32) bool {
|
||||
Panicf("Expected ProposalBlockParts header to be commit header")
|
||||
}
|
||||
|
||||
err := cs.stageBlock(cs.ProposalBlock)
|
||||
err := cs.stageBlock(cs.ProposalBlock, cs.ProposalBlockParts)
|
||||
if err == nil {
|
||||
// Increment height.
|
||||
cs.updateToState(cs.stagedState)
|
||||
@ -838,7 +842,7 @@ func (cs *ConsensusState) AddVote(vote *Vote) (added bool, err error) {
|
||||
|
||||
//-----------------------------------------------------------------------------
|
||||
|
||||
func (cs *ConsensusState) stageBlock(block *Block) error {
|
||||
func (cs *ConsensusState) stageBlock(block *Block, blockParts *PartSet) error {
|
||||
if block == nil {
|
||||
panic("Cannot stage nil block")
|
||||
}
|
||||
@ -853,7 +857,7 @@ func (cs *ConsensusState) stageBlock(block *Block) error {
|
||||
|
||||
// Commit block onto the copied state.
|
||||
// NOTE: Basic validation is done in state.AppendBlock().
|
||||
err := stateCopy.AppendBlock(block, true)
|
||||
err := stateCopy.AppendBlock(block, blockParts.Header(), true)
|
||||
if err != nil {
|
||||
return err
|
||||
} else {
|
||||
@ -882,7 +886,7 @@ func (cs *ConsensusState) signAddVote(type_ byte, hash []byte, header PartSetHea
|
||||
func (cs *ConsensusState) processBlockForCommit(block *Block, blockParts *PartSet) {
|
||||
|
||||
// The proposal must be valid.
|
||||
if err := cs.stageBlock(block); err != nil {
|
||||
if err := cs.stageBlock(block, blockParts); err != nil {
|
||||
// Prevent zombies.
|
||||
log.Warning("+2/3 precommitted an invalid block: %v", err)
|
||||
return
|
||||
|
@ -6,6 +6,7 @@ import (
|
||||
"time"
|
||||
|
||||
. "github.com/tendermint/tendermint/binary"
|
||||
. "github.com/tendermint/tendermint/blocks"
|
||||
. "github.com/tendermint/tendermint/common"
|
||||
db_ "github.com/tendermint/tendermint/db"
|
||||
"github.com/tendermint/tendermint/merkle"
|
||||
@ -65,9 +66,10 @@ func GenesisState(db db_.DB, genesisTime time.Time, accDets []*AccountDetail) *S
|
||||
|
||||
return &State{
|
||||
DB: db,
|
||||
Height: 0,
|
||||
BlockHash: nil,
|
||||
BlockTime: genesisTime,
|
||||
LastBlockHeight: 0,
|
||||
LastBlockHash: nil,
|
||||
LastBlockParts: PartSetHeader{},
|
||||
LastBlockTime: genesisTime,
|
||||
BondedValidators: NewValidatorSet(validators),
|
||||
UnbondingValidators: NewValidatorSet(nil),
|
||||
accountDetails: accountDetails,
|
||||
|
@ -43,9 +43,10 @@ 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
|
||||
BlockTime time.Time // LastKnown block time
|
||||
LastBlockHeight uint32
|
||||
LastBlockHash []byte
|
||||
LastBlockParts PartSetHeader
|
||||
LastBlockTime time.Time
|
||||
BondedValidators *ValidatorSet
|
||||
UnbondingValidators *ValidatorSet
|
||||
accountDetails merkle.Tree // Shouldn't be accessed directly.
|
||||
@ -60,9 +61,10 @@ func LoadState(db db_.DB) *State {
|
||||
reader := bytes.NewReader(buf)
|
||||
var n int64
|
||||
var err error
|
||||
s.Height = ReadUInt32(reader, &n, &err)
|
||||
s.BlockHash = ReadByteSlice(reader, &n, &err)
|
||||
s.BlockTime = ReadTime(reader, &n, &err)
|
||||
s.LastBlockHeight = ReadUInt32(reader, &n, &err)
|
||||
s.LastBlockHash = ReadByteSlice(reader, &n, &err)
|
||||
s.LastBlockParts = ReadPartSetHeader(reader, &n, &err)
|
||||
s.LastBlockTime = ReadTime(reader, &n, &err)
|
||||
s.BondedValidators = ReadValidatorSet(reader, &n, &err)
|
||||
s.UnbondingValidators = ReadValidatorSet(reader, &n, &err)
|
||||
accountDetailsHash := ReadByteSlice(reader, &n, &err)
|
||||
@ -82,9 +84,10 @@ func (s *State) Save() {
|
||||
var buf bytes.Buffer
|
||||
var n int64
|
||||
var err error
|
||||
WriteUInt32(&buf, s.Height, &n, &err)
|
||||
WriteByteSlice(&buf, s.BlockHash, &n, &err)
|
||||
WriteTime(&buf, s.BlockTime, &n, &err)
|
||||
WriteUInt32(&buf, s.LastBlockHeight, &n, &err)
|
||||
WriteByteSlice(&buf, s.LastBlockHash, &n, &err)
|
||||
WriteBinary(&buf, s.LastBlockParts, &n, &err)
|
||||
WriteTime(&buf, s.LastBlockTime, &n, &err)
|
||||
WriteBinary(&buf, s.BondedValidators, &n, &err)
|
||||
WriteBinary(&buf, s.UnbondingValidators, &n, &err)
|
||||
WriteByteSlice(&buf, s.accountDetails.Hash(), &n, &err)
|
||||
@ -97,9 +100,10 @@ func (s *State) Save() {
|
||||
func (s *State) Copy() *State {
|
||||
return &State{
|
||||
DB: s.DB,
|
||||
Height: s.Height,
|
||||
BlockHash: s.BlockHash,
|
||||
BlockTime: s.BlockTime,
|
||||
LastBlockHeight: s.LastBlockHeight,
|
||||
LastBlockHash: s.LastBlockHash,
|
||||
LastBlockParts: s.LastBlockParts,
|
||||
LastBlockTime: s.LastBlockTime,
|
||||
BondedValidators: s.BondedValidators.Copy(),
|
||||
UnbondingValidators: s.UnbondingValidators.Copy(),
|
||||
accountDetails: s.accountDetails.Copy(),
|
||||
@ -172,7 +176,7 @@ func (s *State) ExecTx(tx Tx) error {
|
||||
s.SetAccountDetail(accDet)
|
||||
added := s.BondedValidators.Add(&Validator{
|
||||
Account: accDet.Account,
|
||||
BondHeight: s.Height,
|
||||
BondHeight: s.LastBlockHeight,
|
||||
VotingPower: accDet.Balance,
|
||||
Accum: 0,
|
||||
})
|
||||
@ -260,7 +264,7 @@ func (s *State) unbondValidator(accountId uint64, accDet *AccountDetail) {
|
||||
if !removed {
|
||||
panic("Failed to remove validator")
|
||||
}
|
||||
val.UnbondHeight = s.Height
|
||||
val.UnbondHeight = s.LastBlockHeight
|
||||
added := s.UnbondingValidators.Add(val)
|
||||
if !added {
|
||||
panic("Failed to add validator")
|
||||
@ -286,26 +290,25 @@ func (s *State) releaseValidator(accountId uint64) {
|
||||
// (used for constructing a new proposal)
|
||||
// NOTE: If an error occurs during block execution, state will be left
|
||||
// at an invalid state. Copy the state before calling AppendBlock!
|
||||
func (s *State) AppendBlock(b *Block, checkStateHash bool) error {
|
||||
func (s *State) AppendBlock(block *Block, blockPartsHeader PartSetHeader, checkStateHash bool) error {
|
||||
// Basic block validation.
|
||||
// XXX We need to validate LastBlockParts too.
|
||||
err := b.ValidateBasic(s.Height, s.BlockHash)
|
||||
err := block.ValidateBasic(s.LastBlockHeight, s.LastBlockHash, s.LastBlockParts, s.LastBlockTime)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Validate block Validation.
|
||||
if b.Height == 1 {
|
||||
if len(b.Validation.Commits) != 0 {
|
||||
if block.Height == 1 {
|
||||
if len(block.Validation.Commits) != 0 {
|
||||
return errors.New("Block at height 1 (first block) should have no Validation commits")
|
||||
}
|
||||
} else {
|
||||
if uint(len(b.Validation.Commits)) != s.BondedValidators.Size() {
|
||||
if uint(len(block.Validation.Commits)) != s.BondedValidators.Size() {
|
||||
return errors.New("Invalid block validation size")
|
||||
}
|
||||
var sumVotingPower uint64
|
||||
s.BondedValidators.Iterate(func(index uint, val *Validator) bool {
|
||||
rsig := b.Validation.Commits[index]
|
||||
rsig := block.Validation.Commits[index]
|
||||
if rsig.IsZero() {
|
||||
return false
|
||||
} else {
|
||||
@ -314,11 +317,11 @@ func (s *State) AppendBlock(b *Block, checkStateHash bool) error {
|
||||
return true
|
||||
}
|
||||
vote := &Vote{
|
||||
Height: b.Height,
|
||||
Height: block.Height,
|
||||
Round: rsig.Round,
|
||||
Type: VoteTypeCommit,
|
||||
BlockHash: b.LastBlockHash,
|
||||
BlockParts: b.LastBlockParts,
|
||||
BlockHash: block.LastBlockHash,
|
||||
BlockParts: block.LastBlockParts,
|
||||
Signature: rsig.Signature,
|
||||
}
|
||||
if val.Verify(vote) {
|
||||
@ -339,7 +342,7 @@ func (s *State) AppendBlock(b *Block, checkStateHash bool) error {
|
||||
}
|
||||
|
||||
// Commit each tx
|
||||
for _, tx := range b.Data.Txs {
|
||||
for _, tx := range block.Data.Txs {
|
||||
err := s.ExecTx(tx)
|
||||
if err != nil {
|
||||
return InvalidTxError{tx, err}
|
||||
@ -347,12 +350,12 @@ func (s *State) AppendBlock(b *Block, checkStateHash bool) error {
|
||||
}
|
||||
|
||||
// Update Validator.LastCommitHeight as necessary.
|
||||
for _, rsig := range b.Validation.Commits {
|
||||
for _, rsig := range block.Validation.Commits {
|
||||
_, val := s.BondedValidators.GetById(rsig.SignerId)
|
||||
if val == nil {
|
||||
return ErrStateInvalidSignature
|
||||
}
|
||||
val.LastCommitHeight = b.Height
|
||||
val.LastCommitHeight = block.Height
|
||||
updated := s.BondedValidators.Update(val)
|
||||
if !updated {
|
||||
panic("Failed to update validator LastCommitHeight")
|
||||
@ -363,7 +366,7 @@ func (s *State) AppendBlock(b *Block, checkStateHash bool) error {
|
||||
// reward account with bonded coins.
|
||||
toRelease := []*Validator{}
|
||||
s.UnbondingValidators.Iterate(func(index uint, val *Validator) bool {
|
||||
if val.UnbondHeight+unbondingPeriodBlocks < b.Height {
|
||||
if val.UnbondHeight+unbondingPeriodBlocks < block.Height {
|
||||
toRelease = append(toRelease, val)
|
||||
}
|
||||
return false
|
||||
@ -376,7 +379,7 @@ func (s *State) AppendBlock(b *Block, checkStateHash bool) error {
|
||||
// unbond them, they have timed out.
|
||||
toTimeout := []*Validator{}
|
||||
s.BondedValidators.Iterate(func(index uint, val *Validator) bool {
|
||||
if val.LastCommitHeight+validatorTimeoutBlocks < b.Height {
|
||||
if val.LastCommitHeight+validatorTimeoutBlocks < block.Height {
|
||||
toTimeout = append(toTimeout, val)
|
||||
}
|
||||
return false
|
||||
@ -392,21 +395,22 @@ func (s *State) AppendBlock(b *Block, checkStateHash bool) error {
|
||||
stateHash := s.Hash()
|
||||
if checkStateHash {
|
||||
// State hash should match
|
||||
if !bytes.Equal(stateHash, b.StateHash) {
|
||||
if !bytes.Equal(stateHash, block.StateHash) {
|
||||
return Errorf("Invalid state hash. Got %X, block says %X",
|
||||
stateHash, b.StateHash)
|
||||
stateHash, block.StateHash)
|
||||
}
|
||||
} else {
|
||||
// Set the state hash.
|
||||
if b.StateHash != nil {
|
||||
if block.StateHash != nil {
|
||||
panic("Cannot overwrite block.StateHash")
|
||||
}
|
||||
b.StateHash = stateHash
|
||||
block.StateHash = stateHash
|
||||
}
|
||||
|
||||
s.Height = b.Height
|
||||
s.BlockHash = b.Hash()
|
||||
s.BlockTime = b.Time
|
||||
s.LastBlockHeight = block.Height
|
||||
s.LastBlockHash = block.Hash()
|
||||
s.LastBlockParts = blockPartsHeader
|
||||
s.LastBlockTime = block.Time
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -428,7 +432,7 @@ func (s *State) SetAccountDetail(accDet *AccountDetail) (updated bool) {
|
||||
}
|
||||
|
||||
// Returns a hash that represents the state data,
|
||||
// excluding Height, BlockHash.
|
||||
// excluding LastBlock*
|
||||
func (s *State) Hash() []byte {
|
||||
hashables := []merkle.Hashable{
|
||||
s.BondedValidators,
|
||||
|
@ -1,6 +1,7 @@
|
||||
package state
|
||||
|
||||
import (
|
||||
. "github.com/tendermint/tendermint/binary"
|
||||
. "github.com/tendermint/tendermint/blocks"
|
||||
. "github.com/tendermint/tendermint/common"
|
||||
. "github.com/tendermint/tendermint/config"
|
||||
@ -85,18 +86,23 @@ func TestGenesisSaveLoad(t *testing.T) {
|
||||
// Mutate the state to append one empty block.
|
||||
block := &Block{
|
||||
Header: Header{
|
||||
Network: Config.Network,
|
||||
Height: 1,
|
||||
StateHash: nil,
|
||||
Network: Config.Network,
|
||||
Height: 1,
|
||||
Time: s0.LastBlockTime.Add(time.Minute),
|
||||
Fees: 0,
|
||||
LastBlockHash: s0.LastBlockHash,
|
||||
LastBlockParts: s0.LastBlockParts,
|
||||
StateHash: nil,
|
||||
},
|
||||
Validation: Validation{},
|
||||
Data: Data{
|
||||
Txs: []Tx{},
|
||||
},
|
||||
}
|
||||
blockParts := NewPartSetFromData(BinaryBytes(block))
|
||||
// The second argument to AppendBlock() is false,
|
||||
// which sets Block.Header.StateHash.
|
||||
err := s0.Copy().AppendBlock(block, false)
|
||||
err := s0.Copy().AppendBlock(block, blockParts.Header(), false)
|
||||
if err != nil {
|
||||
t.Error("Error appending initial block:", err)
|
||||
}
|
||||
@ -105,7 +111,7 @@ func TestGenesisSaveLoad(t *testing.T) {
|
||||
}
|
||||
// Now append the block to s0.
|
||||
// This time we also check the StateHash (as computed above).
|
||||
err = s0.AppendBlock(block, true)
|
||||
err = s0.AppendBlock(block, blockParts.Header(), true)
|
||||
if err != nil {
|
||||
t.Error("Error appending initial block:", err)
|
||||
}
|
||||
@ -118,19 +124,19 @@ func TestGenesisSaveLoad(t *testing.T) {
|
||||
if s0.BondedValidators.TotalVotingPower() == 0 {
|
||||
t.Error("s0 BondedValidators TotalVotingPower should not be 0")
|
||||
}
|
||||
if s0.Height != 1 {
|
||||
t.Error("s0 Height should be 1, got", s0.Height)
|
||||
if s0.LastBlockHeight != 1 {
|
||||
t.Error("s0 LastBlockHeight should be 1, got", s0.LastBlockHeight)
|
||||
}
|
||||
|
||||
// Load s1
|
||||
s1 := LoadState(s0.DB)
|
||||
|
||||
// Compare height & blockHash
|
||||
if s0.Height != s1.Height {
|
||||
t.Error("Height mismatch")
|
||||
if s0.LastBlockHeight != s1.LastBlockHeight {
|
||||
t.Error("LastBlockHeight mismatch")
|
||||
}
|
||||
if !bytes.Equal(s0.BlockHash, s1.BlockHash) {
|
||||
t.Error("BlockHash mismatch")
|
||||
if !bytes.Equal(s0.LastBlockHash, s1.LastBlockHash) {
|
||||
t.Error("LastBlockHash mismatch")
|
||||
}
|
||||
// Compare state merkle trees
|
||||
if s0.BondedValidators.Size() != s1.BondedValidators.Size() {
|
||||
@ -165,7 +171,7 @@ func TestTxSequence(t *testing.T) {
|
||||
acc1 := state.GetAccountDetail(1) // Non-validator
|
||||
|
||||
// Try executing a SendTx with various sequence numbers.
|
||||
stx := &SendTx{
|
||||
stxProto := SendTx{
|
||||
BaseTx: BaseTx{
|
||||
Sequence: acc1.Sequence + 1,
|
||||
Fee: 0},
|
||||
@ -176,6 +182,8 @@ func TestTxSequence(t *testing.T) {
|
||||
// Test a variety of sequence numbers for the tx.
|
||||
// The tx should only pass when i == 1.
|
||||
for i := -1; i < 3; i++ {
|
||||
stxCopy := stxProto
|
||||
stx := &stxCopy
|
||||
stx.Sequence = uint(int(acc1.Sequence) + i)
|
||||
privAccounts[1].Sign(stx)
|
||||
stateCopy := state.Copy()
|
||||
@ -285,9 +293,9 @@ func TestTxs(t *testing.T) {
|
||||
if acc1Val == nil {
|
||||
t.Errorf("acc1Val not present")
|
||||
}
|
||||
if acc1Val.BondHeight != state.Height {
|
||||
if acc1Val.BondHeight != state.LastBlockHeight {
|
||||
t.Errorf("Unexpected bond height. Expected %v, got %v",
|
||||
state.Height, acc1Val.BondHeight)
|
||||
state.LastBlockHeight, acc1Val.BondHeight)
|
||||
}
|
||||
if acc1Val.VotingPower != acc1.Balance {
|
||||
t.Errorf("Unexpected voting power. Expected %v, got %v",
|
||||
|
Reference in New Issue
Block a user