From d54de61bf655c8a3e9312c4c42e3ce73cf1f42bf Mon Sep 17 00:00:00 2001 From: Marko Date: Wed, 10 Jun 2020 14:08:47 +0200 Subject: [PATCH] consensus: proto migration (#4984) ## Description migrate consensus to protobuf Closes: #XXX --- consensus/byzantine_test.go | 8 +- consensus/codec.go | 15 -- consensus/msgs.go | 377 ++++++++++++++++++++++++++++ consensus/msgs_test.go | 312 +++++++++++++++++++++++ consensus/reactor.go | 56 ++--- consensus/reactor_test.go | 9 +- consensus/replay_stubs.go | 2 +- consensus/replay_test.go | 9 +- consensus/state.go | 4 +- consensus/types/codec.go | 13 - consensus/types/peer_round_state.go | 28 --- consensus/types/round_state.go | 28 --- consensus/types/round_state_test.go | 1 + consensus/wal.go | 56 +++-- consensus/wal_test.go | 7 +- crypto/merkle/simple_proof_test.go | 35 +++ crypto/merkle/simple_tree_test.go | 2 +- libs/math/safemath.go | 12 + proto/consensus/msgs.pb.go | 114 ++++----- proto/consensus/msgs.proto | 2 +- 20 files changed, 873 insertions(+), 217 deletions(-) delete mode 100644 consensus/codec.go create mode 100644 consensus/msgs.go create mode 100644 consensus/msgs_test.go delete mode 100644 consensus/types/codec.go diff --git a/consensus/byzantine_test.go b/consensus/byzantine_test.go index 4b52f99fe..a9adf0baa 100644 --- a/consensus/byzantine_test.go +++ b/consensus/byzantine_test.go @@ -219,7 +219,7 @@ func sendProposalAndParts( ) { // proposal msg := &ProposalMessage{Proposal: proposal} - peer.Send(DataChannel, cdc.MustMarshalBinaryBare(msg)) + peer.Send(DataChannel, MustEncode(msg)) // parts for i := 0; i < int(parts.Total()); i++ { @@ -229,7 +229,7 @@ func sendProposalAndParts( Round: round, // This tells peer that this part applies to us. Part: part, } - peer.Send(DataChannel, cdc.MustMarshalBinaryBare(msg)) + peer.Send(DataChannel, MustEncode(msg)) } // votes @@ -238,8 +238,8 @@ func sendProposalAndParts( precommit, _ := cs.signVote(tmproto.PrecommitType, blockHash, parts.Header()) cs.mtx.Unlock() - peer.Send(VoteChannel, cdc.MustMarshalBinaryBare(&VoteMessage{prevote})) - peer.Send(VoteChannel, cdc.MustMarshalBinaryBare(&VoteMessage{precommit})) + peer.Send(VoteChannel, MustEncode(&VoteMessage{prevote})) + peer.Send(VoteChannel, MustEncode(&VoteMessage{precommit})) } //---------------------------------------- diff --git a/consensus/codec.go b/consensus/codec.go deleted file mode 100644 index ae7dbaab2..000000000 --- a/consensus/codec.go +++ /dev/null @@ -1,15 +0,0 @@ -package consensus - -import ( - amino "github.com/tendermint/go-amino" - - "github.com/tendermint/tendermint/types" -) - -var cdc = amino.NewCodec() - -func init() { - RegisterMessages(cdc) - RegisterWALMessages(cdc) - types.RegisterBlockAmino(cdc) -} diff --git a/consensus/msgs.go b/consensus/msgs.go new file mode 100644 index 000000000..c218da6c8 --- /dev/null +++ b/consensus/msgs.go @@ -0,0 +1,377 @@ +package consensus + +import ( + "errors" + "fmt" + + "github.com/gogo/protobuf/proto" + + cstypes "github.com/tendermint/tendermint/consensus/types" + "github.com/tendermint/tendermint/libs/bits" + tmmath "github.com/tendermint/tendermint/libs/math" + "github.com/tendermint/tendermint/p2p" + tmcons "github.com/tendermint/tendermint/proto/consensus" + tmproto "github.com/tendermint/tendermint/proto/types" + "github.com/tendermint/tendermint/types" +) + +// MsgToProto takes a consensus message type and returns the proto defined consensus message +func MsgToProto(msg Message) (*tmcons.Message, error) { + if msg == nil { + return nil, errors.New("consensus: message is nil") + } + var pb tmcons.Message + + switch msg := msg.(type) { + case *NewRoundStepMessage: + pb = tmcons.Message{ + Sum: &tmcons.Message_NewRoundStep{ + NewRoundStep: &tmcons.NewRoundStep{ + Height: msg.Height, + Round: msg.Round, + Step: uint32(msg.Step), + SecondsSinceStartTime: msg.SecondsSinceStartTime, + LastCommitRound: msg.LastCommitRound, + }, + }, + } + case *NewValidBlockMessage: + pbPartsHeader := msg.BlockPartsHeader.ToProto() + pbBits := msg.BlockParts.ToProto() + pb = tmcons.Message{ + Sum: &tmcons.Message_NewValidBlock{ + NewValidBlock: &tmcons.NewValidBlock{ + Height: msg.Height, + Round: msg.Round, + BlockPartsHeader: pbPartsHeader, + BlockParts: pbBits, + IsCommit: msg.IsCommit, + }, + }, + } + case *ProposalMessage: + pbP := msg.Proposal.ToProto() + pb = tmcons.Message{ + Sum: &tmcons.Message_Proposal{ + Proposal: &tmcons.Proposal{ + Proposal: *pbP, + }, + }, + } + case *ProposalPOLMessage: + pbBits := msg.ProposalPOL.ToProto() + pb = tmcons.Message{ + Sum: &tmcons.Message_ProposalPol{ + ProposalPol: &tmcons.ProposalPOL{ + Height: msg.Height, + ProposalPolRound: msg.ProposalPOLRound, + ProposalPol: *pbBits, + }, + }, + } + case *BlockPartMessage: + parts, err := msg.Part.ToProto() + if err != nil { + return nil, fmt.Errorf("msg to proto error: %w", err) + } + pb = tmcons.Message{ + Sum: &tmcons.Message_BlockPart{ + BlockPart: &tmcons.BlockPart{ + Height: msg.Height, + Round: msg.Round, + Part: *parts, + }, + }, + } + case *VoteMessage: + vote := msg.Vote.ToProto() + pb = tmcons.Message{ + Sum: &tmcons.Message_Vote{ + Vote: &tmcons.Vote{ + Vote: vote, + }, + }, + } + case *HasVoteMessage: + pb = tmcons.Message{ + Sum: &tmcons.Message_HasVote{ + HasVote: &tmcons.HasVote{ + Height: msg.Height, + Round: msg.Round, + Type: msg.Type, + Index: msg.Index, + }, + }, + } + case *VoteSetMaj23Message: + bi := msg.BlockID.ToProto() + pb = tmcons.Message{ + Sum: &tmcons.Message_VoteSetMaj23{ + VoteSetMaj23: &tmcons.VoteSetMaj23{ + Height: msg.Height, + Round: msg.Round, + Type: msg.Type, + BlockID: bi, + }, + }, + } + case *VoteSetBitsMessage: + bi := msg.BlockID.ToProto() + bits := msg.Votes.ToProto() + + vsb := &tmcons.Message_VoteSetBits{ + VoteSetBits: &tmcons.VoteSetBits{ + Height: msg.Height, + Round: msg.Round, + Type: msg.Type, + BlockID: bi, + }, + } + + if bits != nil { + vsb.VoteSetBits.Votes = *bits + } + + pb = tmcons.Message{ + Sum: vsb, + } + + default: + return nil, fmt.Errorf("consensus: message not recognized: %T", msg) + } + + return &pb, nil +} + +// MsgFromProto takes a consensus proto message and returns the native go type +func MsgFromProto(msg *tmcons.Message) (Message, error) { + if msg == nil { + return nil, errors.New("consensus: nil message") + } + var pb Message + + switch msg := msg.Sum.(type) { + case *tmcons.Message_NewRoundStep: + rs, err := tmmath.SafeConvertUint8(int64(msg.NewRoundStep.Step)) + // deny message based on possible overflow + if err != nil { + return nil, fmt.Errorf("denying message due to possible overflow: %w", err) + } + pb = &NewRoundStepMessage{ + Height: msg.NewRoundStep.Height, + Round: msg.NewRoundStep.Round, + Step: cstypes.RoundStepType(rs), + SecondsSinceStartTime: msg.NewRoundStep.SecondsSinceStartTime, + LastCommitRound: msg.NewRoundStep.LastCommitRound, + } + case *tmcons.Message_NewValidBlock: + pbPartsHeader, err := types.PartSetHeaderFromProto(&msg.NewValidBlock.BlockPartsHeader) + if err != nil { + return nil, fmt.Errorf("parts to proto error: %w", err) + } + + pbBits := new(bits.BitArray) + pbBits.FromProto(msg.NewValidBlock.BlockParts) + + pb = &NewValidBlockMessage{ + Height: msg.NewValidBlock.Height, + Round: msg.NewValidBlock.Round, + BlockPartsHeader: *pbPartsHeader, + BlockParts: pbBits, + IsCommit: msg.NewValidBlock.IsCommit, + } + case *tmcons.Message_Proposal: + pbP, err := types.ProposalFromProto(&msg.Proposal.Proposal) + if err != nil { + return nil, fmt.Errorf("proposal msg to proto error: %w", err) + } + + pb = &ProposalMessage{ + Proposal: pbP, + } + case *tmcons.Message_ProposalPol: + pbBits := new(bits.BitArray) + pbBits.FromProto(&msg.ProposalPol.ProposalPol) + pb = &ProposalPOLMessage{ + Height: msg.ProposalPol.Height, + ProposalPOLRound: msg.ProposalPol.ProposalPolRound, + ProposalPOL: pbBits, + } + case *tmcons.Message_BlockPart: + parts, err := types.PartFromProto(&msg.BlockPart.Part) + if err != nil { + return nil, fmt.Errorf("blockpart msg to proto error: %w", err) + } + pb = &BlockPartMessage{ + Height: msg.BlockPart.Height, + Round: msg.BlockPart.Round, + Part: parts, + } + case *tmcons.Message_Vote: + vote, err := types.VoteFromProto(msg.Vote.Vote) + if err != nil { + return nil, fmt.Errorf("vote msg to proto error: %w", err) + } + + pb = &VoteMessage{ + Vote: vote, + } + case *tmcons.Message_HasVote: + pb = &HasVoteMessage{ + Height: msg.HasVote.Height, + Round: msg.HasVote.Round, + Type: msg.HasVote.Type, + Index: msg.HasVote.Index, + } + case *tmcons.Message_VoteSetMaj23: + bi, err := types.BlockIDFromProto(&msg.VoteSetMaj23.BlockID) + if err != nil { + return nil, fmt.Errorf("voteSetMaj23 msg to proto error: %w", err) + } + pb = &VoteSetMaj23Message{ + Height: msg.VoteSetMaj23.Height, + Round: msg.VoteSetMaj23.Round, + Type: msg.VoteSetMaj23.Type, + BlockID: *bi, + } + case *tmcons.Message_VoteSetBits: + bi, err := types.BlockIDFromProto(&msg.VoteSetBits.BlockID) + if err != nil { + return nil, fmt.Errorf("voteSetBits msg to proto error: %w", err) + } + bits := new(bits.BitArray) + bits.FromProto(&msg.VoteSetBits.Votes) + + pb = &VoteSetBitsMessage{ + Height: msg.VoteSetBits.Height, + Round: msg.VoteSetBits.Round, + Type: msg.VoteSetBits.Type, + BlockID: *bi, + Votes: bits, + } + default: + return nil, fmt.Errorf("consensus: message not recognized: %T", msg) + } + + if err := pb.ValidateBasic(); err != nil { + return nil, err + } + + return pb, nil +} + +// MustEncode takes the reactors msg, makes it proto and marshals it +// this mimics `MustMarshalBinaryBare` in that is panics on error +func MustEncode(msg Message) []byte { + pb, err := MsgToProto(msg) + if err != nil { + panic(err) + } + enc, err := proto.Marshal(pb) + if err != nil { + panic(err) + } + return enc +} + +// WALToProto takes a WAL message and return a proto walMessage and error +func WALToProto(msg WALMessage) (*tmcons.WALMessage, error) { + var pb tmcons.WALMessage + + switch msg := msg.(type) { + case types.EventDataRoundState: + pb = tmcons.WALMessage{ + Sum: &tmcons.WALMessage_EventDataRoundState{ + EventDataRoundState: &tmproto.EventDataRoundState{ + Height: msg.Height, + Round: msg.Round, + Step: msg.Step, + }, + }, + } + case msgInfo: + consMsg, err := MsgToProto(msg.Msg) + if err != nil { + return nil, err + } + pb = tmcons.WALMessage{ + Sum: &tmcons.WALMessage_MsgInfo{ + MsgInfo: &tmcons.MsgInfo{ + Msg: *consMsg, + PeerID: string(msg.PeerID), + }, + }, + } + case timeoutInfo: + pb = tmcons.WALMessage{ + Sum: &tmcons.WALMessage_TimeoutInfo{ + TimeoutInfo: &tmcons.TimeoutInfo{ + Duration: msg.Duration, + Height: msg.Height, + Round: msg.Round, + Step: uint32(msg.Step), + }, + }, + } + case EndHeightMessage: + pb = tmcons.WALMessage{ + Sum: &tmcons.WALMessage_EndHeight{ + EndHeight: &tmcons.EndHeight{ + Height: msg.Height, + }, + }, + } + default: + return nil, fmt.Errorf("to proto: wal message not recognized: %T", msg) + } + + return &pb, nil +} + +// WALFromProto takes a proto wal message and return a consensus walMessage and error +func WALFromProto(msg *tmcons.WALMessage) (WALMessage, error) { + if msg == nil { + return nil, errors.New("nil WAL message") + } + var pb WALMessage + + switch msg := msg.Sum.(type) { + case *tmcons.WALMessage_EventDataRoundState: + pb = types.EventDataRoundState{ + Height: msg.EventDataRoundState.Height, + Round: msg.EventDataRoundState.Round, + Step: msg.EventDataRoundState.Step, + } + case *tmcons.WALMessage_MsgInfo: + walMsg, err := MsgFromProto(&msg.MsgInfo.Msg) + if err != nil { + return nil, fmt.Errorf("msgInfo from proto error: %w", err) + } + pb = msgInfo{ + Msg: walMsg, + PeerID: p2p.ID(msg.MsgInfo.PeerID), + } + + case *tmcons.WALMessage_TimeoutInfo: + tis, err := tmmath.SafeConvertUint8(int64(msg.TimeoutInfo.Step)) + // deny message based on possible overflow + if err != nil { + return nil, fmt.Errorf("denying message due to possible overflow: %w", err) + } + pb = timeoutInfo{ + Duration: msg.TimeoutInfo.Duration, + Height: msg.TimeoutInfo.Height, + Round: msg.TimeoutInfo.Round, + Step: cstypes.RoundStepType(tis), + } + return pb, nil + case *tmcons.WALMessage_EndHeight: + pb := EndHeightMessage{ + Height: msg.EndHeight.Height, + } + return pb, nil + default: + return nil, fmt.Errorf("from proto: wal message not recognized: %T", msg) + } + return pb, nil +} diff --git a/consensus/msgs_test.go b/consensus/msgs_test.go new file mode 100644 index 000000000..c652a82a9 --- /dev/null +++ b/consensus/msgs_test.go @@ -0,0 +1,312 @@ +package consensus + +import ( + "testing" + "time" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "github.com/tendermint/tendermint/crypto/merkle" + "github.com/tendermint/tendermint/libs/bits" + tmrand "github.com/tendermint/tendermint/libs/rand" + "github.com/tendermint/tendermint/p2p" + tmcons "github.com/tendermint/tendermint/proto/consensus" + tmproto "github.com/tendermint/tendermint/proto/types" + "github.com/tendermint/tendermint/types" +) + +func TestMsgToProto(t *testing.T) { + psh := types.PartSetHeader{ + Total: 1, + Hash: tmrand.Bytes(32), + } + pbPsh := psh.ToProto() + bi := types.BlockID{ + Hash: tmrand.Bytes(32), + PartsHeader: psh, + } + pbBi := bi.ToProto() + bits := bits.NewBitArray(1) + pbBits := bits.ToProto() + + parts := types.Part{ + Index: 1, + Bytes: []byte("test"), + Proof: merkle.SimpleProof{ + Total: 1, + Index: 1, + LeafHash: tmrand.Bytes(32), + Aunts: [][]byte{}, + }, + } + pbParts, err := parts.ToProto() + require.NoError(t, err) + + proposal := types.Proposal{ + Type: tmproto.ProposalType, + Height: 1, + Round: 1, + POLRound: 1, + BlockID: bi, + Timestamp: time.Now(), + Signature: tmrand.Bytes(20), + } + pbProposal := proposal.ToProto() + + pv := types.NewMockPV() + pk, err := pv.GetPubKey() + require.NoError(t, err) + val := types.NewValidator(pk, 100) + + vote, err := types.MakeVote( + 1, types.BlockID{}, &types.ValidatorSet{Proposer: val, Validators: []*types.Validator{val}}, + pv, "chainID", time.Now()) + require.NoError(t, err) + pbVote := vote.ToProto() + + testsCases := []struct { + testName string + msg Message + want *tmcons.Message + wantErr bool + }{ + {"successful NewRoundStepMessage", &NewRoundStepMessage{ + Height: 2, + Round: 1, + Step: 1, + SecondsSinceStartTime: 1, + LastCommitRound: 2, + }, &tmcons.Message{ + Sum: &tmcons.Message_NewRoundStep{ + NewRoundStep: &tmcons.NewRoundStep{ + Height: 2, + Round: 1, + Step: 1, + SecondsSinceStartTime: 1, + LastCommitRound: 2, + }, + }, + }, false}, + + {"successful NewValidBlockMessage", &NewValidBlockMessage{ + Height: 1, + Round: 1, + BlockPartsHeader: psh, + BlockParts: bits, + IsCommit: false, + }, &tmcons.Message{ + Sum: &tmcons.Message_NewValidBlock{ + NewValidBlock: &tmcons.NewValidBlock{ + Height: 1, + Round: 1, + BlockPartsHeader: pbPsh, + BlockParts: pbBits, + IsCommit: false, + }, + }, + }, false}, + {"successful BlockPartMessage", &BlockPartMessage{ + Height: 100, + Round: 1, + Part: &parts, + }, &tmcons.Message{ + Sum: &tmcons.Message_BlockPart{ + BlockPart: &tmcons.BlockPart{ + Height: 100, + Round: 1, + Part: *pbParts, + }, + }, + }, false}, + {"successful ProposalPOLMessage", &ProposalPOLMessage{ + Height: 1, + ProposalPOLRound: 1, + ProposalPOL: bits, + }, &tmcons.Message{ + Sum: &tmcons.Message_ProposalPol{ + ProposalPol: &tmcons.ProposalPOL{ + Height: 1, + ProposalPolRound: 1, + ProposalPol: *pbBits, + }, + }}, false}, + {"successful ProposalMessage", &ProposalMessage{ + Proposal: &proposal, + }, &tmcons.Message{ + Sum: &tmcons.Message_Proposal{ + Proposal: &tmcons.Proposal{ + Proposal: *pbProposal, + }, + }, + }, false}, + {"successful VoteMessage", &VoteMessage{ + Vote: vote, + }, &tmcons.Message{ + Sum: &tmcons.Message_Vote{ + Vote: &tmcons.Vote{ + Vote: pbVote, + }, + }, + }, false}, + {"successful VoteSetMaj23", &VoteSetMaj23Message{ + Height: 1, + Round: 1, + Type: 1, + BlockID: bi, + }, &tmcons.Message{ + Sum: &tmcons.Message_VoteSetMaj23{ + VoteSetMaj23: &tmcons.VoteSetMaj23{ + Height: 1, + Round: 1, + Type: 1, + BlockID: pbBi, + }, + }, + }, false}, + {"successful VoteSetBits", &VoteSetBitsMessage{ + Height: 1, + Round: 1, + Type: 1, + BlockID: bi, + Votes: bits, + }, &tmcons.Message{ + Sum: &tmcons.Message_VoteSetBits{ + VoteSetBits: &tmcons.VoteSetBits{ + Height: 1, + Round: 1, + Type: 1, + BlockID: pbBi, + Votes: *pbBits, + }, + }, + }, false}, + {"failure", nil, &tmcons.Message{}, true}, + } + for _, tt := range testsCases { + tt := tt + t.Run(tt.testName, func(t *testing.T) { + pb, err := MsgToProto(tt.msg) + if tt.wantErr == true { + assert.Equal(t, err != nil, tt.wantErr) + return + } + assert.EqualValues(t, tt.want, pb, tt.testName) + + msg, err := MsgFromProto(pb) + + if !tt.wantErr { + require.NoError(t, err) + bcm := assert.Equal(t, tt.msg, msg, tt.testName) + assert.True(t, bcm, tt.testName) + } else { + require.Error(t, err, tt.testName) + } + }) + } +} + +func TestWALMsgProto(t *testing.T) { + + parts := types.Part{ + Index: 1, + Bytes: []byte("test"), + Proof: merkle.SimpleProof{ + Total: 1, + Index: 1, + LeafHash: tmrand.Bytes(32), + Aunts: [][]byte{}, + }, + } + pbParts, err := parts.ToProto() + require.NoError(t, err) + + testsCases := []struct { + testName string + msg WALMessage + want *tmcons.WALMessage + wantErr bool + }{ + {"successful EventDataRoundState", types.EventDataRoundState{ + Height: 2, + Round: 1, + Step: "ronies", + }, &tmcons.WALMessage{ + Sum: &tmcons.WALMessage_EventDataRoundState{ + EventDataRoundState: &tmproto.EventDataRoundState{ + Height: 2, + Round: 1, + Step: "ronies", + }, + }, + }, false}, + {"successful msgInfo", msgInfo{ + Msg: &BlockPartMessage{ + Height: 100, + Round: 1, + Part: &parts, + }, + PeerID: p2p.ID("string"), + }, &tmcons.WALMessage{ + Sum: &tmcons.WALMessage_MsgInfo{ + MsgInfo: &tmcons.MsgInfo{ + Msg: tmcons.Message{ + Sum: &tmcons.Message_BlockPart{ + BlockPart: &tmcons.BlockPart{ + Height: 100, + Round: 1, + Part: *pbParts, + }, + }, + }, + PeerID: "string", + }, + }, + }, false}, + {"successful timeoutInfo", timeoutInfo{ + Duration: time.Duration(100), + Height: 1, + Round: 1, + Step: 1, + }, &tmcons.WALMessage{ + Sum: &tmcons.WALMessage_TimeoutInfo{ + TimeoutInfo: &tmcons.TimeoutInfo{ + Duration: time.Duration(100), + Height: 1, + Round: 1, + Step: 1, + }, + }, + }, false}, + {"successful EndHeightMessage", EndHeightMessage{ + Height: 1, + }, &tmcons.WALMessage{ + Sum: &tmcons.WALMessage_EndHeight{ + EndHeight: &tmcons.EndHeight{ + Height: 1, + }, + }, + }, false}, + {"failure", nil, &tmcons.WALMessage{}, true}, + } + for _, tt := range testsCases { + tt := tt + t.Run(tt.testName, func(t *testing.T) { + pb, err := WALToProto(tt.msg) + if tt.wantErr == true { + assert.Equal(t, err != nil, tt.wantErr) + return + } + assert.EqualValues(t, tt.want, pb, tt.testName) + + msg, err := WALFromProto(pb) + + if !tt.wantErr { + require.NoError(t, err) + assert.Equal(t, tt.msg, msg, tt.testName) // need the concrete type as WAL Message is a empty interface + } else { + require.Error(t, err, tt.testName) + } + }) + } +} diff --git a/consensus/reactor.go b/consensus/reactor.go index b427dcec6..b35d5e5f3 100644 --- a/consensus/reactor.go +++ b/consensus/reactor.go @@ -7,7 +7,7 @@ import ( "sync" "time" - amino "github.com/tendermint/go-amino" + "github.com/gogo/protobuf/proto" cstypes "github.com/tendermint/tendermint/consensus/types" "github.com/tendermint/tendermint/libs/bits" @@ -15,6 +15,7 @@ import ( tmjson "github.com/tendermint/tendermint/libs/json" "github.com/tendermint/tendermint/libs/log" "github.com/tendermint/tendermint/p2p" + tmcons "github.com/tendermint/tendermint/proto/consensus" tmproto "github.com/tendermint/tendermint/proto/types" sm "github.com/tendermint/tendermint/state" "github.com/tendermint/tendermint/types" @@ -277,7 +278,7 @@ func (conR *Reactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) { default: panic("Bad VoteSetBitsMessage field Type. Forgot to add a check in ValidateBasic?") } - src.TrySend(VoteSetBitsChannel, cdc.MustMarshalBinaryBare(&VoteSetBitsMessage{ + src.TrySend(VoteSetBitsChannel, MustEncode(&VoteSetBitsMessage{ Height: msg.Height, Round: msg.Round, Type: msg.Type, @@ -409,7 +410,7 @@ func (conR *Reactor) unsubscribeFromBroadcastEvents() { func (conR *Reactor) broadcastNewRoundStepMessage(rs *cstypes.RoundState) { nrsMsg := makeRoundStepMessage(rs) - conR.Switch.Broadcast(StateChannel, cdc.MustMarshalBinaryBare(nrsMsg)) + conR.Switch.Broadcast(StateChannel, MustEncode(nrsMsg)) } func (conR *Reactor) broadcastNewValidBlockMessage(rs *cstypes.RoundState) { @@ -420,7 +421,7 @@ func (conR *Reactor) broadcastNewValidBlockMessage(rs *cstypes.RoundState) { BlockParts: rs.ProposalBlockParts.BitArray(), IsCommit: rs.Step == cstypes.RoundStepCommit, } - conR.Switch.Broadcast(StateChannel, cdc.MustMarshalBinaryBare(csMsg)) + conR.Switch.Broadcast(StateChannel, MustEncode(csMsg)) } // Broadcasts HasVoteMessage to peers that care. @@ -431,7 +432,7 @@ func (conR *Reactor) broadcastHasVoteMessage(vote *types.Vote) { Type: vote.Type, Index: vote.ValidatorIndex, } - conR.Switch.Broadcast(StateChannel, cdc.MustMarshalBinaryBare(msg)) + conR.Switch.Broadcast(StateChannel, MustEncode(msg)) /* // TODO: Make this broadcast more selective. for _, peer := range conR.Switch.Peers().List() { @@ -457,7 +458,7 @@ func makeRoundStepMessage(rs *cstypes.RoundState) (nrsMsg *NewRoundStepMessage) Height: rs.Height, Round: rs.Round, Step: rs.Step, - SecondsSinceStartTime: int(time.Since(rs.StartTime).Seconds()), + SecondsSinceStartTime: int64(time.Since(rs.StartTime).Seconds()), LastCommitRound: rs.LastCommit.GetRound(), } return @@ -466,7 +467,7 @@ func makeRoundStepMessage(rs *cstypes.RoundState) (nrsMsg *NewRoundStepMessage) func (conR *Reactor) sendNewRoundStepMessage(peer p2p.Peer) { rs := conR.conS.GetRoundState() nrsMsg := makeRoundStepMessage(rs) - peer.Send(StateChannel, cdc.MustMarshalBinaryBare(nrsMsg)) + peer.Send(StateChannel, MustEncode(nrsMsg)) } func (conR *Reactor) gossipDataRoutine(peer p2p.Peer, ps *PeerState) { @@ -492,7 +493,7 @@ OUTER_LOOP: Part: part, } logger.Debug("Sending block part", "height", prs.Height, "round", prs.Round) - if peer.Send(DataChannel, cdc.MustMarshalBinaryBare(msg)) { + if peer.Send(DataChannel, MustEncode(msg)) { ps.SetHasProposalBlockPart(prs.Height, prs.Round, index) } continue OUTER_LOOP @@ -538,7 +539,7 @@ OUTER_LOOP: { msg := &ProposalMessage{Proposal: rs.Proposal} logger.Debug("Sending proposal", "height", prs.Height, "round", prs.Round) - if peer.Send(DataChannel, cdc.MustMarshalBinaryBare(msg)) { + if peer.Send(DataChannel, MustEncode(msg)) { // NOTE[ZM]: A peer might have received different proposal msg so this Proposal msg will be rejected! ps.SetHasProposal(rs.Proposal) } @@ -554,7 +555,7 @@ OUTER_LOOP: ProposalPOL: rs.Votes.Prevotes(rs.Proposal.POLRound).BitArray(), } logger.Debug("Sending POL", "height", prs.Height, "round", prs.Round) - peer.Send(DataChannel, cdc.MustMarshalBinaryBare(msg)) + peer.Send(DataChannel, MustEncode(msg)) } continue OUTER_LOOP } @@ -597,7 +598,7 @@ func (conR *Reactor) gossipDataForCatchup(logger log.Logger, rs *cstypes.RoundSt Part: part, } logger.Debug("Sending block part for catchup", "round", prs.Round, "index", index) - if peer.Send(DataChannel, cdc.MustMarshalBinaryBare(msg)) { + if peer.Send(DataChannel, MustEncode(msg)) { ps.SetHasProposalBlockPart(prs.Height, prs.Round, index) } else { logger.Debug("Sending block part for catchup failed") @@ -757,7 +758,7 @@ OUTER_LOOP: prs := ps.GetRoundState() if rs.Height == prs.Height { if maj23, ok := rs.Votes.Prevotes(prs.Round).TwoThirdsMajority(); ok { - peer.TrySend(StateChannel, cdc.MustMarshalBinaryBare(&VoteSetMaj23Message{ + peer.TrySend(StateChannel, MustEncode(&VoteSetMaj23Message{ Height: prs.Height, Round: prs.Round, Type: tmproto.PrevoteType, @@ -774,7 +775,7 @@ OUTER_LOOP: prs := ps.GetRoundState() if rs.Height == prs.Height { if maj23, ok := rs.Votes.Precommits(prs.Round).TwoThirdsMajority(); ok { - peer.TrySend(StateChannel, cdc.MustMarshalBinaryBare(&VoteSetMaj23Message{ + peer.TrySend(StateChannel, MustEncode(&VoteSetMaj23Message{ Height: prs.Height, Round: prs.Round, Type: tmproto.PrecommitType, @@ -791,7 +792,7 @@ OUTER_LOOP: prs := ps.GetRoundState() if rs.Height == prs.Height && prs.ProposalPOLRound >= 0 { if maj23, ok := rs.Votes.Prevotes(prs.ProposalPOLRound).TwoThirdsMajority(); ok { - peer.TrySend(StateChannel, cdc.MustMarshalBinaryBare(&VoteSetMaj23Message{ + peer.TrySend(StateChannel, MustEncode(&VoteSetMaj23Message{ Height: prs.Height, Round: prs.ProposalPOLRound, Type: tmproto.PrevoteType, @@ -811,7 +812,7 @@ OUTER_LOOP: if prs.CatchupCommitRound != -1 && prs.Height > 0 && prs.Height <= conR.conS.blockStore.Height() && prs.Height >= conR.conS.blockStore.Base() { if commit := conR.conS.LoadCommit(prs.Height); commit != nil { - peer.TrySend(StateChannel, cdc.MustMarshalBinaryBare(&VoteSetMaj23Message{ + peer.TrySend(StateChannel, MustEncode(&VoteSetMaj23Message{ Height: prs.Height, Round: commit.Round, Type: tmproto.PrecommitType, @@ -1032,7 +1033,7 @@ func (ps *PeerState) PickSendVote(votes types.VoteSetReader) bool { if vote, ok := ps.PickVoteToSend(votes); ok { msg := &VoteMessage{vote} ps.logger.Debug("Sending vote message", "ps", ps, "vote", vote) - if ps.peer.Send(VoteChannel, cdc.MustMarshalBinaryBare(msg)) { + if ps.peer.Send(VoteChannel, MustEncode(msg)) { ps.SetHasVote(vote) return true } @@ -1386,19 +1387,6 @@ type Message interface { ValidateBasic() error } -func RegisterMessages(cdc *amino.Codec) { - cdc.RegisterInterface((*Message)(nil), nil) - cdc.RegisterConcrete(&NewRoundStepMessage{}, "tendermint/NewRoundStepMessage", nil) - cdc.RegisterConcrete(&NewValidBlockMessage{}, "tendermint/NewValidBlockMessage", nil) - cdc.RegisterConcrete(&ProposalMessage{}, "tendermint/Proposal", nil) - cdc.RegisterConcrete(&ProposalPOLMessage{}, "tendermint/ProposalPOL", nil) - cdc.RegisterConcrete(&BlockPartMessage{}, "tendermint/BlockPart", nil) - cdc.RegisterConcrete(&VoteMessage{}, "tendermint/Vote", nil) - cdc.RegisterConcrete(&HasVoteMessage{}, "tendermint/HasVote", nil) - cdc.RegisterConcrete(&VoteSetMaj23Message{}, "tendermint/VoteSetMaj23", nil) - cdc.RegisterConcrete(&VoteSetBitsMessage{}, "tendermint/VoteSetBits", nil) -} - func init() { tmjson.RegisterType(&NewRoundStepMessage{}, "tendermint/NewRoundStepMessage") tmjson.RegisterType(&NewValidBlockMessage{}, "tendermint/NewValidBlockMessage") @@ -1412,8 +1400,12 @@ func init() { } func decodeMsg(bz []byte) (msg Message, err error) { - err = cdc.UnmarshalBinaryBare(bz, &msg) - return + pb := &tmcons.Message{} + if err = proto.Unmarshal(bz, pb); err != nil { + return msg, err + } + + return MsgFromProto(pb) } //------------------------------------- @@ -1424,7 +1416,7 @@ type NewRoundStepMessage struct { Height int64 Round int32 Step cstypes.RoundStepType - SecondsSinceStartTime int + SecondsSinceStartTime int64 LastCommitRound int32 } diff --git a/consensus/reactor_test.go b/consensus/reactor_test.go index adfdf4d8e..f85f3b84c 100644 --- a/consensus/reactor_test.go +++ b/consensus/reactor_test.go @@ -109,9 +109,6 @@ func TestReactorBasic(t *testing.T) { // Ensure we can process blocks with evidence func TestReactorWithEvidence(t *testing.T) { - types.RegisterMockEvidences(cdc) - types.RegisterMockEvidences(types.GetCodec()) - nValidators := 4 testName := "consensus_reactor_test" tickerFunc := newMockTickerFunc(true) @@ -273,7 +270,8 @@ func TestReactorReceiveDoesNotPanicIfAddPeerHasntBeenCalledYet(t *testing.T) { var ( reactor = reactors[0] peer = mock.NewPeer(nil) - msg = cdc.MustMarshalBinaryBare(&HasVoteMessage{Height: 1, Round: 1, Index: 1, Type: tmproto.PrevoteType}) + msg = MustEncode(&HasVoteMessage{Height: 1, + Round: 1, Index: 1, Type: tmproto.PrevoteType}) ) reactor.InitPeer(peer) @@ -295,7 +293,8 @@ func TestReactorReceivePanicsIfInitPeerHasntBeenCalledYet(t *testing.T) { var ( reactor = reactors[0] peer = mock.NewPeer(nil) - msg = cdc.MustMarshalBinaryBare(&HasVoteMessage{Height: 1, Round: 1, Index: 1, Type: tmproto.PrevoteType}) + msg = MustEncode(&HasVoteMessage{Height: 1, + Round: 1, Index: 1, Type: tmproto.PrevoteType}) ) // we should call InitPeer here diff --git a/consensus/replay_stubs.go b/consensus/replay_stubs.go index 4df75c65c..de153ae3b 100644 --- a/consensus/replay_stubs.go +++ b/consensus/replay_stubs.go @@ -89,7 +89,7 @@ type mockProxyApp struct { func (mock *mockProxyApp) DeliverTx(req abci.RequestDeliverTx) abci.ResponseDeliverTx { r := mock.abciResponses.DeliverTxs[mock.txCount] mock.txCount++ - if r == nil { //it could be nil because of amino unMarshall, it will cause an empty ResponseDeliverTx to become nil + if r == nil { return abci.ResponseDeliverTx{} } return *r diff --git a/consensus/replay_test.go b/consensus/replay_test.go index cdfe98715..3d77efd08 100644 --- a/consensus/replay_test.go +++ b/consensus/replay_test.go @@ -9,15 +9,13 @@ import ( "os" "path/filepath" "runtime" + "sort" "testing" "time" "github.com/gogo/protobuf/proto" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - - "sort" - dbm "github.com/tendermint/tm-db" "github.com/tendermint/tendermint/abci/example/kvstore" @@ -584,11 +582,12 @@ func TestMockProxyApp(t *testing.T) { abciResWithEmptyDeliverTx.DeliverTxs = append(abciResWithEmptyDeliverTx.DeliverTxs, &abci.ResponseDeliverTx{}) // called when saveABCIResponses: - bytes := cdc.MustMarshalBinaryBare(abciResWithEmptyDeliverTx) + bytes, err := proto.Marshal(abciResWithEmptyDeliverTx) + require.NoError(t, err) loadedAbciRes := new(tmstate.ABCIResponses) // this also happens sm.LoadABCIResponses - err := cdc.UnmarshalBinaryBare(bytes, loadedAbciRes) + err = proto.Unmarshal(bytes, loadedAbciRes) require.NoError(t, err) mock := newMockProxyApp([]byte("mock_hash"), loadedAbciRes) diff --git a/consensus/state.go b/consensus/state.go index 7e8ce5d40..51e44e99d 100644 --- a/consensus/state.go +++ b/consensus/state.go @@ -248,7 +248,7 @@ func (cs *State) GetRoundStateJSON() ([]byte, error) { return tmjson.Marshal(cs.RoundState) } -// GetRoundStateSimpleJSON returns a json of RoundStateSimple, marshalled using go-amino. +// GetRoundStateSimpleJSON returns a json of RoundStateSimple func (cs *State) GetRoundStateSimpleJSON() ([]byte, error) { cs.mtx.RLock() defer cs.mtx.RUnlock() @@ -1976,7 +1976,7 @@ func (cs *State) addVote( } default: - panic(fmt.Sprintf("Unexpected vote type %X", vote.Type)) // go-amino should prevent this. + panic(fmt.Sprintf("Unexpected vote type %v", vote.Type)) } return added, err diff --git a/consensus/types/codec.go b/consensus/types/codec.go deleted file mode 100644 index 69ac8c4a5..000000000 --- a/consensus/types/codec.go +++ /dev/null @@ -1,13 +0,0 @@ -package types - -import ( - amino "github.com/tendermint/go-amino" - - "github.com/tendermint/tendermint/types" -) - -var cdc = amino.NewCodec() - -func init() { - types.RegisterBlockAmino(cdc) -} diff --git a/consensus/types/peer_round_state.go b/consensus/types/peer_round_state.go index 5b27811f4..38ec526ce 100644 --- a/consensus/types/peer_round_state.go +++ b/consensus/types/peer_round_state.go @@ -65,31 +65,3 @@ func (prs PeerRoundState) StringIndented(indent string) string { indent, prs.CatchupCommit, prs.CatchupCommitRound, indent) } - -//----------------------------------------------------------- -// These methods are for Protobuf Compatibility - -// Size returns the size of the amino encoding, in bytes. -func (prs *PeerRoundState) Size() int { - bs, _ := prs.Marshal() - return len(bs) -} - -// Marshal returns the amino encoding. -func (prs *PeerRoundState) Marshal() ([]byte, error) { - return cdc.MarshalBinaryBare(prs) -} - -// MarshalTo calls Marshal and copies to the given buffer. -func (prs *PeerRoundState) MarshalTo(data []byte) (int, error) { - bs, err := prs.Marshal() - if err != nil { - return -1, err - } - return copy(data, bs), nil -} - -// Unmarshal deserializes from amino encoded form. -func (prs *PeerRoundState) Unmarshal(bs []byte) error { - return cdc.UnmarshalBinaryBare(bs, prs) -} diff --git a/consensus/types/round_state.go b/consensus/types/round_state.go index 6fd674f73..532afeac0 100644 --- a/consensus/types/round_state.go +++ b/consensus/types/round_state.go @@ -213,31 +213,3 @@ func (rs *RoundState) StringShort() string { return fmt.Sprintf(`RoundState{H:%v R:%v S:%v ST:%v}`, rs.Height, rs.Round, rs.Step, rs.StartTime) } - -//----------------------------------------------------------- -// These methods are for Protobuf Compatibility - -// Size returns the size of the amino encoding, in bytes. -func (rs *RoundStateSimple) Size() int { - bs, _ := rs.Marshal() - return len(bs) -} - -// Marshal returns the amino encoding. -func (rs *RoundStateSimple) Marshal() ([]byte, error) { - return cdc.MarshalBinaryBare(rs) -} - -// MarshalTo calls Marshal and copies to the given buffer. -func (rs *RoundStateSimple) MarshalTo(data []byte) (int, error) { - bs, err := rs.Marshal() - if err != nil { - return -1, err - } - return copy(data, bs), nil -} - -// Unmarshal deserializes from amino encoded form. -func (rs *RoundStateSimple) Unmarshal(bs []byte) error { - return cdc.UnmarshalBinaryBare(bs, rs) -} diff --git a/consensus/types/round_state_test.go b/consensus/types/round_state_test.go index 131158f0e..749546c0d 100644 --- a/consensus/types/round_state_test.go +++ b/consensus/types/round_state_test.go @@ -84,6 +84,7 @@ func BenchmarkRoundStateDeepCopy(b *testing.B) { LastCommit: nil, // TODO LastValidators: vset, } + b.StartTimer() for i := 0; i < b.N; i++ { diff --git a/consensus/wal.go b/consensus/wal.go index 215bfd1bd..1b434b359 100644 --- a/consensus/wal.go +++ b/consensus/wal.go @@ -2,29 +2,26 @@ package consensus import ( "encoding/binary" + "errors" "fmt" "hash/crc32" "io" "path/filepath" "time" - amino "github.com/tendermint/go-amino" + "github.com/gogo/protobuf/proto" auto "github.com/tendermint/tendermint/libs/autofile" tmjson "github.com/tendermint/tendermint/libs/json" "github.com/tendermint/tendermint/libs/log" tmos "github.com/tendermint/tendermint/libs/os" "github.com/tendermint/tendermint/libs/service" - "github.com/tendermint/tendermint/types" + tmcons "github.com/tendermint/tendermint/proto/consensus" tmtime "github.com/tendermint/tendermint/types/time" ) const ( - // amino overhead + time.Time + max consensus msg size - // - // q: where 24 bytes are coming from? - // a: cdc.MustMarshalBinaryBare(empty consensus part msg) = 14 bytes. +10 - // bytes just in case amino will require more space in the future. + // time.Time + max consensus msg size maxMsgSizeBytes = maxMsgSize + 24 // how often the WAL should be sync'd during period sync'ing @@ -48,14 +45,6 @@ type EndHeightMessage struct { type WALMessage interface{} -func RegisterWALMessages(cdc *amino.Codec) { - cdc.RegisterInterface((*WALMessage)(nil), nil) - cdc.RegisterConcrete(types.EventDataRoundState{}, "tendermint/wal/EventDataRoundState", nil) - cdc.RegisterConcrete(msgInfo{}, "tendermint/wal/MsgInfo", nil) - cdc.RegisterConcrete(timeoutInfo{}, "tendermint/wal/TimeoutInfo", nil) - cdc.RegisterConcrete(EndHeightMessage{}, "tendermint/wal/EndHeightMessage", nil) -} - func init() { tmjson.RegisterType(msgInfo{}, "tendermint/wal/MsgInfo") tmjson.RegisterType(timeoutInfo{}, "tendermint/wal/TimeoutInfo") @@ -291,7 +280,7 @@ func (wal *BaseWAL) SearchForEndHeight( // A WALEncoder writes custom-encoded WAL messages to an output stream. // -// Format: 4 bytes CRC sum + 4 bytes length + arbitrary-length value (go-amino encoded) +// Format: 4 bytes CRC sum + 4 bytes length + arbitrary-length value type WALEncoder struct { wr io.Writer } @@ -302,10 +291,22 @@ func NewWALEncoder(wr io.Writer) *WALEncoder { } // Encode writes the custom encoding of v to the stream. It returns an error if -// the amino-encoded size of v is greater than 1MB. Any error encountered +// the encoded size of v is greater than 1MB. Any error encountered // during the write is also returned. func (enc *WALEncoder) Encode(v *TimedWALMessage) error { - data := cdc.MustMarshalBinaryBare(v) + pbMsg, err := WALToProto(v.Msg) + if err != nil { + return err + } + pv := tmcons.TimedWALMessage{ + Time: v.Time, + Msg: pbMsg, + } + + data, err := proto.Marshal(&pv) + if err != nil { + panic(fmt.Errorf("encode timed wall message failure: %w", err)) + } crc := crc32.Checksum(data, crc32c) length := uint32(len(data)) @@ -319,7 +320,7 @@ func (enc *WALEncoder) Encode(v *TimedWALMessage) error { binary.BigEndian.PutUint32(msg[4:8], length) copy(msg[8:], data) - _, err := enc.wr.Write(msg) + _, err = enc.wr.Write(msg) return err } @@ -363,7 +364,7 @@ func (dec *WALDecoder) Decode() (*TimedWALMessage, error) { b := make([]byte, 4) _, err := dec.rd.Read(b) - if err == io.EOF { + if errors.Is(err, io.EOF) { return nil, err } if err != nil { @@ -397,13 +398,22 @@ func (dec *WALDecoder) Decode() (*TimedWALMessage, error) { return nil, DataCorruptionError{fmt.Errorf("checksums do not match: read: %v, actual: %v", crc, actualCRC)} } - var res = new(TimedWALMessage) - err = cdc.UnmarshalBinaryBare(data, res) + var res = new(tmcons.TimedWALMessage) + err = proto.Unmarshal(data, res) if err != nil { return nil, DataCorruptionError{fmt.Errorf("failed to decode data: %v", err)} } - return res, err + walMsg, err := WALFromProto(res.Msg) + if err != nil { + return nil, DataCorruptionError{fmt.Errorf("failed to convert from proto: %w", err)} + } + tMsgWal := &TimedWALMessage{ + Time: res.Time, + Msg: walMsg, + } + + return tMsgWal, err } type nilWAL struct{} diff --git a/consensus/wal_test.go b/consensus/wal_test.go index 6871f534d..044ea2ddf 100644 --- a/consensus/wal_test.go +++ b/consensus/wal_test.go @@ -82,6 +82,7 @@ func TestWALEncoderDecoder(t *testing.T) { msgs := []TimedWALMessage{ {Time: now, Msg: EndHeightMessage{0}}, {Time: now, Msg: timeoutInfo{Duration: time.Second, Height: 1, Round: 1, Step: types.RoundStepPropose}}, + {Time: now, Msg: tmtypes.EventDataRoundState{Height: 1, Round: 1, Step: ""}}, } b := new(bytes.Buffer) @@ -98,7 +99,6 @@ func TestWALEncoderDecoder(t *testing.T) { dec := NewWALDecoder(b) decoded, err := dec.Decode() require.NoError(t, err) - assert.Equal(t, msg.Time.UTC(), decoded.Time) assert.Equal(t, msg.Msg, decoded.Msg) } @@ -135,7 +135,10 @@ func TestWALWrite(t *testing.T) { }, }, } - err = wal.Write(msg) + + err = wal.Write(msgInfo{ + Msg: msg, + }) if assert.Error(t, err) { assert.Contains(t, err.Error(), "msg is too big") } diff --git a/crypto/merkle/simple_proof_test.go b/crypto/merkle/simple_proof_test.go index 68e6912fb..2dd8b7eb0 100644 --- a/crypto/merkle/simple_proof_test.go +++ b/crypto/merkle/simple_proof_test.go @@ -4,6 +4,9 @@ import ( "testing" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + tmrand "github.com/tendermint/tendermint/libs/rand" ) func TestSimpleProofValidateBasic(t *testing.T) { @@ -39,3 +42,35 @@ func TestSimpleProofValidateBasic(t *testing.T) { }) } } + +func TestSimpleProofProtoBuf(t *testing.T) { + testCases := []struct { + testName string + ps1 *SimpleProof + expPass bool + }{ + {"failure empty", &SimpleProof{}, false}, + {"failure nil", nil, false}, + {"success", + &SimpleProof{ + Total: 1, + Index: 1, + LeafHash: tmrand.Bytes(32), + Aunts: [][]byte{tmrand.Bytes(32), tmrand.Bytes(32), tmrand.Bytes(32)}, + }, true}, + } + + for _, tc := range testCases { + tc := tc + t.Run(tc.testName, func(t *testing.T) { + proto := tc.ps1.ToProto() + p, err := SimpleProofFromProto(proto) + if tc.expPass { + require.NoError(t, err) + require.Equal(t, tc.ps1, p, tc.testName) + } else { + require.Error(t, err, tc.testName) + } + }) + } +} diff --git a/crypto/merkle/simple_tree_test.go b/crypto/merkle/simple_tree_test.go index d6d4345e8..01d6058c3 100644 --- a/crypto/merkle/simple_tree_test.go +++ b/crypto/merkle/simple_tree_test.go @@ -109,7 +109,7 @@ func BenchmarkSimpleHashAlternatives(b *testing.B) { func Test_getSplitPoint(t *testing.T) { tests := []struct { length int64 - want int + want int64 }{ {1, 0}, {2, 1}, diff --git a/libs/math/safemath.go b/libs/math/safemath.go index 458ad9788..2c59c191c 100644 --- a/libs/math/safemath.go +++ b/libs/math/safemath.go @@ -6,6 +6,7 @@ import ( ) var ErrOverflowInt32 = errors.New("int32 overflow") +var ErrOverflowUint8 = errors.New("uint8 overflow") // SafeAddInt32 adds two int32 integers // If there is an overflow this will panic @@ -39,3 +40,14 @@ func SafeConvertInt32(a int64) int32 { } return int32(a) } + +// SafeConvertUint8 takes an int64 and checks if it overflows +// If there is an overflow it returns an error +func SafeConvertUint8(a int64) (uint8, error) { + if a > math.MaxUint8 { + return 0, ErrOverflowUint8 + } else if a < 0 { + return 0, ErrOverflowUint8 + } + return uint8(a), nil +} diff --git a/proto/consensus/msgs.pb.go b/proto/consensus/msgs.pb.go index 3b6a138c0..80d52749a 100644 --- a/proto/consensus/msgs.pb.go +++ b/proto/consensus/msgs.pb.go @@ -399,7 +399,7 @@ type HasVote struct { Height int64 `protobuf:"varint,1,opt,name=height,proto3" json:"height,omitempty"` Round int32 `protobuf:"varint,2,opt,name=round,proto3" json:"round,omitempty"` Type types.SignedMsgType `protobuf:"varint,3,opt,name=type,proto3,enum=tendermint.proto.types.SignedMsgType" json:"type,omitempty"` - Index uint32 `protobuf:"varint,4,opt,name=index,proto3" json:"index,omitempty"` + Index int32 `protobuf:"varint,4,opt,name=index,proto3" json:"index,omitempty"` } func (m *HasVote) Reset() { *m = HasVote{} } @@ -456,7 +456,7 @@ func (m *HasVote) GetType() types.SignedMsgType { return types.SIGNED_MSG_TYPE_UNKNOWN } -func (m *HasVote) GetIndex() uint32 { +func (m *HasVote) GetIndex() int32 { if m != nil { return m.Index } @@ -802,60 +802,60 @@ func init() { proto.RegisterFile("proto/consensus/msgs.proto", fileDescriptor_9d var fileDescriptor_9de64017f8b3fc88 = []byte{ // 863 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x56, 0xcb, 0x6e, 0xdb, 0x46, - 0x14, 0x25, 0x63, 0xc9, 0x92, 0x2f, 0x2d, 0x3b, 0x1d, 0xf4, 0x21, 0x38, 0x85, 0x6c, 0xb0, 0x4d, - 0xab, 0x16, 0x05, 0x15, 0x28, 0x40, 0x1f, 0xbb, 0x94, 0x7d, 0x80, 0x69, 0x2d, 0x47, 0xa0, 0x82, - 0x00, 0xed, 0x86, 0xa0, 0xc4, 0x01, 0x35, 0xad, 0xc8, 0x61, 0x39, 0x23, 0xb9, 0xfa, 0x80, 0x02, - 0x5d, 0xf6, 0x1b, 0xba, 0xee, 0xb6, 0x7f, 0xd0, 0x45, 0x96, 0x59, 0x76, 0x15, 0x14, 0xf2, 0x6f, - 0x74, 0x51, 0xcc, 0x43, 0x22, 0x9d, 0x80, 0xb6, 0xb5, 0x29, 0x90, 0x8d, 0x30, 0x73, 0x1f, 0x67, - 0xee, 0x9c, 0x3b, 0xf7, 0x88, 0x70, 0x94, 0xe5, 0x94, 0xd3, 0xde, 0x84, 0xa6, 0x0c, 0xa7, 0x6c, - 0xce, 0x7a, 0x09, 0x8b, 0x99, 0x23, 0x8d, 0xe8, 0x88, 0xe3, 0x34, 0xc2, 0x79, 0x42, 0x52, 0xae, - 0x2c, 0xce, 0x26, 0xec, 0xe8, 0x3d, 0x3e, 0x25, 0x79, 0x14, 0x64, 0x61, 0xce, 0x97, 0x3d, 0x85, - 0x11, 0xd3, 0x98, 0x16, 0x2b, 0x95, 0x71, 0xf4, 0x96, 0xb2, 0xf0, 0x65, 0x86, 0x99, 0xfa, 0xd5, - 0x8e, 0x3b, 0xca, 0x31, 0x23, 0x63, 0xd6, 0x1b, 0x13, 0x7e, 0xc9, 0x69, 0xff, 0x69, 0xc2, 0xfe, - 0x19, 0x3e, 0xf7, 0xe9, 0x3c, 0x8d, 0x46, 0x1c, 0x67, 0xe8, 0x4d, 0xd8, 0x9d, 0x62, 0x12, 0x4f, - 0x79, 0xdb, 0x3c, 0x31, 0xbb, 0x3b, 0xbe, 0xde, 0xa1, 0xd7, 0xa1, 0x9e, 0x8b, 0xa0, 0xf6, 0xad, - 0x13, 0xb3, 0x5b, 0xf7, 0xd5, 0x06, 0x21, 0xa8, 0x31, 0x8e, 0xb3, 0xf6, 0xce, 0x89, 0xd9, 0x6d, - 0xf9, 0x72, 0x8d, 0x3e, 0x81, 0x36, 0xc3, 0x13, 0x9a, 0x46, 0x2c, 0x60, 0x24, 0x9d, 0xe0, 0x80, - 0xf1, 0x30, 0xe7, 0x01, 0x27, 0x09, 0x6e, 0xd7, 0x24, 0xe6, 0x1b, 0xda, 0x3f, 0x12, 0xee, 0x91, - 0xf0, 0x3e, 0x26, 0x09, 0x46, 0x1f, 0xc2, 0x6b, 0xb3, 0x90, 0xf1, 0x60, 0x42, 0x93, 0x84, 0xf0, - 0x40, 0x1d, 0x57, 0x97, 0xc7, 0x1d, 0x0a, 0xc7, 0x17, 0xd2, 0x2e, 0x4b, 0xb5, 0xff, 0x35, 0xa1, - 0x75, 0x86, 0xcf, 0x9f, 0x84, 0x33, 0x12, 0xb9, 0x33, 0x3a, 0xf9, 0x71, 0xcb, 0xc2, 0xbf, 0x03, - 0x34, 0x16, 0x69, 0x92, 0x57, 0x16, 0x4c, 0x71, 0x18, 0xe1, 0x5c, 0x5e, 0xc3, 0xea, 0xdf, 0x75, - 0x5e, 0x6a, 0x87, 0xa2, 0x6c, 0x18, 0xe6, 0x7c, 0x84, 0xb9, 0x27, 0x83, 0xdd, 0xda, 0xd3, 0xe7, - 0xc7, 0x86, 0x7f, 0x5b, 0xc2, 0x08, 0x0f, 0x53, 0x76, 0xf4, 0x15, 0x58, 0x25, 0x68, 0x79, 0x65, - 0xab, 0xff, 0xee, 0xcb, 0x98, 0xa2, 0x21, 0x8e, 0x68, 0x88, 0xe3, 0x12, 0xfe, 0x79, 0x9e, 0x87, - 0x4b, 0x1f, 0x0a, 0x30, 0x74, 0x07, 0xf6, 0x08, 0xd3, 0x5c, 0x48, 0x16, 0x9a, 0x7e, 0x93, 0x30, - 0xc5, 0x81, 0x7d, 0x06, 0xcd, 0x61, 0x4e, 0x33, 0xca, 0xc2, 0x19, 0x72, 0xa1, 0x99, 0xe9, 0xb5, - 0xbc, 0xba, 0xd5, 0x3f, 0xa9, 0xbc, 0x80, 0x8e, 0xd3, 0xb5, 0x6f, 0xf2, 0xec, 0xdf, 0x4d, 0xb0, - 0xd6, 0xce, 0xe1, 0xa3, 0xd3, 0x4a, 0x32, 0x3f, 0x02, 0xb4, 0xce, 0x09, 0x32, 0x3a, 0x0b, 0xca, - 0xcc, 0xde, 0x5e, 0x7b, 0x86, 0x74, 0x26, 0x9b, 0x84, 0x06, 0xb0, 0x5f, 0x8e, 0xd6, 0xf4, 0xde, - 0x88, 0x0a, 0x5d, 0xa1, 0x55, 0xc2, 0xb4, 0x7f, 0x82, 0x3d, 0x77, 0xcd, 0xcf, 0x96, 0xed, 0xfe, - 0x18, 0x6a, 0xa2, 0x1b, 0xba, 0x82, 0xb7, 0xaf, 0x6a, 0xb0, 0x3e, 0x59, 0xc6, 0xdb, 0x9f, 0x42, - 0xed, 0x09, 0xe5, 0x18, 0xdd, 0x83, 0xda, 0x82, 0x72, 0xac, 0xf9, 0xad, 0xcc, 0x17, 0xb1, 0xbe, - 0x8c, 0xb4, 0x7f, 0x35, 0xa1, 0xe1, 0x85, 0x4c, 0x66, 0x6f, 0x57, 0xeb, 0x67, 0x50, 0x13, 0x68, - 0xb2, 0xd6, 0x83, 0xea, 0xc7, 0x38, 0x22, 0x71, 0x8a, 0xa3, 0x01, 0x8b, 0x1f, 0x2f, 0x33, 0xec, - 0xcb, 0x14, 0x01, 0x48, 0xd2, 0x08, 0xff, 0x2c, 0x1f, 0x5d, 0xcb, 0x57, 0x1b, 0xfb, 0x2f, 0x13, - 0xf6, 0x45, 0x1d, 0x23, 0xcc, 0x07, 0xe1, 0x0f, 0xfd, 0xfb, 0xff, 0x5f, 0x3d, 0xdf, 0x42, 0x53, - 0x8d, 0x02, 0x89, 0xf4, 0x1c, 0x1c, 0x57, 0xa5, 0xcb, 0xce, 0x3e, 0xfc, 0xd2, 0x3d, 0x14, 0xec, - 0xaf, 0x9e, 0x1f, 0x37, 0xb4, 0xc1, 0x6f, 0x48, 0x84, 0x87, 0x91, 0xfd, 0xcb, 0x2d, 0xb0, 0xf4, - 0x35, 0x5c, 0xc2, 0xd9, 0xab, 0x79, 0x0b, 0xf4, 0x00, 0xea, 0xe2, 0x7d, 0x30, 0x39, 0xd2, 0xdb, - 0x0d, 0x83, 0x4a, 0xb4, 0xff, 0xa8, 0x43, 0x63, 0x80, 0x19, 0x0b, 0x63, 0x8c, 0x86, 0x70, 0x90, - 0xe2, 0x73, 0x35, 0x86, 0x81, 0x54, 0x62, 0xf5, 0x42, 0xbb, 0x4e, 0xf5, 0x3f, 0x8a, 0x53, 0xd6, - 0x7b, 0xcf, 0xf0, 0xf7, 0xd3, 0xb2, 0xfe, 0x8f, 0xe0, 0x50, 0x20, 0x2e, 0x84, 0xb0, 0x06, 0xb2, - 0x68, 0xc9, 0xa3, 0xd5, 0xff, 0xe0, 0x1a, 0xc8, 0x42, 0x8a, 0x3d, 0xc3, 0x6f, 0xa5, 0x97, 0xb4, - 0xb9, 0x2c, 0x51, 0x95, 0x22, 0x50, 0xa0, 0xad, 0x95, 0xc8, 0x2b, 0x49, 0x14, 0x3a, 0x7d, 0x41, - 0x4c, 0x54, 0x27, 0xde, 0xbf, 0x09, 0xce, 0xf0, 0xd1, 0xa9, 0x77, 0x59, 0x4b, 0xd0, 0xd7, 0x00, - 0x85, 0x48, 0xeb, 0x5e, 0xdc, 0xbd, 0x0a, 0x6b, 0xa3, 0x3c, 0x9e, 0xe1, 0xef, 0x6d, 0x64, 0x5a, - 0x08, 0x8b, 0x14, 0x86, 0xdd, 0x2a, 0xe1, 0x2d, 0x10, 0xc4, 0xdb, 0xf5, 0x0c, 0x25, 0x0f, 0xe8, - 0x01, 0x34, 0xa7, 0x21, 0x0b, 0x64, 0x6e, 0x43, 0xe6, 0xbe, 0x73, 0x55, 0xae, 0x56, 0x12, 0xcf, - 0xf0, 0x1b, 0x53, 0x2d, 0x2a, 0x43, 0x38, 0x10, 0xd9, 0x01, 0xc3, 0x3c, 0x48, 0xc4, 0x58, 0xb7, - 0x9b, 0xd7, 0xb7, 0xbe, 0x2c, 0x03, 0xa2, 0xf5, 0x8b, 0xb2, 0x2c, 0x0c, 0xa0, 0xb5, 0x41, 0x14, - 0xef, 0xaf, 0xbd, 0x77, 0x3d, 0xc5, 0xa5, 0x81, 0x14, 0x14, 0x2f, 0x8a, 0xad, 0x5b, 0x87, 0x1d, - 0x36, 0x4f, 0xdc, 0x6f, 0x9e, 0xae, 0x3a, 0xe6, 0xb3, 0x55, 0xc7, 0xfc, 0x67, 0xd5, 0x31, 0x7f, - 0xbb, 0xe8, 0x18, 0xcf, 0x2e, 0x3a, 0xc6, 0xdf, 0x17, 0x1d, 0xe3, 0xfb, 0x7b, 0x31, 0xe1, 0xd3, - 0xf9, 0xd8, 0x99, 0xd0, 0xa4, 0x57, 0x1c, 0x51, 0x5e, 0xbe, 0xf0, 0xc9, 0x34, 0xde, 0x95, 0x86, - 0xfb, 0xff, 0x05, 0x00, 0x00, 0xff, 0xff, 0x77, 0x46, 0xa2, 0xe6, 0x4c, 0x09, 0x00, 0x00, + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x56, 0xcb, 0x8e, 0xe3, 0x44, + 0x14, 0xb5, 0xa7, 0x93, 0x4e, 0xfa, 0xba, 0x1f, 0x43, 0x89, 0x47, 0xd4, 0x83, 0xd2, 0x91, 0x61, + 0x20, 0x20, 0xe4, 0x8c, 0x32, 0x12, 0x8f, 0xdd, 0x60, 0x1e, 0xf2, 0x40, 0xa7, 0x27, 0x72, 0x46, + 0x23, 0xc1, 0xc6, 0x72, 0xe2, 0x92, 0x53, 0x10, 0xbb, 0x8c, 0xab, 0x92, 0x26, 0x1f, 0x80, 0xc4, + 0x92, 0x6f, 0x60, 0xcd, 0x96, 0x3f, 0x60, 0x31, 0xcb, 0x59, 0xb2, 0x1a, 0xa1, 0xf4, 0x6f, 0xb0, + 0x40, 0xf5, 0x48, 0xec, 0x9e, 0x91, 0xbb, 0x3b, 0x1b, 0xa4, 0xd9, 0x44, 0x55, 0xf7, 0x71, 0xea, + 0xd6, 0xb9, 0x75, 0x4f, 0x0c, 0xc7, 0x59, 0x4e, 0x39, 0xed, 0x4d, 0x68, 0xca, 0x70, 0xca, 0xe6, + 0xac, 0x97, 0xb0, 0x98, 0x39, 0xd2, 0x88, 0x8e, 0x39, 0x4e, 0x23, 0x9c, 0x27, 0x24, 0xe5, 0xca, + 0xe2, 0x6c, 0xc2, 0x8e, 0xdf, 0xe3, 0x53, 0x92, 0x47, 0x41, 0x16, 0xe6, 0x7c, 0xd9, 0x53, 0x18, + 0x31, 0x8d, 0x69, 0xb1, 0x52, 0x19, 0xc7, 0x6f, 0x29, 0x0b, 0x5f, 0x66, 0x98, 0xa9, 0x5f, 0xed, + 0xb8, 0xa3, 0x1c, 0x33, 0x32, 0x66, 0xbd, 0x31, 0xe1, 0x97, 0x9c, 0xf6, 0x9f, 0x26, 0xec, 0x9f, + 0xe1, 0x73, 0x9f, 0xce, 0xd3, 0x68, 0xc4, 0x71, 0x86, 0xde, 0x84, 0xdd, 0x29, 0x26, 0xf1, 0x94, + 0xb7, 0xcc, 0x8e, 0xd9, 0xdd, 0xf1, 0xf5, 0x0e, 0xbd, 0x0e, 0xf5, 0x5c, 0x04, 0xb5, 0x6e, 0x75, + 0xcc, 0x6e, 0xdd, 0x57, 0x1b, 0x84, 0xa0, 0xc6, 0x38, 0xce, 0x5a, 0x3b, 0x1d, 0xb3, 0x7b, 0xe0, + 0xcb, 0x35, 0xfa, 0x04, 0x5a, 0x0c, 0x4f, 0x68, 0x1a, 0xb1, 0x80, 0x91, 0x74, 0x82, 0x03, 0xc6, + 0xc3, 0x9c, 0x07, 0x9c, 0x24, 0xb8, 0x55, 0x93, 0x98, 0x6f, 0x68, 0xff, 0x48, 0xb8, 0x47, 0xc2, + 0xfb, 0x98, 0x24, 0x18, 0x7d, 0x08, 0xaf, 0xcd, 0x42, 0xc6, 0x83, 0x09, 0x4d, 0x12, 0xc2, 0x03, + 0x75, 0x5c, 0x5d, 0x1e, 0x77, 0x24, 0x1c, 0x5f, 0x48, 0xbb, 0x2c, 0xd5, 0xfe, 0xd7, 0x84, 0x83, + 0x33, 0x7c, 0xfe, 0x24, 0x9c, 0x91, 0xc8, 0x9d, 0xd1, 0xc9, 0x8f, 0x5b, 0x16, 0xfe, 0x1d, 0xa0, + 0xb1, 0x48, 0x93, 0xbc, 0xb2, 0x60, 0x8a, 0xc3, 0x08, 0xe7, 0xf2, 0x1a, 0x56, 0xff, 0xae, 0xf3, + 0x52, 0x3b, 0x14, 0x65, 0xc3, 0x30, 0xe7, 0x23, 0xcc, 0x3d, 0x19, 0xec, 0xd6, 0x9e, 0x3e, 0x3f, + 0x31, 0xfc, 0xdb, 0x12, 0x46, 0x78, 0x98, 0xb2, 0xa3, 0xaf, 0xc0, 0x2a, 0x41, 0xcb, 0x2b, 0x5b, + 0xfd, 0x77, 0x5f, 0xc6, 0x14, 0x0d, 0x71, 0x44, 0x43, 0x1c, 0x97, 0xf0, 0xcf, 0xf3, 0x3c, 0x5c, + 0xfa, 0x50, 0x80, 0xa1, 0x3b, 0xb0, 0x47, 0x98, 0xe6, 0x42, 0xb2, 0xd0, 0xf4, 0x9b, 0x84, 0x29, + 0x0e, 0xec, 0x33, 0x68, 0x0e, 0x73, 0x9a, 0x51, 0x16, 0xce, 0x90, 0x0b, 0xcd, 0x4c, 0xaf, 0xe5, + 0xd5, 0xad, 0x7e, 0xa7, 0xf2, 0x02, 0x3a, 0x4e, 0xd7, 0xbe, 0xc9, 0xb3, 0x7f, 0x37, 0xc1, 0x5a, + 0x3b, 0x87, 0x8f, 0x4e, 0x2b, 0xc9, 0xfc, 0x08, 0xd0, 0x3a, 0x27, 0xc8, 0xe8, 0x2c, 0x28, 0x33, + 0x7b, 0x7b, 0xed, 0x19, 0xd2, 0x99, 0x6c, 0x12, 0x1a, 0xc0, 0x7e, 0x39, 0x5a, 0xd3, 0x7b, 0x23, + 0x2a, 0x74, 0x85, 0x56, 0x09, 0xd3, 0xfe, 0x09, 0xf6, 0xdc, 0x35, 0x3f, 0x5b, 0xb6, 0xfb, 0x63, + 0xa8, 0x89, 0x6e, 0xe8, 0x0a, 0xde, 0xbe, 0xaa, 0xc1, 0xfa, 0x64, 0x19, 0x6f, 0x7f, 0x0a, 0xb5, + 0x27, 0x94, 0x63, 0x74, 0x0f, 0x6a, 0x0b, 0xca, 0xb1, 0xe6, 0xb7, 0x32, 0x5f, 0xc4, 0xfa, 0x32, + 0xd2, 0xfe, 0xd5, 0x84, 0x86, 0x17, 0x32, 0x99, 0xbd, 0x5d, 0xad, 0x9f, 0x41, 0x4d, 0xa0, 0xc9, + 0x5a, 0x0f, 0xab, 0x1f, 0xe3, 0x88, 0xc4, 0x29, 0x8e, 0x06, 0x2c, 0x7e, 0xbc, 0xcc, 0xb0, 0x2f, + 0x53, 0x04, 0x20, 0x49, 0x23, 0xfc, 0xb3, 0x7c, 0x74, 0x75, 0x5f, 0x6d, 0xec, 0xbf, 0x4c, 0xd8, + 0x17, 0x75, 0x8c, 0x30, 0x1f, 0x84, 0x3f, 0xf4, 0xef, 0xff, 0x7f, 0xf5, 0x7c, 0x0b, 0x4d, 0x35, + 0x0a, 0x24, 0xd2, 0x73, 0x70, 0x52, 0x95, 0x2e, 0x3b, 0xfb, 0xf0, 0x4b, 0xf7, 0x48, 0xb0, 0xbf, + 0x7a, 0x7e, 0xd2, 0xd0, 0x06, 0xbf, 0x21, 0x11, 0x1e, 0x46, 0xf6, 0x2f, 0xb7, 0xc0, 0xd2, 0xd7, + 0x70, 0x09, 0x67, 0xaf, 0xe6, 0x2d, 0xd0, 0x03, 0xa8, 0x8b, 0xf7, 0xc1, 0xe4, 0x48, 0x6f, 0x37, + 0x0c, 0x2a, 0xd1, 0xfe, 0xa3, 0x0e, 0x8d, 0x01, 0x66, 0x2c, 0x8c, 0x31, 0x1a, 0xc2, 0x61, 0x8a, + 0xcf, 0xd5, 0x18, 0x06, 0x52, 0x89, 0xd5, 0x0b, 0xed, 0x3a, 0xd5, 0xff, 0x28, 0x4e, 0x59, 0xef, + 0x3d, 0xc3, 0xdf, 0x4f, 0xcb, 0xfa, 0x3f, 0x82, 0x23, 0x81, 0xb8, 0x10, 0xc2, 0x1a, 0xc8, 0xa2, + 0x25, 0x8f, 0x56, 0xff, 0x83, 0x6b, 0x20, 0x0b, 0x29, 0xf6, 0x0c, 0xff, 0x20, 0xbd, 0xa4, 0xcd, + 0x65, 0x89, 0xaa, 0x14, 0x81, 0x02, 0x6d, 0xad, 0x44, 0x5e, 0x49, 0xa2, 0xd0, 0xe9, 0x0b, 0x62, + 0xa2, 0x3a, 0xf1, 0xfe, 0x4d, 0x70, 0x86, 0x8f, 0x4e, 0xbd, 0xcb, 0x5a, 0x82, 0xbe, 0x06, 0x28, + 0x44, 0x5a, 0xf7, 0xe2, 0xee, 0x55, 0x58, 0x1b, 0xe5, 0xf1, 0x0c, 0x7f, 0x6f, 0x23, 0xd3, 0x42, + 0x58, 0xa4, 0x30, 0xec, 0x56, 0x09, 0x6f, 0x81, 0x20, 0xde, 0xae, 0x67, 0x28, 0x79, 0x40, 0x0f, + 0xa0, 0x39, 0x0d, 0x59, 0x20, 0x73, 0x1b, 0x32, 0xf7, 0x9d, 0xab, 0x72, 0xb5, 0x92, 0x78, 0x86, + 0xdf, 0x98, 0x6a, 0x51, 0x19, 0xc2, 0xa1, 0xc8, 0x0e, 0x18, 0xe6, 0x41, 0x22, 0xc6, 0xba, 0xd5, + 0xbc, 0xbe, 0xf5, 0x65, 0x19, 0x10, 0xad, 0x5f, 0x94, 0x65, 0x61, 0x00, 0x07, 0x1b, 0x44, 0xf1, + 0xfe, 0x5a, 0x7b, 0xd7, 0x53, 0x5c, 0x1a, 0x48, 0x41, 0xf1, 0xa2, 0xd8, 0xba, 0x75, 0xd8, 0x61, + 0xf3, 0xc4, 0xfd, 0xe6, 0xe9, 0xaa, 0x6d, 0x3e, 0x5b, 0xb5, 0xcd, 0x7f, 0x56, 0x6d, 0xf3, 0xb7, + 0x8b, 0xb6, 0xf1, 0xec, 0xa2, 0x6d, 0xfc, 0x7d, 0xd1, 0x36, 0xbe, 0xbf, 0x17, 0x13, 0x3e, 0x9d, + 0x8f, 0x9d, 0x09, 0x4d, 0x7a, 0xc5, 0x11, 0xe5, 0xe5, 0x0b, 0x9f, 0x4c, 0xe3, 0x5d, 0x69, 0xb8, + 0xff, 0x5f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x8f, 0xb5, 0xc3, 0x75, 0x4c, 0x09, 0x00, 0x00, } func (m *NewRoundStep) Marshal() (dAtA []byte, err error) { @@ -2653,7 +2653,7 @@ func (m *HasVote) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Index |= uint32(b&0x7F) << shift + m.Index |= int32(b&0x7F) << shift if b < 0x80 { break } diff --git a/proto/consensus/msgs.proto b/proto/consensus/msgs.proto index 245e0f4c4..ad35f308a 100644 --- a/proto/consensus/msgs.proto +++ b/proto/consensus/msgs.proto @@ -57,7 +57,7 @@ message HasVote { int64 height = 1; int32 round = 2; tendermint.proto.types.SignedMsgType type = 3; - uint32 index = 4; + int32 index = 4; } // VoteSetMaj23Message is sent to indicate that a given BlockID has seen +2/3 votes.