Skip to content

Commit

Permalink
Set relay fees per node and save them to database
Browse files Browse the repository at this point in the history
- Fees are set per node instead of per channel (setting different fees for different channels to the same node is most probably an error)
- Fees are saved to a database so that we can keep a trace of historic fees and new channels with a known node use the fee that we set and not the default fee.
  • Loading branch information
thomash-acinq committed Jul 29, 2021
1 parent 6d4da40 commit 7fa170b
Show file tree
Hide file tree
Showing 33 changed files with 393 additions and 96 deletions.
18 changes: 13 additions & 5 deletions eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala
Original file line number Diff line number Diff line change
Expand Up @@ -96,7 +96,7 @@ trait Eclair {

def forceClose(channels: List[ApiTypes.ChannelIdentifier])(implicit timeout: Timeout): Future[Map[ApiTypes.ChannelIdentifier, Either[Throwable, CommandResponse[CMD_FORCECLOSE]]]]

def updateRelayFee(channels: List[ApiTypes.ChannelIdentifier], feeBase: MilliSatoshi, feeProportionalMillionths: Long)(implicit timeout: Timeout): Future[Map[ApiTypes.ChannelIdentifier, Either[Throwable, CommandResponse[CMD_UPDATE_RELAY_FEE]]]]
def updateRelayFee(nodes: List[PublicKey], feeBase: MilliSatoshi, feeProportionalMillionths: Long)(implicit timeout: Timeout): Future[Map[ApiTypes.ChannelIdentifier, Either[Throwable, CommandResponse[CMD_UPDATE_RELAY_FEE]]]]

def channelsInfo(toRemoteNode_opt: Option[PublicKey])(implicit timeout: Timeout): Future[Iterable[RES_GETINFO]]

Expand Down Expand Up @@ -177,15 +177,18 @@ class EclairImpl(appKit: Kit) extends Eclair with Logging {
(appKit.switchboard ? Peer.Disconnect(nodeId)).mapTo[String]
}

override def open(nodeId: PublicKey, fundingAmount: Satoshi, pushAmount_opt: Option[MilliSatoshi], fundingFeeratePerByte_opt: Option[FeeratePerByte], initialRelayFees_opt: Option[(MilliSatoshi, Int)], flags_opt: Option[Int], openTimeout_opt: Option[Timeout])(implicit timeout: Timeout): Future[ChannelOpenResponse] = {
override def open(nodeId: PublicKey, fundingAmount: Satoshi, pushAmount_opt: Option[MilliSatoshi], fundingFeeratePerByte_opt: Option[FeeratePerByte], relayFees_opt: Option[(MilliSatoshi, Int)], flags_opt: Option[Int], openTimeout_opt: Option[Timeout])(implicit timeout: Timeout): Future[ChannelOpenResponse] = {
relayFees_opt match {
case Some((feeBase, feeProportionalMillionths)) => updateRelayFee(List(nodeId), feeBase, feeProportionalMillionths)
case None => ()
}
// we want the open timeout to expire *before* the default ask timeout, otherwise user won't get a generic response
val openTimeout = openTimeout_opt.getOrElse(Timeout(10 seconds))
(appKit.switchboard ? Peer.OpenChannel(
remoteNodeId = nodeId,
fundingSatoshis = fundingAmount,
pushMsat = pushAmount_opt.getOrElse(0 msat),
fundingTxFeeratePerKw_opt = fundingFeeratePerByte_opt.map(FeeratePerKw(_)),
initialRelayFees_opt = initialRelayFees_opt,
channelFlags = flags_opt.map(_.toByte),
timeout_opt = Some(openTimeout))).mapTo[ChannelOpenResponse]
}
Expand All @@ -198,8 +201,13 @@ class EclairImpl(appKit: Kit) extends Eclair with Logging {
sendToChannels[CommandResponse[CMD_FORCECLOSE]](channels, CMD_FORCECLOSE(ActorRef.noSender))
}

override def updateRelayFee(channels: List[ApiTypes.ChannelIdentifier], feeBaseMsat: MilliSatoshi, feeProportionalMillionths: Long)(implicit timeout: Timeout): Future[Map[ApiTypes.ChannelIdentifier, Either[Throwable, CommandResponse[CMD_UPDATE_RELAY_FEE]]]] = {
sendToChannels[CommandResponse[CMD_UPDATE_RELAY_FEE]](channels, CMD_UPDATE_RELAY_FEE(ActorRef.noSender, feeBaseMsat, feeProportionalMillionths))
override def updateRelayFee(nodes: List[PublicKey], feeBaseMsat: MilliSatoshi, feeProportionalMillionths: Long)(implicit timeout: Timeout): Future[Map[ApiTypes.ChannelIdentifier, Either[Throwable, CommandResponse[CMD_UPDATE_RELAY_FEE]]]] = {
for (node_id <- nodes) {
appKit.nodeParams.db.relayFees.addOrUpdateFees(node_id, feeBaseMsat, feeProportionalMillionths)
}
allChannels()
.map(channels => channels.filter(c => nodes.contains(c.a) || nodes.contains(c.b)).map(c => Right(c.shortChannelId)))
.flatMap(channels => sendToChannels[CommandResponse[CMD_UPDATE_RELAY_FEE]](channels.toList, CMD_UPDATE_RELAY_FEE(ActorRef.noSender, feeBaseMsat, feeProportionalMillionths)))
}

override def peers()(implicit timeout: Timeout): Future[Iterable[PeerInfo]] = for {
Expand Down
43 changes: 19 additions & 24 deletions eclair-core/src/main/scala/fr/acinq/eclair/channel/Channel.scala
Original file line number Diff line number Diff line change
Expand Up @@ -195,7 +195,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
startWith(WAIT_FOR_INIT_INTERNAL, Nothing)

when(WAIT_FOR_INIT_INTERNAL)(handleExceptions {
case Event(initFunder@INPUT_INIT_FUNDER(temporaryChannelId, fundingSatoshis, pushMsat, initialFeeratePerKw, fundingTxFeeratePerKw, _, localParams, remote, _, channelFlags, channelConfig, channelFeatures), Nothing) =>
case Event(initFunder@INPUT_INIT_FUNDER(temporaryChannelId, fundingSatoshis, pushMsat, initialFeeratePerKw, fundingTxFeeratePerKw, localParams, remote, _, channelFlags, channelConfig, channelFeatures), Nothing) =>
context.system.eventStream.publish(ChannelCreated(self, peer, remoteNodeId, isFunder = true, temporaryChannelId, initialFeeratePerKw, Some(fundingTxFeeratePerKw)))
activeConnection = remote
txPublisher ! SetChannelId(remoteNodeId, temporaryChannelId)
Expand Down Expand Up @@ -285,17 +285,17 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
context.system.eventStream.publish(ShortChannelIdAssigned(self, normal.channelId, normal.channelUpdate.shortChannelId, None))

// we rebuild a new channel_update with values from the configuration because they may have changed while eclair was down
// NB: we don't update the routing fees, because we don't want to overwrite manual changes made with CMD_UPDATE_RELAY_FEE
// Since CMD_UPDATE_RELAY_FEE is handled even when being offline, that's the preferred solution to update routing fees
val defaultFees = nodeParams.relayParams.defaultFees(data.commitments.announceChannel)
val (feeBase, feeProportionalMillionth) = nodeParams.db.relayFees.getFees(remoteNodeId).getOrElse((defaultFees.feeBase, defaultFees.feeProportionalMillionth.toLong))
val candidateChannelUpdate = Announcements.makeChannelUpdate(
nodeParams.chainHash,
nodeParams.privateKey,
remoteNodeId,
normal.channelUpdate.shortChannelId,
nodeParams.expiryDelta,
normal.commitments.remoteParams.htlcMinimum,
normal.channelUpdate.feeBaseMsat,
normal.channelUpdate.feeProportionalMillionths,
feeBase,
feeProportionalMillionth,
normal.commitments.capacity.toMilliSatoshi,
enable = Announcements.isEnabled(normal.channelUpdate.channelFlags))
val channelUpdate1 = if (Announcements.areSame(candidateChannelUpdate, normal.channelUpdate)) {
Expand Down Expand Up @@ -381,7 +381,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
initFeatures = remoteInit.features,
shutdownScript = remoteShutdownScript)
log.debug("remote params: {}", remoteParams)
goto(WAIT_FOR_FUNDING_CREATED) using DATA_WAIT_FOR_FUNDING_CREATED(open.temporaryChannelId, localParams, remoteParams, open.fundingSatoshis, open.pushMsat, open.feeratePerKw, None, open.firstPerCommitmentPoint, open.channelFlags, channelConfig, channelFeatures, accept) sending accept
goto(WAIT_FOR_FUNDING_CREATED) using DATA_WAIT_FOR_FUNDING_CREATED(open.temporaryChannelId, localParams, remoteParams, open.fundingSatoshis, open.pushMsat, open.feeratePerKw, open.firstPerCommitmentPoint, open.channelFlags, channelConfig, channelFeatures, accept) sending accept
}

case Event(c: CloseCommand, d) => handleFastClose(c, d.channelId)
Expand All @@ -392,7 +392,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
})

when(WAIT_FOR_ACCEPT_CHANNEL)(handleExceptions {
case Event(accept: AcceptChannel, d@DATA_WAIT_FOR_ACCEPT_CHANNEL(INPUT_INIT_FUNDER(temporaryChannelId, fundingSatoshis, pushMsat, initialFeeratePerKw, fundingTxFeeratePerKw, initialRelayFees_opt, localParams, _, remoteInit, _, channelConfig, channelFeatures), open)) =>
case Event(accept: AcceptChannel, d@DATA_WAIT_FOR_ACCEPT_CHANNEL(INPUT_INIT_FUNDER(temporaryChannelId, fundingSatoshis, pushMsat, initialFeeratePerKw, fundingTxFeeratePerKw, localParams, _, remoteInit, _, channelConfig, channelFeatures), open)) =>
log.info(s"received AcceptChannel=$accept")
Helpers.validateParamsFunder(nodeParams, channelFeatures, open, accept) match {
case Left(t) => handleLocalError(t, d, Some(accept))
Expand All @@ -416,7 +416,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
val localFundingPubkey = keyManager.fundingPublicKey(localParams.fundingKeyPath)
val fundingPubkeyScript = Script.write(Script.pay2wsh(Scripts.multiSig2of2(localFundingPubkey.publicKey, remoteParams.fundingPubKey)))
wallet.makeFundingTx(fundingPubkeyScript, fundingSatoshis, fundingTxFeeratePerKw).pipeTo(self)
goto(WAIT_FOR_FUNDING_INTERNAL) using DATA_WAIT_FOR_FUNDING_INTERNAL(temporaryChannelId, localParams, remoteParams, fundingSatoshis, pushMsat, initialFeeratePerKw, initialRelayFees_opt, accept.firstPerCommitmentPoint, channelConfig, channelFeatures, open)
goto(WAIT_FOR_FUNDING_INTERNAL) using DATA_WAIT_FOR_FUNDING_INTERNAL(temporaryChannelId, localParams, remoteParams, fundingSatoshis, pushMsat, initialFeeratePerKw, accept.firstPerCommitmentPoint, channelConfig, channelFeatures, open)
}

case Event(c: CloseCommand, d: DATA_WAIT_FOR_ACCEPT_CHANNEL) =>
Expand All @@ -437,7 +437,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
})

when(WAIT_FOR_FUNDING_INTERNAL)(handleExceptions {
case Event(MakeFundingTxResponse(fundingTx, fundingTxOutputIndex, fundingTxFee), d@DATA_WAIT_FOR_FUNDING_INTERNAL(temporaryChannelId, localParams, remoteParams, fundingAmount, pushMsat, initialFeeratePerKw, initialRelayFees_opt, remoteFirstPerCommitmentPoint, channelConfig, channelFeatures, open)) =>
case Event(MakeFundingTxResponse(fundingTx, fundingTxOutputIndex, fundingTxFee), d@DATA_WAIT_FOR_FUNDING_INTERNAL(temporaryChannelId, localParams, remoteParams, fundingAmount, pushMsat, initialFeeratePerKw, remoteFirstPerCommitmentPoint, channelConfig, channelFeatures, open)) =>
// let's create the first commitment tx that spends the yet uncommitted funding tx
Funding.makeFirstCommitTxs(keyManager, channelConfig, channelFeatures, temporaryChannelId, localParams, remoteParams, fundingAmount, pushMsat, initialFeeratePerKw, fundingTx.hash, fundingTxOutputIndex, remoteFirstPerCommitmentPoint) match {
case Left(ex) => handleLocalError(ex, d, None)
Expand All @@ -456,7 +456,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
txPublisher ! SetChannelId(remoteNodeId, channelId)
context.system.eventStream.publish(ChannelIdAssigned(self, remoteNodeId, temporaryChannelId, channelId))
// NB: we don't send a ChannelSignatureSent for the first commit
goto(WAIT_FOR_FUNDING_SIGNED) using DATA_WAIT_FOR_FUNDING_SIGNED(channelId, localParams, remoteParams, fundingTx, fundingTxFee, initialRelayFees_opt, localSpec, localCommitTx, RemoteCommit(0, remoteSpec, remoteCommitTx.tx.txid, remoteFirstPerCommitmentPoint), open.channelFlags, channelConfig, channelFeatures, fundingCreated) sending fundingCreated
goto(WAIT_FOR_FUNDING_SIGNED) using DATA_WAIT_FOR_FUNDING_SIGNED(channelId, localParams, remoteParams, fundingTx, fundingTxFee, localSpec, localCommitTx, RemoteCommit(0, remoteSpec, remoteCommitTx.tx.txid, remoteFirstPerCommitmentPoint), open.channelFlags, channelConfig, channelFeatures, fundingCreated) sending fundingCreated
}

case Event(Status.Failure(t), d: DATA_WAIT_FOR_FUNDING_INTERNAL) =>
Expand All @@ -482,7 +482,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
})

when(WAIT_FOR_FUNDING_CREATED)(handleExceptions {
case Event(FundingCreated(_, fundingTxHash, fundingTxOutputIndex, remoteSig), d@DATA_WAIT_FOR_FUNDING_CREATED(temporaryChannelId, localParams, remoteParams, fundingAmount, pushMsat, initialFeeratePerKw, initialRelayFees_opt, remoteFirstPerCommitmentPoint, channelFlags, channelConfig, channelFeatures, _)) =>
case Event(FundingCreated(_, fundingTxHash, fundingTxOutputIndex, remoteSig), d@DATA_WAIT_FOR_FUNDING_CREATED(temporaryChannelId, localParams, remoteParams, fundingAmount, pushMsat, initialFeeratePerKw, remoteFirstPerCommitmentPoint, channelFlags, channelConfig, channelFeatures, _)) =>
// they fund the channel with their funding tx, so the money is theirs (but we are paid pushMsat)
Funding.makeFirstCommitTxs(keyManager, channelConfig, channelFeatures, temporaryChannelId, localParams, remoteParams, fundingAmount, pushMsat, initialFeeratePerKw, fundingTxHash, fundingTxOutputIndex, remoteFirstPerCommitmentPoint) match {
case Left(ex) => handleLocalError(ex, d, None)
Expand Down Expand Up @@ -518,7 +518,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
watchFundingTx(commitments)
val fundingMinDepth = Helpers.minDepthForFunding(nodeParams, fundingAmount)
blockchain ! WatchFundingConfirmed(self, commitInput.outPoint.txid, fundingMinDepth)
goto(WAIT_FOR_FUNDING_CONFIRMED) using DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments, None, initialRelayFees_opt, nodeParams.currentBlockHeight, None, Right(fundingSigned)) storing() sending fundingSigned
goto(WAIT_FOR_FUNDING_CONFIRMED) using DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments, None, nodeParams.currentBlockHeight, None, Right(fundingSigned)) storing() sending fundingSigned
}
}

Expand All @@ -532,7 +532,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
})

