1
0
mirror of https://github.com/ACINQ/eclair.git synced 2024-11-19 01:43:22 +01:00

Enforce recommended feerate for on-the-fly funding (#2927)

When using on-the-fly funding, we reject `open_channel2` and
`splice_init` messages that use a smaller feerate than what
we previously recommended.
This commit is contained in:
Bastien Teinturier 2024-10-15 04:20:16 +02:00 committed by GitHub
parent cf6b4e3929
commit b8e6800e9d
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
6 changed files with 95 additions and 16 deletions

View File

@ -37,6 +37,7 @@ class ChannelException(val channelId: ByteVector32, message: String) extends Run
// @formatter:off
case class InvalidChainHash (override val channelId: ByteVector32, local: BlockHash, remote: BlockHash) extends ChannelException(channelId, s"invalid chainHash (local=$local remote=$remote)")
case class FundingFeerateTooLow (override val channelId: ByteVector32, proposed: FeeratePerKw, expected: FeeratePerKw) extends ChannelException(channelId, s"funding feerate is too low: expected at least $expected, but $proposed was proposed")
case class FundingAmountTooLow (override val channelId: ByteVector32, fundingAmount: Satoshi, min: Satoshi) extends ChannelException(channelId, s"invalid funding_amount=$fundingAmount (min=$min)")
case class FundingAmountTooHigh (override val channelId: ByteVector32, fundingAmount: Satoshi, max: Satoshi) extends ChannelException(channelId, s"invalid funding_amount=$fundingAmount (max=$max)")
case class InvalidFundingBalances (override val channelId: ByteVector32, fundingAmount: Satoshi, localBalance: MilliSatoshi, remoteBalance: MilliSatoshi) extends ChannelException(channelId, s"invalid balances funding_amount=$fundingAmount local=$localBalance remote=$remoteBalance")

View File

@ -44,7 +44,7 @@ import fr.acinq.eclair.remote.EclairInternalsSerializer.RemoteTypes
import fr.acinq.eclair.router.Router
import fr.acinq.eclair.wire.protocol
import fr.acinq.eclair.wire.protocol.FailureMessageCodecs.createBadOnionFailure
import fr.acinq.eclair.wire.protocol.{AddFeeCredit, ChannelTlv, CurrentFeeCredit, Error, HasChannelId, HasTemporaryChannelId, LightningMessage, LiquidityAds, NodeAddress, OnTheFlyFundingFailureMessage, OnionMessage, OnionRoutingPacket, RoutingMessage, SpliceInit, TlvStream, UnknownMessage, Warning, WillAddHtlc, WillFailHtlc, WillFailMalformedHtlc}
import fr.acinq.eclair.wire.protocol.{AddFeeCredit, ChannelTlv, CurrentFeeCredit, Error, HasChannelId, HasTemporaryChannelId, LightningMessage, LiquidityAds, NodeAddress, OnTheFlyFundingFailureMessage, OnionMessage, OnionRoutingPacket, RecommendedFeerates, RoutingMessage, SpliceInit, TlvStream, TxAbort, UnknownMessage, Warning, WillAddHtlc, WillFailHtlc, WillFailMalformedHtlc}
/**
* This actor represents a logical peer. There is one [[Peer]] per unique remote node id at all time.
@ -199,13 +199,17 @@ class Peer(val nodeParams: NodeParams,
case Event(open: protocol.OpenDualFundedChannel, d: ConnectedData) =>
d.channels.get(TemporaryChannelId(open.temporaryChannelId)) match {
case None if Features.canUseFeature(d.localFeatures, d.remoteFeatures, Features.DualFunding) =>
openChannelInterceptor ! OpenChannelNonInitiator(remoteNodeId, Right(open), d.localFeatures, d.remoteFeatures, d.peerConnection.toTyped, d.address)
stay()
case None =>
case None if !Features.canUseFeature(d.localFeatures, d.remoteFeatures, Features.DualFunding) =>
log.info("rejecting open_channel2: dual funding is not supported")
self ! Peer.OutgoingMessage(Error(open.temporaryChannelId, "dual funding is not supported"), d.peerConnection)
stay()
case None if open.usesOnTheFlyFunding && !d.fundingFeerateOk(open.fundingFeerate) =>
log.info("rejecting open_channel2: feerate too low ({} < {})", open.fundingFeerate, d.currentFeerates.fundingFeerate)
self ! Peer.OutgoingMessage(Error(open.temporaryChannelId, FundingFeerateTooLow(open.temporaryChannelId, open.fundingFeerate, d.currentFeerates.fundingFeerate).getMessage), d.peerConnection)
stay()
case None =>
openChannelInterceptor ! OpenChannelNonInitiator(remoteNodeId, Right(open), d.localFeatures, d.remoteFeatures, d.peerConnection.toTyped, d.address)
stay()
case Some(_) =>
log.warning("ignoring open_channel2 with duplicate temporaryChannelId={}", open.temporaryChannelId)
stay()
@ -379,6 +383,9 @@ class Peer(val nodeParams: NodeParams,
case Event(msg: SpliceInit, d: ConnectedData) =>
d.channels.get(FinalChannelId(msg.channelId)) match {
case Some(_) if msg.usesOnTheFlyFunding && !d.fundingFeerateOk(msg.feerate) =>
log.info("rejecting open_channel2: feerate too low ({} < {})", msg.feerate, d.currentFeerates.fundingFeerate)
self ! Peer.OutgoingMessage(TxAbort(msg.channelId, FundingFeerateTooLow(msg.channelId, msg.feerate, d.currentFeerates.fundingFeerate).getMessage), d.peerConnection)
case Some(channel) =>
OnTheFlyFunding.validateSplice(msg, nodeParams.channelConf.htlcMinimum, pendingOnTheFlyFunding, feeCredit.getOrElse(0 msat)) match {
case reject: OnTheFlyFunding.ValidationResult.Reject =>
@ -546,9 +553,18 @@ class Peer(val nodeParams: NodeParams,
case Event(_: CurrentFeerates, d) =>
d match {
case d: ConnectedData => d.peerConnection ! nodeParams.recommendedFeerates(remoteNodeId, d.localFeatures, d.remoteFeatures)
case _ => ()
case d: ConnectedData =>
val feerates = nodeParams.recommendedFeerates(remoteNodeId, d.localFeatures, d.remoteFeatures)
d.peerConnection ! feerates
// We keep our previous recommended feerate: if our peer is concurrently sending a message based on the
// previous feerates, we should accept it.
stay() using d.copy(currentFeerates = feerates, previousFeerates_opt = Some(d.currentFeerates))
case _ =>
stay()
}
case Event(msg: RecommendedFeerates, _) =>
log.info("our peer recommends the following feerates: funding={}, commitment={}", msg.fundingFeerate, msg.commitmentFeerate)
stay()
case Event(current: CurrentBlockHeight, d) =>
@ -730,7 +746,8 @@ class Peer(val nodeParams: NodeParams,
channels.values.toSet[ActorRef].foreach(_ ! INPUT_RECONNECTED(connectionReady.peerConnection, connectionReady.localInit, connectionReady.remoteInit)) // we deduplicate with toSet because there might be two entries per channel (tmp id and final id)
// We tell our peer what our current feerates are.
connectionReady.peerConnection ! nodeParams.recommendedFeerates(remoteNodeId, connectionReady.localInit.features, connectionReady.remoteInit.features)
val feerates = nodeParams.recommendedFeerates(remoteNodeId, connectionReady.localInit.features, connectionReady.remoteInit.features)
connectionReady.peerConnection ! feerates
if (Features.canUseFeature(connectionReady.localInit.features, connectionReady.remoteInit.features, Features.FundingFeeCredit)) {
if (feeCredit.isEmpty) {
@ -742,7 +759,7 @@ class Peer(val nodeParams: NodeParams,
connectionReady.peerConnection ! CurrentFeeCredit(nodeParams.chainHash, feeCredit.getOrElse(0 msat))
}
goto(CONNECTED) using ConnectedData(connectionReady.address, connectionReady.peerConnection, connectionReady.localInit, connectionReady.remoteInit, channels)
goto(CONNECTED) using ConnectedData(connectionReady.address, connectionReady.peerConnection, connectionReady.localInit, connectionReady.remoteInit, channels, feerates, None)
}
/**
@ -882,10 +899,12 @@ object Peer {
}
case object Nothing extends Data { override def channels = Map.empty }
case class DisconnectedData(channels: Map[FinalChannelId, ActorRef]) extends Data
case class ConnectedData(address: NodeAddress, peerConnection: ActorRef, localInit: protocol.Init, remoteInit: protocol.Init, channels: Map[ChannelId, ActorRef]) extends Data {
case class ConnectedData(address: NodeAddress, peerConnection: ActorRef, localInit: protocol.Init, remoteInit: protocol.Init, channels: Map[ChannelId, ActorRef], currentFeerates: RecommendedFeerates, previousFeerates_opt: Option[RecommendedFeerates]) extends Data {
val connectionInfo: ConnectionInfo = ConnectionInfo(address, peerConnection, localInit, remoteInit)
def localFeatures: Features[InitFeature] = localInit.features
def remoteFeatures: Features[InitFeature] = remoteInit.features
/** Returns true if the proposed feerate matches one of our recent feerate recommendations. */
def fundingFeerateOk(proposedFeerate: FeeratePerKw): Boolean = currentFeerates.fundingFeerate <= proposedFeerate || previousFeerates_opt.exists(_.fundingFeerate <= proposedFeerate)
}
sealed trait State

