lnd: switch over internal indexes to use the new Channel ID's

This commit is contained in:
Olaoluwa Osuntokun 2017-04-16 15:41:11 -07:00
parent d74d3fa0f3
commit 60c0cebfd5
No known key found for this signature in database
GPG Key ID: 9CC5B105D03521A2
2 changed files with 102 additions and 95 deletions

@ -108,15 +108,15 @@ func (b *boundedLinkChan) restoreSlots(numSlots uint32) {
// metadata such as the current available bandwidth of the link (in satoshis) // metadata such as the current available bandwidth of the link (in satoshis)
// which aid the switch in optimally forwarding HTLCs. // which aid the switch in optimally forwarding HTLCs.
type link struct { type link struct {
chanID lnwire.ChannelID
capacity btcutil.Amount capacity btcutil.Amount
availableBandwidth int64 // atomic availableBandwidth int64 // atomic
*boundedLinkChan
peer *peer peer *peer
chanPoint *wire.OutPoint *boundedLinkChan
} }
// htlcPacket is a wrapper around an lnwire message which adds, times out, or // htlcPacket is a wrapper around an lnwire message which adds, times out, or
@ -127,7 +127,7 @@ type htlcPacket struct {
dest chainhash.Hash dest chainhash.Hash
srcLink wire.OutPoint srcLink lnwire.ChannelID
onion *sphinx.ProcessedPacket onion *sphinx.ProcessedPacket
msg lnwire.Message msg lnwire.Message
@ -192,11 +192,11 @@ type htlcSwitch struct {
started int32 // atomic started int32 // atomic
shutdown int32 // atomic shutdown int32 // atomic
// chanIndex maps a channel's outpoint to a link which contains // chanIndex maps a channel's ID to a link which contains additional
// additional information about the channel, and additionally houses a // information about the channel, and additionally houses a pointer to
// pointer to the peer managing the channel. // the peer managing the channel.
chanIndexMtx sync.RWMutex chanIndexMtx sync.RWMutex
chanIndex map[wire.OutPoint]*link chanIndex map[lnwire.ChannelID]*link
// interfaces maps a node's ID to the set of links (active channels) we // interfaces maps a node's ID to the set of links (active channels) we
// currently have open with that peer. // currently have open with that peer.
@ -240,7 +240,7 @@ type htlcSwitch struct {
// newHtlcSwitch creates a new htlcSwitch. // newHtlcSwitch creates a new htlcSwitch.
func newHtlcSwitch() *htlcSwitch { func newHtlcSwitch() *htlcSwitch {
return &htlcSwitch{ return &htlcSwitch{
chanIndex: make(map[wire.OutPoint]*link), chanIndex: make(map[lnwire.ChannelID]*link),
interfaces: make(map[chainhash.Hash][]*link), interfaces: make(map[chainhash.Hash][]*link),
onionIndex: make(map[[ripemd160.Size]byte][]*link), onionIndex: make(map[[ripemd160.Size]byte][]*link),
paymentCircuits: make(map[circuitKey]*paymentCircuit), paymentCircuits: make(map[circuitKey]*paymentCircuit),
@ -359,7 +359,7 @@ out:
n := atomic.AddInt64(&link.availableBandwidth, n := atomic.AddInt64(&link.availableBandwidth,
-int64(amt)) -int64(amt))
hswcLog.Tracef("Decrementing link %v bandwidth to %v", hswcLog.Tracef("Decrementing link %v bandwidth to %v",
link.chanPoint, n) link.chanID, n)
continue out continue out
} }
@ -435,7 +435,7 @@ out:
hswcLog.Errorf("unable to forward HTLC "+ hswcLog.Errorf("unable to forward HTLC "+
"link %v has insufficient "+ "link %v has insufficient "+
"capacity, have %v need %v", "capacity, have %v need %v",
clearLink[0].chanPoint, linkBandwidth, clearLink[0].chanID, linkBandwidth,
int64(wireMsg.Amount)) int64(wireMsg.Amount))
pkt := &htlcPacket{ pkt := &htlcPacket{
@ -462,8 +462,8 @@ out:
h.paymentCircuits[cKey] = circuit h.paymentCircuits[cKey] = circuit
hswcLog.Debugf("Creating onion circuit for %x: %v<->%v", hswcLog.Debugf("Creating onion circuit for %x: %v<->%v",
cKey[:], clearLink[0].chanPoint, cKey[:], clearLink[0].chanID,
settleLink.chanPoint) settleLink.chanID)
// With the circuit initiated, send the htlcPkt // With the circuit initiated, send the htlcPkt
// to the clearing link within the circuit to // to the clearing link within the circuit to
@ -482,7 +482,7 @@ out:
n := atomic.AddInt64(&circuit.clear.availableBandwidth, n := atomic.AddInt64(&circuit.clear.availableBandwidth,
-int64(pkt.amt)) -int64(pkt.amt))
hswcLog.Tracef("Decrementing link %v bandwidth to %v", hswcLog.Tracef("Decrementing link %v bandwidth to %v",
circuit.clear.chanPoint, n) circuit.clear.chanID, n)
deltaSatRecv += pkt.amt deltaSatRecv += pkt.amt
@ -510,8 +510,8 @@ out:
hswcLog.Debugf("Closing completed onion "+ hswcLog.Debugf("Closing completed onion "+
"circuit for %x: %v<->%v", rHash[:], "circuit for %x: %v<->%v", rHash[:],
circuit.clear.chanPoint, circuit.clear.chanID,
circuit.settle.chanPoint) circuit.settle.chanID)
circuit.settle.sendAndRestore(&htlcPacket{ circuit.settle.sendAndRestore(&htlcPacket{
msg: wireMsg, msg: wireMsg,
@ -525,7 +525,7 @@ out:
n := atomic.AddInt64(&circuit.settle.availableBandwidth, n := atomic.AddInt64(&circuit.settle.availableBandwidth,
int64(pkt.amt)) int64(pkt.amt))
hswcLog.Tracef("Incrementing link %v bandwidth to %v", hswcLog.Tracef("Incrementing link %v bandwidth to %v",
circuit.settle.chanPoint, n) circuit.settle.chanID, n)
deltaSatSent += pkt.amt deltaSatSent += pkt.amt
@ -556,7 +556,7 @@ out:
int64(pkt.amt)) int64(pkt.amt))
hswcLog.Debugf("HTLC %x has been cancelled, "+ hswcLog.Debugf("HTLC %x has been cancelled, "+
"incrementing link %v bandwidth to %v", pkt.payHash, "incrementing link %v bandwidth to %v", pkt.payHash,
circuit.clear.chanPoint, n) circuit.clear.chanID, n)
// With our link info updated, we now continue // With our link info updated, we now continue
// the error propagation by sending the // the error propagation by sending the
@ -636,11 +636,12 @@ out:
// adds the link to the existing set of links for the target interface. // adds the link to the existing set of links for the target interface.
func (h *htlcSwitch) handleRegisterLink(req *registerLinkMsg) { func (h *htlcSwitch) handleRegisterLink(req *registerLinkMsg) {
chanPoint := req.linkInfo.ChannelPoint chanPoint := req.linkInfo.ChannelPoint
chanID := lnwire.NewChanIDFromOutPoint(chanPoint)
newLink := &link{ newLink := &link{
capacity: req.linkInfo.Capacity, capacity: req.linkInfo.Capacity,
availableBandwidth: int64(req.linkInfo.LocalBalance), availableBandwidth: int64(req.linkInfo.LocalBalance),
peer: req.peer, peer: req.peer,
chanPoint: chanPoint, chanID: chanID,
} }
// To ensure we never accidentally cause an HTLC overflow, we'll limit, // To ensure we never accidentally cause an HTLC overflow, we'll limit,
@ -655,7 +656,7 @@ func (h *htlcSwitch) handleRegisterLink(req *registerLinkMsg) {
// close them, update their link capacity, or possibly during multi-hop // close them, update their link capacity, or possibly during multi-hop
// HTLC forwarding. // HTLC forwarding.
h.chanIndexMtx.Lock() h.chanIndexMtx.Lock()
h.chanIndex[*chanPoint] = newLink h.chanIndex[chanID] = newLink
h.chanIndexMtx.Unlock() h.chanIndexMtx.Unlock()
interfaceID := req.peer.lightningID interfaceID := req.peer.lightningID
@ -687,8 +688,8 @@ func (h *htlcSwitch) handleRegisterLink(req *registerLinkMsg) {
// this link leaves the interface empty, then the interface entry itself is // this link leaves the interface empty, then the interface entry itself is
// also deleted. // also deleted.
func (h *htlcSwitch) handleUnregisterLink(req *unregisterLinkMsg) { func (h *htlcSwitch) handleUnregisterLink(req *unregisterLinkMsg) {
hswcLog.Debugf("unregistering active link, interface=%v, chan_point=%v", hswcLog.Debugf("unregistering active link, interface=%v, chan_id=%v",
hex.EncodeToString(req.chanInterface[:]), req.chanPoint) hex.EncodeToString(req.chanInterface[:]), req.chanID)
chanInterface := req.chanInterface chanInterface := req.chanInterface
@ -704,21 +705,21 @@ func (h *htlcSwitch) handleUnregisterLink(req *unregisterLinkMsg) {
// A request with a nil channel point indicates that all the current // A request with a nil channel point indicates that all the current
// links for this channel should be cleared. // links for this channel should be cleared.
if req.chanPoint == nil { if req.chanID == nil {
hswcLog.Debugf("purging all active links for interface %v", hswcLog.Debugf("purging all active links for interface %v",
hex.EncodeToString(chanInterface[:])) hex.EncodeToString(chanInterface[:]))
for _, link := range links { for _, link := range links {
delete(h.chanIndex, *link.chanPoint) delete(h.chanIndex, link.chanID)
} }
links = nil links = nil
} else { } else {
delete(h.chanIndex, *req.chanPoint) delete(h.chanIndex, *req.chanID)
for i := 0; i < len(links); i++ { for i := 0; i < len(links); i++ {
chanLink := links[i] chanLink := links[i]
if chanLink.chanPoint == req.chanPoint { if chanLink.chanID == *req.chanID {
// We perform an in-place delete by sliding // We perform an in-place delete by sliding
// every element down one, then slicing off the // every element down one, then slicing off the
// last element. Additionally, we update the // last element. Additionally, we update the
@ -762,18 +763,20 @@ func (h *htlcSwitch) handleUnregisterLink(req *unregisterLinkMsg) {
// handleCloseLink sends a message to the peer responsible for the target // handleCloseLink sends a message to the peer responsible for the target
// channel point, instructing it to initiate a cooperative channel closure. // channel point, instructing it to initiate a cooperative channel closure.
func (h *htlcSwitch) handleCloseLink(req *closeLinkReq) { func (h *htlcSwitch) handleCloseLink(req *closeLinkReq) {
chanID := lnwire.NewChanIDFromOutPoint(req.chanPoint)
h.chanIndexMtx.RLock() h.chanIndexMtx.RLock()
targetLink, ok := h.chanIndex[*req.chanPoint] targetLink, ok := h.chanIndex[chanID]
h.chanIndexMtx.RUnlock() h.chanIndexMtx.RUnlock()
if !ok { if !ok {
req.err <- fmt.Errorf("channel point %v not found, or peer "+ req.err <- fmt.Errorf("channel %v not found, or peer "+
"offline", req.chanPoint) "offline", req.chanPoint)
return return
} }
hswcLog.Debugf("requesting interface %v to close link %v", hswcLog.Debugf("requesting interface %v to close link %v",
hex.EncodeToString(targetLink.peer.lightningID[:]), req.chanPoint) hex.EncodeToString(targetLink.peer.lightningID[:]), chanID)
targetLink.peer.localCloseChanReqs <- req targetLink.peer.localCloseChanReqs <- req
// TODO(roasbeef): if type was CloseBreach initiate force closure with // TODO(roasbeef): if type was CloseBreach initiate force closure with
@ -784,7 +787,7 @@ func (h *htlcSwitch) handleCloseLink(req *closeLinkReq) {
// channel's available bandwidth by the delta specified within the message. // channel's available bandwidth by the delta specified within the message.
func (h *htlcSwitch) handleLinkUpdate(req *linkInfoUpdateMsg) { func (h *htlcSwitch) handleLinkUpdate(req *linkInfoUpdateMsg) {
h.chanIndexMtx.RLock() h.chanIndexMtx.RLock()
link, ok := h.chanIndex[*req.targetLink] link, ok := h.chanIndex[req.targetLink]
h.chanIndexMtx.RUnlock() h.chanIndexMtx.RUnlock()
if !ok { if !ok {
hswcLog.Errorf("received link update for non-existent link: %v", hswcLog.Errorf("received link update for non-existent link: %v",
@ -828,7 +831,7 @@ func (h *htlcSwitch) RegisterLink(p *peer, linkInfo *channeldb.ChannelSnapshot,
// unregisterLinkMsg is a message which requests the active link be unregistered. // unregisterLinkMsg is a message which requests the active link be unregistered.
type unregisterLinkMsg struct { type unregisterLinkMsg struct {
chanInterface [32]byte chanInterface [32]byte
chanPoint *wire.OutPoint chanID *lnwire.ChannelID
// remoteID is the identity public key of the node we're removing the // remoteID is the identity public key of the node we're removing the
// link between. The public key is expected to be serialized in // link between. The public key is expected to be serialized in
@ -842,13 +845,15 @@ type unregisterLinkMsg struct {
// UnregisterLink requests the htlcSwitch to register the new active link. An // UnregisterLink requests the htlcSwitch to register the new active link. An
// unregistered link will no longer be considered a candidate to forward // unregistered link will no longer be considered a candidate to forward
// HTLCs. // HTLCs.
func (h *htlcSwitch) UnregisterLink(remotePub *btcec.PublicKey, chanPoint *wire.OutPoint) { func (h *htlcSwitch) UnregisterLink(remotePub *btcec.PublicKey,
chanID *lnwire.ChannelID) {
done := make(chan struct{}, 1) done := make(chan struct{}, 1)
rawPub := remotePub.SerializeCompressed() rawPub := remotePub.SerializeCompressed()
h.linkControl <- &unregisterLinkMsg{ h.linkControl <- &unregisterLinkMsg{
chanInterface: sha256.Sum256(rawPub), chanInterface: sha256.Sum256(rawPub),
chanPoint: chanPoint, chanID: chanID,
remoteID: rawPub, remoteID: rawPub,
done: done, done: done,
} }
@ -904,7 +909,7 @@ func (h *htlcSwitch) CloseLink(chanPoint *wire.OutPoint,
// linkInfoUpdateMsg encapsulates a request for the htlc switch to update the // linkInfoUpdateMsg encapsulates a request for the htlc switch to update the
// metadata related to the target link. // metadata related to the target link.
type linkInfoUpdateMsg struct { type linkInfoUpdateMsg struct {
targetLink *wire.OutPoint targetLink lnwire.ChannelID
bandwidthDelta btcutil.Amount bandwidthDelta btcutil.Amount
} }
@ -913,9 +918,9 @@ type linkInfoUpdateMsg struct {
// within the link by the passed satoshi delta. This function may be used when // within the link by the passed satoshi delta. This function may be used when
// re-anchoring to boost the capacity of a channel, or once a peer settles an // re-anchoring to boost the capacity of a channel, or once a peer settles an
// HTLC invoice. // HTLC invoice.
func (h *htlcSwitch) UpdateLink(chanPoint *wire.OutPoint, delta btcutil.Amount) { func (h *htlcSwitch) UpdateLink(chanID lnwire.ChannelID, delta btcutil.Amount) {
h.linkControl <- &linkInfoUpdateMsg{ h.linkControl <- &linkInfoUpdateMsg{
targetLink: chanPoint, targetLink: chanID,
bandwidthDelta: delta, bandwidthDelta: delta,
} }
} }

120
peer.go

@ -118,11 +118,11 @@ type peer struct {
// active channels. Channels are indexed into the map by the txid of // active channels. Channels are indexed into the map by the txid of
// the funding transaction which opened the channel. // the funding transaction which opened the channel.
activeChanMtx sync.RWMutex activeChanMtx sync.RWMutex
activeChannels map[wire.OutPoint]*lnwallet.LightningChannel activeChannels map[lnwire.ChannelID]*lnwallet.LightningChannel
chanSnapshotReqs chan *chanSnapshotReq chanSnapshotReqs chan *chanSnapshotReq
htlcManMtx sync.RWMutex htlcManMtx sync.RWMutex
htlcManagers map[wire.OutPoint]chan lnwire.Message htlcManagers map[lnwire.ChannelID]chan lnwire.Message
// newChannels is used by the fundingManager to send fully opened // newChannels is used by the fundingManager to send fully opened
// channels to the source peer which handled the funding workflow. // channels to the source peer which handled the funding workflow.
@ -176,8 +176,8 @@ func newPeer(conn net.Conn, connReq *connmgr.ConnReq, server *server,
sendQueue: make(chan outgoinMsg, 1), sendQueue: make(chan outgoinMsg, 1),
outgoingQueue: make(chan outgoinMsg, outgoingQueueLen), outgoingQueue: make(chan outgoinMsg, outgoingQueueLen),
activeChannels: make(map[wire.OutPoint]*lnwallet.LightningChannel), activeChannels: make(map[lnwire.ChannelID]*lnwallet.LightningChannel),
htlcManagers: make(map[wire.OutPoint]chan lnwire.Message), htlcManagers: make(map[lnwire.ChannelID]chan lnwire.Message),
chanSnapshotReqs: make(chan *chanSnapshotReq), chanSnapshotReqs: make(chan *chanSnapshotReq),
newChannels: make(chan *newChannelMsg, 1), newChannels: make(chan *newChannelMsg, 1),
@ -216,19 +216,17 @@ func (p *peer) loadActiveChannels(chans []*channeldb.OpenChannel) error {
continue continue
} }
chanID := dbChan.ChanID
lnChan, err := lnwallet.NewLightningChannel(p.server.lnwallet.Signer, lnChan, err := lnwallet.NewLightningChannel(p.server.lnwallet.Signer,
p.server.chainNotifier, dbChan) p.server.chainNotifier, dbChan)
if err != nil { if err != nil {
return err return err
} }
chanPoint := wire.OutPoint{ chanPoint := *dbChan.ChanID
Hash: chanID.Hash, chanID := lnwire.NewChanIDFromOutPoint(&chanPoint)
Index: chanID.Index,
}
p.activeChanMtx.Lock() p.activeChanMtx.Lock()
p.activeChannels[chanPoint] = lnChan p.activeChannels[chanID] = lnChan
p.activeChanMtx.Unlock() p.activeChanMtx.Unlock()
peerLog.Infof("peerID(%v) loaded ChannelPoint(%v)", p.id, chanPoint) peerLog.Infof("peerID(%v) loaded ChannelPoint(%v)", p.id, chanPoint)
@ -244,7 +242,7 @@ func (p *peer) loadActiveChannels(chans []*channeldb.OpenChannel) error {
upstreamLink := make(chan lnwire.Message, 10) upstreamLink := make(chan lnwire.Message, 10)
p.htlcManMtx.Lock() p.htlcManMtx.Lock()
p.htlcManagers[chanPoint] = upstreamLink p.htlcManagers[chanID] = upstreamLink
p.htlcManMtx.Unlock() p.htlcManMtx.Unlock()
p.wg.Add(1) p.wg.Add(1)
@ -421,7 +419,7 @@ out:
var ( var (
isChanUpdate bool isChanUpdate bool
targetChan wire.OutPoint targetChan lnwire.ChannelID
) )
switch msg := nextMsg.(type) { switch msg := nextMsg.(type) {
@ -450,25 +448,25 @@ out:
case *lnwire.CloseRequest: case *lnwire.CloseRequest:
p.remoteCloseChanReqs <- msg p.remoteCloseChanReqs <- msg
case *lnwire.ErrorGeneric: case *lnwire.Error:
p.server.fundingMgr.processErrorGeneric(msg, p.addr) p.server.fundingMgr.processFundingError(msg, p.addr)
// TODO(roasbeef): create ChanUpdater interface for the below // TODO(roasbeef): create ChanUpdater interface for the below
case *lnwire.UpdateAddHTLC: case *lnwire.UpdateAddHTLC:
isChanUpdate = true isChanUpdate = true
targetChan = msg.ChannelPoint targetChan = msg.ChanID
case *lnwire.UpdateFufillHTLC: case *lnwire.UpdateFufillHTLC:
isChanUpdate = true isChanUpdate = true
targetChan = msg.ChannelPoint targetChan = msg.ChanID
case *lnwire.UpdateFailHTLC: case *lnwire.UpdateFailHTLC:
isChanUpdate = true isChanUpdate = true
targetChan = msg.ChannelPoint targetChan = msg.ChanID
case *lnwire.RevokeAndAck: case *lnwire.RevokeAndAck:
isChanUpdate = true isChanUpdate = true
targetChan = msg.ChannelPoint targetChan = msg.ChanID
case *lnwire.CommitSig: case *lnwire.CommitSig:
isChanUpdate = true isChanUpdate = true
targetChan = msg.ChannelPoint targetChan = msg.ChanID
case *lnwire.ChannelUpdateAnnouncement, case *lnwire.ChannelUpdateAnnouncement,
*lnwire.ChannelAnnouncement, *lnwire.ChannelAnnouncement,
@ -721,7 +719,8 @@ out:
select { select {
case req := <-p.chanSnapshotReqs: case req := <-p.chanSnapshotReqs:
p.activeChanMtx.RLock() p.activeChanMtx.RLock()
snapshots := make([]*channeldb.ChannelSnapshot, 0, len(p.activeChannels)) snapshots := make([]*channeldb.ChannelSnapshot, 0,
len(p.activeChannels))
for _, activeChan := range p.activeChannels { for _, activeChan := range p.activeChannels {
snapshot := activeChan.StateSnapshot() snapshot := activeChan.StateSnapshot()
snapshots = append(snapshots, snapshot) snapshots = append(snapshots, snapshot)
@ -730,10 +729,11 @@ out:
req.resp <- snapshots req.resp <- snapshots
case newChanReq := <-p.newChannels: case newChanReq := <-p.newChannels:
chanPoint := *newChanReq.channel.ChannelPoint() chanPoint := newChanReq.channel.ChannelPoint()
chanID := lnwire.NewChanIDFromOutPoint(chanPoint)
p.activeChanMtx.Lock() p.activeChanMtx.Lock()
p.activeChannels[chanPoint] = newChanReq.channel p.activeChannels[chanID] = newChanReq.channel
p.activeChanMtx.Unlock() p.activeChanMtx.Unlock()
peerLog.Infof("New channel active ChannelPoint(%v) "+ peerLog.Infof("New channel active ChannelPoint(%v) "+
@ -753,11 +753,12 @@ out:
// new channel. // new channel.
upstreamLink := make(chan lnwire.Message, 10) upstreamLink := make(chan lnwire.Message, 10)
p.htlcManMtx.Lock() p.htlcManMtx.Lock()
p.htlcManagers[chanPoint] = upstreamLink p.htlcManagers[chanID] = upstreamLink
p.htlcManMtx.Unlock() p.htlcManMtx.Unlock()
p.wg.Add(1) p.wg.Add(1)
go p.htlcManager(newChanReq.channel, plexChan, downstreamLink, upstreamLink) go p.htlcManager(newChanReq.channel, plexChan,
downstreamLink, upstreamLink)
close(newChanReq.done) close(newChanReq.done)
@ -802,7 +803,9 @@ func (p *peer) executeCooperativeClose(channel *lnwallet.LightningChannel) (*cha
if err != nil { if err != nil {
return nil, err return nil, err
} }
closeReq := lnwire.NewCloseRequest(*chanPoint, closeSig)
chanID := lnwire.NewChanIDFromOutPoint(chanPoint)
closeReq := lnwire.NewCloseRequest(chanID, closeSig)
p.queueMsg(closeReq, nil) p.queueMsg(closeReq, nil)
return txid, nil return txid, nil
@ -818,8 +821,10 @@ func (p *peer) handleLocalClose(req *closeLinkReq) {
closingTxid *chainhash.Hash closingTxid *chainhash.Hash
) )
chanID := lnwire.NewChanIDFromOutPoint(req.chanPoint)
p.activeChanMtx.RLock() p.activeChanMtx.RLock()
channel := p.activeChannels[*req.chanPoint] channel := p.activeChannels[chanID]
p.activeChanMtx.RUnlock() p.activeChanMtx.RUnlock()
switch req.CloseType { switch req.CloseType {
@ -911,21 +916,17 @@ func (p *peer) handleLocalClose(req *closeLinkReq) {
// handleRemoteClose completes a request for cooperative channel closure // handleRemoteClose completes a request for cooperative channel closure
// initiated by the remote node. // initiated by the remote node.
func (p *peer) handleRemoteClose(req *lnwire.CloseRequest) { func (p *peer) handleRemoteClose(req *lnwire.CloseRequest) {
chanPoint := req.ChannelPoint
key := wire.OutPoint{
Hash: chanPoint.Hash,
Index: chanPoint.Index,
}
p.activeChanMtx.RLock() p.activeChanMtx.RLock()
channel, ok := p.activeChannels[key] channel, ok := p.activeChannels[req.ChanID]
p.activeChanMtx.RUnlock() p.activeChanMtx.RUnlock()
if !ok { if !ok {
peerLog.Errorf("unable to close channel, ChannelPoint(%v) is "+ peerLog.Errorf("unable to close channel, ChannelID(%v) is "+
"unknown", key) "unknown", req.ChanID)
return return
} }
chanPoint := channel.ChannelPoint()
// Now that we have their signature for the closure transaction, we // Now that we have their signature for the closure transaction, we
// can assemble the final closure transaction, complete with our // can assemble the final closure transaction, complete with our
// signature. // signature.
@ -955,27 +956,26 @@ func (p *peer) handleRemoteClose(req *lnwire.CloseRequest) {
} }
// TODO(roasbeef): also wait for confs before removing state // TODO(roasbeef): also wait for confs before removing state
peerLog.Infof("ChannelPoint(%v) is now "+ peerLog.Infof("ChannelPoint(%v) is now closed", chanPoint)
"closed", key)
if err := wipeChannel(p, channel); err != nil { if err := wipeChannel(p, channel); err != nil {
peerLog.Errorf("unable to wipe channel: %v", err) peerLog.Errorf("unable to wipe channel: %v", err)
} }
p.server.breachArbiter.settledContracts <- &req.ChannelPoint p.server.breachArbiter.settledContracts <- chanPoint
} }
// wipeChannel removes the passed channel from all indexes associated with the // wipeChannel removes the passed channel from all indexes associated with the
// peer, and deletes the channel from the database. // peer, and deletes the channel from the database.
func wipeChannel(p *peer, channel *lnwallet.LightningChannel) error { func wipeChannel(p *peer, channel *lnwallet.LightningChannel) error {
chanID := channel.ChannelPoint() chanID := lnwire.NewChanIDFromOutPoint(channel.ChannelPoint())
p.activeChanMtx.Lock() p.activeChanMtx.Lock()
delete(p.activeChannels, *chanID) delete(p.activeChannels, chanID)
p.activeChanMtx.Unlock() p.activeChanMtx.Unlock()
// Instruct the Htlc Switch to close this link as the channel is no // Instruct the Htlc Switch to close this link as the channel is no
// longer active. // longer active.
p.server.htlcSwitch.UnregisterLink(p.addr.IdentityKey, chanID) p.server.htlcSwitch.UnregisterLink(p.addr.IdentityKey, &chanID)
// Additionally, close up "down stream" link for the htlcManager which // Additionally, close up "down stream" link for the htlcManager which
// has been assigned to this channel. This servers the link between the // has been assigned to this channel. This servers the link between the
@ -985,7 +985,7 @@ func wipeChannel(p *peer, channel *lnwallet.LightningChannel) error {
// If the channel can't be found in the map, then this channel has // If the channel can't be found in the map, then this channel has
// already been wiped. // already been wiped.
htlcWireLink, ok := p.htlcManagers[*chanID] htlcWireLink, ok := p.htlcManagers[chanID]
if !ok { if !ok {
p.htlcManMtx.RUnlock() p.htlcManMtx.RUnlock()
return nil return nil
@ -999,7 +999,7 @@ func wipeChannel(p *peer, channel *lnwallet.LightningChannel) error {
// goroutine should have exited gracefully due to the channel closure // goroutine should have exited gracefully due to the channel closure
// above. // above.
p.htlcManMtx.RLock() p.htlcManMtx.RLock()
delete(p.htlcManagers, *chanID) delete(p.htlcManagers, chanID)
p.htlcManMtx.RUnlock() p.htlcManMtx.RUnlock()
// Finally, we purge the channel's state from the database, leaving a // Finally, we purge the channel's state from the database, leaving a
@ -1072,6 +1072,7 @@ type commitmentState struct {
channel *lnwallet.LightningChannel channel *lnwallet.LightningChannel
chanPoint *wire.OutPoint chanPoint *wire.OutPoint
chanID lnwire.ChannelID
} }
// htlcManager is the primary goroutine which drives a channel's commitment // htlcManager is the primary goroutine which drives a channel's commitment
@ -1104,9 +1105,11 @@ func (p *peer) htlcManager(channel *lnwallet.LightningChannel,
p.queueMsg(rev, nil) p.queueMsg(rev, nil)
} }
chanPoint := channel.ChannelPoint()
state := &commitmentState{ state := &commitmentState{
channel: channel, channel: channel,
chanPoint: channel.ChannelPoint(), chanPoint: chanPoint,
chanID: lnwire.NewChanIDFromOutPoint(chanPoint),
clearedHTCLs: make(map[uint64]*pendingPayment), clearedHTCLs: make(map[uint64]*pendingPayment),
htlcsToSettle: make(map[uint64]*channeldb.Invoice), htlcsToSettle: make(map[uint64]*channeldb.Invoice),
htlcsToCancel: make(map[uint64]lnwire.FailCode), htlcsToCancel: make(map[uint64]lnwire.FailCode),
@ -1253,7 +1256,7 @@ func (p *peer) handleDownStreamPkt(state *commitmentState, pkt *htlcPacket) {
// downstream channel, so we add the new HTLC // downstream channel, so we add the new HTLC
// to our local log, then update the commitment // to our local log, then update the commitment
// chains. // chains.
htlc.ChannelPoint = *state.chanPoint htlc.ChanID = state.chanID
index, err := state.channel.AddHTLC(htlc) index, err := state.channel.AddHTLC(htlc)
if err != nil { if err != nil {
// TODO: possibly perform fallback/retry logic // TODO: possibly perform fallback/retry logic
@ -1272,7 +1275,7 @@ func (p *peer) handleDownStreamPkt(state *commitmentState, pkt *htlcPacket) {
msg: &lnwire.UpdateFailHTLC{ msg: &lnwire.UpdateFailHTLC{
Reason: []byte{byte(0)}, Reason: []byte{byte(0)},
}, },
srcLink: *state.chanPoint, srcLink: state.chanID,
} }
return return
} }
@ -1303,7 +1306,7 @@ func (p *peer) handleDownStreamPkt(state *commitmentState, pkt *htlcPacket) {
// With the HTLC settled, we'll need to populate the wire // With the HTLC settled, we'll need to populate the wire
// message to target the specific channel and HTLC to be // message to target the specific channel and HTLC to be
// cancelled. // cancelled.
htlc.ChannelPoint = *state.chanPoint htlc.ChanID = state.chanID
htlc.ID = logIndex htlc.ID = logIndex
// Then we send the HTLC settle message to the connected peer // Then we send the HTLC settle message to the connected peer
@ -1324,7 +1327,7 @@ func (p *peer) handleDownStreamPkt(state *commitmentState, pkt *htlcPacket) {
// message to target the specific channel and HTLC to be // message to target the specific channel and HTLC to be
// cancelled. The "Reason" field will have already been set // cancelled. The "Reason" field will have already been set
// within the switch. // within the switch.
htlc.ChannelPoint = *state.chanPoint htlc.ChanID = state.chanID
htlc.ID = logIndex htlc.ID = logIndex
// Finally, we send the HTLC message to the peer which // Finally, we send the HTLC message to the peer which
@ -1559,7 +1562,7 @@ func (p *peer) handleUpstreamMsg(state *commitmentState, msg lnwire.Message) {
} }
settleMsg := &lnwire.UpdateFufillHTLC{ settleMsg := &lnwire.UpdateFufillHTLC{
ChannelPoint: *state.chanPoint, ChanID: state.chanID,
ID: logIndex, ID: logIndex,
PaymentPreimage: preimage, PaymentPreimage: preimage,
} }
@ -1589,9 +1592,9 @@ func (p *peer) handleUpstreamMsg(state *commitmentState, msg lnwire.Message) {
} }
cancelMsg := &lnwire.UpdateFailHTLC{ cancelMsg := &lnwire.UpdateFailHTLC{
ChannelPoint: *state.chanPoint, ChanID: state.chanID,
ID: logIndex, ID: logIndex,
Reason: []byte{byte(reason)}, Reason: []byte{byte(reason)},
} }
p.queueMsg(cancelMsg, nil) p.queueMsg(cancelMsg, nil)
delete(state.htlcsToCancel, htlc.Index) delete(state.htlcsToCancel, htlc.Index)
@ -1619,7 +1622,7 @@ func (p *peer) handleUpstreamMsg(state *commitmentState, msg lnwire.Message) {
// Send this fully activated HTLC to the htlc // Send this fully activated HTLC to the htlc
// switch to continue the chained clear/settle. // switch to continue the chained clear/settle.
pkt, err := logEntryToHtlcPkt(*state.chanPoint, pkt, err := logEntryToHtlcPkt(state.chanID,
htlc, onionPkt, reason) htlc, onionPkt, reason)
if err != nil { if err != nil {
peerLog.Errorf("unable to make htlc pkt: %v", peerLog.Errorf("unable to make htlc pkt: %v",
@ -1640,7 +1643,7 @@ func (p *peer) handleUpstreamMsg(state *commitmentState, msg lnwire.Message) {
// payment bandwidth. // payment bandwidth.
// TODO(roasbeef): ideally should wait for next state update. // TODO(roasbeef): ideally should wait for next state update.
if bandwidthUpdate != 0 { if bandwidthUpdate != 0 {
p.server.htlcSwitch.UpdateLink(state.chanPoint, p.server.htlcSwitch.UpdateLink(state.chanID,
bandwidthUpdate) bandwidthUpdate)
} }
@ -1684,8 +1687,8 @@ func (p *peer) updateCommitTx(state *commitmentState) error {
} }
commitSig := &lnwire.CommitSig{ commitSig := &lnwire.CommitSig{
ChannelPoint: *state.chanPoint, ChanID: state.chanID,
CommitSig: parsedSig, CommitSig: parsedSig,
} }
p.queueMsg(commitSig, nil) p.queueMsg(commitSig, nil)
@ -1707,8 +1710,7 @@ func (p *peer) updateCommitTx(state *commitmentState) error {
// log entry the corresponding htlcPacket with src/dest set along with the // log entry the corresponding htlcPacket with src/dest set along with the
// proper wire message. This helper method is provided in order to aid an // proper wire message. This helper method is provided in order to aid an
// htlcManager in forwarding packets to the htlcSwitch. // htlcManager in forwarding packets to the htlcSwitch.
func logEntryToHtlcPkt(chanPoint wire.OutPoint, func logEntryToHtlcPkt(chanID lnwire.ChannelID, pd *lnwallet.PaymentDescriptor,
pd *lnwallet.PaymentDescriptor,
onionPkt *sphinx.ProcessedPacket, onionPkt *sphinx.ProcessedPacket,
reason lnwire.FailCode) (*htlcPacket, error) { reason lnwire.FailCode) (*htlcPacket, error) {
@ -1750,7 +1752,7 @@ func logEntryToHtlcPkt(chanPoint wire.OutPoint,
pkt.amt = pd.Amount pkt.amt = pd.Amount
pkt.msg = msg pkt.msg = msg
pkt.srcLink = chanPoint pkt.srcLink = chanID
pkt.onion = onionPkt pkt.onion = onionPkt
return pkt, nil return pkt, nil