Browse Source

privval: Switch to amino encoding in SignBytes (#2459)

* switch to amino for SignBytes and add Reply with error message

- currently only Vote is done

* switch Reply type in socket for other messages

 - add error description on error

* add TODOs regarding error handling

* address comments from peer review session (thx @xla)

 - contains all changes besides the test-coverage / error'ing branches

* increase test coverage:

 - add tests for each newly introduced error'ing code path

* return error if received wrong response

* add test for wrong response branches (ErrUnexpectedResponse)

* update CHANGELOG_PENDING and related documentation (spec)

* fix typo: s/CanonicallockID/CanonicalBlockID

* fixes from review
pull/2513/head
Ismail Khoffi 6 years ago
committed by Ethan Buchman
parent
commit
fc073746a0
15 changed files with 492 additions and 219 deletions
  1. +6
    -0
      CHANGELOG_PENDING.md
  2. +12
    -4
      docs/spec/blockchain/blockchain.md
  3. +16
    -8
      docs/spec/blockchain/encoding.md
  4. +12
    -19
      privval/priv_validator.go
  5. +92
    -28
      privval/socket.go
  6. +173
    -12
      privval/socket_test.go
  7. +116
    -0
      types/canonical.go
  8. +0
    -115
      types/canonical_json.go
  9. +1
    -1
      types/heartbeat.go
  10. +19
    -11
      types/heartbeat_test.go
  11. +27
    -0
      types/priv_validator.go
  12. +1
    -1
      types/proposal.go
  13. +5
    -10
      types/proposal_test.go
  14. +8
    -4
      types/vote.go
  15. +4
    -6
      types/vote_test.go

+ 6
- 0
CHANGELOG_PENDING.md View File

@ -13,6 +13,12 @@ BREAKING CHANGES:
- [node] Remove node.RunForever
- [config] \#2232 timeouts as time.Duration, not ints
* Blockchain Protocol
* [types] \#2459 `Vote`/`Proposal`/`Heartbeat` use amino encoding instead of JSON in `SignBytes`.
* [privval] \#2459 Split `SocketPVMsg`s implementations into Request and Response, where the Response may contain a error message (returned by the remote signer).
* P2P Protocol
FEATURES:
IMPROVEMENTS:


+ 12
- 4
docs/spec/blockchain/blockchain.md View File

@ -401,14 +401,22 @@ must be greater than 2/3 of the total voting power of the complete validator set
A vote is a signed message broadcast in the consensus for a particular block at a particular height and round.
When stored in the blockchain or propagated over the network, votes are encoded in Amino.
For signing, votes are encoded in JSON, and the ChainID is included, in the form of the `CanonicalSignBytes`.
For signing, votes are represented via `CanonicalVote` and also encoded using amino (protobuf compatible) via
`Vote.SignBytes` which includes the `ChainID`.
We define a method `Verify` that returns `true` if the signature verifies against the pubkey for the CanonicalSignBytes
We define a method `Verify` that returns `true` if the signature verifies against the pubkey for the `SignBytes`
using the given ChainID:
```go
func (v Vote) Verify(chainID string, pubKey PubKey) bool {
return pubKey.Verify(v.Signature, CanonicalSignBytes(chainID, v))
func (vote *Vote) Verify(chainID string, pubKey crypto.PubKey) error {
if !bytes.Equal(pubKey.Address(), vote.ValidatorAddress) {
return ErrVoteInvalidValidatorAddress
}
if !pubKey.VerifyBytes(vote.SignBytes(chainID), vote.Signature) {
return ErrVoteInvalidSignature
}
return nil
}
```


+ 16
- 8
docs/spec/blockchain/encoding.md View File

@ -298,14 +298,22 @@ Where the `"value"` is the base64 encoding of the raw pubkey bytes, and the
### Signed Messages
Signed messages (eg. votes, proposals) in the consensus are encoded using Amino-JSON, rather than in the standard binary format
(NOTE: this is subject to change: https://github.com/tendermint/tendermint/issues/1622)
Signed messages (eg. votes, proposals) in the consensus are encoded using Amino.
When signing, the elements of a message are sorted by key and prepended with
a `@chain_id` and `@type` field.
We call this encoding the CanonicalSignBytes. For instance, CanonicalSignBytes for a vote would look
like:
When signing, the elements of a message are sorted alphabetically by key and prepended with
a `chain_id` and `type` field.
We call this encoding the SignBytes. For instance, SignBytes for a vote is the Amino encoding of the following struct:
```json
{"@chain_id":"test_chain_id","@type":"vote","block_id":{"hash":"8B01023386C371778ECB6368573E539AFC3CC860","parts":{"hash":"72DB3D959635DFF1BB567BEDAA70573392C51596","total":"1000000"}},"height":"12345","round":"2","timestamp":"2017-12-25T03:00:01.234Z","type":2}
```go
type CanonicalVote struct {
ChainID string
Type string
BlockID CanonicalBlockID
Height int64
Round int
Timestamp time.Time
VoteType byte
}
```
NOTE: see [#1622](https://github.com/tendermint/tendermint/issues/1622) for how field ordering will change

+ 12
- 19
privval/priv_validator.go View File

@ -311,21 +311,18 @@ func (pv *FilePV) String() string {
// returns the timestamp from the lastSignBytes.
// returns true if the only difference in the votes is their timestamp.
func checkVotesOnlyDifferByTimestamp(lastSignBytes, newSignBytes []byte) (time.Time, bool) {
var lastVote, newVote types.CanonicalJSONVote
if err := cdc.UnmarshalJSON(lastSignBytes, &lastVote); err != nil {
var lastVote, newVote types.CanonicalVote
if err := cdc.UnmarshalBinary(lastSignBytes, &lastVote); err != nil {
panic(fmt.Sprintf("LastSignBytes cannot be unmarshalled into vote: %v", err))
}
if err := cdc.UnmarshalJSON(newSignBytes, &newVote); err != nil {
if err := cdc.UnmarshalBinary(newSignBytes, &newVote); err != nil {
panic(fmt.Sprintf("signBytes cannot be unmarshalled into vote: %v", err))
}
lastTime, err := time.Parse(types.TimeFormat, lastVote.Timestamp)
if err != nil {
panic(err)
}
lastTime := lastVote.Timestamp
// set the times to the same value and check equality
now := types.CanonicalTime(tmtime.Now())
now := tmtime.Now()
lastVote.Timestamp = now
newVote.Timestamp = now
lastVoteBytes, _ := cdc.MarshalJSON(lastVote)
@ -337,25 +334,21 @@ func checkVotesOnlyDifferByTimestamp(lastSignBytes, newSignBytes []byte) (time.T
// returns the timestamp from the lastSignBytes.
// returns true if the only difference in the proposals is their timestamp
func checkProposalsOnlyDifferByTimestamp(lastSignBytes, newSignBytes []byte) (time.Time, bool) {
var lastProposal, newProposal types.CanonicalJSONProposal
if err := cdc.UnmarshalJSON(lastSignBytes, &lastProposal); err != nil {
var lastProposal, newProposal types.CanonicalProposal
if err := cdc.UnmarshalBinary(lastSignBytes, &lastProposal); err != nil {
panic(fmt.Sprintf("LastSignBytes cannot be unmarshalled into proposal: %v", err))
}
if err := cdc.UnmarshalJSON(newSignBytes, &newProposal); err != nil {
if err := cdc.UnmarshalBinary(newSignBytes, &newProposal); err != nil {
panic(fmt.Sprintf("signBytes cannot be unmarshalled into proposal: %v", err))
}
lastTime, err := time.Parse(types.TimeFormat, lastProposal.Timestamp)
if err != nil {
panic(err)
}
lastTime := lastProposal.Timestamp
// set the times to the same value and check equality
now := types.CanonicalTime(tmtime.Now())
now := tmtime.Now()
lastProposal.Timestamp = now
newProposal.Timestamp = now
lastProposalBytes, _ := cdc.MarshalJSON(lastProposal)
newProposalBytes, _ := cdc.MarshalJSON(newProposal)
lastProposalBytes, _ := cdc.MarshalBinary(lastProposal)
newProposalBytes, _ := cdc.MarshalBinary(newProposal)
return lastTime, bytes.Equal(newProposalBytes, lastProposalBytes)
}

+ 92
- 28
privval/socket.go View File

@ -7,7 +7,7 @@ import (
"net"
"time"
amino "github.com/tendermint/go-amino"
"github.com/tendermint/go-amino"
"github.com/tendermint/tendermint/crypto"
"github.com/tendermint/tendermint/crypto/ed25519"
@ -27,9 +27,10 @@ const (
// Socket errors.
var (
ErrDialRetryMax = errors.New("dialed maximum retries")
ErrConnWaitTimeout = errors.New("waited for remote signer for too long")
ErrConnTimeout = errors.New("remote signer timed out")
ErrDialRetryMax = errors.New("dialed maximum retries")
ErrConnWaitTimeout = errors.New("waited for remote signer for too long")
ErrConnTimeout = errors.New("remote signer timed out")
ErrUnexpectedResponse = errors.New("received unexpected response")
)
var (
@ -150,7 +151,7 @@ func (sc *SocketPV) getPubKey() (crypto.PubKey, error) {
// SignVote implements PrivValidator.
func (sc *SocketPV) SignVote(chainID string, vote *types.Vote) error {
err := writeMsg(sc.conn, &SignVoteMsg{Vote: vote})
err := writeMsg(sc.conn, &SignVoteRequest{Vote: vote})
if err != nil {
return err
}
@ -160,7 +161,16 @@ func (sc *SocketPV) SignVote(chainID string, vote *types.Vote) error {
return err
}
*vote = *res.(*SignVoteMsg).Vote
resp, ok := res.(*SignedVoteResponse)
if !ok {
return ErrUnexpectedResponse
}
if resp.Error != nil {
return fmt.Errorf("remote error occurred: code: %v, description: %s",
resp.Error.Code,
resp.Error.Description)
}
*vote = *resp.Vote
return nil
}
@ -170,7 +180,7 @@ func (sc *SocketPV) SignProposal(
chainID string,
proposal *types.Proposal,
) error {
err := writeMsg(sc.conn, &SignProposalMsg{Proposal: proposal})
err := writeMsg(sc.conn, &SignProposalRequest{Proposal: proposal})
if err != nil {
return err
}
@ -179,8 +189,16 @@ func (sc *SocketPV) SignProposal(
if err != nil {
return err
}
*proposal = *res.(*SignProposalMsg).Proposal
resp, ok := res.(*SignedProposalResponse)
if !ok {
return ErrUnexpectedResponse
}
if resp.Error != nil {
return fmt.Errorf("remote error occurred: code: %v, description: %s",
resp.Error.Code,
resp.Error.Description)
}
*proposal = *resp.Proposal
return nil
}
@ -190,7 +208,7 @@ func (sc *SocketPV) SignHeartbeat(
chainID string,
heartbeat *types.Heartbeat,
) error {
err := writeMsg(sc.conn, &SignHeartbeatMsg{Heartbeat: heartbeat})
err := writeMsg(sc.conn, &SignHeartbeatRequest{Heartbeat: heartbeat})
if err != nil {
return err
}
@ -199,8 +217,16 @@ func (sc *SocketPV) SignHeartbeat(
if err != nil {
return err
}
*heartbeat = *res.(*SignHeartbeatMsg).Heartbeat
resp, ok := res.(*SignedHeartbeatResponse)
if !ok {
return ErrUnexpectedResponse
}
if resp.Error != nil {
return fmt.Errorf("remote error occurred: code: %v, description: %s",
resp.Error.Code,
resp.Error.Description)
}
*heartbeat = *resp.Heartbeat
return nil
}
@ -462,22 +488,34 @@ func (rs *RemoteSigner) handleConnection(conn net.Conn) {
var p crypto.PubKey
p = rs.privVal.GetPubKey()
res = &PubKeyMsg{p}
case *SignVoteMsg:
case *SignVoteRequest:
err = rs.privVal.SignVote(rs.chainID, r.Vote)
res = &SignVoteMsg{r.Vote}
case *SignProposalMsg:
if err != nil {
res = &SignedVoteResponse{nil, &RemoteSignerError{0, err.Error()}}
} else {
res = &SignedVoteResponse{r.Vote, nil}
}
case *SignProposalRequest:
err = rs.privVal.SignProposal(rs.chainID, r.Proposal)
res = &SignProposalMsg{r.Proposal}
case *SignHeartbeatMsg:
if err != nil {
res = &SignedProposalResponse{nil, &RemoteSignerError{0, err.Error()}}
} else {
res = &SignedProposalResponse{r.Proposal, nil}
}
case *SignHeartbeatRequest:
err = rs.privVal.SignHeartbeat(rs.chainID, r.Heartbeat)
res = &SignHeartbeatMsg{r.Heartbeat}
if err != nil {
res = &SignedHeartbeatResponse{nil, &RemoteSignerError{0, err.Error()}}
} else {
res = &SignedHeartbeatResponse{r.Heartbeat, nil}
}
default:
err = fmt.Errorf("unknown msg: %v", r)
}
if err != nil {
// only log the error; we'll reply with an error in res
rs.Logger.Error("handleConnection", "err", err)
return
}
err = writeMsg(conn, res)
@ -496,9 +534,12 @@ type SocketPVMsg interface{}
func RegisterSocketPVMsg(cdc *amino.Codec) {
cdc.RegisterInterface((*SocketPVMsg)(nil), nil)
cdc.RegisterConcrete(&PubKeyMsg{}, "tendermint/socketpv/PubKeyMsg", nil)
cdc.RegisterConcrete(&SignVoteMsg{}, "tendermint/socketpv/SignVoteMsg", nil)
cdc.RegisterConcrete(&SignProposalMsg{}, "tendermint/socketpv/SignProposalMsg", nil)
cdc.RegisterConcrete(&SignHeartbeatMsg{}, "tendermint/socketpv/SignHeartbeatMsg", nil)
cdc.RegisterConcrete(&SignVoteRequest{}, "tendermint/socketpv/SignVoteRequest", nil)
cdc.RegisterConcrete(&SignedVoteResponse{}, "tendermint/socketpv/SignedVoteResponse", nil)
cdc.RegisterConcrete(&SignProposalRequest{}, "tendermint/socketpv/SignProposalRequest", nil)
cdc.RegisterConcrete(&SignedProposalResponse{}, "tendermint/socketpv/SignedProposalResponse", nil)
cdc.RegisterConcrete(&SignHeartbeatRequest{}, "tendermint/socketpv/SignHeartbeatRequest", nil)
cdc.RegisterConcrete(&SignedHeartbeatResponse{}, "tendermint/socketpv/SignedHeartbeatResponse", nil)
}
// PubKeyMsg is a PrivValidatorSocket message containing the public key.
@ -506,21 +547,44 @@ type PubKeyMsg struct {
PubKey crypto.PubKey
}
// SignVoteMsg is a PrivValidatorSocket message containing a vote.
type SignVoteMsg struct {
// SignVoteRequest is a PrivValidatorSocket message containing a vote.
type SignVoteRequest struct {
Vote *types.Vote
}
// SignProposalMsg is a PrivValidatorSocket message containing a Proposal.
type SignProposalMsg struct {
// SignedVoteResponse is a PrivValidatorSocket message containing a signed vote along with a potenial error message.
type SignedVoteResponse struct {
Vote *types.Vote
Error *RemoteSignerError
}
// SignProposalRequest is a PrivValidatorSocket message containing a Proposal.
type SignProposalRequest struct {
Proposal *types.Proposal
}
type SignedProposalResponse struct {
Proposal *types.Proposal
Error *RemoteSignerError
}
// SignHeartbeatMsg is a PrivValidatorSocket message containing a Heartbeat.
type SignHeartbeatMsg struct {
// SignHeartbeatRequest is a PrivValidatorSocket message containing a Heartbeat.
type SignHeartbeatRequest struct {
Heartbeat *types.Heartbeat
}
type SignedHeartbeatResponse struct {
Heartbeat *types.Heartbeat
Error *RemoteSignerError
}
// RemoteSignerError allows (remote) validators to include meaningful error descriptions in their reply.
type RemoteSignerError struct {
// TODO(ismail): create an enum of known errors
Code int
Description string
}
func readMsg(r io.Reader) (msg SocketPVMsg, err error) {
const maxSocketPVMsgSize = 1024 * 10
_, err = cdc.UnmarshalBinaryReader(r, &msg, maxSocketPVMsgSize)


+ 173
- 12
privval/socket_test.go View File

@ -20,7 +20,7 @@ import (
func TestSocketPVAddress(t *testing.T) {
var (
chainID = cmn.RandStr(12)
sc, rs = testSetupSocketPair(t, chainID)
sc, rs = testSetupSocketPair(t, chainID, types.NewMockPV())
)
defer sc.Stop()
defer rs.Stop()
@ -40,7 +40,7 @@ func TestSocketPVAddress(t *testing.T) {
func TestSocketPVPubKey(t *testing.T) {
var (
chainID = cmn.RandStr(12)
sc, rs = testSetupSocketPair(t, chainID)
sc, rs = testSetupSocketPair(t, chainID, types.NewMockPV())
)
defer sc.Stop()
defer rs.Stop()
@ -59,7 +59,7 @@ func TestSocketPVPubKey(t *testing.T) {
func TestSocketPVProposal(t *testing.T) {
var (
chainID = cmn.RandStr(12)
sc, rs = testSetupSocketPair(t, chainID)
sc, rs = testSetupSocketPair(t, chainID, types.NewMockPV())
ts = time.Now()
privProposal = &types.Proposal{Timestamp: ts}
@ -76,7 +76,7 @@ func TestSocketPVProposal(t *testing.T) {
func TestSocketPVVote(t *testing.T) {
var (
chainID = cmn.RandStr(12)
sc, rs = testSetupSocketPair(t, chainID)
sc, rs = testSetupSocketPair(t, chainID, types.NewMockPV())
ts = time.Now()
vType = types.VoteTypePrecommit
@ -94,7 +94,7 @@ func TestSocketPVVote(t *testing.T) {
func TestSocketPVHeartbeat(t *testing.T) {
var (
chainID = cmn.RandStr(12)
sc, rs = testSetupSocketPair(t, chainID)
sc, rs = testSetupSocketPair(t, chainID, types.NewMockPV())
want = &types.Heartbeat{}
have = &types.Heartbeat{}
@ -231,14 +231,163 @@ func TestRemoteSignerRetry(t *testing.T) {
}
}
func TestRemoteSignVoteErrors(t *testing.T) {
var (
chainID = cmn.RandStr(12)
sc, rs = testSetupSocketPair(t, chainID, types.NewErroringMockPV())
ts = time.Now()
vType = types.VoteTypePrecommit
vote = &types.Vote{Timestamp: ts, Type: vType}
)
defer sc.Stop()
defer rs.Stop()
err := writeMsg(sc.conn, &SignVoteRequest{Vote: vote})
require.NoError(t, err)
res, err := readMsg(sc.conn)
require.NoError(t, err)
resp := *res.(*SignedVoteResponse)
require.NotNil(t, resp.Error)
require.Equal(t, resp.Error.Description, types.ErroringMockPVErr.Error())
err = rs.privVal.SignVote(chainID, vote)
require.Error(t, err)
err = sc.SignVote(chainID, vote)
require.Error(t, err)
}
func TestRemoteSignProposalErrors(t *testing.T) {
var (
chainID = cmn.RandStr(12)
sc, rs = testSetupSocketPair(t, chainID, types.NewErroringMockPV())
ts = time.Now()
proposal = &types.Proposal{Timestamp: ts}
)
defer sc.Stop()
defer rs.Stop()
err := writeMsg(sc.conn, &SignProposalRequest{Proposal: proposal})
require.NoError(t, err)
res, err := readMsg(sc.conn)
require.NoError(t, err)
resp := *res.(*SignedProposalResponse)
require.NotNil(t, resp.Error)
require.Equal(t, resp.Error.Description, types.ErroringMockPVErr.Error())
err = rs.privVal.SignProposal(chainID, proposal)
require.Error(t, err)
err = sc.SignProposal(chainID, proposal)
require.Error(t, err)
}
func TestRemoteSignHeartbeatErrors(t *testing.T) {
var (
chainID = cmn.RandStr(12)
sc, rs = testSetupSocketPair(t, chainID, types.NewErroringMockPV())
hb = &types.Heartbeat{}
)
defer sc.Stop()
defer rs.Stop()
err := writeMsg(sc.conn, &SignHeartbeatRequest{Heartbeat: hb})
require.NoError(t, err)
res, err := readMsg(sc.conn)
require.NoError(t, err)
resp := *res.(*SignedHeartbeatResponse)
require.NotNil(t, resp.Error)
require.Equal(t, resp.Error.Description, types.ErroringMockPVErr.Error())
err = rs.privVal.SignHeartbeat(chainID, hb)
require.Error(t, err)
err = sc.SignHeartbeat(chainID, hb)
require.Error(t, err)
}
func TestErrUnexpectedResponse(t *testing.T) {
var (
addr = testFreeAddr(t)
logger = log.TestingLogger()
chainID = cmn.RandStr(12)
readyc = make(chan struct{})
errc = make(chan error, 1)
rs = NewRemoteSigner(
logger,
chainID,
addr,
types.NewMockPV(),
ed25519.GenPrivKey(),
)
sc = NewSocketPV(
logger,
addr,
ed25519.GenPrivKey(),
)
)
testStartSocketPV(t, readyc, sc)
defer sc.Stop()
RemoteSignerConnDeadline(time.Millisecond)(rs)
RemoteSignerConnRetries(1e6)(rs)
// we do not want to Start() the remote signer here and instead use the connection to
// reply with intentionally wrong replies below:
rsConn, err := rs.connect()
defer rsConn.Close()
require.NoError(t, err)
require.NotNil(t, rsConn)
<-readyc
// Heartbeat:
go func(errc chan error) {
errc <- sc.SignHeartbeat(chainID, &types.Heartbeat{})
}(errc)
// read request and write wrong response:
go testReadWriteResponse(t, &SignedVoteResponse{}, rsConn)
err = <-errc
require.Error(t, err)
require.Equal(t, err, ErrUnexpectedResponse)
// Proposal:
go func(errc chan error) {
errc <- sc.SignProposal(chainID, &types.Proposal{})
}(errc)
// read request and write wrong response:
go testReadWriteResponse(t, &SignedHeartbeatResponse{}, rsConn)
err = <-errc
require.Error(t, err)
require.Equal(t, err, ErrUnexpectedResponse)
// Vote:
go func(errc chan error) {
errc <- sc.SignVote(chainID, &types.Vote{})
}(errc)
// read request and write wrong response:
go testReadWriteResponse(t, &SignedHeartbeatResponse{}, rsConn)
err = <-errc
require.Error(t, err)
require.Equal(t, err, ErrUnexpectedResponse)
}
func testSetupSocketPair(
t *testing.T,
chainID string,
privValidator types.PrivValidator,
) (*SocketPV, *RemoteSigner) {
var (
addr = testFreeAddr(t)
logger = log.TestingLogger()
privVal = types.NewMockPV()
privVal = privValidator
readyc = make(chan struct{})
rs = NewRemoteSigner(
logger,
@ -254,12 +403,7 @@ func testSetupSocketPair(
)
)
go func(sc *SocketPV) {
require.NoError(t, sc.Start())
assert.True(t, sc.IsRunning())
readyc <- struct{}{}
}(sc)
testStartSocketPV(t, readyc, sc)
RemoteSignerConnDeadline(time.Millisecond)(rs)
RemoteSignerConnRetries(1e6)(rs)
@ -272,6 +416,23 @@ func testSetupSocketPair(
return sc, rs
}
func testReadWriteResponse(t *testing.T, resp SocketPVMsg, rsConn net.Conn) {
_, err := readMsg(rsConn)
require.NoError(t, err)
err = writeMsg(rsConn, resp)
require.NoError(t, err)
}
func testStartSocketPV(t *testing.T, readyc chan struct{}, sc *SocketPV) {
go func(sc *SocketPV) {
require.NoError(t, sc.Start())
assert.True(t, sc.IsRunning())
readyc <- struct{}{}
}(sc)
}
// testFreeAddr claims a free port so we don't block on listener being ready.
func testFreeAddr(t *testing.T) string {
ln, err := net.Listen("tcp", "127.0.0.1:0")


+ 116
- 0
types/canonical.go View File

@ -0,0 +1,116 @@
package types
import (
"time"
cmn "github.com/tendermint/tendermint/libs/common"
tmtime "github.com/tendermint/tendermint/types/time"
)
// Canonical* wraps the structs in types for amino encoding them for use in SignBytes / the Signable interface.
// TimeFormat is used for generating the sigs
const TimeFormat = time.RFC3339Nano
type CanonicalBlockID struct {
Hash cmn.HexBytes `json:"hash,omitempty"`
PartsHeader CanonicalPartSetHeader `json:"parts,omitempty"`
}
type CanonicalPartSetHeader struct {
Hash cmn.HexBytes `json:"hash,omitempty"`
Total int `json:"total,omitempty"`
}
type CanonicalProposal struct {
ChainID string `json:"@chain_id"`
Type string `json:"@type"`
BlockPartsHeader CanonicalPartSetHeader `json:"block_parts_header"`
Height int64 `json:"height"`
POLBlockID CanonicalBlockID `json:"pol_block_id"`
POLRound int `json:"pol_round"`
Round int `json:"round"`
Timestamp time.Time `json:"timestamp"`
}
type CanonicalVote struct {
ChainID string `json:"@chain_id"`
Type string `json:"@type"`
BlockID CanonicalBlockID `json:"block_id"`
Height int64 `json:"height"`
Round int `json:"round"`
Timestamp time.Time `json:"timestamp"`
VoteType byte `json:"type"`
}
type CanonicalHeartbeat struct {
ChainID string `json:"@chain_id"`
Type string `json:"@type"`
Height int64 `json:"height"`
Round int `json:"round"`
Sequence int `json:"sequence"`
ValidatorAddress Address `json:"validator_address"`
ValidatorIndex int `json:"validator_index"`
}
//-----------------------------------
// Canonicalize the structs
func CanonicalizeBlockID(blockID BlockID) CanonicalBlockID {
return CanonicalBlockID{
Hash: blockID.Hash,
PartsHeader: CanonicalizePartSetHeader(blockID.PartsHeader),
}
}
func CanonicalizePartSetHeader(psh PartSetHeader) CanonicalPartSetHeader {
return CanonicalPartSetHeader{
psh.Hash,
psh.Total,
}
}
func CanonicalizeProposal(chainID string, proposal *Proposal) CanonicalProposal {
return CanonicalProposal{
ChainID: chainID,
Type: "proposal",
BlockPartsHeader: CanonicalizePartSetHeader(proposal.BlockPartsHeader),
Height: proposal.Height,
Timestamp: proposal.Timestamp,
POLBlockID: CanonicalizeBlockID(proposal.POLBlockID),
POLRound: proposal.POLRound,
Round: proposal.Round,
}
}
func CanonicalizeVote(chainID string, vote *Vote) CanonicalVote {
return CanonicalVote{
ChainID: chainID,
Type: "vote",
BlockID: CanonicalizeBlockID(vote.BlockID),
Height: vote.Height,
Round: vote.Round,
Timestamp: vote.Timestamp,
VoteType: vote.Type,
}
}
func CanonicalizeHeartbeat(chainID string, heartbeat *Heartbeat) CanonicalHeartbeat {
return CanonicalHeartbeat{
ChainID: chainID,
Type: "heartbeat",
Height: heartbeat.Height,
Round: heartbeat.Round,
Sequence: heartbeat.Sequence,
ValidatorAddress: heartbeat.ValidatorAddress,
ValidatorIndex: heartbeat.ValidatorIndex,
}
}
// CanonicalTime can be used to stringify time in a canonical way.
func CanonicalTime(t time.Time) string {
// Note that sending time over amino resets it to
// local time, we need to force UTC here, so the
// signatures match
return tmtime.Canonical(t).Format(TimeFormat)
}

+ 0
- 115
types/canonical_json.go View File

@ -1,115 +0,0 @@
package types
import (
"time"
cmn "github.com/tendermint/tendermint/libs/common"
tmtime "github.com/tendermint/tendermint/types/time"
)
// Canonical json is amino's json for structs with fields in alphabetical order
// TimeFormat is used for generating the sigs
const TimeFormat = time.RFC3339Nano
type CanonicalJSONBlockID struct {
Hash cmn.HexBytes `json:"hash,omitempty"`
PartsHeader CanonicalJSONPartSetHeader `json:"parts,omitempty"`
}
type CanonicalJSONPartSetHeader struct {
Hash cmn.HexBytes `json:"hash,omitempty"`
Total int `json:"total,omitempty"`
}
type CanonicalJSONProposal struct {
ChainID string `json:"@chain_id"`
Type string `json:"@type"`
BlockPartsHeader CanonicalJSONPartSetHeader `json:"block_parts_header"`
Height int64 `json:"height"`
POLBlockID CanonicalJSONBlockID `json:"pol_block_id"`
POLRound int `json:"pol_round"`
Round int `json:"round"`
Timestamp string `json:"timestamp"`
}
type CanonicalJSONVote struct {
ChainID string `json:"@chain_id"`
Type string `json:"@type"`
BlockID CanonicalJSONBlockID `json:"block_id"`
Height int64 `json:"height"`
Round int `json:"round"`
Timestamp string `json:"timestamp"`
VoteType byte `json:"type"`
}
type CanonicalJSONHeartbeat struct {
ChainID string `json:"@chain_id"`
Type string `json:"@type"`
Height int64 `json:"height"`
Round int `json:"round"`
Sequence int `json:"sequence"`
ValidatorAddress Address `json:"validator_address"`
ValidatorIndex int `json:"validator_index"`
}
//-----------------------------------
// Canonicalize the structs
func CanonicalBlockID(blockID BlockID) CanonicalJSONBlockID {
return CanonicalJSONBlockID{
Hash: blockID.Hash,
PartsHeader: CanonicalPartSetHeader(blockID.PartsHeader),
}
}
func CanonicalPartSetHeader(psh PartSetHeader) CanonicalJSONPartSetHeader {
return CanonicalJSONPartSetHeader{
psh.Hash,
psh.Total,
}
}
func CanonicalProposal(chainID string, proposal *Proposal) CanonicalJSONProposal {
return CanonicalJSONProposal{
ChainID: chainID,
Type: "proposal",
BlockPartsHeader: CanonicalPartSetHeader(proposal.BlockPartsHeader),
Height: proposal.Height,
Timestamp: CanonicalTime(proposal.Timestamp),
POLBlockID: CanonicalBlockID(proposal.POLBlockID),
POLRound: proposal.POLRound,
Round: proposal.Round,
}
}
func CanonicalVote(chainID string, vote *Vote) CanonicalJSONVote {
return CanonicalJSONVote{
ChainID: chainID,
Type: "vote",
BlockID: CanonicalBlockID(vote.BlockID),
Height: vote.Height,
Round: vote.Round,
Timestamp: CanonicalTime(vote.Timestamp),
VoteType: vote.Type,
}
}
func CanonicalHeartbeat(chainID string, heartbeat *Heartbeat) CanonicalJSONHeartbeat {
return CanonicalJSONHeartbeat{
ChainID: chainID,
Type: "heartbeat",
Height: heartbeat.Height,
Round: heartbeat.Round,
Sequence: heartbeat.Sequence,
ValidatorAddress: heartbeat.ValidatorAddress,
ValidatorIndex: heartbeat.ValidatorIndex,
}
}
func CanonicalTime(t time.Time) string {
// Note that sending time over amino resets it to
// local time, we need to force UTC here, so the
// signatures match
return tmtime.Canonical(t).Format(TimeFormat)
}

+ 1
- 1
types/heartbeat.go View File

@ -23,7 +23,7 @@ type Heartbeat struct {
// SignBytes returns the Heartbeat bytes for signing.
// It panics if the Heartbeat is nil.
func (heartbeat *Heartbeat) SignBytes(chainID string) []byte {
bz, err := cdc.MarshalJSON(CanonicalHeartbeat(chainID, heartbeat))
bz, err := cdc.MarshalBinary(CanonicalizeHeartbeat(chainID, heartbeat))
if err != nil {
panic(err)
}


+ 19
- 11
types/heartbeat_test.go View File

@ -34,19 +34,27 @@ func TestHeartbeatString(t *testing.T) {
}
func TestHeartbeatWriteSignBytes(t *testing.T) {
hb := &Heartbeat{ValidatorIndex: 1, Height: 10, Round: 1}
bz := hb.SignBytes("0xdeadbeef")
// XXX HMMMMMMM
require.Equal(t, string(bz), `{"@chain_id":"0xdeadbeef","@type":"heartbeat","height":"10","round":"1","sequence":"0","validator_address":"","validator_index":"1"}`)
plainHb := &Heartbeat{}
bz = plainHb.SignBytes("0xdeadbeef")
require.Equal(t, string(bz), `{"@chain_id":"0xdeadbeef","@type":"heartbeat","height":"0","round":"0","sequence":"0","validator_address":"","validator_index":"0"}`)
chainID := "test_chain_id"
{
testHeartbeat := &Heartbeat{ValidatorIndex: 1, Height: 10, Round: 1}
signBytes := testHeartbeat.SignBytes(chainID)
expected, err := cdc.MarshalBinary(CanonicalizeHeartbeat(chainID, testHeartbeat))
require.NoError(t, err)
require.Equal(t, expected, signBytes, "Got unexpected sign bytes for Heartbeat")
}
{
testHeartbeat := &Heartbeat{}
signBytes := testHeartbeat.SignBytes(chainID)
expected, err := cdc.MarshalBinary(CanonicalizeHeartbeat(chainID, testHeartbeat))
require.NoError(t, err)
require.Equal(t, expected, signBytes, "Got unexpected sign bytes for Heartbeat")
}
require.Panics(t, func() {
var nilHb *Heartbeat
bz := nilHb.SignBytes("0xdeadbeef")
require.Equal(t, string(bz), "null")
signBytes := nilHb.SignBytes(chainID)
require.Equal(t, string(signBytes), "null")
})
}

+ 27
- 0
types/priv_validator.go View File

@ -2,6 +2,7 @@ package types
import (
"bytes"
"errors"
"fmt"
"github.com/tendermint/tendermint/crypto"
@ -103,3 +104,29 @@ func (pv *MockPV) DisableChecks() {
// Currently this does nothing,
// as MockPV has no safety checks at all.
}
type erroringMockPV struct {
*MockPV
}
var ErroringMockPVErr = errors.New("erroringMockPV always returns an error")
// Implements PrivValidator.
func (pv *erroringMockPV) SignVote(chainID string, vote *Vote) error {
return ErroringMockPVErr
}
// Implements PrivValidator.
func (pv *erroringMockPV) SignProposal(chainID string, proposal *Proposal) error {
return ErroringMockPVErr
}
// signHeartbeat signs the heartbeat without any checking.
func (pv *erroringMockPV) SignHeartbeat(chainID string, heartbeat *Heartbeat) error {
return ErroringMockPVErr
}
// NewErroringMockPV returns a MockPV that fails on each signing request. Again, for testing only.
func NewErroringMockPV() *erroringMockPV {
return &erroringMockPV{&MockPV{ed25519.GenPrivKey()}}
}

+ 1
- 1
types/proposal.go View File

@ -52,7 +52,7 @@ func (p *Proposal) String() string {
// SignBytes returns the Proposal bytes for signing
func (p *Proposal) SignBytes(chainID string) []byte {
bz, err := cdc.MarshalJSON(CanonicalProposal(chainID, p))
bz, err := cdc.MarshalBinary(CanonicalizeProposal(chainID, p))
if err != nil {
panic(err)
}


+ 5
- 10
types/proposal_test.go View File

@ -24,17 +24,12 @@ func init() {
}
func TestProposalSignable(t *testing.T) {
signBytes := testProposal.SignBytes("test_chain_id")
signStr := string(signBytes)
chainID := "test_chain_id"
signBytes := testProposal.SignBytes(chainID)
expected := `{"@chain_id":"test_chain_id","@type":"proposal","block_parts_header":{"hash":"626C6F636B7061727473","total":"111"},"height":"12345","pol_block_id":{},"pol_round":"-1","round":"23456","timestamp":"2018-02-11T07:09:22.765Z"}`
if signStr != expected {
t.Errorf("Got unexpected sign string for Proposal. Expected:\n%v\nGot:\n%v", expected, signStr)
}
if signStr != expected {
t.Errorf("Got unexpected sign string for Proposal. Expected:\n%v\nGot:\n%v", expected, signStr)
}
expected, err := cdc.MarshalBinary(CanonicalizeProposal(chainID, testProposal))
require.NoError(t, err)
require.Equal(t, expected, signBytes, "Got unexpected sign bytes for Proposal")
}
func TestProposalString(t *testing.T) {


+ 8
- 4
types/vote.go View File

@ -6,7 +6,7 @@ import (
"fmt"
"time"
crypto "github.com/tendermint/tendermint/crypto"
"github.com/tendermint/tendermint/crypto"
cmn "github.com/tendermint/tendermint/libs/common"
)
@ -77,7 +77,7 @@ type Vote struct {
}
func (vote *Vote) SignBytes(chainID string) []byte {
bz, err := cdc.MarshalJSON(CanonicalVote(chainID, vote))
bz, err := cdc.MarshalBinary(CanonicalizeVote(chainID, vote))
if err != nil {
panic(err)
}
@ -104,8 +104,12 @@ func (vote *Vote) String() string {
}
return fmt.Sprintf("Vote{%v:%X %v/%02d/%v(%v) %X %X @ %s}",
vote.ValidatorIndex, cmn.Fingerprint(vote.ValidatorAddress),
vote.Height, vote.Round, vote.Type, typeString,
vote.ValidatorIndex,
cmn.Fingerprint(vote.ValidatorAddress),
vote.Height,
vote.Round,
vote.Type,
typeString,
cmn.Fingerprint(vote.BlockID.Hash),
cmn.Fingerprint(vote.Signature),
CanonicalTime(vote.Timestamp))


+ 4
- 6
types/vote_test.go View File

@ -46,13 +46,11 @@ func exampleVote(t byte) *Vote {
func TestVoteSignable(t *testing.T) {
vote := examplePrecommit()
signBytes := vote.SignBytes("test_chain_id")
signStr := string(signBytes)
expected := `{"@chain_id":"test_chain_id","@type":"vote","block_id":{"hash":"8B01023386C371778ECB6368573E539AFC3CC860","parts":{"hash":"72DB3D959635DFF1BB567BEDAA70573392C51596","total":"1000000"}},"height":"12345","round":"2","timestamp":"2017-12-25T03:00:01.234Z","type":2}`
if signStr != expected {
// NOTE: when this fails, you probably want to fix up consensus/replay_test too
t.Errorf("Got unexpected sign string for Vote. Expected:\n%v\nGot:\n%v", expected, signStr)
}
expected, err := cdc.MarshalBinary(CanonicalizeVote("test_chain_id", vote))
require.NoError(t, err)
require.Equal(t, expected, signBytes, "Got unexpected sign bytes for Vote.")
}
func TestVoteVerifySignature(t *testing.T) {


Loading…
Cancel
Save