mirror of
https://github.com/ACINQ/eclair.git
synced 2025-02-23 06:35:11 +01:00
Proper management of expiry-related errors (#469)
Make `Commitment` return `ExpiryTooSmall` and `ExpiryTooBig` when appropriate, and don't do the check in the `Relayer`. Be more restrictive when sending HTLCs, so that counterparty doesn't close the channel when a block just appeared and there is temporarily a 1-block discrepancy between two peers. Proper management of `FinalExpiryTooSoon` in the payment handler. On top of that, added more tests and simplified some.
This commit is contained in:
parent
203288c415
commit
85592de0e5
15 changed files with 210 additions and 164 deletions
|
@ -42,6 +42,10 @@ object Channel {
|
|||
// we won't exchange more than this many signatures when negotiating the closing fee
|
||||
val MAX_NEGOTIATION_ITERATIONS = 20
|
||||
|
||||
// this is defined in BOLT 11
|
||||
val MIN_CLTV_EXPIRY = 9L
|
||||
val MAX_CLTV_EXPIRY = 7 * 144L // one week
|
||||
|
||||
case object TickRefreshChannelUpdate
|
||||
}
|
||||
|
||||
|
|
|
@ -36,7 +36,8 @@ case class HtlcSigCountMismatch (override val channelId: BinaryDa
|
|||
case class ForcedLocalCommit (override val channelId: BinaryData, reason: String) extends ChannelException(channelId, s"forced local commit: reason")
|
||||
case class UnexpectedHtlcId (override val channelId: BinaryData, expected: Long, actual: Long) extends ChannelException(channelId, s"unexpected htlc id: expected=$expected actual=$actual")
|
||||
case class InvalidPaymentHash (override val channelId: BinaryData) extends ChannelException(channelId, "invalid payment hash")
|
||||
case class ExpiryTooSmall (override val channelId: BinaryData, minimum: Long, actual: Long, blockCount: Long) extends ChannelException(channelId, s"expiry too small: required=$minimum actual=$actual blockCount=$blockCount")
|
||||
case class ExpiryTooSmall (override val channelId: BinaryData, minimum: Long, actual: Long, blockCount: Long) extends ChannelException(channelId, s"expiry too small: minimum=$minimum actual=$actual blockCount=$blockCount")
|
||||
case class ExpiryTooBig (override val channelId: BinaryData, maximum: Long, actual: Long, blockCount: Long) extends ChannelException(channelId, s"expiry too big: maximum=$maximum actual=$actual blockCount=$blockCount")
|
||||
case class ExpiryCannotBeInThePast (override val channelId: BinaryData, expiry: Long, blockCount: Long) extends ChannelException(channelId, s"expiry can't be in the past: expiry=$expiry blockCount=$blockCount")
|
||||
case class HtlcValueTooSmall (override val channelId: BinaryData, minimum: Long, actual: Long) extends ChannelException(channelId, s"htlc value too small: minimum=$minimum actual=$actual")
|
||||
case class HtlcValueTooHighInFlight (override val channelId: BinaryData, maximum: UInt64, actual: UInt64) extends ChannelException(channelId, s"in-flight htlcs hold too much value: maximum=$maximum actual=$actual")
|
||||
|
|
|
@ -4,7 +4,7 @@ import akka.event.LoggingAdapter
|
|||
import fr.acinq.bitcoin.Crypto.{Point, PrivateKey, sha256}
|
||||
import fr.acinq.bitcoin.{BinaryData, Crypto, Satoshi, Transaction}
|
||||
import fr.acinq.eclair.crypto.{Generators, ShaChain, Sphinx}
|
||||
import fr.acinq.eclair.payment.Origin
|
||||
import fr.acinq.eclair.payment.{Origin, PaymentLifecycle}
|
||||
import fr.acinq.eclair.transactions.Transactions._
|
||||
import fr.acinq.eclair.transactions._
|
||||
import fr.acinq.eclair.wire._
|
||||
|
@ -87,6 +87,16 @@ object Commitments {
|
|||
if (cmd.expiry <= blockCount) {
|
||||
return Left(ExpiryCannotBeInThePast(commitments.channelId, cmd.expiry, blockCount))
|
||||
}
|
||||
val minExpiry = blockCount + Channel.MIN_CLTV_EXPIRY
|
||||
// we reject expiry=minExpiry, because if a new block has just been found maybe the counterparty will get notified before us, consider that the expiry is too soon and close the channel
|
||||
if (cmd.expiry <= minExpiry) {
|
||||
return Left(ExpiryTooSmall(commitments.channelId, minimum = minExpiry, actual = cmd.expiry, blockCount = blockCount))
|
||||
}
|
||||
val maxExpiry = blockCount + Channel.MAX_CLTV_EXPIRY
|
||||
// we reject expiry=maxExpiry, because if a new block has just been found maybe the counterparty will get notified before us, consider that the expiry is too big and close the channel
|
||||
if (cmd.expiry >= maxExpiry) {
|
||||
return Left(ExpiryTooBig(commitments.channelId, maximum = maxExpiry, actual = cmd.expiry, blockCount = blockCount))
|
||||
}
|
||||
|
||||
if (cmd.amountMsat < commitments.remoteParams.htlcMinimumMsat) {
|
||||
return Left(HtlcValueTooSmall(commitments.channelId, minimum = commitments.remoteParams.htlcMinimumMsat, actual = cmd.amountMsat))
|
||||
|
@ -133,11 +143,18 @@ object Commitments {
|
|||
}
|
||||
|
||||
val blockCount = Globals.blockCount.get()
|
||||
// we need a reasonable amount of time to pull the funds before the sender can get refunded
|
||||
val minExpiry = blockCount + 3
|
||||
if (add.expiry <= blockCount) {
|
||||
throw ExpiryCannotBeInThePast(commitments.channelId, add.expiry, blockCount)
|
||||
}
|
||||
// we need a reasonable amount of time to pull the funds before the sender can get refunded (see BOLT 2 and BOLT 11 for a calculation and rationale)
|
||||
val minExpiry = blockCount + Channel.MIN_CLTV_EXPIRY
|
||||
if (add.expiry < minExpiry) {
|
||||
throw ExpiryTooSmall(commitments.channelId, minimum = minExpiry, actual = add.expiry, blockCount = blockCount)
|
||||
}
|
||||
val maxExpiry = blockCount + Channel.MAX_CLTV_EXPIRY
|
||||
if (add.expiry > maxExpiry) {
|
||||
throw ExpiryTooBig(commitments.channelId, maximum = maxExpiry, actual = add.expiry, blockCount = blockCount)
|
||||
}
|
||||
|
||||
if (add.amountMsat < commitments.localParams.htlcMinimumMsat) {
|
||||
throw HtlcValueTooSmall(commitments.channelId, minimum = commitments.localParams.htlcMinimumMsat, actual = add.amountMsat)
|
||||
|
|
|
@ -21,8 +21,7 @@ trait PaymentsDb {
|
|||
|
||||
def addPayment(payment: Payment)
|
||||
|
||||
@throws(classOf[NoSuchElementException])
|
||||
def findByPaymentHash(paymentHash: BinaryData): Payment
|
||||
def findByPaymentHash(paymentHash: BinaryData): Option[Payment]
|
||||
|
||||
def listPayments(): Seq[Payment]
|
||||
|
||||
|
|
|
@ -35,15 +35,14 @@ class SqlitePaymentsDb(sqlite: Connection) extends PaymentsDb with Logging {
|
|||
}
|
||||
}
|
||||
|
||||
@throws(classOf[NoSuchElementException])
|
||||
override def findByPaymentHash(paymentHash: BinaryData): Payment = {
|
||||
override def findByPaymentHash(paymentHash: BinaryData): Option[Payment] = {
|
||||
using(sqlite.prepareStatement("SELECT payment_hash, amount_msat, timestamp FROM payments WHERE payment_hash = ?")) { statement =>
|
||||
statement.setBytes(1, paymentHash)
|
||||
val rs = statement.executeQuery()
|
||||
if (rs.next()) {
|
||||
Payment(BinaryData(rs.getBytes("payment_hash")), rs.getLong("amount_msat"), rs.getLong("timestamp"))
|
||||
Some(Payment(BinaryData(rs.getBytes("payment_hash")), rs.getLong("amount_msat"), rs.getLong("timestamp")))
|
||||
} else {
|
||||
throw new NoSuchElementException("payment not found")
|
||||
None
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -2,11 +2,12 @@ package fr.acinq.eclair.payment
|
|||
|
||||
import akka.actor.{Actor, ActorLogging, Props, Status}
|
||||
import fr.acinq.bitcoin.{BinaryData, Crypto, MilliSatoshi}
|
||||
import fr.acinq.eclair.channel.{CMD_FAIL_HTLC, CMD_FULFILL_HTLC}
|
||||
import fr.acinq.eclair.channel.{CMD_FAIL_HTLC, CMD_FULFILL_HTLC, Channel}
|
||||
import fr.acinq.eclair.db.Payment
|
||||
import fr.acinq.eclair.wire._
|
||||
|
||||
import scala.concurrent.duration._
|
||||
import fr.acinq.eclair.{NodeParams, randomBytes}
|
||||
import fr.acinq.eclair.{Globals, NodeParams, randomBytes}
|
||||
|
||||
import scala.compat.Platform
|
||||
import scala.concurrent.ExecutionContext
|
||||
|
@ -21,57 +22,58 @@ class LocalPaymentHandler(nodeParams: NodeParams)(implicit ec: ExecutionContext
|
|||
|
||||
override def receive: Receive = run(Map())
|
||||
|
||||
def run(h2r: Map[BinaryData, (BinaryData, PaymentRequest)]): Receive = {
|
||||
def run(hash2preimage: Map[BinaryData, (BinaryData, PaymentRequest)]): Receive = {
|
||||
|
||||
case currentSeconds: Long =>
|
||||
context.become(run(h2r.collect {
|
||||
context.become(run(hash2preimage.collect {
|
||||
case e@(_, (_, pr)) if pr.expiry.isEmpty => e // requests that don't expire are kept forever
|
||||
case e@(_, (_, pr)) if pr.timestamp + pr.expiry.get > currentSeconds => e // clean up expired requests
|
||||
}))
|
||||
|
||||
case ReceivePayment(amount_opt, desc) =>
|
||||
Try {
|
||||
if (h2r.size > nodeParams.maxPendingPaymentRequests) {
|
||||
if (hash2preimage.size > nodeParams.maxPendingPaymentRequests) {
|
||||
throw new RuntimeException(s"too many pending payment requests (max=${nodeParams.maxPendingPaymentRequests})")
|
||||
}
|
||||
val paymentPreimage = randomBytes(32)
|
||||
val paymentHash = Crypto.sha256(paymentPreimage)
|
||||
(paymentPreimage, paymentHash, PaymentRequest(nodeParams.chainHash, amount_opt, paymentHash, nodeParams.privateKey, desc, fallbackAddress = None, expirySeconds = Some(nodeParams.paymentRequestExpiry.toSeconds)))
|
||||
} match {
|
||||
case Success((r, h, pr)) =>
|
||||
log.debug(s"generated payment request=${PaymentRequest.write(pr)} from amount=$amount_opt")
|
||||
sender ! pr
|
||||
context.become(run(h2r + (h -> (r, pr))))
|
||||
case Failure(t) =>
|
||||
sender ! Status.Failure(t)
|
||||
}
|
||||
val paymentRequest = PaymentRequest(nodeParams.chainHash, amount_opt, paymentHash, nodeParams.privateKey, desc, fallbackAddress = None, expirySeconds = Some(nodeParams.paymentRequestExpiry.toSeconds))
|
||||
log.debug(s"generated payment request=${PaymentRequest.write(paymentRequest)} from amount=$amount_opt")
|
||||
sender ! paymentRequest
|
||||
context.become(run(hash2preimage + (paymentHash -> (paymentPreimage, paymentRequest))))
|
||||
} recover { case t => sender ! Status.Failure(t) }
|
||||
|
||||
case CheckPayment(paymentHash) =>
|
||||
val found: Boolean = Try(nodeParams.paymentsDb.findByPaymentHash(paymentHash)) match {
|
||||
case Success(s) if paymentHash == s.payment_hash => true
|
||||
case _ => false
|
||||
nodeParams.paymentsDb.findByPaymentHash(paymentHash) match {
|
||||
case Some(_) => sender ! true
|
||||
case _ => sender ! false
|
||||
}
|
||||
sender ! found
|
||||
|
||||
case htlc: UpdateAddHtlc =>
|
||||
if (h2r.contains(htlc.paymentHash)) {
|
||||
val r = h2r(htlc.paymentHash)._1
|
||||
val pr = h2r(htlc.paymentHash)._2
|
||||
hash2preimage.get(htlc.paymentHash) match {
|
||||
case Some((paymentPreimage, paymentRequest)) =>
|
||||
val minFinalExpiry = Globals.blockCount.get() + paymentRequest.minFinalCltvExpiry.getOrElse(Channel.MIN_CLTV_EXPIRY)
|
||||
// The htlc amount must be equal or greater than the requested amount. A slight overpaying is permitted, however
|
||||
// it must not be greater than two times the requested amount.
|
||||
// see https://github.com/lightningnetwork/lightning-rfc/blob/master/04-onion-routing.md#failure-messages
|
||||
pr.amount match {
|
||||
case Some(amount) if MilliSatoshi(htlc.amountMsat) < amount => sender ! CMD_FAIL_HTLC(htlc.id, Right(IncorrectPaymentAmount), commit = true)
|
||||
case Some(amount) if MilliSatoshi(htlc.amountMsat) > amount * 2 => sender ! CMD_FAIL_HTLC(htlc.id, Right(IncorrectPaymentAmount), commit = true)
|
||||
paymentRequest.amount match {
|
||||
case _ if htlc.expiry < minFinalExpiry =>
|
||||
sender ! CMD_FAIL_HTLC(htlc.id, Right(FinalExpiryTooSoon), commit = true)
|
||||
case Some(amount) if MilliSatoshi(htlc.amountMsat) < amount =>
|
||||
log.warning(s"received payment with amount too small for paymentHash=${htlc.paymentHash} amountMsat=${htlc.amountMsat}")
|
||||
sender ! CMD_FAIL_HTLC(htlc.id, Right(IncorrectPaymentAmount), commit = true)
|
||||
case Some(amount) if MilliSatoshi(htlc.amountMsat) > amount * 2 =>
|
||||
log.warning(s"received payment with amount too large for paymentHash=${htlc.paymentHash} amountMsat=${htlc.amountMsat}")
|
||||
sender ! CMD_FAIL_HTLC(htlc.id, Right(IncorrectPaymentAmount), commit = true)
|
||||
case _ =>
|
||||
log.info(s"received payment for paymentHash=${htlc.paymentHash} amountMsat=${htlc.amountMsat}")
|
||||
// amount is correct or was not specified in the payment request
|
||||
nodeParams.paymentsDb.addPayment(Payment(htlc.paymentHash, htlc.amountMsat, Platform.currentTime / 1000))
|
||||
sender ! CMD_FULFILL_HTLC(htlc.id, r, commit = true)
|
||||
sender ! CMD_FULFILL_HTLC(htlc.id, paymentPreimage, commit = true)
|
||||
context.system.eventStream.publish(PaymentReceived(MilliSatoshi(htlc.amountMsat), htlc.paymentHash))
|
||||
context.become(run(h2r - htlc.paymentHash))
|
||||
context.become(run(hash2preimage - htlc.paymentHash))
|
||||
}
|
||||
} else {
|
||||
case None =>
|
||||
sender ! CMD_FAIL_HTLC(htlc.id, Right(UnknownPaymentHash), commit = true)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -4,7 +4,7 @@ import akka.actor.{ActorRef, FSM, Props, Status}
|
|||
import fr.acinq.bitcoin.Crypto.PublicKey
|
||||
import fr.acinq.bitcoin.{BinaryData, MilliSatoshi}
|
||||
import fr.acinq.eclair._
|
||||
import fr.acinq.eclair.channel.{AddHtlcFailed, CMD_ADD_HTLC, Register}
|
||||
import fr.acinq.eclair.channel.{AddHtlcFailed, CMD_ADD_HTLC, Channel, Register}
|
||||
import fr.acinq.eclair.crypto.Sphinx
|
||||
import fr.acinq.eclair.crypto.Sphinx.{ErrorPacket, Packet}
|
||||
import fr.acinq.eclair.payment.PaymentRequest.ExtraHop
|
||||
|
@ -16,7 +16,10 @@ import scala.util.{Failure, Success}
|
|||
|
||||
// @formatter:off
|
||||
case class ReceivePayment(amountMsat_opt: Option[MilliSatoshi], description: String)
|
||||
case class SendPayment(amountMsat: Long, paymentHash: BinaryData, targetNodeId: PublicKey, assistedRoutes: Seq[Seq[ExtraHop]] = Nil, minFinalCltvExpiry: Long = PaymentLifecycle.defaultMinFinalCltvExpiry, maxAttempts: Int = 5)
|
||||
/**
|
||||
* @param finalCltvExpiry by default we choose finalCltvExpiry = Channel.MIN_CLTV_EXPIRY + 1 to not have our htlc fail when a new block has just been found
|
||||
*/
|
||||
case class SendPayment(amountMsat: Long, paymentHash: BinaryData, targetNodeId: PublicKey, assistedRoutes: Seq[Seq[ExtraHop]] = Nil, finalCltvExpiry: Long = Channel.MIN_CLTV_EXPIRY + 1, maxAttempts: Int = 5)
|
||||
case class CheckPayment(paymentHash: BinaryData)
|
||||
|
||||
sealed trait PaymentResult
|
||||
|
@ -58,7 +61,7 @@ class PaymentLifecycle(sourceNodeId: PublicKey, router: ActorRef, register: Acto
|
|||
case Event(RouteResponse(hops, ignoreNodes, ignoreChannels), WaitingForRoute(s, c, failures)) =>
|
||||
log.info(s"route found: attempt=${failures.size + 1}/${c.maxAttempts} route=${hops.map(_.nextNodeId).mkString("->")} channels=${hops.map(_.lastUpdate.shortChannelId.toHexString).mkString("->")}")
|
||||
val firstHop = hops.head
|
||||
val finalExpiry = Globals.blockCount.get().toInt + c.minFinalCltvExpiry.toInt
|
||||
val finalExpiry = Globals.blockCount.get().toInt + c.finalCltvExpiry.toInt
|
||||
val (cmd, sharedSecrets) = buildCommand(c.amountMsat, finalExpiry, c.paymentHash, hops)
|
||||
register ! Register.ForwardShortId(firstHop.lastUpdate.shortChannelId, cmd)
|
||||
goto(WAITING_FOR_PAYMENT_COMPLETE) using WaitingForComplete(s, c, cmd, failures, sharedSecrets, ignoreNodes, ignoreChannels, hops)
|
||||
|
@ -213,9 +216,6 @@ object PaymentLifecycle {
|
|||
(msat + nextFee, expiry + hop.lastUpdate.cltvExpiryDelta, PerHopPayload(hop.lastUpdate.shortChannelId, msat, expiry) +: payloads)
|
||||
}
|
||||
|
||||
// this is defined in BOLT 11
|
||||
val defaultMinFinalCltvExpiry:Long = 9L
|
||||
|
||||
def buildCommand(finalAmountMsat: Long, finalExpiry: Long, paymentHash: BinaryData, hops: Seq[Hop]): (CMD_ADD_HTLC, Seq[(BinaryData, PublicKey)]) = {
|
||||
val (firstAmountMsat, firstExpiry, payloads) = buildPayloads(finalAmountMsat, finalExpiry, hops.drop(1))
|
||||
val nodes = hops.map(_.nextNodeId)
|
||||
|
|
|
@ -64,8 +64,6 @@ class Relayer(nodeParams: NodeParams, register: ActorRef, paymentHandler: ActorR
|
|||
Left(CMD_FAIL_HTLC(add.id, Right(FinalIncorrectHtlcAmount(add.amountMsat)), commit = true))
|
||||
case PerHopPayload(_, _, finalOutgoingCltvValue) if finalOutgoingCltvValue != add.expiry =>
|
||||
Left(CMD_FAIL_HTLC(add.id, Right(FinalIncorrectCltvExpiry(add.expiry)), commit = true))
|
||||
case _ if add.expiry < Globals.blockCount.get() + 3 => // TODO: check hardcoded value
|
||||
Left(CMD_FAIL_HTLC(add.id, Right(FinalExpiryTooSoon), commit = true))
|
||||
case _ =>
|
||||
Right(add)
|
||||
}
|
||||
|
@ -89,8 +87,6 @@ class Relayer(nodeParams: NodeParams, register: ActorRef, paymentHandler: ActorR
|
|||
Left(CMD_FAIL_HTLC(add.id, Right(AmountBelowMinimum(add.amountMsat, channelUpdate)), commit = true))
|
||||
case Some(channelUpdate) if add.expiry != perHopPayload.outgoingCltvValue + channelUpdate.cltvExpiryDelta =>
|
||||
Left(CMD_FAIL_HTLC(add.id, Right(IncorrectCltvExpiry(add.expiry, channelUpdate)), commit = true))
|
||||
case Some(channelUpdate) if add.expiry < Globals.blockCount.get() + 3 => // TODO: hardcoded value
|
||||
Left(CMD_FAIL_HTLC(add.id, Right(ExpiryTooSoon(channelUpdate)), commit = true))
|
||||
case _ =>
|
||||
Right(CMD_ADD_HTLC(perHopPayload.amtToForward, add.paymentHash, perHopPayload.outgoingCltvValue, nextPacket.serialize, upstream_opt = Some(add), commit = true))
|
||||
}
|
||||
|
@ -123,6 +119,8 @@ class Relayer(nodeParams: NodeParams, register: ActorRef, paymentHandler: ActorR
|
|||
|
||||
case Status.Failure(AddHtlcFailed(_, paymentHash, error, Relayed(originChannelId, originHtlcId, _, _), channelUpdate_opt)) =>
|
||||
val failure = (error, channelUpdate_opt) match {
|
||||
case (_: ExpiryTooSmall, Some(channelUpdate)) => ExpiryTooSoon(channelUpdate)
|
||||
case (_: ExpiryTooBig, _) => ExpiryTooFar
|
||||
case (_: InsufficientFunds, Some(channelUpdate)) => TemporaryChannelFailure(channelUpdate)
|
||||
case (_: TooManyAcceptedHtlcs, Some(channelUpdate)) => TemporaryChannelFailure(channelUpdate)
|
||||
case (_: ChannelUnavailable, Some(channelUpdate)) if !Announcements.isEnabled(channelUpdate.flags) => ChannelDisabled(channelUpdate.flags, channelUpdate)
|
||||
|
|
|
@ -76,7 +76,7 @@ class FuzzySpec extends TestkitBaseClass with StateTestsHelperMethods with Loggi
|
|||
def buildCmdAdd(paymentHash: BinaryData, dest: PublicKey) = {
|
||||
// allow overpaying (no more than 2 times the required amount)
|
||||
val amount = requiredAmount + Random.nextInt(requiredAmount)
|
||||
val expiry = Globals.blockCount.get().toInt + PaymentLifecycle.defaultMinFinalCltvExpiry
|
||||
val expiry = Globals.blockCount.get().toInt + Channel.MIN_CLTV_EXPIRY + 1
|
||||
PaymentLifecycle.buildCommand(amount, expiry, paymentHash, Hop(null, dest, null) :: Nil)._1
|
||||
}
|
||||
|
||||
|
|
|
@ -104,13 +104,42 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
|
|||
}
|
||||
}
|
||||
|
||||
test("recv CMD_ADD_HTLC (expiry in the past)") { case (alice, _, alice2bob, _, _, _, _) =>
|
||||
within(30 seconds) {
|
||||
val sender = TestProbe()
|
||||
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
|
||||
val currentBlockCount = Globals.blockCount.get
|
||||
val add = CMD_ADD_HTLC(500000000, "11" * 32, expiry = 1)
|
||||
sender.send(alice, add)
|
||||
val error = ExpiryCannotBeInThePast(channelId(alice), 1, currentBlockCount)
|
||||
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), add.paymentHash, error, Local(Some(sender.ref)), Some(initialState.channelUpdate))))
|
||||
alice2bob.expectNoMsg(200 millis)
|
||||
}
|
||||
}
|
||||
|
||||
test("recv CMD_ADD_HTLC (expiry too small)") { case (alice, _, alice2bob, _, _, _, _) =>
|
||||
within(30 seconds) {
|
||||
val sender = TestProbe()
|
||||
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
|
||||
val add = CMD_ADD_HTLC(500000000, "11" * 32, expiry = 300000)
|
||||
val currentBlockCount = Globals.blockCount.get
|
||||
val expiryTooSmall = currentBlockCount + 3
|
||||
val add = CMD_ADD_HTLC(500000000, "11" * 32, expiry = expiryTooSmall)
|
||||
sender.send(alice, add)
|
||||
val error = ExpiryCannotBeInThePast(channelId(alice), 300000, 400000)
|
||||
val error = ExpiryTooSmall(channelId(alice), minimum = currentBlockCount + Channel.MIN_CLTV_EXPIRY, actual = expiryTooSmall, blockCount = currentBlockCount)
|
||||
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), add.paymentHash, error, Local(Some(sender.ref)), Some(initialState.channelUpdate))))
|
||||
alice2bob.expectNoMsg(200 millis)
|
||||
}
|
||||
}
|
||||
|
||||
test("recv CMD_ADD_HTLC (expiry too big)") { case (alice, _, alice2bob, _, _, _, _) =>
|
||||
within(30 seconds) {
|
||||
val sender = TestProbe()
|
||||
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
|
||||
val currentBlockCount = Globals.blockCount.get
|
||||
val expiryTooBig = currentBlockCount + Channel.MAX_CLTV_EXPIRY + 1
|
||||
val add = CMD_ADD_HTLC(500000000, "11" * 32, expiry = expiryTooBig)
|
||||
sender.send(alice, add)
|
||||
val error = ExpiryTooBig(channelId(alice), maximum = currentBlockCount + Channel.MAX_CLTV_EXPIRY, actual = expiryTooBig, blockCount = currentBlockCount)
|
||||
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), add.paymentHash, error, Local(Some(sender.ref)), Some(initialState.channelUpdate))))
|
||||
alice2bob.expectNoMsg(200 millis)
|
||||
}
|
||||
|
@ -312,13 +341,46 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
|
|||
}
|
||||
}
|
||||
|
||||
test("recv UpdateAddHtlc (expiry too small)") { case (_, bob, alice2bob, bob2alice, _, bob2blockchain, _) =>
|
||||
test("recv UpdateAddHtlc (expiry in the past)") { case (_, bob, alice2bob, bob2alice, _, bob2blockchain, _) =>
|
||||
within(30 seconds) {
|
||||
val tx = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.localCommit.publishableTxs.commitTx.tx
|
||||
val currentBlockCount = Globals.blockCount.get
|
||||
val htlc = UpdateAddHtlc("00" * 32, 0, 150000, BinaryData("42" * 32), expiry = 1, defaultOnion)
|
||||
alice2bob.forward(bob, htlc)
|
||||
val error = bob2alice.expectMsgType[Error]
|
||||
assert(new String(error.data) === ExpiryTooSmall(channelId(bob), minimum = 400003, actual = 1, blockCount = 400000).getMessage)
|
||||
assert(new String(error.data) === ExpiryCannotBeInThePast(channelId(bob), expiry = 1, blockCount = currentBlockCount).getMessage)
|
||||
awaitCond(bob.stateName == CLOSING)
|
||||
bob2blockchain.expectMsg(PublishAsap(tx))
|
||||
bob2blockchain.expectMsgType[PublishAsap]
|
||||
bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
}
|
||||
}
|
||||
|
||||
test("recv UpdateAddHtlc (expiry too small)") { case (_, bob, alice2bob, bob2alice, _, bob2blockchain, _) =>
|
||||
within(30 seconds) {
|
||||
val tx = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.localCommit.publishableTxs.commitTx.tx
|
||||
val currentBlockCount = Globals.blockCount.get
|
||||
val expiryTooSmall = currentBlockCount + 3
|
||||
val htlc = UpdateAddHtlc("00" * 32, 0, 150000, BinaryData("42" * 32), expiry = expiryTooSmall, defaultOnion)
|
||||
alice2bob.forward(bob, htlc)
|
||||
val error = bob2alice.expectMsgType[Error]
|
||||
assert(new String(error.data) === ExpiryTooSmall(channelId(bob), minimum = currentBlockCount + Channel.MIN_CLTV_EXPIRY, actual = expiryTooSmall, blockCount = currentBlockCount).getMessage)
|
||||
awaitCond(bob.stateName == CLOSING)
|
||||
bob2blockchain.expectMsg(PublishAsap(tx))
|
||||
bob2blockchain.expectMsgType[PublishAsap]
|
||||
bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
}
|
||||
}
|
||||
|
||||
test("recv UpdateAddHtlc (expiry too big)") { case (_, bob, alice2bob, bob2alice, _, bob2blockchain, _) =>
|
||||
within(30 seconds) {
|
||||
val tx = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.localCommit.publishableTxs.commitTx.tx
|
||||
val currentBlockCount = Globals.blockCount.get
|
||||
val expiryTooBig = currentBlockCount + Channel.MAX_CLTV_EXPIRY + 1
|
||||
val htlc = UpdateAddHtlc("00" * 32, 0, 150000, BinaryData("42" * 32), expiry = expiryTooBig, defaultOnion)
|
||||
alice2bob.forward(bob, htlc)
|
||||
val error = bob2alice.expectMsgType[Error]
|
||||
assert(new String(error.data) === ExpiryTooBig(channelId(bob), maximum = currentBlockCount + Channel.MAX_CLTV_EXPIRY, actual = expiryTooBig, blockCount = currentBlockCount).getMessage)
|
||||
awaitCond(bob.stateName == CLOSING)
|
||||
bob2blockchain.expectMsg(PublishAsap(tx))
|
||||
bob2blockchain.expectMsgType[PublishAsap]
|
||||
|
|
|
@ -29,7 +29,7 @@ class SqlitePaymentsDbSpec extends FunSuite {
|
|||
db.addPayment(p1)
|
||||
db.addPayment(p2)
|
||||
assert(db.listPayments().toList === List(p1, p2))
|
||||
assert(db.findByPaymentHash(p1.payment_hash) === p1)
|
||||
intercept[NoSuchElementException](db.findByPaymentHash(BinaryData("6e7e8018f05e169cf1d99e77dc22cb372d09f10b6a81f1eae410718c56cad187")))
|
||||
assert(db.findByPaymentHash(p1.payment_hash) === Some(p1))
|
||||
assert(db.findByPaymentHash("6e7e8018f05e169cf1d99e77dc22cb372d09f10b6a81f1eae410718c56cad187") === None)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
package fr.acinq.eclair.payment
|
||||
|
||||
import fr.acinq.bitcoin.{BinaryData, Block, Crypto}
|
||||
import fr.acinq.eclair.channel.Channel
|
||||
import fr.acinq.eclair.crypto.Sphinx
|
||||
import fr.acinq.eclair.crypto.Sphinx.{PacketAndSecrets, ParsedPacket}
|
||||
import fr.acinq.eclair.payment.PaymentLifecycle._
|
||||
|
@ -151,11 +152,11 @@ object HtlcGenerationSpec {
|
|||
|
||||
val finalAmountMsat = 42000000L
|
||||
val currentBlockCount = 420000
|
||||
val finalExpiry = currentBlockCount + defaultMinFinalCltvExpiry
|
||||
val finalExpiry = currentBlockCount + Channel.MIN_CLTV_EXPIRY
|
||||
val paymentPreimage = BinaryData("42" * 32)
|
||||
val paymentHash = Crypto.sha256(paymentPreimage)
|
||||
|
||||
val expiry_de = currentBlockCount + defaultMinFinalCltvExpiry
|
||||
val expiry_de = currentBlockCount + Channel.MIN_CLTV_EXPIRY
|
||||
val amount_de = finalAmountMsat
|
||||
val fee_d = nodeFee(channelUpdate_de.feeBaseMsat, channelUpdate_de.feeProportionalMillionths, amount_de)
|
||||
|
||||
|
|
|
@ -4,20 +4,23 @@ import akka.actor.{ActorSystem, Status}
|
|||
import akka.actor.Status.Failure
|
||||
import akka.testkit.{TestKit, TestProbe}
|
||||
import fr.acinq.bitcoin.{MilliSatoshi, Satoshi}
|
||||
import fr.acinq.eclair.Globals
|
||||
import fr.acinq.eclair.TestConstants.Alice
|
||||
import fr.acinq.eclair.channel.CMD_FULFILL_HTLC
|
||||
import fr.acinq.eclair.wire.UpdateAddHtlc
|
||||
import fr.acinq.eclair.channel.{CMD_FAIL_HTLC, CMD_FULFILL_HTLC}
|
||||
import fr.acinq.eclair.wire.{FinalExpiryTooSoon, UpdateAddHtlc}
|
||||
import org.junit.runner.RunWith
|
||||
import org.scalatest.FunSuiteLike
|
||||
import org.scalatest.junit.JUnitRunner
|
||||
|
||||
import scala.concurrent.duration._
|
||||
|
||||
/**
|
||||
* Created by PM on 24/03/2017.
|
||||
*/
|
||||
@RunWith(classOf[JUnitRunner])
|
||||
class PaymentHandlerSpec extends TestKit(ActorSystem("test")) with FunSuiteLike {
|
||||
|
||||
test("LocalPaymentHandler should send PaymentReceived and adds payment in DB") {
|
||||
test("LocalPaymentHandler should reply with a fulfill/fail, emit a PaymentReceived and adds payment in DB") {
|
||||
val nodeParams = Alice.nodeParams
|
||||
val handler = system.actorOf(LocalPaymentHandler.props(nodeParams))
|
||||
val sender = TestProbe()
|
||||
|
@ -25,26 +28,46 @@ class PaymentHandlerSpec extends TestKit(ActorSystem("test")) with FunSuiteLike
|
|||
system.eventStream.subscribe(eventListener.ref, classOf[PaymentReceived])
|
||||
|
||||
val amountMsat = MilliSatoshi(42000)
|
||||
val expiry = Globals.blockCount.get() + 12
|
||||
|
||||
{
|
||||
sender.send(handler, ReceivePayment(Some(amountMsat), "1 coffee"))
|
||||
val pr = sender.expectMsgType[PaymentRequest]
|
||||
|
||||
val add = UpdateAddHtlc("11" * 32, 0, amountMsat.amount, pr.paymentHash, 0, "")
|
||||
sender.send(handler, CheckPayment(pr.paymentHash))
|
||||
assert(sender.expectMsgType[Boolean] === false)
|
||||
val add = UpdateAddHtlc("11" * 32, 0, amountMsat.amount, pr.paymentHash, expiry, "")
|
||||
sender.send(handler, add)
|
||||
sender.expectMsgType[CMD_FULFILL_HTLC]
|
||||
eventListener.expectMsg(PaymentReceived(amountMsat, add.paymentHash))
|
||||
sender.send(handler, CheckPayment(pr.paymentHash))
|
||||
assert(sender.expectMsgType[Boolean] === true)
|
||||
}
|
||||
|
||||
{
|
||||
sender.send(handler, ReceivePayment(Some(amountMsat), "another coffee"))
|
||||
val pr_2 = sender.expectMsgType[PaymentRequest]
|
||||
|
||||
val add_2 = UpdateAddHtlc("11" * 32, 0, amountMsat.amount, pr_2.paymentHash, 0, "")
|
||||
sender.send(handler, add_2)
|
||||
val pr = sender.expectMsgType[PaymentRequest]
|
||||
sender.send(handler, CheckPayment(pr.paymentHash))
|
||||
assert(sender.expectMsgType[Boolean] === false)
|
||||
val add = UpdateAddHtlc("11" * 32, 0, amountMsat.amount, pr.paymentHash, expiry, "")
|
||||
sender.send(handler, add)
|
||||
sender.expectMsgType[CMD_FULFILL_HTLC]
|
||||
eventListener.expectMsg(PaymentReceived(amountMsat, add_2.paymentHash))
|
||||
eventListener.expectMsg(PaymentReceived(amountMsat, add.paymentHash))
|
||||
sender.send(handler, CheckPayment(pr.paymentHash))
|
||||
assert(sender.expectMsgType[Boolean] === true)
|
||||
}
|
||||
|
||||
val checkPayment_2 = CheckPayment(add_2.paymentHash)
|
||||
sender.send(handler, checkPayment_2)
|
||||
val found = sender.expectMsgType[Boolean]
|
||||
assert(found)
|
||||
{
|
||||
sender.send(handler, ReceivePayment(Some(amountMsat), "bad expiry"))
|
||||
val pr = sender.expectMsgType[PaymentRequest]
|
||||
sender.send(handler, CheckPayment(pr.paymentHash))
|
||||
assert(sender.expectMsgType[Boolean] === false)
|
||||
val add = UpdateAddHtlc("11" * 32, 0, amountMsat.amount, pr.paymentHash, expiry = Globals.blockCount.get() + 3, "")
|
||||
sender.send(handler, add)
|
||||
assert(sender.expectMsgType[CMD_FAIL_HTLC].reason == Right(FinalExpiryTooSoon))
|
||||
eventListener.expectNoMsg(300 milliseconds)
|
||||
sender.send(handler, CheckPayment(pr.paymentHash))
|
||||
assert(sender.expectMsgType[Boolean] === false)
|
||||
}
|
||||
}
|
||||
|
||||
test("Payment request generation should fail when the amount asked in not valid") {
|
||||
|
|
|
@ -7,6 +7,7 @@ import fr.acinq.bitcoin.{BinaryData, Crypto, MilliSatoshi}
|
|||
import fr.acinq.eclair.randomBytes
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.crypto.Sphinx
|
||||
import fr.acinq.eclair.payment.HtlcGenerationSpec.channelUpdate_bc
|
||||
import fr.acinq.eclair.payment.PaymentLifecycle.buildCommand
|
||||
import fr.acinq.eclair.router.Announcements
|
||||
import fr.acinq.eclair.wire._
|
||||
|
@ -216,24 +217,6 @@ class RelayerSpec extends TestkitBaseClass {
|
|||
paymentHandler.expectNoMsg(100 millis)
|
||||
}
|
||||
|
||||
test("fail to relay an htlc-add when expiry is too soon") { case (relayer, register, paymentHandler) =>
|
||||
val sender = TestProbe()
|
||||
|
||||
val (cmd, _) = buildCommand(finalAmountMsat, 0, paymentHash, hops)
|
||||
// and then manually build an htlc
|
||||
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.expiry, cmd.onion)
|
||||
relayer ! LocalChannelUpdate(null, channelId_bc, channelUpdate_bc.shortChannelId, c, None, channelUpdate_bc)
|
||||
|
||||
sender.send(relayer, ForwardAdd(add_ab))
|
||||
|
||||
val fail = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]].message
|
||||
assert(fail.id === add_ab.id)
|
||||
assert(fail.reason == Right(ExpiryTooSoon(channelUpdate_bc)))
|
||||
|
||||
register.expectNoMsg(100 millis)
|
||||
paymentHandler.expectNoMsg(100 millis)
|
||||
}
|
||||
|
||||
test("fail an htlc-add at the final node when amount has been modified by second-to-last node") { case (relayer, register, paymentHandler) =>
|
||||
val sender = TestProbe()
|
||||
|
||||
|
@ -274,73 +257,30 @@ class RelayerSpec extends TestkitBaseClass {
|
|||
paymentHandler.expectNoMsg(100 millis)
|
||||
}
|
||||
|
||||
test("fail to relay an htlc-add when next channel's balance is too low") { case (relayer, register, paymentHandler) =>
|
||||
test("correctly translates errors returned by channel when attempting to add an htlc") { case (relayer, register, paymentHandler) =>
|
||||
val sender = TestProbe()
|
||||
|
||||
val (cmd, _) = buildCommand(finalAmountMsat, Globals.blockCount.get().toInt + 10, paymentHash, hops)
|
||||
// and then manually build an htlc
|
||||
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.expiry, cmd.onion)
|
||||
relayer ! LocalChannelUpdate(null, channelId_bc, channelUpdate_bc.shortChannelId, c, None, channelUpdate_bc)
|
||||
val paymentHash = randomBytes(32)
|
||||
val origin = Relayed(channelId_ab, originHtlcId = 42, amountMsatIn = 1100000, amountMsatOut = 1000000)
|
||||
|
||||
sender.send(relayer, ForwardAdd(add_ab))
|
||||
sender.send(relayer, Status.Failure(AddHtlcFailed(channelId_bc, paymentHash, ExpiryTooSmall(channelId_bc, 100, 0, 0), origin, Some(channelUpdate_bc))))
|
||||
assert(register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]].message.reason === Right(ExpiryTooSoon(channelUpdate_bc)))
|
||||
|
||||
val fwd = register.expectMsgType[Register.ForwardShortId[CMD_ADD_HTLC]]
|
||||
assert(fwd.shortChannelId === channelUpdate_bc.shortChannelId)
|
||||
assert(fwd.message.upstream_opt === Some(add_ab))
|
||||
sender.send(relayer, Status.Failure(AddHtlcFailed(channelId_bc, paymentHash, ExpiryTooBig(channelId_bc, 100, 200, 0), origin, Some(channelUpdate_bc))))
|
||||
assert(register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]].message.reason === Right(ExpiryTooFar))
|
||||
|
||||
sender.send(relayer, Status.Failure(AddHtlcFailed(channelId_bc, add_ab.paymentHash, new InsufficientFunds(channelId_bc, cmd.amountMsat, 100, 0, 0), Relayed(add_ab.channelId, add_ab.id, add_ab.amountMsat, cmd.amountMsat), Some(channelUpdate_bc))))
|
||||
sender.send(relayer, Status.Failure(AddHtlcFailed(channelId_bc, paymentHash, InsufficientFunds(channelId_bc, origin.amountMsatOut, 100, 0, 0), origin, Some(channelUpdate_bc))))
|
||||
assert(register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]].message.reason === Right(TemporaryChannelFailure(channelUpdate_bc)))
|
||||
|
||||
val fail = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]].message
|
||||
assert(fail.id === add_ab.id)
|
||||
assert(fail.reason == Right(TemporaryChannelFailure(channelUpdate_bc)))
|
||||
val channelUpdate_bc_disabled = channelUpdate_bc.copy(flags = "0002")
|
||||
sender.send(relayer, Status.Failure(AddHtlcFailed(channelId_bc, paymentHash, ChannelUnavailable(channelId_bc), origin, Some(channelUpdate_bc_disabled))))
|
||||
assert(register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]].message.reason === Right(ChannelDisabled(channelUpdate_bc_disabled.flags, channelUpdate_bc_disabled)))
|
||||
|
||||
register.expectNoMsg(100 millis)
|
||||
paymentHandler.expectNoMsg(100 millis)
|
||||
}
|
||||
sender.send(relayer, Status.Failure(AddHtlcFailed(channelId_bc, paymentHash, HtlcTimedout(channelId_bc), origin, None)))
|
||||
assert(register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]].message.reason === Right(PermanentChannelFailure))
|
||||
|
||||
test("fail to relay an htlc-add when next channel has too many inflight htlcs") { case (relayer, register, paymentHandler) =>
|
||||
val sender = TestProbe()
|
||||
|
||||
val (cmd, _) = buildCommand(finalAmountMsat, Globals.blockCount.get().toInt + 10, paymentHash, hops)
|
||||
// and then manually build an htlc
|
||||
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.expiry, cmd.onion)
|
||||
relayer ! LocalChannelUpdate(null, channelId_bc, channelUpdate_bc.shortChannelId, c, None, channelUpdate_bc)
|
||||
|
||||
sender.send(relayer, ForwardAdd(add_ab))
|
||||
|
||||
val fwd = register.expectMsgType[Register.ForwardShortId[CMD_ADD_HTLC]]
|
||||
assert(fwd.shortChannelId === channelUpdate_bc.shortChannelId)
|
||||
assert(fwd.message.upstream_opt === Some(add_ab))
|
||||
|
||||
sender.send(relayer, Status.Failure(AddHtlcFailed(channelId_bc, add_ab.paymentHash, new TooManyAcceptedHtlcs(channelId_bc, 30), Relayed(add_ab.channelId, add_ab.id, add_ab.amountMsat, cmd.amountMsat), Some(channelUpdate_bc))))
|
||||
|
||||
val fail = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]].message
|
||||
assert(fail.id === add_ab.id)
|
||||
assert(fail.reason == Right(TemporaryChannelFailure(channelUpdate_bc)))
|
||||
|
||||
register.expectNoMsg(100 millis)
|
||||
paymentHandler.expectNoMsg(100 millis)
|
||||
}
|
||||
|
||||
test("fail to relay an htlc-add when next channel has a timed out htlc (and is thus closing)") { case (relayer, register, paymentHandler) =>
|
||||
val sender = TestProbe()
|
||||
|
||||
val (cmd, _) = buildCommand(finalAmountMsat, Globals.blockCount.get().toInt + 10, paymentHash, hops)
|
||||
// and then manually build an htlc
|
||||
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.expiry, cmd.onion)
|
||||
relayer ! LocalChannelUpdate(null, channelId_bc, channelUpdate_bc.shortChannelId, c, None, channelUpdate_bc)
|
||||
|
||||
sender.send(relayer, ForwardAdd(add_ab))
|
||||
|
||||
val fwd = register.expectMsgType[Register.ForwardShortId[CMD_ADD_HTLC]]
|
||||
assert(fwd.shortChannelId === channelUpdate_bc.shortChannelId)
|
||||
assert(fwd.message.upstream_opt === Some(add_ab))
|
||||
|
||||
sender.send(relayer, Status.Failure(AddHtlcFailed(channelId_bc, add_ab.paymentHash, new HtlcTimedout(channelId_bc), Relayed(add_ab.channelId, add_ab.id, add_ab.amountMsat, cmd.amountMsat), Some(channelUpdate_bc))))
|
||||
|
||||
val fail = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]].message
|
||||
assert(fail.id === add_ab.id)
|
||||
assert(fail.reason == Right(PermanentChannelFailure))
|
||||
sender.send(relayer, Status.Failure(AddHtlcFailed(channelId_bc, paymentHash, HtlcTimedout(channelId_bc), origin, Some(channelUpdate_bc))))
|
||||
assert(register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]].message.reason === Right(PermanentChannelFailure))
|
||||
|
||||
register.expectNoMsg(100 millis)
|
||||
paymentHandler.expectNoMsg(100 millis)
|
||||
|
|
|
@ -69,7 +69,7 @@ class Handlers(fKit: Future[Kit])(implicit ec: ExecutionContext = ExecutionConte
|
|||
logger.info(s"sending $amountMsat to ${req.paymentHash} @ ${req.nodeId}")
|
||||
val sendPayment = req.minFinalCltvExpiry match {
|
||||
case None => SendPayment(amountMsat, req.paymentHash, req.nodeId, req.routingInfo)
|
||||
case Some(minFinalCltvExpiry) => SendPayment(amountMsat, req.paymentHash, req.nodeId, req.routingInfo, minFinalCltvExpiry = minFinalCltvExpiry)
|
||||
case Some(minFinalCltvExpiry) => SendPayment(amountMsat, req.paymentHash, req.nodeId, req.routingInfo, finalCltvExpiry = minFinalCltvExpiry)
|
||||
}
|
||||
(for {
|
||||
kit <- fKit
|
||||
|
|
Loading…
Add table
Reference in a new issue