draft of consensus+state code, compiles.

This commit is contained in:
Jae Kwon
2014-09-04 03:32:38 -07:00
parent e53b148acf
commit f030c69495
13 changed files with 222 additions and 217 deletions

View File

@ -1,18 +1,15 @@
package binary package binary
import ( import (
"errors" "bytes"
"io"
"time" "time"
) )
type Codec interface { type Codec interface {
WriteTo(io.Writer, interface{}, *int64, *error) Write(interface{}) ([]byte, error)
ReadFrom(io.Reader, *int64, *error) interface{} Read([]byte) (interface{}, error)
} }
//-----------------------------------------------------------------------------
const ( const (
typeNil = byte(0x00) typeNil = byte(0x00)
typeByte = byte(0x01) typeByte = byte(0x01)
@ -33,7 +30,8 @@ var BasicCodec = basicCodec{}
type basicCodec struct{} type basicCodec struct{}
func (bc basicCodec) WriteTo(w io.Writer, o interface{}, n *int64, err *error) { func (bc basicCodec) Write(o interface{}) ([]byte, error) {
n, err, w := new(int64), new(error), new(bytes.Buffer)
switch o.(type) { switch o.(type) {
case nil: case nil:
WriteByte(w, typeNil, n, err) WriteByte(w, typeNil, n, err)
@ -76,63 +74,41 @@ func (bc basicCodec) WriteTo(w io.Writer, o interface{}, n *int64, err *error) {
default: default:
panic("Unsupported type") panic("Unsupported type")
} }
return return w.Bytes(), *err
} }
func (bc basicCodec) ReadFrom(r io.Reader, n *int64, err *error) interface{} { func (bc basicCodec) Read(bz []byte) (interface{}, error) {
n, err, r, o := new(int64), new(error), bytes.NewBuffer(bz), interface{}(nil)
type_ := ReadByte(r, n, err) type_ := ReadByte(r, n, err)
switch type_ { switch type_ {
case typeNil: case typeNil:
return nil o = nil
case typeByte: case typeByte:
return ReadByte(r, n, err) o = ReadByte(r, n, err)
case typeInt8: case typeInt8:
return ReadInt8(r, n, err) o = ReadInt8(r, n, err)
//case typeUInt8: //case typeUInt8:
// return ReadUInt8(r, n, err) // o = ReadUInt8(r, n, err)
case typeInt16: case typeInt16:
return ReadInt16(r, n, err) o = ReadInt16(r, n, err)
case typeUInt16: case typeUInt16:
return ReadUInt16(r, n, err) o = ReadUInt16(r, n, err)
case typeInt32: case typeInt32:
return ReadInt32(r, n, err) o = ReadInt32(r, n, err)
case typeUInt32: case typeUInt32:
return ReadUInt32(r, n, err) o = ReadUInt32(r, n, err)
case typeInt64: case typeInt64:
return ReadInt64(r, n, err) o = ReadInt64(r, n, err)
case typeUInt64: case typeUInt64:
return ReadUInt64(r, n, err) o = ReadUInt64(r, n, err)
case typeString: case typeString:
return ReadString(r, n, err) o = ReadString(r, n, err)
case typeByteSlice: case typeByteSlice:
return ReadByteSlice(r, n, err) o = ReadByteSlice(r, n, err)
case typeTime: case typeTime:
return ReadTime(r, n, err) o = ReadTime(r, n, err)
default: default:
panic("Unsupported type") panic("Unsupported type")
} }
} return o, *err
//-----------------------------------------------------------------------------
// Creates an adapter codec for Binary things.
// Resulting Codec can be used with merkle/*.
type BinaryCodec struct {
decoder func(io.Reader, *int64, *error) interface{}
}
func NewBinaryCodec(decoder func(io.Reader, *int64, *error) interface{}) *BinaryCodec {
return &BinaryCodec{decoder}
}
func (ca *BinaryCodec) WriteTo(w io.Writer, o interface{}, n *int64, err *error) {
if bo, ok := o.(Binary); ok {
WriteTo(w, BinaryBytes(bo), n, err)
} else {
*err = errors.New("BinaryCodec expected Binary object")
}
}
func (ca *BinaryCodec) ReadFrom(r io.Reader, n *int64, err *error) interface{} {
return ca.decoder(r, n, err)
} }

View File

@ -1,63 +0,0 @@
package blocks
import (
. "github.com/tendermint/tendermint/binary"
"io"
)
// NOTE: consensus/Validator embeds this, so..
type Account struct {
Id uint64 // Numeric id of account, incrementing.
PubKey []byte
}
func (self *Account) Verify(msg []byte, sig []byte) bool {
return false
}
//-----------------------------------------------------------------------------
type PrivAccount struct {
Account
PrivKey []byte
}
func (self *PrivAccount) Sign(msg []byte) Signature {
return Signature{}
}
//-----------------------------------------------------------------------------
/*
Signature message wire format:
|A...|SSS...|
A account number, varint encoded (1+ bytes)
S signature of all prior bytes (32 bytes)
It usually follows the message to be signed.
*/
type Signature struct {
SignerId uint64
Bytes []byte
}
func ReadSignature(r io.Reader, n *int64, err *error) Signature {
return Signature{
SignerId: ReadUInt64(r, n, err),
Bytes: ReadByteSlice(r, n, err),
}
}
func (sig Signature) IsZero() bool {
return len(sig.Bytes) == 0
}
func (sig Signature) WriteTo(w io.Writer) (n int64, err error) {
WriteUInt64(w, sig.SignerId, &n, &err)
WriteByteSlice(w, sig.Bytes, &n, &err)
return
}

View File

@ -155,21 +155,6 @@ func (bp *BlockPart) BlockPartHash() []byte {
} }
} }
// Signs the URI, which includes all data and metadata.
// XXX implement or change
func (bp *BlockPart) Sign(acc *PrivAccount) {
// TODO: populate Signature
}
// XXX maybe change.
func (bp *BlockPart) ValidateWithSigner(signer *Account) error {
// TODO: Sanity check height, index, total, bytes, etc.
if !signer.Verify([]byte(bp.URI()), bp.Signature.Bytes) {
return ErrInvalidBlockPartSignature
}
return nil
}
//----------------------------------------------------------------------------- //-----------------------------------------------------------------------------
/* Header is part of a Block */ /* Header is part of a Block */

