Browse Source

blockchain: proto migration (#4969)

## Description

migration of blockchain reactors to protobuf

Closes: #XXX
pull/4988/head
Marko 4 years ago
committed by GitHub
parent
commit
a89f2581fc
No known key found for this signature in database GPG Key ID: 4AEE18F83AFDEB23
22 changed files with 516 additions and 808 deletions
  1. +1
    -0
      CHANGELOG_PENDING.md
  2. +118
    -0
      blockchain/msgs.go
  3. +88
    -0
      blockchain/msgs_test.go
  4. +0
    -14
      blockchain/v0/codec.go
  5. +60
    -149
      blockchain/v0/reactor.go
  6. +0
    -80
      blockchain/v0/reactor_test.go
  7. +0
    -14
      blockchain/v1/codec.go
  8. +53
    -150
      blockchain/v1/reactor.go
  9. +0
    -80
      blockchain/v1/reactor_test.go
  10. +0
    -14
      blockchain/v2/codec.go
  11. +36
    -10
      blockchain/v2/io.go
  12. +16
    -143
      blockchain/v2/reactor.go
  13. +20
    -10
      blockchain/v2/reactor_test.go
  14. +1
    -1
      config/toml.go
  15. +2
    -2
      consensus/reactor.go
  16. +1
    -0
      docs/tendermint-core/configuration.md
  17. +1
    -0
      go.mod
  18. +2
    -1
      go.sum
  19. +45
    -41
      proto/blockchain/msgs.pb.go
  20. +1
    -2
      proto/blockchain/msgs.proto
  21. +1
    -1
      types/block.go
  22. +70
    -96
      types/block_test.go

+ 1
- 0
CHANGELOG_PENDING.md View File

@ -12,6 +12,7 @@ Friendly reminder, we have a [bug bounty program](https://hackerone.com/tendermi
- [evidence] \#4725 Remove `Pubkey` from DuplicateVoteEvidence
- [rpc] [\#4792](https://github.com/tendermint/tendermint/pull/4792) `/validators` are now sorted by voting power (@melekes)
- [blockchain] \#4637 Transition blockchain reactor(s) to Protobuf encoding
- [types] \#4382 `SignedMsgType` has moved to a Protobuf enum types
- [types] \#4382 `Total` has been changed from a `int` to a `uint32`
- [types] \#4582 Vote: `ValidatorIndex` & `Round` are now int32


+ 118
- 0
blockchain/msgs.go View File

@ -0,0 +1,118 @@
package blockchain
import (
"errors"
"fmt"
"github.com/gogo/protobuf/proto"
bcproto "github.com/tendermint/tendermint/proto/blockchain"
"github.com/tendermint/tendermint/types"
)
const (
// NOTE: keep up to date with bcproto.BlockResponse
BlockResponseMessagePrefixSize = 4
BlockResponseMessageFieldKeySize = 1
MaxMsgSize = types.MaxBlockSizeBytes +
BlockResponseMessagePrefixSize +
BlockResponseMessageFieldKeySize
)
// EncodeMsg encodes a Protobuf message
func EncodeMsg(pb proto.Message) ([]byte, error) {
msg := bcproto.Message{}
switch pb := pb.(type) {
case *bcproto.BlockRequest:
msg.Sum = &bcproto.Message_BlockRequest{BlockRequest: pb}
case *bcproto.BlockResponse:
msg.Sum = &bcproto.Message_BlockResponse{BlockResponse: pb}
case *bcproto.NoBlockResponse:
msg.Sum = &bcproto.Message_NoBlockResponse{NoBlockResponse: pb}
case *bcproto.StatusRequest:
msg.Sum = &bcproto.Message_StatusRequest{StatusRequest: pb}
case *bcproto.StatusResponse:
msg.Sum = &bcproto.Message_StatusResponse{StatusResponse: pb}
default:
return nil, fmt.Errorf("unknown message type %T", pb)
}
bz, err := proto.Marshal(&msg)
if err != nil {
return nil, fmt.Errorf("unable to marshal %T: %w", pb, err)
}
return bz, nil
}
// DecodeMsg decodes a Protobuf message.
func DecodeMsg(bz []byte) (proto.Message, error) {
pb := &bcproto.Message{}
err := proto.Unmarshal(bz, pb)
if err != nil {
return nil, err
}
switch msg := pb.Sum.(type) {
case *bcproto.Message_BlockRequest:
return msg.BlockRequest, nil
case *bcproto.Message_BlockResponse:
return msg.BlockResponse, nil
case *bcproto.Message_NoBlockResponse:
return msg.NoBlockResponse, nil
case *bcproto.Message_StatusRequest:
return msg.StatusRequest, nil
case *bcproto.Message_StatusResponse:
return msg.StatusResponse, nil
default:
return nil, fmt.Errorf("unknown message type %T", msg)
}
}
// ValidateMsg validates a message.
func ValidateMsg(pb proto.Message) error {
if pb == nil {
return errors.New("message cannot be nil")
}
switch msg := pb.(type) {
case *bcproto.BlockRequest:
if msg.Height < 0 {
return errors.New("negative Height")
}
case *bcproto.BlockResponse:
_, err := types.BlockFromProto(msg.Block)
if err != nil {
return err
}
case *bcproto.NoBlockResponse:
if msg.Height < 0 {
return errors.New("negative Height")
}
case *bcproto.StatusResponse:
if msg.Base < 0 {
return errors.New("negative Base")
}
if msg.Height < 0 {
return errors.New("negative Height")
}
if msg.Base > msg.Height {
return fmt.Errorf("base %v cannot be greater than height %v", msg.Base, msg.Height)
}
case *bcproto.StatusRequest:
if msg.Base < 0 {
return errors.New("negative Base")
}
if msg.Height < 0 {
return errors.New("negative Height")
}
if msg.Base > msg.Height {
return fmt.Errorf("base %v cannot be greater than height %v", msg.Base, msg.Height)
}
default:
return fmt.Errorf("unknown message type %T", msg)
}
return nil
}

+ 88
- 0
blockchain/msgs_test.go View File

@ -0,0 +1,88 @@
package blockchain
import (
"testing"
"github.com/stretchr/testify/assert"
bcproto "github.com/tendermint/tendermint/proto/blockchain"
)
func TestBcBlockRequestMessageValidateBasic(t *testing.T) {
testCases := []struct {
testName string
requestHeight int64
expectErr bool
}{
{"Valid Request Message", 0, false},
{"Valid Request Message", 1, false},
{"Invalid Request Message", -1, true},
}
for _, tc := range testCases {
tc := tc
t.Run(tc.testName, func(t *testing.T) {
request := bcproto.BlockRequest{Height: tc.requestHeight}
assert.Equal(t, tc.expectErr, ValidateMsg(&request) != nil, "Validate Basic had an unexpected result")
})
}
}
func TestBcNoBlockResponseMessageValidateBasic(t *testing.T) {
testCases := []struct {
testName string
nonResponseHeight int64
expectErr bool
}{
{"Valid Non-Response Message", 0, false},
{"Valid Non-Response Message", 1, false},
{"Invalid Non-Response Message", -1, true},
}
for _, tc := range testCases {
tc := tc
t.Run(tc.testName, func(t *testing.T) {
nonResponse := bcproto.NoBlockResponse{Height: tc.nonResponseHeight}
assert.Equal(t, tc.expectErr, ValidateMsg(&nonResponse) != nil, "Validate Basic had an unexpected result")
})
}
}
func TestBcStatusRequestMessageValidateBasic(t *testing.T) {
testCases := []struct {
testName string
requestHeight int64
expectErr bool
}{
{"Valid Request Message", 0, false},
{"Valid Request Message", 1, false},
{"Invalid Request Message", -1, true},
}
for _, tc := range testCases {
tc := tc
t.Run(tc.testName, func(t *testing.T) {
request := bcproto.StatusRequest{Height: tc.requestHeight}
assert.Equal(t, tc.expectErr, ValidateMsg(&request) != nil, "Validate Basic had an unexpected result")
})
}
}
func TestBcStatusResponseMessageValidateBasic(t *testing.T) {
testCases := []struct {
testName string
responseHeight int64
expectErr bool
}{
{"Valid Response Message", 0, false},
{"Valid Response Message", 1, false},
{"Invalid Response Message", -1, true},
}
for _, tc := range testCases {
tc := tc
t.Run(tc.testName, func(t *testing.T) {
response := bcproto.StatusResponse{Height: tc.responseHeight}
assert.Equal(t, tc.expectErr, ValidateMsg(&response) != nil, "Validate Basic had an unexpected result")
})
}
}

+ 0
- 14
blockchain/v0/codec.go View File

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

+ 60
- 149
blockchain/v0/reactor.go View File

@ -1,15 +1,14 @@
package v0
import (
"errors"
"fmt"
"reflect"
"time"
amino "github.com/tendermint/go-amino"
bc "github.com/tendermint/tendermint/blockchain"
"github.com/tendermint/tendermint/libs/log"
"github.com/tendermint/tendermint/p2p"
bcproto "github.com/tendermint/tendermint/proto/blockchain"
sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/store"
"github.com/tendermint/tendermint/types"
@ -29,13 +28,6 @@ const (
statusUpdateIntervalSeconds = 10
// check if we should switch to consensus reactor
switchToConsensusIntervalSeconds = 1
// NOTE: keep up to date with bcBlockResponseMessage
bcBlockResponseMessagePrefixSize = 4
bcBlockResponseMessageFieldKeySize = 1
maxMsgSize = types.MaxBlockSizeBytes +
bcBlockResponseMessagePrefixSize +
bcBlockResponseMessageFieldKeySize
)
type consensusReactor interface {
@ -147,17 +139,21 @@ func (bcR *BlockchainReactor) GetChannels() []*p2p.ChannelDescriptor {
Priority: 10,
SendQueueCapacity: 1000,
RecvBufferCapacity: 50 * 4096,
RecvMessageCapacity: maxMsgSize,
RecvMessageCapacity: bc.MaxMsgSize,
},
}
}
// AddPeer implements Reactor by sending our state to peer.
func (bcR *BlockchainReactor) AddPeer(peer p2p.Peer) {
msgBytes := cdc.MustMarshalBinaryBare(&bcStatusResponseMessage{
Height: bcR.store.Height(),
msgBytes, err := bc.EncodeMsg(&bcproto.StatusResponse{
Base: bcR.store.Base(),
})
Height: bcR.store.Height()})
if err != nil {
bcR.Logger.Error("could not convert msg to protobuf", "err", err)
return
}
peer.Send(BlockchainChannel, msgBytes)
// it's OK if send fails. will try later in poolRoutine
@ -172,31 +168,47 @@ func (bcR *BlockchainReactor) RemovePeer(peer p2p.Peer, reason interface{}) {
// respondToPeer loads a block and sends it to the requesting peer,
// if we have it. Otherwise, we'll respond saying we don't have it.
func (bcR *BlockchainReactor) respondToPeer(msg *bcBlockRequestMessage,
func (bcR *BlockchainReactor) respondToPeer(msg *bcproto.BlockRequest,
src p2p.Peer) (queued bool) {
block := bcR.store.LoadBlock(msg.Height)
if block != nil {
msgBytes := cdc.MustMarshalBinaryBare(&bcBlockResponseMessage{Block: block})
bl, err := block.ToProto()
if err != nil {
bcR.Logger.Error("could not convert msg to protobuf", "err", err)
return false
}
msgBytes, err := bc.EncodeMsg(&bcproto.BlockResponse{Block: bl})
if err != nil {
bcR.Logger.Error("could not marshal msg", "err", err)
return false
}
return src.TrySend(BlockchainChannel, msgBytes)
}
bcR.Logger.Info("Peer asking for a block we don't have", "src", src, "height", msg.Height)
msgBytes := cdc.MustMarshalBinaryBare(&bcNoBlockResponseMessage{Height: msg.Height})
msgBytes, err := bc.EncodeMsg(&bcproto.NoBlockResponse{Height: msg.Height})
if err != nil {
bcR.Logger.Error("could not convert msg to protobuf", "err", err)
return false
}
return src.TrySend(BlockchainChannel, msgBytes)
}
// Receive implements Reactor by handling 4 types of messages (look below).
func (bcR *BlockchainReactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) {
msg, err := decodeMsg(msgBytes)
msg, err := bc.DecodeMsg(msgBytes)
if err != nil {
bcR.Logger.Error("Error decoding message", "src", src, "chId", chID, "msg", msg, "err", err, "bytes", msgBytes)
bcR.Switch.StopPeerForError(src, err)
return
}
if err = msg.ValidateBasic(); err != nil {
if err = bc.ValidateMsg(msg); err != nil {
bcR.Logger.Error("Peer sent us invalid msg", "peer", src, "msg", msg, "err", err)
bcR.Switch.StopPeerForError(src, err)
return
@ -205,20 +217,30 @@ func (bcR *BlockchainReactor) Receive(chID byte, src p2p.Peer, msgBytes []byte)
bcR.Logger.Debug("Receive", "src", src, "chID", chID, "msg", msg)
switch msg := msg.(type) {
case *bcBlockRequestMessage:
case *bcproto.BlockRequest:
bcR.respondToPeer(msg, src)
case *bcBlockResponseMessage:
bcR.pool.AddBlock(src.ID(), msg.Block, len(msgBytes))
case *bcStatusRequestMessage:
case *bcproto.BlockResponse:
bi, err := types.BlockFromProto(msg.Block)
if err != nil {
bcR.Logger.Error("Block content is invalid", "err", err)
return
}
bcR.pool.AddBlock(src.ID(), bi, len(msgBytes))
case *bcproto.StatusRequest:
// Send peer our state.
src.TrySend(BlockchainChannel, cdc.MustMarshalBinaryBare(&bcStatusResponseMessage{
msgBytes, err := bc.EncodeMsg(&bcproto.StatusResponse{
Height: bcR.store.Height(),
Base: bcR.store.Base(),
}))
case *bcStatusResponseMessage:
})
if err != nil {
bcR.Logger.Error("could not convert msg to protobut", "err", err)
return
}
src.TrySend(BlockchainChannel, msgBytes)
case *bcproto.StatusResponse:
// Got a peer status. Unverified.
bcR.pool.SetPeerRange(src.ID(), msg.Base, msg.Height)
case *bcNoBlockResponseMessage:
case *bcproto.NoBlockResponse:
bcR.Logger.Debug("Peer does not have requested block", "peer", src, "height", msg.Height)
default:
bcR.Logger.Error(fmt.Sprintf("Unknown message type %v", reflect.TypeOf(msg)))
@ -255,7 +277,12 @@ func (bcR *BlockchainReactor) poolRoutine(stateSynced bool) {
if peer == nil {
continue
}
msgBytes := cdc.MustMarshalBinaryBare(&bcBlockRequestMessage{request.Height})
msgBytes, err := bc.EncodeMsg(&bcproto.BlockRequest{Height: request.Height})
if err != nil {
bcR.Logger.Error("could not convert msg to proto", "err", err)
continue
}
queued := peer.TrySend(BlockchainChannel, msgBytes)
if !queued {
bcR.Logger.Debug("Send queue is full, drop block request", "peer", peer.ID(), "height", request.Height)
@ -381,132 +408,16 @@ FOR_LOOP:
// BroadcastStatusRequest broadcasts `BlockStore` base and height.
func (bcR *BlockchainReactor) BroadcastStatusRequest() error {
msgBytes := cdc.MustMarshalBinaryBare(&bcStatusRequestMessage{
bm, err := bc.EncodeMsg(&bcproto.StatusRequest{
Base: bcR.store.Base(),
Height: bcR.store.Height(),
})
bcR.Switch.Broadcast(BlockchainChannel, msgBytes)
return nil
}
//-----------------------------------------------------------------------------
// Messages
// BlockchainMessage is a generic message for this reactor.
type BlockchainMessage interface {
ValidateBasic() error
}
// RegisterBlockchainMessages registers the fast sync messages for amino encoding.
func RegisterBlockchainMessages(cdc *amino.Codec) {
cdc.RegisterInterface((*BlockchainMessage)(nil), nil)
cdc.RegisterConcrete(&bcBlockRequestMessage{}, "tendermint/blockchain/BlockRequest", nil)
cdc.RegisterConcrete(&bcBlockResponseMessage{}, "tendermint/blockchain/BlockResponse", nil)
cdc.RegisterConcrete(&bcNoBlockResponseMessage{}, "tendermint/blockchain/NoBlockResponse", nil)
cdc.RegisterConcrete(&bcStatusResponseMessage{}, "tendermint/blockchain/StatusResponse", nil)
cdc.RegisterConcrete(&bcStatusRequestMessage{}, "tendermint/blockchain/StatusRequest", nil)
}
func decodeMsg(bz []byte) (msg BlockchainMessage, err error) {
err = cdc.UnmarshalBinaryBare(bz, &msg)
return
}
//-------------------------------------
type bcBlockRequestMessage struct {
Height int64
}
// ValidateBasic performs basic validation.
func (m *bcBlockRequestMessage) ValidateBasic() error {
if m.Height < 0 {
return errors.New("negative Height")
}
return nil
}
func (m *bcBlockRequestMessage) String() string {
return fmt.Sprintf("[bcBlockRequestMessage %v]", m.Height)
}
type bcNoBlockResponseMessage struct {
Height int64
}
// ValidateBasic performs basic validation.
func (m *bcNoBlockResponseMessage) ValidateBasic() error {
if m.Height < 0 {
return errors.New("negative Height")
}
return nil
}
func (m *bcNoBlockResponseMessage) String() string {
return fmt.Sprintf("[bcNoBlockResponseMessage %d]", m.Height)
}
//-------------------------------------
type bcBlockResponseMessage struct {
Block *types.Block
}
// ValidateBasic performs basic validation.
func (m *bcBlockResponseMessage) ValidateBasic() error {
return m.Block.ValidateBasic()
}
func (m *bcBlockResponseMessage) String() string {
return fmt.Sprintf("[bcBlockResponseMessage %v]", m.Block.Height)
}
//-------------------------------------
type bcStatusRequestMessage struct {
Height int64
Base int64
}
// ValidateBasic performs basic validation.
func (m *bcStatusRequestMessage) ValidateBasic() error {
if m.Base < 0 {
return errors.New("negative Base")
}
if m.Height < 0 {
return errors.New("negative Height")
}
if m.Base > m.Height {
return fmt.Errorf("base %v cannot be greater than height %v", m.Base, m.Height)
if err != nil {
bcR.Logger.Error("could not convert msg to proto", "err", err)
return fmt.Errorf("could not convert msg to proto: %w", err)
}
return nil
}
func (m *bcStatusRequestMessage) String() string {
return fmt.Sprintf("[bcStatusRequestMessage %v:%v]", m.Base, m.Height)
}
//-------------------------------------
bcR.Switch.Broadcast(BlockchainChannel, bm)
type bcStatusResponseMessage struct {
Height int64
Base int64
}
// ValidateBasic performs basic validation.
func (m *bcStatusResponseMessage) ValidateBasic() error {
if m.Base < 0 {
return errors.New("negative Base")
}
if m.Height < 0 {
return errors.New("negative Height")
}
if m.Base > m.Height {
return fmt.Errorf("base %v cannot be greater than height %v", m.Base, m.Height)
}
return nil
}
func (m *bcStatusResponseMessage) String() string {
return fmt.Sprintf("[bcStatusResponseMessage %v:%v]", m.Base, m.Height)
}

+ 0
- 80
blockchain/v0/reactor_test.go View File

@ -263,86 +263,6 @@ func TestBadBlockStopsPeer(t *testing.T) {
assert.True(t, lastReactorPair.reactor.Switch.Peers().Size() < len(reactorPairs)-1)
}
func TestBcBlockRequestMessageValidateBasic(t *testing.T) {
testCases := []struct {
testName string
requestHeight int64
expectErr bool
}{
{"Valid Request Message", 0, false},
{"Valid Request Message", 1, false},
{"Invalid Request Message", -1, true},
}
for _, tc := range testCases {
tc := tc
t.Run(tc.testName, func(t *testing.T) {
request := bcBlockRequestMessage{Height: tc.requestHeight}
assert.Equal(t, tc.expectErr, request.ValidateBasic() != nil, "Validate Basic had an unexpected result")
})
}
}
func TestBcNoBlockResponseMessageValidateBasic(t *testing.T) {
testCases := []struct {
testName string
nonResponseHeight int64
expectErr bool
}{
{"Valid Non-Response Message", 0, false},
{"Valid Non-Response Message", 1, false},
{"Invalid Non-Response Message", -1, true},
}
for _, tc := range testCases {
tc := tc
t.Run(tc.testName, func(t *testing.T) {
nonResponse := bcNoBlockResponseMessage{Height: tc.nonResponseHeight}
assert.Equal(t, tc.expectErr, nonResponse.ValidateBasic() != nil, "Validate Basic had an unexpected result")
})
}
}
func TestBcStatusRequestMessageValidateBasic(t *testing.T) {
testCases := []struct {
testName string
requestHeight int64
expectErr bool
}{
{"Valid Request Message", 0, false},
{"Valid Request Message", 1, false},
{"Invalid Request Message", -1, true},
}
for _, tc := range testCases {
tc := tc
t.Run(tc.testName, func(t *testing.T) {
request := bcStatusRequestMessage{Height: tc.requestHeight}
assert.Equal(t, tc.expectErr, request.ValidateBasic() != nil, "Validate Basic had an unexpected result")
})
}
}
func TestBcStatusResponseMessageValidateBasic(t *testing.T) {
testCases := []struct {
testName string
responseHeight int64
expectErr bool
}{
{"Valid Response Message", 0, false},
{"Valid Response Message", 1, false},
{"Invalid Response Message", -1, true},
}
for _, tc := range testCases {
tc := tc
t.Run(tc.testName, func(t *testing.T) {
response := bcStatusResponseMessage{Height: tc.responseHeight}
assert.Equal(t, tc.expectErr, response.ValidateBasic() != nil, "Validate Basic had an unexpected result")
})
}
}
//----------------------------------------------
// utility funcs


+ 0
- 14
blockchain/v1/codec.go View File

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

+ 53
- 150
blockchain/v1/reactor.go View File

@ -1,16 +1,15 @@
package v1
import (
"errors"
"fmt"
"reflect"
"time"
amino "github.com/tendermint/go-amino"
"github.com/tendermint/tendermint/behaviour"
bc "github.com/tendermint/tendermint/blockchain"
"github.com/tendermint/tendermint/libs/log"
"github.com/tendermint/tendermint/p2p"
bcproto "github.com/tendermint/tendermint/proto/blockchain"
sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/store"
"github.com/tendermint/tendermint/types"
@ -24,13 +23,6 @@ const (
// ask for best height every 10s
statusUpdateIntervalSeconds = 10
// NOTE: keep up to date with bcBlockResponseMessage
bcBlockResponseMessagePrefixSize = 4
bcBlockResponseMessageFieldKeySize = 1
maxMsgSize = types.MaxBlockSizeBytes +
bcBlockResponseMessagePrefixSize +
bcBlockResponseMessageFieldKeySize
)
var (
@ -176,17 +168,21 @@ func (bcR *BlockchainReactor) GetChannels() []*p2p.ChannelDescriptor {
Priority: 10,
SendQueueCapacity: 2000,
RecvBufferCapacity: 50 * 4096,
RecvMessageCapacity: maxMsgSize,
RecvMessageCapacity: bc.MaxMsgSize,
},
}
}
// AddPeer implements Reactor by sending our state to peer.
func (bcR *BlockchainReactor) AddPeer(peer p2p.Peer) {
msgBytes := cdc.MustMarshalBinaryBare(&bcStatusResponseMessage{
msgBytes, err := bc.EncodeMsg(&bcproto.StatusResponse{
Base: bcR.store.Base(),
Height: bcR.store.Height(),
})
if err != nil {
bcR.Logger.Error("could not convert msg to protobuf", "err", err)
return
}
peer.Send(BlockchainChannel, msgBytes)
// it's OK if send fails. will try later in poolRoutine
@ -197,26 +193,44 @@ func (bcR *BlockchainReactor) AddPeer(peer p2p.Peer) {
// sendBlockToPeer loads a block and sends it to the requesting peer.
// If the block doesn't exist a bcNoBlockResponseMessage is sent.
// If all nodes are honest, no node should be requesting for a block that doesn't exist.
func (bcR *BlockchainReactor) sendBlockToPeer(msg *bcBlockRequestMessage,
func (bcR *BlockchainReactor) sendBlockToPeer(msg *bcproto.BlockRequest,
src p2p.Peer) (queued bool) {
block := bcR.store.LoadBlock(msg.Height)
if block != nil {
msgBytes := cdc.MustMarshalBinaryBare(&bcBlockResponseMessage{Block: block})
pbbi, err := block.ToProto()
if err != nil {
bcR.Logger.Error("Could not send block message to peer", "err", err)
return false
}
msgBytes, err := bc.EncodeMsg(&bcproto.BlockResponse{Block: pbbi})
if err != nil {
bcR.Logger.Error("unable to marshal msg", "err", err)
return false
}
return src.TrySend(BlockchainChannel, msgBytes)
}
bcR.Logger.Info("peer asking for a block we don't have", "src", src, "height", msg.Height)
msgBytes := cdc.MustMarshalBinaryBare(&bcNoBlockResponseMessage{Height: msg.Height})
msgBytes, err := bc.EncodeMsg(&bcproto.NoBlockResponse{Height: msg.Height})
if err != nil {
bcR.Logger.Error("unable to marshal msg", "err", err)
return false
}
return src.TrySend(BlockchainChannel, msgBytes)
}
func (bcR *BlockchainReactor) sendStatusResponseToPeer(msg *bcStatusRequestMessage, src p2p.Peer) (queued bool) {
msgBytes := cdc.MustMarshalBinaryBare(&bcStatusResponseMessage{
func (bcR *BlockchainReactor) sendStatusResponseToPeer(msg *bcproto.StatusRequest, src p2p.Peer) (queued bool) {
msgBytes, err := bc.EncodeMsg(&bcproto.StatusResponse{
Base: bcR.store.Base(),
Height: bcR.store.Height(),
})
if err != nil {
bcR.Logger.Error("unable to marshal msg", "err", err)
return false
}
return src.TrySend(BlockchainChannel, msgBytes)
}
@ -234,7 +248,7 @@ func (bcR *BlockchainReactor) RemovePeer(peer p2p.Peer, reason interface{}) {
// Receive implements Reactor by handling 4 types of messages (look below).
func (bcR *BlockchainReactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) {
msg, err := decodeMsg(msgBytes)
msg, err := bc.DecodeMsg(msgBytes)
if err != nil {
bcR.Logger.Error("error decoding message",
"src", src, "chId", chID, "msg", msg, "err", err, "bytes", msgBytes)
@ -242,7 +256,7 @@ func (bcR *BlockchainReactor) Receive(chID byte, src p2p.Peer, msgBytes []byte)
return
}
if err = msg.ValidateBasic(); err != nil {
if err = bc.ValidateMsg(msg); err != nil {
bcR.Logger.Error("peer sent us invalid msg", "peer", src, "msg", msg, "err", err)
_ = bcR.swReporter.Report(behaviour.BadMessage(src.ID(), err.Error()))
return
@ -251,33 +265,38 @@ func (bcR *BlockchainReactor) Receive(chID byte, src p2p.Peer, msgBytes []byte)
bcR.Logger.Debug("Receive", "src", src, "chID", chID, "msg", msg)
switch msg := msg.(type) {
case *bcBlockRequestMessage:
case *bcproto.BlockRequest:
if queued := bcR.sendBlockToPeer(msg, src); !queued {
// Unfortunately not queued since the queue is full.
bcR.Logger.Error("Could not send block message to peer", "src", src, "height", msg.Height)
}
case *bcStatusRequestMessage:
case *bcproto.StatusRequest:
// Send peer our state.
if queued := bcR.sendStatusResponseToPeer(msg, src); !queued {
// Unfortunately not queued since the queue is full.
bcR.Logger.Error("Could not send status message to peer", "src", src)
}
case *bcBlockResponseMessage:
case *bcproto.BlockResponse:
bi, err := types.BlockFromProto(msg.Block)
if err != nil {
bcR.Logger.Error("error transition block from protobuf", "err", err)
return
}
msgForFSM := bcReactorMessage{
event: blockResponseEv,
data: bReactorEventData{
peerID: src.ID(),
height: msg.Block.Height,
block: msg.Block,
height: bi.Height,
block: bi,
length: len(msgBytes),
},
}
bcR.Logger.Info("Received", "src", src, "height", msg.Block.Height)
bcR.Logger.Info("Received", "src", src, "height", bi.Height)
bcR.messagesForFSMCh <- msgForFSM
case *bcStatusResponseMessage:
case *bcproto.StatusResponse:
// Got a peer status. Unverified.
msgForFSM := bcReactorMessage{
event: statusResponseEv,
@ -461,10 +480,13 @@ func (bcR *BlockchainReactor) processBlock() error {
// Implements bcRNotifier
// sendStatusRequest broadcasts `BlockStore` height.
func (bcR *BlockchainReactor) sendStatusRequest() {
msgBytes := cdc.MustMarshalBinaryBare(&bcStatusRequestMessage{
msgBytes, err := bc.EncodeMsg(&bcproto.StatusRequest{
Base: bcR.store.Base(),
Height: bcR.store.Height(),
})
if err != nil {
panic(err)
}
bcR.Switch.Broadcast(BlockchainChannel, msgBytes)
}
@ -476,7 +498,10 @@ func (bcR *BlockchainReactor) sendBlockRequest(peerID p2p.ID, height int64) erro
return errNilPeerForBlockRequest
}
msgBytes := cdc.MustMarshalBinaryBare(&bcBlockRequestMessage{height})
msgBytes, err := bc.EncodeMsg(&bcproto.BlockRequest{Height: height})
if err != nil {
return err
}
queued := peer.TrySend(BlockchainChannel, msgBytes)
if !queued {
return errSendQueueFull
@ -533,125 +558,3 @@ func (bcR *BlockchainReactor) resetStateTimer(name string, timer **time.Timer, t
(*timer).Reset(timeout)
}
}
//-----------------------------------------------------------------------------
// Messages
// BlockchainMessage is a generic message for this reactor.
type BlockchainMessage interface {
ValidateBasic() error
}
// RegisterBlockchainMessages registers the fast sync messages for amino encoding.
func RegisterBlockchainMessages(cdc *amino.Codec) {
cdc.RegisterInterface((*BlockchainMessage)(nil), nil)
cdc.RegisterConcrete(&bcBlockRequestMessage{}, "tendermint/blockchain/BlockRequest", nil)
cdc.RegisterConcrete(&bcBlockResponseMessage{}, "tendermint/blockchain/BlockResponse", nil)
cdc.RegisterConcrete(&bcNoBlockResponseMessage{}, "tendermint/blockchain/NoBlockResponse", nil)
cdc.RegisterConcrete(&bcStatusResponseMessage{}, "tendermint/blockchain/StatusResponse", nil)
cdc.RegisterConcrete(&bcStatusRequestMessage{}, "tendermint/blockchain/StatusRequest", nil)
}
func decodeMsg(bz []byte) (msg BlockchainMessage, err error) {
err = cdc.UnmarshalBinaryBare(bz, &msg)
return
}
//-------------------------------------
type bcBlockRequestMessage struct {
Height int64
}
// ValidateBasic performs basic validation.
func (m *bcBlockRequestMessage) ValidateBasic() error {
if m.Height < 0 {
return errors.New("negative Height")
}
return nil
}
func (m *bcBlockRequestMessage) String() string {
return fmt.Sprintf("[bcBlockRequestMessage %v]", m.Height)
}
type bcNoBlockResponseMessage struct {
Height int64
}
// ValidateBasic performs basic validation.
func (m *bcNoBlockResponseMessage) ValidateBasic() error {
if m.Height < 0 {
return errors.New("negative Height")
}
return nil
}
func (m *bcNoBlockResponseMessage) String() string {
return fmt.Sprintf("[bcNoBlockResponseMessage %d]", m.Height)
}
//-------------------------------------
type bcBlockResponseMessage struct {
Block *types.Block
}
// ValidateBasic performs basic validation.
func (m *bcBlockResponseMessage) ValidateBasic() error {
return m.Block.ValidateBasic()
}
func (m *bcBlockResponseMessage) String() string {
return fmt.Sprintf("[bcBlockResponseMessage %v]", m.Block.Height)
}
//-------------------------------------
type bcStatusRequestMessage struct {
Height int64
Base int64
}
// ValidateBasic performs basic validation.
func (m *bcStatusRequestMessage) ValidateBasic() error {
if m.Height < 0 {
return errors.New("negative Height")
}
if m.Base < 0 {
return errors.New("negative Base")
}
if m.Base > m.Height {
return fmt.Errorf("base %v cannot be greater than height %v", m.Base, m.Height)
}
return nil
}
func (m *bcStatusRequestMessage) String() string {
return fmt.Sprintf("[bcStatusRequestMessage %v:%v]", m.Base, m.Height)
}
//-------------------------------------
type bcStatusResponseMessage struct {
Height int64
Base int64
}
// ValidateBasic performs basic validation.
func (m *bcStatusResponseMessage) ValidateBasic() error {
if m.Height < 0 {
return errors.New("negative Height")
}
if m.Base < 0 {
return errors.New("negative Base")
}
if m.Base > m.Height {
return fmt.Errorf("base %v cannot be greater than height %v", m.Base, m.Height)
}
return nil
}
func (m *bcStatusResponseMessage) String() string {
return fmt.Sprintf("[bcStatusResponseMessage %v:%v]", m.Base, m.Height)
}

+ 0
- 80
blockchain/v1/reactor_test.go View File

@ -338,86 +338,6 @@ outerFor:
assert.True(t, lastReactorPair.bcR.Switch.Peers().Size() < len(reactorPairs)-1)
}
func TestBcBlockRequestMessageValidateBasic(t *testing.T) {
testCases := []struct {
testName string
requestHeight int64
expectErr bool
}{
{"Valid Request Message", 0, false},
{"Valid Request Message", 1, false},
{"Invalid Request Message", -1, true},
}
for _, tc := range testCases {
tc := tc
t.Run(tc.testName, func(t *testing.T) {
request := bcBlockRequestMessage{Height: tc.requestHeight}
assert.Equal(t, tc.expectErr, request.ValidateBasic() != nil, "Validate Basic had an unexpected result")
})
}
}
func TestBcNoBlockResponseMessageValidateBasic(t *testing.T) {
testCases := []struct {
testName string
nonResponseHeight int64
expectErr bool
}{
{"Valid Non-Response Message", 0, false},
{"Valid Non-Response Message", 1, false},
{"Invalid Non-Response Message", -1, true},
}
for _, tc := range testCases {
tc := tc
t.Run(tc.testName, func(t *testing.T) {
nonResponse := bcNoBlockResponseMessage{Height: tc.nonResponseHeight}
assert.Equal(t, tc.expectErr, nonResponse.ValidateBasic() != nil, "Validate Basic had an unexpected result")
})
}
}
func TestBcStatusRequestMessageValidateBasic(t *testing.T) {
testCases := []struct {
testName string
requestHeight int64
expectErr bool
}{
{"Valid Request Message", 0, false},
{"Valid Request Message", 1, false},
{"Invalid Request Message", -1, true},
}
for _, tc := range testCases {
tc := tc
t.Run(tc.testName, func(t *testing.T) {
request := bcStatusRequestMessage{Height: tc.requestHeight}
assert.Equal(t, tc.expectErr, request.ValidateBasic() != nil, "Validate Basic had an unexpected result")
})
}
}
func TestBcStatusResponseMessageValidateBasic(t *testing.T) {
testCases := []struct {
testName string
responseHeight int64
expectErr bool
}{
{"Valid Response Message", 0, false},
{"Valid Response Message", 1, false},
{"Invalid Response Message", -1, true},
}
for _, tc := range testCases {
tc := tc
t.Run(tc.testName, func(t *testing.T) {
response := bcStatusResponseMessage{Height: tc.responseHeight}
assert.Equal(t, tc.expectErr, response.ValidateBasic() != nil, "Validate Basic had an unexpected result")
})
}
}
//----------------------------------------------
// utility funcs


+ 0
- 14
blockchain/v2/codec.go View File

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

+ 36
- 10
blockchain/v2/io.go View File

@ -3,7 +3,9 @@ package v2
import (
"fmt"
bc "github.com/tendermint/tendermint/blockchain"
"github.com/tendermint/tendermint/p2p"
bcproto "github.com/tendermint/tendermint/proto/blockchain"
"github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/types"
)
@ -14,7 +16,7 @@ type iIO interface {
sendBlockNotFound(height int64, peerID p2p.ID) error
sendStatusResponse(base int64, height int64, peerID p2p.ID) error
broadcastStatusRequest(base int64, height int64)
broadcastStatusRequest(base int64, height int64) error
trySwitchToConsensus(state state.State, skipWAL bool) bool
}
@ -45,8 +47,11 @@ func (sio *switchIO) sendBlockRequest(peerID p2p.ID, height int64) error {
if peer == nil {
return fmt.Errorf("peer not found")
}
msgBytes, err := bc.EncodeMsg(&bcproto.BlockRequest{Height: height})
if err != nil {
return err
}
msgBytes := cdc.MustMarshalBinaryBare(&bcBlockRequestMessage{Height: height})
queued := peer.TrySend(BlockchainChannel, msgBytes)
if !queued {
return fmt.Errorf("send queue full")
@ -59,7 +64,11 @@ func (sio *switchIO) sendStatusResponse(base int64, height int64, peerID p2p.ID)
if peer == nil {
return fmt.Errorf("peer not found")
}
msgBytes := cdc.MustMarshalBinaryBare(&bcStatusResponseMessage{Base: base, Height: height})
msgBytes, err := bc.EncodeMsg(&bcproto.StatusResponse{Height: height, Base: base})
if err != nil {
return err
}
if queued := peer.TrySend(BlockchainChannel, msgBytes); !queued {
return fmt.Errorf("peer queue full")
@ -76,7 +85,16 @@ func (sio *switchIO) sendBlockToPeer(block *types.Block, peerID p2p.ID) error {
if block == nil {
panic("trying to send nil block")
}
msgBytes := cdc.MustMarshalBinaryBare(&bcBlockResponseMessage{Block: block})
bpb, err := block.ToProto()
if err != nil {
return err
}
msgBytes, err := bc.EncodeMsg(&bcproto.BlockResponse{Block: bpb})
if err != nil {
return err
}
if queued := peer.TrySend(BlockchainChannel, msgBytes); !queued {
return fmt.Errorf("peer queue full")
}
@ -89,7 +107,11 @@ func (sio *switchIO) sendBlockNotFound(height int64, peerID p2p.ID) error {
if peer == nil {
return fmt.Errorf("peer not found")
}
msgBytes := cdc.MustMarshalBinaryBare(&bcNoBlockResponseMessage{Height: height})
msgBytes, err := bc.EncodeMsg(&bcproto.NoBlockResponse{Height: height})
if err != nil {
return err
}
if queued := peer.TrySend(BlockchainChannel, msgBytes); !queued {
return fmt.Errorf("peer queue full")
}
@ -105,14 +127,18 @@ func (sio *switchIO) trySwitchToConsensus(state state.State, skipWAL bool) bool
return ok
}
func (sio *switchIO) broadcastStatusRequest(base int64, height int64) {
func (sio *switchIO) broadcastStatusRequest(base, height int64) error {
if height == 0 && base > 0 {
base = 0
}
msgBytes := cdc.MustMarshalBinaryBare(&bcStatusRequestMessage{
Base: base,
Height: height,
})
msgBytes, err := bc.EncodeMsg(&bcproto.StatusRequest{Base: base, Height: height})
if err != nil {
return err
}
// XXX: maybe we should use an io specific peer list here
sio.sw.Broadcast(BlockchainChannel, msgBytes)
return nil
}

+ 16
- 143
blockchain/v2/reactor.go View File

@ -6,11 +6,11 @@ import (
"sync"
"time"
"github.com/tendermint/go-amino"
"github.com/tendermint/tendermint/behaviour"
bc "github.com/tendermint/tendermint/blockchain"
"github.com/tendermint/tendermint/libs/log"
"github.com/tendermint/tendermint/p2p"
bcproto "github.com/tendermint/tendermint/proto/blockchain"
"github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/types"
)
@ -20,109 +20,6 @@ const (
chBufferSize int = 1000
)
//-------------------------------------
type bcBlockRequestMessage struct {
Height int64
}
// ValidateBasic performs basic validation.
func (m *bcBlockRequestMessage) ValidateBasic() error {
if m.Height < 0 {
return errors.New("negative Height")
}
return nil
}
func (m *bcBlockRequestMessage) String() string {
return fmt.Sprintf("[bcBlockRequestMessage %v]", m.Height)
}
type bcNoBlockResponseMessage struct {
Height int64
}
// ValidateBasic performs basic validation.
func (m *bcNoBlockResponseMessage) ValidateBasic() error {
if m.Height < 0 {
return errors.New("negative Height")
}
return nil
}
func (m *bcNoBlockResponseMessage) String() string {
return fmt.Sprintf("[bcNoBlockResponseMessage %d]", m.Height)
}
//-------------------------------------
type bcBlockResponseMessage struct {
Block *types.Block
}
// ValidateBasic performs basic validation.
func (m *bcBlockResponseMessage) ValidateBasic() error {
if m.Block == nil {
return errors.New("block response message has nil block")
}
return m.Block.ValidateBasic()
}
func (m *bcBlockResponseMessage) String() string {
return fmt.Sprintf("[bcBlockResponseMessage %v]", m.Block.Height)
}
//-------------------------------------
type bcStatusRequestMessage struct {
Height int64
Base int64
}
// ValidateBasic performs basic validation.
func (m *bcStatusRequestMessage) ValidateBasic() error {
if m.Base < 0 {
return errors.New("negative Base")
}
if m.Height < 0 {
return errors.New("negative Height")
}
if m.Base > m.Height {
return fmt.Errorf("base %v cannot be greater than height %v", m.Base, m.Height)
}
return nil
}
func (m *bcStatusRequestMessage) String() string {
return fmt.Sprintf("[bcStatusRequestMessage %v:%v]", m.Base, m.Height)
}
//-------------------------------------
type bcStatusResponseMessage struct {
Height int64
Base int64
}
// ValidateBasic performs basic validation.
func (m *bcStatusResponseMessage) ValidateBasic() error {
if m.Base < 0 {
return errors.New("negative Base")
}
if m.Height < 0 {
return errors.New("negative Height")
}
if m.Base > m.Height {
return fmt.Errorf("base %v cannot be greater than height %v", m.Base, m.Height)
}
return nil
}
func (m *bcStatusResponseMessage) String() string {
return fmt.Sprintf("[bcStatusResponseMessage %v:%v]", m.Base, m.Height)
}
type blockStore interface {
LoadBlock(height int64) *types.Block
SaveBlock(*types.Block, *types.PartSet, *types.Commit)
@ -512,38 +409,9 @@ func (r *BlockchainReactor) Stop() error {
return nil
}
const (
// NOTE: keep up to date with bcBlockResponseMessage
bcBlockResponseMessagePrefixSize = 4
bcBlockResponseMessageFieldKeySize = 1
maxMsgSize = types.MaxBlockSizeBytes +
bcBlockResponseMessagePrefixSize +
bcBlockResponseMessageFieldKeySize
)
// BlockchainMessage is a generic message for this reactor.
type BlockchainMessage interface {
ValidateBasic() error
}
// RegisterBlockchainMessages registers the fast sync messages for amino encoding.
func RegisterBlockchainMessages(cdc *amino.Codec) {
cdc.RegisterInterface((*BlockchainMessage)(nil), nil)
cdc.RegisterConcrete(&bcBlockRequestMessage{}, "tendermint/blockchain/BlockRequest", nil)
cdc.RegisterConcrete(&bcBlockResponseMessage{}, "tendermint/blockchain/BlockResponse", nil)
cdc.RegisterConcrete(&bcNoBlockResponseMessage{}, "tendermint/blockchain/NoBlockResponse", nil)
cdc.RegisterConcrete(&bcStatusResponseMessage{}, "tendermint/blockchain/StatusResponse", nil)
cdc.RegisterConcrete(&bcStatusRequestMessage{}, "tendermint/blockchain/StatusRequest", nil)
}
func decodeMsg(bz []byte) (msg BlockchainMessage, err error) {
err = cdc.UnmarshalBinaryBare(bz, &msg)
return
}
// Receive implements Reactor by handling different message types.
func (r *BlockchainReactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) {
msg, err := decodeMsg(msgBytes)
msg, err := bc.DecodeMsg(msgBytes)
if err != nil {
r.logger.Error("error decoding message",
"src", src.ID(), "chId", chID, "msg", msg, "err", err, "bytes", msgBytes)
@ -551,7 +419,7 @@ func (r *BlockchainReactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) {
return
}
if err = msg.ValidateBasic(); err != nil {
if err = bc.ValidateMsg(msg); err != nil {
r.logger.Error("peer sent us invalid msg", "peer", src, "msg", msg, "err", err)
_ = r.reporter.Report(behaviour.BadMessage(src.ID(), err.Error()))
return
@ -560,12 +428,12 @@ func (r *BlockchainReactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) {
r.logger.Debug("Receive", "src", src.ID(), "chID", chID, "msg", msg)
switch msg := msg.(type) {
case *bcStatusRequestMessage:
case *bcproto.StatusRequest:
if err := r.io.sendStatusResponse(r.store.Base(), r.store.Height(), src.ID()); err != nil {
r.logger.Error("Could not send status message to peer", "src", src)
}
case *bcBlockRequestMessage:
case *bcproto.BlockRequest:
block := r.store.LoadBlock(msg.Height)
if block != nil {
if err = r.io.sendBlockToPeer(block, src.ID()); err != nil {
@ -579,26 +447,31 @@ func (r *BlockchainReactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) {
}
}
case *bcStatusResponseMessage:
case *bcproto.StatusResponse:
r.mtx.RLock()
if r.events != nil {
r.events <- bcStatusResponse{peerID: src.ID(), base: msg.Base, height: msg.Height}
}
r.mtx.RUnlock()
case *bcBlockResponseMessage:
case *bcproto.BlockResponse:
r.mtx.RLock()
bi, err := types.BlockFromProto(msg.Block)
if err != nil {
r.logger.Error("error transitioning block from protobuf", "err", err)
return
}
if r.events != nil {
r.events <- bcBlockResponse{
peerID: src.ID(),
block: msg.Block,
block: bi,
size: int64(len(msgBytes)),
time: time.Now(),
}
}
r.mtx.RUnlock()
case *bcNoBlockResponseMessage:
case *bcproto.NoBlockResponse:
r.mtx.RLock()
if r.events != nil {
r.events <- bcNoBlockResponse{peerID: src.ID(), height: msg.Height, time: time.Now()}
@ -640,7 +513,7 @@ func (r *BlockchainReactor) GetChannels() []*p2p.ChannelDescriptor {
Priority: 10,
SendQueueCapacity: 2000,
RecvBufferCapacity: 50 * 4096,
RecvMessageCapacity: maxMsgSize,
RecvMessageCapacity: bc.MaxMsgSize,
},
}
}

+ 20
- 10
blockchain/v2/reactor_test.go View File

@ -14,12 +14,14 @@ import (
abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/behaviour"
bc "github.com/tendermint/tendermint/blockchain"
cfg "github.com/tendermint/tendermint/config"
"github.com/tendermint/tendermint/libs/log"
"github.com/tendermint/tendermint/libs/service"
"github.com/tendermint/tendermint/mempool/mock"
"github.com/tendermint/tendermint/p2p"
"github.com/tendermint/tendermint/p2p/conn"
bcproto "github.com/tendermint/tendermint/proto/blockchain"
"github.com/tendermint/tendermint/proxy"
sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/store"
@ -124,7 +126,8 @@ func (sio *mockSwitchIo) trySwitchToConsensus(state sm.State, skipWAL bool) bool
return true
}
func (sio *mockSwitchIo) broadcastStatusRequest(base int64, height int64) {
func (sio *mockSwitchIo) broadcastStatusRequest(base int64, height int64) error {
return nil
}
type testReactorParams struct {
@ -370,10 +373,10 @@ func TestReactorHelperMode(t *testing.T) {
name: "status request",
params: params,
msgs: []testEvent{
{"P1", bcStatusRequestMessage{}},
{"P1", bcBlockRequestMessage{Height: 13}},
{"P1", bcBlockRequestMessage{Height: 20}},
{"P1", bcBlockRequestMessage{Height: 22}},
{"P1", bcproto.StatusRequest{}},
{"P1", bcproto.BlockRequest{Height: 13}},
{"P1", bcproto.BlockRequest{Height: 20}},
{"P1", bcproto.BlockRequest{Height: 22}},
},
},
}
@ -389,18 +392,25 @@ func TestReactorHelperMode(t *testing.T) {
for i := 0; i < len(tt.msgs); i++ {
step := tt.msgs[i]
switch ev := step.event.(type) {
case bcStatusRequestMessage:
case bcproto.StatusRequest:
old := mockSwitch.numStatusResponse
reactor.Receive(channelID, mockPeer{id: p2p.ID(step.peer)}, cdc.MustMarshalBinaryBare(ev))
msg, err := bc.EncodeMsg(&ev)
assert.NoError(t, err)
reactor.Receive(channelID, mockPeer{id: p2p.ID(step.peer)}, msg)
assert.Equal(t, old+1, mockSwitch.numStatusResponse)
case bcBlockRequestMessage:
case bcproto.BlockRequest:
if ev.Height > params.startHeight {
old := mockSwitch.numNoBlockResponse
reactor.Receive(channelID, mockPeer{id: p2p.ID(step.peer)}, cdc.MustMarshalBinaryBare(ev))
msg, err := bc.EncodeMsg(&ev)
assert.NoError(t, err)
reactor.Receive(channelID, mockPeer{id: p2p.ID(step.peer)}, msg)
assert.Equal(t, old+1, mockSwitch.numNoBlockResponse)
} else {
old := mockSwitch.numBlockResponse
reactor.Receive(channelID, mockPeer{id: p2p.ID(step.peer)}, cdc.MustMarshalBinaryBare(ev))
msg, err := bc.EncodeMsg(&ev)
assert.NoError(t, err)
assert.NoError(t, err)
reactor.Receive(channelID, mockPeer{id: p2p.ID(step.peer)}, msg)
assert.Equal(t, old+1, mockSwitch.numBlockResponse)
}
}


+ 1
- 1
config/toml.go View File

@ -345,7 +345,7 @@ temp_dir = "{{ .StateSync.TempDir }}"
# Fast Sync version to use:
# 1) "v0" (default) - the legacy fast sync implementation
# 2) "v1" - refactor of v0 version for better testability
# 3) "v2" - refactor of v1 version for better usability
# 2) "v2" - complete redesign of v0, optimized for testability & readability
version = "{{ .FastSync.Version }}"
##### consensus configuration options #####


+ 2
- 2
consensus/reactor.go View File

@ -1047,8 +1047,8 @@ func (ps *PeerState) PickVoteToSend(votes types.VoteSetReader) (vote *types.Vote
return nil, false
}
height, round, votesType, size := votes.GetHeight(), votes.GetRound(),
tmproto.SignedMsgType(votes.Type()), votes.Size()
height, round, votesType, size :=
votes.GetHeight(), votes.GetRound(), tmproto.SignedMsgType(votes.Type()), votes.Size()
// Lazily set data using 'votes'.
if votes.IsCommit() {


+ 1
- 0
docs/tendermint-core/configuration.md View File

@ -263,6 +263,7 @@ max_tx_bytes = 1048576
# Fast Sync version to use:
# 1) "v0" (default) - the legacy fast sync implementation
# 2) "v1" - refactor of v0 version for better testability
# 2) "v2" - complete redesign of v0, optimized for testability & readability
version = "v0"
##### consensus configuration options #####


+ 1
- 0
go.mod View File

@ -30,4 +30,5 @@ require (
golang.org/x/crypto v0.0.0-20200406173513-056763e48d71
golang.org/x/net v0.0.0-20200324143707-d3edc9973b7e
google.golang.org/grpc v1.29.1
google.golang.org/protobuf v1.23.0 // indirect
)

+ 2
- 1
go.sum View File

@ -160,7 +160,6 @@ github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5y
github.com/golang/protobuf v1.3.2 h1:6nsPYzhq5kReh6QImI3k5qWzO4PEbvbIW2cwSfR/6xs=
github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw=
github.com/golang/protobuf v1.3.5/go.mod h1:6O5/vntMXwX2lRkT1hjjk0nAC1IDOTvTlVgjlRvqsdk=
github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8=
github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA=
github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs=
@ -653,6 +652,8 @@ google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQ
google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miEFZTKqfCUM6K7xSMQL9OKL/b6hQv+e19PK+JZNE=
google.golang.org/protobuf v1.21.0 h1:qdOKuR/EIArgaWNjetjgTzgVTAZ+S/WXVrq9HW9zimw=
google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo=
google.golang.org/protobuf v1.23.0 h1:4MY060fB1DLGMB/7MBTLnwQUY6+F09GEiz6SsrNqyzM=
google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU=
gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw=
gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405 h1:yhCVgyC4o1eVCa2tZl7eS0r+SDo693bJlVdllGtEeKM=
gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=


+ 45
- 41
proto/blockchain/msgs.pb.go View File

@ -5,7 +5,6 @@ package blockchain
import (
fmt "fmt"
_ "github.com/gogo/protobuf/gogoproto"
proto "github.com/gogo/protobuf/proto"
types "github.com/tendermint/tendermint/proto/types"
io "io"
@ -116,7 +115,7 @@ func (m *NoBlockResponse) GetHeight() int64 {
// BlockResponse returns block to the requested
type BlockResponse struct {
Block types.Block `protobuf:"bytes,1,opt,name=block,proto3" json:"block"`
Block *types.Block `protobuf:"bytes,1,opt,name=block,proto3" json:"block,omitempty"`
}
func (m *BlockResponse) Reset() { *m = BlockResponse{} }
@ -152,11 +151,11 @@ func (m *BlockResponse) XXX_DiscardUnknown() {
var xxx_messageInfo_BlockResponse proto.InternalMessageInfo
func (m *BlockResponse) GetBlock() types.Block {
func (m *BlockResponse) GetBlock() *types.Block {
if m != nil {
return m.Block
}
return types.Block{}
return nil
}
// StatusRequest requests the status of a node (Height & Base)
@ -401,33 +400,31 @@ func init() {
func init() { proto.RegisterFile("proto/blockchain/msgs.proto", fileDescriptor_ecf660069f8bb334) }
var fileDescriptor_ecf660069f8bb334 = []byte{
// 401 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x93, 0x41, 0x4f, 0xe2, 0x40,
0x14, 0xc7, 0xdb, 0x2d, 0xb0, 0xc9, 0x83, 0x42, 0xb6, 0x87, 0x5d, 0x02, 0xd9, 0xee, 0xa6, 0x07,
0x22, 0x6a, 0xda, 0x88, 0x27, 0xa3, 0xa7, 0x9e, 0x88, 0x46, 0x63, 0x4a, 0xe2, 0x81, 0x0b, 0x69,
0x61, 0xd2, 0x36, 0xda, 0x4e, 0xed, 0x4c, 0x0f, 0x7c, 0x0b, 0x3f, 0x92, 0x47, 0x8e, 0x1c, 0x3d,
0x19, 0x03, 0x5f, 0xc4, 0x30, 0x53, 0x0a, 0xad, 0x8a, 0xdc, 0x66, 0xfe, 0x7d, 0xef, 0xf7, 0xfe,
0xf3, 0xfe, 0x29, 0xb4, 0xa3, 0x18, 0x53, 0x6c, 0x38, 0x0f, 0x78, 0x7c, 0x3f, 0xf6, 0x6c, 0x3f,
0x34, 0x02, 0xe2, 0x12, 0x9d, 0xa9, 0x4a, 0x9b, 0xa2, 0x70, 0x82, 0xe2, 0xc0, 0x0f, 0x29, 0x57,
0xf4, 0x4d, 0x5d, 0xab, 0x43, 0x3d, 0x3f, 0x9e, 0x8c, 0x22, 0x3b, 0xa6, 0x53, 0x83, 0x53, 0x5c,
0xec, 0xe2, 0xcd, 0x89, 0xb7, 0xb4, 0xfe, 0x70, 0x85, 0x4e, 0x23, 0x44, 0xf8, 0x1c, 0xfe, 0x41,
0xeb, 0x40, 0xcd, 0x5c, 0x5d, 0x2d, 0xf4, 0x98, 0x20, 0x42, 0x95, 0xdf, 0x50, 0xf1, 0x90, 0xef,
0x7a, 0xb4, 0x29, 0xfe, 0x17, 0x0f, 0x24, 0x2b, 0xbd, 0x69, 0x5d, 0x68, 0xdc, 0xe0, 0xb4, 0x92,
0x44, 0x38, 0x24, 0xe8, 0xcb, 0xd2, 0x4b, 0x90, 0xf3, 0x85, 0x67, 0x50, 0x66, 0x23, 0x59, 0x5d,
0xb5, 0xf7, 0x57, 0xff, 0xf0, 0x22, 0xe6, 0x4b, 0x67, 0x5d, 0x66, 0x69, 0xf6, 0xfa, 0x4f, 0xb0,
0x78, 0x87, 0x76, 0x0e, 0xf2, 0x80, 0xda, 0x34, 0x21, 0xdf, 0xf8, 0x53, 0x14, 0x28, 0x39, 0x36,
0x41, 0xcd, 0x1f, 0x4c, 0x65, 0x67, 0xed, 0x02, 0xea, 0xeb, 0xe6, 0xdd, 0x96, 0x3f, 0xed, 0x7e,
0x96, 0xe0, 0xe7, 0x35, 0x22, 0xc4, 0x76, 0x91, 0x72, 0x0b, 0x32, 0xf3, 0x33, 0x8a, 0xb9, 0x8d,
0xf4, 0x25, 0x5d, 0x7d, 0x47, 0x36, 0xfa, 0xf6, 0x5e, 0xfb, 0x82, 0x55, 0x73, 0xb6, 0xf7, 0x3c,
0x84, 0x5f, 0x21, 0x1e, 0xad, 0xa1, 0xdc, 0x1e, 0x1b, 0x5f, 0xed, 0x1d, 0xef, 0xa4, 0x16, 0x52,
0xe8, 0x0b, 0x56, 0x23, 0x2c, 0x04, 0x33, 0x80, 0x7a, 0x01, 0x2c, 0x31, 0xf0, 0xe1, 0x3e, 0x76,
0x33, 0xac, 0xec, 0x14, 0xa1, 0x84, 0x2d, 0x33, 0xdb, 0x41, 0x69, 0x0f, 0x68, 0x2e, 0xbc, 0x15,
0x94, 0xe4, 0xd2, 0xbc, 0x83, 0x46, 0x06, 0x4d, 0xad, 0x96, 0x19, 0xf5, 0x68, 0x2f, 0x6a, 0xe6,
0xb5, 0x4e, 0x72, 0x8a, 0x59, 0x06, 0x89, 0x24, 0x81, 0x79, 0x35, 0x5b, 0xa8, 0xe2, 0x7c, 0xa1,
0x8a, 0x6f, 0x0b, 0x55, 0x7c, 0x5a, 0xaa, 0xc2, 0x7c, 0xa9, 0x0a, 0x2f, 0x4b, 0x55, 0x18, 0x9e,
0xb8, 0x3e, 0xf5, 0x12, 0x47, 0x1f, 0xe3, 0xc0, 0xd8, 0x4c, 0xda, 0x3e, 0x16, 0x7f, 0x49, 0xa7,
0xc2, 0x94, 0xd3, 0xf7, 0x00, 0x00, 0x00, 0xff, 0xff, 0xaf, 0xc4, 0x3a, 0x27, 0xad, 0x03, 0x00,
0x00,
// 371 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x93, 0xc1, 0x4e, 0xf2, 0x40,
0x14, 0x85, 0xdb, 0xbf, 0xc0, 0x9f, 0x5c, 0x28, 0xc4, 0x2e, 0x94, 0x48, 0x6c, 0x4c, 0x17, 0x46,
0xd4, 0x0c, 0x11, 0x96, 0xba, 0x22, 0x2e, 0x48, 0x8c, 0xc6, 0x94, 0xc4, 0x05, 0x1b, 0xd2, 0xe2,
0xa4, 0x6d, 0xb4, 0x1d, 0xe4, 0x4e, 0x17, 0xbe, 0x85, 0x8f, 0xe4, 0xd2, 0x25, 0x4b, 0x97, 0x06,
0x5e, 0xc4, 0x30, 0x53, 0x0a, 0xad, 0x8a, 0xec, 0x66, 0x6e, 0xce, 0xf9, 0xe6, 0xf4, 0xdc, 0x14,
0x1a, 0xe3, 0x09, 0xe3, 0xac, 0xe5, 0x3e, 0xb1, 0xd1, 0xe3, 0xc8, 0x77, 0x82, 0xa8, 0x15, 0xa2,
0x87, 0x44, 0x4c, 0x8d, 0x06, 0xa7, 0xd1, 0x03, 0x9d, 0x84, 0x41, 0xc4, 0xe5, 0x84, 0xac, 0x74,
0xfb, 0x7b, 0xd2, 0xc9, 0x5f, 0xc6, 0x14, 0xa5, 0x5f, 0x6a, 0xac, 0x23, 0xa8, 0x74, 0x17, 0x57,
0x9b, 0x3e, 0xc7, 0x14, 0xb9, 0xb1, 0x0b, 0x25, 0x9f, 0x06, 0x9e, 0xcf, 0xeb, 0xea, 0xa1, 0x7a,
0xac, 0xd9, 0xc9, 0xcd, 0x6a, 0x42, 0xed, 0x96, 0x25, 0x4a, 0x1c, 0xb3, 0x08, 0xe9, 0xaf, 0xd2,
0x2b, 0xd0, 0xb3, 0xc2, 0x0e, 0x14, 0xc5, 0x93, 0x42, 0x57, 0x6e, 0x1f, 0x90, 0x6f, 0x49, 0x45,
0x2e, 0x22, 0x5d, 0x52, 0x6b, 0x5d, 0x80, 0xde, 0xe7, 0x0e, 0x8f, 0xf1, 0x8f, 0x64, 0x86, 0x01,
0x05, 0xd7, 0x41, 0x5a, 0xff, 0x27, 0xa6, 0xe2, 0x6c, 0x5d, 0x42, 0x75, 0x69, 0xde, 0x1c, 0xf6,
0x47, 0xf7, 0x9b, 0x06, 0xff, 0x6f, 0x28, 0xa2, 0xe3, 0x51, 0xe3, 0x0e, 0x74, 0x91, 0x67, 0x38,
0x91, 0x31, 0x92, 0x6f, 0x68, 0x92, 0x0d, 0x6d, 0x93, 0xf5, 0x46, 0x7b, 0x8a, 0x5d, 0x71, 0xd7,
0x1b, 0x1e, 0xc0, 0x4e, 0xc4, 0x86, 0x4b, 0xa8, 0x8c, 0x27, 0x9e, 0x2f, 0xb7, 0xcf, 0x36, 0x52,
0x73, 0xfd, 0xf7, 0x14, 0xbb, 0x16, 0xe5, 0x56, 0xd2, 0x87, 0x6a, 0x0e, 0xac, 0x09, 0xf0, 0xc9,
0x36, 0x71, 0x53, 0xac, 0xee, 0xe6, 0xa1, 0x28, 0xca, 0x4c, 0x3b, 0x28, 0x6c, 0x01, 0xcd, 0x2c,
0x6f, 0x01, 0xc5, 0xcc, 0x36, 0xef, 0xa1, 0x96, 0x42, 0x93, 0xa8, 0x45, 0x41, 0x3d, 0xdd, 0x8a,
0x9a, 0x66, 0xad, 0x62, 0x66, 0xd2, 0x2d, 0x82, 0x86, 0x71, 0xd8, 0xbd, 0x7e, 0x9f, 0x99, 0xea,
0x74, 0x66, 0xaa, 0x9f, 0x33, 0x53, 0x7d, 0x9d, 0x9b, 0xca, 0x74, 0x6e, 0x2a, 0x1f, 0x73, 0x53,
0x19, 0x9c, 0x7b, 0x01, 0xf7, 0x63, 0x97, 0x8c, 0x58, 0xd8, 0x5a, 0xbd, 0xb4, 0x7e, 0xcc, 0xff,
0x64, 0x6e, 0x49, 0x4c, 0x3a, 0x5f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x55, 0x3b, 0x57, 0xfa, 0x7f,
0x03, 0x00, 0x00,
}
func (m *BlockRequest) Marshal() (dAtA []byte, err error) {
@ -506,16 +503,18 @@ func (m *BlockResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
_ = i
var l int
_ = l
{
size, err := m.Block.MarshalToSizedBuffer(dAtA[:i])
if err != nil {
return 0, err
if m.Block != nil {
{
size, err := m.Block.MarshalToSizedBuffer(dAtA[:i])
if err != nil {
return 0, err
}
i -= size
i = encodeVarintMsgs(dAtA, i, uint64(size))
}
i -= size
i = encodeVarintMsgs(dAtA, i, uint64(size))
i--
dAtA[i] = 0xa
}
i--
dAtA[i] = 0xa
return len(dAtA) - i, nil
}
@ -763,8 +762,10 @@ func (m *BlockResponse) Size() (n int) {
}
var l int
_ = l
l = m.Block.Size()
n += 1 + l + sovMsgs(uint64(l))
if m.Block != nil {
l = m.Block.Size()
n += 1 + l + sovMsgs(uint64(l))
}
return n
}
@ -1079,6 +1080,9 @@ func (m *BlockResponse) Unmarshal(dAtA []byte) error {
if postIndex > l {
return io.ErrUnexpectedEOF
}
if m.Block == nil {
m.Block = &types.Block{}
}
if err := m.Block.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
return err
}


+ 1
- 2
proto/blockchain/msgs.proto View File

@ -3,7 +3,6 @@ package tendermint.proto.blockchain;
option go_package = "github.com/tendermint/tendermint/proto/blockchain";
import "third_party/proto/gogoproto/gogo.proto";
import "proto/types/block.proto";
// BlockRequest requests a block for a specific height
@ -18,7 +17,7 @@ message NoBlockResponse {
// BlockResponse returns block to the requested
message BlockResponse {
tendermint.proto.types.Block block = 1 [(gogoproto.nullable) = false];
tendermint.proto.types.Block block = 1;
}
// StatusRequest requests the status of a node (Height & Base)


+ 1
- 1
types/block.go View File

@ -839,6 +839,7 @@ func (commit *Commit) ValidateBasic() error {
if commit.Round < 0 {
return errors.New("negative Round")
}
if commit.Height >= 1 {
if commit.BlockID.IsZero() {
return errors.New("commit cannot be for nil block")
@ -853,7 +854,6 @@ func (commit *Commit) ValidateBasic() error {
}
}
}
return nil
}


+ 70
- 96
types/block_test.go View File

@ -602,44 +602,18 @@ func TestBlockIDValidateBasic(t *testing.T) {
}
}
func makeRandHeader() Header {
chainID := "test"
t := time.Now()
height := tmrand.Int63()
randBytes := tmrand.Bytes(tmhash.Size)
randAddress := tmrand.Bytes(crypto.AddressSize)
h := Header{
Version: version.Consensus{Block: 1, App: 1},
ChainID: chainID,
Height: height,
Time: t,
LastBlockID: BlockID{},
LastCommitHash: randBytes,
DataHash: randBytes,
ValidatorsHash: randBytes,
NextValidatorsHash: randBytes,
ConsensusHash: randBytes,
AppHash: randBytes,
LastResultsHash: randBytes,
EvidenceHash: randBytes,
ProposerAddress: randAddress,
}
return h
}
func TestBlockProtoBuf(t *testing.T) {
h := tmrand.Int63()
c1 := randCommit(time.Now())
b1 := MakeBlock(h, []Tx{Tx([]byte{1})}, &Commit{Signatures: []CommitSig{}}, []Evidence{})
b1.ProposerAddress = tmrand.Bytes(crypto.AddressSize)
b2 := MakeBlock(h, []Tx{Tx([]byte{1})}, c1, []Evidence{})
b2.ProposerAddress = tmrand.Bytes(crypto.AddressSize)
evi := NewMockEvidence(b2.Height, time.Now(), tmrand.Bytes(32))
b2.Evidence = EvidenceData{Evidence: EvidenceList{evi}}
b2.EvidenceHash = b2.Evidence.Hash()
b3 := MakeBlock(h, []Tx{}, c1, []Evidence{})
b3.ProposerAddress = tmrand.Bytes(crypto.AddressSize)
testCases := []struct {
@ -679,7 +653,6 @@ func TestDataProtoBuf(t *testing.T) {
_ = data.Hash()
data2 := &Data{Txs: Txs{}}
_ = data2.Hash()
testCases := []struct {
msg string
data1 *Data
@ -700,6 +673,74 @@ func TestDataProtoBuf(t *testing.T) {
}
}
func TestEvidenceDataProtoBuf(t *testing.T) {
val := NewMockPV()
blockID := makeBlockID(tmhash.Sum([]byte("blockhash")), math.MaxInt32, tmhash.Sum([]byte("partshash")))
blockID2 := makeBlockID(tmhash.Sum([]byte("blockhash2")), math.MaxInt32, tmhash.Sum([]byte("partshash")))
const chainID = "mychain"
v := makeVote(t, val, chainID, math.MaxInt32, math.MaxInt64, 1, 0x01, blockID, time.Now())
v2 := makeVote(t, val, chainID, math.MaxInt32, math.MaxInt64, 2, 0x01, blockID2, time.Now())
ev := NewDuplicateVoteEvidence(v2, v)
data := &EvidenceData{Evidence: EvidenceList{ev}}
_ = data.Hash()
testCases := []struct {
msg string
data1 *EvidenceData
expPass1 bool
expPass2 bool
}{
{"success", data, true, true},
{"empty evidenceData", &EvidenceData{Evidence: EvidenceList{}}, true, true},
{"fail nil Data", nil, false, false},
}
for _, tc := range testCases {
protoData, err := tc.data1.ToProto()
if tc.expPass1 {
require.NoError(t, err, tc.msg)
} else {
require.Error(t, err, tc.msg)
}
eviD := new(EvidenceData)
err = eviD.FromProto(protoData)
if tc.expPass2 {
require.NoError(t, err, tc.msg)
require.Equal(t, tc.data1, eviD, tc.msg)
} else {
require.Error(t, err, tc.msg)
}
}
}
func makeRandHeader() Header {
chainID := "test"
t := time.Now()
height := tmrand.Int63()
randBytes := tmrand.Bytes(tmhash.Size)
randAddress := tmrand.Bytes(crypto.AddressSize)
h := Header{
Version: version.Consensus{Block: 1, App: 1},
ChainID: chainID,
Height: height,
Time: t,
LastBlockID: BlockID{},
LastCommitHash: randBytes,
DataHash: randBytes,
ValidatorsHash: randBytes,
NextValidatorsHash: randBytes,
ConsensusHash: randBytes,
AppHash: randBytes,
LastResultsHash: randBytes,
EvidenceHash: randBytes,
ProposerAddress: randAddress,
}
return h
}
func TestHeaderProto(t *testing.T) {
h1 := makeRandHeader()
tc := []struct {
@ -779,70 +820,3 @@ func TestSignedHeaderProtoBuf(t *testing.T) {
}
}
}
func TestEvidenceDataProtoBuf(t *testing.T) {
val := NewMockPV()
blockID := makeBlockID(tmhash.Sum([]byte("blockhash")), math.MaxInt32, tmhash.Sum([]byte("partshash")))
blockID2 := makeBlockID(tmhash.Sum([]byte("blockhash2")), math.MaxInt32, tmhash.Sum([]byte("partshash")))
const chainID = "mychain"
v := makeVote(t, val, chainID, math.MaxInt32, math.MaxInt64, 1, 0x01, blockID, time.Now())
v2 := makeVote(t, val, chainID, math.MaxInt32, math.MaxInt64, 2, 0x01, blockID2, time.Now())
ev := NewDuplicateVoteEvidence(v2, v)
data := &EvidenceData{Evidence: EvidenceList{ev}}
_ = data.Hash()
testCases := []struct {
msg string
data1 *EvidenceData
expPass1 bool
expPass2 bool
}{
{"success", data, true, true},
{"empty evidenceData", &EvidenceData{Evidence: EvidenceList{}}, true, true},
{"fail nil Data", nil, false, false},
}
for _, tc := range testCases {
protoData, err := tc.data1.ToProto()
if tc.expPass1 {
require.NoError(t, err, tc.msg)
} else {
require.Error(t, err, tc.msg)
}
eviD := new(EvidenceData)
err = eviD.FromProto(protoData)
if tc.expPass2 {
require.NoError(t, err, tc.msg)
require.Equal(t, tc.data1, eviD, tc.msg)
} else {
require.Error(t, err, tc.msg)
}
}
}
func TestCommitProtoBuf(t *testing.T) {
commit := randCommit(time.Now())
testCases := []struct {
msg string
c1 *Commit
expPass bool
}{
{"success", commit, true},
// Empty value sets signatures to nil, signatures should not be nillable
{"empty commit", &Commit{Signatures: []CommitSig{}}, true},
{"fail Commit nil", nil, false},
}
for _, tc := range testCases {
tc := tc
protoCommit := tc.c1.ToProto()
c, err := CommitFromProto(protoCommit)
if tc.expPass {
require.NoError(t, err, tc.msg)
require.Equal(t, tc.c1, c, tc.msg)
} else {
require.Error(t, err, tc.msg)
}
}
}

Loading…
Cancel
Save