1
0
Fork 0
mirror of https://github.com/ACINQ/eclair.git synced 2025-02-23 14:40:34 +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:
Pierre-Marie Padiou 2018-03-01 18:48:54 +01:00 committed by GitHub
parent 203288c415
commit 85592de0e5
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
15 changed files with 210 additions and 164 deletions

View file

@ -42,6 +42,10 @@ object Channel {
// we won't exchange more than this many signatures when negotiating the closing fee // we won't exchange more than this many signatures when negotiating the closing fee
val MAX_NEGOTIATION_ITERATIONS = 20 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 case object TickRefreshChannelUpdate
} }

View file

@ -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 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 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 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 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 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") 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")

View file

@ -4,7 +4,7 @@ import akka.event.LoggingAdapter
import fr.acinq.bitcoin.Crypto.{Point, PrivateKey, sha256} import fr.acinq.bitcoin.Crypto.{Point, PrivateKey, sha256}
import fr.acinq.bitcoin.{BinaryData, Crypto, Satoshi, Transaction} import fr.acinq.bitcoin.{BinaryData, Crypto, Satoshi, Transaction}
import fr.acinq.eclair.crypto.{Generators, ShaChain, Sphinx} 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.Transactions._
import fr.acinq.eclair.transactions._ import fr.acinq.eclair.transactions._
import fr.acinq.eclair.wire._ import fr.acinq.eclair.wire._
@ -87,6 +87,16 @@ object Commitments {
if (cmd.expiry <= blockCount) { if (cmd.expiry <= blockCount) {
return Left(ExpiryCannotBeInThePast(commitments.channelId, 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) { if (cmd.amountMsat < commitments.remoteParams.htlcMinimumMsat) {
return Left(HtlcValueTooSmall(commitments.channelId, minimum = commitments.remoteParams.htlcMinimumMsat, actual = cmd.amountMsat)) return Left(HtlcValueTooSmall(commitments.channelId, minimum = commitments.remoteParams.htlcMinimumMsat, actual = cmd.amountMsat))
@ -133,11 +143,18 @@ object Commitments {
} }
val blockCount = Globals.blockCount.get() val blockCount = Globals.blockCount.get()
// we need a reasonable amount of time to pull the funds before the sender can get refunded if (add.expiry <= blockCount) {
val minExpiry = blockCount + 3 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) { if (add.expiry < minExpiry) {
throw ExpiryTooSmall(commitments.channelId, minimum = minExpiry, actual = add.expiry, blockCount = blockCount) 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) { if (add.amountMsat < commitments.localParams.htlcMinimumMsat) {
throw HtlcValueTooSmall(commitments.channelId, minimum = commitments.localParams.htlcMinimumMsat, actual = add.amountMsat) throw HtlcValueTooSmall(commitments.channelId, minimum = commitments.localParams.htlcMinimumMsat, actual = add.amountMsat)

View file

@ -21,8 +21,7 @@ trait PaymentsDb {
def addPayment(payment: Payment) def addPayment(payment: Payment)
@throws(classOf[NoSuchElementException]) def findByPaymentHash(paymentHash: BinaryData): Option[Payment]
def findByPaymentHash(paymentHash: BinaryData): Payment
def listPayments(): Seq[Payment] def listPayments(): Seq[Payment]

View file

@ -35,15 +35,14 @@ class SqlitePaymentsDb(sqlite: Connection) extends PaymentsDb with Logging {
} }
} }
@throws(classOf[NoSuchElementException]) override def findByPaymentHash(paymentHash: BinaryData): Option[Payment] = {
override def findByPaymentHash(paymentHash: BinaryData): Payment = {
using(sqlite.prepareStatement("SELECT payment_hash, amount_msat, timestamp FROM payments WHERE payment_hash = ?")) { statement => using(sqlite.prepareStatement("SELECT payment_hash, amount_msat, timestamp FROM payments WHERE payment_hash = ?")) { statement =>
statement.setBytes(1, paymentHash) statement.setBytes(1, paymentHash)
val rs = statement.executeQuery() val rs = statement.executeQuery()
if (rs.next()) { 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 { } else {
throw new NoSuchElementException("payment not found") None
} }
} }
} }

View file

@ -2,11 +2,12 @@ package fr.acinq.eclair.payment
import akka.actor.{Actor, ActorLogging, Props, Status} import akka.actor.{Actor, ActorLogging, Props, Status}
import fr.acinq.bitcoin.{BinaryData, Crypto, MilliSatoshi} 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.db.Payment
import fr.acinq.eclair.wire._ import fr.acinq.eclair.wire._
import scala.concurrent.duration._ import scala.concurrent.duration._
import fr.acinq.eclair.{NodeParams, randomBytes} import fr.acinq.eclair.{Globals, NodeParams, randomBytes}
import scala.compat.Platform import scala.compat.Platform
import scala.concurrent.ExecutionContext import scala.concurrent.ExecutionContext
@ -21,57 +22,58 @@ class LocalPaymentHandler(nodeParams: NodeParams)(implicit ec: ExecutionContext
override def receive: Receive = run(Map()) 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 => 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.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 e@(_, (_, pr)) if pr.timestamp + pr.expiry.get > currentSeconds => e // clean up expired requests
})) }))
case ReceivePayment(amount_opt, desc) => case ReceivePayment(amount_opt, desc) =>
Try { Try {
if (h2r.size > nodeParams.maxPendingPaymentRequests) { if (hash2preimage.size > nodeParams.maxPendingPaymentRequests) {
throw new RuntimeException(s"too many pending payment requests (max=${nodeParams.maxPendingPaymentRequests})") throw new RuntimeException(s"too many pending payment requests (max=${nodeParams.maxPendingPaymentRequests})")
} }
val paymentPreimage = randomBytes(32) val paymentPreimage = randomBytes(32)
val paymentHash = Crypto.sha256(paymentPreimage) val paymentHash = Crypto.sha256(paymentPreimage)
(paymentPreimage, paymentHash, PaymentRequest(nodeParams.chainHash, amount_opt, paymentHash, nodeParams.privateKey, desc, fallbackAddress = None, expirySeconds = Some(nodeParams.paymentRequestExpiry.toSeconds))) val paymentRequest = PaymentRequest(nodeParams.chainHash, amount_opt, paymentHash, nodeParams.privateKey, desc, fallbackAddress = None, expirySeconds = Some(nodeParams.paymentRequestExpiry.toSeconds))
} match { log.debug(s"generated payment request=${PaymentRequest.write(paymentRequest)} from amount=$amount_opt")
case Success((r, h, pr)) => sender ! paymentRequest
log.debug(s"generated payment request=${PaymentRequest.write(pr)} from amount=$amount_opt") context.become(run(hash2preimage + (paymentHash -> (paymentPreimage, paymentRequest))))
sender ! pr } recover { case t => sender ! Status.Failure(t) }
context.become(run(h2r + (h -> (r, pr))))
case Failure(t) =>
sender ! Status.Failure(t)
}
case CheckPayment(paymentHash) => case CheckPayment(paymentHash) =>
val found: Boolean = Try(nodeParams.paymentsDb.findByPaymentHash(paymentHash)) match { nodeParams.paymentsDb.findByPaymentHash(paymentHash) match {
case Success(s) if paymentHash == s.payment_hash => true case Some(_) => sender ! true
case _ => false case _ => sender ! false
} }
sender ! found
case htlc: UpdateAddHtlc => case htlc: UpdateAddHtlc =>
if (h2r.contains(htlc.paymentHash)) { hash2preimage.get(htlc.paymentHash) match {
val r = h2r(htlc.paymentHash)._1 case Some((paymentPreimage, paymentRequest)) =>
val pr = h2r(htlc.paymentHash)._2 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 // 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. // 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 // see https://github.com/lightningnetwork/lightning-rfc/blob/master/04-onion-routing.md#failure-messages
pr.amount match { paymentRequest.amount match {
case Some(amount) if MilliSatoshi(htlc.amountMsat) < amount => sender ! CMD_FAIL_HTLC(htlc.id, Right(IncorrectPaymentAmount), commit = true) case _ if htlc.expiry < minFinalExpiry =>
case Some(amount) if MilliSatoshi(htlc.amountMsat) > amount * 2 => sender ! CMD_FAIL_HTLC(htlc.id, Right(IncorrectPaymentAmount), commit = true) 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 _ => case _ =>
log.info(s"received payment for paymentHash=${htlc.paymentHash} amountMsat=${htlc.amountMsat}") log.info(s"received payment for paymentHash=${htlc.paymentHash} amountMsat=${htlc.amountMsat}")
// amount is correct or was not specified in the payment request // amount is correct or was not specified in the payment request
nodeParams.paymentsDb.addPayment(Payment(htlc.paymentHash, htlc.amountMsat, Platform.currentTime / 1000)) 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.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) sender ! CMD_FAIL_HTLC(htlc.id, Right(UnknownPaymentHash), commit = true)
} }
} }

