@ -1,158 +0,0 @@ | |||||
package blocks | |||||
import ( | |||||
"bytes" | |||||
"errors" | |||||
"sync" | |||||
"github.com/tendermint/tendermint/merkle" | |||||
) | |||||
// A collection of block parts. | |||||
// Doesn't do any validation. | |||||
type BlockPartSet struct { | |||||
mtx sync.Mutex | |||||
height uint32 | |||||
total uint16 // total number of parts | |||||
numParts uint16 // number of parts in this set | |||||
parts []*BlockPart | |||||
_block *Block // cache | |||||
} | |||||
var ( | |||||
ErrInvalidBlockPartConflict = errors.New("Invalid block part conflict") // Signer signed conflicting parts | |||||
) | |||||
// parts may be nil if the parts aren't in hand. | |||||
func NewBlockPartSet(height uint32, parts []*BlockPart) *BlockPartSet { | |||||
bps := &BlockPartSet{ | |||||
height: height, | |||||
parts: parts, | |||||
numParts: uint16(len(parts)), | |||||
} | |||||
if len(parts) > 0 { | |||||
bps.total = parts[0].Total | |||||
} | |||||
return bps | |||||
} | |||||
func (bps *BlockPartSet) Height() uint32 { | |||||
return bps.height | |||||
} | |||||
func (bps *BlockPartSet) BlockParts() []*BlockPart { | |||||
bps.mtx.Lock() | |||||
defer bps.mtx.Unlock() | |||||
return bps.parts | |||||
} | |||||
func (bps *BlockPartSet) BitArray() []byte { | |||||
bps.mtx.Lock() | |||||
defer bps.mtx.Unlock() | |||||
if bps.parts == nil { | |||||
return nil | |||||
} | |||||
bitArray := make([]byte, (len(bps.parts)+7)/8) | |||||
for i, part := range bps.parts { | |||||
if part != nil { | |||||
bitArray[i/8] |= 1 << uint(i%8) | |||||
} | |||||
} | |||||
return bitArray | |||||
} | |||||
// If the part isn't valid, returns an error. | |||||
// err can be ErrInvalidBlockPartConflict | |||||
// NOTE: Caller must check the signature before adding. | |||||
func (bps *BlockPartSet) AddBlockPart(part *BlockPart) (added bool, err error) { | |||||
bps.mtx.Lock() | |||||
defer bps.mtx.Unlock() | |||||
if bps.parts == nil { | |||||
// First received part for this round. | |||||
bps.parts = make([]*BlockPart, part.Total) | |||||
bps.total = uint16(part.Total) | |||||
bps.parts[int(part.Index)] = part | |||||
bps.numParts++ | |||||
return true, nil | |||||
} else { | |||||
// Check part.Index and part.Total | |||||
if uint16(part.Index) >= bps.total { | |||||
return false, ErrInvalidBlockPartConflict | |||||
} | |||||
if uint16(part.Total) != bps.total { | |||||
return false, ErrInvalidBlockPartConflict | |||||
} | |||||
// Check for existing parts. | |||||
existing := bps.parts[part.Index] | |||||
if existing != nil { | |||||
if bytes.Equal(existing.Bytes, part.Bytes) { | |||||
// Ignore duplicate | |||||
return false, nil | |||||
} else { | |||||
return false, ErrInvalidBlockPartConflict | |||||
} | |||||
} else { | |||||
bps.parts[int(part.Index)] = part | |||||
bps.numParts++ | |||||
return true, nil | |||||
} | |||||
} | |||||
} | |||||
func (bps *BlockPartSet) IsComplete() bool { | |||||
bps.mtx.Lock() | |||||
defer bps.mtx.Unlock() | |||||
return bps.total > 0 && bps.total == bps.numParts | |||||
} | |||||
func (bps *BlockPartSet) Block() *Block { | |||||
if !bps.IsComplete() { | |||||
return nil | |||||
} | |||||
bps.mtx.Lock() | |||||
defer bps.mtx.Unlock() | |||||
if bps._block == nil { | |||||
block, err := BlockPartsToBlock(bps.parts) | |||||
if err != nil { | |||||
panic(err) | |||||
} | |||||
bps._block = block | |||||
} | |||||
return bps._block | |||||
} | |||||
func (bps *BlockPartSet) Hash() []byte { | |||||
if !bps.IsComplete() { | |||||
panic("Cannot get hash of an incomplete BlockPartSet") | |||||
} | |||||
hashes := [][]byte{} | |||||
for _, part := range bps.parts { | |||||
partHash := part.Hash() | |||||
hashes = append(hashes, partHash) | |||||
} | |||||
return merkle.HashFromByteSlices(hashes) | |||||
} | |||||
// The proposal hash includes both the block hash | |||||
// as well as the BlockPartSet merkle hash. | |||||
func (bps *BlockPartSet) ProposalHash() []byte { | |||||
bpsHash := bps.Hash() | |||||
blockHash := bps.Block().Hash() | |||||
return merkle.HashFromByteSlices([][]byte{bpsHash, blockHash}) | |||||
} | |||||
//----------------------------------------------------------------------------- | |||||
func BlockPartsToBlock(parts []*BlockPart) (*Block, error) { | |||||
blockBytes := []byte{} | |||||
for _, part := range parts { | |||||
blockBytes = append(blockBytes, part.Bytes...) | |||||
} | |||||
var n int64 | |||||
var err error | |||||
block := ReadBlock(bytes.NewReader(blockBytes), &n, &err) | |||||
return block, err | |||||
} |
@ -0,0 +1,46 @@ | |||||
package blocks | |||||
import ( | |||||
"fmt" | |||||
. "github.com/tendermint/tendermint/config" | |||||
) | |||||
func GenVoteDocument(voteType byte, height uint32, round uint16, blockHash []byte) []byte { | |||||
stepName := "" | |||||
switch voteType { | |||||
case VoteTypeBare: | |||||
stepName = "bare" | |||||
case VoteTypePrecommit: | |||||
stepName = "precommit" | |||||
case VoteTypeCommit: | |||||
stepName = "commit" | |||||
default: | |||||
panic("Unknown vote type") | |||||
} | |||||
return []byte(fmt.Sprintf( | |||||
`!!!!!BEGIN TENDERMINT VOTE!!!!! | |||||
Network: %v | |||||
Height: %v | |||||
Round: %v | |||||
Step: %v | |||||
BlockHash: %v | |||||
!!!!!END TENDERMINT VOTE!!!!!`, | |||||
Config.Network, height, round, stepName, blockHash, | |||||
)) | |||||
} | |||||
func GenProposalDocument(height uint32, round uint16, blockPartsTotal uint16, blockPartsHash []byte, | |||||
polPartsTotal uint16, polPartsHash []byte) []byte { | |||||
return []byte(fmt.Sprintf( | |||||
`!!!!!BEGIN TENDERMINT PROPOSAL!!!!! | |||||
Network: %v | |||||
Height: %v | |||||
Round: %v | |||||
BlockPartsTotal: %v | |||||
BlockPartsHash: %X | |||||
POLPartsTotal: %v | |||||
POLPartsHash: %X | |||||
!!!!!END TENDERMINT PROPOSAL!!!!!`, | |||||
Config.Network, height, round, blockPartsTotal, blockPartsHash, polPartsTotal, polPartsHash, | |||||
)) | |||||
} |
@ -0,0 +1,54 @@ | |||||
package blocks | |||||
import ( | |||||
"errors" | |||||
"io" | |||||
. "github.com/tendermint/tendermint/binary" | |||||
) | |||||
const ( | |||||
VoteTypeBare = byte(0x00) | |||||
VoteTypePrecommit = byte(0x01) | |||||
VoteTypeCommit = byte(0x02) | |||||
) | |||||
var ( | |||||
ErrVoteUnexpectedPhase = errors.New("Unexpected phase") | |||||
ErrVoteInvalidAccount = errors.New("Invalid round vote account") | |||||
ErrVoteInvalidSignature = errors.New("Invalid round vote signature") | |||||
ErrVoteInvalidBlockHash = errors.New("Invalid block hash") | |||||
ErrVoteConflictingSignature = errors.New("Conflicting round vote signature") | |||||
) | |||||
// Represents a bare, precommit, or commit vote for proposals. | |||||
type Vote struct { | |||||
Height uint32 | |||||
Round uint16 | |||||
Type byte | |||||
BlockHash []byte // empty if vote is nil. | |||||
Signature | |||||
} | |||||
func ReadVote(r io.Reader, n *int64, err *error) *Vote { | |||||
return &Vote{ | |||||
Height: ReadUInt32(r, n, err), | |||||
Round: ReadUInt16(r, n, err), | |||||
Type: ReadByte(r, n, err), | |||||
BlockHash: ReadByteSlice(r, n, err), | |||||
Signature: ReadSignature(r, n, err), | |||||
} | |||||
} | |||||
func (v *Vote) WriteTo(w io.Writer) (n int64, err error) { | |||||
WriteUInt32(w, v.Height, &n, &err) | |||||
WriteUInt16(w, v.Round, &n, &err) | |||||
WriteByte(w, v.Type, &n, &err) | |||||
WriteByteSlice(w, v.BlockHash, &n, &err) | |||||
WriteBinary(w, v.Signature, &n, &err) | |||||
return | |||||
} | |||||
func (v *Vote) GenDocument() []byte { | |||||
return GenVoteDocument(v.Type, v.Height, v.Round, v.BlockHash) | |||||
} |
@ -0,0 +1,155 @@ | |||||
package common | |||||
import ( | |||||
"io" | |||||
"math" | |||||
"math/rand" | |||||
. "github.com/tendermint/tendermint/binary" | |||||
) | |||||
// Not goroutine safe | |||||
type BitArray []uint64 | |||||
func NewBitArray(length uint) BitArray { | |||||
return BitArray(make([]uint64, (length+63)/64)) | |||||
} | |||||
func ReadBitArray(r io.Reader, n *int64, err *error) BitArray { | |||||
lengthTotal := ReadUInt32(r, n, err) | |||||
lengthWritten := ReadUInt32(r, n, err) | |||||
if *err != nil { | |||||
return nil | |||||
} | |||||
buf := make([]uint64, int(lengthTotal)) | |||||
for i := uint32(0); i < lengthWritten; i++ { | |||||
buf[i] = ReadUInt64(r, n, err) | |||||
if err != nil { | |||||
return nil | |||||
} | |||||
} | |||||
return BitArray(buf) | |||||
} | |||||
func (bA BitArray) WriteTo(w io.Writer) (n int64, err error) { | |||||
// Count the last element > 0. | |||||
lastNonzeroIndex := -1 | |||||
for i, elem := range bA { | |||||
if elem > 0 { | |||||
lastNonzeroIndex = i | |||||
} | |||||
} | |||||
WriteUInt32(w, uint32(len(bA)), &n, &err) | |||||
WriteUInt32(w, uint32(lastNonzeroIndex+1), &n, &err) | |||||
for i, elem := range bA { | |||||
if i > lastNonzeroIndex { | |||||
break | |||||
} | |||||
WriteUInt64(w, elem, &n, &err) | |||||
} | |||||
return | |||||
} | |||||
func (bA BitArray) GetIndex(i uint) bool { | |||||
return bA[i/64]&uint64(1<<(i%64)) > 0 | |||||
} | |||||
func (bA BitArray) SetIndex(i uint, v bool) { | |||||
if v { | |||||
bA[i/64] |= uint64(1 << (i % 64)) | |||||
} else { | |||||
bA[i/64] &= ^uint64(1 << (i % 64)) | |||||
} | |||||
} | |||||
func (bA BitArray) Copy() BitArray { | |||||
c := make([]uint64, len(bA)) | |||||
copy(c, bA) | |||||
return BitArray(c) | |||||
} | |||||
func (bA BitArray) Or(o BitArray) BitArray { | |||||
c := bA.Copy() | |||||
for i, _ := range c { | |||||
c[i] = o[i] | c[i] | |||||
} | |||||
return c | |||||
} | |||||
func (bA BitArray) And(o BitArray) BitArray { | |||||
c := bA.Copy() | |||||
for i, _ := range c { | |||||
c[i] = o[i] & c[i] | |||||
} | |||||
return c | |||||
} | |||||
func (bA BitArray) Not() BitArray { | |||||
c := bA.Copy() | |||||
for i, _ := range c { | |||||
c[i] = ^c[i] | |||||
} | |||||
return c | |||||
} | |||||
func (bA BitArray) Sub(o BitArray) BitArray { | |||||
return bA.And(o.Not()) | |||||
} | |||||
// NOTE: returns counts or a longer int slice as necessary. | |||||
func (bA BitArray) AddToCounts(counts []int) []int { | |||||
for bytei := 0; bytei < len(bA); bytei++ { | |||||
for biti := 0; biti < 64; biti++ { | |||||
if (bA[bytei] & (1 << uint(biti))) == 0 { | |||||
continue | |||||
} | |||||
index := 64*bytei + biti | |||||
if len(counts) <= index { | |||||
counts = append(counts, make([]int, (index-len(counts)+1))...) | |||||
} | |||||
counts[index]++ | |||||
} | |||||
} | |||||
return counts | |||||
} | |||||
func (bA BitArray) PickRandom() (int, bool) { | |||||
randStart := rand.Intn(len(bA)) | |||||
for i := 0; i < len(bA); i++ { | |||||
bytei := ((i + randStart) % len(bA)) | |||||
if bA[bytei] > 0 { | |||||
randBitStart := rand.Intn(64) | |||||
for j := 0; j < 64; j++ { | |||||
biti := ((j + randBitStart) % 64) | |||||
//fmt.Printf("%X %v %v %v\n", iHas, j, biti, randBitStart) | |||||
if (bA[bytei] & (1 << uint(biti))) > 0 { | |||||
return 64*int(bytei) + int(biti), true | |||||
} | |||||
} | |||||
panic("should not happen") | |||||
} | |||||
} | |||||
return 0, false | |||||
} | |||||
// Pick an index from this BitArray that is 1 && whose count is lowest. | |||||
func (bA BitArray) PickRarest(counts []int) (rarest int, ok bool) { | |||||
smallestCount := math.MaxInt32 | |||||
for bytei := 0; bytei < len(bA); bytei++ { | |||||
if bA[bytei] > 0 { | |||||
for biti := 0; biti < 64; biti++ { | |||||
if (bA[bytei] & (1 << uint(biti))) == 0 { | |||||
continue | |||||
} | |||||
index := 64*bytei + biti | |||||
if counts[index] < smallestCount { | |||||
smallestCount = counts[index] | |||||
rarest = index | |||||
ok = true | |||||
} | |||||
} | |||||
panic("should not happen") | |||||
} | |||||
} | |||||
return | |||||
} |
@ -0,0 +1,20 @@ | |||||
package common | |||||
import ( | |||||
"sort" | |||||
) | |||||
// Sort for []uint64 | |||||
type UInt64Slice []uint64 | |||||
func (p UInt64Slice) Len() int { return len(p) } | |||||
func (p UInt64Slice) Less(i, j int) bool { return p[i] < p[j] } | |||||
func (p UInt64Slice) Swap(i, j int) { p[i], p[j] = p[j], p[i] } | |||||
func (p UInt64Slice) Sort() { sort.Sort(p) } | |||||
func SearchUInt64s(a []uint64, x uint64) int { | |||||
return sort.Search(len(a), func(i int) bool { return a[i] >= x }) | |||||
} | |||||
func (p UInt64Slice) Search(x uint64) int { return SearchUInt64s(p, x) } |
@ -1,41 +0,0 @@ | |||||
package consensus | |||||
import ( | |||||
"fmt" | |||||
. "github.com/tendermint/tendermint/config" | |||||
) | |||||
func GenVoteDocument(voteType byte, height uint32, round uint16, proposalHash []byte) string { | |||||
stepName := "" | |||||
switch voteType { | |||||
case VoteTypeBare: | |||||
stepName = "bare" | |||||
case VoteTypePrecommit: | |||||
stepName = "precommit" | |||||
case VoteTypeCommit: | |||||
stepName = "commit" | |||||
default: | |||||
panic("Unknown vote type") | |||||
} | |||||
return fmt.Sprintf( | |||||
`-----BEGIN TENDERMINT DOCUMENT----- | |||||
URI: %v://consensus/%v/%v/%v | |||||
ProposalHash: %X | |||||
-----END TENDERMINT DOCUMENHT-----`, | |||||
Config.Network, height, round, stepName, | |||||
proposalHash, | |||||
) | |||||
} | |||||
func GenBlockPartDocument(height uint32, round uint16, index uint16, total uint16, blockPartHash []byte) string { | |||||
return fmt.Sprintf( | |||||
`-----BEGIN TENDERMINT DOCUMENT----- | |||||
URI: %v://blockpart/%v/%v/%v | |||||
Total: %v | |||||
BlockPartHash: %X | |||||
-----END TENDERMINT DOCUMENHT-----`, | |||||
Config.Network, height, round, index, | |||||
total, | |||||
blockPartHash, | |||||
) | |||||
} |
@ -0,0 +1,177 @@ | |||||
package consensus | |||||
import ( | |||||
"bytes" | |||||
"crypto/sha256" | |||||
"errors" | |||||
"io" | |||||
"sync" | |||||
. "github.com/tendermint/tendermint/binary" | |||||
. "github.com/tendermint/tendermint/common" | |||||
"github.com/tendermint/tendermint/merkle" | |||||
) | |||||
const ( | |||||
partSize = 4096 // 4KB | |||||
) | |||||
var ( | |||||
ErrPartSetUnexpectedIndex = errors.New("Error part set unexpected index") | |||||
ErrPartSetInvalidTrail = errors.New("Error part set invalid trail") | |||||
) | |||||
type Part struct { | |||||
Index uint16 | |||||
Trail [][]byte | |||||
Bytes []byte | |||||
// Cache | |||||
hash []byte | |||||
} | |||||
func ReadPart(r io.Reader, n *int64, err *error) *Part { | |||||
return &Part{ | |||||
Index: ReadUInt16(r, n, err), | |||||
Trail: ReadByteSlices(r, n, err), | |||||
Bytes: ReadByteSlice(r, n, err), | |||||
} | |||||
} | |||||
func (b *Part) WriteTo(w io.Writer) (n int64, err error) { | |||||
WriteUInt16(w, b.Index, &n, &err) | |||||
WriteByteSlices(w, b.Trail, &n, &err) | |||||
WriteByteSlice(w, b.Bytes, &n, &err) | |||||
return | |||||
} | |||||
func (pt *Part) Hash() []byte { | |||||
if pt.hash != nil { | |||||
return pt.hash | |||||
} else { | |||||
hasher := sha256.New() | |||||
_, err := hasher.Write(pt.Bytes) | |||||
if err != nil { | |||||
panic(err) | |||||
} | |||||
pt.hash = hasher.Sum(nil) | |||||
return pt.hash | |||||
} | |||||
} | |||||
//------------------------------------- | |||||
type PartSet struct { | |||||
rootHash []byte | |||||
total uint16 | |||||
mtx sync.Mutex | |||||
parts []*Part | |||||
partsBitArray BitArray | |||||
count uint16 | |||||
} | |||||
// Returns an immutable, full PartSet. | |||||
func NewPartSetFromData(data []byte) *PartSet { | |||||
// divide data into 4kb parts. | |||||
total := (len(data) + partSize - 1) / partSize | |||||
parts := make([]*Part, total) | |||||
parts_ := make([]merkle.Hashable, total) | |||||
partsBitArray := NewBitArray(uint(total)) | |||||
for i := 0; i < total; i++ { | |||||
part := &Part{ | |||||
Index: uint16(i), | |||||
Bytes: data[i*partSize : MinInt(len(data), (i+1)*partSize)], | |||||
} | |||||
parts[i] = part | |||||
parts_[i] = part | |||||
partsBitArray.SetIndex(uint(i), true) | |||||
} | |||||
// Compute merkle trails | |||||
hashTree := merkle.HashTreeFromHashables(parts_) | |||||
for i := 0; i < total; i++ { | |||||
parts[i].Trail = merkle.HashTrailForIndex(hashTree, i) | |||||
} | |||||
return &PartSet{ | |||||
parts: parts, | |||||
partsBitArray: partsBitArray, | |||||
rootHash: hashTree[len(hashTree)/2], | |||||
total: uint16(total), | |||||
count: uint16(total), | |||||
} | |||||
} | |||||
// Returns an empty PartSet ready to be populated. | |||||
func NewPartSetFromMetadata(total uint16, rootHash []byte) *PartSet { | |||||
return &PartSet{ | |||||
parts: make([]*Part, total), | |||||
partsBitArray: NewBitArray(uint(total)), | |||||
rootHash: rootHash, | |||||
total: total, | |||||
count: 0, | |||||
} | |||||
} | |||||
func (ps *PartSet) BitArray() BitArray { | |||||
ps.mtx.Lock() | |||||
defer ps.mtx.Unlock() | |||||
return ps.partsBitArray.Copy() | |||||
} | |||||
func (ps *PartSet) RootHash() []byte { | |||||
return ps.rootHash | |||||
} | |||||
func (ps *PartSet) Total() uint16 { | |||||
if ps == nil { | |||||
return 0 | |||||
} | |||||
return ps.total | |||||
} | |||||
func (ps *PartSet) AddPart(part *Part) (bool, error) { | |||||
ps.mtx.Lock() | |||||
defer ps.mtx.Unlock() | |||||
// Invalid part index | |||||
if part.Index >= ps.total { | |||||
return false, ErrPartSetUnexpectedIndex | |||||
} | |||||
// If part already exists, return false. | |||||
if ps.parts[part.Index] != nil { | |||||
return false, nil | |||||
} | |||||
// Check hash trail | |||||
if !merkle.VerifyHashTrailForIndex(int(part.Index), part.Hash(), part.Trail, ps.rootHash) { | |||||
return false, ErrPartSetInvalidTrail | |||||
} | |||||
// Add part | |||||
ps.parts[part.Index] = part | |||||
ps.partsBitArray.SetIndex(uint(part.Index), true) | |||||
ps.count++ | |||||
return true, nil | |||||
} | |||||
func (ps *PartSet) GetPart(index uint16) *Part { | |||||
ps.mtx.Lock() | |||||
defer ps.mtx.Unlock() | |||||
return ps.parts[index] | |||||
} | |||||
func (ps *PartSet) IsComplete() bool { | |||||
return ps.count == ps.total | |||||
} | |||||
func (ps *PartSet) GetReader() io.Reader { | |||||
if !ps.IsComplete() { | |||||
panic("Cannot GetReader() on incomplete PartSet") | |||||
} | |||||
buf := []byte{} | |||||
for _, part := range ps.parts { | |||||
buf = append(buf, part.Bytes...) | |||||
} | |||||
return bytes.NewReader(buf) | |||||
} |
@ -0,0 +1,98 @@ | |||||
package consensus | |||||
import ( | |||||
"io" | |||||
. "github.com/tendermint/tendermint/binary" | |||||
. "github.com/tendermint/tendermint/blocks" | |||||
. "github.com/tendermint/tendermint/common" | |||||
. "github.com/tendermint/tendermint/state" | |||||
) | |||||
// Proof of lock. | |||||
// +2/3 of validators' (bare) votes for a given blockhash (or nil) | |||||
type POL struct { | |||||
Height uint32 | |||||
Round uint16 | |||||
BlockHash []byte // Could be nil, which makes this a proof of unlock. | |||||
Votes []Signature // Vote signatures for height/round/hash | |||||
Commits []Signature // Commit signatures for height/hash | |||||
CommitRounds []uint16 // Rounds of the commits, less than POL.Round. | |||||
} | |||||
func ReadPOL(r io.Reader, n *int64, err *error) *POL { | |||||
return &POL{ | |||||
Height: ReadUInt32(r, n, err), | |||||
Round: ReadUInt16(r, n, err), | |||||
BlockHash: ReadByteSlice(r, n, err), | |||||
Votes: ReadSignatures(r, n, err), | |||||
Commits: ReadSignatures(r, n, err), | |||||
CommitRounds: ReadUInt16s(r, n, err), | |||||
} | |||||
} | |||||
func (pol *POL) WriteTo(w io.Writer) (n int64, err error) { | |||||
WriteUInt32(w, pol.Height, &n, &err) | |||||
WriteUInt16(w, pol.Round, &n, &err) | |||||
WriteByteSlice(w, pol.BlockHash, &n, &err) | |||||
WriteSignatures(w, pol.Votes, &n, &err) | |||||
WriteSignatures(w, pol.Commits, &n, &err) | |||||
WriteUInt16s(w, pol.CommitRounds, &n, &err) | |||||
return | |||||
} | |||||
// Returns whether +2/3 have voted/committed for BlockHash. | |||||
func (pol *POL) Verify(vset *ValidatorSet) error { | |||||
talliedVotingPower := uint64(0) | |||||
voteDoc := GenVoteDocument(VoteTypeBare, pol.Height, pol.Round, pol.BlockHash) | |||||
seenValidators := map[uint64]struct{}{} | |||||
for _, sig := range pol.Votes { | |||||
// Validate | |||||
if _, seen := seenValidators[sig.SignerId]; seen { | |||||
return Errorf("Duplicate validator for vote %v for POL %v", sig, pol) | |||||
} | |||||
validator := vset.GetById(sig.SignerId) | |||||
if validator == nil { | |||||
return Errorf("Invalid validator for vote %v for POL %v", sig, pol) | |||||
} | |||||
if !validator.Verify(voteDoc, sig) { | |||||
return Errorf("Invalid signature for vote %v for POL %v", sig, pol) | |||||
} | |||||
// Tally | |||||
seenValidators[validator.Id] = struct{}{} | |||||
talliedVotingPower += validator.VotingPower | |||||
} | |||||
for i, sig := range pol.Commits { | |||||
round := pol.CommitRounds[i] | |||||
// Validate | |||||
if _, seen := seenValidators[sig.SignerId]; seen { | |||||
return Errorf("Duplicate validator for commit %v for POL %v", sig, pol) | |||||
} | |||||
validator := vset.GetById(sig.SignerId) | |||||
if validator == nil { | |||||
return Errorf("Invalid validator for commit %v for POL %v", sig, pol) | |||||
} | |||||
commitDoc := GenVoteDocument(VoteTypeCommit, pol.Height, round, pol.BlockHash) // TODO cache | |||||
if !validator.Verify(commitDoc, sig) { | |||||
return Errorf("Invalid signature for commit %v for POL %v", sig, pol) | |||||
} | |||||
// Tally | |||||
seenValidators[validator.Id] = struct{}{} | |||||
talliedVotingPower += validator.VotingPower | |||||
} | |||||
if talliedVotingPower > vset.TotalVotingPower()*2/3 { | |||||
return nil | |||||
} else { | |||||
return Errorf("Invalid POL, insufficient voting power %v, needed %v", | |||||
talliedVotingPower, (vset.TotalVotingPower()*2/3 + 1)) | |||||
} | |||||
} |
@ -0,0 +1,64 @@ | |||||
package consensus | |||||
import ( | |||||
"errors" | |||||
"io" | |||||
. "github.com/tendermint/tendermint/binary" | |||||
. "github.com/tendermint/tendermint/blocks" | |||||
) | |||||
var ( | |||||
ErrInvalidBlockPartSignature = errors.New("Error invalid block part signature") | |||||
ErrInvalidBlockPartHash = errors.New("Error invalid block part hash") | |||||
) | |||||
type Proposal struct { | |||||
Height uint32 | |||||
Round uint16 | |||||
BlockPartsTotal uint16 | |||||
BlockPartsHash []byte | |||||
POLPartsTotal uint16 | |||||
POLPartsHash []byte | |||||
Signature | |||||
} | |||||
func NewProposal(height uint32, round uint16, blockPartsTotal uint16, blockPartsHash []byte, | |||||
polPartsTotal uint16, polPartsHash []byte) *Proposal { | |||||
return &Proposal{ | |||||
Height: height, | |||||
Round: round, | |||||
BlockPartsTotal: blockPartsTotal, | |||||
BlockPartsHash: blockPartsHash, | |||||
POLPartsTotal: polPartsTotal, | |||||
POLPartsHash: polPartsHash, | |||||
} | |||||
} | |||||
func ReadProposal(r io.Reader, n *int64, err *error) *Proposal { | |||||
return &Proposal{ | |||||
Height: ReadUInt32(r, n, err), | |||||
Round: ReadUInt16(r, n, err), | |||||
BlockPartsTotal: ReadUInt16(r, n, err), | |||||
BlockPartsHash: ReadByteSlice(r, n, err), | |||||
POLPartsTotal: ReadUInt16(r, n, err), | |||||
POLPartsHash: ReadByteSlice(r, n, err), | |||||
Signature: ReadSignature(r, n, err), | |||||
} | |||||
} | |||||
func (p *Proposal) WriteTo(w io.Writer) (n int64, err error) { | |||||
WriteUInt32(w, p.Height, &n, &err) | |||||
WriteUInt16(w, p.Round, &n, &err) | |||||
WriteUInt16(w, p.BlockPartsTotal, &n, &err) | |||||
WriteByteSlice(w, p.BlockPartsHash, &n, &err) | |||||
WriteUInt16(w, p.POLPartsTotal, &n, &err) | |||||
WriteByteSlice(w, p.POLPartsHash, &n, &err) | |||||
WriteBinary(w, p.Signature, &n, &err) | |||||
return | |||||
} | |||||
func (p *Proposal) GenDocument() []byte { | |||||
return GenProposalDocument(p.Height, p.Round, p.BlockPartsTotal, p.BlockPartsHash, | |||||
p.POLPartsTotal, p.POLPartsHash) | |||||
} |
@ -1,204 +1,429 @@ | |||||
package consensus | package consensus | ||||
import ( | import ( | ||||
"errors" | |||||
"sync" | "sync" | ||||
"time" | "time" | ||||
. "github.com/tendermint/tendermint/binary" | |||||
. "github.com/tendermint/tendermint/blocks" | . "github.com/tendermint/tendermint/blocks" | ||||
. "github.com/tendermint/tendermint/common" | . "github.com/tendermint/tendermint/common" | ||||
. "github.com/tendermint/tendermint/mempool" | |||||
. "github.com/tendermint/tendermint/state" | . "github.com/tendermint/tendermint/state" | ||||
) | ) | ||||
const ( | |||||
RoundStepStart = uint8(0x00) // Round started. | |||||
RoundStepPropose = uint8(0x01) // Did propose, broadcasting proposal. | |||||
RoundStepVote = uint8(0x02) // Did vote, broadcasting votes. | |||||
RoundStepPrecommit = uint8(0x03) // Did precommit, broadcasting precommits. | |||||
RoundStepCommit = uint8(0x04) // We committed at this round -- do not progress to the next round. | |||||
) | |||||
var ( | var ( | ||||
ErrInvalidProposalSignature = errors.New("Error invalid proposal signature") | |||||
consensusStateKey = []byte("consensusState") | consensusStateKey = []byte("consensusState") | ||||
) | ) | ||||
// Immutable when returned from ConsensusState.GetRoundState() | |||||
type RoundState struct { | |||||
Height uint32 // Height we are working on | |||||
Round uint16 | |||||
Step uint8 | |||||
StartTime time.Time | |||||
Validators *ValidatorSet | |||||
Proposer *Validator | |||||
Proposal *Proposal | |||||
ProposalBlock *Block | |||||
ProposalBlockPartSet *PartSet | |||||
ProposalPOL *POL | |||||
ProposalPOLPartSet *PartSet | |||||
LockedBlock *Block | |||||
LockedPOL *POL | |||||
Votes *VoteSet | |||||
Precommits *VoteSet | |||||
Commits *VoteSet | |||||
PrivValidator *PrivValidator | |||||
} | |||||
//------------------------------------- | |||||
// Tracks consensus state across block heights and rounds. | // Tracks consensus state across block heights and rounds. | ||||
type ConsensusState struct { | type ConsensusState struct { | ||||
mtx sync.Mutex | |||||
height uint32 // Height we are working on. | |||||
validatorsR0 *ValidatorSet // A copy of the validators at round 0 | |||||
lockedProposal *BlockPartSet // A BlockPartSet of the locked proposal. | |||||
startTime time.Time // Start of round 0 for this height. | |||||
commits *VoteSet // Commits for this height. | |||||
roundState *RoundState // The RoundState object for the current round. | |||||
commitTime time.Time // Time at which a block was found to be committed by +2/3. | |||||
} | |||||
func NewConsensusState(state *State) *ConsensusState { | |||||
cs := &ConsensusState{} | |||||
cs.Update(state) | |||||
mtx sync.Mutex | |||||
RoundState | |||||
blockStore *BlockStore | |||||
mempool *Mempool | |||||
state *State // State until height-1. | |||||
stagedBlock *Block // Cache last staged block. | |||||
stagedState *State // Cache result of staged block. | |||||
} | |||||
func NewConsensusState(state *State, blockStore *BlockStore, mempool *Mempool) *ConsensusState { | |||||
cs := &ConsensusState{ | |||||
blockStore: blockStore, | |||||
mempool: mempool, | |||||
} | |||||
cs.updateToState(state) | |||||
return cs | return cs | ||||
} | } | ||||
func (cs *ConsensusState) LockProposal(blockPartSet *BlockPartSet) { | |||||
func (cs *ConsensusState) GetRoundState() *RoundState { | |||||
cs.mtx.Lock() | cs.mtx.Lock() | ||||
defer cs.mtx.Unlock() | defer cs.mtx.Unlock() | ||||
cs.lockedProposal = blockPartSet | |||||
rs := cs.RoundState // copy | |||||
return &rs | |||||
} | } | ||||
func (cs *ConsensusState) UnlockProposal() { | |||||
func (cs *ConsensusState) updateToState(state *State) { | |||||
// Sanity check state. | |||||
stateHeight := state.Height() | |||||
if stateHeight > 0 && stateHeight != cs.Height+1 { | |||||
Panicf("updateToState() expected state height of %v but found %v", cs.Height+1, stateHeight) | |||||
} | |||||
// Reset fields based on state. | |||||
height := state.Height() | |||||
validators := state.Validators() | |||||
cs.Height = height | |||||
cs.Round = 0 | |||||
cs.Step = RoundStepStart | |||||
cs.StartTime = state.CommitTime().Add(newBlockWaitDuration) | |||||
cs.Validators = validators | |||||
cs.Proposer = validators.GetProposer() | |||||
cs.Proposal = nil | |||||
cs.ProposalBlock = nil | |||||
cs.ProposalBlockPartSet = nil | |||||
cs.ProposalPOL = nil | |||||
cs.ProposalPOLPartSet = nil | |||||
cs.LockedBlock = nil | |||||
cs.LockedPOL = nil | |||||
cs.Votes = NewVoteSet(height, 0, VoteTypeBare, validators) | |||||
cs.Precommits = NewVoteSet(height, 0, VoteTypePrecommit, validators) | |||||
cs.Commits = NewVoteSet(height, 0, VoteTypeCommit, validators) | |||||
cs.stagedBlock = nil | |||||
cs.stagedState = nil | |||||
// Update the round if we need to. | |||||
round := calcRound(cs.StartTime) | |||||
if round > 0 { | |||||
cs.setupRound(round) | |||||
} | |||||
} | |||||
func (cs *ConsensusState) SetupRound(round uint16) { | |||||
cs.mtx.Lock() | cs.mtx.Lock() | ||||
defer cs.mtx.Unlock() | defer cs.mtx.Unlock() | ||||
cs.lockedProposal = nil | |||||
if cs.Round >= round { | |||||
Panicf("ConsensusState round %v not lower than desired round %v", cs.Round, round) | |||||
} | |||||
cs.setupRound(round) | |||||
} | } | ||||
func (cs *ConsensusState) LockedProposal() *BlockPartSet { | |||||
func (cs *ConsensusState) setupRound(round uint16) { | |||||
// Increment all the way to round. | |||||
validators := cs.Validators.Copy() | |||||
for r := cs.Round; r < round; r++ { | |||||
validators.IncrementAccum() | |||||
} | |||||
cs.Round = round | |||||
cs.Step = RoundStepStart | |||||
cs.Validators = validators | |||||
cs.Proposer = validators.GetProposer() | |||||
cs.Proposal = nil | |||||
cs.ProposalBlock = nil | |||||
cs.ProposalBlockPartSet = nil | |||||
cs.ProposalPOL = nil | |||||
cs.ProposalPOLPartSet = nil | |||||
cs.Votes = NewVoteSet(cs.Height, round, VoteTypeBare, validators) | |||||
cs.Votes.AddVotesFromCommits(cs.Commits) | |||||
cs.Precommits = NewVoteSet(cs.Height, round, VoteTypePrecommit, validators) | |||||
cs.Precommits.AddVotesFromCommits(cs.Commits) | |||||
} | |||||
func (cs *ConsensusState) SetStep(step byte) { | |||||
cs.mtx.Lock() | cs.mtx.Lock() | ||||
defer cs.mtx.Unlock() | defer cs.mtx.Unlock() | ||||
return cs.lockedProposal | |||||
if cs.Step < step { | |||||
cs.Step = step | |||||
} else { | |||||
panic("step regression") | |||||
} | |||||
} | } | ||||
func (cs *ConsensusState) RoundState() *RoundState { | |||||
func (cs *ConsensusState) SetPrivValidator(priv *PrivValidator) { | |||||
cs.mtx.Lock() | cs.mtx.Lock() | ||||
defer cs.mtx.Unlock() | defer cs.mtx.Unlock() | ||||
return cs.roundState | |||||
cs.PrivValidator = priv | |||||
} | } | ||||
// Primarily gets called upon block commit by ConsensusAgent. | |||||
func (cs *ConsensusState) Update(state *State) { | |||||
func (cs *ConsensusState) SetProposal(proposal *Proposal) error { | |||||
cs.mtx.Lock() | cs.mtx.Lock() | ||||
defer cs.mtx.Unlock() | defer cs.mtx.Unlock() | ||||
// Sanity check state. | |||||
stateHeight := state.Height() | |||||
if stateHeight > 0 && stateHeight != cs.height+1 { | |||||
Panicf("Update() expected state height of %v but found %v", cs.height+1, stateHeight) | |||||
// Already have one | |||||
if cs.Proposal != nil { | |||||
return nil | |||||
} | } | ||||
// Reset fields based on state. | |||||
cs.height = stateHeight | |||||
cs.validatorsR0 = state.Validators().Copy() // NOTE: immutable. | |||||
cs.lockedProposal = nil | |||||
cs.startTime = state.CommitTime().Add(newBlockWaitDuration) // NOTE: likely future time. | |||||
cs.commits = NewVoteSet(stateHeight, 0, VoteTypeCommit, cs.validatorsR0) | |||||
// Invalid. | |||||
if proposal.Height != cs.Height || proposal.Round != cs.Round { | |||||
return nil | |||||
} | |||||
// Setup the roundState | |||||
cs.roundState = nil | |||||
cs.setupRound(0) | |||||
// Verify signature | |||||
if !cs.Proposer.Verify(proposal.GenDocument(), proposal.Signature) { | |||||
return ErrInvalidProposalSignature | |||||
} | |||||
cs.Proposal = proposal | |||||
cs.ProposalBlockPartSet = NewPartSetFromMetadata(proposal.BlockPartsTotal, proposal.BlockPartsHash) | |||||
cs.ProposalPOLPartSet = NewPartSetFromMetadata(proposal.POLPartsTotal, proposal.POLPartsHash) | |||||
return nil | |||||
} | } | ||||
// If cs.roundState isn't at round, set up new roundState at round. | |||||
func (cs *ConsensusState) SetupRound(round uint16) { | |||||
func (cs *ConsensusState) MakeProposal() { | |||||
cs.mtx.Lock() | cs.mtx.Lock() | ||||
defer cs.mtx.Unlock() | defer cs.mtx.Unlock() | ||||
if cs.roundState != nil && cs.roundState.Round >= round { | |||||
if cs.PrivValidator == nil || cs.Proposer.Id != cs.PrivValidator.Id { | |||||
return | return | ||||
} | } | ||||
cs.setupRound(round) | |||||
} | |||||
// Initialize roundState for given round. | |||||
// Involves incrementing validators for each past rand. | |||||
func (cs *ConsensusState) setupRound(round uint16) { | |||||
// Increment validator accums as necessary. | |||||
// We need to start with cs.validatorsR0 or cs.roundState.Validators | |||||
var validators *ValidatorSet | |||||
var validatorsRound uint16 | |||||
if cs.roundState == nil { | |||||
// We have no roundState so we start from validatorsR0 at round 0. | |||||
validators = cs.validatorsR0.Copy() | |||||
validatorsRound = 0 | |||||
var block *Block | |||||
var blockPartSet *PartSet | |||||
var pol *POL | |||||
var polPartSet *PartSet | |||||
// Decide on block and POL | |||||
if cs.LockedBlock != nil { | |||||
// If we're locked onto a block, just choose that. | |||||
block = cs.LockedBlock | |||||
pol = cs.LockedPOL | |||||
} else { | } else { | ||||
// We have a previous roundState so we start from that. | |||||
validators = cs.roundState.Validators.Copy() | |||||
validatorsRound = cs.roundState.Round | |||||
// TODO: make use of state returned from MakeProposalBlock() | |||||
block, _ = cs.mempool.MakeProposalBlock() | |||||
pol = cs.LockedPOL // If exists, is a PoUnlock. | |||||
} | } | ||||
// Increment all the way to round. | |||||
for r := validatorsRound; r < round; r++ { | |||||
validators.IncrementAccum() | |||||
blockPartSet = NewPartSetFromData(BinaryBytes(block)) | |||||
if pol != nil { | |||||
polPartSet = NewPartSetFromData(BinaryBytes(pol)) | |||||
} | } | ||||
roundState := NewRoundState(cs.height, round, cs.startTime, validators, cs.commits) | |||||
cs.roundState = roundState | |||||
// Make proposal | |||||
proposal := NewProposal(cs.Height, cs.Round, blockPartSet.Total(), blockPartSet.RootHash(), | |||||
polPartSet.Total(), polPartSet.RootHash()) | |||||
cs.PrivValidator.SignProposal(proposal) | |||||
// Set fields | |||||
cs.Proposal = proposal | |||||
cs.ProposalBlock = block | |||||
cs.ProposalBlockPartSet = blockPartSet | |||||
cs.ProposalPOL = pol | |||||
cs.ProposalPOLPartSet = polPartSet | |||||
} | } | ||||
//----------------------------------------------------------------------------- | |||||
// NOTE: block is not necessarily valid. | |||||
func (cs *ConsensusState) AddProposalBlockPart(height uint32, round uint16, part *Part) (added bool, err error) { | |||||
cs.mtx.Lock() | |||||
defer cs.mtx.Unlock() | |||||
const ( | |||||
RoundStepStart = uint8(0x00) // Round started. | |||||
RoundStepProposal = uint8(0x01) // Did propose, broadcasting proposal. | |||||
RoundStepBareVotes = uint8(0x02) // Did vote bare, broadcasting bare votes. | |||||
RoundStepPrecommits = uint8(0x03) // Did precommit, broadcasting precommits. | |||||
RoundStepCommitOrUnlock = uint8(0x04) // We committed at this round -- do not progress to the next round. | |||||
) | |||||
// Blocks might be reused, so round mismatch is OK | |||||
if cs.Height != height { | |||||
return false, nil | |||||
} | |||||
//----------------------------------------------------------------------------- | |||||
// We're not expecting a block part. | |||||
if cs.ProposalBlockPartSet != nil { | |||||
return false, nil // TODO: bad peer? Return error? | |||||
} | |||||
// RoundState encapsulates all the state needed to engage in the consensus protocol. | |||||
type RoundState struct { | |||||
Height uint32 // Immutable | |||||
Round uint16 // Immutable | |||||
StartTime time.Time // Time in which consensus started for this height. | |||||
Expires time.Time // Time after which this round is expired. | |||||
Proposer *Validator // The proposer to propose a block for this round. | |||||
Validators *ValidatorSet // All validators with modified accumPower for this round. | |||||
Proposal *BlockPartSet // All block parts received for this round. | |||||
RoundBareVotes *VoteSet // All votes received for this round. | |||||
RoundPrecommits *VoteSet // All precommits received for this round. | |||||
Commits *VoteSet // A shared object for all commit votes of this height. | |||||
mtx sync.Mutex | |||||
step uint8 // mutable | |||||
} | |||||
func NewRoundState(height uint32, round uint16, startTime time.Time, | |||||
validators *ValidatorSet, commits *VoteSet) *RoundState { | |||||
proposer := validators.GetProposer() | |||||
blockPartSet := NewBlockPartSet(height, nil) | |||||
roundBareVotes := NewVoteSet(height, round, VoteTypeBare, validators) | |||||
roundPrecommits := NewVoteSet(height, round, VoteTypePrecommit, validators) | |||||
rs := &RoundState{ | |||||
Height: height, | |||||
Round: round, | |||||
StartTime: startTime, | |||||
Expires: calcRoundStartTime(round+1, startTime), | |||||
Proposer: proposer, | |||||
Validators: validators, | |||||
Proposal: blockPartSet, | |||||
RoundBareVotes: roundBareVotes, | |||||
RoundPrecommits: roundPrecommits, | |||||
Commits: commits, | |||||
step: RoundStepStart, | |||||
} | |||||
return rs | |||||
} | |||||
// "source" is typically the Peer.Key of the peer that gave us this vote. | |||||
func (rs *RoundState) AddVote(vote *Vote, source string) (added bool, rank uint8, err error) { | |||||
added, err = cs.ProposalBlockPartSet.AddPart(part) | |||||
if err != nil { | |||||
return added, err | |||||
} | |||||
if added && cs.ProposalBlockPartSet.IsComplete() { | |||||
var n int64 | |||||
var err error | |||||
cs.ProposalBlock = ReadBlock(cs.ProposalBlockPartSet.GetReader(), &n, &err) | |||||
return true, err | |||||
} | |||||
return true, nil | |||||
} | |||||
// NOTE: POL is not necessarily valid. | |||||
func (cs *ConsensusState) AddProposalPOLPart(height uint32, round uint16, part *Part) (added bool, err error) { | |||||
cs.mtx.Lock() | |||||
defer cs.mtx.Unlock() | |||||
if cs.Height != height || cs.Round != round { | |||||
return false, nil | |||||
} | |||||
// We're not expecting a POL part. | |||||
if cs.ProposalPOLPartSet != nil { | |||||
return false, nil // TODO: bad peer? Return error? | |||||
} | |||||
added, err = cs.ProposalPOLPartSet.AddPart(part) | |||||
if err != nil { | |||||
return added, err | |||||
} | |||||
if added && cs.ProposalPOLPartSet.IsComplete() { | |||||
var n int64 | |||||
var err error | |||||
cs.ProposalPOL = ReadPOL(cs.ProposalPOLPartSet.GetReader(), &n, &err) | |||||
return true, err | |||||
} | |||||
return true, nil | |||||
} | |||||
func (cs *ConsensusState) AddVote(vote *Vote) (added bool, err error) { | |||||
switch vote.Type { | switch vote.Type { | ||||
case VoteTypeBare: | case VoteTypeBare: | ||||
return rs.RoundBareVotes.AddVote(vote, source) | |||||
// Votes checks for height+round match. | |||||
return cs.Votes.AddVote(vote) | |||||
case VoteTypePrecommit: | case VoteTypePrecommit: | ||||
return rs.RoundPrecommits.AddVote(vote, source) | |||||
// Precommits checks for height+round match. | |||||
return cs.Precommits.AddVote(vote) | |||||
case VoteTypeCommit: | case VoteTypeCommit: | ||||
return rs.Commits.AddVote(vote, source) | |||||
// Commits checks for height match. | |||||
cs.Votes.AddVote(vote) | |||||
cs.Precommits.AddVote(vote) | |||||
return cs.Commits.AddVote(vote) | |||||
default: | default: | ||||
panic("Unknown vote type") | panic("Unknown vote type") | ||||
} | } | ||||
} | } | ||||
func (rs *RoundState) Expired() bool { | |||||
return time.Now().After(rs.Expires) | |||||
// Lock the ProposalBlock if we have enough votes for it, | |||||
// or unlock an existing lock if +2/3 of votes were nil. | |||||
// Returns a blockhash if a block was locked. | |||||
func (cs *ConsensusState) LockOrUnlock(height uint32, round uint16) []byte { | |||||
cs.mtx.Lock() | |||||
defer cs.mtx.Unlock() | |||||
if cs.Height != height || cs.Round != round { | |||||
return nil | |||||
} | |||||
if hash, _, ok := cs.Votes.TwoThirdsMajority(); ok { | |||||
// Remember this POL. (hash may be nil) | |||||
cs.LockedPOL = cs.Votes.MakePOL() | |||||
if len(hash) == 0 { | |||||
// +2/3 voted nil. Just unlock. | |||||
cs.LockedBlock = nil | |||||
return nil | |||||
} else if cs.ProposalBlock.HashesTo(hash) { | |||||
// +2/3 voted for proposal block | |||||
// Validate the block. | |||||
// See note on ZombieValidators to see why. | |||||
if cs.stageBlock(cs.ProposalBlock) != nil { | |||||
log.Warning("+2/3 voted for an invalid block.") | |||||
return nil | |||||
} | |||||
cs.LockedBlock = cs.ProposalBlock | |||||
return hash | |||||
} else if cs.LockedBlock.HashesTo(hash) { | |||||
// +2/3 voted 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 | |||||
return nil | |||||
} | |||||
} else { | |||||
return nil | |||||
} | |||||
} | } | ||||
func (rs *RoundState) Step() uint8 { | |||||
rs.mtx.Lock() | |||||
defer rs.mtx.Unlock() | |||||
return rs.step | |||||
func (cs *ConsensusState) Commit(height uint32, round uint16) *Block { | |||||
cs.mtx.Lock() | |||||
defer cs.mtx.Unlock() | |||||
if cs.Height != height || cs.Round != round { | |||||
return nil | |||||
} | |||||
if hash, commitTime, ok := cs.Precommits.TwoThirdsMajority(); ok { | |||||
// There are some strange cases that shouldn't happen | |||||
// (unless voters are duplicitous). | |||||
// For example, the hash may not be the one that was | |||||
// proposed this round. These cases should be identified | |||||
// and warn the administrator. We should err on the side of | |||||
// caution and not, for example, sign a block. | |||||
// TODO: Identify these strange cases. | |||||
var block *Block | |||||
if cs.LockedBlock.HashesTo(hash) { | |||||
block = cs.LockedBlock | |||||
} else if cs.ProposalBlock.HashesTo(hash) { | |||||
block = cs.ProposalBlock | |||||
} else { | |||||
return nil | |||||
} | |||||
// The proposal must be valid. | |||||
if err := cs.stageBlock(block); err != nil { | |||||
log.Warning("Network is commiting an invalid proposal? %v", err) | |||||
return nil | |||||
} | |||||
// Save to blockStore | |||||
err := cs.blockStore.SaveBlock(block) | |||||
if err != nil { | |||||
return nil | |||||
} | |||||
// What was staged becomes committed. | |||||
state := cs.stagedState | |||||
state.Save(commitTime) | |||||
cs.updateToState(state) | |||||
// Update mempool. | |||||
cs.mempool.ResetForBlockAndState(block, state) | |||||
return block | |||||
} | |||||
return nil | |||||
} | } | ||||
func (rs *RoundState) SetStep(step uint8) bool { | |||||
rs.mtx.Lock() | |||||
defer rs.mtx.Unlock() | |||||
if rs.step < step { | |||||
rs.step = step | |||||
return true | |||||
func (cs *ConsensusState) stageBlock(block *Block) error { | |||||
// Already staged? | |||||
if cs.stagedBlock == block { | |||||
return nil | |||||
} | |||||
// Basic validation is done in state.CommitBlock(). | |||||
//err := block.ValidateBasic() | |||||
//if err != nil { | |||||
// return err | |||||
//} | |||||
// Create a copy of the state for staging | |||||
stateCopy := cs.state.Copy() // Deep copy the state before staging. | |||||
// Commit block onto the copied state. | |||||
err := stateCopy.CommitBlock(block) | |||||
if err != nil { | |||||
return err | |||||
} else { | } else { | ||||
return false | |||||
cs.stagedBlock = block | |||||
cs.stagedState = stateCopy | |||||
return nil | |||||
} | } | ||||
} | } |
@ -1,184 +0,0 @@ | |||||
package consensus | |||||
import ( | |||||
"bytes" | |||||
"errors" | |||||
"io" | |||||
"sync" | |||||
"time" | |||||
. "github.com/tendermint/tendermint/binary" | |||||
. "github.com/tendermint/tendermint/blocks" | |||||
. "github.com/tendermint/tendermint/state" | |||||
) | |||||
const ( | |||||
VoteTypeBare = byte(0x00) | |||||
VoteTypePrecommit = byte(0x01) | |||||
VoteTypeCommit = byte(0x02) | |||||
) | |||||
var ( | |||||
ErrVoteUnexpectedPhase = errors.New("Unexpected phase") | |||||
ErrVoteInvalidAccount = errors.New("Invalid round vote account") | |||||
ErrVoteInvalidSignature = errors.New("Invalid round vote signature") | |||||
ErrVoteInvalidHash = errors.New("Invalid hash") | |||||
ErrVoteConflictingSignature = errors.New("Conflicting round vote signature") | |||||
) | |||||
// Represents a bare, precommit, or commit vote for proposals. | |||||
type Vote struct { | |||||
Height uint32 | |||||
Round uint16 // zero if commit vote. | |||||
Type byte | |||||
Hash []byte // empty if vote is nil. | |||||
Signature | |||||
} | |||||
func ReadVote(r io.Reader, n *int64, err *error) *Vote { | |||||
return &Vote{ | |||||
Height: ReadUInt32(r, n, err), | |||||
Round: ReadUInt16(r, n, err), | |||||
Type: ReadByte(r, n, err), | |||||
Hash: ReadByteSlice(r, n, err), | |||||
Signature: ReadSignature(r, n, err), | |||||
} | |||||
} | |||||
func (v *Vote) WriteTo(w io.Writer) (n int64, err error) { | |||||
WriteUInt32(w, v.Height, &n, &err) | |||||
WriteUInt16(w, v.Round, &n, &err) | |||||
WriteByte(w, v.Type, &n, &err) | |||||
WriteByteSlice(w, v.Hash, &n, &err) | |||||
WriteBinary(w, v.Signature, &n, &err) | |||||
return | |||||
} | |||||
func (v *Vote) GetDocument() string { | |||||
return GenVoteDocument(v.Type, v.Height, v.Round, v.Hash) | |||||
} | |||||
//----------------------------------------------------------------------------- | |||||
// VoteSet helps collect signatures from validators at each height+round | |||||
// for a predefined vote type. | |||||
// TODO: test majority calculations etc. | |||||
type VoteSet struct { | |||||
mtx sync.Mutex | |||||
height uint32 | |||||
round uint16 | |||||
type_ byte | |||||
validators *ValidatorSet | |||||
votes map[uint64]*Vote | |||||
votesSources map[uint64][]string | |||||
votesByHash map[string]uint64 | |||||
totalVotes uint64 | |||||
totalVotingPower uint64 | |||||
oneThirdMajority [][]byte | |||||
twoThirdsCommitTime time.Time | |||||
} | |||||
// Constructs a new VoteSet struct used to accumulate votes for each round. | |||||
func NewVoteSet(height uint32, round uint16, type_ byte, validators *ValidatorSet) *VoteSet { | |||||
if type_ == VoteTypeCommit && round != 0 { | |||||
panic("Expected round 0 for commit vote set") | |||||
} | |||||
totalVotingPower := uint64(0) | |||||
for _, val := range validators.Map() { | |||||
totalVotingPower += val.VotingPower | |||||
} | |||||
return &VoteSet{ | |||||
height: height, | |||||
round: round, | |||||
type_: type_, | |||||
validators: validators, | |||||
votes: make(map[uint64]*Vote, validators.Size()), | |||||
votesSources: make(map[uint64][]string, validators.Size()), | |||||
votesByHash: make(map[string]uint64), | |||||
totalVotes: 0, | |||||
totalVotingPower: totalVotingPower, | |||||
} | |||||
} | |||||
// True if added, false if not. | |||||
// Returns ErrVote[UnexpectedPhase|InvalidAccount|InvalidSignature|InvalidHash|ConflictingSignature] | |||||
func (vs *VoteSet) AddVote(vote *Vote, source string) (bool, uint8, error) { | |||||
vs.mtx.Lock() | |||||
defer vs.mtx.Unlock() | |||||
// Make sure the phase matches. | |||||
if vote.Height != vs.height || | |||||
(vote.Type != VoteTypeCommit && vote.Round != vs.round) || | |||||
vote.Type != vs.type_ { | |||||
return false, 0, ErrVoteUnexpectedPhase | |||||
} | |||||
val := vs.validators.Get(vote.SignerId) | |||||
// Ensure that signer is a validator. | |||||
if val == nil { | |||||
return false, 0, ErrVoteInvalidAccount | |||||
} | |||||
// Check signature. | |||||
if !val.Verify([]byte(vote.GetDocument()), vote.Signature) { | |||||
// Bad signature. | |||||
return false, 0, ErrVoteInvalidSignature | |||||
} | |||||
// Get rank of vote & append provider key | |||||
var priorSources = vs.votesSources[vote.SignerId] | |||||
var rank = uint8(len(priorSources) + 1) | |||||
var alreadyProvidedByPeer = false | |||||
for i, otherPeer := range priorSources { | |||||
if otherPeer == source { | |||||
alreadyProvidedByPeer = true | |||||
rank = uint8(i + 1) | |||||
} | |||||
} | |||||
if !alreadyProvidedByPeer { | |||||
if len(priorSources) < voteRankCutoff { | |||||
vs.votesSources[vote.SignerId] = append(priorSources, source) | |||||
} | |||||
} | |||||
// If vote already exists, return false. | |||||
if existingVote, ok := vs.votes[vote.SignerId]; ok { | |||||
if bytes.Equal(existingVote.Hash, vote.Hash) { | |||||
return false, rank, nil | |||||
} else { | |||||
return false, rank, ErrVoteConflictingSignature | |||||
} | |||||
} | |||||
vs.votes[vote.SignerId] = vote | |||||
totalHashVotes := vs.votesByHash[string(vote.Hash)] + val.VotingPower | |||||
vs.votesByHash[string(vote.Hash)] = totalHashVotes | |||||
vs.totalVotes += val.VotingPower | |||||
// If we just nudged it up to one thirds majority, add it. | |||||
if totalHashVotes > vs.totalVotingPower/3 && | |||||
(totalHashVotes-val.VotingPower) <= vs.totalVotingPower/3 { | |||||
vs.oneThirdMajority = append(vs.oneThirdMajority, vote.Hash) | |||||
} else if totalHashVotes > vs.totalVotingPower*2/3 && | |||||
(totalHashVotes-val.VotingPower) <= vs.totalVotingPower*2/3 { | |||||
vs.twoThirdsCommitTime = time.Now() | |||||
} | |||||
return true, rank, nil | |||||
} | |||||
// Returns either a blockhash (or nil) that received +2/3 majority. | |||||
// If there exists no such majority, returns (nil, false). | |||||
func (vs *VoteSet) TwoThirdsMajority() (hash []byte, commitTime time.Time, ok bool) { | |||||
vs.mtx.Lock() | |||||
defer vs.mtx.Unlock() | |||||
// There's only one or two in the array. | |||||
for _, hash := range vs.oneThirdMajority { | |||||
if vs.votesByHash[string(hash)] > vs.totalVotingPower*2/3 { | |||||
return hash, vs.twoThirdsCommitTime, true | |||||
} | |||||
} | |||||
return nil, time.Time{}, false | |||||
} | |||||
// Returns blockhashes (or nil) that received a +1/3 majority. | |||||
// If there exists no such majority, returns nil. | |||||
func (vs *VoteSet) OneThirdMajority() (hashes [][]byte) { | |||||
vs.mtx.Lock() | |||||
defer vs.mtx.Unlock() | |||||
return vs.oneThirdMajority | |||||
} |
@ -0,0 +1,180 @@ | |||||
package consensus | |||||
import ( | |||||
"bytes" | |||||
"sync" | |||||
"time" | |||||
. "github.com/tendermint/tendermint/blocks" | |||||
. "github.com/tendermint/tendermint/common" | |||||
. "github.com/tendermint/tendermint/state" | |||||
) | |||||
// VoteSet helps collect signatures from validators at each height+round | |||||
// for a predefined vote type. | |||||
// Note that there three kinds of votes: (bare) votes, precommits, and commits. | |||||
// A commit of prior rounds can be added added in lieu of votes/precommits. | |||||
// TODO: test majority calculations etc. | |||||
type VoteSet struct { | |||||
height uint32 | |||||
round uint16 | |||||
type_ byte | |||||
mtx sync.Mutex | |||||
vset *ValidatorSet | |||||
votes map[uint64]*Vote | |||||
votesBitArray BitArray | |||||
votesByBlockHash map[string]uint64 | |||||
totalVotes uint64 | |||||
twoThirdsMajority []byte | |||||
twoThirdsCommitTime time.Time | |||||
} | |||||
// Constructs a new VoteSet struct used to accumulate votes for each round. | |||||
func NewVoteSet(height uint32, round uint16, type_ byte, vset *ValidatorSet) *VoteSet { | |||||
if type_ == VoteTypeCommit && round != 0 { | |||||
panic("Expected round 0 for commit vote set") | |||||
} | |||||
return &VoteSet{ | |||||
height: height, | |||||
round: round, | |||||
type_: type_, | |||||
vset: vset, | |||||
votes: make(map[uint64]*Vote, vset.Size()), | |||||
votesBitArray: NewBitArray(vset.Size()), | |||||
votesByBlockHash: make(map[string]uint64), | |||||
totalVotes: 0, | |||||
} | |||||
} | |||||
// True if added, false if not. | |||||
// Returns ErrVote[UnexpectedPhase|InvalidAccount|InvalidSignature|InvalidBlockHash|ConflictingSignature] | |||||
func (vs *VoteSet) AddVote(vote *Vote) (bool, error) { | |||||
vs.mtx.Lock() | |||||
defer vs.mtx.Unlock() | |||||
// Make sure the phase matches. (or that vote is commit && round < vs.round) | |||||
if vote.Height != vs.height || | |||||
(vote.Type != VoteTypeCommit && vote.Round != vs.round) || | |||||
(vote.Type != VoteTypeCommit && vote.Type != vs.type_) || | |||||
(vote.Type == VoteTypeCommit && vs.type_ != VoteTypeCommit && vote.Round >= vs.round) { | |||||
return false, ErrVoteUnexpectedPhase | |||||
} | |||||
// Ensure that signer is a validator. | |||||
val := vs.vset.GetById(vote.SignerId) | |||||
if val == nil { | |||||
return false, ErrVoteInvalidAccount | |||||
} | |||||
// Check signature. | |||||
if !val.Verify(vote.GenDocument(), vote.Signature) { | |||||
// Bad signature. | |||||
return false, ErrVoteInvalidSignature | |||||
} | |||||
return vs.addVote(vote) | |||||
} | |||||
func (vs *VoteSet) addVote(vote *Vote) (bool, error) { | |||||
// If vote already exists, return false. | |||||
if existingVote, ok := vs.votes[vote.SignerId]; ok { | |||||
if bytes.Equal(existingVote.BlockHash, vote.BlockHash) { | |||||
return false, nil | |||||
} else { | |||||
return false, ErrVoteConflictingSignature | |||||
} | |||||
} | |||||
// Add vote. | |||||
vs.votes[vote.SignerId] = vote | |||||
voterIndex, ok := vs.vset.GetIndexById(vote.SignerId) | |||||
if !ok { | |||||
return false, ErrVoteInvalidAccount | |||||
} | |||||
vs.votesBitArray.SetIndex(uint(voterIndex), true) | |||||
val := vs.vset.GetById(vote.SignerId) | |||||
totalBlockHashVotes := vs.votesByBlockHash[string(vote.BlockHash)] + val.VotingPower | |||||
vs.votesByBlockHash[string(vote.BlockHash)] = totalBlockHashVotes | |||||
vs.totalVotes += val.VotingPower | |||||
// If we just nudged it up to two thirds majority, add it. | |||||
if totalBlockHashVotes > vs.vset.TotalVotingPower()*2/3 && | |||||
(totalBlockHashVotes-val.VotingPower) <= vs.vset.TotalVotingPower()*2/3 { | |||||
vs.twoThirdsMajority = vote.BlockHash | |||||
vs.twoThirdsCommitTime = time.Now() | |||||
} | |||||
return true, nil | |||||
} | |||||
// Assumes that commits VoteSet is valid. | |||||
func (vs *VoteSet) AddVotesFromCommits(commits *VoteSet) { | |||||
commitVotes := commits.AllVotes() | |||||
for _, commit := range commitVotes { | |||||
if commit.Round < vs.round { | |||||
vs.addVote(commit) | |||||
} | |||||
} | |||||
} | |||||
func (vs *VoteSet) BitArray() BitArray { | |||||
vs.mtx.Lock() | |||||
defer vs.mtx.Unlock() | |||||
return vs.votesBitArray.Copy() | |||||
} | |||||
func (vs *VoteSet) GetVote(id uint64) *Vote { | |||||
vs.mtx.Lock() | |||||
defer vs.mtx.Unlock() | |||||
return vs.votes[id] | |||||
} | |||||
func (vs *VoteSet) AllVotes() []*Vote { | |||||
vs.mtx.Lock() | |||||
defer vs.mtx.Unlock() | |||||
votes := []*Vote{} | |||||
for _, vote := range vs.votes { | |||||
votes = append(votes, vote) | |||||
} | |||||
return votes | |||||
} | |||||
// Returns either a blockhash (or nil) that received +2/3 majority. | |||||
// If there exists no such majority, returns (nil, false). | |||||
func (vs *VoteSet) TwoThirdsMajority() (hash []byte, commitTime time.Time, ok bool) { | |||||
vs.mtx.Lock() | |||||
defer vs.mtx.Unlock() | |||||
if vs.twoThirdsCommitTime.IsZero() { | |||||
return nil, time.Time{}, false | |||||
} | |||||
return vs.twoThirdsMajority, vs.twoThirdsCommitTime, true | |||||
} | |||||
func (vs *VoteSet) MakePOL() *POL { | |||||
vs.mtx.Lock() | |||||
defer vs.mtx.Unlock() | |||||
if vs.twoThirdsCommitTime.IsZero() { | |||||
return nil | |||||
} | |||||
majHash := vs.twoThirdsMajority // hash may be nil. | |||||
pol := &POL{ | |||||
Height: vs.height, | |||||
Round: vs.round, | |||||
BlockHash: majHash, | |||||
} | |||||
for _, vote := range vs.votes { | |||||
if bytes.Equal(vote.BlockHash, majHash) { | |||||
if vote.Type == VoteTypeBare { | |||||
pol.Votes = append(pol.Votes, vote.Signature) | |||||
} else if vote.Type == VoteTypeCommit { | |||||
pol.Commits = append(pol.Votes, vote.Signature) | |||||
pol.CommitRounds = append(pol.CommitRounds, vote.Round) | |||||
} else { | |||||
Panicf("Unexpected vote type %X", vote.Type) | |||||
} | |||||
} | |||||
} | |||||
return pol | |||||
} |
@ -1,169 +0,0 @@ | |||||
package mempool | |||||
import ( | |||||
"bytes" | |||||
"fmt" | |||||
"io" | |||||
"sync/atomic" | |||||
. "github.com/tendermint/tendermint/binary" | |||||
. "github.com/tendermint/tendermint/blocks" | |||||
"github.com/tendermint/tendermint/p2p" | |||||
) | |||||
var ( | |||||
MempoolCh = byte(0x30) | |||||
) | |||||
// MempoolAgent handles mempool tx broadcasting amongst peers. | |||||
type MempoolAgent struct { | |||||
sw *p2p.Switch | |||||
swEvents chan interface{} | |||||
quit chan struct{} | |||||
started uint32 | |||||
stopped uint32 | |||||
mempool *Mempool | |||||
} | |||||
func NewMempoolAgent(sw *p2p.Switch, mempool *Mempool) *MempoolAgent { | |||||
swEvents := make(chan interface{}) | |||||
sw.AddEventListener("MempoolAgent.swEvents", swEvents) | |||||
memA := &MempoolAgent{ | |||||
sw: sw, | |||||
swEvents: swEvents, | |||||
quit: make(chan struct{}), | |||||
mempool: mempool, | |||||
} | |||||
return memA | |||||
} | |||||
func (memA *MempoolAgent) Start() { | |||||
if atomic.CompareAndSwapUint32(&memA.started, 0, 1) { | |||||
log.Info("Starting MempoolAgent") | |||||
go memA.switchEventsRoutine() | |||||
go memA.gossipTxRoutine() | |||||
} | |||||
} | |||||
func (memA *MempoolAgent) Stop() { | |||||
if atomic.CompareAndSwapUint32(&memA.stopped, 0, 1) { | |||||
log.Info("Stopping MempoolAgent") | |||||
close(memA.quit) | |||||
close(memA.swEvents) | |||||
} | |||||
} | |||||
func (memA *MempoolAgent) BroadcastTx(tx Tx) error { | |||||
err := memA.mempool.AddTx(tx) | |||||
if err != nil { | |||||
return err | |||||
} | |||||
msg := &TxMessage{Tx: tx} | |||||
memA.sw.Broadcast(MempoolCh, msg) | |||||
return nil | |||||
} | |||||
// Handle peer new/done events | |||||
func (memA *MempoolAgent) switchEventsRoutine() { | |||||
for { | |||||
swEvent, ok := <-memA.swEvents | |||||
if !ok { | |||||
break | |||||
} | |||||
switch swEvent.(type) { | |||||
case p2p.SwitchEventNewPeer: | |||||
// event := swEvent.(p2p.SwitchEventNewPeer) | |||||
case p2p.SwitchEventDonePeer: | |||||
// event := swEvent.(p2p.SwitchEventDonePeer) | |||||
default: | |||||
log.Warning("Unhandled switch event type") | |||||
} | |||||
} | |||||
} | |||||
func (memA *MempoolAgent) gossipTxRoutine() { | |||||
OUTER_LOOP: | |||||
for { | |||||
// Receive incoming message on MempoolCh | |||||
inMsg, ok := memA.sw.Receive(MempoolCh) | |||||
if !ok { | |||||
break OUTER_LOOP // Client has stopped | |||||
} | |||||
_, msg_ := decodeMessage(inMsg.Bytes) | |||||
log.Info("gossipTxRoutine received %v", msg_) | |||||
switch msg_.(type) { | |||||
case *TxMessage: | |||||
msg := msg_.(*TxMessage) | |||||
err := memA.mempool.AddTx(msg.Tx) | |||||
if err != nil { | |||||
// Bad, seen, or conflicting tx. | |||||
log.Debug("Could not add tx %v", msg.Tx) | |||||
continue OUTER_LOOP | |||||
} else { | |||||
log.Debug("Added valid tx %V", msg.Tx) | |||||
} | |||||
// Share tx. | |||||
// We use a simple shotgun approach for now. | |||||
// TODO: improve efficiency | |||||
for _, peer := range memA.sw.Peers().List() { | |||||
if peer.Key == inMsg.MConn.Peer.Key { | |||||
continue | |||||
} | |||||
peer.TrySend(MempoolCh, msg) | |||||
} | |||||
default: | |||||
// Ignore unknown message | |||||
// memA.sw.StopPeerForError(inMsg.MConn.Peer, errInvalidMessage) | |||||
} | |||||
} | |||||
// Cleanup | |||||
} | |||||
//----------------------------------------------------------------------------- | |||||
// Messages | |||||
const ( | |||||
msgTypeUnknown = byte(0x00) | |||||
msgTypeTx = byte(0x10) | |||||
) | |||||
// TODO: check for unnecessary extra bytes at the end. | |||||
func decodeMessage(bz []byte) (msgType byte, msg interface{}) { | |||||
n, err := new(int64), new(error) | |||||
// log.Debug("decoding msg bytes: %X", bz) | |||||
msgType = bz[0] | |||||
switch msgType { | |||||
case msgTypeTx: | |||||
msg = readTxMessage(bytes.NewReader(bz[1:]), n, err) | |||||
// case ...: | |||||
default: | |||||
msg = nil | |||||
} | |||||
return | |||||
} | |||||
//------------------------------------- | |||||
type TxMessage struct { | |||||
Tx Tx | |||||
} | |||||
func readTxMessage(r io.Reader, n *int64, err *error) *TxMessage { | |||||
return &TxMessage{ | |||||
Tx: ReadTx(r, n, err), | |||||
} | |||||
} | |||||
func (m *TxMessage) WriteTo(w io.Writer) (n int64, err error) { | |||||
WriteByte(w, msgTypeTx, &n, &err) | |||||
WriteBinary(w, m.Tx, &n, &err) | |||||
return | |||||
} | |||||
func (m *TxMessage) String() string { | |||||
return fmt.Sprintf("[TxMessage %v]", m.Tx) | |||||
} |
@ -0,0 +1,141 @@ | |||||
package mempool | |||||
import ( | |||||
"bytes" | |||||
"fmt" | |||||
"io" | |||||
"sync/atomic" | |||||
. "github.com/tendermint/tendermint/binary" | |||||
. "github.com/tendermint/tendermint/blocks" | |||||
. "github.com/tendermint/tendermint/p2p" | |||||
) | |||||
var ( | |||||
MempoolCh = byte(0x30) | |||||
) | |||||
// MempoolReactor handles mempool tx broadcasting amongst peers. | |||||
type MempoolReactor struct { | |||||
sw *Switch | |||||
quit chan struct{} | |||||
started uint32 | |||||
stopped uint32 | |||||
mempool *Mempool | |||||
} | |||||
func NewMempoolReactor(sw *Switch, mempool *Mempool) *MempoolReactor { | |||||
memR := &MempoolReactor{ | |||||
sw: sw, | |||||
quit: make(chan struct{}), | |||||
mempool: mempool, | |||||
} | |||||
return memR | |||||
} | |||||
func (memR *MempoolReactor) Start() { | |||||
if atomic.CompareAndSwapUint32(&memR.started, 0, 1) { | |||||
log.Info("Starting MempoolReactor") | |||||
} | |||||
} | |||||
func (memR *MempoolReactor) Stop() { | |||||
if atomic.CompareAndSwapUint32(&memR.stopped, 0, 1) { | |||||
log.Info("Stopping MempoolReactor") | |||||
close(memR.quit) | |||||
} | |||||
} | |||||
func (memR *MempoolReactor) BroadcastTx(tx Tx) error { | |||||
err := memR.mempool.AddTx(tx) | |||||
if err != nil { | |||||
return err | |||||
} | |||||
msg := &TxMessage{Tx: tx} | |||||
memR.sw.Broadcast(MempoolCh, msg) | |||||
return nil | |||||
} | |||||
// Implements Reactor | |||||
func (pexR *MempoolReactor) AddPeer(peer *Peer) { | |||||
} | |||||
// Implements Reactor | |||||
func (pexR *MempoolReactor) RemovePeer(peer *Peer, err error) { | |||||
} | |||||
func (memR *MempoolReactor) Receive(chId byte, src *Peer, msgBytes []byte) { | |||||
_, msg_ := decodeMessage(msgBytes) | |||||
log.Info("MempoolReactor received %v", msg_) | |||||
switch msg_.(type) { | |||||
case *TxMessage: | |||||
msg := msg_.(*TxMessage) | |||||
err := memR.mempool.AddTx(msg.Tx) | |||||
if err != nil { | |||||
// Bad, seen, or conflicting tx. | |||||
log.Debug("Could not add tx %v", msg.Tx) | |||||
return | |||||
} else { | |||||
log.Debug("Added valid tx %V", msg.Tx) | |||||
} | |||||
// Share tx. | |||||
// We use a simple shotgun approach for now. | |||||
// TODO: improve efficiency | |||||
for _, peer := range memR.sw.Peers().List() { | |||||
if peer.Key == src.Key { | |||||
continue | |||||
} | |||||
peer.TrySend(MempoolCh, msg) | |||||
} | |||||
default: | |||||
// Ignore unknown message | |||||
} | |||||
} | |||||
//----------------------------------------------------------------------------- | |||||
// Messages | |||||
const ( | |||||
msgTypeUnknown = byte(0x00) | |||||
msgTypeTx = byte(0x10) | |||||
) | |||||
// TODO: check for unnecessary extra bytes at the end. | |||||
func decodeMessage(bz []byte) (msgType byte, msg interface{}) { | |||||
n, err := new(int64), new(error) | |||||
// log.Debug("decoding msg bytes: %X", bz) | |||||
msgType = bz[0] | |||||
switch msgType { | |||||
case msgTypeTx: | |||||
msg = readTxMessage(bytes.NewReader(bz[1:]), n, err) | |||||
// case ...: | |||||
default: | |||||
msg = nil | |||||
} | |||||
return | |||||
} | |||||
//------------------------------------- | |||||
type TxMessage struct { | |||||
Tx Tx | |||||
} | |||||
func readTxMessage(r io.Reader, n *int64, err *error) *TxMessage { | |||||
return &TxMessage{ | |||||
Tx: ReadTx(r, n, err), | |||||
} | |||||
} | |||||
func (m *TxMessage) WriteTo(w io.Writer) (n int64, err error) { | |||||
WriteByte(w, msgTypeTx, &n, &err) | |||||
WriteBinary(w, m.Tx, &n, &err) | |||||
return | |||||
} | |||||
func (m *TxMessage) String() string { | |||||
return fmt.Sprintf("[TxMessage %v]", m.Tx) | |||||
} |
@ -1,278 +0,0 @@ | |||||
package p2p | |||||
import ( | |||||
"bytes" | |||||
"errors" | |||||
"fmt" | |||||
"io" | |||||
"sync/atomic" | |||||
"time" | |||||
. "github.com/tendermint/tendermint/binary" | |||||
. "github.com/tendermint/tendermint/common" | |||||
) | |||||
var pexErrInvalidMessage = errors.New("Invalid PEX message") | |||||
const ( | |||||
PexCh = byte(0x00) | |||||
ensurePeersPeriodSeconds = 30 | |||||
minNumOutboundPeers = 10 | |||||
maxNumPeers = 50 | |||||
) | |||||
/* | |||||
PEXAgent handles PEX (peer exchange) and ensures that an | |||||
adequate number of peers are connected to the switch. | |||||
*/ | |||||
type PEXAgent struct { | |||||
sw *Switch | |||||
swEvents chan interface{} | |||||
quit chan struct{} | |||||
started uint32 | |||||
stopped uint32 | |||||
book *AddrBook | |||||
} | |||||
func NewPEXAgent(sw *Switch, book *AddrBook) *PEXAgent { | |||||
swEvents := make(chan interface{}) | |||||
sw.AddEventListener("PEXAgent.swEvents", swEvents) | |||||
pexA := &PEXAgent{ | |||||
sw: sw, | |||||
swEvents: swEvents, | |||||
quit: make(chan struct{}), | |||||
book: book, | |||||
} | |||||
return pexA | |||||
} | |||||
func (pexA *PEXAgent) Start() { | |||||
if atomic.CompareAndSwapUint32(&pexA.started, 0, 1) { | |||||
log.Info("Starting PEXAgent") | |||||
go pexA.switchEventsRoutine() | |||||
go pexA.requestRoutine() | |||||
go pexA.ensurePeersRoutine() | |||||
} | |||||
} | |||||
func (pexA *PEXAgent) Stop() { | |||||
if atomic.CompareAndSwapUint32(&pexA.stopped, 0, 1) { | |||||
log.Info("Stopping PEXAgent") | |||||
close(pexA.quit) | |||||
close(pexA.swEvents) | |||||
} | |||||
} | |||||
// Asks peer for more addresses. | |||||
func (pexA *PEXAgent) RequestPEX(peer *Peer) { | |||||
peer.TrySend(PexCh, &pexRequestMessage{}) | |||||
} | |||||
func (pexA *PEXAgent) SendAddrs(peer *Peer, addrs []*NetAddress) { | |||||
peer.Send(PexCh, &pexAddrsMessage{Addrs: addrs}) | |||||
} | |||||
// For new outbound peers, announce our listener addresses if any, | |||||
// and if .book needs more addresses, ask for them. | |||||
func (pexA *PEXAgent) switchEventsRoutine() { | |||||
for { | |||||
swEvent, ok := <-pexA.swEvents | |||||
if !ok { | |||||
break | |||||
} | |||||
switch swEvent.(type) { | |||||
case SwitchEventNewPeer: | |||||
event := swEvent.(SwitchEventNewPeer) | |||||
if event.Peer.IsOutbound() { | |||||
pexA.SendAddrs(event.Peer, pexA.book.OurAddresses()) | |||||
if pexA.book.NeedMoreAddrs() { | |||||
pexA.RequestPEX(event.Peer) | |||||
} | |||||
} | |||||
case SwitchEventDonePeer: | |||||
// TODO | |||||
} | |||||
} | |||||
} | |||||
// Ensures that sufficient peers are connected. (continuous) | |||||
func (pexA *PEXAgent) ensurePeersRoutine() { | |||||
// fire once immediately. | |||||
pexA.ensurePeers() | |||||
// fire periodically | |||||
timer := NewRepeatTimer(ensurePeersPeriodSeconds * time.Second) | |||||
FOR_LOOP: | |||||
for { | |||||
select { | |||||
case <-timer.Ch: | |||||
pexA.ensurePeers() | |||||
case <-pexA.quit: | |||||
break FOR_LOOP | |||||
} | |||||
} | |||||
// Cleanup | |||||
timer.Stop() | |||||
} | |||||
// Ensures that sufficient peers are connected. (once) | |||||
func (pexA *PEXAgent) ensurePeers() { | |||||
numOutPeers, _, numDialing := pexA.sw.NumPeers() | |||||
numToDial := minNumOutboundPeers - (numOutPeers + numDialing) | |||||
if numToDial <= 0 { | |||||
return | |||||
} | |||||
toDial := NewCMap() | |||||
// Try to pick numToDial addresses to dial. | |||||
// TODO: improve logic. | |||||
for i := 0; i < numToDial; i++ { | |||||
newBias := MinInt(numOutPeers, 8)*10 + 10 | |||||
var picked *NetAddress | |||||
// Try to fetch a new peer 3 times. | |||||
// This caps the maximum number of tries to 3 * numToDial. | |||||
for j := 0; i < 3; j++ { | |||||
picked = pexA.book.PickAddress(newBias) | |||||
if picked == nil { | |||||
return | |||||
} | |||||
if toDial.Has(picked.String()) || | |||||
pexA.sw.IsDialing(picked) || | |||||
pexA.sw.Peers().Has(picked.String()) { | |||||
continue | |||||
} else { | |||||
break | |||||
} | |||||
} | |||||
if picked == nil { | |||||
continue | |||||
} | |||||
toDial.Set(picked.String(), picked) | |||||
} | |||||
// Dial picked addresses | |||||
for _, item := range toDial.Values() { | |||||
picked := item.(*NetAddress) | |||||
go func() { | |||||
_, err := pexA.sw.DialPeerWithAddress(picked) | |||||
if err != nil { | |||||
pexA.book.MarkAttempt(picked) | |||||
} | |||||
}() | |||||
} | |||||
} | |||||
// Handles incoming PEX messages. | |||||
func (pexA *PEXAgent) requestRoutine() { | |||||
for { | |||||
inMsg, ok := pexA.sw.Receive(PexCh) // {Peer, Time, Packet} | |||||
if !ok { | |||||
// Client has stopped | |||||
break | |||||
} | |||||
// decode message | |||||
msg := decodeMessage(inMsg.Bytes) | |||||
log.Info("requestRoutine received %v", msg) | |||||
switch msg.(type) { | |||||
case *pexRequestMessage: | |||||
// inMsg.MConn.Peer requested some peers. | |||||
// TODO: prevent abuse. | |||||
addrs := pexA.book.GetSelection() | |||||
msg := &pexAddrsMessage{Addrs: addrs} | |||||
queued := inMsg.MConn.Peer.TrySend(PexCh, msg) | |||||
if !queued { | |||||
// ignore | |||||
} | |||||
case *pexAddrsMessage: | |||||
// We received some peer addresses from inMsg.MConn.Peer. | |||||
// TODO: prevent abuse. | |||||
// (We don't want to get spammed with bad peers) | |||||
srcAddr := inMsg.MConn.RemoteAddress | |||||
for _, addr := range msg.(*pexAddrsMessage).Addrs { | |||||
pexA.book.AddAddress(addr, srcAddr) | |||||
} | |||||
default: | |||||
// Ignore unknown message. | |||||
// pexA.sw.StopPeerForError(inMsg.MConn.Peer, pexErrInvalidMessage) | |||||
} | |||||
} | |||||
// Cleanup | |||||
} | |||||
//----------------------------------------------------------------------------- | |||||
/* Messages */ | |||||
const ( | |||||
msgTypeUnknown = byte(0x00) | |||||
msgTypeRequest = byte(0x01) | |||||
msgTypeAddrs = byte(0x02) | |||||
) | |||||
// TODO: check for unnecessary extra bytes at the end. | |||||
func decodeMessage(bz []byte) (msg interface{}) { | |||||
var n int64 | |||||
var err error | |||||
// log.Debug("decoding msg bytes: %X", bz) | |||||
switch bz[0] { | |||||
case msgTypeRequest: | |||||
return &pexRequestMessage{} | |||||
case msgTypeAddrs: | |||||
return readPexAddrsMessage(bytes.NewReader(bz[1:]), &n, &err) | |||||
default: | |||||
return nil | |||||
} | |||||
} | |||||
/* | |||||
A pexRequestMessage requests additional peer addresses. | |||||
*/ | |||||
type pexRequestMessage struct { | |||||
} | |||||
func (m *pexRequestMessage) WriteTo(w io.Writer) (n int64, err error) { | |||||
WriteByte(w, msgTypeRequest, &n, &err) | |||||
return | |||||
} | |||||
func (m *pexRequestMessage) String() string { | |||||
return "[pexRequest]" | |||||
} | |||||
/* | |||||
A message with announced peer addresses. | |||||
*/ | |||||
type pexAddrsMessage struct { | |||||
Addrs []*NetAddress | |||||
} | |||||
func readPexAddrsMessage(r io.Reader, n *int64, err *error) *pexAddrsMessage { | |||||
numAddrs := int(ReadUInt32(r, n, err)) | |||||
addrs := []*NetAddress{} | |||||
for i := 0; i < numAddrs; i++ { | |||||
addr := ReadNetAddress(r, n, err) | |||||
addrs = append(addrs, addr) | |||||
} | |||||
return &pexAddrsMessage{ | |||||
Addrs: addrs, | |||||
} | |||||
} | |||||
func (m *pexAddrsMessage) WriteTo(w io.Writer) (n int64, err error) { | |||||
WriteByte(w, msgTypeAddrs, &n, &err) | |||||
WriteUInt32(w, uint32(len(m.Addrs)), &n, &err) | |||||
for _, addr := range m.Addrs { | |||||
WriteBinary(w, addr, &n, &err) | |||||
} | |||||
return | |||||
} | |||||
func (m *pexAddrsMessage) String() string { | |||||
return fmt.Sprintf("[pexAddrs %v]", m.Addrs) | |||||
} |
@ -0,0 +1,267 @@ | |||||
package p2p | |||||
import ( | |||||
"bytes" | |||||
"errors" | |||||
"fmt" | |||||
"io" | |||||
"sync/atomic" | |||||
"time" | |||||
. "github.com/tendermint/tendermint/binary" | |||||
. "github.com/tendermint/tendermint/common" | |||||
) | |||||
var pexErrInvalidMessage = errors.New("Invalid PEX message") | |||||
const ( | |||||
PexCh = byte(0x00) | |||||
ensurePeersPeriodSeconds = 30 | |||||
minNumOutboundPeers = 10 | |||||
maxNumPeers = 50 | |||||
) | |||||
/* | |||||
PEXReactor handles PEX (peer exchange) and ensures that an | |||||
adequate number of peers are connected to the switch. | |||||
*/ | |||||
type PEXReactor struct { | |||||
sw *Switch | |||||
quit chan struct{} | |||||
started uint32 | |||||
stopped uint32 | |||||
book *AddrBook | |||||
} | |||||
func NewPEXReactor(sw *Switch, book *AddrBook) *PEXReactor { | |||||
pexR := &PEXReactor{ | |||||
sw: sw, | |||||
quit: make(chan struct{}), | |||||
book: book, | |||||
} | |||||
return pexR | |||||
} | |||||
func (pexR *PEXReactor) Start() { | |||||
if atomic.CompareAndSwapUint32(&pexR.started, 0, 1) { | |||||
log.Info("Starting PEXReactor") | |||||
go pexR.ensurePeersRoutine() | |||||
} | |||||
} | |||||
func (pexR *PEXReactor) Stop() { | |||||
if atomic.CompareAndSwapUint32(&pexR.stopped, 0, 1) { | |||||
log.Info("Stopping PEXReactor") | |||||
close(pexR.quit) | |||||
} | |||||
} | |||||
// Asks peer for more addresses. | |||||
func (pexR *PEXReactor) RequestPEX(peer *Peer) { | |||||
peer.TrySend(PexCh, &pexRequestMessage{}) | |||||
} | |||||
func (pexR *PEXReactor) SendAddrs(peer *Peer, addrs []*NetAddress) { | |||||
peer.Send(PexCh, &pexRddrsMessage{Addrs: addrs}) | |||||
} | |||||
// Implements Reactor | |||||
func (pexR *PEXReactor) GetChannels() []*ChannelDescriptor { | |||||
// TODO optimize | |||||
return []*ChannelDescriptor{ | |||||
&ChannelDescriptor{ | |||||
Id: PexCh, | |||||
SendQueueCapacity: 1, | |||||
RecvQueueCapacity: 2, | |||||
RecvBufferSize: 1024, | |||||
DefaultPriority: 1, | |||||
}, | |||||
} | |||||
} | |||||
// Implements Reactor | |||||
func (pexR *PEXReactor) AddPeer(peer *Peer) { | |||||
if peer.IsOutbound() { | |||||
pexR.SendAddrs(peer, pexR.book.OurAddresses()) | |||||
if pexR.book.NeedMoreAddrs() { | |||||
pexR.RequestPEX(peer) | |||||
} | |||||
} | |||||
} | |||||
// Implements Reactor | |||||
func (pexR *PEXReactor) RemovePeer(peer *Peer, err error) { | |||||
// TODO | |||||
} | |||||
// Implements Reactor | |||||
// Handles incoming PEX messages. | |||||
func (pexR *PEXReactor) Receive(chId byte, src *Peer, msgBytes []byte) { | |||||
// decode message | |||||
msg := decodeMessage(msgBytes) | |||||
log.Info("requestRoutine received %v", msg) | |||||
switch msg.(type) { | |||||
case *pexRequestMessage: | |||||
// src requested some peers. | |||||
// TODO: prevent abuse. | |||||
addrs := pexR.book.GetSelection() | |||||
msg := &pexRddrsMessage{Addrs: addrs} | |||||
queued := src.TrySend(PexCh, msg) | |||||
if !queued { | |||||
// ignore | |||||
} | |||||
case *pexRddrsMessage: | |||||
// We received some peer addresses from src. | |||||
// TODO: prevent abuse. | |||||
// (We don't want to get spammed with bad peers) | |||||
srcAddr := src.RemoteAddress() | |||||
for _, addr := range msg.(*pexRddrsMessage).Addrs { | |||||
pexR.book.AddAddress(addr, srcAddr) | |||||
} | |||||
default: | |||||
// Ignore unknown message. | |||||
} | |||||
} | |||||
// Ensures that sufficient peers are connected. (continuous) | |||||
func (pexR *PEXReactor) ensurePeersRoutine() { | |||||
// fire once immediately. | |||||
pexR.ensurePeers() | |||||
// fire periodically | |||||
timer := NewRepeatTimer(ensurePeersPeriodSeconds * time.Second) | |||||
FOR_LOOP: | |||||
for { | |||||
select { | |||||
case <-timer.Ch: | |||||
pexR.ensurePeers() | |||||
case <-pexR.quit: | |||||
break FOR_LOOP | |||||
} | |||||
} | |||||
// Cleanup | |||||
timer.Stop() | |||||
} | |||||
// Ensures that sufficient peers are connected. (once) | |||||
func (pexR *PEXReactor) ensurePeers() { | |||||
numOutPeers, _, numDialing := pexR.sw.NumPeers() | |||||
numToDial := minNumOutboundPeers - (numOutPeers + numDialing) | |||||
if numToDial <= 0 { | |||||
return | |||||
} | |||||
toDial := NewCMap() | |||||
// Try to pick numToDial addresses to dial. | |||||
// TODO: improve logic. | |||||
for i := 0; i < numToDial; i++ { | |||||
newBias := MinInt(numOutPeers, 8)*10 + 10 | |||||
var picked *NetAddress | |||||
// Try to fetch a new peer 3 times. | |||||
// This caps the maximum number of tries to 3 * numToDial. | |||||
for j := 0; i < 3; j++ { | |||||
picked = pexR.book.PickAddress(newBias) | |||||
if picked == nil { | |||||
return | |||||
} | |||||
if toDial.Has(picked.String()) || | |||||
pexR.sw.IsDialing(picked) || | |||||
pexR.sw.Peers().Has(picked.String()) { | |||||
continue | |||||
} else { | |||||
break | |||||
} | |||||
} | |||||
if picked == nil { | |||||
continue | |||||
} | |||||
toDial.Set(picked.String(), picked) | |||||
} | |||||
// Dial picked addresses | |||||
for _, item := range toDial.Values() { | |||||
picked := item.(*NetAddress) | |||||
go func() { | |||||
_, err := pexR.sw.DialPeerWithAddress(picked) | |||||
if err != nil { | |||||
pexR.book.MarkAttempt(picked) | |||||
} | |||||
}() | |||||
} | |||||
} | |||||
//----------------------------------------------------------------------------- | |||||
/* Messages */ | |||||
const ( | |||||
msgTypeUnknown = byte(0x00) | |||||
msgTypeRequest = byte(0x01) | |||||
msgTypeAddrs = byte(0x02) | |||||
) | |||||
// TODO: check for unnecessary extra bytes at the end. | |||||
func decodeMessage(bz []byte) (msg interface{}) { | |||||
var n int64 | |||||
var err error | |||||
// log.Debug("decoding msg bytes: %X", bz) | |||||
switch bz[0] { | |||||
case msgTypeRequest: | |||||
return &pexRequestMessage{} | |||||
case msgTypeAddrs: | |||||
return readPexAddrsMessage(bytes.NewReader(bz[1:]), &n, &err) | |||||
default: | |||||
return nil | |||||
} | |||||
} | |||||
/* | |||||
A pexRequestMessage requests additional peer addresses. | |||||
*/ | |||||
type pexRequestMessage struct { | |||||
} | |||||
func (m *pexRequestMessage) WriteTo(w io.Writer) (n int64, err error) { | |||||
WriteByte(w, msgTypeRequest, &n, &err) | |||||
return | |||||
} | |||||
func (m *pexRequestMessage) String() string { | |||||
return "[pexRequest]" | |||||
} | |||||
/* | |||||
A message with announced peer addresses. | |||||
*/ | |||||
type pexRddrsMessage struct { | |||||
Addrs []*NetAddress | |||||
} | |||||
func readPexAddrsMessage(r io.Reader, n *int64, err *error) *pexRddrsMessage { | |||||
numAddrs := int(ReadUInt32(r, n, err)) | |||||
addrs := []*NetAddress{} | |||||
for i := 0; i < numAddrs; i++ { | |||||
addr := ReadNetAddress(r, n, err) | |||||
addrs = append(addrs, addr) | |||||
} | |||||
return &pexRddrsMessage{ | |||||
Addrs: addrs, | |||||
} | |||||
} | |||||
func (m *pexRddrsMessage) WriteTo(w io.Writer) (n int64, err error) { | |||||
WriteByte(w, msgTypeAddrs, &n, &err) | |||||
WriteUInt32(w, uint32(len(m.Addrs)), &n, &err) | |||||
for _, addr := range m.Addrs { | |||||
WriteBinary(w, addr, &n, &err) | |||||
} | |||||
return | |||||
} | |||||
func (m *pexRddrsMessage) String() string { | |||||
return fmt.Sprintf("[pexRddrs %v]", m.Addrs) | |||||
} |