1
0
Fork 0
mirror of https://github.com/ACINQ/eclair.git synced 2025-03-13 11:35:47 +01:00
This commit is contained in:
t-bast 2024-12-17 10:06:11 +01:00
parent c390560aa1
commit abb31ad49f
No known key found for this signature in database
GPG key ID: 34F377B0100ED6BB
6 changed files with 119 additions and 28 deletions

30
TODO.md Normal file
View file

@ -0,0 +1,30 @@
# TODO
- Move `ChannelAnnouncement` inside `Commitment`:
- add `blockheight` and `txindex` to `ConfirmedFundingTx` so that `Commitment` can be mapped to an `scid`
- don't prune commitments if we're announcing the channel and don't have a more recent `channel_announcement`
- the scid we use for our `channel_update`s are based on the last announcement we sent
- `Commitments` exposes a `lastChannelAnnouncement_opt` containing channel announcement and commitment
- move `makeAnnouncementSigs` to commitment
- Only update scids (`d.shortIds` and `ShortChannelIdAssigned`) after announcing the channel
- or immediately if private
- send `channel_announcement` to `Router` when we generate it
- refactor `WatchFundingDeeplyBuriedTriggered`
- receive `channel_ready`:
- re-send our `announcement_signatures`
- Implement splice gossip:
- receive `WatchFundingConfirmedTriggered`:
- send `splice_locked`
- if we've received their `splice_locked`, send `announcement_signatures` (no need to store)
- receive `splice_locked`:
- if we've already sent `splice_locked`, send `announcement_signatures` (no need to store)
- receive `announcement_signatures`:
- if `splice_locked` sent and received:
- re-create local announcement sigs
- publish `channel_announcement`
- store it in the `commitment`
- otherwise:
- re-send remote `announcement_signatures` to ourselves after 30 seconds
- reconnection:
- re-send `channel_ready` / `splice_locked`
- announcement signatures will be re-sent when receiving remote

View file