View file

@ -4,7 +4,7 @@ import akka.actor.{ActorRef, FSM, Props, Status}
import fr.acinq.bitcoin.Crypto.PublicKey import fr.acinq.bitcoin.Crypto.PublicKey
import fr.acinq.bitcoin.{BinaryData, MilliSatoshi} import fr.acinq.bitcoin.{BinaryData, MilliSatoshi}
import fr.acinq.eclair._ 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
import fr.acinq.eclair.crypto.Sphinx.{ErrorPacket, Packet} import fr.acinq.eclair.crypto.Sphinx.{ErrorPacket, Packet}
import fr.acinq.eclair.payment.PaymentRequest.ExtraHop import fr.acinq.eclair.payment.PaymentRequest.ExtraHop
@ -16,7 +16,10 @@ import scala.util.{Failure, Success}
// @formatter:off // @formatter:off
case class ReceivePayment(amountMsat_opt: Option[MilliSatoshi], description: String) 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) case class CheckPayment(paymentHash: BinaryData)
sealed trait PaymentResult 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)) => 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("->")}") 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 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) val (cmd, sharedSecrets) = buildCommand(c.amountMsat, finalExpiry, c.paymentHash, hops)
register ! Register.ForwardShortId(firstHop.lastUpdate.shortChannelId, cmd) register ! Register.ForwardShortId(firstHop.lastUpdate.shortChannelId, cmd)
goto(WAITING_FOR_PAYMENT_COMPLETE) using WaitingForComplete(s, c, cmd, failures, sharedSecrets, ignoreNodes, ignoreChannels, hops) 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) (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)]) = { 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 (firstAmountMsat, firstExpiry, payloads) = buildPayloads(finalAmountMsat, finalExpiry, hops.drop(1))
val nodes = hops.map(_.nextNodeId) val nodes = hops.map(_.nextNodeId)

View file

@ -64,8 +64,6 @@ class Relayer(nodeParams: NodeParams, register: ActorRef, paymentHandler: ActorR
Left(CMD_FAIL_HTLC(add.id, Right(FinalIncorrectHtlcAmount(add.amountMsat)), commit = true)) Left(CMD_FAIL_HTLC(add.id, Right(FinalIncorrectHtlcAmount(add.amountMsat)), commit = true))
case PerHopPayload(_, _, finalOutgoingCltvValue) if finalOutgoingCltvValue != add.expiry => case PerHopPayload(_, _, finalOutgoingCltvValue) if finalOutgoingCltvValue != add.expiry =>
Left(CMD_FAIL_HTLC(add.id, Right(FinalIncorrectCltvExpiry(add.expiry)), commit = true)) 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 _ => case _ =>
Right(add) 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)) Left(CMD_FAIL_HTLC(add.id, Right(AmountBelowMinimum(add.amountMsat, channelUpdate)), commit = true))
case Some(channelUpdate) if add.expiry != perHopPayload.outgoingCltvValue + channelUpdate.cltvExpiryDelta => case Some(channelUpdate) if add.expiry != perHopPayload.outgoingCltvValue + channelUpdate.cltvExpiryDelta =>
Left(CMD_FAIL_HTLC(add.id, Right(IncorrectCltvExpiry(add.expiry, channelUpdate)), commit = true)) 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 _ => case _ =>
Right(CMD_ADD_HTLC(perHopPayload.amtToForward, add.paymentHash, perHopPayload.outgoingCltvValue, nextPacket.serialize, upstream_opt = Some(add), commit = true)) 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)) => case Status.Failure(AddHtlcFailed(_, paymentHash, error, Relayed(originChannelId, originHtlcId, _, _), channelUpdate_opt)) =>
val failure = (error, channelUpdate_opt) match { val failure = (error, channelUpdate_opt) match {
case (_: ExpiryTooSmall, Some(channelUpdate)) => ExpiryTooSoon(channelUpdate)
case (_: ExpiryTooBig, _) => ExpiryTooFar
case (_: InsufficientFunds, Some(channelUpdate)) => TemporaryChannelFailure(channelUpdate) case (_: InsufficientFunds, Some(channelUpdate)) => TemporaryChannelFailure(channelUpdate)
case (_: TooManyAcceptedHtlcs, Some(channelUpdate)) => TemporaryChannelFailure(channelUpdate) case (_: TooManyAcceptedHtlcs, Some(channelUpdate)) => TemporaryChannelFailure(channelUpdate)
case (_: ChannelUnavailable, Some(channelUpdate)) if !Announcements.isEnabled(channelUpdate.flags) => ChannelDisabled(channelUpdate.flags, channelUpdate) case (_: ChannelUnavailable, Some(channelUpdate)) if !Announcements.isEnabled(channelUpdate.flags) => ChannelDisabled(channelUpdate.flags, channelUpdate)

View file

@ -76,7 +76,7 @@ class FuzzySpec extends TestkitBaseClass with StateTestsHelperMethods with Loggi
def buildCmdAdd(paymentHash: BinaryData, dest: PublicKey) = { def buildCmdAdd(paymentHash: BinaryData, dest: PublicKey) = {
// allow overpaying (no more than 2 times the required amount) // allow overpaying (no more than 2 times the required amount)
val amount = requiredAmount + Random.nextInt(requiredAmount) 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 PaymentLifecycle.buildCommand(amount, expiry, paymentHash, Hop(null, dest, null) :: Nil)._1
} }

View file

@ -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, _, _, _, _) => test("recv CMD_ADD_HTLC (expiry too small)") { case (alice, _, alice2bob, _, _, _, _) =>
within(30 seconds) { within(30 seconds) {
val sender = TestProbe() val sender = TestProbe()
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL] 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) 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)))) sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), add.paymentHash, error, Local(Some(sender.ref)), Some(initialState.channelUpdate))))
alice2bob.expectNoMsg(200 millis) 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) { within(30 seconds) {
val tx = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.localCommit.publishableTxs.commitTx.tx 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) val htlc = UpdateAddHtlc("00" * 32, 0, 150000, BinaryData("42" * 32), expiry = 1, defaultOnion)
alice2bob.forward(bob, htlc) alice2bob.forward(bob, htlc)
val error = bob2alice.expectMsgType[Error] 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) awaitCond(bob.stateName == CLOSING)
bob2blockchain.expectMsg(PublishAsap(tx)) bob2blockchain.expectMsg(PublishAsap(tx))
bob2blockchain.expectMsgType[PublishAsap] bob2blockchain.expectMsgType[PublishAsap]

