statesync: use Protobuf instead of Amino for p2p traffic (#4943)

## Description



Closes: #XXX
This commit is contained in:
Marko
2020-06-03 10:43:50 +02:00
committed by GitHub
parent 123beeadc4
commit 4e6a844d6f
7 changed files with 166 additions and 216 deletions

View File

@@ -38,6 +38,7 @@ Friendly reminder, we have a [bug bounty program](https://hackerone.com/tendermi
- [evidence] [\#4780](https://github.com/tendermint/tendermint/pull/4780) Cap evidence to an absolute number (@cmwaters) - [evidence] [\#4780](https://github.com/tendermint/tendermint/pull/4780) Cap evidence to an absolute number (@cmwaters)
Add `max_num` to consensus evidence parameters (default: 50 items). Add `max_num` to consensus evidence parameters (default: 50 items).
- [mempool] \#4940 Migrate mempool from amino binary encoding to Protobuf - [mempool] \#4940 Migrate mempool from amino binary encoding to Protobuf
- [statesync] \#4943 Migrate statesync reactor from amino binary encoding to Protobuf
### FEATURES: ### FEATURES:

View File

@@ -4,9 +4,9 @@ import (
"errors" "errors"
"fmt" "fmt"
amino "github.com/tendermint/go-amino" "github.com/gogo/protobuf/proto"
"github.com/tendermint/tendermint/types" ssproto "github.com/tendermint/tendermint/proto/statesync"
) )
const ( const (
@@ -14,116 +14,84 @@ const (
snapshotMsgSize = int(4e6) snapshotMsgSize = int(4e6)
// chunkMsgSize is the maximum size of a chunkResponseMessage // chunkMsgSize is the maximum size of a chunkResponseMessage
chunkMsgSize = int(16e6) chunkMsgSize = int(16e6)
// maxMsgSize is the maximum size of any message
maxMsgSize = chunkMsgSize
) )
var cdc = amino.NewCodec() // mustEncodeMsg encodes a Protobuf message, panicing on error.
func mustEncodeMsg(pb proto.Message) []byte {
func init() { msg := ssproto.Message{}
cdc.RegisterInterface((*Message)(nil), nil) switch pb := pb.(type) {
cdc.RegisterConcrete(&snapshotsRequestMessage{}, "tendermint/SnapshotsRequestMessage", nil) case *ssproto.ChunkRequest:
cdc.RegisterConcrete(&snapshotsResponseMessage{}, "tendermint/SnapshotsResponseMessage", nil) msg.Sum = &ssproto.Message_ChunkRequest{ChunkRequest: pb}
cdc.RegisterConcrete(&chunkRequestMessage{}, "tendermint/ChunkRequestMessage", nil) case *ssproto.ChunkResponse:
cdc.RegisterConcrete(&chunkResponseMessage{}, "tendermint/ChunkResponseMessage", nil) msg.Sum = &ssproto.Message_ChunkResponse{ChunkResponse: pb}
types.RegisterBlockAmino(cdc) case *ssproto.SnapshotsRequest:
msg.Sum = &ssproto.Message_SnapshotsRequest{SnapshotsRequest: pb}
case *ssproto.SnapshotsResponse:
msg.Sum = &ssproto.Message_SnapshotsResponse{SnapshotsResponse: pb}
default:
panic(fmt.Errorf("unknown message type %T", pb))
}
bz, err := proto.Marshal(&msg)
if err != nil {
panic(fmt.Errorf("unable to marshal %T: %w", pb, err))
}
return bz
} }
// decodeMsg decodes a message. // decodeMsg decodes a Protobuf message.
func decodeMsg(bz []byte) (Message, error) { func decodeMsg(bz []byte) (proto.Message, error) {
if len(bz) > maxMsgSize { pb := &ssproto.Message{}
return nil, fmt.Errorf("msg exceeds max size (%d > %d)", len(bz), maxMsgSize) err := proto.Unmarshal(bz, pb)
}
var msg Message
err := cdc.UnmarshalBinaryBare(bz, &msg)
if err != nil { if err != nil {
return nil, err return nil, err
} }
return msg, nil switch msg := pb.Sum.(type) {
case *ssproto.Message_ChunkRequest:
return msg.ChunkRequest, nil
case *ssproto.Message_ChunkResponse:
return msg.ChunkResponse, nil
case *ssproto.Message_SnapshotsRequest:
return msg.SnapshotsRequest, nil
case *ssproto.Message_SnapshotsResponse:
return msg.SnapshotsResponse, nil
default:
return nil, fmt.Errorf("unknown message type %T", msg)
}
} }
// Message is a message sent and received by the reactor. // validateMsg validates a message.
type Message interface { func validateMsg(pb proto.Message) error {
ValidateBasic() error if pb == nil {
} return errors.New("message cannot be nil")
}
// snapshotsRequestMessage requests recent snapshots from a peer. switch msg := pb.(type) {
type snapshotsRequestMessage struct{} case *ssproto.ChunkRequest:
if msg.Height == 0 {
// ValidateBasic implements Message. return errors.New("height cannot be 0")
func (m *snapshotsRequestMessage) ValidateBasic() error { }
if m == nil { case *ssproto.ChunkResponse:
return errors.New("nil message") if msg.Height == 0 {
} return errors.New("height cannot be 0")
return nil }
} if msg.Missing && len(msg.Chunk) > 0 {
return errors.New("missing chunk cannot have contents")
// SnapshotResponseMessage contains information about a single snapshot. }
type snapshotsResponseMessage struct { if !msg.Missing && msg.Chunk == nil {
Height uint64 return errors.New("chunk cannot be nil")
Format uint32 }
Chunks uint32 case *ssproto.SnapshotsRequest:
Hash []byte case *ssproto.SnapshotsResponse:
Metadata []byte if msg.Height == 0 {
} return errors.New("height cannot be 0")
}
// ValidateBasic implements Message. if len(msg.Hash) == 0 {
func (m *snapshotsResponseMessage) ValidateBasic() error { return errors.New("snapshot has no hash")
if m == nil { }
return errors.New("nil message") if msg.Chunks == 0 {
} return errors.New("snapshot has no chunks")
if m.Height == 0 { }
return errors.New("height cannot be 0") default:
} return fmt.Errorf("unknown message type %T", msg)
if len(m.Hash) == 0 {
return errors.New("snapshot has no hash")
}
if m.Chunks == 0 {
return errors.New("snapshot has no chunks")
}
return nil
}
// chunkRequestMessage requests a single chunk from a peer.
type chunkRequestMessage struct {
Height uint64
Format uint32
Index uint32
}
// ValidateBasic implements Message.
func (m *chunkRequestMessage) ValidateBasic() error {
if m == nil {
return errors.New("nil message")
}
if m.Height == 0 {
return errors.New("height cannot be 0")
}
return nil
}
// chunkResponseMessage contains a single chunk from a peer.
type chunkResponseMessage struct {
Height uint64
Format uint32
Index uint32
Chunk []byte
Missing bool
}
// ValidateBasic implements Message.
func (m *chunkResponseMessage) ValidateBasic() error {
if m == nil {
return errors.New("nil message")
}
if m.Height == 0 {
return errors.New("height cannot be 0")
}
if m.Missing && len(m.Chunk) > 0 {
return errors.New("missing chunk cannot have contents")
}
if !m.Missing && m.Chunk == nil {
return errors.New("chunk cannot be nil")
} }
return nil return nil
} }

View File

@@ -3,99 +3,76 @@ package statesync
import ( import (
"testing" "testing"
"github.com/gogo/protobuf/proto"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
ssproto "github.com/tendermint/tendermint/proto/statesync"
tmproto "github.com/tendermint/tendermint/proto/types"
) )
func TestSnapshotsRequestMessage_ValidateBasic(t *testing.T) { func TestValidateMsg(t *testing.T) {
testcases := map[string]struct { testcases := map[string]struct {
msg *snapshotsRequestMessage msg proto.Message
valid bool valid bool
}{ }{
"nil": {nil, false}, "nil": {nil, false},
"valid": {&snapshotsRequestMessage{}, true}, "unrelated": {&tmproto.Block{}, false},
}
for name, tc := range testcases {
tc := tc
t.Run(name, func(t *testing.T) {
err := tc.msg.ValidateBasic()
if tc.valid {
require.NoError(t, err)
} else {
require.Error(t, err)
}
})
}
}
func TestSnapshotsResponseMessage_ValidateBasic(t *testing.T) { "ChunkRequest valid": {&ssproto.ChunkRequest{Height: 1, Format: 1, Index: 1}, true},
testcases := map[string]struct { "ChunkRequest 0 height": {&ssproto.ChunkRequest{Height: 0, Format: 1, Index: 1}, false},
msg *snapshotsResponseMessage "ChunkRequest 0 format": {&ssproto.ChunkRequest{Height: 1, Format: 0, Index: 1}, true},
valid bool "ChunkRequest 0 chunk": {&ssproto.ChunkRequest{Height: 1, Format: 1, Index: 0}, true},
}{
"nil": {nil, false},
"valid": {&snapshotsResponseMessage{Height: 1, Format: 1, Chunks: 2, Hash: []byte{1}}, true},
"0 height": {&snapshotsResponseMessage{Height: 0, Format: 1, Chunks: 2, Hash: []byte{1}}, false},
"0 format": {&snapshotsResponseMessage{Height: 1, Format: 0, Chunks: 2, Hash: []byte{1}}, true},
"0 chunks": {&snapshotsResponseMessage{Height: 1, Format: 1, Hash: []byte{1}}, false},
"no hash": {&snapshotsResponseMessage{Height: 1, Format: 1, Chunks: 2, Hash: []byte{}}, false},
}
for name, tc := range testcases {
tc := tc
t.Run(name, func(t *testing.T) {
err := tc.msg.ValidateBasic()
if tc.valid {
require.NoError(t, err)
} else {
require.Error(t, err)
}
})
}
}
func TestChunkRequestMessage_ValidateBasic(t *testing.T) { "ChunkResponse valid": {
testcases := map[string]struct { &ssproto.ChunkResponse{Height: 1, Format: 1, Index: 1, Chunk: []byte{1}},
msg *chunkRequestMessage true},
valid bool "ChunkResponse 0 height": {
}{ &ssproto.ChunkResponse{Height: 0, Format: 1, Index: 1, Chunk: []byte{1}},
"nil": {nil, false}, false},
"valid": {&chunkRequestMessage{Height: 1, Format: 1, Index: 1}, true}, "ChunkResponse 0 format": {
"0 height": {&chunkRequestMessage{Height: 0, Format: 1, Index: 1}, false}, &ssproto.ChunkResponse{Height: 1, Format: 0, Index: 1, Chunk: []byte{1}},
"0 format": {&chunkRequestMessage{Height: 1, Format: 0, Index: 1}, true}, true},
"0 chunk": {&chunkRequestMessage{Height: 1, Format: 1, Index: 0}, true}, "ChunkResponse 0 chunk": {
} &ssproto.ChunkResponse{Height: 1, Format: 1, Index: 0, Chunk: []byte{1}},
for name, tc := range testcases { true},
tc := tc "ChunkResponse empty body": {
t.Run(name, func(t *testing.T) { &ssproto.ChunkResponse{Height: 1, Format: 1, Index: 1, Chunk: []byte{}},
err := tc.msg.ValidateBasic() true},
if tc.valid { "ChunkResponse nil body": {
require.NoError(t, err) &ssproto.ChunkResponse{Height: 1, Format: 1, Index: 1, Chunk: nil},
} else { false},
require.Error(t, err) "ChunkResponse missing": {
} &ssproto.ChunkResponse{Height: 1, Format: 1, Index: 1, Missing: true},
}) true},
} "ChunkResponse missing with empty": {
} &ssproto.ChunkResponse{Height: 1, Format: 1, Index: 1, Missing: true, Chunk: []byte{}},
true},
"ChunkResponse missing with body": {
&ssproto.ChunkResponse{Height: 1, Format: 1, Index: 1, Missing: true, Chunk: []byte{1}},
false},
func TestChunkResponseMessage_ValidateBasic(t *testing.T) { "SnapshotsRequest valid": {&ssproto.SnapshotsRequest{}, true},
testcases := map[string]struct {
msg *chunkResponseMessage "SnapshotsResponse valid": {
valid bool &ssproto.SnapshotsResponse{Height: 1, Format: 1, Chunks: 2, Hash: []byte{1}},
}{ true},
"nil message": {nil, false}, "SnapshotsResponse 0 height": {
"valid": {&chunkResponseMessage{Height: 1, Format: 1, Index: 1, Chunk: []byte{1}}, true}, &ssproto.SnapshotsResponse{Height: 0, Format: 1, Chunks: 2, Hash: []byte{1}},
"0 height": {&chunkResponseMessage{Height: 0, Format: 1, Index: 1, Chunk: []byte{1}}, false}, false},
"0 format": {&chunkResponseMessage{Height: 1, Format: 0, Index: 1, Chunk: []byte{1}}, true}, "SnapshotsResponse 0 format": {
"0 chunk": {&chunkResponseMessage{Height: 1, Format: 1, Index: 0, Chunk: []byte{1}}, true}, &ssproto.SnapshotsResponse{Height: 1, Format: 0, Chunks: 2, Hash: []byte{1}},
"empty body": {&chunkResponseMessage{Height: 1, Format: 1, Index: 1, Chunk: []byte{}}, true}, true},
"nil body": {&chunkResponseMessage{Height: 1, Format: 1, Index: 1, Chunk: nil}, false}, "SnapshotsResponse 0 chunks": {
"missing": {&chunkResponseMessage{Height: 1, Format: 1, Index: 1, Missing: true}, true}, &ssproto.SnapshotsResponse{Height: 1, Format: 1, Hash: []byte{1}},
"missing with empty": {&chunkResponseMessage{Height: 1, Format: 1, Index: 1, Missing: true, Chunk: []byte{}}, true}, false},
"missing with body": {&chunkResponseMessage{Height: 1, Format: 1, Index: 1, Missing: true, Chunk: []byte{1}}, false}, "SnapshotsResponse no hash": {
&ssproto.SnapshotsResponse{Height: 1, Format: 1, Chunks: 2, Hash: []byte{}},
false},
} }
for name, tc := range testcases { for name, tc := range testcases {
tc := tc tc := tc
t.Run(name, func(t *testing.T) { t.Run(name, func(t *testing.T) {
err := tc.msg.ValidateBasic() err := validateMsg(tc.msg)
if tc.valid { if tc.valid {
require.NoError(t, err) require.NoError(t, err)
} else { } else {

View File

@@ -7,6 +7,7 @@ import (
abci "github.com/tendermint/tendermint/abci/types" abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/p2p" "github.com/tendermint/tendermint/p2p"
ssproto "github.com/tendermint/tendermint/proto/statesync"
"github.com/tendermint/tendermint/proxy" "github.com/tendermint/tendermint/proxy"
sm "github.com/tendermint/tendermint/state" sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/types" "github.com/tendermint/tendermint/types"
@@ -99,7 +100,7 @@ func (r *Reactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) {
r.Switch.StopPeerForError(src, err) r.Switch.StopPeerForError(src, err)
return return
} }
err = msg.ValidateBasic() err = validateMsg(msg)
if err != nil { if err != nil {
r.Logger.Error("Invalid message", "peer", src, "msg", msg, "err", err) r.Logger.Error("Invalid message", "peer", src, "msg", msg, "err", err)
r.Switch.StopPeerForError(src, err) r.Switch.StopPeerForError(src, err)
@@ -109,7 +110,7 @@ func (r *Reactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) {
switch chID { switch chID {
case SnapshotChannel: case SnapshotChannel:
switch msg := msg.(type) { switch msg := msg.(type) {
case *snapshotsRequestMessage: case *ssproto.SnapshotsRequest:
snapshots, err := r.recentSnapshots(recentSnapshots) snapshots, err := r.recentSnapshots(recentSnapshots)
if err != nil { if err != nil {
r.Logger.Error("Failed to fetch snapshots", "err", err) r.Logger.Error("Failed to fetch snapshots", "err", err)
@@ -118,7 +119,7 @@ func (r *Reactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) {
for _, snapshot := range snapshots { for _, snapshot := range snapshots {
r.Logger.Debug("Advertising snapshot", "height", snapshot.Height, r.Logger.Debug("Advertising snapshot", "height", snapshot.Height,
"format", snapshot.Format, "peer", src.ID()) "format", snapshot.Format, "peer", src.ID())
src.Send(chID, cdc.MustMarshalBinaryBare(&snapshotsResponseMessage{ src.Send(chID, mustEncodeMsg(&ssproto.SnapshotsResponse{
Height: snapshot.Height, Height: snapshot.Height,
Format: snapshot.Format, Format: snapshot.Format,
Chunks: snapshot.Chunks, Chunks: snapshot.Chunks,
@@ -127,7 +128,7 @@ func (r *Reactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) {
})) }))
} }
case *snapshotsResponseMessage: case *ssproto.SnapshotsResponse:
r.mtx.RLock() r.mtx.RLock()
defer r.mtx.RUnlock() defer r.mtx.RUnlock()
if r.syncer == nil { if r.syncer == nil {
@@ -154,7 +155,7 @@ func (r *Reactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) {
case ChunkChannel: case ChunkChannel:
switch msg := msg.(type) { switch msg := msg.(type) {
case *chunkRequestMessage: case *ssproto.ChunkRequest:
r.Logger.Debug("Received chunk request", "height", msg.Height, "format", msg.Format, r.Logger.Debug("Received chunk request", "height", msg.Height, "format", msg.Format,
"chunk", msg.Index, "peer", src.ID()) "chunk", msg.Index, "peer", src.ID())
resp, err := r.conn.LoadSnapshotChunkSync(abci.RequestLoadSnapshotChunk{ resp, err := r.conn.LoadSnapshotChunkSync(abci.RequestLoadSnapshotChunk{
@@ -169,7 +170,7 @@ func (r *Reactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) {
} }
r.Logger.Debug("Sending chunk", "height", msg.Height, "format", msg.Format, r.Logger.Debug("Sending chunk", "height", msg.Height, "format", msg.Format,
"chunk", msg.Index, "peer", src.ID()) "chunk", msg.Index, "peer", src.ID())
src.Send(ChunkChannel, cdc.MustMarshalBinaryBare(&chunkResponseMessage{ src.Send(ChunkChannel, mustEncodeMsg(&ssproto.ChunkResponse{
Height: msg.Height, Height: msg.Height,
Format: msg.Format, Format: msg.Format,
Index: msg.Index, Index: msg.Index,
@@ -177,7 +178,7 @@ func (r *Reactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) {
Missing: resp.Chunk == nil, Missing: resp.Chunk == nil,
})) }))
case *chunkResponseMessage: case *ssproto.ChunkResponse:
r.mtx.RLock() r.mtx.RLock()
defer r.mtx.RUnlock() defer r.mtx.RUnlock()
if r.syncer == nil { if r.syncer == nil {

View File

@@ -11,27 +11,28 @@ import (
abci "github.com/tendermint/tendermint/abci/types" abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/p2p" "github.com/tendermint/tendermint/p2p"
p2pmocks "github.com/tendermint/tendermint/p2p/mocks" p2pmocks "github.com/tendermint/tendermint/p2p/mocks"
ssproto "github.com/tendermint/tendermint/proto/statesync"
proxymocks "github.com/tendermint/tendermint/proxy/mocks" proxymocks "github.com/tendermint/tendermint/proxy/mocks"
) )
func TestReactor_Receive_ChunkRequestMessage(t *testing.T) { func TestReactor_Receive_ChunkRequest(t *testing.T) {
testcases := map[string]struct { testcases := map[string]struct {
request *chunkRequestMessage request *ssproto.ChunkRequest
chunk []byte chunk []byte
expectResponse *chunkResponseMessage expectResponse *ssproto.ChunkResponse
}{ }{
"chunk is returned": { "chunk is returned": {
&chunkRequestMessage{Height: 1, Format: 1, Index: 1}, &ssproto.ChunkRequest{Height: 1, Format: 1, Index: 1},
[]byte{1, 2, 3}, []byte{1, 2, 3},
&chunkResponseMessage{Height: 1, Format: 1, Index: 1, Chunk: []byte{1, 2, 3}}}, &ssproto.ChunkResponse{Height: 1, Format: 1, Index: 1, Chunk: []byte{1, 2, 3}}},
"empty chunk is returned, as nil": { "empty chunk is returned, as nil": {
&chunkRequestMessage{Height: 1, Format: 1, Index: 1}, &ssproto.ChunkRequest{Height: 1, Format: 1, Index: 1},
[]byte{}, []byte{},
&chunkResponseMessage{Height: 1, Format: 1, Index: 1, Chunk: nil}}, &ssproto.ChunkResponse{Height: 1, Format: 1, Index: 1, Chunk: nil}},
"nil (missing) chunk is returned as missing": { "nil (missing) chunk is returned as missing": {
&chunkRequestMessage{Height: 1, Format: 1, Index: 1}, &ssproto.ChunkRequest{Height: 1, Format: 1, Index: 1},
nil, nil,
&chunkResponseMessage{Height: 1, Format: 1, Index: 1, Missing: true}, &ssproto.ChunkResponse{Height: 1, Format: 1, Index: 1, Missing: true},
}, },
} }
@@ -49,22 +50,22 @@ func TestReactor_Receive_ChunkRequestMessage(t *testing.T) {
// Mock peer to store response, if found // Mock peer to store response, if found
peer := &p2pmocks.Peer{} peer := &p2pmocks.Peer{}
peer.On("ID").Return(p2p.ID("id")) peer.On("ID").Return(p2p.ID("id"))
var response *chunkResponseMessage var response *ssproto.ChunkResponse
if tc.expectResponse != nil { if tc.expectResponse != nil {
peer.On("Send", ChunkChannel, mock.Anything).Run(func(args mock.Arguments) { peer.On("Send", ChunkChannel, mock.Anything).Run(func(args mock.Arguments) {
msg, err := decodeMsg(args[1].([]byte)) msg, err := decodeMsg(args[1].([]byte))
require.NoError(t, err) require.NoError(t, err)
response = msg.(*chunkResponseMessage) response = msg.(*ssproto.ChunkResponse)
}).Return(true) }).Return(true)
} }
// Start a reactor and send a chunkRequestMessage, then wait for and check response // Start a reactor and send a ssproto.ChunkRequest, then wait for and check response
r := NewReactor(conn, nil, "") r := NewReactor(conn, nil, "")
err := r.Start() err := r.Start()
require.NoError(t, err) require.NoError(t, err)
defer r.Stop() defer r.Stop()
r.Receive(ChunkChannel, peer, cdc.MustMarshalBinaryBare(tc.request)) r.Receive(ChunkChannel, peer, mustEncodeMsg(tc.request))
time.Sleep(100 * time.Millisecond) time.Sleep(100 * time.Millisecond)
assert.Equal(t, tc.expectResponse, response) assert.Equal(t, tc.expectResponse, response)
@@ -74,12 +75,12 @@ func TestReactor_Receive_ChunkRequestMessage(t *testing.T) {
} }
} }
func TestReactor_Receive_SnapshotRequestMessage(t *testing.T) { func TestReactor_Receive_SnapshotsRequest(t *testing.T) {
testcases := map[string]struct { testcases := map[string]struct {
snapshots []*abci.Snapshot snapshots []*abci.Snapshot
expectResponses []*snapshotsResponseMessage expectResponses []*ssproto.SnapshotsResponse
}{ }{
"no snapshots": {nil, []*snapshotsResponseMessage{}}, "no snapshots": {nil, []*ssproto.SnapshotsResponse{}},
">10 unordered snapshots": { ">10 unordered snapshots": {
[]*abci.Snapshot{ []*abci.Snapshot{
{Height: 1, Format: 2, Chunks: 7, Hash: []byte{1, 2}, Metadata: []byte{1}}, {Height: 1, Format: 2, Chunks: 7, Hash: []byte{1, 2}, Metadata: []byte{1}},
@@ -95,7 +96,7 @@ func TestReactor_Receive_SnapshotRequestMessage(t *testing.T) {
{Height: 2, Format: 3, Chunks: 7, Hash: []byte{2, 3}, Metadata: []byte{11}}, {Height: 2, Format: 3, Chunks: 7, Hash: []byte{2, 3}, Metadata: []byte{11}},
{Height: 3, Format: 3, Chunks: 7, Hash: []byte{3, 3}, Metadata: []byte{12}}, {Height: 3, Format: 3, Chunks: 7, Hash: []byte{3, 3}, Metadata: []byte{12}},
}, },
[]*snapshotsResponseMessage{ []*ssproto.SnapshotsResponse{
{Height: 3, Format: 4, Chunks: 7, Hash: []byte{3, 4}, Metadata: []byte{9}}, {Height: 3, Format: 4, Chunks: 7, Hash: []byte{3, 4}, Metadata: []byte{9}},
{Height: 3, Format: 3, Chunks: 7, Hash: []byte{3, 3}, Metadata: []byte{12}}, {Height: 3, Format: 3, Chunks: 7, Hash: []byte{3, 3}, Metadata: []byte{12}},
{Height: 3, Format: 2, Chunks: 7, Hash: []byte{3, 2}, Metadata: []byte{3}}, {Height: 3, Format: 2, Chunks: 7, Hash: []byte{3, 2}, Metadata: []byte{3}},
@@ -120,14 +121,14 @@ func TestReactor_Receive_SnapshotRequestMessage(t *testing.T) {
}, nil) }, nil)
// Mock peer to catch responses and store them in a slice // Mock peer to catch responses and store them in a slice
responses := []*snapshotsResponseMessage{} responses := []*ssproto.SnapshotsResponse{}
peer := &p2pmocks.Peer{} peer := &p2pmocks.Peer{}
if len(tc.expectResponses) > 0 { if len(tc.expectResponses) > 0 {
peer.On("ID").Return(p2p.ID("id")) peer.On("ID").Return(p2p.ID("id"))
peer.On("Send", SnapshotChannel, mock.Anything).Run(func(args mock.Arguments) { peer.On("Send", SnapshotChannel, mock.Anything).Run(func(args mock.Arguments) {
msg, err := decodeMsg(args[1].([]byte)) msg, err := decodeMsg(args[1].([]byte))
require.NoError(t, err) require.NoError(t, err)
responses = append(responses, msg.(*snapshotsResponseMessage)) responses = append(responses, msg.(*ssproto.SnapshotsResponse))
}).Return(true) }).Return(true)
} }
@@ -137,7 +138,7 @@ func TestReactor_Receive_SnapshotRequestMessage(t *testing.T) {
require.NoError(t, err) require.NoError(t, err)
defer r.Stop() defer r.Stop()
r.Receive(SnapshotChannel, peer, cdc.MustMarshalBinaryBare(&snapshotsRequestMessage{})) r.Receive(SnapshotChannel, peer, mustEncodeMsg(&ssproto.SnapshotsRequest{}))
time.Sleep(100 * time.Millisecond) time.Sleep(100 * time.Millisecond)
assert.Equal(t, tc.expectResponses, responses) assert.Equal(t, tc.expectResponses, responses)

View File

@@ -11,6 +11,7 @@ import (
abci "github.com/tendermint/tendermint/abci/types" abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/libs/log" "github.com/tendermint/tendermint/libs/log"
"github.com/tendermint/tendermint/p2p" "github.com/tendermint/tendermint/p2p"
ssproto "github.com/tendermint/tendermint/proto/statesync"
"github.com/tendermint/tendermint/proxy" "github.com/tendermint/tendermint/proxy"
sm "github.com/tendermint/tendermint/state" sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/types" "github.com/tendermint/tendermint/types"
@@ -115,7 +116,7 @@ func (s *syncer) AddSnapshot(peer p2p.Peer, snapshot *snapshot) (bool, error) {
// to discover snapshots, later we may want to do retries and stuff. // to discover snapshots, later we may want to do retries and stuff.
func (s *syncer) AddPeer(peer p2p.Peer) { func (s *syncer) AddPeer(peer p2p.Peer) {
s.logger.Debug("Requesting snapshots from peer", "peer", peer.ID()) s.logger.Debug("Requesting snapshots from peer", "peer", peer.ID())
peer.Send(SnapshotChannel, cdc.MustMarshalBinaryBare(&snapshotsRequestMessage{})) peer.Send(SnapshotChannel, mustEncodeMsg(&ssproto.SnapshotsRequest{}))
} }
// RemovePeer removes a peer from the pool. // RemovePeer removes a peer from the pool.
@@ -411,7 +412,7 @@ func (s *syncer) requestChunk(snapshot *snapshot, chunk uint32) {
} }
s.logger.Debug("Requesting snapshot chunk", "height", snapshot.Height, s.logger.Debug("Requesting snapshot chunk", "height", snapshot.Height,
"format", snapshot.Format, "chunk", chunk, "peer", peer.ID()) "format", snapshot.Format, "chunk", chunk, "peer", peer.ID())
peer.Send(ChunkChannel, cdc.MustMarshalBinaryBare(&chunkRequestMessage{ peer.Send(ChunkChannel, mustEncodeMsg(&ssproto.ChunkRequest{
Height: snapshot.Height, Height: snapshot.Height,
Format: snapshot.Format, Format: snapshot.Format,
Index: chunk, Index: chunk,

View File

@@ -14,6 +14,7 @@ import (
"github.com/tendermint/tendermint/libs/log" "github.com/tendermint/tendermint/libs/log"
"github.com/tendermint/tendermint/p2p" "github.com/tendermint/tendermint/p2p"
p2pmocks "github.com/tendermint/tendermint/p2p/mocks" p2pmocks "github.com/tendermint/tendermint/p2p/mocks"
ssproto "github.com/tendermint/tendermint/proto/statesync"
"github.com/tendermint/tendermint/proxy" "github.com/tendermint/tendermint/proxy"
proxymocks "github.com/tendermint/tendermint/proxy/mocks" proxymocks "github.com/tendermint/tendermint/proxy/mocks"
sm "github.com/tendermint/tendermint/state" sm "github.com/tendermint/tendermint/state"
@@ -90,13 +91,13 @@ func TestSyncer_SyncAny(t *testing.T) {
// Adding a couple of peers should trigger snapshot discovery messages // Adding a couple of peers should trigger snapshot discovery messages
peerA := &p2pmocks.Peer{} peerA := &p2pmocks.Peer{}
peerA.On("ID").Return(p2p.ID("a")) peerA.On("ID").Return(p2p.ID("a"))
peerA.On("Send", SnapshotChannel, cdc.MustMarshalBinaryBare(&snapshotsRequestMessage{})).Return(true) peerA.On("Send", SnapshotChannel, mustEncodeMsg(&ssproto.SnapshotsRequest{})).Return(true)
syncer.AddPeer(peerA) syncer.AddPeer(peerA)
peerA.AssertExpectations(t) peerA.AssertExpectations(t)
peerB := &p2pmocks.Peer{} peerB := &p2pmocks.Peer{}
peerB.On("ID").Return(p2p.ID("b")) peerB.On("ID").Return(p2p.ID("b"))
peerB.On("Send", SnapshotChannel, cdc.MustMarshalBinaryBare(&snapshotsRequestMessage{})).Return(true) peerB.On("Send", SnapshotChannel, mustEncodeMsg(&ssproto.SnapshotsRequest{})).Return(true)
syncer.AddPeer(peerB) syncer.AddPeer(peerB)
peerB.AssertExpectations(t) peerB.AssertExpectations(t)
@@ -139,9 +140,9 @@ func TestSyncer_SyncAny(t *testing.T) {
chunkRequests := make(map[uint32]int) chunkRequests := make(map[uint32]int)
chunkRequestsMtx := sync.Mutex{} chunkRequestsMtx := sync.Mutex{}
onChunkRequest := func(args mock.Arguments) { onChunkRequest := func(args mock.Arguments) {
msg := &chunkRequestMessage{} pb, err := decodeMsg(args[1].([]byte))
err := cdc.UnmarshalBinaryBare(args[1].([]byte), &msg)
require.NoError(t, err) require.NoError(t, err)
msg := pb.(*ssproto.ChunkRequest)
require.EqualValues(t, 1, msg.Height) require.EqualValues(t, 1, msg.Height)
require.EqualValues(t, 1, msg.Format) require.EqualValues(t, 1, msg.Format)
require.LessOrEqual(t, msg.Index, uint32(len(chunks))) require.LessOrEqual(t, msg.Index, uint32(len(chunks)))