Browse Source

server.go + peer: use MessageConn interface instead of brontide.Conn

master
eugene 4 years ago
parent
commit
fada3dd871
  1. 224
      peer/brontide.go
  2. 231
      peer/config.go
  3. 47
      peer/interfaces.go
  4. 2
      server.go

224
peer/brontide.go

@ -17,20 +17,28 @@ import (
"github.com/btcsuite/btcd/wire"
"github.com/davecgh/go-spew/spew"
"github.com/lightningnetwork/lnd/brontide"
"github.com/lightningnetwork/lnd/buffer"
"github.com/lightningnetwork/lnd/chainntnfs"
"github.com/lightningnetwork/lnd/channeldb"
"github.com/lightningnetwork/lnd/channelnotifier"
"github.com/lightningnetwork/lnd/contractcourt"
"github.com/lightningnetwork/lnd/discovery"
"github.com/lightningnetwork/lnd/feature"
"github.com/lightningnetwork/lnd/htlcswitch"
"github.com/lightningnetwork/lnd/htlcswitch/hodl"
"github.com/lightningnetwork/lnd/htlcswitch/hop"
"github.com/lightningnetwork/lnd/input"
"github.com/lightningnetwork/lnd/invoices"
"github.com/lightningnetwork/lnd/lnpeer"
"github.com/lightningnetwork/lnd/lnwallet"
"github.com/lightningnetwork/lnd/lnwallet/chainfee"
"github.com/lightningnetwork/lnd/lnwallet/chancloser"
"github.com/lightningnetwork/lnd/lnwire"
"github.com/lightningnetwork/lnd/netann"
"github.com/lightningnetwork/lnd/pool"
"github.com/lightningnetwork/lnd/queue"
"github.com/lightningnetwork/lnd/ticker"
"github.com/lightningnetwork/lnd/watchtower/wtclient"
)
const (
@ -112,6 +120,209 @@ type TimestampedError struct {
Timestamp time.Time
}
// Config defines configuration fields that are necessary for a peer object
// to function.
type Config struct {
// Conn is the underlying network connection for this peer.
Conn MessageConn
// ConnReq stores information related to the persistent connection request
// for this peer.
ConnReq *connmgr.ConnReq
// PubKeyBytes is the serialized, compressed public key of this peer.
PubKeyBytes [33]byte
// Addr is the network address of the peer.
Addr *lnwire.NetAddress
// Inbound indicates whether or not the peer is an inbound peer.
Inbound bool
// Features is the set of features that we advertise to the remote party.
Features *lnwire.FeatureVector
// LegacyFeatures is the set of features that we advertise to the remote
// peer for backwards compatibility. Nodes that have not implemented
// flat features will still be able to read our feature bits from the
// legacy global field, but we will also advertise everything in the
// default features field.
LegacyFeatures *lnwire.FeatureVector
// OutgoingCltvRejectDelta defines the number of blocks before expiry of
// an htlc where we don't offer it anymore.
OutgoingCltvRejectDelta uint32
// ChanActiveTimeout specifies the duration the peer will wait to request
// a channel reenable, beginning from the time the peer was started.
ChanActiveTimeout time.Duration
// ErrorBuffer stores a set of errors related to a peer. It contains error
// messages that our peer has recently sent us over the wire and records of
// unknown messages that were sent to us so that we can have a full track
// record of the communication errors we have had with our peer. If we
// choose to disconnect from a peer, it also stores the reason we had for
// disconnecting.
ErrorBuffer *queue.CircularBuffer
// WritePool is the task pool that manages reuse of write buffers. Write
// tasks are submitted to the pool in order to conserve the total number of
// write buffers allocated at any one time, and decouple write buffer
// allocation from the peer life cycle.
WritePool *pool.Write
// ReadPool is the task pool that manages reuse of read buffers.
ReadPool *pool.Read
// Switch is a pointer to the htlcswitch. It is used to setup, get, and
// tear-down ChannelLinks.
Switch *htlcswitch.Switch
// InterceptSwitch is a pointer to the InterceptableSwitch, a wrapper around
// the regular Switch. We only export it here to pass ForwardPackets to the
// ChannelLinkConfig.
InterceptSwitch *htlcswitch.InterceptableSwitch
// ChannelDB is used to fetch opened channels, and closed channels.
ChannelDB *channeldb.DB
// ChannelGraph is a pointer to the channel graph which is used to
// query information about the set of known active channels.
ChannelGraph *channeldb.ChannelGraph
// ChainArb is used to subscribe to channel events, update contract signals,
// and force close channels.
ChainArb *contractcourt.ChainArbitrator
// AuthGossiper is needed so that the Brontide impl can register with the
// gossiper and process remote channel announcements.
AuthGossiper *discovery.AuthenticatedGossiper
// ChanStatusMgr is used to set or un-set the disabled bit in channel
// updates.
ChanStatusMgr *netann.ChanStatusManager
// ChainIO is used to retrieve the best block.
ChainIO lnwallet.BlockChainIO
// FeeEstimator is used to compute our target ideal fee-per-kw when
// initializing the coop close process.
FeeEstimator chainfee.Estimator
// Signer is used when creating *lnwallet.LightningChannel instances.
Signer input.Signer
// SigPool is used when creating *lnwallet.LightningChannel instances.
SigPool *lnwallet.SigPool
// Wallet is used to publish transactions and generate delivery scripts
// during the coop close process.
Wallet *lnwallet.LightningWallet
// ChainNotifier is used to receive confirmations of a coop close
// transaction.
ChainNotifier chainntnfs.ChainNotifier
// RoutingPolicy is used to set the forwarding policy for links created by
// the Brontide.
RoutingPolicy htlcswitch.ForwardingPolicy
// Sphinx is used when setting up ChannelLinks so they can decode sphinx
// onion blobs.
Sphinx *hop.OnionProcessor
// WitnessBeacon is used when setting up ChannelLinks so they can add any
// preimages that they learn.
WitnessBeacon contractcourt.WitnessBeacon
// Invoices is passed to the ChannelLink on creation and handles all
// invoice-related logic.
Invoices *invoices.InvoiceRegistry
// ChannelNotifier is used by the link to notify other sub-systems about
// channel-related events and by the Brontide to subscribe to
// ActiveLinkEvents.
ChannelNotifier *channelnotifier.ChannelNotifier
// HtlcNotifier is used when creating a ChannelLink.
HtlcNotifier *htlcswitch.HtlcNotifier
// TowerClient is used when creating a ChannelLink.
TowerClient wtclient.Client
// DisconnectPeer is used to disconnect this peer if the cooperative close
// process fails.
DisconnectPeer func(*btcec.PublicKey) error
// GenNodeAnnouncement is used to send our node announcement to the remote
// on startup.
GenNodeAnnouncement func(bool,
...netann.NodeAnnModifier) (lnwire.NodeAnnouncement, error)
// PrunePersistentPeerConnection is used to remove all internal state
// related to this peer in the server.
PrunePersistentPeerConnection func([33]byte)
// FetchLastChanUpdate fetches our latest channel update for a target
// channel.
FetchLastChanUpdate func(lnwire.ShortChannelID) (*lnwire.ChannelUpdate,
error)
// ProcessFundingOpen is used to hand off an OpenChannel message to the
// funding manager.
ProcessFundingOpen func(*lnwire.OpenChannel, lnpeer.Peer)
// ProcessFundingAccept is used to hand off an AcceptChannel message to the
// funding manager.
ProcessFundingAccept func(*lnwire.AcceptChannel, lnpeer.Peer)
// ProcessFundingCreated is used to hand off a FundingCreated message to
// the funding manager.
ProcessFundingCreated func(*lnwire.FundingCreated, lnpeer.Peer)
// ProcessFundingSigned is used to hand off a FundingSigned message to the
// funding manager.
ProcessFundingSigned func(*lnwire.FundingSigned, lnpeer.Peer)
// ProcessFundingLocked is used to hand off a FundingLocked message to the
// funding manager.
ProcessFundingLocked func(*lnwire.FundingLocked, lnpeer.Peer)
// ProcessFundingError is used to hand off an Error message to the funding
// manager.
ProcessFundingError func(*lnwire.Error, *btcec.PublicKey)
// IsPendingChannel is used to determine whether to send an Error message
// to the funding manager or not.
IsPendingChannel func([32]byte, *btcec.PublicKey) bool
// Hodl is used when creating ChannelLinks to specify HodlFlags as
// breakpoints in dev builds.
Hodl *hodl.Config
// UnsafeReplay is used when creating ChannelLinks to specify whether or
// not to replay adds on its commitment tx.
UnsafeReplay bool
// MaxOutgoingCltvExpiry is used when creating ChannelLinks and is the max
// number of blocks that funds could be locked up for when forwarding
// payments.
MaxOutgoingCltvExpiry uint32
// MaxChannelFeeAllocation is used when creating ChannelLinks and is the
// maximum percentage of total funds that can be allocated to a channel's
// commitment fee. This only applies for the initiator of the channel.
MaxChannelFeeAllocation float64
// ServerPubKey is the serialized, compressed public key of our lnd node.
// It is used to determine which policy (channel edge) to pass to the
// ChannelLink.
ServerPubKey [33]byte
// Quit is the server's quit channel. If this is closed, we halt operation.
Quit chan struct{}
}
// Brontide is an active peer on the Lightning Network. This struct is responsible
// for managing any channel state related to this peer. To do so, it has
// several helper goroutines to handle events such as HTLC timeouts, new
@ -693,11 +904,7 @@ func (p *Brontide) String() string {
// readNextMessage reads, and returns the next message on the wire along with
// any additional raw payload.
func (p *Brontide) readNextMessage() (lnwire.Message, error) {
noiseConn, ok := p.cfg.Conn.(*brontide.Conn)
if !ok {
return nil, fmt.Errorf("brontide.Conn required to read messages")
}
noiseConn := p.cfg.Conn
err := noiseConn.SetReadDeadline(time.Time{})
if err != nil {
return nil, err
@ -1516,10 +1723,7 @@ func (p *Brontide) writeMessage(msg lnwire.Message) error {
p.logWireMessage(msg, false)
}
noiseConn, ok := p.cfg.Conn.(*brontide.Conn)
if !ok {
return fmt.Errorf("brontide.Conn required to write messages")
}
noiseConn := p.cfg.Conn
flushMsg := func() error {
// Ensure the write deadline is set before we attempt to send

231
peer/config.go

@ -1,231 +0,0 @@
package peer
import (
"net"
"time"
"github.com/btcsuite/btcd/btcec"
"github.com/btcsuite/btcd/connmgr"
"github.com/lightningnetwork/lnd/chainntnfs"
"github.com/lightningnetwork/lnd/channeldb"
"github.com/lightningnetwork/lnd/channelnotifier"
"github.com/lightningnetwork/lnd/contractcourt"
"github.com/lightningnetwork/lnd/discovery"
"github.com/lightningnetwork/lnd/htlcswitch"
"github.com/lightningnetwork/lnd/htlcswitch/hodl"
"github.com/lightningnetwork/lnd/htlcswitch/hop"
"github.com/lightningnetwork/lnd/input"
"github.com/lightningnetwork/lnd/invoices"
"github.com/lightningnetwork/lnd/lnpeer"
"github.com/lightningnetwork/lnd/lnwallet"
"github.com/lightningnetwork/lnd/lnwallet/chainfee"
"github.com/lightningnetwork/lnd/lnwire"
"github.com/lightningnetwork/lnd/netann"
"github.com/lightningnetwork/lnd/pool"
"github.com/lightningnetwork/lnd/queue"
"github.com/lightningnetwork/lnd/watchtower/wtclient"
)
// Config defines configuration fields that are necessary for a peer object
// to function.
type Config struct {
// Conn is the underlying network connection for this peer.
Conn net.Conn
// ConnReq stores information related to the persistent connection request
// for this peer.
ConnReq *connmgr.ConnReq
// PubKeyBytes is the serialized, compressed public key of this peer.
PubKeyBytes [33]byte
// Addr is the network address of the peer.
Addr *lnwire.NetAddress
// Inbound indicates whether or not the peer is an inbound peer.
Inbound bool
// Features is the set of features that we advertise to the remote party.
Features *lnwire.FeatureVector
// LegacyFeatures is the set of features that we advertise to the remote
// peer for backwards compatibility. Nodes that have not implemented
// flat features will still be able to read our feature bits from the
// legacy global field, but we will also advertise everything in the
// default features field.
LegacyFeatures *lnwire.FeatureVector
// OutgoingCltvRejectDelta defines the number of blocks before expiry of
// an htlc where we don't offer it anymore.
OutgoingCltvRejectDelta uint32
// ChanActiveTimeout specifies the duration the peer will wait to request
// a channel reenable, beginning from the time the peer was started.
ChanActiveTimeout time.Duration
// ErrorBuffer stores a set of errors related to a peer. It contains error
// messages that our peer has recently sent us over the wire and records of
// unknown messages that were sent to us so that we can have a full track
// record of the communication errors we have had with our peer. If we
// choose to disconnect from a peer, it also stores the reason we had for
// disconnecting.
ErrorBuffer *queue.CircularBuffer
// WritePool is the task pool that manages reuse of write buffers. Write
// tasks are submitted to the pool in order to conserve the total number of
// write buffers allocated at any one time, and decouple write buffer
// allocation from the peer life cycle.
WritePool *pool.Write
// ReadPool is the task pool that manages reuse of read buffers.
ReadPool *pool.Read
// Switch is a pointer to the htlcswitch. It is used to setup, get, and
// tear-down ChannelLinks.
Switch *htlcswitch.Switch
// InterceptSwitch is a pointer to the InterceptableSwitch, a wrapper around
// the regular Switch. We only export it here to pass ForwardPackets to the
// ChannelLinkConfig.
InterceptSwitch *htlcswitch.InterceptableSwitch
// ChannelDB is used to fetch opened channels, and closed channels.
ChannelDB *channeldb.DB
// ChannelGraph is a pointer to the channel graph which is used to
// query information about the set of known active channels.
ChannelGraph *channeldb.ChannelGraph
// ChainArb is used to subscribe to channel events, update contract signals,
// and force close channels.
ChainArb *contractcourt.ChainArbitrator
// AuthGossiper is needed so that the Brontide impl can register with the
// gossiper and process remote channel announcements.
AuthGossiper *discovery.AuthenticatedGossiper
// ChanStatusMgr is used to set or un-set the disabled bit in channel
// updates.
ChanStatusMgr *netann.ChanStatusManager
// ChainIO is used to retrieve the best block.
ChainIO lnwallet.BlockChainIO
// FeeEstimator is used to compute our target ideal fee-per-kw when
// initializing the coop close process.
FeeEstimator chainfee.Estimator
// Signer is used when creating *lnwallet.LightningChannel instances.
Signer input.Signer
// SigPool is used when creating *lnwallet.LightningChannel instances.
SigPool *lnwallet.SigPool
// Wallet is used to publish transactions and generate delivery scripts
// during the coop close process.
Wallet *lnwallet.LightningWallet
// ChainNotifier is used to receive confirmations of a coop close
// transaction.
ChainNotifier chainntnfs.ChainNotifier
// RoutingPolicy is used to set the forwarding policy for links created by
// the Brontide.
RoutingPolicy htlcswitch.ForwardingPolicy
// Sphinx is used when setting up ChannelLinks so they can decode sphinx
// onion blobs.
Sphinx *hop.OnionProcessor
// WitnessBeacon is used when setting up ChannelLinks so they can add any
// preimages that they learn.
WitnessBeacon contractcourt.WitnessBeacon
// Invoices is passed to the ChannelLink on creation and handles all
// invoice-related logic.
Invoices *invoices.InvoiceRegistry
// ChannelNotifier is used by the link to notify other sub-systems about
// channel-related events and by the Brontide to subscribe to
// ActiveLinkEvents.
ChannelNotifier *channelnotifier.ChannelNotifier
// HtlcNotifier is used when creating a ChannelLink.
HtlcNotifier *htlcswitch.HtlcNotifier
// TowerClient is used when creating a ChannelLink.
TowerClient wtclient.Client
// DisconnectPeer is used to disconnect this peer if the cooperative close
// process fails.
DisconnectPeer func(*btcec.PublicKey) error
// GenNodeAnnouncement is used to send our node announcement to the remote
// on startup.
GenNodeAnnouncement func(bool,
...netann.NodeAnnModifier) (lnwire.NodeAnnouncement, error)
// PrunePersistentPeerConnection is used to remove all internal state
// related to this peer in the server.
PrunePersistentPeerConnection func([33]byte)
// FetchLastChanUpdate fetches our latest channel update for a target
// channel.
FetchLastChanUpdate func(lnwire.ShortChannelID) (*lnwire.ChannelUpdate,
error)
// ProcessFundingOpen is used to hand off an OpenChannel message to the
// funding manager.
ProcessFundingOpen func(*lnwire.OpenChannel, lnpeer.Peer)
// ProcessFundingAccept is used to hand off an AcceptChannel message to the
// funding manager.
ProcessFundingAccept func(*lnwire.AcceptChannel, lnpeer.Peer)
// ProcessFundingCreated is used to hand off a FundingCreated message to
// the funding manager.
ProcessFundingCreated func(*lnwire.FundingCreated, lnpeer.Peer)
// ProcessFundingSigned is used to hand off a FundingSigned message to the
// funding manager.
ProcessFundingSigned func(*lnwire.FundingSigned, lnpeer.Peer)
// ProcessFundingLocked is used to hand off a FundingLocked message to the
// funding manager.
ProcessFundingLocked func(*lnwire.FundingLocked, lnpeer.Peer)
// ProcessFundingError is used to hand off an Error message to the funding
// manager.
ProcessFundingError func(*lnwire.Error, *btcec.PublicKey)
// IsPendingChannel is used to determine whether to send an Error message
// to the funding manager or not.
IsPendingChannel func([32]byte, *btcec.PublicKey) bool
// Hodl is used when creating ChannelLinks to specify HodlFlags as
// breakpoints in dev builds.
Hodl *hodl.Config
// UnsafeReplay is used when creating ChannelLinks to specify whether or
// not to replay adds on its commitment tx.
UnsafeReplay bool
// MaxOutgoingCltvExpiry is used when creating ChannelLinks and is the max
// number of blocks that funds could be locked up for when forwarding
// payments.
MaxOutgoingCltvExpiry uint32
// MaxChannelFeeAllocation is used when creating ChannelLinks and is the
// maximum percentage of total funds that can be allocated to a channel's
// commitment fee. This only applies for the initiator of the channel.
MaxChannelFeeAllocation float64
// ServerPubKey is the serialized, compressed public key of our lnd node.
// It is used to determine which policy (channel edge) to pass to the
// ChannelLink.
ServerPubKey [33]byte
// Quit is the server's quit channel. If this is closed, we halt operation.
Quit chan struct{}
}

47
peer/interfaces.go

@ -1,6 +1,11 @@
package peer
import "github.com/lightningnetwork/lnd/lnwire"
import (
"net"
"time"
"github.com/lightningnetwork/lnd/lnwire"
)
// LinkUpdater is an interface implemented by most messages in BOLT 2 that are
// allowed to update the channel state.
@ -9,3 +14,43 @@ type LinkUpdater interface {
// is intended.
TargetChanID() lnwire.ChannelID
}
// MessageConn is an interface implemented by anything that delivers
// an lnwire.Message using a net.Conn interface.
type MessageConn interface {
// RemoteAddr returns the remote address on the other end of the connection.
RemoteAddr() net.Addr
// LocalAddr returns the local address on our end of the connection.
LocalAddr() net.Addr
// Read reads bytes from the connection.
Read([]byte) (int, error)
// Write writes bytes to the connection.
Write([]byte) (int, error)
// SetDeadline sets the deadline for the connection.
SetDeadline(time.Time) error
// SetReadDeadline sets the read deadline.
SetReadDeadline(time.Time) error
// SetWriteDeadline sets the write deadline.
SetWriteDeadline(time.Time) error
// Close closes the connection.
Close() error
// Flush attempts a flush.
Flush() (int, error)
// WriteMessage writes the message.
WriteMessage([]byte) error
// ReadNextHeader reads the next header.
ReadNextHeader() (uint32, error)
// ReadNextBody reads the next body.
ReadNextBody([]byte) ([]byte, error)
}

2
server.go

@ -2946,7 +2946,7 @@ func (s *server) peerConnected(conn net.Conn, connReq *connmgr.ConnReq,
// htlcs, an extra block is added to prevent the channel from being
// closed when the htlc is outstanding and a new block comes in.
pCfg := peer.Config{
Conn: conn,
Conn: brontideConn,
ConnReq: connReq,
Addr: peerAddr,
Inbound: inbound,

Loading…
Cancel
Save