mirror of
https://github.com/ACINQ/eclair.git
synced 2025-02-23 06:35:11 +01:00
Compute max fee before route calculation (#1417)
Move the maximum fee computation outside of `findRoute`: this should be done earlier in the payment pipeline if we want to allow accurate fee control for MPP retries. Right now MPP uses approximations when retrying which can lead to payments that exceed the maximum configured fees. This is a first step towards ensuring that this situation cannot happen anymore.
This commit is contained in:
parent
ad44ab3631
commit
ce3629c98b
8 changed files with 220 additions and 190 deletions
|
@ -210,7 +210,8 @@ class EclairImpl(appKit: Kit) extends Eclair {
|
|||
}
|
||||
|
||||
override def findRoute(targetNodeId: PublicKey, amount: MilliSatoshi, assistedRoutes: Seq[Seq[PaymentRequest.ExtraHop]] = Seq.empty)(implicit timeout: Timeout): Future[RouteResponse] = {
|
||||
(appKit.router ? RouteRequest(appKit.nodeParams.nodeId, targetNodeId, amount, assistedRoutes)).mapTo[RouteResponse]
|
||||
val maxFee = RouteCalculation.getDefaultRouteParams(appKit.nodeParams.routerConf).getMaxFee(amount)
|
||||
(appKit.router ? RouteRequest(appKit.nodeParams.nodeId, targetNodeId, amount, maxFee, assistedRoutes)).mapTo[RouteResponse]
|
||||
}
|
||||
|
||||
override def sendToRoute(amount: MilliSatoshi, recipientAmount_opt: Option[MilliSatoshi], externalId_opt: Option[String], parentId_opt: Option[UUID], invoice: PaymentRequest, finalCltvExpiryDelta: CltvExpiryDelta, route: Seq[PublicKey], trampolineSecret_opt: Option[ByteVector32], trampolineFees_opt: Option[MilliSatoshi], trampolineExpiryDelta_opt: Option[CltvExpiryDelta], trampolineNodes_opt: Seq[PublicKey])(implicit timeout: Timeout): Future[SendPaymentToRouteResponse] = {
|
||||
|
|
|
@ -95,7 +95,7 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A
|
|||
// If the sender already provided a route to the target, no need to involve the router.
|
||||
self ! RouteResponse(Seq(Route(c.finalPayload.amount, Nil, allowEmpty = true)))
|
||||
} else {
|
||||
router ! RouteRequest(c.getRouteRequestStart(nodeParams), c.targetNodeId, c.finalPayload.amount, c.assistedRoutes, routeParams = c.routeParams, ignoreNodes = ignoredNodes)
|
||||
router ! RouteRequest(c.getRouteRequestStart(nodeParams), c.targetNodeId, c.finalPayload.amount, c.getMaxFee(nodeParams), c.assistedRoutes, routeParams = c.routeParams, ignoreNodes = ignoredNodes)
|
||||
}
|
||||
if (cfg.storeInDb) {
|
||||
paymentsDb.addOutgoingPayment(OutgoingPayment(id, cfg.parentId, cfg.externalId, paymentHash, PaymentType.Standard, c.finalPayload.amount, cfg.recipientAmount, cfg.recipientNodeId, System.currentTimeMillis, cfg.paymentRequest, OutgoingPaymentStatus.Pending))
|
||||
|
@ -204,12 +204,12 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A
|
|||
case extraHop => extraHop
|
||||
})
|
||||
// let's try again, router will have updated its state
|
||||
router ! RouteRequest(c.getRouteRequestStart(nodeParams), c.targetNodeId, c.finalPayload.amount, assistedRoutes1, ignoreNodes, ignoreChannels, c.routeParams)
|
||||
router ! RouteRequest(c.getRouteRequestStart(nodeParams), c.targetNodeId, c.finalPayload.amount, c.getMaxFee(nodeParams), assistedRoutes1, ignoreNodes, ignoreChannels, c.routeParams)
|
||||
ignoreNodes
|
||||
} else {
|
||||
// this node is fishy, it gave us a bad sig!! let's filter it out
|
||||
log.warning(s"got bad signature from node=$nodeId update=${failureMessage.update}")
|
||||
router ! RouteRequest(c.getRouteRequestStart(nodeParams), c.targetNodeId, c.finalPayload.amount, c.assistedRoutes, ignoreNodes + nodeId, ignoreChannels, c.routeParams)
|
||||
router ! RouteRequest(c.getRouteRequestStart(nodeParams), c.targetNodeId, c.finalPayload.amount, c.getMaxFee(nodeParams), c.assistedRoutes, ignoreNodes + nodeId, ignoreChannels, c.routeParams)
|
||||
ignoreNodes + nodeId
|
||||
}
|
||||
goto(WAITING_FOR_ROUTE) using WaitingForRoute(s, c, failures :+ RemoteFailure(cfg.fullRoute(route), e), ignoreNodes1, ignoreChannels)
|
||||
|
@ -267,7 +267,7 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A
|
|||
|
||||
private def retry(failure: PaymentFailure, data: WaitingForComplete): FSM.State[PaymentLifecycle.State, PaymentLifecycle.Data] = {
|
||||
val (ignoreNodes1, ignoreChannels1) = PaymentFailure.updateIgnored(failure, data.ignoreNodes, data.ignoreChannels)
|
||||
router ! RouteRequest(data.c.getRouteRequestStart(nodeParams), data.c.targetNodeId, data.c.finalPayload.amount, data.c.assistedRoutes, ignoreNodes1, ignoreChannels1, data.c.routeParams)
|
||||
router ! RouteRequest(data.c.getRouteRequestStart(nodeParams), data.c.targetNodeId, data.c.finalPayload.amount, data.c.getMaxFee(nodeParams), data.c.assistedRoutes, ignoreNodes1, ignoreChannels1, data.c.routeParams)
|
||||
goto(WAITING_FOR_ROUTE) using WaitingForRoute(data.sender, data.c, data.failures :+ failure, ignoreNodes1, ignoreChannels1)
|
||||
}
|
||||
|
||||
|
@ -340,6 +340,9 @@ object PaymentLifecycle {
|
|||
routePrefix: Seq[ChannelHop] = Nil) {
|
||||
require(finalPayload.amount > 0.msat, s"amount must be > 0")
|
||||
|
||||
def getMaxFee(nodeParams: NodeParams): MilliSatoshi =
|
||||
routeParams.getOrElse(RouteCalculation.getDefaultRouteParams(nodeParams.routerConf)).getMaxFee(finalPayload.amount)
|
||||
|
||||
/** Returns the node from which the path-finding algorithm should start. */
|
||||
def getRouteRequestStart(nodeParams: NodeParams): PublicKey = routePrefix match {
|
||||
case Nil => nodeParams.nodeId
|
||||
|
|
|
@ -26,6 +26,7 @@ object Monitoring {
|
|||
|
||||
object Metrics {
|
||||
val FindRouteDuration = Kamon.timer("router.find-route.duration", "Path-finding duration")
|
||||
val FindRouteErrors = Kamon.counter("router.find-route.errors", "Path-finding errors")
|
||||
val RouteLength = Kamon.histogram("router.find-route.length", "Path-finding result length")
|
||||
|
||||
object QueryChannelRange {
|
||||
|
@ -69,6 +70,7 @@ object Monitoring {
|
|||
val Amount = "amount"
|
||||
val Announced = "announced"
|
||||
val Direction = "direction"
|
||||
val Error = "error"
|
||||
val NumberOfRoutes = "numRoutes"
|
||||
|
||||
object Directions {
|
||||
|
|
|
@ -29,8 +29,9 @@ import fr.acinq.eclair.router.Router._
|
|||
import fr.acinq.eclair.wire.ChannelUpdate
|
||||
import fr.acinq.eclair.{ShortChannelId, _}
|
||||
|
||||
import scala.annotation.tailrec
|
||||
import scala.concurrent.duration._
|
||||
import scala.util.{Random, Try}
|
||||
import scala.util.{Failure, Random, Success, Try}
|
||||
|
||||
object RouteCalculation {
|
||||
|
||||
|
@ -71,9 +72,16 @@ object RouteCalculation {
|
|||
|
||||
log.info(s"finding a route ${r.source}->${r.target} with assistedChannels={} ignoreNodes={} ignoreChannels={} excludedChannels={}", assistedChannels.keys.mkString(","), r.ignoreNodes.map(_.value).mkString(","), r.ignoreChannels.mkString(","), d.excludedChannels.mkString(","))
|
||||
log.info(s"finding a route with randomize={} params={}", routesToFind > 1, params)
|
||||
findRoute(d.graph, r.source, r.target, r.amount, numRoutes = routesToFind, extraEdges = extraEdges, ignoredEdges = ignoredEdges, ignoredVertices = r.ignoreNodes, routeParams = params, currentBlockHeight)
|
||||
.map(route => ctx.sender ! RouteResponse(route :: Nil))
|
||||
.recover { case t => ctx.sender ! Status.Failure(t) }
|
||||
KamonExt.time(Metrics.FindRouteDuration.withTag(Tags.NumberOfRoutes, routesToFind).withTag(Tags.Amount, Tags.amountBucket(r.amount))) {
|
||||
findRoute(d.graph, r.source, r.target, r.amount, r.maxFee, routesToFind, extraEdges, ignoredEdges, r.ignoreNodes, params, currentBlockHeight) match {
|
||||
case Success(routes) =>
|
||||
Metrics.RouteLength.withTag(Tags.Amount, Tags.amountBucket(r.amount)).record(routes.head.length)
|
||||
ctx.sender ! RouteResponse(routes)
|
||||
case Failure(t) =>
|
||||
Metrics.FindRouteErrors.withTag(Tags.Amount, Tags.amountBucket(r.amount)).withTag(Tags.Error, t.getClass.getSimpleName).increment()
|
||||
ctx.sender ! Status.Failure(t)
|
||||
}
|
||||
}
|
||||
d
|
||||
}
|
||||
|
||||
|
@ -146,63 +154,68 @@ object RouteCalculation {
|
|||
* @param g graph of the whole network
|
||||
* @param localNodeId sender node (payer)
|
||||
* @param targetNodeId target node (final recipient)
|
||||
* @param amount the amount that will be sent along this route
|
||||
* @param numRoutes the number of shortest-paths to find
|
||||
* @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
|
||||
* @return the computed route to the destination @targetNodeId
|
||||
* @return the computed routes to the destination @param targetNodeId
|
||||
*/
|
||||
def findRoute(g: DirectedGraph,
|
||||
localNodeId: PublicKey,
|
||||
targetNodeId: PublicKey,
|
||||
amount: MilliSatoshi,
|
||||
maxFee: MilliSatoshi,
|
||||
numRoutes: Int,
|
||||
extraEdges: Set[GraphEdge] = Set.empty,
|
||||
ignoredEdges: Set[ChannelDesc] = Set.empty,
|
||||
ignoredVertices: Set[PublicKey] = Set.empty,
|
||||
routeParams: RouteParams,
|
||||
currentBlockHeight: Long): Try[Route] = Try {
|
||||
|
||||
if (localNodeId == targetNodeId) throw CannotRouteToSelf
|
||||
|
||||
def feeBaseOk(fee: MilliSatoshi): Boolean = fee <= routeParams.maxFeeBase
|
||||
|
||||
def feePctOk(fee: MilliSatoshi, amount: MilliSatoshi): Boolean = {
|
||||
val maxFee = amount * routeParams.maxFeePct
|
||||
fee <= maxFee
|
||||
currentBlockHeight: Long): Try[Seq[Route]] = Try {
|
||||
findRouteInternal(g, localNodeId, targetNodeId, amount, maxFee, numRoutes, extraEdges, ignoredEdges, ignoredVertices, routeParams, currentBlockHeight) match {
|
||||
case Right(routes) => routes.map(route => Route(amount, route.path.map(graphEdgeToHop)))
|
||||
case Left(ex) => return Failure(ex)
|
||||
}
|
||||
}
|
||||
|
||||
def feeOk(fee: MilliSatoshi, amount: MilliSatoshi): Boolean = feeBaseOk(fee) || feePctOk(fee, amount)
|
||||
@tailrec
|
||||
private def findRouteInternal(g: DirectedGraph,
|
||||
localNodeId: PublicKey,
|
||||
targetNodeId: PublicKey,
|
||||
amount: MilliSatoshi,
|
||||
maxFee: MilliSatoshi,
|
||||
numRoutes: Int,
|
||||
extraEdges: Set[GraphEdge] = Set.empty,
|
||||
ignoredEdges: Set[ChannelDesc] = Set.empty,
|
||||
ignoredVertices: Set[PublicKey] = Set.empty,
|
||||
routeParams: RouteParams,
|
||||
currentBlockHeight: Long): Either[RouterException, Seq[Graph.WeightedPath]] = {
|
||||
if (localNodeId == targetNodeId) return Left(CannotRouteToSelf)
|
||||
|
||||
def feeOk(fee: MilliSatoshi): Boolean = fee <= maxFee
|
||||
|
||||
def lengthOk(length: Int): Boolean = length <= routeParams.routeMaxLength && length <= ROUTE_MAX_LENGTH
|
||||
|
||||
def cltvOk(cltv: CltvExpiryDelta): Boolean = cltv <= routeParams.routeMaxCltv
|
||||
|
||||
val boundaries: RichWeight => Boolean = { weight =>
|
||||
feeOk(weight.cost - amount, amount) && lengthOk(weight.length) && cltvOk(weight.cltv)
|
||||
}
|
||||
val boundaries: RichWeight => Boolean = { weight => feeOk(weight.cost - amount) && lengthOk(weight.length) && cltvOk(weight.cltv) }
|
||||
|
||||
val foundRoutes = KamonExt.time(Metrics.FindRouteDuration.withTag(Tags.NumberOfRoutes, numRoutes).withTag(Tags.Amount, Tags.amountBucket(amount))) {
|
||||
Graph.yenKshortestPaths(g, localNodeId, targetNodeId, amount, ignoredEdges, ignoredVertices, extraEdges, numRoutes, routeParams.ratios, currentBlockHeight, boundaries).toList
|
||||
}
|
||||
foundRoutes match {
|
||||
case Nil if routeParams.routeMaxLength < ROUTE_MAX_LENGTH => // if not found within the constraints we relax and repeat the search
|
||||
Metrics.RouteLength.withTag(Tags.Amount, Tags.amountBucket(amount)).record(0)
|
||||
return findRoute(g, localNodeId, targetNodeId, amount, numRoutes, extraEdges, ignoredEdges, ignoredVertices, routeParams.copy(routeMaxLength = ROUTE_MAX_LENGTH, routeMaxCltv = DEFAULT_ROUTE_MAX_CLTV), currentBlockHeight)
|
||||
case Nil =>
|
||||
Metrics.RouteLength.withTag(Tags.Amount, Tags.amountBucket(amount)).record(0)
|
||||
throw RouteNotFound
|
||||
case foundRoutes =>
|
||||
val routes = foundRoutes.find(_.path.size == 1) match {
|
||||
case Some(directRoute) => directRoute :: Nil
|
||||
case _ => foundRoutes
|
||||
}
|
||||
// At this point 'routes' cannot be empty
|
||||
val randomizedRoutes = if (routeParams.randomize) Random.shuffle(routes) else routes
|
||||
val route = randomizedRoutes.head.path.map(graphEdgeToHop)
|
||||
Metrics.RouteLength.withTag(Tags.Amount, Tags.amountBucket(amount)).record(route.length)
|
||||
Route(amount, route)
|
||||
val foundRoutes: Seq[Graph.WeightedPath] = Graph.yenKshortestPaths(g, localNodeId, targetNodeId, amount, ignoredEdges, ignoredVertices, extraEdges, numRoutes, routeParams.ratios, currentBlockHeight, boundaries)
|
||||
if (foundRoutes.nonEmpty) {
|
||||
val (directRoutes, indirectRoutes) = foundRoutes.partition(_.path.length == 1)
|
||||
val routes = if (routeParams.randomize) {
|
||||
Random.shuffle(directRoutes) ++ Random.shuffle(indirectRoutes)
|
||||
} else {
|
||||
directRoutes ++ indirectRoutes
|
||||
}
|
||||
Right(routes)
|
||||
} else if (routeParams.routeMaxLength < ROUTE_MAX_LENGTH) {
|
||||
// if not found within the constraints we relax and repeat the search
|
||||
val relaxedRouteParams = routeParams.copy(routeMaxLength = ROUTE_MAX_LENGTH, routeMaxCltv = DEFAULT_ROUTE_MAX_CLTV)
|
||||
findRouteInternal(g, localNodeId, targetNodeId, amount, maxFee, numRoutes, extraEdges, ignoredEdges, ignoredVertices, relaxedRouteParams, currentBlockHeight)
|
||||
} else {
|
||||
Left(RouteNotFound)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -363,11 +363,17 @@ 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 RouteParams(randomize: Boolean, maxFeeBase: MilliSatoshi, maxFeePct: Double, routeMaxLength: Int, routeMaxCltv: CltvExpiryDelta, ratios: Option[WeightRatios]) {
|
||||
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)
|
||||
}
|
||||
}
|
||||
|
||||
case class RouteRequest(source: PublicKey,
|
||||
target: PublicKey,
|
||||
amount: MilliSatoshi,
|
||||
maxFee: MilliSatoshi,
|
||||
assistedRoutes: Seq[Seq[ExtraHop]] = Nil,
|
||||
ignoreNodes: Set[PublicKey] = Set.empty,
|
||||
ignoreChannels: Set[ChannelDesc] = Set.empty,
|
||||
|
|
|
@ -21,6 +21,7 @@ import java.util.UUID
|
|||
import akka.actor.FSM.{CurrentState, SubscribeTransitionCallBack, Transition}
|
||||
import akka.actor.{ActorRef, Status}
|
||||
import akka.testkit.{TestFSMRef, TestProbe}
|
||||
import fr.acinq.bitcoin.Crypto.PublicKey
|
||||
import fr.acinq.bitcoin.Script.{pay2wsh, write}
|
||||
import fr.acinq.bitcoin.{Block, ByteVector32, Crypto, Transaction, TxOut}
|
||||
import fr.acinq.eclair._
|
||||
|
@ -54,6 +55,7 @@ import scala.concurrent.duration._
|
|||
class PaymentLifecycleSpec extends BaseRouterSpec {
|
||||
|
||||
val defaultAmountMsat = 142000000 msat
|
||||
val defaultMaxFee = 4260000 msat // 3% of defaultAmountMsat
|
||||
val defaultExpiry = Channel.MIN_CLTV_EXPIRY_DELTA.toCltvExpiry(40000)
|
||||
val defaultPaymentPreimage = randomBytes32
|
||||
val defaultPaymentHash = Crypto.sha256(defaultPaymentPreimage)
|
||||
|
@ -61,6 +63,8 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
val defaultExternalId = UUID.randomUUID().toString
|
||||
val defaultPaymentRequest = SendPaymentRequest(defaultAmountMsat, defaultPaymentHash, d, 1, externalId = Some(defaultExternalId))
|
||||
|
||||
def defaultRouteRequest(source: PublicKey, target: PublicKey): RouteRequest = RouteRequest(source, target, defaultAmountMsat, defaultMaxFee)
|
||||
|
||||
case class PaymentFixture(id: UUID,
|
||||
parentId: UUID,
|
||||
nodeParams: NodeParams,
|
||||
|
@ -148,7 +152,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
|
||||
val request = SendPayment(d, FinalLegacyPayload(defaultAmountMsat, defaultExpiry), 3, routePrefix = Seq(ChannelHop(a, b, update_ab), ChannelHop(b, c, update_bc)))
|
||||
sender.send(paymentFSM, request)
|
||||
routerForwarder.expectMsg(RouteRequest(c, d, defaultAmountMsat, ignoreNodes = Set(a, b)))
|
||||
routerForwarder.expectMsg(defaultRouteRequest(c, d).copy(ignoreNodes = Set(a, b)))
|
||||
val Transition(_, WAITING_FOR_REQUEST, WAITING_FOR_ROUTE) = monitor.expectMsgClass(classOf[Transition[_]])
|
||||
awaitCond(nodeParams.db.payments.getOutgoingPayment(id).exists(_.status == OutgoingPaymentStatus.Pending))
|
||||
|
||||
|
@ -174,7 +178,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
|
||||
val request = SendPayment(d, FinalLegacyPayload(defaultAmountMsat, defaultExpiry), 3, routePrefix = Seq(ChannelHop(a, b, update_ab), ChannelHop(b, c, update_bc)))
|
||||
sender.send(paymentFSM, request)
|
||||
routerForwarder.expectMsg(RouteRequest(c, d, defaultAmountMsat, ignoreNodes = Set(a, b)))
|
||||
routerForwarder.expectMsg(defaultRouteRequest(c, d).copy(ignoreNodes = Set(a, b)))
|
||||
val Transition(_, WAITING_FOR_REQUEST, WAITING_FOR_ROUTE) = monitor.expectMsgClass(classOf[Transition[_]])
|
||||
awaitCond(nodeParams.db.payments.getOutgoingPayment(id).exists(_.status == OutgoingPaymentStatus.Pending))
|
||||
|
||||
|
@ -182,7 +186,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
val Transition(_, WAITING_FOR_ROUTE, WAITING_FOR_PAYMENT_COMPLETE) = monitor.expectMsgClass(classOf[Transition[_]])
|
||||
|
||||
sender.send(paymentFSM, UpdateFailHtlc(randomBytes32, 0, randomBytes(Sphinx.FailurePacket.PacketLength)))
|
||||
routerForwarder.expectMsg(RouteRequest(c, d, defaultAmountMsat, ignoreNodes = Set(a, b, c)))
|
||||
routerForwarder.expectMsg(defaultRouteRequest(c, d).copy(ignoreNodes = Set(a, b, c)))
|
||||
val Transition(_, WAITING_FOR_PAYMENT_COMPLETE, WAITING_FOR_ROUTE) = monitor.expectMsgClass(classOf[Transition[_]])
|
||||
assert(nodeParams.db.payments.getOutgoingPayment(id).exists(_.status == OutgoingPaymentStatus.Pending))
|
||||
}
|
||||
|
@ -222,7 +226,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
|
||||
val request = SendPayment(d, FinalLegacyPayload(defaultAmountMsat, defaultExpiry), 2)
|
||||
sender.send(paymentFSM, request)
|
||||
routerForwarder.expectMsg(RouteRequest(a, d, defaultAmountMsat, ignoreNodes = Set.empty, ignoreChannels = Set.empty))
|
||||
routerForwarder.expectMsg(defaultRouteRequest(a, d))
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && nodeParams.db.payments.getOutgoingPayment(id).exists(_.status === OutgoingPaymentStatus.Pending))
|
||||
|
||||
val WaitingForRoute(_, _, Nil, _, _) = paymentFSM.stateData
|
||||
|
@ -235,7 +239,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
sender.send(paymentFSM, Relayer.ForwardRemoteFail(UpdateFailHtlc(ByteVector32.Zeroes, 0, randomBytes32), defaultOrigin, UpdateAddHtlc(ByteVector32.Zeroes, 0, defaultAmountMsat, defaultPaymentHash, defaultExpiry, TestConstants.emptyOnionPacket))) // unparsable message
|
||||
|
||||
// then the payment lifecycle will ask for a new route excluding all intermediate nodes
|
||||
routerForwarder.expectMsg(RouteRequest(nodeParams.nodeId, d, defaultAmountMsat, ignoreNodes = Set(c), ignoreChannels = Set.empty))
|
||||
routerForwarder.expectMsg(defaultRouteRequest(nodeParams.nodeId, d).copy(ignoreNodes = Set(c)))
|
||||
|
||||
// let's simulate a response by the router with another route
|
||||
sender.send(paymentFSM, RouteResponse(route :: Nil))
|
||||
|
@ -260,7 +264,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && nodeParams.db.payments.getOutgoingPayment(id).exists(_.status === OutgoingPaymentStatus.Pending))
|
||||
|
||||
val WaitingForRoute(_, _, Nil, _, _) = paymentFSM.stateData
|
||||
routerForwarder.expectMsg(RouteRequest(nodeParams.nodeId, d, defaultAmountMsat, assistedRoutes = Nil, ignoreNodes = Set.empty, ignoreChannels = Set.empty))
|
||||
routerForwarder.expectMsg(defaultRouteRequest(nodeParams.nodeId, d))
|
||||
routerForwarder.forward(routerFixture.router)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_PAYMENT_COMPLETE)
|
||||
val WaitingForComplete(_, _, cmd1, Nil, _, _, _, _) = paymentFSM.stateData
|
||||
|
@ -269,7 +273,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
sender.send(paymentFSM, Status.Failure(AddHtlcFailed(ByteVector32.Zeroes, defaultPaymentHash, ChannelUnavailable(ByteVector32.Zeroes), Local(id, Some(paymentFSM.underlying.self)), None, None)))
|
||||
|
||||
// then the payment lifecycle will ask for a new route excluding the channel
|
||||
routerForwarder.expectMsg(RouteRequest(nodeParams.nodeId, d, defaultAmountMsat, assistedRoutes = Nil, ignoreNodes = Set.empty, ignoreChannels = Set(ChannelDesc(channelId_ab, a, b))))
|
||||
routerForwarder.expectMsg(defaultRouteRequest(nodeParams.nodeId, d).copy(ignoreChannels = Set(ChannelDesc(channelId_ab, a, b))))
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && nodeParams.db.payments.getOutgoingPayment(id).exists(_.status === OutgoingPaymentStatus.Pending)) // payment is still pending because the error is recoverable
|
||||
}
|
||||
|
||||
|
@ -282,7 +286,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && nodeParams.db.payments.getOutgoingPayment(id).exists(_.status === OutgoingPaymentStatus.Pending))
|
||||
|
||||
val WaitingForRoute(_, _, Nil, _, _) = paymentFSM.stateData
|
||||
routerForwarder.expectMsg(RouteRequest(nodeParams.nodeId, d, defaultAmountMsat, assistedRoutes = Nil, ignoreNodes = Set.empty, ignoreChannels = Set.empty))
|
||||
routerForwarder.expectMsg(defaultRouteRequest(nodeParams.nodeId, d))
|
||||
routerForwarder.forward(routerFixture.router)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_PAYMENT_COMPLETE)
|
||||
val WaitingForComplete(_, _, cmd1, Nil, _, _, _, _) = paymentFSM.stateData
|
||||
|
@ -291,7 +295,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
sender.send(paymentFSM, UpdateFailMalformedHtlc(ByteVector32.Zeroes, 0, randomBytes32, FailureMessageCodecs.BADONION))
|
||||
|
||||
// then the payment lifecycle will ask for a new route excluding the channel
|
||||
routerForwarder.expectMsg(RouteRequest(a, d, defaultAmountMsat, assistedRoutes = Nil, ignoreNodes = Set.empty, ignoreChannels = Set(ChannelDesc(channelId_ab, a, b))))
|
||||
routerForwarder.expectMsg(defaultRouteRequest(a, d).copy(ignoreChannels = Set(ChannelDesc(channelId_ab, a, b))))
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && nodeParams.db.payments.getOutgoingPayment(id).exists(_.status === OutgoingPaymentStatus.Pending))
|
||||
}
|
||||
|
||||
|
@ -303,7 +307,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
sender.send(paymentFSM, request)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE)
|
||||
val WaitingForRoute(_, _, Nil, _, _) = paymentFSM.stateData
|
||||
routerForwarder.expectMsg(RouteRequest(nodeParams.nodeId, d, defaultAmountMsat, assistedRoutes = Nil, ignoreNodes = Set.empty, ignoreChannels = Set.empty))
|
||||
routerForwarder.expectMsg(defaultRouteRequest(nodeParams.nodeId, d))
|
||||
routerForwarder.forward(routerFixture.router)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_PAYMENT_COMPLETE)
|
||||
val WaitingForComplete(_, _, cmd1, Nil, sharedSecrets1, _, _, route) = paymentFSM.stateData
|
||||
|
@ -318,7 +322,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
// payment lifecycle forwards the embedded channelUpdate to the router
|
||||
routerForwarder.expectMsg(update_bc)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE)
|
||||
routerForwarder.expectMsg(RouteRequest(a, d, defaultAmountMsat, assistedRoutes = Nil, ignoreNodes = Set.empty, ignoreChannels = Set.empty))
|
||||
routerForwarder.expectMsg(defaultRouteRequest(a, d))
|
||||
routerForwarder.forward(routerFixture.router)
|
||||
// we allow 2 tries, so we send a 2nd request to the router
|
||||
assert(sender.expectMsgType[PaymentFailed].failures === RemoteFailure(route.hops, Sphinx.DecryptedFailurePacket(b, failure)) :: LocalFailure(Nil, RouteNotFound) :: Nil)
|
||||
|
@ -333,7 +337,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && nodeParams.db.payments.getOutgoingPayment(id).exists(_.status === OutgoingPaymentStatus.Pending))
|
||||
|
||||
val WaitingForRoute(_, _, Nil, _, _) = paymentFSM.stateData
|
||||
routerForwarder.expectMsg(RouteRequest(nodeParams.nodeId, d, defaultAmountMsat, assistedRoutes = Nil, ignoreNodes = Set.empty, ignoreChannels = Set.empty))
|
||||
routerForwarder.expectMsg(defaultRouteRequest(nodeParams.nodeId, d))
|
||||
routerForwarder.forward(routerFixture.router)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_PAYMENT_COMPLETE)
|
||||
val WaitingForComplete(_, _, cmd1, Nil, sharedSecrets1, _, _, route1) = paymentFSM.stateData
|
||||
|
@ -348,7 +352,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
// payment lifecycle forwards the embedded channelUpdate to the router
|
||||
routerForwarder.expectMsg(channelUpdate_bc_modified)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && nodeParams.db.payments.getOutgoingPayment(id).exists(_.status === OutgoingPaymentStatus.Pending)) // 1 failure but not final, the payment is still PENDING
|
||||
routerForwarder.expectMsg(RouteRequest(nodeParams.nodeId, d, defaultAmountMsat, assistedRoutes = Nil, ignoreNodes = Set.empty, ignoreChannels = Set.empty))
|
||||
routerForwarder.expectMsg(defaultRouteRequest(nodeParams.nodeId, d))
|
||||
routerForwarder.forward(routerFixture.router)
|
||||
|
||||
// router answers with a new route, taking into account the new update
|
||||
|
@ -368,7 +372,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
// but it will still forward the embedded channelUpdate to the router
|
||||
routerForwarder.expectMsg(channelUpdate_bc_modified_2)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE)
|
||||
routerForwarder.expectMsg(RouteRequest(nodeParams.nodeId, d, defaultAmountMsat, assistedRoutes = Nil, ignoreNodes = Set.empty, ignoreChannels = Set.empty))
|
||||
routerForwarder.expectMsg(defaultRouteRequest(nodeParams.nodeId, d))
|
||||
routerForwarder.forward(routerFixture.router)
|
||||
|
||||
// this time the router can't find a route: game over
|
||||
|
@ -391,7 +395,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && nodeParams.db.payments.getOutgoingPayment(id).exists(_.status === OutgoingPaymentStatus.Pending))
|
||||
|
||||
val WaitingForRoute(_, _, Nil, _, _) = paymentFSM.stateData
|
||||
routerForwarder.expectMsg(RouteRequest(nodeParams.nodeId, d, defaultAmountMsat, assistedRoutes = assistedRoutes, ignoreNodes = Set.empty, ignoreChannels = Set.empty))
|
||||
routerForwarder.expectMsg(defaultRouteRequest(nodeParams.nodeId, d).copy(assistedRoutes = assistedRoutes))
|
||||
routerForwarder.forward(routerFixture.router)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_PAYMENT_COMPLETE)
|
||||
val WaitingForComplete(_, _, cmd1, Nil, sharedSecrets1, _, _, _) = paymentFSM.stateData
|
||||
|
@ -410,7 +414,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
ExtraHop(b, channelId_bc, update_bc.feeBaseMsat, update_bc.feeProportionalMillionths, channelUpdate_bc_modified.cltvExpiryDelta),
|
||||
ExtraHop(c, channelId_cd, update_cd.feeBaseMsat, update_cd.feeProportionalMillionths, update_cd.cltvExpiryDelta)
|
||||
))
|
||||
routerForwarder.expectMsg(RouteRequest(nodeParams.nodeId, d, defaultAmountMsat, assistedRoutes = assistedRoutes1, ignoreNodes = Set.empty, ignoreChannels = Set.empty))
|
||||
routerForwarder.expectMsg(defaultRouteRequest(nodeParams.nodeId, d).copy(assistedRoutes = assistedRoutes1))
|
||||
routerForwarder.forward(routerFixture.router)
|
||||
|
||||
// router answers with a new route, taking into account the new update
|
||||
|
@ -429,7 +433,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && nodeParams.db.payments.getOutgoingPayment(id).exists(_.status === OutgoingPaymentStatus.Pending))
|
||||
|
||||
val WaitingForRoute(_, _, Nil, _, _) = paymentFSM.stateData
|
||||
routerForwarder.expectMsg(RouteRequest(nodeParams.nodeId, d, defaultAmountMsat, assistedRoutes = Nil, ignoreNodes = Set.empty, ignoreChannels = Set.empty))
|
||||
routerForwarder.expectMsg(defaultRouteRequest(nodeParams.nodeId, d))
|
||||
routerForwarder.forward(router)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_PAYMENT_COMPLETE)
|
||||
val WaitingForComplete(_, _, cmd1, Nil, sharedSecrets1, _, _, route1) = paymentFSM.stateData
|
||||
|
@ -439,7 +443,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
|
||||
// payment lifecycle forwards the embedded channelUpdate to the router
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE)
|
||||
routerForwarder.expectMsg(RouteRequest(nodeParams.nodeId, d, defaultAmountMsat, assistedRoutes = Nil, ignoreNodes = Set.empty, ignoreChannels = Set(ChannelDesc(channelId_bc, b, c))))
|
||||
routerForwarder.expectMsg(defaultRouteRequest(nodeParams.nodeId, d).copy(ignoreChannels = Set(ChannelDesc(channelId_bc, b, c))))
|
||||
routerForwarder.forward(router)
|
||||
// we allow 2 tries, so we send a 2nd request to the router, which won't find another route
|
||||
|
||||
|
|
|
@ -27,8 +27,8 @@ 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.funsuite.AnyFunSuite
|
||||
import org.scalatest.ParallelTestExecution
|
||||
import org.scalatest.funsuite.AnyFunSuite
|
||||
import scodec.bits._
|
||||
|
||||
import scala.collection.immutable.SortedMap
|
||||
|
@ -52,18 +52,19 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
makeEdge(4L, d, e, 1 msat, 10, cltvDelta = CltvExpiryDelta(1))
|
||||
))
|
||||
|
||||
val route = findRoute(g, a, e, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route.map(route2Ids) === Success(1 :: 2 :: 3 :: 4 :: Nil))
|
||||
val Success(route :: Nil) = findRoute(g, a, e, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route2Ids(route) === 1 :: 2 :: 3 :: 4 :: Nil)
|
||||
}
|
||||
|
||||
test("check fee against max pct properly") {
|
||||
// fee is acceptable if it is either
|
||||
// - below our maximum fee base
|
||||
// - below our maximum fraction of the paid amount
|
||||
|
||||
// fee is acceptable if it is either:
|
||||
// - below our maximum fee base
|
||||
// - below our maximum fraction of the paid amount
|
||||
// here we have a maximum fee base of 1 msat, and all our updates have a base fee of 10 msat
|
||||
// so our fee will always be above the base fee, and we will always check that it is below our maximum percentage
|
||||
// of the amount being paid
|
||||
val routeParams = DEFAULT_ROUTE_PARAMS.copy(maxFeeBase = 1 msat)
|
||||
val maxFee = routeParams.getMaxFee(DEFAULT_AMOUNT_MSAT)
|
||||
|
||||
val g = DirectedGraph(List(
|
||||
makeEdge(1L, a, b, 10 msat, 10, cltvDelta = CltvExpiryDelta(1)),
|
||||
|
@ -72,9 +73,8 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
makeEdge(4L, d, e, 10 msat, 10, cltvDelta = CltvExpiryDelta(1))
|
||||
))
|
||||
|
||||
val route = findRoute(g, a, e, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS.copy(maxFeeBase = 1 msat), currentBlockHeight = 400000)
|
||||
|
||||
assert(route.map(route2Ids) === Success(1 :: 2 :: 3 :: 4 :: Nil))
|
||||
val Success(route :: Nil) = findRoute(g, a, e, DEFAULT_AMOUNT_MSAT, maxFee, numRoutes = 1, routeParams = routeParams, currentBlockHeight = 400000)
|
||||
assert(route2Ids(route) === 1 :: 2 :: 3 :: 4 :: Nil)
|
||||
}
|
||||
|
||||
test("calculate the shortest path (correct fees)") {
|
||||
|
@ -116,7 +116,7 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
makeEdge(6L, f, d, feeBase = 1 msat, feeProportionalMillionth = 100, minHtlc = 0 msat)
|
||||
))
|
||||
|
||||
val Success(route) = findRoute(graph, a, d, amount, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
val Success(route :: Nil) = findRoute(graph, a, d, amount, maxFee = 7 msat, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
val weightedPath = Graph.pathWeight(a, route2Edges(route), amount, 0, None)
|
||||
assert(route2Ids(route) === 4 :: 5 :: 6 :: Nil)
|
||||
assert(weightedPath.length === 3)
|
||||
|
@ -127,7 +127,7 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
val graph2 = graph.addEdge(makeEdge(5L, e, f, feeBase = 1 msat, feeProportionalMillionth = 400, minHtlc = 0 msat, capacity = 10 sat))
|
||||
val graph3 = graph.addEdge(makeEdge(5L, e, f, feeBase = 1 msat, feeProportionalMillionth = 400, minHtlc = 0 msat, balance_opt = Some(10001 msat)))
|
||||
for (g <- Seq(graph1, graph2, graph3)) {
|
||||
val Success(route1) = findRoute(g, a, d, amount, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
val Success(route1 :: Nil) = findRoute(g, a, d, amount, maxFee = 10 msat, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route2Ids(route1) === 1 :: 2 :: 3 :: Nil)
|
||||
}
|
||||
}
|
||||
|
@ -141,8 +141,8 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
makeEdge(5L, d, e, 5 msat, 0) // d -> e
|
||||
))
|
||||
|
||||
val route = findRoute(g, a, e, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route.map(route2Ids) === Success(2 :: 5 :: Nil))
|
||||
val Success(route :: Nil) = findRoute(g, a, e, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route2Ids(route) === 2 :: 5 :: Nil)
|
||||
}
|
||||
|
||||
test("calculate simple route (add and remove edges") {
|
||||
|
@ -153,12 +153,12 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
makeEdge(4L, d, e, 0 msat, 0)
|
||||
))
|
||||
|
||||
val route1 = findRoute(g, a, e, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route1.map(route2Ids) === Success(1 :: 2 :: 3 :: 4 :: Nil))
|
||||
val Success(route1 :: Nil) = findRoute(g, a, e, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route2Ids(route1) === 1 :: 2 :: 3 :: 4 :: Nil)
|
||||
|
||||
val graphWithRemovedEdge = g.removeEdge(ChannelDesc(ShortChannelId(3L), c, d))
|
||||
val route2 = findRoute(graphWithRemovedEdge, a, e, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route2.map(route2Ids) === Failure(RouteNotFound))
|
||||
val route2 = findRoute(graphWithRemovedEdge, a, e, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route2 === Failure(RouteNotFound))
|
||||
}
|
||||
|
||||
test("calculate the shortest path (hardcoded nodes)") {
|
||||
|
@ -176,8 +176,8 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
makeEdge(4L, f, h, 50 msat, 0) // more expensive but fee will be ignored since f is the payer
|
||||
))
|
||||
|
||||
val route = findRoute(graph, f, i, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route.map(route2Ids) === Success(4 :: 3 :: Nil))
|
||||
val Success(route :: Nil) = findRoute(graph, f, i, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route2Ids(route) === 4 :: 3 :: Nil)
|
||||
}
|
||||
|
||||
test("calculate the shortest path (select direct channel)") {
|
||||
|
@ -195,8 +195,9 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
makeEdge(3L, h, i, 0 msat, 0)
|
||||
))
|
||||
|
||||
val route = findRoute(graph, f, i, DEFAULT_AMOUNT_MSAT, numRoutes = 2, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route.map(route2Ids) === Success(4 :: Nil))
|
||||
val Success(route1 :: route2 :: Nil) = findRoute(graph, f, i, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 2, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route2Ids(route1) === 4 :: Nil)
|
||||
assert(route2Ids(route2) === 1 :: 2 :: 3 :: Nil)
|
||||
}
|
||||
|
||||
test("find a route using channels with htlMaximumMsat close to the payment amount") {
|
||||
|
@ -214,8 +215,8 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
makeEdge(3L, h, i, 1 msat, 0)
|
||||
))
|
||||
|
||||
val route = findRoute(graph, f, i, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route.map(route2Ids) == Success(1 :: 2 :: 3 :: Nil))
|
||||
val Success(route :: Nil) = findRoute(graph, f, i, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route2Ids(route) === 1 :: 2 :: 3 :: Nil)
|
||||
}
|
||||
|
||||
test("find a route using channels with htlMinimumMsat close to the payment amount") {
|
||||
|
@ -233,8 +234,8 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
makeEdge(3L, h, i, 1 msat, 0)
|
||||
))
|
||||
|
||||
val route = findRoute(graph, f, i, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route.map(route2Ids) === Failure(RouteNotFound))
|
||||
val route = findRoute(graph, f, i, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route === Failure(RouteNotFound))
|
||||
}
|
||||
|
||||
test("if there are multiple channels between the same node, select the cheapest") {
|
||||
|
@ -252,8 +253,8 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
makeEdge(3L, h, i, 0 msat, 0)
|
||||
))
|
||||
|
||||
val route = findRoute(graph, f, i, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route.map(route2Ids) === Success(1 :: 6 :: 3 :: Nil))
|
||||
val Success(route :: Nil) = findRoute(graph, f, i, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route2Ids(route) === 1 :: 6 :: 3 :: Nil)
|
||||
}
|
||||
|
||||
test("if there are multiple channels between the same node, select one that has enough balance") {
|
||||
|
@ -271,8 +272,8 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
makeEdge(3L, h, i, 0 msat, 0)
|
||||
))
|
||||
|
||||
val route = findRoute(graph, f, i, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route.map(route2Ids) === Success(1 :: 2 :: 3 :: Nil))
|
||||
val Success(route :: Nil) = findRoute(graph, f, i, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route2Ids(route) === 1 :: 2 :: 3 :: Nil)
|
||||
}
|
||||
|
||||
test("calculate longer but cheaper route") {
|
||||
|
@ -284,8 +285,8 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
makeEdge(5L, b, e, 10 msat, 10)
|
||||
))
|
||||
|
||||
val route = findRoute(g, a, e, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route.map(route2Ids) === Success(1 :: 2 :: 3 :: 4 :: Nil))
|
||||
val Success(route :: Nil) = findRoute(g, a, e, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route2Ids(route) === 1 :: 2 :: 3 :: 4 :: Nil)
|
||||
}
|
||||
|
||||
test("no local channels") {
|
||||
|
@ -294,8 +295,8 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
makeEdge(4L, d, e, 0 msat, 0)
|
||||
))
|
||||
|
||||
val route = findRoute(g, a, e, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route.map(route2Ids) === Failure(RouteNotFound))
|
||||
val route = findRoute(g, a, e, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route === Failure(RouteNotFound))
|
||||
}
|
||||
|
||||
test("route not found") {
|
||||
|
@ -305,8 +306,8 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
makeEdge(4L, d, e, 0 msat, 0)
|
||||
))
|
||||
|
||||
val route = findRoute(g, a, e, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route.map(route2Ids) === Failure(RouteNotFound))
|
||||
val route = findRoute(g, a, e, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route === Failure(RouteNotFound))
|
||||
}
|
||||
|
||||
test("route not found (source OR target node not connected)") {
|
||||
|
@ -315,8 +316,8 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
makeEdge(4L, c, d, 0 msat, 0)
|
||||
)).addVertex(a).addVertex(e)
|
||||
|
||||
assert(findRoute(g, a, d, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) === Failure(RouteNotFound))
|
||||
assert(findRoute(g, b, e, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) === Failure(RouteNotFound))
|
||||
assert(findRoute(g, a, d, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) === Failure(RouteNotFound))
|
||||
assert(findRoute(g, b, e, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) === Failure(RouteNotFound))
|
||||
}
|
||||
|
||||
test("route not found (amount too high OR too low)") {
|
||||
|
@ -338,8 +339,8 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
val g = DirectedGraph(edgesHi)
|
||||
val g1 = DirectedGraph(edgesLo)
|
||||
|
||||
assert(findRoute(g, a, d, highAmount, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) === Failure(RouteNotFound))
|
||||
assert(findRoute(g1, a, d, lowAmount, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) === Failure(RouteNotFound))
|
||||
assert(findRoute(g, a, d, highAmount, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) === Failure(RouteNotFound))
|
||||
assert(findRoute(g1, a, d, lowAmount, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) === Failure(RouteNotFound))
|
||||
}
|
||||
|
||||
test("route not found (balance too low)") {
|
||||
|
@ -348,7 +349,7 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
makeEdge(2L, b, c, 1 msat, 2, minHtlc = 10000 msat),
|
||||
makeEdge(3L, c, d, 1 msat, 2, minHtlc = 10000 msat)
|
||||
))
|
||||
assert(findRoute(g, a, d, 15000 msat, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000).isSuccess)
|
||||
assert(findRoute(g, a, d, 15000 msat, 100 msat, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000).isSuccess)
|
||||
|
||||
// not enough balance on the last edge
|
||||
val g1 = DirectedGraph(List(
|
||||
|
@ -368,7 +369,7 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
makeEdge(2L, b, c, 1 msat, 2, minHtlc = 10000 msat),
|
||||
makeEdge(3L, c, d, 1 msat, 2, minHtlc = 10000 msat)
|
||||
))
|
||||
Seq(g1, g2, g3).foreach(g => assert(findRoute(g, a, d, 15000 msat, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) === Failure(RouteNotFound)))
|
||||
Seq(g1, g2, g3).foreach(g => assert(findRoute(g, a, d, 15000 msat, 100 msat, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) === Failure(RouteNotFound)))
|
||||
}
|
||||
|
||||
test("route to self") {
|
||||
|
@ -378,8 +379,8 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
makeEdge(3L, c, d, 0 msat, 0)
|
||||
))
|
||||
|
||||
val route = findRoute(g, a, a, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route.map(route2Ids) === Failure(CannotRouteToSelf))
|
||||
val route = findRoute(g, a, a, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route === Failure(CannotRouteToSelf))
|
||||
}
|
||||
|
||||
test("route to immediate neighbor") {
|
||||
|
@ -390,8 +391,8 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
makeEdge(4L, d, e, 0 msat, 0)
|
||||
))
|
||||
|
||||
val route = findRoute(g, a, b, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route.map(route2Ids) === Success(1 :: Nil))
|
||||
val Success(route :: Nil) = findRoute(g, a, b, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route2Ids(route) === 1 :: Nil)
|
||||
}
|
||||
|
||||
test("directed graph") {
|
||||
|
@ -403,11 +404,11 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
makeEdge(4L, d, e, 0 msat, 0)
|
||||
))
|
||||
|
||||
val route1 = findRoute(g, a, e, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route1.map(route2Ids) === Success(1 :: 2 :: 3 :: 4 :: Nil))
|
||||
val Success(route1 :: Nil) = findRoute(g, a, e, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route2Ids(route1) === 1 :: 2 :: 3 :: 4 :: Nil)
|
||||
|
||||
val route2 = findRoute(g, e, a, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route2.map(route2Ids) === Failure(RouteNotFound))
|
||||
val route2 = findRoute(g, e, a, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route2 === Failure(RouteNotFound))
|
||||
}
|
||||
|
||||
test("calculate route and return metadata") {
|
||||
|
@ -434,8 +435,8 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
)
|
||||
|
||||
val g = DirectedGraph(edges)
|
||||
val hops = findRoute(g, a, e, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000).get.hops
|
||||
assert(hops === ChannelHop(a, b, uab) :: ChannelHop(b, c, ubc) :: ChannelHop(c, d, ucd) :: ChannelHop(d, e, ude) :: Nil)
|
||||
val Success(route :: Nil) = findRoute(g, a, e, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route.hops === ChannelHop(a, b, uab) :: ChannelHop(b, c, ubc) :: ChannelHop(c, d, ucd) :: ChannelHop(d, e, ude) :: Nil)
|
||||
}
|
||||
|
||||
test("convert extra hops to assisted channels") {
|
||||
|
@ -468,8 +469,8 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
makeEdge(4L, d, e, 0 msat, 0)
|
||||
))
|
||||
|
||||
val route1 = findRoute(g, a, e, DEFAULT_AMOUNT_MSAT, numRoutes = 1, ignoredEdges = Set(ChannelDesc(ShortChannelId(3L), c, d)), routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route1.map(route2Ids) === Failure(RouteNotFound))
|
||||
val route1 = findRoute(g, a, e, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, ignoredEdges = Set(ChannelDesc(ShortChannelId(3L), c, d)), routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route1 === Failure(RouteNotFound))
|
||||
|
||||
// verify that we left the graph untouched
|
||||
assert(g.containsEdge(ChannelDesc(ShortChannelId(3), c, d)))
|
||||
|
@ -477,8 +478,8 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
assert(g.containsVertex(d))
|
||||
|
||||
// make sure we can find a route if without the blacklist
|
||||
val route2 = findRoute(g, a, e, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route2.map(route2Ids) === Success(1 :: 2 :: 3 :: 4 :: Nil))
|
||||
val Success(route2 :: Nil) = findRoute(g, a, e, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route2Ids(route2) === 1 :: 2 :: 3 :: 4 :: Nil)
|
||||
}
|
||||
|
||||
test("route to a destination that is not in the graph (with assisted routes)") {
|
||||
|
@ -488,13 +489,13 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
makeEdge(3L, c, d, 10 msat, 10)
|
||||
))
|
||||
|
||||
val route = findRoute(g, a, e, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route.map(route2Ids) === Failure(RouteNotFound))
|
||||
val route = findRoute(g, a, e, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route === Failure(RouteNotFound))
|
||||
|
||||
// now we add the missing edge to reach the destination
|
||||
val extraGraphEdges = Set(makeEdge(4L, d, e, 5 msat, 5))
|
||||
val route1 = findRoute(g, a, e, DEFAULT_AMOUNT_MSAT, numRoutes = 1, extraEdges = extraGraphEdges, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route1.map(route2Ids) === Success(1 :: 2 :: 3 :: 4 :: Nil))
|
||||
val Success(route1 :: Nil) = findRoute(g, a, e, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, extraEdges = extraGraphEdges, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route2Ids(route1) === 1 :: 2 :: 3 :: 4 :: Nil)
|
||||
}
|
||||
|
||||
test("route from a source that is not in the graph (with assisted routes)") {
|
||||
|
@ -503,13 +504,13 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
makeEdge(3L, c, d, 10 msat, 10)
|
||||
))
|
||||
|
||||
val route = findRoute(g, a, d, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route.map(route2Ids) === Failure(RouteNotFound))
|
||||
val route = findRoute(g, a, d, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route === Failure(RouteNotFound))
|
||||
|
||||
// now we add the missing starting edge
|
||||
val extraGraphEdges = Set(makeEdge(1L, a, b, 5 msat, 5))
|
||||
val route1 = findRoute(g, a, d, DEFAULT_AMOUNT_MSAT, numRoutes = 1, extraEdges = extraGraphEdges, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route1.map(route2Ids) === Success(1 :: 2 :: 3 :: Nil))
|
||||
val Success(route1 :: Nil) = findRoute(g, a, d, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, extraEdges = extraGraphEdges, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route2Ids(route1) === 1 :: 2 :: 3 :: Nil)
|
||||
}
|
||||
|
||||
test("verify that extra hops takes precedence over known channels") {
|
||||
|
@ -520,14 +521,14 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
makeEdge(4L, d, e, 10 msat, 10)
|
||||
))
|
||||
|
||||
val route1 = findRoute(g, a, e, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route1.map(route2Ids) === Success(1 :: 2 :: 3 :: 4 :: Nil))
|
||||
assert(route1.get.hops(1).lastUpdate.feeBaseMsat === 10.msat)
|
||||
val Success(route1 :: Nil) = findRoute(g, a, e, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route2Ids(route1) === 1 :: 2 :: 3 :: 4 :: Nil)
|
||||
assert(route1.hops(1).lastUpdate.feeBaseMsat === 10.msat)
|
||||
|
||||
val extraGraphEdges = Set(makeEdge(2L, b, c, 5 msat, 5))
|
||||
val route2 = findRoute(g, a, e, DEFAULT_AMOUNT_MSAT, numRoutes = 1, extraEdges = extraGraphEdges, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route2.map(route2Ids) === Success(1 :: 2 :: 3 :: 4 :: Nil))
|
||||
assert(route2.get.hops(1).lastUpdate.feeBaseMsat === 5.msat)
|
||||
val Success(route2 :: Nil) = findRoute(g, a, e, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, extraEdges = extraGraphEdges, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route2Ids(route2) === 1 :: 2 :: 3 :: 4 :: Nil)
|
||||
assert(route2.hops(1).lastUpdate.feeBaseMsat === 5.msat)
|
||||
}
|
||||
|
||||
test("compute ignored channels") {
|
||||
|
@ -583,10 +584,10 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
|
||||
val g = DirectedGraph(edges)
|
||||
|
||||
assert(findRoute(g, nodes(0), nodes(18), DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000).map(route2Ids) === Success(0 until 18))
|
||||
assert(findRoute(g, nodes(0), nodes(19), DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000).map(route2Ids) === Success(0 until 19))
|
||||
assert(findRoute(g, nodes(0), nodes(20), DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000).map(route2Ids) === Success(0 until 20))
|
||||
assert(findRoute(g, nodes(0), nodes(21), DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000).map(route2Ids) === Failure(RouteNotFound))
|
||||
assert(findRoute(g, nodes(0), nodes(18), DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000).map(r => route2Ids(r.head)) === Success(0 until 18))
|
||||
assert(findRoute(g, nodes(0), nodes(19), DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000).map(r => route2Ids(r.head)) === Success(0 until 19))
|
||||
assert(findRoute(g, nodes(0), nodes(20), DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000).map(r => route2Ids(r.head)) === Success(0 until 20))
|
||||
assert(findRoute(g, nodes(0), nodes(21), DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000) === Failure(RouteNotFound))
|
||||
}
|
||||
|
||||
test("ignore cheaper route when it has more than 20 hops") {
|
||||
|
@ -601,8 +602,8 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
|
||||
val g = DirectedGraph(expensiveShortEdge :: edges)
|
||||
|
||||
val route = findRoute(g, nodes(0), nodes(49), DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route.map(route2Ids) === Success(0 :: 1 :: 99 :: 48 :: Nil))
|
||||
val Success(route :: Nil) = findRoute(g, nodes(0), nodes(49), DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route2Ids(route) === 0 :: 1 :: 99 :: 48 :: Nil)
|
||||
}
|
||||
|
||||
test("ignore cheaper route when it has more than the requested CLTV") {
|
||||
|
@ -616,8 +617,8 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
makeEdge(6, f, d, feeBase = 5 msat, 0, minHtlc = 0 msat, maxHtlc = None, CltvExpiryDelta(9))
|
||||
))
|
||||
|
||||
val route = findRoute(g, a, d, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS.copy(routeMaxCltv = CltvExpiryDelta(28)), currentBlockHeight = 400000)
|
||||
assert(route.map(route2Ids) === Success(4 :: 5 :: 6 :: Nil))
|
||||
val Success(route :: Nil) = findRoute(g, a, d, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS.copy(routeMaxCltv = CltvExpiryDelta(28)), currentBlockHeight = 400000)
|
||||
assert(route2Ids(route) === 4 :: 5 :: 6 :: Nil)
|
||||
}
|
||||
|
||||
test("ignore cheaper route when it grows longer than the requested size") {
|
||||
|
@ -631,8 +632,8 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
makeEdge(6, b, f, feeBase = 5 msat, 0, minHtlc = 0 msat, maxHtlc = None, CltvExpiryDelta(9))
|
||||
))
|
||||
|
||||
val route = findRoute(g, a, f, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS.copy(routeMaxLength = 3), currentBlockHeight = 400000)
|
||||
assert(route.map(route2Ids) === Success(1 :: 6 :: Nil))
|
||||
val Success(route :: Nil) = findRoute(g, a, f, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS.copy(routeMaxLength = 3), currentBlockHeight = 400000)
|
||||
assert(route2Ids(route) === 1 :: 6 :: Nil)
|
||||
}
|
||||
|
||||
test("ignore loops") {
|
||||
|
@ -644,8 +645,8 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
makeEdge(5L, d, e, 10 msat, 10)
|
||||
))
|
||||
|
||||
val route1 = findRoute(g, a, e, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route1.map(route2Ids) === Success(1 :: 2 :: 4 :: 5 :: Nil))
|
||||
val Success(route :: Nil) = findRoute(g, a, e, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route2Ids(route) === 1 :: 2 :: 4 :: 5 :: Nil)
|
||||
}
|
||||
|
||||
test("ensure the route calculation terminates correctly when selecting 0-fees edges") {
|
||||
|
@ -660,8 +661,8 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
makeEdge(5L, e, d, 0 msat, 0) // e -> d
|
||||
))
|
||||
|
||||
val route1 = findRoute(g, a, d, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route1.map(route2Ids) === Success(1 :: 3 :: 5 :: Nil))
|
||||
val Success(route :: Nil) = findRoute(g, a, d, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route2Ids(route) === 1 :: 3 :: 5 :: Nil)
|
||||
}
|
||||
|
||||
// +---+ +---+ +---+
|
||||
|
@ -769,29 +770,31 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
}
|
||||
|
||||
test("select a random route below the requested fee") {
|
||||
val strictFeeParams = DEFAULT_ROUTE_PARAMS.copy(maxFeeBase = 7 msat, maxFeePct = 0)
|
||||
val strictFeeParams = DEFAULT_ROUTE_PARAMS.copy(maxFeeBase = 7 msat, maxFeePct = 0, randomize = true)
|
||||
val strictFee = strictFeeParams.getMaxFee(DEFAULT_AMOUNT_MSAT)
|
||||
assert(strictFee === 7.msat)
|
||||
|
||||
// A -> B -> C -> D has total cost of 10000005
|
||||
// A -> E -> C -> D has total cost of 11080003 !!
|
||||
// A -> E -> C -> D has total cost of 10000103 !!
|
||||
// A -> E -> F -> D has total cost of 10000006
|
||||
val g = DirectedGraph(List(
|
||||
makeEdge(1L, a, b, feeBase = 1 msat, 0),
|
||||
makeEdge(4L, a, e, feeBase = 1 msat, 0),
|
||||
makeEdge(2L, b, c, feeBase = 2 msat, 0),
|
||||
makeEdge(3L, c, d, feeBase = 3 msat, 0),
|
||||
makeEdge(4L, a, e, feeBase = 1 msat, 0),
|
||||
makeEdge(5L, e, f, feeBase = 3 msat, 0),
|
||||
makeEdge(6L, f, d, feeBase = 3 msat, 0),
|
||||
makeEdge(7L, e, c, feeBase = 9 msat, 0)
|
||||
makeEdge(7L, e, c, feeBase = 100 msat, 0)
|
||||
))
|
||||
|
||||
(for {_ <- 0 to 10} yield findRoute(g, a, d, DEFAULT_AMOUNT_MSAT, numRoutes = 3, routeParams = strictFeeParams, currentBlockHeight = 400000)).map {
|
||||
case Failure(thr) => fail(thr)
|
||||
case Success(someRoute) =>
|
||||
val weightedPath = Graph.pathWeight(a, route2Edges(someRoute), DEFAULT_AMOUNT_MSAT, 0, None)
|
||||
val totalFees = weightedPath.cost - DEFAULT_AMOUNT_MSAT
|
||||
// over the three routes we could only get the 2 cheapest because the third is too expensive (over 7 msat of fees)
|
||||
assert(totalFees === 5.msat || totalFees === 6.msat)
|
||||
assert(weightedPath.length === 3)
|
||||
for (_ <- 0 to 10) {
|
||||
val Success(routes) = findRoute(g, a, d, DEFAULT_AMOUNT_MSAT, strictFee, numRoutes = 3, routeParams = strictFeeParams, currentBlockHeight = 400000)
|
||||
assert(routes.length === 2, routes)
|
||||
val weightedPath = Graph.pathWeight(a, route2Edges(routes.head), DEFAULT_AMOUNT_MSAT, 400000, None)
|
||||
val totalFees = weightedPath.cost - DEFAULT_AMOUNT_MSAT
|
||||
// over the three routes we could only get the 2 cheapest because the third is too expensive (over 7 msat of fees)
|
||||
assert(totalFees === 5.msat || totalFees === 6.msat)
|
||||
assert(weightedPath.length === 3)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -812,17 +815,17 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
makeEdge(7L, e, c, feeBase = 2 msat, 0, minHtlc = 0 msat, capacity = largeCapacity, cltvDelta = CltvExpiryDelta(12))
|
||||
))
|
||||
|
||||
val Success(routeFeeOptimized) = findRoute(g, a, d, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
val Success(routeFeeOptimized :: Nil) = findRoute(g, a, d, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = 400000)
|
||||
assert(route2Nodes(routeFeeOptimized) === (a, b) :: (b, c) :: (c, d) :: Nil)
|
||||
|
||||
val Success(routeCltvOptimized) = findRoute(g, a, d, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS.copy(ratios = Some(WeightRatios(
|
||||
val Success(routeCltvOptimized :: Nil) = findRoute(g, a, d, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS.copy(ratios = Some(WeightRatios(
|
||||
cltvDeltaFactor = 1,
|
||||
ageFactor = 0,
|
||||
capacityFactor = 0
|
||||
))), currentBlockHeight = 400000)
|
||||
assert(route2Nodes(routeCltvOptimized) === (a, e) :: (e, f) :: (f, d) :: Nil)
|
||||
|
||||
val Success(routeCapacityOptimized) = findRoute(g, a, d, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS.copy(ratios = Some(WeightRatios(
|
||||
val Success(routeCapacityOptimized :: Nil) = findRoute(g, a, d, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS.copy(ratios = Some(WeightRatios(
|
||||
cltvDeltaFactor = 0,
|
||||
ageFactor = 0,
|
||||
capacityFactor = 1
|
||||
|
@ -842,7 +845,7 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
makeEdge(ShortChannelId(s"${currentBlockHeight}x0x6").toLong, f, d, feeBase = 1 msat, 0, minHtlc = 0 msat, maxHtlc = None, cltvDelta = CltvExpiryDelta(144))
|
||||
))
|
||||
|
||||
val Success(routeScoreOptimized) = findRoute(g, a, d, DEFAULT_AMOUNT_MSAT / 2, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS.copy(ratios = Some(WeightRatios(
|
||||
val Success(routeScoreOptimized :: Nil) = findRoute(g, a, d, DEFAULT_AMOUNT_MSAT / 2, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS.copy(ratios = Some(WeightRatios(
|
||||
ageFactor = 0.33,
|
||||
cltvDeltaFactor = 0.33,
|
||||
capacityFactor = 0.33
|
||||
|
@ -861,7 +864,7 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
makeEdge(6, f, d, feeBase = 1 msat, 0, minHtlc = 0 msat, maxHtlc = None, cltvDelta = CltvExpiryDelta(12))
|
||||
))
|
||||
|
||||
val Success(routeScoreOptimized) = findRoute(g, a, d, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS.copy(ratios = Some(WeightRatios(
|
||||
val Success(routeScoreOptimized :: Nil) = findRoute(g, a, d, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS.copy(ratios = Some(WeightRatios(
|
||||
ageFactor = 0.33,
|
||||
cltvDeltaFactor = 0.33,
|
||||
capacityFactor = 0.33
|
||||
|
@ -882,7 +885,7 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
makeEdge(6, f, d, feeBase = 1 msat, 0, minHtlc = 0 msat, maxHtlc = None, cltvDelta = CltvExpiryDelta(144))
|
||||
))
|
||||
|
||||
val Success(routeScoreOptimized) = findRoute(g, a, d, DEFAULT_AMOUNT_MSAT / 2, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS.copy(ratios = Some(WeightRatios(
|
||||
val Success(routeScoreOptimized :: Nil) = findRoute(g, a, d, DEFAULT_AMOUNT_MSAT / 2, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS.copy(ratios = Some(WeightRatios(
|
||||
ageFactor = 0.33,
|
||||
cltvDeltaFactor = 0.33,
|
||||
capacityFactor = 0.33
|
||||
|
@ -929,7 +932,7 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution {
|
|||
val targetNode = PublicKey(hex"024655b768ef40951b20053a5c4b951606d4d86085d51238f2c67c7dec29c792ca")
|
||||
val amount = 351000 msat
|
||||
|
||||
val Success(route) = findRoute(g, thisNode, targetNode, amount, 1, Set.empty, Set.empty, Set.empty, params, currentBlockHeight = 567634) // simulate mainnet block for heuristic
|
||||
val Success(route :: Nil) = findRoute(g, thisNode, targetNode, amount, DEFAULT_MAX_FEE, 1, Set.empty, Set.empty, Set.empty, params, currentBlockHeight = 567634) // simulate mainnet block for heuristic
|
||||
assert(route.length == 2)
|
||||
assert(route.hops.last.nextNodeId == targetNode)
|
||||
}
|
||||
|
@ -961,6 +964,7 @@ object RouteCalculationSpec {
|
|||
val noopBoundaries = { _: RichWeight => true }
|
||||
|
||||
val DEFAULT_AMOUNT_MSAT = 10000000 msat
|
||||
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)
|
||||
|
|
|
@ -27,14 +27,13 @@ import fr.acinq.eclair.crypto.TransportHandler
|
|||
import fr.acinq.eclair.io.Peer.PeerRoutingMessage
|
||||
import fr.acinq.eclair.payment.PaymentRequest.ExtraHop
|
||||
import fr.acinq.eclair.router.Announcements.{makeChannelUpdate, makeNodeAnnouncement}
|
||||
import fr.acinq.eclair.router.RouteCalculationSpec.DEFAULT_AMOUNT_MSAT
|
||||
import fr.acinq.eclair.router.RouteCalculationSpec.{DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE}
|
||||
import fr.acinq.eclair.router.Router._
|
||||
import fr.acinq.eclair.transactions.Scripts
|
||||
import fr.acinq.eclair.wire.{Color, QueryShortChannelIds}
|
||||
import fr.acinq.eclair.{CltvExpiryDelta, Features, LongToBtcAmount, MilliSatoshi, ShortChannelId, TestConstants, randomKey}
|
||||
import scodec.bits._
|
||||
|
||||
import scala.compat.Platform
|
||||
import scala.concurrent.duration._
|
||||
|
||||
/**
|
||||
|
@ -43,8 +42,6 @@ import scala.concurrent.duration._
|
|||
|
||||
class RouterSpec extends BaseRouterSpec {
|
||||
|
||||
val relaxedRouteParams = Some(RouteCalculationSpec.DEFAULT_ROUTE_PARAMS.copy(maxFeePct = 0.3))
|
||||
|
||||
test("properly announce valid new channels and ignore invalid ones") { fixture =>
|
||||
import fixture._
|
||||
val eventListener = TestProbe()
|
||||
|
@ -315,7 +312,7 @@ class RouterSpec extends BaseRouterSpec {
|
|||
import fixture._
|
||||
val sender = TestProbe()
|
||||
// no route a->f
|
||||
sender.send(router, RouteRequest(a, f, DEFAULT_AMOUNT_MSAT))
|
||||
sender.send(router, RouteRequest(a, f, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE))
|
||||
sender.expectMsg(Failure(RouteNotFound))
|
||||
}
|
||||
|
||||
|
@ -323,7 +320,7 @@ class RouterSpec extends BaseRouterSpec {
|
|||
import fixture._
|
||||
val sender = TestProbe()
|
||||
// no route a->f
|
||||
sender.send(router, RouteRequest(randomKey.publicKey, f, DEFAULT_AMOUNT_MSAT))
|
||||
sender.send(router, RouteRequest(randomKey.publicKey, f, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE))
|
||||
sender.expectMsg(Failure(RouteNotFound))
|
||||
}
|
||||
|
||||
|
@ -331,19 +328,19 @@ class RouterSpec extends BaseRouterSpec {
|
|||
import fixture._
|
||||
val sender = TestProbe()
|
||||
// no route a->f
|
||||
sender.send(router, RouteRequest(a, randomKey.publicKey, DEFAULT_AMOUNT_MSAT))
|
||||
sender.send(router, RouteRequest(a, randomKey.publicKey, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE))
|
||||
sender.expectMsg(Failure(RouteNotFound))
|
||||
}
|
||||
|
||||
test("route found") { fixture =>
|
||||
import fixture._
|
||||
val sender = TestProbe()
|
||||
sender.send(router, RouteRequest(a, d, DEFAULT_AMOUNT_MSAT, routeParams = relaxedRouteParams))
|
||||
sender.send(router, RouteRequest(a, d, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE))
|
||||
val res = sender.expectMsgType[RouteResponse]
|
||||
assert(res.routes.head.hops.map(_.nodeId).toList === a :: b :: c :: Nil)
|
||||
assert(res.routes.head.hops.last.nextNodeId === d)
|
||||
|
||||
sender.send(router, RouteRequest(a, h, DEFAULT_AMOUNT_MSAT))
|
||||
sender.send(router, RouteRequest(a, h, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE))
|
||||
val res1 = sender.expectMsgType[RouteResponse]
|
||||
assert(res1.routes.head.hops.map(_.nodeId).toList === a :: g :: Nil)
|
||||
assert(res1.routes.head.hops.last.nextNodeId === h)
|
||||
|
@ -358,7 +355,7 @@ class RouterSpec extends BaseRouterSpec {
|
|||
val extraHop_cx = ExtraHop(c, ShortChannelId(1), 10 msat, 11, CltvExpiryDelta(12))
|
||||
val extraHop_xy = ExtraHop(x, ShortChannelId(2), 10 msat, 11, CltvExpiryDelta(12))
|
||||
val extraHop_yz = ExtraHop(y, ShortChannelId(3), 20 msat, 21, CltvExpiryDelta(22))
|
||||
sender.send(router, RouteRequest(a, z, DEFAULT_AMOUNT_MSAT, assistedRoutes = Seq(extraHop_cx :: extraHop_xy :: extraHop_yz :: Nil)))
|
||||
sender.send(router, RouteRequest(a, z, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, assistedRoutes = Seq(extraHop_cx :: extraHop_xy :: extraHop_yz :: Nil)))
|
||||
val res = sender.expectMsgType[RouteResponse]
|
||||
assert(res.routes.head.hops.map(_.nodeId).toList === a :: b :: c :: x :: y :: Nil)
|
||||
assert(res.routes.head.hops.last.nextNodeId === z)
|
||||
|
@ -368,7 +365,7 @@ class RouterSpec extends BaseRouterSpec {
|
|||
import fixture._
|
||||
val sender = TestProbe()
|
||||
val peerConnection = TestProbe()
|
||||
sender.send(router, RouteRequest(a, d, DEFAULT_AMOUNT_MSAT, routeParams = relaxedRouteParams))
|
||||
sender.send(router, RouteRequest(a, d, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE))
|
||||
val res = sender.expectMsgType[RouteResponse]
|
||||
assert(res.routes.head.hops.map(_.nodeId).toList === a :: b :: c :: Nil)
|
||||
assert(res.routes.head.hops.last.nextNodeId === d)
|
||||
|
@ -376,21 +373,21 @@ class RouterSpec extends BaseRouterSpec {
|
|||
val channelUpdate_cd1 = makeChannelUpdate(Block.RegtestGenesisBlock.hash, priv_c, d, channelId_cd, CltvExpiryDelta(3), 0 msat, 153000 msat, 4, htlcMaximum, enable = false)
|
||||
peerConnection.send(router, PeerRoutingMessage(peerConnection.ref, remoteNodeId, channelUpdate_cd1))
|
||||
peerConnection.expectMsg(TransportHandler.ReadAck(channelUpdate_cd1))
|
||||
sender.send(router, RouteRequest(a, d, DEFAULT_AMOUNT_MSAT, routeParams = relaxedRouteParams))
|
||||
sender.send(router, RouteRequest(a, d, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE))
|
||||
sender.expectMsg(Failure(RouteNotFound))
|
||||
}
|
||||
|
||||
test("route not found (private channel disabled)") { fixture =>
|
||||
import fixture._
|
||||
val sender = TestProbe()
|
||||
sender.send(router, RouteRequest(a, h, DEFAULT_AMOUNT_MSAT))
|
||||
sender.send(router, RouteRequest(a, h, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE))
|
||||
val res = sender.expectMsgType[RouteResponse]
|
||||
assert(res.routes.head.hops.map(_.nodeId).toList === a :: g :: Nil)
|
||||
assert(res.routes.head.hops.last.nextNodeId === h)
|
||||
|
||||
val channelUpdate_ag1 = makeChannelUpdate(Block.RegtestGenesisBlock.hash, priv_a, g, channelId_ag, CltvExpiryDelta(7), 0 msat, 10 msat, 10, htlcMaximum, enable = false)
|
||||
sender.send(router, LocalChannelUpdate(sender.ref, null, channelId_ag, g, None, channelUpdate_ag1, CommitmentsSpec.makeCommitments(10000 msat, 15000 msat, a, g, announceChannel = false)))
|
||||
sender.send(router, RouteRequest(a, h, DEFAULT_AMOUNT_MSAT))
|
||||
sender.send(router, RouteRequest(a, h, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE))
|
||||
sender.expectMsg(Failure(RouteNotFound))
|
||||
}
|
||||
|
||||
|
@ -399,38 +396,38 @@ class RouterSpec extends BaseRouterSpec {
|
|||
val sender = TestProbe()
|
||||
|
||||
// Via private channels.
|
||||
sender.send(router, RouteRequest(a, h, DEFAULT_AMOUNT_MSAT))
|
||||
sender.send(router, RouteRequest(a, h, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE))
|
||||
sender.expectMsgType[RouteResponse]
|
||||
sender.send(router, RouteRequest(a, h, 50000000 msat))
|
||||
sender.send(router, RouteRequest(a, h, 50000000 msat, Long.MaxValue.msat))
|
||||
sender.expectMsg(Failure(RouteNotFound))
|
||||
|
||||
// Via public channels.
|
||||
sender.send(router, RouteRequest(a, d, DEFAULT_AMOUNT_MSAT))
|
||||
sender.send(router, RouteRequest(a, d, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE))
|
||||
sender.expectMsgType[RouteResponse]
|
||||
val commitments1 = CommitmentsSpec.makeCommitments(10000000 msat, 20000000 msat, a, b, announceChannel = true)
|
||||
sender.send(router, LocalChannelUpdate(sender.ref, null, channelId_ab, b, Some(chan_ab), update_ab, commitments1))
|
||||
sender.send(router, RouteRequest(a, d, 12000000 msat))
|
||||
sender.send(router, RouteRequest(a, d, 12000000 msat, Long.MaxValue.msat))
|
||||
sender.expectMsg(Failure(RouteNotFound))
|
||||
sender.send(router, RouteRequest(a, d, 5000000 msat))
|
||||
sender.send(router, RouteRequest(a, d, 5000000 msat, Long.MaxValue.msat))
|
||||
sender.expectMsgType[RouteResponse]
|
||||
}
|
||||
|
||||
test("temporary channel exclusion") { fixture =>
|
||||
import fixture._
|
||||
val sender = TestProbe()
|
||||
sender.send(router, RouteRequest(a, d, DEFAULT_AMOUNT_MSAT, routeParams = relaxedRouteParams))
|
||||
sender.send(router, RouteRequest(a, d, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE))
|
||||
sender.expectMsgType[RouteResponse]
|
||||
val bc = ChannelDesc(channelId_bc, b, c)
|
||||
// let's exclude channel b->c
|
||||
sender.send(router, ExcludeChannel(bc))
|
||||
sender.send(router, RouteRequest(a, d, DEFAULT_AMOUNT_MSAT, routeParams = relaxedRouteParams))
|
||||
sender.send(router, RouteRequest(a, d, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE))
|
||||
sender.expectMsg(Failure(RouteNotFound))
|
||||
// note that cb is still available!
|
||||
sender.send(router, RouteRequest(d, a, DEFAULT_AMOUNT_MSAT, routeParams = relaxedRouteParams))
|
||||
sender.send(router, RouteRequest(d, a, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE))
|
||||
sender.expectMsgType[RouteResponse]
|
||||
// let's remove the exclusion
|
||||
sender.send(router, LiftChannelExclusion(bc))
|
||||
sender.send(router, RouteRequest(a, d, DEFAULT_AMOUNT_MSAT, routeParams = relaxedRouteParams))
|
||||
sender.send(router, RouteRequest(a, d, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE))
|
||||
sender.expectMsgType[RouteResponse]
|
||||
}
|
||||
|
||||
|
|
Loading…
Add table
Reference in a new issue