Browse Source

everything but binary, common, and blocks are explicitly imported.

pull/9/head
Jae Kwon 10 years ago
parent
commit
ac147e2353
13 changed files with 84 additions and 75 deletions
  1. +3
    -3
      consensus/consensus.go
  2. +9
    -0
      consensus/part_set_test.go
  3. +2
    -2
      consensus/pol.go
  4. +2
    -2
      consensus/priv_validator.go
  5. +10
    -10
      consensus/state.go
  6. +3
    -3
      consensus/vote_set.go
  7. +8
    -8
      consensus/vote_set_test.go
  8. +5
    -5
      mempool/mempool.go
  9. +6
    -6
      mempool/reactor.go
  10. +4
    -4
      merkle/iavl_tree.go
  11. +25
    -25
      sim/bench_votes.go
  12. +4
    -4
      state/state.go
  13. +3
    -3
      state/state_test.go

+ 3
- 3
consensus/consensus.go View File

@ -13,9 +13,9 @@ import (
. "github.com/tendermint/tendermint/binary"
. "github.com/tendermint/tendermint/blocks"
. "github.com/tendermint/tendermint/common"
. "github.com/tendermint/tendermint/mempool"
"github.com/tendermint/tendermint/mempool"
"github.com/tendermint/tendermint/p2p"
. "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/state"
)
const (
@ -102,7 +102,7 @@ type ConsensusReactor struct {
doActionCh chan RoundAction
}
func NewConsensusReactor(sw *p2p.Switch, blockStore *BlockStore, mempool *Mempool, state *State) *ConsensusReactor {
func NewConsensusReactor(sw *p2p.Switch, blockStore *BlockStore, mempool *mempool.Mempool, state *state.State) *ConsensusReactor {
conS := NewConsensusState(state, blockStore, mempool)
conR := &ConsensusReactor{
sw: sw,


+ 9
- 0
consensus/part_set_test.go View File

@ -0,0 +1,9 @@
package consensus
import (
"testing"
)
func TestBasicPartSet(t *testing.T) {
// XXX this is fun!
}

+ 2
- 2
consensus/pol.go View File

@ -6,7 +6,7 @@ import (
. "github.com/tendermint/tendermint/binary"
. "github.com/tendermint/tendermint/blocks"
. "github.com/tendermint/tendermint/common"
. "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/state"
)
// Proof of lock.
@ -42,7 +42,7 @@ func (pol *POL) WriteTo(w io.Writer) (n int64, err error) {
}
// Returns whether +2/3 have voted/committed for BlockHash.
func (pol *POL) Verify(vset *ValidatorSet) error {
func (pol *POL) Verify(vset *state.ValidatorSet) error {
talliedVotingPower := uint64(0)
voteDoc := BinaryBytes(&Vote{Height: pol.Height, Round: pol.Round,


+ 2
- 2
consensus/priv_validator.go View File

@ -3,13 +3,13 @@ package consensus
import (
. "github.com/tendermint/tendermint/blocks"
db_ "github.com/tendermint/tendermint/db"
. "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/state"
)
//-----------------------------------------------------------------------------
type PrivValidator struct {
PrivAccount
state.PrivAccount
db *db_.LevelDB
}


+ 10
- 10
consensus/state.go View File

@ -8,8 +8,8 @@ import (
. "github.com/tendermint/tendermint/binary"
. "github.com/tendermint/tendermint/blocks"
. "github.com/tendermint/tendermint/common"
. "github.com/tendermint/tendermint/mempool"
. "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/mempool"
"github.com/tendermint/tendermint/state"
)
const (
@ -32,8 +32,8 @@ type RoundState struct {
Round uint16
Step uint8
StartTime time.Time
Validators *ValidatorSet
Proposer *Validator
Validators *state.ValidatorSet
Proposer *state.Validator
Proposal *Proposal
ProposalBlock *Block
ProposalBlockPartSet *PartSet
@ -52,16 +52,16 @@ type RoundState struct {
// Tracks consensus state across block heights and rounds.
type ConsensusState struct {
blockStore *BlockStore
mempool *Mempool
mempool *mempool.Mempool
mtx sync.Mutex
RoundState
state *State // State until height-1.
stagedBlock *Block // Cache last staged block.
stagedState *State // Cache result of staged block.
state *state.State // State until height-1.
stagedBlock *Block // Cache last staged block.
stagedState *state.State // Cache result of staged block.
}
func NewConsensusState(state *State, blockStore *BlockStore, mempool *Mempool) *ConsensusState {
func NewConsensusState(state *state.State, blockStore *BlockStore, mempool *mempool.Mempool) *ConsensusState {
cs := &ConsensusState{
blockStore: blockStore,
mempool: mempool,
@ -77,7 +77,7 @@ func (cs *ConsensusState) GetRoundState() *RoundState {
return &rs
}
func (cs *ConsensusState) updateToState(state *State) {
func (cs *ConsensusState) updateToState(state *state.State) {
// Sanity check state.
stateHeight := state.Height
if stateHeight > 0 && stateHeight != cs.Height+1 {


+ 3
- 3
consensus/vote_set.go View File

@ -9,7 +9,7 @@ import (
. "github.com/tendermint/tendermint/blocks"
. "github.com/tendermint/tendermint/common"
. "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/state"
)
// VoteSet helps collect signatures from validators at each height+round
@ -23,7 +23,7 @@ type VoteSet struct {
type_ byte
mtx sync.Mutex
vset *ValidatorSet
vset *state.ValidatorSet
votes map[uint64]*Vote
votesBitArray BitArray
votesByBlockHash map[string]uint64
@ -33,7 +33,7 @@ type VoteSet struct {
}
// Constructs a new VoteSet struct used to accumulate votes for each round.
func NewVoteSet(height uint32, round uint16, type_ byte, vset *ValidatorSet) *VoteSet {
func NewVoteSet(height uint32, round uint16, type_ byte, vset *state.ValidatorSet) *VoteSet {
if type_ == VoteTypeCommit && round != 0 {
panic("Expected round 0 for commit vote set")
}


+ 8
- 8
consensus/vote_set_test.go View File

@ -3,29 +3,29 @@ package consensus
import (
. "github.com/tendermint/tendermint/blocks"
. "github.com/tendermint/tendermint/common"
. "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/state"
"testing"
)
func makeValidator(id uint64, votingPower uint64) (*Validator, *PrivAccount) {
privAccount := GenPrivAccount()
func makeValidator(id uint64, votingPower uint64) (*state.Validator, *state.PrivAccount) {
privAccount := state.GenPrivAccount()
privAccount.Id = id
return &Validator{
return &state.Validator{
Account: privAccount.Account,
VotingPower: votingPower,
}, privAccount
}
func makeVoteSet(height uint32, round uint16, numValidators int, votingPower uint64) (*VoteSet, *ValidatorSet, []*PrivAccount) {
vals := make([]*Validator, numValidators)
privAccounts := make([]*PrivAccount, numValidators)
func makeVoteSet(height uint32, round uint16, numValidators int, votingPower uint64) (*VoteSet, *state.ValidatorSet, []*state.PrivAccount) {
vals := make([]*state.Validator, numValidators)
privAccounts := make([]*state.PrivAccount, numValidators)
for i := 0; i < numValidators; i++ {
val, privAccount := makeValidator(uint64(i), votingPower)
vals[i] = val
privAccounts[i] = privAccount
}
valSet := NewValidatorSet(vals)
valSet := state.NewValidatorSet(vals)
return NewVoteSet(height, round, VoteTypeBare, valSet), valSet, privAccounts
}


+ 5
- 5
mempool/mempool.go View File

@ -13,17 +13,17 @@ import (
. "github.com/tendermint/tendermint/binary"
. "github.com/tendermint/tendermint/blocks"
. "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/state"
)
type Mempool struct {
mtx sync.Mutex
lastBlock *Block
state *State
state *state.State
txs []Tx
}
func NewMempool(lastBlock *Block, state *State) *Mempool {
func NewMempool(lastBlock *Block, state *state.State) *Mempool {
return &Mempool{
lastBlock: lastBlock,
state: state,
@ -45,7 +45,7 @@ func (mem *Mempool) AddTx(tx Tx) (err error) {
// Returns a new block from the current state and associated transactions.
// The block's Validation is empty, and some parts of the header too.
func (mem *Mempool) MakeProposalBlock() (*Block, *State) {
func (mem *Mempool) MakeProposalBlock() (*Block, *state.State) {
mem.mtx.Lock()
defer mem.mtx.Unlock()
nextBlock := mem.lastBlock.MakeNextBlock()
@ -57,7 +57,7 @@ func (mem *Mempool) MakeProposalBlock() (*Block, *State) {
// "state" is the result of state.AppendBlock("block").
// Txs that are present in "block" are discarded from mempool.
// Txs that have become invalid in the new "state" are also discarded.
func (mem *Mempool) ResetForBlockAndState(block *Block, state *State) {
func (mem *Mempool) ResetForBlockAndState(block *Block, state *state.State) {
mem.mtx.Lock()
defer mem.mtx.Unlock()
mem.lastBlock = block


+ 6
- 6
mempool/reactor.go View File

@ -8,7 +8,7 @@ import (
. "github.com/tendermint/tendermint/binary"
. "github.com/tendermint/tendermint/blocks"
. "github.com/tendermint/tendermint/p2p"
"github.com/tendermint/tendermint/p2p"
)
var (
@ -17,7 +17,7 @@ var (
// MempoolReactor handles mempool tx broadcasting amongst peers.
type MempoolReactor struct {
sw *Switch
sw *p2p.Switch
quit chan struct{}
started uint32
stopped uint32
@ -25,7 +25,7 @@ type MempoolReactor struct {
mempool *Mempool
}
func NewMempoolReactor(sw *Switch, mempool *Mempool) *MempoolReactor {
func NewMempoolReactor(sw *p2p.Switch, mempool *Mempool) *MempoolReactor {
memR := &MempoolReactor{
sw: sw,
quit: make(chan struct{}),
@ -58,14 +58,14 @@ func (memR *MempoolReactor) BroadcastTx(tx Tx) error {
}
// Implements Reactor
func (pexR *MempoolReactor) AddPeer(peer *Peer) {
func (pexR *MempoolReactor) AddPeer(peer *p2p.Peer) {
}
// Implements Reactor
func (pexR *MempoolReactor) RemovePeer(peer *Peer, err error) {
func (pexR *MempoolReactor) RemovePeer(peer *p2p.Peer, err error) {
}
func (memR *MempoolReactor) Receive(chId byte, src *Peer, msgBytes []byte) {
func (memR *MempoolReactor) Receive(chId byte, src *p2p.Peer, msgBytes []byte) {
_, msg_ := decodeMessage(msgBytes)
log.Info("MempoolReactor received %v", msg_)


+ 4
- 4
merkle/iavl_tree.go View File

@ -6,7 +6,7 @@ import (
"sync"
. "github.com/tendermint/tendermint/binary"
. "github.com/tendermint/tendermint/db"
db_ "github.com/tendermint/tendermint/db"
)
/*
@ -20,7 +20,7 @@ type IAVLTree struct {
ndb *nodeDB
}
func NewIAVLTree(keyCodec, valueCodec Codec, cacheSize int, db DB) *IAVLTree {
func NewIAVLTree(keyCodec, valueCodec Codec, cacheSize int, db db_.DB) *IAVLTree {
if db == nil {
// In-memory IAVLTree
return &IAVLTree{
@ -178,10 +178,10 @@ type nodeDB struct {
cache map[string]nodeElement
cacheSize int
cacheQueue *list.List
db DB
db db_.DB
}
func newNodeDB(cacheSize int, db DB) *nodeDB {
func newNodeDB(cacheSize int, db db_.DB) *nodeDB {
return &nodeDB{
cache: make(map[string]nodeElement),
cacheSize: cacheSize,


+ 25
- 25
sim/bench_votes.go View File

@ -11,13 +11,13 @@ import (
const seed = 0
const numNodes = 50000 // Total number of nodes to simulate
const numNodes8 = (numNodes + 7) / 8
const minNumPeers = 8 // Each node should be connected to at least this many peers
const maxNumPeers = 12 // ... and at most this many
const latencyMS = uint16(500) // One way packet latency
const partTxMS = uint16(3) // Transmission time per peer of 100B of data.
const minNumPeers = 8 // Each node should be connected to at least this many peers
const maxNumPeers = 12 // ... and at most this many
const latencyMS = uint16(500) // One way packet latency
const partTxMS = uint16(3) // Transmission time per peer of 100B of data.
const sendQueueCapacity = 3200 // Amount of messages to queue between peers.
const maxAllowableRank = 2 // After this, the data is considered waste.
const tryUnsolicited = 0.02 // Chance of sending an unsolicited piece of data.
const maxAllowableRank = 2 // After this, the data is considered waste.
const tryUnsolicited = 0.02 // Chance of sending an unsolicited piece of data.
var log *bufio.Writer
@ -47,7 +47,7 @@ func logWrite(s string) {
type Peer struct {
node *Node // Pointer to node
sent uint16 // Time of last packet send, including transmit time.
remote uint8 // SomeNode.peers[x].node.peers[remote].node is SomeNode for all x.
remote uint8 // SomeNode.peers[x].node.peers[remote].node is SomeNode for all x.
wanted []byte // Bitarray of wanted pieces.
given []byte // Bitarray of given pieces.
}
@ -217,7 +217,7 @@ type Event interface {
type EventData struct {
time uint16 // time of receipt.
src uint8 // src node's peer index on destination node
src uint8 // src node's peer index on destination node
part uint16
}
@ -231,7 +231,7 @@ func (e EventData) String() string {
type EventDataResponse struct {
time uint16 // time of receipt.
src uint8 // src node's peer index on destination node.
src uint8 // src node's peer index on destination node.
part uint16 // in response to given part
rank uint8 // if this is 1, node was first to give peer part.
}
@ -289,7 +289,7 @@ func countFull(nodes []*Node) (fullCount int) {
}
type runStat struct {
time uint16 // time for all events to propagate
time uint16 // time for all events to propagate
fill float64 // avg % of pieces gotten
succ float64 // % of times the sendQueue was not full
dups float64 // % of times that a received data was duplicate
@ -417,21 +417,21 @@ func main() {
//fmt.Print("!")
// Peer doesn't want it, but sporadically we'll try sending it anyways.
/*
if rand.Float32() < tryUnsolicited {
sent := peer.sendEventData(EventData{
time: event.time + latencyMS + partTxMS,
src: peer.remote,
part: event.part,
})
if sent {
//logWrite(fmt.Sprintf("[%v] t:%v S:%v n:%v -> p:%v %v TS\n", len(runStats), event.time, srcPeer.node.index, node.index, peer.node.index, event.part))
peer.setGiven(event.part)
// numSendSuccess++
} else {
//logWrite(fmt.Sprintf("[%v] t:%v S:%v n:%v -> p:%v %v TF\n", len(runStats), event.time, srcPeer.node.index, node.index, peer.node.index, event.part))
// numSendFailure++
}
}*/
if rand.Float32() < tryUnsolicited {
sent := peer.sendEventData(EventData{
time: event.time + latencyMS + partTxMS,
src: peer.remote,
part: event.part,
})
if sent {
//logWrite(fmt.Sprintf("[%v] t:%v S:%v n:%v -> p:%v %v TS\n", len(runStats), event.time, srcPeer.node.index, node.index, peer.node.index, event.part))
peer.setGiven(event.part)
// numSendSuccess++
} else {
//logWrite(fmt.Sprintf("[%v] t:%v S:%v n:%v -> p:%v %v TF\n", len(runStats), event.time, srcPeer.node.index, node.index, peer.node.index, event.part))
// numSendFailure++
}
}*/
}
}


+ 4
- 4
state/state.go View File

@ -9,7 +9,7 @@ import (
. "github.com/tendermint/tendermint/binary"
. "github.com/tendermint/tendermint/blocks"
. "github.com/tendermint/tendermint/common"
. "github.com/tendermint/tendermint/db"
db_ "github.com/tendermint/tendermint/db"
"github.com/tendermint/tendermint/merkle"
)
@ -42,7 +42,7 @@ func (txErr InvalidTxError) Error() string {
// NOTE: not goroutine-safe.
type State struct {
DB DB
DB db_.DB
Height uint32 // Last known block height
BlockHash []byte // Last known block hash
CommitTime time.Time
@ -51,7 +51,7 @@ type State struct {
UnbondingValidators *ValidatorSet
}
func GenesisState(db DB, genesisTime time.Time, accDets []*AccountDetail) *State {
func GenesisState(db db_.DB, genesisTime time.Time, accDets []*AccountDetail) *State {
// TODO: Use "uint64Codec" instead of BasicCodec
accountDetails := merkle.NewIAVLTree(BasicCodec, AccountDetailCodec, defaultAccountDetailsCacheCapacity, db)
@ -84,7 +84,7 @@ func GenesisState(db DB, genesisTime time.Time, accDets []*AccountDetail) *State
}
}
func LoadState(db DB) *State {
func LoadState(db db_.DB) *State {
s := &State{DB: db}
buf := db.Get(stateKey)
if len(buf) == 0 {


+ 3
- 3
state/state_test.go View File

@ -4,7 +4,7 @@ import (
. "github.com/tendermint/tendermint/blocks"
. "github.com/tendermint/tendermint/common"
. "github.com/tendermint/tendermint/config"
. "github.com/tendermint/tendermint/db"
db_ "github.com/tendermint/tendermint/db"
"bytes"
"testing"
@ -25,7 +25,7 @@ func randAccountDetail(id uint64, status byte) (*AccountDetail, *PrivAccount) {
// The first numValidators accounts are validators.
func randGenesisState(numAccounts int, numValidators int) (*State, []*PrivAccount) {
db := NewMemDB()
db := db_.NewMemDB()
accountDetails := make([]*AccountDetail, numAccounts)
privAccounts := make([]*PrivAccount, numAccounts)
for i := 0; i < numAccounts; i++ {
@ -114,7 +114,7 @@ func TestGenesisSaveLoad(t *testing.T) {
s0.Save(commitTime)
// Sanity check s0
//s0.DB.(*MemDB).Print()
//s0.DB.(*db_.MemDB).Print()
if s0.BondedValidators.TotalVotingPower() == 0 {
t.Error("s0 BondedValidators TotalVotingPower should not be 0")
}


Loading…
Cancel
Save