Browse Source

Merge pull request #40 from tendermint/blockpool

Blockpool
pull/43/merge
Jae Kwon 9 years ago
parent
commit
9266f8e27c
35 changed files with 1077 additions and 341 deletions
  1. +7
    -0
      blockchain/log.go
  2. +354
    -0
      blockchain/pool.go
  3. +112
    -0
      blockchain/pool_test.go
  4. +248
    -0
      blockchain/store.go
  5. +6
    -6
      cmd/gen_tx.go
  6. +8
    -8
      consensus/pol.go
  7. +23
    -23
      consensus/pol_test.go
  8. +33
    -33
      consensus/reactor.go
  9. +54
    -54
      consensus/state.go
  10. +15
    -15
      consensus/state_test.go
  11. +2
    -2
      consensus/test.go
  12. +4
    -4
      consensus/types/proposal.go
  13. +26
    -26
      consensus/vote_set.go
  14. +30
    -30
      consensus/vote_set_test.go
  15. +3
    -3
      daemon/daemon.go
  16. +7
    -7
      mempool/mempool.go
  17. +3
    -3
      mempool/reactor.go
  18. +5
    -5
      rpc/blocks.go
  19. +3
    -3
      rpc/mempool.go
  20. +3
    -3
      rpc/rpc.go
  21. +11
    -11
      rpc/txs.go
  22. +4
    -4
      state/genesis.go
  23. +8
    -8
      state/priv_validator.go
  24. +46
    -46
      state/state.go
  25. +35
    -35
      state/state_test.go
  26. +2
    -2
      state/test.go
  27. +2
    -2
      state/validator.go
  28. +0
    -0
      types/README.md
  29. +16
    -1
      types/block.go
  30. +2
    -2
      types/log.go
  31. +1
    -1
      types/part_set.go
  32. +1
    -1
      types/part_set_test.go
  33. +1
    -1
      types/store.go
  34. +1
    -1
      types/tx.go
  35. +1
    -1
      types/vote.go

+ 7
- 0
blockchain/log.go View File

@ -0,0 +1,7 @@
package blockchain
import (
"github.com/tendermint/tendermint/logger"
)
var log = logger.New("module", "blockchain")

+ 354
- 0
blockchain/pool.go View File

@ -0,0 +1,354 @@
package blockchain
import (
"math/rand"
"sync/atomic"
"time"
. "github.com/tendermint/tendermint/common"
"github.com/tendermint/tendermint/types"
)
const (
maxOutstandingRequestsPerPeer = 10
eventsChannelCapacity = 100
requestTimeoutSeconds = 10
maxTries = 3
requestIntervalMS = 500
requestBatchSize = 50
maxPendingRequests = 50
maxTotalRequests = 100
maxPeersPerRequest = 1
)
type BlockRequest struct {
Height uint
PeerId string
}
type BlockPool struct {
peers map[string]*bpPeer
blockInfos map[uint]*bpBlockInfo
height uint // the lowest key in blockInfos.
started int32 // atomic
stopped int32 // atomic
numPending int32
numTotal int32
eventsCh chan interface{} // internal events.
requestsCh chan<- BlockRequest // output of new requests to make.
timeoutsCh chan<- string // output of peers that timed out.
blocksCh chan<- *types.Block // output of ordered blocks.
repeater *RepeatTimer // for requesting more bocks.
quit chan struct{}
}
func NewBlockPool(start uint, timeoutsCh chan<- string, requestsCh chan<- BlockRequest, blocksCh chan<- *types.Block) *BlockPool {
return &BlockPool{
peers: make(map[string]*bpPeer),
blockInfos: make(map[uint]*bpBlockInfo),
height: start,
started: 0,
stopped: 0,
numPending: 0,
numTotal: 0,
quit: make(chan struct{}),
eventsCh: make(chan interface{}, eventsChannelCapacity),
requestsCh: requestsCh,
timeoutsCh: timeoutsCh,
blocksCh: blocksCh,
repeater: NewRepeatTimer("", requestIntervalMS*time.Millisecond),
}
}
func (bp *BlockPool) Start() {
if atomic.CompareAndSwapInt32(&bp.started, 0, 1) {
log.Info("Starting BlockPool")
go bp.run()
}
}
func (bp *BlockPool) Stop() {
if atomic.CompareAndSwapInt32(&bp.stopped, 0, 1) {
log.Info("Stopping BlockPool")
close(bp.quit)
close(bp.eventsCh)
close(bp.requestsCh)
close(bp.timeoutsCh)
close(bp.blocksCh)
bp.repeater.Stop()
}
}
// AddBlock should be called when a block is received.
func (bp *BlockPool) AddBlock(block *types.Block, peerId string) {
bp.eventsCh <- bpBlockResponse{block, peerId}
}
func (bp *BlockPool) SetPeerStatus(peerId string, height uint) {
bp.eventsCh <- bpPeerStatus{peerId, height}
}
// Runs in a goroutine and processes messages.
func (bp *BlockPool) run() {
FOR_LOOP:
for {
select {
case msg := <-bp.eventsCh:
bp.handleEvent(msg)
case <-bp.repeater.Ch:
bp.makeMoreBlockInfos()
bp.requestBlocksFromRandomPeers(10)
case <-bp.quit:
break FOR_LOOP
}
}
}
func (bp *BlockPool) handleEvent(event_ interface{}) {
switch event := event_.(type) {
case bpBlockResponse:
peer := bp.peers[event.peerId]
blockInfo := bp.blockInfos[event.block.Height]
if blockInfo == nil {
// block was unwanted.
if peer != nil {
peer.bad++
}
} else {
// block was wanted.
if peer != nil {
peer.good++
}
delete(peer.requests, event.block.Height)
if blockInfo.block == nil {
// peer is the first to give it to us.
blockInfo.block = event.block
blockInfo.blockBy = peer.id
bp.numPending--
if event.block.Height == bp.height {
go bp.pushBlocksFromStart()
}
}
}
case bpPeerStatus: // updated or new status from peer
// request blocks if possible.
peer := bp.peers[event.peerId]
if peer == nil {
peer = bpNewPeer(event.peerId, event.height)
bp.peers[peer.id] = peer
}
bp.requestBlocksFromPeer(peer)
case bpRequestTimeout: // unconditional timeout for each peer's request.
peer := bp.peers[event.peerId]
if peer == nil {
// cleanup was already handled.
return
}
height := event.height
request := peer.requests[height]
if request == nil || request.block != nil {
// the request was fulfilled by some peer or this peer.
return
}
// A request for peer timed out.
peer.bad++
if request.tries < maxTries {
log.Warn("Timeout: Trying again.", "tries", request.tries, "peerId", peer.id)
// try again.
select {
case bp.requestsCh <- BlockRequest{height, peer.id}:
request.startAndTimeoutTo(bp.eventsCh) // also bumps request.tries
default:
// The request cannot be made because requestCh is full.
// Just delete the request.
delete(peer.requests, height)
}
} else {
log.Warn("Timeout: Deleting request")
// delete the request.
delete(peer.requests, height)
blockInfo := bp.blockInfos[height]
if blockInfo != nil {
delete(blockInfo.requests, peer.id)
}
select {
case bp.timeoutsCh <- peer.id:
default:
}
}
}
}
// NOTE: This function is sufficient, but we should find pending blocks
// and sample the peers in one go rather than the current O(n^2) impl.
func (bp *BlockPool) requestBlocksFromRandomPeers(maxPeers int) {
chosen := bp.pickAvailablePeers(maxPeers)
log.Debug("requestBlocksFromRandomPeers", "chosen", len(chosen))
for _, peer := range chosen {
bp.requestBlocksFromPeer(peer)
}
}
func (bp *BlockPool) requestBlocksFromPeer(peer *bpPeer) {
// If peer is available and can provide something...
for height := bp.height; peer.available(); height++ {
blockInfo := bp.blockInfos[height]
if blockInfo == nil {
// We're out of range.
return
}
needsMorePeers := blockInfo.needsMorePeers()
alreadyAskedPeer := blockInfo.requests[peer.id] != nil
if needsMorePeers && !alreadyAskedPeer {
select {
case bp.requestsCh <- BlockRequest{height, peer.id}:
// Create a new request and start the timer.
request := &bpBlockRequest{
height: height,
peer: peer,
}
blockInfo.requests[peer.id] = request
peer.requests[height] = request
request.startAndTimeoutTo(bp.eventsCh) // also bumps request.tries
default:
// The request cannot be made because requestCh is full.
// Just stop.
return
}
}
}
}
func (bp *BlockPool) makeMoreBlockInfos() {
// make more requests if necessary.
for i := 0; i < requestBatchSize; i++ {
//log.Debug("Confused?",
// "numPending", bp.numPending, "maxPendingRequests", maxPendingRequests, "numtotal", bp.numTotal, "maxTotalRequests", maxTotalRequests)
if bp.numPending < maxPendingRequests && bp.numTotal < maxTotalRequests {
// Make a request for the next block height
requestHeight := bp.height + uint(bp.numTotal)
log.Debug("New blockInfo", "height", requestHeight)
blockInfo := bpNewBlockInfo(requestHeight)
bp.blockInfos[requestHeight] = blockInfo
bp.numPending++
bp.numTotal++
} else {
break
}
}
}
func (bp *BlockPool) pickAvailablePeers(choose int) []*bpPeer {
available := []*bpPeer{}
for _, peer := range bp.peers {
if peer.available() {
available = append(available, peer)
}
}
perm := rand.Perm(MinInt(choose, len(available)))
chosen := make([]*bpPeer, len(perm))
for i, idx := range perm {
chosen[i] = available[idx]
}
return chosen
}
// blocking
func (bp *BlockPool) pushBlocksFromStart() {
for height := bp.height; ; height++ {
// push block to blocksCh.
blockInfo := bp.blockInfos[height]
if blockInfo == nil || blockInfo.block == nil {
break
}
bp.numTotal--
bp.height++
delete(bp.blockInfos, height)
bp.blocksCh <- blockInfo.block
}
}
//-----------------------------------------------------------------------------
type bpBlockInfo struct {
height uint
requests map[string]*bpBlockRequest
block *types.Block // first block received
blockBy string // peerId of source
}
func bpNewBlockInfo(height uint) *bpBlockInfo {
return &bpBlockInfo{
height: height,
requests: make(map[string]*bpBlockRequest),
}
}
func (blockInfo *bpBlockInfo) needsMorePeers() bool {
return len(blockInfo.requests) < maxPeersPerRequest
}
//-------------------------------------
type bpBlockRequest struct {
peer *bpPeer
height uint
block *types.Block
tries int
}
// bump tries++ and set timeout.
// NOTE: the timer is unconditional.
func (request *bpBlockRequest) startAndTimeoutTo(eventsCh chan<- interface{}) {
request.tries++
time.AfterFunc(requestTimeoutSeconds*time.Second, func() {
eventsCh <- bpRequestTimeout{
peerId: request.peer.id,
height: request.height,
}
})
}
//-------------------------------------
type bpPeer struct {
id string
height uint
requests map[uint]*bpBlockRequest
// Count good/bad events from peer.
good uint
bad uint
}
func bpNewPeer(peerId string, height uint) *bpPeer {
return &bpPeer{
id: peerId,
height: height,
requests: make(map[uint]*bpBlockRequest),
}
}
func (peer *bpPeer) available() bool {
return len(peer.requests) < maxOutstandingRequestsPerPeer
}
//-------------------------------------
// bp.eventsCh messages
type bpBlockResponse struct {
block *types.Block
peerId string
}
type bpPeerStatus struct {
peerId string
height uint // blockchain tip of peer
}
type bpRequestTimeout struct {
peerId string
height uint
}