View File

@ -33,13 +33,6 @@ func randTime() time.Time {
return time.Unix(int64(randUInt64()), 0) return time.Unix(int64(randUInt64()), 0)
} }
func randAccount() Account {
return Account{
Id: randUInt64(),
PubKey: randBytes(32),
}
}
func randBytes(n int) []byte { func randBytes(n int) []byte {
bs := make([]byte, n) bs := make([]byte, n)
for i := 0; i < n; i++ { for i := 0; i < n; i++ {

40
blocks/signature.go Normal file
View File

@ -0,0 +1,40 @@
package blocks
import (
. "github.com/tendermint/tendermint/binary"
"io"
)
/*
Signature message wire format:
|a...|sss...|
a Account number, varint encoded (1+ bytes)
s Signature of all prior bytes (32 bytes)
It usually follows the message to be signed.
*/
type Signature struct {
SignerId uint64
Bytes []byte
}
func ReadSignature(r io.Reader, n *int64, err *error) Signature {
return Signature{
SignerId: ReadUInt64(r, n, err),
Bytes: ReadByteSlice(r, n, err),
}
}
func (sig Signature) IsZero() bool {
return len(sig.Bytes) == 0
}
func (sig Signature) WriteTo(w io.Writer) (n int64, err error) {
WriteUInt64(w, sig.SignerId, &n, &err)
WriteByteSlice(w, sig.Bytes, &n, &err)
return
}

View File

@ -1,9 +1,12 @@
package blocks package consensus
import ( import (
"bytes" "bytes"
"errors" "errors"
"sync" "sync"
. "github.com/tendermint/tendermint/blocks"
. "github.com/tendermint/tendermint/state"
) )
// Helper for keeping track of block parts. // Helper for keeping track of block parts.
@ -72,10 +75,12 @@ func (bps *BlockPartSet) AddBlockPart(part *BlockPart) (added bool, err error) {
defer bps.mtx.Unlock() defer bps.mtx.Unlock()
// If part is invalid, return an error. // If part is invalid, return an error.
/* XXX
err = part.ValidateWithSigner(bps.signer) err = part.ValidateWithSigner(bps.signer)
if err != nil { if err != nil {
return false, err return false, err
} }
*/
if bps.parts == nil { if bps.parts == nil {
// First received part for this round. // First received part for this round.

View File

@ -320,7 +320,7 @@ func (cm *ConsensusManager) stageProposal(proposal *BlockPartSet) error {
if !proposal.IsComplete() { if !proposal.IsComplete() {
return errors.New("Incomplete proposal BlockPartSet") return errors.New("Incomplete proposal BlockPartSet")
} }
block, blockParts := blockPartSet.Block(), blockPartSet.BlockParts() block := proposal.Block()
err := block.ValidateBasic() err := block.ValidateBasic()
if err != nil { if err != nil {
return err return err
@ -332,7 +332,7 @@ func (cm *ConsensusManager) stageProposal(proposal *BlockPartSet) error {
cm.mtx.Unlock() cm.mtx.Unlock()
// Commit block onto the copied state. // Commit block onto the copied state.
err := stateCopy.CommitBlock(block, block.Header.Time) // NOTE: fake commit time. err = stateCopy.CommitBlock(block, block.Header.Time) // NOTE: fake commit time.
if err != nil { if err != nil {
return err return err
} }
@ -340,7 +340,7 @@ func (cm *ConsensusManager) stageProposal(proposal *BlockPartSet) error {
// Looks good! // Looks good!
cm.mtx.Lock() cm.mtx.Lock()
cm.stagedProposal = proposal cm.stagedProposal = proposal
cm.stagedState = state cm.stagedState = stateCopy
cm.mtx.Unlock() cm.mtx.Unlock()
return nil return nil
} }
@ -379,7 +379,7 @@ func (cm *ConsensusManager) voteProposal(rs *RoundState) error {
return err return err
} }
// Vote for block. // Vote for block.
err := cm.signAndVote(&Vote{ err = cm.signAndVote(&Vote{
Height: rs.Height, Height: rs.Height,
Round: rs.Round, Round: rs.Round,
Type: VoteTypeBare, Type: VoteTypeBare,
@ -788,25 +788,26 @@ func (ps *PeerState) ApplyVoteRankMessage(msg *VoteRankMessage) error {
// Messages // Messages
const ( const (
msgTypeUnknown = Byte(0x00) msgTypeUnknown = byte(0x00)
msgTypeBlockPart = Byte(0x10) msgTypeBlockPart = byte(0x10)
msgTypeKnownBlockParts = Byte(0x11) msgTypeKnownBlockParts = byte(0x11)
msgTypeVote = Byte(0x20) msgTypeVote = byte(0x20)
msgTypeVoteRank = Byte(0x21) msgTypeVoteRank = byte(0x21)
) )
// TODO: check for unnecessary extra bytes at the end. // TODO: check for unnecessary extra bytes at the end.
func decodeMessage(bz ByteSlice) (msg interface{}) { func decodeMessage(bz []byte) (msg interface{}) {
n, err := new(int64), new(error)
// log.Debug("decoding msg bytes: %X", bz) // log.Debug("decoding msg bytes: %X", bz)
switch Byte(bz[0]) { switch bz[0] {
case msgTypeBlockPart: case msgTypeBlockPart:
return readBlockPartMessage(bytes.NewReader(bz[1:])) return readBlockPartMessage(bytes.NewReader(bz[1:]), n, err)
case msgTypeKnownBlockParts: case msgTypeKnownBlockParts:
return readKnownBlockPartsMessage(bytes.NewReader(bz[1:])) return readKnownBlockPartsMessage(bytes.NewReader(bz[1:]), n, err)
case msgTypeVote: case msgTypeVote:
return ReadVote(bytes.NewReader(bz[1:])) return ReadVote(bytes.NewReader(bz[1:]), n, err)
case msgTypeVoteRank: case msgTypeVoteRank:
return readVoteRankMessage(bytes.NewReader(bz[1:])) return readVoteRankMessage(bytes.NewReader(bz[1:]), n, err)
default: default:
return nil return nil
} }
@ -818,15 +819,15 @@ type BlockPartMessage struct {
BlockPart *BlockPart BlockPart *BlockPart
} }
func readBlockPartMessage(r io.Reader) *BlockPartMessage { func readBlockPartMessage(r io.Reader, n *int64, err *error) *BlockPartMessage {
return &BlockPartMessage{ return &BlockPartMessage{
BlockPart: ReadBlockPart(r), BlockPart: ReadBlockPart(r, n, err),
} }
} }
func (m *BlockPartMessage) WriteTo(w io.Writer) (n int64, err error) { func (m *BlockPartMessage) WriteTo(w io.Writer) (n int64, err error) {
n, err = WriteTo(msgTypeBlockPart, w, n, err) WriteByte(w, msgTypeBlockPart, &n, &err)
n, err = WriteTo(m.BlockPart, w, n, err) WriteBinary(w, m.BlockPart, &n, &err)
return return
} }
@ -839,22 +840,22 @@ func (m *BlockPartMessage) String() string {
type KnownBlockPartsMessage struct { type KnownBlockPartsMessage struct {
Height uint32 Height uint32
SecondsSinceStartTime uint32 SecondsSinceStartTime uint32
BlockPartsBitArray ByteSlice BlockPartsBitArray []byte
} }
func readKnownBlockPartsMessage(r io.Reader) *KnownBlockPartsMessage { func readKnownBlockPartsMessage(r io.Reader, n *int64, err *error) *KnownBlockPartsMessage {
return &KnownBlockPartsMessage{ return &KnownBlockPartsMessage{
Height: Readuint32(r), Height: ReadUInt32(r, n, err),
SecondsSinceStartTime: Readuint32(r), SecondsSinceStartTime: ReadUInt32(r, n, err),
BlockPartsBitArray: ReadByteSlice(r), BlockPartsBitArray: ReadByteSlice(r, n, err),
} }
} }
func (m *KnownBlockPartsMessage) WriteTo(w io.Writer) (n int64, err error) { func (m *KnownBlockPartsMessage) WriteTo(w io.Writer) (n int64, err error) {
n, err = WriteTo(msgTypeKnownBlockParts, w, n, err) WriteByte(w, msgTypeKnownBlockParts, &n, &err)
n, err = WriteTo(UInt32(m.Height), w, n, err) WriteUInt32(w, m.Height, &n, &err)
n, err = WriteTo(UInt32(m.SecondsSinceStartTime), w, n, err) WriteUInt32(w, m.SecondsSinceStartTime, &n, &err)
n, err = WriteTo(m.BlockPartsBitArray, w, n, err) WriteByteSlice(w, m.BlockPartsBitArray, &n, &err)
return return
} }
@ -871,17 +872,17 @@ type VoteRankMessage struct {
Rank uint8 Rank uint8
} }
func readVoteRankMessage(r io.Reader) *VoteRankMessage { func readVoteRankMessage(r io.Reader, n *int64, err *error) *VoteRankMessage {
return &VoteRankMessage{ return &VoteRankMessage{
ValidatorId: Readuint64(r), ValidatorId: ReadUInt64(r, n, err),
Rank: Readuint8(r), Rank: ReadUInt8(r, n, err),
} }
} }
func (m *VoteRankMessage) WriteTo(w io.Writer) (n int64, err error) { func (m *VoteRankMessage) WriteTo(w io.Writer) (n int64, err error) {
n, err = WriteTo(msgTypeVoteRank, w, n, err) WriteByte(w, msgTypeVoteRank, &n, &err)
n, err = WriteTo(UInt64(m.ValidatorId), w, n, err) WriteUInt64(w, m.ValidatorId, &n, &err)
n, err = WriteTo(UInt8(m.Rank), w, n, err) WriteUInt8(w, m.Rank, &n, &err)
return return
} }

View File

@ -0,0 +1,20 @@
package consensus
import (
db_ "github.com/tendermint/tendermint/db"
. "github.com/tendermint/tendermint/state"
)
//-----------------------------------------------------------------------------
// TODO: Ensure that double signing never happens via an external persistent check.
type PrivValidator struct {
PrivAccount
db *db_.LevelDB
}
// Modifies the vote object in memory.
// Double signing results in an error.
func (pv *PrivValidator) SignVote(vote *Vote) error {
return nil
}

View File

@ -1,12 +1,9 @@
package consensus package consensus
import ( import (
"bytes"
"sync" "sync"
"time" "time"
. "github.com/tendermint/tendermint/binary"
. "github.com/tendermint/tendermint/blocks"
. "github.com/tendermint/tendermint/common" . "github.com/tendermint/tendermint/common"
. "github.com/tendermint/tendermint/state" . "github.com/tendermint/tendermint/state"
) )
@ -18,12 +15,12 @@ var (
// 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 mtx sync.Mutex
height uint32 // Height we are working on. height uint32 // Height we are working on.
validatorsR0 map[uint64]*Validator // A copy of the validators at round 0 validatorsR0 *ValidatorSet // A copy of the validators at round 0
lockedProposal *BlockPartSet // A BlockPartSet of the locked proposal. lockedProposal *BlockPartSet // A BlockPartSet of the locked proposal.
startTime time.Time // Start of round 0 for this height. startTime time.Time // Start of round 0 for this height.
commits *VoteSet // Commits for this height. commits *VoteSet // Commits for this height.
roundState *RoundState // The RoundState object for the current round. roundState *RoundState // The RoundState object for the current round.
} }
func NewConsensusState(state *State) *ConsensusState { func NewConsensusState(state *State) *ConsensusState {
@ -70,7 +67,7 @@ func (cs *ConsensusState) Update(state *State) {
cs.height = stateHeight cs.height = stateHeight
cs.validatorsR0 = state.Validators().Copy() // NOTE: immutable. cs.validatorsR0 = state.Validators().Copy() // NOTE: immutable.
cs.lockedProposal = nil cs.lockedProposal = nil
cs.startTime = state.commitTime // XXX is this what we want? cs.startTime = state.CommitTime() // XXX is this what we want?
cs.commits = NewVoteSet(stateHeight, 0, VoteTypeCommit, cs.validatorsR0) cs.commits = NewVoteSet(stateHeight, 0, VoteTypeCommit, cs.validatorsR0)
// Setup the roundState // Setup the roundState
@ -94,20 +91,20 @@ func (cs *ConsensusState) SetupRound(round uint16) {
func (cs *ConsensusState) setupRound(round uint16) { func (cs *ConsensusState) setupRound(round uint16) {
// Increment validator accums as necessary. // Increment validator accums as necessary.
// We need to start with cs.validatorsR0 or cs.roundState.Validators // We need to start with cs.validatorsR0 or cs.roundState.Validators
var validators map[uint64]*Validator = nil var validators *ValidatorSet
var validatorsRound uint16 var validatorsRound uint16
if cs.roundState == nil { if cs.roundState == nil {
// We have no roundState so we start from validatorsR0 at round 0. // We have no roundState so we start from validatorsR0 at round 0.
validators = copyValidators(cs.validatorsR0) validators = cs.validatorsR0.Copy()
validatorsRound = 0 validatorsRound = 0
} else { } else {
// We have a previous roundState so we start from that. // We have a previous roundState so we start from that.
validators = copyValidators(cs.roundState.Validators) validators = cs.roundState.Validators.Copy()
validatorsRound = cs.roundState.Round validatorsRound = cs.roundState.Round
} }
// Increment all the way to round. // Increment all the way to round.
for r := validatorsRound; r < round; r++ { for r := validatorsRound; r < round; r++ {
incrementAccum(validators) validators.IncrementAccum()
} }
roundState := NewRoundState(cs.height, round, cs.startTime, validators, cs.commits) roundState := NewRoundState(cs.height, round, cs.startTime, validators, cs.commits)
@ -128,25 +125,25 @@ const (
// RoundState encapsulates all the state needed to engage in the consensus protocol. // RoundState encapsulates all the state needed to engage in the consensus protocol.
type RoundState struct { type RoundState struct {
Height uint32 // Immutable Height uint32 // Immutable
Round uint16 // Immutable Round uint16 // Immutable
StartTime time.Time // Time in which consensus started for this height. StartTime time.Time // Time in which consensus started for this height.
Expires time.Time // Time after which this round is expired. Expires time.Time // Time after which this round is expired.
Proposer *Validator // The proposer to propose a block for this round. Proposer *Validator // The proposer to propose a block for this round.
Validators map[uint64]*Validator // All validators with modified accumPower for this round. Validators *ValidatorSet // All validators with modified accumPower for this round.
BlockPartSet *BlockPartSet // All block parts received for this round. BlockPartSet *BlockPartSet // All block parts received for this round.
RoundBareVotes *VoteSet // All votes received for this round. RoundBareVotes *VoteSet // All votes received for this round.
RoundPrecommits *VoteSet // All precommits received for this round. RoundPrecommits *VoteSet // All precommits received for this round.
Commits *VoteSet // A shared object for all commit votes of this height. Commits *VoteSet // A shared object for all commit votes of this height.
mtx sync.Mutex mtx sync.Mutex
step uint8 // mutable step uint8 // mutable
} }
func NewRoundState(height uint32, round uint16, startTime time.Time, func NewRoundState(height uint32, round uint16, startTime time.Time,
validators map[uint64]*Validator, commits *VoteSet) *RoundState { validators *ValidatorSet, commits *VoteSet) *RoundState {
proposer := getProposer(validators) proposer := validators.GetProposer()
blockPartSet := NewBlockPartSet(height, round, &(proposer.Account)) blockPartSet := NewBlockPartSet(height, round, &(proposer.Account))
roundBareVotes := NewVoteSet(height, round, VoteTypeBare, validators) roundBareVotes := NewVoteSet(height, round, VoteTypeBare, validators)
roundPrecommits := NewVoteSet(height, round, VoteTypePrecommit, validators) roundPrecommits := NewVoteSet(height, round, VoteTypePrecommit, validators)

View File

@ -1,4 +1,4 @@
package state package consensus
import ( import (
"bytes" "bytes"
@ -10,6 +10,7 @@ import (
. "github.com/tendermint/tendermint/binary" . "github.com/tendermint/tendermint/binary"
. "github.com/tendermint/tendermint/blocks" . "github.com/tendermint/tendermint/blocks"
"github.com/tendermint/tendermint/config" "github.com/tendermint/tendermint/config"
. "github.com/tendermint/tendermint/state"
) )
const ( const (
@ -104,7 +105,7 @@ type VoteSet struct {
height uint32 height uint32
round uint16 round uint16
type_ byte type_ byte
validators map[uint64]*Validator validators *ValidatorSet
votes map[uint64]*Vote votes map[uint64]*Vote
votesByHash map[string]uint64 votesByHash map[string]uint64
totalVotes uint64 totalVotes uint64
@ -112,9 +113,9 @@ type VoteSet struct {
} }
// Constructs a new VoteSet struct used to accumulate votes for each round. // Constructs a new VoteSet struct used to accumulate votes for each round.
func NewVoteSet(height uint32, round uint16, type_ byte, validators map[uint64]*Validator) *VoteSet { func NewVoteSet(height uint32, round uint16, type_ byte, validators *ValidatorSet) *VoteSet {
totalVotingPower := uint64(0) totalVotingPower := uint64(0)
for _, val := range validators { for _, val := range validators.Map() {
totalVotingPower += val.VotingPower totalVotingPower += val.VotingPower
} }
return &VoteSet{ return &VoteSet{
@ -122,7 +123,7 @@ func NewVoteSet(height uint32, round uint16, type_ byte, validators map[uint64]*
round: round, round: round,
type_: type_, type_: type_,
validators: validators, validators: validators,
votes: make(map[uint64]*Vote, len(validators)), votes: make(map[uint64]*Vote, validators.Size()),
votesByHash: make(map[string]uint64), votesByHash: make(map[string]uint64),
totalVotes: 0, totalVotes: 0,
totalVotingPower: totalVotingPower, totalVotingPower: totalVotingPower,
@ -140,7 +141,7 @@ func (vs *VoteSet) AddVote(vote *Vote) (bool, error) {
return false, ErrVoteUnexpectedPhase return false, ErrVoteUnexpectedPhase
} }
val := vs.validators[vote.SignerId] val := vs.validators.Get(vote.SignerId)
// Ensure that signer is a validator. // Ensure that signer is a validator.
if val == nil { if val == nil {
return false, ErrVoteInvalidAccount return false, ErrVoteInvalidAccount

53
state/account.go Normal file
View File

@ -0,0 +1,53 @@
package state
import (
. "github.com/tendermint/tendermint/binary"
. "github.com/tendermint/tendermint/blocks"
"io"
)
// NOTE: consensus/Validator embeds this, so..
type Account struct {
Id uint64 // Numeric id of account, incrementing.
PubKey []byte
}
func ReadAccount(r io.Reader, n *int64, err *error) *Account {
return &Account{
Id: ReadUInt64(r, n, err),
PubKey: ReadByteSlice(r, n, err),
}
}
func (self *Account) Verify(msg []byte, sig []byte) bool {
return false
}
//-----------------------------------------------------------------------------
type PrivAccount struct {
Account
PrivKey []byte
}
func (self *PrivAccount) Sign(msg []byte) Signature {
return Signature{}
}
/*
// Signs the URI, which includes all data and metadata.
// XXX implement or change
func (bp *BlockPart) Sign(acc *PrivAccount) {
// TODO: populate Signature
}
// XXX maybe change.
func (bp *BlockPart) ValidateWithSigner(signer *Account) error {
// TODO: Sanity check height, index, total, bytes, etc.
if !signer.Verify([]byte(bp.URI()), bp.Signature.Bytes) {
return ErrInvalidBlockPartSignature
}
return nil
}
*/

View File

@ -119,9 +119,18 @@ func (s *State) Validators() *ValidatorSet {
return s.validators return s.validators
} }
func (s *State) Account(accountId uint64) *Account { func (s *State) Account(accountId uint64) (*Account, error) {
s.mtx.Lock() s.mtx.Lock()
defer s.mtx.Unlock() defer s.mtx.Unlock()
// XXX: figure out a way to load an Account Binary type. idBytes, err := BasicCodec.Write(accountId)
return s.accounts.Get(accountId) if err != nil {
return nil, err
}
accountBytes := s.accounts.Get(idBytes)
if accountBytes == nil {
return nil, nil
}
n, err := int64(0), error(nil)
account := ReadAccount(bytes.NewBuffer(accountBytes), &n, &err)
return account, err
} }

View File

@ -4,9 +4,7 @@ import (
"io" "io"
. "github.com/tendermint/tendermint/binary" . "github.com/tendermint/tendermint/binary"
. "github.com/tendermint/tendermint/blocks"
. "github.com/tendermint/tendermint/common" . "github.com/tendermint/tendermint/common"
db_ "github.com/tendermint/tendermint/db"
) )
// Holds state for a Validator at a given height+round. // Holds state for a Validator at a given height+round.
@ -54,20 +52,6 @@ func (v *Validator) WriteTo(w io.Writer) (n int64, err error) {
//----------------------------------------------------------------------------- //-----------------------------------------------------------------------------
// TODO: Ensure that double signing never happens via an external persistent check.
type PrivValidator struct {
PrivAccount
db *db_.LevelDB
}
// Modifies the vote object in memory.
// Double signing results in an error.
func (pv *PrivValidator) SignVote(vote *Vote) error {
return nil
}
//-----------------------------------------------------------------------------
// Not goroutine-safe. // Not goroutine-safe.
type ValidatorSet struct { type ValidatorSet struct {
validators map[uint64]*Validator validators map[uint64]*Validator
@ -124,6 +108,10 @@ func (v *ValidatorSet) Map() map[uint64]*Validator {
return v.validators return v.validators
} }
func (v *ValidatorSet) Size() int {
return len(v.validators)
}
// TODO: cache proposer. invalidate upon increment. // TODO: cache proposer. invalidate upon increment.
func (v *ValidatorSet) GetProposer() (proposer *Validator) { func (v *ValidatorSet) GetProposer() (proposer *Validator) {
highestAccum := int64(0) highestAccum := int64(0)