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

Rework channel announcement signatures handling

Now that we wait for at least 6 confirmations before considering a
channel confirmed, we can simplify our channel announcement logic.
Whenever a channel reaches the confirmed state, it can be announced
to the network (if nodes wish to announce it). We thus don't need
the "deeply buried" state and the "temporary" scid anymore.

The logic is much simpler to follow: when the channel confirms, we
internally update the real scid to match the confirmed funding tx
and send our `announcement_signatures`. When we receive our peer's
`announcement_signatures`, we stash them if the funding tx doesn't
have enough confirmations yet, otherwise we announce the channel and
create a new `channel_update` that uses the real scid.

Whenever we create a `channel_update`, we simply look at whether the
channel is announced or not to choose which scid to use.

This will make it much simpler to announce splice transactions, which
don't need a "deeply buried" state either and will instead simply rely
on whether the splice transaction is confirmed or not to generate
`announcement_signatures`.
This commit is contained in:
t-bast 2024-12-26 21:34:06 +01:00
parent 96183a93aa
commit 48a7da21ab
No known key found for this signature in database
GPG key ID: 34F377B0100ED6BB
57 changed files with 576 additions and 735 deletions

View file

@ -137,9 +137,6 @@ object ZmqWatcher {
case class WatchFundingConfirmed(replyTo: ActorRef[WatchFundingConfirmedTriggered], txId: TxId, minDepth: Long) extends WatchConfirmed[WatchFundingConfirmedTriggered]
case class WatchFundingConfirmedTriggered(blockHeight: BlockHeight, txIndex: Int, tx: Transaction) extends WatchConfirmedTriggered
case class WatchFundingDeeplyBuried(replyTo: ActorRef[WatchFundingDeeplyBuriedTriggered], txId: TxId, minDepth: Long) extends WatchConfirmed[WatchFundingDeeplyBuriedTriggered]
case class WatchFundingDeeplyBuriedTriggered(blockHeight: BlockHeight, txIndex: Int, tx: Transaction) extends WatchConfirmedTriggered
case class RelativeDelay(parentTxId: TxId, delay: Long)
case class WatchTxConfirmed(replyTo: ActorRef[WatchTxConfirmedTriggered], txId: TxId, minDepth: Long, delay_opt: Option[RelativeDelay] = None) extends WatchConfirmed[WatchTxConfirmedTriggered]
case class WatchTxConfirmedTriggered(blockHeight: BlockHeight, txIndex: Int, tx: Transaction) extends WatchConfirmedTriggered
@ -429,7 +426,6 @@ private class ZmqWatcher(nodeParams: NodeParams, blockHeight: AtomicLong, client
client.getTransactionShortId(w.txId).map {
case (height, index) => w match {
case w: WatchFundingConfirmed => context.self ! TriggerEvent(w.replyTo, w, WatchFundingConfirmedTriggered(height, index, tx))
case w: WatchFundingDeeplyBuried => context.self ! TriggerEvent(w.replyTo, w, WatchFundingDeeplyBuriedTriggered(height, index, tx))
case w: WatchTxConfirmed => context.self ! TriggerEvent(w.replyTo, w, WatchTxConfirmedTriggered(height, index, tx))
case w: WatchParentTxConfirmed => context.self ! TriggerEvent(w.replyTo, w, WatchParentTxConfirmedTriggered(height, index, tx))
case w: WatchAlternativeCommitTxConfirmed => context.self ! TriggerEvent(w.replyTo, w, WatchAlternativeCommitTxConfirmedTriggered(height, index, tx))

View file

@ -416,27 +416,17 @@ case class RevokedCommitPublished(commitTx: Transaction, claimMainOutputTx: Opti
}
}
sealed trait RealScidStatus { def toOption: Option[RealShortChannelId] }
object RealScidStatus {
/** The funding transaction has been confirmed but hasn't reached min_depth, we must be ready for a reorg. */
case class Temporary(realScid: RealShortChannelId) extends RealScidStatus { override def toOption: Option[RealShortChannelId] = Some(realScid) }
/** The funding transaction has been deeply confirmed. */
case class Final(realScid: RealShortChannelId) extends RealScidStatus { override def toOption: Option[RealShortChannelId] = Some(realScid) }
/** We don't know the status of the funding transaction. */
case object Unknown extends RealScidStatus { override def toOption: Option[RealShortChannelId] = None }
}
/**
* Short identifiers for the channel
* Short identifiers for the channel.
*
* @param real the real scid, it may change if a reorg happens before the channel reaches 6 conf
* @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: RealScidStatus, localAlias: Alias, remoteAlias_opt: Option[Alias])
case class ShortIds(real_opt: Option[RealShortChannelId], localAlias: Alias, remoteAlias_opt: Option[Alias])
sealed trait LocalFundingStatus {
def signedTx_opt: Option[Transaction]

View file

@ -45,7 +45,7 @@ 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) extends ChannelEvent
case class ShortChannelIdAssigned(channel: ActorRef, channelId: ByteVector32, shortIds: ShortIds, remoteNodeId: PublicKey, isAnnounced: Boolean) 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
@ -64,7 +64,7 @@ case class LocalChannelUpdate(channel: ActorRef, channelId: ByteVector32, shortI
def scidsForRouting: Seq[ShortChannelId] = {
val canUseRealScid = !commitments.params.channelFeatures.hasFeature(Features.ScidAlias)
if (canUseRealScid) {
shortIds.real.toOption.toSeq :+ shortIds.localAlias
shortIds.real_opt.toSeq :+ shortIds.localAlias
} else {
Seq(shortIds.localAlias)
}

View file

@ -306,7 +306,7 @@ object Helpers {
def channelUpdateForDirectPeer(nodeParams: NodeParams, channelUpdate: ChannelUpdate, shortIds: ShortIds): ChannelUpdate = {
shortIds.remoteAlias_opt match {
case Some(remoteAlias) => Announcements.updateScid(nodeParams.privateKey, channelUpdate, remoteAlias)
case None => shortIds.real.toOption match {
case None => shortIds.real_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.

View file

@ -60,7 +60,7 @@ 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.toOption.toSeq :+ scidAssigned.shortIds.localAlias).map(_ -> scidAssigned.channelId).toMap
val m = (scidAssigned.shortIds.real_opt.toSeq :+ scidAssigned.shortIds.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 {
case Some(channelId) if channelId != scidAssigned.channelId =>

View file

@ -126,9 +126,6 @@ object Channel {
def props(nodeParams: NodeParams, wallet: OnChainChannelFunder with OnchainPubkeyCache, remoteNodeId: PublicKey, blockchain: typed.ActorRef[ZmqWatcher.Command], relayer: ActorRef, txPublisherFactory: TxPublisherFactory): Props =
Props(new Channel(nodeParams, wallet, remoteNodeId, blockchain, relayer, txPublisherFactory))
// see https://github.com/lightningnetwork/lightning-rfc/blob/master/07-routing-gossip.md#requirements
val ANNOUNCEMENTS_MINCONF = 6
// https://github.com/lightningnetwork/lightning-rfc/blob/master/02-peer-protocol.md#requirements
val MAX_FUNDING_WITHOUT_WUMBO: Satoshi = 16777216 sat // = 2^24
val MAX_ACCEPTED_HTLCS = 483
@ -217,6 +214,8 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
var activeConnection = context.system.deadLetters
// we aggregate sigs for splices before processing
var sigStash = Seq.empty[CommitSig]
// we stash announcement_signatures if we receive them earlier than expected
var announcementSigsStash = Map.empty[RealShortChannelId, AnnouncementSignatures]
val txPublisher = txPublisherFactory.spawnTxPublisher(context, remoteNodeId)
@ -355,7 +354,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))
context.system.eventStream.publish(ShortChannelIdAssigned(self, normal.channelId, normal.shortIds, remoteNodeId, normal.channelAnnouncement.nonEmpty))
// 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)
@ -755,70 +754,56 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
case Event(c: CurrentFeerates.BitcoinCore, d: DATA_NORMAL) => handleCurrentFeerate(c, d)
case Event(WatchFundingDeeplyBuriedTriggered(blockHeight, txIndex, fundingTx), d: DATA_NORMAL) if d.channelAnnouncement.isEmpty =>
val finalRealShortId = RealScidStatus.Final(RealShortChannelId(blockHeight, txIndex, d.commitments.latest.commitInput.outPoint.index.toInt))
log.info(s"funding tx is deeply buried at blockHeight=$blockHeight txIndex=$txIndex shortChannelId=${finalRealShortId.realScid}")
val shortIds1 = d.shortIds.copy(real = finalRealShortId)
context.system.eventStream.publish(ShortChannelIdAssigned(self, d.channelId, shortIds1, remoteNodeId))
if (d.shortIds.real == RealScidStatus.Unknown) {
// this is a zero-conf channel and it is the first time we know for sure that the funding tx has been confirmed
context.system.eventStream.publish(TransactionConfirmed(d.channelId, remoteNodeId, fundingTx))
}
val scidForChannelUpdate = Helpers.scidForChannelUpdate(d.channelAnnouncement, shortIds1.localAlias)
// if the shortChannelId is different from the one we had before, we need to re-announce it
val channelUpdate1 = if (d.channelUpdate.shortChannelId != scidForChannelUpdate) {
log.info(s"using new scid in channel_update: old=${d.channelUpdate.shortChannelId} new=$scidForChannelUpdate")
// we re-announce the channelUpdate for the same reason
Helpers.makeChannelUpdate(nodeParams, remoteNodeId, scidForChannelUpdate, d.commitments, d.channelUpdate.relayFees)
} else {
d.channelUpdate
}
if (d.commitments.announceChannel) {
// if channel is public we need to send our announcement_signatures in order to generate the channel_announcement
val localAnnSigs = Helpers.makeAnnouncementSignatures(nodeParams, d.commitments.params, d.commitments.latest.remoteFundingPubKey, finalRealShortId.realScid)
// we use goto() instead of stay() because we want to fire transitions
goto(NORMAL) using d.copy(shortIds = shortIds1, channelUpdate = channelUpdate1) storing() sending localAnnSigs
} else {
// we use goto() instead of stay() because we want to fire transitions
goto(NORMAL) using d.copy(shortIds = shortIds1, channelUpdate = channelUpdate1) storing()
case Event(_: ChannelReady, d: DATA_NORMAL) =>
d.shortIds.real_opt match {
case Some(realScid) if d.channelAnnouncement.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
case _ => stay()
}
case Event(remoteAnnSigs: AnnouncementSignatures, d: DATA_NORMAL) if d.commitments.announceChannel =>
// channels are publicly announced if both parties want it (defined as feature bit)
d.shortIds.real match {
case RealScidStatus.Final(realScid) =>
// we are aware that the channel has reached enough confirmations
// we already had sent our announcement_signatures but we don't store them so we need to recompute it
val localAnnSigs = Helpers.makeAnnouncementSignatures(nodeParams, d.commitments.params, d.commitments.latest.remoteFundingPubKey, realScid)
// 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 =>
require(localAnnSigs.shortChannelId == remoteAnnSigs.shortChannelId, s"shortChannelId mismatch: local=${localAnnSigs.shortChannelId} remote=${remoteAnnSigs.shortChannelId}")
log.info(s"announcing channelId=${d.channelId} on the network with shortId=${localAnnSigs.shortChannelId}")
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)
if (!Announcements.checkSigs(channelAnn)) {
handleLocalError(InvalidAnnouncementSignatures(d.channelId, remoteAnnSigs), d, Some(remoteAnnSigs))
} else {
// we generate a new channel_update because the scid used may change if we were previously using an alias
// 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)
// we use goto() instead of stay() because we want to fire transitions
// We use goto() instead of stay() because we want to fire transitions.
goto(NORMAL) using d.copy(channelAnnouncement = Some(channelAnn), channelUpdate = channelUpdate) storing()
}
case Some(_) =>
// they have sent their announcement sigs, but we already have a valid channel announcement
// this can happen if our announcement_signatures was lost during a disconnection
// specs says that we "MUST respond to the first announcement_signatures message after reconnection with its own announcement_signatures message"
// current implementation always replies to announcement_signatures, not only the first time
// TODO: we should only be nice once, current behaviour opens way to DOS, but this should be handled higher in the stack anyway
log.debug("re-sending our announcement sigs")
stay() sending localAnnSigs
log.debug("ignoring remote announcement_signatures for scid={}, channel is already announced", remoteAnnSigs.shortChannelId)
stay()
}
case _ =>
// our watcher didn't notify yet that the tx has reached ANNOUNCEMENTS_MINCONF confirmations, let's delay remote's message
// note: no need to persist their message, in case of disconnection they will resend it
log.debug("received remote announcement signatures, delaying")
context.system.scheduler.scheduleOnce(5 seconds, self, remoteAnnSigs)
// 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
}
announcementSigsStash += (remoteAnnSigs.shortChannelId -> remoteAnnSigs)
stay()
}
@ -1346,16 +1331,32 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
case Event(w: WatchFundingConfirmedTriggered, d: DATA_NORMAL) =>
acceptFundingTxConfirmed(w, d) match {
case Right((commitments1, commitment)) =>
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))
// 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 {
// this was a zero-conf splice and we already sent our splice_locked
None
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)
maybeUpdateMaxHtlcAmount(d.channelUpdate.htlcMaximumMsat, commitments1)
stay() using d.copy(commitments = commitments1) storing() sending toSend.toSeq
}
maybeEmitEventsPostSplice(d.shortIds, d.commitments, commitments1)
maybeUpdateMaxHtlcAmount(d.channelUpdate.htlcMaximumMsat, commitments1)
stay() using d.copy(commitments = commitments1) storing() sending toSend.toSeq
case Left(_) => stay()
}
@ -1393,8 +1394,6 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
case Event(e: Error, d: DATA_NORMAL) => handleRemoteError(e, d)
case Event(_: ChannelReady, _: DATA_NORMAL) => stay() // will happen after a reconnection if no updates were ever committed to the channel
})
/*
@ -2127,9 +2126,6 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
case Event(BITCOIN_FUNDING_TIMEOUT, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) => handleFundingTimeout(d)
case Event(e: BITCOIN_FUNDING_DOUBLE_SPENT, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => handleDualFundingDoubleSpent(e, d)
// just ignore this, we will put a new watch when we reconnect, and we'll be notified again
case Event(_: WatchFundingDeeplyBuriedTriggered, _) => stay()
})
when(SYNCING)(handleExceptions {
@ -2277,18 +2273,14 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
sendQueue = sendQueue :+ localShutdown
}
d.shortIds.real match {
case RealScidStatus.Final(realShortChannelId) =>
// should we (re)send our announcement sigs?
if (d.commitments.announceChannel && d.channelAnnouncement.isEmpty) {
// BOLT 7: a node SHOULD retransmit the announcement_signatures message if it has not received an announcement_signatures message
val localAnnSigs = Helpers.makeAnnouncementSignatures(nodeParams, d.commitments.params, d.commitments.latest.remoteFundingPubKey, realShortChannelId)
sendQueue = sendQueue :+ localAnnSigs
}
case _ =>
// even if we were just disconnected/reconnected, we need to put back the watch because the event may have been
// fired while we were in OFFLINE (if not, the operation is idempotent anyway)
blockchain ! WatchFundingDeeplyBuried(self, d.commitments.latest.fundingTxId, ANNOUNCEMENTS_MINCONF)
d.shortIds.real_opt match {
case Some(realShortChannelId) if d.commitments.announceChannel && d.channelAnnouncement.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
case _ => ()
}
if (d.commitments.announceChannel) {
@ -2383,9 +2375,6 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
case Event(e: BITCOIN_FUNDING_DOUBLE_SPENT, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => handleDualFundingDoubleSpent(e, d)
// just ignore this, we will put a new watch when we reconnect, and we'll be notified again
case Event(_: WatchFundingDeeplyBuriedTriggered, _) => stay()
case Event(e: Error, d: PersistentChannelData) => handleRemoteError(e, d)
})
@ -2495,14 +2484,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 =>
val realScidStatus = RealScidStatus.Unknown
val shortIds = createShortIds(d.channelId, realScidStatus)
DATA_WAIT_FOR_CHANNEL_READY(commitments1, shortIds)
case d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED =>
val realScidStatus = RealScidStatus.Unknown
val shortIds = createShortIds(d.channelId, realScidStatus)
DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments1, shortIds)
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_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)
@ -2519,20 +2502,24 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
case Event(w: WatchFundingConfirmedTriggered, d: ChannelDataWithCommitments) =>
acceptFundingTxConfirmed(w, d) match {
case Right((commitments1, commitment)) =>
log.info(s"funding txid=${w.tx.txid} has been confirmed")
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 =>
val realScidStatus = RealScidStatus.Temporary(RealShortChannelId(w.blockHeight, w.txIndex, commitment.commitInput.outPoint.index.toInt))
val shortIds = createShortIds(d.channelId, realScidStatus)
DATA_WAIT_FOR_CHANNEL_READY(commitments1, shortIds)
case d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED =>
val realScidStatus = RealScidStatus.Temporary(RealShortChannelId(w.blockHeight, w.txIndex, commitment.commitInput.outPoint.index.toInt))
val shortIds = createShortIds(d.channelId, realScidStatus)
DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments1, shortIds)
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_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_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)
@ -2620,7 +2607,7 @@ 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.toOption != d2.shortIds.real.toOption || d1.channelUpdate != d2.channelUpdate || d1.channelAnnouncement != d2.channelAnnouncement => Some(EmitLocalChannelUpdate("normal->normal", d2, sendToPeer = d2.channelAnnouncement.isEmpty && d1.channelUpdate != d2.channelUpdate))
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))
@ -2629,7 +2616,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
}
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} channel_update={} channel_announcement={}", d.channelUpdate, d.channelAnnouncement.map(_ => "yes").getOrElse("no"))
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)
context.system.eventStream.publish(lcu)
if (sendToPeer) {
@ -2706,10 +2693,11 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
}
}
/** On disconnection we clear up the sig stash */
/** On disconnection we clear up stashes. */
onTransition {
case _ -> OFFLINE =>
sigStash = Nil
announcementSigsStash = Map.empty
}
/*

View file

@ -728,8 +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 realScidStatus = RealScidStatus.Unknown
val shortIds = createShortIds(d.channelId, realScidStatus)
val shortIds = createShortIds(d.channelId, None)
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
@ -739,8 +738,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 realScidStatus = RealScidStatus.Temporary(RealShortChannelId(w.blockHeight, w.txIndex, commitment.commitInput.outPoint.index.toInt))
val shortIds = createShortIds(d.channelId, realScidStatus)
val realScid = RealShortChannelId(w.blockHeight, w.txIndex, commitment.commitInput.outPoint.index.toInt)
val shortIds = createShortIds(d.channelId, Some(realScid))
val channelReady = createChannelReady(shortIds, d.commitments.params)
reportRbfFailure(d.status, InvalidRbfTxConfirmed(d.channelId))
val toSend = d.status match {
@ -787,7 +786,11 @@ 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)
goto(NORMAL) using d1 storing()
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
}
goto(NORMAL) using d1 storing() sending annSigs_opt.toSeq
case Event(_: TxInitRbf, d: DATA_WAIT_FOR_DUAL_FUNDING_READY) =>
// Our peer may not have received the funding transaction confirmation.

View file

@ -397,8 +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 realScidStatus = RealScidStatus.Unknown
val shortIds = createShortIds(d.channelId, realScidStatus)
val shortIds = createShortIds(d.channelId, None)
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
@ -408,8 +407,8 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers {
case Event(w: WatchFundingConfirmedTriggered, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) =>
acceptFundingTxConfirmed(w, d) match {
case Right((commitments1, _)) =>
val realScidStatus = RealScidStatus.Temporary(RealShortChannelId(w.blockHeight, w.txIndex, d.commitments.latest.commitInput.outPoint.index.toInt))
val shortIds = createShortIds(d.channelId, realScidStatus)
val realScid = RealShortChannelId(w.blockHeight, w.txIndex, d.commitments.latest.commitInput.outPoint.index.toInt)
val shortIds = createShortIds(d.channelId, Some(realScid))
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
@ -441,7 +440,11 @@ 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)
goto(NORMAL) using d1 storing()
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
}
goto(NORMAL) using d1 storing() sending annSigs_opt.toSeq
case Event(remoteAnnSigs: AnnouncementSignatures, d: DATA_WAIT_FOR_CHANNEL_READY) if d.commitments.announceChannel =>
delayEarlyAnnouncementSigs(remoteAnnSigs)

View file

@ -20,12 +20,12 @@ 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.ShortChannelId
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}
import fr.acinq.eclair.channel._
import fr.acinq.eclair.channel.fsm.Channel.{ANNOUNCEMENTS_MINCONF, BroadcastChannelUpdate, PeriodicRefresh, REFRESH_CHANNEL_UPDATE_INTERVAL}
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}
@ -107,11 +107,11 @@ trait CommonFundingHandlers extends CommonHandlers {
}
}
def createShortIds(channelId: ByteVector32, realScidStatus: RealScidStatus): ShortIds = {
// the alias will use in our peer's channel_update message, the goal is to be able to use our channel as soon
// as it reaches NORMAL state, and before it is announced on the network
val shortIds = ShortIds(realScidStatus, ShortChannelId.generateLocalAlias(), remoteAlias_opt = None)
context.system.eventStream.publish(ShortChannelIdAssigned(self, channelId, shortIds, remoteNodeId))
def createShortIds(channelId: ByteVector32, real_opt: Option[RealShortChannelId]): ShortIds = {
// 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
}
@ -124,19 +124,17 @@ trait CommonFundingHandlers extends CommonHandlers {
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, shortIds = shortIds1, remoteNodeId = remoteNodeId)))
log.info("shortIds: real={} localAlias={} remoteAlias={}", shortIds1.real.toOption.getOrElse("none"), shortIds1.localAlias, shortIds1.remoteAlias_opt.getOrElse("none"))
// we notify that the channel is now ready to route payments
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"))
// 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
// 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)
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)
// we need to periodically re-send channel updates, otherwise channel will be considered stale and get pruned by network
// We need to periodically re-send channel updates, otherwise channel will be considered stale and get pruned by network.
context.system.scheduler.scheduleWithFixedDelay(initialDelay = REFRESH_CHANNEL_UPDATE_INTERVAL, delay = REFRESH_CHANNEL_UPDATE_INTERVAL, receiver = self, message = BroadcastChannelUpdate(PeriodicRefresh))
// used to get the final shortChannelId, used in announcements (if minDepth >= ANNOUNCEMENTS_MINCONF this event will fire instantly)
blockchain ! WatchFundingDeeplyBuried(self, commitments.latest.fundingTxId, ANNOUNCEMENTS_MINCONF)
val commitments1 = commitments.modify(_.remoteNextCommitInfo).setTo(Right(channelReady.nextPerCommitmentPoint))
peer ! ChannelReadyForPayments(self, remoteNodeId, commitments.channelId, fundingTxIndex = 0)
DATA_NORMAL(commitments1, shortIds1, None, initialChannelUpdate, None, None, None, SpliceStatus.NoSplice)

View file

@ -35,7 +35,7 @@ import fr.acinq.eclair.router.Router._
import fr.acinq.eclair.transactions.DirectedHtlc
import fr.acinq.eclair.transactions.Transactions._
import fr.acinq.eclair.wire.protocol._
import fr.acinq.eclair.{Alias, BlockHeight, CltvExpiry, CltvExpiryDelta, Feature, FeatureSupport, MilliSatoshi, ShortChannelId, TimestampMilli, TimestampSecond, UInt64, UnknownFeature}
import fr.acinq.eclair.{Alias, BlockHeight, CltvExpiry, CltvExpiryDelta, Feature, FeatureSupport, MilliSatoshi, RealShortChannelId, ShortChannelId, TimestampMilli, TimestampSecond, UInt64, UnknownFeature}
import org.json4s
import org.json4s.JsonAST._
import org.json4s.jackson.Serialization
@ -565,8 +565,8 @@ 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: RealScidStatus, localAlias: Alias, remoteAlias: Option[ShortChannelId])
object ShortIdsSerializer extends ConvertClassSerializer[ShortIds](s => ShortIdsJson(s.real, s.localAlias, s.remoteAlias_opt))
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))
// @formatter:on
// @formatter:off
@ -647,12 +647,6 @@ object CustomTypeHints {
classOf[DATA_WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT]
), typeHintFieldName = "type")
val realScidStatuses: CustomTypeHints = CustomTypeHints(Map(
classOf[RealScidStatus.Unknown.type] -> "unknown",
classOf[RealScidStatus.Temporary] -> "temporary",
classOf[RealScidStatus.Final] -> "final",
), typeHintFieldName = "status")
val remoteFundingStatuses: CustomTypeHints = CustomTypeHints(Map(
classOf[RemoteFundingStatus.NotLocked.type] -> "not-locked",
classOf[RemoteFundingStatus.Locked.type] -> "locked",
@ -670,7 +664,6 @@ object JsonSerializers {
CustomTypeHints.onionMessageEvent +
CustomTypeHints.channelSources +
CustomTypeHints.channelStates +
CustomTypeHints.realScidStatuses +
CustomTypeHints.remoteFundingStatuses +
ActorRefSerializer +
TypedActorRefSerializer +

View file

@ -137,7 +137,7 @@ class ChannelRelay private(nodeParams: NodeParams,
case Left(_) => None
case Right(outgoingChannelId) => channels.collectFirst {
case (channelId, channel) if channel.shortIds.localAlias == outgoingChannelId => channelId
case (channelId, channel) if channel.shortIds.real.toOption.contains(outgoingChannelId) => channelId
case (channelId, channel) if channel.shortIds.real_opt.contains(outgoingChannelId) => channelId
}
}

View file

@ -95,7 +95,7 @@ object ChannelRelayer {
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} localAlias=${shortIds.localAlias} remoteNodeId=$remoteNodeId channelUpdate={} commitments={}", 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)
val prevChannelUpdate = channels.get(channelId).map(_.channelUpdate)
val channel = Relayer.OutgoingChannel(shortIds, remoteNodeId, channelUpdate, prevChannelUpdate, commitments)
val channels1 = channels + (channelId -> channel)
@ -108,7 +108,7 @@ object ChannelRelayer {
case WrappedLocalChannelDown(LocalChannelDown(_, channelId, shortIds, remoteNodeId)) =>
context.log.debug(s"removed local channel info for channelId=$channelId localAlias=${shortIds.localAlias}")
val channels1 = channels - channelId
val scid2Channels1 = scid2channels - shortIds.localAlias -- shortIds.real.toOption
val scid2Channels1 = scid2channels - shortIds.localAlias -- shortIds.real_opt
val node2channels1 = node2channels.subtractOne(remoteNodeId, channelId)
apply(nodeParams, register, channels1, scid2Channels1, node2channels1)

View file

@ -28,7 +28,6 @@ import fr.acinq.eclair._
import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher
import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher._
import fr.acinq.eclair.channel._
import fr.acinq.eclair.channel.fsm.Channel.ANNOUNCEMENTS_MINCONF
import fr.acinq.eclair.crypto.TransportHandler
import fr.acinq.eclair.db.NetworkDb
import fr.acinq.eclair.io.Peer.PeerRoutingMessage
@ -343,6 +342,9 @@ class Router(val nodeParams: NodeParams, watcher: typed.ActorRef[ZmqWatcher.Comm
object Router {
// see https://github.com/lightningnetwork/lightning-rfc/blob/master/07-routing-gossip.md#requirements
val ANNOUNCEMENTS_MINCONF = 6
def props(nodeParams: NodeParams, watcher: typed.ActorRef[ZmqWatcher.Command], initialized: Option[Promise[Done]] = None) = Props(new Router(nodeParams, watcher, initialized))
case class SearchBoundaries(maxFeeFlat: MilliSatoshi,
@ -451,7 +453,7 @@ object Router {
// we want the incoming channel_update
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.toOption)
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 =>

View file

@ -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.toOption), lcu.channelUpdate, Set(LocalGossip))
case Left(lcu) => (d.resolve(lcu.channelId, lcu.shortIds.real_opt), lcu.channelUpdate, Set(LocalGossip))
case Right(rcu) =>
rcu.origins.collect {
case RemoteGossip(peerConnection, _) if !wasStashed => // stashed changes have already been acknowledged
@ -535,26 +535,36 @@ object Validation {
}
/**
* We will receive this event before [[LocalChannelUpdate]] or [[ChannelUpdate]]
* Note that we may receive this event before [[ChannelAnnouncement]], [[LocalChannelUpdate]] or [[ChannelUpdate]].
* This function must correctly handle cases where the channel isn't yet in the public graph but will be soon.
*/
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.toOption match {
// 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)
}
log.debug("handleShortChannelIdAssigned scia={} mappings={}", scia, mappings)
val d1 = d.copy(scid2PrivateChannels = d.scid2PrivateChannels ++ mappings)
d1.resolve(scia.channelId, scia.shortIds.real.toOption) match {
d1.resolve(scia.channelId, scia.shortIds.real_opt) match {
case Some(_) =>
// channel is known, nothing more to do
// This channel is already known, nothing more to do.
d1
case None if scia.isAnnounced =>
// 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.
d1
case None =>
// this is a local channel that hasn't yet been announced (maybe it is a private channel or maybe it is a public
// channel that doesn't yet have 6 confirmations), we create a corresponding private channel
val pc = PrivateChannel(scia.channelId, scia.shortIds, localNodeId, scia.remoteNodeId, None, None, ChannelMeta(0 msat, 0 msat))
// This is either:
// - a private channel that hasn't been added yet
// - a public channel that hasn't reached enough confirmations
// 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))
d1.copy(privateChannels = d1.privateChannels + (scia.channelId -> pc))
}
}
@ -563,49 +573,55 @@ 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.toOption) match {
d.resolve(lcu.channelId, lcu.shortIds.real_opt) match {
case Some(publicChannel: PublicChannel) =>
// this a known public channel, we can process the channel_update
// 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 {
case Some(ann) =>
log.debug("private channel graduating to public privateChannel={}", privateChannel)
// channel is graduating from private to public
// since this is a local channel, we can trust the announcement, no need to go through the full
// verification process and make calls to bitcoin core
// 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))
log.debug("processing channel_update")
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 (we now may have the remote_alias) and the balances
// This a known private channel, we update the short ids and the balances.
val pc1 = privateChannel.copy(shortIds = lcu.shortIds).updateBalances(lcu.commitments)
val d1 = d.copy(privateChannels = d.privateChannels + (privateChannel.channelId -> pc1))
// then we can process the channel_update
handleChannelUpdate(d1, db, nodeParams.currentBlockHeight, Left(lcu))
}
case None =>
lcu.shortIds.real.toOption match {
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)
handleChannelUpdate(d, db, nodeParams.currentBlockHeight, Left(lcu))
case _ =>
// should never happen
log.warning("unrecognized local channel update for channelId={} localAlias={}", lcu.channelId, lcu.shortIds.localAlias)
// 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 None =>
log.warning("unrecognized local channel update for private channelId={} localAlias={}", lcu.channelId, lcu.shortIds.localAlias)
d
}
// Process the update: it will be rejected if there is no related channel.
handleChannelUpdate(d1, db, nodeParams.currentBlockHeight, Left(lcu))
}
// handle the update: it will be rejected if there is no related channel
handleChannelUpdate(d, db, nodeParams.currentBlockHeight, Left(lcu))
}
}
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.toOption.map(_.toLong)
val scid2PrivateChannels1 = d.scid2PrivateChannels - lcd.shortIds.localAlias.toLong -- lcd.shortIds.real_opt.map(_.toLong)
// a local channel has permanently gone down
if (lcd.shortIds.real.toOption.exists(d.channels.contains)) {
if (lcd.shortIds.real_opt.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)
@ -624,7 +640,7 @@ object Validation {
}
def handleAvailableBalanceChanged(d: Data, e: AvailableBalanceChanged)(implicit log: LoggingAdapter): Data = {
val (publicChannels1, graphWithBalances1) = e.shortIds.real.toOption.flatMap(d.channels.get) match {
val (publicChannels1, graphWithBalances1) = e.shortIds.real_opt.flatMap(d.channels.get) match {
case Some(pc) =>
val pc1 = pc.updateBalances(e.commitments)
log.debug("public channel balance updated: {}", pc1)

View file

@ -387,8 +387,8 @@ private[channel] object ChannelCodecs0 {
("commitments" | commitmentsCodec) ::
("shortChannelId" | realshortchannelid) ::
("lastSent" | channelReadyCodec)).map {
case commitments :: shortChannelId :: lastSent :: HNil =>
DATA_WAIT_FOR_CHANNEL_READY(commitments, shortIds = ShortIds(real = RealScidStatus.Temporary(shortChannelId), localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None))
case commitments :: shortChannelId :: _ :: HNil =>
DATA_WAIT_FOR_CHANNEL_READY(commitments, shortIds = ShortIds(real_opt = Some(shortChannelId), localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None))
}.decodeOnly
val shutdownCodec: Codec[Shutdown] = (
@ -406,8 +406,8 @@ private[channel] object ChannelCodecs0 {
("localShutdown" | optional(bool, shutdownCodec)) ::
("remoteShutdown" | optional(bool, shutdownCodec)) ::
("closingFeerates" | provide(Option.empty[ClosingFeerates]))).map {
case commitments :: shortChannelId :: buried :: channelAnnouncement :: channelUpdate :: localShutdown :: remoteShutdown :: closingFeerates :: HNil =>
DATA_NORMAL(commitments, shortIds = ShortIds(real = if (buried) RealScidStatus.Final(shortChannelId) else RealScidStatus.Temporary(shortChannelId), localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None), channelAnnouncement, channelUpdate, localShutdown, remoteShutdown, closingFeerates, SpliceStatus.NoSplice)
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)
}.decodeOnly
val DATA_NORMAL_10_Codec: Codec[DATA_NORMAL] = (
@ -419,8 +419,8 @@ private[channel] object ChannelCodecs0 {
("localShutdown" | optional(bool, shutdownCodec)) ::
("remoteShutdown" | optional(bool, shutdownCodec)) ::
("closingFeerates" | provide(Option.empty[ClosingFeerates]))).map {
case commitments :: shortChannelId :: buried :: channelAnnouncement :: channelUpdate :: localShutdown :: remoteShutdown :: closingFeerates :: HNil =>
DATA_NORMAL(commitments, shortIds = ShortIds(real = if (buried) RealScidStatus.Final(shortChannelId) else RealScidStatus.Temporary(shortChannelId), localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None), channelAnnouncement, channelUpdate, localShutdown, remoteShutdown, closingFeerates, SpliceStatus.NoSplice)
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)
}.decodeOnly
val DATA_SHUTDOWN_04_Codec: Codec[DATA_SHUTDOWN] = (

View file

@ -250,8 +250,8 @@ private[channel] object ChannelCodecs1 {
("commitments" | commitmentsCodec) ::
("shortChannelId" | realshortchannelid) ::
("lastSent" | lengthDelimited(channelReadyCodec))).map {
case commitments :: shortChannelId :: lastSent :: HNil =>
DATA_WAIT_FOR_CHANNEL_READY(commitments, shortIds = ShortIds(real = RealScidStatus.Temporary(shortChannelId), localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None))
case commitments :: shortChannelId :: _ :: HNil =>
DATA_WAIT_FOR_CHANNEL_READY(commitments, shortIds = ShortIds(real_opt = Some(shortChannelId), localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None))
}.decodeOnly
val DATA_NORMAL_22_Codec: Codec[DATA_NORMAL] = (
@ -263,8 +263,8 @@ private[channel] object ChannelCodecs1 {
("localShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) ::
("remoteShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) ::
("closingFeerates" | provide(Option.empty[ClosingFeerates]))).map {
case commitments :: shortChannelId :: buried :: channelAnnouncement :: channelUpdate :: localShutdown :: remoteShutdown :: closingFeerates :: HNil =>
DATA_NORMAL(commitments, shortIds = ShortIds(real = if (buried) RealScidStatus.Final(shortChannelId) else RealScidStatus.Temporary(shortChannelId), localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None), channelAnnouncement, channelUpdate, localShutdown, remoteShutdown, closingFeerates, SpliceStatus.NoSplice)
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)
}.decodeOnly
val DATA_SHUTDOWN_23_Codec: Codec[DATA_SHUTDOWN] = (

View file

@ -277,8 +277,8 @@ private[channel] object ChannelCodecs2 {
("commitments" | commitmentsCodec) ::
("shortChannelId" | realshortchannelid) ::
("lastSent" | lengthDelimited(channelReadyCodec))).map {
case commitments :: shortChannelId :: lastSent :: HNil =>
DATA_WAIT_FOR_CHANNEL_READY(commitments, shortIds = ShortIds(real = RealScidStatus.Temporary(shortChannelId), localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None))
case commitments :: shortChannelId :: _ :: HNil =>
DATA_WAIT_FOR_CHANNEL_READY(commitments, shortIds = ShortIds(real_opt = Some(shortChannelId), localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None))
}.decodeOnly
val DATA_NORMAL_02_Codec: Codec[DATA_NORMAL] = (
@ -290,8 +290,8 @@ private[channel] object ChannelCodecs2 {
("localShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) ::
("remoteShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) ::
("closingFeerates" | provide(Option.empty[ClosingFeerates]))).map {
case commitments :: shortChannelId :: buried :: channelAnnouncement :: channelUpdate :: localShutdown :: remoteShutdown :: closingFeerates :: HNil =>
DATA_NORMAL(commitments, shortIds = ShortIds(real = if (buried) RealScidStatus.Final(shortChannelId) else RealScidStatus.Temporary(shortChannelId), localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None), channelAnnouncement, channelUpdate, localShutdown, remoteShutdown, closingFeerates, SpliceStatus.NoSplice)
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)
}.decodeOnly
val DATA_SHUTDOWN_03_Codec: Codec[DATA_SHUTDOWN] = (

View file

@ -343,7 +343,7 @@ private[channel] object ChannelCodecs3 {
("shortChannelId" | realshortchannelid) ::
("lastSent" | lengthDelimited(channelReadyCodec))).map {
case commitments :: shortChannelId :: _ :: HNil =>
DATA_WAIT_FOR_CHANNEL_READY(commitments, shortIds = ShortIds(real = RealScidStatus.Temporary(shortChannelId), localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None))
DATA_WAIT_FOR_CHANNEL_READY(commitments, shortIds = ShortIds(real_opt = Some(shortChannelId), localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None))
}.decodeOnly
val DATA_WAIT_FOR_CHANNEL_READY_0a_Codec: Codec[DATA_WAIT_FOR_CHANNEL_READY] = (
@ -366,8 +366,8 @@ private[channel] object ChannelCodecs3 {
("localShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) ::
("remoteShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) ::
("closingFeerates" | provide(Option.empty[ClosingFeerates]))).map {
case commitments :: shortChannelId :: buried :: channelAnnouncement :: channelUpdate :: localShutdown :: remoteShutdown :: closingFeerates :: HNil =>
DATA_NORMAL(commitments, shortIds = ShortIds(real = if (buried) RealScidStatus.Final(shortChannelId) else RealScidStatus.Temporary(shortChannelId), localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None), channelAnnouncement, channelUpdate, localShutdown, remoteShutdown, closingFeerates, SpliceStatus.NoSplice)
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)
}.decodeOnly
val DATA_NORMAL_07_Codec: Codec[DATA_NORMAL] = (
@ -379,8 +379,8 @@ private[channel] object ChannelCodecs3 {
("localShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) ::
("remoteShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) ::
("closingFeerates" | optional(bool8, closingFeeratesCodec))).map {
case commitments :: shortChannelId :: buried :: channelAnnouncement :: channelUpdate :: localShutdown :: remoteShutdown :: closingFeerates :: HNil =>
DATA_NORMAL(commitments, shortIds = ShortIds(real = if (buried) RealScidStatus.Final(shortChannelId) else RealScidStatus.Temporary(shortChannelId), localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None), channelAnnouncement, channelUpdate, localShutdown, remoteShutdown, closingFeerates, SpliceStatus.NoSplice)
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)
}.decodeOnly
val DATA_NORMAL_09_Codec: Codec[DATA_NORMAL] = (

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, RealScidStatus, ShortIds}
import fr.acinq.eclair.channel.{ChannelFlags, ShortIds}
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,13 +146,8 @@ object CommonCodecs {
val alias: Codec[Alias] = shortchannelid.narrow[Alias](scid => Attempt.successful(Alias(scid.toLong)), scid => scid)
val realShortChannelIdStatus: Codec[RealScidStatus] = discriminated[RealScidStatus].by(uint8)
.typecase(0, provide(RealScidStatus.Unknown))
.typecase(1, realshortchannelid.as[RealScidStatus.Temporary])
.typecase(2, realshortchannelid.as[RealScidStatus.Final])
val shortids: Codec[ShortIds] = (
("real" | realShortChannelIdStatus) ::
("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
("remoteAlias_opt" | optional(bool8, alias))
).as[ShortIds]

View file

@ -112,10 +112,7 @@
"originChannels" : { }
},
"shortIds" : {
"real" : {
"status" : "final",
"realScid" : "1513532x23x1"
},
"real" : "1513532x23x1",
"localAlias" : "0x17183c0000170001"
},
"channelAnnouncement" : {

View file

@ -112,10 +112,7 @@
"originChannels" : { }
},
"shortIds" : {
"real" : {
"status" : "final",
"realScid" : "1413373x969x0"
},
"real" : "1413373x969x0",
"localAlias" : "0x1590fd0003c90000"
},
"channelUpdate" : {

View file

@ -126,10 +126,7 @@
"originChannels" : { }
},
"shortIds" : {
"real" : {
"status" : "final",
"realScid" : "2026958x1x0"
},
"real" : "2026958x1x0",
"localAlias" : "0x1eedce0000010000"
},
"channelAnnouncement" : {

View file

@ -121,10 +121,7 @@
"originChannels" : { }
},
"shortIds" : {
"real" : {
"status" : "temporary",
"realScid" : "400000x42x0"
},
"real" : "400000x42x0",
"localAlias" : "0x3dab1f7dc6942fd"
}
}

View file

@ -127,10 +127,7 @@
"originChannels" : { }
},
"shortIds" : {
"real" : {
"status" : "temporary",
"realScid" : "400000x42x0"
},
"real" : "400000x42x0",
"localAlias" : "0x2d1583d8409d217"
}
}

View file

@ -363,8 +363,8 @@ 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(RealScidStatus.Unknown, Alias(13), None))
val (channelId4, shortIds4) = (randomBytes32(), ShortIds(RealScidStatus.Final(RealShortChannelId(4)), Alias(14), None))
val (channelId3, shortIds3) = (randomBytes32(), ShortIds(None, Alias(13), None))
val (channelId4, shortIds4) = (randomBytes32(), ShortIds(Some(RealShortChannelId(4)), 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)),
@ -372,7 +372,7 @@ class EclairImplSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with I
val scidMapping = Map(
shortIds3.localAlias.toLong -> channelId3,
shortIds4.localAlias.toLong -> channelId4,
shortIds4.real.toOption.get.toLong -> channelId4,
shortIds4.real_opt.get.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 +384,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.toOption.get, unknownScid))
eclair.findRoute(c, 250_000 msat, None, ignoreNodeIds = Seq(b, unknownNodeId), ignoreShortChannelIds = Seq(channel1.shortChannelId, shortIds3.localAlias, shortIds4.real_opt.get, unknownScid))
router.expectMsg(Router.GetRouterData)
router.reply(routerData)
val routeRequest2 = router.expectMsgType[RouteRequest]
@ -395,8 +395,8 @@ class EclairImplSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with I
Router.ChannelDesc(channel1.shortChannelId, b, a),
Router.ChannelDesc(shortIds3.localAlias, a, b),
Router.ChannelDesc(shortIds3.localAlias, b, a),
Router.ChannelDesc(shortIds4.real.toOption.get, a, c),
Router.ChannelDesc(shortIds4.real.toOption.get, c, a),
Router.ChannelDesc(shortIds4.real_opt.get, a, c),
Router.ChannelDesc(shortIds4.real_opt.get, c, a),
))
}

View file

@ -193,10 +193,10 @@ class ZmqWatcherSpec extends TestKitBaseClass with AnyFunSuiteLike with Bitcoind
val tx2 = sendToAddress(address, Btc(0.5), probe)
watcher ! WatchFundingConfirmed(probe.ref, tx1.txid, 1)
watcher ! WatchFundingDeeplyBuried(probe.ref, tx1.txid, 4)
watcher ! WatchFundingDeeplyBuried(probe.ref, tx1.txid, 4) // setting the watch multiple times should be a no-op
watcher ! WatchFundingConfirmed(probe.ref, tx1.txid, 4)
watcher ! WatchFundingConfirmed(probe.ref, tx1.txid, 4) // setting the watch multiple times should be a no-op
watcher ! WatchFundingConfirmed(probe.ref, tx2.txid, 3)
watcher ! WatchFundingDeeplyBuried(probe.ref, tx2.txid, 6)
watcher ! WatchFundingConfirmed(probe.ref, tx2.txid, 6)
probe.expectNoMessage(100 millis)
watcher ! ListWatches(probe.ref)
@ -221,7 +221,7 @@ class ZmqWatcherSpec extends TestKitBaseClass with AnyFunSuiteLike with Bitcoind
assert(probe.expectMsgType[Set[Watch[_]]].size == 1)
generateBlocks(1)
assert(probe.expectMsgType[WatchFundingDeeplyBuriedTriggered].tx.txid == tx1.txid)
assert(probe.expectMsgType[WatchFundingConfirmedTriggered].tx.txid == tx1.txid)
probe.expectNoMessage(100 millis)
watcher ! ListWatches(probe.ref)
@ -232,10 +232,6 @@ class ZmqWatcherSpec extends TestKitBaseClass with AnyFunSuiteLike with Bitcoind
assert(probe.expectMsgType[WatchFundingConfirmedTriggered].tx.txid == tx1.txid)
watcher ! WatchFundingConfirmed(probe.ref, tx2.txid, 2)
assert(probe.expectMsgType[WatchFundingConfirmedTriggered].tx.txid == tx2.txid)
watcher ! WatchFundingDeeplyBuried(probe.ref, tx1.txid, 4)
assert(probe.expectMsgType[WatchFundingDeeplyBuriedTriggered].tx.txid == tx1.txid)
watcher ! WatchFundingDeeplyBuried(probe.ref, tx2.txid, 4)
assert(probe.expectMsgType[WatchFundingDeeplyBuriedTriggered].tx.txid == tx2.txid)
probe.expectNoMessage(100 millis)
watcher ! ListWatches(probe.ref)
@ -463,22 +459,20 @@ class ZmqWatcherSpec extends TestKitBaseClass with AnyFunSuiteLike with Bitcoind
val txid = randomTxId()
watcher ! WatchFundingConfirmed(actor1.ref, txid, 2)
watcher ! WatchFundingConfirmed(actor1.ref, txid, 3)
watcher ! WatchFundingDeeplyBuried(actor1.ref, txid, 3)
watcher ! WatchFundingConfirmed(actor1.ref, TxId(txid.value.reverse), 3)
watcher ! WatchOutputSpent(actor1.ref, txid, 0, Set.empty)
watcher ! WatchOutputSpent(actor1.ref, txid, 1, Set.empty)
watcher ! ListWatches(actor1.ref)
val watches1 = actor1.expectMsgType[Set[Watch[_]]]
assert(watches1.size == 6)
assert(watches1.size == 5)
watcher ! WatchFundingConfirmed(actor2.ref, txid, 2)
watcher ! WatchFundingDeeplyBuried(actor2.ref, txid, 3)
watcher ! WatchFundingConfirmed(actor2.ref, TxId(txid.value.reverse), 3)
watcher ! WatchOutputSpent(actor2.ref, txid, 0, Set.empty)
watcher ! WatchOutputSpent(actor2.ref, txid, 1, Set.empty)
watcher ! ListWatches(actor2.ref)
val watches2 = actor2.expectMsgType[Set[Watch[_]]]
assert(watches2.size == 11)
assert(watches2.size == 9)
assert(watches1.forall(w => watches2.contains(w)))
watcher ! StopWatching(actor2.ref)

View file

@ -47,8 +47,6 @@ import scala.concurrent.Await
import scala.concurrent.duration._
object ChannelStateTestsTags {
/** If set, the channel funding transaction will have more than 6 confirmations. */
val FundingDeeplyBuried = "funding_deeply_buried"
/** If set, channels will not use option_support_large_channel. */
val DisableWumbo = "disable_wumbo"
/** If set, channels will use option_dual_fund. */
@ -65,6 +63,8 @@ object ChannelStateTestsTags {
val ShutdownAnySegwit = "shutdown_anysegwit"
/** If set, channels will be public (otherwise we don't announce them by default). */
val ChannelsPublic = "channels_public"
/** If set, initial announcement_signatures and channel_updates will not be intercepted and ignored. */
val DoNotInterceptGossip = "do_not_intercept_gossip"
/** If set, no amount will be pushed when opening a channel (by default the initiator pushes a small amount). */
val NoPushAmount = "no_push_amount"
/** If set, the non-initiator will push a small amount when opening a dual-funded channel. */
@ -232,8 +232,7 @@ trait ChannelStateTestsBase extends Assertions with Eventually {
(aliceParams, bobParams, channelType)
}
def reachNormal(setup: SetupFixture, tags: Set[String] = Set.empty, interceptChannelUpdates: Boolean = true): Transaction = {
def reachNormal(setup: SetupFixture, tags: Set[String] = Set.empty): Transaction = {
import setup._
val channelConfig = ChannelConfig.standard
@ -357,18 +356,15 @@ trait ChannelStateTestsBase extends Assertions with Eventually {
fundingTx
}
if (interceptChannelUpdates) {
if (!tags.contains(ChannelStateTestsTags.DoNotInterceptGossip)) {
if (tags.contains(ChannelStateTestsTags.ChannelsPublic) && !channelType.features.contains(Features.ZeroConf)) {
alice2bob.expectMsgType[AnnouncementSignatures]
bob2alice.expectMsgType[AnnouncementSignatures]
}
// we don't forward the channel updates, in reality they would be processed by the router
alice2bob.expectMsgType[ChannelUpdate]
bob2alice.expectMsgType[ChannelUpdate]
}
alice2blockchain.expectMsgType[WatchFundingDeeplyBuried]
bob2blockchain.expectMsgType[WatchFundingDeeplyBuried]
if (tags.contains(ChannelStateTestsTags.FundingDeeplyBuried)) {
val fundingTx = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localFundingStatus.signedTx_opt.get
alice ! WatchFundingDeeplyBuriedTriggered(BlockHeight(400000), 42, fundingTx)
bob ! WatchFundingDeeplyBuriedTriggered(BlockHeight(400000), 42, fundingTx)
}
eventually(assert(alice.stateName == NORMAL))
eventually(assert(bob.stateName == NORMAL))

View file

@ -23,7 +23,7 @@ import fr.acinq.bitcoin.scalacompat.{ByteVector32, Transaction}
import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher._
import fr.acinq.eclair.channel._
import fr.acinq.eclair.channel.fsm.Channel
import fr.acinq.eclair.channel.publish.{TxPublisher, TxPublisherSpec}
import fr.acinq.eclair.channel.publish.TxPublisher
import fr.acinq.eclair.channel.states.{ChannelStateTestsBase, ChannelStateTestsTags}
import fr.acinq.eclair.payment.relay.Relayer.RelayFees
import fr.acinq.eclair.router.Announcements
@ -108,9 +108,9 @@ class WaitForChannelReadyStateSpec extends TestKitBaseClass with FixtureAnyFunSu
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.isInstanceOf[RealScidStatus.Temporary])
assert(aliceIds.real_opt.nonEmpty)
val bobIds = bob.stateData.asInstanceOf[DATA_WAIT_FOR_CHANNEL_READY].shortIds
assert(bobIds.real.isInstanceOf[RealScidStatus.Temporary])
assert(bobIds.real_opt.nonEmpty)
val channelReady = bob2alice.expectMsgType[ChannelReady]
assert(channelReady.alias_opt.contains(bobIds.localAlias))
val listener = TestProbe()
@ -126,7 +126,7 @@ class WaitForChannelReadyStateSpec extends TestKitBaseClass with FixtureAnyFunSu
assert(channelUpdateSentToPeer.shortChannelId == bobIds.localAlias)
assert(Announcements.areSameRelayParams(initialChannelUpdate, channelUpdateSentToPeer))
assert(Announcements.checkSig(channelUpdateSentToPeer, alice.underlyingActor.nodeParams.nodeId))
alice2blockchain.expectMsgType[WatchFundingDeeplyBuried]
alice2blockchain.expectNoMessage(100 millis)
bob2alice.expectNoMessage(100 millis)
awaitCond(alice.stateName == NORMAL)
}
@ -135,7 +135,7 @@ class WaitForChannelReadyStateSpec extends TestKitBaseClass with FixtureAnyFunSu
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.asInstanceOf[RealScidStatus.Temporary].realScid
val realScid = aliceIds.real_opt.get
val channelReady = bob2alice.expectMsgType[ChannelReady]
val channelReadyNoAlias = channelReady.modify(_.tlvStream.records).using(_.filterNot(_.isInstanceOf[ChannelReadyTlv.ShortChannelIdTlv]))
bob2alice.forward(alice, channelReadyNoAlias)
@ -148,7 +148,6 @@ class WaitForChannelReadyStateSpec extends TestKitBaseClass with FixtureAnyFunSu
assert(channelUpdateSentToPeer.shortChannelId == realScid)
assert(Announcements.areSameRelayParams(initialChannelUpdate, channelUpdateSentToPeer))
assert(Announcements.checkSig(channelUpdateSentToPeer, alice.underlyingActor.nodeParams.nodeId))
alice2blockchain.expectMsgType[WatchFundingDeeplyBuried]
bob2alice.expectNoMessage(100 millis)
awaitCond(alice.stateName == NORMAL)
}
@ -157,9 +156,9 @@ class WaitForChannelReadyStateSpec extends TestKitBaseClass with FixtureAnyFunSu
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 == RealScidStatus.Unknown)
assert(aliceIds.real_opt.isEmpty)
val bobIds = bob.stateData.asInstanceOf[DATA_WAIT_FOR_CHANNEL_READY].shortIds
assert(bobIds.real == RealScidStatus.Unknown)
assert(bobIds.real_opt.isEmpty)
val channelReady = bob2alice.expectMsgType[ChannelReady]
assert(channelReady.alias_opt.contains(bobIds.localAlias))
val listener = TestProbe()
@ -175,7 +174,6 @@ class WaitForChannelReadyStateSpec extends TestKitBaseClass with FixtureAnyFunSu
assert(channelUpdateSentToPeer.shortChannelId == bobIds.localAlias)
assert(Announcements.areSameRelayParams(initialChannelUpdate, channelUpdateSentToPeer))
assert(Announcements.checkSig(channelUpdateSentToPeer, alice.underlyingActor.nodeParams.nodeId))
alice2blockchain.expectMsgType[WatchFundingDeeplyBuried]
bob2alice.expectNoMessage(100 millis)
awaitCond(alice.stateName == NORMAL)
}
@ -184,9 +182,9 @@ class WaitForChannelReadyStateSpec extends TestKitBaseClass with FixtureAnyFunSu
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 == RealScidStatus.Unknown)
assert(aliceIds.real_opt.isEmpty)
val bobIds = bob.stateData.asInstanceOf[DATA_WAIT_FOR_CHANNEL_READY].shortIds
assert(bobIds.real == RealScidStatus.Unknown)
assert(bobIds.real_opt.isEmpty)
val channelReady = bob2alice.expectMsgType[ChannelReady]
val channelReadyNoAlias = channelReady.modify(_.tlvStream.records).using(_.filterNot(_.isInstanceOf[ChannelReadyTlv.ShortChannelIdTlv]))
bob2alice.forward(alice, channelReadyNoAlias)
@ -199,7 +197,6 @@ class WaitForChannelReadyStateSpec extends TestKitBaseClass with FixtureAnyFunSu
// they can't understand it, too bad for them
assert(channelUpdateSentToPeer.shortChannelId == aliceIds.localAlias)
assert(Announcements.areSameRelayParams(initialChannelUpdate, channelUpdateSentToPeer))
alice2blockchain.expectMsgType[WatchFundingDeeplyBuried]
bob2alice.expectNoMessage(100 millis)
awaitCond(alice.stateName == NORMAL)
}
@ -208,14 +205,16 @@ class WaitForChannelReadyStateSpec extends TestKitBaseClass with FixtureAnyFunSu
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.isInstanceOf[RealScidStatus.Temporary])
assert(aliceIds.real_opt.nonEmpty)
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.isInstanceOf[RealScidStatus.Temporary])
assert(bobIds.real_opt.nonEmpty)
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))
val initialChannelUpdate = alice.stateData.asInstanceOf[DATA_NORMAL].channelUpdate
assert(initialChannelUpdate.shortChannelId == aliceIds.localAlias)
assert(initialChannelUpdate.feeBaseMsat == relayFees.feeBase)
@ -225,7 +224,6 @@ class WaitForChannelReadyStateSpec extends TestKitBaseClass with FixtureAnyFunSu
assert(channelUpdateSentToPeer.shortChannelId == bobIds.localAlias)
assert(Announcements.areSameRelayParams(initialChannelUpdate, channelUpdateSentToPeer))
assert(Announcements.checkSig(channelUpdateSentToPeer, alice.underlyingActor.nodeParams.nodeId))
alice2blockchain.expectMsgType[WatchFundingDeeplyBuried]
bob2alice.expectNoMessage(100 millis)
awaitCond(alice.stateName == NORMAL)
}
@ -234,9 +232,9 @@ class WaitForChannelReadyStateSpec extends TestKitBaseClass with FixtureAnyFunSu
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 == RealScidStatus.Unknown)
assert(aliceIds.real_opt.isEmpty)
val bobIds = bob.stateData.asInstanceOf[DATA_WAIT_FOR_CHANNEL_READY].shortIds
assert(bobIds.real == RealScidStatus.Unknown)
assert(bobIds.real_opt.isEmpty)
val channelReady = bob2alice.expectMsgType[ChannelReady]
assert(channelReady.alias_opt.contains(bobIds.localAlias))
bob2alice.forward(alice)
@ -249,7 +247,6 @@ class WaitForChannelReadyStateSpec extends TestKitBaseClass with FixtureAnyFunSu
assert(channelUpdateSentToPeer.shortChannelId == bobIds.localAlias)
assert(Announcements.areSameRelayParams(initialChannelUpdate, channelUpdateSentToPeer))
assert(Announcements.checkSig(channelUpdateSentToPeer, alice.underlyingActor.nodeParams.nodeId))
alice2blockchain.expectMsgType[WatchFundingDeeplyBuried]
bob2alice.expectNoMessage(100 millis)
awaitCond(alice.stateName == NORMAL)
}

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 == RealScidStatus.Unknown)
assert(aliceListener.expectMsgType[ShortChannelIdAssigned].shortIds.real_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 == RealScidStatus.Unknown)
assert(bobListener.expectMsgType[ShortChannelIdAssigned].shortIds.real_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 == RealScidStatus.Unknown)
assert(aliceListener.expectMsgType[ShortChannelIdAssigned].shortIds.real_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.isInstanceOf[RealScidStatus.Temporary])
assert(aliceListener.expectMsgType[ShortChannelIdAssigned].shortIds.real_opt.nonEmpty)
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.isInstanceOf[RealScidStatus.Temporary])
assert(bobListener.expectMsgType[ShortChannelIdAssigned].shortIds.real_opt.nonEmpty)
awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_READY)
}
@ -258,8 +258,9 @@ 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.isInstanceOf[RealScidStatus.Temporary])
assert(aliceListener.expectMsgType[ShortChannelIdAssigned].shortIds.real_opt.nonEmpty)
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 +688,10 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture
alice ! WatchPublishedTriggered(fundingTx)
alice2blockchain.expectMsgType[WatchFundingConfirmed]
val scids = aliceListener.expectMsgType[ShortChannelIdAssigned].shortIds
assert(scids.real == RealScidStatus.Unknown)
assert(scids.real_opt.isEmpty)
val aliceChannelReady = alice2bob.expectMsgType[ChannelReady]
assert(aliceChannelReady.alias_opt.contains(scids.localAlias))
assert(alice2blockchain.expectMsgType[WatchFundingDeeplyBuried].txId == fundingTx.txid)
alice2blockchain.expectNoMessage(100 millis)
awaitCond(alice.stateName == NORMAL)
}
@ -742,7 +743,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.isInstanceOf[RealScidStatus.Temporary])
assert(aliceListener.expectMsgType[ShortChannelIdAssigned].shortIds.real_opt.nonEmpty)
assert(alice2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTx.txid)
alice2blockchain.expectNoMessage(100 millis)
alice2bob.expectNoMessage(100 millis)
@ -776,7 +777,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.isInstanceOf[RealScidStatus.Temporary])
assert(aliceListener.expectMsgType[ShortChannelIdAssigned].shortIds.real_opt.nonEmpty)
assert(alice2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTx1.txid)
alice2blockchain.expectMsg(UnwatchTxConfirmed(fundingTx2.txId))
alice2blockchain.expectNoMessage(100 millis)

