multi: use models.ChannelEdgeInfo throughout

This commit is contained in:
Elle Mouton 2024-08-21 16:32:43 +02:00
parent 74d03e7658
commit 84abc0a311
No known key found for this signature in database
GPG Key ID: D7D916376026F177
28 changed files with 898 additions and 543 deletions

@ -90,7 +90,7 @@ func (d *dbNode) Addrs() []net.Addr {
// NOTE: Part of the autopilot.Node interface.
func (d *dbNode) ForEachChannel(cb func(ChannelEdge) error) error {
return d.db.ForEachNodeChannelTx(d.tx, d.node.PubKeyBytes,
func(tx kvdb.RTx, ei *models.ChannelEdgeInfo1, ep,
func(tx kvdb.RTx, ei models.ChannelEdgeInfo, ep,
_ *models.ChannelEdgePolicy1) error {
// Skip channels for which no outgoing edge policy is
@ -116,7 +116,7 @@ func (d *dbNode) ForEachChannel(cb func(ChannelEdge) error) error {
ChanID: lnwire.NewShortChanIDFromInt(
ep.ChannelID,
),
Capacity: ei.Capacity,
Capacity: ei.GetCapacity(),
Peer: &dbNode{
tx: tx,
db: d.db,

@ -239,7 +239,7 @@ func NewChannelGraph(db kvdb.Backend, rejectCacheSize, chanCacheSize int,
return nil, err
}
err = g.ForEachChannel(func(info *models.ChannelEdgeInfo1,
err = g.ForEachChannel(func(info models.ChannelEdgeInfo,
policy1, policy2 *models.ChannelEdgePolicy1) error {
g.graphCache.AddChannel(info, policy1, policy2)
@ -425,7 +425,7 @@ func (c *ChannelGraph) NewPathFindTx() (kvdb.RTx, error) {
// NOTE: If an edge can't be found, or wasn't advertised, then a nil pointer
// for that particular channel edge routing policy will be passed into the
// callback.
func (c *ChannelGraph) ForEachChannel(cb func(*models.ChannelEdgeInfo1,
func (c *ChannelGraph) ForEachChannel(cb func(models.ChannelEdgeInfo,
*models.ChannelEdgePolicy1, *models.ChannelEdgePolicy1) error) error {
return c.db.View(func(tx kvdb.RTx) error {
@ -453,28 +453,21 @@ func (c *ChannelGraph) ForEachChannel(cb func(*models.ChannelEdgeInfo1,
copy(chanID[:], k)
edgeInfoReader := bytes.NewReader(edgeInfoBytes)
info, err := deserializeChanEdgeInfo(edgeInfoReader)
edgeInfo, err := deserializeChanEdgeInfo(edgeInfoReader)
if err != nil {
return err
}
policy1 := channelMap[channelMapKey{
nodeKey: info.Node1Bytes(),
nodeKey: edgeInfo.Node1Bytes(),
chanID: chanID,
}]
policy2 := channelMap[channelMapKey{
nodeKey: info.Node2Bytes(),
nodeKey: edgeInfo.Node2Bytes(),
chanID: chanID,
}]
edgeInfo, ok := info.(*models.ChannelEdgeInfo1)
if !ok {
return fmt.Errorf("expected "+
"*models.ChannelEdgeInfo1, got %T",
edgeInfo)
}
return cb(edgeInfo, policy1, policy2)
})
}, func() {})
@ -502,7 +495,7 @@ func (c *ChannelGraph) ForEachNodeDirectedChannel(tx kvdb.RTx,
return err
}
dbCallback := func(tx kvdb.RTx, e *models.ChannelEdgeInfo1, p1,
dbCallback := func(tx kvdb.RTx, e models.ChannelEdgeInfo, p1,
p2 *models.ChannelEdgePolicy1) error {
var cachedInPolicy *models.CachedEdgePolicy
@ -523,21 +516,22 @@ func (c *ChannelGraph) ForEachNodeDirectedChannel(tx kvdb.RTx,
}
directedChannel := &DirectedChannel{
ChannelID: e.ChannelID,
IsNode1: node == e.NodeKey1Bytes,
OtherNode: e.NodeKey2Bytes,
Capacity: e.Capacity,
ChannelID: e.GetChanID(),
IsNode1: node == e.Node1Bytes(),
OtherNode: e.Node2Bytes(),
Capacity: e.GetCapacity(),
OutPolicySet: p1 != nil,
InPolicy: cachedInPolicy,
InboundFee: inboundFee,
}
if node == e.NodeKey2Bytes {
directedChannel.OtherNode = e.NodeKey1Bytes
if node == e.Node2Bytes() {
directedChannel.OtherNode = e.Node1Bytes()
}
return cb(directedChannel)
}
return nodeTraversal(tx, node[:], c.db, dbCallback)
}
@ -588,7 +582,7 @@ func (c *ChannelGraph) ForEachNodeCached(cb func(node route.Vertex,
channels := make(map[uint64]*DirectedChannel)
err := c.ForEachNodeChannelTx(tx, node.PubKeyBytes,
func(tx kvdb.RTx, e *models.ChannelEdgeInfo1,
func(tx kvdb.RTx, e models.ChannelEdgeInfo,
p1 *models.ChannelEdgePolicy1,
p2 *models.ChannelEdgePolicy1) error {
@ -613,21 +607,21 @@ func (c *ChannelGraph) ForEachNodeCached(cb func(node route.Vertex,
}
directedChannel := &DirectedChannel{
ChannelID: e.ChannelID,
ChannelID: e.GetChanID(),
IsNode1: node.PubKeyBytes ==
e.NodeKey1Bytes,
OtherNode: e.NodeKey2Bytes,
Capacity: e.Capacity,
e.Node1Bytes(),
OtherNode: e.Node2Bytes(),
Capacity: e.GetCapacity(),
OutPolicySet: p1 != nil,
InPolicy: cachedInPolicy,
}
if node.PubKeyBytes == e.NodeKey2Bytes {
if node.PubKeyBytes == e.Node2Bytes() {
directedChannel.OtherNode =
e.NodeKey1Bytes
e.Node1Bytes()
}
channels[e.ChannelID] = directedChannel
channels[e.GetChanID()] = directedChannel
return nil
})
@ -1001,7 +995,7 @@ func (c *ChannelGraph) deleteLightningNode(nodes kvdb.RwBucket,
// involved in creation of the channel, and the set of features that the channel
// supports. The chanPoint and chanID are used to uniquely identify the edge
// globally within the database.
func (c *ChannelGraph) AddChannelEdge(edge *models.ChannelEdgeInfo1,
func (c *ChannelGraph) AddChannelEdge(edge models.ChannelEdgeInfo,
op ...batch.SchedulerOption) error {
var alreadyExists bool
@ -1028,8 +1022,8 @@ func (c *ChannelGraph) AddChannelEdge(edge *models.ChannelEdgeInfo1,
case alreadyExists:
return ErrEdgeAlreadyExist
default:
c.rejectCache.remove(edge.ChannelID)
c.chanCache.remove(edge.ChannelID)
c.rejectCache.remove(edge.GetChanID())
c.chanCache.remove(edge.GetChanID())
return nil
}
},
@ -1045,11 +1039,11 @@ func (c *ChannelGraph) AddChannelEdge(edge *models.ChannelEdgeInfo1,
// addChannelEdge is the private form of AddChannelEdge that allows callers to
// utilize an existing db transaction.
func (c *ChannelGraph) addChannelEdge(tx kvdb.RwTx,
edge *models.ChannelEdgeInfo1) error {
edge models.ChannelEdgeInfo) error {
// Construct the channel's primary key which is the 8-byte channel ID.
var chanKey [8]byte
binary.BigEndian.PutUint64(chanKey[:], edge.ChannelID)
binary.BigEndian.PutUint64(chanKey[:], edge.GetChanID())
nodes, err := tx.CreateTopLevelBucket(nodeBucket)
if err != nil {
@ -1078,37 +1072,42 @@ func (c *ChannelGraph) addChannelEdge(tx kvdb.RwTx,
c.graphCache.AddChannel(edge, nil, nil)
}
var (
node1Bytes = edge.Node1Bytes()
node2Bytes = edge.Node2Bytes()
)
// Before we insert the channel into the database, we'll ensure that
// both nodes already exist in the channel graph. If either node
// doesn't, then we'll insert a "shell" node that just includes its
// public key, so subsequent validation and queries can work properly.
_, node1Err := fetchLightningNode(nodes, edge.NodeKey1Bytes[:])
_, node1Err := fetchLightningNode(nodes, node1Bytes[:])
switch {
case node1Err == ErrGraphNodeNotFound:
node1Shell := LightningNode{
PubKeyBytes: edge.NodeKey1Bytes,
PubKeyBytes: node1Bytes,
HaveNodeAnnouncement: false,
}
err := addLightningNode(tx, &node1Shell)
if err != nil {
return fmt.Errorf("unable to create shell node "+
"for: %x", edge.NodeKey1Bytes)
"for: %x", node1Bytes)
}
case node1Err != nil:
return err
}
_, node2Err := fetchLightningNode(nodes, edge.NodeKey2Bytes[:])
_, node2Err := fetchLightningNode(nodes, node2Bytes[:])
switch {
case node2Err == ErrGraphNodeNotFound:
node2Shell := LightningNode{
PubKeyBytes: edge.NodeKey2Bytes,
PubKeyBytes: node2Bytes,
HaveNodeAnnouncement: false,
}
err := addLightningNode(tx, &node2Shell)
if err != nil {
return fmt.Errorf("unable to create shell node "+
"for: %x", edge.NodeKey2Bytes)
"for: %x", node2Bytes)
}
case node2Err != nil:
return err
@ -1124,11 +1123,11 @@ func (c *ChannelGraph) addChannelEdge(tx kvdb.RwTx,
// Mark edge policies for both sides as unknown. This is to enable
// efficient incoming channel lookup for a node.
keys := []*[33]byte{
&edge.NodeKey1Bytes,
&edge.NodeKey2Bytes,
&node1Bytes,
&node2Bytes,
}
for _, key := range keys {
err := putChanEdgePolicyUnknown(edges, edge.ChannelID, key[:])
err := putChanEdgePolicyUnknown(edges, edge.GetChanID(), key[:])
if err != nil {
return err
}
@ -1137,7 +1136,8 @@ func (c *ChannelGraph) addChannelEdge(tx kvdb.RwTx,
// Finally we add it to the channel index which maps channel points
// (outpoints) to the shorter channel ID's.
var b bytes.Buffer
if err := writeOutpoint(&b, &edge.ChannelPoint); err != nil {
chanPoint := edge.GetChanPoint()
if err := writeOutpoint(&b, &chanPoint); err != nil {
return err
}
return chanIndex.Put(b.Bytes(), chanKey[:])
@ -1257,10 +1257,10 @@ func (c *ChannelGraph) HasChannelEdge(
// In order to maintain this constraints, we return an error in the scenario
// that an edge info hasn't yet been created yet, but someone attempts to update
// it.
func (c *ChannelGraph) UpdateChannelEdge(edge *models.ChannelEdgeInfo1) error {
func (c *ChannelGraph) UpdateChannelEdge(edge models.ChannelEdgeInfo) error {
// Construct the channel's primary key which is the 8-byte channel ID.
var chanKey [8]byte
binary.BigEndian.PutUint64(chanKey[:], edge.ChannelID)
binary.BigEndian.PutUint64(chanKey[:], edge.GetChanID())
return kvdb.Update(c.db, func(tx kvdb.RwTx) error {
edges := tx.ReadWriteBucket(edgeBucket)
@ -1303,12 +1303,12 @@ const (
// the target block are returned if the function succeeds without error.
func (c *ChannelGraph) PruneGraph(spentOutputs []*wire.OutPoint,
blockHash *chainhash.Hash, blockHeight uint32) (
[]*models.ChannelEdgeInfo1, error) {
[]models.ChannelEdgeInfo, error) {
c.cacheMu.Lock()
defer c.cacheMu.Unlock()
var chansClosed []*models.ChannelEdgeInfo1
var chansClosed []models.ChannelEdgeInfo
err := kvdb.Update(c.db, func(tx kvdb.RwTx) error {
// First grab the edges bucket which houses the information
@ -1318,12 +1318,17 @@ func (c *ChannelGraph) PruneGraph(spentOutputs []*wire.OutPoint,
return err
}
// Next grab the two edge indexes which will also need to be updated.
edgeIndex, err := edges.CreateBucketIfNotExists(edgeIndexBucket)
// Next grab the two edge indexes which will also need to be
// updated.
edgeIndex, err := edges.CreateBucketIfNotExists(
edgeIndexBucket,
)
if err != nil {
return err
}
chanIndex, err := edges.CreateBucketIfNotExists(channelPointBucket)
chanIndex, err := edges.CreateBucketIfNotExists(
channelPointBucket,
)
if err != nil {
return err
}
@ -1344,7 +1349,8 @@ func (c *ChannelGraph) PruneGraph(spentOutputs []*wire.OutPoint,
// if NOT if filter
var opBytes bytes.Buffer
if err := writeOutpoint(&opBytes, chanPoint); err != nil {
err := writeOutpoint(&opBytes, chanPoint)
if err != nil {
return err
}
@ -1375,14 +1381,7 @@ func (c *ChannelGraph) PruneGraph(spentOutputs []*wire.OutPoint,
return err
}
info, ok := edgeInfo.(*models.ChannelEdgeInfo1)
if !ok {
return fmt.Errorf("expected "+
"*models.ChannelEdgeInfo1, got %T",
edgeInfo)
}
chansClosed = append(chansClosed, info)
chansClosed = append(chansClosed, edgeInfo)
}
metaBucket, err := tx.CreateTopLevelBucket(graphMetaBucket)
@ -1390,7 +1389,9 @@ func (c *ChannelGraph) PruneGraph(spentOutputs []*wire.OutPoint,
return err
}
pruneBucket, err := metaBucket.CreateBucketIfNotExists(pruneLogBucket)
pruneBucket, err := metaBucket.CreateBucketIfNotExists(
pruneLogBucket,
)
if err != nil {
return err
}
@ -1421,8 +1422,8 @@ func (c *ChannelGraph) PruneGraph(spentOutputs []*wire.OutPoint,
}
for _, channel := range chansClosed {
c.rejectCache.remove(channel.ChannelID)
c.chanCache.remove(channel.ChannelID)
c.rejectCache.remove(channel.GetChanID())
c.chanCache.remove(channel.GetChanID())
}
if c.graphCache != nil {
@ -1570,7 +1571,7 @@ func (c *ChannelGraph) pruneGraphNodes(nodes kvdb.RwBucket,
// Channels that were removed from the graph resulting from the
// disconnected block are returned.
func (c *ChannelGraph) DisconnectBlockAtHeight(height uint32) (
[]*models.ChannelEdgeInfo1, error) {
[]models.ChannelEdgeInfo, error) {
// Every channel having a ShortChannelID starting at 'height'
// will no longer be confirmed.
@ -1592,7 +1593,7 @@ func (c *ChannelGraph) DisconnectBlockAtHeight(height uint32) (
defer c.cacheMu.Unlock()
// Keep track of the channels that are removed from the graph.
var removedChans []*models.ChannelEdgeInfo1
var removedChans []models.ChannelEdgeInfo
if err := kvdb.Update(c.db, func(tx kvdb.RwTx) error {
edges, err := tx.CreateTopLevelBucket(edgeBucket)
@ -1631,15 +1632,7 @@ func (c *ChannelGraph) DisconnectBlockAtHeight(height uint32) (
}
keys = append(keys, k)
info, ok := edgeInfo.(*models.ChannelEdgeInfo1)
if !ok {
return fmt.Errorf("expected "+
"*models.ChannelEdgeInfo1, got %T",
edgeInfo)
}
keys = append(keys, k)
removedChans = append(removedChans, info)
removedChans = append(removedChans, edgeInfo)
}
for _, k := range keys {
@ -1694,8 +1687,8 @@ func (c *ChannelGraph) DisconnectBlockAtHeight(height uint32) (
}
for _, channel := range removedChans {
c.rejectCache.remove(channel.ChannelID)
c.chanCache.remove(channel.ChannelID)
c.rejectCache.remove(channel.GetChanID())
c.chanCache.remove(channel.GetChanID())
}
return removedChans, nil
@ -1903,7 +1896,7 @@ func (c *ChannelGraph) HighestChanID() (uint64, error) {
// edge as well as each of the known advertised edge policies.
type ChannelEdge struct {
// Info contains all the static information describing the channel.
Info *models.ChannelEdgeInfo1
Info models.ChannelEdgeInfo
// Policy1 points to the "first" edge policy of the channel containing
// the dynamic information required to properly route through the edge.
@ -1995,20 +1988,13 @@ func (c *ChannelGraph) ChanUpdatesInHorizon(startTime,
}
// First, we'll fetch the static edge information.
info, err := fetchChanEdgeInfo(edgeIndex, chanID)
edgeInfo, err := fetchChanEdgeInfo(edgeIndex, chanID)
if err != nil {
chanID := byteOrder.Uint64(chanID)
return fmt.Errorf("unable to fetch info for "+
"edge with chan_id=%v: %v", chanID, err)
}
edgeInfo, ok := info.(*models.ChannelEdgeInfo1)
if !ok {
return fmt.Errorf("expected "+
"*models.ChannelEdgeInfo1, got %T",
edgeInfo)
}
// With the static information obtained, we'll now
// fetch the dynamic policy info.
edge1, edge2, err := fetchChanEdgePolicies(
@ -2021,16 +2007,17 @@ func (c *ChannelGraph) ChanUpdatesInHorizon(startTime,
err)
}
node1, err := fetchLightningNode(
nodes, edgeInfo.NodeKey1Bytes[:],
var (
node1Bytes = edgeInfo.Node1Bytes()
node2Bytes = edgeInfo.Node2Bytes()
)
node1, err := fetchLightningNode(nodes, node1Bytes[:])
if err != nil {
return err
}
node2, err := fetchLightningNode(
nodes, edgeInfo.NodeKey2Bytes[:],
)
node2, err := fetchLightningNode(nodes, node2Bytes[:])
if err != nil {
return err
}
@ -2511,7 +2498,7 @@ func (c *ChannelGraph) fetchChanInfos(tx kvdb.RTx, chanIDs []uint64) (
// First, we'll fetch the static edge information. If
// the edge is unknown, we will skip the edge and
// continue gathering all known edges.
info, err := fetchChanEdgeInfo(
edgeInfo, err := fetchChanEdgeInfo(
edgeIndex, cidBytes[:],
)
switch {
@ -2530,23 +2517,17 @@ func (c *ChannelGraph) fetchChanInfos(tx kvdb.RTx, chanIDs []uint64) (
return err
}
edgeInfo, ok := info.(*models.ChannelEdgeInfo1)
if !ok {
return fmt.Errorf("expected "+
"*models.ChannelEdgeInfo1, got %T",
info)
}
node1, err := fetchLightningNode(
nodes, edgeInfo.NodeKey1Bytes[:],
var (
node1Bytes = edgeInfo.Node1Bytes()
node2Bytes = edgeInfo.Node2Bytes()
)
node1, err := fetchLightningNode(nodes, node1Bytes[:])
if err != nil {
return err
}
node2, err := fetchLightningNode(
nodes, edgeInfo.NodeKey2Bytes[:],
)
node2, err := fetchLightningNode(nodes, node2Bytes[:])
if err != nil {
return err
}
@ -2630,21 +2611,20 @@ func (c *ChannelGraph) delChannelEdgeUnsafe(edges, edgeIndex, chanIndex,
zombieIndex kvdb.RwBucket, chanID []byte, isZombie,
strictZombie bool) error {
info, err := fetchChanEdgeInfo(edgeIndex, chanID)
edgeInfo, err := fetchChanEdgeInfo(edgeIndex, chanID)
if err != nil {
return err
}
edgeInfo, ok := info.(*models.ChannelEdgeInfo1)
if !ok {
return fmt.Errorf("expected *models.ChannelEdgeInfo1, got %T",
info)
}
var (
node1Bytes = edgeInfo.Node1Bytes()
node2Bytes = edgeInfo.Node2Bytes()
chanPoint = edgeInfo.GetChanPoint()
)
if c.graphCache != nil {
c.graphCache.RemoveChannel(
edgeInfo.NodeKey1Bytes, edgeInfo.NodeKey2Bytes,
edgeInfo.ChannelID,
node1Bytes, node2Bytes, edgeInfo.GetChanID(),
)
}
@ -2669,13 +2649,13 @@ func (c *ChannelGraph) delChannelEdgeUnsafe(edges, edgeIndex, chanIndex,
// With the latter half constructed, copy over the first public key to
// delete the edge in this direction, then the second to delete the
// edge in the opposite direction.
copy(edgeKey[:33], edgeInfo.NodeKey1Bytes[:])
copy(edgeKey[:33], node1Bytes[:])
if edges.Get(edgeKey[:]) != nil {
if err := edges.Delete(edgeKey[:]); err != nil {
return err
}
}
copy(edgeKey[:33], edgeInfo.NodeKey2Bytes[:])
copy(edgeKey[:33], node2Bytes[:])
if edges.Get(edgeKey[:]) != nil {
if err := edges.Delete(edgeKey[:]); err != nil {
return err
@ -2693,7 +2673,7 @@ func (c *ChannelGraph) delChannelEdgeUnsafe(edges, edgeIndex, chanIndex,
return err
}
var b bytes.Buffer
if err := writeOutpoint(&b, &edgeInfo.ChannelPoint); err != nil {
if err := writeOutpoint(&b, &chanPoint); err != nil {
return err
}
if err := chanIndex.Delete(b.Bytes()); err != nil {
@ -2707,9 +2687,15 @@ func (c *ChannelGraph) delChannelEdgeUnsafe(edges, edgeIndex, chanIndex,
return nil
}
nodeKey1, nodeKey2 := edgeInfo.NodeKey1Bytes, edgeInfo.NodeKey2Bytes
nodeKey1, nodeKey2 := node1Bytes, node2Bytes
if strictZombie {
nodeKey1, nodeKey2 = makeZombiePubkeys(edgeInfo, edge1, edge2)
var err error
nodeKey1, nodeKey2, err = makeZombiePubkeys(
edgeInfo, edge1, edge2,
)
if err != nil {
return err
}
}
return markEdgeZombie(
@ -2733,27 +2719,46 @@ func (c *ChannelGraph) delChannelEdgeUnsafe(edges, edgeIndex, chanIndex,
// the channel. If the channel were to be marked zombie again, it would be
// marked with the correct lagging channel since we received an update from only
// one side.
func makeZombiePubkeys(info *models.ChannelEdgeInfo1,
e1, e2 *models.ChannelEdgePolicy1) ([33]byte, [33]byte) {
func makeZombiePubkeys(info models.ChannelEdgeInfo,
e1, e2 *models.ChannelEdgePolicy1) ([33]byte, [33]byte, error) {
var (
node1Bytes = info.Node1Bytes()
node2Bytes = info.Node2Bytes()
)
switch {
// If we don't have either edge policy, we'll return both pubkeys so
// that the channel can be resurrected by either party.
case e1 == nil && e2 == nil:
return info.NodeKey1Bytes, info.NodeKey2Bytes
return node1Bytes, node2Bytes, nil
// If we're missing edge1, or if both edges are present but edge1 is
// older, we'll return edge1's pubkey and a blank pubkey for edge2. This
// means that only an update from edge1 will be able to resurrect the
// channel.
case e1 == nil || (e2 != nil && e1.LastUpdate.Before(e2.LastUpdate)):
return info.NodeKey1Bytes, [33]byte{}
// If we're only missing edge1, then we return edge1's pubkey and a
// blank pubkey for edge2 so that only an update from edge1 can
// resurrect the channel.
case e1 == nil:
return node1Bytes, [33]byte{}, nil
// Otherwise, we're missing edge2 or edge2 is the older side, so we
// return a blank pubkey for edge1. In this case, only an update from
// edge2 can resurect the channel.
// If we're only missing edge2, then we return edge2's pubkey and a
// blank pubkey for edge1 so that only an update from edge2 can
// resurrect the channel.
case e2 == nil:
return [33]byte{}, node2Bytes, nil
// If we have both edges, then we check which one is older. We return
// the pubkey of the oldest update so that only an update from that
// edge can resurrect the channel.
default:
return [33]byte{}, info.NodeKey2Bytes
e1Before, err := e1.Before(e2)
if err != nil {
return [33]byte{}, [33]byte{}, err
}
if e1Before {
return node1Bytes, [33]byte{}, nil
}
return [33]byte{}, node2Bytes, nil
}
}
@ -3050,15 +3055,16 @@ func (c *ChannelGraph) isPublic(tx kvdb.RTx, nodePub route.Vertex,
nodeIsPublic := false
errDone := errors.New("done")
err := c.ForEachNodeChannelTx(tx, nodePub, func(tx kvdb.RTx,
info *models.ChannelEdgeInfo1, _ *models.ChannelEdgePolicy1,
info models.ChannelEdgeInfo, _ *models.ChannelEdgePolicy1,
_ *models.ChannelEdgePolicy1) error {
// If this edge doesn't extend to the source node, we'll
// terminate our search as we can now conclude that the node is
// publicly advertised within the graph due to the local node
// knowing of the current edge.
if !bytes.Equal(info.NodeKey1Bytes[:], sourcePubKey) &&
!bytes.Equal(info.NodeKey2Bytes[:], sourcePubKey) {
node1Bytes, node2Bytes := info.Node1Bytes(), info.Node2Bytes()
if !bytes.Equal(node1Bytes[:], sourcePubKey) &&
!bytes.Equal(node2Bytes[:], sourcePubKey) {
nodeIsPublic = true
return errDone
@ -3066,7 +3072,7 @@ func (c *ChannelGraph) isPublic(tx kvdb.RTx, nodePub route.Vertex,
// Since the edge _does_ extend to the source node, we'll also
// need to ensure that this is a public edge.
if info.AuthProof != nil {
if info.GetAuthProof() != nil {
nodeIsPublic = true
return errDone
}
@ -3193,7 +3199,7 @@ func (n *graphCacheNode) Features() *lnwire.FeatureVector {
//
// Unknown policies are passed into the callback as nil values.
func (n *graphCacheNode) ForEachChannel(tx kvdb.RTx,
cb func(kvdb.RTx, *models.ChannelEdgeInfo1, *models.ChannelEdgePolicy1,
cb func(kvdb.RTx, models.ChannelEdgeInfo, *models.ChannelEdgePolicy1,
*models.ChannelEdgePolicy1) error) error {
return nodeTraversal(tx, n.pubKeyBytes[:], nil, cb)
@ -3254,7 +3260,7 @@ func (c *ChannelGraph) HasLightningNode(nodePub [33]byte) (time.Time, bool, erro
// nodeTraversal is used to traverse all channels of a node given by its
// public key and passes channel information into the specified callback.
func nodeTraversal(tx kvdb.RTx, nodePub []byte, db kvdb.Backend,
cb func(kvdb.RTx, *models.ChannelEdgeInfo1, *models.ChannelEdgePolicy1,
cb func(kvdb.RTx, models.ChannelEdgeInfo, *models.ChannelEdgePolicy1,
*models.ChannelEdgePolicy1) error) error {
traversal := func(tx kvdb.RTx) error {
@ -3289,18 +3295,11 @@ func nodeTraversal(tx kvdb.RTx, nodePub []byte, db kvdb.Backend,
// the node at the other end of the channel and both
// edge policies.
chanID := nodeEdge[33:]
info, err := fetchChanEdgeInfo(edgeIndex, chanID)
edgeInfo, err := fetchChanEdgeInfo(edgeIndex, chanID)
if err != nil {
return err
}
edgeInfo, ok := info.(*models.ChannelEdgeInfo1)
if !ok {
return fmt.Errorf("expected "+
"*models.ChannelEdgeInfo1, got %T",
edgeInfo)
}
outgoingPolicy, err := fetchChanEdgePolicy(
edges, chanID, nodePub,
)
@ -3308,9 +3307,19 @@ func nodeTraversal(tx kvdb.RTx, nodePub []byte, db kvdb.Backend,
return err
}
otherNode, err := edgeInfo.OtherNodeKeyBytes(nodePub)
if err != nil {
return err
var (
otherNode [33]byte
node1Bytes = edgeInfo.Node1Bytes()
node2Bytes = edgeInfo.Node2Bytes()
)
switch {
case bytes.Equal(node1Bytes[:], nodePub):
otherNode = node2Bytes
case bytes.Equal(node2Bytes[:], nodePub):
otherNode = node1Bytes
default:
return fmt.Errorf("node not participating in " +
"this channel")
}
incomingPolicy, err := fetchChanEdgePolicy(
@ -3350,7 +3359,7 @@ func nodeTraversal(tx kvdb.RTx, nodePub []byte, db kvdb.Backend,
//
// Unknown policies are passed into the callback as nil values.
func (c *ChannelGraph) ForEachNodeChannel(nodePub route.Vertex,
cb func(kvdb.RTx, *models.ChannelEdgeInfo1, *models.ChannelEdgePolicy1,
cb func(kvdb.RTx, models.ChannelEdgeInfo, *models.ChannelEdgePolicy1,
*models.ChannelEdgePolicy1) error) error {
return nodeTraversal(nil, nodePub[:], c.db, cb)
@ -3370,7 +3379,7 @@ func (c *ChannelGraph) ForEachNodeChannel(nodePub route.Vertex,
// be nil and a fresh transaction will be created to execute the graph
// traversal.
func (c *ChannelGraph) ForEachNodeChannelTx(tx kvdb.RTx,
nodePub route.Vertex, cb func(kvdb.RTx, *models.ChannelEdgeInfo1,
nodePub route.Vertex, cb func(kvdb.RTx, models.ChannelEdgeInfo,
*models.ChannelEdgePolicy1,
*models.ChannelEdgePolicy1) error) error {
@ -3382,16 +3391,21 @@ func (c *ChannelGraph) ForEachNodeChannelTx(tx kvdb.RTx,
// one of the nodes, and wishes to obtain the full LightningNode for the other
// end of the channel.
func (c *ChannelGraph) FetchOtherNode(tx kvdb.RTx,
channel *models.ChannelEdgeInfo1, thisNodeKey []byte) (*LightningNode,
edge models.ChannelEdgeInfo, thisNodeKey []byte) (*LightningNode,
error) {
var (
targetNodeBytes [33]byte
node1Bytes = edge.Node1Bytes()
node2Bytes = edge.Node2Bytes()
)
// Ensure that the node passed in is actually a member of the channel.
var targetNodeBytes [33]byte
switch {
case bytes.Equal(channel.NodeKey1Bytes[:], thisNodeKey):
targetNodeBytes = channel.NodeKey2Bytes
case bytes.Equal(channel.NodeKey2Bytes[:], thisNodeKey):
targetNodeBytes = channel.NodeKey1Bytes
case bytes.Equal(node1Bytes[:], thisNodeKey):
targetNodeBytes = node2Bytes
case bytes.Equal(node2Bytes[:], thisNodeKey):
targetNodeBytes = node1Bytes
default:
return nil, fmt.Errorf("node not participating in this channel")
}
@ -3454,11 +3468,11 @@ func computeEdgePolicyKeys(info models.ChannelEdgeInfo) ([]byte, []byte) {
// information for the channel itself is returned as well as two structs that
// contain the routing policies for the channel in either direction.
func (c *ChannelGraph) FetchChannelEdgesByOutpoint(op *wire.OutPoint) (
*models.ChannelEdgeInfo1, *models.ChannelEdgePolicy1,
models.ChannelEdgeInfo, *models.ChannelEdgePolicy1,
*models.ChannelEdgePolicy1, error) {
var (
edgeInfo *models.ChannelEdgeInfo1
edgeInfo models.ChannelEdgeInfo
policy1 *models.ChannelEdgePolicy1
policy2 *models.ChannelEdgePolicy1
)
@ -3500,20 +3514,12 @@ func (c *ChannelGraph) FetchChannelEdgesByOutpoint(op *wire.OutPoint) (
// If the channel is found to exists, then we'll first retrieve
// the general information for the channel.
edge, err := fetchChanEdgeInfo(edgeIndex, chanID)
var err error
edgeInfo, err = fetchChanEdgeInfo(edgeIndex, chanID)
if err != nil {
return fmt.Errorf("%w: chanID=%x", err, chanID)
}
info, ok := edge.(*models.ChannelEdgeInfo1)
if !ok {
return fmt.Errorf("expected "+
"*models.ChannelEdgeInfo1, got %T",
edge)
}
edgeInfo = info
// Once we have the information about the channels' parameters,
// we'll fetch the routing policies for each for the directed
// edges.
@ -3547,11 +3553,11 @@ func (c *ChannelGraph) FetchChannelEdgesByOutpoint(op *wire.OutPoint) (
// within the database. In this case, the ChannelEdgePolicy1's will be nil, and
// the ChannelEdgeInfo1 will only include the public keys of each node.
func (c *ChannelGraph) FetchChannelEdgesByID(chanID uint64) (
*models.ChannelEdgeInfo1, *models.ChannelEdgePolicy1,
models.ChannelEdgeInfo, *models.ChannelEdgePolicy1,
*models.ChannelEdgePolicy1, error) {
var (
edgeInfo *models.ChannelEdgeInfo1
edgeInfo models.ChannelEdgeInfo
policy1 *models.ChannelEdgePolicy1
policy2 *models.ChannelEdgePolicy1
channelID [8]byte
@ -3580,7 +3586,8 @@ func (c *ChannelGraph) FetchChannelEdgesByID(chanID uint64) (
byteOrder.PutUint64(channelID[:], chanID)
// Now, attempt to fetch edge.
edge, err := fetchChanEdgeInfo(edgeIndex, channelID[:])
var err error
edgeInfo, err = fetchChanEdgeInfo(edgeIndex, channelID[:])
// If it doesn't exist, we'll quickly check our zombie index to
// see if we've previously marked it as so.
@ -3616,14 +3623,6 @@ func (c *ChannelGraph) FetchChannelEdgesByID(chanID uint64) (
return err
}
info, ok := edge.(*models.ChannelEdgeInfo1)
if !ok {
return fmt.Errorf("expected "+
"*models.ChannelEdgeInfo1, got %T", edge)
}
edgeInfo = info
// Then we'll attempt to fetch the accompanying policies of this
// edge.
e1, e2, err := fetchChanEdgePolicies(

@ -28,7 +28,7 @@ type GraphCacheNode interface {
// error, then the iteration is halted with the error propagated back up
// to the caller.
ForEachChannel(kvdb.RTx,
func(kvdb.RTx, *models.ChannelEdgeInfo1,
func(kvdb.RTx, models.ChannelEdgeInfo,
*models.ChannelEdgePolicy1,
*models.ChannelEdgePolicy1) error) error
}
@ -142,7 +142,7 @@ func (c *GraphCache) AddNode(tx kvdb.RTx, node GraphCacheNode) error {
c.AddNodeFeatures(node)
return node.ForEachChannel(
tx, func(tx kvdb.RTx, info *models.ChannelEdgeInfo1,
tx, func(tx kvdb.RTx, info models.ChannelEdgeInfo,
outPolicy *models.ChannelEdgePolicy1,
inPolicy *models.ChannelEdgePolicy1) error {

@ -42,7 +42,7 @@ func (n *node) Features() *lnwire.FeatureVector {
}
func (n *node) ForEachChannel(tx kvdb.RTx,
cb func(kvdb.RTx, *models.ChannelEdgeInfo1, *models.ChannelEdgePolicy1,
cb func(kvdb.RTx, models.ChannelEdgeInfo, *models.ChannelEdgePolicy1,
*models.ChannelEdgePolicy1) error) error {
for idx := range n.edgeInfos {

@ -509,10 +509,10 @@ func TestDisconnectBlockAtHeight(t *testing.T) {
t.Fatalf("expected two edges to be removed from graph, "+
"only %d were", len(removed))
}
if removed[0].ChannelID != edgeInfo.ChannelID {
if removed[0].GetChanID() != edgeInfo.ChannelID {
t.Fatalf("expected edge to be removed from graph")
}
if removed[1].ChannelID != edgeInfo2.ChannelID {
if removed[1].GetChanID() != edgeInfo2.ChannelID {
t.Fatalf("expected edge to be removed from graph")
}
@ -556,7 +556,19 @@ func TestDisconnectBlockAtHeight(t *testing.T) {
}
}
func assertEdgeInfoEqual(t *testing.T, e1 *models.ChannelEdgeInfo1,
func assertEdgeInfoEqual(t *testing.T, e1, e2 models.ChannelEdgeInfo) {
switch edge1 := e1.(type) {
case *models.ChannelEdgeInfo1:
edge2, ok := e2.(*models.ChannelEdgeInfo1)
require.True(t, ok)
assertEdgeInfo1Equal(t, edge1, edge2)
default:
t.Fatalf("unhandled ChannelEdgeInfo type: %T", e1)
}
}
func assertEdgeInfo1Equal(t *testing.T, e1 *models.ChannelEdgeInfo1,
e2 *models.ChannelEdgeInfo1) {
if e1.ChannelID != e2.ChannelID {
@ -1042,11 +1054,11 @@ func TestGraphTraversal(t *testing.T) {
// Iterate through all the known channels within the graph DB, once
// again if the map is empty that indicates that all edges have
// properly been reached.
err = graph.ForEachChannel(func(ei *models.ChannelEdgeInfo1,
err = graph.ForEachChannel(func(ei models.ChannelEdgeInfo,
_ *models.ChannelEdgePolicy1,
_ *models.ChannelEdgePolicy1) error {
delete(chanIndex, ei.ChannelID)
delete(chanIndex, ei.GetChanID())
return nil
})
require.NoError(t, err)
@ -1057,7 +1069,7 @@ func TestGraphTraversal(t *testing.T) {
numNodeChans := 0
firstNode, secondNode := nodeList[0], nodeList[1]
err = graph.ForEachNodeChannel(firstNode.PubKeyBytes,
func(_ kvdb.RTx, _ *models.ChannelEdgeInfo1, outEdge,
func(_ kvdb.RTx, _ models.ChannelEdgeInfo, outEdge,
inEdge *models.ChannelEdgePolicy1) error {
// All channels between first and second node should
@ -1138,11 +1150,11 @@ func TestGraphTraversalCacheable(t *testing.T) {
for _, node := range nodes {
err := node.ForEachChannel(
tx, func(tx kvdb.RTx,
info *models.ChannelEdgeInfo1,
info models.ChannelEdgeInfo,
policy *models.ChannelEdgePolicy1,
policy2 *models.ChannelEdgePolicy1) error { //nolint:lll
delete(chanIndex, info.ChannelID)
delete(chanIndex, info.GetChanID())
return nil
},
)
@ -1322,7 +1334,7 @@ func assertPruneTip(t *testing.T, graph *ChannelGraph, blockHash *chainhash.Hash
func assertNumChans(t *testing.T, graph *ChannelGraph, n int) {
numChans := 0
if err := graph.ForEachChannel(func(*models.ChannelEdgeInfo1,
if err := graph.ForEachChannel(func(models.ChannelEdgeInfo,
*models.ChannelEdgePolicy1,
*models.ChannelEdgePolicy1) error {
@ -2749,7 +2761,7 @@ func TestIncompleteChannelPolicies(t *testing.T) {
checkPolicies := func(node *LightningNode, expectedIn, expectedOut bool) {
calls := 0
err := graph.ForEachNodeChannel(node.PubKeyBytes,
func(_ kvdb.RTx, _ *models.ChannelEdgeInfo1, outEdge,
func(_ kvdb.RTx, _ models.ChannelEdgeInfo, outEdge,
inEdge *models.ChannelEdgePolicy1) error {
if !expectedOut && outEdge != nil {
@ -3887,7 +3899,7 @@ func BenchmarkForEachChannel(b *testing.B) {
err = graph.db.View(func(tx kvdb.RTx) error {
for _, n := range nodes {
cb := func(tx kvdb.RTx,
info *models.ChannelEdgeInfo1,
info models.ChannelEdgeInfo,
policy *models.ChannelEdgePolicy1,
policy2 *models.ChannelEdgePolicy1) error { //nolint:lll
@ -3896,7 +3908,7 @@ func BenchmarkForEachChannel(b *testing.B) {
// compiler is going to optimize
// this away, and we get bogus
// results.
totalCapacity += info.Capacity
totalCapacity += info.GetCapacity()
maxHTLCs += policy.MaxHTLC
maxHTLCs += policy2.MaxHTLC
@ -3977,10 +3989,10 @@ func TestGraphCacheForEachNodeChannel(t *testing.T) {
directedChan := getSingleChannel()
require.NotNil(t, directedChan)
require.Equal(t, directedChan.InboundFee, lnwire.Fee{
require.Equal(t, lnwire.Fee{
BaseFee: 10,
FeeRate: 20,
})
}, directedChan.InboundFee)
// Set an invalid inbound fee and check that the edge is no longer
// returned.

@ -347,6 +347,8 @@ func (c *ChannelEdgeInfo1) FundingScript() ([]byte, error) {
return nil, err
}
// TODO(roasbeef): add tapscript root to gossip v1.5
return fundingScript, nil
}

@ -120,13 +120,13 @@ func (c *ChanSeries) UpdatesInHorizon(chain chainhash.Hash,
// If the channel hasn't been fully advertised yet, or is a
// private channel, then we'll skip it as we can't construct a
// full authentication proof if one is requested.
if channel.Info.AuthProof == nil {
if channel.Info.GetAuthProof() == nil {
continue
}
chanAnn, edge1, edge2, err := netann.CreateChanAnnouncement(
channel.Info.AuthProof, channel.Info, channel.Policy1,
channel.Policy2,
channel.Info.GetAuthProof(), channel.Info,
channel.Policy1, channel.Policy2,
)
if err != nil {
return nil, err
@ -264,13 +264,13 @@ func (c *ChanSeries) FetchChanAnns(chain chainhash.Hash,
// If the channel doesn't have an authentication proof, then we
// won't send it over as it may not yet be finalized, or be a
// non-advertised channel.
if channel.Info.AuthProof == nil {
if channel.Info.GetAuthProof() == nil {
continue
}
chanAnn, edge1, edge2, err := netann.CreateChanAnnouncement(
channel.Info.AuthProof, channel.Info, channel.Policy1,
channel.Policy2,
channel.Info.GetAuthProof(), channel.Info,
channel.Policy1, channel.Policy2,
)
if err != nil {
return nil, err

@ -553,7 +553,7 @@ func New(cfg Config, selfKeyDesc *keychain.KeyDescriptor) *AuthenticatedGossiper
// EdgeWithInfo contains the information that is required to update an edge.
type EdgeWithInfo struct {
// Info describes the channel.
Info *models.ChannelEdgeInfo1
Info models.ChannelEdgeInfo
// Edge describes the policy in one direction of the channel.
Edge *models.ChannelEdgePolicy1
@ -1617,7 +1617,7 @@ func (d *AuthenticatedGossiper) retransmitStaleAnns(now time.Time) error {
// Iterate over all of our channels and check if any of them fall
// within the prune interval or re-broadcast interval.
type updateTuple struct {
info *models.ChannelEdgeInfo1
info models.ChannelEdgeInfo
edge *models.ChannelEdgePolicy1
}
@ -1627,7 +1627,7 @@ func (d *AuthenticatedGossiper) retransmitStaleAnns(now time.Time) error {
)
err := d.cfg.Graph.ForAllOutgoingChannels(func(
_ kvdb.RTx,
info *models.ChannelEdgeInfo1,
info models.ChannelEdgeInfo,
edge *models.ChannelEdgePolicy1) error {
// If there's no auth proof attached to this edge, it means
@ -1635,9 +1635,9 @@ func (d *AuthenticatedGossiper) retransmitStaleAnns(now time.Time) error {
// the greater network, so avoid sending channel updates for
// this channel to not leak its
// existence.
if info.AuthProof == nil {
if info.GetAuthProof() == nil {
log.Debugf("Skipping retransmission of channel "+
"without AuthProof: %v", info.ChannelID)
"without AuthProof: %v", info.GetChanID())
return nil
}
@ -1653,7 +1653,9 @@ func (d *AuthenticatedGossiper) retransmitStaleAnns(now time.Time) error {
// We'll make sure we support the new max_htlc field if
// not already present.
edge.MessageFlags |= lnwire.ChanUpdateRequiredMaxHtlc
edge.MaxHTLC = lnwire.NewMSatFromSatoshis(info.Capacity)
edge.MaxHTLC = lnwire.NewMSatFromSatoshis(
info.GetCapacity(),
)
edgesToUpdate = append(edgesToUpdate, updateTuple{
info: info,
@ -1771,7 +1773,7 @@ func (d *AuthenticatedGossiper) processChanPolicyUpdate(
// We'll avoid broadcasting any updates for private channels to
// avoid directly giving away their existence. Instead, we'll
// send the update directly to the remote party.
if edgeInfo.Info.AuthProof == nil {
if edgeInfo.Info.GetAuthProof() == nil {
// If AuthProof is nil and an alias was found for this
// ChannelID (meaning the option-scid-alias feature was
// negotiated), we'll replace the ShortChannelID in the
@ -1779,7 +1781,7 @@ func (d *AuthenticatedGossiper) processChanPolicyUpdate(
// updateChannel so that the alias isn't persisted to
// the database.
chanID := lnwire.NewChanIDFromOutPoint(
edgeInfo.Info.ChannelPoint,
edgeInfo.Info.GetChanPoint(),
)
var defaultAlias lnwire.ShortChannelID
@ -1805,7 +1807,7 @@ func (d *AuthenticatedGossiper) processChanPolicyUpdate(
}
remotePubKey := remotePubFromChanInfo(
edgeInfo.Info, chanUpdate.ChannelFlags,
edgeInfo.Info, chanUpdate.IsNode1(),
)
err := d.reliableSender.sendMessage(
chanUpdate, remotePubKey,
@ -1834,18 +1836,14 @@ func (d *AuthenticatedGossiper) processChanPolicyUpdate(
// remotePubFromChanInfo returns the public key of the remote peer given a
// ChannelEdgeInfo1 that describe a channel we have with them.
func remotePubFromChanInfo(chanInfo *models.ChannelEdgeInfo1,
chanFlags lnwire.ChanUpdateChanFlags) [33]byte {
func remotePubFromChanInfo(chanInfo models.ChannelEdgeInfo,
isNode1 bool) [33]byte {
var remotePubKey [33]byte
switch {
case chanFlags&lnwire.ChanUpdateDirection == 0:
remotePubKey = chanInfo.NodeKey2Bytes
case chanFlags&lnwire.ChanUpdateDirection == 1:
remotePubKey = chanInfo.NodeKey1Bytes
if isNode1 {
return chanInfo.Node2Bytes()
}
return remotePubKey
return chanInfo.Node1Bytes()
}
// processRejectedEdge examines a rejected edge to see if we can extract any
@ -1870,7 +1868,7 @@ func (d *AuthenticatedGossiper) processRejectedEdge(
// The edge is in the graph, and has a proof attached, then we'll just
// reject it as normal.
if chanInfo.AuthProof != nil {
if chanInfo.GetAuthProof() != nil {
return nil, nil
}
@ -2076,13 +2074,9 @@ func (d *AuthenticatedGossiper) processNetworkAnnouncement(
// NOTE: only the NodeKey1Bytes and NodeKey2Bytes members of the
// ChannelEdgeInfo1 should be inspected.
func (d *AuthenticatedGossiper) processZombieUpdate(
chanInfo *models.ChannelEdgeInfo1, scid lnwire.ShortChannelID,
chanInfo models.ChannelEdgeInfo, scid lnwire.ShortChannelID,
msg *lnwire.ChannelUpdate1) error {
// The least-significant bit in the flag on the channel update tells us
// which edge is being updated.
isNode1 := msg.ChannelFlags&lnwire.ChanUpdateDirection == 0
// Since we've deemed the update as not stale above, before marking it
// live, we'll make sure it has been signed by the correct party. If we
// have both pubkeys, either party can resurrect the channel. If we've
@ -2090,9 +2084,9 @@ func (d *AuthenticatedGossiper) processZombieUpdate(
// will only have the pubkey of the node with the oldest timestamp.
var pubKey *btcec.PublicKey
switch {
case isNode1 && chanInfo.NodeKey1Bytes != emptyPubkey:
case msg.IsNode1() && chanInfo.Node1Bytes() != emptyPubkey:
pubKey, _ = chanInfo.NodeKey1()
case !isNode1 && chanInfo.NodeKey2Bytes != emptyPubkey:
case !msg.IsNode1() && chanInfo.Node2Bytes() != emptyPubkey:
pubKey, _ = chanInfo.NodeKey2()
}
if pubKey == nil {
@ -2161,14 +2155,14 @@ func (d *AuthenticatedGossiper) isMsgStale(msg lnwire.Message) bool {
}
if err != nil {
log.Debugf("Unable to retrieve channel=%v from graph: "+
"%v", chanInfo.ChannelID, err)
"%v", chanInfo.GetChanID(), err)
return false
}
// If the proof exists in the graph, then we have successfully
// received the remote proof and assembled the full proof, so we
// can safely delete the local proof from the database.
return chanInfo.AuthProof != nil
return chanInfo.GetAuthProof() != nil
case *lnwire.ChannelUpdate1:
_, p1, p2, err := d.cfg.Graph.GetChannelByID(msg.ShortChannelID)
@ -2213,16 +2207,21 @@ func (d *AuthenticatedGossiper) isMsgStale(msg lnwire.Message) bool {
// updateChannel creates a new fully signed update for the channel, and updates
// the underlying graph with the new state.
func (d *AuthenticatedGossiper) updateChannel(info *models.ChannelEdgeInfo1,
func (d *AuthenticatedGossiper) updateChannel(edgeInfo models.ChannelEdgeInfo,
edge *models.ChannelEdgePolicy1) (*lnwire.ChannelAnnouncement1,
*lnwire.ChannelUpdate1, error) {
// Parse the unsigned edge into a channel update.
chanUpdate := netann.UnsignedChannelUpdateFromEdge(info, edge)
chanUpdate, err := netann.UnsignedChannelUpdateFromEdge(
edgeInfo.GetChainHash(), edge,
)
if err != nil {
return nil, nil, err
}
// We'll generate a new signature over a digest of the channel
// announcement itself and update the timestamp to ensure it propagate.
err := netann.SignChannelUpdate(
err = netann.SignChannelUpdate(
d.cfg.AnnSigner, d.selfKeyLoc, chanUpdate,
netann.ChanUpdSetTimestamp,
)
@ -2238,7 +2237,7 @@ func (d *AuthenticatedGossiper) updateChannel(info *models.ChannelEdgeInfo1,
// To ensure that our signature is valid, we'll verify it ourself
// before committing it to the slice returned.
err = lnwire.ValidateChannelUpdateAnn(
d.selfKey, info.Capacity, chanUpdate,
d.selfKey, edgeInfo.GetCapacity(), chanUpdate,
)
if err != nil {
return nil, nil, fmt.Errorf("generated invalid channel "+
@ -2254,47 +2253,67 @@ func (d *AuthenticatedGossiper) updateChannel(info *models.ChannelEdgeInfo1,
// be broadcast along side each other (if necessary), but only if we
// have a full channel announcement for this channel.
var chanAnn *lnwire.ChannelAnnouncement1
if info.AuthProof != nil {
chanID := lnwire.NewShortChanIDFromInt(info.ChannelID)
chanAnn = &lnwire.ChannelAnnouncement1{
ShortChannelID: chanID,
NodeID1: info.NodeKey1Bytes,
NodeID2: info.NodeKey2Bytes,
ChainHash: info.ChainHash,
BitcoinKey1: info.BitcoinKey1Bytes,
Features: lnwire.NewRawFeatureVector(),
BitcoinKey2: info.BitcoinKey2Bytes,
ExtraOpaqueData: info.ExtraOpaqueData,
}
chanAnn.NodeSig1, err = lnwire.NewSigFromECDSARawSignature(
info.AuthProof.NodeSig1Bytes,
)
if err != nil {
return nil, nil, err
}
chanAnn.NodeSig2, err = lnwire.NewSigFromECDSARawSignature(
info.AuthProof.NodeSig2Bytes,
)
if err != nil {
return nil, nil, err
}
chanAnn.BitcoinSig1, err = lnwire.NewSigFromECDSARawSignature(
info.AuthProof.BitcoinSig1Bytes,
)
if err != nil {
return nil, nil, err
}
chanAnn.BitcoinSig2, err = lnwire.NewSigFromECDSARawSignature(
info.AuthProof.BitcoinSig2Bytes,
)
if err != nil {
return nil, nil, err
if edgeInfo.GetAuthProof() != nil {
switch info := edgeInfo.(type) {
case *models.ChannelEdgeInfo1:
chanAnn, err = chanAnn1FromEdgeInfo1(info)
if err != nil {
return nil, nil, err
}
default:
return nil, nil, fmt.Errorf("unhandled "+
"implementation of models.ChannelEdgeInfo: "+
"%T", info)
}
}
return chanAnn, chanUpdate, err
}
func chanAnn1FromEdgeInfo1(info *models.ChannelEdgeInfo1) (
*lnwire.ChannelAnnouncement1, error) {
var err error
chanID := lnwire.NewShortChanIDFromInt(info.ChannelID)
chanAnn := &lnwire.ChannelAnnouncement1{
ShortChannelID: chanID,
NodeID1: info.NodeKey1Bytes,
NodeID2: info.NodeKey2Bytes,
ChainHash: info.ChainHash,
BitcoinKey1: info.BitcoinKey1Bytes,
Features: lnwire.NewRawFeatureVector(),
BitcoinKey2: info.BitcoinKey2Bytes,
ExtraOpaqueData: info.ExtraOpaqueData,
}
chanAnn.NodeSig1, err = lnwire.NewSigFromECDSARawSignature(
info.AuthProof.NodeSig1Bytes,
)
if err != nil {
return nil, err
}
chanAnn.NodeSig2, err = lnwire.NewSigFromECDSARawSignature(
info.AuthProof.NodeSig2Bytes,
)
if err != nil {
return nil, err
}
chanAnn.BitcoinSig1, err = lnwire.NewSigFromECDSARawSignature(
info.AuthProof.BitcoinSig1Bytes,
)
if err != nil {
return nil, err
}
chanAnn.BitcoinSig2, err = lnwire.NewSigFromECDSARawSignature(
info.AuthProof.BitcoinSig2Bytes,
)
if err != nil {
return nil, err
}
return chanAnn, nil
}
// SyncManager returns the gossiper's SyncManager instance.
func (d *AuthenticatedGossiper) SyncManager() *SyncManager {
return d.syncMgr
@ -2568,35 +2587,21 @@ func (d *AuthenticatedGossiper) handleChanAnnouncement(nMsg *networkMsg,
// With the proof validated (if necessary), we can now store it within
// the database for our path finding and syncing needs.
var featureBuf bytes.Buffer
if err := ann.Features.Encode(&featureBuf); err != nil {
log.Errorf("unable to encode features: %v", err)
edge, err := d.buildEdgeInfo(ann, nMsg.optionalMsgFields)
if err != nil {
log.Errorf("unable to build edge info from announcement: %v",
err)
nMsg.err <- err
return nil, false
}
edge := &models.ChannelEdgeInfo1{
ChannelID: scid.ToUint64(),
ChainHash: ann.ChainHash,
NodeKey1Bytes: ann.NodeID1,
NodeKey2Bytes: ann.NodeID2,
BitcoinKey1Bytes: ann.BitcoinKey1,
BitcoinKey2Bytes: ann.BitcoinKey2,
AuthProof: proof,
Features: featureBuf.Bytes(),
ExtraOpaqueData: ann.ExtraOpaqueData,
}
err = edge.SetAuthProof(proof)
if err != nil {
log.Errorf("unable to set auth proof: %v", err)
nMsg.err <- err
// If there were any optional message fields provided, we'll include
// them in its serialized disk representation now.
if nMsg.optionalMsgFields != nil {
if nMsg.optionalMsgFields.capacity != nil {
edge.Capacity = *nMsg.optionalMsgFields.capacity
}
if nMsg.optionalMsgFields.channelPoint != nil {
cp := *nMsg.optionalMsgFields.channelPoint
edge.ChannelPoint = cp
}
return nil, false
}
log.Debugf("Adding edge for short_chan_id: %v", scid.ToUint64())
@ -3021,14 +3026,18 @@ func (d *AuthenticatedGossiper) handleChanUpdate(nMsg *networkMsg,
edgeToUpdate = e2
}
var chanID = chanInfo.GetChanID()
log.Debugf("Validating ChannelUpdate: channel=%v, from node=%x, has "+
"edge=%v", chanInfo.ChannelID, pubKey.SerializeCompressed(),
"edge=%v", chanID, pubKey.SerializeCompressed(),
edgeToUpdate != nil)
// Validate the channel announcement with the expected public key and
// channel capacity. In the case of an invalid channel update, we'll
// return an error to the caller and exit early.
err = lnwire.ValidateChannelUpdateAnn(pubKey, chanInfo.Capacity, upd)
err = lnwire.ValidateChannelUpdateAnn(
pubKey, chanInfo.GetCapacity(), upd,
)
if err != nil {
rErr := fmt.Errorf("unable to validate channel update "+
"announcement for short_chan_id=%v: %v",
@ -3067,7 +3076,7 @@ func (d *AuthenticatedGossiper) handleChanUpdate(nMsg *networkMsg,
// multiple aliases for a channel and we may otherwise
// rate-limit only a single alias of the channel,
// instead of the whole channel.
baseScid := chanInfo.ChannelID
baseScid := chanID
d.Lock()
rls, ok := d.chanUpdateRateLimiter[baseScid]
if !ok {
@ -3100,7 +3109,7 @@ func (d *AuthenticatedGossiper) handleChanUpdate(nMsg *networkMsg,
// only be a difference if AuthProof == nil, this is fine.
update := &models.ChannelEdgePolicy1{
SigBytes: upd.Signature.ToSignatureBytes(),
ChannelID: chanInfo.ChannelID,
ChannelID: chanID,
LastUpdate: timestamp,
MessageFlags: upd.MessageFlags,
ChannelFlags: upd.ChannelFlags,
@ -3125,8 +3134,7 @@ func (d *AuthenticatedGossiper) handleChanUpdate(nMsg *networkMsg,
// Since we know the stored SCID in the graph, we'll
// cache that SCID.
key := newRejectCacheKey(
chanInfo.ChannelID,
sourceToPub(nMsg.source),
chanID, sourceToPub(nMsg.source),
)
_, _ = d.recentRejects.Put(key, &cachedReject{})
@ -3143,7 +3151,7 @@ func (d *AuthenticatedGossiper) handleChanUpdate(nMsg *networkMsg,
// to the greater network. However, our channel counter party will need
// to be given the update, so we'll try sending the update directly to
// the remote peer.
if !nMsg.isRemote && chanInfo.AuthProof == nil {
if !nMsg.isRemote && chanInfo.GetAuthProof() == nil {
if nMsg.optionalMsgFields != nil {
remoteAlias := nMsg.optionalMsgFields.remoteAlias
if remoteAlias != nil {
@ -3174,9 +3182,7 @@ func (d *AuthenticatedGossiper) handleChanUpdate(nMsg *networkMsg,
}
// Get our peer's public key.
remotePubKey := remotePubFromChanInfo(
chanInfo, upd.ChannelFlags,
)
remotePubKey := remotePubFromChanInfo(chanInfo, upd.IsNode1())
log.Debugf("The message %v has no AuthProof, sending the "+
"update to remote peer %x", upd.MsgType(), remotePubKey)
@ -3200,7 +3206,9 @@ func (d *AuthenticatedGossiper) handleChanUpdate(nMsg *networkMsg,
// authentication proof. We also won't broadcast the update if it
// contains an alias because the network would reject this.
var announcements []networkMsg
if chanInfo.AuthProof != nil && !d.cfg.IsAlias(upd.ShortChannelID) {
if chanInfo.GetAuthProof() != nil &&
!d.cfg.IsAlias(upd.ShortChannelID) {
announcements = append(announcements, networkMsg{
peer: nMsg.peer,
source: nMsg.source,
@ -3289,9 +3297,14 @@ func (d *AuthenticatedGossiper) handleAnnSig(nMsg *networkMsg,
return nil, false
}
var (
node1Bytes = chanInfo.Node1Bytes()
node2Bytes = chanInfo.Node2Bytes()
)
nodeID := nMsg.source.SerializeCompressed()
isFirstNode := bytes.Equal(nodeID, chanInfo.NodeKey1Bytes[:])
isSecondNode := bytes.Equal(nodeID, chanInfo.NodeKey2Bytes[:])
isFirstNode := bytes.Equal(nodeID, node1Bytes[:])
isSecondNode := bytes.Equal(nodeID, node2Bytes[:])
// Ensure that channel that was retrieved belongs to the peer which
// sent the proof announcement.
@ -3310,9 +3323,9 @@ func (d *AuthenticatedGossiper) handleAnnSig(nMsg *networkMsg,
if !nMsg.isRemote {
var remotePubKey [33]byte
if isFirstNode {
remotePubKey = chanInfo.NodeKey2Bytes
remotePubKey = node2Bytes
} else {
remotePubKey = chanInfo.NodeKey1Bytes
remotePubKey = node1Bytes
}
// Since the remote peer might not be online we'll call a
@ -3329,7 +3342,8 @@ func (d *AuthenticatedGossiper) handleAnnSig(nMsg *networkMsg,
}
// Check if we already have the full proof for this channel.
if chanInfo.AuthProof != nil {
authInfo := chanInfo.GetAuthProof()
if authInfo != nil {
// If we already have the fully assembled proof, then the peer
// sending us their proof has probably not received our local
// proof yet. So be kind and send them the full proof.
@ -3348,7 +3362,7 @@ func (d *AuthenticatedGossiper) handleAnnSig(nMsg *networkMsg,
ann.ChannelID, peerID)
ca, _, _, err := netann.CreateChanAnnouncement(
chanInfo.AuthProof, chanInfo, e1, e2,
authInfo, chanInfo, e1, e2,
)
if err != nil {
log.Errorf("unable to gen ann: %v",
@ -3502,10 +3516,10 @@ func (d *AuthenticatedGossiper) handleAnnSig(nMsg *networkMsg,
// it since the source gets skipped. This isn't necessary for channel
// updates and announcement signatures since we send those directly to
// our channel counterparty through the gossiper's reliable sender.
node1Ann, err := d.fetchNodeAnn(chanInfo.NodeKey1Bytes)
node1Ann, err := d.fetchNodeAnn(node1Bytes)
if err != nil {
log.Debugf("Unable to fetch node announcement for %x: %v",
chanInfo.NodeKey1Bytes, err)
node1Bytes, err)
} else {
if nodeKey1, err := chanInfo.NodeKey1(); err == nil {
announcements = append(announcements, networkMsg{
@ -3516,10 +3530,10 @@ func (d *AuthenticatedGossiper) handleAnnSig(nMsg *networkMsg,
}
}
node2Ann, err := d.fetchNodeAnn(chanInfo.NodeKey2Bytes)
node2Ann, err := d.fetchNodeAnn(node2Bytes)
if err != nil {
log.Debugf("Unable to fetch node announcement for %x: %v",
chanInfo.NodeKey2Bytes, err)
node2Bytes, err)
} else {
if nodeKey2, err := chanInfo.NodeKey2(); err == nil {
announcements = append(announcements, networkMsg{
@ -3566,3 +3580,73 @@ func (d *AuthenticatedGossiper) ShouldDisconnect(pubkey *btcec.PublicKey) (
return false, nil
}
// buildEdgeInfo builds constructs an appropriate models.ChannelEdgeInfo using
// the given lnwire.ChannelAnnouncement and some optional fields.
func (d *AuthenticatedGossiper) buildEdgeInfo(ann lnwire.ChannelAnnouncement,
opts *optionalMsgFields) (models.ChannelEdgeInfo, error) {
switch a := ann.(type) {
case *lnwire.ChannelAnnouncement1:
var featureBuf bytes.Buffer
if err := a.Features.Encode(&featureBuf); err != nil {
return nil, err
}
edge := &models.ChannelEdgeInfo1{
ChannelID: a.ShortChannelID.ToUint64(),
ChainHash: a.ChainHash,
NodeKey1Bytes: a.NodeID1,
NodeKey2Bytes: a.NodeID2,
BitcoinKey1Bytes: a.BitcoinKey1,
BitcoinKey2Bytes: a.BitcoinKey2,
Features: featureBuf.Bytes(),
ExtraOpaqueData: a.ExtraOpaqueData,
}
// If there were any optional message fields provided, we'll
// include them in its serialized disk representation now.
if opts != nil {
if opts.capacity != nil {
edge.Capacity = *opts.capacity
}
if opts.channelPoint != nil {
cp := *opts.channelPoint
edge.ChannelPoint = cp
}
}
return edge, nil
case *lnwire.ChannelAnnouncement2:
edge := &models.ChannelEdgeInfo2{
ChannelAnnouncement2: *a,
ChannelPoint: wire.OutPoint{},
}
// If there were any optional message fields provided, we'll
// include them in its serialized disk representation now.
if opts != nil {
if opts.channelPoint != nil {
cp := *opts.channelPoint
edge.ChannelPoint = cp
}
}
// If no bitcoin keys are provided, then there is no way to
// construct the pk script from the announcement, and so we
// instead need to fetch the pk script.
if a.BitcoinKey1.IsNone() && a.BitcoinKey2.IsNone() {
pkScript, err := d.fetchPKScript(&a.ShortChannelID.Val)
if err != nil {
return nil, err
}
edge.FundingPkScript = pkScript
}
return edge, nil
default:
return nil, fmt.Errorf("unhandled lnwire.ChannelAnnouncement "+
"implementation: %T", a)
}
}

@ -93,7 +93,7 @@ type mockGraphSource struct {
mu sync.Mutex
nodes []channeldb.LightningNode
infos map[uint64]models.ChannelEdgeInfo1
infos map[uint64]models.ChannelEdgeInfo
edges map[uint64][]models.ChannelEdgePolicy1
zombies map[uint64][][33]byte
chansToReject map[uint64]struct{}
@ -103,7 +103,7 @@ type mockGraphSource struct {
func newMockRouter(height uint32) *mockGraphSource {
return &mockGraphSource{
bestHeight: height,
infos: make(map[uint64]models.ChannelEdgeInfo1),
infos: make(map[uint64]models.ChannelEdgeInfo),
edges: make(map[uint64][]models.ChannelEdgePolicy1),
zombies: make(map[uint64][][33]byte),
chansToReject: make(map[uint64]struct{}),
@ -122,7 +122,7 @@ func (r *mockGraphSource) AddNode(node *channeldb.LightningNode,
return nil
}
func (r *mockGraphSource) AddEdge(info *models.ChannelEdgeInfo1,
func (r *mockGraphSource) AddEdge(info models.ChannelEdgeInfo,
_ ...batch.SchedulerOption) error {
r.mu.Lock()
@ -134,15 +134,16 @@ func (r *mockGraphSource) AddEdge(info *models.ChannelEdgeInfo1,
)
}
if _, ok := r.infos[info.ChannelID]; ok {
if _, ok := r.infos[info.GetChanID()]; ok {
return errors.New("info already exist")
}
if _, ok := r.chansToReject[info.ChannelID]; ok {
if _, ok := r.chansToReject[info.GetChanID()]; ok {
return errors.New("validation failed")
}
r.infos[info.ChannelID] = *info
r.infos[info.GetChanID()] = info
return nil
}
@ -196,8 +197,14 @@ func (r *mockGraphSource) AddProof(chanID lnwire.ShortChannelID,
return errors.New("channel does not exist")
}
info.AuthProof = proof
r.infos[chanIDInt] = info
infoCP := info.Copy()
err := infoCP.SetAuthProof(proof)
if err != nil {
return err
}
r.infos[chanIDInt] = infoCP
return nil
}
@ -207,7 +214,7 @@ func (r *mockGraphSource) ForEachNode(func(node *channeldb.LightningNode) error)
}
func (r *mockGraphSource) ForAllOutgoingChannels(cb func(tx kvdb.RTx,
i *models.ChannelEdgeInfo1,
i models.ChannelEdgeInfo,
c *models.ChannelEdgePolicy1) error) error {
r.mu.Lock()
@ -217,9 +224,9 @@ func (r *mockGraphSource) ForAllOutgoingChannels(cb func(tx kvdb.RTx,
for _, info := range r.infos {
info := info
edgeInfo := chans[info.ChannelID]
edgeInfo.Info = &info
chans[info.ChannelID] = edgeInfo
edgeInfo := chans[info.GetChanID()]
edgeInfo.Info = info
chans[info.GetChanID()] = edgeInfo
}
for _, edges := range r.edges {
edges := edges
@ -239,7 +246,7 @@ func (r *mockGraphSource) ForAllOutgoingChannels(cb func(tx kvdb.RTx,
}
func (r *mockGraphSource) GetChannelByID(chanID lnwire.ShortChannelID) (
*models.ChannelEdgeInfo1,
models.ChannelEdgeInfo,
*models.ChannelEdgePolicy1,
*models.ChannelEdgePolicy1, error) {
@ -262,7 +269,7 @@ func (r *mockGraphSource) GetChannelByID(chanID lnwire.ShortChannelID) (
edges := r.edges[chanID.ToUint64()]
if len(edges) == 0 {
return &chanInfo, nil, nil, nil
return chanInfo, nil, nil, nil
}
var edge1 *models.ChannelEdgePolicy1
@ -275,7 +282,7 @@ func (r *mockGraphSource) GetChannelByID(chanID lnwire.ShortChannelID) (
edge2 = &edges[1]
}
return &chanInfo, edge1, edge2, nil
return chanInfo, edge1, edge2, nil
}
func (r *mockGraphSource) FetchLightningNode(
@ -307,10 +314,10 @@ func (r *mockGraphSource) IsStaleNode(nodePub route.Vertex, timestamp time.Time)
// require the node to already have a channel in the graph to not be
// considered stale.
for _, info := range r.infos {
if info.NodeKey1Bytes == nodePub {
if info.Node1Bytes() == nodePub {
return false
}
if info.NodeKey2Bytes == nodePub {
if info.Node2Bytes() == nodePub {
return false
}
}
@ -321,12 +328,15 @@ func (r *mockGraphSource) IsStaleNode(nodePub route.Vertex, timestamp time.Time)
// the graph from the graph's source node's point of view.
func (r *mockGraphSource) IsPublicNode(node route.Vertex) (bool, error) {
for _, info := range r.infos {
if !bytes.Equal(node[:], info.NodeKey1Bytes[:]) &&
!bytes.Equal(node[:], info.NodeKey2Bytes[:]) {
node1 := info.Node1Bytes()
node2 := info.Node2Bytes()
if !bytes.Equal(node[:], node1[:]) &&
!bytes.Equal(node[:], node2[:]) {
continue
}
if info.AuthProof != nil {
if info.GetAuthProof() != nil {
return true, nil
}
}
@ -3483,7 +3493,7 @@ out:
var edgesToUpdate []EdgeWithInfo
err = ctx.router.ForAllOutgoingChannels(func(
_ kvdb.RTx,
info *models.ChannelEdgeInfo1,
info models.ChannelEdgeInfo,
edge *models.ChannelEdgePolicy1) error {
edge.TimeLockDelta = uint16(newTimeLockDelta)
@ -3594,13 +3604,13 @@ func TestProcessChannelAnnouncementOptionalMsgFields(t *testing.T) {
if err != nil {
t.Fatalf("unable to get channel by id: %v", err)
}
if edge.Capacity != capacity {
if edge.GetCapacity() != capacity {
t.Fatalf("expected capacity %v, got %v", capacity,
edge.Capacity)
edge.GetCapacity())
}
if edge.ChannelPoint != channelPoint {
if edge.GetChanPoint() != channelPoint {
t.Fatalf("expected channel point %v, got %v",
channelPoint, edge.ChannelPoint)
channelPoint, edge.GetChanPoint())
}
}

@ -1,7 +1,6 @@
package graph
import (
"bytes"
"fmt"
"runtime"
"strings"
@ -11,15 +10,12 @@ import (
"github.com/btcsuite/btcd/btcec/v2"
"github.com/btcsuite/btcd/btcutil"
"github.com/btcsuite/btcd/chaincfg/chainhash"
"github.com/btcsuite/btcd/wire"
"github.com/go-errors/errors"
"github.com/lightningnetwork/lnd/batch"
"github.com/lightningnetwork/lnd/chainntnfs"
"github.com/lightningnetwork/lnd/channeldb"
"github.com/lightningnetwork/lnd/channeldb/models"
"github.com/lightningnetwork/lnd/fn"
"github.com/lightningnetwork/lnd/input"
"github.com/lightningnetwork/lnd/kvdb"
"github.com/lightningnetwork/lnd/lnutils"
"github.com/lightningnetwork/lnd/lnwallet"
@ -540,19 +536,21 @@ func (b *Builder) pruneZombieChans() error {
log.Infof("Examining channel graph for zombie channels")
// A helper method to detect if the channel belongs to this node
isSelfChannelEdge := func(info *models.ChannelEdgeInfo1) bool {
return info.NodeKey1Bytes == b.cfg.SelfNode ||
info.NodeKey2Bytes == b.cfg.SelfNode
isSelfChannelEdge := func(info models.ChannelEdgeInfo) bool {
return info.Node1Bytes() == b.cfg.SelfNode ||
info.Node2Bytes() == b.cfg.SelfNode
}
// First, we'll collect all the channels which are eligible for garbage
// collection due to being zombies.
filterPruneChans := func(info *models.ChannelEdgeInfo1,
filterPruneChans := func(info models.ChannelEdgeInfo,
e1, e2 *models.ChannelEdgePolicy1) error {
chanID := info.GetChanID()
// Exit early in case this channel is already marked to be
// pruned
_, markedToPrune := chansToPrune[info.ChannelID]
_, markedToPrune := chansToPrune[chanID]
if markedToPrune {
return nil
}
@ -568,12 +566,12 @@ func (b *Builder) pruneZombieChans() error {
if e1Zombie {
log.Tracef("Node1 pubkey=%x of chan_id=%v is zombie",
info.NodeKey1Bytes, info.ChannelID)
info.Node1Bytes(), chanID)
}
if e2Zombie {
log.Tracef("Node2 pubkey=%x of chan_id=%v is zombie",
info.NodeKey2Bytes, info.ChannelID)
info.Node2Bytes(), chanID)
}
// If either edge hasn't been updated for a period of
@ -584,10 +582,10 @@ func (b *Builder) pruneZombieChans() error {
}
log.Debugf("ChannelID(%v) is a zombie, collecting to prune",
info.ChannelID)
chanID)
// TODO(roasbeef): add ability to delete single directional edge
chansToPrune[info.ChannelID] = struct{}{}
chansToPrune[chanID] = struct{}{}
return nil
}
@ -613,8 +611,8 @@ func (b *Builder) pruneZombieChans() error {
// Ensuring we won't prune our own channel from the graph.
for _, disabledEdge := range disabledEdges {
if !isSelfChannelEdge(disabledEdge.Info) {
chansToPrune[disabledEdge.Info.ChannelID] =
struct{}{}
chanID := disabledEdge.Info.GetChanID()
chansToPrune[chanID] = struct{}{}
}
}
}
@ -1088,72 +1086,6 @@ func (b *Builder) addZombieEdge(chanID uint64) error {
return nil
}
// makeFundingScript is used to make the funding script for both segwit v0 and
// segwit v1 (taproot) channels.
//
// TODO(roasbeef: export and use elsewhere?
func makeFundingScript(bitcoinKey1, bitcoinKey2 []byte,
chanFeatures []byte) ([]byte, error) {
legacyFundingScript := func() ([]byte, error) {
witnessScript, err := input.GenMultiSigScript(
bitcoinKey1, bitcoinKey2,
)
if err != nil {
return nil, err
}
pkScript, err := input.WitnessScriptHash(witnessScript)
if err != nil {
return nil, err
}
return pkScript, nil
}
if len(chanFeatures) == 0 {
return legacyFundingScript()
}
// In order to make the correct funding script, we'll need to parse the
// chanFeatures bytes into a feature vector we can interact with.
rawFeatures := lnwire.NewRawFeatureVector()
err := rawFeatures.Decode(bytes.NewReader(chanFeatures))
if err != nil {
return nil, fmt.Errorf("unable to parse chan feature "+
"bits: %w", err)
}
chanFeatureBits := lnwire.NewFeatureVector(
rawFeatures, lnwire.Features,
)
if chanFeatureBits.HasFeature(
lnwire.SimpleTaprootChannelsOptionalStaging,
) {
pubKey1, err := btcec.ParsePubKey(bitcoinKey1)
if err != nil {
return nil, err
}
pubKey2, err := btcec.ParsePubKey(bitcoinKey2)
if err != nil {
return nil, err
}
fundingScript, _, err := input.GenTaprootFundingScript(
pubKey1, pubKey2, 0, fn.None[chainhash.Hash](),
)
if err != nil {
return nil, err
}
// TODO(roasbeef): add tapscript root to gossip v1.5
return fundingScript, nil
}
return legacyFundingScript()
}
// processUpdate processes a new relate authenticated channel/edge, node or
// channel/edge update network update. If the update didn't affect the internal
// state of the draft due to either being out of date, invalid, or redundant,
@ -1181,14 +1113,19 @@ func (b *Builder) processUpdate(msg interface{},
log.Tracef("Updated vertex data for node=%x", msg.PubKeyBytes)
b.stats.incNumNodeUpdates()
case *models.ChannelEdgeInfo1:
log.Debugf("Received ChannelEdgeInfo1 for channel %v",
msg.ChannelID)
case models.ChannelEdgeInfo:
var (
chanID = msg.GetChanID()
node1Bytes = msg.Node1Bytes()
node2Bytes = msg.Node2Bytes()
)
log.Debugf("Received ChannelEdgeInfo for channel %v", chanID)
// Prior to processing the announcement we first check if we
// already know of this channel, if so, then we can exit early.
_, _, exists, isZombie, err := b.cfg.Graph.HasChannelEdge(
msg.ChannelID,
chanID,
)
if err != nil &&
!errors.Is(err, channeldb.ErrGraphNoEdgesFound) {
@ -1198,11 +1135,11 @@ func (b *Builder) processUpdate(msg interface{},
}
if isZombie {
return NewErrf(ErrIgnored, "ignoring msg for zombie "+
"chan_id=%v", msg.ChannelID)
"chan_id=%v", chanID)
}
if exists {
return NewErrf(ErrIgnored, "ignoring msg for known "+
"chan_id=%v", msg.ChannelID)
"chan_id=%v", chanID)
}
// If AssumeChannelValid is present, then we are unable to
@ -1212,7 +1149,7 @@ func (b *Builder) processUpdate(msg interface{},
// skip validation as it will not map to a legitimate tx. This
// is not a DoS vector as only we can add an alias
// ChannelAnnouncement from the gossiper.
scid := lnwire.NewShortChanIDFromInt(msg.ChannelID)
scid := lnwire.NewShortChanIDFromInt(chanID)
if b.cfg.AssumeChannelValid || b.cfg.IsAlias(scid) {
err := b.cfg.Graph.AddChannelEdge(msg, op...)
if err != nil {
@ -1220,8 +1157,7 @@ func (b *Builder) processUpdate(msg interface{},
}
log.Tracef("New channel discovered! Link "+
"connects %x and %x with ChannelID(%v)",
msg.NodeKey1Bytes, msg.NodeKey2Bytes,
msg.ChannelID)
node1Bytes, node2Bytes, chanID)
b.stats.incNumEdgesDiscovered()
break
@ -1230,7 +1166,7 @@ func (b *Builder) processUpdate(msg interface{},
// Before we can add the channel to the channel graph, we need
// to obtain the full funding outpoint that's encoded within
// the channel ID.
channelID := lnwire.NewShortChanIDFromInt(msg.ChannelID)
channelID := lnwire.NewShortChanIDFromInt(chanID)
fundingTx, err := lnwallet.FetchFundingTxWrapper(
b.cfg.Chain, &channelID, b.quit,
)
@ -1257,7 +1193,7 @@ func (b *Builder) processUpdate(msg interface{},
// zombie so we don't continue to request it.
// We use the "zero key" for both node pubkeys
// so this edge can't be resurrected.
zErr := b.addZombieEdge(msg.ChannelID)
zErr := b.addZombieEdge(chanID)
if zErr != nil {
return zErr
}
@ -1272,10 +1208,7 @@ func (b *Builder) processUpdate(msg interface{},
// Recreate witness output to be sure that declared in channel
// edge bitcoin keys and channel value corresponds to the
// reality.
fundingPkScript, err := makeFundingScript(
msg.BitcoinKey1Bytes[:], msg.BitcoinKey2Bytes[:],
msg.Features,
)
fundingPkScript, err := msg.FundingScript()
if err != nil {
return err
}
@ -1296,7 +1229,7 @@ func (b *Builder) processUpdate(msg interface{},
if err != nil {
// Mark the edge as a zombie so we won't try to
// re-validate it on start up.
if err := b.addZombieEdge(msg.ChannelID); err != nil {
if err := b.addZombieEdge(chanID); err != nil {
return err
}
@ -1313,7 +1246,7 @@ func (b *Builder) processUpdate(msg interface{},
)
if err != nil {
if errors.Is(err, btcwallet.ErrOutputSpent) {
zErr := b.addZombieEdge(msg.ChannelID)
zErr := b.addZombieEdge(chanID)
if zErr != nil {
return zErr
}
@ -1321,22 +1254,37 @@ func (b *Builder) processUpdate(msg interface{},
return NewErrf(ErrChannelSpent, "unable to fetch utxo "+
"for chan_id=%v, chan_point=%v: %v",
msg.ChannelID, fundingPoint, err)
chanID, fundingPoint, err)
}
// TODO(roasbeef): this is a hack, needs to be removed
// after commitment fees are dynamic.
msg.Capacity = btcutil.Amount(chanUtxo.Value)
msg.ChannelPoint = *fundingPoint
switch m := msg.(type) {
case *models.ChannelEdgeInfo1:
m.Capacity = btcutil.Amount(chanUtxo.Value)
m.ChannelPoint = *fundingPoint
case *models.ChannelEdgeInfo2:
m.ChannelPoint = *fundingPoint
// We only store the funding script if the bitcoin keys
// were not provided in the announcement.
if m.BitcoinKey1.IsNone() && m.BitcoinKey2.IsNone() {
m.FundingPkScript = fundingPkScript
}
default:
return errors.Errorf("unhandled implementation of "+
"ChannelEdgeInfo: %T", msg)
}
if err := b.cfg.Graph.AddChannelEdge(msg, op...); err != nil {
return errors.Errorf("unable to add edge: %v", err)
}
log.Debugf("New channel discovered! Link "+
"connects %x and %x with ChannelPoint(%v): "+
"chan_id=%v, capacity=%v",
msg.NodeKey1Bytes, msg.NodeKey2Bytes,
fundingPoint, msg.ChannelID, msg.Capacity)
"chan_id=%v, capacity=%v", node1Bytes, node2Bytes,
fundingPoint, chanID, msg.GetCapacity())
b.stats.incNumEdgesDiscovered()
// As a new edge has been added to the channel graph, we'll
@ -1484,7 +1432,7 @@ func (b *Builder) ApplyChannelUpdate(msg *lnwire.ChannelUpdate1) bool {
return false
}
err = lnwire.ValidateChannelUpdateAnn(pubKey, ch.Capacity, msg)
err = lnwire.ValidateChannelUpdateAnn(pubKey, ch.GetCapacity(), msg)
if err != nil {
log.Errorf("Unable to validate channel update: %v", err)
return false
@ -1543,7 +1491,7 @@ func (b *Builder) AddNode(node *channeldb.LightningNode,
// in construction of payment path.
//
// NOTE: This method is part of the ChannelGraphSource interface.
func (b *Builder) AddEdge(edge *models.ChannelEdgeInfo1,
func (b *Builder) AddEdge(edge models.ChannelEdgeInfo,
op ...batch.SchedulerOption) error {
rMsg := &routingMsg{
@ -1610,7 +1558,7 @@ func (b *Builder) SyncedHeight() uint32 {
//
// NOTE: This method is part of the ChannelGraphSource interface.
func (b *Builder) GetChannelByID(chanID lnwire.ShortChannelID) (
*models.ChannelEdgeInfo1,
models.ChannelEdgeInfo,
*models.ChannelEdgePolicy1,
*models.ChannelEdgePolicy1, error) {
@ -1645,10 +1593,10 @@ func (b *Builder) ForEachNode(
//
// NOTE: This method is part of the ChannelGraphSource interface.
func (b *Builder) ForAllOutgoingChannels(cb func(kvdb.RTx,
*models.ChannelEdgeInfo1, *models.ChannelEdgePolicy1) error) error {
models.ChannelEdgeInfo, *models.ChannelEdgePolicy1) error) error {
return b.cfg.Graph.ForEachNodeChannel(b.cfg.SelfNode,
func(tx kvdb.RTx, c *models.ChannelEdgeInfo1,
func(tx kvdb.RTx, c models.ChannelEdgeInfo,
e *models.ChannelEdgePolicy1,
_ *models.ChannelEdgePolicy1) error {
@ -1674,7 +1622,10 @@ func (b *Builder) AddProof(chanID lnwire.ShortChannelID,
return err
}
info.AuthProof = proof
err = info.SetAuthProof(proof)
if err != nil {
return err
}
return b.cfg.Graph.UpdateChannelEdge(info)
}

@ -22,10 +22,13 @@ import (
"github.com/lightningnetwork/lnd/chainntnfs"
"github.com/lightningnetwork/lnd/channeldb"
"github.com/lightningnetwork/lnd/channeldb/models"
"github.com/lightningnetwork/lnd/fn"
"github.com/lightningnetwork/lnd/htlcswitch"
"github.com/lightningnetwork/lnd/input"
"github.com/lightningnetwork/lnd/lntest/wait"
"github.com/lightningnetwork/lnd/lnwire"
"github.com/lightningnetwork/lnd/routing/route"
"github.com/lightningnetwork/lnd/tlv"
"github.com/stretchr/testify/require"
)
@ -80,7 +83,7 @@ func TestAddProof(t *testing.T) {
info, _, _, err := ctx.builder.GetChannelByID(*chanID)
require.NoError(t, err, "unable to get channel")
require.NotNil(t, info.AuthProof)
require.NotNil(t, info.GetAuthProof())
}
// TestIgnoreNodeAnnouncement tests that adding a node to the router that is
@ -2071,3 +2074,242 @@ func (m *mockLink) EligibleToForward() bool {
func (m *mockLink) MayAddOutgoingHtlc(_ lnwire.MilliSatoshi) error {
return m.mayAddOutgoingErr
}
// TestChanAnn2Validation tests that the router can validate the various forms
// of ChannelEdgeInfo2.
func TestChanAnn2Validation(t *testing.T) {
t.Parallel()
var rootHash [32]byte
_, err := rand.Read(rootHash[:])
require.NoError(t, err)
tests := []struct {
name string
makeFundingTx func(t *testing.T, ctx *testCtx) (*wire.MsgTx,
*lnwire.ShortChannelID, []byte)
buildEdgeInfo func(node1 *channeldb.LightningNode,
node2 *channeldb.LightningNode,
chanID *lnwire.ShortChannelID,
pkScript []byte) models.ChannelEdgeInfo
}{
{
// This test covers the case where two bitcoin keys
// are provided in the channel announcement but no
// merkle root hash is provided. In this case, the
// on-chain funding script is expected to be equal to
// the MuSig2 combination of the two bitcoin keys along
// with a BIP86 tweak.
name: "bitcoin keys with bip 86 tweak",
makeFundingTx: func(t *testing.T, ctx *testCtx) (
*wire.MsgTx, *lnwire.ShortChannelID, []byte) {
pkScript, tx, err := input.GenTaprootFundingScript( //nolint:lll
bitcoinKey1, bitcoinKey2, int64(100),
fn.None[chainhash.Hash](),
)
require.NoError(t, err)
fundingTx := wire.NewMsgTx(2)
_, chanID := addFundingTxToChain(
ctx, fundingTx, tx, 0,
)
return fundingTx, chanID, pkScript
},
buildEdgeInfo: func(node1 *channeldb.LightningNode,
node2 *channeldb.LightningNode,
chanID *lnwire.ShortChannelID,
_ []byte) models.ChannelEdgeInfo {
ann := lnwire.ChannelAnnouncement2{}
ann.ShortChannelID.Val = *chanID
ann.NodeID1.Val = node1.PubKeyBytes
ann.NodeID2.Val = node2.PubKeyBytes
btc1 := tlv.ZeroRecordT[
tlv.TlvType12, [33]byte,
]()
copy(
btc1.Val[:],
bitcoinKey1.SerializeCompressed(),
)
ann.BitcoinKey1 = tlv.SomeRecordT(btc1)
btc2 := tlv.ZeroRecordT[
tlv.TlvType14, [33]byte,
]()
copy(
btc2.Val[:],
bitcoinKey2.SerializeCompressed(),
)
ann.BitcoinKey2 = tlv.SomeRecordT(btc2)
return &models.ChannelEdgeInfo2{
ChannelAnnouncement2: ann,
}
},
},
{
// In this case, no bitcoin keys and no merkle root hash
// is included in the channel announcement. In this
// case, it is not necessary to validate that the
// on-chain pk script is equal to anything particular
// since the signature check in discovery would have
// checked that the announcement signature is also
// signed by the output key found on-chain.
name: "no bitcoin keys",
makeFundingTx: func(t *testing.T, ctx *testCtx) (
*wire.MsgTx, *lnwire.ShortChannelID, []byte) {
pkScript, tx, err := input.GenTaprootFundingScript( //nolint:lll
bitcoinKey1, bitcoinKey2, int64(100),
fn.None[chainhash.Hash](),
)
require.NoError(t, err)
fundingTx := wire.NewMsgTx(2)
_, chanID := addFundingTxToChain(
ctx, fundingTx, tx, 0,
)
return fundingTx, chanID, pkScript
},
buildEdgeInfo: func(node1 *channeldb.LightningNode,
node2 *channeldb.LightningNode,
chanID *lnwire.ShortChannelID,
pkScript []byte) models.ChannelEdgeInfo {
ann := lnwire.ChannelAnnouncement2{}
ann.ShortChannelID.Val = *chanID
ann.NodeID1.Val = node1.PubKeyBytes
ann.NodeID2.Val = node2.PubKeyBytes
return &models.ChannelEdgeInfo2{
ChannelAnnouncement2: ann,
FundingPkScript: pkScript,
}
},
},
{
// This test covers the case where bitcoin keys are
// included in the channel announcement along with a
// merkle root hash.
name: "bitcoin keys with non-bip86 tweak",
makeFundingTx: func(t *testing.T, ctx *testCtx) (
*wire.MsgTx, *lnwire.ShortChannelID, []byte) {
fundingTx := wire.NewMsgTx(2)
pkScript, tx, err := input.GenTaprootFundingScript( //nolint:lll
bitcoinKey1, bitcoinKey2, int64(100),
fn.Some[chainhash.Hash](rootHash),
)
require.NoError(t, err)
_, chanID := addFundingTxToChain(
ctx, fundingTx, tx, 0,
)
return fundingTx, chanID, pkScript
},
buildEdgeInfo: func(node1 *channeldb.LightningNode,
node2 *channeldb.LightningNode,
chanID *lnwire.ShortChannelID,
pkScript []byte) models.ChannelEdgeInfo {
ann := lnwire.ChannelAnnouncement2{}
ann.ShortChannelID.Val = *chanID
ann.NodeID1.Val = node1.PubKeyBytes
ann.NodeID2.Val = node2.PubKeyBytes
btc1 := tlv.ZeroRecordT[
tlv.TlvType12, [33]byte,
]()
copy(
btc1.Val[:],
bitcoinKey1.SerializeCompressed(),
)
ann.BitcoinKey1 = tlv.SomeRecordT(btc1)
btc2 := tlv.ZeroRecordT[
tlv.TlvType14, [33]byte,
]()
copy(
btc2.Val[:],
bitcoinKey2.SerializeCompressed(),
)
ann.BitcoinKey2 = tlv.SomeRecordT(btc2)
merkleRootHash := tlv.ZeroRecordT[
tlv.TlvType16, [32]byte,
]()
merkleRootHash.Val = rootHash
ann.MerkleRootHash = tlv.SomeRecordT(
merkleRootHash,
)
return &models.ChannelEdgeInfo2{
ChannelAnnouncement2: ann,
}
},
},
}
for _, test := range tests {
test := test
t.Run(test.name, func(t *testing.T) {
t.Parallel()
ctx := createTestCtxSingleNode(t, 0)
// Create two new nodes within the network that the
// channel will connect.
node1 := createTestNode(t)
node2 := createTestNode(t)
fundingTx, chanID, pkScript := test.makeFundingTx(
t, ctx,
)
fundingBlock := &wire.MsgBlock{
Transactions: []*wire.MsgTx{fundingTx},
}
ctx.chain.addBlock(
fundingBlock, chanID.BlockHeight,
chanID.BlockHeight,
)
edge := test.buildEdgeInfo(
node1, node2, chanID, pkScript,
)
require.NoError(t, ctx.builder.AddEdge(edge))
})
}
}
func addFundingTxToChain(ctx *testCtx, fundingTx *wire.MsgTx,
fundingOutput *wire.TxOut, fundingHeight uint32) (*wire.OutPoint,
*lnwire.ShortChannelID) {
fundingTx.TxOut = append(fundingTx.TxOut, fundingOutput)
chanUtxo := wire.OutPoint{
Hash: fundingTx.TxHash(),
Index: 0,
}
// With the utxo constructed, we'll mark it as closed.
ctx.chain.addUtxo(chanUtxo, fundingOutput)
// Our fake channel will be "confirmed" at height 101.
chanID := &lnwire.ShortChannelID{
BlockHeight: fundingHeight,
TxIndex: 0,
TxPosition: 0,
}
return &chanUtxo, chanID
}

@ -29,7 +29,7 @@ type ChannelGraphSource interface {
// AddEdge is used to add edge/channel to the topology of the router,
// after all information about channel will be gathered this
// edge/channel might be used in construction of payment path.
AddEdge(edge *models.ChannelEdgeInfo1,
AddEdge(edge models.ChannelEdgeInfo,
op ...batch.SchedulerOption) error
// AddProof updates the channel edge info with proof which is needed to
@ -70,7 +70,7 @@ type ChannelGraphSource interface {
// emanating from the "source" node which is the center of the
// star-graph.
ForAllOutgoingChannels(cb func(tx kvdb.RTx,
c *models.ChannelEdgeInfo1,
c models.ChannelEdgeInfo,
e *models.ChannelEdgePolicy1) error) error
// CurrentBlockHeight returns the block height from POV of the router
@ -79,7 +79,7 @@ type ChannelGraphSource interface {
// GetChannelByID return the channel by the channel id.
GetChannelByID(chanID lnwire.ShortChannelID) (
*models.ChannelEdgeInfo1, *models.ChannelEdgePolicy1,
models.ChannelEdgeInfo, *models.ChannelEdgePolicy1,
*models.ChannelEdgePolicy1, error)
// FetchLightningNode attempts to look up a target node by its identity
@ -110,7 +110,7 @@ type DB interface {
// slice of channels that have been closed by the target block are
// returned if the function succeeds without error.
PruneGraph(spentOutputs []*wire.OutPoint, blockHash *chainhash.Hash,
blockHeight uint32) ([]*models.ChannelEdgeInfo1, error)
blockHeight uint32) ([]models.ChannelEdgeInfo, error)
// ChannelView returns the verifiable edge information for each active
// channel within the known channel graph. The set of UTXO's (along with
@ -169,8 +169,7 @@ type DB interface {
// set to the last prune height valid for the remaining chain.
// Channels that were removed from the graph resulting from the
// disconnected block are returned.
DisconnectBlockAtHeight(height uint32) ([]*models.ChannelEdgeInfo1,
error)
DisconnectBlockAtHeight(height uint32) ([]models.ChannelEdgeInfo, error)
// HasChannelEdge returns true if the database knows of a channel edge
// with the passed channel ID, and false otherwise. If an edge with that
@ -191,7 +190,7 @@ type DB interface {
// zombie within the database. In this case, the ChannelEdgePolicy1's
// will be nil, and the ChannelEdgeInfo1 will only include the public
// keys of each node.
FetchChannelEdgesByID(chanID uint64) (*models.ChannelEdgeInfo1,
FetchChannelEdgesByID(chanID uint64) (models.ChannelEdgeInfo,
*models.ChannelEdgePolicy1, *models.ChannelEdgePolicy1, error)
// AddLightningNode adds a vertex/node to the graph database. If the
@ -210,7 +209,7 @@ type DB interface {
// and the set of features that the channel supports. The chanPoint and
// chanID are used to uniquely identify the edge globally within the
// database.
AddChannelEdge(edge *models.ChannelEdgeInfo1,
AddChannelEdge(edge models.ChannelEdgeInfo,
op ...batch.SchedulerOption) error
// MarkEdgeZombie attempts to mark a channel identified by its channel
@ -258,7 +257,7 @@ type DB interface {
//
// Unknown policies are passed into the callback as nil values.
ForEachNodeChannel(nodePub route.Vertex, cb func(kvdb.RTx,
*models.ChannelEdgeInfo1,
models.ChannelEdgeInfo,
*models.ChannelEdgePolicy1,
*models.ChannelEdgePolicy1) error) error
@ -267,7 +266,7 @@ type DB interface {
// created. In order to maintain this constraints, we return an error in
// the scenario that an edge info hasn't yet been created yet, but
// someone attempts to update it.
UpdateChannelEdge(edge *models.ChannelEdgeInfo1) error
UpdateChannelEdge(edge models.ChannelEdgeInfo) error
// IsPublicNode is a helper method that determines whether the node with
// the given public key is seen as a public node in the graph from the

@ -211,15 +211,15 @@ type ClosedChanSummary struct {
// 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 ...*models.ChannelEdgeInfo1) []*ClosedChanSummary {
closedChans ...models.ChannelEdgeInfo) []*ClosedChanSummary {
closeSummaries := make([]*ClosedChanSummary, len(closedChans))
for i, closedChan := range closedChans {
closeSummaries[i] = &ClosedChanSummary{
ChanID: closedChan.ChannelID,
Capacity: closedChan.Capacity,
ChanID: closedChan.GetChanID(),
Capacity: closedChan.GetCapacity(),
ClosedHeight: blockHeight,
ChanPoint: closedChan.ChannelPoint,
ChanPoint: closedChan.GetChanPoint(),
}
}
@ -337,7 +337,7 @@ func addToTopologyChange(graph DB, update *TopologyChange,
// We ignore initial channel announcements as we'll only send out
// updates once the individual edges themselves have been updated.
case *models.ChannelEdgeInfo1:
case models.ChannelEdgeInfo:
return nil
// Any new ChannelUpdateAnnouncements will generate a corresponding
@ -372,9 +372,9 @@ func addToTopologyChange(graph DB, update *TopologyChange,
edgeUpdate := &ChannelEdgeUpdate{
ChanID: m.ChannelID,
ChanPoint: edgeInfo.ChannelPoint,
ChanPoint: edgeInfo.GetChanPoint(),
TimeLockDelta: m.TimeLockDelta,
Capacity: edgeInfo.Capacity,
Capacity: edgeInfo.GetCapacity(),
MinHTLC: m.MinHTLC,
MaxHTLC: m.MaxHTLC,
BaseFee: m.FeeBaseMSat,

@ -126,9 +126,8 @@ func (v *ValidationBarrier) InitJobDependencies(job interface{}) {
v.nodeAnnDependencies[route.Vertex(msg.NodeID1)] = signals
v.nodeAnnDependencies[route.Vertex(msg.NodeID2)] = signals
}
case *models.ChannelEdgeInfo1:
shortID := lnwire.NewShortChanIDFromInt(msg.ChannelID)
case models.ChannelEdgeInfo:
shortID := lnwire.NewShortChanIDFromInt(msg.GetChanID())
if _, ok := v.chanAnnFinSignal[shortID]; !ok {
signals := &validationSignals{
allow: make(chan struct{}),
@ -138,8 +137,8 @@ func (v *ValidationBarrier) InitJobDependencies(job interface{}) {
v.chanAnnFinSignal[shortID] = signals
v.chanEdgeDependencies[shortID] = signals
v.nodeAnnDependencies[route.Vertex(msg.NodeKey1Bytes)] = signals
v.nodeAnnDependencies[route.Vertex(msg.NodeKey2Bytes)] = signals
v.nodeAnnDependencies[msg.Node1Bytes()] = signals
v.nodeAnnDependencies[msg.Node2Bytes()] = signals
}
// These other types don't have any dependants, so no further
@ -218,7 +217,7 @@ func (v *ValidationBarrier) WaitForDependants(job interface{}) error {
// return directly.
case *lnwire.AnnounceSignatures1:
// TODO(roasbeef): need to wait on chan ann?
case *models.ChannelEdgeInfo1:
case models.ChannelEdgeInfo:
case *lnwire.ChannelAnnouncement1:
}
@ -264,8 +263,8 @@ func (v *ValidationBarrier) SignalDependants(job interface{}, allow bool) {
// If we've just finished executing a ChannelAnnouncement, then we'll
// close out the signal, and remove the signal from the map of active
// ones. This will allow/deny any dependent jobs to continue execution.
case *models.ChannelEdgeInfo1:
shortID := lnwire.NewShortChanIDFromInt(msg.ChannelID)
case models.ChannelEdgeInfo:
shortID := lnwire.NewShortChanIDFromInt(msg.GetChanID())
finSignals, ok := v.chanAnnFinSignal[shortID]
if ok {
if allow {

@ -624,7 +624,7 @@ func AddInvoice(ctx context.Context, cfg *AddInvoiceConfig,
// chanCanBeHopHint returns true if the target channel is eligible to be a hop
// hint.
func chanCanBeHopHint(channel *HopHintInfo, cfg *SelectHopHintsCfg) (
*models.ChannelEdgePolicy1, bool) {
models.ChannelEdgePolicy, bool) {
// Since we're only interested in our private channels, we'll skip
// public ones.
@ -679,8 +679,11 @@ func chanCanBeHopHint(channel *HopHintInfo, cfg *SelectHopHintsCfg) (
// Now, we'll need to determine which is the correct policy for HTLCs
// being sent from the remote node.
var remotePolicy *models.ChannelEdgePolicy1
if bytes.Equal(remotePub[:], info.NodeKey1Bytes[:]) {
var (
remotePolicy models.ChannelEdgePolicy
node1Bytes = info.Node1Bytes()
)
if bytes.Equal(remotePub[:], node1Bytes[:]) {
remotePolicy = p1
} else {
remotePolicy = p2
@ -739,16 +742,16 @@ func newHopHintInfo(c *channeldb.OpenChannel, isActive bool) *HopHintInfo {
// newHopHint returns a new hop hint using the relevant data from a hopHintInfo
// and a ChannelEdgePolicy1.
func newHopHint(hopHintInfo *HopHintInfo,
chanPolicy *models.ChannelEdgePolicy1) zpay32.HopHint {
chanPolicy models.ChannelEdgePolicy) zpay32.HopHint {
policy := chanPolicy.ForwardingPolicy()
return zpay32.HopHint{
NodeID: hopHintInfo.RemotePubkey,
ChannelID: hopHintInfo.ShortChannelID,
FeeBaseMSat: uint32(chanPolicy.FeeBaseMSat),
FeeProportionalMillionths: uint32(
chanPolicy.FeeProportionalMillionths,
),
CLTVExpiryDelta: chanPolicy.TimeLockDelta,
NodeID: hopHintInfo.RemotePubkey,
ChannelID: hopHintInfo.ShortChannelID,
FeeBaseMSat: uint32(policy.BaseFee),
FeeProportionalMillionths: uint32(policy.FeeRate),
CLTVExpiryDelta: policy.TimeLockDelta,
}
}
@ -762,7 +765,7 @@ type SelectHopHintsCfg struct {
// FetchChannelEdgesByID attempts to lookup the two directed edges for
// the channel identified by the channel ID.
FetchChannelEdgesByID func(chanID uint64) (*models.ChannelEdgeInfo1,
FetchChannelEdgesByID func(chanID uint64) (models.ChannelEdgeInfo,
*models.ChannelEdgePolicy1, *models.ChannelEdgePolicy1,
error)

@ -67,7 +67,7 @@ func (h *hopHintsConfigMock) FetchAllChannels() ([]*channeldb.OpenChannel,
// FetchChannelEdgesByID attempts to lookup the two directed edges for
// the channel identified by the channel ID.
func (h *hopHintsConfigMock) FetchChannelEdgesByID(chanID uint64) (
*models.ChannelEdgeInfo1, *models.ChannelEdgePolicy1,
models.ChannelEdgeInfo, *models.ChannelEdgePolicy1,
*models.ChannelEdgePolicy1, error) {
args := h.Mock.Called(chanID)

@ -661,7 +661,8 @@ func (m *ChanStatusManager) fetchLastChanUpdateByOutPoint(op wire.OutPoint) (
update, err := ExtractChannelUpdate(
m.ourPubKeyBytes, info, edge1, edge2,
)
return update, info.AuthProof == nil, err
return update, info.GetAuthProof() == nil, err
}
// loadInitialChanState determines the initial ChannelState for a particular

@ -4,6 +4,7 @@ import (
"bytes"
"crypto/rand"
"encoding/binary"
"encoding/hex"
"fmt"
"io"
"sync"
@ -12,6 +13,7 @@ import (
"time"
"github.com/btcsuite/btcd/btcec/v2"
"github.com/btcsuite/btcd/btcec/v2/ecdsa"
"github.com/btcsuite/btcd/wire"
"github.com/lightningnetwork/lnd/channeldb"
"github.com/lightningnetwork/lnd/channeldb/models"
@ -24,10 +26,17 @@ import (
var (
testKeyLoc = keychain.KeyLocator{Family: keychain.KeyFamilyNodeKey}
// testSigBytes specifies a testing signature with the minimal length.
testSigBytes = []byte{
0x30, 0x06, 0x02, 0x01, 0x00, 0x02, 0x01, 0x00,
}
testRBytes, _ = hex.DecodeString("8ce2bc69281ce27da07e6683571319d18e" +
"949ddfa2965fb6caa1bf0314f882d7")
testSBytes, _ = hex.DecodeString("299105481d63e0f4bc2a88121167221b67" +
"00d72a0ead154c03be696a292d24ae")
testRScalar = new(btcec.ModNScalar)
testSScalar = new(btcec.ModNScalar)
_ = testRScalar.SetByteSlice(testRBytes)
_ = testSScalar.SetByteSlice(testSBytes)
testSig = ecdsa.NewSignature(testRScalar, testSScalar)
testSigBytes = testSig.Serialize()
)
// randOutpoint creates a random wire.Outpoint.
@ -121,7 +130,7 @@ func createEdgePolicies(t *testing.T, channel *channeldb.OpenChannel,
type mockGraph struct {
mu sync.Mutex
channels []*channeldb.OpenChannel
chanInfos map[wire.OutPoint]*models.ChannelEdgeInfo1
chanInfos map[wire.OutPoint]models.ChannelEdgeInfo
chanPols1 map[wire.OutPoint]*models.ChannelEdgePolicy1
chanPols2 map[wire.OutPoint]*models.ChannelEdgePolicy1
sidToCid map[lnwire.ShortChannelID]wire.OutPoint
@ -134,7 +143,7 @@ func newMockGraph(t *testing.T, numChannels int,
g := &mockGraph{
channels: make([]*channeldb.OpenChannel, 0, numChannels),
chanInfos: make(map[wire.OutPoint]*models.ChannelEdgeInfo1),
chanInfos: make(map[wire.OutPoint]models.ChannelEdgeInfo),
chanPols1: make(map[wire.OutPoint]*models.ChannelEdgePolicy1),
chanPols2: make(map[wire.OutPoint]*models.ChannelEdgePolicy1),
sidToCid: make(map[lnwire.ShortChannelID]wire.OutPoint),
@ -160,7 +169,7 @@ func (g *mockGraph) FetchAllOpenChannels() ([]*channeldb.OpenChannel, error) {
}
func (g *mockGraph) FetchChannelEdgesByOutpoint(
op *wire.OutPoint) (*models.ChannelEdgeInfo1,
op *wire.OutPoint) (models.ChannelEdgeInfo,
*models.ChannelEdgePolicy1, *models.ChannelEdgePolicy1, error) {
g.mu.Lock()

@ -2,6 +2,7 @@ package netann
import (
"bytes"
"fmt"
"github.com/lightningnetwork/lnd/channeldb/models"
"github.com/lightningnetwork/lnd/lnwire"
@ -12,7 +13,29 @@ import (
// function is used to transform out database structs into the corresponding wire
// structs for announcing new channels to other peers, or simply syncing up a
// peer's initial routing table upon connect.
func CreateChanAnnouncement(chanProof *models.ChannelAuthProof1,
func CreateChanAnnouncement(chanProof models.ChannelAuthProof,
chanInfo models.ChannelEdgeInfo, e1, e2 *models.ChannelEdgePolicy1) (
*lnwire.ChannelAnnouncement1, *lnwire.ChannelUpdate1,
*lnwire.ChannelUpdate1, error) {
switch proof := chanProof.(type) {
case *models.ChannelAuthProof1:
info, ok := chanInfo.(*models.ChannelEdgeInfo1)
if !ok {
return nil, nil, nil, fmt.Errorf("expected type "+
"ChannelEdgeInfo1 to be paired with "+
"ChannelAuthProof1, got: %T", chanInfo)
}
return createChanAnnouncement1(proof, info, e1, e2)
default:
return nil, nil, nil, fmt.Errorf("unhandled "+
"models.ChannelAuthProof type: %T", chanProof)
}
}
func createChanAnnouncement1(chanProof *models.ChannelAuthProof1,
chanInfo *models.ChannelEdgeInfo1,
e1, e2 *models.ChannelEdgePolicy1) (*lnwire.ChannelAnnouncement1,
*lnwire.ChannelUpdate1, *lnwire.ChannelUpdate1, error) {

@ -6,6 +6,7 @@ import (
"time"
"github.com/btcsuite/btcd/btcec/v2"
"github.com/btcsuite/btcd/chaincfg/chainhash"
"github.com/lightningnetwork/lnd/channeldb/models"
"github.com/lightningnetwork/lnd/keychain"
"github.com/lightningnetwork/lnd/lnwallet"
@ -84,7 +85,7 @@ func SignChannelUpdate(signer lnwallet.MessageSigner, keyLoc keychain.KeyLocator
//
// NOTE: The passed policies can be nil.
func ExtractChannelUpdate(ownerPubKey []byte,
info *models.ChannelEdgeInfo1,
info models.ChannelEdgeInfo,
policies ...*models.ChannelEdgePolicy1) (
*lnwire.ChannelUpdate1, error) {
@ -117,11 +118,24 @@ func ExtractChannelUpdate(ownerPubKey []byte,
// UnsignedChannelUpdateFromEdge reconstructs an unsigned ChannelUpdate from the
// given edge info and policy.
func UnsignedChannelUpdateFromEdge(info *models.ChannelEdgeInfo1,
func UnsignedChannelUpdateFromEdge(chainHash chainhash.Hash,
policy models.ChannelEdgePolicy) (*lnwire.ChannelUpdate1, error) {
switch p := policy.(type) {
case *models.ChannelEdgePolicy1:
return unsignedChanPolicy1ToUpdate(chainHash, p), nil
default:
return nil, fmt.Errorf("unhandled implementation of the "+
"models.ChanelEdgePolicy interface: %T", policy)
}
}
func unsignedChanPolicy1ToUpdate(chainHash chainhash.Hash,
policy *models.ChannelEdgePolicy1) *lnwire.ChannelUpdate1 {
return &lnwire.ChannelUpdate1{
ChainHash: info.ChainHash,
ChainHash: chainHash,
ShortChannelID: lnwire.NewShortChanIDFromInt(policy.ChannelID),
Timestamp: uint32(policy.LastUpdate.Unix()),
ChannelFlags: policy.ChannelFlags,
@ -135,20 +149,23 @@ func UnsignedChannelUpdateFromEdge(info *models.ChannelEdgeInfo1,
}
}
// ChannelUpdateFromEdge reconstructs a signed ChannelUpdate from the given edge
// info and policy.
func ChannelUpdateFromEdge(info *models.ChannelEdgeInfo1,
// ChannelUpdateFromEdge reconstructs a signed ChannelUpdate from the given
// edge info and policy.
func ChannelUpdateFromEdge(info models.ChannelEdgeInfo,
policy *models.ChannelEdgePolicy1) (*lnwire.ChannelUpdate1, error) {
update := UnsignedChannelUpdateFromEdge(info, policy)
var err error
update.Signature, err = lnwire.NewSigFromECDSARawSignature(
policy.SigBytes,
)
sig, err := policy.Signature()
if err != nil {
return nil, err
}
s, err := lnwire.NewSigFromSignature(sig)
if err != nil {
return nil, err
}
update := unsignedChanPolicy1ToUpdate(info.GetChainHash(), policy)
update.Signature = s
return update, nil
}

@ -19,6 +19,6 @@ type DB interface {
type ChannelGraph interface {
// FetchChannelEdgesByOutpoint returns the channel edge info and most
// recent channel edge policies for a given outpoint.
FetchChannelEdgesByOutpoint(*wire.OutPoint) (*models.ChannelEdgeInfo1,
FetchChannelEdgesByOutpoint(*wire.OutPoint) (models.ChannelEdgeInfo,
*models.ChannelEdgePolicy1, *models.ChannelEdgePolicy1, error)
}

@ -1035,13 +1035,14 @@ func (p *Brontide) loadActiveChannels(chans []*channeldb.OpenChannel) (
//
// TODO(roasbeef): can add helper method to get policy for
// particular channel.
var selfPolicy *models.ChannelEdgePolicy1
if info != nil && bytes.Equal(info.NodeKey1Bytes[:],
p.cfg.ServerPubKey[:]) {
selfPolicy := p2
if info != nil {
node1Bytes := info.Node1Bytes()
if bytes.Equal(node1Bytes[:],
p.cfg.ServerPubKey[:]) {
selfPolicy = p1
} else {
selfPolicy = p2
selfPolicy = p1
}
}
// If we don't yet have an advertised routing policy, then

@ -42,7 +42,7 @@ type BuildBlindedPathCfg struct {
// FetchChannelEdgesByID attempts to look up the two directed edges for
// the channel identified by the channel ID.
FetchChannelEdgesByID func(chanID uint64) (*models.ChannelEdgeInfo1,
FetchChannelEdgesByID func(chanID uint64) (models.ChannelEdgeInfo,
*models.ChannelEdgePolicy1, *models.ChannelEdgePolicy1, error)
// FetchOurOpenChannels fetches this node's set of open channels.

@ -598,7 +598,7 @@ func TestBuildBlindedPath(t *testing.T) {
return []*route.Route{realRoute}, nil
},
FetchChannelEdgesByID: func(chanID uint64) (
*models.ChannelEdgeInfo1, *models.ChannelEdgePolicy1,
models.ChannelEdgeInfo, *models.ChannelEdgePolicy1,
*models.ChannelEdgePolicy1, error) {
return nil, realPolicies[chanID], nil, nil
@ -766,7 +766,7 @@ func TestBuildBlindedPathWithDummyHops(t *testing.T) {
return []*route.Route{realRoute}, nil
},
FetchChannelEdgesByID: func(chanID uint64) (
*models.ChannelEdgeInfo1, *models.ChannelEdgePolicy1,
models.ChannelEdgeInfo, *models.ChannelEdgePolicy1,
*models.ChannelEdgePolicy1, error) {
policy, ok := realPolicies[chanID]
@ -937,7 +937,7 @@ func TestBuildBlindedPathWithDummyHops(t *testing.T) {
nil
},
FetchChannelEdgesByID: func(chanID uint64) (
*models.ChannelEdgeInfo1, *models.ChannelEdgePolicy1,
models.ChannelEdgeInfo, *models.ChannelEdgePolicy1,
*models.ChannelEdgePolicy1, error) {
// Force the call to error for the first 2 channels.

@ -32,7 +32,7 @@ type Manager struct {
// ForAllOutgoingChannels is required to iterate over all our local
// channels.
ForAllOutgoingChannels func(cb func(kvdb.RTx,
*models.ChannelEdgeInfo1,
models.ChannelEdgeInfo,
*models.ChannelEdgePolicy1) error) error
// FetchChannel is used to query local channel parameters. Optionally an
@ -74,25 +74,27 @@ func (r *Manager) UpdatePolicy(newSchema routing.ChannelPolicy,
// otherwise we'll collect them all.
err := r.ForAllOutgoingChannels(func(
tx kvdb.RTx,
info *models.ChannelEdgeInfo1,
info models.ChannelEdgeInfo,
edge *models.ChannelEdgePolicy1) error {
var chanPoint = info.GetChanPoint()
// If we have a channel filter, and this channel isn't a part
// of it, then we'll skip it.
_, ok := unprocessedChans[info.ChannelPoint]
_, ok := unprocessedChans[chanPoint]
if !ok && haveChanFilter {
return nil
}
// Mark this channel as found by removing it. unprocessedChans
// will be used to report invalid channels later on.
delete(unprocessedChans, info.ChannelPoint)
delete(unprocessedChans, chanPoint)
// Apply the new policy to the edge.
err := r.updateEdge(tx, info.ChannelPoint, edge, newSchema)
err := r.updateEdge(tx, chanPoint, edge, newSchema)
if err != nil {
failedUpdates = append(failedUpdates,
makeFailureItem(info.ChannelPoint,
makeFailureItem(chanPoint,
lnrpc.UpdateFailure_UPDATE_FAILURE_INVALID_PARAMETER,
err.Error(),
))
@ -115,7 +117,7 @@ func (r *Manager) UpdatePolicy(newSchema routing.ChannelPolicy,
inboundFee := models.NewInboundFeeFromWire(inboundWireFee)
// Add updated policy to list of policies to send to switch.
policiesToUpdate[info.ChannelPoint] = models.ForwardingPolicy{
policiesToUpdate[chanPoint] = models.ForwardingPolicy{
BaseFee: edge.FeeBaseMSat,
FeeRate: edge.FeeProportionalMillionths,
TimeLockDelta: uint32(edge.TimeLockDelta),

@ -107,7 +107,7 @@ func TestManager(t *testing.T) {
}
forAllOutgoingChannels := func(cb func(kvdb.RTx,
*models.ChannelEdgeInfo1,
models.ChannelEdgeInfo,
*models.ChannelEdgePolicy1) error) error {
for _, c := range channelSet {

@ -690,7 +690,7 @@ func (r *rpcServer) addDeps(s *server, macService *macaroons.Service,
if err != nil {
return 0, err
}
return info.Capacity, nil
return info.GetCapacity(), nil
},
FetchAmountPairCapacity: func(nodeFrom, nodeTo route.Vertex,
amount lnwire.MilliSatoshi) (btcutil.Amount, error) {
@ -713,7 +713,7 @@ func (r *rpcServer) addDeps(s *server, macService *macaroons.Service,
chanID, err)
}
return info.NodeKey1Bytes, info.NodeKey2Bytes, nil
return info.Node1Bytes(), info.Node1Bytes(), nil
},
FindRoute: s.chanRouter.FindRoute,
MissionControl: s.missionControl,
@ -6220,14 +6220,14 @@ func (r *rpcServer) DescribeGraph(ctx context.Context,
// Next, for each active channel we know of within the graph, create a
// similar response which details both the edge information as well as
// the routing policies of th nodes connecting the two edges.
err = graph.ForEachChannel(func(edgeInfo *models.ChannelEdgeInfo1,
err = graph.ForEachChannel(func(edgeInfo models.ChannelEdgeInfo,
c1, c2 *models.ChannelEdgePolicy1) error {
// Do not include unannounced channels unless specifically
// requested. Unannounced channels include both private channels as
// well as public channels whose authentication proof were not
// confirmed yet, hence were not announced.
if !includeUnannounced && edgeInfo.AuthProof == nil {
if !includeUnannounced && edgeInfo.GetAuthProof() == nil {
return nil
}
@ -6454,7 +6454,7 @@ func (r *rpcServer) GetChanInfo(_ context.Context,
graph := r.server.graphDB
var (
edgeInfo *models.ChannelEdgeInfo1
edgeInfo models.ChannelEdgeInfo
edge1, edge2 *models.ChannelEdgePolicy1
err error
)
@ -6527,11 +6527,11 @@ func (r *rpcServer) GetNodeInfo(ctx context.Context,
)
err = graph.ForEachNodeChannel(node.PubKeyBytes,
func(_ kvdb.RTx, edge *models.ChannelEdgeInfo1,
func(_ kvdb.RTx, edge models.ChannelEdgeInfo,
c1, c2 *models.ChannelEdgePolicy1) error {
numChannels++
totalCapacity += edge.Capacity
totalCapacity += edge.GetCapacity()
// Only populate the node's channels if the user
// requested them.
@ -6539,7 +6539,7 @@ func (r *rpcServer) GetNodeInfo(ctx context.Context,
// Do not include unannounced channels - private
// channels or public channels whose
// authentication proof were not confirmed yet.
if edge.AuthProof == nil {
if edge.GetAuthProof() == nil {
return nil
}
@ -7190,14 +7190,14 @@ func (r *rpcServer) FeeReport(ctx context.Context,
var feeReports []*lnrpc.ChannelFeeReport
err = channelGraph.ForEachNodeChannel(selfNode.PubKeyBytes,
func(_ kvdb.RTx, chanInfo *models.ChannelEdgeInfo1,
func(_ kvdb.RTx, chanInfo models.ChannelEdgeInfo,
edgePolicy, _ *models.ChannelEdgePolicy1) error {
// Self node should always have policies for its
// channels.
if edgePolicy == nil {
return fmt.Errorf("no policy for outgoing "+
"channel %v ", chanInfo.ChannelID)
"channel %v ", chanInfo.GetChanID())
}
// We'll compute the effective fee rate by converting
@ -7221,8 +7221,8 @@ func (r *rpcServer) FeeReport(ctx context.Context,
// TODO(roasbeef): also add stats for revenue for each
// channel
feeReports = append(feeReports, &lnrpc.ChannelFeeReport{
ChanId: chanInfo.ChannelID,
ChannelPoint: chanInfo.ChannelPoint.String(),
ChanId: chanInfo.GetChanID(),
ChannelPoint: chanInfo.GetChanPoint().String(),
BaseFeeMsat: int64(edgePolicy.FeeBaseMSat),
FeePerMil: int64(feeRateFixedPoint),
FeeRate: feeRate,

@ -1310,7 +1310,7 @@ func newServer(cfg *Config, listenAddrs []net.Addr,
copy(ourKey[:], nodeKeyDesc.PubKey.SerializeCompressed())
var ourPolicy *models.ChannelEdgePolicy1
if info != nil && info.NodeKey1Bytes == ourKey {
if info != nil && info.Node1Bytes() == ourKey {
ourPolicy = e1
} else {
ourPolicy = e2
@ -3229,15 +3229,17 @@ func (s *server) establishPersistentConnections() error {
selfPub := s.identityECDH.PubKey().SerializeCompressed()
err = s.graphDB.ForEachNodeChannel(sourceNode.PubKeyBytes, func(
tx kvdb.RTx,
chanInfo *models.ChannelEdgeInfo1,
chanInfo models.ChannelEdgeInfo,
policy, _ *models.ChannelEdgePolicy1) error {
chanPoint := chanInfo.GetChanPoint()
// If the remote party has announced the channel to us, but we
// haven't yet, then we won't have a policy. However, we don't
// need this to connect to the peer, so we'll log it and move on.
if policy == nil {
srvrLog.Warnf("No channel policy found for "+
"ChannelPoint(%v): ", chanInfo.ChannelPoint)
"ChannelPoint(%v): ", chanPoint)
}
// We'll now fetch the peer opposite from us within this
@ -3247,8 +3249,7 @@ func (s *server) establishPersistentConnections() error {
)
if err != nil {
return fmt.Errorf("unable to fetch channel peer for "+
"ChannelPoint(%v): %v", chanInfo.ChannelPoint,
err)
"ChannelPoint(%v): %v", chanPoint, err)
}
pubStr := string(channelPeer.PubKeyBytes[:])