Skip to content

Commit

Permalink
Add on_the_fly_funding feature bit and messages
Browse files Browse the repository at this point in the history
Add the (disabled by default) `on_the_fly_funding` feature bit and
codecs for the corresponding messages:

- `will_add_htlc`
- `will_fail_htlc`
- `will_fail_malformed_htlc`
- `cancel_on_the_fly_funding`

We also add a TLV to `update_add_htlc` to notify the recipient that we
relayed less data than what the onion encodes, in exchange for the fees
of the specified funding transaction.
  • Loading branch information
t-bast committed Jun 13, 2024
1 parent 9271c1e commit 41700ab
Show file tree
Hide file tree
Showing 9 changed files with 274 additions and 52 deletions.
1 change: 1 addition & 0 deletions eclair-core/src/main/resources/reference.conf
Original file line number Diff line number Diff line change
Expand Up @@ -83,6 +83,7 @@ eclair {
keysend = disabled
trampoline_payment_prototype = disabled
async_payment_prototype = disabled
on_the_fly_funding = disabled
}
// The following section lets you customize features for specific nodes.
// The overrides will be applied on top of the default features settings.
Expand Down
12 changes: 11 additions & 1 deletion eclair-core/src/main/scala/fr/acinq/eclair/Features.scala
Original file line number Diff line number Diff line change
Expand Up @@ -323,6 +323,14 @@ object Features {
val mandatory = 154
}

// TODO:
// - add NodeFeature once stable
// - add link to bLIP
case object OnTheFlyFunding extends Feature with InitFeature {
val rfcName = "on_the_fly_funding"
val mandatory = 560
}

