Skip to content

Commit a69dc8f

Browse files
committed
Add funding_fee_credit feature
We add an optional feature that lets on-the-fly funding clients accept payments that are too small to pay the fees for an on-the-fly funding. When that happens, the payment amount is added as "fee credit" without performing an on-chain operation. Once enough fee credit has been obtained, we can initiate an on-chain operation to create a channel or a splice by paying part of the fees from the fee credit. This feature makes more efficient use of on-chain transactions by trusting that the seller will honor our fee credit in the future. The fee credit takes precedence over other ways of paying the fees (from the channel balance or future HTLCs), which guarantees that the fee credit eventually converges to 0.
1 parent 69014d2 commit a69dc8f

24 files changed

+879
-116
lines changed

eclair-core/src/main/scala/fr/acinq/eclair/Features.scala

Lines changed: 12 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -332,6 +332,14 @@ object Features {
332332
val mandatory = 560
333333
}
334334

335+
// TODO:
336+
// - add NodeFeature once stable
337+
// - add link to bLIP
338+
case object FundingFeeCredit extends Feature with InitFeature {
339+
val rfcName = "funding_fee_credit"
340+
val mandatory = 562
341+
}
342+
335343
val knownFeatures: Set[Feature] = Set(
336344
DataLossProtect,
337345
InitialRoutingSync,
@@ -358,7 +366,8 @@ object Features {
358366
TrampolinePaymentPrototype,
359367
AsyncPaymentPrototype,
360368
SplicePrototype,
361-
OnTheFlyFunding
369+
OnTheFlyFunding,
370+
FundingFeeCredit
362371
)
363372

364373
// Features may depend on other features, as specified in Bolt 9.
@@ -372,7 +381,8 @@ object Features {
372381
TrampolinePaymentPrototype -> (PaymentSecret :: Nil),
373382
KeySend -> (VariableLengthOnion :: Nil),
374383
AsyncPaymentPrototype -> (TrampolinePaymentPrototype :: Nil),
375-
OnTheFlyFunding -> (SplicePrototype :: Nil)
384+
OnTheFlyFunding -> (SplicePrototype :: Nil),
385+
FundingFeeCredit -> (OnTheFlyFunding :: Nil)
376386
)
377387

378388
case class FeatureException(message: String) extends IllegalArgumentException(message)

eclair-core/src/main/scala/fr/acinq/eclair/channel/Helpers.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -171,7 +171,7 @@ object Helpers {
171171

172172
for {
173173
script_opt <- extractShutdownScript(open.temporaryChannelId, localFeatures, remoteFeatures, open.upfrontShutdownScript_opt)
174-
willFund_opt <- LiquidityAds.validateRequest(nodeParams.privateKey, open.temporaryChannelId, fundingScript, open.fundingFeerate, isChannelCreation = true, open.requestFunding_opt, addFunding_opt.flatMap(_.rates_opt))
174+
willFund_opt <- LiquidityAds.validateRequest(nodeParams.privateKey, open.temporaryChannelId, fundingScript, open.fundingFeerate, isChannelCreation = true, open.requestFunding_opt, addFunding_opt.flatMap(_.rates_opt), open.useFeeCredit_opt)
175175
} yield (channelFeatures, script_opt, willFund_opt)
176176
}
177177

eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -952,7 +952,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
952952
val parentCommitment = d.commitments.latest.commitment
953953
val localFundingPubKey = nodeParams.channelKeyManager.fundingPublicKey(d.commitments.params.localParams.fundingKeyPath, parentCommitment.fundingTxIndex + 1).publicKey
954954
val fundingScript = Funding.makeFundingPubKeyScript(localFundingPubKey, msg.fundingPubKey)
955-
LiquidityAds.validateRequest(nodeParams.privateKey, d.channelId, fundingScript, msg.feerate, isChannelCreation = false, msg.requestFunding_opt, nodeParams.willFundRates_opt) match {
955+
LiquidityAds.validateRequest(nodeParams.privateKey, d.channelId, fundingScript, msg.feerate, isChannelCreation = false, msg.requestFunding_opt, nodeParams.willFundRates_opt, msg.useFeeCredit_opt) match {
956956
case Left(t) =>
957957
log.warning("rejecting splice request with invalid liquidity ads: {}", t.getMessage)
958958
stay() using d.copy(spliceStatus = SpliceStatus.SpliceAborted) sending TxAbort(d.channelId, t.getMessage)
@@ -963,7 +963,8 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
963963
fundingPubKey = localFundingPubKey,
964964
pushAmount = 0.msat,
965965
requireConfirmedInputs = nodeParams.channelConf.requireConfirmedInputsForDualFunding,
966-
willFund_opt = willFund_opt.map(_.willFund)
966+
willFund_opt = willFund_opt.map(_.willFund),
967+
feeCreditUsed_opt = msg.useFeeCredit_opt
967968
)
968969
val fundingParams = InteractiveTxParams(
969970
channelId = d.channelId,

eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -180,6 +180,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
180180
Some(ChannelTlv.ChannelTypeTlv(d.init.channelType)),
181181
if (d.init.requireConfirmedInputs) Some(ChannelTlv.RequireConfirmedInputsTlv()) else None,
182182
willFund_opt.map(l => ChannelTlv.ProvideFundingTlv(l.willFund)),
183+
open.useFeeCredit_opt.map(c => ChannelTlv.FeeCreditUsedTlv(c)),
183184
d.init.pushAmount_opt.map(amount => ChannelTlv.PushAmountTlv(amount)),
184185
).flatten
185186
val accept = AcceptDualFundedChannel(
@@ -547,7 +548,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
547548
stay() using d.copy(rbfStatus = RbfStatus.RbfAborted) sending TxAbort(d.channelId, InvalidRbfAttemptTooSoon(d.channelId, d.latestFundingTx.createdAt, d.latestFundingTx.createdAt + nodeParams.channelConf.remoteRbfLimits.attemptDeltaBlocks).getMessage)
548549
} else {
549550
val fundingScript = d.commitments.latest.commitInput.txOut.publicKeyScript
550-
LiquidityAds.validateRequest(nodeParams.privateKey, d.channelId, fundingScript, msg.feerate, isChannelCreation = true, msg.requestFunding_opt, nodeParams.willFundRates_opt) match {
551+
LiquidityAds.validateRequest(nodeParams.privateKey, d.channelId, fundingScript, msg.feerate, isChannelCreation = true, msg.requestFunding_opt, nodeParams.willFundRates_opt, None) match {
551552
case Left(t) =>
552553
log.warning("rejecting rbf attempt: invalid liquidity ads request ({})", t.getMessage)
553554
stay() using d.copy(rbfStatus = RbfStatus.RbfAborted) sending TxAbort(d.channelId, t.getMessage)

eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxBuilder.scala

Lines changed: 20 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -37,7 +37,7 @@ import fr.acinq.eclair.crypto.keymanager.ChannelKeyManager
3737
import fr.acinq.eclair.transactions.Transactions.{CommitTx, HtlcTx, InputInfo, TxOwner}
3838
import fr.acinq.eclair.transactions.{CommitmentSpec, DirectedHtlc, Scripts, Transactions}
3939
import fr.acinq.eclair.wire.protocol._
40-
import fr.acinq.eclair.{Logs, MilliSatoshi, MilliSatoshiLong, NodeParams, UInt64}
40+
import fr.acinq.eclair.{Logs, MilliSatoshi, MilliSatoshiLong, NodeParams, ToMilliSatoshiConversion, UInt64}
4141
import scodec.bits.ByteVector
4242

4343
import scala.concurrent.{ExecutionContext, Future}
@@ -157,13 +157,18 @@ object InteractiveTxBuilder {
157157
// BOLT 2: the initiator's serial IDs MUST use even values and the non-initiator odd values.
158158
val serialIdParity: Int = if (isInitiator) 0 else 1
159159

160-
def liquidityFees(liquidityPurchase_opt: Option[LiquidityAds.Purchase]): Satoshi = {
160+
def liquidityFees(liquidityPurchase_opt: Option[LiquidityAds.Purchase]): MilliSatoshi = {
161161
liquidityPurchase_opt.map(l => l.paymentDetails match {
162162
// The initiator of the interactive-tx is the liquidity buyer (if liquidity ads is used).
163-
case LiquidityAds.PaymentDetails.FromChannelBalance | _: LiquidityAds.PaymentDetails.FromChannelBalanceForFutureHtlc => if (isInitiator) l.fees.total else -l.fees.total
163+
case LiquidityAds.PaymentDetails.FromChannelBalance | _: LiquidityAds.PaymentDetails.FromChannelBalanceForFutureHtlc =>
164+
val feesOwed = l match {
165+
case l: LiquidityAds.Purchase.Standard => l.fees.total.toMilliSatoshi
166+
case l: LiquidityAds.Purchase.WithFeeCredit => l.fees.total.toMilliSatoshi - l.feeCreditUsed
167+
}
168+
if (isInitiator) feesOwed else -feesOwed
164169
// Fees will be paid later, when relaying HTLCs.
165-
case _: LiquidityAds.PaymentDetails.FromFutureHtlc | _: LiquidityAds.PaymentDetails.FromFutureHtlcWithPreimage => 0.sat
166-
}).getOrElse(0 sat)
170+
case _: LiquidityAds.PaymentDetails.FromFutureHtlc | _: LiquidityAds.PaymentDetails.FromFutureHtlcWithPreimage => 0 msat
171+
}).getOrElse(0 msat)
167172
}
168173
}
169174

@@ -744,6 +749,16 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
744749
return Left(InvalidCompleteInteractiveTx(fundingParams.channelId))
745750
}
746751

752+
liquidityPurchase_opt match {
753+
case Some(p: LiquidityAds.Purchase.WithFeeCredit) if !fundingParams.isInitiator =>
754+
val currentFeeCredit = nodeParams.db.liquidity.getFeeCredit(remoteNodeId)
755+
if (currentFeeCredit < p.feeCreditUsed) {
756+
log.warn("not enough fee credit: our peer may be malicious ({} < {})", currentFeeCredit, p.feeCreditUsed)
757+
return Left(InvalidCompleteInteractiveTx(fundingParams.channelId))
758+
}
759+
case _ => ()
760+
}
761+
747762
previousTransactions.headOption match {
748763
case Some(previousTx) =>
749764
// This is an RBF attempt: even if our peer does not contribute to the feerate increase, we'd like to broadcast

eclair-core/src/main/scala/fr/acinq/eclair/db/DualDatabases.scala

Lines changed: 15 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -463,4 +463,19 @@ case class DualLiquidityDb(primary: LiquidityDb, secondary: LiquidityDb) extends
463463
primary.getOnTheFlyFundingPreimage(paymentHash)
464464
}
465465

466+
override def addFeeCredit(nodeId: PublicKey, amount: MilliSatoshi, receivedAt: TimestampMilli): MilliSatoshi = {
467+
runAsync(secondary.addFeeCredit(nodeId, amount, receivedAt))
468+
primary.addFeeCredit(nodeId, amount, receivedAt)
469+
}
470+
471+
override def getFeeCredit(nodeId: PublicKey): MilliSatoshi = {
472+
runAsync(secondary.getFeeCredit(nodeId))
473+
primary.getFeeCredit(nodeId)
474+
}
475+
476+
override def removeFeeCredit(nodeId: PublicKey, amountUsed: MilliSatoshi): MilliSatoshi = {
477+
runAsync(secondary.removeFeeCredit(nodeId, amountUsed))
478+
primary.removeFeeCredit(nodeId, amountUsed)
479+
}
480+
466481
}

eclair-core/src/main/scala/fr/acinq/eclair/db/LiquidityDb.scala

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@ import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
2020
import fr.acinq.bitcoin.scalacompat.{ByteVector32, TxId}
2121
import fr.acinq.eclair.channel.{ChannelLiquidityPurchased, LiquidityPurchase}
2222
import fr.acinq.eclair.payment.relay.OnTheFlyFunding
23+
import fr.acinq.eclair.{MilliSatoshi, TimestampMilli}
2324

2425
/**
2526
* Created by t-bast on 13/09/2024.
@@ -57,4 +58,13 @@ trait LiquidityDb {
5758
/** Check if we received the preimage for the given payment hash of an on-the-fly payment. */
5859
def getOnTheFlyFundingPreimage(paymentHash: ByteVector32): Option[ByteVector32]
5960

61+
/** Add fee credit for the given remote node and return the updated fee credit. */
62+
def addFeeCredit(nodeId: PublicKey, amount: MilliSatoshi, receivedAt: TimestampMilli = TimestampMilli.now()): MilliSatoshi
63+
64+
/** Return the amount owed to the given remote node as fee credit. */
65+
def getFeeCredit(nodeId: PublicKey): MilliSatoshi
66+
67+
/** Remove fee credit for the given remote node and return the remaining fee credit. */
68+
def removeFeeCredit(nodeId: PublicKey, amountUsed: MilliSatoshi): MilliSatoshi
69+
6070
}

eclair-core/src/main/scala/fr/acinq/eclair/db/pg/PgLiquidityDb.scala

Lines changed: 42 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,7 @@ import fr.acinq.eclair.db.Monitoring.Tags.DbBackends
2525
import fr.acinq.eclair.db.pg.PgUtils.PgLock.NoLock.withLock
2626
import fr.acinq.eclair.payment.relay.OnTheFlyFunding
2727
import fr.acinq.eclair.wire.protocol.LiquidityAds
28-
import fr.acinq.eclair.{MilliSatoshi, MilliSatoshiLong}
28+
import fr.acinq.eclair.{MilliSatoshi, MilliSatoshiLong, TimestampMilli}
2929
import grizzled.slf4j.Logging
3030
import scodec.bits.BitVector
3131

@@ -58,6 +58,7 @@ class PgLiquidityDb(implicit ds: DataSource) extends LiquidityDb with Logging {
5858
// On-the-fly funding.
5959
statement.executeUpdate("CREATE TABLE liquidity.on_the_fly_funding_preimages (payment_hash TEXT NOT NULL PRIMARY KEY, preimage TEXT NOT NULL, received_at TIMESTAMP WITH TIME ZONE NOT NULL)")
6060
statement.executeUpdate("CREATE TABLE liquidity.pending_on_the_fly_funding (node_id TEXT NOT NULL, payment_hash TEXT NOT NULL, channel_id TEXT NOT NULL, tx_id TEXT NOT NULL, funding_tx_index BIGINT NOT NULL, remaining_fees_msat BIGINT NOT NULL, proposed BYTEA NOT NULL, funded_at TIMESTAMP WITH TIME ZONE NOT NULL, PRIMARY KEY (node_id, payment_hash))")
61+
statement.executeUpdate("CREATE TABLE liquidity.fee_credits (node_id TEXT NOT NULL PRIMARY KEY, amount_msat BIGINT NOT NULL, updated_at TIMESTAMP WITH TIME ZONE NOT NULL)")
6162
// Indexes.
6263
statement.executeUpdate("CREATE INDEX liquidity_purchases_node_id_idx ON liquidity.purchases(node_id)")
6364
case Some(CURRENT_VERSION) => () // table is up-to-date, nothing to do
@@ -129,6 +130,7 @@ class PgLiquidityDb(implicit ds: DataSource) extends LiquidityDb with Logging {
129130
override def addPendingOnTheFlyFunding(remoteNodeId: Crypto.PublicKey, pending: OnTheFlyFunding.Pending): Unit = withMetrics("liquidity/add-pending-on-the-fly-funding", DbBackends.Postgres) {
130131
pending.status match {
131132
case _: OnTheFlyFunding.Status.Proposed => ()
133+
case _: OnTheFlyFunding.Status.AddedToFeeCredit => ()
132134
case status: OnTheFlyFunding.Status.Funded => withLock { pg =>
133135
using(pg.prepareStatement("INSERT INTO liquidity.pending_on_the_fly_funding (node_id, payment_hash, channel_id, tx_id, funding_tx_index, remaining_fees_msat, proposed, funded_at) VALUES (?, ?, ?, ?, ?, ?, ?, ?) ON CONFLICT DO NOTHING")) { statement =>
134136
statement.setString(1, remoteNodeId.toHex)
@@ -237,4 +239,43 @@ class PgLiquidityDb(implicit ds: DataSource) extends LiquidityDb with Logging {
237239
}
238240
}
239241

242+
override def addFeeCredit(nodeId: PublicKey, amount: MilliSatoshi, receivedAt: TimestampMilli): MilliSatoshi = withMetrics("liquidity/add-fee-credit", DbBackends.Postgres) {
243+
withLock { pg =>
244+
using(pg.prepareStatement("INSERT INTO liquidity.fee_credits(node_id, amount_msat, updated_at) VALUES (?, ?, ?) ON CONFLICT (node_id) DO UPDATE SET (amount_msat, updated_at) = (liquidity.fee_credits.amount_msat + EXCLUDED.amount_msat, EXCLUDED.updated_at) RETURNING amount_msat")) { statement =>
245+
statement.setString(1, nodeId.toHex)
246+
statement.setLong(2, amount.toLong)
247+
statement.setTimestamp(3, receivedAt.toSqlTimestamp)
248+
statement.executeQuery().map(_.getLong("amount_msat").msat).headOption.getOrElse(0 msat)
249+
}
250+
}
251+
}
252+
253+
override def getFeeCredit(nodeId: PublicKey): MilliSatoshi = withMetrics("liquidity/get-fee-credit", DbBackends.Postgres) {
254+
withLock { pg =>
255+
using(pg.prepareStatement("SELECT amount_msat FROM liquidity.fee_credits WHERE node_id = ?")) { statement =>
256+
statement.setString(1, nodeId.toHex)
257+
statement.executeQuery().map(_.getLong("amount_msat").msat).headOption.getOrElse(0 msat)
258+
}
259+
}
260+
}
261+
262+
override def removeFeeCredit(nodeId: PublicKey, amountUsed: MilliSatoshi): MilliSatoshi = withMetrics("liquidity/remove-fee-credit", DbBackends.Postgres) {
263+
withLock { pg =>
264+
using(pg.prepareStatement("SELECT amount_msat FROM liquidity.fee_credits WHERE node_id = ?")) { statement =>
265+
statement.setString(1, nodeId.toHex)
266+
statement.executeQuery().map(_.getLong("amount_msat").msat).headOption match {
267+
case Some(current) => using(pg.prepareStatement("UPDATE liquidity.fee_credits SET (amount_msat, updated_at) = (?, ?) WHERE node_id = ?")) { statement =>
268+
val updated = (current - amountUsed).max(0 msat)
269+
statement.setLong(1, updated.toLong)
270+
statement.setTimestamp(2, Timestamp.from(Instant.now()))
271+
statement.setString(3, nodeId.toHex)
272+
statement.executeUpdate()
273+
updated
274+
}
275+
case None => 0 msat
276+
}
277+
}
278+
}
279+
}
280+
240281
}

eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteLiquidityDb.scala

Lines changed: 48 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -53,6 +53,7 @@ class SqliteLiquidityDb(val sqlite: Connection) extends LiquidityDb with Logging
5353
// On-the-fly funding.
5454
statement.executeUpdate("CREATE TABLE on_the_fly_funding_preimages (payment_hash BLOB NOT NULL PRIMARY KEY, preimage BLOB NOT NULL, received_at INTEGER NOT NULL)")
5555
statement.executeUpdate("CREATE TABLE on_the_fly_funding_pending (node_id BLOB NOT NULL, payment_hash BLOB NOT NULL, channel_id BLOB NOT NULL, tx_id BLOB NOT NULL, funding_tx_index INTEGER NOT NULL, remaining_fees_msat INTEGER NOT NULL, proposed BLOB NOT NULL, funded_at INTEGER NOT NULL, PRIMARY KEY (node_id, payment_hash))")
56+
statement.executeUpdate("CREATE TABLE fee_credits (node_id BLOB NOT NULL PRIMARY KEY, amount_msat INTEGER NOT NULL, updated_at INTEGER NOT NULL)")
5657
// Indexes.
5758
statement.executeUpdate("CREATE INDEX liquidity_purchases_node_id_idx ON liquidity_purchases(node_id)")
5859
case Some(CURRENT_VERSION) => () // table is up-to-date, nothing to do
@@ -117,6 +118,7 @@ class SqliteLiquidityDb(val sqlite: Connection) extends LiquidityDb with Logging
117118
override def addPendingOnTheFlyFunding(remoteNodeId: Crypto.PublicKey, pending: OnTheFlyFunding.Pending): Unit = withMetrics("liquidity/add-pending-on-the-fly-funding", DbBackends.Sqlite) {
118119
pending.status match {
119120
case _: OnTheFlyFunding.Status.Proposed => ()
121+
case _: OnTheFlyFunding.Status.AddedToFeeCredit => ()
120122
case status: OnTheFlyFunding.Status.Funded =>
121123
using(sqlite.prepareStatement("INSERT OR IGNORE INTO on_the_fly_funding_pending (node_id, payment_hash, channel_id, tx_id, funding_tx_index, remaining_fees_msat, proposed, funded_at) VALUES (?, ?, ?, ?, ?, ?, ?, ?)")) { statement =>
122124
statement.setBytes(1, remoteNodeId.value.toArray)
@@ -212,4 +214,50 @@ class SqliteLiquidityDb(val sqlite: Connection) extends LiquidityDb with Logging
212214
}
213215
}
214216

217+
override def addFeeCredit(nodeId: PublicKey, amount: MilliSatoshi, receivedAt: TimestampMilli): MilliSatoshi = withMetrics("liquidity/add-fee-credit", DbBackends.Sqlite) {
218+
using(sqlite.prepareStatement("SELECT amount_msat FROM fee_credits WHERE node_id = ?")) { statement =>
219+
statement.setBytes(1, nodeId.value.toArray)
220+
statement.executeQuery().map(_.getLong("amount_msat").msat).headOption match {
221+
case Some(current) => using(sqlite.prepareStatement("UPDATE fee_credits SET (amount_msat, updated_at) = (?, ?) WHERE node_id = ?")) { statement =>
222+
statement.setLong(1, (current + amount).toLong)
223+
statement.setLong(2, receivedAt.toLong)
224+
statement.setBytes(3, nodeId.value.toArray)
225+
statement.executeUpdate()
226+
amount + current
227+
}
228+
case None => using(sqlite.prepareStatement("INSERT OR IGNORE INTO fee_credits(node_id, amount_msat, updated_at) VALUES (?, ?, ?)")) { statement =>
229+
statement.setBytes(1, nodeId.value.toArray)
230+
statement.setLong(2, amount.toLong)
231+
statement.setLong(3, receivedAt.toLong)
232+
statement.executeUpdate()
233+
amount
234+
}
235+
}
236+
}
237+
}
238+
239+
override def getFeeCredit(nodeId: PublicKey): MilliSatoshi = withMetrics("liquidity/get-fee-credit", DbBackends.Sqlite) {
240+
using(sqlite.prepareStatement("SELECT amount_msat FROM fee_credits WHERE node_id = ?")) { statement =>
241+
statement.setBytes(1, nodeId.value.toArray)
242+
statement.executeQuery().map(_.getLong("amount_msat").msat).headOption.getOrElse(0 msat)
243+
}
244+
}
245+
246+
override def removeFeeCredit(nodeId: PublicKey, amountUsed: MilliSatoshi): MilliSatoshi = withMetrics("liquidity/remove-fee-credit", DbBackends.Sqlite) {
247+
using(sqlite.prepareStatement("SELECT amount_msat FROM fee_credits WHERE node_id = ?")) { statement =>
248+
statement.setBytes(1, nodeId.value.toArray)
249+
statement.executeQuery().map(_.getLong("amount_msat").msat).headOption match {
250+
case Some(current) => using(sqlite.prepareStatement("UPDATE fee_credits SET (amount_msat, updated_at) = (?, ?) WHERE node_id = ?")) { statement =>
251+
val updated = (current - amountUsed).max(0 msat)
252+
statement.setLong(1, updated.toLong)
253+
statement.setLong(2, TimestampMilli.now().toLong)
254+
statement.setBytes(3, nodeId.value.toArray)
255+
statement.executeUpdate()
256+
updated
257+
}
258+
case None => 0 msat
259+
}
260+
}
261+
}
262+
215263
}

eclair-core/src/main/scala/fr/acinq/eclair/io/Monitoring.scala

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -75,6 +75,7 @@ object Monitoring {
7575
val Rejected = "rejected"
7676
val Expired = "expired"
7777
val Timeout = "timeout"
78+
val AddedToFeeCredit = "added-to-fee-credit"
7879
val Funded = "funded"
7980
val RelaySucceeded = "relay-succeeded"
8081

0 commit comments

Comments
 (0)