Skip to content

Commit 0e2d7da

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 881bd40 commit 0e2d7da

22 files changed

+850
-103
lines changed

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

+12-2
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

+1-1
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, open.requestFunding_opt, addFunding_opt.flatMap(_.rates_opt))
174+
willFund_opt <- LiquidityAds.validateRequest(nodeParams.privateKey, open.temporaryChannelId, fundingScript, open.fundingFeerate, 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

+3-2
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, msg.requestFunding_opt, nodeParams.willFundRates_opt) match {
955+
LiquidityAds.validateRequest(nodeParams.privateKey, d.channelId, fundingScript, msg.feerate, 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

+2-1
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 (nodeParams.channelConf.requireConfirmedInputsForDualFunding) 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, msg.requestFunding_opt, nodeParams.willFundRates_opt) match {
551+
LiquidityAds.validateRequest(nodeParams.privateKey, d.channelId, fundingScript, msg.feerate, 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

+20-5
Original file line numberDiff line numberDiff line change
@@ -35,7 +35,7 @@ import fr.acinq.eclair.crypto.keymanager.ChannelKeyManager
3535
import fr.acinq.eclair.transactions.Transactions.{CommitTx, HtlcTx, InputInfo, TxOwner}
3636
import fr.acinq.eclair.transactions.{CommitmentSpec, DirectedHtlc, Scripts, Transactions}
3737
import fr.acinq.eclair.wire.protocol._
38-
import fr.acinq.eclair.{Logs, MilliSatoshi, MilliSatoshiLong, NodeParams, UInt64}
38+
import fr.acinq.eclair.{Logs, MilliSatoshi, MilliSatoshiLong, NodeParams, ToMilliSatoshiConversion, UInt64}
3939
import scodec.bits.ByteVector
4040

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

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

@@ -734,6 +739,16 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
734739
return Left(InvalidCompleteInteractiveTx(fundingParams.channelId))
735740
}
736741

742+
liquidityPurchase_opt match {
743+
case Some(p: LiquidityAds.Purchase.WithFeeCredit) if !fundingParams.isInitiator =>
744+
val currentFeeCredit = nodeParams.db.onTheFlyFunding.getFeeCredit(remoteNodeId)
745+
if (currentFeeCredit < p.feeCreditUsed) {
746+
log.warn("not enough fee credit: our peer may be malicious ({} < {})", currentFeeCredit, p.feeCreditUsed)
747+
return Left(InvalidCompleteInteractiveTx(fundingParams.channelId))
748+
}
749+
case _ => ()
750+
}
751+
737752
previousTransactions.headOption match {
738753
case Some(previousTx) =>
739754
// 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

+15
Original file line numberDiff line numberDiff line change
@@ -442,4 +442,19 @@ case class DualOnTheFlyFundingDb(primary: OnTheFlyFundingDb, secondary: OnTheFly
442442
runAsync(secondary.listPendingPayments())
443443
primary.listPendingPayments()
444444
}
445+
446+
override def addFeeCredit(nodeId: PublicKey, amount: MilliSatoshi, receivedAt: TimestampMilli): MilliSatoshi = {
447+
runAsync(secondary.addFeeCredit(nodeId, amount, receivedAt))
448+
primary.addFeeCredit(nodeId, amount, receivedAt)
449+
}
450+
451+
override def getFeeCredit(nodeId: PublicKey): MilliSatoshi = {
452+
runAsync(secondary.getFeeCredit(nodeId))
453+
primary.getFeeCredit(nodeId)
454+
}
455+
456+
override def removeFeeCredit(nodeId: PublicKey, amountUsed: MilliSatoshi): MilliSatoshi = {
457+
runAsync(secondary.removeFeeCredit(nodeId, amountUsed))
458+
primary.removeFeeCredit(nodeId, amountUsed)
459+
}
445460
}

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

+10
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,7 @@ package fr.acinq.eclair.db
1818

1919
import fr.acinq.bitcoin.scalacompat.ByteVector32
2020
import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
21+
import fr.acinq.eclair.{MilliSatoshi, TimestampMilli}
2122
import fr.acinq.eclair.payment.relay.OnTheFlyFunding
2223

2324
/**
@@ -44,4 +45,13 @@ trait OnTheFlyFundingDb {
4445
/** List the payment_hashes of pending proposals we funded for all remote nodes. */
4546
def listPendingPayments(): Map[PublicKey, Set[ByteVector32]]
4647

48+
/** Add fee credit for the given remote node and return the updated fee credit. */
49+
def addFeeCredit(nodeId: PublicKey, amount: MilliSatoshi, receivedAt: TimestampMilli = TimestampMilli.now()): MilliSatoshi
50+
51+
/** Return the amount owed to the given remote node as fee credit. */
52+
def getFeeCredit(nodeId: PublicKey): MilliSatoshi
53+
54+
/** Remove fee credit for the given remote node and return the remaining fee credit. */
55+
def removeFeeCredit(nodeId: PublicKey, amountUsed: MilliSatoshi): MilliSatoshi
56+
4757
}

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

+42-1
Original file line numberDiff line numberDiff line change
@@ -18,12 +18,12 @@ package fr.acinq.eclair.db.pg
1818

1919
import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
2020
import fr.acinq.bitcoin.scalacompat.{ByteVector32, Crypto, TxId}
21-
import fr.acinq.eclair.MilliSatoshiLong
2221
import fr.acinq.eclair.db.Monitoring.Metrics.withMetrics
2322
import fr.acinq.eclair.db.Monitoring.Tags.DbBackends
2423
import fr.acinq.eclair.db.OnTheFlyFundingDb
2524
import fr.acinq.eclair.db.pg.PgUtils.PgLock
2625
import fr.acinq.eclair.payment.relay.OnTheFlyFunding
26+
import fr.acinq.eclair.{MilliSatoshi, MilliSatoshiLong, TimestampMilli}
2727
import scodec.bits.BitVector
2828

2929
import java.sql.Timestamp
@@ -53,6 +53,7 @@ class PgOnTheFlyFundingDb(implicit ds: DataSource, lock: PgLock) extends OnTheFl
5353
statement.executeUpdate("CREATE SCHEMA IF NOT EXISTS on_the_fly_funding")
5454
statement.executeUpdate("CREATE TABLE on_the_fly_funding.preimages (payment_hash TEXT NOT NULL PRIMARY KEY, preimage TEXT NOT NULL, received_at TIMESTAMP WITH TIME ZONE NOT NULL)")
5555
statement.executeUpdate("CREATE TABLE on_the_fly_funding.pending (remote_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 (remote_node_id, payment_hash))")
56+
statement.executeUpdate("CREATE TABLE on_the_fly_funding.fee_credit (remote_node_id TEXT NOT NULL PRIMARY KEY, amount_msat BIGINT NOT NULL, updated_at TIMESTAMP WITH TIME ZONE NOT NULL)")
5657
case Some(CURRENT_VERSION) => () // table is up-to-date, nothing to do
5758
case Some(unknownVersion) => throw new RuntimeException(s"Unknown version of DB $DB_NAME found, version=$unknownVersion")
5859
}
@@ -83,6 +84,7 @@ class PgOnTheFlyFundingDb(implicit ds: DataSource, lock: PgLock) extends OnTheFl
8384
override def addPending(remoteNodeId: Crypto.PublicKey, pending: OnTheFlyFunding.Pending): Unit = withMetrics("on-the-fly-funding/add-pending", DbBackends.Postgres) {
8485
pending.status match {
8586
case _: OnTheFlyFunding.Status.Proposed => ()
87+
case _: OnTheFlyFunding.Status.AddedToFeeCredit => ()
8688
case status: OnTheFlyFunding.Status.Funded => withLock { pg =>
8789
using(pg.prepareStatement("INSERT INTO on_the_fly_funding.pending (remote_node_id, payment_hash, channel_id, tx_id, funding_tx_index, remaining_fees_msat, proposed, funded_at) VALUES (?, ?, ?, ?, ?, ?, ?, ?) ON CONFLICT DO NOTHING")) { statement =>
8890
statement.setString(1, remoteNodeId.toHex)
@@ -144,4 +146,43 @@ class PgOnTheFlyFundingDb(implicit ds: DataSource, lock: PgLock) extends OnTheFl
144146
}
145147
}
146148

149+
override def addFeeCredit(nodeId: PublicKey, amount: MilliSatoshi, receivedAt: TimestampMilli): MilliSatoshi = withMetrics("on-the-fly-funding/add-fee-credit", DbBackends.Postgres) {
150+
withLock { pg =>
151+
using(pg.prepareStatement("INSERT INTO on_the_fly_funding.fee_credit(remote_node_id, amount_msat, updated_at) VALUES (?, ?, ?) ON CONFLICT (remote_node_id) DO UPDATE SET (amount_msat, updated_at) = (on_the_fly_funding.fee_credit.amount_msat + EXCLUDED.amount_msat, EXCLUDED.updated_at) RETURNING amount_msat")) { statement =>
152+
statement.setString(1, nodeId.toHex)
153+
statement.setLong(2, amount.toLong)
154+
statement.setTimestamp(3, receivedAt.toSqlTimestamp)
155+
statement.executeQuery().map(_.getLong("amount_msat").msat).headOption.getOrElse(0 msat)
156+
}
157+
}
158+
}
159+
160+
override def getFeeCredit(nodeId: PublicKey): MilliSatoshi = withMetrics("on-the-fly-funding/get-fee-credit", DbBackends.Postgres) {
161+
withLock { pg =>
162+
using(pg.prepareStatement("SELECT amount_msat FROM on_the_fly_funding.fee_credit WHERE remote_node_id = ?")) { statement =>
163+
statement.setString(1, nodeId.toHex)
164+
statement.executeQuery().map(_.getLong("amount_msat").msat).headOption.getOrElse(0 msat)
165+
}
166+
}
167+
}
168+
169+
override def removeFeeCredit(nodeId: PublicKey, amountUsed: MilliSatoshi): MilliSatoshi = withMetrics("on-the-fly-funding/remove-fee-credit", DbBackends.Postgres) {
170+
withLock { pg =>
171+
using(pg.prepareStatement("SELECT amount_msat FROM on_the_fly_funding.fee_credit WHERE remote_node_id = ?")) { statement =>
172+
statement.setString(1, nodeId.toHex)
173+
statement.executeQuery().map(_.getLong("amount_msat").msat).headOption match {
174+
case Some(current) => using(pg.prepareStatement("UPDATE on_the_fly_funding.fee_credit SET (amount_msat, updated_at) = (?, ?) WHERE remote_node_id = ?")) { statement =>
175+
val updated = (current - amountUsed).max(0 msat)
176+
statement.setLong(1, updated.toLong)
177+
statement.setTimestamp(2, Timestamp.from(Instant.now()))
178+
statement.setString(3, nodeId.toHex)
179+
statement.executeUpdate()
180+
updated
181+
}
182+
case None => 0 msat
183+
}
184+
}
185+
}
186+
}
187+
147188
}

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

+49-1
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,7 @@ import fr.acinq.eclair.db.Monitoring.Metrics.withMetrics
2222
import fr.acinq.eclair.db.Monitoring.Tags.DbBackends
2323
import fr.acinq.eclair.db.OnTheFlyFundingDb
2424
import fr.acinq.eclair.payment.relay.OnTheFlyFunding
25-
import fr.acinq.eclair.{MilliSatoshiLong, TimestampMilli}
25+
import fr.acinq.eclair.{MilliSatoshi, MilliSatoshiLong, TimestampMilli}
2626
import scodec.bits.BitVector
2727

2828
import java.sql.Connection
@@ -47,6 +47,7 @@ class SqliteOnTheFlyFundingDb(val sqlite: Connection) extends OnTheFlyFundingDb
4747
case None =>
4848
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)")
4949
statement.executeUpdate("CREATE TABLE on_the_fly_funding_pending (remote_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 (remote_node_id, payment_hash))")
50+
statement.executeUpdate("CREATE TABLE fee_credit (remote_node_id BLOB NOT NULL PRIMARY KEY, amount_msat INTEGER NOT NULL, updated_at INTEGER NOT NULL)")
5051
case Some(CURRENT_VERSION) => () // table is up-to-date, nothing to do
5152
case Some(unknownVersion) => throw new RuntimeException(s"Unknown version of DB $DB_NAME found, version=$unknownVersion")
5253
}
@@ -72,6 +73,7 @@ class SqliteOnTheFlyFundingDb(val sqlite: Connection) extends OnTheFlyFundingDb
7273
override def addPending(remoteNodeId: Crypto.PublicKey, pending: OnTheFlyFunding.Pending): Unit = withMetrics("on-the-fly-funding/add-pending", DbBackends.Sqlite) {
7374
pending.status match {
7475
case _: OnTheFlyFunding.Status.Proposed => ()
76+
case _: OnTheFlyFunding.Status.AddedToFeeCredit => ()
7577
case status: OnTheFlyFunding.Status.Funded =>
7678
using(sqlite.prepareStatement("INSERT OR IGNORE INTO on_the_fly_funding_pending (remote_node_id, payment_hash, channel_id, tx_id, funding_tx_index, remaining_fees_msat, proposed, funded_at) VALUES (?, ?, ?, ?, ?, ?, ?, ?)")) { statement =>
7779
statement.setBytes(1, remoteNodeId.value.toArray)
@@ -126,4 +128,50 @@ class SqliteOnTheFlyFundingDb(val sqlite: Connection) extends OnTheFlyFundingDb
126128
}
127129
}
128130

131+
override def addFeeCredit(nodeId: PublicKey, amount: MilliSatoshi, receivedAt: TimestampMilli): MilliSatoshi = withMetrics("on-the-fly-funding/add-fee-credit", DbBackends.Sqlite) {
132+
using(sqlite.prepareStatement("SELECT amount_msat FROM fee_credit WHERE remote_node_id = ?")) { statement =>
133+
statement.setBytes(1, nodeId.value.toArray)
134+
statement.executeQuery().map(_.getLong("amount_msat").msat).headOption match {
135+
case Some(current) => using(sqlite.prepareStatement("UPDATE fee_credit SET (amount_msat, updated_at) = (?, ?) WHERE remote_node_id = ?")) { statement =>
136+
statement.setLong(1, (current + amount).toLong)
137+
statement.setLong(2, receivedAt.toLong)
138+
statement.setBytes(3, nodeId.value.toArray)
139+
statement.executeUpdate()
140+
amount + current
141+
}
142+
case None => using(sqlite.prepareStatement("INSERT OR IGNORE INTO fee_credit(remote_node_id, amount_msat, updated_at) VALUES (?, ?, ?)")) { statement =>
143+
statement.setBytes(1, nodeId.value.toArray)
144+
statement.setLong(2, amount.toLong)
145+
statement.setLong(3, receivedAt.toLong)
146+
statement.executeUpdate()
147+
amount
148+
}
149+
}
150+
}
151+
}
152+
153+
override def getFeeCredit(nodeId: PublicKey): MilliSatoshi = withMetrics("on-the-fly-funding/get-fee-credit", DbBackends.Sqlite) {
154+
using(sqlite.prepareStatement("SELECT amount_msat FROM fee_credit WHERE remote_node_id = ?")) { statement =>
155+
statement.setBytes(1, nodeId.value.toArray)
156+
statement.executeQuery().map(_.getLong("amount_msat").msat).headOption.getOrElse(0 msat)
157+
}
158+
}
159+
160+
override def removeFeeCredit(nodeId: PublicKey, amountUsed: MilliSatoshi): MilliSatoshi = withMetrics("on-the-fly-funding/remove-fee-credit", DbBackends.Sqlite) {
161+
using(sqlite.prepareStatement("SELECT amount_msat FROM fee_credit WHERE remote_node_id = ?")) { statement =>
162+
statement.setBytes(1, nodeId.value.toArray)
163+
statement.executeQuery().map(_.getLong("amount_msat").msat).headOption match {
164+
case Some(current) => using(sqlite.prepareStatement("UPDATE fee_credit SET (amount_msat, updated_at) = (?, ?) WHERE remote_node_id = ?")) { statement =>
165+
val updated = (current - amountUsed).max(0 msat)
166+
statement.setLong(1, updated.toLong)
167+
statement.setLong(2, TimestampMilli.now().toLong)
168+
statement.setBytes(3, nodeId.value.toArray)
169+
statement.executeUpdate()
170+
updated
171+
}
172+
case None => 0 msat
173+
}
174+
}
175+
}
176+
129177
}

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

+1
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)