WIP commit; Fix types/results_test

This commit is contained in:
Jae Kwon
2018-04-03 07:03:08 -07:00
parent 89cdde7f1e
commit 196f8410ba
9 changed files with 111 additions and 208 deletions

View File

@ -1,21 +1,16 @@
package blockchain package blockchain
import ( import (
"bytes"
"errors"
"fmt" "fmt"
"reflect" "reflect"
"sync"
"time" "time"
wire "github.com/tendermint/go-wire" "github.com/tendermint/go-amino"
cmn "github.com/tendermint/tmlibs/common"
"github.com/tendermint/tmlibs/log"
"github.com/tendermint/tendermint/p2p" "github.com/tendermint/tendermint/p2p"
sm "github.com/tendermint/tendermint/state" sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/types" "github.com/tendermint/tendermint/types"
cmn "github.com/tendermint/tmlibs/common"
"github.com/tendermint/tmlibs/log"
) )
const ( const (
@ -52,9 +47,6 @@ func (e peerError) Error() string {
type BlockchainReactor struct { type BlockchainReactor struct {
p2p.BaseReactor p2p.BaseReactor
mtx sync.Mutex
params types.ConsensusParams
// immutable // immutable
initialState sm.State initialState sm.State
@ -87,7 +79,6 @@ func NewBlockchainReactor(state sm.State, blockExec *sm.BlockExecutor, store *Bl
) )
bcR := &BlockchainReactor{ bcR := &BlockchainReactor{
params: state.ConsensusParams,
initialState: state, initialState: state,
blockExec: blockExec, blockExec: blockExec,
store: store, store: store,
@ -140,8 +131,8 @@ func (bcR *BlockchainReactor) GetChannels() []*p2p.ChannelDescriptor {
// AddPeer implements Reactor by sending our state to peer. // AddPeer implements Reactor by sending our state to peer.
func (bcR *BlockchainReactor) AddPeer(peer p2p.Peer) { func (bcR *BlockchainReactor) AddPeer(peer p2p.Peer) {
if !peer.Send(BlockchainChannel, msgBytes := cdc.MustMarshalBinaryBare(&bcStatusResponseMessage{bcR.store.Height()})
struct{ BlockchainMessage }{&bcStatusResponseMessage{bcR.store.Height()}}) { if !peer.Send(BlockchainChannel, msgBytes) {
// doing nothing, will try later in `poolRoutine` // doing nothing, will try later in `poolRoutine`
} }
// peer is added to the pool once we receive the first // peer is added to the pool once we receive the first
@ -162,20 +153,19 @@ func (bcR *BlockchainReactor) respondToPeer(msg *bcBlockRequestMessage,
block := bcR.store.LoadBlock(msg.Height) block := bcR.store.LoadBlock(msg.Height)
if block != nil { if block != nil {
msg := &bcBlockResponseMessage{Block: block} msgBytes := cdc.MustMarshalBinaryBare(&bcBlockResponseMessage{Block: block})
return src.TrySend(BlockchainChannel, struct{ BlockchainMessage }{msg}) return src.TrySend(BlockchainChannel, msgBytes)
} }
bcR.Logger.Info("Peer asking for a block we don't have", "src", src, "height", msg.Height) bcR.Logger.Info("Peer asking for a block we don't have", "src", src, "height", msg.Height)
return src.TrySend(BlockchainChannel, struct{ BlockchainMessage }{ msgBytes := cdc.MustMarshalBinaryBare(&bcNoBlockResponseMessage{Height: msg.Height})
&bcNoBlockResponseMessage{Height: msg.Height}, return src.TrySend(BlockchainChannel, msgBytes)
})
} }
// Receive implements Reactor by handling 4 types of messages (look below). // Receive implements Reactor by handling 4 types of messages (look below).
func (bcR *BlockchainReactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) { func (bcR *BlockchainReactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) {
_, msg, err := DecodeMessage(msgBytes, bcR.maxMsgSize()) msg, err := DecodeMessage(msgBytes)
if err != nil { if err != nil {
bcR.Logger.Error("Error decoding message", "src", src, "chId", chID, "msg", msg, "err", err, "bytes", msgBytes) bcR.Logger.Error("Error decoding message", "src", src, "chId", chID, "msg", msg, "err", err, "bytes", msgBytes)
bcR.Switch.StopPeerForError(src, err) bcR.Switch.StopPeerForError(src, err)
@ -194,8 +184,8 @@ func (bcR *BlockchainReactor) Receive(chID byte, src p2p.Peer, msgBytes []byte)
bcR.pool.AddBlock(src.ID(), msg.Block, len(msgBytes)) bcR.pool.AddBlock(src.ID(), msg.Block, len(msgBytes))
case *bcStatusRequestMessage: case *bcStatusRequestMessage:
// Send peer our state. // Send peer our state.
queued := src.TrySend(BlockchainChannel, msgBytes := cdc.MustMarshalBinaryBare(&bcStatusResponseMessage{bcR.store.Height()})
struct{ BlockchainMessage }{&bcStatusResponseMessage{bcR.store.Height()}}) queued := src.TrySend(BlockchainChannel, msgBytes)
if !queued { if !queued {
// sorry // sorry
} }
@ -207,21 +197,6 @@ func (bcR *BlockchainReactor) Receive(chID byte, src p2p.Peer, msgBytes []byte)
} }
} }
// maxMsgSize returns the maximum allowable size of a
// message on the blockchain reactor.
func (bcR *BlockchainReactor) maxMsgSize() int {
bcR.mtx.Lock()
defer bcR.mtx.Unlock()
return bcR.params.BlockSize.MaxBytes + 2
}
// updateConsensusParams updates the internal consensus params
func (bcR *BlockchainReactor) updateConsensusParams(params types.ConsensusParams) {
bcR.mtx.Lock()
defer bcR.mtx.Unlock()
bcR.params = params
}
// Handle messages from the poolReactor telling the reactor what to do. // Handle messages from the poolReactor telling the reactor what to do.
// NOTE: Don't sleep in the FOR_LOOP or otherwise slow it down! // NOTE: Don't sleep in the FOR_LOOP or otherwise slow it down!
// (Except for the SYNC_LOOP, which is the primary purpose and must be synchronous.) // (Except for the SYNC_LOOP, which is the primary purpose and must be synchronous.)
@ -247,8 +222,8 @@ FOR_LOOP:
if peer == nil { if peer == nil {
continue FOR_LOOP // Peer has since been disconnected. continue FOR_LOOP // Peer has since been disconnected.
} }
msg := &bcBlockRequestMessage{request.Height} msgBytes := cdc.MustMarshalBinaryBare(&bcBlockRequestMessage{request.Height})
queued := peer.TrySend(BlockchainChannel, struct{ BlockchainMessage }{msg}) queued := peer.TrySend(BlockchainChannel, msgBytes)
if !queued { if !queued {
// We couldn't make the request, send-queue full. // We couldn't make the request, send-queue full.
// The pool handles timeouts, just let it go. // The pool handles timeouts, just let it go.
@ -321,9 +296,6 @@ FOR_LOOP:
} }
blocksSynced++ blocksSynced++
// update the consensus params
bcR.updateConsensusParams(state.ConsensusParams)
if blocksSynced%100 == 0 { if blocksSynced%100 == 0 {
lastRate = 0.9*lastRate + 0.1*(100/time.Since(lastHundred).Seconds()) lastRate = 0.9*lastRate + 0.1*(100/time.Since(lastHundred).Seconds())
bcR.Logger.Info("Fast Sync Rate", "height", bcR.pool.height, bcR.Logger.Info("Fast Sync Rate", "height", bcR.pool.height,
@ -341,43 +313,37 @@ FOR_LOOP:
// BroadcastStatusRequest broadcasts `BlockStore` height. // BroadcastStatusRequest broadcasts `BlockStore` height.
func (bcR *BlockchainReactor) BroadcastStatusRequest() error { func (bcR *BlockchainReactor) BroadcastStatusRequest() error {
bcR.Switch.Broadcast(BlockchainChannel, msgBytes := cdc.MustMarshalBinaryBare(&bcStatusRequestMessage{bcR.store.Height()})
struct{ BlockchainMessage }{&bcStatusRequestMessage{bcR.store.Height()}}) bcR.Switch.Broadcast(BlockchainChannel, msgBytes)
return nil return nil
} }
//----------------------------------------------------------------------------- //-----------------------------------------------------------------------------
// Messages // Messages
const (
msgTypeBlockRequest = byte(0x10)
msgTypeBlockResponse = byte(0x11)
msgTypeNoBlockResponse = byte(0x12)
msgTypeStatusResponse = byte(0x20)
msgTypeStatusRequest = byte(0x21)
)
// BlockchainMessage is a generic message for this reactor. // BlockchainMessage is a generic message for this reactor.
type BlockchainMessage interface{} type BlockchainMessage interface{}
var _ = wire.RegisterInterface( func RegisterBlockchainMessages(cdc *amino.Codec) {
struct{ BlockchainMessage }{}, cdc.RegisterInterface((*BlockchainMessage)(nil), nil)
wire.ConcreteType{&bcBlockRequestMessage{}, msgTypeBlockRequest}, cdc.RegisterConcrete(&bcBlockRequestMessage{}, "tendermint/mempool/BlockRequest", nil)
wire.ConcreteType{&bcBlockResponseMessage{}, msgTypeBlockResponse}, cdc.RegisterConcrete(&bcBlockResponseMessage{}, "tendermint/mempool/BlockResponse", nil)
wire.ConcreteType{&bcNoBlockResponseMessage{}, msgTypeNoBlockResponse}, cdc.RegisterConcrete(&bcNoBlockResponseMessage{}, "tendermint/mempool/NoBlockResponse", nil)
wire.ConcreteType{&bcStatusResponseMessage{}, msgTypeStatusResponse}, cdc.RegisterConcrete(&bcStatusResponseMessage{}, "tendermint/mempool/StatusResponse", nil)
wire.ConcreteType{&bcStatusRequestMessage{}, msgTypeStatusRequest}, cdc.RegisterConcrete(&bcStatusRequestMessage{}, "tendermint/mempool/StatusRequest", nil)
) }
// DecodeMessage decodes BlockchainMessage. // DecodeMessage decodes BlockchainMessage.
// TODO: ensure that bz is completely read. // TODO: ensure that bz is completely read.
func DecodeMessage(bz []byte, maxSize int) (msgType byte, msg BlockchainMessage, err error) { func DecodeMessage(bz []byte) (msg BlockchainMessage, err error) {
msgType = bz[0] const (
n := int(0) prefixSize = 4
r := bytes.NewReader(bz) fieldKeySize = 1
msg = wire.ReadBinary(struct{ BlockchainMessage }{}, r, maxSize, &n, &err).(struct{ BlockchainMessage }).BlockchainMessage maxMessageSize = types.MaxBlockSizeBytes + prefixSize + fieldKeySize
if err != nil && n != len(bz) { )
err = errors.New("DecodeMessage() had bytes left over") err = cdc.UnmarshalBinaryBare(bz, &msg)
if err != nil {
err = cmn.ErrorWrap(err, "DecodeMessage() had bytes left over")
} }
return return
} }
@ -402,7 +368,6 @@ func (brm *bcNoBlockResponseMessage) String() string {
//------------------------------------- //-------------------------------------
// NOTE: keep up-to-date with maxBlockchainResponseSize
type bcBlockResponseMessage struct { type bcBlockResponseMessage struct {
Block *types.Block Block *types.Block
} }

View File

@ -3,8 +3,6 @@ package blockchain
import ( import (
"testing" "testing"
wire "github.com/tendermint/go-wire"
cmn "github.com/tendermint/tmlibs/common" cmn "github.com/tendermint/tmlibs/common"
dbm "github.com/tendermint/tmlibs/db" dbm "github.com/tendermint/tmlibs/db"
"github.com/tendermint/tmlibs/log" "github.com/tendermint/tmlibs/log"
@ -19,7 +17,10 @@ import (
func makeStateAndBlockStore(logger log.Logger) (sm.State, *BlockStore) { func makeStateAndBlockStore(logger log.Logger) (sm.State, *BlockStore) {
config := cfg.ResetTestRoot("blockchain_reactor_test") config := cfg.ResetTestRoot("blockchain_reactor_test")
blockStore := NewBlockStore(dbm.NewMemDB()) blockStore := NewBlockStore(dbm.NewMemDB())
state, _ := sm.LoadStateFromDBOrGenesisFile(dbm.NewMemDB(), config.GenesisFile()) state, err := sm.LoadStateFromDBOrGenesisFile(dbm.NewMemDB(), config.GenesisFile())
if err != nil {
panic(cmn.ErrorWrap(err, "error constructing state from genesis file"))
}
return state, blockStore return state, blockStore
} }
@ -76,10 +77,9 @@ func TestNoBlockResponse(t *testing.T) {
// wait for our response to be received on the peer // wait for our response to be received on the peer
for _, tt := range tests { for _, tt := range tests {
reqBlockMsg := &bcBlockRequestMessage{tt.height} reqBlockMsg := &bcBlockRequestMessage{tt.height}
reqBlockBytes := wire.BinaryBytes(struct{ BlockchainMessage }{reqBlockMsg}) reqBlockBytes := cdc.MustMarshalBinaryBare(reqBlockMsg)
bcr.Receive(chID, peer, reqBlockBytes) bcr.Receive(chID, peer, reqBlockBytes)
value := peer.lastValue() msg := peer.lastBlockchainMessage()
msg := value.(struct{ BlockchainMessage }).BlockchainMessage
if tt.existent { if tt.existent {
if blockMsg, ok := msg.(*bcBlockResponseMessage); !ok { if blockMsg, ok := msg.(*bcBlockResponseMessage); !ok {
@ -173,26 +173,30 @@ func newbcrTestPeer(id p2p.ID) *bcrTestPeer {
return bcr return bcr
} }
func (tp *bcrTestPeer) lastValue() interface{} { return <-tp.ch } func (tp *bcrTestPeer) lastBlockchainMessage() interface{} { return <-tp.ch }
func (tp *bcrTestPeer) TrySend(chID byte, value interface{}) bool { func (tp *bcrTestPeer) TrySend(chID byte, msgBytes []byte) bool {
if _, ok := value.(struct{ BlockchainMessage }). var msg BlockchainMessage
BlockchainMessage.(*bcStatusResponseMessage); ok { err := cdc.UnmarshalBinaryBare(msgBytes, &msg)
if err != nil {
panic(cmn.ErrorWrap(err, "Error while trying to parse a BlockchainMessage"))
}
if _, ok := msg.(*bcStatusResponseMessage); ok {
// Discard status response messages since they skew our results // Discard status response messages since they skew our results
// We only want to deal with: // We only want to deal with:
// + bcBlockResponseMessage // + bcBlockResponseMessage
// + bcNoBlockResponseMessage // + bcNoBlockResponseMessage
} else { } else {
tp.ch <- value tp.ch <- msg
} }
return true return true
} }
func (tp *bcrTestPeer) Send(chID byte, data interface{}) bool { return tp.TrySend(chID, data) } func (tp *bcrTestPeer) Send(chID byte, msgBytes []byte) bool { return tp.TrySend(chID, msgBytes) }
func (tp *bcrTestPeer) NodeInfo() p2p.NodeInfo { return p2p.NodeInfo{} } func (tp *bcrTestPeer) NodeInfo() p2p.NodeInfo { return p2p.NodeInfo{} }
func (tp *bcrTestPeer) Status() p2p.ConnectionStatus { return p2p.ConnectionStatus{} } func (tp *bcrTestPeer) Status() p2p.ConnectionStatus { return p2p.ConnectionStatus{} }
func (tp *bcrTestPeer) ID() p2p.ID { return tp.id } func (tp *bcrTestPeer) ID() p2p.ID { return tp.id }
func (tp *bcrTestPeer) IsOutbound() bool { return false } func (tp *bcrTestPeer) IsOutbound() bool { return false }
func (tp *bcrTestPeer) IsPersistent() bool { return true } func (tp *bcrTestPeer) IsPersistent() bool { return true }
func (tp *bcrTestPeer) Get(s string) interface{} { return s } func (tp *bcrTestPeer) Get(s string) interface{} { return s }
func (tp *bcrTestPeer) Set(string, interface{}) {} func (tp *bcrTestPeer) Set(string, interface{}) {}

View File

@ -1,14 +1,10 @@
package blockchain package blockchain
import ( import (
"bytes"
"encoding/json" "encoding/json"
"fmt" "fmt"
"io"
"sync" "sync"
wire "github.com/tendermint/go-wire"
cmn "github.com/tendermint/tmlibs/common" cmn "github.com/tendermint/tmlibs/common"
dbm "github.com/tendermint/tmlibs/db" dbm "github.com/tendermint/tmlibs/db"
@ -54,38 +50,25 @@ func (bs *BlockStore) Height() int64 {
return bs.height return bs.height
} }
// GetReader returns the value associated with the given key wrapped in an io.Reader.
// If no value is found, it returns nil.
// It's mainly for use with wire.ReadBinary.
func (bs *BlockStore) GetReader(key []byte) io.Reader {
bytez := bs.db.Get(key)
if bytez == nil {
return nil
}
return bytes.NewReader(bytez)
}
// LoadBlock returns the block with the given height. // LoadBlock returns the block with the given height.
// If no block is found for that height, it returns nil. // If no block is found for that height, it returns nil.
func (bs *BlockStore) LoadBlock(height int64) *types.Block { func (bs *BlockStore) LoadBlock(height int64) *types.Block {
var n int var blockMeta *types.BlockMeta
var err error bz := bs.db.Get(calcBlockMetaKey(height))
r := bs.GetReader(calcBlockMetaKey(height)) err := cdc.UnmarshalBinaryBare(bz, blockMeta)
if r == nil {
return nil
}
blockMeta := wire.ReadBinary(&types.BlockMeta{}, r, 0, &n, &err).(*types.BlockMeta)
if err != nil { if err != nil {
panic(fmt.Sprintf("Error reading block meta: %v", err)) panic(cmn.ErrorWrap(err, "Error reading block meta"))
} }
bytez := []byte{}
var block *types.Block
buf := []byte{}
for i := 0; i < blockMeta.BlockID.PartsHeader.Total; i++ { for i := 0; i < blockMeta.BlockID.PartsHeader.Total; i++ {
part := bs.LoadBlockPart(height, i) part := bs.LoadBlockPart(height, i)
bytez = append(bytez, part.Bytes...) buf = append(buf, part.Bytes...)
} }
block := wire.ReadBinary(&types.Block{}, bytes.NewReader(bytez), 0, &n, &err).(*types.Block) err = cdc.UnmarshalBinaryBare(buf, block)
if err != nil { if err != nil {
panic(fmt.Sprintf("Error reading block: %v", err)) panic(cmn.ErrorWrap(err, "Error reading block"))
} }
return block return block
} }
@ -94,15 +77,11 @@ func (bs *BlockStore) LoadBlock(height int64) *types.Block {
// from the block at the given height. // from the block at the given height.
// If no part is found for the given height and index, it returns nil. // If no part is found for the given height and index, it returns nil.
func (bs *BlockStore) LoadBlockPart(height int64, index int) *types.Part { func (bs *BlockStore) LoadBlockPart(height int64, index int) *types.Part {
var n int var part *types.Part
var err error bz := bs.db.Get(calcBlockPartKey(height, index))
r := bs.GetReader(calcBlockPartKey(height, index)) err := cdc.UnmarshalBinaryBare(bz, part)
if r == nil {
return nil
}
part := wire.ReadBinary(&types.Part{}, r, 0, &n, &err).(*types.Part)
if err != nil { if err != nil {
panic(fmt.Sprintf("Error reading block part: %v", err)) panic(cmn.ErrorWrap(err, "Error reading block part"))
} }
return part return part
} }
@ -110,15 +89,11 @@ func (bs *BlockStore) LoadBlockPart(height int64, index int) *types.Part {
// LoadBlockMeta returns the BlockMeta for the given height. // LoadBlockMeta returns the BlockMeta for the given height.
// If no block is found for the given height, it returns nil. // If no block is found for the given height, it returns nil.
func (bs *BlockStore) LoadBlockMeta(height int64) *types.BlockMeta { func (bs *BlockStore) LoadBlockMeta(height int64) *types.BlockMeta {
var n int var blockMeta *types.BlockMeta
var err error bz := bs.db.Get(calcBlockMetaKey(height))
r := bs.GetReader(calcBlockMetaKey(height)) err := cdc.UnmarshalBinaryBare(bz, blockMeta)
if r == nil {
return nil
}
blockMeta := wire.ReadBinary(&types.BlockMeta{}, r, 0, &n, &err).(*types.BlockMeta)
if err != nil { if err != nil {
panic(fmt.Sprintf("Error reading block meta: %v", err)) panic(cmn.ErrorWrap(err, "Error reading block meta"))
} }
return blockMeta return blockMeta
} }
@ -128,15 +103,11 @@ func (bs *BlockStore) LoadBlockMeta(height int64) *types.BlockMeta {
// and it comes from the block.LastCommit for `height+1`. // and it comes from the block.LastCommit for `height+1`.
// If no commit is found for the given height, it returns nil. // If no commit is found for the given height, it returns nil.
func (bs *BlockStore) LoadBlockCommit(height int64) *types.Commit { func (bs *BlockStore) LoadBlockCommit(height int64) *types.Commit {
var n int var commit *types.Commit
var err error bz := bs.db.Get(calcBlockCommitKey(height))
r := bs.GetReader(calcBlockCommitKey(height)) err := cdc.UnmarshalBinaryBare(bz, commit)
if r == nil {
return nil
}
commit := wire.ReadBinary(&types.Commit{}, r, 0, &n, &err).(*types.Commit)
if err != nil { if err != nil {
panic(fmt.Sprintf("Error reading commit: %v", err)) panic(cmn.ErrorWrap(err, "Error reading block commit"))
} }
return commit return commit
} }
@ -145,15 +116,11 @@ func (bs *BlockStore) LoadBlockCommit(height int64) *types.Commit {
// This is useful when we've seen a commit, but there has not yet been // This is useful when we've seen a commit, but there has not yet been
// a new block at `height + 1` that includes this commit in its block.LastCommit. // a new block at `height + 1` that includes this commit in its block.LastCommit.
func (bs *BlockStore) LoadSeenCommit(height int64) *types.Commit { func (bs *BlockStore) LoadSeenCommit(height int64) *types.Commit {
var n int var commit *types.Commit
var err error bz := bs.db.Get(calcSeenCommitKey(height))
r := bs.GetReader(calcSeenCommitKey(height)) err := cdc.UnmarshalBinaryBare(bz, commit)
if r == nil {
return nil
}
commit := wire.ReadBinary(&types.Commit{}, r, 0, &n, &err).(*types.Commit)
if err != nil { if err != nil {
panic(fmt.Sprintf("Error reading commit: %v", err)) panic(cmn.ErrorWrap(err, "Error reading block commit"))
} }
return commit return commit
} }
@ -178,7 +145,7 @@ func (bs *BlockStore) SaveBlock(block *types.Block, blockParts *types.PartSet, s
// Save block meta // Save block meta
blockMeta := types.NewBlockMeta(block, blockParts) blockMeta := types.NewBlockMeta(block, blockParts)
metaBytes := wire.BinaryBytes(blockMeta) metaBytes := cdc.MustMarshalBinaryBare(blockMeta)
bs.db.Set(calcBlockMetaKey(height), metaBytes) bs.db.Set(calcBlockMetaKey(height), metaBytes)
// Save block parts // Save block parts
@ -187,12 +154,12 @@ func (bs *BlockStore) SaveBlock(block *types.Block, blockParts *types.PartSet, s
} }
// Save block commit (duplicate and separate from the Block) // Save block commit (duplicate and separate from the Block)
blockCommitBytes := wire.BinaryBytes(block.LastCommit) blockCommitBytes := cdc.MustMarshalBinaryBare(block.LastCommit)
bs.db.Set(calcBlockCommitKey(height-1), blockCommitBytes) bs.db.Set(calcBlockCommitKey(height-1), blockCommitBytes)
// Save seen commit (seen +2/3 precommits for block) // Save seen commit (seen +2/3 precommits for block)
// NOTE: we can delete this at a later height // NOTE: we can delete this at a later height
seenCommitBytes := wire.BinaryBytes(seenCommit) seenCommitBytes := cdc.MustMarshalBinaryBare(seenCommit)
bs.db.Set(calcSeenCommitKey(height), seenCommitBytes) bs.db.Set(calcSeenCommitKey(height), seenCommitBytes)
// Save new BlockStoreStateJSON descriptor // Save new BlockStoreStateJSON descriptor
@ -211,7 +178,7 @@ func (bs *BlockStore) saveBlockPart(height int64, index int, part *types.Part) {
if height != bs.Height()+1 { if height != bs.Height()+1 {
cmn.PanicSanity(cmn.Fmt("BlockStore can only save contiguous blocks. Wanted %v, got %v", bs.Height()+1, height)) cmn.PanicSanity(cmn.Fmt("BlockStore can only save contiguous blocks. Wanted %v, got %v", bs.Height()+1, height))
} }
partBytes := wire.BinaryBytes(part) partBytes := cdc.MustMarshalBinaryBare(part)
bs.db.Set(calcBlockPartKey(height, index), partBytes) bs.db.Set(calcBlockPartKey(height, index), partBytes)
} }

View File

@ -3,7 +3,6 @@ package blockchain
import ( import (
"bytes" "bytes"
"fmt" "fmt"
"io/ioutil"
"runtime/debug" "runtime/debug"
"strings" "strings"
"testing" "testing"
@ -11,9 +10,6 @@ import (
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
wire "github.com/tendermint/go-wire"
"github.com/tendermint/tmlibs/db" "github.com/tendermint/tmlibs/db"
"github.com/tendermint/tmlibs/log" "github.com/tendermint/tmlibs/log"
@ -61,38 +57,6 @@ func TestNewBlockStore(t *testing.T) {
assert.Equal(t, bs.Height(), int64(0), "expecting nil bytes to be unmarshaled alright") assert.Equal(t, bs.Height(), int64(0), "expecting nil bytes to be unmarshaled alright")
} }
func TestBlockStoreGetReader(t *testing.T) {
db := db.NewMemDB()
// Initial setup
db.Set([]byte("Foo"), []byte("Bar"))
db.Set([]byte("Foo1"), nil)
bs := NewBlockStore(db)
tests := [...]struct {
key []byte
want []byte
}{
0: {key: []byte("Foo"), want: []byte("Bar")},
1: {key: []byte("KnoxNonExistent"), want: nil},
2: {key: []byte("Foo1"), want: []byte{}},
}
for i, tt := range tests {
r := bs.GetReader(tt.key)
if r == nil {
assert.Nil(t, tt.want, "#%d: expected a non-nil reader", i)
continue
}
slurp, err := ioutil.ReadAll(r)
if err != nil {
t.Errorf("#%d: unexpected Read err: %v", i, err)
} else {
assert.Equal(t, slurp, tt.want, "#%d: mismatch", i)
}
}
}
func freshBlockStore() (*BlockStore, db.DB) { func freshBlockStore() (*BlockStore, db.DB) {
db := db.NewMemDB() db := db.NewMemDB()
return NewBlockStore(db), db return NewBlockStore(db), db
@ -305,14 +269,6 @@ func TestBlockStoreSaveLoadBlock(t *testing.T) {
} }
} }
func binarySerializeIt(v interface{}) []byte {
var n int
var err error
buf := new(bytes.Buffer)
wire.WriteBinary(v, buf, &n, &err)
return buf.Bytes()
}
func TestLoadBlockPart(t *testing.T) { func TestLoadBlockPart(t *testing.T) {
bs, db := freshBlockStore() bs, db := freshBlockStore()
height, index := int64(10), 1 height, index := int64(10), 1
@ -334,7 +290,7 @@ func TestLoadBlockPart(t *testing.T) {
require.Contains(t, panicErr.Error(), "Error reading block part") require.Contains(t, panicErr.Error(), "Error reading block part")
// 3. A good block serialized and saved to the DB should be retrievable // 3. A good block serialized and saved to the DB should be retrievable
db.Set(calcBlockPartKey(height, index), binarySerializeIt(part1)) db.Set(calcBlockPartKey(height, index), cdc.MustMarshalBinaryBare(part1))
gotPart, _, panicErr := doFn(loadPart) gotPart, _, panicErr := doFn(loadPart)
require.Nil(t, panicErr, "an existent and proper block should not panic") require.Nil(t, panicErr, "an existent and proper block should not panic")
require.Nil(t, res, "a properly saved block should return a proper block") require.Nil(t, res, "a properly saved block should return a proper block")
@ -364,11 +320,11 @@ func TestLoadBlockMeta(t *testing.T) {
// 3. A good blockMeta serialized and saved to the DB should be retrievable // 3. A good blockMeta serialized and saved to the DB should be retrievable
meta := &types.BlockMeta{} meta := &types.BlockMeta{}
db.Set(calcBlockMetaKey(height), binarySerializeIt(meta)) db.Set(calcBlockMetaKey(height), cdc.MustMarshalBinaryBare(meta))
gotMeta, _, panicErr := doFn(loadMeta) gotMeta, _, panicErr := doFn(loadMeta)
require.Nil(t, panicErr, "an existent and proper block should not panic") require.Nil(t, panicErr, "an existent and proper block should not panic")
require.Nil(t, res, "a properly saved blockMeta should return a proper blocMeta ") require.Nil(t, res, "a properly saved blockMeta should return a proper blocMeta ")
require.Equal(t, binarySerializeIt(meta), binarySerializeIt(gotMeta), require.Equal(t, cdc.MustMarshalBinaryBare(meta), cdc.MustMarshalBinaryBare(gotMeta),
"expecting successful retrieval of previously saved blockMeta") "expecting successful retrieval of previously saved blockMeta")
} }

11
blockchain/wire.go Normal file
View File

@ -0,0 +1,11 @@
package blockchain
import (
"github.com/tendermint/go-amino"
)
var cdc = amino.NewCodec()
func init() {
RegisterBlockchainMessages(cdc)
}

View File

@ -405,9 +405,9 @@ func getSwitchIndex(switches []*p2p.Switch, peer p2p.Peer) int {
//------------------------------------------------------------------------------- //-------------------------------------------------------------------------------
// genesis // genesis
func randGenesisDoc(numValidators int, randPower bool, minPower int64) (*types.GenesisDoc, []*types.PrivValidatorFS) { func randGenesisDoc(numValidators int, randPower bool, minPower int64) (*types.GenesisDoc, []types.PrivValidator) {
validators := make([]types.GenesisValidator, numValidators) validators := make([]types.GenesisValidator, numValidators)
privValidators := make([]*types.PrivValidatorFS, numValidators) privValidators := make([]types.PrivValidator, numValidators)
for i := 0; i < numValidators; i++ { for i := 0; i < numValidators; i++ {
val, privVal := types.RandValidator(randPower, minPower) val, privVal := types.RandValidator(randPower, minPower)
validators[i] = types.GenesisValidator{ validators[i] = types.GenesisValidator{
@ -425,7 +425,7 @@ func randGenesisDoc(numValidators int, randPower bool, minPower int64) (*types.G
}, privValidators }, privValidators
} }
func randGenesisState(numValidators int, randPower bool, minPower int64) (sm.State, []*types.PrivValidatorFS) { func randGenesisState(numValidators int, randPower bool, minPower int64) (sm.State, []types.PrivValidator) {
genDoc, privValidators := randGenesisDoc(numValidators, randPower, minPower) genDoc, privValidators := randGenesisDoc(numValidators, randPower, minPower)
s0, _ := sm.MakeGenesisState(genDoc) s0, _ := sm.MakeGenesisState(genDoc)
db := dbm.NewMemDB() db := dbm.NewMemDB()

View File

@ -1301,10 +1301,10 @@ func (cs *ConsensusState) addProposalBlockPart(height int64, part *types.Part, v
} }
if added && cs.ProposalBlockParts.IsComplete() { if added && cs.ProposalBlockParts.IsComplete() {
// Added and completed! // Added and completed!
var n int err = cdc.UnmarshalBinaryBare(cs.ProposalBlockParts.GetReader(), &cs.ProposalBlock, cs.state.ConsensusParams.BlockSize.MaxBytes)
var err error if err != nil {
cs.ProposalBlock = wire.ReadBinary(&types.Block{}, cs.ProposalBlockParts.GetReader(), return true, err
cs.state.ConsensusParams.BlockSize.MaxBytes, &n, &err).(*types.Block) }
// NOTE: it's possible to receive complete proposal blocks for future rounds without having the proposal // NOTE: it's possible to receive complete proposal blocks for future rounds without having the proposal
cs.Logger.Info("Received complete proposal block", "height", cs.ProposalBlock.Height, "hash", cs.ProposalBlock.Hash()) cs.Logger.Info("Received complete proposal block", "height", cs.ProposalBlock.Height, "hash", cs.ProposalBlock.Hash())
if cs.Step == cstypes.RoundStepPropose && cs.isProposalComplete() { if cs.Step == cstypes.RoundStepPropose && cs.isProposalComplete() {
@ -1314,7 +1314,7 @@ func (cs *ConsensusState) addProposalBlockPart(height int64, part *types.Part, v
// If we're waiting on the proposal block... // If we're waiting on the proposal block...
cs.tryFinalizeCommit(height) cs.tryFinalizeCommit(height)
} }
return true, err return true, nil
} }
return added, nil return added, nil
} }

View File

@ -48,7 +48,7 @@ func TestPeerCatchupRounds(t *testing.T) {
} }
func makeVoteHR(t *testing.T, height int64, round int, privVals []*types.PrivValidatorFS, valIndex int) *types.Vote { func makeVoteHR(t *testing.T, height int64, round int, privVals []types.PrivValidator, valIndex int) *types.Vote {
privVal := privVals[valIndex] privVal := privVals[valIndex]
vote := &types.Vote{ vote := &types.Vote{
ValidatorAddress: privVal.GetAddress(), ValidatorAddress: privVal.GetAddress(),

View File

@ -15,10 +15,10 @@ func TestABCIResults(t *testing.T) {
e := ABCIResult{Code: 14, Data: []byte("foo")} e := ABCIResult{Code: 14, Data: []byte("foo")}
f := ABCIResult{Code: 14, Data: []byte("bar")} f := ABCIResult{Code: 14, Data: []byte("bar")}
// Nil and []byte{} should not produce the same hash. // Nil and []byte{} should produce the same hash.
require.Equal(t, a.Hash(), a.Hash()) require.Equal(t, a.Hash(), a.Hash())
require.Equal(t, b.Hash(), b.Hash()) require.Equal(t, b.Hash(), b.Hash())
require.NotEqual(t, a.Hash(), b.Hash()) require.Equal(t, a.Hash(), b.Hash())
// a and b should be the same, don't go in results. // a and b should be the same, don't go in results.
results := ABCIResults{a, c, d, e, f} results := ABCIResults{a, c, d, e, f}