p2p: tmconn->conn and types->p2p

This commit is contained in:
Ethan Buchman
2018-01-21 00:33:53 -05:00
parent 0d7d16005a
commit 44e967184a
28 changed files with 160 additions and 177 deletions

View File

@ -24,7 +24,6 @@ import (
"github.com/tendermint/tendermint/p2p" "github.com/tendermint/tendermint/p2p"
"github.com/tendermint/tendermint/p2p/pex" "github.com/tendermint/tendermint/p2p/pex"
"github.com/tendermint/tendermint/p2p/trust" "github.com/tendermint/tendermint/p2p/trust"
p2ptypes "github.com/tendermint/tendermint/p2p/types"
"github.com/tendermint/tendermint/proxy" "github.com/tendermint/tendermint/proxy"
rpccore "github.com/tendermint/tendermint/rpc/core" rpccore "github.com/tendermint/tendermint/rpc/core"
grpccore "github.com/tendermint/tendermint/rpc/grpc" grpccore "github.com/tendermint/tendermint/rpc/grpc"
@ -376,7 +375,7 @@ func (n *Node) OnStart() error {
// Generate node PrivKey // Generate node PrivKey
// TODO: pass in like priv_val // TODO: pass in like priv_val
nodeKey, err := p2ptypes.LoadOrGenNodeKey(n.config.NodeKeyFile()) nodeKey, err := p2p.LoadOrGenNodeKey(n.config.NodeKeyFile())
if err != nil { if err != nil {
return err return err
} }

View File

@ -1,7 +1,7 @@
package p2p package p2p
import ( import (
"github.com/tendermint/tendermint/p2p/tmconn" "github.com/tendermint/tendermint/p2p/conn"
cmn "github.com/tendermint/tmlibs/common" cmn "github.com/tendermint/tmlibs/common"
) )
@ -9,7 +9,7 @@ type Reactor interface {
cmn.Service // Start, Stop cmn.Service // Start, Stop
SetSwitch(*Switch) SetSwitch(*Switch)
GetChannels() []*tmconn.ChannelDescriptor GetChannels() []*conn.ChannelDescriptor
AddPeer(peer Peer) AddPeer(peer Peer)
RemovePeer(peer Peer, reason interface{}) RemovePeer(peer Peer, reason interface{})
Receive(chID byte, peer Peer, msgBytes []byte) // CONTRACT: msgBytes are not nil Receive(chID byte, peer Peer, msgBytes []byte) // CONTRACT: msgBytes are not nil
@ -32,7 +32,7 @@ func NewBaseReactor(name string, impl Reactor) *BaseReactor {
func (br *BaseReactor) SetSwitch(sw *Switch) { func (br *BaseReactor) SetSwitch(sw *Switch) {
br.Switch = sw br.Switch = sw
} }
func (_ *BaseReactor) GetChannels() []*tmconn.ChannelDescriptor { return nil } func (_ *BaseReactor) GetChannels() []*conn.ChannelDescriptor { return nil }
func (_ *BaseReactor) AddPeer(peer Peer) {} func (_ *BaseReactor) AddPeer(peer Peer) {}
func (_ *BaseReactor) RemovePeer(peer Peer, reason interface{}) {} func (_ *BaseReactor) RemovePeer(peer Peer, reason interface{}) {}
func (_ *BaseReactor) Receive(chID byte, peer Peer, msgBytes []byte) {} func (_ *BaseReactor) Receive(chID byte, peer Peer, msgBytes []byte) {}

View File

@ -1,6 +1,6 @@
// +build go1.10 // +build go1.10
package tmconn package conn
// Go1.10 has a proper net.Conn implementation that // Go1.10 has a proper net.Conn implementation that
// has the SetDeadline method implemented as per // has the SetDeadline method implemented as per

View File

@ -1,6 +1,6 @@
// +build !go1.10 // +build !go1.10
package tmconn package conn
import ( import (
"net" "net"

View File

@ -1,4 +1,4 @@
package tmconn package conn
import ( import (
"bufio" "bufio"

View File

@ -1,4 +1,4 @@
package tmconn package conn
import ( import (
"net" "net"

View File

@ -4,7 +4,7 @@
// is known ahead of time, and thus we are technically // is known ahead of time, and thus we are technically
// still vulnerable to MITM. (TODO!) // still vulnerable to MITM. (TODO!)
// See docs/sts-final.pdf for more info // See docs/sts-final.pdf for more info
package tmconn package conn
import ( import (
"bytes" "bytes"

View File

@ -1,4 +1,4 @@
package tmconn package conn
import ( import (
"io" "io"

View File

@ -1,4 +1,4 @@
package types package p2p
import ( import (
"errors" "errors"

View File

@ -1,4 +1,4 @@
package types package p2p
import ( import (
"bytes" "bytes"

View File

@ -1,4 +1,4 @@
package types package p2p
import ( import (
"bytes" "bytes"

View File

@ -6,7 +6,6 @@ import (
"strconv" "strconv"
"time" "time"
"github.com/tendermint/tendermint/p2p/types"
"github.com/tendermint/tendermint/p2p/upnp" "github.com/tendermint/tendermint/p2p/upnp"
cmn "github.com/tendermint/tmlibs/common" cmn "github.com/tendermint/tmlibs/common"
"github.com/tendermint/tmlibs/log" "github.com/tendermint/tmlibs/log"
@ -14,8 +13,8 @@ import (
type Listener interface { type Listener interface {
Connections() <-chan net.Conn Connections() <-chan net.Conn
InternalAddress() *types.NetAddress InternalAddress() *NetAddress
ExternalAddress() *types.NetAddress ExternalAddress() *NetAddress
String() string String() string
Stop() error Stop() error
} }
@ -25,8 +24,8 @@ type DefaultListener struct {
cmn.BaseService cmn.BaseService
listener net.Listener listener net.Listener
intAddr *types.NetAddress intAddr *NetAddress
extAddr *types.NetAddress extAddr *NetAddress
connections chan net.Conn connections chan net.Conn
} }
@ -72,14 +71,14 @@ func NewDefaultListener(protocol string, lAddr string, skipUPNP bool, logger log
logger.Info("Local listener", "ip", listenerIP, "port", listenerPort) logger.Info("Local listener", "ip", listenerIP, "port", listenerPort)
// Determine internal address... // Determine internal address...
var intAddr *types.NetAddress var intAddr *NetAddress
intAddr, err = types.NewNetAddressString(lAddr) intAddr, err = NewNetAddressString(lAddr)
if err != nil { if err != nil {
panic(err) panic(err)
} }
// Determine external address... // Determine external address...
var extAddr *types.NetAddress var extAddr *NetAddress
if !skipUPNP { if !skipUPNP {
// If the lAddrIP is INADDR_ANY, try UPnP // If the lAddrIP is INADDR_ANY, try UPnP
if lAddrIP == "" || lAddrIP == "0.0.0.0" { if lAddrIP == "" || lAddrIP == "0.0.0.0" {
@ -152,11 +151,11 @@ func (l *DefaultListener) Connections() <-chan net.Conn {
return l.connections return l.connections
} }
func (l *DefaultListener) InternalAddress() *types.NetAddress { func (l *DefaultListener) InternalAddress() *NetAddress {
return l.intAddr return l.intAddr
} }
func (l *DefaultListener) ExternalAddress() *types.NetAddress { func (l *DefaultListener) ExternalAddress() *NetAddress {
return l.extAddr return l.extAddr
} }
@ -173,7 +172,7 @@ func (l *DefaultListener) String() string {
/* external address helpers */ /* external address helpers */
// UPNP external address discovery & port mapping // UPNP external address discovery & port mapping
func getUPNPExternalAddress(externalPort, internalPort int, logger log.Logger) *types.NetAddress { func getUPNPExternalAddress(externalPort, internalPort int, logger log.Logger) *NetAddress {
logger.Info("Getting UPNP external address") logger.Info("Getting UPNP external address")
nat, err := upnp.Discover() nat, err := upnp.Discover()
if err != nil { if err != nil {
@ -199,11 +198,11 @@ func getUPNPExternalAddress(externalPort, internalPort int, logger log.Logger) *
} }
logger.Info("Got UPNP external address", "address", ext) logger.Info("Got UPNP external address", "address", ext)
return types.NewNetAddressIPPort(ext, uint16(externalPort)) return NewNetAddressIPPort(ext, uint16(externalPort))
} }
// TODO: use syscalls: see issue #712 // TODO: use syscalls: see issue #712
func getNaiveExternalAddress(port int, settleForLocal bool, logger log.Logger) *types.NetAddress { func getNaiveExternalAddress(port int, settleForLocal bool, logger log.Logger) *NetAddress {
addrs, err := net.InterfaceAddrs() addrs, err := net.InterfaceAddrs()
if err != nil { if err != nil {
panic(cmn.Fmt("Could not fetch interface addresses: %v", err)) panic(cmn.Fmt("Could not fetch interface addresses: %v", err))
@ -218,7 +217,7 @@ func getNaiveExternalAddress(port int, settleForLocal bool, logger log.Logger) *
if v4 == nil || (!settleForLocal && v4[0] == 127) { if v4 == nil || (!settleForLocal && v4[0] == 127) {
continue continue
} // loopback } // loopback
return types.NewNetAddressIPPort(ipnet.IP, uint16(port)) return NewNetAddressIPPort(ipnet.IP, uint16(port))
} }
// try again, but settle for local // try again, but settle for local

View File

@ -2,7 +2,7 @@
// Originally Copyright (c) 2013-2014 Conformal Systems LLC. // Originally Copyright (c) 2013-2014 Conformal Systems LLC.
// https://github.com/conformal/btcd/blob/master/LICENSE // https://github.com/conformal/btcd/blob/master/LICENSE
package types package p2p
import ( import (
"encoding/hex" "encoding/hex"

View File

@ -1,4 +1,4 @@
package types package p2p
import ( import (
"net" "net"

View File

@ -1,4 +1,4 @@
package types package p2p
import ( import (
"fmt" "fmt"

View File

@ -12,18 +12,17 @@ import (
cmn "github.com/tendermint/tmlibs/common" cmn "github.com/tendermint/tmlibs/common"
"github.com/tendermint/tmlibs/log" "github.com/tendermint/tmlibs/log"
"github.com/tendermint/tendermint/p2p/tmconn" tmconn "github.com/tendermint/tendermint/p2p/conn"
"github.com/tendermint/tendermint/p2p/types"
) )
// Peer is an interface representing a peer connected on a reactor. // Peer is an interface representing a peer connected on a reactor.
type Peer interface { type Peer interface {
cmn.Service cmn.Service
ID() types.ID // peer's cryptographic ID ID() ID // peer's cryptographic ID
IsOutbound() bool // did we dial the peer IsOutbound() bool // did we dial the peer
IsPersistent() bool // do we redial this peer when we disconnect IsPersistent() bool // do we redial this peer when we disconnect
NodeInfo() types.NodeInfo // peer's info NodeInfo() NodeInfo // peer's info
Status() tmconn.ConnectionStatus Status() tmconn.ConnectionStatus
Send(byte, interface{}) bool Send(byte, interface{}) bool
@ -49,7 +48,7 @@ type peer struct {
persistent bool persistent bool
config *PeerConfig config *PeerConfig
nodeInfo types.NodeInfo nodeInfo NodeInfo
Data *cmn.CMap // User data. Data *cmn.CMap // User data.
} }
@ -79,7 +78,7 @@ func DefaultPeerConfig() *PeerConfig {
} }
} }
func newOutboundPeer(addr *types.NetAddress, reactorsByCh map[byte]Reactor, chDescs []*tmconn.ChannelDescriptor, func newOutboundPeer(addr *NetAddress, reactorsByCh map[byte]Reactor, chDescs []*tmconn.ChannelDescriptor,
onPeerError func(Peer, interface{}), ourNodePrivKey crypto.PrivKey, config *PeerConfig, persistent bool) (*peer, error) { onPeerError func(Peer, interface{}), ourNodePrivKey crypto.PrivKey, config *PeerConfig, persistent bool) (*peer, error) {
conn, err := dial(addr, config) conn, err := dial(addr, config)
@ -174,8 +173,8 @@ func (p *peer) OnStop() {
// Implements Peer // Implements Peer
// ID returns the peer's ID - the hex encoded hash of its pubkey. // ID returns the peer's ID - the hex encoded hash of its pubkey.
func (p *peer) ID() types.ID { func (p *peer) ID() ID {
return types.PubKeyToID(p.PubKey()) return PubKeyToID(p.PubKey())
} }
// IsOutbound returns true if the connection is outbound, false otherwise. // IsOutbound returns true if the connection is outbound, false otherwise.
@ -189,7 +188,7 @@ func (p *peer) IsPersistent() bool {
} }
// NodeInfo returns a copy of the peer's NodeInfo. // NodeInfo returns a copy of the peer's NodeInfo.
func (p *peer) NodeInfo() types.NodeInfo { func (p *peer) NodeInfo() NodeInfo {
return p.nodeInfo return p.nodeInfo
} }
@ -239,13 +238,13 @@ func (p *peer) CloseConn() {
// HandshakeTimeout performs the Tendermint P2P handshake between a given node and the peer // 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. // by exchanging their NodeInfo. It sets the received nodeInfo on the peer.
// NOTE: blocking // NOTE: blocking
func (p *peer) HandshakeTimeout(ourNodeInfo types.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 // Set deadline for handshake so we don't block forever on conn.ReadFull
if err := p.conn.SetDeadline(time.Now().Add(timeout)); err != nil { if err := p.conn.SetDeadline(time.Now().Add(timeout)); err != nil {
return errors.Wrap(err, "Error setting deadline") return errors.Wrap(err, "Error setting deadline")
} }
var peerNodeInfo types.NodeInfo var peerNodeInfo NodeInfo
var err1 error var err1 error
var err2 error var err2 error
cmn.Parallel( cmn.Parallel(
@ -255,7 +254,7 @@ func (p *peer) HandshakeTimeout(ourNodeInfo types.NodeInfo, timeout time.Duratio
}, },
func() { func() {
var n int var n int
wire.ReadBinary(&peerNodeInfo, p.conn, types.MaxNodeInfoSize(), &n, &err2) wire.ReadBinary(&peerNodeInfo, p.conn, MaxNodeInfoSize(), &n, &err2)
p.Logger.Info("Peer handshake", "peerNodeInfo", peerNodeInfo) p.Logger.Info("Peer handshake", "peerNodeInfo", peerNodeInfo)
}) })
if err1 != nil { if err1 != nil {
@ -311,7 +310,7 @@ func (p *peer) String() string {
//------------------------------------------------------------------ //------------------------------------------------------------------
// helper funcs // helper funcs
func dial(addr *types.NetAddress, config *PeerConfig) (net.Conn, error) { func dial(addr *NetAddress, config *PeerConfig) (net.Conn, error) {
conn, err := addr.DialTimeout(config.DialTimeout * time.Second) conn, err := addr.DialTimeout(config.DialTimeout * time.Second)
if err != nil { if err != nil {
return nil, err return nil, err

View File

@ -2,14 +2,12 @@ package p2p
import ( import (
"sync" "sync"
"github.com/tendermint/tendermint/p2p/types"
) )
// IPeerSet has a (immutable) subset of the methods of PeerSet. // IPeerSet has a (immutable) subset of the methods of PeerSet.
type IPeerSet interface { type IPeerSet interface {
Has(key types.ID) bool Has(key ID) bool
Get(key types.ID) Peer Get(key ID) Peer
List() []Peer List() []Peer
Size() int Size() int
} }
@ -20,7 +18,7 @@ type IPeerSet interface {
// Iteration over the peers is super fast and thread-safe. // Iteration over the peers is super fast and thread-safe.
type PeerSet struct { type PeerSet struct {
mtx sync.Mutex mtx sync.Mutex
lookup map[types.ID]*peerSetItem lookup map[ID]*peerSetItem
list []Peer list []Peer
} }
@ -32,7 +30,7 @@ type peerSetItem struct {
// NewPeerSet creates a new peerSet with a list of initial capacity of 256 items. // NewPeerSet creates a new peerSet with a list of initial capacity of 256 items.
func NewPeerSet() *PeerSet { func NewPeerSet() *PeerSet {
return &PeerSet{ return &PeerSet{
lookup: make(map[types.ID]*peerSetItem), lookup: make(map[ID]*peerSetItem),
list: make([]Peer, 0, 256), list: make([]Peer, 0, 256),
} }
} }
@ -43,7 +41,7 @@ func (ps *PeerSet) Add(peer Peer) error {
ps.mtx.Lock() ps.mtx.Lock()
defer ps.mtx.Unlock() defer ps.mtx.Unlock()
if ps.lookup[peer.ID()] != nil { if ps.lookup[peer.ID()] != nil {
return types.ErrSwitchDuplicatePeer return ErrSwitchDuplicatePeer
} }
index := len(ps.list) index := len(ps.list)
@ -56,7 +54,7 @@ func (ps *PeerSet) Add(peer Peer) error {
// Has returns true iff the PeerSet contains // Has returns true iff the PeerSet contains
// the peer referred to by this peerKey. // the peer referred to by this peerKey.
func (ps *PeerSet) Has(peerKey types.ID) bool { func (ps *PeerSet) Has(peerKey ID) bool {
ps.mtx.Lock() ps.mtx.Lock()
_, ok := ps.lookup[peerKey] _, ok := ps.lookup[peerKey]
ps.mtx.Unlock() ps.mtx.Unlock()
@ -64,7 +62,7 @@ func (ps *PeerSet) Has(peerKey types.ID) bool {
} }
// Get looks up a peer by the provided peerKey. // Get looks up a peer by the provided peerKey.
func (ps *PeerSet) Get(peerKey types.ID) Peer { func (ps *PeerSet) Get(peerKey ID) Peer {
ps.mtx.Lock() ps.mtx.Lock()
defer ps.mtx.Unlock() defer ps.mtx.Unlock()
item, ok := ps.lookup[peerKey] item, ok := ps.lookup[peerKey]

View File

@ -8,14 +8,13 @@ import (
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
crypto "github.com/tendermint/go-crypto" crypto "github.com/tendermint/go-crypto"
"github.com/tendermint/tendermint/p2p/types"
cmn "github.com/tendermint/tmlibs/common" cmn "github.com/tendermint/tmlibs/common"
) )
// Returns an empty dummy peer // Returns an empty dummy peer
func randPeer() *peer { func randPeer() *peer {
return &peer{ return &peer{
nodeInfo: types.NodeInfo{ nodeInfo: NodeInfo{
ListenAddr: cmn.Fmt("%v.%v.%v.%v:46656", rand.Int()%256, rand.Int()%256, rand.Int()%256, rand.Int()%256), 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(), PubKey: crypto.GenPrivKeyEd25519().Wrap().PubKey(),
}, },
@ -120,7 +119,7 @@ func TestPeerSetAddDuplicate(t *testing.T) {
// Our next procedure is to ensure that only one addition // Our next procedure is to ensure that only one addition
// succeeded and that the rest are each ErrSwitchDuplicatePeer. // succeeded and that the rest are each ErrSwitchDuplicatePeer.
wantErrCount, gotErrCount := n-1, errsTally[types.ErrSwitchDuplicatePeer] wantErrCount, gotErrCount := n-1, errsTally[ErrSwitchDuplicatePeer]
assert.Equal(t, wantErrCount, gotErrCount, "invalid ErrSwitchDuplicatePeer count") assert.Equal(t, wantErrCount, gotErrCount, "invalid ErrSwitchDuplicatePeer count")
wantNilErrCount, gotNilErrCount := 1, errsTally[nil] wantNilErrCount, gotNilErrCount := 1, errsTally[nil]

View File

@ -10,8 +10,7 @@ import (
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
crypto "github.com/tendermint/go-crypto" crypto "github.com/tendermint/go-crypto"
"github.com/tendermint/tendermint/p2p/tmconn" tmconn "github.com/tendermint/tendermint/p2p/conn"
"github.com/tendermint/tendermint/p2p/types"
) )
func TestPeerBasic(t *testing.T) { func TestPeerBasic(t *testing.T) {
@ -82,7 +81,7 @@ func TestPeerSend(t *testing.T) {
assert.True(p.Send(0x01, "Asylum")) assert.True(p.Send(0x01, "Asylum"))
} }
func createOutboundPeerAndPerformHandshake(addr *types.NetAddress, config *PeerConfig) (*peer, error) { func createOutboundPeerAndPerformHandshake(addr *NetAddress, config *PeerConfig) (*peer, error) {
chDescs := []*tmconn.ChannelDescriptor{ chDescs := []*tmconn.ChannelDescriptor{
{ID: 0x01, Priority: 1}, {ID: 0x01, Priority: 1},
} }
@ -92,7 +91,7 @@ func createOutboundPeerAndPerformHandshake(addr *types.NetAddress, config *PeerC
if err != nil { if err != nil {
return nil, err return nil, err
} }
err = p.HandshakeTimeout(types.NodeInfo{ err = p.HandshakeTimeout(NodeInfo{
PubKey: pk.PubKey(), PubKey: pk.PubKey(),
Moniker: "host_peer", Moniker: "host_peer",
Network: "testing", Network: "testing",
@ -107,11 +106,11 @@ func createOutboundPeerAndPerformHandshake(addr *types.NetAddress, config *PeerC
type remotePeer struct { type remotePeer struct {
PrivKey crypto.PrivKey PrivKey crypto.PrivKey
Config *PeerConfig Config *PeerConfig
addr *types.NetAddress addr *NetAddress
quit chan struct{} quit chan struct{}
} }
func (p *remotePeer) Addr() *types.NetAddress { func (p *remotePeer) Addr() *NetAddress {
return p.addr return p.addr
} }
@ -124,7 +123,7 @@ func (p *remotePeer) Start() {
if e != nil { if e != nil {
golog.Fatalf("net.Listen tcp :0: %+v", e) golog.Fatalf("net.Listen tcp :0: %+v", e)
} }
p.addr = types.NewNetAddress("", l.Addr()) p.addr = NewNetAddress("", l.Addr())
p.quit = make(chan struct{}) p.quit = make(chan struct{})
go p.accept(l) go p.accept(l)
} }
@ -143,7 +142,7 @@ func (p *remotePeer) accept(l net.Listener) {
if err != nil { if err != nil {
golog.Fatalf("Failed to create a peer: %+v", err) golog.Fatalf("Failed to create a peer: %+v", err)
} }
err = peer.HandshakeTimeout(types.NodeInfo{ err = peer.HandshakeTimeout(NodeInfo{
PubKey: p.PrivKey.PubKey(), PubKey: p.PrivKey.PubKey(),
Moniker: "remote_peer", Moniker: "remote_peer",
Network: "testing", Network: "testing",

View File

@ -15,9 +15,8 @@ import (
"time" "time"
crypto "github.com/tendermint/go-crypto" crypto "github.com/tendermint/go-crypto"
"github.com/tendermint/tendermint/p2p"
cmn "github.com/tendermint/tmlibs/common" cmn "github.com/tendermint/tmlibs/common"
"github.com/tendermint/tendermint/p2p/types"
) )
const ( const (
@ -32,25 +31,25 @@ type AddrBook interface {
cmn.Service cmn.Service
// Add our own addresses so we don't later add ourselves // Add our own addresses so we don't later add ourselves
AddOurAddress(*types.NetAddress) AddOurAddress(*p2p.NetAddress)
// Add and remove an address // Add and remove an address
AddAddress(addr *types.NetAddress, src *types.NetAddress) error AddAddress(addr *p2p.NetAddress, src *p2p.NetAddress) error
RemoveAddress(addr *types.NetAddress) RemoveAddress(addr *p2p.NetAddress)
// Do we need more peers? // Do we need more peers?
NeedMoreAddrs() bool NeedMoreAddrs() bool
// Pick an address to dial // Pick an address to dial
PickAddress(newBias int) *types.NetAddress PickAddress(newBias int) *p2p.NetAddress
// Mark address // Mark address
MarkGood(*types.NetAddress) MarkGood(*p2p.NetAddress)
MarkAttempt(*types.NetAddress) MarkAttempt(*p2p.NetAddress)
MarkBad(*types.NetAddress) MarkBad(*p2p.NetAddress)
// Send a selection of addresses to peers // Send a selection of addresses to peers
GetSelection() []*types.NetAddress GetSelection() []*p2p.NetAddress
// TODO: remove // TODO: remove
ListOfKnownAddresses() []*knownAddress ListOfKnownAddresses() []*knownAddress
@ -71,8 +70,8 @@ type addrBook struct {
// accessed concurrently // accessed concurrently
mtx sync.Mutex mtx sync.Mutex
rand *rand.Rand rand *rand.Rand
ourAddrs map[string]*types.NetAddress ourAddrs map[string]*p2p.NetAddress
addrLookup map[types.ID]*knownAddress // new & old addrLookup map[p2p.ID]*knownAddress // new & old
bucketsOld []map[string]*knownAddress bucketsOld []map[string]*knownAddress
bucketsNew []map[string]*knownAddress bucketsNew []map[string]*knownAddress
nOld int nOld int
@ -86,8 +85,8 @@ type addrBook struct {
func NewAddrBook(filePath string, routabilityStrict bool) *addrBook { func NewAddrBook(filePath string, routabilityStrict bool) *addrBook {
am := &addrBook{ am := &addrBook{
rand: rand.New(rand.NewSource(time.Now().UnixNano())), // TODO: seed from outside rand: rand.New(rand.NewSource(time.Now().UnixNano())), // TODO: seed from outside
ourAddrs: make(map[string]*types.NetAddress), ourAddrs: make(map[string]*p2p.NetAddress),
addrLookup: make(map[types.ID]*knownAddress), addrLookup: make(map[p2p.ID]*knownAddress),
filePath: filePath, filePath: filePath,
routabilityStrict: routabilityStrict, routabilityStrict: routabilityStrict,
} }
@ -139,7 +138,7 @@ func (a *addrBook) Wait() {
//------------------------------------------------------- //-------------------------------------------------------
// AddOurAddress one of our addresses. // AddOurAddress one of our addresses.
func (a *addrBook) AddOurAddress(addr *types.NetAddress) { func (a *addrBook) AddOurAddress(addr *p2p.NetAddress) {
a.mtx.Lock() a.mtx.Lock()
defer a.mtx.Unlock() defer a.mtx.Unlock()
a.Logger.Info("Add our address to book", "addr", addr) a.Logger.Info("Add our address to book", "addr", addr)
@ -148,14 +147,14 @@ func (a *addrBook) AddOurAddress(addr *types.NetAddress) {
// AddAddress implements AddrBook - adds the given address as received from the given source. // AddAddress implements AddrBook - adds the given address as received from the given source.
// NOTE: addr must not be nil // NOTE: addr must not be nil
func (a *addrBook) AddAddress(addr *types.NetAddress, src *types.NetAddress) error { func (a *addrBook) AddAddress(addr *p2p.NetAddress, src *p2p.NetAddress) error {
a.mtx.Lock() a.mtx.Lock()
defer a.mtx.Unlock() defer a.mtx.Unlock()
return a.addAddress(addr, src) return a.addAddress(addr, src)
} }
// RemoveAddress implements AddrBook - removes the address from the book. // RemoveAddress implements AddrBook - removes the address from the book.
func (a *addrBook) RemoveAddress(addr *types.NetAddress) { func (a *addrBook) RemoveAddress(addr *p2p.NetAddress) {
a.mtx.Lock() a.mtx.Lock()
defer a.mtx.Unlock() defer a.mtx.Unlock()
ka := a.addrLookup[addr.ID] ka := a.addrLookup[addr.ID]
@ -177,7 +176,7 @@ func (a *addrBook) NeedMoreAddrs() bool {
// and determines how biased we are to pick an address from a new bucket. // and determines how biased we are to pick an address from a new bucket.
// PickAddress returns nil if the AddrBook is empty or if we try to pick // PickAddress returns nil if the AddrBook is empty or if we try to pick
// from an empty bucket. // from an empty bucket.
func (a *addrBook) PickAddress(newBias int) *types.NetAddress { func (a *addrBook) PickAddress(newBias int) *p2p.NetAddress {
a.mtx.Lock() a.mtx.Lock()
defer a.mtx.Unlock() defer a.mtx.Unlock()
@ -223,7 +222,7 @@ func (a *addrBook) PickAddress(newBias int) *types.NetAddress {
// MarkGood implements AddrBook - it marks the peer as good and // MarkGood implements AddrBook - it marks the peer as good and
// moves it into an "old" bucket. // moves it into an "old" bucket.
func (a *addrBook) MarkGood(addr *types.NetAddress) { func (a *addrBook) MarkGood(addr *p2p.NetAddress) {
a.mtx.Lock() a.mtx.Lock()
defer a.mtx.Unlock() defer a.mtx.Unlock()
ka := a.addrLookup[addr.ID] ka := a.addrLookup[addr.ID]
@ -237,7 +236,7 @@ func (a *addrBook) MarkGood(addr *types.NetAddress) {
} }
// MarkAttempt implements AddrBook - it marks that an attempt was made to connect to the address. // MarkAttempt implements AddrBook - it marks that an attempt was made to connect to the address.
func (a *addrBook) MarkAttempt(addr *types.NetAddress) { func (a *addrBook) MarkAttempt(addr *p2p.NetAddress) {
a.mtx.Lock() a.mtx.Lock()
defer a.mtx.Unlock() defer a.mtx.Unlock()
ka := a.addrLookup[addr.ID] ka := a.addrLookup[addr.ID]
@ -249,13 +248,13 @@ func (a *addrBook) MarkAttempt(addr *types.NetAddress) {
// MarkBad implements AddrBook. Currently it just ejects the address. // MarkBad implements AddrBook. Currently it just ejects the address.
// TODO: black list for some amount of time // TODO: black list for some amount of time
func (a *addrBook) MarkBad(addr *types.NetAddress) { func (a *addrBook) MarkBad(addr *p2p.NetAddress) {
a.RemoveAddress(addr) a.RemoveAddress(addr)
} }
// GetSelection implements AddrBook. // GetSelection implements AddrBook.
// It randomly selects some addresses (old & new). Suitable for peer-exchange protocols. // It randomly selects some addresses (old & new). Suitable for peer-exchange protocols.
func (a *addrBook) GetSelection() []*types.NetAddress { func (a *addrBook) GetSelection() []*p2p.NetAddress {
a.mtx.Lock() a.mtx.Lock()
defer a.mtx.Unlock() defer a.mtx.Unlock()
@ -263,7 +262,7 @@ func (a *addrBook) GetSelection() []*types.NetAddress {
return nil return nil
} }
allAddr := make([]*types.NetAddress, a.size()) allAddr := make([]*p2p.NetAddress, a.size())
i := 0 i := 0
for _, ka := range a.addrLookup { for _, ka := range a.addrLookup {
allAddr[i] = ka.Addr allAddr[i] = ka.Addr
@ -466,7 +465,7 @@ func (a *addrBook) pickOldest(bucketType byte, bucketIdx int) *knownAddress {
// adds the address to a "new" bucket. if its already in one, // adds the address to a "new" bucket. if its already in one,
// it only adds it probabilistically // it only adds it probabilistically
func (a *addrBook) addAddress(addr, src *types.NetAddress) error { func (a *addrBook) addAddress(addr, src *p2p.NetAddress) error {
if a.routabilityStrict && !addr.Routable() { if a.routabilityStrict && !addr.Routable() {
return fmt.Errorf("Cannot add non-routable address %v", addr) return fmt.Errorf("Cannot add non-routable address %v", addr)
} }
@ -573,7 +572,7 @@ func (a *addrBook) moveToOld(ka *knownAddress) {
// doublesha256( key + sourcegroup + // doublesha256( key + sourcegroup +
// int64(doublesha256(key + group + sourcegroup))%bucket_per_group ) % num_new_buckets // int64(doublesha256(key + group + sourcegroup))%bucket_per_group ) % num_new_buckets
func (a *addrBook) calcNewBucket(addr, src *types.NetAddress) int { func (a *addrBook) calcNewBucket(addr, src *p2p.NetAddress) int {
data1 := []byte{} data1 := []byte{}
data1 = append(data1, []byte(a.key)...) data1 = append(data1, []byte(a.key)...)
data1 = append(data1, []byte(a.groupKey(addr))...) data1 = append(data1, []byte(a.groupKey(addr))...)
@ -594,7 +593,7 @@ func (a *addrBook) calcNewBucket(addr, src *types.NetAddress) int {
// doublesha256( key + group + // doublesha256( key + group +
// int64(doublesha256(key + addr))%buckets_per_group ) % num_old_buckets // int64(doublesha256(key + addr))%buckets_per_group ) % num_old_buckets
func (a *addrBook) calcOldBucket(addr *types.NetAddress) int { func (a *addrBook) calcOldBucket(addr *p2p.NetAddress) int {
data1 := []byte{} data1 := []byte{}
data1 = append(data1, []byte(a.key)...) data1 = append(data1, []byte(a.key)...)
data1 = append(data1, []byte(addr.String())...) data1 = append(data1, []byte(addr.String())...)
@ -616,7 +615,7 @@ func (a *addrBook) calcOldBucket(addr *types.NetAddress) int {
// This is the /16 for IPv4, the /32 (/36 for he.net) for IPv6, the string // This is the /16 for IPv4, the /32 (/36 for he.net) for IPv6, the string
// "local" for a local address and the string "unroutable" for an unroutable // "local" for a local address and the string "unroutable" for an unroutable
// address. // address.
func (a *addrBook) groupKey(na *types.NetAddress) string { func (a *addrBook) groupKey(na *p2p.NetAddress) string {
if a.routabilityStrict && na.Local() { if a.routabilityStrict && na.Local() {
return "local" return "local"
} }

View File

@ -8,7 +8,7 @@ import (
"testing" "testing"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
"github.com/tendermint/tendermint/p2p/types" "github.com/tendermint/tendermint/p2p"
cmn "github.com/tendermint/tmlibs/common" cmn "github.com/tendermint/tmlibs/common"
"github.com/tendermint/tmlibs/log" "github.com/tendermint/tmlibs/log"
) )
@ -169,8 +169,8 @@ func TestAddrBookHandlesDuplicates(t *testing.T) {
} }
type netAddressPair struct { type netAddressPair struct {
addr *types.NetAddress addr *p2p.NetAddress
src *types.NetAddress src *p2p.NetAddress
} }
func randNetAddressPairs(t *testing.T, n int) []netAddressPair { func randNetAddressPairs(t *testing.T, n int) []netAddressPair {
@ -181,7 +181,7 @@ func randNetAddressPairs(t *testing.T, n int) []netAddressPair {
return randAddrs return randAddrs
} }
func randIPv4Address(t *testing.T) *types.NetAddress { func randIPv4Address(t *testing.T) *p2p.NetAddress {
for { for {
ip := fmt.Sprintf("%v.%v.%v.%v", ip := fmt.Sprintf("%v.%v.%v.%v",
rand.Intn(254)+1, rand.Intn(254)+1,
@ -190,9 +190,9 @@ func randIPv4Address(t *testing.T) *types.NetAddress {
rand.Intn(255), rand.Intn(255),
) )
port := rand.Intn(65535-1) + 1 port := rand.Intn(65535-1) + 1
id := types.ID(hex.EncodeToString(cmn.RandBytes(types.IDByteLength))) id := p2p.ID(hex.EncodeToString(cmn.RandBytes(p2p.IDByteLength)))
idAddr := types.IDAddressString(id, fmt.Sprintf("%v:%v", ip, port)) idAddr := p2p.IDAddressString(id, fmt.Sprintf("%v:%v", ip, port))
addr, err := types.NewNetAddressString(idAddr) addr, err := p2p.NewNetAddressString(idAddr)
assert.Nil(t, err, "error generating rand network address") assert.Nil(t, err, "error generating rand network address")
if addr.Routable() { if addr.Routable() {
return addr return addr

View File

@ -3,14 +3,14 @@ package pex
import ( import (
"time" "time"
"github.com/tendermint/tendermint/p2p/types" "github.com/tendermint/tendermint/p2p"
) )
// knownAddress tracks information about a known network address // knownAddress tracks information about a known network address
// that is used to determine how viable an address is. // that is used to determine how viable an address is.
type knownAddress struct { type knownAddress struct {
Addr *types.NetAddress Addr *p2p.NetAddress
Src *types.NetAddress Src *p2p.NetAddress
Attempts int32 Attempts int32
LastAttempt time.Time LastAttempt time.Time
LastSuccess time.Time LastSuccess time.Time
@ -18,7 +18,7 @@ type knownAddress struct {
Buckets []int Buckets []int
} }
func newKnownAddress(addr *types.NetAddress, src *types.NetAddress) *knownAddress { func newKnownAddress(addr *p2p.NetAddress, src *p2p.NetAddress) *knownAddress {
return &knownAddress{ return &knownAddress{
Addr: addr, Addr: addr,
Src: src, Src: src,
@ -29,7 +29,7 @@ func newKnownAddress(addr *types.NetAddress, src *types.NetAddress) *knownAddres
} }
} }
func (ka *knownAddress) ID() types.ID { func (ka *knownAddress) ID() p2p.ID {
return ka.Addr.ID return ka.Addr.ID
} }

View File

@ -13,8 +13,7 @@ import (
cmn "github.com/tendermint/tmlibs/common" cmn "github.com/tendermint/tmlibs/common"
"github.com/tendermint/tendermint/p2p" "github.com/tendermint/tendermint/p2p"
"github.com/tendermint/tendermint/p2p/tmconn" "github.com/tendermint/tendermint/p2p/conn"
"github.com/tendermint/tendermint/p2p/types"
) )
type Peer = p2p.Peer type Peer = p2p.Peer
@ -117,8 +116,8 @@ func (r *PEXReactor) OnStop() {
} }
// GetChannels implements Reactor // GetChannels implements Reactor
func (r *PEXReactor) GetChannels() []*tmconn.ChannelDescriptor { func (r *PEXReactor) GetChannels() []*conn.ChannelDescriptor {
return []*tmconn.ChannelDescriptor{ return []*conn.ChannelDescriptor{
{ {
ID: PexChannel, ID: PexChannel,
Priority: 1, Priority: 1,
@ -231,7 +230,7 @@ func (r *PEXReactor) RequestAddrs(p Peer) {
// ReceiveAddrs adds the given addrs to the addrbook if theres an open // ReceiveAddrs adds the given addrs to the addrbook if theres an open
// request for this peer and deletes the open request. // request for this peer and deletes the open request.
// If there's no open request for the src peer, it returns an error. // If there's no open request for the src peer, it returns an error.
func (r *PEXReactor) ReceiveAddrs(addrs []*types.NetAddress, src Peer) error { func (r *PEXReactor) ReceiveAddrs(addrs []*p2p.NetAddress, src Peer) error {
id := string(src.ID()) id := string(src.ID())
if !r.requestsSent.Has(id) { if !r.requestsSent.Has(id) {
@ -250,7 +249,7 @@ func (r *PEXReactor) ReceiveAddrs(addrs []*types.NetAddress, src Peer) error {
} }
// SendAddrs sends addrs to the peer. // SendAddrs sends addrs to the peer.
func (r *PEXReactor) SendAddrs(p Peer, netAddrs []*types.NetAddress) { func (r *PEXReactor) SendAddrs(p Peer, netAddrs []*p2p.NetAddress) {
p.Send(PexChannel, struct{ PexMessage }{&pexAddrsMessage{Addrs: netAddrs}}) p.Send(PexChannel, struct{ PexMessage }{&pexAddrsMessage{Addrs: netAddrs}})
} }
@ -300,7 +299,7 @@ func (r *PEXReactor) ensurePeers() {
// NOTE: range here is [10, 90]. Too high ? // NOTE: range here is [10, 90]. Too high ?
newBias := cmn.MinInt(numOutPeers, 8)*10 + 10 newBias := cmn.MinInt(numOutPeers, 8)*10 + 10
toDial := make(map[types.ID]*types.NetAddress) toDial := make(map[p2p.ID]*p2p.NetAddress)
// Try maxAttempts times to pick numToDial addresses to dial // Try maxAttempts times to pick numToDial addresses to dial
maxAttempts := numToDial * 3 maxAttempts := numToDial * 3
for i := 0; i < maxAttempts && len(toDial) < numToDial; i++ { for i := 0; i < maxAttempts && len(toDial) < numToDial; i++ {
@ -323,11 +322,11 @@ func (r *PEXReactor) ensurePeers() {
// Dial picked addresses // Dial picked addresses
for _, item := range toDial { for _, item := range toDial {
go func(picked *types.NetAddress) { go func(picked *p2p.NetAddress) {
_, err := r.Switch.DialPeerWithAddress(picked, false) _, err := r.Switch.DialPeerWithAddress(picked, false)
if err != nil { if err != nil {
// TODO: detect more "bad peer" scenarios // TODO: detect more "bad peer" scenarios
if _, ok := err.(types.ErrSwitchAuthenticationFailure); ok { if _, ok := err.(p2p.ErrSwitchAuthenticationFailure); ok {
r.book.MarkBad(picked) r.book.MarkBad(picked)
} else { } else {
r.book.MarkAttempt(picked) r.book.MarkAttempt(picked)
@ -360,7 +359,7 @@ func (r *PEXReactor) checkSeeds() error {
if lSeeds == 0 { if lSeeds == 0 {
return nil return nil
} }
_, errs := types.NewNetAddressStrings(r.config.Seeds) _, errs := p2p.NewNetAddressStrings(r.config.Seeds)
for _, err := range errs { for _, err := range errs {
if err != nil { if err != nil {
return err return err
@ -375,7 +374,7 @@ func (r *PEXReactor) dialSeeds() {
if lSeeds == 0 { if lSeeds == 0 {
return return
} }
seedAddrs, _ := types.NewNetAddressStrings(r.config.Seeds) seedAddrs, _ := p2p.NewNetAddressStrings(r.config.Seeds)
perm := rand.Perm(lSeeds) perm := rand.Perm(lSeeds)
// perm := r.Switch.rng.Perm(lSeeds) // perm := r.Switch.rng.Perm(lSeeds)
@ -420,7 +419,7 @@ func (r *PEXReactor) crawlPeersRoutine() {
// network crawling performed during seed/crawler mode. // network crawling performed during seed/crawler mode.
type crawlPeerInfo struct { type crawlPeerInfo struct {
// The listening address of a potential peer we learned about // The listening address of a potential peer we learned about
Addr *types.NetAddress Addr *p2p.NetAddress
// The last time we attempt to reach this address // The last time we attempt to reach this address
LastAttempt time.Time LastAttempt time.Time
@ -544,7 +543,7 @@ func (m *pexRequestMessage) String() string {
A message with announced peer addresses. A message with announced peer addresses.
*/ */
type pexAddrsMessage struct { type pexAddrsMessage struct {
Addrs []*types.NetAddress Addrs []*p2p.NetAddress
} }
func (m *pexAddrsMessage) String() string { func (m *pexAddrsMessage) String() string {

View File

@ -17,8 +17,7 @@ import (
cfg "github.com/tendermint/tendermint/config" cfg "github.com/tendermint/tendermint/config"
"github.com/tendermint/tendermint/p2p" "github.com/tendermint/tendermint/p2p"
"github.com/tendermint/tendermint/p2p/tmconn" "github.com/tendermint/tendermint/p2p/conn"
"github.com/tendermint/tendermint/p2p/types"
) )
var ( var (
@ -101,7 +100,7 @@ func TestPEXReactorRunning(t *testing.T) {
// fill the address book and add listeners // fill the address book and add listeners
for _, s := range switches { for _, s := range switches {
addr, _ := types.NewNetAddressString(s.NodeInfo().ListenAddr) addr, _ := p2p.NewNetAddressString(s.NodeInfo().ListenAddr)
book.AddAddress(addr, addr) book.AddAddress(addr, addr)
s.AddListener(p2p.NewDefaultListener("tcp", s.NodeInfo().ListenAddr, true, log.TestingLogger())) s.AddListener(p2p.NewDefaultListener("tcp", s.NodeInfo().ListenAddr, true, log.TestingLogger()))
} }
@ -171,7 +170,7 @@ func TestPEXReactorReceive(t *testing.T) {
r.RequestAddrs(peer) r.RequestAddrs(peer)
size := book.Size() size := book.Size()
addrs := []*types.NetAddress{peer.NodeInfo().NetAddress()} addrs := []*p2p.NetAddress{peer.NodeInfo().NetAddress()}
msg := wire.BinaryBytes(struct{ PexMessage }{&pexAddrsMessage{Addrs: addrs}}) msg := wire.BinaryBytes(struct{ PexMessage }{&pexAddrsMessage{Addrs: addrs}})
r.Receive(PexChannel, peer, msg) r.Receive(PexChannel, peer, msg)
assert.Equal(size+1, book.Size()) assert.Equal(size+1, book.Size())
@ -246,7 +245,7 @@ func TestPEXReactorAddrsMessageAbuse(t *testing.T) {
assert.True(r.requestsSent.Has(id)) assert.True(r.requestsSent.Has(id))
assert.True(sw.Peers().Has(peer.ID())) assert.True(sw.Peers().Has(peer.ID()))
addrs := []*types.NetAddress{peer.NodeInfo().NetAddress()} addrs := []*p2p.NetAddress{peer.NodeInfo().NetAddress()}
msg := wire.BinaryBytes(struct{ PexMessage }{&pexAddrsMessage{Addrs: addrs}}) msg := wire.BinaryBytes(struct{ PexMessage }{&pexAddrsMessage{Addrs: addrs}})
// receive some addrs. should clear the request // receive some addrs. should clear the request
@ -340,7 +339,7 @@ func TestPEXReactorCrawlStatus(t *testing.T) {
type mockPeer struct { type mockPeer struct {
*cmn.BaseService *cmn.BaseService
pubKey crypto.PubKey pubKey crypto.PubKey
addr *types.NetAddress addr *p2p.NetAddress
outbound, persistent bool outbound, persistent bool
} }
@ -355,16 +354,16 @@ func newMockPeer() mockPeer {
return mp return mp
} }
func (mp mockPeer) ID() types.ID { return types.PubKeyToID(mp.pubKey) } func (mp mockPeer) ID() p2p.ID { return p2p.PubKeyToID(mp.pubKey) }
func (mp mockPeer) IsOutbound() bool { return mp.outbound } func (mp mockPeer) IsOutbound() bool { return mp.outbound }
func (mp mockPeer) IsPersistent() bool { return mp.persistent } func (mp mockPeer) IsPersistent() bool { return mp.persistent }
func (mp mockPeer) NodeInfo() types.NodeInfo { func (mp mockPeer) NodeInfo() p2p.NodeInfo {
return types.NodeInfo{ return p2p.NodeInfo{
PubKey: mp.pubKey, PubKey: mp.pubKey,
ListenAddr: mp.addr.DialString(), ListenAddr: mp.addr.DialString(),
} }
} }
func (mp mockPeer) Status() tmconn.ConnectionStatus { return tmconn.ConnectionStatus{} } func (mp mockPeer) Status() conn.ConnectionStatus { return conn.ConnectionStatus{} }
func (mp mockPeer) Send(byte, interface{}) bool { return false } func (mp mockPeer) Send(byte, interface{}) bool { return false }
func (mp mockPeer) TrySend(byte, interface{}) bool { return false } func (mp mockPeer) TrySend(byte, interface{}) bool { return false }
func (mp mockPeer) Set(string, interface{}) {} func (mp mockPeer) Set(string, interface{}) {}

View File

@ -11,8 +11,7 @@ import (
crypto "github.com/tendermint/go-crypto" crypto "github.com/tendermint/go-crypto"
cfg "github.com/tendermint/tendermint/config" cfg "github.com/tendermint/tendermint/config"
"github.com/tendermint/tendermint/p2p/tmconn" "github.com/tendermint/tendermint/p2p/conn"
"github.com/tendermint/tendermint/p2p/types"
cmn "github.com/tendermint/tmlibs/common" cmn "github.com/tendermint/tmlibs/common"
) )
@ -35,7 +34,7 @@ const (
//----------------------------------------------------------------------------- //-----------------------------------------------------------------------------
type AddrBook interface { type AddrBook interface {
AddAddress(addr *types.NetAddress, src *types.NetAddress) error AddAddress(addr *NetAddress, src *NetAddress) error
} }
//----------------------------------------------------------------------------- //-----------------------------------------------------------------------------
@ -51,12 +50,12 @@ type Switch struct {
peerConfig *PeerConfig peerConfig *PeerConfig
listeners []Listener listeners []Listener
reactors map[string]Reactor reactors map[string]Reactor
chDescs []*tmconn.ChannelDescriptor chDescs []*conn.ChannelDescriptor
reactorsByCh map[byte]Reactor reactorsByCh map[byte]Reactor
peers *PeerSet peers *PeerSet
dialing *cmn.CMap dialing *cmn.CMap
nodeInfo types.NodeInfo // our node info nodeInfo NodeInfo // our node info
nodeKey *types.NodeKey // our node privkey nodeKey *NodeKey // our node privkey
filterConnByAddr func(net.Addr) error filterConnByAddr func(net.Addr) error
filterConnByPubKey func(crypto.PubKey) error filterConnByPubKey func(crypto.PubKey) error
@ -69,7 +68,7 @@ func NewSwitch(config *cfg.P2PConfig) *Switch {
config: config, config: config,
peerConfig: DefaultPeerConfig(), peerConfig: DefaultPeerConfig(),
reactors: make(map[string]Reactor), reactors: make(map[string]Reactor),
chDescs: make([]*tmconn.ChannelDescriptor, 0), chDescs: make([]*conn.ChannelDescriptor, 0),
reactorsByCh: make(map[byte]Reactor), reactorsByCh: make(map[byte]Reactor),
peers: NewPeerSet(), peers: NewPeerSet(),
dialing: cmn.NewCMap(), dialing: cmn.NewCMap(),
@ -143,19 +142,19 @@ func (sw *Switch) IsListening() bool {
// SetNodeInfo sets the switch's NodeInfo for checking compatibility and handshaking with other nodes. // SetNodeInfo sets the switch's NodeInfo for checking compatibility and handshaking with other nodes.
// NOTE: Not goroutine safe. // NOTE: Not goroutine safe.
func (sw *Switch) SetNodeInfo(nodeInfo types.NodeInfo) { func (sw *Switch) SetNodeInfo(nodeInfo NodeInfo) {
sw.nodeInfo = nodeInfo sw.nodeInfo = nodeInfo
} }
// NodeInfo returns the switch's NodeInfo. // NodeInfo returns the switch's NodeInfo.
// NOTE: Not goroutine safe. // NOTE: Not goroutine safe.
func (sw *Switch) NodeInfo() types.NodeInfo { func (sw *Switch) NodeInfo() NodeInfo {
return sw.nodeInfo return sw.nodeInfo
} }
// SetNodeKey sets the switch's private key for authenticated encryption. // SetNodeKey sets the switch's private key for authenticated encryption.
// NOTE: Not goroutine safe. // NOTE: Not goroutine safe.
func (sw *Switch) SetNodeKey(nodeKey *types.NodeKey) { func (sw *Switch) SetNodeKey(nodeKey *NodeKey) {
sw.nodeKey = nodeKey sw.nodeKey = nodeKey
} }
@ -314,13 +313,13 @@ func (sw *Switch) reconnectToPeer(peer Peer) {
// Dialing // Dialing
// IsDialing returns true if the switch is currently dialing the given ID. // IsDialing returns true if the switch is currently dialing the given ID.
func (sw *Switch) IsDialing(id types.ID) bool { func (sw *Switch) IsDialing(id ID) bool {
return sw.dialing.Has(string(id)) return sw.dialing.Has(string(id))
} }
// DialPeersAsync dials a list of peers asynchronously in random order (optionally, making them persistent). // 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 { func (sw *Switch) DialPeersAsync(addrBook AddrBook, peers []string, persistent bool) error {
netAddrs, errs := types.NewNetAddressStrings(peers) netAddrs, errs := NewNetAddressStrings(peers)
for _, err := range errs { for _, err := range errs {
sw.Logger.Error("Error in peer's address", "err", err) sw.Logger.Error("Error in peer's address", "err", err)
} }
@ -357,7 +356,7 @@ func (sw *Switch) DialPeersAsync(addrBook AddrBook, peers []string, persistent b
// DialPeerWithAddress dials the given peer and runs sw.addPeer if it connects and authenticates successfully. // 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. // If `persistent == true`, the switch will always try to reconnect to this peer if the connection ever fails.
func (sw *Switch) DialPeerWithAddress(addr *types.NetAddress, persistent bool) (Peer, error) { func (sw *Switch) DialPeerWithAddress(addr *NetAddress, persistent bool) (Peer, error) {
sw.dialing.Set(string(addr.ID), addr) sw.dialing.Set(string(addr.ID), addr)
defer sw.dialing.Delete(string(addr.ID)) defer sw.dialing.Delete(string(addr.ID))
return sw.addOutboundPeerWithConfig(addr, sw.peerConfig, persistent) return sw.addOutboundPeerWithConfig(addr, sw.peerConfig, persistent)
@ -443,7 +442,7 @@ func (sw *Switch) addInboundPeerWithConfig(conn net.Conn, config *PeerConfig) er
// dial the peer; make secret connection; authenticate against the dialed ID; // dial the peer; make secret connection; authenticate against the dialed ID;
// add the peer. // add the peer.
func (sw *Switch) addOutboundPeerWithConfig(addr *types.NetAddress, config *PeerConfig, persistent bool) (Peer, error) { func (sw *Switch) addOutboundPeerWithConfig(addr *NetAddress, config *PeerConfig, persistent bool) (Peer, error) {
sw.Logger.Info("Dialing peer", "address", addr) sw.Logger.Info("Dialing peer", "address", addr)
peer, err := newOutboundPeer(addr, sw.reactorsByCh, sw.chDescs, sw.StopPeerForError, sw.nodeKey.PrivKey, config, persistent) peer, err := newOutboundPeer(addr, sw.reactorsByCh, sw.chDescs, sw.StopPeerForError, sw.nodeKey.PrivKey, config, persistent)
if err != nil { if err != nil {
@ -457,7 +456,7 @@ func (sw *Switch) addOutboundPeerWithConfig(addr *types.NetAddress, config *Peer
peer.Logger.Info("Dialed peer with unknown ID - unable to authenticate", "addr", addr) peer.Logger.Info("Dialed peer with unknown ID - unable to authenticate", "addr", addr)
} else if addr.ID != peer.ID() { } else if addr.ID != peer.ID() {
peer.CloseConn() peer.CloseConn()
return nil, types.ErrSwitchAuthenticationFailure{addr, peer.ID()} return nil, ErrSwitchAuthenticationFailure{addr, peer.ID()}
} }
err = sw.addPeer(peer) err = sw.addPeer(peer)
@ -478,12 +477,12 @@ func (sw *Switch) addOutboundPeerWithConfig(addr *types.NetAddress, config *Peer
func (sw *Switch) addPeer(peer *peer) error { func (sw *Switch) addPeer(peer *peer) error {
// Avoid self // Avoid self
if sw.nodeKey.ID() == peer.ID() { if sw.nodeKey.ID() == peer.ID() {
return types.ErrSwitchConnectToSelf return ErrSwitchConnectToSelf
} }
// Avoid duplicate // Avoid duplicate
if sw.peers.Has(peer.ID()) { if sw.peers.Has(peer.ID()) {
return types.ErrSwitchDuplicatePeer return ErrSwitchDuplicatePeer
} }

View File

@ -16,8 +16,7 @@ import (
"github.com/tendermint/tmlibs/log" "github.com/tendermint/tmlibs/log"
cfg "github.com/tendermint/tendermint/config" cfg "github.com/tendermint/tendermint/config"
"github.com/tendermint/tendermint/p2p/tmconn" "github.com/tendermint/tendermint/p2p/conn"
"github.com/tendermint/tendermint/p2p/types"
) )
var ( var (
@ -30,7 +29,7 @@ func init() {
} }
type PeerMessage struct { type PeerMessage struct {
PeerID types.ID PeerID ID
Bytes []byte Bytes []byte
Counter int Counter int
} }
@ -39,7 +38,7 @@ type TestReactor struct {
BaseReactor BaseReactor
mtx sync.Mutex mtx sync.Mutex
channels []*tmconn.ChannelDescriptor channels []*conn.ChannelDescriptor
peersAdded []Peer peersAdded []Peer
peersRemoved []Peer peersRemoved []Peer
logMessages bool logMessages bool
@ -47,7 +46,7 @@ type TestReactor struct {
msgsReceived map[byte][]PeerMessage msgsReceived map[byte][]PeerMessage
} }
func NewTestReactor(channels []*tmconn.ChannelDescriptor, logMessages bool) *TestReactor { func NewTestReactor(channels []*conn.ChannelDescriptor, logMessages bool) *TestReactor {
tr := &TestReactor{ tr := &TestReactor{
channels: channels, channels: channels,
logMessages: logMessages, logMessages: logMessages,
@ -58,7 +57,7 @@ func NewTestReactor(channels []*tmconn.ChannelDescriptor, logMessages bool) *Tes
return tr return tr
} }
func (tr *TestReactor) GetChannels() []*tmconn.ChannelDescriptor { func (tr *TestReactor) GetChannels() []*conn.ChannelDescriptor {
return tr.channels return tr.channels
} }
@ -102,11 +101,11 @@ func MakeSwitchPair(t testing.TB, initSwitch func(int, *Switch) *Switch) (*Switc
func initSwitchFunc(i int, sw *Switch) *Switch { func initSwitchFunc(i int, sw *Switch) *Switch {
// Make two reactors of two channels each // Make two reactors of two channels each
sw.AddReactor("foo", NewTestReactor([]*tmconn.ChannelDescriptor{ sw.AddReactor("foo", NewTestReactor([]*conn.ChannelDescriptor{
{ID: byte(0x00), Priority: 10}, {ID: byte(0x00), Priority: 10},
{ID: byte(0x01), Priority: 10}, {ID: byte(0x01), Priority: 10},
}, true)) }, true))
sw.AddReactor("bar", NewTestReactor([]*tmconn.ChannelDescriptor{ sw.AddReactor("bar", NewTestReactor([]*conn.ChannelDescriptor{
{ID: byte(0x02), Priority: 10}, {ID: byte(0x02), Priority: 10},
{ID: byte(0x03), Priority: 10}, {ID: byte(0x03), Priority: 10},
}, true)) }, true))
@ -163,7 +162,7 @@ func TestConnAddrFilter(t *testing.T) {
defer s1.Stop() defer s1.Stop()
defer s2.Stop() defer s2.Stop()
c1, c2 := tmconn.NetPipe() c1, c2 := conn.NetPipe()
s1.SetAddrFilter(func(addr net.Addr) error { s1.SetAddrFilter(func(addr net.Addr) error {
if addr.String() == c1.RemoteAddr().String() { if addr.String() == c1.RemoteAddr().String() {
@ -199,7 +198,7 @@ func TestConnPubKeyFilter(t *testing.T) {
defer s1.Stop() defer s1.Stop()
defer s2.Stop() defer s2.Stop()
c1, c2 := tmconn.NetPipe() c1, c2 := conn.NetPipe()
// set pubkey filter // set pubkey filter
s1.SetPubKeyFilter(func(pubkey crypto.PubKey) error { s1.SetPubKeyFilter(func(pubkey crypto.PubKey) error {
@ -306,11 +305,11 @@ func BenchmarkSwitches(b *testing.B) {
s1, s2 := MakeSwitchPair(b, func(i int, sw *Switch) *Switch { s1, s2 := MakeSwitchPair(b, func(i int, sw *Switch) *Switch {
// Make bar reactors of bar channels each // Make bar reactors of bar channels each
sw.AddReactor("foo", NewTestReactor([]*tmconn.ChannelDescriptor{ sw.AddReactor("foo", NewTestReactor([]*conn.ChannelDescriptor{
{ID: byte(0x00), Priority: 10}, {ID: byte(0x00), Priority: 10},
{ID: byte(0x01), Priority: 10}, {ID: byte(0x01), Priority: 10},
}, false)) }, false))
sw.AddReactor("bar", NewTestReactor([]*tmconn.ChannelDescriptor{ sw.AddReactor("bar", NewTestReactor([]*conn.ChannelDescriptor{
{ID: byte(0x02), Priority: 10}, {ID: byte(0x02), Priority: 10},
{ID: byte(0x03), Priority: 10}, {ID: byte(0x03), Priority: 10},
}, false)) }, false))

View File

@ -9,8 +9,7 @@ import (
"github.com/tendermint/tmlibs/log" "github.com/tendermint/tmlibs/log"
cfg "github.com/tendermint/tendermint/config" cfg "github.com/tendermint/tendermint/config"
"github.com/tendermint/tendermint/p2p/tmconn" "github.com/tendermint/tendermint/p2p/conn"
"github.com/tendermint/tendermint/p2p/types"
) )
func AddPeerToSwitch(sw *Switch, peer Peer) { func AddPeerToSwitch(sw *Switch, peer Peer) {
@ -20,22 +19,22 @@ func AddPeerToSwitch(sw *Switch, peer Peer) {
func CreateRandomPeer(outbound bool) *peer { func CreateRandomPeer(outbound bool) *peer {
addr, netAddr := CreateRoutableAddr() addr, netAddr := CreateRoutableAddr()
p := &peer{ p := &peer{
nodeInfo: types.NodeInfo{ nodeInfo: NodeInfo{
ListenAddr: netAddr.DialString(), ListenAddr: netAddr.DialString(),
PubKey: crypto.GenPrivKeyEd25519().Wrap().PubKey(), PubKey: crypto.GenPrivKeyEd25519().Wrap().PubKey(),
}, },
outbound: outbound, outbound: outbound,
mconn: &tmconn.MConnection{}, mconn: &conn.MConnection{},
} }
p.SetLogger(log.TestingLogger().With("peer", addr)) p.SetLogger(log.TestingLogger().With("peer", addr))
return p return p
} }
func CreateRoutableAddr() (addr string, netAddr *types.NetAddress) { func CreateRoutableAddr() (addr string, netAddr *NetAddress) {
for { for {
var err error var err error
addr = cmn.Fmt("%X@%v.%v.%v.%v:46656", cmn.RandBytes(20), rand.Int()%256, rand.Int()%256, rand.Int()%256, rand.Int()%256) addr = cmn.Fmt("%X@%v.%v.%v.%v:46656", cmn.RandBytes(20), rand.Int()%256, rand.Int()%256, rand.Int()%256, rand.Int()%256)
netAddr, err = types.NewNetAddressString(addr) netAddr, err = NewNetAddressString(addr)
if err != nil { if err != nil {
panic(err) panic(err)
} }
@ -78,7 +77,7 @@ func MakeConnectedSwitches(cfg *cfg.P2PConfig, n int, initSwitch func(int, *Swit
func Connect2Switches(switches []*Switch, i, j int) { func Connect2Switches(switches []*Switch, i, j int) {
switchI := switches[i] switchI := switches[i]
switchJ := switches[j] switchJ := switches[j]
c1, c2 := tmconn.NetPipe() c1, c2 := conn.NetPipe()
doneCh := make(chan struct{}) doneCh := make(chan struct{})
go func() { go func() {
err := switchI.addPeerWithConnection(c1) err := switchI.addPeerWithConnection(c1)
@ -130,13 +129,13 @@ func StartSwitches(switches []*Switch) error {
func MakeSwitch(cfg *cfg.P2PConfig, i int, network, version string, initSwitch func(int, *Switch) *Switch) *Switch { func MakeSwitch(cfg *cfg.P2PConfig, i int, network, version string, initSwitch func(int, *Switch) *Switch) *Switch {
// new switch, add reactors // new switch, add reactors
// TODO: let the config be passed in? // TODO: let the config be passed in?
nodeKey := &types.NodeKey{ nodeKey := &NodeKey{
PrivKey: crypto.GenPrivKeyEd25519().Wrap(), PrivKey: crypto.GenPrivKeyEd25519().Wrap(),
} }
s := NewSwitch(cfg) s := NewSwitch(cfg)
s.SetLogger(log.TestingLogger()) s.SetLogger(log.TestingLogger())
s = initSwitch(i, s) s = initSwitch(i, s)
s.SetNodeInfo(types.NodeInfo{ s.SetNodeInfo(NodeInfo{
PubKey: nodeKey.PubKey(), PubKey: nodeKey.PubKey(),
Moniker: cmn.Fmt("switch%d", i), Moniker: cmn.Fmt("switch%d", i),
Network: network, Network: network,

View File

@ -1,12 +1,8 @@
package p2p package p2p
import ( import (
"github.com/tendermint/tendermint/p2p/tmconn" "github.com/tendermint/tendermint/p2p/conn"
"github.com/tendermint/tendermint/p2p/types"
) )
type ID = types.ID type ChannelDescriptor = conn.ChannelDescriptor
type NodeInfo = types.NodeInfo type ConnectionStatus = conn.ConnectionStatus
type ChannelDescriptor = tmconn.ChannelDescriptor
type ConnectionStatus = tmconn.ConnectionStatus