1
0
Fork 0
mirror of https://github.com/ACINQ/eclair.git synced 2025-02-23 14:40:34 +01:00

Fix blinded route failure messages (#2490)

* Make totalAmount required in blinded final payloads

And update the reference test vector for blinded payments.

* Handle failures inside blinded routes

When a failure occurs inside a blinded route, we must avoid leaking any
information to upstream nodes.

We do that by returning `update_fail_malformed_htlc` with the
`invalid_onion_blinding` code whenever we are inside the blinded route,
and `update_fail_htlc` with the `invalid_onion_blinding` code when we are
the introduction node (and we add a delay).

When we are using only dummy hops or not using any blinded hop, we can
return normal errors.

We also fix an issue we had with `update_fail_malformed_htlc`: when we
received that kind of error from the downstream node, we also returned
an `update_fail_malformed_htlc` error upstream, whereas the specification
says we must convert it to an `update_fail_htlc`.

We also add many e2e tests for blinded payments.
This commit is contained in:
Bastien Teinturier 2022-12-16 18:24:35 +01:00 committed by GitHub
parent 83edd8c41d
commit b5a2d3a665
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
28 changed files with 722 additions and 120 deletions

View file

@ -184,8 +184,8 @@ sealed trait HasOptionalReplyToCommand extends Command { def replyTo_opt: Option
final case class CMD_ADD_HTLC(replyTo: ActorRef, amount: MilliSatoshi, paymentHash: ByteVector32, cltvExpiry: CltvExpiry, onion: OnionRoutingPacket, nextBlindingKey_opt: Option[PublicKey], origin: Origin.Hot, commit: Boolean = false) extends HasReplyToCommand
sealed trait HtlcSettlementCommand extends HasOptionalReplyToCommand { def id: Long }
final case class CMD_FULFILL_HTLC(id: Long, r: ByteVector32, commit: Boolean = false, replyTo_opt: Option[ActorRef] = None) extends HtlcSettlementCommand
final case class CMD_FAIL_HTLC(id: Long, reason: Either[ByteVector, FailureMessage], commit: Boolean = false, replyTo_opt: Option[ActorRef] = None) extends HtlcSettlementCommand
final case class CMD_FAIL_MALFORMED_HTLC(id: Long, onionHash: ByteVector32, failureCode: Int, delay_opt: Option[FiniteDuration] = None, commit: Boolean = false, replyTo_opt: Option[ActorRef] = None) extends HtlcSettlementCommand
final case class CMD_FAIL_HTLC(id: Long, reason: Either[ByteVector, FailureMessage], delay_opt: Option[FiniteDuration] = None, commit: Boolean = false, replyTo_opt: Option[ActorRef] = None) extends HtlcSettlementCommand
final case class CMD_FAIL_MALFORMED_HTLC(id: Long, onionHash: ByteVector32, failureCode: Int, commit: Boolean = false, replyTo_opt: Option[ActorRef] = None) extends HtlcSettlementCommand
final case class CMD_UPDATE_FEE(feeratePerKw: FeeratePerKw, commit: Boolean = false, replyTo_opt: Option[ActorRef] = None) extends HasOptionalReplyToCommand
final case class CMD_SIGN(replyTo_opt: Option[ActorRef] = None) extends HasOptionalReplyToCommand

View file

@ -514,7 +514,7 @@ object Commitments {
case None => Left(UnknownHtlcId(commitments.channelId, fulfill.id))
}
def sendFail(commitments: Commitments, cmd: CMD_FAIL_HTLC, nodeSecret: PrivateKey): Either[ChannelException, (Commitments, UpdateFailHtlc)] =
def sendFail(commitments: Commitments, cmd: CMD_FAIL_HTLC, nodeSecret: PrivateKey): Either[ChannelException, (Commitments, HtlcFailureMessage)] =
commitments.getIncomingHtlcCrossSigned(cmd.id) match {
case Some(htlc) if alreadyProposed(commitments.localChanges.proposed, htlc.id) =>
// we have already sent a fail/fulfill for this htlc

View file

@ -382,7 +382,12 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val
}
case Event(c: CMD_FAIL_HTLC, d: DATA_NORMAL) =>
Commitments.sendFail(d.commitments, c, nodeParams.privateKey) match {
c.delay_opt match {
case Some(delay) =>
log.debug("delaying CMD_FAIL_HTLC with id={} for {}", c.id, delay)
context.system.scheduler.scheduleOnce(delay, self, c.copy(delay_opt = None))
stay()
case None => Commitments.sendFail(d.commitments, c, nodeParams.privateKey) match {
case Right((commitments1, fail)) =>
if (c.commit) self ! CMD_SIGN()
context.system.eventStream.publish(AvailableBalanceChanged(self, d.channelId, d.shortIds, commitments1))
@ -391,14 +396,10 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val
// we acknowledge the command right away in case of failure
handleCommandError(cause, c).acking(d.channelId, c)
}
}
case Event(c: CMD_FAIL_MALFORMED_HTLC, d: DATA_NORMAL) =>
c.delay_opt match {
case Some(delay) =>
log.debug("delaying CMD_FAIL_MALFORMED_HTLC with id={} for {}", c.id, delay)
context.system.scheduler.scheduleOnce(delay, self, c.copy(delay_opt = None))
stay()
case None => Commitments.sendFailMalformed(d.commitments, c) match {
Commitments.sendFailMalformed(d.commitments, c) match {
case Right((commitments1, fail)) =>
if (c.commit) self ! CMD_SIGN()
context.system.eventStream.publish(AvailableBalanceChanged(self, d.channelId, d.shortIds, commitments1))
@ -407,7 +408,6 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val
// we acknowledge the command right away in case of failure
handleCommandError(cause, c).acking(d.channelId, c)
}
}
case Event(fail: UpdateFailHtlc, d: DATA_NORMAL) =>
Commitments.receiveFail(d.commitments, fail) match {

View file

@ -158,6 +158,7 @@ object Bolt11Invoice {
paymentMetadata: Option[ByteVector] = None,
features: Features[InvoiceFeature] = defaultFeatures): Bolt11Invoice = {
require(features.hasFeature(Features.PaymentSecret, Some(FeatureSupport.Mandatory)), "invoices must require a payment secret")
require(!features.hasFeature(Features.RouteBlinding), "bolt11 invoices cannot use route blinding")
val prefix = prefixes(chainHash)
val tags = {
val defaultTags = List(

View file

@ -303,7 +303,7 @@ object OutgoingPaymentPacket {
}
}
def buildHtlcFailure(nodeSecret: PrivateKey, reason: Either[ByteVector, FailureMessage], add: UpdateAddHtlc): Either[CannotExtractSharedSecret, ByteVector] = {
private def buildHtlcFailure(nodeSecret: PrivateKey, reason: Either[ByteVector, FailureMessage], add: UpdateAddHtlc): Either[CannotExtractSharedSecret, ByteVector] = {
Sphinx.peel(nodeSecret, Some(add.paymentHash), add.onionRoutingPacket) match {
case Right(Sphinx.DecryptedPacket(_, _, sharedSecret)) =>
val encryptedReason = reason match {
@ -315,8 +315,15 @@ object OutgoingPaymentPacket {
}
}
def buildHtlcFailure(nodeSecret: PrivateKey, cmd: CMD_FAIL_HTLC, add: UpdateAddHtlc): Either[CannotExtractSharedSecret, UpdateFailHtlc] = {
def buildHtlcFailure(nodeSecret: PrivateKey, cmd: CMD_FAIL_HTLC, add: UpdateAddHtlc): Either[CannotExtractSharedSecret, HtlcFailureMessage] = {
add.blinding_opt match {
case Some(_) =>
// We are part of a blinded route and we're not the introduction node.
val failure = InvalidOnionBlinding(Sphinx.hash(add.onionRoutingPacket))
Right(UpdateFailMalformedHtlc(add.channelId, add.id, failure.onionHash, failure.code))
case None =>
buildHtlcFailure(nodeSecret, cmd.reason, add).map(encryptedReason => UpdateFailHtlc(add.channelId, cmd.id, encryptedReason))
}
}
}

View file

@ -28,6 +28,7 @@ import fr.acinq.eclair.db.PendingCommandsDb
import fr.acinq.eclair.payment.Monitoring.{Metrics, Tags}
import fr.acinq.eclair.payment.relay.Relayer.{OutgoingChannel, OutgoingChannelParams}
import fr.acinq.eclair.payment.{ChannelPaymentRelayed, IncomingPaymentPacket}
import fr.acinq.eclair.wire.protocol.FailureMessageCodecs.createBadOnionFailure
import fr.acinq.eclair.wire.protocol.PaymentOnion.IntermediatePayload
import fr.acinq.eclair.wire.protocol._
import fr.acinq.eclair.{Logs, NodeParams, TimestampSecond, channel, nodeFee}
@ -83,21 +84,10 @@ object ChannelRelay {
}
}
def translateRelayFailure(originHtlcId: Long, fail: HtlcResult.Fail, relayPacket_opt: Option[IncomingPaymentPacket.ChannelRelayPacket]): channel.Command with channel.HtlcSettlementCommand = {
def translateRelayFailure(originHtlcId: Long, fail: HtlcResult.Fail): CMD_FAIL_HTLC = {
fail match {
case f: HtlcResult.RemoteFail => CMD_FAIL_HTLC(originHtlcId, Left(f.fail.reason), commit = true)
case f: HtlcResult.RemoteFailMalformed => relayPacket_opt match {
case Some(IncomingPaymentPacket.ChannelRelayPacket(add, payload: IntermediatePayload.ChannelRelay.Blinded, _)) =>
// Bolt 2:
// - if it is part of a blinded route:
// - MUST return an `update_fail_malformed_htlc` error using the `invalid_onion_blinding` failure code, with the `sha256_of_onion` of the onion it received.
// - If its onion payload contains `current_blinding_point`:
// - SHOULD add a random delay before sending `update_fail_malformed_htlc`.
val delay_opt = payload.records.get[OnionPaymentPayloadTlv.BlindingPoint].map(_ => Random.nextLong(1000).millis)
CMD_FAIL_MALFORMED_HTLC(originHtlcId, Sphinx.hash(add.onionRoutingPacket), InvalidOnionBlinding(ByteVector32.Zeroes).code, delay_opt, commit = true)
case _ =>
CMD_FAIL_MALFORMED_HTLC(originHtlcId, f.fail.onionHash, f.fail.failureCode, commit = true)
}
case f: HtlcResult.RemoteFailMalformed => CMD_FAIL_HTLC(originHtlcId, Right(createBadOnionFailure(f.fail.onionHash, f.fail.failureCode)), commit = true)
case _: HtlcResult.OnChainFail => CMD_FAIL_HTLC(originHtlcId, Right(PermanentChannelFailure), commit = true)
case HtlcResult.ChannelFailureBeforeSigned => CMD_FAIL_HTLC(originHtlcId, Right(PermanentChannelFailure), commit = true)
case f: HtlcResult.DisconnectedBeforeSigned => CMD_FAIL_HTLC(originHtlcId, Right(TemporaryChannelFailure(f.channelUpdate)), commit = true)
@ -171,13 +161,31 @@ class ChannelRelay private(nodeParams: NodeParams,
case WrappedAddResponse(RES_ADD_SETTLED(o: Origin.ChannelRelayedHot, _, fail: HtlcResult.Fail)) =>
context.log.info("relaying fail to upstream")
Metrics.recordPaymentRelayFailed(Tags.FailureType.Remote, Tags.RelayType.Channel)
val cmd = translateRelayFailure(o.originHtlcId, fail, Some(r))
val cmd = translateRelayFailure(o.originHtlcId, fail)
safeSendAndStop(o.originChannelId, cmd)
}
def safeSendAndStop(channelId: ByteVector32, cmd: channel.Command with channel.HtlcSettlementCommand): Behavior[Command] = {
def safeSendAndStop(channelId: ByteVector32, cmd: channel.HtlcSettlementCommand): Behavior[Command] = {
val toSend = cmd match {
case _: CMD_FULFILL_HTLC => cmd
case _: CMD_FAIL_HTLC | _: CMD_FAIL_MALFORMED_HTLC => r.payload match {
case payload: IntermediatePayload.ChannelRelay.Blinded =>
// We are inside a blinded route, so we must carefully choose the error we return to avoid leaking information.
val failure = InvalidOnionBlinding(Sphinx.hash(r.add.onionRoutingPacket))
payload.records.get[OnionPaymentPayloadTlv.BlindingPoint] match {
case Some(_) =>
// We are the introduction node: we add a delay to make it look like it could come from further downstream.
val delay = Some(Random.nextLong(1000).millis)
CMD_FAIL_HTLC(cmd.id, Right(failure), delay, commit = true)
case None =>
// We are not the introduction node.
CMD_FAIL_MALFORMED_HTLC(cmd.id, failure.onionHash, failure.code, commit = true)
}
case _: IntermediatePayload.ChannelRelay.Standard => cmd
}
}
// NB: we are not using an adapter here because we are stopping anyway so we won't be there to get the result
PendingCommandsDb.safeSend(register, nodeParams.db.pendingCommands, channelId, cmd)
PendingCommandsDb.safeSend(register, nodeParams.db.pendingCommands, channelId, toSend)
Behaviors.stopped
}
@ -236,7 +244,7 @@ class ChannelRelay private(nodeParams: NodeParams,
channel.channelUpdate,
relayResult match {
case _: RelaySuccess => "success"
case RelayFailure(CMD_FAIL_HTLC(_, Right(failureReason), _, _)) => failureReason
case RelayFailure(CMD_FAIL_HTLC(_, Right(failureReason), _, _, _)) => failureReason
case other => other
})
(channel, relayResult)

View file

@ -235,7 +235,7 @@ class PostRestartHtlcCleaner(nodeParams: NodeParams, register: ActorRef, initial
case Origin.ChannelRelayedCold(originChannelId, originHtlcId, _, _) =>
log.warning(s"payment failed for paymentHash=${failedHtlc.paymentHash}: failing 1 HTLC upstream")
Metrics.Resolved.withTag(Tags.Success, value = false).withTag(Metrics.Relayed, value = true).increment()
val cmd = ChannelRelay.translateRelayFailure(originHtlcId, fail, None)
val cmd = ChannelRelay.translateRelayFailure(originHtlcId, fail)
PendingCommandsDb.safeSend(register, nodeParams.db.pendingCommands, originChannelId, cmd)
case Origin.TrampolineRelayedCold(origins) =>
log.warning(s"payment failed for paymentHash=${failedHtlc.paymentHash}: failing ${origins.length} HTLCs upstream")

View file

@ -78,14 +78,16 @@ class Relayer(nodeParams: NodeParams, router: ActorRef, register: ActorRef, paym
}
case Left(badOnion: BadOnion) =>
log.warning(s"couldn't parse onion: reason=${badOnion.message}")
val delay_opt = badOnion match {
val cmdFail = badOnion match {
case InvalidOnionBlinding(_) if add.blinding_opt.isEmpty =>
// We are the introduction point of a blinded path: we add a non-negligible delay to make it look like it
// could come from a downstream node.
case InvalidOnionBlinding(_) if add.blinding_opt.isEmpty => Some(500.millis + Random.nextLong(1500).millis)
case _ => None
val delay = Some(500.millis + Random.nextLong(1500).millis)
CMD_FAIL_HTLC(add.id, Right(InvalidOnionBlinding(badOnion.onionHash)), delay, commit = true)
case _ =>
CMD_FAIL_MALFORMED_HTLC(add.id, badOnion.onionHash, badOnion.code, commit = true)
}
val cmdFail = CMD_FAIL_MALFORMED_HTLC(add.id, badOnion.onionHash, badOnion.code, delay_opt, commit = true)
log.warning(s"rejecting htlc #${add.id} from channelId=${add.channelId} reason=malformed onionHash=${cmdFail.onionHash} failureCode=${cmdFail.failureCode}")
log.warning(s"rejecting htlc #${add.id} from channelId=${add.channelId} reason=malformed onionHash=${badOnion.onionHash} failureCode=${badOnion.code}")
PendingCommandsDb.safeSend(register, nodeParams.db.pendingCommands, add.channelId, cmdFail)
case Left(failure) =>
log.warning(s"rejecting htlc #${add.id} from channelId=${add.channelId} reason=$failure")

View file

@ -36,6 +36,8 @@ object CommandCodecs {
val cmdFailCodec: Codec[CMD_FAIL_HTLC] =
(("id" | int64) ::
("reason" | either(bool, varsizebinarydata, failureMessageCodec)) ::
// No need to delay commands after a restart, we've been offline which already created a random delay.
("delay_opt" | provide(Option.empty[FiniteDuration])) ::
("commit" | provide(false)) ::
("replyTo_opt" | provide(Option.empty[ActorRef]))).as[CMD_FAIL_HTLC]
@ -43,8 +45,6 @@ object CommandCodecs {
(("id" | int64) ::
("onionHash" | bytes32) ::
("failureCode" | uint16) ::
// No need to delay commands after a restart, we've been offline which already created a random delay.
("delay_opt" | provide(Option.empty[FiniteDuration])) ::
("commit" | provide(false)) ::
("replyTo_opt" | provide(Option.empty[ActorRef]))).as[CMD_FAIL_MALFORMED_HTLC]

View file

@ -163,4 +163,20 @@ object FailureMessageCodecs {
*/
def failureOnionCodec(mac: Mac32, payloadAndPadLength: Int = 256): Codec[FailureMessage] = CommonCodecs.prependmac(failureOnionPayload(payloadAndPadLength).complete, mac)
/** Create a BadOnion failure matching the failure code provided. */
def createBadOnionFailure(onionHash: ByteVector32, failureCode: Int): BadOnion = {
if (failureCode == (BADONION | PERM | 4)) {
InvalidOnionVersion(onionHash)
} else if (failureCode == (BADONION | PERM | 5)) {
InvalidOnionHmac(onionHash)
} else if (failureCode == (BADONION | PERM | 6)) {
InvalidOnionKey(onionHash)
} else if (failureCode == (BADONION | PERM | 24)) {
InvalidOnionBlinding(onionHash)
} else {
// unknown failure code, we default to a generic error
InvalidOnionVersion(onionHash)
}
}
}

View file

@ -51,6 +51,7 @@ sealed trait HasChainHash extends LightningMessage { def chainHash: ByteVector32
sealed trait HasSerialId extends LightningMessage { def serialId: UInt64 } // <- not in the spec
sealed trait UpdateMessage extends HtlcMessage // <- not in the spec
sealed trait HtlcSettlementMessage extends UpdateMessage { def id: Long } // <- not in the spec
sealed trait HtlcFailureMessage extends HtlcSettlementMessage // <- not in the spec
// @formatter:on
case class Init(features: Features[InitFeature], tlvStream: TlvStream[InitTlv] = TlvStream.empty) extends SetupMessage {
@ -303,13 +304,13 @@ case class UpdateFulfillHtlc(channelId: ByteVector32,
case class UpdateFailHtlc(channelId: ByteVector32,
id: Long,
reason: ByteVector,
tlvStream: TlvStream[UpdateFailHtlcTlv] = TlvStream.empty) extends HtlcMessage with UpdateMessage with HasChannelId with HtlcSettlementMessage
tlvStream: TlvStream[UpdateFailHtlcTlv] = TlvStream.empty) extends HtlcMessage with UpdateMessage with HasChannelId with HtlcFailureMessage
case class UpdateFailMalformedHtlc(channelId: ByteVector32,
id: Long,
onionHash: ByteVector32,
failureCode: Int,
tlvStream: TlvStream[UpdateFailMalformedHtlcTlv] = TlvStream.empty) extends HtlcMessage with UpdateMessage with HasChannelId with HtlcSettlementMessage
tlvStream: TlvStream[UpdateFailMalformedHtlcTlv] = TlvStream.empty) extends HtlcMessage with UpdateMessage with HasChannelId with HtlcFailureMessage
case class CommitSig(channelId: ByteVector32,
signature: ByteVector64,

View file

@ -406,7 +406,7 @@ object PaymentOnion {
*/
case class Blinded(records: TlvStream[OnionPaymentPayloadTlv], blindedRecords: TlvStream[RouteBlindingEncryptedDataTlv]) extends FinalPayload {
override val amount = records.get[AmountToForward].get.amount
override val totalAmount = records.get[TotalAmount].map(_.totalAmount).getOrElse(amount)
override val totalAmount = records.get[TotalAmount].get.totalAmount
override val expiry = records.get[OutgoingCltv].get.cltv
val blinding_opt: Option[PublicKey] = records.get[BlindingPoint].map(_.publicKey)
val pathId = blindedRecords.get[RouteBlindingEncryptedDataTlv.PathId].get.data
@ -419,6 +419,7 @@ object PaymentOnion {
if (records.get[AmountToForward].isEmpty) return Left(MissingRequiredTlv(UInt64(2)))
if (records.get[OutgoingCltv].isEmpty) return Left(MissingRequiredTlv(UInt64(4)))
if (records.get[EncryptedRecipientData].isEmpty) return Left(MissingRequiredTlv(UInt64(10)))
if (records.get[TotalAmount].isEmpty) return Left(MissingRequiredTlv(UInt64(18)))
// Bolt 4: MUST return an error if the payload contains other tlv fields than `encrypted_recipient_data`, `current_blinding_point`, `amt_to_forward`, `outgoing_cltv_value` and `total_amount_msat`.
if (records.unknown.nonEmpty) return Left(ForbiddenTlv(records.unknown.head.tag))
records.records.find {

View file

@ -95,6 +95,7 @@ object TestConstants {
PaymentSecret -> Mandatory,
BasicMultiPartPayment -> Optional,
PaymentMetadata -> Optional,
RouteBlinding -> Optional,
),
unknown = Set(UnknownFeature(TestFeature.optional))
),
@ -245,6 +246,7 @@ object TestConstants {
PaymentSecret -> Mandatory,
BasicMultiPartPayment -> Optional,
PaymentMetadata -> Optional,
RouteBlinding -> Optional,
),
pluginParams = Nil,
overrideInitFeatures = Map.empty,

View file

@ -26,7 +26,7 @@ import fr.acinq.eclair.crypto.ShaChain
import fr.acinq.eclair.crypto.keymanager.LocalChannelKeyManager
import fr.acinq.eclair.transactions.CommitmentSpec
import fr.acinq.eclair.transactions.Transactions.CommitTx
import fr.acinq.eclair.wire.protocol.{IncorrectOrUnknownPaymentDetails, UpdateAddHtlc}
import fr.acinq.eclair.wire.protocol.{IncorrectOrUnknownPaymentDetails, UpdateAddHtlc, UpdateFailHtlc}
import fr.acinq.eclair.{TestKitBaseClass, _}
import org.scalatest.funsuite.FixtureAnyFunSuiteLike
import org.scalatest.{Outcome, Tag}
@ -199,7 +199,7 @@ class CommitmentsSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
assert(bc4.availableBalanceForReceive == a - p - htlcOutputFee)
val cmdFail = CMD_FAIL_HTLC(0, Right(IncorrectOrUnknownPaymentDetails(p, BlockHeight(42))))
val Right((bc5, fail)) = sendFail(bc4, cmdFail, bob.underlyingActor.nodeParams.privateKey)
val Right((bc5, fail: UpdateFailHtlc)) = sendFail(bc4, cmdFail, bob.underlyingActor.nodeParams.privateKey)
assert(bc5.availableBalanceForSend == b)
assert(bc5.availableBalanceForReceive == a - p - htlcOutputFee) // a's balance won't return to previous before she acknowledges the fail
@ -322,7 +322,7 @@ class CommitmentsSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
assert(bc8.availableBalanceForReceive == a - p1 - htlcOutputFee - p2 - htlcOutputFee - htlcOutputFee)
val cmdFail2 = CMD_FAIL_HTLC(1, Right(IncorrectOrUnknownPaymentDetails(p2, BlockHeight(42))))
val Right((bc9, fail2)) = sendFail(bc8, cmdFail2, bob.underlyingActor.nodeParams.privateKey)
val Right((bc9, fail2: UpdateFailHtlc)) = sendFail(bc8, cmdFail2, bob.underlyingActor.nodeParams.privateKey)
assert(bc9.availableBalanceForSend == b + p1 - p3)
assert(bc9.availableBalanceForReceive == a - p1 - htlcOutputFee - p2 - htlcOutputFee - htlcOutputFee) // a's balance won't return to previous before she acknowledges the fail

View file

@ -1716,6 +1716,22 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
testCmdFailHtlc _
}
test("recv CMD_FAIL_HTLC (with delay)") { f =>
import f._
val (_, htlc) = addHtlc(50000000 msat, alice, bob, alice2bob, bob2alice)
crossSign(alice, bob, alice2bob, bob2alice)
val initialState = bob.stateData.asInstanceOf[DATA_NORMAL]
val cmd = CMD_FAIL_HTLC(htlc.id, Right(PermanentChannelFailure), delay_opt = Some(50 millis))
val Right(fail) = OutgoingPaymentPacket.buildHtlcFailure(Bob.nodeParams.privateKey, cmd, htlc)
assert(fail.id == htlc.id)
bob ! cmd
bob2alice.expectMsg(fail)
awaitCond(bob.stateData == initialState.copy(
commitments = initialState.commitments.copy(
localChanges = initialState.commitments.localChanges.copy(initialState.commitments.localChanges.proposed :+ fail))))
}
test("recv CMD_FAIL_HTLC (unknown htlc id)") { f =>
import f._
val sender = TestProbe()
@ -1771,20 +1787,6 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
localChanges = initialState.commitments.localChanges.copy(initialState.commitments.localChanges.proposed :+ fail))))
}
test("recv CMD_FAIL_MALFORMED_HTLC (with delay)") { f =>
import f._
val (_, htlc) = addHtlc(50000000 msat, alice, bob, alice2bob, bob2alice)
crossSign(alice, bob, alice2bob, bob2alice)
// actual test begins
val initialState = bob.stateData.asInstanceOf[DATA_NORMAL]
bob ! CMD_FAIL_MALFORMED_HTLC(htlc.id, Sphinx.hash(htlc.onionRoutingPacket), FailureMessageCodecs.BADONION | FailureMessageCodecs.PERM | 24, delay_opt = Some(50 millis))
val fail = bob2alice.expectMsgType[UpdateFailMalformedHtlc]
awaitCond(bob.stateData == initialState.copy(
commitments = initialState.commitments.copy(
localChanges = initialState.commitments.localChanges.copy(initialState.commitments.localChanges.proposed :+ fail))))
}
test("recv CMD_FAIL_MALFORMED_HTLC (unknown htlc id)") { f =>
import f._
val sender = TestProbe()

View file

@ -471,12 +471,19 @@ class SphinxSpec extends AnyFunSuite {
// The sender includes the correct encrypted recipient data in each blinded node's payload.
TlvStream[OnionPaymentPayloadTlv](OnionPaymentPayloadTlv.EncryptedRecipientData(blindedRoute.encryptedPayloads(1))),
TlvStream[OnionPaymentPayloadTlv](OnionPaymentPayloadTlv.EncryptedRecipientData(blindedRoute.encryptedPayloads(2))),
TlvStream[OnionPaymentPayloadTlv](OnionPaymentPayloadTlv.AmountToForward(100_000 msat), OnionPaymentPayloadTlv.OutgoingCltv(CltvExpiry(749000)), OnionPaymentPayloadTlv.EncryptedRecipientData(blindedRoute.encryptedPayloads(3))),
TlvStream[OnionPaymentPayloadTlv](OnionPaymentPayloadTlv.AmountToForward(100_000 msat), OnionPaymentPayloadTlv.TotalAmount(150_000 msat), OnionPaymentPayloadTlv.OutgoingCltv(CltvExpiry(749000)), OnionPaymentPayloadTlv.EncryptedRecipientData(blindedRoute.encryptedPayloads(3))),
).map(tlvs => PaymentOnionCodecs.perHopPayloadCodec.encode(tlvs).require.bytes)
assert(payloads == Seq(
hex"14020301ae2d04030b6e5e0608000000000000000a",
hex"740a4fcd7b00ff9c09ed28102b210ac73aa12d63e90852cebc496c49f57c499a2888b49f2e72b19446f7e60a818aa2938d8c625415b992b8928a7321edb8f7cea40de362bed082ad51acc6156dca5532fb680c21024d4b6cd1361032ca9bd2aeb9d900aa4d45d9ead80ac9423374c451a7254d0766",
hex"510a4fcc0f16524fd7f8bb0f4e8d40ad71709ef140174c76faa574cac401bb8992fef76c4d004aa485dd599ed1cf2715f570f656a5aaecaf1ee8dc9d0fa1d424759be1932a8f29fac08bc2d2a1ed7159f28b",
hex"510a4f0fa1a72cff3b64a3d6e1e4903cf8c8b0a17144aeb249dcb86561adee1f679ee8db3e561d9e49895fd4bcebf6f58d6f61a6d41a9bf5aa4b0453437856632e8255c351873143ddf2bb2b0832b091e1b4",
hex"6002030186a004030b6dc80a4fda1c7e5f7881219884beae6ae68971de73bab4c3055d9865b1afb60722a63c688768042ade22f2c22f5724767d171fd221d3e579e43b354cc72e3ef146ada91a892d95fc48662f5b158add0af457da12030249f0",
))
val nodeIds = Seq(alice, bob).map(_.publicKey) ++ blindedRoute.blindedNodeIds.tail
val Success(PacketAndSecrets(onion, sharedSecrets)) = create(sessionKey, 1300, nodeIds, payloads, associatedData)
assert(serializePaymentOnion(onion) == hex"0002531fe6068134503d2723133227c867ac8fa6c83c537e9a44c3c5bdbdcb1fe337dadf610256c6ab518495dce9cdedf9391e21a71daddfe667387c384267a4c6453777590fc38e591b4f04a1e96bd1dec4af605d6adda2690de4ebe5d56ad2013b520af2a3c49316bc590ee83e8c31b1eb11ff766dad27ca993326b1ed582fb451a2ad87fbf6601134c6341c4a2deb6850e25a355be68dbb6923dc89444fdd74a0f700433b667bda345926099f5547b07e97ad903e8a01566a78ae177366239e793dac719de805565b6d0a942f42722a79dba29ebf4f9ec40cf579191716aac3a79f78c1d43398fba3f304786435976102a924ba4ba3de6150c829ce01c25428f2f5d05ef023be7d590ecdf6603730db3948f80ca1ed3d85227e64ef77200b9b557f427b6e1073cfa0e63e4485441768b98ab11ba8104a6cee1d7af7bb9d3167503ea010fabcd207b0b37a68b84be55663802d96faee291e8241b5e6c4b38e0c6d17ef6ba7bbe93f02046975bb01b7f766fcfc5a755af11a90cc7eb3505986b56e07a7855534d03b79f0dfbfe645b0d6d4185c038771fd25b800aa26b2ed2e30b1e713659468618a2fea04fcd04732a6fa9e77db73d0efa5253e123d5c2306ddb9ebf7bc897b559cf9870715039c6183082d762b6417f99d0f71ff7c060f6b564ad6827edaffa72eefcc4ce633a8da8d41c19d8f6aebd8878869eb518ccc16dccae6a94c690957598ce0295c1c46af5d7a2f0955b5400526bfd1430f554562614b5d00feff3946427be520cce52bfe9b6a9c2b1da6701c8ca628a69d6d40e20dd69d6e879d7a052d9c16f52c26e3bf745daeb3578c211475f2953e3c42308af89f3fd3c93bb4ba7320b35721bfdf2ad3db94b711fbdccdbe8465d9ff7bc9a293861dcea15bfa4f64993e9a751f571ab24a3219446483968821aa19a8d89ec611d686ff5f8fdc340aa8185ae29b01e60fb5a4c5c4bf8054c711522fc74e1d60976c33d2dfd782bbd555b8d06af6e688b3f541f1275706d045c607eea5926c49ced5bd368914f5ef793c3d6c1ab08dae689f0d71d64ec9c136cd38ac038cfa37846e3df7ce4bf63f44fce412bf3c9b8f21eabc34186a9c660b23fb7f3fa26cc9d830b40b499c613c2569d5e5f10823854471d3ac8bf655b020c37309fbaa0d0af5f14babd9485347ccd891bbd1e3b73e800c500be25073ee8a3844aca1cb9fa06d5579532da09a480cbec171b2ca9f83985d1a8cf60092fedaa88d4ccc711243298beb3d9d46c87542072aebb33d5a5ee671d4974b93c901eb1b5b4eaefc3669a7daa5154dced8cdc1bf49c1ba829bcbdee4e1f2f703c983872a7bff0669c9322c13a7cfb3f7f98b7ddcb47042a4786368a182f9c667d495438b6dee2d2a6ad0f8795ac499c3c3e9d584f6cf8279497fecc51c9203510858d738cec815a13d35d220ea297333068d8b64f4bcb627d127ab1e7732c840da45d35647e9e319bac2e95bb49f070e32772e2a8a6b55ca35d2391de4269cd6c5030203ab14abfca973a032b6ce10e958f1be2399c98ee70da0363c2f9a4e52546d8eef0b63cbab415a9341dbb9099df5e1ba2a83c2be15a96518741eacbe0f5d45e81ed5ddb76438a45cc5bb8d87abba0dd8c9181eff8b1f7c3939f3600883a3139515c53a07429247db278384d727d9b3b327c0f47dd4319d12e24ac2713f8c828217491df60f5b002cc58476a7b857dffb148179ffa5c62060d26dc3a9df11beccf77929e5d752d7351e58dc7f5265946792e7733886240efa0994868aa28a66754dccee99abd37a78558c858ddc9ca52aee32e263dd5165cdaf8ff74dfa9b61506af68b2fc9c0b887d3e49cc534040221f72fe6ec705e3964ad1e6d686840dd821c7a386baecb841369c98f5b493820be03c3b726cba925c72b05ea3d1b")
assert(serializePaymentOnion(onion) == hex"0002531fe6068134503d2723133227c867ac8fa6c83c537e9a44c3c5bdbdcb1fe337dadf610256c6ab518495dce9cdedf9391e21a71dada75be905267ba82f326c0513dda706908cfee834996700f881b2aed106585d61a2690de4ebe5d56ad2013b520af2a3c49316bc590ee83e8c31b1eb11ff766dad27ca993326b1ed582fb451a2ad87fbf6601134c6341c4a2deb6850e25a355be68dbb6923dc89444fdd74a0f700433b667bda345926099f5547b07e97ad903e8a01566a78ae177366239e793dac719de805565b6d0a1d290e273f705cfc56873f8b5e28225f7ded7a1d4ceffae63f91e477be8c917c786435976102a924ba4ba3de6150c829ce01c25428f2f5d05ef023be7d590ecdf6603730db3948f80ca1ed3d85227e64ef77200b9b557f427b6e1073cfa0e63e4485441768b98ab11ba8104a6cee1d7af7bb5ee9c05cf9cf4718901e92e09dfe5cb3af336a953072391c1e91fc2f4b92e124b38e0c6d17ef6ba7bbe93f02046975bb01b7f766fcfc5a755af11a90cc7eb3505986b56e07a7855534d03b79f0dfbfe645b0d6d4185c038771fd25b800aa26b2ed2e30b1e713659468618a2fea04fcd0473284598f76b11b0d159d343bc9711d3bea8d561547bcc8fff12317c0e7b1ee75bcb8082d762b6417f99d0f71ff7c060f6b564ad6827edaffa72eefcc4ce633a8da8d41c19d8f6aebd8878869eb518ccc16dccae6a94c690957598ce0295c1c46af5d7a2f0955b5400526bfd1430f554562614b5d00feff3946427be520dee629b76b6a9c2b1da6701c8ca628a69d6d40e20dd69d6e879d7a052d9c16f544b49738c7ff3cdd0613e9ed00ead7707702d1a6a0b88de1927a50c36beb78f4ff81e3dd97b706307596eebb363d418a891e1cb4589ce86ce81cdc0e1473d7a7dd5f6bb6e147c1f7c46fa879b4512c25704da6cdbb3c123a72e3585dc07b3e5cbe7fecf3a08426eee8c70ddc46ebf98b0bcb14a08c469cb5cfb6702acc0befd17640fa60244eca491280a95fbbc5833d26e4be70fcf798b55e06eb9fcb156942dcf108236f32a5a6c605687ba4f037eddbb1834dcbcd5293a0b66c621346ca5d893d239c26619b24c71f25cecc275e1ab24436ac01c80c0006fab2d95e82e3a0c3ea02d08ec5b24eb39205c49f4b549dcab7a88962336c4624716902f4e08f2b23cfd324f18405d66e9da3627ac34a6873ba2238386313af20d5a13bbd507fdc73015a17e3bd38fae1145f7f70d7cb8c5e1cdf9cf06d1246592a25d56ec2ae44cd7f75aa7f5f4a2b2ee49a41a26be4fab3f3f2ceb7b08510c5e2b7255326e4c417325b333cafe96dde1314a15dd6779a7d5a8a40622260041e936247eec8ec39ca29a1e18161db37497bdd4447a7d5ef3b8d22a2acd7f486b152bb66d3a15afc41dc9245a8d75e1d33704d4471e417ccc8d31645fdd647a2c191692675cf97664951d6ce98237d78b0962ad1433b5a3e49ddddbf57a391b14dcce00b4d7efe5cbb1e78f30d5ef53d66c381a45e275d2dcf6be559acb3c42494a9a2156eb8dcf03dd92b2ebaa697ea628fa0f75f125e4a7daa10f8dcf56ebaf7814557708c75580fad2bbb33e66ad7a4788a7aaac792aaae76138d7ff09df6a1a1920ddcf22e5e7007b15171b51ff81799355232ce39f7d5ceeaf704255d790041d6390a69f42816cba641ec81faa3d7c0fdec59dfe4ca41f31a692eaffc66b083995d86c575aea4514a3e09e8b3a1fa4d1591a2505f253ad0b6bfd9d87f063d2be414d3a427c0506a88ac5bdbef9b50d73bce876f85c196dca435e210e1d6713695b529ddda3350fb5065a6a8288abd265380917bac8ebbc7d5ced564587471dddf90c22ce6dbadea7e7a6723438d4cf6ac6dae27d033a8cadd77ab262e8defb33445ddb2056ec364c7629c33745e2338")
// Alice can decrypt the onion as usual.
val Right(DecryptedPacket(onionPayloadAlice, packetForBob, sharedSecretAlice)) = peel(alice, associatedData, onion)
@ -555,10 +562,10 @@ class SphinxSpec extends AnyFunSuite {
assert(Seq(sharedSecretAlice, sharedSecretBob, sharedSecretCarol, sharedSecretDave, sharedSecretEve) == sharedSecrets.map(_._1))
val packets = Seq(packetForBob, packetForCarol, packetForDave, packetForEve, packetForNobody)
assert(packets(0).hmac == ByteVector32(hex"0b462fb9321df3f139d2efccdc54471840e5cb50b4f7dae44df9c8c3e5ffabde"))
assert(packets(1).hmac == ByteVector32(hex"5c7b8d4f3061b3e58194edfb76ac339932c61ff77b024192508c9628a0206bb7"))
assert(packets(2).hmac == ByteVector32(hex"6a8df602e649e459b456df92327d7cf28132b735d38d3692c7c199e27d298c85"))
assert(packets(3).hmac == ByteVector32(hex"6db2bc62c58cd931570f8b7eb13b96e40b8a34e13655eb4f4b3a3ec87824403d"))
assert(packets(0).hmac == ByteVector32(hex"73fba184685e19b9af78afe876aa4e4b4242382b293133771d95a2bd83fa9c62"))
assert(packets(1).hmac == ByteVector32(hex"d57dc11a256834bb49ac9deeec88e4bf563c7340f44a240caec941c7e50f09cf"))
assert(packets(2).hmac == ByteVector32(hex"a955510e2126b3bb989a4ac21cf948f965e48bc363d2997437797b4f770e8b65"))
assert(packets(3).hmac == ByteVector32(hex"4f11ad63afe404cb6f1e8ea5fd7a8e085b65ca5136146febf4d47928dcc9a9e0"))
assert(packets(4).hmac == ByteVector32(hex"0000000000000000000000000000000000000000000000000000000000000000"))
}
}

View file

@ -100,6 +100,7 @@ abstract class IntegrationSpec extends TestKitBaseClass with BitcoindService wit
s"eclair.features.${Wumbo.rfcName}" -> "optional",
s"eclair.features.${ShutdownAnySegwit.rfcName}" -> "optional",
s"eclair.features.${ChannelType.rfcName}" -> "optional",
s"eclair.features.${RouteBlinding.rfcName}" -> "optional",
).asJava)
val withDefaultCommitment = commonFeatures.withFallback(ConfigFactory.parseMap(Map(

View file

@ -33,15 +33,16 @@ import fr.acinq.eclair.crypto.TransportHandler
import fr.acinq.eclair.db._
import fr.acinq.eclair.io.Peer.PeerRoutingMessage
import fr.acinq.eclair.payment._
import fr.acinq.eclair.payment.receive.MultiPartHandler.ReceiveStandardPayment
import fr.acinq.eclair.payment.receive.MultiPartHandler.{DummyBlindedHop, ReceiveOfferPayment, ReceiveStandardPayment, ReceivingRoute}
import fr.acinq.eclair.payment.relay.Relayer
import fr.acinq.eclair.payment.relay.Relayer.RelayFees
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}
import fr.acinq.eclair.wire.protocol.OfferTypes.{InvoiceRequest, Offer}
import fr.acinq.eclair.wire.protocol.{ChannelAnnouncement, ChannelUpdate, IncorrectOrUnknownPaymentDetails}
import fr.acinq.eclair.{CltvExpiryDelta, Features, Kit, MilliSatoshiLong, ShortChannelId, TimestampMilli, randomBytes32}
import fr.acinq.eclair.{CltvExpiryDelta, Features, Kit, MilliSatoshiLong, ShortChannelId, TimestampMilli, randomBytes32, randomKey}
import org.json4s.JsonAST.{JString, JValue}
import scodec.bits.ByteVector
@ -682,6 +683,93 @@ class PaymentIntegrationSpec extends IntegrationSpec {
assert(status.route.lastOption.contains(HopSummary(nodes("C").nodeParams.nodeId, nodes("D").nodeParams.nodeId)), status)
}
test("send a blinded payment A->D with many blinded routes") {
val sender = TestProbe()
val recipientKey = randomKey()
val amount = 50_000_000 msat
val chain = nodes("D").nodeParams.chainHash
val offer = Offer(Some(amount), "test offer", recipientKey.publicKey, nodes("D").nodeParams.features.invoiceFeatures(), chain)
val invoiceRequest = InvoiceRequest(offer, amount, 1, nodes("A").nodeParams.features.invoiceFeatures(), randomKey(), chain)
val receivingRoutes = Seq(
ReceivingRoute(Seq(nodes("G").nodeParams.nodeId, nodes("C").nodeParams.nodeId, nodes("D").nodeParams.nodeId), CltvExpiryDelta(1000)),
ReceivingRoute(Seq(nodes("B").nodeParams.nodeId, nodes("C").nodeParams.nodeId, nodes("D").nodeParams.nodeId), CltvExpiryDelta(1000)),
ReceivingRoute(Seq(nodes("E").nodeParams.nodeId, nodes("C").nodeParams.nodeId, nodes("D").nodeParams.nodeId), CltvExpiryDelta(1000)),
)
sender.send(nodes("D").paymentHandler, ReceiveOfferPayment(recipientKey, offer, invoiceRequest, receivingRoutes, nodes("D").router))
val invoice = sender.expectMsgType[Bolt12Invoice]
assert(invoice.blindedPaths.length == 3)
assert(invoice.blindedPathsInfo.length == 3)
assert(invoice.nodeId == recipientKey.publicKey)
sender.send(nodes("A").paymentInitiator, SendPaymentToNode(amount, invoice, maxAttempts = 3, routeParams = integrationTestRouteParams))
val paymentId = sender.expectMsgType[UUID]
val paymentSent = sender.expectMsgType[PaymentSent](max = 30 seconds)
assert(paymentSent.id == paymentId, paymentSent)
assert(paymentSent.paymentHash == invoice.paymentHash, paymentSent)
assert(paymentSent.recipientAmount == amount, paymentSent)
assert(paymentSent.feesPaid > 0.msat, paymentSent)
awaitCond(nodes("D").nodeParams.db.payments.getIncomingPayment(invoice.paymentHash).exists(_.status.isInstanceOf[IncomingPaymentStatus.Received]))
val Some(IncomingBlindedPayment(_, _, _, _, _, IncomingPaymentStatus.Received(receivedAmount, _))) = nodes("D").nodeParams.db.payments.getIncomingPayment(invoice.paymentHash)
assert(receivedAmount >= amount)
}
test("send a blinded payment D->C with empty blinded routes") {
val sender = TestProbe()
val amount = 25_000_000 msat
val chain = nodes("C").nodeParams.chainHash
val offer = Offer(Some(amount), "test offer", nodes("C").nodeParams.nodeId, nodes("C").nodeParams.features.invoiceFeatures(), chain)
val invoiceRequest = InvoiceRequest(offer, amount, 1, nodes("D").nodeParams.features.invoiceFeatures(), randomKey(), chain)
// C uses a 0-hop blinded route and signs the invoice with its public nodeId.
val receivingRoutes = Seq(
ReceivingRoute(Seq(nodes("C").nodeParams.nodeId), CltvExpiryDelta(1000)),
ReceivingRoute(Seq(nodes("C").nodeParams.nodeId), CltvExpiryDelta(1000)),
)
sender.send(nodes("C").paymentHandler, ReceiveOfferPayment(nodes("C").nodeParams.privateKey, offer, invoiceRequest, receivingRoutes, nodes("C").router))
val invoice = sender.expectMsgType[Bolt12Invoice]
assert(invoice.blindedPaths.length == 2)
assert(invoice.blindedPaths.forall(_.length == 0))
assert(invoice.nodeId == nodes("C").nodeParams.nodeId)
sender.send(nodes("D").paymentInitiator, SendPaymentToNode(amount, invoice, maxAttempts = 3, routeParams = integrationTestRouteParams))
val paymentId = sender.expectMsgType[UUID]
val paymentSent = sender.expectMsgType[PaymentSent](max = 30 seconds)
assert(paymentSent.id == paymentId)
assert(paymentSent.recipientAmount == amount, paymentSent)
assert(paymentSent.feesPaid == 0.msat, paymentSent)
awaitCond(nodes("C").nodeParams.db.payments.getIncomingPayment(invoice.paymentHash).exists(_.status.isInstanceOf[IncomingPaymentStatus.Received]))
val Some(IncomingBlindedPayment(_, _, _, _, _, IncomingPaymentStatus.Received(receivedAmount, _))) = nodes("C").nodeParams.db.payments.getIncomingPayment(invoice.paymentHash)
assert(receivedAmount == amount)
}
test("send a blinded payment B->A with dummy hops") {
val sender = TestProbe()
val recipientKey = randomKey()
val amount = 50_000_000 msat
val chain = nodes("A").nodeParams.chainHash
val offer = Offer(Some(amount), "test offer", recipientKey.publicKey, nodes("A").nodeParams.features.invoiceFeatures(), chain)
val invoiceRequest = InvoiceRequest(offer, amount, 1, nodes("B").nodeParams.features.invoiceFeatures(), randomKey(), chain)
val receivingRoutes = Seq(
ReceivingRoute(Seq(nodes("A").nodeParams.nodeId), CltvExpiryDelta(1000), Seq(DummyBlindedHop(100 msat, 100, CltvExpiryDelta(48)), DummyBlindedHop(150 msat, 50, CltvExpiryDelta(36))))
)
sender.send(nodes("A").paymentHandler, ReceiveOfferPayment(recipientKey, offer, invoiceRequest, receivingRoutes, nodes("A").router))
val invoice = sender.expectMsgType[Bolt12Invoice]
assert(invoice.blindedPaths.length == 1)
assert(invoice.nodeId == recipientKey.publicKey)
sender.send(nodes("B").paymentInitiator, SendPaymentToNode(amount, invoice, maxAttempts = 3, routeParams = integrationTestRouteParams))
val paymentId = sender.expectMsgType[UUID]
val paymentSent = sender.expectMsgType[PaymentSent](max = 30 seconds)
assert(paymentSent.id == paymentId)
assert(paymentSent.recipientAmount == amount, paymentSent)
assert(paymentSent.feesPaid >= 0.msat, paymentSent)
awaitCond(nodes("A").nodeParams.db.payments.getIncomingPayment(invoice.paymentHash).exists(_.status.isInstanceOf[IncomingPaymentStatus.Received]))
val Some(IncomingBlindedPayment(_, _, _, _, _, IncomingPaymentStatus.Received(receivedAmount, _))) = nodes("A").nodeParams.db.payments.getIncomingPayment(invoice.paymentHash)
assert(receivedAmount >= amount)
}
test("generate and validate lots of channels") {
val bitcoinClient = new BitcoinCoreClient(bitcoinrpcclient)
// we simulate fake channels by publishing a funding tx and sending announcement messages to a node at random

View file

@ -54,6 +54,7 @@ case class MinimalNodeFixture private(nodeParams: NodeParams,
wallet: DummyOnChainWallet,
bitcoinClient: TestBitcoinCoreClient) {
val nodeId = nodeParams.nodeId
val routeParams = nodeParams.routerConf.pathFindingExperimentConf.experiments.values.head.getDefaultRouteParams
}
object MinimalNodeFixture extends Assertions with Eventually with IntegrationPatience with EitherValues {
@ -300,6 +301,8 @@ object MinimalNodeFixture extends Assertions with Eventually with IntegrationPat
case None => timers.startSingleTimer(watch, 10 millis)
}
Behaviors.same
case _ =>
Behaviors.same
}
}
}

View file

@ -0,0 +1,323 @@
/*
* Copyright 2022 ACINQ SAS
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package fr.acinq.eclair.integration.basic.payment
import akka.testkit.TestProbe
import com.softwaremill.quicklens.ModifyPimp
import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
import fr.acinq.bitcoin.scalacompat.{ByteVector32, SatoshiLong}
import fr.acinq.eclair.FeatureSupport.Optional
import fr.acinq.eclair.Features.{KeySend, RouteBlinding}
import fr.acinq.eclair.channel.{DATA_NORMAL, RealScidStatus}
import fr.acinq.eclair.integration.basic.fixtures.MinimalNodeFixture
import fr.acinq.eclair.integration.basic.fixtures.MinimalNodeFixture.{connect, getChannelData, knownFundingTxs, nodeParamsFor, openChannel, watcherAutopilot}
import fr.acinq.eclair.integration.basic.fixtures.composite.ThreeNodesFixture
import fr.acinq.eclair.payment._
import fr.acinq.eclair.payment.receive.MultiPartHandler
import fr.acinq.eclair.payment.receive.MultiPartHandler.{DummyBlindedHop, ReceivingRoute}
import fr.acinq.eclair.payment.send.PaymentInitiator.{SendPaymentToNode, SendSpontaneousPayment}
import fr.acinq.eclair.payment.send.PaymentLifecycle
import fr.acinq.eclair.testutils.FixtureSpec
import fr.acinq.eclair.wire.protocol.OfferTypes.{InvoiceRequest, Offer}
import fr.acinq.eclair.wire.protocol.{IncorrectOrUnknownPaymentDetails, InvalidOnionBlinding}
import fr.acinq.eclair.{CltvExpiryDelta, Features, MilliSatoshi, MilliSatoshiLong, randomBytes32, randomKey}
import org.scalatest.concurrent.IntegrationPatience
import org.scalatest.{Tag, TestData}
import scodec.bits.HexStringSyntax
import java.util.UUID
class BlindedPaymentSpec extends FixtureSpec with IntegrationPatience {
type FixtureParam = ThreeNodesFixture
val PrivateChannels = "private_channels"
val RouteBlindingDisabledBob = "route_blinding_disabled_bob"
val RouteBlindingDisabledCarol = "route_blinding_disabled_carol"
val maxFinalExpiryDelta = CltvExpiryDelta(1000)
override def createFixture(testData: TestData): FixtureParam = {
// seeds have been chosen so that node ids start with 02aaaa for alice, 02bbbb for bob, etc.
val aliceParams = nodeParamsFor("alice", ByteVector32(hex"b4acd47335b25ab7b84b8c020997b12018592bb4631b868762154d77fa8b93a3"))
.modify(_.channelConf.maxHtlcValueInFlightPercent).setTo(100)
.modify(_.features.activated).using(_ + (RouteBlinding -> Optional))
.modify(_.channelConf.channelFlags.announceChannel).setTo(!testData.tags.contains(PrivateChannels))
val bobParams = nodeParamsFor("bob", ByteVector32(hex"7620226fec887b0b2ebe76492e5a3fd3eb0e47cd3773263f6a81b59a704dc492"))
.modify(_.channelConf.maxHtlcValueInFlightPercent).setTo(100)
.modify(_.features.activated).using(_ + (RouteBlinding -> Optional))
.modify(_.features.activated).usingIf(testData.tags.contains(RouteBlindingDisabledBob))(_ - RouteBlinding)
.modify(_.channelConf.channelFlags.announceChannel).setTo(!testData.tags.contains(PrivateChannels))
val carolParams = nodeParamsFor("carol", ByteVector32(hex"ebd5a5d3abfb3ef73731eb3418d918f247445183180522674666db98a66411cc"))
.modify(_.channelConf.maxHtlcValueInFlightPercent).setTo(100)
.modify(_.features.activated).using(_ + (RouteBlinding -> Optional))
.modify(_.features.activated).using(_ + (KeySend -> Optional))
.modify(_.features.activated).usingIf(testData.tags.contains(RouteBlindingDisabledCarol))(_ - RouteBlinding)
.modify(_.channelConf.channelFlags.announceChannel).setTo(!testData.tags.contains(PrivateChannels))
val f = ThreeNodesFixture(aliceParams, bobParams, carolParams, testData.name)
createChannels(f)
f
}
override def cleanupFixture(fixture: FixtureParam): Unit = {
fixture.cleanup()
}
private def createChannels(f: FixtureParam): Unit = {
import f._
alice.watcher.setAutoPilot(watcherAutopilot(knownFundingTxs(alice, bob, carol)))
bob.watcher.setAutoPilot(watcherAutopilot(knownFundingTxs(alice, bob, carol)))
carol.watcher.setAutoPilot(watcherAutopilot(knownFundingTxs(alice, bob, carol)))
connect(alice, bob)
connect(bob, carol)
val channelId_ab = openChannel(alice, bob, 500_000 sat).channelId
val channelId_bc_1 = openChannel(bob, carol, 100_000 sat).channelId
val channelId_bc_2 = openChannel(bob, carol, 100_000 sat).channelId
eventually {
assert(getChannelData(alice, channelId_ab).asInstanceOf[DATA_NORMAL].shortIds.real.isInstanceOf[RealScidStatus.Final])
assert(getChannelData(bob, channelId_bc_1).asInstanceOf[DATA_NORMAL].shortIds.real.isInstanceOf[RealScidStatus.Final])
assert(getChannelData(bob, channelId_bc_2).asInstanceOf[DATA_NORMAL].shortIds.real.isInstanceOf[RealScidStatus.Final])
}
}
def createInvoice(recipient: MinimalNodeFixture, amount: MilliSatoshi, routes: Seq[ReceivingRoute], sender: TestProbe): Bolt12Invoice = {
val offerKey = randomKey()
val offer = Offer(None, "test", offerKey.publicKey, Features.empty, recipient.nodeParams.chainHash)
val invoiceReq = InvoiceRequest(offer, amount, 1, Features.empty, randomKey(), recipient.nodeParams.chainHash)
sender.send(recipient.paymentHandler, MultiPartHandler.ReceiveOfferPayment(offerKey, offer, invoiceReq, routes, recipient.router))
val invoice = sender.expectMsgType[Bolt12Invoice]
assert(invoice.nodeId != recipient.nodeParams.nodeId)
invoice
}
def sendPaymentToCarol(f: FixtureParam, payer: MinimalNodeFixture, amount: MilliSatoshi, routes: Seq[ReceivingRoute]): (Bolt12Invoice, PaymentEvent) = {
import f._
val sender = TestProbe("sender")
val invoice = createInvoice(carol, amount, routes, sender)
sender.send(payer.paymentInitiator, SendPaymentToNode(amount, invoice, maxAttempts = 1, routeParams = payer.routeParams, blockUntilComplete = true))
(invoice, sender.expectMsgType[PaymentEvent])
}
def sendPaymentAliceToCarol(f: FixtureParam, amount: MilliSatoshi, routes: Seq[ReceivingRoute]): (Bolt12Invoice, PaymentEvent) = sendPaymentToCarol(f, f.alice, amount, routes)
def sendPaymentBobToCarol(f: FixtureParam, amount: MilliSatoshi, routes: Seq[ReceivingRoute]): (Bolt12Invoice, PaymentEvent) = sendPaymentToCarol(f, f.bob, amount, routes)
def verifyPaymentSuccess(invoice: Bolt12Invoice, result: PaymentEvent): PaymentSent = {
assert(result.isInstanceOf[PaymentSent])
val payment = result.asInstanceOf[PaymentSent]
assert(payment.recipientAmount == invoice.amount)
assert(payment.recipientNodeId == invoice.nodeId)
assert(payment.parts.map(_.amount).sum == invoice.amount)
payment
}
test("send blinded payment a->b->c") { f =>
import f._
val amount = 25_000_000 msat
val routes = Seq(ReceivingRoute(Seq(bob.nodeId, carol.nodeId), maxFinalExpiryDelta))
val (invoice, result) = sendPaymentAliceToCarol(f, amount, routes)
val payment = verifyPaymentSuccess(invoice, result)
assert(payment.parts.length == 1)
}
test("send blinded multi-part payment a->b->c") { f =>
import f._
val amount = 125_000_000 msat
val routes = Seq(
ReceivingRoute(Seq(bob.nodeId, carol.nodeId), maxFinalExpiryDelta),
ReceivingRoute(Seq(bob.nodeId, carol.nodeId), maxFinalExpiryDelta),
)
val (invoice, result) = sendPaymentAliceToCarol(f, amount, routes)
val payment = verifyPaymentSuccess(invoice, result)
assert(payment.parts.length == 2)
}
test("send blinded payment a->b->c with dummy hops") { f =>
import f._
val amount = 125_000_000 msat
val routes = Seq(
ReceivingRoute(Seq(bob.nodeId, carol.nodeId), maxFinalExpiryDelta, Seq(DummyBlindedHop(150 msat, 0, CltvExpiryDelta(50)))),
ReceivingRoute(Seq(bob.nodeId, carol.nodeId), maxFinalExpiryDelta, Seq(DummyBlindedHop(50 msat, 0, CltvExpiryDelta(20)), DummyBlindedHop(100 msat, 0, CltvExpiryDelta(30)))),
)
val (invoice, result) = sendPaymentAliceToCarol(f, amount, routes)
val payment = verifyPaymentSuccess(invoice, result)
assert(payment.parts.length == 2)
}
test("send blinded payment a->b->c through private channels", Tag(PrivateChannels)) { f =>
import f._
val amount = 50_000_000 msat
val routes = Seq(ReceivingRoute(Seq(bob.nodeId, carol.nodeId), maxFinalExpiryDelta))
val (invoice, result) = sendPaymentAliceToCarol(f, amount, routes)
verifyPaymentSuccess(invoice, result)
}
test("send blinded payment a->b") { f =>
import f._
val sender = TestProbe("sender")
val amount = 75_000_000 msat
val routes = Seq(ReceivingRoute(Seq(bob.nodeId), maxFinalExpiryDelta))
val invoice = createInvoice(bob, amount, routes, sender)
sender.send(alice.paymentInitiator, SendPaymentToNode(amount, invoice, maxAttempts = 1, routeParams = alice.routeParams, blockUntilComplete = true))
val payment = sender.expectMsgType[PaymentSent]
assert(payment.recipientAmount == invoice.amount)
assert(payment.recipientNodeId == invoice.nodeId)
assert(payment.parts.map(_.amount).sum == invoice.amount)
}
test("send blinded payment a->b with dummy hops") { f =>
import f._
val sender = TestProbe("sender")
val amount = 250_000_000 msat
val routes = Seq(ReceivingRoute(Seq(bob.nodeId), maxFinalExpiryDelta, Seq(DummyBlindedHop(10 msat, 25, CltvExpiryDelta(24)), DummyBlindedHop(5 msat, 10, CltvExpiryDelta(36)))))
val invoice = createInvoice(bob, amount, routes, sender)
sender.send(alice.paymentInitiator, SendPaymentToNode(amount, invoice, maxAttempts = 1, routeParams = alice.routeParams, blockUntilComplete = true))
val payment = sender.expectMsgType[PaymentSent]
assert(payment.recipientAmount == invoice.amount)
assert(payment.recipientNodeId == invoice.nodeId)
assert(payment.parts.map(_.amount).sum == invoice.amount)
}
test("send fully blinded payment b->c") { f =>
import f._
val amount = 50_000_000 msat
val routes = Seq(ReceivingRoute(Seq(bob.nodeId, carol.nodeId), maxFinalExpiryDelta))
val (invoice, result) = sendPaymentBobToCarol(f, amount, routes)
val payment = verifyPaymentSuccess(invoice, result)
assert(payment.parts.length == 1)
}
test("send fully blinded payment b->c with dummy hops") { f =>
import f._
val amount = 50_000_000 msat
val routes = Seq(ReceivingRoute(Seq(bob.nodeId, carol.nodeId), maxFinalExpiryDelta, Seq(DummyBlindedHop(25 msat, 250, CltvExpiryDelta(75)))))
val (invoice, result) = sendPaymentBobToCarol(f, amount, routes)
val payment = verifyPaymentSuccess(invoice, result)
assert(payment.parts.length == 1)
}
def verifyBlindedFailure(payment: PaymentFailed, expectedNode: PublicKey): Unit = {
assert(payment.failures.head.isInstanceOf[RemoteFailure])
val failure = payment.failures.head.asInstanceOf[RemoteFailure]
assert(failure.e.originNode == expectedNode)
assert(failure.e.failureMessage.isInstanceOf[InvalidOnionBlinding])
}
test("send blinded payment a->b->c failing at b") { f =>
import f._
val sender = TestProbe("sender")
val routes = Seq(ReceivingRoute(Seq(bob.nodeId, carol.nodeId), maxFinalExpiryDelta))
val invoice = createInvoice(carol, 75_000_000 msat, routes, sender)
// Bob sends payments to Carol to reduce the liquidity on both of his channels.
Seq(1, 2).foreach(_ => {
sender.send(bob.paymentInitiator, SendSpontaneousPayment(50_000_000 msat, carol.nodeId, randomBytes32(), 1, routeParams = bob.routeParams))
sender.expectMsgType[UUID]
sender.expectMsgType[PaymentSent]
})
// Bob now doesn't have enough funds to relay the payment.
sender.send(alice.paymentInitiator, SendPaymentToNode(invoice.amount, invoice, maxAttempts = 1, routeParams = alice.routeParams, blockUntilComplete = true))
val payment = sender.expectMsgType[PaymentFailed]
verifyBlindedFailure(payment, bob.nodeId)
}
test("send blinded payment a->b->c using expired route") { f =>
import f._
val sender = TestProbe("sender")
val routes = Seq(ReceivingRoute(Seq(bob.nodeId, carol.nodeId), CltvExpiryDelta(-500)))
val invoice = createInvoice(carol, 25_000_000 msat, routes, sender)
sender.send(alice.paymentInitiator, SendPaymentToNode(invoice.amount, invoice, maxAttempts = 1, routeParams = alice.routeParams, blockUntilComplete = true))
val payment = sender.expectMsgType[PaymentFailed]
verifyBlindedFailure(payment, bob.nodeId)
}
test("send blinded payment a->b->c failing at c") { f =>
import f._
val sender = TestProbe("sender")
val routes = Seq(ReceivingRoute(Seq(bob.nodeId, carol.nodeId), maxFinalExpiryDelta))
val invoice = createInvoice(carol, 25_000_000 msat, routes, sender)
// The amount is below what Carol expects.
sender.send(alice.paymentInitiator, SendPaymentToNode(20_000_000 msat, invoice, maxAttempts = 1, routeParams = alice.routeParams, blockUntilComplete = true))
val payment = sender.expectMsgType[PaymentFailed]
verifyBlindedFailure(payment, bob.nodeId)
}
test("send blinded payment a->b failing at b") { f =>
import f._
val sender = TestProbe("sender")
val routes = Seq(ReceivingRoute(Seq(bob.nodeId), maxFinalExpiryDelta))
val invoice = createInvoice(bob, 50_000_000 msat, routes, sender)
// The amount is below what Bob expects: since he is both the introduction node and the final recipient, he sends
// back a normal error.
sender.send(alice.paymentInitiator, SendPaymentToNode(25_000_000 msat, invoice, maxAttempts = 1, routeParams = alice.routeParams, blockUntilComplete = true))
val payment = sender.expectMsgType[PaymentFailed]
assert(payment.failures.head.isInstanceOf[RemoteFailure])
val failure = payment.failures.head.asInstanceOf[RemoteFailure]
assert(failure.e.originNode == bob.nodeId)
assert(failure.e.failureMessage.isInstanceOf[IncorrectOrUnknownPaymentDetails])
assert(failure.e.failureMessage.asInstanceOf[IncorrectOrUnknownPaymentDetails].amount == 25_000_000.msat)
}
test("send blinded payment a->b with dummy hops failing at b") { f =>
import f._
val sender = TestProbe("sender")
val routes = Seq(ReceivingRoute(Seq(bob.nodeId), maxFinalExpiryDelta, Seq(DummyBlindedHop(1 msat, 100, CltvExpiryDelta(48)))))
val invoice = createInvoice(bob, 50_000_000 msat, routes, sender)
// The amount is below what Bob expects: since he is both the introduction node and the final recipient, he sends
// back a normal error.
sender.send(alice.paymentInitiator, SendPaymentToNode(25_000_000 msat, invoice, maxAttempts = 1, routeParams = alice.routeParams, blockUntilComplete = true))
val payment = sender.expectMsgType[PaymentFailed]
assert(payment.failures.head.isInstanceOf[RemoteFailure])
val failure = payment.failures.head.asInstanceOf[RemoteFailure]
assert(failure.e.originNode == bob.nodeId)
assert(failure.e.failureMessage.isInstanceOf[IncorrectOrUnknownPaymentDetails])
assert(failure.e.failureMessage.asInstanceOf[IncorrectOrUnknownPaymentDetails].amount == 25_000_000.msat)
}
test("send fully blinded payment b->c failing at c") { f =>
import f._
val sender = TestProbe("sender")
val routes = Seq(ReceivingRoute(Seq(bob.nodeId, carol.nodeId), maxFinalExpiryDelta))
val invoice = createInvoice(carol, 50_000_000 msat, routes, sender)
// The amount is below what Carol expects.
sender.send(bob.paymentInitiator, SendPaymentToNode(45_000_000 msat, invoice, maxAttempts = 1, routeParams = bob.routeParams, blockUntilComplete = true))
val payment = sender.expectMsgType[PaymentFailed]
assert(payment.failures.head.isInstanceOf[LocalFailure])
val failure = payment.failures.head.asInstanceOf[LocalFailure]
assert(failure.t == PaymentLifecycle.UpdateMalformedException)
}
}

View file

@ -632,8 +632,9 @@ class Bolt11InvoiceSpec extends AnyFunSuite {
}
test("no unknown feature in invoice") {
assert(TestConstants.Alice.nodeParams.features.invoiceFeatures().unknown.nonEmpty)
val invoice = Bolt11Invoice(Block.LivenetGenesisBlock.hash, Some(123 msat), ByteVector32.One, priv, Left("Some invoice"), CltvExpiryDelta(18), features = TestConstants.Alice.nodeParams.features.invoiceFeatures())
val invoiceFeatures = TestConstants.Alice.nodeParams.features.invoiceFeatures().remove(RouteBlinding)
assert(invoiceFeatures.unknown.nonEmpty)
val invoice = Bolt11Invoice(Block.LivenetGenesisBlock.hash, Some(123 msat), ByteVector32.One, priv, Left("Some invoice"), CltvExpiryDelta(18), features = invoiceFeatures)
assert(invoice.features == Features(PaymentSecret -> Mandatory, BasicMultiPartPayment -> Optional, PaymentMetadata -> Optional, VariableLengthOnion -> Mandatory))
assert(Bolt11Invoice.fromString(invoice.toString).get == invoice)
}

View file

@ -22,7 +22,7 @@ import akka.testkit.{TestActorRef, TestProbe}
import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
import fr.acinq.bitcoin.scalacompat.{Block, ByteVector32, Crypto}
import fr.acinq.eclair.FeatureSupport.{Mandatory, Optional}
import fr.acinq.eclair.Features._
import fr.acinq.eclair.Features.{KeySend, _}
import fr.acinq.eclair.TestConstants.Alice
import fr.acinq.eclair.channel.{CMD_FAIL_HTLC, CMD_FULFILL_HTLC, Register}
import fr.acinq.eclair.db.{IncomingBlindedPayment, IncomingPaymentStatus}
@ -34,7 +34,7 @@ import fr.acinq.eclair.payment.receive.{MultiPartPaymentFSM, PaymentHandler}
import fr.acinq.eclair.router.Router
import fr.acinq.eclair.router.Router.RouteResponse
import fr.acinq.eclair.wire.protocol.OfferTypes.{InvoiceRequest, Offer, PaymentInfo}
import fr.acinq.eclair.wire.protocol.OnionPaymentPayloadTlv.{AmountToForward, BlindingPoint, EncryptedRecipientData, OutgoingCltv}
import fr.acinq.eclair.wire.protocol.OnionPaymentPayloadTlv._
import fr.acinq.eclair.wire.protocol.PaymentOnion.FinalPayload
import fr.acinq.eclair.wire.protocol.RouteBlindingEncryptedDataTlv.{PathId, PaymentConstraints}
import fr.acinq.eclair.wire.protocol._
@ -97,7 +97,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
def createBlindedPacket(amount: MilliSatoshi, paymentHash: ByteVector32, expiry: CltvExpiry, pathId: ByteVector, blinding_opt: Option[PublicKey]): IncomingPaymentPacket.FinalPacket = {
val add = UpdateAddHtlc(ByteVector32.One, 0, amount, paymentHash, expiry, TestConstants.emptyOnionPacket, blinding_opt)
val payload = FinalPayload.Blinded(TlvStream(AmountToForward(amount), OutgoingCltv(expiry), EncryptedRecipientData(hex"deadbeef")), TlvStream(PathId(pathId), PaymentConstraints(CltvExpiry(500_000), 1 msat)))
val payload = FinalPayload.Blinded(TlvStream(AmountToForward(amount), TotalAmount(amount), OutgoingCltv(expiry), EncryptedRecipientData(hex"deadbeef")), TlvStream(PathId(pathId), PaymentConstraints(CltvExpiry(500_000), 1 msat)))
IncomingPaymentPacket.FinalPacket(add, payload)
}
@ -543,7 +543,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
assert(pathIds.size == 1)
val add = UpdateAddHtlc(ByteVector32.One, 0, 5000 msat, invoice.paymentHash, defaultExpiry, TestConstants.emptyOnionPacket, None)
val payload = FinalPayload.Blinded(TlvStream(BlindingPoint(pathIds.keys.head), AmountToForward(5000 msat), OutgoingCltv(defaultExpiry), EncryptedRecipientData(hex"deadbeef")), TlvStream(PathId(pathIds.values.head), PaymentConstraints(CltvExpiry(500_000), 1 msat)))
val payload = FinalPayload.Blinded(TlvStream(BlindingPoint(pathIds.keys.head), AmountToForward(5000 msat), TotalAmount(5000 msat), OutgoingCltv(defaultExpiry), EncryptedRecipientData(hex"deadbeef")), TlvStream(PathId(pathIds.values.head), PaymentConstraints(CltvExpiry(500_000), 1 msat)))
val packet = IncomingPaymentPacket.FinalPacket(add, payload)
sender.send(handlerWithRouteBlinding, packet)
register.expectMsgType[Register.Forward[CMD_FULFILL_HTLC]].message

View file

@ -75,7 +75,6 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
VariableLengthOnion -> Mandatory,
PaymentSecret -> Mandatory,
BasicMultiPartPayment -> Optional,
RouteBlinding -> Optional,
TrampolinePaymentPrototype -> Optional,
)
@ -204,7 +203,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
test("forward single-part payment when multi-part deactivated", Tag(Tags.DisableMPP)) { f =>
import f._
val finalExpiryDelta = CltvExpiryDelta(24)
val invoice = Bolt11Invoice(Block.LivenetGenesisBlock.hash, Some(finalAmount), paymentHash, priv_c.privateKey, Left("Some MPP invoice"), finalExpiryDelta, features = featuresWithMpp)
val invoice = Bolt11Invoice(Block.LivenetGenesisBlock.hash, Some(finalAmount), paymentHash, priv_c.privateKey, Left("Some MPP invoice"), finalExpiryDelta, features = featuresWithoutRouteBlinding)
val req = SendPaymentToNode(finalAmount, invoice, 1, routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams)
assert(req.finalExpiry(nodeParams) == (finalExpiryDelta + 1).toCltvExpiry(nodeParams.currentBlockHeight))
sender.send(initiator, req)
@ -228,7 +227,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
test("forward multi-part payment") { f =>
import f._
val invoice = Bolt11Invoice(Block.LivenetGenesisBlock.hash, Some(finalAmount), paymentHash, priv_c.privateKey, Left("Some invoice"), CltvExpiryDelta(18), features = featuresWithMpp)
val invoice = Bolt11Invoice(Block.LivenetGenesisBlock.hash, Some(finalAmount), paymentHash, priv_c.privateKey, Left("Some invoice"), CltvExpiryDelta(18), features = featuresWithoutRouteBlinding)
val req = SendPaymentToNode(finalAmount + 100.msat, invoice, 1, routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams)
sender.send(initiator, req)
val id = sender.expectMsgType[UUID]
@ -252,7 +251,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
test("forward multi-part payment with randomized final expiry", Tag(Tags.RandomizeFinalExpiry)) { f =>
import f._
val invoiceFinalExpiryDelta = CltvExpiryDelta(6)
val invoice = Bolt11Invoice(Block.LivenetGenesisBlock.hash, Some(finalAmount), paymentHash, priv_c.privateKey, Left("Some invoice"), invoiceFinalExpiryDelta, features = featuresWithMpp)
val invoice = Bolt11Invoice(Block.LivenetGenesisBlock.hash, Some(finalAmount), paymentHash, priv_c.privateKey, Left("Some invoice"), invoiceFinalExpiryDelta, features = featuresWithoutRouteBlinding)
val req = SendPaymentToNode(finalAmount, invoice, 1, routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams)
sender.send(initiator, req)
val id = sender.expectMsgType[UUID]
@ -265,7 +264,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
test("forward multi-part payment with pre-defined route") { f =>
import f._
val invoice = Bolt11Invoice(Block.LivenetGenesisBlock.hash, Some(finalAmount), paymentHash, priv_c.privateKey, Left("Some invoice"), CltvExpiryDelta(18), features = featuresWithMpp)
val invoice = Bolt11Invoice(Block.LivenetGenesisBlock.hash, Some(finalAmount), paymentHash, priv_c.privateKey, Left("Some invoice"), CltvExpiryDelta(18), features = featuresWithoutRouteBlinding)
val route = PredefinedChannelRoute(finalAmount / 2, c, Seq(channelUpdate_ab.shortChannelId, channelUpdate_bc.shortChannelId))
val req = SendPaymentToRoute(finalAmount, invoice, route, None, None, None)
sender.send(initiator, req)
@ -423,7 +422,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
import f._
// This is disabled because it would let the trampoline node steal the whole payment (if malicious).
val routingHints = List(List(Bolt11Invoice.ExtraHop(b, channelUpdate_bc.shortChannelId, 10 msat, 100, CltvExpiryDelta(144))))
val invoice = Bolt11Invoice(Block.RegtestGenesisBlock.hash, None, paymentHash, priv_a.privateKey, Left("#abittooreckless"), CltvExpiryDelta(18), None, None, routingHints, features = featuresWithMpp)
val invoice = Bolt11Invoice(Block.RegtestGenesisBlock.hash, None, paymentHash, priv_a.privateKey, Left("#abittooreckless"), CltvExpiryDelta(18), None, None, routingHints, features = featuresWithoutRouteBlinding)
val trampolineFees = 21_000 msat
val req = SendTrampolinePayment(finalAmount, invoice, b, Seq((trampolineFees, CltvExpiryDelta(12))), routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams)
sender.send(initiator, req)

View file

@ -641,13 +641,13 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
// e returns a failure
val failure = IncorrectOrUnknownPaymentDetails(finalAmount, BlockHeight(currentBlockCount))
val Right(fail_e) = buildHtlcFailure(priv_e.privateKey, CMD_FAIL_HTLC(add_e.id, Right(failure)), add_e)
val Right(fail_e: UpdateFailHtlc) = buildHtlcFailure(priv_e.privateKey, CMD_FAIL_HTLC(add_e.id, Right(failure)), add_e)
assert(fail_e.id == add_e.id)
val Right(fail_d) = buildHtlcFailure(priv_d.privateKey, CMD_FAIL_HTLC(add_d.id, Left(fail_e.reason)), add_d)
val Right(fail_d: UpdateFailHtlc) = buildHtlcFailure(priv_d.privateKey, CMD_FAIL_HTLC(add_d.id, Left(fail_e.reason)), add_d)
assert(fail_d.id == add_d.id)
val Right(fail_c) = buildHtlcFailure(priv_c.privateKey, CMD_FAIL_HTLC(add_c.id, Left(fail_d.reason)), add_c)
val Right(fail_c: UpdateFailHtlc) = buildHtlcFailure(priv_c.privateKey, CMD_FAIL_HTLC(add_c.id, Left(fail_d.reason)), add_c)
assert(fail_c.id == add_c.id)
val Right(fail_b) = buildHtlcFailure(priv_b.privateKey, CMD_FAIL_HTLC(add_b.id, Left(fail_c.reason)), add_b)
val Right(fail_b: UpdateFailHtlc) = buildHtlcFailure(priv_b.privateKey, CMD_FAIL_HTLC(add_b.id, Left(fail_c.reason)), add_b)
assert(fail_b.id == add_b.id)
val Success(Sphinx.DecryptedFailurePacket(failingNode, decryptedFailure)) = Sphinx.FailurePacket.decrypt(fail_b.reason, payment.sharedSecrets)
assert(failingNode == e)
@ -661,14 +661,29 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
val add_b = UpdateAddHtlc(randomBytes32(), 0, payment.cmd.amount, payment.cmd.paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, payment.cmd.nextBlindingKey_opt)
val Right(ChannelRelayPacket(_, _, packet_c)) = decrypt(add_b, priv_b.privateKey, Features.empty)
val add_c = UpdateAddHtlc(randomBytes32(), 1, amount_bc, paymentHash, expiry_bc, packet_c, None)
val Right(_: ChannelRelayPacket) = decrypt(add_c, priv_c.privateKey, Features(RouteBlinding -> Optional))
val Right(ChannelRelayPacket(_, payload_c, packet_d)) = decrypt(add_c, priv_c.privateKey, Features(RouteBlinding -> Optional))
val blinding_d = payload_c.asInstanceOf[IntermediatePayload.ChannelRelay.Blinded].nextBlinding
val add_d = UpdateAddHtlc(randomBytes32(), 2, amount_cd, paymentHash, expiry_cd, packet_d, Some(blinding_d))
val Right(ChannelRelayPacket(_, payload_d, packet_e)) = decrypt(add_d, priv_d.privateKey, Features(RouteBlinding -> Optional))
val blinding_e = payload_d.asInstanceOf[IntermediatePayload.ChannelRelay.Blinded].nextBlinding
val add_e = UpdateAddHtlc(randomBytes32(), 3, amount_de, paymentHash, expiry_de, packet_e, Some(blinding_e))
val Right(FinalPacket(_, payload_e)) = decrypt(add_e, priv_e.privateKey, Features(RouteBlinding -> Optional))
assert(payload_e.isInstanceOf[FinalPayload.Blinded])
// only the introduction node is allowed to send an `update_fail_htlc` message: downstream nodes must send
// `update_fail_malformed_htlc` which doesn't use onion encryption
// nodes after the introduction node cannot send `update_fail_htlc` messages
val Right(fail_e: UpdateFailMalformedHtlc) = buildHtlcFailure(priv_e.privateKey, CMD_FAIL_HTLC(add_e.id, Right(TemporaryNodeFailure)), add_e)
assert(fail_e.id == add_e.id)
assert(fail_e.onionHash == Sphinx.hash(add_e.onionRoutingPacket))
assert(fail_e.failureCode == InvalidOnionBlinding(fail_e.onionHash).code)
val Right(fail_d: UpdateFailMalformedHtlc) = buildHtlcFailure(priv_d.privateKey, CMD_FAIL_HTLC(add_d.id, Right(UnknownNextPeer)), add_d)
assert(fail_d.id == add_d.id)
assert(fail_d.onionHash == Sphinx.hash(add_d.onionRoutingPacket))
assert(fail_d.failureCode == InvalidOnionBlinding(fail_d.onionHash).code)
// only the introduction node is allowed to send an `update_fail_htlc` message
val failure = InvalidOnionBlinding(Sphinx.hash(add_c.onionRoutingPacket))
val Right(fail_c) = buildHtlcFailure(priv_c.privateKey, CMD_FAIL_HTLC(add_c.id, Right(failure)), add_c)
val Right(fail_c: UpdateFailHtlc) = buildHtlcFailure(priv_c.privateKey, CMD_FAIL_HTLC(add_c.id, Right(failure)), add_c)
assert(fail_c.id == add_c.id)
val Right(fail_b) = buildHtlcFailure(priv_b.privateKey, CMD_FAIL_HTLC(add_b.id, Left(fail_c.reason)), add_b)
val Right(fail_b: UpdateFailHtlc) = buildHtlcFailure(priv_b.privateKey, CMD_FAIL_HTLC(add_b.id, Left(fail_c.reason)), add_b)
assert(fail_b.id == add_b.id)
val Success(Sphinx.DecryptedFailurePacket(failingNode, decryptedFailure)) = Sphinx.FailurePacket.decrypt(fail_b.reason, payment.sharedSecrets)
assert(failingNode == c)

View file

@ -28,6 +28,7 @@ import fr.acinq.eclair.Features.ScidAlias
import fr.acinq.eclair.TestConstants.emptyOnionPacket
import fr.acinq.eclair.blockchain.fee.FeeratePerKw
import fr.acinq.eclair.channel._
import fr.acinq.eclair.crypto.Sphinx
import fr.acinq.eclair.payment.IncomingPaymentPacket.ChannelRelayPacket
import fr.acinq.eclair.payment.relay.ChannelRelayer._
import fr.acinq.eclair.payment.{ChannelPaymentRelayed, IncomingPaymentPacket, PaymentPacketSpec}
@ -153,12 +154,12 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a
expectFwdAdd(register, lcu2.channelId, outgoingAmount, outgoingExpiry)
}
test("relay with onion tlv payload") { f =>
test("relay blinded payment") { f =>
import f._
val payload = ChannelRelay.Standard(realScid1, outgoingAmount, outgoingExpiry)
val r = createValidIncomingPacket(payload)
val u = createLocalUpdate(channelId1)
val u = createLocalUpdate(channelId1, feeBaseMsat = 2500 msat, feeProportionalMillionths = 0)
val payload = createBlindedPayload(u.channelUpdate, isIntroduction = false)
val r = createValidIncomingPacket(payload, outgoingAmount + u.channelUpdate.feeBaseMsat, outgoingExpiry + u.channelUpdate.cltvExpiryDelta)
channelRelayer ! WrappedLocalChannelUpdate(u)
channelRelayer ! Relay(r)
@ -183,9 +184,9 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a
channelRelayer ! Relay(r)
// first try
val fwd1 = expectFwdAdd(register, channelIds(realScId2), outgoingAmount, outgoingExpiry)
val fwd1 = expectFwdAdd(register, channelIds(realScid2), outgoingAmount, outgoingExpiry)
// channel returns an error
fwd1.message.replyTo ! RES_ADD_FAILED(fwd1.message, HtlcValueTooHighInFlight(channelIds(realScId2), 1000000000 msat, 1516977616 msat), Some(u2.channelUpdate))
fwd1.message.replyTo ! RES_ADD_FAILED(fwd1.message, HtlcValueTooHighInFlight(channelIds(realScid2), 1000000000 msat, 1516977616 msat), Some(u2.channelUpdate))
// second try
val fwd2 = expectFwdAdd(register, channelIds(realScid1), outgoingAmount, outgoingExpiry)
@ -264,6 +265,35 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a
expectFwdFail(register, r.add.channelId, CMD_FAIL_HTLC(r.add.id, Right(AmountBelowMinimum(outgoingAmount, u.channelUpdate)), commit = true))
}
test("fail to relay blinded payment") { f =>
import f._
Seq(true, false).foreach { isIntroduction =>
// The outgoing channel is disabled, so we won't be able to relay the payment.
val u = createLocalUpdate(channelId1, feeBaseMsat = 5000 msat, feeProportionalMillionths = 0, enabled = false)
val r = createValidIncomingPacket(createBlindedPayload(u.channelUpdate, isIntroduction), outgoingAmount + u.channelUpdate.feeBaseMsat, outgoingExpiry + u.channelUpdate.cltvExpiryDelta)
channelRelayer ! WrappedLocalChannelUpdate(u)
channelRelayer ! Relay(r)
val cmd = register.expectMessageType[Register.Forward[channel.Command]]
assert(cmd.channelId == r.add.channelId)
if (isIntroduction) {
assert(cmd.message.isInstanceOf[CMD_FAIL_HTLC])
val fail = cmd.message.asInstanceOf[CMD_FAIL_HTLC]
assert(fail.id == r.add.id)
assert(fail.reason == Right(InvalidOnionBlinding(Sphinx.hash(r.add.onionRoutingPacket))))
assert(fail.delay_opt.nonEmpty)
} else {
assert(cmd.message.isInstanceOf[CMD_FAIL_MALFORMED_HTLC])
val fail = cmd.message.asInstanceOf[CMD_FAIL_MALFORMED_HTLC]
assert(fail.id == r.add.id)
assert(fail.onionHash == Sphinx.hash(r.add.onionRoutingPacket))
assert(fail.failureCode == InvalidOnionBlinding(Sphinx.hash(r.add.onionRoutingPacket)).code)
}
}
}
test("relay when expiry larger than our requirements") { f =>
import f._
@ -448,10 +478,9 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a
test("settlement failure") { f =>
import f._
val channelId1 = channelIds(realScid1)
val u = createLocalUpdate(channelId1, feeBaseMsat = 5000 msat, feeProportionalMillionths = 0)
val payload = ChannelRelay.Standard(realScid1, outgoingAmount, outgoingExpiry)
val r = createValidIncomingPacket(payload)
val u = createLocalUpdate(channelId1)
val r = createValidIncomingPacket(payload, outgoingAmount + u.channelUpdate.feeBaseMsat, outgoingExpiry + u.channelUpdate.cltvExpiryDelta)
val u_disabled = createLocalUpdate(channelId1, enabled = false)
val downstream_htlc = UpdateAddHtlc(channelId1, 7, outgoingAmount, paymentHash, outgoingExpiry, emptyOnionPacket, None)
@ -459,7 +488,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a
val testCases = Seq(
TestCase(HtlcResult.RemoteFail(UpdateFailHtlc(channelId1, downstream_htlc.id, hex"deadbeef")), CMD_FAIL_HTLC(r.add.id, Left(hex"deadbeef"), commit = true)),
TestCase(HtlcResult.RemoteFailMalformed(UpdateFailMalformedHtlc(channelId1, downstream_htlc.id, ByteVector32.One, FailureMessageCodecs.BADONION)), CMD_FAIL_MALFORMED_HTLC(r.add.id, ByteVector32.One, FailureMessageCodecs.BADONION, commit = true)),
TestCase(HtlcResult.RemoteFailMalformed(UpdateFailMalformedHtlc(channelId1, downstream_htlc.id, ByteVector32.One, FailureMessageCodecs.BADONION | FailureMessageCodecs.PERM | 5)), CMD_FAIL_HTLC(r.add.id, Right(InvalidOnionHmac(ByteVector32.One)), commit = true)),
TestCase(HtlcResult.OnChainFail(HtlcOverriddenByLocalCommit(channelId1, downstream_htlc)), CMD_FAIL_HTLC(r.add.id, Right(PermanentChannelFailure), commit = true)),
TestCase(HtlcResult.DisconnectedBeforeSigned(u_disabled.channelUpdate), CMD_FAIL_HTLC(r.add.id, Right(TemporaryChannelFailure(u_disabled.channelUpdate)), commit = true)),
TestCase(HtlcResult.ChannelFailureBeforeSigned, CMD_FAIL_HTLC(r.add.id, Right(PermanentChannelFailure), commit = true))
@ -475,6 +504,47 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a
}
}
test("settlement failure (blinded payment)") { f =>
import f._
val u = createLocalUpdate(channelId1, feeBaseMsat = 5000 msat, feeProportionalMillionths = 0)
val downstream = UpdateAddHtlc(channelId1, 7, outgoingAmount, paymentHash, outgoingExpiry, emptyOnionPacket, None)
val testCases = Seq(
HtlcResult.RemoteFail(UpdateFailHtlc(channelId1, downstream.id, hex"deadbeef")),
HtlcResult.RemoteFailMalformed(UpdateFailMalformedHtlc(channelId1, downstream.id, randomBytes32(), FailureMessageCodecs.BADONION | FailureMessageCodecs.PERM | 5)),
HtlcResult.OnChainFail(HtlcOverriddenByLocalCommit(channelId1, downstream)),
HtlcResult.DisconnectedBeforeSigned(createLocalUpdate(channelId1, enabled = false).channelUpdate),
HtlcResult.ChannelFailureBeforeSigned,
)
Seq(true, false).foreach { isIntroduction =>
testCases.foreach { htlcResult =>
val r = createValidIncomingPacket(createBlindedPayload(u.channelUpdate, isIntroduction), outgoingAmount + u.channelUpdate.feeBaseMsat, outgoingExpiry + u.channelUpdate.cltvExpiryDelta)
channelRelayer ! WrappedLocalChannelUpdate(u)
channelRelayer ! Relay(r)
val fwd = expectFwdAdd(register, channelId1, outgoingAmount, outgoingExpiry)
fwd.message.replyTo ! RES_SUCCESS(fwd.message, channelId1)
fwd.message.origin.replyTo ! RES_ADD_SETTLED(fwd.message.origin, downstream, htlcResult)
val cmd = register.expectMessageType[Register.Forward[channel.Command]]
assert(cmd.channelId == r.add.channelId)
if (isIntroduction) {
assert(cmd.message.isInstanceOf[CMD_FAIL_HTLC])
val fail = cmd.message.asInstanceOf[CMD_FAIL_HTLC]
assert(fail.id == r.add.id)
assert(fail.reason == Right(InvalidOnionBlinding(Sphinx.hash(r.add.onionRoutingPacket))))
assert(fail.delay_opt.nonEmpty)
} else {
assert(cmd.message.isInstanceOf[CMD_FAIL_MALFORMED_HTLC])
val fail = cmd.message.asInstanceOf[CMD_FAIL_MALFORMED_HTLC]
assert(fail.id == r.add.id)
assert(fail.onionHash == Sphinx.hash(r.add.onionRoutingPacket))
assert(fail.failureCode == InvalidOnionBlinding(Sphinx.hash(r.add.onionRoutingPacket)).code)
}
}
}
}
test("settlement success") { f =>
import f._
val eventListener = TestProbe[ChannelPaymentRelayed]()
@ -569,7 +639,7 @@ object ChannelRelayerSpec {
val outgoingNodeId: PublicKey = randomKey().publicKey
val realScid1: RealShortChannelId = RealShortChannelId(111111)
val realScId2: RealShortChannelId = RealShortChannelId(222222)
val realScid2: RealShortChannelId = RealShortChannelId(222222)
val localAlias1: Alias = Alias(111000)
val localAlias2: Alias = Alias(222000)
@ -579,13 +649,30 @@ object ChannelRelayerSpec {
val channelIds = Map(
realScid1 -> channelId1,
realScId2 -> channelId2,
realScid2 -> channelId2,
localAlias1 -> channelId1,
localAlias2 -> channelId2,
)
def createBlindedPayload(update: ChannelUpdate, isIntroduction: Boolean): ChannelRelay.Blinded = {
val tlvs = TlvStream[OnionPaymentPayloadTlv](Seq(
Some(OnionPaymentPayloadTlv.EncryptedRecipientData(hex"2a")),
if (isIntroduction) Some(OnionPaymentPayloadTlv.BlindingPoint(randomKey().publicKey)) else None,
).flatten)
val blindedTlvs = TlvStream[RouteBlindingEncryptedDataTlv](
RouteBlindingEncryptedDataTlv.OutgoingChannelId(update.shortChannelId),
RouteBlindingEncryptedDataTlv.PaymentRelay(update.cltvExpiryDelta, update.feeProportionalMillionths, update.feeBaseMsat),
RouteBlindingEncryptedDataTlv.PaymentConstraints(CltvExpiry(500_000), 0 msat),
)
ChannelRelay.Blinded(tlvs, blindedTlvs, randomKey().publicKey)
}
def createValidIncomingPacket(payload: IntermediatePayload.ChannelRelay, amountIn: MilliSatoshi = 11_000_000 msat, expiryIn: CltvExpiry = CltvExpiry(400_100)): IncomingPaymentPacket.ChannelRelayPacket = {
val add_ab = UpdateAddHtlc(channelId = randomBytes32(), id = 123456, amountIn, paymentHash, expiryIn, emptyOnionPacket, None)
val nextBlinding_opt = payload match {
case p: ChannelRelay.Blinded => Some(p.nextBlinding)
case _: ChannelRelay.Standard => None
}
val add_ab = UpdateAddHtlc(channelId = randomBytes32(), id = 123456, amountIn, paymentHash, expiryIn, emptyOnionPacket, nextBlinding_opt)
ChannelRelayPacket(add_ab, payload, emptyOnionPacket)
}

View file

@ -33,7 +33,7 @@ import fr.acinq.eclair.payment.OutgoingPaymentPacket.{NodePayload, Upstream, bui
import fr.acinq.eclair.payment.PaymentPacketSpec._
import fr.acinq.eclair.payment.relay.Relayer._
import fr.acinq.eclair.payment.send.{ClearRecipient, ClearTrampolineRecipient}
import fr.acinq.eclair.router.BaseRouterSpec.channelHopFromUpdate
import fr.acinq.eclair.router.BaseRouterSpec.{blindedRouteFromHops, channelHopFromUpdate}
import fr.acinq.eclair.router.Router.{NodeHop, Route}
import fr.acinq.eclair.wire.protocol.PaymentOnion.FinalPayload
import fr.acinq.eclair.wire.protocol._
@ -41,6 +41,7 @@ import fr.acinq.eclair.{NodeParams, TestConstants, randomBytes32, _}
import org.scalatest.concurrent.PatienceConfiguration
import org.scalatest.funsuite.FixtureAnyFunSuiteLike
import org.scalatest.{Outcome, Tag}
import scodec.bits.HexStringSyntax
import java.util.UUID
import scala.concurrent.duration.DurationInt
@ -153,6 +154,42 @@ class RelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat
register.expectNoMessage(50 millis)
}
test("fail to relay an htlc-add with invalid blinding data (introduction node)") { f =>
import f._
// we use an expired blinded route.
val routeExpiry = CltvExpiry(nodeParams.currentBlockHeight - 10)
val (_, blindedHop, recipient) = blindedRouteFromHops(finalAmount, finalExpiry, Seq(channelHopFromUpdate(b, c, channelUpdate_bc)), routeExpiry, paymentPreimage, hex"deadbeef")
val route = Route(finalAmount, Seq(channelHopFromUpdate(priv_a.publicKey, b, channelUpdate_ab)), Some(blindedHop))
val Right(payment) = buildOutgoingPayment(ActorRef.noSender, priv_a.privateKey, Upstream.Local(UUID.randomUUID()), paymentHash, route, recipient)
val add_ab = UpdateAddHtlc(channelId_ab, 0, payment.cmd.amount, payment.cmd.paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, payment.cmd.nextBlindingKey_opt)
relayer ! RelayForward(add_ab)
val fail = register.expectMessageType[Register.Forward[CMD_FAIL_HTLC]].message
assert(fail.id == add_ab.id)
assert(fail.reason == Right(InvalidOnionBlinding(Sphinx.hash(add_ab.onionRoutingPacket))))
assert(fail.delay_opt.nonEmpty)
register.expectNoMessage(50 millis)
}
test("fail to relay an htlc-add with invalid blinding data (intermediate node)") { f =>
import f._
// we use an expired blinded route.
val (route, recipient) = singleBlindedHop(routeExpiry = CltvExpiry(nodeParams.currentBlockHeight - 1))
val Right(payment) = buildOutgoingPayment(ActorRef.noSender, priv_a.privateKey, Upstream.Local(UUID.randomUUID()), paymentHash, route, recipient)
val add_ab = UpdateAddHtlc(channelId_ab, 0, payment.cmd.amount, payment.cmd.paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, payment.cmd.nextBlindingKey_opt)
relayer ! RelayForward(add_ab)
val fail = register.expectMessageType[Register.Forward[CMD_FAIL_MALFORMED_HTLC]].message
assert(fail.id == add_ab.id)
assert(fail.onionHash == Sphinx.hash(add_ab.onionRoutingPacket))
assert(fail.failureCode == (FailureMessageCodecs.BADONION | FailureMessageCodecs.PERM | 24))
register.expectNoMessage(50 millis)
}
test("fail to relay a trampoline htlc-add when trampoline is disabled", Tag("trampoline-disabled")) { f =>
import f._

View file

@ -64,7 +64,7 @@ class CommandCodecsSpec extends AnyFunSuite {
("reason" | either(bool, varsizebinarydata, failureMessageCodec)) ::
("commit" | provide(false))
assert(CommandCodecs.cmdFailCodec.decode(legacyCmdFailCodec.encode(42 :: Left(data123) :: true :: HNil).require).require ==
DecodeResult(CMD_FAIL_HTLC(42, Left(data123), commit = false, None), BitVector.empty))
DecodeResult(CMD_FAIL_HTLC(42, Left(data123), None, commit = false, None), BitVector.empty))
val legacyCmdFailMalformedCodec =
("id" | int64) ::
@ -72,7 +72,7 @@ class CommandCodecsSpec extends AnyFunSuite {
("failureCode" | uint16) ::
("commit" | provide(false))
assert(CommandCodecs.cmdFailMalformedCodec.decode(legacyCmdFailMalformedCodec.encode(42 :: data32 :: 456 :: true :: HNil).require).require ==
DecodeResult(CMD_FAIL_MALFORMED_HTLC(42, data32, 456, None, commit = false, None), BitVector.empty))
DecodeResult(CMD_FAIL_MALFORMED_HTLC(42, data32, 456, commit = false, None), BitVector.empty))
}
}

View file

@ -201,8 +201,7 @@ class PaymentOnionSpec extends AnyFunSuite {
RouteBlindingEncryptedDataTlv.PaymentConstraints(CltvExpiry(1500), 1 msat),
)
val testCases = Map(
TlvStream[OnionPaymentPayloadTlv](AmountToForward(561 msat), OutgoingCltv(CltvExpiry(42)), EncryptedRecipientData(hex"deadbeef")) -> hex"0d 02020231 04012a 0a04deadbeef",
TlvStream[OnionPaymentPayloadTlv](AmountToForward(561 msat), OutgoingCltv(CltvExpiry(42)), EncryptedRecipientData(hex"deadbeef"), BlindingPoint(PublicKey(hex"036d6caac248af96f6afa7f904f550253a0f3ef3f5aa2fe6838a95b216691468e2"))) -> hex"30 02020231 04012a 0a04deadbeef 0c21036d6caac248af96f6afa7f904f550253a0f3ef3f5aa2fe6838a95b216691468e2",
TlvStream[OnionPaymentPayloadTlv](AmountToForward(561 msat), OutgoingCltv(CltvExpiry(42)), EncryptedRecipientData(hex"deadbeef"), TotalAmount(1105 msat)) -> hex"11 02020231 04012a 0a04deadbeef 12020451",
TlvStream[OnionPaymentPayloadTlv](AmountToForward(561 msat), OutgoingCltv(CltvExpiry(42)), EncryptedRecipientData(hex"deadbeef"), BlindingPoint(PublicKey(hex"036d6caac248af96f6afa7f904f550253a0f3ef3f5aa2fe6838a95b216691468e2")), TotalAmount(1105 msat)) -> hex"34 02020231 04012a 0a04deadbeef 0c21036d6caac248af96f6afa7f904f550253a0f3ef3f5aa2fe6838a95b216691468e2 12020451",
)
@ -211,6 +210,7 @@ class PaymentOnionSpec extends AnyFunSuite {
assert(decoded == expected)
val Right(payload) = FinalPayload.Blinded.validate(decoded, blindedTlvs)
assert(payload.amount == 561.msat)
assert(payload.totalAmount == 1105.msat)
assert(payload.expiry == CltvExpiry(42))
assert(payload.pathId == hex"2a2a2a2a")
val encoded = perHopPayloadCodec.encode(expected).require.bytes
@ -312,13 +312,14 @@ class PaymentOnionSpec extends AnyFunSuite {
RouteBlindingEncryptedDataTlv.PaymentConstraints(CltvExpiry(1500), 1 msat),
)
val testCases = Seq(
(MissingRequiredTlv(UInt64(2)), hex"0d 04012a 0a080123456789abcdef"), // missing amount
(MissingRequiredTlv(UInt64(4)), hex"0e 02020231 0a080123456789abcdef"), // missing expiry
(MissingRequiredTlv(UInt64(10)), hex"07 02020231 04012a"), // missing encrypted data
(ForbiddenTlv(UInt64(0)), hex"1b 02020231 04012a 06080000000000000451 0a080123456789abcdef"), // forbidden outgoing_channel_id
(ForbiddenTlv(UInt64(0)), hex"35 02020231 04012a 0822eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f2836866190451 0a080123456789abcdef"), // forbidden payment_data
(ForbiddenTlv(UInt64(0)), hex"17 02020231 04012a 0a080123456789abcdef 1004deadbeef"), // forbidden payment_metadata
(ForbiddenTlv(UInt64(65535)), hex"17 02020231 04012a 0a080123456789abcdef fdffff0206c1"), // forbidden unknown tlv
(MissingRequiredTlv(UInt64(2)), hex"11 04012a 0a080123456789abcdef 12020451"), // missing amount
(MissingRequiredTlv(UInt64(4)), hex"12 02020231 0a080123456789abcdef 12020451"), // missing expiry
(MissingRequiredTlv(UInt64(10)), hex"0b 02020231 04012a 12020451"), // missing encrypted data
(MissingRequiredTlv(UInt64(18)), hex"11 02020231 04012a 0a080123456789abcdef"), // missing total amount
(ForbiddenTlv(UInt64(0)), hex"1f 02020231 04012a 06080000000000000451 0a080123456789abcdef 12020451"), // forbidden outgoing_channel_id
(ForbiddenTlv(UInt64(0)), hex"39 02020231 04012a 0822eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f2836866190451 0a080123456789abcdef 12020451"), // forbidden payment_data
(ForbiddenTlv(UInt64(0)), hex"1b 02020231 04012a 0a080123456789abcdef 1004deadbeef 12020451"), // forbidden payment_metadata
(ForbiddenTlv(UInt64(65535)), hex"1b 02020231 04012a 0a080123456789abcdef 12020451 fdffff0206c1"), // forbidden unknown tlv
)
for ((expectedErr, bin) <- testCases) {