when(WAIT_FOR_FUNDING_SIGNED)(handleExceptions {
case Event(msg@FundingSigned(_, remoteSig), d@DATA_WAIT_FOR_FUNDING_SIGNED(channelId, localParams, remoteParams, fundingTx, fundingTxFee, initialRelayFees_opt, localSpec, localCommitTx, remoteCommit, channelFlags, channelConfig, channelFeatures, fundingCreated)) =>
case Event(msg@FundingSigned(_, remoteSig), d@DATA_WAIT_FOR_FUNDING_SIGNED(channelId, localParams, remoteParams, fundingTx, fundingTxFee, localSpec, localCommitTx, remoteCommit, channelFlags, channelConfig, channelFeatures, fundingCreated)) =>
// we make sure that their sig checks out and that our first commit tx is spendable
val fundingPubKey = keyManager.fundingPublicKey(localParams.fundingKeyPath)
val localSigOfLocalTx = keyManager.sign(localCommitTx, fundingPubKey, TxOwner.Local, channelFeatures.commitmentFormat)
Expand Down Expand Up @@ -576,7 +576,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
}
}

goto(WAIT_FOR_FUNDING_CONFIRMED) using DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments, Some(fundingTx), initialRelayFees_opt, now, None, Left(fundingCreated)) storing() calling publishFundingTx()
goto(WAIT_FOR_FUNDING_CONFIRMED) using DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments, Some(fundingTx), now, None, Left(fundingCreated)) storing() calling publishFundingTx()
}

