diff --git a/CHANGELOG.md b/CHANGELOG.md index 7caa8efef..0205c427f 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -26,6 +26,15 @@ BUG FIXES: - Graceful handling/recovery for apps that have non-determinism or fail to halt - Graceful handling/recovery for violations of safety, or liveness +## 0.16.0 (TBD) + +BREAKING CHANGES: +- [p2p] old `seeds` is now `persistent_peers` (persistent peers to which TM will always connect to) +- [p2p] now `seeds` only used for getting addresses (if addrbook is empty; not persistent) + +FEATURES: +- [p2p] added new `/dial_persistent_peers` **unsafe** endpoint + ## 0.15.0 (December 29, 2017) BREAKING CHANGES: diff --git a/benchmarks/blockchain/localsync.sh b/benchmarks/blockchain/localsync.sh index e181c5655..389464b6c 100755 --- a/benchmarks/blockchain/localsync.sh +++ b/benchmarks/blockchain/localsync.sh @@ -51,7 +51,7 @@ tendermint node \ --proxy_app dummy \ --p2p.laddr tcp://127.0.0.1:56666 \ --rpc.laddr tcp://127.0.0.1:56667 \ - --p2p.seeds 127.0.0.1:56656 \ + --p2p.persistent_peers 127.0.0.1:56656 \ --log_level error & # wait for node to start up so we only count time where we are actually syncing diff --git a/benchmarks/codec_test.go b/benchmarks/codec_test.go index 631b303eb..8ac62a247 100644 --- a/benchmarks/codec_test.go +++ b/benchmarks/codec_test.go @@ -16,11 +16,10 @@ func BenchmarkEncodeStatusWire(b *testing.B) { b.StopTimer() pubKey := crypto.GenPrivKeyEd25519().PubKey() status := &ctypes.ResultStatus{ - NodeInfo: &p2p.NodeInfo{ - PubKey: pubKey.Unwrap().(crypto.PubKeyEd25519), + NodeInfo: p2p.NodeInfo{ + PubKey: pubKey, Moniker: "SOMENAME", Network: "SOMENAME", - RemoteAddr: "SOMEADDR", ListenAddr: "SOMEADDR", Version: "SOMEVER", Other: []string{"SOMESTRING", "OTHERSTRING"}, @@ -42,12 +41,11 @@ func BenchmarkEncodeStatusWire(b *testing.B) { func BenchmarkEncodeNodeInfoWire(b *testing.B) { b.StopTimer() - pubKey := crypto.GenPrivKeyEd25519().PubKey().Unwrap().(crypto.PubKeyEd25519) - nodeInfo := &p2p.NodeInfo{ + pubKey := crypto.GenPrivKeyEd25519().PubKey() + nodeInfo := p2p.NodeInfo{ PubKey: pubKey, Moniker: "SOMENAME", Network: "SOMENAME", - RemoteAddr: "SOMEADDR", ListenAddr: "SOMEADDR", Version: "SOMEVER", Other: []string{"SOMESTRING", "OTHERSTRING"}, @@ -63,12 +61,11 @@ func BenchmarkEncodeNodeInfoWire(b *testing.B) { func BenchmarkEncodeNodeInfoBinary(b *testing.B) { b.StopTimer() - pubKey := crypto.GenPrivKeyEd25519().PubKey().Unwrap().(crypto.PubKeyEd25519) - nodeInfo := &p2p.NodeInfo{ + pubKey := crypto.GenPrivKeyEd25519().PubKey() + nodeInfo := p2p.NodeInfo{ PubKey: pubKey, Moniker: "SOMENAME", Network: "SOMENAME", - RemoteAddr: "SOMEADDR", ListenAddr: "SOMEADDR", Version: "SOMEVER", Other: []string{"SOMESTRING", "OTHERSTRING"}, @@ -87,11 +84,10 @@ func BenchmarkEncodeNodeInfoProto(b *testing.B) { b.StopTimer() pubKey := crypto.GenPrivKeyEd25519().PubKey().Unwrap().(crypto.PubKeyEd25519) pubKey2 := &proto.PubKey{Ed25519: &proto.PubKeyEd25519{Bytes: pubKey[:]}} - nodeInfo := &proto.NodeInfo{ + nodeInfo := proto.NodeInfo{ PubKey: pubKey2, Moniker: "SOMENAME", Network: "SOMENAME", - RemoteAddr: "SOMEADDR", ListenAddr: "SOMEADDR", Version: "SOMEVER", Other: []string{"SOMESTRING", "OTHERSTRING"}, diff --git a/blockchain/pool.go b/blockchain/pool.go index e39749dc0..f3148e6c5 100644 --- a/blockchain/pool.go +++ b/blockchain/pool.go @@ -5,6 +5,7 @@ import ( "sync" "time" + "github.com/tendermint/tendermint/p2p" "github.com/tendermint/tendermint/types" cmn "github.com/tendermint/tmlibs/common" flow "github.com/tendermint/tmlibs/flowrate" @@ -56,16 +57,16 @@ type BlockPool struct { height int64 // the lowest key in requesters. numPending int32 // number of requests pending assignment or block response // peers - peers map[string]*bpPeer + peers map[p2p.ID]*bpPeer maxPeerHeight int64 requestsCh chan<- BlockRequest - timeoutsCh chan<- string + timeoutsCh chan<- p2p.ID } -func NewBlockPool(start int64, requestsCh chan<- BlockRequest, timeoutsCh chan<- string) *BlockPool { +func NewBlockPool(start int64, requestsCh chan<- BlockRequest, timeoutsCh chan<- p2p.ID) *BlockPool { bp := &BlockPool{ - peers: make(map[string]*bpPeer), + peers: make(map[p2p.ID]*bpPeer), requesters: make(map[int64]*bpRequester), height: start, @@ -210,7 +211,7 @@ func (pool *BlockPool) RedoRequest(height int64) { } // TODO: ensure that blocks come in order for each peer. -func (pool *BlockPool) AddBlock(peerID string, block *types.Block, blockSize int) { +func (pool *BlockPool) AddBlock(peerID p2p.ID, block *types.Block, blockSize int) { pool.mtx.Lock() defer pool.mtx.Unlock() @@ -240,7 +241,7 @@ func (pool *BlockPool) MaxPeerHeight() int64 { } // Sets the peer's alleged blockchain height. -func (pool *BlockPool) SetPeerHeight(peerID string, height int64) { +func (pool *BlockPool) SetPeerHeight(peerID p2p.ID, height int64) { pool.mtx.Lock() defer pool.mtx.Unlock() @@ -258,14 +259,14 @@ func (pool *BlockPool) SetPeerHeight(peerID string, height int64) { } } -func (pool *BlockPool) RemovePeer(peerID string) { +func (pool *BlockPool) RemovePeer(peerID p2p.ID) { pool.mtx.Lock() defer pool.mtx.Unlock() pool.removePeer(peerID) } -func (pool *BlockPool) removePeer(peerID string) { +func (pool *BlockPool) removePeer(peerID p2p.ID) { for _, requester := range pool.requesters { if requester.getPeerID() == peerID { if requester.getBlock() != nil { @@ -321,14 +322,14 @@ func (pool *BlockPool) requestersLen() int64 { return int64(len(pool.requesters)) } -func (pool *BlockPool) sendRequest(height int64, peerID string) { +func (pool *BlockPool) sendRequest(height int64, peerID p2p.ID) { if !pool.IsRunning() { return } pool.requestsCh <- BlockRequest{height, peerID} } -func (pool *BlockPool) sendTimeout(peerID string) { +func (pool *BlockPool) sendTimeout(peerID p2p.ID) { if !pool.IsRunning() { return } @@ -357,7 +358,7 @@ func (pool *BlockPool) debug() string { type bpPeer struct { pool *BlockPool - id string + id p2p.ID recvMonitor *flow.Monitor height int64 @@ -368,7 +369,7 @@ type bpPeer struct { logger log.Logger } -func newBPPeer(pool *BlockPool, peerID string, height int64) *bpPeer { +func newBPPeer(pool *BlockPool, peerID p2p.ID, height int64) *bpPeer { peer := &bpPeer{ pool: pool, id: peerID, @@ -434,7 +435,7 @@ type bpRequester struct { redoCh chan struct{} mtx sync.Mutex - peerID string + peerID p2p.ID block *types.Block } @@ -458,7 +459,7 @@ func (bpr *bpRequester) OnStart() error { } // Returns true if the peer matches -func (bpr *bpRequester) setBlock(block *types.Block, peerID string) bool { +func (bpr *bpRequester) setBlock(block *types.Block, peerID p2p.ID) bool { bpr.mtx.Lock() if bpr.block != nil || bpr.peerID != peerID { bpr.mtx.Unlock() @@ -477,7 +478,7 @@ func (bpr *bpRequester) getBlock() *types.Block { return bpr.block } -func (bpr *bpRequester) getPeerID() string { +func (bpr *bpRequester) getPeerID() p2p.ID { bpr.mtx.Lock() defer bpr.mtx.Unlock() return bpr.peerID @@ -551,5 +552,5 @@ OUTER_LOOP: type BlockRequest struct { Height int64 - PeerID string + PeerID p2p.ID } diff --git a/blockchain/pool_test.go b/blockchain/pool_test.go index 3e347fd29..0cdbc3a9e 100644 --- a/blockchain/pool_test.go +++ b/blockchain/pool_test.go @@ -5,6 +5,7 @@ import ( "testing" "time" + "github.com/tendermint/tendermint/p2p" "github.com/tendermint/tendermint/types" cmn "github.com/tendermint/tmlibs/common" "github.com/tendermint/tmlibs/log" @@ -15,14 +16,14 @@ func init() { } type testPeer struct { - id string + id p2p.ID height int64 } -func makePeers(numPeers int, minHeight, maxHeight int64) map[string]testPeer { - peers := make(map[string]testPeer, numPeers) +func makePeers(numPeers int, minHeight, maxHeight int64) map[p2p.ID]testPeer { + peers := make(map[p2p.ID]testPeer, numPeers) for i := 0; i < numPeers; i++ { - peerID := cmn.RandStr(12) + peerID := p2p.ID(cmn.RandStr(12)) height := minHeight + rand.Int63n(maxHeight-minHeight) peers[peerID] = testPeer{peerID, height} } @@ -32,7 +33,7 @@ func makePeers(numPeers int, minHeight, maxHeight int64) map[string]testPeer { func TestBasic(t *testing.T) { start := int64(42) peers := makePeers(10, start+1, 1000) - timeoutsCh := make(chan string, 100) + timeoutsCh := make(chan p2p.ID, 100) requestsCh := make(chan BlockRequest, 100) pool := NewBlockPool(start, requestsCh, timeoutsCh) pool.SetLogger(log.TestingLogger()) @@ -89,7 +90,7 @@ func TestBasic(t *testing.T) { func TestTimeout(t *testing.T) { start := int64(42) peers := makePeers(10, start+1, 1000) - timeoutsCh := make(chan string, 100) + timeoutsCh := make(chan p2p.ID, 100) requestsCh := make(chan BlockRequest, 100) pool := NewBlockPool(start, requestsCh, timeoutsCh) pool.SetLogger(log.TestingLogger()) @@ -127,7 +128,7 @@ func TestTimeout(t *testing.T) { // Pull from channels counter := 0 - timedOut := map[string]struct{}{} + timedOut := map[p2p.ID]struct{}{} for { select { case peerID := <-timeoutsCh: diff --git a/blockchain/reactor.go b/blockchain/reactor.go index d4b803dd6..701e04f68 100644 --- a/blockchain/reactor.go +++ b/blockchain/reactor.go @@ -52,7 +52,7 @@ type BlockchainReactor struct { pool *BlockPool fastSync bool requestsCh chan BlockRequest - timeoutsCh chan string + timeoutsCh chan p2p.ID } // NewBlockchainReactor returns new reactor instance. @@ -62,7 +62,7 @@ func NewBlockchainReactor(state sm.State, blockExec *sm.BlockExecutor, store *Bl } requestsCh := make(chan BlockRequest, defaultChannelCapacity) - timeoutsCh := make(chan string, defaultChannelCapacity) + timeoutsCh := make(chan p2p.ID, defaultChannelCapacity) pool := NewBlockPool( store.Height()+1, requestsCh, @@ -131,7 +131,7 @@ func (bcR *BlockchainReactor) AddPeer(peer p2p.Peer) { // RemovePeer implements Reactor by removing peer from the pool. func (bcR *BlockchainReactor) RemovePeer(peer p2p.Peer, reason interface{}) { - bcR.pool.RemovePeer(peer.Key()) + bcR.pool.RemovePeer(peer.ID()) } // respondToPeer loads a block and sends it to the requesting peer, @@ -170,7 +170,7 @@ func (bcR *BlockchainReactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) } case *bcBlockResponseMessage: // Got a block. - bcR.pool.AddBlock(src.Key(), msg.Block, len(msgBytes)) + bcR.pool.AddBlock(src.ID(), msg.Block, len(msgBytes)) case *bcStatusRequestMessage: // Send peer our state. queued := src.TrySend(BlockchainChannel, struct{ BlockchainMessage }{&bcStatusResponseMessage{bcR.store.Height()}}) @@ -179,7 +179,7 @@ func (bcR *BlockchainReactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) } case *bcStatusResponseMessage: // Got a peer status. Unverified. - bcR.pool.SetPeerHeight(src.Key(), msg.Height) + bcR.pool.SetPeerHeight(src.ID(), msg.Height) default: bcR.Logger.Error(cmn.Fmt("Unknown message type %v", reflect.TypeOf(msg))) } diff --git a/blockchain/reactor_test.go b/blockchain/reactor_test.go index fcb8a6f86..06f6c36c5 100644 --- a/blockchain/reactor_test.go +++ b/blockchain/reactor_test.go @@ -55,7 +55,7 @@ func TestNoBlockMessageResponse(t *testing.T) { defer bcr.Stop() // Add some peers in - peer := newbcrTestPeer(cmn.RandStr(12)) + peer := newbcrTestPeer(p2p.ID(cmn.RandStr(12))) bcr.AddPeer(peer) chID := byte(0x01) @@ -113,16 +113,16 @@ func makeBlock(height int64, state sm.State) *types.Block { // The Test peer type bcrTestPeer struct { cmn.Service - key string - ch chan interface{} + id p2p.ID + ch chan interface{} } var _ p2p.Peer = (*bcrTestPeer)(nil) -func newbcrTestPeer(key string) *bcrTestPeer { +func newbcrTestPeer(id p2p.ID) *bcrTestPeer { return &bcrTestPeer{ Service: cmn.NewBaseService(nil, "bcrTestPeer", nil), - key: key, + id: id, ch: make(chan interface{}, 2), } } @@ -142,9 +142,9 @@ func (tp *bcrTestPeer) TrySend(chID byte, value interface{}) bool { } func (tp *bcrTestPeer) Send(chID byte, data interface{}) bool { return tp.TrySend(chID, data) } -func (tp *bcrTestPeer) NodeInfo() *p2p.NodeInfo { return nil } +func (tp *bcrTestPeer) NodeInfo() p2p.NodeInfo { return p2p.NodeInfo{} } func (tp *bcrTestPeer) Status() p2p.ConnectionStatus { return p2p.ConnectionStatus{} } -func (tp *bcrTestPeer) Key() string { return tp.key } +func (tp *bcrTestPeer) ID() p2p.ID { return tp.id } func (tp *bcrTestPeer) IsOutbound() bool { return false } func (tp *bcrTestPeer) IsPersistent() bool { return true } func (tp *bcrTestPeer) Get(s string) interface{} { return s } diff --git a/cmd/tendermint/commands/run_node.go b/cmd/tendermint/commands/run_node.go index 0f37bb319..0eb7a4259 100644 --- a/cmd/tendermint/commands/run_node.go +++ b/cmd/tendermint/commands/run_node.go @@ -29,6 +29,7 @@ func AddNodeFlags(cmd *cobra.Command) { // p2p flags cmd.Flags().String("p2p.laddr", config.P2P.ListenAddress, "Node listen address. (0.0.0.0:0 means any interface, any port)") cmd.Flags().String("p2p.seeds", config.P2P.Seeds, "Comma delimited host:port seed nodes") + cmd.Flags().String("p2p.persistent_peers", config.P2P.PersistentPeers, "Comma delimited host:port persistent peers") cmd.Flags().Bool("p2p.skip_upnp", config.P2P.SkipUPNP, "Skip UPNP configuration") cmd.Flags().Bool("p2p.pex", config.P2P.PexReactor, "Enable/disable Peer-Exchange") diff --git a/config/config.go b/config/config.go index 6689adfcc..f93b79241 100644 --- a/config/config.go +++ b/config/config.go @@ -20,10 +20,12 @@ var ( defaultConfigFileName = "config.toml" defaultGenesisJSONName = "genesis.json" defaultPrivValName = "priv_validator.json" + defaultNodeKeyName = "node_key.json" defaultConfigFilePath = filepath.Join(defaultConfigDir, defaultConfigFileName) defaultGenesisJSONPath = filepath.Join(defaultConfigDir, defaultGenesisJSONName) defaultPrivValPath = filepath.Join(defaultConfigDir, defaultPrivValName) + defaultNodeKeyPath = filepath.Join(defaultConfigDir, defaultNodeKeyName) ) // Config defines the top level configuration for a Tendermint node @@ -92,6 +94,9 @@ type BaseConfig struct { // Path to the JSON file containing the private key to use as a validator in the consensus protocol PrivValidator string `mapstructure:"priv_validator_file"` + // A JSON file containing the private key to use for p2p authenticated encryption + NodeKey string `mapstructure:"node_key_file"` + // A custom human readable name for this node Moniker string `mapstructure:"moniker"` @@ -133,6 +138,7 @@ func DefaultBaseConfig() BaseConfig { return BaseConfig{ Genesis: defaultGenesisJSONPath, PrivValidator: defaultPrivValPath, + NodeKey: defaultNodeKeyPath, Moniker: defaultMoniker, ProxyApp: "tcp://127.0.0.1:46658", ABCI: "socket", @@ -165,6 +171,11 @@ func (b BaseConfig) PrivValidatorFile() string { return rootify(b.PrivValidator, b.RootDir) } +// NodeKeyFile returns the full path to the node_key.json file +func (b BaseConfig) NodeKeyFile() string { + return rootify(b.NodeKey, b.RootDir) +} + // DBDir returns the full path to the database directory func (b BaseConfig) DBDir() string { return rootify(b.DBPath, b.RootDir) @@ -194,7 +205,7 @@ type RPCConfig struct { // NOTE: This server only supports /broadcast_tx_commit GRPCListenAddress string `mapstructure:"grpc_laddr"` - // Activate unsafe RPC commands like /dial_seeds and /unsafe_flush_mempool + // Activate unsafe RPC commands like /dial_persistent_peers and /unsafe_flush_mempool Unsafe bool `mapstructure:"unsafe"` } @@ -227,8 +238,13 @@ type P2PConfig struct { ListenAddress string `mapstructure:"laddr"` // Comma separated list of seed nodes to connect to + // We only use these if we can’t connect to peers in the addrbook Seeds string `mapstructure:"seeds"` + // Comma separated list of persistent peers to connect to + // We always connect to these + PersistentPeers string `mapstructure:"persistent_peers"` + // Skip UPNP port forwarding SkipUPNP bool `mapstructure:"skip_upnp"` diff --git a/config/toml.go b/config/toml.go index 92ceb7de4..e7bd26103 100644 --- a/config/toml.go +++ b/config/toml.go @@ -87,6 +87,9 @@ genesis_file = "{{ .BaseConfig.Genesis }}" # Path to the JSON file containing the private key to use as a validator in the consensus protocol priv_validator_file = "{{ .BaseConfig.PrivValidator }}" +# Path to the JSON file containing the private key to use for node authentication in the p2p protocol +node_key_file = "{{ .BaseConfig.NodeKey}}" + # Mechanism to connect to the ABCI application: socket | grpc abci = "{{ .BaseConfig.ABCI }}" @@ -121,6 +124,9 @@ laddr = "{{ .P2P.ListenAddress }}" # Comma separated list of seed nodes to connect to seeds = "" +# Comma separated list of nodes to keep persistent connections to +persistent_peers = "" + # Path to address book addr_book_file = "{{ .P2P.AddrBook }}" diff --git a/consensus/reactor.go b/consensus/reactor.go index 9b3393e94..3f6ab506c 100644 --- a/consensus/reactor.go +++ b/consensus/reactor.go @@ -205,7 +205,7 @@ func (conR *ConsensusReactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) return } // Peer claims to have a maj23 for some BlockID at H,R,S, - votes.SetPeerMaj23(msg.Round, msg.Type, ps.Peer.Key(), msg.BlockID) + votes.SetPeerMaj23(msg.Round, msg.Type, ps.Peer.ID(), msg.BlockID) // Respond with a VoteSetBitsMessage showing which votes we have. // (and consequently shows which we don't have) var ourVotes *cmn.BitArray @@ -242,12 +242,12 @@ func (conR *ConsensusReactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) switch msg := msg.(type) { case *ProposalMessage: ps.SetHasProposal(msg.Proposal) - conR.conS.peerMsgQueue <- msgInfo{msg, src.Key()} + conR.conS.peerMsgQueue <- msgInfo{msg, src.ID()} case *ProposalPOLMessage: ps.ApplyProposalPOLMessage(msg) case *BlockPartMessage: ps.SetHasProposalBlockPart(msg.Height, msg.Round, msg.Part.Index) - conR.conS.peerMsgQueue <- msgInfo{msg, src.Key()} + conR.conS.peerMsgQueue <- msgInfo{msg, src.ID()} default: conR.Logger.Error(cmn.Fmt("Unknown message type %v", reflect.TypeOf(msg))) } @@ -267,7 +267,7 @@ func (conR *ConsensusReactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) ps.EnsureVoteBitArrays(height-1, lastCommitSize) ps.SetHasVote(msg.Vote) - cs.peerMsgQueue <- msgInfo{msg, src.Key()} + cs.peerMsgQueue <- msgInfo{msg, src.ID()} default: // don't punish (leave room for soft upgrades) @@ -1200,7 +1200,7 @@ func (ps *PeerState) StringIndented(indent string) string { %s Key %v %s PRS %v %s}`, - indent, ps.Peer.Key(), + indent, ps.Peer.ID(), indent, ps.PeerRoundState.StringIndented(indent+" "), indent) } diff --git a/consensus/replay.go b/consensus/replay.go index 784e8bd6e..881571077 100644 --- a/consensus/replay.go +++ b/consensus/replay.go @@ -61,21 +61,21 @@ func (cs *ConsensusState) readReplayMessage(msg *TimedWALMessage, newStepCh chan } } case msgInfo: - peerKey := m.PeerKey - if peerKey == "" { - peerKey = "local" + peerID := m.PeerID + if peerID == "" { + peerID = "local" } switch msg := m.Msg.(type) { case *ProposalMessage: p := msg.Proposal cs.Logger.Info("Replay: Proposal", "height", p.Height, "round", p.Round, "header", - p.BlockPartsHeader, "pol", p.POLRound, "peer", peerKey) + p.BlockPartsHeader, "pol", p.POLRound, "peer", peerID) case *BlockPartMessage: - cs.Logger.Info("Replay: BlockPart", "height", msg.Height, "round", msg.Round, "peer", peerKey) + cs.Logger.Info("Replay: BlockPart", "height", msg.Height, "round", msg.Round, "peer", peerID) case *VoteMessage: v := msg.Vote cs.Logger.Info("Replay: Vote", "height", v.Height, "round", v.Round, "type", v.Type, - "blockID", v.BlockID, "peer", peerKey) + "blockID", v.BlockID, "peer", peerID) } cs.handleMsg(m) diff --git a/consensus/state.go b/consensus/state.go index 518d81c58..56070b03a 100644 --- a/consensus/state.go +++ b/consensus/state.go @@ -17,6 +17,7 @@ import ( cfg "github.com/tendermint/tendermint/config" cstypes "github.com/tendermint/tendermint/consensus/types" + "github.com/tendermint/tendermint/p2p" sm "github.com/tendermint/tendermint/state" "github.com/tendermint/tendermint/types" ) @@ -46,8 +47,8 @@ var ( // msgs from the reactor which may update the state type msgInfo struct { - Msg ConsensusMessage `json:"msg"` - PeerKey string `json:"peer_key"` + Msg ConsensusMessage `json:"msg"` + PeerID p2p.ID `json:"peer_key"` } // internally generated messages which may update the state @@ -303,17 +304,17 @@ func (cs *ConsensusState) OpenWAL(walFile string) (WAL, error) { //------------------------------------------------------------ // Public interface for passing messages into the consensus state, possibly causing a state transition. -// If peerKey == "", the msg is considered internal. +// If peerID == "", the msg is considered internal. // Messages are added to the appropriate queue (peer or internal). // If the queue is full, the function may block. // TODO: should these return anything or let callers just use events? // AddVote inputs a vote. -func (cs *ConsensusState) AddVote(vote *types.Vote, peerKey string) (added bool, err error) { - if peerKey == "" { +func (cs *ConsensusState) AddVote(vote *types.Vote, peerID p2p.ID) (added bool, err error) { + if peerID == "" { cs.internalMsgQueue <- msgInfo{&VoteMessage{vote}, ""} } else { - cs.peerMsgQueue <- msgInfo{&VoteMessage{vote}, peerKey} + cs.peerMsgQueue <- msgInfo{&VoteMessage{vote}, peerID} } // TODO: wait for event?! @@ -321,12 +322,12 @@ func (cs *ConsensusState) AddVote(vote *types.Vote, peerKey string) (added bool, } // SetProposal inputs a proposal. -func (cs *ConsensusState) SetProposal(proposal *types.Proposal, peerKey string) error { +func (cs *ConsensusState) SetProposal(proposal *types.Proposal, peerID p2p.ID) error { - if peerKey == "" { + if peerID == "" { cs.internalMsgQueue <- msgInfo{&ProposalMessage{proposal}, ""} } else { - cs.peerMsgQueue <- msgInfo{&ProposalMessage{proposal}, peerKey} + cs.peerMsgQueue <- msgInfo{&ProposalMessage{proposal}, peerID} } // TODO: wait for event?! @@ -334,12 +335,12 @@ func (cs *ConsensusState) SetProposal(proposal *types.Proposal, peerKey string) } // AddProposalBlockPart inputs a part of the proposal block. -func (cs *ConsensusState) AddProposalBlockPart(height int64, round int, part *types.Part, peerKey string) error { +func (cs *ConsensusState) AddProposalBlockPart(height int64, round int, part *types.Part, peerID p2p.ID) error { - if peerKey == "" { + if peerID == "" { cs.internalMsgQueue <- msgInfo{&BlockPartMessage{height, round, part}, ""} } else { - cs.peerMsgQueue <- msgInfo{&BlockPartMessage{height, round, part}, peerKey} + cs.peerMsgQueue <- msgInfo{&BlockPartMessage{height, round, part}, peerID} } // TODO: wait for event?! @@ -347,13 +348,13 @@ func (cs *ConsensusState) AddProposalBlockPart(height int64, round int, part *ty } // SetProposalAndBlock inputs the proposal and all block parts. -func (cs *ConsensusState) SetProposalAndBlock(proposal *types.Proposal, block *types.Block, parts *types.PartSet, peerKey string) error { - if err := cs.SetProposal(proposal, peerKey); err != nil { +func (cs *ConsensusState) SetProposalAndBlock(proposal *types.Proposal, block *types.Block, parts *types.PartSet, peerID p2p.ID) error { + if err := cs.SetProposal(proposal, peerID); err != nil { return err } for i := 0; i < parts.Total(); i++ { part := parts.GetPart(i) - if err := cs.AddProposalBlockPart(proposal.Height, proposal.Round, part, peerKey); err != nil { + if err := cs.AddProposalBlockPart(proposal.Height, proposal.Round, part, peerID); err != nil { return err } } @@ -561,7 +562,7 @@ func (cs *ConsensusState) handleMsg(mi msgInfo) { defer cs.mtx.Unlock() var err error - msg, peerKey := mi.Msg, mi.PeerKey + msg, peerID := mi.Msg, mi.PeerID switch msg := msg.(type) { case *ProposalMessage: // will not cause transition. @@ -569,14 +570,14 @@ func (cs *ConsensusState) handleMsg(mi msgInfo) { err = cs.setProposal(msg.Proposal) case *BlockPartMessage: // if the proposal is complete, we'll enterPrevote or tryFinalizeCommit - _, err = cs.addProposalBlockPart(msg.Height, msg.Part, peerKey != "") + _, err = cs.addProposalBlockPart(msg.Height, msg.Part, peerID != "") if err != nil && msg.Round != cs.Round { err = nil } case *VoteMessage: // attempt to add the vote and dupeout the validator if its a duplicate signature // if the vote gives us a 2/3-any or 2/3-one, we transition - err := cs.tryAddVote(msg.Vote, peerKey) + err := cs.tryAddVote(msg.Vote, peerID) if err == ErrAddingVote { // TODO: punish peer } @@ -591,7 +592,7 @@ func (cs *ConsensusState) handleMsg(mi msgInfo) { cs.Logger.Error("Unknown msg type", reflect.TypeOf(msg)) } if err != nil { - cs.Logger.Error("Error with msg", "type", reflect.TypeOf(msg), "peer", peerKey, "err", err, "msg", msg) + cs.Logger.Error("Error with msg", "type", reflect.TypeOf(msg), "peer", peerID, "err", err, "msg", msg) } } @@ -1308,8 +1309,8 @@ func (cs *ConsensusState) addProposalBlockPart(height int64, part *types.Part, v } // Attempt to add the vote. if its a duplicate signature, dupeout the validator -func (cs *ConsensusState) tryAddVote(vote *types.Vote, peerKey string) error { - _, err := cs.addVote(vote, peerKey) +func (cs *ConsensusState) tryAddVote(vote *types.Vote, peerID p2p.ID) error { + _, err := cs.addVote(vote, peerID) if err != nil { // If the vote height is off, we'll just ignore it, // But if it's a conflicting sig, add it to the cs.evpool. @@ -1335,7 +1336,7 @@ func (cs *ConsensusState) tryAddVote(vote *types.Vote, peerKey string) error { //----------------------------------------------------------------------------- -func (cs *ConsensusState) addVote(vote *types.Vote, peerKey string) (added bool, err error) { +func (cs *ConsensusState) addVote(vote *types.Vote, peerID p2p.ID) (added bool, err error) { cs.Logger.Debug("addVote", "voteHeight", vote.Height, "voteType", vote.Type, "valIndex", vote.ValidatorIndex, "csHeight", cs.Height) // A precommit for the previous height? @@ -1365,7 +1366,7 @@ func (cs *ConsensusState) addVote(vote *types.Vote, peerKey string) (added bool, // A prevote/precommit for this height? if vote.Height == cs.Height { height := cs.Height - added, err = cs.Votes.AddVote(vote, peerKey) + added, err = cs.Votes.AddVote(vote, peerID) if added { cs.eventBus.PublishEventVote(types.EventDataVote{vote}) diff --git a/consensus/types/height_vote_set.go b/consensus/types/height_vote_set.go index 0a0a25fea..1435cf421 100644 --- a/consensus/types/height_vote_set.go +++ b/consensus/types/height_vote_set.go @@ -4,6 +4,7 @@ import ( "strings" "sync" + "github.com/tendermint/tendermint/p2p" "github.com/tendermint/tendermint/types" cmn "github.com/tendermint/tmlibs/common" ) @@ -35,7 +36,7 @@ type HeightVoteSet struct { mtx sync.Mutex round int // max tracked round roundVoteSets map[int]RoundVoteSet // keys: [0...round] - peerCatchupRounds map[string][]int // keys: peer.Key; values: at most 2 rounds + peerCatchupRounds map[p2p.ID][]int // keys: peer.ID; values: at most 2 rounds } func NewHeightVoteSet(chainID string, height int64, valSet *types.ValidatorSet) *HeightVoteSet { @@ -53,7 +54,7 @@ func (hvs *HeightVoteSet) Reset(height int64, valSet *types.ValidatorSet) { hvs.height = height hvs.valSet = valSet hvs.roundVoteSets = make(map[int]RoundVoteSet) - hvs.peerCatchupRounds = make(map[string][]int) + hvs.peerCatchupRounds = make(map[p2p.ID][]int) hvs.addRound(0) hvs.round = 0 @@ -101,8 +102,8 @@ func (hvs *HeightVoteSet) addRound(round int) { } // Duplicate votes return added=false, err=nil. -// By convention, peerKey is "" if origin is self. -func (hvs *HeightVoteSet) AddVote(vote *types.Vote, peerKey string) (added bool, err error) { +// By convention, peerID is "" if origin is self. +func (hvs *HeightVoteSet) AddVote(vote *types.Vote, peerID p2p.ID) (added bool, err error) { hvs.mtx.Lock() defer hvs.mtx.Unlock() if !types.IsVoteTypeValid(vote.Type) { @@ -110,10 +111,10 @@ func (hvs *HeightVoteSet) AddVote(vote *types.Vote, peerKey string) (added bool, } voteSet := hvs.getVoteSet(vote.Round, vote.Type) if voteSet == nil { - if rndz := hvs.peerCatchupRounds[peerKey]; len(rndz) < 2 { + if rndz := hvs.peerCatchupRounds[peerID]; len(rndz) < 2 { hvs.addRound(vote.Round) voteSet = hvs.getVoteSet(vote.Round, vote.Type) - hvs.peerCatchupRounds[peerKey] = append(rndz, vote.Round) + hvs.peerCatchupRounds[peerID] = append(rndz, vote.Round) } else { // Peer has sent a vote that does not match our round, // for more than one round. Bad peer! @@ -206,7 +207,7 @@ func (hvs *HeightVoteSet) StringIndented(indent string) string { // NOTE: if there are too many peers, or too much peer churn, // this can cause memory issues. // TODO: implement ability to remove peers too -func (hvs *HeightVoteSet) SetPeerMaj23(round int, type_ byte, peerID string, blockID types.BlockID) { +func (hvs *HeightVoteSet) SetPeerMaj23(round int, type_ byte, peerID p2p.ID, blockID types.BlockID) { hvs.mtx.Lock() defer hvs.mtx.Unlock() if !types.IsVoteTypeValid(type_) { diff --git a/consensus/wal.go b/consensus/wal.go index dfbef8790..88218940d 100644 --- a/consensus/wal.go +++ b/consensus/wal.go @@ -121,7 +121,7 @@ func (wal *baseWAL) Save(msg WALMessage) { if wal.light { // in light mode we only write new steps, timeouts, and our own votes (no proposals, block parts) if mi, ok := msg.(msgInfo); ok { - if mi.PeerKey != "" { + if mi.PeerID != "" { return } } diff --git a/docs/deploy-testnets.rst b/docs/deploy-testnets.rst index 658330fe5..f59d4b650 100644 --- a/docs/deploy-testnets.rst +++ b/docs/deploy-testnets.rst @@ -24,13 +24,13 @@ Here are the steps to setting up a testnet manually: ``tendermint gen_validator`` 4) Compile a list of public keys for each validator into a ``genesis.json`` file. -5) Run ``tendermint node --p2p.seeds=< seed addresses >`` on each node, - where ``< seed addresses >`` is a comma separated list of the IP:PORT +5) Run ``tendermint node --p2p.persistent_peers=< peer addresses >`` on each node, + where ``< peer addresses >`` is a comma separated list of the IP:PORT combination for each node. The default port for Tendermint is ``46656``. Thus, if the IP addresses of your nodes were ``192.168.0.1, 192.168.0.2, 192.168.0.3, 192.168.0.4``, the command would look like: - ``tendermint node --p2p.seeds=192.168.0.1:46656,192.168.0.2:46656,192.168.0.3:46656,192.168.0.4:46656``. + ``tendermint node --p2p.persistent_peers=192.168.0.1:46656,192.168.0.2:46656,192.168.0.3:46656,192.168.0.4:46656``. After a few seconds, all the nodes should connect to eachother and start making blocks! For more information, see the Tendermint Networks section diff --git a/docs/specification/configuration.rst b/docs/specification/configuration.rst index 37359885c..cf6002908 100644 --- a/docs/specification/configuration.rst +++ b/docs/specification/configuration.rst @@ -88,6 +88,9 @@ like the file below, however, double check by inspecting the # Comma separated list of seed nodes to connect to seeds = "" + # Comma separated list of nodes to keep persistent connections to + persistent_peers = "" + # Path to address book addr_book_file = "addrbook.json" diff --git a/docs/specification/rpc.rst b/docs/specification/rpc.rst index f8e9bd3a9..7df394d77 100644 --- a/docs/specification/rpc.rst +++ b/docs/specification/rpc.rst @@ -112,6 +112,7 @@ An HTTP Get request to the root RPC endpoint (e.g. http://localhost:46657/broadcast_tx_sync?tx=_ http://localhost:46657/commit?height=_ http://localhost:46657/dial_seeds?seeds=_ + http://localhost:46657/dial_peers?peers=_&persistent=_ http://localhost:46657/subscribe?event=_ http://localhost:46657/tx?hash=_&prove=_ http://localhost:46657/unsafe_start_cpu_profiler?filename=_ diff --git a/docs/using-tendermint.rst b/docs/using-tendermint.rst index f4b0cb089..2fab60f8f 100644 --- a/docs/using-tendermint.rst +++ b/docs/using-tendermint.rst @@ -326,7 +326,9 @@ Peers ~~~~~ To connect to peers on start-up, specify them in the ``$TMHOME/config/config.toml`` or -on the command line. +on the command line. Use `seeds` to specify seed nodes from which you can get many other +peer addresses, and ``persistent_peers`` to specify peers that your node will maintain +persistent connections with. For instance, @@ -335,26 +337,35 @@ For instance, tendermint node --p2p.seeds "1.2.3.4:46656,5.6.7.8:46656" Alternatively, you can use the ``/dial_seeds`` endpoint of the RPC to -specify peers for a running node to connect to: +specify seeds for a running node to connect to: :: - curl --data-urlencode "seeds=[\"1.2.3.4:46656\",\"5.6.7.8:46656\"]" localhost:46657/dial_seeds + curl 'localhost:46657/dial_seeds?seeds=\["1.2.3.4:46656","5.6.7.8:46656"\]' -Additionally, the peer-exchange protocol can be enabled using the -``--pex`` flag, though this feature is `still under -development `__. If -``--pex`` is enabled, peers will gossip about known peers and form a -more resilient network. +Note, if the peer-exchange protocol (PEX) is enabled (default), you should not +normally need seeds after the first start. Peers will be gossipping about known +peers and forming a network, storing peer addresses in the addrbook. + +If you want Tendermint to connect to specific set of addresses and maintain a +persistent connection with each, you can use the ``--p2p.persistent_peers`` +flag or the corresponding setting in the ``config.toml`` or the +``/dial_peers`` RPC endpoint to do it without stopping Tendermint +core instance. + +:: + + tendermint node --p2p.persistent_peers "10.11.12.13:46656,10.11.12.14:46656" + curl 'localhost:46657/dial_peers?persistent=true&peers=\["1.2.3.4:46656","5.6.7.8:46656"\]' Adding a Non-Validator ~~~~~~~~~~~~~~~~~~~~~~ Adding a non-validator is simple. Just copy the original ``genesis.json`` to ``~/.tendermint/config`` on the new machine and start the -node, specifying seeds as necessary. If no seeds are specified, the node -won't make any blocks, because it's not a validator, and it won't hear -about any blocks, because it's not connected to the other peer. +node, specifying seeds or persistent peers as necessary. If no seeds or persistent +peers are specified, the node won't make any blocks, because it's not a validator, +and it won't hear about any blocks, because it's not connected to the other peer. Adding a Validator ~~~~~~~~~~~~~~~~~~ @@ -425,7 +436,7 @@ and the new ``priv_validator.json`` to the ``~/.tendermint/config`` on a new machine. Now run ``tendermint node`` on both machines, and use either -``--p2p.seeds`` or the ``/dial_seeds`` to get them to peer up. They +``--p2p.persistent_peers`` or the ``/dial_peers`` to get them to peer up. They should start making blocks, and will only continue to do so as long as both of them are online. diff --git a/node/node.go b/node/node.go index fde8e1e0a..6c816a9ef 100644 --- a/node/node.go +++ b/node/node.go @@ -96,7 +96,6 @@ type Node struct { privValidator types.PrivValidator // local node's validator key // network - privKey crypto.PrivKeyEd25519 // local node's p2p key sw *p2p.Switch // p2p connections addrBook *p2p.AddrBook // known peers trustMetricStore *trust.TrustMetricStore // trust metrics for all peers @@ -170,9 +169,6 @@ func NewNode(config *cfg.Config, // reload the state (it may have been updated by the handshake) state = sm.LoadState(stateDB) - // Generate node PrivKey - privKey := crypto.GenPrivKeyEd25519() - // Decide whether to fast-sync or not // We don't fast-sync when the only validator is us. fastSync := config.FastSync @@ -255,7 +251,12 @@ func NewNode(config *cfg.Config, trustMetricStore = trust.NewTrustMetricStore(trustHistoryDB, trust.DefaultConfig()) trustMetricStore.SetLogger(p2pLogger) - pexReactor := p2p.NewPEXReactor(addrBook) + var seeds []string + if config.P2P.Seeds != "" { + seeds = strings.Split(config.P2P.Seeds, ",") + } + pexReactor := p2p.NewPEXReactor(addrBook, + &p2p.PEXReactorConfig{Seeds: seeds}) pexReactor.SetLogger(p2pLogger) sw.AddReactor("PEX", pexReactor) } @@ -275,7 +276,7 @@ func NewNode(config *cfg.Config, } return nil }) - sw.SetPubKeyFilter(func(pubkey crypto.PubKeyEd25519) error { + sw.SetPubKeyFilter(func(pubkey crypto.PubKey) error { resQuery, err := proxyApp.Query().QuerySync(abci.RequestQuery{Path: cmn.Fmt("/p2p/filter/pubkey/%X", pubkey.Bytes())}) if err != nil { return err @@ -328,7 +329,6 @@ func NewNode(config *cfg.Config, genesisDoc: genDoc, privValidator: privValidator, - privKey: privKey, sw: sw, addrBook: addrBook, trustMetricStore: trustMetricStore, @@ -371,19 +371,26 @@ func (n *Node) OnStart() error { l := p2p.NewDefaultListener(protocol, address, n.config.P2P.SkipUPNP, n.Logger.With("module", "p2p")) n.sw.AddListener(l) + // Generate node PrivKey + // TODO: pass in like priv_val + nodeKey, err := p2p.LoadOrGenNodeKey(n.config.NodeKeyFile()) + if err != nil { + return err + } + n.Logger.Info("P2P Node ID", "ID", nodeKey.ID(), "file", n.config.NodeKeyFile()) + // Start the switch - n.sw.SetNodeInfo(n.makeNodeInfo()) - n.sw.SetNodePrivKey(n.privKey) + n.sw.SetNodeInfo(n.makeNodeInfo(nodeKey.PubKey())) + n.sw.SetNodeKey(nodeKey) err = n.sw.Start() if err != nil { return err } - // If seeds exist, add them to the address book and dial out - if n.config.P2P.Seeds != "" { - // dial out - seeds := strings.Split(n.config.P2P.Seeds, ",") - if err := n.DialSeeds(seeds); err != nil { + // Always connect to persistent peers + if n.config.P2P.PersistentPeers != "" { + err = n.sw.DialPeersAsync(n.addrBook, strings.Split(n.config.P2P.PersistentPeers, ","), true) + if err != nil { return err } } @@ -534,16 +541,16 @@ func (n *Node) ProxyApp() proxy.AppConns { return n.proxyApp } -func (n *Node) makeNodeInfo() *p2p.NodeInfo { +func (n *Node) makeNodeInfo(pubKey crypto.PubKey) p2p.NodeInfo { txIndexerStatus := "on" if _, ok := n.txIndexer.(*null.TxIndex); ok { txIndexerStatus = "off" } - nodeInfo := &p2p.NodeInfo{ - PubKey: n.privKey.PubKey().Unwrap().(crypto.PubKeyEd25519), - Moniker: n.config.Moniker, + nodeInfo := p2p.NodeInfo{ + PubKey: pubKey, Network: n.genesisDoc.ChainID, Version: version.Version, + Moniker: n.config.Moniker, Other: []string{ cmn.Fmt("wire_version=%v", wire.Version), cmn.Fmt("p2p_version=%v", p2p.Version), @@ -571,15 +578,10 @@ func (n *Node) makeNodeInfo() *p2p.NodeInfo { //------------------------------------------------------------------------------ // NodeInfo returns the Node's Info from the Switch. -func (n *Node) NodeInfo() *p2p.NodeInfo { +func (n *Node) NodeInfo() p2p.NodeInfo { return n.sw.NodeInfo() } -// DialSeeds dials the given seeds on the Switch. -func (n *Node) DialSeeds(seeds []string) error { - return n.sw.DialSeeds(n.addrBook, seeds) -} - //------------------------------------------------------------------------------ var ( diff --git a/p2p/README.md b/p2p/README.md index f386c5112..9a8ddc6c3 100644 --- a/p2p/README.md +++ b/p2p/README.md @@ -8,4 +8,4 @@ Docs: - [Peer](../docs/specification/new-spec/p2p/peer.md) for details on peer ID, handshakes, and peer exchange - [Node](../docs/specification/new-spec/p2p/node.md) for details about different types of nodes and how they should work - [Pex](../docs/specification/new-spec/p2p/pex.md) for details on peer discovery and exchange -- [Config](../docs/specification/new-spec/p2p/config.md) for details on some config option +- [Config](../docs/specification/new-spec/p2p/config.md) for details on some config option \ No newline at end of file diff --git a/p2p/addrbook.go b/p2p/addrbook.go index 8f924d129..7591c3074 100644 --- a/p2p/addrbook.go +++ b/p2p/addrbook.go @@ -5,6 +5,7 @@ package p2p import ( + "crypto/sha256" "encoding/binary" "encoding/json" "fmt" @@ -89,7 +90,7 @@ type AddrBook struct { mtx sync.Mutex rand *rand.Rand ourAddrs map[string]*NetAddress - addrLookup map[string]*knownAddress // new & old + addrLookup map[ID]*knownAddress // new & old bucketsOld []map[string]*knownAddress bucketsNew []map[string]*knownAddress nOld int @@ -104,7 +105,7 @@ func NewAddrBook(filePath string, routabilityStrict bool) *AddrBook { am := &AddrBook{ rand: rand.New(rand.NewSource(time.Now().UnixNano())), ourAddrs: make(map[string]*NetAddress), - addrLookup: make(map[string]*knownAddress), + addrLookup: make(map[ID]*knownAddress), filePath: filePath, routabilityStrict: routabilityStrict, } @@ -244,11 +245,11 @@ func (a *AddrBook) PickAddress(newBias int) *NetAddress { } // MarkGood marks the peer as good and moves it into an "old" bucket. -// XXX: we never call this! +// TODO: call this from somewhere func (a *AddrBook) MarkGood(addr *NetAddress) { a.mtx.Lock() defer a.mtx.Unlock() - ka := a.addrLookup[addr.String()] + ka := a.addrLookup[addr.ID] if ka == nil { return } @@ -262,7 +263,7 @@ func (a *AddrBook) MarkGood(addr *NetAddress) { func (a *AddrBook) MarkAttempt(addr *NetAddress) { a.mtx.Lock() defer a.mtx.Unlock() - ka := a.addrLookup[addr.String()] + ka := a.addrLookup[addr.ID] if ka == nil { return } @@ -279,11 +280,11 @@ func (a *AddrBook) MarkBad(addr *NetAddress) { func (a *AddrBook) RemoveAddress(addr *NetAddress) { a.mtx.Lock() defer a.mtx.Unlock() - ka := a.addrLookup[addr.String()] + ka := a.addrLookup[addr.ID] if ka == nil { return } - a.Logger.Info("Remove address from book", "addr", addr) + a.Logger.Info("Remove address from book", "addr", ka.Addr, "ID", ka.ID) a.removeFromAllBuckets(ka) } @@ -300,8 +301,8 @@ func (a *AddrBook) GetSelection() []*NetAddress { allAddr := make([]*NetAddress, a.size()) i := 0 - for _, v := range a.addrLookup { - allAddr[i] = v.Addr + for _, ka := range a.addrLookup { + allAddr[i] = ka.Addr i++ } @@ -388,7 +389,7 @@ func (a *AddrBook) loadFromFile(filePath string) bool { bucket := a.getBucket(ka.BucketType, bucketIndex) bucket[ka.Addr.String()] = ka } - a.addrLookup[ka.Addr.String()] = ka + a.addrLookup[ka.ID()] = ka if ka.BucketType == bucketTypeNew { a.nNew++ } else { @@ -466,7 +467,7 @@ func (a *AddrBook) addToNewBucket(ka *knownAddress, bucketIdx int) bool { } // Ensure in addrLookup - a.addrLookup[addrStr] = ka + a.addrLookup[ka.ID()] = ka return true } @@ -503,7 +504,7 @@ func (a *AddrBook) addToOldBucket(ka *knownAddress, bucketIdx int) bool { } // Ensure in addrLookup - a.addrLookup[addrStr] = ka + a.addrLookup[ka.ID()] = ka return true } @@ -521,7 +522,7 @@ func (a *AddrBook) removeFromBucket(ka *knownAddress, bucketType byte, bucketIdx } else { a.nOld-- } - delete(a.addrLookup, ka.Addr.String()) + delete(a.addrLookup, ka.ID()) } } @@ -536,7 +537,7 @@ func (a *AddrBook) removeFromAllBuckets(ka *knownAddress) { } else { a.nOld-- } - delete(a.addrLookup, ka.Addr.String()) + delete(a.addrLookup, ka.ID()) } func (a *AddrBook) pickOldest(bucketType byte, bucketIdx int) *knownAddress { @@ -559,7 +560,7 @@ func (a *AddrBook) addAddress(addr, src *NetAddress) error { return fmt.Errorf("Cannot add ourselves with address %v", addr) } - ka := a.addrLookup[addr.String()] + ka := a.addrLookup[addr.ID] if ka != nil { // Already old. @@ -768,6 +769,10 @@ func newKnownAddress(addr *NetAddress, src *NetAddress) *knownAddress { } } +func (ka *knownAddress) ID() ID { + return ka.Addr.ID +} + func (ka *knownAddress) isOld() bool { return ka.BucketType == bucketTypeOld } @@ -863,3 +868,15 @@ func (ka *knownAddress) isBad() bool { return false } + +//----------------------------------------------------------------------------- + +// doubleSha256 calculates sha256(sha256(b)) and returns the resulting bytes. +func doubleSha256(b []byte) []byte { + hasher := sha256.New() + hasher.Write(b) // nolint: errcheck, gas + sum := hasher.Sum(nil) + hasher.Reset() + hasher.Write(sum) // nolint: errcheck, gas + return hasher.Sum(nil) +} diff --git a/p2p/addrbook_test.go b/p2p/addrbook_test.go index d84c008ed..00051ae1f 100644 --- a/p2p/addrbook_test.go +++ b/p2p/addrbook_test.go @@ -1,12 +1,14 @@ package p2p import ( + "encoding/hex" "fmt" "io/ioutil" "math/rand" "testing" "github.com/stretchr/testify/assert" + cmn "github.com/tendermint/tmlibs/common" "github.com/tendermint/tmlibs/log" ) @@ -102,7 +104,7 @@ func TestAddrBookLookup(t *testing.T) { src := addrSrc.src book.AddAddress(addr, src) - ka := book.addrLookup[addr.String()] + ka := book.addrLookup[addr.ID] assert.NotNil(t, ka, "Expected to find KnownAddress %v but wasn't there.", addr) if !(ka.Addr.Equals(addr) && ka.Src.Equals(src)) { @@ -187,7 +189,9 @@ func randIPv4Address(t *testing.T) *NetAddress { rand.Intn(255), ) port := rand.Intn(65535-1) + 1 - addr, err := NewNetAddressString(fmt.Sprintf("%v:%v", ip, port)) + id := ID(hex.EncodeToString(cmn.RandBytes(IDByteLength))) + idAddr := IDAddressString(id, fmt.Sprintf("%v:%v", ip, port)) + addr, err := NewNetAddressString(idAddr) assert.Nil(t, err, "error generating rand network address") if addr.Routable() { return addr diff --git a/p2p/base_reactor.go b/p2p/base_reactor.go new file mode 100644 index 000000000..e8107d730 --- /dev/null +++ b/p2p/base_reactor.go @@ -0,0 +1,35 @@ +package p2p + +import cmn "github.com/tendermint/tmlibs/common" + +type Reactor interface { + cmn.Service // Start, Stop + + SetSwitch(*Switch) + GetChannels() []*ChannelDescriptor + AddPeer(peer Peer) + RemovePeer(peer Peer, reason interface{}) + Receive(chID byte, peer Peer, msgBytes []byte) // CONTRACT: msgBytes are not nil +} + +//-------------------------------------- + +type BaseReactor struct { + cmn.BaseService // Provides Start, Stop, .Quit + Switch *Switch +} + +func NewBaseReactor(name string, impl Reactor) *BaseReactor { + return &BaseReactor{ + BaseService: *cmn.NewBaseService(nil, name, impl), + Switch: nil, + } +} + +func (br *BaseReactor) SetSwitch(sw *Switch) { + br.Switch = sw +} +func (_ *BaseReactor) GetChannels() []*ChannelDescriptor { return nil } +func (_ *BaseReactor) AddPeer(peer Peer) {} +func (_ *BaseReactor) RemovePeer(peer Peer, reason interface{}) {} +func (_ *BaseReactor) Receive(chID byte, peer Peer, msgBytes []byte) {} diff --git a/p2p/connection.go b/p2p/connection.go index 626aeb10f..306eaf7eb 100644 --- a/p2p/connection.go +++ b/p2p/connection.go @@ -88,9 +88,6 @@ type MConnection struct { flushTimer *cmn.ThrottleTimer // flush writes as necessary but throttled. pingTimer *cmn.RepeatTimer // send pings periodically chStatsTimer *cmn.RepeatTimer // update channel stats periodically - - LocalAddress *NetAddress - RemoteAddress *NetAddress } // MConnConfig is a MConnection configuration. @@ -140,9 +137,6 @@ func NewMConnectionWithConfig(conn net.Conn, chDescs []*ChannelDescriptor, onRec onReceive: onReceive, onError: onError, config: config, - - LocalAddress: NewNetAddress(conn.LocalAddr()), - RemoteAddress: NewNetAddress(conn.RemoteAddr()), } // Create channels diff --git a/p2p/key.go b/p2p/key.go new file mode 100644 index 000000000..ea0f0b071 --- /dev/null +++ b/p2p/key.go @@ -0,0 +1,113 @@ +package p2p + +import ( + "bytes" + "encoding/hex" + "encoding/json" + "fmt" + "io/ioutil" + + crypto "github.com/tendermint/go-crypto" + cmn "github.com/tendermint/tmlibs/common" +) + +// ID is a hex-encoded crypto.Address +type ID string + +// IDByteLength is the length of a crypto.Address. Currently only 20. +// TODO: support other length addresses ? +const IDByteLength = 20 + +//------------------------------------------------------------------------------ +// Persistent peer ID +// TODO: encrypt on disk + +// NodeKey is the persistent peer key. +// It contains the nodes private key for authentication. +type NodeKey struct { + PrivKey crypto.PrivKey `json:"priv_key"` // our priv key +} + +// ID returns the peer's canonical ID - the hash of its public key. +func (nodeKey *NodeKey) ID() ID { + return PubKeyToID(nodeKey.PubKey()) +} + +// PubKey returns the peer's PubKey +func (nodeKey *NodeKey) PubKey() crypto.PubKey { + return nodeKey.PrivKey.PubKey() +} + +// PubKeyToID returns the ID corresponding to the given PubKey. +// It's the hex-encoding of the pubKey.Address(). +func PubKeyToID(pubKey crypto.PubKey) ID { + return ID(hex.EncodeToString(pubKey.Address())) +} + +// LoadOrGenNodeKey attempts to load the NodeKey from the given filePath. +// If the file does not exist, it generates and saves a new NodeKey. +func LoadOrGenNodeKey(filePath string) (*NodeKey, error) { + if cmn.FileExists(filePath) { + nodeKey, err := loadNodeKey(filePath) + if err != nil { + return nil, err + } + return nodeKey, nil + } else { + return genNodeKey(filePath) + } +} + +func loadNodeKey(filePath string) (*NodeKey, error) { + jsonBytes, err := ioutil.ReadFile(filePath) + if err != nil { + return nil, err + } + nodeKey := new(NodeKey) + err = json.Unmarshal(jsonBytes, nodeKey) + if err != nil { + return nil, fmt.Errorf("Error reading NodeKey from %v: %v\n", filePath, err) + } + return nodeKey, nil +} + +func genNodeKey(filePath string) (*NodeKey, error) { + privKey := crypto.GenPrivKeyEd25519().Wrap() + nodeKey := &NodeKey{ + PrivKey: privKey, + } + + jsonBytes, err := json.Marshal(nodeKey) + if err != nil { + return nil, err + } + err = ioutil.WriteFile(filePath, jsonBytes, 0600) + if err != nil { + return nil, err + } + return nodeKey, nil +} + +//------------------------------------------------------------------------------ + +// MakePoWTarget returns the big-endian encoding of 2^(targetBits - difficulty) - 1. +// It can be used as a Proof of Work target. +// NOTE: targetBits must be a multiple of 8 and difficulty must be less than targetBits. +func MakePoWTarget(difficulty, targetBits uint) []byte { + if targetBits%8 != 0 { + panic(fmt.Sprintf("targetBits (%d) not a multiple of 8", targetBits)) + } + if difficulty >= targetBits { + panic(fmt.Sprintf("difficulty (%d) >= targetBits (%d)", difficulty, targetBits)) + } + targetBytes := targetBits / 8 + zeroPrefixLen := (int(difficulty) / 8) + prefix := bytes.Repeat([]byte{0}, zeroPrefixLen) + mod := (difficulty % 8) + if mod > 0 { + nonZeroPrefix := byte(1<<(8-mod) - 1) + prefix = append(prefix, nonZeroPrefix) + } + tailLen := int(targetBytes) - len(prefix) + return append(prefix, bytes.Repeat([]byte{0xFF}, tailLen)...) +} diff --git a/p2p/key_test.go b/p2p/key_test.go new file mode 100644 index 000000000..c2e1f3e0e --- /dev/null +++ b/p2p/key_test.go @@ -0,0 +1,50 @@ +package p2p + +import ( + "bytes" + "os" + "path/filepath" + "testing" + + "github.com/stretchr/testify/assert" + cmn "github.com/tendermint/tmlibs/common" +) + +func TestLoadOrGenNodeKey(t *testing.T) { + filePath := filepath.Join(os.TempDir(), cmn.RandStr(12)+"_peer_id.json") + + nodeKey, err := LoadOrGenNodeKey(filePath) + assert.Nil(t, err) + + nodeKey2, err := LoadOrGenNodeKey(filePath) + assert.Nil(t, err) + + assert.Equal(t, nodeKey, nodeKey2) +} + +//---------------------------------------------------------- + +func padBytes(bz []byte, targetBytes int) []byte { + return append(bz, bytes.Repeat([]byte{0xFF}, targetBytes-len(bz))...) +} + +func TestPoWTarget(t *testing.T) { + + targetBytes := 20 + cases := []struct { + difficulty uint + target []byte + }{ + {0, padBytes([]byte{}, targetBytes)}, + {1, padBytes([]byte{127}, targetBytes)}, + {8, padBytes([]byte{0}, targetBytes)}, + {9, padBytes([]byte{0, 127}, targetBytes)}, + {10, padBytes([]byte{0, 63}, targetBytes)}, + {16, padBytes([]byte{0, 0}, targetBytes)}, + {17, padBytes([]byte{0, 0, 127}, targetBytes)}, + } + + for _, c := range cases { + assert.Equal(t, MakePoWTarget(c.difficulty, 20*8), c.target) + } +} diff --git a/p2p/netaddress.go b/p2p/netaddress.go index 41c2cc976..333d16e5d 100644 --- a/p2p/netaddress.go +++ b/p2p/netaddress.go @@ -5,6 +5,7 @@ package p2p import ( + "encoding/hex" "flag" "fmt" "net" @@ -12,41 +13,69 @@ import ( "strings" "time" + "github.com/pkg/errors" cmn "github.com/tendermint/tmlibs/common" ) // NetAddress defines information about a peer on the network -// including its IP address, and port. +// including its ID, IP address, and port. type NetAddress struct { + ID ID IP net.IP Port uint16 str string } +// IDAddressString returns id@hostPort. +func IDAddressString(id ID, hostPort string) string { + return fmt.Sprintf("%s@%s", id, hostPort) +} + // NewNetAddress returns a new NetAddress using the provided TCP // address. When testing, other net.Addr (except TCP) will result in // using 0.0.0.0:0. When normal run, other net.Addr (except TCP) will // panic. // TODO: socks proxies? -func NewNetAddress(addr net.Addr) *NetAddress { +func NewNetAddress(id ID, addr net.Addr) *NetAddress { tcpAddr, ok := addr.(*net.TCPAddr) if !ok { if flag.Lookup("test.v") == nil { // normal run cmn.PanicSanity(cmn.Fmt("Only TCPAddrs are supported. Got: %v", addr)) } else { // in testing - return NewNetAddressIPPort(net.IP("0.0.0.0"), 0) + netAddr := NewNetAddressIPPort(net.IP("0.0.0.0"), 0) + netAddr.ID = id + return netAddr } } ip := tcpAddr.IP port := uint16(tcpAddr.Port) - return NewNetAddressIPPort(ip, port) + netAddr := NewNetAddressIPPort(ip, port) + netAddr.ID = id + return netAddr } // NewNetAddressString returns a new NetAddress using the provided -// address in the form of "IP:Port". Also resolves the host if host -// is not an IP. +// address in the form of "ID@IP:Port", where the ID is optional. +// Also resolves the host if host is not an IP. func NewNetAddressString(addr string) (*NetAddress, error) { - host, portStr, err := net.SplitHostPort(removeProtocolIfDefined(addr)) + addr = removeProtocolIfDefined(addr) + + var id ID + spl := strings.Split(addr, "@") + if len(spl) == 2 { + idStr := spl[0] + idBytes, err := hex.DecodeString(idStr) + if err != nil { + return nil, errors.Wrap(err, fmt.Sprintf("Address (%s) contains invalid ID", addr)) + } + if len(idBytes) != IDByteLength { + return nil, fmt.Errorf("Address (%s) contains ID of invalid length (%d). Should be %d hex-encoded bytes", + addr, len(idBytes), IDByteLength) + } + id, addr = ID(idStr), spl[1] + } + + host, portStr, err := net.SplitHostPort(addr) if err != nil { return nil, err } @@ -68,6 +97,7 @@ func NewNetAddressString(addr string) (*NetAddress, error) { } na := NewNetAddressIPPort(ip, uint16(port)) + na.ID = id return na, nil } @@ -93,46 +123,54 @@ func NewNetAddressIPPort(ip net.IP, port uint16) *NetAddress { na := &NetAddress{ IP: ip, Port: port, - str: net.JoinHostPort( - ip.String(), - strconv.FormatUint(uint64(port), 10), - ), } return na } -// Equals reports whether na and other are the same addresses. +// Equals reports whether na and other are the same addresses, +// including their ID, IP, and Port. func (na *NetAddress) Equals(other interface{}) bool { if o, ok := other.(*NetAddress); ok { return na.String() == o.String() } - return false } -func (na *NetAddress) Less(other interface{}) bool { +// Same returns true is na has the same non-empty ID or DialString as other. +func (na *NetAddress) Same(other interface{}) bool { if o, ok := other.(*NetAddress); ok { - return na.String() < o.String() + if na.DialString() == o.DialString() { + return true + } + if na.ID != "" && na.ID == o.ID { + return true + } } - - cmn.PanicSanity("Cannot compare unequal types") return false } -// String representation. +// String representation: @: func (na *NetAddress) String() string { if na.str == "" { - na.str = net.JoinHostPort( - na.IP.String(), - strconv.FormatUint(uint64(na.Port), 10), - ) + addrStr := na.DialString() + if na.ID != "" { + addrStr = IDAddressString(na.ID, addrStr) + } + na.str = addrStr } return na.str } +func (na *NetAddress) DialString() string { + return net.JoinHostPort( + na.IP.String(), + strconv.FormatUint(uint64(na.Port), 10), + ) +} + // Dial calls net.Dial on the address. func (na *NetAddress) Dial() (net.Conn, error) { - conn, err := net.Dial("tcp", na.String()) + conn, err := net.Dial("tcp", na.DialString()) if err != nil { return nil, err } @@ -141,7 +179,7 @@ func (na *NetAddress) Dial() (net.Conn, error) { // DialTimeout calls net.DialTimeout on the address. func (na *NetAddress) DialTimeout(timeout time.Duration) (net.Conn, error) { - conn, err := net.DialTimeout("tcp", na.String(), timeout) + conn, err := net.DialTimeout("tcp", na.DialString(), timeout) if err != nil { return nil, err } diff --git a/p2p/netaddress_test.go b/p2p/netaddress_test.go index 137be090c..6c1930a2f 100644 --- a/p2p/netaddress_test.go +++ b/p2p/netaddress_test.go @@ -13,12 +13,12 @@ func TestNewNetAddress(t *testing.T) { tcpAddr, err := net.ResolveTCPAddr("tcp", "127.0.0.1:8080") require.Nil(err) - addr := NewNetAddress(tcpAddr) + addr := NewNetAddress("", tcpAddr) assert.Equal("127.0.0.1:8080", addr.String()) assert.NotPanics(func() { - NewNetAddress(&net.UDPAddr{IP: net.ParseIP("127.0.0.1"), Port: 8000}) + NewNetAddress("", &net.UDPAddr{IP: net.ParseIP("127.0.0.1"), Port: 8000}) }, "Calling NewNetAddress with UDPAddr should not panic in testing") } @@ -38,6 +38,23 @@ func TestNewNetAddressString(t *testing.T) { {"notahost:8080", "", false}, {"8082", "", false}, {"127.0.0:8080000", "", false}, + + {"deadbeef@127.0.0.1:8080", "", false}, + {"this-isnot-hex@127.0.0.1:8080", "", false}, + {"xxxxbeefdeadbeefdeadbeefdeadbeefdeadbeef@127.0.0.1:8080", "", false}, + {"deadbeefdeadbeefdeadbeefdeadbeefdeadbeef@127.0.0.1:8080", "deadbeefdeadbeefdeadbeefdeadbeefdeadbeef@127.0.0.1:8080", true}, + + {"tcp://deadbeef@127.0.0.1:8080", "", false}, + {"tcp://this-isnot-hex@127.0.0.1:8080", "", false}, + {"tcp://xxxxbeefdeadbeefdeadbeefdeadbeefdeadbeef@127.0.0.1:8080", "", false}, + {"tcp://deadbeefdeadbeefdeadbeefdeadbeefdeadbeef@127.0.0.1:8080", "deadbeefdeadbeefdeadbeefdeadbeefdeadbeef@127.0.0.1:8080", true}, + + {"tcp://@127.0.0.1:8080", "", false}, + {"tcp://@", "", false}, + {"", "", false}, + {"@", "", false}, + {" @", "", false}, + {" @ ", "", false}, } for _, tc := range testCases { diff --git a/p2p/peer.go b/p2p/peer.go index cc7f4927a..596b92168 100644 --- a/p2p/peer.go +++ b/p2p/peer.go @@ -17,10 +17,10 @@ import ( type Peer interface { cmn.Service - Key() string - IsOutbound() bool - IsPersistent() bool - NodeInfo() *NodeInfo + ID() ID // peer's cryptographic ID + IsOutbound() bool // did we dial the peer + IsPersistent() bool // do we redial this peer when we disconnect + NodeInfo() NodeInfo // peer's info Status() ConnectionStatus Send(byte, interface{}) bool @@ -30,9 +30,9 @@ type Peer interface { Get(string) interface{} } -// Peer could be marked as persistent, in which case you can use -// Redial function to reconnect. Note that inbound peers can't be -// made persistent. They should be made persistent on the other end. +//---------------------------------------------------------- + +// peer implements Peer. // // Before using a peer, you will need to perform a handshake on connection. type peer struct { @@ -46,7 +46,7 @@ type peer struct { persistent bool config *PeerConfig - nodeInfo *NodeInfo + nodeInfo NodeInfo Data *cmn.CMap // User data. } @@ -77,7 +77,7 @@ func DefaultPeerConfig() *PeerConfig { } func newOutboundPeer(addr *NetAddress, reactorsByCh map[byte]Reactor, chDescs []*ChannelDescriptor, - onPeerError func(Peer, interface{}), ourNodePrivKey crypto.PrivKeyEd25519, config *PeerConfig) (*peer, error) { + onPeerError func(Peer, interface{}), ourNodePrivKey crypto.PrivKey, config *PeerConfig, persistent bool) (*peer, error) { conn, err := dial(addr, config) if err != nil { @@ -91,17 +91,21 @@ func newOutboundPeer(addr *NetAddress, reactorsByCh map[byte]Reactor, chDescs [] } return nil, err } + peer.persistent = persistent + return peer, nil } func newInboundPeer(conn net.Conn, reactorsByCh map[byte]Reactor, chDescs []*ChannelDescriptor, - onPeerError func(Peer, interface{}), ourNodePrivKey crypto.PrivKeyEd25519, config *PeerConfig) (*peer, error) { + onPeerError func(Peer, interface{}), ourNodePrivKey crypto.PrivKey, config *PeerConfig) (*peer, error) { + + // TODO: issue PoW challenge return newPeerFromConnAndConfig(conn, false, reactorsByCh, chDescs, onPeerError, ourNodePrivKey, config) } func newPeerFromConnAndConfig(rawConn net.Conn, outbound bool, reactorsByCh map[byte]Reactor, chDescs []*ChannelDescriptor, - onPeerError func(Peer, interface{}), ourNodePrivKey crypto.PrivKeyEd25519, config *PeerConfig) (*peer, error) { + onPeerError func(Peer, interface{}), ourNodePrivKey crypto.PrivKey, config *PeerConfig) (*peer, error) { conn := rawConn @@ -124,7 +128,7 @@ func newPeerFromConnAndConfig(rawConn net.Conn, outbound bool, reactorsByCh map[ } } - // Key and NodeInfo are set after Handshake + // NodeInfo is set after Handshake p := &peer{ outbound: outbound, conn: conn, @@ -139,23 +143,41 @@ func newPeerFromConnAndConfig(rawConn net.Conn, outbound bool, reactorsByCh map[ return p, nil } +//--------------------------------------------------- +// Implements cmn.Service + +// SetLogger implements BaseService. func (p *peer) SetLogger(l log.Logger) { p.Logger = l p.mconn.SetLogger(l) } -// CloseConn should be used when the peer was created, but never started. -func (p *peer) CloseConn() { - p.conn.Close() // nolint: errcheck +// OnStart implements BaseService. +func (p *peer) OnStart() error { + if err := p.BaseService.OnStart(); err != nil { + return err + } + err := p.mconn.Start() + return err } -// makePersistent marks the peer as persistent. -func (p *peer) makePersistent() { - if !p.outbound { - panic("inbound peers can't be made persistent") - } +// OnStop implements BaseService. +func (p *peer) OnStop() { + p.BaseService.OnStop() + p.mconn.Stop() // stop everything and close the conn +} + +//--------------------------------------------------- +// Implements Peer - p.persistent = true +// ID returns the peer's ID - the hex encoded hash of its pubkey. +func (p *peer) ID() ID { + return PubKeyToID(p.PubKey()) +} + +// IsOutbound returns true if the connection is outbound, false otherwise. +func (p *peer) IsOutbound() bool { + return p.outbound } // IsPersistent returns true if the peer is persitent, false otherwise. @@ -163,25 +185,74 @@ func (p *peer) IsPersistent() bool { return p.persistent } -// HandshakeTimeout performs a handshake between a given node and the peer. +// NodeInfo returns a copy of the peer's NodeInfo. +func (p *peer) NodeInfo() NodeInfo { + return p.nodeInfo +} + +// Status returns the peer's ConnectionStatus. +func (p *peer) Status() ConnectionStatus { + return p.mconn.Status() +} + +// Send msg to the channel identified by chID byte. Returns false if the send +// queue is full after timeout, specified by MConnection. +func (p *peer) Send(chID byte, msg interface{}) bool { + if !p.IsRunning() { + // see Switch#Broadcast, where we fetch the list of peers and loop over + // them - while we're looping, one peer may be removed and stopped. + return false + } + return p.mconn.Send(chID, msg) +} + +// TrySend msg to the channel identified by chID byte. Immediately returns +// false if the send queue is full. +func (p *peer) TrySend(chID byte, msg interface{}) bool { + if !p.IsRunning() { + return false + } + return p.mconn.TrySend(chID, msg) +} + +// Get the data for a given key. +func (p *peer) Get(key string) interface{} { + return p.Data.Get(key) +} + +// Set sets the data for the given key. +func (p *peer) Set(key string, data interface{}) { + p.Data.Set(key, data) +} + +//--------------------------------------------------- +// methods used by the Switch + +// CloseConn should be called by the Switch if the peer was created but never started. +func (p *peer) CloseConn() { + p.conn.Close() // nolint: errcheck +} + +// HandshakeTimeout performs the Tendermint P2P handshake between a given node and the peer +// by exchanging their NodeInfo. It sets the received nodeInfo on the peer. // NOTE: blocking -func (p *peer) HandshakeTimeout(ourNodeInfo *NodeInfo, timeout time.Duration) error { +func (p *peer) HandshakeTimeout(ourNodeInfo NodeInfo, timeout time.Duration) error { // Set deadline for handshake so we don't block forever on conn.ReadFull if err := p.conn.SetDeadline(time.Now().Add(timeout)); err != nil { return errors.Wrap(err, "Error setting deadline") } - var peerNodeInfo = new(NodeInfo) + var peerNodeInfo NodeInfo var err1 error var err2 error cmn.Parallel( func() { var n int - wire.WriteBinary(ourNodeInfo, p.conn, &n, &err1) + wire.WriteBinary(&ourNodeInfo, p.conn, &n, &err1) }, func() { var n int - wire.ReadBinary(peerNodeInfo, p.conn, maxNodeInfoSize, &n, &err2) + wire.ReadBinary(&peerNodeInfo, p.conn, maxNodeInfoSize, &n, &err2) p.Logger.Info("Peer handshake", "peerNodeInfo", peerNodeInfo) }) if err1 != nil { @@ -191,20 +262,12 @@ func (p *peer) HandshakeTimeout(ourNodeInfo *NodeInfo, timeout time.Duration) er return errors.Wrap(err2, "Error during handshake/read") } - if p.config.AuthEnc { - // Check that the professed PubKey matches the sconn's. - if !peerNodeInfo.PubKey.Equals(p.PubKey().Wrap()) { - return fmt.Errorf("Ignoring connection with unmatching pubkey: %v vs %v", - peerNodeInfo.PubKey, p.PubKey()) - } - } - // Remove deadline if err := p.conn.SetDeadline(time.Time{}); err != nil { return errors.Wrap(err, "Error removing deadline") } - peerNodeInfo.RemoteAddr = p.Addr().String() + // TODO: fix the peerNodeInfo.ListenAddr p.nodeInfo = peerNodeInfo return nil @@ -216,59 +279,13 @@ func (p *peer) Addr() net.Addr { } // PubKey returns peer's public key. -func (p *peer) PubKey() crypto.PubKeyEd25519 { - if p.config.AuthEnc { +func (p *peer) PubKey() crypto.PubKey { + if !p.nodeInfo.PubKey.Empty() { + return p.nodeInfo.PubKey + } else if p.config.AuthEnc { return p.conn.(*SecretConnection).RemotePubKey() } - if p.NodeInfo() == nil { - panic("Attempt to get peer's PubKey before calling Handshake") - } - return p.PubKey() -} - -// OnStart implements BaseService. -func (p *peer) OnStart() error { - if err := p.BaseService.OnStart(); err != nil { - return err - } - err := p.mconn.Start() - return err -} - -// OnStop implements BaseService. -func (p *peer) OnStop() { - p.BaseService.OnStop() - p.mconn.Stop() -} - -// Connection returns underlying MConnection. -func (p *peer) Connection() *MConnection { - return p.mconn -} - -// IsOutbound returns true if the connection is outbound, false otherwise. -func (p *peer) IsOutbound() bool { - return p.outbound -} - -// Send msg to the channel identified by chID byte. Returns false if the send -// queue is full after timeout, specified by MConnection. -func (p *peer) Send(chID byte, msg interface{}) bool { - if !p.IsRunning() { - // see Switch#Broadcast, where we fetch the list of peers and loop over - // them - while we're looping, one peer may be removed and stopped. - return false - } - return p.mconn.Send(chID, msg) -} - -// TrySend msg to the channel identified by chID byte. Immediately returns -// false if the send queue is full. -func (p *peer) TrySend(chID byte, msg interface{}) bool { - if !p.IsRunning() { - return false - } - return p.mconn.TrySend(chID, msg) + panic("Attempt to get peer's PubKey before calling Handshake") } // CanSend returns true if the send queue is not full, false otherwise. @@ -282,45 +299,14 @@ func (p *peer) CanSend(chID byte) bool { // String representation. func (p *peer) String() string { if p.outbound { - return fmt.Sprintf("Peer{%v %v out}", p.mconn, p.Key()) + return fmt.Sprintf("Peer{%v %v out}", p.mconn, p.ID()) } - return fmt.Sprintf("Peer{%v %v in}", p.mconn, p.Key()) -} - -// Equals reports whenever 2 peers are actually represent the same node. -func (p *peer) Equals(other Peer) bool { - return p.Key() == other.Key() + return fmt.Sprintf("Peer{%v %v in}", p.mconn, p.ID()) } -// Get the data for a given key. -func (p *peer) Get(key string) interface{} { - return p.Data.Get(key) -} - -// Set sets the data for the given key. -func (p *peer) Set(key string, data interface{}) { - p.Data.Set(key, data) -} - -// Key returns the peer's id key. -func (p *peer) Key() string { - return p.nodeInfo.ListenAddr // XXX: should probably be PubKey.KeyString() -} - -// NodeInfo returns a copy of the peer's NodeInfo. -func (p *peer) NodeInfo() *NodeInfo { - if p.nodeInfo == nil { - return nil - } - n := *p.nodeInfo // copy - return &n -} - -// Status returns the peer's ConnectionStatus. -func (p *peer) Status() ConnectionStatus { - return p.mconn.Status() -} +//------------------------------------------------------------------ +// helper funcs func dial(addr *NetAddress, config *PeerConfig) (net.Conn, error) { conn, err := addr.DialTimeout(config.DialTimeout * time.Second) diff --git a/p2p/peer_set.go b/p2p/peer_set.go index c21748cf9..dc53174a1 100644 --- a/p2p/peer_set.go +++ b/p2p/peer_set.go @@ -6,8 +6,8 @@ import ( // IPeerSet has a (immutable) subset of the methods of PeerSet. type IPeerSet interface { - Has(key string) bool - Get(key string) Peer + Has(key ID) bool + Get(key ID) Peer List() []Peer Size() int } @@ -18,7 +18,7 @@ type IPeerSet interface { // Iteration over the peers is super fast and thread-safe. type PeerSet struct { mtx sync.Mutex - lookup map[string]*peerSetItem + lookup map[ID]*peerSetItem list []Peer } @@ -30,7 +30,7 @@ type peerSetItem struct { // NewPeerSet creates a new peerSet with a list of initial capacity of 256 items. func NewPeerSet() *PeerSet { return &PeerSet{ - lookup: make(map[string]*peerSetItem), + lookup: make(map[ID]*peerSetItem), list: make([]Peer, 0, 256), } } @@ -40,7 +40,7 @@ func NewPeerSet() *PeerSet { func (ps *PeerSet) Add(peer Peer) error { ps.mtx.Lock() defer ps.mtx.Unlock() - if ps.lookup[peer.Key()] != nil { + if ps.lookup[peer.ID()] != nil { return ErrSwitchDuplicatePeer } @@ -48,13 +48,13 @@ func (ps *PeerSet) Add(peer Peer) error { // Appending is safe even with other goroutines // iterating over the ps.list slice. ps.list = append(ps.list, peer) - ps.lookup[peer.Key()] = &peerSetItem{peer, index} + ps.lookup[peer.ID()] = &peerSetItem{peer, index} return nil } // Has returns true iff the PeerSet contains // the peer referred to by this peerKey. -func (ps *PeerSet) Has(peerKey string) bool { +func (ps *PeerSet) Has(peerKey ID) bool { ps.mtx.Lock() _, ok := ps.lookup[peerKey] ps.mtx.Unlock() @@ -62,7 +62,7 @@ func (ps *PeerSet) Has(peerKey string) bool { } // Get looks up a peer by the provided peerKey. -func (ps *PeerSet) Get(peerKey string) Peer { +func (ps *PeerSet) Get(peerKey ID) Peer { ps.mtx.Lock() defer ps.mtx.Unlock() item, ok := ps.lookup[peerKey] @@ -77,7 +77,7 @@ func (ps *PeerSet) Get(peerKey string) Peer { func (ps *PeerSet) Remove(peer Peer) { ps.mtx.Lock() defer ps.mtx.Unlock() - item := ps.lookup[peer.Key()] + item := ps.lookup[peer.ID()] if item == nil { return } @@ -90,18 +90,18 @@ func (ps *PeerSet) Remove(peer Peer) { // If it's the last peer, that's an easy special case. if index == len(ps.list)-1 { ps.list = newList - delete(ps.lookup, peer.Key()) + delete(ps.lookup, peer.ID()) return } // Replace the popped item with the last item in the old list. lastPeer := ps.list[len(ps.list)-1] - lastPeerKey := lastPeer.Key() + lastPeerKey := lastPeer.ID() lastPeerItem := ps.lookup[lastPeerKey] newList[index] = lastPeer lastPeerItem.index = index ps.list = newList - delete(ps.lookup, peer.Key()) + delete(ps.lookup, peer.ID()) } // Size returns the number of unique items in the peerSet. diff --git a/p2p/peer_set_test.go b/p2p/peer_set_test.go index a7f29315a..e906eb8e7 100644 --- a/p2p/peer_set_test.go +++ b/p2p/peer_set_test.go @@ -7,15 +7,16 @@ import ( "github.com/stretchr/testify/assert" + crypto "github.com/tendermint/go-crypto" cmn "github.com/tendermint/tmlibs/common" ) // Returns an empty dummy peer func randPeer() *peer { return &peer{ - nodeInfo: &NodeInfo{ - RemoteAddr: cmn.Fmt("%v.%v.%v.%v:46656", rand.Int()%256, rand.Int()%256, rand.Int()%256, rand.Int()%256), + nodeInfo: NodeInfo{ ListenAddr: cmn.Fmt("%v.%v.%v.%v:46656", rand.Int()%256, rand.Int()%256, rand.Int()%256, rand.Int()%256), + PubKey: crypto.GenPrivKeyEd25519().Wrap().PubKey(), }, } } @@ -39,7 +40,7 @@ func TestPeerSetAddRemoveOne(t *testing.T) { peerSet.Remove(peerAtFront) wantSize := n - i - 1 for j := 0; j < 2; j++ { - assert.Equal(t, false, peerSet.Has(peerAtFront.Key()), "#%d Run #%d: failed to remove peer", i, j) + assert.Equal(t, false, peerSet.Has(peerAtFront.ID()), "#%d Run #%d: failed to remove peer", i, j) assert.Equal(t, wantSize, peerSet.Size(), "#%d Run #%d: failed to remove peer and decrement size", i, j) // Test the route of removing the now non-existent element peerSet.Remove(peerAtFront) @@ -58,7 +59,7 @@ func TestPeerSetAddRemoveOne(t *testing.T) { for i := n - 1; i >= 0; i-- { peerAtEnd := peerList[i] peerSet.Remove(peerAtEnd) - assert.Equal(t, false, peerSet.Has(peerAtEnd.Key()), "#%d: failed to remove item at end", i) + assert.Equal(t, false, peerSet.Has(peerAtEnd.ID()), "#%d: failed to remove item at end", i) assert.Equal(t, i, peerSet.Size(), "#%d: differing sizes after peerSet.Remove(atEndPeer)", i) } } @@ -82,7 +83,7 @@ func TestPeerSetAddRemoveMany(t *testing.T) { for i, peer := range peers { peerSet.Remove(peer) - if peerSet.Has(peer.Key()) { + if peerSet.Has(peer.ID()) { t.Errorf("Failed to remove peer") } if peerSet.Size() != len(peers)-i-1 { @@ -129,7 +130,7 @@ func TestPeerSetGet(t *testing.T) { t.Parallel() peerSet := NewPeerSet() peer := randPeer() - assert.Nil(t, peerSet.Get(peer.Key()), "expecting a nil lookup, before .Add") + assert.Nil(t, peerSet.Get(peer.ID()), "expecting a nil lookup, before .Add") if err := peerSet.Add(peer); err != nil { t.Fatalf("Failed to add new peer: %v", err) @@ -142,7 +143,7 @@ func TestPeerSetGet(t *testing.T) { wg.Add(1) go func(i int) { defer wg.Done() - got, want := peerSet.Get(peer.Key()), peer + got, want := peerSet.Get(peer.ID()), peer assert.Equal(t, got, want, "#%d: got=%v want=%v", i, got, want) }(i) } diff --git a/p2p/peer_test.go b/p2p/peer_test.go index b53b0bb12..d99fff5e4 100644 --- a/p2p/peer_test.go +++ b/p2p/peer_test.go @@ -16,7 +16,7 @@ func TestPeerBasic(t *testing.T) { assert, require := assert.New(t), require.New(t) // simulate remote peer - rp := &remotePeer{PrivKey: crypto.GenPrivKeyEd25519(), Config: DefaultPeerConfig()} + rp := &remotePeer{PrivKey: crypto.GenPrivKeyEd25519().Wrap(), Config: DefaultPeerConfig()} rp.Start() defer rp.Stop() @@ -30,7 +30,7 @@ func TestPeerBasic(t *testing.T) { assert.True(p.IsRunning()) assert.True(p.IsOutbound()) assert.False(p.IsPersistent()) - p.makePersistent() + p.persistent = true assert.True(p.IsPersistent()) assert.Equal(rp.Addr().String(), p.Addr().String()) assert.Equal(rp.PubKey(), p.PubKey()) @@ -43,7 +43,7 @@ func TestPeerWithoutAuthEnc(t *testing.T) { config.AuthEnc = false // simulate remote peer - rp := &remotePeer{PrivKey: crypto.GenPrivKeyEd25519(), Config: config} + rp := &remotePeer{PrivKey: crypto.GenPrivKeyEd25519().Wrap(), Config: config} rp.Start() defer rp.Stop() @@ -64,7 +64,7 @@ func TestPeerSend(t *testing.T) { config.AuthEnc = false // simulate remote peer - rp := &remotePeer{PrivKey: crypto.GenPrivKeyEd25519(), Config: config} + rp := &remotePeer{PrivKey: crypto.GenPrivKeyEd25519().Wrap(), Config: config} rp.Start() defer rp.Stop() @@ -85,13 +85,13 @@ func createOutboundPeerAndPerformHandshake(addr *NetAddress, config *PeerConfig) {ID: 0x01, Priority: 1}, } reactorsByCh := map[byte]Reactor{0x01: NewTestReactor(chDescs, true)} - pk := crypto.GenPrivKeyEd25519() - p, err := newOutboundPeer(addr, reactorsByCh, chDescs, func(p Peer, r interface{}) {}, pk, config) + pk := crypto.GenPrivKeyEd25519().Wrap() + p, err := newOutboundPeer(addr, reactorsByCh, chDescs, func(p Peer, r interface{}) {}, pk, config, false) if err != nil { return nil, err } - err = p.HandshakeTimeout(&NodeInfo{ - PubKey: pk.PubKey().Unwrap().(crypto.PubKeyEd25519), + err = p.HandshakeTimeout(NodeInfo{ + PubKey: pk.PubKey(), Moniker: "host_peer", Network: "testing", Version: "123.123.123", @@ -103,7 +103,7 @@ func createOutboundPeerAndPerformHandshake(addr *NetAddress, config *PeerConfig) } type remotePeer struct { - PrivKey crypto.PrivKeyEd25519 + PrivKey crypto.PrivKey Config *PeerConfig addr *NetAddress quit chan struct{} @@ -113,8 +113,8 @@ func (p *remotePeer) Addr() *NetAddress { return p.addr } -func (p *remotePeer) PubKey() crypto.PubKeyEd25519 { - return p.PrivKey.PubKey().Unwrap().(crypto.PubKeyEd25519) +func (p *remotePeer) PubKey() crypto.PubKey { + return p.PrivKey.PubKey() } func (p *remotePeer) Start() { @@ -122,7 +122,7 @@ func (p *remotePeer) Start() { if e != nil { golog.Fatalf("net.Listen tcp :0: %+v", e) } - p.addr = NewNetAddress(l.Addr()) + p.addr = NewNetAddress("", l.Addr()) p.quit = make(chan struct{}) go p.accept(l) } @@ -141,11 +141,12 @@ func (p *remotePeer) accept(l net.Listener) { if err != nil { golog.Fatalf("Failed to create a peer: %+v", err) } - err = peer.HandshakeTimeout(&NodeInfo{ - PubKey: p.PrivKey.PubKey().Unwrap().(crypto.PubKeyEd25519), - Moniker: "remote_peer", - Network: "testing", - Version: "123.123.123", + err = peer.HandshakeTimeout(NodeInfo{ + PubKey: p.PrivKey.PubKey(), + Moniker: "remote_peer", + Network: "testing", + Version: "123.123.123", + ListenAddr: l.Addr().String(), }, 1*time.Second) if err != nil { golog.Fatalf("Failed to perform handshake: %+v", err) diff --git a/p2p/pex_reactor.go b/p2p/pex_reactor.go index 2bfe7dcab..93fddc111 100644 --- a/p2p/pex_reactor.go +++ b/p2p/pex_reactor.go @@ -7,6 +7,7 @@ import ( "reflect" "time" + "github.com/pkg/errors" wire "github.com/tendermint/go-wire" cmn "github.com/tendermint/tmlibs/common" ) @@ -19,10 +20,6 @@ const ( defaultEnsurePeersPeriod = 30 * time.Second minNumOutboundPeers = 10 maxPexMessageSize = 1048576 // 1MB - - // maximum pex messages one peer can send to us during `msgCountByPeerFlushInterval` - defaultMaxMsgCountByPeer = 1000 - msgCountByPeerFlushInterval = 1 * time.Hour ) // PEXReactor handles PEX (peer exchange) and ensures that an @@ -32,33 +29,35 @@ const ( // // ## Preventing abuse // -// For now, it just limits the number of messages from one peer to -// `defaultMaxMsgCountByPeer` messages per `msgCountByPeerFlushInterval` (1000 -// msg/hour). -// -// NOTE [2017-01-17]: -// Limiting is fine for now. Maybe down the road we want to keep track of the -// quality of peer messages so if peerA keeps telling us about peers we can't -// connect to then maybe we should care less about peerA. But I don't think -// that kind of complexity is priority right now. +// Only accept pexAddrsMsg from peers we sent a corresponding pexRequestMsg too. +// Only accept one pexRequestMsg every ~defaultEnsurePeersPeriod. type PEXReactor struct { BaseReactor book *AddrBook + config *PEXReactorConfig ensurePeersPeriod time.Duration - // tracks message count by peer, so we can prevent abuse - msgCountByPeer *cmn.CMap - maxMsgCountByPeer uint16 + // maps to prevent abuse + requestsSent *cmn.CMap // ID->struct{}: unanswered send requests + lastReceivedRequests *cmn.CMap // ID->time.Time: last time peer requested from us +} + +// PEXReactorConfig holds reactor specific configuration data. +type PEXReactorConfig struct { + // Seeds is a list of addresses reactor may use if it can't connect to peers + // in the addrbook. + Seeds []string } // NewPEXReactor creates new PEX reactor. -func NewPEXReactor(b *AddrBook) *PEXReactor { +func NewPEXReactor(b *AddrBook, config *PEXReactorConfig) *PEXReactor { r := &PEXReactor{ - book: b, - ensurePeersPeriod: defaultEnsurePeersPeriod, - msgCountByPeer: cmn.NewCMap(), - maxMsgCountByPeer: defaultMaxMsgCountByPeer, + book: b, + config: config, + ensurePeersPeriod: defaultEnsurePeersPeriod, + requestsSent: cmn.NewCMap(), + lastReceivedRequests: cmn.NewCMap(), } r.BaseReactor = *NewBaseReactor("PEXReactor", r) return r @@ -73,8 +72,13 @@ func (r *PEXReactor) OnStart() error { if err != nil && err != cmn.ErrAlreadyStarted { return err } + + // return err if user provided a bad seed address + if err := r.checkSeeds(); err != nil { + return err + } + go r.ensurePeersRoutine() - go r.flushMsgCountByPeer() return nil } @@ -99,46 +103,31 @@ func (r *PEXReactor) GetChannels() []*ChannelDescriptor { // or by requesting more addresses (if outbound). func (r *PEXReactor) AddPeer(p Peer) { if p.IsOutbound() { - // For outbound peers, the address is already in the books. - // Either it was added in DialSeeds or when we - // received the peer's address in r.Receive + // For outbound peers, the address is already in the books - + // either via DialPeersAsync or r.Receive. + // Ask it for more peers if we need. if r.book.NeedMoreAddrs() { r.RequestPEX(p) } - } else { // For inbound connections, the peer is its own source - addr, err := NewNetAddressString(p.NodeInfo().ListenAddr) - if err != nil { - // peer gave us a bad ListenAddr. TODO: punish - r.Logger.Error("Error in AddPeer: invalid peer address", "addr", p.NodeInfo().ListenAddr, "err", err) - return - } + } else { + // For inbound peers, the peer is its own source, + // and its NodeInfo has already been validated. + // Let the ensurePeersRoutine handle asking for more + // peers when we need - we don't trust inbound peers as much. + addr := p.NodeInfo().NetAddress() r.book.AddAddress(addr, addr) } } // RemovePeer implements Reactor. func (r *PEXReactor) RemovePeer(p Peer, reason interface{}) { - // If we aren't keeping track of local temp data for each peer here, then we - // don't have to do anything. + id := string(p.ID()) + r.requestsSent.Delete(id) + r.lastReceivedRequests.Delete(id) } // Receive implements Reactor by handling incoming PEX messages. func (r *PEXReactor) Receive(chID byte, src Peer, msgBytes []byte) { - srcAddrStr := src.NodeInfo().RemoteAddr - srcAddr, err := NewNetAddressString(srcAddrStr) - if err != nil { - // this should never happen. TODO: cancel conn - r.Logger.Error("Error in Receive: invalid peer address", "addr", srcAddrStr, "err", err) - return - } - - r.IncrementMsgCountForPeer(srcAddrStr) - if r.ReachedMaxMsgCountForPeer(srcAddrStr) { - r.Logger.Error("Maximum number of messages reached for peer", "peer", srcAddrStr) - // TODO remove src from peers? - return - } - _, msg, err := DecodeMessage(msgBytes) if err != nil { r.Logger.Error("Error decoding message", "err", err) @@ -148,58 +137,89 @@ func (r *PEXReactor) Receive(chID byte, src Peer, msgBytes []byte) { switch msg := msg.(type) { case *pexRequestMessage: - // src requested some peers. - // NOTE: we might send an empty selection + // We received a request for peers from src. + if err := r.receiveRequest(src); err != nil { + r.Switch.StopPeerForError(src, err) + return + } r.SendAddrs(src, r.book.GetSelection()) case *pexAddrsMessage: // We received some peer addresses from src. - // TODO: (We don't want to get spammed with bad peers) - for _, addr := range msg.Addrs { - if addr != nil { - r.book.AddAddress(addr, srcAddr) - } + if err := r.ReceivePEX(msg.Addrs, src); err != nil { + r.Switch.StopPeerForError(src, err) + return } default: r.Logger.Error(fmt.Sprintf("Unknown message type %v", reflect.TypeOf(msg))) } } -// RequestPEX asks peer for more addresses. +func (r *PEXReactor) receiveRequest(src Peer) error { + id := string(src.ID()) + v := r.lastReceivedRequests.Get(id) + if v == nil { + // initialize with empty time + lastReceived := time.Time{} + r.lastReceivedRequests.Set(id, lastReceived) + return nil + } + + lastReceived := v.(time.Time) + if lastReceived.Equal(time.Time{}) { + // first time gets a free pass. then we start tracking the time + lastReceived = time.Now() + r.lastReceivedRequests.Set(id, lastReceived) + return nil + } + + now := time.Now() + if now.Sub(lastReceived) < r.ensurePeersPeriod/3 { + return fmt.Errorf("Peer (%v) is sending too many PEX requests. Disconnecting", src.ID()) + } + r.lastReceivedRequests.Set(id, now) + return nil +} + +// RequestPEX asks peer for more addresses if we do not already +// have a request out for this peer. func (r *PEXReactor) RequestPEX(p Peer) { + id := string(p.ID()) + if r.requestsSent.Has(id) { + return + } + r.requestsSent.Set(id, struct{}{}) p.Send(PexChannel, struct{ PexMessage }{&pexRequestMessage{}}) } -// SendAddrs sends addrs to the peer. -func (r *PEXReactor) SendAddrs(p Peer, addrs []*NetAddress) { - p.Send(PexChannel, struct{ PexMessage }{&pexAddrsMessage{Addrs: addrs}}) -} +// ReceivePEX adds the given addrs to the addrbook if theres an open +// request for this peer and deletes the open request. +// If there's no open request for the src peer, it returns an error. +func (r *PEXReactor) ReceivePEX(addrs []*NetAddress, src Peer) error { + id := string(src.ID()) -// SetEnsurePeersPeriod sets period to ensure peers connected. -func (r *PEXReactor) SetEnsurePeersPeriod(d time.Duration) { - r.ensurePeersPeriod = d -} + if !r.requestsSent.Has(id) { + return errors.New("Received unsolicited pexAddrsMessage") + } + + r.requestsSent.Delete(id) -// SetMaxMsgCountByPeer sets maximum messages one peer can send to us during 'msgCountByPeerFlushInterval'. -func (r *PEXReactor) SetMaxMsgCountByPeer(v uint16) { - r.maxMsgCountByPeer = v + srcAddr := src.NodeInfo().NetAddress() + for _, netAddr := range addrs { + if netAddr != nil { + r.book.AddAddress(netAddr, srcAddr) + } + } + return nil } -// ReachedMaxMsgCountForPeer returns true if we received too many -// messages from peer with address `addr`. -// NOTE: assumes the value in the CMap is non-nil -func (r *PEXReactor) ReachedMaxMsgCountForPeer(addr string) bool { - return r.msgCountByPeer.Get(addr).(uint16) >= r.maxMsgCountByPeer +// SendAddrs sends addrs to the peer. +func (r *PEXReactor) SendAddrs(p Peer, netAddrs []*NetAddress) { + p.Send(PexChannel, struct{ PexMessage }{&pexAddrsMessage{Addrs: netAddrs}}) } -// Increment or initialize the msg count for the peer in the CMap -func (r *PEXReactor) IncrementMsgCountForPeer(addr string) { - var count uint16 - countI := r.msgCountByPeer.Get(addr) - if countI != nil { - count = countI.(uint16) - } - count++ - r.msgCountByPeer.Set(addr, count) +// SetEnsurePeersPeriod sets period to ensure peers connected. +func (r *PEXReactor) SetEnsurePeersPeriod(d time.Duration) { + r.ensurePeersPeriod = d } // Ensures that sufficient peers are connected. (continuous) @@ -209,11 +229,11 @@ func (r *PEXReactor) ensurePeersRoutine() { time.Sleep(time.Duration(rand.Int63n(ensurePeersPeriodMs)) * time.Millisecond) // fire once immediately. + // ensures we dial the seeds right away if the book is empty r.ensurePeers() // fire periodically ticker := time.NewTicker(r.ensurePeersPeriod) - for { select { case <-ticker.C: @@ -238,7 +258,7 @@ func (r *PEXReactor) ensurePeersRoutine() { // placeholder. It should not be the case that an address becomes old/vetted // upon a single successful connection. func (r *PEXReactor) ensurePeers() { - numOutPeers, _, numDialing := r.Switch.NumPeers() + numOutPeers, numInPeers, numDialing := r.Switch.NumPeers() numToDial := minNumOutboundPeers - (numOutPeers + numDialing) r.Logger.Info("Ensure peers", "numOutPeers", numOutPeers, "numDialing", numDialing, "numToDial", numToDial) if numToDial <= 0 { @@ -250,7 +270,7 @@ func (r *PEXReactor) ensurePeers() { // NOTE: range here is [10, 90]. Too high ? newBias := cmn.MinInt(numOutPeers, 8)*10 + 10 - toDial := make(map[string]*NetAddress) + toDial := make(map[ID]*NetAddress) // Try maxAttempts times to pick numToDial addresses to dial maxAttempts := numToDial * 3 for i := 0; i < maxAttempts && len(toDial) < numToDial; i++ { @@ -258,18 +278,17 @@ func (r *PEXReactor) ensurePeers() { if try == nil { continue } - if _, selected := toDial[try.IP.String()]; selected { + if _, selected := toDial[try.ID]; selected { continue } - if dialling := r.Switch.IsDialing(try); dialling { + if dialling := r.Switch.IsDialing(try.ID); dialling { continue } - // XXX: Should probably use pubkey as peer key ... - if connected := r.Switch.Peers().Has(try.String()); connected { + if connected := r.Switch.Peers().Has(try.ID); connected { continue } r.Logger.Info("Will dial address", "addr", try) - toDial[try.IP.String()] = try + toDial[try.ID] = try } // Dial picked addresses @@ -284,27 +303,58 @@ func (r *PEXReactor) ensurePeers() { // If we need more addresses, pick a random peer and ask for more. if r.book.NeedMoreAddrs() { - if peers := r.Switch.Peers().List(); len(peers) > 0 { - i := rand.Int() % len(peers) // nolint: gas - peer := peers[i] - r.Logger.Info("No addresses to dial. Sending pexRequest to random peer", "peer", peer) + peers := r.Switch.Peers().List() + peersCount := len(peers) + if peersCount > 0 { + peer := peers[rand.Int()%peersCount] // nolint: gas + r.Logger.Info("We need more addresses. Sending pexRequest to random peer", "peer", peer) r.RequestPEX(peer) } } + + // If we are not connected to nor dialing anybody, fallback to dialing a seed. + if numOutPeers+numInPeers+numDialing+len(toDial) == 0 { + r.Logger.Info("No addresses to dial nor connected peers. Falling back to seeds") + r.dialSeed() + } } -func (r *PEXReactor) flushMsgCountByPeer() { - ticker := time.NewTicker(msgCountByPeerFlushInterval) +// check seed addresses are well formed +func (r *PEXReactor) checkSeeds() error { + lSeeds := len(r.config.Seeds) + if lSeeds == 0 { + return nil + } + _, errs := NewNetAddressStrings(r.config.Seeds) + for _, err := range errs { + if err != nil { + return err + } + } + return nil +} - for { - select { - case <-ticker.C: - r.msgCountByPeer.Clear() - case <-r.Quit: - ticker.Stop() +// randomly dial seeds until we connect to one or exhaust them +func (r *PEXReactor) dialSeed() { + lSeeds := len(r.config.Seeds) + if lSeeds == 0 { + return + } + seedAddrs, _ := NewNetAddressStrings(r.config.Seeds) + + perm := r.Switch.rng.Perm(lSeeds) + for _, i := range perm { + // dial a random seed + seedAddr := seedAddrs[i] + peer, err := r.Switch.DialPeerWithAddress(seedAddr, false) + if err != nil { + r.Switch.Logger.Error("Error dialing seed", "err", err, "seed", seedAddr) + } else { + r.Switch.Logger.Info("Connected to seed", "peer", peer) return } } + r.Switch.Logger.Error("Couldn't connect to any seeds") } //----------------------------------------------------------------------------- diff --git a/p2p/pex_reactor_test.go b/p2p/pex_reactor_test.go index a14f0eb2a..c0681586a 100644 --- a/p2p/pex_reactor_test.go +++ b/p2p/pex_reactor_test.go @@ -10,6 +10,7 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + crypto "github.com/tendermint/go-crypto" wire "github.com/tendermint/go-wire" cmn "github.com/tendermint/tmlibs/common" "github.com/tendermint/tmlibs/log" @@ -24,7 +25,7 @@ func TestPEXReactorBasic(t *testing.T) { book := NewAddrBook(dir+"addrbook.json", true) book.SetLogger(log.TestingLogger()) - r := NewPEXReactor(book) + r := NewPEXReactor(book, &PEXReactorConfig{}) r.SetLogger(log.TestingLogger()) assert.NotNil(r) @@ -40,7 +41,7 @@ func TestPEXReactorAddRemovePeer(t *testing.T) { book := NewAddrBook(dir+"addrbook.json", true) book.SetLogger(log.TestingLogger()) - r := NewPEXReactor(book) + r := NewPEXReactor(book, &PEXReactorConfig{}) r.SetLogger(log.TestingLogger()) size := book.Size() @@ -76,7 +77,7 @@ func TestPEXReactorRunning(t *testing.T) { switches[i] = makeSwitch(config, i, "127.0.0.1", "123.123.123", func(i int, sw *Switch) *Switch { sw.SetLogger(log.TestingLogger().With("switch", i)) - r := NewPEXReactor(book) + r := NewPEXReactor(book, &PEXReactorConfig{}) r.SetLogger(log.TestingLogger()) r.SetEnsurePeersPeriod(250 * time.Millisecond) sw.AddReactor("pex", r) @@ -107,6 +108,7 @@ func TestPEXReactorRunning(t *testing.T) { func assertSomePeersWithTimeout(t *testing.T, switches []*Switch, checkPeriod, timeout time.Duration) { ticker := time.NewTicker(checkPeriod) + remaining := timeout for { select { case <-ticker.C: @@ -118,16 +120,21 @@ func assertSomePeersWithTimeout(t *testing.T, switches []*Switch, checkPeriod, t allGood = false } } + remaining -= checkPeriod + if remaining < 0 { + remaining = 0 + } if allGood { return } - case <-time.After(timeout): + case <-time.After(remaining): numPeersStr := "" for i, s := range switches { outbound, inbound, _ := s.NumPeers() numPeersStr += fmt.Sprintf("%d => {outbound: %d, inbound: %d}, ", i, outbound, inbound) } t.Errorf("expected all switches to be connected to at least one peer (switches: %s)", numPeersStr) + return } } } @@ -141,14 +148,16 @@ func TestPEXReactorReceive(t *testing.T) { book := NewAddrBook(dir+"addrbook.json", false) book.SetLogger(log.TestingLogger()) - r := NewPEXReactor(book) + r := NewPEXReactor(book, &PEXReactorConfig{}) r.SetLogger(log.TestingLogger()) peer := createRandomPeer(false) + // we have to send a request to receive responses + r.RequestPEX(peer) + size := book.Size() - netAddr, _ := NewNetAddressString(peer.NodeInfo().ListenAddr) - addrs := []*NetAddress{netAddr} + addrs := []*NetAddress{peer.NodeInfo().NetAddress()} msg := wire.BinaryBytes(struct{ PexMessage }{&pexAddrsMessage{Addrs: addrs}}) r.Receive(PexChannel, peer, msg) assert.Equal(size+1, book.Size()) @@ -157,7 +166,7 @@ func TestPEXReactorReceive(t *testing.T) { r.Receive(PexChannel, peer, msg) } -func TestPEXReactorAbuseFromPeer(t *testing.T) { +func TestPEXReactorRequestMessageAbuse(t *testing.T) { assert, require := assert.New(t), require.New(t) dir, err := ioutil.TempDir("", "pex_reactor") @@ -166,18 +175,115 @@ func TestPEXReactorAbuseFromPeer(t *testing.T) { book := NewAddrBook(dir+"addrbook.json", true) book.SetLogger(log.TestingLogger()) - r := NewPEXReactor(book) + r := NewPEXReactor(book, &PEXReactorConfig{}) + sw := makeSwitch(config, 0, "127.0.0.1", "123.123.123", func(i int, sw *Switch) *Switch { return sw }) + sw.SetLogger(log.TestingLogger()) + sw.AddReactor("PEX", r) + r.SetSwitch(sw) r.SetLogger(log.TestingLogger()) - r.SetMaxMsgCountByPeer(5) - peer := createRandomPeer(false) + peer := newMockPeer() + sw.peers.Add(peer) + assert.True(sw.Peers().Has(peer.ID())) + id := string(peer.ID()) msg := wire.BinaryBytes(struct{ PexMessage }{&pexRequestMessage{}}) - for i := 0; i < 10; i++ { - r.Receive(PexChannel, peer, msg) - } - assert.True(r.ReachedMaxMsgCountForPeer(peer.NodeInfo().ListenAddr)) + // first time creates the entry + r.Receive(PexChannel, peer, msg) + assert.True(r.lastReceivedRequests.Has(id)) + assert.True(sw.Peers().Has(peer.ID())) + + // next time sets the last time value + r.Receive(PexChannel, peer, msg) + assert.True(r.lastReceivedRequests.Has(id)) + assert.True(sw.Peers().Has(peer.ID())) + + // third time is too many too soon - peer is removed + r.Receive(PexChannel, peer, msg) + assert.False(r.lastReceivedRequests.Has(id)) + assert.False(sw.Peers().Has(peer.ID())) +} + +func TestPEXReactorAddrsMessageAbuse(t *testing.T) { + assert, require := assert.New(t), require.New(t) + + dir, err := ioutil.TempDir("", "pex_reactor") + require.Nil(err) + defer os.RemoveAll(dir) // nolint: errcheck + book := NewAddrBook(dir+"addrbook.json", true) + book.SetLogger(log.TestingLogger()) + + r := NewPEXReactor(book, &PEXReactorConfig{}) + sw := makeSwitch(config, 0, "127.0.0.1", "123.123.123", func(i int, sw *Switch) *Switch { return sw }) + sw.SetLogger(log.TestingLogger()) + sw.AddReactor("PEX", r) + r.SetSwitch(sw) + r.SetLogger(log.TestingLogger()) + + peer := newMockPeer() + sw.peers.Add(peer) + assert.True(sw.Peers().Has(peer.ID())) + + id := string(peer.ID()) + + // request addrs from the peer + r.RequestPEX(peer) + assert.True(r.requestsSent.Has(id)) + assert.True(sw.Peers().Has(peer.ID())) + + addrs := []*NetAddress{peer.NodeInfo().NetAddress()} + msg := wire.BinaryBytes(struct{ PexMessage }{&pexAddrsMessage{Addrs: addrs}}) + + // receive some addrs. should clear the request + r.Receive(PexChannel, peer, msg) + assert.False(r.requestsSent.Has(id)) + assert.True(sw.Peers().Has(peer.ID())) + + // receiving more addrs causes a disconnect + r.Receive(PexChannel, peer, msg) + assert.False(sw.Peers().Has(peer.ID())) +} + +func TestPEXReactorUsesSeedsIfNeeded(t *testing.T) { + dir, err := ioutil.TempDir("", "pex_reactor") + require.Nil(t, err) + defer os.RemoveAll(dir) // nolint: errcheck + + book := NewAddrBook(dir+"addrbook.json", false) + book.SetLogger(log.TestingLogger()) + + // 1. create seed + seed := makeSwitch(config, 0, "127.0.0.1", "123.123.123", func(i int, sw *Switch) *Switch { + sw.SetLogger(log.TestingLogger()) + + r := NewPEXReactor(book, &PEXReactorConfig{}) + r.SetLogger(log.TestingLogger()) + r.SetEnsurePeersPeriod(250 * time.Millisecond) + sw.AddReactor("pex", r) + return sw + }) + seed.AddListener(NewDefaultListener("tcp", seed.NodeInfo().ListenAddr, true, log.TestingLogger())) + err = seed.Start() + require.Nil(t, err) + defer seed.Stop() + + // 2. create usual peer + sw := makeSwitch(config, 1, "127.0.0.1", "123.123.123", func(i int, sw *Switch) *Switch { + sw.SetLogger(log.TestingLogger()) + + r := NewPEXReactor(book, &PEXReactorConfig{Seeds: []string{seed.NodeInfo().ListenAddr}}) + r.SetLogger(log.TestingLogger()) + r.SetEnsurePeersPeriod(250 * time.Millisecond) + sw.AddReactor("pex", r) + return sw + }) + err = sw.Start() + require.Nil(t, err) + defer sw.Stop() + + // 3. check that peer at least connects to seed + assertSomePeersWithTimeout(t, []*Switch{sw}, 10*time.Millisecond, 10*time.Second) } func createRoutableAddr() (addr string, netAddr *NetAddress) { @@ -194,13 +300,46 @@ func createRoutableAddr() (addr string, netAddr *NetAddress) { func createRandomPeer(outbound bool) *peer { addr, netAddr := createRoutableAddr() p := &peer{ - nodeInfo: &NodeInfo{ - ListenAddr: addr, - RemoteAddr: netAddr.String(), + nodeInfo: NodeInfo{ + ListenAddr: netAddr.String(), + PubKey: crypto.GenPrivKeyEd25519().Wrap().PubKey(), }, outbound: outbound, - mconn: &MConnection{RemoteAddress: netAddr}, + mconn: &MConnection{}, } p.SetLogger(log.TestingLogger().With("peer", addr)) return p } + +type mockPeer struct { + *cmn.BaseService + pubKey crypto.PubKey + addr *NetAddress + outbound, persistent bool +} + +func newMockPeer() mockPeer { + _, netAddr := createRoutableAddr() + mp := mockPeer{ + addr: netAddr, + pubKey: crypto.GenPrivKeyEd25519().Wrap().PubKey(), + } + mp.BaseService = cmn.NewBaseService(nil, "MockPeer", mp) + mp.Start() + return mp +} + +func (mp mockPeer) ID() ID { return PubKeyToID(mp.pubKey) } +func (mp mockPeer) IsOutbound() bool { return mp.outbound } +func (mp mockPeer) IsPersistent() bool { return mp.persistent } +func (mp mockPeer) NodeInfo() NodeInfo { + return NodeInfo{ + PubKey: mp.pubKey, + ListenAddr: mp.addr.DialString(), + } +} +func (mp mockPeer) Status() ConnectionStatus { return ConnectionStatus{} } +func (mp mockPeer) Send(byte, interface{}) bool { return false } +func (mp mockPeer) TrySend(byte, interface{}) bool { return false } +func (mp mockPeer) Set(string, interface{}) {} +func (mp mockPeer) Get(string) interface{} { return nil } diff --git a/p2p/secret_connection.go b/p2p/secret_connection.go index aec0a7519..f022d9c35 100644 --- a/p2p/secret_connection.go +++ b/p2p/secret_connection.go @@ -38,7 +38,7 @@ type SecretConnection struct { recvBuffer []byte recvNonce *[24]byte sendNonce *[24]byte - remPubKey crypto.PubKeyEd25519 + remPubKey crypto.PubKey shrSecret *[32]byte // shared secret } @@ -46,9 +46,9 @@ type SecretConnection struct { // Returns nil if error in handshake. // Caller should call conn.Close() // See docs/sts-final.pdf for more information. -func MakeSecretConnection(conn io.ReadWriteCloser, locPrivKey crypto.PrivKeyEd25519) (*SecretConnection, error) { +func MakeSecretConnection(conn io.ReadWriteCloser, locPrivKey crypto.PrivKey) (*SecretConnection, error) { - locPubKey := locPrivKey.PubKey().Unwrap().(crypto.PubKeyEd25519) + locPubKey := locPrivKey.PubKey() // Generate ephemeral keys for perfect forward secrecy. locEphPub, locEphPriv := genEphKeys() @@ -100,12 +100,12 @@ func MakeSecretConnection(conn io.ReadWriteCloser, locPrivKey crypto.PrivKeyEd25 } // We've authorized. - sc.remPubKey = remPubKey.Unwrap().(crypto.PubKeyEd25519) + sc.remPubKey = remPubKey return sc, nil } // Returns authenticated remote pubkey -func (sc *SecretConnection) RemotePubKey() crypto.PubKeyEd25519 { +func (sc *SecretConnection) RemotePubKey() crypto.PubKey { return sc.remPubKey } @@ -258,8 +258,8 @@ func genChallenge(loPubKey, hiPubKey *[32]byte) (challenge *[32]byte) { return hash32(append(loPubKey[:], hiPubKey[:]...)) } -func signChallenge(challenge *[32]byte, locPrivKey crypto.PrivKeyEd25519) (signature crypto.SignatureEd25519) { - signature = locPrivKey.Sign(challenge[:]).Unwrap().(crypto.SignatureEd25519) +func signChallenge(challenge *[32]byte, locPrivKey crypto.PrivKey) (signature crypto.Signature) { + signature = locPrivKey.Sign(challenge[:]) return } @@ -268,7 +268,7 @@ type authSigMessage struct { Sig crypto.Signature } -func shareAuthSignature(sc *SecretConnection, pubKey crypto.PubKeyEd25519, signature crypto.SignatureEd25519) (*authSigMessage, error) { +func shareAuthSignature(sc *SecretConnection, pubKey crypto.PubKey, signature crypto.Signature) (*authSigMessage, error) { var recvMsg authSigMessage var err1, err2 error diff --git a/p2p/secret_connection_test.go b/p2p/secret_connection_test.go index 8b58fb417..5e0611a87 100644 --- a/p2p/secret_connection_test.go +++ b/p2p/secret_connection_test.go @@ -1,7 +1,6 @@ package p2p import ( - "bytes" "io" "testing" @@ -32,10 +31,10 @@ func makeDummyConnPair() (fooConn, barConn dummyConn) { func makeSecretConnPair(tb testing.TB) (fooSecConn, barSecConn *SecretConnection) { fooConn, barConn := makeDummyConnPair() - fooPrvKey := crypto.GenPrivKeyEd25519() - fooPubKey := fooPrvKey.PubKey().Unwrap().(crypto.PubKeyEd25519) - barPrvKey := crypto.GenPrivKeyEd25519() - barPubKey := barPrvKey.PubKey().Unwrap().(crypto.PubKeyEd25519) + fooPrvKey := crypto.GenPrivKeyEd25519().Wrap() + fooPubKey := fooPrvKey.PubKey() + barPrvKey := crypto.GenPrivKeyEd25519().Wrap() + barPubKey := barPrvKey.PubKey() cmn.Parallel( func() { @@ -46,7 +45,7 @@ func makeSecretConnPair(tb testing.TB) (fooSecConn, barSecConn *SecretConnection return } remotePubBytes := fooSecConn.RemotePubKey() - if !bytes.Equal(remotePubBytes[:], barPubKey[:]) { + if !remotePubBytes.Equals(barPubKey) { tb.Errorf("Unexpected fooSecConn.RemotePubKey. Expected %v, got %v", barPubKey, fooSecConn.RemotePubKey()) } @@ -59,7 +58,7 @@ func makeSecretConnPair(tb testing.TB) (fooSecConn, barSecConn *SecretConnection return } remotePubBytes := barSecConn.RemotePubKey() - if !bytes.Equal(remotePubBytes[:], fooPubKey[:]) { + if !remotePubBytes.Equals(fooPubKey) { tb.Errorf("Unexpected barSecConn.RemotePubKey. Expected %v, got %v", fooPubKey, barSecConn.RemotePubKey()) } @@ -93,7 +92,7 @@ func TestSecretConnectionReadWrite(t *testing.T) { genNodeRunner := func(nodeConn dummyConn, nodeWrites []string, nodeReads *[]string) func() { return func() { // Node handskae - nodePrvKey := crypto.GenPrivKeyEd25519() + nodePrvKey := crypto.GenPrivKeyEd25519().Wrap() nodeSecretConn, err := MakeSecretConnection(nodeConn, nodePrvKey) if err != nil { t.Errorf("Failed to establish SecretConnection for node: %v", err) diff --git a/p2p/switch.go b/p2p/switch.go index 76b019806..3f026556a 100644 --- a/p2p/switch.go +++ b/p2p/switch.go @@ -16,7 +16,7 @@ import ( const ( // wait a random amount of time from this interval - // before dialing seeds or reconnecting to help prevent DoS + // before dialing peers or reconnecting to help prevent DoS dialRandomizerIntervalMilliseconds = 3000 // repeatedly try to reconnect for a few minutes @@ -30,46 +30,17 @@ const ( reconnectBackOffBaseSeconds = 3 ) -type Reactor interface { - cmn.Service // Start, Stop - - SetSwitch(*Switch) - GetChannels() []*ChannelDescriptor - AddPeer(peer Peer) - RemovePeer(peer Peer, reason interface{}) - Receive(chID byte, peer Peer, msgBytes []byte) // CONTRACT: msgBytes are not nil -} - -//-------------------------------------- - -type BaseReactor struct { - cmn.BaseService // Provides Start, Stop, .Quit - Switch *Switch -} - -func NewBaseReactor(name string, impl Reactor) *BaseReactor { - return &BaseReactor{ - BaseService: *cmn.NewBaseService(nil, name, impl), - Switch: nil, - } -} - -func (br *BaseReactor) SetSwitch(sw *Switch) { - br.Switch = sw -} -func (_ *BaseReactor) GetChannels() []*ChannelDescriptor { return nil } -func (_ *BaseReactor) AddPeer(peer Peer) {} -func (_ *BaseReactor) RemovePeer(peer Peer, reason interface{}) {} -func (_ *BaseReactor) Receive(chID byte, peer Peer, msgBytes []byte) {} +var ( + ErrSwitchDuplicatePeer = errors.New("Duplicate peer") + ErrSwitchConnectToSelf = errors.New("Connect to self") +) //----------------------------------------------------------------------------- -/* -The `Switch` handles peer connections and exposes an API to receive incoming messages -on `Reactors`. Each `Reactor` is responsible for handling incoming messages of one -or more `Channels`. So while sending outgoing messages is typically performed on the peer, -incoming messages are received on the reactor. -*/ +// `Switch` handles peer connections and exposes an API to receive incoming messages +// on `Reactors`. Each `Reactor` is responsible for handling incoming messages of one +// or more `Channels`. So while sending outgoing messages is typically performed on the peer, +// incoming messages are received on the reactor. type Switch struct { cmn.BaseService @@ -81,19 +52,15 @@ type Switch struct { reactorsByCh map[byte]Reactor peers *PeerSet dialing *cmn.CMap - nodeInfo *NodeInfo // our node info - nodePrivKey crypto.PrivKeyEd25519 // our node privkey + nodeInfo NodeInfo // our node info + nodeKey *NodeKey // our node privkey filterConnByAddr func(net.Addr) error - filterConnByPubKey func(crypto.PubKeyEd25519) error + filterConnByPubKey func(crypto.PubKey) error rng *rand.Rand // seed for randomizing dial times and orders } -var ( - ErrSwitchDuplicatePeer = errors.New("Duplicate peer") -) - func NewSwitch(config *cfg.P2PConfig) *Switch { sw := &Switch{ config: config, @@ -103,7 +70,6 @@ func NewSwitch(config *cfg.P2PConfig) *Switch { reactorsByCh: make(map[byte]Reactor), peers: NewPeerSet(), dialing: cmn.NewCMap(), - nodeInfo: nil, } // Ensure we have a completely undeterministic PRNG. cmd.RandInt64() draws @@ -120,6 +86,9 @@ func NewSwitch(config *cfg.P2PConfig) *Switch { return sw } +//--------------------------------------------------------------------- +// Switch setup + // AddReactor adds the given reactor to the switch. // NOTE: Not goroutine safe. func (sw *Switch) AddReactor(name string, reactor Reactor) Reactor { @@ -171,26 +140,25 @@ func (sw *Switch) IsListening() bool { // SetNodeInfo sets the switch's NodeInfo for checking compatibility and handshaking with other nodes. // NOTE: Not goroutine safe. -func (sw *Switch) SetNodeInfo(nodeInfo *NodeInfo) { +func (sw *Switch) SetNodeInfo(nodeInfo NodeInfo) { sw.nodeInfo = nodeInfo } // NodeInfo returns the switch's NodeInfo. // NOTE: Not goroutine safe. -func (sw *Switch) NodeInfo() *NodeInfo { +func (sw *Switch) NodeInfo() NodeInfo { return sw.nodeInfo } -// SetNodePrivKey sets the switch's private key for authenticated encryption. -// NOTE: Overwrites sw.nodeInfo.PubKey. +// SetNodeKey sets the switch's private key for authenticated encryption. // NOTE: Not goroutine safe. -func (sw *Switch) SetNodePrivKey(nodePrivKey crypto.PrivKeyEd25519) { - sw.nodePrivKey = nodePrivKey - if sw.nodeInfo != nil { - sw.nodeInfo.PubKey = nodePrivKey.PubKey().Unwrap().(crypto.PubKeyEd25519) - } +func (sw *Switch) SetNodeKey(nodeKey *NodeKey) { + sw.nodeKey = nodeKey } +//--------------------------------------------------------------------- +// Service start/stop + // OnStart implements BaseService. It starts all the reactors, peers, and listeners. func (sw *Switch) OnStart() error { // Start reactors @@ -226,172 +194,26 @@ func (sw *Switch) OnStop() { } } -// addPeer checks the given peer's validity, performs a handshake, and adds the -// peer to the switch and to all registered reactors. -// NOTE: This performs a blocking handshake before the peer is added. -// NOTE: If error is returned, caller is responsible for calling peer.CloseConn() -func (sw *Switch) addPeer(peer *peer) error { - - if err := sw.FilterConnByAddr(peer.Addr()); err != nil { - return err - } - - if err := sw.FilterConnByPubKey(peer.PubKey()); err != nil { - return err - } - - if err := peer.HandshakeTimeout(sw.nodeInfo, time.Duration(sw.peerConfig.HandshakeTimeout*time.Second)); err != nil { - return err - } - - // Avoid self - if sw.nodeInfo.PubKey.Equals(peer.PubKey().Wrap()) { - return errors.New("Ignoring connection from self") - } - - // Check version, chain id - if err := sw.nodeInfo.CompatibleWith(peer.NodeInfo()); err != nil { - return err - } - - // Check for duplicate peer - if sw.peers.Has(peer.Key()) { - return ErrSwitchDuplicatePeer - - } - - // Start peer - if sw.IsRunning() { - sw.startInitPeer(peer) - } - - // Add the peer to .peers. - // We start it first so that a peer in the list is safe to Stop. - // It should not err since we already checked peers.Has(). - if err := sw.peers.Add(peer); err != nil { - return err - } - - sw.Logger.Info("Added peer", "peer", peer) - return nil -} - -// FilterConnByAddr returns an error if connecting to the given address is forbidden. -func (sw *Switch) FilterConnByAddr(addr net.Addr) error { - if sw.filterConnByAddr != nil { - return sw.filterConnByAddr(addr) - } - return nil -} - -// FilterConnByPubKey returns an error if connecting to the given public key is forbidden. -func (sw *Switch) FilterConnByPubKey(pubkey crypto.PubKeyEd25519) error { - if sw.filterConnByPubKey != nil { - return sw.filterConnByPubKey(pubkey) - } - return nil - -} - -// SetAddrFilter sets the function for filtering connections by address. -func (sw *Switch) SetAddrFilter(f func(net.Addr) error) { - sw.filterConnByAddr = f -} - -// SetPubKeyFilter sets the function for filtering connections by public key. -func (sw *Switch) SetPubKeyFilter(f func(crypto.PubKeyEd25519) error) { - sw.filterConnByPubKey = f -} - -func (sw *Switch) startInitPeer(peer *peer) { - err := peer.Start() // spawn send/recv routines - if err != nil { - // Should never happen - sw.Logger.Error("Error starting peer", "peer", peer, "err", err) - } +//--------------------------------------------------------------------- +// Peers - for _, reactor := range sw.reactors { - reactor.AddPeer(peer) - } +// Peers returns the set of peers that are connected to the switch. +func (sw *Switch) Peers() IPeerSet { + return sw.peers } -// DialSeeds dials a list of seeds asynchronously in random order. -func (sw *Switch) DialSeeds(addrBook *AddrBook, seeds []string) error { - netAddrs, errs := NewNetAddressStrings(seeds) - for _, err := range errs { - sw.Logger.Error("Error in seed's address", "err", err) - } - - if addrBook != nil { - // add seeds to `addrBook` - ourAddrS := sw.nodeInfo.ListenAddr - ourAddr, _ := NewNetAddressString(ourAddrS) - for _, netAddr := range netAddrs { - // do not add ourselves - if netAddr.Equals(ourAddr) { - continue - } - addrBook.AddAddress(netAddr, ourAddr) +// NumPeers returns the count of outbound/inbound and outbound-dialing peers. +func (sw *Switch) NumPeers() (outbound, inbound, dialing int) { + peers := sw.peers.List() + for _, peer := range peers { + if peer.IsOutbound() { + outbound++ + } else { + inbound++ } - addrBook.Save() - } - - // permute the list, dial them in random order. - perm := sw.rng.Perm(len(netAddrs)) - for i := 0; i < len(perm); i++ { - go func(i int) { - sw.randomSleep(0) - j := perm[i] - sw.dialSeed(netAddrs[j]) - }(i) - } - return nil -} - -// sleep for interval plus some random amount of ms on [0, dialRandomizerIntervalMilliseconds] -func (sw *Switch) randomSleep(interval time.Duration) { - r := time.Duration(sw.rng.Int63n(dialRandomizerIntervalMilliseconds)) * time.Millisecond - time.Sleep(r + interval) -} - -func (sw *Switch) dialSeed(addr *NetAddress) { - peer, err := sw.DialPeerWithAddress(addr, true) - if err != nil { - sw.Logger.Error("Error dialing seed", "err", err) - } else { - sw.Logger.Info("Connected to seed", "peer", peer) - } -} - -// DialPeerWithAddress dials the given peer and runs sw.addPeer if it connects successfully. -// If `persistent == true`, the switch will always try to reconnect to this peer if the connection ever fails. -func (sw *Switch) DialPeerWithAddress(addr *NetAddress, persistent bool) (Peer, error) { - sw.dialing.Set(addr.IP.String(), addr) - defer sw.dialing.Delete(addr.IP.String()) - - sw.Logger.Info("Dialing peer", "address", addr) - peer, err := newOutboundPeer(addr, sw.reactorsByCh, sw.chDescs, sw.StopPeerForError, sw.nodePrivKey, sw.peerConfig) - if err != nil { - sw.Logger.Error("Failed to dial peer", "address", addr, "err", err) - return nil, err - } - peer.SetLogger(sw.Logger.With("peer", addr)) - if persistent { - peer.makePersistent() - } - err = sw.addPeer(peer) - if err != nil { - sw.Logger.Error("Failed to add peer", "address", addr, "err", err) - peer.CloseConn() - return nil, err } - sw.Logger.Info("Dialed and added peer", "address", addr, "peer", peer) - return peer, nil -} - -// IsDialing returns true if the switch is currently dialing the given address. -func (sw *Switch) IsDialing(addr *NetAddress) bool { - return sw.dialing.Has(addr.IP.String()) + dialing = sw.dialing.Size() + return } // Broadcast runs a go routine for each attempted send, which will block @@ -411,25 +233,6 @@ func (sw *Switch) Broadcast(chID byte, msg interface{}) chan bool { return successChan } -// NumPeers returns the count of outbound/inbound and outbound-dialing peers. -func (sw *Switch) NumPeers() (outbound, inbound, dialing int) { - peers := sw.peers.List() - for _, peer := range peers { - if peer.IsOutbound() { - outbound++ - } else { - inbound++ - } - } - dialing = sw.dialing.Size() - return -} - -// Peers returns the set of peers that are connected to the switch. -func (sw *Switch) Peers() IPeerSet { - return sw.peers -} - // StopPeerForError disconnects from a peer due to external error. // If the peer is persistent, it will attempt to reconnect. // TODO: make record depending on reason. @@ -442,12 +245,27 @@ func (sw *Switch) StopPeerForError(peer Peer, reason interface{}) { } } +// StopPeerGracefully disconnects from a peer gracefully. +// TODO: handle graceful disconnects. +func (sw *Switch) StopPeerGracefully(peer Peer) { + sw.Logger.Info("Stopping peer gracefully") + sw.stopAndRemovePeer(peer, nil) +} + +func (sw *Switch) stopAndRemovePeer(peer Peer, reason interface{}) { + sw.peers.Remove(peer) + peer.Stop() + for _, reactor := range sw.reactors { + reactor.RemovePeer(peer, reason) + } +} + // reconnectToPeer tries to reconnect to the peer, first repeatedly // with a fixed interval, then with exponential backoff. // If no success after all that, it stops trying, and leaves it // to the PEX/Addrbook to find the peer again func (sw *Switch) reconnectToPeer(peer Peer) { - addr, _ := NewNetAddressString(peer.NodeInfo().RemoteAddr) + netAddr := peer.NodeInfo().NetAddress() start := time.Now() sw.Logger.Info("Reconnecting to peer", "peer", peer) for i := 0; i < reconnectAttempts; i++ { @@ -455,7 +273,7 @@ func (sw *Switch) reconnectToPeer(peer Peer) { return } - peer, err := sw.DialPeerWithAddress(addr, true) + peer, err := sw.DialPeerWithAddress(netAddr, true) if err != nil { sw.Logger.Info("Error reconnecting to peer. Trying again", "tries", i, "err", err, "peer", peer) // sleep a set amount @@ -477,7 +295,7 @@ func (sw *Switch) reconnectToPeer(peer Peer) { // sleep an exponentially increasing amount sleepIntervalSeconds := math.Pow(reconnectBackOffBaseSeconds, float64(i)) sw.randomSleep(time.Duration(sleepIntervalSeconds) * time.Second) - peer, err := sw.DialPeerWithAddress(addr, true) + peer, err := sw.DialPeerWithAddress(netAddr, true) if err != nil { sw.Logger.Info("Error reconnecting to peer. Trying again", "tries", i, "err", err, "peer", peer) continue @@ -489,21 +307,97 @@ func (sw *Switch) reconnectToPeer(peer Peer) { sw.Logger.Error("Failed to reconnect to peer. Giving up", "peer", peer, "elapsed", time.Since(start)) } -// StopPeerGracefully disconnects from a peer gracefully. -// TODO: handle graceful disconnects. -func (sw *Switch) StopPeerGracefully(peer Peer) { - sw.Logger.Info("Stopping peer gracefully") - sw.stopAndRemovePeer(peer, nil) +//--------------------------------------------------------------------- +// Dialing + +// IsDialing returns true if the switch is currently dialing the given ID. +func (sw *Switch) IsDialing(id ID) bool { + return sw.dialing.Has(string(id)) } -func (sw *Switch) stopAndRemovePeer(peer Peer, reason interface{}) { - sw.peers.Remove(peer) - peer.Stop() - for _, reactor := range sw.reactors { - reactor.RemovePeer(peer, reason) +// DialPeersAsync dials a list of peers asynchronously in random order (optionally, making them persistent). +func (sw *Switch) DialPeersAsync(addrBook *AddrBook, peers []string, persistent bool) error { + netAddrs, errs := NewNetAddressStrings(peers) + for _, err := range errs { + sw.Logger.Error("Error in peer's address", "err", err) + } + + if addrBook != nil { + // add peers to `addrBook` + ourAddr := sw.nodeInfo.NetAddress() + for _, netAddr := range netAddrs { + // do not add our address or ID + if netAddr.Same(ourAddr) { + continue + } + addrBook.AddAddress(netAddr, ourAddr) + } + addrBook.Save() + } + + // permute the list, dial them in random order. + perm := sw.rng.Perm(len(netAddrs)) + for i := 0; i < len(perm); i++ { + go func(i int) { + sw.randomSleep(0) + j := perm[i] + peer, err := sw.DialPeerWithAddress(netAddrs[j], persistent) + if err != nil { + sw.Logger.Error("Error dialing peer", "err", err) + } else { + sw.Logger.Info("Connected to peer", "peer", peer) + } + }(i) + } + return nil +} + +// DialPeerWithAddress dials the given peer and runs sw.addPeer if it connects and authenticates successfully. +// If `persistent == true`, the switch will always try to reconnect to this peer if the connection ever fails. +func (sw *Switch) DialPeerWithAddress(addr *NetAddress, persistent bool) (Peer, error) { + sw.dialing.Set(string(addr.ID), addr) + defer sw.dialing.Delete(string(addr.ID)) + return sw.addOutboundPeerWithConfig(addr, sw.peerConfig, persistent) +} + +// sleep for interval plus some random amount of ms on [0, dialRandomizerIntervalMilliseconds] +func (sw *Switch) randomSleep(interval time.Duration) { + r := time.Duration(sw.rng.Int63n(dialRandomizerIntervalMilliseconds)) * time.Millisecond + time.Sleep(r + interval) +} + +//------------------------------------------------------------------------------------ +// Connection filtering + +// FilterConnByAddr returns an error if connecting to the given address is forbidden. +func (sw *Switch) FilterConnByAddr(addr net.Addr) error { + if sw.filterConnByAddr != nil { + return sw.filterConnByAddr(addr) } + return nil +} + +// FilterConnByPubKey returns an error if connecting to the given public key is forbidden. +func (sw *Switch) FilterConnByPubKey(pubkey crypto.PubKey) error { + if sw.filterConnByPubKey != nil { + return sw.filterConnByPubKey(pubkey) + } + return nil + +} + +// SetAddrFilter sets the function for filtering connections by address. +func (sw *Switch) SetAddrFilter(f func(net.Addr) error) { + sw.filterConnByAddr = f } +// SetPubKeyFilter sets the function for filtering connections by public key. +func (sw *Switch) SetPubKeyFilter(f func(crypto.PubKey) error) { + sw.filterConnByPubKey = f +} + +//------------------------------------------------------------------------------------ + func (sw *Switch) listenerRoutine(l Listener) { for { inConn, ok := <-l.Connections() @@ -519,131 +413,124 @@ func (sw *Switch) listenerRoutine(l Listener) { } // New inbound connection! - err := sw.addPeerWithConnectionAndConfig(inConn, sw.peerConfig) + err := sw.addInboundPeerWithConfig(inConn, sw.peerConfig) if err != nil { sw.Logger.Info("Ignoring inbound connection: error while adding peer", "address", inConn.RemoteAddr().String(), "err", err) continue } - - // NOTE: We don't yet have the listening port of the - // remote (if they have a listener at all). - // The peerHandshake will handle that. } // cleanup } -//------------------------------------------------------------------ -// Connects switches via arbitrary net.Conn. Used for testing. - -// MakeConnectedSwitches returns n switches, connected according to the connect func. -// If connect==Connect2Switches, the switches will be fully connected. -// initSwitch defines how the i'th switch should be initialized (ie. with what reactors). -// NOTE: panics if any switch fails to start. -func MakeConnectedSwitches(cfg *cfg.P2PConfig, n int, initSwitch func(int, *Switch) *Switch, connect func([]*Switch, int, int)) []*Switch { - switches := make([]*Switch, n) - for i := 0; i < n; i++ { - switches[i] = makeSwitch(cfg, i, "testing", "123.123.123", initSwitch) +func (sw *Switch) addInboundPeerWithConfig(conn net.Conn, config *PeerConfig) error { + peer, err := newInboundPeer(conn, sw.reactorsByCh, sw.chDescs, sw.StopPeerForError, sw.nodeKey.PrivKey, config) + if err != nil { + peer.CloseConn() + return err } + peer.SetLogger(sw.Logger.With("peer", conn.RemoteAddr())) + if err = sw.addPeer(peer); err != nil { + peer.CloseConn() + return err + } + + return nil +} - if err := StartSwitches(switches); err != nil { - panic(err) +// dial the peer; make secret connection; authenticate against the dialed ID; +// add the peer. +func (sw *Switch) addOutboundPeerWithConfig(addr *NetAddress, config *PeerConfig, persistent bool) (Peer, error) { + sw.Logger.Info("Dialing peer", "address", addr) + peer, err := newOutboundPeer(addr, sw.reactorsByCh, sw.chDescs, sw.StopPeerForError, sw.nodeKey.PrivKey, config, persistent) + if err != nil { + sw.Logger.Error("Failed to dial peer", "address", addr, "err", err) + return nil, err } + peer.SetLogger(sw.Logger.With("peer", addr)) - for i := 0; i < n; i++ { - for j := i + 1; j < n; j++ { - connect(switches, i, j) - } + // authenticate peer + if addr.ID == "" { + peer.Logger.Info("Dialed peer with unknown ID - unable to authenticate", "addr", addr) + } else if addr.ID != peer.ID() { + peer.CloseConn() + return nil, fmt.Errorf("Failed to authenticate peer %v. Connected to peer with ID %s", addr, peer.ID()) } - return switches + err = sw.addPeer(peer) + if err != nil { + sw.Logger.Error("Failed to add peer", "address", addr, "err", err) + peer.CloseConn() + return nil, err + } + sw.Logger.Info("Dialed and added peer", "address", addr, "peer", peer) + return peer, nil } -// Connect2Switches will connect switches i and j via net.Pipe(). -// Blocks until a connection is established. -// NOTE: caller ensures i and j are within bounds. -func Connect2Switches(switches []*Switch, i, j int) { - switchI := switches[i] - switchJ := switches[j] - c1, c2 := netPipe() - doneCh := make(chan struct{}) - go func() { - err := switchI.addPeerWithConnection(c1) - if err != nil { - panic(err) - } - doneCh <- struct{}{} - }() - go func() { - err := switchJ.addPeerWithConnection(c2) - if err != nil { - panic(err) - } - doneCh <- struct{}{} - }() - <-doneCh - <-doneCh -} +// addPeer performs the Tendermint P2P handshake with a peer +// that already has a SecretConnection. If all goes well, +// it starts the peer and adds it to the switch. +// NOTE: This performs a blocking handshake before the peer is added. +// NOTE: If error is returned, caller is responsible for calling peer.CloseConn() +func (sw *Switch) addPeer(peer *peer) error { + // Avoid self + if sw.nodeKey.ID() == peer.ID() { + return ErrSwitchConnectToSelf + } + + // Avoid duplicate + if sw.peers.Has(peer.ID()) { + return ErrSwitchDuplicatePeer -// StartSwitches calls sw.Start() for each given switch. -// It returns the first encountered error. -func StartSwitches(switches []*Switch) error { - for _, s := range switches { - err := s.Start() // start switch and reactors - if err != nil { - return err - } } - return nil -} -func makeSwitch(cfg *cfg.P2PConfig, i int, network, version string, initSwitch func(int, *Switch) *Switch) *Switch { - privKey := crypto.GenPrivKeyEd25519() - // new switch, add reactors - // TODO: let the config be passed in? - s := initSwitch(i, NewSwitch(cfg)) - s.SetNodeInfo(&NodeInfo{ - PubKey: privKey.PubKey().Unwrap().(crypto.PubKeyEd25519), - Moniker: cmn.Fmt("switch%d", i), - Network: network, - Version: version, - RemoteAddr: cmn.Fmt("%v:%v", network, rand.Intn(64512)+1023), - ListenAddr: cmn.Fmt("%v:%v", network, rand.Intn(64512)+1023), - }) - s.SetNodePrivKey(privKey) - return s -} - -func (sw *Switch) addPeerWithConnection(conn net.Conn) error { - peer, err := newInboundPeer(conn, sw.reactorsByCh, sw.chDescs, sw.StopPeerForError, sw.nodePrivKey, sw.peerConfig) - if err != nil { - if err := conn.Close(); err != nil { - sw.Logger.Error("Error closing connection", "err", err) - } + // Filter peer against white list + if err := sw.FilterConnByAddr(peer.Addr()); err != nil { return err } - peer.SetLogger(sw.Logger.With("peer", conn.RemoteAddr())) - if err = sw.addPeer(peer); err != nil { - peer.CloseConn() + if err := sw.FilterConnByPubKey(peer.PubKey()); err != nil { return err } - return nil -} + // Exchange NodeInfo with the peer + if err := peer.HandshakeTimeout(sw.nodeInfo, time.Duration(sw.peerConfig.HandshakeTimeout*time.Second)); err != nil { + return err + } -func (sw *Switch) addPeerWithConnectionAndConfig(conn net.Conn, config *PeerConfig) error { - peer, err := newInboundPeer(conn, sw.reactorsByCh, sw.chDescs, sw.StopPeerForError, sw.nodePrivKey, config) - if err != nil { - if err := conn.Close(); err != nil { - sw.Logger.Error("Error closing connection", "err", err) - } + // Validate the peers nodeInfo against the pubkey + if err := peer.NodeInfo().Validate(peer.PubKey()); err != nil { return err } - peer.SetLogger(sw.Logger.With("peer", conn.RemoteAddr())) - if err = sw.addPeer(peer); err != nil { - peer.CloseConn() + + // Check version, chain id + if err := sw.nodeInfo.CompatibleWith(peer.NodeInfo()); err != nil { + return err + } + + // All good. Start peer + if sw.IsRunning() { + sw.startInitPeer(peer) + } + + // Add the peer to .peers. + // We start it first so that a peer in the list is safe to Stop. + // It should not err since we already checked peers.Has(). + if err := sw.peers.Add(peer); err != nil { return err } + sw.Logger.Info("Added peer", "peer", peer) return nil } + +func (sw *Switch) startInitPeer(peer *peer) { + err := peer.Start() // spawn send/recv routines + if err != nil { + // Should never happen + sw.Logger.Error("Error starting peer", "peer", peer, "err", err) + } + + for _, reactor := range sw.reactors { + reactor.AddPeer(peer) + } +} diff --git a/p2p/switch_test.go b/p2p/switch_test.go index 72807d36a..a729698e9 100644 --- a/p2p/switch_test.go +++ b/p2p/switch_test.go @@ -28,7 +28,7 @@ func init() { } type PeerMessage struct { - PeerKey string + PeerID ID Bytes []byte Counter int } @@ -77,7 +77,7 @@ func (tr *TestReactor) Receive(chID byte, peer Peer, msgBytes []byte) { tr.mtx.Lock() defer tr.mtx.Unlock() //fmt.Printf("Received: %X, %X\n", chID, msgBytes) - tr.msgsReceived[chID] = append(tr.msgsReceived[chID], PeerMessage{peer.Key(), msgBytes, tr.msgsCounter}) + tr.msgsReceived[chID] = append(tr.msgsReceived[chID], PeerMessage{peer.ID(), msgBytes, tr.msgsCounter}) tr.msgsCounter++ } } @@ -200,7 +200,7 @@ func TestConnPubKeyFilter(t *testing.T) { c1, c2 := netPipe() // set pubkey filter - s1.SetPubKeyFilter(func(pubkey crypto.PubKeyEd25519) error { + s1.SetPubKeyFilter(func(pubkey crypto.PubKey) error { if bytes.Equal(pubkey.Bytes(), s2.nodeInfo.PubKey.Bytes()) { return fmt.Errorf("Error: pipe is blacklisted") } @@ -232,11 +232,11 @@ func TestSwitchStopsNonPersistentPeerOnError(t *testing.T) { defer sw.Stop() // simulate remote peer - rp := &remotePeer{PrivKey: crypto.GenPrivKeyEd25519(), Config: DefaultPeerConfig()} + rp := &remotePeer{PrivKey: crypto.GenPrivKeyEd25519().Wrap(), Config: DefaultPeerConfig()} rp.Start() defer rp.Stop() - peer, err := newOutboundPeer(rp.Addr(), sw.reactorsByCh, sw.chDescs, sw.StopPeerForError, sw.nodePrivKey, DefaultPeerConfig()) + peer, err := newOutboundPeer(rp.Addr(), sw.reactorsByCh, sw.chDescs, sw.StopPeerForError, sw.nodeKey.PrivKey, DefaultPeerConfig(), false) require.Nil(err) err = sw.addPeer(peer) require.Nil(err) @@ -259,12 +259,11 @@ func TestSwitchReconnectsToPersistentPeer(t *testing.T) { defer sw.Stop() // simulate remote peer - rp := &remotePeer{PrivKey: crypto.GenPrivKeyEd25519(), Config: DefaultPeerConfig()} + rp := &remotePeer{PrivKey: crypto.GenPrivKeyEd25519().Wrap(), Config: DefaultPeerConfig()} rp.Start() defer rp.Stop() - peer, err := newOutboundPeer(rp.Addr(), sw.reactorsByCh, sw.chDescs, sw.StopPeerForError, sw.nodePrivKey, DefaultPeerConfig()) - peer.makePersistent() + peer, err := newOutboundPeer(rp.Addr(), sw.reactorsByCh, sw.chDescs, sw.StopPeerForError, sw.nodeKey.PrivKey, DefaultPeerConfig(), true) require.Nil(err) err = sw.addPeer(peer) require.Nil(err) diff --git a/p2p/test_util.go b/p2p/test_util.go new file mode 100644 index 000000000..18167e209 --- /dev/null +++ b/p2p/test_util.go @@ -0,0 +1,111 @@ +package p2p + +import ( + "math/rand" + "net" + + crypto "github.com/tendermint/go-crypto" + cfg "github.com/tendermint/tendermint/config" + cmn "github.com/tendermint/tmlibs/common" + "github.com/tendermint/tmlibs/log" +) + +//------------------------------------------------------------------ +// Connects switches via arbitrary net.Conn. Used for testing. + +// MakeConnectedSwitches returns n switches, connected according to the connect func. +// If connect==Connect2Switches, the switches will be fully connected. +// initSwitch defines how the i'th switch should be initialized (ie. with what reactors). +// NOTE: panics if any switch fails to start. +func MakeConnectedSwitches(cfg *cfg.P2PConfig, n int, initSwitch func(int, *Switch) *Switch, connect func([]*Switch, int, int)) []*Switch { + switches := make([]*Switch, n) + for i := 0; i < n; i++ { + switches[i] = makeSwitch(cfg, i, "testing", "123.123.123", initSwitch) + } + + if err := StartSwitches(switches); err != nil { + panic(err) + } + + for i := 0; i < n; i++ { + for j := i + 1; j < n; j++ { + connect(switches, i, j) + } + } + + return switches +} + +// Connect2Switches will connect switches i and j via net.Pipe(). +// Blocks until a connection is established. +// NOTE: caller ensures i and j are within bounds. +func Connect2Switches(switches []*Switch, i, j int) { + switchI := switches[i] + switchJ := switches[j] + c1, c2 := netPipe() + doneCh := make(chan struct{}) + go func() { + err := switchI.addPeerWithConnection(c1) + if err != nil { + panic(err) + } + doneCh <- struct{}{} + }() + go func() { + err := switchJ.addPeerWithConnection(c2) + if err != nil { + panic(err) + } + doneCh <- struct{}{} + }() + <-doneCh + <-doneCh +} + +func (sw *Switch) addPeerWithConnection(conn net.Conn) error { + peer, err := newInboundPeer(conn, sw.reactorsByCh, sw.chDescs, sw.StopPeerForError, sw.nodeKey.PrivKey, sw.peerConfig) + if err != nil { + if err := conn.Close(); err != nil { + sw.Logger.Error("Error closing connection", "err", err) + } + return err + } + peer.SetLogger(sw.Logger.With("peer", conn.RemoteAddr())) + if err = sw.addPeer(peer); err != nil { + peer.CloseConn() + return err + } + + return nil +} + +// StartSwitches calls sw.Start() for each given switch. +// It returns the first encountered error. +func StartSwitches(switches []*Switch) error { + for _, s := range switches { + err := s.Start() // start switch and reactors + if err != nil { + return err + } + } + return nil +} + +func makeSwitch(cfg *cfg.P2PConfig, i int, network, version string, initSwitch func(int, *Switch) *Switch) *Switch { + // new switch, add reactors + // TODO: let the config be passed in? + nodeKey := &NodeKey{ + PrivKey: crypto.GenPrivKeyEd25519().Wrap(), + } + s := initSwitch(i, NewSwitch(cfg)) + s.SetNodeInfo(NodeInfo{ + PubKey: nodeKey.PubKey(), + Moniker: cmn.Fmt("switch%d", i), + Network: network, + Version: version, + ListenAddr: cmn.Fmt("%v:%v", network, rand.Intn(64512)+1023), + }) + s.SetNodeKey(nodeKey) + s.SetLogger(log.TestingLogger()) + return s +} diff --git a/p2p/trust/metric_test.go b/p2p/trust/metric_test.go index 00219a19e..69c9f8f2a 100644 --- a/p2p/trust/metric_test.go +++ b/p2p/trust/metric_test.go @@ -68,7 +68,9 @@ func TestTrustMetricStopPause(t *testing.T) { tt.NextTick() tm.Pause() + // could be 1 or 2 because Pause and NextTick race first := tm.Copy().numIntervals + // Allow more time to pass and check the intervals are unchanged tt.NextTick() tt.NextTick() diff --git a/p2p/trust/ticker.go b/p2p/trust/ticker.go index bce9fcc24..3f0f30919 100644 --- a/p2p/trust/ticker.go +++ b/p2p/trust/ticker.go @@ -24,7 +24,7 @@ type TestTicker struct { // NewTestTicker returns our ticker used within test routines func NewTestTicker() *TestTicker { - c := make(chan time.Time, 1) + c := make(chan time.Time) return &TestTicker{ C: c, } diff --git a/p2p/types.go b/p2p/types.go index 4e0994b71..d93adc9b6 100644 --- a/p2p/types.go +++ b/p2p/types.go @@ -11,18 +11,35 @@ import ( const maxNodeInfoSize = 10240 // 10Kb +// NodeInfo is the basic node information exchanged +// between two peers during the Tendermint P2P handshake. type NodeInfo struct { - PubKey crypto.PubKeyEd25519 `json:"pub_key"` - Moniker string `json:"moniker"` - Network string `json:"network"` - RemoteAddr string `json:"remote_addr"` - ListenAddr string `json:"listen_addr"` - Version string `json:"version"` // major.minor.revision - Other []string `json:"other"` // other application specific data + // Authenticate + PubKey crypto.PubKey `json:"pub_key"` // authenticated pubkey + ListenAddr string `json:"listen_addr"` // accepting incoming + + // Check compatibility + Network string `json:"network"` // network/chain ID + Version string `json:"version"` // major.minor.revision + + // Sanitize + Moniker string `json:"moniker"` // arbitrary moniker + Other []string `json:"other"` // other application specific data +} + +// Validate checks the self-reported NodeInfo is safe. +// It returns an error if the info.PubKey doesn't match the given pubKey. +// TODO: constraints for Moniker/Other? Or is that for the UI ? +func (info NodeInfo) Validate(pubKey crypto.PubKey) error { + if !info.PubKey.Equals(pubKey) { + return fmt.Errorf("info.PubKey (%v) doesn't match peer.PubKey (%v)", + info.PubKey, pubKey) + } + return nil } // CONTRACT: two nodes are compatible if the major/minor versions match and network match -func (info *NodeInfo) CompatibleWith(other *NodeInfo) error { +func (info NodeInfo) CompatibleWith(other NodeInfo) error { iMajor, iMinor, _, iErr := splitVersion(info.Version) oMajor, oMinor, _, oErr := splitVersion(other.Version) @@ -54,12 +71,26 @@ func (info *NodeInfo) CompatibleWith(other *NodeInfo) error { return nil } -func (info *NodeInfo) ListenHost() string { +func (info NodeInfo) ID() ID { + return PubKeyToID(info.PubKey) +} + +func (info NodeInfo) NetAddress() *NetAddress { + id := PubKeyToID(info.PubKey) + addr := info.ListenAddr + netAddr, err := NewNetAddressString(IDAddressString(id, addr)) + if err != nil { + panic(err) // everything should be well formed by now + } + return netAddr +} + +func (info NodeInfo) ListenHost() string { host, _, _ := net.SplitHostPort(info.ListenAddr) // nolint: errcheck, gas return host } -func (info *NodeInfo) ListenPort() int { +func (info NodeInfo) ListenPort() int { _, port, _ := net.SplitHostPort(info.ListenAddr) // nolint: errcheck, gas port_i, err := strconv.Atoi(port) if err != nil { @@ -69,7 +100,7 @@ func (info *NodeInfo) ListenPort() int { } func (info NodeInfo) String() string { - return fmt.Sprintf("NodeInfo{pk: %v, moniker: %v, network: %v [remote %v, listen %v], version: %v (%v)}", info.PubKey, info.Moniker, info.Network, info.RemoteAddr, info.ListenAddr, info.Version, info.Other) + return fmt.Sprintf("NodeInfo{pk: %v, moniker: %v, network: %v [listen %v], version: %v (%v)}", info.PubKey, info.Moniker, info.Network, info.ListenAddr, info.Version, info.Other) } func splitVersion(version string) (string, string, string, error) { diff --git a/p2p/util.go b/p2p/util.go deleted file mode 100644 index a4c3ad58b..000000000 --- a/p2p/util.go +++ /dev/null @@ -1,15 +0,0 @@ -package p2p - -import ( - "crypto/sha256" -) - -// doubleSha256 calculates sha256(sha256(b)) and returns the resulting bytes. -func doubleSha256(b []byte) []byte { - hasher := sha256.New() - hasher.Write(b) // nolint: errcheck, gas - sum := hasher.Sum(nil) - hasher.Reset() - hasher.Write(sum) // nolint: errcheck, gas - return hasher.Sum(nil) -} diff --git a/rpc/client/localclient.go b/rpc/client/localclient.go index 71f25ef23..5e0573a1b 100644 --- a/rpc/client/localclient.go +++ b/rpc/client/localclient.go @@ -88,6 +88,10 @@ func (Local) DialSeeds(seeds []string) (*ctypes.ResultDialSeeds, error) { return core.UnsafeDialSeeds(seeds) } +func (Local) DialPeers(peers []string, persistent bool) (*ctypes.ResultDialPeers, error) { + return core.UnsafeDialPeers(peers, persistent) +} + func (Local) BlockchainInfo(minHeight, maxHeight int64) (*ctypes.ResultBlockchainInfo, error) { return core.BlockchainInfo(minHeight, maxHeight) } diff --git a/rpc/client/mock/client.go b/rpc/client/mock/client.go index dc75e04cb..6c4728986 100644 --- a/rpc/client/mock/client.go +++ b/rpc/client/mock/client.go @@ -111,6 +111,10 @@ func (c Client) DialSeeds(seeds []string) (*ctypes.ResultDialSeeds, error) { return core.UnsafeDialSeeds(seeds) } +func (c Client) DialPeers(peers []string, persistent bool) (*ctypes.ResultDialPeers, error) { + return core.UnsafeDialPeers(peers, persistent) +} + func (c Client) BlockchainInfo(minHeight, maxHeight int64) (*ctypes.ResultBlockchainInfo, error) { return core.BlockchainInfo(minHeight, maxHeight) } diff --git a/rpc/core/consensus.go b/rpc/core/consensus.go index 65c9fc364..25b67925c 100644 --- a/rpc/core/consensus.go +++ b/rpc/core/consensus.go @@ -3,6 +3,7 @@ package core import ( cm "github.com/tendermint/tendermint/consensus" cstypes "github.com/tendermint/tendermint/consensus/types" + p2p "github.com/tendermint/tendermint/p2p" ctypes "github.com/tendermint/tendermint/rpc/core/types" sm "github.com/tendermint/tendermint/state" "github.com/tendermint/tendermint/types" @@ -82,11 +83,11 @@ func Validators(heightPtr *int64) (*ctypes.ResultValidators, error) { // } // ``` func DumpConsensusState() (*ctypes.ResultDumpConsensusState, error) { - peerRoundStates := make(map[string]*cstypes.PeerRoundState) + peerRoundStates := make(map[p2p.ID]*cstypes.PeerRoundState) for _, peer := range p2pSwitch.Peers().List() { peerState := peer.Get(types.PeerStateKey).(*cm.PeerState) peerRoundState := peerState.GetRoundState() - peerRoundStates[peer.Key()] = peerRoundState + peerRoundStates[peer.ID()] = peerRoundState } return &ctypes.ResultDumpConsensusState{consensusState.GetRoundState(), peerRoundStates}, nil } diff --git a/rpc/core/doc.go b/rpc/core/doc.go index d3ec5d3b6..5f3e2dced 100644 --- a/rpc/core/doc.go +++ b/rpc/core/doc.go @@ -95,6 +95,7 @@ Endpoints that require arguments: /broadcast_tx_sync?tx=_ /commit?height=_ /dial_seeds?seeds=_ +/dial_persistent_peers?persistent_peers=_ /subscribe?event=_ /tx?hash=_&prove=_ /unsafe_start_cpu_profiler?filename=_ diff --git a/rpc/core/net.go b/rpc/core/net.go index b3f1c7ce5..14e7389d5 100644 --- a/rpc/core/net.go +++ b/rpc/core/net.go @@ -1,8 +1,7 @@ package core import ( - "fmt" - + "github.com/pkg/errors" ctypes "github.com/tendermint/tendermint/rpc/core/types" ) @@ -42,7 +41,7 @@ func NetInfo() (*ctypes.ResultNetInfo, error) { peers := []ctypes.Peer{} for _, peer := range p2pSwitch.Peers().List() { peers = append(peers, ctypes.Peer{ - NodeInfo: *peer.NodeInfo(), + NodeInfo: peer.NodeInfo(), IsOutbound: peer.IsOutbound(), ConnectionStatus: peer.Status(), }) @@ -55,19 +54,31 @@ func NetInfo() (*ctypes.ResultNetInfo, error) { } func UnsafeDialSeeds(seeds []string) (*ctypes.ResultDialSeeds, error) { - if len(seeds) == 0 { - return &ctypes.ResultDialSeeds{}, fmt.Errorf("No seeds provided") + return &ctypes.ResultDialSeeds{}, errors.New("No seeds provided") } - // starts go routines to dial each seed after random delays + // starts go routines to dial each peer after random delays logger.Info("DialSeeds", "addrBook", addrBook, "seeds", seeds) - err := p2pSwitch.DialSeeds(addrBook, seeds) + err := p2pSwitch.DialPeersAsync(addrBook, seeds, false) if err != nil { return &ctypes.ResultDialSeeds{}, err } return &ctypes.ResultDialSeeds{"Dialing seeds in progress. See /net_info for details"}, nil } +func UnsafeDialPeers(peers []string, persistent bool) (*ctypes.ResultDialPeers, error) { + if len(peers) == 0 { + return &ctypes.ResultDialPeers{}, errors.New("No peers provided") + } + // starts go routines to dial each peer after random delays + logger.Info("DialPeers", "addrBook", addrBook, "peers", peers, "persistent", persistent) + err := p2pSwitch.DialPeersAsync(addrBook, peers, persistent) + if err != nil { + return &ctypes.ResultDialPeers{}, err + } + return &ctypes.ResultDialPeers{"Dialing peers in progress. See /net_info for details"}, nil +} + // Get genesis file. // // ```shell diff --git a/rpc/core/pipe.go b/rpc/core/pipe.go index 927d7ccad..301977ac3 100644 --- a/rpc/core/pipe.go +++ b/rpc/core/pipe.go @@ -30,9 +30,9 @@ type P2P interface { Listeners() []p2p.Listener Peers() p2p.IPeerSet NumPeers() (outbound, inbound, dialig int) - NodeInfo() *p2p.NodeInfo + NodeInfo() p2p.NodeInfo IsListening() bool - DialSeeds(*p2p.AddrBook, []string) error + DialPeersAsync(*p2p.AddrBook, []string, bool) error } //---------------------------------------------- diff --git a/rpc/core/routes.go b/rpc/core/routes.go index fb5a1fd36..3ea7aa08c 100644 --- a/rpc/core/routes.go +++ b/rpc/core/routes.go @@ -39,6 +39,7 @@ var Routes = map[string]*rpc.RPCFunc{ func AddUnsafeRoutes() { // control API Routes["dial_seeds"] = rpc.NewRPCFunc(UnsafeDialSeeds, "seeds") + Routes["dial_peers"] = rpc.NewRPCFunc(UnsafeDialPeers, "peers,persistent") Routes["unsafe_flush_mempool"] = rpc.NewRPCFunc(UnsafeFlushMempool, "") // profiler API diff --git a/rpc/core/types/responses.go b/rpc/core/types/responses.go index dae7c0046..233b44e93 100644 --- a/rpc/core/types/responses.go +++ b/rpc/core/types/responses.go @@ -54,7 +54,7 @@ func NewResultCommit(header *types.Header, commit *types.Commit, } type ResultStatus struct { - NodeInfo *p2p.NodeInfo `json:"node_info"` + NodeInfo p2p.NodeInfo `json:"node_info"` PubKey crypto.PubKey `json:"pub_key"` LatestBlockHash data.Bytes `json:"latest_block_hash"` LatestAppHash data.Bytes `json:"latest_app_hash"` @@ -64,7 +64,7 @@ type ResultStatus struct { } func (s *ResultStatus) TxIndexEnabled() bool { - if s == nil || s.NodeInfo == nil { + if s == nil { return false } for _, s := range s.NodeInfo.Other { @@ -86,6 +86,10 @@ type ResultDialSeeds struct { Log string `json:"log"` } +type ResultDialPeers struct { + Log string `json:"log"` +} + type Peer struct { p2p.NodeInfo `json:"node_info"` IsOutbound bool `json:"is_outbound"` @@ -99,7 +103,7 @@ type ResultValidators struct { type ResultDumpConsensusState struct { RoundState *cstypes.RoundState `json:"round_state"` - PeerRoundStates map[string]*cstypes.PeerRoundState `json:"peer_round_states"` + PeerRoundStates map[p2p.ID]*cstypes.PeerRoundState `json:"peer_round_states"` } type ResultBroadcastTx struct { diff --git a/rpc/core/types/responses_test.go b/rpc/core/types/responses_test.go index fa0da3fdf..e410d47ae 100644 --- a/rpc/core/types/responses_test.go +++ b/rpc/core/types/responses_test.go @@ -17,7 +17,7 @@ func TestStatusIndexer(t *testing.T) { status = &ResultStatus{} assert.False(status.TxIndexEnabled()) - status.NodeInfo = &p2p.NodeInfo{} + status.NodeInfo = p2p.NodeInfo{} assert.False(status.TxIndexEnabled()) cases := []struct { diff --git a/test/p2p/README.md b/test/p2p/README.md index e2a577cfa..b68f7a819 100644 --- a/test/p2p/README.md +++ b/test/p2p/README.md @@ -38,7 +38,7 @@ for i in $(seq 1 4); do --name local_testnet_$i \ --entrypoint tendermint \ -e TMHOME=/go/src/github.com/tendermint/tendermint/test/p2p/data/mach$i/core \ - tendermint_tester node --p2p.seeds 172.57.0.101:46656,172.57.0.102:46656,172.57.0.103:46656,172.57.0.104:46656 --proxy_app=dummy + tendermint_tester node --p2p.persistent_peers 172.57.0.101:46656,172.57.0.102:46656,172.57.0.103:46656,172.57.0.104:46656 --proxy_app=dummy done ``` diff --git a/test/p2p/fast_sync/test_peer.sh b/test/p2p/fast_sync/test_peer.sh index 8174be0e7..1f341bf5d 100644 --- a/test/p2p/fast_sync/test_peer.sh +++ b/test/p2p/fast_sync/test_peer.sh @@ -23,11 +23,11 @@ docker rm -vf local_testnet_$ID set -e # restart peer - should have an empty blockchain -SEEDS="$(test/p2p/ip.sh 1):46656" +PERSISTENT_PEERS="$(test/p2p/ip.sh 1):46656" for j in `seq 2 $N`; do - SEEDS="$SEEDS,$(test/p2p/ip.sh $j):46656" + PERSISTENT_PEERS="$PERSISTENT_PEERS,$(test/p2p/ip.sh $j):46656" done -bash test/p2p/peer.sh $DOCKER_IMAGE $NETWORK_NAME $ID $PROXY_APP "--p2p.seeds $SEEDS --p2p.pex --rpc.unsafe" +bash test/p2p/peer.sh $DOCKER_IMAGE $NETWORK_NAME $ID $PROXY_APP "--p2p.persistent_peers $PERSISTENT_PEERS --p2p.pex --rpc.unsafe" # wait for peer to sync and check the app hash bash test/p2p/client.sh $DOCKER_IMAGE $NETWORK_NAME fs_$ID "test/p2p/fast_sync/check_peer.sh $ID" diff --git a/test/p2p/local_testnet_start.sh b/test/p2p/local_testnet_start.sh index f70bdf04c..25b3c6d3e 100644 --- a/test/p2p/local_testnet_start.sh +++ b/test/p2p/local_testnet_start.sh @@ -7,10 +7,10 @@ N=$3 APP_PROXY=$4 set +u -SEEDS=$5 -if [[ "$SEEDS" != "" ]]; then - echo "Seeds: $SEEDS" - SEEDS="--p2p.seeds $SEEDS" +PERSISTENT_PEERS=$5 +if [[ "$PERSISTENT_PEERS" != "" ]]; then + echo "PersistentPeers: $PERSISTENT_PEERS" + PERSISTENT_PEERS="--p2p.persistent_peers $PERSISTENT_PEERS" fi set -u @@ -20,5 +20,5 @@ cd "$GOPATH/src/github.com/tendermint/tendermint" docker network create --driver bridge --subnet 172.57.0.0/16 "$NETWORK_NAME" for i in $(seq 1 "$N"); do - bash test/p2p/peer.sh "$DOCKER_IMAGE" "$NETWORK_NAME" "$i" "$APP_PROXY" "$SEEDS --p2p.pex --rpc.unsafe" + bash test/p2p/peer.sh "$DOCKER_IMAGE" "$NETWORK_NAME" "$i" "$APP_PROXY" "$PERSISTENT_PEERS --p2p.pex --rpc.unsafe" done diff --git a/test/p2p/persistent_peers.sh b/test/p2p/persistent_peers.sh new file mode 100644 index 000000000..4ad55bc03 --- /dev/null +++ b/test/p2p/persistent_peers.sh @@ -0,0 +1,12 @@ +#! /bin/bash +set -eu + +N=$1 + +cd "$GOPATH/src/github.com/tendermint/tendermint" + +persistent_peers="$(test/p2p/ip.sh 1):46656" +for i in $(seq 2 $N); do + persistent_peers="$persistent_peers,$(test/p2p/ip.sh $i):46656" +done +echo "$persistent_peers" diff --git a/test/p2p/pex/dial_seeds.sh b/test/p2p/pex/dial_peers.sh similarity index 63% rename from test/p2p/pex/dial_seeds.sh rename to test/p2p/pex/dial_peers.sh index 15c22af67..ddda7dbe7 100644 --- a/test/p2p/pex/dial_seeds.sh +++ b/test/p2p/pex/dial_peers.sh @@ -1,4 +1,4 @@ -#! /bin/bash +#! /bin/bash set -u N=$1 @@ -11,7 +11,7 @@ for i in `seq 1 $N`; do curl -s $addr/status > /dev/null ERR=$? while [ "$ERR" != 0 ]; do - sleep 1 + sleep 1 curl -s $addr/status > /dev/null ERR=$? done @@ -19,13 +19,14 @@ for i in `seq 1 $N`; do done set -e -# seeds need quotes -seeds="\"$(test/p2p/ip.sh 1):46656\"" +# peers need quotes +peers="\"$(test/p2p/ip.sh 1):46656\"" for i in `seq 2 $N`; do - seeds="$seeds,\"$(test/p2p/ip.sh $i):46656\"" + peers="$peers,\"$(test/p2p/ip.sh $i):46656\"" done -echo $seeds +echo $peers -echo $seeds +echo $peers IP=$(test/p2p/ip.sh 1) -curl --data-urlencode "seeds=[$seeds]" "$IP:46657/dial_seeds" +curl "$IP:46657/dial_peers?persistent=true&peers=\[$peers\]" + diff --git a/test/p2p/pex/test.sh b/test/p2p/pex/test.sh index d54d81350..ffecd6510 100644 --- a/test/p2p/pex/test.sh +++ b/test/p2p/pex/test.sh @@ -6,10 +6,10 @@ NETWORK_NAME=$2 N=$3 PROXY_APP=$4 -cd $GOPATH/src/github.com/tendermint/tendermint +cd "$GOPATH/src/github.com/tendermint/tendermint" echo "Test reconnecting from the address book" -bash test/p2p/pex/test_addrbook.sh $DOCKER_IMAGE $NETWORK_NAME $N $PROXY_APP +bash test/p2p/pex/test_addrbook.sh "$DOCKER_IMAGE" "$NETWORK_NAME" "$N" "$PROXY_APP" -echo "Test connecting via /dial_seeds" -bash test/p2p/pex/test_dial_seeds.sh $DOCKER_IMAGE $NETWORK_NAME $N $PROXY_APP +echo "Test connecting via /dial_peers" +bash test/p2p/pex/test_dial_peers.sh "$DOCKER_IMAGE" "$NETWORK_NAME" "$N" "$PROXY_APP" diff --git a/test/p2p/pex/test_addrbook.sh b/test/p2p/pex/test_addrbook.sh index 35dcb89d3..1dd26b172 100644 --- a/test/p2p/pex/test_addrbook.sh +++ b/test/p2p/pex/test_addrbook.sh @@ -9,7 +9,7 @@ PROXY_APP=$4 ID=1 echo "----------------------------------------------------------------------" -echo "Testing pex creates the addrbook and uses it if seeds are not provided" +echo "Testing pex creates the addrbook and uses it if persistent_peers are not provided" echo "(assuming peers are started with pex enabled)" CLIENT_NAME="pex_addrbook_$ID" @@ -22,7 +22,7 @@ set +e #CIRCLE docker rm -vf "local_testnet_$ID" set -e -# NOTE that we do not provide seeds +# NOTE that we do not provide persistent_peers bash test/p2p/peer.sh "$DOCKER_IMAGE" "$NETWORK_NAME" "$ID" "$PROXY_APP" "--p2p.pex --rpc.unsafe" docker cp "/tmp/addrbook.json" "local_testnet_$ID:/go/src/github.com/tendermint/tendermint/test/p2p/data/mach1/core/addrbook.json" echo "with the following addrbook:" @@ -35,7 +35,7 @@ echo "" bash test/p2p/client.sh "$DOCKER_IMAGE" "$NETWORK_NAME" "$CLIENT_NAME" "test/p2p/pex/check_peer.sh $ID $N" echo "----------------------------------------------------------------------" -echo "Testing other peers connect to us if we have neither seeds nor the addrbook" +echo "Testing other peers connect to us if we have neither persistent_peers nor the addrbook" echo "(assuming peers are started with pex enabled)" CLIENT_NAME="pex_no_addrbook_$ID" @@ -46,7 +46,7 @@ set +e #CIRCLE docker rm -vf "local_testnet_$ID" set -e -# NOTE that we do not provide seeds +# NOTE that we do not provide persistent_peers bash test/p2p/peer.sh "$DOCKER_IMAGE" "$NETWORK_NAME" "$ID" "$PROXY_APP" "--p2p.pex --rpc.unsafe" # if the client runs forever, it means other peers have removed us from their books (which should not happen) diff --git a/test/p2p/pex/test_dial_seeds.sh b/test/p2p/pex/test_dial_peers.sh similarity index 77% rename from test/p2p/pex/test_dial_seeds.sh rename to test/p2p/pex/test_dial_peers.sh index ea72004db..d0b042342 100644 --- a/test/p2p/pex/test_dial_seeds.sh +++ b/test/p2p/pex/test_dial_peers.sh @@ -11,7 +11,7 @@ ID=1 cd $GOPATH/src/github.com/tendermint/tendermint echo "----------------------------------------------------------------------" -echo "Testing full network connection using one /dial_seeds call" +echo "Testing full network connection using one /dial_peers call" echo "(assuming peers are started with pex enabled)" # stop the existing testnet and remove local network @@ -21,16 +21,16 @@ set -e # start the testnet on a local network # NOTE we re-use the same network for all tests -SEEDS="" -bash test/p2p/local_testnet_start.sh $DOCKER_IMAGE $NETWORK_NAME $N $PROXY_APP $SEEDS +PERSISTENT_PEERS="" +bash test/p2p/local_testnet_start.sh $DOCKER_IMAGE $NETWORK_NAME $N $PROXY_APP $PERSISTENT_PEERS -# dial seeds from one node -CLIENT_NAME="dial_seeds" -bash test/p2p/client.sh $DOCKER_IMAGE $NETWORK_NAME $CLIENT_NAME "test/p2p/pex/dial_seeds.sh $N" +# dial peers from one node +CLIENT_NAME="dial_peers" +bash test/p2p/client.sh $DOCKER_IMAGE $NETWORK_NAME $CLIENT_NAME "test/p2p/pex/dial_peers.sh $N" # test basic connectivity and consensus # start client container and check the num peers and height for all nodes -CLIENT_NAME="dial_seeds_basic" +CLIENT_NAME="dial_peers_basic" bash test/p2p/client.sh $DOCKER_IMAGE $NETWORK_NAME $CLIENT_NAME "test/p2p/basic/test.sh $N" diff --git a/test/p2p/seeds.sh b/test/p2p/seeds.sh deleted file mode 100644 index 4bf866cba..000000000 --- a/test/p2p/seeds.sh +++ /dev/null @@ -1,12 +0,0 @@ -#! /bin/bash -set -eu - -N=$1 - -cd "$GOPATH/src/github.com/tendermint/tendermint" - -seeds="$(test/p2p/ip.sh 1):46656" -for i in $(seq 2 $N); do - seeds="$seeds,$(test/p2p/ip.sh $i):46656" -done -echo "$seeds" diff --git a/test/p2p/test.sh b/test/p2p/test.sh index 6a5537b98..c95f69733 100644 --- a/test/p2p/test.sh +++ b/test/p2p/test.sh @@ -13,11 +13,11 @@ set +e bash test/p2p/local_testnet_stop.sh "$NETWORK_NAME" "$N" set -e -SEEDS=$(bash test/p2p/seeds.sh $N) +PERSISTENT_PEERS=$(bash test/p2p/persistent_peers.sh $N) # start the testnet on a local network # NOTE we re-use the same network for all tests -bash test/p2p/local_testnet_start.sh "$DOCKER_IMAGE" "$NETWORK_NAME" "$N" "$PROXY_APP" "$SEEDS" +bash test/p2p/local_testnet_start.sh "$DOCKER_IMAGE" "$NETWORK_NAME" "$N" "$PROXY_APP" "$PERSISTENT_PEERS" # test basic connectivity and consensus # start client container and check the num peers and height for all nodes diff --git a/types/vote_set.go b/types/vote_set.go index 34f989567..a97676f6e 100644 --- a/types/vote_set.go +++ b/types/vote_set.go @@ -8,6 +8,7 @@ import ( "github.com/pkg/errors" + "github.com/tendermint/tendermint/p2p" cmn "github.com/tendermint/tmlibs/common" ) @@ -58,7 +59,7 @@ type VoteSet struct { sum int64 // Sum of voting power for seen votes, discounting conflicts maj23 *BlockID // First 2/3 majority seen votesByBlock map[string]*blockVotes // string(blockHash|blockParts) -> blockVotes - peerMaj23s map[string]BlockID // Maj23 for each peer + peerMaj23s map[p2p.ID]BlockID // Maj23 for each peer } // Constructs a new VoteSet struct used to accumulate votes for given height/round. @@ -77,7 +78,7 @@ func NewVoteSet(chainID string, height int64, round int, type_ byte, valSet *Val sum: 0, maj23: nil, votesByBlock: make(map[string]*blockVotes, valSet.Size()), - peerMaj23s: make(map[string]BlockID), + peerMaj23s: make(map[p2p.ID]BlockID), } } @@ -290,7 +291,7 @@ func (voteSet *VoteSet) addVerifiedVote(vote *Vote, blockKey string, votingPower // this can cause memory issues. // TODO: implement ability to remove peers too // NOTE: VoteSet must not be nil -func (voteSet *VoteSet) SetPeerMaj23(peerID string, blockID BlockID) { +func (voteSet *VoteSet) SetPeerMaj23(peerID p2p.ID, blockID BlockID) { if voteSet == nil { cmn.PanicSanity("SetPeerMaj23() on nil VoteSet") }