Browse Source

RebondTx and more efficient IncrementAccum()

pull/9/head
Jae Kwon 10 years ago
parent
commit
6cacf6f09b
16 changed files with 286 additions and 141 deletions
  1. +44
    -0
      account/account.go
  2. +4
    -1
      account/pubkey.go
  3. +33
    -3
      blocks/block.go
  4. +22
    -4
      blocks/tx.go
  5. +1
    -1
      cmd/daemon.go
  6. +5
    -2
      common/heap.go
  7. +18
    -1
      consensus/priv_validator.go
  8. +2
    -2
      consensus/reactor.go
  9. +23
    -15
      consensus/state.go
  10. +1
    -1
      consensus/vote_set.go
  11. +4
    -4
      mempool/reactor.go
  12. +0
    -47
      state/account.go
  13. +1
    -0
      state/genesis.go
  14. +52
    -10
      state/state.go
  15. +28
    -17
      state/state_test.go
  16. +48
    -33
      state/validator_set.go

+ 44
- 0
account/account.go View File

@ -2,7 +2,10 @@ package account
import (
"bytes"
"fmt"
"io"
. "github.com/tendermint/tendermint/binary"
)
type Signable interface {
@ -17,3 +20,44 @@ func SignBytes(o Signable) []byte {
}
return buf.Bytes()
}
//-----------------------------------------------------------------------------
type Account struct {
Address []byte
PubKey PubKey
Sequence uint
Balance uint64
}
func NewAccount(pubKey PubKey) *Account {
address := pubKey.Address()
return &Account{
Address: address,
PubKey: pubKey,
Sequence: uint(0),
Balance: uint64(0),
}
}
func (account *Account) Copy() *Account {
accountCopy := *account
return &accountCopy
}
func (account *Account) String() string {
return fmt.Sprintf("Account{%X:%v}", account.Address, account.PubKey)
}
func AccountEncoder(o interface{}, w io.Writer, n *int64, err *error) {
WriteBinary(o.(*Account), w, n, err)
}
func AccountDecoder(r io.Reader, n *int64, err *error) interface{} {
return ReadBinary(&Account{}, r, n, err)
}
var AccountCodec = Codec{
Encode: AccountEncoder,
Decode: AccountDecoder,
}

+ 4
- 1
account/pubkey.go View File

