mirror of
https://github.com/fluencelabs/tendermint
synced 2025-06-24 18:21:38 +00:00
Don't send bitarrays, just send height/round/type/index.
This commit is contained in:
@ -10,14 +10,15 @@ import (
|
|||||||
"time"
|
"time"
|
||||||
|
|
||||||
. "github.com/tendermint/tendermint/binary"
|
. "github.com/tendermint/tendermint/binary"
|
||||||
|
. "github.com/tendermint/tendermint/common"
|
||||||
. "github.com/tendermint/tendermint/config"
|
. "github.com/tendermint/tendermint/config"
|
||||||
"github.com/tendermint/tendermint/merkle"
|
"github.com/tendermint/tendermint/merkle"
|
||||||
)
|
)
|
||||||
|
|
||||||
type Block struct {
|
type Block struct {
|
||||||
Header
|
*Header
|
||||||
Validation
|
*Validation
|
||||||
Data
|
*Data
|
||||||
|
|
||||||
// Volatile
|
// Volatile
|
||||||
hash []byte
|
hash []byte
|
||||||
@ -32,9 +33,9 @@ func ReadBlock(r io.Reader, n *int64, err *error) *Block {
|
|||||||
}
|
}
|
||||||
|
|
||||||
func (b *Block) WriteTo(w io.Writer) (n int64, err error) {
|
func (b *Block) WriteTo(w io.Writer) (n int64, err error) {
|
||||||
WriteBinary(w, &b.Header, &n, &err)
|
WriteBinary(w, b.Header, &n, &err)
|
||||||
WriteBinary(w, &b.Validation, &n, &err)
|
WriteBinary(w, b.Validation, &n, &err)
|
||||||
WriteBinary(w, &b.Data, &n, &err)
|
WriteBinary(w, b.Data, &n, &err)
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -125,11 +126,11 @@ type Header struct {
|
|||||||
hash []byte
|
hash []byte
|
||||||
}
|
}
|
||||||
|
|
||||||
func ReadHeader(r io.Reader, n *int64, err *error) (h Header) {
|
func ReadHeader(r io.Reader, n *int64, err *error) *Header {
|
||||||
if *err != nil {
|
if *err != nil {
|
||||||
return Header{}
|
return nil
|
||||||
}
|
}
|
||||||
return Header{
|
return &Header{
|
||||||
Network: ReadString(r, n, err),
|
Network: ReadString(r, n, err),
|
||||||
Height: ReadUInt32(r, n, err),
|
Height: ReadUInt32(r, n, err),
|
||||||
Time: ReadTime(r, n, err),
|
Time: ReadTime(r, n, err),
|
||||||
@ -190,10 +191,11 @@ type Validation struct {
|
|||||||
|
|
||||||
// Volatile
|
// Volatile
|
||||||
hash []byte
|
hash []byte
|
||||||
|
bitArray BitArray
|
||||||
}
|
}
|
||||||
|
|
||||||
func ReadValidation(r io.Reader, n *int64, err *error) Validation {
|
func ReadValidation(r io.Reader, n *int64, err *error) *Validation {
|
||||||
return Validation{
|
return &Validation{
|
||||||
Commits: ReadRoundSignatures(r, n, err),
|
Commits: ReadRoundSignatures(r, n, err),
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -226,6 +228,16 @@ func (v *Validation) StringWithIndent(indent string) string {
|
|||||||
indent, v.hash)
|
indent, v.hash)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (v *Validation) BitArray() BitArray {
|
||||||
|
if v.bitArray.IsZero() {
|
||||||
|
v.bitArray = NewBitArray(uint(len(v.Commits)))
|
||||||
|
for i, rsig := range v.Commits {
|
||||||
|
v.bitArray.SetIndex(uint(i), !rsig.IsZero())
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return v.bitArray
|
||||||
|
}
|
||||||
|
|
||||||
//-----------------------------------------------------------------------------
|
//-----------------------------------------------------------------------------
|
||||||
|
|
||||||
type Data struct {
|
type Data struct {
|
||||||
@ -235,13 +247,13 @@ type Data struct {
|
|||||||
hash []byte
|
hash []byte
|
||||||
}
|
}
|
||||||
|
|
||||||
func ReadData(r io.Reader, n *int64, err *error) Data {
|
func ReadData(r io.Reader, n *int64, err *error) *Data {
|
||||||
numTxs := ReadUInt32(r, n, err)
|
numTxs := ReadUInt32(r, n, err)
|
||||||
txs := make([]Tx, 0, numTxs)
|
txs := make([]Tx, 0, numTxs)
|
||||||
for i := uint32(0); i < numTxs; i++ {
|
for i := uint32(0); i < numTxs; i++ {
|
||||||
txs = append(txs, ReadTx(r, n, err))
|
txs = append(txs, ReadTx(r, n, err))
|
||||||
}
|
}
|
||||||
return Data{Txs: txs}
|
return &Data{Txs: txs}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (data *Data) WriteTo(w io.Writer) (n int64, err error) {
|
func (data *Data) WriteTo(w io.Writer) (n int64, err error) {
|
||||||
|
@ -59,7 +59,7 @@ func randBlock() *Block {
|
|||||||
|
|
||||||
// Block
|
// Block
|
||||||
block := &Block{
|
block := &Block{
|
||||||
Header: Header{
|
Header: &Header{
|
||||||
Network: "Tendermint",
|
Network: "Tendermint",
|
||||||
Height: RandUInt32Exp(),
|
Height: RandUInt32Exp(),
|
||||||
Fees: RandUInt64Exp(),
|
Fees: RandUInt64Exp(),
|
||||||
@ -67,10 +67,10 @@ func randBlock() *Block {
|
|||||||
LastBlockHash: RandBytes(32),
|
LastBlockHash: RandBytes(32),
|
||||||
StateHash: RandBytes(32),
|
StateHash: RandBytes(32),
|
||||||
},
|
},
|
||||||
Validation: Validation{
|
Validation: &Validation{
|
||||||
Commits: []RoundSignature{randRoundSig(), randRoundSig()},
|
Commits: []RoundSignature{randRoundSig(), randRoundSig()},
|
||||||
},
|
},
|
||||||
Data: Data{
|
Data: &Data{
|
||||||
Txs: []Tx{sendTx, nameTx, bondTx, unbondTx, dupeoutTx},
|
Txs: []Tx{sendTx, nameTx, bondTx, unbondTx, dupeoutTx},
|
||||||
},
|
},
|
||||||
}
|
}
|
||||||
|
@ -71,6 +71,16 @@ func (bs *BlockStore) LoadBlockPart(height uint32, index uint16) *Part {
|
|||||||
return part
|
return part
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (bs *BlockStore) LoadBlockMeta(height uint32) *BlockMeta {
|
||||||
|
var n int64
|
||||||
|
var err error
|
||||||
|
meta := ReadBlockMeta(bs.GetReader(calcBlockMetaKey(height)), &n, &err)
|
||||||
|
if err != nil {
|
||||||
|
Panicf("Error reading block meta: %v", err)
|
||||||
|
}
|
||||||
|
return meta
|
||||||
|
}
|
||||||
|
|
||||||
func (bs *BlockStore) LoadBlockValidation(height uint32) *Validation {
|
func (bs *BlockStore) LoadBlockValidation(height uint32) *Validation {
|
||||||
var n int64
|
var n int64
|
||||||
var err error
|
var err error
|
||||||
@ -78,7 +88,7 @@ func (bs *BlockStore) LoadBlockValidation(height uint32) *Validation {
|
|||||||
if err != nil {
|
if err != nil {
|
||||||
Panicf("Error reading validation: %v", err)
|
Panicf("Error reading validation: %v", err)
|
||||||
}
|
}
|
||||||
return &validation
|
return validation
|
||||||
}
|
}
|
||||||
|
|
||||||
func (bs *BlockStore) SaveBlock(block *Block, blockParts *PartSet) {
|
func (bs *BlockStore) SaveBlock(block *Block, blockParts *PartSet) {
|
||||||
@ -89,7 +99,11 @@ func (bs *BlockStore) SaveBlock(block *Block, blockParts *PartSet) {
|
|||||||
if !blockParts.IsComplete() {
|
if !blockParts.IsComplete() {
|
||||||
Panicf("BlockStore can only save complete block part sets")
|
Panicf("BlockStore can only save complete block part sets")
|
||||||
}
|
}
|
||||||
meta := BlockMeta{Hash: block.Hash(), Parts: blockParts.Header()}
|
meta := &BlockMeta{
|
||||||
|
Hash: block.Hash(),
|
||||||
|
Parts: blockParts.Header(),
|
||||||
|
Header: block.Header,
|
||||||
|
}
|
||||||
// Save block meta
|
// Save block meta
|
||||||
metaBytes := BinaryBytes(meta)
|
metaBytes := BinaryBytes(meta)
|
||||||
bs.db.Set(calcBlockMetaKey(height), metaBytes)
|
bs.db.Set(calcBlockMetaKey(height), metaBytes)
|
||||||
@ -98,7 +112,7 @@ func (bs *BlockStore) SaveBlock(block *Block, blockParts *PartSet) {
|
|||||||
bs.saveBlockPart(height, i, blockParts.GetPart(i))
|
bs.saveBlockPart(height, i, blockParts.GetPart(i))
|
||||||
}
|
}
|
||||||
// Save block validation (duplicate and separate)
|
// Save block validation (duplicate and separate)
|
||||||
validationBytes := BinaryBytes(&block.Validation)
|
validationBytes := BinaryBytes(block.Validation)
|
||||||
bs.db.Set(calcBlockValidationKey(height), validationBytes)
|
bs.db.Set(calcBlockValidationKey(height), validationBytes)
|
||||||
// Save new BlockStoreJSON descriptor
|
// Save new BlockStoreJSON descriptor
|
||||||
BlockStoreJSON{Height: height}.Save(bs.db)
|
BlockStoreJSON{Height: height}.Save(bs.db)
|
||||||
@ -115,20 +129,23 @@ func (bs *BlockStore) saveBlockPart(height uint32, index uint16, part *Part) {
|
|||||||
//-----------------------------------------------------------------------------
|
//-----------------------------------------------------------------------------
|
||||||
|
|
||||||
type BlockMeta struct {
|
type BlockMeta struct {
|
||||||
Hash []byte
|
Hash []byte // The BlockHash
|
||||||
Parts PartSetHeader
|
Parts PartSetHeader // The PartSetHeader, for transfer
|
||||||
|
Header *Header // The block's Header
|
||||||
}
|
}
|
||||||
|
|
||||||
func ReadBlockMeta(r io.Reader, n *int64, err *error) BlockMeta {
|
func ReadBlockMeta(r io.Reader, n *int64, err *error) *BlockMeta {
|
||||||
return BlockMeta{
|
return &BlockMeta{
|
||||||
Hash: ReadByteSlice(r, n, err),
|
Hash: ReadByteSlice(r, n, err),
|
||||||
Parts: ReadPartSetHeader(r, n, err),
|
Parts: ReadPartSetHeader(r, n, err),
|
||||||
|
Header: ReadHeader(r, n, err),
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (bm BlockMeta) WriteTo(w io.Writer) (n int64, err error) {
|
func (bm *BlockMeta) WriteTo(w io.Writer) (n int64, err error) {
|
||||||
WriteByteSlice(w, bm.Hash, &n, &err)
|
WriteByteSlice(w, bm.Hash, &n, &err)
|
||||||
WriteBinary(w, bm.Parts, &n, &err)
|
WriteBinary(w, bm.Parts, &n, &err)
|
||||||
|
WriteBinary(w, bm.Header, &n, &err)
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -60,6 +60,10 @@ func (bA BitArray) WriteTo(w io.Writer) (n int64, err error) {
|
|||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (bA BitArray) Size() uint {
|
||||||
|
return bA.bits
|
||||||
|
}
|
||||||
|
|
||||||
func (bA BitArray) IsZero() bool {
|
func (bA BitArray) IsZero() bool {
|
||||||
return bA.bits == 0
|
return bA.bits == 0
|
||||||
}
|
}
|
||||||
|
@ -25,7 +25,6 @@ const (
|
|||||||
peerStateKey = "ConsensusReactor.peerState"
|
peerStateKey = "ConsensusReactor.peerState"
|
||||||
|
|
||||||
peerGossipSleepDuration = 50 * time.Millisecond // Time to sleep if there's nothing to send.
|
peerGossipSleepDuration = 50 * time.Millisecond // Time to sleep if there's nothing to send.
|
||||||
hasVotesThreshold = 50 // After this many new votes we'll send a HasVotesMessage.
|
|
||||||
)
|
)
|
||||||
|
|
||||||
//-----------------------------------------------------------------------------
|
//-----------------------------------------------------------------------------
|
||||||
@ -113,7 +112,6 @@ func (conR *ConsensusReactor) Receive(chId byte, peer *p2p.Peer, msgBytes []byte
|
|||||||
rs := conR.conS.GetRoundState()
|
rs := conR.conS.GetRoundState()
|
||||||
ps := peer.Data.Get(peerStateKey).(*PeerState)
|
ps := peer.Data.Get(peerStateKey).(*PeerState)
|
||||||
_, msg_ := decodeMessage(msgBytes)
|
_, msg_ := decodeMessage(msgBytes)
|
||||||
voteAddCounter := 0
|
|
||||||
var err error = nil
|
var err error = nil
|
||||||
|
|
||||||
log.Debug("[%X][%v] Receive: %v", chId, peer, msg_)
|
log.Debug("[%X][%v] Receive: %v", chId, peer, msg_)
|
||||||
@ -129,9 +127,9 @@ func (conR *ConsensusReactor) Receive(chId byte, peer *p2p.Peer, msgBytes []byte
|
|||||||
msg := msg_.(*CommitMessage)
|
msg := msg_.(*CommitMessage)
|
||||||
ps.ApplyCommitMessage(msg)
|
ps.ApplyCommitMessage(msg)
|
||||||
|
|
||||||
case *HasVotesMessage:
|
case *HasVoteMessage:
|
||||||
msg := msg_.(*HasVotesMessage)
|
msg := msg_.(*HasVoteMessage)
|
||||||
ps.ApplyHasVotesMessage(msg)
|
ps.ApplyHasVoteMessage(msg)
|
||||||
|
|
||||||
default:
|
default:
|
||||||
// Ignore unknown message
|
// Ignore unknown message
|
||||||
@ -164,37 +162,22 @@ func (conR *ConsensusReactor) Receive(chId byte, peer *p2p.Peer, msgBytes []byte
|
|||||||
switch msg_.(type) {
|
switch msg_.(type) {
|
||||||
case *Vote:
|
case *Vote:
|
||||||
vote := msg_.(*Vote)
|
vote := msg_.(*Vote)
|
||||||
// We can't deal with votes from another height,
|
added, index, err := conR.conS.AddVote(vote)
|
||||||
// as they have a different validator set.
|
|
||||||
if vote.Height != rs.Height || vote.Height != ps.Height {
|
|
||||||
return
|
|
||||||
}
|
|
||||||
index, val := rs.Validators.GetById(vote.SignerId)
|
|
||||||
if val == nil {
|
|
||||||
log.Warning("Peer gave us an invalid vote.")
|
|
||||||
return
|
|
||||||
}
|
|
||||||
ps.EnsureVoteBitArrays(rs.Height, rs.Round, rs.Validators.Size())
|
|
||||||
ps.SetHasVote(rs.Height, rs.Round, index, vote)
|
|
||||||
added, err := conR.conS.AddVote(vote)
|
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Warning("Error attempting to add vote: %v", err)
|
log.Warning("Error attempting to add vote: %v", err)
|
||||||
}
|
}
|
||||||
|
// Initialize Prevotes/Precommits/Commits if needed
|
||||||
|
ps.EnsureVoteBitArrays(rs.Height, rs.Validators.Size())
|
||||||
|
ps.SetHasVote(vote, index)
|
||||||
if added {
|
if added {
|
||||||
// Maybe send HasVotesMessage
|
msg := &HasVoteMessage{
|
||||||
// TODO optimize. It would be better to just acks for each vote!
|
Height: vote.Height,
|
||||||
voteAddCounter++
|
Round: vote.Round,
|
||||||
if voteAddCounter%hasVotesThreshold == 0 {
|
Type: vote.Type,
|
||||||
msg := &HasVotesMessage{
|
Index: index,
|
||||||
Height: rs.Height,
|
|
||||||
Round: rs.Round,
|
|
||||||
Prevotes: rs.Prevotes.BitArray(),
|
|
||||||
Precommits: rs.Precommits.BitArray(),
|
|
||||||
Commits: rs.Commits.BitArray(),
|
|
||||||
}
|
}
|
||||||
conR.sw.Broadcast(StateCh, msg)
|
conR.sw.Broadcast(StateCh, msg)
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
|
||||||
default:
|
default:
|
||||||
// Ignore unknown message
|
// Ignore unknown message
|
||||||
@ -320,7 +303,6 @@ OUTER_LOOP:
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// XXX Need to also send commits for LastComits.
|
|
||||||
func (conR *ConsensusReactor) gossipVotesRoutine(peer *p2p.Peer, ps *PeerState) {
|
func (conR *ConsensusReactor) gossipVotesRoutine(peer *p2p.Peer, ps *PeerState) {
|
||||||
OUTER_LOOP:
|
OUTER_LOOP:
|
||||||
for {
|
for {
|
||||||
@ -332,15 +314,6 @@ OUTER_LOOP:
|
|||||||
rs := conR.conS.GetRoundState()
|
rs := conR.conS.GetRoundState()
|
||||||
prs := ps.GetRoundState()
|
prs := ps.GetRoundState()
|
||||||
|
|
||||||
// If height doesn't match, sleep.
|
|
||||||
if rs.Height != prs.Height {
|
|
||||||
time.Sleep(peerGossipSleepDuration)
|
|
||||||
continue OUTER_LOOP
|
|
||||||
}
|
|
||||||
|
|
||||||
// Ensure that peer's prevote/precommit/commit bitarrays of of sufficient capacity
|
|
||||||
ps.EnsureVoteBitArrays(rs.Height, rs.Round, rs.Validators.Size())
|
|
||||||
|
|
||||||
trySendVote := func(voteSet *VoteSet, peerVoteSet BitArray) (sent bool) {
|
trySendVote := func(voteSet *VoteSet, peerVoteSet BitArray) (sent bool) {
|
||||||
// TODO: give priority to our vote.
|
// TODO: give priority to our vote.
|
||||||
index, ok := voteSet.BitArray().Sub(peerVoteSet).PickRandom()
|
index, ok := voteSet.BitArray().Sub(peerVoteSet).PickRandom()
|
||||||
@ -349,12 +322,27 @@ OUTER_LOOP:
|
|||||||
// NOTE: vote may be a commit.
|
// NOTE: vote may be a commit.
|
||||||
msg := p2p.TypedMessage{msgTypeVote, vote}
|
msg := p2p.TypedMessage{msgTypeVote, vote}
|
||||||
peer.Send(VoteCh, msg)
|
peer.Send(VoteCh, msg)
|
||||||
ps.SetHasVote(rs.Height, rs.Round, index, vote)
|
ps.SetHasVote(vote, index)
|
||||||
return true
|
return true
|
||||||
}
|
}
|
||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// If height matches, then send LastCommits, Prevotes, Precommits, or Commits.
|
||||||
|
if rs.Height == prs.Height {
|
||||||
|
|
||||||
|
// If there are lastcommits to send...
|
||||||
|
if prs.Round == 0 && prs.Step == RoundStepNewHeight {
|
||||||
|
if prs.LastCommits.Size() == rs.LastCommits.Size() {
|
||||||
|
if trySendVote(rs.LastCommits, prs.LastCommits) {
|
||||||
|
continue OUTER_LOOP
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Initialize Prevotes/Precommits/Commits if needed
|
||||||
|
ps.EnsureVoteBitArrays(rs.Height, rs.Validators.Size())
|
||||||
|
|
||||||
// If there are prevotes to send...
|
// If there are prevotes to send...
|
||||||
if rs.Round == prs.Round && prs.Step <= RoundStepPrevote {
|
if rs.Round == prs.Round && prs.Step <= RoundStepPrevote {
|
||||||
if trySendVote(rs.Prevotes, prs.Prevotes) {
|
if trySendVote(rs.Prevotes, prs.Prevotes) {
|
||||||
@ -373,6 +361,50 @@ OUTER_LOOP:
|
|||||||
if trySendVote(rs.Commits, prs.Commits) {
|
if trySendVote(rs.Commits, prs.Commits) {
|
||||||
continue OUTER_LOOP
|
continue OUTER_LOOP
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// If peer is lagging by height 1, match our LastCommits to peer's Commits.
|
||||||
|
if rs.Height == prs.Height+1 {
|
||||||
|
|
||||||
|
// Initialize Commits if needed
|
||||||
|
ps.EnsureVoteBitArrays(rs.Height-1, rs.LastCommits.Size())
|
||||||
|
|
||||||
|
// If there are lastcommits to send...
|
||||||
|
if trySendVote(rs.LastCommits, prs.Commits) {
|
||||||
|
continue OUTER_LOOP
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
// If peer is lagging by more than 1, load and send Validation and send Commits.
|
||||||
|
if rs.Height >= prs.Height+2 {
|
||||||
|
|
||||||
|
// Load the block header and validation for prs.Height+1,
|
||||||
|
// which contains commit signatures for prs.Height.
|
||||||
|
header, validation := conR.conS.LoadHeaderValidation(prs.Height + 1)
|
||||||
|
size := uint(len(validation.Commits))
|
||||||
|
|
||||||
|
// Initialize Commits if needed
|
||||||
|
ps.EnsureVoteBitArrays(prs.Height, size)
|
||||||
|
|
||||||
|
index, ok := validation.BitArray().Sub(prs.Commits).PickRandom()
|
||||||
|
if ok {
|
||||||
|
rsig := validation.Commits[index]
|
||||||
|
// Reconstruct vote.
|
||||||
|
vote := &Vote{
|
||||||
|
Height: prs.Height,
|
||||||
|
Round: rsig.Round,
|
||||||
|
Type: VoteTypeCommit,
|
||||||
|
BlockHash: header.LastBlockHash,
|
||||||
|
BlockParts: header.LastBlockParts,
|
||||||
|
Signature: rsig.Signature,
|
||||||
|
}
|
||||||
|
msg := p2p.TypedMessage{msgTypeVote, vote}
|
||||||
|
peer.Send(VoteCh, msg)
|
||||||
|
ps.SetHasVote(vote, index)
|
||||||
|
continue OUTER_LOOP
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
// We sent nothing. Sleep...
|
// We sent nothing. Sleep...
|
||||||
time.Sleep(peerGossipSleepDuration)
|
time.Sleep(peerGossipSleepDuration)
|
||||||
@ -396,6 +428,7 @@ type PeerRoundState struct {
|
|||||||
Prevotes BitArray // All votes peer has for this round
|
Prevotes BitArray // All votes peer has for this round
|
||||||
Precommits BitArray // All precommits peer has for this round
|
Precommits BitArray // All precommits peer has for this round
|
||||||
Commits BitArray // All commits peer has for this height
|
Commits BitArray // All commits peer has for this height
|
||||||
|
LastCommits BitArray // All commits peer has for last height
|
||||||
}
|
}
|
||||||
|
|
||||||
//-----------------------------------------------------------------------------
|
//-----------------------------------------------------------------------------
|
||||||
@ -463,11 +496,11 @@ func (ps *PeerState) SetHasProposalPOLPart(height uint32, round uint16, index ui
|
|||||||
ps.ProposalPOLBitArray.SetIndex(uint(index), true)
|
ps.ProposalPOLBitArray.SetIndex(uint(index), true)
|
||||||
}
|
}
|
||||||
|
|
||||||
func (ps *PeerState) EnsureVoteBitArrays(height uint32, round uint16, numValidators uint) {
|
func (ps *PeerState) EnsureVoteBitArrays(height uint32, numValidators uint) {
|
||||||
ps.mtx.Lock()
|
ps.mtx.Lock()
|
||||||
defer ps.mtx.Unlock()
|
defer ps.mtx.Unlock()
|
||||||
|
|
||||||
if ps.Height != height || ps.Round != round {
|
if ps.Height != height {
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -482,21 +515,29 @@ func (ps *PeerState) EnsureVoteBitArrays(height uint32, round uint16, numValidat
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (ps *PeerState) SetHasVote(height uint32, round uint16, index uint, vote *Vote) {
|
func (ps *PeerState) SetHasVote(vote *Vote, index uint) {
|
||||||
ps.mtx.Lock()
|
ps.mtx.Lock()
|
||||||
defer ps.mtx.Unlock()
|
defer ps.mtx.Unlock()
|
||||||
|
ps.setHasVote(vote.Height, vote.Round, vote.Type, index)
|
||||||
|
}
|
||||||
|
|
||||||
if ps.Height != height {
|
func (ps *PeerState) setHasVote(height uint32, round uint16, type_ byte, index uint) {
|
||||||
|
if ps.Height == height+1 && type_ == VoteTypeCommit {
|
||||||
|
// Special case for LastCommits.
|
||||||
|
ps.LastCommits.SetIndex(index, true)
|
||||||
|
return
|
||||||
|
} else if ps.Height != height {
|
||||||
|
// Does not apply.
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
switch vote.Type {
|
switch type_ {
|
||||||
case VoteTypePrevote:
|
case VoteTypePrevote:
|
||||||
ps.Prevotes.SetIndex(index, true)
|
ps.Prevotes.SetIndex(index, true)
|
||||||
case VoteTypePrecommit:
|
case VoteTypePrecommit:
|
||||||
ps.Precommits.SetIndex(index, true)
|
ps.Precommits.SetIndex(index, true)
|
||||||
case VoteTypeCommit:
|
case VoteTypeCommit:
|
||||||
if vote.Round < round {
|
if round < ps.Round {
|
||||||
ps.Prevotes.SetIndex(index, true)
|
ps.Prevotes.SetIndex(index, true)
|
||||||
ps.Precommits.SetIndex(index, true)
|
ps.Precommits.SetIndex(index, true)
|
||||||
}
|
}
|
||||||
@ -531,6 +572,12 @@ func (ps *PeerState) ApplyNewRoundStepMessage(msg *NewRoundStepMessage, rs *Roun
|
|||||||
ps.Precommits = BitArray{}
|
ps.Precommits = BitArray{}
|
||||||
}
|
}
|
||||||
if psHeight != msg.Height {
|
if psHeight != msg.Height {
|
||||||
|
// Shift Commits to LastCommits
|
||||||
|
if psHeight+1 == msg.Height {
|
||||||
|
ps.LastCommits = ps.Commits
|
||||||
|
} else {
|
||||||
|
ps.LastCommits = BitArray{}
|
||||||
|
}
|
||||||
// We'll update the BitArray capacity later.
|
// We'll update the BitArray capacity later.
|
||||||
ps.Commits = BitArray{}
|
ps.Commits = BitArray{}
|
||||||
}
|
}
|
||||||
@ -548,22 +595,19 @@ func (ps *PeerState) ApplyCommitMessage(msg *CommitMessage) {
|
|||||||
ps.ProposalBlockBitArray = msg.BlockBitArray
|
ps.ProposalBlockBitArray = msg.BlockBitArray
|
||||||
}
|
}
|
||||||
|
|
||||||
func (ps *PeerState) ApplyHasVotesMessage(msg *HasVotesMessage) {
|
func (ps *PeerState) ApplyHasVoteMessage(msg *HasVoteMessage) {
|
||||||
ps.mtx.Lock()
|
ps.mtx.Lock()
|
||||||
defer ps.mtx.Unlock()
|
defer ps.mtx.Unlock()
|
||||||
|
|
||||||
if ps.Height != msg.Height {
|
// Special case for LastCommits
|
||||||
|
if ps.Height == msg.Height+1 && msg.Type == VoteTypeCommit {
|
||||||
|
ps.LastCommits.SetIndex(msg.Index, true)
|
||||||
|
return
|
||||||
|
} else if ps.Height != msg.Height {
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
ps.Commits = ps.Commits.Or(msg.Commits)
|
ps.setHasVote(msg.Height, msg.Round, msg.Type, msg.Index)
|
||||||
if ps.Round == msg.Round {
|
|
||||||
ps.Prevotes = ps.Prevotes.Or(msg.Prevotes)
|
|
||||||
ps.Precommits = ps.Precommits.Or(msg.Precommits)
|
|
||||||
} else {
|
|
||||||
ps.Prevotes = msg.Prevotes
|
|
||||||
ps.Precommits = msg.Precommits
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
//-----------------------------------------------------------------------------
|
//-----------------------------------------------------------------------------
|
||||||
@ -574,11 +618,11 @@ const (
|
|||||||
// Messages for communicating state changes
|
// Messages for communicating state changes
|
||||||
msgTypeNewRoundStep = byte(0x01)
|
msgTypeNewRoundStep = byte(0x01)
|
||||||
msgTypeCommit = byte(0x02)
|
msgTypeCommit = byte(0x02)
|
||||||
msgTypeHasVotes = byte(0x03)
|
|
||||||
// Messages of data
|
// Messages of data
|
||||||
msgTypeProposal = byte(0x11)
|
msgTypeProposal = byte(0x11)
|
||||||
msgTypePart = byte(0x12) // both block & POL
|
msgTypePart = byte(0x12) // both block & POL
|
||||||
msgTypeVote = byte(0x13)
|
msgTypeVote = byte(0x13)
|
||||||
|
msgTypeHasVote = byte(0x14)
|
||||||
)
|
)
|
||||||
|
|
||||||
// TODO: check for unnecessary extra bytes at the end.
|
// TODO: check for unnecessary extra bytes at the end.
|
||||||
@ -593,8 +637,6 @@ func decodeMessage(bz []byte) (msgType byte, msg interface{}) {
|
|||||||
msg = readNewRoundStepMessage(r, n, err)
|
msg = readNewRoundStepMessage(r, n, err)
|
||||||
case msgTypeCommit:
|
case msgTypeCommit:
|
||||||
msg = readCommitMessage(r, n, err)
|
msg = readCommitMessage(r, n, err)
|
||||||
case msgTypeHasVotes:
|
|
||||||
msg = readHasVotesMessage(r, n, err)
|
|
||||||
// Messages of data
|
// Messages of data
|
||||||
case msgTypeProposal:
|
case msgTypeProposal:
|
||||||
msg = ReadProposal(r, n, err)
|
msg = ReadProposal(r, n, err)
|
||||||
@ -602,6 +644,8 @@ func decodeMessage(bz []byte) (msgType byte, msg interface{}) {
|
|||||||
msg = readPartMessage(r, n, err)
|
msg = readPartMessage(r, n, err)
|
||||||
case msgTypeVote:
|
case msgTypeVote:
|
||||||
msg = ReadVote(r, n, err)
|
msg = ReadVote(r, n, err)
|
||||||
|
case msgTypeHasVote:
|
||||||
|
msg = readHasVoteMessage(r, n, err)
|
||||||
default:
|
default:
|
||||||
msg = nil
|
msg = nil
|
||||||
}
|
}
|
||||||
@ -669,40 +713,6 @@ func (m *CommitMessage) String() string {
|
|||||||
|
|
||||||
//-------------------------------------
|
//-------------------------------------
|
||||||
|
|
||||||
type HasVotesMessage struct {
|
|
||||||
Height uint32
|
|
||||||
Round uint16
|
|
||||||
Prevotes BitArray
|
|
||||||
Precommits BitArray
|
|
||||||
Commits BitArray
|
|
||||||
}
|
|
||||||
|
|
||||||
func readHasVotesMessage(r io.Reader, n *int64, err *error) *HasVotesMessage {
|
|
||||||
return &HasVotesMessage{
|
|
||||||
Height: ReadUInt32(r, n, err),
|
|
||||||
Round: ReadUInt16(r, n, err),
|
|
||||||
Prevotes: ReadBitArray(r, n, err),
|
|
||||||
Precommits: ReadBitArray(r, n, err),
|
|
||||||
Commits: ReadBitArray(r, n, err),
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
func (m *HasVotesMessage) WriteTo(w io.Writer) (n int64, err error) {
|
|
||||||
WriteByte(w, msgTypeHasVotes, &n, &err)
|
|
||||||
WriteUInt32(w, m.Height, &n, &err)
|
|
||||||
WriteUInt16(w, m.Round, &n, &err)
|
|
||||||
WriteBinary(w, m.Prevotes, &n, &err)
|
|
||||||
WriteBinary(w, m.Precommits, &n, &err)
|
|
||||||
WriteBinary(w, m.Commits, &n, &err)
|
|
||||||
return
|
|
||||||
}
|
|
||||||
|
|
||||||
func (m *HasVotesMessage) String() string {
|
|
||||||
return fmt.Sprintf("[HasVotesMessage H:%v R:%v]", m.Height, m.Round)
|
|
||||||
}
|
|
||||||
|
|
||||||
//-------------------------------------
|
|
||||||
|
|
||||||
const (
|
const (
|
||||||
partTypeProposalBlock = byte(0x01)
|
partTypeProposalBlock = byte(0x01)
|
||||||
partTypeProposalPOL = byte(0x02)
|
partTypeProposalPOL = byte(0x02)
|
||||||
@ -736,3 +746,34 @@ func (m *PartMessage) WriteTo(w io.Writer) (n int64, err error) {
|
|||||||
func (m *PartMessage) String() string {
|
func (m *PartMessage) String() string {
|
||||||
return fmt.Sprintf("[PartMessage H:%v R:%v T:%X]", m.Height, m.Round, m.Type)
|
return fmt.Sprintf("[PartMessage H:%v R:%v T:%X]", m.Height, m.Round, m.Type)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
//-------------------------------------
|
||||||
|
|
||||||
|
type HasVoteMessage struct {
|
||||||
|
Height uint32
|
||||||
|
Round uint16
|
||||||
|
Type byte
|
||||||
|
Index uint
|
||||||
|
}
|
||||||
|
|
||||||
|
func readHasVoteMessage(r io.Reader, n *int64, err *error) *HasVoteMessage {
|
||||||
|
return &HasVoteMessage{
|
||||||
|
Height: ReadUInt32(r, n, err),
|
||||||
|
Round: ReadUInt16(r, n, err),
|
||||||
|
Type: ReadByte(r, n, err),
|
||||||
|
Index: ReadUVarInt(r, n, err),
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (m *HasVoteMessage) WriteTo(w io.Writer) (n int64, err error) {
|
||||||
|
WriteByte(w, msgTypeHasVote, &n, &err)
|
||||||
|
WriteUInt32(w, m.Height, &n, &err)
|
||||||
|
WriteUInt16(w, m.Round, &n, &err)
|
||||||
|
WriteByte(w, m.Type, &n, &err)
|
||||||
|
WriteUVarInt(w, m.Index, &n, &err)
|
||||||
|
return
|
||||||
|
}
|
||||||
|
|
||||||
|
func (m *HasVoteMessage) String() string {
|
||||||
|
return fmt.Sprintf("[HasVoteMessage H:%v R:%v T:%X]", m.Height, m.Round, m.Type)
|
||||||
|
}
|
||||||
|
@ -496,10 +496,11 @@ func (cs *ConsensusState) RunActionPropose(height uint32, round uint16) {
|
|||||||
pol = cs.LockedPOL
|
pol = cs.LockedPOL
|
||||||
} else {
|
} else {
|
||||||
// Otherwise we should create a new proposal.
|
// Otherwise we should create a new proposal.
|
||||||
var validation Validation
|
var validation *Validation
|
||||||
if cs.Height == 1 {
|
if cs.Height == 1 {
|
||||||
// We're creating a proposal for the first block.
|
// We're creating a proposal for the first block.
|
||||||
// The validation is empty.
|
// The validation is empty.
|
||||||
|
validation = &Validation{}
|
||||||
} else {
|
} else {
|
||||||
// We need to create a proposal.
|
// We need to create a proposal.
|
||||||
// If we don't have enough commits from the last height,
|
// If we don't have enough commits from the last height,
|
||||||
@ -512,7 +513,7 @@ func (cs *ConsensusState) RunActionPropose(height uint32, round uint16) {
|
|||||||
}
|
}
|
||||||
txs, state := cs.mempool.GetProposalTxs() // TODO: cache state
|
txs, state := cs.mempool.GetProposalTxs() // TODO: cache state
|
||||||
block = &Block{
|
block = &Block{
|
||||||
Header: Header{
|
Header: &Header{
|
||||||
Network: Config.Network,
|
Network: Config.Network,
|
||||||
Height: cs.Height,
|
Height: cs.Height,
|
||||||
Time: time.Now(),
|
Time: time.Now(),
|
||||||
@ -522,7 +523,7 @@ func (cs *ConsensusState) RunActionPropose(height uint32, round uint16) {
|
|||||||
StateHash: state.Hash(),
|
StateHash: state.Hash(),
|
||||||
},
|
},
|
||||||
Validation: validation,
|
Validation: validation,
|
||||||
Data: Data{
|
Data: &Data{
|
||||||
Txs: txs,
|
Txs: txs,
|
||||||
},
|
},
|
||||||
}
|
}
|
||||||
@ -827,8 +828,26 @@ func (cs *ConsensusState) AddProposalPOLPart(height uint32, round uint16, part *
|
|||||||
return true, nil
|
return true, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// NOTE: This function may increment the height.
|
func (cs *ConsensusState) AddVote(vote *Vote) (added bool, index uint, err error) {
|
||||||
func (cs *ConsensusState) AddVote(vote *Vote) (added bool, err error) {
|
cs.mtx.Lock()
|
||||||
|
defer cs.mtx.Unlock()
|
||||||
|
|
||||||
|
return cs.addVote(vote)
|
||||||
|
}
|
||||||
|
|
||||||
|
// TODO: Maybe move this out of here?
|
||||||
|
func (cs *ConsensusState) LoadHeaderValidation(height uint32) (*Header, *Validation) {
|
||||||
|
meta := cs.blockStore.LoadBlockMeta(height)
|
||||||
|
if meta == nil {
|
||||||
|
return nil, nil
|
||||||
|
}
|
||||||
|
validation := cs.blockStore.LoadBlockValidation(height)
|
||||||
|
return meta.Header, validation
|
||||||
|
}
|
||||||
|
|
||||||
|
//-----------------------------------------------------------------------------
|
||||||
|
|
||||||
|
func (cs *ConsensusState) addVote(vote *Vote) (added bool, index uint, err error) {
|
||||||
switch vote.Type {
|
switch vote.Type {
|
||||||
case VoteTypePrevote:
|
case VoteTypePrevote:
|
||||||
// Prevotes checks for height+round match.
|
// Prevotes checks for height+round match.
|
||||||
@ -837,23 +856,26 @@ func (cs *ConsensusState) AddVote(vote *Vote) (added bool, err error) {
|
|||||||
// Precommits checks for height+round match.
|
// Precommits checks for height+round match.
|
||||||
return cs.Precommits.Add(vote)
|
return cs.Precommits.Add(vote)
|
||||||
case VoteTypeCommit:
|
case VoteTypeCommit:
|
||||||
// Commits checks for height match.
|
if vote.Height == cs.Height {
|
||||||
// No need to check if vote.Round < cs.Round ...
|
// No need to check if vote.Round < cs.Round ...
|
||||||
// Prevotes && Precommits already checks that.
|
// Prevotes && Precommits already checks that.
|
||||||
cs.Prevotes.Add(vote)
|
cs.Prevotes.Add(vote)
|
||||||
cs.Precommits.Add(vote)
|
cs.Precommits.Add(vote)
|
||||||
added, err = cs.Commits.Add(vote)
|
added, index, err = cs.Commits.Add(vote)
|
||||||
if added && cs.Commits.HasTwoThirdsMajority() {
|
if added && cs.Commits.HasTwoThirdsMajority() {
|
||||||
cs.runActionCh <- RoundAction{cs.Height, cs.Round, RoundActionTryFinalize}
|
cs.runActionCh <- RoundAction{cs.Height, cs.Round, RoundActionTryFinalize}
|
||||||
}
|
}
|
||||||
return added, err
|
return added, index, err
|
||||||
|
}
|
||||||
|
if vote.Height+1 == cs.Height {
|
||||||
|
return cs.LastCommits.Add(vote)
|
||||||
|
}
|
||||||
|
return false, 0, nil
|
||||||
default:
|
default:
|
||||||
panic("Unknown vote type")
|
panic("Unknown vote type")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
//-----------------------------------------------------------------------------
|
|
||||||
|
|
||||||
func (cs *ConsensusState) stageBlock(block *Block, blockParts *PartSet) error {
|
func (cs *ConsensusState) stageBlock(block *Block, blockParts *PartSet) error {
|
||||||
if block == nil {
|
if block == nil {
|
||||||
panic("Cannot stage nil block")
|
panic("Cannot stage nil block")
|
||||||
@ -891,7 +913,7 @@ func (cs *ConsensusState) signAddVote(type_ byte, hash []byte, header PartSetHea
|
|||||||
BlockParts: header,
|
BlockParts: header,
|
||||||
}
|
}
|
||||||
cs.PrivValidator.Sign(vote)
|
cs.PrivValidator.Sign(vote)
|
||||||
cs.AddVote(vote)
|
cs.addVote(vote)
|
||||||
return vote
|
return vote
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -54,10 +54,14 @@ func NewVoteSet(height uint32, round uint16, type_ byte, vset *state.ValidatorSe
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (vs *VoteSet) Size() uint {
|
||||||
|
return vs.vset.Size()
|
||||||
|
}
|
||||||
|
|
||||||
// True if added, false if not.
|
// True if added, false if not.
|
||||||
// Returns ErrVote[UnexpectedStep|InvalidAccount|InvalidSignature|InvalidBlockHash|ConflictingSignature]
|
// Returns ErrVote[UnexpectedStep|InvalidAccount|InvalidSignature|InvalidBlockHash|ConflictingSignature]
|
||||||
// NOTE: vote should not be mutated after adding.
|
// NOTE: vote should not be mutated after adding.
|
||||||
func (vs *VoteSet) Add(vote *Vote) (bool, error) {
|
func (vs *VoteSet) Add(vote *Vote) (bool, uint, error) {
|
||||||
vs.mtx.Lock()
|
vs.mtx.Lock()
|
||||||
defer vs.mtx.Unlock()
|
defer vs.mtx.Unlock()
|
||||||
|
|
||||||
@ -66,31 +70,31 @@ func (vs *VoteSet) Add(vote *Vote) (bool, error) {
|
|||||||
(vote.Type != VoteTypeCommit && vote.Round != vs.round) ||
|
(vote.Type != VoteTypeCommit && vote.Round != vs.round) ||
|
||||||
(vote.Type != VoteTypeCommit && vote.Type != vs.type_) ||
|
(vote.Type != VoteTypeCommit && vote.Type != vs.type_) ||
|
||||||
(vote.Type == VoteTypeCommit && vs.type_ != VoteTypeCommit && vote.Round >= vs.round) {
|
(vote.Type == VoteTypeCommit && vs.type_ != VoteTypeCommit && vote.Round >= vs.round) {
|
||||||
return false, ErrVoteUnexpectedStep
|
return false, 0, ErrVoteUnexpectedStep
|
||||||
}
|
}
|
||||||
|
|
||||||
// Ensure that signer is a validator.
|
// Ensure that signer is a validator.
|
||||||
_, val := vs.vset.GetById(vote.SignerId)
|
_, val := vs.vset.GetById(vote.SignerId)
|
||||||
if val == nil {
|
if val == nil {
|
||||||
return false, ErrVoteInvalidAccount
|
return false, 0, ErrVoteInvalidAccount
|
||||||
}
|
}
|
||||||
|
|
||||||
// Check signature.
|
// Check signature.
|
||||||
if !val.Verify(vote) {
|
if !val.Verify(vote) {
|
||||||
// Bad signature.
|
// Bad signature.
|
||||||
return false, ErrVoteInvalidSignature
|
return false, 0, ErrVoteInvalidSignature
|
||||||
}
|
}
|
||||||
|
|
||||||
return vs.addVote(vote)
|
return vs.addVote(vote)
|
||||||
}
|
}
|
||||||
|
|
||||||
func (vs *VoteSet) addVote(vote *Vote) (bool, error) {
|
func (vs *VoteSet) addVote(vote *Vote) (bool, uint, error) {
|
||||||
// If vote already exists, return false.
|
// If vote already exists, return false.
|
||||||
if existingVote, ok := vs.votes[vote.SignerId]; ok {
|
if existingVote, ok := vs.votes[vote.SignerId]; ok {
|
||||||
if bytes.Equal(existingVote.BlockHash, vote.BlockHash) {
|
if bytes.Equal(existingVote.BlockHash, vote.BlockHash) {
|
||||||
return false, nil
|
return false, 0, nil
|
||||||
} else {
|
} else {
|
||||||
return false, ErrVoteConflictingSignature
|
return false, 0, ErrVoteConflictingSignature
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -98,7 +102,7 @@ func (vs *VoteSet) addVote(vote *Vote) (bool, error) {
|
|||||||
vs.votes[vote.SignerId] = vote
|
vs.votes[vote.SignerId] = vote
|
||||||
voterIndex, val := vs.vset.GetById(vote.SignerId)
|
voterIndex, val := vs.vset.GetById(vote.SignerId)
|
||||||
if val == nil {
|
if val == nil {
|
||||||
return false, ErrVoteInvalidAccount
|
return false, 0, ErrVoteInvalidAccount
|
||||||
}
|
}
|
||||||
vs.votesBitArray.SetIndex(uint(voterIndex), true)
|
vs.votesBitArray.SetIndex(uint(voterIndex), true)
|
||||||
blockKey := string(vote.BlockHash) + string(BinaryBytes(vote.BlockParts))
|
blockKey := string(vote.BlockHash) + string(BinaryBytes(vote.BlockParts))
|
||||||
@ -114,7 +118,7 @@ func (vs *VoteSet) addVote(vote *Vote) (bool, error) {
|
|||||||
vs.maj23Exists = true
|
vs.maj23Exists = true
|
||||||
}
|
}
|
||||||
|
|
||||||
return true, nil
|
return true, voterIndex, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// Assumes that commits VoteSet is valid.
|
// Assumes that commits VoteSet is valid.
|
||||||
@ -215,7 +219,7 @@ func (vs *VoteSet) MakePOL() *POL {
|
|||||||
return pol
|
return pol
|
||||||
}
|
}
|
||||||
|
|
||||||
func (vs *VoteSet) MakeValidation() Validation {
|
func (vs *VoteSet) MakeValidation() *Validation {
|
||||||
if vs.type_ != VoteTypeCommit {
|
if vs.type_ != VoteTypeCommit {
|
||||||
panic("Cannot MakeValidation() unless VoteSet.Type is VoteTypeCommit")
|
panic("Cannot MakeValidation() unless VoteSet.Type is VoteTypeCommit")
|
||||||
}
|
}
|
||||||
@ -240,7 +244,7 @@ func (vs *VoteSet) MakeValidation() Validation {
|
|||||||
rsigs[index] = RoundSignature{vote.Round, vote.Signature}
|
rsigs[index] = RoundSignature{vote.Round, vote.Signature}
|
||||||
return false
|
return false
|
||||||
})
|
})
|
||||||
return Validation{
|
return &Validation{
|
||||||
Commits: rsigs,
|
Commits: rsigs,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -169,7 +169,7 @@ func TestBadVotes(t *testing.T) {
|
|||||||
// val0 votes for nil.
|
// val0 votes for nil.
|
||||||
vote := &Vote{Height: height, Round: round, Type: VoteTypePrevote, BlockHash: nil}
|
vote := &Vote{Height: height, Round: round, Type: VoteTypePrevote, BlockHash: nil}
|
||||||
privAccounts[0].Sign(vote)
|
privAccounts[0].Sign(vote)
|
||||||
added, err := voteSet.Add(vote)
|
added, _, err := voteSet.Add(vote)
|
||||||
if !added || err != nil {
|
if !added || err != nil {
|
||||||
t.Errorf("Expected Add(vote) to succeed")
|
t.Errorf("Expected Add(vote) to succeed")
|
||||||
}
|
}
|
||||||
@ -177,7 +177,7 @@ func TestBadVotes(t *testing.T) {
|
|||||||
// val0 votes again for some block.
|
// val0 votes again for some block.
|
||||||
vote = &Vote{Height: height, Round: round, Type: VoteTypePrevote, BlockHash: CRandBytes(32)}
|
vote = &Vote{Height: height, Round: round, Type: VoteTypePrevote, BlockHash: CRandBytes(32)}
|
||||||
privAccounts[0].Sign(vote)
|
privAccounts[0].Sign(vote)
|
||||||
added, err = voteSet.Add(vote)
|
added, _, err = voteSet.Add(vote)
|
||||||
if added || err == nil {
|
if added || err == nil {
|
||||||
t.Errorf("Expected Add(vote) to fail, dupeout.")
|
t.Errorf("Expected Add(vote) to fail, dupeout.")
|
||||||
}
|
}
|
||||||
@ -185,7 +185,7 @@ func TestBadVotes(t *testing.T) {
|
|||||||
// val1 votes on another height
|
// val1 votes on another height
|
||||||
vote = &Vote{Height: height + 1, Round: round, Type: VoteTypePrevote, BlockHash: nil}
|
vote = &Vote{Height: height + 1, Round: round, Type: VoteTypePrevote, BlockHash: nil}
|
||||||
privAccounts[1].Sign(vote)
|
privAccounts[1].Sign(vote)
|
||||||
added, err = voteSet.Add(vote)
|
added, _, err = voteSet.Add(vote)
|
||||||
if added {
|
if added {
|
||||||
t.Errorf("Expected Add(vote) to fail, wrong height")
|
t.Errorf("Expected Add(vote) to fail, wrong height")
|
||||||
}
|
}
|
||||||
@ -193,7 +193,7 @@ func TestBadVotes(t *testing.T) {
|
|||||||
// val2 votes on another round
|
// val2 votes on another round
|
||||||
vote = &Vote{Height: height, Round: round + 1, Type: VoteTypePrevote, BlockHash: nil}
|
vote = &Vote{Height: height, Round: round + 1, Type: VoteTypePrevote, BlockHash: nil}
|
||||||
privAccounts[2].Sign(vote)
|
privAccounts[2].Sign(vote)
|
||||||
added, err = voteSet.Add(vote)
|
added, _, err = voteSet.Add(vote)
|
||||||
if added {
|
if added {
|
||||||
t.Errorf("Expected Add(vote) to fail, wrong round")
|
t.Errorf("Expected Add(vote) to fail, wrong round")
|
||||||
}
|
}
|
||||||
@ -201,7 +201,7 @@ func TestBadVotes(t *testing.T) {
|
|||||||
// val3 votes of another type.
|
// val3 votes of another type.
|
||||||
vote = &Vote{Height: height, Round: round, Type: VoteTypePrecommit, BlockHash: nil}
|
vote = &Vote{Height: height, Round: round, Type: VoteTypePrecommit, BlockHash: nil}
|
||||||
privAccounts[3].Sign(vote)
|
privAccounts[3].Sign(vote)
|
||||||
added, err = voteSet.Add(vote)
|
added, _, err = voteSet.Add(vote)
|
||||||
if added {
|
if added {
|
||||||
t.Errorf("Expected Add(vote) to fail, wrong type")
|
t.Errorf("Expected Add(vote) to fail, wrong type")
|
||||||
}
|
}
|
||||||
@ -226,7 +226,7 @@ func TestAddCommitsToPrevoteVotes(t *testing.T) {
|
|||||||
// Attempt to add a commit from val6 at a previous height
|
// Attempt to add a commit from val6 at a previous height
|
||||||
vote := &Vote{Height: height - 1, Round: round, Type: VoteTypeCommit, BlockHash: nil}
|
vote := &Vote{Height: height - 1, Round: round, Type: VoteTypeCommit, BlockHash: nil}
|
||||||
privAccounts[6].Sign(vote)
|
privAccounts[6].Sign(vote)
|
||||||
added, _ := voteSet.Add(vote)
|
added, _, _ := voteSet.Add(vote)
|
||||||
if added {
|
if added {
|
||||||
t.Errorf("Expected Add(vote) to fail, wrong height.")
|
t.Errorf("Expected Add(vote) to fail, wrong height.")
|
||||||
}
|
}
|
||||||
@ -234,7 +234,7 @@ func TestAddCommitsToPrevoteVotes(t *testing.T) {
|
|||||||
// Attempt to add a commit from val6 at a later round
|
// Attempt to add a commit from val6 at a later round
|
||||||
vote = &Vote{Height: height, Round: round + 1, Type: VoteTypeCommit, BlockHash: nil}
|
vote = &Vote{Height: height, Round: round + 1, Type: VoteTypeCommit, BlockHash: nil}
|
||||||
privAccounts[6].Sign(vote)
|
privAccounts[6].Sign(vote)
|
||||||
added, _ = voteSet.Add(vote)
|
added, _, _ = voteSet.Add(vote)
|
||||||
if added {
|
if added {
|
||||||
t.Errorf("Expected Add(vote) to fail, cannot add future round vote.")
|
t.Errorf("Expected Add(vote) to fail, cannot add future round vote.")
|
||||||
}
|
}
|
||||||
@ -242,7 +242,7 @@ func TestAddCommitsToPrevoteVotes(t *testing.T) {
|
|||||||
// Attempt to add a commit from val6 for currrent height/round.
|
// Attempt to add a commit from val6 for currrent height/round.
|
||||||
vote = &Vote{Height: height, Round: round, Type: VoteTypeCommit, BlockHash: nil}
|
vote = &Vote{Height: height, Round: round, Type: VoteTypeCommit, BlockHash: nil}
|
||||||
privAccounts[6].Sign(vote)
|
privAccounts[6].Sign(vote)
|
||||||
added, err := voteSet.Add(vote)
|
added, _, err := voteSet.Add(vote)
|
||||||
if added || err == nil {
|
if added || err == nil {
|
||||||
t.Errorf("Expected Add(vote) to fail, only prior round commits can be added.")
|
t.Errorf("Expected Add(vote) to fail, only prior round commits can be added.")
|
||||||
}
|
}
|
||||||
@ -250,7 +250,7 @@ func TestAddCommitsToPrevoteVotes(t *testing.T) {
|
|||||||
// Add commit from val6 at a previous round
|
// Add commit from val6 at a previous round
|
||||||
vote = &Vote{Height: height, Round: round - 1, Type: VoteTypeCommit, BlockHash: nil}
|
vote = &Vote{Height: height, Round: round - 1, Type: VoteTypeCommit, BlockHash: nil}
|
||||||
privAccounts[6].Sign(vote)
|
privAccounts[6].Sign(vote)
|
||||||
added, err = voteSet.Add(vote)
|
added, _, err = voteSet.Add(vote)
|
||||||
if !added || err != nil {
|
if !added || err != nil {
|
||||||
t.Errorf("Expected Add(vote) to succeed, commit for prior rounds are relevant.")
|
t.Errorf("Expected Add(vote) to succeed, commit for prior rounds are relevant.")
|
||||||
}
|
}
|
||||||
@ -258,7 +258,7 @@ func TestAddCommitsToPrevoteVotes(t *testing.T) {
|
|||||||
// Also add commit from val7 for previous round.
|
// Also add commit from val7 for previous round.
|
||||||
vote = &Vote{Height: height, Round: round - 2, Type: VoteTypeCommit, BlockHash: nil}
|
vote = &Vote{Height: height, Round: round - 2, Type: VoteTypeCommit, BlockHash: nil}
|
||||||
privAccounts[7].Sign(vote)
|
privAccounts[7].Sign(vote)
|
||||||
added, err = voteSet.Add(vote)
|
added, _, err = voteSet.Add(vote)
|
||||||
if !added || err != nil {
|
if !added || err != nil {
|
||||||
t.Errorf("Expected Add(vote) to succeed. err: %v", err)
|
t.Errorf("Expected Add(vote) to succeed. err: %v", err)
|
||||||
}
|
}
|
||||||
|
Reference in New Issue
Block a user