routing: add new notificaiton client for topology changes

This commit adds some new functionality to the channel router: the
ability to dispatch notification to registered clients upon either a
channel being closed, a new node appearing, or an exiting client being
updated or opened for the first time.

With this change, the integration tests will now be able to eliminate
most of the sleep as we gain a new syntonization point into the
propagation of information within the test network. Additionally, this
also paves the way for client side software to dynamically visualize
the channel graph in real-time as nodes+channels are updated.
This commit is contained in:
Olaoluwa Osuntokun 2017-03-08 14:27:46 -08:00
parent 7bdf02bc9e
commit 1a78c73f7e
No known key found for this signature in database
GPG Key ID: 9CC5B105D03521A2
2 changed files with 484 additions and 114 deletions

321
routing/notifications.go Normal file

@ -0,0 +1,321 @@
package routing
import (
"errors"
"fmt"
"net"
"sync/atomic"
"github.com/lightningnetwork/lnd/channeldb"
"github.com/lightningnetwork/lnd/lnwire"
"github.com/roasbeef/btcd/btcec"
"github.com/roasbeef/btcd/wire"
"github.com/roasbeef/btcutil"
)
// TopologyClient represents an intent to receive notifications from the
// channel router regarding changes to the topology of the channel graph. The
// TopologyChanges channel will be sent upon with new updates to the channel
// graph in real-time as they're encountered.
type TopologyClient struct {
// TopologyChanges is a recieve only channel that new channel graph
// updates will be sent over.
TopologyChanges <-chan *TopologyChange
// Cancel is a function closure that should be executed when the client
// wishes to cancel their notification intent. Doing so allows the
// ChannelRouter to free up resources.
Cancel func()
}
// topologyClientUpdate is a message sent to the channel router to either
// register a new topology client or re-register an existing client.
type topologyClientUpdate struct {
// cancel indicates if the update to the client is cancelling an
// existing client's notifications. If not then this update will be to
// register a new set of notifications.
cancel bool
// clientID is the unique identifier for this client. Any further
// updates (deleting or adding) to this notification client will be
// dispatched according to the target clientID.
clientID uint64
// ntfnChan is a *send-only* channel in which notifications should be
// sent over from router -> client.
ntfnChan chan<- *TopologyChange
}
// SubscribeTopology returns a new topology client which can be used by the
// caller to receive notifications when ever a change in the channel graph
// topology occurs. Changes that will be sent at notifications include: new
// nodes appearing, node updating their attributes, new channels, channels
// closing, and updates in the routing policies of a channel's directed edges.
func (r *ChannelRouter) SubscribeTopology() (*TopologyClient, error) {
// We'll first atomitcally obtain the next ID for this client from the
// incrementing client ID counter.
clientID := atomic.AddUint64(&r.ntfnClientCounter, 1)
ntfnChan := make(chan *TopologyChange)
select {
case r.ntfnClientUpdates <- &topologyClientUpdate{
cancel: false,
clientID: clientID,
ntfnChan: ntfnChan,
}:
case <-r.quit:
return nil, errors.New("ChannelRouter shutting down")
}
return &TopologyClient{
TopologyChanges: ntfnChan,
Cancel: func() {
select {
case r.ntfnClientUpdates <- &topologyClientUpdate{
cancel: true,
clientID: clientID,
}:
case <-r.quit:
return
}
},
}, nil
}
// topologyClient is a data-structure use by the channel router to couple the
// client's notification channel along with a special "exit" channel that can
// be used to cancel all lingering goroutines blocked on a send to the
// notification channel.
type topologyClient struct {
// ntfnChan is a send-only channel that's used to propagate
// notification s from the channel router to an instance of a
// topologyClient client.
ntfnChan chan<- *TopologyChange
// exit is a channel that is used internally by the channel router to
// cancel any active un-consumed goroutine notifications.
exit chan struct{}
}
// notifyTopologyChange notifies all registered clients of a new change in
// graph topology in a non-blocking.
func (r *ChannelRouter) notifyTopologyChange(topologyDiff *TopologyChange) {
for _, client := range r.topologyClients {
go func(c *topologyClient) {
select {
// In this case we'll try to send the notification
// directly to the upstream client consumer.
case client.ntfnChan <- topologyDiff:
// If the client cancel's the notifications, then we'll
// exit early.
case <-client.exit:
// Similarly, if the ChannelRouter itself exists early,
// then we'll also exit ourselves.
case <-r.quit:
}
}(&client)
}
}
// TopologyChange represents a new set of modifications to the channel graph.
// Topology changes will be dispatched in real-time as the ChannelGraph
// validates and process modifications to the authenticated channel graph.
type TopologyChange struct {
// NodeUpdates is a slice of nodes which are either new to the channel
// graph, or have had their attributes updated in an authenticated
// manner.
NodeUpdates []*NetworkNodeUpdate
// ChanelEdgeUpdates is a slice of channel edges which are either newly
// opened and authenticated, or have had their routing policies
// updated.
ChannelEdgeUpdates []*ChannelEdgeUpdate
// ClosedChannels contains a slice of close channel summaries which
// described which block a channel was closed at, and also carry
// supplemental information such as the capacity of the former channel.
ClosedChannels []*ClosedChanSummary
}
// isEmpty returns true if the TopologyChange is empty. A TopologyChange is
// considered empty, if it contains no *new* updates of any type.
func (t *TopologyChange) isEmpty() bool {
return len(t.NodeUpdates) == 0 && len(t.ChannelEdgeUpdates) == 0 &&
len(t.ClosedChannels) == 0
}
// ClosedChanSummary is a summary of a channel that was detected as being
// closed by monitoring the blockchain. Once a channel's funding point has been
// spent, the channel will automatically be marked as closed by the
// ChainNotifier.
//
// TODO(roasbeef): add nodes involved?
type ClosedChanSummary struct {
// ChanID is the short-channel ID which uniquely identifies the
// channel.
ChanID uint64
// Capacity was the total capacity of the channel before it was closed.
Capacity btcutil.Amount
// ClosedHeight is the height in the chain that the channel was closed
// at.
ClosedHeight uint32
// ChanPoint is the funding point, or the multi-sig utxo which
// previously represented the channel.
ChanPoint wire.OutPoint
}
// createCloseSummaries takes in a slice of channels closed at the target block
// height and creates a slice of summaries which of each channel closure.
func createCloseSummaries(blockHeight uint32,
closedChans ...*channeldb.ChannelEdgeInfo) []*ClosedChanSummary {
closeSummaries := make([]*ClosedChanSummary, len(closedChans))
for i, closedChan := range closedChans {
closeSummaries[i] = &ClosedChanSummary{
ChanID: closedChan.ChannelID,
Capacity: closedChan.Capacity,
ClosedHeight: blockHeight,
ChanPoint: closedChan.ChannelPoint,
}
}
return closeSummaries
}
// NetworkNodeUpdate is an update for a node within the Lightning Network. A
// NetworkNodeUpdate is sent out either when a new node joins the network, or a
// node broadcasts a new update with a newer time stamp that supersedes it's
// old update. All updates are properly authenticated.
type NetworkNodeUpdate struct {
// Addresses is a slice of all the node's known addresses.
Addresses []net.Addr
// IdentityKey is the identity public key of the target node. This is
// used to encrypt onion blobs as well as to authenticate any new
// updates.
IdentityKey *btcec.PublicKey
// GlobalFeatures is a set of opaque bytes that describe the set of
// features supported by the node.
GlobalFeatures []byte
// Alias is the alias or nick name of the node.
Alias string
}
// ChannelEdgeUpdate is an update for a new channel within the ChannelGraph.
// This update is sent out once a new authenticated channel edge is discovered
// within the network. These updates are directional, so if a channel is fully
// public, then there will be two updates sent out: one for each direction
// within the channel. Each update will carry that particular routing edge
// policy for the channel direction.
//
// An edge is a channel in the direction of AdvertisingNode -> ConnectingNode.
type ChannelEdgeUpdate struct {
// ChanID is the unique short channel ID for the channel. This encodes
// where in the blockchain the channel's funding transaction was
// originally confirmed.
ChanID uint64
// Capacity is the capacity of the newly created channel.
Capacity btcutil.Amount
// MinHTLC is the minimum HTLC amount that this channel will forward.
MinHTLC btcutil.Amount
// BaseFee is the base fee that will charged for all HTLC's forwarded
// across the this channel direction.
BaseFee btcutil.Amount
// FeeRate is the fee rate that will be shared for all HTLC's forwarded
// across this channel direction.
FeeRate btcutil.Amount
// TimeLockDelta is the time-lock expressed in blocks that will be
// added to outgoing HTLC's from incoming HTLC's. This value is the
// difference of the incoming and outgoing HTLC's time-locks routed
// through this hop.
TimeLockDelta uint16
// AdvertisingNode is the node that's advertising this edge.
AdvertisingNode *btcec.PublicKey
// ConnectingNode is the node that the advertising node connects to.
ConnectingNode *btcec.PublicKey
}
// appendTopologyChange appends the passed update message to the passed
// TopologyChange, properly identifying which type of update the message
// constitutes. This function will also fetch any required auxiliary
// information required to create the topology change update from the graph
// database.
func addToTopologyChange(graph *channeldb.ChannelGraph, update *TopologyChange,
msg lnwire.Message) error {
switch m := msg.(type) {
// Any node announcement maps directly to a NetworkNodeUpdate struct.
// No further data munging or db queries are required.
case *lnwire.NodeAnnouncement:
update.NodeUpdates = append(update.NodeUpdates, &NetworkNodeUpdate{
Addresses: []net.Addr{m.Address},
IdentityKey: m.NodeID,
Alias: m.Alias.String(),
})
return nil
// We ignore initial channel announcements as we'll only send out
// updates once the individual edges themselves have been updated.
case *lnwire.ChannelAnnouncement:
return nil
// Any new ChannelUpdateAnnouncements will generate a corresponding
// ChannelEdgeUpdate notification.
case *lnwire.ChannelUpdateAnnouncement:
// We'll need to fetch the edge's information from the database
// in order to get the information concerning which nodes are
// being connected.
chanID := m.ChannelID.ToUint64()
edgeInfo, _, _, err := graph.FetchChannelEdgesByID(chanID)
if err != nil {
return err
}
// If the flag is one, then the advertising node is actually
// the second node.
sourceNode := edgeInfo.NodeKey1
connectingNode := edgeInfo.NodeKey2
if m.Flags == 1 {
sourceNode = edgeInfo.NodeKey2
connectingNode = edgeInfo.NodeKey1
}
edgeUpdate := &ChannelEdgeUpdate{
ChanID: chanID,
TimeLockDelta: m.TimeLockDelta,
Capacity: edgeInfo.Capacity,
MinHTLC: btcutil.Amount(m.HtlcMinimumMsat),
BaseFee: btcutil.Amount(m.FeeBaseMsat),
FeeRate: btcutil.Amount(m.FeeProportionalMillionths),
AdvertisingNode: sourceNode,
ConnectingNode: connectingNode,
}
// TODO(roasbeef): add bit to toggle
update.ChannelEdgeUpdates = append(update.ChannelEdgeUpdates,
edgeUpdate)
return nil
default:
return fmt.Errorf("Unable to add to topology change, "+
"unknown message type %T", msg)
}
}

