Browse Source

block module -> import as blk

pull/39/head
Jae Kwon 10 years ago
parent
commit
0a6c28c2da
22 changed files with 323 additions and 323 deletions
  1. +3
    -3
      cmd/daemon.go
  2. +6
    -6
      cmd/gen_tx.go
  3. +8
    -8
      consensus/pol.go
  4. +23
    -23
      consensus/pol_test.go
  5. +31
    -31
      consensus/reactor.go
  6. +71
    -71
      consensus/state.go
  7. +15
    -15
      consensus/state_test.go
  8. +2
    -2
      consensus/test.go
  9. +4
    -4
      consensus/types/proposal.go
  10. +26
    -26
      consensus/vote_set.go
  11. +30
    -30
      consensus/vote_set_test.go
  12. +8
    -8
      mempool/mempool.go
  13. +3
    -3
      mempool/reactor.go
  14. +3
    -3
      rpc/accounts.go
  15. +7
    -7
      rpc/blocks.go
  16. +2
    -2
      rpc/mempool.go
  17. +3
    -3
      rpc/rpc.go
  18. +4
    -4
      state/genesis.go
  19. +8
    -8
      state/priv_validator.go
  20. +62
    -62
      state/state.go
  21. +2
    -2
      state/test.go
  22. +2
    -2
      state/validator.go

+ 3
- 3
cmd/daemon.go View File