val knownFeatures: Set[Feature] = Set(
DataLossProtect,
InitialRoutingSync,
Expand All @@ -349,6 +357,7 @@ object Features {
TrampolinePaymentPrototype,
AsyncPaymentPrototype,
SplicePrototype,
OnTheFlyFunding
)

// Features may depend on other features, as specified in Bolt 9.
Expand All @@ -361,7 +370,8 @@ object Features {
RouteBlinding -> (VariableLengthOnion :: Nil),
TrampolinePaymentPrototype -> (PaymentSecret :: Nil),
KeySend -> (VariableLengthOnion :: Nil),
AsyncPaymentPrototype -> (TrampolinePaymentPrototype :: Nil)
AsyncPaymentPrototype -> (TrampolinePaymentPrototype :: Nil),
OnTheFlyFunding -> (SplicePrototype :: Nil)
)

case class FeatureException(message: String) extends IllegalArgumentException(message)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ import fr.acinq.eclair.channel.fund.InteractiveTxSigningSession.UnsignedLocalCom
import fr.acinq.eclair.crypto.keymanager.ChannelKeyManager
import fr.acinq.eclair.transactions.Transactions.{CommitTx, HtlcTx, InputInfo, TxOwner}
import fr.acinq.eclair.transactions.{CommitmentSpec, DirectedHtlc, Scripts, Transactions}
import fr.acinq.eclair.wire.protocol.LiquidityAds.PaymentDetails
import fr.acinq.eclair.wire.protocol._
import fr.acinq.eclair.{Logs, MilliSatoshi, MilliSatoshiLong, NodeParams, UInt64}
import scodec.bits.ByteVector
Expand Down Expand Up @@ -154,6 +155,16 @@ object InteractiveTxBuilder {
val minNextFeerate: FeeratePerKw = targetFeerate * 25 / 24
// BOLT 2: the initiator's serial IDs MUST use even values and the non-initiator odd values.
val serialIdParity: Int = if (isInitiator) 0 else 1

def liquidityFees(lease_opt: Option[LiquidityAds.Lease]): Satoshi = {
lease_opt.map(l => l.paymentDetails match {
// The initiator of the interactive-tx is the liquidity buyer (if liquidity ads is used).
case LiquidityAds.PaymentDetails.FromChannelBalance => if (isInitiator) l.fees.total else -l.fees.total
// Fees will be paid later, when relaying HTLCs.
case _: LiquidityAds.PaymentDetails.FromFutureHtlc => 0.sat
case _: LiquidityAds.PaymentDetails.FromFutureHtlcWithPreimage => 0.sat
}).getOrElse(0 sat)
}
}

// @formatter:off
Expand Down Expand Up @@ -357,10 +368,7 @@ object InteractiveTxBuilder {
Behaviors.withMdc(Logs.mdc(remoteNodeId_opt = Some(channelParams.remoteParams.nodeId), channelId_opt = Some(fundingParams.channelId))) {
Behaviors.receiveMessagePartial {
case Start(replyTo) =>
val liquidityFee = liquidityLease_opt.map(l => l.paymentDetails match {
// The initiator of the interactive-tx is the liquidity buyer (if liquidity ads is used).
case LiquidityAds.PaymentDetails.FromChannelBalance => if (fundingParams.isInitiator) l.fees.total else -l.fees.total
}).getOrElse(0 sat)
val liquidityFee = fundingParams.liquidityFees(liquidityLease_opt)
// Note that pending HTLCs are ignored: splices only affect the main outputs.
val nextLocalBalance = purpose.previousLocalBalance + fundingParams.localContribution - localPushAmount + remotePushAmount - liquidityFee
val nextRemoteBalance = purpose.previousRemoteBalance + fundingParams.remoteContribution - remotePushAmount + localPushAmount + liquidityFee
Expand Down Expand Up @@ -752,10 +760,7 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
private def signCommitTx(completeTx: SharedTransaction): Behavior[Command] = {
val fundingTx = completeTx.buildUnsignedTx()
val fundingOutputIndex = fundingTx.txOut.indexWhere(_.publicKeyScript == fundingPubkeyScript)
val liquidityFee = liquidityLease_opt.map(l => l.paymentDetails match {
// The initiator of the interactive-tx is the liquidity buyer (if liquidity ads is used).
case LiquidityAds.PaymentDetails.FromChannelBalance => if (fundingParams.isInitiator) l.fees.total else -l.fees.total
}).getOrElse(0 sat)
val liquidityFee = fundingParams.liquidityFees(liquidityLease_opt)
Funding.makeCommitTxs(keyManager, channelParams,
fundingAmount = fundingParams.fundingAmount,
toLocal = completeTx.sharedOutput.localAmount - localPushAmount + remotePushAmount - liquidityFee,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,15 @@ object UpdateAddHtlcTlv {

private val blindingPoint: Codec[BlindingPoint] = (("length" | constant(hex"21")) :: ("blinding" | publicKey)).as[BlindingPoint]

val addHtlcTlvCodec: Codec[TlvStream[UpdateAddHtlcTlv]] = tlvStream(discriminated[UpdateAddHtlcTlv].by(varint).typecase(UInt64(0), blindingPoint))
/** When on-the-fly funding is used, the liquidity fees may be taken from HTLCs relayed after funding. */
case class FundingFeeTlv(fee: LiquidityAds.FundingFee) extends UpdateAddHtlcTlv

private val fundingFee: Codec[FundingFeeTlv] = tlvField((("amount" | millisatoshi) :: ("txId" | txIdAsHash)).as[LiquidityAds.FundingFee])

val addHtlcTlvCodec: Codec[TlvStream[UpdateAddHtlcTlv]] = tlvStream(discriminated[UpdateAddHtlcTlv].by(varint)
.typecase(UInt64(0), blindingPoint)
.typecase(UInt64(41041), fundingFee)
)
}

sealed trait UpdateFulfillHtlcTlv extends Tlv
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -434,6 +434,31 @@ object LightningMessageCodecs {
("fundingFeerate" | feeratePerKw) ::
("commitmentFeerate" | feeratePerKw)).as[RecommendedFeerates]

val willAddHtlcCodec: Codec[WillAddHtlc] = (
("chainHash" | blockHash) ::
("id" | bytes32) ::
("amount" | millisatoshi) ::
("paymentHash" | bytes32) ::
("expiry" | cltvExpiry) ::
("onionRoutingPacket" | PaymentOnionCodecs.paymentOnionPacketCodec) ::
("tlvStream" | WillAddHtlcTlv.willAddHtlcTlvCodec)).as[WillAddHtlc]

val willFailHtlcCodec: Codec[WillFailHtlc] = (
("id" | bytes32) ::
("paymentHash" | bytes32) ::
("reason" | varsizebinarydata)).as[WillFailHtlc]

val willFailMalformedHtlcCodec: Codec[WillFailMalformedHtlc] = (
("id" | bytes32) ::
("paymentHash" | bytes32) ::
("onionHash" | bytes32) ::
("failureCode" | uint16)).as[WillFailMalformedHtlc]

val cancelOnTheFlyFundingCodec: Codec[CancelOnTheFlyFunding] = (
("channelId" | bytes32) ::
("paymentHashes" | listOfN(uint16, bytes32)) ::
("reason" | varsizebinarydata)).as[CancelOnTheFlyFunding]

val unknownMessageCodec: Codec[UnknownMessage] = (
("tag" | uint16) ::
("message" | bytes)
Expand Down Expand Up @@ -487,6 +512,11 @@ object LightningMessageCodecs {
//
.typecase(35025, recommendedFeeratesCodec)
//
.typecase(41041, willAddHtlcCodec)
.typecase(41042, willFailHtlcCodec)
.typecase(41043, willFailMalformedHtlcCodec)
.typecase(41044, cancelOnTheFlyFundingCodec)
//
.typecase(37000, spliceInitCodec)
.typecase(37002, spliceAckCodec)
.typecase(37004, spliceLockedCodec)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -363,6 +363,7 @@ case class UpdateAddHtlc(channelId: ByteVector32,
onionRoutingPacket: OnionRoutingPacket,
tlvStream: TlvStream[UpdateAddHtlcTlv]) extends HtlcMessage with UpdateMessage with HasChannelId {
val blinding_opt: Option[PublicKey] = tlvStream.get[UpdateAddHtlcTlv.BlindingPoint].map(_.publicKey)
val fundingFee_opt: Option[LiquidityAds.FundingFee] = tlvStream.get[UpdateAddHtlcTlv.FundingFeeTlv].map(_.fee)
}

object UpdateAddHtlc {
Expand All @@ -372,8 +373,12 @@ object UpdateAddHtlc {
paymentHash: ByteVector32,
cltvExpiry: CltvExpiry,
onionRoutingPacket: OnionRoutingPacket,
blinding_opt: Option[PublicKey]): UpdateAddHtlc = {
val tlvs = blinding_opt.map(UpdateAddHtlcTlv.BlindingPoint).toSet[UpdateAddHtlcTlv]
blinding_opt: Option[PublicKey],
fundingFee_opt: Option[LiquidityAds.FundingFee] = None): UpdateAddHtlc = {
val tlvs = Set(
blinding_opt.map(UpdateAddHtlcTlv.BlindingPoint),
fundingFee_opt.map(UpdateAddHtlcTlv.FundingFeeTlv),
).flatten[UpdateAddHtlcTlv]
UpdateAddHtlc(channelId, id, amountMsat, paymentHash, cltvExpiry, onionRoutingPacket, TlvStream(tlvs))
}
}
Expand Down Expand Up @@ -607,4 +612,51 @@ case class OnionMessage(blindingKey: PublicKey, onionRoutingPacket: OnionRouting
*/
case class RecommendedFeerates(chainHash: BlockHash, fundingFeerate: FeeratePerKw, commitmentFeerate: FeeratePerKw) extends SetupMessage with HasChainHash

/**
* This message is sent when an HTLC couldn't be relayed to our node because we don't have enough inbound liquidity.
* This allows us to treat it as an incoming payment, and request on-the-fly liquidity accordingly if we wish to receive that payment.
* If we accept the payment, we will send an [[OpenDualFundedChannel]] or [[SpliceInit]] message containing [[ChannelTlv.RequestFundingTlv]].
* Our peer will then provide the requested funding liquidity and will relay the corresponding HTLC(s) afterwards.
*/
case class WillAddHtlc(chainHash: BlockHash,
id: ByteVector32,
amount: MilliSatoshi,
paymentHash: ByteVector32,
expiry: CltvExpiry,
finalPacket: OnionRoutingPacket,
tlvStream: TlvStream[WillAddHtlcTlv] = TlvStream.empty) extends LightningMessage {
val blinding_opt: Option[PublicKey] = tlvStream.get[WillAddHtlcTlv.BlindingPoint].map(_.publicKey)
}

object WillAddHtlc {
def apply(chainHash: BlockHash,
id: ByteVector32,
amount: MilliSatoshi,
paymentHash: ByteVector32,
expiry: CltvExpiry,
finalPacket: OnionRoutingPacket,
blinding_opt: Option[PublicKey]): WillAddHtlc = {
val tlvs = blinding_opt.map(WillAddHtlcTlv.BlindingPoint).toSet[WillAddHtlcTlv]
WillAddHtlc(chainHash, id, amount, paymentHash, expiry, finalPacket, TlvStream(tlvs))
}
}

/** This message is similar to [[UpdateFailHtlc]], but for [[WillAddHtlc]]. */
case class WillFailHtlc(id: ByteVector32, paymentHash: ByteVector32, reason: ByteVector) extends LightningMessage

/** This message is similar to [[UpdateFailMalformedHtlc]], but for [[WillAddHtlc]]. */
case class WillFailMalformedHtlc(id: ByteVector32, paymentHash: ByteVector32, onionHash: ByteVector32, failureCode: Int) extends LightningMessage

/**
* This message is sent in response to an [[OpenDualFundedChannel]] or [[SpliceInit]] message containing an invalid [[LiquidityAds.RequestFunds]].
* The receiver must consider the funding attempt failed when receiving this message.
*/
case class CancelOnTheFlyFunding(channelId: ByteVector32, paymentHashes: List[ByteVector32], reason: ByteVector) extends LightningMessage with HasChannelId {
def toAscii: String = if (isAsciiPrintable(reason)) new String(reason.toArray, StandardCharsets.US_ASCII) else "n/a"
}

object CancelOnTheFlyFunding {
def apply(channelId: ByteVector32, paymentHashes: List[ByteVector32], reason: String): CancelOnTheFlyFunding = CancelOnTheFlyFunding(channelId, paymentHashes, ByteVector.view(reason.getBytes(Charsets.US_ASCII)))
}

case class UnknownMessage(tag: Int, data: ByteVector) extends LightningMessage
Original file line number Diff line number Diff line change
Expand Up @@ -18,13 +18,13 @@ package fr.acinq.eclair.wire.protocol

import com.google.common.base.Charsets
import fr.acinq.bitcoin.scalacompat.Crypto.{PrivateKey, PublicKey}
import fr.acinq.bitcoin.scalacompat.{ByteVector32, ByteVector64, Crypto, Satoshi}
import fr.acinq.bitcoin.scalacompat.{ByteVector32, ByteVector64, Crypto, Satoshi, TxId}
import fr.acinq.eclair.blockchain.fee.FeeratePerKw
import fr.acinq.eclair.channel._
import fr.acinq.eclair.transactions.Transactions
import fr.acinq.eclair.wire.protocol.CommonCodecs._
import fr.acinq.eclair.wire.protocol.TlvCodecs.{genericTlv, tlvField, tsatoshi32}
import fr.acinq.eclair.{ToMilliSatoshiConversion, UInt64}
import fr.acinq.eclair.{MilliSatoshi, ToMilliSatoshiConversion, UInt64}
import scodec.Codec
import scodec.bits.{BitVector, ByteVector}
import scodec.codecs._
Expand All @@ -48,6 +48,9 @@ object LiquidityAds {
val total: Satoshi = miningFee + serviceFee
}

/** Fees paid for the funding transaction that provides liquidity. */
case class FundingFee(amount: MilliSatoshi, fundingTxId: TxId)

sealed trait FundingLease extends Tlv {
/** Fees paid by the liquidity buyer. */
def fees(feerate: FeeratePerKw, requestedAmount: Satoshi, contributedAmount: Satoshi): LeaseFees
Expand Down Expand Up @@ -106,6 +109,10 @@ object LiquidityAds {
// @formatter:off
/** Fees are transferred from the buyer's channel balance to the seller's during the interactive-tx construction. */
case object FromChannelBalance extends PaymentType { override val rfcName: String = "from_channel_balance" }
/** Fees will be deducted from future HTLCs that will be relayed to the buyer. */
case object FromFutureHtlc extends PaymentType { override val rfcName: String = "from_future_htlc" }
/** Fees will be deducted from future HTLCs that will be relayed to the buyer, but the preimage is revealed immediately. */
case object FromFutureHtlcWithPreimage extends PaymentType { override val rfcName: String = "from_future_htlc_with_preimage" }
/** Sellers may support unknown payment types, which we must ignore. */
case class Unknown(bitIndex: Int) extends PaymentType { override val rfcName: String = s"unknown_$bitIndex" }
// @formatter:on
Expand All @@ -119,6 +126,8 @@ object LiquidityAds {
object PaymentDetails {
// @formatter:off
case object FromChannelBalance extends PaymentDetails { override val paymentType: PaymentType = PaymentType.FromChannelBalance }
case class FromFutureHtlc(paymentHashes: List[ByteVector32]) extends PaymentDetails { override val paymentType: PaymentType = PaymentType.FromFutureHtlc }
case class FromFutureHtlcWithPreimage(preimages: List[ByteVector32]) extends PaymentDetails { override val paymentType: PaymentType = PaymentType.FromFutureHtlcWithPreimage }
// @formatter:on
}

Expand Down Expand Up @@ -244,6 +253,8 @@ object LiquidityAds {

private val paymentDetails: Codec[PaymentDetails] = discriminated[PaymentDetails].by(varint)
.typecase(UInt64(0), tlvField(provide(PaymentDetails.FromChannelBalance)))
.typecase(UInt64(128), tlvField(list(bytes32).as[PaymentDetails.FromFutureHtlc]))
.typecase(UInt64(129), tlvField(list(bytes32).as[PaymentDetails.FromFutureHtlcWithPreimage]))

val requestFunds: Codec[RequestFunds] = (
("requestedAmount" | satoshi) ::
Expand All @@ -260,12 +271,16 @@ object LiquidityAds {
f = { bytes =>
bytes.bits.toIndexedSeq.reverse.zipWithIndex.collect {
case (true, 0) => PaymentType.FromChannelBalance
case (true, 128) => PaymentType.FromFutureHtlc
case (true, 129) => PaymentType.FromFutureHtlcWithPreimage
case (true, idx) => PaymentType.Unknown(idx)
}.toSet
},
g = { paymentTypes =>
val indexes = paymentTypes.collect {
case PaymentType.FromChannelBalance => 0
case PaymentType.FromFutureHtlc => 128
case PaymentType.FromFutureHtlcWithPreimage => 129
case PaymentType.Unknown(idx) => idx
}
// When converting from BitVector to ByteVector, scodec pads right instead of left, so we make sure we pad to bytes *before* setting bits.
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,42 @@
/*
* Copyright 2024 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.wire.protocol

import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
import fr.acinq.eclair.UInt64
import fr.acinq.eclair.wire.protocol.CommonCodecs.{publicKey, varint}
import fr.acinq.eclair.wire.protocol.TlvCodecs.tlvStream
import scodec.Codec
import scodec.bits.HexStringSyntax
import scodec.codecs._

/**
* Created by t-bast on 07/06/2024.
*/

sealed trait WillAddHtlcTlv extends Tlv

object WillAddHtlcTlv {
/** Blinding ephemeral public key that should be used to derive shared secrets when using route blinding. */
case class BlindingPoint(publicKey: PublicKey) extends WillAddHtlcTlv

private val blindingPoint: Codec[BlindingPoint] = (("length" | constant(hex"21")) :: ("blinding" | publicKey)).as[BlindingPoint]

val willAddHtlcTlvCodec: Codec[TlvStream[WillAddHtlcTlv]] = tlvStream(discriminated[WillAddHtlcTlv].by(varint)
.typecase(UInt64(0), blindingPoint)
)
}
Loading

0 comments on commit 41700ab

Please sign in to comment.