@ -95,7 +95,10 @@ type Config struct {
// duty of the router to sync up newly connected peers with the latest state of
// the channel graph.
type ChannelRouter struct {
sync.RWMutex
ntfnClientCounter uint64
started uint32
stopped uint32
// cfg is a copy of the configuration struct that the ChannelRouter was
// initialized with.
@ -109,7 +112,7 @@ type ChannelRouter struct {
// TODO(roasbeef): make LRU, invalidate upon new block connect
shortestPathCache map[[33]byte][]*Route
nodeCache map[[33]byte]*channeldb.LightningNode
edgeCache map[wire.OutPoint]*channeldb.ChannelEdge
edgeCache map[wire.OutPoint]*channeldb.ChannelEdgePolicy
// newBlocks is a channel in which new blocks connected to the end of
// the main chain are sent over.
@ -125,23 +128,35 @@ type ChannelRouter struct {
// prematureAnnouncements maps a blockheight to a set of announcements
// which are "premature" from our PoV. An announcement is premature if
// it claims to be anchored in a block which is beyond the current
// mainchain tip as we know it. Premature announcements will be
// processed once the chain tip as we know it extends to/past the
// premature height.
// it claims to be anchored in a block which is beyond the current main
// chain tip as we know it. Premature announcements will be processed
// once the chain tip as we know it extends to/past the premature
// height.
//
// TODO(roasbeef): limit premature announcements to N
prematureAnnouncements map[uint32][]lnwire.Message
// topologyClients maps a client's unique notification ID to a
// topologyClient client that contains its notification dispatch
// channel.
topologyClients map[uint64]topologyClient
// ntfnClientUpdates is a channel that's used to send new updates to
// topology notification clients to the ChannelRouter. Updates either
// add a new notification client, or cancel notifications for an
// existing client.
ntfnClientUpdates chan *topologyClientUpdate
// bestHeight is the height of the block at the tip of the main chain
// as we know it.
bestHeight uint32
fakeSig *btcec.Signature
started uint32
stopped uint32
quit chan struct{}
wg sync.WaitGroup
sync.RWMutex
quit chan struct{}
wg sync.WaitGroup
}
// New creates a new instance of the ChannelRouter with the specified
@ -176,7 +191,9 @@ func New(cfg Config) (*ChannelRouter, error) {
networkMsgs: make(chan *routingMsg),
syncRequests: make(chan *syncRequest),
prematureAnnouncements: make(map[uint32][]lnwire.Message),
quit: make(chan struct{}),
topologyClients: make(map[uint64]topologyClient),
ntfnClientUpdates: make(chan *topologyClientUpdate),
quit: make(chan struct{}),
}, nil
}
@ -307,12 +324,13 @@ func (r *ChannelRouter) syncGraphWithChain() error {
// With the spent outputs gathered, attempt to prune the
// channel graph, also passing in the hash+height of the block
// being pruned so the prune tip can be updated.
numClosed, err := r.cfg.Graph.PruneGraph(spentOutputs, nextHash,
closedChans, err := r.cfg.Graph.PruneGraph(spentOutputs, nextHash,
nextHeight)
if err != nil {
return err
}
numClosed := uint32(len(closedChans))
log.Infof("Block %v (height=%v) closed %v channels",
nextHash, nextHeight, numClosed)
@ -359,7 +377,22 @@ func (r *ChannelRouter) networkHandler() {
// announcement batch to be broadcast once the trickle
// timer ticks gain.
if accepted {
// TODO(roasbeef): exclude peer that sent
announcementBatch = append(announcementBatch, netMsg.msg)
// Send off a new notification for the newly
// accepted announcement.
topChange := &TopologyChange{}
err := addToTopologyChange(r.cfg.Graph, topChange,
netMsg.msg)
if err != nil {
log.Errorf("unable to update topology "+
"change notification: %v", err)
}
if !topChange.isEmpty() {
r.notifyTopologyChange(topChange)
}
}
// TODO(roasbeef): remove all unconnected vertexes
@ -389,13 +422,30 @@ func (r *ChannelRouter) networkHandler() {
"height %v", len(prematureAnns), blockHeight)
}
topChange := &TopologyChange{}
for _, ann := range prematureAnns {
if ok := r.processNetworkAnnouncement(ann); ok {
announcementBatch = append(announcementBatch, ann)
// As the announcement was accepted,
// accumulate it to the running set of
// announcements for this block.
err := addToTopologyChange(r.cfg.Graph,
topChange, ann)
if err != nil {
log.Errorf("unable to update topology "+
"change notification: %v", err)
}
}
}
delete(r.prematureAnnouncements, blockHeight)
// If the pending notification generated above isn't
// empty, then send it out to all registered clients.
if !topChange.isEmpty() {
r.notifyTopologyChange(topChange)
}
log.Infof("Pruning channel graph using block %v (height=%v)",
newBlock.Hash, blockHeight)
@ -421,7 +471,7 @@ func (r *ChannelRouter) networkHandler() {
// the channel graph, also passing in the hash+height
// of the block being pruned so the prune tip can be
// updated.
numClosed, err := r.cfg.Graph.PruneGraph(spentOutputs,
chansClosed, err := r.cfg.Graph.PruneGraph(spentOutputs,
newBlock.Hash, blockHeight)
if err != nil {
log.Errorf("unable to prune routing table: %v", err)
@ -429,7 +479,18 @@ func (r *ChannelRouter) networkHandler() {
}
log.Infof("Block %v (height=%v) closed %v channels",
newBlock.Hash, blockHeight, numClosed)
newBlock.Hash, blockHeight, len(chansClosed))
if len(chansClosed) == 0 {
continue
}
// Notify all currently registered clients of the newly
// closed channels.
closeSummaries := createCloseSummaries(blockHeight, chansClosed...)
r.notifyTopologyChange(&TopologyChange{
ClosedChannels: closeSummaries,
})
// The retransmission timer has ticked which indicates that we
// should broadcast our personal channel to the network. This
@ -439,7 +500,9 @@ func (r *ChannelRouter) networkHandler() {
var selfChans []lnwire.Message
selfPub := r.selfNode.PubKey.SerializeCompressed()
err := r.selfNode.ForEachChannel(nil, func(c *channeldb.ChannelEdge) error {
err := r.selfNode.ForEachChannel(nil, func(_ *channeldb.ChannelEdgeInfo,
c *channeldb.ChannelEdgePolicy) error {
chanNodePub := c.Node.PubKey.SerializeCompressed()
// Compare our public key with that of the
@ -456,9 +519,9 @@ func (r *ChannelRouter) networkHandler() {
ChannelID: lnwire.NewChanIDFromInt(c.ChannelID),
Timestamp: uint32(c.LastUpdate.Unix()),
Flags: flags,
Expiry: c.Expiry,
HtlcMinimumMstat: uint32(c.MinHTLC),
FeeBaseMstat: uint32(c.FeeBaseMSat),
TimeLockDelta: c.TimeLockDelta,
HtlcMinimumMsat: uint32(c.MinHTLC),
FeeBaseMsat: uint32(c.FeeBaseMSat),
FeeProportionalMillionths: uint32(c.FeeProportionalMillionths),
})
return nil
@ -469,7 +532,11 @@ func (r *ChannelRouter) networkHandler() {
continue
}
log.Infof("Retransmitting %v outgoing channels",
if len(selfChans) == 0 {
continue
}
log.Debugf("Retransmitting %v outgoing channels",
len(selfChans))
// With all the wire messages properly crafted, we'll
@ -520,6 +587,27 @@ func (r *ChannelRouter) networkHandler() {
nodePub, err)
}
// A new notification client update has arrived. We're either
// gaining a new client, or cancelling notifications for an
// existing client.
case ntfnUpdate := <-r.ntfnClientUpdates:
clientID := ntfnUpdate.clientID
if ntfnUpdate.cancel {
if client, ok := r.topologyClients[ntfnUpdate.clientID]; ok {
delete(r.topologyClients, clientID)
close(client.ntfnChan)
close(client.exit)
}
continue
}
r.topologyClients[ntfnUpdate.clientID] = topologyClient{
ntfnChan: ntfnUpdate.ntfnChan,
exit: make(chan struct{}),
}
// The router has been signalled to exit, to we exit our main
// loop so the wait group can be decremented.
case <-r.quit:
@ -630,18 +718,32 @@ func (r *ChannelRouter) processNetworkAnnouncement(msg lnwire.Message) bool {
// Now that we have the funding outpoint of the channel, ensure
// that it hasn't yet been spent. If so, then this channel has
// been closed so we'll ignore it.
if _, err := r.cfg.Chain.GetUtxo(&fundingPoint.Hash,
fundingPoint.Index); err != nil {
chanUtxo, err := r.cfg.Chain.GetUtxo(&fundingPoint.Hash,
fundingPoint.Index)
if err != nil {
log.Errorf("unable to fetch utxo for chan_id=%v: %v",
channelID, err)
return false
}
// TODO(roasbeef): also add capacity here two instead of on the
// directed edges.
err = r.cfg.Graph.AddChannelEdge(msg.FirstNodeID,
msg.SecondNodeID, fundingPoint, channelID)
if err != nil {
edge := &channeldb.ChannelEdgeInfo{
ChannelID: channelID,
NodeKey1: msg.FirstNodeID,
NodeKey2: msg.SecondNodeID,
BitcoinKey1: msg.FirstBitcoinKey,
BitcoinKey2: msg.SecondBitcoinKey,
AuthProof: &channeldb.ChannelAuthProof{
NodeSig1: msg.FirstNodeSig,
NodeSig2: msg.SecondNodeSig,
BitcoinSig1: msg.FirstBitcoinSig,
BitcoinSig2: msg.SecondBitcoinSig,
},
ChannelPoint: *fundingPoint,
// TODO(roasbeef): this is a hack, needs to be removed
// after commitment fees are dynamic.
Capacity: btcutil.Amount(chanUtxo.Value) - btcutil.Amount(5000),
}
if err := r.cfg.Graph.AddChannelEdge(edge); err != nil {
log.Errorf("unable to add channel: %v", err)
return false
}
@ -688,8 +790,8 @@ func (r *ChannelRouter) processNetworkAnnouncement(msg lnwire.Message) bool {
updateTimestamp := time.Unix(int64(msg.Timestamp), 0)
switch msg.Flags {
// A flag set of 0 indicates this is an announcement for
// the "first" node in the channel.
// A flag set of 0 indicates this is an announcement for the
// "first" node in the channel.
case 0:
if edge1Timestamp.After(updateTimestamp) ||
edge1Timestamp.Equal(updateTimestamp) {
@ -700,8 +802,8 @@ func (r *ChannelRouter) processNetworkAnnouncement(msg lnwire.Message) bool {
return false
}
// Similarly, a flag set of 1 indicates this is an
// announcement for the "second" node in the channel.
// Similarly, a flag set of 1 indicates this is an announcement
// for the "second" node in the channel.
case 1:
if edge2Timestamp.After(updateTimestamp) ||
edge2Timestamp.Equal(updateTimestamp) {
@ -720,31 +822,24 @@ func (r *ChannelRouter) processNetworkAnnouncement(msg lnwire.Message) bool {
log.Errorf("unable to fetch chan point for chan_id=%v: %v", chanID, err)
return false
}
utxo, err := r.cfg.Chain.GetUtxo(&chanPoint.Hash,
chanPoint.Index)
if err != nil {
if _, err := r.cfg.Chain.GetUtxo(&chanPoint.Hash,
chanPoint.Index); err != nil {
log.Errorf("unable to fetch utxo for chan_id=%v: %v",
chanID, err)
return false
}
// TODO(roasbeef): should be msat here
chanUpdate := &channeldb.ChannelEdge{
chanUpdate := &channeldb.ChannelEdgePolicy{
ChannelID: chanID,
ChannelPoint: *chanPoint,
LastUpdate: updateTimestamp,
Flags: msg.Flags,
Expiry: msg.Expiry,
MinHTLC: btcutil.Amount(msg.HtlcMinimumMstat),
FeeBaseMSat: btcutil.Amount(msg.FeeBaseMstat),
TimeLockDelta: msg.TimeLockDelta,
MinHTLC: btcutil.Amount(msg.HtlcMinimumMsat),
FeeBaseMSat: btcutil.Amount(msg.FeeBaseMsat),
FeeProportionalMillionths: btcutil.Amount(msg.FeeProportionalMillionths),
// TODO(roasbeef): this is a hack, needs to be removed
// after commitment fees are dynamic.
Capacity: btcutil.Amount(utxo.Value) - btcutil.Amount(5000),
}
err = r.cfg.Graph.UpdateEdgeInfo(chanUpdate)
if err != nil {
if err = r.cfg.Graph.UpdateEdgePolicy(chanUpdate); err != nil {
log.Errorf("unable to add channel: %v", err)
return false
}
@ -819,52 +914,26 @@ func (r *ChannelRouter) syncChannelGraph(syncReq *syncRequest) error {
// With the vertexes gathered, we'll no retrieve the initial
// announcement, as well as the latest channel update announcement for
// both of the directed edges that make up the channel.
// TODO(roasbeef): multi-sig keys should also be stored in DB
var numEdges uint32
if err := r.cfg.Graph.ForEachChannel(func(e1, e2 *channeldb.ChannelEdge) error {
// First we'll need to obtain the channel ID for the channel
// advertisement. As an edge may not be advertised, we'll grab
// the channel ID from the edge that was.
var chanID lnwire.ChannelID
switch {
case e1 != nil:
chanID = lnwire.NewChanIDFromInt(e1.ChannelID)
case e2 != nil:
chanID = lnwire.NewChanIDFromInt(e2.ChannelID)
case e1 == nil && e2 == nil:
return nil
default:
chanID = lnwire.NewChanIDFromInt(e1.ChannelID)
}
if err := r.cfg.Graph.ForEachChannel(func(chanInfo *channeldb.ChannelEdgeInfo,
e1, e2 *channeldb.ChannelEdgePolicy) error {
chanID := lnwire.NewChanIDFromInt(chanInfo.ChannelID)
// First, using the parameters of the channel, along with the
// channel authentication proof, we'll create re-create the
// original authenticated channel announcement.
authProof := chanInfo.AuthProof
chanAnn := &lnwire.ChannelAnnouncement{
FirstNodeSig: r.fakeSig,
SecondNodeSig: r.fakeSig,
FirstNodeSig: authProof.NodeSig1,
SecondNodeSig: authProof.NodeSig2,
ChannelID: chanID,
FirstBitcoinSig: r.fakeSig,
SecondBitcoinSig: r.fakeSig,
}
// If the edge was advertised, then we'll use the node's
// identity within the announcement we send to the sync node.
// Otherwise, we'll fill in a dummy key.
//
// TODO(roasbeef): both else clauses need to be removed
// once we fully validate, andrew's PR will reconcile
// this
if e1 != nil {
chanAnn.FirstNodeID = e1.Node.PubKey
chanAnn.FirstBitcoinKey = e1.Node.PubKey
} else {
chanAnn.FirstNodeID = e2.Node.PubKey
chanAnn.FirstBitcoinKey = e2.Node.PubKey
}
if e2 != nil {
chanAnn.SecondNodeID = e2.Node.PubKey
chanAnn.SecondBitcoinKey = e2.Node.PubKey
} else {
chanAnn.SecondNodeID = e1.Node.PubKey
chanAnn.SecondBitcoinKey = e1.Node.PubKey
FirstBitcoinSig: authProof.BitcoinSig1,
SecondBitcoinSig: authProof.BitcoinSig2,
FirstNodeID: chanInfo.NodeKey1,
SecondNodeID: chanInfo.NodeKey2,
FirstBitcoinKey: chanInfo.BitcoinKey1,
SecondBitcoinKey: chanInfo.BitcoinKey2,
}
// We'll unconditionally queue the channel's existence proof as
@ -881,9 +950,9 @@ func (r *ChannelRouter) syncChannelGraph(syncReq *syncRequest) error {
ChannelID: chanID,
Timestamp: uint32(e1.LastUpdate.Unix()),
Flags: 0,
Expiry: e1.Expiry,
HtlcMinimumMstat: uint32(e1.MinHTLC),
FeeBaseMstat: uint32(e1.FeeBaseMSat),
TimeLockDelta: e1.TimeLockDelta,
HtlcMinimumMsat: uint32(e1.MinHTLC),
FeeBaseMsat: uint32(e1.FeeBaseMSat),
FeeProportionalMillionths: uint32(e1.FeeProportionalMillionths),
})
}
@ -893,9 +962,9 @@ func (r *ChannelRouter) syncChannelGraph(syncReq *syncRequest) error {
ChannelID: chanID,
Timestamp: uint32(e2.LastUpdate.Unix()),
Flags: 1,
Expiry: e2.Expiry,
HtlcMinimumMstat: uint32(e2.MinHTLC),
FeeBaseMstat: uint32(e2.FeeBaseMSat),
TimeLockDelta: e2.TimeLockDelta,
HtlcMinimumMsat: uint32(e2.MinHTLC),
FeeBaseMsat: uint32(e2.FeeBaseMSat),
FeeProportionalMillionths: uint32(e2.FeeProportionalMillionths),
})
}
@ -1088,7 +1157,7 @@ type LightningPayment struct {
// Target is the node in which the payment should be routed towards.
Target *btcec.PublicKey
// Amount is the value of the payment to send throuhg the network in
// Amount is the value of the payment to send through the network in
// satoshis.
// TODO(roasbeef): this should be milli satoshis
Amount btcutil.Amount
@ -1097,7 +1166,7 @@ type LightningPayment struct {
// the first hop.
PaymentHash [32]byte
// TODO(roasbeef): add message?
// TODO(roasbeef): add e2e message?
}
// SendPayment attempts to send a payment as described within the passed
@ -1148,23 +1217,3 @@ func (r *ChannelRouter) SendPayment(payment *LightningPayment) ([32]byte, *Route
return preImage, route, nil
}
// TopologyClient...
// TODO(roasbeef): put in discovery package?
type TopologyClient struct {
}
// TopologyChange...
type TopologyChange struct {
NewNodes []*channeldb.LinkNode
NewChannels []*channeldb.ChannelEdge
}
// notifyTopologyChange...
func (r *ChannelRouter) notifyTopologyChange() {
}
// SubscribeTopology....
func (r *ChannelRouter) SubscribeTopology() (*TopologyClient, error) {
return nil, nil
}