Skip to content

Commit

Permalink
Announce public splice transactions
Browse files Browse the repository at this point in the history
We now support splicing on public channels: once the splice transaction
is confirmed and locked on both sides, nodes will exchange announcement
signatures that allows them to create a `channel_announcement` that they
then broadcast to the network.

This requires reworking the data model to include the announcement and
the real `short_channel_id` in each commitment, which lets us cleanly
distinguish real `short_channel_id`s from aliases (which are set at the
channel level regardless of the current commitments).

The flow now becomes:

- when the funding transaction of a commitment confirms, we set the
  corresponding real `short_channel_id` in that commitment
- if the channel is public and we've received `channel_ready` or
  `splice_locked`, we send our `announcement_signatures`
- if we receive `announcement_signatures` for a commitment for which
  the funding transaction is unconfirmed, we stash it and replay it
  when the transaction confirms
- if we receive `announcement_signatures` for a confirmed commitment,
  and we don't have a more recently announced commitment, we generate
  a `channel_announcement`, store it with the commitment and update
  our router data

When creating a `channel_update` for a public channel, we always use
the `short_channel_id` that matches the latest announcement we created.
This is very important to guarantee that nodes receiving our updates
will not discard them because they cannot match it to a channel.

For private channels, we stop allowing usage of the `short_channel_id`
for routing: `scid_alias` MUST be used, which ensures that the channel
utxo isn't revealed.

Note that when migrating to taproot channels, `splice_locked` will be
used to transmit nonces for the announcement signatures, which will be
compatible with the existing flow (and similarly, `channel_ready` will
be used for the initial funding transaction). They are retransmitted
on reconnection to ensure that the announcements can be generated.
  • Loading branch information
t-bast committed Jan 10, 2025
1 parent 5ba0f56 commit 1bf54c1
Show file tree
Hide file tree
Showing 74 changed files with 1,988 additions and 627 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -417,23 +417,24 @@ case class RevokedCommitPublished(commitTx: Transaction, claimMainOutputTx: Opti
}

/**
* Short identifiers for the channel.
* Short identifiers for the channel that aren't related to the on-chain utxo.
*
* @param real_opt the real scid of the latest announced (and thus confirmed) funding transaction.
* @param localAlias we must remember the alias that we sent to our peer because we use it to:
* - identify incoming [[ChannelUpdate]] at the connection level
* - route outgoing payments to that channel
* @param remoteAlias_opt we only remember the last alias received from our peer, we use this to generate
* routing hints in [[fr.acinq.eclair.payment.Bolt11Invoice]]
*/
case class ShortIds(real_opt: Option[RealShortChannelId], localAlias: Alias, remoteAlias_opt: Option[Alias])
case class ShortIdAliases(localAlias: Alias, remoteAlias_opt: Option[Alias])

sealed trait LocalFundingStatus {
def signedTx_opt: Option[Transaction]
/** We store local signatures for the purpose of retransmitting if the funding/splicing flow is interrupted. */
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 @@ -449,15 +450,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 @@ -589,7 +593,7 @@ final case class DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments: Commitments,
lastSent: Either[FundingCreated, FundingSigned]) extends ChannelDataWithCommitments {
def fundingTx_opt: Option[Transaction] = commitments.latest.localFundingStatus.signedTx_opt
}
final case class DATA_WAIT_FOR_CHANNEL_READY(commitments: Commitments, shortIds: ShortIds) extends ChannelDataWithCommitments
final case class DATA_WAIT_FOR_CHANNEL_READY(commitments: Commitments, aliases: ShortIdAliases) extends ChannelDataWithCommitments

final case class DATA_WAIT_FOR_OPEN_DUAL_FUNDED_CHANNEL(init: INPUT_INIT_CHANNEL_NON_INITIATOR) extends TransientChannelData {
val channelId: ByteVector32 = init.temporaryChannelId
Expand Down Expand Up @@ -622,16 +626,17 @@ final case class DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments: Commitments,
def latestFundingTx: DualFundedUnconfirmedFundingTx = commitments.latest.localFundingStatus.asInstanceOf[DualFundedUnconfirmedFundingTx]
def previousFundingTxs: Seq[DualFundedUnconfirmedFundingTx] = allFundingTxs diff Seq(latestFundingTx)
}
final case class DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments: Commitments, shortIds: ShortIds) extends ChannelDataWithCommitments
final case class DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments: Commitments, aliases: ShortIdAliases) extends ChannelDataWithCommitments

