Skip to content

Commit

Permalink
routerrpc+htlcswitch: move intercepted htlc tracking to switch
Browse files Browse the repository at this point in the history
In this commit we move the tracking of the outstanding intercepted htlcs
to InterceptableSwitch. This is a preparation for making the htlc
interceptor required.

Required interception involves tracking outstanding htlcs across
multiple grpc client sessions. The per-session routerrpc
forwardInterceptor object is therefore no longer the best place for
that.
  • Loading branch information
joostjager committed Mar 15, 2022
1 parent 95c270d commit 169f0c0
Show file tree
Hide file tree
Showing 6 changed files with 378 additions and 206 deletions.
292 changes: 251 additions & 41 deletions htlcswitch/interceptable_switch.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,80 +30,289 @@ var (
// Settle - routes UpdateFulfillHTLC to the originating link.
// Fail - routes UpdateFailHTLC to the originating link.
type InterceptableSwitch struct {
sync.RWMutex

// htlcSwitch is the underline switch
htlcSwitch *Switch

// fwdInterceptor is the callback that is called for each forward of
// an incoming htlc. It should return true if it is interested in handling
// it.
fwdInterceptor ForwardInterceptor
// intercepted is where we stream all intercepted packets coming from
// the switch.
intercepted chan *interceptedPackets

// resolutionChan is where we stream all responses coming from the
// interceptor client.
resolutionChan chan *fwdResolution

// interceptorRegistration is a channel that we use to synchronize
// client connect and disconnect.
interceptorRegistration chan ForwardInterceptor

// interceptor is the handler for intercepted packets.
interceptor ForwardInterceptor

// holdForwards keeps track of outstanding intercepted forwards.
holdForwards map[channeldb.CircuitKey]InterceptedForward

wg sync.WaitGroup
quit chan struct{}
}

type interceptedPackets struct {
packets []*htlcPacket
linkQuit chan struct{}
}

// FwdAction defines the various resolution types.
type FwdAction int

const (
// FwdActionResume forwards the intercepted packet to the switch.
FwdActionResume FwdAction = iota

// FwdActionSettle settles the intercepted packet with a preimage.
FwdActionSettle

// FwdActionFail fails the intercepted packet back to the sender.
FwdActionFail
)

// FwdResolution defines the action to be taken on an intercepted packet.
type FwdResolution struct {
// Key is the incoming circuit key of the htlc.
Key channeldb.CircuitKey

// Action is the action to take on the intercepted htlc.
Action FwdAction

// Preimage is the preimage that is to be used for settling if Action is
// FwdActionSettle.
Preimage lntypes.Preimage

// FailureMessage is the encrypted failure message that is to be passed
// back to the sender if action is FwdActionFail.
FailureMessage []byte

// FailureCode is the failure code that is to be passed back to the
// sender if action is FwdActionFail.
FailureCode lnwire.FailCode
}

type fwdResolution struct {
resolution *FwdResolution
errChan chan error
}

// NewInterceptableSwitch returns an instance of InterceptableSwitch.
func NewInterceptableSwitch(s *Switch) *InterceptableSwitch {
return &InterceptableSwitch{htlcSwitch: s}
return &InterceptableSwitch{
htlcSwitch: s,
intercepted: make(chan *interceptedPackets),
interceptorRegistration: make(chan ForwardInterceptor),
holdForwards: make(map[channeldb.CircuitKey]InterceptedForward),
resolutionChan: make(chan *fwdResolution),

quit: make(chan struct{}),
}
}

// SetInterceptor sets the ForwardInterceptor to be used.
// SetInterceptor sets the ForwardInterceptor to be used. A nil argument
// unregisters the current interceptor.
func (s *InterceptableSwitch) SetInterceptor(
interceptor ForwardInterceptor) {

s.Lock()
defer s.Unlock()
s.fwdInterceptor = interceptor
// Synchronize setting the handler with the main loop to prevent race
// conditions.
select {
case s.interceptorRegistration <- interceptor:

case <-s.quit:
}
}

// ForwardPackets attempts to forward the batch of htlcs through the
// switch, any failed packets will be returned to the provided
// ChannelLink. The link's quit signal should be provided to allow
// cancellation of forwarding during link shutdown.
func (s *InterceptableSwitch) ForwardPackets(linkQuit chan struct{},
packets ...*htlcPacket) error {
func (s *InterceptableSwitch) Start() error {
s.wg.Add(1)
go func() {
defer s.wg.Done()

s.run()
}()

return nil
}

var interceptor ForwardInterceptor
s.Lock()
interceptor = s.fwdInterceptor
s.Unlock()
func (s *InterceptableSwitch) Stop() error {
close(s.quit)
s.wg.Wait()

// Optimize for the case we don't have an interceptor.
if interceptor == nil {
return s.htlcSwitch.ForwardPackets(linkQuit, packets...)
return nil
}

func (s *InterceptableSwitch) run() {
for {
select {
// An interceptor registration or de-registration came in.
case interceptor := <-s.interceptorRegistration:
s.setInterceptor(interceptor)

case packets := <-s.intercepted:
var notIntercepted []*htlcPacket
for _, p := range packets.packets {
if s.interceptor == nil ||
!s.interceptForward(p) {

notIntercepted = append(
notIntercepted, p,
)
}
}
err := s.htlcSwitch.ForwardPackets(
packets.linkQuit, notIntercepted...,
)
if err != nil {
log.Errorf("Cannot forward packets: %v", err)
}

case res := <-s.resolutionChan:
res.errChan <- s.resolve(res.resolution)

case <-s.quit:
return
}
}
}

func (s *InterceptableSwitch) sendForward(fwd InterceptedForward) {
err := s.interceptor(fwd.Packet())
if err != nil {
// Only log the error. If we couldn't send the packet, we assume
// that the interceptor will reconnect so that we can retry.
log.Debugf("Interceptor cannot handle forward: %v", err)
}
}

func (s *InterceptableSwitch) setInterceptor(interceptor ForwardInterceptor) {
s.interceptor = interceptor

if interceptor != nil {
log.Debugf("Interceptor connected")

var notIntercepted []*htlcPacket
for _, p := range packets {
if !s.interceptForward(p, interceptor, linkQuit) {
notIntercepted = append(notIntercepted, p)
return
}

log.Infof("Interceptor disconnected, resolving held packets")

for _, fwd := range s.holdForwards {
if err := fwd.Resume(); err != nil {
log.Errorf("Failed to resume hold forward %v", err)
}
}
s.holdForwards = make(map[channeldb.CircuitKey]InterceptedForward)
}

func (s *InterceptableSwitch) resolve(res *FwdResolution) error {
intercepted, ok := s.holdForwards[res.Key]
if !ok {
return fmt.Errorf("fwd %v not found", res.Key)
}
delete(s.holdForwards, res.Key)

switch res.Action {
case FwdActionResume:
return intercepted.Resume()

case FwdActionSettle:
return intercepted.Settle(res.Preimage)

case FwdActionFail:
if len(res.FailureMessage) > 0 {
return intercepted.Fail(res.FailureMessage)
}

return intercepted.FailWithCode(res.FailureCode)

default:
return fmt.Errorf("unrecognized action %v", res.Action)
}
return s.htlcSwitch.ForwardPackets(linkQuit, notIntercepted...)
}

// interceptForward checks if there is any external interceptor interested in
// this packet. Currently only htlc type of UpdateAddHTLC that are forwarded
// are being checked for interception. It can be extended in the future given
// the right use case.
func (s *InterceptableSwitch) interceptForward(packet *htlcPacket,
interceptor ForwardInterceptor, linkQuit chan struct{}) bool {
// Resolve resolves an intercepted packet.
func (s *InterceptableSwitch) Resolve(res *FwdResolution) error {
internalRes := &fwdResolution{
resolution: res,
errChan: make(chan error, 1),
}

select {
case s.resolutionChan <- internalRes:

case <-s.quit:
return errors.New("switch shutting down")
}

select {
case err := <-internalRes.errChan:
return err

case <-s.quit:
return errors.New("switch shutting down")
}
}

// ForwardPackets attempts to forward the batch of htlcs to a connected
// interceptor. If the interceptor signals the resume action, the htlcs are
// forwarded to the switch. The link's quit signal should be provided to allow
// cancellation of forwarding during link shutdown.
func (s *InterceptableSwitch) ForwardPackets(linkQuit chan struct{},
packets ...*htlcPacket) error {

// Synchronize with the main event loop. This should be light in the
// case where there is no interceptor.
select {
case s.intercepted <- &interceptedPackets{
packets: packets,
linkQuit: linkQuit,
}:

case <-linkQuit:
log.Debugf("Forward cancelled because link quit")

case <-s.quit:
return errors.New("interceptable switch quit")
}

return nil
}

// interceptForward forwards the packet to the external interceptor after
// checking the interception criteria.
func (s *InterceptableSwitch) interceptForward(packet *htlcPacket) bool {
switch htlc := packet.htlc.(type) {
case *lnwire.UpdateAddHTLC:
// We are not interested in intercepting initiated payments.
if packet.incomingChanID == hop.Source {
return false
}

inKey := channeldb.CircuitKey{
ChanID: packet.incomingChanID,
HtlcID: packet.incomingHTLCID,
}

// Ignore already held htlcs.
if _, ok := s.holdForwards[inKey]; ok {
return true
}

intercepted := &interceptedForward{
linkQuit: linkQuit,
htlc: htlc,
packet: packet,
htlcSwitch: s.htlcSwitch,
}

// If this htlc was intercepted, don't handle the forward.
return interceptor(intercepted)
s.holdForwards[inKey] = intercepted

s.sendForward(intercepted)

return true

default:
return false
}
Expand All @@ -113,7 +322,6 @@ func (s *InterceptableSwitch) interceptForward(packet *htlcPacket,
// It is passed from the switch to external interceptors that are interested
// in holding forwards and resolve them manually.
type interceptedForward struct {
linkQuit chan struct{}
htlc *lnwire.UpdateAddHTLC
packet *htlcPacket
htlcSwitch *Switch
Expand All @@ -139,10 +347,12 @@ func (f *interceptedForward) Packet() InterceptedPacket {

// Resume resumes the default behavior as if the packet was not intercepted.
func (f *interceptedForward) Resume() error {
return f.htlcSwitch.ForwardPackets(f.linkQuit, f.packet)
// Forward to the switch. A link quit channel isn't needed, because we
// are on a different thread now.
return f.htlcSwitch.ForwardPackets(nil, f.packet)
}

// Fail notifies the intention to fail an existing hold forward with an
// Fail notifies the intention to Fail an existing hold forward with an
// encrypted failure reason.
func (f *interceptedForward) Fail(reason []byte) error {
obfuscatedReason := f.packet.obfuscator.IntermediateEncrypt(reason)
Expand Down
5 changes: 4 additions & 1 deletion htlcswitch/interfaces.go
Original file line number Diff line number Diff line change
Expand Up @@ -234,6 +234,9 @@ type TowerClient interface {
type InterceptableHtlcForwarder interface {
// SetInterceptor sets a ForwardInterceptor.
SetInterceptor(interceptor ForwardInterceptor)

// Resolve resolves an intercepted packet.
Resolve(res *FwdResolution) error
}

// ForwardInterceptor is a function that is invoked from the switch for every
Expand All @@ -242,7 +245,7 @@ type InterceptableHtlcForwarder interface {
// to resolve it manually later in case it is held.
// The return value indicates if this handler will take control of this forward
// and resolve it later or let the switch execute its default behavior.
type ForwardInterceptor func(InterceptedForward) bool
type ForwardInterceptor func(InterceptedPacket) error

// InterceptedPacket contains the relevant information for the interceptor about
// an htlc.
Expand Down
Loading

0 comments on commit 169f0c0

Please sign in to comment.