View file

@ -145,7 +145,7 @@ class WaitForDualFundingReadyStateSpec extends TestKitBaseClass with FixtureAnyF
case _ => false
}
assert(alice.stateData.asInstanceOf[DATA_NORMAL].shortIds.real.isInstanceOf[RealScidStatus.Temporary])
assert(alice.stateData.asInstanceOf[DATA_NORMAL].shortIds.real_opt.nonEmpty)
val aliceCommitments = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest
val aliceUpdate = alice.stateData.asInstanceOf[DATA_NORMAL].channelUpdate
assert(aliceUpdate.shortChannelId == aliceChannelReady.alias_opt.value)
@ -153,7 +153,7 @@ class WaitForDualFundingReadyStateSpec extends TestKitBaseClass with FixtureAnyF
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.isInstanceOf[RealScidStatus.Temporary])
assert(bob.stateData.asInstanceOf[DATA_NORMAL].shortIds.real_opt.nonEmpty)
val bobCommitments = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.latest
val bobUpdate = bob.stateData.asInstanceOf[DATA_NORMAL].channelUpdate
assert(bobUpdate.shortChannelId == bobChannelReady.alias_opt.value)
@ -185,12 +185,12 @@ 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 == RealScidStatus.Unknown)
assert(alice.stateData.asInstanceOf[DATA_NORMAL].shortIds.real_opt.isEmpty)
val aliceCommitments = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest
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 == RealScidStatus.Unknown)
assert(bob.stateData.asInstanceOf[DATA_NORMAL].shortIds.real_opt.isEmpty)
val bobCommitments = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.latest
assert(bob.stateData.asInstanceOf[DATA_NORMAL].channelUpdate.shortChannelId == bobChannelReady.alias_opt.value)
assert(bobCommitments.localChannelReserve == aliceCommitments.remoteChannelReserve)
@ -202,6 +202,55 @@ class WaitForDualFundingReadyStateSpec extends TestKitBaseClass with FixtureAnyF
bob2alice.expectNoMessage(100 millis)
}
test("recv ChannelReady (public channel)", Tag(ChannelStateTestsTags.DualFunding), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs), Tag(ChannelStateTestsTags.ChannelsPublic)) { f =>
import f._
val aliceChannelReady = alice2bob.expectMsgType[ChannelReady]
alice2bob.forward(bob, aliceChannelReady)
awaitCond(bob.stateName == NORMAL)
val bobChannelReady = bob2alice.expectMsgType[ChannelReady]
bob2alice.forward(alice, bobChannelReady)
awaitCond(alice.stateName == NORMAL)
// Alice sends announcement_signatures to Bob.
val aliceAnnSigs = alice2bob.expectMsgType[AnnouncementSignatures]
alice2bob.forward(bob, aliceAnnSigs)
val aliceChannelUpdate = alice2bob.expectMsgType[ChannelUpdate]
assert(aliceChannelUpdate.shortChannelId == bobChannelReady.alias_opt.value)
// Bob also sends announcement_signatures, but Alice doesn't receive it.
bob2alice.expectMsgType[AnnouncementSignatures]
val bobChannelUpdate = bob2alice.expectMsgType[ChannelUpdate]
assert(bobChannelUpdate.shortChannelId == aliceChannelReady.alias_opt.value)
awaitCond(bob.stateData.asInstanceOf[DATA_NORMAL].channelAnnouncement.nonEmpty)
// Alice disconnects without receiving Bob's announcement_signatures.
alice ! INPUT_DISCONNECTED
bob ! INPUT_DISCONNECTED
awaitCond(alice.stateName == OFFLINE)
awaitCond(bob.stateName == OFFLINE)
// On reconnection, Alice retransmits channel_ready and announcement_signatures.
val aliceInit = Init(TestConstants.Alice.nodeParams.features.initFeatures())
val bobInit = Init(TestConstants.Bob.nodeParams.features.initFeatures())
alice ! INPUT_RECONNECTED(alice2bob.ref, aliceInit, bobInit)
bob ! INPUT_RECONNECTED(bob2alice.ref, bobInit, aliceInit)
alice2bob.expectMsgType[ChannelReestablish]
alice2bob.forward(bob)
bob2alice.expectMsgType[ChannelReestablish]
bob2alice.forward(alice)
bob2alice.expectMsgType[ChannelReady]
bob2alice.forward(alice)
bob2alice.expectNoMessage(100 millis)
// When receiving channel_ready, Bob retransmits announcement_signatures.
alice2bob.expectMsgType[ChannelReady]
alice2bob.forward(bob)
alice2bob.expectMsgType[AnnouncementSignatures]
alice2bob.forward(bob)
bob2alice.expectMsgType[AnnouncementSignatures]
bob2alice.forward(alice)
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].channelAnnouncement.nonEmpty)
}
test("recv TxInitRbf", Tag(ChannelStateTestsTags.DualFunding), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>
import f._
alice2bob.expectMsgType[ChannelReady]

View file

@ -96,7 +96,6 @@ class WaitForFundingConfirmedStateSpec extends TestKitBaseClass with FixtureAnyF
assert(alice2blockchain.expectMsgType[WatchPublished].txId == fundingTx.txid)
alice ! WatchPublishedTriggered(fundingTx)
alice2blockchain.expectMsgType[WatchFundingConfirmed]
alice2blockchain.expectMsgType[WatchFundingDeeplyBuried]
val aliceChannelReady = alice2bob.expectMsgType[ChannelReady]
assert(aliceChannelReady.alias_opt.nonEmpty)
awaitAssert(assert(alice.stateName == NORMAL))

View file

@ -76,8 +76,6 @@ class NormalQuiescentStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteL
bob2alice.expectMsgType[ChannelReestablish]
bob2alice.forward(alice)
alice2blockchain.expectMsgType[WatchFundingDeeplyBuried]
bob2blockchain.expectMsgType[WatchFundingDeeplyBuried]
awaitCond(alice.stateName == NORMAL)
awaitCond(bob.stateName == NORMAL)
}
@ -189,7 +187,7 @@ class NormalQuiescentStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteL
import f._
val (preimage, add) = addHtlc(50_000_000 msat, bob, alice, bob2alice, alice2bob)
val cmd = c match {
val cmd = c match {
case FulfillHtlc => CMD_FULFILL_HTLC(add.id, preimage)
case FailHtlc => CMD_FAIL_HTLC(add.id, FailureReason.EncryptedDownstreamFailure(randomBytes(252)))
}

View file

@ -60,7 +60,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
implicit val log: akka.event.LoggingAdapter = akka.event.NoLogging
override def withFixture(test: OneArgTest): Outcome = {
val tags = test.tags + ChannelStateTestsTags.DualFunding + ChannelStateTestsTags.FundingDeeplyBuried
val tags = test.tags + ChannelStateTestsTags.DualFunding
val setup = init(tags = tags)
import setup._
reachNormal(setup, tags)
@ -2000,7 +2000,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
rbfTx
}
test("disconnect (RBF commit_sig not received)", Tag(ChannelStateTestsTags.FundingDeeplyBuried)) { f =>
test("disconnect (RBF commit_sig not received)") { f =>
import f._
val htlcs = setupHtlcs(f)
@ -2046,7 +2046,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
resolveHtlcs(f, htlcs)
}
test("disconnect (RBF commit_sig received by alice)", Tag(ChannelStateTestsTags.FundingDeeplyBuried)) { f =>
test("disconnect (RBF commit_sig received by alice)") { f =>
import f._
val htlcs = setupHtlcs(f)
@ -2093,7 +2093,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
resolveHtlcs(f, htlcs)
}
test("disconnect (RBF tx_signatures received by alice)", Tag(ChannelStateTestsTags.FundingDeeplyBuried)) { f =>
test("disconnect (RBF tx_signatures received by alice)") { f =>
import f._
val htlcs = setupHtlcs(f)

View file

@ -28,7 +28,6 @@ import fr.acinq.eclair._
import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher._
import fr.acinq.eclair.blockchain.fee._
import fr.acinq.eclair.blockchain.{CurrentBlockHeight, CurrentFeerates}
import fr.acinq.eclair.channel.RealScidStatus.Final
import fr.acinq.eclair.channel._
import fr.acinq.eclair.channel.fsm.Channel._
import fr.acinq.eclair.channel.publish.TxPublisher.{PublishFinalTx, PublishReplaceableTx, PublishTx}
@ -42,7 +41,8 @@ import fr.acinq.eclair.router.Announcements
import fr.acinq.eclair.transactions.DirectedHtlc.{incoming, outgoing}
import fr.acinq.eclair.transactions.Transactions
import fr.acinq.eclair.transactions.Transactions._
import fr.acinq.eclair.wire.protocol.{AnnouncementSignatures, ClosingSigned, CommitSig, Error, FailureMessageCodecs, FailureReason, PermanentChannelFailure, RevokeAndAck, Shutdown, TemporaryNodeFailure, TlvStream, UpdateAddHtlc, UpdateFailHtlc, UpdateFailMalformedHtlc, UpdateFee, UpdateFulfillHtlc, Warning}
import fr.acinq.eclair.wire.protocol.{AnnouncementSignatures, ChannelUpdate, ClosingSigned, CommitSig, Error, FailureMessageCodecs, FailureReason, PermanentChannelFailure, RevokeAndAck, Shutdown, TemporaryNodeFailure, TlvStream, UpdateAddHtlc, UpdateFailHtlc, UpdateFailMalformedHtlc, UpdateFee, UpdateFulfillHtlc, Warning}
import org.scalatest.Inside.inside
import org.scalatest.funsuite.FixtureAnyFunSuiteLike
import org.scalatest.{Outcome, Tag}
import scodec.bits._
@ -911,7 +911,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.remoteNextCommitInfo == Left(waitForRevocation))
}
test("recv CMD_SIGN (going above balance threshold)", Tag(ChannelStateTestsTags.NoPushAmount), Tag(ChannelStateTestsTags.ChannelsPublic), Tag(ChannelStateTestsTags.AdaptMaxHtlcAmount)) { f =>
test("recv CMD_SIGN (going above balance threshold)", Tag(ChannelStateTestsTags.NoPushAmount), Tag(ChannelStateTestsTags.ChannelsPublic), Tag(ChannelStateTestsTags.DoNotInterceptGossip), Tag(ChannelStateTestsTags.AdaptMaxHtlcAmount)) { f =>
import f._
val aliceListener = TestProbe()
@ -919,22 +919,26 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
val bobListener = TestProbe()
bob.underlying.system.eventStream.subscribe(bobListener.ref, classOf[LocalChannelUpdate])
// We make sure the funding transaction is deeply buried: before that, we won't broadcast channel updates to the network.
val fundingTx = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localFundingStatus.signedTx_opt.get
alice ! WatchFundingDeeplyBuriedTriggered(BlockHeight(42), 0, fundingTx)
// Alice and Bob exchange announcement_signatures and a first channel update using scid aliases.
alice2bob.expectMsgType[AnnouncementSignatures]
alice2bob.forward(bob)
bob ! WatchFundingDeeplyBuriedTriggered(BlockHeight(42), 0, fundingTx)
assert(alice2bob.expectMsgType[ChannelUpdate].shortChannelId.isInstanceOf[Alias])
bob2alice.expectMsgType[AnnouncementSignatures]
bob2alice.forward(alice)
assert(bob2alice.expectMsgType[ChannelUpdate].shortChannelId.isInstanceOf[Alias])
// The channel starts with all funds on alice's side, so htlcMaximumMsat will be initially set to 0 on bob's side.
// A second channel update sets the scid after funding confirmation.
assert(aliceListener.expectMsgType[LocalChannelUpdate].channelUpdate.htlcMaximumMsat == 500_000_000.msat)
assert(aliceListener.expectMsgType[LocalChannelUpdate].channelUpdate.shortChannelId.isInstanceOf[RealShortChannelId])
// The channel starts with all funds on alice's side, so htlc_maximum_msat will be initially set to 0 on bob's side.
inside(aliceListener.expectMsgType[LocalChannelUpdate]) { lcu =>
assert(lcu.channelUpdate.htlcMaximumMsat == 500_000_000.msat)
assert(lcu.channelUpdate.shortChannelId.isInstanceOf[RealShortChannelId])
assert(lcu.channelUpdate.channelFlags.isEnabled)
}
assert(alice.stateData.asInstanceOf[DATA_NORMAL].channelUpdate.channelFlags.isEnabled)
assert(bobListener.expectMsgType[LocalChannelUpdate].channelUpdate.htlcMaximumMsat == 0.msat)
assert(bobListener.expectMsgType[LocalChannelUpdate].channelUpdate.shortChannelId.isInstanceOf[RealShortChannelId])
inside(bobListener.expectMsgType[LocalChannelUpdate]) { lcu =>
assert(lcu.channelUpdate.htlcMaximumMsat == 0.msat)
assert(lcu.channelUpdate.shortChannelId.isInstanceOf[RealShortChannelId])
assert(lcu.channelUpdate.channelFlags.isEnabled)
}
assert(bob.stateData.asInstanceOf[DATA_NORMAL].channelUpdate.channelFlags.isEnabled)
// Alice and Bob use the following balance thresholds:
@ -3590,154 +3594,79 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
assert(addSettled.htlc == htlc1)
}
test("recv WatchFundingDeeplyBuriedTriggered (public channel)", Tag(ChannelStateTestsTags.ChannelsPublic)) { f =>
test("recv WatchFundingConfirmedTriggered (public channel, zero-conf)", Tag(ChannelStateTestsTags.ChannelsPublic), Tag(ChannelStateTestsTags.DoNotInterceptGossip), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs), Tag(ChannelStateTestsTags.ZeroConf)) { f =>
import f._
val aliceIds = alice.stateData.asInstanceOf[DATA_NORMAL].shortIds
val realShortChannelId = aliceIds.real.asInstanceOf[RealScidStatus.Temporary].realScid
// existing funding tx coordinates
val TxCoordinates(blockHeight, txIndex, _) = ShortChannelId.coordinates(realShortChannelId)
alice ! WatchFundingDeeplyBuriedTriggered(blockHeight, txIndex, null)
val annSigs = alice2bob.expectMsgType[AnnouncementSignatures]
assert(annSigs.shortChannelId == realShortChannelId)
// alice updates her internal state
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].shortIds.real == RealScidStatus.Final(realShortChannelId))
alice2bob.expectNoMessage(100 millis)
channelUpdateListener.expectNoMessage(100 millis)
assert(alice.stateData.asInstanceOf[DATA_NORMAL].channelUpdate.shortChannelId == aliceIds.localAlias)
}
test("recv WatchFundingDeeplyBuriedTriggered (public channel, dual funding)", Tag(ChannelStateTestsTags.DualFunding), Tag(ChannelStateTestsTags.ChannelsPublic)) { f =>
import f._
alice ! WatchFundingDeeplyBuriedTriggered(BlockHeight(400000), 42, null)
val annSigs = alice2bob.expectMsgType[AnnouncementSignatures]
// public channel: we don't send the channel_update directly to the peer
alice2bob.expectNoMessage(1 second)
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].shortIds.real == RealScidStatus.Final(annSigs.shortChannelId))
// we don't re-publish the same channel_update if there was no change
channelUpdateListener.expectNoMessage(1 second)
}
test("recv WatchFundingDeeplyBuriedTriggered (public channel, zero-conf)", Tag(ChannelStateTestsTags.ChannelsPublic), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs), Tag(ChannelStateTestsTags.ZeroConf)) { f =>
import f._
// in zero-conf channel we don't have a real short channel id when going to NORMAL state
// 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 == RealScidStatus.Unknown)
// funding tx coordinates (unknown before)
val (blockHeight, txIndex) = (BlockHeight(400000), 42)
alice ! WatchFundingDeeplyBuriedTriggered(blockHeight, txIndex, null)
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)
// 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)
alice ! WatchFundingConfirmedTriggered(blockHeight, txIndex, fundingTx)
val realShortChannelId = RealShortChannelId(blockHeight, txIndex, alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.commitInput.outPoint.index.toInt)
val annSigs = alice2bob.expectMsgType[AnnouncementSignatures]
assert(annSigs.shortChannelId == realShortChannelId)
// alice updates her internal state
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].shortIds.real == RealScidStatus.Final(realShortChannelId))
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))
alice2bob.forward(bob, annSigsA)
alice2bob.expectNoMessage(100 millis)
// we emit a new local channel update containing the same channel_update, but also the new real scid
// Bob doesn't know that the funding transaction is confirmed, so he doesn't send his announcement_signatures yet.
bob2alice.expectNoMessage(100 millis)
bob ! WatchFundingConfirmedTriggered(blockHeight, txIndex, fundingTx)
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)
// 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 == Final(realShortChannelId))
assert(lcu.channelUpdate == aliceState.channelUpdate)
assert(alice.stateData.asInstanceOf[DATA_NORMAL].channelUpdate.shortChannelId == aliceState.shortIds.localAlias)
assert(lcu.shortIds.real_opt.contains(realShortChannelId))
assert(alice.stateData.asInstanceOf[DATA_NORMAL].channelUpdate.shortChannelId == realShortChannelId)
}
test("recv WatchFundingDeeplyBuriedTriggered (public channel, short channel id changed)", Tag(ChannelStateTestsTags.ChannelsPublic)) { f =>
import f._
val aliceState = alice.stateData.asInstanceOf[DATA_NORMAL]
val realShortChannelId = aliceState.shortIds.real.asInstanceOf[RealScidStatus.Temporary].realScid
// existing funding tx coordinates
val TxCoordinates(blockHeight, txIndex, _) = ShortChannelId.coordinates(realShortChannelId)
// new funding tx coordinates (there was a reorg)
val (blockHeight1, txIndex1) = (blockHeight + 10, txIndex + 10)
alice ! WatchFundingDeeplyBuriedTriggered(blockHeight1, txIndex1, null)
val newRealShortChannelId = RealShortChannelId(blockHeight1, txIndex1, alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.commitInput.outPoint.index.toInt)
val annSigs = alice2bob.expectMsgType[AnnouncementSignatures]
assert(annSigs.shortChannelId == newRealShortChannelId)
// update data with real short channel id
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].shortIds.real == RealScidStatus.Final(newRealShortChannelId))
// we emit a new local channel update containing the same channel_update, but also the new real scid
val lcu = channelUpdateListener.expectMsgType[LocalChannelUpdate]
assert(lcu.shortIds.real == Final(newRealShortChannelId))
assert(lcu.channelUpdate == aliceState.channelUpdate)
channelUpdateListener.expectNoMessage(100 millis)
assert(alice.stateData.asInstanceOf[DATA_NORMAL].channelUpdate.shortChannelId == aliceState.shortIds.localAlias)
}
test("recv WatchFundingDeeplyBuriedTriggered (private channel)") { f =>
import f._
val aliceIds = alice.stateData.asInstanceOf[DATA_NORMAL].shortIds
val realShortChannelId = alice.stateData.asInstanceOf[DATA_NORMAL].shortIds.real.asInstanceOf[RealScidStatus.Temporary].realScid
// existing funding tx coordinates
val TxCoordinates(blockHeight, txIndex, _) = ShortChannelId.coordinates(realShortChannelId)
alice ! WatchFundingDeeplyBuriedTriggered(blockHeight, txIndex, null)
// update data with real short channel id
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].shortIds.real == RealScidStatus.Final(realShortChannelId))
// private channel: we'll use the remote alias in the channel_update we sent to our peer, so there is no change and we don't send a new one
alice2bob.expectNoMessage(100 millis)
channelUpdateListener.expectNoMessage(100 millis)
assert(alice.stateData.asInstanceOf[DATA_NORMAL].channelUpdate.shortChannelId == aliceIds.localAlias)
}
test("recv WatchFundingDeeplyBuriedTriggered (private channel, zero-conf)", Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs), Tag(ChannelStateTestsTags.ZeroConf)) { f =>
test("recv WatchFundingConfirmedTriggered (private channel, zero-conf)", Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs), Tag(ChannelStateTestsTags.ZeroConf)) { f =>
import f._
// we create a new listener that registers after alice has published the funding tx
val listener = TestProbe()
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 == RealScidStatus.Unknown)
alice ! WatchFundingDeeplyBuriedTriggered(BlockHeight(42000), 42, null)
val realShortChannelId = RealShortChannelId(BlockHeight(42000), 42, 0)
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 == RealScidStatus.Final(realShortChannelId))
// private channel: we'll use the remote alias in the channel_update we sent to our peer, so there is no change and we don't send a new one
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].shortIds.real_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)
// we emit a new local channel update containing the same channel_update, but also the new real scid
val lcu = channelUpdateListener.expectMsgType[LocalChannelUpdate]
assert(lcu.shortIds.real == Final(realShortChannelId))
assert(lcu.channelUpdate == aliceState.channelUpdate)
assert(alice.stateData.asInstanceOf[DATA_NORMAL].channelUpdate.shortChannelId == aliceState.shortIds.localAlias)
channelUpdateListener.expectNoMessage(100 millis)
// this is the first time we know the funding tx has been confirmed
listener.expectMsgType[TransactionConfirmed]
}
test("recv WatchFundingDeeplyBuriedTriggered (private channel, short channel id changed)") { f =>
import f._
val aliceState = alice.stateData.asInstanceOf[DATA_NORMAL]
val realShortChannelId = aliceState.shortIds.real.asInstanceOf[RealScidStatus.Temporary].realScid
// existing funding tx coordinates
val TxCoordinates(blockHeight, txIndex, _) = ShortChannelId.coordinates(realShortChannelId)
// new funding tx coordinates (there was a reorg)
val (blockHeight1, txIndex1) = (blockHeight + 10, txIndex + 10)
alice ! WatchFundingDeeplyBuriedTriggered(blockHeight1, txIndex1, null)
val newRealShortChannelId = RealShortChannelId(blockHeight1, txIndex1, alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.commitInput.outPoint.index.toInt)
// update data with real short channel id
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].shortIds.real == RealScidStatus.Final(newRealShortChannelId))
// private channel: we'll use the remote alias in the channel_update we sent to our peer, so there is no change and we don't send a new one
alice2bob.expectNoMessage(100 millis)
// we emit a new local channel update containing the same channel_update, but also the new real scid
val lcu = channelUpdateListener.expectMsgType[LocalChannelUpdate]
assert(lcu.shortIds.real == Final(newRealShortChannelId))
assert(lcu.channelUpdate == aliceState.channelUpdate)
assert(alice.stateData.asInstanceOf[DATA_NORMAL].channelUpdate.shortChannelId == aliceState.shortIds.localAlias)
assert(alice.stateData.asInstanceOf[DATA_NORMAL].channelUpdate.shortChannelId == aliceState.shortIds.localAlias)
}
test("recv AnnouncementSignatures", Tag(ChannelStateTestsTags.ChannelsPublic)) { f =>
test("recv AnnouncementSignatures", Tag(ChannelStateTestsTags.ChannelsPublic), Tag(ChannelStateTestsTags.DoNotInterceptGossip)) { f =>
import f._
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
val realShortChannelId = initialState.shortIds.real.asInstanceOf[RealScidStatus.Temporary].realScid
// existing funding tx coordinates
val TxCoordinates(blockHeight, txIndex, _) = ShortChannelId.coordinates(realShortChannelId)
alice ! WatchFundingDeeplyBuriedTriggered(blockHeight, txIndex, null)
val realShortChannelId = initialState.shortIds.real_opt.get
// Alice and Bob exchange announcement_signatures.
val annSigsA = alice2bob.expectMsgType[AnnouncementSignatures]
bob ! WatchFundingDeeplyBuriedTriggered(blockHeight, txIndex, null)
assert(annSigsA.shortChannelId == realShortChannelId)
alice2bob.expectMsgType[ChannelUpdate]
val annSigsB = bob2alice.expectMsgType[AnnouncementSignatures]
import initialState.commitments.latest.{localParams, remoteParams}
val channelAnn = Announcements.makeChannelAnnouncement(Alice.nodeParams.chainHash, annSigsA.shortChannelId, Alice.nodeParams.nodeId, remoteParams.nodeId, Alice.channelKeyManager.fundingPublicKey(localParams.fundingKeyPath, fundingTxIndex = 0).publicKey, initialState.commitments.latest.remoteFundingPubKey, annSigsA.nodeSignature, annSigsB.nodeSignature, annSigsA.bitcoinSignature, annSigsB.bitcoinSignature)
assert(annSigsB.shortChannelId == realShortChannelId)
bob2alice.expectMsgType[ChannelUpdate]
val aliceFundingKey = Alice.channelKeyManager.fundingPublicKey(initialState.commitments.params.localParams.fundingKeyPath, fundingTxIndex = 0).publicKey
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
bob2alice.forward(alice, annSigsB)
awaitAssert {
val normal = alice.stateData.asInstanceOf[DATA_NORMAL]
assert(normal.shortIds.real == RealScidStatus.Final(annSigsA.shortChannelId) && normal.channelAnnouncement.contains(channelAnn) && normal.channelUpdate.shortChannelId == annSigsA.shortChannelId)
assert(normal.shortIds.real_opt.contains(realShortChannelId))
assert(normal.channelAnnouncement.contains(channelAnn))
assert(normal.channelUpdate.shortChannelId == realShortChannelId)
}
// we use the real scid instead of remote alias as soon as the channel is announced
val lcu = channelUpdateListener.expectMsgType[LocalChannelUpdate]
@ -3745,38 +3674,19 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
assert(lcu.channelAnnouncement_opt.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
bob2alice.forward(alice, annSigsB)
alice2bob.expectNoMessage(100 millis)
channelUpdateListener.expectNoMessage(100 millis)
}
test("recv AnnouncementSignatures (re-send)", Tag(ChannelStateTestsTags.ChannelsPublic)) { f =>
import f._
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
val realShortChannelId = initialState.shortIds.real.asInstanceOf[RealScidStatus.Temporary].realScid
// existing funding tx coordinates
val TxCoordinates(blockHeight, txIndex, _) = ShortChannelId.coordinates(realShortChannelId)
alice ! WatchFundingDeeplyBuriedTriggered(blockHeight, txIndex, null)
val annSigsA = alice2bob.expectMsgType[AnnouncementSignatures]
bob ! WatchFundingDeeplyBuriedTriggered(blockHeight, txIndex, null)
val annSigsB = bob2alice.expectMsgType[AnnouncementSignatures]
import initialState.commitments.latest.{localParams, remoteParams}
val channelAnn = Announcements.makeChannelAnnouncement(Alice.nodeParams.chainHash, annSigsA.shortChannelId, Alice.nodeParams.nodeId, remoteParams.nodeId, Alice.channelKeyManager.fundingPublicKey(localParams.fundingKeyPath, fundingTxIndex = 0).publicKey, initialState.commitments.latest.remoteFundingPubKey, annSigsA.nodeSignature, annSigsB.nodeSignature, annSigsA.bitcoinSignature, annSigsB.bitcoinSignature)
bob2alice.forward(alice)
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].channelAnnouncement.contains(channelAnn))
// actual test starts here
// simulate bob re-sending its sigs
bob2alice.send(alice, annSigsA)
// alice re-sends her sigs
alice2bob.expectMsg(annSigsA)
}
test("recv AnnouncementSignatures (invalid)", Tag(ChannelStateTestsTags.ChannelsPublic)) { f =>
test("recv AnnouncementSignatures (invalid)", Tag(ChannelStateTestsTags.ChannelsPublic), Tag(ChannelStateTestsTags.DoNotInterceptGossip)) { f =>
import f._
val channelId = alice.stateData.asInstanceOf[DATA_NORMAL].channelId
alice ! WatchFundingDeeplyBuriedTriggered(BlockHeight(400000), 42, null)
alice2bob.expectMsgType[AnnouncementSignatures]
bob ! WatchFundingDeeplyBuriedTriggered(BlockHeight(400000), 42, null)
alice2bob.expectMsgType[ChannelUpdate]
val annSigsB = bob2alice.expectMsgType[AnnouncementSignatures]
bob2alice.expectMsgType[ChannelUpdate]
// actual test starts here - Bob sends an invalid signature
val annSigsB_invalid = annSigsB.copy(bitcoinSignature = annSigsB.nodeSignature, nodeSignature = annSigsB.bitcoinSignature)
bob2alice.forward(alice, annSigsB_invalid)
@ -3786,10 +3696,8 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
awaitCond(alice.stateName == CLOSING)
}
test("recv BroadcastChannelUpdate", Tag(ChannelStateTestsTags.ChannelsPublic)) { f =>
test("recv BroadcastChannelUpdate", Tag(ChannelStateTestsTags.ChannelsPublic), Tag(ChannelStateTestsTags.DoNotInterceptGossip)) { f =>
import f._
alice ! WatchFundingDeeplyBuriedTriggered(BlockHeight(400000), 42, null)
bob ! WatchFundingDeeplyBuriedTriggered(BlockHeight(400000), 42, null)
val realScid = bob2alice.expectMsgType[AnnouncementSignatures].shortChannelId
bob2alice.forward(alice)
val update1 = channelUpdateListener.expectMsgType[LocalChannelUpdate]
@ -3803,10 +3711,8 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
assert(update1.channelUpdate.timestamp < update2.channelUpdate.timestamp)
}
test("recv BroadcastChannelUpdate (no changes)", Tag(ChannelStateTestsTags.ChannelsPublic)) { f =>
test("recv BroadcastChannelUpdate (no changes)", Tag(ChannelStateTestsTags.ChannelsPublic), Tag(ChannelStateTestsTags.DoNotInterceptGossip)) { f =>
import f._
alice ! WatchFundingDeeplyBuriedTriggered(BlockHeight(400000), 42, null)
bob ! WatchFundingDeeplyBuriedTriggered(BlockHeight(400000), 42, null)
bob2alice.expectMsgType[AnnouncementSignatures]
bob2alice.forward(alice)
channelUpdateListener.expectMsgType[LocalChannelUpdate]
@ -3852,10 +3758,8 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
awaitCond(alice.stateName == OFFLINE)
}
test("recv INPUT_DISCONNECTED (public channel)", Tag(ChannelStateTestsTags.ChannelsPublic)) { f =>
test("recv INPUT_DISCONNECTED (public channel)", Tag(ChannelStateTestsTags.ChannelsPublic), Tag(ChannelStateTestsTags.DoNotInterceptGossip)) { f =>
import f._
alice ! WatchFundingDeeplyBuriedTriggered(BlockHeight(400000), 42, null)
bob ! WatchFundingDeeplyBuriedTriggered(BlockHeight(400000), 42, null)
bob2alice.expectMsgType[AnnouncementSignatures]
bob2alice.forward(alice)
val update1 = channelUpdateListener.expectMsgType[LocalChannelUpdate]
@ -3867,18 +3771,19 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
channelUpdateListener.expectNoMessage(1 second)
}
test("recv INPUT_DISCONNECTED (public channel, with pending unsigned htlcs)", Tag(ChannelStateTestsTags.ChannelsPublic)) { f =>
test("recv INPUT_DISCONNECTED (public channel, with pending unsigned htlcs)", Tag(ChannelStateTestsTags.ChannelsPublic), Tag(ChannelStateTestsTags.DoNotInterceptGossip)) { f =>
import f._
val sender = TestProbe()
alice ! WatchFundingDeeplyBuriedTriggered(BlockHeight(400000), 42, null)
bob ! WatchFundingDeeplyBuriedTriggered(BlockHeight(400000), 42, null)
bob2alice.expectMsgType[AnnouncementSignatures]
bob2alice.forward(alice)
bob2alice.expectMsgType[ChannelUpdate]
alice2bob.expectMsgType[AnnouncementSignatures]
alice2bob.forward(bob)
alice2bob.expectMsgType[ChannelUpdate]
val update1a = channelUpdateListener.expectMsgType[LocalChannelUpdate]
val update1b = channelUpdateListener.expectMsgType[LocalChannelUpdate]
assert(update1a.channelUpdate.channelFlags.isEnabled)
assert(update1b.channelUpdate.channelFlags.isEnabled)
val (_, htlc1) = addHtlc(10000 msat, alice, bob, alice2bob, bob2alice, sender.ref)
sender.expectMsgType[RES_SUCCESS[CMD_ADD_HTLC]]
val (_, htlc2) = addHtlc(10000 msat, alice, bob, alice2bob, bob2alice, sender.ref)

View file

@ -29,8 +29,8 @@ import fr.acinq.eclair.blockchain.{CurrentBlockHeight, CurrentFeerates}
import fr.acinq.eclair.channel._
import fr.acinq.eclair.channel.fsm.Channel
import fr.acinq.eclair.channel.publish.TxPublisher.{PublishFinalTx, PublishReplaceableTx, PublishTx}
import fr.acinq.eclair.channel.states.{ChannelStateTestsBase, ChannelStateTestsTags}
import fr.acinq.eclair.channel.states.ChannelStateTestsBase.PimpTestFSM
import fr.acinq.eclair.channel.states.{ChannelStateTestsBase, ChannelStateTestsTags}
import fr.acinq.eclair.transactions.Transactions.{ClaimHtlcTimeoutTx, HtlcSuccessTx}
import fr.acinq.eclair.wire.protocol._
import fr.acinq.eclair.{BlockHeight, CltvExpiry, CltvExpiryDelta, MilliSatoshiLong, TestConstants, TestKitBaseClass, TestUtils, randomBytes32}
@ -403,7 +403,6 @@ class OfflineStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
alice2bob.forward(bob, reestablishA)
bob2alice.expectMsgType[RevokeAndAck]
bob2alice.expectMsgType[CommitSig]
bob2blockchain.expectMsgType[WatchFundingDeeplyBuried]
bob2alice.expectNoMessage(100 millis)
bob2blockchain.expectNoMessage(100 millis)
@ -699,7 +698,7 @@ class OfflineStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
if (shouldClose) {
assert(alice2blockchain.expectMsgType[PublishFinalTx].tx.txid == aliceCommitTx.txid)
} else {
alice2blockchain.expectNoMessage()
alice2blockchain.expectNoMessage(100 millis)
}
}
@ -723,8 +722,8 @@ class OfflineStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
// this time Alice will ignore feerate changes for the offline channel
alice.setFeerates(networkFeerates)
alice ! CurrentFeerates.BitcoinCore(networkFeerates)
alice2blockchain.expectNoMessage()
alice2bob.expectNoMessage()
alice2blockchain.expectNoMessage(100 millis)
alice2bob.expectNoMessage(100 millis)
}
def testUpdateFeeOnReconnect(f: FixtureParam, shouldUpdateFee: Boolean): Unit = {
@ -740,8 +739,8 @@ class OfflineStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
// Alice ignores feerate changes while offline
alice.setFeerates(networkFeerates)
alice ! CurrentFeerates.BitcoinCore(networkFeerates)
alice2blockchain.expectNoMessage()
alice2bob.expectNoMessage()
alice2blockchain.expectNoMessage(100 millis)
alice2bob.expectNoMessage(100 millis)
// then we reconnect them; Alice should send the feerate changes to Bob
reconnect(alice, bob, alice2bob, bob2alice)
@ -810,7 +809,7 @@ class OfflineStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
if (shouldClose) {
assert(bob2blockchain.expectMsgType[PublishFinalTx].tx.txid == bobCommitTx.txid)
} else {
bob2blockchain.expectNoMessage()
bob2blockchain.expectNoMessage(100 millis)
}
}
@ -834,9 +833,8 @@ class OfflineStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
// alice and bob resend their channel update at reconnection (unannounced channel)
alice2bob.expectMsgType[ChannelUpdate]
bob2alice.expectMsgType[ChannelUpdate]
alice2bob.expectNoMessage()
bob2alice.expectNoMessage()
alice2bob.expectNoMessage(100 millis)
bob2alice.expectNoMessage(100 millis)
// we make the peers exchange a few messages
addHtlc(250000000 msat, alice, bob, alice2bob, bob2alice)
@ -855,15 +853,8 @@ class OfflineStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
// alice and bob resend their channel update at reconnection (unannounced channel)
alice2bob.expectMsgType[ChannelUpdate]
bob2alice.expectMsgType[ChannelUpdate]
alice2bob.expectNoMessage()
bob2alice.expectNoMessage()
// funding tx gets 6 confirmations
alice ! WatchFundingDeeplyBuriedTriggered(BlockHeight(400000), 42, null)
bob ! WatchFundingDeeplyBuriedTriggered(BlockHeight(400000), 42, null)
// channel is private so there is no announcement sigs
// we use aliases so there is no need to resend a channel_update
alice2bob.expectNoMessage(100 millis)
bob2alice.expectNoMessage(100 millis)
// we get disconnected again
disconnect(alice, bob)

View file

@ -32,7 +32,7 @@ import fr.acinq.eclair.payment._
import fr.acinq.eclair.payment.relay.Relayer._
import fr.acinq.eclair.payment.send.SpontaneousRecipient
import fr.acinq.eclair.transactions.Transactions.ClaimLocalAnchorOutputTx
import fr.acinq.eclair.wire.protocol.{AnnouncementSignatures, ClosingSigned, CommitSig, Error, FailureMessageCodecs, FailureReason, PermanentChannelFailure, RevokeAndAck, Shutdown, UpdateAddHtlc, UpdateFailHtlc, UpdateFailMalformedHtlc, UpdateFee, UpdateFulfillHtlc}
import fr.acinq.eclair.wire.protocol.{AnnouncementSignatures, ChannelUpdate, ClosingSigned, CommitSig, Error, FailureMessageCodecs, FailureReason, PermanentChannelFailure, RevokeAndAck, Shutdown, UpdateAddHtlc, UpdateFailHtlc, UpdateFailMalformedHtlc, UpdateFee, UpdateFulfillHtlc}
import fr.acinq.eclair.{BlockHeight, CltvExpiry, CltvExpiryDelta, MilliSatoshiLong, TestConstants, TestKitBaseClass, randomBytes32}
import org.scalatest.funsuite.FixtureAnyFunSuiteLike
import org.scalatest.{Outcome, Tag}
@ -102,23 +102,23 @@ class ShutdownStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wit
}
}
test("emit disabled channel update", Tag(ChannelStateTestsTags.ChannelsPublic)) { () =>
test("emit disabled channel update", Tag(ChannelStateTestsTags.ChannelsPublic), Tag(ChannelStateTestsTags.DoNotInterceptGossip)) { () =>
val setup = init()
import setup._
within(30 seconds) {
reachNormal(setup, Set(ChannelStateTestsTags.ChannelsPublic))
reachNormal(setup, Set(ChannelStateTestsTags.ChannelsPublic, ChannelStateTestsTags.DoNotInterceptGossip))
val aliceListener = TestProbe()
systemA.eventStream.subscribe(aliceListener.ref, classOf[LocalChannelUpdate])
val bobListener = TestProbe()
systemB.eventStream.subscribe(bobListener.ref, classOf[LocalChannelUpdate])
alice ! WatchFundingDeeplyBuriedTriggered(BlockHeight(400_000), 42, null)
alice2bob.expectMsgType[AnnouncementSignatures]
alice2bob.forward(bob)
bob ! WatchFundingDeeplyBuriedTriggered(BlockHeight(400_000), 42, null)
alice2bob.expectMsgType[ChannelUpdate]
bob2alice.expectMsgType[AnnouncementSignatures]
bob2alice.forward(alice)
bob2alice.expectMsgType[ChannelUpdate]
assert(aliceListener.expectMsgType[LocalChannelUpdate].channelUpdate.channelFlags.isEnabled)
assert(bobListener.expectMsgType[LocalChannelUpdate].channelUpdate.channelFlags.isEnabled)

View file

@ -29,8 +29,8 @@ import fr.acinq.eclair.channel.states.{ChannelStateTestsBase, ChannelStateTestsT
import fr.acinq.eclair.transactions.Transactions
import fr.acinq.eclair.transactions.Transactions.ZeroFeeHtlcTxAnchorOutputsCommitmentFormat
import fr.acinq.eclair.wire.protocol.ClosingSignedTlv.FeeRange
import fr.acinq.eclair.wire.protocol.{AnnouncementSignatures, ClosingSigned, Error, Shutdown, TlvStream, Warning}
import fr.acinq.eclair.{BlockHeight, CltvExpiry, Features, MilliSatoshiLong, TestConstants, TestKitBaseClass, randomBytes32}
import fr.acinq.eclair.wire.protocol.{AnnouncementSignatures, ChannelUpdate, ClosingSigned, Error, Shutdown, TlvStream, Warning}
import fr.acinq.eclair.{CltvExpiry, Features, MilliSatoshiLong, TestConstants, TestKitBaseClass, randomBytes32}
import org.scalatest.funsuite.FixtureAnyFunSuiteLike
import org.scalatest.{Outcome, Tag}
@ -89,20 +89,20 @@ class NegotiatingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
def buildFeerates(feerate: FeeratePerKw, minFeerate: FeeratePerKw = FeeratePerKw(250 sat)): FeeratesPerKw =
FeeratesPerKw.single(feerate).copy(minimum = minFeerate, slow = minFeerate)
test("emit disabled channel update", Tag(ChannelStateTestsTags.ChannelsPublic)) { f =>
test("emit disabled channel update", Tag(ChannelStateTestsTags.ChannelsPublic), Tag(ChannelStateTestsTags.DoNotInterceptGossip)) { f =>
import f._
val aliceListener = TestProbe()
systemA.eventStream.subscribe(aliceListener.ref, classOf[LocalChannelUpdate])
val bobListener = TestProbe()
systemB.eventStream.subscribe(bobListener.ref, classOf[LocalChannelUpdate])
alice ! WatchFundingDeeplyBuriedTriggered(BlockHeight(400_000), 42, null)
alice2bob.expectMsgType[AnnouncementSignatures]
alice2bob.forward(bob)
bob ! WatchFundingDeeplyBuriedTriggered(BlockHeight(400_000), 42, null)
alice2bob.expectMsgType[ChannelUpdate]
bob2alice.expectMsgType[AnnouncementSignatures]
bob2alice.forward(alice)
bob2alice.expectMsgType[ChannelUpdate]
assert(aliceListener.expectMsgType[LocalChannelUpdate].channelUpdate.channelFlags.isEnabled)
assert(bobListener.expectMsgType[LocalChannelUpdate].channelUpdate.channelFlags.isEnabled)

View file

@ -104,6 +104,14 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
} else {
within(30 seconds) {
reachNormal(setup, test.tags)
if (test.tags.contains(ChannelStateTestsTags.ChannelsPublic) && test.tags.contains(ChannelStateTestsTags.DoNotInterceptGossip)) {
alice2bob.expectMsgType[AnnouncementSignatures]
alice2bob.forward(bob)
alice2bob.expectMsgType[ChannelUpdate]
bob2alice.expectMsgType[AnnouncementSignatures]
bob2alice.forward(alice)
bob2alice.expectMsgType[ChannelUpdate]
}
alice.underlying.system.eventStream.subscribe(txListener.ref, classOf[TransactionPublished])
alice.underlying.system.eventStream.subscribe(txListener.ref, classOf[TransactionConfirmed])
bob.underlying.system.eventStream.subscribe(txListener.ref, classOf[TransactionPublished])
@ -349,24 +357,16 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
assert(alice.stateData == initialState) // this was a no-op
}
test("recv WatchFundingSpentTriggered (local commit, public channel)", Tag(ChannelStateTestsTags.ChannelsPublic)) { f =>
test("recv WatchFundingSpentTriggered (local commit, public channel)", Tag(ChannelStateTestsTags.ChannelsPublic), Tag(ChannelStateTestsTags.DoNotInterceptGossip)) { f =>
import f._
val listener = TestProbe()
systemA.eventStream.subscribe(listener.ref, classOf[LocalChannelUpdate])
alice ! WatchFundingDeeplyBuriedTriggered(BlockHeight(400_000), 42, null)
alice2bob.expectMsgType[AnnouncementSignatures]
alice2bob.forward(bob)
bob ! WatchFundingDeeplyBuriedTriggered(BlockHeight(400_000), 42, null)
bob2alice.expectMsgType[AnnouncementSignatures]
bob2alice.forward(alice)
assert(listener.expectMsgType[LocalChannelUpdate].channelUpdate.channelFlags.isEnabled)
// an error occurs and alice publishes her commit tx
localClose(alice, alice2blockchain)
// she notifies the network that the channel shouldn't be used anymore
assert(!listener.expectMsgType[LocalChannelUpdate].channelUpdate.channelFlags.isEnabled)
inside(listener.expectMsgType[LocalChannelUpdate]) { u => assert(!u.channelUpdate.channelFlags.isEnabled) }
}
test("recv WatchOutputSpentTriggered") { f =>
@ -838,25 +838,17 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
assert(txPublished.miningFee > 0.sat) // alice is funder, she pays the fee for the remote commit
}
test("recv WatchFundingSpentTriggered (remote commit, public channel)", Tag(ChannelStateTestsTags.ChannelsPublic)) { f =>
test("recv WatchFundingSpentTriggered (remote commit, public channel)", Tag(ChannelStateTestsTags.ChannelsPublic), Tag(ChannelStateTestsTags.DoNotInterceptGossip)) { f =>
import f._
val listener = TestProbe()
systemA.eventStream.subscribe(listener.ref, classOf[LocalChannelUpdate])
alice ! WatchFundingDeeplyBuriedTriggered(BlockHeight(400_000), 42, null)
alice2bob.expectMsgType[AnnouncementSignatures]
alice2bob.forward(bob)
bob ! WatchFundingDeeplyBuriedTriggered(BlockHeight(400_000), 42, null)
bob2alice.expectMsgType[AnnouncementSignatures]
bob2alice.forward(alice)
assert(listener.expectMsgType[LocalChannelUpdate].channelUpdate.channelFlags.isEnabled)
// bob publishes his commit tx
val bobCommitTx = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localCommit.commitTxAndRemoteSig.commitTx.tx
remoteClose(bobCommitTx, alice, alice2blockchain)
// alice notifies the network that the channel shouldn't be used anymore
assert(!listener.expectMsgType[LocalChannelUpdate].channelUpdate.channelFlags.isEnabled)
inside(listener.expectMsgType[LocalChannelUpdate]) { u => assert(!u.channelUpdate.channelFlags.isEnabled) }
}
test("recv CMD_BUMP_FORCE_CLOSE_FEE (remote commit)", Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>

View file

@ -20,7 +20,6 @@ import com.softwaremill.quicklens._
import fr.acinq.bitcoin.scalacompat.ByteVector32
import fr.acinq.bitcoin.scalacompat.Crypto.{PrivateKey, PublicKey}
import fr.acinq.eclair.TestDatabases.{TestPgDatabases, TestSqliteDatabases, migrationCheck}
import fr.acinq.eclair.channel.RealScidStatus
import fr.acinq.eclair.db.ChannelsDbSpec.{getPgTimestamp, getTimestamp, testCases}
import fr.acinq.eclair.db.DbEventHandler.ChannelEvent
import fr.acinq.eclair.db.jdbc.JdbcUtils.using
@ -61,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).setTo(RealScidStatus.Final(RealShortChannelId(189371)))
val channel2b = channel2a.modify(_.shortIds.real_opt).setTo(Some(RealShortChannelId(189371)))
val commitNumber = 42
val paymentHash1 = ByteVector32.Zeroes

View file

@ -17,7 +17,7 @@ class TwoNodesIntegrationSpec extends FixtureSpec with IntegrationPatience {
type FixtureParam = TwoNodesFixture
import fr.acinq.eclair.integration.basic.fixtures.MinimalNodeFixture.{confirmChannel, confirmChannelDeep, connect, getChannelData, getChannelState, getRouterData, knownFundingTxs, nodeParamsFor, openChannel, sendSuccessfulPayment, watcherAutopilot}
import fr.acinq.eclair.integration.basic.fixtures.MinimalNodeFixture.{confirmChannel, connect, getChannelData, getRouterData, knownFundingTxs, nodeParamsFor, openChannel, sendSuccessfulPayment, watcherAutopilot}
override def createFixture(testData: TestData): FixtureParam = {
// seeds have been chosen so that node ids start with 02aaaa for alice, 02bbbb for bob, etc.
@ -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(getChannelState(alice, channelId) == NORMAL)
assert(getChannelState(bob, channelId) == NORMAL)
assert(getChannelData(alice, channelId).asInstanceOf[DATA_NORMAL].shortIds.real_opt.nonEmpty)
assert(getChannelData(bob, channelId).asInstanceOf[DATA_NORMAL].shortIds.real_opt.nonEmpty)
}
test("open multiple channels alice-bob") { f =>
@ -64,30 +64,8 @@ class TwoNodesIntegrationSpec extends FixtureSpec with IntegrationPatience {
connect(alice, bob)
val channelId = openChannel(alice, bob, 100_000 sat).channelId
eventually {
assert(getChannelState(alice, channelId) == NORMAL)
assert(getChannelState(bob, channelId) == NORMAL)
}
}
test("open a channel alice-bob and confirm deeply") { f =>
import f._
connect(alice, bob)
val channelId = openChannel(alice, bob, 100_000 sat).channelId
confirmChannel(alice, bob, channelId, BlockHeight(420_000), 21)
confirmChannelDeep(alice, bob, channelId, BlockHeight(420_000), 21)
assert(getChannelData(alice, channelId).asInstanceOf[DATA_NORMAL].shortIds.real.isInstanceOf[RealScidStatus.Final])
assert(getChannelData(bob, channelId).asInstanceOf[DATA_NORMAL].shortIds.real.isInstanceOf[RealScidStatus.Final])
}
test("open a channel alice-bob and confirm deeply (autoconfirm)") { f =>
import f._
alice.watcher.setAutoPilot(watcherAutopilot(knownFundingTxs(alice)))
bob.watcher.setAutoPilot(watcherAutopilot(knownFundingTxs(alice)))
connect(alice, bob)
val channelId = openChannel(alice, bob, 100_000 sat).channelId
eventually {
assert(getChannelData(alice, channelId).asInstanceOf[DATA_NORMAL].shortIds.real.isInstanceOf[RealScidStatus.Final])
assert(getChannelData(bob, channelId).asInstanceOf[DATA_NORMAL].shortIds.real.isInstanceOf[RealScidStatus.Final])
assert(getChannelData(alice, channelId).asInstanceOf[DATA_NORMAL].shortIds.real_opt.nonEmpty)
assert(getChannelData(bob, channelId).asInstanceOf[DATA_NORMAL].shortIds.real_opt.nonEmpty)
}
}

View file

@ -2,7 +2,7 @@ package fr.acinq.eclair.integration.basic.channel
import fr.acinq.bitcoin.scalacompat.{ByteVector32, SatoshiLong}
import fr.acinq.eclair.channel.states.ChannelStateTestsTags
import fr.acinq.eclair.channel.{DATA_NORMAL, RES_SPLICE, RealScidStatus}
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.integration.basic.fixtures.composite.ThreeNodesFixture
@ -52,11 +52,11 @@ class GossipIntegrationSpec extends ThreeNodesIntegrationSpec {
// channels confirm deeply
eventually {
assert(getChannelData(alice, channelId_ab).asInstanceOf[DATA_NORMAL].shortIds.real.isInstanceOf[RealScidStatus.Final])
assert(getChannelData(bob, channelId_bc).asInstanceOf[DATA_NORMAL].shortIds.real.isInstanceOf[RealScidStatus.Final])
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.asInstanceOf[RealScidStatus.Final].realScid
val scid_bc = getChannelData(bob, channelId_bc).asInstanceOf[DATA_NORMAL].shortIds.real.asInstanceOf[RealScidStatus.Final].realScid
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
@ -67,8 +67,8 @@ class GossipIntegrationSpec extends ThreeNodesIntegrationSpec {
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.toOption.get == channelData_bob1.shortIds.real.toOption.get)
val scid_ab1 = getChannelData(alice, channelId_ab).asInstanceOf[DATA_NORMAL].shortIds.real.asInstanceOf[RealScidStatus.Final].realScid
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)

View file

@ -12,7 +12,7 @@ import fr.acinq.bitcoin.scalacompat.{Block, ByteVector32, OutPoint, Satoshi, Sat
import fr.acinq.eclair.ShortChannelId.txIndex
import fr.acinq.eclair.blockchain.SingleKeyOnChainWallet
import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher
import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher.{WatchFundingConfirmed, WatchFundingConfirmedTriggered, WatchFundingDeeplyBuried, WatchFundingDeeplyBuriedTriggered}
import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher.{WatchFundingConfirmed, WatchFundingConfirmedTriggered}
import fr.acinq.eclair.blockchain.fee.{FeeratePerKw, FeeratesPerKw}
import fr.acinq.eclair.channel._
import fr.acinq.eclair.channel.fsm.Channel
@ -30,7 +30,7 @@ import fr.acinq.eclair.payment.relay.{ChannelRelayer, PostRestartHtlcCleaner, Re
import fr.acinq.eclair.payment.send.PaymentInitiator
import fr.acinq.eclair.router.Router
import fr.acinq.eclair.wire.protocol.IPAddress
import fr.acinq.eclair.{BlockHeight, MilliSatoshi, MilliSatoshiLong, NodeParams, SubscriptionsComplete, TestBitcoinCoreClient, TestDatabases}
import fr.acinq.eclair.{BlockHeight, MilliSatoshi, MilliSatoshiLong, NodeParams, RealShortChannelId, SubscriptionsComplete, TestBitcoinCoreClient, TestDatabases}
import org.scalatest.concurrent.{Eventually, IntegrationPatience}
import org.scalatest.{Assertions, EitherValues}
@ -193,7 +193,7 @@ object MinimalNodeFixture extends Assertions with Eventually with IntegrationPat
sender.expectMsgType[CommandResponse[CMD_SPLICE]]
}
def confirmChannel(node1: MinimalNodeFixture, node2: MinimalNodeFixture, channelId: ByteVector32, blockHeight: BlockHeight, txIndex: Int)(implicit system: ActorSystem): Option[RealScidStatus.Temporary] = {
def confirmChannel(node1: MinimalNodeFixture, node2: MinimalNodeFixture, channelId: ByteVector32, blockHeight: BlockHeight, txIndex: Int)(implicit system: ActorSystem): RealShortChannelId = {
val fundingTx = getChannelData(node1, channelId) match {
case d: DATA_WAIT_FOR_DUAL_FUNDING_SIGNED => d.signingSession.fundingTx.tx.buildUnsignedTx()
case d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => d.latestFundingTx.sharedTx.tx.buildUnsignedTx()
@ -213,30 +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.asInstanceOf[RealScidStatus.Temporary]
val realScid2 = data2After.shortIds.real.asInstanceOf[RealScidStatus.Temporary]
assert(realScid1 == realScid2)
Some(realScid1)
}
def confirmChannelDeep(node1: MinimalNodeFixture, node2: MinimalNodeFixture, channelId: ByteVector32, blockHeight: BlockHeight, txIndex: Int)(implicit system: ActorSystem): RealScidStatus.Final = {
assert(getChannelState(node1, channelId) == NORMAL)
val data1Before = getChannelData(node1, channelId).asInstanceOf[DATA_NORMAL]
val fundingTx = data1Before.commitments.latest.localFundingStatus.signedTx_opt.get
val watch1 = node1.watcher.fishForMessage() { case w: WatchFundingDeeplyBuried if w.txId == fundingTx.txid => true; case _ => false }.asInstanceOf[WatchFundingDeeplyBuried]
val watch2 = node2.watcher.fishForMessage() { case w: WatchFundingDeeplyBuried if w.txId == fundingTx.txid => true; case _ => false }.asInstanceOf[WatchFundingDeeplyBuried]
watch1.replyTo ! WatchFundingDeeplyBuriedTriggered(blockHeight, txIndex, fundingTx)
watch2.replyTo ! WatchFundingDeeplyBuriedTriggered(blockHeight, txIndex, fundingTx)
waitReady(node1, channelId)
waitReady(node2, channelId)
val data1After = getChannelData(node1, channelId).asInstanceOf[DATA_NORMAL]
val data2After = getChannelData(node2, channelId).asInstanceOf[DATA_NORMAL]
val realScid1 = data1After.shortIds.real.asInstanceOf[RealScidStatus.Final]
val realScid2 = data2After.shortIds.real.asInstanceOf[RealScidStatus.Final]
val realScid1 = data1After.shortIds.real_opt.get
val realScid2 = data2After.shortIds.real_opt.get
assert(realScid1 == realScid2)
realScid1
}
@ -279,9 +257,9 @@ object MinimalNodeFixture extends Assertions with Eventually with IntegrationPat
* An autopilot method for the watcher, that handled funding confirmation requests from the channel and channel
* validation requests from the router
*/
def watcherAutopilot(knownFundingTxs: () => Iterable[Transaction], confirm: Boolean = true, deepConfirm: Boolean = true)(implicit system: ActorSystem): TestActor.AutoPilot = {
def watcherAutopilot(knownFundingTxs: () => Iterable[Transaction], confirm: Boolean = true)(implicit system: ActorSystem): TestActor.AutoPilot = {
// we forward messages to an actor to emulate a stateful autopilot
val fundingTxWatcher = system.spawnAnonymous(FundingTxWatcher(knownFundingTxs, confirm = confirm, deepConfirm = deepConfirm))
val fundingTxWatcher = system.spawnAnonymous(FundingTxWatcher(knownFundingTxs, confirm))
(_, msg) =>
msg match {
case msg: ZmqWatcher.Command => fundingTxWatcher ! msg
@ -293,7 +271,7 @@ object MinimalNodeFixture extends Assertions with Eventually with IntegrationPat
// When opening a channel, only one of the two nodes publishes the funding transaction, which creates a race when the
// other node sets a watch before that happens. We simply retry until the funding transaction is published.
private object FundingTxWatcher {
def apply(knownFundingTxs: () => Iterable[Transaction], confirm: Boolean, deepConfirm: Boolean): Behavior[ZmqWatcher.Command] = {
def apply(knownFundingTxs: () => Iterable[Transaction], confirm: Boolean): Behavior[ZmqWatcher.Command] = {
Behaviors.setup { _ =>
Behaviors.withTimers { timers =>
Behaviors.receiveMessagePartial {
@ -317,13 +295,6 @@ object MinimalNodeFixture extends Assertions with Eventually with IntegrationPat
case None => timers.startSingleTimer(watch, 10 millis)
}
Behaviors.same
case watch: ZmqWatcher.WatchFundingDeeplyBuried if deepConfirm =>
val (blockHeight, txIndex) = deterministicTxCoordinates(watch.txId)
knownFundingTxs().find(_.txid == watch.txId) match {
case Some(fundingTx) => watch.replyTo ! ZmqWatcher.WatchFundingDeeplyBuriedTriggered(blockHeight, txIndex, fundingTx)
case None => timers.startSingleTimer(watch, 10 millis)
}
Behaviors.same
case watch: ZmqWatcher.WatchExternalChannelSpent =>
knownFundingTxs().find(_.txIn.exists(_.outPoint == OutPoint(watch.txId, watch.outputIndex))) match {
case Some(nextFundingTx) =>
@ -342,7 +313,7 @@ object MinimalNodeFixture extends Assertions with Eventually with IntegrationPat
* We don't use a blockchain in this test setup, but we want to simulate channels being confirmed.
* We choose a block height and transaction index at which the channel confirms deterministically from its txid.
*/
def deterministicTxCoordinates(txId: TxId): (BlockHeight, Int) = {
private def deterministicTxCoordinates(txId: TxId): (BlockHeight, Int) = {
val blockHeight = txId.value.take(3).toInt(signed = false)
val txIndex = txId.value.takeRight(2).toInt(signed = false)
(BlockHeight(blockHeight), txIndex)

View file

@ -25,9 +25,9 @@ import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
import fr.acinq.bitcoin.scalacompat.{ByteVector32, SatoshiLong}
import fr.acinq.eclair.FeatureSupport.Optional
import fr.acinq.eclair.Features.{KeySend, RouteBlinding}
import fr.acinq.eclair.channel.{DATA_NORMAL, RealScidStatus}
import fr.acinq.eclair.channel.{DATA_NORMAL, NORMAL}
import fr.acinq.eclair.integration.basic.fixtures.MinimalNodeFixture
import fr.acinq.eclair.integration.basic.fixtures.MinimalNodeFixture.{connect, getChannelData, getPeerChannels, getRouterData, knownFundingTxs, nodeParamsFor, openChannel, sendPayment, watcherAutopilot}
import fr.acinq.eclair.integration.basic.fixtures.MinimalNodeFixture.{connect, getChannelState, getPeerChannels, getRouterData, knownFundingTxs, nodeParamsFor, openChannel, sendPayment, watcherAutopilot}
import fr.acinq.eclair.integration.basic.fixtures.composite.ThreeNodesFixture
import fr.acinq.eclair.message.OnionMessages
import fr.acinq.eclair.message.OnionMessages.{IntermediateNode, Recipient, buildRoute}
@ -117,8 +117,8 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience {
import f._
eventually {
assert(getChannelData(alice, channelId).asInstanceOf[DATA_NORMAL].shortIds.real.isInstanceOf[RealScidStatus.Final])
assert(getChannelData(bob, channelId).asInstanceOf[DATA_NORMAL].shortIds.real.isInstanceOf[RealScidStatus.Final])
assert(getChannelState(alice, channelId) == NORMAL)
assert(getChannelState(bob, channelId) == NORMAL)
}
}
@ -126,8 +126,8 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience {
import f._
eventually {
assert(getChannelData(bob, channelId).asInstanceOf[DATA_NORMAL].shortIds.real.isInstanceOf[RealScidStatus.Final])
assert(getChannelData(carol, channelId).asInstanceOf[DATA_NORMAL].shortIds.real.isInstanceOf[RealScidStatus.Final])
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))
}
@ -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.toOption.get
val scid_bc = getPeerChannels(bob, carol.nodeId).head.data.asInstanceOf[DATA_NORMAL].shortIds.real_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

@ -46,8 +46,8 @@ class ZeroConfActivationSpec extends FixtureSpec with IntegrationPatience {
private def createChannel(f: FixtureParam, channelType_opt: Option[SupportedChannelType] = None): ByteVector32 = {
import f._
alice.watcher.setAutoPilot(watcherAutopilot(knownFundingTxs(alice, bob), confirm = false, deepConfirm = false))
bob.watcher.setAutoPilot(watcherAutopilot(knownFundingTxs(alice, bob), confirm = false, deepConfirm = false))
alice.watcher.setAutoPilot(watcherAutopilot(knownFundingTxs(alice, bob), confirm = false))
bob.watcher.setAutoPilot(watcherAutopilot(knownFundingTxs(alice, bob), confirm = false))
connect(alice, bob)
openChannel(alice, bob, 100_000 sat, channelType_opt).channelId

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, RealScidStatus}
import fr.acinq.eclair.channel.DATA_NORMAL
import fr.acinq.eclair.crypto.Sphinx
import fr.acinq.eclair.integration.basic.fixtures.composite.ThreeNodesFixture
import fr.acinq.eclair.payment.Bolt11Invoice.ExtraHop
@ -51,12 +51,12 @@ class ZeroConfAliasIntegrationSpec extends FixtureSpec with IntegrationPatience
fixture.cleanup()
}
private def createChannels(f: FixtureParam)(deepConfirm: Boolean): (ByteVector32, ByteVector32) = {
private def createChannels(f: FixtureParam, confirm: Boolean): (ByteVector32, ByteVector32) = {
import f._
alice.watcher.setAutoPilot(watcherAutopilot(knownFundingTxs(alice, bob, carol), deepConfirm = deepConfirm))
bob.watcher.setAutoPilot(watcherAutopilot(knownFundingTxs(alice, bob, carol), deepConfirm = deepConfirm))
carol.watcher.setAutoPilot(watcherAutopilot(knownFundingTxs(alice, bob, carol), deepConfirm = deepConfirm))
alice.watcher.setAutoPilot(watcherAutopilot(knownFundingTxs(alice, bob, carol), confirm = confirm))
bob.watcher.setAutoPilot(watcherAutopilot(knownFundingTxs(alice, bob, carol), confirm = confirm))
carol.watcher.setAutoPilot(watcherAutopilot(knownFundingTxs(alice, bob, carol), confirm = confirm))
connect(alice, bob)
connect(bob, carol)
@ -110,7 +110,7 @@ class ZeroConfAliasIntegrationSpec extends FixtureSpec with IntegrationPatience
}
private def internalTest(f: FixtureParam,
deepConfirm: Boolean,
confirm: Boolean,
bcPublic: Boolean,
bcZeroConf: Boolean,
bcScidAlias: Boolean,
@ -119,23 +119,21 @@ class ZeroConfAliasIntegrationSpec extends FixtureSpec with IntegrationPatience
paymentWithRealScidHint_opt: Option[Either[Either[Throwable, FailureMessage], Ok.type]]): Unit = {
import f._
val (_, channelId_bc) = createChannels(f)(deepConfirm = deepConfirm)
val (_, channelId_bc) = createChannels(f, confirm)
eventually {
assert(getChannelData(bob, channelId_bc).asInstanceOf[DATA_NORMAL].commitments.params.channelFeatures.features.contains(ZeroConf) == bcZeroConf)
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 (deepConfirm) {
assert(getChannelData(bob, channelId_bc).asInstanceOf[DATA_NORMAL].shortIds.real.isInstanceOf[RealScidStatus.Final])
} else if (bcZeroConf) {
assert(getChannelData(bob, channelId_bc).asInstanceOf[DATA_NORMAL].shortIds.real == RealScidStatus.Unknown)
if (confirm) {
assert(getChannelData(bob, channelId_bc).asInstanceOf[DATA_NORMAL].shortIds.real_opt.nonEmpty)
} else {
assert(getChannelData(bob, channelId_bc).asInstanceOf[DATA_NORMAL].shortIds.real.isInstanceOf[RealScidStatus.Temporary])
assert(getChannelData(bob, channelId_bc).asInstanceOf[DATA_NORMAL].shortIds.real_opt.isEmpty)
}
}
eventually {
if (bcPublic && deepConfirm) {
if (bcPublic && confirm) {
// if channel bob-carol is public, we wait for alice to learn about it
val data = getRouterData(alice)
assert(data.channels.size == 2)
@ -155,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.toOption.value))
sendPaymentAliceToCarol(f, paymentWithRealScidHint, useHint = true, overrideHintScid_opt = Some(getChannelData(bob, channelId_bc).asInstanceOf[DATA_NORMAL].shortIds.real_opt.value))
}
}
eventually {
if (deepConfirm) {
if (confirm) {
val scidsBob = getChannelData(bob, channelId_bc).asInstanceOf[DATA_NORMAL].shortIds
val scid_bc = if (bcPublic) scidsBob.real.toOption.get else scidsBob.localAlias
val scid_bc = if (bcPublic) scidsBob.real_opt.get else scidsBob.localAlias
createSelfRouteCarol(f, scid_bc)
}
}
@ -170,7 +168,7 @@ class ZeroConfAliasIntegrationSpec extends FixtureSpec with IntegrationPatience
test("a->b->c (b-c private)") { f =>
internalTest(f,
deepConfirm = true,
confirm = true,
bcPublic = false,
bcZeroConf = false,
bcScidAlias = false,
@ -182,7 +180,7 @@ class ZeroConfAliasIntegrationSpec extends FixtureSpec with IntegrationPatience
test("a->b->c (b-c scid-alias private)", Tag(ScidAliasBobCarol)) { f =>
internalTest(f,
deepConfirm = true,
confirm = true,
bcPublic = false,
bcZeroConf = false,
bcScidAlias = true,
@ -194,7 +192,7 @@ class ZeroConfAliasIntegrationSpec extends FixtureSpec with IntegrationPatience
test("a->b->c (b-c zero-conf unconfirmed private)", Tag(ZeroConfBobCarol)) { f =>
internalTest(f,
deepConfirm = false,
confirm = false,
bcPublic = false,
bcZeroConf = true,
bcScidAlias = false,
@ -206,7 +204,7 @@ class ZeroConfAliasIntegrationSpec extends FixtureSpec with IntegrationPatience
test("a->b->c (b-c zero-conf deeply confirmed private)", Tag(ZeroConfBobCarol)) { f =>
internalTest(f,
deepConfirm = true,
confirm = true,
bcPublic = false,
bcZeroConf = true,
bcScidAlias = false,
@ -223,7 +221,7 @@ class ZeroConfAliasIntegrationSpec extends FixtureSpec with IntegrationPatience
test("a->b->c (b-c zero-conf scid-alias deeply confirmed private)", Tag(ZeroConfBobCarol), Tag(ScidAliasBobCarol)) { f =>
internalTest(f,
deepConfirm = true,
confirm = true,
bcPublic = false,
bcZeroConf = true,
bcScidAlias = true,
@ -235,7 +233,7 @@ class ZeroConfAliasIntegrationSpec extends FixtureSpec with IntegrationPatience
test("a->b->c (b-c zero-conf unconfirmed public)", Tag(ZeroConfBobCarol), Tag(PublicBobCarol)) { f =>
internalTest(f,
deepConfirm = false,
confirm = false,
bcPublic = true,
bcZeroConf = true,
bcScidAlias = false,
@ -247,7 +245,7 @@ class ZeroConfAliasIntegrationSpec extends FixtureSpec with IntegrationPatience
test("a->b->c (b-c zero-conf deeply confirmed public)", Tag(ZeroConfBobCarol), Tag(PublicBobCarol)) { f =>
internalTest(f,
deepConfirm = true,
confirm = true,
bcPublic = true,
bcZeroConf = true,
bcScidAlias = false,
@ -260,12 +258,12 @@ class ZeroConfAliasIntegrationSpec extends FixtureSpec with IntegrationPatience
test("temporary channel failures don't leak the real scid", Tag(ScidAliasBobCarol), Tag(ZeroConfBobCarol)) { f =>
import f._
val (_, channelId_bc) = createChannels(f)(deepConfirm = false)
val (_, channelId_bc) = createChannels(f, confirm = false)
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 == RealScidStatus.Unknown)
assert(getChannelData(bob, channelId_bc).asInstanceOf[DATA_NORMAL].shortIds.real_opt.isEmpty)
assert(getRouterData(bob).privateChannels.values.exists(_.nodeId2 == carol.nodeParams.nodeId))
}

View file

@ -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(RealScidStatus.Unknown, ShortChannelId.generateLocalAlias(), None)),
DATA_WAIT_FOR_CHANNEL_READY(commitments(peerOnWhitelistAtLimit, randomBytes32()), ShortIds(None, 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(RealScidStatus.Unknown, ShortChannelId.generateLocalAlias(), None)),
DATA_WAIT_FOR_CHANNEL_READY(commitments(peerAtLimit1, randomBytes32()), ShortIds(None, 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(RealScidStatus.Unknown, ShortChannelId.generateLocalAlias(), None)),
DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments(peerAtLimit2, randomBytes32()), ShortIds(None, 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(RealScidStatus.Unknown, ShortChannelId.generateLocalAlias(), None)),
DATA_NORMAL(commitments(peerBelowLimit2, randomBytes32()), ShortIds(RealScidStatus.Unknown, ShortChannelId.generateLocalAlias(), None), None, null, None, None, None, SpliceStatus.NoSplice),
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_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(RealScidStatus.Unknown, ShortChannelId.generateLocalAlias(), None)),
DATA_NORMAL(commitments(privatePeer2, randomBytes32()), ShortIds(RealScidStatus.Unknown, ShortChannelId.generateLocalAlias(), None), None, null, None, None, None, SpliceStatus.NoSplice),
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),
)
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(RealScidStatus.Unknown, ShortChannelId.generateLocalAlias(), None)),
DATA_WAIT_FOR_CHANNEL_READY(commitments(peerBelowLimit1, randomBytes32(), isOpener = true), ShortIds(None, 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(RealScidStatus.Unknown, ShortChannelId.generateLocalAlias(), None)),
DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments(peerAtLimit1, randomBytes32(), isOpener = true), ShortIds(None, ShortChannelId.generateLocalAlias(), None)),
)
val publicChannels = channelsOnWhitelistAtLimit ++ channelsAtLimit1 ++ channelsAtLimit2 ++ channelsBelowLimit1 ++ channelsBelowLimit2
val allChannels = publicChannels ++ privateChannels ++ initiatorChannels
@ -315,9 +315,9 @@ class PendingChannelsRateLimiterSpec extends ScalaTestWithActorTestKit(ConfigFac
import f._
val channels = Seq(
DATA_WAIT_FOR_CHANNEL_READY(commitments(randomKey().publicKey, randomBytes32()), ShortIds(RealScidStatus.Unknown, ShortChannelId.generateLocalAlias(), None)),
DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments(randomKey().publicKey, randomBytes32()), ShortIds(RealScidStatus.Unknown, ShortChannelId.generateLocalAlias(), None)),
DATA_NORMAL(commitments(randomKey().publicKey, randomBytes32()), ShortIds(RealScidStatus.Unknown, ShortChannelId.generateLocalAlias(), None), None, null, None, None, None, SpliceStatus.NoSplice),
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_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,7 +139,7 @@ class JsonSerializersSpec extends TestKitBaseClass with AnyFunSuiteLike with Mat
ShaChain.init,
Map.empty,
),
ShortIds(RealScidStatus.Unknown, Alias(42), None),
ShortIds(None, Alias(42), None),
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 +220,7 @@ class JsonSerializersSpec extends TestKitBaseClass with AnyFunSuiteLike with Mat
| "remotePerCommitmentSecrets": null,
| "originChannels": {}
| },
| "shortIds": { "real": { "status": "unknown" }, "localAlias": "0x2a" },
| "shortIds": { "localAlias": "0x2a" },
| "channelUpdate": {
| "signature": "345b2b05ec046ffe0c14d3b61838c79980713ad1cf8ae7a45c172ce90c9c0b9f345b2b05ec046ffe0c14d3b61838c79980713ad1cf8ae7a45c172ce90c9c0b9f",
| "chainHash": "06226e46111a0b59caaf126043eb5bbf28c34f3a5e332a1fc7b2b73cf188910f",
@ -449,12 +449,10 @@ class JsonSerializersSpec extends TestKitBaseClass with AnyFunSuiteLike with Mat
test("serialize short ids") {
val testCases = Map(
ShortIds(real = RealScidStatus.Unknown, localAlias = Alias(0x4455), remoteAlias_opt = Some(Alias(0x88888888L))) ->
"""{"real":{"status":"unknown"},"localAlias":"0x4455","remoteAlias":"0x88888888"}""",
ShortIds(real = RealScidStatus.Temporary(RealShortChannelId(BlockHeight(500000), 42, 1)), localAlias = Alias(0x4455), remoteAlias_opt = None) ->
"""{"real":{"status":"temporary","realScid":"500000x42x1"},"localAlias":"0x4455"}""",
ShortIds(real = RealScidStatus.Final(RealShortChannelId(BlockHeight(500000), 42, 1)), localAlias = Alias(0x4455), remoteAlias_opt = None) ->
"""{"real":{"status":"final","realScid":"500000x42x1"},"localAlias":"0x4455"}""",
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"}""",
)
for ((obj, json) <- testCases) {
JsonSerializers.serialization.write(obj)(JsonSerializers.formats) shouldBe json

View file

@ -165,7 +165,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a
// reorg happens
val realScid1AfterReorg = RealShortChannelId(111112)
val lcu2 = createLocalUpdate(channelId1).modify(_.shortIds.real).setTo(RealScidStatus.Final(realScid1AfterReorg))
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)
@ -577,7 +577,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a
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 = RealScidStatus.Final(shortChannelId), localAlias = ShortChannelId.generateLocalAlias(), remoteAlias_opt = None)
val shortIds = ShortIds(real_opt = Some(shortChannelId), localAlias = ShortChannelId.generateLocalAlias(), remoteAlias_opt = None)
LocalChannelUpdate(null, channelId, shortIds, remoteNodeId, None, update, commitments)
}
@ -761,8 +761,8 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a
import f._
val channelId_ab = randomBytes32()
val channelId_bc = randomBytes32()
val shortIds_ab = ShortIds(RealScidStatus.Final(RealShortChannelId(channelUpdate_ab.shortChannelId.toLong)), ShortChannelId.generateLocalAlias(), remoteAlias_opt = None)
val shortIds_bc = ShortIds(RealScidStatus.Final(RealShortChannelId(channelUpdate_bc.shortChannelId.toLong)), ShortChannelId.generateLocalAlias(), remoteAlias_opt = None)
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 a = PaymentPacketSpec.a
val sender = TestProbe[Relayer.OutgoingChannels]()
@ -858,12 +858,12 @@ object ChannelRelayerSpec {
def createShortIds(channelId: ByteVector32) = {
val realScid = channelIds.collectFirst { case (realScid: RealShortChannelId, cid) if cid == channelId => realScid }.get
val localAlias = channelIds.collectFirst { case (localAlias: Alias, cid) if cid == channelId => localAlias }.get
ShortIds(real = RealScidStatus.Final(realScid), localAlias, remoteAlias_opt = None)
ShortIds(real_opt = Some(realScid), 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.toOption.get)
val channelUpdateScid = channelUpdateScid_opt.getOrElse(shortIds.real_opt.get)
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,

View file

@ -166,7 +166,7 @@ class OnTheFlyFundingSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike {
val commitments = CommitmentsSpec.makeCommitments(500_000_000 msat, 500_000_000 msat, nodeParams.nodeId, remoteNodeId, announceChannel = false)
.modify(_.params.remoteParams.htlcMinimum).setTo(htlcMinimum)
.modify(_.changes.localChanges).setTo(localChanges)
DATA_NORMAL(commitments, ShortIds(RealScidStatus.Unknown, Alias(42), None), None, null, None, None, None, SpliceStatus.NoSplice)
DATA_NORMAL(commitments, ShortIds(None, Alias(42), None), None, null, None, None, None, SpliceStatus.NoSplice)
}
}

View file

@ -81,7 +81,7 @@ 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(RealScidStatus.Final(RealShortChannelId(channelUpdate_bc.shortChannelId.toLong)), ShortChannelId.generateLocalAlias(), remoteAlias_opt = None)
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)))
eventually(PatienceConfiguration.Timeout(30 seconds), PatienceConfiguration.Interval(1 second)) {
childActors.channelRelayer ! ChannelRelayer.GetOutgoingChannels(sender.ref.toClassic, GetOutgoingChannels())

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(RealScidStatus.Final(scid_ab), alias_ab, Some(alias_ba))
val scids_ag_private = ShortIds(RealScidStatus.Final(scid_ag_private), alias_ag_private, Some(alias_ga_private))
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 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,7 +167,7 @@ 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))
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, PeerRoutingMessage(peerConnection.ref, remoteNodeId, update_ga_private))
// watcher receives the get tx requests

View file

@ -2,16 +2,15 @@ package fr.acinq.eclair.router
import akka.actor.ActorSystem
import akka.testkit.{TestFSMRef, TestProbe}
import fr.acinq.bitcoin.scalacompat.Transaction
import fr.acinq.eclair.blockchain.CurrentBlockHeight
import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher.{WatchExternalChannelSpent, WatchExternalChannelSpentTriggered, WatchFundingDeeplyBuriedTriggered, WatchTxConfirmed, WatchTxConfirmedTriggered}
import fr.acinq.bitcoin.scalacompat.{SatoshiLong, Script, Transaction, TxOut}
import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher._
import fr.acinq.eclair.channel.states.{ChannelStateTestsBase, ChannelStateTestsTags}
import fr.acinq.eclair.channel.{CMD_CLOSE, DATA_NORMAL}
import fr.acinq.eclair.io.Peer.PeerRoutingMessage
import fr.acinq.eclair.router.Graph.GraphStructure.GraphEdge
import fr.acinq.eclair.router.Router._
import fr.acinq.eclair.wire.protocol.{AnnouncementSignatures, ChannelUpdate, Shutdown}
import fr.acinq.eclair.{BlockHeight, TestKitBaseClass}
import fr.acinq.eclair.{BlockHeight, TestKitBaseClass, randomKey}
import org.scalatest.OptionValues.convertOptionToValuable
import org.scalatest.funsuite.FixtureAnyFunSuiteLike
import org.scalatest.{Outcome, Tag}
@ -47,71 +46,81 @@ class ChannelRouterIntegrationSpec extends TestKitBaseClass with FixtureAnyFunSu
import f._
val (aliceNodeId, bobNodeId) = (channels.alice.underlyingActor.nodeParams.nodeId, channels.bob.underlyingActor.nodeParams.nodeId)
reachNormal(channels, testTags, interceptChannelUpdates = false)
val fundingTx = reachNormal(channels, testTags + ChannelStateTestsTags.DoNotInterceptGossip)
// the router learns about the local, still unannounced, channel
// The router learns about the local, still unannounced, channel.
awaitCond(router.stateData.privateChannels.size == 1)
// only alice's channel_update is known (NB : due to how node ids are constructed, 1 = alice and 2 = bob)
// Alice only has her local channel_update at that point (NB: due to how node ids are constructed, 1 = alice and 2 = bob).
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.toOption.isEmpty == f.testTags.contains(ChannelStateTestsTags.ZeroConf))
// 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)
// alice uses her alias for her internal channel update
// Alice uses her alias for her internal channel update.
val aliceInitialChannelUpdate = privateChannel.update_1_opt.value
assert(aliceInitialChannelUpdate.shortChannelId == privateChannel.shortIds.localAlias)
// alice and bob send their channel_updates using remote alias when they go to NORMAL state
// 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)) {
val annSigsA = channels.alice2bob.expectMsgType[AnnouncementSignatures]
val annSigsB = channels.bob2alice.expectMsgType[AnnouncementSignatures]
(Some(annSigsA), Some(annSigsB))
} else {
(None, None)
}
// In all cases, Alice and bob send their channel_updates using the remote alias when they go to NORMAL state.
val aliceChannelUpdate1 = channels.alice2bob.expectMsgType[ChannelUpdate]
val bobChannelUpdate1 = channels.bob2alice.expectMsgType[ChannelUpdate]
// alice's channel_update uses bob's alias, and vice versa
// 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)
// channel_updates are handled by the peer connection and sent to the router
// The channel_updates are handled by the peer connection and sent to the router.
val peerConnection = TestProbe()
router ! PeerRoutingMessage(peerConnection.ref, bobNodeId, bobChannelUpdate1)
// router processes bob's channel_update and now knows both channel updates
awaitCond {
privateChannel.update_1_opt.contains(aliceInitialChannelUpdate) && privateChannel.update_2_opt.contains(bobChannelUpdate1)
}
// The router processes bob's channel_update and now knows both channel updates.
awaitCond(privateChannel.update_1_opt.contains(aliceInitialChannelUpdate) && privateChannel.update_2_opt.contains(bobChannelUpdate1))
// there is nothing for the router to rebroadcast, channel is not announced
// There is nothing for the router to rebroadcast, because the channel is not announced yet.
assert(router.stateData.rebroadcast == Rebroadcast(Map.empty, Map.empty, Map.empty))
// router graph contains a single channel
// The router graph contains a single channel between Alice and Bob.
assert(router.stateData.graphWithBalances.graph.vertexSet() == Set(aliceNodeId, bobNodeId))
assert(router.stateData.graphWithBalances.graph.edgeSet().toSet == Set(GraphEdge(aliceInitialChannelUpdate, privateChannel), GraphEdge(bobChannelUpdate1, privateChannel)))
// The channel now confirms, if it hadn't confirmed already.
if (testTags.contains(ChannelStateTestsTags.ZeroConf)) {
channels.alice ! WatchFundingConfirmedTriggered(BlockHeight(400_000), 42, fundingTx)
channels.alice2blockchain.expectMsgType[WatchFundingSpent]
channels.bob ! WatchFundingConfirmedTriggered(BlockHeight(400_000), 42, fundingTx)
channels.bob2blockchain.expectMsgType[WatchFundingSpent]
}
if (testTags.contains(ChannelStateTestsTags.ChannelsPublic)) {
// this is a public channel
// funding tx reaches 6 blocks, announcements are exchanged
channels.alice ! WatchFundingDeeplyBuriedTriggered(BlockHeight(400000), 42, null)
channels.alice2bob.expectMsgType[AnnouncementSignatures]
channels.alice2bob.forward(channels.bob)
// The channel is public, so Alice and Bob exchange announcement signatures.
val annSigsA = annSigsA_opt.getOrElse(channels.alice2bob.expectMsgType[AnnouncementSignatures])
val annSigsB = annSigsB_opt.getOrElse(channels.bob2alice.expectMsgType[AnnouncementSignatures])
channels.alice2bob.forward(channels.bob, annSigsA)
channels.bob2alice.forward(channels.alice, annSigsB)
channels.bob ! WatchFundingDeeplyBuriedTriggered(BlockHeight(400000), 42, null)
channels.bob2alice.expectMsgType[AnnouncementSignatures]
channels.bob2alice.forward(channels.alice)
// the router learns about the announcement and channel graduates from private to public
awaitCond {
router.stateData.privateChannels.isEmpty && router.stateData.channels.size == 1
// The router learns about the announcement and the channel graduates from private to public.
awaitAssert {
assert(router.stateData.privateChannels.isEmpty)
assert(router.stateData.scid2PrivateChannels.isEmpty)
assert(router.stateData.channels.size == 1)
}
// router has cleaned up the scid mapping
assert(router.stateData.scid2PrivateChannels.isEmpty)
// alice and bob won't send their channel_update directly to each other because the channel has been announced
// 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
awaitCond {
channels.alice.stateData.asInstanceOf[DATA_NORMAL].channelAnnouncement.isDefined &&
channels.bob.stateData.asInstanceOf[DATA_NORMAL].channelAnnouncement.isDefined
awaitAssert {
assert(channels.alice.stateData.asInstanceOf[DATA_NORMAL].channelAnnouncement.isDefined)
assert(channels.bob.stateData.asInstanceOf[DATA_NORMAL].channelAnnouncement.isDefined)
}
val aliceChannelUpdate2 = channels.alice.stateData.asInstanceOf[DATA_NORMAL].channelUpdate
val bobChannelUpdate2 = channels.bob.stateData.asInstanceOf[DATA_NORMAL].channelUpdate
// this time, they use the real scid
// 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
assert(aliceAnn == bobAnn)
@ -120,25 +129,23 @@ class ChannelRouterIntegrationSpec extends TestKitBaseClass with FixtureAnyFunSu
assert(bobChannelUpdate2.shortChannelId == bobAnn.shortChannelId)
assert(!bobChannelUpdate2.dontForward)
// the router has already processed the new local channel update from alice which uses the real scid, and keeps bob's previous channel update
// The router has already processed the new local channel update from alice which uses the real scid, and keeps bob's previous channel update.
assert(publicChannel.update_1_opt.contains(aliceChannelUpdate2) && publicChannel.update_2_opt.contains(bobChannelUpdate1))
// the router prepares to rebroadcast the channel announcement, the local update which uses the real scid, and the first node announcement
// The router will rebroadcast the channel announcement, the local update which uses the real scid, and the first node announcement.
assert(router.stateData.rebroadcast == Rebroadcast(
channels = Map(aliceAnn -> Set[GossipOrigin](LocalGossip)),
updates = Map(aliceChannelUpdate2 -> Set[GossipOrigin](LocalGossip)),
nodes = Map(router.stateData.nodes.values.head -> Set[GossipOrigin](LocalGossip)))
)
// bob's channel_update reaches the router
// Bob's new channel_update (using the real scid) reaches the router.
router ! PeerRoutingMessage(peerConnection.ref, bobNodeId, bobChannelUpdate2)
// router processes bob's channel_update and now knows both channel updates with real scids
awaitCond {
publicChannel.update_1_opt.contains(aliceChannelUpdate2) && publicChannel.update_2_opt.contains(bobChannelUpdate2)
}
// The router processes bob's channel_update and now uses channel updates with real scids.
awaitCond(publicChannel.update_1_opt.contains(aliceChannelUpdate2) && publicChannel.update_2_opt.contains(bobChannelUpdate2))
// router is now ready to rebroadcast both channel updates
// The router is now ready to rebroadcast both channel updates.
assert(router.stateData.rebroadcast == Rebroadcast(
channels = Map(aliceAnn -> Set[GossipOrigin](LocalGossip)),
updates = Map(
@ -148,40 +155,39 @@ class ChannelRouterIntegrationSpec extends TestKitBaseClass with FixtureAnyFunSu
nodes = Map(router.stateData.nodes.values.head -> Set[GossipOrigin](LocalGossip)))
)
// router graph contains a single channel
// The router graph still contains a single channel, with public updates.
assert(router.stateData.graphWithBalances.graph.vertexSet() == Set(aliceNodeId, bobNodeId))
assert(router.stateData.graphWithBalances.graph.edgeSet().size == 2)
assert(router.stateData.graphWithBalances.graph.edgeSet().toSet == Set(GraphEdge(aliceChannelUpdate2, publicChannel), GraphEdge(bobChannelUpdate2, publicChannel)))
} else {
// this is a private channel
// funding tx reaches 6 blocks, no announcements are exchanged because the channel is private
channels.alice ! WatchFundingDeeplyBuriedTriggered(BlockHeight(400000), 42, null)
channels.bob ! WatchFundingDeeplyBuriedTriggered(BlockHeight(400000), 42, null)
// alice and bob won't send their channel_update directly to each other because they haven't changed
// This is a private channel: alice and bob won't send a new channel_update to each other, even though the channel
// is confirmed, because they will keep using the scid aliases.
channels.alice2bob.expectNoMessage(100 millis)
channels.bob2alice.expectNoMessage(100 millis)
assert(privateChannel.update_1_opt.get.dontForward)
assert(privateChannel.update_2_opt.get.dontForward)
// router graph contains a single channel
// The router graph still contains a single private channel.
assert(router.stateData.graphWithBalances.graph.vertexSet() == Set(aliceNodeId, bobNodeId))
assert(router.stateData.graphWithBalances.graph.edgeSet().toSet == Set(GraphEdge(aliceInitialChannelUpdate, privateChannel), GraphEdge(bobChannelUpdate1, privateChannel)))
}
// channel closes
// The channel closes.
channels.alice ! CMD_CLOSE(TestProbe().ref, scriptPubKey = None, feerates = None)
channels.alice2bob.expectMsgType[Shutdown]
channels.alice2bob.forward(channels.bob)
channels.bob2alice.expectMsgType[Shutdown]
channels.bob2alice.forward(channels.alice)
// If the channel was public, the router is notified when the funding tx is spent.
if (testTags.contains(ChannelStateTestsTags.ChannelsPublic)) {
// if the channel was public, the router asked the watcher to watch the funding tx and will be notified when it confirms
val closingTx = Transaction(2, Nil, Seq(TxOut(100_000 sat, Script.pay2wpkh(randomKey().publicKey))), 0)
val watchSpent = channels.alice2blockchain.expectMsgType[WatchExternalChannelSpent]
watchSpent.replyTo ! WatchExternalChannelSpentTriggered(watchSpent.shortChannelId, Transaction(0, Nil, Nil, 0))
watchSpent.replyTo ! WatchExternalChannelSpentTriggered(watchSpent.shortChannelId, closingTx)
val watchConfirmed = channels.alice2blockchain.expectMsgType[WatchTxConfirmed]
watchConfirmed.replyTo ! WatchTxConfirmedTriggered(BlockHeight(400000), 42, Transaction(0, Nil, Nil, 0))
assert(watchConfirmed.txId == closingTx.txid)
watchConfirmed.replyTo ! WatchTxConfirmedTriggered(BlockHeight(400_000), 42, closingTx)
}
// router cleans up the channel
// The router can now clean up the closed channel.
awaitAssert {
assert(router.stateData.nodes == Map.empty)
assert(router.stateData.channels == Map.empty)

View file

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

View file

@ -1 +1 @@
[{"remoteNodeId":"03af0ed6052cf28d670665549bc86f4b721c9fdb309d40c58f5811f63966e005d0","shortIds":{"real":{"status":"temporary","realScid":"0x0x1"},"localAlias":"0x2"},"canSend":100000000,"canReceive":20000000,"isPublic":true,"isEnabled":true},{"remoteNodeId":"03af0ed6052cf28d670665549bc86f4b721c9fdb309d40c58f5811f63966e005d0","shortIds":{"real":{"status":"final","realScid":"0x0x3"},"localAlias":"0x3","remoteAlias":"0x3"},"canSend":0,"canReceive":30000000,"isPublic":false,"isEnabled":false}]
[{"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}]

View file

@ -1 +1 @@
[{"remoteNodeId":"03af0ed6052cf28d670665549bc86f4b721c9fdb309d40c58f5811f63966e005d0","shortIds":{"real":{"status":"unknown"},"localAlias":"0x1"},"canSend":100000000,"canReceive":20000000,"isPublic":true,"isEnabled":true},{"remoteNodeId":"03af0ed6052cf28d670665549bc86f4b721c9fdb309d40c58f5811f63966e005d0","shortIds":{"real":{"status":"final","realScid":"0x0x2"},"localAlias":"0x3","remoteAlias":"0x4"},"canSend":400000000,"canReceive":30000000,"isPublic":false,"isEnabled":true}]
[{"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}]

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(RealScidStatus.Unknown, Alias(1), None), 100000000 msat, 20000000 msat, isPublic = true, isEnabled = true),
ChannelBalance(aliceNodeId, ShortIds(RealScidStatus.Final(RealShortChannelId(2)), Alias(3), Some(Alias(4))), 400000000 msat, 30000000 msat, isPublic = false, isEnabled = true)
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)
))
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(RealScidStatus.Temporary(RealShortChannelId(1)), Alias(2), None), 100000000 msat, 20000000 msat, isPublic = true, isEnabled = true),
ChannelBalance(aliceNodeId, ShortIds(RealScidStatus.Final(RealShortChannelId(3)), Alias(3), Some(Alias(3))), 0 msat, 30000000 msat, isPublic = false, isEnabled = false)
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)
))
val mockService = mockApi(eclair)