+ 112
- 0
blockchain/pool_test.go View File

@ -0,0 +1,112 @@
package blockchain
import (
"math/rand"
"testing"
. "github.com/tendermint/tendermint/common"
"github.com/tendermint/tendermint/types"
)
type testPeer struct {
id string
height uint
}
func makePeers(numPeers int, minHeight, maxHeight uint) map[string]testPeer {
peers := make(map[string]testPeer, numPeers)
for i := 0; i < numPeers; i++ {
peerId := RandStr(12)
height := minHeight + uint(rand.Intn(int(maxHeight-minHeight)))
peers[peerId] = testPeer{peerId, height}
}
return peers
}
func TestBasic(t *testing.T) {
// 100 peers anywhere at height 0 to 1000.
peers := makePeers(100, 0, 1000)
start := uint(42)
maxHeight := uint(300)
timeoutsCh := make(chan string, 100)
requestsCh := make(chan BlockRequest, 100)
blocksCh := make(chan *types.Block, 100)
pool := NewBlockPool(start, timeoutsCh, requestsCh, blocksCh)
pool.Start()
// Introduce each peer.
go func() {
for _, peer := range peers {
pool.SetPeerStatus(peer.id, peer.height)
}
}()
lastSeenBlock := uint(41)
// Pull from channels
for {
select {
case peerId := <-timeoutsCh:
t.Errorf("timeout: %v", peerId)
case request := <-requestsCh:
log.Debug("TEST: Pulled new BlockRequest", "request", request)
// After a while, pretend like we got a block from the peer.
go func() {
block := &types.Block{Header: &types.Header{Height: request.Height}}
pool.AddBlock(block, request.PeerId)
log.Debug("TEST: Added block", "block", request.Height, "peer", request.PeerId)
}()
case block := <-blocksCh:
log.Debug("TEST: Pulled new Block", "height", block.Height)
if block.Height != lastSeenBlock+1 {
t.Fatalf("Wrong order of blocks seen. Expected: %v Got: %v", lastSeenBlock+1, block.Height)
}
lastSeenBlock++
if block.Height == maxHeight {
return // Done!
}
}
}
pool.Stop()
}
func TestTimeout(t *testing.T) {
peers := makePeers(100, 0, 1000)
start := uint(42)
timeoutsCh := make(chan string, 10)
requestsCh := make(chan BlockRequest, 10)
blocksCh := make(chan *types.Block, 100)
pool := NewBlockPool(start, timeoutsCh, requestsCh, blocksCh)
pool.Start()
// Introduce each peer.
go func() {
for _, peer := range peers {
pool.SetPeerStatus(peer.id, peer.height)
}
}()
// Pull from channels
for {
select {
case peerId := <-timeoutsCh:
// Timed out. Done!
if peers[peerId].id != peerId {
t.Errorf("Unexpected peer from timeoutsCh")
}
return
case _ = <-requestsCh:
// Don't do anything, let it time out.
case _ = <-blocksCh:
t.Errorf("Got block when none expected")
return
}
}
pool.Stop()
}

+ 248
- 0
blockchain/store.go View File

@ -0,0 +1,248 @@
package blockchain
import (
"bytes"
"encoding/json"
"fmt"
"io"
"github.com/tendermint/tendermint/binary"
. "github.com/tendermint/tendermint/common"
dbm "github.com/tendermint/tendermint/db"
"github.com/tendermint/tendermint/types"
)
/*
Simple low level store for blocks.
There are three types of information stored:
- BlockMeta: Meta information about each block
- Block part: Parts of each block, aggregated w/ PartSet
- Validation: The Validation part of each block, for gossiping commit votes
Currently the commit signatures are duplicated in the Block parts as
well as the Validation. In the future this may change, perhaps by moving
the Validation data outside the Block.
*/
type BlockStore struct {
height uint
db dbm.DB
}
func NewBlockStore(db dbm.DB) *BlockStore {
bsjson := LoadBlockStoreStateJSON(db)
return &BlockStore{
height: bsjson.Height,
db: db,
}
}
// Height() returns the last known contiguous block height.
func (bs *BlockStore) Height() uint {
return bs.height
}
func (bs *BlockStore) GetReader(key []byte) io.Reader {
bytez := bs.db.Get(key)
if bytez == nil {
return nil
}
return bytes.NewReader(bytez)
}
func (bs *BlockStore) LoadBlock(height uint) *types.Block {
var n int64
var err error
r := bs.GetReader(calcBlockMetaKey(height))
if r == nil {
panic(Fmt("Block does not exist at height %v", height))
}
meta := binary.ReadBinary(&BlockMeta{}, r, &n, &err).(*BlockMeta)
if err != nil {
panic(Fmt("Error reading block meta: %v", err))
}
bytez := []byte{}
for i := uint(0); i < meta.Parts.Total; i++ {
part := bs.LoadBlockPart(height, i)
bytez = append(bytez, part.Bytes...)
}
block := binary.ReadBinary(&types.Block{}, bytes.NewReader(bytez), &n, &err).(*types.Block)
if err != nil {
panic(Fmt("Error reading block: %v", err))
}
return block
}
func (bs *BlockStore) LoadBlockPart(height uint, index uint) *types.Part {
var n int64
var err error
r := bs.GetReader(calcBlockPartKey(height, index))
if r == nil {
panic(Fmt("BlockPart does not exist for height %v index %v", height, index))
}
part := binary.ReadBinary(&types.Part{}, r, &n, &err).(*types.Part)
if err != nil {
panic(Fmt("Error reading block part: %v", err))
}
return part
}
func (bs *BlockStore) LoadBlockMeta(height uint) *BlockMeta {
var n int64
var err error
r := bs.GetReader(calcBlockMetaKey(height))
if r == nil {
panic(Fmt("BlockMeta does not exist for height %v", height))
}
meta := binary.ReadBinary(&BlockMeta{}, r, &n, &err).(*BlockMeta)
if err != nil {
panic(Fmt("Error reading block meta: %v", err))
}
return meta
}
// NOTE: the Commit-vote heights are for the block at `height-1`
// Since these are included in the subsequent block, the height
// is off by 1.
func (bs *BlockStore) LoadBlockValidation(height uint) *types.Validation {
var n int64
var err error
r := bs.GetReader(calcBlockValidationKey(height))
if r == nil {
panic(Fmt("BlockValidation does not exist for height %v", height))
}
validation := binary.ReadBinary(&types.Validation{}, r, &n, &err).(*types.Validation)
if err != nil {
panic(Fmt("Error reading validation: %v", err))
}
return validation
}
// NOTE: the Commit-vote heights are for the block at `height`
func (bs *BlockStore) LoadSeenValidation(height uint) *types.Validation {
var n int64
var err error
r := bs.GetReader(calcSeenValidationKey(height))
if r == nil {
panic(Fmt("SeenValidation does not exist for height %v", height))
}
validation := binary.ReadBinary(&types.Validation{}, r, &n, &err).(*types.Validation)
if err != nil {
panic(Fmt("Error reading validation: %v", err))
}
return validation
}
// blockParts: Must be parts of the block
// seenValidation: The +2/3 commits that were seen which finalized the height.
// If all the nodes restart after committing a block,
// we need this to reload the commits to catch-up nodes to the
// most recent height. Otherwise they'd stall at H-1.
// Also good to have to debug consensus issues & punish wrong-signers
// whose commits weren't included in the block.
func (bs *BlockStore) SaveBlock(block *types.Block, blockParts *types.PartSet, seenValidation *types.Validation) {
height := block.Height
if height != bs.height+1 {
panic(Fmt("BlockStore can only save contiguous blocks. Wanted %v, got %v", bs.height+1, height))
}
if !blockParts.IsComplete() {
panic(Fmt("BlockStore can only save complete block part sets"))
}
// Save block meta
meta := makeBlockMeta(block, blockParts)
metaBytes := binary.BinaryBytes(meta)
bs.db.Set(calcBlockMetaKey(height), metaBytes)
// Save block parts
for i := uint(0); i < blockParts.Total(); i++ {
bs.saveBlockPart(height, i, blockParts.GetPart(i))
}
// Save block validation (duplicate and separate from the Block)
blockValidationBytes := binary.BinaryBytes(block.Validation)
bs.db.Set(calcBlockValidationKey(height), blockValidationBytes)
// Save seen validation (seen +2/3 commits)
seenValidationBytes := binary.BinaryBytes(seenValidation)
bs.db.Set(calcSeenValidationKey(height), seenValidationBytes)
// Save new BlockStoreStateJSON descriptor
BlockStoreStateJSON{Height: height}.Save(bs.db)
// Done!
bs.height = height
}
func (bs *BlockStore) saveBlockPart(height uint, index uint, part *types.Part) {
if height != bs.height+1 {
panic(Fmt("BlockStore can only save contiguous blocks. Wanted %v, got %v", bs.height+1, height))
}
partBytes := binary.BinaryBytes(part)
bs.db.Set(calcBlockPartKey(height, index), partBytes)
}
//-----------------------------------------------------------------------------
type BlockMeta struct {
Hash []byte // The block hash
Header *types.Header // The block's Header
Parts types.PartSetHeader // The PartSetHeader, for transfer
}
func makeBlockMeta(block *types.Block, blockParts *types.PartSet) *BlockMeta {
return &BlockMeta{
Hash: block.Hash(),
Header: block.Header,
Parts: blockParts.Header(),
}
}
//-----------------------------------------------------------------------------
func calcBlockMetaKey(height uint) []byte {
return []byte(fmt.Sprintf("H:%v", height))
}
func calcBlockPartKey(height uint, partIndex uint) []byte {
return []byte(fmt.Sprintf("P:%v:%v", height, partIndex))
}
func calcBlockValidationKey(height uint) []byte {
return []byte(fmt.Sprintf("V:%v", height))
}
func calcSeenValidationKey(height uint) []byte {
return []byte(fmt.Sprintf("SV:%v", height))
}
//-----------------------------------------------------------------------------
var blockStoreKey = []byte("blockStore")
type BlockStoreStateJSON struct {
Height uint
}
func (bsj BlockStoreStateJSON) Save(db dbm.DB) {
bytes, err := json.Marshal(bsj)
if err != nil {
panic(Fmt("Could not marshal state bytes: %v", err))
}
db.Set(blockStoreKey, bytes)
}
func LoadBlockStoreStateJSON(db dbm.DB) BlockStoreStateJSON {
bytes := db.Get(blockStoreKey)
if bytes == nil {
return BlockStoreStateJSON{
Height: 0,
}
}
bsj := BlockStoreStateJSON{}
err := json.Unmarshal(bytes, &bsj)
if err != nil {
panic(Fmt("Could not unmarshal bytes: %X", bytes))
}
return bsj
}

+ 6
- 6
cmd/gen_tx.go View File

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


+ 8
- 8
consensus/pol.go View File

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


+ 23
- 23
consensus/pol_test.go View File

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


+ 33
- 33
consensus/reactor.go View File

