Skip to content

Commit

Permalink
Move channel_announcement to Commitment
Browse files Browse the repository at this point in the history
In preparation for announcing splices on public channels, we move the
`channel_announcement` that was previously included in `DATA_NORMAL`
to the `Commitment`'s funding status. Every confirmed commitment may
be announced to the network once we support splicing, to let the rest
of the network know that the channel wasn't closed by the spending tx
but only changed its capacity.
  • Loading branch information
t-bast committed Dec 18, 2024
1 parent c390560 commit b01ea02
Show file tree
Hide file tree
Showing 22 changed files with 252 additions and 152 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -444,6 +444,8 @@ sealed trait LocalFundingStatus {
def localSigs_opt: Option[TxSignatures]
/** Basic information about the liquidity purchase negotiated in this transaction, if any. */
def liquidityPurchase_opt: Option[LiquidityAds.PurchaseBasicInfo]
/** After confirmation, we store the channel announcement matching this funding transaction, once we've created it. */
def announcement_opt: Option[ChannelAnnouncement]
}
object LocalFundingStatus {
sealed trait NotLocked extends LocalFundingStatus
Expand All @@ -459,15 +461,18 @@ object LocalFundingStatus {
case class SingleFundedUnconfirmedFundingTx(signedTx_opt: Option[Transaction]) extends UnconfirmedFundingTx with NotLocked {
override val localSigs_opt: Option[TxSignatures] = None
override val liquidityPurchase_opt: Option[LiquidityAds.PurchaseBasicInfo] = None
override val announcement_opt: Option[ChannelAnnouncement] = None
}
case class DualFundedUnconfirmedFundingTx(sharedTx: SignedSharedTransaction, createdAt: BlockHeight, fundingParams: InteractiveTxParams, liquidityPurchase_opt: Option[LiquidityAds.PurchaseBasicInfo]) extends UnconfirmedFundingTx with NotLocked {
override val signedTx_opt: Option[Transaction] = sharedTx.signedTx_opt
override val localSigs_opt: Option[TxSignatures] = Some(sharedTx.localSigs)
override val announcement_opt: Option[ChannelAnnouncement] = None
}
case class ZeroconfPublishedFundingTx(tx: Transaction, localSigs_opt: Option[TxSignatures], liquidityPurchase_opt: Option[LiquidityAds.PurchaseBasicInfo]) extends UnconfirmedFundingTx with Locked {
override val signedTx_opt: Option[Transaction] = Some(tx)
override val announcement_opt: Option[ChannelAnnouncement] = None
}
case class ConfirmedFundingTx(tx: Transaction, localSigs_opt: Option[TxSignatures], liquidityPurchase_opt: Option[LiquidityAds.PurchaseBasicInfo]) extends LocalFundingStatus with Locked {
case class ConfirmedFundingTx(tx: Transaction, shortChannelId: RealShortChannelId, announcement_opt: Option[ChannelAnnouncement], localSigs_opt: Option[TxSignatures], liquidityPurchase_opt: Option[LiquidityAds.PurchaseBasicInfo]) extends LocalFundingStatus with Locked {
override val signedTx_opt: Option[Transaction] = Some(tx)
}
}
Expand Down Expand Up @@ -636,12 +641,12 @@ final case class DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments: Commitments, shor

final case class DATA_NORMAL(commitments: Commitments,
shortIds: ShortIds,
channelAnnouncement: Option[ChannelAnnouncement],
channelUpdate: ChannelUpdate,
localShutdown: Option[Shutdown],
remoteShutdown: Option[Shutdown],
closingFeerates: Option[ClosingFeerates],
spliceStatus: SpliceStatus) extends ChannelDataWithCommitments {
val lastAnnouncement_opt: Option[ChannelAnnouncement] = commitments.lastAnnouncement_opt
val isNegotiatingQuiescence: Boolean = spliceStatus.isNegotiatingQuiescence
val isQuiescent: Boolean = spliceStatus.isQuiescent
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -12,10 +12,11 @@ import fr.acinq.eclair.channel.fund.InteractiveTxBuilder.SharedTransaction
import fr.acinq.eclair.crypto.keymanager.ChannelKeyManager
import fr.acinq.eclair.crypto.{Generators, ShaChain}
import fr.acinq.eclair.payment.OutgoingPaymentPacket
import fr.acinq.eclair.router.Announcements
import fr.acinq.eclair.transactions.Transactions._
import fr.acinq.eclair.transactions._
import fr.acinq.eclair.wire.protocol._
import fr.acinq.eclair.{BlockHeight, CltvExpiry, CltvExpiryDelta, Features, MilliSatoshi, MilliSatoshiLong, payment}
import fr.acinq.eclair.{BlockHeight, CltvExpiry, CltvExpiryDelta, Feature, Features, MilliSatoshi, MilliSatoshiLong, NodeParams, RealShortChannelId, payment}
import scodec.bits.ByteVector

/** Static channel parameters shared by all commitments. */
Expand Down Expand Up @@ -278,6 +279,7 @@ case class Commitment(fundingTxIndex: Long,
val commitInput: InputInfo = localCommit.commitTxAndRemoteSig.commitTx.input
val fundingTxId: TxId = commitInput.outPoint.txid
val capacity: Satoshi = commitInput.txOut.amount
val announcement_opt: Option[ChannelAnnouncement] = localFundingStatus.announcement_opt

/** Channel reserve that applies to our funds. */
def localChannelReserve(params: ChannelParams): Satoshi = params.localChannelReserveForCapacity(capacity, fundingTxIndex > 0)
Expand Down Expand Up @@ -367,6 +369,34 @@ case class Commitment(fundingTxIndex: Long,
}
}

/** Sign the announcement for this commitment, if the funding transaction is confirmed. */
def signAnnouncement(nodeParams: NodeParams, params: ChannelParams): Option[AnnouncementSignatures] = {
localFundingStatus match {
case funding: LocalFundingStatus.ConfirmedFundingTx if params.announceChannel =>
val features = Features.empty[Feature] // empty features for now
val fundingPubKey = nodeParams.channelKeyManager.fundingPublicKey(params.localParams.fundingKeyPath, fundingTxIndex)
val witness = Announcements.generateChannelAnnouncementWitness(
nodeParams.chainHash,
funding.shortChannelId,
nodeParams.nodeKeyManager.nodeId,
params.remoteParams.nodeId,
fundingPubKey.publicKey,
remoteFundingPubKey,
features
)
val localBitcoinSig = nodeParams.channelKeyManager.signChannelAnnouncement(witness, fundingPubKey.path)
val localNodeSig = nodeParams.nodeKeyManager.signChannelAnnouncement(witness)
Some(AnnouncementSignatures(params.channelId, funding.shortChannelId, localNodeSig, localBitcoinSig))
case _ => None
}
}

/** Add the channel_announcement provided if it is for this commitment. */
def addAnnouncementIfMatches(ann: ChannelAnnouncement): Commitment = localFundingStatus match {
case f: LocalFundingStatus.ConfirmedFundingTx if f.shortChannelId == ann.shortChannelId => copy(localFundingStatus = f.copy(announcement_opt = Some(ann)))
case _ => this
}

def hasNoPendingHtlcs: Boolean = localCommit.spec.htlcs.isEmpty && remoteCommit.spec.htlcs.isEmpty && nextRemoteCommit_opt.isEmpty

def hasNoPendingHtlcsOrFeeUpdate(changes: CommitmentChanges): Boolean = hasNoPendingHtlcs &&
Expand Down Expand Up @@ -810,13 +840,19 @@ case class Commitments(params: ChannelParams,
lazy val availableBalanceForSend: MilliSatoshi = active.map(_.availableBalanceForSend(params, changes)).min
lazy val availableBalanceForReceive: MilliSatoshi = active.map(_.availableBalanceForReceive(params, changes)).min

val all: Seq[Commitment] = active ++ inactive

// We always use the last commitment that was created, to make sure we never go back in time.
val latest = FullCommitment(params, changes, active.head.fundingTxIndex, active.head.firstRemoteCommitIndex, active.head.remoteFundingPubKey, active.head.localFundingStatus, active.head.remoteFundingStatus, active.head.localCommit, active.head.remoteCommit, active.head.nextRemoteCommit_opt)

val all: Seq[Commitment] = active ++ inactive
val lastAnnouncement_opt: Option[ChannelAnnouncement] = all.flatMap(_.announcement_opt).headOption

def add(commitment: Commitment): Commitments = copy(active = commitment +: active)

def addAnnouncement(ann: ChannelAnnouncement): Commitments = copy(
active = active.map(_.addAnnouncementIfMatches(ann)),
inactive = inactive.map(_.addAnnouncementIfMatches(ann)),
)

// @formatter:off
def localIsQuiescent: Boolean = changes.localChanges.all.isEmpty
def remoteIsQuiescent: Boolean = changes.remoteChanges.all.isEmpty
Expand Down Expand Up @@ -1155,7 +1191,7 @@ case class Commitments(params: ChannelParams,
def localFundingSigs(fundingTxId: TxId): Option[TxSignatures] = {
all.find(_.fundingTxId == fundingTxId).flatMap(_.localFundingStatus.localSigs_opt)
}

def liquidityPurchase(fundingTxId: TxId): Option[LiquidityAds.PurchaseBasicInfo] = {
all.find(_.fundingTxId == fundingTxId).flatMap(_.localFundingStatus.liquidityPurchase_opt)
}
Expand Down Expand Up @@ -1247,10 +1283,25 @@ case class Commitments(params: ChannelParams,
.filter(_.localFundingStatus.isInstanceOf[LocalFundingStatus.ConfirmedFundingTx])
.sortBy(_.fundingTxIndex)
.lastOption match {
// NB: we cannot prune active commitments, even if we know that they have been double-spent, because our peer
// may not yet be aware of it, and will expect us to send commit_sig.
case Some(lastConfirmed) if params.announceChannel =>
// If the most recently confirmed commitment isn't announced yet, we cannot prune the last commitment we
// announced, because our channel updates are based on its announcement (and its short_channel_id).
val pruningIndex = all
.filter(_.announcement_opt.nonEmpty)
.map(_.fundingTxIndex)
.sorted
.lastOption
// If we never announced the channel, we don't need to announce old commitments, we will directly announce
// the last one.
.getOrElse(lastConfirmed.fundingTxIndex)
// We can prune all RBF candidates, and commitments that came before the last announced one.
val pruned = inactive.filter(c => c.fundingTxIndex < pruningIndex || (c.fundingTxIndex == lastConfirmed.fundingTxIndex && c.fundingTxId != lastConfirmed.fundingTxId))
pruned.foreach(c => log.info("pruning commitment fundingTxIndex={} fundingTxId={}", c.fundingTxIndex, c.fundingTxId))
copy(inactive = inactive diff pruned)
case Some(lastConfirmed) =>
// We can prune all other commitments with the same or lower funding index.
// NB: we cannot prune active commitments, even if we know that they have been double-spent, because our peer
// may not yet be aware of it, and will expect us to send commit_sig.
val pruned = inactive.filter(c => c.fundingTxId != lastConfirmed.fundingTxId && c.fundingTxIndex <= lastConfirmed.fundingTxIndex)
pruned.foreach(c => log.info("pruning commitment fundingTxIndex={} fundingTxId={}", c.fundingTxIndex, c.fundingTxId))
copy(inactive = inactive diff pruned)
Expand All @@ -1267,6 +1318,14 @@ case class Commitments(params: ChannelParams,
def resolveCommitment(spendingTx: Transaction): Option[Commitment] = {
all.find(c => spendingTx.txIn.map(_.outPoint).contains(c.commitInput.outPoint))
}

/** Find the corresponding commitment based on its short_channel_id (once funding transaction is confirmed). */
def resolveCommitment(shortChannelId: RealShortChannelId): Option[Commitment] = {
all.find(c => c.localFundingStatus match {
case f: LocalFundingStatus.ConfirmedFundingTx => f.shortChannelId == shortChannelId
case _ => false
})
}
}

object Commitments {
Expand Down
19 changes: 1 addition & 18 deletions eclair-core/src/main/scala/fr/acinq/eclair/channel/Helpers.scala
Original file line number Diff line number Diff line change
Expand Up @@ -296,7 +296,7 @@ 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 = scidForChannelUpdate(d.lastAnnouncement_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
Expand Down Expand Up @@ -350,23 +350,6 @@ 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 = {
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 witness = Announcements.generateChannelAnnouncementWitness(
nodeParams.chainHash,
shortChannelId,
nodeParams.nodeKeyManager.nodeId,
channelParams.remoteParams.nodeId,
fundingPubKey.publicKey,
remoteFundingPubKey,
features
)
val localBitcoinSig = nodeParams.channelKeyManager.signChannelAnnouncement(witness, fundingPubKey.path)
val localNodeSig = nodeParams.nodeKeyManager.signChannelAnnouncement(witness)
AnnouncementSignatures(channelParams.channelId, shortChannelId, localNodeSig, localBitcoinSig)
}

def getRelayFees(nodeParams: NodeParams, remoteNodeId: PublicKey, announceChannel: Boolean): RelayFees = {
val defaultFees = nodeParams.relayParams.defaultFees(announceChannel)
nodeParams.db.peers.getRelayFees(remoteNodeId).getOrElse(defaultFees)
Expand Down
Loading

0 comments on commit b01ea02

Please sign in to comment.