View File

@ -254,6 +254,7 @@ case class OpenDualFundedChannel(chainHash: BlockHash,
val channelType_opt: Option[ChannelType] = tlvStream.get[ChannelTlv.ChannelTypeTlv].map(_.channelType)
val requireConfirmedInputs: Boolean = tlvStream.get[ChannelTlv.RequireConfirmedInputsTlv].nonEmpty
val requestFunding_opt: Option[LiquidityAds.RequestFunding] = tlvStream.get[ChannelTlv.RequestFundingTlv].map(_.request)
val usesOnTheFlyFunding: Boolean = requestFunding_opt.exists(_.paymentDetails.paymentType.isInstanceOf[LiquidityAds.OnTheFlyFundingPaymentType])
val useFeeCredit_opt: Option[MilliSatoshi] = tlvStream.get[ChannelTlv.UseFeeCredit].map(_.amount)
val pushAmount: MilliSatoshi = tlvStream.get[ChannelTlv.PushAmountTlv].map(_.amount).getOrElse(0 msat)
}
@ -308,6 +309,7 @@ case class SpliceInit(channelId: ByteVector32,
tlvStream: TlvStream[SpliceInitTlv] = TlvStream.empty) extends ChannelMessage with HasChannelId {
val requireConfirmedInputs: Boolean = tlvStream.get[ChannelTlv.RequireConfirmedInputsTlv].nonEmpty
val requestFunding_opt: Option[LiquidityAds.RequestFunding] = tlvStream.get[ChannelTlv.RequestFundingTlv].map(_.request)
val usesOnTheFlyFunding: Boolean = requestFunding_opt.exists(_.paymentDetails.paymentType.isInstanceOf[LiquidityAds.OnTheFlyFundingPaymentType])
val useFeeCredit_opt: Option[MilliSatoshi] = tlvStream.get[ChannelTlv.UseFeeCredit].map(_.amount)
val pushAmount: MilliSatoshi = tlvStream.get[ChannelTlv.PushAmountTlv].map(_.amount).getOrElse(0 msat)
}