@ -26,7 +26,7 @@ import fr.acinq.eclair.channel.fund.{InteractiveTxBuilder, InteractiveTxSigningS
import fr.acinq.eclair.io.Peer
import fr.acinq.eclair.transactions.CommitmentSpec
import fr.acinq.eclair.transactions.Transactions._
import fr.acinq.eclair.wire.protocol.{ChannelAnnouncement, ChannelReady, ChannelReestablish, ChannelUpdate, ClosingSigned, CommitSig, FailureReason, FundingCreated, FundingSigned, Init, LiquidityAds, OnionRoutingPacket, OpenChannel, OpenDualFundedChannel, Shutdown, SpliceInit, Stfu, TxInitRbf, TxSignatures, UpdateAddHtlc, UpdateFailHtlc, UpdateFailMalformedHtlc, UpdateFulfillHtlc}
import fr.acinq.eclair.wire.protocol.{AnnouncementSignatures, ChannelAnnouncement, ChannelReady, ChannelReestablish, ChannelUpdate, ClosingSigned, CommitSig, FailureReason, FundingCreated, FundingSigned, Init, LiquidityAds, OnionRoutingPacket, OpenChannel, OpenDualFundedChannel, Shutdown, SpliceInit, Stfu, TxInitRbf, TxSignatures, UpdateAddHtlc, UpdateFailHtlc, UpdateFailMalformedHtlc, UpdateFulfillHtlc}
import fr.acinq.eclair.{Alias, BlockHeight, CltvExpiry, CltvExpiryDelta, Features, InitFeature, MilliSatoshi, MilliSatoshiLong, RealShortChannelId, TimestampMilli, UInt64}
import scodec.bits.ByteVector
@ -478,6 +478,14 @@ object RemoteFundingStatus {
case object Locked extends RemoteFundingStatus
}
sealed trait ChannelAnnouncementStatus
object ChannelAnnouncementStatus {
case class RemoteSigsReceived(sigs: AnnouncementSignatures, previous_opt: Option[ChannelAnnouncement]) extends ChannelAnnouncementStatus
case class LocalSigsSent(fundingTxId: TxId, sigs: AnnouncementSignatures, previous_opt: Option[ChannelAnnouncement]) extends ChannelAnnouncementStatus
case class Sent(ann: ChannelAnnouncement) extends ChannelAnnouncementStatus
case object NotAnnounced extends ChannelAnnouncementStatus
}
sealed trait DualFundingStatus
object DualFundingStatus {
/** We're waiting for one of the funding transactions to confirm. */
@ -636,7 +644,7 @@ final case class DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments: Commitments, shor
final case class DATA_NORMAL(commitments: Commitments,
shortIds: ShortIds,
channelAnnouncement: Option[ChannelAnnouncement],
channelAnnouncementStatus: ChannelAnnouncementStatus,
channelUpdate: ChannelUpdate,
localShutdown: Option[Shutdown],
remoteShutdown: Option[Shutdown],

View file

@ -296,7 +296,15 @@ object Helpers {
channelAnnouncement_opt.map(_.shortChannelId).getOrElse(localAlias)
}
def scidForChannelUpdate(d: DATA_NORMAL): ShortChannelId = scidForChannelUpdate(d.channelAnnouncement, d.shortIds.localAlias)
def scidForChannelUpdate(d: DATA_NORMAL): ShortChannelId = {
val channelAnnouncement_opt = d.channelAnnouncementStatus match {
case ann: ChannelAnnouncementStatus.RemoteSigsReceived => ann.previous_opt
case ann: ChannelAnnouncementStatus.LocalSigsSent => ann.previous_opt
case ChannelAnnouncementStatus.Sent(ann) => Some(ann)
case ChannelAnnouncementStatus.NotAnnounced => None
}
scidForChannelUpdate(channelAnnouncement_opt, d.shortIds.localAlias)
}
/**
* If our peer sent us an alias, that's what we must use in the channel_update we send them to ensure they're able to
@ -350,9 +358,10 @@ object Helpers {
def makeChannelUpdate(nodeParams: NodeParams, remoteNodeId: PublicKey, scid: ShortChannelId, commitments: Commitments, relayFees: RelayFees, enable: Boolean = true): ChannelUpdate =
Announcements.makeChannelUpdate(nodeParams.chainHash, nodeParams.privateKey, remoteNodeId, scid, nodeParams.channelConf.expiryDelta, commitments.params.remoteParams.htlcMinimum, relayFees.feeBase, relayFees.feeProportionalMillionths, maxHtlcAmount(nodeParams, commitments), isPrivate = !commitments.announceChannel, enable)
def makeAnnouncementSignatures(nodeParams: NodeParams, channelParams: ChannelParams, remoteFundingPubKey: PublicKey, shortChannelId: RealShortChannelId): AnnouncementSignatures = {
// TODO: move to Commitments object with nodeParams, fundingTxIndex and shortChannelId as param? Or even omitting shortChannelId and getting it from localFundingStatus
def makeAnnouncementSignatures(nodeParams: NodeParams, channelParams: ChannelParams, fundingTxIndex: Long, remoteFundingPubKey: PublicKey, shortChannelId: RealShortChannelId): AnnouncementSignatures = {
val features = Features.empty[Feature] // empty features for now
val fundingPubKey = nodeParams.channelKeyManager.fundingPublicKey(channelParams.localParams.fundingKeyPath, fundingTxIndex = 0) // TODO: public announcements are not yet supported with splices
val fundingPubKey = nodeParams.channelKeyManager.fundingPublicKey(channelParams.localParams.fundingKeyPath, fundingTxIndex)
val witness = Announcements.generateChannelAnnouncementWitness(
nodeParams.chainHash,
shortChannelId,

View file

@ -754,38 +754,73 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
case Event(c: CurrentFeerates.BitcoinCore, d: DATA_NORMAL) => handleCurrentFeerate(c, d)
case Event(WatchFundingDeeplyBuriedTriggered(blockHeight, txIndex, fundingTx), d: DATA_NORMAL) if d.channelAnnouncement.isEmpty =>
case Event(WatchFundingDeeplyBuriedTriggered(blockHeight, txIndex, fundingTx), d: DATA_NORMAL) =>
val finalRealShortId = RealScidStatus.Final(RealShortChannelId(blockHeight, txIndex, d.commitments.latest.commitInput.outPoint.index.toInt))
log.info(s"funding tx is deeply buried at blockHeight=$blockHeight txIndex=$txIndex shortChannelId=${finalRealShortId.realScid}")
log.info("funding tx is deeply buried at blockHeight={} txIndex={} shortChannelId={}", blockHeight, txIndex, finalRealShortId.realScid)
val shortIds1 = d.shortIds.copy(real = finalRealShortId)
context.system.eventStream.publish(ShortChannelIdAssigned(self, d.channelId, shortIds1, remoteNodeId))
if (d.shortIds.real == RealScidStatus.Unknown) {
// this is a zero-conf channel and it is the first time we know for sure that the funding tx has been confirmed
// This is a zero-conf channel and it is the first time we know for sure that the funding tx has been confirmed.
context.system.eventStream.publish(TransactionConfirmed(d.channelId, remoteNodeId, fundingTx))
}
val scidForChannelUpdate = Helpers.scidForChannelUpdate(d.channelAnnouncement, shortIds1.localAlias)
// if the shortChannelId is different from the one we had before, we need to re-announce it
val channelUpdate1 = if (d.channelUpdate.shortChannelId != scidForChannelUpdate) {
log.info(s"using new scid in channel_update: old=${d.channelUpdate.shortChannelId} new=$scidForChannelUpdate")
// we re-announce the channelUpdate for the same reason
Helpers.makeChannelUpdate(nodeParams, remoteNodeId, scidForChannelUpdate, d.commitments, d.channelUpdate.relayFees)
} else {
d.channelUpdate
if (!d.commitments.announceChannel) {
// If this is a private channel, we store every scid mapping, even if we may already have a more recent one from a splice.
context.system.eventStream.publish(ShortChannelIdAssigned(self, d.channelId, shortIds1, remoteNodeId))
}
if (d.commitments.announceChannel) {
// if channel is public we need to send our announcement_signatures in order to generate the channel_announcement
val localAnnSigs = Helpers.makeAnnouncementSignatures(nodeParams, d.commitments.params, d.commitments.latest.remoteFundingPubKey, finalRealShortId.realScid)
// we use goto() instead of stay() because we want to fire transitions
goto(NORMAL) using d.copy(shortIds = shortIds1, channelUpdate = channelUpdate1) storing() sending localAnnSigs
if (d.commitments.latest.fundingTxId != fundingTx.txid) {
log.warning("skipping announcement_signatures for fundingTxId={}, we're already at fundingTxIndex={}", fundingTx.txid, d.commitments.latest.fundingTxIndex)
// We only create a channel_announcement if it matches our latest commitment, otherwise the announcement we
// create may become obsolete before it is even received by other nodes on the network. This may be an issue
// for channels that splice very frequently while using 0-conf: the latest splice will only be announced if
// they wait for the announcement to be signed (after on-chain confirmations) before splicing again.
stay()
} else {
// we use goto() instead of stay() because we want to fire transitions
goto(NORMAL) using d.copy(shortIds = shortIds1, channelUpdate = channelUpdate1) storing()
d.channelAnnouncementStatus match {
case ChannelAnnouncementStatus.NotAnnounced if !d.commitments.announceChannel =>
stay() using d.copy(shortIds = shortIds1) storing()
case ChannelAnnouncementStatus.NotAnnounced if d.commitments.announceChannel =>
val localSigs = Helpers.makeAnnouncementSignatures(nodeParams, d.commitments.params, d.commitments.latest.fundingTxIndex, d.commitments.latest.remoteFundingPubKey, finalRealShortId.realScid)
stay() using d.copy(shortIds = shortIds1, channelAnnouncementStatus = ChannelAnnouncementStatus.LocalSigsSent(localSigs, previous_opt = None)) storing() sending localSigs
case ChannelAnnouncementStatus.RemoteSigsReceived(remoteSigs, _) if remoteSigs.shortChannelId != finalRealShortId.realScid =>
// TODO: if our scid is newer, we must drop theirs and go to LocalSigsSent?
// -> ignore the case where we have two scids from the same block, only possible for 0-conf splices and unlikely?
log.warning("skipping announcement_signatures for scid={}, remote signatures are for scid={}", finalRealShortId.realScid, remoteSigs.shortChannelId)
stay()
case ChannelAnnouncementStatus.RemoteSigsReceived(remoteSigs, _) =>
log.info("announcing channelId={} on the network with shortId={}", d.channelId, finalRealShortId.realScid)
val localSigs = Helpers.makeAnnouncementSignatures(nodeParams, d.commitments.params, d.commitments.latest.fundingTxIndex, d.commitments.latest.remoteFundingPubKey, finalRealShortId.realScid)
val fundingPubKey = keyManager.fundingPublicKey(d.commitments.params.localParams.fundingKeyPath, d.commitments.latest.fundingTxIndex)
val channelAnn = Announcements.makeChannelAnnouncement(nodeParams.chainHash, finalRealShortId.realScid, nodeParams.nodeId, d.commitments.params.remoteParams.nodeId, fundingPubKey.publicKey, d.commitments.latest.remoteFundingPubKey, localSigs.nodeSignature, remoteSigs.nodeSignature, localSigs.bitcoinSignature, remoteSigs.bitcoinSignature)
if (!Announcements.checkSigs(channelAnn)) {
handleLocalError(InvalidAnnouncementSignatures(d.channelId, remoteSigs), d, Some(remoteSigs))
} else {
// We generate a new channel_update because the scid used will change if we were previously using an alias.
val scidForChannelUpdate = Helpers.scidForChannelUpdate(Some(channelAnn), d.shortIds.localAlias)
context.system.eventStream.publish(ShortChannelIdAssigned(self, d.channelId, shortIds1, remoteNodeId))
val channelUpdate = Helpers.makeChannelUpdate(nodeParams, remoteNodeId, scidForChannelUpdate, d.commitments, d.channelUpdate.relayFees)
// TODO: send channel announcement to the router? Should be done in the onTransition
// We use goto() instead of stay() because we want to fire transitions.
goto(NORMAL) using d.copy(shortIds = shortIds1, channelAnnouncementStatus = ChannelAnnouncementStatus.Sent(channelAnn), channelUpdate = channelUpdate) storing() sending localSigs
}
case _: ChannelAnnouncementStatus.LocalSigsSent =>
log.warning("unexpected funding_deeply_buried event: announcement_signatures already sent")
stay()
case ChannelAnnouncementStatus.Sent(_) =>
log.warning("unexpected funding_deeply_buried event: channel already announced")
stay()
}
}
case Event(remoteAnnSigs: AnnouncementSignatures, d: DATA_NORMAL) if d.commitments.announceChannel =>
// channels are publicly announced if both parties want it (defined as feature bit)
d.shortIds.real match {
case RealScidStatus.Final(realScid) =>
// TODO: handle splicing case
// -> if we've already sent our announcement_sigs:
// -> send channel_announcement to our peer and to the router
// -> update d.shortIds with new scid
// -> emit ShortChannelIdAssigned
// -> create new channel updates
// -> otherwise just store their remote sigs, they sent it too early?
// we are aware that the channel has reached enough confirmations
// we already had sent our announcement_signatures but we don't store them so we need to recompute it
val localAnnSigs = Helpers.makeAnnouncementSignatures(nodeParams, d.commitments.params, d.commitments.latest.remoteFundingPubKey, realScid)
@ -1353,6 +1388,12 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
None
}
maybeEmitEventsPostSplice(d.shortIds, d.commitments, commitments1)
// TODO:
// - if private channel, update d.shortIds and send ShortChannelIdAssigned
// - not trivial to handle the case where two splices confirm in the same block -> only update scid if making progress in terms of fundingTxIndex somehow?
// - if public channel, send announcement_sigs if:
// - we haven't already sent them
// - the splice is fully locked (local+remote)
maybeUpdateMaxHtlcAmount(d.channelUpdate.htlcMaximumMsat, commitments1)
stay() using d.copy(commitments = commitments1) storing() sending toSend.toSeq
case Left(_) => stay()
@ -1362,6 +1403,10 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
d.commitments.updateRemoteFundingStatus(msg.fundingTxId) match {
case Right((commitments1, _)) =>
maybeEmitEventsPostSplice(d.shortIds, d.commitments, commitments1)
// TODO: send announcement_sigs if:
// - this is a public channel
// - we haven't already sent them
// - the splice is fully locked (local+remote)
maybeUpdateMaxHtlcAmount(d.channelUpdate.htlcMaximumMsat, commitments1)
stay() using d.copy(commitments = commitments1) storing()
case Left(_) => stay()
@ -2305,7 +2350,6 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
// we send it (if needed) when reconnected.
val shutdownInProgress = d.localShutdown.nonEmpty || d.remoteShutdown.nonEmpty
if (d.commitments.params.localParams.paysCommitTxFees && !shutdownInProgress) {
// TODO: all active commitments use the same feerate, but may have a different channel capacity: how should we compute networkFeeratePerKw?
val currentFeeratePerKw = d.commitments.latest.localCommit.spec.commitTxFeerate
val networkFeeratePerKw = nodeParams.onChainFeeConf.getCommitmentFeerate(nodeParams.currentBitcoinCoreFeerates, remoteNodeId, d.commitments.params.commitmentFormat, d.commitments.latest.capacity)
if (nodeParams.onChainFeeConf.shouldUpdateFee(currentFeeratePerKw, networkFeeratePerKw)) {
@ -2600,6 +2644,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
cancelTimer(QuiescenceTimeout.toString)
}
// TODO: channel announcement should be sent to the router if not done already!
sealed trait EmitLocalChannelEvent
/*
* This event is for:
@ -2736,7 +2781,6 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
}
private def handleCurrentFeerate(c: CurrentFeerates, d: ChannelDataWithCommitments) = {
// TODO: all active commitments use the same feerate, but may have a different channel capacity: how should we compute networkFeeratePerKw?
val commitments = d.commitments.latest
val networkFeeratePerKw = nodeParams.onChainFeeConf.getCommitmentFeerate(nodeParams.currentBitcoinCoreFeerates, remoteNodeId, d.commitments.params.commitmentFormat, commitments.capacity)
val currentFeeratePerKw = commitments.localCommit.spec.commitTxFeerate
@ -2762,7 +2806,6 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
* @return
*/
private def handleCurrentFeerateDisconnected(c: CurrentFeerates, d: ChannelDataWithCommitments) = {
// TODO: all active commitments use the same feerate, but may have a different channel capacity: how should we compute networkFeeratePerKw?
val commitments = d.commitments.latest
val networkFeeratePerKw = nodeParams.onChainFeeConf.getCommitmentFeerate(nodeParams.currentBitcoinCoreFeerates, remoteNodeId, d.commitments.params.commitmentFormat, commitments.capacity)
val currentFeeratePerKw = commitments.localCommit.spec.commitTxFeerate

View file

@ -139,7 +139,7 @@ trait CommonFundingHandlers extends CommonHandlers {
blockchain ! WatchFundingDeeplyBuried(self, commitments.latest.fundingTxId, ANNOUNCEMENTS_MINCONF)
val commitments1 = commitments.modify(_.remoteNextCommitInfo).setTo(Right(channelReady.nextPerCommitmentPoint))
peer ! ChannelReadyForPayments(self, remoteNodeId, commitments.channelId, fundingTxIndex = 0)
DATA_NORMAL(commitments1, shortIds1, None, initialChannelUpdate, None, None, None, SpliceStatus.NoSplice)
DATA_NORMAL(commitments1, shortIds1, ChannelAnnouncementStatus.NotAnnounced, initialChannelUpdate, None, None, None, SpliceStatus.NoSplice)
}
def delayEarlyAnnouncementSigs(remoteAnnSigs: AnnouncementSignatures): Unit = {

View file

@ -551,6 +551,7 @@ object Validation {
// channel is known, nothing more to do
d1
case None =>
// TODO: ShortChannelIdAssigned should have a "splice" flag and be ignored here
// this is a local channel that hasn't yet been announced (maybe it is a private channel or maybe it is a public
// channel that doesn't yet have 6 confirmations), we create a corresponding private channel
val pc = PrivateChannel(scia.channelId, scia.shortIds, localNodeId, scia.remoteNodeId, None, None, ChannelMeta(0 msat, 0 msat))