mirror of
https://github.com/ACINQ/eclair.git
synced 2024-11-19 01:43:22 +01:00
More fine grained support for fee diff errors (#2815)
When there is a mismatch between the feerate of a channel and the feerate we get from our estimator, we may want to force-close because that could be exploited by our peer to steal HTLCs. But that's only the case if the feerate is too low, not if it's too high. We previously force-closed in both cases, whereas we only need to do it when the feerate is too low. This should avoid some unnecessary force-close that we've observed and are due to buggy fee estimators (fee estimation is hard!), or to peers who simply do some smoothing and slightly delay lowering the feerate of our channels.
This commit is contained in:
parent
5d6a1db9fb
commit
36a3c8897c
@ -20,7 +20,7 @@ import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
|
||||
import fr.acinq.bitcoin.scalacompat.Satoshi
|
||||
import fr.acinq.eclair.BlockHeight
|
||||
import fr.acinq.eclair.transactions.Transactions
|
||||
import fr.acinq.eclair.transactions.Transactions.{CommitmentFormat, DefaultCommitmentFormat, UnsafeLegacyAnchorOutputsCommitmentFormat, ZeroFeeHtlcTxAnchorOutputsCommitmentFormat}
|
||||
import fr.acinq.eclair.transactions.Transactions.{CommitmentFormat, UnsafeLegacyAnchorOutputsCommitmentFormat, ZeroFeeHtlcTxAnchorOutputsCommitmentFormat}
|
||||
|
||||
// @formatter:off
|
||||
sealed trait ConfirmationPriority extends Ordered[ConfirmationPriority] {
|
||||
@ -71,12 +71,21 @@ case class FeerateTolerance(ratioLow: Double, ratioHigh: Double, anchorOutputMax
|
||||
* @return true if the difference between proposed and reference fee rates is too high.
|
||||
*/
|
||||
def isFeeDiffTooHigh(commitmentFormat: CommitmentFormat, networkFeerate: FeeratePerKw, proposedFeerate: FeeratePerKw): Boolean = {
|
||||
isProposedFeerateTooLow(commitmentFormat, networkFeerate, proposedFeerate) || isProposedFeerateTooHigh(commitmentFormat, networkFeerate, proposedFeerate)
|
||||
}
|
||||
|
||||
def isProposedFeerateTooHigh(commitmentFormat: CommitmentFormat, networkFeerate: FeeratePerKw, proposedFeerate: FeeratePerKw): Boolean = {
|
||||
commitmentFormat match {
|
||||
case DefaultCommitmentFormat =>
|
||||
proposedFeerate < networkFeerate * ratioLow || networkFeerate * ratioHigh < proposedFeerate
|
||||
case ZeroFeeHtlcTxAnchorOutputsCommitmentFormat | UnsafeLegacyAnchorOutputsCommitmentFormat =>
|
||||
// when using anchor outputs, we allow any feerate: fees will be set with CPFP and RBF at broadcast time
|
||||
false
|
||||
case Transactions.DefaultCommitmentFormat => networkFeerate * ratioHigh < proposedFeerate
|
||||
case ZeroFeeHtlcTxAnchorOutputsCommitmentFormat | UnsafeLegacyAnchorOutputsCommitmentFormat => networkFeerate * ratioHigh < proposedFeerate
|
||||
}
|
||||
}
|
||||
|
||||
def isProposedFeerateTooLow(commitmentFormat: CommitmentFormat, networkFeerate: FeeratePerKw, proposedFeerate: FeeratePerKw): Boolean = {
|
||||
commitmentFormat match {
|
||||
case Transactions.DefaultCommitmentFormat => proposedFeerate < networkFeerate * ratioLow
|
||||
// When using anchor outputs, we allow low feerates: fees will be set with CPFP and RBF at broadcast time.
|
||||
case ZeroFeeHtlcTxAnchorOutputsCommitmentFormat | UnsafeLegacyAnchorOutputsCommitmentFormat => false
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -87,7 +96,7 @@ case class OnChainFeeConf(feeTargets: FeeTargets,
|
||||
anchorWithoutHtlcsMaxFee: Satoshi,
|
||||
closeOnOfflineMismatch: Boolean,
|
||||
updateFeeMinDiffRatio: Double,
|
||||
private val defaultFeerateTolerance: FeerateTolerance,
|
||||
defaultFeerateTolerance: FeerateTolerance,
|
||||
private val perNodeFeerateTolerance: Map[PublicKey, FeerateTolerance]) {
|
||||
|
||||
def feerateToleranceFor(nodeId: PublicKey): FeerateTolerance = perNodeFeerateTolerance.getOrElse(nodeId, defaultFeerateTolerance)
|
||||
@ -103,8 +112,8 @@ case class OnChainFeeConf(feeTargets: FeeTargets,
|
||||
* - if we're using anchor outputs, we use a feerate that allows network propagation of the commit tx: we will use CPFP to speed up confirmation if needed
|
||||
* - otherwise we use a feerate that should get the commit tx confirmed within the configured block target
|
||||
*
|
||||
* @param remoteNodeId nodeId of our channel peer
|
||||
* @param commitmentFormat commitment format
|
||||
* @param remoteNodeId nodeId of our channel peer
|
||||
* @param commitmentFormat commitment format
|
||||
*/
|
||||
def getCommitmentFeerate(feerates: FeeratesPerKw, remoteNodeId: PublicKey, commitmentFormat: CommitmentFormat, channelCapacity: Satoshi): FeeratePerKw = {
|
||||
val networkFeerate = feerates.fast
|
||||
|
@ -433,10 +433,12 @@ case class Commitment(fundingTxIndex: Long,
|
||||
// we allowed mismatches between our feerates and our remote's as long as commitments didn't contain any HTLC at risk
|
||||
// we need to verify that we're not disagreeing on feerates anymore before offering new HTLCs
|
||||
// NB: there may be a pending update_fee that hasn't been applied yet that needs to be taken into account
|
||||
val localFeeratePerKw = feeConf.getCommitmentFeerate(feerates, params.remoteNodeId, params.commitmentFormat, capacity)
|
||||
val remoteFeeratePerKw = localCommit.spec.commitTxFeerate +: changes.remoteChanges.all.collect { case f: UpdateFee => f.feeratePerKw }
|
||||
remoteFeeratePerKw.find(feerate => feeConf.feerateToleranceFor(params.remoteNodeId).isFeeDiffTooHigh(params.commitmentFormat, localFeeratePerKw, feerate)) match {
|
||||
case Some(feerate) => return Left(FeerateTooDifferent(params.channelId, localFeeratePerKw = localFeeratePerKw, remoteFeeratePerKw = feerate))
|
||||
val localFeerate = feeConf.getCommitmentFeerate(feerates, params.remoteNodeId, params.commitmentFormat, capacity)
|
||||
val remoteFeerate = localCommit.spec.commitTxFeerate +: changes.remoteChanges.all.collect { case f: UpdateFee => f.feeratePerKw }
|
||||
// What we want to avoid is having an HTLC in a commitment transaction that has a very low feerate, which we won't
|
||||
// be able to confirm in time to claim the HTLC, so we only need to check that the feerate isn't too low.
|
||||
remoteFeerate.find(feerate => feeConf.feerateToleranceFor(params.remoteNodeId).isProposedFeerateTooLow(params.commitmentFormat, localFeerate, feerate)) match {
|
||||
case Some(feerate) => return Left(FeerateTooDifferent(params.channelId, localFeeratePerKw = localFeerate, remoteFeeratePerKw = feerate))
|
||||
case None =>
|
||||
}
|
||||
|
||||
@ -510,10 +512,10 @@ case class Commitment(fundingTxIndex: Long,
|
||||
// we allowed mismatches between our feerates and our remote's as long as commitments didn't contain any HTLC at risk
|
||||
// we need to verify that we're not disagreeing on feerates anymore before accepting new HTLCs
|
||||
// NB: there may be a pending update_fee that hasn't been applied yet that needs to be taken into account
|
||||
val localFeeratePerKw = feeConf.getCommitmentFeerate(feerates, params.remoteNodeId, params.commitmentFormat, capacity)
|
||||
val remoteFeeratePerKw = localCommit.spec.commitTxFeerate +: changes.remoteChanges.all.collect { case f: UpdateFee => f.feeratePerKw }
|
||||
remoteFeeratePerKw.find(feerate => feeConf.feerateToleranceFor(params.remoteNodeId).isFeeDiffTooHigh(params.commitmentFormat, localFeeratePerKw, feerate)) match {
|
||||
case Some(feerate) => return Left(FeerateTooDifferent(params.channelId, localFeeratePerKw = localFeeratePerKw, remoteFeeratePerKw = feerate))
|
||||
val localFeerate = feeConf.getCommitmentFeerate(feerates, params.remoteNodeId, params.commitmentFormat, capacity)
|
||||
val remoteFeerate = localCommit.spec.commitTxFeerate +: changes.remoteChanges.all.collect { case f: UpdateFee => f.feeratePerKw }
|
||||
remoteFeerate.find(feerate => feeConf.feerateToleranceFor(params.remoteNodeId).isProposedFeerateTooLow(params.commitmentFormat, localFeerate, feerate)) match {
|
||||
case Some(feerate) => return Left(FeerateTooDifferent(params.channelId, localFeeratePerKw = localFeerate, remoteFeeratePerKw = feerate))
|
||||
case None =>
|
||||
}
|
||||
|
||||
@ -583,9 +585,12 @@ case class Commitment(fundingTxIndex: Long,
|
||||
}
|
||||
|
||||
def canReceiveFee(targetFeerate: FeeratePerKw, params: ChannelParams, changes: CommitmentChanges, feerates: FeeratesPerKw, feeConf: OnChainFeeConf): Either[ChannelException, Unit] = {
|
||||
val localFeeratePerKw = feeConf.getCommitmentFeerate(feerates, params.remoteNodeId, params.commitmentFormat, capacity)
|
||||
if (feeConf.feerateToleranceFor(params.remoteNodeId).isFeeDiffTooHigh(params.commitmentFormat, localFeeratePerKw, targetFeerate) && hasPendingOrProposedHtlcs(changes)) {
|
||||
return Left(FeerateTooDifferent(params.channelId, localFeeratePerKw = localFeeratePerKw, remoteFeeratePerKw = targetFeerate))
|
||||
val localFeerate = feeConf.getCommitmentFeerate(feerates, params.remoteNodeId, params.commitmentFormat, capacity)
|
||||
if (feeConf.feerateToleranceFor(params.remoteNodeId).isProposedFeerateTooHigh(params.commitmentFormat, localFeerate, targetFeerate)) {
|
||||
return Left(FeerateTooDifferent(params.channelId, localFeeratePerKw = localFeerate, remoteFeeratePerKw = targetFeerate))
|
||||
} else if (feeConf.feerateToleranceFor(params.remoteNodeId).isProposedFeerateTooLow(params.commitmentFormat, localFeerate, targetFeerate) && hasPendingOrProposedHtlcs(changes)) {
|
||||
// If the proposed feerate is too low, but we don't have any pending HTLC, we temporarily accept it.
|
||||
return Left(FeerateTooDifferent(params.channelId, localFeeratePerKw = localFeerate, remoteFeeratePerKw = targetFeerate))
|
||||
} else {
|
||||
// let's compute the current commitment *as seen by us* including this change
|
||||
// NB: we check that the initiator can afford this new fee even if spec allows to do it at next signature
|
||||
|
@ -2542,7 +2542,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
|
||||
val currentFeeratePerKw = commitments.localCommit.spec.commitTxFeerate
|
||||
val shouldUpdateFee = d.commitments.params.localParams.isInitiator && nodeParams.onChainFeeConf.shouldUpdateFee(currentFeeratePerKw, networkFeeratePerKw)
|
||||
val shouldClose = !d.commitments.params.localParams.isInitiator &&
|
||||
nodeParams.onChainFeeConf.feerateToleranceFor(d.commitments.remoteNodeId).isFeeDiffTooHigh(d.commitments.params.commitmentFormat, networkFeeratePerKw, currentFeeratePerKw) &&
|
||||
nodeParams.onChainFeeConf.feerateToleranceFor(d.commitments.remoteNodeId).isProposedFeerateTooLow(d.commitments.params.commitmentFormat, networkFeeratePerKw, currentFeeratePerKw) &&
|
||||
d.commitments.hasPendingOrProposedHtlcs // we close only if we have HTLCs potentially at risk
|
||||
if (shouldUpdateFee) {
|
||||
self ! CMD_UPDATE_FEE(networkFeeratePerKw, commit = true)
|
||||
@ -2568,7 +2568,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
|
||||
val currentFeeratePerKw = commitments.localCommit.spec.commitTxFeerate
|
||||
// if the network fees are too high we risk to not be able to confirm our current commitment
|
||||
val shouldClose = networkFeeratePerKw > currentFeeratePerKw &&
|
||||
nodeParams.onChainFeeConf.feerateToleranceFor(d.commitments.remoteNodeId).isFeeDiffTooHigh(d.commitments.params.commitmentFormat, networkFeeratePerKw, currentFeeratePerKw) &&
|
||||
nodeParams.onChainFeeConf.feerateToleranceFor(d.commitments.remoteNodeId).isProposedFeerateTooLow(d.commitments.params.commitmentFormat, networkFeeratePerKw, currentFeeratePerKw) &&
|
||||
d.commitments.hasPendingOrProposedHtlcs // we close only if we have HTLCs potentially at risk
|
||||
if (shouldClose) {
|
||||
if (nodeParams.onChainFeeConf.closeOnOfflineMismatch) {
|
||||
|
@ -17,7 +17,6 @@
|
||||
package fr.acinq.eclair.blockchain.fee
|
||||
|
||||
import fr.acinq.bitcoin.scalacompat.SatoshiLong
|
||||
import fr.acinq.eclair.channel.ChannelTypes
|
||||
import fr.acinq.eclair.randomKey
|
||||
import fr.acinq.eclair.transactions.Transactions.{DefaultCommitmentFormat, UnsafeLegacyAnchorOutputsCommitmentFormat, ZeroFeeHtlcTxAnchorOutputsCommitmentFormat}
|
||||
import org.scalatest.funsuite.AnyFunSuite
|
||||
@ -87,7 +86,6 @@ class OnChainFeeConfSpec extends AnyFunSuite {
|
||||
|
||||
test("fee difference too high") {
|
||||
val tolerance = FeerateTolerance(ratioLow = 0.5, ratioHigh = 4.0, anchorOutputMaxCommitFeerate = FeeratePerKw(2500 sat), DustTolerance(25000 sat, closeOnUpdateFeeOverflow = false))
|
||||
val channelType = ChannelTypes.Standard()
|
||||
val testCases = Seq(
|
||||
(FeeratePerKw(500 sat), FeeratePerKw(500 sat), false),
|
||||
(FeeratePerKw(500 sat), FeeratePerKw(250 sat), false),
|
||||
@ -107,21 +105,19 @@ class OnChainFeeConfSpec extends AnyFunSuite {
|
||||
test("fee difference too high (anchor outputs)") {
|
||||
val tolerance = FeerateTolerance(ratioLow = 0.5, ratioHigh = 4.0, anchorOutputMaxCommitFeerate = FeeratePerKw(2500 sat), DustTolerance(25000 sat, closeOnUpdateFeeOverflow = false))
|
||||
val testCases = Seq(
|
||||
(FeeratePerKw(500 sat), FeeratePerKw(500 sat)),
|
||||
(FeeratePerKw(500 sat), FeeratePerKw(2500 sat)),
|
||||
(FeeratePerKw(500 sat), FeeratePerKw(10000 sat)),
|
||||
(FeeratePerKw(500 sat), FeeratePerKw(10001 sat)),
|
||||
(FeeratePerKw(2500 sat), FeeratePerKw(10000 sat)),
|
||||
(FeeratePerKw(2500 sat), FeeratePerKw(10001 sat)),
|
||||
(FeeratePerKw(2500 sat), FeeratePerKw(1250 sat)),
|
||||
(FeeratePerKw(2500 sat), FeeratePerKw(1249 sat)),
|
||||
(FeeratePerKw(2500 sat), FeeratePerKw(1000 sat)),
|
||||
(FeeratePerKw(1000 sat), FeeratePerKw(500 sat)),
|
||||
(FeeratePerKw(1000 sat), FeeratePerKw(499 sat)),
|
||||
(FeeratePerKw(500 sat), FeeratePerKw(500 sat), false),
|
||||
(FeeratePerKw(500 sat), FeeratePerKw(1000 sat), false),
|
||||
(FeeratePerKw(500 sat), FeeratePerKw(2000 sat), false),
|
||||
(FeeratePerKw(500 sat), FeeratePerKw(2001 sat), true),
|
||||
(FeeratePerKw(2500 sat), FeeratePerKw(10000 sat), false),
|
||||
(FeeratePerKw(2500 sat), FeeratePerKw(10001 sat), true),
|
||||
(FeeratePerKw(2500 sat), FeeratePerKw(1250 sat), false),
|
||||
(FeeratePerKw(2500 sat), FeeratePerKw(1000 sat), false),
|
||||
(FeeratePerKw(1000 sat), FeeratePerKw(500 sat), false),
|
||||
)
|
||||
testCases.foreach { case (networkFeerate, proposedFeerate) =>
|
||||
assert(!tolerance.isFeeDiffTooHigh(UnsafeLegacyAnchorOutputsCommitmentFormat, networkFeerate, proposedFeerate))
|
||||
assert(!tolerance.isFeeDiffTooHigh(ZeroFeeHtlcTxAnchorOutputsCommitmentFormat, networkFeerate, proposedFeerate))
|
||||
testCases.foreach { case (networkFeerate, proposedFeerate, expected) =>
|
||||
assert(tolerance.isFeeDiffTooHigh(UnsafeLegacyAnchorOutputsCommitmentFormat, networkFeerate, proposedFeerate) == expected)
|
||||
assert(tolerance.isFeeDiffTooHigh(ZeroFeeHtlcTxAnchorOutputsCommitmentFormat, networkFeerate, proposedFeerate) == expected)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -78,6 +78,8 @@ object ChannelStateTestsTags {
|
||||
val HighDustLimitDifferenceAliceBob = "high_dust_limit_difference_alice_bob"
|
||||
/** If set, Bob will have a much higher dust limit than Alice. */
|
||||
val HighDustLimitDifferenceBobAlice = "high_dust_limit_difference_bob_alice"
|
||||
/** If set, Alice and Bob will use a very large tolerance for feerate mismatch. */
|
||||
val HighFeerateMismatchTolerance = "high_feerate_mismatch_tolerance"
|
||||
/** If set, channels will use option_channel_type. */
|
||||
val ChannelType = "option_channel_type"
|
||||
/** If set, channels will use option_zeroconf. */
|
||||
@ -145,6 +147,8 @@ trait ChannelStateTestsBase extends Assertions with Eventually {
|
||||
.modify(_.channelConf.dustLimit).setToIf(tags.contains(ChannelStateTestsTags.HighDustLimitDifferenceBobAlice))(1000 sat)
|
||||
.modify(_.channelConf.maxRemoteDustLimit).setToIf(tags.contains(ChannelStateTestsTags.HighDustLimitDifferenceAliceBob))(10000 sat)
|
||||
.modify(_.channelConf.maxRemoteDustLimit).setToIf(tags.contains(ChannelStateTestsTags.HighDustLimitDifferenceBobAlice))(10000 sat)
|
||||
.modify(_.onChainFeeConf.defaultFeerateTolerance.ratioLow).setToIf(tags.contains(ChannelStateTestsTags.HighFeerateMismatchTolerance))(0.000001)
|
||||
.modify(_.onChainFeeConf.defaultFeerateTolerance.ratioHigh).setToIf(tags.contains(ChannelStateTestsTags.HighFeerateMismatchTolerance))(1000000)
|
||||
.modify(_.onChainFeeConf.spendAnchorWithoutHtlcs).setToIf(tags.contains(ChannelStateTestsTags.DontSpendAnchorWithoutHtlcs))(false)
|
||||
.modify(_.channelConf.balanceThresholds).setToIf(tags.contains(ChannelStateTestsTags.AdaptMaxHtlcAmount))(Seq(Channel.BalanceThreshold(1_000 sat, 0 sat), Channel.BalanceThreshold(5_000 sat, 1_000 sat), Channel.BalanceThreshold(10_000 sat, 5_000 sat)))
|
||||
val finalNodeParamsB = nodeParamsB
|
||||
@ -154,6 +158,8 @@ trait ChannelStateTestsBase extends Assertions with Eventually {
|
||||
.modify(_.channelConf.maxRemoteDustLimit).setToIf(tags.contains(ChannelStateTestsTags.HighDustLimitDifferenceBobAlice))(10000 sat)
|
||||
.modify(_.channelConf.remoteRbfLimits.maxAttempts).setToIf(tags.contains(ChannelStateTestsTags.RejectRbfAttempts))(0)
|
||||
.modify(_.channelConf.remoteRbfLimits.attemptDeltaBlocks).setToIf(tags.contains(ChannelStateTestsTags.DelayRbfAttempts))(1)
|
||||
.modify(_.onChainFeeConf.defaultFeerateTolerance.ratioLow).setToIf(tags.contains(ChannelStateTestsTags.HighFeerateMismatchTolerance))(0.000001)
|
||||
.modify(_.onChainFeeConf.defaultFeerateTolerance.ratioHigh).setToIf(tags.contains(ChannelStateTestsTags.HighFeerateMismatchTolerance))(1000000)
|
||||
.modify(_.onChainFeeConf.spendAnchorWithoutHtlcs).setToIf(tags.contains(ChannelStateTestsTags.DontSpendAnchorWithoutHtlcs))(false)
|
||||
.modify(_.channelConf.balanceThresholds).setToIf(tags.contains(ChannelStateTestsTags.AdaptMaxHtlcAmount))(Seq(Channel.BalanceThreshold(1_000 sat, 0 sat), Channel.BalanceThreshold(5_000 sat, 1_000 sat), Channel.BalanceThreshold(10_000 sat, 5_000 sat)))
|
||||
val wallet = wallet_opt match {
|
||||
|
@ -29,6 +29,7 @@ import fr.acinq.eclair.channel.fund.InteractiveTxBuilder.{FullySignedSharedTrans
|
||||
import fr.acinq.eclair.channel.publish.TxPublisher
|
||||
import fr.acinq.eclair.channel.states.{ChannelStateTestsBase, ChannelStateTestsTags}
|
||||
import fr.acinq.eclair.io.Peer.OpenChannelResponse
|
||||
import fr.acinq.eclair.transactions.Transactions
|
||||
import fr.acinq.eclair.wire.protocol._
|
||||
import fr.acinq.eclair.{Features, MilliSatoshiLong, TestConstants, TestKitBaseClass, ToMilliSatoshiConversion}
|
||||
import org.scalatest.funsuite.FixtureAnyFunSuiteLike
|
||||
@ -51,12 +52,16 @@ class WaitForDualFundingSignedStateSpec extends TestKitBaseClass with FixtureAny
|
||||
val bobInit = Init(bobParams.initFeatures)
|
||||
val bobContribution = if (channelType.features.contains(Features.ZeroConf)) None else Some(TestConstants.nonInitiatorFundingSatoshis)
|
||||
val (initiatorPushAmount, nonInitiatorPushAmount) = if (test.tags.contains("both_push_amount")) (Some(TestConstants.initiatorPushAmount), Some(TestConstants.nonInitiatorPushAmount)) else (None, None)
|
||||
val commitFeerate = channelType.commitmentFormat match {
|
||||
case Transactions.DefaultCommitmentFormat => TestConstants.feeratePerKw
|
||||
case _: Transactions.AnchorOutputsCommitmentFormat => TestConstants.anchorOutputsFeeratePerKw
|
||||
}
|
||||
val aliceListener = TestProbe()
|
||||
val bobListener = TestProbe()
|
||||
within(30 seconds) {
|
||||
alice.underlying.system.eventStream.subscribe(aliceListener.ref, classOf[ChannelAborted])
|
||||
bob.underlying.system.eventStream.subscribe(bobListener.ref, classOf[ChannelAborted])
|
||||
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = true, TestConstants.feeratePerKw, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, initiatorPushAmount, requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped)
|
||||
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = true, commitFeerate, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, initiatorPushAmount, requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped)
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, bobContribution, dualFunded = true, nonInitiatorPushAmount, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
|
||||
alice2blockchain.expectMsgType[TxPublisher.SetChannelId] // temporary channel id
|
||||
bob2blockchain.expectMsgType[TxPublisher.SetChannelId] // temporary channel id
|
||||
|
@ -28,6 +28,7 @@ import fr.acinq.eclair.channel.fsm.Channel.TickChannelOpenTimeout
|
||||
import fr.acinq.eclair.channel.publish.TxPublisher
|
||||
import fr.acinq.eclair.channel.states.{ChannelStateTestsBase, ChannelStateTestsTags}
|
||||
import fr.acinq.eclair.io.Peer.OpenChannelResponse
|
||||
import fr.acinq.eclair.transactions.Transactions
|
||||
import fr.acinq.eclair.wire.protocol.{AcceptChannel, Error, FundingCreated, FundingSigned, Init, OpenChannel}
|
||||
import fr.acinq.eclair.{TestConstants, TestKitBaseClass}
|
||||
import org.scalatest.funsuite.FixtureAnyFunSuiteLike
|
||||
@ -58,12 +59,16 @@ class WaitForFundingSignedStateSpec extends TestKitBaseClass with FixtureAnyFunS
|
||||
val channelConfig = ChannelConfig.standard
|
||||
val channelFlags = ChannelFlags.Private
|
||||
val (aliceParams, bobParams, channelType) = computeFeatures(setup, test.tags, channelFlags)
|
||||
val commitFeerate = channelType.commitmentFormat match {
|
||||
case Transactions.DefaultCommitmentFormat => TestConstants.feeratePerKw
|
||||
case _: Transactions.AnchorOutputsCommitmentFormat => TestConstants.anchorOutputsFeeratePerKw
|
||||
}
|
||||
val aliceInit = Init(aliceParams.initFeatures)
|
||||
val bobInit = Init(bobParams.initFeatures)
|
||||
val listener = TestProbe()
|
||||
within(30 seconds) {
|
||||
alice.underlying.system.eventStream.subscribe(listener.ref, classOf[ChannelAborted])
|
||||
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, fundingSatoshis, dualFunded = false, TestConstants.feeratePerKw, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, Some(pushMsat), requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped)
|
||||
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, fundingSatoshis, dualFunded = false, commitFeerate, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, Some(pushMsat), requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped)
|
||||
alice2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, None, dualFunded = false, None, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
|
||||
bob2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
|
@ -27,6 +27,7 @@ import fr.acinq.eclair.channel.publish.TxPublisher
|
||||
import fr.acinq.eclair.channel.states.{ChannelStateTestsBase, ChannelStateTestsTags}
|
||||
import fr.acinq.eclair.payment.relay.Relayer.RelayFees
|
||||
import fr.acinq.eclair.router.Announcements
|
||||
import fr.acinq.eclair.transactions.Transactions
|
||||
import fr.acinq.eclair.wire.protocol._
|
||||
import fr.acinq.eclair.{BlockHeight, MilliSatoshiLong, TestConstants, TestKitBaseClass}
|
||||
import org.scalatest.funsuite.FixtureAnyFunSuiteLike
|
||||
@ -50,6 +51,10 @@ class WaitForChannelReadyStateSpec extends TestKitBaseClass with FixtureAnyFunSu
|
||||
val channelConfig = ChannelConfig.standard
|
||||
val channelFlags = ChannelFlags(announceChannel = test.tags.contains(ChannelStateTestsTags.ChannelsPublic))
|
||||
val (aliceParams, bobParams, channelType) = computeFeatures(setup, test.tags, channelFlags)
|
||||
val commitFeerate = channelType.commitmentFormat match {
|
||||
case Transactions.DefaultCommitmentFormat => TestConstants.feeratePerKw
|
||||
case _: Transactions.AnchorOutputsCommitmentFormat => TestConstants.anchorOutputsFeeratePerKw
|
||||
}
|
||||
val pushMsat = if (test.tags.contains(ChannelStateTestsTags.NoPushAmount)) None else Some(TestConstants.initiatorPushAmount)
|
||||
val aliceInit = Init(aliceParams.initFeatures)
|
||||
val bobInit = Init(bobParams.initFeatures)
|
||||
@ -60,7 +65,7 @@ class WaitForChannelReadyStateSpec extends TestKitBaseClass with FixtureAnyFunSu
|
||||
alice.underlying.system.eventStream.subscribe(aliceListener.ref, classOf[ChannelAborted])
|
||||
bob.underlying.system.eventStream.subscribe(bobListener.ref, classOf[ChannelAborted])
|
||||
alice.underlyingActor.nodeParams.db.peers.addOrUpdateRelayFees(bobParams.nodeId, relayFees)
|
||||
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = false, TestConstants.feeratePerKw, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, pushMsat, requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped)
|
||||
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = false, commitFeerate, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, pushMsat, requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped)
|
||||
alice2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, None, dualFunded = false, None, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
|
||||
bob2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
|
@ -30,6 +30,7 @@ import fr.acinq.eclair.channel.publish.TxPublisher
|
||||
import fr.acinq.eclair.channel.publish.TxPublisher.{PublishFinalTx, SetChannelId}
|
||||
import fr.acinq.eclair.channel.states.ChannelStateTestsBase.FakeTxPublisherFactory
|
||||
import fr.acinq.eclair.channel.states.{ChannelStateTestsBase, ChannelStateTestsTags}
|
||||
import fr.acinq.eclair.transactions.Transactions
|
||||
import fr.acinq.eclair.wire.protocol._
|
||||
import fr.acinq.eclair.{BlockHeight, TestConstants, TestKitBaseClass, ToMilliSatoshiConversion}
|
||||
import org.scalatest.funsuite.FixtureAnyFunSuiteLike
|
||||
@ -62,12 +63,16 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture
|
||||
val channelConfig = ChannelConfig.standard
|
||||
val channelFlags = ChannelFlags.Private
|
||||
val (aliceParams, bobParams, channelType) = computeFeatures(setup, test.tags, channelFlags)
|
||||
val commitFeerate = channelType.commitmentFormat match {
|
||||
case Transactions.DefaultCommitmentFormat => TestConstants.feeratePerKw
|
||||
case _: Transactions.AnchorOutputsCommitmentFormat => TestConstants.anchorOutputsFeeratePerKw
|
||||
}
|
||||
val aliceInit = Init(aliceParams.initFeatures)
|
||||
val bobInit = Init(bobParams.initFeatures)
|
||||
val bobContribution = if (test.tags.contains("no-funding-contribution")) None else Some(TestConstants.nonInitiatorFundingSatoshis)
|
||||
val (initiatorPushAmount, nonInitiatorPushAmount) = if (test.tags.contains("both_push_amount")) (Some(TestConstants.initiatorPushAmount), Some(TestConstants.nonInitiatorPushAmount)) else (None, None)
|
||||
within(30 seconds) {
|
||||
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = true, TestConstants.feeratePerKw, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, initiatorPushAmount, requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped)
|
||||
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = true, commitFeerate, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, initiatorPushAmount, requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped)
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, bobContribution, dualFunded = true, nonInitiatorPushAmount, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
|
||||
alice2blockchain.expectMsgType[SetChannelId] // temporary channel id
|
||||
bob2blockchain.expectMsgType[SetChannelId] // temporary channel id
|
||||
|
@ -2248,7 +2248,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
bob2blockchain.expectMsgType[WatchTxConfirmed]
|
||||
}
|
||||
|
||||
test("recv UpdateFee (sender can't afford it, anchor outputs)", Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>
|
||||
test("recv UpdateFee (sender can't afford it, anchor outputs)", Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs), Tag(ChannelStateTestsTags.HighFeerateMismatchTolerance)) { f =>
|
||||
import f._
|
||||
val tx = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localCommit.commitTxAndRemoteSig.commitTx.tx
|
||||
// This feerate is just above the threshold: (800000 (alice balance) - 20000 (reserve) - 660 (anchors)) / 1124 (commit tx weight) = 693363
|
||||
@ -2267,7 +2267,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
val initialState = bob.stateData.asInstanceOf[DATA_NORMAL]
|
||||
val commitTx = initialState.commitments.latest.localCommit.commitTxAndRemoteSig.commitTx.tx
|
||||
assert(initialState.commitments.latest.localCommit.spec.commitTxFeerate == TestConstants.feeratePerKw)
|
||||
alice2bob.send(bob, UpdateFee(ByteVector32.Zeroes, TestConstants.feeratePerKw * 3))
|
||||
alice2bob.send(bob, UpdateFee(ByteVector32.Zeroes, TestConstants.feeratePerKw / 2))
|
||||
bob2alice.expectNoMessage(250 millis) // we don't close because the commitment doesn't contain any HTLC
|
||||
|
||||
// when we try to add an HTLC, we still disagree on the feerate so we close
|
||||
@ -2287,14 +2287,11 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
|
||||
val initialState = bob.stateData.asInstanceOf[DATA_NORMAL]
|
||||
assert(initialState.commitments.latest.localCommit.spec.commitTxFeerate == TestConstants.anchorOutputsFeeratePerKw)
|
||||
val add = UpdateAddHtlc(ByteVector32.Zeroes, 0, 2500000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None)
|
||||
alice2bob.send(bob, add)
|
||||
val fee = UpdateFee(initialState.channelId, TestConstants.anchorOutputsFeeratePerKw * 3)
|
||||
alice2bob.send(bob, fee)
|
||||
awaitCond(bob.stateData == initialState
|
||||
.modify(_.commitments.changes.remoteChanges.proposed).using(_ :+ add :+ fee)
|
||||
.modify(_.commitments.changes.remoteNextHtlcId).setTo(1))
|
||||
bob2alice.expectNoMessage(250 millis) // we don't close because we're using anchor outputs
|
||||
val error = bob2alice.expectMsgType[Error]
|
||||
assert(new String(error.data.toArray).contains("local/remote feerates are too different"))
|
||||
awaitCond(bob.stateName == CLOSING)
|
||||
}
|
||||
|
||||
test("recv UpdateFee (remote feerate is too small, anchor outputs)", Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>
|
||||
@ -2683,7 +2680,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
test("recv Shutdown (with unsigned fee update)") { f =>
|
||||
import f._
|
||||
val sender = TestProbe()
|
||||
alice ! CMD_UPDATE_FEE(FeeratePerKw(20000 sat), commit = true)
|
||||
alice ! CMD_UPDATE_FEE(FeeratePerKw(10_000 sat), commit = true)
|
||||
alice2bob.expectMsgType[UpdateFee]
|
||||
alice2bob.forward(bob)
|
||||
val sig = alice2bob.expectMsgType[CommitSig]
|
||||
@ -3054,7 +3051,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
assert(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localCommit.spec.commitTxFeerate == TestConstants.anchorOutputsFeeratePerKw / 2)
|
||||
|
||||
// The network fees spike, but Bob doesn't close the channel because we're using anchor outputs.
|
||||
val event = CurrentFeerates(FeeratesPerKw.single(TestConstants.anchorOutputsFeeratePerKw * 2))
|
||||
val event = CurrentFeerates(FeeratesPerKw.single(TestConstants.anchorOutputsFeeratePerKw * 10))
|
||||
bob.setFeerates(event.feeratesPerKw)
|
||||
bob ! event
|
||||
bob2alice.expectNoMessage(250 millis)
|
||||
@ -3064,7 +3061,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
test("recv CurrentFeerate (when fundee, commit-fee/network-fee are very different, without HTLCs)") { f =>
|
||||
import f._
|
||||
|
||||
val event = CurrentFeerates(FeeratesPerKw.single(FeeratePerKw(1000 sat)))
|
||||
val event = CurrentFeerates(FeeratesPerKw.single(FeeratePerKw(15_000 sat)))
|
||||
bob.setFeerates(event.feeratesPerKw)
|
||||
bob ! event
|
||||
bob2alice.expectNoMessage(250 millis) // we don't close because the commitment doesn't contain any HTLC
|
||||
|
@ -707,7 +707,7 @@ class ShutdownStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wit
|
||||
|
||||
test("recv CurrentFeerate (when fundee, commit-fee/network-fee are very different)") { f =>
|
||||
import f._
|
||||
val event = CurrentFeerates(FeeratesPerKw.single(FeeratePerKw(1000 sat)))
|
||||
val event = CurrentFeerates(FeeratesPerKw.single(FeeratePerKw(25000 sat)))
|
||||
bob.setFeerates(event.feeratesPerKw)
|
||||
bob ! event
|
||||
bob2alice.expectMsgType[Error]
|
||||
|
Loading…
Reference in New Issue
Block a user