final case class DATA_NORMAL(commitments: Commitments,
shortIds: ShortIds,
channelAnnouncement: Option[ChannelAnnouncement],
aliases: ShortIdAliases,
channelUpdate: ChannelUpdate,
localShutdown: Option[Shutdown],
remoteShutdown: Option[Shutdown],
closingFeerates: Option[ClosingFeerates],
spliceStatus: SpliceStatus) extends ChannelDataWithCommitments {
val lastAnnouncedCommitment_opt: Option[AnnouncedCommitment] = commitments.lastAnnouncement_opt
val lastAnnouncement_opt: Option[ChannelAnnouncement] = lastAnnouncedCommitment_opt.map(_.announcement)
val isNegotiatingQuiescence: Boolean = spliceStatus.isNegotiatingQuiescence
val isQuiescent: Boolean = spliceStatus.isQuiescent
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ import fr.acinq.bitcoin.scalacompat.{ByteVector32, Satoshi, Transaction, TxId}
import fr.acinq.eclair.blockchain.fee.FeeratePerKw
import fr.acinq.eclair.channel.Helpers.Closing.ClosingType
import fr.acinq.eclair.wire.protocol.{ChannelAnnouncement, ChannelUpdate, LiquidityAds}
import fr.acinq.eclair.{BlockHeight, Features, MilliSatoshi, ShortChannelId}
import fr.acinq.eclair.{BlockHeight, Features, MilliSatoshi, RealShortChannelId, ShortChannelId}

/**
* Created by PM on 17/08/2016.
Expand All @@ -44,8 +44,8 @@ case class ChannelRestored(channel: ActorRef, channelId: ByteVector32, peer: Act

case class ChannelIdAssigned(channel: ActorRef, remoteNodeId: PublicKey, temporaryChannelId: ByteVector32, channelId: ByteVector32) extends ChannelEvent

/** This event will be sent whenever a new scid is assigned to the channel, be it a real, local alias or remote alias. */
case class ShortChannelIdAssigned(channel: ActorRef, channelId: ByteVector32, shortIds: ShortIds, remoteNodeId: PublicKey, isAnnounced: Boolean) extends ChannelEvent
/** This event will be sent whenever a new scid is assigned to the channel: local alias, remote alias or announcement. */
case class ShortChannelIdAssigned(channel: ActorRef, channelId: ByteVector32, announcement_opt: Option[ChannelAnnouncement], aliases: ShortIdAliases, remoteNodeId: PublicKey) extends ChannelEvent

/** This event will be sent if a channel was aborted before completing the opening flow. */
case class ChannelAborted(channel: ActorRef, remoteNodeId: PublicKey, channelId: ByteVector32) extends ChannelEvent
Expand All @@ -56,24 +56,24 @@ case class ChannelOpened(channel: ActorRef, remoteNodeId: PublicKey, channelId:
/** This event is sent once channel_ready or splice_locked have been exchanged. */
case class ChannelReadyForPayments(channel: ActorRef, remoteNodeId: PublicKey, channelId: ByteVector32, fundingTxIndex: Long) extends ChannelEvent

case class LocalChannelUpdate(channel: ActorRef, channelId: ByteVector32, shortIds: ShortIds, remoteNodeId: PublicKey, channelAnnouncement_opt: Option[ChannelAnnouncement], channelUpdate: ChannelUpdate, commitments: Commitments) extends ChannelEvent {
case class LocalChannelUpdate(channel: ActorRef, channelId: ByteVector32, aliases: ShortIdAliases, remoteNodeId: PublicKey, announcement_opt: Option[AnnouncedCommitment], channelUpdate: ChannelUpdate, commitments: Commitments) extends ChannelEvent {
/**
* We always include the local alias because we must always be able to route based on it.
* However we only include the real scid if option_scid_alias is disabled, because we otherwise want to hide it.
*/
def scidsForRouting: Seq[ShortChannelId] = {
val canUseRealScid = !commitments.params.channelFeatures.hasFeature(Features.ScidAlias)
if (canUseRealScid) {
shortIds.real_opt.toSeq :+ shortIds.localAlias
announcement_opt.map(_.shortChannelId).toSeq :+ aliases.localAlias
} else {
Seq(shortIds.localAlias)
Seq(aliases.localAlias)
}
}
}

case class ChannelUpdateParametersChanged(channel: ActorRef, channelId: ByteVector32, remoteNodeId: PublicKey, channelUpdate: ChannelUpdate) extends ChannelEvent

case class LocalChannelDown(channel: ActorRef, channelId: ByteVector32, shortIds: ShortIds, remoteNodeId: PublicKey) extends ChannelEvent
case class LocalChannelDown(channel: ActorRef, channelId: ByteVector32, realScids: Seq[RealShortChannelId], aliases: ShortIdAliases, remoteNodeId: PublicKey) extends ChannelEvent

case class ChannelStateChanged(channel: ActorRef, channelId: ByteVector32, peer: ActorRef, remoteNodeId: PublicKey, previousState: ChannelState, currentState: ChannelState, commitments_opt: Option[Commitments]) extends ChannelEvent

Expand All @@ -97,7 +97,7 @@ case class TransactionPublished(channelId: ByteVector32, remoteNodeId: PublicKey
case class TransactionConfirmed(channelId: ByteVector32, remoteNodeId: PublicKey, tx: Transaction) extends ChannelEvent

// NB: this event is only sent when the channel is available.
case class AvailableBalanceChanged(channel: ActorRef, channelId: ByteVector32, shortIds: ShortIds, commitments: Commitments) extends ChannelEvent
case class AvailableBalanceChanged(channel: ActorRef, channelId: ByteVector32, aliases: ShortIdAliases, commitments: Commitments) extends ChannelEvent

case class ChannelPersisted(channel: ActorRef, remoteNodeId: PublicKey, channelId: ByteVector32, data: PersistentChannelData) extends ChannelEvent

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,12 @@ case class Commitment(fundingTxIndex: Long,
val commitInput: InputInfo = localCommit.commitTxAndRemoteSig.commitTx.input
val fundingTxId: TxId = commitInput.outPoint.txid
val capacity: Satoshi = commitInput.txOut.amount
/** Once the funding transaction is confirmed, short_channel_id matching this transaction. */
val shortChannelId_opt: Option[RealShortChannelId] = localFundingStatus match {
case f: LocalFundingStatus.ConfirmedFundingTx => Some(f.shortChannelId)
case _ => None
}
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 +374,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 @@ -732,6 +767,13 @@ object Commitment {
}
}

/** A commitment for which a channel announcement has been created. */
case class AnnouncedCommitment(commitment: Commitment, announcement: ChannelAnnouncement) {
val shortChannelId: RealShortChannelId = announcement.shortChannelId
val fundingTxId: TxId = commitment.fundingTxId
val fundingTxIndex: Long = commitment.fundingTxIndex
}

/** Subset of Commitments when we want to work with a single, specific commitment. */
case class FullCommitment(params: ChannelParams, changes: CommitmentChanges,
fundingTxIndex: Long,
Expand All @@ -740,6 +782,10 @@ case class FullCommitment(params: ChannelParams, changes: CommitmentChanges,
localFundingStatus: LocalFundingStatus, remoteFundingStatus: RemoteFundingStatus,
localCommit: LocalCommit, remoteCommit: RemoteCommit, nextRemoteCommit_opt: Option[NextRemoteCommit]) {
val channelId = params.channelId
val shortChannelId_opt = localFundingStatus match {
case f: LocalFundingStatus.ConfirmedFundingTx => Some(f.shortChannelId)
case _ => None
}
val localParams = params.localParams
val remoteParams = params.remoteParams
val commitInput = localCommit.commitTxAndRemoteSig.commitTx.input
Expand Down Expand Up @@ -810,13 +856,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[AnnouncedCommitment] = all.collectFirst { case c if c.announcement_opt.nonEmpty => AnnouncedCommitment(c, c.announcement_opt.get) }

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 @@ -1158,7 +1210,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 @@ -1251,10 +1303,19 @@ case class Commitments(params: ChannelParams,
.sortBy(_.fundingTxIndex)
.lastOption match {
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)
val pruned = 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).
// If we never announced the channel, we don't need to announce old commitments, we will directly announce the last one.
val pruningIndex = lastAnnouncement_opt.map(_.fundingTxIndex).getOrElse(lastConfirmed.fundingTxIndex)
// We can prune all RBF candidates, and commitments that came before the last announced one.
inactive.filter(c => c.fundingTxIndex < pruningIndex || (c.fundingTxIndex == lastConfirmed.fundingTxIndex && c.fundingTxId != lastConfirmed.fundingTxId))
} else {
// We can prune all other commitments with the same or lower funding index.
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)
case _ =>
Expand All @@ -1270,6 +1331,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
Loading

0 comments on commit 1bf54c1

Please sign in to comment.