Browse Source

draft of consensus+state code, compiles.

pull/9/head
Jae Kwon 10 years ago
parent
commit
f030c69495
13 changed files with 222 additions and 217 deletions
  1. +22
    -46
      binary/codec.go
  2. +0
    -63
      blocks/accounts.go
  3. +0
    -15
      blocks/block.go
  4. +0
    -7
      blocks/block_test.go
  5. +40
    -0
      blocks/signature.go
  6. +6
    -1
      consensus/block_part_set.go
  7. +35
    -34
      consensus/consensus.go
  8. +20
    -0
      consensus/priv_validator.go
  9. +23
    -26
      consensus/state.go
  10. +7
    -6
      consensus/vote.go
  11. +53
    -0
      state/account.go
  12. +12
    -3
      state/state.go
  13. +4
    -16
      state/validator.go

+ 22
- 46
binary/codec.go View File

@ -1,18 +1,15 @@
package binary
import (
"errors"
"io"
"bytes"
"time"
)
type Codec interface {
WriteTo(io.Writer, interface{}, *int64, *error)
ReadFrom(io.Reader, *int64, *error) interface{}
Write(interface{}) ([]byte, error)
Read([]byte) (interface{}, error)
}
//-----------------------------------------------------------------------------
const (
typeNil = byte(0x00)
typeByte = byte(0x01)
@ -33,7 +30,8 @@ var BasicCodec = basicCodec{}
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) {
case nil:
WriteByte(w, typeNil, n, err)
@ -76,63 +74,41 @@ func (bc basicCodec) WriteTo(w io.Writer, o interface{}, n *int64, err *error) {
default:
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)
switch type_ {
case typeNil:
return nil
o = nil
case typeByte:
return ReadByte(r, n, err)
o = ReadByte(r, n, err)
case typeInt8:
return ReadInt8(r, n, err)
o = ReadInt8(r, n, err)
//case typeUInt8:
// return ReadUInt8(r, n, err)
// o = ReadUInt8(r, n, err)
case typeInt16:
return ReadInt16(r, n, err)
o = ReadInt16(r, n, err)
case typeUInt16:
return ReadUInt16(r, n, err)
o = ReadUInt16(r, n, err)
case typeInt32:
return ReadInt32(r, n, err)
o = ReadInt32(r, n, err)
case typeUInt32:
return ReadUInt32(r, n, err)
o = ReadUInt32(r, n, err)
case typeInt64:
return ReadInt64(r, n, err)
o = ReadInt64(r, n, err)
case typeUInt64:
return ReadUInt64(r, n, err)
o = ReadUInt64(r, n, err)
case typeString:
return ReadString(r, n, err)
o = ReadString(r, n, err)
case typeByteSlice:
return ReadByteSlice(r, n, err)
o = ReadByteSlice(r, n, err)
case typeTime:
return ReadTime(r, n, err)
o = ReadTime(r, n, err)
default:
panic("Unsupported type")
}
}
//-----------------------------------------------------------------------------
// 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)
return o, *err
}

+ 0
- 63
blocks/accounts.go 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
}

+ 0
- 15
blocks/block.go 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 */


+ 0
- 7
blocks/block_test.go View File

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


+ 40
- 0
blocks/signature.go 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
}

blocks/block_part_set.go → consensus/block_part_set.go View File


+ 35
- 34
consensus/consensus.go View File

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


+ 20
- 0
consensus/priv_validator.go 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
}

+ 23
- 26
consensus/state.go View File

@ -1,12 +1,9 @@
package consensus
import (
"bytes"
"sync"
"time"
. "github.com/tendermint/tendermint/binary"
. "github.com/tendermint/tendermint/blocks"
. "github.com/tendermint/tendermint/common"
. "github.com/tendermint/tendermint/state"
)
@ -18,12 +15,12 @@ var (
// Tracks consensus state across block heights and rounds.
type ConsensusState struct {
mtx sync.Mutex
height uint32 // Height we are working on.
validatorsR0 map[uint64]*Validator // 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.
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.
}
func NewConsensusState(state *State) *ConsensusState {
@ -70,7 +67,7 @@ func (cs *ConsensusState) Update(state *State) {
cs.height = stateHeight
cs.validatorsR0 = state.Validators().Copy() // NOTE: immutable.
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)
// Setup the roundState
@ -94,20 +91,20 @@ func (cs *ConsensusState) SetupRound(round uint16) {
func (cs *ConsensusState) setupRound(round uint16) {
// Increment validator accums as necessary.
// We need to start with cs.validatorsR0 or cs.roundState.Validators
var validators map[uint64]*Validator = nil
var validators *ValidatorSet
var validatorsRound uint16
if cs.roundState == nil {
// We have no roundState so we start from validatorsR0 at round 0.
validators = copyValidators(cs.validatorsR0)
validators = cs.validatorsR0.Copy()
validatorsRound = 0
} else {
// We have a previous roundState so we start from that.
validators = copyValidators(cs.roundState.Validators)
validators = cs.roundState.Validators.Copy()
validatorsRound = cs.roundState.Round
}
// Increment all the way to round.
for r := validatorsRound; r < round; r++ {
incrementAccum(validators)
validators.IncrementAccum()
}
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.
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 map[uint64]*Validator // All validators with modified accumPower for this round.
BlockPartSet *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.
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.
BlockPartSet *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 map[uint64]*Validator, commits *VoteSet) *RoundState {
validators *ValidatorSet, commits *VoteSet) *RoundState {
proposer := getProposer(validators)
proposer := validators.GetProposer()
blockPartSet := NewBlockPartSet(height, round, &(proposer.Account))
roundBareVotes := NewVoteSet(height, round, VoteTypeBare, validators)
roundPrecommits := NewVoteSet(height, round, VoteTypePrecommit, validators)


state/vote.go → consensus/vote.go View File


+ 53
- 0
state/account.go 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
}
*/

+ 12
- 3
state/state.go View File

@ -119,9 +119,18 @@ func (s *State) Validators() *ValidatorSet {
return s.validators
}
func (s *State) Account(accountId uint64) *Account {
func (s *State) Account(accountId uint64) (*Account, error) {
s.mtx.Lock()
defer s.mtx.Unlock()
// XXX: figure out a way to load an Account Binary type.
return s.accounts.Get(accountId)
idBytes, err := BasicCodec.Write(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
}

+ 4
- 16
state/validator.go View File

@ -4,9 +4,7 @@ import (
"io"
. "github.com/tendermint/tendermint/binary"
. "github.com/tendermint/tendermint/blocks"
. "github.com/tendermint/tendermint/common"
db_ "github.com/tendermint/tendermint/db"
)
// 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.
type ValidatorSet struct {
validators map[uint64]*Validator
@ -124,6 +108,10 @@ func (v *ValidatorSet) Map() map[uint64]*Validator {
return v.validators
}
func (v *ValidatorSet) Size() int {
return len(v.validators)
}
// TODO: cache proposer. invalidate upon increment.
func (v *ValidatorSet) GetProposer() (proposer *Validator) {
highestAccum := int64(0)


Loading…
Cancel
Save