From 48b490e448fffccfc1a4fb9c5c7c635a80711104 Mon Sep 17 00:00:00 2001 From: t-bast Date: Fri, 15 May 2020 19:33:18 +0200 Subject: [PATCH 1/4] Implement MPP split inside the Router Leverage Yen's k-shortest paths and a simple split algorithm to move MPP entirely inside the Router. This is currently unused, the multipart payment lifecycle needs to be updated to leverage this new algorithm. --- eclair-core/src/main/resources/reference.conf | 5 + .../scala/fr/acinq/eclair/NodeParams.scala | 4 +- .../scala/fr/acinq/eclair/router/Graph.scala | 12 +- .../eclair/router/RouteCalculation.scala | 155 ++++- .../scala/fr/acinq/eclair/router/Router.scala | 12 +- .../scala/fr/acinq/eclair/TestConstants.scala | 8 +- .../eclair/integration/IntegrationSpec.scala | 7 +- .../MultiPartPaymentLifecycleSpec.scala | 4 +- .../eclair/payment/PaymentInitiatorSpec.scala | 4 +- .../eclair/payment/PaymentLifecycleSpec.scala | 2 +- .../eclair/router/RouteCalculationSpec.scala | 569 +++++++++++++++++- 11 files changed, 748 insertions(+), 34 deletions(-) diff --git a/eclair-core/src/main/resources/reference.conf b/eclair-core/src/main/resources/reference.conf index f67b0d4779..5aaf61d0a7 100644 --- a/eclair-core/src/main/resources/reference.conf +++ b/eclair-core/src/main/resources/reference.conf @@ -151,6 +151,11 @@ eclair { ratio-cltv = 0.15 // when computing the weight for a channel, consider its CLTV delta in this proportion ratio-channel-age = 0.35 // when computing the weight for a channel, consider its AGE in this proportion ratio-channel-capacity = 0.5 // when computing the weight for a channel, consider its CAPACITY in this proportion + + mpp { + min-amount-satoshis = 15000 // minimum amount sent via partial HTLCs + max-parts = 6 // maximum number of HTLCs sent per payment: increasing this value will impact performance + } } } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/NodeParams.scala b/eclair-core/src/main/scala/fr/acinq/eclair/NodeParams.scala index cca678cfa9..b52fa8494b 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/NodeParams.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/NodeParams.scala @@ -292,7 +292,9 @@ object NodeParams { searchHeuristicsEnabled = config.getBoolean("router.path-finding.heuristics-enable"), searchRatioCltv = config.getDouble("router.path-finding.ratio-cltv"), searchRatioChannelAge = config.getDouble("router.path-finding.ratio-channel-age"), - searchRatioChannelCapacity = config.getDouble("router.path-finding.ratio-channel-capacity") + searchRatioChannelCapacity = config.getDouble("router.path-finding.ratio-channel-capacity"), + mppMinPartAmount = Satoshi(config.getLong("router.path-finding.mpp.min-amount-satoshis")).toMilliSatoshi, + mppMaxParts = config.getInt("router.path-finding.mpp.max-parts") ), socksProxy_opt = socksProxy_opt, maxPaymentAttempts = config.getInt("max-payment-attempts"), diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/router/Graph.scala b/eclair-core/src/main/scala/fr/acinq/eclair/router/Graph.scala index 186de6f6ca..a72c920872 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/router/Graph.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/router/Graph.scala @@ -375,7 +375,17 @@ object Graph { * @param capacity channel capacity * @param balance_opt (optional) available balance that can be sent through this edge */ - case class GraphEdge(desc: ChannelDesc, update: ChannelUpdate, capacity: Satoshi, balance_opt: Option[MilliSatoshi]) + case class GraphEdge(desc: ChannelDesc, update: ChannelUpdate, capacity: Satoshi, balance_opt: Option[MilliSatoshi]) { + + def maxHtlcAmount(reservedCapacity: MilliSatoshi): MilliSatoshi = Seq( + balance_opt.map(balance => balance - reservedCapacity), + update.htlcMaximumMsat, + Some(capacity.toMilliSatoshi - reservedCapacity) + ).flatten.min.max(0 msat) + + def fee(amount: MilliSatoshi): MilliSatoshi = nodeFee(update.feeBaseMsat, update.feeProportionalMillionths, amount) + + } /** A graph data structure that uses an adjacency list, stores the incoming edges of the neighbors */ case class DirectedGraph(private val vertices: Map[PublicKey, List[GraphEdge]]) { diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/router/RouteCalculation.scala b/eclair-core/src/main/scala/fr/acinq/eclair/router/RouteCalculation.scala index a5eea8328e..33bcaef0c9 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/router/RouteCalculation.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/router/RouteCalculation.scala @@ -30,6 +30,7 @@ import fr.acinq.eclair.wire.ChannelUpdate import fr.acinq.eclair.{ShortChannelId, _} import scala.annotation.tailrec +import scala.collection.mutable import scala.concurrent.duration._ import scala.util.{Failure, Random, Success, Try} @@ -144,22 +145,24 @@ object RouteCalculation { ageFactor = routerConf.searchRatioChannelAge, capacityFactor = routerConf.searchRatioChannelCapacity )) - } + }, + mpp = MultiPartParams(routerConf.mppMinPartAmount, routerConf.mppMaxParts) ) /** * Find a route in the graph between localNodeId and targetNodeId, returns the route. * Will perform a k-shortest path selection given the @param numRoutes and randomly select one of the result. * - * @param g graph of the whole network - * @param localNodeId sender node (payer) - * @param targetNodeId target node (final recipient) - * @param amount the amount that the target node should receive - * @param maxFee the maximum fee of a resulting route - * @param numRoutes the number of routes to find - * @param extraEdges a set of extra edges we want to CONSIDER during the search - * @param ignoredEdges a set of extra edges we want to IGNORE during the search - * @param routeParams a set of parameters that can restrict the route search + * @param g graph of the whole network + * @param localNodeId sender node (payer) + * @param targetNodeId target node (final recipient) + * @param amount the amount that the target node should receive + * @param maxFee the maximum fee of a resulting route + * @param numRoutes the number of routes to find + * @param extraEdges a set of extra edges we want to CONSIDER during the search + * @param ignoredEdges a set of extra edges we want to IGNORE during the search + * @param ignoredVertices a set of extra vertices we want to IGNORE during the search + * @param routeParams a set of parameters that can restrict the route search * @return the computed routes to the destination @param targetNodeId */ def findRoute(g: DirectedGraph, @@ -219,4 +222,136 @@ object RouteCalculation { } } + /** + * Find a multi-part route in the graph between localNodeId and targetNodeId. + * + * @param g graph of the whole network + * @param localNodeId sender node (payer) + * @param targetNodeId target node (final recipient) + * @param amount the amount that the target node should receive + * @param maxFee the maximum fee of a resulting route + * @param extraEdges a set of extra edges we want to CONSIDER during the search + * @param ignoredEdges a set of extra edges we want to IGNORE during the search + * @param ignoredVertices a set of extra vertices we want to IGNORE during the search + * @param pendingHtlcs a list of htlcs that have already been sent for that multi-part payment (used to avoid finding conflicting HTLCs) + * @param routeParams a set of parameters that can restrict the route search + * @return a set of disjoint routes to the destination @param targetNodeId with the payment amount split between them + */ + def findMultiPartRoute(g: DirectedGraph, + localNodeId: PublicKey, + targetNodeId: PublicKey, + amount: MilliSatoshi, + maxFee: MilliSatoshi, + extraEdges: Set[GraphEdge] = Set.empty, + ignoredEdges: Set[ChannelDesc] = Set.empty, + ignoredVertices: Set[PublicKey] = Set.empty, + pendingHtlcs: Seq[Route] = Nil, + routeParams: RouteParams, + currentBlockHeight: Long): Try[Seq[Route]] = Try { + val result = findMultiPartRouteInternal(g, localNodeId, targetNodeId, amount, maxFee, extraEdges, ignoredEdges, ignoredVertices, pendingHtlcs, routeParams, currentBlockHeight) match { + case Right(routes) => Right(routes) + case Left(RouteNotFound) if routeParams.randomize => + // If we couldn't find a randomized solution, fallback to a deterministic one. + findMultiPartRouteInternal(g, localNodeId, targetNodeId, amount, maxFee, extraEdges, ignoredEdges, ignoredVertices, pendingHtlcs, routeParams.copy(randomize = false), currentBlockHeight) + case Left(ex) => Left(ex) + } + result match { + case Right(routes) => routes + case Left(ex) => return Failure(ex) + } + } + + private def findMultiPartRouteInternal(g: DirectedGraph, + localNodeId: PublicKey, + targetNodeId: PublicKey, + amount: MilliSatoshi, + maxFee: MilliSatoshi, + extraEdges: Set[GraphEdge] = Set.empty, + ignoredEdges: Set[ChannelDesc] = Set.empty, + ignoredVertices: Set[PublicKey] = Set.empty, + pendingHtlcs: Seq[Route] = Nil, + routeParams: RouteParams, + currentBlockHeight: Long): Either[RouterException, Seq[Route]] = { + // We use Yen's k-shortest paths to find many paths for chunks of the total amount. + val numRoutes = { + val directChannelsCount = g.getEdgesBetween(localNodeId, targetNodeId).length + routeParams.mpp.maxParts.max(directChannelsCount) // if we have direct channels to the target, we can use them all + } + findRouteInternal(g, localNodeId, targetNodeId, routeParams.mpp.minPartAmount, maxFee, numRoutes, extraEdges, ignoredEdges, ignoredVertices, routeParams, currentBlockHeight) match { + case Right(routes) => + // We use these shortest paths to find a set of non-conflicting HTLCs that send the total amount. + split(amount, mutable.Queue(routes: _*), initializeUsedCapacity(pendingHtlcs), routeParams) match { + case Right(routes) if validateMultiPartRoute(amount, maxFee, routes) => Right(routes) + case _ => Left(RouteNotFound) + } + case Left(ex) => Left(ex) + } + } + + private def split(amount: MilliSatoshi, paths: mutable.Queue[Graph.WeightedPath], usedCapacity: mutable.Map[ShortChannelId, MilliSatoshi], routeParams: RouteParams): Either[RouterException, Seq[Route]] = { + if (amount == 0.msat) { + Right(Nil) + } else if (paths.isEmpty) { + Left(RouteNotFound) + } else { + val current = paths.dequeue() + val candidate = computeRouteMaxAmount(current.path, usedCapacity) + if (candidate.amount < routeParams.mpp.minPartAmount.min(amount)) { + // this route doesn't have enough capacity left: we remove it and continue. + split(amount, paths, usedCapacity, routeParams) + } else { + val route = if (routeParams.randomize) { + // randomly choose the amount to be between 20% and 100% of the available capacity. + val randomizedAmount = candidate.amount * ((20d + Random.nextInt(81)) / 100) + if (randomizedAmount < routeParams.mpp.minPartAmount) { + candidate.copy(amount = routeParams.mpp.minPartAmount.min(amount)) + } else { + candidate.copy(amount = randomizedAmount.min(amount)) + } + } else { + candidate.copy(amount = candidate.amount.min(amount)) + } + updateUsedCapacity(route, usedCapacity) + // NB: we re-enqueue the current path, it may still have capacity for a second HTLC. + split(amount - route.amount, paths.enqueue(current), usedCapacity, routeParams).map(routes => route +: routes) + } + } + } + + /** Compute the maximum amount that we can send through the given route. */ + private def computeRouteMaxAmount(route: Seq[GraphEdge], usedCapacity: mutable.Map[ShortChannelId, MilliSatoshi]): Route = { + val firstHopMaxAmount = route.head.maxHtlcAmount(usedCapacity.getOrElse(route.head.update.shortChannelId, 0 msat)) + val amount = route.drop(1).foldLeft(firstHopMaxAmount) { case (amount, edge) => + // We compute fees going forward instead of backwards. That means we will slightly overestimate the fees of some + // edges, but we will always stay inside the capacity bounds we computed. + val amountMinusFees = amount - edge.fee(amount) + val edgeMaxAmount = edge.maxHtlcAmount(usedCapacity.getOrElse(edge.update.shortChannelId, 0 msat)) + amountMinusFees.min(edgeMaxAmount) + } + Route(amount.max(0 msat), route.map(graphEdgeToHop)) + } + + /** Initialize known used capacity based on pending HTLCs. */ + private def initializeUsedCapacity(pendingHtlcs: Seq[Route]): mutable.Map[ShortChannelId, MilliSatoshi] = { + val usedCapacity = mutable.Map.empty[ShortChannelId, MilliSatoshi] + // We always skip the first hop: since they are local channels, we already take into account those sent HTLCs in the + // channel balance (which overrides the channel capacity in route calculation). + pendingHtlcs.filter(_.hops.length > 1).foreach(route => updateUsedCapacity(route.copy(hops = route.hops.tail), usedCapacity)) + usedCapacity + } + + /** Update used capacity by taking into account an HTLC sent to the given route. */ + private def updateUsedCapacity(route: Route, usedCapacity: mutable.Map[ShortChannelId, MilliSatoshi]): Unit = { + route.hops.reverse.foldLeft(route.amount) { case (amount, hop) => + usedCapacity.updateWith(hop.lastUpdate.shortChannelId)(previous => Some(amount + previous.getOrElse(0 msat))) + amount + hop.fee(amount) + } + } + + private def validateMultiPartRoute(amount: MilliSatoshi, maxFee: MilliSatoshi, routes: Seq[Route]): Boolean = { + val amountOk = routes.map(_.amount).sum == amount + val feeOk = routes.map(_.fee).sum <= maxFee + amountOk && feeOk + } + } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/router/Router.scala b/eclair-core/src/main/scala/fr/acinq/eclair/router/Router.scala index 194e3590e2..bce17f02bf 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/router/Router.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/router/Router.scala @@ -274,7 +274,9 @@ object Router { searchHeuristicsEnabled: Boolean, searchRatioCltv: Double, searchRatioChannelAge: Double, - searchRatioChannelCapacity: Double) + searchRatioChannelCapacity: Double, + mppMinPartAmount: MilliSatoshi, + mppMaxParts: Int) // @formatter:off case class ChannelDesc(shortChannelId: ShortChannelId, a: PublicKey, b: PublicKey) @@ -363,7 +365,9 @@ object Router { override def fee(amount: MilliSatoshi): MilliSatoshi = fee } - case class RouteParams(randomize: Boolean, maxFeeBase: MilliSatoshi, maxFeePct: Double, routeMaxLength: Int, routeMaxCltv: CltvExpiryDelta, ratios: Option[WeightRatios]) { + case class MultiPartParams(minPartAmount: MilliSatoshi, maxParts: Int) + + case class RouteParams(randomize: Boolean, maxFeeBase: MilliSatoshi, maxFeePct: Double, routeMaxLength: Int, routeMaxCltv: CltvExpiryDelta, ratios: Option[WeightRatios], mpp: MultiPartParams) { def getMaxFee(amount: MilliSatoshi): MilliSatoshi = { // The payment fee must satisfy either the flat fee or the percentage fee, not necessarily both. maxFeeBase.max(amount * maxFeePct) @@ -384,6 +388,10 @@ object Router { case class Route(amount: MilliSatoshi, hops: Seq[ChannelHop], allowEmpty: Boolean = false) { require(allowEmpty || 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) } + amountToSend - amount + } /** 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) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/TestConstants.scala b/eclair-core/src/test/scala/fr/acinq/eclair/TestConstants.scala index 5ec345b912..562b2e0ac9 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/TestConstants.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/TestConstants.scala @@ -133,7 +133,9 @@ object TestConstants { searchHeuristicsEnabled = false, searchRatioCltv = 0.0, searchRatioChannelAge = 0.0, - searchRatioChannelCapacity = 0.0 + searchRatioChannelCapacity = 0.0, + mppMinPartAmount = 15000000 msat, + mppMaxParts = 10 ), socksProxy_opt = None, maxPaymentAttempts = 5, @@ -215,7 +217,9 @@ object TestConstants { searchHeuristicsEnabled = false, searchRatioCltv = 0.0, searchRatioChannelAge = 0.0, - searchRatioChannelCapacity = 0.0 + searchRatioChannelCapacity = 0.0, + mppMinPartAmount = 15000000 msat, + mppMaxParts = 10 ), socksProxy_opt = None, maxPaymentAttempts = 5, diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/integration/IntegrationSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/integration/IntegrationSpec.scala index b4326df36d..0413335b5a 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/integration/IntegrationSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/integration/IntegrationSpec.scala @@ -37,8 +37,8 @@ import fr.acinq.eclair.channel._ import fr.acinq.eclair.crypto.Sphinx.DecryptedFailurePacket import fr.acinq.eclair.crypto.TransportHandler import fr.acinq.eclair.db._ -import fr.acinq.eclair.io.{Peer, PeerConnection} import fr.acinq.eclair.io.Peer.{Disconnect, PeerRoutingMessage} +import fr.acinq.eclair.io.{Peer, PeerConnection} import fr.acinq.eclair.payment.PaymentRequest.ExtraHop import fr.acinq.eclair.payment._ import fr.acinq.eclair.payment.receive.MultiPartHandler.ReceivePayment @@ -49,7 +49,7 @@ import fr.acinq.eclair.payment.send.PaymentInitiator.{SendPaymentRequest, SendTr import fr.acinq.eclair.payment.send.PaymentLifecycle.{State => _} import fr.acinq.eclair.router.Graph.WeightRatios import fr.acinq.eclair.router.RouteCalculation.ROUTE_MAX_LENGTH -import fr.acinq.eclair.router.Router.{GossipDecision, PublicChannel, RouteParams, NORMAL => _, State => _} +import fr.acinq.eclair.router.Router.{GossipDecision, MultiPartParams, PublicChannel, RouteParams, NORMAL => _, State => _} import fr.acinq.eclair.router.{Announcements, AnnouncementsBatchValidationSpec, Router} import fr.acinq.eclair.transactions.Transactions import fr.acinq.eclair.transactions.Transactions.{HtlcSuccessTx, HtlcTimeoutTx} @@ -86,7 +86,8 @@ class IntegrationSpec extends TestKitBaseClass with BitcoindService with AnyFunS cltvDeltaFactor = 0.1, ageFactor = 0, capacityFactor = 0 - )) + )), + mpp = MultiPartParams(15000000 msat, 6) )) val commonConfig = ConfigFactory.parseMap(Map( diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartPaymentLifecycleSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartPaymentLifecycleSpec.scala index 5881fe6c34..26270c6a8c 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartPaymentLifecycleSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartPaymentLifecycleSpec.scala @@ -283,7 +283,7 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS test("split fees between child payments") { f => import f._ - val routeParams = RouteParams(randomize = false, 100 msat, 0.05, 20, CltvExpiryDelta(144), None) + 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) @@ -494,7 +494,7 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS // 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) + 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) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentInitiatorSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentInitiatorSpec.scala index 0c0add846d..c73d743c7c 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentInitiatorSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentInitiatorSpec.scala @@ -32,7 +32,7 @@ import fr.acinq.eclair.payment.send.MultiPartPaymentLifecycle.SendMultiPartPayme import fr.acinq.eclair.payment.send.PaymentInitiator._ import fr.acinq.eclair.payment.send.PaymentLifecycle.{SendPayment, SendPaymentToRoute} import fr.acinq.eclair.payment.send.{PaymentError, PaymentInitiator} -import fr.acinq.eclair.router.Router.{NodeHop, RouteParams} +import fr.acinq.eclair.router.Router.{MultiPartParams, NodeHop, RouteParams} import fr.acinq.eclair.wire.Onion.{FinalLegacyPayload, FinalTlvPayload} import fr.acinq.eclair.wire.OnionTlv.{AmountToForward, OutgoingCltv} import fr.acinq.eclair.wire.{Onion, OnionCodecs, OnionTlv, TrampolineFeeInsufficient, _} @@ -122,7 +122,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike test("forward legacy payment") { f => import f._ val hints = Seq(Seq(ExtraHop(b, channelUpdate_bc.shortChannelId, feeBase = 10 msat, feeProportionalMillionths = 1, cltvExpiryDelta = CltvExpiryDelta(12)))) - val routeParams = RouteParams(randomize = true, 15 msat, 1.5, 5, CltvExpiryDelta(561), None) + val routeParams = RouteParams(randomize = true, 15 msat, 1.5, 5, CltvExpiryDelta(561), None, MultiPartParams(10000 msat, 5)) sender.send(initiator, SendPaymentRequest(finalAmount, paymentHash, c, 1, CltvExpiryDelta(42), assistedRoutes = hints, routeParams = Some(routeParams))) val id1 = sender.expectMsgType[UUID] payFsm.expectMsg(SendPaymentConfig(id1, id1, None, paymentHash, finalAmount, c, Upstream.Local(id1), None, storeInDb = true, publishEvent = true, Nil)) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentLifecycleSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentLifecycleSpec.scala index 4bd386437e..c6572696df 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentLifecycleSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentLifecycleSpec.scala @@ -209,7 +209,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec { val payFixture = createPaymentLifecycle() import payFixture._ - val request = SendPayment(d, FinalLegacyPayload(defaultAmountMsat, defaultExpiry), 5, routeParams = Some(RouteParams(randomize = false, maxFeeBase = 100 msat, maxFeePct = 0.0, routeMaxLength = 20, routeMaxCltv = CltvExpiryDelta(2016), ratios = None))) + val request = SendPayment(d, FinalLegacyPayload(defaultAmountMsat, defaultExpiry), 5, routeParams = Some(RouteParams(randomize = false, 100 msat, 0.0, 20, CltvExpiryDelta(2016), None, MultiPartParams(10000 msat, 5)))) sender.send(paymentFSM, request) val routeRequest = routerForwarder.expectMsgType[RouteRequest] val Transition(_, WAITING_FOR_REQUEST, WAITING_FOR_ROUTE) = monitor.expectMsgClass(classOf[Transition[_]]) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/router/RouteCalculationSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/router/RouteCalculationSpec.scala index 7dce698422..22168e78b2 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/router/RouteCalculationSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/router/RouteCalculationSpec.scala @@ -27,12 +27,12 @@ import fr.acinq.eclair.router.Router._ import fr.acinq.eclair.transactions.Transactions import fr.acinq.eclair.wire._ import fr.acinq.eclair.{CltvExpiryDelta, Features, LongToBtcAmount, MilliSatoshi, ShortChannelId, ToMilliSatoshiConversion, randomKey} -import org.scalatest.ParallelTestExecution import org.scalatest.funsuite.AnyFunSuite +import org.scalatest.{ParallelTestExecution, Tag} import scodec.bits._ import scala.collection.immutable.SortedMap -import scala.util.{Failure, Success} +import scala.util.{Failure, Random, Success} /** * Created by PM on 31/05/2016. @@ -925,9 +925,10 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution { ) val g = DirectedGraph.makeGraph(updates) - val params = RouteParams(randomize = false, maxFeeBase = 21000 msat, maxFeePct = 0.03, routeMaxCltv = CltvExpiryDelta(1008), routeMaxLength = 6, ratios = Some( - WeightRatios(cltvDeltaFactor = 0.15, ageFactor = 0.35, capacityFactor = 0.5) - )) + val params = DEFAULT_ROUTE_PARAMS.copy( + routeMaxCltv = CltvExpiryDelta(1008), + ratios = Some(WeightRatios(cltvDeltaFactor = 0.15, ageFactor = 0.35, capacityFactor = 0.5)), + ) val thisNode = PublicKey(hex"036d65409c41ab7380a43448f257809e7496b52bf92057c09c4f300cbd61c50d96") val targetNode = PublicKey(hex"024655b768ef40951b20053a5c4b951606d4d86085d51238f2c67c7dec29c792ca") val amount = 351000 msat @@ -957,6 +958,541 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution { assert(!Graph.validatePath(Seq(ab, bc, cd), 250 msat)) // above balance (AB) } + test("calculate multipart route to neighbor (many channels, known balance)") { + val amount = 65000 msat + val g = DirectedGraph(List( + makeEdge(1L, a, b, 50 msat, 100, minHtlc = 1 msat, balance_opt = Some(15000 msat)), + makeEdge(2L, a, b, 15 msat, 10, minHtlc = 1 msat, balance_opt = Some(25000 msat)), + makeEdge(3L, a, b, 1 msat, 50, minHtlc = 1 msat, balance_opt = Some(20000 msat)), + makeEdge(4L, a, b, 100 msat, 20, minHtlc = 1 msat, balance_opt = Some(10000 msat)), + )) + // We set max-parts to 3, but it should be ignored when sending to a direct neighbor. + val routeParams = DEFAULT_ROUTE_PARAMS.copy(mpp = MultiPartParams(2500 msat, 3)) + + { + val Success(routes) = findMultiPartRoute(g, a, b, amount, 1 msat, routeParams = routeParams, currentBlockHeight = 400000) + assert(routes.length === 4, routes) + assert(routes.forall(_.length == 1), routes) + checkRouteAmounts(routes, amount, 0 msat) + } + { + val Success(routes) = findMultiPartRoute(g, a, b, amount, 1 msat, routeParams = routeParams.copy(randomize = true), currentBlockHeight = 400000) + assert(routes.length >= 4, routes) + assert(routes.forall(_.length == 1), routes) + checkRouteAmounts(routes, amount, 0 msat) + } + } + + test("calculate multipart route to neighbor (single channel, known balance)") { + val g = DirectedGraph(List( + makeEdge(1L, a, b, 50 msat, 100, minHtlc = 1 msat, balance_opt = Some(25000 msat)), + makeEdge(2L, a, c, 1 msat, 0, minHtlc = 1 msat, balance_opt = Some(50000 msat)), + makeEdge(3L, c, b, 1 msat, 0, minHtlc = 1 msat), + makeEdge(4L, a, d, 1 msat, 0, minHtlc = 1 msat, balance_opt = Some(45000 msat)), + )) + + val amount = 25000 msat + val Success(routes) = findMultiPartRoute(g, a, b, amount, 1 msat, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) + assert(routes.length === 1, routes) + checkRouteAmounts(routes, amount, 0 msat) + assert(route2Ids(routes.head) === 1L :: Nil) + } + + test("calculate multipart route to neighbor (many channels, some balance unknown)") { + val g = DirectedGraph(List( + makeEdge(1L, a, b, 50 msat, 100, minHtlc = 1 msat, balance_opt = Some(15000 msat)), + makeEdge(2L, a, b, 15 msat, 10, minHtlc = 1 msat, balance_opt = Some(25000 msat)), + makeEdge(3L, a, b, 1 msat, 50, minHtlc = 1 msat, balance_opt = None, capacity = 20 sat), + makeEdge(4L, a, b, 100 msat, 20, minHtlc = 1 msat, balance_opt = Some(10000 msat)), + makeEdge(5L, a, d, 1 msat, 0, minHtlc = 1 msat, balance_opt = Some(45000 msat)), + )) + + val amount = 65000 msat + val Success(routes) = findMultiPartRoute(g, a, b, amount, 1 msat, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) + assert(routes.length === 4, routes) + assert(routes.forall(_.length == 1), routes) + checkRouteAmounts(routes, amount, 0 msat) + } + + test("calculate multipart route to neighbor (many channels, some empty)") { + val amount = 35000 msat + val g = DirectedGraph(List( + makeEdge(1L, a, b, 50 msat, 100, minHtlc = 1 msat, balance_opt = Some(15000 msat)), + makeEdge(2L, a, b, 15 msat, 10, minHtlc = 1 msat, balance_opt = Some(0 msat)), + makeEdge(3L, a, b, 1 msat, 50, minHtlc = 1 msat, balance_opt = None, capacity = 15 sat), + makeEdge(4L, a, b, 100 msat, 20, minHtlc = 1 msat, balance_opt = Some(10000 msat)), + makeEdge(5L, a, d, 1 msat, 0, minHtlc = 1 msat, balance_opt = Some(45000 msat)), + )) + + { + val Success(routes) = findMultiPartRoute(g, a, b, amount, 1 msat, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) + assert(routes.length === 3, routes) + assert(routes.forall(_.length == 1), routes) + checkIgnoredChannels(routes, 2L) + checkRouteAmounts(routes, amount, 0 msat) + } + { + val Success(routes) = findMultiPartRoute(g, a, b, amount, 1 msat, routeParams = DEFAULT_ROUTE_PARAMS.copy(randomize = true), currentBlockHeight = 400000) + assert(routes.length >= 3, routes) + assert(routes.forall(_.length == 1), routes) + checkIgnoredChannels(routes, 2L) + checkRouteAmounts(routes, amount, 0 msat) + } + } + + test("calculate multipart route to neighbor (ignored channels)") { + val g = DirectedGraph(List( + makeEdge(1L, a, b, 50 msat, 100, minHtlc = 1 msat, balance_opt = Some(15000 msat)), + makeEdge(2L, a, b, 15 msat, 10, minHtlc = 1 msat, balance_opt = Some(25000 msat)), + makeEdge(3L, a, b, 1 msat, 50, minHtlc = 1 msat, balance_opt = None, capacity = 50 sat), + makeEdge(4L, a, b, 100 msat, 20, minHtlc = 1 msat, balance_opt = Some(10000 msat)), + makeEdge(5L, a, b, 1 msat, 10, minHtlc = 1 msat, balance_opt = None, capacity = 10 sat), + makeEdge(6L, a, d, 1 msat, 0, minHtlc = 1 msat, balance_opt = Some(45000 msat)), + )) + + val amount = 20000 msat + val ignoredEdges = Set(ChannelDesc(ShortChannelId(2L), a, b), ChannelDesc(ShortChannelId(3L), a, b)) + val Success(routes) = findMultiPartRoute(g, a, b, amount, 1 msat, ignoredEdges = ignoredEdges, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) + assert(routes.forall(_.length == 1), routes) + checkIgnoredChannels(routes, 2L, 3L) + checkRouteAmounts(routes, amount, 0 msat) + } + + test("calculate multipart route to neighbor (pending htlcs ignored for local channels)") { + val edge_ab_1 = makeEdge(1L, a, b, 50 msat, 100, minHtlc = 1 msat, balance_opt = Some(15000 msat)) + val edge_ab_2 = makeEdge(2L, a, b, 15 msat, 10, minHtlc = 1 msat, balance_opt = Some(25000 msat)) + val edge_ab_3 = makeEdge(3L, a, b, 1 msat, 50, minHtlc = 1 msat, balance_opt = None, capacity = 15 sat) + val g = DirectedGraph(List( + edge_ab_1, + edge_ab_2, + edge_ab_3, + makeEdge(4L, a, d, 1 msat, 0, minHtlc = 1 msat, balance_opt = Some(45000 msat)), + )) + + val amount = 50000 msat + // These pending HTLCs will have already been taken into account in the edge's `balance_opt` field: findMultiPartRoute + // should ignore this information. + val pendingHtlcs = Seq(Route(10000 msat, ChannelHop(a, b, edge_ab_1.update) :: Nil), Route(5000 msat, ChannelHop(a, b, edge_ab_2.update) :: Nil)) + val Success(routes) = findMultiPartRoute(g, a, b, amount, 1 msat, pendingHtlcs = pendingHtlcs, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) + assert(routes.forall(_.length == 1), routes) + checkRouteAmounts(routes, amount, 0 msat) + } + + test("calculate multipart route to neighbor (restricted htlc_maximum_msat)") { + val g = DirectedGraph(List( + makeEdge(1L, a, b, 25 msat, 15, minHtlc = 1 msat, maxHtlc = Some(5000 msat), balance_opt = Some(18000 msat)), + makeEdge(2L, a, b, 15 msat, 10, minHtlc = 1 msat, maxHtlc = Some(5000 msat), balance_opt = Some(23000 msat)), + makeEdge(3L, a, b, 1 msat, 50, minHtlc = 1 msat, maxHtlc = Some(5000 msat), balance_opt = Some(21000 msat)), + makeEdge(4L, a, d, 1 msat, 0, minHtlc = 1 msat, balance_opt = Some(45000 msat)), + )) + + val amount = 50000 msat + val Success(routes) = findMultiPartRoute(g, a, b, amount, 1 msat, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) + assert(routes.forall(_.length == 1), routes) + assert(routes.length >= 10, routes) + assert(routes.forall(_.amount <= 5000.msat), routes) + checkRouteAmounts(routes, amount, 0 msat) + } + + test("calculate multipart route to neighbor (restricted htlc_minimum_msat)") { + val g = DirectedGraph(List( + makeEdge(1L, a, b, 25 msat, 15, minHtlc = 2500 msat, balance_opt = Some(18000 msat)), + makeEdge(2L, a, b, 15 msat, 10, minHtlc = 2500 msat, balance_opt = Some(7000 msat)), + makeEdge(3L, a, b, 1 msat, 50, minHtlc = 2500 msat, balance_opt = Some(10000 msat)), + makeEdge(4L, a, d, 1 msat, 0, minHtlc = 1 msat, balance_opt = Some(45000 msat)), + )) + + val amount = 30000 msat + val routeParams = DEFAULT_ROUTE_PARAMS.copy(mpp = MultiPartParams(2500 msat, 5)) + val Success(routes) = findMultiPartRoute(g, a, b, amount, 1 msat, routeParams = routeParams, currentBlockHeight = 400000) + assert(routes.forall(_.length == 1), routes) + assert(routes.length == 3, routes) + checkRouteAmounts(routes, amount, 0 msat) + } + + test("calculate multipart route to neighbor (through remote channels)") { + val g = DirectedGraph(List( + makeEdge(1L, a, b, 25 msat, 15, minHtlc = 1000 msat, balance_opt = Some(18000 msat)), + makeEdge(2L, a, b, 15 msat, 10, minHtlc = 1000 msat, balance_opt = Some(7000 msat)), + makeEdge(3L, a, c, 1000 msat, 10000, minHtlc = 1000 msat, balance_opt = Some(10000 msat)), + makeEdge(4L, c, b, 10 msat, 1000, minHtlc = 1000 msat), + makeEdge(5L, a, d, 1 msat, 0, minHtlc = 1 msat, balance_opt = Some(25000 msat)), + )) + + val amount = 30000 msat + val maxFeeTooLow = findMultiPartRoute(g, a, b, amount, 1 msat, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) + assert(maxFeeTooLow === Failure(RouteNotFound)) + + val Success(routes) = findMultiPartRoute(g, a, b, amount, 20 msat, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) + assert(routes.forall(_.length <= 2), routes) + assert(routes.length == 3, routes) + checkRouteAmounts(routes, amount, 20 msat) + } + + test("cannot find multipart route to neighbor (not enough balance)") { + val g = DirectedGraph(List( + makeEdge(1L, a, b, 0 msat, 0, minHtlc = 1 msat, balance_opt = Some(15000 msat)), + makeEdge(2L, a, b, 0 msat, 0, minHtlc = 1 msat, balance_opt = Some(5000 msat)), + makeEdge(3L, a, b, 0 msat, 0, minHtlc = 1 msat, balance_opt = Some(10000 msat)), + makeEdge(4L, a, d, 0 msat, 0, minHtlc = 1 msat, balance_opt = Some(45000 msat)), + )) + + { + val result = findMultiPartRoute(g, a, b, 40000 msat, 1 msat, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) + assert(result === Failure(RouteNotFound)) + } + { + val result = findMultiPartRoute(g, a, b, 40000 msat, 1 msat, routeParams = DEFAULT_ROUTE_PARAMS.copy(randomize = true), currentBlockHeight = 400000) + assert(result === Failure(RouteNotFound)) + } + } + + test("cannot find multipart route to neighbor (not enough capacity)") { + val g = DirectedGraph(List( + makeEdge(1L, a, b, 0 msat, 0, minHtlc = 1 msat, capacity = 1500 sat), + makeEdge(2L, a, b, 0 msat, 0, minHtlc = 1 msat, capacity = 2000 sat), + makeEdge(3L, a, b, 0 msat, 0, minHtlc = 1 msat, capacity = 1200 sat), + makeEdge(4L, a, d, 0 msat, 0, minHtlc = 1 msat, capacity = 4500 sat), + )) + + val result = findMultiPartRoute(g, a, b, 5000000 msat, 1 msat, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) + assert(result === Failure(RouteNotFound)) + } + + test("cannot find multipart route to neighbor (restricted htlc_minimum_msat)") { + val g = DirectedGraph(List( + makeEdge(1L, a, b, 25 msat, 15, minHtlc = 5000 msat, balance_opt = Some(6000 msat)), + makeEdge(2L, a, b, 15 msat, 10, minHtlc = 5000 msat, balance_opt = Some(7000 msat)), + makeEdge(3L, a, d, 0 msat, 0, minHtlc = 5000 msat, balance_opt = Some(9000 msat)), + )) + + { + val result = findMultiPartRoute(g, a, b, 10000 msat, 1 msat, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) + assert(result === Failure(RouteNotFound)) + } + { + val result = findMultiPartRoute(g, a, b, 10000 msat, 1 msat, routeParams = DEFAULT_ROUTE_PARAMS.copy(randomize = true), currentBlockHeight = 400000) + assert(result === Failure(RouteNotFound)) + } + } + + test("calculate multipart route to remote node (many local channels)") { + // +-------+ + // | | + // A ----- C ----- E + // | | + // +--- B --- D ---+ + val (amount, maxFee) = (30000 msat, 150 msat) + val edge_ab = makeEdge(1L, a, b, 50 msat, 100, minHtlc = 1 msat, balance_opt = Some(15000 msat)) + val g = DirectedGraph(List( + edge_ab, + makeEdge(2L, b, d, 15 msat, 0, minHtlc = 1 msat, capacity = 25 sat), + makeEdge(3L, d, e, 15 msat, 0, minHtlc = 1 msat, capacity = 20 sat), + makeEdge(4L, a, c, 1 msat, 50, minHtlc = 1 msat, balance_opt = Some(10000 msat)), + makeEdge(5L, a, c, 1 msat, 50, minHtlc = 1 msat, balance_opt = Some(8000 msat)), + makeEdge(6L, c, e, 50 msat, 30, minHtlc = 1 msat, capacity = 20 sat), + )) + + { + val Success(routes) = findMultiPartRoute(g, a, e, amount, maxFee, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) + checkRouteAmounts(routes, amount, maxFee) + assert(routes2Ids(routes) === Set(Seq(1L, 2L, 3L), Seq(4L, 6L), Seq(5L, 6L))) + } + { + // Update A - B with unknown balance, capacity should be used instead. + val g1 = g.addEdge(edge_ab.copy(capacity = 15 sat, balance_opt = None)) + val Success(routes) = findMultiPartRoute(g1, a, e, amount, maxFee, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) + checkRouteAmounts(routes, amount, maxFee) + assert(routes2Ids(routes) === Set(Seq(1L, 2L, 3L), Seq(4L, 6L), Seq(5L, 6L))) + } + { + // Randomize routes. + val Success(routes) = findMultiPartRoute(g, a, e, amount, maxFee, routeParams = DEFAULT_ROUTE_PARAMS.copy(randomize = true), currentBlockHeight = 400000) + checkRouteAmounts(routes, amount, maxFee) + } + { + // Update balance A - B to be too low. + val g1 = g.addEdge(edge_ab.copy(balance_opt = Some(2000 msat))) + val failure = findMultiPartRoute(g1, a, e, amount, maxFee, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) + assert(failure === Failure(RouteNotFound)) + } + { + // Update capacity A - B to be too low. + val g1 = g.addEdge(edge_ab.copy(capacity = 5 sat, balance_opt = None)) + val failure = findMultiPartRoute(g1, a, e, amount, maxFee, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) + assert(failure === Failure(RouteNotFound)) + } + { + // Try to find a route with a maxFee that's too low. + val maxFeeTooLow = 100 msat + val failure = findMultiPartRoute(g, a, e, amount, maxFeeTooLow, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) + assert(failure === Failure(RouteNotFound)) + } + } + + test("calculate multipart route to remote node (single local channel)") { + // +--- C ---+ + // | | + // A --- B ------- D --- F + // | | + // +----- E -------+ + val (amount, maxFee) = (400000 msat, 250 msat) + val edge_ab = makeEdge(1L, a, b, 50 msat, 100, minHtlc = 1 msat, balance_opt = Some(500000 msat)) + val g = DirectedGraph(List( + edge_ab, + makeEdge(2L, b, c, 10 msat, 30, minHtlc = 1 msat, capacity = 150 sat), + makeEdge(3L, c, d, 15 msat, 50, minHtlc = 1 msat, capacity = 150 sat), + makeEdge(4L, b, d, 20 msat, 75, minHtlc = 1 msat, capacity = 180 sat), + makeEdge(5L, d, f, 5 msat, 50, minHtlc = 1 msat, capacity = 300 sat), + makeEdge(6L, b, e, 15 msat, 80, minHtlc = 1 msat, capacity = 210 sat), + makeEdge(7L, e, f, 15 msat, 100, minHtlc = 1 msat, capacity = 200 sat), + )) + + { + val Success(routes) = findMultiPartRoute(g, a, f, amount, maxFee, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) + checkRouteAmounts(routes, amount, maxFee) + assert(routes2Ids(routes) === Set(Seq(1L, 2L, 3L, 5L), Seq(1L, 4L, 5L), Seq(1L, 6L, 7L))) + } + { + // Randomize routes. + val Success(routes) = findMultiPartRoute(g, a, f, amount, maxFee, routeParams = DEFAULT_ROUTE_PARAMS.copy(randomize = true), currentBlockHeight = 400000) + checkRouteAmounts(routes, amount, maxFee) + } + { + // Update A - B with unknown balance, capacity should be used instead. + val g1 = g.addEdge(edge_ab.copy(capacity = 500 sat, balance_opt = None)) + val Success(routes) = findMultiPartRoute(g1, a, f, amount, maxFee, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) + checkRouteAmounts(routes, amount, maxFee) + assert(routes2Ids(routes) === Set(Seq(1L, 2L, 3L, 5L), Seq(1L, 4L, 5L), Seq(1L, 6L, 7L))) + } + { + // Update balance A - B to be too low to cover fees. + val g1 = g.addEdge(edge_ab.copy(balance_opt = Some(400000 msat))) + val failure = findMultiPartRoute(g1, a, f, amount, maxFee, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) + assert(failure === Failure(RouteNotFound)) + } + { + // Update capacity A - B to be too low to cover fees. + val g1 = g.addEdge(edge_ab.copy(capacity = 400 sat, balance_opt = None)) + val failure = findMultiPartRoute(g1, a, f, amount, maxFee, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) + assert(failure === Failure(RouteNotFound)) + } + { + // Try to find a route with a maxFee that's too low. + val maxFeeTooLow = 100 msat + val failure = findMultiPartRoute(g, a, f, amount, maxFeeTooLow, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) + assert(failure === Failure(RouteNotFound)) + } + } + + test("calculate multipart route to remote node (ignored channels and nodes)") { + // +----- B --xxx-- C -----+ + // | +-------- D --------+ | + // | | | | + // +---+ (empty) +---+ + // | A | --------------- | F | + // +---+ +---+ + // | | (not empty) | | + // | +-------------------+ | + // +---------- E ----------+ + val (amount, maxFee) = (25000 msat, 5 msat) + val g = DirectedGraph(List( + makeEdge(1L, a, b, 1 msat, 0, minHtlc = 1 msat, balance_opt = Some(75000 msat)), + makeEdge(2L, b, c, 1 msat, 0, minHtlc = 1 msat, capacity = 150 sat), + makeEdge(3L, c, f, 1 msat, 0, minHtlc = 1 msat, capacity = 150 sat), + makeEdge(4L, a, d, 1 msat, 0, minHtlc = 1 msat, balance_opt = Some(85000 msat)), + makeEdge(5L, d, f, 1 msat, 0, minHtlc = 1 msat, capacity = 300 sat), + makeEdge(6L, a, f, 1 msat, 0, minHtlc = 1 msat, balance_opt = Some(0 msat)), + makeEdge(7L, a, f, 1 msat, 0, minHtlc = 1 msat, balance_opt = Some(10000 msat)), + makeEdge(8L, a, e, 1 msat, 0, minHtlc = 1 msat, balance_opt = Some(18000 msat)), + makeEdge(9L, e, f, 1 msat, 0, minHtlc = 1 msat, capacity = 15 sat), + )) + + val ignoredNodes = Set(d) + val ignoredChannels = Set(ChannelDesc(ShortChannelId(2L), b, c)) + val Success(routes) = findMultiPartRoute(g, a, f, amount, maxFee, ignoredEdges = ignoredChannels, ignoredVertices = ignoredNodes, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) + checkRouteAmounts(routes, amount, maxFee) + assert(routes2Ids(routes) === Set(Seq(7L), Seq(8L, 9L))) + } + + test("calculate multipart route to remote node (restricted htlc_minimum_msat and htlc_maximum_msat)") { + // +----- B -----+ + // | | + // A----- C ---- E + // | | + // +----- D -----+ + val (amount, maxFee) = (15000 msat, 5 msat) + val g = DirectedGraph(List( + // The A -> B -> E path is impossible because the A -> B balance is lower than the B -> E htlc_minimum_msat. + makeEdge(1L, a, b, 1 msat, 0, minHtlc = 500 msat, balance_opt = Some(7000 msat)), + makeEdge(2L, b, e, 1 msat, 0, minHtlc = 10000 msat, capacity = 50 sat), + makeEdge(3L, a, c, 1 msat, 0, minHtlc = 500 msat, balance_opt = Some(10000 msat)), + makeEdge(4L, c, e, 1 msat, 0, minHtlc = 500 msat, maxHtlc = Some(4000 msat), capacity = 50 sat), + makeEdge(5L, a, d, 1 msat, 0, minHtlc = 500 msat, balance_opt = Some(10000 msat)), + makeEdge(6L, d, e, 1 msat, 0, minHtlc = 500 msat, maxHtlc = Some(4000 msat), capacity = 50 sat), + )) + + val Success(routes) = findMultiPartRoute(g, a, e, amount, maxFee, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) + checkRouteAmounts(routes, amount, maxFee) + assert(routes.length >= 4, routes) + assert(routes.forall(_.amount <= 4000.msat), routes) + assert(routes.forall(_.amount >= 500.msat), routes) + checkIgnoredChannels(routes, 1L, 2L) + + val maxFeeTooLow = 3 msat + val failure = findMultiPartRoute(g, a, e, amount, maxFeeTooLow, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) + assert(failure === Failure(RouteNotFound)) + } + + test("calculate multipart route to remote node (complex graph)") { + // +---+ +---+ +---+ + // | A |-----+ +--->| B |--->| C | + // +---+ | | +---+ +---+ + // ^ | +---+ | | + // | +--->| E |---+ | + // | | +---+ | | + // +---+ | | +---+ | + // | D |-----+ +--->| F |<-----+ + // +---+ +---+ + val g = DirectedGraph(Seq( + makeEdge(1L, d, a, 100 msat, 1000, minHtlc = 1000 msat, balance_opt = Some(80000 msat)), + makeEdge(2L, d, e, 100 msat, 1000, minHtlc = 1500 msat, balance_opt = Some(20000 msat)), + makeEdge(3L, a, e, 5 msat, 50, minHtlc = 1200 msat, capacity = 100 sat), + makeEdge(4L, e, f, 25 msat, 1000, minHtlc = 1300 msat, capacity = 25 sat), + makeEdge(5L, e, b, 10 msat, 100, minHtlc = 1100 msat, capacity = 75 sat), + makeEdge(6L, b, c, 5 msat, 50, minHtlc = 1000 msat, capacity = 20 sat), + makeEdge(7L, c, f, 5 msat, 10, minHtlc = 1500 msat, capacity = 50 sat) + )) + val routeParams = DEFAULT_ROUTE_PARAMS.copy(mpp = MultiPartParams(1500 msat, 10)) + + { + val (amount, maxFee) = (15000 msat, 50 msat) + val Success(routes) = findMultiPartRoute(g, d, f, amount, maxFee, routeParams = routeParams, currentBlockHeight = 400000) + checkRouteAmounts(routes, amount, maxFee) + } + { + val (amount, maxFee) = (25000 msat, 100 msat) + val Success(routes) = findMultiPartRoute(g, d, f, amount, maxFee, routeParams = routeParams, currentBlockHeight = 400000) + checkRouteAmounts(routes, amount, maxFee) + } + { + val (amount, maxFee) = (25000 msat, 50 msat) + val failure = findMultiPartRoute(g, d, f, amount, maxFee, routeParams = routeParams, currentBlockHeight = 400000) + assert(failure === Failure(RouteNotFound)) + } + { + val (amount, maxFee) = (40000 msat, 100 msat) + val Success(routes) = findMultiPartRoute(g, d, f, amount, maxFee, routeParams = routeParams, currentBlockHeight = 400000) + checkRouteAmounts(routes, amount, maxFee) + } + { + val (amount, maxFee) = (40000 msat, 100 msat) + val Success(routes) = findMultiPartRoute(g, d, f, amount, maxFee, routeParams = routeParams.copy(randomize = true), currentBlockHeight = 400000) + checkRouteAmounts(routes, amount, maxFee) + } + { + val (amount, maxFee) = (40000 msat, 50 msat) + val failure = findMultiPartRoute(g, d, f, amount, maxFee, routeParams = routeParams, currentBlockHeight = 400000) + assert(failure === Failure(RouteNotFound)) + } + } + + test("calculate multipart route to remote node (with extra edges)") { + // +--- B ---+ + // A D (---) E (---) F + // +--- C ---+ + val (amount, maxFeeE, maxFeeF) = (10000 msat, 50 msat, 100 msat) + val g = DirectedGraph(List( + makeEdge(1L, a, b, 1 msat, 0, minHtlc = 1 msat, maxHtlc = Some(4000 msat), balance_opt = Some(7000 msat)), + makeEdge(2L, b, d, 1 msat, 0, minHtlc = 1 msat, capacity = 50 sat), + makeEdge(3L, a, c, 1 msat, 0, minHtlc = 1 msat, maxHtlc = Some(4000 msat), balance_opt = Some(6000 msat)), + makeEdge(4L, c, d, 1 msat, 0, minHtlc = 1 msat, capacity = 40 sat), + )) + val extraEdges = Set( + makeEdge(10L, d, e, 10 msat, 100, minHtlc = 500 msat, capacity = 15 sat), + makeEdge(11L, e, f, 5 msat, 100, minHtlc = 500 msat, capacity = 10 sat), + ) + + val Success(routes1) = findMultiPartRoute(g, a, e, amount, maxFeeE, extraEdges = extraEdges, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) + checkRouteAmounts(routes1, amount, maxFeeE) + assert(routes1.length >= 3, routes1) + assert(routes1.forall(_.amount <= 4000.msat), routes1) + + val Success(routes2) = findMultiPartRoute(g, a, f, amount, maxFeeF, extraEdges = extraEdges, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) + checkRouteAmounts(routes2, amount, maxFeeF) + assert(routes2.length >= 3, routes2) + assert(routes2.forall(_.amount <= 4000.msat), routes2) + + val maxFeeTooLow = 40 msat + val failure = findMultiPartRoute(g, a, f, amount, maxFeeTooLow, extraEdges = extraEdges, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) + assert(failure === Failure(RouteNotFound)) + } + + test("calculate multipart route to remote node (pending htlcs)") { + // +----- B -----+ + // | | + // A----- C ---- E + // | | + // +----- D -----+ + val (amount, maxFee) = (15000 msat, 100 msat) + val edge_ab = makeEdge(1L, a, b, 1 msat, 0, minHtlc = 100 msat, balance_opt = Some(5000 msat)) + val edge_be = makeEdge(2L, b, e, 1 msat, 0, minHtlc = 100 msat, capacity = 5 sat) + val g = DirectedGraph(List( + // The A -> B -> E route is the most economic one, but we already have a pending HTLC in it. + edge_ab, + edge_be, + makeEdge(3L, a, c, 50 msat, 0, minHtlc = 100 msat, balance_opt = Some(10000 msat)), + makeEdge(4L, c, e, 50 msat, 0, minHtlc = 100 msat, capacity = 25 sat), + makeEdge(5L, a, d, 50 msat, 0, minHtlc = 100 msat, balance_opt = Some(10000 msat)), + makeEdge(6L, d, e, 50 msat, 0, minHtlc = 100 msat, capacity = 25 sat), + )) + + val pendingHtlcs = Seq(Route(5000 msat, ChannelHop(a, b, edge_ab.update) :: ChannelHop(b, e, edge_be.update) :: Nil)) + val Success(routes) = findMultiPartRoute(g, a, e, amount, maxFee, pendingHtlcs = pendingHtlcs, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) + assert(routes.forall(_.length == 2), routes) + checkRouteAmounts(routes, amount, maxFee) + checkIgnoredChannels(routes, 1L, 2L) + } + + test("calculate multipart route for full amount or fail", Tag("fuzzy")) { + // +------------------------------------+ + // | | + // | v + // +---+ +---+ +---+ + // | A |-----+ +--------->| B |--->| C | + // +---+ | | +---+ +---+ + // ^ | +---+ | + // | +--->| E |----------+ | + // | +---+ | | + // | ^ v | + // +---+ | +---+ | + // | D |------------+ | F |<-----+ + // +---+ +---+ + // | ^ + // | | + // +---------------------------+ + for (_ <- 1 to 100) { + val amount = (100 + Random.nextLong(200000)).msat + val maxFee = 50.msat.max(amount * 0.03) + val g = DirectedGraph(List( + makeEdge(1L, d, f, Random.nextLong(250).msat, Random.nextInt(10000), minHtlc = Random.nextLong(100).msat, maxHtlc = Some((20000 + Random.nextLong(80000)).msat), CltvExpiryDelta(Random.nextInt(288)), capacity = (10 + Random.nextLong(100)).sat, balance_opt = Some(Random.nextLong(2 * amount.toLong).msat)), + makeEdge(2L, d, a, Random.nextLong(250).msat, Random.nextInt(10000), minHtlc = Random.nextLong(100).msat, maxHtlc = Some((20000 + Random.nextLong(80000)).msat), CltvExpiryDelta(Random.nextInt(288)), capacity = (10 + Random.nextLong(100)).sat, balance_opt = Some(Random.nextLong(2 * amount.toLong).msat)), + makeEdge(3L, d, e, Random.nextLong(250).msat, Random.nextInt(10000), minHtlc = Random.nextLong(100).msat, maxHtlc = Some((20000 + Random.nextLong(80000)).msat), CltvExpiryDelta(Random.nextInt(288)), capacity = (10 + Random.nextLong(100)).sat, balance_opt = Some(Random.nextLong(2 * amount.toLong).msat)), + makeEdge(4L, a, c, Random.nextLong(250).msat, Random.nextInt(10000), minHtlc = Random.nextLong(100).msat, maxHtlc = Some((20000 + Random.nextLong(80000)).msat), CltvExpiryDelta(Random.nextInt(288)), capacity = (10 + Random.nextLong(100)).sat), + makeEdge(5L, a, e, Random.nextLong(250).msat, Random.nextInt(10000), minHtlc = Random.nextLong(100).msat, maxHtlc = Some((20000 + Random.nextLong(80000)).msat), CltvExpiryDelta(Random.nextInt(288)), capacity = (10 + Random.nextLong(100)).sat), + makeEdge(6L, e, f, Random.nextLong(250).msat, Random.nextInt(10000), minHtlc = Random.nextLong(100).msat, maxHtlc = Some((20000 + Random.nextLong(80000)).msat), CltvExpiryDelta(Random.nextInt(288)), capacity = (10 + Random.nextLong(100)).sat), + makeEdge(7L, e, b, Random.nextLong(250).msat, Random.nextInt(10000), minHtlc = Random.nextLong(100).msat, maxHtlc = Some((20000 + Random.nextLong(80000)).msat), CltvExpiryDelta(Random.nextInt(288)), capacity = (10 + Random.nextLong(100)).sat), + makeEdge(8L, b, c, Random.nextLong(250).msat, Random.nextInt(10000), minHtlc = Random.nextLong(100).msat, maxHtlc = Some((20000 + Random.nextLong(80000)).msat), CltvExpiryDelta(Random.nextInt(288)), capacity = (10 + Random.nextLong(100)).sat), + makeEdge(9L, c, f, Random.nextLong(250).msat, Random.nextInt(10000), minHtlc = Random.nextLong(100).msat, maxHtlc = Some((20000 + Random.nextLong(80000)).msat), CltvExpiryDelta(Random.nextInt(288)), capacity = (10 + Random.nextLong(100)).sat) + )) + + findMultiPartRoute(g, d, f, amount, maxFee, routeParams = DEFAULT_ROUTE_PARAMS.copy(randomize = true), currentBlockHeight = 400000) match { + case Success(routes) => checkRouteAmounts(routes, amount, maxFee) + case Failure(ex) => assert(ex === RouteNotFound) + } + } + } + } object RouteCalculationSpec { @@ -967,7 +1503,7 @@ object RouteCalculationSpec { val DEFAULT_MAX_FEE = 100000 msat val DEFAULT_CAPACITY = 100000 sat - val DEFAULT_ROUTE_PARAMS = RouteParams(randomize = false, maxFeeBase = 21000 msat, maxFeePct = 0.03, routeMaxCltv = CltvExpiryDelta(2016), routeMaxLength = 6, ratios = None) + val DEFAULT_ROUTE_PARAMS = RouteParams(randomize = false, 21000 msat, 0.03, 6, CltvExpiryDelta(2016), None, MultiPartParams(1000 msat, 10)) val DUMMY_SIG = Transactions.PlaceHolderSig @@ -1008,12 +1544,25 @@ object RouteCalculationSpec { htlcMaximumMsat = maxHtlc ) - def hops2Ids(hops: Seq[ChannelHop]) = hops.map(hop => hop.lastUpdate.shortChannelId.toLong) + def hops2Ids(hops: Seq[ChannelHop]): Seq[Long] = hops.map(hop => hop.lastUpdate.shortChannelId.toLong) + + def route2Ids(route: Route): Seq[Long] = hops2Ids(route.hops) - def route2Ids(route: Route) = hops2Ids(route.hops) + def routes2Ids(routes: Seq[Route]): Set[Seq[Long]] = routes.map(route2Ids).toSet - def route2Edges(route: Route) = route.hops.map(hop => GraphEdge(ChannelDesc(hop.lastUpdate.shortChannelId, hop.nodeId, hop.nextNodeId), hop.lastUpdate, 0 sat, None)) + def route2Edges(route: Route): Seq[GraphEdge] = route.hops.map(hop => GraphEdge(ChannelDesc(hop.lastUpdate.shortChannelId, hop.nodeId, hop.nextNodeId), hop.lastUpdate, 0 sat, None)) - def route2Nodes(route: Route) = route.hops.map(hop => (hop.nodeId, hop.nextNodeId)) + def route2Nodes(route: Route): Seq[(PublicKey, PublicKey)] = route.hops.map(hop => (hop.nodeId, hop.nextNodeId)) + + def checkIgnoredChannels(routes: Seq[Route], shortChannelIds: Long*): Unit = { + shortChannelIds.foreach(shortChannelId => routes.foreach(route => { + assert(route.hops.forall(_.lastUpdate.shortChannelId.toLong != shortChannelId), route) + })) + } + + def checkRouteAmounts(routes: Seq[Route], totalAmount: MilliSatoshi, maxFee: MilliSatoshi): Unit = { + assert(routes.map(_.amount).sum == totalAmount, routes) + assert(routes.map(_.fee).sum <= maxFee, routes) + } } From 85ad011c2635fee75c71e400d19f6deee842706e Mon Sep 17 00:00:00 2001 From: t-bast Date: Tue, 26 May 2020 17:55:25 +0200 Subject: [PATCH 2/4] fixup! Implement MPP split inside the Router --- .../eclair/router/RouteCalculation.scala | 3 +- .../eclair/router/RouteCalculationSpec.scala | 28 +++++++++++++++++++ 2 files changed, 30 insertions(+), 1 deletion(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/router/RouteCalculation.scala b/eclair-core/src/main/scala/fr/acinq/eclair/router/RouteCalculation.scala index 33bcaef0c9..7c9d0a079d 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/router/RouteCalculation.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/router/RouteCalculation.scala @@ -277,7 +277,8 @@ object RouteCalculation { val directChannelsCount = g.getEdgesBetween(localNodeId, targetNodeId).length routeParams.mpp.maxParts.max(directChannelsCount) // if we have direct channels to the target, we can use them all } - findRouteInternal(g, localNodeId, targetNodeId, routeParams.mpp.minPartAmount, maxFee, numRoutes, extraEdges, ignoredEdges, ignoredVertices, routeParams, currentBlockHeight) match { + val routeAmount = routeParams.mpp.minPartAmount.min(amount) + findRouteInternal(g, localNodeId, targetNodeId, routeAmount, maxFee, numRoutes, extraEdges, ignoredEdges, ignoredVertices, routeParams, currentBlockHeight) match { case Right(routes) => // We use these shortest paths to find a set of non-conflicting HTLCs that send the total amount. split(amount, mutable.Queue(routes: _*), initializeUsedCapacity(pendingHtlcs), routeParams) match { diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/router/RouteCalculationSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/router/RouteCalculationSpec.scala index 22168e78b2..c475fdefaa 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/router/RouteCalculationSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/router/RouteCalculationSpec.scala @@ -1230,6 +1230,34 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution { } } + test("calculate multipart route to remote node (tiny amount)") { + // A ----- C ----- E + // | | + // +--- B --- D ---+ + // Our balance and the amount we want to send are below the minimum part amount. + val routeParams = DEFAULT_ROUTE_PARAMS.copy(mpp = MultiPartParams(5000 msat, 5)) + val g = DirectedGraph(List( + makeEdge(1L, a, b, 50 msat, 100, minHtlc = 1 msat, balance_opt = Some(1500 msat)), + makeEdge(2L, b, d, 15 msat, 0, minHtlc = 1 msat, capacity = 25 sat), + makeEdge(3L, d, e, 15 msat, 0, minHtlc = 1 msat, capacity = 20 sat), + makeEdge(4L, a, c, 1 msat, 50, minHtlc = 1 msat, balance_opt = Some(1000 msat)), + makeEdge(5L, c, e, 50 msat, 30, minHtlc = 1 msat, capacity = 20 sat), + )) + + { + // We can send single-part tiny payments. + val (amount, maxFee) = (1400 msat, 30 msat) + val Success(routes) = findMultiPartRoute(g, a, e, amount, maxFee, routeParams = routeParams, currentBlockHeight = 400000) + checkRouteAmounts(routes, amount, maxFee) + } + { + // But we don't want to split such tiny amounts. + val (amount, maxFee) = (2000 msat, 150 msat) + val failure = findMultiPartRoute(g, a, e, amount, maxFee, routeParams = routeParams, currentBlockHeight = 400000) + assert(failure === Failure(RouteNotFound)) + } + } + test("calculate multipart route to remote node (single local channel)") { // +--- C ---+ // | | From 181e1c2e929780819109b54c19edd1860c763943 Mon Sep 17 00:00:00 2001 From: t-bast Date: Fri, 19 Jun 2020 11:43:53 +0200 Subject: [PATCH 3/4] Add a test for single path to remote destination --- .../acinq/eclair/router/RouteCalculationSpec.scala | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/router/RouteCalculationSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/router/RouteCalculationSpec.scala index c475fdefaa..8a6c96759f 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/router/RouteCalculationSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/router/RouteCalculationSpec.scala @@ -1258,6 +1258,20 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution { } } + test("calculate multipart route to remote node (single path)") { + val (amount, maxFee) = (100000 msat, 500 msat) + val g = DirectedGraph(List( + makeEdge(1L, a, b, 50 msat, 100, minHtlc = 1 msat, balance_opt = Some(500000 msat)), + makeEdge(2L, b, c, 10 msat, 30, minHtlc = 1 msat, capacity = 150 sat), + makeEdge(3L, c, d, 15 msat, 50, minHtlc = 1 msat, capacity = 150 sat), + )) + + val Success(routes) = findMultiPartRoute(g, a, d, amount, maxFee, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) + checkRouteAmounts(routes, amount, maxFee) + assert(routes.length === 1, "payment shouldn't be split when we have one path with enough capacity") + assert(routes2Ids(routes) === Set(Seq(1L, 2L, 3L))) + } + test("calculate multipart route to remote node (single local channel)") { // +--- C ---+ // | | From a78baed7df824a68f477d73c8c9c38743f87cce3 Mon Sep 17 00:00:00 2001 From: t-bast Date: Fri, 19 Jun 2020 11:44:46 +0200 Subject: [PATCH 4/4] Make split tailrec --- .../scala/fr/acinq/eclair/router/RouteCalculation.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/router/RouteCalculation.scala b/eclair-core/src/main/scala/fr/acinq/eclair/router/RouteCalculation.scala index 7c9d0a079d..2728e7f442 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/router/RouteCalculation.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/router/RouteCalculation.scala @@ -289,9 +289,10 @@ object RouteCalculation { } } - private def split(amount: MilliSatoshi, paths: mutable.Queue[Graph.WeightedPath], usedCapacity: mutable.Map[ShortChannelId, MilliSatoshi], routeParams: RouteParams): Either[RouterException, Seq[Route]] = { + @tailrec + private def split(amount: MilliSatoshi, paths: mutable.Queue[Graph.WeightedPath], usedCapacity: mutable.Map[ShortChannelId, MilliSatoshi], routeParams: RouteParams, selectedRoutes: Seq[Route] = Nil): Either[RouterException, Seq[Route]] = { if (amount == 0.msat) { - Right(Nil) + Right(selectedRoutes) } else if (paths.isEmpty) { Left(RouteNotFound) } else { @@ -299,7 +300,7 @@ object RouteCalculation { val candidate = computeRouteMaxAmount(current.path, usedCapacity) if (candidate.amount < routeParams.mpp.minPartAmount.min(amount)) { // this route doesn't have enough capacity left: we remove it and continue. - split(amount, paths, usedCapacity, routeParams) + split(amount, paths, usedCapacity, routeParams, selectedRoutes) } else { val route = if (routeParams.randomize) { // randomly choose the amount to be between 20% and 100% of the available capacity. @@ -314,7 +315,7 @@ object RouteCalculation { } updateUsedCapacity(route, usedCapacity) // NB: we re-enqueue the current path, it may still have capacity for a second HTLC. - split(amount - route.amount, paths.enqueue(current), usedCapacity, routeParams).map(routes => route +: routes) + split(amount - route.amount, paths.enqueue(current), usedCapacity, routeParams, route +: selectedRoutes) } } }