Skip to content

Commit

Permalink
Handle fees increases when channel is OFFLINE (#1080)
Browse files Browse the repository at this point in the history
* Handle feerate changes when OFFLINE, SYNCING

* Add 'close-on-offline-feerate-mismatch' configuration to avoid closing offline channel when the feerate mismatch if over the threshold.
  • Loading branch information
araspitzu authored Sep 20, 2019
1 parent b5461b8 commit abf3907
Show file tree
Hide file tree
Showing 7 changed files with 140 additions and 39 deletions.
1 change: 1 addition & 0 deletions eclair-core/src/main/resources/reference.conf
Original file line number Diff line number Diff line change
Expand Up @@ -90,6 +90,7 @@ eclair {
// maximum local vs remote feerate mismatch; 1.0 means 100%
// actual check is abs((local feerate - remote fee rate) / (local fee rate + remote fee rate)/2) > fee rate mismatch
max-feerate-mismatch = 1.56 // will allow remote fee rates up to 8x bigger or smaller than our local fee rate
close-on-offline-feerate-mismatch = true // do not change this unless you know what you are doing

// funder will send an UpdateFee message if the difference between current commitment fee and actual current network fee is greater
// than this ratio.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -217,6 +217,7 @@ object NodeParams {
feeTargets = feeTargets,
feeEstimator = feeEstimator,
maxFeerateMismatch = config.getDouble("on-chain-fees.max-feerate-mismatch"),
closeOnOfflineMismatch = config.getBoolean("on-chain-fees.close-on-offline-feerate-mismatch"),
updateFeeMinDiffRatio = config.getDouble("on-chain-fees.update-fee-min-diff-ratio")
),
maxHtlcValueInFlightMsat = UInt64(config.getLong("max-htlc-value-in-flight-msat")),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,4 +26,4 @@ trait FeeEstimator {

case class FeeTargets(fundingBlockTarget: Int, commitmentBlockTarget: Int, mutualCloseBlockTarget: Int, claimMainBlockTarget: Int)

case class OnChainFeeConf(feeTargets: FeeTargets, feeEstimator: FeeEstimator, maxFeerateMismatch: Double, updateFeeMinDiffRatio: Double)
case class OnChainFeeConf(feeTargets: FeeTargets, feeEstimator: FeeEstimator, maxFeerateMismatch: Double, closeOnOfflineMismatch: Boolean, updateFeeMinDiffRatio: Double)
64 changes: 45 additions & 19 deletions eclair-core/src/main/scala/fr/acinq/eclair/channel/Channel.scala
Original file line number Diff line number Diff line change
Expand Up @@ -854,15 +854,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
case Event(c: CurrentBlockCount, d: DATA_NORMAL) => handleNewBlock(c, d)

case Event(c@CurrentFeerates(feeratesPerKw), d: DATA_NORMAL) =>
val networkFeeratePerKw = feeratesPerKw.feePerBlock(target = nodeParams.onChainFeeConf.feeTargets.commitmentBlockTarget)
d.commitments.localParams.isFunder match {
case true if Helpers.shouldUpdateFee(d.commitments.localCommit.spec.feeratePerKw, networkFeeratePerKw, nodeParams.onChainFeeConf.updateFeeMinDiffRatio) =>
self ! CMD_UPDATE_FEE(networkFeeratePerKw, commit = true)
stay
case false if Helpers.isFeeDiffTooHigh(d.commitments.localCommit.spec.feeratePerKw, networkFeeratePerKw, nodeParams.onChainFeeConf.maxFeerateMismatch) =>
handleLocalError(FeerateTooDifferent(d.channelId, localFeeratePerKw = networkFeeratePerKw, remoteFeeratePerKw = d.commitments.localCommit.spec.feeratePerKw), d, Some(c))
case _ => stay
}
handleCurrentFeerate(c, d)

case Event(WatchEventConfirmed(BITCOIN_FUNDING_DEEPLYBURIED, blockHeight, txIndex, _), d: DATA_NORMAL) if d.channelAnnouncement.isEmpty =>
val shortChannelId = ShortChannelId(blockHeight, txIndex, d.commitments.commitInput.outPoint.index.toInt)
Expand Down Expand Up @@ -1135,16 +1127,8 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId

case Event(c: CurrentBlockCount, d: DATA_SHUTDOWN) => handleNewBlock(c, d)

case Event(c@CurrentFeerates(feeratesPerKw), d: DATA_SHUTDOWN) =>
val networkFeeratePerKw = feeratesPerKw.feePerBlock(target = nodeParams.onChainFeeConf.feeTargets.commitmentBlockTarget)
d.commitments.localParams.isFunder match {
case true if Helpers.shouldUpdateFee(d.commitments.localCommit.spec.feeratePerKw, networkFeeratePerKw, nodeParams.onChainFeeConf.updateFeeMinDiffRatio) =>
self ! CMD_UPDATE_FEE(networkFeeratePerKw, commit = true)
stay
case false if Helpers.isFeeDiffTooHigh(d.commitments.localCommit.spec.feeratePerKw, networkFeeratePerKw, nodeParams.onChainFeeConf.maxFeerateMismatch) =>
handleLocalError(FeerateTooDifferent(d.channelId, localFeeratePerKw = networkFeeratePerKw, remoteFeeratePerKw = d.commitments.localCommit.spec.feeratePerKw), d, Some(c))
case _ => stay
}
case Event(c@CurrentFeerates(feerates), d: DATA_SHUTDOWN) =>
handleCurrentFeerate(c, d)

case Event(WatchEventSpent(BITCOIN_FUNDING_SPENT, tx), d: DATA_SHUTDOWN) if tx.txid == d.commitments.remoteCommit.txid => handleRemoteSpentCurrent(tx, d)

Expand Down Expand Up @@ -1419,6 +1403,9 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
// -> in CLOSING we either have mutual closed (so no more htlcs), or already have unilaterally closed (so no action required), and we can't be in OFFLINE state anyway
case Event(c: CurrentBlockCount, d: HasCommitments) => handleNewBlock(c, d)

case Event(c: CurrentFeerates, d: HasCommitments) =>
handleOfflineFeerate(c, d)

case Event(c: CMD_ADD_HTLC, d: DATA_NORMAL) => handleAddDisconnected(c, d)

case Event(CMD_UPDATE_RELAY_FEE(feeBaseMsat, feeProportionalMillionths), d: DATA_NORMAL) =>
Expand Down Expand Up @@ -1553,6 +1540,9 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId

case Event(c: CurrentBlockCount, d: HasCommitments) => handleNewBlock(c, d)

case Event(c: CurrentFeerates, d: HasCommitments) =>
handleOfflineFeerate(c, d)

case Event(getTxResponse: GetTxWithMetaResponse, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) if getTxResponse.txid == d.commitments.commitInput.outPoint.txid => handleGetFundingTx(getTxResponse, d.waitingSince, d.fundingTx)

case Event(BITCOIN_FUNDING_PUBLISH_FAILED, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) => handleFundingPublishFailed(d)
Expand Down Expand Up @@ -1722,6 +1712,42 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
origin_opt.foreach(_ ! m)
}

def handleCurrentFeerate(c: CurrentFeerates, d: HasCommitments) = {
val networkFeeratePerKw = c.feeratesPerKw.feePerBlock(target = nodeParams.onChainFeeConf.feeTargets.commitmentBlockTarget)
val currentFeeratePerKw = d.commitments.localCommit.spec.feeratePerKw
d.commitments.localParams.isFunder match {
case true if Helpers.shouldUpdateFee(currentFeeratePerKw, networkFeeratePerKw, nodeParams.onChainFeeConf.updateFeeMinDiffRatio) =>
self ! CMD_UPDATE_FEE(networkFeeratePerKw, commit = true)
stay
case false if Helpers.isFeeDiffTooHigh(currentFeeratePerKw, networkFeeratePerKw, nodeParams.onChainFeeConf.maxFeerateMismatch) =>
handleLocalError(FeerateTooDifferent(d.channelId, localFeeratePerKw = networkFeeratePerKw, remoteFeeratePerKw = d.commitments.localCommit.spec.feeratePerKw), d, Some(c))
case _ => stay
}
}

/**
* This is used to check for the commitment fees when the channel is not operational but we have something at stake
* @param c the new feerates
* @param d the channel commtiments
* @return
*/
def handleOfflineFeerate(c: CurrentFeerates, d: HasCommitments) = {
val networkFeeratePerKw = c.feeratesPerKw.feePerBlock(target = nodeParams.onChainFeeConf.feeTargets.commitmentBlockTarget)
val currentFeeratePerKw = d.commitments.localCommit.spec.feeratePerKw
// if the fees are too high we risk to not be able to confirm our current commitment
if(networkFeeratePerKw > currentFeeratePerKw && Helpers.isFeeDiffTooHigh(currentFeeratePerKw, networkFeeratePerKw, nodeParams.onChainFeeConf.maxFeerateMismatch)){
if(nodeParams.onChainFeeConf.closeOnOfflineMismatch) {
log.warning(s"closing OFFLINE ${d.channelId} due to fee mismatch: currentFeeratePerKw=$currentFeeratePerKw networkFeeratePerKw=$networkFeeratePerKw")
handleLocalError(FeerateTooDifferent(d.channelId, localFeeratePerKw = currentFeeratePerKw, remoteFeeratePerKw = networkFeeratePerKw), d, Some(c))
} else {
log.warning(s"channel ${d.channelId} is OFFLINE but its fee mismatch is over the threshold: currentFeeratePerKw=$currentFeeratePerKw networkFeeratePerKw=$networkFeeratePerKw")
stay
}
} else {
stay
}
}

def handleCommandSuccess(sender: ActorRef, newData: Data) = {
stay using newData replying "ok"
}
Expand Down
32 changes: 16 additions & 16 deletions eclair-core/src/main/scala/fr/acinq/eclair/channel/Helpers.scala
Original file line number Diff line number Diff line change
Expand Up @@ -171,27 +171,27 @@ object Helpers {
}

/**
*
* @param remoteFeeratePerKw remote fee rate per kiloweight
* @param localFeeratePerKw local fee rate per kiloweight
* @return the "normalized" difference between local and remote fee rate, i.e. |remote - local| / avg(local, remote)
*/
def feeRateMismatch(remoteFeeratePerKw: Long, localFeeratePerKw: Long): Double =
Math.abs((2.0 * (remoteFeeratePerKw - localFeeratePerKw)) / (localFeeratePerKw + remoteFeeratePerKw))
*
* @param referenceFeePerKw reference fee rate per kiloweight
* @param currentFeePerKw current fee rate per kiloweight
* @return the "normalized" difference between i.e local and remote fee rate: |reference - current| / avg(current, reference)
*/
def feeRateMismatch(referenceFeePerKw: Long, currentFeePerKw: Long): Double =
Math.abs((2.0 * (referenceFeePerKw - currentFeePerKw)) / (currentFeePerKw + referenceFeePerKw))

def shouldUpdateFee(commitmentFeeratePerKw: Long, networkFeeratePerKw: Long, updateFeeMinDiffRatio: Double): Boolean =
feeRateMismatch(networkFeeratePerKw, commitmentFeeratePerKw) > updateFeeMinDiffRatio

/**
*
* @param remoteFeeratePerKw remote fee rate per kiloweight
* @param localFeeratePerKw local fee rate per kiloweight
* @param maxFeerateMismatchRatio maximum fee rate mismatch ratio
* @return true if the difference between local and remote fee rates is too high.
* the actual check is |remote - local| / avg(local, remote) > mismatch ratio
*/
def isFeeDiffTooHigh(remoteFeeratePerKw: Long, localFeeratePerKw: Long, maxFeerateMismatchRatio: Double): Boolean =
feeRateMismatch(remoteFeeratePerKw, localFeeratePerKw) > maxFeerateMismatchRatio
*
* @param referenceFeePerKw reference fee rate per kiloweight
* @param currentFeePerKw current fee rate per kiloweight
* @param maxFeerateMismatchRatio maximum fee rate mismatch ratio
* @return true if the difference between current and reference fee rates is too high.
* the actual check is |reference - current| / avg(current, reference) > mismatch ratio
*/
def isFeeDiffTooHigh(referenceFeePerKw: Long, currentFeePerKw: Long, maxFeerateMismatchRatio: Double): Boolean =
feeRateMismatch(referenceFeePerKw, currentFeePerKw) > maxFeerateMismatchRatio

/**
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -79,6 +79,7 @@ object TestConstants {
feeTargets = FeeTargets(6, 2, 2, 6),
feeEstimator = new TestFeeEstimator,
maxFeerateMismatch = 1.5,
closeOnOfflineMismatch = true,
updateFeeMinDiffRatio = 0.1
),
maxHtlcValueInFlightMsat = UInt64(150000000),
Expand Down Expand Up @@ -156,6 +157,7 @@ object TestConstants {
feeTargets = FeeTargets(6, 2, 2, 6),
feeEstimator = new TestFeeEstimator,
maxFeerateMismatch = 1.0,
closeOnOfflineMismatch = true,
updateFeeMinDiffRatio = 0.1
),
maxHtlcValueInFlightMsat = UInt64.MaxValue, // Bob has no limit on the combined max value of in-flight htlcs
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,9 @@ import akka.actor.Status
import akka.testkit.{TestActorRef, TestProbe}
import fr.acinq.bitcoin.Crypto.PrivateKey
import fr.acinq.bitcoin.{ByteVector32, ScriptFlags, Transaction}
import fr.acinq.eclair.blockchain.{CurrentBlockCount, PublishAsap, WatchConfirmed, WatchEventSpent}
import fr.acinq.eclair.TestConstants.Alice
import fr.acinq.eclair.blockchain.fee.FeeratesPerKw
import fr.acinq.eclair.blockchain.{CurrentBlockCount, CurrentFeerates, PublishAsap, WatchConfirmed, WatchEventSpent}
import fr.acinq.eclair.channel.Channel.LocalError
import fr.acinq.eclair.channel._
import fr.acinq.eclair.channel.states.StateTestsHelperMethods
Expand All @@ -32,7 +34,7 @@ import fr.acinq.eclair.router.Announcements
import fr.acinq.eclair.transactions.Transactions.HtlcSuccessTx
import fr.acinq.eclair.wire._
import fr.acinq.eclair.{CltvExpiry, CltvExpiryDelta, LongToBtcAmount, TestConstants, TestkitBaseClass, randomBytes32}
import org.scalatest.Outcome
import org.scalatest.{Outcome, Tag}

import scala.concurrent.duration._

Expand All @@ -45,7 +47,10 @@ class OfflineStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
type FixtureParam = SetupFixture

override def withFixture(test: OneArgTest): Outcome = {
val setup = init()
val setup = test.tags.contains("disable-offline-mismatch") match {
case false => init()
case true => init(nodeParamsA = Alice.nodeParams.copy(onChainFeeConf = Alice.nodeParams.onChainFeeConf.copy(closeOnOfflineMismatch = false)))
}
import setup._
within(30 seconds) {
reachNormal(setup)
Expand Down Expand Up @@ -459,4 +464,70 @@ class OfflineStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
alice2blockchain.expectNoMsg(250 millis)
}

test("handle feerate changes while offline (funder scenario)") { f =>
import f._
val sender = TestProbe()

// we simulate a disconnection
sender.send(alice, INPUT_DISCONNECTED)
sender.send(bob, INPUT_DISCONNECTED)
awaitCond(alice.stateName == OFFLINE)
awaitCond(bob.stateName == OFFLINE)

val aliceStateData = alice.stateData.asInstanceOf[DATA_NORMAL]
val aliceCommitTx = aliceStateData.commitments.localCommit.publishableTxs.commitTx.tx

val localFeeratePerKw = aliceStateData.commitments.localCommit.spec.feeratePerKw
val tooHighFeeratePerKw = ((alice.underlyingActor.nodeParams.onChainFeeConf.maxFeerateMismatch + 6) * localFeeratePerKw).toLong
val highFeerate = FeeratesPerKw.single(tooHighFeeratePerKw)

// alice is funder
sender.send(alice, CurrentFeerates(highFeerate))
alice2blockchain.expectMsg(PublishAsap(aliceCommitTx))
}

test("handle feerate changes while offline (don't close on mismatch)", Tag("disable-offline-mismatch")) { f =>
import f._
val sender = TestProbe()

// we simulate a disconnection
sender.send(alice, INPUT_DISCONNECTED)
sender.send(bob, INPUT_DISCONNECTED)
awaitCond(alice.stateName == OFFLINE)
awaitCond(bob.stateName == OFFLINE)

val aliceStateData = alice.stateData.asInstanceOf[DATA_NORMAL]
val aliceCommitTx = aliceStateData.commitments.localCommit.publishableTxs.commitTx.tx

val localFeeratePerKw = aliceStateData.commitments.localCommit.spec.feeratePerKw
val tooHighFeeratePerKw = ((alice.underlyingActor.nodeParams.onChainFeeConf.maxFeerateMismatch + 6) * localFeeratePerKw).toLong
val highFeerate = FeeratesPerKw.single(tooHighFeeratePerKw)

// this time Alice will ignore feerate changes for the offline channel
sender.send(alice, CurrentFeerates(highFeerate))
alice2blockchain.expectNoMsg()
alice2bob.expectNoMsg()
}

test("handle feerate changes while offline (fundee scenario)") { f =>
import f._
val sender = TestProbe()

// we simulate a disconnection
sender.send(alice, INPUT_DISCONNECTED)
sender.send(bob, INPUT_DISCONNECTED)
awaitCond(alice.stateName == OFFLINE)
awaitCond(bob.stateName == OFFLINE)

val bobStateData = bob.stateData.asInstanceOf[DATA_NORMAL]
val bobCommitTx = bobStateData.commitments.localCommit.publishableTxs.commitTx.tx

val localFeeratePerKw = bobStateData.commitments.localCommit.spec.feeratePerKw
val tooHighFeeratePerKw = ((bob.underlyingActor.nodeParams.onChainFeeConf.maxFeerateMismatch + 6) * localFeeratePerKw).toLong
val highFeerate = FeeratesPerKw.single(tooHighFeeratePerKw)

// bob is fundee
sender.send(bob, CurrentFeerates(highFeerate))
bob2blockchain.expectMsg(PublishAsap(bobCommitTx))
}
}

0 comments on commit abf3907

Please sign in to comment.