@ -44,12 +44,15 @@ var _ = RegisterType(&TypeInfo{
//-------------------------------------
type PubKeyUnknown struct {
address []byte
}
func NewPubKeyUnknown(address []byte) PubKeyUnknown { return PubKeyUnknown{address} }
func (key PubKeyUnknown) TypeByte() byte { return PubKeyTypeUnknown }
func (key PubKeyUnknown) Address() []byte {
panic("PubKeyUnknown has no address")
return key.address
}
func (key PubKeyUnknown) VerifyBytes(msg []byte, sig_ Signature) bool {


+ 33
- 3
blocks/block.go View File

@ -42,6 +42,9 @@ func (b *Block) ValidateBasic(lastBlockHeight uint, lastBlockHash []byte,
if !b.Time.After(lastBlockTime) {
return errors.New("Invalid block time")
}
if err := b.Validation.ValidateBasic(); err != nil {
return err
}
// XXX more validation
return nil
}
@ -146,7 +149,7 @@ func (h *Header) StringWithIndent(indent string) string {
//-----------------------------------------------------------------------------
type Commit struct {
// It's not strictly needed here, but consider adding address here for convenience
Address []byte
Round uint
Signature SignatureEd25519
}
@ -156,19 +159,46 @@ func (commit Commit) IsZero() bool {
}
func (commit Commit) String() string {
return fmt.Sprintf("Commit{R:%v %X}", commit.Round, Fingerprint(commit.Signature.Bytes))
return fmt.Sprintf("Commit{A:%X R:%v %X}", commit.Address, commit.Round, Fingerprint(commit.Signature.Bytes))
}
//-------------------------------------
// NOTE: The Commits are in order of address to preserve the active ValidatorSet order.
// Any peer with a block can gossip commits by index with a peer catching up without recalculating the
// active ValidatorSet.
type Validation struct {
Commits []Commit
Commits []Commit // Commits (or nil) of all active validators in address order.
// Volatile
hash []byte
bitArray BitArray
}
func (v *Validation) ValidateBasic() error {
if len(v.Commits) == 0 {
return errors.New("No commits in validation")
}
lastAddress := []byte{}
for i := 0; i < len(v.Commits); i++ {
commit := v.Commits[i]
if commit.IsZero() {
if len(commit.Address) > 0 {
return errors.New("Zero commits should not have an address")
}
} else {
if len(commit.Address) == 0 {
return errors.New("Nonzero commits should have an address")
}
if len(lastAddress) > 0 && bytes.Compare(lastAddress, commit.Address) != -1 {
return errors.New("Invalid commit order")
}
lastAddress = commit.Address
}
}
return nil
}
func (v *Validation) Hash() []byte {
if v.hash == nil {
bs := make([]interface{}, len(v.Commits))


+ 22
- 4
blocks/tx.go View File

@ -32,7 +32,8 @@ const (
// Validation transactions
TxTypeBond = byte(0x11)
TxTypeUnbond = byte(0x12)
TxTypeDupeout = byte(0x13)
TxTypeRebond = byte(0x13)
TxTypeDupeout = byte(0x14)
)
var (
@ -52,6 +53,8 @@ func TxDecoder(r io.Reader, n *int64, err *error) interface{} {
return ReadBinary(&BondTx{}, r, n, err)
case TxTypeUnbond:
return ReadBinary(&UnbondTx{}, r, n, err)
case TxTypeRebond:
return ReadBinary(&RebondTx{}, r, n, err)
case TxTypeDupeout:
return ReadBinary(&DupeoutTx{}, r, n, err)
default:
@ -68,9 +71,9 @@ var _ = RegisterType(&TypeInfo{
//-----------------------------------------------------------------------------
type TxInput struct {
Address []byte // Hash of the PubKey
Amount uint64 // Must not exceed account balance
Sequence uint // Must be 1 greater than the last committed TxInput
Address []byte // Hash of the PubKey
Amount uint64 // Must not exceed account balance
Sequence uint // Must be 1 greater than the last committed TxInput
Signature Signature // Depends on the PubKey type and the whole Tx
}
@ -171,6 +174,21 @@ func (tx *UnbondTx) WriteSignBytes(w io.Writer, n *int64, err *error) {
//-----------------------------------------------------------------------------
type RebondTx struct {
Address []byte
Height uint
Signature SignatureEd25519
}
func (tx *RebondTx) TypeByte() byte { return TxTypeRebond }
func (tx *RebondTx) WriteSignBytes(w io.Writer, n *int64, err *error) {
WriteByteSlice(tx.Address, w, n, err)
WriteUVarInt(tx.Height, w, n, err)
}
//-----------------------------------------------------------------------------
type DupeoutTx struct {
Address []byte
VoteA Vote


+ 1
- 1
cmd/daemon.go View File

@ -52,7 +52,7 @@ func NewNode() *Node {
mempoolReactor := mempool_.NewMempoolReactor(mempool)
// Get ConsensusReactor
consensusReactor := consensus.NewConsensusReactor(blockStore, mempool, state)
consensusReactor := consensus.NewConsensusReactor(blockStore, mempoolReactor, state)
if privValidator != nil {
consensusReactor.SetPrivValidator(privValidator)
}


+ 5
- 2
common/heap.go View File

@ -46,6 +46,10 @@ func (h *Heap) Peek() interface{} {
return h.pq[0].value
}
func (h *Heap) Update(value interface{}, priority Comparable) {
h.pq.Update(h.pq[0], value, priority)
}
func (h *Heap) Pop() interface{} {
item := heap.Pop(&h.pq).(*pqItem)
return item.value
@ -93,8 +97,7 @@ func (pq *priorityQueue) Pop() interface{} {
}
func (pq *priorityQueue) Update(item *pqItem, value interface{}, priority Comparable) {
heap.Remove(pq, item.index)
item.value = value
item.priority = priority
heap.Push(pq, item)
heap.Fix(pq, item.index)
}

+ 18
- 1
consensus/priv_validator.go View File

@ -8,6 +8,7 @@ import (
"encoding/json"
"fmt"
"io/ioutil"
"math"
. "github.com/tendermint/tendermint/account"
. "github.com/tendermint/tendermint/binary"
@ -161,6 +162,22 @@ func (privVal *PrivValidator) SignProposal(proposal *Proposal) SignatureEd25519
// Sign
return privVal.PrivKey.Sign(SignBytes(proposal)).(SignatureEd25519)
} else {
panic(fmt.Sprintf("Attempt of duplicate signing of proposal: Height %v, Round %v, Type %v", proposal.Height, proposal.Round))
panic(fmt.Sprintf("Attempt of duplicate signing of proposal: Height %v, Round %v", proposal.Height, proposal.Round))
}
}
func (privVal *PrivValidator) SignRebondTx(rebondTx *RebondTx) SignatureEd25519 {
if privVal.LastHeight < rebondTx.Height {
// Persist height/round/step
privVal.LastHeight = rebondTx.Height
privVal.LastRound = math.MaxUint64 // We can't do anything else for this rebondTx.Height.
privVal.LastStep = math.MaxUint8
privVal.Save()
// Sign
return privVal.PrivKey.Sign(SignBytes(rebondTx)).(SignatureEd25519)
} else {
panic(fmt.Sprintf("Attempt of duplicate signing of rebondTx: Height %v", rebondTx.Height))
}
}

+ 2
- 2
consensus/reactor.go View File

@ -37,8 +37,8 @@ type ConsensusReactor struct {
conS *ConsensusState
}
func NewConsensusReactor(blockStore *BlockStore, mempool *mempool.Mempool, state *state.State) *ConsensusReactor {
conS := NewConsensusState(state, blockStore, mempool)
func NewConsensusReactor(blockStore *BlockStore, mempoolReactor *mempool.MempoolReactor, state *state.State) *ConsensusReactor {
conS := NewConsensusState(state, blockStore, mempoolReactor)
conR := &ConsensusReactor{
quit: make(chan struct{}),
conS: conS,


+ 23
- 15
consensus/state.go View File

@ -178,10 +178,10 @@ type ConsensusState struct {
stopped uint32
quit chan struct{}
blockStore *BlockStore
mempool *mempool.Mempool
runActionCh chan RoundAction
newStepCh chan *RoundState
blockStore *BlockStore
mempoolReactor *mempool.MempoolReactor
runActionCh chan RoundAction
newStepCh chan *RoundState
mtx sync.Mutex
RoundState
@ -190,13 +190,13 @@ type ConsensusState struct {
stagedState *state.State // Cache result of staged block.
}
func NewConsensusState(state *state.State, blockStore *BlockStore, mempool *mempool.Mempool) *ConsensusState {
func NewConsensusState(state *state.State, blockStore *BlockStore, mempoolReactor *mempool.MempoolReactor) *ConsensusState {
cs := &ConsensusState{
quit: make(chan struct{}),
blockStore: blockStore,
mempool: mempool,
runActionCh: make(chan RoundAction, 1),
newStepCh: make(chan *RoundState, 1),
quit: make(chan struct{}),
blockStore: blockStore,
mempoolReactor: mempoolReactor,
runActionCh: make(chan RoundAction, 1),
newStepCh: make(chan *RoundState, 1),
}
cs.updateToState(state)
return cs
@ -432,6 +432,16 @@ func (cs *ConsensusState) updateToState(state *state.State) {
if round > 0 {
cs.setupNewRound(round)
}
// If we've timed out, then send rebond tx.
if cs.PrivValidator != nil && cs.state.UnbondingValidators.HasAddress(cs.PrivValidator.Address) {
rebondTx := &RebondTx{
Address: cs.PrivValidator.Address,
Height: cs.Height + 1,
}
rebondTx.Signature = cs.PrivValidator.SignRebondTx(rebondTx)
cs.mempoolReactor.BroadcastTx(rebondTx)
}
}
// After the call cs.Step becomes RoundStepNewRound.
@ -443,9 +453,7 @@ func (cs *ConsensusState) setupNewRound(round uint) {
// Increment all the way to round.
validators := cs.Validators.Copy()
for r := cs.Round; r < round; r++ {
validators.IncrementAccum()
}
validators.IncrementAccum(round - cs.Round)
cs.Round = round
cs.Step = RoundStepNewRound
@ -529,7 +537,7 @@ func (cs *ConsensusState) RunActionPropose(height uint, round uint) {
validation = cs.LastCommits.MakeValidation()
}
}
txs := cs.mempool.GetProposalTxs()
txs := cs.mempoolReactor.Mempool.GetProposalTxs()
block = &Block{
Header: &Header{
Network: Config.Network,
@ -975,7 +983,7 @@ func (cs *ConsensusState) saveCommitVoteBlock(block *Block, blockParts *PartSet)
cs.stagedState.Save()
// Update mempool.
cs.mempool.ResetForBlockAndState(block, cs.stagedState)
cs.mempoolReactor.Mempool.ResetForBlockAndState(block, cs.stagedState)
cs.signAddVote(VoteTypeCommit, block.Hash(), blockParts.Header())
}


+ 1
- 1
consensus/vote_set.go View File

@ -232,7 +232,7 @@ func (voteSet *VoteSet) MakeValidation() *Validation {
if !vote.BlockParts.Equals(voteSet.maj23Parts) {
return false
}
commits[valIndex] = Commit{vote.Round, vote.Signature}
commits[valIndex] = Commit{val.Address, vote.Round, vote.Signature}
return false
})
return &Validation{


+ 4
- 4
mempool/reactor.go View File

@ -21,13 +21,13 @@ type MempoolReactor struct {
started uint32
stopped uint32
mempool *Mempool
Mempool *Mempool
}
func NewMempoolReactor(mempool *Mempool) *MempoolReactor {
memR := &MempoolReactor{
quit: make(chan struct{}),
mempool: mempool,
Mempool: mempool,
}
return memR
}
@ -74,7 +74,7 @@ func (memR *MempoolReactor) Receive(chId byte, src *p2p.Peer, msgBytes []byte) {
switch msg_.(type) {
case *TxMessage:
msg := msg_.(*TxMessage)
err := memR.mempool.AddTx(msg.Tx)
err := memR.Mempool.AddTx(msg.Tx)
if err != nil {
// Bad, seen, or conflicting tx.
log.Debug("Could not add tx %v", msg.Tx)
@ -98,7 +98,7 @@ func (memR *MempoolReactor) Receive(chId byte, src *p2p.Peer, msgBytes []byte) {
}
func (memR *MempoolReactor) BroadcastTx(tx Tx) error {
err := memR.mempool.AddTx(tx)
err := memR.Mempool.AddTx(tx)
if err != nil {
return err
}


+ 0
- 47
state/account.go View File

@ -1,47 +0,0 @@
package state
import (
"fmt"
"io"
. "github.com/tendermint/tendermint/account"
. "github.com/tendermint/tendermint/binary"
)
type Account struct {
Address []byte
PubKey PubKey
Sequence uint
Balance uint64
}
func NewAccount(address []byte, pubKey PubKey) *Account {
return &Account{
Address: address,
PubKey: pubKey,
Sequence: uint(0),
Balance: uint64(0),
}
}
func (account *Account) Copy() *Account {
accountCopy := *account
return &accountCopy
}
func (account *Account) String() string {
return fmt.Sprintf("Account{%X:%v}", account.Address, account.PubKey)
}
func AccountEncoder(o interface{}, w io.Writer, n *int64, err *error) {
WriteBinary(o.(*Account), w, n, err)
}
func AccountDecoder(r io.Reader, n *int64, err *error) interface{} {
return ReadBinary(&Account{}, r, n, err)
}
var AccountCodec = Codec{
Encode: AccountEncoder,
Decode: AccountDecoder,
}

+ 1
- 0
state/genesis.go View File

@ -5,6 +5,7 @@ import (
"io/ioutil"
"time"
. "github.com/tendermint/tendermint/account"
. "github.com/tendermint/tendermint/binary"
. "github.com/tendermint/tendermint/blocks"
. "github.com/tendermint/tendermint/common"


+ 52
- 10
state/state.go View File

@ -128,7 +128,7 @@ func (s *State) GetOrMakeAccounts(ins []*TxInput, outs []*TxOutput) (map[string]
account := s.GetAccount(out.Address)
// output account may be nil (new)
if account == nil {
account = NewAccount(out.Address, PubKeyUnknown{})
account = NewAccount(NewPubKeyUnknown(out.Address))
}
accounts[string(out.Address)] = account
}
@ -236,6 +236,12 @@ func (s *State) ExecTx(tx_ Tx) error {
case *BondTx:
tx := tx_.(*BondTx)
valInfo := s.GetValidatorInfo(tx.PubKey.Address())
if valInfo != nil {
// TODO: In the future, check that the validator wasn't destroyed,
// add funds, merge UnbondTo outputs, and unbond validator.
return errors.New("Adding coins to existing validators not yet supported")
}
accounts, err := s.GetOrMakeAccounts(tx.Inputs, tx.UnbondTo)
if err != nil {
return err
@ -262,15 +268,12 @@ func (s *State) ExecTx(tx_ Tx) error {
s.AdjustByInputs(accounts, tx.Inputs)
s.SetAccounts(accounts)
// Add ValidatorInfo
updated := s.SetValidatorInfo(&ValidatorInfo{
s.SetValidatorInfo(&ValidatorInfo{
Address: tx.PubKey.Address(),
PubKey: tx.PubKey,
UnbondTo: tx.UnbondTo,
FirstBondHeight: s.LastBlockHeight + 1,
})
if !updated {
panic("Failed to add validator info")
}
// Add Validator
added := s.BondedValidators.Add(&Validator{
Address: tx.PubKey.Address(),
@ -300,14 +303,40 @@ func (s *State) ExecTx(tx_ Tx) error {
}
// tx.Height must be greater than val.LastCommitHeight
if tx.Height < val.LastCommitHeight {
return errors.New("Invalid bond height")
if tx.Height <= val.LastCommitHeight {
return errors.New("Invalid unbond height")
}
// Good!
s.unbondValidator(val)
return nil
case *RebondTx:
tx := tx_.(*RebondTx)
// The validator must be inactive
_, val := s.UnbondingValidators.GetByAddress(tx.Address)
if val == nil {
return ErrTxInvalidAddress
}
// Verify the signature
signBytes := SignBytes(tx)
if !val.PubKey.VerifyBytes(signBytes, tx.Signature) {
return ErrTxInvalidSignature
}
// tx.Height must be equal to the next height
if tx.Height != s.LastBlockHeight+1 {
return errors.New("Invalid rebond height")
}
// tx.Height must be
// Good!
s.rebondValidator(val)
return nil
case *DupeoutTx:
tx := tx_.(*DupeoutTx)
@ -356,13 +385,26 @@ func (s *State) unbondValidator(val *Validator) {
if !removed {
panic("Couldn't remove validator for unbonding")
}
val.UnbondHeight = s.LastBlockHeight
val.UnbondHeight = s.LastBlockHeight + 1
added := s.UnbondingValidators.Add(val)
if !added {
panic("Couldn't add validator for unbonding")
}
}
func (s *State) rebondValidator(val *Validator) {
// Move validator to BondingValidators
val, removed := s.UnbondingValidators.Remove(val.Address)
if !removed {
panic("Couldn't remove validator for rebonding")
}
val.BondHeight = s.LastBlockHeight + 1
added := s.BondedValidators.Add(val)
if !added {
panic("Couldn't add validator for rebonding")
}
}
func (s *State) releaseValidator(val *Validator) {
// Update validatorInfo
valInfo := s.GetValidatorInfo(val.Address)
@ -475,7 +517,7 @@ func (s *State) AppendBlock(block *Block, blockPartsHeader PartSetHeader, checkS
}
_, val := s.BondedValidators.GetByIndex(uint(i))
if val == nil {
return ErrTxInvalidSignature
Panicf("Failed to fetch validator at index %v", i)
}
val.LastCommitHeight = block.Height - 1
updated := s.BondedValidators.Update(val)
@ -511,7 +553,7 @@ func (s *State) AppendBlock(block *Block, blockPartsHeader PartSetHeader, checkS
}
// Increment validator AccumPowers
s.BondedValidators.IncrementAccum()
s.BondedValidators.IncrementAccum(1)
// Check or set block.StateHash
stateHash := s.Hash()


+ 28
- 17
state/state_test.go View File

@ -1,44 +1,55 @@
package state
import (
. "github.com/tendermint/tendermint/account"
. "github.com/tendermint/tendermint/binary"
. "github.com/tendermint/tendermint/blocks"
. "github.com/tendermint/tendermint/common"
. "github.com/tendermint/tendermint/config"
db_ "github.com/tendermint/tendermint/db"
"github.com/tendermint/tendermint/wallet"
"bytes"
"testing"
"time"
)
func randAccountDetail(id uint64, status byte) (*AccountDetail, *PrivAccount) {
privAccount := GenPrivAccount()
privAccount.Id = id
account := privAccount.Account
return &AccountDetail{
Account: account,
Sequence: RandUInt(),
Balance: RandUInt64() + 1000, // At least 1000.
Status: status,
}, privAccount
func randAccount() (*Account, *PrivAccount) {
privAccount := wallet.GenPrivAccount()
account := NewAccount(privAccount.PubKey)
account.Sequence = RandUInt()
account.Balance = RandUInt32() + 1000 // At least 1000.
return account, privAccount
}
func genValidator(account *Account) *Validator, *ValidatorInfo {
valInfo := &ValidatorInfo{
Address: account.Address,
PubKey: account.PubKey,
UnbondTo: []*TxOutput{&TxOutput{
Address:
Address []byte
PubKey PubKeyEd25519
UnbondTo []*TxOutput
FirstBondHeight uint
}
}
// The first numValidators accounts are validators.
func randGenesisState(numAccounts int, numValidators int) (*State, []*PrivAccount) {
db := db_.NewMemDB()
accountDetails := make([]*AccountDetail, numAccounts)
accounts := make([]*Account, numAccounts)
privAccounts := make([]*PrivAccount, numAccounts)
validators := make([]*Validator, numValidators)
for i := 0; i < numAccounts; i++ {
account, privAccount := randAccount()
accounts[i], privAccounts[i] = account, privAccount
if i < numValidators {
accountDetails[i], privAccounts[i] =
randAccountDetail(uint64(i), AccountStatusBonded)
} else {
accountDetails[i], privAccounts[i] =
randAccountDetail(uint64(i), AccountStatusNominal)
validators[i] = &
}
}
s0 := GenesisState(db, time.Now(), accountDetails)
s0 := GenesisState(db, time.Now(), accounts)
s0.Save()
return s0, privAccounts
}


+ 48
- 33
state/validator_set.go View File

@ -6,30 +6,10 @@ import (
"sort"
"strings"
. "github.com/tendermint/tendermint/common"
"github.com/tendermint/tendermint/merkle"
)
//-------------------------------------
// Implements sort for sorting validators by id.
type ValidatorSlice []*Validator
func (vs ValidatorSlice) Len() int {
return len(vs)
}
func (vs ValidatorSlice) Less(i, j int) bool {
return bytes.Compare(vs[i].Address, vs[j].Address) == -1
}
func (vs ValidatorSlice) Swap(i, j int) {
it := vs[i]
vs[i] = vs[j]
vs[j] = it
}
//-------------------------------------
// ValidatorSet represent a set of *Validator at a given height.
// The validators can be fetched by address or index.
// The index is in order of .Address, so the index are the same
@ -54,25 +34,31 @@ func NewValidatorSet(vals []*Validator) *ValidatorSet {
for i, val := range vals {
validators[i] = val.Copy()
}
sort.Sort(ValidatorSlice(validators))
sort.Sort(ValidatorsByAddress(validators))
return &ValidatorSet{
validators: validators,
}
}
func (valSet *ValidatorSet) IncrementAccum() {
// Decrement from previous proposer
oldProposer := valSet.Proposer()
oldProposer.Accum -= int64(valSet.TotalVotingPower())
valSet.Update(oldProposer)
var newProposer *Validator
// Increment accum and find new proposer
// NOTE: updates validators in place.
// TODO: mind the overflow when times and votingPower shares too large.
func (valSet *ValidatorSet) IncrementAccum(times uint) {
// Add VotingPower * times to each validator and order into heap.
validatorsHeap := NewHeap()
for _, val := range valSet.validators {
val.Accum += int64(val.VotingPower)
newProposer = newProposer.CompareAccum(val)
val.Accum += int64(val.VotingPower) * int64(times) // TODO: mind overflow
validatorsHeap.Push(val, accumComparable(val.Accum))
}
valSet.proposer = newProposer
// Decrement the validator with most accum, times times.
for i := uint(0); i < times; i++ {
mostest := validatorsHeap.Peek().(*Validator)
mostest.Accum -= int64(valSet.TotalVotingPower())
validatorsHeap.Update(mostest, accumComparable(mostest.Accum))
}
// The next proposer is the next most accums remaining
valSet.proposer = validatorsHeap.Peek().(*Validator)
}
func (valSet *ValidatorSet) Copy() *ValidatorSet {
@ -231,3 +217,32 @@ func (valSet *ValidatorSet) StringWithIndent(indent string) string {
indent)
}
//-------------------------------------
// Implements sort for sorting validators by address.
type ValidatorsByAddress []*Validator
func (vs ValidatorsByAddress) Len() int {
return len(vs)
}
func (vs ValidatorsByAddress) Less(i, j int) bool {
return bytes.Compare(vs[i].Address, vs[j].Address) == -1
}
func (vs ValidatorsByAddress) Swap(i, j int) {
it := vs[i]
vs[i] = vs[j]
vs[j] = it
}
//-------------------------------------
// Use with Heap for sorting validators by accum
type accumComparable uint64
// We want to find the validator with the greatest accum.
func (ac accumComparable) Less(o interface{}) bool {
return uint64(ac) > o.(uint64)
}

Loading…
Cancel
Save