1
0
Fork 0
mirror of https://github.com/ACINQ/eclair.git synced 2025-03-13 11:35:47 +01:00

Announce public splice transactions

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.
This commit is contained in:
t-bast 2024-12-31 16:41:28 +01:00
parent 5ba0f563f9
commit 1bf54c1fcc
No known key found for this signature in database
GPG key ID: 34F377B0100ED6BB
74 changed files with 1993 additions and 632 deletions

View file

@ -417,16 +417,15 @@ 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]
@ -434,6 +433,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
@ -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)
}
}
@ -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
@ -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
}

View file

@ -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.
@ -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
@ -56,7 +56,7 @@ 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.
@ -64,16 +64,16 @@ case class LocalChannelUpdate(channel: ActorRef, channelId: ByteVector32, shortI
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
@ -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

View file

@ -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. */
@ -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)
@ -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 &&
@ -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,
@ -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
@ -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
// 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
// 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 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
@ -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)
}
@ -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 _ =>
@ -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 {

View file

@ -296,17 +296,17 @@ 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.aliases.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
* match this update with the corresponding local channel. If they didn't send us an alias, it means we're not using
* 0-conf and we'll use the real scid.
*/
def channelUpdateForDirectPeer(nodeParams: NodeParams, channelUpdate: ChannelUpdate, shortIds: ShortIds): ChannelUpdate = {
shortIds.remoteAlias_opt match {
def channelUpdateForDirectPeer(nodeParams: NodeParams, channelUpdate: ChannelUpdate, realScid_opt: Option[RealShortChannelId], aliases: ShortIdAliases): ChannelUpdate = {
aliases.remoteAlias_opt match {
case Some(remoteAlias) => Announcements.updateScid(nodeParams.privateKey, channelUpdate, remoteAlias)
case None => shortIds.real_opt match {
case None => realScid_opt match {
case Some(realScid) => Announcements.updateScid(nodeParams.privateKey, channelUpdate, realScid)
// This case is a spec violation: this is a 0-conf channel, so our peer MUST send their alias.
// They won't be able to match our channel_update with their local channel, too bad for them.
@ -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)

View file

@ -60,11 +60,11 @@ class Register extends Actor with ActorLogging {
case scidAssigned: ShortChannelIdAssigned =>
// We map all known scids (real or alias) to the channel_id. The relayer is in charge of deciding whether a real
// scid can be used or not for routing (see option_scid_alias), but the register is neutral.
val m = (scidAssigned.shortIds.real_opt.toSeq :+ scidAssigned.shortIds.localAlias).map(_ -> scidAssigned.channelId).toMap
val m = (scidAssigned.announcement_opt.map(_.shortChannelId).toSeq :+ scidAssigned.aliases.localAlias).map(_ -> scidAssigned.channelId).toMap
// duplicate check for aliases (we use a random value in a large enough space that there should never be collisions)
shortIds.get(scidAssigned.shortIds.localAlias) match {
shortIds.get(scidAssigned.aliases.localAlias) match {
case Some(channelId) if channelId != scidAssigned.channelId =>
log.error("duplicate alias={} for channelIds={},{} this should never happen!", scidAssigned.shortIds.localAlias, channelId, scidAssigned.channelId)
log.error("duplicate alias={} for channelIds={},{} this should never happen!", scidAssigned.aliases.localAlias, channelId, scidAssigned.channelId)
case _ => ()
}
context become main(channels, shortIds ++ m, channelsTo, nodeIdToPeer)

View file

@ -217,6 +217,16 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
// we stash announcement_signatures if we receive them earlier than expected
var announcementSigsStash = Map.empty[RealShortChannelId, AnnouncementSignatures]
private def trimAnnouncementSigsStashIfNeeded(): Unit = {
if (announcementSigsStash.size >= 10) {
// We shouldn't store an unbounded number of announcement_signatures for scids that we don't have in our
// commitments, otherwise it can be used as a DoS vector.
val oldestScid = announcementSigsStash.keys.minBy(ShortChannelId.blockHeight(_))
log.warning("too many pending announcement_signatures: dropping scid={}", oldestScid)
announcementSigsStash -= oldestScid
}
}
val txPublisher = txPublisherFactory.spawnTxPublisher(context, remoteNodeId)
// this will be used to detect htlc timeouts
@ -354,7 +364,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
goto(CLOSING) using closing
case normal: DATA_NORMAL =>
context.system.eventStream.publish(ShortChannelIdAssigned(self, normal.channelId, normal.shortIds, remoteNodeId, normal.channelAnnouncement.nonEmpty))
context.system.eventStream.publish(ShortChannelIdAssigned(self, normal.channelId, normal.lastAnnouncement_opt, normal.aliases, remoteNodeId))
// we check the configuration because the values for channel_update may have changed while eclair was down
val fees = getRelayFees(nodeParams, remoteNodeId, normal.commitments.announceChannel)
@ -439,7 +449,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
d.commitments.sendAdd(c, nodeParams.currentBlockHeight, nodeParams.channelConf, nodeParams.currentBitcoinCoreFeerates, nodeParams.onChainFeeConf) match {
case Right((commitments1, add)) =>
if (c.commit) self ! CMD_SIGN()
context.system.eventStream.publish(AvailableBalanceChanged(self, d.channelId, d.shortIds, commitments1))
context.system.eventStream.publish(AvailableBalanceChanged(self, d.channelId, d.aliases, commitments1))
handleCommandSuccess(c, d.copy(commitments = commitments1)) sending add
case Left(cause) => handleAddHtlcCommandError(c, cause, Some(d.channelUpdate))
}
@ -454,7 +464,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
d.commitments.sendFulfill(c) match {
case Right((commitments1, fulfill)) =>
if (c.commit) self ! CMD_SIGN()
context.system.eventStream.publish(AvailableBalanceChanged(self, d.channelId, d.shortIds, commitments1))
context.system.eventStream.publish(AvailableBalanceChanged(self, d.channelId, d.aliases, commitments1))
handleCommandSuccess(c, d.copy(commitments = commitments1)) sending fulfill
case Left(cause) =>
// we acknowledge the command right away in case of failure
@ -479,7 +489,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
case None => d.commitments.sendFail(c, nodeParams.privateKey) match {
case Right((commitments1, fail)) =>
if (c.commit) self ! CMD_SIGN()
context.system.eventStream.publish(AvailableBalanceChanged(self, d.channelId, d.shortIds, commitments1))
context.system.eventStream.publish(AvailableBalanceChanged(self, d.channelId, d.aliases, commitments1))
handleCommandSuccess(c, d.copy(commitments = commitments1)) sending fail
case Left(cause) =>
// we acknowledge the command right away in case of failure
@ -491,7 +501,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
d.commitments.sendFailMalformed(c) match {
case Right((commitments1, fail)) =>
if (c.commit) self ! CMD_SIGN()
context.system.eventStream.publish(AvailableBalanceChanged(self, d.channelId, d.shortIds, commitments1))
context.system.eventStream.publish(AvailableBalanceChanged(self, d.channelId, d.aliases, commitments1))
handleCommandSuccess(c, d.copy(commitments = commitments1)) sending fail
case Left(cause) =>
// we acknowledge the command right away in case of failure
@ -514,7 +524,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
d.commitments.sendFee(c, nodeParams.onChainFeeConf) match {
case Right((commitments1, fee)) =>
if (c.commit) self ! CMD_SIGN()
context.system.eventStream.publish(AvailableBalanceChanged(self, d.channelId, d.shortIds, commitments1))
context.system.eventStream.publish(AvailableBalanceChanged(self, d.channelId, d.aliases, commitments1))
handleCommandSuccess(c, d.copy(commitments = commitments1)) sending fee
case Left(cause) => handleCommandError(cause, c)
}
@ -607,7 +617,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
}
if (d.commitments.availableBalanceForSend != commitments1.availableBalanceForSend) {
// we send this event only when our balance changes
context.system.eventStream.publish(AvailableBalanceChanged(self, d.channelId, d.shortIds, commitments1))
context.system.eventStream.publish(AvailableBalanceChanged(self, d.channelId, d.aliases, commitments1))
}
context.system.eventStream.publish(ChannelSignatureReceived(self, commitments1))
// If we're now quiescent, we may send our stfu message.
@ -755,54 +765,56 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
case Event(c: CurrentFeerates.BitcoinCore, d: DATA_NORMAL) => handleCurrentFeerate(c, d)
case Event(_: ChannelReady, d: DATA_NORMAL) =>
d.shortIds.real_opt match {
case Some(realScid) if d.channelAnnouncement.nonEmpty =>
d.commitments.all.find(_.fundingTxIndex == 0) match {
case Some(c) if c.announcement_opt.nonEmpty =>
// The channel hasn't been used yet, and our peer may be missing our announcement_signatures if:
// - we received their announcement_signatures before sending ours
// - then the funding transaction confirmed and we created the channel_announcement
// - but we got disconnected before they received our announcement_signatures
log.debug("received channel_ready, retransmitting announcement_signatures")
val localAnnSigs = Helpers.makeAnnouncementSignatures(nodeParams, d.commitments.params, d.commitments.latest.remoteFundingPubKey, realScid)
stay() sending localAnnSigs
val localAnnSigs_opt = c.signAnnouncement(nodeParams, d.commitments.params)
stay() sending localAnnSigs_opt.toSeq
case _ => stay()
}
// Channels are publicly announced if both parties want it: we ignore this message if we don't want to announce the channel.
case Event(remoteAnnSigs: AnnouncementSignatures, d: DATA_NORMAL) if d.commitments.announceChannel =>
// Channels are publicly announced if both parties want it: we ignore this message if we don't want to announce the channel.
d.shortIds.real_opt match {
case Some(realScid) if remoteAnnSigs.shortChannelId == realScid =>
d.channelAnnouncement match {
case None =>
log.info("announcing channelId={} on the network with shortChannelId={}", d.channelId, realScid)
// We already sent our announcement_signatures but we don't store them so we need to recompute them.
val localAnnSigs = Helpers.makeAnnouncementSignatures(nodeParams, d.commitments.params, d.commitments.latest.remoteFundingPubKey, realScid)
val fundingPubKey = keyManager.fundingPublicKey(d.commitments.params.localParams.fundingKeyPath, fundingTxIndex = 0) // TODO: public announcements are not yet supported with splices
val channelAnn = Announcements.makeChannelAnnouncement(nodeParams.chainHash, localAnnSigs.shortChannelId, nodeParams.nodeId, d.commitments.params.remoteParams.nodeId, fundingPubKey.publicKey, d.commitments.latest.remoteFundingPubKey, localAnnSigs.nodeSignature, remoteAnnSigs.nodeSignature, localAnnSigs.bitcoinSignature, remoteAnnSigs.bitcoinSignature)
d.commitments.resolveCommitment(remoteAnnSigs.shortChannelId) match {
case Some(c) if d.lastAnnouncedCommitment_opt.exists(_.fundingTxIndex > c.fundingTxIndex) =>
log.info("ignoring remote announcement_signatures for scid={} with fundingTxIndex={}, we have a more recent announcement with scid={} and fundingTxIndex={}", remoteAnnSigs.shortChannelId, c.fundingTxIndex, d.lastAnnouncement_opt.map(_.shortChannelId), d.lastAnnouncedCommitment_opt.map(_.fundingTxIndex))
stay()
case Some(c) if c.announcement_opt.nonEmpty =>
log.debug("ignoring remote announcement_signatures for scid={} with fundingTxIndex={}, channel is already announced", remoteAnnSigs.shortChannelId, c.fundingTxIndex)
stay()
case Some(c) =>
c.signAnnouncement(nodeParams, d.commitments.params) match {
case Some(localAnnSigs) =>
log.info("announcing channelId={} on the network with shortChannelId={} for fundingTxIndex={}", d.channelId, localAnnSigs.shortChannelId, c.fundingTxIndex)
val fundingPubKey = keyManager.fundingPublicKey(d.commitments.params.localParams.fundingKeyPath, c.fundingTxIndex)
val channelAnn = Announcements.makeChannelAnnouncement(nodeParams.chainHash, localAnnSigs.shortChannelId, nodeParams.nodeId, remoteNodeId, fundingPubKey.publicKey, c.remoteFundingPubKey, localAnnSigs.nodeSignature, remoteAnnSigs.nodeSignature, localAnnSigs.bitcoinSignature, remoteAnnSigs.bitcoinSignature)
if (!Announcements.checkSigs(channelAnn)) {
handleLocalError(InvalidAnnouncementSignatures(d.channelId, remoteAnnSigs), d, Some(remoteAnnSigs))
} else {
// We generate a new channel_update because we can now use the scid of the announced funding transaction.
val scidForChannelUpdate = Helpers.scidForChannelUpdate(Some(channelAnn), d.shortIds.localAlias)
val channelUpdate = Helpers.makeChannelUpdate(nodeParams, remoteNodeId, scidForChannelUpdate, d.commitments, d.channelUpdate.relayFees)
val commitments1 = d.commitments.addAnnouncement(channelAnn)
val scidForChannelUpdate = Helpers.scidForChannelUpdate(Some(channelAnn), d.aliases.localAlias)
val channelUpdate = Helpers.makeChannelUpdate(nodeParams, remoteNodeId, scidForChannelUpdate, commitments1, d.channelUpdate.relayFees)
context.system.eventStream.publish(ShortChannelIdAssigned(self, d.channelId, Some(channelAnn), d.aliases, remoteNodeId))
// We use goto() instead of stay() because we want to fire transitions.
goto(NORMAL) using d.copy(channelAnnouncement = Some(channelAnn), channelUpdate = channelUpdate) storing()
goto(NORMAL) using d.copy(commitments = commitments1, channelUpdate = channelUpdate) storing()
}
case Some(_) =>
log.debug("ignoring remote announcement_signatures for scid={}, channel is already announced", remoteAnnSigs.shortChannelId)
case None =>
log.warning("cannot generate announcement_signatures for scid={} with fundingTxIndex={}", remoteAnnSigs.shortChannelId, c.fundingTxIndex)
trimAnnouncementSigsStashIfNeeded()
announcementSigsStash += (remoteAnnSigs.shortChannelId -> remoteAnnSigs)
stay()
}
case _ =>
case None =>
// The funding transaction doesn't have enough confirmations yet: we stash the remote message and will replay
// it when we receive WatchFundingConfirmedTriggered. We don't need to persist this message, it will be
// retransmitted on reconnection.
log.debug("received remote announcement signatures for scid={}, delaying", remoteAnnSigs.shortChannelId)
if (announcementSigsStash.size >= 10) {
// We shouldn't store an unbounded number of announcement_signatures for scids that we don't have in our
// commitments, otherwise it can be used as a DoS vector.
val oldestScid = announcementSigsStash.keys.minBy(ShortChannelId.blockHeight(_))
log.warning("too many pending announcement_signatures: dropping scid={}", oldestScid)
announcementSigsStash -= oldestScid
}
trimAnnouncementSigsStashIfNeeded()
announcementSigsStash += (remoteAnnSigs.shortChannelId -> remoteAnnSigs)
stay()
}
@ -1322,7 +1334,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
d.commitments.updateLocalFundingStatus(w.tx.txid, fundingStatus) match {
case Right((commitments1, _)) =>
watchFundingConfirmed(w.tx.txid, Some(nodeParams.channelConf.minDepthFunding), delay_opt = None)
maybeEmitEventsPostSplice(d.shortIds, d.commitments, commitments1)
maybeEmitEventsPostSplice(d.aliases, d.commitments, commitments1)
maybeUpdateMaxHtlcAmount(d.channelUpdate.htlcMaximumMsat, commitments1)
stay() using d.copy(commitments = commitments1) storing() sending SpliceLocked(d.channelId, w.tx.txid)
case Left(_) => stay()
@ -1331,41 +1343,34 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
case Event(w: WatchFundingConfirmedTriggered, d: DATA_NORMAL) =>
acceptFundingTxConfirmed(w, d) match {
case Right((commitments1, commitment)) =>
// We only announce the channel creation, announcing splice transactions isn't supported yet.
if (commitment.fundingTxIndex == 0) {
val realScid = RealShortChannelId(w.blockHeight, w.txIndex, commitment.commitInput.outPoint.index.toInt)
val shortIds1 = d.shortIds.copy(real_opt = Some(realScid))
context.system.eventStream.publish(ShortChannelIdAssigned(self, d.channelId, shortIds1, remoteNodeId, d.channelAnnouncement.nonEmpty))
// If the channel is public we need to send our announcement_signatures in order to generate the channel_announcement.
val localAnnSigs_opt = if (d.commitments.announceChannel) {
Some(Helpers.makeAnnouncementSignatures(nodeParams, d.commitments.params, commitment.remoteFundingPubKey, realScid))
} else {
None
}
// If we've already received the remote announcement_signatures, we're now ready to process them.
announcementSigsStash.get(realScid).foreach(self ! _)
stay() using d.copy(shortIds = shortIds1, commitments = commitments1) storing() sending localAnnSigs_opt.toSeq
} else {
val toSend = if (d.commitments.all.exists(c => c.fundingTxId == commitment.fundingTxId && c.localFundingStatus.isInstanceOf[LocalFundingStatus.NotLocked])) {
// this commitment just moved from NotLocked to Locked
Some(SpliceLocked(d.channelId, w.tx.txid))
} else {
// this was a zero-conf splice and we already sent our splice_locked
None
}
maybeEmitEventsPostSplice(d.shortIds, d.commitments, commitments1)
// We check if this commitment was already locked before receiving the event (which happens when using 0-conf
// or for the initial funding transaction). If it was previously not locked, we must send splice_locked now.
val previouslyNotLocked = d.commitments.all.exists(c => c.fundingTxId == commitment.fundingTxId && c.localFundingStatus.isInstanceOf[LocalFundingStatus.NotLocked])
val spliceLocked_opt = if (previouslyNotLocked) Some(SpliceLocked(d.channelId, w.tx.txid)) else None
// If the channel is public and we've received the remote splice_locked, we send our announcement_signatures
// in order to generate the channel_announcement.
val remoteLocked = commitment.fundingTxIndex == 0 || d.commitments.all.exists(c => c.fundingTxId == commitment.fundingTxId && c.remoteFundingStatus == RemoteFundingStatus.Locked)
val localAnnSigs_opt = if (d.commitments.announceChannel && remoteLocked) commitment.signAnnouncement(nodeParams, d.commitments.params) else None
// If we've already received the remote announcement_signatures, we're now ready to process them.
localAnnSigs_opt.flatMap(ann => announcementSigsStash.get(ann.shortChannelId)).foreach(self ! _)
if (commitment.fundingTxIndex > 0) {
maybeEmitEventsPostSplice(d.aliases, d.commitments, commitments1)
maybeUpdateMaxHtlcAmount(d.channelUpdate.htlcMaximumMsat, commitments1)
stay() using d.copy(commitments = commitments1) storing() sending toSend.toSeq
}
stay() using d.copy(commitments = commitments1) storing() sending spliceLocked_opt.toSeq ++ localAnnSigs_opt.toSeq
case Left(_) => stay()
}
case Event(msg: SpliceLocked, d: DATA_NORMAL) =>
d.commitments.updateRemoteFundingStatus(msg.fundingTxId) match {
case Right((commitments1, _)) =>
maybeEmitEventsPostSplice(d.shortIds, d.commitments, commitments1)
case Right((commitments1, commitment)) =>
// If the commitment is confirmed, we were waiting to receive the remote splice_locked before sending our announcement_signatures.
val localAnnSigs_opt = if (d.commitments.announceChannel) commitment.signAnnouncement(nodeParams, commitments1.params) else None
// If we've already received the remote announcement_signatures, we're now ready to process them.
localAnnSigs_opt.flatMap(ann => announcementSigsStash.get(ann.shortChannelId)).foreach(self ! _)
maybeEmitEventsPostSplice(d.aliases, d.commitments, commitments1)
maybeUpdateMaxHtlcAmount(d.channelUpdate.htlcMaximumMsat, commitments1)
stay() using d.copy(commitments = commitments1) storing()
stay() using d.copy(commitments = commitments1) storing() sending localAnnSigs_opt.toSeq
case Left(_) => stay()
}
@ -2170,12 +2175,12 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
case Event(_: ChannelReestablish, d: DATA_WAIT_FOR_CHANNEL_READY) =>
log.debug("re-sending channelReady")
val channelReady = createChannelReady(d.shortIds, d.commitments.params)
val channelReady = createChannelReady(d.aliases, d.commitments.params)
goto(WAIT_FOR_CHANNEL_READY) sending channelReady
case Event(_: ChannelReestablish, d: DATA_WAIT_FOR_DUAL_FUNDING_READY) =>
log.debug("re-sending channelReady")
val channelReady = createChannelReady(d.shortIds, d.commitments.params)
val channelReady = createChannelReady(d.aliases, d.commitments.params)
goto(WAIT_FOR_DUAL_FUNDING_READY) sending channelReady
case Event(channelReestablish: ChannelReestablish, d: DATA_NORMAL) =>
@ -2273,13 +2278,13 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
sendQueue = sendQueue :+ localShutdown
}
d.shortIds.real_opt match {
case Some(realShortChannelId) if d.commitments.announceChannel && d.channelAnnouncement.isEmpty =>
d.commitments.all.find(_.fundingTxIndex == 0) match {
case Some(c) if d.commitments.announceChannel && c.shortChannelId_opt.nonEmpty && c.announcement_opt.isEmpty =>
// The funding transaction is confirmed, so we've already sent our announcement_signatures.
// We haven't announced the channel yet, which means we haven't received our peer's announcement_signatures.
// We retransmit our announcement_signatures to let our peer know that we're ready to announce the channel.
val localAnnSigs = Helpers.makeAnnouncementSignatures(nodeParams, d.commitments.params, d.commitments.latest.remoteFundingPubKey, realShortChannelId)
sendQueue = sendQueue :+ localAnnSigs
val localAnnSigs_opt = c.signAnnouncement(nodeParams, d.commitments.params)
sendQueue = sendQueue ++ localAnnSigs_opt.toSeq
case _ => ()
}
@ -2298,7 +2303,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)) {
@ -2473,7 +2477,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
// slightly before us. In that case, the WatchConfirmed may trigger first, and it would be inefficient to let the
// WatchPublished override our funding status: it will make us set a new WatchConfirmed that will instantly
// trigger and rewrite the funding status again.
val alreadyConfirmed = d.commitments.active.map(_.localFundingStatus).collect { case LocalFundingStatus.ConfirmedFundingTx(tx, _, _) => tx }.exists(_.txid == w.tx.txid)
val alreadyConfirmed = d.commitments.active.map(_.localFundingStatus).collect { case f: LocalFundingStatus.ConfirmedFundingTx => f.tx }.exists(_.txid == w.tx.txid)
if (alreadyConfirmed) {
stay()
} else {
@ -2484,8 +2488,8 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
watchFundingConfirmed(w.tx.txid, Some(nodeParams.channelConf.minDepthFunding), delay_opt = None)
val d1 = d match {
// NB: we discard remote's stashed channel_ready, they will send it back at reconnection
case d: DATA_WAIT_FOR_FUNDING_CONFIRMED => DATA_WAIT_FOR_CHANNEL_READY(commitments1, shortIds = createShortIds(d.channelId, None))
case d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments1, shortIds = createShortIds(d.channelId, None))
case d: DATA_WAIT_FOR_FUNDING_CONFIRMED => DATA_WAIT_FOR_CHANNEL_READY(commitments1, aliases = createShortIdAliases(d.channelId))
case d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments1, aliases = createShortIdAliases(d.channelId))
case d: DATA_WAIT_FOR_CHANNEL_READY => d.copy(commitments = commitments1)
case d: DATA_WAIT_FOR_DUAL_FUNDING_READY => d.copy(commitments = commitments1)
case d: DATA_NORMAL => d.copy(commitments = commitments1)
@ -2501,25 +2505,15 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
case Event(w: WatchFundingConfirmedTriggered, d: ChannelDataWithCommitments) =>
acceptFundingTxConfirmed(w, d) match {
case Right((commitments1, commitment)) =>
case Right((commitments1, _)) =>
log.info("funding txid={} has been confirmed", w.tx.txid)
val realScid = RealShortChannelId(w.blockHeight, w.txIndex, commitment.commitInput.outPoint.index.toInt)
val d1 = d match {
// NB: we discard remote's stashed channel_ready, they will send it back at reconnection
case d: DATA_WAIT_FOR_FUNDING_CONFIRMED => DATA_WAIT_FOR_CHANNEL_READY(commitments1, shortIds = createShortIds(d.channelId, Some(realScid)))
case d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments1, shortIds = createShortIds(d.channelId, Some(realScid)))
// No need to emit ShortChannelIdAssigned, we will emit it when receiving the remote channel_ready.
case d: DATA_WAIT_FOR_CHANNEL_READY => d.copy(shortIds = d.shortIds.copy(real_opt = Some(realScid)), commitments = commitments1)
case d: DATA_WAIT_FOR_DUAL_FUNDING_READY => d.copy(shortIds = d.shortIds.copy(real_opt = Some(realScid)), commitments = commitments1)
case d: DATA_NORMAL =>
if (commitment.fundingTxIndex == 0) {
val shortIds1 = d.shortIds.copy(real_opt = Some(realScid))
context.system.eventStream.publish(ShortChannelIdAssigned(self, d.channelId, shortIds1, remoteNodeId, d.channelAnnouncement.nonEmpty))
d.copy(shortIds = shortIds1, commitments = commitments1)
} else {
// We don't support announcing splice transactions yet, so we don't update our scid.
d.copy(commitments = commitments1)
}
case d: DATA_WAIT_FOR_FUNDING_CONFIRMED => DATA_WAIT_FOR_CHANNEL_READY(commitments1, aliases = createShortIdAliases(d.channelId))
case d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments1, aliases = createShortIdAliases(d.channelId))
case d: DATA_WAIT_FOR_CHANNEL_READY => d.copy(commitments = commitments1)
case d: DATA_WAIT_FOR_DUAL_FUNDING_READY => d.copy(commitments = commitments1)
case d: DATA_NORMAL => d.copy(commitments = commitments1)
case d: DATA_SHUTDOWN => d.copy(commitments = commitments1)
case d: DATA_NEGOTIATING => d.copy(commitments = commitments1)
case d: DATA_WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT => d.copy(commitments = commitments1)
@ -2607,29 +2601,32 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
val emitEvent_opt: Option[EmitLocalChannelEvent] = (state, nextState, stateData, nextStateData) match {
case (WAIT_FOR_INIT_INTERNAL, OFFLINE, _, d: DATA_NORMAL) => Some(EmitLocalChannelUpdate("restore", d, sendToPeer = false))
case (WAIT_FOR_CHANNEL_READY | WAIT_FOR_DUAL_FUNDING_READY, NORMAL, _, d: DATA_NORMAL) => Some(EmitLocalChannelUpdate("initial", d, sendToPeer = true))
case (NORMAL, NORMAL, d1: DATA_NORMAL, d2: DATA_NORMAL) if d1.shortIds.real_opt != d2.shortIds.real_opt || d1.channelUpdate != d2.channelUpdate || d1.channelAnnouncement != d2.channelAnnouncement => Some(EmitLocalChannelUpdate("normal->normal", d2, sendToPeer = d2.channelAnnouncement.isEmpty && d1.channelUpdate != d2.channelUpdate))
case (SYNCING, NORMAL, d1: DATA_NORMAL, d2: DATA_NORMAL) if d1.channelUpdate != d2.channelUpdate || d1.channelAnnouncement != d2.channelAnnouncement => Some(EmitLocalChannelUpdate("syncing->normal", d2, sendToPeer = d2.channelAnnouncement.isEmpty))
case (NORMAL, OFFLINE, d1: DATA_NORMAL, d2: DATA_NORMAL) if d1.channelUpdate != d2.channelUpdate || d1.channelAnnouncement != d2.channelAnnouncement => Some(EmitLocalChannelUpdate("normal->offline", d2, sendToPeer = false))
case (OFFLINE, OFFLINE, d1: DATA_NORMAL, d2: DATA_NORMAL) if d1.channelUpdate != d2.channelUpdate || d1.channelAnnouncement != d2.channelAnnouncement => Some(EmitLocalChannelUpdate("offline->offline", d2, sendToPeer = false))
case (NORMAL, NORMAL, d1: DATA_NORMAL, d2: DATA_NORMAL) if d1.channelUpdate != d2.channelUpdate || d1.lastAnnouncement_opt != d2.lastAnnouncement_opt => Some(EmitLocalChannelUpdate("normal->normal", d2, sendToPeer = d2.lastAnnouncement_opt.isEmpty && d1.channelUpdate != d2.channelUpdate))
case (SYNCING, NORMAL, d1: DATA_NORMAL, d2: DATA_NORMAL) if d1.channelUpdate != d2.channelUpdate || d1.lastAnnouncement_opt != d2.lastAnnouncement_opt => Some(EmitLocalChannelUpdate("syncing->normal", d2, sendToPeer = d2.lastAnnouncement_opt.isEmpty))
case (NORMAL, OFFLINE, d1: DATA_NORMAL, d2: DATA_NORMAL) if d1.channelUpdate != d2.channelUpdate || d1.lastAnnouncement_opt != d2.lastAnnouncement_opt => Some(EmitLocalChannelUpdate("normal->offline", d2, sendToPeer = false))
case (OFFLINE, OFFLINE, d1: DATA_NORMAL, d2: DATA_NORMAL) if d1.channelUpdate != d2.channelUpdate || d1.lastAnnouncement_opt != d2.lastAnnouncement_opt => Some(EmitLocalChannelUpdate("offline->offline", d2, sendToPeer = false))
case (NORMAL | SYNCING | OFFLINE, SHUTDOWN | NEGOTIATING | CLOSING | CLOSED | ERR_INFORMATION_LEAK | WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT, d: DATA_NORMAL, _) => Some(EmitLocalChannelDown(d))
case _ => None
}
emitEvent_opt.foreach {
case EmitLocalChannelUpdate(reason, d, sendToPeer) =>
log.debug(s"emitting channel update event: reason=$reason enabled=${d.channelUpdate.channelFlags.isEnabled} sendToPeer=$sendToPeer realScid=${d.shortIds.real_opt} channel_update={} channel_announcement={}", d.channelUpdate, d.channelAnnouncement.map(_ => "yes").getOrElse("no"))
val lcu = LocalChannelUpdate(self, d.channelId, d.shortIds, remoteNodeId, d.channelAnnouncement, d.channelUpdate, d.commitments)
log.debug("emitting channel update event: reason={} sendToPeer={} channel_update={} channel_announcement={}", reason, sendToPeer, d.channelUpdate, d.lastAnnouncement_opt)
val lcu = LocalChannelUpdate(self, d.channelId, d.aliases, remoteNodeId, d.lastAnnouncedCommitment_opt, d.channelUpdate, d.commitments)
context.system.eventStream.publish(lcu)
if (sendToPeer) {
send(Helpers.channelUpdateForDirectPeer(nodeParams, d.channelUpdate, d.shortIds))
// We may need a real scid to fallback to if our peer didn't send us an alias, even if we're using a private
// channel, otherwise they won't be able to match our update to a channel.
val realScid_opt = d.lastAnnouncement_opt.map(_.shortChannelId).orElse(d.commitments.all.flatMap(_.shortChannelId_opt).headOption)
send(Helpers.channelUpdateForDirectPeer(nodeParams, d.channelUpdate, realScid_opt, d.aliases))
}
case EmitLocalChannelDown(d) =>
log.debug("emitting channel down event")
if (d.channelAnnouncement.nonEmpty) {
if (d.lastAnnouncement_opt.nonEmpty) {
// We tell the rest of the network that this channel shouldn't be used anymore.
val disabledUpdate = Helpers.makeChannelUpdate(nodeParams, remoteNodeId, Helpers.scidForChannelUpdate(d), d.commitments, d.channelUpdate.relayFees, enable = false)
context.system.eventStream.publish(LocalChannelUpdate(self, d.channelId, d.shortIds, remoteNodeId, d.channelAnnouncement, disabledUpdate, d.commitments))
context.system.eventStream.publish(LocalChannelUpdate(self, d.channelId, d.aliases, remoteNodeId, d.lastAnnouncedCommitment_opt, disabledUpdate, d.commitments))
}
val lcd = LocalChannelDown(self, d.channelId, d.shortIds, remoteNodeId)
val lcd = LocalChannelDown(self, d.channelId, d.commitments.all.flatMap(_.shortChannelId_opt), d.aliases, remoteNodeId)
context.system.eventStream.publish(lcd)
}
@ -2725,7 +2722,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
@ -2751,7 +2747,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
@ -2874,10 +2869,10 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
}
/** Splices change balances and capacity, we send events to notify other actors (router, relayer) */
private def maybeEmitEventsPostSplice(shortIds: ShortIds, oldCommitments: Commitments, newCommitments: Commitments): Unit = {
private def maybeEmitEventsPostSplice(aliases: ShortIdAliases, oldCommitments: Commitments, newCommitments: Commitments): Unit = {
// NB: we consider the send and receive balance, because router tracks both
if (oldCommitments.availableBalanceForSend != newCommitments.availableBalanceForSend || oldCommitments.availableBalanceForReceive != newCommitments.availableBalanceForReceive) {
context.system.eventStream.publish(AvailableBalanceChanged(self, newCommitments.channelId, shortIds, newCommitments))
context.system.eventStream.publish(AvailableBalanceChanged(self, newCommitments.channelId, aliases, newCommitments))
}
if (oldCommitments.active.size != newCommitments.active.size) {
// Some commitments have been deactivated, which means our available balance changed, which may allow forwarding

View file

@ -29,7 +29,7 @@ import fr.acinq.eclair.channel.publish.TxPublisher.SetChannelId
import fr.acinq.eclair.crypto.ShaChain
import fr.acinq.eclair.io.Peer.{LiquidityPurchaseSigned, OpenChannelResponse}
import fr.acinq.eclair.wire.protocol._
import fr.acinq.eclair.{RealShortChannelId, ToMilliSatoshiConversion, UInt64, randomBytes32}
import fr.acinq.eclair.{ToMilliSatoshiConversion, UInt64, randomBytes32}
/**
* Created by t-bast on 19/04/2022.
@ -728,7 +728,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
case Right((commitments1, _)) =>
// we still watch the funding tx for confirmation even if we can use the zero-conf channel right away
watchFundingConfirmed(w.tx.txid, Some(nodeParams.channelConf.minDepthFunding), delay_opt = None)
val shortIds = createShortIds(d.channelId, None)
val shortIds = createShortIdAliases(d.channelId)
val channelReady = createChannelReady(shortIds, d.commitments.params)
d.deferred.foreach(self ! _)
goto(WAIT_FOR_DUAL_FUNDING_READY) using DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments1, shortIds) storing() sending channelReady
@ -737,9 +737,8 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
case Event(w: WatchFundingConfirmedTriggered, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) =>
acceptFundingTxConfirmed(w, d) match {
case Right((commitments1, commitment)) =>
val realScid = RealShortChannelId(w.blockHeight, w.txIndex, commitment.commitInput.outPoint.index.toInt)
val shortIds = createShortIds(d.channelId, Some(realScid))
case Right((commitments1, _)) =>
val shortIds = createShortIdAliases(d.channelId)
val channelReady = createChannelReady(shortIds, d.commitments.params)
reportRbfFailure(d.status, InvalidRbfTxConfirmed(d.channelId))
val toSend = d.status match {
@ -785,11 +784,8 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
when(WAIT_FOR_DUAL_FUNDING_READY)(handleExceptions {
case Event(channelReady: ChannelReady, d: DATA_WAIT_FOR_DUAL_FUNDING_READY) =>
val d1 = receiveChannelReady(d.shortIds, channelReady, d.commitments)
val annSigs_opt = d.shortIds.real_opt match {
case Some(realScid) if d.commitments.announceChannel => Some(Helpers.makeAnnouncementSignatures(nodeParams, d.commitments.params, d.commitments.latest.remoteFundingPubKey, realScid))
case _ => None
}
val d1 = receiveChannelReady(d.aliases, channelReady, d.commitments)
val annSigs_opt = d1.commitments.all.find(_.fundingTxIndex == 0).flatMap(_.signAnnouncement(nodeParams, d1.commitments.params))
goto(NORMAL) using d1 storing() sending annSigs_opt.toSeq
case Event(_: TxInitRbf, d: DATA_WAIT_FOR_DUAL_FUNDING_READY) =>

View file

@ -32,7 +32,7 @@ import fr.acinq.eclair.io.Peer.OpenChannelResponse
import fr.acinq.eclair.transactions.Transactions.TxOwner
import fr.acinq.eclair.transactions.{Scripts, Transactions}
import fr.acinq.eclair.wire.protocol.{AcceptChannel, AnnouncementSignatures, ChannelReady, ChannelTlv, Error, FundingCreated, FundingSigned, OpenChannel, TlvStream}
import fr.acinq.eclair.{Features, MilliSatoshiLong, RealShortChannelId, UInt64, randomKey, toLongId}
import fr.acinq.eclair.{Features, MilliSatoshiLong, UInt64, randomKey, toLongId}
import scodec.bits.ByteVector
import scala.util.{Failure, Success}
@ -397,7 +397,7 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers {
log.info("funding txid={} was successfully published for zero-conf channelId={}", w.tx.txid, d.channelId)
// we still watch the funding tx for confirmation even if we can use the zero-conf channel right away
watchFundingConfirmed(w.tx.txid, Some(nodeParams.channelConf.minDepthFunding), delay_opt = None)
val shortIds = createShortIds(d.channelId, None)
val shortIds = createShortIdAliases(d.channelId)
val channelReady = createChannelReady(shortIds, d.commitments.params)
d.deferred.foreach(self ! _)
goto(WAIT_FOR_CHANNEL_READY) using DATA_WAIT_FOR_CHANNEL_READY(commitments1, shortIds) storing() sending channelReady
@ -407,8 +407,7 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers {
case Event(w: WatchFundingConfirmedTriggered, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) =>
acceptFundingTxConfirmed(w, d) match {
case Right((commitments1, _)) =>
val realScid = RealShortChannelId(w.blockHeight, w.txIndex, d.commitments.latest.commitInput.outPoint.index.toInt)
val shortIds = createShortIds(d.channelId, Some(realScid))
val shortIds = createShortIdAliases(d.channelId)
val channelReady = createChannelReady(shortIds, d.commitments.params)
d.deferred.foreach(self ! _)
goto(WAIT_FOR_CHANNEL_READY) using DATA_WAIT_FOR_CHANNEL_READY(commitments1, shortIds) storing() sending channelReady
@ -439,11 +438,8 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers {
when(WAIT_FOR_CHANNEL_READY)(handleExceptions {
case Event(channelReady: ChannelReady, d: DATA_WAIT_FOR_CHANNEL_READY) =>
val d1 = receiveChannelReady(d.shortIds, channelReady, d.commitments)
val annSigs_opt = d.shortIds.real_opt match {
case Some(realScid) if d.commitments.announceChannel => Some(Helpers.makeAnnouncementSignatures(nodeParams, d.commitments.params, d.commitments.latest.remoteFundingPubKey, realScid))
case _ => None
}
val d1 = receiveChannelReady(d.aliases, channelReady, d.commitments)
val annSigs_opt = d1.commitments.all.find(_.fundingTxIndex == 0).flatMap(_.signAnnouncement(nodeParams, d1.commitments.params))
goto(NORMAL) using d1 storing() sending annSigs_opt.toSeq
case Event(remoteAnnSigs: AnnouncementSignatures, d: DATA_WAIT_FOR_CHANNEL_READY) if d.commitments.announceChannel =>

View file

@ -20,7 +20,6 @@ import akka.actor.typed.scaladsl.adapter.{TypedActorRefOps, actorRefAdapter}
import com.softwaremill.quicklens.ModifyPimp
import fr.acinq.bitcoin.ScriptFlags
import fr.acinq.bitcoin.scalacompat.{ByteVector32, Transaction, TxId}
import fr.acinq.eclair.{RealShortChannelId, ShortChannelId}
import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher._
import fr.acinq.eclair.channel.Helpers.getRelayFees
import fr.acinq.eclair.channel.LocalFundingStatus.{ConfirmedFundingTx, DualFundedUnconfirmedFundingTx, SingleFundedUnconfirmedFundingTx}
@ -28,6 +27,7 @@ import fr.acinq.eclair.channel._
import fr.acinq.eclair.channel.fsm.Channel.{BroadcastChannelUpdate, PeriodicRefresh, REFRESH_CHANNEL_UPDATE_INTERVAL}
import fr.acinq.eclair.db.RevokedHtlcInfoCleaner
import fr.acinq.eclair.wire.protocol.{AnnouncementSignatures, ChannelReady, ChannelReadyTlv, TlvStream}
import fr.acinq.eclair.{RealShortChannelId, ShortChannelId}
import scala.concurrent.duration.{DurationInt, FiniteDuration}
import scala.util.{Failure, Success, Try}
@ -82,54 +82,57 @@ trait CommonFundingHandlers extends CommonHandlers {
}
case _ => () // in the dual-funding case, we have already verified the funding tx
}
val fundingStatus = ConfirmedFundingTx(w.tx, d.commitments.localFundingSigs(w.tx.txid), d.commitments.liquidityPurchase(w.tx.txid))
context.system.eventStream.publish(TransactionConfirmed(d.channelId, remoteNodeId, w.tx))
// When a splice transaction confirms, it double-spends all the commitment transactions that only applied to the
// previous funding transaction. Our peer cannot publish the corresponding revoked commitments anymore, so we can
// clean-up the htlc data that we were storing for the matching penalty transactions.
d.commitments.all.find(_.fundingTxId == w.tx.txid).map(_.firstRemoteCommitIndex).foreach {
commitIndex => context.system.eventStream.publish(RevokedHtlcInfoCleaner.ForgetHtlcInfos(d.channelId, beforeCommitIndex = commitIndex))
}
d.commitments.updateLocalFundingStatus(w.tx.txid, fundingStatus).map {
case (commitments1, commitment) =>
// First of all, we watch the funding tx that is now confirmed.
// Children splice transactions may already spend that confirmed funding transaction.
val spliceSpendingTxs = commitments1.all.collect { case c if c.fundingTxIndex == commitment.fundingTxIndex + 1 => c.fundingTxId }
watchFundingSpent(commitment, additionalKnownSpendingTxs = spliceSpendingTxs.toSet, None)
// In the dual-funding/splicing case we can forget all other transactions (RBF attempts), they have been
// double-spent by the tx that just confirmed.
val conflictingTxs = d.commitments.active // note how we use the unpruned original commitments
.filter(c => c.fundingTxIndex == commitment.fundingTxIndex && c.fundingTxId != commitment.fundingTxId)
.map(_.localFundingStatus).collect { case fundingTx: DualFundedUnconfirmedFundingTx => fundingTx.sharedTx }
conflictingTxs.foreach(tx => blockchain ! UnwatchTxConfirmed(tx.txId))
rollbackDualFundingTxs(conflictingTxs)
(commitments1, commitment)
d.commitments.all.find(_.fundingTxId == w.tx.txid) match {
case Some(c) =>
val scid = RealShortChannelId(w.blockHeight, w.txIndex, c.commitInput.outPoint.index.toInt)
val fundingStatus = ConfirmedFundingTx(w.tx, scid, c.announcement_opt, d.commitments.localFundingSigs(w.tx.txid), d.commitments.liquidityPurchase(w.tx.txid))
// When a splice transaction confirms, it double-spends all the commitment transactions that only applied to the
// previous funding transaction. Our peer cannot publish the corresponding revoked commitments anymore, so we can
// clean-up the htlc data that we were storing for the matching penalty transactions.
context.system.eventStream.publish(RevokedHtlcInfoCleaner.ForgetHtlcInfos(d.channelId, beforeCommitIndex = c.firstRemoteCommitIndex))
d.commitments.updateLocalFundingStatus(w.tx.txid, fundingStatus).map {
case (commitments1, commitment) =>
// First of all, we watch the funding tx that is now confirmed.
// Children splice transactions may already spend that confirmed funding transaction.
val spliceSpendingTxs = commitments1.all.collect { case c if c.fundingTxIndex == commitment.fundingTxIndex + 1 => c.fundingTxId }
watchFundingSpent(commitment, additionalKnownSpendingTxs = spliceSpendingTxs.toSet, None)
// In the dual-funding/splicing case we can forget all other transactions (RBF attempts), they have been
// double-spent by the tx that just confirmed.
val conflictingTxs = d.commitments.active // note how we use the unpruned original commitments
.filter(c => c.fundingTxIndex == commitment.fundingTxIndex && c.fundingTxId != commitment.fundingTxId)
.map(_.localFundingStatus).collect { case fundingTx: DualFundedUnconfirmedFundingTx => fundingTx.sharedTx }
conflictingTxs.foreach(tx => blockchain ! UnwatchTxConfirmed(tx.txId))
rollbackDualFundingTxs(conflictingTxs)
(commitments1, commitment)
}
case None => Left(d.commitments)
}
}
def createShortIds(channelId: ByteVector32, real_opt: Option[RealShortChannelId]): ShortIds = {
def createShortIdAliases(channelId: ByteVector32): ShortIdAliases = {
// The alias will be used in our peer's channel_update message, the goal is to be able to use our channel as soon
// as it reaches the NORMAL state, before it is announced on the network.
val shortIds = ShortIds(real_opt, ShortChannelId.generateLocalAlias(), remoteAlias_opt = None)
context.system.eventStream.publish(ShortChannelIdAssigned(self, channelId, shortIds, remoteNodeId, isAnnounced = false))
shortIds
val aliases = ShortIdAliases(ShortChannelId.generateLocalAlias(), remoteAlias_opt = None)
context.system.eventStream.publish(ShortChannelIdAssigned(self, channelId, None, aliases, remoteNodeId))
aliases
}
def createChannelReady(shortIds: ShortIds, params: ChannelParams): ChannelReady = {
def createChannelReady(aliases: ShortIdAliases, params: ChannelParams): ChannelReady = {
val channelKeyPath = keyManager.keyPath(params.localParams, params.channelConfig)
val nextPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, 1)
// we always send our local alias, even if it isn't explicitly supported, that's an optional TLV anyway
ChannelReady(params.channelId, nextPerCommitmentPoint, TlvStream(ChannelReadyTlv.ShortChannelIdTlv(shortIds.localAlias)))
ChannelReady(params.channelId, nextPerCommitmentPoint, TlvStream(ChannelReadyTlv.ShortChannelIdTlv(aliases.localAlias)))
}
def receiveChannelReady(shortIds: ShortIds, channelReady: ChannelReady, commitments: Commitments): DATA_NORMAL = {
val shortIds1 = shortIds.copy(remoteAlias_opt = channelReady.alias_opt)
shortIds1.remoteAlias_opt.foreach(_ => context.system.eventStream.publish(ShortChannelIdAssigned(self, commitments.channelId, shortIds1, remoteNodeId, isAnnounced = false)))
log.info("shortIds: real={} localAlias={} remoteAlias={}", shortIds1.real_opt.getOrElse("none"), shortIds1.localAlias, shortIds1.remoteAlias_opt.getOrElse("none"))
def receiveChannelReady(aliases: ShortIdAliases, channelReady: ChannelReady, commitments: Commitments): DATA_NORMAL = {
val aliases1 = aliases.copy(remoteAlias_opt = channelReady.alias_opt)
aliases1.remoteAlias_opt.foreach(_ => context.system.eventStream.publish(ShortChannelIdAssigned(self, commitments.channelId, None, aliases1, remoteNodeId)))
log.info("shortIds: real={} localAlias={} remoteAlias={}", commitments.latest.shortChannelId_opt.getOrElse("none"), aliases1.localAlias, aliases1.remoteAlias_opt.getOrElse("none"))
// We notify that the channel is now ready to route payments.
context.system.eventStream.publish(ChannelOpened(self, remoteNodeId, commitments.channelId))
// 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 scidForChannelUpdate = Helpers.scidForChannelUpdate(channelAnnouncement_opt = None, shortIds1.localAlias)
val scidForChannelUpdate = Helpers.scidForChannelUpdate(channelAnnouncement_opt = None, aliases1.localAlias)
log.info("using shortChannelId={} for initial channel_update", scidForChannelUpdate)
val relayFees = getRelayFees(nodeParams, remoteNodeId, commitments.announceChannel)
val initialChannelUpdate = Helpers.makeChannelUpdate(nodeParams, remoteNodeId, scidForChannelUpdate, commitments, relayFees)
@ -137,7 +140,7 @@ trait CommonFundingHandlers extends CommonHandlers {
context.system.scheduler.scheduleWithFixedDelay(initialDelay = REFRESH_CHANNEL_UPDATE_INTERVAL, delay = REFRESH_CHANNEL_UPDATE_INTERVAL, receiver = self, message = BroadcastChannelUpdate(PeriodicRefresh))
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, aliases1, initialChannelUpdate, None, None, None, SpliceStatus.NoSplice)
}
def delayEarlyAnnouncementSigs(remoteAnnSigs: AnnouncementSignatures): Unit = {

View file

@ -565,15 +565,15 @@ object OnionMessageReceivedSerializer extends ConvertClassSerializer[OnionMessag
// @formatter:off
/** this is cosmetic, just to not have a '_opt' field in json, which will only appear if the option is defined anyway */
private case class ShortIdsJson(real: Option[RealShortChannelId], localAlias: Alias, remoteAlias: Option[ShortChannelId])
object ShortIdsSerializer extends ConvertClassSerializer[ShortIds](s => ShortIdsJson(s.real_opt, s.localAlias, s.remoteAlias_opt))
private case class ShortIdAliasesJson(localAlias: Alias, remoteAlias: Option[ShortChannelId])
object ShortIdAliasesSerializer extends ConvertClassSerializer[ShortIdAliases](s => ShortIdAliasesJson(s.localAlias, s.remoteAlias_opt))
// @formatter:on
// @formatter:off
private case class FundingTxStatusJson(status: String, txid: Option[TxId])
private case class FundingTxStatusJson(status: String, txid: Option[TxId], shortChannelId: Option[RealShortChannelId], announcement: Option[ChannelAnnouncement])
object FundingTxStatusSerializer extends ConvertClassSerializer[LocalFundingStatus]({
case s: LocalFundingStatus.UnconfirmedFundingTx => FundingTxStatusJson("unconfirmed", s.signedTx_opt.map(_.txid))
case s: LocalFundingStatus.ConfirmedFundingTx => FundingTxStatusJson("confirmed", s.signedTx_opt.map(_.txid))
case s: LocalFundingStatus.UnconfirmedFundingTx => FundingTxStatusJson("unconfirmed", s.signedTx_opt.map(_.txid), None, None)
case s: LocalFundingStatus.ConfirmedFundingTx => FundingTxStatusJson("confirmed", s.signedTx_opt.map(_.txid), Some(s.shortChannelId), s.announcement_opt)
})
// @formatter:on
@ -719,7 +719,7 @@ object JsonSerializers {
PeerInfoSerializer +
PaymentFailedSummarySerializer +
OnionMessageReceivedSerializer +
ShortIdsSerializer +
ShortIdAliasesSerializer +
FundingTxStatusSerializer +
CommitmentSerializer +
TlvStreamSerializer +

View file

@ -136,8 +136,8 @@ class ChannelRelay private(nodeParams: NodeParams,
private val requestedChannelId_opt = r.payload.outgoing match {
case Left(_) => None
case Right(outgoingChannelId) => channels.collectFirst {
case (channelId, channel) if channel.shortIds.localAlias == outgoingChannelId => channelId
case (channelId, channel) if channel.shortIds.real_opt.contains(outgoingChannelId) => channelId
case (channelId, channel) if channel.aliases.localAlias == outgoingChannelId => channelId
case (channelId, channel) if channel.realScid_opt.contains(outgoingChannelId) => channelId
}
}

View file

@ -94,8 +94,8 @@ object ChannelRelayer {
replyTo ! Relayer.OutgoingChannels(selected.toSeq)
Behaviors.same
case WrappedLocalChannelUpdate(lcu@LocalChannelUpdate(_, channelId, shortIds, remoteNodeId, _, channelUpdate, commitments)) =>
context.log.debug(s"updating local channel info for channelId=$channelId realScid=${shortIds.real_opt} localAlias=${shortIds.localAlias} remoteNodeId=$remoteNodeId channelUpdate={} commitments={}", channelUpdate, commitments)
case WrappedLocalChannelUpdate(lcu@LocalChannelUpdate(_, channelId, shortIds, remoteNodeId, announcement_opt, channelUpdate, commitments)) =>
context.log.debug("updating local channel info for channelId={} realScid={} localAlias={} remoteNodeId={} channelUpdate={}", channelId, announcement_opt.map(_.shortChannelId), shortIds.localAlias, remoteNodeId, channelUpdate)
val prevChannelUpdate = channels.get(channelId).map(_.channelUpdate)
val channel = Relayer.OutgoingChannel(shortIds, remoteNodeId, channelUpdate, prevChannelUpdate, commitments)
val channels1 = channels + (channelId -> channel)
@ -105,17 +105,17 @@ object ChannelRelayer {
val node2channels1 = node2channels.addOne(remoteNodeId, channelId)
apply(nodeParams, register, channels1, scid2channels1, node2channels1)
case WrappedLocalChannelDown(LocalChannelDown(_, channelId, shortIds, remoteNodeId)) =>
context.log.debug(s"removed local channel info for channelId=$channelId localAlias=${shortIds.localAlias}")
case WrappedLocalChannelDown(LocalChannelDown(_, channelId, realScids, aliases, remoteNodeId)) =>
context.log.debug("removed local channel info for channelId={} localAlias={}", channelId, aliases.localAlias)
val channels1 = channels - channelId
val scid2Channels1 = scid2channels - shortIds.localAlias -- shortIds.real_opt
val scid2Channels1 = scid2channels - aliases.localAlias -- realScids
val node2channels1 = node2channels.subtractOne(remoteNodeId, channelId)
apply(nodeParams, register, channels1, scid2Channels1, node2channels1)
case WrappedAvailableBalanceChanged(AvailableBalanceChanged(_, channelId, shortIds, commitments)) =>
case WrappedAvailableBalanceChanged(AvailableBalanceChanged(_, channelId, aliases, commitments)) =>
val channels1 = channels.get(channelId) match {
case Some(c: Relayer.OutgoingChannel) =>
context.log.debug(s"available balance changed for channelId=$channelId localAlias=${shortIds.localAlias} availableForSend={} availableForReceive={}", commitments.availableBalanceForSend, commitments.availableBalanceForReceive)
context.log.debug("available balance changed for channelId={} localAlias={} availableForSend={} availableForReceive={}", channelId, aliases.localAlias, commitments.availableBalanceForSend, commitments.availableBalanceForReceive)
channels + (channelId -> c.copy(commitments = commitments))
case None => channels // we only consider the balance if we have the channel_update
}

View file

@ -29,7 +29,7 @@ import fr.acinq.eclair.channel._
import fr.acinq.eclair.db.PendingCommandsDb
import fr.acinq.eclair.payment._
import fr.acinq.eclair.wire.protocol._
import fr.acinq.eclair.{CltvExpiryDelta, Logs, MilliSatoshi, NodeParams}
import fr.acinq.eclair.{CltvExpiryDelta, Logs, MilliSatoshi, NodeParams, RealShortChannelId}
import grizzled.slf4j.Logging
import scala.concurrent.Promise
@ -151,7 +151,7 @@ object Relayer extends Logging {
}
case class RelayForward(add: UpdateAddHtlc, originNode: PublicKey)
case class ChannelBalance(remoteNodeId: PublicKey, shortIds: ShortIds, canSend: MilliSatoshi, canReceive: MilliSatoshi, isPublic: Boolean, isEnabled: Boolean)
case class ChannelBalance(remoteNodeId: PublicKey, realScid: Option[RealShortChannelId], aliases: ShortIdAliases, canSend: MilliSatoshi, canReceive: MilliSatoshi, isPublic: Boolean, isEnabled: Boolean)
sealed trait OutgoingChannelParams {
def channelId: ByteVector32
@ -165,11 +165,13 @@ object Relayer extends Logging {
* @param enabledOnly if true, filter out disabled channels.
*/
case class GetOutgoingChannels(enabledOnly: Boolean = true)
case class OutgoingChannel(shortIds: ShortIds, nextNodeId: PublicKey, channelUpdate: ChannelUpdate, prevChannelUpdate: Option[ChannelUpdate], commitments: Commitments) extends OutgoingChannelParams {
case class OutgoingChannel(aliases: ShortIdAliases, nextNodeId: PublicKey, channelUpdate: ChannelUpdate, prevChannelUpdate: Option[ChannelUpdate], commitments: Commitments) extends OutgoingChannelParams {
override val channelId: ByteVector32 = commitments.channelId
val realScid_opt: Option[RealShortChannelId] = commitments.lastAnnouncement_opt.map(_.shortChannelId)
def toChannelBalance: ChannelBalance = ChannelBalance(
remoteNodeId = nextNodeId,
shortIds = shortIds,
realScid = realScid_opt,
aliases = aliases,
canSend = commitments.availableBalanceForSend,
canReceive = commitments.availableBalanceForReceive,
isPublic = commitments.announceChannel,

View file

@ -41,7 +41,7 @@ object RouteCalculation {
// We are the destination of that edge: local graph edges always use either the local alias or the real scid.
// We want to use the remote alias when available, because our peer won't understand our local alias.
d.resolve(e.desc.shortChannelId) match {
case Some(c: PrivateChannel) => c.shortIds.remoteAlias_opt.getOrElse(e.desc.shortChannelId)
case Some(c: PrivateChannel) => c.aliases.remoteAlias_opt.getOrElse(e.desc.shortChannelId)
case _ => e.desc.shortChannelId
}
} else {
@ -94,8 +94,8 @@ object RouteCalculation {
case _ => None
}
case Some(c: PrivateChannel) => currentNode match {
case c.nodeId1 => Some(ChannelDesc(c.shortIds.localAlias, c.nodeId1, c.nodeId2))
case c.nodeId2 => Some(ChannelDesc(c.shortIds.localAlias, c.nodeId2, c.nodeId1))
case c.nodeId1 => Some(ChannelDesc(c.aliases.localAlias, c.nodeId1, c.nodeId2))
case c.nodeId2 => Some(ChannelDesc(c.aliases.localAlias, c.nodeId2, c.nodeId1))
case _ => None
}
case None => extraEdges.find(e => e.desc.shortChannelId == shortChannelId && e.desc.a == currentNode).map(_.desc)

View file

@ -393,7 +393,7 @@ object Router {
}
def apply(u: ChannelUpdate, pc: PrivateChannel): ChannelDesc = {
// the least significant bit tells us if it is node1 or node2
if (u.channelFlags.isNode1) ChannelDesc(pc.shortIds.localAlias, pc.nodeId1, pc.nodeId2) else ChannelDesc(pc.shortIds.localAlias, pc.nodeId2, pc.nodeId1)
if (u.channelFlags.isNode1) ChannelDesc(pc.aliases.localAlias, pc.nodeId1, pc.nodeId2) else ChannelDesc(pc.aliases.localAlias, pc.nodeId2, pc.nodeId1)
}
}
case class ChannelMeta(balance1: MilliSatoshi, balance2: MilliSatoshi)
@ -431,7 +431,7 @@ object Router {
case Right(rcu) => updateChannelUpdateSameSideAs(rcu.channelUpdate)
}
}
case class PrivateChannel(channelId: ByteVector32, shortIds: ShortIds, localNodeId: PublicKey, remoteNodeId: PublicKey, update_1_opt: Option[ChannelUpdate], update_2_opt: Option[ChannelUpdate], meta: ChannelMeta) extends KnownChannel {
case class PrivateChannel(channelId: ByteVector32, aliases: ShortIdAliases, localNodeId: PublicKey, remoteNodeId: PublicKey, update_1_opt: Option[ChannelUpdate], update_2_opt: Option[ChannelUpdate], meta: ChannelMeta) extends KnownChannel {
val (nodeId1, nodeId2) = if (Announcements.isNode1(localNodeId, remoteNodeId)) (localNodeId, remoteNodeId) else (remoteNodeId, localNodeId)
val capacity: Satoshi = (meta.balance1 + meta.balance2).truncateToSatoshi
@ -448,18 +448,17 @@ object Router {
case Left(lcu) => updateChannelUpdateSameSideAs(lcu.channelUpdate).updateBalances(lcu.commitments)
case Right(rcu) => updateChannelUpdateSameSideAs(rcu.channelUpdate)
}
/** Create an invoice routing hint from that channel. Note that if the channel is private, the invoice will leak its existence. */
/** Create an invoice routing hint for that channel. */
def toIncomingExtraHop: Option[Bolt11Invoice.ExtraHop] = {
// we want the incoming channel_update
// We use the incoming channel_update because we want to route to ourselves.
val remoteUpdate_opt = if (localNodeId == nodeId1) update_2_opt else update_1_opt
// for incoming payments we preferably use the *remote alias*, otherwise the real scid if we have it
val scid_opt = shortIds.remoteAlias_opt.orElse(shortIds.real_opt)
// we override the remote update's scid, because it contains either the real scid or our local alias
scid_opt.flatMap { scid =>
remoteUpdate_opt.map { remoteUpdate =>
Bolt11Invoice.ExtraHop(remoteNodeId, scid, remoteUpdate.feeBaseMsat, remoteUpdate.feeProportionalMillionths, remoteUpdate.cltvExpiryDelta)
}
}
// They sent us their channel update using either the real scid or our local alias (to allow us to map the update
// to the right channel): we need to override it with the alias *they* are using for that channel. If they didn't
// provide one, we won't use this channel as routing hint, because it would leak its short_channel_id.
for {
scid <- aliases.remoteAlias_opt
update <- remoteUpdate_opt
} yield (Bolt11Invoice.ExtraHop(remoteNodeId, scid, update.feeBaseMsat, update.feeProportionalMillionths, update.cltvExpiryDelta))
}
}
// @formatter:on

View file

@ -232,7 +232,7 @@ object Validation {
// mutable variable is simpler here
var graph = d.graphWithBalances
// remove previous private edges
graph = graph.removeChannel(ChannelDesc(privateChannel.shortIds.localAlias, privateChannel.nodeId1, privateChannel.nodeId2))
graph = graph.removeChannel(ChannelDesc(privateChannel.aliases.localAlias, privateChannel.nodeId1, privateChannel.nodeId2))
// add new public edges
pubChan.update_1_opt.foreach(u => graph = graph.addEdge(GraphEdge(u, pubChan)))
pubChan.update_2_opt.foreach(u => graph = graph.addEdge(GraphEdge(u, pubChan)))
@ -249,7 +249,7 @@ object Validation {
// we remove the corresponding unannounced channel that we may have until now
privateChannels = d.privateChannels -- privChan_opt.map(_.channelId).toSeq,
// we also remove the scid -> channelId mappings
scid2PrivateChannels = d.scid2PrivateChannels - pubChan.shortChannelId.toLong -- privChan_opt.map(_.shortIds.localAlias.toLong),
scid2PrivateChannels = d.scid2PrivateChannels - pubChan.shortChannelId.toLong -- privChan_opt.map(_.aliases.localAlias.toLong),
// we also add the newly validated channels to the rebroadcast queue
rebroadcast = d.rebroadcast.copy(
// we rebroadcast the channel to our peers
@ -363,7 +363,7 @@ object Validation {
def handleChannelUpdate(d: Data, db: NetworkDb, currentBlockHeight: BlockHeight, update: Either[LocalChannelUpdate, RemoteChannelUpdate], wasStashed: Boolean = false)(implicit ctx: ActorContext, log: LoggingAdapter): Data = {
implicit val sender: ActorRef = ctx.self // necessary to preserve origin when sending messages to other actors
val (pc_opt: Option[KnownChannel], u: ChannelUpdate, origins: Set[GossipOrigin]) = update match {
case Left(lcu) => (d.resolve(lcu.channelId, lcu.shortIds.real_opt), lcu.channelUpdate, Set(LocalGossip))
case Left(lcu) => (d.resolve(lcu.channelId, lcu.announcement_opt.map(_.shortChannelId)), lcu.channelUpdate, Set(LocalGossip))
case Right(rcu) =>
rcu.origins.collect {
case RemoteGossip(peerConnection, _) if !wasStashed => // stashed changes have already been acknowledged
@ -541,17 +541,17 @@ object Validation {
def handleShortChannelIdAssigned(d: Data, localNodeId: PublicKey, scia: ShortChannelIdAssigned)(implicit ctx: ActorContext, log: LoggingAdapter): Data = {
implicit val sender: ActorRef = ctx.self // necessary to preserve origin when sending messages to other actors
// NB: we don't map remote aliases because they are decided by our peer and could overlap with ours.
val mappings = scia.shortIds.real_opt match {
case Some(realScid) => Map(realScid.toLong -> scia.channelId, scia.shortIds.localAlias.toLong -> scia.channelId)
case None => Map(scia.shortIds.localAlias.toLong -> scia.channelId)
val mappings = scia.announcement_opt.map(_.shortChannelId) match {
case Some(realScid) => Map(realScid.toLong -> scia.channelId, scia.aliases.localAlias.toLong -> scia.channelId)
case None => Map(scia.aliases.localAlias.toLong -> scia.channelId)
}
log.debug("handleShortChannelIdAssigned scia={} mappings={}", scia, mappings)
val d1 = d.copy(scid2PrivateChannels = d.scid2PrivateChannels ++ mappings)
d1.resolve(scia.channelId, scia.shortIds.real_opt) match {
d1.resolve(scia.channelId, scia.announcement_opt.map(_.shortChannelId)) match {
case Some(_) =>
// This channel is already known, nothing more to do.
d1
case None if scia.isAnnounced =>
case None if scia.announcement_opt.nonEmpty =>
// This channel has been announced: it must be a public channel for which we haven't processed the announcement
// yet (or a splice that updates the real scid). We don't have anything to do, the scid will be updated in the
// public channels map when we process the announcement.
@ -563,8 +563,8 @@ object Validation {
// 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 enough confirmations). We create a corresponding private channel in both cases,
// which will be converted to a public channel later if it is announced.
log.debug("adding unannounced local channel to remote={} channelId={} localAlias={}", scia.remoteNodeId, scia.channelId, scia.shortIds.localAlias)
val pc = PrivateChannel(scia.channelId, scia.shortIds, localNodeId, scia.remoteNodeId, None, None, ChannelMeta(0 msat, 0 msat))
log.debug("adding unannounced local channel to remote={} channelId={} localAlias={}", scia.remoteNodeId, scia.channelId, scia.aliases.localAlias)
val pc = PrivateChannel(scia.channelId, scia.aliases, localNodeId, scia.remoteNodeId, None, None, ChannelMeta(0 msat, 0 msat))
d1.copy(privateChannels = d1.privateChannels + (scia.channelId -> pc))
}
}
@ -573,45 +573,47 @@ object Validation {
implicit val sender: ActorRef = ctx.self // necessary to preserve origin when sending messages to other actors
import nodeParams.db.{network => db}
log.debug("handleLocalChannelUpdate lcu={}", lcu)
d.resolve(lcu.channelId, lcu.shortIds.real_opt) match {
d.resolve(lcu.channelId, lcu.announcement_opt.map(_.shortChannelId)) match {
case Some(publicChannel: PublicChannel) =>
// This a known public channel, we only need to process the channel_update.
log.debug("this is a known public channel, processing channel_update publicChannel={}", publicChannel)
handleChannelUpdate(d, db, nodeParams.currentBlockHeight, Left(lcu))
case Some(privateChannel: PrivateChannel) =>
lcu.channelAnnouncement_opt match {
lcu.announcement_opt match {
case Some(ann) =>
log.debug("private channel graduating to public privateChannel={}", privateChannel)
// This channel is now being announced and thus graduating from private to public.
// Since this is a local channel, we can trust the announcement, no need to verify the utxo.
val fundingTxId = lcu.commitments.latest.fundingTxId
val d1 = addPublicChannel(d, nodeParams, watcher, ann, fundingTxId, lcu.commitments.latest.capacity, Some(privateChannel))
val d1 = addPublicChannel(d, nodeParams, watcher, ann.announcement, ann.fundingTxId, lcu.commitments.capacity, Some(privateChannel))
handleChannelUpdate(d1, db, nodeParams.currentBlockHeight, Left(lcu))
case None =>
log.debug("this is a known private channel, processing channel_update privateChannel={}", privateChannel)
// This a known private channel, we update the short ids and the balances.
val pc1 = privateChannel.copy(shortIds = lcu.shortIds).updateBalances(lcu.commitments)
val pc1 = privateChannel.copy(aliases = lcu.aliases).updateBalances(lcu.commitments)
val d1 = d.copy(privateChannels = d.privateChannels + (privateChannel.channelId -> pc1))
handleChannelUpdate(d1, db, nodeParams.currentBlockHeight, Left(lcu))
}
case None =>
lcu.shortIds.real_opt match {
case Some(realScid) if d.prunedChannels.contains(realScid) =>
log.debug("this is a known pruned local channel, processing channel_update for channelId={} scid={}", lcu.channelId, realScid)
lcu.announcement_opt match {
case Some(ann) if d.prunedChannels.contains(ann.shortChannelId) =>
log.debug("this is a known pruned local channel, processing channel_update for channelId={} scid={}", lcu.channelId, ann.shortChannelId)
handleChannelUpdate(d, db, nodeParams.currentBlockHeight, Left(lcu))
case _ =>
// This may be a public channel for which we haven't processed the channel_announcement yet.
// If we have a channel announcement, we first add it to the channels list and then process the update.
val d1 = lcu.channelAnnouncement_opt match {
case Some(ann) =>
val fundingTxId = lcu.commitments.latest.fundingTxId
addPublicChannel(d, nodeParams, watcher, ann, fundingTxId, lcu.commitments.latest.capacity, None)
case Some(ann) =>
val d1 = d.spentChannels.get(ann.fundingTxId).flatMap(parentScid => d.channels.get(parentScid)) match {
case Some(parentChannel) =>
// This is a splice for which we haven't processed the (local) channel_announcement yet.
log.debug("processing channel_announcement for local splice with fundingTxId={} channelId={} scid={} (previous={})", ann.fundingTxId, lcu.channelId, ann.shortChannelId, parentChannel.shortChannelId)
updateSplicedPublicChannel(d, nodeParams, watcher, ann.announcement, ann.fundingTxId, lcu.commitments.capacity, parentChannel)
case None =>
log.warning("unrecognized local channel update for private channelId={} localAlias={}", lcu.channelId, lcu.shortIds.localAlias)
d
// This is a public channel for which we haven't processed the (local) channel_announcement yet.
log.debug("processing channel_announcement for unknown local channel with fundingTxId={} channelId={} scid={}", ann.fundingTxId, lcu.channelId, ann.shortChannelId)
addPublicChannel(d, nodeParams, watcher, ann.announcement, ann.fundingTxId, lcu.commitments.capacity, None)
}
// Process the update: it will be rejected if there is no related channel.
handleChannelUpdate(d1, db, nodeParams.currentBlockHeight, Left(lcu))
case None =>
log.warning("unrecognized local channel update for private channelId={} localAlias={}", lcu.channelId, lcu.aliases.localAlias)
// Process the update: it will be rejected if there is no related channel.
handleChannelUpdate(d, db, nodeParams.currentBlockHeight, Left(lcu))
}
}
}
@ -619,15 +621,15 @@ object Validation {
def handleLocalChannelDown(d: Data, localNodeId: PublicKey, lcd: LocalChannelDown)(implicit log: LoggingAdapter): Data = {
import lcd.{channelId, remoteNodeId}
log.debug("handleLocalChannelDown lcd={}", lcd)
val scid2PrivateChannels1 = d.scid2PrivateChannels - lcd.shortIds.localAlias.toLong -- lcd.shortIds.real_opt.map(_.toLong)
val scid2PrivateChannels1 = d.scid2PrivateChannels - lcd.aliases.localAlias.toLong -- lcd.realScids.map(_.toLong)
// a local channel has permanently gone down
if (lcd.shortIds.real_opt.exists(d.channels.contains)) {
if (lcd.realScids.exists(d.channels.contains)) {
// the channel was public, we will receive (or have already received) a WatchSpent event, that will trigger a clean up of the channel
// so let's not do anything here
d.copy(scid2PrivateChannels = scid2PrivateChannels1)
} else if (d.privateChannels.contains(lcd.channelId)) {
// the channel was private or public-but-not-yet-announced, let's do the clean up
val localAlias = d.privateChannels(channelId).shortIds.localAlias
val localAlias = d.privateChannels(channelId).aliases.localAlias
log.debug("removing private local channel and channel_update for channelId={} localAlias={}", channelId, localAlias)
// we remove the corresponding updates from the graph
val graphWithBalances1 = d.graphWithBalances
@ -640,7 +642,7 @@ object Validation {
}
def handleAvailableBalanceChanged(d: Data, e: AvailableBalanceChanged)(implicit log: LoggingAdapter): Data = {
val (publicChannels1, graphWithBalances1) = e.shortIds.real_opt.flatMap(d.channels.get) match {
val (publicChannels1, graphWithBalances1) = e.commitments.lastAnnouncement_opt.map(_.shortChannelId).flatMap(d.channels.get) match {
case Some(pc) =>
val pc1 = pc.updateBalances(e.commitments)
log.debug("public channel balance updated: {}", pc1)

View file

@ -388,7 +388,7 @@ private[channel] object ChannelCodecs0 {
("shortChannelId" | realshortchannelid) ::
("lastSent" | channelReadyCodec)).map {
case commitments :: shortChannelId :: _ :: HNil =>
DATA_WAIT_FOR_CHANNEL_READY(commitments, shortIds = ShortIds(real_opt = Some(shortChannelId), localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None))
DATA_WAIT_FOR_CHANNEL_READY(commitments, aliases = ShortIdAliases(localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None))
}.decodeOnly
val shutdownCodec: Codec[Shutdown] = (
@ -406,8 +406,9 @@ private[channel] object ChannelCodecs0 {
("localShutdown" | optional(bool, shutdownCodec)) ::
("remoteShutdown" | optional(bool, shutdownCodec)) ::
("closingFeerates" | provide(Option.empty[ClosingFeerates]))).map {
case commitments :: shortChannelId :: _ :: channelAnnouncement :: channelUpdate :: localShutdown :: remoteShutdown :: closingFeerates :: HNil =>
DATA_NORMAL(commitments, shortIds = ShortIds(real_opt = Some(shortChannelId), localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None), channelAnnouncement, channelUpdate, localShutdown, remoteShutdown, closingFeerates, SpliceStatus.NoSplice)
case commitments :: shortChannelId :: _ :: _ :: channelUpdate :: localShutdown :: remoteShutdown :: closingFeerates :: HNil =>
val aliases = ShortIdAliases(localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None)
DATA_NORMAL(commitments, aliases, channelUpdate, localShutdown, remoteShutdown, closingFeerates, SpliceStatus.NoSplice)
}.decodeOnly
val DATA_NORMAL_10_Codec: Codec[DATA_NORMAL] = (
@ -419,8 +420,9 @@ private[channel] object ChannelCodecs0 {
("localShutdown" | optional(bool, shutdownCodec)) ::
("remoteShutdown" | optional(bool, shutdownCodec)) ::
("closingFeerates" | provide(Option.empty[ClosingFeerates]))).map {
case commitments :: shortChannelId :: _ :: channelAnnouncement :: channelUpdate :: localShutdown :: remoteShutdown :: closingFeerates :: HNil =>
DATA_NORMAL(commitments, shortIds = ShortIds(real_opt = Some(shortChannelId), localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None), channelAnnouncement, channelUpdate, localShutdown, remoteShutdown, closingFeerates, SpliceStatus.NoSplice)
case commitments :: shortChannelId :: _ :: _ :: channelUpdate :: localShutdown :: remoteShutdown :: closingFeerates :: HNil =>
val aliases = ShortIdAliases(localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None)
DATA_NORMAL(commitments, aliases, channelUpdate, localShutdown, remoteShutdown, closingFeerates, SpliceStatus.NoSplice)
}.decodeOnly
val DATA_SHUTDOWN_04_Codec: Codec[DATA_SHUTDOWN] = (

View file

@ -251,7 +251,7 @@ private[channel] object ChannelCodecs1 {
("shortChannelId" | realshortchannelid) ::
("lastSent" | lengthDelimited(channelReadyCodec))).map {
case commitments :: shortChannelId :: _ :: HNil =>
DATA_WAIT_FOR_CHANNEL_READY(commitments, shortIds = ShortIds(real_opt = Some(shortChannelId), localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None))
DATA_WAIT_FOR_CHANNEL_READY(commitments, aliases = ShortIdAliases(localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None))
}.decodeOnly
val DATA_NORMAL_22_Codec: Codec[DATA_NORMAL] = (
@ -263,8 +263,9 @@ private[channel] object ChannelCodecs1 {
("localShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) ::
("remoteShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) ::
("closingFeerates" | provide(Option.empty[ClosingFeerates]))).map {
case commitments :: shortChannelId :: _ :: channelAnnouncement :: channelUpdate :: localShutdown :: remoteShutdown :: closingFeerates :: HNil =>
DATA_NORMAL(commitments, shortIds = ShortIds(real_opt = Some(shortChannelId), localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None), channelAnnouncement, channelUpdate, localShutdown, remoteShutdown, closingFeerates, SpliceStatus.NoSplice)
case commitments :: shortChannelId :: _ :: _ :: channelUpdate :: localShutdown :: remoteShutdown :: closingFeerates :: HNil =>
val aliases = ShortIdAliases(localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None)
DATA_NORMAL(commitments, aliases, channelUpdate, localShutdown, remoteShutdown, closingFeerates, SpliceStatus.NoSplice)
}.decodeOnly
val DATA_SHUTDOWN_23_Codec: Codec[DATA_SHUTDOWN] = (

View file

@ -278,7 +278,7 @@ private[channel] object ChannelCodecs2 {
("shortChannelId" | realshortchannelid) ::
("lastSent" | lengthDelimited(channelReadyCodec))).map {
case commitments :: shortChannelId :: _ :: HNil =>
DATA_WAIT_FOR_CHANNEL_READY(commitments, shortIds = ShortIds(real_opt = Some(shortChannelId), localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None))
DATA_WAIT_FOR_CHANNEL_READY(commitments, aliases = ShortIdAliases(localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None))
}.decodeOnly
val DATA_NORMAL_02_Codec: Codec[DATA_NORMAL] = (
@ -290,8 +290,9 @@ private[channel] object ChannelCodecs2 {
("localShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) ::
("remoteShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) ::
("closingFeerates" | provide(Option.empty[ClosingFeerates]))).map {
case commitments :: shortChannelId :: _ :: channelAnnouncement :: channelUpdate :: localShutdown :: remoteShutdown :: closingFeerates :: HNil =>
DATA_NORMAL(commitments, shortIds = ShortIds(real_opt = Some(shortChannelId), localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None), channelAnnouncement, channelUpdate, localShutdown, remoteShutdown, closingFeerates, SpliceStatus.NoSplice)
case commitments :: shortChannelId :: _ :: _ :: channelUpdate :: localShutdown :: remoteShutdown :: closingFeerates :: HNil =>
val aliases = ShortIdAliases(localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None)
DATA_NORMAL(commitments, aliases, channelUpdate, localShutdown, remoteShutdown, closingFeerates, SpliceStatus.NoSplice)
}.decodeOnly
val DATA_SHUTDOWN_03_Codec: Codec[DATA_SHUTDOWN] = (

View file

@ -327,6 +327,14 @@ private[channel] object ChannelCodecs3 {
("claimHtlcDelayedPenaltyTxs" | listOfN(uint16, claimHtlcDelayedOutputPenaltyTxCodec)) ::
("spent" | spentMapCodec)).as[RevokedCommitPublished]
private val shortids: Codec[ShortIdAliases] = (
("real_opt" | optional(bool8, realshortchannelid)) ::
("localAlias" | discriminated[Alias].by(uint16).typecase(1, alias)) ::
("remoteAlias_opt" | optional(bool8, alias))
).map {
case _ :: localAlias :: remoteAlias_opt :: HNil => ShortIdAliases(localAlias, remoteAlias_opt)
}.decodeOnly
val DATA_WAIT_FOR_FUNDING_CONFIRMED_00_Codec: Codec[DATA_WAIT_FOR_FUNDING_CONFIRMED] = (
("commitments" | commitmentsCodec) ::
("fundingTx_opt" | optional(bool8, txCodec)) ::
@ -343,7 +351,7 @@ private[channel] object ChannelCodecs3 {
("shortChannelId" | realshortchannelid) ::
("lastSent" | lengthDelimited(channelReadyCodec))).map {
case commitments :: shortChannelId :: _ :: HNil =>
DATA_WAIT_FOR_CHANNEL_READY(commitments, shortIds = ShortIds(real_opt = Some(shortChannelId), localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None))
DATA_WAIT_FOR_CHANNEL_READY(commitments, aliases = ShortIdAliases(localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None))
}.decodeOnly
val DATA_WAIT_FOR_CHANNEL_READY_0a_Codec: Codec[DATA_WAIT_FOR_CHANNEL_READY] = (
@ -366,8 +374,9 @@ private[channel] object ChannelCodecs3 {
("localShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) ::
("remoteShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) ::
("closingFeerates" | provide(Option.empty[ClosingFeerates]))).map {
case commitments :: shortChannelId :: _ :: channelAnnouncement :: channelUpdate :: localShutdown :: remoteShutdown :: closingFeerates :: HNil =>
DATA_NORMAL(commitments, shortIds = ShortIds(real_opt = Some(shortChannelId), localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None), channelAnnouncement, channelUpdate, localShutdown, remoteShutdown, closingFeerates, SpliceStatus.NoSplice)
case commitments :: shortChannelId :: _ :: _ :: channelUpdate :: localShutdown :: remoteShutdown :: closingFeerates :: HNil =>
val aliases = ShortIdAliases(localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None)
DATA_NORMAL(commitments, aliases, channelUpdate, localShutdown, remoteShutdown, closingFeerates, SpliceStatus.NoSplice)
}.decodeOnly
val DATA_NORMAL_07_Codec: Codec[DATA_NORMAL] = (
@ -379,8 +388,9 @@ private[channel] object ChannelCodecs3 {
("localShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) ::
("remoteShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) ::
("closingFeerates" | optional(bool8, closingFeeratesCodec))).map {
case commitments :: shortChannelId :: _ :: channelAnnouncement :: channelUpdate :: localShutdown :: remoteShutdown :: closingFeerates :: HNil =>
DATA_NORMAL(commitments, shortIds = ShortIds(real_opt = Some(shortChannelId), localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None), channelAnnouncement, channelUpdate, localShutdown, remoteShutdown, closingFeerates, SpliceStatus.NoSplice)
case commitments :: shortChannelId :: _ :: _ :: channelUpdate :: localShutdown :: remoteShutdown :: closingFeerates :: HNil =>
val aliases = ShortIdAliases(localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None)
DATA_NORMAL(commitments, aliases, channelUpdate, localShutdown, remoteShutdown, closingFeerates, SpliceStatus.NoSplice)
}.decodeOnly
val DATA_NORMAL_09_Codec: Codec[DATA_NORMAL] = (
@ -391,7 +401,10 @@ private[channel] object ChannelCodecs3 {
("localShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) ::
("remoteShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) ::
("closingFeerates" | optional(bool8, closingFeeratesCodec)) ::
("spliceStatus" | provide[SpliceStatus](SpliceStatus.NoSplice))).as[DATA_NORMAL]
("spliceStatus" | provide[SpliceStatus](SpliceStatus.NoSplice))).map {
case commitments :: shortIds :: _ :: channelUpdate :: localShutdown :: remoteShutdown :: closingFeerates :: spliceStatus :: HNil =>
DATA_NORMAL(commitments, shortIds, channelUpdate, localShutdown, remoteShutdown, closingFeerates, spliceStatus)
}.decodeOnly
val DATA_SHUTDOWN_03_Codec: Codec[DATA_SHUTDOWN] = (
("commitments" | commitmentsCodec) ::

View file

@ -14,8 +14,8 @@ import fr.acinq.eclair.transactions.Transactions._
import fr.acinq.eclair.transactions.{CommitmentSpec, DirectedHtlc, IncomingHtlc, OutgoingHtlc}
import fr.acinq.eclair.wire.protocol.CommonCodecs._
import fr.acinq.eclair.wire.protocol.LightningMessageCodecs._
import fr.acinq.eclair.wire.protocol.{LiquidityAds, TxSignatures, UpdateAddHtlc, UpdateMessage}
import fr.acinq.eclair.{BlockHeight, FeatureSupport, Features, MilliSatoshiLong, PermanentChannelFeature, channel}
import fr.acinq.eclair.wire.protocol._
import fr.acinq.eclair.{Alias, BlockHeight, FeatureSupport, Features, MilliSatoshiLong, PermanentChannelFeature, RealShortChannelId, channel}
import scodec.bits.{BitVector, ByteVector}
import scodec.codecs._
import scodec.{Attempt, Codec}
@ -424,15 +424,16 @@ private[channel] object ChannelCodecs4 {
}
val fundingTxStatusCodec: Codec[LocalFundingStatus] = discriminated[LocalFundingStatus].by(uint8)
.typecase(0x0a, (txCodec :: realshortchannelid :: optional(bool8, lengthDelimited(channelAnnouncementCodec)) :: optional(bool8, lengthDelimited(txSignaturesCodec)) :: optional(bool8, liquidityPurchaseCodec)).as[ConfirmedFundingTx])
.typecase(0x01, optional(bool8, txCodec).as[SingleFundedUnconfirmedFundingTx])
.typecase(0x07, dualFundedUnconfirmedFundingTxCodec)
.typecase(0x08, (txCodec :: optional(bool8, lengthDelimited(txSignaturesCodec)) :: optional(bool8, liquidityPurchaseCodec)).as[ZeroconfPublishedFundingTx])
.typecase(0x09, (txCodec :: optional(bool8, lengthDelimited(txSignaturesCodec)) :: optional(bool8, liquidityPurchaseCodec)).as[ConfirmedFundingTx])
.typecase(0x09, (txCodec :: provide(RealShortChannelId(0)) :: provide(Option.empty[ChannelAnnouncement]) :: optional(bool8, lengthDelimited(txSignaturesCodec)) :: optional(bool8, liquidityPurchaseCodec)).as[ConfirmedFundingTx])
.typecase(0x02, dualFundedUnconfirmedFundingTxWithoutLiquidityPurchaseCodec)
.typecase(0x05, (txCodec :: optional(bool8, lengthDelimited(txSignaturesCodec)) :: provide(Option.empty[LiquidityAds.PurchaseBasicInfo])).as[ZeroconfPublishedFundingTx])
.typecase(0x06, (txCodec :: optional(bool8, lengthDelimited(txSignaturesCodec)) :: provide(Option.empty[LiquidityAds.PurchaseBasicInfo])).as[ConfirmedFundingTx])
.typecase(0x06, (txCodec :: provide(RealShortChannelId(0)) :: provide(Option.empty[ChannelAnnouncement]) :: optional(bool8, lengthDelimited(txSignaturesCodec)) :: provide(Option.empty[LiquidityAds.PurchaseBasicInfo])).as[ConfirmedFundingTx])
.typecase(0x03, (txCodec :: provide(Option.empty[TxSignatures]) :: provide(Option.empty[LiquidityAds.PurchaseBasicInfo])).as[ZeroconfPublishedFundingTx])
.typecase(0x04, (txCodec :: provide(Option.empty[TxSignatures]) :: provide(Option.empty[LiquidityAds.PurchaseBasicInfo])).as[ConfirmedFundingTx])
.typecase(0x04, (txCodec :: provide(RealShortChannelId(0)) :: provide(Option.empty[ChannelAnnouncement]) :: provide(Option.empty[TxSignatures]) :: provide(Option.empty[LiquidityAds.PurchaseBasicInfo])).as[ConfirmedFundingTx])
val remoteFundingStatusCodec: Codec[RemoteFundingStatus] = discriminated[RemoteFundingStatus].by(uint8)
.typecase(0x01, provide(RemoteFundingStatus.NotLocked))
@ -647,6 +648,12 @@ private[channel] object ChannelCodecs4 {
.\(0x03) { case status: SpliceStatus.SpliceWaitingForSigs => status }(interactiveTxWaitingForSigsCodec.as[channel.SpliceStatus.SpliceWaitingForSigs])
.\(0x02) { case status: SpliceStatus.SpliceWaitingForSigs => status }(interactiveTxWaitingForSigsWithoutLiquidityPurchaseCodec.as[channel.SpliceStatus.SpliceWaitingForSigs])
private val shortids: Codec[ChannelTypes4.ShortIds] = (
("real_opt" | optional(bool8, realshortchannelid)) ::
("localAlias" | discriminated[Alias].by(uint16).typecase(1, alias)) ::
("remoteAlias_opt" | optional(bool8, alias))
).as[ChannelTypes4.ShortIds]
val DATA_WAIT_FOR_FUNDING_CONFIRMED_00_Codec: Codec[DATA_WAIT_FOR_FUNDING_CONFIRMED] = (
("commitments" | commitmentsCodecWithoutFirstRemoteCommitIndex) ::
("waitingSince" | blockHeight) ::
@ -661,11 +668,15 @@ private[channel] object ChannelCodecs4 {
val DATA_WAIT_FOR_CHANNEL_READY_01_Codec: Codec[DATA_WAIT_FOR_CHANNEL_READY] = (
("commitments" | commitmentsCodecWithoutFirstRemoteCommitIndex) ::
("shortIds" | shortids)).as[DATA_WAIT_FOR_CHANNEL_READY]
("shortIds" | shortids)).as[ChannelTypes4.DATA_WAIT_FOR_CHANNEL_READY_0b].xmap(d => d.migrate(), d => ChannelTypes4.DATA_WAIT_FOR_CHANNEL_READY_0b.from(d))
val DATA_WAIT_FOR_CHANNEL_READY_0b_Codec: Codec[DATA_WAIT_FOR_CHANNEL_READY] = (
("commitments" | versionedCommitmentsCodec) ::
("shortIds" | shortids)).as[DATA_WAIT_FOR_CHANNEL_READY]
("shortIds" | shortids)).as[ChannelTypes4.DATA_WAIT_FOR_CHANNEL_READY_0b].xmap(d => d.migrate(), d => ChannelTypes4.DATA_WAIT_FOR_CHANNEL_READY_0b.from(d))
val DATA_WAIT_FOR_CHANNEL_READY_15_Codec: Codec[DATA_WAIT_FOR_CHANNEL_READY] = (
("commitments" | versionedCommitmentsCodec) ::
("aliases" | aliases)).as[DATA_WAIT_FOR_CHANNEL_READY]
val DATA_WAIT_FOR_DUAL_FUNDING_SIGNED_09_Codec: Codec[DATA_WAIT_FOR_DUAL_FUNDING_SIGNED] = (
("channelParams" | paramsCodec) ::
@ -682,7 +693,7 @@ private[channel] object ChannelCodecs4 {
("remotePushAmount" | millisatoshi) ::
("status" | interactiveTxWaitingForSigsCodec) ::
("remoteChannelData_opt" | optional(bool8, varsizebinarydata))).as[DATA_WAIT_FOR_DUAL_FUNDING_SIGNED]
val DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED_02_Codec: Codec[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED] = (
("commitments" | commitmentsCodecWithoutFirstRemoteCommitIndex) ::
("localPushAmount" | millisatoshi) ::
@ -703,11 +714,15 @@ private[channel] object ChannelCodecs4 {
val DATA_WAIT_FOR_DUAL_FUNDING_READY_03_Codec: Codec[DATA_WAIT_FOR_DUAL_FUNDING_READY] = (
("commitments" | commitmentsCodecWithoutFirstRemoteCommitIndex) ::
("shortIds" | shortids)).as[DATA_WAIT_FOR_DUAL_FUNDING_READY]
("shortIds" | shortids)).as[ChannelTypes4.DATA_WAIT_FOR_DUAL_FUNDING_READY_0d].xmap(d => d.migrate(), d => ChannelTypes4.DATA_WAIT_FOR_DUAL_FUNDING_READY_0d.from(d))
val DATA_WAIT_FOR_DUAL_FUNDING_READY_0d_Codec: Codec[DATA_WAIT_FOR_DUAL_FUNDING_READY] = (
("commitments" | versionedCommitmentsCodec) ::
("shortIds" | shortids)).as[DATA_WAIT_FOR_DUAL_FUNDING_READY]
("shortIds" | shortids)).as[ChannelTypes4.DATA_WAIT_FOR_DUAL_FUNDING_READY_0d].xmap(d => d.migrate(), d => ChannelTypes4.DATA_WAIT_FOR_DUAL_FUNDING_READY_0d.from(d))
val DATA_WAIT_FOR_DUAL_FUNDING_READY_16_Codec: Codec[DATA_WAIT_FOR_DUAL_FUNDING_READY] = (
("commitments" | versionedCommitmentsCodec) ::
("aliases" | aliases)).as[DATA_WAIT_FOR_DUAL_FUNDING_READY]
val DATA_NORMAL_04_Codec: Codec[DATA_NORMAL] = (
("commitments" | commitmentsCodecWithoutFirstRemoteCommitIndex) ::
@ -717,7 +732,7 @@ private[channel] object ChannelCodecs4 {
("localShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) ::
("remoteShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) ::
("closingFeerates" | optional(bool8, closingFeeratesCodec)) ::
("spliceStatus" | spliceStatusCodec)).as[DATA_NORMAL]
("spliceStatus" | spliceStatusCodec)).as[ChannelTypes4.DATA_NORMAL_0e].xmap(d => d.migrate(), d => ChannelTypes4.DATA_NORMAL_0e.from(d))
val DATA_NORMAL_0e_Codec: Codec[DATA_NORMAL] = (
("commitments" | versionedCommitmentsCodec) ::
@ -727,6 +742,15 @@ private[channel] object ChannelCodecs4 {
("localShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) ::
("remoteShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) ::
("closingFeerates" | optional(bool8, closingFeeratesCodec)) ::
("spliceStatus" | spliceStatusCodec)).as[ChannelTypes4.DATA_NORMAL_0e].xmap(d => d.migrate(), d => ChannelTypes4.DATA_NORMAL_0e.from(d))
val DATA_NORMAL_14_Codec: Codec[DATA_NORMAL] = (
("commitments" | versionedCommitmentsCodec) ::
("aliases" | aliases) ::
("channelUpdate" | lengthDelimited(channelUpdateCodec)) ::
("localShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) ::
("remoteShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) ::
("closingFeerates" | optional(bool8, closingFeeratesCodec)) ::
("spliceStatus" | spliceStatusCodec)).as[DATA_NORMAL]
val DATA_SHUTDOWN_05_Codec: Codec[DATA_SHUTDOWN] = (
@ -790,6 +814,9 @@ private[channel] object ChannelCodecs4 {
// Order matters!
val channelDataCodec: Codec[PersistentChannelData] = discriminated[PersistentChannelData].by(uint16)
.typecase(0x16, Codecs.DATA_WAIT_FOR_DUAL_FUNDING_READY_16_Codec)
.typecase(0x15, Codecs.DATA_WAIT_FOR_CHANNEL_READY_15_Codec)
.typecase(0x14, Codecs.DATA_NORMAL_14_Codec)
.typecase(0x13, Codecs.DATA_WAIT_FOR_DUAL_FUNDING_SIGNED_13_Codec)
.typecase(0x12, Codecs.DATA_WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT_12_Codec)
.typecase(0x11, Codecs.DATA_CLOSING_11_Codec)

View file

@ -0,0 +1,106 @@
/*
* Copyright 2024 ACINQ SAS
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package fr.acinq.eclair.wire.internal.channel.version4
import fr.acinq.eclair.channel.LocalFundingStatus.ConfirmedFundingTx
import fr.acinq.eclair.channel._
import fr.acinq.eclair.wire.protocol.{ChannelAnnouncement, ChannelUpdate, Shutdown}
import fr.acinq.eclair.{Alias, RealShortChannelId}
private[channel] object ChannelTypes4 {
// We moved the real scid inside each commitment object when adding DATA_NORMAL_14_Codec.
case class ShortIds(real_opt: Option[RealShortChannelId], localAlias: Alias, remoteAlias_opt: Option[Alias])
// We moved the channel_announcement inside each commitment object when adding DATA_NORMAL_14_Codec.
case class DATA_NORMAL_0e(commitments: Commitments,
shortIds: ShortIds,
channelAnnouncement: Option[ChannelAnnouncement],
channelUpdate: ChannelUpdate,
localShutdown: Option[Shutdown],
remoteShutdown: Option[Shutdown],
closingFeerates: Option[ClosingFeerates],
spliceStatus: SpliceStatus) {
def migrate(): DATA_NORMAL = {
val commitments1 = commitments.copy(
active = commitments.active.map(c => setAnnouncementAndScidIfMatches(c, shortIds, channelAnnouncement)),
inactive = commitments.inactive.map(c => setAnnouncementAndScidIfMatches(c, shortIds, channelAnnouncement)),
)
val aliases = ShortIdAliases(shortIds.localAlias, shortIds.remoteAlias_opt)
DATA_NORMAL(commitments1, aliases, channelUpdate, localShutdown, remoteShutdown, closingFeerates, spliceStatus)
}
}
object DATA_NORMAL_0e {
// shouldn't be used since we only decode old data and never encode it
def from(d: DATA_NORMAL): DATA_NORMAL_0e = {
val shortIds = ShortIds(d.lastAnnouncement_opt.map(_.shortChannelId), d.aliases.localAlias, d.aliases.remoteAlias_opt)
DATA_NORMAL_0e(d.commitments, shortIds, d.lastAnnouncement_opt, d.channelUpdate, d.localShutdown, d.remoteShutdown, d.closingFeerates, d.spliceStatus)
}
}
case class DATA_WAIT_FOR_CHANNEL_READY_0b(commitments: Commitments, shortIds: ShortIds) {
def migrate(): DATA_WAIT_FOR_CHANNEL_READY = {
val commitments1 = commitments.copy(
active = commitments.active.map(c => setAnnouncementAndScidIfMatches(c, shortIds, None)),
inactive = commitments.inactive.map(c => setAnnouncementAndScidIfMatches(c, shortIds, None)),
)
val aliases = ShortIdAliases(shortIds.localAlias, shortIds.remoteAlias_opt)
DATA_WAIT_FOR_CHANNEL_READY(commitments1, aliases)
}
}
object DATA_WAIT_FOR_CHANNEL_READY_0b {
// shouldn't be used since we only decode old data and never encode it
def from(d: DATA_WAIT_FOR_CHANNEL_READY): DATA_WAIT_FOR_CHANNEL_READY_0b = {
val shortIds = ShortIds(None, d.aliases.localAlias, d.aliases.remoteAlias_opt)
DATA_WAIT_FOR_CHANNEL_READY_0b(d.commitments, shortIds)
}
}
case class DATA_WAIT_FOR_DUAL_FUNDING_READY_0d(commitments: Commitments, shortIds: ShortIds) {
def migrate(): DATA_WAIT_FOR_DUAL_FUNDING_READY = {
val commitments1 = commitments.copy(
active = commitments.active.map(c => setAnnouncementAndScidIfMatches(c, shortIds, None)),
inactive = commitments.inactive.map(c => setAnnouncementAndScidIfMatches(c, shortIds, None)),
)
val aliases = ShortIdAliases(shortIds.localAlias, shortIds.remoteAlias_opt)
DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments1, aliases)
}
}
object DATA_WAIT_FOR_DUAL_FUNDING_READY_0d {
// shouldn't be used since we only decode old data and never encode it
def from(d: DATA_WAIT_FOR_DUAL_FUNDING_READY): DATA_WAIT_FOR_DUAL_FUNDING_READY_0d = {
val shortIds = ShortIds(None, d.aliases.localAlias, d.aliases.remoteAlias_opt)
DATA_WAIT_FOR_DUAL_FUNDING_READY_0d(d.commitments, shortIds)
}
}
private def setAnnouncementAndScidIfMatches(c: Commitment, shortIds: ShortIds, announcement_opt: Option[ChannelAnnouncement]): Commitment = {
c.localFundingStatus match {
// We didn't support splicing on public channels in this version: the scid and announcement (if any) are for the
// initial funding transaction. For private channels we don't care about the real scid, it will be set correctly
// after the next splice.
case f: ConfirmedFundingTx if c.fundingTxIndex == 0 =>
val scid = shortIds.real_opt.getOrElse(f.shortChannelId)
c.copy(localFundingStatus = f.copy(shortChannelId = scid, announcement_opt = announcement_opt))
case _ => c
}
}
}

View file

@ -19,7 +19,7 @@ package fr.acinq.eclair.wire.protocol
import fr.acinq.bitcoin.scalacompat.Crypto.{PrivateKey, PublicKey}
import fr.acinq.bitcoin.scalacompat.{BlockHash, ByteVector32, ByteVector64, Satoshi, Transaction, TxHash, TxId}
import fr.acinq.eclair.blockchain.fee.FeeratePerKw
import fr.acinq.eclair.channel.{ChannelFlags, ShortIds}
import fr.acinq.eclair.channel.{ChannelFlags, ShortIdAliases}
import fr.acinq.eclair.crypto.Mac32
import fr.acinq.eclair.{Alias, BlockHeight, CltvExpiry, CltvExpiryDelta, Feature, Features, InitFeature, MilliSatoshi, RealShortChannelId, ShortChannelId, TimestampSecond, UInt64, UnspecifiedShortChannelId}
import org.apache.commons.codec.binary.Base32
@ -146,11 +146,11 @@ object CommonCodecs {
val alias: Codec[Alias] = shortchannelid.narrow[Alias](scid => Attempt.successful(Alias(scid.toLong)), scid => scid)
val shortids: Codec[ShortIds] = (
("real_opt" | optional(bool8, realshortchannelid)) ::
("localAlias" | discriminated[Alias].by(uint16).typecase(1, alias)) :: // forward-compatible with listOfN(uint16, aliashortchannelid) in case we want to store a list of local aliases later
val aliases: Codec[ShortIdAliases] = (
// forward-compatible with listOfN(uint16, alias) in case we want to store a list of local aliases later
("localAlias" | discriminated[Alias].by(uint16).typecase(1, alias)) ::
("remoteAlias_opt" | optional(bool8, alias))
).as[ShortIds]
).as[ShortIdAliases]
val privateKey: Codec[PrivateKey] = Codec[PrivateKey](
(priv: PrivateKey) => bytes(32).encode(priv.value),

View file

@ -111,27 +111,9 @@
"remotePerCommitmentSecrets" : null,
"originChannels" : { }
},
"shortIds" : {
"real" : "1513532x23x1",
"aliases" : {
"localAlias" : "0x17183c0000170001"
},
"channelAnnouncement" : {
"nodeSignature1" : "d2366163f4d5a51be3210b66b2e4a2736b9ccc20ce8d0d69413d5b5e42d991401183b271ba032764151ba8f3c4b03f11df5749fd876eeaf3fd401bb383cb3174",
"nodeSignature2" : "075779c27157e5b4024ecee12308cf3bde976a0891983b0655b669b38e7e700362c25ce4af05aaa130f000aa6a04037534a7a23a8d99454948dd689277eab321",
"bitcoinSignature1" : "4049b7649693d92139bf3f1f41da3825d1b3dbed2884797b76fd8e1c77390d1b4f3bf76b8d890485d7555619160a2bf18d58626f2ec9a8ca1f887eba3ba130b5",
"bitcoinSignature2" : "0d55e84fb4059bea082d443934af74dcbfd5c4c2fd54eba3ea2823114df932e7759805207f1182062f99af028aa4b62c7723a0c5b9198fe637a3d18d4d99dc70",
"features" : {
"activated" : { },
"unknown" : [ ]
},
"chainHash" : "43497fd7f826957108f4a30fd9cec3aeba79972084e90ead01ea330900000000",
"shortChannelId" : "1513532x23x1",
"nodeId1" : "034fe52e98a0e9d3c21b767e1b371881265d8c7578c21f5afd6d6438da10348b36",
"nodeId2" : "03933884aaf1d6b108397e5efe5c86bcf2d8ca8d2f700eda99db9214fc2712b134",
"bitcoinKey1" : "028cef3ef020cfda09692afc29e38ac4756ca60736563a93220481091c9cd05b64",
"bitcoinKey2" : "03660d280e24a9b16772a6e6418029719620a5caa29ebdf8339e5d700c611ab9e3",
"tlvStream" : { }
},
"channelUpdate" : {
"signature" : "4e34a547c424182812bd39b35c1c244b98f2bbb5b7d07812b9a008bb69f3fd77788f4ad338a102c331892afa8d076167a6a6cfb4eac3b890387f0fdc98b5b8c3",
"chainHash" : "43497fd7f826957108f4a30fd9cec3aeba79972084e90ead01ea330900000000",

View file

@ -111,8 +111,7 @@
"remotePerCommitmentSecrets" : null,
"originChannels" : { }
},
"shortIds" : {
"real" : "1413373x969x0",
"aliases" : {
"localAlias" : "0x1590fd0003c90000"
},
"channelUpdate" : {

View file

@ -125,27 +125,9 @@
"remotePerCommitmentSecrets" : null,
"originChannels" : { }
},
"shortIds" : {
"real" : "2026958x1x0",
"aliases" : {
"localAlias" : "0x1eedce0000010000"
},
"channelAnnouncement" : {
"nodeSignature1" : "98d7a81bc1aa92fcfb74ced2213e85e0d92ae8ac622bf294b3551c7c27f6f84f782f3b318e4d0eb2c67ac719a7c65afcf85bf159f6ceea9427be549201341969",
"nodeSignature2" : "92f6ed0e059db72105a13ec0e799bb08896cad8b4feb7e9ec7283c309b5f43123af1bd9e913fc2db018edadde8932d6992408f10c1ad020504361972dfa7fef0",
"bitcoinSignature1" : "9bbc2b568cef3c8c006f7860106fd5984bcc271ff06c4829db2a665e59b7c0b22c311a340ff2ab9bcb74a50db10ed85503ad2d248d95af8151aca8ef96248e8f",
"bitcoinSignature2" : "84b3075922385fbaf012f057e7ee84ecbc14c84880520b26d6fd22ab5f107db606a906efdcf0f88ffbe32dc6ecc10131e1ff0dc8d68dad89c98562557f00448b",
"features" : {
"activated" : { },
"unknown" : [ ]
},
"chainHash" : "43497fd7f826957108f4a30fd9cec3aeba79972084e90ead01ea330900000000",
"shortChannelId" : "2026958x1x0",
"nodeId1" : "027455aef8453d92f4706b560b61527cc217ddf14da41770e8ed6607190a1851b8",
"nodeId2" : "03933884aaf1d6b108397e5efe5c86bcf2d8ca8d2f700eda99db9214fc2712b134",
"bitcoinKey1" : "02eff5309b9368340edc6114d738b3590e6969bec4e95d8a080cf185e8b9ce5e4b",
"bitcoinKey2" : "023ced05ed1ab328b67477376d68a69ecd0f371a9d5843c6c3be4d31498d516d8d",
"tlvStream" : { }
},
"channelUpdate" : {
"signature" : "710d73875607575f3d84bb507dd87cca5b85f0cdac84f4ccecce7af3a55897525a45070fe26c0ea43e9580d4ea4cfa62ee3273e5546911145cba6bbf56e59d8e",
"chainHash" : "43497fd7f826957108f4a30fd9cec3aeba79972084e90ead01ea330900000000",

View file

@ -120,8 +120,7 @@
"remotePerCommitmentSecrets" : null,
"originChannels" : { }
},
"shortIds" : {
"real" : "400000x42x0",
"aliases" : {
"localAlias" : "0x3dab1f7dc6942fd"
}
}

View file

@ -126,8 +126,7 @@
"remotePerCommitmentSecrets" : null,
"originChannels" : { }
},
"shortIds" : {
"real" : "400000x42x0",
"aliases" : {
"localAlias" : "0x2d1583d8409d217"
}
}

File diff suppressed because one or more lines are too long

View file

@ -0,0 +1,138 @@
{
"type" : "DATA_WAIT_FOR_CHANNEL_READY",
"commitments" : {
"params" : {
"channelId" : "7d975ecb75e1497076150f745b83dace95a189eecb6172c20a9a4fe0f91b8d1d",
"channelConfig" : [ "funding_pubkey_based_channel_keypath" ],
"channelFeatures" : [ "option_static_remotekey" ],
"localParams" : {
"nodeId" : "02bbbb671d15145722fb8c28d732cddb249bcc6652ed2b297ff1f77a18371b1e63",
"fundingKeyPath" : [ 3266606892, 3454308995, 379409286, 2058386039, 150235166, 1337553882, 292124276, 1286028724, 2147483648 ],
"dustLimit" : 1000,
"maxHtlcValueInFlightMsat" : 1000000000,
"initialRequestedChannelReserve_opt" : 20000,
"htlcMinimum" : 1000,
"toSelfDelay" : 144,
"maxAcceptedHtlcs" : 30,
"isChannelOpener" : false,
"paysCommitTxFees" : false,
"walletStaticPaymentBasepoint" : "028feba10d0eafd0fad8fe20e6d9206e6bd30242826de05c63f459a00aced24b12",
"initFeatures" : {
"activated" : {
"option_route_blinding" : "optional",
"splice_prototype" : "optional",
"payment_secret" : "mandatory",
"gossip_queries_ex" : "optional",
"option_quiesce" : "optional",
"option_data_loss_protect" : "optional",
"var_onion_optin" : "mandatory",
"option_static_remotekey" : "mandatory",
"option_support_large_channel" : "optional",
"option_anchors_zero_fee_htlc_tx" : "optional",
"basic_mpp" : "optional",
"gossip_queries" : "optional"
},
"unknown" : [ ]
}
},
"remoteParams" : {
"nodeId" : "02aaaa00ce2f18a967dc4f25f414e671ba6585f8ef0b8c5fb812c21064f55a2eaa",
"dustLimit" : 1100,
"maxHtlcValueInFlightMsat" : 500000000,
"initialRequestedChannelReserve_opt" : 10000,
"htlcMinimum" : 0,
"toSelfDelay" : 144,
"maxAcceptedHtlcs" : 100,
"revocationBasepoint" : "02e1a7010650cd5cd6fbf7505f5f213b36e5cc0d127064c74619c83dfa7434ce25",
"paymentBasepoint" : "028feba10d0eafd0fad8fe20e6d9206e6bd30242826de05c63f459a00aced24b12",
"delayedPaymentBasepoint" : "031723f6ffc8e552694f09d067844851f14357fb2a294fd94fd0b596ee808d2a17",
"htlcBasepoint" : "031c3b7172c024396dd7931cd835d50e8eaef23c80e5b47a6c35fcf5c0ea89c6ab",
"initFeatures" : {
"activated" : {
"option_support_large_channel" : "optional",
"option_route_blinding" : "optional",
"option_provide_storage" : "optional",
"splice_prototype" : "optional",
"payment_secret" : "mandatory",
"gossip_queries_ex" : "optional",
"option_quiesce" : "optional",
"option_data_loss_protect" : "optional",
"var_onion_optin" : "mandatory",
"option_static_remotekey" : "mandatory",
"basic_mpp" : "optional",
"gossip_queries" : "optional"
},
"unknown" : [ 50001 ]
}
},
"channelFlags" : {
"nonInitiatorPaysCommitFees" : false,
"announceChannel" : false
}
},
"changes" : {
"localChanges" : {
"proposed" : [ ],
"signed" : [ ],
"acked" : [ ]
},
"remoteChanges" : {
"proposed" : [ ],
"acked" : [ ],
"signed" : [ ]
},
"localNextHtlcId" : 0,
"remoteNextHtlcId" : 0
},
"active" : [ {
"fundingTxIndex" : 0,
"fundingTx" : {
"outPoint" : "1d8d1bf9e04f9a0ac27261cbee89a195ceda835b740f15767049e175cb5e977d:0",
"amountSatoshis" : 1000000
},
"localFunding" : {
"status" : "confirmed",
"txid" : "1d8d1bf9e04f9a0ac27261cbee89a195ceda835b740f15767049e175cb5e977d",
"shortChannelId" : "400000x42x0"
},
"remoteFunding" : {
"status" : "not-locked"
},
"localCommit" : {
"index" : 0,
"spec" : {
"htlcs" : [ ],
"commitTxFeerate" : 10000,
"toLocal" : 200000000,
"toRemote" : 800000000
},
"commitTxAndRemoteSig" : {
"commitTx" : {
"txid" : "d873d760728d9cf12d61c43360c383ab83799c9153f38c6012d9d6f6e6026df1",
"tx" : "02000000017d975ecb75e1497076150f745b83dace95a189eecb6172c20a9a4fe0f91b8d1d000000000036a2ab8002400d0300000000002200207d4136c71e1c702d6e56ea300e1a56298b70fc8e96d67e72cfab3a27d39c9bf2b8180c0000000000160014761879f7b274ce995f87150a02e75cc0c037e8e3ac678920"
},
"remoteSig" : "02b8ce0a760ae6052a0d6c5770ae626ae71ea1b64853f97bb2c784839264054c64dc8717789f804342cbd9c5a364b32f4cbd9e9a4f91f48127a58f95014f6663"
},
"htlcTxsAndRemoteSigs" : [ ]
},
"remoteCommit" : {
"index" : 0,
"spec" : {
"htlcs" : [ ],
"commitTxFeerate" : 10000,
"toLocal" : 800000000,
"toRemote" : 200000000
},
"txid" : "ad28d946d13d331539d08da9d131311d1bec7308ddf072a59b07b2dc22779c3f",
"remotePerCommitmentPoint" : "02e813fa8f4480fcbfa92d65037fe2d5d99cb72d6987381cf253d13f071a45f049"
}
} ],
"inactive" : [ ],
"remoteNextCommitInfo" : "02305a6b82bd468d5eb1d80d36117119d1144f0f7439b88e64c8e164f43d9ea969",
"remotePerCommitmentSecrets" : null,
"originChannels" : { }
},
"aliases" : {
"localAlias" : "0x2a7224ea3627762"
}
}

File diff suppressed because one or more lines are too long

View file

@ -0,0 +1,138 @@
{
"type" : "DATA_WAIT_FOR_CHANNEL_READY",
"commitments" : {
"params" : {
"channelId" : "7d975ecb75e1497076150f745b83dace95a189eecb6172c20a9a4fe0f91b8d1d",
"channelConfig" : [ "funding_pubkey_based_channel_keypath" ],
"channelFeatures" : [ "option_static_remotekey" ],
"localParams" : {
"nodeId" : "02aaaa00ce2f18a967dc4f25f414e671ba6585f8ef0b8c5fb812c21064f55a2eaa",
"fundingKeyPath" : [ 222163200, 3836794389, 3217943953, 1565449417, 2862146275, 3633046910, 2547274215, 4050695153, 2147483649 ],
"dustLimit" : 1100,
"maxHtlcValueInFlightMsat" : 500000000,
"initialRequestedChannelReserve_opt" : 10000,
"htlcMinimum" : 0,
"toSelfDelay" : 144,
"maxAcceptedHtlcs" : 100,
"isChannelOpener" : true,
"paysCommitTxFees" : true,
"walletStaticPaymentBasepoint" : "028feba10d0eafd0fad8fe20e6d9206e6bd30242826de05c63f459a00aced24b12",
"initFeatures" : {
"activated" : {
"option_support_large_channel" : "optional",
"option_route_blinding" : "optional",
"option_provide_storage" : "optional",
"splice_prototype" : "optional",
"payment_secret" : "mandatory",
"gossip_queries_ex" : "optional",
"option_quiesce" : "optional",
"option_data_loss_protect" : "optional",
"var_onion_optin" : "mandatory",
"option_static_remotekey" : "mandatory",
"basic_mpp" : "optional",
"gossip_queries" : "optional"
},
"unknown" : [ 50001 ]
}
},
"remoteParams" : {
"nodeId" : "02bbbb671d15145722fb8c28d732cddb249bcc6652ed2b297ff1f77a18371b1e63",
"dustLimit" : 1000,
"maxHtlcValueInFlightMsat" : 1000000000,
"initialRequestedChannelReserve_opt" : 20000,
"htlcMinimum" : 1000,
"toSelfDelay" : 144,
"maxAcceptedHtlcs" : 30,
"revocationBasepoint" : "0201dd773d2108f3ed8f59669240c7f098d3083963bef5b36ebf940f4bc40724c8",
"paymentBasepoint" : "028feba10d0eafd0fad8fe20e6d9206e6bd30242826de05c63f459a00aced24b12",
"delayedPaymentBasepoint" : "0243f77d02fbb07f25d6fe14f9c7ee904e1a19442a4aea0dc1ce959d5d2caea9b4",
"htlcBasepoint" : "02e3ae52529ae89f6d7ab478474e0ebe1845d3f7a36967209caad616b0d43353f5",
"initFeatures" : {
"activated" : {
"option_route_blinding" : "optional",
"splice_prototype" : "optional",
"payment_secret" : "mandatory",
"gossip_queries_ex" : "optional",
"option_quiesce" : "optional",
"option_data_loss_protect" : "optional",
"var_onion_optin" : "mandatory",
"option_static_remotekey" : "mandatory",
"option_support_large_channel" : "optional",
"option_anchors_zero_fee_htlc_tx" : "optional",
"basic_mpp" : "optional",
"gossip_queries" : "optional"
},
"unknown" : [ ]
}
},
"channelFlags" : {
"nonInitiatorPaysCommitFees" : false,
"announceChannel" : false
}
},
"changes" : {
"localChanges" : {
"proposed" : [ ],
"signed" : [ ],
"acked" : [ ]
},
"remoteChanges" : {
"proposed" : [ ],
"acked" : [ ],
"signed" : [ ]
},
"localNextHtlcId" : 0,
"remoteNextHtlcId" : 0
},
"active" : [ {
"fundingTxIndex" : 0,
"fundingTx" : {
"outPoint" : "1d8d1bf9e04f9a0ac27261cbee89a195ceda835b740f15767049e175cb5e977d:0",
"amountSatoshis" : 1000000
},
"localFunding" : {
"status" : "confirmed",
"txid" : "1d8d1bf9e04f9a0ac27261cbee89a195ceda835b740f15767049e175cb5e977d",
"shortChannelId" : "400000x42x0"
},
"remoteFunding" : {
"status" : "not-locked"
},
"localCommit" : {
"index" : 0,
"spec" : {
"htlcs" : [ ],
"commitTxFeerate" : 10000,
"toLocal" : 800000000,
"toRemote" : 200000000
},
"commitTxAndRemoteSig" : {
"commitTx" : {
"txid" : "ad28d946d13d331539d08da9d131311d1bec7308ddf072a59b07b2dc22779c3f",
"tx" : "02000000017d975ecb75e1497076150f745b83dace95a189eecb6172c20a9a4fe0f91b8d1d000000000036a2ab8002400d030000000000160014761879f7b274ce995f87150a02e75cc0c037e8e3b8180c000000000022002029b18fc4cf2f2b4634802a1b950d9e27e93b9be9ce638351143432fd0db09163ac678920"
},
"remoteSig" : "8d3c57514dd1c01073b3d3704eddcfd8ca6ac3bb58f145f2c2dd921b77f238e842dce706a711b69b250e9ec3b827a9d9e144720a41fadce820bb18280667bffd"
},
"htlcTxsAndRemoteSigs" : [ ]
},
"remoteCommit" : {
"index" : 0,
"spec" : {
"htlcs" : [ ],
"commitTxFeerate" : 10000,
"toLocal" : 200000000,
"toRemote" : 800000000
},
"txid" : "d873d760728d9cf12d61c43360c383ab83799c9153f38c6012d9d6f6e6026df1",
"remotePerCommitmentPoint" : "026b84590050c242d572dd2a45b873ce1e108303e1a54a6492e641f3ec1a70a7aa"
}
} ],
"inactive" : [ ],
"remoteNextCommitInfo" : "02449092e08f529ffd0b45361f70f877e865f045b23b80e7658d050690c2280125",
"remotePerCommitmentSecrets" : null,
"originChannels" : { }
},
"aliases" : {
"localAlias" : "0x1955fc8674b7616"
}
}

File diff suppressed because one or more lines are too long

View file

@ -0,0 +1,140 @@
{
"type" : "DATA_WAIT_FOR_DUAL_FUNDING_READY",
"commitments" : {
"params" : {
"channelId" : "ae58e31828b115b8a900a9d20b060ef2b2da2fcfe18991aff34168579d246b54",
"channelConfig" : [ "funding_pubkey_based_channel_keypath" ],
"channelFeatures" : [ "option_static_remotekey", "option_anchors_zero_fee_htlc_tx", "option_dual_fund" ],
"localParams" : {
"nodeId" : "02bbbb671d15145722fb8c28d732cddb249bcc6652ed2b297ff1f77a18371b1e63",
"fundingKeyPath" : [ 2566431519, 2591595317, 1320214673, 2312083324, 3942626068, 2510065287, 1961707336, 236707474, 2147483648 ],
"dustLimit" : 1000,
"maxHtlcValueInFlightMsat" : 1000000000,
"htlcMinimum" : 1000,
"toSelfDelay" : 144,
"maxAcceptedHtlcs" : 30,
"isChannelOpener" : false,
"paysCommitTxFees" : false,
"initFeatures" : {
"activated" : {
"option_route_blinding" : "optional",
"option_dual_fund" : "optional",
"splice_prototype" : "optional",
"payment_secret" : "mandatory",
"gossip_queries_ex" : "optional",
"option_anchor_outputs" : "optional",
"option_quiesce" : "optional",
"option_data_loss_protect" : "optional",
"var_onion_optin" : "mandatory",
"option_static_remotekey" : "optional",
"option_support_large_channel" : "optional",
"option_anchors_zero_fee_htlc_tx" : "optional",
"basic_mpp" : "optional",
"gossip_queries" : "optional"
},
"unknown" : [ ]
}
},
"remoteParams" : {
"nodeId" : "02aaaa00ce2f18a967dc4f25f414e671ba6585f8ef0b8c5fb812c21064f55a2eaa",
"dustLimit" : 1100,
"maxHtlcValueInFlightMsat" : 500000000,
"htlcMinimum" : 0,
"toSelfDelay" : 144,
"maxAcceptedHtlcs" : 100,
"revocationBasepoint" : "038039cb4b63c81052cf0893e27f7be58a5808453dffd44c662b1a75aebb1fc74e",
"paymentBasepoint" : "0394e61db71e79c20f2ef131e99121996c7ae04330c4c50dae6bccc5afcf493661",
"delayedPaymentBasepoint" : "029a15b644f2800a507565666956446c05ef8dd7b60023852607c97471d233d62e",
"htlcBasepoint" : "02c633358670a7235d82a5b732d3f7c21e010ea411c805f0f319d922300df453c0",
"initFeatures" : {
"activated" : {
"option_route_blinding" : "optional",
"option_provide_storage" : "optional",
"option_dual_fund" : "optional",
"splice_prototype" : "optional",
"payment_secret" : "mandatory",
"gossip_queries_ex" : "optional",
"option_anchor_outputs" : "optional",
"option_quiesce" : "optional",
"option_data_loss_protect" : "optional",
"var_onion_optin" : "mandatory",
"option_static_remotekey" : "optional",
"option_support_large_channel" : "optional",
"option_anchors_zero_fee_htlc_tx" : "optional",
"basic_mpp" : "optional",
"gossip_queries" : "optional"
},
"unknown" : [ 50001 ]
}
},
"channelFlags" : {
"nonInitiatorPaysCommitFees" : false,
"announceChannel" : false
}
},
"changes" : {
"localChanges" : {
"proposed" : [ ],
"signed" : [ ],
"acked" : [ ]
},
"remoteChanges" : {
"proposed" : [ ],
"acked" : [ ],
"signed" : [ ]
},
"localNextHtlcId" : 0,
"remoteNextHtlcId" : 0
},
"active" : [ {
"fundingTxIndex" : 0,
"fundingTx" : {
"outPoint" : "e0052c9a6cebcf139248732fcc7c563b022c2ae2d665a670f09389e4ff514393:0",
"amountSatoshis" : 1500000
},
"localFunding" : {
"status" : "confirmed",
"txid" : "e0052c9a6cebcf139248732fcc7c563b022c2ae2d665a670f09389e4ff514393",
"shortChannelId" : "400000x42x0"
},
"remoteFunding" : {
"status" : "not-locked"
},
"localCommit" : {
"index" : 0,
"spec" : {
"htlcs" : [ ],
"commitTxFeerate" : 2500,
"toLocal" : 500000000,
"toRemote" : 1000000000
},
"commitTxAndRemoteSig" : {
"commitTx" : {
"txid" : "800a5eab3f585d54f19d6e574b611d94fa7414af651e17104310bf004ccd5cd6",
"tx" : "0200000001934351ffe48993f070a665d6e22a2c023b567ccc2f73489213cfeb6c9a2c05e000000000001a741d80044a010000000000002200209698cbfb709ad73804217ae67e86ea857e8c1b317c82e7740670eb6def38ea334a01000000000000220020d468d18ef4bd63800e3e59bbb204a2921dc75cec9786e363ba2d33d467ae98e820a10700000000002200200e84b31c024b1498353109b6127546c1c1999edf47edb4c772c2108edeb524b4b2340f0000000000220020c38ddef3ceb25118b96983889833cbecd3fdeaf190083eef04ee451b70b47e04a71aae20"
},
"remoteSig" : "c70d3fc3e720cc4fd1a2e07b97d454392ecfbe27c0ef2eed401538466924f6c24accb834eed784536fe5d30656eb8308a8b0f48bdba2ccc9812ce49a12b9dd80"
},
"htlcTxsAndRemoteSigs" : [ ]
},
"remoteCommit" : {
"index" : 0,
"spec" : {
"htlcs" : [ ],
"commitTxFeerate" : 2500,
"toLocal" : 1000000000,
"toRemote" : 500000000
},
"txid" : "c98fac755c41c1688a55f820a3ac9c4c8b2adde1d6687b2a9d843b364211965f",
"remotePerCommitmentPoint" : "02df0aedf5ee7eba9dbe640dfd3b2da6e108cf52568e4de3e492ed4366355a33c0"
}
} ],
"inactive" : [ ],
"remoteNextCommitInfo" : "028a7d253bc83df4357f3c1c89d9040cc55bc5a64da5828e523840a50832543629",
"remotePerCommitmentSecrets" : null,
"originChannels" : { }
},
"aliases" : {
"localAlias" : "0x15e6c55cb4a946e"
}
}

File diff suppressed because one or more lines are too long

View file

@ -0,0 +1,140 @@
{
"type" : "DATA_WAIT_FOR_DUAL_FUNDING_READY",
"commitments" : {
"params" : {
"channelId" : "ae58e31828b115b8a900a9d20b060ef2b2da2fcfe18991aff34168579d246b54",
"channelConfig" : [ "funding_pubkey_based_channel_keypath" ],
"channelFeatures" : [ "option_static_remotekey", "option_anchors_zero_fee_htlc_tx", "option_dual_fund" ],
"localParams" : {
"nodeId" : "02aaaa00ce2f18a967dc4f25f414e671ba6585f8ef0b8c5fb812c21064f55a2eaa",
"fundingKeyPath" : [ 461130578, 2441458723, 3315200959, 1624559903, 421875145, 1108846792, 2605590614, 1621284536, 2147483649 ],
"dustLimit" : 1100,
"maxHtlcValueInFlightMsat" : 500000000,
"htlcMinimum" : 0,
"toSelfDelay" : 144,
"maxAcceptedHtlcs" : 100,
"isChannelOpener" : true,
"paysCommitTxFees" : true,
"initFeatures" : {
"activated" : {
"option_route_blinding" : "optional",
"option_provide_storage" : "optional",
"option_dual_fund" : "optional",
"splice_prototype" : "optional",
"payment_secret" : "mandatory",
"gossip_queries_ex" : "optional",
"option_anchor_outputs" : "optional",
"option_quiesce" : "optional",
"option_data_loss_protect" : "optional",
"var_onion_optin" : "mandatory",
"option_static_remotekey" : "optional",
"option_support_large_channel" : "optional",
"option_anchors_zero_fee_htlc_tx" : "optional",
"basic_mpp" : "optional",
"gossip_queries" : "optional"
},
"unknown" : [ 50001 ]
}
},
"remoteParams" : {
"nodeId" : "02bbbb671d15145722fb8c28d732cddb249bcc6652ed2b297ff1f77a18371b1e63",
"dustLimit" : 1000,
"maxHtlcValueInFlightMsat" : 1000000000,
"htlcMinimum" : 1000,
"toSelfDelay" : 144,
"maxAcceptedHtlcs" : 30,
"revocationBasepoint" : "031a9b266c6a1f5d05f6f2b040b0eea59819c2fb5715945628541c58c2605e5c42",
"paymentBasepoint" : "02cb3e21fe0825663cfa1d8a8accc93dedbdb906b0ec4b779bb41b0f030b38fd33",
"delayedPaymentBasepoint" : "0370c28e0286bfcf6bfb18c2b98d5b265594ec363822cbb323e7f7451ca60b0330",
"htlcBasepoint" : "03fac293804f27e6823ba11b41063d35f20a844864de7cc2f9586445af26b1a03e",
"initFeatures" : {
"activated" : {
"option_route_blinding" : "optional",
"option_dual_fund" : "optional",
"splice_prototype" : "optional",
"payment_secret" : "mandatory",
"gossip_queries_ex" : "optional",
"option_anchor_outputs" : "optional",
"option_quiesce" : "optional",
"option_data_loss_protect" : "optional",
"var_onion_optin" : "mandatory",
"option_static_remotekey" : "optional",
"option_support_large_channel" : "optional",
"option_anchors_zero_fee_htlc_tx" : "optional",
"basic_mpp" : "optional",
"gossip_queries" : "optional"
},
"unknown" : [ ]
}
},
"channelFlags" : {
"nonInitiatorPaysCommitFees" : false,
"announceChannel" : false
}
},
"changes" : {
"localChanges" : {
"proposed" : [ ],
"signed" : [ ],
"acked" : [ ]
},
"remoteChanges" : {
"proposed" : [ ],
"acked" : [ ],
"signed" : [ ]
},
"localNextHtlcId" : 0,
"remoteNextHtlcId" : 0
},
"active" : [ {
"fundingTxIndex" : 0,
"fundingTx" : {
"outPoint" : "e0052c9a6cebcf139248732fcc7c563b022c2ae2d665a670f09389e4ff514393:0",
"amountSatoshis" : 1500000
},
"localFunding" : {
"status" : "confirmed",
"txid" : "e0052c9a6cebcf139248732fcc7c563b022c2ae2d665a670f09389e4ff514393",
"shortChannelId" : "400000x42x0"
},
"remoteFunding" : {
"status" : "not-locked"
},
"localCommit" : {
"index" : 0,
"spec" : {
"htlcs" : [ ],
"commitTxFeerate" : 2500,
"toLocal" : 1000000000,
"toRemote" : 500000000
},
"commitTxAndRemoteSig" : {
"commitTx" : {
"txid" : "c98fac755c41c1688a55f820a3ac9c4c8b2adde1d6687b2a9d843b364211965f",
"tx" : "0200000001934351ffe48993f070a665d6e22a2c023b567ccc2f73489213cfeb6c9a2c05e000000000001a741d80044a010000000000002200209698cbfb709ad73804217ae67e86ea857e8c1b317c82e7740670eb6def38ea334a01000000000000220020d468d18ef4bd63800e3e59bbb204a2921dc75cec9786e363ba2d33d467ae98e820a1070000000000220020fbc40c7329e9ed94bb1a94bed8d66eba59e96aba45dc5d59150b6ac6ab24436ab2340f00000000002200201e1e972764d9ad2836c50f22bcb80b031dd989239b89db8d3c62c83e14ad88c0a71aae20"
},
"remoteSig" : "353c51755a0c852a4c5dd9e2d33d4a9c0934e9a07a43da89f061a3b4499f374d1857dfa513ed714287baf4ac3f4375f06bf435577962f121182ad9ce4b19f549"
},
"htlcTxsAndRemoteSigs" : [ ]
},
"remoteCommit" : {
"index" : 0,
"spec" : {
"htlcs" : [ ],
"commitTxFeerate" : 2500,
"toLocal" : 500000000,
"toRemote" : 1000000000
},
"txid" : "800a5eab3f585d54f19d6e574b611d94fa7414af651e17104310bf004ccd5cd6",
"remotePerCommitmentPoint" : "03a59c87106cab1b24df7f9a3843a728a45bca74680f65118ca6c5a3d45c1934e6"
}
} ],
"inactive" : [ ],
"remoteNextCommitInfo" : "035642c8d53e45f57ffdbcfd1bdb3c931479649d6c3c3a4ffe766c1a3f8835212d",
"remotePerCommitmentSecrets" : null,
"originChannels" : { }
},
"aliases" : {
"localAlias" : "0x1277132bb4ca09a"
}
}

File diff suppressed because one or more lines are too long

View file

@ -0,0 +1,178 @@
{
"type" : "DATA_NORMAL",
"commitments" : {
"params" : {
"channelId" : "c380aa11700db0a6d797dfd0be8aecfadad9397e4975f0fa8c9d10db71feac38",
"channelConfig" : [ "funding_pubkey_based_channel_keypath" ],
"channelFeatures" : [ "option_static_remotekey" ],
"localParams" : {
"nodeId" : "02aaaa00ce2f18a967dc4f25f414e671ba6585f8ef0b8c5fb812c21064f55a2eaa",
"fundingKeyPath" : [ 1057736322, 487845715, 3068040012, 2828312867, 3790939166, 2388092911, 4276180524, 217832603, 2147483649 ],
"dustLimit" : 1100,
"maxHtlcValueInFlightMsat" : 500000000,
"initialRequestedChannelReserve_opt" : 10000,
"htlcMinimum" : 0,
"toSelfDelay" : 144,
"maxAcceptedHtlcs" : 100,
"isChannelOpener" : true,
"paysCommitTxFees" : true,
"walletStaticPaymentBasepoint" : "028feba10d0eafd0fad8fe20e6d9206e6bd30242826de05c63f459a00aced24b12",
"initFeatures" : {
"activated" : {
"option_support_large_channel" : "optional",
"option_route_blinding" : "optional",
"option_provide_storage" : "optional",
"splice_prototype" : "optional",
"payment_secret" : "mandatory",
"gossip_queries_ex" : "optional",
"option_quiesce" : "optional",
"option_data_loss_protect" : "optional",
"var_onion_optin" : "mandatory",
"option_static_remotekey" : "mandatory",
"basic_mpp" : "optional",
"gossip_queries" : "optional"
},
"unknown" : [ 50001 ]
}
},
"remoteParams" : {
"nodeId" : "02bbbb671d15145722fb8c28d732cddb249bcc6652ed2b297ff1f77a18371b1e63",
"dustLimit" : 1000,
"maxHtlcValueInFlightMsat" : 1000000000,
"initialRequestedChannelReserve_opt" : 20000,
"htlcMinimum" : 1000,
"toSelfDelay" : 144,
"maxAcceptedHtlcs" : 30,
"revocationBasepoint" : "02f943c4f199d1425fc6e52f160be536d526e9643af1430cfed4ce63f88beebd2f",
"paymentBasepoint" : "028feba10d0eafd0fad8fe20e6d9206e6bd30242826de05c63f459a00aced24b12",
"delayedPaymentBasepoint" : "03f3b6c8919d313d5bd14a2e567649ea80ed1dd299ebc72766775a1f4fdc3cc1b5",
"htlcBasepoint" : "027ebc1b165dfe376d2a5b7bfba6419af4670b1b89b3aabf8bcbc2b5f33f0db274",
"initFeatures" : {
"activated" : {
"option_route_blinding" : "optional",
"splice_prototype" : "optional",
"payment_secret" : "mandatory",
"gossip_queries_ex" : "optional",
"option_quiesce" : "optional",
"option_data_loss_protect" : "optional",
"var_onion_optin" : "mandatory",
"option_static_remotekey" : "mandatory",
"option_support_large_channel" : "optional",
"option_anchors_zero_fee_htlc_tx" : "optional",
"basic_mpp" : "optional",
"gossip_queries" : "optional"
},
"unknown" : [ ]
}
},
"channelFlags" : {
"nonInitiatorPaysCommitFees" : false,
"announceChannel" : true
}
},
"changes" : {
"localChanges" : {
"proposed" : [ ],
"signed" : [ ],
"acked" : [ ]
},
"remoteChanges" : {
"proposed" : [ ],
"acked" : [ ],
"signed" : [ ]
},
"localNextHtlcId" : 0,
"remoteNextHtlcId" : 0
},
"active" : [ {
"fundingTxIndex" : 0,
"fundingTx" : {
"outPoint" : "38acfe71db109d8cfaf075497e39d9dafaec8abed0df97d7a6b00d7011aa80c3:0",
"amountSatoshis" : 1000000
},
"localFunding" : {
"status" : "confirmed",
"txid" : "38acfe71db109d8cfaf075497e39d9dafaec8abed0df97d7a6b00d7011aa80c3",
"shortChannelId" : "400000x42x0",
"announcement" : {
"nodeSignature1" : "c5a72f183a9fc823c5f5e28c2803d2042c563f291aafa2fb5e97b3cad51014b50aafbad94e32c6326bf2865a4a3d5f2cd57c9eff6173b4328dae116ed5a04e38",
"nodeSignature2" : "9d6e2543f51f5cee25d2bfdf2273f278362f09935f4749433cf81ca1b8004a223d7196b8c8e241f8b723d7e47124557d977b0cb40fffef57d8123d0a633e2452",
"bitcoinSignature1" : "9f91b0bacabbf1008077f58571453ba25bf9b1bf01c3511e255f533eb10776e34a04390a4ad8539825346f9a122016c3b3cc8e176ed5235e27ec1fa7ff0b3899",
"bitcoinSignature2" : "7fc5d33308e2a870ff1feb1cad3271801da93aed362fe13ea9f33f543e2a372a5f4de2566bba858b95e3df8d7ba9c10647a6acea455598a9b84d21a876525de9",
"features" : {
"activated" : { },
"unknown" : [ ]
},
"chainHash" : "06226e46111a0b59caaf126043eb5bbf28c34f3a5e332a1fc7b2b73cf188910f",
"shortChannelId" : "400000x42x0",
"nodeId1" : "02aaaa00ce2f18a967dc4f25f414e671ba6585f8ef0b8c5fb812c21064f55a2eaa",
"nodeId2" : "02bbbb671d15145722fb8c28d732cddb249bcc6652ed2b297ff1f77a18371b1e63",
"bitcoinKey1" : "029fbe1c2fe0d86e562a09aedaca23dabee01159d288ef3d6ea85ed107dd51db8a",
"bitcoinKey2" : "03613ca8550a9ce1252ed80d39dce2e56f5befdedd70019e195bc0e1448e48b4d6",
"tlvStream" : { }
}
},
"remoteFunding" : {
"status" : "not-locked"
},
"localCommit" : {
"index" : 0,
"spec" : {
"htlcs" : [ ],
"commitTxFeerate" : 10000,
"toLocal" : 800000000,
"toRemote" : 200000000
},
"commitTxAndRemoteSig" : {
"commitTx" : {
"txid" : "41418383164935b0753fa1f9b128dff2f1b3608e6b50e8beea4525edf7c3959d",
"tx" : "0200000001c380aa11700db0a6d797dfd0be8aecfadad9397e4975f0fa8c9d10db71feac38000000000036a2ab8002400d030000000000160014761879f7b274ce995f87150a02e75cc0c037e8e3b8180c000000000022002010a0e90d648e62c6aa0d9b9883ee391449f1a1ce28926eb26c3bbdf76c25f631ac678920"
},
"remoteSig" : "62de565629126afb02a76e67ddf18dfc883cf93bfeb7c807826dd6c8d1bd1f4d55528717f0d2303dc254b4512e32ce82dc7947d5d8e4d38647c2407fc53b874a"
},
"htlcTxsAndRemoteSigs" : [ ]
},
"remoteCommit" : {
"index" : 0,
"spec" : {
"htlcs" : [ ],
"commitTxFeerate" : 10000,
"toLocal" : 200000000,
"toRemote" : 800000000
},
"txid" : "287a0627974d69a85199233f5cb8e86e7d04d8afdbc0faa4e56587833838c5af",
"remotePerCommitmentPoint" : "03293b6fcd6474a6c8dff63eeed1ca704c15d5bb80177025c2c45a4f995723393d"
}
} ],
"inactive" : [ ],
"remoteNextCommitInfo" : "021492cff600e90bf43cfa885be167e2596428e3ab962ecc1f86df9d0ab6d0440c",
"remotePerCommitmentSecrets" : null,
"originChannels" : { }
},
"aliases" : {
"localAlias" : "0x37ca53d75c1340",
"remoteAlias" : "0x189a8cb29e4b168"
},
"channelUpdate" : {
"signature" : "61051293e9bb63a6e2182bce3dba766e0b0f69d546dcc059aee70e81fac8db1e56d2b10e4ecee3b99aaad8751fd396dbe9a6ba68d8c153dbc5e862afafd31daa",
"chainHash" : "06226e46111a0b59caaf126043eb5bbf28c34f3a5e332a1fc7b2b73cf188910f",
"shortChannelId" : "400000x42x0",
"timestamp" : {
"iso" : "2024-12-31T16:03:46Z",
"unix" : 1735661026
},
"messageFlags" : {
"dontForward" : false
},
"channelFlags" : {
"isEnabled" : true,
"isNode1" : true
},
"cltvExpiryDelta" : 144,
"htlcMinimumMsat" : 1000,
"feeBaseMsat" : 546000,
"feeProportionalMillionths" : 10,
"htlcMaximumMsat" : 500000000,
"tlvStream" : { }
}
}

File diff suppressed because one or more lines are too long

View file

@ -0,0 +1,244 @@
{
"type" : "DATA_NORMAL",
"commitments" : {
"params" : {
"channelId" : "9c5655e99c1ef6d219eb093ca9f29e642167b4c776928df11c73e2293086b749",
"channelConfig" : [ "funding_pubkey_based_channel_keypath" ],
"channelFeatures" : [ "option_static_remotekey", "option_dual_fund" ],
"localParams" : {
"nodeId" : "02bbbb671d15145722fb8c28d732cddb249bcc6652ed2b297ff1f77a18371b1e63",
"fundingKeyPath" : [ 2707154742, 1913609705, 398200054, 2009144985, 2117492679, 302300795, 2676284185, 1690699462, 2147483648 ],
"dustLimit" : 1000,
"maxHtlcValueInFlightMsat" : 9223372036854775807,
"htlcMinimum" : 1000,
"toSelfDelay" : 144,
"maxAcceptedHtlcs" : 30,
"isChannelOpener" : false,
"paysCommitTxFees" : false,
"walletStaticPaymentBasepoint" : "038bff1253b7b8e40532508a53d31b95b295df31edf0d067d734479a680bd395de",
"initFeatures" : {
"activated" : {
"option_route_blinding" : "optional",
"option_dual_fund" : "optional",
"splice_prototype" : "optional",
"payment_secret" : "mandatory",
"gossip_queries_ex" : "optional",
"option_quiesce" : "optional",
"option_data_loss_protect" : "optional",
"var_onion_optin" : "mandatory",
"option_static_remotekey" : "mandatory",
"option_support_large_channel" : "optional",
"option_anchors_zero_fee_htlc_tx" : "optional",
"basic_mpp" : "optional",
"gossip_queries" : "optional"
},
"unknown" : [ ]
}
},
"remoteParams" : {
"nodeId" : "02aaaa00ce2f18a967dc4f25f414e671ba6585f8ef0b8c5fb812c21064f55a2eaa",
"dustLimit" : 1100,
"maxHtlcValueInFlightMsat" : 9223372036854775807,
"htlcMinimum" : 0,
"toSelfDelay" : 144,
"maxAcceptedHtlcs" : 100,
"revocationBasepoint" : "02fab0332aecd8006f4937b220ea073b0e814a21e36f4d21de9bb3e5165c3ffa02",
"paymentBasepoint" : "038bff1253b7b8e40532508a53d31b95b295df31edf0d067d734479a680bd395de",
"delayedPaymentBasepoint" : "023a5359a4e12c66e5b6a1847161f837695ab6e7591e67cb5b1da82706b491ea12",
"htlcBasepoint" : "032d7ab7162c56d1d273b09e67312833bcc8891c76b73fc6f69843b0f053e15b7c",
"initFeatures" : {
"activated" : {
"option_support_large_channel" : "optional",
"option_route_blinding" : "optional",
"option_provide_storage" : "optional",
"option_dual_fund" : "optional",
"splice_prototype" : "optional",
"payment_secret" : "mandatory",
"gossip_queries_ex" : "optional",
"option_quiesce" : "optional",
"option_data_loss_protect" : "optional",
"var_onion_optin" : "mandatory",
"option_static_remotekey" : "mandatory",
"basic_mpp" : "optional",
"gossip_queries" : "optional"
},
"unknown" : [ 50001 ]
}
},
"channelFlags" : {
"nonInitiatorPaysCommitFees" : false,
"announceChannel" : false
}
},
"changes" : {
"localChanges" : {
"proposed" : [ ],
"signed" : [ ],
"acked" : [ ]
},
"remoteChanges" : {
"proposed" : [ ],
"acked" : [ ],
"signed" : [ ]
},
"localNextHtlcId" : 0,
"remoteNextHtlcId" : 0
},
"active" : [ {
"fundingTxIndex" : 2,
"fundingTx" : {
"outPoint" : "8883382b78ace90653166f985e331d0e0a759a5215cd0598ab5d1f72330d3101:0",
"amountSatoshis" : 2500000
},
"localFunding" : {
"status" : "unconfirmed",
"txid" : "8883382b78ace90653166f985e331d0e0a759a5215cd0598ab5d1f72330d3101"
},
"remoteFunding" : {
"status" : "not-locked"
},
"localCommit" : {
"index" : 0,
"spec" : {
"htlcs" : [ ],
"commitTxFeerate" : 10000,
"toLocal" : 700000000,
"toRemote" : 1800000000
},
"commitTxAndRemoteSig" : {
"commitTx" : {
"txid" : "ec1fdbdffd12f20851f33319fb3ab6ecb8528a34a9a2c76e53aaf91cf4a18da4",
"tx" : "020000000101310d33721f5dab9805cd15529a750a0e1d335e986f165306e9ac782b3883880000000000310038800260ae0a0000000000220020a1ca24b5ac3c96e066fa7466d34505e7e62bfe75017116d988328f97f4358160f85a1b0000000000160014b8a6877e8036939b79fcebe6c478d4bb234656b4a0f85620"
},
"remoteSig" : "52ea4ba9421f4b4ad8a9a1e3041a940b13751504faf7d79338d73bbc1187fb551d9340de809849d6d7dcd745b813e01c07809cb8eacc57a211990b0c021d4c45"
},
"htlcTxsAndRemoteSigs" : [ ]
},
"remoteCommit" : {
"index" : 0,
"spec" : {
"htlcs" : [ ],
"commitTxFeerate" : 10000,
"toLocal" : 1800000000,
"toRemote" : 700000000
},
"txid" : "ac57fb7fa3b5493412ad83d7d056f269f54133e69026306d34a9ede545ef145c",
"remotePerCommitmentPoint" : "0362028d59fc5fd2598f1fad1df3933b64367fd6a16ba88a90d31d396b588eac2a"
}
}, {
"fundingTxIndex" : 1,
"fundingTx" : {
"outPoint" : "91556cf5881038eb239cac05123f0be1daface928690e5f33ae1cc653f3d3005:0",
"amountSatoshis" : 2000000
},
"localFunding" : {
"status" : "confirmed",
"txid" : "91556cf5881038eb239cac05123f0be1daface928690e5f33ae1cc653f3d3005",
"shortChannelId" : "0x0x0"
},
"remoteFunding" : {
"status" : "not-locked"
},
"localCommit" : {
"index" : 0,
"spec" : {
"htlcs" : [ ],
"commitTxFeerate" : 10000,
"toLocal" : 700000000,
"toRemote" : 1300000000
},
"commitTxAndRemoteSig" : {
"commitTx" : {
"txid" : "27f7218158a68f5bb23c9d8a280f85bfca32d7636529e1db2ba23172241161f4",
"tx" : "020000000105303d3f65cce13af3e5908692cefadae10b3f1205ac9c23eb381088f56c55910000000000310038800260ae0a0000000000220020a1ca24b5ac3c96e066fa7466d34505e7e62bfe75017116d988328f97f4358160d8b9130000000000160014b8a6877e8036939b79fcebe6c478d4bb234656b4a0f85620"
},
"remoteSig" : "32a246840dd67eb72b59fc170303c508c77ee8666ac6be36abced02bf79dbfbe114cdd4e56600ae6e3573e6e10d3517ddc857fbd89c00e2cbcdf5186c6592f26"
},
"htlcTxsAndRemoteSigs" : [ ]
},
"remoteCommit" : {
"index" : 0,
"spec" : {
"htlcs" : [ ],
"commitTxFeerate" : 10000,
"toLocal" : 1300000000,
"toRemote" : 700000000
},
"txid" : "1f1392e4cee2d3fc5f4fed4afed2b962ea8df1a802dc9274043f337e8767838e",
"remotePerCommitmentPoint" : "0362028d59fc5fd2598f1fad1df3933b64367fd6a16ba88a90d31d396b588eac2a"
}
}, {
"fundingTxIndex" : 0,
"fundingTx" : {
"outPoint" : "421298cf7c9a9d12d35fd68231986b43f6fe90e4b499fff4bc3e36d52d2d215f:2",
"amountSatoshis" : 1500000
},
"localFunding" : {
"status" : "confirmed",
"txid" : "421298cf7c9a9d12d35fd68231986b43f6fe90e4b499fff4bc3e36d52d2d215f",
"shortChannelId" : "400000x42x2"
},
"remoteFunding" : {
"status" : "not-locked"
},
"localCommit" : {
"index" : 0,
"spec" : {
"htlcs" : [ ],
"commitTxFeerate" : 10000,
"toLocal" : 700000000,
"toRemote" : 800000000
},
"commitTxAndRemoteSig" : {
"commitTx" : {
"txid" : "381ee5a54f127b776114b197b2695c39f2d9b81cb967628a32e12766cdf8f1b5",
"tx" : "02000000015f212d2dd5363ebcf4ff99b4e490fef6436b983182d65fd3129d9a7ccf9812420200000000310038800260ae0a0000000000220020a1ca24b5ac3c96e066fa7466d34505e7e62bfe75017116d988328f97f4358160b8180c0000000000160014b8a6877e8036939b79fcebe6c478d4bb234656b4a0f85620"
},
"remoteSig" : "47f525be7304ef4fa4d868eccfa2389fe1d8f058d08305b0f1792cebbd14ec7b6b43f98d0f3aa22e60d7ef024785f3576d0c1d74b666c7c5d2da88c7e7f7bd18"
},
"htlcTxsAndRemoteSigs" : [ ]
},
"remoteCommit" : {
"index" : 0,
"spec" : {
"htlcs" : [ ],
"commitTxFeerate" : 10000,
"toLocal" : 800000000,
"toRemote" : 700000000
},
"txid" : "ea2430e6f9d234b9ca79d5edbdd0edbe429aa31026ad79ecc207299a62151d33",
"remotePerCommitmentPoint" : "0362028d59fc5fd2598f1fad1df3933b64367fd6a16ba88a90d31d396b588eac2a"
}
} ],
"inactive" : [ ],
"remoteNextCommitInfo" : "02c8a6e5a10f60d65a8d9aec86137898f4293bf8692917fd59a676efdc7956df27",
"remotePerCommitmentSecrets" : null,
"originChannels" : { }
},
"aliases" : {
"localAlias" : "0x1e9544574043065",
"remoteAlias" : "0x10644a8d60b0c11"
},
"channelUpdate" : {
"signature" : "1f06e865a73a5f7ca2cd4f4df80110ffb72623fab4a3b3edaab2cbd14886f94f1e8d7c1586a668a5e0c7acb17a6fdf9539b63be05a2a650eca95f79d9af23ee1",
"chainHash" : "06226e46111a0b59caaf126043eb5bbf28c34f3a5e332a1fc7b2b73cf188910f",
"shortChannelId" : "125268x4551684x12389",
"timestamp" : {
"iso" : "2024-12-31T16:12:58Z",
"unix" : 1735661578
},
"messageFlags" : {
"dontForward" : true
},
"channelFlags" : {
"isEnabled" : true,
"isNode1" : false
},
"cltvExpiryDelta" : 144,
"htlcMinimumMsat" : 0,
"feeBaseMsat" : 547000,
"feeProportionalMillionths" : 20,
"htlcMaximumMsat" : 9223372036854775807,
"tlvStream" : { }
}
}

View file

@ -363,16 +363,15 @@ class EclairImplSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with I
channel1.shortChannelId -> PublicChannel(channel1, TxId(ByteVector32.Zeroes), 100_000 sat, None, None, None),
channel2.shortChannelId -> PublicChannel(channel2, TxId(ByteVector32.Zeroes), 150_000 sat, None, None, None),
)
val (channelId3, shortIds3) = (randomBytes32(), ShortIds(None, Alias(13), None))
val (channelId4, shortIds4) = (randomBytes32(), ShortIds(Some(RealShortChannelId(4)), Alias(14), None))
val (channelId3, aliases3) = (randomBytes32(), ShortIdAliases(Alias(13), None))
val (channelId4, aliases4) = (randomBytes32(), ShortIdAliases(Alias(14), None))
val privateChannels = Map(
channelId3 -> PrivateChannel(channelId3, shortIds3, a, b, None, None, Router.ChannelMeta(25_000 msat, 50_000 msat)),
channelId4 -> PrivateChannel(channelId4, shortIds4, a, c, None, None, Router.ChannelMeta(75_000 msat, 10_000 msat)),
channelId3 -> PrivateChannel(channelId3, aliases3, a, b, None, None, Router.ChannelMeta(25_000 msat, 50_000 msat)),
channelId4 -> PrivateChannel(channelId4, aliases4, a, c, None, None, Router.ChannelMeta(75_000 msat, 10_000 msat)),
)
val scidMapping = Map(
shortIds3.localAlias.toLong -> channelId3,
shortIds4.localAlias.toLong -> channelId4,
shortIds4.real_opt.get.toLong -> channelId4,
aliases3.localAlias.toLong -> channelId3,
aliases4.localAlias.toLong -> channelId4,
)
val g = GraphWithBalanceEstimates(DirectedGraph(Nil), 1 hour)
val routerData = Router.Data(Map.empty, publicChannels, SortedMap.empty, Router.Stash(Map.empty, Map.empty), Router.Rebroadcast(Map.empty, Map.empty, Map.empty), Map.empty, privateChannels, scidMapping, Map.empty, g, Map.empty, Map.empty)
@ -384,7 +383,7 @@ class EclairImplSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with I
val unknownNodeId = randomKey().publicKey
val unknownScid = Alias(42)
eclair.findRoute(c, 250_000 msat, None, ignoreNodeIds = Seq(b, unknownNodeId), ignoreShortChannelIds = Seq(channel1.shortChannelId, shortIds3.localAlias, shortIds4.real_opt.get, unknownScid))
eclair.findRoute(c, 250_000 msat, None, ignoreNodeIds = Seq(b, unknownNodeId), ignoreShortChannelIds = Seq(channel1.shortChannelId, aliases3.localAlias, unknownScid))
router.expectMsg(Router.GetRouterData)
router.reply(routerData)
val routeRequest2 = router.expectMsgType[RouteRequest]
@ -393,10 +392,8 @@ class EclairImplSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with I
assert(routeRequest2.ignore.channels == Set(
Router.ChannelDesc(channel1.shortChannelId, a, b),
Router.ChannelDesc(channel1.shortChannelId, b, a),
Router.ChannelDesc(shortIds3.localAlias, a, b),
Router.ChannelDesc(shortIds3.localAlias, b, a),
Router.ChannelDesc(shortIds4.real_opt.get, a, c),
Router.ChannelDesc(shortIds4.real_opt.get, c, a),
Router.ChannelDesc(aliases3.localAlias, a, b),
Router.ChannelDesc(aliases3.localAlias, b, a),
))
}

View file

@ -17,7 +17,7 @@
package fr.acinq.eclair.channel
import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
import fr.acinq.bitcoin.scalacompat.{Block, ByteVector32, ByteVector64, DeterministicWallet, Satoshi, SatoshiLong, Transaction}
import fr.acinq.bitcoin.scalacompat.{Block, ByteVector32, ByteVector64, DeterministicWallet, OutPoint, Satoshi, SatoshiLong, Transaction, TxOut}
import fr.acinq.eclair.TestUtils.randomTxId
import fr.acinq.eclair._
import fr.acinq.eclair.blockchain.fee._
@ -25,9 +25,9 @@ import fr.acinq.eclair.channel.Helpers.Funding
import fr.acinq.eclair.channel.states.ChannelStateTestsBase
import fr.acinq.eclair.crypto.ShaChain
import fr.acinq.eclair.crypto.keymanager.LocalChannelKeyManager
import fr.acinq.eclair.transactions.CommitmentSpec
import fr.acinq.eclair.transactions.Transactions.CommitTx
import fr.acinq.eclair.wire.protocol.{FailureReason, IncorrectOrUnknownPaymentDetails, UpdateAddHtlc, UpdateFailHtlc}
import fr.acinq.eclair.transactions.{CommitmentSpec, Scripts, Transactions}
import fr.acinq.eclair.wire.protocol._
import org.scalatest.funsuite.FixtureAnyFunSuiteLike
import org.scalatest.{Outcome, Tag}
@ -485,18 +485,24 @@ class CommitmentsSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
object CommitmentsSpec {
def makeCommitments(toLocal: MilliSatoshi, toRemote: MilliSatoshi, feeRatePerKw: FeeratePerKw = FeeratePerKw(0 sat), dustLimit: Satoshi = 0 sat, isOpener: Boolean = true, announceChannel: Boolean = true): Commitments = {
def makeCommitments(toLocal: MilliSatoshi, toRemote: MilliSatoshi, feeRatePerKw: FeeratePerKw = FeeratePerKw(0 sat), dustLimit: Satoshi = 0 sat, isOpener: Boolean = true, announcement_opt: Option[ChannelAnnouncement] = None): Commitments = {
val channelReserve = (toLocal + toRemote).truncateToSatoshi * 0.01
val localParams = LocalParams(randomKey().publicKey, DeterministicWallet.KeyPath(Seq(42L)), dustLimit, Long.MaxValue.msat, Some(channelReserve), 1 msat, CltvExpiryDelta(144), 50, isOpener, isOpener, None, None, Features.empty)
val remoteParams = RemoteParams(randomKey().publicKey, dustLimit, UInt64.MaxValue, Some(channelReserve), 1 msat, CltvExpiryDelta(144), 50, randomKey().publicKey, randomKey().publicKey, randomKey().publicKey, randomKey().publicKey, Features.empty, None)
val localFundingPubKey = randomKey().publicKey
val remoteFundingPubKey = randomKey().publicKey
val commitmentInput = Funding.makeFundingInputInfo(randomTxId(), 0, (toLocal + toRemote).truncateToSatoshi, randomKey().publicKey, remoteFundingPubKey)
val fundingTx = Transaction(2, Nil, Seq(TxOut((toLocal + toRemote).truncateToSatoshi, Funding.makeFundingPubKeyScript(localFundingPubKey, remoteFundingPubKey))), 0)
val commitmentInput = Transactions.InputInfo(OutPoint(fundingTx, 0), fundingTx.txOut.head, Scripts.multiSig2of2(localFundingPubKey, remoteFundingPubKey))
val localCommit = LocalCommit(0, CommitmentSpec(Set.empty, feeRatePerKw, toLocal, toRemote), CommitTxAndRemoteSig(CommitTx(commitmentInput, Transaction(2, Nil, Nil, 0)), ByteVector64.Zeroes), Nil)
val remoteCommit = RemoteCommit(0, CommitmentSpec(Set.empty, feeRatePerKw, toRemote, toLocal), randomTxId(), randomKey().publicKey)
val localFundingStatus = announcement_opt match {
case Some(ann) => LocalFundingStatus.ConfirmedFundingTx(fundingTx, ann.shortChannelId, Some(ann), None, None)
case None => LocalFundingStatus.SingleFundedUnconfirmedFundingTx(None)
}
Commitments(
ChannelParams(randomBytes32(), ChannelConfig.standard, ChannelFeatures(), localParams, remoteParams, ChannelFlags(announceChannel = announceChannel)),
ChannelParams(randomBytes32(), ChannelConfig.standard, ChannelFeatures(), localParams, remoteParams, ChannelFlags(announceChannel = announcement_opt.nonEmpty)),
CommitmentChanges(LocalChanges(Nil, Nil, Nil), RemoteChanges(Nil, Nil, Nil), localNextHtlcId = 1, remoteNextHtlcId = 1),
List(Commitment(0, 0, remoteFundingPubKey, LocalFundingStatus.SingleFundedUnconfirmedFundingTx(None), RemoteFundingStatus.Locked, localCommit, remoteCommit, None)),
List(Commitment(0, 0, remoteFundingPubKey, localFundingStatus, RemoteFundingStatus.Locked, localCommit, remoteCommit, None)),
inactive = Nil,
Right(randomKey().publicKey),
ShaChain.init,
@ -504,18 +510,24 @@ object CommitmentsSpec {
)
}
def makeCommitments(toLocal: MilliSatoshi, toRemote: MilliSatoshi, localNodeId: PublicKey, remoteNodeId: PublicKey, announceChannel: Boolean): Commitments = {
def makeCommitments(toLocal: MilliSatoshi, toRemote: MilliSatoshi, localNodeId: PublicKey, remoteNodeId: PublicKey, announcement_opt: Option[ChannelAnnouncement]): Commitments = {
val channelReserve = (toLocal + toRemote).truncateToSatoshi * 0.01
val localParams = LocalParams(localNodeId, DeterministicWallet.KeyPath(Seq(42L)), 0 sat, Long.MaxValue.msat, Some(channelReserve), 1 msat, CltvExpiryDelta(144), 50, isChannelOpener = true, paysCommitTxFees = true, None, None, Features.empty)
val remoteParams = RemoteParams(remoteNodeId, 0 sat, UInt64.MaxValue, Some(channelReserve), 1 msat, CltvExpiryDelta(144), 50, randomKey().publicKey, randomKey().publicKey, randomKey().publicKey, randomKey().publicKey, Features.empty, None)
val localFundingPubKey = randomKey().publicKey
val remoteFundingPubKey = randomKey().publicKey
val commitmentInput = Funding.makeFundingInputInfo(randomTxId(), 0, (toLocal + toRemote).truncateToSatoshi, randomKey().publicKey, remoteFundingPubKey)
val fundingTx = Transaction(2, Nil, Seq(TxOut((toLocal + toRemote).truncateToSatoshi, Funding.makeFundingPubKeyScript(localFundingPubKey, remoteFundingPubKey))), 0)
val commitmentInput = Transactions.InputInfo(OutPoint(fundingTx, 0), fundingTx.txOut.head, Scripts.multiSig2of2(localFundingPubKey, remoteFundingPubKey))
val localCommit = LocalCommit(0, CommitmentSpec(Set.empty, FeeratePerKw(0 sat), toLocal, toRemote), CommitTxAndRemoteSig(CommitTx(commitmentInput, Transaction(2, Nil, Nil, 0)), ByteVector64.Zeroes), Nil)
val remoteCommit = RemoteCommit(0, CommitmentSpec(Set.empty, FeeratePerKw(0 sat), toRemote, toLocal), randomTxId(), randomKey().publicKey)
val localFundingStatus = announcement_opt match {
case Some(ann) => LocalFundingStatus.ConfirmedFundingTx(fundingTx, ann.shortChannelId, Some(ann), None, None)
case None => LocalFundingStatus.SingleFundedUnconfirmedFundingTx(None)
}
Commitments(
ChannelParams(randomBytes32(), ChannelConfig.standard, ChannelFeatures(), localParams, remoteParams, ChannelFlags(announceChannel = announceChannel)),
ChannelParams(randomBytes32(), ChannelConfig.standard, ChannelFeatures(), localParams, remoteParams, ChannelFlags(announceChannel = announcement_opt.nonEmpty)),
CommitmentChanges(LocalChanges(Nil, Nil, Nil), RemoteChanges(Nil, Nil, Nil), localNextHtlcId = 1, remoteNextHtlcId = 1),
List(Commitment(0, 0, remoteFundingPubKey, LocalFundingStatus.SingleFundedUnconfirmedFundingTx(None), RemoteFundingStatus.Locked, localCommit, remoteCommit, None)),
List(Commitment(0, 0, remoteFundingPubKey, localFundingStatus, RemoteFundingStatus.Locked, localCommit, remoteCommit, None)),
inactive = Nil,
Right(randomKey().publicKey),
ShaChain.init,

View file

@ -107,10 +107,10 @@ class WaitForChannelReadyStateSpec extends TestKitBaseClass with FixtureAnyFunSu
test("recv ChannelReady") { f =>
import f._
// we have a real scid at this stage, because this isn't a zero-conf channel
val aliceIds = alice.stateData.asInstanceOf[DATA_WAIT_FOR_CHANNEL_READY].shortIds
assert(aliceIds.real_opt.nonEmpty)
val bobIds = bob.stateData.asInstanceOf[DATA_WAIT_FOR_CHANNEL_READY].shortIds
assert(bobIds.real_opt.nonEmpty)
val aliceIds = alice.stateData.asInstanceOf[DATA_WAIT_FOR_CHANNEL_READY].aliases
assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_CHANNEL_READY].commitments.latest.shortChannelId_opt.nonEmpty)
val bobIds = bob.stateData.asInstanceOf[DATA_WAIT_FOR_CHANNEL_READY].aliases
assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_CHANNEL_READY].commitments.latest.shortChannelId_opt.nonEmpty)
val channelReady = bob2alice.expectMsgType[ChannelReady]
assert(channelReady.alias_opt.contains(bobIds.localAlias))
val listener = TestProbe()
@ -134,8 +134,8 @@ class WaitForChannelReadyStateSpec extends TestKitBaseClass with FixtureAnyFunSu
test("recv ChannelReady (no alias)") { f =>
import f._
// we have a real scid at this stage, because this isn't a zero-conf channel
val aliceIds = alice.stateData.asInstanceOf[DATA_WAIT_FOR_CHANNEL_READY].shortIds
val realScid = aliceIds.real_opt.get
val aliceIds = alice.stateData.asInstanceOf[DATA_WAIT_FOR_CHANNEL_READY].aliases
val realScid = alice.stateData.asInstanceOf[DATA_WAIT_FOR_CHANNEL_READY].commitments.latest.shortChannelId_opt.get
val channelReady = bob2alice.expectMsgType[ChannelReady]
val channelReadyNoAlias = channelReady.modify(_.tlvStream.records).using(_.filterNot(_.isInstanceOf[ChannelReadyTlv.ShortChannelIdTlv]))
bob2alice.forward(alice, channelReadyNoAlias)
@ -155,10 +155,8 @@ class WaitForChannelReadyStateSpec extends TestKitBaseClass with FixtureAnyFunSu
test("recv ChannelReady (zero-conf)", Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs), Tag(ChannelStateTestsTags.ZeroConf)) { f =>
import f._
// zero-conf channel: we don't have a real scid
val aliceIds = alice.stateData.asInstanceOf[DATA_WAIT_FOR_CHANNEL_READY].shortIds
assert(aliceIds.real_opt.isEmpty)
val bobIds = bob.stateData.asInstanceOf[DATA_WAIT_FOR_CHANNEL_READY].shortIds
assert(bobIds.real_opt.isEmpty)
val aliceIds = alice.stateData.asInstanceOf[DATA_WAIT_FOR_CHANNEL_READY].aliases
val bobIds = bob.stateData.asInstanceOf[DATA_WAIT_FOR_CHANNEL_READY].aliases
val channelReady = bob2alice.expectMsgType[ChannelReady]
assert(channelReady.alias_opt.contains(bobIds.localAlias))
val listener = TestProbe()
@ -181,10 +179,7 @@ class WaitForChannelReadyStateSpec extends TestKitBaseClass with FixtureAnyFunSu
test("recv ChannelReady (zero-conf, no alias)", Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs), Tag(ChannelStateTestsTags.ZeroConf)) { f =>
import f._
// zero-conf channel: we don't have a real scid
val aliceIds = alice.stateData.asInstanceOf[DATA_WAIT_FOR_CHANNEL_READY].shortIds
assert(aliceIds.real_opt.isEmpty)
val bobIds = bob.stateData.asInstanceOf[DATA_WAIT_FOR_CHANNEL_READY].shortIds
assert(bobIds.real_opt.isEmpty)
val aliceIds = alice.stateData.asInstanceOf[DATA_WAIT_FOR_CHANNEL_READY].aliases
val channelReady = bob2alice.expectMsgType[ChannelReady]
val channelReadyNoAlias = channelReady.modify(_.tlvStream.records).using(_.filterNot(_.isInstanceOf[ChannelReadyTlv.ShortChannelIdTlv]))
bob2alice.forward(alice, channelReadyNoAlias)
@ -204,17 +199,14 @@ class WaitForChannelReadyStateSpec extends TestKitBaseClass with FixtureAnyFunSu
test("recv ChannelReady (public)", Tag(ChannelStateTestsTags.ChannelsPublic)) { f =>
import f._
// we have a real scid at this stage, because this isn't a zero-conf channel
val aliceIds = alice.stateData.asInstanceOf[DATA_WAIT_FOR_CHANNEL_READY].shortIds
assert(aliceIds.real_opt.nonEmpty)
val aliceIds = alice.stateData.asInstanceOf[DATA_WAIT_FOR_CHANNEL_READY].aliases
assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_CHANNEL_READY].commitments.params.channelFlags.announceChannel)
val bobIds = bob.stateData.asInstanceOf[DATA_WAIT_FOR_CHANNEL_READY].shortIds
assert(bobIds.real_opt.nonEmpty)
val bobIds = bob.stateData.asInstanceOf[DATA_WAIT_FOR_CHANNEL_READY].aliases
assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_CHANNEL_READY].commitments.params.channelFlags.announceChannel)
val channelReady = bob2alice.expectMsgType[ChannelReady]
assert(channelReady.alias_opt.contains(bobIds.localAlias))
bob2alice.forward(alice)
val annSigs = alice2bob.expectMsgType[AnnouncementSignatures]
assert(aliceIds.real_opt.contains(annSigs.shortChannelId))
alice2bob.expectMsgType[AnnouncementSignatures]
val initialChannelUpdate = alice.stateData.asInstanceOf[DATA_NORMAL].channelUpdate
assert(initialChannelUpdate.shortChannelId == aliceIds.localAlias)
assert(initialChannelUpdate.feeBaseMsat == relayFees.feeBase)
@ -231,10 +223,8 @@ class WaitForChannelReadyStateSpec extends TestKitBaseClass with FixtureAnyFunSu
test("recv ChannelReady (public, zero-conf)", Tag(ChannelStateTestsTags.ChannelsPublic), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs), Tag(ChannelStateTestsTags.ZeroConf)) { f =>
import f._
// zero-conf channel: we don't have a real scid
val aliceIds = alice.stateData.asInstanceOf[DATA_WAIT_FOR_CHANNEL_READY].shortIds
assert(aliceIds.real_opt.isEmpty)
val bobIds = bob.stateData.asInstanceOf[DATA_WAIT_FOR_CHANNEL_READY].shortIds
assert(bobIds.real_opt.isEmpty)
val aliceIds = alice.stateData.asInstanceOf[DATA_WAIT_FOR_CHANNEL_READY].aliases
val bobIds = bob.stateData.asInstanceOf[DATA_WAIT_FOR_CHANNEL_READY].aliases
val channelReady = bob2alice.expectMsgType[ChannelReady]
assert(channelReady.alias_opt.contains(bobIds.localAlias))
bob2alice.forward(alice)

View file

@ -195,7 +195,7 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture
assert(alice2blockchain.expectMsgType[WatchFundingConfirmed].txId == fundingTx.txid)
alice2blockchain.expectNoMessage(100 millis) // we don't set WatchFundingSpent
alice2bob.expectMsgType[ChannelReady]
assert(aliceListener.expectMsgType[ShortChannelIdAssigned].shortIds.real_opt.isEmpty)
assert(aliceListener.expectMsgType[ShortChannelIdAssigned].announcement_opt.isEmpty)
awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_READY)
}
@ -206,7 +206,7 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture
assert(bob2blockchain.expectMsgType[WatchFundingConfirmed].txId == fundingTx.txid)
bob2blockchain.expectNoMessage(100 millis) // we don't set WatchFundingSpent
bob2alice.expectMsgType[ChannelReady]
assert(bobListener.expectMsgType[ShortChannelIdAssigned].shortIds.real_opt.isEmpty)
assert(bobListener.expectMsgType[ShortChannelIdAssigned].announcement_opt.isEmpty)
awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_READY)
}
@ -220,7 +220,7 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture
assert(alice2blockchain.expectMsgType[WatchFundingConfirmed].txId == fundingTx.txid)
alice2blockchain.expectNoMessage(100 millis) // we don't set WatchFundingSpent
alice2bob.expectNoMessage(100 millis)
assert(aliceListener.expectMsgType[ShortChannelIdAssigned].shortIds.real_opt.isEmpty)
assert(aliceListener.expectMsgType[ShortChannelIdAssigned].announcement_opt.isEmpty)
awaitCond(alice.stateData.isInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_READY])
assert(alice.stateName == OFFLINE)
}
@ -232,7 +232,7 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture
assert(alice2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTx.txid)
alice2bob.expectMsgType[ChannelReady]
assert(aliceListener.expectMsgType[TransactionConfirmed].tx == fundingTx)
assert(aliceListener.expectMsgType[ShortChannelIdAssigned].shortIds.real_opt.nonEmpty)
assert(aliceListener.expectMsgType[ShortChannelIdAssigned].announcement_opt.isEmpty)
awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_READY)
}
@ -243,7 +243,7 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture
assert(bob2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTx.txid)
bob2alice.expectMsgType[ChannelReady]
assert(bobListener.expectMsgType[TransactionConfirmed].tx == fundingTx)
assert(bobListener.expectMsgType[ShortChannelIdAssigned].shortIds.real_opt.nonEmpty)
assert(bobListener.expectMsgType[ShortChannelIdAssigned].announcement_opt.isEmpty)
awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_READY)
}
@ -258,9 +258,8 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture
alice2blockchain.expectNoMessage(100 millis)
alice2bob.expectNoMessage(100 millis)
assert(aliceListener.expectMsgType[TransactionConfirmed].tx == fundingTx)
assert(aliceListener.expectMsgType[ShortChannelIdAssigned].shortIds.real_opt.nonEmpty)
assert(aliceListener.expectMsgType[ShortChannelIdAssigned].announcement_opt.isEmpty)
awaitCond(alice.stateData.isInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_READY])
assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_READY].shortIds.real_opt.nonEmpty)
assert(alice.stateName == OFFLINE)
}
@ -687,10 +686,9 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture
assert(alice2blockchain.expectMsgType[WatchPublished].txId == fundingTx.txid)
alice ! WatchPublishedTriggered(fundingTx)
alice2blockchain.expectMsgType[WatchFundingConfirmed]
val scids = aliceListener.expectMsgType[ShortChannelIdAssigned].shortIds
assert(scids.real_opt.isEmpty)
val aliases = aliceListener.expectMsgType[ShortChannelIdAssigned].aliases
val aliceChannelReady = alice2bob.expectMsgType[ChannelReady]
assert(aliceChannelReady.alias_opt.contains(scids.localAlias))
assert(aliceChannelReady.alias_opt.contains(aliases.localAlias))
alice2blockchain.expectNoMessage(100 millis)
awaitCond(alice.stateName == NORMAL)
}
@ -743,7 +741,7 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture
// The funding tx confirms while we're offline.
alice ! WatchFundingConfirmedTriggered(BlockHeight(42000), 42, fundingTx)
assert(aliceListener.expectMsgType[TransactionConfirmed].tx == fundingTx)
assert(aliceListener.expectMsgType[ShortChannelIdAssigned].shortIds.real_opt.nonEmpty)
assert(aliceListener.expectMsgType[ShortChannelIdAssigned].announcement_opt.isEmpty)
assert(alice2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTx.txid)
alice2blockchain.expectNoMessage(100 millis)
alice2bob.expectNoMessage(100 millis)
@ -777,7 +775,7 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture
// A previous funding tx confirms while we're offline.
alice ! WatchFundingConfirmedTriggered(BlockHeight(42000), 42, fundingTx1)
assert(aliceListener.expectMsgType[TransactionConfirmed].tx == fundingTx1)
assert(aliceListener.expectMsgType[ShortChannelIdAssigned].shortIds.real_opt.nonEmpty)
assert(aliceListener.expectMsgType[ShortChannelIdAssigned].announcement_opt.isEmpty)
assert(alice2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTx1.txid)
alice2blockchain.expectMsg(UnwatchTxConfirmed(fundingTx2.txId))
alice2blockchain.expectNoMessage(100 millis)

View file

@ -145,16 +145,16 @@ class WaitForDualFundingReadyStateSpec extends TestKitBaseClass with FixtureAnyF
case _ => false
}
assert(alice.stateData.asInstanceOf[DATA_NORMAL].shortIds.real_opt.nonEmpty)
val aliceCommitments = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest
assert(aliceCommitments.commitment.shortChannelId_opt.nonEmpty)
val aliceUpdate = alice.stateData.asInstanceOf[DATA_NORMAL].channelUpdate
assert(aliceUpdate.shortChannelId == aliceChannelReady.alias_opt.value)
assert(aliceUpdate.feeBaseMsat == 20.msat)
assert(aliceUpdate.feeProportionalMillionths == 125)
assert(aliceCommitments.localChannelReserve == aliceCommitments.commitInput.txOut.amount / 100)
assert(aliceCommitments.localChannelReserve == aliceCommitments.remoteChannelReserve)
assert(bob.stateData.asInstanceOf[DATA_NORMAL].shortIds.real_opt.nonEmpty)
val bobCommitments = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.latest
assert(bobCommitments.commitment.shortChannelId_opt.nonEmpty)
val bobUpdate = bob.stateData.asInstanceOf[DATA_NORMAL].channelUpdate
assert(bobUpdate.shortChannelId == bobChannelReady.alias_opt.value)
assert(bobUpdate.feeBaseMsat == 25.msat)
@ -185,13 +185,13 @@ class WaitForDualFundingReadyStateSpec extends TestKitBaseClass with FixtureAnyF
listenerA.expectMsg(ChannelOpened(alice, bob.underlyingActor.nodeParams.nodeId, channelId(alice)))
awaitCond(alice.stateName == NORMAL)
assert(alice.stateData.asInstanceOf[DATA_NORMAL].shortIds.real_opt.isEmpty)
val aliceCommitments = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest
assert(aliceCommitments.commitment.shortChannelId_opt.isEmpty)
assert(alice.stateData.asInstanceOf[DATA_NORMAL].channelUpdate.shortChannelId == aliceChannelReady.alias_opt.value)
assert(aliceCommitments.localChannelReserve == aliceCommitments.commitInput.txOut.amount / 100)
assert(aliceCommitments.localChannelReserve == aliceCommitments.remoteChannelReserve)
assert(bob.stateData.asInstanceOf[DATA_NORMAL].shortIds.real_opt.isEmpty)
val bobCommitments = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.latest
assert(bobCommitments.commitment.shortChannelId_opt.isEmpty)
assert(bob.stateData.asInstanceOf[DATA_NORMAL].channelUpdate.shortChannelId == bobChannelReady.alias_opt.value)
assert(bobCommitments.localChannelReserve == aliceCommitments.remoteChannelReserve)
assert(bobCommitments.localChannelReserve == bobCommitments.remoteChannelReserve)
@ -221,7 +221,7 @@ class WaitForDualFundingReadyStateSpec extends TestKitBaseClass with FixtureAnyF
bob2alice.expectMsgType[AnnouncementSignatures]
val bobChannelUpdate = bob2alice.expectMsgType[ChannelUpdate]
assert(bobChannelUpdate.shortChannelId == aliceChannelReady.alias_opt.value)
awaitCond(bob.stateData.asInstanceOf[DATA_NORMAL].channelAnnouncement.nonEmpty)
awaitCond(bob.stateData.asInstanceOf[DATA_NORMAL].lastAnnouncement_opt.nonEmpty)
// Alice disconnects without receiving Bob's announcement_signatures.
alice ! INPUT_DISCONNECTED
@ -248,7 +248,7 @@ class WaitForDualFundingReadyStateSpec extends TestKitBaseClass with FixtureAnyF
alice2bob.forward(bob)
bob2alice.expectMsgType[AnnouncementSignatures]
bob2alice.forward(alice)
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].channelAnnouncement.nonEmpty)
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].lastAnnouncement_opt.nonEmpty)
}
test("recv TxInitRbf", Tag(ChannelStateTestsTags.DualFunding), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>

View file

@ -1349,6 +1349,165 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
}
}
test("recv announcement_signatures", Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs), Tag(ChannelStateTestsTags.ChannelsPublic), Tag(ChannelStateTestsTags.DoNotInterceptGossip)) { f =>
import f._
val aliceListener = TestProbe()
alice.underlyingActor.context.system.eventStream.subscribe(aliceListener.ref, classOf[ShortChannelIdAssigned])
val bobListener = TestProbe()
bob.underlyingActor.context.system.eventStream.subscribe(bobListener.ref, classOf[ShortChannelIdAssigned])
// Alice and Bob announce the initial funding transaction.
alice2bob.expectMsgType[AnnouncementSignatures]
alice2bob.forward(bob)
alice2bob.expectMsgType[ChannelUpdate]
bob2alice.expectMsgType[AnnouncementSignatures]
bob2alice.forward(alice)
bob2alice.expectMsgType[ChannelUpdate]
awaitAssert(assert(alice.stateData.asInstanceOf[DATA_NORMAL].lastAnnouncement_opt.nonEmpty))
val ann = alice.stateData.asInstanceOf[DATA_NORMAL].lastAnnouncement_opt.get
assert(aliceListener.expectMsgType[ShortChannelIdAssigned].announcement_opt.contains(ann))
assert(bobListener.expectMsgType[ShortChannelIdAssigned].announcement_opt.contains(ann))
// Alice and Bob create a first splice transaction.
val spliceTx1 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat)))
alice2blockchain.expectWatchFundingConfirmed(spliceTx1.txid)
bob2blockchain.expectWatchFundingConfirmed(spliceTx1.txid)
// Alice sees the splice transaction confirm.
alice ! WatchFundingConfirmedTriggered(BlockHeight(1105), 37, spliceTx1)
alice2blockchain.expectWatchFundingSpent(spliceTx1.txid)
assert(alice2bob.expectMsgType[SpliceLocked].fundingTxId == spliceTx1.txid)
alice2bob.forward(bob)
bob2alice.expectNoMessage(100 millis)
// Bob sees the splice transaction confirm and receives Alice's announcement_signatures.
bob ! WatchFundingConfirmedTriggered(BlockHeight(1105), 37, spliceTx1)
bob2blockchain.expectWatchFundingSpent(spliceTx1.txid)
assert(bob2alice.expectMsgType[SpliceLocked].fundingTxId == spliceTx1.txid)
bob2alice.forward(alice)
alice2bob.expectMsgType[AnnouncementSignatures]
alice2bob.forward(bob)
val bobAnnSigs1 = bob2alice.expectMsgType[AnnouncementSignatures] // Alice doesn't receive Bob's signatures.
awaitAssert(assert(bob.stateData.asInstanceOf[DATA_NORMAL].lastAnnouncement_opt.exists(_ != ann)))
val spliceAnn1 = bob.stateData.asInstanceOf[DATA_NORMAL].lastAnnouncement_opt.get
assert(spliceAnn1.shortChannelId != ann.shortChannelId)
assert(bobListener.expectMsgType[ShortChannelIdAssigned].announcement_opt.contains(spliceAnn1))
assert(alice.stateData.asInstanceOf[DATA_NORMAL].lastAnnouncement_opt.contains(ann))
aliceListener.expectNoMessage(100 millis)
// Bob can prune previous commitments, but Alice cannot because she hasn't created the announcement yet.
awaitAssert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.size == 1)
awaitAssert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.inactive.size == 1)
awaitAssert(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active.size == 1)
awaitAssert(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.inactive.isEmpty)
// Alice and Bob create a second splice transaction.
val spliceTx2 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(100_000 sat)))
alice2blockchain.expectWatchFundingConfirmed(spliceTx2.txid)
bob2blockchain.expectWatchFundingConfirmed(spliceTx2.txid)
// Alice sees the splice transaction confirm.
alice ! WatchFundingConfirmedTriggered(BlockHeight(1729), 27, spliceTx2)
alice2blockchain.expectWatchFundingSpent(spliceTx2.txid)
assert(alice2bob.expectMsgType[SpliceLocked].fundingTxId == spliceTx2.txid)
alice2bob.forward(bob)
// Bob sees the splice transaction confirm.
bob ! WatchFundingConfirmedTriggered(BlockHeight(1729), 27, spliceTx2)
bob2blockchain.expectWatchFundingSpent(spliceTx2.txid)
assert(bob2alice.expectMsgType[SpliceLocked].fundingTxId == spliceTx2.txid)
bob2alice.forward(alice)
alice2bob.expectMsgType[AnnouncementSignatures]
alice2bob.forward(bob)
val bobAnnSigs2 = bob2alice.expectMsgType[AnnouncementSignatures] // Alice doesn't receive Bob's signatures.
awaitAssert(assert(bob.stateData.asInstanceOf[DATA_NORMAL].lastAnnouncement_opt.exists(_ != spliceAnn1)))
val spliceAnn2 = bob.stateData.asInstanceOf[DATA_NORMAL].lastAnnouncement_opt.get
assert(spliceAnn2.shortChannelId != spliceAnn1.shortChannelId)
assert(bobListener.expectMsgType[ShortChannelIdAssigned].announcement_opt.contains(spliceAnn2))
assert(alice.stateData.asInstanceOf[DATA_NORMAL].lastAnnouncement_opt.contains(ann))
aliceListener.expectNoMessage(100 millis)
// Bob can prune previous commitments, but Alice cannot because she hasn't created the announcement yet.
awaitAssert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.size == 1)
awaitAssert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.inactive.size == 2)
awaitAssert(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active.size == 1)
awaitAssert(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.inactive.isEmpty)
// Alice receives Bob's announcement_signatures.
bob2alice.forward(alice, bobAnnSigs2)
assert(aliceListener.expectMsgType[ShortChannelIdAssigned].announcement_opt.contains(spliceAnn2))
awaitAssert(assert(alice.stateData.asInstanceOf[DATA_NORMAL].lastAnnouncement_opt.contains(spliceAnn2)))
awaitAssert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.size == 1)
awaitAssert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.inactive.isEmpty)
// Alice receives Bob's previous announcement_signatures.
bob2alice.forward(alice, bobAnnSigs1)
alice2bob.expectNoMessage(100 millis)
aliceListener.expectNoMessage(100 millis)
assert(alice.stateData.asInstanceOf[DATA_NORMAL].lastAnnouncement_opt.contains(spliceAnn2))
}
test("recv announcement_signatures (after restart)", Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs), Tag(ChannelStateTestsTags.ChannelsPublic), Tag(ChannelStateTestsTags.DoNotInterceptGossip)) { f =>
import f._
val aliceListener = TestProbe()
alice.underlyingActor.context.system.eventStream.subscribe(aliceListener.ref, classOf[ShortChannelIdAssigned])
val bobListener = TestProbe()
bob.underlyingActor.context.system.eventStream.subscribe(bobListener.ref, classOf[ShortChannelIdAssigned])
// Alice and Bob want to announce the initial funding transaction, but the messages are dropped.
val shortChannelId = alice2bob.expectMsgType[AnnouncementSignatures].shortChannelId
alice2bob.expectMsgType[ChannelUpdate]
bob2alice.expectMsgType[AnnouncementSignatures]
bob2alice.expectMsgType[ChannelUpdate]
assert(alice.stateData.asInstanceOf[DATA_NORMAL].lastAnnouncement_opt.isEmpty)
assert(bob.stateData.asInstanceOf[DATA_NORMAL].lastAnnouncement_opt.isEmpty)
// Alice and Bob create a splice transaction.
val spliceTx = initiateSplice(f, spliceIn_opt = Some(SpliceIn(250_000 sat)))
alice2blockchain.expectWatchFundingConfirmed(spliceTx.txid)
bob2blockchain.expectWatchFundingConfirmed(spliceTx.txid)
// Alice sees the splice transaction confirm.
alice ! WatchFundingConfirmedTriggered(BlockHeight(1105), 37, spliceTx)
alice2blockchain.expectWatchFundingSpent(spliceTx.txid)
assert(alice2bob.expectMsgType[SpliceLocked].fundingTxId == spliceTx.txid)
alice2bob.forward(bob)
// Bob sees the splice transaction confirm and receives Alice's announcement_signatures.
bob ! WatchFundingConfirmedTriggered(BlockHeight(1105), 37, spliceTx)
bob2blockchain.expectWatchFundingSpent(spliceTx.txid)
assert(bob2alice.expectMsgType[SpliceLocked].fundingTxId == spliceTx.txid)
bob2alice.forward(alice)
alice2bob.expectMsgType[AnnouncementSignatures]
alice2bob.forward(bob)
bob2alice.expectMsgType[AnnouncementSignatures] // Alice doesn't receive Bob's signatures.
awaitAssert(bob.stateData.asInstanceOf[DATA_NORMAL].lastAnnouncement_opt.nonEmpty)
val spliceAnn = bob.stateData.asInstanceOf[DATA_NORMAL].lastAnnouncement_opt.get
assert(spliceAnn.shortChannelId != shortChannelId)
assert(bobListener.expectMsgType[ShortChannelIdAssigned].announcement_opt.contains(spliceAnn))
assert(alice.stateData.asInstanceOf[DATA_NORMAL].lastAnnouncement_opt.isEmpty)
aliceListener.expectNoMessage(100 millis)
// Alice restarts.
disconnect(f)
val aliceDataBeforeRestart = alice.stateData.asInstanceOf[DATA_NORMAL]
alice.setState(WAIT_FOR_INIT_INTERNAL, Nothing)
alice ! INPUT_RESTORED(aliceDataBeforeRestart)
alice2blockchain.expectMsgType[SetChannelId]
alice2blockchain.expectWatchFundingSpent(spliceTx.txid)
assert(aliceListener.expectMsgType[ShortChannelIdAssigned].announcement_opt.isEmpty)
awaitAssert(assert(alice.stateName == OFFLINE))
// Alice and Bob reconnect.
reconnect(f)
assert(alice2bob.expectMsgType[SpliceLocked].fundingTxId == spliceTx.txid)
alice2bob.forward(bob)
alice2bob.expectNoMessage(100 millis)
assert(bob2alice.expectMsgType[SpliceLocked].fundingTxId == spliceTx.txid)
bob2alice.forward(alice)
assert(bob2alice.expectMsgType[AnnouncementSignatures].shortChannelId == spliceAnn.shortChannelId)
bob2alice.forward(alice)
bob2alice.expectNoMessage(100 millis)
assert(aliceListener.expectMsgType[ShortChannelIdAssigned].announcement_opt.contains(spliceAnn))
awaitAssert(assert(alice.stateData.asInstanceOf[DATA_NORMAL].lastAnnouncement_opt.contains(spliceAnn)))
awaitAssert(assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.all.size == 1))
awaitAssert(assert(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.all.size == 1))
}
test("recv CMD_ADD_HTLC with multiple commitments") { f =>
import f._
initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat)))

View file

@ -3598,9 +3598,8 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
import f._
// For zero-conf channels we don't have a real short_channel_id when going to the NORMAL state.
val aliceState = alice.stateData.asInstanceOf[DATA_NORMAL]
assert(aliceState.shortIds.real_opt.isEmpty)
assert(alice2bob.expectMsgType[ChannelUpdate].shortChannelId == bob.stateData.asInstanceOf[DATA_NORMAL].shortIds.localAlias)
assert(bob2alice.expectMsgType[ChannelUpdate].shortChannelId == alice.stateData.asInstanceOf[DATA_NORMAL].shortIds.localAlias)
assert(alice2bob.expectMsgType[ChannelUpdate].shortChannelId == bob.stateData.asInstanceOf[DATA_NORMAL].aliases.localAlias)
assert(bob2alice.expectMsgType[ChannelUpdate].shortChannelId == alice.stateData.asInstanceOf[DATA_NORMAL].aliases.localAlias)
// When the funding transaction confirms, we obtain a real short_channel_id.
val fundingTx = aliceState.commitments.latest.localFundingStatus.signedTx_opt.get
val (blockHeight, txIndex) = (BlockHeight(400_000), 42)
@ -3609,7 +3608,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
val annSigsA = alice2bob.expectMsgType[AnnouncementSignatures]
assert(annSigsA.shortChannelId == realShortChannelId)
// Alice updates her internal state wih the real scid.
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].shortIds.real_opt.contains(realShortChannelId))
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.shortChannelId_opt.contains(realShortChannelId))
alice2bob.forward(bob, annSigsA)
alice2bob.expectNoMessage(100 millis)
// Bob doesn't know that the funding transaction is confirmed, so he doesn't send his announcement_signatures yet.
@ -3618,11 +3617,11 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
val annSigsB = bob2alice.expectMsgType[AnnouncementSignatures]
assert(annSigsB.shortChannelId == realShortChannelId)
bob2alice.forward(alice, annSigsB)
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].channelAnnouncement.nonEmpty)
awaitCond(bob.stateData.asInstanceOf[DATA_NORMAL].channelAnnouncement.nonEmpty)
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].lastAnnouncement_opt.map(_.shortChannelId).contains(realShortChannelId))
awaitCond(bob.stateData.asInstanceOf[DATA_NORMAL].lastAnnouncement_opt.map(_.shortChannelId).contains(realShortChannelId))
// We emit a new local channel update containing the same channel_update, but with the new real scid.
val lcu = channelUpdateListener.expectMsgType[LocalChannelUpdate]
assert(lcu.shortIds.real_opt.contains(realShortChannelId))
assert(lcu.announcement_opt.map(_.shortChannelId).contains(realShortChannelId))
assert(alice.stateData.asInstanceOf[DATA_NORMAL].channelUpdate.shortChannelId == realShortChannelId)
}
@ -3633,12 +3632,11 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
alice.underlying.system.eventStream.subscribe(listener.ref, classOf[TransactionConfirmed])
// zero-conf channel: the funding tx isn't confirmed
val aliceState = alice.stateData.asInstanceOf[DATA_NORMAL]
assert(aliceState.shortIds.real_opt.isEmpty)
val fundingTx = aliceState.commitments.latest.localFundingStatus.signedTx_opt.get
alice ! WatchFundingConfirmedTriggered(BlockHeight(400_000), 42, fundingTx)
val realShortChannelId = RealShortChannelId(BlockHeight(400_000), 42, 0)
// update data with real short channel id
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].shortIds.real_opt.contains(realShortChannelId))
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.shortChannelId_opt.contains(realShortChannelId))
// private channel: we'll use the remote alias in the channel_update we sent to our peer, there is no change so we don't create a new channel_update
alice2bob.expectNoMessage(100 millis)
channelUpdateListener.expectNoMessage(100 millis)
@ -3649,7 +3647,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
test("recv AnnouncementSignatures", Tag(ChannelStateTestsTags.ChannelsPublic), Tag(ChannelStateTestsTags.DoNotInterceptGossip)) { f =>
import f._
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
val realShortChannelId = initialState.shortIds.real_opt.get
val realShortChannelId = initialState.commitments.latest.shortChannelId_opt.get
// Alice and Bob exchange announcement_signatures.
val annSigsA = alice2bob.expectMsgType[AnnouncementSignatures]
assert(annSigsA.shortChannelId == realShortChannelId)
@ -3661,20 +3659,22 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
val bobFundingKey = initialState.commitments.latest.remoteFundingPubKey
val channelAnn = Announcements.makeChannelAnnouncement(Alice.nodeParams.chainHash, annSigsA.shortChannelId, Alice.nodeParams.nodeId, Bob.nodeParams.nodeId, aliceFundingKey, bobFundingKey, annSigsA.nodeSignature, annSigsB.nodeSignature, annSigsA.bitcoinSignature, annSigsB.bitcoinSignature)
// actual test starts here
val listener = TestProbe()
alice.underlying.system.eventStream.subscribe(listener.ref, classOf[ShortChannelIdAssigned])
bob2alice.forward(alice, annSigsB)
awaitAssert {
val normal = alice.stateData.asInstanceOf[DATA_NORMAL]
assert(normal.shortIds.real_opt.contains(realShortChannelId))
assert(normal.channelAnnouncement.contains(channelAnn))
assert(normal.lastAnnouncement_opt.contains(channelAnn))
assert(normal.channelUpdate.shortChannelId == realShortChannelId)
}
// we use the real scid instead of remote alias as soon as the channel is announced
assert(listener.expectMsgType[ShortChannelIdAssigned].announcement_opt.contains(channelAnn))
// We use the real scid in channel updates instead of the remote alias as soon as the channel is announced.
val lcu = channelUpdateListener.expectMsgType[LocalChannelUpdate]
assert(lcu.channelUpdate.shortChannelId == realShortChannelId)
assert(lcu.channelAnnouncement_opt.contains(channelAnn))
// we don't send directly the channel_update to our peer, public announcements are handled by the router
assert(lcu.announcement_opt.map(_.announcement).contains(channelAnn))
// We don't send directly the channel_update to our peer, public announcements are handled by the router.
alice2bob.expectNoMessage(100 millis)
// we ignore redundant announcement_signatures
// We ignore redundant announcement_signatures.
bob2alice.forward(alice, annSigsB)
alice2bob.expectNoMessage(100 millis)
channelUpdateListener.expectNoMessage(100 millis)

View file

@ -813,6 +813,7 @@ class OfflineStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
}
}
// TODO
test("re-send channel_update at reconnection for unannounced channels") { f =>
import f._

View file

@ -29,7 +29,7 @@ import fr.acinq.eclair.db.sqlite.SqliteChannelsDb
import fr.acinq.eclair.db.sqlite.SqliteUtils.ExtendedResultSet._
import fr.acinq.eclair.wire.internal.channel.ChannelCodecs.channelDataCodec
import fr.acinq.eclair.wire.internal.channel.ChannelCodecsSpec
import fr.acinq.eclair.{CltvExpiry, RealShortChannelId, TestDatabases, randomBytes32, randomKey}
import fr.acinq.eclair.{Alias, CltvExpiry, TestDatabases, randomBytes32, randomKey, randomLong}
import org.scalatest.funsuite.AnyFunSuite
import scodec.bits.ByteVector
@ -60,7 +60,7 @@ class ChannelsDbSpec extends AnyFunSuite {
val channel1 = ChannelCodecsSpec.normal
val channel2a = ChannelCodecsSpec.normal.modify(_.commitments.params.channelId).setTo(randomBytes32())
val channel2b = channel2a.modify(_.shortIds.real_opt).setTo(Some(RealShortChannelId(189371)))
val channel2b = channel2a.modify(_.aliases.remoteAlias_opt).setTo(Some(Alias(randomLong())))
val commitNumber = 42
val paymentHash1 = ByteVector32.Zeroes

View file

@ -231,6 +231,7 @@ class MessageIntegrationSpec extends IntegrationSpec {
eventListener.expectNoMessage()
}
// TODO: fails...
test("open channels") {
val probe = TestProbe()

View file

@ -187,7 +187,7 @@ class PaymentIntegrationSpec extends IntegrationSpec {
// we then forge a new channel_update for B-C...
val channelUpdateBC = Announcements.makeChannelUpdate(Block.RegtestGenesisBlock.hash, nodes("B").nodeParams.privateKey, nodes("C").nodeParams.nodeId, shortIdBC, nodes("B").nodeParams.channelConf.expiryDelta + 1, nodes("C").nodeParams.channelConf.htlcMinimum, nodes("B").nodeParams.relayParams.publicChannelFees.feeBase, nodes("B").nodeParams.relayParams.publicChannelFees.feeProportionalMillionths, 500000000 msat)
// ...and notify B's relayer
nodes("B").system.eventStream.publish(LocalChannelUpdate(system.deadLetters, normalBC.channelId, normalBC.shortIds, normalBC.commitments.remoteNodeId, normalBC.channelAnnouncement, channelUpdateBC, normalBC.commitments))
nodes("B").system.eventStream.publish(LocalChannelUpdate(system.deadLetters, normalBC.channelId, normalBC.aliases, normalBC.commitments.remoteNodeId, normalBC.lastAnnouncedCommitment_opt, channelUpdateBC, normalBC.commitments))
// we retrieve a payment hash from D
val amountMsat = 4200000.msat
sender.send(nodes("D").paymentHandler, ReceiveStandardPayment(sender.ref, Some(amountMsat), Left("1 coffee")))

View file

@ -40,8 +40,8 @@ class TwoNodesIntegrationSpec extends FixtureSpec with IntegrationPatience {
connect(alice, bob)
val channelId = openChannel(alice, bob, 100_000 sat).channelId
confirmChannel(alice, bob, channelId, BlockHeight(420_000), 21)
assert(getChannelData(alice, channelId).asInstanceOf[DATA_NORMAL].shortIds.real_opt.nonEmpty)
assert(getChannelData(bob, channelId).asInstanceOf[DATA_NORMAL].shortIds.real_opt.nonEmpty)
assert(getChannelData(alice, channelId).asInstanceOf[DATA_NORMAL].commitments.latest.localFundingStatus.isInstanceOf[LocalFundingStatus.ConfirmedFundingTx])
assert(getChannelData(bob, channelId).asInstanceOf[DATA_NORMAL].commitments.latest.localFundingStatus.isInstanceOf[LocalFundingStatus.ConfirmedFundingTx])
}
test("open multiple channels alice-bob") { f =>
@ -64,8 +64,8 @@ class TwoNodesIntegrationSpec extends FixtureSpec with IntegrationPatience {
connect(alice, bob)
val channelId = openChannel(alice, bob, 100_000 sat).channelId
eventually {
assert(getChannelData(alice, channelId).asInstanceOf[DATA_NORMAL].shortIds.real_opt.nonEmpty)
assert(getChannelData(bob, channelId).asInstanceOf[DATA_NORMAL].shortIds.real_opt.nonEmpty)
assert(getChannelData(alice, channelId).asInstanceOf[DATA_NORMAL].commitments.latest.localFundingStatus.isInstanceOf[LocalFundingStatus.ConfirmedFundingTx])
assert(getChannelData(bob, channelId).asInstanceOf[DATA_NORMAL].commitments.latest.localFundingStatus.isInstanceOf[LocalFundingStatus.ConfirmedFundingTx])
}
}

View file

@ -1,46 +1,48 @@
package fr.acinq.eclair.integration.basic.channel
import com.softwaremill.quicklens.ModifyPimp
import fr.acinq.bitcoin.scalacompat.{ByteVector32, SatoshiLong}
import fr.acinq.eclair.channel.states.ChannelStateTestsTags
import fr.acinq.eclair.channel.{DATA_NORMAL, RES_SPLICE}
import fr.acinq.eclair.integration.basic.ThreeNodesIntegrationSpec
import fr.acinq.eclair.integration.basic.fixtures.MinimalNodeFixture.{getChannelData, getPeerChannels, spliceIn}
import fr.acinq.eclair.channel.{DATA_NORMAL, LocalFundingStatus, RES_SPLICE}
import fr.acinq.eclair.integration.basic.fixtures.MinimalNodeFixture.{getChannelData, getPeerChannels, sendPayment, spliceIn}
import fr.acinq.eclair.integration.basic.fixtures.composite.ThreeNodesFixture
import fr.acinq.eclair.{FeatureSupport, Features}
import fr.acinq.eclair.testutils.FixtureSpec
import fr.acinq.eclair.{FeatureSupport, Features, MilliSatoshiLong}
import org.scalatest.Inside.inside
import org.scalatest.concurrent.IntegrationPatience
import org.scalatest.{Tag, TestData}
import scodec.bits.HexStringSyntax
/**
* These test checks the gossip sent between nodes by the Router
*/
class GossipIntegrationSpec extends ThreeNodesIntegrationSpec {
class GossipIntegrationSpec extends FixtureSpec with IntegrationPatience {
type FixtureParam = ThreeNodesFixture
import fr.acinq.eclair.integration.basic.fixtures.MinimalNodeFixture.{connect, getRouterData, knownFundingTxs, nodeParamsFor, openChannel, watcherAutopilot}
override def createFixture(testData: TestData): FixtureParam = {
// seeds have been chosen so that node ids start with 02aaaa for alice, 02bbbb for bob, etc.
val aliceParams = nodeParamsFor("alice", ByteVector32(hex"b4acd47335b25ab7b84b8c020997b12018592bb4631b868762154d77fa8b93a3"))
val aliceParams1 = aliceParams.copy(
features = aliceParams.features.add(Features.SplicePrototype, FeatureSupport.Optional)
)
.modify(_.features).using(_.add(Features.SplicePrototype, FeatureSupport.Optional))
val bobParams = nodeParamsFor("bob", ByteVector32(hex"7620226fec887b0b2ebe76492e5a3fd3eb0e47cd3773263f6a81b59a704dc492"))
val bobParams1 = bobParams.copy(
features = bobParams.features.add(Features.SplicePrototype, FeatureSupport.Optional)
)
.modify(_.features).using(_.add(Features.SplicePrototype, FeatureSupport.Optional))
val carolParams = nodeParamsFor("carol", ByteVector32(hex"ebd5a5d3abfb3ef73731eb3418d918f247445183180522674666db98a66411cc"))
ThreeNodesFixture(aliceParams1, bobParams1, carolParams, testData.name)
ThreeNodesFixture(aliceParams, bobParams, carolParams, testData.name)
}
override def cleanupFixture(fixture: ThreeNodesFixture): Unit = {
fixture.cleanup()
}
test("send gossip when alice->bob channel is spliced", Tag(ChannelStateTestsTags.DualFunding)) { f =>
test("propagate channel_announcement and channel_update after splicing channel", Tag(ChannelStateTestsTags.DualFunding)) { f =>
import f._
connect(alice, bob)
connect(bob, carol)
// we put watchers on auto pilot to confirm funding txs
// We put the watchers on auto pilot to confirm funding txs.
alice.watcher.setAutoPilot(watcherAutopilot(knownFundingTxs(alice, bob, carol)))
bob.watcher.setAutoPilot(watcherAutopilot(knownFundingTxs(alice, bob, carol)))
carol.watcher.setAutoPilot(watcherAutopilot(knownFundingTxs(alice, bob, carol)))
@ -50,33 +52,76 @@ class GossipIntegrationSpec extends ThreeNodesIntegrationSpec {
val channels = getPeerChannels(alice, bob.nodeId) ++ getPeerChannels(bob, carol.nodeId)
assert(channels.map(_.data.channelId).toSet == Set(channelId_ab, channelId_bc))
// channels confirm deeply
eventually {
assert(getChannelData(alice, channelId_ab).asInstanceOf[DATA_NORMAL].shortIds.real_opt.nonEmpty)
assert(getChannelData(bob, channelId_bc).asInstanceOf[DATA_NORMAL].shortIds.real_opt.nonEmpty)
}
val scid_ab = getChannelData(alice, channelId_ab).asInstanceOf[DATA_NORMAL].shortIds.real_opt.get
val scid_bc = getChannelData(bob, channelId_bc).asInstanceOf[DATA_NORMAL].shortIds.real_opt.get
// splice in to increase capacity of alice->bob channel
spliceIn(alice, channelId_ab, 100_000 sat, None).asInstanceOf[RES_SPLICE].fundingTxId
// verify that the new capacity and scid are correctly propagated
eventually {
val channelData_alice1 = getChannelData(alice, channelId_ab).asInstanceOf[DATA_NORMAL]
val channelData_bob1 = getChannelData(bob, channelId_ab).asInstanceOf[DATA_NORMAL]
assert(channelData_alice1.commitments.latest.capacity == 200_000.sat)
assert(channelData_bob1.commitments.latest.capacity == 200_000.sat)
assert(channelData_alice1.shortIds.real_opt.get == channelData_bob1.shortIds.real_opt.get)
val scid_ab1 = getChannelData(alice, channelId_ab).asInstanceOf[DATA_NORMAL].shortIds.real_opt.get
val ann_splice = getRouterData(alice).channels(scid_ab1)
assert(ann_splice.capacity == 200_000.sat)
assert(getRouterData(bob).channels(scid_ab1) == ann_splice)
// TODO: after PR 2941, the slice ChannelAnnouncement will have a new scid and not be ignore by carol
assert(getRouterData(carol).spentChannels.exists(_._2 == ann_splice.shortChannelId))
// assert(scid_ab != scid_ab1)
// assert(getRouterData(carol).channels(scid_ab1).capacity == 200_000.sat)
assert(getChannelData(alice, channelId_ab).asInstanceOf[DATA_NORMAL].commitments.latest.localFundingStatus.isInstanceOf[LocalFundingStatus.ConfirmedFundingTx])
assert(getChannelData(bob, channelId_bc).asInstanceOf[DATA_NORMAL].commitments.latest.localFundingStatus.isInstanceOf[LocalFundingStatus.ConfirmedFundingTx])
}
// We splice in to increase the capacity of the alice->bob channel.
val scid_ab = getChannelData(alice, channelId_ab).asInstanceOf[DATA_NORMAL].commitments.latest.shortChannelId_opt.get
val spliceTxId = spliceIn(alice, channelId_ab, 100_000 sat, None).asInstanceOf[RES_SPLICE].fundingTxId
// The announcement for the splice transaction and the corresponding channel updates are broadcast.
eventually {
val channelData_alice = getChannelData(alice, channelId_ab).asInstanceOf[DATA_NORMAL]
val channelData_bob = getChannelData(bob, channelId_ab).asInstanceOf[DATA_NORMAL]
// The splice increased the channel capacity.
assert(channelData_alice.commitments.latest.fundingTxId == spliceTxId)
assert(channelData_alice.commitments.latest.capacity == 200_000.sat)
assert(channelData_bob.commitments.latest.capacity == 200_000.sat)
// The splice transaction changed the short_channel_id.
val splice_scid_ab = channelData_alice.commitments.latest.shortChannelId_opt.get
assert(splice_scid_ab != scid_ab)
assert(channelData_bob.commitments.latest.shortChannelId_opt.contains(splice_scid_ab))
// Alice creates a channel_announcement for the splice transaction and updates the graph.
val spliceAnn = inside(getRouterData(alice)) { routerData =>
assert(routerData.channels.contains(splice_scid_ab))
val channel_ab = routerData.channels(splice_scid_ab)
assert(channel_ab.capacity == 200_000.sat)
assert(channel_ab.update_1_opt.nonEmpty && channel_ab.update_2_opt.nonEmpty)
assert(channel_ab.meta_opt.nonEmpty)
Seq(channel_ab.update_1_opt, channel_ab.update_2_opt).flatten.foreach(u => assert(u.shortChannelId == splice_scid_ab))
val edge_ab_opt = routerData.graphWithBalances.graph.getEdgesBetween(alice.nodeId, bob.nodeId).find(_.desc.shortChannelId == splice_scid_ab)
assert(edge_ab_opt.nonEmpty)
assert(edge_ab_opt.get.capacity == 200_000.sat)
assert(edge_ab_opt.get.balance_opt.get > 100_000_000.msat)
channel_ab.ann
}
// Bob also creates a channel_announcement for the splice transaction and updates the graph.
inside(getRouterData(bob)) { routerData =>
assert(routerData.channels.get(splice_scid_ab).map(_.ann).contains(spliceAnn))
routerData.channels.get(splice_scid_ab).foreach(c => {
assert(c.capacity == 200_000.sat)
assert(c.fundingTxId == spliceTxId)
assert(c.update_1_opt.nonEmpty && c.update_2_opt.nonEmpty)
assert(c.meta_opt.nonEmpty)
Seq(c.update_1_opt, c.update_2_opt).flatten.foreach(u => assert(u.shortChannelId == splice_scid_ab))
})
val edge_ab_opt = routerData.graphWithBalances.graph.getEdgesBetween(alice.nodeId, bob.nodeId).find(_.desc.shortChannelId == splice_scid_ab)
assert(edge_ab_opt.get.capacity == 200_000.sat)
assert(edge_ab_opt.get.balance_opt.get > 100_000_000.msat)
}
// The channel_announcement for the splice propagates to Carol.
inside(getRouterData(carol)) { routerData =>
assert(routerData.channels.get(splice_scid_ab).map(_.ann).contains(spliceAnn))
routerData.channels.get(splice_scid_ab).foreach(c => {
assert(c.capacity == 200_000.sat)
assert(c.fundingTxId == spliceTxId)
assert(c.update_1_opt.nonEmpty && c.update_2_opt.nonEmpty)
assert(c.meta_opt.isEmpty)
Seq(c.update_1_opt, c.update_2_opt).flatten.foreach(u => assert(u.shortChannelId == splice_scid_ab))
})
val edge_ba_opt = routerData.graphWithBalances.graph.getEdgesBetween(bob.nodeId, alice.nodeId).find(_.desc.shortChannelId == splice_scid_ab)
assert(edge_ba_opt.get.capacity == 200_000.sat)
assert(edge_ba_opt.get.balance_opt.isEmpty)
}
}
// Payments can be made using the spliced channel.
assert(sendPayment(alice, carol, 75_000_000 msat).isRight)
assert(sendPayment(carol, alice, 60_000_000 msat).isRight)
}
}

View file

@ -213,8 +213,8 @@ object MinimalNodeFixture extends Assertions with Eventually with IntegrationPat
val data1After = getChannelData(node1, channelId).asInstanceOf[DATA_NORMAL]
val data2After = getChannelData(node2, channelId).asInstanceOf[DATA_NORMAL]
val realScid1 = data1After.shortIds.real_opt.get
val realScid2 = data2After.shortIds.real_opt.get
val realScid1 = data1After.commitments.latest.shortChannelId_opt.get
val realScid2 = data2After.commitments.latest.shortChannelId_opt.get
assert(realScid1 == realScid2)
realScid1
}

View file

@ -129,7 +129,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience {
assert(getChannelState(bob, channelId) == NORMAL)
assert(getChannelState(carol, channelId) == NORMAL)
// Carol must have received Bob's alias to create usable blinded routes to herself.
assert(getRouterData(carol).privateChannels.values.forall(_.shortIds.remoteAlias_opt.nonEmpty))
assert(getRouterData(carol).privateChannels.values.forall(_.aliases.remoteAlias_opt.nonEmpty))
}
}
@ -554,7 +554,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience {
val blindedRoute = buildRoute(randomKey(), Seq(IntermediateNode(bob.nodeId), IntermediateNode(carol.nodeId)), Recipient(carol.nodeId, Some(pathId))).route
val offer = Offer(None, Some("test"), recipientKey.publicKey, Features.empty, carol.nodeParams.chainHash, additionalTlvs = Set(OfferPaths(Seq(blindedRoute))))
val scid_bc = getPeerChannels(bob, carol.nodeId).head.data.asInstanceOf[DATA_NORMAL].shortIds.real_opt.get
val scid_bc = getPeerChannels(bob, carol.nodeId).head.data.asInstanceOf[DATA_NORMAL].commitments.latest.shortChannelId_opt.get
val compactBlindedRoute = buildRoute(randomKey(), Seq(IntermediateNode(bob.nodeId, EncodedNodeId(bob.nodeId), Some(scid_bc)), IntermediateNode(carol.nodeId, EncodedNodeId(carol.nodeId), Some(ShortChannelId.toSelf))), Recipient(carol.nodeId, Some(pathId))).route
val compactOffer = Offer(None, Some("test"), recipientKey.publicKey, Features.empty, carol.nodeParams.chainHash, additionalTlvs = Set(OfferPaths(Seq(compactBlindedRoute))))
assert(compactOffer.toString.length < offer.toString.length)

View file

@ -6,7 +6,7 @@ import com.softwaremill.quicklens._
import fr.acinq.bitcoin.scalacompat.{ByteVector32, SatoshiLong}
import fr.acinq.eclair.FeatureSupport.{Mandatory, Optional}
import fr.acinq.eclair.Features.{ScidAlias, ZeroConf}
import fr.acinq.eclair.channel.DATA_NORMAL
import fr.acinq.eclair.channel.{DATA_NORMAL, LocalFundingStatus}
import fr.acinq.eclair.crypto.Sphinx
import fr.acinq.eclair.integration.basic.fixtures.composite.ThreeNodesFixture
import fr.acinq.eclair.payment.Bolt11Invoice.ExtraHop
@ -72,7 +72,7 @@ class ZeroConfAliasIntegrationSpec extends FixtureSpec with IntegrationPatience
if (useHint) {
val Some(carolHint) = getRouterData(carol).privateChannels.values.head.toIncomingExtraHop
// due to how node ids are built, bob < carol so carol is always the node 2
val bobAlias = getRouterData(bob).privateChannels.values.find(_.nodeId2 == carol.nodeParams.nodeId).value.shortIds.localAlias
val bobAlias = getRouterData(bob).privateChannels.values.find(_.nodeId2 == carol.nodeParams.nodeId).value.aliases.localAlias
// the hint is always using the alias
assert(carolHint.shortChannelId == bobAlias)
List(carolHint.modify(_.shortChannelId).setToIfDefined(overrideHintScid_opt))
@ -126,9 +126,9 @@ class ZeroConfAliasIntegrationSpec extends FixtureSpec with IntegrationPatience
assert(getChannelData(bob, channelId_bc).asInstanceOf[DATA_NORMAL].commitments.params.channelFeatures.features.contains(ScidAlias) == bcScidAlias)
assert(getChannelData(bob, channelId_bc).asInstanceOf[DATA_NORMAL].commitments.params.channelFlags.announceChannel == bcPublic)
if (confirm) {
assert(getChannelData(bob, channelId_bc).asInstanceOf[DATA_NORMAL].shortIds.real_opt.nonEmpty)
assert(getChannelData(bob, channelId_bc).asInstanceOf[DATA_NORMAL].commitments.latest.localFundingStatus.isInstanceOf[LocalFundingStatus.ConfirmedFundingTx])
} else {
assert(getChannelData(bob, channelId_bc).asInstanceOf[DATA_NORMAL].shortIds.real_opt.isEmpty)
assert(getChannelData(bob, channelId_bc).asInstanceOf[DATA_NORMAL].commitments.latest.shortChannelId_opt.isEmpty)
}
}
@ -153,14 +153,14 @@ class ZeroConfAliasIntegrationSpec extends FixtureSpec with IntegrationPatience
paymentWithRealScidHint_opt.foreach { paymentWithRealScidHint =>
eventually {
sendPaymentAliceToCarol(f, paymentWithRealScidHint, useHint = true, overrideHintScid_opt = Some(getChannelData(bob, channelId_bc).asInstanceOf[DATA_NORMAL].shortIds.real_opt.value))
sendPaymentAliceToCarol(f, paymentWithRealScidHint, useHint = true, overrideHintScid_opt = Some(getChannelData(bob, channelId_bc).asInstanceOf[DATA_NORMAL].commitments.latest.shortChannelId_opt.value))
}
}
eventually {
if (confirm) {
val scidsBob = getChannelData(bob, channelId_bc).asInstanceOf[DATA_NORMAL].shortIds
val scid_bc = if (bcPublic) scidsBob.real_opt.get else scidsBob.localAlias
val channelDataBob = getChannelData(bob, channelId_bc).asInstanceOf[DATA_NORMAL]
val scid_bc = if (bcPublic) channelDataBob.commitments.latest.shortChannelId_opt.value else channelDataBob.aliases.localAlias
createSelfRouteCarol(f, scid_bc)
}
}
@ -174,7 +174,7 @@ class ZeroConfAliasIntegrationSpec extends FixtureSpec with IntegrationPatience
bcScidAlias = false,
paymentWithoutHint = Left(Left(RouteNotFound)), // alice can't find a route to carol because bob-carol isn't announced
paymentWithHint_opt = Some(Right(Ok)), // with a routing hint the payment works (and it will use the alias, even if the feature isn't enabled)
paymentWithRealScidHint_opt = Some(Right(Ok)) // if alice uses the real scid instead of the bob-carol alias, it still works
paymentWithRealScidHint_opt = Some(Left(Right(UnknownNextPeer()))) // if alice uses the real scid instead of the bob-carol alias, it doesn't work: peers must use option_scid_alias
)
}
@ -263,12 +263,12 @@ class ZeroConfAliasIntegrationSpec extends FixtureSpec with IntegrationPatience
eventually {
assert(getChannelData(bob, channelId_bc).asInstanceOf[DATA_NORMAL].commitments.params.channelFeatures.features.contains(ZeroConf))
assert(getChannelData(bob, channelId_bc).asInstanceOf[DATA_NORMAL].commitments.params.channelFeatures.features.contains(ScidAlias))
assert(getChannelData(bob, channelId_bc).asInstanceOf[DATA_NORMAL].shortIds.real_opt.isEmpty)
assert(getChannelData(bob, channelId_bc).asInstanceOf[DATA_NORMAL].commitments.latest.shortChannelId_opt.isEmpty)
assert(getRouterData(bob).privateChannels.values.exists(_.nodeId2 == carol.nodeParams.nodeId))
}
val Some(carolHint) = getRouterData(carol).privateChannels.values.head.toIncomingExtraHop
val bobAlias = getRouterData(bob).privateChannels.values.find(_.nodeId2 == carol.nodeParams.nodeId).value.shortIds.localAlias
val bobAlias = getRouterData(bob).privateChannels.values.find(_.nodeId2 == carol.nodeParams.nodeId).value.aliases.localAlias
assert(carolHint.shortChannelId == bobAlias)
// We make sure Bob won't have enough liquidity to relay another payment.

View file

@ -82,7 +82,7 @@ class OpenChannelInterceptorSpec extends ScalaTestWithActorTestKit(ConfigFactory
case class FixtureParam(openChannelInterceptor: ActorRef[OpenChannelInterceptor.Command], peer: TestProbe[Any], pluginInterceptor: TestProbe[InterceptOpenChannelCommand], pendingChannelsRateLimiter: TestProbe[PendingChannelsRateLimiter.Command], peerConnection: TestProbe[Any], eventListener: TestProbe[ChannelAborted], wallet: DummyOnChainWallet)
private def commitments(isOpener: Boolean = false): Commitments = {
val commitments = CommitmentsSpec.makeCommitments(500_000 msat, 400_000 msat, TestConstants.Alice.nodeParams.nodeId, remoteNodeId, announceChannel = false)
val commitments = CommitmentsSpec.makeCommitments(500_000 msat, 400_000 msat, TestConstants.Alice.nodeParams.nodeId, remoteNodeId, announcement_opt = None)
commitments.copy(params = commitments.params.copy(localParams = commitments.params.localParams.copy(isChannelOpener = isOpener, paysCommitTxFees = isOpener)))
}

View file

@ -782,7 +782,7 @@ class PeerSpec extends FixtureSpec {
}
// Our channel closes, so we stop storing backups for that peer.
peer ! LocalChannelDown(ActorRef.noSender, channel.channelId, channel.shortIds, channel.remoteNodeId)
peer ! LocalChannelDown(ActorRef.noSender, channel.channelId, channel.lastAnnouncement_opt.map(_.shortChannelId).toSeq, channel.aliases, channel.remoteNodeId)
peerConnection3.send(peer, PeerStorageStore(hex"2222"))
assert(!peer.isTimerActive("peer-storage-write"))
assert(nodeParams.db.peers.getStorage(remoteNodeId).contains(hex"1111"))

View file

@ -20,14 +20,14 @@ import akka.actor.testkit.typed.scaladsl.{ScalaTestWithActorTestKit, TestProbe}
import akka.actor.typed.eventstream.EventStream.Publish
import com.typesafe.config.ConfigFactory
import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
import fr.acinq.bitcoin.scalacompat.{ByteVector32, SatoshiLong, Transaction, TxId, TxOut}
import fr.acinq.bitcoin.scalacompat.{Block, ByteVector32, SatoshiLong, Transaction, TxId, TxOut}
import fr.acinq.eclair.channel._
import fr.acinq.eclair.io.PendingChannelsRateLimiter.filterPendingChannels
import fr.acinq.eclair.router.Router
import fr.acinq.eclair.router.Router.{GetNode, PublicNode, UnknownNode}
import fr.acinq.eclair.router.{Announcements, Router}
import fr.acinq.eclair.transactions.Transactions.{ClosingTx, InputInfo}
import fr.acinq.eclair.wire.protocol._
import fr.acinq.eclair.{BlockHeight, Features, MilliSatoshiLong, NodeParams, ShortChannelId, TestConstants, TimestampSecondLong, randomBytes32, randomBytes64, randomKey}
import fr.acinq.eclair.{BlockHeight, Features, MilliSatoshiLong, NodeParams, RealShortChannelId, ShortChannelId, TestConstants, TimestampSecondLong, randomBytes32, randomBytes64, randomKey}
import org.scalatest.Outcome
import org.scalatest.funsuite.FixtureAnyFunSuiteLike
import scodec.bits.{ByteVector, HexStringSyntax}
@ -75,34 +75,34 @@ class PendingChannelsRateLimiterSpec extends ScalaTestWithActorTestKit(ConfigFac
val closingTx = ClosingTx(InputInfo(tx.txIn.head.outPoint, TxOut(10_000 sat, Nil), Nil), tx, None)
val channelsOnWhitelistAtLimit: Seq[PersistentChannelData] = Seq(
DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments(peerOnWhitelistAtLimit, randomBytes32()), BlockHeight(0), None, Left(FundingCreated(randomBytes32(), TxId(ByteVector32.Zeroes), 3, randomBytes64()))),
DATA_WAIT_FOR_CHANNEL_READY(commitments(peerOnWhitelistAtLimit, randomBytes32()), ShortIds(None, ShortChannelId.generateLocalAlias(), None)),
DATA_WAIT_FOR_CHANNEL_READY(commitments(peerOnWhitelistAtLimit, randomBytes32()), ShortIdAliases(ShortChannelId.generateLocalAlias(), None)),
)
val channelsAtLimit1 = Seq(
DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments(peerAtLimit1, channelIdAtLimit1), BlockHeight(0), None, Left(FundingCreated(channelIdAtLimit1, TxId(ByteVector32.Zeroes), 3, randomBytes64()))),
DATA_WAIT_FOR_CHANNEL_READY(commitments(peerAtLimit1, randomBytes32()), ShortIds(None, ShortChannelId.generateLocalAlias(), None)),
DATA_WAIT_FOR_CHANNEL_READY(commitments(peerAtLimit1, randomBytes32()), ShortIdAliases(ShortChannelId.generateLocalAlias(), None)),
)
val channelsAtLimit2 = Seq(
DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments(peerAtLimit2, channelIdAtLimit2), 0 msat, 0 msat, BlockHeight(0), BlockHeight(0), DualFundingStatus.WaitingForConfirmations, None),
DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments(peerAtLimit2, randomBytes32()), ShortIds(None, ShortChannelId.generateLocalAlias(), None)),
DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments(peerAtLimit2, randomBytes32()), ShortIdAliases(ShortChannelId.generateLocalAlias(), None)),
)
val channelsBelowLimit1 = Seq(
DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments(peerBelowLimit1, channelIdBelowLimit1), BlockHeight(0), None, Left(FundingCreated(channelIdBelowLimit1, TxId(ByteVector32.Zeroes), 3, randomBytes64()))),
)
val channelsBelowLimit2 = Seq(
DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments(peerBelowLimit2, channelIdBelowLimit2), ShortIds(None, ShortChannelId.generateLocalAlias(), None)),
DATA_NORMAL(commitments(peerBelowLimit2, randomBytes32()), ShortIds(None, ShortChannelId.generateLocalAlias(), None), None, null, None, None, None, SpliceStatus.NoSplice),
DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments(peerBelowLimit2, channelIdBelowLimit2), ShortIdAliases(ShortChannelId.generateLocalAlias(), None)),
DATA_NORMAL(commitments(peerBelowLimit2, randomBytes32()), ShortIdAliases(ShortChannelId.generateLocalAlias(), None), null, None, None, None, SpliceStatus.NoSplice),
DATA_SHUTDOWN(commitments(peerBelowLimit2, randomBytes32()), Shutdown(randomBytes32(), ByteVector.empty), Shutdown(randomBytes32(), ByteVector.empty), None),
DATA_CLOSING(commitments(peerBelowLimit2, randomBytes32()), BlockHeight(0), ByteVector.empty, List(), List(closingTx))
)
val privateChannels = Seq(
DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments(privatePeer1, channelIdPrivate1), ShortIds(None, ShortChannelId.generateLocalAlias(), None)),
DATA_NORMAL(commitments(privatePeer2, randomBytes32()), ShortIds(None, ShortChannelId.generateLocalAlias(), None), None, null, None, None, None, SpliceStatus.NoSplice),
DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments(privatePeer1, channelIdPrivate1), ShortIdAliases(ShortChannelId.generateLocalAlias(), None)),
DATA_NORMAL(commitments(privatePeer2, randomBytes32()), ShortIdAliases(ShortChannelId.generateLocalAlias(), None), null, None, None, None, SpliceStatus.NoSplice),
)
val initiatorChannels = Seq(
DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments(peerBelowLimit1, randomBytes32(), isOpener = true), BlockHeight(0), None, Left(FundingCreated(channelIdAtLimit1, TxId(ByteVector32.Zeroes), 3, randomBytes64()))),
DATA_WAIT_FOR_CHANNEL_READY(commitments(peerBelowLimit1, randomBytes32(), isOpener = true), ShortIds(None, ShortChannelId.generateLocalAlias(), None)),
DATA_WAIT_FOR_CHANNEL_READY(commitments(peerBelowLimit1, randomBytes32(), isOpener = true), ShortIdAliases(ShortChannelId.generateLocalAlias(), None)),
DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments(peerAtLimit1, randomBytes32(), isOpener = true), 0 msat, 0 msat, BlockHeight(0), BlockHeight(0), DualFundingStatus.WaitingForConfirmations, None),
DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments(peerAtLimit1, randomBytes32(), isOpener = true), ShortIds(None, ShortChannelId.generateLocalAlias(), None)),
DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments(peerAtLimit1, randomBytes32(), isOpener = true), ShortIdAliases(ShortChannelId.generateLocalAlias(), None)),
)
val publicChannels = channelsOnWhitelistAtLimit ++ channelsAtLimit1 ++ channelsAtLimit2 ++ channelsBelowLimit1 ++ channelsBelowLimit2
val allChannels = publicChannels ++ privateChannels ++ initiatorChannels
@ -113,7 +113,8 @@ class PendingChannelsRateLimiterSpec extends ScalaTestWithActorTestKit(ConfigFac
def announcement(nodeId: PublicKey): NodeAnnouncement = NodeAnnouncement(randomBytes64(), Features.empty, 1 unixsec, nodeId, Color(100.toByte, 200.toByte, 300.toByte), "node-alias", NodeAddress.fromParts("1.2.3.4", 42000).get :: Nil)
def commitments(remoteNodeId: PublicKey, channelId: ByteVector32, isOpener: Boolean = false): Commitments = {
val commitments = CommitmentsSpec.makeCommitments(500_000 msat, 400_000 msat, TestConstants.Alice.nodeParams.nodeId, remoteNodeId, announceChannel = true)
val ann = Announcements.makeChannelAnnouncement(Block.RegtestGenesisBlock.hash, RealShortChannelId(42), TestConstants.Alice.nodeParams.nodeId, remoteNodeId, randomKey().publicKey, randomKey().publicKey, randomBytes64(), randomBytes64(), randomBytes64(), randomBytes64())
val commitments = CommitmentsSpec.makeCommitments(500_000 msat, 400_000 msat, TestConstants.Alice.nodeParams.nodeId, remoteNodeId, announcement_opt = Some(ann))
commitments.copy(params = commitments.params.copy(channelId = channelId, localParams = commitments.params.localParams.copy(isChannelOpener = isOpener)))
}
@ -315,9 +316,9 @@ class PendingChannelsRateLimiterSpec extends ScalaTestWithActorTestKit(ConfigFac
import f._
val channels = Seq(
DATA_WAIT_FOR_CHANNEL_READY(commitments(randomKey().publicKey, randomBytes32()), ShortIds(None, ShortChannelId.generateLocalAlias(), None)),
DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments(randomKey().publicKey, randomBytes32()), ShortIds(None, ShortChannelId.generateLocalAlias(), None)),
DATA_NORMAL(commitments(randomKey().publicKey, randomBytes32()), ShortIds(None, ShortChannelId.generateLocalAlias(), None), None, null, None, None, None, SpliceStatus.NoSplice),
DATA_WAIT_FOR_CHANNEL_READY(commitments(randomKey().publicKey, randomBytes32()), ShortIdAliases(ShortChannelId.generateLocalAlias(), None)),
DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments(randomKey().publicKey, randomBytes32()), ShortIdAliases(ShortChannelId.generateLocalAlias(), None)),
DATA_NORMAL(commitments(randomKey().publicKey, randomBytes32()), ShortIdAliases(ShortChannelId.generateLocalAlias(), None), null, None, None, None, SpliceStatus.NoSplice),
DATA_SHUTDOWN(commitments(randomKey().publicKey, randomBytes32()), Shutdown(randomBytes32(), ByteVector.empty), Shutdown(randomBytes32(), ByteVector.empty), None),
DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments(randomKey().publicKey, randomBytes32()), BlockHeight(0), None, Left(FundingCreated(randomBytes32(), TxId(ByteVector32.Zeroes), 3, randomBytes64()))),
)

View file

@ -139,8 +139,7 @@ class JsonSerializersSpec extends TestKitBaseClass with AnyFunSuiteLike with Mat
ShaChain.init,
Map.empty,
),
ShortIds(None, Alias(42), None),
None,
ShortIdAliases(Alias(42), None),
ChannelUpdate(ByteVector64(hex"345b2b05ec046ffe0c14d3b61838c79980713ad1cf8ae7a45c172ce90c9c0b9f345b2b05ec046ffe0c14d3b61838c79980713ad1cf8ae7a45c172ce90c9c0b9f"), Block.RegtestGenesisBlock.hash, ShortChannelId(0), 0 unixsec, ChannelUpdate.MessageFlags(dontForward = false), ChannelUpdate.ChannelFlags.DUMMY, CltvExpiryDelta(12), 1 msat, 100 msat, 0, 2_000_000 msat),
None, None, None, SpliceStatus.NoSplice
)
@ -220,7 +219,7 @@ class JsonSerializersSpec extends TestKitBaseClass with AnyFunSuiteLike with Mat
| "remotePerCommitmentSecrets": null,
| "originChannels": {}
| },
| "shortIds": { "localAlias": "0x2a" },
| "aliases": { "localAlias": "0x2a" },
| "channelUpdate": {
| "signature": "345b2b05ec046ffe0c14d3b61838c79980713ad1cf8ae7a45c172ce90c9c0b9f345b2b05ec046ffe0c14d3b61838c79980713ad1cf8ae7a45c172ce90c9c0b9f",
| "chainHash": "06226e46111a0b59caaf126043eb5bbf28c34f3a5e332a1fc7b2b73cf188910f",
@ -447,12 +446,10 @@ class JsonSerializersSpec extends TestKitBaseClass with AnyFunSuiteLike with Mat
JsonSerializers.serialization.write(map)(JsonSerializers.formats) shouldBe s"""{"e2fc57221cfb1942224082174022f3f70a32005aa209956f9c94c6903f7669ff":"ok","8e3ec6e16436b7dc61b86340192603d05f16d4f8e06c8aaa02fbe2ad63209af3":"cannot execute command=CMD_UPDATE_RELAY_FEE in state=CLOSING","74ca7a86e52d597aa2248cd2ff3b24428ede71345262be7fb31afddfe18dc0d8":"channel 74ca7a86e52d597aa2248cd2ff3b24428ede71345262be7fb31afddfe18dc0d8 not found"}"""
}
test("serialize short ids") {
test("serialize short id aliases") {
val testCases = Map(
ShortIds(real_opt = None, localAlias = Alias(0x4455), remoteAlias_opt = Some(Alias(0x88888888L))) ->
"""{"localAlias":"0x4455","remoteAlias":"0x88888888"}""",
ShortIds(real_opt = Some(RealShortChannelId(BlockHeight(500000), 42, 1)), localAlias = Alias(0x4455), remoteAlias_opt = None) ->
"""{"real":"500000x42x1","localAlias":"0x4455"}""",
ShortIdAliases(localAlias = Alias(0x4455), remoteAlias_opt = Some(Alias(0x88888888L))) -> """{"localAlias":"0x4455","remoteAlias":"0x88888888"}""",
ShortIdAliases(localAlias = Alias(0x4455), remoteAlias_opt = None) -> """{"localAlias":"0x4455"}""",
)
for ((obj, json) <- testCases) {
JsonSerializers.serialization.write(obj)(JsonSerializers.formats) shouldBe json

View file

@ -18,10 +18,9 @@ package fr.acinq.eclair.payment
import akka.actor.ActorRef
import fr.acinq.bitcoin.scalacompat.DeterministicWallet.ExtendedPrivateKey
import fr.acinq.bitcoin.scalacompat.{Block, ByteVector32, Crypto, DeterministicWallet, OutPoint, Satoshi, SatoshiLong, TxOut}
import fr.acinq.bitcoin.scalacompat.{Block, ByteVector32, Crypto, DeterministicWallet, OutPoint, Satoshi, SatoshiLong, Transaction, TxOut}
import fr.acinq.eclair.FeatureSupport.{Mandatory, Optional}
import fr.acinq.eclair.Features._
import fr.acinq.eclair.TestUtils.randomTxId
import fr.acinq.eclair.channel._
import fr.acinq.eclair.channel.fsm.Channel
import fr.acinq.eclair.crypto.{ShaChain, Sphinx}
@ -715,20 +714,25 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
object PaymentPacketSpec {
def makeCommitments(channelId: ByteVector32, testAvailableBalanceForSend: MilliSatoshi = 50000000 msat, testAvailableBalanceForReceive: MilliSatoshi = 50000000 msat, testCapacity: Satoshi = 100000 sat, channelFeatures: ChannelFeatures = ChannelFeatures()): Commitments = {
def makeCommitments(channelId: ByteVector32, testAvailableBalanceForSend: MilliSatoshi = 50000000 msat, testAvailableBalanceForReceive: MilliSatoshi = 50000000 msat, testCapacity: Satoshi = 100000 sat, channelFeatures: ChannelFeatures = ChannelFeatures(), announcement_opt: Option[ChannelAnnouncement] = None): Commitments = {
val channelReserve = testCapacity * 0.01
val localParams = LocalParams(null, null, null, Long.MaxValue.msat, Some(channelReserve), null, null, 0, isChannelOpener = true, paysCommitTxFees = true, None, None, null)
val remoteParams = RemoteParams(randomKey().publicKey, null, UInt64.MaxValue, Some(channelReserve), null, null, maxAcceptedHtlcs = 0, null, null, null, null, null, None)
val commitInput = InputInfo(OutPoint(randomTxId(), 1), TxOut(testCapacity, Nil), Nil)
val fundingTx = Transaction(2, Nil, Seq(TxOut(testCapacity, Nil)), 0)
val commitInput = InputInfo(OutPoint(fundingTx, 0), fundingTx.txOut.head, Nil)
val localCommit = LocalCommit(0, null, CommitTxAndRemoteSig(Transactions.CommitTx(commitInput, null), null), Nil)
val remoteCommit = RemoteCommit(0, null, null, randomKey().publicKey)
val localChanges = LocalChanges(Nil, Nil, Nil)
val remoteChanges = RemoteChanges(Nil, Nil, Nil)
val channelFlags = ChannelFlags(announceChannel = false)
val localFundingStatus = announcement_opt match {
case Some(ann) => LocalFundingStatus.ConfirmedFundingTx(fundingTx, ann.shortChannelId, Some(ann), None, None)
case None => LocalFundingStatus.SingleFundedUnconfirmedFundingTx(None)
}
val channelFlags = ChannelFlags(announceChannel = announcement_opt.nonEmpty)
new Commitments(
ChannelParams(channelId, ChannelConfig.standard, channelFeatures, localParams, remoteParams, channelFlags),
CommitmentChanges(localChanges, remoteChanges, 0, 0),
List(Commitment(0, 0, null, LocalFundingStatus.SingleFundedUnconfirmedFundingTx(None), RemoteFundingStatus.Locked, localCommit, remoteCommit, None)),
List(Commitment(0, 0, null, localFundingStatus, RemoteFundingStatus.Locked, localCommit, remoteCommit, None)),
inactive = Nil,
Right(randomKey().publicKey),
ShaChain.init,

View file

@ -152,32 +152,6 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a
basicRelayTest(f)(relayPayloadScid = localAlias1, lcu = createLocalUpdate(channelId1, optionScidAlias = true, channelUpdateScid_opt = Some(localAlias1)), success = true)
}
test("relay with new real scid after reorg") { f =>
import f._
// initial channel update
val lcu1 = createLocalUpdate(channelId1)
val payload1 = ChannelRelay.Standard(realScid1, outgoingAmount, outgoingExpiry)
val r1 = createValidIncomingPacket(payload1)
channelRelayer ! WrappedLocalChannelUpdate(lcu1)
channelRelayer ! Relay(r1, TestConstants.Alice.nodeParams.nodeId)
expectFwdAdd(register, lcu1.channelId, outgoingAmount, outgoingExpiry, 7)
// reorg happens
val realScid1AfterReorg = RealShortChannelId(111112)
val lcu2 = createLocalUpdate(channelId1).modify(_.shortIds.real_opt).setTo(Some(realScid1AfterReorg))
val payload2 = ChannelRelay.Standard(realScid1AfterReorg, outgoingAmount, outgoingExpiry)
val r2 = createValidIncomingPacket(payload2)
channelRelayer ! WrappedLocalChannelUpdate(lcu2)
// both old and new real scids work
channelRelayer ! Relay(r1, TestConstants.Alice.nodeParams.nodeId)
expectFwdAdd(register, lcu1.channelId, outgoingAmount, outgoingExpiry, 7)
// new real scid works
channelRelayer ! Relay(r2, TestConstants.Alice.nodeParams.nodeId)
expectFwdAdd(register, lcu2.channelId, outgoingAmount, outgoingExpiry, 7)
}
test("relay blinded payment") { f =>
import f._
@ -384,7 +358,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a
val payload = ChannelRelay.Standard(realScid1, outgoingAmount, outgoingExpiry)
val r = createValidIncomingPacket(payload)
val u = createLocalUpdate(channelId1)
val d = LocalChannelDown(null, channelId1, createShortIds(channelId1), outgoingNodeId)
val d = LocalChannelDown(null, channelId1, Seq(realScid1), createAliases(channelId1), outgoingNodeId)
channelRelayer ! WrappedLocalChannelUpdate(u)
channelRelayer ! WrappedLocalChannelDown(d)
@ -576,9 +550,10 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a
def dummyLocalUpdate(shortChannelId: RealShortChannelId, remoteNodeId: PublicKey, availableBalanceForSend: MilliSatoshi, capacity: Satoshi) = {
val channelId = randomBytes32()
val update = Announcements.makeChannelUpdate(Block.RegtestGenesisBlock.hash, randomKey(), remoteNodeId, shortChannelId, CltvExpiryDelta(10), 100 msat, 1000 msat, 100, capacity.toMilliSatoshi)
val commitments = PaymentPacketSpec.makeCommitments(channelId, availableBalanceForSend, testCapacity = capacity)
val shortIds = ShortIds(real_opt = Some(shortChannelId), localAlias = ShortChannelId.generateLocalAlias(), remoteAlias_opt = None)
LocalChannelUpdate(null, channelId, shortIds, remoteNodeId, None, update, commitments)
val ann = Announcements.makeChannelAnnouncement(Block.RegtestGenesisBlock.hash, shortChannelId, TestConstants.Alice.nodeParams.nodeId, outgoingNodeId, randomKey().publicKey, randomKey().publicKey, randomBytes64(), randomBytes64(), randomBytes64(), randomBytes64())
val commitments = PaymentPacketSpec.makeCommitments(channelId, availableBalanceForSend, testCapacity = capacity, announcement_opt = Some(ann))
val aliases = ShortIdAliases(localAlias = ShortChannelId.generateLocalAlias(), remoteAlias_opt = None)
LocalChannelUpdate(null, channelId, aliases, remoteNodeId, Some(AnnouncedCommitment(commitments.latest.commitment, ann)), update, commitments)
}
val (a, b) = (randomKey().publicKey, randomKey().publicKey)
@ -761,8 +736,8 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a
import f._
val channelId_ab = randomBytes32()
val channelId_bc = randomBytes32()
val shortIds_ab = ShortIds(Some(RealShortChannelId(channelUpdate_ab.shortChannelId.toLong)), ShortChannelId.generateLocalAlias(), remoteAlias_opt = None)
val shortIds_bc = ShortIds(Some(RealShortChannelId(channelUpdate_bc.shortChannelId.toLong)), ShortChannelId.generateLocalAlias(), remoteAlias_opt = None)
val aliases_ab = ShortIdAliases(ShortChannelId.generateLocalAlias(), remoteAlias_opt = None)
val aliases_bc = ShortIdAliases(ShortChannelId.generateLocalAlias(), remoteAlias_opt = None)
val a = PaymentPacketSpec.a
val sender = TestProbe[Relayer.OutgoingChannels]()
@ -773,32 +748,32 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a
channels
}
channelRelayer ! WrappedLocalChannelUpdate(LocalChannelUpdate(null, channelId_ab, shortIds_ab, a, None, channelUpdate_ab, makeCommitments(channelId_ab, -2000 msat, 300000 msat)))
channelRelayer ! WrappedLocalChannelUpdate(LocalChannelUpdate(null, channelId_bc, shortIds_bc, c, None, channelUpdate_bc, makeCommitments(channelId_bc, 400000 msat, -5000 msat)))
channelRelayer ! WrappedLocalChannelUpdate(LocalChannelUpdate(null, channelId_ab, aliases_ab, a, None, channelUpdate_ab, makeCommitments(channelId_ab, -2000 msat, 300000 msat)))
channelRelayer ! WrappedLocalChannelUpdate(LocalChannelUpdate(null, channelId_bc, aliases_bc, c, None, channelUpdate_bc, makeCommitments(channelId_bc, 400000 msat, -5000 msat)))
val channels1 = getOutgoingChannels(true)
assert(channels1.size == 2)
assert(channels1.head.channelUpdate == channelUpdate_ab)
assert(channels1.head.toChannelBalance == Relayer.ChannelBalance(a, shortIds_ab, 0 msat, 300000 msat, isPublic = false, isEnabled = true))
assert(channels1.head.toChannelBalance == Relayer.ChannelBalance(a, None, aliases_ab, 0 msat, 300000 msat, isPublic = false, isEnabled = true))
assert(channels1.last.channelUpdate == channelUpdate_bc)
assert(channels1.last.toChannelBalance == Relayer.ChannelBalance(c, shortIds_bc, 400000 msat, 0 msat, isPublic = false, isEnabled = true))
assert(channels1.last.toChannelBalance == Relayer.ChannelBalance(c, None, aliases_bc, 400000 msat, 0 msat, isPublic = false, isEnabled = true))
channelRelayer ! WrappedAvailableBalanceChanged(AvailableBalanceChanged(null, channelId_bc, shortIds_ab, makeCommitments(channelId_bc, 200000 msat, 500000 msat)))
channelRelayer ! WrappedAvailableBalanceChanged(AvailableBalanceChanged(null, channelId_bc, aliases_ab, makeCommitments(channelId_bc, 200000 msat, 500000 msat)))
val channels2 = getOutgoingChannels(true)
assert(channels2.last.commitments.availableBalanceForReceive == 500000.msat && channels2.last.commitments.availableBalanceForSend == 200000.msat)
channelRelayer ! WrappedAvailableBalanceChanged(AvailableBalanceChanged(null, channelId_ab, shortIds_ab, makeCommitments(channelId_ab, 100000 msat, 200000 msat)))
channelRelayer ! WrappedLocalChannelDown(LocalChannelDown(null, channelId_bc, shortIds_ab, c))
channelRelayer ! WrappedAvailableBalanceChanged(AvailableBalanceChanged(null, channelId_ab, aliases_ab, makeCommitments(channelId_ab, 100000 msat, 200000 msat)))
channelRelayer ! WrappedLocalChannelDown(LocalChannelDown(null, channelId_bc, Nil, aliases_ab, c))
val channels3 = getOutgoingChannels(true)
assert(channels3.size == 1 && channels3.head.commitments.availableBalanceForSend == 100000.msat)
channelRelayer ! WrappedLocalChannelUpdate(LocalChannelUpdate(null, channelId_ab, shortIds_ab, a, None, channelUpdate_ab.copy(channelFlags = ChannelUpdate.ChannelFlags(isEnabled = false, isNode1 = true)), makeCommitments(channelId_ab, 100000 msat, 200000 msat)))
channelRelayer ! WrappedLocalChannelUpdate(LocalChannelUpdate(null, channelId_ab, aliases_ab, a, None, channelUpdate_ab.copy(channelFlags = ChannelUpdate.ChannelFlags(isEnabled = false, isNode1 = true)), makeCommitments(channelId_ab, 100000 msat, 200000 msat)))
val channels4 = getOutgoingChannels(true)
assert(channels4.isEmpty)
val channels5 = getOutgoingChannels(false)
assert(channels5.size == 1)
channelRelayer ! WrappedLocalChannelUpdate(LocalChannelUpdate(null, channelId_ab, shortIds_ab, a, None, channelUpdate_ab, makeCommitments(channelId_ab, 100000 msat, 200000 msat)))
channelRelayer ! WrappedLocalChannelUpdate(LocalChannelUpdate(null, channelId_ab, aliases_ab, a, None, channelUpdate_ab, makeCommitments(channelId_ab, 100000 msat, 200000 msat)))
val channels6 = getOutgoingChannels(true)
assert(channels6.size == 1)
}
@ -855,21 +830,22 @@ object ChannelRelayerSpec {
ChannelRelayPacket(add_ab, payload, emptyOnionPacket)
}
def createShortIds(channelId: ByteVector32) = {
val realScid = channelIds.collectFirst { case (realScid: RealShortChannelId, cid) if cid == channelId => realScid }.get
def createAliases(channelId: ByteVector32): ShortIdAliases = {
val localAlias = channelIds.collectFirst { case (localAlias: Alias, cid) if cid == channelId => localAlias }.get
ShortIds(real_opt = Some(realScid), localAlias, remoteAlias_opt = None)
ShortIdAliases(localAlias, remoteAlias_opt = None)
}
def createLocalUpdate(channelId: ByteVector32, channelUpdateScid_opt: Option[ShortChannelId] = None, balance: MilliSatoshi = 100_000_000 msat, capacity: Satoshi = 5_000_000 sat, enabled: Boolean = true, htlcMinimum: MilliSatoshi = 0 msat, timestamp: TimestampSecond = 0 unixsec, feeBaseMsat: MilliSatoshi = 1000 msat, feeProportionalMillionths: Long = 100, optionScidAlias: Boolean = false): LocalChannelUpdate = {
val shortIds = createShortIds(channelId)
val channelUpdateScid = channelUpdateScid_opt.getOrElse(shortIds.real_opt.get)
val aliases = createAliases(channelId)
val realScid = channelIds.collectFirst { case (realScid: RealShortChannelId, cid) if cid == channelId => realScid }.get
val channelUpdateScid = channelUpdateScid_opt.getOrElse(realScid)
val update = ChannelUpdate(ByteVector64(randomBytes(64)), Block.RegtestGenesisBlock.hash, channelUpdateScid, timestamp, ChannelUpdate.MessageFlags(dontForward = false), ChannelUpdate.ChannelFlags(isNode1 = true, isEnabled = enabled), CltvExpiryDelta(100), htlcMinimum, feeBaseMsat, feeProportionalMillionths, capacity.toMilliSatoshi)
val features: Set[PermanentChannelFeature] = Set(
if (optionScidAlias) Some(ScidAlias) else None,
).flatten
val channelFeatures = ChannelFeatures(features)
val commitments = PaymentPacketSpec.makeCommitments(channelId, testAvailableBalanceForSend = balance, testCapacity = capacity, channelFeatures = channelFeatures)
LocalChannelUpdate(null, channelId, shortIds, outgoingNodeId, None, update, commitments)
val announcement = Announcements.makeChannelAnnouncement(Block.RegtestGenesisBlock.hash, realScid, TestConstants.Alice.nodeParams.nodeId, outgoingNodeId, randomKey().publicKey, randomKey().publicKey, randomBytes64(), randomBytes64(), randomBytes64(), randomBytes64())
val commitments = PaymentPacketSpec.makeCommitments(channelId, testAvailableBalanceForSend = balance, testCapacity = capacity, channelFeatures = channelFeatures, announcement_opt = Some(announcement))
LocalChannelUpdate(null, channelId, aliases, outgoingNodeId, Some(AnnouncedCommitment(commitments.latest.commitment, announcement)), update, commitments)
}
}

View file

@ -163,10 +163,10 @@ class OnTheFlyFundingSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike {
}
def makeChannelData(htlcMinimum: MilliSatoshi = 1 msat, localChanges: LocalChanges = LocalChanges(Nil, Nil, Nil)): DATA_NORMAL = {
val commitments = CommitmentsSpec.makeCommitments(500_000_000 msat, 500_000_000 msat, nodeParams.nodeId, remoteNodeId, announceChannel = false)
val commitments = CommitmentsSpec.makeCommitments(500_000_000 msat, 500_000_000 msat, nodeParams.nodeId, remoteNodeId, announcement_opt = None)
.modify(_.params.remoteParams.htlcMinimum).setTo(htlcMinimum)
.modify(_.changes.localChanges).setTo(localChanges)
DATA_NORMAL(commitments, ShortIds(None, Alias(42), None), None, null, None, None, None, SpliceStatus.NoSplice)
DATA_NORMAL(commitments, ShortIdAliases(Alias(42), None), null, None, None, None, SpliceStatus.NoSplice)
}
}

View file

@ -81,8 +81,8 @@ class RelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat
assert(sender.expectMessageType[Relayer.OutgoingChannels].channels.isEmpty)
// We publish a channel update, that should be picked up by the channel relayer
val shortIds_bc = ShortIds(Some(RealShortChannelId(channelUpdate_bc.shortChannelId.toLong)), ShortChannelId.generateLocalAlias(), remoteAlias_opt = None)
system.eventStream ! EventStream.Publish(LocalChannelUpdate(null, channelId_bc, shortIds_bc, c, None, channelUpdate_bc, makeCommitments(channelId_bc)))
val aliases_bc = ShortIdAliases(Alias(channelUpdate_bc.shortChannelId.toLong), remoteAlias_opt = None)
system.eventStream ! EventStream.Publish(LocalChannelUpdate(null, channelId_bc, aliases_bc, c, None, channelUpdate_bc, makeCommitments(channelId_bc)))
eventually(PatienceConfiguration.Timeout(30 seconds), PatienceConfiguration.Interval(1 second)) {
childActors.channelRelayer ! ChannelRelayer.GetOutgoingChannels(sender.ref.toClassic, GetOutgoingChannels())
val channels = sender.expectMessageType[Relayer.OutgoingChannels].channels

View file

@ -92,8 +92,8 @@ abstract class BaseRouterSpec extends TestKitBaseClass with FixtureAnyFunSuiteLi
val alias_ag_private = ShortChannelId.generateLocalAlias()
val alias_ga_private = ShortChannelId.generateLocalAlias()
val scids_ab = ShortIds(Some(scid_ab), alias_ab, Some(alias_ba))
val scids_ag_private = ShortIds(Some(scid_ag_private), alias_ag_private, Some(alias_ga_private))
val scids_ab = ShortIdAliases(alias_ab, Some(alias_ba))
val scids_ag_private = ShortIdAliases(alias_ag_private, Some(alias_ga_private))
val chan_ab = channelAnnouncement(scid_ab, priv_a, priv_b, priv_funding_a, priv_funding_b)
val chan_bc = channelAnnouncement(scid_bc, priv_b, priv_c, priv_funding_b, priv_funding_c)
@ -167,8 +167,8 @@ abstract class BaseRouterSpec extends TestKitBaseClass with FixtureAnyFunSuiteLi
peerConnection.send(router, PeerRoutingMessage(peerConnection.ref, remoteNodeId, update_gh))
peerConnection.send(router, PeerRoutingMessage(peerConnection.ref, remoteNodeId, update_hg))
// then private channels
sender.send(router, ShortChannelIdAssigned(sender.ref, channelId_ag_private, scids_ag_private, remoteNodeId = g, isAnnounced = false))
sender.send(router, LocalChannelUpdate(sender.ref, channelId_ag_private, scids_ag_private, g, None, update_ag_private, CommitmentsSpec.makeCommitments(30000000 msat, 8000000 msat, a, g, announceChannel = false)))
sender.send(router, ShortChannelIdAssigned(sender.ref, channelId_ag_private, announcement_opt = None, scids_ag_private, g))
sender.send(router, LocalChannelUpdate(sender.ref, channelId_ag_private, scids_ag_private, g, None, update_ag_private, CommitmentsSpec.makeCommitments(30000000 msat, 8000000 msat, a, g, announcement_opt = None)))
sender.send(router, PeerRoutingMessage(peerConnection.ref, remoteNodeId, update_ga_private))
// watcher receives the get tx requests
assert(watcher.expectMsgType[ValidateRequest].ann == chan_ab)

View file

@ -55,11 +55,11 @@ class ChannelRouterIntegrationSpec extends TestKitBaseClass with FixtureAnyFunSu
assert(privateChannel.update_1_opt.isDefined)
assert(privateChannel.update_2_opt.isEmpty)
// Alice will only have a real scid if this is not a zeroconf channel.
assert(channels.alice.stateData.asInstanceOf[DATA_NORMAL].shortIds.real_opt.isEmpty == testTags.contains(ChannelStateTestsTags.ZeroConf))
assert(channels.alice.stateData.asInstanceOf[DATA_NORMAL].shortIds.remoteAlias_opt.isDefined)
assert(channels.alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.shortChannelId_opt.isEmpty == testTags.contains(ChannelStateTestsTags.ZeroConf))
assert(channels.alice.stateData.asInstanceOf[DATA_NORMAL].aliases.remoteAlias_opt.isDefined)
// Alice uses her alias for her internal channel update.
val aliceInitialChannelUpdate = privateChannel.update_1_opt.value
assert(aliceInitialChannelUpdate.shortChannelId == privateChannel.shortIds.localAlias)
assert(aliceInitialChannelUpdate.shortChannelId == privateChannel.aliases.localAlias)
// If the channel is public and confirmed, announcement signatures are sent.
val (annSigsA_opt, annSigsB_opt) = if (testTags.contains(ChannelStateTestsTags.ChannelsPublic) && !testTags.contains(ChannelStateTestsTags.ZeroConf)) {
@ -74,8 +74,8 @@ class ChannelRouterIntegrationSpec extends TestKitBaseClass with FixtureAnyFunSu
val aliceChannelUpdate1 = channels.alice2bob.expectMsgType[ChannelUpdate]
val bobChannelUpdate1 = channels.bob2alice.expectMsgType[ChannelUpdate]
// Alice's channel_update uses bob's alias, and vice versa.
assert(aliceChannelUpdate1.shortChannelId == channels.bob.stateData.asInstanceOf[DATA_NORMAL].shortIds.localAlias)
assert(bobChannelUpdate1.shortChannelId == channels.alice.stateData.asInstanceOf[DATA_NORMAL].shortIds.localAlias)
assert(aliceChannelUpdate1.shortChannelId == channels.bob.stateData.asInstanceOf[DATA_NORMAL].aliases.localAlias)
assert(bobChannelUpdate1.shortChannelId == channels.alice.stateData.asInstanceOf[DATA_NORMAL].aliases.localAlias)
// The channel_updates are handled by the peer connection and sent to the router.
val peerConnection = TestProbe()
router ! PeerRoutingMessage(peerConnection.ref, bobNodeId, bobChannelUpdate1)
@ -115,14 +115,14 @@ class ChannelRouterIntegrationSpec extends TestKitBaseClass with FixtureAnyFunSu
// Alice and Bob won't send their channel_update directly to each other because the channel has been announced
// but we can get the update from their data
awaitAssert {
assert(channels.alice.stateData.asInstanceOf[DATA_NORMAL].channelAnnouncement.isDefined)
assert(channels.bob.stateData.asInstanceOf[DATA_NORMAL].channelAnnouncement.isDefined)
assert(channels.alice.stateData.asInstanceOf[DATA_NORMAL].lastAnnouncement_opt.isDefined)
assert(channels.bob.stateData.asInstanceOf[DATA_NORMAL].lastAnnouncement_opt.isDefined)
}
val aliceChannelUpdate2 = channels.alice.stateData.asInstanceOf[DATA_NORMAL].channelUpdate
val bobChannelUpdate2 = channels.bob.stateData.asInstanceOf[DATA_NORMAL].channelUpdate
// Channel updates now use the real scid because the channel has been announced.
val aliceAnn = channels.alice.stateData.asInstanceOf[DATA_NORMAL].channelAnnouncement.get
val bobAnn = channels.bob.stateData.asInstanceOf[DATA_NORMAL].channelAnnouncement.get
val aliceAnn = channels.alice.stateData.asInstanceOf[DATA_NORMAL].lastAnnouncement_opt.get
val bobAnn = channels.bob.stateData.asInstanceOf[DATA_NORMAL].lastAnnouncement_opt.get
assert(aliceAnn == bobAnn)
assert(aliceChannelUpdate2.shortChannelId == aliceAnn.shortChannelId)
assert(!aliceChannelUpdate2.dontForward)

View file

@ -23,7 +23,7 @@ import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
import fr.acinq.bitcoin.scalacompat.Script.{pay2wsh, write}
import fr.acinq.bitcoin.scalacompat.{Block, ByteVector32, OutPoint, Satoshi, SatoshiLong, Transaction, TxIn, TxOut}
import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher._
import fr.acinq.eclair.channel.{AvailableBalanceChanged, CommitmentsSpec, LocalChannelUpdate}
import fr.acinq.eclair.channel._
import fr.acinq.eclair.crypto.TransportHandler
import fr.acinq.eclair.io.Peer.PeerRoutingMessage
import fr.acinq.eclair.payment.Bolt11Invoice.ExtraHop
@ -37,7 +37,7 @@ import fr.acinq.eclair.router.RouteCalculationSpec.{DEFAULT_AMOUNT_MSAT, DEFAULT
import fr.acinq.eclair.router.Router._
import fr.acinq.eclair.transactions.Scripts
import fr.acinq.eclair.wire.protocol._
import fr.acinq.eclair.{BlockHeight, CltvExpiryDelta, Features, MilliSatoshiLong, RealShortChannelId, ShortChannelId, TestConstants, TimestampSecond, randomBytes32, randomKey}
import fr.acinq.eclair.{Alias, BlockHeight, CltvExpiryDelta, Features, MilliSatoshiLong, RealShortChannelId, ShortChannelId, TestConstants, TimestampSecond, randomBytes32, randomKey}
import org.scalatest.Inside.inside
import scodec.bits._
@ -645,7 +645,7 @@ class RouterSpec extends BaseRouterSpec {
assert(res.routes.head.hops.last.nextNodeId == h)
val channelUpdate_ag1 = makeChannelUpdate(Block.RegtestGenesisBlock.hash, priv_a, g, alias_ga_private, CltvExpiryDelta(7), 0 msat, 10 msat, 10, htlcMaximum, enable = false)
sender.send(router, LocalChannelUpdate(sender.ref, channelId_ag_private, scids_ag_private, g, None, channelUpdate_ag1, CommitmentsSpec.makeCommitments(10000 msat, 15000 msat, a, g, announceChannel = false)))
sender.send(router, LocalChannelUpdate(sender.ref, channelId_ag_private, scids_ag_private, g, None, channelUpdate_ag1, CommitmentsSpec.makeCommitments(10000 msat, 15000 msat, a, g, announcement_opt = None)))
router ! RouteRequest(requester.ref, a, SpontaneousRecipient(h, DEFAULT_AMOUNT_MSAT, DEFAULT_EXPIRY, ByteVector32.One), DEFAULT_ROUTE_PARAMS)
requester.expectMessage(PaymentRouteNotFound(RouteNotFound))
}
@ -666,8 +666,8 @@ class RouterSpec extends BaseRouterSpec {
// Via public channels.
router ! RouteRequest(requester.ref, a, SpontaneousRecipient(b, DEFAULT_AMOUNT_MSAT, DEFAULT_EXPIRY, ByteVector32.One), DEFAULT_ROUTE_PARAMS)
requester.expectMessageType[RouteResponse]
val commitments1 = CommitmentsSpec.makeCommitments(10000000 msat, 20000000 msat, a, b, announceChannel = true)
sender.send(router, LocalChannelUpdate(sender.ref, null, scids_ab, b, Some(chan_ab), update_ab, commitments1))
val commitments1 = CommitmentsSpec.makeCommitments(10000000 msat, 20000000 msat, a, b, announcement_opt = Some(chan_ab))
sender.send(router, LocalChannelUpdate(sender.ref, commitments1.channelId, scids_ab, b, Some(AnnouncedCommitment(commitments1.latest.commitment, chan_ab)), update_ab, commitments1))
router ! RouteRequest(requester.ref, a, SpontaneousRecipient(b, 12000000 msat, DEFAULT_EXPIRY, ByteVector32.One), DEFAULT_ROUTE_PARAMS)
requester.expectMessage(PaymentRouteNotFound(BalanceTooLow))
router ! RouteRequest(requester.ref, a, SpontaneousRecipient(b, 12000000 msat, DEFAULT_EXPIRY, ByteVector32.One), DEFAULT_ROUTE_PARAMS, allowMultiPart = true)
@ -811,29 +811,6 @@ class RouterSpec extends BaseRouterSpec {
assert(route.hops.map(_.nextNodeId) == Seq(g, a))
assert(route.hops.map(_.shortChannelId) == Seq(alias_ag_private, alias_ga_private))
}
{
// using the real scid
val preComputedRoute = PredefinedChannelRoute(10000 msat, g, Seq(scid_ag_private))
router ! FinalizeRoute(sender.ref, preComputedRoute)
val response = sender.expectMessageType[RouteResponse]
assert(response.routes.length == 1)
val route = response.routes.head
assert(route.hops.map(_.params) == Seq(HopRelayParams.FromAnnouncement(update_ag_private)))
assert(route.hops.head.nodeId == a)
assert(route.hops.head.nextNodeId == g)
assert(route.hops.head.shortChannelId == alias_ag_private)
}
{
val preComputedRoute = PredefinedChannelRoute(10000 msat, h, Seq(scid_ag_private, scid_gh))
router ! FinalizeRoute(sender.ref, preComputedRoute)
val response = sender.expectMessageType[RouteResponse]
assert(response.routes.length == 1)
val route = response.routes.head
assert(route.hops.map(_.nodeId) == Seq(a, g))
assert(route.hops.map(_.nextNodeId) == Seq(g, h))
assert(route.hops.map(_.shortChannelId) == Seq(alias_ag_private, scid_gh))
assert(route.hops.map(_.params) == Seq(HopRelayParams.FromAnnouncement(update_ag_private), HopRelayParams.FromAnnouncement(update_gh)))
}
}
test("given a pre-defined channels route with routing hints add the proper channel updates") { fixture =>
@ -860,7 +837,7 @@ class RouterSpec extends BaseRouterSpec {
{
val amount = RoutingHeuristics.CAPACITY_CHANNEL_LOW * 2
val invoiceRoutingHint = Invoice.ExtraEdge(h, targetNodeId, RealShortChannelId(BlockHeight(420000), 516, 1105), 10 msat, 150, CltvExpiryDelta(96), 1 msat, None)
val preComputedRoute = PredefinedChannelRoute(amount, targetNodeId, Seq(scid_ag_private, scid_gh, invoiceRoutingHint.shortChannelId))
val preComputedRoute = PredefinedChannelRoute(amount, targetNodeId, Seq(alias_ag_private, scid_gh, invoiceRoutingHint.shortChannelId))
// the amount affects the way we estimate the channel capacity of the hinted channel
assert(amount > RoutingHeuristics.CAPACITY_CHANNEL_LOW)
router ! FinalizeRoute(sender.ref, preComputedRoute, extraEdges = Seq(invoiceRoutingHint))
@ -1010,9 +987,9 @@ class RouterSpec extends BaseRouterSpec {
{
// When the local channel comes back online, it will send a LocalChannelUpdate to the router.
val commitments = CommitmentsSpec.makeCommitments(channel_ab.capacity - 50_000_000.msat, 50_000_000 msat, a, b, announceChannel = true)
val commitments = CommitmentsSpec.makeCommitments(channel_ab.capacity - 50_000_000.msat, 50_000_000 msat, a, b, announcement_opt = Some(chan_ab))
val balances = Set(commitments.availableBalanceForSend, commitments.availableBalanceForReceive)
sender.send(router, LocalChannelUpdate(sender.ref, null, scids_ab, b, Some(chan_ab), update_ab, commitments))
sender.send(router, LocalChannelUpdate(sender.ref, commitments.channelId, scids_ab, b, Some(AnnouncedCommitment(commitments.latest.commitment, chan_ab)), update_ab, commitments))
sender.send(router, GetRoutingState)
val channel_ab1 = sender.expectMsgType[RoutingState].channels.find(_.ann == chan_ab).get
assert(Set(channel_ab1.meta_opt.map(_.balance1), channel_ab1.meta_opt.map(_.balance2)).flatten == balances)
@ -1025,7 +1002,6 @@ class RouterSpec extends BaseRouterSpec {
assert(edge_ab.balance_opt.contains(commitments.availableBalanceForSend))
assert(edge_ba.balance_opt.isEmpty)
}
{
// First we make sure we aren't in the "pending rebroadcast" state for this channel update.
sender.send(router, TickBroadcast)
@ -1033,9 +1009,9 @@ class RouterSpec extends BaseRouterSpec {
assert(sender.expectMsgType[Data].rebroadcast.updates.isEmpty)
// Then we update the balance without changing the contents of the channel update; the graph should still be updated.
val commitments = CommitmentsSpec.makeCommitments(channel_ab.capacity - 40_000_000.msat, 40_000_000 msat, a, b, announceChannel = true)
val commitments = CommitmentsSpec.makeCommitments(channel_ab.capacity - 40_000_000.msat, 40_000_000 msat, a, b, announcement_opt = Some(chan_ab))
val balances = Set(commitments.availableBalanceForSend, commitments.availableBalanceForReceive)
sender.send(router, LocalChannelUpdate(sender.ref, null, scids_ab, b, Some(chan_ab), update_ab, commitments))
sender.send(router, LocalChannelUpdate(sender.ref, commitments.channelId, scids_ab, b, Some(AnnouncedCommitment(commitments.latest.commitment, chan_ab)), update_ab, commitments))
sender.send(router, GetRoutingState)
val channel_ab1 = sender.expectMsgType[RoutingState].channels.find(_.ann == chan_ab).get
assert(Set(channel_ab1.meta_opt.map(_.balance1), channel_ab1.meta_opt.map(_.balance2)).flatten == balances)
@ -1048,10 +1024,9 @@ class RouterSpec extends BaseRouterSpec {
assert(edge_ab.balance_opt.contains(commitments.availableBalanceForSend))
assert(edge_ba.balance_opt.isEmpty)
}
{
// When HTLCs are relayed through the channel, balance changes are sent to the router.
val commitments = CommitmentsSpec.makeCommitments(channel_ab.capacity - 55_000_000.msat, 55_000_000 msat, a, b, announceChannel = true)
val commitments = CommitmentsSpec.makeCommitments(channel_ab.capacity - 55_000_000.msat, 55_000_000 msat, a, b, announcement_opt = Some(chan_ab))
val balances = Set(commitments.availableBalanceForSend, commitments.availableBalanceForReceive)
sender.send(router, AvailableBalanceChanged(sender.ref, null, scids_ab, commitments))
sender.send(router, GetRoutingState)
@ -1066,12 +1041,11 @@ class RouterSpec extends BaseRouterSpec {
assert(edge_ab.balance_opt.contains(commitments.availableBalanceForSend))
assert(edge_ba.balance_opt.isEmpty)
}
{
// Private channels should also update the graph when HTLCs are relayed through them.
sender.send(router, GetRouterData)
val channel_ag = sender.expectMsgType[Data].privateChannels(channelId_ag_private)
val commitments = CommitmentsSpec.makeCommitments(channel_ag.meta.balance1 + 10_000_000.msat, channel_ag.meta.balance2 - 10_000_000.msat, a, g, announceChannel = false)
val commitments = CommitmentsSpec.makeCommitments(channel_ag.meta.balance1 + 10_000_000.msat, channel_ag.meta.balance2 - 10_000_000.msat, a, g, announcement_opt = None)
sender.send(router, AvailableBalanceChanged(sender.ref, channelId_ag_private, scids_ab, commitments))
sender.send(router, Router.GetRouterData)
val data = sender.expectMsgType[Data]
@ -1084,6 +1058,63 @@ class RouterSpec extends BaseRouterSpec {
}
}
test("restore channels after restart") { fixture =>
import fixture._
val sender = TestProbe()
// There are two channels between A and X, where the first channel has been announced but not the second one.
val x = randomKey()
val scid1 = RealShortChannelId(BlockHeight(817_031), 12, 8)
val aliases1 = ShortIdAliases(Alias(37), None)
val scid2 = RealShortChannelId(BlockHeight(819_506), 3, 1)
val aliases2 = ShortIdAliases(Alias(42), None)
val announcement1 = channelAnnouncement(scid1, priv_a, x, randomKey(), randomKey())
val commitments1 = CommitmentsSpec.makeCommitments(250_000_000 msat, 150_000_000 msat, a, x.publicKey, announcement_opt = Some(announcement1))
val update1 = makeChannelUpdate(Block.RegtestGenesisBlock.hash, priv_a, x.publicKey, scid1, CltvExpiryDelta(36), htlcMinimumMsat = 0 msat, feeBaseMsat = 10 msat, feeProportionalMillionths = 10, htlcMaximumMsat = htlcMaximum)
val announcement2 = channelAnnouncement(scid2, priv_a, x, randomKey(), randomKey())
val commitments2 = CommitmentsSpec.makeCommitments(100_000_000 msat, 0 msat, a, x.publicKey, announcement_opt = None)
val update2 = makeChannelUpdate(Block.RegtestGenesisBlock.hash, priv_a, x.publicKey, scid2, CltvExpiryDelta(24), htlcMinimumMsat = 0 msat, feeBaseMsat = 50 msat, feeProportionalMillionths = 15, htlcMaximumMsat = htlcMaximum)
// The first channel comes back online and is added to the public channels.
sender.send(router, ShortChannelIdAssigned(sender.ref, commitments1.channelId, announcement_opt = Some(announcement1), aliases1, x.publicKey))
sender.send(router, LocalChannelUpdate(sender.ref, commitments1.channelId, aliases1, x.publicKey, Some(AnnouncedCommitment(commitments1.latest.commitment, announcement1)), update1, commitments1))
sender.send(router, GetRouterData)
inside(sender.expectMsgType[Data]) { data =>
assert(data.channels.contains(scid1))
val chan1 = data.channels(scid1)
assert(chan1.capacity == 400_000.sat)
assert(chan1.ann == announcement1)
assert((chan1.update_1_opt.toSet ++ chan1.update_2_opt.toSet) == Set(update1))
}
// The second channel comes back online and is added to the private channels.
sender.send(router, ShortChannelIdAssigned(sender.ref, commitments2.channelId, announcement_opt = None, aliases2, x.publicKey))
sender.send(router, LocalChannelUpdate(sender.ref, commitments2.channelId, aliases2, x.publicKey, None, update2, commitments2))
sender.send(router, GetRouterData)
inside(sender.expectMsgType[Data]) { data =>
assert(!data.channels.contains(scid2))
assert(data.scid2PrivateChannels.get(aliases2.localAlias.toLong).contains(commitments2.channelId))
val chan2 = data.privateChannels(commitments2.channelId)
assert(chan2.capacity == 99_000.sat) // for private channels, we use the balance to compute the channel's capacity
assert((chan2.update_1_opt.toSet ++ chan2.update_2_opt.toSet) == Set(update2))
}
// The second channel is announced and moves from the private channels to the public channels.
val fundingConfirmed = LocalFundingStatus.ConfirmedFundingTx(Transaction(2, Nil, TxOut(100_000 sat, Nil) :: Nil, 0), scid2, Some(announcement2), None, None)
val commitments3 = commitments2.updateLocalFundingStatus(commitments2.latest.fundingTxId, fundingConfirmed)(akka.event.NoLogging).toOption.get._1
assert(commitments3.channelId == commitments2.channelId)
sender.send(router, LocalChannelUpdate(sender.ref, commitments3.channelId, aliases2, x.publicKey, Some(AnnouncedCommitment(commitments3.latest.commitment, announcement2)), update2, commitments3))
sender.send(router, GetRouterData)
inside(sender.expectMsgType[Data]) { data =>
assert(data.channels.contains(scid2))
assert(!data.privateChannels.contains(commitments3.channelId))
val chan2 = data.channels(scid2)
assert(chan2.capacity == 100_000.sat)
assert(chan2.ann == announcement2)
assert((chan2.update_1_opt.toSet ++ chan2.update_2_opt.toSet) == Set(update2))
}
}
test("stream updates to front") { fixture =>
import fixture._

View file

@ -335,7 +335,7 @@ object ChannelCodecsSpec {
remoteNextCommitInfo = Right(randomKey().publicKey),
remotePerCommitmentSecrets = ShaChain.init,
originChannels = origins)
DATA_NORMAL(commitments, ShortIds(Some(RealShortChannelId(42)), ShortChannelId.generateLocalAlias(), None), None, channelUpdate, None, None, None, SpliceStatus.NoSplice)
DATA_NORMAL(commitments, ShortIdAliases(ShortChannelId.generateLocalAlias(), None), channelUpdate, None, None, None, SpliceStatus.NoSplice)
}
}

View file

@ -1 +1 @@
[{"remoteNodeId":"03af0ed6052cf28d670665549bc86f4b721c9fdb309d40c58f5811f63966e005d0","shortIds":{"real":"0x0x1","localAlias":"0x2"},"canSend":100000000,"canReceive":20000000,"isPublic":true,"isEnabled":true},{"remoteNodeId":"03af0ed6052cf28d670665549bc86f4b721c9fdb309d40c58f5811f63966e005d0","shortIds":{"real":"0x0x3","localAlias":"0x3","remoteAlias":"0x3"},"canSend":0,"canReceive":30000000,"isPublic":false,"isEnabled":false}]
[{"remoteNodeId":"03af0ed6052cf28d670665549bc86f4b721c9fdb309d40c58f5811f63966e005d0","realScid":"0x0x1","aliases":{"localAlias":"0x2"},"canSend":100000000,"canReceive":20000000,"isPublic":true,"isEnabled":true},{"remoteNodeId":"03af0ed6052cf28d670665549bc86f4b721c9fdb309d40c58f5811f63966e005d0","realScid":"0x0x3","aliases":{"localAlias":"0x3","remoteAlias":"0x3"},"canSend":0,"canReceive":30000000,"isPublic":false,"isEnabled":false}]

View file

@ -1 +1 @@
[{"remoteNodeId":"03af0ed6052cf28d670665549bc86f4b721c9fdb309d40c58f5811f63966e005d0","shortIds":{"localAlias":"0x1"},"canSend":100000000,"canReceive":20000000,"isPublic":true,"isEnabled":true},{"remoteNodeId":"03af0ed6052cf28d670665549bc86f4b721c9fdb309d40c58f5811f63966e005d0","shortIds":{"real":"0x0x2","localAlias":"0x3","remoteAlias":"0x4"},"canSend":400000000,"canReceive":30000000,"isPublic":false,"isEnabled":true}]
[{"remoteNodeId":"03af0ed6052cf28d670665549bc86f4b721c9fdb309d40c58f5811f63966e005d0","aliases":{"localAlias":"0x1"},"canSend":100000000,"canReceive":20000000,"isPublic":true,"isEnabled":true},{"remoteNodeId":"03af0ed6052cf28d670665549bc86f4b721c9fdb309d40c58f5811f63966e005d0","realScid":"0x0x2","aliases":{"localAlias":"0x3","remoteAlias":"0x4"},"canSend":400000000,"canReceive":30000000,"isPublic":false,"isEnabled":true}]

View file

@ -206,8 +206,8 @@ class ApiServiceSpec extends AnyFunSuite with ScalatestRouteTest with IdiomaticM
test("'usablebalances' returns expected balance json only for enabled channels") {
val eclair = mock[Eclair]
eclair.usableBalances()(any[Timeout]) returns Future.successful(List(
ChannelBalance(aliceNodeId, ShortIds(None, Alias(1), None), 100000000 msat, 20000000 msat, isPublic = true, isEnabled = true),
ChannelBalance(aliceNodeId, ShortIds(Some(RealShortChannelId(2)), Alias(3), Some(Alias(4))), 400000000 msat, 30000000 msat, isPublic = false, isEnabled = true)
ChannelBalance(aliceNodeId, None, ShortIdAliases(Alias(1), None), 100000000 msat, 20000000 msat, isPublic = true, isEnabled = true),
ChannelBalance(aliceNodeId, Some(RealShortChannelId(2)), ShortIdAliases(Alias(3), Some(Alias(4))), 400000000 msat, 30000000 msat, isPublic = false, isEnabled = true)
))
val mockService = mockApi(eclair)
@ -226,8 +226,8 @@ class ApiServiceSpec extends AnyFunSuite with ScalatestRouteTest with IdiomaticM
test("'channelbalances' returns expected balance json for all channels") {
val eclair = mock[Eclair]
eclair.channelBalances()(any[Timeout]) returns Future.successful(List(
ChannelBalance(aliceNodeId, ShortIds(Some(RealShortChannelId(1)), Alias(2), None), 100000000 msat, 20000000 msat, isPublic = true, isEnabled = true),
ChannelBalance(aliceNodeId, ShortIds(Some(RealShortChannelId(3)), Alias(3), Some(Alias(3))), 0 msat, 30000000 msat, isPublic = false, isEnabled = false)
ChannelBalance(aliceNodeId, Some(RealShortChannelId(1)), ShortIdAliases(Alias(2), None), 100000000 msat, 20000000 msat, isPublic = true, isEnabled = true),
ChannelBalance(aliceNodeId, Some(RealShortChannelId(3)), ShortIdAliases(Alias(3), Some(Alias(3))), 0 msat, 30000000 msat, isPublic = false, isEnabled = false)
))
val mockService = mockApi(eclair)