@ -9,11 +9,11 @@ import (
"time" "time"
"github.com/tendermint/tendermint/binary" "github.com/tendermint/tendermint/binary"
blk "github.com/tendermint/tendermint/block"
. "github.com/tendermint/tendermint/common" . "github.com/tendermint/tendermint/common"
. "github.com/tendermint/tendermint/consensus/types" . "github.com/tendermint/tendermint/consensus/types"
"github.com/tendermint/tendermint/p2p" "github.com/tendermint/tendermint/p2p"
sm "github.com/tendermint/tendermint/state" sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/types"
) )
const ( const (
@ -34,11 +34,11 @@ type ConsensusReactor struct {
stopped uint32 stopped uint32
quit chan struct{} quit chan struct{}
blockStore *blk.BlockStore
blockStore *types.BlockStore
conS *ConsensusState conS *ConsensusState
} }
func NewConsensusReactor(consensusState *ConsensusState, blockStore *blk.BlockStore) *ConsensusReactor {
func NewConsensusReactor(consensusState *ConsensusState, blockStore *types.BlockStore) *ConsensusReactor {
conR := &ConsensusReactor{ conR := &ConsensusReactor{
blockStore: blockStore, blockStore: blockStore,
quit: make(chan struct{}), quit: make(chan struct{}),
@ -167,9 +167,9 @@ func (conR *ConsensusReactor) Receive(chId byte, peer *p2p.Peer, msgBytes []byte
added, index, err := conR.conS.AddVote(address, vote) added, index, err := conR.conS.AddVote(address, vote)
if err != nil { if err != nil {
// If conflicting sig, broadcast evidence tx for slashing. Else punish peer. // If conflicting sig, broadcast evidence tx for slashing. Else punish peer.
if errDupe, ok := err.(*blk.ErrVoteConflictingSignature); ok {
if errDupe, ok := err.(*types.ErrVoteConflictingSignature); ok {
log.Warn("Found conflicting vote. Publish evidence") log.Warn("Found conflicting vote. Publish evidence")
evidenceTx := &blk.DupeoutTx{
evidenceTx := &types.DupeoutTx{
Address: address, Address: address,
VoteA: *errDupe.VoteA, VoteA: *errDupe.VoteA,
VoteB: *errDupe.VoteB, VoteB: *errDupe.VoteB,
@ -404,7 +404,7 @@ OUTER_LOOP:
return false return false
} }
trySendCommitFromValidation := func(blockMeta *blk.BlockMeta, validation *blk.Validation, peerVoteSet BitArray) (sent bool) {
trySendCommitFromValidation := func(blockMeta *types.BlockMeta, validation *types.Validation, peerVoteSet BitArray) (sent bool) {
// Initialize Commits if needed // Initialize Commits if needed
ps.EnsureVoteBitArrays(prs.Height, uint(len(validation.Commits))) ps.EnsureVoteBitArrays(prs.Height, uint(len(validation.Commits)))
@ -412,10 +412,10 @@ OUTER_LOOP:
commit := validation.Commits[index] commit := validation.Commits[index]
log.Debug("Picked commit to send", "index", index, "commit", commit) log.Debug("Picked commit to send", "index", index, "commit", commit)
// Reconstruct vote. // Reconstruct vote.
vote := &blk.Vote{
vote := &types.Vote{
Height: prs.Height, Height: prs.Height,
Round: commit.Round, Round: commit.Round,
Type: blk.VoteTypeCommit,
Type: types.VoteTypeCommit,
BlockHash: blockMeta.Hash, BlockHash: blockMeta.Hash,
BlockParts: blockMeta.Parts, BlockParts: blockMeta.Parts,
Signature: commit.Signature, Signature: commit.Signature,
@ -515,20 +515,20 @@ OUTER_LOOP:
// Read only when returned by PeerState.GetRoundState(). // Read only when returned by PeerState.GetRoundState().
type PeerRoundState struct { 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 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
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 types.PartSetHeader //
ProposalBlockBitArray BitArray // True bit -> has part
ProposalPOLParts types.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
} }
//----------------------------------------------------------------------------- //-----------------------------------------------------------------------------
@ -616,7 +616,7 @@ func (ps *PeerState) EnsureVoteBitArrays(height uint, numValidators uint) {
} }
} }
func (ps *PeerState) SetHasVote(vote *blk.Vote, index uint) {
func (ps *PeerState) SetHasVote(vote *types.Vote, index uint) {
ps.mtx.Lock() ps.mtx.Lock()
defer ps.mtx.Unlock() defer ps.mtx.Unlock()
@ -624,7 +624,7 @@ func (ps *PeerState) SetHasVote(vote *blk.Vote, index uint) {
} }
func (ps *PeerState) setHasVote(height uint, round uint, type_ byte, index uint) { func (ps *PeerState) setHasVote(height uint, round uint, type_ byte, index uint) {
if ps.Height == height+1 && type_ == blk.VoteTypeCommit {
if ps.Height == height+1 && type_ == types.VoteTypeCommit {
// Special case for LastCommits. // Special case for LastCommits.
ps.LastCommits.SetIndex(index, true) ps.LastCommits.SetIndex(index, true)
return return
@ -634,11 +634,11 @@ func (ps *PeerState) setHasVote(height uint, round uint, type_ byte, index uint)
} }
switch type_ { switch type_ {
case blk.VoteTypePrevote:
case types.VoteTypePrevote:
ps.Prevotes.SetIndex(index, true) ps.Prevotes.SetIndex(index, true)
case blk.VoteTypePrecommit:
case types.VoteTypePrecommit:
ps.Precommits.SetIndex(index, true) ps.Precommits.SetIndex(index, true)
case blk.VoteTypeCommit:
case types.VoteTypeCommit:
if round < ps.Round { if round < ps.Round {
ps.Prevotes.SetIndex(index, true) ps.Prevotes.SetIndex(index, true)
ps.Precommits.SetIndex(index, true) ps.Precommits.SetIndex(index, true)
@ -676,9 +676,9 @@ func (ps *PeerState) ApplyNewRoundStepMessage(msg *NewRoundStepMessage, rs *Roun
ps.StartTime = startTime ps.StartTime = startTime
if psHeight != msg.Height || psRound != msg.Round { if psHeight != msg.Height || psRound != msg.Round {
ps.Proposal = false ps.Proposal = false
ps.ProposalBlockParts = blk.PartSetHeader{}
ps.ProposalBlockParts = types.PartSetHeader{}
ps.ProposalBlockBitArray = BitArray{} ps.ProposalBlockBitArray = BitArray{}
ps.ProposalPOLParts = blk.PartSetHeader{}
ps.ProposalPOLParts = types.PartSetHeader{}
ps.ProposalPOLBitArray = BitArray{} ps.ProposalPOLBitArray = BitArray{}
// We'll update the BitArray capacity later. // We'll update the BitArray capacity later.
ps.Prevotes = BitArray{} ps.Prevotes = BitArray{}
@ -714,7 +714,7 @@ func (ps *PeerState) ApplyHasVoteMessage(msg *HasVoteMessage) {
defer ps.mtx.Unlock() defer ps.mtx.Unlock()
// Special case for LastCommits // Special case for LastCommits
if ps.Height == msg.Height+1 && msg.Type == blk.VoteTypeCommit {
if ps.Height == msg.Height+1 && msg.Type == types.VoteTypeCommit {
ps.LastCommits.SetIndex(msg.Index, true) ps.LastCommits.SetIndex(msg.Index, true)
return return
} else if ps.Height != msg.Height { } else if ps.Height != msg.Height {
@ -784,7 +784,7 @@ func (m *NewRoundStepMessage) String() string {
type CommitStepMessage struct { type CommitStepMessage struct {
Height uint Height uint
BlockParts blk.PartSetHeader
BlockParts types.PartSetHeader
BlockBitArray BitArray BlockBitArray BitArray
} }
@ -805,7 +805,7 @@ type PartMessage struct {
Height uint Height uint
Round uint Round uint
Type byte Type byte
Part *blk.Part
Part *types.Part
} }
func (m *PartMessage) TypeByte() byte { return msgTypePart } func (m *PartMessage) TypeByte() byte { return msgTypePart }
@ -818,7 +818,7 @@ func (m *PartMessage) String() string {
type VoteMessage struct { type VoteMessage struct {
ValidatorIndex uint ValidatorIndex uint
Vote *blk.Vote
Vote *types.Vote
} }
func (m *VoteMessage) TypeByte() byte { return msgTypeVote } func (m *VoteMessage) TypeByte() byte { return msgTypeVote }


+ 54
- 54
consensus/state.go View File

@ -62,12 +62,12 @@ import (
"github.com/tendermint/tendermint/account" "github.com/tendermint/tendermint/account"
"github.com/tendermint/tendermint/binary" "github.com/tendermint/tendermint/binary"
blk "github.com/tendermint/tendermint/block"
. "github.com/tendermint/tendermint/common" . "github.com/tendermint/tendermint/common"
"github.com/tendermint/tendermint/config" "github.com/tendermint/tendermint/config"
. "github.com/tendermint/tendermint/consensus/types" . "github.com/tendermint/tendermint/consensus/types"
mempl "github.com/tendermint/tendermint/mempool" mempl "github.com/tendermint/tendermint/mempool"
sm "github.com/tendermint/tendermint/state" sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/types"
) )
const ( const (
@ -171,12 +171,12 @@ type RoundState struct {
CommitTime time.Time // Time when +2/3 commits were found CommitTime time.Time // Time when +2/3 commits were found
Validators *sm.ValidatorSet Validators *sm.ValidatorSet
Proposal *Proposal Proposal *Proposal
ProposalBlock *blk.Block
ProposalBlockParts *blk.PartSet
ProposalBlock *types.Block
ProposalBlockParts *types.PartSet
ProposalPOL *POL ProposalPOL *POL
ProposalPOLParts *blk.PartSet
LockedBlock *blk.Block
LockedBlockParts *blk.PartSet
ProposalPOLParts *types.PartSet
LockedBlock *types.Block
LockedBlockParts *types.PartSet
LockedPOL *POL // Rarely needed, so no LockedPOLParts. LockedPOL *POL // Rarely needed, so no LockedPOLParts.
Prevotes *VoteSet Prevotes *VoteSet
Precommits *VoteSet Precommits *VoteSet
@ -234,20 +234,20 @@ type ConsensusState struct {
stopped uint32 stopped uint32
quit chan struct{} quit chan struct{}
blockStore *blk.BlockStore
blockStore *types.BlockStore
mempoolReactor *mempl.MempoolReactor mempoolReactor *mempl.MempoolReactor
runActionCh chan RoundAction runActionCh chan RoundAction
newStepCh chan *RoundState newStepCh chan *RoundState
mtx sync.Mutex mtx sync.Mutex
RoundState RoundState
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.
state *sm.State // State until height-1.
stagedBlock *types.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 *blk.BlockStore, mempoolReactor *mempl.MempoolReactor) *ConsensusState {
func NewConsensusState(state *sm.State, blockStore *types.BlockStore, mempoolReactor *mempl.MempoolReactor) *ConsensusState {
cs := &ConsensusState{ cs := &ConsensusState{
quit: make(chan struct{}), quit: make(chan struct{}),
blockStore: blockStore, blockStore: blockStore,
@ -484,10 +484,10 @@ func (cs *ConsensusState) updateToState(state *sm.State) {
cs.LockedBlock = nil cs.LockedBlock = nil
cs.LockedBlockParts = nil cs.LockedBlockParts = nil
cs.LockedPOL = nil cs.LockedPOL = nil
cs.Prevotes = NewVoteSet(height, 0, blk.VoteTypePrevote, validators)
cs.Precommits = NewVoteSet(height, 0, blk.VoteTypePrecommit, validators)
cs.Prevotes = NewVoteSet(height, 0, types.VoteTypePrevote, validators)
cs.Precommits = NewVoteSet(height, 0, types.VoteTypePrecommit, validators)
cs.LastCommits = cs.Commits cs.LastCommits = cs.Commits
cs.Commits = NewVoteSet(height, 0, blk.VoteTypeCommit, validators)
cs.Commits = NewVoteSet(height, 0, types.VoteTypeCommit, validators)
cs.state = state cs.state = state
cs.stagedBlock = nil cs.stagedBlock = nil
@ -501,7 +501,7 @@ func (cs *ConsensusState) updateToState(state *sm.State) {
// If we've timed out, then send rebond tx. // If we've timed out, then send rebond tx.
if cs.PrivValidator != nil && cs.state.UnbondingValidators.HasAddress(cs.PrivValidator.Address) { if cs.PrivValidator != nil && cs.state.UnbondingValidators.HasAddress(cs.PrivValidator.Address) {
rebondTx := &blk.RebondTx{
rebondTx := &types.RebondTx{
Address: cs.PrivValidator.Address, Address: cs.PrivValidator.Address,
Height: cs.Height + 1, Height: cs.Height + 1,
} }
@ -534,9 +534,9 @@ func (cs *ConsensusState) setupNewRound(round uint) {
cs.ProposalBlockParts = nil cs.ProposalBlockParts = nil
cs.ProposalPOL = nil cs.ProposalPOL = nil
cs.ProposalPOLParts = nil cs.ProposalPOLParts = nil
cs.Prevotes = NewVoteSet(cs.Height, round, blk.VoteTypePrevote, validators)
cs.Prevotes = NewVoteSet(cs.Height, round, types.VoteTypePrevote, validators)
cs.Prevotes.AddFromCommits(cs.Commits) cs.Prevotes.AddFromCommits(cs.Commits)
cs.Precommits = NewVoteSet(cs.Height, round, blk.VoteTypePrecommit, validators)
cs.Precommits = NewVoteSet(cs.Height, round, types.VoteTypePrecommit, validators)
cs.Precommits.AddFromCommits(cs.Commits) cs.Precommits.AddFromCommits(cs.Commits)
} }
@ -586,10 +586,10 @@ func (cs *ConsensusState) RunActionPropose(height uint, round uint) {
log.Debug("Our turn to propose", "proposer", cs.Validators.Proposer().Address, "privValidator", cs.PrivValidator) log.Debug("Our turn to propose", "proposer", cs.Validators.Proposer().Address, "privValidator", cs.PrivValidator)
} }
var block *blk.Block
var blockParts *blk.PartSet
var block *types.Block
var blockParts *types.PartSet
var pol *POL var pol *POL
var polParts *blk.PartSet
var polParts *types.PartSet
// Decide on block and POL // Decide on block and POL
if cs.LockedBlock != nil { if cs.LockedBlock != nil {
@ -599,11 +599,11 @@ func (cs *ConsensusState) RunActionPropose(height uint, round uint) {
pol = cs.LockedPOL pol = cs.LockedPOL
} else { } else {
// Otherwise we should create a new proposal. // Otherwise we should create a new proposal.
var validation *blk.Validation
var validation *types.Validation
if cs.Height == 1 { if cs.Height == 1 {
// We're creating a proposal for the first block. // We're creating a proposal for the first block.
// The validation is empty. // The validation is empty.
validation = &blk.Validation{}
validation = &types.Validation{}
} else if cs.LastCommits.HasTwoThirdsMajority() { } else if cs.LastCommits.HasTwoThirdsMajority() {
// Make the validation from LastCommits // Make the validation from LastCommits
validation = cs.LastCommits.MakeValidation() validation = cs.LastCommits.MakeValidation()
@ -617,8 +617,8 @@ func (cs *ConsensusState) RunActionPropose(height uint, round uint) {
} }
} }
txs := cs.mempoolReactor.Mempool.GetProposalTxs() txs := cs.mempoolReactor.Mempool.GetProposalTxs()
block = &blk.Block{
Header: &blk.Header{
block = &types.Block{
Header: &types.Header{
Network: config.App().GetString("Network"), Network: config.App().GetString("Network"),
Height: cs.Height, Height: cs.Height,
Time: time.Now(), Time: time.Now(),
@ -629,24 +629,24 @@ func (cs *ConsensusState) RunActionPropose(height uint, round uint) {
StateHash: nil, // Will set afterwards. StateHash: nil, // Will set afterwards.
}, },
Validation: validation, Validation: validation,
Data: &blk.Data{
Data: &types.Data{
Txs: txs, Txs: txs,
}, },
} }
// Set the blk.Header.StateHash.
// Set the types.Header.StateHash.
err := cs.state.SetBlockStateHash(block) err := cs.state.SetBlockStateHash(block)
if err != nil { if err != nil {
log.Error("Error setting state hash", "error", err) log.Error("Error setting state hash", "error", err)
return return
} }
blockParts = blk.NewPartSetFromData(binary.BinaryBytes(block))
blockParts = types.NewPartSetFromData(binary.BinaryBytes(block))
pol = cs.LockedPOL // If exists, is a PoUnlock. pol = cs.LockedPOL // If exists, is a PoUnlock.
} }
if pol != nil { if pol != nil {
polParts = blk.NewPartSetFromData(binary.BinaryBytes(pol))
polParts = types.NewPartSetFromData(binary.BinaryBytes(pol))
} }
// Make proposal // Make proposal
@ -681,14 +681,14 @@ func (cs *ConsensusState) RunActionPrevote(height uint, round uint) {
// If a block is locked, prevote that. // If a block is locked, prevote that.
if cs.LockedBlock != nil { if cs.LockedBlock != nil {
cs.signAddVote(blk.VoteTypePrevote, cs.LockedBlock.Hash(), cs.LockedBlockParts.Header())
cs.signAddVote(types.VoteTypePrevote, cs.LockedBlock.Hash(), cs.LockedBlockParts.Header())
return return
} }
// If ProposalBlock is nil, prevote nil. // If ProposalBlock is nil, prevote nil.
if cs.ProposalBlock == nil { if cs.ProposalBlock == nil {
log.Warn("ProposalBlock is nil") log.Warn("ProposalBlock is nil")
cs.signAddVote(blk.VoteTypePrevote, nil, blk.PartSetHeader{})
cs.signAddVote(types.VoteTypePrevote, nil, types.PartSetHeader{})
return return
} }
@ -697,12 +697,12 @@ func (cs *ConsensusState) RunActionPrevote(height uint, round uint) {
if err != nil { if err != nil {
// ProposalBlock is invalid, prevote nil. // ProposalBlock is invalid, prevote nil.
log.Warn("ProposalBlock is invalid", "error", err) log.Warn("ProposalBlock is invalid", "error", err)
cs.signAddVote(blk.VoteTypePrevote, nil, blk.PartSetHeader{})
cs.signAddVote(types.VoteTypePrevote, nil, types.PartSetHeader{})
return return
} }
// Prevote cs.ProposalBlock // Prevote cs.ProposalBlock
cs.signAddVote(blk.VoteTypePrevote, cs.ProposalBlock.Hash(), cs.ProposalBlockParts.Header())
cs.signAddVote(types.VoteTypePrevote, cs.ProposalBlock.Hash(), cs.ProposalBlockParts.Header())
return return
} }
@ -738,7 +738,7 @@ func (cs *ConsensusState) RunActionPrecommit(height uint, round uint) {
// If +2/3 prevoted for already locked block, precommit it. // If +2/3 prevoted for already locked block, precommit it.
if cs.LockedBlock.HashesTo(hash) { if cs.LockedBlock.HashesTo(hash) {
cs.signAddVote(blk.VoteTypePrecommit, hash, partsHeader)
cs.signAddVote(types.VoteTypePrecommit, hash, partsHeader)
return return
} }
@ -752,7 +752,7 @@ func (cs *ConsensusState) RunActionPrecommit(height uint, round uint) {
} }
cs.LockedBlock = cs.ProposalBlock cs.LockedBlock = cs.ProposalBlock
cs.LockedBlockParts = cs.ProposalBlockParts cs.LockedBlockParts = cs.ProposalBlockParts
cs.signAddVote(blk.VoteTypePrecommit, hash, partsHeader)
cs.signAddVote(types.VoteTypePrecommit, hash, partsHeader)
return return
} }
@ -806,7 +806,7 @@ func (cs *ConsensusState) RunActionCommit(height uint) {
// We're getting the wrong block. // We're getting the wrong block.
// Set up ProposalBlockParts and keep waiting. // Set up ProposalBlockParts and keep waiting.
cs.ProposalBlock = nil cs.ProposalBlock = nil
cs.ProposalBlockParts = blk.NewPartSetFromHeader(partsHeader)
cs.ProposalBlockParts = types.NewPartSetFromHeader(partsHeader)
} else { } else {
// We just need to keep waiting. // We just need to keep waiting.
@ -896,14 +896,14 @@ func (cs *ConsensusState) SetProposal(proposal *Proposal) error {
} }
cs.Proposal = proposal cs.Proposal = proposal
cs.ProposalBlockParts = blk.NewPartSetFromHeader(proposal.BlockParts)
cs.ProposalPOLParts = blk.NewPartSetFromHeader(proposal.POLParts)
cs.ProposalBlockParts = types.NewPartSetFromHeader(proposal.BlockParts)
cs.ProposalPOLParts = types.NewPartSetFromHeader(proposal.POLParts)
return nil return nil
} }
// NOTE: block is not necessarily valid. // NOTE: block is not necessarily valid.
// NOTE: This function may increment the height. // NOTE: This function may increment the height.
func (cs *ConsensusState) AddProposalBlockPart(height uint, round uint, part *blk.Part) (added bool, err error) {
func (cs *ConsensusState) AddProposalBlockPart(height uint, round uint, part *types.Part) (added bool, err error) {
cs.mtx.Lock() cs.mtx.Lock()
defer cs.mtx.Unlock() defer cs.mtx.Unlock()
@ -924,7 +924,7 @@ func (cs *ConsensusState) AddProposalBlockPart(height uint, round uint, part *bl
if added && cs.ProposalBlockParts.IsComplete() { if added && cs.ProposalBlockParts.IsComplete() {
var n int64 var n int64
var err error var err error
cs.ProposalBlock = binary.ReadBinary(&blk.Block{}, cs.ProposalBlockParts.GetReader(), &n, &err).(*blk.Block)
cs.ProposalBlock = binary.ReadBinary(&types.Block{}, cs.ProposalBlockParts.GetReader(), &n, &err).(*types.Block)
// If we're already in the commit step, try to finalize round. // If we're already in the commit step, try to finalize round.
if cs.Step == RoundStepCommit { if cs.Step == RoundStepCommit {
cs.queueAction(RoundAction{cs.Height, cs.Round, RoundActionTryFinalize}) cs.queueAction(RoundAction{cs.Height, cs.Round, RoundActionTryFinalize})
@ -936,7 +936,7 @@ func (cs *ConsensusState) AddProposalBlockPart(height uint, round uint, part *bl
} }
// NOTE: POL is not necessarily valid. // NOTE: POL is not necessarily valid.
func (cs *ConsensusState) AddProposalPOLPart(height uint, round uint, part *blk.Part) (added bool, err error) {
func (cs *ConsensusState) AddProposalPOLPart(height uint, round uint, part *types.Part) (added bool, err error) {
cs.mtx.Lock() cs.mtx.Lock()
defer cs.mtx.Unlock() defer cs.mtx.Unlock()
@ -962,7 +962,7 @@ func (cs *ConsensusState) AddProposalPOLPart(height uint, round uint, part *blk.
return true, nil return true, nil
} }
func (cs *ConsensusState) AddVote(address []byte, vote *blk.Vote) (added bool, index uint, err error) {
func (cs *ConsensusState) AddVote(address []byte, vote *types.Vote) (added bool, index uint, err error) {
cs.mtx.Lock() cs.mtx.Lock()
defer cs.mtx.Unlock() defer cs.mtx.Unlock()
@ -971,15 +971,15 @@ func (cs *ConsensusState) AddVote(address []byte, vote *blk.Vote) (added bool, i
//----------------------------------------------------------------------------- //-----------------------------------------------------------------------------
func (cs *ConsensusState) addVote(address []byte, vote *blk.Vote) (added bool, index uint, err error) {
func (cs *ConsensusState) addVote(address []byte, vote *types.Vote) (added bool, index uint, err error) {
switch vote.Type { switch vote.Type {
case blk.VoteTypePrevote:
case types.VoteTypePrevote:
// Prevotes checks for height+round match. // Prevotes checks for height+round match.
return cs.Prevotes.Add(address, vote) return cs.Prevotes.Add(address, vote)
case blk.VoteTypePrecommit:
case types.VoteTypePrecommit:
// Precommits checks for height+round match. // Precommits checks for height+round match.
return cs.Precommits.Add(address, vote) return cs.Precommits.Add(address, vote)
case blk.VoteTypeCommit:
case types.VoteTypeCommit:
if vote.Height == cs.Height { if vote.Height == cs.Height {
// No need to check if vote.Round < cs.Round ... // No need to check if vote.Round < cs.Round ...
// Prevotes && Precommits already checks that. // Prevotes && Precommits already checks that.
@ -1006,7 +1006,7 @@ func (cs *ConsensusState) addVote(address []byte, vote *blk.Vote) (added bool, i
} }
} }
func (cs *ConsensusState) stageBlock(block *blk.Block, blockParts *blk.PartSet) error {
func (cs *ConsensusState) stageBlock(block *types.Block, blockParts *types.PartSet) error {
if block == nil { if block == nil {
panic("Cannot stage nil block") panic("Cannot stage nil block")
} }
@ -1031,11 +1031,11 @@ func (cs *ConsensusState) stageBlock(block *blk.Block, blockParts *blk.PartSet)
} }
} }
func (cs *ConsensusState) signAddVote(type_ byte, hash []byte, header blk.PartSetHeader) *blk.Vote {
func (cs *ConsensusState) signAddVote(type_ byte, hash []byte, header types.PartSetHeader) *types.Vote {
if cs.PrivValidator == nil || !cs.Validators.HasAddress(cs.PrivValidator.Address) { if cs.PrivValidator == nil || !cs.Validators.HasAddress(cs.PrivValidator.Address) {
return nil return nil
} }
vote := &blk.Vote{
vote := &types.Vote{
Height: cs.Height, Height: cs.Height,
Round: cs.Round, Round: cs.Round,
Type: type_, Type: type_,
@ -1054,7 +1054,7 @@ func (cs *ConsensusState) signAddVote(type_ byte, hash []byte, header blk.PartSe
} }
// sign a Commit-Vote // sign a Commit-Vote
func (cs *ConsensusState) commitVoteBlock(block *blk.Block, blockParts *blk.PartSet) {
func (cs *ConsensusState) commitVoteBlock(block *types.Block, blockParts *types.PartSet) {
// The proposal must be valid. // The proposal must be valid.
if err := cs.stageBlock(block, blockParts); err != nil { if err := cs.stageBlock(block, blockParts); err != nil {
@ -1065,16 +1065,16 @@ func (cs *ConsensusState) commitVoteBlock(block *blk.Block, blockParts *blk.Part
// Commit-vote. // Commit-vote.
if cs.lastCommitVoteHeight < block.Height { if cs.lastCommitVoteHeight < block.Height {
cs.signAddVote(blk.VoteTypeCommit, block.Hash(), blockParts.Header())
cs.signAddVote(types.VoteTypeCommit, block.Hash(), blockParts.Header())
cs.lastCommitVoteHeight = block.Height cs.lastCommitVoteHeight = block.Height
} else { } else {
log.Error("Duplicate commitVoteBlock() attempt", "lastCommitVoteHeight", cs.lastCommitVoteHeight, "blk.Height", block.Height)
log.Error("Duplicate commitVoteBlock() attempt", "lastCommitVoteHeight", cs.lastCommitVoteHeight, "types.Height", block.Height)
} }
} }
// Save Block, save the +2/3 Commits we've seen, // Save Block, save the +2/3 Commits we've seen,
// and sign a Commit-Vote if we haven't already // and sign a Commit-Vote if we haven't already
func (cs *ConsensusState) saveCommitVoteBlock(block *blk.Block, blockParts *blk.PartSet, commits *VoteSet) {
func (cs *ConsensusState) saveCommitVoteBlock(block *types.Block, blockParts *types.PartSet, commits *VoteSet) {
// The proposal must be valid. // The proposal must be valid.
if err := cs.stageBlock(block, blockParts); err != nil { if err := cs.stageBlock(block, blockParts); err != nil {
@ -1097,7 +1097,7 @@ func (cs *ConsensusState) saveCommitVoteBlock(block *blk.Block, blockParts *blk.
// Commit-vote if we haven't already. // Commit-vote if we haven't already.
if cs.lastCommitVoteHeight < block.Height { if cs.lastCommitVoteHeight < block.Height {
cs.signAddVote(blk.VoteTypeCommit, block.Hash(), blockParts.Header())
cs.signAddVote(types.VoteTypeCommit, block.Hash(), blockParts.Header())
cs.lastCommitVoteHeight = block.Height cs.lastCommitVoteHeight = block.Height
} }
} }


+ 15
- 15
consensus/state_test.go View File

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


+ 2
- 2
consensus/test.go View File

@ -3,15 +3,15 @@ package consensus
import ( import (
"sort" "sort"
blk "github.com/tendermint/tendermint/block"
dbm "github.com/tendermint/tendermint/db" dbm "github.com/tendermint/tendermint/db"
mempl "github.com/tendermint/tendermint/mempool" mempl "github.com/tendermint/tendermint/mempool"
sm "github.com/tendermint/tendermint/state" sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/types"
) )
func randConsensusState() (*ConsensusState, []*sm.PrivValidator) { func randConsensusState() (*ConsensusState, []*sm.PrivValidator) {
state, _, privValidators := sm.RandGenesisState(20, false, 1000, 10, false, 1000) state, _, privValidators := sm.RandGenesisState(20, false, 1000, 10, false, 1000)
blockStore := blk.NewBlockStore(dbm.NewMemDB())
blockStore := types.NewBlockStore(dbm.NewMemDB())
mempool := mempl.NewMempool(state) mempool := mempl.NewMempool(state)
mempoolReactor := mempl.NewMempoolReactor(mempool) mempoolReactor := mempl.NewMempoolReactor(mempool)
cs := NewConsensusState(state, blockStore, mempoolReactor) 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/account"
"github.com/tendermint/tendermint/binary" "github.com/tendermint/tendermint/binary"
blk "github.com/tendermint/tendermint/block"
"github.com/tendermint/tendermint/types"
) )
var ( var (
@ -18,12 +18,12 @@ var (
type Proposal struct { type Proposal struct {
Height uint Height uint
Round uint Round uint
BlockParts blk.PartSetHeader
POLParts blk.PartSetHeader
BlockParts types.PartSetHeader
POLParts types.PartSetHeader
Signature account.SignatureEd25519 Signature account.SignatureEd25519
} }
func NewProposal(height uint, round uint, blockParts, polParts blk.PartSetHeader) *Proposal {
func NewProposal(height uint, round uint, blockParts, polParts types.PartSetHeader) *Proposal {
return &Proposal{ return &Proposal{
Height: height, Height: height,
Round: round, Round: round,


+ 26
- 26
consensus/vote_set.go View File

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


+ 30
- 30
consensus/vote_set_test.go View File

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


+ 3
- 3
daemon/daemon.go View File

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


+ 7
- 7
mempool/mempool.go View File

@ -12,14 +12,14 @@ import (
"sync" "sync"
"github.com/tendermint/tendermint/binary" "github.com/tendermint/tendermint/binary"
blk "github.com/tendermint/tendermint/block"
sm "github.com/tendermint/tendermint/state" sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/types"
) )
type Mempool struct { type Mempool struct {
mtx sync.Mutex mtx sync.Mutex
state *sm.State state *sm.State
txs []blk.Tx
txs []types.Tx
} }
func NewMempool(state *sm.State) *Mempool { func NewMempool(state *sm.State) *Mempool {
@ -33,7 +33,7 @@ func (mem *Mempool) GetState() *sm.State {
} }
// Apply tx to the state and remember it. // Apply tx to the state and remember it.
func (mem *Mempool) AddTx(tx blk.Tx) (err error) {
func (mem *Mempool) AddTx(tx types.Tx) (err error) {
mem.mtx.Lock() mem.mtx.Lock()
defer mem.mtx.Unlock() defer mem.mtx.Unlock()
err = mem.state.ExecTx(tx, false) err = mem.state.ExecTx(tx, false)
@ -47,7 +47,7 @@ func (mem *Mempool) AddTx(tx blk.Tx) (err error) {
} }
} }
func (mem *Mempool) GetProposalTxs() []blk.Tx {
func (mem *Mempool) GetProposalTxs() []types.Tx {
mem.mtx.Lock() mem.mtx.Lock()
defer mem.mtx.Unlock() defer mem.mtx.Unlock()
log.Debug("GetProposalTxs:", "txs", mem.txs) log.Debug("GetProposalTxs:", "txs", mem.txs)
@ -58,7 +58,7 @@ func (mem *Mempool) GetProposalTxs() []blk.Tx {
// "state" is the result of state.AppendBlock("block"). // "state" is the result of state.AppendBlock("block").
// Txs that are present in "block" are discarded from mempool. // Txs that are present in "block" are discarded from mempool.
// Txs that have become invalid in the new "state" are also discarded. // Txs that have become invalid in the new "state" are also discarded.
func (mem *Mempool) ResetForBlockAndState(block *blk.Block, state *sm.State) {
func (mem *Mempool) ResetForBlockAndState(block *types.Block, state *sm.State) {
mem.mtx.Lock() mem.mtx.Lock()
defer mem.mtx.Unlock() defer mem.mtx.Unlock()
mem.state = state.Copy() mem.state = state.Copy()
@ -71,7 +71,7 @@ func (mem *Mempool) ResetForBlockAndState(block *blk.Block, state *sm.State) {
} }
// Next, filter all txs from mem.txs that are in blockTxsMap // Next, filter all txs from mem.txs that are in blockTxsMap
txs := []blk.Tx{}
txs := []types.Tx{}
for _, tx := range mem.txs { for _, tx := range mem.txs {
txHash := binary.BinarySha256(tx) txHash := binary.BinarySha256(tx)
if _, ok := blockTxsMap[string(txHash)]; ok { if _, ok := blockTxsMap[string(txHash)]; ok {
@ -84,7 +84,7 @@ func (mem *Mempool) ResetForBlockAndState(block *blk.Block, state *sm.State) {
} }
// Next, filter all txs that aren't valid given new state. // Next, filter all txs that aren't valid given new state.
validTxs := []blk.Tx{}
validTxs := []types.Tx{}
for _, tx := range txs { for _, tx := range txs {
err := mem.state.ExecTx(tx, false) err := mem.state.ExecTx(tx, false)
if err == nil { if err == nil {


+ 3
- 3
mempool/reactor.go View File

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


+ 5
- 5
rpc/blocks.go View File

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

+ 3
- 3
rpc/mempool.go View File

@ -4,16 +4,16 @@ import (
"net/http" "net/http"
"github.com/tendermint/tendermint/binary" "github.com/tendermint/tendermint/binary"
blk "github.com/tendermint/tendermint/block"
. "github.com/tendermint/tendermint/common" . "github.com/tendermint/tendermint/common"
"github.com/tendermint/tendermint/merkle" "github.com/tendermint/tendermint/merkle"
"github.com/tendermint/tendermint/state" "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/types"
) )
func BroadcastTxHandler(w http.ResponseWriter, r *http.Request) { func BroadcastTxHandler(w http.ResponseWriter, r *http.Request) {
txJSON := GetParam(r, "tx") txJSON := GetParam(r, "tx")
var err error var err error
var tx blk.Tx
var tx types.Tx
binary.ReadJSON(&tx, []byte(txJSON), &err) binary.ReadJSON(&tx, []byte(txJSON), &err)
if err != nil { if err != nil {
WriteAPIResponse(w, API_INVALID_PARAM, Fmt("Invalid tx: %v", err)) WriteAPIResponse(w, API_INVALID_PARAM, Fmt("Invalid tx: %v", err))
@ -30,7 +30,7 @@ func BroadcastTxHandler(w http.ResponseWriter, r *http.Request) {
var createsContract bool var createsContract bool
var contractAddr []byte var contractAddr []byte
if callTx, ok := tx.(*blk.CallTx); ok {
if callTx, ok := tx.(*types.CallTx); ok {
if callTx.Address == nil { if callTx.Address == nil {
createsContract = true createsContract = true
contractAddr = state.NewContractAddress(callTx.Input.Address, uint64(callTx.Input.Sequence)) contractAddr = state.NewContractAddress(callTx.Input.Address, uint64(callTx.Input.Sequence))


+ 3
- 3
rpc/rpc.go View File

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


+ 11
- 11
rpc/txs.go View File

@ -5,8 +5,8 @@ import (
"github.com/tendermint/tendermint/account" "github.com/tendermint/tendermint/account"
"github.com/tendermint/tendermint/binary" "github.com/tendermint/tendermint/binary"
blk "github.com/tendermint/tendermint/block"
. "github.com/tendermint/tendermint/common" . "github.com/tendermint/tendermint/common"
"github.com/tendermint/tendermint/types"
) )
func SignTxHandler(w http.ResponseWriter, r *http.Request) { func SignTxHandler(w http.ResponseWriter, r *http.Request) {
@ -14,7 +14,7 @@ func SignTxHandler(w http.ResponseWriter, r *http.Request) {
privAccountsStr := GetParam(r, "privAccounts") privAccountsStr := GetParam(r, "privAccounts")
var err error var err error
var tx blk.Tx
var tx types.Tx
binary.ReadJSON(&tx, []byte(txStr), &err) binary.ReadJSON(&tx, []byte(txStr), &err)
if err != nil { if err != nil {
WriteAPIResponse(w, API_INVALID_PARAM, Fmt("Invalid tx: %v", err)) WriteAPIResponse(w, API_INVALID_PARAM, Fmt("Invalid tx: %v", err))
@ -33,25 +33,25 @@ func SignTxHandler(w http.ResponseWriter, r *http.Request) {
} }
switch tx.(type) { switch tx.(type) {
case *blk.SendTx:
sendTx := tx.(*blk.SendTx)
case *types.SendTx:
sendTx := tx.(*types.SendTx)
for i, input := range sendTx.Inputs { for i, input := range sendTx.Inputs {
input.PubKey = privAccounts[i].PubKey input.PubKey = privAccounts[i].PubKey
input.Signature = privAccounts[i].Sign(sendTx) input.Signature = privAccounts[i].Sign(sendTx)
} }
case *blk.BondTx:
bondTx := tx.(*blk.BondTx)
case *types.BondTx:
bondTx := tx.(*types.BondTx)
for i, input := range bondTx.Inputs { for i, input := range bondTx.Inputs {
input.PubKey = privAccounts[i].PubKey input.PubKey = privAccounts[i].PubKey
input.Signature = privAccounts[i].Sign(bondTx) input.Signature = privAccounts[i].Sign(bondTx)
} }
case *blk.UnbondTx:
unbondTx := tx.(*blk.UnbondTx)
case *types.UnbondTx:
unbondTx := tx.(*types.UnbondTx)
unbondTx.Signature = privAccounts[0].Sign(unbondTx).(account.SignatureEd25519) unbondTx.Signature = privAccounts[0].Sign(unbondTx).(account.SignatureEd25519)
case *blk.RebondTx:
rebondTx := tx.(*blk.RebondTx)
case *types.RebondTx:
rebondTx := tx.(*types.RebondTx)
rebondTx.Signature = privAccounts[0].Sign(rebondTx).(account.SignatureEd25519) rebondTx.Signature = privAccounts[0].Sign(rebondTx).(account.SignatureEd25519)
} }
WriteAPIResponse(w, API_OK, struct{ blk.Tx }{tx})
WriteAPIResponse(w, API_OK, struct{ types.Tx }{tx})
} }

+ 4
- 4
state/genesis.go View File

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


+ 8
- 8
state/priv_validator.go View File

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


+ 46
- 46
state/state.go View File

@ -8,10 +8,10 @@ import (
"github.com/tendermint/tendermint/account" "github.com/tendermint/tendermint/account"
"github.com/tendermint/tendermint/binary" "github.com/tendermint/tendermint/binary"
blk "github.com/tendermint/tendermint/block"
. "github.com/tendermint/tendermint/common" . "github.com/tendermint/tendermint/common"
dbm "github.com/tendermint/tendermint/db" dbm "github.com/tendermint/tendermint/db"
"github.com/tendermint/tendermint/merkle" "github.com/tendermint/tendermint/merkle"
"github.com/tendermint/tendermint/types"
"github.com/tendermint/tendermint/vm" "github.com/tendermint/tendermint/vm"
) )
@ -26,7 +26,7 @@ var (
//----------------------------------------------------------------------------- //-----------------------------------------------------------------------------
type InvalidTxError struct { type InvalidTxError struct {
Tx blk.Tx
Tx types.Tx
Reason error Reason error
} }
@ -41,7 +41,7 @@ type State struct {
DB dbm.DB DB dbm.DB
LastBlockHeight uint LastBlockHeight uint
LastBlockHash []byte LastBlockHash []byte
LastBlockParts blk.PartSetHeader
LastBlockParts types.PartSetHeader
LastBlockTime time.Time LastBlockTime time.Time
BondedValidators *ValidatorSet BondedValidators *ValidatorSet
LastBondedValidators *ValidatorSet LastBondedValidators *ValidatorSet
@ -59,7 +59,7 @@ func LoadState(db dbm.DB) *State {
r, n, err := bytes.NewReader(buf), new(int64), new(error) r, n, err := bytes.NewReader(buf), new(int64), new(error)
s.LastBlockHeight = binary.ReadUvarint(r, n, err) s.LastBlockHeight = binary.ReadUvarint(r, n, err)
s.LastBlockHash = binary.ReadByteSlice(r, n, err) s.LastBlockHash = binary.ReadByteSlice(r, n, err)
s.LastBlockParts = binary.ReadBinary(blk.PartSetHeader{}, r, n, err).(blk.PartSetHeader)
s.LastBlockParts = binary.ReadBinary(types.PartSetHeader{}, r, n, err).(types.PartSetHeader)
s.LastBlockTime = binary.ReadTime(r, n, err) s.LastBlockTime = binary.ReadTime(r, n, err)
s.BondedValidators = binary.ReadBinary(&ValidatorSet{}, r, n, err).(*ValidatorSet) s.BondedValidators = binary.ReadBinary(&ValidatorSet{}, r, n, err).(*ValidatorSet)
s.LastBondedValidators = binary.ReadBinary(&ValidatorSet{}, r, n, err).(*ValidatorSet) s.LastBondedValidators = binary.ReadBinary(&ValidatorSet{}, r, n, err).(*ValidatorSet)
@ -117,16 +117,16 @@ func (s *State) Copy() *State {
// account.PubKey.(type) != PubKeyNil, (it must be known), // account.PubKey.(type) != PubKeyNil, (it must be known),
// or it must be specified in the TxInput. If redeclared, // or it must be specified in the TxInput. If redeclared,
// the TxInput is modified and input.PubKey set to PubKeyNil. // the TxInput is modified and input.PubKey set to PubKeyNil.
func (s *State) GetOrMakeAccounts(ins []*blk.TxInput, outs []*blk.TxOutput) (map[string]*account.Account, error) {
func (s *State) GetOrMakeAccounts(ins []*types.TxInput, outs []*types.TxOutput) (map[string]*account.Account, error) {
accounts := map[string]*account.Account{} accounts := map[string]*account.Account{}
for _, in := range ins { for _, in := range ins {
// Account shouldn't be duplicated // Account shouldn't be duplicated
if _, ok := accounts[string(in.Address)]; ok { if _, ok := accounts[string(in.Address)]; ok {
return nil, blk.ErrTxDuplicateAddress
return nil, types.ErrTxDuplicateAddress
} }
acc := s.GetAccount(in.Address) acc := s.GetAccount(in.Address)
if acc == nil { if acc == nil {
return nil, blk.ErrTxInvalidAddress
return nil, types.ErrTxInvalidAddress
} }
// PubKey should be present in either "account" or "in" // PubKey should be present in either "account" or "in"
if err := checkInputPubKey(acc, in); err != nil { if err := checkInputPubKey(acc, in); err != nil {
@ -137,7 +137,7 @@ func (s *State) GetOrMakeAccounts(ins []*blk.TxInput, outs []*blk.TxOutput) (map
for _, out := range outs { for _, out := range outs {
// Account shouldn't be duplicated // Account shouldn't be duplicated
if _, ok := accounts[string(out.Address)]; ok { if _, ok := accounts[string(out.Address)]; ok {
return nil, blk.ErrTxDuplicateAddress
return nil, types.ErrTxDuplicateAddress
} }
acc := s.GetAccount(out.Address) acc := s.GetAccount(out.Address)
// output account may be nil (new) // output account may be nil (new)
@ -154,13 +154,13 @@ func (s *State) GetOrMakeAccounts(ins []*blk.TxInput, outs []*blk.TxOutput) (map
return accounts, nil return accounts, nil
} }
func checkInputPubKey(acc *account.Account, in *blk.TxInput) error {
func checkInputPubKey(acc *account.Account, in *types.TxInput) error {
if _, isNil := acc.PubKey.(account.PubKeyNil); isNil { if _, isNil := acc.PubKey.(account.PubKeyNil); isNil {
if _, isNil := in.PubKey.(account.PubKeyNil); isNil { if _, isNil := in.PubKey.(account.PubKeyNil); isNil {
return blk.ErrTxUnknownPubKey
return types.ErrTxUnknownPubKey
} }
if !bytes.Equal(in.PubKey.Address(), acc.Address) { if !bytes.Equal(in.PubKey.Address(), acc.Address) {
return blk.ErrTxInvalidPubKey
return types.ErrTxInvalidPubKey
} }
acc.PubKey = in.PubKey acc.PubKey = in.PubKey
} else { } else {
@ -169,7 +169,7 @@ func checkInputPubKey(acc *account.Account, in *blk.TxInput) error {
return nil return nil
} }
func (s *State) ValidateInputs(accounts map[string]*account.Account, signBytes []byte, ins []*blk.TxInput) (total uint64, err error) {
func (s *State) ValidateInputs(accounts map[string]*account.Account, signBytes []byte, ins []*types.TxInput) (total uint64, err error) {
for _, in := range ins { for _, in := range ins {
acc := accounts[string(in.Address)] acc := accounts[string(in.Address)]
if acc == nil { if acc == nil {
@ -185,30 +185,30 @@ func (s *State) ValidateInputs(accounts map[string]*account.Account, signBytes [
return total, nil return total, nil
} }
func (s *State) ValidateInput(acc *account.Account, signBytes []byte, in *blk.TxInput) (err error) {
func (s *State) ValidateInput(acc *account.Account, signBytes []byte, in *types.TxInput) (err error) {
// Check TxInput basic // Check TxInput basic
if err := in.ValidateBasic(); err != nil { if err := in.ValidateBasic(); err != nil {
return err return err
} }
// Check signatures // Check signatures
if !acc.PubKey.VerifyBytes(signBytes, in.Signature) { if !acc.PubKey.VerifyBytes(signBytes, in.Signature) {
return blk.ErrTxInvalidSignature
return types.ErrTxInvalidSignature
} }
// Check sequences // Check sequences
if acc.Sequence+1 != in.Sequence { if acc.Sequence+1 != in.Sequence {
return blk.ErrTxInvalidSequence{
return types.ErrTxInvalidSequence{
Got: uint64(in.Sequence), Got: uint64(in.Sequence),
Expected: uint64(acc.Sequence + 1), Expected: uint64(acc.Sequence + 1),
} }
} }
// Check amount // Check amount
if acc.Balance < in.Amount { if acc.Balance < in.Amount {
return blk.ErrTxInsufficientFunds
return types.ErrTxInsufficientFunds
} }
return nil return nil
} }
func (s *State) ValidateOutputs(outs []*blk.TxOutput) (total uint64, err error) {
func (s *State) ValidateOutputs(outs []*types.TxOutput) (total uint64, err error) {
for _, out := range outs { for _, out := range outs {
// Check TxOutput basic // Check TxOutput basic
if err := out.ValidateBasic(); err != nil { if err := out.ValidateBasic(); err != nil {
@ -220,7 +220,7 @@ func (s *State) ValidateOutputs(outs []*blk.TxOutput) (total uint64, err error)
return total, nil return total, nil
} }
func (s *State) AdjustByInputs(accounts map[string]*account.Account, ins []*blk.TxInput) {
func (s *State) AdjustByInputs(accounts map[string]*account.Account, ins []*types.TxInput) {
for _, in := range ins { for _, in := range ins {
acc := accounts[string(in.Address)] acc := accounts[string(in.Address)]
if acc == nil { if acc == nil {
@ -234,7 +234,7 @@ func (s *State) AdjustByInputs(accounts map[string]*account.Account, ins []*blk.
} }
} }
func (s *State) AdjustByOutputs(accounts map[string]*account.Account, outs []*blk.TxOutput) {
func (s *State) AdjustByOutputs(accounts map[string]*account.Account, outs []*types.TxOutput) {
for _, out := range outs { for _, out := range outs {
acc := accounts[string(out.Address)] acc := accounts[string(out.Address)]
if acc == nil { if acc == nil {
@ -246,14 +246,14 @@ func (s *State) AdjustByOutputs(accounts map[string]*account.Account, outs []*bl
// If the tx is invalid, an error will be returned. // If the tx is invalid, an error will be returned.
// Unlike AppendBlock(), state will not be altered. // Unlike AppendBlock(), state will not be altered.
func (s *State) ExecTx(tx_ blk.Tx, runCall bool) error {
func (s *State) ExecTx(tx_ types.Tx, runCall bool) error {
// TODO: do something with fees // TODO: do something with fees
fees := uint64(0) fees := uint64(0)
// Exec tx // Exec tx
switch tx := tx_.(type) { switch tx := tx_.(type) {
case *blk.SendTx:
case *types.SendTx:
accounts, err := s.GetOrMakeAccounts(tx.Inputs, tx.Outputs) accounts, err := s.GetOrMakeAccounts(tx.Inputs, tx.Outputs)
if err != nil { if err != nil {
return err return err
@ -268,7 +268,7 @@ func (s *State) ExecTx(tx_ blk.Tx, runCall bool) error {
return err return err
} }
if outTotal > inTotal { if outTotal > inTotal {
return blk.ErrTxInsufficientFunds
return types.ErrTxInsufficientFunds
} }
fee := inTotal - outTotal fee := inTotal - outTotal
fees += fee fees += fee
@ -279,14 +279,14 @@ func (s *State) ExecTx(tx_ blk.Tx, runCall bool) error {
s.UpdateAccounts(accounts) s.UpdateAccounts(accounts)
return nil return nil
case *blk.CallTx:
case *types.CallTx:
var inAcc, outAcc *account.Account var inAcc, outAcc *account.Account
// Validate input // Validate input
inAcc = s.GetAccount(tx.Input.Address) inAcc = s.GetAccount(tx.Input.Address)
if inAcc == nil { if inAcc == nil {
log.Debug(Fmt("Can't find in account %X", tx.Input.Address)) log.Debug(Fmt("Can't find in account %X", tx.Input.Address))
return blk.ErrTxInvalidAddress
return types.ErrTxInvalidAddress
} }
// pubKey should be present in either "inAcc" or "tx.Input" // pubKey should be present in either "inAcc" or "tx.Input"
if err := checkInputPubKey(inAcc, tx.Input); err != nil { if err := checkInputPubKey(inAcc, tx.Input); err != nil {
@ -301,7 +301,7 @@ func (s *State) ExecTx(tx_ blk.Tx, runCall bool) error {
} }
if tx.Input.Amount < tx.Fee { if tx.Input.Amount < tx.Fee {
log.Debug(Fmt("Sender did not send enough to cover the fee %X", tx.Input.Address)) log.Debug(Fmt("Sender did not send enough to cover the fee %X", tx.Input.Address))
return blk.ErrTxInsufficientFunds
return types.ErrTxInsufficientFunds
} }
createAccount := len(tx.Address) == 0 createAccount := len(tx.Address) == 0
@ -309,7 +309,7 @@ func (s *State) ExecTx(tx_ blk.Tx, runCall bool) error {
// Validate output // Validate output
if len(tx.Address) != 20 { if len(tx.Address) != 20 {
log.Debug(Fmt("Destination address is not 20 bytes %X", tx.Address)) log.Debug(Fmt("Destination address is not 20 bytes %X", tx.Address))
return blk.ErrTxInvalidAddress
return types.ErrTxInvalidAddress
} }
// this may be nil if we are still in mempool and contract was created in same block as this tx // this may be nil if we are still in mempool and contract was created in same block as this tx
// but that's fine, because the account will be created properly when the create tx runs in the block // but that's fine, because the account will be created properly when the create tx runs in the block
@ -348,7 +348,7 @@ func (s *State) ExecTx(tx_ blk.Tx, runCall bool) error {
inAcc.Balance -= tx.Fee inAcc.Balance -= tx.Fee
s.UpdateAccount(inAcc) s.UpdateAccount(inAcc)
log.Debug(Fmt("Cannot find destination address %X. Deducting fee from caller", tx.Address)) log.Debug(Fmt("Cannot find destination address %X. Deducting fee from caller", tx.Address))
return blk.ErrTxInvalidAddress
return types.ErrTxInvalidAddress
} }
callee = toVMAccount(outAcc) callee = toVMAccount(outAcc)
@ -401,7 +401,7 @@ func (s *State) ExecTx(tx_ blk.Tx, runCall bool) error {
return nil return nil
case *blk.BondTx:
case *types.BondTx:
valInfo := s.GetValidatorInfo(tx.PubKey.Address()) valInfo := s.GetValidatorInfo(tx.PubKey.Address())
if valInfo != nil { if valInfo != nil {
// TODO: In the future, check that the validator wasn't destroyed, // TODO: In the future, check that the validator wasn't destroyed,
@ -425,7 +425,7 @@ func (s *State) ExecTx(tx_ blk.Tx, runCall bool) error {
return err return err
} }
if outTotal > inTotal { if outTotal > inTotal {
return blk.ErrTxInsufficientFunds
return types.ErrTxInsufficientFunds
} }
fee := inTotal - outTotal fee := inTotal - outTotal
fees += fee fees += fee
@ -454,17 +454,17 @@ func (s *State) ExecTx(tx_ blk.Tx, runCall bool) error {
} }
return nil return nil
case *blk.UnbondTx:
case *types.UnbondTx:
// The validator must be active // The validator must be active
_, val := s.BondedValidators.GetByAddress(tx.Address) _, val := s.BondedValidators.GetByAddress(tx.Address)
if val == nil { if val == nil {
return blk.ErrTxInvalidAddress
return types.ErrTxInvalidAddress
} }
// Verify the signature // Verify the signature
signBytes := account.SignBytes(tx) signBytes := account.SignBytes(tx)
if !val.PubKey.VerifyBytes(signBytes, tx.Signature) { if !val.PubKey.VerifyBytes(signBytes, tx.Signature) {
return blk.ErrTxInvalidSignature
return types.ErrTxInvalidSignature
} }
// tx.Height must be greater than val.LastCommitHeight // tx.Height must be greater than val.LastCommitHeight
@ -476,17 +476,17 @@ func (s *State) ExecTx(tx_ blk.Tx, runCall bool) error {
s.unbondValidator(val) s.unbondValidator(val)
return nil return nil
case *blk.RebondTx:
case *types.RebondTx:
// The validator must be inactive // The validator must be inactive
_, val := s.UnbondingValidators.GetByAddress(tx.Address) _, val := s.UnbondingValidators.GetByAddress(tx.Address)
if val == nil { if val == nil {
return blk.ErrTxInvalidAddress
return types.ErrTxInvalidAddress
} }
// Verify the signature // Verify the signature
signBytes := account.SignBytes(tx) signBytes := account.SignBytes(tx)
if !val.PubKey.VerifyBytes(signBytes, tx.Signature) { if !val.PubKey.VerifyBytes(signBytes, tx.Signature) {
return blk.ErrTxInvalidSignature
return types.ErrTxInvalidSignature
} }
// tx.Height must be equal to the next height // tx.Height must be equal to the next height
@ -498,20 +498,20 @@ func (s *State) ExecTx(tx_ blk.Tx, runCall bool) error {
s.rebondValidator(val) s.rebondValidator(val)
return nil return nil
case *blk.DupeoutTx:
case *types.DupeoutTx:
// Verify the signatures // Verify the signatures
_, accused := s.BondedValidators.GetByAddress(tx.Address) _, accused := s.BondedValidators.GetByAddress(tx.Address)
if accused == nil { if accused == nil {
_, accused = s.UnbondingValidators.GetByAddress(tx.Address) _, accused = s.UnbondingValidators.GetByAddress(tx.Address)
if accused == nil { if accused == nil {
return blk.ErrTxInvalidAddress
return types.ErrTxInvalidAddress
} }
} }
voteASignBytes := account.SignBytes(&tx.VoteA) voteASignBytes := account.SignBytes(&tx.VoteA)
voteBSignBytes := account.SignBytes(&tx.VoteB) voteBSignBytes := account.SignBytes(&tx.VoteB)
if !accused.PubKey.VerifyBytes(voteASignBytes, tx.VoteA.Signature) || if !accused.PubKey.VerifyBytes(voteASignBytes, tx.VoteA.Signature) ||
!accused.PubKey.VerifyBytes(voteBSignBytes, tx.VoteB.Signature) { !accused.PubKey.VerifyBytes(voteBSignBytes, tx.VoteB.Signature) {
return blk.ErrTxInvalidSignature
return types.ErrTxInvalidSignature
} }
// Verify equivocation // Verify equivocation
@ -520,10 +520,10 @@ func (s *State) ExecTx(tx_ blk.Tx, runCall bool) error {
if tx.VoteA.Height != tx.VoteB.Height { if tx.VoteA.Height != tx.VoteB.Height {
return errors.New("DupeoutTx heights don't match") return errors.New("DupeoutTx heights don't match")
} }
if tx.VoteA.Type == blk.VoteTypeCommit && tx.VoteA.Round < tx.VoteB.Round {
if tx.VoteA.Type == types.VoteTypeCommit && tx.VoteA.Round < tx.VoteB.Round {
// Check special case (not an error, validator must be slashed!) // Check special case (not an error, validator must be slashed!)
// Validators should not sign another vote after committing. // Validators should not sign another vote after committing.
} else if tx.VoteB.Type == blk.VoteTypeCommit && tx.VoteB.Round < tx.VoteA.Round {
} else if tx.VoteB.Type == types.VoteTypeCommit && tx.VoteB.Round < tx.VoteA.Round {
// We need to check both orderings of the votes // We need to check both orderings of the votes
} else { } else {
if tx.VoteA.Round != tx.VoteB.Round { if tx.VoteA.Round != tx.VoteB.Round {
@ -619,7 +619,7 @@ func (s *State) destroyValidator(val *Validator) {
// NOTE: If an error occurs during block execution, state will be left // NOTE: If an error occurs during block execution, state will be left
// at an invalid state. Copy the state before calling AppendBlock! // at an invalid state. Copy the state before calling AppendBlock!
func (s *State) AppendBlock(block *blk.Block, blockPartsHeader blk.PartSetHeader) error {
func (s *State) AppendBlock(block *types.Block, blockPartsHeader types.PartSetHeader) error {
err := s.appendBlock(block, blockPartsHeader) err := s.appendBlock(block, blockPartsHeader)
if err != nil { if err != nil {
return err return err
@ -633,9 +633,9 @@ func (s *State) AppendBlock(block *blk.Block, blockPartsHeader blk.PartSetHeader
return nil return nil
} }
func (s *State) SetBlockStateHash(block *blk.Block) error {
func (s *State) SetBlockStateHash(block *types.Block) error {
sCopy := s.Copy() sCopy := s.Copy()
err := sCopy.appendBlock(block, blk.PartSetHeader{})
err := sCopy.appendBlock(block, types.PartSetHeader{})
if err != nil { if err != nil {
return err return err
} }
@ -647,7 +647,7 @@ func (s *State) SetBlockStateHash(block *blk.Block) error {
// Appends the block, does not check block.StateHash // Appends the block, does not check block.StateHash
// NOTE: If an error occurs during block execution, state will be left // NOTE: If an error occurs during block execution, state will be left
// at an invalid state. Copy the state before calling appendBlock! // at an invalid state. Copy the state before calling appendBlock!
func (s *State) appendBlock(block *blk.Block, blockPartsHeader blk.PartSetHeader) error {
func (s *State) appendBlock(block *types.Block, blockPartsHeader types.PartSetHeader) error {
// Basic block validation. // 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 { if err != nil {
@ -670,10 +670,10 @@ func (s *State) appendBlock(block *blk.Block, blockPartsHeader blk.PartSetHeader
if commit.IsZero() { if commit.IsZero() {
return false return false
} else { } else {
vote := &blk.Vote{
vote := &types.Vote{
Height: block.Height - 1, Height: block.Height - 1,
Round: commit.Round, Round: commit.Round,
Type: blk.VoteTypeCommit,
Type: types.VoteTypeCommit,
BlockHash: block.LastBlockHash, BlockHash: block.LastBlockHash,
BlockParts: block.LastBlockParts, BlockParts: block.LastBlockParts,
} }


+ 35
- 35
state/state_test.go View File

@ -3,8 +3,8 @@ package state
import ( import (
"github.com/tendermint/tendermint/account" "github.com/tendermint/tendermint/account"
"github.com/tendermint/tendermint/binary" "github.com/tendermint/tendermint/binary"
blk "github.com/tendermint/tendermint/block"
"github.com/tendermint/tendermint/config" "github.com/tendermint/tendermint/config"
"github.com/tendermint/tendermint/types"
"bytes" "bytes"
"testing" "testing"
@ -52,9 +52,9 @@ func TestCopyState(t *testing.T) {
} }
} }
func makeBlock(t *testing.T, state *State, commits []blk.Commit, txs []blk.Tx) *blk.Block {
block := &blk.Block{
Header: &blk.Header{
func makeBlock(t *testing.T, state *State, commits []types.Commit, txs []types.Tx) *types.Block {
block := &types.Block{
Header: &types.Header{
Network: config.App().GetString("Network"), Network: config.App().GetString("Network"),
Height: state.LastBlockHeight + 1, Height: state.LastBlockHeight + 1,
Time: state.LastBlockTime.Add(time.Minute), Time: state.LastBlockTime.Add(time.Minute),
@ -64,10 +64,10 @@ func makeBlock(t *testing.T, state *State, commits []blk.Commit, txs []blk.Tx) *
LastBlockParts: state.LastBlockParts, LastBlockParts: state.LastBlockParts,
StateHash: nil, StateHash: nil,
}, },
Validation: &blk.Validation{
Validation: &types.Validation{
Commits: commits, Commits: commits,
}, },
Data: &blk.Data{
Data: &types.Data{
Txs: txs, Txs: txs,
}, },
} }
@ -91,7 +91,7 @@ func TestGenesisSaveLoad(t *testing.T) {
// Make complete block and blockParts // Make complete block and blockParts
block := makeBlock(t, s0, nil, nil) block := makeBlock(t, s0, nil, nil)
blockParts := blk.NewPartSetFromData(binary.BinaryBytes(block))
blockParts := types.NewPartSetFromData(binary.BinaryBytes(block))
// Now append the block to s0. // Now append the block to s0.
err := s0.AppendBlock(block, blockParts.Header()) err := s0.AppendBlock(block, blockParts.Header())
@ -157,18 +157,18 @@ func TestTxSequence(t *testing.T) {
acc1 := state.GetAccount(privAccounts[1].PubKey.Address()) acc1 := state.GetAccount(privAccounts[1].PubKey.Address())
// Try executing a SendTx with various sequence numbers. // Try executing a SendTx with various sequence numbers.
makeSendTx := func(sequence uint) *blk.SendTx {
return &blk.SendTx{
Inputs: []*blk.TxInput{
&blk.TxInput{
makeSendTx := func(sequence uint) *types.SendTx {
return &types.SendTx{
Inputs: []*types.TxInput{
&types.TxInput{
Address: acc0.Address, Address: acc0.Address,
Amount: 1, Amount: 1,
Sequence: sequence, Sequence: sequence,
PubKey: acc0PubKey, PubKey: acc0PubKey,
}, },
}, },
Outputs: []*blk.TxOutput{
&blk.TxOutput{
Outputs: []*types.TxOutput{
&types.TxOutput{
Address: acc1.Address, Address: acc1.Address,
Amount: 1, Amount: 1,
}, },
@ -224,17 +224,17 @@ func TestTxs(t *testing.T) {
// SendTx. // SendTx.
{ {
state := state.Copy() state := state.Copy()
tx := &blk.SendTx{
Inputs: []*blk.TxInput{
&blk.TxInput{
tx := &types.SendTx{
Inputs: []*types.TxInput{
&types.TxInput{
Address: acc0.Address, Address: acc0.Address,
Amount: 1, Amount: 1,
Sequence: acc0.Sequence + 1, Sequence: acc0.Sequence + 1,
PubKey: acc0PubKey, PubKey: acc0PubKey,
}, },
}, },
Outputs: []*blk.TxOutput{
&blk.TxOutput{
Outputs: []*types.TxOutput{
&types.TxOutput{
Address: acc1.Address, Address: acc1.Address,
Amount: 1, Amount: 1,
}, },
@ -261,18 +261,18 @@ func TestTxs(t *testing.T) {
// BondTx. // BondTx.
{ {
state := state.Copy() state := state.Copy()
tx := &blk.BondTx{
tx := &types.BondTx{
PubKey: acc0PubKey.(account.PubKeyEd25519), PubKey: acc0PubKey.(account.PubKeyEd25519),
Inputs: []*blk.TxInput{
&blk.TxInput{
Inputs: []*types.TxInput{
&types.TxInput{
Address: acc0.Address, Address: acc0.Address,
Amount: 1, Amount: 1,
Sequence: acc0.Sequence + 1, Sequence: acc0.Sequence + 1,
PubKey: acc0PubKey, PubKey: acc0PubKey,
}, },
}, },
UnbondTo: []*blk.TxOutput{
&blk.TxOutput{
UnbondTo: []*types.TxOutput{
&types.TxOutput{
Address: acc0.Address, Address: acc0.Address,
Amount: 1, Amount: 1,
}, },
@ -317,18 +317,18 @@ func TestAddValidator(t *testing.T) {
// The first privAccount will become a validator // The first privAccount will become a validator
acc0 := privAccounts[0] acc0 := privAccounts[0]
bondTx := &blk.BondTx{
bondTx := &types.BondTx{
PubKey: acc0.PubKey.(account.PubKeyEd25519), PubKey: acc0.PubKey.(account.PubKeyEd25519),
Inputs: []*blk.TxInput{
&blk.TxInput{
Inputs: []*types.TxInput{
&types.TxInput{
Address: acc0.Address, Address: acc0.Address,
Amount: 1000, Amount: 1000,
Sequence: 1, Sequence: 1,
PubKey: acc0.PubKey, PubKey: acc0.PubKey,
}, },
}, },
UnbondTo: []*blk.TxOutput{
&blk.TxOutput{
UnbondTo: []*types.TxOutput{
&types.TxOutput{
Address: acc0.Address, Address: acc0.Address,
Amount: 1000, Amount: 1000,
}, },
@ -337,8 +337,8 @@ func TestAddValidator(t *testing.T) {
bondTx.Inputs[0].Signature = acc0.Sign(bondTx) bondTx.Inputs[0].Signature = acc0.Sign(bondTx)
// Make complete block and blockParts // Make complete block and blockParts
block0 := makeBlock(t, s0, nil, []blk.Tx{bondTx})
block0Parts := blk.NewPartSetFromData(binary.BinaryBytes(block0))
block0 := makeBlock(t, s0, nil, []types.Tx{bondTx})
block0Parts := types.NewPartSetFromData(binary.BinaryBytes(block0))
// Sanity check // Sanity check
if s0.BondedValidators.Size() != 1 { if s0.BondedValidators.Size() != 1 {
@ -361,25 +361,25 @@ func TestAddValidator(t *testing.T) {
// The validation for the next block should only require 1 signature // The validation for the next block should only require 1 signature
// (the new validator wasn't active for block0) // (the new validator wasn't active for block0)
commit0 := &blk.Vote{
commit0 := &types.Vote{
Height: 1, Height: 1,
Round: 0, Round: 0,
Type: blk.VoteTypeCommit,
Type: types.VoteTypeCommit,
BlockHash: block0.Hash(), BlockHash: block0.Hash(),
BlockParts: block0Parts.Header(), BlockParts: block0Parts.Header(),
} }
privValidators[0].SignVote(commit0) privValidators[0].SignVote(commit0)
block1 := makeBlock(t, s0, block1 := makeBlock(t, s0,
[]blk.Commit{
blk.Commit{
[]types.Commit{
types.Commit{
Address: privValidators[0].Address, Address: privValidators[0].Address,
Round: 0, Round: 0,
Signature: commit0.Signature, Signature: commit0.Signature,
}, },
}, nil, }, nil,
) )
block1Parts := blk.NewPartSetFromData(binary.BinaryBytes(block1))
block1Parts := types.NewPartSetFromData(binary.BinaryBytes(block1))
err = s0.AppendBlock(block1, block1Parts.Header()) err = s0.AppendBlock(block1, block1Parts.Header())
if err != nil { if err != nil {
t.Error("Error appending secondary block:", err) t.Error("Error appending secondary block:", err)


+ 2
- 2
state/test.go View File

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


+ 2
- 2
state/validator.go View File

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


block/README.md → types/README.md View File


block/block.go → types/block.go View File


block/log.go → types/log.go View File


block/part_set.go → types/part_set.go View File


block/part_set_test.go → types/part_set_test.go View File


block/store.go → types/store.go View File


block/tx.go → types/tx.go View File


block/vote.go → types/vote.go View File


Loading…
Cancel
Save