Skip to content

Commit

Permalink
Fix blinded route failure messages (#2490)
Browse files Browse the repository at this point in the history
* 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.
  • Loading branch information
t-bast authored Dec 16, 2022
1 parent 83edd8c commit b5a2d3a
Show file tree
Hide file tree
Showing 28 changed files with 722 additions and 120 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -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

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -382,23 +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 {
case Right((commitments1, fail)) =>
if (c.commit) self ! CMD_SIGN()
context.system.eventStream.publish(AvailableBalanceChanged(self, d.channelId, d.shortIds, commitments1))
handleCommandSuccess(c, d.copy(commitments = commitments1)) sending fail
case Left(cause) =>
// 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)
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.sendFailMalformed(d.commitments, c) match {
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))
Expand All @@ -409,6 +398,17 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val
}
}

case Event(c: CMD_FAIL_MALFORMED_HTLC, d: DATA_NORMAL) =>
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))
handleCommandSuccess(c, d.copy(commitments = commitments1)) sending fail
case Left(cause) =>
// 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 {
case Right((commitments1, _, _)) => stay() using d.copy(commitments = commitments1)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand All @@ -315,8 +315,15 @@ object OutgoingPaymentPacket {
}
}

def buildHtlcFailure(nodeSecret: PrivateKey, cmd: CMD_FAIL_HTLC, add: UpdateAddHtlc): Either[CannotExtractSharedSecret, UpdateFailHtlc] = {
buildHtlcFailure(nodeSecret, cmd.reason, add).map(encryptedReason => UpdateFailHtlc(add.channelId, cmd.id, encryptedReason))
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))
}
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -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}
Expand Down Expand Up @@ -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)
Expand Down Expand Up @@ -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
}

Expand Down Expand Up @@ -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)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
// 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 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.
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")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,15 +36,15 @@ 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]

val cmdFailMalformedCodec: Codec[CMD_FAIL_MALFORMED_HTLC] =
(("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]

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down Expand Up @@ -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,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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 {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -95,6 +95,7 @@ object TestConstants {
PaymentSecret -> Mandatory,
BasicMultiPartPayment -> Optional,
PaymentMetadata -> Optional,
RouteBlinding -> Optional,
),
unknown = Set(UnknownFeature(TestFeature.optional))
),
Expand Down Expand Up @@ -245,6 +246,7 @@ object TestConstants {
PaymentSecret -> Mandatory,
BasicMultiPartPayment -> Optional,
PaymentMetadata -> Optional,
RouteBlinding -> Optional,
),
pluginParams = Nil,
overrideInitFeatures = Map.empty,
Expand Down
Loading

0 comments on commit b5a2d3a

Please sign in to comment.