mirror of
https://github.com/ACINQ/eclair.git
synced 2025-02-23 06:35:11 +01:00
Find multi part route (#1427)
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.
This commit is contained in:
parent
676a45c19c
commit
c52508d216
11 changed files with 792 additions and 34 deletions
|
@ -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
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -293,7 +293,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"),
|
||||
|
|
|
@ -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]]) {
|
||||
|
|
|
@ -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,138 @@ 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
|
||||
}
|
||||
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 {
|
||||
case Right(routes) if validateMultiPartRoute(amount, maxFee, routes) => Right(routes)
|
||||
case _ => Left(RouteNotFound)
|
||||
}
|
||||
case Left(ex) => Left(ex)
|
||||
}
|
||||
}
|
||||
|
||||
@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(selectedRoutes)
|
||||
} 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, selectedRoutes)
|
||||
} 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, route +: selectedRoutes)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** 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
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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,
|
||||
|
@ -217,7 +219,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,
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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))
|
||||
|
|
|
@ -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[_]])
|
||||
|
|
|
@ -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,583 @@ 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 (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 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 ---+
|
||||
// | |
|
||||
// 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 +1545,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 +1586,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) = hops2Ids(route.hops)
|
||||
def route2Ids(route: Route): Seq[Long] = hops2Ids(route.hops)
|
||||
|
||||
def route2Edges(route: Route) = route.hops.map(hop => GraphEdge(ChannelDesc(hop.lastUpdate.shortChannelId, hop.nodeId, hop.nextNodeId), hop.lastUpdate, 0 sat, None))
|
||||
def routes2Ids(routes: Seq[Route]): Set[Seq[Long]] = routes.map(route2Ids).toSet
|
||||
|
||||
def route2Nodes(route: Route) = route.hops.map(hop => (hop.nodeId, hop.nextNodeId))
|
||||
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): 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)
|
||||
}
|
||||
|
||||
}
|
||||
|
|
Loading…
Add table
Reference in a new issue