case Event(c: CloseCommand, d: DATA_WAIT_FOR_FUNDING_SIGNED) =>
Expand Down Expand Up @@ -609,7 +609,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
log.info(s"received their FundingLocked, deferring message")
stay() using d.copy(deferred = Some(msg)) // no need to store, they will re-send if we get disconnected

case Event(WatchFundingConfirmedTriggered(blockHeight, txIndex, fundingTx), d@DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments, _, initialRelayFees_opt, _, deferred, _)) =>
case Event(WatchFundingConfirmedTriggered(blockHeight, txIndex, fundingTx), d@DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments, _, _, deferred, _)) =>
Try(Transaction.correctlySpends(commitments.fullySignedLocalCommitTx(keyManager).tx, Seq(fundingTx), ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)) match {
case Success(_) =>
log.info(s"channelId=${commitments.channelId} was confirmed at blockHeight=$blockHeight txIndex=$txIndex")
Expand All @@ -622,7 +622,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
// as soon as it reaches NORMAL state, and before it is announced on the network
// (this id might be updated when the funding tx gets deeply buried, if there was a reorg in the meantime)
val shortChannelId = ShortChannelId(blockHeight, txIndex, commitments.commitInput.outPoint.index.toInt)
goto(WAIT_FOR_FUNDING_LOCKED) using DATA_WAIT_FOR_FUNDING_LOCKED(commitments, shortChannelId, fundingLocked, initialRelayFees_opt) storing() sending fundingLocked
goto(WAIT_FOR_FUNDING_LOCKED) using DATA_WAIT_FOR_FUNDING_LOCKED(commitments, shortChannelId, fundingLocked) storing() sending fundingLocked
case Failure(t) =>
log.error(t, s"rejecting channel with invalid funding tx: ${fundingTx.bin}")
goto(CLOSED)
Expand Down Expand Up @@ -658,18 +658,13 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
})

