1
0
Fork 0
mirror of https://github.com/ACINQ/eclair.git synced 2025-03-13 11:35:47 +01:00

Merge commit 'd9f257ada' into android

This commit is contained in:
pm47 2020-10-08 18:30:57 +02:00
commit 2735e0f419
No known key found for this signature in database
GPG key ID: E434ED292E85643A
23 changed files with 775 additions and 946 deletions

View file

@ -44,6 +44,8 @@ eclair {
gossip_queries = optional
gossip_queries_ex = optional
var_onion_optin = optional
payment_secret = optional
basic_mpp = optional
}
override-features = [ // optional per-node features
# {

View file

@ -248,7 +248,7 @@ class Setup(datadir: File,
_ <- postRestartCleanUpInitialized.future
switchboard = system.actorOf(SimpleSupervisor.props(Switchboard.props(nodeParams, watcher, relayer, wallet), "switchboard", SupervisorStrategy.Resume))
clientSpawner = system.actorOf(SimpleSupervisor.props(ClientSpawner.props(nodeParams, switchboard, router), "client-spawner", SupervisorStrategy.Restart))
paymentInitiator = system.actorOf(SimpleSupervisor.props(PaymentInitiator.props(nodeParams, router, relayer, register), "payment-initiator", SupervisorStrategy.Restart))
paymentInitiator = system.actorOf(SimpleSupervisor.props(PaymentInitiator.props(nodeParams, router, register), "payment-initiator", SupervisorStrategy.Restart))
kit = Kit(
nodeParams = nodeParams,

View file

@ -27,9 +27,14 @@ object Monitoring {
val PaymentParts = Kamon.histogram("payment.parts", "Number of HTLCs per payment (MPP)")
val PaymentFailed = Kamon.counter("payment.failed", "Number of failed payment")
val PaymentError = Kamon.counter("payment.error", "Non-fatal errors encountered during payment attempts")
val PaymentAttempt = Kamon.histogram("payment.attempt", "Number of attempts before a payment succeeds")
val SentPaymentDuration = Kamon.timer("payment.duration.sent", "Outgoing payment duration")
val ReceivedPaymentDuration = Kamon.timer("payment.duration.received", "Incoming payment duration")
// The goal of this metric is to measure whether retrying MPP payments on failing channels yields useful results.
// Once enough data has been collected, we will update the MultiPartPaymentLifecycle logic accordingly.
val RetryFailedChannelsResult = Kamon.counter("payment.mpp.retry-failed-channels-result")
def recordPaymentRelayFailed(failureType: String, relayType: String): Unit =
Metrics.PaymentFailed
.withTag(Tags.Direction, Tags.Directions.Relayed)

View file

@ -23,7 +23,7 @@ import fr.acinq.bitcoin.Crypto.PublicKey
import fr.acinq.eclair.MilliSatoshi
import fr.acinq.eclair.crypto.Sphinx
import fr.acinq.eclair.router.Announcements
import fr.acinq.eclair.router.Router.{ChannelDesc, ChannelHop, Hop}
import fr.acinq.eclair.router.Router.{ChannelDesc, ChannelHop, Hop, Ignore}
import fr.acinq.eclair.wire.Node
/**
@ -162,43 +162,43 @@ object PaymentFailure {
.isDefined
/** Update the set of nodes and channels to ignore in retries depending on the failure we received. */
def updateIgnored(failure: PaymentFailure, ignoreNodes: Set[PublicKey], ignoreChannels: Set[ChannelDesc]): (Set[PublicKey], Set[ChannelDesc]) = failure match {
def updateIgnored(failure: PaymentFailure, ignore: Ignore): Ignore = failure match {
case RemoteFailure(hops, Sphinx.DecryptedFailurePacket(nodeId, _)) if nodeId == hops.last.nextNodeId =>
// The failure came from the final recipient: the payment should be aborted without penalizing anyone in the route.
(ignoreNodes, ignoreChannels)
ignore
case RemoteFailure(_, Sphinx.DecryptedFailurePacket(nodeId, _: Node)) =>
(ignoreNodes + nodeId, ignoreChannels)
ignore + nodeId
case RemoteFailure(_, Sphinx.DecryptedFailurePacket(nodeId, failureMessage: Update)) =>
if (Announcements.checkSig(failureMessage.update, nodeId)) {
// We were using an outdated channel update, we should retry with the new one and nobody should be penalized.
(ignoreNodes, ignoreChannels)
ignore
} else {
// This node is fishy, it gave us a bad signature, so let's filter it out.
(ignoreNodes + nodeId, ignoreChannels)
ignore + nodeId
}
case RemoteFailure(hops, Sphinx.DecryptedFailurePacket(nodeId, _)) =>
// Let's ignore the channel outgoing from nodeId.
hops.collectFirst {
case hop: ChannelHop if hop.nodeId == nodeId => ChannelDesc(hop.lastUpdate.shortChannelId, hop.nodeId, hop.nextNodeId)
} match {
case Some(faultyChannel) => (ignoreNodes, ignoreChannels + faultyChannel)
case None => (ignoreNodes, ignoreChannels)
case Some(faultyChannel) => ignore + faultyChannel
case None => ignore
}
case UnreadableRemoteFailure(hops) =>
// We don't know which node is sending garbage, let's blacklist all nodes except the one we are directly connected to and the final recipient.
val blacklist = hops.map(_.nextNodeId).drop(1).dropRight(1)
(ignoreNodes ++ blacklist, ignoreChannels)
val blacklist = hops.map(_.nextNodeId).drop(1).dropRight(1).toSet
ignore ++ blacklist
case LocalFailure(hops, _) => hops.headOption match {
case Some(hop: ChannelHop) =>
val faultyChannel = ChannelDesc(hop.lastUpdate.shortChannelId, hop.nodeId, hop.nextNodeId)
(ignoreNodes, ignoreChannels + faultyChannel)
case _ => (ignoreNodes, ignoreChannels)
ignore + faultyChannel
case _ => ignore
}
}
/** Update the set of nodes and channels to ignore in retries depending on the failures we received. */
def updateIgnored(failures: Seq[PaymentFailure], ignoreNodes: Set[PublicKey], ignoreChannels: Set[ChannelDesc]): (Set[PublicKey], Set[ChannelDesc]) = {
failures.foldLeft((ignoreNodes, ignoreChannels)) { case ((nodes, channels), failure) => updateIgnored(failure, nodes, channels) }
def updateIgnored(failures: Seq[PaymentFailure], ignore: Ignore): Ignore = {
failures.foldLeft(ignore) { case (current, failure) => updateIgnored(failure, current) }
}
}

View file

@ -29,11 +29,11 @@ import fr.acinq.eclair.payment.receive.MultiPartPaymentFSM
import fr.acinq.eclair.payment.send.MultiPartPaymentLifecycle.SendMultiPartPayment
import fr.acinq.eclair.payment.send.PaymentInitiator.SendPaymentConfig
import fr.acinq.eclair.payment.send.PaymentLifecycle.SendPayment
import fr.acinq.eclair.payment.send.{MultiPartPaymentLifecycle, PaymentError, PaymentLifecycle}
import fr.acinq.eclair.payment.send.{MultiPartPaymentLifecycle, PaymentLifecycle}
import fr.acinq.eclair.router.Router.RouteParams
import fr.acinq.eclair.router.{RouteCalculation, RouteNotFound}
import fr.acinq.eclair.router.{BalanceTooLow, RouteCalculation, RouteNotFound}
import fr.acinq.eclair.wire._
import fr.acinq.eclair.{CltvExpiry, Logs, MilliSatoshi, NodeParams, nodeFee, randomBytes32}
import fr.acinq.eclair.{CltvExpiry, Features, Logs, MilliSatoshi, NodeParams, nodeFee, randomBytes32}
import scala.collection.immutable.Queue
@ -46,7 +46,7 @@ import scala.collection.immutable.Queue
* It aggregates incoming HTLCs (in case multi-part was used upstream) and then forwards the requested amount (using the
* router to find a route to the remote node and potentially splitting the payment using multi-part).
*/
class NodeRelayer(nodeParams: NodeParams, relayer: ActorRef, router: ActorRef, commandBuffer: ActorRef, register: ActorRef) extends Actor with DiagnosticActorLogging {
class NodeRelayer(nodeParams: NodeParams, router: ActorRef, commandBuffer: ActorRef, register: ActorRef) extends Actor with DiagnosticActorLogging {
import NodeRelayer._
@ -147,7 +147,7 @@ class NodeRelayer(nodeParams: NodeParams, relayer: ActorRef, router: ActorRef, c
def spawnOutgoingPayFSM(cfg: SendPaymentConfig, multiPart: Boolean): ActorRef = {
if (multiPart) {
context.actorOf(MultiPartPaymentLifecycle.props(nodeParams, cfg, relayer, router, register))
context.actorOf(MultiPartPaymentLifecycle.props(nodeParams, cfg, router, register))
} else {
context.actorOf(PaymentLifecycle.props(nodeParams, cfg, router, register))
}
@ -157,15 +157,21 @@ class NodeRelayer(nodeParams: NodeParams, relayer: ActorRef, router: ActorRef, c
val paymentId = UUID.randomUUID()
val paymentCfg = SendPaymentConfig(paymentId, paymentId, None, paymentHash, payloadOut.amountToForward, payloadOut.outgoingNodeId, upstream, None, storeInDb = false, publishEvent = false, Nil)
val routeParams = computeRouteParams(nodeParams, upstream.amountIn, upstream.expiryIn, payloadOut.amountToForward, payloadOut.outgoingCltv)
// If invoice features are provided in the onion, the sender is asking us to relay to a non-trampoline recipient.
payloadOut.invoiceFeatures match {
case Some(_) =>
log.debug("relaying trampoline payment to non-trampoline recipient")
case Some(features) =>
val routingHints = payloadOut.invoiceRoutingInfo.map(_.map(_.toSeq).toSeq).getOrElse(Nil)
// TODO: @t-bast: MPP is disabled for trampoline to non-trampoline payments until we improve the splitting algorithm for nodes with a lot of channels.
val payFSM = spawnOutgoingPayFSM(paymentCfg, multiPart = false)
val finalPayload = Onion.createSinglePartPayload(payloadOut.amountToForward, payloadOut.outgoingCltv, payloadOut.paymentSecret)
val payment = SendPayment(payloadOut.outgoingNodeId, finalPayload, nodeParams.maxPaymentAttempts, routingHints, Some(routeParams))
payFSM ! payment
payloadOut.paymentSecret match {
case Some(paymentSecret) if Features(features).hasFeature(Features.BasicMultiPartPayment) =>
log.debug("relaying trampoline payment to non-trampoline recipient using MPP")
val payment = SendMultiPartPayment(paymentSecret, payloadOut.outgoingNodeId, payloadOut.amountToForward, payloadOut.outgoingCltv, nodeParams.maxPaymentAttempts, routingHints, Some(routeParams))
spawnOutgoingPayFSM(paymentCfg, multiPart = true) ! payment
case _ =>
log.debug("relaying trampoline payment to non-trampoline recipient without MPP")
val finalPayload = Onion.createSinglePartPayload(payloadOut.amountToForward, payloadOut.outgoingCltv, payloadOut.paymentSecret)
val payment = SendPayment(payloadOut.outgoingNodeId, finalPayload, nodeParams.maxPaymentAttempts, routingHints, Some(routeParams))
spawnOutgoingPayFSM(paymentCfg, multiPart = false) ! payment
}
case None =>
log.debug("relaying trampoline payment to next trampoline node")
val payFSM = spawnOutgoingPayFSM(paymentCfg, multiPart = true)
@ -208,7 +214,7 @@ class NodeRelayer(nodeParams: NodeParams, relayer: ActorRef, router: ActorRef, c
object NodeRelayer {
def props(nodeParams: NodeParams, relayer: ActorRef, router: ActorRef, commandBuffer: ActorRef, register: ActorRef) = Props(new NodeRelayer(nodeParams, relayer, router, commandBuffer, register))
def props(nodeParams: NodeParams, router: ActorRef, commandBuffer: ActorRef, register: ActorRef) = Props(new NodeRelayer(nodeParams, router, commandBuffer, register))
/**
* We start by aggregating an incoming HTLC set. Once we received the whole set, we will compute a route to the next
@ -259,15 +265,11 @@ object NodeRelayer {
* should return upstream.
*/
private def translateError(failures: Seq[PaymentFailure], outgoingNodeId: PublicKey): Option[FailureMessage] = {
def tooManyRouteNotFound(failures: Seq[PaymentFailure]): Boolean = {
val routeNotFoundCount = failures.collect { case f@LocalFailure(_, RouteNotFound) => f }.length
routeNotFoundCount > failures.length / 2
}
val routeNotFound = failures.collectFirst { case f@LocalFailure(_, RouteNotFound) => f }.nonEmpty
failures match {
case Nil => None
case LocalFailure(_, PaymentError.BalanceTooLow) :: Nil => Some(TemporaryNodeFailure) // we don't have enough outgoing liquidity at the moment
case _ if tooManyRouteNotFound(failures) => Some(TrampolineFeeInsufficient) // if we couldn't find routes, it's likely that the fee/cltv was insufficient
case LocalFailure(_, BalanceTooLow) :: Nil => Some(TemporaryNodeFailure) // we don't have enough outgoing liquidity at the moment
case _ if routeNotFound => Some(TrampolineFeeInsufficient) // if we couldn't find routes, it's likely that the fee/cltv was insufficient
case _ =>
// Otherwise, we try to find a downstream error that we could decrypt.
val outgoingNodeFailure = failures.collectFirst { case RemoteFailure(_, e) if e.originNode == outgoingNodeId => e.failureMessage }

View file

@ -79,7 +79,7 @@ class Relayer(nodeParams: NodeParams, router: ActorRef, register: ActorRef, comm
private val postRestartCleaner = context.actorOf(PostRestartHtlcCleaner.props(nodeParams, commandBuffer, initialized))
private val channelRelayer = context.actorOf(ChannelRelayer.props(nodeParams, self, register, commandBuffer))
private val nodeRelayer = context.actorOf(NodeRelayer.props(nodeParams, self, router, commandBuffer, register))
private val nodeRelayer = context.actorOf(NodeRelayer.props(nodeParams, router, commandBuffer, register))
override def receive: Receive = main(Map.empty, new mutable.HashMap[PublicKey, mutable.Set[ShortChannelId]] with mutable.MultiMap[PublicKey, ShortChannelId])
@ -202,7 +202,7 @@ class Relayer(nodeParams: NodeParams, router: ActorRef, register: ActorRef, comm
object Relayer extends Logging {
def props(nodeParams: NodeParams, router: ActorRef, register: ActorRef, commandBuffer: ActorRef, paymentHandler: ActorRef, initialized: Option[Promise[Done]] = None) =
Props(classOf[Relayer], nodeParams, router, register, commandBuffer, paymentHandler, initialized)
Props(new Relayer(nodeParams, router, register, commandBuffer, paymentHandler, initialized))
type ChannelUpdates = Map[ShortChannelId, OutgoingChannel]
type NodeChannels = mutable.HashMap[PublicKey, mutable.Set[ShortChannelId]] with mutable.MultiMap[PublicKey, ShortChannelId]

View file

@ -19,29 +19,23 @@ package fr.acinq.eclair.payment.send
import java.util.UUID
import java.util.concurrent.TimeUnit
import akka.actor.{ActorRef, FSM, Props}
import akka.actor.{ActorRef, FSM, Props, Status}
import akka.event.Logging.MDC
import fr.acinq.bitcoin.ByteVector32
import fr.acinq.bitcoin.Crypto.PublicKey
import fr.acinq.eclair.channel.{Commitments, Upstream}
import fr.acinq.eclair.crypto.Sphinx
import fr.acinq.eclair.channel.Upstream
import fr.acinq.eclair.payment.Monitoring.{Metrics, Tags}
import fr.acinq.eclair.payment.PaymentRequest.ExtraHop
import fr.acinq.eclair.payment.PaymentSent.PartialPayment
import fr.acinq.eclair.payment._
import fr.acinq.eclair.payment.relay.Relayer.{GetOutgoingChannels, OutgoingChannel, OutgoingChannels}
import fr.acinq.eclair.payment.send.PaymentInitiator.SendPaymentConfig
import fr.acinq.eclair.payment.send.PaymentLifecycle.SendPayment
import fr.acinq.eclair.payment.send.PaymentLifecycle.SendPaymentToRoute
import fr.acinq.eclair.router.RouteCalculation
import fr.acinq.eclair.router.Router._
import fr.acinq.eclair.router._
import fr.acinq.eclair.wire._
import fr.acinq.eclair.{CltvExpiry, FSMDiagnosticActorLogging, Logs, LongToBtcAmount, MilliSatoshi, NodeParams, ShortChannelId, ToMilliSatoshiConversion}
import fr.acinq.eclair.{CltvExpiry, FSMDiagnosticActorLogging, Logs, LongToBtcAmount, MilliSatoshi, NodeParams}
import kamon.Kamon
import kamon.context.Context
import scodec.bits.ByteVector
import scala.annotation.tailrec
import scala.util.Random
/**
* Created by t-bast on 18/07/2019.
@ -51,7 +45,7 @@ import scala.util.Random
* Sender for a multi-part payment (see https://github.com/lightningnetwork/lightning-rfc/blob/master/04-onion-routing.md#basic-multi-part-payments).
* The payment will be split into multiple sub-payments that will be sent in parallel.
*/
class MultiPartPaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, relayer: ActorRef, router: ActorRef, register: ActorRef) extends FSMDiagnosticActorLogging[MultiPartPaymentLifecycle.State, MultiPartPaymentLifecycle.Data] {
class MultiPartPaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: ActorRef, register: ActorRef) extends FSMDiagnosticActorLogging[MultiPartPaymentLifecycle.State, MultiPartPaymentLifecycle.Data] {
import MultiPartPaymentLifecycle._
@ -60,6 +54,7 @@ class MultiPartPaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig,
val id = cfg.id
val paymentHash = cfg.paymentHash
val start = System.currentTimeMillis
private var retriedFailedChannels = false
private val span = Kamon.spanBuilder("multi-part-payment")
.tag(Tags.ParentId, cfg.parentId.toString)
@ -72,109 +67,95 @@ class MultiPartPaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig,
when(WAIT_FOR_PAYMENT_REQUEST) {
case Event(r: SendMultiPartPayment, _) =>
router ! GetNetworkStats
goto(WAIT_FOR_NETWORK_STATS) using WaitingForNetworkStats(sender, r)
val routeParams = r.getRouteParams(nodeParams, randomize = false) // we don't randomize the first attempt, regardless of configuration choices
val maxFee = routeParams.getMaxFee(r.totalAmount)
log.debug("sending {} with maximum fee {}", r.totalAmount, maxFee)
val d = PaymentProgress(sender, r, r.maxAttempts, Map.empty, Ignore.empty, Nil)
router ! createRouteRequest(nodeParams, r.totalAmount, maxFee, routeParams, d)
goto(WAIT_FOR_ROUTES) using d
}
when(WAIT_FOR_NETWORK_STATS) {
case Event(s: GetNetworkStatsResponse, d: WaitingForNetworkStats) =>
log.debug("network stats: {}", s.stats.map(_.capacity))
// If we don't have network stats it's ok, we'll use data about our local channels instead.
// We tell the router to compute those stats though: in case our payment attempt fails, they will be available for
// another payment attempt.
if (s.stats.isEmpty) {
router ! TickComputeNetworkStats
}
relayer ! GetOutgoingChannels()
goto(WAIT_FOR_CHANNEL_BALANCES) using WaitingForChannelBalances(d.sender, d.request, s.stats)
}
when(WAIT_FOR_CHANNEL_BALANCES) {
case Event(OutgoingChannels(channels), d: WaitingForChannelBalances) =>
log.debug("trying to send {} with local channels: {}", d.request.totalAmount, channels.map(_.toUsableBalance).mkString(","))
val (remaining, payments) = splitPayment(nodeParams, d.request.totalAmount, channels, d.networkStats, d.request, randomize = false)
if (remaining > 0.msat) {
log.warning(s"cannot send ${d.request.totalAmount} with our current balance")
Metrics.PaymentError.withTag(Tags.Failure, Tags.FailureType(LocalFailure(Nil, PaymentError.BalanceTooLow)))
goto(PAYMENT_ABORTED) using PaymentAborted(d.sender, d.request, LocalFailure(Nil, PaymentError.BalanceTooLow) :: Nil, Set.empty)
} else {
val pending = setFees(d.request.routeParams, payments, payments.size)
when(WAIT_FOR_ROUTES) {
case Event(RouteResponse(routes), d: PaymentProgress) =>
log.info("{} routes found (attempt={}/{})", routes.length, d.request.maxAttempts - d.remainingAttempts + 1, d.request.maxAttempts)
// We may have already succeeded sending parts of the payment and only need to take care of the rest.
val (toSend, maxFee) = remainingToSend(nodeParams, d.request, d.pending.values)
if (routes.map(_.amount).sum == toSend) {
val childPayments = routes.map(route => (UUID.randomUUID(), route)).toMap
Kamon.runWithContextEntry(parentPaymentIdKey, cfg.parentId) {
Kamon.runWithSpan(span, finishSpan = true) {
pending.foreach { case (childId, payment) => spawnChildPaymentFsm(childId) ! payment }
childPayments.foreach { case (childId, route) => spawnChildPaymentFsm(childId) ! createChildPayment(route, d.request) }
}
}
goto(PAYMENT_IN_PROGRESS) using PaymentProgress(d.sender, d.request, d.networkStats, channels.length, 0 msat, d.request.maxAttempts - 1, pending, Set.empty, Nil)
goto(PAYMENT_IN_PROGRESS) using d.copy(remainingAttempts = (d.remainingAttempts - 1).max(0), pending = d.pending ++ childPayments)
} else {
// If a child payment failed while we were waiting for routes, the routes we received don't cover the whole
// remaining amount. In that case we discard these routes and send a new request to the router.
log.info("discarding routes, another child payment failed so we need to recompute them (amount = {}, maximum fee = {})", toSend, maxFee)
val routeParams = d.request.getRouteParams(nodeParams, randomize = true) // we randomize route selection when we retry
router ! createRouteRequest(nodeParams, toSend, maxFee, routeParams, d)
stay
}
case Event(Status.Failure(t), d: PaymentProgress) =>
log.warning("router error: {}", t.getMessage)
if (d.ignore.channels.nonEmpty) {
// If no route can be found, we will retry once with the channels that we previously ignored.
// Channels are mostly ignored for temporary reasons, likely because they didn't have enough balance to forward
// the payment. When we're retrying an MPP split, it may make sense to retry those ignored channels because with
// a different split, they may have enough balance to forward the payment.
val (toSend, maxFee) = remainingToSend(nodeParams, d.request, d.pending.values)
log.debug("retry sending {} with maximum fee {} without ignoring channels ({})", toSend, maxFee, d.ignore.channels.map(_.shortChannelId).mkString(","))
val routeParams = d.request.getRouteParams(nodeParams, randomize = true) // we randomize route selection when we retry
router ! createRouteRequest(nodeParams, toSend, maxFee, routeParams, d).copy(ignore = d.ignore.emptyChannels())
retriedFailedChannels = true
stay using d.copy(remainingAttempts = (d.remainingAttempts - 1).max(0), ignore = d.ignore.emptyChannels())
} else {
val failure = LocalFailure(Nil, t)
Metrics.PaymentError.withTag(Tags.Failure, Tags.FailureType(failure)).increment()
gotoAbortedOrStop(PaymentAborted(d.sender, d.request, d.failures :+ failure, d.pending.keySet))
}
case Event(pf: PaymentFailed, d: PaymentProgress) =>
if (isFinalRecipientFailure(pf, d)) {
gotoAbortedOrStop(PaymentAborted(d.sender, d.request, d.failures ++ pf.failures, d.pending.keySet - pf.id))
} else {
val ignore1 = PaymentFailure.updateIgnored(pf.failures, d.ignore)
stay using d.copy(pending = d.pending - pf.id, ignore = ignore1, failures = d.failures ++ pf.failures)
}
// The recipient released the preimage without receiving the full payment amount.
// This is a spec violation and is too bad for them, we obtained a proof of payment without paying the full amount.
case Event(ps: PaymentSent, d: PaymentProgress) =>
require(ps.parts.length == 1, "child payment must contain only one part")
// As soon as we get the preimage we can consider that the whole payment succeeded (we have a proof of payment).
gotoSucceededOrStop(PaymentSucceeded(d.sender, d.request, ps.paymentPreimage, ps.parts, d.pending.keySet - ps.parts.head.id))
}
when(PAYMENT_IN_PROGRESS) {
case Event(pf: PaymentFailed, d: PaymentProgress) => handleChildFailure(pf, d) match {
case Some(paymentAborted) =>
goto(PAYMENT_ABORTED) using paymentAborted
case None =>
// Get updated local channels (will take into account the child payments that are in-flight).
relayer ! GetOutgoingChannels()
val failedPayment = d.pending(pf.id)
val shouldBlacklist = shouldBlacklistChannel(pf)
if (shouldBlacklist) {
log.debug(s"ignoring channel ${getFirstHopShortChannelId(failedPayment)} to ${failedPayment.routePrefix.head.nextNodeId}")
}
val ignoreChannels = if (shouldBlacklist) d.ignoreChannels + getFirstHopShortChannelId(failedPayment) else d.ignoreChannels
val remainingAttempts = if (shouldBlacklist && Random.nextDouble() * math.log(d.channelsCount) > 2.0) {
// When we have a lot of channels, many of them may end up being a bad route prefix for the destination we're
// trying to reach. This is a cheap error that is detected quickly (RouteNotFound), so we don't want to count
// it in our payment attempts to avoid failing too fast.
// However we don't want to test all of our channels either which would be expensive, so we only probabilistically
// count the failure in our payment attempts.
// With the log-scale used, here are the probabilities and the corresponding number of retries:
// * 10 channels -> refund 13% of failures -> with 5 initial retries we will actually try 5/(1-0.13) = ~6 times
// * 20 channels -> refund 32% of failures -> with 5 initial retries we will actually try 5/(1-0.32) = ~7 times
// * 50 channels -> refund 50% of failures -> with 5 initial retries we will actually try 5/(1-0.50) = ~10 times
// * 100 channels -> refund 56% of failures -> with 5 initial retries we will actually try 5/(1-0.56) = ~11 times
// * 1000 channels -> refund 70% of failures -> with 5 initial retries we will actually try 5/(1-0.70) = ~17 times
// NB: this hack won't be necessary once multi-part is directly handled by the router.
d.remainingAttempts + 1
} else {
d.remainingAttempts
}
goto(RETRY_WITH_UPDATED_BALANCES) using d.copy(toSend = d.toSend + failedPayment.finalPayload.amount, pending = d.pending - pf.id, failures = d.failures ++ pf.failures, ignoreChannels = ignoreChannels, remainingAttempts = remainingAttempts)
}
case Event(ps: PaymentSent, d: PaymentProgress) =>
require(ps.parts.length == 1, "child payment must contain only one part")
// As soon as we get the preimage we can consider that the whole payment succeeded (we have a proof of payment).
goto(PAYMENT_SUCCEEDED) using PaymentSucceeded(d.sender, d.request, ps.paymentPreimage, ps.parts, d.pending.keySet - ps.parts.head.id)
}
when(RETRY_WITH_UPDATED_BALANCES) {
case Event(OutgoingChannels(channels), d: PaymentProgress) =>
log.debug("trying to send {} with local channels: {}", d.toSend, channels.map(_.toUsableBalance).mkString(","))
val filteredChannels = channels.filter(c => !d.ignoreChannels.contains(c.channelUpdate.shortChannelId))
val (remaining, payments) = splitPayment(nodeParams, d.toSend, filteredChannels, d.networkStats, d.request, randomize = true) // we randomize channel selection when we retry
if (remaining > 0.msat) {
log.warning(s"cannot send ${d.toSend} with our current balance")
Metrics.PaymentError.withTag(Tags.Failure, Tags.FailureType(LocalFailure(Nil, PaymentError.BalanceTooLow)))
goto(PAYMENT_ABORTED) using PaymentAborted(d.sender, d.request, d.failures :+ LocalFailure(Nil, PaymentError.BalanceTooLow), d.pending.keySet)
case Event(pf: PaymentFailed, d: PaymentProgress) =>
if (isFinalRecipientFailure(pf, d)) {
gotoAbortedOrStop(PaymentAborted(d.sender, d.request, d.failures ++ pf.failures, d.pending.keySet - pf.id))
} else if (d.remainingAttempts == 0) {
val failure = LocalFailure(Nil, PaymentError.RetryExhausted)
Metrics.PaymentError.withTag(Tags.Failure, Tags.FailureType(failure)).increment()
gotoAbortedOrStop(PaymentAborted(d.sender, d.request, d.failures ++ pf.failures :+ failure, d.pending.keySet - pf.id))
} else {
val pending = setFees(d.request.routeParams, payments, payments.size + d.pending.size)
pending.foreach { case (childId, payment) => spawnChildPaymentFsm(childId) ! payment }
goto(PAYMENT_IN_PROGRESS) using d.copy(toSend = 0 msat, remainingAttempts = d.remainingAttempts - 1, pending = d.pending ++ pending, channelsCount = channels.length)
val ignore1 = PaymentFailure.updateIgnored(pf.failures, d.ignore)
val stillPending = d.pending - pf.id
val (toSend, maxFee) = remainingToSend(nodeParams, d.request, stillPending.values)
log.debug("child payment failed, retry sending {} with maximum fee {}", toSend, maxFee)
val routeParams = d.request.getRouteParams(nodeParams, randomize = true) // we randomize route selection when we retry
val d1 = d.copy(pending = stillPending, ignore = ignore1, failures = d.failures ++ pf.failures)
router ! createRouteRequest(nodeParams, toSend, maxFee, routeParams, d1)
goto(WAIT_FOR_ROUTES) using d1
}
case Event(pf: PaymentFailed, d: PaymentProgress) => handleChildFailure(pf, d) match {
case Some(paymentAborted) =>
goto(PAYMENT_ABORTED) using paymentAborted
case None =>
val failedPayment = d.pending(pf.id)
val ignoreChannels = if (shouldBlacklistChannel(pf)) d.ignoreChannels + getFirstHopShortChannelId(failedPayment) else d.ignoreChannels
stay using d.copy(toSend = d.toSend + failedPayment.finalPayload.amount, pending = d.pending - pf.id, failures = d.failures ++ pf.failures, ignoreChannels = ignoreChannels)
}
case Event(ps: PaymentSent, d: PaymentProgress) =>
require(ps.parts.length == 1, "child payment must contain only one part")
// As soon as we get the preimage we can consider that the whole payment succeeded (we have a proof of payment).
goto(PAYMENT_SUCCEEDED) using PaymentSucceeded(d.sender, d.request, ps.paymentPreimage, ps.parts, d.pending.keySet - ps.parts.head.id)
Metrics.PaymentAttempt.withTag(Tags.MultiPart, value = true).record(d.request.maxAttempts - d.remainingAttempts)
gotoSucceededOrStop(PaymentSucceeded(d.sender, d.request, ps.paymentPreimage, ps.parts, d.pending.keySet - ps.parts.head.id))
}
when(PAYMENT_ABORTED) {
@ -192,7 +173,10 @@ class MultiPartPaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig,
case Event(ps: PaymentSent, d: PaymentAborted) =>
require(ps.parts.length == 1, "child payment must contain only one part")
log.warning(s"payment recipient fulfilled incomplete multi-part payment (id=${ps.parts.head.id})")
goto(PAYMENT_SUCCEEDED) using PaymentSucceeded(d.sender, d.request, ps.paymentPreimage, ps.parts, d.pending - ps.parts.head.id)
gotoSucceededOrStop(PaymentSucceeded(d.sender, d.request, ps.paymentPreimage, ps.parts, d.pending - ps.parts.head.id))
case Event(_: RouteResponse, _) => stay
case Event(_: Status.Failure, _) => stay
}
when(PAYMENT_SUCCEEDED) {
@ -216,20 +200,9 @@ class MultiPartPaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig,
} else {
stay using d.copy(pending = pending)
}
}
onTransition {
case _ -> PAYMENT_ABORTED => nextStateData match {
case d: PaymentAborted if d.pending.isEmpty =>
myStop(d.sender, Left(PaymentFailed(id, paymentHash, d.failures)))
case _ =>
}
case _ -> PAYMENT_SUCCEEDED => nextStateData match {
case d: PaymentSucceeded if d.pending.isEmpty =>
myStop(d.sender, Right(cfg.createPaymentSent(d.preimage, d.parts)))
case _ =>
}
case Event(_: RouteResponse, _) => stay
case Event(_: Status.Failure, _) => stay
}
def spawnChildPaymentFsm(childId: UUID): ActorRef = {
@ -241,6 +214,20 @@ class MultiPartPaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig,
context.actorOf(PaymentLifecycle.props(nodeParams, childCfg, router, register))
}
private def gotoAbortedOrStop(d: PaymentAborted): State = {
if (d.pending.isEmpty) {
myStop(d.sender, Left(PaymentFailed(id, paymentHash, d.failures)))
} else
goto(PAYMENT_ABORTED) using d
}
private def gotoSucceededOrStop(d: PaymentSucceeded): State = {
if (d.pending.isEmpty) {
myStop(d.sender, Right(cfg.createPaymentSent(d.preimage, d.parts)))
} else
goto(PAYMENT_SUCCEEDED) using d
}
def myStop(origin: ActorRef, event: Either[PaymentFailed, PaymentSent]): State = {
event match {
case Left(paymentFailed) =>
@ -255,6 +242,9 @@ class MultiPartPaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig,
.withTag(Tags.MultiPart, Tags.MultiPartType.Parent)
.withTag(Tags.Success, value = event.isRight)
.record(System.currentTimeMillis - start, TimeUnit.MILLISECONDS)
if (retriedFailedChannels) {
Metrics.RetryFailedChannelsResult.withTag(Tags.Success, event.isRight).increment()
}
span.finish()
stop(FSM.Normal)
}
@ -264,21 +254,13 @@ class MultiPartPaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig,
if (cfg.publishEvent) context.system.eventStream.publish(e)
}
def handleChildFailure(pf: PaymentFailed, d: PaymentProgress): Option[PaymentAborted] = {
val isFromFinalRecipient = pf.failures.collectFirst { case f: RemoteFailure if f.e.originNode == d.request.targetNodeId => true }.isDefined
if (isFromFinalRecipient) {
Some(PaymentAborted(d.sender, d.request, d.failures ++ pf.failures, d.pending.keySet - pf.id))
} else if (d.remainingAttempts == 0) {
val failure = LocalFailure(Nil, PaymentError.RetryExhausted)
Metrics.PaymentError.withTag(Tags.Failure, Tags.FailureType(failure))
Some(PaymentAborted(d.sender, d.request, d.failures ++ pf.failures :+ failure, d.pending.keySet - pf.id))
} else {
None
}
}
override def mdc(currentMessage: Any): MDC = {
Logs.mdc(category_opt = Some(Logs.LogCategory.PAYMENT), parentPaymentId_opt = Some(cfg.parentId), paymentId_opt = Some(id), paymentHash_opt = Some(paymentHash))
Logs.mdc(
category_opt = Some(Logs.LogCategory.PAYMENT),
parentPaymentId_opt = Some(cfg.parentId),
paymentId_opt = Some(id),
paymentHash_opt = Some(paymentHash),
remoteNodeId_opt = Some(cfg.recipientNodeId))
}
initialize()
@ -289,7 +271,7 @@ object MultiPartPaymentLifecycle {
val parentPaymentIdKey = Context.key[UUID]("parentPaymentId", UUID.fromString("00000000-0000-0000-0000-000000000000"))
def props(nodeParams: NodeParams, cfg: SendPaymentConfig, relayer: ActorRef, router: ActorRef, register: ActorRef) = Props(new MultiPartPaymentLifecycle(nodeParams, cfg, relayer, router, register))
def props(nodeParams: NodeParams, cfg: SendPaymentConfig, router: ActorRef, register: ActorRef) = Props(new MultiPartPaymentLifecycle(nodeParams, cfg, router, register))
/**
* Send a payment to a given node. The payment may be split into multiple child payments, for which a path-finding
@ -316,53 +298,46 @@ object MultiPartPaymentLifecycle {
additionalTlvs: Seq[OnionTlv] = Nil,
userCustomTlvs: Seq[GenericTlv] = Nil) {
require(totalAmount > 0.msat, s"total amount must be > 0")
def getRouteParams(nodeParams: NodeParams, randomize: Boolean): RouteParams =
routeParams.getOrElse(RouteCalculation.getDefaultRouteParams(nodeParams.routerConf)).copy(randomize = randomize)
}
// @formatter:off
sealed trait State
case object WAIT_FOR_PAYMENT_REQUEST extends State
case object WAIT_FOR_NETWORK_STATS extends State
case object WAIT_FOR_CHANNEL_BALANCES extends State
case object WAIT_FOR_PAYMENT_REQUEST extends State
case object WAIT_FOR_ROUTES extends State
case object PAYMENT_IN_PROGRESS extends State
case object RETRY_WITH_UPDATED_BALANCES extends State
case object PAYMENT_ABORTED extends State
case object PAYMENT_SUCCEEDED extends State
// @formatter:on
sealed trait Data
/**
* During initialization, we wait for a multi-part payment request containing the total amount to send.
* During initialization, we wait for a multi-part payment request containing the total amount to send and the maximum
* fee budget.
*/
case object WaitingForRequest extends Data
/**
* During initialization, we collect network statistics to help us decide how to best split a big payment.
*
* @param sender the sender of the payment request.
* @param request payment request containing the total amount to send.
*/
case class WaitingForNetworkStats(sender: ActorRef, request: SendMultiPartPayment) extends Data
/**
* During initialization, we request our local channels balances.
*
* @param sender the sender of the payment request.
* @param request payment request containing the total amount to send.
* @param networkStats network statistics help us decide how to best split a big payment.
*/
case class WaitingForChannelBalances(sender: ActorRef, request: SendMultiPartPayment, networkStats: Option[NetworkStats]) extends Data
/**
* While the payment is in progress, we listen to child payment failures. When we receive such failures, we request
* our up-to-date local channels balances and retry the failed child payments with a potentially different route.
* While the payment is in progress, we listen to child payment failures. When we receive such failures, we retry the
* failed amount with different routes.
*
* @param sender the sender of the payment request.
* @param request payment request containing the total amount to send.
* @param networkStats network statistics help us decide how to best split a big payment.
* @param channelsCount number of local channels.
* @param toSend remaining amount that should be split and sent.
* @param remainingAttempts remaining attempts (after child payments fail).
* @param pending pending child payments (payment sent, we are waiting for a fulfill or a failure).
* @param ignoreChannels channels that should be ignored (previously returned a permanent error).
* @param ignore channels and nodes that should be ignored (previously returned a permanent error).
* @param failures previous child payment failures.
*/
case class PaymentProgress(sender: ActorRef, request: SendMultiPartPayment, networkStats: Option[NetworkStats], channelsCount: Int, toSend: MilliSatoshi, remainingAttempts: Int, pending: Map[UUID, SendPayment], ignoreChannels: Set[ShortChannelId], failures: Seq[PaymentFailure]) extends Data
case class PaymentProgress(sender: ActorRef,
request: SendMultiPartPayment,
remainingAttempts: Int,
pending: Map[UUID, Route],
ignore: Ignore,
failures: Seq[PaymentFailure]) extends Data
/**
* When we exhaust our retry attempts without success, we abort the payment.
* Once we're in that state, we wait for all the pending child payments to settle.
@ -373,6 +348,7 @@ object MultiPartPaymentLifecycle {
* @param pending pending child payments (we are waiting for them to be failed downstream).
*/
case class PaymentAborted(sender: ActorRef, request: SendMultiPartPayment, failures: Seq[PaymentFailure], pending: Set[UUID]) extends Data
/**
* Once we receive a first fulfill for a child payment, we can consider that the whole payment succeeded (because we
* received the payment preimage that we can use as a proof of payment).
@ -385,160 +361,33 @@ object MultiPartPaymentLifecycle {
* @param pending pending child payments (we are waiting for them to be fulfilled downstream).
*/
case class PaymentSucceeded(sender: ActorRef, request: SendMultiPartPayment, preimage: ByteVector32, parts: Seq[PartialPayment], pending: Set[UUID]) extends Data
// @formatter:on
/** If the payment failed immediately with a RouteNotFound, the channel we selected should be ignored in retries. */
private def shouldBlacklistChannel(pf: PaymentFailed): Boolean = pf.failures match {
case LocalFailure(_, RouteNotFound) :: Nil => true
case _ => false
private def createRouteRequest(nodeParams: NodeParams, toSend: MilliSatoshi, maxFee: MilliSatoshi, routeParams: RouteParams, d: PaymentProgress): RouteRequest =
RouteRequest(
nodeParams.nodeId,
d.request.targetNodeId,
toSend,
maxFee,
d.request.assistedRoutes,
d.ignore,
Some(routeParams),
allowMultiPart = true,
d.pending.values.toSeq)
private def createChildPayment(route: Route, request: SendMultiPartPayment): SendPaymentToRoute = {
val finalPayload = Onion.createMultiPartPayload(route.amount, request.totalAmount, request.targetExpiry, request.paymentSecret, request.additionalTlvs, request.userCustomTlvs)
SendPaymentToRoute(Right(route), finalPayload)
}
def getFirstHopShortChannelId(payment: SendPayment): ShortChannelId = {
require(payment.routePrefix.nonEmpty, "multi-part payment must have a route prefix")
payment.routePrefix.head.lastUpdate.shortChannelId
}
/** When we receive an error from the final recipient, we should fail the whole payment, it's useless to retry. */
private def isFinalRecipientFailure(pf: PaymentFailed, d: PaymentProgress): Boolean = pf.failures.collectFirst {
case f: RemoteFailure if f.e.originNode == d.request.targetNodeId => true
}.getOrElse(false)
/**
* If fee limits are provided, we need to divide them between all child payments. Otherwise we could end up paying
* N * maxFee (where N is the number of child payments).
* Note that payment retries may mess up this calculation and make us pay a bit more than our fee limit.
*
* TODO: @t-bast: the router should expose a GetMultiRouteRequest API; this is where fee calculations will be more
* accurate and path-finding will be more efficient.
*/
private def setFees(routeParams: Option[RouteParams], payments: Seq[SendPayment], paymentsCount: Int): Map[UUID, SendPayment] =
payments.map(p => {
val payment = routeParams match {
case Some(routeParams) => p.copy(routeParams = Some(routeParams.copy(maxFeeBase = routeParams.maxFeeBase / paymentsCount)))
case None => p
}
(UUID.randomUUID(), payment)
}).toMap
private def createChildPayment(nodeParams: NodeParams, request: SendMultiPartPayment, childAmount: MilliSatoshi, channel: OutgoingChannel): SendPayment = {
SendPayment(
request.targetNodeId,
Onion.createMultiPartPayload(childAmount, request.totalAmount, request.targetExpiry, request.paymentSecret, request.additionalTlvs, request.userCustomTlvs),
request.maxAttempts,
request.assistedRoutes,
request.routeParams,
ChannelHop(nodeParams.nodeId, channel.nextNodeId, channel.channelUpdate) :: Nil)
}
/** Compute the maximum amount we should send in a single child payment. */
private def computeThreshold(networkStats: Option[NetworkStats], localChannels: Seq[OutgoingChannel]): MilliSatoshi = {
import com.google.common.math.Quantiles.median
import scala.collection.JavaConverters.asJavaCollectionConverter
// We use network statistics with a random factor to decide on the maximum amount for child payments.
// The current choice of parameters is completely arbitrary and could be made configurable.
// We could also learn from previous payment failures to dynamically tweak that value.
val maxAmount = networkStats.map(_.capacity.percentile75.toMilliSatoshi * ((75.0 + Random.nextInt(25)) / 100))
// If network statistics aren't available, we'll use our local channels to choose a value.
maxAmount.getOrElse({
val localBalanceMedian = median().compute(localChannels.map(b => java.lang.Long.valueOf(b.commitments.availableBalanceForSend.toLong)).asJavaCollection)
MilliSatoshi(localBalanceMedian.toLong)
})
}
/**
* Split a payment to a remote node inside a given channel.
*
* @param nodeParams node params.
* @param toSend total amount to send (may exceed the channel capacity if we have other channels available).
* @param request parent payment request.
* @param maxChildAmount maximum amount of each child payment inside that channel.
* @param maxFeeBase maximum base fee (for the future payment route).
* @param maxFeePct maximum proportional fee (for the future payment route).
* @param channel channel to use.
* @param channelCommitments channel commitments.
* @param channelPayments already-constructed child payments inside this channel.
* @return child payments to send through this channel.
*/
@tailrec
private def splitInsideChannel(nodeParams: NodeParams,
toSend: MilliSatoshi,
request: SendMultiPartPayment,
maxChildAmount: MilliSatoshi,
maxFeeBase: MilliSatoshi,
maxFeePct: Double,
channel: OutgoingChannel,
channelCommitments: Commitments,
channelPayments: Seq[SendPayment]): Seq[SendPayment] = {
// We can't use all the available balance because we need to take the fees for each child payment into account and
// we don't know the exact fee before-hand because we don't know the rest of the route yet (so we assume the worst
// case where the max fee is used).
val previousFees = channelPayments.map(p => maxFeeBase.max(p.finalPayload.amount * maxFeePct))
val totalPreviousFee = previousFees.sum
val withFeeBase = channelCommitments.availableBalanceForSend - maxFeeBase - totalPreviousFee
val withFeePct = channelCommitments.availableBalanceForSend * (1 - maxFeePct) - totalPreviousFee
val childAmount = Seq(maxChildAmount, toSend - channelPayments.map(_.finalPayload.amount).sum, withFeeBase, withFeePct).min
if (childAmount <= 0.msat) {
channelPayments
} else if (previousFees.nonEmpty && childAmount < previousFees.max) {
// We avoid sending tiny HTLCs: that would be a waste of fees.
channelPayments
} else {
val childPayment = createChildPayment(nodeParams, request, childAmount, channel)
// Splitting into multiple HTLCs in the same channel will also increase the size of the CommitTx (and thus its
// fee), which decreases the available balance.
// We need to take that into account when trying to send multiple payments through the same channel, which is
// why we simulate adding the HTLC to the commitments.
val fakeOnion = OnionRoutingPacket(0, ByteVector.fill(33)(0), ByteVector.fill(Sphinx.PaymentPacket.PayloadLength)(0), ByteVector32.Zeroes)
val add = UpdateAddHtlc(channelCommitments.channelId, channelCommitments.localNextHtlcId + channelPayments.size, childAmount, ByteVector32.Zeroes, CltvExpiry(0), fakeOnion)
val updatedCommitments = channelCommitments.addLocalProposal(add)
splitInsideChannel(nodeParams, toSend, request, maxChildAmount, maxFeeBase, maxFeePct, channel, updatedCommitments, childPayment +: channelPayments)
}
}
/**
* Split a payment into many child payments.
*
* @param toSend amount to split.
* @param localChannels local channels balances.
* @param request payment request containing the total amount to send and routing hints and parameters.
* @param randomize randomize the channel selection.
* @return the child payments that should be then sent to PaymentLifecycle actors.
*/
def splitPayment(nodeParams: NodeParams, toSend: MilliSatoshi, localChannels: Seq[OutgoingChannel], networkStats: Option[NetworkStats], request: SendMultiPartPayment, randomize: Boolean): (MilliSatoshi, Seq[SendPayment]) = {
require(toSend > 0.msat, "amount to send must be greater than 0")
val maxFeePct = request.routeParams.map(_.maxFeePct).getOrElse(nodeParams.routerConf.searchMaxFeePct)
val maxFeeBase = request.routeParams.map(_.maxFeeBase).getOrElse(nodeParams.routerConf.searchMaxFeeBase.toMilliSatoshi)
@tailrec
def split(remaining: MilliSatoshi, payments: Seq[SendPayment], channels: Seq[OutgoingChannel], splitInsideChannel: (MilliSatoshi, OutgoingChannel) => Seq[SendPayment]): Seq[SendPayment] = channels match {
case Nil => payments
case _ if remaining == 0.msat => payments
case _ if remaining < 0.msat => throw new RuntimeException(s"payment splitting error: remaining amount must not be negative ($remaining): sending $toSend to ${request.targetNodeId} with local channels=${localChannels.map(_.toUsableBalance)}, current channels=${channels.map(_.toUsableBalance)}, network=${networkStats.map(_.capacity)}, fees=($maxFeeBase, $maxFeePct)")
case channel :: rest if channel.commitments.availableBalanceForSend == 0.msat => split(remaining, payments, rest, splitInsideChannel)
case channel :: rest =>
val childPayments = splitInsideChannel(remaining, channel)
split(remaining - childPayments.map(_.finalPayload.amount).sum, payments ++ childPayments, rest, splitInsideChannel)
}
// If we have direct channels to the target, we use them without splitting the payment inside each channel.
val channelsToTarget = localChannels.filter(p => p.nextNodeId == request.targetNodeId).sortBy(_.commitments.availableBalanceForSend)
val directPayments = split(toSend, Seq.empty, channelsToTarget, (remaining: MilliSatoshi, channel: OutgoingChannel) => {
// When using direct channels to the destination, it doesn't make sense to use retries so we set maxAttempts to 1.
createChildPayment(nodeParams, request.copy(maxAttempts = 1), remaining.min(channel.commitments.availableBalanceForSend), channel) :: Nil
})
// Otherwise we need to split the amount based on network statistics and pessimistic fees estimates.
// Note that this will be handled more gracefully once this logic is migrated inside the router.
val channels = if (randomize) {
Random.shuffle(localChannels.filter(p => p.nextNodeId != request.targetNodeId))
} else {
localChannels.filter(p => p.nextNodeId != request.targetNodeId).sortBy(_.commitments.availableBalanceForSend)
}
val remotePayments = split(toSend - directPayments.map(_.finalPayload.amount).sum, Seq.empty, channels, (remaining: MilliSatoshi, channel: OutgoingChannel) => {
// We re-generate a split threshold for each channel to randomize the amounts.
val maxChildAmount = computeThreshold(networkStats, localChannels)
splitInsideChannel(nodeParams, remaining, request, maxChildAmount, maxFeeBase, maxFeePct, channel, channel.commitments, Nil)
})
val childPayments = directPayments ++ remotePayments
(toSend - childPayments.map(_.finalPayload.amount).sum, childPayments)
private def remainingToSend(nodeParams: NodeParams, request: SendMultiPartPayment, pending: Iterable[Route]): (MilliSatoshi, MilliSatoshi) = {
val sentAmount = pending.map(_.amount).sum
val sentFees = pending.map(_.fee).sum
(request.totalAmount - sentAmount, request.getRouteParams(nodeParams, randomize = false).getMaxFee(request.totalAmount) - sentFees)
}
}

View file

@ -41,8 +41,6 @@ object PaymentError {
// @formatter:on
// @formatter:off
/** Outbound capacity is too low. */
case object BalanceTooLow extends PaymentError
/** Payment attempts exhausted without success. */
case object RetryExhausted extends PaymentError
// @formatter:on

View file

@ -37,7 +37,7 @@ import fr.acinq.eclair.{CltvExpiry, CltvExpiryDelta, LongToBtcAmount, MilliSatos
/**
* Created by PM on 29/08/2016.
*/
class PaymentInitiator(nodeParams: NodeParams, router: ActorRef, relayer: ActorRef, register: ActorRef) extends Actor with ActorLogging {
class PaymentInitiator(nodeParams: NodeParams, router: ActorRef, register: ActorRef) extends Actor with ActorLogging {
import PaymentInitiator._
@ -114,12 +114,12 @@ class PaymentInitiator(nodeParams: NodeParams, router: ActorRef, relayer: ActorR
val trampolineSecret = r.trampolineSecret.getOrElse(randomBytes32)
sender ! SendPaymentToRouteResponse(paymentId, parentPaymentId, Some(trampolineSecret))
val (trampolineAmount, trampolineExpiry, trampolineOnion) = buildTrampolinePayment(SendTrampolinePaymentRequest(r.recipientAmount, r.paymentRequest, trampoline, Seq((r.trampolineFees, r.trampolineExpiryDelta)), r.finalExpiryDelta), r.trampolineFees, r.trampolineExpiryDelta)
payFsm forward SendPaymentToRoute(r.route, Onion.createMultiPartPayload(r.amount, trampolineAmount, trampolineExpiry, trampolineSecret, Seq(OnionTlv.TrampolineOnion(trampolineOnion))), r.paymentRequest.routingInfo)
payFsm forward SendPaymentToRoute(Left(r.route), Onion.createMultiPartPayload(r.amount, trampolineAmount, trampolineExpiry, trampolineSecret, Seq(OnionTlv.TrampolineOnion(trampolineOnion))), r.paymentRequest.routingInfo)
case Nil =>
sender ! SendPaymentToRouteResponse(paymentId, parentPaymentId, None)
r.paymentRequest.paymentSecret match {
case Some(paymentSecret) => payFsm forward SendPaymentToRoute(r.route, Onion.createMultiPartPayload(r.amount, r.recipientAmount, finalExpiry, paymentSecret), r.paymentRequest.routingInfo)
case None => payFsm forward SendPaymentToRoute(r.route, FinalLegacyPayload(r.recipientAmount, finalExpiry), r.paymentRequest.routingInfo)
case Some(paymentSecret) => payFsm forward SendPaymentToRoute(Left(r.route), Onion.createMultiPartPayload(r.amount, r.recipientAmount, finalExpiry, paymentSecret), r.paymentRequest.routingInfo)
case None => payFsm forward SendPaymentToRoute(Left(r.route), FinalLegacyPayload(r.recipientAmount, finalExpiry), r.paymentRequest.routingInfo)
}
case _ =>
sender ! PaymentFailed(paymentId, r.paymentHash, LocalFailure(Nil, TrampolineMultiNodeNotSupported) :: Nil)
@ -128,7 +128,7 @@ class PaymentInitiator(nodeParams: NodeParams, router: ActorRef, relayer: ActorR
def spawnPaymentFsm(paymentCfg: SendPaymentConfig): ActorRef = context.actorOf(PaymentLifecycle.props(nodeParams, paymentCfg, router, register))
def spawnMultiPartPaymentFsm(paymentCfg: SendPaymentConfig): ActorRef = context.actorOf(MultiPartPaymentLifecycle.props(nodeParams, paymentCfg, relayer, router, register))
def spawnMultiPartPaymentFsm(paymentCfg: SendPaymentConfig): ActorRef = context.actorOf(MultiPartPaymentLifecycle.props(nodeParams, paymentCfg, router, register))
private def buildTrampolinePayment(r: SendTrampolinePaymentRequest, trampolineFees: MilliSatoshi, trampolineExpiryDelta: CltvExpiryDelta): (MilliSatoshi, CltvExpiry, OnionRoutingPacket) = {
val trampolineRoute = Seq(
@ -161,7 +161,7 @@ class PaymentInitiator(nodeParams: NodeParams, router: ActorRef, relayer: ActorR
object PaymentInitiator {
def props(nodeParams: NodeParams, router: ActorRef, relayer: ActorRef, register: ActorRef) = Props(new PaymentInitiator(nodeParams, router, relayer, register))
def props(nodeParams: NodeParams, router: ActorRef, register: ActorRef) = Props(new PaymentInitiator(nodeParams, router, register))
case class PendingPayment(sender: ActorRef, remainingAttempts: Seq[(MilliSatoshi, CltvExpiryDelta)], r: SendTrampolinePaymentRequest)

View file

@ -75,44 +75,39 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A
span.tag(Tags.Amount, c.finalPayload.amount.toLong)
span.tag(Tags.TotalAmount, c.finalPayload.totalAmount.toLong)
span.tag(Tags.Expiry, c.finalPayload.expiry.toLong)
log.debug("sending {} to route {}", c.finalPayload.amount, c.hops.mkString("->"))
val send = SendPayment(c.hops.last, c.finalPayload, maxAttempts = 1)
router ! FinalizeRoute(c.finalPayload.amount, c.hops, c.assistedRoutes)
log.debug("sending {} to route {}", c.finalPayload.amount, c.printRoute())
val send = SendPayment(c.targetNodeId, c.finalPayload, maxAttempts = 1, assistedRoutes = c.assistedRoutes)
c.route.fold(
hops => router ! FinalizeRoute(c.finalPayload.amount, hops, c.assistedRoutes),
route => self ! RouteResponse(route :: Nil)
)
if (cfg.storeInDb) {
paymentsDb.addOutgoingPayment(OutgoingPayment(id, cfg.parentId, cfg.externalId, paymentHash, PaymentType.Standard, c.finalPayload.amount, cfg.recipientAmount, cfg.recipientNodeId, System.currentTimeMillis, cfg.paymentRequest, OutgoingPaymentStatus.Pending))
}
goto(WAITING_FOR_ROUTE) using WaitingForRoute(sender, send, Nil, Set.empty, Set.empty)
goto(WAITING_FOR_ROUTE) using WaitingForRoute(sender, send, Nil, Ignore.empty)
case Event(c: SendPayment, WaitingForRequest) =>
span.tag(Tags.TargetNodeId, c.targetNodeId.toString())
span.tag(Tags.Amount, c.finalPayload.amount.toLong)
span.tag(Tags.TotalAmount, c.finalPayload.totalAmount.toLong)
span.tag(Tags.Expiry, c.finalPayload.expiry.toLong)
log.debug("sending {} to {}{}", c.finalPayload.amount, c.targetNodeId, c.routePrefix.mkString(" with route prefix ", "->", ""))
// We don't want the router to try cycling back to nodes that are at the beginning of the route.
val ignoredNodes = c.routePrefix.map(_.nodeId).toSet
if (c.routePrefix.lastOption.exists(_.nextNodeId == c.targetNodeId)) {
// If the sender already provided a route to the target, no need to involve the router.
self ! RouteResponse(Seq(Route(c.finalPayload.amount, Nil, allowEmpty = true)))
} else {
router ! RouteRequest(c.getRouteRequestStart(nodeParams), c.targetNodeId, c.finalPayload.amount, c.getMaxFee(nodeParams), c.assistedRoutes, routeParams = c.routeParams, ignoreNodes = ignoredNodes)
}
log.debug("sending {} to {}", c.finalPayload.amount, c.targetNodeId)
router ! RouteRequest(nodeParams.nodeId, c.targetNodeId, c.finalPayload.amount, c.getMaxFee(nodeParams), c.assistedRoutes, routeParams = c.routeParams)
if (cfg.storeInDb) {
paymentsDb.addOutgoingPayment(OutgoingPayment(id, cfg.parentId, cfg.externalId, paymentHash, PaymentType.Standard, c.finalPayload.amount, cfg.recipientAmount, cfg.recipientNodeId, System.currentTimeMillis, cfg.paymentRequest, OutgoingPaymentStatus.Pending))
}
goto(WAITING_FOR_ROUTE) using WaitingForRoute(sender, c, Nil, ignoredNodes, Set.empty)
goto(WAITING_FOR_ROUTE) using WaitingForRoute(sender, c, Nil, Ignore.empty)
}
when(WAITING_FOR_ROUTE) {
case Event(RouteResponse(routes), WaitingForRoute(s, c, failures, ignoreNodes, ignoreChannels)) =>
val hops = c.routePrefix ++ routes.head.hops
log.info(s"route found: attempt=${failures.size + 1}/${c.maxAttempts} route=${hops.map(_.nextNodeId).mkString("->")} channels=${hops.map(_.lastUpdate.shortChannelId).mkString("->")}")
val firstHop = hops.head
val (cmd, sharedSecrets) = OutgoingPacket.buildCommand(cfg.upstream, paymentHash, hops, c.finalPayload)
register ! Register.ForwardShortId(firstHop.lastUpdate.shortChannelId, cmd)
goto(WAITING_FOR_PAYMENT_COMPLETE) using WaitingForComplete(s, c, cmd, failures, sharedSecrets, ignoreNodes, ignoreChannels, Route(c.finalPayload.amount, hops))
case Event(RouteResponse(route +: _), WaitingForRoute(s, c, failures, ignore)) =>
log.info(s"route found: attempt=${failures.size + 1}/${c.maxAttempts} route=${route.printNodes()} channels=${route.printChannels()}")
val (cmd, sharedSecrets) = OutgoingPacket.buildCommand(cfg.upstream, paymentHash, route.hops, c.finalPayload)
register ! Register.ForwardShortId(route.hops.head.lastUpdate.shortChannelId, cmd)
goto(WAITING_FOR_PAYMENT_COMPLETE) using WaitingForComplete(s, c, cmd, failures, sharedSecrets, ignore, route)
case Event(Status.Failure(t), WaitingForRoute(s, _, failures, _, _)) =>
case Event(Status.Failure(t), WaitingForRoute(s, _, failures, _)) =>
log.warning("router error: {}", t.getMessage)
Metrics.PaymentError.withTag(Tags.Failure, Tags.FailureType(LocalFailure(Nil, t))).increment()
onFailure(s, PaymentFailed(id, paymentHash, failures :+ LocalFailure(Nil, t)))
myStop()
@ -121,7 +116,8 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A
when(WAITING_FOR_PAYMENT_COMPLETE) {
case Event(ChannelCommandResponse.Ok, _) => stay
case Event(fulfill: Relayer.ForwardFulfill, WaitingForComplete(s, c, cmd, _, _, _, _, route)) =>
case Event(fulfill: Relayer.ForwardFulfill, WaitingForComplete(s, c, cmd, failures, _, _, route)) =>
Metrics.PaymentAttempt.withTag(Tags.MultiPart, value = false).record(failures.size + 1)
val p = PartialPayment(id, c.finalPayload.amount, cmd.amount - c.finalPayload.amount, fulfill.htlc.channelId, Some(cfg.fullRoute(route)))
onSuccess(s, cfg.createPaymentSent(fulfill.paymentPreimage, p :: Nil))
myStop()
@ -134,7 +130,7 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A
}
stay
case Event(fail: UpdateFailHtlc, data@WaitingForComplete(s, c, _, failures, sharedSecrets, ignoreNodes, ignoreChannels, route)) =>
case Event(fail: UpdateFailHtlc, data@WaitingForComplete(s, c, _, failures, sharedSecrets, ignore, route)) =>
(Sphinx.FailurePacket.decrypt(fail.reason, sharedSecrets) match {
case success@Success(e) =>
Metrics.PaymentError.withTag(Tags.Failure, Tags.FailureType(RemoteFailure(Nil, e))).increment()
@ -162,7 +158,7 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A
onFailure(s, PaymentFailed(id, paymentHash, failures :+ failure))
myStop()
case Failure(t) =>
log.warning(s"cannot parse returned error: ${t.getMessage}, route=${route.hops.map(_.nextNodeId)}")
log.warning(s"cannot parse returned error: ${t.getMessage}, route=${route.printNodes()}")
val failure = UnreadableRemoteFailure(cfg.fullRoute(route))
retry(failure, data)
case Success(e@Sphinx.DecryptedFailurePacket(nodeId, failureMessage: Node)) =>
@ -171,7 +167,7 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A
retry(failure, data)
case Success(e@Sphinx.DecryptedFailurePacket(nodeId, failureMessage: Update)) =>
log.info(s"received 'Update' type error message from nodeId=$nodeId, retrying payment (failure=$failureMessage)")
val ignoreNodes1 = if (Announcements.checkSig(failureMessage.update, nodeId)) {
val ignore1 = if (Announcements.checkSig(failureMessage.update, nodeId)) {
route.getChannelUpdateForNode(nodeId) match {
case Some(u) if u.shortChannelId != failureMessage.update.shortChannelId =>
// it is possible that nodes in the route prefer using a different channel (to the same N+1 node) than the one we requested, that's fine
@ -204,15 +200,15 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A
case extraHop => extraHop
})
// let's try again, router will have updated its state
router ! RouteRequest(c.getRouteRequestStart(nodeParams), c.targetNodeId, c.finalPayload.amount, c.getMaxFee(nodeParams), assistedRoutes1, ignoreNodes, ignoreChannels, c.routeParams)
ignoreNodes
router ! RouteRequest(nodeParams.nodeId, c.targetNodeId, c.finalPayload.amount, c.getMaxFee(nodeParams), assistedRoutes1, ignore, c.routeParams)
ignore
} else {
// this node is fishy, it gave us a bad sig!! let's filter it out
log.warning(s"got bad signature from node=$nodeId update=${failureMessage.update}")
router ! RouteRequest(c.getRouteRequestStart(nodeParams), c.targetNodeId, c.finalPayload.amount, c.getMaxFee(nodeParams), c.assistedRoutes, ignoreNodes + nodeId, ignoreChannels, c.routeParams)
ignoreNodes + nodeId
router ! RouteRequest(nodeParams.nodeId, c.targetNodeId, c.finalPayload.amount, c.getMaxFee(nodeParams), c.assistedRoutes, ignore + nodeId, c.routeParams)
ignore + nodeId
}
goto(WAITING_FOR_ROUTE) using WaitingForRoute(s, c, failures :+ RemoteFailure(cfg.fullRoute(route), e), ignoreNodes1, ignoreChannels)
goto(WAITING_FOR_ROUTE) using WaitingForRoute(s, c, failures :+ RemoteFailure(cfg.fullRoute(route), e), ignore1)
case Success(e@Sphinx.DecryptedFailurePacket(nodeId, failureMessage)) =>
log.info(s"received an error message from nodeId=$nodeId, trying to use a different channel (failure=$failureMessage)")
val failure = RemoteFailure(cfg.fullRoute(route), e)
@ -228,10 +224,9 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A
self ! Status.Failure(new RuntimeException("first hop returned an UpdateFailMalformedHtlc message"))
stay
case Event(Status.Failure(t), data@WaitingForComplete(s, c, _, failures, _, _, _, hops)) =>
case Event(Status.Failure(t), data@WaitingForComplete(s, c, _, failures, _, _, hops)) =>
Metrics.PaymentError.withTag(Tags.Failure, Tags.FailureType(LocalFailure(cfg.fullRoute(hops), t))).increment()
val isFatal = failures.size + 1 >= c.maxAttempts || // retries exhausted
c.routePrefix.nonEmpty || // first hop was selected by the sender and failed, it doesn't make sense to retry
t.isInstanceOf[HtlcsTimedoutDownstream] // htlc timed out so retrying won't help, we need to re-compute cltvs
if (isFatal) {
onFailure(s, PaymentFailed(id, paymentHash, failures :+ LocalFailure(cfg.fullRoute(hops), t)))
@ -266,9 +261,9 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A
}
private def retry(failure: PaymentFailure, data: WaitingForComplete): FSM.State[PaymentLifecycle.State, PaymentLifecycle.Data] = {
val (ignoreNodes1, ignoreChannels1) = PaymentFailure.updateIgnored(failure, data.ignoreNodes, data.ignoreChannels)
router ! RouteRequest(data.c.getRouteRequestStart(nodeParams), data.c.targetNodeId, data.c.finalPayload.amount, data.c.getMaxFee(nodeParams), data.c.assistedRoutes, ignoreNodes1, ignoreChannels1, data.c.routeParams)
goto(WAITING_FOR_ROUTE) using WaitingForRoute(data.sender, data.c, data.failures :+ failure, ignoreNodes1, ignoreChannels1)
val ignore1 = PaymentFailure.updateIgnored(failure, data.ignore)
router ! RouteRequest(nodeParams.nodeId, data.c.targetNodeId, data.c.finalPayload.amount, data.c.getMaxFee(nodeParams), data.c.assistedRoutes, ignore1, data.c.routeParams)
goto(WAITING_FOR_ROUTE) using WaitingForRoute(data.sender, data.c, data.failures :+ failure, ignore1)
}
private def myStop(): State = {
@ -312,12 +307,15 @@ object PaymentLifecycle {
/**
* Send a payment to a pre-defined route without running the path-finding algorithm.
*
* @param hops payment route to use.
* @param route payment route to use.
* @param finalPayload onion payload for the target node.
*/
case class SendPaymentToRoute(hops: Seq[PublicKey], finalPayload: FinalPayload, assistedRoutes: Seq[Seq[ExtraHop]] = Nil) {
require(hops.nonEmpty, s"payment route must not be empty")
val targetNodeId = hops.last
case class SendPaymentToRoute(route: Either[Seq[PublicKey], Route], finalPayload: FinalPayload, assistedRoutes: Seq[Seq[ExtraHop]] = Nil) {
require(route.fold(_.nonEmpty, _.hops.nonEmpty), "payment route must not be empty")
val targetNodeId = route.fold(_.last, _.hops.last.nextNodeId)
def printRoute(): String = route.fold(nodes => nodes, _.hops.map(_.nextNodeId)).mkString("->")
}
/**
@ -329,32 +327,24 @@ object PaymentLifecycle {
* @param maxAttempts maximum number of retries.
* @param assistedRoutes routing hints (usually from a Bolt 11 invoice).
* @param routeParams parameters to fine-tune the routing algorithm.
* @param routePrefix when provided, the payment route will start with these hops. Path-finding will run only to
* find how to route from the last node of the route prefix to the target node.
*/
case class SendPayment(targetNodeId: PublicKey,
finalPayload: FinalPayload,
maxAttempts: Int,
assistedRoutes: Seq[Seq[ExtraHop]] = Nil,
routeParams: Option[RouteParams] = None,
routePrefix: Seq[ChannelHop] = Nil) {
routeParams: Option[RouteParams] = None) {
require(finalPayload.amount > 0.msat, s"amount must be > 0")
def getMaxFee(nodeParams: NodeParams): MilliSatoshi =
routeParams.getOrElse(RouteCalculation.getDefaultRouteParams(nodeParams.routerConf)).getMaxFee(finalPayload.amount)
/** Returns the node from which the path-finding algorithm should start. */
def getRouteRequestStart(nodeParams: NodeParams): PublicKey = routePrefix match {
case Nil => nodeParams.nodeId
case prefix => prefix.last.nextNodeId
}
}
// @formatter:off
sealed trait Data
case object WaitingForRequest extends Data
case class WaitingForRoute(sender: ActorRef, c: SendPayment, failures: Seq[PaymentFailure], ignoreNodes: Set[PublicKey], ignoreChannels: Set[ChannelDesc]) extends Data
case class WaitingForComplete(sender: ActorRef, c: SendPayment, cmd: CMD_ADD_HTLC, failures: Seq[PaymentFailure], sharedSecrets: Seq[(ByteVector32, PublicKey)], ignoreNodes: Set[PublicKey], ignoreChannels: Set[ChannelDesc], route: Route) extends Data
case class WaitingForRoute(sender: ActorRef, c: SendPayment, failures: Seq[PaymentFailure], ignore: Ignore) extends Data
case class WaitingForComplete(sender: ActorRef, c: SendPayment, cmd: CMD_ADD_HTLC, failures: Seq[PaymentFailure], sharedSecrets: Seq[(ByteVector32, PublicKey)], ignore: Ignore, route: Route) extends Data
sealed trait State
case object WAITING_FOR_REQUEST extends State

View file

@ -28,6 +28,7 @@ object Monitoring {
val FindRouteDuration = Kamon.timer("router.find-route.duration", "Path-finding duration")
val FindRouteErrors = Kamon.counter("router.find-route.errors", "Path-finding errors")
val RouteLength = Kamon.histogram("router.find-route.length", "Path-finding result length")
val RouteResults = Kamon.histogram("router.find-route.results", "Path-finding number of routes found")
object QueryChannelRange {
val Blocks = Kamon.histogram("router.gossip.query-channel-range.blocks", "Number of blocks requested in query-channel-range")
@ -71,6 +72,7 @@ object Monitoring {
val Announced = "announced"
val Direction = "direction"
val Error = "error"
val MultiPart = "multiPart"
val NumberOfRoutes = "numRoutes"
object Directions {

View file

@ -28,6 +28,7 @@ import fr.acinq.eclair.router.Monitoring.{Metrics, Tags}
import fr.acinq.eclair.router.Router._
import fr.acinq.eclair.wire.ChannelUpdate
import fr.acinq.eclair.{ShortChannelId, _}
import kamon.tag.TagSet
import scala.annotation.tailrec
import scala.collection.mutable
@ -66,23 +67,31 @@ object RouteCalculation {
val extraEdges = assistedChannels.values.map(ac =>
GraphEdge(ChannelDesc(ac.extraHop.shortChannelId, ac.extraHop.nodeId, ac.nextNodeId), toFakeUpdate(ac.extraHop, ac.htlcMaximum), htlcMaxToCapacity(ac.htlcMaximum), Some(ac.htlcMaximum))
).toSet
val ignoredEdges = r.ignoreChannels ++ d.excludedChannels
val defaultRouteParams: RouteParams = getDefaultRouteParams(routerConf)
val params = r.routeParams.getOrElse(defaultRouteParams)
val ignoredEdges = r.ignore.channels ++ d.excludedChannels
val params = r.routeParams.getOrElse(getDefaultRouteParams(routerConf))
val routesToFind = if (params.randomize) DEFAULT_ROUTES_COUNT else 1
log.info(s"finding a route ${r.source}->${r.target} with assistedChannels={} ignoreNodes={} ignoreChannels={} excludedChannels={}", assistedChannels.keys.mkString(","), r.ignoreNodes.map(_.value).mkString(","), r.ignoreChannels.mkString(","), d.excludedChannels.mkString(","))
log.info(s"finding a route with randomize={} params={}", routesToFind > 1, params)
KamonExt.time(Metrics.FindRouteDuration.withTag(Tags.NumberOfRoutes, routesToFind).withTag(Tags.Amount, Tags.amountBucket(r.amount))) {
findRoute(d.graph, r.source, r.target, r.amount, r.maxFee, routesToFind, extraEdges, ignoredEdges, r.ignoreNodes, params, currentBlockHeight) match {
log.info(s"finding routes ${r.source}->${r.target} with assistedChannels={} ignoreNodes={} ignoreChannels={} excludedChannels={}", assistedChannels.keys.mkString(","), r.ignore.nodes.map(_.value).mkString(","), r.ignore.channels.mkString(","), d.excludedChannels.mkString(","))
log.info("finding routes with randomize={} params={}", params.randomize, params)
val tags = TagSet.Empty.withTag(Tags.MultiPart, r.allowMultiPart).withTag(Tags.Amount, Tags.amountBucket(r.amount))
KamonExt.time(Metrics.FindRouteDuration.withTags(tags.withTag(Tags.NumberOfRoutes, routesToFind.toLong))) {
val result = if (r.allowMultiPart) {
findMultiPartRoute(d.graph, r.source, r.target, r.amount, r.maxFee, extraEdges, ignoredEdges, r.ignore.nodes, r.pendingPayments, params, currentBlockHeight)
} else {
findRoute(d.graph, r.source, r.target, r.amount, r.maxFee, routesToFind, extraEdges, ignoredEdges, r.ignore.nodes, params, currentBlockHeight)
}
result match {
case Success(routes) =>
Metrics.RouteLength.withTag(Tags.Amount, Tags.amountBucket(r.amount)).record(routes.head.length)
Metrics.RouteResults.withTags(tags).record(routes.length)
routes.foreach(route => Metrics.RouteLength.withTags(tags).record(route.length))
ctx.sender ! RouteResponse(routes)
case Failure(t) =>
Metrics.FindRouteErrors.withTag(Tags.Amount, Tags.amountBucket(r.amount)).withTag(Tags.Error, t.getClass.getSimpleName).increment()
ctx.sender ! Status.Failure(t)
val failure = if (isNeighborBalanceTooLow(d.graph, r)) BalanceTooLow else t
Metrics.FindRouteErrors.withTags(tags.withTag(Tags.Error, failure.getClass.getSimpleName)).increment()
ctx.sender ! Status.Failure(failure)
}
}
d
}
@ -357,4 +366,14 @@ object RouteCalculation {
amountOk && feeOk
}
/**
* Checks if we are directly connected to the target but don't have enough balance in our local channels to send the
* requested amount. We could potentially relay the payment by using indirect routes, but since we're connected to
* the target node it means we'd like to reach it via direct channels as much as possible.
*/
private def isNeighborBalanceTooLow(g: DirectedGraph, r: RouteRequest): Boolean = {
val neighborEdges = g.getEdgesBetween(r.source, r.target)
neighborEdges.nonEmpty && neighborEdges.map(e => e.balance_opt.getOrElse(e.capacity.toMilliSatoshi)).sum < r.amount
}
}

View file

@ -341,19 +341,35 @@ object Router {
}
}
case class Ignore(nodes: Set[PublicKey], channels: Set[ChannelDesc]) {
// @formatter:off
def +(ignoreNode: PublicKey): Ignore = copy(nodes = nodes + ignoreNode)
def ++(ignoreNodes: Set[PublicKey]): Ignore = copy(nodes = nodes ++ ignoreNodes)
def +(ignoreChannel: ChannelDesc): Ignore = copy(channels = channels + ignoreChannel)
def emptyNodes(): Ignore = copy(nodes = Set.empty)
def emptyChannels(): Ignore = copy(channels = Set.empty)
// @formatter:on
}
object Ignore {
def empty: Ignore = Ignore(Set.empty, Set.empty)
}
case class RouteRequest(source: PublicKey,
target: PublicKey,
amount: MilliSatoshi,
maxFee: MilliSatoshi,
assistedRoutes: Seq[Seq[ExtraHop]] = Nil,
ignoreNodes: Set[PublicKey] = Set.empty,
ignoreChannels: Set[ChannelDesc] = Set.empty,
routeParams: Option[RouteParams] = None)
ignore: Ignore = Ignore.empty,
routeParams: Option[RouteParams] = None,
allowMultiPart: Boolean = false,
pendingPayments: Seq[Route] = Nil)
case class FinalizeRoute(amount: MilliSatoshi, hops: Seq[PublicKey], assistedRoutes: Seq[Seq[ExtraHop]] = Nil)
case class Route(amount: MilliSatoshi, hops: Seq[ChannelHop], allowEmpty: Boolean = false) {
require(allowEmpty || hops.nonEmpty, "route cannot be empty")
case class Route(amount: MilliSatoshi, hops: Seq[ChannelHop]) {
require(hops.nonEmpty, "route cannot be empty")
val length = hops.length
lazy val fee: MilliSatoshi = {
val amountToSend = hops.drop(1).reverse.foldLeft(amount) { case (amount1, hop) => amount1 + hop.fee(amount1) }
@ -362,6 +378,11 @@ object Router {
/** This method retrieves the channel update that we used when we built the route. */
def getChannelUpdateForNode(nodeId: PublicKey): Option[ChannelUpdate] = hops.find(_.nodeId == nodeId).map(_.lastUpdate)
def printNodes(): String = hops.map(_.nextNodeId).mkString("->")
def printChannels(): String = hops.map(_.lastUpdate.shortChannelId).mkString("->")
}
case class RouteResponse(routes: Seq[Route]) {

View file

@ -24,4 +24,6 @@ class RouterException(message: String) extends RuntimeException(message)
object RouteNotFound extends RouterException("route not found")
object BalanceTooLow extends RouterException("balance too low")
object CannotRouteToSelf extends RouterException("cannot route to self")

View file

@ -21,7 +21,7 @@ object Kamon {
def withoutTags() = this
def withTags(args: TagSet, a: Boolean) = this
def withTags(args: TagSet) = this
def withTags(a: TagSet, b: TagSet, c: Boolean) = this

View file

@ -1,6 +1,10 @@
package kamon.tag
trait TagSet
trait TagSet {
def withTag(t: String, s: Boolean) = this
def withTag(a: String, value: Long) = this
def withTag(a: String, value: String) = this
}
object TagSet extends TagSet {
def Empty: TagSet = this
def of(t: String, s: String) = this

View file

@ -22,10 +22,10 @@ import com.typesafe.config.{Config, ConfigFactory}
import fr.acinq.bitcoin.Block
import fr.acinq.bitcoin.Crypto.PublicKey
import fr.acinq.eclair.FeatureSupport.Mandatory
import fr.acinq.eclair.Features.{BasicMultiPartPayment, ChannelRangeQueries, ChannelRangeQueriesExtended, InitialRoutingSync, OptionDataLossProtect, PaymentSecret, VariableLengthOnion}
import fr.acinq.eclair.Features._
import fr.acinq.eclair.crypto.LocalKeyManager
import scodec.bits.ByteVector
import org.scalatest.funsuite.AnyFunSuite
import scodec.bits.ByteVector
import scala.collection.JavaConversions._
import scala.collection.JavaConverters._
@ -84,6 +84,10 @@ class StartupSpec extends AnyFunSuite {
}
test("NodeParams should fail if features are inconsistent") {
// Because of https://github.com/ACINQ/eclair/issues/1434, we need to remove the default features when falling back
// to the default configuration.
def finalizeConf(testCfg: Config): Config = testCfg.withFallback(defaultConf.withoutPath("features"))
val legalFeaturesConf = ConfigFactory.parseMap(Map(
s"features.${OptionDataLossProtect.rfcName}" -> "optional",
s"features.${InitialRoutingSync.rfcName}" -> "optional",
@ -105,9 +109,9 @@ class StartupSpec extends AnyFunSuite {
s"features.${BasicMultiPartPayment.rfcName}" -> "optional"
).asJava)
assert(Try(makeNodeParamsWithDefaults(legalFeaturesConf.withFallback(defaultConf))).isSuccess)
assert(Try(makeNodeParamsWithDefaults(illegalButAllowedFeaturesConf.withFallback(defaultConf))).isSuccess)
assert(Try(makeNodeParamsWithDefaults(illegalFeaturesConf.withFallback(defaultConf))).isFailure)
assert(Try(makeNodeParamsWithDefaults(finalizeConf(legalFeaturesConf))).isSuccess)
assert(Try(makeNodeParamsWithDefaults(finalizeConf(illegalButAllowedFeaturesConf))).isSuccess)
assert(Try(makeNodeParamsWithDefaults(finalizeConf(illegalFeaturesConf))).isFailure)
}
test("parse human readable override features") {

View file

@ -18,29 +18,25 @@ package fr.acinq.eclair.payment
import java.util.UUID
import akka.actor.ActorRef
import akka.actor.{ActorRef, Status}
import akka.testkit.{TestFSMRef, TestProbe}
import fr.acinq.bitcoin.{Block, Crypto, Satoshi}
import fr.acinq.eclair.TestConstants.TestFeeEstimator
import fr.acinq.bitcoin.{Block, Crypto}
import fr.acinq.eclair._
import fr.acinq.eclair.blockchain.fee.FeeratesPerKw
import fr.acinq.eclair.channel.{ChannelFlags, Commitments, CommitmentsSpec, Upstream}
import fr.acinq.eclair.channel.{AddHtlcFailed, ChannelFlags, ChannelUnavailable, Upstream}
import fr.acinq.eclair.crypto.Sphinx
import fr.acinq.eclair.payment.PaymentSent.PartialPayment
import fr.acinq.eclair.payment.relay.Relayer.{GetOutgoingChannels, OutgoingChannel, OutgoingChannels}
import fr.acinq.eclair.payment.send.MultiPartPaymentLifecycle
import fr.acinq.eclair.payment.send.MultiPartPaymentLifecycle._
import fr.acinq.eclair.payment.send.PaymentError.RetryExhausted
import fr.acinq.eclair.payment.send.PaymentInitiator.SendPaymentConfig
import fr.acinq.eclair.payment.send.PaymentLifecycle.SendPayment
import fr.acinq.eclair.payment.send.{MultiPartPaymentLifecycle, PaymentError}
import fr.acinq.eclair.payment.send.PaymentLifecycle.SendPaymentToRoute
import fr.acinq.eclair.router.RouteNotFound
import fr.acinq.eclair.router.Router._
import fr.acinq.eclair.router._
import fr.acinq.eclair.wire._
import org.scalatest.Outcome
import org.scalatest.funsuite.FixtureAnyFunSuiteLike
import org.scalatest.{Outcome, Tag}
import scodec.bits.ByteVector
import scodec.bits.{ByteVector, HexStringSyntax}
import scala.concurrent.duration._
import scala.util.Random
/**
* Created by t-bast on 18/07/2019.
@ -54,7 +50,6 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS
nodeParams: NodeParams,
payFsm: TestFSMRef[MultiPartPaymentLifecycle.State, MultiPartPaymentLifecycle.Data, MultiPartPaymentLifecycle],
router: TestProbe,
relayer: TestProbe,
sender: TestProbe,
childPayFsm: TestProbe,
eventListener: TestProbe)
@ -63,445 +58,392 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS
val id = UUID.randomUUID()
val cfg = SendPaymentConfig(id, id, Some("42"), paymentHash, finalAmount, finalRecipient, Upstream.Local(id), None, storeInDb = true, publishEvent = true, Nil)
val nodeParams = TestConstants.Alice.nodeParams
nodeParams.onChainFeeConf.feeEstimator.asInstanceOf[TestFeeEstimator].setFeerate(FeeratesPerKw.single(500))
val (childPayFsm, router, relayer, sender, eventListener) = (TestProbe(), TestProbe(), TestProbe(), TestProbe(), TestProbe())
class TestMultiPartPaymentLifecycle extends MultiPartPaymentLifecycle(nodeParams, cfg, relayer.ref, router.ref, TestProbe().ref) {
val (childPayFsm, router, sender, eventListener) = (TestProbe(), TestProbe(), TestProbe(), TestProbe())
class TestMultiPartPaymentLifecycle extends MultiPartPaymentLifecycle(nodeParams, cfg, router.ref, TestProbe().ref) {
override def spawnChildPaymentFsm(childId: UUID): ActorRef = childPayFsm.ref
}
val paymentHandler = TestFSMRef(new TestMultiPartPaymentLifecycle().asInstanceOf[MultiPartPaymentLifecycle])
system.eventStream.subscribe(eventListener.ref, classOf[PaymentEvent])
withFixture(test.toNoArgTest(FixtureParam(id, nodeParams, paymentHandler, router, relayer, sender, childPayFsm, eventListener)))
withFixture(test.toNoArgTest(FixtureParam(id, nodeParams, paymentHandler, router, sender, childPayFsm, eventListener)))
}
def initPayment(f: FixtureParam, request: SendMultiPartPayment, networkStats: NetworkStats, localChannels: OutgoingChannels): Unit = {
import f._
sender.send(payFsm, request)
router.expectMsg(GetNetworkStats)
router.send(payFsm, GetNetworkStatsResponse(Some(networkStats)))
relayer.expectMsg(GetOutgoingChannels())
relayer.send(payFsm, localChannels)
}
def waitUntilAmountSent(f: FixtureParam, amount: MilliSatoshi): Unit = {
Iterator.iterate(0 msat)(sent => {
sent + f.childPayFsm.expectMsgType[SendPayment].finalPayload.amount
}).takeWhile(sent => sent < amount)
}
test("get network statistics and usable balances before paying") { f =>
test("successful first attempt (single part)") { f =>
import f._
assert(payFsm.stateName === WAIT_FOR_PAYMENT_REQUEST)
val payment = SendMultiPartPayment(randomBytes32, b, 1500 * 1000 msat, expiry, 1)
val payment = SendMultiPartPayment(randomBytes32, e, finalAmount, expiry, 1, routeParams = Some(routeParams.copy(randomize = true)))
sender.send(payFsm, payment)
router.expectMsg(GetNetworkStats)
assert(payFsm.stateName === WAIT_FOR_NETWORK_STATS)
router.send(payFsm, GetNetworkStatsResponse(Some(emptyStats)))
relayer.expectMsg(GetOutgoingChannels())
awaitCond(payFsm.stateName === WAIT_FOR_CHANNEL_BALANCES)
assert(payFsm.stateData.asInstanceOf[WaitingForChannelBalances].networkStats === Some(emptyStats))
router.expectMsg(RouteRequest(nodeParams.nodeId, e, finalAmount, maxFee, routeParams = Some(routeParams.copy(randomize = false)), allowMultiPart = true))
assert(payFsm.stateName === WAIT_FOR_ROUTES)
val singleRoute = Route(finalAmount, hop_ab_1 :: hop_be :: Nil)
router.send(payFsm, RouteResponse(Seq(singleRoute)))
val childPayment = childPayFsm.expectMsgType[SendPaymentToRoute]
assert(childPayment.route === Right(singleRoute))
assert(childPayment.finalPayload.expiry === expiry)
assert(childPayment.finalPayload.paymentSecret === Some(payment.paymentSecret))
assert(childPayment.finalPayload.amount === finalAmount)
assert(childPayment.finalPayload.totalAmount === finalAmount)
assert(payFsm.stateName === PAYMENT_IN_PROGRESS)
val result = fulfillPendingPayments(f, 1)
assert(result.amountWithFees === finalAmount + 100.msat)
assert(result.trampolineFees === 0.msat)
assert(result.nonTrampolineFees === 100.msat)
}
test("get network statistics not available") { f =>
test("successful first attempt (multiple parts)") { f =>
import f._
assert(payFsm.stateName === WAIT_FOR_PAYMENT_REQUEST)
val payment = SendMultiPartPayment(randomBytes32, b, 2500 * 1000 msat, expiry, 1)
val payment = SendMultiPartPayment(randomBytes32, e, 1200000 msat, expiry, 1, routeParams = Some(routeParams.copy(randomize = false)))
sender.send(payFsm, payment)
router.expectMsg(GetNetworkStats)
assert(payFsm.stateName === WAIT_FOR_NETWORK_STATS)
router.send(payFsm, GetNetworkStatsResponse(None))
// If network stats aren't available we'll use local channel balance information instead.
// We should ask the router to compute statistics (for next payment attempts).
router.expectMsg(TickComputeNetworkStats)
relayer.expectMsg(GetOutgoingChannels())
awaitCond(payFsm.stateName === WAIT_FOR_CHANNEL_BALANCES)
assert(payFsm.stateData.asInstanceOf[WaitingForChannelBalances].networkStats === None)
relayer.send(payFsm, localChannels())
awaitCond(payFsm.stateName === PAYMENT_IN_PROGRESS)
waitUntilAmountSent(f, payment.totalAmount)
val payments = payFsm.stateData.asInstanceOf[PaymentProgress].pending.values
assert(payments.size > 1)
}
router.expectMsg(RouteRequest(nodeParams.nodeId, e, 1200000 msat, maxFee, routeParams = Some(routeParams.copy(randomize = false)), allowMultiPart = true))
assert(payFsm.stateName === WAIT_FOR_ROUTES)
test("send to peer node via multiple channels") { f =>
import f._
val payment = SendMultiPartPayment(randomBytes32, b, 2000 * 1000 msat, expiry, 3)
// When sending to a peer node, we should not filter out unannounced channels.
val channels = OutgoingChannels(Seq(
OutgoingChannel(c, channelUpdate_ac_2, makeCommitments(1000 * 1000 msat, 0)),
OutgoingChannel(c, channelUpdate_ac_3, makeCommitments(1500 * 1000 msat, 0)),
OutgoingChannel(b, channelUpdate_ab_1.copy(channelFlags = ChannelFlags.Empty), makeCommitments(1000 * 1000 msat, 0, announceChannel = false)),
OutgoingChannel(b, channelUpdate_ab_2.copy(channelFlags = ChannelFlags.Empty), makeCommitments(1500 * 1000 msat, 0, announceChannel = false)),
OutgoingChannel(d, channelUpdate_ad_1, makeCommitments(1000 * 1000 msat, 0))))
// Network statistics should be ignored when sending to peer.
initPayment(f, payment, emptyStats, channels)
// The payment should be split in two, using direct channels with b.
// MaxAttempts should be set to 1 when using direct channels to the destination.
childPayFsm.expectMsgAllOf(
SendPayment(b, Onion.createMultiPartPayload(1000 * 1000 msat, payment.totalAmount, expiry, payment.paymentSecret), 1, routePrefix = Seq(ChannelHop(nodeParams.nodeId, b, channelUpdate_ab_1.copy(channelFlags = ChannelFlags.Empty)))),
SendPayment(b, Onion.createMultiPartPayload(1000 * 1000 msat, payment.totalAmount, expiry, payment.paymentSecret), 1, routePrefix = Seq(ChannelHop(nodeParams.nodeId, b, channelUpdate_ab_2.copy(channelFlags = ChannelFlags.Empty))))
val routes = Seq(
Route(500000 msat, hop_ab_1 :: hop_be :: Nil),
Route(700000 msat, hop_ac_1 :: hop_ce :: Nil)
)
childPayFsm.expectNoMsg(50 millis)
val childIds = payFsm.stateData.asInstanceOf[PaymentProgress].pending.keys.toSeq
assert(childIds.length === 2)
router.send(payFsm, RouteResponse(routes))
val childPayments = childPayFsm.expectMsgType[SendPaymentToRoute] :: childPayFsm.expectMsgType[SendPaymentToRoute] :: Nil
assert(childPayments.map(_.route).toSet === routes.map(r => Right(r)).toSet)
assert(childPayments.map(_.finalPayload.expiry).toSet === Set(expiry))
assert(childPayments.map(_.finalPayload.paymentSecret.get).toSet === Set(payment.paymentSecret))
assert(childPayments.map(_.finalPayload.amount).toSet === Set(500000 msat, 700000 msat))
assert(childPayments.map(_.finalPayload.totalAmount).toSet === Set(1200000 msat))
assert(payFsm.stateName === PAYMENT_IN_PROGRESS)
val pp1 = PartialPayment(childIds.head, 1000 * 1000 msat, 0 msat, randomBytes32, None)
val pp2 = PartialPayment(childIds(1), 1000 * 1000 msat, 0 msat, randomBytes32, None)
childPayFsm.send(payFsm, PaymentSent(paymentId, paymentHash, paymentPreimage, finalAmount, b, Seq(pp1)))
childPayFsm.send(payFsm, PaymentSent(paymentId, paymentHash, paymentPreimage, finalAmount, b, Seq(pp2)))
val expectedMsg = PaymentSent(paymentId, paymentHash, paymentPreimage, finalAmount, finalRecipient, Seq(pp1, pp2))
sender.expectMsg(expectedMsg)
eventListener.expectMsg(expectedMsg)
assert(expectedMsg.recipientAmount === finalAmount)
assert(expectedMsg.amountWithFees === (2000 * 1000).msat)
assert(expectedMsg.trampolineFees === (1000 * 1000).msat)
assert(expectedMsg.nonTrampolineFees === 0.msat)
assert(expectedMsg.feesPaid === expectedMsg.trampolineFees)
val result = fulfillPendingPayments(f, 2)
assert(result.amountWithFees === 1200200.msat)
assert(result.trampolineFees === 200000.msat)
assert(result.nonTrampolineFees === 200.msat)
}
test("send to peer node via single big channel") { f =>
test("send custom tlv records") { f =>
import f._
val payment = SendMultiPartPayment(randomBytes32, b, 1000 * 1000 msat, expiry, 1)
// Network statistics should be ignored when sending to peer (otherwise we should have split into multiple payments).
initPayment(f, payment, emptyStats.copy(capacity = Stats.generate(Seq(100), d => Satoshi(d.toLong))), localChannels(0))
childPayFsm.expectMsg(SendPayment(b, Onion.createMultiPartPayload(payment.totalAmount, payment.totalAmount, expiry, payment.paymentSecret), 1, routePrefix = Seq(ChannelHop(nodeParams.nodeId, b, channelUpdate_ab_1))))
childPayFsm.expectNoMsg(50 millis)
// We include a bunch of additional tlv records.
val trampolineTlv = OnionTlv.TrampolineOnion(OnionRoutingPacket(0, ByteVector.fill(33)(0), ByteVector.fill(400)(0), randomBytes32))
val userCustomTlv = GenericTlv(UInt64(561), hex"deadbeef")
val payment = SendMultiPartPayment(randomBytes32, e, finalAmount + 1000.msat, expiry, 1, routeParams = Some(routeParams), additionalTlvs = Seq(trampolineTlv), userCustomTlvs = Seq(userCustomTlv))
sender.send(payFsm, payment)
router.expectMsgType[RouteRequest]
router.send(payFsm, RouteResponse(Seq(Route(500000 msat, hop_ab_1 :: hop_be :: Nil), Route(501000 msat, hop_ac_1 :: hop_ce :: Nil))))
val childPayments = childPayFsm.expectMsgType[SendPaymentToRoute] :: childPayFsm.expectMsgType[SendPaymentToRoute] :: Nil
childPayments.map(_.finalPayload.asInstanceOf[Onion.FinalTlvPayload]).foreach(p => {
assert(p.records.get[OnionTlv.TrampolineOnion] === Some(trampolineTlv))
assert(p.records.unknown.toSeq === Seq(userCustomTlv))
})
val result = fulfillPendingPayments(f, 2)
assert(result.trampolineFees === 1000.msat)
}
test("send to peer node via remote channels") { f =>
test("successful retry") { f =>
import f._
// d only has a single channel with capacity 1000 sat, we try to send more.
val payment = SendMultiPartPayment(randomBytes32, d, 2000 * 1000 msat, expiry, 1)
val testChannels = localChannels()
val balanceToTarget = testChannels.channels.filter(_.nextNodeId == d).map(_.commitments.availableBalanceForSend).sum
assert(balanceToTarget < (1000 * 1000).msat) // the commit tx fee prevents us from completely emptying our channel
initPayment(f, payment, emptyStats.copy(capacity = Stats.generate(Seq(500), d => Satoshi(d.toLong))), testChannels)
waitUntilAmountSent(f, payment.totalAmount)
val payments = payFsm.stateData.asInstanceOf[PaymentProgress].pending.values
assert(payments.size > 1)
val directPayments = payments.filter(p => p.routePrefix.head.nextNodeId == d)
assert(directPayments.size === 1)
assert(directPayments.head.finalPayload.amount === balanceToTarget)
val payment = SendMultiPartPayment(randomBytes32, e, finalAmount, expiry, 3, routeParams = Some(routeParams))
sender.send(payFsm, payment)
router.expectMsgType[RouteRequest]
val failingRoute = Route(finalAmount, hop_ab_1 :: hop_be :: Nil)
router.send(payFsm, RouteResponse(Seq(failingRoute)))
childPayFsm.expectMsgType[SendPaymentToRoute]
childPayFsm.expectNoMsg(100 millis)
val childId = payFsm.stateData.asInstanceOf[PaymentProgress].pending.keys.head
childPayFsm.send(payFsm, PaymentFailed(childId, paymentHash, Seq(RemoteFailure(failingRoute.hops, Sphinx.DecryptedFailurePacket(b, PermanentChannelFailure)))))
// We retry ignoring the failing channel.
router.expectMsg(RouteRequest(nodeParams.nodeId, e, finalAmount, maxFee, routeParams = Some(routeParams.copy(randomize = true)), allowMultiPart = true, ignore = Ignore(Set.empty, Set(ChannelDesc(channelId_be, b, e)))))
router.send(payFsm, RouteResponse(Seq(Route(400000 msat, hop_ac_1 :: hop_ce :: Nil), Route(600000 msat, hop_ad :: hop_de :: Nil))))
childPayFsm.expectMsgType[SendPaymentToRoute]
childPayFsm.expectMsgType[SendPaymentToRoute]
assert(!payFsm.stateData.asInstanceOf[PaymentProgress].pending.contains(childId))
val result = fulfillPendingPayments(f, 2)
assert(result.amountWithFees === 1000200.msat)
assert(result.trampolineFees === 0.msat)
assert(result.nonTrampolineFees === 200.msat)
}
test("send to remote node without splitting") { f =>
test("retry failures while waiting for routes") { f =>
import f._
val payment = SendMultiPartPayment(randomBytes32, e, 300 * 1000 msat, expiry, 1)
initPayment(f, payment, emptyStats.copy(capacity = Stats.generate(Seq(1500), d => Satoshi(d.toLong))), localChannels())
waitUntilAmountSent(f, payment.totalAmount)
payFsm.stateData.asInstanceOf[PaymentProgress].pending.foreach {
case (id, payment) => childPayFsm.send(payFsm, PaymentSent(paymentId, paymentHash, paymentPreimage, finalAmount, e, Seq(PartialPayment(id, payment.finalPayload.amount, 5 msat, randomBytes32, None))))
}
val payment = SendMultiPartPayment(randomBytes32, e, finalAmount, expiry, 3, routeParams = Some(routeParams))
sender.send(payFsm, payment)
router.expectMsgType[RouteRequest]
router.send(payFsm, RouteResponse(Seq(Route(400000 msat, hop_ab_1 :: hop_be :: Nil), Route(600000 msat, hop_ab_2 :: hop_be :: Nil))))
childPayFsm.expectMsgType[SendPaymentToRoute]
childPayFsm.expectMsgType[SendPaymentToRoute]
childPayFsm.expectNoMsg(100 millis)
val (failedId1, failedRoute1) :: (failedId2, failedRoute2) :: Nil = payFsm.stateData.asInstanceOf[PaymentProgress].pending.toList
childPayFsm.send(payFsm, PaymentFailed(failedId1, paymentHash, Seq(RemoteFailure(failedRoute1.hops, Sphinx.DecryptedFailurePacket(b, TemporaryNodeFailure)))))
// When we retry, we ignore the failing node and we let the router know about the remaining pending route.
router.expectMsg(RouteRequest(nodeParams.nodeId, e, failedRoute1.amount, maxFee - failedRoute1.fee, ignore = Ignore(Set(b), Set.empty), pendingPayments = Seq(failedRoute2), allowMultiPart = true, routeParams = Some(routeParams.copy(randomize = true))))
// The second part fails while we're still waiting for new routes.
childPayFsm.send(payFsm, PaymentFailed(failedId2, paymentHash, Seq(RemoteFailure(failedRoute2.hops, Sphinx.DecryptedFailurePacket(b, TemporaryNodeFailure)))))
// We receive a response to our first request, but it's now obsolete: we re-sent a new route request that takes into
// account the latest failures.
router.send(payFsm, RouteResponse(Seq(Route(failedRoute1.amount, hop_ac_1 :: hop_ce :: Nil))))
router.expectMsg(RouteRequest(nodeParams.nodeId, e, finalAmount, maxFee, ignore = Ignore(Set(b), Set.empty), allowMultiPart = true, routeParams = Some(routeParams.copy(randomize = true))))
awaitCond(payFsm.stateData.asInstanceOf[PaymentProgress].pending.isEmpty)
childPayFsm.expectNoMsg(100 millis)
// We receive new routes that work.
router.send(payFsm, RouteResponse(Seq(Route(300000 msat, hop_ac_1 :: hop_ce :: Nil), Route(700000 msat, hop_ad :: hop_de :: Nil))))
childPayFsm.expectMsgType[SendPaymentToRoute]
childPayFsm.expectMsgType[SendPaymentToRoute]
val result = fulfillPendingPayments(f, 2)
assert(result.amountWithFees === 1000200.msat)
assert(result.nonTrampolineFees === 200.msat)
}
test("retry without ignoring channels") { f =>
import f._
val payment = SendMultiPartPayment(randomBytes32, e, finalAmount, expiry, 3, routeParams = Some(routeParams))
sender.send(payFsm, payment)
router.expectMsgType[RouteRequest]
router.send(payFsm, RouteResponse(Seq(Route(500000 msat, hop_ab_1 :: hop_be :: Nil), Route(500000 msat, hop_ab_1 :: hop_be :: Nil))))
childPayFsm.expectMsgType[SendPaymentToRoute]
childPayFsm.expectMsgType[SendPaymentToRoute]
childPayFsm.expectNoMsg(100 millis)
val (failedId, failedRoute) :: (_, pendingRoute) :: Nil = payFsm.stateData.asInstanceOf[PaymentProgress].pending.toList
childPayFsm.send(payFsm, PaymentFailed(failedId, paymentHash, Seq(LocalFailure(failedRoute.hops, AddHtlcFailed(randomBytes32, paymentHash, ChannelUnavailable(randomBytes32), null, None, None)))))
// If the router doesn't find routes, we will retry without ignoring the channel: it may work with a different split
// of the amount to send.
val expectedRouteRequest = RouteRequest(
nodeParams.nodeId, e,
failedRoute.amount, maxFee - failedRoute.fee,
ignore = Ignore(Set.empty, Set(ChannelDesc(channelId_ab_1, a, b))),
pendingPayments = Seq(pendingRoute),
allowMultiPart = true,
routeParams = Some(routeParams.copy(randomize = true)))
router.expectMsg(expectedRouteRequest)
router.send(payFsm, Status.Failure(RouteNotFound))
router.expectMsg(expectedRouteRequest.copy(ignore = Ignore.empty))
router.send(payFsm, RouteResponse(Seq(Route(500000 msat, hop_ac_1 :: hop_ce :: Nil))))
childPayFsm.expectMsgType[SendPaymentToRoute]
val result = fulfillPendingPayments(f, 2)
assert(result.amountWithFees === 1000200.msat)
}
test("abort after too many failed attempts") { f =>
import f._
val payment = SendMultiPartPayment(randomBytes32, e, finalAmount, expiry, 2, routeParams = Some(routeParams))
sender.send(payFsm, payment)
router.expectMsgType[RouteRequest]
router.send(payFsm, RouteResponse(Seq(Route(500000 msat, hop_ab_1 :: hop_be :: Nil), Route(500000 msat, hop_ac_1 :: hop_ce :: Nil))))
childPayFsm.expectMsgType[SendPaymentToRoute]
childPayFsm.expectMsgType[SendPaymentToRoute]
val (failedId1, failedRoute1) = payFsm.stateData.asInstanceOf[PaymentProgress].pending.head
childPayFsm.send(payFsm, PaymentFailed(failedId1, paymentHash, Seq(UnreadableRemoteFailure(failedRoute1.hops))))
router.expectMsgType[RouteRequest]
router.send(payFsm, RouteResponse(Seq(Route(500000 msat, hop_ad :: hop_de :: Nil))))
childPayFsm.expectMsgType[SendPaymentToRoute]
assert(!payFsm.stateData.asInstanceOf[PaymentProgress].pending.contains(failedId1))
val (failedId2, failedRoute2) = payFsm.stateData.asInstanceOf[PaymentProgress].pending.head
val result = abortAfterFailure(f, PaymentFailed(failedId2, paymentHash, Seq(UnreadableRemoteFailure(failedRoute2.hops))))
assert(result.failures.length >= 3)
assert(result.failures.contains(LocalFailure(Nil, RetryExhausted)))
}
test("abort if no routes found") { f =>
import f._
sender.watch(payFsm)
val payment = SendMultiPartPayment(randomBytes32, e, finalAmount, expiry, 5, routeParams = Some(routeParams))
sender.send(payFsm, payment)
router.expectMsgType[RouteRequest]
router.send(payFsm, Status.Failure(RouteNotFound))
val result = sender.expectMsgType[PaymentFailed]
assert(result.id === paymentId)
assert(result.paymentHash === paymentHash)
assert(result.failures === Seq(LocalFailure(Nil, RouteNotFound)))
sender.expectTerminated(payFsm)
sender.expectNoMsg(100 millis)
router.expectNoMsg(100 millis)
childPayFsm.expectNoMsg(100 millis)
}
test("abort if recipient sends error") { f =>
import f._
val payment = SendMultiPartPayment(randomBytes32, e, finalAmount, expiry, 5, routeParams = Some(routeParams))
sender.send(payFsm, payment)
router.expectMsgType[RouteRequest]
router.send(payFsm, RouteResponse(Seq(Route(finalAmount, hop_ab_1 :: hop_be :: Nil))))
childPayFsm.expectMsgType[SendPaymentToRoute]
val (failedId, failedRoute) = payFsm.stateData.asInstanceOf[PaymentProgress].pending.head
val result = abortAfterFailure(f, PaymentFailed(failedId, paymentHash, Seq(RemoteFailure(failedRoute.hops, Sphinx.DecryptedFailurePacket(e, IncorrectOrUnknownPaymentDetails(600000 msat, 0))))))
assert(result.failures.length === 1)
}
test("abort if recipient sends error during retry") { f =>
import f._
val payment = SendMultiPartPayment(randomBytes32, e, finalAmount, expiry, 5, routeParams = Some(routeParams))
sender.send(payFsm, payment)
router.expectMsgType[RouteRequest]
router.send(payFsm, RouteResponse(Seq(Route(400000 msat, hop_ab_1 :: hop_be :: Nil), Route(600000 msat, hop_ac_1 :: hop_ce :: Nil))))
childPayFsm.expectMsgType[SendPaymentToRoute]
childPayFsm.expectMsgType[SendPaymentToRoute]
val (failedId1, failedRoute1) :: (failedId2, failedRoute2) :: Nil = payFsm.stateData.asInstanceOf[PaymentProgress].pending.toList
childPayFsm.send(payFsm, PaymentFailed(failedId1, paymentHash, Seq(UnreadableRemoteFailure(failedRoute1.hops))))
router.expectMsgType[RouteRequest]
val result = abortAfterFailure(f, PaymentFailed(failedId2, paymentHash, Seq(RemoteFailure(failedRoute2.hops, Sphinx.DecryptedFailurePacket(e, PaymentTimeout)))))
assert(result.failures.length === 2)
}
test("receive partial success after retriable failure (recipient spec violation)") { f =>
import f._
val payment = SendMultiPartPayment(randomBytes32, e, finalAmount, expiry, 5, routeParams = Some(routeParams))
sender.send(payFsm, payment)
router.expectMsgType[RouteRequest]
router.send(payFsm, RouteResponse(Seq(Route(400000 msat, hop_ab_1 :: hop_be :: Nil), Route(600000 msat, hop_ac_1 :: hop_ce :: Nil))))
childPayFsm.expectMsgType[SendPaymentToRoute]
childPayFsm.expectMsgType[SendPaymentToRoute]
val (failedId, failedRoute) :: (successId, successRoute) :: Nil = payFsm.stateData.asInstanceOf[PaymentProgress].pending.toList
childPayFsm.send(payFsm, PaymentFailed(failedId, paymentHash, Seq(UnreadableRemoteFailure(failedRoute.hops))))
router.expectMsgType[RouteRequest]
val result = fulfillPendingPayments(f, 1)
assert(result.amountWithFees < finalAmount) // we got the preimage without paying the full amount
assert(result.nonTrampolineFees === successRoute.fee) // we paid the fee for only one of the partial payments
assert(result.parts.length === 1 && result.parts.head.id === successId)
}
test("receive partial success after abort (recipient spec violation)") { f =>
import f._
val payment = SendMultiPartPayment(randomBytes32, e, finalAmount, expiry, 5, routeParams = Some(routeParams))
sender.send(payFsm, payment)
router.expectMsgType[RouteRequest]
router.send(payFsm, RouteResponse(Seq(Route(400000 msat, hop_ab_1 :: hop_be :: Nil), Route(600000 msat, hop_ac_1 :: hop_ce :: Nil))))
childPayFsm.expectMsgType[SendPaymentToRoute]
childPayFsm.expectMsgType[SendPaymentToRoute]
val (failedId, failedRoute) :: (successId, successRoute) :: Nil = payFsm.stateData.asInstanceOf[PaymentProgress].pending.toList
childPayFsm.send(payFsm, PaymentFailed(failedId, paymentHash, Seq(RemoteFailure(failedRoute.hops, Sphinx.DecryptedFailurePacket(e, PaymentTimeout)))))
awaitCond(payFsm.stateName === PAYMENT_ABORTED)
sender.watch(payFsm)
childPayFsm.send(payFsm, PaymentSent(paymentId, paymentHash, paymentPreimage, finalAmount, e, Seq(PaymentSent.PartialPayment(successId, successRoute.amount, successRoute.fee, randomBytes32, Some(successRoute.hops)))))
val result = sender.expectMsgType[PaymentSent]
assert(result.id === paymentId)
assert(result.amountWithFees === payment.totalAmount + result.nonTrampolineFees)
assert(result.parts.length === 1)
assert(result.paymentHash === paymentHash)
assert(result.paymentPreimage === paymentPreimage)
assert(result.parts.length === 1 && result.parts.head.id === successId)
assert(result.recipientAmount === finalAmount)
assert(result.recipientNodeId === finalRecipient)
assert(result.amountWithFees < finalAmount) // we got the preimage without paying the full amount
assert(result.nonTrampolineFees === successRoute.fee) // we paid the fee for only one of the partial payments
sender.expectTerminated(payFsm)
sender.expectNoMsg(100 millis)
router.expectNoMsg(100 millis)
childPayFsm.expectNoMsg(100 millis)
}
test("send to remote node via multiple channels") { f =>
test("receive partial failure after success (recipient spec violation)") { f =>
import f._
val payment = SendMultiPartPayment(randomBytes32, e, 3200 * 1000 msat, expiry, 3)
// A network capacity of 1000 sat should split the payment in at least 3 parts.
initPayment(f, payment, emptyStats.copy(capacity = Stats.generate(Seq(1000), d => Satoshi(d.toLong))), localChannels())
val payments = Iterator.iterate(0 msat)(sent => {
val child = childPayFsm.expectMsgType[SendPayment]
assert(child.targetNodeId === e)
assert(child.maxAttempts === 3)
assert(child.finalPayload.expiry === expiry)
assert(child.finalPayload.paymentSecret === Some(payment.paymentSecret))
assert(child.finalPayload.totalAmount === payment.totalAmount)
assert(child.routePrefix.length === 1 && child.routePrefix.head.nodeId === nodeParams.nodeId)
assert(sent + child.finalPayload.amount <= payment.totalAmount)
sent + child.finalPayload.amount
}).takeWhile(sent => sent != payment.totalAmount).toSeq
assert(payments.length > 2)
assert(payments.length < 10)
childPayFsm.expectNoMsg(50 millis)
val payment = SendMultiPartPayment(randomBytes32, e, finalAmount, expiry, 5, routeParams = Some(routeParams))
sender.send(payFsm, payment)
router.expectMsgType[RouteRequest]
router.send(payFsm, RouteResponse(Seq(Route(400000 msat, hop_ab_1 :: hop_be :: Nil), Route(600000 msat, hop_ac_1 :: hop_ce :: Nil))))
childPayFsm.expectMsgType[SendPaymentToRoute]
childPayFsm.expectMsgType[SendPaymentToRoute]
val (childId, route) :: (failedId, failedRoute) :: Nil = payFsm.stateData.asInstanceOf[PaymentProgress].pending.toList
childPayFsm.send(payFsm, PaymentSent(paymentId, paymentHash, paymentPreimage, finalAmount, e, Seq(PaymentSent.PartialPayment(childId, route.amount, route.fee, randomBytes32, Some(route.hops)))))
awaitCond(payFsm.stateName === PAYMENT_SUCCEEDED)
sender.watch(payFsm)
childPayFsm.send(payFsm, PaymentFailed(failedId, paymentHash, Seq(RemoteFailure(failedRoute.hops, Sphinx.DecryptedFailurePacket(e, PaymentTimeout)))))
val result = sender.expectMsgType[PaymentSent]
assert(result.parts.length === 1 && result.parts.head.id === childId)
assert(result.amountWithFees < finalAmount) // we got the preimage without paying the full amount
assert(result.nonTrampolineFees === route.fee) // we paid the fee for only one of the partial payments
sender.expectTerminated(payFsm)
sender.expectNoMsg(100 millis)
router.expectNoMsg(100 millis)
childPayFsm.expectNoMsg(100 millis)
}
def fulfillPendingPayments(f: FixtureParam, childCount: Int): PaymentSent = {
import f._
sender.watch(payFsm)
val pending = payFsm.stateData.asInstanceOf[PaymentProgress].pending
assert(pending.size === childCount)
val partialPayments = pending.map {
case (id, payment) => PartialPayment(id, payment.finalPayload.amount, 1 msat, randomBytes32, Some(hop_ac_1 :: hop_ab_2 :: Nil))
case (childId, route) => PaymentSent.PartialPayment(childId, route.amount, route.fee, randomBytes32, Some(route.hops))
}
partialPayments.foreach(pp => childPayFsm.send(payFsm, PaymentSent(paymentId, paymentHash, paymentPreimage, finalAmount, e, Seq(pp))))
val result = sender.expectMsgType[PaymentSent]
assert(result.id === paymentId)
assert(result.paymentHash === paymentHash)
assert(result.paymentPreimage === paymentPreimage)
assert(result.parts === partialPayments)
assert(result.parts.toSet === partialPayments.toSet)
assert(result.recipientAmount === finalAmount)
assert(result.amountWithFees > (3200 * 1000).msat)
assert(result.trampolineFees === (2200 * 1000).msat)
assert(result.nonTrampolineFees === partialPayments.map(_.feesPaid).sum)
assert(result.recipientNodeId === finalRecipient)
sender.expectTerminated(payFsm)
sender.expectNoMsg(100 millis)
router.expectNoMsg(100 millis)
childPayFsm.expectNoMsg(100 millis)
result
}
test("send to remote node via single big channel") { f =>
def abortAfterFailure(f: FixtureParam, childFailure: PaymentFailed): PaymentFailed = {
import f._
val payment = SendMultiPartPayment(randomBytes32, e, 3500 * 1000 msat, expiry, 3)
// When splitting inside a channel, we need to take the fees of the commit tx into account (multiple outgoing HTLCs
// will increase the size of the commit tx and thus its fee.
val feeRatePerKw = 100
// A network capacity of 1500 sat should split the payment in at least 2 parts.
// We have a single big channel inside which we'll send multiple payments.
val localChannel = OutgoingChannels(Seq(OutgoingChannel(b, channelUpdate_ab_1, makeCommitments(5000 * 1000 msat, feeRatePerKw))))
initPayment(f, payment, emptyStats.copy(capacity = Stats.generate(Seq(1500), d => Satoshi(d.toLong))), localChannel)
waitUntilAmountSent(f, payment.totalAmount)
val pending = payFsm.stateData.asInstanceOf[PaymentProgress].pending
assert(pending.size >= 2)
val partialPayments = pending.map {
case (id, payment) => PartialPayment(id, payment.finalPayload.amount, 1 msat, randomBytes32, None)
}
partialPayments.foreach(pp => childPayFsm.send(payFsm, PaymentSent(paymentId, paymentHash, paymentPreimage, payment.totalAmount, e, Seq(pp))))
val result = sender.expectMsgType[PaymentSent]
assert(result.id === paymentId)
assert(result.paymentHash === paymentHash)
assert(result.paymentPreimage === paymentPreimage)
assert(result.parts === partialPayments)
assert(result.amountWithFees - result.nonTrampolineFees === (3500 * 1000).msat)
assert(result.recipientNodeId === finalRecipient) // the recipient is obtained from the config, not from the request (which may be to the first trampoline node)
assert(result.nonTrampolineFees === partialPayments.map(_.feesPaid).sum)
}
test("send to remote trampoline node") { f =>
import f._
val trampolineTlv = OnionTlv.TrampolineOnion(OnionRoutingPacket(0, ByteVector.fill(33)(0), ByteVector.fill(400)(0), randomBytes32))
val payment = SendMultiPartPayment(randomBytes32, e, 3000 * 1000 msat, expiry, 3, additionalTlvs = Seq(trampolineTlv))
initPayment(f, payment, emptyStats.copy(capacity = Stats.generate(Seq(1000), d => Satoshi(d.toLong))), localChannels())
waitUntilAmountSent(f, payment.totalAmount)
val pending = payFsm.stateData.asInstanceOf[PaymentProgress].pending
pending.foreach {
case (_, p) => assert(p.finalPayload.asInstanceOf[Onion.FinalTlvPayload].records.get[OnionTlv.TrampolineOnion] === Some(trampolineTlv))
}
}
test("split fees between child payments") { f =>
import f._
val routeParams = RouteParams(randomize = false, 100 msat, 0.05, 20, CltvExpiryDelta(144), None, MultiPartParams(10000 msat, 5))
val payment = SendMultiPartPayment(randomBytes32, e, 3000 * 1000 msat, expiry, 3, routeParams = Some(routeParams))
initPayment(f, payment, emptyStats.copy(capacity = Stats.generate(Seq(1000), d => Satoshi(d.toLong))), localChannels())
waitUntilAmountSent(f, 3000 * 1000 msat)
val pending = payFsm.stateData.asInstanceOf[PaymentProgress].pending
assert(pending.size >= 2)
pending.foreach {
case (_, p) =>
assert(p.routeParams.get.maxFeeBase < 50.msat)
assert(p.routeParams.get.maxFeePct == 0.05) // fee percent doesn't need to change
}
}
test("skip empty channels") { f =>
import f._
val payment = SendMultiPartPayment(randomBytes32, e, 3000 * 1000 msat, expiry, 3)
val testChannels = localChannels()
val testChannels1 = testChannels.copy(channels = testChannels.channels ++ Seq(
OutgoingChannel(b, channelUpdate_ab_1.copy(shortChannelId = ShortChannelId(42)), makeCommitments(0 msat, 10)),
OutgoingChannel(e, channelUpdate_ab_1.copy(shortChannelId = ShortChannelId(43)), makeCommitments(0 msat, 10)
)))
initPayment(f, payment, emptyStats.copy(capacity = Stats.generate(Seq(1000), d => Satoshi(d.toLong))), testChannels1)
waitUntilAmountSent(f, payment.totalAmount)
payFsm.stateData.asInstanceOf[PaymentProgress].pending.foreach {
case (id, p) => childPayFsm.send(payFsm, PaymentSent(paymentId, paymentHash, paymentPreimage, payment.totalAmount, e, Seq(PartialPayment(id, p.finalPayload.amount, 5 msat, randomBytes32, None))))
sender.watch(payFsm)
val pendingCount = payFsm.stateData.asInstanceOf[PaymentProgress].pending.size
childPayFsm.send(payFsm, childFailure) // this failure should trigger an abort
if (pendingCount > 1) {
awaitCond(payFsm.stateName === PAYMENT_ABORTED)
assert(payFsm.stateData.asInstanceOf[PaymentAborted].pending.size === pendingCount - 1)
// Fail all remaining child payments.
payFsm.stateData.asInstanceOf[PaymentAborted].pending.foreach(childId =>
childPayFsm.send(payFsm, PaymentFailed(childId, paymentHash, Seq(RemoteFailure(hop_ab_1 :: hop_be :: Nil, Sphinx.DecryptedFailurePacket(e, PaymentTimeout)))))
)
}
val result = sender.expectMsgType[PaymentSent]
assert(result.id === paymentId)
assert(result.amountWithFees > payment.totalAmount)
}
test("retry after error") { f =>
import f._
val payment = SendMultiPartPayment(randomBytes32, e, 3000 * 1000 msat, expiry, 3)
val testChannels = localChannels()
// A network capacity of 1000 sat should split the payment in at least 3 parts.
initPayment(f, payment, emptyStats.copy(capacity = Stats.generate(Seq(1000), d => Satoshi(d.toLong))), testChannels)
waitUntilAmountSent(f, payment.totalAmount)
val pending = payFsm.stateData.asInstanceOf[PaymentProgress].pending
assert(pending.size > 2)
// Simulate a local channel failure and a remote failure.
val faultyLocalChannelId = getFirstHopShortChannelId(pending.head._2)
val faultyLocalPayments = pending.filter { case (_, p) => getFirstHopShortChannelId(p) == faultyLocalChannelId }
val faultyRemotePayment = pending.filter { case (_, p) => getFirstHopShortChannelId(p) != faultyLocalChannelId }.head
faultyLocalPayments.keys.foreach(id => {
childPayFsm.send(payFsm, PaymentFailed(id, paymentHash, LocalFailure(Nil, RouteNotFound) :: Nil))
})
childPayFsm.send(payFsm, PaymentFailed(faultyRemotePayment._1, paymentHash, UnreadableRemoteFailure(Nil) :: Nil))
// We should ask for updated balance to take into account pending payments.
relayer.expectMsg(GetOutgoingChannels())
relayer.send(payFsm, testChannels.copy(channels = testChannels.channels.dropRight(2)))
// The channel that lead to a RouteNotFound should be ignored.
assert(payFsm.stateData.asInstanceOf[PaymentProgress].ignoreChannels === Set(faultyLocalChannelId))
// New payments should be sent that match the failed amount.
waitUntilAmountSent(f, faultyRemotePayment._2.finalPayload.amount + faultyLocalPayments.values.map(_.finalPayload.amount).sum)
val stateData = payFsm.stateData.asInstanceOf[PaymentProgress]
assert(stateData.failures.toSet === Set(LocalFailure(Nil, RouteNotFound), UnreadableRemoteFailure(Nil)))
assert(stateData.pending.values.forall(p => getFirstHopShortChannelId(p) != faultyLocalChannelId))
}
test("cannot send (not enough capacity on local channels)") { f =>
import f._
val payment = SendMultiPartPayment(randomBytes32, e, 3000 * 1000 msat, expiry, 3)
initPayment(f, payment, emptyStats.copy(capacity = Stats.generate(Seq(1000), d => Satoshi(d.toLong))), OutgoingChannels(Seq(
OutgoingChannel(b, channelUpdate_ab_1, makeCommitments(1000 * 1000 msat, 10)),
OutgoingChannel(c, channelUpdate_ac_2, makeCommitments(1000 * 1000 msat, 10)),
OutgoingChannel(d, channelUpdate_ad_1, makeCommitments(1000 * 1000 msat, 10))))
)
val result = sender.expectMsgType[PaymentFailed]
assert(result.id === paymentId)
assert(result.paymentHash === paymentHash)
assert(result.failures.length === 1)
assert(result.failures.head.asInstanceOf[LocalFailure].t === PaymentError.BalanceTooLow)
}
assert(result.failures.nonEmpty)
test("cannot send (fee rate too high)") { f =>
import f._
val payment = SendMultiPartPayment(randomBytes32, e, 2500 * 1000 msat, expiry, 3)
initPayment(f, payment, emptyStats.copy(capacity = Stats.generate(Seq(1000), d => Satoshi(d.toLong))), OutgoingChannels(Seq(
OutgoingChannel(b, channelUpdate_ab_1, makeCommitments(1500 * 1000 msat, 1000)),
OutgoingChannel(c, channelUpdate_ac_2, makeCommitments(1500 * 1000 msat, 1000)),
OutgoingChannel(d, channelUpdate_ad_1, makeCommitments(1500 * 1000 msat, 1000))))
)
val result = sender.expectMsgType[PaymentFailed]
assert(result.id === paymentId)
assert(result.paymentHash === paymentHash)
assert(result.failures.length === 1)
assert(result.failures.head.asInstanceOf[LocalFailure].t === PaymentError.BalanceTooLow)
}
sender.expectTerminated(payFsm)
sender.expectNoMsg(100 millis)
router.expectNoMsg(100 millis)
childPayFsm.expectNoMsg(100 millis)
test("payment timeout") { f =>
import f._
val payment = SendMultiPartPayment(randomBytes32, e, 3000 * 1000 msat, expiry, 5)
initPayment(f, payment, emptyStats.copy(capacity = Stats.generate(Seq(1000), d => Satoshi(d.toLong))), localChannels())
waitUntilAmountSent(f, payment.totalAmount)
val (childId1, _) = payFsm.stateData.asInstanceOf[PaymentProgress].pending.head
// If we receive a timeout failure, we directly abort the payment instead of retrying.
childPayFsm.send(payFsm, PaymentFailed(childId1, paymentHash, RemoteFailure(Nil, Sphinx.DecryptedFailurePacket(e, PaymentTimeout)) :: Nil))
relayer.expectNoMsg(50 millis)
awaitCond(payFsm.stateName === PAYMENT_ABORTED)
}
test("failure received from final recipient") { f =>
import f._
val payment = SendMultiPartPayment(randomBytes32, e, 3000 * 1000 msat, expiry, 5)
initPayment(f, payment, emptyStats.copy(capacity = Stats.generate(Seq(1000), d => Satoshi(d.toLong))), localChannels())
waitUntilAmountSent(f, payment.totalAmount)
val (childId1, _) = payFsm.stateData.asInstanceOf[PaymentProgress].pending.head
// If we receive a failure from the final node, we directly abort the payment instead of retrying.
childPayFsm.send(payFsm, PaymentFailed(childId1, paymentHash, RemoteFailure(Nil, Sphinx.DecryptedFailurePacket(e, IncorrectOrUnknownPaymentDetails(3000 * 1000 msat, 42))) :: Nil))
relayer.expectNoMsg(50 millis)
awaitCond(payFsm.stateName === PAYMENT_ABORTED)
}
test("fail after too many attempts") { f =>
import f._
val payment = SendMultiPartPayment(randomBytes32, e, 3000 * 1000 msat, expiry, 2)
initPayment(f, payment, emptyStats.copy(capacity = Stats.generate(Seq(1000), d => Satoshi(d.toLong))), localChannels())
waitUntilAmountSent(f, payment.totalAmount)
val (childId1, childPayment1) = payFsm.stateData.asInstanceOf[PaymentProgress].pending.head
// We retry one failure.
val failures = Seq(UnreadableRemoteFailure(hop_ab_1 :: Nil), UnreadableRemoteFailure(hop_ac_1 :: hop_ab_2 :: Nil))
childPayFsm.send(payFsm, PaymentFailed(childId1, paymentHash, failures.slice(0, 1)))
relayer.expectMsg(GetOutgoingChannels())
relayer.send(payFsm, localChannels())
waitUntilAmountSent(f, childPayment1.finalPayload.amount)
// But another failure occurs...
val (childId2, _) = payFsm.stateData.asInstanceOf[PaymentProgress].pending.head
childPayFsm.send(payFsm, PaymentFailed(childId2, paymentHash, failures.slice(1, 2)))
relayer.expectNoMsg(50 millis)
awaitCond(payFsm.stateName === PAYMENT_ABORTED)
// And then all other payments time out.
payFsm.stateData.asInstanceOf[PaymentAborted].pending.foreach(childId => childPayFsm.send(payFsm, PaymentFailed(childId, paymentHash, Nil)))
val result = sender.expectMsgType[PaymentFailed]
assert(result.id === paymentId)
assert(result.paymentHash === paymentHash)
assert(result.failures.length === 3)
assert(result.failures.slice(0, 2) === failures)
assert(result.failures.last.asInstanceOf[LocalFailure].t === PaymentError.RetryExhausted)
}
test("receive partial failure after success (recipient spec violation)") { f =>
import f._
val payment = SendMultiPartPayment(randomBytes32, e, 4000 * 1000 msat, expiry, 2)
initPayment(f, payment, emptyStats.copy(capacity = Stats.generate(Seq(1500), d => Satoshi(d.toLong))), localChannels())
waitUntilAmountSent(f, payment.totalAmount)
val pending = payFsm.stateData.asInstanceOf[PaymentProgress].pending
// If one of the payments succeeds, the recipient MUST succeed them all: we can consider the whole payment succeeded.
val (id1, payment1) = pending.head
childPayFsm.send(payFsm, PaymentSent(paymentId, paymentHash, paymentPreimage, payment.totalAmount, e, Seq(PartialPayment(id1, payment1.finalPayload.amount, 0 msat, randomBytes32, None))))
awaitCond(payFsm.stateName === PAYMENT_SUCCEEDED)
// A partial failure should simply be ignored.
val (id2, payment2) = pending.tail.head
childPayFsm.send(payFsm, PaymentFailed(id2, paymentHash, Nil))
pending.tail.tail.foreach {
case (id, p) => childPayFsm.send(payFsm, PaymentSent(paymentId, paymentHash, paymentPreimage, payment.totalAmount, e, Seq(PartialPayment(id, p.finalPayload.amount, 0 msat, randomBytes32, None))))
}
val result = sender.expectMsgType[PaymentSent]
assert(result.id === paymentId)
assert(result.amountWithFees === payment.totalAmount - payment2.finalPayload.amount)
}
test("receive partial success after abort (recipient spec violation)") { f =>
import f._
val payment = SendMultiPartPayment(randomBytes32, e, 5000 * 1000 msat, expiry, 1)
initPayment(f, payment, emptyStats.copy(capacity = Stats.generate(Seq(2000), d => Satoshi(d.toLong))), localChannels())
waitUntilAmountSent(f, payment.totalAmount)
val pending = payFsm.stateData.asInstanceOf[PaymentProgress].pending
// One of the payments failed and we configured maxAttempts = 1, so we abort.
val (id1, _) = pending.head
childPayFsm.send(payFsm, PaymentFailed(id1, paymentHash, Nil))
awaitCond(payFsm.stateName === PAYMENT_ABORTED)
// The in-flight HTLC set doesn't pay the full amount, so the recipient MUST not fulfill any of those.
// But if he does, it's too bad for him as we have obtained a cheaper proof of payment.
val (id2, payment2) = pending.tail.head
childPayFsm.send(payFsm, PaymentSent(paymentId, paymentHash, paymentPreimage, payment.totalAmount, e, Seq(PartialPayment(id2, payment2.finalPayload.amount, 5 msat, randomBytes32, None))))
awaitCond(payFsm.stateName === PAYMENT_SUCCEEDED)
// Even if all other child payments fail, we obtained the preimage so the payment is a success from our point of view.
pending.tail.tail.foreach {
case (id, _) => childPayFsm.send(payFsm, PaymentFailed(id, paymentHash, Nil))
}
val result = sender.expectMsgType[PaymentSent]
assert(result.id === paymentId)
assert(result.amountWithFees === payment2.finalPayload.amount + 5.msat)
assert(result.nonTrampolineFees === 5.msat)
}
test("split payment", Tag("fuzzy")) { f =>
// The fees for a single HTLC will be 100 * 172 / 1000 = 17 satoshis.
val testChannels = localChannels(100)
for (_ <- 1 to 100) {
// We have a total of 6500 satoshis across all channels. We try to send lower amounts to take fees into account.
val toSend = ((1 + Random.nextInt(3500)) * 1000).msat
val networkStats = emptyStats.copy(capacity = Stats.generate(Seq(400 + Random.nextInt(1600)), d => Satoshi(d.toLong)))
val routeParams = RouteParams(randomize = true, Random.nextInt(1000).msat, Random.nextInt(10).toDouble / 100, 20, CltvExpiryDelta(144), None, MultiPartParams(10000 msat, 5))
val request = SendMultiPartPayment(randomBytes32, e, toSend, CltvExpiry(561), 1, Nil, Some(routeParams))
val fuzzParams = s"(sending $toSend with network capacity ${networkStats.capacity.percentile75.toMilliSatoshi}, fee base ${routeParams.maxFeeBase} and fee percentage ${routeParams.maxFeePct})"
val (remaining, payments) = splitPayment(f.nodeParams, toSend, testChannels.channels, Some(networkStats), request, randomize = true)
assert(remaining === 0.msat, fuzzParams)
assert(payments.nonEmpty, fuzzParams)
assert(payments.map(_.finalPayload.amount).sum === toSend, fuzzParams)
}
result
}
}
@ -513,6 +455,8 @@ object MultiPartPaymentLifecycleSpec {
val expiry = CltvExpiry(1105)
val finalAmount = 1000000 msat
val finalRecipient = randomKey.publicKey
val routeParams = RouteParams(randomize = false, 15000 msat, 0.01, 6, CltvExpiryDelta(1008), None, MultiPartParams(1000 msat, 5))
val maxFee = 15000 msat // max fee for the defaultAmount
/**
* We simulate a multi-part-friendly network:
@ -527,35 +471,29 @@ object MultiPartPaymentLifecycleSpec {
val a :: b :: c :: d :: e :: Nil = Seq.fill(5)(randomKey.publicKey)
val channelId_ab_1 = ShortChannelId(1)
val channelId_ab_2 = ShortChannelId(2)
val channelId_be = ShortChannelId(3)
val channelId_ac_1 = ShortChannelId(11)
val channelId_ac_2 = ShortChannelId(12)
val channelId_ac_3 = ShortChannelId(13)
val channelId_ad_1 = ShortChannelId(21)
val defaultChannelUpdate = ChannelUpdate(randomBytes64, Block.RegtestGenesisBlock.hash, ShortChannelId(0), 0, 1, ChannelFlags.AnnounceChannel, CltvExpiryDelta(12), 1 msat, 0 msat, 0, Some(2000 * 1000 msat))
val channelUpdate_ab_1 = defaultChannelUpdate.copy(shortChannelId = channelId_ab_1, cltvExpiryDelta = CltvExpiryDelta(4), feeBaseMsat = 100 msat, feeProportionalMillionths = 70)
val channelUpdate_ab_2 = defaultChannelUpdate.copy(shortChannelId = channelId_ab_2, cltvExpiryDelta = CltvExpiryDelta(4), feeBaseMsat = 100 msat, feeProportionalMillionths = 70)
val channelUpdate_ac_1 = defaultChannelUpdate.copy(shortChannelId = channelId_ac_1, cltvExpiryDelta = CltvExpiryDelta(5), feeBaseMsat = 150 msat, feeProportionalMillionths = 40)
val channelUpdate_ac_2 = defaultChannelUpdate.copy(shortChannelId = channelId_ac_2, cltvExpiryDelta = CltvExpiryDelta(5), feeBaseMsat = 150 msat, feeProportionalMillionths = 40)
val channelUpdate_ac_3 = defaultChannelUpdate.copy(shortChannelId = channelId_ac_3, cltvExpiryDelta = CltvExpiryDelta(5), feeBaseMsat = 150 msat, feeProportionalMillionths = 40)
val channelUpdate_ad_1 = defaultChannelUpdate.copy(shortChannelId = channelId_ad_1, cltvExpiryDelta = CltvExpiryDelta(6), feeBaseMsat = 200 msat, feeProportionalMillionths = 50)
// With a fee rate of 10, the fees for a single HTLC will be 10 * 172 / 1000 = 1 satoshi.
def localChannels(feeRatePerKw: Long = 10): OutgoingChannels = OutgoingChannels(Seq(
OutgoingChannel(b, channelUpdate_ab_1, makeCommitments(1000 * 1000 msat, feeRatePerKw)),
OutgoingChannel(b, channelUpdate_ab_2, makeCommitments(1500 * 1000 msat, feeRatePerKw)),
OutgoingChannel(c, channelUpdate_ac_1, makeCommitments(500 * 1000 msat, feeRatePerKw)),
OutgoingChannel(c, channelUpdate_ac_2, makeCommitments(1000 * 1000 msat, feeRatePerKw)),
OutgoingChannel(c, channelUpdate_ac_3, makeCommitments(1500 * 1000 msat, feeRatePerKw)),
OutgoingChannel(d, channelUpdate_ad_1, makeCommitments(1000 * 1000 msat, feeRatePerKw))))
val channelId_ce = ShortChannelId(13)
val channelId_ad = ShortChannelId(21)
val channelId_de = ShortChannelId(22)
val defaultChannelUpdate = ChannelUpdate(randomBytes64, Block.RegtestGenesisBlock.hash, ShortChannelId(0), 0, 1, ChannelFlags.AnnounceChannel, CltvExpiryDelta(12), 1 msat, 100 msat, 0, Some(2000000 msat))
val channelUpdate_ab_1 = defaultChannelUpdate.copy(shortChannelId = channelId_ab_1)
val channelUpdate_ab_2 = defaultChannelUpdate.copy(shortChannelId = channelId_ab_2)
val channelUpdate_be = defaultChannelUpdate.copy(shortChannelId = channelId_be)
val channelUpdate_ac_1 = defaultChannelUpdate.copy(shortChannelId = channelId_ac_1)
val channelUpdate_ac_2 = defaultChannelUpdate.copy(shortChannelId = channelId_ac_2)
val channelUpdate_ce = defaultChannelUpdate.copy(shortChannelId = channelId_ce)
val channelUpdate_ad = defaultChannelUpdate.copy(shortChannelId = channelId_ad)
val channelUpdate_de = defaultChannelUpdate.copy(shortChannelId = channelId_de)
val hop_ab_1 = ChannelHop(a, b, channelUpdate_ab_1)
val hop_ab_2 = ChannelHop(a, b, channelUpdate_ab_2)
val hop_be = ChannelHop(b, e, channelUpdate_be)
val hop_ac_1 = ChannelHop(a, c, channelUpdate_ac_1)
val emptyStats = NetworkStats(0, 0, Stats.generate(Seq(0), d => Satoshi(d.toLong)), Stats.generate(Seq(0), d => CltvExpiryDelta(d.toInt)), Stats.generate(Seq(0), d => MilliSatoshi(d.toLong)), Stats.generate(Seq(0), d => d.toLong))
// We are only interested in availableBalanceForSend so we can put dummy values for the rest.
def makeCommitments(canSend: MilliSatoshi, feeRatePerKw: Long, announceChannel: Boolean = true): Commitments =
CommitmentsSpec.makeCommitments(canSend, 0 msat, feeRatePerKw, 0 sat, announceChannel = announceChannel)
val hop_ac_2 = ChannelHop(a, c, channelUpdate_ac_2)
val hop_ce = ChannelHop(c, e, channelUpdate_ce)
val hop_ad = ChannelHop(a, d, channelUpdate_ad)
val hop_de = ChannelHop(d, e, channelUpdate_de)
}

View file

@ -31,7 +31,7 @@ import fr.acinq.eclair.payment.send.MultiPartPaymentLifecycle.SendMultiPartPayme
import fr.acinq.eclair.payment.send.PaymentError
import fr.acinq.eclair.payment.send.PaymentInitiator.SendPaymentConfig
import fr.acinq.eclair.payment.send.PaymentLifecycle.SendPayment
import fr.acinq.eclair.router.RouteNotFound
import fr.acinq.eclair.router.{BalanceTooLow, RouteNotFound}
import fr.acinq.eclair.wire._
import fr.acinq.eclair.{CltvExpiry, CltvExpiryDelta, LongToBtcAmount, MilliSatoshi, NodeParams, ShortChannelId, TestConstants, TestKitBaseClass, nodeFee, randomBytes, randomBytes32, randomKey}
import org.scalatest.Outcome
@ -56,16 +56,16 @@ class NodeRelayerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike {
within(30 seconds) {
val nodeParams = TestConstants.Bob.nodeParams
val outgoingPayFSM = TestProbe()
val (relayer, router, commandBuffer, register, eventListener) = (TestProbe(), TestProbe(), TestProbe(), TestProbe(), TestProbe())
val (router, commandBuffer, register, eventListener) = (TestProbe(), TestProbe(), TestProbe(), TestProbe())
system.eventStream.subscribe(eventListener.ref, classOf[PaymentEvent])
class TestNodeRelayer extends NodeRelayer(nodeParams, relayer.ref, router.ref, commandBuffer.ref, register.ref) {
class TestNodeRelayer extends NodeRelayer(nodeParams, router.ref, commandBuffer.ref, register.ref) {
override def spawnOutgoingPayFSM(cfg: SendPaymentConfig, multiPart: Boolean): ActorRef = {
outgoingPayFSM.ref ! cfg
outgoingPayFSM.ref
}
}
val nodeRelayer = TestActorRef(new TestNodeRelayer().asInstanceOf[NodeRelayer])
withFixture(test.toNoArgTest(FixtureParam(nodeParams, nodeRelayer, relayer, outgoingPayFSM, commandBuffer, eventListener)))
withFixture(test.toNoArgTest(FixtureParam(nodeParams, nodeRelayer, TestProbe(), outgoingPayFSM, commandBuffer, eventListener)))
}
}
@ -219,7 +219,7 @@ class NodeRelayerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike {
val outgoingPaymentId = outgoingPayFSM.expectMsgType[SendPaymentConfig].id
outgoingPayFSM.expectMsgType[SendMultiPartPayment]
outgoingPayFSM.send(nodeRelayer, PaymentFailed(outgoingPaymentId, paymentHash, LocalFailure(Nil, PaymentError.BalanceTooLow) :: Nil))
outgoingPayFSM.send(nodeRelayer, PaymentFailed(outgoingPaymentId, paymentHash, LocalFailure(Nil, BalanceTooLow) :: Nil))
incomingMultiPart.foreach(p => commandBuffer.expectMsg(CommandBuffer.CommandSend(p.add.channelId, CMD_FAIL_HTLC(p.add.id, Right(TemporaryNodeFailure), commit = true))))
commandBuffer.expectNoMsg(100 millis)
eventListener.expectNoMsg(100 millis)
@ -249,7 +249,7 @@ class NodeRelayerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike {
val outgoingPaymentId = outgoingPayFSM.expectMsgType[SendPaymentConfig].id
outgoingPayFSM.expectMsgType[SendMultiPartPayment]
val failures = RemoteFailure(Nil, Sphinx.DecryptedFailurePacket(outgoingNodeId, FinalIncorrectHtlcAmount(42 msat))) :: UnreadableRemoteFailure(Nil) :: LocalFailure(Nil, RouteNotFound) :: Nil
val failures = RemoteFailure(Nil, Sphinx.DecryptedFailurePacket(outgoingNodeId, FinalIncorrectHtlcAmount(42 msat))) :: UnreadableRemoteFailure(Nil) :: Nil
outgoingPayFSM.send(nodeRelayer, PaymentFailed(outgoingPaymentId, paymentHash, failures))
incomingMultiPart.foreach(p => commandBuffer.expectMsg(CommandBuffer.CommandSend(p.add.channelId, CMD_FAIL_HTLC(p.add.id, Right(FinalIncorrectHtlcAmount(42 msat)), commit = true))))
commandBuffer.expectNoMsg(100 millis)
@ -329,8 +329,7 @@ class NodeRelayerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike {
commandBuffer.expectNoMsg(100 millis)
}
// TODO: re-activate this test once we have better MPP split to remote legacy recipients
ignore("relay to non-trampoline recipient supporting multi-part") { f =>
test("relay to non-trampoline recipient supporting multi-part") { f =>
import f._
// Receive an upstream multi-part payment.
@ -378,7 +377,6 @@ class NodeRelayerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike {
val outgoingCfg = outgoingPayFSM.expectMsgType[SendPaymentConfig]
validateOutgoingCfg(outgoingCfg, Upstream.TrampolineRelayed(incomingMultiPart.map(_.add)))
val outgoingPayment = outgoingPayFSM.expectMsgType[SendPayment]
assert(outgoingPayment.routePrefix === Nil)
assert(outgoingPayment.finalPayload.amount === outgoingAmount)
assert(outgoingPayment.finalPayload.expiry === outgoingExpiry)
assert(outgoingPayment.targetNodeId === outgoingNodeId)

View file

@ -69,7 +69,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
val (sender, payFsm, multiPartPayFsm) = (TestProbe(), TestProbe(), TestProbe())
val eventListener = TestProbe()
system.eventStream.subscribe(eventListener.ref, classOf[PaymentEvent])
class TestPaymentInitiator extends PaymentInitiator(nodeParams, TestProbe().ref, TestProbe().ref, TestProbe().ref) {
class TestPaymentInitiator extends PaymentInitiator(nodeParams, TestProbe().ref, TestProbe().ref) {
// @formatter:off
override def spawnPaymentFsm(cfg: SendPaymentConfig): ActorRef = {
payFsm.ref ! cfg
@ -116,7 +116,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
sender.send(initiator, SendPaymentToRouteRequest(finalAmount, finalAmount, None, None, pr, Channel.MIN_CLTV_EXPIRY_DELTA, Seq(a, b, c), None, 0 msat, CltvExpiryDelta(0), Nil))
val payment = sender.expectMsgType[SendPaymentToRouteResponse]
payFsm.expectMsg(SendPaymentConfig(payment.paymentId, payment.parentId, None, paymentHash, finalAmount, c, Upstream.Local(payment.paymentId), Some(pr), storeInDb = true, publishEvent = true, Nil))
payFsm.expectMsg(SendPaymentToRoute(Seq(a, b, c), FinalLegacyPayload(finalAmount, Channel.MIN_CLTV_EXPIRY_DELTA.toCltvExpiry(nodeParams.currentBlockHeight + 1))))
payFsm.expectMsg(SendPaymentToRoute(Left(Seq(a, b, c)), FinalLegacyPayload(finalAmount, Channel.MIN_CLTV_EXPIRY_DELTA.toCltvExpiry(nodeParams.currentBlockHeight + 1))))
}
test("forward legacy payment") { f =>
@ -162,7 +162,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
val payment = sender.expectMsgType[SendPaymentToRouteResponse]
payFsm.expectMsg(SendPaymentConfig(payment.paymentId, payment.parentId, None, paymentHash, finalAmount, c, Upstream.Local(payment.paymentId), Some(pr), storeInDb = true, publishEvent = true, Nil))
val msg = payFsm.expectMsgType[SendPaymentToRoute]
assert(msg.hops === Seq(a, b, c))
assert(msg.route === Left(Seq(a, b, c)))
assert(msg.finalPayload.amount === finalAmount / 2)
assert(msg.finalPayload.paymentSecret === pr.paymentSecret)
assert(msg.finalPayload.totalAmount === finalAmount)
@ -326,7 +326,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
assert(payment.trampolineSecret.nonEmpty)
payFsm.expectMsg(SendPaymentConfig(payment.paymentId, payment.parentId, None, paymentHash, finalAmount, c, Upstream.Local(payment.paymentId), Some(pr), storeInDb = true, publishEvent = true, Seq(NodeHop(b, c, CltvExpiryDelta(0), 0 msat))))
val msg = payFsm.expectMsgType[SendPaymentToRoute]
assert(msg.hops === Seq(a, b))
assert(msg.route === Left(Seq(a, b)))
assert(msg.finalPayload.amount === finalAmount + trampolineFees)
assert(msg.finalPayload.paymentSecret === payment.trampolineSecret)
assert(msg.finalPayload.totalAmount === finalAmount + trampolineFees)

View file

@ -86,12 +86,35 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
PaymentFixture(id, parentId, nodeParams, paymentFSM, routerForwarder, register, sender, monitor, eventListener)
}
test("send to route") { routerFixture =>
test("send to route") { _ =>
val payFixture = createPaymentLifecycle()
import payFixture._
// pre-computed route going from A to D
val request = SendPaymentToRoute(Seq(a, b, c, d), FinalLegacyPayload(defaultAmountMsat, defaultExpiry))
val route = Route(defaultAmountMsat, ChannelHop(a, b, update_ab) :: ChannelHop(b, c, update_bc) :: ChannelHop(c, d, update_cd) :: Nil)
val request = SendPaymentToRoute(Right(route), FinalLegacyPayload(defaultAmountMsat, defaultExpiry))
sender.send(paymentFSM, request)
routerForwarder.expectNoMsg(100 millis) // we don't need the router, we have the pre-computed route
val Transition(_, WAITING_FOR_REQUEST, WAITING_FOR_ROUTE) = monitor.expectMsgClass(classOf[Transition[_]])
val Transition(_, WAITING_FOR_ROUTE, WAITING_FOR_PAYMENT_COMPLETE) = monitor.expectMsgClass(classOf[Transition[_]])
awaitCond(nodeParams.db.payments.getOutgoingPayment(id).exists(_.status == OutgoingPaymentStatus.Pending))
val Some(outgoing) = nodeParams.db.payments.getOutgoingPayment(id)
assert(outgoing.copy(createdAt = 0) === OutgoingPayment(id, parentId, Some(defaultExternalId), defaultPaymentHash, PaymentType.Standard, defaultAmountMsat, defaultAmountMsat, d, 0, None, OutgoingPaymentStatus.Pending))
sender.send(paymentFSM, Relayer.ForwardRemoteFulfill(UpdateFulfillHtlc(ByteVector32.Zeroes, 0, defaultPaymentPreimage), defaultOrigin, UpdateAddHtlc(ByteVector32.Zeroes, 0, defaultAmountMsat, defaultPaymentHash, defaultExpiry, TestConstants.emptyOnionPacket)))
val ps = sender.expectMsgType[PaymentSent]
assert(ps.id === parentId)
assert(ps.parts.head.route === Some(route.hops))
awaitCond(nodeParams.db.payments.getOutgoingPayment(id).exists(_.status.isInstanceOf[OutgoingPaymentStatus.Succeeded]))
}
test("send to route (node_id only)") { routerFixture =>
val payFixture = createPaymentLifecycle()
import payFixture._
// pre-computed route going from A to D
val request = SendPaymentToRoute(Left(Seq(a, b, c, d)), FinalLegacyPayload(defaultAmountMsat, defaultExpiry))
sender.send(paymentFSM, request)
routerForwarder.expectMsg(FinalizeRoute(defaultAmountMsat, Seq(a, b, c, d)))
@ -113,7 +136,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
val payFixture = createPaymentLifecycle()
import payFixture._
val brokenRoute = SendPaymentToRoute(Seq(randomKey.publicKey, randomKey.publicKey, randomKey.publicKey), FinalLegacyPayload(defaultAmountMsat, defaultExpiry))
val brokenRoute = SendPaymentToRoute(Left(Seq(randomKey.publicKey, randomKey.publicKey, randomKey.publicKey)), FinalLegacyPayload(defaultAmountMsat, defaultExpiry))
sender.send(paymentFSM, brokenRoute)
routerForwarder.expectMsgType[FinalizeRoute]
routerForwarder.forward(routerFixture.router)
@ -128,7 +151,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
val recipient = randomKey.publicKey
val routingHint = Seq(Seq(ExtraHop(c, ShortChannelId(561), 1 msat, 100, CltvExpiryDelta(144))))
val request = SendPaymentToRoute(Seq(a, b, c, recipient), FinalLegacyPayload(defaultAmountMsat, defaultExpiry), routingHint)
val request = SendPaymentToRoute(Left(Seq(a, b, c, recipient)), FinalLegacyPayload(defaultAmountMsat, defaultExpiry), routingHint)
sender.send(paymentFSM, request)
routerForwarder.expectMsg(FinalizeRoute(defaultAmountMsat, Seq(a, b, c, recipient), routingHint))
@ -145,51 +168,6 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
awaitCond(nodeParams.db.payments.getOutgoingPayment(id).exists(_.status.isInstanceOf[OutgoingPaymentStatus.Succeeded]))
}
test("send with route prefix") { _ =>
val payFixture = createPaymentLifecycle()
import payFixture._
val request = SendPayment(d, FinalLegacyPayload(defaultAmountMsat, defaultExpiry), 3, routePrefix = Seq(ChannelHop(a, b, update_ab), ChannelHop(b, c, update_bc)))
sender.send(paymentFSM, request)
routerForwarder.expectMsg(defaultRouteRequest(c, d).copy(ignoreNodes = Set(a, b)))
val Transition(_, WAITING_FOR_REQUEST, WAITING_FOR_ROUTE) = monitor.expectMsgClass(classOf[Transition[_]])
awaitCond(nodeParams.db.payments.getOutgoingPayment(id).exists(_.status == OutgoingPaymentStatus.Pending))
routerForwarder.send(paymentFSM, RouteResponse(Route(defaultAmountMsat, Seq(ChannelHop(c, d, update_cd))) :: Nil))
val Transition(_, WAITING_FOR_ROUTE, WAITING_FOR_PAYMENT_COMPLETE) = monitor.expectMsgClass(classOf[Transition[_]])
}
test("send with whole route prefix") { _ =>
val payFixture = createPaymentLifecycle()
import payFixture._
val request = SendPayment(c, FinalLegacyPayload(defaultAmountMsat, defaultExpiry), 3, routePrefix = Seq(ChannelHop(a, b, update_ab), ChannelHop(b, c, update_bc)))
sender.send(paymentFSM, request)
routerForwarder.expectNoMsg(50 millis) // we don't need the router when we already have the whole route
val Transition(_, WAITING_FOR_REQUEST, WAITING_FOR_ROUTE) = monitor.expectMsgClass(classOf[Transition[_]])
val Transition(_, WAITING_FOR_ROUTE, WAITING_FOR_PAYMENT_COMPLETE) = monitor.expectMsgClass(classOf[Transition[_]])
awaitCond(nodeParams.db.payments.getOutgoingPayment(id).exists(_.status == OutgoingPaymentStatus.Pending))
}
test("send with route prefix and retry") { _ =>
val payFixture = createPaymentLifecycle()
import payFixture._
val request = SendPayment(d, FinalLegacyPayload(defaultAmountMsat, defaultExpiry), 3, routePrefix = Seq(ChannelHop(a, b, update_ab), ChannelHop(b, c, update_bc)))
sender.send(paymentFSM, request)
routerForwarder.expectMsg(defaultRouteRequest(c, d).copy(ignoreNodes = Set(a, b)))
val Transition(_, WAITING_FOR_REQUEST, WAITING_FOR_ROUTE) = monitor.expectMsgClass(classOf[Transition[_]])
awaitCond(nodeParams.db.payments.getOutgoingPayment(id).exists(_.status == OutgoingPaymentStatus.Pending))
routerForwarder.send(paymentFSM, RouteResponse(Route(defaultAmountMsat, Seq(ChannelHop(c, d, update_cd))) :: Nil))
val Transition(_, WAITING_FOR_ROUTE, WAITING_FOR_PAYMENT_COMPLETE) = monitor.expectMsgClass(classOf[Transition[_]])
sender.send(paymentFSM, UpdateFailHtlc(randomBytes32, 0, randomBytes(Sphinx.FailurePacket.PacketLength)))
routerForwarder.expectMsg(defaultRouteRequest(c, d).copy(ignoreNodes = Set(a, b, c)))
val Transition(_, WAITING_FOR_PAYMENT_COMPLETE, WAITING_FOR_ROUTE) = monitor.expectMsgClass(classOf[Transition[_]])
assert(nodeParams.db.payments.getOutgoingPayment(id).exists(_.status == OutgoingPaymentStatus.Pending))
}
test("payment failed (route not found)") { routerFixture =>
val payFixture = createPaymentLifecycle()
import payFixture._
@ -228,23 +206,23 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
routerForwarder.expectMsg(defaultRouteRequest(a, d))
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && nodeParams.db.payments.getOutgoingPayment(id).exists(_.status === OutgoingPaymentStatus.Pending))
val WaitingForRoute(_, _, Nil, _, _) = paymentFSM.stateData
val WaitingForRoute(_, _, Nil, _) = paymentFSM.stateData
routerForwarder.forward(routerFixture.router)
awaitCond(paymentFSM.stateName == WAITING_FOR_PAYMENT_COMPLETE)
val WaitingForComplete(_, _, cmd1, Nil, _, ignoreNodes1, _, route) = paymentFSM.stateData
assert(ignoreNodes1.isEmpty)
val WaitingForComplete(_, _, cmd1, Nil, _, ignore1, route) = paymentFSM.stateData
assert(ignore1.nodes.isEmpty)
register.expectMsg(ForwardShortId(channelId_ab, cmd1))
sender.send(paymentFSM, Relayer.ForwardRemoteFail(UpdateFailHtlc(ByteVector32.Zeroes, 0, randomBytes32), defaultOrigin, UpdateAddHtlc(ByteVector32.Zeroes, 0, defaultAmountMsat, defaultPaymentHash, defaultExpiry, TestConstants.emptyOnionPacket))) // unparsable message
// then the payment lifecycle will ask for a new route excluding all intermediate nodes
routerForwarder.expectMsg(defaultRouteRequest(nodeParams.nodeId, d).copy(ignoreNodes = Set(c)))
routerForwarder.expectMsg(defaultRouteRequest(nodeParams.nodeId, d).copy(ignore = Ignore(Set(c), Set.empty)))
// let's simulate a response by the router with another route
sender.send(paymentFSM, RouteResponse(route :: Nil))
awaitCond(paymentFSM.stateName == WAITING_FOR_PAYMENT_COMPLETE)
val WaitingForComplete(_, _, cmd2, _, _, ignoreNodes2, _, _) = paymentFSM.stateData
assert(ignoreNodes2 === Set(c))
val WaitingForComplete(_, _, cmd2, _, _, ignore2, _) = paymentFSM.stateData
assert(ignore2.nodes === Set(c))
// and reply a 2nd time with an unparsable failure
register.expectMsg(ForwardShortId(channelId_ab, cmd2))
sender.send(paymentFSM, UpdateFailHtlc(ByteVector32.Zeroes, 0, defaultPaymentHash)) // unparsable message
@ -262,17 +240,17 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
sender.send(paymentFSM, request)
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && nodeParams.db.payments.getOutgoingPayment(id).exists(_.status === OutgoingPaymentStatus.Pending))
val WaitingForRoute(_, _, Nil, _, _) = paymentFSM.stateData
val WaitingForRoute(_, _, Nil, _) = paymentFSM.stateData
routerForwarder.expectMsg(defaultRouteRequest(nodeParams.nodeId, d))
routerForwarder.forward(routerFixture.router)
awaitCond(paymentFSM.stateName == WAITING_FOR_PAYMENT_COMPLETE)
val WaitingForComplete(_, _, cmd1, Nil, _, _, _, _) = paymentFSM.stateData
val WaitingForComplete(_, _, cmd1, Nil, _, _, _) = paymentFSM.stateData
register.expectMsg(ForwardShortId(channelId_ab, cmd1))
sender.send(paymentFSM, Status.Failure(AddHtlcFailed(ByteVector32.Zeroes, defaultPaymentHash, ChannelUnavailable(ByteVector32.Zeroes), Local(id, Some(paymentFSM.underlying.self)), None, None)))
// then the payment lifecycle will ask for a new route excluding the channel
routerForwarder.expectMsg(defaultRouteRequest(nodeParams.nodeId, d).copy(ignoreChannels = Set(ChannelDesc(channelId_ab, a, b))))
routerForwarder.expectMsg(defaultRouteRequest(nodeParams.nodeId, d).copy(ignore = Ignore(Set.empty, Set(ChannelDesc(channelId_ab, a, b)))))
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && nodeParams.db.payments.getOutgoingPayment(id).exists(_.status === OutgoingPaymentStatus.Pending)) // payment is still pending because the error is recoverable
}
@ -284,17 +262,17 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
sender.send(paymentFSM, request)
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && nodeParams.db.payments.getOutgoingPayment(id).exists(_.status === OutgoingPaymentStatus.Pending))
val WaitingForRoute(_, _, Nil, _, _) = paymentFSM.stateData
val WaitingForRoute(_, _, Nil, _) = paymentFSM.stateData
routerForwarder.expectMsg(defaultRouteRequest(nodeParams.nodeId, d))
routerForwarder.forward(routerFixture.router)
awaitCond(paymentFSM.stateName == WAITING_FOR_PAYMENT_COMPLETE)
val WaitingForComplete(_, _, cmd1, Nil, _, _, _, _) = paymentFSM.stateData
val WaitingForComplete(_, _, cmd1, Nil, _, _, _) = paymentFSM.stateData
register.expectMsg(ForwardShortId(channelId_ab, cmd1))
sender.send(paymentFSM, UpdateFailMalformedHtlc(ByteVector32.Zeroes, 0, randomBytes32, FailureMessageCodecs.BADONION))
// then the payment lifecycle will ask for a new route excluding the channel
routerForwarder.expectMsg(defaultRouteRequest(a, d).copy(ignoreChannels = Set(ChannelDesc(channelId_ab, a, b))))
routerForwarder.expectMsg(defaultRouteRequest(a, d).copy(ignore = Ignore(Set.empty, Set(ChannelDesc(channelId_ab, a, b)))))
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && nodeParams.db.payments.getOutgoingPayment(id).exists(_.status === OutgoingPaymentStatus.Pending))
}
@ -305,11 +283,11 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
val request = SendPayment(d, FinalLegacyPayload(defaultAmountMsat, defaultExpiry), 2)
sender.send(paymentFSM, request)
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE)
val WaitingForRoute(_, _, Nil, _, _) = paymentFSM.stateData
val WaitingForRoute(_, _, Nil, _) = paymentFSM.stateData
routerForwarder.expectMsg(defaultRouteRequest(nodeParams.nodeId, d))
routerForwarder.forward(routerFixture.router)
awaitCond(paymentFSM.stateName == WAITING_FOR_PAYMENT_COMPLETE)
val WaitingForComplete(_, _, cmd1, Nil, sharedSecrets1, _, _, route) = paymentFSM.stateData
val WaitingForComplete(_, _, cmd1, Nil, sharedSecrets1, _, route) = paymentFSM.stateData
register.expectMsg(ForwardShortId(channelId_ab, cmd1))
val failure = TemporaryChannelFailure(update_bc)
@ -335,11 +313,11 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
sender.send(paymentFSM, request)
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && nodeParams.db.payments.getOutgoingPayment(id).exists(_.status === OutgoingPaymentStatus.Pending))
val WaitingForRoute(_, _, Nil, _, _) = paymentFSM.stateData
val WaitingForRoute(_, _, Nil, _) = paymentFSM.stateData
routerForwarder.expectMsg(defaultRouteRequest(nodeParams.nodeId, d))
routerForwarder.forward(routerFixture.router)
awaitCond(paymentFSM.stateName == WAITING_FOR_PAYMENT_COMPLETE)
val WaitingForComplete(_, _, cmd1, Nil, sharedSecrets1, _, _, route1) = paymentFSM.stateData
val WaitingForComplete(_, _, cmd1, Nil, sharedSecrets1, _, route1) = paymentFSM.stateData
register.expectMsg(ForwardShortId(channelId_ab, cmd1))
// we change the cltv expiry
@ -356,7 +334,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
// router answers with a new route, taking into account the new update
awaitCond(paymentFSM.stateName == WAITING_FOR_PAYMENT_COMPLETE)
val WaitingForComplete(_, _, cmd2, _, sharedSecrets2, _, _, route2) = paymentFSM.stateData
val WaitingForComplete(_, _, cmd2, _, sharedSecrets2, _, route2) = paymentFSM.stateData
register.expectMsg(ForwardShortId(channelId_ab, cmd2))
// we change the cltv expiry one more time
@ -393,11 +371,11 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
sender.send(paymentFSM, request)
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && nodeParams.db.payments.getOutgoingPayment(id).exists(_.status === OutgoingPaymentStatus.Pending))
val WaitingForRoute(_, _, Nil, _, _) = paymentFSM.stateData
val WaitingForRoute(_, _, Nil, _) = paymentFSM.stateData
routerForwarder.expectMsg(defaultRouteRequest(nodeParams.nodeId, d).copy(assistedRoutes = assistedRoutes))
routerForwarder.forward(routerFixture.router)
awaitCond(paymentFSM.stateName == WAITING_FOR_PAYMENT_COMPLETE)
val WaitingForComplete(_, _, cmd1, Nil, sharedSecrets1, _, _, _) = paymentFSM.stateData
val WaitingForComplete(_, _, cmd1, Nil, sharedSecrets1, _, _) = paymentFSM.stateData
register.expectMsg(ForwardShortId(channelId_ab, cmd1))
// we change the cltv expiry
@ -418,7 +396,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
// router answers with a new route, taking into account the new update
awaitCond(paymentFSM.stateName == WAITING_FOR_PAYMENT_COMPLETE)
val WaitingForComplete(_, _, cmd2, _, _, _, _, _) = paymentFSM.stateData
val WaitingForComplete(_, _, cmd2, _, _, _, _) = paymentFSM.stateData
register.expectMsg(ForwardShortId(channelId_ab, cmd2))
assert(cmd2.cltvExpiry > cmd1.cltvExpiry)
}
@ -431,18 +409,18 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
sender.send(paymentFSM, request)
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && nodeParams.db.payments.getOutgoingPayment(id).exists(_.status === OutgoingPaymentStatus.Pending))
val WaitingForRoute(_, _, Nil, _, _) = paymentFSM.stateData
val WaitingForRoute(_, _, Nil, _) = paymentFSM.stateData
routerForwarder.expectMsg(defaultRouteRequest(nodeParams.nodeId, d))
routerForwarder.forward(router)
awaitCond(paymentFSM.stateName == WAITING_FOR_PAYMENT_COMPLETE)
val WaitingForComplete(_, _, cmd1, Nil, sharedSecrets1, _, _, route1) = paymentFSM.stateData
val WaitingForComplete(_, _, cmd1, Nil, sharedSecrets1, _, route1) = paymentFSM.stateData
register.expectMsg(ForwardShortId(channelId_ab, cmd1))
sender.send(paymentFSM, UpdateFailHtlc(ByteVector32.Zeroes, 0, Sphinx.FailurePacket.create(sharedSecrets1.head._1, failure)))
// payment lifecycle forwards the embedded channelUpdate to the router
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE)
routerForwarder.expectMsg(defaultRouteRequest(nodeParams.nodeId, d).copy(ignoreChannels = Set(ChannelDesc(channelId_bc, b, c))))
routerForwarder.expectMsg(defaultRouteRequest(nodeParams.nodeId, d).copy(ignore = Ignore(Set.empty, Set(ChannelDesc(channelId_bc, b, c)))))
routerForwarder.forward(router)
// we allow 2 tries, so we send a 2nd request to the router, which won't find another route
@ -569,9 +547,9 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
)
for ((failure, expectedNodes, expectedChannels) <- testCases) {
val (ignoreNodes, ignoreChannels) = PaymentFailure.updateIgnored(failure, Set.empty[PublicKey], Set.empty[ChannelDesc])
assert(ignoreNodes === expectedNodes, failure)
assert(ignoreChannels === expectedChannels, failure)
val ignore = PaymentFailure.updateIgnored(failure, Ignore.empty)
assert(ignore.nodes === expectedNodes, failure)
assert(ignore.channels === expectedChannels, failure)
}
val failures = Seq(
@ -579,9 +557,9 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
RemoteFailure(route_abcd, Sphinx.DecryptedFailurePacket(b, UnknownNextPeer)),
LocalFailure(route_abcd, new RuntimeException("fatal"))
)
val (ignoreNodes, ignoreChannels) = PaymentFailure.updateIgnored(failures, Set.empty[PublicKey], Set.empty[ChannelDesc])
assert(ignoreNodes === Set(c))
assert(ignoreChannels === Set(ChannelDesc(channelId_ab, a, b), ChannelDesc(channelId_bc, b, c)))
val ignore = PaymentFailure.updateIgnored(failures, Ignore.empty)
assert(ignore.nodes === Set(c))
assert(ignore.channels === Set(ChannelDesc(channelId_ab, a, b), ChannelDesc(channelId_bc, b, c)))
}
test("disable database and events") { routerFixture =>

View file

@ -28,7 +28,7 @@ import fr.acinq.eclair.payment.OutgoingPacket.{buildCommand, buildOnion, buildPa
import fr.acinq.eclair.payment.relay.Origin._
import fr.acinq.eclair.payment.relay.Relayer._
import fr.acinq.eclair.payment.relay.{CommandBuffer, Relayer}
import fr.acinq.eclair.router.Router.{ChannelHop, GetNetworkStats, GetNetworkStatsResponse, NodeHop, TickComputeNetworkStats}
import fr.acinq.eclair.router.Router.{ChannelHop, Ignore, NodeHop}
import fr.acinq.eclair.router.{Announcements, _}
import fr.acinq.eclair.wire.Onion.{ChannelRelayTlvPayload, FinalLegacyPayload, FinalTlvPayload, PerHopPayload}
import fr.acinq.eclair.wire._
@ -177,9 +177,13 @@ class RelayerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike {
sender.send(relayer, ForwardAdd(add_ab2))
// A multi-part payment FSM should start to relay the payment.
router.expectMsg(GetNetworkStats)
router.send(router.lastSender, GetNetworkStatsResponse(None))
router.expectMsg(TickComputeNetworkStats)
val routeRequest1 = router.expectMsgType[Router.RouteRequest]
assert(routeRequest1.source === b)
assert(routeRequest1.target === c)
assert(routeRequest1.amount === finalAmount)
assert(routeRequest1.allowMultiPart)
assert(routeRequest1.ignore === Ignore.empty)
router.send(router.lastSender, Router.RouteResponse(Router.Route(finalAmount, ChannelHop(b, c, channelUpdate_bc) :: Nil) :: Nil))
// first try
val fwd1 = register.expectMsgType[Register.ForwardShortId[CMD_ADD_HTLC]]
@ -191,6 +195,10 @@ class RelayerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike {
sender.send(relayer, Status.Failure(AddHtlcFailed(channelId_bc, paymentHash, HtlcValueTooHighInFlight(channelId_bc, UInt64(1000000000L), 1516977616L msat), origin1, Some(channelUpdate_bc), originalCommand = Some(fwd1.message))))
// second try
val routeRequest2 = router.expectMsgType[Router.RouteRequest]
assert(routeRequest2.ignore.channels.map(_.shortChannelId) === Set(channelUpdate_bc.shortChannelId))
router.send(router.lastSender, Router.RouteResponse(Router.Route(finalAmount, ChannelHop(b, c, channelUpdate_bc) :: Nil) :: Nil))
val fwd2 = register.expectMsgType[Register.ForwardShortId[CMD_ADD_HTLC]]
assert(fwd2.shortChannelId === channelUpdate_bc.shortChannelId)
assert(fwd2.message.upstream.asInstanceOf[Upstream.TrampolineRelayed].adds === Seq(add_ab1, add_ab2))
@ -527,7 +535,10 @@ class RelayerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike {
val add_ab2 = UpdateAddHtlc(channelId_ab, 565, cmd2.amount, cmd2.paymentHash, cmd2.cltvExpiry, cmd2.onion)
sender.send(relayer, ForwardAdd(add_ab1))
sender.send(relayer, ForwardAdd(add_ab2))
router.expectMsg(GetNetworkStats) // A multi-part payment FSM is started to relay the payment downstream.
// A multi-part payment FSM is started to relay the payment downstream.
val routeRequest = router.expectMsgType[Router.RouteRequest]
assert(routeRequest.allowMultiPart)
// We simulate a fake htlc fulfill for the downstream channel.
val payFSM = TestProbe()

View file

@ -402,19 +402,25 @@ class RouterSpec extends BaseRouterSpec {
val sender = TestProbe()
// Via private channels.
sender.send(router, RouteRequest(a, h, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE))
sender.send(router, RouteRequest(a, g, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE))
sender.expectMsgType[RouteResponse]
sender.send(router, RouteRequest(a, h, 50000000 msat, Long.MaxValue.msat))
sender.expectMsg(Failure(RouteNotFound))
sender.send(router, RouteRequest(a, g, 50000000 msat, Long.MaxValue.msat))
sender.expectMsg(Failure(BalanceTooLow))
sender.send(router, RouteRequest(a, g, 50000000 msat, Long.MaxValue.msat, allowMultiPart = true))
sender.expectMsg(Failure(BalanceTooLow))
// Via public channels.
sender.send(router, RouteRequest(a, d, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE))
sender.send(router, RouteRequest(a, b, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE))
sender.expectMsgType[RouteResponse]
val commitments1 = CommitmentsSpec.makeCommitments(10000000 msat, 20000000 msat, a, b, announceChannel = true)
sender.send(router, LocalChannelUpdate(sender.ref, null, channelId_ab, b, Some(chan_ab), update_ab, commitments1))
sender.send(router, RouteRequest(a, d, 12000000 msat, Long.MaxValue.msat))
sender.expectMsg(Failure(RouteNotFound))
sender.send(router, RouteRequest(a, d, 5000000 msat, Long.MaxValue.msat))
sender.send(router, RouteRequest(a, b, 12000000 msat, Long.MaxValue.msat))
sender.expectMsg(Failure(BalanceTooLow))
sender.send(router, RouteRequest(a, b, 12000000 msat, Long.MaxValue.msat, allowMultiPart = true))
sender.expectMsg(Failure(BalanceTooLow))
sender.send(router, RouteRequest(a, b, 5000000 msat, Long.MaxValue.msat))
sender.expectMsgType[RouteResponse]
sender.send(router, RouteRequest(a, b, 5000000 msat, Long.MaxValue.msat, allowMultiPart = true))
sender.expectMsgType[RouteResponse]
}