diff --git a/blockchain/log.go b/blockchain/log.go new file mode 100644 index 000000000..f04e99943 --- /dev/null +++ b/blockchain/log.go @@ -0,0 +1,7 @@ +package blockchain + +import ( + "github.com/tendermint/tendermint/logger" +) + +var log = logger.New("module", "blockchain") diff --git a/blockchain/pool.go b/blockchain/pool.go new file mode 100644 index 000000000..528c66548 --- /dev/null +++ b/blockchain/pool.go @@ -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 +} diff --git a/blockchain/pool_test.go b/blockchain/pool_test.go new file mode 100644 index 000000000..4376f3ac0 --- /dev/null +++ b/blockchain/pool_test.go @@ -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() + +} diff --git a/blockchain/store.go b/blockchain/store.go new file mode 100644 index 000000000..8938273cb --- /dev/null +++ b/blockchain/store.go @@ -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 +} diff --git a/cmd/gen_tx.go b/cmd/gen_tx.go index f1fe13ea3..687339bed 100644 --- a/cmd/gen_tx.go +++ b/cmd/gen_tx.go @@ -10,10 +10,10 @@ import ( "github.com/tendermint/tendermint/account" "github.com/tendermint/tendermint/binary" - blk "github.com/tendermint/tendermint/block" . "github.com/tendermint/tendermint/common" dbm "github.com/tendermint/tendermint/db" sm "github.com/tendermint/tendermint/state" + "github.com/tendermint/tendermint/types" ) func getString(prompt string) string { @@ -80,9 +80,9 @@ func gen_tx() { dstSendAmount := getUint64(Fmt("Enter amount to send to %X: ", dstAddress)) // Construct SendTx - tx := &blk.SendTx{ - Inputs: []*blk.TxInput{ - &blk.TxInput{ + tx := &types.SendTx{ + Inputs: []*types.TxInput{ + &types.TxInput{ Address: srcAddress, Amount: srcSendAmount, Sequence: srcSendSequence, @@ -90,8 +90,8 @@ func gen_tx() { PubKey: srcPubKey, }, }, - Outputs: []*blk.TxOutput{ - &blk.TxOutput{ + Outputs: []*types.TxOutput{ + &types.TxOutput{ Address: dstAddress, Amount: dstSendAmount, }, diff --git a/consensus/pol.go b/consensus/pol.go index 3483b2f1b..c87b4ee5d 100644 --- a/consensus/pol.go +++ b/consensus/pol.go @@ -4,9 +4,9 @@ import ( "fmt" "github.com/tendermint/tendermint/account" - blk "github.com/tendermint/tendermint/block" . "github.com/tendermint/tendermint/common" sm "github.com/tendermint/tendermint/state" + "github.com/tendermint/tendermint/types" ) // Each signature of a POL (proof-of-lock, see whitepaper) is @@ -23,9 +23,9 @@ type POLVoteSignature struct { type POL struct { Height uint Round uint - BlockHash []byte // Could be nil, which makes this a proof of unlock. - BlockParts 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. @@ -37,8 +37,8 @@ func (pol *POL) Verify(valSet *sm.ValidatorSet) error { } 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, BlockParts: pol.BlockParts, }) @@ -54,8 +54,8 @@ func (pol *POL) Verify(valSet *sm.ValidatorSet) error { // Commit vote? if vote.Round < pol.Round { - voteDoc = account.SignBytes(&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, BlockParts: pol.BlockParts, }) diff --git a/consensus/pol_test.go b/consensus/pol_test.go index 5420e0f44..e1d67e4dd 100644 --- a/consensus/pol_test.go +++ b/consensus/pol_test.go @@ -2,9 +2,9 @@ package consensus import ( "github.com/tendermint/tendermint/binary" - blk "github.com/tendermint/tendermint/block" . "github.com/tendermint/tendermint/common" sm "github.com/tendermint/tendermint/state" + "github.com/tendermint/tendermint/types" "bytes" "testing" @@ -15,7 +15,7 @@ import ( // Convenience method. // Signs the vote and sets the POL's vote at the desired index // Returns the POLVoteSignature pointer, so you can modify it afterwards. -func signAddPOLVoteSignature(val *sm.PrivValidator, valSet *sm.ValidatorSet, vote *blk.Vote, pol *POL) *POLVoteSignature { +func signAddPOLVoteSignature(val *sm.PrivValidator, valSet *sm.ValidatorSet, vote *types.Vote, pol *POL) *POLVoteSignature { vote = vote.Copy() err := val.SignVote(vote) if err != nil { @@ -28,7 +28,7 @@ func signAddPOLVoteSignature(val *sm.PrivValidator, valSet *sm.ValidatorSet, vot func TestVerifyVotes(t *testing.T) { height, round := uint(1), uint(0) - _, valSet, privValidators := randVoteSet(height, round, blk.VoteTypePrevote, 10, 1) + _, valSet, privValidators := randVoteSet(height, round, types.VoteTypePrevote, 10, 1) // Make a POL with -2/3 votes. blockHash := RandBytes(32) @@ -36,8 +36,8 @@ func TestVerifyVotes(t *testing.T) { Height: height, Round: round, BlockHash: blockHash, Votes: make([]POLVoteSignature, valSet.Size()), } - voteProto := &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++ { signAddPOLVoteSignature(privValidators[i], valSet, voteProto, pol) @@ -59,7 +59,7 @@ func TestVerifyVotes(t *testing.T) { func TestVerifyInvalidVote(t *testing.T) { height, round := uint(1), uint(0) - _, valSet, privValidators := randVoteSet(height, round, blk.VoteTypePrevote, 10, 1) + _, valSet, privValidators := randVoteSet(height, round, types.VoteTypePrevote, 10, 1) // Make a POL with +2/3 votes with the wrong signature. blockHash := RandBytes(32) @@ -67,8 +67,8 @@ func TestVerifyInvalidVote(t *testing.T) { Height: height, Round: round, BlockHash: blockHash, Votes: make([]POLVoteSignature, valSet.Size()), } - voteProto := &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++ { polVoteSig := signAddPOLVoteSignature(privValidators[i], valSet, voteProto, pol) @@ -83,7 +83,7 @@ func TestVerifyInvalidVote(t *testing.T) { func TestVerifyCommits(t *testing.T) { height, round := uint(1), uint(2) - _, valSet, privValidators := randVoteSet(height, round, blk.VoteTypePrevote, 10, 1) + _, valSet, privValidators := randVoteSet(height, round, types.VoteTypePrevote, 10, 1) // Make a POL with +2/3 votes. blockHash := RandBytes(32) @@ -91,8 +91,8 @@ func TestVerifyCommits(t *testing.T) { Height: height, Round: round, BlockHash: blockHash, Votes: make([]POLVoteSignature, valSet.Size()), } - voteProto := &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++ { signAddPOLVoteSignature(privValidators[i], valSet, voteProto, pol) @@ -106,7 +106,7 @@ func TestVerifyCommits(t *testing.T) { func TestVerifyInvalidCommits(t *testing.T) { height, round := uint(1), uint(2) - _, valSet, privValidators := randVoteSet(height, round, blk.VoteTypePrevote, 10, 1) + _, valSet, privValidators := randVoteSet(height, round, types.VoteTypePrevote, 10, 1) // Make a POL with +2/3 votes with the wrong signature. blockHash := RandBytes(32) @@ -114,8 +114,8 @@ func TestVerifyInvalidCommits(t *testing.T) { Height: height, Round: round, BlockHash: blockHash, Votes: make([]POLVoteSignature, valSet.Size()), } - voteProto := &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++ { polVoteSig := signAddPOLVoteSignature(privValidators[i], valSet, voteProto, pol) @@ -130,7 +130,7 @@ func TestVerifyInvalidCommits(t *testing.T) { func TestVerifyInvalidCommitRounds(t *testing.T) { height, round := uint(1), uint(2) - _, valSet, privValidators := randVoteSet(height, round, blk.VoteTypePrevote, 10, 1) + _, valSet, privValidators := randVoteSet(height, round, types.VoteTypePrevote, 10, 1) // Make a POL with +2/3 commits for the current round. blockHash := RandBytes(32) @@ -138,8 +138,8 @@ func TestVerifyInvalidCommitRounds(t *testing.T) { Height: height, Round: round, BlockHash: blockHash, Votes: make([]POLVoteSignature, valSet.Size()), } - voteProto := &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++ { signAddPOLVoteSignature(privValidators[i], valSet, voteProto, pol) @@ -153,7 +153,7 @@ func TestVerifyInvalidCommitRounds(t *testing.T) { func TestVerifyInvalidCommitRounds2(t *testing.T) { height, round := uint(1), uint(2) - _, valSet, privValidators := randVoteSet(height, round, blk.VoteTypePrevote, 10, 1) + _, valSet, privValidators := randVoteSet(height, round, types.VoteTypePrevote, 10, 1) // Make a POL with +2/3 commits for future round. blockHash := RandBytes(32) @@ -161,8 +161,8 @@ func TestVerifyInvalidCommitRounds2(t *testing.T) { Height: height, Round: round, BlockHash: blockHash, Votes: make([]POLVoteSignature, valSet.Size()), } - voteProto := &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++ { polVoteSig := signAddPOLVoteSignature(privValidators[i], valSet, voteProto, pol) @@ -177,7 +177,7 @@ func TestVerifyInvalidCommitRounds2(t *testing.T) { func TestReadWrite(t *testing.T) { height, round := uint(1), uint(2) - _, valSet, privValidators := randVoteSet(height, round, blk.VoteTypePrevote, 10, 1) + _, valSet, privValidators := randVoteSet(height, round, types.VoteTypePrevote, 10, 1) // Make a POL with +2/3 votes. blockHash := RandBytes(32) @@ -185,8 +185,8 @@ func TestReadWrite(t *testing.T) { Height: height, Round: round, BlockHash: blockHash, Votes: make([]POLVoteSignature, valSet.Size()), } - voteProto := &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++ { signAddPOLVoteSignature(privValidators[i], valSet, voteProto, pol) diff --git a/consensus/reactor.go b/consensus/reactor.go index 9b3f37de6..83c244995 100644 --- a/consensus/reactor.go +++ b/consensus/reactor.go @@ -9,11 +9,11 @@ import ( "time" "github.com/tendermint/tendermint/binary" - blk "github.com/tendermint/tendermint/block" . "github.com/tendermint/tendermint/common" . "github.com/tendermint/tendermint/consensus/types" "github.com/tendermint/tendermint/p2p" sm "github.com/tendermint/tendermint/state" + "github.com/tendermint/tendermint/types" ) const ( @@ -34,11 +34,11 @@ type ConsensusReactor struct { stopped uint32 quit chan struct{} - blockStore *blk.BlockStore + blockStore *types.BlockStore conS *ConsensusState } -func NewConsensusReactor(consensusState *ConsensusState, blockStore *blk.BlockStore) *ConsensusReactor { +func NewConsensusReactor(consensusState *ConsensusState, blockStore *types.BlockStore) *ConsensusReactor { conR := &ConsensusReactor{ blockStore: blockStore, 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) if err != nil { // 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") - evidenceTx := &blk.DupeoutTx{ + evidenceTx := &types.DupeoutTx{ Address: address, VoteA: *errDupe.VoteA, VoteB: *errDupe.VoteB, @@ -404,7 +404,7 @@ OUTER_LOOP: 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 ps.EnsureVoteBitArrays(prs.Height, uint(len(validation.Commits))) @@ -412,10 +412,10 @@ OUTER_LOOP: commit := validation.Commits[index] log.Debug("Picked commit to send", "index", index, "commit", commit) // Reconstruct vote. - vote := &blk.Vote{ + vote := &types.Vote{ Height: prs.Height, Round: commit.Round, - Type: blk.VoteTypeCommit, + Type: types.VoteTypeCommit, BlockHash: blockMeta.Hash, BlockParts: blockMeta.Parts, Signature: commit.Signature, @@ -515,20 +515,20 @@ OUTER_LOOP: // Read only when returned by PeerState.GetRoundState(). type PeerRoundState struct { - Height uint // Height peer is at - Round uint // Round peer is at - Step RoundStep // Step peer is at - StartTime time.Time // Estimated start of round 0 at this height - Proposal bool // True if peer has proposal for this round - ProposalBlockParts 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() 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) { - if ps.Height == height+1 && type_ == blk.VoteTypeCommit { + if ps.Height == height+1 && type_ == types.VoteTypeCommit { // Special case for LastCommits. ps.LastCommits.SetIndex(index, true) return @@ -634,11 +634,11 @@ func (ps *PeerState) setHasVote(height uint, round uint, type_ byte, index uint) } switch type_ { - case blk.VoteTypePrevote: + case types.VoteTypePrevote: ps.Prevotes.SetIndex(index, true) - case blk.VoteTypePrecommit: + case types.VoteTypePrecommit: ps.Precommits.SetIndex(index, true) - case blk.VoteTypeCommit: + case types.VoteTypeCommit: if round < ps.Round { ps.Prevotes.SetIndex(index, true) ps.Precommits.SetIndex(index, true) @@ -676,9 +676,9 @@ func (ps *PeerState) ApplyNewRoundStepMessage(msg *NewRoundStepMessage, rs *Roun ps.StartTime = startTime if psHeight != msg.Height || psRound != msg.Round { ps.Proposal = false - ps.ProposalBlockParts = blk.PartSetHeader{} + ps.ProposalBlockParts = types.PartSetHeader{} ps.ProposalBlockBitArray = BitArray{} - ps.ProposalPOLParts = blk.PartSetHeader{} + ps.ProposalPOLParts = types.PartSetHeader{} ps.ProposalPOLBitArray = BitArray{} // We'll update the BitArray capacity later. ps.Prevotes = BitArray{} @@ -714,7 +714,7 @@ func (ps *PeerState) ApplyHasVoteMessage(msg *HasVoteMessage) { defer ps.mtx.Unlock() // 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) return } else if ps.Height != msg.Height { @@ -784,7 +784,7 @@ func (m *NewRoundStepMessage) String() string { type CommitStepMessage struct { Height uint - BlockParts blk.PartSetHeader + BlockParts types.PartSetHeader BlockBitArray BitArray } @@ -805,7 +805,7 @@ type PartMessage struct { Height uint Round uint Type byte - Part *blk.Part + Part *types.Part } func (m *PartMessage) TypeByte() byte { return msgTypePart } @@ -818,7 +818,7 @@ func (m *PartMessage) String() string { type VoteMessage struct { ValidatorIndex uint - Vote *blk.Vote + Vote *types.Vote } func (m *VoteMessage) TypeByte() byte { return msgTypeVote } diff --git a/consensus/state.go b/consensus/state.go index 5d5ac8f96..d44dd7b5a 100644 --- a/consensus/state.go +++ b/consensus/state.go @@ -62,12 +62,12 @@ import ( "github.com/tendermint/tendermint/account" "github.com/tendermint/tendermint/binary" - blk "github.com/tendermint/tendermint/block" . "github.com/tendermint/tendermint/common" "github.com/tendermint/tendermint/config" . "github.com/tendermint/tendermint/consensus/types" mempl "github.com/tendermint/tendermint/mempool" sm "github.com/tendermint/tendermint/state" + "github.com/tendermint/tendermint/types" ) const ( @@ -171,12 +171,12 @@ type RoundState struct { CommitTime time.Time // Time when +2/3 commits were found Validators *sm.ValidatorSet Proposal *Proposal - ProposalBlock *blk.Block - ProposalBlockParts *blk.PartSet + ProposalBlock *types.Block + ProposalBlockParts *types.PartSet 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. Prevotes *VoteSet Precommits *VoteSet @@ -234,20 +234,20 @@ type ConsensusState struct { stopped uint32 quit chan struct{} - blockStore *blk.BlockStore + blockStore *types.BlockStore mempoolReactor *mempl.MempoolReactor runActionCh chan RoundAction newStepCh chan *RoundState mtx sync.Mutex 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{ quit: make(chan struct{}), blockStore: blockStore, @@ -484,10 +484,10 @@ func (cs *ConsensusState) updateToState(state *sm.State) { cs.LockedBlock = nil cs.LockedBlockParts = nil cs.LockedPOL = nil - cs.Prevotes = NewVoteSet(height, 0, 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.Commits = NewVoteSet(height, 0, blk.VoteTypeCommit, validators) + cs.Commits = NewVoteSet(height, 0, types.VoteTypeCommit, validators) cs.state = state cs.stagedBlock = nil @@ -501,7 +501,7 @@ func (cs *ConsensusState) updateToState(state *sm.State) { // If we've timed out, then send rebond tx. if cs.PrivValidator != nil && cs.state.UnbondingValidators.HasAddress(cs.PrivValidator.Address) { - rebondTx := &blk.RebondTx{ + rebondTx := &types.RebondTx{ Address: cs.PrivValidator.Address, Height: cs.Height + 1, } @@ -534,9 +534,9 @@ func (cs *ConsensusState) setupNewRound(round uint) { cs.ProposalBlockParts = nil cs.ProposalPOL = nil cs.ProposalPOLParts = nil - cs.Prevotes = NewVoteSet(cs.Height, round, blk.VoteTypePrevote, validators) + cs.Prevotes = NewVoteSet(cs.Height, round, types.VoteTypePrevote, validators) 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) } @@ -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) } - var block *blk.Block - var blockParts *blk.PartSet + var block *types.Block + var blockParts *types.PartSet var pol *POL - var polParts *blk.PartSet + var polParts *types.PartSet // Decide on block and POL if cs.LockedBlock != nil { @@ -599,11 +599,11 @@ func (cs *ConsensusState) RunActionPropose(height uint, round uint) { pol = cs.LockedPOL } else { // Otherwise we should create a new proposal. - var validation *blk.Validation + var validation *types.Validation if cs.Height == 1 { // We're creating a proposal for the first block. // The validation is empty. - validation = &blk.Validation{} + validation = &types.Validation{} } else if cs.LastCommits.HasTwoThirdsMajority() { // Make the validation from LastCommits validation = cs.LastCommits.MakeValidation() @@ -617,8 +617,8 @@ func (cs *ConsensusState) RunActionPropose(height uint, round uint) { } } txs := cs.mempoolReactor.Mempool.GetProposalTxs() - block = &blk.Block{ - Header: &blk.Header{ + block = &types.Block{ + Header: &types.Header{ Network: config.App().GetString("Network"), Height: cs.Height, Time: time.Now(), @@ -629,24 +629,24 @@ func (cs *ConsensusState) RunActionPropose(height uint, round uint) { StateHash: nil, // Will set afterwards. }, Validation: validation, - Data: &blk.Data{ + Data: &types.Data{ Txs: txs, }, } - // Set the blk.Header.StateHash. + // Set the types.Header.StateHash. err := cs.state.SetBlockStateHash(block) if err != nil { log.Error("Error setting state hash", "error", err) return } - blockParts = blk.NewPartSetFromData(binary.BinaryBytes(block)) + blockParts = types.NewPartSetFromData(binary.BinaryBytes(block)) pol = cs.LockedPOL // If exists, is a PoUnlock. } if pol != nil { - polParts = blk.NewPartSetFromData(binary.BinaryBytes(pol)) + polParts = types.NewPartSetFromData(binary.BinaryBytes(pol)) } // Make proposal @@ -681,14 +681,14 @@ func (cs *ConsensusState) RunActionPrevote(height uint, round uint) { // If a block is locked, prevote that. 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 } // If ProposalBlock is nil, prevote nil. if cs.ProposalBlock == nil { log.Warn("ProposalBlock is nil") - cs.signAddVote(blk.VoteTypePrevote, nil, blk.PartSetHeader{}) + cs.signAddVote(types.VoteTypePrevote, nil, types.PartSetHeader{}) return } @@ -697,12 +697,12 @@ func (cs *ConsensusState) RunActionPrevote(height uint, round uint) { if err != nil { // ProposalBlock is invalid, prevote nil. log.Warn("ProposalBlock is invalid", "error", err) - cs.signAddVote(blk.VoteTypePrevote, nil, blk.PartSetHeader{}) + cs.signAddVote(types.VoteTypePrevote, nil, types.PartSetHeader{}) return } // Prevote cs.ProposalBlock - cs.signAddVote(blk.VoteTypePrevote, cs.ProposalBlock.Hash(), cs.ProposalBlockParts.Header()) + cs.signAddVote(types.VoteTypePrevote, cs.ProposalBlock.Hash(), cs.ProposalBlockParts.Header()) return } @@ -738,7 +738,7 @@ func (cs *ConsensusState) RunActionPrecommit(height uint, round uint) { // If +2/3 prevoted for already locked block, precommit it. if cs.LockedBlock.HashesTo(hash) { - cs.signAddVote(blk.VoteTypePrecommit, hash, partsHeader) + cs.signAddVote(types.VoteTypePrecommit, hash, partsHeader) return } @@ -752,7 +752,7 @@ func (cs *ConsensusState) RunActionPrecommit(height uint, round uint) { } cs.LockedBlock = cs.ProposalBlock cs.LockedBlockParts = cs.ProposalBlockParts - cs.signAddVote(blk.VoteTypePrecommit, hash, partsHeader) + cs.signAddVote(types.VoteTypePrecommit, hash, partsHeader) return } @@ -806,7 +806,7 @@ func (cs *ConsensusState) RunActionCommit(height uint) { // We're getting the wrong block. // Set up ProposalBlockParts and keep waiting. cs.ProposalBlock = nil - cs.ProposalBlockParts = blk.NewPartSetFromHeader(partsHeader) + cs.ProposalBlockParts = types.NewPartSetFromHeader(partsHeader) } else { // We just need to keep waiting. @@ -896,14 +896,14 @@ func (cs *ConsensusState) SetProposal(proposal *Proposal) error { } 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 } // NOTE: block is not necessarily valid. // 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() defer cs.mtx.Unlock() @@ -924,7 +924,7 @@ func (cs *ConsensusState) AddProposalBlockPart(height uint, round uint, part *bl if added && cs.ProposalBlockParts.IsComplete() { var n int64 var err error - cs.ProposalBlock = binary.ReadBinary(&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 cs.Step == RoundStepCommit { 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. -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() defer cs.mtx.Unlock() @@ -962,7 +962,7 @@ func (cs *ConsensusState) AddProposalPOLPart(height uint, round uint, part *blk. 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() 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 { - case blk.VoteTypePrevote: + case types.VoteTypePrevote: // Prevotes checks for height+round match. return cs.Prevotes.Add(address, vote) - case blk.VoteTypePrecommit: + case types.VoteTypePrecommit: // Precommits checks for height+round match. return cs.Precommits.Add(address, vote) - case blk.VoteTypeCommit: + case types.VoteTypeCommit: if vote.Height == cs.Height { // No need to check if vote.Round < cs.Round ... // 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 { 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) { return nil } - vote := &blk.Vote{ + vote := &types.Vote{ Height: cs.Height, Round: cs.Round, Type: type_, @@ -1054,7 +1054,7 @@ func (cs *ConsensusState) signAddVote(type_ byte, hash []byte, header blk.PartSe } // 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. if err := cs.stageBlock(block, blockParts); err != nil { @@ -1065,16 +1065,16 @@ func (cs *ConsensusState) commitVoteBlock(block *blk.Block, blockParts *blk.Part // Commit-vote. 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 } 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, // 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. 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. 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 } } diff --git a/consensus/state_test.go b/consensus/state_test.go index 423f09aa0..60e54a692 100644 --- a/consensus/state_test.go +++ b/consensus/state_test.go @@ -4,7 +4,7 @@ import ( "bytes" "testing" - blk "github.com/tendermint/tendermint/block" + "github.com/tendermint/tendermint/types" ) func TestSetupRound(t *testing.T) { @@ -12,9 +12,9 @@ func TestSetupRound(t *testing.T) { val0 := privValidators[0] // Add a vote, precommit, and commit by val0. - voteTypes := []byte{blk.VoteTypePrevote, blk.VoteTypePrecommit, blk.VoteTypeCommit} + voteTypes := []byte{types.VoteTypePrevote, types.VoteTypePrecommit, types.VoteTypeCommit} 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) if err != nil { t.Error("Error signing vote: %v", err) @@ -24,13 +24,13 @@ func TestSetupRound(t *testing.T) { // Ensure that vote appears in RoundState. rs0 := cs.GetRoundState() - if vote := rs0.Prevotes.GetByAddress(val0.Address); vote == nil || vote.Type != 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) } - 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) } - 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) } @@ -40,13 +40,13 @@ func TestSetupRound(t *testing.T) { // Now the commit should be copied over to prevotes and precommits. rs1 := cs.GetRoundState() - if vote := rs1.Prevotes.GetByAddress(val0.Address); vote == nil || vote.Type != 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) } - 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) } - 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) } @@ -116,10 +116,10 @@ func TestRunActionPrecommitCommitFinalize(t *testing.T) { // Add at least +2/3 prevotes. for i := 0; i < 7; i++ { - vote := &blk.Vote{ + vote := &types.Vote{ Height: 1, Round: 0, - Type: blk.VoteTypePrevote, + Type: types.VoteTypePrevote, BlockHash: cs.ProposalBlock.Hash(), BlockParts: cs.ProposalBlockParts.Header(), } @@ -146,10 +146,10 @@ func TestRunActionPrecommitCommitFinalize(t *testing.T) { } continue } - vote := &blk.Vote{ + vote := &types.Vote{ Height: 1, Round: 0, - Type: blk.VoteTypePrecommit, + Type: types.VoteTypePrecommit, BlockHash: cs.ProposalBlock.Hash(), BlockParts: cs.ProposalBlockParts.Header(), } @@ -184,10 +184,10 @@ func TestRunActionPrecommitCommitFinalize(t *testing.T) { } continue } - vote := &blk.Vote{ + vote := &types.Vote{ Height: 1, Round: uint(i), // Doesn't matter what round - Type: blk.VoteTypeCommit, + Type: types.VoteTypeCommit, BlockHash: cs.ProposalBlock.Hash(), BlockParts: cs.ProposalBlockParts.Header(), } diff --git a/consensus/test.go b/consensus/test.go index 2ccdeb3b7..e86c6a075 100644 --- a/consensus/test.go +++ b/consensus/test.go @@ -3,15 +3,15 @@ package consensus import ( "sort" - blk "github.com/tendermint/tendermint/block" dbm "github.com/tendermint/tendermint/db" mempl "github.com/tendermint/tendermint/mempool" sm "github.com/tendermint/tendermint/state" + "github.com/tendermint/tendermint/types" ) func randConsensusState() (*ConsensusState, []*sm.PrivValidator) { state, _, privValidators := sm.RandGenesisState(20, false, 1000, 10, false, 1000) - blockStore := blk.NewBlockStore(dbm.NewMemDB()) + blockStore := types.NewBlockStore(dbm.NewMemDB()) mempool := mempl.NewMempool(state) mempoolReactor := mempl.NewMempoolReactor(mempool) cs := NewConsensusState(state, blockStore, mempoolReactor) diff --git a/consensus/types/proposal.go b/consensus/types/proposal.go index 22a3963e4..a78570a53 100644 --- a/consensus/types/proposal.go +++ b/consensus/types/proposal.go @@ -7,7 +7,7 @@ import ( "github.com/tendermint/tendermint/account" "github.com/tendermint/tendermint/binary" - blk "github.com/tendermint/tendermint/block" + "github.com/tendermint/tendermint/types" ) var ( @@ -18,12 +18,12 @@ var ( type Proposal struct { Height uint Round uint - BlockParts blk.PartSetHeader - POLParts blk.PartSetHeader + BlockParts types.PartSetHeader + POLParts types.PartSetHeader 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{ Height: height, Round: round, diff --git a/consensus/vote_set.go b/consensus/vote_set.go index 627d0e2bb..c640fc51b 100644 --- a/consensus/vote_set.go +++ b/consensus/vote_set.go @@ -8,9 +8,9 @@ import ( "github.com/tendermint/tendermint/account" "github.com/tendermint/tendermint/binary" - blk "github.com/tendermint/tendermint/block" . "github.com/tendermint/tendermint/common" sm "github.com/tendermint/tendermint/state" + "github.com/tendermint/tendermint/types" ) // VoteSet helps collect signatures from validators at each height+round @@ -25,12 +25,12 @@ type VoteSet struct { mtx sync.Mutex valSet *sm.ValidatorSet - votes []*blk.Vote // validator index -> vote + votes []*types.Vote // validator index -> vote votesBitArray BitArray // validator index -> has vote? votesByBlock map[string]uint64 // string(blockHash)+string(blockParts) -> vote sum. totalVotes uint64 maj23Hash []byte - maj23Parts blk.PartSetHeader + maj23Parts types.PartSetHeader maj23Exists bool } @@ -39,7 +39,7 @@ func NewVoteSet(height uint, round uint, type_ byte, valSet *sm.ValidatorSet) *V if height == 0 { panic("Cannot make VoteSet for height == 0, doesn't make sense.") } - if type_ == blk.VoteTypeCommit && round != 0 { + if type_ == types.VoteTypeCommit && round != 0 { panic("Expected round 0 for commit vote set") } return &VoteSet{ @@ -47,7 +47,7 @@ func NewVoteSet(height uint, round uint, type_ byte, valSet *sm.ValidatorSet) *V round: round, type_: type_, valSet: valSet, - votes: make([]*blk.Vote, valSet.Size()), + votes: make([]*types.Vote, valSet.Size()), votesBitArray: NewBitArray(valSet.Size()), votesByBlock: make(map[string]uint64), totalVotes: 0, @@ -73,40 +73,40 @@ func (voteSet *VoteSet) Size() uint { // True if added, false if not. // Returns ErrVote[UnexpectedStep|InvalidAccount|InvalidSignature|InvalidBlockHash|ConflictingSignature] // NOTE: vote should not be mutated after adding. -func (voteSet *VoteSet) Add(address []byte, vote *blk.Vote) (bool, uint, error) { +func (voteSet *VoteSet) Add(address []byte, vote *types.Vote) (bool, uint, error) { voteSet.mtx.Lock() defer voteSet.mtx.Unlock() // Make sure the step matches. (or that vote is commit && round < voteSet.round) if vote.Height != voteSet.height || - (vote.Type != 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. valIndex, val := voteSet.valSet.GetByAddress(address) if val == nil { - return false, 0, blk.ErrVoteInvalidAccount + return false, 0, types.ErrVoteInvalidAccount } // Check signature. if !val.PubKey.VerifyBytes(account.SignBytes(vote), vote.Signature) { // Bad signature. - return false, 0, blk.ErrVoteInvalidSignature + return false, 0, types.ErrVoteInvalidSignature } 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 existingVote := voteSet.votes[valIndex]; existingVote != nil { if bytes.Equal(existingVote.BlockHash, vote.BlockHash) { return false, 0, nil } else { - return false, 0, &blk.ErrVoteConflictingSignature{ + return false, 0, &types.ErrVoteConflictingSignature{ VoteA: existingVote, VoteB: vote, } @@ -157,13 +157,13 @@ func (voteSet *VoteSet) BitArray() BitArray { return voteSet.votesBitArray.Copy() } -func (voteSet *VoteSet) GetByIndex(valIndex uint) *blk.Vote { +func (voteSet *VoteSet) GetByIndex(valIndex uint) *types.Vote { voteSet.mtx.Lock() defer voteSet.mtx.Unlock() return voteSet.votes[valIndex] } -func (voteSet *VoteSet) GetByAddress(address []byte) *blk.Vote { +func (voteSet *VoteSet) GetByAddress(address []byte) *types.Vote { voteSet.mtx.Lock() defer voteSet.mtx.Unlock() 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. // 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() defer voteSet.mtx.Unlock() if voteSet.maj23Exists { return voteSet.maj23Hash, voteSet.maj23Parts, true } else { - return nil, blk.PartSetHeader{}, false + return nil, types.PartSetHeader{}, false } } 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() defer voteSet.mtx.Unlock() @@ -228,16 +228,16 @@ func (voteSet *VoteSet) MakePOL() *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() defer voteSet.mtx.Unlock() if len(voteSet.maj23Hash) == 0 { 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 { vote := voteSet.votes[valIndex] if vote == nil { @@ -249,10 +249,10 @@ func (voteSet *VoteSet) MakeValidation() *blk.Validation { if !vote.BlockParts.Equals(voteSet.maj23Parts) { 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 &blk.Validation{ + return &types.Validation{ Commits: commits, } } diff --git a/consensus/vote_set_test.go b/consensus/vote_set_test.go index db7a271b5..c18892f23 100644 --- a/consensus/vote_set_test.go +++ b/consensus/vote_set_test.go @@ -3,10 +3,10 @@ package consensus import ( "bytes" - blk "github.com/tendermint/tendermint/block" . "github.com/tendermint/tendermint/common" . "github.com/tendermint/tendermint/common/test" sm "github.com/tendermint/tendermint/state" + "github.com/tendermint/tendermint/types" "testing" ) @@ -14,41 +14,41 @@ import ( // NOTE: see consensus/test.go for common test methods. // 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.Height = height return vote } // 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.Round = round return vote } // 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.Type = type_ return vote } // 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.BlockHash = blockHash return vote } // 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.BlockParts = blockParts 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) added, _, err := voteSet.Add(privVal.Address, vote) return added, err @@ -56,7 +56,7 @@ func signAddVote(privVal *sm.PrivValidator, vote *blk.Vote, voteSet *VoteSet) (b func TestAddVote(t *testing.T) { 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] // t.Logf(">> %v", voteSet) @@ -72,7 +72,7 @@ func TestAddVote(t *testing.T) { 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) if voteSet.GetByAddress(val0.Address) == nil { @@ -89,9 +89,9 @@ func TestAddVote(t *testing.T) { func Test2_3Majority(t *testing.T) { height, round := uint(1), uint(0) - voteSet, _, privValidators := randVoteSet(height, round, 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. for i := 0; i < 6; i++ { @@ -123,13 +123,13 @@ func Test2_3Majority(t *testing.T) { func Test2_3MajorityRedux(t *testing.T) { height, round := uint(1), uint(0) - voteSet, _, privValidators := randVoteSet(height, round, blk.VoteTypePrevote, 100, 1) + voteSet, _, privValidators := randVoteSet(height, round, types.VoteTypePrevote, 100, 1) blockHash := CRandBytes(32) 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. for i := 0; i < 66; i++ { @@ -151,7 +151,7 @@ func Test2_3MajorityRedux(t *testing.T) { // 68th validator voted for a different BlockParts PartSetHeader { - blockParts := blk.PartSetHeader{blockPartsTotal, CRandBytes(32)} + blockParts := types.PartSetHeader{blockPartsTotal, CRandBytes(32)} signAddVote(privValidators[67], withBlockParts(vote, blockParts), voteSet) hash, header, ok = voteSet.TwoThirdsMajority() if hash != nil || !header.IsZero() || ok { @@ -161,7 +161,7 @@ func Test2_3MajorityRedux(t *testing.T) { // 69th validator voted for different BlockParts Total { - blockParts := blk.PartSetHeader{blockPartsTotal + 1, blockParts.Hash} + blockParts := types.PartSetHeader{blockPartsTotal + 1, blockParts.Hash} signAddVote(privValidators[68], withBlockParts(vote, blockParts), voteSet) hash, header, ok = voteSet.TwoThirdsMajority() if hash != nil || !header.IsZero() || ok { @@ -190,10 +190,10 @@ func Test2_3MajorityRedux(t *testing.T) { func TestBadVotes(t *testing.T) { height, round := uint(1), uint(0) - voteSet, _, privValidators := randVoteSet(height, round, blk.VoteTypePrevote, 10, 1) + voteSet, _, privValidators := randVoteSet(height, round, types.VoteTypePrevote, 10, 1) // 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) if !added || err != nil { t.Errorf("Expected Add() to succeed") @@ -218,7 +218,7 @@ func TestBadVotes(t *testing.T) { } // val3 votes of another type. - added, err = signAddVote(privValidators[3], withType(vote, blk.VoteTypePrecommit), voteSet) + added, err = signAddVote(privValidators[3], withType(vote, types.VoteTypePrecommit), voteSet) if added { t.Errorf("Expected Add() to fail, wrong type") } @@ -226,10 +226,10 @@ func TestBadVotes(t *testing.T) { func TestAddCommitsToPrevoteVotes(t *testing.T) { height, round := uint(2), uint(5) - voteSet, _, privValidators := randVoteSet(height, round, blk.VoteTypePrevote, 10, 1) + voteSet, _, privValidators := randVoteSet(height, round, types.VoteTypePrevote, 10, 1) // 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++ { signAddVote(privValidators[i], vote, voteSet) } @@ -239,35 +239,35 @@ func TestAddCommitsToPrevoteVotes(t *testing.T) { } // Attempt to add a commit from val6 at a previous height - vote = &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) if added { t.Errorf("Expected Add() to fail, wrong height.") } // 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) if added { t.Errorf("Expected Add() to fail, cannot add future round vote.") } // Attempt to add a commit from val6 for currrent height/round. - vote = &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) if added || err == nil { t.Errorf("Expected Add() to fail, only prior round commits can be added.") } // Add commit from val6 at a previous round - vote = &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) if !added || err != nil { t.Errorf("Expected Add() to succeed, commit for prior rounds are relevant.") } // Also add commit from val7 for previous round. - vote = &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) if !added || err != nil { t.Errorf("Expected Add() to succeed. err: %v", err) @@ -283,10 +283,10 @@ func TestAddCommitsToPrevoteVotes(t *testing.T) { func TestMakeValidation(t *testing.T) { height, round := uint(1), uint(0) - voteSet, _, privValidators := randVoteSet(height, round, 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} // 6 out of 10 voted for some block. @@ -300,7 +300,7 @@ func TestMakeValidation(t *testing.T) { // 7th voted for some other block. { vote := withBlockHash(vote, RandBytes(32)) - vote = withBlockParts(vote, blk.PartSetHeader{123, RandBytes(32)}) + vote = withBlockParts(vote, types.PartSetHeader{123, RandBytes(32)}) signAddVote(privValidators[6], vote, voteSet) } diff --git a/daemon/daemon.go b/daemon/daemon.go index 72a615c20..ccb2932de 100644 --- a/daemon/daemon.go +++ b/daemon/daemon.go @@ -4,7 +4,6 @@ import ( "os" "os/signal" - blk "github.com/tendermint/tendermint/block" . "github.com/tendermint/tendermint/common" "github.com/tendermint/tendermint/config" "github.com/tendermint/tendermint/consensus" @@ -13,6 +12,7 @@ import ( "github.com/tendermint/tendermint/p2p" "github.com/tendermint/tendermint/rpc" sm "github.com/tendermint/tendermint/state" + "github.com/tendermint/tendermint/types" ) type Node struct { @@ -20,7 +20,7 @@ type Node struct { sw *p2p.Switch book *p2p.AddrBook pexReactor *p2p.PEXReactor - blockStore *blk.BlockStore + blockStore *types.BlockStore mempoolReactor *mempl.MempoolReactor consensusState *consensus.ConsensusState consensusReactor *consensus.ConsensusReactor @@ -30,7 +30,7 @@ type Node struct { func NewNode() *Node { // Get BlockStore blockStoreDB := dbm.GetDB("blockstore") - blockStore := blk.NewBlockStore(blockStoreDB) + blockStore := types.NewBlockStore(blockStoreDB) // Get State stateDB := dbm.GetDB("state") diff --git a/mempool/mempool.go b/mempool/mempool.go index 4743e6380..e955dfacf 100644 --- a/mempool/mempool.go +++ b/mempool/mempool.go @@ -12,14 +12,14 @@ import ( "sync" "github.com/tendermint/tendermint/binary" - blk "github.com/tendermint/tendermint/block" sm "github.com/tendermint/tendermint/state" + "github.com/tendermint/tendermint/types" ) type Mempool struct { mtx sync.Mutex state *sm.State - txs []blk.Tx + txs []types.Tx } func NewMempool(state *sm.State) *Mempool { @@ -33,7 +33,7 @@ func (mem *Mempool) GetState() *sm.State { } // 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() defer mem.mtx.Unlock() 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() defer mem.mtx.Unlock() log.Debug("GetProposalTxs:", "txs", mem.txs) @@ -58,7 +58,7 @@ func (mem *Mempool) GetProposalTxs() []blk.Tx { // "state" is the result of state.AppendBlock("block"). // Txs that are present in "block" are discarded from mempool. // Txs that have become invalid in the new "state" are also discarded. -func (mem *Mempool) ResetForBlockAndState(block *blk.Block, state *sm.State) { +func (mem *Mempool) ResetForBlockAndState(block *types.Block, state *sm.State) { mem.mtx.Lock() defer mem.mtx.Unlock() 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 - txs := []blk.Tx{} + txs := []types.Tx{} for _, tx := range mem.txs { txHash := binary.BinarySha256(tx) 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. - validTxs := []blk.Tx{} + validTxs := []types.Tx{} for _, tx := range txs { err := mem.state.ExecTx(tx, false) if err == nil { diff --git a/mempool/reactor.go b/mempool/reactor.go index 7381d1ad2..5bed4e18b 100644 --- a/mempool/reactor.go +++ b/mempool/reactor.go @@ -6,8 +6,8 @@ import ( "sync/atomic" "github.com/tendermint/tendermint/binary" - blk "github.com/tendermint/tendermint/block" "github.com/tendermint/tendermint/p2p" + "github.com/tendermint/tendermint/types" ) 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) if err != nil { return err @@ -135,7 +135,7 @@ func DecodeMessage(bz []byte) (msgType byte, msg interface{}, err error) { //------------------------------------- type TxMessage struct { - Tx blk.Tx + Tx types.Tx } func (m *TxMessage) TypeByte() byte { return msgTypeTx } diff --git a/rpc/blocks.go b/rpc/blocks.go index 2703767f1..1415f1ff0 100644 --- a/rpc/blocks.go +++ b/rpc/blocks.go @@ -3,8 +3,8 @@ package rpc import ( "net/http" - blk "github.com/tendermint/tendermint/block" . "github.com/tendermint/tendermint/common" + "github.com/tendermint/tendermint/types" ) 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) - blockMetas := []*blk.BlockMeta{} + blockMetas := []*types.BlockMeta{} for height := maxHeight; height >= minHeight; height-- { blockMeta := blockStore.LoadBlockMeta(height) blockMetas = append(blockMetas, blockMeta) @@ -28,7 +28,7 @@ func BlockchainInfoHandler(w http.ResponseWriter, r *http.Request) { WriteAPIResponse(w, API_OK, struct { LastHeight uint - BlockMetas []*blk.BlockMeta + BlockMetas []*types.BlockMeta }{blockStore.Height(), blockMetas}) } @@ -49,7 +49,7 @@ func GetBlockHandler(w http.ResponseWriter, r *http.Request) { block := blockStore.LoadBlock(height) WriteAPIResponse(w, API_OK, struct { - BlockMeta *blk.BlockMeta - Block *blk.Block + BlockMeta *types.BlockMeta + Block *types.Block }{blockMeta, block}) } diff --git a/rpc/mempool.go b/rpc/mempool.go index 0296ee69d..81402ebb3 100644 --- a/rpc/mempool.go +++ b/rpc/mempool.go @@ -4,16 +4,16 @@ import ( "net/http" "github.com/tendermint/tendermint/binary" - blk "github.com/tendermint/tendermint/block" . "github.com/tendermint/tendermint/common" "github.com/tendermint/tendermint/merkle" "github.com/tendermint/tendermint/state" + "github.com/tendermint/tendermint/types" ) func BroadcastTxHandler(w http.ResponseWriter, r *http.Request) { txJSON := GetParam(r, "tx") var err error - var tx blk.Tx + var tx types.Tx binary.ReadJSON(&tx, []byte(txJSON), &err) if err != nil { 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 contractAddr []byte - if callTx, ok := tx.(*blk.CallTx); ok { + if callTx, ok := tx.(*types.CallTx); ok { if callTx.Address == nil { createsContract = true contractAddr = state.NewContractAddress(callTx.Input.Address, uint64(callTx.Input.Sequence)) diff --git a/rpc/rpc.go b/rpc/rpc.go index 7d4526b8b..94631249c 100644 --- a/rpc/rpc.go +++ b/rpc/rpc.go @@ -1,18 +1,18 @@ package rpc import ( - blk "github.com/tendermint/tendermint/block" "github.com/tendermint/tendermint/consensus" mempl "github.com/tendermint/tendermint/mempool" "github.com/tendermint/tendermint/p2p" + "github.com/tendermint/tendermint/types" ) -var blockStore *blk.BlockStore +var blockStore *types.BlockStore var consensusState *consensus.ConsensusState var mempoolReactor *mempl.MempoolReactor var p2pSwitch *p2p.Switch -func SetRPCBlockStore(bs *blk.BlockStore) { +func SetRPCBlockStore(bs *types.BlockStore) { blockStore = bs } diff --git a/rpc/txs.go b/rpc/txs.go index 92fdea9d0..5c7588f77 100644 --- a/rpc/txs.go +++ b/rpc/txs.go @@ -5,8 +5,8 @@ import ( "github.com/tendermint/tendermint/account" "github.com/tendermint/tendermint/binary" - blk "github.com/tendermint/tendermint/block" . "github.com/tendermint/tendermint/common" + "github.com/tendermint/tendermint/types" ) func SignTxHandler(w http.ResponseWriter, r *http.Request) { @@ -14,7 +14,7 @@ func SignTxHandler(w http.ResponseWriter, r *http.Request) { privAccountsStr := GetParam(r, "privAccounts") var err error - var tx blk.Tx + var tx types.Tx binary.ReadJSON(&tx, []byte(txStr), &err) if err != nil { 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) { - case *blk.SendTx: - sendTx := tx.(*blk.SendTx) + case *types.SendTx: + sendTx := tx.(*types.SendTx) for i, input := range sendTx.Inputs { input.PubKey = privAccounts[i].PubKey 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 { input.PubKey = privAccounts[i].PubKey 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) - case *blk.RebondTx: - rebondTx := tx.(*blk.RebondTx) + case *types.RebondTx: + rebondTx := tx.(*types.RebondTx) rebondTx.Signature = privAccounts[0].Sign(rebondTx).(account.SignatureEd25519) } - WriteAPIResponse(w, API_OK, struct{ blk.Tx }{tx}) + WriteAPIResponse(w, API_OK, struct{ types.Tx }{tx}) } diff --git a/state/genesis.go b/state/genesis.go index d5590c408..9b75db976 100644 --- a/state/genesis.go +++ b/state/genesis.go @@ -6,10 +6,10 @@ import ( "github.com/tendermint/tendermint/account" "github.com/tendermint/tendermint/binary" - blk "github.com/tendermint/tendermint/block" . "github.com/tendermint/tendermint/common" dbm "github.com/tendermint/tendermint/db" "github.com/tendermint/tendermint/merkle" + "github.com/tendermint/tendermint/types" ) type GenesisAccount struct { @@ -79,12 +79,12 @@ func MakeGenesisState(db dbm.DB, genDoc *GenesisDoc) *State { valInfo := &ValidatorInfo{ Address: address, PubKey: pubKey, - UnbondTo: make([]*blk.TxOutput, len(val.UnbondTo)), + UnbondTo: make([]*types.TxOutput, len(val.UnbondTo)), FirstBondHeight: 0, FirstBondAmount: val.Amount, } for i, unbondTo := range val.UnbondTo { - valInfo.UnbondTo[i] = &blk.TxOutput{ + valInfo.UnbondTo[i] = &types.TxOutput{ Address: unbondTo.Address, Amount: unbondTo.Amount, } @@ -107,7 +107,7 @@ func MakeGenesisState(db dbm.DB, genDoc *GenesisDoc) *State { DB: db, LastBlockHeight: 0, LastBlockHash: nil, - LastBlockParts: blk.PartSetHeader{}, + LastBlockParts: types.PartSetHeader{}, LastBlockTime: genDoc.GenesisTime, BondedValidators: NewValidatorSet(validators), LastBondedValidators: NewValidatorSet(nil), diff --git a/state/priv_validator.go b/state/priv_validator.go index 854df45a7..733d1151c 100644 --- a/state/priv_validator.go +++ b/state/priv_validator.go @@ -11,10 +11,10 @@ import ( "github.com/tendermint/tendermint/account" "github.com/tendermint/tendermint/binary" - blk "github.com/tendermint/tendermint/block" . "github.com/tendermint/tendermint/common" "github.com/tendermint/tendermint/config" . "github.com/tendermint/tendermint/consensus/types" + "github.com/tendermint/tendermint/types" "github.com/tendermint/ed25519" ) @@ -27,13 +27,13 @@ const ( stepCommit = 4 ) -func voteToStep(vote *blk.Vote) uint8 { +func voteToStep(vote *types.Vote) uint8 { switch vote.Type { - case blk.VoteTypePrevote: + case types.VoteTypePrevote: return stepPrevote - case blk.VoteTypePrecommit: + case types.VoteTypePrecommit: return stepPrecommit - case blk.VoteTypeCommit: + case types.VoteTypeCommit: return stepCommit default: panic("Unknown vote type") @@ -100,7 +100,7 @@ func (privVal *PrivValidator) save() { } // TODO: test -func (privVal *PrivValidator) SignVote(vote *blk.Vote) error { +func (privVal *PrivValidator) SignVote(vote *types.Vote) error { privVal.mtx.Lock() defer privVal.mtx.Unlock() @@ -135,7 +135,7 @@ func (privVal *PrivValidator) SignVote(vote *blk.Vote) error { 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) } @@ -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() defer privVal.mtx.Unlock() if privVal.LastHeight < rebondTx.Height { diff --git a/state/state.go b/state/state.go index e4e21cbbf..a3f3586eb 100644 --- a/state/state.go +++ b/state/state.go @@ -8,10 +8,10 @@ import ( "github.com/tendermint/tendermint/account" "github.com/tendermint/tendermint/binary" - blk "github.com/tendermint/tendermint/block" . "github.com/tendermint/tendermint/common" dbm "github.com/tendermint/tendermint/db" "github.com/tendermint/tendermint/merkle" + "github.com/tendermint/tendermint/types" "github.com/tendermint/tendermint/vm" ) @@ -26,7 +26,7 @@ var ( //----------------------------------------------------------------------------- type InvalidTxError struct { - Tx blk.Tx + Tx types.Tx Reason error } @@ -41,7 +41,7 @@ type State struct { DB dbm.DB LastBlockHeight uint LastBlockHash []byte - LastBlockParts blk.PartSetHeader + LastBlockParts types.PartSetHeader LastBlockTime time.Time BondedValidators *ValidatorSet LastBondedValidators *ValidatorSet @@ -59,7 +59,7 @@ func LoadState(db dbm.DB) *State { r, n, err := bytes.NewReader(buf), new(int64), new(error) s.LastBlockHeight = binary.ReadUvarint(r, n, err) s.LastBlockHash = binary.ReadByteSlice(r, n, err) - s.LastBlockParts = binary.ReadBinary(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.BondedValidators = 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), // or it must be specified in the TxInput. If redeclared, // the TxInput is modified and input.PubKey set to PubKeyNil. -func (s *State) GetOrMakeAccounts(ins []*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{} for _, in := range ins { // Account shouldn't be duplicated if _, ok := accounts[string(in.Address)]; ok { - return nil, blk.ErrTxDuplicateAddress + return nil, types.ErrTxDuplicateAddress } acc := s.GetAccount(in.Address) if acc == nil { - return nil, blk.ErrTxInvalidAddress + return nil, types.ErrTxInvalidAddress } // PubKey should be present in either "account" or "in" 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 { // Account shouldn't be duplicated if _, ok := accounts[string(out.Address)]; ok { - return nil, blk.ErrTxDuplicateAddress + return nil, types.ErrTxDuplicateAddress } acc := s.GetAccount(out.Address) // output account may be nil (new) @@ -154,13 +154,13 @@ func (s *State) GetOrMakeAccounts(ins []*blk.TxInput, outs []*blk.TxOutput) (map 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 := in.PubKey.(account.PubKeyNil); isNil { - return blk.ErrTxUnknownPubKey + return types.ErrTxUnknownPubKey } if !bytes.Equal(in.PubKey.Address(), acc.Address) { - return blk.ErrTxInvalidPubKey + return types.ErrTxInvalidPubKey } acc.PubKey = in.PubKey } else { @@ -169,7 +169,7 @@ func checkInputPubKey(acc *account.Account, in *blk.TxInput) error { 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 { acc := accounts[string(in.Address)] if acc == nil { @@ -185,30 +185,30 @@ func (s *State) ValidateInputs(accounts map[string]*account.Account, signBytes [ 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 if err := in.ValidateBasic(); err != nil { return err } // Check signatures if !acc.PubKey.VerifyBytes(signBytes, in.Signature) { - return blk.ErrTxInvalidSignature + return types.ErrTxInvalidSignature } // Check sequences if acc.Sequence+1 != in.Sequence { - return blk.ErrTxInvalidSequence{ + return types.ErrTxInvalidSequence{ Got: uint64(in.Sequence), Expected: uint64(acc.Sequence + 1), } } // Check amount if acc.Balance < in.Amount { - return blk.ErrTxInsufficientFunds + return types.ErrTxInsufficientFunds } 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 { // Check TxOutput basic if err := out.ValidateBasic(); err != nil { @@ -220,7 +220,7 @@ func (s *State) ValidateOutputs(outs []*blk.TxOutput) (total uint64, err error) 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 { acc := accounts[string(in.Address)] 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 { acc := accounts[string(out.Address)] 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. // 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 fees := uint64(0) // Exec tx switch tx := tx_.(type) { - case *blk.SendTx: + case *types.SendTx: accounts, err := s.GetOrMakeAccounts(tx.Inputs, tx.Outputs) if err != nil { return err @@ -268,7 +268,7 @@ func (s *State) ExecTx(tx_ blk.Tx, runCall bool) error { return err } if outTotal > inTotal { - return blk.ErrTxInsufficientFunds + return types.ErrTxInsufficientFunds } fee := inTotal - outTotal fees += fee @@ -279,14 +279,14 @@ func (s *State) ExecTx(tx_ blk.Tx, runCall bool) error { s.UpdateAccounts(accounts) return nil - case *blk.CallTx: + case *types.CallTx: var inAcc, outAcc *account.Account // Validate input inAcc = s.GetAccount(tx.Input.Address) if inAcc == nil { 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" 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 { 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 @@ -309,7 +309,7 @@ func (s *State) ExecTx(tx_ blk.Tx, runCall bool) error { // Validate output if len(tx.Address) != 20 { 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 // 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 s.UpdateAccount(inAcc) log.Debug(Fmt("Cannot find destination address %X. Deducting fee from caller", tx.Address)) - return blk.ErrTxInvalidAddress + return types.ErrTxInvalidAddress } callee = toVMAccount(outAcc) @@ -401,7 +401,7 @@ func (s *State) ExecTx(tx_ blk.Tx, runCall bool) error { return nil - case *blk.BondTx: + case *types.BondTx: valInfo := s.GetValidatorInfo(tx.PubKey.Address()) if valInfo != nil { // 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 } if outTotal > inTotal { - return blk.ErrTxInsufficientFunds + return types.ErrTxInsufficientFunds } fee := inTotal - outTotal fees += fee @@ -454,17 +454,17 @@ func (s *State) ExecTx(tx_ blk.Tx, runCall bool) error { } return nil - case *blk.UnbondTx: + case *types.UnbondTx: // The validator must be active _, val := s.BondedValidators.GetByAddress(tx.Address) if val == nil { - return blk.ErrTxInvalidAddress + return types.ErrTxInvalidAddress } // Verify the signature signBytes := account.SignBytes(tx) if !val.PubKey.VerifyBytes(signBytes, tx.Signature) { - return blk.ErrTxInvalidSignature + return types.ErrTxInvalidSignature } // 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) return nil - case *blk.RebondTx: + case *types.RebondTx: // The validator must be inactive _, val := s.UnbondingValidators.GetByAddress(tx.Address) if val == nil { - return blk.ErrTxInvalidAddress + return types.ErrTxInvalidAddress } // Verify the signature signBytes := account.SignBytes(tx) if !val.PubKey.VerifyBytes(signBytes, tx.Signature) { - return blk.ErrTxInvalidSignature + return types.ErrTxInvalidSignature } // 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) return nil - case *blk.DupeoutTx: + case *types.DupeoutTx: // Verify the signatures _, accused := s.BondedValidators.GetByAddress(tx.Address) if accused == nil { _, accused = s.UnbondingValidators.GetByAddress(tx.Address) if accused == nil { - return blk.ErrTxInvalidAddress + return types.ErrTxInvalidAddress } } voteASignBytes := account.SignBytes(&tx.VoteA) voteBSignBytes := account.SignBytes(&tx.VoteB) if !accused.PubKey.VerifyBytes(voteASignBytes, tx.VoteA.Signature) || !accused.PubKey.VerifyBytes(voteBSignBytes, tx.VoteB.Signature) { - return blk.ErrTxInvalidSignature + return types.ErrTxInvalidSignature } // Verify equivocation @@ -520,10 +520,10 @@ func (s *State) ExecTx(tx_ blk.Tx, runCall bool) error { if tx.VoteA.Height != tx.VoteB.Height { 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!) // 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 } else { 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 // 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) if err != nil { return err @@ -633,9 +633,9 @@ func (s *State) AppendBlock(block *blk.Block, blockPartsHeader blk.PartSetHeader return nil } -func (s *State) SetBlockStateHash(block *blk.Block) error { +func (s *State) SetBlockStateHash(block *types.Block) error { sCopy := s.Copy() - err := sCopy.appendBlock(block, blk.PartSetHeader{}) + err := sCopy.appendBlock(block, types.PartSetHeader{}) if err != nil { return err } @@ -647,7 +647,7 @@ func (s *State) SetBlockStateHash(block *blk.Block) error { // Appends the block, does not check block.StateHash // NOTE: If an error occurs during block execution, state will be left // at an invalid state. Copy the state before calling appendBlock! -func (s *State) appendBlock(block *blk.Block, blockPartsHeader blk.PartSetHeader) error { +func (s *State) appendBlock(block *types.Block, blockPartsHeader types.PartSetHeader) error { // Basic block validation. err := block.ValidateBasic(s.LastBlockHeight, s.LastBlockHash, s.LastBlockParts, s.LastBlockTime) if err != nil { @@ -670,10 +670,10 @@ func (s *State) appendBlock(block *blk.Block, blockPartsHeader blk.PartSetHeader if commit.IsZero() { return false } else { - vote := &blk.Vote{ + vote := &types.Vote{ Height: block.Height - 1, Round: commit.Round, - Type: blk.VoteTypeCommit, + Type: types.VoteTypeCommit, BlockHash: block.LastBlockHash, BlockParts: block.LastBlockParts, } diff --git a/state/state_test.go b/state/state_test.go index 63bd0c470..da7960a77 100644 --- a/state/state_test.go +++ b/state/state_test.go @@ -3,8 +3,8 @@ package state import ( "github.com/tendermint/tendermint/account" "github.com/tendermint/tendermint/binary" - blk "github.com/tendermint/tendermint/block" "github.com/tendermint/tendermint/config" + "github.com/tendermint/tendermint/types" "bytes" "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"), Height: state.LastBlockHeight + 1, 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, StateHash: nil, }, - Validation: &blk.Validation{ + Validation: &types.Validation{ Commits: commits, }, - Data: &blk.Data{ + Data: &types.Data{ Txs: txs, }, } @@ -91,7 +91,7 @@ func TestGenesisSaveLoad(t *testing.T) { // Make complete block and blockParts block := makeBlock(t, s0, nil, nil) - blockParts := blk.NewPartSetFromData(binary.BinaryBytes(block)) + blockParts := types.NewPartSetFromData(binary.BinaryBytes(block)) // Now append the block to s0. err := s0.AppendBlock(block, blockParts.Header()) @@ -157,18 +157,18 @@ func TestTxSequence(t *testing.T) { acc1 := state.GetAccount(privAccounts[1].PubKey.Address()) // 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, Amount: 1, Sequence: sequence, PubKey: acc0PubKey, }, }, - Outputs: []*blk.TxOutput{ - &blk.TxOutput{ + Outputs: []*types.TxOutput{ + &types.TxOutput{ Address: acc1.Address, Amount: 1, }, @@ -224,17 +224,17 @@ func TestTxs(t *testing.T) { // SendTx. { state := state.Copy() - tx := &blk.SendTx{ - Inputs: []*blk.TxInput{ - &blk.TxInput{ + tx := &types.SendTx{ + Inputs: []*types.TxInput{ + &types.TxInput{ Address: acc0.Address, Amount: 1, Sequence: acc0.Sequence + 1, PubKey: acc0PubKey, }, }, - Outputs: []*blk.TxOutput{ - &blk.TxOutput{ + Outputs: []*types.TxOutput{ + &types.TxOutput{ Address: acc1.Address, Amount: 1, }, @@ -261,18 +261,18 @@ func TestTxs(t *testing.T) { // BondTx. { state := state.Copy() - tx := &blk.BondTx{ + tx := &types.BondTx{ PubKey: acc0PubKey.(account.PubKeyEd25519), - Inputs: []*blk.TxInput{ - &blk.TxInput{ + Inputs: []*types.TxInput{ + &types.TxInput{ Address: acc0.Address, Amount: 1, Sequence: acc0.Sequence + 1, PubKey: acc0PubKey, }, }, - UnbondTo: []*blk.TxOutput{ - &blk.TxOutput{ + UnbondTo: []*types.TxOutput{ + &types.TxOutput{ Address: acc0.Address, Amount: 1, }, @@ -317,18 +317,18 @@ func TestAddValidator(t *testing.T) { // The first privAccount will become a validator acc0 := privAccounts[0] - bondTx := &blk.BondTx{ + bondTx := &types.BondTx{ PubKey: acc0.PubKey.(account.PubKeyEd25519), - Inputs: []*blk.TxInput{ - &blk.TxInput{ + Inputs: []*types.TxInput{ + &types.TxInput{ Address: acc0.Address, Amount: 1000, Sequence: 1, PubKey: acc0.PubKey, }, }, - UnbondTo: []*blk.TxOutput{ - &blk.TxOutput{ + UnbondTo: []*types.TxOutput{ + &types.TxOutput{ Address: acc0.Address, Amount: 1000, }, @@ -337,8 +337,8 @@ func TestAddValidator(t *testing.T) { bondTx.Inputs[0].Signature = acc0.Sign(bondTx) // 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 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 new validator wasn't active for block0) - commit0 := &blk.Vote{ + commit0 := &types.Vote{ Height: 1, Round: 0, - Type: blk.VoteTypeCommit, + Type: types.VoteTypeCommit, BlockHash: block0.Hash(), BlockParts: block0Parts.Header(), } privValidators[0].SignVote(commit0) block1 := makeBlock(t, s0, - []blk.Commit{ - blk.Commit{ + []types.Commit{ + types.Commit{ Address: privValidators[0].Address, Round: 0, Signature: commit0.Signature, }, }, nil, ) - block1Parts := blk.NewPartSetFromData(binary.BinaryBytes(block1)) + block1Parts := types.NewPartSetFromData(binary.BinaryBytes(block1)) err = s0.AppendBlock(block1, block1Parts.Header()) if err != nil { t.Error("Error appending secondary block:", err) diff --git a/state/test.go b/state/test.go index 05b079ec0..49ba0d5e3 100644 --- a/state/test.go +++ b/state/test.go @@ -5,9 +5,9 @@ import ( "sort" "github.com/tendermint/tendermint/account" - blk "github.com/tendermint/tendermint/block" . "github.com/tendermint/tendermint/common" dbm "github.com/tendermint/tendermint/db" + "github.com/tendermint/tendermint/types" "io/ioutil" "os" @@ -46,7 +46,7 @@ func RandValidator(randBonded bool, minBonded uint64) (*ValidatorInfo, *Validato valInfo := &ValidatorInfo{ Address: privVal.Address, PubKey: privVal.PubKey, - UnbondTo: []*blk.TxOutput{&blk.TxOutput{ + UnbondTo: []*types.TxOutput{&types.TxOutput{ Amount: bonded, Address: privVal.Address, }}, diff --git a/state/validator.go b/state/validator.go index c0181f4e0..17cbeb5a5 100644 --- a/state/validator.go +++ b/state/validator.go @@ -7,14 +7,14 @@ import ( "github.com/tendermint/tendermint/account" "github.com/tendermint/tendermint/binary" - blk "github.com/tendermint/tendermint/block" + "github.com/tendermint/tendermint/types" ) // Persistent (mostly) static data for each Validator type ValidatorInfo struct { Address []byte PubKey account.PubKeyEd25519 - UnbondTo []*blk.TxOutput + UnbondTo []*types.TxOutput FirstBondHeight uint FirstBondAmount uint64 diff --git a/block/README.md b/types/README.md similarity index 100% rename from block/README.md rename to types/README.md diff --git a/block/block.go b/types/block.go similarity index 95% rename from block/block.go rename to types/block.go index bfa8c28f4..d56f4e467 100644 --- a/block/block.go +++ b/types/block.go @@ -1,4 +1,4 @@ -package block +package types import ( "bytes" @@ -54,6 +54,9 @@ func (b *Block) ValidateBasic(lastBlockHeight uint, lastBlockHash []byte, } func (b *Block) Hash() []byte { + if b.Header == nil || b.Validation == nil || b.Data == nil { + return nil + } hashes := [][]byte{ b.Header.Hash(), b.Validation.Hash(), @@ -81,6 +84,9 @@ func (b *Block) String() string { } func (b *Block) StringIndented(indent string) string { + if b == nil { + return "nil-Block" + } return fmt.Sprintf(`Block{ %s %v %s %v @@ -126,6 +132,9 @@ func (h *Header) Hash() []byte { } func (h *Header) StringIndented(indent string) string { + if h == nil { + return "nil-Header" + } return fmt.Sprintf(`Header{ %s Network: %v %s Height: %v @@ -212,6 +221,9 @@ func (v *Validation) Hash() []byte { } func (v *Validation) StringIndented(indent string) string { + if v == nil { + return "nil-Validation" + } commitStrings := make([]string, len(v.Commits)) for i, commit := range v.Commits { commitStrings[i] = commit.String() @@ -254,6 +266,9 @@ func (data *Data) Hash() []byte { } func (data *Data) StringIndented(indent string) string { + if data == nil { + return "nil-Data" + } txStrings := make([]string, len(data.Txs)) for i, tx := range data.Txs { txStrings[i] = fmt.Sprintf("Tx:%v", tx) diff --git a/block/log.go b/types/log.go similarity index 50% rename from block/log.go rename to types/log.go index bdc316758..fed0c559f 100644 --- a/block/log.go +++ b/types/log.go @@ -1,7 +1,7 @@ -package block +package types import ( "github.com/tendermint/tendermint/logger" ) -var log = logger.New("module", "block") +var log = logger.New("module", "types") diff --git a/block/part_set.go b/types/part_set.go similarity index 99% rename from block/part_set.go rename to types/part_set.go index 046b95430..d403809d1 100644 --- a/block/part_set.go +++ b/types/part_set.go @@ -1,4 +1,4 @@ -package block +package types import ( "bytes" diff --git a/block/part_set_test.go b/types/part_set_test.go similarity index 99% rename from block/part_set_test.go rename to types/part_set_test.go index ebfda4524..d27a85c4d 100644 --- a/block/part_set_test.go +++ b/types/part_set_test.go @@ -1,4 +1,4 @@ -package block +package types import ( "bytes" diff --git a/block/store.go b/types/store.go similarity index 99% rename from block/store.go rename to types/store.go index e49083ca1..3afc8cb58 100644 --- a/block/store.go +++ b/types/store.go @@ -1,4 +1,4 @@ -package block +package types import ( "bytes" diff --git a/block/tx.go b/types/tx.go similarity index 99% rename from block/tx.go rename to types/tx.go index f03d3d945..1df6f6be1 100644 --- a/block/tx.go +++ b/types/tx.go @@ -1,4 +1,4 @@ -package block +package types import ( "errors" diff --git a/block/vote.go b/types/vote.go similarity index 99% rename from block/vote.go rename to types/vote.go index b3425cf90..3e3a30e08 100644 --- a/block/vote.go +++ b/types/vote.go @@ -1,4 +1,4 @@ -package block +package types import ( "errors"