@ -4,7 +4,7 @@ import (
"os"
"os/signal"
"github.com/tendermint/tendermint/block"
blk "github.com/tendermint/tendermint/block"
. "github.com/tendermint/tendermint/common"
"github.com/tendermint/tendermint/config"
"github.com/tendermint/tendermint/consensus"
@ -20,7 +20,7 @@ type Node struct {
sw *p2p.Switch
book *p2p.AddrBook
pexReactor *p2p.PEXReactor
blockStore *block.BlockStore
blockStore *blk.BlockStore
mempoolReactor *mempl.MempoolReactor
consensusState *consensus.ConsensusState
consensusReactor *consensus.ConsensusReactor
@ -30,7 +30,7 @@ type Node struct {
func NewNode() *Node {
// Get BlockStore
blockStoreDB := dbm.GetDB("blockstore")
blockStore := block.NewBlockStore(blockStoreDB)
blockStore := blk.NewBlockStore(blockStoreDB)
// Get State
stateDB := dbm.GetDB("state")


+ 6
- 6
cmd/gen_tx.go View File

@ -10,7 +10,7 @@ import (
"github.com/tendermint/tendermint/account"
"github.com/tendermint/tendermint/binary"
"github.com/tendermint/tendermint/block"
blk "github.com/tendermint/tendermint/block"
. "github.com/tendermint/tendermint/common"
dbm "github.com/tendermint/tendermint/db"
sm "github.com/tendermint/tendermint/state"
@ -80,9 +80,9 @@ func gen_tx() {
dstSendAmount := getUint64(Fmt("Enter amount to send to %X: ", dstAddress))
// Construct SendTx
tx := &block.SendTx{
Inputs: []*block.TxInput{
&block.TxInput{
tx := &blk.SendTx{
Inputs: []*blk.TxInput{
&blk.TxInput{
Address: srcAddress,
Amount: srcSendAmount,
Sequence: srcSendSequence,
@ -90,8 +90,8 @@ func gen_tx() {
PubKey: srcPubKey,
},
},
Outputs: []*block.TxOutput{
&block.TxOutput{
Outputs: []*blk.TxOutput{
&blk.TxOutput{
Address: dstAddress,
Amount: dstSendAmount,
},


+ 8
- 8
consensus/pol.go View File

@ -4,7 +4,7 @@ import (
"fmt"
"github.com/tendermint/tendermint/account"
"github.com/tendermint/tendermint/block"
blk "github.com/tendermint/tendermint/block"
. "github.com/tendermint/tendermint/common"
sm "github.com/tendermint/tendermint/state"
)
@ -23,9 +23,9 @@ type POLVoteSignature struct {
type POL struct {
Height uint
Round uint
BlockHash []byte // Could be nil, which makes this a proof of unlock.
BlockParts block.PartSetHeader // When BlockHash is nil, this is zero.
Votes []POLVoteSignature // Prevote and commit signatures in ValidatorSet order.
BlockHash []byte // Could be nil, which makes this a proof of unlock.
BlockParts blk.PartSetHeader // When BlockHash is nil, this is zero.
Votes []POLVoteSignature // Prevote and commit signatures in ValidatorSet order.
}
// Returns whether +2/3 have prevoted/committed for BlockHash.
@ -37,8 +37,8 @@ func (pol *POL) Verify(valSet *sm.ValidatorSet) error {
}
talliedVotingPower := uint64(0)
prevoteDoc := account.SignBytes(&block.Vote{
Height: pol.Height, Round: pol.Round, Type: block.VoteTypePrevote,
prevoteDoc := account.SignBytes(&blk.Vote{
Height: pol.Height, Round: pol.Round, Type: blk.VoteTypePrevote,
BlockHash: pol.BlockHash,
BlockParts: pol.BlockParts,
})
@ -54,8 +54,8 @@ func (pol *POL) Verify(valSet *sm.ValidatorSet) error {
// Commit vote?
if vote.Round < pol.Round {
voteDoc = account.SignBytes(&block.Vote{
Height: pol.Height, Round: vote.Round, Type: block.VoteTypeCommit,
voteDoc = account.SignBytes(&blk.Vote{
Height: pol.Height, Round: vote.Round, Type: blk.VoteTypeCommit,
BlockHash: pol.BlockHash,
BlockParts: pol.BlockParts,
})


+ 23
- 23
consensus/pol_test.go View File

@ -2,7 +2,7 @@ package consensus
import (
"github.com/tendermint/tendermint/binary"
"github.com/tendermint/tendermint/block"
blk "github.com/tendermint/tendermint/block"
. "github.com/tendermint/tendermint/common"
sm "github.com/tendermint/tendermint/state"
@ -15,7 +15,7 @@ import (
// Convenience method.
// Signs the vote and sets the POL's vote at the desired index
// Returns the POLVoteSignature pointer, so you can modify it afterwards.
func signAddPOLVoteSignature(val *sm.PrivValidator, valSet *sm.ValidatorSet, vote *block.Vote, pol *POL) *POLVoteSignature {
func signAddPOLVoteSignature(val *sm.PrivValidator, valSet *sm.ValidatorSet, vote *blk.Vote, pol *POL) *POLVoteSignature {
vote = vote.Copy()
err := val.SignVote(vote)
if err != nil {
@ -28,7 +28,7 @@ func signAddPOLVoteSignature(val *sm.PrivValidator, valSet *sm.ValidatorSet, vot
func TestVerifyVotes(t *testing.T) {
height, round := uint(1), uint(0)
_, valSet, privValidators := randVoteSet(height, round, block.VoteTypePrevote, 10, 1)
_, valSet, privValidators := randVoteSet(height, round, blk.VoteTypePrevote, 10, 1)
// Make a POL with -2/3 votes.
blockHash := RandBytes(32)
@ -36,8 +36,8 @@ func TestVerifyVotes(t *testing.T) {
Height: height, Round: round, BlockHash: blockHash,
Votes: make([]POLVoteSignature, valSet.Size()),
}
voteProto := &block.Vote{
Height: height, Round: round, Type: block.VoteTypePrevote, BlockHash: blockHash,
voteProto := &blk.Vote{
Height: height, Round: round, Type: blk.VoteTypePrevote, BlockHash: blockHash,
}
for i := 0; i < 6; i++ {
signAddPOLVoteSignature(privValidators[i], valSet, voteProto, pol)
@ -59,7 +59,7 @@ func TestVerifyVotes(t *testing.T) {
func TestVerifyInvalidVote(t *testing.T) {
height, round := uint(1), uint(0)
_, valSet, privValidators := randVoteSet(height, round, block.VoteTypePrevote, 10, 1)
_, valSet, privValidators := randVoteSet(height, round, blk.VoteTypePrevote, 10, 1)
// Make a POL with +2/3 votes with the wrong signature.
blockHash := RandBytes(32)
@ -67,8 +67,8 @@ func TestVerifyInvalidVote(t *testing.T) {
Height: height, Round: round, BlockHash: blockHash,
Votes: make([]POLVoteSignature, valSet.Size()),
}
voteProto := &block.Vote{
Height: height, Round: round, Type: block.VoteTypePrevote, BlockHash: blockHash,
voteProto := &blk.Vote{
Height: height, Round: round, Type: blk.VoteTypePrevote, BlockHash: blockHash,
}
for i := 0; i < 7; i++ {
polVoteSig := signAddPOLVoteSignature(privValidators[i], valSet, voteProto, pol)
@ -83,7 +83,7 @@ func TestVerifyInvalidVote(t *testing.T) {
func TestVerifyCommits(t *testing.T) {
height, round := uint(1), uint(2)
_, valSet, privValidators := randVoteSet(height, round, block.VoteTypePrevote, 10, 1)
_, valSet, privValidators := randVoteSet(height, round, blk.VoteTypePrevote, 10, 1)
// Make a POL with +2/3 votes.
blockHash := RandBytes(32)
@ -91,8 +91,8 @@ func TestVerifyCommits(t *testing.T) {
Height: height, Round: round, BlockHash: blockHash,
Votes: make([]POLVoteSignature, valSet.Size()),
}
voteProto := &block.Vote{
Height: height, Round: round - 1, Type: block.VoteTypeCommit, BlockHash: blockHash,
voteProto := &blk.Vote{
Height: height, Round: round - 1, Type: blk.VoteTypeCommit, BlockHash: blockHash,
}
for i := 0; i < 7; i++ {
signAddPOLVoteSignature(privValidators[i], valSet, voteProto, pol)
@ -106,7 +106,7 @@ func TestVerifyCommits(t *testing.T) {
func TestVerifyInvalidCommits(t *testing.T) {
height, round := uint(1), uint(2)
_, valSet, privValidators := randVoteSet(height, round, block.VoteTypePrevote, 10, 1)
_, valSet, privValidators := randVoteSet(height, round, blk.VoteTypePrevote, 10, 1)
// Make a POL with +2/3 votes with the wrong signature.
blockHash := RandBytes(32)
@ -114,8 +114,8 @@ func TestVerifyInvalidCommits(t *testing.T) {
Height: height, Round: round, BlockHash: blockHash,
Votes: make([]POLVoteSignature, valSet.Size()),
}
voteProto := &block.Vote{
Height: height, Round: round - 1, Type: block.VoteTypeCommit, BlockHash: blockHash,
voteProto := &blk.Vote{
Height: height, Round: round - 1, Type: blk.VoteTypeCommit, BlockHash: blockHash,
}
for i := 0; i < 7; i++ {
polVoteSig := signAddPOLVoteSignature(privValidators[i], valSet, voteProto, pol)
@ -130,7 +130,7 @@ func TestVerifyInvalidCommits(t *testing.T) {
func TestVerifyInvalidCommitRounds(t *testing.T) {
height, round := uint(1), uint(2)
_, valSet, privValidators := randVoteSet(height, round, block.VoteTypePrevote, 10, 1)
_, valSet, privValidators := randVoteSet(height, round, blk.VoteTypePrevote, 10, 1)
// Make a POL with +2/3 commits for the current round.
blockHash := RandBytes(32)
@ -138,8 +138,8 @@ func TestVerifyInvalidCommitRounds(t *testing.T) {
Height: height, Round: round, BlockHash: blockHash,
Votes: make([]POLVoteSignature, valSet.Size()),
}
voteProto := &block.Vote{
Height: height, Round: round, Type: block.VoteTypeCommit, BlockHash: blockHash,
voteProto := &blk.Vote{
Height: height, Round: round, Type: blk.VoteTypeCommit, BlockHash: blockHash,
}
for i := 0; i < 7; i++ {
signAddPOLVoteSignature(privValidators[i], valSet, voteProto, pol)
@ -153,7 +153,7 @@ func TestVerifyInvalidCommitRounds(t *testing.T) {
func TestVerifyInvalidCommitRounds2(t *testing.T) {
height, round := uint(1), uint(2)
_, valSet, privValidators := randVoteSet(height, round, block.VoteTypePrevote, 10, 1)
_, valSet, privValidators := randVoteSet(height, round, blk.VoteTypePrevote, 10, 1)
// Make a POL with +2/3 commits for future round.
blockHash := RandBytes(32)
@ -161,8 +161,8 @@ func TestVerifyInvalidCommitRounds2(t *testing.T) {
Height: height, Round: round, BlockHash: blockHash,
Votes: make([]POLVoteSignature, valSet.Size()),
}
voteProto := &block.Vote{
Height: height, Round: round + 1, Type: block.VoteTypeCommit, BlockHash: blockHash,
voteProto := &blk.Vote{
Height: height, Round: round + 1, Type: blk.VoteTypeCommit, BlockHash: blockHash,
}
for i := 0; i < 7; i++ {
polVoteSig := signAddPOLVoteSignature(privValidators[i], valSet, voteProto, pol)
@ -177,7 +177,7 @@ func TestVerifyInvalidCommitRounds2(t *testing.T) {
func TestReadWrite(t *testing.T) {
height, round := uint(1), uint(2)
_, valSet, privValidators := randVoteSet(height, round, block.VoteTypePrevote, 10, 1)
_, valSet, privValidators := randVoteSet(height, round, blk.VoteTypePrevote, 10, 1)
// Make a POL with +2/3 votes.
blockHash := RandBytes(32)
@ -185,8 +185,8 @@ func TestReadWrite(t *testing.T) {
Height: height, Round: round, BlockHash: blockHash,
Votes: make([]POLVoteSignature, valSet.Size()),
}
voteProto := &block.Vote{
Height: height, Round: round, Type: block.VoteTypePrevote, BlockHash: blockHash,
voteProto := &blk.Vote{
Height: height, Round: round, Type: blk.VoteTypePrevote, BlockHash: blockHash,
}
for i := 0; i < 7; i++ {
signAddPOLVoteSignature(privValidators[i], valSet, voteProto, pol)


+ 31
- 31
consensus/reactor.go View File

@ -9,7 +9,7 @@ import (
"time"
"github.com/tendermint/tendermint/binary"
"github.com/tendermint/tendermint/block"
blk "github.com/tendermint/tendermint/block"
. "github.com/tendermint/tendermint/common"
. "github.com/tendermint/tendermint/consensus/types"
"github.com/tendermint/tendermint/p2p"
@ -34,11 +34,11 @@ type ConsensusReactor struct {
stopped uint32
quit chan struct{}
blockStore *block.BlockStore
blockStore *blk.BlockStore
conS *ConsensusState
}
func NewConsensusReactor(consensusState *ConsensusState, blockStore *block.BlockStore) *ConsensusReactor {
func NewConsensusReactor(consensusState *ConsensusState, blockStore *blk.BlockStore) *ConsensusReactor {
conR := &ConsensusReactor{
blockStore: blockStore,
quit: make(chan struct{}),
@ -398,7 +398,7 @@ OUTER_LOOP:
return false
}
trySendCommitFromValidation := func(blockMeta *block.BlockMeta, validation *block.Validation, peerVoteSet BitArray) (sent bool) {
trySendCommitFromValidation := func(blockMeta *blk.BlockMeta, validation *blk.Validation, peerVoteSet BitArray) (sent bool) {
// Initialize Commits if needed
ps.EnsureVoteBitArrays(prs.Height, uint(len(validation.Commits)))
@ -406,10 +406,10 @@ OUTER_LOOP:
commit := validation.Commits[index]
log.Debug("Picked commit to send", "index", index, "commit", commit)
// Reconstruct vote.
vote := &block.Vote{
vote := &blk.Vote{
Height: prs.Height,
Round: commit.Round,
Type: block.VoteTypeCommit,
Type: blk.VoteTypeCommit,
BlockHash: blockMeta.Hash,
BlockParts: blockMeta.Parts,
Signature: commit.Signature,
@ -509,20 +509,20 @@ OUTER_LOOP:
// Read only when returned by PeerState.GetRoundState().
type PeerRoundState struct {
Height uint // Height peer is at
Round uint // Round peer is at
Step RoundStep // Step peer is at
StartTime time.Time // Estimated start of round 0 at this height
Proposal bool // True if peer has proposal for this round
ProposalBlockParts block.PartSetHeader //
ProposalBlockBitArray BitArray // True bit -> has part
ProposalPOLParts block.PartSetHeader //
ProposalPOLBitArray BitArray // True bit -> has part
Prevotes BitArray // All votes peer has for this round
Precommits BitArray // All precommits peer has for this round
Commits BitArray // All commits peer has for this height
LastCommits BitArray // All commits peer has for last height
HasAllCatchupCommits bool // Used for catch-up
Height uint // Height peer is at
Round uint // Round peer is at
Step RoundStep // Step peer is at
StartTime time.Time // Estimated start of round 0 at this height
Proposal bool // True if peer has proposal for this round
ProposalBlockParts blk.PartSetHeader //
ProposalBlockBitArray BitArray // True bit -> has part
ProposalPOLParts blk.PartSetHeader //
ProposalPOLBitArray BitArray // True bit -> has part
Prevotes BitArray // All votes peer has for this round
Precommits BitArray // All precommits peer has for this round
Commits BitArray // All commits peer has for this height
LastCommits BitArray // All commits peer has for last height
HasAllCatchupCommits bool // Used for catch-up
}
//-----------------------------------------------------------------------------
@ -610,7 +610,7 @@ func (ps *PeerState) EnsureVoteBitArrays(height uint, numValidators uint) {
}
}
func (ps *PeerState) SetHasVote(vote *block.Vote, index uint) {
func (ps *PeerState) SetHasVote(vote *blk.Vote, index uint) {
ps.mtx.Lock()
defer ps.mtx.Unlock()
@ -618,7 +618,7 @@ func (ps *PeerState) SetHasVote(vote *block.Vote, index uint) {
}
func (ps *PeerState) setHasVote(height uint, round uint, type_ byte, index uint) {
if ps.Height == height+1 && type_ == block.VoteTypeCommit {
if ps.Height == height+1 && type_ == blk.VoteTypeCommit {
// Special case for LastCommits.
ps.LastCommits.SetIndex(index, true)
return
@ -628,11 +628,11 @@ func (ps *PeerState) setHasVote(height uint, round uint, type_ byte, index uint)
}
switch type_ {
case block.VoteTypePrevote:
case blk.VoteTypePrevote:
ps.Prevotes.SetIndex(index, true)
case block.VoteTypePrecommit:
case blk.VoteTypePrecommit:
ps.Precommits.SetIndex(index, true)
case block.VoteTypeCommit:
case blk.VoteTypeCommit:
if round < ps.Round {
ps.Prevotes.SetIndex(index, true)
ps.Precommits.SetIndex(index, true)
@ -670,9 +670,9 @@ func (ps *PeerState) ApplyNewRoundStepMessage(msg *NewRoundStepMessage, rs *Roun
ps.StartTime = startTime
if psHeight != msg.Height || psRound != msg.Round {
ps.Proposal = false
ps.ProposalBlockParts = block.PartSetHeader{}
ps.ProposalBlockParts = blk.PartSetHeader{}
ps.ProposalBlockBitArray = BitArray{}
ps.ProposalPOLParts = block.PartSetHeader{}
ps.ProposalPOLParts = blk.PartSetHeader{}
ps.ProposalPOLBitArray = BitArray{}
// We'll update the BitArray capacity later.
ps.Prevotes = BitArray{}
@ -708,7 +708,7 @@ func (ps *PeerState) ApplyHasVoteMessage(msg *HasVoteMessage) {
defer ps.mtx.Unlock()
// Special case for LastCommits
if ps.Height == msg.Height+1 && msg.Type == block.VoteTypeCommit {
if ps.Height == msg.Height+1 && msg.Type == blk.VoteTypeCommit {
ps.LastCommits.SetIndex(msg.Index, true)
return
} else if ps.Height != msg.Height {
@ -778,7 +778,7 @@ func (m *NewRoundStepMessage) String() string {
type CommitStepMessage struct {
Height uint
BlockParts block.PartSetHeader
BlockParts blk.PartSetHeader
BlockBitArray BitArray
}
@ -799,7 +799,7 @@ type PartMessage struct {
Height uint
Round uint
Type byte
Part *block.Part
Part *blk.Part
}
func (m *PartMessage) TypeByte() byte { return msgTypePart }
@ -812,7 +812,7 @@ func (m *PartMessage) String() string {
type VoteMessage struct {
ValidatorIndex uint
Vote *block.Vote
Vote *blk.Vote
}
func (m *VoteMessage) TypeByte() byte { return msgTypeVote }


+ 71
- 71
consensus/state.go View File

@ -62,7 +62,7 @@ import (
"github.com/tendermint/tendermint/account"
"github.com/tendermint/tendermint/binary"
"github.com/tendermint/tendermint/block"
blk "github.com/tendermint/tendermint/block"
. "github.com/tendermint/tendermint/common"
"github.com/tendermint/tendermint/config"
. "github.com/tendermint/tendermint/consensus/types"
@ -171,12 +171,12 @@ type RoundState struct {
CommitTime time.Time // Time when +2/3 commits were found
Validators *sm.ValidatorSet
Proposal *Proposal
ProposalBlock *block.Block
ProposalBlockParts *block.PartSet
ProposalBlock *blk.Block
ProposalBlockParts *blk.PartSet
ProposalPOL *POL
ProposalPOLParts *block.PartSet
LockedBlock *block.Block
LockedBlockParts *block.PartSet
ProposalPOLParts *blk.PartSet
LockedBlock *blk.Block
LockedBlockParts *blk.PartSet
LockedPOL *POL // Rarely needed, so no LockedPOLParts.
Prevotes *VoteSet
Precommits *VoteSet
@ -234,20 +234,20 @@ type ConsensusState struct {
stopped uint32
quit chan struct{}
blockStore *block.BlockStore
blockStore *blk.BlockStore
mempoolReactor *mempl.MempoolReactor
runActionCh chan RoundAction
newStepCh chan *RoundState
mtx sync.Mutex
RoundState
state *sm.State // State until height-1.
stagedBlock *block.Block // Cache last staged block.
stagedState *sm.State // Cache result of staged block.
lastCommitVoteHeight uint // Last called commitVoteBlock() or saveCommitVoteBlock() on.
state *sm.State // State until height-1.
stagedBlock *blk.Block // Cache last staged block.
stagedState *sm.State // Cache result of staged block.
lastCommitVoteHeight uint // Last called commitVoteBlock() or saveCommitVoteBlock() on.
}
func NewConsensusState(state *sm.State, blockStore *block.BlockStore, mempoolReactor *mempl.MempoolReactor) *ConsensusState {
func NewConsensusState(state *sm.State, blockStore *blk.BlockStore, mempoolReactor *mempl.MempoolReactor) *ConsensusState {
cs := &ConsensusState{
quit: make(chan struct{}),
blockStore: blockStore,
@ -484,10 +484,10 @@ func (cs *ConsensusState) updateToState(state *sm.State) {
cs.LockedBlock = nil
cs.LockedBlockParts = nil
cs.LockedPOL = nil
cs.Prevotes = NewVoteSet(height, 0, block.VoteTypePrevote, validators)
cs.Precommits = NewVoteSet(height, 0, block.VoteTypePrecommit, validators)
cs.Prevotes = NewVoteSet(height, 0, blk.VoteTypePrevote, validators)
cs.Precommits = NewVoteSet(height, 0, blk.VoteTypePrecommit, validators)
cs.LastCommits = cs.Commits
cs.Commits = NewVoteSet(height, 0, block.VoteTypeCommit, validators)
cs.Commits = NewVoteSet(height, 0, blk.VoteTypeCommit, validators)
cs.state = state
cs.stagedBlock = nil
@ -501,7 +501,7 @@ func (cs *ConsensusState) updateToState(state *sm.State) {
// If we've timed out, then send rebond tx.
if cs.PrivValidator != nil && cs.state.UnbondingValidators.HasAddress(cs.PrivValidator.Address) {
rebondTx := &block.RebondTx{
rebondTx := &blk.RebondTx{
Address: cs.PrivValidator.Address,
Height: cs.Height + 1,
}
@ -534,9 +534,9 @@ func (cs *ConsensusState) setupNewRound(round uint) {
cs.ProposalBlockParts = nil
cs.ProposalPOL = nil
cs.ProposalPOLParts = nil
cs.Prevotes = NewVoteSet(cs.Height, round, block.VoteTypePrevote, validators)
cs.Prevotes = NewVoteSet(cs.Height, round, blk.VoteTypePrevote, validators)
cs.Prevotes.AddFromCommits(cs.Commits)
cs.Precommits = NewVoteSet(cs.Height, round, block.VoteTypePrecommit, validators)
cs.Precommits = NewVoteSet(cs.Height, round, blk.VoteTypePrecommit, validators)
cs.Precommits.AddFromCommits(cs.Commits)
}
@ -586,24 +586,24 @@ func (cs *ConsensusState) RunActionPropose(height uint, round uint) {
log.Debug("Our turn to propose", "proposer", cs.Validators.Proposer().Address, "privValidator", cs.PrivValidator)
}
var block_ *block.Block
var blockParts *block.PartSet
var block *blk.Block
var blockParts *blk.PartSet
var pol *POL
var polParts *block.PartSet
var polParts *blk.PartSet
// Decide on block and POL
if cs.LockedBlock != nil {
// If we're locked onto a block, just choose that.
block_ = cs.LockedBlock
block = cs.LockedBlock
blockParts = cs.LockedBlockParts
pol = cs.LockedPOL
} else {
// Otherwise we should create a new proposal.
var validation *block.Validation
var validation *blk.Validation
if cs.Height == 1 {
// We're creating a proposal for the first block.
// The validation is empty.
validation = &block.Validation{}
validation = &blk.Validation{}
} else if cs.LastCommits.HasTwoThirdsMajority() {
// Make the validation from LastCommits
validation = cs.LastCommits.MakeValidation()
@ -617,8 +617,8 @@ func (cs *ConsensusState) RunActionPropose(height uint, round uint) {
}
}
txs := cs.mempoolReactor.Mempool.GetProposalTxs()
block_ = &block.Block{
Header: &block.Header{
block = &blk.Block{
Header: &blk.Header{
Network: config.App.GetString("Network"),
Height: cs.Height,
Time: time.Now(),
@ -629,22 +629,22 @@ func (cs *ConsensusState) RunActionPropose(height uint, round uint) {
StateHash: nil, // Will set afterwards.
},
Validation: validation,
Data: &block.Data{
Data: &blk.Data{
Txs: txs,
},
}
// Set the block.Header.StateHash.
// Set the blk.Header.StateHash.
// TODO: we could cache the resulting state to cs.stagedState.
// TODO: This is confusing, not clear that we're mutating block.
cs.state.Copy().AppendBlock(block_, block.PartSetHeader{}, false)
cs.state.Copy().AppendBlock(block, blk.PartSetHeader{}, false)
blockParts = block.NewPartSetFromData(binary.BinaryBytes(block_))
blockParts = blk.NewPartSetFromData(binary.BinaryBytes(block))
pol = cs.LockedPOL // If exists, is a PoUnlock.
}
if pol != nil {
polParts = block.NewPartSetFromData(binary.BinaryBytes(pol))
polParts = blk.NewPartSetFromData(binary.BinaryBytes(pol))
}
// Make proposal
@ -652,10 +652,10 @@ func (cs *ConsensusState) RunActionPropose(height uint, round uint) {
err := cs.PrivValidator.SignProposal(proposal)
if err == nil {
log.Info("Signed and set proposal", "height", cs.Height, "round", cs.Round, "proposal", proposal)
log.Debug(Fmt("Signed and set proposal block: %v", block_))
log.Debug(Fmt("Signed and set proposal block: %v", block))
// Set fields
cs.Proposal = proposal
cs.ProposalBlock = block_
cs.ProposalBlock = block
cs.ProposalBlockParts = blockParts
cs.ProposalPOL = pol
cs.ProposalPOLParts = polParts
@ -679,14 +679,14 @@ func (cs *ConsensusState) RunActionPrevote(height uint, round uint) {
// If a block is locked, prevote that.
if cs.LockedBlock != nil {
cs.signAddVote(block.VoteTypePrevote, cs.LockedBlock.Hash(), cs.LockedBlockParts.Header())
cs.signAddVote(blk.VoteTypePrevote, cs.LockedBlock.Hash(), cs.LockedBlockParts.Header())
return
}
// If ProposalBlock is nil, prevote nil.
if cs.ProposalBlock == nil {
log.Warn("ProposalBlock is nil")
cs.signAddVote(block.VoteTypePrevote, nil, block.PartSetHeader{})
cs.signAddVote(blk.VoteTypePrevote, nil, blk.PartSetHeader{})
return
}
@ -695,12 +695,12 @@ func (cs *ConsensusState) RunActionPrevote(height uint, round uint) {
if err != nil {
// ProposalBlock is invalid, prevote nil.
log.Warn("ProposalBlock is invalid", "error", err)
cs.signAddVote(block.VoteTypePrevote, nil, block.PartSetHeader{})
cs.signAddVote(blk.VoteTypePrevote, nil, blk.PartSetHeader{})
return
}
// Prevote cs.ProposalBlock
cs.signAddVote(block.VoteTypePrevote, cs.ProposalBlock.Hash(), cs.ProposalBlockParts.Header())
cs.signAddVote(blk.VoteTypePrevote, cs.ProposalBlock.Hash(), cs.ProposalBlockParts.Header())
return
}
@ -736,7 +736,7 @@ func (cs *ConsensusState) RunActionPrecommit(height uint, round uint) {
// If +2/3 prevoted for already locked block, precommit it.
if cs.LockedBlock.HashesTo(hash) {
cs.signAddVote(block.VoteTypePrecommit, hash, partsHeader)
cs.signAddVote(blk.VoteTypePrecommit, hash, partsHeader)
return
}
@ -750,7 +750,7 @@ func (cs *ConsensusState) RunActionPrecommit(height uint, round uint) {
}
cs.LockedBlock = cs.ProposalBlock
cs.LockedBlockParts = cs.ProposalBlockParts
cs.signAddVote(block.VoteTypePrecommit, hash, partsHeader)
cs.signAddVote(blk.VoteTypePrecommit, hash, partsHeader)
return
}
@ -804,7 +804,7 @@ func (cs *ConsensusState) RunActionCommit(height uint) {
// We're getting the wrong block.
// Set up ProposalBlockParts and keep waiting.
cs.ProposalBlock = nil
cs.ProposalBlockParts = block.NewPartSetFromHeader(partsHeader)
cs.ProposalBlockParts = blk.NewPartSetFromHeader(partsHeader)
} else {
// We just need to keep waiting.
@ -894,14 +894,14 @@ func (cs *ConsensusState) SetProposal(proposal *Proposal) error {
}
cs.Proposal = proposal
cs.ProposalBlockParts = block.NewPartSetFromHeader(proposal.BlockParts)
cs.ProposalPOLParts = block.NewPartSetFromHeader(proposal.POLParts)
cs.ProposalBlockParts = blk.NewPartSetFromHeader(proposal.BlockParts)
cs.ProposalPOLParts = blk.NewPartSetFromHeader(proposal.POLParts)
return nil
}
// NOTE: block is not necessarily valid.
// NOTE: This function may increment the height.
func (cs *ConsensusState) AddProposalBlockPart(height uint, round uint, part *block.Part) (added bool, err error) {
func (cs *ConsensusState) AddProposalBlockPart(height uint, round uint, part *blk.Part) (added bool, err error) {
cs.mtx.Lock()
defer cs.mtx.Unlock()
@ -922,7 +922,7 @@ func (cs *ConsensusState) AddProposalBlockPart(height uint, round uint, part *bl
if added && cs.ProposalBlockParts.IsComplete() {
var n int64
var err error
cs.ProposalBlock = binary.ReadBinary(&block.Block{}, cs.ProposalBlockParts.GetReader(), &n, &err).(*block.Block)
cs.ProposalBlock = binary.ReadBinary(&blk.Block{}, cs.ProposalBlockParts.GetReader(), &n, &err).(*blk.Block)
// If we're already in the commit step, try to finalize round.
if cs.Step == RoundStepCommit {
cs.queueAction(RoundAction{cs.Height, cs.Round, RoundActionTryFinalize})
@ -934,7 +934,7 @@ func (cs *ConsensusState) AddProposalBlockPart(height uint, round uint, part *bl
}
// NOTE: POL is not necessarily valid.
func (cs *ConsensusState) AddProposalPOLPart(height uint, round uint, part *block.Part) (added bool, err error) {
func (cs *ConsensusState) AddProposalPOLPart(height uint, round uint, part *blk.Part) (added bool, err error) {
cs.mtx.Lock()
defer cs.mtx.Unlock()
@ -960,7 +960,7 @@ func (cs *ConsensusState) AddProposalPOLPart(height uint, round uint, part *bloc
return true, nil
}
func (cs *ConsensusState) AddVote(address []byte, vote *block.Vote) (added bool, index uint, err error) {
func (cs *ConsensusState) AddVote(address []byte, vote *blk.Vote) (added bool, index uint, err error) {
cs.mtx.Lock()
defer cs.mtx.Unlock()
@ -969,15 +969,15 @@ func (cs *ConsensusState) AddVote(address []byte, vote *block.Vote) (added bool,
//-----------------------------------------------------------------------------
func (cs *ConsensusState) addVote(address []byte, vote *block.Vote) (added bool, index uint, err error) {
func (cs *ConsensusState) addVote(address []byte, vote *blk.Vote) (added bool, index uint, err error) {
switch vote.Type {
case block.VoteTypePrevote:
case blk.VoteTypePrevote:
// Prevotes checks for height+round match.
return cs.Prevotes.Add(address, vote)
case block.VoteTypePrecommit:
case blk.VoteTypePrecommit:
// Precommits checks for height+round match.
return cs.Precommits.Add(address, vote)
case block.VoteTypeCommit:
case blk.VoteTypeCommit:
if vote.Height == cs.Height {
// No need to check if vote.Round < cs.Round ...
// Prevotes && Precommits already checks that.
@ -1004,13 +1004,13 @@ func (cs *ConsensusState) addVote(address []byte, vote *block.Vote) (added bool,
}
}
func (cs *ConsensusState) stageBlock(block_ *block.Block, blockParts *block.PartSet) error {
if block_ == nil {
func (cs *ConsensusState) stageBlock(block *blk.Block, blockParts *blk.PartSet) error {
if block == nil {
panic("Cannot stage nil block")
}
// Already staged?
if cs.stagedBlock == block_ {
if cs.stagedBlock == block {
return nil
}
@ -1019,21 +1019,21 @@ func (cs *ConsensusState) stageBlock(block_ *block.Block, blockParts *block.Part
// Commit block onto the copied state.
// NOTE: Basic validation is done in state.AppendBlock().
err := stateCopy.AppendBlock(block_, blockParts.Header(), true)
err := stateCopy.AppendBlock(block, blockParts.Header(), true)
if err != nil {
return err
} else {
cs.stagedBlock = block_
cs.stagedBlock = block
cs.stagedState = stateCopy
return nil
}
}
func (cs *ConsensusState) signAddVote(type_ byte, hash []byte, header block.PartSetHeader) *block.Vote {
func (cs *ConsensusState) signAddVote(type_ byte, hash []byte, header blk.PartSetHeader) *blk.Vote {
if cs.PrivValidator == nil || !cs.Validators.HasAddress(cs.PrivValidator.Address) {
return nil
}
vote := &block.Vote{
vote := &blk.Vote{
Height: cs.Height,
Round: cs.Round,
Type: type_,
@ -1052,51 +1052,51 @@ func (cs *ConsensusState) signAddVote(type_ byte, hash []byte, header block.Part
}
// sign a Commit-Vote
func (cs *ConsensusState) commitVoteBlock(block_ *block.Block, blockParts *block.PartSet) {
func (cs *ConsensusState) commitVoteBlock(block *blk.Block, blockParts *blk.PartSet) {
// The proposal must be valid.
if err := cs.stageBlock(block_, blockParts); err != nil {
if err := cs.stageBlock(block, blockParts); err != nil {
// Prevent zombies.
log.Warn("commitVoteBlock() an invalid block", "error", err)
return
}
// Commit-vote.
if cs.lastCommitVoteHeight < block_.Height {
cs.signAddVote(block.VoteTypeCommit, block_.Hash(), blockParts.Header())
cs.lastCommitVoteHeight = block_.Height
if cs.lastCommitVoteHeight < block.Height {
cs.signAddVote(blk.VoteTypeCommit, block.Hash(), blockParts.Header())
cs.lastCommitVoteHeight = block.Height
} else {
log.Error("Duplicate commitVoteBlock() attempt", "lastCommitVoteHeight", cs.lastCommitVoteHeight, "block.Height", block_.Height)
log.Error("Duplicate commitVoteBlock() attempt", "lastCommitVoteHeight", cs.lastCommitVoteHeight, "blk.Height", block.Height)
}
}
// Save Block, save the +2/3 Commits we've seen,
// and sign a Commit-Vote if we haven't already
func (cs *ConsensusState) saveCommitVoteBlock(block_ *block.Block, blockParts *block.PartSet, commits *VoteSet) {
func (cs *ConsensusState) saveCommitVoteBlock(block *blk.Block, blockParts *blk.PartSet, commits *VoteSet) {
// The proposal must be valid.
if err := cs.stageBlock(block_, blockParts); err != nil {
if err := cs.stageBlock(block, blockParts); err != nil {
// Prevent zombies.
log.Warn("saveCommitVoteBlock() an invalid block", "error", err)
return
}
// Save to blockStore.
if cs.blockStore.Height() < block_.Height {
if cs.blockStore.Height() < block.Height {
seenValidation := commits.MakeValidation()
cs.blockStore.SaveBlock(block_, blockParts, seenValidation)
cs.blockStore.SaveBlock(block, blockParts, seenValidation)
}
// Save the state.
cs.stagedState.Save()
// Update mempool.
cs.mempoolReactor.Mempool.ResetForBlockAndState(block_, cs.stagedState)
cs.mempoolReactor.Mempool.ResetForBlockAndState(block, cs.stagedState)
// Commit-vote if we haven't already.
if cs.lastCommitVoteHeight < block_.Height {
cs.signAddVote(block.VoteTypeCommit, block_.Hash(), blockParts.Header())
cs.lastCommitVoteHeight = block_.Height
if cs.lastCommitVoteHeight < block.Height {
cs.signAddVote(blk.VoteTypeCommit, block.Hash(), blockParts.Header())
cs.lastCommitVoteHeight = block.Height
}
}


+ 15
- 15
consensus/state_test.go View File

@ -4,7 +4,7 @@ import (
"bytes"
"testing"
"github.com/tendermint/tendermint/block"
blk "github.com/tendermint/tendermint/block"
)
func TestSetupRound(t *testing.T) {
@ -12,9 +12,9 @@ func TestSetupRound(t *testing.T) {
val0 := privValidators[0]
// Add a vote, precommit, and commit by val0.
voteTypes := []byte{block.VoteTypePrevote, block.VoteTypePrecommit, block.VoteTypeCommit}
voteTypes := []byte{blk.VoteTypePrevote, blk.VoteTypePrecommit, blk.VoteTypeCommit}
for _, voteType := range voteTypes {
vote := &block.Vote{Height: 1, Round: 0, Type: voteType} // nil vote
vote := &blk.Vote{Height: 1, Round: 0, Type: voteType} // nil vote
err := val0.SignVote(vote)
if err != nil {
t.Error("Error signing vote: %v", err)
@ -24,13 +24,13 @@ func TestSetupRound(t *testing.T) {
// Ensure that vote appears in RoundState.
rs0 := cs.GetRoundState()
if vote := rs0.Prevotes.GetByAddress(val0.Address); vote == nil || vote.Type != block.VoteTypePrevote {
if vote := rs0.Prevotes.GetByAddress(val0.Address); vote == nil || vote.Type != blk.VoteTypePrevote {
t.Errorf("Expected to find prevote but got %v", vote)
}
if vote := rs0.Precommits.GetByAddress(val0.Address); vote == nil || vote.Type != block.VoteTypePrecommit {
if vote := rs0.Precommits.GetByAddress(val0.Address); vote == nil || vote.Type != blk.VoteTypePrecommit {
t.Errorf("Expected to find precommit but got %v", vote)
}
if vote := rs0.Commits.GetByAddress(val0.Address); vote == nil || vote.Type != block.VoteTypeCommit {
if vote := rs0.Commits.GetByAddress(val0.Address); vote == nil || vote.Type != blk.VoteTypeCommit {
t.Errorf("Expected to find commit but got %v", vote)
}
@ -40,13 +40,13 @@ func TestSetupRound(t *testing.T) {
// Now the commit should be copied over to prevotes and precommits.
rs1 := cs.GetRoundState()
if vote := rs1.Prevotes.GetByAddress(val0.Address); vote == nil || vote.Type != block.VoteTypeCommit {
if vote := rs1.Prevotes.GetByAddress(val0.Address); vote == nil || vote.Type != blk.VoteTypeCommit {
t.Errorf("Expected to find commit but got %v", vote)
}
if vote := rs1.Precommits.GetByAddress(val0.Address); vote == nil || vote.Type != block.VoteTypeCommit {
if vote := rs1.Precommits.GetByAddress(val0.Address); vote == nil || vote.Type != blk.VoteTypeCommit {
t.Errorf("Expected to find commit but got %v", vote)
}
if vote := rs1.Commits.GetByAddress(val0.Address); vote == nil || vote.Type != block.VoteTypeCommit {
if vote := rs1.Commits.GetByAddress(val0.Address); vote == nil || vote.Type != blk.VoteTypeCommit {
t.Errorf("Expected to find commit but got %v", vote)
}
@ -116,10 +116,10 @@ func TestRunActionPrecommitCommitFinalize(t *testing.T) {
// Add at least +2/3 prevotes.
for i := 0; i < 7; i++ {
vote := &block.Vote{
vote := &blk.Vote{
Height: 1,
Round: 0,
Type: block.VoteTypePrevote,
Type: blk.VoteTypePrevote,
BlockHash: cs.ProposalBlock.Hash(),
BlockParts: cs.ProposalBlockParts.Header(),
}
@ -146,10 +146,10 @@ func TestRunActionPrecommitCommitFinalize(t *testing.T) {
}
continue
}
vote := &block.Vote{
vote := &blk.Vote{
Height: 1,
Round: 0,
Type: block.VoteTypePrecommit,
Type: blk.VoteTypePrecommit,
BlockHash: cs.ProposalBlock.Hash(),
BlockParts: cs.ProposalBlockParts.Header(),
}
@ -184,10 +184,10 @@ func TestRunActionPrecommitCommitFinalize(t *testing.T) {
}
continue
}
vote := &block.Vote{
vote := &blk.Vote{
Height: 1,
Round: uint(i), // Doesn't matter what round
Type: block.VoteTypeCommit,
Type: blk.VoteTypeCommit,
BlockHash: cs.ProposalBlock.Hash(),
BlockParts: cs.ProposalBlockParts.Header(),
}


+ 2
- 2
consensus/test.go View File

@ -3,7 +3,7 @@ package consensus
import (
"sort"
"github.com/tendermint/tendermint/block"
blk "github.com/tendermint/tendermint/block"
dbm "github.com/tendermint/tendermint/db"
mempl "github.com/tendermint/tendermint/mempool"
sm "github.com/tendermint/tendermint/state"
@ -39,7 +39,7 @@ func randVoteSet(height uint, round uint, type_ byte, numValidators int, votingP
func randConsensusState() (*ConsensusState, []*sm.PrivValidator) {
state, _, privValidators := sm.RandGenesisState(20, false, 1000, 10, false, 1000)
blockStore := block.NewBlockStore(dbm.NewMemDB())
blockStore := blk.NewBlockStore(dbm.NewMemDB())
mempool := mempl.NewMempool(state)
mempoolReactor := mempl.NewMempoolReactor(mempool)
cs := NewConsensusState(state, blockStore, mempoolReactor)


+ 4
- 4
consensus/types/proposal.go View File

@ -7,7 +7,7 @@ import (
"github.com/tendermint/tendermint/account"
"github.com/tendermint/tendermint/binary"
. "github.com/tendermint/tendermint/block"
blk "github.com/tendermint/tendermint/block"
)
var (
@ -18,12 +18,12 @@ var (
type Proposal struct {
Height uint
Round uint
BlockParts PartSetHeader
POLParts PartSetHeader
BlockParts blk.PartSetHeader
POLParts blk.PartSetHeader
Signature account.SignatureEd25519
}
func NewProposal(height uint, round uint, blockParts, polParts PartSetHeader) *Proposal {
func NewProposal(height uint, round uint, blockParts, polParts blk.PartSetHeader) *Proposal {
return &Proposal{
Height: height,
Round: round,


+ 26
- 26
consensus/vote_set.go View File

@ -8,7 +8,7 @@ import (
"github.com/tendermint/tendermint/account"
"github.com/tendermint/tendermint/binary"
"github.com/tendermint/tendermint/block"
blk "github.com/tendermint/tendermint/block"
. "github.com/tendermint/tendermint/common"
sm "github.com/tendermint/tendermint/state"
)
@ -25,12 +25,12 @@ type VoteSet struct {
mtx sync.Mutex
valSet *sm.ValidatorSet
votes []*block.Vote // validator index -> vote
votes []*blk.Vote // validator index -> vote
votesBitArray BitArray // validator index -> has vote?
votesByBlock map[string]uint64 // string(blockHash)+string(blockParts) -> vote sum.
totalVotes uint64
maj23Hash []byte
maj23Parts block.PartSetHeader
maj23Parts blk.PartSetHeader
maj23Exists bool
}
@ -39,7 +39,7 @@ func NewVoteSet(height uint, round uint, type_ byte, valSet *sm.ValidatorSet) *V
if height == 0 {
panic("Cannot make VoteSet for height == 0, doesn't make sense.")
}
if type_ == block.VoteTypeCommit && round != 0 {
if type_ == blk.VoteTypeCommit && round != 0 {
panic("Expected round 0 for commit vote set")
}
return &VoteSet{
@ -47,7 +47,7 @@ func NewVoteSet(height uint, round uint, type_ byte, valSet *sm.ValidatorSet) *V
round: round,
type_: type_,
valSet: valSet,
votes: make([]*block.Vote, valSet.Size()),
votes: make([]*blk.Vote, valSet.Size()),
votesBitArray: NewBitArray(valSet.Size()),
votesByBlock: make(map[string]uint64),
totalVotes: 0,
@ -65,40 +65,40 @@ func (voteSet *VoteSet) Size() uint {
// True if added, false if not.
// Returns ErrVote[UnexpectedStep|InvalidAccount|InvalidSignature|InvalidBlockHash|ConflictingSignature]
// NOTE: vote should not be mutated after adding.
func (voteSet *VoteSet) Add(address []byte, vote *block.Vote) (bool, uint, error) {
func (voteSet *VoteSet) Add(address []byte, vote *blk.Vote) (bool, uint, error) {
voteSet.mtx.Lock()
defer voteSet.mtx.Unlock()
// Make sure the step matches. (or that vote is commit && round < voteSet.round)
if vote.Height != voteSet.height ||
(vote.Type != block.VoteTypeCommit && vote.Round != voteSet.round) ||
(vote.Type != block.VoteTypeCommit && vote.Type != voteSet.type_) ||
(vote.Type == block.VoteTypeCommit && voteSet.type_ != block.VoteTypeCommit && vote.Round >= voteSet.round) {
return false, 0, block.ErrVoteUnexpectedStep
(vote.Type != blk.VoteTypeCommit && vote.Round != voteSet.round) ||
(vote.Type != blk.VoteTypeCommit && vote.Type != voteSet.type_) ||
(vote.Type == blk.VoteTypeCommit && voteSet.type_ != blk.VoteTypeCommit && vote.Round >= voteSet.round) {
return false, 0, blk.ErrVoteUnexpectedStep
}
// Ensure that signer is a validator.
valIndex, val := voteSet.valSet.GetByAddress(address)
if val == nil {
return false, 0, block.ErrVoteInvalidAccount
return false, 0, blk.ErrVoteInvalidAccount
}
// Check signature.
if !val.PubKey.VerifyBytes(account.SignBytes(vote), vote.Signature) {
// Bad signature.
return false, 0, block.ErrVoteInvalidSignature
return false, 0, blk.ErrVoteInvalidSignature
}
return voteSet.addVote(valIndex, vote)
}
func (voteSet *VoteSet) addVote(valIndex uint, vote *block.Vote) (bool, uint, error) {
func (voteSet *VoteSet) addVote(valIndex uint, vote *blk.Vote) (bool, uint, error) {
// If vote already exists, return false.
if existingVote := voteSet.votes[valIndex]; existingVote != nil {
if bytes.Equal(existingVote.BlockHash, vote.BlockHash) {
return false, 0, nil
} else {
return false, 0, block.ErrVoteConflictingSignature
return false, 0, blk.ErrVoteConflictingSignature
}
}
@ -146,13 +146,13 @@ func (voteSet *VoteSet) BitArray() BitArray {
return voteSet.votesBitArray.Copy()
}
func (voteSet *VoteSet) GetByIndex(valIndex uint) *block.Vote {
func (voteSet *VoteSet) GetByIndex(valIndex uint) *blk.Vote {
voteSet.mtx.Lock()
defer voteSet.mtx.Unlock()
return voteSet.votes[valIndex]
}
func (voteSet *VoteSet) GetByAddress(address []byte) *block.Vote {
func (voteSet *VoteSet) GetByAddress(address []byte) *blk.Vote {
voteSet.mtx.Lock()
defer voteSet.mtx.Unlock()
valIndex, val := voteSet.valSet.GetByAddress(address)
@ -173,19 +173,19 @@ func (voteSet *VoteSet) HasTwoThirdsMajority() bool {
// Returns either a blockhash (or nil) that received +2/3 majority.
// If there exists no such majority, returns (nil, false).
func (voteSet *VoteSet) TwoThirdsMajority() (hash []byte, parts block.PartSetHeader, ok bool) {
func (voteSet *VoteSet) TwoThirdsMajority() (hash []byte, parts blk.PartSetHeader, ok bool) {
voteSet.mtx.Lock()
defer voteSet.mtx.Unlock()
if voteSet.maj23Exists {
return voteSet.maj23Hash, voteSet.maj23Parts, true
} else {
return nil, block.PartSetHeader{}, false
return nil, blk.PartSetHeader{}, false
}
}
func (voteSet *VoteSet) MakePOL() *POL {
if voteSet.type_ != block.VoteTypePrevote {
panic("Cannot MakePOL() unless VoteSet.Type is block.VoteTypePrevote")
if voteSet.type_ != blk.VoteTypePrevote {
panic("Cannot MakePOL() unless VoteSet.Type is blk.VoteTypePrevote")
}
voteSet.mtx.Lock()
defer voteSet.mtx.Unlock()
@ -217,16 +217,16 @@ func (voteSet *VoteSet) MakePOL() *POL {
return pol
}
func (voteSet *VoteSet) MakeValidation() *block.Validation {
if voteSet.type_ != block.VoteTypeCommit {
panic("Cannot MakeValidation() unless VoteSet.Type is block.VoteTypeCommit")
func (voteSet *VoteSet) MakeValidation() *blk.Validation {
if voteSet.type_ != blk.VoteTypeCommit {
panic("Cannot MakeValidation() unless VoteSet.Type is blk.VoteTypeCommit")
}
voteSet.mtx.Lock()
defer voteSet.mtx.Unlock()
if len(voteSet.maj23Hash) == 0 {
panic("Cannot MakeValidation() unless a blockhash has +2/3")
}
commits := make([]block.Commit, voteSet.valSet.Size())
commits := make([]blk.Commit, voteSet.valSet.Size())
voteSet.valSet.Iterate(func(valIndex uint, val *sm.Validator) bool {
vote := voteSet.votes[valIndex]
if vote == nil {
@ -238,10 +238,10 @@ func (voteSet *VoteSet) MakeValidation() *block.Validation {
if !vote.BlockParts.Equals(voteSet.maj23Parts) {
return false
}
commits[valIndex] = block.Commit{val.Address, vote.Round, vote.Signature}
commits[valIndex] = blk.Commit{val.Address, vote.Round, vote.Signature}
return false
})
return &block.Validation{
return &blk.Validation{
Commits: commits,
}
}


+ 30
- 30
consensus/vote_set_test.go View File

@ -3,7 +3,7 @@ package consensus
import (
"bytes"
"github.com/tendermint/tendermint/block"
blk "github.com/tendermint/tendermint/block"
. "github.com/tendermint/tendermint/common"
. "github.com/tendermint/tendermint/common/test"
sm "github.com/tendermint/tendermint/state"
@ -14,41 +14,41 @@ import (
// NOTE: see consensus/test.go for common test methods.
// Convenience: Return new vote with different height
func withHeight(vote *block.Vote, height uint) *block.Vote {
func withHeight(vote *blk.Vote, height uint) *blk.Vote {
vote = vote.Copy()
vote.Height = height
return vote
}
// Convenience: Return new vote with different round
func withRound(vote *block.Vote, round uint) *block.Vote {
func withRound(vote *blk.Vote, round uint) *blk.Vote {
vote = vote.Copy()
vote.Round = round
return vote
}
// Convenience: Return new vote with different type
func withType(vote *block.Vote, type_ byte) *block.Vote {
func withType(vote *blk.Vote, type_ byte) *blk.Vote {
vote = vote.Copy()
vote.Type = type_
return vote
}
// Convenience: Return new vote with different blockHash
func withBlockHash(vote *block.Vote, blockHash []byte) *block.Vote {
func withBlockHash(vote *blk.Vote, blockHash []byte) *blk.Vote {
vote = vote.Copy()
vote.BlockHash = blockHash
return vote
}
// Convenience: Return new vote with different blockParts
func withBlockParts(vote *block.Vote, blockParts block.PartSetHeader) *block.Vote {
func withBlockParts(vote *blk.Vote, blockParts blk.PartSetHeader) *blk.Vote {
vote = vote.Copy()
vote.BlockParts = blockParts
return vote
}
func signAddVote(privVal *sm.PrivValidator, vote *block.Vote, voteSet *VoteSet) (bool, error) {
func signAddVote(privVal *sm.PrivValidator, vote *blk.Vote, voteSet *VoteSet) (bool, error) {
privVal.SignVoteUnsafe(vote)
added, _, err := voteSet.Add(privVal.Address, vote)
return added, err
@ -56,7 +56,7 @@ func signAddVote(privVal *sm.PrivValidator, vote *block.Vote, voteSet *VoteSet)
func TestAddVote(t *testing.T) {
height, round := uint(1), uint(0)
voteSet, _, privValidators := randVoteSet(height, round, block.VoteTypePrevote, 10, 1)
voteSet, _, privValidators := randVoteSet(height, round, blk.VoteTypePrevote, 10, 1)
val0 := privValidators[0]
// t.Logf(">> %v", voteSet)
@ -72,7 +72,7 @@ func TestAddVote(t *testing.T) {
t.Errorf("There should be no 2/3 majority")
}
vote := &block.Vote{Height: height, Round: round, Type: block.VoteTypePrevote, BlockHash: nil}
vote := &blk.Vote{Height: height, Round: round, Type: blk.VoteTypePrevote, BlockHash: nil}
signAddVote(val0, vote, voteSet)
if voteSet.GetByAddress(val0.Address) == nil {
@ -89,9 +89,9 @@ func TestAddVote(t *testing.T) {
func Test2_3Majority(t *testing.T) {
height, round := uint(1), uint(0)
voteSet, _, privValidators := randVoteSet(height, round, block.VoteTypePrevote, 10, 1)
voteSet, _, privValidators := randVoteSet(height, round, blk.VoteTypePrevote, 10, 1)
vote := &block.Vote{Height: height, Round: round, Type: block.VoteTypePrevote, BlockHash: nil}
vote := &blk.Vote{Height: height, Round: round, Type: blk.VoteTypePrevote, BlockHash: nil}
// 6 out of 10 voted for nil.
for i := 0; i < 6; i++ {
@ -123,13 +123,13 @@ func Test2_3Majority(t *testing.T) {
func Test2_3MajorityRedux(t *testing.T) {
height, round := uint(1), uint(0)
voteSet, _, privValidators := randVoteSet(height, round, block.VoteTypePrevote, 100, 1)
voteSet, _, privValidators := randVoteSet(height, round, blk.VoteTypePrevote, 100, 1)
blockHash := CRandBytes(32)
blockPartsTotal := uint(123)
blockParts := block.PartSetHeader{blockPartsTotal, CRandBytes(32)}
blockParts := blk.PartSetHeader{blockPartsTotal, CRandBytes(32)}
vote := &block.Vote{Height: height, Round: round, Type: block.VoteTypePrevote, BlockHash: blockHash, BlockParts: blockParts}
vote := &blk.Vote{Height: height, Round: round, Type: blk.VoteTypePrevote, BlockHash: blockHash, BlockParts: blockParts}
// 66 out of 100 voted for nil.
for i := 0; i < 66; i++ {
@ -151,7 +151,7 @@ func Test2_3MajorityRedux(t *testing.T) {
// 68th validator voted for a different BlockParts PartSetHeader
{
blockParts := block.PartSetHeader{blockPartsTotal, CRandBytes(32)}
blockParts := blk.PartSetHeader{blockPartsTotal, CRandBytes(32)}
signAddVote(privValidators[67], withBlockParts(vote, blockParts), voteSet)
hash, header, ok = voteSet.TwoThirdsMajority()
if hash != nil || !header.IsZero() || ok {
@ -161,7 +161,7 @@ func Test2_3MajorityRedux(t *testing.T) {
// 69th validator voted for different BlockParts Total
{
blockParts := block.PartSetHeader{blockPartsTotal + 1, blockParts.Hash}
blockParts := blk.PartSetHeader{blockPartsTotal + 1, blockParts.Hash}
signAddVote(privValidators[68], withBlockParts(vote, blockParts), voteSet)
hash, header, ok = voteSet.TwoThirdsMajority()
if hash != nil || !header.IsZero() || ok {
@ -190,10 +190,10 @@ func Test2_3MajorityRedux(t *testing.T) {
func TestBadVotes(t *testing.T) {
height, round := uint(1), uint(0)
voteSet, _, privValidators := randVoteSet(height, round, block.VoteTypePrevote, 10, 1)
voteSet, _, privValidators := randVoteSet(height, round, blk.VoteTypePrevote, 10, 1)
// val0 votes for nil.
vote := &block.Vote{Height: height, Round: round, Type: block.VoteTypePrevote, BlockHash: nil}
vote := &blk.Vote{Height: height, Round: round, Type: blk.VoteTypePrevote, BlockHash: nil}
added, err := signAddVote(privValidators[0], vote, voteSet)
if !added || err != nil {
t.Errorf("Expected Add() to succeed")
@ -218,7 +218,7 @@ func TestBadVotes(t *testing.T) {
}
// val3 votes of another type.
added, err = signAddVote(privValidators[3], withType(vote, block.VoteTypePrecommit), voteSet)
added, err = signAddVote(privValidators[3], withType(vote, blk.VoteTypePrecommit), voteSet)
if added {
t.Errorf("Expected Add() to fail, wrong type")
}
@ -226,10 +226,10 @@ func TestBadVotes(t *testing.T) {
func TestAddCommitsToPrevoteVotes(t *testing.T) {
height, round := uint(2), uint(5)
voteSet, _, privValidators := randVoteSet(height, round, block.VoteTypePrevote, 10, 1)
voteSet, _, privValidators := randVoteSet(height, round, blk.VoteTypePrevote, 10, 1)
// val0, val1, val2, val3, val4, val5 vote for nil.
vote := &block.Vote{Height: height, Round: round, Type: block.VoteTypePrevote, BlockHash: nil}
vote := &blk.Vote{Height: height, Round: round, Type: blk.VoteTypePrevote, BlockHash: nil}
for i := 0; i < 6; i++ {
signAddVote(privValidators[i], vote, voteSet)
}
@ -239,35 +239,35 @@ func TestAddCommitsToPrevoteVotes(t *testing.T) {
}
// Attempt to add a commit from val6 at a previous height
vote = &block.Vote{Height: height - 1, Round: round, Type: block.VoteTypeCommit, BlockHash: nil}
vote = &blk.Vote{Height: height - 1, Round: round, Type: blk.VoteTypeCommit, BlockHash: nil}
added, _ := signAddVote(privValidators[6], vote, voteSet)
if added {
t.Errorf("Expected Add() to fail, wrong height.")
}
// Attempt to add a commit from val6 at a later round
vote = &block.Vote{Height: height, Round: round + 1, Type: block.VoteTypeCommit, BlockHash: nil}
vote = &blk.Vote{Height: height, Round: round + 1, Type: blk.VoteTypeCommit, BlockHash: nil}
added, _ = signAddVote(privValidators[6], vote, voteSet)
if added {
t.Errorf("Expected Add() to fail, cannot add future round vote.")
}
// Attempt to add a commit from val6 for currrent height/round.
vote = &block.Vote{Height: height, Round: round, Type: block.VoteTypeCommit, BlockHash: nil}
vote = &blk.Vote{Height: height, Round: round, Type: blk.VoteTypeCommit, BlockHash: nil}
added, err := signAddVote(privValidators[6], vote, voteSet)
if added || err == nil {
t.Errorf("Expected Add() to fail, only prior round commits can be added.")
}
// Add commit from val6 at a previous round
vote = &block.Vote{Height: height, Round: round - 1, Type: block.VoteTypeCommit, BlockHash: nil}
vote = &blk.Vote{Height: height, Round: round - 1, Type: blk.VoteTypeCommit, BlockHash: nil}
added, err = signAddVote(privValidators[6], vote, voteSet)
if !added || err != nil {
t.Errorf("Expected Add() to succeed, commit for prior rounds are relevant.")
}
// Also add commit from val7 for previous round.
vote = &block.Vote{Height: height, Round: round - 2, Type: block.VoteTypeCommit, BlockHash: nil}
vote = &blk.Vote{Height: height, Round: round - 2, Type: blk.VoteTypeCommit, BlockHash: nil}
added, err = signAddVote(privValidators[7], vote, voteSet)
if !added || err != nil {
t.Errorf("Expected Add() to succeed. err: %v", err)
@ -283,10 +283,10 @@ func TestAddCommitsToPrevoteVotes(t *testing.T) {
func TestMakeValidation(t *testing.T) {
height, round := uint(1), uint(0)
voteSet, _, privValidators := randVoteSet(height, round, block.VoteTypeCommit, 10, 1)
blockHash, blockParts := CRandBytes(32), block.PartSetHeader{123, CRandBytes(32)}
voteSet, _, privValidators := randVoteSet(height, round, blk.VoteTypeCommit, 10, 1)
blockHash, blockParts := CRandBytes(32), blk.PartSetHeader{123, CRandBytes(32)}
vote := &block.Vote{Height: height, Round: round, Type: block.VoteTypeCommit,
vote := &blk.Vote{Height: height, Round: round, Type: blk.VoteTypeCommit,
BlockHash: blockHash, BlockParts: blockParts}
// 6 out of 10 voted for some block.
@ -300,7 +300,7 @@ func TestMakeValidation(t *testing.T) {
// 7th voted for some other block.
{
vote := withBlockHash(vote, RandBytes(32))
vote = withBlockParts(vote, block.PartSetHeader{123, RandBytes(32)})
vote = withBlockParts(vote, blk.PartSetHeader{123, RandBytes(32)})
signAddVote(privValidators[6], vote, voteSet)
}


+ 8
- 8
mempool/mempool.go View File

@ -12,14 +12,14 @@ import (
"sync"
"github.com/tendermint/tendermint/binary"
"github.com/tendermint/tendermint/block"
blk "github.com/tendermint/tendermint/block"
sm "github.com/tendermint/tendermint/state"
)
type Mempool struct {
mtx sync.Mutex
state *sm.State
txs []block.Tx
txs []blk.Tx
}
func NewMempool(state *sm.State) *Mempool {
@ -29,7 +29,7 @@ func NewMempool(state *sm.State) *Mempool {
}
// Apply tx to the state and remember it.
func (mem *Mempool) AddTx(tx block.Tx) (err error) {
func (mem *Mempool) AddTx(tx blk.Tx) (err error) {
mem.mtx.Lock()
defer mem.mtx.Unlock()
err = mem.state.ExecTx(tx)
@ -43,7 +43,7 @@ func (mem *Mempool) AddTx(tx block.Tx) (err error) {
}
}
func (mem *Mempool) GetProposalTxs() []block.Tx {
func (mem *Mempool) GetProposalTxs() []blk.Tx {
mem.mtx.Lock()
defer mem.mtx.Unlock()
log.Debug("GetProposalTxs:", "txs", mem.txs)
@ -54,20 +54,20 @@ func (mem *Mempool) GetProposalTxs() []block.Tx {
// "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.Block, state *sm.State) {
func (mem *Mempool) ResetForBlockAndState(block *blk.Block, state *sm.State) {
mem.mtx.Lock()
defer mem.mtx.Unlock()
mem.state = state.Copy()
// First, create a lookup map of txns in new block.
blockTxsMap := make(map[string]struct{})
for _, tx := range block_.Data.Txs {
for _, tx := range block.Data.Txs {
txHash := binary.BinarySha256(tx)
blockTxsMap[string(txHash)] = struct{}{}
}
// Next, filter all txs from mem.txs that are in blockTxsMap
txs := []block.Tx{}
txs := []blk.Tx{}
for _, tx := range mem.txs {
txHash := binary.BinarySha256(tx)
if _, ok := blockTxsMap[string(txHash)]; ok {
@ -80,7 +80,7 @@ func (mem *Mempool) ResetForBlockAndState(block_ *block.Block, state *sm.State)
}
// Next, filter all txs that aren't valid given new state.
validTxs := []block.Tx{}
validTxs := []blk.Tx{}
for _, tx := range txs {
err := mem.state.ExecTx(tx)
if err == nil {


+ 3
- 3
mempool/reactor.go View File

@ -6,7 +6,7 @@ import (
"sync/atomic"
"github.com/tendermint/tendermint/binary"
"github.com/tendermint/tendermint/block"
blk "github.com/tendermint/tendermint/block"
"github.com/tendermint/tendermint/p2p"
)
@ -101,7 +101,7 @@ func (memR *MempoolReactor) Receive(chId byte, src *p2p.Peer, msgBytes []byte) {
}
}
func (memR *MempoolReactor) BroadcastTx(tx block.Tx) error {
func (memR *MempoolReactor) BroadcastTx(tx blk.Tx) error {
err := memR.Mempool.AddTx(tx)
if err != nil {
return err
@ -136,7 +136,7 @@ func DecodeMessage(bz []byte) (msgType byte, msg interface{}, err error) {
//-------------------------------------
type TxMessage struct {
Tx block.Tx
Tx blk.Tx
}
func (m *TxMessage) TypeByte() byte { return msgTypeTx }


+ 3
- 3
rpc/accounts.go View File

@ -5,7 +5,7 @@ import (
"github.com/tendermint/tendermint/account"
"github.com/tendermint/tendermint/binary"
"github.com/tendermint/tendermint/block"
blk "github.com/tendermint/tendermint/block"
. "github.com/tendermint/tendermint/common"
)
@ -68,7 +68,7 @@ func SignSendTxHandler(w http.ResponseWriter, r *http.Request) {
privAccountsStr := GetParam(r, "privAccounts")
var err error
sendTx := binary.ReadJSON(&block.SendTx{}, []byte(sendTxStr), &err).(*block.SendTx)
sendTx := binary.ReadJSON(&blk.SendTx{}, []byte(sendTxStr), &err).(*blk.SendTx)
if err != nil {
WriteAPIResponse(w, API_INVALID_PARAM, Fmt("Invalid sendTx: %v", err))
return
@ -91,6 +91,6 @@ func SignSendTxHandler(w http.ResponseWriter, r *http.Request) {
}
WriteAPIResponse(w, API_OK, struct {
SendTx *block.SendTx
SendTx *blk.SendTx
}{sendTx})
}

+ 7
- 7
rpc/blocks.go View File

@ -3,7 +3,7 @@ package rpc
import (
"net/http"
"github.com/tendermint/tendermint/block"
blk "github.com/tendermint/tendermint/block"
. "github.com/tendermint/tendermint/common"
)
@ -20,7 +20,7 @@ func BlockchainInfoHandler(w http.ResponseWriter, r *http.Request) {
}
log.Debug("BlockchainInfoHandler", "maxHeight", maxHeight, "minHeight", minHeight)
blockMetas := []*block.BlockMeta{}
blockMetas := []*blk.BlockMeta{}
for height := maxHeight; height >= minHeight; height-- {
blockMeta := blockStore.LoadBlockMeta(height)
blockMetas = append(blockMetas, blockMeta)
@ -28,7 +28,7 @@ func BlockchainInfoHandler(w http.ResponseWriter, r *http.Request) {
WriteAPIResponse(w, API_OK, struct {
LastHeight uint
BlockMetas []*block.BlockMeta
BlockMetas []*blk.BlockMeta
}{blockStore.Height(), blockMetas})
}
@ -46,10 +46,10 @@ func GetBlockHandler(w http.ResponseWriter, r *http.Request) {
}
blockMeta := blockStore.LoadBlockMeta(height)
block_ := blockStore.LoadBlock(height)
block := blockStore.LoadBlock(height)
WriteAPIResponse(w, API_OK, struct {
BlockMeta *block.BlockMeta
Block *block.Block
}{blockMeta, block_})
BlockMeta *blk.BlockMeta
Block *blk.Block
}{blockMeta, block})
}

+ 2
- 2
rpc/mempool.go View File

@ -4,14 +4,14 @@ import (
"net/http"
"github.com/tendermint/tendermint/binary"
"github.com/tendermint/tendermint/block"
blk "github.com/tendermint/tendermint/block"
. "github.com/tendermint/tendermint/common"
)
func BroadcastTxHandler(w http.ResponseWriter, r *http.Request) {
txJSON := GetParam(r, "tx")
var err error
var tx block.Tx
var tx blk.Tx
binary.ReadJSON(&tx, []byte(txJSON), &err)
if err != nil {
WriteAPIResponse(w, API_INVALID_PARAM, Fmt("Invalid tx: %v", err))


+ 3
- 3
rpc/rpc.go View File

@ -1,16 +1,16 @@
package rpc
import (
"github.com/tendermint/tendermint/block"
blk "github.com/tendermint/tendermint/block"
"github.com/tendermint/tendermint/consensus"
mempl "github.com/tendermint/tendermint/mempool"
)
var blockStore *block.BlockStore
var blockStore *blk.BlockStore
var consensusState *consensus.ConsensusState
var mempoolReactor *mempl.MempoolReactor
func SetRPCBlockStore(bs *block.BlockStore) {
func SetRPCBlockStore(bs *blk.BlockStore) {
blockStore = bs
}


+ 4
- 4
state/genesis.go View File

@ -6,7 +6,7 @@ import (
"github.com/tendermint/tendermint/account"
"github.com/tendermint/tendermint/binary"
"github.com/tendermint/tendermint/block"
blk "github.com/tendermint/tendermint/block"
. "github.com/tendermint/tendermint/common"
dbm "github.com/tendermint/tendermint/db"
"github.com/tendermint/tendermint/merkle"
@ -79,12 +79,12 @@ func MakeGenesisState(db dbm.DB, genDoc *GenesisDoc) *State {
valInfo := &ValidatorInfo{
Address: address,
PubKey: pubKey,
UnbondTo: make([]*block.TxOutput, len(val.UnbondTo)),
UnbondTo: make([]*blk.TxOutput, len(val.UnbondTo)),
FirstBondHeight: 0,
FirstBondAmount: val.Amount,
}
for i, unbondTo := range val.UnbondTo {
valInfo.UnbondTo[i] = &block.TxOutput{
valInfo.UnbondTo[i] = &blk.TxOutput{
Address: unbondTo.Address,
Amount: unbondTo.Amount,
}
@ -107,7 +107,7 @@ func MakeGenesisState(db dbm.DB, genDoc *GenesisDoc) *State {
DB: db,
LastBlockHeight: 0,
LastBlockHash: nil,
LastBlockParts: block.PartSetHeader{},
LastBlockParts: blk.PartSetHeader{},
LastBlockTime: genDoc.GenesisTime,
BondedValidators: NewValidatorSet(validators),
UnbondingValidators: NewValidatorSet(nil),


+ 8
- 8
state/priv_validator.go View File

@ -11,7 +11,7 @@ import (
"github.com/tendermint/tendermint/account"
"github.com/tendermint/tendermint/binary"
"github.com/tendermint/tendermint/block"
blk "github.com/tendermint/tendermint/block"
. "github.com/tendermint/tendermint/common"
"github.com/tendermint/tendermint/config"
. "github.com/tendermint/tendermint/consensus/types"
@ -27,13 +27,13 @@ const (
stepCommit = 4
)
func voteToStep(vote *block.Vote) uint8 {
func voteToStep(vote *blk.Vote) uint8 {
switch vote.Type {
case block.VoteTypePrevote:
case blk.VoteTypePrevote:
return stepPrevote
case block.VoteTypePrecommit:
case blk.VoteTypePrecommit:
return stepPrecommit
case block.VoteTypeCommit:
case blk.VoteTypeCommit:
return stepCommit
default:
panic("Unknown vote type")
@ -99,7 +99,7 @@ func (privVal *PrivValidator) save() {
}
// TODO: test
func (privVal *PrivValidator) SignVote(vote *block.Vote) error {
func (privVal *PrivValidator) SignVote(vote *blk.Vote) error {
privVal.mtx.Lock()
defer privVal.mtx.Unlock()
@ -134,7 +134,7 @@ func (privVal *PrivValidator) SignVote(vote *block.Vote) error {
return nil
}
func (privVal *PrivValidator) SignVoteUnsafe(vote *block.Vote) {
func (privVal *PrivValidator) SignVoteUnsafe(vote *blk.Vote) {
vote.Signature = privVal.PrivKey.Sign(account.SignBytes(vote)).(account.SignatureEd25519)
}
@ -159,7 +159,7 @@ func (privVal *PrivValidator) SignProposal(proposal *Proposal) error {
}
}
func (privVal *PrivValidator) SignRebondTx(rebondTx *block.RebondTx) error {
func (privVal *PrivValidator) SignRebondTx(rebondTx *blk.RebondTx) error {
privVal.mtx.Lock()
defer privVal.mtx.Unlock()
if privVal.LastHeight < rebondTx.Height {


+ 62
- 62
state/state.go View File

@ -8,7 +8,7 @@ import (
"github.com/tendermint/tendermint/account"
"github.com/tendermint/tendermint/binary"
"github.com/tendermint/tendermint/block"
blk "github.com/tendermint/tendermint/block"
. "github.com/tendermint/tendermint/common"
dbm "github.com/tendermint/tendermint/db"
"github.com/tendermint/tendermint/merkle"
@ -25,7 +25,7 @@ var (
//-----------------------------------------------------------------------------
type InvalidTxError struct {
Tx block.Tx
Tx blk.Tx
Reason error
}
@ -40,7 +40,7 @@ type State struct {
DB dbm.DB
LastBlockHeight uint
LastBlockHash []byte
LastBlockParts block.PartSetHeader
LastBlockParts blk.PartSetHeader
LastBlockTime time.Time
BondedValidators *ValidatorSet
UnbondingValidators *ValidatorSet
@ -57,7 +57,7 @@ func LoadState(db dbm.DB) *State {
r, n, err := bytes.NewReader(buf), new(int64), new(error)
s.LastBlockHeight = binary.ReadUvarint(r, n, err)
s.LastBlockHash = binary.ReadByteSlice(r, n, err)
s.LastBlockParts = binary.ReadBinary(block.PartSetHeader{}, r, n, err).(block.PartSetHeader)
s.LastBlockParts = binary.ReadBinary(blk.PartSetHeader{}, r, n, err).(blk.PartSetHeader)
s.LastBlockTime = binary.ReadTime(r, n, err)
s.BondedValidators = binary.ReadBinary(&ValidatorSet{}, r, n, err).(*ValidatorSet)
s.UnbondingValidators = binary.ReadBinary(&ValidatorSet{}, r, n, err).(*ValidatorSet)
@ -112,24 +112,24 @@ func (s *State) Copy() *State {
// account.PubKey.(type) != PubKeyNil, (it must be known),
// or it must be specified in the TxInput. If redeclared,
// the TxInput is modified and input.PubKey set to PubKeyNil.
func (s *State) GetOrMakeAccounts(ins []*block.TxInput, outs []*block.TxOutput) (map[string]*account.Account, error) {
func (s *State) GetOrMakeAccounts(ins []*blk.TxInput, outs []*blk.TxOutput) (map[string]*account.Account, error) {
accounts := map[string]*account.Account{}
for _, in := range ins {
// Account shouldn't be duplicated
if _, ok := accounts[string(in.Address)]; ok {
return nil, block.ErrTxDuplicateAddress
return nil, blk.ErrTxDuplicateAddress
}
acc := s.GetAccount(in.Address)
if acc == nil {
return nil, block.ErrTxInvalidAddress
return nil, blk.ErrTxInvalidAddress
}
// PubKey should be present in either "account" or "in"
if _, isNil := acc.PubKey.(account.PubKeyNil); isNil {
if _, isNil := in.PubKey.(account.PubKeyNil); isNil {
return nil, block.ErrTxUnknownPubKey
return nil, blk.ErrTxUnknownPubKey
}
if !bytes.Equal(in.PubKey.Address(), acc.Address) {
return nil, block.ErrTxInvalidPubKey
return nil, blk.ErrTxInvalidPubKey
}
acc.PubKey = in.PubKey
} else {
@ -140,7 +140,7 @@ func (s *State) GetOrMakeAccounts(ins []*block.TxInput, outs []*block.TxOutput)
for _, out := range outs {
// Account shouldn't be duplicated
if _, ok := accounts[string(out.Address)]; ok {
return nil, block.ErrTxDuplicateAddress
return nil, blk.ErrTxDuplicateAddress
}
acc := s.GetAccount(out.Address)
// output account may be nil (new)
@ -157,7 +157,7 @@ func (s *State) GetOrMakeAccounts(ins []*block.TxInput, outs []*block.TxOutput)
return accounts, nil
}
func (s *State) ValidateInputs(accounts map[string]*account.Account, signBytes []byte, ins []*block.TxInput) (total uint64, err error) {
func (s *State) ValidateInputs(accounts map[string]*account.Account, signBytes []byte, ins []*blk.TxInput) (total uint64, err error) {
for _, in := range ins {
acc := accounts[string(in.Address)]
if acc == nil {
@ -169,15 +169,15 @@ func (s *State) ValidateInputs(accounts map[string]*account.Account, signBytes [
}
// Check signatures
if !acc.PubKey.VerifyBytes(signBytes, in.Signature) {
return 0, block.ErrTxInvalidSignature
return 0, blk.ErrTxInvalidSignature
}
// Check sequences
if acc.Sequence+1 != in.Sequence {
return 0, block.ErrTxInvalidSequence
return 0, blk.ErrTxInvalidSequence
}
// Check amount
if acc.Balance < in.Amount {
return 0, block.ErrTxInsufficientFunds
return 0, blk.ErrTxInsufficientFunds
}
// Good. Add amount to total
total += in.Amount
@ -185,7 +185,7 @@ func (s *State) ValidateInputs(accounts map[string]*account.Account, signBytes [
return total, nil
}
func (s *State) ValidateOutputs(outs []*block.TxOutput) (total uint64, err error) {
func (s *State) ValidateOutputs(outs []*blk.TxOutput) (total uint64, err error) {
for _, out := range outs {
// Check TxOutput basic
if err := out.ValidateBasic(); err != nil {
@ -197,7 +197,7 @@ func (s *State) ValidateOutputs(outs []*block.TxOutput) (total uint64, err error
return total, nil
}
func (s *State) AdjustByInputs(accounts map[string]*account.Account, ins []*block.TxInput) {
func (s *State) AdjustByInputs(accounts map[string]*account.Account, ins []*blk.TxInput) {
for _, in := range ins {
acc := accounts[string(in.Address)]
if acc == nil {
@ -211,7 +211,7 @@ func (s *State) AdjustByInputs(accounts map[string]*account.Account, ins []*bloc
}
}
func (s *State) AdjustByOutputs(accounts map[string]*account.Account, outs []*block.TxOutput) {
func (s *State) AdjustByOutputs(accounts map[string]*account.Account, outs []*blk.TxOutput) {
for _, out := range outs {
acc := accounts[string(out.Address)]
if acc == nil {
@ -223,15 +223,15 @@ func (s *State) AdjustByOutputs(accounts map[string]*account.Account, outs []*bl
// If the tx is invalid, an error will be returned.
// Unlike AppendBlock(), state will not be altered.
func (s *State) ExecTx(tx_ block.Tx) error {
func (s *State) ExecTx(tx_ blk.Tx) error {
// TODO: do something with fees
fees := uint64(0)
// Exec tx
switch tx_.(type) {
case *block.SendTx:
tx := tx_.(*block.SendTx)
case *blk.SendTx:
tx := tx_.(*blk.SendTx)
accounts, err := s.GetOrMakeAccounts(tx.Inputs, tx.Outputs)
if err != nil {
return err
@ -246,7 +246,7 @@ func (s *State) ExecTx(tx_ block.Tx) error {
return err
}
if outTotal > inTotal {
return block.ErrTxInsufficientFunds
return blk.ErrTxInsufficientFunds
}
fee := inTotal - outTotal
fees += fee
@ -257,8 +257,8 @@ func (s *State) ExecTx(tx_ block.Tx) error {
s.UpdateAccounts(accounts)
return nil
case *block.BondTx:
tx := tx_.(*block.BondTx)
case *blk.BondTx:
tx := tx_.(*blk.BondTx)
valInfo := s.GetValidatorInfo(tx.PubKey.Address())
if valInfo != nil {
// TODO: In the future, check that the validator wasn't destroyed,
@ -282,7 +282,7 @@ func (s *State) ExecTx(tx_ block.Tx) error {
return err
}
if outTotal > inTotal {
return block.ErrTxInsufficientFunds
return blk.ErrTxInsufficientFunds
}
fee := inTotal - outTotal
fees += fee
@ -311,19 +311,19 @@ func (s *State) ExecTx(tx_ block.Tx) error {
}
return nil
case *block.UnbondTx:
tx := tx_.(*block.UnbondTx)
case *blk.UnbondTx:
tx := tx_.(*blk.UnbondTx)
// The validator must be active
_, val := s.BondedValidators.GetByAddress(tx.Address)
if val == nil {
return block.ErrTxInvalidAddress
return blk.ErrTxInvalidAddress
}
// Verify the signature
signBytes := account.SignBytes(tx)
if !val.PubKey.VerifyBytes(signBytes, tx.Signature) {
return block.ErrTxInvalidSignature
return blk.ErrTxInvalidSignature
}
// tx.Height must be greater than val.LastCommitHeight
@ -335,19 +335,19 @@ func (s *State) ExecTx(tx_ block.Tx) error {
s.unbondValidator(val)
return nil
case *block.RebondTx:
tx := tx_.(*block.RebondTx)
case *blk.RebondTx:
tx := tx_.(*blk.RebondTx)
// The validator must be inactive
_, val := s.UnbondingValidators.GetByAddress(tx.Address)
if val == nil {
return block.ErrTxInvalidAddress
return blk.ErrTxInvalidAddress
}
// Verify the signature
signBytes := account.SignBytes(tx)
if !val.PubKey.VerifyBytes(signBytes, tx.Signature) {
return block.ErrTxInvalidSignature
return blk.ErrTxInvalidSignature
}
// tx.Height must be equal to the next height
@ -359,8 +359,8 @@ func (s *State) ExecTx(tx_ block.Tx) error {
s.rebondValidator(val)
return nil
case *block.DupeoutTx:
tx := tx_.(*block.DupeoutTx)
case *blk.DupeoutTx:
tx := tx_.(*blk.DupeoutTx)
// Verify the signatures
_, accused := s.BondedValidators.GetByAddress(tx.Address)
@ -368,7 +368,7 @@ func (s *State) ExecTx(tx_ block.Tx) error {
voteBSignBytes := account.SignBytes(&tx.VoteB)
if !accused.PubKey.VerifyBytes(voteASignBytes, tx.VoteA.Signature) ||
!accused.PubKey.VerifyBytes(voteBSignBytes, tx.VoteB.Signature) {
return block.ErrTxInvalidSignature
return blk.ErrTxInvalidSignature
}
// Verify equivocation
@ -377,7 +377,7 @@ func (s *State) ExecTx(tx_ block.Tx) error {
if tx.VoteA.Height != tx.VoteB.Height {
return errors.New("DupeoutTx heights don't match")
}
if tx.VoteA.Type == block.VoteTypeCommit && tx.VoteA.Round < tx.VoteB.Round {
if tx.VoteA.Type == blk.VoteTypeCommit && tx.VoteA.Round < tx.VoteB.Round {
// Check special case.
// Validators should not sign another vote after committing.
} else {
@ -473,38 +473,38 @@ func (s *State) destroyValidator(val *Validator) {
}
// "checkStateHash": If false, instead of checking the resulting
// state.Hash() against block.StateHash, it *sets* the block.StateHash.
// state.Hash() against blk.StateHash, it *sets* the blk.StateHash.
// (used for constructing a new proposal)
// NOTE: If an error occurs during block execution, state will be left
// at an invalid state. Copy the state before calling AppendBlock!
func (s *State) AppendBlock(block_ *block.Block, blockPartsHeader block.PartSetHeader, checkStateHash bool) error {
func (s *State) AppendBlock(block *blk.Block, blockPartsHeader blk.PartSetHeader, checkStateHash bool) error {
// Basic block validation.
err := block_.ValidateBasic(s.LastBlockHeight, s.LastBlockHash, s.LastBlockParts, s.LastBlockTime)
err := block.ValidateBasic(s.LastBlockHeight, s.LastBlockHash, s.LastBlockParts, s.LastBlockTime)
if err != nil {
return err
}
// Validate block Validation.
if block_.Height == 1 {
if len(block_.Validation.Commits) != 0 {
if block.Height == 1 {
if len(block.Validation.Commits) != 0 {
return errors.New("Block at height 1 (first block) should have no Validation commits")
}
} else {
if uint(len(block_.Validation.Commits)) != s.BondedValidators.Size() {
if uint(len(block.Validation.Commits)) != s.BondedValidators.Size() {
return errors.New("Invalid block validation size")
}
var sumVotingPower uint64
s.BondedValidators.Iterate(func(index uint, val *Validator) bool {
commit := block_.Validation.Commits[index]
commit := block.Validation.Commits[index]
if commit.IsZero() {
return false
} else {
vote := &block.Vote{
Height: block_.Height - 1,
vote := &blk.Vote{
Height: block.Height - 1,
Round: commit.Round,
Type: block.VoteTypeCommit,
BlockHash: block_.LastBlockHash,
BlockParts: block_.LastBlockParts,
Type: blk.VoteTypeCommit,
BlockHash: block.LastBlockHash,
BlockParts: block.LastBlockParts,
}
if val.PubKey.VerifyBytes(account.SignBytes(vote), commit.Signature) {
sumVotingPower += val.VotingPower
@ -525,7 +525,7 @@ func (s *State) AppendBlock(block_ *block.Block, blockPartsHeader block.PartSetH
}
// Commit each tx
for _, tx := range block_.Data.Txs {
for _, tx := range block.Data.Txs {
err := s.ExecTx(tx)
if err != nil {
return InvalidTxError{tx, err}
@ -533,7 +533,7 @@ func (s *State) AppendBlock(block_ *block.Block, blockPartsHeader block.PartSetH
}
// Update Validator.LastCommitHeight as necessary.
for i, commit := range block_.Validation.Commits {
for i, commit := range block.Validation.Commits {
if commit.IsZero() {
continue
}
@ -541,7 +541,7 @@ func (s *State) AppendBlock(block_ *block.Block, blockPartsHeader block.PartSetH
if val == nil {
panic(Fmt("Failed to fetch validator at index %v", i))
}
val.LastCommitHeight = block_.Height - 1
val.LastCommitHeight = block.Height - 1
updated := s.BondedValidators.Update(val)
if !updated {
panic("Failed to update validator LastCommitHeight")
@ -552,7 +552,7 @@ func (s *State) AppendBlock(block_ *block.Block, blockPartsHeader block.PartSetH
// reward account with bonded coins.
toRelease := []*Validator{}
s.UnbondingValidators.Iterate(func(index uint, val *Validator) bool {
if val.UnbondHeight+unbondingPeriodBlocks < block_.Height {
if val.UnbondHeight+unbondingPeriodBlocks < block.Height {
toRelease = append(toRelease, val)
}
return false
@ -565,7 +565,7 @@ func (s *State) AppendBlock(block_ *block.Block, blockPartsHeader block.PartSetH
// unbond them, they have timed out.
toTimeout := []*Validator{}
s.BondedValidators.Iterate(func(index uint, val *Validator) bool {
if val.LastCommitHeight+validatorTimeoutBlocks < block_.Height {
if val.LastCommitHeight+validatorTimeoutBlocks < block.Height {
toTimeout = append(toTimeout, val)
}
return false
@ -577,26 +577,26 @@ func (s *State) AppendBlock(block_ *block.Block, blockPartsHeader block.PartSetH
// Increment validator AccumPowers
s.BondedValidators.IncrementAccum(1)
// Check or set block.StateHash
// Check or set blk.StateHash
stateHash := s.Hash()
if checkStateHash {
// State hash should match
if !bytes.Equal(stateHash, block_.StateHash) {
if !bytes.Equal(stateHash, block.StateHash) {
return Errorf("Invalid state hash. Got %X, block says %X",
stateHash, block_.StateHash)
stateHash, block.StateHash)
}
} else {
// Set the state hash.
if block_.StateHash != nil {
panic("Cannot overwrite block_.StateHash")
if block.StateHash != nil {
panic("Cannot overwrite block.StateHash")
}
block_.StateHash = stateHash
block.StateHash = stateHash
}
s.LastBlockHeight = block_.Height
s.LastBlockHash = block_.Hash()
s.LastBlockHeight = block.Height
s.LastBlockHash = block.Hash()
s.LastBlockParts = blockPartsHeader
s.LastBlockTime = block_.Time
s.LastBlockTime = block.Time
return nil
}


+ 2
- 2
state/test.go View File

@ -5,7 +5,7 @@ import (
"sort"
"github.com/tendermint/tendermint/account"
"github.com/tendermint/tendermint/block"
blk "github.com/tendermint/tendermint/block"
. "github.com/tendermint/tendermint/common"
dbm "github.com/tendermint/tendermint/db"
@ -46,7 +46,7 @@ func RandValidator(randBonded bool, minBonded uint64) (*ValidatorInfo, *PrivVali
valInfo := &ValidatorInfo{
Address: privVal.Address,
PubKey: privVal.PubKey,
UnbondTo: []*block.TxOutput{&block.TxOutput{
UnbondTo: []*blk.TxOutput{&blk.TxOutput{
Amount: bonded,
Address: privVal.Address,
}},


+ 2
- 2
state/validator.go View File

@ -7,14 +7,14 @@ import (
"github.com/tendermint/tendermint/account"
"github.com/tendermint/tendermint/binary"
"github.com/tendermint/tendermint/block"
blk "github.com/tendermint/tendermint/block"
)
// Persistent (mostly) static data for each Validator
type ValidatorInfo struct {
Address []byte
PubKey account.PubKeyEd25519
UnbondTo []*block.TxOutput
UnbondTo []*blk.TxOutput
FirstBondHeight uint
FirstBondAmount uint64


Loading…
Cancel
Save