mirror of
https://github.com/ACINQ/eclair.git
synced 2025-02-22 06:21:42 +01:00
Make route params explicit (#1923)
For path finding we always need `routeParams`, however all of the messages asking for finding a route have `routeParams` as an optional parameter. This makes it hard to enforce that a given payments uses a given `routeParams` as there may be a `None` somewhere that reverts to the global defaults. It seems that the reason all the `routeParams` are optional is that the messages asking for finding a route are sometimes used when we already have a route. This is a hacky solution and I've tried to replace these messages by traits that work both when we want to find a route and when we already have it. I'm trying to enable AB testing which requires using different `routeParams` for different payments and this is a blocker.
This commit is contained in:
parent
d53f57fed9
commit
275581df96
23 changed files with 286 additions and 280 deletions
|
@ -179,7 +179,6 @@ eclair {
|
|||
autoprobe-count = 0 // number of parallel tasks that send test payments to detect invalid channels
|
||||
|
||||
router {
|
||||
randomize-route-selection = true // when computing a route for a payment we randomize the final selection
|
||||
channel-exclude-duration = 60 seconds // when a temporary channel failure is returned, we exclude the channel from our payment routes for this duration
|
||||
broadcast-interval = 60 seconds // see BOLT #7
|
||||
network-stats-interval = 6 hours // frequency at which we refresh global network statistics (expensive operation)
|
||||
|
@ -194,6 +193,8 @@ eclair {
|
|||
|
||||
// the values below will be used to perform route searching
|
||||
path-finding {
|
||||
randomize-route-selection = true // when computing a route for a payment we randomize the final selection
|
||||
|
||||
max-route-length = 6 // max route length for the 'first pass', if none is found then a second pass is made with no limit
|
||||
max-cltv = 1008 // max acceptable cltv expiry for the payment (1008 ~ 1 week)
|
||||
fee-threshold-sat = 21 // if fee is below this value we skip the max-fee-pct check
|
||||
|
|
|
@ -39,7 +39,7 @@ import fr.acinq.eclair.payment._
|
|||
import fr.acinq.eclair.payment.receive.MultiPartHandler.ReceivePayment
|
||||
import fr.acinq.eclair.payment.relay.Relayer.{GetOutgoingChannels, OutgoingChannels, RelayFees, UsableBalance}
|
||||
import fr.acinq.eclair.payment.send.MultiPartPaymentLifecycle.PreimageReceived
|
||||
import fr.acinq.eclair.payment.send.PaymentInitiator.{SendPayment, SendPaymentToRoute, SendPaymentToRouteResponse, SendSpontaneousPayment}
|
||||
import fr.acinq.eclair.payment.send.PaymentInitiator.{SendPaymentToNode, SendPaymentToRoute, SendPaymentToRouteResponse, SendSpontaneousPayment}
|
||||
import fr.acinq.eclair.router.Router._
|
||||
import fr.acinq.eclair.router.{NetworkStats, RouteCalculation, Router}
|
||||
import fr.acinq.eclair.wire.protocol._
|
||||
|
@ -284,8 +284,9 @@ class EclairImpl(appKit: Kit) extends Eclair with Logging {
|
|||
findRouteBetween(appKit.nodeParams.nodeId, targetNodeId, amount, assistedRoutes)
|
||||
|
||||
override def findRouteBetween(sourceNodeId: PublicKey, targetNodeId: PublicKey, amount: MilliSatoshi, assistedRoutes: Seq[Seq[PaymentRequest.ExtraHop]] = Seq.empty)(implicit timeout: Timeout): Future[RouteResponse] = {
|
||||
val maxFee = RouteCalculation.getDefaultRouteParams(appKit.nodeParams.routerConf).getMaxFee(amount)
|
||||
(appKit.router ? RouteRequest(sourceNodeId, targetNodeId, amount, maxFee, assistedRoutes)).mapTo[RouteResponse]
|
||||
val routeParams = RouteCalculation.getDefaultRouteParams(appKit.nodeParams.routerConf.pathFindingConf)
|
||||
val maxFee = routeParams.getMaxFee(amount)
|
||||
(appKit.router ? RouteRequest(sourceNodeId, targetNodeId, amount, maxFee, assistedRoutes, routeParams = routeParams)).mapTo[RouteResponse]
|
||||
}
|
||||
|
||||
override def sendToRoute(amount: MilliSatoshi, recipientAmount_opt: Option[MilliSatoshi], externalId_opt: Option[String], parentId_opt: Option[UUID], invoice: PaymentRequest, finalCltvExpiryDelta: CltvExpiryDelta, route: PredefinedRoute, trampolineSecret_opt: Option[ByteVector32], trampolineFees_opt: Option[MilliSatoshi], trampolineExpiryDelta_opt: Option[CltvExpiryDelta], trampolineNodes_opt: Seq[PublicKey])(implicit timeout: Timeout): Future[SendPaymentToRouteResponse] = {
|
||||
|
@ -306,9 +307,9 @@ class EclairImpl(appKit: Kit) extends Eclair with Logging {
|
|||
}
|
||||
}
|
||||
|
||||
private def createPaymentRequest(externalId_opt: Option[String], amount: MilliSatoshi, invoice: PaymentRequest, maxAttempts_opt: Option[Int], feeThreshold_opt: Option[Satoshi], maxFeePct_opt: Option[Double]): Either[IllegalArgumentException, SendPayment] = {
|
||||
private def createPaymentRequest(externalId_opt: Option[String], amount: MilliSatoshi, invoice: PaymentRequest, maxAttempts_opt: Option[Int], feeThreshold_opt: Option[Satoshi], maxFeePct_opt: Option[Double]): Either[IllegalArgumentException, SendPaymentToNode] = {
|
||||
val maxAttempts = maxAttempts_opt.getOrElse(appKit.nodeParams.maxPaymentAttempts)
|
||||
val defaultRouteParams = RouteCalculation.getDefaultRouteParams(appKit.nodeParams.routerConf)
|
||||
val defaultRouteParams = RouteCalculation.getDefaultRouteParams(appKit.nodeParams.routerConf.pathFindingConf)
|
||||
val routeParams = defaultRouteParams.copy(
|
||||
maxFeePct = maxFeePct_opt.getOrElse(defaultRouteParams.maxFeePct),
|
||||
maxFeeBase = feeThreshold_opt.map(_.toMilliSatoshi).getOrElse(defaultRouteParams.maxFeeBase)
|
||||
|
@ -318,8 +319,8 @@ class EclairImpl(appKit: Kit) extends Eclair with Logging {
|
|||
case Some(externalId) if externalId.length > externalIdMaxLength => Left(new IllegalArgumentException(s"externalId is too long: cannot exceed $externalIdMaxLength characters"))
|
||||
case _ if invoice.isExpired => Left(new IllegalArgumentException("invoice has expired"))
|
||||
case _ => invoice.minFinalCltvExpiryDelta match {
|
||||
case Some(minFinalCltvExpiryDelta) => Right(SendPayment(amount, invoice, maxAttempts, minFinalCltvExpiryDelta, externalId_opt, assistedRoutes = invoice.routingInfo, routeParams = Some(routeParams)))
|
||||
case None => Right(SendPayment(amount, invoice, maxAttempts, externalId = externalId_opt, assistedRoutes = invoice.routingInfo, routeParams = Some(routeParams)))
|
||||
case Some(minFinalCltvExpiryDelta) => Right(SendPaymentToNode(amount, invoice, maxAttempts, minFinalCltvExpiryDelta, externalId_opt, assistedRoutes = invoice.routingInfo, routeParams = routeParams))
|
||||
case None => Right(SendPaymentToNode(amount, invoice, maxAttempts, externalId = externalId_opt, assistedRoutes = invoice.routingInfo, routeParams = routeParams))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -343,12 +344,12 @@ class EclairImpl(appKit: Kit) extends Eclair with Logging {
|
|||
|
||||
override def sendWithPreimage(externalId_opt: Option[String], recipientNodeId: PublicKey, amount: MilliSatoshi, paymentPreimage: ByteVector32, maxAttempts_opt: Option[Int], feeThreshold_opt: Option[Satoshi], maxFeePct_opt: Option[Double])(implicit timeout: Timeout): Future[UUID] = {
|
||||
val maxAttempts = maxAttempts_opt.getOrElse(appKit.nodeParams.maxPaymentAttempts)
|
||||
val defaultRouteParams = RouteCalculation.getDefaultRouteParams(appKit.nodeParams.routerConf)
|
||||
val defaultRouteParams = RouteCalculation.getDefaultRouteParams(appKit.nodeParams.routerConf.pathFindingConf)
|
||||
val routeParams = defaultRouteParams.copy(
|
||||
maxFeePct = maxFeePct_opt.getOrElse(defaultRouteParams.maxFeePct),
|
||||
maxFeeBase = feeThreshold_opt.map(_.toMilliSatoshi).getOrElse(defaultRouteParams.maxFeeBase)
|
||||
)
|
||||
val sendPayment = SendSpontaneousPayment(amount, recipientNodeId, paymentPreimage, maxAttempts, externalId_opt, Some(routeParams))
|
||||
val sendPayment = SendSpontaneousPayment(amount, recipientNodeId, paymentPreimage, maxAttempts, externalId_opt, routeParams)
|
||||
(appKit.paymentInitiator ? sendPayment).mapTo[UUID]
|
||||
}
|
||||
|
||||
|
|
|
@ -27,7 +27,7 @@ import fr.acinq.eclair.crypto.keymanager.{ChannelKeyManager, NodeKeyManager}
|
|||
import fr.acinq.eclair.db._
|
||||
import fr.acinq.eclair.io.PeerConnection
|
||||
import fr.acinq.eclair.payment.relay.Relayer.{RelayFees, RelayParams}
|
||||
import fr.acinq.eclair.router.Router.RouterConf
|
||||
import fr.acinq.eclair.router.Router.{PathFindingConf, RouterConf}
|
||||
import fr.acinq.eclair.tor.Socks5ProxyParams
|
||||
import fr.acinq.eclair.wire.protocol.{Color, EncodingType, NodeAddress}
|
||||
import grizzled.slf4j.Logging
|
||||
|
@ -196,7 +196,9 @@ object NodeParams extends Logging {
|
|||
"feerate-provider-timeout" -> "on-chain-fees.provider-timeout",
|
||||
// v0.6.1
|
||||
"enable-db-backup" -> "file-backup.enabled",
|
||||
"backup-notify-script" -> "file-backup.notify-script"
|
||||
"backup-notify-script" -> "file-backup.notify-script",
|
||||
// v0.6.2
|
||||
"router.randomize-route-selection" -> "router.path-finding.randomize-route-selection",
|
||||
)
|
||||
deprecatedKeyPaths.foreach {
|
||||
case (old, new_) => require(!config.hasPath(old), s"configuration key '$old' has been replaced by '$new_'")
|
||||
|
@ -302,6 +304,23 @@ object NodeParams extends Logging {
|
|||
RelayFees(feeBase, relayFeesConfig.getInt("fee-proportional-millionths"))
|
||||
}
|
||||
|
||||
def getPathFindingConf(config: Config): PathFindingConf = {
|
||||
PathFindingConf(
|
||||
randomizeRouteSelection = config.getBoolean("randomize-route-selection"),
|
||||
searchMaxRouteLength = config.getInt("max-route-length"),
|
||||
searchMaxCltv = CltvExpiryDelta(config.getInt("max-cltv")),
|
||||
searchMaxFeeBase = Satoshi(config.getLong("fee-threshold-sat")),
|
||||
searchMaxFeePct = config.getDouble("max-fee-pct"),
|
||||
searchRatioBase = config.getDouble("ratio-base"),
|
||||
searchRatioCltv = config.getDouble("ratio-cltv"),
|
||||
searchRatioChannelAge = config.getDouble("ratio-channel-age"),
|
||||
searchRatioChannelCapacity = config.getDouble("ratio-channel-capacity"),
|
||||
searchHopCostBase = MilliSatoshi(config.getLong("hop-cost-base-msat")),
|
||||
searchHopCostMillionths = config.getLong("hop-cost-millionths"),
|
||||
mppMinPartAmount = Satoshi(config.getLong("mpp.min-amount-satoshis")).toMilliSatoshi,
|
||||
mppMaxParts = config.getInt("mpp.max-parts"))
|
||||
}
|
||||
|
||||
val routerSyncEncodingType = config.getString("router.sync.encoding-type") match {
|
||||
case "uncompressed" => EncodingType.UNCOMPRESSED
|
||||
case "zlib" => EncodingType.COMPRESSED_ZLIB
|
||||
|
@ -383,23 +402,11 @@ object NodeParams extends Logging {
|
|||
channelExcludeDuration = FiniteDuration(config.getDuration("router.channel-exclude-duration").getSeconds, TimeUnit.SECONDS),
|
||||
routerBroadcastInterval = FiniteDuration(config.getDuration("router.broadcast-interval").getSeconds, TimeUnit.SECONDS),
|
||||
networkStatsRefreshInterval = FiniteDuration(config.getDuration("router.network-stats-interval").getSeconds, TimeUnit.SECONDS),
|
||||
randomizeRouteSelection = config.getBoolean("router.randomize-route-selection"),
|
||||
requestNodeAnnouncements = config.getBoolean("router.sync.request-node-announcements"),
|
||||
encodingType = routerSyncEncodingType,
|
||||
channelRangeChunkSize = config.getInt("router.sync.channel-range-chunk-size"),
|
||||
channelQueryChunkSize = config.getInt("router.sync.channel-query-chunk-size"),
|
||||
searchMaxRouteLength = config.getInt("router.path-finding.max-route-length"),
|
||||
searchMaxCltv = CltvExpiryDelta(config.getInt("router.path-finding.max-cltv")),
|
||||
searchMaxFeeBase = Satoshi(config.getLong("router.path-finding.fee-threshold-sat")),
|
||||
searchMaxFeePct = config.getDouble("router.path-finding.max-fee-pct"),
|
||||
searchRatioBase = config.getDouble("router.path-finding.ratio-base"),
|
||||
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"),
|
||||
searchHopCostBase = MilliSatoshi(config.getLong("router.path-finding.hop-cost-base-msat")),
|
||||
searchHopCostMillionths = config.getLong("router.path-finding.hop-cost-millionths"),
|
||||
mppMinPartAmount = Satoshi(config.getLong("router.path-finding.mpp.min-amount-satoshis")).toMilliSatoshi,
|
||||
mppMaxParts = config.getInt("router.path-finding.mpp.max-parts")
|
||||
pathFindingConf = getPathFindingConf(config.getConfig("router.path-finding"))
|
||||
),
|
||||
socksProxy_opt = socksProxy_opt,
|
||||
maxPaymentAttempts = config.getInt("max-payment-attempts"),
|
||||
|
|
|
@ -122,7 +122,7 @@ object NodeRelay {
|
|||
def computeRouteParams(nodeParams: NodeParams, amountIn: MilliSatoshi, expiryIn: CltvExpiry, amountOut: MilliSatoshi, expiryOut: CltvExpiry): RouteParams = {
|
||||
val routeMaxCltv = expiryIn - expiryOut
|
||||
val routeMaxFee = amountIn - amountOut
|
||||
RouteCalculation.getDefaultRouteParams(nodeParams.routerConf).copy(
|
||||
RouteCalculation.getDefaultRouteParams(nodeParams.routerConf.pathFindingConf).copy(
|
||||
maxFeeBase = routeMaxFee,
|
||||
routeMaxCltv = routeMaxCltv,
|
||||
maxFeePct = 0, // we disable percent-based max fee calculation, we're only interested in collecting our node fee
|
||||
|
@ -273,14 +273,14 @@ class NodeRelay private(nodeParams: NodeParams,
|
|||
val paymentSecret = payloadOut.paymentSecret.get // NB: we've verified that there was a payment secret in validateRelay
|
||||
if (Features(features).hasFeature(Features.BasicMultiPartPayment)) {
|
||||
context.log.debug("sending the payment to non-trampoline recipient using MPP")
|
||||
val payment = SendMultiPartPayment(payFsmAdapters, paymentSecret, payloadOut.outgoingNodeId, payloadOut.amountToForward, payloadOut.outgoingCltv, nodeParams.maxPaymentAttempts, routingHints, Some(routeParams))
|
||||
val payment = SendMultiPartPayment(payFsmAdapters, paymentSecret, payloadOut.outgoingNodeId, payloadOut.amountToForward, payloadOut.outgoingCltv, nodeParams.maxPaymentAttempts, routingHints, routeParams)
|
||||
val payFSM = outgoingPaymentFactory.spawnOutgoingPayFSM(context, paymentCfg, multiPart = true)
|
||||
payFSM ! payment
|
||||
payFSM
|
||||
} else {
|
||||
context.log.debug("sending the payment to non-trampoline recipient without MPP")
|
||||
val finalPayload = Onion.createSinglePartPayload(payloadOut.amountToForward, payloadOut.outgoingCltv, paymentSecret)
|
||||
val payment = SendPaymentToNode(payFsmAdapters, payloadOut.outgoingNodeId, finalPayload, nodeParams.maxPaymentAttempts, routingHints, Some(routeParams))
|
||||
val payment = SendPaymentToNode(payFsmAdapters, payloadOut.outgoingNodeId, finalPayload, nodeParams.maxPaymentAttempts, routingHints, routeParams)
|
||||
val payFSM = outgoingPaymentFactory.spawnOutgoingPayFSM(context, paymentCfg, multiPart = false)
|
||||
payFSM ! payment
|
||||
payFSM
|
||||
|
@ -289,7 +289,7 @@ class NodeRelay private(nodeParams: NodeParams,
|
|||
context.log.debug("sending the payment to the next trampoline node")
|
||||
val payFSM = outgoingPaymentFactory.spawnOutgoingPayFSM(context, paymentCfg, multiPart = true)
|
||||
val paymentSecret = randomBytes32() // we generate a new secret to protect against probing attacks
|
||||
val payment = SendMultiPartPayment(payFsmAdapters, paymentSecret, payloadOut.outgoingNodeId, payloadOut.amountToForward, payloadOut.outgoingCltv, nodeParams.maxPaymentAttempts, routeParams = Some(routeParams), additionalTlvs = Seq(OnionTlv.TrampolineOnion(packetOut)))
|
||||
val payment = SendMultiPartPayment(payFsmAdapters, paymentSecret, payloadOut.outgoingNodeId, payloadOut.amountToForward, payloadOut.outgoingCltv, nodeParams.maxPaymentAttempts, routeParams = routeParams, additionalTlvs = Seq(OnionTlv.TrampolineOnion(packetOut)))
|
||||
payFSM ! payment
|
||||
payFSM
|
||||
}
|
||||
|
|
|
@ -20,7 +20,7 @@ import akka.actor.{Actor, ActorLogging, ActorRef, Props}
|
|||
import fr.acinq.bitcoin.Crypto.PublicKey
|
||||
import fr.acinq.eclair.crypto.Sphinx.DecryptedFailurePacket
|
||||
import fr.acinq.eclair.payment.{PaymentEvent, PaymentFailed, PaymentRequest, RemoteFailure}
|
||||
import fr.acinq.eclair.router.{Announcements, Router}
|
||||
import fr.acinq.eclair.router.{Announcements, RouteCalculation, Router}
|
||||
import fr.acinq.eclair.wire.protocol.IncorrectOrUnknownPaymentDetails
|
||||
import fr.acinq.eclair.{MilliSatoshiLong, NodeParams, randomBytes32, randomLong}
|
||||
import scodec.bits.ByteVector
|
||||
|
@ -64,7 +64,8 @@ class Autoprobe(nodeParams: NodeParams, router: ActorRef, paymentInitiator: Acto
|
|||
),
|
||||
ByteVector.empty)
|
||||
log.info(s"sending payment probe to node=$targetNodeId payment_hash=${fakeInvoice.paymentHash}")
|
||||
paymentInitiator ! PaymentInitiator.SendPayment(PAYMENT_AMOUNT_MSAT, fakeInvoice, maxAttempts = 1)
|
||||
val routeParams = RouteCalculation.getDefaultRouteParams(nodeParams.routerConf.pathFindingConf)
|
||||
paymentInitiator ! PaymentInitiator.SendPaymentToNode(PAYMENT_AMOUNT_MSAT, fakeInvoice, maxAttempts = 1, routeParams = routeParams)
|
||||
case None =>
|
||||
log.info(s"could not find a destination, re-scheduling")
|
||||
scheduleProbe()
|
||||
|
|
|
@ -60,7 +60,7 @@ class MultiPartPaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig,
|
|||
|
||||
when(WAIT_FOR_PAYMENT_REQUEST) {
|
||||
case Event(r: SendMultiPartPayment, _) =>
|
||||
val routeParams = r.getRouteParams(nodeParams, randomize = false) // we don't randomize the first attempt, regardless of configuration choices
|
||||
val routeParams = r.routeParams.copy(randomize = false) // we don't randomize the first attempt, regardless of configuration choices
|
||||
val maxFee = routeParams.getMaxFee(r.totalAmount)
|
||||
log.debug("sending {} with maximum fee {}", r.totalAmount, maxFee)
|
||||
val d = PaymentProgress(r, r.maxAttempts, Map.empty, Ignore.empty, Nil)
|
||||
|
@ -81,7 +81,7 @@ class MultiPartPaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig,
|
|||
// If a child payment failed while we were waiting for routes, the routes we received don't cover the whole
|
||||
// remaining amount. In that case we discard these routes and send a new request to the router.
|
||||
log.info("discarding routes, another child payment failed so we need to recompute them (amount = {}, maximum fee = {})", toSend, maxFee)
|
||||
val routeParams = d.request.getRouteParams(nodeParams, randomize = true) // we randomize route selection when we retry
|
||||
val routeParams = d.request.routeParams.copy(randomize = true) // we randomize route selection when we retry
|
||||
router ! createRouteRequest(nodeParams, toSend, maxFee, routeParams, d, cfg)
|
||||
stay()
|
||||
}
|
||||
|
@ -95,7 +95,7 @@ class MultiPartPaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig,
|
|||
// a different split, they may have enough balance to forward the payment.
|
||||
val (toSend, maxFee) = remainingToSend(nodeParams, d.request, d.pending.values)
|
||||
log.debug("retry sending {} with maximum fee {} without ignoring channels ({})", toSend, maxFee, d.ignore.channels.map(_.shortChannelId).mkString(","))
|
||||
val routeParams = d.request.getRouteParams(nodeParams, randomize = true) // we randomize route selection when we retry
|
||||
val routeParams = d.request.routeParams.copy(randomize = true) // we randomize route selection when we retry
|
||||
router ! createRouteRequest(nodeParams, toSend, maxFee, routeParams, d, cfg).copy(ignore = d.ignore.emptyChannels())
|
||||
retriedFailedChannels = true
|
||||
stay() using d.copy(remainingAttempts = (d.remainingAttempts - 1).max(0), ignore = d.ignore.emptyChannels())
|
||||
|
@ -143,7 +143,7 @@ class MultiPartPaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig,
|
|||
val stillPending = d.pending - pf.id
|
||||
val (toSend, maxFee) = remainingToSend(nodeParams, d.request, stillPending.values)
|
||||
log.debug("child payment failed, retry sending {} with maximum fee {}", toSend, maxFee)
|
||||
val routeParams = d.request.getRouteParams(nodeParams, randomize = true) // we randomize route selection when we retry
|
||||
val routeParams = d.request.routeParams.copy(randomize = true) // we randomize route selection when we retry
|
||||
val d1 = d.copy(pending = stillPending, ignore = ignore1, failures = d.failures ++ pf.failures, request = d.request.copy(assistedRoutes = assistedRoutes1))
|
||||
router ! createRouteRequest(nodeParams, toSend, maxFee, routeParams, d1, cfg)
|
||||
goto(WAIT_FOR_ROUTES) using d1
|
||||
|
@ -290,13 +290,10 @@ object MultiPartPaymentLifecycle {
|
|||
targetExpiry: CltvExpiry,
|
||||
maxAttempts: Int,
|
||||
assistedRoutes: Seq[Seq[ExtraHop]] = Nil,
|
||||
routeParams: Option[RouteParams] = None,
|
||||
routeParams: RouteParams,
|
||||
additionalTlvs: Seq[OnionTlv] = Nil,
|
||||
userCustomTlvs: Seq[GenericTlv] = Nil) {
|
||||
require(totalAmount > 0.msat, s"total amount must be > 0")
|
||||
|
||||
def getRouteParams(nodeParams: NodeParams, randomize: Boolean): RouteParams =
|
||||
routeParams.getOrElse(RouteCalculation.getDefaultRouteParams(nodeParams.routerConf)).copy(randomize = randomize)
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -368,7 +365,7 @@ object MultiPartPaymentLifecycle {
|
|||
maxFee,
|
||||
d.request.assistedRoutes,
|
||||
d.ignore,
|
||||
Some(routeParams),
|
||||
routeParams,
|
||||
allowMultiPart = true,
|
||||
d.pending.values.toSeq,
|
||||
Some(cfg.paymentContext))
|
||||
|
@ -390,7 +387,7 @@ object MultiPartPaymentLifecycle {
|
|||
private def remainingToSend(nodeParams: NodeParams, request: SendMultiPartPayment, pending: Iterable[Route]): (MilliSatoshi, MilliSatoshi) = {
|
||||
val sentAmount = pending.map(_.amount).sum
|
||||
val sentFees = pending.map(_.fee).sum
|
||||
(request.totalAmount - sentAmount, request.getRouteParams(nodeParams, randomize = false).getMaxFee(request.totalAmount) - sentFees)
|
||||
(request.totalAmount - sentAmount, request.routeParams.copy(randomize = false).getMaxFee(request.totalAmount) - sentFees)
|
||||
}
|
||||
|
||||
}
|
|
@ -44,7 +44,7 @@ class PaymentInitiator(nodeParams: NodeParams, outgoingPaymentFactory: PaymentIn
|
|||
override def receive: Receive = main(Map.empty)
|
||||
|
||||
def main(pending: Map[UUID, PendingPayment]): Receive = {
|
||||
case r: SendPayment =>
|
||||
case r: SendPaymentToNode =>
|
||||
val paymentId = UUID.randomUUID()
|
||||
if (!r.blockUntilComplete) {
|
||||
// Immediately return the paymentId
|
||||
|
@ -139,7 +139,7 @@ class PaymentInitiator(nodeParams: NodeParams, outgoingPaymentFactory: PaymentIn
|
|||
val trampolineSecret = r.trampolineSecret.getOrElse(randomBytes32())
|
||||
sender() ! SendPaymentToRouteResponse(paymentId, parentPaymentId, Some(trampolineSecret))
|
||||
val payFsm = outgoingPaymentFactory.spawnOutgoingPayment(context, paymentCfg)
|
||||
val (trampolineAmount, trampolineExpiry, trampolineOnion) = buildTrampolinePayment(SendTrampolinePayment(r.recipientAmount, r.paymentRequest, trampoline, Seq((r.trampolineFees, r.trampolineExpiryDelta)), r.fallbackFinalExpiryDelta), r.trampolineFees, r.trampolineExpiryDelta)
|
||||
val (trampolineAmount, trampolineExpiry, trampolineOnion) = buildTrampolinePayment(r, trampoline, r.trampolineFees, r.trampolineExpiryDelta)
|
||||
payFsm ! PaymentLifecycle.SendPaymentToRoute(sender(), Left(r.route), Onion.createMultiPartPayload(r.amount, trampolineAmount, trampolineExpiry, trampolineSecret, Seq(OnionTlv.TrampolineOnion(trampolineOnion))), r.paymentRequest.routingInfo)
|
||||
case Nil =>
|
||||
sender() ! SendPaymentToRouteResponse(paymentId, parentPaymentId, None)
|
||||
|
@ -150,10 +150,10 @@ class PaymentInitiator(nodeParams: NodeParams, outgoingPaymentFactory: PaymentIn
|
|||
}
|
||||
}
|
||||
|
||||
private def buildTrampolinePayment(r: SendTrampolinePayment, trampolineFees: MilliSatoshi, trampolineExpiryDelta: CltvExpiryDelta): (MilliSatoshi, CltvExpiry, OnionRoutingPacket) = {
|
||||
private def buildTrampolinePayment(r: SendRequestedPayment, trampolineNodeId : PublicKey, trampolineFees: MilliSatoshi, trampolineExpiryDelta: CltvExpiryDelta): (MilliSatoshi, CltvExpiry, OnionRoutingPacket) = {
|
||||
val trampolineRoute = Seq(
|
||||
NodeHop(nodeParams.nodeId, r.trampolineNodeId, nodeParams.expiryDelta, 0 msat),
|
||||
NodeHop(r.trampolineNodeId, r.recipientNodeId, trampolineExpiryDelta, trampolineFees) // for now we only use a single trampoline hop
|
||||
NodeHop(nodeParams.nodeId, trampolineNodeId, nodeParams.expiryDelta, 0 msat),
|
||||
NodeHop(trampolineNodeId, r.recipientNodeId, trampolineExpiryDelta, trampolineFees) // for now we only use a single trampoline hop
|
||||
)
|
||||
val finalPayload = if (r.paymentRequest.features.allowMultiPart) {
|
||||
Onion.createMultiPartPayload(r.recipientAmount, r.recipientAmount, r.finalExpiry(nodeParams.currentBlockHeight), r.paymentRequest.paymentSecret.get)
|
||||
|
@ -173,7 +173,7 @@ class PaymentInitiator(nodeParams: NodeParams, outgoingPaymentFactory: PaymentIn
|
|||
val paymentCfg = SendPaymentConfig(paymentId, paymentId, None, r.paymentHash, r.recipientAmount, r.recipientNodeId, Upstream.Local(paymentId), Some(r.paymentRequest), storeInDb = true, publishEvent = false, Seq(NodeHop(r.trampolineNodeId, r.recipientNodeId, trampolineExpiryDelta, trampolineFees)))
|
||||
// We generate a random secret for this payment to avoid leaking the invoice secret to the first trampoline node.
|
||||
val trampolineSecret = randomBytes32()
|
||||
val (trampolineAmount, trampolineExpiry, trampolineOnion) = buildTrampolinePayment(r, trampolineFees, trampolineExpiryDelta)
|
||||
val (trampolineAmount, trampolineExpiry, trampolineOnion) = buildTrampolinePayment(r, r.trampolineNodeId, trampolineFees, trampolineExpiryDelta)
|
||||
val fsm = outgoingPaymentFactory.spawnOutgoingMultiPartPayment(context, paymentCfg)
|
||||
fsm ! SendMultiPartPayment(self, trampolineSecret, r.trampolineNodeId, trampolineAmount, trampolineExpiry, nodeParams.maxPaymentAttempts, r.paymentRequest.routingInfo, r.routeParams, Seq(OnionTlv.TrampolineOnion(trampolineOnion)))
|
||||
}
|
||||
|
@ -204,6 +204,17 @@ object PaymentInitiator {
|
|||
|
||||
case class PendingPayment(sender: ActorRef, remainingAttempts: Seq[(MilliSatoshi, CltvExpiryDelta)], r: SendTrampolinePayment)
|
||||
|
||||
sealed trait SendRequestedPayment{
|
||||
def recipientAmount: MilliSatoshi
|
||||
def paymentRequest: PaymentRequest
|
||||
def recipientNodeId: PublicKey = paymentRequest.nodeId
|
||||
def paymentHash: ByteVector32 = paymentRequest.paymentHash
|
||||
def fallbackFinalExpiryDelta: CltvExpiryDelta
|
||||
// We add one block in order to not have our htlcs fail when a new block has just been found.
|
||||
def finalExpiry(currentBlockHeight: Long): CltvExpiry =
|
||||
paymentRequest.minFinalCltvExpiryDelta.getOrElse(fallbackFinalExpiryDelta).toCltvExpiry(currentBlockHeight + 1)
|
||||
}
|
||||
|
||||
/**
|
||||
* We temporarily let the caller decide to use Trampoline (instead of a normal payment) and set the fees/cltv.
|
||||
* Once we have trampoline fee estimation built into the router, the decision to use Trampoline or not should be done
|
||||
|
@ -224,13 +235,7 @@ object PaymentInitiator {
|
|||
trampolineNodeId: PublicKey,
|
||||
trampolineAttempts: Seq[(MilliSatoshi, CltvExpiryDelta)],
|
||||
fallbackFinalExpiryDelta: CltvExpiryDelta = Channel.MIN_CLTV_EXPIRY_DELTA,
|
||||
routeParams: Option[RouteParams] = None) {
|
||||
val recipientNodeId = paymentRequest.nodeId
|
||||
val paymentHash = paymentRequest.paymentHash
|
||||
|
||||
// We add one block in order to not have our htlcs fail when a new block has just been found.
|
||||
def finalExpiry(currentBlockHeight: Long) = paymentRequest.minFinalCltvExpiryDelta.getOrElse(fallbackFinalExpiryDelta).toCltvExpiry(currentBlockHeight + 1)
|
||||
}
|
||||
routeParams: RouteParams) extends SendRequestedPayment
|
||||
|
||||
/**
|
||||
* @param recipientAmount amount that should be received by the final recipient (usually from a Bolt 11 invoice).
|
||||
|
@ -243,21 +248,15 @@ object PaymentInitiator {
|
|||
* @param userCustomTlvs (optional) user-defined custom tlvs that will be added to the onion sent to the target node.
|
||||
* @param blockUntilComplete (optional) if true, wait until the payment completes before returning a result.
|
||||
*/
|
||||
case class SendPayment(recipientAmount: MilliSatoshi,
|
||||
paymentRequest: PaymentRequest,
|
||||
maxAttempts: Int,
|
||||
fallbackFinalExpiryDelta: CltvExpiryDelta = Channel.MIN_CLTV_EXPIRY_DELTA,
|
||||
externalId: Option[String] = None,
|
||||
assistedRoutes: Seq[Seq[ExtraHop]] = Nil,
|
||||
routeParams: Option[RouteParams] = None,
|
||||
userCustomTlvs: Seq[GenericTlv] = Nil,
|
||||
blockUntilComplete: Boolean = false) {
|
||||
val recipientNodeId = paymentRequest.nodeId
|
||||
val paymentHash = paymentRequest.paymentHash
|
||||
|
||||
// We add one block in order to not have our htlcs fail when a new block has just been found.
|
||||
def finalExpiry(currentBlockHeight: Long) = paymentRequest.minFinalCltvExpiryDelta.getOrElse(fallbackFinalExpiryDelta).toCltvExpiry(currentBlockHeight + 1)
|
||||
}
|
||||
case class SendPaymentToNode(recipientAmount: MilliSatoshi,
|
||||
paymentRequest: PaymentRequest,
|
||||
maxAttempts: Int,
|
||||
fallbackFinalExpiryDelta: CltvExpiryDelta = Channel.MIN_CLTV_EXPIRY_DELTA,
|
||||
externalId: Option[String] = None,
|
||||
assistedRoutes: Seq[Seq[ExtraHop]] = Nil,
|
||||
routeParams: RouteParams,
|
||||
userCustomTlvs: Seq[GenericTlv] = Nil,
|
||||
blockUntilComplete: Boolean = false) extends SendRequestedPayment
|
||||
|
||||
/**
|
||||
* @param recipientAmount amount that should be received by the final recipient.
|
||||
|
@ -273,7 +272,7 @@ object PaymentInitiator {
|
|||
paymentPreimage: ByteVector32,
|
||||
maxAttempts: Int,
|
||||
externalId: Option[String] = None,
|
||||
routeParams: Option[RouteParams] = None,
|
||||
routeParams: RouteParams,
|
||||
userCustomTlvs: Seq[GenericTlv] = Nil) {
|
||||
val paymentHash = Crypto.sha256(paymentPreimage)
|
||||
}
|
||||
|
@ -322,13 +321,7 @@ object PaymentInitiator {
|
|||
trampolineSecret: Option[ByteVector32],
|
||||
trampolineFees: MilliSatoshi,
|
||||
trampolineExpiryDelta: CltvExpiryDelta,
|
||||
trampolineNodes: Seq[PublicKey]) {
|
||||
val recipientNodeId = paymentRequest.nodeId
|
||||
val paymentHash = paymentRequest.paymentHash
|
||||
|
||||
// We add one block in order to not have our htlcs fail when a new block has just been found.
|
||||
def finalExpiry(currentBlockHeight: Long) = paymentRequest.minFinalCltvExpiryDelta.getOrElse(fallbackFinalExpiryDelta).toCltvExpiry(currentBlockHeight + 1)
|
||||
}
|
||||
trampolineNodes: Seq[PublicKey]) extends SendRequestedPayment
|
||||
|
||||
/**
|
||||
* @param paymentId id of the outgoing payment (mapped to a single outgoing HTLC).
|
||||
|
|
|
@ -65,7 +65,7 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A
|
|||
|
||||
case Event(c: SendPaymentToNode, WaitingForRequest) =>
|
||||
log.debug("sending {} to {}", c.finalPayload.amount, c.targetNodeId)
|
||||
router ! RouteRequest(nodeParams.nodeId, c.targetNodeId, c.finalPayload.amount, c.getMaxFee(nodeParams), c.assistedRoutes, routeParams = c.routeParams, paymentContext = Some(cfg.paymentContext))
|
||||
router ! RouteRequest(nodeParams.nodeId, c.targetNodeId, c.finalPayload.amount, c.maxFee, c.assistedRoutes, routeParams = c.routeParams, paymentContext = Some(cfg.paymentContext))
|
||||
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))
|
||||
}
|
||||
|
@ -132,7 +132,7 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A
|
|||
data.c match {
|
||||
case (sendPaymentToNode: SendPaymentToNode) =>
|
||||
val ignore1 = PaymentFailure.updateIgnored(failure, data.ignore)
|
||||
router ! RouteRequest(nodeParams.nodeId, data.c.targetNodeId, data.c.finalPayload.amount, sendPaymentToNode.getMaxFee(nodeParams), data.c.assistedRoutes, ignore1, sendPaymentToNode.routeParams, paymentContext = Some(cfg.paymentContext))
|
||||
router ! RouteRequest(nodeParams.nodeId, data.c.targetNodeId, data.c.finalPayload.amount, sendPaymentToNode.maxFee, data.c.assistedRoutes, ignore1, sendPaymentToNode.routeParams, paymentContext = Some(cfg.paymentContext))
|
||||
goto(WAITING_FOR_ROUTE) using WaitingForRoute(data.c, data.failures :+ failure, ignore1)
|
||||
case (_: SendPaymentToRoute) =>
|
||||
log.error("unexpected retry during SendPaymentToRoute")
|
||||
|
@ -212,7 +212,7 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A
|
|||
log.error("unexpected retry during SendPaymentToRoute")
|
||||
stop(FSM.Normal)
|
||||
case (c: SendPaymentToNode) =>
|
||||
router ! RouteRequest(nodeParams.nodeId, c.targetNodeId, c.finalPayload.amount, c.getMaxFee(nodeParams), assistedRoutes1, ignore1, c.routeParams, paymentContext = Some(cfg.paymentContext))
|
||||
router ! RouteRequest(nodeParams.nodeId, c.targetNodeId, c.finalPayload.amount, c.maxFee, assistedRoutes1, ignore1, c.routeParams, paymentContext = Some(cfg.paymentContext))
|
||||
goto(WAITING_FOR_ROUTE) using WaitingForRoute(c, failures :+ failure, ignore1)
|
||||
}
|
||||
} else {
|
||||
|
@ -223,7 +223,7 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A
|
|||
log.error("unexpected retry during SendPaymentToRoute")
|
||||
stop(FSM.Normal)
|
||||
case (c: SendPaymentToNode) =>
|
||||
router ! RouteRequest(nodeParams.nodeId, c.targetNodeId, c.finalPayload.amount, c.getMaxFee(nodeParams), c.assistedRoutes, ignore + nodeId, c.routeParams, paymentContext = Some(cfg.paymentContext))
|
||||
router ! RouteRequest(nodeParams.nodeId, c.targetNodeId, c.finalPayload.amount, c.maxFee, c.assistedRoutes, ignore + nodeId, c.routeParams, paymentContext = Some(cfg.paymentContext))
|
||||
goto(WAITING_FOR_ROUTE) using WaitingForRoute(c, failures :+ failure, ignore + nodeId)
|
||||
}
|
||||
}
|
||||
|
@ -363,11 +363,11 @@ object PaymentLifecycle {
|
|||
finalPayload: FinalPayload,
|
||||
maxAttempts: Int,
|
||||
assistedRoutes: Seq[Seq[ExtraHop]] = Nil,
|
||||
routeParams: Option[RouteParams] = None) extends SendPayment {
|
||||
routeParams: RouteParams) extends SendPayment {
|
||||
require(finalPayload.amount > 0.msat, s"amount must be > 0")
|
||||
|
||||
def getMaxFee(nodeParams: NodeParams): MilliSatoshi =
|
||||
routeParams.getOrElse(RouteCalculation.getDefaultRouteParams(nodeParams.routerConf)).getMaxFee(finalPayload.amount)
|
||||
val maxFee: MilliSatoshi = routeParams.getMaxFee(finalPayload.amount)
|
||||
|
||||
}
|
||||
|
||||
// @formatter:off
|
||||
|
|
|
@ -23,7 +23,7 @@ import fr.acinq.eclair.crypto.TransportHandler
|
|||
import fr.acinq.eclair.io.Peer.PeerRoutingMessage
|
||||
import fr.acinq.eclair.io.Switchboard.RouterPeerConf
|
||||
import fr.acinq.eclair.io.{ClientSpawner, Peer, PeerConnection, Switchboard}
|
||||
import fr.acinq.eclair.router.Router.{GossipDecision, RouterConf, SendChannelQuery}
|
||||
import fr.acinq.eclair.router.Router.{GossipDecision, PathFindingConf, RouterConf, SendChannelQuery}
|
||||
import fr.acinq.eclair.router._
|
||||
import fr.acinq.eclair.wire.protocol.CommonCodecs._
|
||||
import fr.acinq.eclair.wire.protocol.LightningMessageCodecs._
|
||||
|
@ -46,17 +46,8 @@ object EclairInternalsSerializer {
|
|||
|
||||
def iterable[A](codec: Codec[A]): Codec[Iterable[A]] = listOfN(uint16, codec).xmap(_.toIterable, _.toList)
|
||||
|
||||
val routerConfCodec: Codec[RouterConf] = (
|
||||
val pathFindingConfCodec: Codec[PathFindingConf] = (
|
||||
("randomizeRouteSelection" | bool(8)) ::
|
||||
("channelExcludeDuration" | finiteDurationCodec) ::
|
||||
("routerBroadcastInterval" | finiteDurationCodec) ::
|
||||
("networkStatsRefreshInterval" | finiteDurationCodec) ::
|
||||
("requestNodeAnnouncements" | bool(8)) ::
|
||||
("encodingType" | discriminated[EncodingType].by(uint8)
|
||||
.typecase(0, provide(EncodingType.UNCOMPRESSED))
|
||||
.typecase(1, provide(EncodingType.COMPRESSED_ZLIB))) ::
|
||||
("channelRangeChunkSize" | int32) ::
|
||||
("channelQueryChunkSize" | int32) ::
|
||||
("searchMaxFeeBase" | satoshi) ::
|
||||
("searchMaxFeePct" | double) ::
|
||||
("searchMaxRouteLength" | int32) ::
|
||||
|
@ -68,7 +59,19 @@ object EclairInternalsSerializer {
|
|||
("searchHopCostBase" | millisatoshi) ::
|
||||
("searchHopCostMillionths" | int64) ::
|
||||
("mppMinPartAmount" | millisatoshi) ::
|
||||
("mppMaxParts" | int32)).as[RouterConf]
|
||||
("mppMaxParts" | int32)).as[PathFindingConf]
|
||||
|
||||
val routerConfCodec: Codec[RouterConf] = (
|
||||
("channelExcludeDuration" | finiteDurationCodec) ::
|
||||
("routerBroadcastInterval" | finiteDurationCodec) ::
|
||||
("networkStatsRefreshInterval" | finiteDurationCodec) ::
|
||||
("requestNodeAnnouncements" | bool(8)) ::
|
||||
("encodingType" | discriminated[EncodingType].by(uint8)
|
||||
.typecase(0, provide(EncodingType.UNCOMPRESSED))
|
||||
.typecase(1, provide(EncodingType.COMPRESSED_ZLIB))) ::
|
||||
("channelRangeChunkSize" | int32) ::
|
||||
("channelQueryChunkSize" | int32) ::
|
||||
("pathFindingConf" | pathFindingConfCodec)).as[RouterConf]
|
||||
|
||||
val overrideFeaturesListCodec: Codec[List[(PublicKey, Features)]] = listOfN(uint16, publicKey ~ variableSizeBytes(uint16, featuresCodec))
|
||||
|
||||
|
|
|
@ -105,7 +105,7 @@ object RouteCalculation {
|
|||
GraphEdge(ChannelDesc(ac.extraHop.shortChannelId, ac.extraHop.nodeId, ac.nextNodeId), toFakeUpdate(ac.extraHop, ac.htlcMaximum), htlcMaxToCapacity(ac.htlcMaximum), Some(ac.htlcMaximum))
|
||||
).toSet
|
||||
val ignoredEdges = r.ignore.channels ++ d.excludedChannels
|
||||
val params = r.routeParams.getOrElse(getDefaultRouteParams(routerConf))
|
||||
val params = r.routeParams
|
||||
val routesToFind = if (params.randomize) DEFAULT_ROUTES_COUNT else 1
|
||||
|
||||
log.info(s"finding routes ${r.source}->${r.target} with assistedChannels={} ignoreNodes={} ignoreChannels={} excludedChannels={}", assistedChannels.keys.mkString(","), r.ignore.nodes.map(_.value).mkString(","), r.ignore.channels.mkString(","), d.excludedChannels.mkString(","))
|
||||
|
@ -183,21 +183,21 @@ object RouteCalculation {
|
|||
/** The default number of routes we'll search for when findRoute is called with randomize = true */
|
||||
val DEFAULT_ROUTES_COUNT = 3
|
||||
|
||||
def getDefaultRouteParams(routerConf: RouterConf): RouteParams = RouteParams(
|
||||
randomize = routerConf.randomizeRouteSelection,
|
||||
maxFeeBase = routerConf.searchMaxFeeBase.toMilliSatoshi,
|
||||
maxFeePct = routerConf.searchMaxFeePct,
|
||||
routeMaxLength = routerConf.searchMaxRouteLength,
|
||||
routeMaxCltv = routerConf.searchMaxCltv,
|
||||
def getDefaultRouteParams(pathFindingConf: PathFindingConf): RouteParams = RouteParams(
|
||||
randomize = pathFindingConf.randomizeRouteSelection,
|
||||
maxFeeBase = pathFindingConf.searchMaxFeeBase.toMilliSatoshi,
|
||||
maxFeePct = pathFindingConf.searchMaxFeePct,
|
||||
routeMaxLength = pathFindingConf.searchMaxRouteLength,
|
||||
routeMaxCltv = pathFindingConf.searchMaxCltv,
|
||||
ratios = WeightRatios(
|
||||
baseFactor = routerConf.searchRatioBase,
|
||||
cltvDeltaFactor = routerConf.searchRatioCltv,
|
||||
ageFactor = routerConf.searchRatioChannelAge,
|
||||
capacityFactor = routerConf.searchRatioChannelCapacity,
|
||||
hopCostBase = routerConf.searchHopCostBase,
|
||||
hopCostMillionths = routerConf.searchHopCostMillionths
|
||||
baseFactor = pathFindingConf.searchRatioBase,
|
||||
cltvDeltaFactor = pathFindingConf.searchRatioCltv,
|
||||
ageFactor = pathFindingConf.searchRatioChannelAge,
|
||||
capacityFactor = pathFindingConf.searchRatioChannelCapacity,
|
||||
hopCostBase = pathFindingConf.searchHopCostBase,
|
||||
hopCostMillionths = pathFindingConf.searchHopCostMillionths
|
||||
),
|
||||
mpp = MultiPartParams(routerConf.mppMinPartAmount, routerConf.mppMaxParts),
|
||||
mpp = MultiPartParams(pathFindingConf.mppMinPartAmount, pathFindingConf.mppMaxParts),
|
||||
includeLocalChannelCost = false,
|
||||
)
|
||||
|
||||
|
|
|
@ -297,26 +297,19 @@ object Router {
|
|||
|
||||
def props(nodeParams: NodeParams, watcher: typed.ActorRef[ZmqWatcher.Command], initialized: Option[Promise[Done]] = None) = Props(new Router(nodeParams, watcher, initialized))
|
||||
|
||||
case class RouterConf(randomizeRouteSelection: Boolean,
|
||||
channelExcludeDuration: FiniteDuration,
|
||||
routerBroadcastInterval: FiniteDuration,
|
||||
networkStatsRefreshInterval: FiniteDuration,
|
||||
requestNodeAnnouncements: Boolean,
|
||||
encodingType: EncodingType,
|
||||
channelRangeChunkSize: Int,
|
||||
channelQueryChunkSize: Int,
|
||||
searchMaxFeeBase: Satoshi,
|
||||
searchMaxFeePct: Double,
|
||||
searchMaxRouteLength: Int,
|
||||
searchMaxCltv: CltvExpiryDelta,
|
||||
searchRatioBase: Double,
|
||||
searchRatioCltv: Double,
|
||||
searchRatioChannelAge: Double,
|
||||
searchRatioChannelCapacity: Double,
|
||||
searchHopCostBase: MilliSatoshi,
|
||||
searchHopCostMillionths: Long,
|
||||
mppMinPartAmount: MilliSatoshi,
|
||||
mppMaxParts: Int) {
|
||||
case class PathFindingConf(randomizeRouteSelection: Boolean,
|
||||
searchMaxFeeBase: Satoshi,
|
||||
searchMaxFeePct: Double,
|
||||
searchMaxRouteLength: Int,
|
||||
searchMaxCltv: CltvExpiryDelta,
|
||||
searchRatioBase: Double,
|
||||
searchRatioCltv: Double,
|
||||
searchRatioChannelAge: Double,
|
||||
searchRatioChannelCapacity: Double,
|
||||
searchHopCostBase: MilliSatoshi,
|
||||
searchHopCostMillionths: Long,
|
||||
mppMinPartAmount: MilliSatoshi,
|
||||
mppMaxParts: Int) {
|
||||
require(searchRatioBase >= 0.0, "ratio-base must be nonnegative")
|
||||
require(searchRatioCltv >= 0.0, "ratio-cltv must be nonnegative")
|
||||
require(searchRatioChannelAge >= 0.0, "ratio-channel-age must be nonnegative")
|
||||
|
@ -326,6 +319,15 @@ object Router {
|
|||
require(searchHopCostMillionths >= 0.0, "hop-cost-millionths must be nonnegative")
|
||||
}
|
||||
|
||||
case class RouterConf(channelExcludeDuration: FiniteDuration,
|
||||
routerBroadcastInterval: FiniteDuration,
|
||||
networkStatsRefreshInterval: FiniteDuration,
|
||||
requestNodeAnnouncements: Boolean,
|
||||
encodingType: EncodingType,
|
||||
channelRangeChunkSize: Int,
|
||||
channelQueryChunkSize: Int,
|
||||
pathFindingConf: PathFindingConf)
|
||||
|
||||
// @formatter:off
|
||||
case class ChannelDesc(shortChannelId: ShortChannelId, a: PublicKey, b: PublicKey)
|
||||
case class ChannelMeta(balance1: MilliSatoshi, balance2: MilliSatoshi)
|
||||
|
@ -469,7 +471,7 @@ object Router {
|
|||
maxFee: MilliSatoshi,
|
||||
assistedRoutes: Seq[Seq[ExtraHop]] = Nil,
|
||||
ignore: Ignore = Ignore.empty,
|
||||
routeParams: Option[RouteParams] = None,
|
||||
routeParams: RouteParams,
|
||||
allowMultiPart: Boolean = false,
|
||||
pendingPayments: Seq[Route] = Nil,
|
||||
paymentContext: Option[PaymentContext] = None)
|
||||
|
|
|
@ -33,7 +33,7 @@ import fr.acinq.eclair.payment.PaymentRequest.ExtraHop
|
|||
import fr.acinq.eclair.payment.receive.MultiPartHandler.ReceivePayment
|
||||
import fr.acinq.eclair.payment.receive.PaymentHandler
|
||||
import fr.acinq.eclair.payment.relay.Relayer.RelayFees
|
||||
import fr.acinq.eclair.payment.send.PaymentInitiator.{SendPayment, SendPaymentToRoute, SendSpontaneousPayment}
|
||||
import fr.acinq.eclair.payment.send.PaymentInitiator.{SendPaymentToNode, SendPaymentToRoute, SendSpontaneousPayment}
|
||||
import fr.acinq.eclair.router.RouteCalculationSpec.makeUpdateShort
|
||||
import fr.acinq.eclair.router.Router.{GetNetworkStats, GetNetworkStatsResponse, PredefinedNodeRoute, PublicChannel}
|
||||
import fr.acinq.eclair.router.{Announcements, NetworkStats, Router, Stats}
|
||||
|
@ -108,7 +108,7 @@ class EclairImplSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with I
|
|||
val nodePrivKey = randomKey()
|
||||
val invoice0 = PaymentRequest(Block.RegtestGenesisBlock.hash, Some(123 msat), ByteVector32.Zeroes, nodePrivKey, "description", CltvExpiryDelta(18))
|
||||
eclair.send(None, 123 msat, invoice0)
|
||||
val send = paymentInitiator.expectMsgType[SendPayment]
|
||||
val send = paymentInitiator.expectMsgType[SendPaymentToNode]
|
||||
assert(send.externalId === None)
|
||||
assert(send.recipientNodeId === nodePrivKey.publicKey)
|
||||
assert(send.recipientAmount === 123.msat)
|
||||
|
@ -121,7 +121,7 @@ class EclairImplSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with I
|
|||
val hints = List(List(ExtraHop(Bob.nodeParams.nodeId, ShortChannelId("569178x2331x1"), feeBase = 10 msat, feeProportionalMillionths = 1, cltvExpiryDelta = CltvExpiryDelta(12))))
|
||||
val invoice1 = PaymentRequest(Block.RegtestGenesisBlock.hash, Some(123 msat), ByteVector32.Zeroes, nodePrivKey, "description", CltvExpiryDelta(18), None, None, hints)
|
||||
eclair.send(Some(externalId1), 123 msat, invoice1)
|
||||
val send1 = paymentInitiator.expectMsgType[SendPayment]
|
||||
val send1 = paymentInitiator.expectMsgType[SendPaymentToNode]
|
||||
assert(send1.externalId === Some(externalId1))
|
||||
assert(send1.recipientNodeId === nodePrivKey.publicKey)
|
||||
assert(send1.recipientAmount === 123.msat)
|
||||
|
@ -133,7 +133,7 @@ class EclairImplSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with I
|
|||
val externalId2 = "487da196-a4dc-4b1e-92b4-3e5e905e9f3f"
|
||||
val invoice2 = PaymentRequest("lntb", Some(123 msat), System.currentTimeMillis() / 1000L, nodePrivKey.publicKey, List(PaymentRequest.MinFinalCltvExpiry(96), PaymentRequest.PaymentHash(ByteVector32.Zeroes), PaymentRequest.Description("description")), ByteVector.empty)
|
||||
eclair.send(Some(externalId2), 123 msat, invoice2)
|
||||
val send2 = paymentInitiator.expectMsgType[SendPayment]
|
||||
val send2 = paymentInitiator.expectMsgType[SendPaymentToNode]
|
||||
assert(send2.externalId === Some(externalId2))
|
||||
assert(send2.recipientNodeId === nodePrivKey.publicKey)
|
||||
assert(send2.recipientAmount === 123.msat)
|
||||
|
@ -143,13 +143,13 @@ class EclairImplSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with I
|
|||
|
||||
// with custom route fees parameters
|
||||
eclair.send(None, 123 msat, invoice0, feeThreshold_opt = Some(123 sat), maxFeePct_opt = Some(4.20))
|
||||
val send3 = paymentInitiator.expectMsgType[SendPayment]
|
||||
val send3 = paymentInitiator.expectMsgType[SendPaymentToNode]
|
||||
assert(send3.externalId === None)
|
||||
assert(send3.recipientNodeId === nodePrivKey.publicKey)
|
||||
assert(send3.recipientAmount === 123.msat)
|
||||
assert(send3.paymentHash === ByteVector32.Zeroes)
|
||||
assert(send3.routeParams.get.maxFeeBase === 123000.msat) // conversion sat -> msat
|
||||
assert(send3.routeParams.get.maxFeePct === 4.20)
|
||||
assert(send3.routeParams.maxFeeBase === 123000.msat) // conversion sat -> msat
|
||||
assert(send3.routeParams.maxFeePct === 4.20)
|
||||
|
||||
val invalidExternalId = "Robert'); DROP TABLE received_payments; DROP TABLE sent_payments; DROP TABLE payments;"
|
||||
assertThrows[IllegalArgumentException](Await.result(eclair.send(Some(invalidExternalId), 123 msat, invoice0), 50 millis))
|
||||
|
|
|
@ -19,12 +19,12 @@ package fr.acinq.eclair
|
|||
import fr.acinq.bitcoin.{Block, ByteVector32, Satoshi, SatoshiLong, Script}
|
||||
import fr.acinq.eclair.FeatureSupport.{Mandatory, Optional}
|
||||
import fr.acinq.eclair.Features._
|
||||
import fr.acinq.eclair.blockchain.fee.{FeeEstimator, FeeTargets, FeeratesPerKw, OnChainFeeConf, _}
|
||||
import fr.acinq.eclair.blockchain.fee._
|
||||
import fr.acinq.eclair.channel.LocalParams
|
||||
import fr.acinq.eclair.crypto.keymanager.{LocalChannelKeyManager, LocalNodeKeyManager}
|
||||
import fr.acinq.eclair.io.{Peer, PeerConnection}
|
||||
import fr.acinq.eclair.payment.relay.Relayer.{RelayFees, RelayParams}
|
||||
import fr.acinq.eclair.router.Router.RouterConf
|
||||
import fr.acinq.eclair.router.Router.{PathFindingConf, RouterConf}
|
||||
import fr.acinq.eclair.wire.protocol.{Color, EncodingType, NodeAddress, OnionRoutingPacket}
|
||||
import org.scalatest.Tag
|
||||
import scodec.bits.ByteVector
|
||||
|
@ -151,7 +151,6 @@ object TestConstants {
|
|||
maxRebroadcastDelay = 5 seconds
|
||||
),
|
||||
routerConf = RouterConf(
|
||||
randomizeRouteSelection = false,
|
||||
channelExcludeDuration = 60 seconds,
|
||||
routerBroadcastInterval = 5 seconds,
|
||||
networkStatsRefreshInterval = 1 hour,
|
||||
|
@ -159,18 +158,20 @@ object TestConstants {
|
|||
encodingType = EncodingType.COMPRESSED_ZLIB,
|
||||
channelRangeChunkSize = 20,
|
||||
channelQueryChunkSize = 5,
|
||||
searchMaxFeeBase = 21 sat,
|
||||
searchMaxFeePct = 0.03,
|
||||
searchMaxCltv = CltvExpiryDelta(2016),
|
||||
searchMaxRouteLength = 20,
|
||||
searchRatioBase = 1.0,
|
||||
searchRatioCltv = 0.0,
|
||||
searchRatioChannelAge = 0.0,
|
||||
searchRatioChannelCapacity = 0.0,
|
||||
searchHopCostBase = 0 msat,
|
||||
searchHopCostMillionths = 0,
|
||||
mppMinPartAmount = 15000000 msat,
|
||||
mppMaxParts = 10
|
||||
pathFindingConf = PathFindingConf(
|
||||
randomizeRouteSelection = false,
|
||||
searchMaxFeeBase = 21 sat,
|
||||
searchMaxFeePct = 0.03,
|
||||
searchMaxCltv = CltvExpiryDelta(2016),
|
||||
searchMaxRouteLength = 20,
|
||||
searchRatioBase = 1.0,
|
||||
searchRatioCltv = 0.0,
|
||||
searchRatioChannelAge = 0.0,
|
||||
searchRatioChannelCapacity = 0.0,
|
||||
searchHopCostBase = 0 msat,
|
||||
searchHopCostMillionths = 0,
|
||||
mppMinPartAmount = 15000000 msat,
|
||||
mppMaxParts = 10)
|
||||
),
|
||||
socksProxy_opt = None,
|
||||
maxPaymentAttempts = 5,
|
||||
|
@ -268,7 +269,6 @@ object TestConstants {
|
|||
maxRebroadcastDelay = 5 seconds
|
||||
),
|
||||
routerConf = RouterConf(
|
||||
randomizeRouteSelection = false,
|
||||
channelExcludeDuration = 60 seconds,
|
||||
routerBroadcastInterval = 5 seconds,
|
||||
networkStatsRefreshInterval = 1 hour,
|
||||
|
@ -276,18 +276,20 @@ object TestConstants {
|
|||
encodingType = EncodingType.UNCOMPRESSED,
|
||||
channelRangeChunkSize = 20,
|
||||
channelQueryChunkSize = 5,
|
||||
searchMaxFeeBase = 21 sat,
|
||||
searchMaxFeePct = 0.03,
|
||||
searchMaxCltv = CltvExpiryDelta(2016),
|
||||
searchMaxRouteLength = 20,
|
||||
searchRatioBase = 1.0,
|
||||
searchRatioCltv = 0.0,
|
||||
searchRatioChannelAge = 0.0,
|
||||
searchRatioChannelCapacity = 0.0,
|
||||
searchHopCostBase = 0 msat,
|
||||
searchHopCostMillionths = 0,
|
||||
mppMinPartAmount = 15000000 msat,
|
||||
mppMaxParts = 10
|
||||
pathFindingConf = PathFindingConf(
|
||||
randomizeRouteSelection = false,
|
||||
searchMaxFeeBase = 21 sat,
|
||||
searchMaxFeePct = 0.03,
|
||||
searchMaxCltv = CltvExpiryDelta(2016),
|
||||
searchMaxRouteLength = 20,
|
||||
searchRatioBase = 1.0,
|
||||
searchRatioCltv = 0.0,
|
||||
searchRatioChannelAge = 0.0,
|
||||
searchRatioChannelCapacity = 0.0,
|
||||
searchHopCostBase = 0 msat,
|
||||
searchHopCostMillionths = 0,
|
||||
mppMinPartAmount = 15000000 msat,
|
||||
mppMaxParts = 10)
|
||||
),
|
||||
socksProxy_opt = None,
|
||||
maxPaymentAttempts = 5,
|
||||
|
|
|
@ -32,7 +32,7 @@ import fr.acinq.eclair.payment._
|
|||
import fr.acinq.eclair.payment.receive.MultiPartHandler.ReceivePayment
|
||||
import fr.acinq.eclair.payment.receive.{ForwardHandler, PaymentHandler}
|
||||
import fr.acinq.eclair.payment.send.MultiPartPaymentLifecycle.PreimageReceived
|
||||
import fr.acinq.eclair.payment.send.PaymentInitiator.SendPayment
|
||||
import fr.acinq.eclair.payment.send.PaymentInitiator.SendPaymentToNode
|
||||
import fr.acinq.eclair.router.Router
|
||||
import fr.acinq.eclair.transactions.Transactions.TxOwner
|
||||
import fr.acinq.eclair.transactions.{Scripts, Transactions}
|
||||
|
@ -146,7 +146,7 @@ abstract class ChannelIntegrationSpec extends IntegrationSpec {
|
|||
val preimage = randomBytes32()
|
||||
val paymentHash = Crypto.sha256(preimage)
|
||||
// A sends a payment to F
|
||||
val paymentReq = SendPayment(100000000 msat, PaymentRequest(Block.RegtestGenesisBlock.hash, None, paymentHash, nodes("F").nodeParams.privateKey, "test", finalCltvExpiryDelta), maxAttempts = 1, routeParams = integrationTestRouteParams)
|
||||
val paymentReq = SendPaymentToNode(100000000 msat, PaymentRequest(Block.RegtestGenesisBlock.hash, None, paymentHash, nodes("F").nodeParams.privateKey, "test", finalCltvExpiryDelta), maxAttempts = 1, routeParams = integrationTestRouteParams)
|
||||
val paymentSender = TestProbe()
|
||||
paymentSender.send(nodes("A").paymentInitiator, paymentReq)
|
||||
val paymentId = paymentSender.expectMsgType[UUID]
|
||||
|
@ -369,7 +369,7 @@ abstract class ChannelIntegrationSpec extends IntegrationSpec {
|
|||
def send(amountMsat: MilliSatoshi, paymentHandler: ActorRef, paymentInitiator: ActorRef): UUID = {
|
||||
sender.send(paymentHandler, ReceivePayment(Some(amountMsat), "1 coffee"))
|
||||
val pr = sender.expectMsgType[PaymentRequest]
|
||||
val sendReq = SendPayment(amountMsat, pr, maxAttempts = 1, fallbackFinalExpiryDelta = finalCltvExpiryDelta, routeParams = integrationTestRouteParams)
|
||||
val sendReq = SendPaymentToNode(amountMsat, pr, maxAttempts = 1, fallbackFinalExpiryDelta = finalCltvExpiryDelta, routeParams = integrationTestRouteParams)
|
||||
sender.send(paymentInitiator, sendReq)
|
||||
sender.expectMsgType[UUID]
|
||||
}
|
||||
|
@ -685,7 +685,7 @@ class AnchorOutputChannelIntegrationSpec extends ChannelIntegrationSpec {
|
|||
val pr = sender.expectMsgType[PaymentRequest]
|
||||
|
||||
// then we make the actual payment
|
||||
sender.send(nodes("C").paymentInitiator, SendPayment(amountMsat, pr, maxAttempts = 1, fallbackFinalExpiryDelta = finalCltvExpiryDelta))
|
||||
sender.send(nodes("C").paymentInitiator, SendPaymentToNode(amountMsat, pr, maxAttempts = 1, fallbackFinalExpiryDelta = finalCltvExpiryDelta, routeParams = integrationTestRouteParams))
|
||||
val paymentId = sender.expectMsgType[UUID]
|
||||
val preimage = sender.expectMsgType[PreimageReceived].paymentPreimage
|
||||
assert(Crypto.sha256(preimage) === pr.paymentHash)
|
||||
|
|
|
@ -49,7 +49,7 @@ abstract class IntegrationSpec extends TestKitBaseClass with BitcoindService wit
|
|||
var nodes: Map[String, Kit] = Map()
|
||||
|
||||
// we override the default because these test were designed to use cost-optimized routes
|
||||
val integrationTestRouteParams = Some(RouteParams(
|
||||
val integrationTestRouteParams = RouteParams(
|
||||
randomize = false,
|
||||
maxFeeBase = 21000 msat,
|
||||
maxFeePct = 0.03,
|
||||
|
@ -65,7 +65,7 @@ abstract class IntegrationSpec extends TestKitBaseClass with BitcoindService wit
|
|||
),
|
||||
mpp = MultiPartParams(15000000 msat, 6),
|
||||
includeLocalChannelCost = false,
|
||||
))
|
||||
)
|
||||
|
||||
// we need to provide a value higher than every node's fulfill-safety-before-timeout
|
||||
val finalCltvExpiryDelta = CltvExpiryDelta(36)
|
||||
|
|
|
@ -37,7 +37,7 @@ import fr.acinq.eclair.payment._
|
|||
import fr.acinq.eclair.payment.receive.MultiPartHandler.ReceivePayment
|
||||
import fr.acinq.eclair.payment.relay.Relayer
|
||||
import fr.acinq.eclair.payment.send.MultiPartPaymentLifecycle.PreimageReceived
|
||||
import fr.acinq.eclair.payment.send.PaymentInitiator.{SendPayment, SendTrampolinePayment}
|
||||
import fr.acinq.eclair.payment.send.PaymentInitiator.{SendPaymentToNode, SendTrampolinePayment}
|
||||
import fr.acinq.eclair.router.Graph.WeightRatios
|
||||
import fr.acinq.eclair.router.Router.{GossipDecision, PublicChannel}
|
||||
import fr.acinq.eclair.router.{Announcements, AnnouncementsBatchValidationSpec, Router}
|
||||
|
@ -159,7 +159,7 @@ class PaymentIntegrationSpec extends IntegrationSpec {
|
|||
sender.send(nodes("D").paymentHandler, ReceivePayment(Some(amountMsat), "1 coffee"))
|
||||
val pr = sender.expectMsgType[PaymentRequest]
|
||||
// then we make the actual payment
|
||||
sender.send(nodes("A").paymentInitiator, SendPayment(amountMsat, pr, fallbackFinalExpiryDelta = finalCltvExpiryDelta, routeParams = integrationTestRouteParams, maxAttempts = 1))
|
||||
sender.send(nodes("A").paymentInitiator, SendPaymentToNode(amountMsat, pr, fallbackFinalExpiryDelta = finalCltvExpiryDelta, routeParams = integrationTestRouteParams, maxAttempts = 1))
|
||||
val paymentId = sender.expectMsgType[UUID]
|
||||
val preimage = sender.expectMsgType[PreimageReceived].paymentPreimage
|
||||
assert(Crypto.sha256(preimage) === pr.paymentHash)
|
||||
|
@ -185,7 +185,7 @@ class PaymentIntegrationSpec extends IntegrationSpec {
|
|||
sender.send(nodes("D").paymentHandler, ReceivePayment(Some(amountMsat), "1 coffee"))
|
||||
val pr = sender.expectMsgType[PaymentRequest]
|
||||
// then we make the actual payment, do not randomize the route to make sure we route through node B
|
||||
val sendReq = SendPayment(amountMsat, pr, fallbackFinalExpiryDelta = finalCltvExpiryDelta, routeParams = integrationTestRouteParams, maxAttempts = 5)
|
||||
val sendReq = SendPaymentToNode(amountMsat, pr, fallbackFinalExpiryDelta = finalCltvExpiryDelta, routeParams = integrationTestRouteParams, maxAttempts = 5)
|
||||
sender.send(nodes("A").paymentInitiator, sendReq)
|
||||
// A will receive an error from B that include the updated channel update, then will retry the payment
|
||||
val paymentId = sender.expectMsgType[UUID]
|
||||
|
@ -227,7 +227,7 @@ class PaymentIntegrationSpec extends IntegrationSpec {
|
|||
sender.send(nodes("D").paymentHandler, ReceivePayment(Some(amountMsat), "1 coffee"))
|
||||
val pr = sender.expectMsgType[PaymentRequest]
|
||||
// then we make the payment (B-C has a smaller capacity than A-B and C-D)
|
||||
val sendReq = SendPayment(amountMsat, pr, fallbackFinalExpiryDelta = finalCltvExpiryDelta, routeParams = integrationTestRouteParams, maxAttempts = 5)
|
||||
val sendReq = SendPaymentToNode(amountMsat, pr, fallbackFinalExpiryDelta = finalCltvExpiryDelta, routeParams = integrationTestRouteParams, maxAttempts = 5)
|
||||
sender.send(nodes("A").paymentInitiator, sendReq)
|
||||
// A will first receive an error from C, then retry and route around C: A->B->E->C->D
|
||||
sender.expectMsgType[UUID]
|
||||
|
@ -239,7 +239,7 @@ class PaymentIntegrationSpec extends IntegrationSpec {
|
|||
val sender = TestProbe()
|
||||
val amount = 100000000 msat
|
||||
val unknownInvoice = PaymentRequest(Block.RegtestGenesisBlock.hash, Some(amount), randomBytes32(), nodes("D").nodeParams.privateKey, "test", finalCltvExpiryDelta)
|
||||
val pr = SendPayment(amount, unknownInvoice, routeParams = integrationTestRouteParams, maxAttempts = 5)
|
||||
val pr = SendPaymentToNode(amount, unknownInvoice, routeParams = integrationTestRouteParams, maxAttempts = 5)
|
||||
sender.send(nodes("A").paymentInitiator, pr)
|
||||
|
||||
// A will receive an error from D and won't retry
|
||||
|
@ -259,7 +259,7 @@ class PaymentIntegrationSpec extends IntegrationSpec {
|
|||
val pr = sender.expectMsgType[PaymentRequest]
|
||||
|
||||
// A send payment of only 1 mBTC
|
||||
val sendReq = SendPayment(100000000 msat, pr, fallbackFinalExpiryDelta = finalCltvExpiryDelta, routeParams = integrationTestRouteParams, maxAttempts = 5)
|
||||
val sendReq = SendPaymentToNode(100000000 msat, pr, fallbackFinalExpiryDelta = finalCltvExpiryDelta, routeParams = integrationTestRouteParams, maxAttempts = 5)
|
||||
sender.send(nodes("A").paymentInitiator, sendReq)
|
||||
|
||||
// A will first receive an IncorrectPaymentAmount error from D
|
||||
|
@ -279,7 +279,7 @@ class PaymentIntegrationSpec extends IntegrationSpec {
|
|||
val pr = sender.expectMsgType[PaymentRequest]
|
||||
|
||||
// A send payment of 6 mBTC
|
||||
val sendReq = SendPayment(600000000 msat, pr, fallbackFinalExpiryDelta = finalCltvExpiryDelta, routeParams = integrationTestRouteParams, maxAttempts = 5)
|
||||
val sendReq = SendPaymentToNode(600000000 msat, pr, fallbackFinalExpiryDelta = finalCltvExpiryDelta, routeParams = integrationTestRouteParams, maxAttempts = 5)
|
||||
sender.send(nodes("A").paymentInitiator, sendReq)
|
||||
|
||||
// A will first receive an IncorrectPaymentAmount error from D
|
||||
|
@ -299,7 +299,7 @@ class PaymentIntegrationSpec extends IntegrationSpec {
|
|||
val pr = sender.expectMsgType[PaymentRequest]
|
||||
|
||||
// A send payment of 3 mBTC, more than asked but it should still be accepted
|
||||
val sendReq = SendPayment(300000000 msat, pr, fallbackFinalExpiryDelta = finalCltvExpiryDelta, routeParams = integrationTestRouteParams, maxAttempts = 5)
|
||||
val sendReq = SendPaymentToNode(300000000 msat, pr, fallbackFinalExpiryDelta = finalCltvExpiryDelta, routeParams = integrationTestRouteParams, maxAttempts = 5)
|
||||
sender.send(nodes("A").paymentInitiator, sendReq)
|
||||
sender.expectMsgType[UUID]
|
||||
}
|
||||
|
@ -312,7 +312,7 @@ class PaymentIntegrationSpec extends IntegrationSpec {
|
|||
sender.send(nodes("D").paymentHandler, ReceivePayment(Some(amountMsat), "1 payment"))
|
||||
val pr = sender.expectMsgType[PaymentRequest]
|
||||
|
||||
val sendReq = SendPayment(amountMsat, pr, fallbackFinalExpiryDelta = finalCltvExpiryDelta, routeParams = integrationTestRouteParams, maxAttempts = 5)
|
||||
val sendReq = SendPaymentToNode(amountMsat, pr, fallbackFinalExpiryDelta = finalCltvExpiryDelta, routeParams = integrationTestRouteParams, maxAttempts = 5)
|
||||
sender.send(nodes("A").paymentInitiator, sendReq)
|
||||
sender.expectMsgType[UUID]
|
||||
sender.expectMsgType[PreimageReceived]
|
||||
|
@ -328,7 +328,7 @@ class PaymentIntegrationSpec extends IntegrationSpec {
|
|||
val pr = sender.expectMsgType[PaymentRequest]
|
||||
|
||||
// the payment is requesting to use a capacity-optimized route which will select node G even though it's a bit more expensive
|
||||
sender.send(nodes("A").paymentInitiator, SendPayment(amountMsat, pr, maxAttempts = 1, fallbackFinalExpiryDelta = finalCltvExpiryDelta, routeParams = integrationTestRouteParams.map(_.copy(ratios = WeightRatios(0, 0, 0, 1, 0 msat, 0)))))
|
||||
sender.send(nodes("A").paymentInitiator, SendPaymentToNode(amountMsat, pr, maxAttempts = 1, fallbackFinalExpiryDelta = finalCltvExpiryDelta, routeParams = integrationTestRouteParams.copy(ratios = WeightRatios(0, 0, 0, 1, 0 msat, 0))))
|
||||
sender.expectMsgType[UUID]
|
||||
sender.expectMsgType[PreimageReceived]
|
||||
val ps = sender.expectMsgType[PaymentSent]
|
||||
|
@ -343,7 +343,7 @@ class PaymentIntegrationSpec extends IntegrationSpec {
|
|||
val pr = sender.expectMsgType[PaymentRequest]
|
||||
assert(pr.features.allowMultiPart)
|
||||
|
||||
sender.send(nodes("B").paymentInitiator, SendPayment(amount, pr, maxAttempts = 5))
|
||||
sender.send(nodes("B").paymentInitiator, SendPaymentToNode(amount, pr, maxAttempts = 5, routeParams = integrationTestRouteParams))
|
||||
val paymentId = sender.expectMsgType[UUID]
|
||||
assert(sender.expectMsgType[PreimageReceived].paymentHash === pr.paymentHash)
|
||||
val paymentSent = sender.expectMsgType[PaymentSent](max = 30 seconds)
|
||||
|
@ -387,7 +387,7 @@ class PaymentIntegrationSpec extends IntegrationSpec {
|
|||
val canSend = sender.expectMsgType[Relayer.OutgoingChannels].channels.map(_.commitments.availableBalanceForSend).sum
|
||||
assert(canSend > amount)
|
||||
|
||||
sender.send(nodes("B").paymentInitiator, SendPayment(amount, pr, maxAttempts = 1))
|
||||
sender.send(nodes("B").paymentInitiator, SendPaymentToNode(amount, pr, maxAttempts = 1, routeParams = integrationTestRouteParams))
|
||||
val paymentId = sender.expectMsgType[UUID]
|
||||
val paymentFailed = sender.expectMsgType[PaymentFailed](max = 30 seconds)
|
||||
assert(paymentFailed.id === paymentId, paymentFailed)
|
||||
|
@ -410,7 +410,7 @@ class PaymentIntegrationSpec extends IntegrationSpec {
|
|||
val pr = sender.expectMsgType[PaymentRequest]
|
||||
assert(pr.features.allowMultiPart)
|
||||
|
||||
sender.send(nodes("D").paymentInitiator, SendPayment(amount, pr, maxAttempts = 3))
|
||||
sender.send(nodes("D").paymentInitiator, SendPaymentToNode(amount, pr, maxAttempts = 3, routeParams = integrationTestRouteParams))
|
||||
val paymentId = sender.expectMsgType[UUID]
|
||||
assert(sender.expectMsgType[PreimageReceived].paymentHash === pr.paymentHash)
|
||||
val paymentSent = sender.expectMsgType[PaymentSent](max = 30 seconds)
|
||||
|
@ -443,7 +443,7 @@ class PaymentIntegrationSpec extends IntegrationSpec {
|
|||
val canSend = sender.expectMsgType[Relayer.OutgoingChannels].channels.map(_.commitments.availableBalanceForSend).sum
|
||||
assert(canSend < amount)
|
||||
|
||||
sender.send(nodes("D").paymentInitiator, SendPayment(amount, pr, maxAttempts = 1))
|
||||
sender.send(nodes("D").paymentInitiator, SendPaymentToNode(amount, pr, maxAttempts = 1, routeParams = integrationTestRouteParams))
|
||||
val paymentId = sender.expectMsgType[UUID]
|
||||
val paymentFailed = sender.expectMsgType[PaymentFailed](max = 30 seconds)
|
||||
assert(paymentFailed.id === paymentId, paymentFailed)
|
||||
|
@ -469,7 +469,7 @@ class PaymentIntegrationSpec extends IntegrationSpec {
|
|||
|
||||
// The first attempt should fail, but the second one should succeed.
|
||||
val attempts = (1000 msat, CltvExpiryDelta(42)) :: (1000000 msat, CltvExpiryDelta(288)) :: Nil
|
||||
val payment = SendTrampolinePayment(amount, pr, nodes("G").nodeParams.nodeId, attempts)
|
||||
val payment = SendTrampolinePayment(amount, pr, nodes("G").nodeParams.nodeId, attempts, routeParams = integrationTestRouteParams)
|
||||
sender.send(nodes("B").paymentInitiator, payment)
|
||||
val paymentId = sender.expectMsgType[UUID]
|
||||
val paymentSent = sender.expectMsgType[PaymentSent](max = 30 seconds)
|
||||
|
@ -509,7 +509,7 @@ class PaymentIntegrationSpec extends IntegrationSpec {
|
|||
assert(pr.features.allowMultiPart)
|
||||
assert(pr.features.allowTrampoline)
|
||||
|
||||
val payment = SendTrampolinePayment(amount, pr, nodes("C").nodeParams.nodeId, Seq((350000 msat, CltvExpiryDelta(288))))
|
||||
val payment = SendTrampolinePayment(amount, pr, nodes("C").nodeParams.nodeId, Seq((350000 msat, CltvExpiryDelta(288))), routeParams = integrationTestRouteParams)
|
||||
sender.send(nodes("D").paymentInitiator, payment)
|
||||
val paymentId = sender.expectMsgType[UUID]
|
||||
val paymentSent = sender.expectMsgType[PaymentSent](max = 30 seconds)
|
||||
|
@ -558,7 +558,7 @@ class PaymentIntegrationSpec extends IntegrationSpec {
|
|||
assert(pr.features.allowMultiPart)
|
||||
assert(!pr.features.allowTrampoline)
|
||||
|
||||
val payment = SendTrampolinePayment(amount, pr, nodes("C").nodeParams.nodeId, Seq((1000000 msat, CltvExpiryDelta(432))))
|
||||
val payment = SendTrampolinePayment(amount, pr, nodes("C").nodeParams.nodeId, Seq((1000000 msat, CltvExpiryDelta(432))), routeParams = integrationTestRouteParams)
|
||||
sender.send(nodes("F").paymentInitiator, payment)
|
||||
val paymentId = sender.expectMsgType[UUID]
|
||||
val paymentSent = sender.expectMsgType[PaymentSent](max = 30 seconds)
|
||||
|
@ -593,7 +593,7 @@ class PaymentIntegrationSpec extends IntegrationSpec {
|
|||
// We put most of the capacity C <-> D on D's side.
|
||||
sender.send(nodes("D").paymentHandler, ReceivePayment(Some(8000000000L msat), "plz send everything"))
|
||||
val pr1 = sender.expectMsgType[PaymentRequest]
|
||||
sender.send(nodes("C").paymentInitiator, SendPayment(8000000000L msat, pr1, maxAttempts = 3))
|
||||
sender.send(nodes("C").paymentInitiator, SendPaymentToNode(8000000000L msat, pr1, maxAttempts = 3, routeParams = integrationTestRouteParams))
|
||||
sender.expectMsgType[UUID]
|
||||
sender.expectMsgType[PreimageReceived](max = 30 seconds)
|
||||
sender.expectMsgType[PaymentSent](max = 30 seconds)
|
||||
|
@ -605,7 +605,7 @@ class PaymentIntegrationSpec extends IntegrationSpec {
|
|||
assert(pr.features.allowMultiPart)
|
||||
assert(pr.features.allowTrampoline)
|
||||
|
||||
val payment = SendTrampolinePayment(amount, pr, nodes("C").nodeParams.nodeId, Seq((250000 msat, CltvExpiryDelta(288))))
|
||||
val payment = SendTrampolinePayment(amount, pr, nodes("C").nodeParams.nodeId, Seq((250000 msat, CltvExpiryDelta(288))), routeParams = integrationTestRouteParams)
|
||||
sender.send(nodes("B").paymentInitiator, payment)
|
||||
val paymentId = sender.expectMsgType[UUID]
|
||||
val paymentFailed = sender.expectMsgType[PaymentFailed](max = 30 seconds)
|
||||
|
@ -626,7 +626,7 @@ class PaymentIntegrationSpec extends IntegrationSpec {
|
|||
assert(pr.features.allowMultiPart)
|
||||
assert(pr.features.allowTrampoline)
|
||||
|
||||
val payment = SendTrampolinePayment(amount, pr, nodes("B").nodeParams.nodeId, Seq((450000 msat, CltvExpiryDelta(288))))
|
||||
val payment = SendTrampolinePayment(amount, pr, nodes("B").nodeParams.nodeId, Seq((450000 msat, CltvExpiryDelta(288))), routeParams = integrationTestRouteParams)
|
||||
sender.send(nodes("A").paymentInitiator, payment)
|
||||
val paymentId = sender.expectMsgType[UUID]
|
||||
val paymentFailed = sender.expectMsgType[PaymentFailed](max = 30 seconds)
|
||||
|
|
|
@ -85,7 +85,7 @@ class PerformanceIntegrationSpec extends IntegrationSpec {
|
|||
sender.send(nodes("B").paymentHandler, ReceivePayment(Some(amountMsat), "1 coffee"))
|
||||
val pr = sender.expectMsgType[PaymentRequest]
|
||||
// then we make the actual payment
|
||||
sender.send(nodes("A").paymentInitiator, PaymentInitiator.SendPayment(amountMsat, pr, fallbackFinalExpiryDelta = finalCltvExpiryDelta, routeParams = integrationTestRouteParams, maxAttempts = 1))
|
||||
sender.send(nodes("A").paymentInitiator, PaymentInitiator.SendPaymentToNode(amountMsat, pr, fallbackFinalExpiryDelta = finalCltvExpiryDelta, routeParams = integrationTestRouteParams, maxAttempts = 1))
|
||||
val paymentId = sender.expectMsgType[UUID]
|
||||
sender.expectMsgType[PreimageReceived]
|
||||
val ps = sender.expectMsgType[PaymentSent]
|
||||
|
|
|
@ -75,10 +75,10 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS
|
|||
import f._
|
||||
|
||||
assert(payFsm.stateName === WAIT_FOR_PAYMENT_REQUEST)
|
||||
val payment = SendMultiPartPayment(sender.ref, randomBytes32(), e, finalAmount, expiry, 1, routeParams = Some(routeParams.copy(randomize = true)))
|
||||
val payment = SendMultiPartPayment(sender.ref, randomBytes32(), e, finalAmount, expiry, 1, routeParams = routeParams.copy(randomize = true))
|
||||
sender.send(payFsm, payment)
|
||||
|
||||
router.expectMsg(RouteRequest(nodeParams.nodeId, e, finalAmount, maxFee, routeParams = Some(routeParams.copy(randomize = false)), allowMultiPart = true, paymentContext = Some(cfg.paymentContext)))
|
||||
router.expectMsg(RouteRequest(nodeParams.nodeId, e, finalAmount, maxFee, routeParams = routeParams.copy(randomize = false), allowMultiPart = true, paymentContext = Some(cfg.paymentContext)))
|
||||
assert(payFsm.stateName === WAIT_FOR_ROUTES)
|
||||
|
||||
val singleRoute = Route(finalAmount, hop_ab_1 :: hop_be :: Nil)
|
||||
|
@ -101,10 +101,10 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS
|
|||
import f._
|
||||
|
||||
assert(payFsm.stateName === WAIT_FOR_PAYMENT_REQUEST)
|
||||
val payment = SendMultiPartPayment(sender.ref, randomBytes32(), e, 1200000 msat, expiry, 1, routeParams = Some(routeParams.copy(randomize = false)))
|
||||
val payment = SendMultiPartPayment(sender.ref, randomBytes32(), e, 1200000 msat, expiry, 1, routeParams = routeParams.copy(randomize = false))
|
||||
sender.send(payFsm, payment)
|
||||
|
||||
router.expectMsg(RouteRequest(nodeParams.nodeId, e, 1200000 msat, maxFee, routeParams = Some(routeParams.copy(randomize = false)), allowMultiPart = true, paymentContext = Some(cfg.paymentContext)))
|
||||
router.expectMsg(RouteRequest(nodeParams.nodeId, e, 1200000 msat, maxFee, routeParams = routeParams.copy(randomize = false), allowMultiPart = true, paymentContext = Some(cfg.paymentContext)))
|
||||
assert(payFsm.stateName === WAIT_FOR_ROUTES)
|
||||
|
||||
val routes = Seq(
|
||||
|
@ -132,7 +132,7 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS
|
|||
// We include a bunch of additional tlv records.
|
||||
val trampolineTlv = OnionTlv.TrampolineOnion(OnionRoutingPacket(0, ByteVector.fill(33)(0), ByteVector.fill(400)(0), randomBytes32()))
|
||||
val userCustomTlv = GenericTlv(UInt64(561), hex"deadbeef")
|
||||
val payment = SendMultiPartPayment(sender.ref, randomBytes32(), e, finalAmount + 1000.msat, expiry, 1, routeParams = Some(routeParams), additionalTlvs = Seq(trampolineTlv), userCustomTlvs = Seq(userCustomTlv))
|
||||
val payment = SendMultiPartPayment(sender.ref, randomBytes32(), e, finalAmount + 1000.msat, expiry, 1, routeParams = routeParams, additionalTlvs = Seq(trampolineTlv), userCustomTlvs = Seq(userCustomTlv))
|
||||
sender.send(payFsm, payment)
|
||||
router.expectMsgType[RouteRequest]
|
||||
router.send(payFsm, RouteResponse(Seq(Route(500000 msat, hop_ab_1 :: hop_be :: Nil), Route(501000 msat, hop_ac_1 :: hop_ce :: Nil))))
|
||||
|
@ -149,7 +149,7 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS
|
|||
test("successful retry") { f =>
|
||||
import f._
|
||||
|
||||
val payment = SendMultiPartPayment(sender.ref, randomBytes32(), e, finalAmount, expiry, 3, routeParams = Some(routeParams))
|
||||
val payment = SendMultiPartPayment(sender.ref, randomBytes32(), e, finalAmount, expiry, 3, routeParams = routeParams)
|
||||
sender.send(payFsm, payment)
|
||||
router.expectMsgType[RouteRequest]
|
||||
val failingRoute = Route(finalAmount, hop_ab_1 :: hop_be :: Nil)
|
||||
|
@ -160,7 +160,7 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS
|
|||
val childId = payFsm.stateData.asInstanceOf[PaymentProgress].pending.keys.head
|
||||
childPayFsm.send(payFsm, PaymentFailed(childId, paymentHash, Seq(RemoteFailure(failingRoute.hops, Sphinx.DecryptedFailurePacket(b, PermanentChannelFailure)))))
|
||||
// We retry ignoring the failing channel.
|
||||
router.expectMsg(RouteRequest(nodeParams.nodeId, e, finalAmount, maxFee, routeParams = Some(routeParams.copy(randomize = true)), allowMultiPart = true, ignore = Ignore(Set.empty, Set(ChannelDesc(channelId_be, b, e))), paymentContext = Some(cfg.paymentContext)))
|
||||
router.expectMsg(RouteRequest(nodeParams.nodeId, e, finalAmount, maxFee, routeParams = routeParams.copy(randomize = true), allowMultiPart = true, ignore = Ignore(Set.empty, Set(ChannelDesc(channelId_be, b, e))), paymentContext = Some(cfg.paymentContext)))
|
||||
router.send(payFsm, RouteResponse(Seq(Route(400000 msat, hop_ac_1 :: hop_ce :: Nil), Route(600000 msat, hop_ad :: hop_de :: Nil))))
|
||||
childPayFsm.expectMsgType[SendPaymentToRoute]
|
||||
childPayFsm.expectMsgType[SendPaymentToRoute]
|
||||
|
@ -175,7 +175,7 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS
|
|||
test("retry failures while waiting for routes") { f =>
|
||||
import f._
|
||||
|
||||
val payment = SendMultiPartPayment(sender.ref, randomBytes32(), e, finalAmount, expiry, 3, routeParams = Some(routeParams))
|
||||
val payment = SendMultiPartPayment(sender.ref, randomBytes32(), e, finalAmount, expiry, 3, routeParams = routeParams)
|
||||
sender.send(payFsm, payment)
|
||||
router.expectMsgType[RouteRequest]
|
||||
router.send(payFsm, RouteResponse(Seq(Route(400000 msat, hop_ab_1 :: hop_be :: Nil), Route(600000 msat, hop_ab_2 :: hop_be :: Nil))))
|
||||
|
@ -187,13 +187,13 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS
|
|||
childPayFsm.send(payFsm, PaymentFailed(failedId1, paymentHash, Seq(RemoteFailure(failedRoute1.hops, Sphinx.DecryptedFailurePacket(b, TemporaryNodeFailure)))))
|
||||
|
||||
// When we retry, we ignore the failing node and we let the router know about the remaining pending route.
|
||||
router.expectMsg(RouteRequest(nodeParams.nodeId, e, failedRoute1.amount, maxFee - failedRoute1.fee, ignore = Ignore(Set(b), Set.empty), pendingPayments = Seq(failedRoute2), allowMultiPart = true, routeParams = Some(routeParams.copy(randomize = true)), paymentContext = Some(cfg.paymentContext)))
|
||||
router.expectMsg(RouteRequest(nodeParams.nodeId, e, failedRoute1.amount, maxFee - failedRoute1.fee, ignore = Ignore(Set(b), Set.empty), pendingPayments = Seq(failedRoute2), allowMultiPart = true, routeParams = routeParams.copy(randomize = true), paymentContext = Some(cfg.paymentContext)))
|
||||
// The second part fails while we're still waiting for new routes.
|
||||
childPayFsm.send(payFsm, PaymentFailed(failedId2, paymentHash, Seq(RemoteFailure(failedRoute2.hops, Sphinx.DecryptedFailurePacket(b, TemporaryNodeFailure)))))
|
||||
// We receive a response to our first request, but it's now obsolete: we re-sent a new route request that takes into
|
||||
// account the latest failures.
|
||||
router.send(payFsm, RouteResponse(Seq(Route(failedRoute1.amount, hop_ac_1 :: hop_ce :: Nil))))
|
||||
router.expectMsg(RouteRequest(nodeParams.nodeId, e, finalAmount, maxFee, ignore = Ignore(Set(b), Set.empty), allowMultiPart = true, routeParams = Some(routeParams.copy(randomize = true)), paymentContext = Some(cfg.paymentContext)))
|
||||
router.expectMsg(RouteRequest(nodeParams.nodeId, e, finalAmount, maxFee, ignore = Ignore(Set(b), Set.empty), allowMultiPart = true, routeParams = routeParams.copy(randomize = true), paymentContext = Some(cfg.paymentContext)))
|
||||
awaitCond(payFsm.stateData.asInstanceOf[PaymentProgress].pending.isEmpty)
|
||||
childPayFsm.expectNoMessage(100 millis)
|
||||
|
||||
|
@ -210,7 +210,7 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS
|
|||
test("retry local channel failures") { f =>
|
||||
import f._
|
||||
|
||||
val payment = SendMultiPartPayment(sender.ref, randomBytes32(), e, finalAmount, expiry, 3, routeParams = Some(routeParams))
|
||||
val payment = SendMultiPartPayment(sender.ref, randomBytes32(), e, finalAmount, expiry, 3, routeParams = routeParams)
|
||||
sender.send(payFsm, payment)
|
||||
router.expectMsgType[RouteRequest]
|
||||
router.send(payFsm, RouteResponse(Seq(Route(finalAmount, hop_ab_1 :: hop_be :: Nil))))
|
||||
|
@ -227,7 +227,7 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS
|
|||
ignore = Ignore(Set.empty, Set(ChannelDesc(channelId_ab_1, a, b))),
|
||||
pendingPayments = Nil,
|
||||
allowMultiPart = true,
|
||||
routeParams = Some(routeParams.copy(randomize = true)),
|
||||
routeParams = routeParams.copy(randomize = true),
|
||||
paymentContext = Some(cfg.paymentContext))
|
||||
router.expectMsg(expectedRouteRequest)
|
||||
}
|
||||
|
@ -235,7 +235,7 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS
|
|||
test("retry without ignoring channels") { f =>
|
||||
import f._
|
||||
|
||||
val payment = SendMultiPartPayment(sender.ref, randomBytes32(), e, finalAmount, expiry, 3, routeParams = Some(routeParams))
|
||||
val payment = SendMultiPartPayment(sender.ref, randomBytes32(), e, finalAmount, expiry, 3, routeParams = routeParams)
|
||||
sender.send(payFsm, payment)
|
||||
router.expectMsgType[RouteRequest]
|
||||
router.send(payFsm, RouteResponse(Seq(Route(500000 msat, hop_ab_1 :: hop_be :: Nil), Route(500000 msat, hop_ab_1 :: hop_be :: Nil))))
|
||||
|
@ -254,7 +254,7 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS
|
|||
ignore = Ignore(Set.empty, Set(ChannelDesc(channelId_ab_1, a, b))),
|
||||
pendingPayments = Seq(pendingRoute),
|
||||
allowMultiPart = true,
|
||||
routeParams = Some(routeParams.copy(randomize = true)),
|
||||
routeParams = routeParams.copy(randomize = true),
|
||||
paymentContext = Some(cfg.paymentContext))
|
||||
router.expectMsg(expectedRouteRequest)
|
||||
router.send(payFsm, Status.Failure(RouteNotFound))
|
||||
|
@ -272,7 +272,7 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS
|
|||
|
||||
// The B -> E channel is private and provided in the invoice routing hints.
|
||||
val routingHint = ExtraHop(b, hop_be.lastUpdate.shortChannelId, hop_be.lastUpdate.feeBaseMsat, hop_be.lastUpdate.feeProportionalMillionths, hop_be.lastUpdate.cltvExpiryDelta)
|
||||
val payment = SendMultiPartPayment(sender.ref, randomBytes32(), e, finalAmount, expiry, 3, routeParams = Some(routeParams), assistedRoutes = List(List(routingHint)))
|
||||
val payment = SendMultiPartPayment(sender.ref, randomBytes32(), e, finalAmount, expiry, 3, routeParams = routeParams, assistedRoutes = List(List(routingHint)))
|
||||
sender.send(payFsm, payment)
|
||||
assert(router.expectMsgType[RouteRequest].assistedRoutes.head.head === routingHint)
|
||||
val route = Route(finalAmount, hop_ab_1 :: hop_be :: Nil)
|
||||
|
@ -293,7 +293,7 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS
|
|||
|
||||
// The B -> E channel is private and provided in the invoice routing hints.
|
||||
val routingHint = ExtraHop(b, hop_be.lastUpdate.shortChannelId, hop_be.lastUpdate.feeBaseMsat, hop_be.lastUpdate.feeProportionalMillionths, hop_be.lastUpdate.cltvExpiryDelta)
|
||||
val payment = SendMultiPartPayment(sender.ref, randomBytes32(), e, finalAmount, expiry, 3, routeParams = Some(routeParams), assistedRoutes = List(List(routingHint)))
|
||||
val payment = SendMultiPartPayment(sender.ref, randomBytes32(), e, finalAmount, expiry, 3, routeParams = routeParams, assistedRoutes = List(List(routingHint)))
|
||||
sender.send(payFsm, payment)
|
||||
assert(router.expectMsgType[RouteRequest].assistedRoutes.head.head === routingHint)
|
||||
val route = Route(finalAmount, hop_ab_1 :: hop_be :: Nil)
|
||||
|
@ -352,7 +352,7 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS
|
|||
test("abort after too many failed attempts") { f =>
|
||||
import f._
|
||||
|
||||
val payment = SendMultiPartPayment(sender.ref, randomBytes32(), e, finalAmount, expiry, 2, routeParams = Some(routeParams))
|
||||
val payment = SendMultiPartPayment(sender.ref, randomBytes32(), e, finalAmount, expiry, 2, routeParams = routeParams)
|
||||
sender.send(payFsm, payment)
|
||||
router.expectMsgType[RouteRequest]
|
||||
router.send(payFsm, RouteResponse(Seq(Route(500000 msat, hop_ab_1 :: hop_be :: Nil), Route(500000 msat, hop_ac_1 :: hop_ce :: Nil))))
|
||||
|
@ -376,7 +376,7 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS
|
|||
import f._
|
||||
|
||||
sender.watch(payFsm)
|
||||
val payment = SendMultiPartPayment(sender.ref, randomBytes32(), e, finalAmount, expiry, 5, routeParams = Some(routeParams))
|
||||
val payment = SendMultiPartPayment(sender.ref, randomBytes32(), e, finalAmount, expiry, 5, routeParams = routeParams)
|
||||
sender.send(payFsm, payment)
|
||||
router.expectMsgType[RouteRequest]
|
||||
router.send(payFsm, Status.Failure(RouteNotFound))
|
||||
|
@ -399,7 +399,7 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS
|
|||
test("abort if recipient sends error") { f =>
|
||||
import f._
|
||||
|
||||
val payment = SendMultiPartPayment(sender.ref, randomBytes32(), e, finalAmount, expiry, 5, routeParams = Some(routeParams))
|
||||
val payment = SendMultiPartPayment(sender.ref, randomBytes32(), e, finalAmount, expiry, 5, routeParams = routeParams)
|
||||
sender.send(payFsm, payment)
|
||||
router.expectMsgType[RouteRequest]
|
||||
router.send(payFsm, RouteResponse(Seq(Route(finalAmount, hop_ab_1 :: hop_be :: Nil))))
|
||||
|
@ -413,7 +413,7 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS
|
|||
test("abort if payment gets settled on chain") { f =>
|
||||
import f._
|
||||
|
||||
val payment = SendMultiPartPayment(sender.ref, randomBytes32(), e, finalAmount, expiry, 5, routeParams = Some(routeParams))
|
||||
val payment = SendMultiPartPayment(sender.ref, randomBytes32(), e, finalAmount, expiry, 5, routeParams = routeParams)
|
||||
sender.send(payFsm, payment)
|
||||
router.expectMsgType[RouteRequest]
|
||||
router.send(payFsm, RouteResponse(Seq(Route(finalAmount, hop_ab_1 :: hop_be :: Nil))))
|
||||
|
@ -427,7 +427,7 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS
|
|||
test("abort if recipient sends error during retry") { f =>
|
||||
import f._
|
||||
|
||||
val payment = SendMultiPartPayment(sender.ref, randomBytes32(), e, finalAmount, expiry, 5, routeParams = Some(routeParams))
|
||||
val payment = SendMultiPartPayment(sender.ref, randomBytes32(), e, finalAmount, expiry, 5, routeParams = routeParams)
|
||||
sender.send(payFsm, payment)
|
||||
router.expectMsgType[RouteRequest]
|
||||
router.send(payFsm, RouteResponse(Seq(Route(400000 msat, hop_ab_1 :: hop_be :: Nil), Route(600000 msat, hop_ac_1 :: hop_ce :: Nil))))
|
||||
|
@ -445,7 +445,7 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS
|
|||
test("receive partial success after retriable failure (recipient spec violation)") { f =>
|
||||
import f._
|
||||
|
||||
val payment = SendMultiPartPayment(sender.ref, randomBytes32(), e, finalAmount, expiry, 5, routeParams = Some(routeParams))
|
||||
val payment = SendMultiPartPayment(sender.ref, randomBytes32(), e, finalAmount, expiry, 5, routeParams = routeParams)
|
||||
sender.send(payFsm, payment)
|
||||
router.expectMsgType[RouteRequest]
|
||||
router.send(payFsm, RouteResponse(Seq(Route(400000 msat, hop_ab_1 :: hop_be :: Nil), Route(600000 msat, hop_ac_1 :: hop_ce :: Nil))))
|
||||
|
@ -465,7 +465,7 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS
|
|||
test("receive partial success after abort (recipient spec violation)") { f =>
|
||||
import f._
|
||||
|
||||
val payment = SendMultiPartPayment(sender.ref, randomBytes32(), e, finalAmount, expiry, 5, routeParams = Some(routeParams))
|
||||
val payment = SendMultiPartPayment(sender.ref, randomBytes32(), e, finalAmount, expiry, 5, routeParams = routeParams)
|
||||
sender.send(payFsm, payment)
|
||||
router.expectMsgType[RouteRequest]
|
||||
router.send(payFsm, RouteResponse(Seq(Route(400000 msat, hop_ab_1 :: hop_be :: Nil), Route(600000 msat, hop_ac_1 :: hop_ce :: Nil))))
|
||||
|
@ -498,7 +498,7 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS
|
|||
test("receive partial failure after success (recipient spec violation)") { f =>
|
||||
import f._
|
||||
|
||||
val payment = SendMultiPartPayment(sender.ref, randomBytes32(), e, finalAmount, expiry, 5, routeParams = Some(routeParams))
|
||||
val payment = SendMultiPartPayment(sender.ref, randomBytes32(), e, finalAmount, expiry, 5, routeParams = routeParams)
|
||||
sender.send(payFsm, payment)
|
||||
router.expectMsgType[RouteRequest]
|
||||
router.send(payFsm, RouteResponse(Seq(Route(400000 msat, hop_ab_1 :: hop_be :: Nil), Route(600000 msat, hop_ac_1 :: hop_ce :: Nil))))
|
||||
|
|
|
@ -30,7 +30,7 @@ import fr.acinq.eclair.payment.PaymentRequest.{ExtraHop, PaymentRequestFeatures}
|
|||
import fr.acinq.eclair.payment.send.MultiPartPaymentLifecycle.SendMultiPartPayment
|
||||
import fr.acinq.eclair.payment.send.PaymentInitiator._
|
||||
import fr.acinq.eclair.payment.send.{PaymentError, PaymentInitiator, PaymentLifecycle}
|
||||
import fr.acinq.eclair.router.RouteNotFound
|
||||
import fr.acinq.eclair.router.{RouteCalculation, RouteNotFound}
|
||||
import fr.acinq.eclair.router.Router._
|
||||
import fr.acinq.eclair.wire.protocol.Onion.FinalTlvPayload
|
||||
import fr.acinq.eclair.wire.protocol.OnionTlv.{AmountToForward, KeySend, OutgoingCltv}
|
||||
|
@ -89,7 +89,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
|||
import f._
|
||||
val customRecords = Seq(GenericTlv(500L, hex"01020304"), GenericTlv(501L, hex"d34db33f"))
|
||||
val pr = PaymentRequest(Block.LivenetGenesisBlock.hash, None, paymentHash, priv_c.privateKey, "test", Channel.MIN_CLTV_EXPIRY_DELTA)
|
||||
val req = SendPayment(finalAmount, pr, 1, Channel.MIN_CLTV_EXPIRY_DELTA, userCustomTlvs = customRecords)
|
||||
val req = SendPaymentToNode(finalAmount, pr, 1, Channel.MIN_CLTV_EXPIRY_DELTA, userCustomTlvs = customRecords, routeParams = RouteCalculation.getDefaultRouteParams(nodeParams.routerConf.pathFindingConf))
|
||||
sender.send(initiator, req)
|
||||
sender.expectMsgType[UUID]
|
||||
payFsm.expectMsgType[SendPaymentConfig]
|
||||
|
@ -101,7 +101,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
|||
|
||||
test("forward keysend payment") { f =>
|
||||
import f._
|
||||
val req = SendSpontaneousPayment(finalAmount, c, paymentPreimage, 1)
|
||||
val req = SendSpontaneousPayment(finalAmount, c, paymentPreimage, 1, routeParams = RouteCalculation.getDefaultRouteParams(nodeParams.routerConf.pathFindingConf))
|
||||
sender.send(initiator, req)
|
||||
sender.expectMsgType[UUID]
|
||||
payFsm.expectMsgType[SendPaymentConfig]
|
||||
|
@ -116,7 +116,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
|||
import f._
|
||||
val unknownFeature = 42
|
||||
val pr = PaymentRequest(Block.LivenetGenesisBlock.hash, Some(finalAmount), paymentHash, randomKey(), "Some invoice", CltvExpiryDelta(18), features = PaymentRequestFeatures(Features.VariableLengthOnion.mandatory, Features.PaymentSecret.mandatory, unknownFeature))
|
||||
val req = SendPayment(finalAmount + 100.msat, pr, 1, CltvExpiryDelta(42))
|
||||
val req = SendPaymentToNode(finalAmount + 100.msat, pr, 1, CltvExpiryDelta(42), routeParams = RouteCalculation.getDefaultRouteParams(nodeParams.routerConf.pathFindingConf))
|
||||
sender.send(initiator, req)
|
||||
val id = sender.expectMsgType[UUID]
|
||||
val fail = sender.expectMsgType[PaymentFailed]
|
||||
|
@ -141,22 +141,22 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
|||
import f._
|
||||
val finalExpiryDelta = CltvExpiryDelta(24)
|
||||
val pr = PaymentRequest(Block.LivenetGenesisBlock.hash, Some(finalAmount), paymentHash, priv_c.privateKey, "Some MPP invoice", finalExpiryDelta, features = PaymentRequestFeatures(VariableLengthOnion.mandatory, PaymentSecret.mandatory, BasicMultiPartPayment.optional))
|
||||
val req = SendPayment(finalAmount, pr, 1, /* ignored since the invoice provides it */ CltvExpiryDelta(12))
|
||||
val req = SendPaymentToNode(finalAmount, pr, 1, /* ignored since the invoice provides it */ CltvExpiryDelta(12), routeParams = RouteCalculation.getDefaultRouteParams(nodeParams.routerConf.pathFindingConf))
|
||||
assert(req.finalExpiry(nodeParams.currentBlockHeight) === (finalExpiryDelta + 1).toCltvExpiry(nodeParams.currentBlockHeight))
|
||||
sender.send(initiator, req)
|
||||
val id = sender.expectMsgType[UUID]
|
||||
payFsm.expectMsg(SendPaymentConfig(id, id, None, paymentHash, finalAmount, c, Upstream.Local(id), Some(pr), storeInDb = true, publishEvent = true, Nil))
|
||||
payFsm.expectMsg(PaymentLifecycle.SendPaymentToNode(sender.ref, c, FinalTlvPayload(TlvStream(OnionTlv.AmountToForward(finalAmount), OnionTlv.OutgoingCltv(req.finalExpiry(nodeParams.currentBlockHeight)), OnionTlv.PaymentData(pr.paymentSecret.get, finalAmount))), 1))
|
||||
payFsm.expectMsg(PaymentLifecycle.SendPaymentToNode(sender.ref, c, FinalTlvPayload(TlvStream(OnionTlv.AmountToForward(finalAmount), OnionTlv.OutgoingCltv(req.finalExpiry(nodeParams.currentBlockHeight)), OnionTlv.PaymentData(pr.paymentSecret.get, finalAmount))), 1, routeParams = RouteCalculation.getDefaultRouteParams(nodeParams.routerConf.pathFindingConf)))
|
||||
}
|
||||
|
||||
test("forward multi-part payment") { f =>
|
||||
import f._
|
||||
val pr = PaymentRequest(Block.LivenetGenesisBlock.hash, Some(finalAmount), paymentHash, priv_c.privateKey, "Some invoice", CltvExpiryDelta(18), features = PaymentRequestFeatures(VariableLengthOnion.mandatory, PaymentSecret.mandatory, BasicMultiPartPayment.optional))
|
||||
val req = SendPayment(finalAmount + 100.msat, pr, 1, CltvExpiryDelta(42))
|
||||
val req = SendPaymentToNode(finalAmount + 100.msat, pr, 1, CltvExpiryDelta(42), routeParams = RouteCalculation.getDefaultRouteParams(nodeParams.routerConf.pathFindingConf))
|
||||
sender.send(initiator, req)
|
||||
val id = sender.expectMsgType[UUID]
|
||||
multiPartPayFsm.expectMsg(SendPaymentConfig(id, id, None, paymentHash, finalAmount + 100.msat, c, Upstream.Local(id), Some(pr), storeInDb = true, publishEvent = true, Nil))
|
||||
multiPartPayFsm.expectMsg(SendMultiPartPayment(sender.ref, pr.paymentSecret.get, c, finalAmount + 100.msat, req.finalExpiry(nodeParams.currentBlockHeight), 1))
|
||||
multiPartPayFsm.expectMsg(SendMultiPartPayment(sender.ref, pr.paymentSecret.get, c, finalAmount + 100.msat, req.finalExpiry(nodeParams.currentBlockHeight), 1, routeParams = RouteCalculation.getDefaultRouteParams(nodeParams.routerConf.pathFindingConf)))
|
||||
}
|
||||
|
||||
test("forward multi-part payment with pre-defined route") { f =>
|
||||
|
@ -181,7 +181,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
|||
val ignoredRoutingHints = List(List(ExtraHop(b, channelUpdate_bc.shortChannelId, feeBase = 10 msat, feeProportionalMillionths = 1, cltvExpiryDelta = CltvExpiryDelta(12))))
|
||||
val pr = PaymentRequest(Block.LivenetGenesisBlock.hash, Some(finalAmount), paymentHash, priv_c.privateKey, "Some phoenix invoice", CltvExpiryDelta(9), features = features, extraHops = ignoredRoutingHints)
|
||||
val trampolineFees = 21000 msat
|
||||
val req = SendTrampolinePayment(finalAmount, pr, b, Seq((trampolineFees, CltvExpiryDelta(12))), /* ignored since the invoice provides it */ CltvExpiryDelta(18))
|
||||
val req = SendTrampolinePayment(finalAmount, pr, b, Seq((trampolineFees, CltvExpiryDelta(12))), /* ignored since the invoice provides it */ CltvExpiryDelta(18), routeParams = RouteCalculation.getDefaultRouteParams(nodeParams.routerConf.pathFindingConf))
|
||||
sender.send(initiator, req)
|
||||
sender.expectMsgType[UUID]
|
||||
multiPartPayFsm.expectMsgType[SendPaymentConfig]
|
||||
|
@ -221,7 +221,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
|||
import f._
|
||||
val pr = PaymentRequest(Block.LivenetGenesisBlock.hash, Some(finalAmount), paymentHash, priv_c.privateKey, "Some eclair-mobile invoice", CltvExpiryDelta(9))
|
||||
val trampolineFees = 21000 msat
|
||||
val req = SendTrampolinePayment(finalAmount, pr, b, Seq((trampolineFees, CltvExpiryDelta(12))))
|
||||
val req = SendTrampolinePayment(finalAmount, pr, b, Seq((trampolineFees, CltvExpiryDelta(12))), routeParams = RouteCalculation.getDefaultRouteParams(nodeParams.routerConf.pathFindingConf))
|
||||
sender.send(initiator, req)
|
||||
sender.expectMsgType[UUID]
|
||||
multiPartPayFsm.expectMsgType[SendPaymentConfig]
|
||||
|
@ -253,7 +253,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
|||
val features = PaymentRequestFeatures(VariableLengthOnion.mandatory, PaymentSecret.mandatory, BasicMultiPartPayment.optional)
|
||||
val pr = PaymentRequest(Block.RegtestGenesisBlock.hash, None, paymentHash, priv_a.privateKey, "#abittooreckless", CltvExpiryDelta(18), None, None, routingHints, features = features)
|
||||
val trampolineFees = 21000 msat
|
||||
val req = SendTrampolinePayment(finalAmount, pr, b, Seq((trampolineFees, CltvExpiryDelta(12))), CltvExpiryDelta(9))
|
||||
val req = SendTrampolinePayment(finalAmount, pr, b, Seq((trampolineFees, CltvExpiryDelta(12))), CltvExpiryDelta(9), routeParams = RouteCalculation.getDefaultRouteParams(nodeParams.routerConf.pathFindingConf))
|
||||
sender.send(initiator, req)
|
||||
val id = sender.expectMsgType[UUID]
|
||||
val fail = sender.expectMsgType[PaymentFailed]
|
||||
|
@ -269,7 +269,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
|||
val features = PaymentRequestFeatures(VariableLengthOnion.mandatory, PaymentSecret.mandatory, BasicMultiPartPayment.optional, TrampolinePayment.optional)
|
||||
val pr = PaymentRequest(Block.LivenetGenesisBlock.hash, Some(finalAmount), paymentHash, priv_c.privateKey, "Some phoenix invoice", CltvExpiryDelta(18), features = features)
|
||||
val trampolineAttempts = (21000 msat, CltvExpiryDelta(12)) :: (25000 msat, CltvExpiryDelta(24)) :: Nil
|
||||
val req = SendTrampolinePayment(finalAmount, pr, b, trampolineAttempts, CltvExpiryDelta(9))
|
||||
val req = SendTrampolinePayment(finalAmount, pr, b, trampolineAttempts, CltvExpiryDelta(9), routeParams = RouteCalculation.getDefaultRouteParams(nodeParams.routerConf.pathFindingConf))
|
||||
sender.send(initiator, req)
|
||||
sender.expectMsgType[UUID]
|
||||
val cfg = multiPartPayFsm.expectMsgType[SendPaymentConfig]
|
||||
|
@ -299,7 +299,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
|||
val features = PaymentRequestFeatures(VariableLengthOnion.mandatory, PaymentSecret.mandatory, BasicMultiPartPayment.optional, TrampolinePayment.optional)
|
||||
val pr = PaymentRequest(Block.LivenetGenesisBlock.hash, Some(finalAmount), paymentHash, priv_c.privateKey, "Some phoenix invoice", CltvExpiryDelta(18), features = features)
|
||||
val trampolineAttempts = (21000 msat, CltvExpiryDelta(12)) :: (25000 msat, CltvExpiryDelta(24)) :: Nil
|
||||
val req = SendTrampolinePayment(finalAmount, pr, b, trampolineAttempts, CltvExpiryDelta(9))
|
||||
val req = SendTrampolinePayment(finalAmount, pr, b, trampolineAttempts, CltvExpiryDelta(9), routeParams = RouteCalculation.getDefaultRouteParams(nodeParams.routerConf.pathFindingConf))
|
||||
sender.send(initiator, req)
|
||||
sender.expectMsgType[UUID]
|
||||
val cfg = multiPartPayFsm.expectMsgType[SendPaymentConfig]
|
||||
|
@ -329,7 +329,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
|||
val features = PaymentRequestFeatures(VariableLengthOnion.mandatory, PaymentSecret.mandatory, BasicMultiPartPayment.optional, TrampolinePayment.optional)
|
||||
val pr = PaymentRequest(Block.LivenetGenesisBlock.hash, Some(finalAmount), paymentHash, priv_c.privateKey, "Some phoenix invoice", CltvExpiryDelta(18), features = features)
|
||||
val trampolineAttempts = (21000 msat, CltvExpiryDelta(12)) :: (25000 msat, CltvExpiryDelta(24)) :: Nil
|
||||
val req = SendTrampolinePayment(finalAmount, pr, b, trampolineAttempts, CltvExpiryDelta(9))
|
||||
val req = SendTrampolinePayment(finalAmount, pr, b, trampolineAttempts, CltvExpiryDelta(9), routeParams = RouteCalculation.getDefaultRouteParams(nodeParams.routerConf.pathFindingConf))
|
||||
sender.send(initiator, req)
|
||||
sender.expectMsgType[UUID]
|
||||
|
||||
|
|
|
@ -60,8 +60,9 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
val defaultOrigin = Origin.LocalCold(UUID.randomUUID())
|
||||
val defaultExternalId = UUID.randomUUID().toString
|
||||
val defaultInvoice = PaymentRequest(Block.RegtestGenesisBlock.hash, None, defaultPaymentHash, priv_d, "test", Channel.MIN_CLTV_EXPIRY_DELTA)
|
||||
val defaultRouteParams = RouteCalculation.getDefaultRouteParams(TestConstants.Alice.nodeParams.routerConf.pathFindingConf)
|
||||
|
||||
def defaultRouteRequest(source: PublicKey, target: PublicKey, cfg: SendPaymentConfig): RouteRequest = RouteRequest(source, target, defaultAmountMsat, defaultMaxFee, paymentContext = Some(cfg.paymentContext))
|
||||
def defaultRouteRequest(source: PublicKey, target: PublicKey, cfg: SendPaymentConfig): RouteRequest = RouteRequest(source, target, defaultAmountMsat, defaultMaxFee, paymentContext = Some(cfg.paymentContext), routeParams = defaultRouteParams)
|
||||
|
||||
case class PaymentFixture(cfg: SendPaymentConfig,
|
||||
nodeParams: NodeParams,
|
||||
|
@ -196,7 +197,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
import payFixture._
|
||||
import cfg._
|
||||
|
||||
val request = SendPaymentToNode(sender.ref, f, Onion.createSinglePartPayload(defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret.get), 5)
|
||||
val request = SendPaymentToNode(sender.ref, f, Onion.createSinglePartPayload(defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret.get), 5, routeParams = defaultRouteParams)
|
||||
sender.send(paymentFSM, request)
|
||||
val Transition(_, WAITING_FOR_REQUEST, WAITING_FOR_ROUTE) = monitor.expectMsgClass(classOf[Transition[_]])
|
||||
val routeRequest = routerForwarder.expectMsgType[RouteRequest]
|
||||
|
@ -212,7 +213,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
import payFixture._
|
||||
import cfg._
|
||||
|
||||
val request = SendPaymentToNode(sender.ref, d, Onion.createSinglePartPayload(defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret.get), 5, routeParams = Some(RouteParams(randomize = false, 100 msat, 0.0, 20, CltvExpiryDelta(2016), WeightRatios(1, 0, 0, 0, 0 msat, 0), MultiPartParams(10000 msat, 5), false)))
|
||||
val request = SendPaymentToNode(sender.ref, d, Onion.createSinglePartPayload(defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret.get), 5, routeParams = RouteParams(randomize = false, 100 msat, 0.0, 20, CltvExpiryDelta(2016), WeightRatios(1, 0, 0, 0, 0 msat, 0), MultiPartParams(10000 msat, 5), false))
|
||||
sender.send(paymentFSM, request)
|
||||
val routeRequest = routerForwarder.expectMsgType[RouteRequest]
|
||||
val Transition(_, WAITING_FOR_REQUEST, WAITING_FOR_ROUTE) = monitor.expectMsgClass(classOf[Transition[_]])
|
||||
|
@ -227,7 +228,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
import payFixture._
|
||||
import cfg._
|
||||
|
||||
val request = SendPaymentToNode(sender.ref, d, Onion.createSinglePartPayload(defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret.get), 2)
|
||||
val request = SendPaymentToNode(sender.ref, d, Onion.createSinglePartPayload(defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret.get), 2, routeParams = defaultRouteParams)
|
||||
sender.send(paymentFSM, request)
|
||||
routerForwarder.expectMsg(defaultRouteRequest(a, d, cfg))
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && nodeParams.db.payments.getOutgoingPayment(id).exists(_.status === OutgoingPaymentStatus.Pending))
|
||||
|
@ -263,7 +264,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
import payFixture._
|
||||
import cfg._
|
||||
|
||||
val request = SendPaymentToNode(sender.ref, d, Onion.createSinglePartPayload(defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret.get), 2)
|
||||
val request = SendPaymentToNode(sender.ref, d, Onion.createSinglePartPayload(defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret.get), 2, routeParams = defaultRouteParams)
|
||||
sender.send(paymentFSM, request)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && nodeParams.db.payments.getOutgoingPayment(id).exists(_.status === OutgoingPaymentStatus.Pending))
|
||||
routerForwarder.expectMsgType[RouteRequest]
|
||||
|
@ -284,7 +285,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
import payFixture._
|
||||
import cfg._
|
||||
|
||||
val request = SendPaymentToNode(sender.ref, d, Onion.createSinglePartPayload(defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret.get), 2)
|
||||
val request = SendPaymentToNode(sender.ref, d, Onion.createSinglePartPayload(defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret.get), 2, routeParams = defaultRouteParams)
|
||||
sender.send(paymentFSM, request)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && nodeParams.db.payments.getOutgoingPayment(id).exists(_.status === OutgoingPaymentStatus.Pending))
|
||||
routerForwarder.expectMsgType[RouteRequest]
|
||||
|
@ -304,7 +305,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
import payFixture._
|
||||
import cfg._
|
||||
|
||||
val request = SendPaymentToNode(sender.ref, d, Onion.createSinglePartPayload(defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret.get), 2)
|
||||
val request = SendPaymentToNode(sender.ref, d, Onion.createSinglePartPayload(defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret.get), 2, routeParams = defaultRouteParams)
|
||||
sender.send(paymentFSM, request)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && nodeParams.db.payments.getOutgoingPayment(id).exists(_.status === OutgoingPaymentStatus.Pending))
|
||||
|
||||
|
@ -327,7 +328,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
import payFixture._
|
||||
import cfg._
|
||||
|
||||
val request = SendPaymentToNode(sender.ref, d, Onion.createSinglePartPayload(defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret.get), 2)
|
||||
val request = SendPaymentToNode(sender.ref, d, Onion.createSinglePartPayload(defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret.get), 2, routeParams = defaultRouteParams)
|
||||
sender.send(paymentFSM, request)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && nodeParams.db.payments.getOutgoingPayment(id).exists(_.status === OutgoingPaymentStatus.Pending))
|
||||
|
||||
|
@ -350,7 +351,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
import payFixture._
|
||||
import cfg._
|
||||
|
||||
val request = SendPaymentToNode(sender.ref, d, Onion.createSinglePartPayload(defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret.get), 2)
|
||||
val request = SendPaymentToNode(sender.ref, d, Onion.createSinglePartPayload(defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret.get), 2, routeParams = defaultRouteParams)
|
||||
sender.send(paymentFSM, request)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && nodeParams.db.payments.getOutgoingPayment(id).exists(_.status === OutgoingPaymentStatus.Pending))
|
||||
|
||||
|
@ -373,7 +374,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
val payFixture = createPaymentLifecycle()
|
||||
import payFixture._
|
||||
|
||||
val request = SendPaymentToNode(sender.ref, d, Onion.createSinglePartPayload(defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret.get), 2)
|
||||
val request = SendPaymentToNode(sender.ref, d, Onion.createSinglePartPayload(defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret.get), 2, routeParams = defaultRouteParams)
|
||||
sender.send(paymentFSM, request)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE)
|
||||
val WaitingForRoute(_, Nil, _) = paymentFSM.stateData
|
||||
|
@ -403,7 +404,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
import payFixture._
|
||||
import cfg._
|
||||
|
||||
val request = SendPaymentToNode(sender.ref, d, Onion.createSinglePartPayload(defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret.get), 5)
|
||||
val request = SendPaymentToNode(sender.ref, d, Onion.createSinglePartPayload(defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret.get), 5, routeParams = defaultRouteParams)
|
||||
sender.send(paymentFSM, request)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && nodeParams.db.payments.getOutgoingPayment(id).exists(_.status === OutgoingPaymentStatus.Pending))
|
||||
|
||||
|
@ -455,7 +456,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
val payFixture = createPaymentLifecycle()
|
||||
import payFixture._
|
||||
|
||||
val request = SendPaymentToNode(sender.ref, d, Onion.createSinglePartPayload(defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret.get), 1)
|
||||
val request = SendPaymentToNode(sender.ref, d, Onion.createSinglePartPayload(defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret.get), 1, routeParams = defaultRouteParams)
|
||||
sender.send(paymentFSM, request)
|
||||
routerForwarder.expectMsg(defaultRouteRequest(nodeParams.nodeId, d, cfg))
|
||||
routerForwarder.forward(routerFixture.router)
|
||||
|
@ -485,7 +486,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
ExtraHop(c, channelId_cd, update_cd.feeBaseMsat, update_cd.feeProportionalMillionths, update_cd.cltvExpiryDelta)
|
||||
))
|
||||
|
||||
val request = SendPaymentToNode(sender.ref, d, Onion.createSinglePartPayload(defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret.get), 5, assistedRoutes = assistedRoutes)
|
||||
val request = SendPaymentToNode(sender.ref, d, Onion.createSinglePartPayload(defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret.get), 5, assistedRoutes = assistedRoutes, routeParams = defaultRouteParams)
|
||||
sender.send(paymentFSM, request)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && nodeParams.db.payments.getOutgoingPayment(id).exists(_.status === OutgoingPaymentStatus.Pending))
|
||||
|
||||
|
@ -526,7 +527,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
|
||||
// we build an assisted route for channel cd
|
||||
val assistedRoutes = Seq(Seq(ExtraHop(c, channelId_cd, update_cd.feeBaseMsat, update_cd.feeProportionalMillionths, update_cd.cltvExpiryDelta)))
|
||||
val request = SendPaymentToNode(sender.ref, d, Onion.createSinglePartPayload(defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret.get), 1, assistedRoutes = assistedRoutes)
|
||||
val request = SendPaymentToNode(sender.ref, d, Onion.createSinglePartPayload(defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret.get), 1, assistedRoutes = assistedRoutes, routeParams = defaultRouteParams)
|
||||
sender.send(paymentFSM, request)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && nodeParams.db.payments.getOutgoingPayment(id).exists(_.status === OutgoingPaymentStatus.Pending))
|
||||
|
||||
|
@ -551,7 +552,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
import payFixture._
|
||||
import cfg._
|
||||
|
||||
val request = SendPaymentToNode(sender.ref, d, Onion.createSinglePartPayload(defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret.get), 2)
|
||||
val request = SendPaymentToNode(sender.ref, d, Onion.createSinglePartPayload(defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret.get), 2, routeParams = defaultRouteParams)
|
||||
sender.send(paymentFSM, request)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && nodeParams.db.payments.getOutgoingPayment(id).exists(_.status === OutgoingPaymentStatus.Pending))
|
||||
|
||||
|
@ -589,7 +590,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
import payFixture._
|
||||
import cfg._
|
||||
|
||||
val request = SendPaymentToNode(sender.ref, d, Onion.createSinglePartPayload(defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret.get), 5)
|
||||
val request = SendPaymentToNode(sender.ref, d, Onion.createSinglePartPayload(defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret.get), 5, routeParams = defaultRouteParams)
|
||||
sender.send(paymentFSM, request)
|
||||
routerForwarder.expectMsgType[RouteRequest]
|
||||
val Transition(_, WAITING_FOR_REQUEST, WAITING_FOR_ROUTE) = monitor.expectMsgClass(classOf[Transition[_]])
|
||||
|
@ -636,7 +637,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
import payFixture._
|
||||
|
||||
// we send a payment to H
|
||||
val request = SendPaymentToNode(sender.ref, h, Onion.createSinglePartPayload(defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret.get), 5)
|
||||
val request = SendPaymentToNode(sender.ref, h, Onion.createSinglePartPayload(defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret.get), 5, routeParams = defaultRouteParams)
|
||||
sender.send(paymentFSM, request)
|
||||
routerForwarder.expectMsgType[RouteRequest]
|
||||
|
||||
|
@ -713,7 +714,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
import payFixture._
|
||||
import cfg._
|
||||
|
||||
val request = SendPaymentToNode(sender.ref, d, Onion.createSinglePartPayload(defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret.get), 3)
|
||||
val request = SendPaymentToNode(sender.ref, d, Onion.createSinglePartPayload(defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret.get), 3, routeParams = defaultRouteParams)
|
||||
sender.send(paymentFSM, request)
|
||||
routerForwarder.expectMsgType[RouteRequest]
|
||||
val Transition(_, WAITING_FOR_REQUEST, WAITING_FOR_ROUTE) = monitor.expectMsgClass(classOf[Transition[_]])
|
||||
|
|
|
@ -486,7 +486,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl
|
|||
nodeRelayer ! NodeRelay.Relay(incomingSinglePart)
|
||||
|
||||
val routeRequest = router.expectMessageType[RouteRequest]
|
||||
val routeParams = routeRequest.routeParams.get
|
||||
val routeParams = routeRequest.routeParams
|
||||
assert(routeParams.maxFeePct === 0) // should be disabled
|
||||
assert(routeParams.maxFeeBase === incomingAmount - outgoingAmount)
|
||||
assert(routeParams.routeMaxCltv === incomingSinglePart.add.cltvExpiry - outgoingExpiry)
|
||||
|
@ -582,7 +582,6 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl
|
|||
assert(outgoingPayment.targetExpiry === outgoingExpiry)
|
||||
assert(outgoingPayment.targetNodeId === outgoingNodeId)
|
||||
assert(outgoingPayment.additionalTlvs === Nil)
|
||||
assert(outgoingPayment.routeParams.isDefined)
|
||||
assert(outgoingPayment.assistedRoutes === hints)
|
||||
// those are adapters for pay-fsm messages
|
||||
val nodeRelayerAdapters = outgoingPayment.replyTo
|
||||
|
@ -622,7 +621,6 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl
|
|||
assert(outgoingPayment.finalPayload.amount === outgoingAmount)
|
||||
assert(outgoingPayment.finalPayload.expiry === outgoingExpiry)
|
||||
assert(outgoingPayment.targetNodeId === outgoingNodeId)
|
||||
assert(outgoingPayment.routeParams.isDefined)
|
||||
assert(outgoingPayment.assistedRoutes === hints)
|
||||
// those are adapters for pay-fsm messages
|
||||
val nodeRelayerAdapters = outgoingPayment.replyTo
|
||||
|
@ -679,7 +677,6 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl
|
|||
assert(outgoingPayment.targetExpiry === outgoingExpiry)
|
||||
assert(outgoingPayment.targetNodeId === outgoingNodeId)
|
||||
assert(outgoingPayment.additionalTlvs === Seq(OnionTlv.TrampolineOnion(nextTrampolinePacket)))
|
||||
assert(outgoingPayment.routeParams.isDefined)
|
||||
assert(outgoingPayment.assistedRoutes === Nil)
|
||||
}
|
||||
|
||||
|
|
|
@ -29,7 +29,7 @@ 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.BaseRouterSpec.channelAnnouncement
|
||||
import fr.acinq.eclair.router.RouteCalculationSpec.{DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE}
|
||||
import fr.acinq.eclair.router.RouteCalculationSpec.{DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, DEFAULT_ROUTE_PARAMS}
|
||||
import fr.acinq.eclair.router.Router._
|
||||
import fr.acinq.eclair.transactions.Scripts
|
||||
import fr.acinq.eclair.wire.protocol._
|
||||
|
@ -314,7 +314,7 @@ class RouterSpec extends BaseRouterSpec {
|
|||
import fixture._
|
||||
val sender = TestProbe()
|
||||
// no route a->f
|
||||
sender.send(router, RouteRequest(a, f, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE))
|
||||
sender.send(router, RouteRequest(a, f, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, routeParams = DEFAULT_ROUTE_PARAMS))
|
||||
sender.expectMsg(Failure(RouteNotFound))
|
||||
}
|
||||
|
||||
|
@ -322,7 +322,7 @@ class RouterSpec extends BaseRouterSpec {
|
|||
import fixture._
|
||||
val sender = TestProbe()
|
||||
// no route a->f
|
||||
sender.send(router, RouteRequest(randomKey().publicKey, f, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE))
|
||||
sender.send(router, RouteRequest(randomKey().publicKey, f, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, routeParams = DEFAULT_ROUTE_PARAMS))
|
||||
sender.expectMsg(Failure(RouteNotFound))
|
||||
}
|
||||
|
||||
|
@ -330,19 +330,19 @@ class RouterSpec extends BaseRouterSpec {
|
|||
import fixture._
|
||||
val sender = TestProbe()
|
||||
// no route a->f
|
||||
sender.send(router, RouteRequest(a, randomKey().publicKey, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE))
|
||||
sender.send(router, RouteRequest(a, randomKey().publicKey, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, routeParams = DEFAULT_ROUTE_PARAMS))
|
||||
sender.expectMsg(Failure(RouteNotFound))
|
||||
}
|
||||
|
||||
test("route found") { fixture =>
|
||||
import fixture._
|
||||
val sender = TestProbe()
|
||||
sender.send(router, RouteRequest(a, d, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE))
|
||||
sender.send(router, RouteRequest(a, d, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, routeParams = DEFAULT_ROUTE_PARAMS))
|
||||
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, DEFAULT_MAX_FEE))
|
||||
sender.send(router, RouteRequest(a, h, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, routeParams = DEFAULT_ROUTE_PARAMS))
|
||||
val res1 = sender.expectMsgType[RouteResponse]
|
||||
assert(res1.routes.head.hops.map(_.nodeId).toList === a :: g :: Nil)
|
||||
assert(res1.routes.head.hops.last.nextNodeId === h)
|
||||
|
@ -357,7 +357,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, DEFAULT_MAX_FEE, 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), routeParams = DEFAULT_ROUTE_PARAMS))
|
||||
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)
|
||||
|
@ -367,7 +367,7 @@ class RouterSpec extends BaseRouterSpec {
|
|||
import fixture._
|
||||
val sender = TestProbe()
|
||||
val peerConnection = TestProbe()
|
||||
sender.send(router, RouteRequest(a, d, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE))
|
||||
sender.send(router, RouteRequest(a, d, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, routeParams = DEFAULT_ROUTE_PARAMS))
|
||||
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)
|
||||
|
@ -375,21 +375,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, DEFAULT_MAX_FEE))
|
||||
sender.send(router, RouteRequest(a, d, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, routeParams = DEFAULT_ROUTE_PARAMS))
|
||||
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, DEFAULT_MAX_FEE))
|
||||
sender.send(router, RouteRequest(a, h, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, routeParams = DEFAULT_ROUTE_PARAMS))
|
||||
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, None, CommitmentsSpec.makeCommitments(10000 msat, 15000 msat, a, g, announceChannel = false)))
|
||||
sender.send(router, RouteRequest(a, h, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE))
|
||||
sender.send(router, RouteRequest(a, h, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, routeParams = DEFAULT_ROUTE_PARAMS))
|
||||
sender.expectMsg(Failure(RouteNotFound))
|
||||
}
|
||||
|
||||
|
@ -398,44 +398,44 @@ class RouterSpec extends BaseRouterSpec {
|
|||
val sender = TestProbe()
|
||||
|
||||
// Via private channels.
|
||||
sender.send(router, RouteRequest(a, g, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE))
|
||||
sender.send(router, RouteRequest(a, g, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, routeParams = DEFAULT_ROUTE_PARAMS))
|
||||
sender.expectMsgType[RouteResponse]
|
||||
sender.send(router, RouteRequest(a, g, 50000000 msat, Long.MaxValue.msat))
|
||||
sender.send(router, RouteRequest(a, g, 50000000 msat, Long.MaxValue.msat, routeParams = DEFAULT_ROUTE_PARAMS))
|
||||
sender.expectMsg(Failure(BalanceTooLow))
|
||||
sender.send(router, RouteRequest(a, g, 50000000 msat, Long.MaxValue.msat, allowMultiPart = true))
|
||||
sender.send(router, RouteRequest(a, g, 50000000 msat, Long.MaxValue.msat, allowMultiPart = true, routeParams = DEFAULT_ROUTE_PARAMS))
|
||||
sender.expectMsg(Failure(BalanceTooLow))
|
||||
|
||||
// Via public channels.
|
||||
sender.send(router, RouteRequest(a, b, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE))
|
||||
sender.send(router, RouteRequest(a, b, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, routeParams = DEFAULT_ROUTE_PARAMS))
|
||||
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, None, commitments1))
|
||||
sender.send(router, RouteRequest(a, b, 12000000 msat, Long.MaxValue.msat))
|
||||
sender.send(router, RouteRequest(a, b, 12000000 msat, Long.MaxValue.msat, routeParams = DEFAULT_ROUTE_PARAMS))
|
||||
sender.expectMsg(Failure(BalanceTooLow))
|
||||
sender.send(router, RouteRequest(a, b, 12000000 msat, Long.MaxValue.msat, allowMultiPart = true))
|
||||
sender.send(router, RouteRequest(a, b, 12000000 msat, Long.MaxValue.msat, allowMultiPart = true, routeParams = DEFAULT_ROUTE_PARAMS))
|
||||
sender.expectMsg(Failure(BalanceTooLow))
|
||||
sender.send(router, RouteRequest(a, b, 5000000 msat, Long.MaxValue.msat))
|
||||
sender.send(router, RouteRequest(a, b, 5000000 msat, Long.MaxValue.msat, routeParams = DEFAULT_ROUTE_PARAMS))
|
||||
sender.expectMsgType[RouteResponse]
|
||||
sender.send(router, RouteRequest(a, b, 5000000 msat, Long.MaxValue.msat, allowMultiPart = true))
|
||||
sender.send(router, RouteRequest(a, b, 5000000 msat, Long.MaxValue.msat, allowMultiPart = true, routeParams = DEFAULT_ROUTE_PARAMS))
|
||||
sender.expectMsgType[RouteResponse]
|
||||
}
|
||||
|
||||
test("temporary channel exclusion") { fixture =>
|
||||
import fixture._
|
||||
val sender = TestProbe()
|
||||
sender.send(router, RouteRequest(a, d, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE))
|
||||
sender.send(router, RouteRequest(a, d, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, routeParams = DEFAULT_ROUTE_PARAMS))
|
||||
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, DEFAULT_MAX_FEE))
|
||||
sender.send(router, RouteRequest(a, d, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, routeParams = DEFAULT_ROUTE_PARAMS))
|
||||
sender.expectMsg(Failure(RouteNotFound))
|
||||
// note that cb is still available!
|
||||
sender.send(router, RouteRequest(d, a, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE))
|
||||
sender.send(router, RouteRequest(d, a, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, routeParams = DEFAULT_ROUTE_PARAMS))
|
||||
sender.expectMsgType[RouteResponse]
|
||||
// let's remove the exclusion
|
||||
sender.send(router, LiftChannelExclusion(bc))
|
||||
sender.send(router, RouteRequest(a, d, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE))
|
||||
sender.send(router, RouteRequest(a, d, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, routeParams = DEFAULT_ROUTE_PARAMS))
|
||||
sender.expectMsgType[RouteResponse]
|
||||
}
|
||||
|
||||
|
|
|
@ -17,7 +17,6 @@
|
|||
package fr.acinq.eclair.gui
|
||||
|
||||
import java.util.UUID
|
||||
|
||||
import akka.pattern.{AskTimeoutException, ask}
|
||||
import akka.util.Timeout
|
||||
import fr.acinq.eclair.blockchain.fee.FeeratePerKB
|
||||
|
@ -25,7 +24,8 @@ import fr.acinq.eclair.gui.controllers._
|
|||
import fr.acinq.eclair.io.{NodeURI, Peer}
|
||||
import fr.acinq.eclair.payment._
|
||||
import fr.acinq.eclair.payment.receive.MultiPartHandler.ReceivePayment
|
||||
import fr.acinq.eclair.payment.send.PaymentInitiator.SendPayment
|
||||
import fr.acinq.eclair.payment.send.PaymentInitiator.SendPaymentToNode
|
||||
import fr.acinq.eclair.router.RouteCalculation
|
||||
import fr.acinq.eclair.{MilliSatoshi, _}
|
||||
import grizzled.slf4j.Logging
|
||||
|
||||
|
@ -85,9 +85,10 @@ class Handlers(fKit: Future[Kit])(implicit ec: ExecutionContext = ExecutionConte
|
|||
logger.info(s"sending $amountMsat to ${req.paymentHash} @ ${req.nodeId}")
|
||||
(for {
|
||||
kit <- fKit
|
||||
routeParams = RouteCalculation.getDefaultRouteParams(kit.nodeParams.routerConf.pathFindingConf)
|
||||
sendPayment = req.minFinalCltvExpiryDelta match {
|
||||
case None => SendPayment(MilliSatoshi(amountMsat), req, kit.nodeParams.maxPaymentAttempts, assistedRoutes = req.routingInfo)
|
||||
case Some(minFinalCltvExpiry) => SendPayment(MilliSatoshi(amountMsat), req, kit.nodeParams.maxPaymentAttempts, assistedRoutes = req.routingInfo, fallbackFinalExpiryDelta = minFinalCltvExpiry)
|
||||
case None => SendPaymentToNode(MilliSatoshi(amountMsat), req, kit.nodeParams.maxPaymentAttempts, assistedRoutes = req.routingInfo, routeParams = routeParams)
|
||||
case Some(minFinalCltvExpiry) => SendPaymentToNode(MilliSatoshi(amountMsat), req, kit.nodeParams.maxPaymentAttempts, assistedRoutes = req.routingInfo, fallbackFinalExpiryDelta = minFinalCltvExpiry, routeParams = routeParams)
|
||||
}
|
||||
res <- (kit.paymentInitiator ? sendPayment).mapTo[UUID]
|
||||
} yield res).recover {
|
||||
|
|
Loading…
Add table
Reference in a new issue