View File

@ -93,16 +93,19 @@ object LiquidityAds {
// @formatter:on
}
/** Payment type used for on-the-fly funding for incoming HTLCs. */
sealed trait OnTheFlyFundingPaymentType extends PaymentType
object PaymentType {
// @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" }
case object FromFutureHtlc extends OnTheFlyFundingPaymentType { 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" }
case object FromFutureHtlcWithPreimage extends OnTheFlyFundingPaymentType { override val rfcName: String = "from_future_htlc_with_preimage" }
/** Similar to [[FromChannelBalance]] but expects HTLCs to be relayed after funding. */
case object FromChannelBalanceForFutureHtlc extends PaymentType { override val rfcName: String = "from_channel_balance_for_future_htlc" }
case object FromChannelBalanceForFutureHtlc extends OnTheFlyFundingPaymentType { override val rfcName: String = "from_channel_balance_for_future_htlc" }
/** 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

View File

@ -348,6 +348,59 @@ class PeerSpec extends FixtureSpec {
))
}
test("reject funding requests if funding feerate is too low for on-the-fly funding", Tag(ChannelStateTestsTags.DualFunding), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>
import f._
connect(remoteNodeId, peer, peerConnection, switchboard, remoteInit = protocol.Init(Features(StaticRemoteKey -> Optional, AnchorOutputsZeroFeeHtlcTx -> Optional, DualFunding -> Optional)))
val requestFunds = LiquidityAds.RequestFunding(50_000 sat, LiquidityAds.FundingRate(10_000 sat, 100_000 sat, 0, 0, 0 sat, 0 sat), LiquidityAds.PaymentDetails.FromFutureHtlc(randomBytes32() :: Nil))
val open = {
val open = createOpenDualFundedChannelMessage()
open.copy(fundingFeerate = FeeratePerKw(5000 sat), tlvStream = TlvStream(ChannelTlv.RequestFundingTlv(requestFunds)))
}
// Our current and previous feerates are higher than what will be proposed.
Seq(FeeratePerKw(7500 sat), FeeratePerKw(6000 sat)).foreach(feerate => {
val feerates = FeeratesPerKw.single(feerate)
nodeParams.setBitcoinCoreFeerates(feerates)
peer ! CurrentFeerates.BitcoinCore(feerates)
assert(peerConnection.expectMsgType[RecommendedFeerates].fundingFeerate == feerate)
})
// The channel request is rejected.
peerConnection.send(peer, open)
peerConnection.expectMsg(Error(open.temporaryChannelId, FundingFeerateTooLow(open.temporaryChannelId, FeeratePerKw(5000 sat), FeeratePerKw(6000 sat)).getMessage))
// Our latest feerate matches the channel request.
val feerates3 = FeeratesPerKw.single(FeeratePerKw(5000 sat))
nodeParams.setBitcoinCoreFeerates(feerates3)
peer ! CurrentFeerates.BitcoinCore(feerates3)
assert(peerConnection.expectMsgType[RecommendedFeerates].fundingFeerate == FeeratePerKw(5000 sat))
// The channel request is accepted.
peerConnection.send(peer, open)
channel.expectMsgType[INPUT_INIT_CHANNEL_NON_INITIATOR]
channel.expectMsg(open)
val channelId = randomBytes32()
peer ! ChannelIdAssigned(channel.ref, remoteNodeId, open.temporaryChannelId, channelId)
peerConnection.expectMsgType[PeerConnection.DoSync]
// The feerate of the splice request is lower than our last two feerates.
val splice = SpliceInit(channelId, 100_000 sat, FeeratePerKw(4500 sat), 0, randomKey().publicKey, TlvStream(ChannelTlv.RequestFundingTlv(requestFunds)))
peerConnection.send(peer, splice)
peerConnection.expectMsg(TxAbort(channelId, FundingFeerateTooLow(channelId, FeeratePerKw(4500 sat), FeeratePerKw(5000 sat)).getMessage))
// Our latest feerate matches the splice request.
val feerates4 = FeeratesPerKw.single(FeeratePerKw(4000 sat))
nodeParams.setBitcoinCoreFeerates(feerates4)
peer ! CurrentFeerates.BitcoinCore(feerates4)
assert(peerConnection.expectMsgType[RecommendedFeerates].fundingFeerate == FeeratePerKw(4000 sat))
// The splice is accepted.
peerConnection.send(peer, splice)
channel.expectMsg(splice)
}
test("don't spawn a channel with duplicate temporary channel id", Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>
import f._

View File

@ -17,28 +17,29 @@
package fr.acinq.eclair.io
import akka.testkit.{TestFSMRef, TestProbe}
import fr.acinq.bitcoin.scalacompat.Block
import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
import fr.acinq.eclair._
import fr.acinq.eclair.io.Peer.ChannelId
import fr.acinq.eclair.io.ReconnectionTask.WaitingData
import fr.acinq.eclair.tor.Socks5ProxyParams
import fr.acinq.eclair.wire.protocol.{Color, IPv4, NodeAddress, NodeAnnouncement}
import fr.acinq.eclair.wire.protocol.{Color, NodeAddress, NodeAnnouncement, RecommendedFeerates}
import org.mockito.IdiomaticMockito.StubbingOps
import org.mockito.MockitoSugar.mock
import org.scalatest.funsuite.FixtureAnyFunSuiteLike
import org.scalatest.{Outcome, ParallelTestExecution, Tag}
import java.net.Inet4Address
import scala.concurrent.duration._
class ReconnectionTaskSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with ParallelTestExecution {
private val fakeIPAddress = NodeAddress.fromParts("1.2.3.4", 42000).get
private val channels = Map(Peer.FinalChannelId(randomBytes32()) -> system.deadLetters)
private val recommendedFeerates = RecommendedFeerates(Block.RegtestGenesisBlock.hash, TestConstants.feeratePerKw, TestConstants.anchorOutputsFeeratePerKw)
private val PeerNothingData = Peer.Nothing
private val PeerDisconnectedData = Peer.DisconnectedData(channels)
private val PeerConnectedData = Peer.ConnectedData(fakeIPAddress, system.deadLetters, null, null, channels.map { case (k: ChannelId, v) => (k, v) })
private val PeerConnectedData = Peer.ConnectedData(fakeIPAddress, system.deadLetters, null, null, channels.map { case (k: ChannelId, v) => (k, v) }, recommendedFeerates, None)
case class FixtureParam(nodeParams: NodeParams, remoteNodeId: PublicKey, reconnectionTask: TestFSMRef[ReconnectionTask.State, ReconnectionTask.Data, ReconnectionTask], monitor: TestProbe)