when(WAIT_FOR_FUNDING_LOCKED)(handleExceptions {
case Event(FundingLocked(_, nextPerCommitmentPoint), d@DATA_WAIT_FOR_FUNDING_LOCKED(commitments, shortChannelId, _, initialRelayFees_opt)) =>
case Event(FundingLocked(_, nextPerCommitmentPoint), d@DATA_WAIT_FOR_FUNDING_LOCKED(commitments, shortChannelId, _)) =>
// used to get the final shortChannelId, used in announcements (if minDepth >= ANNOUNCEMENTS_MINCONF this event will fire instantly)
blockchain ! WatchFundingDeeplyBuried(self, commitments.commitInput.outPoint.txid, ANNOUNCEMENTS_MINCONF)
context.system.eventStream.publish(ShortChannelIdAssigned(self, commitments.channelId, shortChannelId, None))
// we create a channel_update early so that we can use it to send payments through this channel, but it won't be propagated to other nodes since the channel is not yet announced
val defaultFees =
if (commitments.announceChannel) {
(nodeParams.relayParams.publicChannelFees.feeBase, nodeParams.relayParams.publicChannelFees.feeProportionalMillionth)
} else {
(nodeParams.relayParams.privateChannelFees.feeBase, nodeParams.relayParams.privateChannelFees.feeProportionalMillionth)
}
val (feeBase, feeProportionalMillionths) = initialRelayFees_opt.getOrElse(defaultFees)
val defaultFees = nodeParams.relayParams.defaultFees(commitments.announceChannel)
val (feeBase, feeProportionalMillionths) = nodeParams.db.relayFees.getFees(remoteNodeId).getOrElse((defaultFees.feeBase, defaultFees.feeProportionalMillionth.toLong))
val initialChannelUpdate = Announcements.makeChannelUpdate(nodeParams.chainHash, nodeParams.privateKey, remoteNodeId, shortChannelId, nodeParams.expiryDelta, d.commitments.remoteParams.htlcMinimum, feeBase, feeProportionalMillionths, commitments.capacity.toMilliSatoshi, enable = Helpers.aboveReserve(d.commitments))
// we need to periodically re-send channel updates, otherwise channel will be considered stale and get pruned by network
context.system.scheduler.scheduleWithFixedDelay(initialDelay = REFRESH_CHANNEL_UPDATE_INTERVAL, delay = REFRESH_CHANNEL_UPDATE_INTERVAL, receiver = self, message = BroadcastChannelUpdate(PeriodicRefresh))
Expand Down
Loading

0 comments on commit 7fa170b

Please sign in to comment.