View file

@ -29,7 +29,7 @@ class SqlitePaymentsDbSpec extends FunSuite {
db.addPayment(p1) db.addPayment(p1)
db.addPayment(p2) db.addPayment(p2)
assert(db.listPayments().toList === List(p1, p2)) assert(db.listPayments().toList === List(p1, p2))
assert(db.findByPaymentHash(p1.payment_hash) === p1) assert(db.findByPaymentHash(p1.payment_hash) === Some(p1))
intercept[NoSuchElementException](db.findByPaymentHash(BinaryData("6e7e8018f05e169cf1d99e77dc22cb372d09f10b6a81f1eae410718c56cad187"))) assert(db.findByPaymentHash("6e7e8018f05e169cf1d99e77dc22cb372d09f10b6a81f1eae410718c56cad187") === None)
} }
} }

View file

@ -1,6 +1,7 @@
package fr.acinq.eclair.payment package fr.acinq.eclair.payment
import fr.acinq.bitcoin.{BinaryData, Block, Crypto} 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
import fr.acinq.eclair.crypto.Sphinx.{PacketAndSecrets, ParsedPacket} import fr.acinq.eclair.crypto.Sphinx.{PacketAndSecrets, ParsedPacket}
import fr.acinq.eclair.payment.PaymentLifecycle._ import fr.acinq.eclair.payment.PaymentLifecycle._
@ -151,11 +152,11 @@ object HtlcGenerationSpec {
val finalAmountMsat = 42000000L val finalAmountMsat = 42000000L
val currentBlockCount = 420000 val currentBlockCount = 420000
val finalExpiry = currentBlockCount + defaultMinFinalCltvExpiry val finalExpiry = currentBlockCount + Channel.MIN_CLTV_EXPIRY
val paymentPreimage = BinaryData("42" * 32) val paymentPreimage = BinaryData("42" * 32)
val paymentHash = Crypto.sha256(paymentPreimage) val paymentHash = Crypto.sha256(paymentPreimage)
val expiry_de = currentBlockCount + defaultMinFinalCltvExpiry val expiry_de = currentBlockCount + Channel.MIN_CLTV_EXPIRY
val amount_de = finalAmountMsat val amount_de = finalAmountMsat
val fee_d = nodeFee(channelUpdate_de.feeBaseMsat, channelUpdate_de.feeProportionalMillionths, amount_de) val fee_d = nodeFee(channelUpdate_de.feeBaseMsat, channelUpdate_de.feeProportionalMillionths, amount_de)

View file

@ -4,20 +4,23 @@ import akka.actor.{ActorSystem, Status}
import akka.actor.Status.Failure import akka.actor.Status.Failure
import akka.testkit.{TestKit, TestProbe} import akka.testkit.{TestKit, TestProbe}
import fr.acinq.bitcoin.{MilliSatoshi, Satoshi} import fr.acinq.bitcoin.{MilliSatoshi, Satoshi}
import fr.acinq.eclair.Globals
import fr.acinq.eclair.TestConstants.Alice import fr.acinq.eclair.TestConstants.Alice
import fr.acinq.eclair.channel.CMD_FULFILL_HTLC import fr.acinq.eclair.channel.{CMD_FAIL_HTLC, CMD_FULFILL_HTLC}
import fr.acinq.eclair.wire.UpdateAddHtlc import fr.acinq.eclair.wire.{FinalExpiryTooSoon, UpdateAddHtlc}
import org.junit.runner.RunWith import org.junit.runner.RunWith
import org.scalatest.FunSuiteLike import org.scalatest.FunSuiteLike
import org.scalatest.junit.JUnitRunner import org.scalatest.junit.JUnitRunner
import scala.concurrent.duration._
/** /**
* Created by PM on 24/03/2017. * Created by PM on 24/03/2017.
*/ */
@RunWith(classOf[JUnitRunner]) @RunWith(classOf[JUnitRunner])
class PaymentHandlerSpec extends TestKit(ActorSystem("test")) with FunSuiteLike { 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 nodeParams = Alice.nodeParams
val handler = system.actorOf(LocalPaymentHandler.props(nodeParams)) val handler = system.actorOf(LocalPaymentHandler.props(nodeParams))
val sender = TestProbe() val sender = TestProbe()
@ -25,26 +28,46 @@ class PaymentHandlerSpec extends TestKit(ActorSystem("test")) with FunSuiteLike
system.eventStream.subscribe(eventListener.ref, classOf[PaymentReceived]) system.eventStream.subscribe(eventListener.ref, classOf[PaymentReceived])
val amountMsat = MilliSatoshi(42000) val amountMsat = MilliSatoshi(42000)
val expiry = Globals.blockCount.get() + 12
{
sender.send(handler, ReceivePayment(Some(amountMsat), "1 coffee")) sender.send(handler, ReceivePayment(Some(amountMsat), "1 coffee"))
val pr = sender.expectMsgType[PaymentRequest] val pr = sender.expectMsgType[PaymentRequest]
sender.send(handler, CheckPayment(pr.paymentHash))
val add = UpdateAddHtlc("11" * 32, 0, amountMsat.amount, pr.paymentHash, 0, "") assert(sender.expectMsgType[Boolean] === false)
val add = UpdateAddHtlc("11" * 32, 0, amountMsat.amount, pr.paymentHash, expiry, "")
sender.send(handler, add) sender.send(handler, add)
sender.expectMsgType[CMD_FULFILL_HTLC] sender.expectMsgType[CMD_FULFILL_HTLC]
eventListener.expectMsg(PaymentReceived(amountMsat, add.paymentHash)) 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")) sender.send(handler, ReceivePayment(Some(amountMsat), "another coffee"))
val pr_2 = sender.expectMsgType[PaymentRequest] val pr = sender.expectMsgType[PaymentRequest]
sender.send(handler, CheckPayment(pr.paymentHash))
val add_2 = UpdateAddHtlc("11" * 32, 0, amountMsat.amount, pr_2.paymentHash, 0, "") assert(sender.expectMsgType[Boolean] === false)
sender.send(handler, add_2) val add = UpdateAddHtlc("11" * 32, 0, amountMsat.amount, pr.paymentHash, expiry, "")
sender.send(handler, add)
sender.expectMsgType[CMD_FULFILL_HTLC] 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) sender.send(handler, ReceivePayment(Some(amountMsat), "bad expiry"))
val found = sender.expectMsgType[Boolean] val pr = sender.expectMsgType[PaymentRequest]
assert(found) 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") { test("Payment request generation should fail when the amount asked in not valid") {

View file

@ -7,6 +7,7 @@ import fr.acinq.bitcoin.{BinaryData, Crypto, MilliSatoshi}
import fr.acinq.eclair.randomBytes import fr.acinq.eclair.randomBytes
import fr.acinq.eclair.channel._ import fr.acinq.eclair.channel._
import fr.acinq.eclair.crypto.Sphinx 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.payment.PaymentLifecycle.buildCommand
import fr.acinq.eclair.router.Announcements import fr.acinq.eclair.router.Announcements
import fr.acinq.eclair.wire._ import fr.acinq.eclair.wire._
@ -216,24 +217,6 @@ class RelayerSpec extends TestkitBaseClass {
paymentHandler.expectNoMsg(100 millis) 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) => 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() val sender = TestProbe()
@ -274,73 +257,30 @@ class RelayerSpec extends TestkitBaseClass {
paymentHandler.expectNoMsg(100 millis) 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 sender = TestProbe()
val (cmd, _) = buildCommand(finalAmountMsat, Globals.blockCount.get().toInt + 10, paymentHash, hops) val paymentHash = randomBytes(32)
// and then manually build an htlc val origin = Relayed(channelId_ab, originHtlcId = 42, amountMsatIn = 1100000, amountMsatOut = 1000000)
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)) 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]] sender.send(relayer, Status.Failure(AddHtlcFailed(channelId_bc, paymentHash, ExpiryTooBig(channelId_bc, 100, 200, 0), origin, Some(channelUpdate_bc))))
assert(fwd.shortChannelId === channelUpdate_bc.shortChannelId) assert(register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]].message.reason === Right(ExpiryTooFar))
assert(fwd.message.upstream_opt === Some(add_ab))
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 val channelUpdate_bc_disabled = channelUpdate_bc.copy(flags = "0002")
assert(fail.id === add_ab.id) sender.send(relayer, Status.Failure(AddHtlcFailed(channelId_bc, paymentHash, ChannelUnavailable(channelId_bc), origin, Some(channelUpdate_bc_disabled))))
assert(fail.reason == Right(TemporaryChannelFailure(channelUpdate_bc))) assert(register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]].message.reason === Right(ChannelDisabled(channelUpdate_bc_disabled.flags, channelUpdate_bc_disabled)))
register.expectNoMsg(100 millis) sender.send(relayer, Status.Failure(AddHtlcFailed(channelId_bc, paymentHash, HtlcTimedout(channelId_bc), origin, None)))
paymentHandler.expectNoMsg(100 millis) 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) => sender.send(relayer, Status.Failure(AddHtlcFailed(channelId_bc, paymentHash, HtlcTimedout(channelId_bc), origin, Some(channelUpdate_bc))))
val sender = TestProbe() assert(register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]].message.reason === Right(PermanentChannelFailure))
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))
register.expectNoMsg(100 millis) register.expectNoMsg(100 millis)
paymentHandler.expectNoMsg(100 millis) paymentHandler.expectNoMsg(100 millis)

View file

@ -69,7 +69,7 @@ class Handlers(fKit: Future[Kit])(implicit ec: ExecutionContext = ExecutionConte
logger.info(s"sending $amountMsat to ${req.paymentHash} @ ${req.nodeId}") logger.info(s"sending $amountMsat to ${req.paymentHash} @ ${req.nodeId}")
val sendPayment = req.minFinalCltvExpiry match { val sendPayment = req.minFinalCltvExpiry match {
case None => SendPayment(amountMsat, req.paymentHash, req.nodeId, req.routingInfo) 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 { (for {
kit <- fKit kit <- fKit