Browse Source

Remove privval.GetAddress(), memoize pubkey (#2948)

privval: remove GetAddress(), memoize pubkey
pull/3075/head
Ismail Khoffi 6 years ago
committed by Ethan Buchman
parent
commit
6a80412a01
20 changed files with 194 additions and 143 deletions
  1. +7
    -2
      CHANGELOG_PENDING.md
  2. +1
    -1
      blockchain/reactor_test.go
  3. +3
    -2
      cmd/tendermint/commands/init.go
  4. +8
    -4
      consensus/common_test.go
  5. +6
    -3
      consensus/reactor_test.go
  6. +0
    -6
      consensus/replay_test.go
  7. +13
    -10
      consensus/state.go
  8. +28
    -14
      consensus/state_test.go
  9. +2
    -1
      consensus/types/height_vote_set_test.go
  10. +9
    -5
      node/node.go
  11. +4
    -1
      privval/ipc.go
  12. +39
    -32
      privval/remote_signer.go
  13. +4
    -2
      privval/tcp.go
  14. +9
    -19
      privval/tcp_test.go
  15. +2
    -1
      types/evidence_test.go
  16. +3
    -8
      types/priv_validator.go
  17. +3
    -2
      types/protobuf_test.go
  18. +2
    -2
      types/test_util.go
  19. +2
    -1
      types/validator.go
  20. +49
    -27
      types/vote_set_test.go

+ 7
- 2
CHANGELOG_PENDING.md View File

@ -11,9 +11,14 @@ Special thanks to external contributors on this release:
- [cli] Renamed `node` `--proxy_app=nilapp` to `--proxy_app=noop`.
- [config] \#2992 `allow_duplicate_ip` is now set to false
- [privval] \#2926 split up `PubKeyMsg` into `PubKeyRequest` and `PubKeyResponse` to be consistent with other message types
* Apps
* Go API
* Go API
- [types] \#2926 memoize consensus public key on initialization of remote signer and return the memoized key on
`PrivValidator.GetPubKey()` instead of requesting it again
- [types] \#2981 Remove `PrivValidator.GetAddress()`
* Blockchain Protocol
@ -26,4 +31,4 @@ Special thanks to external contributors on this release:
### IMPROVEMENTS:
### BUG FIXES:
- [types] \#2926 do not panic if retrieving the private validator's public key fails

+ 1
- 1
blockchain/reactor_test.go View File

@ -42,7 +42,7 @@ func randGenesisDoc(numValidators int, randPower bool, minPower int64) (*types.G
}
func makeVote(header *types.Header, blockID types.BlockID, valset *types.ValidatorSet, privVal types.PrivValidator) *types.Vote {
addr := privVal.GetAddress()
addr := privVal.GetPubKey().Address()
idx, _ := valset.GetByAddress(addr)
vote := &types.Vote{
ValidatorAddress: addr,


+ 3
- 2
cmd/tendermint/commands/init.go View File

@ -59,9 +59,10 @@ func initFilesWithConfig(config *cfg.Config) error {
GenesisTime: tmtime.Now(),
ConsensusParams: types.DefaultConsensusParams(),
}
key := pv.GetPubKey()
genDoc.Validators = []types.GenesisValidator{{
Address: pv.GetPubKey().Address(),
PubKey: pv.GetPubKey(),
Address: key.Address(),
PubKey: key,
Power: 10,
}}


+ 8
- 4
consensus/common_test.go View File

@ -71,9 +71,10 @@ func NewValidatorStub(privValidator types.PrivValidator, valIndex int) *validato
}
func (vs *validatorStub) signVote(voteType types.SignedMsgType, hash []byte, header types.PartSetHeader) (*types.Vote, error) {
addr := vs.PrivValidator.GetPubKey().Address()
vote := &types.Vote{
ValidatorIndex: vs.Index,
ValidatorAddress: vs.PrivValidator.GetAddress(),
ValidatorAddress: addr,
Height: vs.Height,
Round: vs.Round,
Timestamp: tmtime.Now(),
@ -150,8 +151,9 @@ func signAddVotes(to *ConsensusState, voteType types.SignedMsgType, hash []byte,
func validatePrevote(t *testing.T, cs *ConsensusState, round int, privVal *validatorStub, blockHash []byte) {
prevotes := cs.Votes.Prevotes(round)
address := privVal.GetPubKey().Address()
var vote *types.Vote
if vote = prevotes.GetByAddress(privVal.GetAddress()); vote == nil {
if vote = prevotes.GetByAddress(address); vote == nil {
panic("Failed to find prevote from validator")
}
if blockHash == nil {
@ -167,8 +169,9 @@ func validatePrevote(t *testing.T, cs *ConsensusState, round int, privVal *valid
func validateLastPrecommit(t *testing.T, cs *ConsensusState, privVal *validatorStub, blockHash []byte) {
votes := cs.LastCommit
address := privVal.GetPubKey().Address()
var vote *types.Vote
if vote = votes.GetByAddress(privVal.GetAddress()); vote == nil {
if vote = votes.GetByAddress(address); vote == nil {
panic("Failed to find precommit from validator")
}
if !bytes.Equal(vote.BlockID.Hash, blockHash) {
@ -178,8 +181,9 @@ func validateLastPrecommit(t *testing.T, cs *ConsensusState, privVal *validatorS
func validatePrecommit(t *testing.T, cs *ConsensusState, thisRound, lockRound int, privVal *validatorStub, votedBlockHash, lockedBlockHash []byte) {
precommits := cs.Votes.Precommits(thisRound)
address := privVal.GetPubKey().Address()
var vote *types.Vote
if vote = precommits.GetByAddress(privVal.GetAddress()); vote == nil {
if vote = precommits.GetByAddress(address); vote == nil {
panic("Failed to find precommit from validator")
}


+ 6
- 3
consensus/reactor_test.go View File

@ -143,7 +143,8 @@ func TestReactorWithEvidence(t *testing.T) {
// mock the evidence pool
// everyone includes evidence of another double signing
vIdx := (i + 1) % nValidators
evpool := newMockEvidencePool(privVals[vIdx].GetAddress())
addr := privVals[vIdx].GetPubKey().Address()
evpool := newMockEvidencePool(addr)
// Make ConsensusState
blockExec := sm.NewBlockExecutor(stateDB, log.TestingLogger(), proxyAppConnCon, mempool, evpool)
@ -268,7 +269,8 @@ func TestReactorVotingPowerChange(t *testing.T) {
// map of active validators
activeVals := make(map[string]struct{})
for i := 0; i < nVals; i++ {
activeVals[string(css[i].privValidator.GetAddress())] = struct{}{}
addr := css[i].privValidator.GetPubKey().Address()
activeVals[string(addr)] = struct{}{}
}
// wait till everyone makes block 1
@ -331,7 +333,8 @@ func TestReactorValidatorSetChanges(t *testing.T) {
// map of active validators
activeVals := make(map[string]struct{})
for i := 0; i < nVals; i++ {
activeVals[string(css[i].privValidator.GetAddress())] = struct{}{}
addr := css[i].privValidator.GetPubKey().Address()
activeVals[string(addr)] = struct{}{}
}
// wait till everyone makes block 1


+ 0
- 6
consensus/replay_test.go View File

@ -659,12 +659,6 @@ func TestInitChainUpdateValidators(t *testing.T) {
assert.Equal(t, newValAddr, expectValAddr)
}
func newInitChainApp(vals []abci.ValidatorUpdate) *initChainApp {
return &initChainApp{
vals: vals,
}
}
// returns the vals on InitChain
type initChainApp struct {
abci.BaseApplication


+ 13
- 10
consensus/state.go View File

@ -2,13 +2,14 @@ package consensus
import (
"bytes"
"errors"
"fmt"
"reflect"
"runtime/debug"
"sync"
"time"
"github.com/pkg/errors"
cmn "github.com/tendermint/tendermint/libs/common"
"github.com/tendermint/tendermint/libs/fail"
"github.com/tendermint/tendermint/libs/log"
@ -829,13 +830,14 @@ func (cs *ConsensusState) enterPropose(height int64, round int) {
}
// if not a validator, we're done
if !cs.Validators.HasAddress(cs.privValidator.GetAddress()) {
logger.Debug("This node is not a validator", "addr", cs.privValidator.GetAddress(), "vals", cs.Validators)
address := cs.privValidator.GetPubKey().Address()
if !cs.Validators.HasAddress(address) {
logger.Debug("This node is not a validator", "addr", address, "vals", cs.Validators)
return
}
logger.Debug("This node is a validator")
if cs.isProposer() {
if cs.isProposer(address) {
logger.Info("enterPropose: Our turn to propose", "proposer", cs.Validators.GetProposer().Address, "privValidator", cs.privValidator)
cs.decideProposal(height, round)
} else {
@ -843,8 +845,8 @@ func (cs *ConsensusState) enterPropose(height int64, round int) {
}
}
func (cs *ConsensusState) isProposer() bool {
return bytes.Equal(cs.Validators.GetProposer().Address, cs.privValidator.GetAddress())
func (cs *ConsensusState) isProposer(address []byte) bool {
return bytes.Equal(cs.Validators.GetProposer().Address, address)
}
func (cs *ConsensusState) defaultDecideProposal(height int64, round int) {
@ -929,7 +931,7 @@ func (cs *ConsensusState) createProposalBlock() (block *types.Block, blockParts
cs.state.Validators.Size(),
len(evidence),
), maxGas)
proposerAddr := cs.privValidator.GetAddress()
proposerAddr := cs.privValidator.GetPubKey().Address()
block, parts := cs.state.MakeBlock(cs.Height, txs, commit, evidence, proposerAddr)
return block, parts
@ -1474,7 +1476,8 @@ func (cs *ConsensusState) tryAddVote(vote *types.Vote, peerID p2p.ID) (bool, err
if err == ErrVoteHeightMismatch {
return added, err
} else if voteErr, ok := err.(*types.ErrVoteConflictingVotes); ok {
if bytes.Equal(vote.ValidatorAddress, cs.privValidator.GetAddress()) {
addr := cs.privValidator.GetPubKey().Address()
if bytes.Equal(vote.ValidatorAddress, addr) {
cs.Logger.Error("Found conflicting vote from ourselves. Did you unsafe_reset a validator?", "height", vote.Height, "round", vote.Round, "type", vote.Type)
return added, err
}
@ -1639,7 +1642,7 @@ func (cs *ConsensusState) addVote(vote *types.Vote, peerID p2p.ID) (added bool,
}
func (cs *ConsensusState) signVote(type_ types.SignedMsgType, hash []byte, header types.PartSetHeader) (*types.Vote, error) {
addr := cs.privValidator.GetAddress()
addr := cs.privValidator.GetPubKey().Address()
valIndex, _ := cs.Validators.GetByAddress(addr)
vote := &types.Vote{
@ -1675,7 +1678,7 @@ func (cs *ConsensusState) voteTime() time.Time {
// sign the vote and publish on internalMsgQueue
func (cs *ConsensusState) signAddVote(type_ types.SignedMsgType, hash []byte, header types.PartSetHeader) *types.Vote {
// if we don't have a key or we're not in the validator set, do nothing
if cs.privValidator == nil || !cs.Validators.HasAddress(cs.privValidator.GetAddress()) {
if cs.privValidator == nil || !cs.Validators.HasAddress(cs.privValidator.GetPubKey().Address()) {
return nil
}
vote, err := cs.signVote(type_, hash, header)


+ 28
- 14
consensus/state_test.go View File

@ -73,7 +73,8 @@ func TestStateProposerSelection0(t *testing.T) {
// Commit a block and ensure proposer for the next height is correct.
prop := cs1.GetRoundState().Validators.GetProposer()
if !bytes.Equal(prop.Address, cs1.privValidator.GetAddress()) {
address := cs1.privValidator.GetPubKey().Address()
if !bytes.Equal(prop.Address, address) {
t.Fatalf("expected proposer to be validator %d. Got %X", 0, prop.Address)
}
@ -87,7 +88,8 @@ func TestStateProposerSelection0(t *testing.T) {
ensureNewRound(newRoundCh, height+1, 0)
prop = cs1.GetRoundState().Validators.GetProposer()
if !bytes.Equal(prop.Address, vss[1].GetAddress()) {
addr := vss[1].GetPubKey().Address()
if !bytes.Equal(prop.Address, addr) {
panic(fmt.Sprintf("expected proposer to be validator %d. Got %X", 1, prop.Address))
}
}
@ -110,7 +112,8 @@ func TestStateProposerSelection2(t *testing.T) {
// everyone just votes nil. we get a new proposer each round
for i := 0; i < len(vss); i++ {
prop := cs1.GetRoundState().Validators.GetProposer()
correctProposer := vss[(i+round)%len(vss)].GetAddress()
addr := vss[(i+round)%len(vss)].GetPubKey().Address()
correctProposer := addr
if !bytes.Equal(prop.Address, correctProposer) {
panic(fmt.Sprintf("expected RoundState.Validators.GetProposer() to be validator %d. Got %X", (i+2)%len(vss), prop.Address))
}
@ -505,7 +508,8 @@ func TestStateLockPOLRelock(t *testing.T) {
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
voteCh := subscribeToVoter(cs1, cs1.privValidator.GetAddress())
addr := cs1.privValidator.GetPubKey().Address()
voteCh := subscribeToVoter(cs1, addr)
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
newBlockCh := subscribe(cs1.eventBus, types.EventQueryNewBlockHeader)
@ -596,7 +600,8 @@ func TestStateLockPOLUnlock(t *testing.T) {
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
unlockCh := subscribe(cs1.eventBus, types.EventQueryUnlock)
voteCh := subscribeToVoter(cs1, cs1.privValidator.GetAddress())
addr := cs1.privValidator.GetPubKey().Address()
voteCh := subscribeToVoter(cs1, addr)
// everything done from perspective of cs1
@ -689,7 +694,8 @@ func TestStateLockPOLSafety1(t *testing.T) {
timeoutProposeCh := subscribe(cs1.eventBus, types.EventQueryTimeoutPropose)
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
voteCh := subscribeToVoter(cs1, cs1.privValidator.GetAddress())
addr := cs1.privValidator.GetPubKey().Address()
voteCh := subscribeToVoter(cs1, addr)
// start round and wait for propose and prevote
startTestRound(cs1, cs1.Height, round)
@ -805,7 +811,8 @@ func TestStateLockPOLSafety2(t *testing.T) {
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
unlockCh := subscribe(cs1.eventBus, types.EventQueryUnlock)
voteCh := subscribeToVoter(cs1, cs1.privValidator.GetAddress())
addr := cs1.privValidator.GetPubKey().Address()
voteCh := subscribeToVoter(cs1, addr)
// the block for R0: gets polkad but we miss it
// (even though we signed it, shhh)
@ -896,7 +903,8 @@ func TestProposeValidBlock(t *testing.T) {
timeoutProposeCh := subscribe(cs1.eventBus, types.EventQueryTimeoutPropose)
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
unlockCh := subscribe(cs1.eventBus, types.EventQueryUnlock)
voteCh := subscribeToVoter(cs1, cs1.privValidator.GetAddress())
addr := cs1.privValidator.GetPubKey().Address()
voteCh := subscribeToVoter(cs1, addr)
// start round and wait for propose and prevote
startTestRound(cs1, cs1.Height, round)
@ -982,7 +990,8 @@ func TestSetValidBlockOnDelayedPrevote(t *testing.T) {
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
validBlockCh := subscribe(cs1.eventBus, types.EventQueryValidBlock)
voteCh := subscribeToVoter(cs1, cs1.privValidator.GetAddress())
addr := cs1.privValidator.GetPubKey().Address()
voteCh := subscribeToVoter(cs1, addr)
// start round and wait for propose and prevote
startTestRound(cs1, cs1.Height, round)
@ -1041,7 +1050,8 @@ func TestSetValidBlockOnDelayedProposal(t *testing.T) {
timeoutProposeCh := subscribe(cs1.eventBus, types.EventQueryTimeoutPropose)
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
validBlockCh := subscribe(cs1.eventBus, types.EventQueryValidBlock)
voteCh := subscribeToVoter(cs1, cs1.privValidator.GetAddress())
addr := cs1.privValidator.GetPubKey().Address()
voteCh := subscribeToVoter(cs1, addr)
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
round = round + 1 // move to round in which P0 is not proposer
@ -1111,7 +1121,8 @@ func TestWaitingTimeoutProposeOnNewRound(t *testing.T) {
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutPropose)
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
voteCh := subscribeToVoter(cs1, cs1.privValidator.GetAddress())
addr := cs1.privValidator.GetPubKey().Address()
voteCh := subscribeToVoter(cs1, addr)
// start round
startTestRound(cs1, height, round)
@ -1144,7 +1155,8 @@ func TestRoundSkipOnNilPolkaFromHigherRound(t *testing.T) {
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
voteCh := subscribeToVoter(cs1, cs1.privValidator.GetAddress())
addr := cs1.privValidator.GetPubKey().Address()
voteCh := subscribeToVoter(cs1, addr)
// start round
startTestRound(cs1, height, round)
@ -1177,7 +1189,8 @@ func TestWaitTimeoutProposeOnNilPolkaForTheCurrentRound(t *testing.T) {
timeoutProposeCh := subscribe(cs1.eventBus, types.EventQueryTimeoutPropose)
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
voteCh := subscribeToVoter(cs1, cs1.privValidator.GetAddress())
addr := cs1.privValidator.GetPubKey().Address()
voteCh := subscribeToVoter(cs1, addr)
// start round in which PO is not proposer
startTestRound(cs1, height, round)
@ -1361,7 +1374,8 @@ func TestStateHalt1(t *testing.T) {
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
newBlockCh := subscribe(cs1.eventBus, types.EventQueryNewBlock)
voteCh := subscribeToVoter(cs1, cs1.privValidator.GetAddress())
addr := cs1.privValidator.GetPubKey().Address()
voteCh := subscribeToVoter(cs1, addr)
// start round and wait for propose and prevote
startTestRound(cs1, height, round)


+ 2
- 1
consensus/types/height_vote_set_test.go View File

@ -50,8 +50,9 @@ func TestPeerCatchupRounds(t *testing.T) {
func makeVoteHR(t *testing.T, height int64, round int, privVals []types.PrivValidator, valIndex int) *types.Vote {
privVal := privVals[valIndex]
addr := privVal.GetPubKey().Address()
vote := &types.Vote{
ValidatorAddress: privVal.GetAddress(),
ValidatorAddress: addr,
ValidatorIndex: valIndex,
Height: height,
Round: round,


+ 9
- 5
node/node.go View File

@ -259,16 +259,19 @@ func NewNode(config *cfg.Config,
fastSync := config.FastSync
if state.Validators.Size() == 1 {
addr, _ := state.Validators.GetByIndex(0)
if bytes.Equal(privValidator.GetAddress(), addr) {
privValAddr := privValidator.GetPubKey().Address()
if bytes.Equal(privValAddr, addr) {
fastSync = false
}
}
pubKey := privValidator.GetPubKey()
addr := pubKey.Address()
// Log whether this node is a validator or an observer
if state.Validators.HasAddress(privValidator.GetAddress()) {
consensusLogger.Info("This node is a validator", "addr", privValidator.GetAddress(), "pubKey", privValidator.GetPubKey())
if state.Validators.HasAddress(addr) {
consensusLogger.Info("This node is a validator", "addr", addr, "pubKey", pubKey)
} else {
consensusLogger.Info("This node is not a validator", "addr", privValidator.GetAddress(), "pubKey", privValidator.GetPubKey())
consensusLogger.Info("This node is not a validator", "addr", addr, "pubKey", pubKey)
}
csMetrics, p2pMetrics, memplMetrics, smMetrics := metricsProvider()
@ -636,7 +639,8 @@ func (n *Node) ConfigureRPC() {
rpccore.SetEvidencePool(n.evidencePool)
rpccore.SetP2PPeers(n.sw)
rpccore.SetP2PTransport(n)
rpccore.SetPubKey(n.privValidator.GetPubKey())
pubKey := n.privValidator.GetPubKey()
rpccore.SetPubKey(pubKey)
rpccore.SetGenesisDoc(n.genesisDoc)
rpccore.SetAddrBook(n.addrBook)
rpccore.SetProxyAppQuery(n.proxyApp.Query())


+ 4
- 1
privval/ipc.go View File

@ -67,7 +67,10 @@ func (sc *IPCVal) OnStart() error {
return err
}
sc.RemoteSignerClient = NewRemoteSignerClient(sc.conn)
sc.RemoteSignerClient, err = NewRemoteSignerClient(sc.conn)
if err != nil {
return err
}
// Start a routine to keep the connection alive
sc.cancelPing = make(chan struct{}, 1)


+ 39
- 32
privval/remote_signer.go View File

@ -6,6 +6,8 @@ import (
"net"
"sync"
"github.com/pkg/errors"
"github.com/tendermint/go-amino"
"github.com/tendermint/tendermint/crypto"
cmn "github.com/tendermint/tendermint/libs/common"
@ -15,8 +17,9 @@ import (
// RemoteSignerClient implements PrivValidator, it uses a socket to request signatures
// from an external process.
type RemoteSignerClient struct {
conn net.Conn
lock sync.Mutex
conn net.Conn
consensusPubKey crypto.PubKey
mtx sync.Mutex
}
// Check that RemoteSignerClient implements PrivValidator.
@ -25,38 +28,29 @@ var _ types.PrivValidator = (*RemoteSignerClient)(nil)
// NewRemoteSignerClient returns an instance of RemoteSignerClient.
func NewRemoteSignerClient(
conn net.Conn,
) *RemoteSignerClient {
) (*RemoteSignerClient, error) {
sc := &RemoteSignerClient{
conn: conn,
}
return sc
}
// GetAddress implements PrivValidator.
func (sc *RemoteSignerClient) GetAddress() types.Address {
pubKey, err := sc.getPubKey()
if err != nil {
panic(err)
return nil, cmn.ErrorWrap(err, "error while retrieving public key for remote signer")
}
return pubKey.Address()
// retrieve and memoize the consensus public key once:
sc.consensusPubKey = pubKey
return sc, nil
}
// GetPubKey implements PrivValidator.
func (sc *RemoteSignerClient) GetPubKey() crypto.PubKey {
pubKey, err := sc.getPubKey()
if err != nil {
panic(err)
}
return pubKey
return sc.consensusPubKey
}
func (sc *RemoteSignerClient) getPubKey() (crypto.PubKey, error) {
sc.lock.Lock()
defer sc.lock.Unlock()
sc.mtx.Lock()
defer sc.mtx.Unlock()
err := writeMsg(sc.conn, &PubKeyMsg{})
err := writeMsg(sc.conn, &PubKeyRequest{})
if err != nil {
return nil, err
}
@ -65,14 +59,22 @@ func (sc *RemoteSignerClient) getPubKey() (crypto.PubKey, error) {
if err != nil {
return nil, err
}
pubKeyResp, ok := res.(*PubKeyResponse)
if !ok {
return nil, errors.Wrap(ErrUnexpectedResponse, "response is not PubKeyResponse")
}
return res.(*PubKeyMsg).PubKey, nil
if pubKeyResp.Error != nil {
return nil, errors.Wrap(pubKeyResp.Error, "failed to get private validator's public key")
}
return pubKeyResp.PubKey, nil
}
// SignVote implements PrivValidator.
func (sc *RemoteSignerClient) SignVote(chainID string, vote *types.Vote) error {
sc.lock.Lock()
defer sc.lock.Unlock()
sc.mtx.Lock()
defer sc.mtx.Unlock()
err := writeMsg(sc.conn, &SignVoteRequest{Vote: vote})
if err != nil {
@ -101,8 +103,8 @@ func (sc *RemoteSignerClient) SignProposal(
chainID string,
proposal *types.Proposal,
) error {
sc.lock.Lock()
defer sc.lock.Unlock()
sc.mtx.Lock()
defer sc.mtx.Unlock()
err := writeMsg(sc.conn, &SignProposalRequest{Proposal: proposal})
if err != nil {
@ -127,8 +129,8 @@ func (sc *RemoteSignerClient) SignProposal(
// Ping is used to check connection health.
func (sc *RemoteSignerClient) Ping() error {
sc.lock.Lock()
defer sc.lock.Unlock()
sc.mtx.Lock()
defer sc.mtx.Unlock()
err := writeMsg(sc.conn, &PingRequest{})
if err != nil {
@ -152,7 +154,8 @@ type RemoteSignerMsg interface{}
func RegisterRemoteSignerMsg(cdc *amino.Codec) {
cdc.RegisterInterface((*RemoteSignerMsg)(nil), nil)
cdc.RegisterConcrete(&PubKeyMsg{}, "tendermint/remotesigner/PubKeyMsg", nil)
cdc.RegisterConcrete(&PubKeyRequest{}, "tendermint/remotesigner/PubKeyRequest", nil)
cdc.RegisterConcrete(&PubKeyResponse{}, "tendermint/remotesigner/PubKeyResponse", nil)
cdc.RegisterConcrete(&SignVoteRequest{}, "tendermint/remotesigner/SignVoteRequest", nil)
cdc.RegisterConcrete(&SignedVoteResponse{}, "tendermint/remotesigner/SignedVoteResponse", nil)
cdc.RegisterConcrete(&SignProposalRequest{}, "tendermint/remotesigner/SignProposalRequest", nil)
@ -161,9 +164,13 @@ func RegisterRemoteSignerMsg(cdc *amino.Codec) {
cdc.RegisterConcrete(&PingResponse{}, "tendermint/remotesigner/PingResponse", nil)
}
// PubKeyMsg is a PrivValidatorSocket message containing the public key.
type PubKeyMsg struct {
// PubKeyRequest requests the consensus public key from the remote signer.
type PubKeyRequest struct{}
// PubKeyResponse is a PrivValidatorSocket message containing the public key.
type PubKeyResponse struct {
PubKey crypto.PubKey
Error *RemoteSignerError
}
// SignVoteRequest is a PrivValidatorSocket message containing a vote.
@ -227,10 +234,10 @@ func handleRequest(req RemoteSignerMsg, chainID string, privVal types.PrivValida
var err error
switch r := req.(type) {
case *PubKeyMsg:
case *PubKeyRequest:
var p crypto.PubKey
p = privVal.GetPubKey()
res = &PubKeyMsg{p}
res = &PubKeyResponse{p, nil}
case *SignVoteRequest:
err = privVal.SignVote(chainID, r.Vote)
if err != nil {


+ 4
- 2
privval/tcp.go View File

@ -107,8 +107,10 @@ func (sc *TCPVal) OnStart() error {
}
sc.conn = conn
sc.RemoteSignerClient = NewRemoteSignerClient(sc.conn)
sc.RemoteSignerClient, err = NewRemoteSignerClient(sc.conn)
if err != nil {
return err
}
// Start a routine to keep the connection alive
sc.cancelPing = make(chan struct{}, 1)


+ 9
- 19
privval/tcp_test.go View File

@ -25,15 +25,10 @@ func TestSocketPVAddress(t *testing.T) {
defer sc.Stop()
defer rs.Stop()
serverAddr := rs.privVal.GetAddress()
clientAddr := sc.GetAddress()
serverAddr := rs.privVal.GetPubKey().Address()
clientAddr := sc.GetPubKey().Address()
assert.Equal(t, serverAddr, clientAddr)
// TODO(xla): Remove when PrivValidator2 replaced PrivValidator.
assert.Equal(t, serverAddr, sc.GetAddress())
}
func TestSocketPVPubKey(t *testing.T) {
@ -47,12 +42,9 @@ func TestSocketPVPubKey(t *testing.T) {
clientKey, err := sc.getPubKey()
require.NoError(t, err)
privKey := rs.privVal.GetPubKey()
privvalPubKey := rs.privVal.GetPubKey()
assert.Equal(t, privKey, clientKey)
// TODO(xla): Remove when PrivValidator2 replaced PrivValidator.
assert.Equal(t, privKey, sc.GetPubKey())
assert.Equal(t, privvalPubKey, clientKey)
}
func TestSocketPVProposal(t *testing.T) {
@ -153,9 +145,9 @@ func TestSocketPVDeadline(t *testing.T) {
go func(sc *TCPVal) {
defer close(listenc)
require.NoError(t, sc.Start())
assert.Equal(t, sc.Start().(cmn.Error).Data(), ErrConnTimeout)
assert.True(t, sc.IsRunning())
assert.False(t, sc.IsRunning())
}(sc)
for {
@ -174,9 +166,6 @@ func TestSocketPVDeadline(t *testing.T) {
}
<-listenc
_, err := sc.getPubKey()
assert.Equal(t, err.(cmn.Error).Data(), ErrConnTimeout)
}
func TestRemoteSignerRetry(t *testing.T) {
@ -310,14 +299,15 @@ func TestErrUnexpectedResponse(t *testing.T) {
testStartSocketPV(t, readyc, sc)
defer sc.Stop()
RemoteSignerConnDeadline(time.Millisecond)(rs)
RemoteSignerConnRetries(1e6)(rs)
RemoteSignerConnRetries(100)(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)
// send over public key to get the remote signer running:
go testReadWriteResponse(t, &PubKeyResponse{}, rsConn)
<-readyc
// Proposal:


+ 2
- 1
types/evidence_test.go View File

@ -17,8 +17,9 @@ type voteData struct {
}
func makeVote(val PrivValidator, chainID string, valIndex int, height int64, round, step int, blockID BlockID) *Vote {
addr := val.GetPubKey().Address()
v := &Vote{
ValidatorAddress: val.GetAddress(),
ValidatorAddress: addr,
ValidatorIndex: valIndex,
Height: height,
Round: round,


+ 3
- 8
types/priv_validator.go View File

@ -12,7 +12,6 @@ import (
// PrivValidator defines the functionality of a local Tendermint validator
// that signs votes and proposals, and never double signs.
type PrivValidator interface {
GetAddress() Address // redundant since .PubKey().Address()
GetPubKey() crypto.PubKey
SignVote(chainID string, vote *Vote) error
@ -29,7 +28,7 @@ func (pvs PrivValidatorsByAddress) Len() int {
}
func (pvs PrivValidatorsByAddress) Less(i, j int) bool {
return bytes.Compare(pvs[i].GetAddress(), pvs[j].GetAddress()) == -1
return bytes.Compare(pvs[i].GetPubKey().Address(), pvs[j].GetPubKey().Address()) == -1
}
func (pvs PrivValidatorsByAddress) Swap(i, j int) {
@ -51,11 +50,6 @@ func NewMockPV() *MockPV {
return &MockPV{ed25519.GenPrivKey()}
}
// Implements PrivValidator.
func (pv *MockPV) GetAddress() Address {
return pv.privKey.PubKey().Address()
}
// Implements PrivValidator.
func (pv *MockPV) GetPubKey() crypto.PubKey {
return pv.privKey.PubKey()
@ -85,7 +79,8 @@ func (pv *MockPV) SignProposal(chainID string, proposal *Proposal) error {
// String returns a string representation of the MockPV.
func (pv *MockPV) String() string {
return fmt.Sprintf("MockPV{%v}", pv.GetAddress())
addr := pv.GetPubKey().Address()
return fmt.Sprintf("MockPV{%v}", addr)
}
// XXX: Implement.


+ 3
- 2
types/protobuf_test.go View File

@ -142,14 +142,15 @@ func TestABCIEvidence(t *testing.T) {
blockID := makeBlockID([]byte("blockhash"), 1000, []byte("partshash"))
blockID2 := makeBlockID([]byte("blockhash2"), 1000, []byte("partshash"))
const chainID = "mychain"
pubKey := val.GetPubKey()
ev := &DuplicateVoteEvidence{
PubKey: val.GetPubKey(),
PubKey: pubKey,
VoteA: makeVote(val, chainID, 0, 10, 2, 1, blockID),
VoteB: makeVote(val, chainID, 0, 10, 2, 1, blockID2),
}
abciEv := TM2PB.Evidence(
ev,
NewValidatorSet([]*Validator{NewValidator(val.GetPubKey(), 10)}),
NewValidatorSet([]*Validator{NewValidator(pubKey, 10)}),
time.Now(),
)


+ 2
- 2
types/test_util.go View File

@ -10,9 +10,9 @@ func MakeCommit(blockID BlockID, height int64, round int,
// all sign
for i := 0; i < len(validators); i++ {
addr := validators[i].GetPubKey().Address()
vote := &Vote{
ValidatorAddress: validators[i].GetAddress(),
ValidatorAddress: addr,
ValidatorIndex: i,
Height: height,
Round: round,


+ 2
- 1
types/validator.go View File

@ -101,6 +101,7 @@ func RandValidator(randPower bool, minPower int64) (*Validator, PrivValidator) {
if randPower {
votePower += int64(cmn.RandUint32())
}
val := NewValidator(privVal.GetPubKey(), votePower)
pubKey := privVal.GetPubKey()
val := NewValidator(pubKey, votePower)
return val, privVal
}

+ 49
- 27
types/vote_set_test.go View File

@ -66,7 +66,8 @@ func TestAddVote(t *testing.T) {
// t.Logf(">> %v", voteSet)
if voteSet.GetByAddress(val0.GetAddress()) != nil {
val0Addr := val0.GetPubKey().Address()
if voteSet.GetByAddress(val0Addr) != nil {
t.Errorf("Expected GetByAddress(val0.Address) to be nil")
}
if voteSet.BitArray().GetIndex(0) {
@ -78,7 +79,7 @@ func TestAddVote(t *testing.T) {
}
vote := &Vote{
ValidatorAddress: val0.GetAddress(),
ValidatorAddress: val0Addr,
ValidatorIndex: 0, // since privValidators are in order
Height: height,
Round: round,
@ -91,7 +92,7 @@ func TestAddVote(t *testing.T) {
t.Error(err)
}
if voteSet.GetByAddress(val0.GetAddress()) == nil {
if voteSet.GetByAddress(val0Addr) == nil {
t.Errorf("Expected GetByAddress(val0.Address) to be present")
}
if !voteSet.BitArray().GetIndex(0) {
@ -118,7 +119,8 @@ func Test2_3Majority(t *testing.T) {
}
// 6 out of 10 voted for nil.
for i := 0; i < 6; i++ {
vote := withValidator(voteProto, privValidators[i].GetAddress(), i)
addr := privValidators[i].GetPubKey().Address()
vote := withValidator(voteProto, addr, i)
_, err := signAddVote(privValidators[i], vote, voteSet)
if err != nil {
t.Error(err)
@ -131,7 +133,8 @@ func Test2_3Majority(t *testing.T) {
// 7th validator voted for some blockhash
{
vote := withValidator(voteProto, privValidators[6].GetAddress(), 6)
addr := privValidators[6].GetPubKey().Address()
vote := withValidator(voteProto, addr, 6)
_, err := signAddVote(privValidators[6], withBlockHash(vote, cmn.RandBytes(32)), voteSet)
if err != nil {
t.Error(err)
@ -144,7 +147,8 @@ func Test2_3Majority(t *testing.T) {
// 8th validator voted for nil.
{
vote := withValidator(voteProto, privValidators[7].GetAddress(), 7)
addr := privValidators[7].GetPubKey().Address()
vote := withValidator(voteProto, addr, 7)
_, err := signAddVote(privValidators[7], vote, voteSet)
if err != nil {
t.Error(err)
@ -176,7 +180,8 @@ func Test2_3MajorityRedux(t *testing.T) {
// 66 out of 100 voted for nil.
for i := 0; i < 66; i++ {
vote := withValidator(voteProto, privValidators[i].GetAddress(), i)
addr := privValidators[i].GetPubKey().Address()
vote := withValidator(voteProto, addr, i)
_, err := signAddVote(privValidators[i], vote, voteSet)
if err != nil {
t.Error(err)
@ -189,7 +194,8 @@ func Test2_3MajorityRedux(t *testing.T) {
// 67th validator voted for nil
{
vote := withValidator(voteProto, privValidators[66].GetAddress(), 66)
adrr := privValidators[66].GetPubKey().Address()
vote := withValidator(voteProto, adrr, 66)
_, err := signAddVote(privValidators[66], withBlockHash(vote, nil), voteSet)
if err != nil {
t.Error(err)
@ -202,7 +208,8 @@ func Test2_3MajorityRedux(t *testing.T) {
// 68th validator voted for a different BlockParts PartSetHeader
{
vote := withValidator(voteProto, privValidators[67].GetAddress(), 67)
addr := privValidators[67].GetPubKey().Address()
vote := withValidator(voteProto, addr, 67)
blockPartsHeader := PartSetHeader{blockPartsTotal, crypto.CRandBytes(32)}
_, err := signAddVote(privValidators[67], withBlockPartsHeader(vote, blockPartsHeader), voteSet)
if err != nil {
@ -216,7 +223,8 @@ func Test2_3MajorityRedux(t *testing.T) {
// 69th validator voted for different BlockParts Total
{
vote := withValidator(voteProto, privValidators[68].GetAddress(), 68)
addr := privValidators[68].GetPubKey().Address()
vote := withValidator(voteProto, addr, 68)
blockPartsHeader := PartSetHeader{blockPartsTotal + 1, blockPartsHeader.Hash}
_, err := signAddVote(privValidators[68], withBlockPartsHeader(vote, blockPartsHeader), voteSet)
if err != nil {
@ -230,7 +238,8 @@ func Test2_3MajorityRedux(t *testing.T) {
// 70th validator voted for different BlockHash
{
vote := withValidator(voteProto, privValidators[69].GetAddress(), 69)
addr := privValidators[69].GetPubKey().Address()
vote := withValidator(voteProto, addr, 69)
_, err := signAddVote(privValidators[69], withBlockHash(vote, cmn.RandBytes(32)), voteSet)
if err != nil {
t.Error(err)
@ -243,7 +252,8 @@ func Test2_3MajorityRedux(t *testing.T) {
// 71st validator voted for the right BlockHash & BlockPartsHeader
{
vote := withValidator(voteProto, privValidators[70].GetAddress(), 70)
addr := privValidators[70].GetPubKey().Address()
vote := withValidator(voteProto, addr, 70)
_, err := signAddVote(privValidators[70], vote, voteSet)
if err != nil {
t.Error(err)
@ -271,7 +281,8 @@ func TestBadVotes(t *testing.T) {
// val0 votes for nil.
{
vote := withValidator(voteProto, privValidators[0].GetAddress(), 0)
addr := privValidators[0].GetPubKey().Address()
vote := withValidator(voteProto, addr, 0)
added, err := signAddVote(privValidators[0], vote, voteSet)
if !added || err != nil {
t.Errorf("Expected VoteSet.Add to succeed")
@ -280,7 +291,8 @@ func TestBadVotes(t *testing.T) {
// val0 votes again for some block.
{
vote := withValidator(voteProto, privValidators[0].GetAddress(), 0)
addr := privValidators[0].GetPubKey().Address()
vote := withValidator(voteProto, addr, 0)
added, err := signAddVote(privValidators[0], withBlockHash(vote, cmn.RandBytes(32)), voteSet)
if added || err == nil {
t.Errorf("Expected VoteSet.Add to fail, conflicting vote.")
@ -289,7 +301,8 @@ func TestBadVotes(t *testing.T) {
// val1 votes on another height
{
vote := withValidator(voteProto, privValidators[1].GetAddress(), 1)
addr := privValidators[1].GetPubKey().Address()
vote := withValidator(voteProto, addr, 1)
added, err := signAddVote(privValidators[1], withHeight(vote, height+1), voteSet)
if added || err == nil {
t.Errorf("Expected VoteSet.Add to fail, wrong height")
@ -298,7 +311,8 @@ func TestBadVotes(t *testing.T) {
// val2 votes on another round
{
vote := withValidator(voteProto, privValidators[2].GetAddress(), 2)
addr := privValidators[2].GetPubKey().Address()
vote := withValidator(voteProto, addr, 2)
added, err := signAddVote(privValidators[2], withRound(vote, round+1), voteSet)
if added || err == nil {
t.Errorf("Expected VoteSet.Add to fail, wrong round")
@ -307,7 +321,8 @@ func TestBadVotes(t *testing.T) {
// val3 votes of another type.
{
vote := withValidator(voteProto, privValidators[3].GetAddress(), 3)
addr := privValidators[3].GetPubKey().Address()
vote := withValidator(voteProto, addr, 3)
added, err := signAddVote(privValidators[3], withType(vote, byte(PrecommitType)), voteSet)
if added || err == nil {
t.Errorf("Expected VoteSet.Add to fail, wrong type")
@ -331,9 +346,10 @@ func TestConflicts(t *testing.T) {
BlockID: BlockID{nil, PartSetHeader{}},
}
val0Addr := privValidators[0].GetPubKey().Address()
// val0 votes for nil.
{
vote := withValidator(voteProto, privValidators[0].GetAddress(), 0)
vote := withValidator(voteProto, val0Addr, 0)
added, err := signAddVote(privValidators[0], vote, voteSet)
if !added || err != nil {
t.Errorf("Expected VoteSet.Add to succeed")
@ -342,7 +358,7 @@ func TestConflicts(t *testing.T) {
// val0 votes again for blockHash1.
{
vote := withValidator(voteProto, privValidators[0].GetAddress(), 0)
vote := withValidator(voteProto, val0Addr, 0)
added, err := signAddVote(privValidators[0], withBlockHash(vote, blockHash1), voteSet)
if added {
t.Errorf("Expected VoteSet.Add to fail, conflicting vote.")
@ -357,7 +373,7 @@ func TestConflicts(t *testing.T) {
// val0 votes again for blockHash1.
{
vote := withValidator(voteProto, privValidators[0].GetAddress(), 0)
vote := withValidator(voteProto, val0Addr, 0)
added, err := signAddVote(privValidators[0], withBlockHash(vote, blockHash1), voteSet)
if !added {
t.Errorf("Expected VoteSet.Add to succeed, called SetPeerMaj23().")
@ -372,7 +388,7 @@ func TestConflicts(t *testing.T) {
// val0 votes again for blockHash1.
{
vote := withValidator(voteProto, privValidators[0].GetAddress(), 0)
vote := withValidator(voteProto, val0Addr, 0)
added, err := signAddVote(privValidators[0], withBlockHash(vote, blockHash2), voteSet)
if added {
t.Errorf("Expected VoteSet.Add to fail, duplicate SetPeerMaj23() from peerA")
@ -384,7 +400,8 @@ func TestConflicts(t *testing.T) {
// val1 votes for blockHash1.
{
vote := withValidator(voteProto, privValidators[1].GetAddress(), 1)
addr := privValidators[1].GetPubKey().Address()
vote := withValidator(voteProto, addr, 1)
added, err := signAddVote(privValidators[1], withBlockHash(vote, blockHash1), voteSet)
if !added || err != nil {
t.Errorf("Expected VoteSet.Add to succeed")
@ -401,7 +418,8 @@ func TestConflicts(t *testing.T) {
// val2 votes for blockHash2.
{
vote := withValidator(voteProto, privValidators[2].GetAddress(), 2)
addr := privValidators[2].GetPubKey().Address()
vote := withValidator(voteProto, addr, 2)
added, err := signAddVote(privValidators[2], withBlockHash(vote, blockHash2), voteSet)
if !added || err != nil {
t.Errorf("Expected VoteSet.Add to succeed")
@ -421,7 +439,8 @@ func TestConflicts(t *testing.T) {
// val2 votes for blockHash1.
{
vote := withValidator(voteProto, privValidators[2].GetAddress(), 2)
addr := privValidators[2].GetPubKey().Address()
vote := withValidator(voteProto, addr, 2)
added, err := signAddVote(privValidators[2], withBlockHash(vote, blockHash1), voteSet)
if !added {
t.Errorf("Expected VoteSet.Add to succeed")
@ -462,7 +481,8 @@ func TestMakeCommit(t *testing.T) {
// 6 out of 10 voted for some block.
for i := 0; i < 6; i++ {
vote := withValidator(voteProto, privValidators[i].GetAddress(), i)
addr := privValidators[i].GetPubKey().Address()
vote := withValidator(voteProto, addr, i)
_, err := signAddVote(privValidators[i], vote, voteSet)
if err != nil {
t.Error(err)
@ -474,7 +494,8 @@ func TestMakeCommit(t *testing.T) {
// 7th voted for some other block.
{
vote := withValidator(voteProto, privValidators[6].GetAddress(), 6)
addr := privValidators[6].GetPubKey().Address()
vote := withValidator(voteProto, addr, 6)
vote = withBlockHash(vote, cmn.RandBytes(32))
vote = withBlockPartsHeader(vote, PartSetHeader{123, cmn.RandBytes(32)})
@ -486,7 +507,8 @@ func TestMakeCommit(t *testing.T) {
// The 8th voted like everyone else.
{
vote := withValidator(voteProto, privValidators[7].GetAddress(), 7)
addr := privValidators[7].GetPubKey().Address()
vote := withValidator(voteProto, addr, 7)
_, err := signAddVote(privValidators[7], vote, voteSet)
if err != nil {
t.Error(err)


Loading…
Cancel
Save