mirror of
https://github.com/ACINQ/eclair.git
synced 2024-11-19 01:43:22 +01:00
Implement on-the-fly funding based on splicing and liquidity ads (#2861)
* Add `on_the_fly_funding` feature bit and messages Add the (disabled by default) `on_the_fly_funding` feature bit and codecs for the corresponding messages: - `will_add_htlc` - `will_fail_htlc` - `will_fail_malformed_htlc` - `cancel_on_the_fly_funding` We also add a TLV to `update_add_htlc` to notify the recipient that we relayed less data than what the onion encodes, in exchange for the fees of the specified funding transaction. * Add `non_initiator_pays_commit_fees` channel flag We add a non-standard channel flag to `open_channel2` to allow wallets to ask their peer to pay the commit tx fees, even when they're not the channel opener. This is necessary for on-the-fly funding, until we can move to 0-fee commit txs which will make it obsolete. * Allow underpaying feerate when using future HTLCs When an interactive-tx session is created for a liquidity purchase that uses future HTLCs to pay fees, the initiator may not have enough funds to honor the target feerate. We allow the transaction anyway, because we want to get paid for the liquidity we're providing. If the feerate is too low and the transaction doesn't confirm, we can double-spend it if we need that liquidity elsewhere. * Add `funding_fee` field to `CMD_ADD_HTLC` This commit adds the funding fee field to HTLCs, but never sets it. We update a lot of test files, but there is no functional change. * Implement on-the-fly funding Implement the on-the-fly funding protocol: when a payment cannot be relayed because of a liquidity issue, we notify the `Peer` actor that we'd like to trigger on-the-fly funding if available. If available, we we send a funding proposal to our peer and keep track of its status. Once a matching funding transaction is signed, we persist this funding attempt and wait for the additional liquidity to be available (once the channel is ready or the splice locked). We will then frequently try to relay the payment to get paid our liquidity fees. If the payment keeps getting rejected, or we cannot connect to our peer, we abandon the payment when it reaches its CLTV expiry, which ensures that the upstream channels are not at risk. When using on-the-fly funding, we use a single channel with our peer. If they try to open another channel while one is available, we reject their request and expect a splice instead.
This commit is contained in:
parent
db8290f80e
commit
de42c8aa1b
@ -83,6 +83,7 @@ eclair {
|
||||
keysend = disabled
|
||||
trampoline_payment_prototype = disabled
|
||||
async_payment_prototype = disabled
|
||||
on_the_fly_funding = disabled
|
||||
}
|
||||
// The following section lets you customize features for specific nodes.
|
||||
// The overrides will be applied on top of the default features settings.
|
||||
@ -340,6 +341,13 @@ eclair {
|
||||
]
|
||||
}
|
||||
|
||||
// On-the-fly funding leverages liquidity ads to fund channels with wallet peers based on their payment patterns.
|
||||
on-the-fly-funding {
|
||||
// If our peer doesn't respond to our funding proposal, we must fail the corresponding upstream HTLCs.
|
||||
// Since MPP may be used, we should use a timeout greater than the MPP timeout.
|
||||
proposal-timeout = 90 seconds
|
||||
}
|
||||
|
||||
peer-connection {
|
||||
auth-timeout = 15 seconds // will disconnect if connection authentication doesn't happen within that timeframe
|
||||
init-timeout = 15 seconds // will disconnect if initialization doesn't happen within that timeframe
|
||||
|
@ -323,6 +323,15 @@ object Features {
|
||||
val mandatory = 154
|
||||
}
|
||||
|
||||
/**
|
||||
* Activate this feature to provide on-the-fly funding to remote nodes, as specified in bLIP 36: https://github.com/lightning/blips/blob/master/blip-0036.md.
|
||||
* TODO: add NodeFeature once bLIP is merged.
|
||||
*/
|
||||
case object OnTheFlyFunding extends Feature with InitFeature {
|
||||
val rfcName = "on_the_fly_funding"
|
||||
val mandatory = 560
|
||||
}
|
||||
|
||||
val knownFeatures: Set[Feature] = Set(
|
||||
DataLossProtect,
|
||||
InitialRoutingSync,
|
||||
@ -349,6 +358,7 @@ object Features {
|
||||
TrampolinePaymentPrototype,
|
||||
AsyncPaymentPrototype,
|
||||
SplicePrototype,
|
||||
OnTheFlyFunding
|
||||
)
|
||||
|
||||
// Features may depend on other features, as specified in Bolt 9.
|
||||
@ -361,7 +371,8 @@ object Features {
|
||||
RouteBlinding -> (VariableLengthOnion :: Nil),
|
||||
TrampolinePaymentPrototype -> (PaymentSecret :: Nil),
|
||||
KeySend -> (VariableLengthOnion :: Nil),
|
||||
AsyncPaymentPrototype -> (TrampolinePaymentPrototype :: Nil)
|
||||
AsyncPaymentPrototype -> (TrampolinePaymentPrototype :: Nil),
|
||||
OnTheFlyFunding -> (SplicePrototype :: Nil)
|
||||
)
|
||||
|
||||
case class FeatureException(message: String) extends IllegalArgumentException(message)
|
||||
|
@ -30,6 +30,7 @@ import fr.acinq.eclair.db._
|
||||
import fr.acinq.eclair.io.MessageRelay.{RelayAll, RelayChannelsOnly, RelayPolicy}
|
||||
import fr.acinq.eclair.io.{PeerConnection, PeerReadyNotifier}
|
||||
import fr.acinq.eclair.message.OnionMessages.OnionMessageConfig
|
||||
import fr.acinq.eclair.payment.relay.OnTheFlyFunding
|
||||
import fr.acinq.eclair.payment.relay.Relayer.{AsyncPaymentsParams, RelayFees, RelayParams}
|
||||
import fr.acinq.eclair.router.Announcements.AddressException
|
||||
import fr.acinq.eclair.router.Graph.{HeuristicsConstants, WeightRatios}
|
||||
@ -90,7 +91,8 @@ case class NodeParams(nodeKeyManager: NodeKeyManager,
|
||||
purgeInvoicesInterval: Option[FiniteDuration],
|
||||
revokedHtlcInfoCleanerConfig: RevokedHtlcInfoCleaner.Config,
|
||||
willFundRates_opt: Option[LiquidityAds.WillFundRates],
|
||||
peerWakeUpConfig: PeerReadyNotifier.WakeUpConfig) {
|
||||
peerWakeUpConfig: PeerReadyNotifier.WakeUpConfig,
|
||||
onTheFlyFundingConfig: OnTheFlyFunding.Config) {
|
||||
val privateKey: Crypto.PrivateKey = nodeKeyManager.nodeKey.privateKey
|
||||
|
||||
val nodeId: PublicKey = nodeKeyManager.nodeId
|
||||
@ -504,7 +506,10 @@ object NodeParams extends Logging {
|
||||
|
||||
val willFundRates_opt = {
|
||||
val supportedPaymentTypes = Map(
|
||||
LiquidityAds.PaymentType.FromChannelBalance.rfcName -> LiquidityAds.PaymentType.FromChannelBalance
|
||||
LiquidityAds.PaymentType.FromChannelBalance.rfcName -> LiquidityAds.PaymentType.FromChannelBalance,
|
||||
LiquidityAds.PaymentType.FromChannelBalanceForFutureHtlc.rfcName -> LiquidityAds.PaymentType.FromChannelBalanceForFutureHtlc,
|
||||
LiquidityAds.PaymentType.FromFutureHtlc.rfcName -> LiquidityAds.PaymentType.FromFutureHtlc,
|
||||
LiquidityAds.PaymentType.FromFutureHtlcWithPreimage.rfcName -> LiquidityAds.PaymentType.FromFutureHtlcWithPreimage,
|
||||
)
|
||||
val paymentTypes: Set[LiquidityAds.PaymentType] = config.getStringList("liquidity-ads.payment-types").asScala.map(s => {
|
||||
supportedPaymentTypes.get(s) match {
|
||||
@ -668,7 +673,10 @@ object NodeParams extends Logging {
|
||||
willFundRates_opt = willFundRates_opt,
|
||||
peerWakeUpConfig = PeerReadyNotifier.WakeUpConfig(
|
||||
enabled = config.getBoolean("peer-wake-up.enabled"),
|
||||
timeout = FiniteDuration(config.getDuration("peer-wake-up.timeout").getSeconds, TimeUnit.SECONDS)
|
||||
timeout = FiniteDuration(config.getDuration("peer-wake-up.timeout").getSeconds, TimeUnit.SECONDS),
|
||||
),
|
||||
onTheFlyFundingConfig = OnTheFlyFunding.Config(
|
||||
proposalTimeout = FiniteDuration(config.getDuration("on-the-fly-funding.proposal-timeout").getSeconds, TimeUnit.SECONDS),
|
||||
),
|
||||
)
|
||||
}
|
||||
|
@ -113,6 +113,7 @@ case class INPUT_INIT_CHANNEL_NON_INITIATOR(temporaryChannelId: ByteVector32,
|
||||
fundingContribution_opt: Option[LiquidityAds.AddFunding],
|
||||
dualFunded: Boolean,
|
||||
pushAmount_opt: Option[MilliSatoshi],
|
||||
requireConfirmedInputs: Boolean,
|
||||
localParams: LocalParams,
|
||||
remote: ActorRef,
|
||||
remoteInit: Init,
|
||||
@ -146,7 +147,7 @@ object Upstream {
|
||||
val expiryIn: CltvExpiry = add.cltvExpiry
|
||||
}
|
||||
/** Our node is forwarding a payment based on a set of HTLCs from potentially multiple upstream channels. */
|
||||
case class Trampoline(received: Seq[Channel]) extends Hot {
|
||||
case class Trampoline(received: List[Channel]) extends Hot {
|
||||
override val amountIn: MilliSatoshi = received.map(_.add.amountMsat).sum
|
||||
// We must use the lowest expiry of the incoming HTLC set.
|
||||
val expiryIn: CltvExpiry = received.map(_.add.cltvExpiry).min
|
||||
@ -165,6 +166,10 @@ object Upstream {
|
||||
|
||||
/** Our node is forwarding a single incoming HTLC. */
|
||||
case class Channel(originChannelId: ByteVector32, originHtlcId: Long, amountIn: MilliSatoshi) extends Cold
|
||||
object Channel {
|
||||
def apply(add: UpdateAddHtlc): Channel = Channel(add.channelId, add.id, add.amountMsat)
|
||||
}
|
||||
|
||||
/** Our node is forwarding a payment based on a set of HTLCs from potentially multiple upstream channels. */
|
||||
case class Trampoline(originHtlcs: List[Channel]) extends Cold { override val amountIn: MilliSatoshi = originHtlcs.map(_.amountIn).sum }
|
||||
}
|
||||
@ -197,7 +202,17 @@ sealed trait HasOptionalReplyToCommand extends Command { def replyTo_opt: Option
|
||||
sealed trait ForbiddenCommandDuringSplice extends Command
|
||||
sealed trait ForbiddenCommandDuringQuiescence extends Command
|
||||
|
||||
final case class CMD_ADD_HTLC(replyTo: ActorRef, amount: MilliSatoshi, paymentHash: ByteVector32, cltvExpiry: CltvExpiry, onion: OnionRoutingPacket, nextBlindingKey_opt: Option[PublicKey], confidence: Double, origin: Origin.Hot, commit: Boolean = false) extends HasReplyToCommand with ForbiddenCommandDuringSplice with ForbiddenCommandDuringQuiescence
|
||||
final case class CMD_ADD_HTLC(replyTo: ActorRef,
|
||||
amount: MilliSatoshi,
|
||||
paymentHash: ByteVector32,
|
||||
cltvExpiry: CltvExpiry,
|
||||
onion: OnionRoutingPacket,
|
||||
nextBlindingKey_opt: Option[PublicKey],
|
||||
confidence: Double,
|
||||
fundingFee_opt: Option[LiquidityAds.FundingFee],
|
||||
origin: Origin.Hot,
|
||||
commit: Boolean = false) extends HasReplyToCommand with ForbiddenCommandDuringSplice with ForbiddenCommandDuringQuiescence
|
||||
|
||||
sealed trait HtlcSettlementCommand extends HasOptionalReplyToCommand with ForbiddenCommandDuringSplice with ForbiddenCommandDuringQuiescence { def id: Long }
|
||||
final case class CMD_FULFILL_HTLC(id: Long, r: ByteVector32, commit: Boolean = false, replyTo_opt: Option[ActorRef] = None) extends HtlcSettlementCommand
|
||||
final case class CMD_FAIL_HTLC(id: Long, reason: Either[ByteVector, FailureMessage], delay_opt: Option[FiniteDuration] = None, commit: Boolean = false, replyTo_opt: Option[ActorRef] = None) extends HtlcSettlementCommand
|
||||
@ -666,8 +681,16 @@ case class RemoteParams(nodeId: PublicKey,
|
||||
initFeatures: Features[InitFeature],
|
||||
upfrontShutdownScript_opt: Option[ByteVector])
|
||||
|
||||
case class ChannelFlags(announceChannel: Boolean) {
|
||||
override def toString: String = s"ChannelFlags(announceChannel=$announceChannel)"
|
||||
/**
|
||||
* The [[nonInitiatorPaysCommitFees]] parameter is set to true when the sender wants the receiver to pay the commitment transaction fees.
|
||||
* This is not part of the BOLTs and won't be needed anymore once commitment transactions don't pay any on-chain fees.
|
||||
*/
|
||||
case class ChannelFlags(nonInitiatorPaysCommitFees: Boolean, announceChannel: Boolean) {
|
||||
override def toString: String = s"ChannelFlags(announceChannel=$announceChannel, nonInitiatorPaysCommitFees=$nonInitiatorPaysCommitFees)"
|
||||
}
|
||||
|
||||
object ChannelFlags {
|
||||
def apply(announceChannel: Boolean): ChannelFlags = ChannelFlags(nonInitiatorPaysCommitFees = false, announceChannel = announceChannel)
|
||||
}
|
||||
|
||||
/** Information about what triggered the opening of the channel */
|
||||
|
@ -53,6 +53,9 @@ case class ChannelAborted(channel: ActorRef, remoteNodeId: PublicKey, channelId:
|
||||
/** This event will be sent once a channel has been successfully opened and is ready to process payments. */
|
||||
case class ChannelOpened(channel: ActorRef, remoteNodeId: PublicKey, channelId: ByteVector32) extends ChannelEvent
|
||||
|
||||
/** This event is sent once channel_ready or splice_locked have been exchanged. */
|
||||
case class ChannelReadyForPayments(channel: ActorRef, remoteNodeId: PublicKey, channelId: ByteVector32, fundingTxIndex: Long) extends ChannelEvent
|
||||
|
||||
case class LocalChannelUpdate(channel: ActorRef, channelId: ByteVector32, shortIds: ShortIds, remoteNodeId: PublicKey, channelAnnouncement_opt: Option[ChannelAnnouncement], channelUpdate: ChannelUpdate, commitments: Commitments) extends ChannelEvent {
|
||||
/**
|
||||
* We always include the local alias because we must always be able to route based on it.
|
||||
|
@ -854,7 +854,7 @@ case class Commitments(params: ChannelParams,
|
||||
return Left(HtlcValueTooSmall(params.channelId, minimum = htlcMinimum, actual = cmd.amount))
|
||||
}
|
||||
|
||||
val add = UpdateAddHtlc(channelId, changes.localNextHtlcId, cmd.amount, cmd.paymentHash, cmd.cltvExpiry, cmd.onion, cmd.nextBlindingKey_opt, cmd.confidence)
|
||||
val add = UpdateAddHtlc(channelId, changes.localNextHtlcId, cmd.amount, cmd.paymentHash, cmd.cltvExpiry, cmd.onion, cmd.nextBlindingKey_opt, cmd.confidence, cmd.fundingFee_opt)
|
||||
// we increment the local htlc index and add an entry to the origins map
|
||||
val changes1 = changes.addLocalProposal(add).copy(localNextHtlcId = changes.localNextHtlcId + 1)
|
||||
val originChannels1 = originChannels + (add.id -> cmd.origin)
|
||||
|
@ -44,6 +44,7 @@ import fr.acinq.eclair.crypto.keymanager.ChannelKeyManager
|
||||
import fr.acinq.eclair.db.DbEventHandler.ChannelEvent.EventType
|
||||
import fr.acinq.eclair.db.PendingCommandsDb
|
||||
import fr.acinq.eclair.io.Peer
|
||||
import fr.acinq.eclair.io.Peer.LiquidityPurchaseSigned
|
||||
import fr.acinq.eclair.payment.relay.Relayer
|
||||
import fr.acinq.eclair.payment.{Bolt11Invoice, PaymentSettlingOnChain}
|
||||
import fr.acinq.eclair.router.Announcements
|
||||
@ -1095,10 +1096,13 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
|
||||
log.info("ignoring outgoing interactive-tx message {} from previous session", msg.getClass.getSimpleName)
|
||||
stay()
|
||||
}
|
||||
case InteractiveTxBuilder.Succeeded(signingSession, commitSig) =>
|
||||
case InteractiveTxBuilder.Succeeded(signingSession, commitSig, liquidityPurchase_opt) =>
|
||||
log.info(s"splice tx created with fundingTxIndex=${signingSession.fundingTxIndex} fundingTxId=${signingSession.fundingTx.txId}")
|
||||
cmd_opt.foreach(cmd => cmd.replyTo ! RES_SPLICE(fundingTxIndex = signingSession.fundingTxIndex, signingSession.fundingTx.txId, signingSession.fundingParams.fundingAmount, signingSession.localCommit.fold(_.spec, _.spec).toLocal))
|
||||
remoteCommitSig_opt.foreach(self ! _)
|
||||
liquidityPurchase_opt.collect {
|
||||
case purchase if !signingSession.fundingParams.isInitiator => peer ! LiquidityPurchaseSigned(d.channelId, signingSession.fundingTx.txId, signingSession.fundingTxIndex, d.commitments.params.remoteParams.htlcMinimum, purchase)
|
||||
}
|
||||
val d1 = d.copy(spliceStatus = SpliceStatus.SpliceWaitingForSigs(signingSession))
|
||||
stay() using d1 storing() sending commitSig
|
||||
case f: InteractiveTxBuilder.Failed =>
|
||||
@ -2139,6 +2143,12 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
|
||||
}
|
||||
}
|
||||
|
||||
// We tell the peer that the channel is ready to process payments that may be queued.
|
||||
if (!shutdownInProgress) {
|
||||
val fundingTxIndex = commitments1.active.map(_.fundingTxIndex).min
|
||||
peer ! ChannelReadyForPayments(self, remoteNodeId, d.channelId, fundingTxIndex)
|
||||
}
|
||||
|
||||
goto(NORMAL) using d.copy(commitments = commitments1, spliceStatus = spliceStatus1) sending sendQueue
|
||||
}
|
||||
|
||||
@ -2710,6 +2720,12 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
|
||||
if (oldCommitments.availableBalanceForSend != newCommitments.availableBalanceForSend || oldCommitments.availableBalanceForReceive != newCommitments.availableBalanceForReceive) {
|
||||
context.system.eventStream.publish(AvailableBalanceChanged(self, newCommitments.channelId, shortIds, newCommitments))
|
||||
}
|
||||
if (oldCommitments.active.size != newCommitments.active.size) {
|
||||
// Some commitments have been deactivated, which means our available balance changed, which may allow forwarding
|
||||
// payments that couldn't be forwarded before.
|
||||
val fundingTxIndex = newCommitments.active.map(_.fundingTxIndex).min
|
||||
peer ! ChannelReadyForPayments(self, remoteNodeId, newCommitments.channelId, fundingTxIndex)
|
||||
}
|
||||
}
|
||||
|
||||
private def maybeUpdateMaxHtlcAmount(currentMaxHtlcAmount: MilliSatoshi, newCommitments: Commitments): Unit = {
|
||||
|
@ -27,7 +27,7 @@ import fr.acinq.eclair.channel.fund.InteractiveTxBuilder.{FullySignedSharedTrans
|
||||
import fr.acinq.eclair.channel.fund.{InteractiveTxBuilder, InteractiveTxSigningSession}
|
||||
import fr.acinq.eclair.channel.publish.TxPublisher.SetChannelId
|
||||
import fr.acinq.eclair.crypto.ShaChain
|
||||
import fr.acinq.eclair.io.Peer.OpenChannelResponse
|
||||
import fr.acinq.eclair.io.Peer.{LiquidityPurchaseSigned, OpenChannelResponse}
|
||||
import fr.acinq.eclair.wire.protocol._
|
||||
import fr.acinq.eclair.{MilliSatoshiLong, RealShortChannelId, ToMilliSatoshiConversion, UInt64, randomBytes32}
|
||||
|
||||
@ -178,7 +178,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
|
||||
val tlvs: Set[AcceptDualFundedChannelTlv] = Set(
|
||||
upfrontShutdownScript_opt,
|
||||
Some(ChannelTlv.ChannelTypeTlv(d.init.channelType)),
|
||||
if (nodeParams.channelConf.requireConfirmedInputsForDualFunding) Some(ChannelTlv.RequireConfirmedInputsTlv()) else None,
|
||||
if (d.init.requireConfirmedInputs) Some(ChannelTlv.RequireConfirmedInputsTlv()) else None,
|
||||
willFund_opt.map(l => ChannelTlv.ProvideFundingTlv(l.willFund)),
|
||||
d.init.pushAmount_opt.map(amount => ChannelTlv.PushAmountTlv(amount)),
|
||||
).flatten
|
||||
@ -339,9 +339,12 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
|
||||
|
||||
case Event(msg: InteractiveTxBuilder.Response, d: DATA_WAIT_FOR_DUAL_FUNDING_CREATED) => msg match {
|
||||
case InteractiveTxBuilder.SendMessage(_, msg) => stay() sending msg
|
||||
case InteractiveTxBuilder.Succeeded(status, commitSig) =>
|
||||
case InteractiveTxBuilder.Succeeded(status, commitSig, liquidityPurchase_opt) =>
|
||||
d.deferred.foreach(self ! _)
|
||||
d.replyTo_opt.foreach(_ ! OpenChannelResponse.Created(d.channelId, status.fundingTx.txId, status.fundingTx.tx.localFees.truncateToSatoshi))
|
||||
liquidityPurchase_opt.collect {
|
||||
case purchase if !status.fundingParams.isInitiator => peer ! LiquidityPurchaseSigned(d.channelId, status.fundingTx.txId, status.fundingTxIndex, d.channelParams.remoteParams.htlcMinimum, purchase)
|
||||
}
|
||||
val d1 = DATA_WAIT_FOR_DUAL_FUNDING_SIGNED(d.channelParams, d.secondRemotePerCommitmentPoint, d.localPushAmount, d.remotePushAmount, status, None)
|
||||
goto(WAIT_FOR_DUAL_FUNDING_SIGNED) using d1 storing() sending commitSig
|
||||
case f: InteractiveTxBuilder.Failed =>
|
||||
@ -506,7 +509,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
|
||||
cmd.replyTo ! RES_FAILURE(cmd, InvalidRbfFeerate(d.channelId, cmd.targetFeerate, minNextFeerate))
|
||||
stay()
|
||||
} else {
|
||||
val txInitRbf = TxInitRbf(d.channelId, cmd.lockTime, cmd.targetFeerate, d.latestFundingTx.fundingParams.localContribution, nodeParams.channelConf.requireConfirmedInputsForDualFunding, cmd.requestFunding_opt)
|
||||
val txInitRbf = TxInitRbf(d.channelId, cmd.lockTime, cmd.targetFeerate, d.latestFundingTx.fundingParams.localContribution, d.latestFundingTx.fundingParams.requireConfirmedInputs.forRemote, cmd.requestFunding_opt)
|
||||
stay() using d.copy(rbfStatus = RbfStatus.RbfRequested(cmd)) sending txInitRbf
|
||||
}
|
||||
case _ =>
|
||||
@ -556,7 +559,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
|
||||
remoteContribution = msg.fundingContribution,
|
||||
lockTime = msg.lockTime,
|
||||
targetFeerate = msg.feerate,
|
||||
requireConfirmedInputs = RequireConfirmedInputs(forLocal = msg.requireConfirmedInputs, forRemote = nodeParams.channelConf.requireConfirmedInputsForDualFunding)
|
||||
requireConfirmedInputs = d.latestFundingTx.fundingParams.requireConfirmedInputs.copy(forLocal = msg.requireConfirmedInputs)
|
||||
)
|
||||
val txBuilder = context.spawnAnonymous(InteractiveTxBuilder(
|
||||
randomBytes32(),
|
||||
@ -568,7 +571,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
|
||||
wallet))
|
||||
txBuilder ! InteractiveTxBuilder.Start(self)
|
||||
val toSend = Seq(
|
||||
Some(TxAckRbf(d.channelId, fundingParams.localContribution, nodeParams.channelConf.requireConfirmedInputsForDualFunding, willFund_opt.map(_.willFund))),
|
||||
Some(TxAckRbf(d.channelId, fundingParams.localContribution, d.latestFundingTx.fundingParams.requireConfirmedInputs.forRemote, willFund_opt.map(_.willFund))),
|
||||
if (remainingRbfAttempts <= 3) Some(Warning(d.channelId, s"will accept at most ${remainingRbfAttempts - 1} future rbf attempts")) else None,
|
||||
).flatten
|
||||
stay() using d.copy(rbfStatus = RbfStatus.RbfInProgress(cmd_opt = None, txBuilder, remoteCommitSig = None)) sending toSend
|
||||
@ -687,9 +690,12 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
|
||||
case RbfStatus.RbfInProgress(cmd_opt, _, remoteCommitSig_opt) =>
|
||||
msg match {
|
||||
case InteractiveTxBuilder.SendMessage(_, msg) => stay() sending msg
|
||||
case InteractiveTxBuilder.Succeeded(signingSession, commitSig) =>
|
||||
case InteractiveTxBuilder.Succeeded(signingSession, commitSig, liquidityPurchase_opt) =>
|
||||
cmd_opt.foreach(cmd => cmd.replyTo ! RES_BUMP_FUNDING_FEE(rbfIndex = d.previousFundingTxs.length, signingSession.fundingTx.txId, signingSession.fundingTx.tx.localFees.truncateToSatoshi))
|
||||
remoteCommitSig_opt.foreach(self ! _)
|
||||
liquidityPurchase_opt.collect {
|
||||
case purchase if !signingSession.fundingParams.isInitiator => peer ! LiquidityPurchaseSigned(d.channelId, signingSession.fundingTx.txId, signingSession.fundingTxIndex, d.commitments.params.remoteParams.htlcMinimum, purchase)
|
||||
}
|
||||
val d1 = d.copy(rbfStatus = RbfStatus.RbfWaitingForSigs(signingSession))
|
||||
stay() using d1 storing() sending commitSig
|
||||
case f: InteractiveTxBuilder.Failed =>
|
||||
|
@ -135,6 +135,7 @@ trait CommonFundingHandlers extends CommonHandlers {
|
||||
// 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)
|
||||
}
|
||||
|
||||
|
@ -92,7 +92,7 @@ object InteractiveTxBuilder {
|
||||
|
||||
sealed trait Response
|
||||
case class SendMessage(sessionId: ByteVector32, msg: LightningMessage) extends Response
|
||||
case class Succeeded(signingSession: InteractiveTxSigningSession.WaitingForSigs, commitSig: CommitSig) extends Response
|
||||
case class Succeeded(signingSession: InteractiveTxSigningSession.WaitingForSigs, commitSig: CommitSig, liquidityPurchase_opt: Option[LiquidityAds.Purchase]) extends Response
|
||||
sealed trait Failed extends Response { def cause: ChannelException }
|
||||
case class LocalFailure(cause: ChannelException) extends Failed
|
||||
case class RemoteFailure(cause: ChannelException) extends Failed
|
||||
@ -156,6 +156,15 @@ object InteractiveTxBuilder {
|
||||
val minNextFeerate: FeeratePerKw = targetFeerate * 25 / 24
|
||||
// BOLT 2: the initiator's serial IDs MUST use even values and the non-initiator odd values.
|
||||
val serialIdParity: Int = if (isInitiator) 0 else 1
|
||||
|
||||
def liquidityFees(liquidityPurchase_opt: Option[LiquidityAds.Purchase]): Satoshi = {
|
||||
liquidityPurchase_opt.map(l => l.paymentDetails match {
|
||||
// The initiator of the interactive-tx is the liquidity buyer (if liquidity ads is used).
|
||||
case LiquidityAds.PaymentDetails.FromChannelBalance | _: LiquidityAds.PaymentDetails.FromChannelBalanceForFutureHtlc => if (isInitiator) l.fees.total else -l.fees.total
|
||||
// Fees will be paid later, when relaying HTLCs.
|
||||
case _: LiquidityAds.PaymentDetails.FromFutureHtlc | _: LiquidityAds.PaymentDetails.FromFutureHtlcWithPreimage => 0.sat
|
||||
}).getOrElse(0 sat)
|
||||
}
|
||||
}
|
||||
|
||||
// @formatter:off
|
||||
@ -367,19 +376,28 @@ object InteractiveTxBuilder {
|
||||
Behaviors.withMdc(Logs.mdc(remoteNodeId_opt = Some(channelParams.remoteParams.nodeId), channelId_opt = Some(fundingParams.channelId))) {
|
||||
Behaviors.receiveMessagePartial {
|
||||
case Start(replyTo) =>
|
||||
val liquidityFee = liquidityPurchase_opt.map(l => l.paymentDetails match {
|
||||
// The initiator of the interactive-tx is the liquidity buyer (if liquidity ads is used).
|
||||
case LiquidityAds.PaymentDetails.FromChannelBalance => if (fundingParams.isInitiator) l.fees.total else -l.fees.total
|
||||
}).getOrElse(0 sat)
|
||||
val liquidityFee = fundingParams.liquidityFees(liquidityPurchase_opt)
|
||||
// Note that pending HTLCs are ignored: splices only affect the main outputs.
|
||||
val nextLocalBalance = purpose.previousLocalBalance + fundingParams.localContribution - localPushAmount + remotePushAmount - liquidityFee
|
||||
val nextRemoteBalance = purpose.previousRemoteBalance + fundingParams.remoteContribution - remotePushAmount + localPushAmount + liquidityFee
|
||||
val liquidityPaymentTypeOk = liquidityPurchase_opt match {
|
||||
case Some(l) if !fundingParams.isInitiator => l.paymentDetails match {
|
||||
case LiquidityAds.PaymentDetails.FromChannelBalance | _: LiquidityAds.PaymentDetails.FromChannelBalanceForFutureHtlc => true
|
||||
// If our peer has enough balance to pay the liquidity fees, they shouldn't use future HTLCs which
|
||||
// involves trust: they should directly pay from their channel balance.
|
||||
case _: LiquidityAds.PaymentDetails.FromFutureHtlc | _: LiquidityAds.PaymentDetails.FromFutureHtlcWithPreimage => nextRemoteBalance < l.fees.total
|
||||
}
|
||||
case _ => true
|
||||
}
|
||||
if (fundingParams.fundingAmount < fundingParams.dustLimit) {
|
||||
replyTo ! LocalFailure(FundingAmountTooLow(channelParams.channelId, fundingParams.fundingAmount, fundingParams.dustLimit))
|
||||
Behaviors.stopped
|
||||
} else if (nextLocalBalance < 0.msat || nextRemoteBalance < 0.msat) {
|
||||
replyTo ! LocalFailure(InvalidFundingBalances(channelParams.channelId, fundingParams.fundingAmount, nextLocalBalance, nextRemoteBalance))
|
||||
Behaviors.stopped
|
||||
} else if (!liquidityPaymentTypeOk) {
|
||||
replyTo ! LocalFailure(InvalidLiquidityAdsPaymentType(channelParams.channelId, liquidityPurchase_opt.get.paymentDetails.paymentType, Set(LiquidityAds.PaymentType.FromChannelBalance, LiquidityAds.PaymentType.FromChannelBalanceForFutureHtlc)))
|
||||
Behaviors.stopped
|
||||
} else {
|
||||
val actor = new InteractiveTxBuilder(replyTo, sessionId, nodeParams, channelParams, fundingParams, purpose, localPushAmount, remotePushAmount, liquidityPurchase_opt, wallet, stash, context)
|
||||
actor.start()
|
||||
@ -745,7 +763,17 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
|
||||
return Left(InvalidCompleteInteractiveTx(fundingParams.channelId))
|
||||
}
|
||||
case None =>
|
||||
val minimumFee = Transactions.weight2fee(fundingParams.targetFeerate, tx.weight())
|
||||
val feeWithoutWitness = Transactions.weight2fee(fundingParams.targetFeerate, tx.weight())
|
||||
val minimumFee = liquidityPurchase_opt.map(_.paymentDetails) match {
|
||||
case Some(paymentDetails) => paymentDetails match {
|
||||
case LiquidityAds.PaymentDetails.FromChannelBalance | _: LiquidityAds.PaymentDetails.FromChannelBalanceForFutureHtlc => feeWithoutWitness
|
||||
// We allow the feerate to be lower than requested when using on-the-fly funding, because our peer may not
|
||||
// be able to contribute as much as expected to the funding transaction itself since they don't have funds.
|
||||
// It's acceptable because they will be paying liquidity fees from future HTLCs.
|
||||
case _: LiquidityAds.PaymentDetails.FromFutureHtlc | _: LiquidityAds.PaymentDetails.FromFutureHtlcWithPreimage => feeWithoutWitness * 0.5
|
||||
}
|
||||
case None => feeWithoutWitness
|
||||
}
|
||||
if (sharedTx.fees < minimumFee) {
|
||||
log.warn("invalid interactive tx: below the target feerate (target={}, actual={})", fundingParams.targetFeerate, Transactions.fee2rate(sharedTx.fees, tx.weight()))
|
||||
return Left(InvalidCompleteInteractiveTx(fundingParams.channelId))
|
||||
@ -767,10 +795,7 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
|
||||
private def signCommitTx(completeTx: SharedTransaction): Behavior[Command] = {
|
||||
val fundingTx = completeTx.buildUnsignedTx()
|
||||
val fundingOutputIndex = fundingTx.txOut.indexWhere(_.publicKeyScript == fundingPubkeyScript)
|
||||
val liquidityFee = liquidityPurchase_opt.map(l => l.paymentDetails match {
|
||||
// The initiator of the interactive-tx is the liquidity buyer (if liquidity ads is used).
|
||||
case LiquidityAds.PaymentDetails.FromChannelBalance => if (fundingParams.isInitiator) l.fees.total else -l.fees.total
|
||||
}).getOrElse(0 sat)
|
||||
val liquidityFee = fundingParams.liquidityFees(liquidityPurchase_opt)
|
||||
Funding.makeCommitTxs(keyManager, channelParams,
|
||||
fundingAmount = fundingParams.fundingAmount,
|
||||
toLocal = completeTx.sharedOutput.localAmount - localPushAmount + remotePushAmount - liquidityFee,
|
||||
@ -825,7 +850,7 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
|
||||
)
|
||||
context.system.eventStream ! EventStream.Publish(ChannelLiquidityPurchased(replyTo.toClassic, channelParams.channelId, remoteNodeId, purchase))
|
||||
}
|
||||
replyTo ! Succeeded(InteractiveTxSigningSession.WaitingForSigs(fundingParams, purpose.fundingTxIndex, signedTx, Left(localCommit), remoteCommit), commitSig)
|
||||
replyTo ! Succeeded(InteractiveTxSigningSession.WaitingForSigs(fundingParams, purpose.fundingTxIndex, signedTx, Left(localCommit), remoteCommit), commitSig, liquidityPurchase_opt)
|
||||
Behaviors.stopped
|
||||
case WalletFailure(t) =>
|
||||
log.error("could not sign funding transaction: ", t)
|
||||
@ -981,7 +1006,11 @@ object InteractiveTxSigningSession {
|
||||
return Left(InvalidFundingSignature(fundingParams.channelId, Some(partiallySignedTx.txId)))
|
||||
}
|
||||
// We allow a 5% error margin since witness size prediction could be inaccurate.
|
||||
if (fundingParams.localContribution != 0.sat && txWithSigs.feerate < fundingParams.targetFeerate * 0.95) {
|
||||
// If they didn't contribute to the transaction, they're not responsible, so we don't check the feerate.
|
||||
// If we didn't contribute to the transaction, we don't care if they use a lower feerate than expected.
|
||||
val localContributed = txWithSigs.tx.localInputs.nonEmpty || txWithSigs.tx.localOutputs.nonEmpty
|
||||
val remoteContributed = txWithSigs.tx.remoteInputs.nonEmpty || txWithSigs.tx.remoteOutputs.nonEmpty
|
||||
if (localContributed && remoteContributed && txWithSigs.feerate < fundingParams.targetFeerate * 0.95) {
|
||||
return Left(InvalidFundingFeerate(fundingParams.channelId, fundingParams.targetFeerate, txWithSigs.feerate))
|
||||
}
|
||||
val previousOutputs = {
|
||||
|
@ -91,6 +91,9 @@ class DbEventHandler(nodeParams: NodeParams) extends Actor with DiagnosticActorL
|
||||
case ChannelPaymentRelayed(_, _, _, fromChannelId, toChannelId, _, _) =>
|
||||
channelsDb.updateChannelMeta(fromChannelId, ChannelEvent.EventType.PaymentReceived)
|
||||
channelsDb.updateChannelMeta(toChannelId, ChannelEvent.EventType.PaymentSent)
|
||||
case OnTheFlyFundingPaymentRelayed(_, incoming, outgoing) =>
|
||||
incoming.foreach(p => channelsDb.updateChannelMeta(p.channelId, ChannelEvent.EventType.PaymentReceived))
|
||||
outgoing.foreach(p => channelsDb.updateChannelMeta(p.channelId, ChannelEvent.EventType.PaymentSent))
|
||||
}
|
||||
auditDb.add(e)
|
||||
|
||||
|
@ -9,6 +9,7 @@ import fr.acinq.eclair.db.Databases.{FileBackup, PostgresDatabases, SqliteDataba
|
||||
import fr.acinq.eclair.db.DbEventHandler.ChannelEvent
|
||||
import fr.acinq.eclair.db.DualDatabases.runAsync
|
||||
import fr.acinq.eclair.payment._
|
||||
import fr.acinq.eclair.payment.relay.OnTheFlyFunding
|
||||
import fr.acinq.eclair.payment.relay.Relayer.RelayFees
|
||||
import fr.acinq.eclair.router.Router
|
||||
import fr.acinq.eclair.wire.protocol.{ChannelAnnouncement, ChannelUpdate, NodeAddress, NodeAnnouncement}
|
||||
@ -427,4 +428,39 @@ case class DualLiquidityDb(primary: LiquidityDb, secondary: LiquidityDb) extends
|
||||
primary.listPurchases(remoteNodeId)
|
||||
}
|
||||
|
||||
}
|
||||
override def addPendingOnTheFlyFunding(remoteNodeId: PublicKey, pending: OnTheFlyFunding.Pending): Unit = {
|
||||
runAsync(secondary.addPendingOnTheFlyFunding(remoteNodeId, pending))
|
||||
primary.addPendingOnTheFlyFunding(remoteNodeId, pending)
|
||||
}
|
||||
|
||||
override def removePendingOnTheFlyFunding(remoteNodeId: PublicKey, paymentHash: ByteVector32): Unit = {
|
||||
runAsync(secondary.removePendingOnTheFlyFunding(remoteNodeId, paymentHash))
|
||||
primary.removePendingOnTheFlyFunding(remoteNodeId, paymentHash)
|
||||
}
|
||||
|
||||
override def listPendingOnTheFlyFunding(remoteNodeId: PublicKey): Map[ByteVector32, OnTheFlyFunding.Pending] = {
|
||||
runAsync(secondary.listPendingOnTheFlyFunding(remoteNodeId))
|
||||
primary.listPendingOnTheFlyFunding(remoteNodeId)
|
||||
}
|
||||
|
||||
override def listPendingOnTheFlyFunding(): Map[PublicKey, Map[ByteVector32, OnTheFlyFunding.Pending]] = {
|
||||
runAsync(secondary.listPendingOnTheFlyFunding())
|
||||
primary.listPendingOnTheFlyFunding()
|
||||
}
|
||||
|
||||
override def listPendingOnTheFlyPayments(): Map[PublicKey, Set[ByteVector32]] = {
|
||||
runAsync(secondary.listPendingOnTheFlyPayments())
|
||||
primary.listPendingOnTheFlyPayments()
|
||||
}
|
||||
|
||||
override def addOnTheFlyFundingPreimage(preimage: ByteVector32): Unit = {
|
||||
runAsync(secondary.addOnTheFlyFundingPreimage(preimage))
|
||||
primary.addOnTheFlyFundingPreimage(preimage)
|
||||
}
|
||||
|
||||
override def getOnTheFlyFundingPreimage(paymentHash: ByteVector32): Option[ByteVector32] = {
|
||||
runAsync(secondary.getOnTheFlyFundingPreimage(paymentHash))
|
||||
primary.getOnTheFlyFundingPreimage(paymentHash)
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -17,8 +17,9 @@
|
||||
package fr.acinq.eclair.db
|
||||
|
||||
import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
|
||||
import fr.acinq.bitcoin.scalacompat.TxId
|
||||
import fr.acinq.bitcoin.scalacompat.{ByteVector32, TxId}
|
||||
import fr.acinq.eclair.channel.{ChannelLiquidityPurchased, LiquidityPurchase}
|
||||
import fr.acinq.eclair.payment.relay.OnTheFlyFunding
|
||||
|
||||
/**
|
||||
* Created by t-bast on 13/09/2024.
|
||||
@ -26,10 +27,34 @@ import fr.acinq.eclair.channel.{ChannelLiquidityPurchased, LiquidityPurchase}
|
||||
|
||||
trait LiquidityDb {
|
||||
|
||||
/** We save liquidity purchases as soon as the corresponding transaction is signed. */
|
||||
def addPurchase(liquidityPurchase: ChannelLiquidityPurchased): Unit
|
||||
|
||||
/** When a transaction confirms, we mark the corresponding liquidity purchase (if any) as confirmed. */
|
||||
def setConfirmed(remoteNodeId: PublicKey, txId: TxId): Unit
|
||||
|
||||
/** List all liquidity purchases with the given remote node. */
|
||||
def listPurchases(remoteNodeId: PublicKey): Seq[LiquidityPurchase]
|
||||
|
||||
/** We save on-the-fly funded proposals to allow completing the payment after a disconnection or a restart. */
|
||||
def addPendingOnTheFlyFunding(remoteNodeId: PublicKey, pending: OnTheFlyFunding.Pending): Unit
|
||||
|
||||
/** Once complete (succeeded or failed), we forget the pending on-the-fly funding proposal. */
|
||||
def removePendingOnTheFlyFunding(remoteNodeId: PublicKey, paymentHash: ByteVector32): Unit
|
||||
|
||||
/** List pending on-the-fly funding proposals we funded for the given remote node. */
|
||||
def listPendingOnTheFlyFunding(remoteNodeId: PublicKey): Map[ByteVector32, OnTheFlyFunding.Pending]
|
||||
|
||||
/** List all pending on-the-fly funding proposals we funded. */
|
||||
def listPendingOnTheFlyFunding(): Map[PublicKey, Map[ByteVector32, OnTheFlyFunding.Pending]]
|
||||
|
||||
/** List the payment_hashes of pending on-the-fly funding proposals we funded for all remote nodes. */
|
||||
def listPendingOnTheFlyPayments(): Map[PublicKey, Set[ByteVector32]]
|
||||
|
||||
/** When we receive the preimage for an on-the-fly payment, we save it to protect against replays. */
|
||||
def addOnTheFlyFundingPreimage(preimage: ByteVector32): Unit
|
||||
|
||||
/** Check if we received the preimage for the given payment hash of an on-the-fly payment. */
|
||||
def getOnTheFlyFundingPreimage(paymentHash: ByteVector32): Option[ByteVector32]
|
||||
|
||||
}
|
||||
|
@ -238,7 +238,9 @@ class PgAuditDb(implicit ds: DataSource) extends AuditDb with Logging {
|
||||
val payments = e match {
|
||||
case ChannelPaymentRelayed(amountIn, amountOut, _, fromChannelId, toChannelId, startedAt, settledAt) =>
|
||||
// non-trampoline relayed payments have one input and one output
|
||||
Seq(RelayedPart(fromChannelId, amountIn, "IN", "channel", startedAt), RelayedPart(toChannelId, amountOut, "OUT", "channel", settledAt))
|
||||
val in = Seq(RelayedPart(fromChannelId, amountIn, "IN", "channel", startedAt))
|
||||
val out = Seq(RelayedPart(toChannelId, amountOut, "OUT", "channel", settledAt))
|
||||
in ++ out
|
||||
case TrampolinePaymentRelayed(_, incoming, outgoing, nextTrampolineNodeId, nextTrampolineAmount) =>
|
||||
using(pg.prepareStatement("INSERT INTO audit.relayed_trampoline VALUES (?, ?, ?, ?)")) { statement =>
|
||||
statement.setString(1, e.paymentHash.toHex)
|
||||
@ -248,7 +250,13 @@ class PgAuditDb(implicit ds: DataSource) extends AuditDb with Logging {
|
||||
statement.executeUpdate()
|
||||
}
|
||||
// trampoline relayed payments do MPP aggregation and may have M inputs and N outputs
|
||||
incoming.map(i => RelayedPart(i.channelId, i.amount, "IN", "trampoline", i.receivedAt)) ++ outgoing.map(o => RelayedPart(o.channelId, o.amount, "OUT", "trampoline", o.settledAt))
|
||||
val in = incoming.map(i => RelayedPart(i.channelId, i.amount, "IN", "trampoline", i.receivedAt))
|
||||
val out = outgoing.map(o => RelayedPart(o.channelId, o.amount, "OUT", "trampoline", o.settledAt))
|
||||
in ++ out
|
||||
case OnTheFlyFundingPaymentRelayed(_, incoming, outgoing) =>
|
||||
val in = incoming.map(i => RelayedPart(i.channelId, i.amount, "IN", "on-the-fly-funding", i.receivedAt))
|
||||
val out = outgoing.map(o => RelayedPart(o.channelId, o.amount, "OUT", "on-the-fly-funding", o.settledAt))
|
||||
in ++ out
|
||||
}
|
||||
for (p <- payments) {
|
||||
using(pg.prepareStatement("INSERT INTO audit.relayed VALUES (?, ?, ?, ?, ?, ?)")) { statement =>
|
||||
@ -453,6 +461,8 @@ class PgAuditDb(implicit ds: DataSource) extends AuditDb with Logging {
|
||||
case Some(RelayedPart(_, _, _, "trampoline", _)) =>
|
||||
val (nextTrampolineAmount, nextTrampolineNodeId) = trampolineByHash.getOrElse(paymentHash, (0 msat, PlaceHolderPubKey))
|
||||
TrampolinePaymentRelayed(paymentHash, incoming, outgoing, nextTrampolineNodeId, nextTrampolineAmount) :: Nil
|
||||
case Some(RelayedPart(_, _, _, "on-the-fly-funding", _)) =>
|
||||
Seq(OnTheFlyFundingPaymentRelayed(paymentHash, incoming, outgoing))
|
||||
case _ => Nil
|
||||
}
|
||||
}.toSeq.sortBy(_.timestamp)
|
||||
@ -480,10 +490,21 @@ class PgAuditDb(implicit ds: DataSource) extends AuditDb with Logging {
|
||||
}
|
||||
|
||||
override def stats(from: TimestampMilli, to: TimestampMilli, paginated_opt: Option[Paginated]): Seq[Stats] = {
|
||||
val networkFees = listNetworkFees(from, to).foldLeft(Map.empty[ByteVector32, Satoshi]) { (feeByChannelId, f) =>
|
||||
feeByChannelId + (f.channelId -> (feeByChannelId.getOrElse(f.channelId, 0 sat) + f.fee))
|
||||
}
|
||||
case class Relayed(amount: MilliSatoshi, fee: MilliSatoshi, direction: String)
|
||||
|
||||
def aggregateRelayStats(previous: Map[ByteVector32, Seq[Relayed]], incoming: PaymentRelayed.Incoming, outgoing: PaymentRelayed.Outgoing): Map[ByteVector32, Seq[Relayed]] = {
|
||||
// We ensure trampoline payments are counted only once per channel and per direction (if multiple HTLCs were sent
|
||||
// from/to the same channel, we group them).
|
||||
val amountIn = incoming.map(_.amount).sum
|
||||
val amountOut = outgoing.map(_.amount).sum
|
||||
val in = incoming.groupBy(_.channelId).map { case (channelId, parts) => (channelId, Relayed(parts.map(_.amount).sum, 0 msat, "IN")) }.toSeq
|
||||
val out = outgoing.groupBy(_.channelId).map { case (channelId, parts) =>
|
||||
val fee = (amountIn - amountOut) * parts.length / outgoing.length // we split the fee among outgoing channels
|
||||
(channelId, Relayed(parts.map(_.amount).sum, fee, "OUT"))
|
||||
}.toSeq
|
||||
(in ++ out).groupBy(_._1).map { case (channelId, payments) => (channelId, payments.map(_._2) ++ previous.getOrElse(channelId, Nil)) }
|
||||
}
|
||||
|
||||
val relayed = listRelayed(from, to).foldLeft(Map.empty[ByteVector32, Seq[Relayed]]) { (previous, e) =>
|
||||
// NB: we must avoid counting the fee twice: we associate it to the outgoing channels rather than the incoming ones.
|
||||
val current = e match {
|
||||
@ -492,17 +513,17 @@ class PgAuditDb(implicit ds: DataSource) extends AuditDb with Logging {
|
||||
c.toChannelId -> (Relayed(c.amountOut, c.amountIn - c.amountOut, "OUT") +: previous.getOrElse(c.toChannelId, Nil)),
|
||||
)
|
||||
case t: TrampolinePaymentRelayed =>
|
||||
// We ensure a trampoline payment is counted only once per channel and per direction (if multiple HTLCs were
|
||||
// sent from/to the same channel, we group them).
|
||||
val in = t.incoming.groupBy(_.channelId).map { case (channelId, parts) => (channelId, Relayed(parts.map(_.amount).sum, 0 msat, "IN")) }.toSeq
|
||||
val out = t.outgoing.groupBy(_.channelId).map { case (channelId, parts) =>
|
||||
val fee = (t.amountIn - t.amountOut) * parts.length / t.outgoing.length // we split the fee among outgoing channels
|
||||
(channelId, Relayed(parts.map(_.amount).sum, fee, "OUT"))
|
||||
}.toSeq
|
||||
(in ++ out).groupBy(_._1).map { case (channelId, payments) => (channelId, payments.map(_._2) ++ previous.getOrElse(channelId, Nil)) }
|
||||
aggregateRelayStats(previous, t.incoming, t.outgoing)
|
||||
case f: OnTheFlyFundingPaymentRelayed =>
|
||||
aggregateRelayStats(previous, f.incoming, f.outgoing)
|
||||
}
|
||||
previous ++ current
|
||||
}
|
||||
|
||||
val networkFees = listNetworkFees(from, to).foldLeft(Map.empty[ByteVector32, Satoshi]) { (feeByChannelId, f) =>
|
||||
feeByChannelId + (f.channelId -> (feeByChannelId.getOrElse(f.channelId, 0 sat) + f.fee))
|
||||
}
|
||||
|
||||
// Channels opened by our peers won't have any network fees paid by us, but we still want to compute stats for them.
|
||||
val allChannels = networkFees.keySet ++ relayed.keySet
|
||||
val result = allChannels.toSeq.flatMap(channelId => {
|
||||
|
@ -17,14 +17,17 @@
|
||||
package fr.acinq.eclair.db.pg
|
||||
|
||||
import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
|
||||
import fr.acinq.bitcoin.scalacompat.{Satoshi, TxId}
|
||||
import fr.acinq.eclair.MilliSatoshi
|
||||
import fr.acinq.bitcoin.scalacompat.{ByteVector32, Crypto, Satoshi, TxId}
|
||||
import fr.acinq.eclair.channel.{ChannelLiquidityPurchased, LiquidityPurchase}
|
||||
import fr.acinq.eclair.db.LiquidityDb
|
||||
import fr.acinq.eclair.db.Monitoring.Metrics.withMetrics
|
||||
import fr.acinq.eclair.db.Monitoring.Tags.DbBackends
|
||||
import fr.acinq.eclair.db.pg.PgUtils.PgLock.NoLock.withLock
|
||||
import fr.acinq.eclair.payment.relay.OnTheFlyFunding
|
||||
import fr.acinq.eclair.wire.protocol.LiquidityAds
|
||||
import fr.acinq.eclair.{MilliSatoshi, MilliSatoshiLong}
|
||||
import grizzled.slf4j.Logging
|
||||
import scodec.bits.BitVector
|
||||
|
||||
import java.sql.Timestamp
|
||||
import java.time.Instant
|
||||
@ -50,7 +53,12 @@ class PgLiquidityDb(implicit ds: DataSource) extends LiquidityDb with Logging {
|
||||
getVersion(statement, DB_NAME) match {
|
||||
case None =>
|
||||
statement.executeUpdate("CREATE SCHEMA liquidity")
|
||||
// Liquidity purchases.
|
||||
statement.executeUpdate("CREATE TABLE liquidity.purchases (tx_id TEXT NOT NULL, channel_id TEXT NOT NULL, node_id TEXT NOT NULL, is_buyer BOOLEAN NOT NULL, amount_sat BIGINT NOT NULL, mining_fee_sat BIGINT NOT NULL, service_fee_sat BIGINT NOT NULL, funding_tx_index BIGINT NOT NULL, capacity_sat BIGINT NOT NULL, local_contribution_sat BIGINT NOT NULL, remote_contribution_sat BIGINT NOT NULL, local_balance_msat BIGINT NOT NULL, remote_balance_msat BIGINT NOT NULL, outgoing_htlc_count BIGINT NOT NULL, incoming_htlc_count BIGINT NOT NULL, created_at TIMESTAMP WITH TIME ZONE NOT NULL, confirmed_at TIMESTAMP WITH TIME ZONE)")
|
||||
// On-the-fly funding.
|
||||
statement.executeUpdate("CREATE TABLE liquidity.on_the_fly_funding_preimages (payment_hash TEXT NOT NULL PRIMARY KEY, preimage TEXT NOT NULL, received_at TIMESTAMP WITH TIME ZONE NOT NULL)")
|
||||
statement.executeUpdate("CREATE TABLE liquidity.pending_on_the_fly_funding (node_id TEXT NOT NULL, payment_hash TEXT NOT NULL, channel_id TEXT NOT NULL, tx_id TEXT NOT NULL, funding_tx_index BIGINT NOT NULL, remaining_fees_msat BIGINT NOT NULL, proposed BYTEA NOT NULL, funded_at TIMESTAMP WITH TIME ZONE NOT NULL, PRIMARY KEY (node_id, payment_hash))")
|
||||
// Indexes.
|
||||
statement.executeUpdate("CREATE INDEX liquidity_purchases_node_id_idx ON liquidity.purchases(node_id)")
|
||||
case Some(CURRENT_VERSION) => () // table is up-to-date, nothing to do
|
||||
case Some(unknownVersion) => throw new RuntimeException(s"Unknown version of DB $DB_NAME found, version=$unknownVersion")
|
||||
@ -118,4 +126,115 @@ class PgLiquidityDb(implicit ds: DataSource) extends LiquidityDb with Logging {
|
||||
}
|
||||
}
|
||||
|
||||
override def addPendingOnTheFlyFunding(remoteNodeId: Crypto.PublicKey, pending: OnTheFlyFunding.Pending): Unit = withMetrics("liquidity/add-pending-on-the-fly-funding", DbBackends.Postgres) {
|
||||
pending.status match {
|
||||
case _: OnTheFlyFunding.Status.Proposed => ()
|
||||
case status: OnTheFlyFunding.Status.Funded => withLock { pg =>
|
||||
using(pg.prepareStatement("INSERT INTO liquidity.pending_on_the_fly_funding (node_id, payment_hash, channel_id, tx_id, funding_tx_index, remaining_fees_msat, proposed, funded_at) VALUES (?, ?, ?, ?, ?, ?, ?, ?) ON CONFLICT DO NOTHING")) { statement =>
|
||||
statement.setString(1, remoteNodeId.toHex)
|
||||
statement.setString(2, pending.paymentHash.toHex)
|
||||
statement.setString(3, status.channelId.toHex)
|
||||
statement.setString(4, status.txId.value.toHex)
|
||||
statement.setLong(5, status.fundingTxIndex)
|
||||
statement.setLong(6, status.remainingFees.toLong)
|
||||
statement.setBytes(7, OnTheFlyFunding.Codecs.proposals.encode(pending.proposed).require.bytes.toArray)
|
||||
statement.setTimestamp(8, Timestamp.from(Instant.now()))
|
||||
statement.executeUpdate()
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
override def removePendingOnTheFlyFunding(remoteNodeId: Crypto.PublicKey, paymentHash: ByteVector32): Unit = withMetrics("liquidity/remove-pending-on-the-fly-funding", DbBackends.Postgres) {
|
||||
withLock { pg =>
|
||||
using(pg.prepareStatement("DELETE FROM liquidity.pending_on_the_fly_funding WHERE node_id = ? AND payment_hash = ?")) { statement =>
|
||||
statement.setString(1, remoteNodeId.toHex)
|
||||
statement.setString(2, paymentHash.toHex)
|
||||
statement.executeUpdate()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
override def listPendingOnTheFlyFunding(remoteNodeId: Crypto.PublicKey): Map[ByteVector32, OnTheFlyFunding.Pending] = withMetrics("liquidity/list-pending-on-the-fly-funding", DbBackends.Postgres) {
|
||||
withLock { pg =>
|
||||
using(pg.prepareStatement("SELECT * FROM liquidity.pending_on_the_fly_funding WHERE node_id = ?")) { statement =>
|
||||
statement.setString(1, remoteNodeId.toHex)
|
||||
statement.executeQuery().map { rs =>
|
||||
val paymentHash = rs.getByteVector32FromHex("payment_hash")
|
||||
val pending = OnTheFlyFunding.Pending(
|
||||
proposed = OnTheFlyFunding.Codecs.proposals.decode(BitVector(rs.getBytes("proposed"))).require.value,
|
||||
status = OnTheFlyFunding.Status.Funded(
|
||||
channelId = rs.getByteVector32FromHex("channel_id"),
|
||||
txId = TxId(rs.getByteVector32FromHex("tx_id")),
|
||||
fundingTxIndex = rs.getLong("funding_tx_index"),
|
||||
remainingFees = rs.getLong("remaining_fees_msat").msat
|
||||
)
|
||||
)
|
||||
paymentHash -> pending
|
||||
}.toMap
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
override def listPendingOnTheFlyFunding(): Map[PublicKey, Map[ByteVector32, OnTheFlyFunding.Pending]] = withMetrics("liquidity/list-pending-on-the-fly-funding-all", DbBackends.Postgres) {
|
||||
withLock { pg =>
|
||||
using(pg.prepareStatement("SELECT * FROM liquidity.pending_on_the_fly_funding")) { statement =>
|
||||
statement.executeQuery().map { rs =>
|
||||
val remoteNodeId = PublicKey(rs.getByteVectorFromHex("node_id"))
|
||||
val paymentHash = rs.getByteVector32FromHex("payment_hash")
|
||||
val pending = OnTheFlyFunding.Pending(
|
||||
proposed = OnTheFlyFunding.Codecs.proposals.decode(BitVector(rs.getBytes("proposed"))).require.value,
|
||||
status = OnTheFlyFunding.Status.Funded(
|
||||
channelId = rs.getByteVector32FromHex("channel_id"),
|
||||
txId = TxId(rs.getByteVector32FromHex("tx_id")),
|
||||
fundingTxIndex = rs.getLong("funding_tx_index"),
|
||||
remainingFees = rs.getLong("remaining_fees_msat").msat
|
||||
)
|
||||
)
|
||||
(remoteNodeId, paymentHash, pending)
|
||||
}.groupBy {
|
||||
case (remoteNodeId, _, _) => remoteNodeId
|
||||
}.map {
|
||||
case (remoteNodeId, payments) =>
|
||||
val paymentsMap = payments.map { case (_, paymentHash, pending) => paymentHash -> pending }.toMap
|
||||
remoteNodeId -> paymentsMap
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
override def listPendingOnTheFlyPayments(): Map[Crypto.PublicKey, Set[ByteVector32]] = withMetrics("liquidity/list-pending-on-the-fly-payments", DbBackends.Postgres) {
|
||||
withLock { pg =>
|
||||
using(pg.prepareStatement("SELECT node_id, payment_hash FROM liquidity.pending_on_the_fly_funding")) { statement =>
|
||||
statement.executeQuery().map { rs =>
|
||||
val remoteNodeId = PublicKey(rs.getByteVectorFromHex("node_id"))
|
||||
val paymentHash = rs.getByteVector32FromHex("payment_hash")
|
||||
remoteNodeId -> paymentHash
|
||||
}.groupMap(_._1)(_._2).map {
|
||||
case (remoteNodeId, payments) => remoteNodeId -> payments.toSet
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
override def addOnTheFlyFundingPreimage(preimage: ByteVector32): Unit = withMetrics("liquidity/add-on-the-fly-funding-preimage", DbBackends.Postgres) {
|
||||
withLock { pg =>
|
||||
using(pg.prepareStatement("INSERT INTO liquidity.on_the_fly_funding_preimages (payment_hash, preimage, received_at) VALUES (?, ?, ?) ON CONFLICT DO NOTHING")) { statement =>
|
||||
statement.setString(1, Crypto.sha256(preimage).toHex)
|
||||
statement.setString(2, preimage.toHex)
|
||||
statement.setTimestamp(3, Timestamp.from(Instant.now()))
|
||||
statement.executeUpdate()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
override def getOnTheFlyFundingPreimage(paymentHash: ByteVector32): Option[ByteVector32] = withMetrics("liquidity/get-on-the-fly-funding-preimage", DbBackends.Postgres) {
|
||||
withLock { pg =>
|
||||
using(pg.prepareStatement("SELECT preimage FROM liquidity.on_the_fly_funding_preimages WHERE payment_hash = ?")) { statement =>
|
||||
statement.setString(1, paymentHash.toHex)
|
||||
statement.executeQuery().map { rs => rs.getByteVector32FromHex("preimage") }.lastOption
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -226,7 +226,9 @@ class SqliteAuditDb(val sqlite: Connection) extends AuditDb with Logging {
|
||||
val payments = e match {
|
||||
case ChannelPaymentRelayed(amountIn, amountOut, _, fromChannelId, toChannelId, startedAt, settledAt) =>
|
||||
// non-trampoline relayed payments have one input and one output
|
||||
Seq(RelayedPart(fromChannelId, amountIn, "IN", "channel", startedAt), RelayedPart(toChannelId, amountOut, "OUT", "channel", settledAt))
|
||||
val in = Seq(RelayedPart(fromChannelId, amountIn, "IN", "channel", startedAt))
|
||||
val out = Seq(RelayedPart(toChannelId, amountOut, "OUT", "channel", settledAt))
|
||||
in ++ out
|
||||
case TrampolinePaymentRelayed(_, incoming, outgoing, nextTrampolineNodeId, nextTrampolineAmount) =>
|
||||
using(sqlite.prepareStatement("INSERT INTO relayed_trampoline VALUES (?, ?, ?, ?)")) { statement =>
|
||||
statement.setBytes(1, e.paymentHash.toArray)
|
||||
@ -236,8 +238,13 @@ class SqliteAuditDb(val sqlite: Connection) extends AuditDb with Logging {
|
||||
statement.executeUpdate()
|
||||
}
|
||||
// trampoline relayed payments do MPP aggregation and may have M inputs and N outputs
|
||||
incoming.map(i => RelayedPart(i.channelId, i.amount, "IN", "trampoline", i.receivedAt)) ++
|
||||
outgoing.map(o => RelayedPart(o.channelId, o.amount, "OUT", "trampoline", o.settledAt))
|
||||
val in = incoming.map(i => RelayedPart(i.channelId, i.amount, "IN", "trampoline", i.receivedAt))
|
||||
val out = outgoing.map(o => RelayedPart(o.channelId, o.amount, "OUT", "trampoline", o.settledAt))
|
||||
in ++ out
|
||||
case OnTheFlyFundingPaymentRelayed(_, incoming, outgoing) =>
|
||||
val in = incoming.map(i => RelayedPart(i.channelId, i.amount, "IN", "on-the-fly-funding", i.receivedAt))
|
||||
val out = outgoing.map(o => RelayedPart(o.channelId, o.amount, "OUT", "on-the-fly-funding", o.settledAt))
|
||||
in ++ out
|
||||
}
|
||||
for (p <- payments) {
|
||||
using(sqlite.prepareStatement("INSERT INTO relayed VALUES (?, ?, ?, ?, ?, ?)")) { statement =>
|
||||
@ -423,6 +430,8 @@ class SqliteAuditDb(val sqlite: Connection) extends AuditDb with Logging {
|
||||
case Some(RelayedPart(_, _, _, "trampoline", _)) =>
|
||||
val (nextTrampolineAmount, nextTrampolineNodeId) = trampolineByHash.getOrElse(paymentHash, (0 msat, PlaceHolderPubKey))
|
||||
TrampolinePaymentRelayed(paymentHash, incoming, outgoing, nextTrampolineNodeId, nextTrampolineAmount) :: Nil
|
||||
case Some(RelayedPart(_, _, _, "on-the-fly-funding", _)) =>
|
||||
Seq(OnTheFlyFundingPaymentRelayed(paymentHash, incoming, outgoing))
|
||||
case _ => Nil
|
||||
}
|
||||
}.toSeq.sortBy(_.timestamp)
|
||||
@ -449,10 +458,21 @@ class SqliteAuditDb(val sqlite: Connection) extends AuditDb with Logging {
|
||||
}
|
||||
|
||||
override def stats(from: TimestampMilli, to: TimestampMilli, paginated_opt: Option[Paginated]): Seq[Stats] = {
|
||||
val networkFees = listNetworkFees(from, to).foldLeft(Map.empty[ByteVector32, Satoshi]) { (feeByChannelId, f) =>
|
||||
feeByChannelId + (f.channelId -> (feeByChannelId.getOrElse(f.channelId, 0 sat) + f.fee))
|
||||
}
|
||||
case class Relayed(amount: MilliSatoshi, fee: MilliSatoshi, direction: String)
|
||||
|
||||
def aggregateRelayStats(previous: Map[ByteVector32, Seq[Relayed]], incoming: PaymentRelayed.Incoming, outgoing: PaymentRelayed.Outgoing): Map[ByteVector32, Seq[Relayed]] = {
|
||||
// We ensure trampoline payments are counted only once per channel and per direction (if multiple HTLCs were sent
|
||||
// from/to the same channel, we group them).
|
||||
val amountIn = incoming.map(_.amount).sum
|
||||
val amountOut = outgoing.map(_.amount).sum
|
||||
val in = incoming.groupBy(_.channelId).map { case (channelId, parts) => (channelId, Relayed(parts.map(_.amount).sum, 0 msat, "IN")) }.toSeq
|
||||
val out = outgoing.groupBy(_.channelId).map { case (channelId, parts) =>
|
||||
val fee = (amountIn - amountOut) * parts.length / outgoing.length // we split the fee among outgoing channels
|
||||
(channelId, Relayed(parts.map(_.amount).sum, fee, "OUT"))
|
||||
}.toSeq
|
||||
(in ++ out).groupBy(_._1).map { case (channelId, payments) => (channelId, payments.map(_._2) ++ previous.getOrElse(channelId, Nil)) }
|
||||
}
|
||||
|
||||
val relayed = listRelayed(from, to).foldLeft(Map.empty[ByteVector32, Seq[Relayed]]) { (previous, e) =>
|
||||
// NB: we must avoid counting the fee twice: we associate it to the outgoing channels rather than the incoming ones.
|
||||
val current = e match {
|
||||
@ -461,17 +481,17 @@ class SqliteAuditDb(val sqlite: Connection) extends AuditDb with Logging {
|
||||
c.toChannelId -> (Relayed(c.amountOut, c.amountIn - c.amountOut, "OUT") +: previous.getOrElse(c.toChannelId, Nil)),
|
||||
)
|
||||
case t: TrampolinePaymentRelayed =>
|
||||
// We ensure a trampoline payment is counted only once per channel and per direction (if multiple HTLCs were
|
||||
// sent from/to the same channel, we group them).
|
||||
val in = t.incoming.groupBy(_.channelId).map { case (channelId, parts) => (channelId, Relayed(parts.map(_.amount).sum, 0 msat, "IN")) }.toSeq
|
||||
val out = t.outgoing.groupBy(_.channelId).map { case (channelId, parts) =>
|
||||
val fee = (t.amountIn - t.amountOut) * parts.length / t.outgoing.length // we split the fee among outgoing channels
|
||||
(channelId, Relayed(parts.map(_.amount).sum, fee, "OUT"))
|
||||
}.toSeq
|
||||
(in ++ out).groupBy(_._1).map { case (channelId, payments) => (channelId, payments.map(_._2) ++ previous.getOrElse(channelId, Nil)) }
|
||||
aggregateRelayStats(previous, t.incoming, t.outgoing)
|
||||
case f: OnTheFlyFundingPaymentRelayed =>
|
||||
aggregateRelayStats(previous, f.incoming, f.outgoing)
|
||||
}
|
||||
previous ++ current
|
||||
}
|
||||
|
||||
val networkFees = listNetworkFees(from, to).foldLeft(Map.empty[ByteVector32, Satoshi]) { (feeByChannelId, f) =>
|
||||
feeByChannelId + (f.channelId -> (feeByChannelId.getOrElse(f.channelId, 0 sat) + f.fee))
|
||||
}
|
||||
|
||||
// Channels opened by our peers won't have any network fees paid by us, but we still want to compute stats for them.
|
||||
val allChannels = networkFees.keySet ++ relayed.keySet
|
||||
val result = allChannels.toSeq.flatMap(channelId => {
|
||||
|
@ -17,14 +17,16 @@
|
||||
package fr.acinq.eclair.db.sqlite
|
||||
|
||||
import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
|
||||
import fr.acinq.bitcoin.scalacompat.{Satoshi, TxId}
|
||||
import fr.acinq.bitcoin.scalacompat.{ByteVector32, Crypto, Satoshi, TxId}
|
||||
import fr.acinq.eclair.channel.{ChannelLiquidityPurchased, LiquidityPurchase}
|
||||
import fr.acinq.eclair.db.LiquidityDb
|
||||
import fr.acinq.eclair.db.Monitoring.Metrics.withMetrics
|
||||
import fr.acinq.eclair.db.Monitoring.Tags.DbBackends
|
||||
import fr.acinq.eclair.payment.relay.OnTheFlyFunding
|
||||
import fr.acinq.eclair.wire.protocol.LiquidityAds
|
||||
import fr.acinq.eclair.{MilliSatoshi, TimestampMilli}
|
||||
import fr.acinq.eclair.{MilliSatoshi, MilliSatoshiLong, TimestampMilli}
|
||||
import grizzled.slf4j.Logging
|
||||
import scodec.bits.BitVector
|
||||
|
||||
import java.sql.Connection
|
||||
|
||||
@ -46,7 +48,12 @@ class SqliteLiquidityDb(val sqlite: Connection) extends LiquidityDb with Logging
|
||||
using(sqlite.createStatement(), inTransaction = true) { statement =>
|
||||
getVersion(statement, DB_NAME) match {
|
||||
case None =>
|
||||
// Liquidity purchases.
|
||||
statement.executeUpdate("CREATE TABLE liquidity_purchases (tx_id BLOB NOT NULL, channel_id BLOB NOT NULL, node_id BLOB NOT NULL, is_buyer BOOLEAN NOT NULL, amount_sat INTEGER NOT NULL, mining_fee_sat INTEGER NOT NULL, service_fee_sat INTEGER NOT NULL, funding_tx_index INTEGER NOT NULL, capacity_sat INTEGER NOT NULL, local_contribution_sat INTEGER NOT NULL, remote_contribution_sat INTEGER NOT NULL, local_balance_msat INTEGER NOT NULL, remote_balance_msat INTEGER NOT NULL, outgoing_htlc_count INTEGER NOT NULL, incoming_htlc_count INTEGER NOT NULL, created_at INTEGER NOT NULL, confirmed_at INTEGER)")
|
||||
// On-the-fly funding.
|
||||
statement.executeUpdate("CREATE TABLE on_the_fly_funding_preimages (payment_hash BLOB NOT NULL PRIMARY KEY, preimage BLOB NOT NULL, received_at INTEGER NOT NULL)")
|
||||
statement.executeUpdate("CREATE TABLE on_the_fly_funding_pending (node_id BLOB NOT NULL, payment_hash BLOB NOT NULL, channel_id BLOB NOT NULL, tx_id BLOB NOT NULL, funding_tx_index INTEGER NOT NULL, remaining_fees_msat INTEGER NOT NULL, proposed BLOB NOT NULL, funded_at INTEGER NOT NULL, PRIMARY KEY (node_id, payment_hash))")
|
||||
// Indexes.
|
||||
statement.executeUpdate("CREATE INDEX liquidity_purchases_node_id_idx ON liquidity_purchases(node_id)")
|
||||
case Some(CURRENT_VERSION) => () // table is up-to-date, nothing to do
|
||||
case Some(unknownVersion) => throw new RuntimeException(s"Unknown version of DB $DB_NAME found, version=$unknownVersion")
|
||||
@ -107,4 +114,102 @@ class SqliteLiquidityDb(val sqlite: Connection) extends LiquidityDb with Logging
|
||||
}
|
||||
}
|
||||
|
||||
override def addPendingOnTheFlyFunding(remoteNodeId: Crypto.PublicKey, pending: OnTheFlyFunding.Pending): Unit = withMetrics("liquidity/add-pending-on-the-fly-funding", DbBackends.Sqlite) {
|
||||
pending.status match {
|
||||
case _: OnTheFlyFunding.Status.Proposed => ()
|
||||
case status: OnTheFlyFunding.Status.Funded =>
|
||||
using(sqlite.prepareStatement("INSERT OR IGNORE INTO on_the_fly_funding_pending (node_id, payment_hash, channel_id, tx_id, funding_tx_index, remaining_fees_msat, proposed, funded_at) VALUES (?, ?, ?, ?, ?, ?, ?, ?)")) { statement =>
|
||||
statement.setBytes(1, remoteNodeId.value.toArray)
|
||||
statement.setBytes(2, pending.paymentHash.toArray)
|
||||
statement.setBytes(3, status.channelId.toArray)
|
||||
statement.setBytes(4, status.txId.value.toArray)
|
||||
statement.setLong(5, status.fundingTxIndex)
|
||||
statement.setLong(6, status.remainingFees.toLong)
|
||||
statement.setBytes(7, OnTheFlyFunding.Codecs.proposals.encode(pending.proposed).require.bytes.toArray)
|
||||
statement.setLong(8, TimestampMilli.now().toLong)
|
||||
statement.executeUpdate()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
override def removePendingOnTheFlyFunding(remoteNodeId: Crypto.PublicKey, paymentHash: ByteVector32): Unit = withMetrics("liquidity/remove-pending-on-the-fly-funding", DbBackends.Sqlite) {
|
||||
using(sqlite.prepareStatement("DELETE FROM on_the_fly_funding_pending WHERE node_id = ? AND payment_hash = ?")) { statement =>
|
||||
statement.setBytes(1, remoteNodeId.value.toArray)
|
||||
statement.setBytes(2, paymentHash.toArray)
|
||||
statement.executeUpdate()
|
||||
}
|
||||
}
|
||||
|
||||
override def listPendingOnTheFlyFunding(remoteNodeId: Crypto.PublicKey): Map[ByteVector32, OnTheFlyFunding.Pending] = withMetrics("liquidity/list-pending-on-the-fly-funding", DbBackends.Sqlite) {
|
||||
using(sqlite.prepareStatement("SELECT * FROM on_the_fly_funding_pending WHERE node_id = ?")) { statement =>
|
||||
statement.setBytes(1, remoteNodeId.value.toArray)
|
||||
statement.executeQuery().map { rs =>
|
||||
val paymentHash = rs.getByteVector32("payment_hash")
|
||||
val pending = OnTheFlyFunding.Pending(
|
||||
proposed = OnTheFlyFunding.Codecs.proposals.decode(BitVector(rs.getBytes("proposed"))).require.value,
|
||||
status = OnTheFlyFunding.Status.Funded(
|
||||
channelId = rs.getByteVector32("channel_id"),
|
||||
txId = TxId(rs.getByteVector32("tx_id")),
|
||||
fundingTxIndex = rs.getLong("funding_tx_index"),
|
||||
remainingFees = rs.getLong("remaining_fees_msat").msat
|
||||
)
|
||||
)
|
||||
paymentHash -> pending
|
||||
}.toMap
|
||||
}
|
||||
}
|
||||
|
||||
override def listPendingOnTheFlyFunding(): Map[PublicKey, Map[ByteVector32, OnTheFlyFunding.Pending]] = withMetrics("liquidity/list-pending-on-the-fly-funding-all", DbBackends.Sqlite) {
|
||||
using(sqlite.prepareStatement("SELECT * FROM on_the_fly_funding_pending")) { statement =>
|
||||
statement.executeQuery().map { rs =>
|
||||
val remoteNodeId = PublicKey(rs.getByteVector("node_id"))
|
||||
val paymentHash = rs.getByteVector32("payment_hash")
|
||||
val pending = OnTheFlyFunding.Pending(
|
||||
proposed = OnTheFlyFunding.Codecs.proposals.decode(BitVector(rs.getBytes("proposed"))).require.value,
|
||||
status = OnTheFlyFunding.Status.Funded(
|
||||
channelId = rs.getByteVector32("channel_id"),
|
||||
txId = TxId(rs.getByteVector32("tx_id")),
|
||||
fundingTxIndex = rs.getLong("funding_tx_index"),
|
||||
remainingFees = rs.getLong("remaining_fees_msat").msat
|
||||
)
|
||||
)
|
||||
(remoteNodeId, paymentHash, pending)
|
||||
}.groupBy {
|
||||
case (remoteNodeId, _, _) => remoteNodeId
|
||||
}.map {
|
||||
case (remoteNodeId, payments) =>
|
||||
val paymentsMap = payments.map { case (_, paymentHash, pending) => paymentHash -> pending }.toMap
|
||||
remoteNodeId -> paymentsMap
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
override def listPendingOnTheFlyPayments(): Map[Crypto.PublicKey, Set[ByteVector32]] = withMetrics("liquidity/list-pending-on-the-fly-payments", DbBackends.Sqlite) {
|
||||
using(sqlite.prepareStatement("SELECT node_id, payment_hash FROM on_the_fly_funding_pending")) { statement =>
|
||||
statement.executeQuery().map { rs =>
|
||||
val remoteNodeId = PublicKey(rs.getByteVector("node_id"))
|
||||
val paymentHash = rs.getByteVector32("payment_hash")
|
||||
remoteNodeId -> paymentHash
|
||||
}.groupMap(_._1)(_._2).map {
|
||||
case (remoteNodeId, payments) => remoteNodeId -> payments.toSet
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
override def addOnTheFlyFundingPreimage(preimage: ByteVector32): Unit = withMetrics("liquidity/add-on-the-fly-funding-preimage", DbBackends.Sqlite) {
|
||||
using(sqlite.prepareStatement("INSERT OR IGNORE INTO on_the_fly_funding_preimages (payment_hash, preimage, received_at) VALUES (?, ?, ?)")) { statement =>
|
||||
statement.setBytes(1, Crypto.sha256(preimage).toArray)
|
||||
statement.setBytes(2, preimage.toArray)
|
||||
statement.setLong(3, TimestampMilli.now().toLong)
|
||||
statement.executeUpdate()
|
||||
}
|
||||
}
|
||||
|
||||
override def getOnTheFlyFundingPreimage(paymentHash: ByteVector32): Option[ByteVector32] = withMetrics("liquidity/get-on-the-fly-funding-preimage", DbBackends.Sqlite) {
|
||||
using(sqlite.prepareStatement("SELECT preimage FROM on_the_fly_funding_preimages WHERE payment_hash = ?")) { statement =>
|
||||
statement.setBytes(1, paymentHash.toArray)
|
||||
statement.executeQuery().map { rs => rs.getByteVector32("preimage") }.lastOption
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -154,7 +154,7 @@ private class MessageRelay(nodeParams: NodeParams,
|
||||
|
||||
private def waitForPreviousPeerForPolicyCheck(msg: OnionMessage, nextNodeId: PublicKey): Behavior[Command] = {
|
||||
Behaviors.receiveMessagePartial {
|
||||
case WrappedPeerInfo(PeerInfo(_, _, _, _, channels)) if channels.nonEmpty =>
|
||||
case WrappedPeerInfo(info: PeerInfo) if info.channels.nonEmpty =>
|
||||
switchboard ! GetPeerInfo(context.messageAdapter(WrappedPeerInfo), nextNodeId)
|
||||
waitForNextPeerForPolicyCheck(msg, nextNodeId)
|
||||
case _ =>
|
||||
@ -167,8 +167,8 @@ private class MessageRelay(nodeParams: NodeParams,
|
||||
|
||||
private def waitForNextPeerForPolicyCheck(msg: OnionMessage, nextNodeId: PublicKey): Behavior[Command] = {
|
||||
Behaviors.receiveMessagePartial {
|
||||
case WrappedPeerInfo(PeerInfo(peer, _, _, _, channels)) if channels.nonEmpty =>
|
||||
peer ! Peer.RelayOnionMessage(messageId, msg, replyTo_opt)
|
||||
case WrappedPeerInfo(info: PeerInfo) if info.channels.nonEmpty =>
|
||||
info.peer ! Peer.RelayOnionMessage(messageId, msg, replyTo_opt)
|
||||
Behaviors.stopped
|
||||
case _ =>
|
||||
Metrics.OnionMessagesNotRelayed.withTag(Tags.Reason, Tags.Reasons.NoChannelWithNextPeer).increment()
|
||||
|
@ -16,6 +16,7 @@
|
||||
|
||||
package fr.acinq.eclair.io
|
||||
|
||||
import fr.acinq.eclair.payment.relay.OnTheFlyFunding
|
||||
import kamon.Kamon
|
||||
|
||||
object Monitoring {
|
||||
@ -36,6 +37,9 @@ object Monitoring {
|
||||
|
||||
val IncomingConnectionsNoChannels = Kamon.gauge("incomingconnections.nochannels")
|
||||
val IncomingConnectionsDisconnected = Kamon.counter("incomingconnections.disconnected")
|
||||
|
||||
val OnTheFlyFunding = Kamon.counter("on-the-fly-funding.attempts")
|
||||
val OnTheFlyFundingFees = Kamon.histogram("on-the-fly-funding.fees-msat")
|
||||
}
|
||||
|
||||
object Tags {
|
||||
@ -64,6 +68,18 @@ object Monitoring {
|
||||
val NoChannelWithNextPeer = "NoChannelWithNextPeer"
|
||||
val ConnectionFailure = "ConnectionFailure"
|
||||
}
|
||||
|
||||
val OnTheFlyFundingState = "state"
|
||||
object OnTheFlyFundingStates {
|
||||
val Proposed = "proposed"
|
||||
val Rejected = "rejected"
|
||||
val Expired = "expired"
|
||||
val Timeout = "timeout"
|
||||
val Funded = "funded"
|
||||
val RelaySucceeded = "relay-succeeded"
|
||||
|
||||
def relayFailed(failure: OnTheFlyFunding.PaymentRelayer.RelayFailure) = s"relay-failed-${failure.getClass.getSimpleName}"
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -30,7 +30,7 @@ import fr.acinq.eclair.channel.fsm.Channel
|
||||
import fr.acinq.eclair.io.Peer.{OpenChannelResponse, SpawnChannelNonInitiator}
|
||||
import fr.acinq.eclair.io.PendingChannelsRateLimiter.AddOrRejectChannel
|
||||
import fr.acinq.eclair.wire.protocol
|
||||
import fr.acinq.eclair.wire.protocol.{Error, NodeAddress}
|
||||
import fr.acinq.eclair.wire.protocol.{Error, LiquidityAds, NodeAddress}
|
||||
import fr.acinq.eclair.{AcceptOpenChannel, CltvExpiryDelta, Features, InitFeature, InterceptOpenChannelPlugin, InterceptOpenChannelReceived, InterceptOpenChannelResponse, Logs, MilliSatoshi, NodeParams, RejectOpenChannel, ToMilliSatoshiConversion}
|
||||
import scodec.bits.ByteVector
|
||||
|
||||
@ -63,6 +63,8 @@ object OpenChannelInterceptor {
|
||||
private sealed trait CheckRateLimitsCommands extends Command
|
||||
private case class PendingChannelsRateLimiterResponse(response: PendingChannelsRateLimiter.Response) extends CheckRateLimitsCommands
|
||||
|
||||
private case class WrappedPeerChannels(channels: Seq[Peer.ChannelInfo]) extends Command
|
||||
|
||||
private sealed trait QueryPluginCommands extends Command
|
||||
private case class PluginOpenChannelResponse(pluginResponse: InterceptOpenChannelResponse) extends QueryPluginCommands
|
||||
private case object PluginTimeout extends QueryPluginCommands
|
||||
@ -160,7 +162,17 @@ private class OpenChannelInterceptor(peer: ActorRef[Any],
|
||||
val dualFunded = Features.canUseFeature(request.localFeatures, request.remoteFeatures, Features.DualFunding)
|
||||
val upfrontShutdownScript = Features.canUseFeature(request.localFeatures, request.remoteFeatures, Features.UpfrontShutdownScript)
|
||||
val localParams = createLocalParams(nodeParams, request.localFeatures, upfrontShutdownScript, channelType, isChannelOpener = false, dualFunded = dualFunded, request.fundingAmount, disableMaxHtlcValueInFlight = false)
|
||||
checkRateLimits(request, channelType, localParams)
|
||||
// We only accept paying the commit fees if:
|
||||
// - our peer supports on-the-fly funding, indicating that they're a mobile wallet
|
||||
// - they are purchasing liquidity for this channel
|
||||
val nonInitiatorPaysCommitTxFees = request.channelFlags.nonInitiatorPaysCommitFees &&
|
||||
Features.canUseFeature(request.localFeatures, request.remoteFeatures, Features.OnTheFlyFunding) &&
|
||||
request.open.fold(_ => false, _.requestFunding_opt.isDefined)
|
||||
if (nonInitiatorPaysCommitTxFees) {
|
||||
checkRateLimits(request, channelType, localParams.copy(paysCommitTxFees = true))
|
||||
} else {
|
||||
checkRateLimits(request, channelType, localParams)
|
||||
}
|
||||
case Left(ex) =>
|
||||
context.log.warn(s"ignoring remote channel open: ${ex.getMessage}")
|
||||
sendFailure(ex.getMessage, request)
|
||||
@ -173,13 +185,7 @@ private class OpenChannelInterceptor(peer: ActorRef[Any],
|
||||
pendingChannelsRateLimiter ! AddOrRejectChannel(adapter, request.remoteNodeId, request.temporaryChannelId)
|
||||
receiveCommandMessage[CheckRateLimitsCommands](context, "checkRateLimits") {
|
||||
case PendingChannelsRateLimiterResponse(PendingChannelsRateLimiter.AcceptOpenChannel) =>
|
||||
nodeParams.pluginOpenChannelInterceptor match {
|
||||
case Some(plugin) => queryPlugin(plugin, request, localParams, ChannelConfig.standard, channelType)
|
||||
case None =>
|
||||
// We don't honor liquidity ads for new channels: we let the node operator's plugin decide what to do.
|
||||
peer ! SpawnChannelNonInitiator(request.open, ChannelConfig.standard, channelType, addFunding_opt = None, localParams, request.peerConnection.toClassic)
|
||||
waitForRequest()
|
||||
}
|
||||
checkLiquidityAdsRequest(request, channelType, localParams)
|
||||
case PendingChannelsRateLimiterResponse(PendingChannelsRateLimiter.ChannelRateLimited) =>
|
||||
context.log.warn(s"ignoring remote channel open: rate limited")
|
||||
sendFailure("rate limit reached", request)
|
||||
@ -187,6 +193,46 @@ private class OpenChannelInterceptor(peer: ActorRef[Any],
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* If an external plugin was configured, we forward the channel request for further analysis. Otherwise, we accept
|
||||
* the channel and honor the optional liquidity request only for on-the-fly funding where we enforce a single channel.
|
||||
*/
|
||||
private def checkLiquidityAdsRequest(request: OpenChannelNonInitiator, channelType: SupportedChannelType, localParams: LocalParams): Behavior[Command] = {
|
||||
nodeParams.pluginOpenChannelInterceptor match {
|
||||
case Some(plugin) => queryPlugin(plugin, request, localParams, ChannelConfig.standard, channelType)
|
||||
case None =>
|
||||
request.open.fold(_ => None, _.requestFunding_opt) match {
|
||||
case Some(requestFunding) if Features.canUseFeature(request.localFeatures, request.remoteFeatures, Features.OnTheFlyFunding) && localParams.paysCommitTxFees =>
|
||||
val addFunding = LiquidityAds.AddFunding(requestFunding.requestedAmount, nodeParams.willFundRates_opt)
|
||||
val accept = SpawnChannelNonInitiator(request.open, ChannelConfig.standard, channelType, Some(addFunding), localParams, request.peerConnection.toClassic)
|
||||
checkNoExistingChannel(request, accept)
|
||||
case _ =>
|
||||
// We don't honor liquidity ads for new channels: node operators should use plugin for that.
|
||||
peer ! SpawnChannelNonInitiator(request.open, ChannelConfig.standard, channelType, addFunding_opt = None, localParams, request.peerConnection.toClassic)
|
||||
waitForRequest()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* In some cases we want to reject additional channels when we already have one: it is usually better to splice the
|
||||
* existing channel instead of opening another one.
|
||||
*/
|
||||
private def checkNoExistingChannel(request: OpenChannelNonInitiator, accept: SpawnChannelNonInitiator): Behavior[Command] = {
|
||||
peer ! Peer.GetPeerChannels(context.messageAdapter[Peer.PeerChannels](r => WrappedPeerChannels(r.channels)))
|
||||
receiveCommandMessage[WrappedPeerChannels](context, "checkNoExistingChannel") {
|
||||
case WrappedPeerChannels(channels) =>
|
||||
if (channels.forall(isClosing)) {
|
||||
peer ! accept
|
||||
waitForRequest()
|
||||
} else {
|
||||
context.log.warn("we already have an active channel, so we won't accept another one: our peer should request a splice instead")
|
||||
sendFailure("we already have an active channel: you should splice instead of requesting another channel", request)
|
||||
waitForRequest()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private def queryPlugin(plugin: InterceptOpenChannelPlugin, request: OpenChannelInterceptor.OpenChannelNonInitiator, localParams: LocalParams, channelConfig: ChannelConfig, channelType: SupportedChannelType): Behavior[Command] =
|
||||
Behaviors.withTimers { timers =>
|
||||
timers.startSingleTimer(PluginTimeout, pluginTimeout)
|
||||
@ -210,6 +256,23 @@ private class OpenChannelInterceptor(peer: ActorRef[Any],
|
||||
}
|
||||
}
|
||||
|
||||
private def isClosing(channel: Peer.ChannelInfo): Boolean = channel.state match {
|
||||
case CLOSED => true
|
||||
case _ => channel.data match {
|
||||
case _: TransientChannelData => false
|
||||
case _: ChannelDataWithoutCommitments => false
|
||||
case _: DATA_WAIT_FOR_FUNDING_CONFIRMED => false
|
||||
case _: DATA_WAIT_FOR_CHANNEL_READY => false
|
||||
case _: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => false
|
||||
case _: DATA_WAIT_FOR_DUAL_FUNDING_READY => false
|
||||
case _: DATA_NORMAL => false
|
||||
case _: DATA_SHUTDOWN => true
|
||||
case _: DATA_NEGOTIATING => true
|
||||
case _: DATA_CLOSING => true
|
||||
case _: DATA_WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT => true
|
||||
}
|
||||
}
|
||||
|
||||
private def sendFailure(error: String, request: OpenChannelNonInitiator): Unit = {
|
||||
peer ! Peer.OutgoingMessage(Error(request.temporaryChannelId, error), request.peerConnection.toClassic)
|
||||
context.system.eventStream ! Publish(ChannelAborted(actor.ActorRef.noSender, request.remoteNodeId, request.temporaryChannelId))
|
||||
|
@ -23,24 +23,29 @@ import akka.event.Logging.MDC
|
||||
import akka.event.{BusLogging, DiagnosticLoggingAdapter}
|
||||
import akka.util.Timeout
|
||||
import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
|
||||
import fr.acinq.bitcoin.scalacompat.{ByteVector32, Satoshi, SatoshiLong, TxId}
|
||||
import fr.acinq.bitcoin.scalacompat.{ByteVector32, Crypto, Satoshi, SatoshiLong, TxId}
|
||||
import fr.acinq.eclair.Logs.LogCategory
|
||||
import fr.acinq.eclair.NotificationsLogger.NotifyNodeOperator
|
||||
import fr.acinq.eclair._
|
||||
import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher
|
||||
import fr.acinq.eclair.blockchain.fee.FeeratePerKw
|
||||
import fr.acinq.eclair.blockchain.{CurrentFeerates, OnChainChannelFunder, OnchainPubkeyCache}
|
||||
import fr.acinq.eclair.blockchain.{CurrentBlockHeight, CurrentFeerates, OnChainChannelFunder, OnchainPubkeyCache}
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.channel.fsm.Channel
|
||||
import fr.acinq.eclair.db.PendingCommandsDb
|
||||
import fr.acinq.eclair.io.MessageRelay.Status
|
||||
import fr.acinq.eclair.io.Monitoring.Metrics
|
||||
import fr.acinq.eclair.io.Monitoring.{Metrics, Tags}
|
||||
import fr.acinq.eclair.io.OpenChannelInterceptor.{OpenChannelInitiator, OpenChannelNonInitiator}
|
||||
import fr.acinq.eclair.io.PeerConnection.KillReason
|
||||
import fr.acinq.eclair.message.OnionMessages
|
||||
import fr.acinq.eclair.payment.relay.OnTheFlyFunding
|
||||
import fr.acinq.eclair.payment.{OnTheFlyFundingPaymentRelayed, PaymentRelayed}
|
||||
import fr.acinq.eclair.remote.EclairInternalsSerializer.RemoteTypes
|
||||
import fr.acinq.eclair.router.Router
|
||||
import fr.acinq.eclair.wire.protocol
|
||||
import fr.acinq.eclair.wire.protocol.{Error, HasChannelId, HasTemporaryChannelId, LightningMessage, LiquidityAds, NodeAddress, OnionMessage, RoutingMessage, UnknownMessage, Warning}
|
||||
import fr.acinq.eclair.wire.protocol.FailureMessageCodecs.createBadOnionFailure
|
||||
import fr.acinq.eclair.wire.protocol.LiquidityAds.PaymentDetails
|
||||
import fr.acinq.eclair.wire.protocol.{Error, HasChannelId, HasTemporaryChannelId, LightningMessage, LiquidityAds, NodeAddress, OnTheFlyFundingFailureMessage, OnionMessage, OnionRoutingPacket, RoutingMessage, SpliceInit, UnknownMessage, Warning, WillAddHtlc, WillFailHtlc, WillFailMalformedHtlc}
|
||||
|
||||
/**
|
||||
* This actor represents a logical peer. There is one [[Peer]] per unique remote node id at all time.
|
||||
@ -63,13 +68,17 @@ class Peer(val nodeParams: NodeParams,
|
||||
|
||||
import Peer._
|
||||
|
||||
private var pendingOnTheFlyFunding = Map.empty[ByteVector32, OnTheFlyFunding.Pending]
|
||||
|
||||
context.system.eventStream.subscribe(self, classOf[CurrentFeerates])
|
||||
context.system.eventStream.subscribe(self, classOf[CurrentBlockHeight])
|
||||
|
||||
startWith(INSTANTIATING, Nothing)
|
||||
|
||||
when(INSTANTIATING) {
|
||||
case Event(Init(storedChannels), _) =>
|
||||
val channels = storedChannels.map { state =>
|
||||
case Event(init: Init, _) =>
|
||||
pendingOnTheFlyFunding = init.pendingOnTheFlyFunding
|
||||
val channels = init.storedChannels.map { state =>
|
||||
val channel = spawnChannel()
|
||||
channel ! INPUT_RESTORED(state)
|
||||
FinalChannelId(state.channelId) -> channel
|
||||
@ -91,10 +100,10 @@ class Peer(val nodeParams: NodeParams,
|
||||
val channelIds = d.channels.filter(_._2 == actor).keys
|
||||
log.info(s"channel closed: channelId=${channelIds.mkString("/")}")
|
||||
val channels1 = d.channels -- channelIds
|
||||
if (channels1.isEmpty) {
|
||||
if (channels1.isEmpty && !pendingSignedOnTheFlyFunding()) {
|
||||
log.info("that was the last open channel")
|
||||
context.system.eventStream.publish(LastChannelClosed(self, remoteNodeId))
|
||||
// we have no existing channels, we can forget about this peer
|
||||
// We have no existing channels or pending signed transaction, we can forget about this peer.
|
||||
stopPeer()
|
||||
} else {
|
||||
stay() using d.copy(channels = channels1)
|
||||
@ -104,8 +113,8 @@ class Peer(val nodeParams: NodeParams,
|
||||
Logs.withMdc(diagLog)(Logs.mdc(category_opt = Some(Logs.LogCategory.CONNECTION))) {
|
||||
log.debug("connection lost while negotiating connection")
|
||||
}
|
||||
if (d.channels.isEmpty) {
|
||||
// we have no existing channels, we can forget about this peer
|
||||
if (d.channels.isEmpty && !pendingSignedOnTheFlyFunding()) {
|
||||
// We have no existing channels or pending signed transaction, we can forget about this peer.
|
||||
stopPeer()
|
||||
} else {
|
||||
stay()
|
||||
@ -205,23 +214,153 @@ class Peer(val nodeParams: NodeParams,
|
||||
case Event(SpawnChannelNonInitiator(open, channelConfig, channelType, addFunding_opt, localParams, peerConnection), d: ConnectedData) =>
|
||||
val temporaryChannelId = open.fold(_.temporaryChannelId, _.temporaryChannelId)
|
||||
if (peerConnection == d.peerConnection) {
|
||||
val channel = spawnChannel()
|
||||
log.info(s"accepting a new channel with type=$channelType temporaryChannelId=$temporaryChannelId localParams=$localParams")
|
||||
open match {
|
||||
case Left(open) =>
|
||||
channel ! INPUT_INIT_CHANNEL_NON_INITIATOR(open.temporaryChannelId, None, dualFunded = false, None, localParams, d.peerConnection, d.remoteInit, channelConfig, channelType)
|
||||
channel ! open
|
||||
case Right(open) =>
|
||||
channel ! INPUT_INIT_CHANNEL_NON_INITIATOR(open.temporaryChannelId, addFunding_opt, dualFunded = true, None, localParams, d.peerConnection, d.remoteInit, channelConfig, channelType)
|
||||
channel ! open
|
||||
OnTheFlyFunding.validateOpen(open, pendingOnTheFlyFunding) match {
|
||||
case reject: OnTheFlyFunding.ValidationResult.Reject =>
|
||||
log.warning("rejecting on-the-fly channel: {}", reject.cancel.toAscii)
|
||||
self ! Peer.OutgoingMessage(reject.cancel, d.peerConnection)
|
||||
cancelUnsignedOnTheFlyFunding(reject.paymentHashes)
|
||||
context.system.eventStream.publish(ChannelAborted(ActorRef.noSender, remoteNodeId, temporaryChannelId))
|
||||
stay()
|
||||
case accept: OnTheFlyFunding.ValidationResult.Accept =>
|
||||
val channel = spawnChannel()
|
||||
log.info(s"accepting a new channel with type=$channelType temporaryChannelId=$temporaryChannelId localParams=$localParams")
|
||||
open match {
|
||||
case Left(open) =>
|
||||
channel ! INPUT_INIT_CHANNEL_NON_INITIATOR(open.temporaryChannelId, None, dualFunded = false, None, requireConfirmedInputs = false, localParams, d.peerConnection, d.remoteInit, channelConfig, channelType)
|
||||
channel ! open
|
||||
case Right(open) =>
|
||||
val requireConfirmedInputs = nodeParams.channelConf.requireConfirmedInputsForDualFunding
|
||||
channel ! INPUT_INIT_CHANNEL_NON_INITIATOR(open.temporaryChannelId, addFunding_opt, dualFunded = true, None, requireConfirmedInputs, localParams, d.peerConnection, d.remoteInit, channelConfig, channelType)
|
||||
channel ! open
|
||||
}
|
||||
fulfillOnTheFlyFundingHtlcs(accept.preimages)
|
||||
stay() using d.copy(channels = d.channels + (TemporaryChannelId(temporaryChannelId) -> channel))
|
||||
}
|
||||
stay() using d.copy(channels = d.channels + (TemporaryChannelId(temporaryChannelId) -> channel))
|
||||
} else {
|
||||
log.warning("ignoring open_channel request that reconnected during channel intercept, temporaryChannelId={}", temporaryChannelId)
|
||||
context.system.eventStream.publish(ChannelAborted(ActorRef.noSender, remoteNodeId, temporaryChannelId))
|
||||
stay()
|
||||
}
|
||||
|
||||
case Event(cmd: ProposeOnTheFlyFunding, d: ConnectedData) if !d.remoteFeatures.hasFeature(Features.OnTheFlyFunding) =>
|
||||
cmd.replyTo ! ProposeOnTheFlyFundingResponse.NotAvailable("peer does not support on-the-fly funding")
|
||||
stay()
|
||||
|
||||
case Event(cmd: ProposeOnTheFlyFunding, d: ConnectedData) =>
|
||||
// We send the funding proposal to our peer, and report it to the sender.
|
||||
val htlc = WillAddHtlc(nodeParams.chainHash, randomBytes32(), cmd.amount, cmd.paymentHash, cmd.expiry, cmd.onion, cmd.nextBlindingKey_opt)
|
||||
cmd.replyTo ! ProposeOnTheFlyFundingResponse.Proposed
|
||||
// We update our list of pending proposals for that payment.
|
||||
val pending = pendingOnTheFlyFunding.get(htlc.paymentHash) match {
|
||||
case Some(pending) =>
|
||||
pending.status match {
|
||||
case status: OnTheFlyFunding.Status.Proposed =>
|
||||
self ! Peer.OutgoingMessage(htlc, d.peerConnection)
|
||||
// We extend the previous timer.
|
||||
status.timer.cancel()
|
||||
val timer = context.system.scheduler.scheduleOnce(nodeParams.onTheFlyFundingConfig.proposalTimeout, self, OnTheFlyFundingTimeout(cmd.paymentHash))(context.dispatcher)
|
||||
pending.copy(
|
||||
proposed = pending.proposed :+ OnTheFlyFunding.Proposal(htlc, cmd.upstream),
|
||||
status = OnTheFlyFunding.Status.Proposed(timer)
|
||||
)
|
||||
case status: OnTheFlyFunding.Status.Funded =>
|
||||
log.info("received extra payment for on-the-fly funding that has already been funded with txId={} (payment_hash={}, amount={})", status.txId, cmd.paymentHash, cmd.amount)
|
||||
pending.copy(proposed = pending.proposed :+ OnTheFlyFunding.Proposal(htlc, cmd.upstream))
|
||||
}
|
||||
case None =>
|
||||
self ! Peer.OutgoingMessage(htlc, d.peerConnection)
|
||||
Metrics.OnTheFlyFunding.withTag(Tags.OnTheFlyFundingState, Tags.OnTheFlyFundingStates.Proposed).increment()
|
||||
val timer = context.system.scheduler.scheduleOnce(nodeParams.onTheFlyFundingConfig.proposalTimeout, self, OnTheFlyFundingTimeout(cmd.paymentHash))(context.dispatcher)
|
||||
OnTheFlyFunding.Pending(Seq(OnTheFlyFunding.Proposal(htlc, cmd.upstream)), OnTheFlyFunding.Status.Proposed(timer))
|
||||
}
|
||||
pendingOnTheFlyFunding += (htlc.paymentHash -> pending)
|
||||
stay()
|
||||
|
||||
case Event(msg: OnTheFlyFundingFailureMessage, d: ConnectedData) =>
|
||||
pendingOnTheFlyFunding.get(msg.paymentHash) match {
|
||||
case Some(pending) =>
|
||||
pending.status match {
|
||||
case status: OnTheFlyFunding.Status.Proposed =>
|
||||
pending.proposed.find(_.htlc.id == msg.id) match {
|
||||
case Some(htlc) =>
|
||||
val failure = msg match {
|
||||
case msg: WillFailHtlc => Left(msg.reason)
|
||||
case msg: WillFailMalformedHtlc => Right(createBadOnionFailure(msg.onionHash, msg.failureCode))
|
||||
}
|
||||
htlc.createFailureCommands(Some(failure)).foreach { case (channelId, cmd) => PendingCommandsDb.safeSend(register, nodeParams.db.pendingCommands, channelId, cmd) }
|
||||
val proposed1 = pending.proposed.filterNot(_.htlc.id == msg.id)
|
||||
if (proposed1.isEmpty) {
|
||||
Metrics.OnTheFlyFunding.withTag(Tags.OnTheFlyFundingState, Tags.OnTheFlyFundingStates.Rejected).increment()
|
||||
status.timer.cancel()
|
||||
pendingOnTheFlyFunding -= msg.paymentHash
|
||||
} else {
|
||||
pendingOnTheFlyFunding += (msg.paymentHash -> pending.copy(proposed = proposed1))
|
||||
}
|
||||
case None =>
|
||||
log.warning("ignoring will_fail_htlc: no matching proposal for id={}", msg.id)
|
||||
self ! Peer.OutgoingMessage(Warning(s"ignoring will_fail_htlc: no matching proposal for id=${msg.id}"), d.peerConnection)
|
||||
}
|
||||
case status: OnTheFlyFunding.Status.Funded =>
|
||||
log.warning("ignoring will_fail_htlc: on-the-fly funding already signed with txId={}", status.txId)
|
||||
self ! Peer.OutgoingMessage(Warning(s"ignoring will_fail_htlc: on-the-fly funding already signed with txId=${status.txId}"), d.peerConnection)
|
||||
}
|
||||
case None =>
|
||||
log.warning("ignoring will_fail_htlc: no matching proposal for payment_hash={}", msg.paymentHash)
|
||||
self ! Peer.OutgoingMessage(Warning(s"ignoring will_fail_htlc: no matching proposal for payment_hash=${msg.paymentHash}"), d.peerConnection)
|
||||
}
|
||||
stay()
|
||||
|
||||
case Event(timeout: OnTheFlyFundingTimeout, d: ConnectedData) =>
|
||||
pendingOnTheFlyFunding.get(timeout.paymentHash) match {
|
||||
case Some(pending) =>
|
||||
pending.status match {
|
||||
case _: OnTheFlyFunding.Status.Proposed =>
|
||||
log.warning("on-the-fly funding proposal timed out for payment_hash={}", timeout.paymentHash)
|
||||
pending.createFailureCommands().foreach { case (channelId, cmd) => PendingCommandsDb.safeSend(register, nodeParams.db.pendingCommands, channelId, cmd) }
|
||||
Metrics.OnTheFlyFunding.withTag(Tags.OnTheFlyFundingState, Tags.OnTheFlyFundingStates.Expired).increment()
|
||||
pendingOnTheFlyFunding -= timeout.paymentHash
|
||||
self ! Peer.OutgoingMessage(Warning(s"on-the-fly funding proposal timed out for payment_hash=${timeout.paymentHash}"), d.peerConnection)
|
||||
case status: OnTheFlyFunding.Status.Funded =>
|
||||
log.warning("ignoring on-the-fly funding proposal timeout, already funded with txId={}", status.txId)
|
||||
}
|
||||
case None =>
|
||||
log.debug("ignoring on-the-fly funding timeout for payment_hash={} (already completed)", timeout.paymentHash)
|
||||
}
|
||||
stay()
|
||||
|
||||
case Event(msg: SpliceInit, d: ConnectedData) =>
|
||||
d.channels.get(FinalChannelId(msg.channelId)) match {
|
||||
case Some(channel) =>
|
||||
OnTheFlyFunding.validateSplice(msg, nodeParams.channelConf.htlcMinimum, pendingOnTheFlyFunding) match {
|
||||
case reject: OnTheFlyFunding.ValidationResult.Reject =>
|
||||
log.warning("rejecting on-the-fly splice: {}", reject.cancel.toAscii)
|
||||
self ! Peer.OutgoingMessage(reject.cancel, d.peerConnection)
|
||||
cancelUnsignedOnTheFlyFunding(reject.paymentHashes)
|
||||
case accept: OnTheFlyFunding.ValidationResult.Accept =>
|
||||
fulfillOnTheFlyFundingHtlcs(accept.preimages)
|
||||
channel forward msg
|
||||
}
|
||||
case None => replyUnknownChannel(d.peerConnection, msg.channelId)
|
||||
}
|
||||
stay()
|
||||
|
||||
case Event(e: ChannelReadyForPayments, _: ConnectedData) =>
|
||||
pendingOnTheFlyFunding.foreach {
|
||||
case (paymentHash, pending) =>
|
||||
pending.status match {
|
||||
case _: OnTheFlyFunding.Status.Proposed => ()
|
||||
case status: OnTheFlyFunding.Status.Funded =>
|
||||
context.child(paymentHash.toHex) match {
|
||||
case Some(_) => log.debug("already relaying payment_hash={}", paymentHash)
|
||||
case None if e.fundingTxIndex < status.fundingTxIndex => log.debug("too early to relay payment_hash={}, funding not locked ({} < {})", paymentHash, e.fundingTxIndex, status.fundingTxIndex)
|
||||
case None =>
|
||||
val relayer = context.spawn(Behaviors.supervise(OnTheFlyFunding.PaymentRelayer(nodeParams, remoteNodeId, e.channelId, paymentHash)).onFailure(typed.SupervisorStrategy.stop), paymentHash.toHex)
|
||||
relayer ! OnTheFlyFunding.PaymentRelayer.TryRelay(self.toTyped, e.channel.toTyped, pending.proposed, status)
|
||||
}
|
||||
}
|
||||
}
|
||||
stay()
|
||||
|
||||
case Event(msg: HasChannelId, d: ConnectedData) =>
|
||||
d.channels.get(FinalChannelId(msg.channelId)) match {
|
||||
case Some(channel) => channel forward msg
|
||||
@ -257,8 +396,8 @@ class Peer(val nodeParams: NodeParams,
|
||||
Logs.withMdc(diagLog)(Logs.mdc(category_opt = Some(Logs.LogCategory.CONNECTION))) {
|
||||
log.debug("connection lost")
|
||||
}
|
||||
if (d.channels.isEmpty) {
|
||||
// we have no existing channels, we can forget about this peer
|
||||
if (d.channels.isEmpty && !pendingSignedOnTheFlyFunding()) {
|
||||
// We have no existing channels or pending signed transaction, we can forget about this peer.
|
||||
stopPeer()
|
||||
} else {
|
||||
d.channels.values.toSet[ActorRef].foreach(_ ! INPUT_DISCONNECTED) // we deduplicate with toSet because there might be two entries per channel (tmp id and final id)
|
||||
@ -325,6 +464,10 @@ class Peer(val nodeParams: NodeParams,
|
||||
sender() ! Status.Failure(new RuntimeException("not connected"))
|
||||
stay()
|
||||
|
||||
case Event(r: Peer.ProposeOnTheFlyFunding, _) =>
|
||||
r.replyTo ! ProposeOnTheFlyFundingResponse.NotAvailable("peer not connected")
|
||||
stay()
|
||||
|
||||
case Event(Disconnect(nodeId, replyTo_opt), _) =>
|
||||
val replyTo = replyTo_opt.getOrElse(sender().toTyped)
|
||||
replyTo ! NotConnected(nodeId)
|
||||
@ -332,10 +475,11 @@ class Peer(val nodeParams: NodeParams,
|
||||
|
||||
case Event(r: GetPeerInfo, d) =>
|
||||
val replyTo = r.replyTo.getOrElse(sender().toTyped)
|
||||
replyTo ! PeerInfo(self, remoteNodeId, stateName, d match {
|
||||
case c: ConnectedData => Some(c.address)
|
||||
case _ => None
|
||||
}, d.channels.values.toSet)
|
||||
val peerInfo = d match {
|
||||
case c: ConnectedData => PeerInfo(self, remoteNodeId, stateName, Some(c.remoteFeatures), Some(c.address), c.channels.values.toSet)
|
||||
case _ => PeerInfo(self, remoteNodeId, stateName, None, None, d.channels.values.toSet)
|
||||
}
|
||||
replyTo ! peerInfo
|
||||
stay()
|
||||
|
||||
case Event(r: GetPeerChannels, d) =>
|
||||
@ -354,6 +498,111 @@ class Peer(val nodeParams: NodeParams,
|
||||
}
|
||||
stay()
|
||||
|
||||
case Event(current: CurrentBlockHeight, d) =>
|
||||
// If we have pending will_add_htlc that are timing out, it doesn't make any sense to keep them, even if we have
|
||||
// already funded the corresponding channel: our peer will force-close if we relay them.
|
||||
// Our only option is to fail the upstream HTLCs to ensure that the upstream channels don't force-close.
|
||||
// Note that we won't be paid for the liquidity we've provided, but we don't have a choice.
|
||||
val expired = pendingOnTheFlyFunding.filter {
|
||||
case (_, pending) => pending.proposed.exists(_.htlc.expiry.blockHeight <= current.blockHeight)
|
||||
}
|
||||
expired.foreach {
|
||||
case (paymentHash, pending) =>
|
||||
log.warning("will_add_htlc expired for payment_hash={}, our peer may be malicious", paymentHash)
|
||||
Metrics.OnTheFlyFunding.withTag(Tags.OnTheFlyFundingState, Tags.OnTheFlyFundingStates.Timeout).increment()
|
||||
pending.createFailureCommands().foreach { case (channelId, cmd) => PendingCommandsDb.safeSend(register, nodeParams.db.pendingCommands, channelId, cmd) }
|
||||
}
|
||||
expired.foreach {
|
||||
case (paymentHash, pending) => pending.status match {
|
||||
case _: OnTheFlyFunding.Status.Proposed => ()
|
||||
case _: OnTheFlyFunding.Status.Funded => nodeParams.db.liquidity.removePendingOnTheFlyFunding(remoteNodeId, paymentHash)
|
||||
}
|
||||
}
|
||||
pendingOnTheFlyFunding = pendingOnTheFlyFunding.removedAll(expired.keys)
|
||||
d match {
|
||||
case d: DisconnectedData if d.channels.isEmpty && pendingOnTheFlyFunding.isEmpty => stopPeer()
|
||||
case _ => stay()
|
||||
}
|
||||
|
||||
case Event(e: LiquidityPurchaseSigned, _: ConnectedData) =>
|
||||
// We signed a liquidity purchase from our peer. At that point we're not 100% sure yet it will succeed: if
|
||||
// we disconnect before our peer sends their signature, the funding attempt may be cancelled when reconnecting.
|
||||
// If that happens, the on-the-fly proposal will stay in our state until we reach the CLTV expiry, at which
|
||||
// point we will forget it and fail the upstream HTLCs. This is also what would happen if we successfully
|
||||
// funded the channel, but it closed before we could relay the HTLCs.
|
||||
val (paymentHashes, fees) = e.purchase.paymentDetails match {
|
||||
case PaymentDetails.FromChannelBalance => (Nil, 0 sat)
|
||||
case p: PaymentDetails.FromChannelBalanceForFutureHtlc => (p.paymentHashes, 0 sat)
|
||||
case p: PaymentDetails.FromFutureHtlc => (p.paymentHashes, e.purchase.fees.total)
|
||||
case p: PaymentDetails.FromFutureHtlcWithPreimage => (p.preimages.map(preimage => Crypto.sha256(preimage)), e.purchase.fees.total)
|
||||
}
|
||||
// We split the fees across payments. We could dynamically re-split depending on whether some payments are failed
|
||||
// instead of fulfilled, but that's overkill: if our peer fails one of those payment, they're likely malicious
|
||||
// and will fail anyway, even if we try to be clever with fees splitting.
|
||||
var remainingFees = fees.toMilliSatoshi
|
||||
pendingOnTheFlyFunding
|
||||
.filter { case (paymentHash, _) => paymentHashes.contains(paymentHash) }
|
||||
.values.toSeq
|
||||
// In case our peer goes offline, we start with payments that are as far as possible from timing out.
|
||||
.sortBy(_.expiry).reverse
|
||||
.foreach(payment => {
|
||||
payment.status match {
|
||||
case status: OnTheFlyFunding.Status.Proposed =>
|
||||
status.timer.cancel()
|
||||
val paymentFees = remainingFees.min(payment.maxFees(e.htlcMinimum))
|
||||
remainingFees -= paymentFees
|
||||
log.info("liquidity purchase signed for payment_hash={}, waiting to relay HTLCs (txId={}, fundingTxIndex={}, fees={})", payment.paymentHash, e.txId, e.fundingTxIndex, paymentFees)
|
||||
val payment1 = payment.copy(status = OnTheFlyFunding.Status.Funded(e.channelId, e.txId, e.fundingTxIndex, paymentFees))
|
||||
Metrics.OnTheFlyFunding.withTag(Tags.OnTheFlyFundingState, Tags.OnTheFlyFundingStates.Funded).increment()
|
||||
nodeParams.db.liquidity.addPendingOnTheFlyFunding(remoteNodeId, payment1)
|
||||
pendingOnTheFlyFunding += payment.paymentHash -> payment1
|
||||
case status: OnTheFlyFunding.Status.Funded =>
|
||||
log.warning("liquidity purchase was already signed for payment_hash={} (previousTxId={}, currentTxId={})", payment.paymentHash, status.txId, e.txId)
|
||||
}
|
||||
})
|
||||
stay()
|
||||
|
||||
case Event(e: OnTheFlyFunding.PaymentRelayer.RelayResult, _) =>
|
||||
e match {
|
||||
case success: OnTheFlyFunding.PaymentRelayer.RelaySuccess =>
|
||||
pendingOnTheFlyFunding.get(success.paymentHash) match {
|
||||
case Some(pending) =>
|
||||
log.info("successfully relayed on-the-fly HTLC for payment_hash={}", success.paymentHash)
|
||||
// We've been paid for our liquidity fees: we can now fulfill upstream.
|
||||
pending.createFulfillCommands(success.preimage).foreach {
|
||||
case (channelId, cmd) => PendingCommandsDb.safeSend(register, nodeParams.db.pendingCommands, channelId, cmd)
|
||||
}
|
||||
// We emit a relay event: since we waited for on-chain funding before relaying the payment, the timestamps
|
||||
// won't be accurate, but everything else is.
|
||||
pending.proposed.foreach {
|
||||
case OnTheFlyFunding.Proposal(htlc, upstream) => upstream match {
|
||||
case _: Upstream.Local => ()
|
||||
case u: Upstream.Hot.Channel =>
|
||||
val incoming = PaymentRelayed.IncomingPart(u.add.amountMsat, u.add.channelId, u.receivedAt)
|
||||
val outgoing = PaymentRelayed.OutgoingPart(htlc.amount, success.channelId, TimestampMilli.now())
|
||||
context.system.eventStream.publish(OnTheFlyFundingPaymentRelayed(htlc.paymentHash, Seq(incoming), Seq(outgoing)))
|
||||
case u: Upstream.Hot.Trampoline =>
|
||||
val incoming = u.received.map(r => PaymentRelayed.IncomingPart(r.add.amountMsat, r.add.channelId, r.receivedAt))
|
||||
val outgoing = PaymentRelayed.OutgoingPart(htlc.amount, success.channelId, TimestampMilli.now())
|
||||
context.system.eventStream.publish(OnTheFlyFundingPaymentRelayed(htlc.paymentHash, incoming, Seq(outgoing)))
|
||||
}
|
||||
}
|
||||
Metrics.OnTheFlyFunding.withTag(Tags.OnTheFlyFundingState, Tags.OnTheFlyFundingStates.RelaySucceeded).increment()
|
||||
Metrics.OnTheFlyFundingFees.withoutTags().record(success.fees.toLong)
|
||||
nodeParams.db.liquidity.removePendingOnTheFlyFunding(remoteNodeId, success.paymentHash)
|
||||
pendingOnTheFlyFunding -= success.paymentHash
|
||||
case None => ()
|
||||
}
|
||||
stay()
|
||||
case OnTheFlyFunding.PaymentRelayer.RelayFailed(paymentHash, failure) =>
|
||||
log.warning("on-the-fly HTLC failure for payment_hash={}: {}", paymentHash, failure.toString)
|
||||
Metrics.OnTheFlyFunding.withTag(Tags.OnTheFlyFundingState, Tags.OnTheFlyFundingStates.relayFailed(failure)).increment()
|
||||
// We don't give up yet by relaying the failure upstream: we may have simply been disconnected, or the added
|
||||
// liquidity may have been consumed by concurrent HTLCs. We'll retry at the next reconnection with that peer
|
||||
// or after the next splice, and will only give up when the outgoing will_add_htlc timeout.
|
||||
stay()
|
||||
}
|
||||
|
||||
case Event(_: Peer.OutgoingMessage, _) => stay() // we got disconnected or reconnected and this message was for the previous connection
|
||||
|
||||
case Event(RelayOnionMessage(messageId, _, replyTo_opt), _) =>
|
||||
@ -373,9 +622,11 @@ class Peer(val nodeParams: NodeParams,
|
||||
context.system.eventStream.publish(PeerConnected(self, remoteNodeId, nextStateData.asInstanceOf[Peer.ConnectedData].connectionInfo))
|
||||
case CONNECTED -> CONNECTED => // connection switch
|
||||
context.system.eventStream.publish(PeerConnected(self, remoteNodeId, nextStateData.asInstanceOf[Peer.ConnectedData].connectionInfo))
|
||||
cancelUnsignedOnTheFlyFunding()
|
||||
case CONNECTED -> DISCONNECTED =>
|
||||
Metrics.PeersConnected.withoutTags().decrement()
|
||||
context.system.eventStream.publish(PeerDisconnected(self, remoteNodeId))
|
||||
cancelUnsignedOnTheFlyFunding()
|
||||
}
|
||||
|
||||
onTermination {
|
||||
@ -427,11 +678,50 @@ class Peer(val nodeParams: NodeParams,
|
||||
self ! Peer.OutgoingMessage(msg, peerConnection)
|
||||
}
|
||||
|
||||
private def cancelUnsignedOnTheFlyFunding(): Unit = cancelUnsignedOnTheFlyFunding(pendingOnTheFlyFunding.keySet)
|
||||
|
||||
private def cancelUnsignedOnTheFlyFunding(paymentHashes: Set[ByteVector32]): Unit = {
|
||||
val unsigned = pendingOnTheFlyFunding.filter {
|
||||
case (paymentHash, pending) if paymentHashes.contains(paymentHash) =>
|
||||
pending.status match {
|
||||
case status: OnTheFlyFunding.Status.Proposed =>
|
||||
status.timer.cancel()
|
||||
true
|
||||
case _: OnTheFlyFunding.Status.Funded => false
|
||||
}
|
||||
case _ => false
|
||||
}
|
||||
unsigned.foreach {
|
||||
case (paymentHash, pending) =>
|
||||
log.info("cancelling on-the-fly funding for payment_hash={}", paymentHash)
|
||||
pending.createFailureCommands().foreach { case (channelId, cmd) => PendingCommandsDb.safeSend(register, nodeParams.db.pendingCommands, channelId, cmd) }
|
||||
}
|
||||
pendingOnTheFlyFunding = pendingOnTheFlyFunding.removedAll(unsigned.keys)
|
||||
}
|
||||
|
||||
private def fulfillOnTheFlyFundingHtlcs(preimages: Set[ByteVector32]): Unit = {
|
||||
preimages.foreach(preimage => pendingOnTheFlyFunding.get(Crypto.sha256(preimage)) match {
|
||||
case Some(pending) => pending.createFulfillCommands(preimage).foreach { case (channelId, cmd) => PendingCommandsDb.safeSend(register, nodeParams.db.pendingCommands, channelId, cmd) }
|
||||
case None => ()
|
||||
})
|
||||
}
|
||||
|
||||
/** Return true if we have signed on-the-fly funding transactions and haven't settled the corresponding HTLCs yet. */
|
||||
private def pendingSignedOnTheFlyFunding(): Boolean = {
|
||||
pendingOnTheFlyFunding.exists {
|
||||
case (_, pending) => pending.status match {
|
||||
case _: OnTheFlyFunding.Status.Proposed => false
|
||||
case _: OnTheFlyFunding.Status.Funded => true
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// resume the openChannelInterceptor in case of failure, we always want the open channel request to succeed or fail
|
||||
private val openChannelInterceptor = context.spawnAnonymous(Behaviors.supervise(OpenChannelInterceptor(context.self.toTyped, nodeParams, remoteNodeId, wallet, pendingChannelsRateLimiter)).onFailure(typed.SupervisorStrategy.resume))
|
||||
|
||||
private def stopPeer(): State = {
|
||||
log.info("removing peer from db")
|
||||
cancelUnsignedOnTheFlyFunding()
|
||||
nodeParams.db.peers.removePeer(remoteNodeId)
|
||||
stop(FSM.Normal)
|
||||
}
|
||||
@ -507,7 +797,7 @@ object Peer {
|
||||
case object DISCONNECTED extends State
|
||||
case object CONNECTED extends State
|
||||
|
||||
case class Init(storedChannels: Set[PersistentChannelData])
|
||||
case class Init(storedChannels: Set[PersistentChannelData], pendingOnTheFlyFunding: Map[ByteVector32, OnTheFlyFunding.Pending])
|
||||
case class Connect(nodeId: PublicKey, address_opt: Option[NodeAddress], replyTo: ActorRef, isPersistent: Boolean) {
|
||||
def uri: Option[NodeURI] = address_opt.map(NodeURI(nodeId, _))
|
||||
}
|
||||
@ -561,9 +851,23 @@ object Peer {
|
||||
case class SpawnChannelInitiator(replyTo: akka.actor.typed.ActorRef[OpenChannelResponse], cmd: Peer.OpenChannel, channelConfig: ChannelConfig, channelType: SupportedChannelType, localParams: LocalParams)
|
||||
case class SpawnChannelNonInitiator(open: Either[protocol.OpenChannel, protocol.OpenDualFundedChannel], channelConfig: ChannelConfig, channelType: SupportedChannelType, addFunding_opt: Option[LiquidityAds.AddFunding], localParams: LocalParams, peerConnection: ActorRef)
|
||||
|
||||
/** If [[Features.OnTheFlyFunding]] is supported and we're connected, relay a funding proposal to our peer. */
|
||||
case class ProposeOnTheFlyFunding(replyTo: typed.ActorRef[ProposeOnTheFlyFundingResponse], amount: MilliSatoshi, paymentHash: ByteVector32, expiry: CltvExpiry, onion: OnionRoutingPacket, nextBlindingKey_opt: Option[PublicKey], upstream: Upstream.Hot)
|
||||
|
||||
sealed trait ProposeOnTheFlyFundingResponse
|
||||
object ProposeOnTheFlyFundingResponse {
|
||||
case object Proposed extends ProposeOnTheFlyFundingResponse
|
||||
case class NotAvailable(reason: String) extends ProposeOnTheFlyFundingResponse
|
||||
}
|
||||
|
||||
/** We signed a funding transaction where our peer purchased some liquidity. */
|
||||
case class LiquidityPurchaseSigned(channelId: ByteVector32, txId: TxId, fundingTxIndex: Long, htlcMinimum: MilliSatoshi, purchase: LiquidityAds.Purchase)
|
||||
|
||||
case class OnTheFlyFundingTimeout(paymentHash: ByteVector32)
|
||||
|
||||
case class GetPeerInfo(replyTo: Option[typed.ActorRef[PeerInfoResponse]])
|
||||
sealed trait PeerInfoResponse { def nodeId: PublicKey }
|
||||
case class PeerInfo(peer: ActorRef, nodeId: PublicKey, state: State, address: Option[NodeAddress], channels: Set[ActorRef]) extends PeerInfoResponse
|
||||
case class PeerInfo(peer: ActorRef, nodeId: PublicKey, state: State, features: Option[Features[InitFeature]], address: Option[NodeAddress], channels: Set[ActorRef]) extends PeerInfoResponse
|
||||
case class PeerNotFound(nodeId: PublicKey) extends PeerInfoResponse with DisconnectResponse { override def toString: String = s"peer $nodeId not found" }
|
||||
|
||||
/** Return the peer's current channels: note that the data may change concurrently, never assume it is fully up-to-date. */
|
||||
@ -571,7 +875,6 @@ object Peer {
|
||||
case class ChannelInfo(channel: typed.ActorRef[Command], state: ChannelState, data: ChannelData)
|
||||
case class PeerChannels(nodeId: PublicKey, channels: Seq[ChannelInfo])
|
||||
|
||||
|
||||
case class PeerRoutingMessage(peerConnection: ActorRef, remoteNodeId: PublicKey, message: RoutingMessage) extends RemoteTypes
|
||||
|
||||
/**
|
||||
|
@ -23,7 +23,7 @@ import akka.actor.typed.scaladsl.{ActorContext, Behaviors, TimerScheduler}
|
||||
import akka.actor.typed.{ActorRef, Behavior, SupervisorStrategy}
|
||||
import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
|
||||
import fr.acinq.eclair.blockchain.CurrentBlockHeight
|
||||
import fr.acinq.eclair.{BlockHeight, Logs, channel}
|
||||
import fr.acinq.eclair.{BlockHeight, Features, InitFeature, Logs, channel}
|
||||
|
||||
import scala.concurrent.duration.{DurationInt, FiniteDuration}
|
||||
|
||||
@ -107,7 +107,7 @@ object PeerReadyNotifier {
|
||||
private case object PeerNotConnected extends Command
|
||||
private case object PeerConnected extends Command
|
||||
private case object PeerDisconnected extends Command
|
||||
private case class WrappedPeerInfo(peer: ActorRef[Peer.GetPeerChannels], channelCount: Int) extends Command
|
||||
private case class WrappedPeerInfo(peer: ActorRef[Peer.GetPeerChannels], remoteFeatures: Features[InitFeature], channelCount: Int) extends Command
|
||||
private case class NewBlockNotTimedOut(currentBlockHeight: BlockHeight) extends Command
|
||||
private case object CheckChannelsReady extends Command
|
||||
private case class WrappedPeerChannels(wrapped: Peer.PeerChannels) extends Command
|
||||
@ -115,7 +115,7 @@ object PeerReadyNotifier {
|
||||
private case object ToBeIgnored extends Command
|
||||
|
||||
sealed trait Result { def remoteNodeId: PublicKey }
|
||||
case class PeerReady(remoteNodeId: PublicKey, peer: akka.actor.ActorRef, channelInfos: Seq[Peer.ChannelInfo]) extends Result { val channelsCount: Int = channelInfos.size }
|
||||
case class PeerReady(remoteNodeId: PublicKey, peer: akka.actor.ActorRef, remoteFeatures: Features[InitFeature], channelInfos: Seq[Peer.ChannelInfo]) extends Result { val channelsCount: Int = channelInfos.size }
|
||||
case class PeerUnavailable(remoteNodeId: PublicKey) extends Result
|
||||
|
||||
private case object ChannelsReadyTimerKey
|
||||
@ -243,7 +243,7 @@ private class PeerReadyNotifier(replyTo: ActorRef[PeerReadyNotifier.Result],
|
||||
// In that case we still want to wait for a connection, because we may want to open a channel to them.
|
||||
case _: Peer.PeerNotFound => PeerNotConnected
|
||||
case info: Peer.PeerInfo if info.state != Peer.CONNECTED => PeerNotConnected
|
||||
case info: Peer.PeerInfo => WrappedPeerInfo(info.peer.toTyped, info.channels.size)
|
||||
case info: Peer.PeerInfo => WrappedPeerInfo(info.peer.toTyped, info.features.getOrElse(Features.empty), info.channels.size)
|
||||
}
|
||||
// We check whether the peer is already connected.
|
||||
switchboard ! Switchboard.GetPeerInfo(peerInfoAdapter, remoteNodeId)
|
||||
@ -256,14 +256,14 @@ private class PeerReadyNotifier(replyTo: ActorRef[PeerReadyNotifier.Result],
|
||||
Behaviors.same
|
||||
case PeerDisconnected =>
|
||||
Behaviors.same
|
||||
case WrappedPeerInfo(peer, channelCount) =>
|
||||
case WrappedPeerInfo(peer, remoteFeatures, channelCount) =>
|
||||
if (channelCount == 0) {
|
||||
log.info("peer is ready with no channels")
|
||||
replyTo ! PeerReady(remoteNodeId, peer.toClassic, Seq.empty)
|
||||
replyTo ! PeerReady(remoteNodeId, peer.toClassic, remoteFeatures, Seq.empty)
|
||||
Behaviors.stopped
|
||||
} else {
|
||||
log.debug("peer is connected with {} channels", channelCount)
|
||||
waitForChannelsReady(peer, switchboard)
|
||||
waitForChannelsReady(peer, switchboard, remoteFeatures)
|
||||
}
|
||||
case NewBlockNotTimedOut(currentBlockHeight) =>
|
||||
log.debug("waiting for peer to connect at block {}", currentBlockHeight)
|
||||
@ -277,7 +277,7 @@ private class PeerReadyNotifier(replyTo: ActorRef[PeerReadyNotifier.Result],
|
||||
}
|
||||
}
|
||||
|
||||
private def waitForChannelsReady(peer: ActorRef[Peer.GetPeerChannels], switchboard: ActorRef[Switchboard.GetPeerInfo]): Behavior[Command] = {
|
||||
private def waitForChannelsReady(peer: ActorRef[Peer.GetPeerChannels], switchboard: ActorRef[Switchboard.GetPeerInfo], remoteFeatures: Features[InitFeature]): Behavior[Command] = {
|
||||
timers.startTimerWithFixedDelay(ChannelsReadyTimerKey, CheckChannelsReady, initialDelay = 50 millis, delay = 1 second)
|
||||
Behaviors.receiveMessagePartial {
|
||||
case CheckChannelsReady =>
|
||||
@ -286,7 +286,7 @@ private class PeerReadyNotifier(replyTo: ActorRef[PeerReadyNotifier.Result],
|
||||
Behaviors.same
|
||||
case WrappedPeerChannels(peerChannels) =>
|
||||
if (peerChannels.channels.map(_.state).forall(isChannelReady)) {
|
||||
replyTo ! PeerReady(remoteNodeId, peer.toClassic, peerChannels.channels)
|
||||
replyTo ! PeerReady(remoteNodeId, peer.toClassic, remoteFeatures, peerChannels.channels)
|
||||
Behaviors.stopped
|
||||
} else {
|
||||
log.debug("peer has {} channels that are not ready", peerChannels.channels.count(s => !isChannelReady(s.state)))
|
||||
|
@ -20,12 +20,14 @@ import akka.actor.typed.receptionist.{Receptionist, ServiceKey}
|
||||
import akka.actor.typed.scaladsl.Behaviors
|
||||
import akka.actor.typed.scaladsl.adapter.{ClassicActorContextOps, ClassicActorRefOps, ClassicActorSystemOps, TypedActorRefOps}
|
||||
import akka.actor.{Actor, ActorContext, ActorLogging, ActorRef, OneForOneStrategy, Props, Stash, Status, SupervisorStrategy, typed}
|
||||
import fr.acinq.bitcoin.scalacompat.ByteVector32
|
||||
import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
|
||||
import fr.acinq.eclair.blockchain.OnchainPubkeyCache
|
||||
import fr.acinq.eclair.channel.Helpers.Closing
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.io.IncomingConnectionsTracker.TrackIncomingConnection
|
||||
import fr.acinq.eclair.io.Peer.{PeerInfoResponse, PeerNotFound}
|
||||
import fr.acinq.eclair.payment.relay.OnTheFlyFunding
|
||||
import fr.acinq.eclair.remote.EclairInternalsSerializer.RemoteTypes
|
||||
import fr.acinq.eclair.router.Router
|
||||
import fr.acinq.eclair.router.Router.RouterConf
|
||||
@ -57,11 +59,17 @@ class Switchboard(nodeParams: NodeParams, peerFactory: Switchboard.PeerFactory)
|
||||
log.info(s"closing channel ${c.channelId}")
|
||||
nodeParams.db.channels.removeChannel(c.channelId)
|
||||
})
|
||||
val peerChannels = channels.groupBy(_.remoteNodeId)
|
||||
peerChannels.foreach { case (remoteNodeId, states) => createOrGetPeer(remoteNodeId, offlineChannels = states.toSet) }
|
||||
log.info("restoring {} peer(s) with {} channel(s)", peerChannels.size, channels.size)
|
||||
val peersWithChannels = channels.groupBy(_.remoteNodeId)
|
||||
val peersWithOnTheFlyFunding = nodeParams.db.liquidity.listPendingOnTheFlyFunding()
|
||||
peersWithChannels.foreach { case (remoteNodeId, states) => createOrGetPeer(remoteNodeId, offlineChannels = states.toSet, peersWithOnTheFlyFunding.getOrElse(remoteNodeId, Map.empty)) }
|
||||
// We must re-create peers that have a funded on-the-fly payment, even if they don't have a channel yet.
|
||||
// We will retry relaying that payment and complete the on-the-fly funding.
|
||||
(peersWithOnTheFlyFunding -- peersWithChannels.keySet).foreach {
|
||||
case (remoteNodeId, pending) => createOrGetPeer(remoteNodeId, Set.empty, pending)
|
||||
}
|
||||
log.info("restoring {} peer(s) with {} channel(s) and {} peers with pending on-the-fly funding", peersWithChannels.size, channels.size, (peersWithOnTheFlyFunding.keySet -- peersWithChannels.keySet).size)
|
||||
unstashAll()
|
||||
context.become(normal(peerChannels.keySet))
|
||||
context.become(normal(peersWithChannels.keySet))
|
||||
case _ =>
|
||||
stash()
|
||||
}
|
||||
@ -72,9 +80,11 @@ class Switchboard(nodeParams: NodeParams, peerFactory: Switchboard.PeerFactory)
|
||||
sender() ! Status.Failure(new RuntimeException("cannot open connection with oneself"))
|
||||
|
||||
case Peer.Connect(nodeId, address_opt, replyTo, isPersistent) =>
|
||||
// we create a peer if it doesn't exist: when the peer doesn't exist, we can be sure that we don't have channels,
|
||||
// We create a peer if it doesn't exist: when the peer doesn't exist, we can be sure that we don't have channels,
|
||||
// otherwise the peer would have been created during the initialization step.
|
||||
val peer = createOrGetPeer(nodeId, offlineChannels = Set.empty)
|
||||
// We're also sure that we don't have pending on-the-fly funding attempts, otherwise the peer would have also
|
||||
// been created during the initialization step.
|
||||
val peer = createOrGetPeer(nodeId, offlineChannels = Set.empty, pendingOnTheFlyFunding = Map.empty)
|
||||
val c = if (replyTo == ActorRef.noSender) {
|
||||
Peer.Connect(nodeId, address_opt, sender(), isPersistent)
|
||||
} else {
|
||||
@ -96,8 +106,8 @@ class Switchboard(nodeParams: NodeParams, peerFactory: Switchboard.PeerFactory)
|
||||
}
|
||||
|
||||
case authenticated: PeerConnection.Authenticated =>
|
||||
// if this is an incoming connection, we might not yet have created the peer
|
||||
val peer = createOrGetPeer(authenticated.remoteNodeId, offlineChannels = Set.empty)
|
||||
// If this is an incoming connection, we might not yet have created the peer.
|
||||
val peer = createOrGetPeer(authenticated.remoteNodeId, offlineChannels = Set.empty, pendingOnTheFlyFunding = Map.empty)
|
||||
val features = nodeParams.initFeaturesFor(authenticated.remoteNodeId)
|
||||
val hasChannels = peersWithChannels.contains(authenticated.remoteNodeId)
|
||||
// if the peer is whitelisted, we sync with them, otherwise we only sync with peers with whom we have at least one channel
|
||||
@ -134,14 +144,14 @@ class Switchboard(nodeParams: NodeParams, peerFactory: Switchboard.PeerFactory)
|
||||
|
||||
def createPeer(remoteNodeId: PublicKey): ActorRef = peerFactory.spawn(context, remoteNodeId)
|
||||
|
||||
def createOrGetPeer(remoteNodeId: PublicKey, offlineChannels: Set[PersistentChannelData]): ActorRef = {
|
||||
def createOrGetPeer(remoteNodeId: PublicKey, offlineChannels: Set[PersistentChannelData], pendingOnTheFlyFunding: Map[ByteVector32, OnTheFlyFunding.Pending]): ActorRef = {
|
||||
getPeer(remoteNodeId) match {
|
||||
case Some(peer) => peer
|
||||
case None =>
|
||||
// do not count the incoming-connections-tracker child actor that is not a peer
|
||||
log.debug(s"creating new peer (current={})", context.children.size - 1)
|
||||
val peer = createPeer(remoteNodeId)
|
||||
peer ! Peer.Init(offlineChannels)
|
||||
peer ! Peer.Init(offlineChannels, pendingOnTheFlyFunding)
|
||||
peer
|
||||
}
|
||||
}
|
||||
|
@ -100,10 +100,12 @@ object Monitoring {
|
||||
object RelayType {
|
||||
val Channel = "channel"
|
||||
val Trampoline = "trampoline"
|
||||
val OnTheFly = "on-the-fly"
|
||||
|
||||
def apply(e: PaymentRelayed): String = e match {
|
||||
case _: ChannelPaymentRelayed => Channel
|
||||
case _: TrampolinePaymentRelayed => Trampoline
|
||||
case _: OnTheFlyFundingPaymentRelayed => OnTheFly
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -102,6 +102,14 @@ case class TrampolinePaymentRelayed(paymentHash: ByteVector32, incoming: Payment
|
||||
override val timestamp: TimestampMilli = settledAt
|
||||
}
|
||||
|
||||
case class OnTheFlyFundingPaymentRelayed(paymentHash: ByteVector32, incoming: PaymentRelayed.Incoming, outgoing: PaymentRelayed.Outgoing) extends PaymentRelayed {
|
||||
override val amountIn: MilliSatoshi = incoming.map(_.amount).sum
|
||||
override val amountOut: MilliSatoshi = outgoing.map(_.amount).sum
|
||||
override val startedAt: TimestampMilli = incoming.map(_.receivedAt).minOption.getOrElse(TimestampMilli.now())
|
||||
override val settledAt: TimestampMilli = outgoing.map(_.settledAt).maxOption.getOrElse(TimestampMilli.now())
|
||||
override val timestamp: TimestampMilli = settledAt
|
||||
}
|
||||
|
||||
object PaymentRelayed {
|
||||
|
||||
case class IncomingPart(amount: MilliSatoshi, channelId: ByteVector32, receivedAt: TimestampMilli)
|
||||
|
@ -290,7 +290,7 @@ object OutgoingPaymentPacket {
|
||||
payment <- recipient.buildPayloads(paymentHash, route)
|
||||
onion <- buildOnion(payment.payloads, paymentHash, Some(PaymentOnionCodecs.paymentOnionPayloadLength)) // BOLT 2 requires that associatedData == paymentHash
|
||||
} yield {
|
||||
val cmd = CMD_ADD_HTLC(origin.replyTo, payment.amount, paymentHash, payment.expiry, onion.packet, payment.outerBlinding_opt, confidence, origin, commit = true)
|
||||
val cmd = CMD_ADD_HTLC(origin.replyTo, payment.amount, paymentHash, payment.expiry, onion.packet, payment.outerBlinding_opt, confidence, fundingFee_opt = None, origin, commit = true)
|
||||
OutgoingPaymentPacket(cmd, route.hops.head.shortChannelId, onion.sharedSecrets)
|
||||
}
|
||||
}
|
||||
|
@ -26,14 +26,15 @@ import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.crypto.Sphinx
|
||||
import fr.acinq.eclair.db.PendingCommandsDb
|
||||
import fr.acinq.eclair.io.PeerReadyNotifier
|
||||
import fr.acinq.eclair.io.Peer.ProposeOnTheFlyFundingResponse
|
||||
import fr.acinq.eclair.io.{Peer, PeerReadyNotifier}
|
||||
import fr.acinq.eclair.payment.Monitoring.{Metrics, Tags}
|
||||
import fr.acinq.eclair.payment.relay.Relayer.{OutgoingChannel, OutgoingChannelParams}
|
||||
import fr.acinq.eclair.payment.{ChannelPaymentRelayed, IncomingPaymentPacket}
|
||||
import fr.acinq.eclair.wire.protocol.FailureMessageCodecs.createBadOnionFailure
|
||||
import fr.acinq.eclair.wire.protocol.PaymentOnion.IntermediatePayload
|
||||
import fr.acinq.eclair.wire.protocol._
|
||||
import fr.acinq.eclair.{Logs, NodeParams, TimestampMilli, TimestampSecond, channel, nodeFee}
|
||||
import fr.acinq.eclair.{Features, InitFeature, Logs, NodeParams, TimestampMilli, TimestampSecond, channel, nodeFee}
|
||||
|
||||
import java.util.UUID
|
||||
import java.util.concurrent.TimeUnit
|
||||
@ -48,11 +49,13 @@ object ChannelRelay {
|
||||
private case class WrappedPeerReadyResult(result: PeerReadyNotifier.Result) extends Command
|
||||
private case class WrappedForwardFailure(failure: Register.ForwardFailure[CMD_ADD_HTLC]) extends Command
|
||||
private case class WrappedAddResponse(res: CommandResponse[CMD_ADD_HTLC]) extends Command
|
||||
private case class WrappedOnTheFlyFundingResponse(result: Peer.ProposeOnTheFlyFundingResponse) extends Command
|
||||
// @formatter:on
|
||||
|
||||
// @formatter:off
|
||||
sealed trait RelayResult
|
||||
case class RelayFailure(cmdFail: CMD_FAIL_HTLC) extends RelayResult
|
||||
case class RelayNeedsFunding(nextNode: PublicKey, cmdFail: CMD_FAIL_HTLC) extends RelayResult
|
||||
case class RelaySuccess(selectedChannelId: ByteVector32, cmdAdd: CMD_ADD_HTLC) extends RelayResult
|
||||
// @formatter:on
|
||||
|
||||
@ -121,6 +124,8 @@ class ChannelRelay private(nodeParams: NodeParams,
|
||||
|
||||
private val forwardFailureAdapter = context.messageAdapter[Register.ForwardFailure[CMD_ADD_HTLC]](WrappedForwardFailure)
|
||||
private val addResponseAdapter = context.messageAdapter[CommandResponse[CMD_ADD_HTLC]](WrappedAddResponse)
|
||||
private val forwardNodeIdFailureAdapter = context.messageAdapter[Register.ForwardNodeIdFailure[Peer.ProposeOnTheFlyFunding]](_ => WrappedOnTheFlyFundingResponse(Peer.ProposeOnTheFlyFundingResponse.NotAvailable("peer not found")))
|
||||
private val onTheFlyFundingResponseAdapter = context.messageAdapter[Peer.ProposeOnTheFlyFundingResponse](WrappedOnTheFlyFundingResponse)
|
||||
|
||||
private val nextBlindingKey_opt = r.payload match {
|
||||
case payload: IntermediatePayload.ChannelRelay.Blinded => Some(payload.nextBlinding)
|
||||
@ -148,7 +153,7 @@ class ChannelRelay private(nodeParams: NodeParams,
|
||||
case Some(walletNodeId) if nodeParams.peerWakeUpConfig.enabled => wakeUp(walletNodeId)
|
||||
case _ =>
|
||||
context.self ! DoRelay
|
||||
relay(Seq.empty)
|
||||
relay(None, Seq.empty)
|
||||
}
|
||||
}
|
||||
|
||||
@ -161,13 +166,13 @@ class ChannelRelay private(nodeParams: NodeParams,
|
||||
Metrics.recordPaymentRelayFailed(Tags.FailureType.WakeUp, Tags.RelayType.Channel)
|
||||
context.log.info("rejecting htlc: failed to wake-up remote peer")
|
||||
safeSendAndStop(r.add.channelId, CMD_FAIL_HTLC(r.add.id, Right(UnknownNextPeer()), commit = true))
|
||||
case WrappedPeerReadyResult(_: PeerReadyNotifier.PeerReady) =>
|
||||
case WrappedPeerReadyResult(r: PeerReadyNotifier.PeerReady) =>
|
||||
context.self ! DoRelay
|
||||
relay(Seq.empty)
|
||||
relay(Some(r.remoteFeatures), Seq.empty)
|
||||
}
|
||||
}
|
||||
|
||||
def relay(previousFailures: Seq[PreviouslyTried]): Behavior[Command] = {
|
||||
def relay(remoteFeatures_opt: Option[Features[InitFeature]], previousFailures: Seq[PreviouslyTried]): Behavior[Command] = {
|
||||
Behaviors.receiveMessagePartial {
|
||||
case DoRelay =>
|
||||
if (previousFailures.isEmpty) {
|
||||
@ -175,20 +180,24 @@ class ChannelRelay private(nodeParams: NodeParams,
|
||||
context.log.info("relaying htlc #{} from channelId={} to requestedShortChannelId={} nextNode={}", r.add.id, r.add.channelId, requestedShortChannelId_opt, nextNodeId_opt.getOrElse(""))
|
||||
}
|
||||
context.log.debug("attempting relay previousAttempts={}", previousFailures.size)
|
||||
handleRelay(previousFailures) match {
|
||||
handleRelay(remoteFeatures_opt, previousFailures) match {
|
||||
case RelayFailure(cmdFail) =>
|
||||
Metrics.recordPaymentRelayFailed(Tags.FailureType(cmdFail), Tags.RelayType.Channel)
|
||||
context.log.info("rejecting htlc reason={}", cmdFail.reason)
|
||||
safeSendAndStop(r.add.channelId, cmdFail)
|
||||
case RelayNeedsFunding(nextNodeId, cmdFail) =>
|
||||
val cmd = Peer.ProposeOnTheFlyFunding(onTheFlyFundingResponseAdapter, r.amountToForward, r.add.paymentHash, r.outgoingCltv, r.nextPacket, nextBlindingKey_opt, upstream)
|
||||
register ! Register.ForwardNodeId(forwardNodeIdFailureAdapter, nextNodeId, cmd)
|
||||
waitForOnTheFlyFundingResponse(cmdFail)
|
||||
case RelaySuccess(selectedChannelId, cmdAdd) =>
|
||||
context.log.info("forwarding htlc #{} from channelId={} to channelId={}", r.add.id, r.add.channelId, selectedChannelId)
|
||||
register ! Register.Forward(forwardFailureAdapter, selectedChannelId, cmdAdd)
|
||||
waitForAddResponse(selectedChannelId, previousFailures)
|
||||
waitForAddResponse(selectedChannelId, remoteFeatures_opt, previousFailures)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private def waitForAddResponse(selectedChannelId: ByteVector32, previousFailures: Seq[PreviouslyTried]): Behavior[Command] =
|
||||
private def waitForAddResponse(selectedChannelId: ByteVector32, remoteFeatures_opt: Option[Features[InitFeature]], previousFailures: Seq[PreviouslyTried]): Behavior[Command] =
|
||||
Behaviors.receiveMessagePartial {
|
||||
case WrappedForwardFailure(Register.ForwardFailure(Register.Forward(_, channelId, _))) =>
|
||||
context.log.warn(s"couldn't resolve downstream channel $channelId, failing htlc #${upstream.add.id}")
|
||||
@ -199,7 +208,7 @@ class ChannelRelay private(nodeParams: NodeParams,
|
||||
case WrappedAddResponse(addFailed: RES_ADD_FAILED[_]) =>
|
||||
context.log.info("attempt failed with reason={}", addFailed.t.getClass.getSimpleName)
|
||||
context.self ! DoRelay
|
||||
relay(previousFailures :+ PreviouslyTried(selectedChannelId, addFailed))
|
||||
relay(remoteFeatures_opt, previousFailures :+ PreviouslyTried(selectedChannelId, addFailed))
|
||||
|
||||
case WrappedAddResponse(_: RES_SUCCESS[_]) =>
|
||||
context.log.debug("sent htlc to the downstream channel")
|
||||
@ -225,6 +234,21 @@ class ChannelRelay private(nodeParams: NodeParams,
|
||||
safeSendAndStop(upstream.add.channelId, cmd)
|
||||
}
|
||||
|
||||
private def waitForOnTheFlyFundingResponse(cmdFail: CMD_FAIL_HTLC): Behavior[Command] = Behaviors.receiveMessagePartial {
|
||||
case WrappedOnTheFlyFundingResponse(response) =>
|
||||
response match {
|
||||
case ProposeOnTheFlyFundingResponse.Proposed =>
|
||||
context.log.info("on-the-fly funding proposed for htlc #{} from channelId={}", r.add.id, r.add.channelId)
|
||||
// We're not responsible for the payment relay anymore: another actor will take care of relaying the payment
|
||||
// once on-the-fly funding completes.
|
||||
Behaviors.stopped
|
||||
case ProposeOnTheFlyFundingResponse.NotAvailable(reason) =>
|
||||
context.log.warn("could not propose on-the-fly funding for htlc #{} from channelId={}: {}", r.add.id, r.add.channelId, reason)
|
||||
Metrics.recordPaymentRelayFailed(Tags.FailureType(cmdFail), Tags.RelayType.Channel)
|
||||
safeSendAndStop(r.add.channelId, cmdFail)
|
||||
}
|
||||
}
|
||||
|
||||
private def safeSendAndStop(channelId: ByteVector32, cmd: channel.HtlcSettlementCommand): Behavior[Command] = {
|
||||
val toSend = cmd match {
|
||||
case _: CMD_FULFILL_HTLC => cmd
|
||||
@ -256,7 +280,7 @@ class ChannelRelay private(nodeParams: NodeParams,
|
||||
* - a CMD_FAIL_HTLC to be sent back upstream
|
||||
* - a CMD_ADD_HTLC to propagate downstream
|
||||
*/
|
||||
private def handleRelay(previousFailures: Seq[PreviouslyTried]): RelayResult = {
|
||||
private def handleRelay(remoteFeatures_opt: Option[Features[InitFeature]], previousFailures: Seq[PreviouslyTried]): RelayResult = {
|
||||
val alreadyTried = previousFailures.map(_.channelId)
|
||||
selectPreferredChannel(alreadyTried) match {
|
||||
case Some(outgoingChannel) => relayOrFail(outgoingChannel)
|
||||
@ -273,7 +297,10 @@ class ChannelRelay private(nodeParams: NodeParams,
|
||||
} else {
|
||||
CMD_FAIL_HTLC(r.add.id, Right(UnknownNextPeer()), commit = true)
|
||||
}
|
||||
RelayFailure(cmdFail)
|
||||
walletNodeId_opt match {
|
||||
case Some(walletNodeId) if shouldAttemptOnTheFlyFunding(remoteFeatures_opt, previousFailures) => RelayNeedsFunding(walletNodeId, cmdFail)
|
||||
case _ => RelayFailure(cmdFail)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -349,7 +376,7 @@ class ChannelRelay private(nodeParams: NodeParams,
|
||||
RelayFailure(CMD_FAIL_HTLC(r.add.id, Right(ChannelDisabled(update.messageFlags, update.channelFlags, Some(update))), commit = true))
|
||||
case None =>
|
||||
val origin = Origin.Hot(addResponseAdapter.toClassic, upstream)
|
||||
RelaySuccess(outgoingChannel.channelId, CMD_ADD_HTLC(addResponseAdapter.toClassic, r.amountToForward, r.add.paymentHash, r.outgoingCltv, r.nextPacket, nextBlindingKey_opt, confidence, origin, commit = true))
|
||||
RelaySuccess(outgoingChannel.channelId, CMD_ADD_HTLC(addResponseAdapter.toClassic, r.amountToForward, r.add.paymentHash, r.outgoingCltv, r.nextPacket, nextBlindingKey_opt, confidence, fundingFee_opt = None, origin, commit = true))
|
||||
}
|
||||
}
|
||||
|
||||
@ -372,6 +399,20 @@ class ChannelRelay private(nodeParams: NodeParams,
|
||||
}
|
||||
}
|
||||
|
||||
/** If we fail to relay a payment, we may want to attempt on-the-fly funding. */
|
||||
private def shouldAttemptOnTheFlyFunding(remoteFeatures_opt: Option[Features[InitFeature]], previousFailures: Seq[PreviouslyTried]): Boolean = {
|
||||
val featureOk = Features.canUseFeature(nodeParams.features.initFeatures(), remoteFeatures_opt.getOrElse(Features.empty), Features.OnTheFlyFunding)
|
||||
// If we have a channel with the next node, we only want to perform on-the-fly funding for liquidity issues.
|
||||
val liquidityIssue = previousFailures.forall {
|
||||
case PreviouslyTried(_, RES_ADD_FAILED(_, _: InsufficientFunds, _)) => true
|
||||
case _ => false
|
||||
}
|
||||
// If we have a channel with the next peer, but we skipped it because the sender is using invalid relay parameters,
|
||||
// we don't want to perform on-the-fly funding: the sender should send a valid payment first.
|
||||
val relayParamsOk = channels.values.forall(c => validateRelayParams(c).isEmpty)
|
||||
featureOk && liquidityIssue && relayParamsOk
|
||||
}
|
||||
|
||||
private def recordRelayDuration(isSuccess: Boolean): Unit =
|
||||
Metrics.RelayedPaymentDuration
|
||||
.withTag(Tags.Relay, Tags.RelayType.Channel)
|
||||
|
@ -24,9 +24,10 @@ import akka.actor.{ActorRef, typed}
|
||||
import com.softwaremill.quicklens.ModifyPimp
|
||||
import fr.acinq.bitcoin.scalacompat.ByteVector32
|
||||
import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
|
||||
import fr.acinq.eclair.channel.{CMD_FAIL_HTLC, CMD_FULFILL_HTLC, Upstream}
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.db.PendingCommandsDb
|
||||
import fr.acinq.eclair.io.PeerReadyNotifier
|
||||
import fr.acinq.eclair.io.Peer.ProposeOnTheFlyFundingResponse
|
||||
import fr.acinq.eclair.io.{Peer, PeerReadyNotifier}
|
||||
import fr.acinq.eclair.payment.IncomingPaymentPacket.NodeRelayPacket
|
||||
import fr.acinq.eclair.payment.Monitoring.{Metrics, Tags}
|
||||
import fr.acinq.eclair.payment._
|
||||
@ -37,11 +38,11 @@ import fr.acinq.eclair.payment.send.MultiPartPaymentLifecycle.{PreimageReceived,
|
||||
import fr.acinq.eclair.payment.send.PaymentInitiator.SendPaymentConfig
|
||||
import fr.acinq.eclair.payment.send.PaymentLifecycle.SendPaymentToNode
|
||||
import fr.acinq.eclair.payment.send._
|
||||
import fr.acinq.eclair.router.Router.RouteParams
|
||||
import fr.acinq.eclair.router.Router.{ChannelHop, HopRelayParams, Route, RouteParams}
|
||||
import fr.acinq.eclair.router.{BalanceTooLow, RouteNotFound}
|
||||
import fr.acinq.eclair.wire.protocol.PaymentOnion.IntermediatePayload
|
||||
import fr.acinq.eclair.wire.protocol._
|
||||
import fr.acinq.eclair.{CltvExpiry, EncodedNodeId, Features, Logs, MilliSatoshi, NodeParams, TimestampMilli, UInt64, nodeFee, randomBytes32}
|
||||
import fr.acinq.eclair.{Alias, CltvExpiry, CltvExpiryDelta, EncodedNodeId, Features, InitFeature, Logs, MilliSatoshi, MilliSatoshiLong, NodeParams, TimestampMilli, UInt64, nodeFee, randomBytes32}
|
||||
|
||||
import java.util.UUID
|
||||
import java.util.concurrent.TimeUnit
|
||||
@ -65,6 +66,7 @@ object NodeRelay {
|
||||
private case class WrappedPaymentFailed(paymentFailed: PaymentFailed) extends Command
|
||||
private case class WrappedPeerReadyResult(result: PeerReadyNotifier.Result) extends Command
|
||||
private case class WrappedResolvedPaths(resolved: Seq[ResolvedPath]) extends Command
|
||||
private case class WrappedOnTheFlyFundingResponse(result: Peer.ProposeOnTheFlyFundingResponse) extends Command
|
||||
// @formatter:on
|
||||
|
||||
trait OutgoingPaymentFactory {
|
||||
@ -161,6 +163,17 @@ object NodeRelay {
|
||||
.modify(_.includeLocalChannelCost).setTo(true)
|
||||
}
|
||||
|
||||
/** If we fail to relay a payment, we may want to attempt on-the-fly funding if it makes sense. */
|
||||
private def shouldAttemptOnTheFlyFunding(nodeParams: NodeParams, recipientFeatures_opt: Option[Features[InitFeature]], failures: Seq[PaymentFailure])(implicit context: ActorContext[Command]): Boolean = {
|
||||
val featureOk = Features.canUseFeature(nodeParams.features.initFeatures(), recipientFeatures_opt.getOrElse(Features.empty), Features.OnTheFlyFunding)
|
||||
val routerBalanceTooLow = failures.collectFirst { case f@LocalFailure(_, _, BalanceTooLow) => f }.nonEmpty
|
||||
val channelBalanceTooLow = failures.collectFirst { case f@LocalFailure(_, _, _: InsufficientFunds) => f }.nonEmpty
|
||||
val routeNotFound = failures.collectFirst { case f@LocalFailure(_, _, RouteNotFound) => f }.nonEmpty
|
||||
val res = featureOk && (routerBalanceTooLow || channelBalanceTooLow || routeNotFound)
|
||||
context.log.info(s"on-the-fly funding assessment: shouldAttempt=$res featureOk=$featureOk routerBalanceTooLow=$routerBalanceTooLow channelBalanceTooLow=$channelBalanceTooLow localFailures={}", failures.collect { case f: LocalFailure => f.t.getClass.getSimpleName }.mkString(","))
|
||||
res
|
||||
}
|
||||
|
||||
/**
|
||||
* This helper method translates relaying errors (returned by the downstream nodes) to a BOLT 4 standard error that we
|
||||
* should return upstream.
|
||||
@ -230,7 +243,7 @@ class NodeRelay private(nodeParams: NodeParams,
|
||||
stopping()
|
||||
case WrappedMultiPartPaymentSucceeded(MultiPartPaymentFSM.MultiPartPaymentSucceeded(_, parts)) =>
|
||||
context.log.info("completed incoming multi-part payment with parts={} paidAmount={}", parts.size, parts.map(_.amount).sum)
|
||||
val upstream = Upstream.Hot.Trampoline(htlcs)
|
||||
val upstream = Upstream.Hot.Trampoline(htlcs.toList)
|
||||
validateRelay(nodeParams, upstream, nextPayload) match {
|
||||
case Some(failure) =>
|
||||
context.log.warn(s"rejecting trampoline payment reason=$failure")
|
||||
@ -288,7 +301,7 @@ class NodeRelay private(nodeParams: NodeParams,
|
||||
private def ensureRecipientReady(upstream: Upstream.Hot.Trampoline, recipient: Recipient, nextPayload: IntermediatePayload.NodeRelay, nextPacket_opt: Option[OnionRoutingPacket]): Behavior[Command] = {
|
||||
nextWalletNodeId(nodeParams, recipient) match {
|
||||
case Some(walletNodeId) if nodeParams.peerWakeUpConfig.enabled => waitForPeerReady(upstream, walletNodeId, recipient, nextPayload, nextPacket_opt)
|
||||
case _ => relay(upstream, recipient, nextPayload, nextPacket_opt)
|
||||
case _ => relay(upstream, recipient, None, None, nextPayload, nextPacket_opt)
|
||||
}
|
||||
}
|
||||
|
||||
@ -306,16 +319,18 @@ class NodeRelay private(nodeParams: NodeParams,
|
||||
context.log.warn("rejecting payment: failed to wake-up remote peer")
|
||||
rejectPayment(upstream, Some(UnknownNextPeer()))
|
||||
stopping()
|
||||
case WrappedPeerReadyResult(_: PeerReadyNotifier.PeerReady) =>
|
||||
relay(upstream, recipient, nextPayload, nextPacket_opt)
|
||||
case WrappedPeerReadyResult(r: PeerReadyNotifier.PeerReady) =>
|
||||
relay(upstream, recipient, Some(walletNodeId), Some(r.remoteFeatures), nextPayload, nextPacket_opt)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** Relay the payment to the next identified node: this is similar to sending an outgoing payment. */
|
||||
private def relay(upstream: Upstream.Hot.Trampoline, recipient: Recipient, payloadOut: IntermediatePayload.NodeRelay, packetOut_opt: Option[OnionRoutingPacket]): Behavior[Command] = {
|
||||
private def relay(upstream: Upstream.Hot.Trampoline, recipient: Recipient, walletNodeId_opt: Option[PublicKey], recipientFeatures_opt: Option[Features[InitFeature]], payloadOut: IntermediatePayload.NodeRelay, packetOut_opt: Option[OnionRoutingPacket]): Behavior[Command] = {
|
||||
context.log.debug("relaying trampoline payment (amountIn={} expiryIn={} amountOut={} expiryOut={})", upstream.amountIn, upstream.expiryIn, payloadOut.amountToForward, payloadOut.outgoingCltv)
|
||||
val confidence = (upstream.received.map(_.add.endorsement).min + 0.5) / 8
|
||||
// We only make one try when it's a direct payment to a wallet.
|
||||
val maxPaymentAttempts = if (walletNodeId_opt.isDefined) 1 else nodeParams.maxPaymentAttempts
|
||||
val paymentCfg = SendPaymentConfig(relayId, relayId, None, paymentHash, recipient.nodeId, upstream, None, None, storeInDb = false, publishEvent = false, recordPathFindingMetrics = true, confidence)
|
||||
val routeParams = computeRouteParams(nodeParams, upstream.amountIn, upstream.expiryIn, payloadOut.amountToForward, payloadOut.outgoingCltv)
|
||||
// If the next node is using trampoline, we assume that they support MPP.
|
||||
@ -326,13 +341,13 @@ class NodeRelay private(nodeParams: NodeParams,
|
||||
context.messageAdapter[PaymentFailed](WrappedPaymentFailed)
|
||||
}.toClassic
|
||||
val payment = if (useMultiPart) {
|
||||
SendMultiPartPayment(payFsmAdapters, recipient, nodeParams.maxPaymentAttempts, routeParams)
|
||||
SendMultiPartPayment(payFsmAdapters, recipient, maxPaymentAttempts, routeParams)
|
||||
} else {
|
||||
SendPaymentToNode(payFsmAdapters, recipient, nodeParams.maxPaymentAttempts, routeParams)
|
||||
SendPaymentToNode(payFsmAdapters, recipient, maxPaymentAttempts, routeParams)
|
||||
}
|
||||
val payFSM = outgoingPaymentFactory.spawnOutgoingPayFSM(context, paymentCfg, useMultiPart)
|
||||
payFSM ! payment
|
||||
sending(upstream, payloadOut, recipient, TimestampMilli.now(), fulfilledUpstream = false)
|
||||
sending(upstream, recipient, recipientFeatures_opt, payloadOut, TimestampMilli.now(), fulfilledUpstream = false)
|
||||
}
|
||||
|
||||
/**
|
||||
@ -343,8 +358,9 @@ class NodeRelay private(nodeParams: NodeParams,
|
||||
* @param fulfilledUpstream true if we already fulfilled the payment upstream.
|
||||
*/
|
||||
private def sending(upstream: Upstream.Hot.Trampoline,
|
||||
nextPayload: IntermediatePayload.NodeRelay,
|
||||
recipient: Recipient,
|
||||
recipientFeatures_opt: Option[Features[InitFeature]],
|
||||
nextPayload: IntermediatePayload.NodeRelay,
|
||||
startedAt: TimestampMilli,
|
||||
fulfilledUpstream: Boolean): Behavior[Command] =
|
||||
Behaviors.receiveMessagePartial {
|
||||
@ -355,7 +371,7 @@ class NodeRelay private(nodeParams: NodeParams,
|
||||
// We want to fulfill upstream as soon as we receive the preimage (even if not all HTLCs have fulfilled downstream).
|
||||
context.log.debug("got preimage from downstream")
|
||||
fulfillPayment(upstream, paymentPreimage)
|
||||
sending(upstream, nextPayload, recipient, startedAt, fulfilledUpstream = true)
|
||||
sending(upstream, recipient, recipientFeatures_opt, nextPayload, startedAt, fulfilledUpstream = true)
|
||||
} else {
|
||||
// we don't want to fulfill multiple times
|
||||
Behaviors.same
|
||||
@ -365,16 +381,66 @@ class NodeRelay private(nodeParams: NodeParams,
|
||||
success(upstream, fulfilledUpstream, paymentSent)
|
||||
recordRelayDuration(startedAt, isSuccess = true)
|
||||
stopping()
|
||||
case WrappedPaymentFailed(PaymentFailed(_, _, failures, _)) =>
|
||||
context.log.debug(s"trampoline payment failed downstream")
|
||||
if (!fulfilledUpstream) {
|
||||
rejectPayment(upstream, translateError(nodeParams, failures, upstream, nextPayload))
|
||||
}
|
||||
recordRelayDuration(startedAt, isSuccess = fulfilledUpstream)
|
||||
case _: WrappedPaymentFailed if fulfilledUpstream =>
|
||||
context.log.warn("trampoline payment failed downstream but was fulfilled upstream")
|
||||
recordRelayDuration(startedAt, isSuccess = true)
|
||||
stopping()
|
||||
case WrappedPaymentFailed(PaymentFailed(_, _, failures, _)) =>
|
||||
nextWalletNodeId(nodeParams, recipient) match {
|
||||
case Some(walletNodeId) if shouldAttemptOnTheFlyFunding(nodeParams, recipientFeatures_opt, failures)(context) =>
|
||||
context.log.info("trampoline payment failed, attempting on-the-fly funding")
|
||||
attemptOnTheFlyFunding(upstream, walletNodeId, recipient, nextPayload, failures, startedAt)
|
||||
case _ =>
|
||||
rejectPayment(upstream, translateError(nodeParams, failures, upstream, nextPayload))
|
||||
recordRelayDuration(startedAt, isSuccess = false)
|
||||
stopping()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** We couldn't forward the payment, but the next node may accept on-the-fly funding. */
|
||||
private def attemptOnTheFlyFunding(upstream: Upstream.Hot.Trampoline, walletNodeId: PublicKey, recipient: Recipient, nextPayload: IntermediatePayload.NodeRelay, failures: Seq[PaymentFailure], startedAt: TimestampMilli): Behavior[Command] = {
|
||||
// We create a payment onion, using a dummy channel hop between our node and the wallet node.
|
||||
val dummyEdge = Invoice.ExtraEdge(nodeParams.nodeId, walletNodeId, Alias(0), 0 msat, 0, CltvExpiryDelta(0), 1 msat, None)
|
||||
val dummyHop = ChannelHop(Alias(0), nodeParams.nodeId, walletNodeId, HopRelayParams.FromHint(dummyEdge))
|
||||
val finalHop_opt = recipient match {
|
||||
case _: ClearRecipient => None
|
||||
case _: SpontaneousRecipient => None
|
||||
case _: TrampolineRecipient => None
|
||||
case r: BlindedRecipient => r.blindedHops.headOption
|
||||
}
|
||||
val dummyRoute = Route(nextPayload.amountToForward, Seq(dummyHop), finalHop_opt)
|
||||
OutgoingPaymentPacket.buildOutgoingPayment(Origin.Hot(ActorRef.noSender, upstream), paymentHash, dummyRoute, recipient, 1.0) match {
|
||||
case Left(f) =>
|
||||
context.log.warn("could not create payment onion for on-the-fly funding: {}", f.getMessage)
|
||||
rejectPayment(upstream, translateError(nodeParams, failures, upstream, nextPayload))
|
||||
recordRelayDuration(startedAt, isSuccess = false)
|
||||
stopping()
|
||||
case Right(nextPacket) =>
|
||||
val forwardNodeIdFailureAdapter = context.messageAdapter[Register.ForwardNodeIdFailure[Peer.ProposeOnTheFlyFunding]](_ => WrappedOnTheFlyFundingResponse(Peer.ProposeOnTheFlyFundingResponse.NotAvailable("peer not found")))
|
||||
val onTheFlyFundingResponseAdapter = context.messageAdapter[Peer.ProposeOnTheFlyFundingResponse](WrappedOnTheFlyFundingResponse)
|
||||
val cmd = Peer.ProposeOnTheFlyFunding(onTheFlyFundingResponseAdapter, nextPayload.amountToForward, paymentHash, nextPayload.outgoingCltv, nextPacket.cmd.onion, nextPacket.cmd.nextBlindingKey_opt, upstream)
|
||||
register ! Register.ForwardNodeId(forwardNodeIdFailureAdapter, walletNodeId, cmd)
|
||||
Behaviors.receiveMessagePartial {
|
||||
rejectExtraHtlcPartialFunction orElse {
|
||||
case WrappedOnTheFlyFundingResponse(response) =>
|
||||
response match {
|
||||
case ProposeOnTheFlyFundingResponse.Proposed =>
|
||||
context.log.info("on-the-fly funding proposed")
|
||||
// We're not responsible for the payment relay anymore: another actor will take care of relaying the
|
||||
// payment once on-the-fly funding completes.
|
||||
stopping()
|
||||
case ProposeOnTheFlyFundingResponse.NotAvailable(reason) =>
|
||||
context.log.warn("could not propose on-the-fly funding: {}", reason)
|
||||
rejectPayment(upstream, Some(UnknownNextPeer()))
|
||||
recordRelayDuration(startedAt, isSuccess = false)
|
||||
stopping()
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Once the downstream payment is settled (fulfilled or failed), we reject new upstream payments while we wait for our parent to stop us.
|
||||
*/
|
||||
|
@ -0,0 +1,318 @@
|
||||
/*
|
||||
* Copyright 2024 ACINQ SAS
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package fr.acinq.eclair.payment.relay
|
||||
|
||||
import akka.actor.Cancellable
|
||||
import akka.actor.typed.scaladsl.adapter.TypedActorRefOps
|
||||
import akka.actor.typed.scaladsl.{ActorContext, Behaviors, StashBuffer}
|
||||
import akka.actor.typed.{ActorRef, Behavior}
|
||||
import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
|
||||
import fr.acinq.bitcoin.scalacompat.{ByteVector32, Crypto, TxId}
|
||||
import fr.acinq.eclair.blockchain.fee.FeeratePerKw
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.crypto.Sphinx
|
||||
import fr.acinq.eclair.wire.protocol.LiquidityAds.PaymentDetails
|
||||
import fr.acinq.eclair.wire.protocol._
|
||||
import fr.acinq.eclair.{Logs, MilliSatoshi, NodeParams, TimestampMilli, ToMilliSatoshiConversion}
|
||||
import scodec.bits.ByteVector
|
||||
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
|
||||
/**
|
||||
* Created by t-bast on 19/06/2024.
|
||||
*/
|
||||
|
||||
object OnTheFlyFunding {
|
||||
|
||||
case class Config(proposalTimeout: FiniteDuration)
|
||||
|
||||
// @formatter:off
|
||||
sealed trait Status
|
||||
object Status {
|
||||
/** We sent will_add_htlc, but didn't fund a transaction yet. */
|
||||
case class Proposed(timer: Cancellable) extends Status
|
||||
/**
|
||||
* We signed a transaction matching the on-the-fly funding proposed. We're waiting for the liquidity to be
|
||||
* available (channel ready or splice locked) to relay the HTLCs and complete the payment.
|
||||
*/
|
||||
case class Funded(channelId: ByteVector32, txId: TxId, fundingTxIndex: Long, remainingFees: MilliSatoshi) extends Status
|
||||
}
|
||||
// @formatter:on
|
||||
|
||||
/** An on-the-fly funding proposal sent to our peer. */
|
||||
case class Proposal(htlc: WillAddHtlc, upstream: Upstream.Hot) {
|
||||
/** Maximum fees that can be collected from this HTLC. */
|
||||
def maxFees(htlcMinimum: MilliSatoshi): MilliSatoshi = htlc.amount - htlcMinimum
|
||||
|
||||
/** Create commands to fail all upstream HTLCs. */
|
||||
def createFailureCommands(failure_opt: Option[Either[ByteVector, FailureMessage]]): Seq[(ByteVector32, CMD_FAIL_HTLC)] = upstream match {
|
||||
case _: Upstream.Local => Nil
|
||||
case u: Upstream.Hot.Channel =>
|
||||
val failure = htlc.blinding_opt match {
|
||||
case Some(_) => Right(InvalidOnionBlinding(Sphinx.hash(u.add.onionRoutingPacket)))
|
||||
case None => failure_opt.getOrElse(Right(UnknownNextPeer()))
|
||||
}
|
||||
Seq(u.add.channelId -> CMD_FAIL_HTLC(u.add.id, failure, commit = true))
|
||||
case u: Upstream.Hot.Trampoline =>
|
||||
// In the trampoline case, we currently ignore downstream failures: we should add dedicated failures to the
|
||||
// BOLTs to better handle those cases.
|
||||
val failure = failure_opt match {
|
||||
case Some(f) => f.getOrElse(TemporaryNodeFailure())
|
||||
case None => UnknownNextPeer()
|
||||
}
|
||||
u.received.map(_.add).map(add => add.channelId -> CMD_FAIL_HTLC(add.id, Right(failure), commit = true))
|
||||
}
|
||||
|
||||
/** Create commands to fulfill all upstream HTLCs. */
|
||||
def createFulfillCommands(preimage: ByteVector32): Seq[(ByteVector32, CMD_FULFILL_HTLC)] = upstream match {
|
||||
case _: Upstream.Local => Nil
|
||||
case u: Upstream.Hot.Channel => Seq(u.add.channelId -> CMD_FULFILL_HTLC(u.add.id, preimage, commit = true))
|
||||
case u: Upstream.Hot.Trampoline => u.received.map(_.add).map(add => add.channelId -> CMD_FULFILL_HTLC(add.id, preimage, commit = true))
|
||||
}
|
||||
}
|
||||
|
||||
/** A set of funding proposals for a given payment. */
|
||||
case class Pending(proposed: Seq[Proposal], status: Status) {
|
||||
val paymentHash = proposed.head.htlc.paymentHash
|
||||
val expiry = proposed.map(_.htlc.expiry).min
|
||||
|
||||
/** Maximum fees that can be collected from this HTLC set. */
|
||||
def maxFees(htlcMinimum: MilliSatoshi): MilliSatoshi = proposed.map(_.maxFees(htlcMinimum)).sum
|
||||
|
||||
/** Create commands to fail all upstream HTLCs. */
|
||||
def createFailureCommands(): Seq[(ByteVector32, CMD_FAIL_HTLC)] = proposed.flatMap(_.createFailureCommands(None))
|
||||
|
||||
/** Create commands to fulfill all upstream HTLCs. */
|
||||
def createFulfillCommands(preimage: ByteVector32): Seq[(ByteVector32, CMD_FULFILL_HTLC)] = proposed.flatMap(_.createFulfillCommands(preimage))
|
||||
}
|
||||
|
||||
// @formatter:off
|
||||
sealed trait ValidationResult
|
||||
object ValidationResult {
|
||||
/** The incoming channel or splice cannot pay the liquidity fees: we must reject it and fail the corresponding upstream HTLCs. */
|
||||
case class Reject(cancel: CancelOnTheFlyFunding, paymentHashes: Set[ByteVector32]) extends ValidationResult
|
||||
/** We are on-the-fly funding a channel: if we received preimages, we must fulfill the corresponding upstream HTLCs. */
|
||||
case class Accept(preimages: Set[ByteVector32]) extends ValidationResult
|
||||
}
|
||||
// @formatter:on
|
||||
|
||||
/** Validate an incoming channel that may use on-the-fly funding. */
|
||||
def validateOpen(open: Either[OpenChannel, OpenDualFundedChannel], pendingOnTheFlyFunding: Map[ByteVector32, Pending]): ValidationResult = {
|
||||
open match {
|
||||
case Left(_) => ValidationResult.Accept(Set.empty)
|
||||
case Right(open) => open.requestFunding_opt match {
|
||||
case Some(requestFunding) => validate(open.temporaryChannelId, requestFunding, isChannelCreation = true, open.fundingFeerate, open.htlcMinimum, pendingOnTheFlyFunding)
|
||||
case None => ValidationResult.Accept(Set.empty)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** Validate an incoming splice that may use on-the-fly funding. */
|
||||
def validateSplice(splice: SpliceInit, htlcMinimum: MilliSatoshi, pendingOnTheFlyFunding: Map[ByteVector32, Pending]): ValidationResult = {
|
||||
splice.requestFunding_opt match {
|
||||
case Some(requestFunding) => validate(splice.channelId, requestFunding, isChannelCreation = false, splice.feerate, htlcMinimum, pendingOnTheFlyFunding)
|
||||
case None => ValidationResult.Accept(Set.empty)
|
||||
}
|
||||
}
|
||||
|
||||
private def validate(channelId: ByteVector32,
|
||||
requestFunding: LiquidityAds.RequestFunding,
|
||||
isChannelCreation: Boolean,
|
||||
feerate: FeeratePerKw,
|
||||
htlcMinimum: MilliSatoshi,
|
||||
pendingOnTheFlyFunding: Map[ByteVector32, Pending]): ValidationResult = {
|
||||
val paymentHashes = requestFunding.paymentDetails match {
|
||||
case PaymentDetails.FromChannelBalance => Nil
|
||||
case PaymentDetails.FromChannelBalanceForFutureHtlc(paymentHashes) => paymentHashes
|
||||
case PaymentDetails.FromFutureHtlc(paymentHashes) => paymentHashes
|
||||
case PaymentDetails.FromFutureHtlcWithPreimage(preimages) => preimages.map(preimage => Crypto.sha256(preimage))
|
||||
}
|
||||
val pending = paymentHashes.flatMap(paymentHash => pendingOnTheFlyFunding.get(paymentHash)).filter(_.status.isInstanceOf[OnTheFlyFunding.Status.Proposed])
|
||||
val totalPaymentAmount = pending.flatMap(_.proposed.map(_.htlc.amount)).sum
|
||||
// We will deduce fees from HTLCs: we check that the amount is large enough to cover the fees.
|
||||
val availableAmountForFees = pending.map(_.maxFees(htlcMinimum)).sum
|
||||
val fees = requestFunding.fees(feerate, isChannelCreation)
|
||||
val cancelAmountTooLow = CancelOnTheFlyFunding(channelId, paymentHashes, s"requested amount is too low to relay HTLCs: ${requestFunding.requestedAmount} < $totalPaymentAmount")
|
||||
val cancelFeesTooLow = CancelOnTheFlyFunding(channelId, paymentHashes, s"htlc amount is too low to pay liquidity fees: $availableAmountForFees < ${fees.total}")
|
||||
requestFunding.paymentDetails match {
|
||||
case PaymentDetails.FromChannelBalance => ValidationResult.Accept(Set.empty)
|
||||
case _ if requestFunding.requestedAmount.toMilliSatoshi < totalPaymentAmount => ValidationResult.Reject(cancelAmountTooLow, paymentHashes.toSet)
|
||||
case _: PaymentDetails.FromChannelBalanceForFutureHtlc => ValidationResult.Accept(Set.empty)
|
||||
case _: PaymentDetails.FromFutureHtlc if availableAmountForFees < fees.total => ValidationResult.Reject(cancelFeesTooLow, paymentHashes.toSet)
|
||||
case _: PaymentDetails.FromFutureHtlc => ValidationResult.Accept(Set.empty)
|
||||
case _: PaymentDetails.FromFutureHtlcWithPreimage if availableAmountForFees < fees.total => ValidationResult.Reject(cancelFeesTooLow, paymentHashes.toSet)
|
||||
case p: PaymentDetails.FromFutureHtlcWithPreimage => ValidationResult.Accept(p.preimages.toSet)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This actor relays HTLCs that were proposed with [[WillAddHtlc]] once funding is complete.
|
||||
* It verifies that this payment was not previously relayed, to protect against over-paying and paying multiple times.
|
||||
*/
|
||||
object PaymentRelayer {
|
||||
// @formatter:off
|
||||
sealed trait Command
|
||||
case class TryRelay(replyTo: ActorRef[RelayResult], channel: ActorRef[fr.acinq.eclair.channel.Command], proposed: Seq[Proposal], status: Status.Funded) extends Command
|
||||
private case class WrappedChannelInfo(state: ChannelState, data: ChannelData) extends Command
|
||||
private case class WrappedCommandResponse(response: CommandResponse[CMD_ADD_HTLC]) extends Command
|
||||
private case class WrappedHtlcSettled(result: RES_ADD_SETTLED[Origin.Hot, HtlcResult]) extends Command
|
||||
|
||||
sealed trait RelayResult
|
||||
case class RelaySuccess(channelId: ByteVector32, paymentHash: ByteVector32, preimage: ByteVector32, fees: MilliSatoshi) extends RelayResult
|
||||
case class RelayFailed(paymentHash: ByteVector32, failure: RelayFailure) extends RelayResult
|
||||
|
||||
sealed trait RelayFailure
|
||||
case object ExpiryTooClose extends RelayFailure { override def toString: String = "htlcs are too close to expiry to be relayed" }
|
||||
case class ChannelNotAvailable(state: ChannelState) extends RelayFailure { override def toString: String = s"channel is not ready for payments (state=${state.toString})" }
|
||||
case class CannotAddToChannel(t: Throwable) extends RelayFailure { override def toString: String = s"could not relay on-the-fly HTLC: ${t.getMessage}" }
|
||||
case class RemoteFailure(failure: HtlcResult.Fail) extends RelayFailure { override def toString: String = s"relayed on-the-fly HTLC was failed: ${failure.getClass.getSimpleName}" }
|
||||
// @formatter:on
|
||||
|
||||
def apply(nodeParams: NodeParams, remoteNodeId: PublicKey, channelId: ByteVector32, paymentHash: ByteVector32): Behavior[Command] =
|
||||
Behaviors.setup { context =>
|
||||
Behaviors.withMdc(Logs.mdc(category_opt = Some(Logs.LogCategory.PAYMENT), remoteNodeId_opt = Some(remoteNodeId), channelId_opt = Some(channelId), paymentHash_opt = Some(paymentHash))) {
|
||||
Behaviors.receiveMessagePartial {
|
||||
case cmd: TryRelay => new PaymentRelayer(nodeParams, channelId, paymentHash, cmd, context).start()
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class PaymentRelayer private(nodeParams: NodeParams, channelId: ByteVector32, paymentHash: ByteVector32, cmd: PaymentRelayer.TryRelay, context: ActorContext[PaymentRelayer.Command]) {
|
||||
|
||||
import PaymentRelayer._
|
||||
|
||||
def start(): Behavior[Command] = {
|
||||
if (cmd.proposed.exists(_.htlc.expiry.blockHeight <= nodeParams.currentBlockHeight + 12)) {
|
||||
// The funding proposal expires soon: we shouldn't relay HTLCs to avoid risking a force-close.
|
||||
cmd.replyTo ! RelayFailed(paymentHash, ExpiryTooClose)
|
||||
Behaviors.stopped
|
||||
} else {
|
||||
checkChannelState()
|
||||
}
|
||||
}
|
||||
|
||||
private def checkChannelState(): Behavior[Command] = {
|
||||
cmd.channel ! CMD_GET_CHANNEL_INFO(context.messageAdapter[RES_GET_CHANNEL_INFO](r => WrappedChannelInfo(r.state, r.data)))
|
||||
Behaviors.receiveMessagePartial {
|
||||
case WrappedChannelInfo(_, data: DATA_NORMAL) if paymentAlreadyRelayed(paymentHash, data) =>
|
||||
context.log.warn("payment is already being relayed, waiting for it to be settled")
|
||||
Behaviors.stopped
|
||||
case WrappedChannelInfo(_, data: DATA_NORMAL) =>
|
||||
nodeParams.db.liquidity.getOnTheFlyFundingPreimage(paymentHash) match {
|
||||
case Some(preimage) =>
|
||||
// We have already received the preimage for that payment, but we probably restarted before removing the
|
||||
// on-the-fly funding proposal from our DB. We must not relay the payment again, otherwise we will pay
|
||||
// the next node twice.
|
||||
cmd.replyTo ! RelaySuccess(channelId, paymentHash, preimage, cmd.status.remainingFees)
|
||||
Behaviors.stopped
|
||||
case None => relay(data)
|
||||
}
|
||||
case WrappedChannelInfo(state, _) =>
|
||||
cmd.replyTo ! RelayFailed(paymentHash, ChannelNotAvailable(state))
|
||||
Behaviors.stopped
|
||||
}
|
||||
}
|
||||
|
||||
private def paymentAlreadyRelayed(paymentHash: ByteVector32, data: DATA_NORMAL): Boolean = {
|
||||
data.commitments.changes.localChanges.all.exists {
|
||||
case add: UpdateAddHtlc => add.paymentHash == paymentHash && add.fundingFee_opt.nonEmpty
|
||||
case _ => false
|
||||
}
|
||||
}
|
||||
|
||||
private def relay(data: DATA_NORMAL): Behavior[Command] = {
|
||||
context.log.debug("relaying {} on-the-fly HTLCs that have been funded", cmd.proposed.size)
|
||||
val htlcMinimum = data.commitments.params.remoteParams.htlcMinimum
|
||||
val cmdAdapter = context.messageAdapter[CommandResponse[CMD_ADD_HTLC]](WrappedCommandResponse)
|
||||
val htlcSettledAdapter = context.messageAdapter[RES_ADD_SETTLED[Origin.Hot, HtlcResult]](WrappedHtlcSettled)
|
||||
cmd.proposed.foldLeft(cmd.status.remainingFees) {
|
||||
case (remainingFees, p) =>
|
||||
// We always set the funding_fee field, even if the fee for this specific HTLC is 0.
|
||||
// This lets us detect that this HTLC is an on-the-fly funded HTLC.
|
||||
val htlcFees = LiquidityAds.FundingFee(remainingFees.min(p.maxFees(htlcMinimum)), cmd.status.txId)
|
||||
val origin = Origin.Hot(htlcSettledAdapter.toClassic, p.upstream)
|
||||
// We only sign at the end of the whole batch.
|
||||
val commit = p.htlc.id == cmd.proposed.last.htlc.id
|
||||
val add = CMD_ADD_HTLC(cmdAdapter.toClassic, p.htlc.amount - htlcFees.amount, paymentHash, p.htlc.expiry, p.htlc.finalPacket, p.htlc.blinding_opt, 1.0, Some(htlcFees), origin, commit)
|
||||
cmd.channel ! add
|
||||
remainingFees - htlcFees.amount
|
||||
}
|
||||
Behaviors.withStash(cmd.proposed.size) { stash =>
|
||||
waitForCommandResult(stash, cmd.proposed.size, htlcSent = 0)
|
||||
}
|
||||
}
|
||||
|
||||
private def waitForCommandResult(stash: StashBuffer[Command], remaining: Int, htlcSent: Int): Behavior[Command] = {
|
||||
if (remaining == 0) {
|
||||
stash.unstashAll(waitForSettlement(htlcSent))
|
||||
} else {
|
||||
Behaviors.receiveMessagePartial {
|
||||
case WrappedCommandResponse(response) => response match {
|
||||
case _: CommandSuccess[_] =>
|
||||
waitForCommandResult(stash, remaining - 1, htlcSent + 1)
|
||||
case failure: CommandFailure[_, _] =>
|
||||
cmd.replyTo ! RelayFailed(paymentHash, CannotAddToChannel(failure.t))
|
||||
waitForCommandResult(stash, remaining - 1, htlcSent)
|
||||
}
|
||||
case msg: WrappedHtlcSettled =>
|
||||
stash.stash(msg)
|
||||
Behaviors.same
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private def waitForSettlement(remaining: Int): Behavior[Command] = {
|
||||
if (remaining == 0) {
|
||||
Behaviors.stopped
|
||||
} else {
|
||||
Behaviors.receiveMessagePartial {
|
||||
case WrappedHtlcSettled(settled) =>
|
||||
settled.result match {
|
||||
case fulfill: HtlcResult.Fulfill => cmd.replyTo ! RelaySuccess(channelId, paymentHash, fulfill.paymentPreimage, cmd.status.remainingFees)
|
||||
case fail: HtlcResult.Fail => cmd.replyTo ! RelayFailed(paymentHash, RemoteFailure(fail))
|
||||
}
|
||||
waitForSettlement(remaining - 1)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
object Codecs {
|
||||
|
||||
import fr.acinq.eclair.wire.protocol.CommonCodecs._
|
||||
import fr.acinq.eclair.wire.protocol.LightningMessageCodecs._
|
||||
import scodec.Codec
|
||||
import scodec.codecs._
|
||||
|
||||
private val upstreamLocal: Codec[Upstream.Local] = uuid.as[Upstream.Local]
|
||||
private val upstreamChannel: Codec[Upstream.Hot.Channel] = (lengthDelimited(updateAddHtlcCodec) :: uint64overflow.as[TimestampMilli] :: publicKey).as[Upstream.Hot.Channel]
|
||||
private val upstreamTrampoline: Codec[Upstream.Hot.Trampoline] = listOfN(uint16, upstreamChannel).as[Upstream.Hot.Trampoline]
|
||||
|
||||
val upstream: Codec[Upstream.Hot] = discriminated[Upstream.Hot].by(uint16)
|
||||
.typecase(0x00, upstreamLocal)
|
||||
.typecase(0x01, upstreamChannel)
|
||||
.typecase(0x02, upstreamTrampoline)
|
||||
|
||||
val proposal: Codec[Proposal] = (("willAddHtlc" | lengthDelimited(willAddHtlcCodec)) :: ("upstream" | upstream)).as[Proposal]
|
||||
|
||||
val proposals: Codec[Seq[Proposal]] = listOfN(uint16, proposal).xmap(_.toSeq, _.toList)
|
||||
|
||||
}
|
||||
|
||||
}
|
@ -68,6 +68,7 @@ class PostRestartHtlcCleaner(nodeParams: NodeParams, register: ActorRef, initial
|
||||
// result upstream to preserve channels.
|
||||
val brokenHtlcs: BrokenHtlcs = {
|
||||
val channels = listLocalChannels(init.channels)
|
||||
val onTheFlyPayments = nodeParams.db.liquidity.listPendingOnTheFlyPayments().values.flatten.toSet
|
||||
val nonStandardIncomingHtlcs: Seq[IncomingHtlc] = nodeParams.pluginParams.collect { case p: CustomCommitmentsPlugin => p.getIncomingHtlcs(nodeParams, log) }.flatten
|
||||
val htlcsIn: Seq[IncomingHtlc] = getIncomingHtlcs(channels, nodeParams.db.payments, nodeParams.privateKey, nodeParams.features) ++ nonStandardIncomingHtlcs
|
||||
val nonStandardRelayedOutHtlcs: Map[Origin.Cold, Set[(ByteVector32, Long)]] = nodeParams.pluginParams.collect { case p: CustomCommitmentsPlugin => p.getHtlcsRelayedOut(htlcsIn, nodeParams, log) }.flatten.toMap
|
||||
@ -85,7 +86,7 @@ class PostRestartHtlcCleaner(nodeParams: NodeParams, register: ActorRef, initial
|
||||
log.info(s"htlcsIn=${htlcsIn.length} notRelayed=${notRelayed.length} relayedOut=${relayedOut.values.flatten.size}")
|
||||
log.info("notRelayed={}", notRelayed.map(htlc => (htlc.add.channelId, htlc.add.id)))
|
||||
log.info("relayedOut={}", relayedOut)
|
||||
BrokenHtlcs(notRelayed, relayedOut, Set.empty)
|
||||
BrokenHtlcs(notRelayed, relayedOut, Set.empty, onTheFlyPayments)
|
||||
}
|
||||
|
||||
Metrics.PendingNotRelayed.update(brokenHtlcs.notRelayed.size)
|
||||
@ -120,6 +121,10 @@ class PostRestartHtlcCleaner(nodeParams: NodeParams, register: ActorRef, initial
|
||||
} else {
|
||||
log.info(s"got preimage but upstream channel is closed for htlc=$htlc")
|
||||
}
|
||||
case None if brokenHtlcs.pendingPayments.contains(htlc.paymentHash) =>
|
||||
// We don't fail on-the-fly HTLCs that have been funded: we haven't been paid our fee yet, so we will
|
||||
// retry relaying them unless we reach the HTLC timeout.
|
||||
log.info("htlc #{} from channelId={} wasn't relayed, but has a pending on-the-fly relay (paymentHash={})", htlc.id, htlc.channelId, htlc.paymentHash)
|
||||
case None =>
|
||||
Metrics.Resolved.withTag(Tags.Success, value = false).withTag(Metrics.Relayed, value = false).increment()
|
||||
if (e.currentState != CLOSING && e.currentState != CLOSED) {
|
||||
@ -150,12 +155,17 @@ class PostRestartHtlcCleaner(nodeParams: NodeParams, register: ActorRef, initial
|
||||
case _: ChannelStateChanged => // ignore other channel state changes
|
||||
|
||||
case RES_ADD_SETTLED(o: Origin.Cold, htlc, fulfill: HtlcResult.Fulfill) =>
|
||||
log.info("htlc fulfilled downstream: ({},{})", htlc.channelId, htlc.id)
|
||||
log.info("htlc #{} from channelId={} fulfilled downstream", htlc.id, htlc.channelId)
|
||||
handleDownstreamFulfill(brokenHtlcs, o, htlc, fulfill.paymentPreimage)
|
||||
|
||||
case RES_ADD_SETTLED(o: Origin.Cold, htlc, fail: HtlcResult.Fail) =>
|
||||
log.info("htlc failed downstream: ({},{},{})", htlc.channelId, htlc.id, fail.getClass.getSimpleName)
|
||||
handleDownstreamFailure(brokenHtlcs, o, htlc, fail)
|
||||
if (htlc.fundingFee_opt.nonEmpty) {
|
||||
log.info("htlc #{} from channelId={} failed downstream but has a pending on-the-fly funding", htlc.id, htlc.channelId)
|
||||
// We don't fail upstream: we haven't been paid our funding fee yet, so we will try relaying again.
|
||||
} else {
|
||||
log.info("htlc #{} from channelId={} failed downstream: {}", htlc.id, htlc.channelId, fail.getClass.getSimpleName)
|
||||
handleDownstreamFailure(brokenHtlcs, o, htlc, fail)
|
||||
}
|
||||
|
||||
case GetBrokenHtlcs => sender() ! brokenHtlcs
|
||||
}
|
||||
@ -329,8 +339,9 @@ object PostRestartHtlcCleaner {
|
||||
* @param notRelayed incoming HTLCs that were committed upstream but not relayed downstream.
|
||||
* @param relayedOut outgoing HTLC sets that may have been incompletely sent and need to be watched.
|
||||
* @param settledUpstream upstream payments that have already been settled (failed or fulfilled) by this actor.
|
||||
* @param pendingPayments payments that are pending and will be relayed: we mustn't fail them upstream.
|
||||
*/
|
||||
case class BrokenHtlcs(notRelayed: Seq[IncomingHtlc], relayedOut: Map[Origin.Cold, Set[(ByteVector32, Long)]], settledUpstream: Set[Origin.Cold])
|
||||
case class BrokenHtlcs(notRelayed: Seq[IncomingHtlc], relayedOut: Map[Origin.Cold, Set[(ByteVector32, Long)]], settledUpstream: Set[Origin.Cold], pendingPayments: Set[ByteVector32])
|
||||
|
||||
/** Returns true if the given HTLC matches the given origin. */
|
||||
private def matchesOrigin(htlcIn: UpdateAddHtlc, origin: Origin.Cold): Boolean = origin.upstream match {
|
||||
|
@ -96,10 +96,15 @@ class Relayer(nodeParams: NodeParams, router: ActorRef, register: ActorRef, paym
|
||||
PendingCommandsDb.safeSend(register, nodeParams.db.pendingCommands, add.channelId, cmdFail)
|
||||
}
|
||||
|
||||
case r: RES_ADD_SETTLED[_, _] => r.origin match {
|
||||
case _: Origin.Cold => postRestartCleaner ! r
|
||||
case o: Origin.Hot => o.replyTo ! r
|
||||
}
|
||||
case r: RES_ADD_SETTLED[_, HtlcResult] =>
|
||||
r.result match {
|
||||
case fulfill: HtlcResult.Fulfill if r.htlc.fundingFee_opt.nonEmpty => nodeParams.db.liquidity.addOnTheFlyFundingPreimage(fulfill.paymentPreimage)
|
||||
case _ => ()
|
||||
}
|
||||
r.origin match {
|
||||
case _: Origin.Cold => postRestartCleaner ! r
|
||||
case o: Origin.Hot => o.replyTo ! r
|
||||
}
|
||||
|
||||
case g: GetOutgoingChannels => channelRelayer ! ChannelRelayer.GetOutgoingChannels(sender(), g)
|
||||
|
||||
|
@ -119,7 +119,7 @@ object CommonCodecs {
|
||||
|
||||
val listofsignatures: Codec[List[ByteVector64]] = listOfN(uint16, bytes64)
|
||||
|
||||
val channelflags: Codec[ChannelFlags] = (ignore(7) :: bool).as[ChannelFlags]
|
||||
val channelflags: Codec[ChannelFlags] = (ignore(6) :: bool :: bool).as[ChannelFlags]
|
||||
|
||||
val ipv4address: Codec[Inet4Address] = bytes(4).xmap(b => InetAddress.getByAddress(b.toArray).asInstanceOf[Inet4Address], a => ByteVector(a.getAddress))
|
||||
|
||||
|
@ -34,15 +34,22 @@ object UpdateAddHtlcTlv {
|
||||
/** Blinding ephemeral public key that should be used to derive shared secrets when using route blinding. */
|
||||
case class BlindingPoint(publicKey: PublicKey) extends UpdateAddHtlcTlv
|
||||
|
||||
case class Endorsement(level: Int) extends UpdateAddHtlcTlv
|
||||
|
||||
private val blindingPoint: Codec[BlindingPoint] = (("length" | constant(hex"21")) :: ("blinding" | publicKey)).as[BlindingPoint]
|
||||
|
||||
case class Endorsement(level: Int) extends UpdateAddHtlcTlv
|
||||
|
||||
private val endorsement: Codec[Endorsement] = tlvField(uint8.narrow[Endorsement](n => if (n >= 8) Attempt.failure(Err(s"invalid endorsement level: $n")) else Attempt.successful(Endorsement(n)), _.level))
|
||||
|
||||
/** When on-the-fly funding is used, the liquidity fees may be taken from HTLCs relayed after funding. */
|
||||
case class FundingFeeTlv(fee: LiquidityAds.FundingFee) extends UpdateAddHtlcTlv
|
||||
|
||||
private val fundingFee: Codec[FundingFeeTlv] = tlvField((("amount" | millisatoshi) :: ("txId" | txIdAsHash)).as[LiquidityAds.FundingFee])
|
||||
|
||||
val addHtlcTlvCodec: Codec[TlvStream[UpdateAddHtlcTlv]] = tlvStream(discriminated[UpdateAddHtlcTlv].by(varint)
|
||||
.typecase(UInt64(0), blindingPoint)
|
||||
.typecase(UInt64(106823), endorsement))
|
||||
.typecase(UInt64(41041), fundingFee)
|
||||
.typecase(UInt64(106823), endorsement)
|
||||
)
|
||||
}
|
||||
|
||||
sealed trait UpdateFulfillHtlcTlv extends Tlv
|
||||
|
@ -435,6 +435,31 @@ object LightningMessageCodecs {
|
||||
("commitmentFeerate" | feeratePerKw) ::
|
||||
("tlvStream" | RecommendedFeeratesTlv.recommendedFeeratesTlvCodec)).as[RecommendedFeerates]
|
||||
|
||||
val willAddHtlcCodec: Codec[WillAddHtlc] = (
|
||||
("chainHash" | blockHash) ::
|
||||
("id" | bytes32) ::
|
||||
("amount" | millisatoshi) ::
|
||||
("paymentHash" | bytes32) ::
|
||||
("expiry" | cltvExpiry) ::
|
||||
("onionRoutingPacket" | PaymentOnionCodecs.paymentOnionPacketCodec) ::
|
||||
("tlvStream" | WillAddHtlcTlv.willAddHtlcTlvCodec)).as[WillAddHtlc]
|
||||
|
||||
val willFailHtlcCodec: Codec[WillFailHtlc] = (
|
||||
("id" | bytes32) ::
|
||||
("paymentHash" | bytes32) ::
|
||||
("reason" | varsizebinarydata)).as[WillFailHtlc]
|
||||
|
||||
val willFailMalformedHtlcCodec: Codec[WillFailMalformedHtlc] = (
|
||||
("id" | bytes32) ::
|
||||
("paymentHash" | bytes32) ::
|
||||
("onionHash" | bytes32) ::
|
||||
("failureCode" | uint16)).as[WillFailMalformedHtlc]
|
||||
|
||||
val cancelOnTheFlyFundingCodec: Codec[CancelOnTheFlyFunding] = (
|
||||
("channelId" | bytes32) ::
|
||||
("paymentHashes" | listOfN(uint16, bytes32)) ::
|
||||
("reason" | varsizebinarydata)).as[CancelOnTheFlyFunding]
|
||||
|
||||
val unknownMessageCodec: Codec[UnknownMessage] = (
|
||||
("tag" | uint16) ::
|
||||
("message" | bytes)
|
||||
@ -487,6 +512,11 @@ object LightningMessageCodecs {
|
||||
|
||||
//
|
||||
//
|
||||
.typecase(41041, willAddHtlcCodec)
|
||||
.typecase(41042, willFailHtlcCodec)
|
||||
.typecase(41043, willFailMalformedHtlcCodec)
|
||||
.typecase(41044, cancelOnTheFlyFundingCodec)
|
||||
//
|
||||
.typecase(37000, spliceInitCodec)
|
||||
.typecase(37002, spliceAckCodec)
|
||||
.typecase(37004, spliceLockedCodec)
|
||||
|
@ -44,6 +44,8 @@ sealed trait InteractiveTxConstructionMessage extends InteractiveTxMessage // <-
|
||||
sealed trait HtlcMessage extends LightningMessage
|
||||
sealed trait RoutingMessage extends LightningMessage
|
||||
sealed trait AnnouncementMessage extends RoutingMessage // <- not in the spec
|
||||
sealed trait OnTheFlyFundingMessage extends LightningMessage { def paymentHash: ByteVector32 }
|
||||
sealed trait OnTheFlyFundingFailureMessage extends OnTheFlyFundingMessage { def id: ByteVector32 }
|
||||
sealed trait HasTimestamp extends LightningMessage { def timestamp: TimestampSecond }
|
||||
sealed trait HasTemporaryChannelId extends LightningMessage { def temporaryChannelId: ByteVector32 } // <- not in the spec
|
||||
sealed trait HasChannelId extends LightningMessage { def channelId: ByteVector32 } // <- not in the spec
|
||||
@ -361,8 +363,9 @@ case class UpdateAddHtlc(channelId: ByteVector32,
|
||||
paymentHash: ByteVector32,
|
||||
cltvExpiry: CltvExpiry,
|
||||
onionRoutingPacket: OnionRoutingPacket,
|
||||
tlvStream: TlvStream[UpdateAddHtlcTlv] = TlvStream.empty) extends HtlcMessage with UpdateMessage with HasChannelId {
|
||||
tlvStream: TlvStream[UpdateAddHtlcTlv]) extends HtlcMessage with UpdateMessage with HasChannelId {
|
||||
val blinding_opt: Option[PublicKey] = tlvStream.get[UpdateAddHtlcTlv.BlindingPoint].map(_.publicKey)
|
||||
val fundingFee_opt: Option[LiquidityAds.FundingFee] = tlvStream.get[UpdateAddHtlcTlv.FundingFeeTlv].map(_.fee)
|
||||
|
||||
val endorsement: Int = tlvStream.get[UpdateAddHtlcTlv.Endorsement].map(_.level).getOrElse(0)
|
||||
|
||||
@ -378,8 +381,13 @@ object UpdateAddHtlc {
|
||||
cltvExpiry: CltvExpiry,
|
||||
onionRoutingPacket: OnionRoutingPacket,
|
||||
blinding_opt: Option[PublicKey],
|
||||
confidence: Double): UpdateAddHtlc = {
|
||||
val tlvs: Set[UpdateAddHtlcTlv] = Set(blinding_opt.map(UpdateAddHtlcTlv.BlindingPoint), Some(UpdateAddHtlcTlv.Endorsement((confidence * 7.999).toInt))).flatten
|
||||
confidence: Double,
|
||||
fundingFee_opt: Option[LiquidityAds.FundingFee]): UpdateAddHtlc = {
|
||||
val tlvs = Set(
|
||||
blinding_opt.map(UpdateAddHtlcTlv.BlindingPoint),
|
||||
fundingFee_opt.map(UpdateAddHtlcTlv.FundingFeeTlv),
|
||||
Some(UpdateAddHtlcTlv.Endorsement((confidence * 7.999).toInt)),
|
||||
).flatten[UpdateAddHtlcTlv]
|
||||
UpdateAddHtlc(channelId, id, amountMsat, paymentHash, cltvExpiry, onionRoutingPacket, TlvStream(tlvs))
|
||||
}
|
||||
}
|
||||
@ -618,4 +626,51 @@ case class RecommendedFeerates(chainHash: BlockHash, fundingFeerate: FeeratePerK
|
||||
val maxCommitmentFeerate: FeeratePerKw = tlvStream.get[RecommendedFeeratesTlv.CommitmentFeerateRange].map(_.max).getOrElse(commitmentFeerate)
|
||||
}
|
||||
|
||||
/**
|
||||
* This message is sent when an HTLC couldn't be relayed to our node because we don't have enough inbound liquidity.
|
||||
* This allows us to treat it as an incoming payment, and request on-the-fly liquidity accordingly if we wish to receive that payment.
|
||||
* If we accept the payment, we will send an [[OpenDualFundedChannel]] or [[SpliceInit]] message containing [[ChannelTlv.RequestFundingTlv]].
|
||||
* Our peer will then provide the requested funding liquidity and will relay the corresponding HTLC(s) afterwards.
|
||||
*/
|
||||
case class WillAddHtlc(chainHash: BlockHash,
|
||||
id: ByteVector32,
|
||||
amount: MilliSatoshi,
|
||||
paymentHash: ByteVector32,
|
||||
expiry: CltvExpiry,
|
||||
finalPacket: OnionRoutingPacket,
|
||||
tlvStream: TlvStream[WillAddHtlcTlv] = TlvStream.empty) extends OnTheFlyFundingMessage {
|
||||
val blinding_opt: Option[PublicKey] = tlvStream.get[WillAddHtlcTlv.BlindingPoint].map(_.publicKey)
|
||||
}
|
||||
|
||||
object WillAddHtlc {
|
||||
def apply(chainHash: BlockHash,
|
||||
id: ByteVector32,
|
||||
amount: MilliSatoshi,
|
||||
paymentHash: ByteVector32,
|
||||
expiry: CltvExpiry,
|
||||
finalPacket: OnionRoutingPacket,
|
||||
blinding_opt: Option[PublicKey]): WillAddHtlc = {
|
||||
val tlvs = blinding_opt.map(WillAddHtlcTlv.BlindingPoint).toSet[WillAddHtlcTlv]
|
||||
WillAddHtlc(chainHash, id, amount, paymentHash, expiry, finalPacket, TlvStream(tlvs))
|
||||
}
|
||||
}
|
||||
|
||||
/** This message is similar to [[UpdateFailHtlc]], but for [[WillAddHtlc]]. */
|
||||
case class WillFailHtlc(id: ByteVector32, paymentHash: ByteVector32, reason: ByteVector) extends OnTheFlyFundingFailureMessage
|
||||
|
||||
/** This message is similar to [[UpdateFailMalformedHtlc]], but for [[WillAddHtlc]]. */
|
||||
case class WillFailMalformedHtlc(id: ByteVector32, paymentHash: ByteVector32, onionHash: ByteVector32, failureCode: Int) extends OnTheFlyFundingFailureMessage
|
||||
|
||||
/**
|
||||
* This message is sent in response to an [[OpenDualFundedChannel]] or [[SpliceInit]] message containing an invalid [[LiquidityAds.RequestFunding]].
|
||||
* The receiver must consider the funding attempt failed when receiving this message.
|
||||
*/
|
||||
case class CancelOnTheFlyFunding(channelId: ByteVector32, paymentHashes: List[ByteVector32], reason: ByteVector) extends LightningMessage with HasChannelId {
|
||||
def toAscii: String = if (isAsciiPrintable(reason)) new String(reason.toArray, StandardCharsets.US_ASCII) else "n/a"
|
||||
}
|
||||
|
||||
object CancelOnTheFlyFunding {
|
||||
def apply(channelId: ByteVector32, paymentHashes: List[ByteVector32], reason: String): CancelOnTheFlyFunding = CancelOnTheFlyFunding(channelId, paymentHashes, ByteVector.view(reason.getBytes(Charsets.US_ASCII)))
|
||||
}
|
||||
|
||||
case class UnknownMessage(tag: Int, data: ByteVector) extends LightningMessage
|
@ -18,13 +18,13 @@ package fr.acinq.eclair.wire.protocol
|
||||
|
||||
import com.google.common.base.Charsets
|
||||
import fr.acinq.bitcoin.scalacompat.Crypto.{PrivateKey, PublicKey}
|
||||
import fr.acinq.bitcoin.scalacompat.{ByteVector32, ByteVector64, Crypto, Satoshi}
|
||||
import fr.acinq.bitcoin.scalacompat.{ByteVector32, ByteVector64, Crypto, Satoshi, TxId}
|
||||
import fr.acinq.eclair.blockchain.fee.FeeratePerKw
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.transactions.Transactions
|
||||
import fr.acinq.eclair.wire.protocol.CommonCodecs._
|
||||
import fr.acinq.eclair.wire.protocol.TlvCodecs.tlvField
|
||||
import fr.acinq.eclair.{ToMilliSatoshiConversion, UInt64}
|
||||
import fr.acinq.eclair.wire.protocol.TlvCodecs.{genericTlv, tlvField, tsatoshi32}
|
||||
import fr.acinq.eclair.{MilliSatoshi, ToMilliSatoshiConversion, UInt64}
|
||||
import scodec.Codec
|
||||
import scodec.bits.{BitVector, ByteVector}
|
||||
import scodec.codecs._
|
||||
@ -48,6 +48,9 @@ object LiquidityAds {
|
||||
val total: Satoshi = miningFee + serviceFee
|
||||
}
|
||||
|
||||
/** Fees paid for the funding transaction that provides liquidity. */
|
||||
case class FundingFee(amount: MilliSatoshi, fundingTxId: TxId)
|
||||
|
||||
/**
|
||||
* Rate at which a liquidity seller sells its liquidity.
|
||||
* Liquidity fees are computed based on multiple components.
|
||||
@ -94,6 +97,12 @@ object LiquidityAds {
|
||||
// @formatter:off
|
||||
/** Fees are transferred from the buyer's channel balance to the seller's during the interactive-tx construction. */
|
||||
case object FromChannelBalance extends PaymentType { override val rfcName: String = "from_channel_balance" }
|
||||
/** Fees will be deducted from future HTLCs that will be relayed to the buyer. */
|
||||
case object FromFutureHtlc extends PaymentType { override val rfcName: String = "from_future_htlc" }
|
||||
/** Fees will be deducted from future HTLCs that will be relayed to the buyer, but the preimage is revealed immediately. */
|
||||
case object FromFutureHtlcWithPreimage extends PaymentType { override val rfcName: String = "from_future_htlc_with_preimage" }
|
||||
/** Similar to [[FromChannelBalance]] but expects HTLCs to be relayed after funding. */
|
||||
case object FromChannelBalanceForFutureHtlc extends PaymentType { override val rfcName: String = "from_channel_balance_for_future_htlc" }
|
||||
/** Sellers may support unknown payment types, which we must ignore. */
|
||||
case class Unknown(bitIndex: Int) extends PaymentType { override val rfcName: String = s"unknown_$bitIndex" }
|
||||
// @formatter:on
|
||||
@ -107,6 +116,9 @@ object LiquidityAds {
|
||||
object PaymentDetails {
|
||||
// @formatter:off
|
||||
case object FromChannelBalance extends PaymentDetails { override val paymentType: PaymentType = PaymentType.FromChannelBalance }
|
||||
case class FromFutureHtlc(paymentHashes: List[ByteVector32]) extends PaymentDetails { override val paymentType: PaymentType = PaymentType.FromFutureHtlc }
|
||||
case class FromFutureHtlcWithPreimage(preimages: List[ByteVector32]) extends PaymentDetails { override val paymentType: PaymentType = PaymentType.FromFutureHtlcWithPreimage }
|
||||
case class FromChannelBalanceForFutureHtlc(paymentHashes: List[ByteVector32]) extends PaymentDetails { override val paymentType: PaymentType = PaymentType.FromChannelBalanceForFutureHtlc }
|
||||
// @formatter:on
|
||||
}
|
||||
|
||||
@ -230,6 +242,9 @@ object LiquidityAds {
|
||||
|
||||
private val paymentDetails: Codec[PaymentDetails] = discriminated[PaymentDetails].by(varint)
|
||||
.typecase(UInt64(0), tlvField(provide(PaymentDetails.FromChannelBalance)))
|
||||
.typecase(UInt64(128), tlvField(list(bytes32).as[PaymentDetails.FromFutureHtlc]))
|
||||
.typecase(UInt64(129), tlvField(list(bytes32).as[PaymentDetails.FromFutureHtlcWithPreimage]))
|
||||
.typecase(UInt64(130), tlvField(list(bytes32).as[PaymentDetails.FromChannelBalanceForFutureHtlc]))
|
||||
|
||||
val requestFunding: Codec[RequestFunding] = (
|
||||
("requestedAmount" | satoshi) ::
|
||||
@ -247,12 +262,18 @@ object LiquidityAds {
|
||||
f = { bytes =>
|
||||
bytes.bits.toIndexedSeq.reverse.zipWithIndex.collect {
|
||||
case (true, 0) => PaymentType.FromChannelBalance
|
||||
case (true, 128) => PaymentType.FromFutureHtlc
|
||||
case (true, 129) => PaymentType.FromFutureHtlcWithPreimage
|
||||
case (true, 130) => PaymentType.FromChannelBalanceForFutureHtlc
|
||||
case (true, idx) => PaymentType.Unknown(idx)
|
||||
}.toSet
|
||||
},
|
||||
g = { paymentTypes =>
|
||||
val indexes = paymentTypes.collect {
|
||||
case PaymentType.FromChannelBalance => 0
|
||||
case PaymentType.FromFutureHtlc => 128
|
||||
case PaymentType.FromFutureHtlcWithPreimage => 129
|
||||
case PaymentType.FromChannelBalanceForFutureHtlc => 130
|
||||
case PaymentType.Unknown(idx) => idx
|
||||
}
|
||||
// When converting from BitVector to ByteVector, scodec pads right instead of left, so we make sure we pad to bytes *before* setting bits.
|
||||
|
@ -0,0 +1,42 @@
|
||||
/*
|
||||
* Copyright 2024 ACINQ SAS
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package fr.acinq.eclair.wire.protocol
|
||||
|
||||
import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
|
||||
import fr.acinq.eclair.UInt64
|
||||
import fr.acinq.eclair.wire.protocol.CommonCodecs.{publicKey, varint}
|
||||
import fr.acinq.eclair.wire.protocol.TlvCodecs.tlvStream
|
||||
import scodec.Codec
|
||||
import scodec.bits.HexStringSyntax
|
||||
import scodec.codecs._
|
||||
|
||||
/**
|
||||
* Created by t-bast on 07/06/2024.
|
||||
*/
|
||||
|
||||
sealed trait WillAddHtlcTlv extends Tlv
|
||||
|
||||
object WillAddHtlcTlv {
|
||||
/** Blinding ephemeral public key that should be used to derive shared secrets when using route blinding. */
|
||||
case class BlindingPoint(publicKey: PublicKey) extends WillAddHtlcTlv
|
||||
|
||||
private val blindingPoint: Codec[BlindingPoint] = (("length" | constant(hex"21")) :: ("blinding" | publicKey)).as[BlindingPoint]
|
||||
|
||||
val willAddHtlcTlvCodec: Codec[TlvStream[WillAddHtlcTlv]] = tlvStream(discriminated[WillAddHtlcTlv].by(varint)
|
||||
.typecase(UInt64(0), blindingPoint)
|
||||
)
|
||||
}
|
@ -47,6 +47,7 @@
|
||||
}
|
||||
},
|
||||
"channelFlags" : {
|
||||
"nonInitiatorPaysCommitFees" : false,
|
||||
"announceChannel" : true
|
||||
}
|
||||
},
|
||||
|
@ -47,6 +47,7 @@
|
||||
}
|
||||
},
|
||||
"channelFlags" : {
|
||||
"nonInitiatorPaysCommitFees" : false,
|
||||
"announceChannel" : true
|
||||
}
|
||||
},
|
||||
|
@ -61,6 +61,7 @@
|
||||
}
|
||||
},
|
||||
"channelFlags" : {
|
||||
"nonInitiatorPaysCommitFees" : false,
|
||||
"announceChannel" : true
|
||||
}
|
||||
},
|
||||
|
@ -54,6 +54,7 @@
|
||||
}
|
||||
},
|
||||
"channelFlags" : {
|
||||
"nonInitiatorPaysCommitFees" : false,
|
||||
"announceChannel" : false
|
||||
}
|
||||
},
|
||||
|
@ -56,6 +56,7 @@
|
||||
}
|
||||
},
|
||||
"channelFlags" : {
|
||||
"nonInitiatorPaysCommitFees" : false,
|
||||
"announceChannel" : false
|
||||
}
|
||||
},
|
||||
|
@ -62,6 +62,7 @@
|
||||
}
|
||||
},
|
||||
"channelFlags" : {
|
||||
"nonInitiatorPaysCommitFees" : false,
|
||||
"announceChannel" : false
|
||||
}
|
||||
},
|
||||
|
@ -18,8 +18,6 @@ package fr.acinq.eclair
|
||||
|
||||
import akka.actor.ActorRef
|
||||
import fr.acinq.bitcoin.scalacompat.{Block, ByteVector32, Satoshi, SatoshiLong}
|
||||
import fr.acinq.eclair.FeatureSupport.{Mandatory, Optional}
|
||||
import fr.acinq.eclair.Features._
|
||||
import fr.acinq.eclair.blockchain.fee._
|
||||
import fr.acinq.eclair.channel.fsm.Channel.{ChannelConf, RemoteRbfLimits, UnhandledExceptionStrategy}
|
||||
import fr.acinq.eclair.channel.{ChannelFlags, LocalParams, Origin, Upstream}
|
||||
@ -28,6 +26,7 @@ import fr.acinq.eclair.db.RevokedHtlcInfoCleaner
|
||||
import fr.acinq.eclair.io.MessageRelay.RelayAll
|
||||
import fr.acinq.eclair.io.{OpenChannelInterceptor, PeerConnection, PeerReadyNotifier}
|
||||
import fr.acinq.eclair.message.OnionMessages.OnionMessageConfig
|
||||
import fr.acinq.eclair.payment.relay.OnTheFlyFunding
|
||||
import fr.acinq.eclair.payment.relay.Relayer.{AsyncPaymentsParams, RelayFees, RelayParams}
|
||||
import fr.acinq.eclair.router.Graph.{MessagePath, WeightRatios}
|
||||
import fr.acinq.eclair.router.PathFindingExperimentConf
|
||||
@ -97,16 +96,16 @@ object TestConstants {
|
||||
torAddress_opt = None,
|
||||
features = Features(
|
||||
Map(
|
||||
DataLossProtect -> Optional,
|
||||
ChannelRangeQueries -> Optional,
|
||||
ChannelRangeQueriesExtended -> Optional,
|
||||
VariableLengthOnion -> Mandatory,
|
||||
PaymentSecret -> Mandatory,
|
||||
BasicMultiPartPayment -> Optional,
|
||||
Wumbo -> Optional,
|
||||
PaymentMetadata -> Optional,
|
||||
RouteBlinding -> Optional,
|
||||
StaticRemoteKey -> Mandatory
|
||||
Features.DataLossProtect -> FeatureSupport.Optional,
|
||||
Features.ChannelRangeQueries -> FeatureSupport.Optional,
|
||||
Features.ChannelRangeQueriesExtended -> FeatureSupport.Optional,
|
||||
Features.VariableLengthOnion -> FeatureSupport.Mandatory,
|
||||
Features.PaymentSecret -> FeatureSupport.Mandatory,
|
||||
Features.BasicMultiPartPayment -> FeatureSupport.Optional,
|
||||
Features.Wumbo -> FeatureSupport.Optional,
|
||||
Features.PaymentMetadata -> FeatureSupport.Optional,
|
||||
Features.RouteBlinding -> FeatureSupport.Optional,
|
||||
Features.StaticRemoteKey -> FeatureSupport.Mandatory,
|
||||
),
|
||||
unknown = Set(UnknownFeature(TestFeature.optional))
|
||||
),
|
||||
@ -238,6 +237,7 @@ object TestConstants {
|
||||
revokedHtlcInfoCleanerConfig = RevokedHtlcInfoCleaner.Config(10, 100 millis),
|
||||
willFundRates_opt = Some(defaultLiquidityRates),
|
||||
peerWakeUpConfig = PeerReadyNotifier.WakeUpConfig(enabled = false, timeout = 30 seconds),
|
||||
onTheFlyFundingConfig = OnTheFlyFunding.Config(proposalTimeout = 90 seconds),
|
||||
)
|
||||
|
||||
def channelParams: LocalParams = OpenChannelInterceptor.makeChannelParams(
|
||||
@ -270,17 +270,17 @@ object TestConstants {
|
||||
publicAddresses = NodeAddress.fromParts("localhost", 9732).get :: Nil,
|
||||
torAddress_opt = None,
|
||||
features = Features(
|
||||
DataLossProtect -> Optional,
|
||||
ChannelRangeQueries -> Optional,
|
||||
ChannelRangeQueriesExtended -> Optional,
|
||||
VariableLengthOnion -> Mandatory,
|
||||
PaymentSecret -> Mandatory,
|
||||
BasicMultiPartPayment -> Optional,
|
||||
Wumbo -> Optional,
|
||||
PaymentMetadata -> Optional,
|
||||
RouteBlinding -> Optional,
|
||||
StaticRemoteKey -> Mandatory,
|
||||
AnchorOutputsZeroFeeHtlcTx -> Optional
|
||||
Features.DataLossProtect -> FeatureSupport.Optional,
|
||||
Features.ChannelRangeQueries -> FeatureSupport.Optional,
|
||||
Features.ChannelRangeQueriesExtended -> FeatureSupport.Optional,
|
||||
Features.VariableLengthOnion -> FeatureSupport.Mandatory,
|
||||
Features.PaymentSecret -> FeatureSupport.Mandatory,
|
||||
Features.BasicMultiPartPayment -> FeatureSupport.Optional,
|
||||
Features.Wumbo -> FeatureSupport.Optional,
|
||||
Features.PaymentMetadata -> FeatureSupport.Optional,
|
||||
Features.RouteBlinding -> FeatureSupport.Optional,
|
||||
Features.StaticRemoteKey -> FeatureSupport.Mandatory,
|
||||
Features.AnchorOutputsZeroFeeHtlcTx -> FeatureSupport.Optional,
|
||||
),
|
||||
pluginParams = Nil,
|
||||
overrideInitFeatures = Map.empty,
|
||||
@ -410,6 +410,7 @@ object TestConstants {
|
||||
revokedHtlcInfoCleanerConfig = RevokedHtlcInfoCleaner.Config(10, 100 millis),
|
||||
willFundRates_opt = Some(defaultLiquidityRates),
|
||||
peerWakeUpConfig = PeerReadyNotifier.WakeUpConfig(enabled = false, timeout = 30 seconds),
|
||||
onTheFlyFundingConfig = OnTheFlyFunding.Config(proposalTimeout = 90 seconds),
|
||||
)
|
||||
|
||||
def channelParams: LocalParams = OpenChannelInterceptor.makeChannelParams(
|
||||
|
@ -409,7 +409,7 @@ class CommitmentsSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
test("can receive availableForReceive") { f =>
|
||||
for (isInitiator <- Seq(true, false)) {
|
||||
val c = CommitmentsSpec.makeCommitments(31000000 msat, 702000000 msat, FeeratePerKw(2679 sat), 546 sat, isInitiator)
|
||||
val add = UpdateAddHtlc(randomBytes32(), c.changes.remoteNextHtlcId, c.availableBalanceForReceive, randomBytes32(), CltvExpiry(f.currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val add = UpdateAddHtlc(randomBytes32(), c.changes.remoteNextHtlcId, c.availableBalanceForReceive, randomBytes32(), CltvExpiry(f.currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
c.receiveAdd(add, feerates, feeConfNoMismatch)
|
||||
}
|
||||
}
|
||||
@ -460,14 +460,14 @@ class CommitmentsSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
// Add some initial HTLCs to the pending list (bigger commit tx).
|
||||
for (_ <- 1 to t.pendingHtlcs) {
|
||||
val amount = Random.nextInt(maxPendingHtlcAmount.toLong.toInt).msat.max(1 msat)
|
||||
val add = UpdateAddHtlc(randomBytes32(), c.changes.remoteNextHtlcId, amount, randomBytes32(), CltvExpiry(f.currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val add = UpdateAddHtlc(randomBytes32(), c.changes.remoteNextHtlcId, amount, randomBytes32(), CltvExpiry(f.currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
c.receiveAdd(add, feerates, feeConfNoMismatch) match {
|
||||
case Right(cc) => c = cc
|
||||
case Left(e) => ignore(s"$t -> could not setup initial htlcs: $e")
|
||||
}
|
||||
}
|
||||
if (c.availableBalanceForReceive > 0.msat) {
|
||||
val add = UpdateAddHtlc(randomBytes32(), c.changes.remoteNextHtlcId, c.availableBalanceForReceive, randomBytes32(), CltvExpiry(f.currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val add = UpdateAddHtlc(randomBytes32(), c.changes.remoteNextHtlcId, c.availableBalanceForReceive, randomBytes32(), CltvExpiry(f.currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
c.receiveAdd(add, feerates, feeConfNoMismatch) match {
|
||||
case Right(_) => ()
|
||||
case Left(e) => fail(s"$t -> $e")
|
||||
|
@ -26,7 +26,7 @@ import org.scalatest.funsuite.AnyFunSuiteLike
|
||||
class DustExposureSpec extends AnyFunSuiteLike {
|
||||
|
||||
def createHtlc(id: Long, amount: MilliSatoshi): UpdateAddHtlc = {
|
||||
UpdateAddHtlc(ByteVector32.Zeroes, id, amount, randomBytes32(), CltvExpiry(500), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
UpdateAddHtlc(ByteVector32.Zeroes, id, amount, randomBytes32(), CltvExpiry(500), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
}
|
||||
|
||||
test("compute dust exposure") {
|
||||
|
@ -79,7 +79,7 @@ class FuzzySpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with Channe
|
||||
// no announcements
|
||||
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = false, TestConstants.feeratePerKw, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, Some(TestConstants.initiatorPushAmount), requireConfirmedInputs = false, requestFunding_opt = None, Alice.channelParams, pipe, bobInit, channelFlags, ChannelConfig.standard, ChannelTypes.Standard(), replyTo = system.deadLetters)
|
||||
alice2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, None, dualFunded = false, None, Bob.channelParams, pipe, aliceInit, ChannelConfig.standard, ChannelTypes.Standard())
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, None, dualFunded = false, None, requireConfirmedInputs = false, Bob.channelParams, pipe, aliceInit, ChannelConfig.standard, ChannelTypes.Standard())
|
||||
bob2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
pipe ! (alice, bob)
|
||||
alice2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
|
@ -68,14 +68,14 @@ class HelpersSpec extends TestKitBaseClass with AnyFunSuiteLike with ChannelStat
|
||||
awaitCond(bob.stateName == NORMAL)
|
||||
// We have two identical HTLCs (MPP):
|
||||
val (_, htlca1a) = addHtlc(15_000_000 msat, alice, bob, alice2bob, bob2alice)
|
||||
val aliceMppCmd = CMD_ADD_HTLC(TestProbe().ref, 15_000_000 msat, htlca1a.paymentHash, htlca1a.cltvExpiry, htlca1a.onionRoutingPacket, None, 1.0, Origin.Hot(TestProbe().ref, Upstream.Local(UUID.randomUUID())))
|
||||
val aliceMppCmd = CMD_ADD_HTLC(TestProbe().ref, 15_000_000 msat, htlca1a.paymentHash, htlca1a.cltvExpiry, htlca1a.onionRoutingPacket, None, 1.0, None, Origin.Hot(TestProbe().ref, Upstream.Local(UUID.randomUUID())))
|
||||
val htlca1b = addHtlc(aliceMppCmd, alice, bob, alice2bob, bob2alice)
|
||||
val (ra2, htlca2) = addHtlc(16_000_000 msat, alice, bob, alice2bob, bob2alice)
|
||||
addHtlc(500_000 msat, alice, bob, alice2bob, bob2alice) // below dust
|
||||
crossSign(alice, bob, alice2bob, bob2alice)
|
||||
// We have two identical HTLCs (MPP):
|
||||
val (_, htlcb1a) = addHtlc(17_000_000 msat, bob, alice, bob2alice, alice2bob)
|
||||
val bobMppCmd = CMD_ADD_HTLC(TestProbe().ref, 17_000_000 msat, htlcb1a.paymentHash, htlcb1a.cltvExpiry, htlcb1a.onionRoutingPacket, None, 1.0, Origin.Hot(TestProbe().ref, Upstream.Local(UUID.randomUUID())))
|
||||
val bobMppCmd = CMD_ADD_HTLC(TestProbe().ref, 17_000_000 msat, htlcb1a.paymentHash, htlcb1a.cltvExpiry, htlcb1a.onionRoutingPacket, None, 1.0, None, Origin.Hot(TestProbe().ref, Upstream.Local(UUID.randomUUID())))
|
||||
val htlcb1b = addHtlc(bobMppCmd, bob, alice, bob2alice, alice2bob)
|
||||
val (rb2, htlcb2) = addHtlc(18_000_000 msat, bob, alice, bob2alice, alice2bob)
|
||||
addHtlc(400_000 msat, bob, alice, bob2alice, alice2bob) // below dust
|
||||
|
@ -211,11 +211,11 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit
|
||||
}
|
||||
}
|
||||
|
||||
private def createFixtureParams(fundingAmountA: Satoshi, fundingAmountB: Satoshi, targetFeerate: FeeratePerKw, dustLimit: Satoshi, lockTime: Long, requireConfirmedInputs: RequireConfirmedInputs = RequireConfirmedInputs(forLocal = false, forRemote = false)): FixtureParams = {
|
||||
private def createFixtureParams(fundingAmountA: Satoshi, fundingAmountB: Satoshi, targetFeerate: FeeratePerKw, dustLimit: Satoshi, lockTime: Long, requireConfirmedInputs: RequireConfirmedInputs = RequireConfirmedInputs(forLocal = false, forRemote = false), nonInitiatorPaysCommitTxFees: Boolean = false): FixtureParams = {
|
||||
val channelFeatures = ChannelFeatures(ChannelTypes.AnchorOutputsZeroFeeHtlcTx(), Features[InitFeature](Features.DualFunding -> FeatureSupport.Optional), Features[InitFeature](Features.DualFunding -> FeatureSupport.Optional), announceChannel = true)
|
||||
val Seq(nodeParamsA, nodeParamsB) = Seq(TestConstants.Alice.nodeParams, TestConstants.Bob.nodeParams).map(_.copy(features = Features(channelFeatures.features.map(f => f -> FeatureSupport.Optional).toMap[Feature, FeatureSupport])))
|
||||
val localParamsA = makeChannelParams(nodeParamsA, nodeParamsA.features.initFeatures(), None, None, isChannelOpener = true, dualFunded = true, fundingAmountA, unlimitedMaxHtlcValueInFlight = false)
|
||||
val localParamsB = makeChannelParams(nodeParamsB, nodeParamsB.features.initFeatures(), None, None, isChannelOpener = false, dualFunded = true, fundingAmountB, unlimitedMaxHtlcValueInFlight = false)
|
||||
val localParamsA = makeChannelParams(nodeParamsA, nodeParamsA.features.initFeatures(), None, None, isChannelOpener = true, dualFunded = true, fundingAmountA, unlimitedMaxHtlcValueInFlight = false).copy(paysCommitTxFees = !nonInitiatorPaysCommitTxFees)
|
||||
val localParamsB = makeChannelParams(nodeParamsB, nodeParamsB.features.initFeatures(), None, None, isChannelOpener = false, dualFunded = true, fundingAmountB, unlimitedMaxHtlcValueInFlight = false).copy(paysCommitTxFees = nonInitiatorPaysCommitTxFees)
|
||||
|
||||
val Seq(remoteParamsA, remoteParamsB) = Seq((nodeParamsA, localParamsA), (nodeParamsB, localParamsB)).map {
|
||||
case (nodeParams, localParams) =>
|
||||
@ -287,7 +287,7 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit
|
||||
utxosB.foreach(amount => addUtxo(walletB, amount, probe))
|
||||
generateBlocks(1)
|
||||
|
||||
val fixtureParams = createFixtureParams(fundingAmountA, fundingAmountB, targetFeerate, dustLimit, lockTime, requireConfirmedInputs)
|
||||
val fixtureParams = createFixtureParams(fundingAmountA, fundingAmountB, targetFeerate, dustLimit, lockTime, requireConfirmedInputs, nonInitiatorPaysCommitTxFees = liquidityPurchase_opt.nonEmpty)
|
||||
val alice = fixtureParams.spawnTxBuilderAlice(walletA, liquidityPurchase_opt = liquidityPurchase_opt)
|
||||
val bob = fixtureParams.spawnTxBuilderBob(walletB, liquidityPurchase_opt = liquidityPurchase_opt)
|
||||
testFun(Fixture(alice, bob, fixtureParams, walletA, rpcClientA, walletB, rpcClientB, TestProbe(), TestProbe()))
|
||||
@ -564,6 +564,59 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit
|
||||
}
|
||||
}
|
||||
|
||||
test("initiator does not contribute -- on-the-fly funding") {
|
||||
val targetFeerate = FeeratePerKw(5000 sat)
|
||||
val fundingB = 150_000.sat
|
||||
val utxosB = Seq(200_000 sat)
|
||||
// When on-the-fly funding is used, the initiator may not contribute to the funding transaction.
|
||||
// It will receive HTLCs later that use the purchased inbound liquidity, and liquidity fees will be deduced from those HTLCs.
|
||||
val purchase = LiquidityAds.Purchase.Standard(fundingB, LiquidityAds.Fees(2500 sat, 7500 sat), LiquidityAds.PaymentDetails.FromFutureHtlc(Nil))
|
||||
withFixture(0 sat, Nil, fundingB, utxosB, targetFeerate, 330 sat, 0, RequireConfirmedInputs(forLocal = false, forRemote = false), Some(purchase)) { f =>
|
||||
import f._
|
||||
|
||||
alice ! Start(alice2bob.ref)
|
||||
bob ! Start(bob2alice.ref)
|
||||
|
||||
// Alice --- tx_add_output --> Bob
|
||||
fwd.forwardAlice2Bob[TxAddOutput]
|
||||
// Alice <-- tx_add_input --- Bob
|
||||
fwd.forwardBob2Alice[TxAddInput]
|
||||
// Alice --- tx_complete --> Bob
|
||||
fwd.forwardAlice2Bob[TxComplete]
|
||||
// Alice <-- tx_add_output --- Bob
|
||||
fwd.forwardBob2Alice[TxAddOutput]
|
||||
// Alice --- tx_complete --> Bob
|
||||
fwd.forwardAlice2Bob[TxComplete]
|
||||
// Alice <-- tx_complete --- Bob
|
||||
fwd.forwardBob2Alice[TxComplete]
|
||||
|
||||
// Alice is responsible for adding the shared output, but Bob is paying for everything.
|
||||
assert(aliceParams.fundingAmount == fundingB)
|
||||
|
||||
// Alice sends signatures first as she did not contribute at all.
|
||||
val successA = alice2bob.expectMsgType[Succeeded]
|
||||
val successB = bob2alice.expectMsgType[Succeeded]
|
||||
val (txA, _, txB, commitmentB) = fixtureParams.exchangeSigsAliceFirst(aliceParams, successA, successB)
|
||||
// Alice doesn't pay any fees to Bob during the interactive-tx, fees will be paid from future HTLCs.
|
||||
assert(commitmentB.localCommit.spec.toLocal == fundingB.toMilliSatoshi)
|
||||
|
||||
// The resulting transaction is valid but has a lower feerate than expected.
|
||||
assert(txA.txId == txB.txId)
|
||||
assert(txA.tx.localAmountIn == 0.msat)
|
||||
assert(txA.tx.localFees == 0.msat)
|
||||
assert(txB.tx.remoteAmountIn == 0.msat)
|
||||
assert(txB.tx.remoteFees == 0.msat)
|
||||
assert(txB.tx.localFees > 0.msat)
|
||||
val probe = TestProbe()
|
||||
walletA.publishTransaction(txA.signedTx).pipeTo(probe.ref)
|
||||
probe.expectMsg(txA.txId)
|
||||
walletA.getMempoolTx(txA.txId).pipeTo(probe.ref)
|
||||
val mempoolTx = probe.expectMsgType[MempoolTx]
|
||||
assert(mempoolTx.fees == txA.tx.fees)
|
||||
assert(targetFeerate * 0.5 <= txA.feerate && txA.feerate < targetFeerate, s"unexpected feerate (target=$targetFeerate actual=${txA.feerate})")
|
||||
}
|
||||
}
|
||||
|
||||
test("initiator and non-initiator splice-in") {
|
||||
val targetFeerate = FeeratePerKw(1000 sat)
|
||||
// We chose those amounts to ensure that Bob always signs first:
|
||||
@ -2194,13 +2247,22 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit
|
||||
val bob = params.spawnTxBuilderBob(wallet, params.fundingParamsB, Some(purchase))
|
||||
bob ! Start(probe.ref)
|
||||
assert(probe.expectMsgType[LocalFailure].cause == InvalidFundingBalances(params.channelId, 524_000 sat, 525_000_000 msat, -1_000_000 msat))
|
||||
// Bob reject a splice proposed by Alice where she doesn't have enough funds to pay the liquidity fees.
|
||||
// Bob rejects a splice proposed by Alice where she doesn't have enough funds to pay the liquidity fees.
|
||||
val previousCommitment = CommitmentsSpec.makeCommitments(450_000_000 msat, 50_000_000 msat).active.head
|
||||
val sharedInput = params.dummySharedInputB(500_000 sat)
|
||||
val spliceParams = params.fundingParamsB.copy(localContribution = 150_000 sat, remoteContribution = -30_000 sat, sharedInput_opt = Some(sharedInput))
|
||||
val bobSplice = params.spawnTxBuilderSpliceBob(spliceParams, previousCommitment, wallet, Some(purchase))
|
||||
bobSplice ! Start(probe.ref)
|
||||
assert(probe.expectMsgType[LocalFailure].cause == InvalidFundingBalances(params.channelId, 620_000 sat, 625_000_000 msat, -5_000_000 msat))
|
||||
// If we use a payment type where fees are paid outside of the interactive-tx session, the funding attempt is valid.
|
||||
val bobFutureHtlc = params.spawnTxBuilderBob(wallet, params.fundingParamsB, Some(purchase.copy(paymentDetails = LiquidityAds.PaymentDetails.FromFutureHtlc(Nil))))
|
||||
bobFutureHtlc ! Start(probe.ref)
|
||||
probe.expectNoMessage(100 millis)
|
||||
// Bob rejects a splice proposed by Alice where she has enough funds to pay the liquidity fees, but wants to pay
|
||||
// them outside of the interactive-tx session, which requires some trust.
|
||||
val bobFutureHtlcWithBalance = params.spawnTxBuilderSpliceBob(spliceParams, previousCommitment, wallet, Some(purchase.copy(fees = LiquidityAds.Fees(1000 sat, 4000 sat), paymentDetails = LiquidityAds.PaymentDetails.FromFutureHtlc(Nil))))
|
||||
bobFutureHtlcWithBalance ! Start(probe.ref)
|
||||
assert(probe.expectMsgType[LocalFailure].cause == InvalidLiquidityAdsPaymentType(params.channelId, LiquidityAds.PaymentType.FromFutureHtlc, Set(LiquidityAds.PaymentType.FromChannelBalance, LiquidityAds.PaymentType.FromChannelBalanceForFutureHtlc)))
|
||||
}
|
||||
|
||||
test("invalid input") {
|
||||
|
@ -270,7 +270,7 @@ trait ChannelStateTestsBase extends Assertions with Eventually {
|
||||
val bobInit = Init(bobParams.initFeatures)
|
||||
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, fundingAmount, dualFunded, commitTxFeerate, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, initiatorPushAmount, requireConfirmedInputs = false, requestFunds_opt, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped)
|
||||
assert(alice2blockchain.expectMsgType[TxPublisher.SetChannelId].channelId == ByteVector32.Zeroes)
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, nonInitiatorFunding_opt, dualFunded, nonInitiatorPushAmount, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, nonInitiatorFunding_opt, dualFunded, nonInitiatorPushAmount, requireConfirmedInputs = false, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
|
||||
assert(bob2blockchain.expectMsgType[TxPublisher.SetChannelId].channelId == ByteVector32.Zeroes)
|
||||
|
||||
val fundingTx = if (!dualFunded) {
|
||||
|
@ -67,7 +67,7 @@ class WaitForAcceptChannelStateSpec extends TestKitBaseClass with FixtureAnyFunS
|
||||
alice.underlying.system.eventStream.subscribe(listener.ref, classOf[ChannelAborted])
|
||||
val fundingAmount = if (test.tags.contains(LargeChannel)) Btc(5).toSatoshi else TestConstants.fundingSatoshis
|
||||
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, fundingAmount, dualFunded = false, commitTxFeerate, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, Some(TestConstants.initiatorPushAmount), requireConfirmedInputs = false, requestFunding_opt = None, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped)
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, None, dualFunded = false, None, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, None, dualFunded = false, None, requireConfirmedInputs = false, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
|
||||
alice2bob.expectMsgType[OpenChannel]
|
||||
alice2bob.forward(bob)
|
||||
awaitCond(alice.stateName == WAIT_FOR_ACCEPT_CHANNEL)
|
||||
@ -173,7 +173,7 @@ class WaitForAcceptChannelStateSpec extends TestKitBaseClass with FixtureAnyFunS
|
||||
val aliceParams = Alice.channelParams
|
||||
val bobParams = Bob.channelParams.copy(initFeatures = Features(Features.StaticRemoteKey -> FeatureSupport.Optional, Features.AnchorOutputs -> FeatureSupport.Optional))
|
||||
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = false, TestConstants.anchorOutputsFeeratePerKw, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, Some(TestConstants.initiatorPushAmount), requireConfirmedInputs = false, requestFunding_opt = None, aliceParams, alice2bob.ref, Init(bobParams.initFeatures), channelFlags, channelConfig, ChannelTypes.AnchorOutputs(), replyTo = aliceOpenReplyTo.ref.toTyped)
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, None, dualFunded = false, None, bobParams, bob2alice.ref, Init(bobParams.initFeatures), channelConfig, ChannelTypes.AnchorOutputs())
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, None, dualFunded = false, None, requireConfirmedInputs = false, bobParams, bob2alice.ref, Init(bobParams.initFeatures), channelConfig, ChannelTypes.AnchorOutputs())
|
||||
val open = alice2bob.expectMsgType[OpenChannel]
|
||||
assert(open.channelType_opt.contains(ChannelTypes.AnchorOutputs()))
|
||||
alice2bob.forward(bob, open)
|
||||
|
@ -65,7 +65,7 @@ class WaitForAcceptDualFundedChannelStateSpec extends TestKitBaseClass with Fixt
|
||||
within(30 seconds) {
|
||||
alice.underlying.system.eventStream.subscribe(listener.ref, classOf[ChannelAborted])
|
||||
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = true, TestConstants.anchorOutputsFeeratePerKw, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, None, requireConfirmedInputs = false, requestFunds_opt, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped)
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, nonInitiatorContribution, dualFunded = true, nonInitiatorPushAmount, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, nonInitiatorContribution, dualFunded = true, nonInitiatorPushAmount, requireConfirmedInputs = test.tags.contains(bobRequiresConfirmedInputs), bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
|
||||
val open = alice2bob.expectMsgType[OpenDualFundedChannel]
|
||||
alice2bob.forward(bob, open)
|
||||
awaitCond(alice.stateName == WAIT_FOR_ACCEPT_DUAL_FUNDED_CHANNEL)
|
||||
|
@ -58,7 +58,7 @@ class WaitForOpenChannelStateSpec extends TestKitBaseClass with FixtureAnyFunSui
|
||||
within(30 seconds) {
|
||||
bob.underlying.system.eventStream.subscribe(listener.ref, classOf[ChannelAborted])
|
||||
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = false, commitTxFeerate, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, Some(TestConstants.initiatorPushAmount), requireConfirmedInputs = false, requestFunding_opt = None, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped)
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, None, dualFunded = false, None, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, None, dualFunded = false, None, requireConfirmedInputs = false, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
|
||||
awaitCond(bob.stateName == WAIT_FOR_OPEN_CHANNEL)
|
||||
withFixture(test.toNoArgTest(FixtureParam(alice, bob, alice2bob, bob2alice, bob2blockchain, listener)))
|
||||
}
|
||||
|
@ -59,7 +59,7 @@ class WaitForOpenDualFundedChannelStateSpec extends TestKitBaseClass with Fixtur
|
||||
val requireConfirmedInputs = test.tags.contains(aliceRequiresConfirmedInputs)
|
||||
within(30 seconds) {
|
||||
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = true, TestConstants.anchorOutputsFeeratePerKw, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, pushAmount, requireConfirmedInputs, None, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped)
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, nonInitiatorContribution, dualFunded = true, None, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, nonInitiatorContribution, dualFunded = true, None, requireConfirmedInputs = false, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
|
||||
awaitCond(bob.stateName == WAIT_FOR_OPEN_DUAL_FUNDED_CHANNEL)
|
||||
withFixture(test.toNoArgTest(FixtureParam(alice, bob, alice2bob, bob2alice, aliceListener, bobListener)))
|
||||
}
|
||||
|
@ -54,7 +54,7 @@ class WaitForDualFundingCreatedStateSpec extends TestKitBaseClass with FixtureAn
|
||||
alice.underlying.system.eventStream.subscribe(aliceListener.ref, classOf[ChannelAborted])
|
||||
bob.underlying.system.eventStream.subscribe(bobListener.ref, classOf[ChannelAborted])
|
||||
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = true, TestConstants.feeratePerKw, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, None, requireConfirmedInputs = false, requestFunding_opt = None, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped)
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, Some(LiquidityAds.AddFunding(TestConstants.nonInitiatorFundingSatoshis, None)), dualFunded = true, None, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, Some(LiquidityAds.AddFunding(TestConstants.nonInitiatorFundingSatoshis, None)), dualFunded = true, None, requireConfirmedInputs = false, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
|
||||
alice2blockchain.expectMsgType[TxPublisher.SetChannelId] // temporary channel id
|
||||
bob2blockchain.expectMsgType[TxPublisher.SetChannelId] // temporary channel id
|
||||
alice2bob.expectMsgType[OpenDualFundedChannel]
|
||||
|
@ -28,7 +28,7 @@ import fr.acinq.eclair.channel.fsm.Channel
|
||||
import fr.acinq.eclair.channel.fund.InteractiveTxBuilder.{FullySignedSharedTransaction, PartiallySignedSharedTransaction}
|
||||
import fr.acinq.eclair.channel.publish.TxPublisher
|
||||
import fr.acinq.eclair.channel.states.{ChannelStateTestsBase, ChannelStateTestsTags}
|
||||
import fr.acinq.eclair.io.Peer.OpenChannelResponse
|
||||
import fr.acinq.eclair.io.Peer.{LiquidityPurchaseSigned, OpenChannelResponse}
|
||||
import fr.acinq.eclair.transactions.Transactions
|
||||
import fr.acinq.eclair.wire.protocol._
|
||||
import fr.acinq.eclair.{Features, MilliSatoshiLong, TestConstants, TestKitBaseClass, ToMilliSatoshiConversion}
|
||||
@ -39,7 +39,7 @@ import scala.concurrent.duration.DurationInt
|
||||
|
||||
class WaitForDualFundingSignedStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with ChannelStateTestsBase {
|
||||
|
||||
case class FixtureParam(alice: TestFSMRef[ChannelState, ChannelData, Channel], bob: TestFSMRef[ChannelState, ChannelData, Channel], alice2bob: TestProbe, bob2alice: TestProbe, alice2blockchain: TestProbe, bob2blockchain: TestProbe, wallet: SingleKeyOnChainWallet, aliceListener: TestProbe, bobListener: TestProbe)
|
||||
case class FixtureParam(alice: TestFSMRef[ChannelState, ChannelData, Channel], bob: TestFSMRef[ChannelState, ChannelData, Channel], alicePeer: TestProbe, bobPeer: TestProbe, alice2bob: TestProbe, bob2alice: TestProbe, alice2blockchain: TestProbe, bob2blockchain: TestProbe, wallet: SingleKeyOnChainWallet, aliceListener: TestProbe, bobListener: TestProbe)
|
||||
|
||||
override def withFixture(test: OneArgTest): Outcome = {
|
||||
val wallet = new SingleKeyOnChainWallet()
|
||||
@ -50,7 +50,8 @@ class WaitForDualFundingSignedStateSpec extends TestKitBaseClass with FixtureAny
|
||||
val (aliceParams, bobParams, channelType) = computeFeatures(setup, test.tags, channelFlags)
|
||||
val aliceInit = Init(aliceParams.initFeatures)
|
||||
val bobInit = Init(bobParams.initFeatures)
|
||||
val bobContribution = if (channelType.features.contains(Features.ZeroConf)) None else Some(LiquidityAds.AddFunding(TestConstants.nonInitiatorFundingSatoshis, None))
|
||||
val bobContribution = if (channelType.features.contains(Features.ZeroConf)) None else Some(LiquidityAds.AddFunding(TestConstants.nonInitiatorFundingSatoshis, Some(TestConstants.defaultLiquidityRates)))
|
||||
val requestFunding_opt = if (test.tags.contains(ChannelStateTestsTags.LiquidityAds)) Some(LiquidityAds.RequestFunding(TestConstants.nonInitiatorFundingSatoshis, TestConstants.defaultLiquidityRates.fundingRates.head, LiquidityAds.PaymentDetails.FromChannelBalance)) else None
|
||||
val (initiatorPushAmount, nonInitiatorPushAmount) = if (test.tags.contains("both_push_amount")) (Some(TestConstants.initiatorPushAmount), Some(TestConstants.nonInitiatorPushAmount)) else (None, None)
|
||||
val commitFeerate = channelType.commitmentFormat match {
|
||||
case Transactions.DefaultCommitmentFormat => TestConstants.feeratePerKw
|
||||
@ -61,8 +62,8 @@ class WaitForDualFundingSignedStateSpec extends TestKitBaseClass with FixtureAny
|
||||
within(30 seconds) {
|
||||
alice.underlying.system.eventStream.subscribe(aliceListener.ref, classOf[ChannelAborted])
|
||||
bob.underlying.system.eventStream.subscribe(bobListener.ref, classOf[ChannelAborted])
|
||||
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = true, commitFeerate, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, initiatorPushAmount, requireConfirmedInputs = false, requestFunding_opt = None, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped)
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, bobContribution, dualFunded = true, nonInitiatorPushAmount, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
|
||||
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = true, commitFeerate, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, initiatorPushAmount, requireConfirmedInputs = false, requestFunding_opt, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped)
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, bobContribution, dualFunded = true, nonInitiatorPushAmount, requireConfirmedInputs = false, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
|
||||
alice2blockchain.expectMsgType[TxPublisher.SetChannelId] // temporary channel id
|
||||
bob2blockchain.expectMsgType[TxPublisher.SetChannelId] // temporary channel id
|
||||
alice2bob.expectMsgType[OpenDualFundedChannel]
|
||||
@ -95,7 +96,7 @@ class WaitForDualFundingSignedStateSpec extends TestKitBaseClass with FixtureAny
|
||||
aliceOpenReplyTo.expectMsgType[OpenChannelResponse.Created]
|
||||
awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_SIGNED)
|
||||
awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_SIGNED)
|
||||
withFixture(test.toNoArgTest(FixtureParam(alice, bob, alice2bob, bob2alice, alice2blockchain, bob2blockchain, wallet, aliceListener, bobListener)))
|
||||
withFixture(test.toNoArgTest(FixtureParam(alice, bob, alicePeer, bobPeer, alice2bob, bob2alice, alice2blockchain, bob2blockchain, wallet, aliceListener, bobListener)))
|
||||
}
|
||||
}
|
||||
|
||||
@ -196,6 +197,44 @@ class WaitForDualFundingSignedStateSpec extends TestKitBaseClass with FixtureAny
|
||||
assert(aliceData.commitments.latest.localCommit.spec.toRemote == expectedBalanceBob)
|
||||
}
|
||||
|
||||
test("complete interactive-tx protocol (with liquidity ads)", Tag(ChannelStateTestsTags.DualFunding), Tag(ChannelStateTestsTags.LiquidityAds)) { f =>
|
||||
import f._
|
||||
|
||||
val listener = TestProbe()
|
||||
alice.underlyingActor.context.system.eventStream.subscribe(listener.ref, classOf[TransactionPublished])
|
||||
|
||||
bob2alice.expectMsgType[CommitSig]
|
||||
bob2alice.forward(alice)
|
||||
alice2bob.expectMsgType[CommitSig]
|
||||
alice2bob.forward(bob)
|
||||
|
||||
// Bob sends its signatures first as he contributed less than Alice.
|
||||
bob2alice.expectMsgType[TxSignatures]
|
||||
awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)
|
||||
val bobData = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED]
|
||||
val fundingTxId = bobData.latestFundingTx.sharedTx.asInstanceOf[PartiallySignedSharedTransaction].txId
|
||||
assert(bob2blockchain.expectMsgType[WatchFundingConfirmed].txId == fundingTxId)
|
||||
|
||||
// Bob signed a liquidity purchase.
|
||||
bobPeer.fishForMessage() {
|
||||
case l: LiquidityPurchaseSigned =>
|
||||
assert(l.purchase.paymentDetails == LiquidityAds.PaymentDetails.FromChannelBalance)
|
||||
assert(l.fundingTxIndex == 0)
|
||||
assert(l.txId == fundingTxId)
|
||||
true
|
||||
case _ => false
|
||||
}
|
||||
|
||||
// Alice receives Bob's signatures and sends her own signatures.
|
||||
bob2alice.forward(alice)
|
||||
assert(listener.expectMsgType[TransactionPublished].tx.txid == fundingTxId)
|
||||
assert(alice2blockchain.expectMsgType[WatchFundingConfirmed].txId == fundingTxId)
|
||||
alice2bob.expectMsgType[TxSignatures]
|
||||
awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)
|
||||
val aliceData = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED]
|
||||
assert(aliceData.latestFundingTx.sharedTx.asInstanceOf[FullySignedSharedTransaction].signedTx.txid == fundingTxId)
|
||||
}
|
||||
|
||||
test("recv invalid CommitSig", Tag(ChannelStateTestsTags.DualFunding)) { f =>
|
||||
import f._
|
||||
|
||||
|
@ -67,7 +67,7 @@ class WaitForFundingCreatedStateSpec extends TestKitBaseClass with FixtureAnyFun
|
||||
bob.underlying.system.eventStream.subscribe(listener.ref, classOf[ChannelAborted])
|
||||
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, fundingSatoshis, dualFunded = false, TestConstants.feeratePerKw, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, Some(pushMsat), requireConfirmedInputs = false, requestFunding_opt = None, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped)
|
||||
alice2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, None, dualFunded = false, None, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, None, dualFunded = false, None, requireConfirmedInputs = false, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
|
||||
bob2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
alice2bob.expectMsgType[OpenChannel]
|
||||
alice2bob.forward(bob)
|
||||
|
@ -53,7 +53,7 @@ class WaitForFundingInternalStateSpec extends TestKitBaseClass with FixtureAnyFu
|
||||
within(30 seconds) {
|
||||
alice.underlying.system.eventStream.subscribe(listener.ref, classOf[ChannelAborted])
|
||||
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = false, TestConstants.feeratePerKw, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, Some(TestConstants.initiatorPushAmount), requireConfirmedInputs = false, requestFunding_opt = None, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped)
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, None, dualFunded = false, None, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, None, dualFunded = false, None, requireConfirmedInputs = false, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
|
||||
alice2bob.expectMsgType[OpenChannel]
|
||||
alice2bob.forward(bob)
|
||||
bob2alice.expectMsgType[AcceptChannel]
|
||||
|
@ -70,7 +70,7 @@ class WaitForFundingSignedStateSpec extends TestKitBaseClass with FixtureAnyFunS
|
||||
alice.underlying.system.eventStream.subscribe(listener.ref, classOf[ChannelAborted])
|
||||
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, fundingSatoshis, dualFunded = false, commitFeerate, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, Some(pushMsat), requireConfirmedInputs = false, requestFunding_opt = None, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped)
|
||||
alice2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, None, dualFunded = false, None, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, None, dualFunded = false, None, requireConfirmedInputs = false, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
|
||||
bob2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
alice2bob.expectMsgType[OpenChannel]
|
||||
alice2bob.forward(bob)
|
||||
|
@ -67,7 +67,7 @@ class WaitForChannelReadyStateSpec extends TestKitBaseClass with FixtureAnyFunSu
|
||||
alice.underlyingActor.nodeParams.db.peers.addOrUpdateRelayFees(bobParams.nodeId, relayFees)
|
||||
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = false, commitFeerate, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, pushMsat, requireConfirmedInputs = false, requestFunding_opt = None, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped)
|
||||
alice2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, None, dualFunded = false, None, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, None, dualFunded = false, None, requireConfirmedInputs = false, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
|
||||
bob2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
alice2bob.expectMsgType[OpenChannel]
|
||||
alice2bob.forward(bob)
|
||||
|
@ -83,7 +83,7 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture
|
||||
val (initiatorPushAmount, nonInitiatorPushAmount) = if (test.tags.contains(bothPushAmount)) (Some(TestConstants.initiatorPushAmount), Some(TestConstants.nonInitiatorPushAmount)) else (None, None)
|
||||
within(30 seconds) {
|
||||
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = true, commitFeerate, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, initiatorPushAmount, requireConfirmedInputs = false, requestFunds_opt, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped)
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, bobContribution, dualFunded = true, nonInitiatorPushAmount, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, bobContribution, dualFunded = true, nonInitiatorPushAmount, requireConfirmedInputs = false, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
|
||||
alice2blockchain.expectMsgType[SetChannelId] // temporary channel id
|
||||
bob2blockchain.expectMsgType[SetChannelId] // temporary channel id
|
||||
alice2bob.expectMsgType[OpenDualFundedChannel]
|
||||
|
@ -38,7 +38,7 @@ import scala.concurrent.duration.DurationInt
|
||||
|
||||
class WaitForDualFundingReadyStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with ChannelStateTestsBase {
|
||||
|
||||
case class FixtureParam(alice: TestFSMRef[ChannelState, ChannelData, Channel], bob: TestFSMRef[ChannelState, ChannelData, Channel], alice2bob: TestProbe, bob2alice: TestProbe, alice2blockchain: TestProbe, bob2blockchain: TestProbe, listener: TestProbe)
|
||||
case class FixtureParam(alice: TestFSMRef[ChannelState, ChannelData, Channel], bob: TestFSMRef[ChannelState, ChannelData, Channel], alicePeer: TestProbe, bobPeer: TestProbe, alice2bob: TestProbe, bob2alice: TestProbe, alice2blockchain: TestProbe, bob2blockchain: TestProbe, listener: TestProbe)
|
||||
|
||||
override def withFixture(test: OneArgTest): Outcome = {
|
||||
val setup = init(tags = test.tags)
|
||||
@ -53,7 +53,7 @@ class WaitForDualFundingReadyStateSpec extends TestKitBaseClass with FixtureAnyF
|
||||
within(30 seconds) {
|
||||
alice.underlying.system.eventStream.subscribe(listener.ref, classOf[ChannelAborted])
|
||||
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = true, TestConstants.anchorOutputsFeeratePerKw, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, None, requireConfirmedInputs = false, requestFunding_opt = None, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped)
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, Some(LiquidityAds.AddFunding(TestConstants.nonInitiatorFundingSatoshis, None)), dualFunded = true, None, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, Some(LiquidityAds.AddFunding(TestConstants.nonInitiatorFundingSatoshis, None)), dualFunded = true, None, requireConfirmedInputs = false, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
|
||||
alice2blockchain.expectMsgType[SetChannelId] // temporary channel id
|
||||
bob2blockchain.expectMsgType[SetChannelId] // temporary channel id
|
||||
alice2bob.expectMsgType[OpenDualFundedChannel]
|
||||
@ -105,7 +105,7 @@ class WaitForDualFundingReadyStateSpec extends TestKitBaseClass with FixtureAnyF
|
||||
}
|
||||
awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_READY)
|
||||
awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_READY)
|
||||
withFixture(test.toNoArgTest(FixtureParam(alice, bob, alice2bob, bob2alice, alice2blockchain, bob2blockchain, listener)))
|
||||
withFixture(test.toNoArgTest(FixtureParam(alice, bob, alicePeer, bobPeer, alice2bob, bob2alice, alice2blockchain, bob2blockchain, listener)))
|
||||
}
|
||||
}
|
||||
|
||||
@ -129,6 +129,22 @@ class WaitForDualFundingReadyStateSpec extends TestKitBaseClass with FixtureAnyF
|
||||
listenerA.expectMsg(ChannelOpened(alice, bob.underlyingActor.nodeParams.nodeId, channelId(alice)))
|
||||
awaitCond(alice.stateName == NORMAL)
|
||||
|
||||
// The channel is now ready to process payments.
|
||||
alicePeer.fishForMessage() {
|
||||
case e: ChannelReadyForPayments =>
|
||||
assert(e.fundingTxIndex == 0)
|
||||
assert(e.channelId == aliceChannelReady.channelId)
|
||||
true
|
||||
case _ => false
|
||||
}
|
||||
bobPeer.fishForMessage() {
|
||||
case e: ChannelReadyForPayments =>
|
||||
assert(e.fundingTxIndex == 0)
|
||||
assert(e.channelId == aliceChannelReady.channelId)
|
||||
true
|
||||
case _ => false
|
||||
}
|
||||
|
||||
assert(alice.stateData.asInstanceOf[DATA_NORMAL].shortIds.real.isInstanceOf[RealScidStatus.Temporary])
|
||||
val aliceCommitments = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest
|
||||
val aliceUpdate = alice.stateData.asInstanceOf[DATA_NORMAL].channelUpdate
|
||||
|
@ -63,7 +63,7 @@ class WaitForFundingConfirmedStateSpec extends TestKitBaseClass with FixtureAnyF
|
||||
val commitTxFeerate = if (test.tags.contains(ChannelStateTestsTags.AnchorOutputs) || test.tags.contains(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) TestConstants.anchorOutputsFeeratePerKw else TestConstants.feeratePerKw
|
||||
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = false, commitTxFeerate, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, Some(pushMsat), requireConfirmedInputs = false, requestFunding_opt = None, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped)
|
||||
alice2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, None, dualFunded = false, None, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, None, dualFunded = false, None, requireConfirmedInputs = false, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
|
||||
bob2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
alice2bob.expectMsgType[OpenChannel]
|
||||
alice2bob.forward(bob)
|
||||
|
@ -148,7 +148,7 @@ class NormalQuiescentStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteL
|
||||
// initiator should reject commands that change the commitment once it became quiescent
|
||||
val sender1, sender2, sender3 = TestProbe()
|
||||
val cmds = Seq(
|
||||
CMD_ADD_HTLC(sender1.ref, 1_000_000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender1.ref)),
|
||||
CMD_ADD_HTLC(sender1.ref, 1_000_000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender1.ref)),
|
||||
CMD_UPDATE_FEE(FeeratePerKw(100 sat), replyTo_opt = Some(sender2.ref)),
|
||||
CMD_CLOSE(sender3.ref, None, None)
|
||||
)
|
||||
@ -164,7 +164,7 @@ class NormalQuiescentStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteL
|
||||
// both should reject commands that change the commitment while quiescent
|
||||
val sender1, sender2, sender3 = TestProbe()
|
||||
val cmds = Seq(
|
||||
CMD_ADD_HTLC(sender1.ref, 1_000_000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender1.ref)),
|
||||
CMD_ADD_HTLC(sender1.ref, 1_000_000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender1.ref)),
|
||||
CMD_UPDATE_FEE(FeeratePerKw(100 sat), replyTo_opt = Some(sender2.ref)),
|
||||
CMD_CLOSE(sender3.ref, None, None)
|
||||
)
|
||||
@ -352,7 +352,7 @@ class NormalQuiescentStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteL
|
||||
import f._
|
||||
initiateQuiescence(f, sendInitialStfu = true)
|
||||
// have to build a htlc manually because eclair would refuse to accept this command as it's forbidden
|
||||
val forbiddenMsg = UpdateAddHtlc(channelId = randomBytes32(), id = 5656, amountMsat = 50000000 msat, cltvExpiry = CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), paymentHash = randomBytes32(), onionRoutingPacket = TestConstants.emptyOnionPacket, blinding_opt = None, confidence = 1.0)
|
||||
val forbiddenMsg = UpdateAddHtlc(channelId = randomBytes32(), id = 5656, amountMsat = 50000000 msat, cltvExpiry = CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), paymentHash = randomBytes32(), onionRoutingPacket = TestConstants.emptyOnionPacket, blinding_opt = None, confidence = 1.0, fundingFee_opt = None)
|
||||
// both parties will respond to a forbidden msg while quiescent with a warning (and disconnect)
|
||||
bob2alice.forward(alice, forbiddenMsg)
|
||||
alice2bob.expectMsg(Warning(channelId(alice), ForbiddenDuringSplice(channelId(alice), "UpdateAddHtlc").getMessage))
|
||||
|
@ -36,6 +36,7 @@ import fr.acinq.eclair.channel.publish.TxPublisher.{PublishFinalTx, PublishRepla
|
||||
import fr.acinq.eclair.channel.states.ChannelStateTestsBase.{FakeTxPublisherFactory, PimpTestFSM}
|
||||
import fr.acinq.eclair.channel.states.{ChannelStateTestsBase, ChannelStateTestsTags}
|
||||
import fr.acinq.eclair.db.RevokedHtlcInfoCleaner.ForgetHtlcInfos
|
||||
import fr.acinq.eclair.io.Peer.LiquidityPurchaseSigned
|
||||
import fr.acinq.eclair.payment.relay.Relayer
|
||||
import fr.acinq.eclair.testutils.PimpTestProbe.convert
|
||||
import fr.acinq.eclair.transactions.DirectedHtlc.{incoming, outgoing}
|
||||
@ -341,9 +342,19 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
|
||||
assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.capacity == 2_400_000.sat)
|
||||
assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localCommit.spec.toLocal < 1_300_000_000.msat)
|
||||
assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localCommit.spec.toRemote > 1_100_000_000.msat)
|
||||
|
||||
// Bob signed a liquidity purchase.
|
||||
bobPeer.fishForMessage() {
|
||||
case l: LiquidityPurchaseSigned =>
|
||||
assert(l.purchase.paymentDetails == LiquidityAds.PaymentDetails.FromChannelBalance)
|
||||
assert(l.fundingTxIndex == 1)
|
||||
assert(l.txId == alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.fundingTxId)
|
||||
true
|
||||
case _ => false
|
||||
}
|
||||
}
|
||||
|
||||
test("recv CMD_SPLICE (splice-in, liquidity ads, invalid lease witness)", Tag(ChannelStateTestsTags.Quiescence)) { f =>
|
||||
test("recv CMD_SPLICE (splice-in, liquidity ads, invalid will_fund signature)", Tag(ChannelStateTestsTags.Quiescence)) { f =>
|
||||
import f._
|
||||
|
||||
val sender = TestProbe()
|
||||
@ -707,7 +718,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
|
||||
|
||||
val sender = TestProbe()
|
||||
// command for a large payment (larger than local balance pre-slice)
|
||||
val cmd = CMD_ADD_HTLC(sender.ref, 1_000_000_000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
val cmd = CMD_ADD_HTLC(sender.ref, 1_000_000_000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
// first attempt at payment fails (not enough balance)
|
||||
alice ! cmd
|
||||
sender.expectMsgType[RES_ADD_FAILED[_]]
|
||||
@ -938,6 +949,16 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
|
||||
aliceEvents.expectNoMessage(100 millis)
|
||||
bobEvents.expectNoMessage(100 millis)
|
||||
|
||||
// The channel is now ready to use liquidity from the first splice.
|
||||
alicePeer.fishForMessage() {
|
||||
case e: ChannelReadyForPayments => e.fundingTxIndex == 1
|
||||
case _ => false
|
||||
}
|
||||
bobPeer.fishForMessage() {
|
||||
case e: ChannelReadyForPayments => e.fundingTxIndex == 1
|
||||
case _ => false
|
||||
}
|
||||
|
||||
bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx2)
|
||||
bob2alice.expectMsgType[SpliceLocked]
|
||||
bob2alice.forward(alice)
|
||||
@ -953,13 +974,23 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
|
||||
bobEvents.expectAvailableBalanceChanged(balance = 650_000_000.msat, capacity = 2_500_000.sat)
|
||||
aliceEvents.expectNoMessage(100 millis)
|
||||
bobEvents.expectNoMessage(100 millis)
|
||||
|
||||
// The channel is now ready to use liquidity from the second splice.
|
||||
alicePeer.fishForMessage() {
|
||||
case e: ChannelReadyForPayments => e.fundingTxIndex == 2
|
||||
case _ => false
|
||||
}
|
||||
bobPeer.fishForMessage() {
|
||||
case e: ChannelReadyForPayments => e.fundingTxIndex == 2
|
||||
case _ => false
|
||||
}
|
||||
}
|
||||
|
||||
test("recv CMD_ADD_HTLC with multiple commitments") { f =>
|
||||
import f._
|
||||
initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat)))
|
||||
val sender = TestProbe()
|
||||
alice ! CMD_ADD_HTLC(sender.ref, 500_000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
alice ! CMD_ADD_HTLC(sender.ref, 500_000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
sender.expectMsgType[RES_SUCCESS[CMD_ADD_HTLC]]
|
||||
alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
alice2bob.forward(bob)
|
||||
@ -988,7 +1019,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
|
||||
import f._
|
||||
initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat)))
|
||||
val sender = TestProbe()
|
||||
alice ! CMD_ADD_HTLC(sender.ref, 500_000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
alice ! CMD_ADD_HTLC(sender.ref, 500_000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
sender.expectMsgType[RES_SUCCESS[CMD_ADD_HTLC]]
|
||||
alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
alice2bob.forward(bob)
|
||||
@ -1022,7 +1053,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
|
||||
val cmd = CMD_SPLICE(sender.ref, spliceIn_opt = Some(SpliceIn(500_000 sat, pushAmount = 0 msat)), spliceOut_opt = None, requestFunding_opt = None)
|
||||
alice ! cmd
|
||||
alice2bob.expectMsgType[SpliceInit]
|
||||
alice ! CMD_ADD_HTLC(sender.ref, 500000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
alice ! CMD_ADD_HTLC(sender.ref, 500000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
sender.expectMsgType[RES_ADD_FAILED[_]]
|
||||
alice2bob.expectNoMessage(100 millis)
|
||||
}
|
||||
@ -1037,7 +1068,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
|
||||
bob2alice.expectMsgType[SpliceAck]
|
||||
bob2alice.forward(alice)
|
||||
alice2bob.expectMsgType[TxAddInput]
|
||||
alice ! CMD_ADD_HTLC(sender.ref, 500000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
alice ! CMD_ADD_HTLC(sender.ref, 500000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
sender.expectMsgType[RES_ADD_FAILED[_]]
|
||||
alice2bob.expectNoMessage(100 millis)
|
||||
}
|
||||
@ -1079,7 +1110,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
|
||||
alice2bob.expectMsgType[TxAddInput]
|
||||
|
||||
// have to build a htlc manually because eclair would refuse to accept this command as it's forbidden
|
||||
val fakeHtlc = UpdateAddHtlc(channelId = randomBytes32(), id = 5656, amountMsat = 50000000 msat, cltvExpiry = CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), paymentHash = randomBytes32(), onionRoutingPacket = TestConstants.emptyOnionPacket, blinding_opt = None, confidence = 1.0)
|
||||
val fakeHtlc = UpdateAddHtlc(channelId = randomBytes32(), id = 5656, amountMsat = 50000000 msat, cltvExpiry = CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), paymentHash = randomBytes32(), onionRoutingPacket = TestConstants.emptyOnionPacket, blinding_opt = None, confidence = 1.0, fundingFee_opt = None)
|
||||
bob2alice.forward(alice, fakeHtlc)
|
||||
// alice returns a warning and schedules a disconnect after receiving UpdateAddHtlc
|
||||
alice2bob.expectMsg(Warning(channelId(alice), ForbiddenDuringSplice(channelId(alice), "UpdateAddHtlc").getMessage))
|
||||
@ -1514,6 +1545,16 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
|
||||
alice2bob.expectNoMessage(100 millis)
|
||||
bob2alice.expectNoMessage(100 millis)
|
||||
|
||||
// splice transactions are not locked yet: we're still at the initial funding index
|
||||
alicePeer.fishForMessage() {
|
||||
case e: ChannelReadyForPayments => e.fundingTxIndex == 0
|
||||
case _ => false
|
||||
}
|
||||
bobPeer.fishForMessage() {
|
||||
case e: ChannelReadyForPayments => e.fundingTxIndex == 0
|
||||
case _ => false
|
||||
}
|
||||
|
||||
// splice 1 confirms on alice's side
|
||||
watchConfirmed1a.replyTo ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1)
|
||||
assert(alice2bob.expectMsgType[SpliceLocked].fundingTxId == fundingTx1.txid)
|
||||
@ -1540,12 +1581,28 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
|
||||
alice2bob.expectNoMessage(100 millis)
|
||||
bob2alice.expectNoMessage(100 millis)
|
||||
|
||||
// splice transactions are not locked by bob yet: we're still at the initial funding index
|
||||
alicePeer.fishForMessage() {
|
||||
case e: ChannelReadyForPayments => e.fundingTxIndex == 0
|
||||
case _ => false
|
||||
}
|
||||
|
||||
// splice 1 confirms on bob's side
|
||||
watchConfirmed1b.replyTo ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1)
|
||||
assert(bob2alice.expectMsgType[SpliceLocked].fundingTxId == fundingTx1.txid)
|
||||
bob2alice.forward(alice)
|
||||
bob2blockchain.expectMsgType[WatchFundingSpent]
|
||||
|
||||
// splice 1 is locked on both sides
|
||||
alicePeer.fishForMessage() {
|
||||
case e: ChannelReadyForPayments => e.fundingTxIndex == 1
|
||||
case _ => false
|
||||
}
|
||||
bobPeer.fishForMessage() {
|
||||
case e: ChannelReadyForPayments => e.fundingTxIndex == 1
|
||||
case _ => false
|
||||
}
|
||||
|
||||
disconnect(f)
|
||||
reconnect(f)
|
||||
|
||||
@ -1556,11 +1613,26 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
|
||||
alice2bob.expectNoMessage(100 millis)
|
||||
bob2alice.expectNoMessage(100 millis)
|
||||
|
||||
alicePeer.fishForMessage() {
|
||||
case e: ChannelReadyForPayments => e.fundingTxIndex == 1
|
||||
case _ => false
|
||||
}
|
||||
bobPeer.fishForMessage() {
|
||||
case e: ChannelReadyForPayments => e.fundingTxIndex == 1
|
||||
case _ => false
|
||||
}
|
||||
|
||||
// splice 2 confirms on bob's side
|
||||
watchConfirmed2b.replyTo ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx2)
|
||||
assert(bob2alice.expectMsgType[SpliceLocked].fundingTxId == fundingTx2.txid)
|
||||
bob2blockchain.expectMsgType[WatchFundingSpent]
|
||||
|
||||
// splice 2 is locked on both sides
|
||||
bobPeer.fishForMessage() {
|
||||
case e: ChannelReadyForPayments => e.fundingTxIndex == 2
|
||||
case _ => false
|
||||
}
|
||||
|
||||
// NB: we disconnect *before* transmitting the splice_confirmed to alice
|
||||
disconnect(f)
|
||||
reconnect(f)
|
||||
@ -1582,6 +1654,16 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
|
||||
bob2alice.forward(alice)
|
||||
alice2bob.expectNoMessage(100 millis)
|
||||
bob2alice.expectNoMessage(100 millis)
|
||||
|
||||
// splice 2 is locked on both sides
|
||||
alicePeer.fishForMessage() {
|
||||
case e: ChannelReadyForPayments => e.fundingTxIndex == 2
|
||||
case _ => false
|
||||
}
|
||||
bobPeer.fishForMessage() {
|
||||
case e: ChannelReadyForPayments => e.fundingTxIndex == 2
|
||||
case _ => false
|
||||
}
|
||||
}
|
||||
|
||||
/** Check type of published transactions */
|
||||
|
@ -77,7 +77,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
val listener = TestProbe()
|
||||
alice.underlying.system.eventStream.subscribe(listener.ref, classOf[AvailableBalanceChanged])
|
||||
val h = randomBytes32()
|
||||
val add = CMD_ADD_HTLC(sender.ref, 50000000 msat, h, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
val add = CMD_ADD_HTLC(sender.ref, 50000000 msat, h, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
alice ! add
|
||||
sender.expectMsgType[RES_SUCCESS[CMD_ADD_HTLC]]
|
||||
val e = listener.expectMsgType[AvailableBalanceChanged]
|
||||
@ -103,7 +103,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
val sender = TestProbe()
|
||||
val h = randomBytes32()
|
||||
for (i <- 0 until 10) {
|
||||
alice ! CMD_ADD_HTLC(sender.ref, 500000 msat, h, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
alice ! CMD_ADD_HTLC(sender.ref, 500000 msat, h, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
sender.expectMsgType[RES_SUCCESS[CMD_ADD_HTLC]]
|
||||
val htlc = alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
assert(htlc.id == i && htlc.paymentHash == h)
|
||||
@ -115,9 +115,9 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
|
||||
val sender = TestProbe()
|
||||
val h = randomBytes32()
|
||||
val originHtlc = UpdateAddHtlc(channelId = randomBytes32(), id = 5656, amountMsat = 50000000 msat, cltvExpiry = CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), paymentHash = h, onionRoutingPacket = TestConstants.emptyOnionPacket, blinding_opt = None, confidence = 1.0)
|
||||
val originHtlc = UpdateAddHtlc(channelId = randomBytes32(), id = 5656, amountMsat = 50000000 msat, cltvExpiry = CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), paymentHash = h, onionRoutingPacket = TestConstants.emptyOnionPacket, blinding_opt = None, confidence = 1.0, fundingFee_opt = None)
|
||||
val origin = Origin.Hot(sender.ref, Upstream.Hot.Channel(originHtlc, TimestampMilli.now(), randomKey().publicKey))
|
||||
val cmd = CMD_ADD_HTLC(sender.ref, originHtlc.amountMsat - 10_000.msat, h, originHtlc.cltvExpiry - CltvExpiryDelta(7), TestConstants.emptyOnionPacket, None, 1.0, origin)
|
||||
val cmd = CMD_ADD_HTLC(sender.ref, originHtlc.amountMsat - 10_000.msat, h, originHtlc.cltvExpiry - CltvExpiryDelta(7), TestConstants.emptyOnionPacket, None, 1.0, None, origin)
|
||||
alice ! cmd
|
||||
sender.expectMsgType[RES_SUCCESS[CMD_ADD_HTLC]]
|
||||
val htlc = alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
@ -133,10 +133,10 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
|
||||
val sender = TestProbe()
|
||||
val h = randomBytes32()
|
||||
val originHtlc1 = UpdateAddHtlc(randomBytes32(), 47, 30000000 msat, h, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val originHtlc2 = UpdateAddHtlc(randomBytes32(), 32, 20000000 msat, h, CltvExpiryDelta(160).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val origin = Origin.Hot(sender.ref, Upstream.Hot.Trampoline(Seq(originHtlc1, originHtlc2).map(htlc => Upstream.Hot.Channel(htlc, TimestampMilli.now(), randomKey().publicKey))))
|
||||
val cmd = CMD_ADD_HTLC(sender.ref, originHtlc1.amountMsat + originHtlc2.amountMsat - 10000.msat, h, originHtlc2.cltvExpiry - CltvExpiryDelta(7), TestConstants.emptyOnionPacket, None, 1.0, origin)
|
||||
val originHtlc1 = UpdateAddHtlc(randomBytes32(), 47, 30000000 msat, h, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
val originHtlc2 = UpdateAddHtlc(randomBytes32(), 32, 20000000 msat, h, CltvExpiryDelta(160).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
val origin = Origin.Hot(sender.ref, Upstream.Hot.Trampoline(List(originHtlc1, originHtlc2).map(htlc => Upstream.Hot.Channel(htlc, TimestampMilli.now(), randomKey().publicKey))))
|
||||
val cmd = CMD_ADD_HTLC(sender.ref, originHtlc1.amountMsat + originHtlc2.amountMsat - 10000.msat, h, originHtlc2.cltvExpiry - CltvExpiryDelta(7), TestConstants.emptyOnionPacket, None, 1.0, None, origin)
|
||||
alice ! cmd
|
||||
sender.expectMsgType[RES_SUCCESS[CMD_ADD_HTLC]]
|
||||
val htlc = alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
@ -152,7 +152,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
val sender = TestProbe()
|
||||
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
|
||||
val expiryTooSmall = CltvExpiry(currentBlockHeight)
|
||||
val add = CMD_ADD_HTLC(sender.ref, 500000000 msat, randomBytes32(), expiryTooSmall, TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
val add = CMD_ADD_HTLC(sender.ref, 500000000 msat, randomBytes32(), expiryTooSmall, TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
alice ! add
|
||||
val error = ExpiryTooSmall(channelId(alice), CltvExpiry(currentBlockHeight + 3), expiryTooSmall, currentBlockHeight)
|
||||
sender.expectMsg(RES_ADD_FAILED(add, error, Some(initialState.channelUpdate)))
|
||||
@ -165,7 +165,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
|
||||
val maxAllowedExpiryDelta = alice.underlyingActor.nodeParams.channelConf.maxExpiryDelta
|
||||
val expiryTooBig = (maxAllowedExpiryDelta + 1).toCltvExpiry(currentBlockHeight)
|
||||
val add = CMD_ADD_HTLC(sender.ref, 500000000 msat, randomBytes32(), expiryTooBig, TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
val add = CMD_ADD_HTLC(sender.ref, 500000000 msat, randomBytes32(), expiryTooBig, TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
alice ! add
|
||||
val error = ExpiryTooBig(channelId(alice), maximum = maxAllowedExpiryDelta.toCltvExpiry(currentBlockHeight), actual = expiryTooBig, blockHeight = currentBlockHeight)
|
||||
sender.expectMsg(RES_ADD_FAILED(add, error, Some(initialState.channelUpdate)))
|
||||
@ -176,7 +176,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
import f._
|
||||
val sender = TestProbe()
|
||||
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
|
||||
val add = CMD_ADD_HTLC(sender.ref, 50 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
val add = CMD_ADD_HTLC(sender.ref, 50 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
alice ! add
|
||||
val error = HtlcValueTooSmall(channelId(alice), 1000 msat, 50 msat)
|
||||
sender.expectMsg(RES_ADD_FAILED(add, error, Some(initialState.channelUpdate)))
|
||||
@ -189,7 +189,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
// Alice has a minimum set to 0 msat (which should be invalid, but may mislead Bob into relaying 0-value HTLCs which is forbidden by the spec).
|
||||
assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.params.localParams.htlcMinimum == 0.msat)
|
||||
val initialState = bob.stateData.asInstanceOf[DATA_NORMAL]
|
||||
val add = CMD_ADD_HTLC(sender.ref, 0 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
val add = CMD_ADD_HTLC(sender.ref, 0 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
bob ! add
|
||||
val error = HtlcValueTooSmall(channelId(bob), 1 msat, 0 msat)
|
||||
sender.expectMsg(RES_ADD_FAILED(add, error, Some(initialState.channelUpdate)))
|
||||
@ -201,7 +201,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
val sender = TestProbe()
|
||||
// channel starts with all funds on alice's side, alice sends some funds to bob, but not enough to make it go above reserve
|
||||
val h = randomBytes32()
|
||||
val add = CMD_ADD_HTLC(sender.ref, 50000000 msat, h, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
val add = CMD_ADD_HTLC(sender.ref, 50000000 msat, h, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
alice ! add
|
||||
sender.expectMsgType[RES_SUCCESS[CMD_ADD_HTLC]]
|
||||
}
|
||||
@ -210,7 +210,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
import f._
|
||||
val sender = TestProbe()
|
||||
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
|
||||
val add = CMD_ADD_HTLC(sender.ref, MilliSatoshi(Int.MaxValue), randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
val add = CMD_ADD_HTLC(sender.ref, MilliSatoshi(Int.MaxValue), randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
alice ! add
|
||||
val error = InsufficientFunds(channelId(alice), amount = MilliSatoshi(Int.MaxValue), missing = 1388843 sat, reserve = 20000 sat, fees = 8960 sat)
|
||||
sender.expectMsg(RES_ADD_FAILED(add, error, Some(initialState.channelUpdate)))
|
||||
@ -223,7 +223,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
|
||||
// The anchor outputs commitment format costs more fees for the funder (bigger commit tx + cost of anchor outputs)
|
||||
assert(initialState.commitments.availableBalanceForSend < initialState.commitments.modify(_.params.channelFeatures).setTo(ChannelFeatures()).availableBalanceForSend)
|
||||
val add = CMD_ADD_HTLC(sender.ref, initialState.commitments.availableBalanceForSend + 1.msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
val add = CMD_ADD_HTLC(sender.ref, initialState.commitments.availableBalanceForSend + 1.msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
alice ! add
|
||||
|
||||
val error = InsufficientFunds(channelId(alice), amount = add.amount, missing = 0 sat, reserve = 20000 sat, fees = 3900 sat)
|
||||
@ -235,7 +235,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
import f._
|
||||
val sender = TestProbe()
|
||||
val initialState = bob.stateData.asInstanceOf[DATA_NORMAL]
|
||||
val add = CMD_ADD_HTLC(sender.ref, initialState.commitments.availableBalanceForSend + 1.msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
val add = CMD_ADD_HTLC(sender.ref, initialState.commitments.availableBalanceForSend + 1.msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
bob ! add
|
||||
|
||||
val error = InsufficientFunds(channelId(alice), amount = add.amount, missing = 0 sat, reserve = 10000 sat, fees = 0 sat)
|
||||
@ -253,7 +253,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
// At this point alice has the minimal amount to sustain a channel.
|
||||
// Alice maintains an extra reserve to accommodate for a few more HTLCs, so the first few HTLCs should be allowed.
|
||||
val htlcs = (1 to 7).map { _ =>
|
||||
bob ! CMD_ADD_HTLC(sender.ref, 12_000_000 msat, randomBytes32(), CltvExpiry(400144), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
bob ! CMD_ADD_HTLC(sender.ref, 12_000_000 msat, randomBytes32(), CltvExpiry(400144), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
sender.expectMsgType[RES_SUCCESS[CMD_ADD_HTLC]]
|
||||
val add = bob2alice.expectMsgType[UpdateAddHtlc]
|
||||
bob2alice.forward(alice, add)
|
||||
@ -261,7 +261,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
}
|
||||
|
||||
// But this one will dip alice below her reserve: we must wait for the previous HTLCs to settle before sending any more.
|
||||
val failedAdd = CMD_ADD_HTLC(sender.ref, 11_000_000 msat, randomBytes32(), CltvExpiry(400144), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
val failedAdd = CMD_ADD_HTLC(sender.ref, 11_000_000 msat, randomBytes32(), CltvExpiry(400144), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
bob ! failedAdd
|
||||
val error = RemoteCannotAffordFeesForNewHtlc(channelId(bob), failedAdd.amount, missing = 1360 sat, 20_000 sat, 22_720 sat)
|
||||
sender.expectMsg(RES_ADD_FAILED(failedAdd, error, Some(bob.stateData.asInstanceOf[DATA_NORMAL].channelUpdate)))
|
||||
@ -285,13 +285,13 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
|
||||
// At this point alice has the minimal amount to sustain a channel.
|
||||
// Alice maintains an extra reserve to accommodate for a one more HTLCs, so the first few HTLCs should be allowed.
|
||||
bob ! CMD_ADD_HTLC(sender.ref, 25_000_000 msat, randomBytes32(), CltvExpiry(400144), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
bob ! CMD_ADD_HTLC(sender.ref, 25_000_000 msat, randomBytes32(), CltvExpiry(400144), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
sender.expectMsgType[RES_SUCCESS[CMD_ADD_HTLC]]
|
||||
val add = bob2alice.expectMsgType[UpdateAddHtlc]
|
||||
bob2alice.forward(alice, add)
|
||||
|
||||
// But this one will dip alice below her reserve: we must wait for the previous HTLCs to settle before sending any more.
|
||||
val failedAdd = CMD_ADD_HTLC(sender.ref, 25_000_000 msat, randomBytes32(), CltvExpiry(400144), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
val failedAdd = CMD_ADD_HTLC(sender.ref, 25_000_000 msat, randomBytes32(), CltvExpiry(400144), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
bob ! failedAdd
|
||||
val error = RemoteCannotAffordFeesForNewHtlc(channelId(bob), failedAdd.amount, missing = 340 sat, 20_000 sat, 21_700 sat)
|
||||
sender.expectMsg(RES_ADD_FAILED(failedAdd, error, Some(bob.stateData.asInstanceOf[DATA_NORMAL].channelUpdate)))
|
||||
@ -306,16 +306,16 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
import f._
|
||||
val sender = TestProbe()
|
||||
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
|
||||
alice ! CMD_ADD_HTLC(sender.ref, 500000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
alice ! CMD_ADD_HTLC(sender.ref, 500000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
sender.expectMsgType[RES_SUCCESS[CMD_ADD_HTLC]]
|
||||
alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
alice ! CMD_ADD_HTLC(sender.ref, 200000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
alice ! CMD_ADD_HTLC(sender.ref, 200000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
sender.expectMsgType[RES_SUCCESS[CMD_ADD_HTLC]]
|
||||
alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
alice ! CMD_ADD_HTLC(sender.ref, 51760000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
alice ! CMD_ADD_HTLC(sender.ref, 51760000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
sender.expectMsgType[RES_SUCCESS[CMD_ADD_HTLC]]
|
||||
alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
val add = CMD_ADD_HTLC(sender.ref, 1000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
val add = CMD_ADD_HTLC(sender.ref, 1000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
alice ! add
|
||||
val error = InsufficientFunds(channelId(alice), amount = 1000000 msat, missing = 1000 sat, reserve = 20000 sat, fees = 12400 sat)
|
||||
sender.expectMsg(RES_ADD_FAILED(add, error, Some(initialState.channelUpdate)))
|
||||
@ -326,13 +326,13 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
import f._
|
||||
val sender = TestProbe()
|
||||
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
|
||||
alice ! CMD_ADD_HTLC(sender.ref, 300000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
alice ! CMD_ADD_HTLC(sender.ref, 300000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
sender.expectMsgType[RES_SUCCESS[CMD_ADD_HTLC]]
|
||||
alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
alice ! CMD_ADD_HTLC(sender.ref, 300000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
alice ! CMD_ADD_HTLC(sender.ref, 300000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
sender.expectMsgType[RES_SUCCESS[CMD_ADD_HTLC]]
|
||||
alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
val add = CMD_ADD_HTLC(sender.ref, 500000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
val add = CMD_ADD_HTLC(sender.ref, 500000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
alice ! add
|
||||
val error = InsufficientFunds(channelId(alice), amount = 500000000 msat, missing = 348240 sat, reserve = 20000 sat, fees = 12400 sat)
|
||||
sender.expectMsg(RES_ADD_FAILED(add, error, Some(initialState.channelUpdate)))
|
||||
@ -345,7 +345,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
val initialState = bob.stateData.asInstanceOf[DATA_NORMAL]
|
||||
assert(initialState.commitments.params.localParams.maxHtlcValueInFlightMsat == initialState.commitments.latest.capacity.toMilliSatoshi)
|
||||
assert(initialState.commitments.params.remoteParams.maxHtlcValueInFlightMsat == UInt64(150000000))
|
||||
val add = CMD_ADD_HTLC(sender.ref, 151000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
val add = CMD_ADD_HTLC(sender.ref, 151000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
bob ! add
|
||||
val error = HtlcValueTooHighInFlight(channelId(bob), maximum = 150000000 msat, actual = 151000000 msat)
|
||||
sender.expectMsg(RES_ADD_FAILED(add, error, Some(initialState.channelUpdate)))
|
||||
@ -358,11 +358,11 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
val initialState = bob.stateData.asInstanceOf[DATA_NORMAL]
|
||||
assert(initialState.commitments.params.localParams.maxHtlcValueInFlightMsat == initialState.commitments.latest.capacity.toMilliSatoshi)
|
||||
assert(initialState.commitments.params.remoteParams.maxHtlcValueInFlightMsat == UInt64(150000000))
|
||||
val add = CMD_ADD_HTLC(sender.ref, 75500000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
val add = CMD_ADD_HTLC(sender.ref, 75500000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
bob ! add
|
||||
sender.expectMsgType[RES_SUCCESS[CMD_ADD_HTLC]]
|
||||
bob2alice.expectMsgType[UpdateAddHtlc]
|
||||
val add1 = CMD_ADD_HTLC(sender.ref, 75500000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
val add1 = CMD_ADD_HTLC(sender.ref, 75500000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
bob ! add1
|
||||
val error = HtlcValueTooHighInFlight(channelId(bob), maximum = 150000000 msat, actual = 151000000 msat)
|
||||
sender.expectMsg(RES_ADD_FAILED(add1, error, Some(initialState.channelUpdate)))
|
||||
@ -375,7 +375,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
|
||||
assert(initialState.commitments.params.localParams.maxHtlcValueInFlightMsat == 150000000.msat)
|
||||
assert(initialState.commitments.params.remoteParams.maxHtlcValueInFlightMsat == UInt64(initialState.commitments.latest.capacity.toMilliSatoshi.toLong))
|
||||
val add = CMD_ADD_HTLC(sender.ref, 151000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
val add = CMD_ADD_HTLC(sender.ref, 151000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
alice ! add
|
||||
val error = HtlcValueTooHighInFlight(channelId(alice), maximum = 150000000 msat, actual = 151000000 msat)
|
||||
sender.expectMsg(RES_ADD_FAILED(add, error, Some(initialState.channelUpdate)))
|
||||
@ -389,11 +389,11 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
assert(initialState.commitments.params.localParams.maxAcceptedHtlcs == 100)
|
||||
assert(initialState.commitments.params.remoteParams.maxAcceptedHtlcs == 30) // Bob accepts a maximum of 30 htlcs
|
||||
for (_ <- 0 until 30) {
|
||||
alice ! CMD_ADD_HTLC(sender.ref, 10000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
alice ! CMD_ADD_HTLC(sender.ref, 10000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
sender.expectMsgType[RES_SUCCESS[CMD_ADD_HTLC]]
|
||||
alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
}
|
||||
val add = CMD_ADD_HTLC(sender.ref, 10000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
val add = CMD_ADD_HTLC(sender.ref, 10000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
alice ! add
|
||||
val error = TooManyAcceptedHtlcs(channelId(alice), maximum = 30)
|
||||
sender.expectMsg(RES_ADD_FAILED(add, error, Some(initialState.channelUpdate)))
|
||||
@ -407,11 +407,11 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
assert(initialState.commitments.params.localParams.maxAcceptedHtlcs == 30) // Bob accepts a maximum of 30 htlcs
|
||||
assert(initialState.commitments.params.remoteParams.maxAcceptedHtlcs == 100) // Alice accepts more, but Bob will stop at 30 HTLCs
|
||||
for (_ <- 0 until 30) {
|
||||
bob ! CMD_ADD_HTLC(sender.ref, 500000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
bob ! CMD_ADD_HTLC(sender.ref, 500000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
sender.expectMsgType[RES_SUCCESS[CMD_ADD_HTLC]]
|
||||
bob2alice.expectMsgType[UpdateAddHtlc]
|
||||
}
|
||||
val add = CMD_ADD_HTLC(sender.ref, 500000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
val add = CMD_ADD_HTLC(sender.ref, 500000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
bob ! add
|
||||
val error = TooManyAcceptedHtlcs(channelId(bob), maximum = 30)
|
||||
sender.expectMsg(RES_ADD_FAILED(add, error, Some(initialState.channelUpdate)))
|
||||
@ -444,18 +444,18 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
crossSign(bob, alice, bob2alice, alice2bob)
|
||||
|
||||
// HTLCs that take Alice's dust exposure above her threshold are rejected.
|
||||
val dustAdd = CMD_ADD_HTLC(sender.ref, 501.sat.toMilliSatoshi, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
val dustAdd = CMD_ADD_HTLC(sender.ref, 501.sat.toMilliSatoshi, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
alice ! dustAdd
|
||||
sender.expectMsg(RES_ADD_FAILED(dustAdd, LocalDustHtlcExposureTooHigh(channelId(alice), 25000.sat, 25001.sat.toMilliSatoshi), Some(initialState.channelUpdate)))
|
||||
val trimmedAdd = CMD_ADD_HTLC(sender.ref, 5000.sat.toMilliSatoshi, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
val trimmedAdd = CMD_ADD_HTLC(sender.ref, 5000.sat.toMilliSatoshi, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
alice ! trimmedAdd
|
||||
sender.expectMsg(RES_ADD_FAILED(trimmedAdd, LocalDustHtlcExposureTooHigh(channelId(alice), 25000.sat, 29500.sat.toMilliSatoshi), Some(initialState.channelUpdate)))
|
||||
val justAboveTrimmedAdd = CMD_ADD_HTLC(sender.ref, 8500.sat.toMilliSatoshi, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
val justAboveTrimmedAdd = CMD_ADD_HTLC(sender.ref, 8500.sat.toMilliSatoshi, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
alice ! justAboveTrimmedAdd
|
||||
sender.expectMsg(RES_ADD_FAILED(justAboveTrimmedAdd, LocalDustHtlcExposureTooHigh(channelId(alice), 25000.sat, 33000.sat.toMilliSatoshi), Some(initialState.channelUpdate)))
|
||||
|
||||
// HTLCs that don't contribute to dust exposure are accepted.
|
||||
alice ! CMD_ADD_HTLC(sender.ref, 25000.sat.toMilliSatoshi, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
alice ! CMD_ADD_HTLC(sender.ref, 25000.sat.toMilliSatoshi, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
sender.expectMsgType[RES_SUCCESS[CMD_ADD_HTLC]]
|
||||
alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
}
|
||||
@ -481,7 +481,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
addHtlc(1500.sat.toMilliSatoshi, alice, bob, alice2bob, bob2alice)
|
||||
|
||||
// HTLCs that take Alice's dust exposure above her threshold are rejected.
|
||||
val add = CMD_ADD_HTLC(sender.ref, 1001.sat.toMilliSatoshi, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
val add = CMD_ADD_HTLC(sender.ref, 1001.sat.toMilliSatoshi, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
alice ! add
|
||||
sender.expectMsg(RES_ADD_FAILED(add, LocalDustHtlcExposureTooHigh(channelId(alice), 25000.sat, 25001.sat.toMilliSatoshi), Some(initialState.channelUpdate)))
|
||||
}
|
||||
@ -505,7 +505,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
(1 to 3).foreach(_ => addHtlc(1050.sat.toMilliSatoshi, alice, bob, alice2bob, bob2alice))
|
||||
|
||||
// HTLCs that take Alice's dust exposure above her threshold are rejected.
|
||||
val add = CMD_ADD_HTLC(sender.ref, 1050.sat.toMilliSatoshi, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
val add = CMD_ADD_HTLC(sender.ref, 1050.sat.toMilliSatoshi, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
alice ! add
|
||||
sender.expectMsg(RES_ADD_FAILED(add, LocalDustHtlcExposureTooHigh(channelId(alice), 25000.sat, 25200.sat.toMilliSatoshi), Some(initialState.channelUpdate)))
|
||||
}
|
||||
@ -529,7 +529,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
(1 to 8).foreach(_ => addHtlc(1050.sat.toMilliSatoshi, bob, alice, bob2alice, alice2bob))
|
||||
|
||||
// HTLCs that take Bob's dust exposure above his threshold are rejected.
|
||||
val add = CMD_ADD_HTLC(sender.ref, 1050.sat.toMilliSatoshi, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
val add = CMD_ADD_HTLC(sender.ref, 1050.sat.toMilliSatoshi, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
bob ! add
|
||||
sender.expectMsg(RES_ADD_FAILED(add, RemoteDustHtlcExposureTooHigh(channelId(bob), 30000.sat, 30450.sat.toMilliSatoshi), Some(initialState.channelUpdate)))
|
||||
}
|
||||
@ -538,14 +538,14 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
import f._
|
||||
val sender = TestProbe()
|
||||
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
|
||||
val add1 = CMD_ADD_HTLC(sender.ref, TestConstants.fundingSatoshis.toMilliSatoshi * 2 / 3, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
val add1 = CMD_ADD_HTLC(sender.ref, TestConstants.fundingSatoshis.toMilliSatoshi * 2 / 3, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
alice ! add1
|
||||
sender.expectMsgType[RES_SUCCESS[CMD_ADD_HTLC]]
|
||||
alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
alice ! CMD_SIGN()
|
||||
alice2bob.expectMsgType[CommitSig]
|
||||
// this is over channel-capacity
|
||||
val add2 = CMD_ADD_HTLC(sender.ref, TestConstants.fundingSatoshis.toMilliSatoshi * 2 / 3, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
val add2 = CMD_ADD_HTLC(sender.ref, TestConstants.fundingSatoshis.toMilliSatoshi * 2 / 3, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
alice ! add2
|
||||
val error = InsufficientFunds(channelId(alice), add2.amount, 578133 sat, 20000 sat, 10680 sat)
|
||||
sender.expectMsg(RES_ADD_FAILED(add2, error, Some(initialState.channelUpdate)))
|
||||
@ -562,7 +562,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
|
||||
val initialState = bob.stateData.asInstanceOf[DATA_NORMAL]
|
||||
val upstream = localOrigin(sender.ref)
|
||||
val add = CMD_ADD_HTLC(sender.ref, 500000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, upstream)
|
||||
val add = CMD_ADD_HTLC(sender.ref, 500000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, upstream)
|
||||
bob ! add
|
||||
val error = FeerateTooDifferent(channelId(bob), FeeratePerKw(20000 sat), FeeratePerKw(10000 sat))
|
||||
sender.expectMsg(RES_ADD_FAILED(add, error, Some(initialState.channelUpdate)))
|
||||
@ -587,7 +587,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].localShutdown.isDefined && alice.stateData.asInstanceOf[DATA_NORMAL].remoteShutdown.isEmpty)
|
||||
|
||||
// actual test starts here
|
||||
val add = CMD_ADD_HTLC(sender.ref, 500000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
val add = CMD_ADD_HTLC(sender.ref, 500000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
alice ! add
|
||||
val error = NoMoreHtlcsClosingInProgress(channelId(alice))
|
||||
sender.expectMsg(RES_ADD_FAILED(add, error, Some(initialState.channelUpdate)))
|
||||
@ -599,14 +599,14 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
val sender = TestProbe()
|
||||
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
|
||||
// let's make alice send an htlc
|
||||
val add1 = CMD_ADD_HTLC(sender.ref, 50000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
val add1 = CMD_ADD_HTLC(sender.ref, 50000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
alice ! add1
|
||||
sender.expectMsgType[RES_SUCCESS[CMD_ADD_HTLC]]
|
||||
// at the same time bob initiates a closing
|
||||
bob ! CMD_CLOSE(sender.ref, None, None)
|
||||
sender.expectMsgType[RES_SUCCESS[CMD_CLOSE]]
|
||||
// this command will be received by alice right after having received the shutdown
|
||||
val add2 = CMD_ADD_HTLC(sender.ref, 10000000 msat, randomBytes32(), CltvExpiry(300000), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
val add2 = CMD_ADD_HTLC(sender.ref, 10000000 msat, randomBytes32(), CltvExpiry(300000), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
// messages cross
|
||||
alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
alice2bob.forward(bob)
|
||||
@ -620,7 +620,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
test("recv UpdateAddHtlc") { f =>
|
||||
import f._
|
||||
val initialState = bob.stateData.asInstanceOf[DATA_NORMAL]
|
||||
val htlc = UpdateAddHtlc(ByteVector32.Zeroes, 0, 150000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val htlc = UpdateAddHtlc(ByteVector32.Zeroes, 0, 150000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
bob ! htlc
|
||||
awaitCond(bob.stateData == initialState
|
||||
.modify(_.commitments.changes.remoteChanges.proposed).using(_ :+ htlc)
|
||||
@ -632,7 +632,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
test("recv UpdateAddHtlc (unexpected id)") { f =>
|
||||
import f._
|
||||
val tx = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localCommit.commitTxAndRemoteSig.commitTx.tx
|
||||
val htlc = UpdateAddHtlc(ByteVector32.Zeroes, 42, 150000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val htlc = UpdateAddHtlc(ByteVector32.Zeroes, 42, 150000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
bob ! htlc.copy(id = 0)
|
||||
bob ! htlc.copy(id = 1)
|
||||
bob ! htlc.copy(id = 2)
|
||||
@ -649,7 +649,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
test("recv UpdateAddHtlc (value too small)") { f =>
|
||||
import f._
|
||||
val tx = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localCommit.commitTxAndRemoteSig.commitTx.tx
|
||||
val htlc = UpdateAddHtlc(ByteVector32.Zeroes, 0, 150 msat, randomBytes32(), cltvExpiry = CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val htlc = UpdateAddHtlc(ByteVector32.Zeroes, 0, 150 msat, randomBytes32(), cltvExpiry = CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
alice2bob.forward(bob, htlc)
|
||||
val error = bob2alice.expectMsgType[Error]
|
||||
assert(new String(error.data.toArray) == HtlcValueTooSmall(channelId(bob), minimum = 1000 msat, actual = 150 msat).getMessage)
|
||||
@ -664,7 +664,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
test("recv UpdateAddHtlc (insufficient funds)") { f =>
|
||||
import f._
|
||||
val tx = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localCommit.commitTxAndRemoteSig.commitTx.tx
|
||||
val htlc = UpdateAddHtlc(ByteVector32.Zeroes, 0, MilliSatoshi(Long.MaxValue), randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val htlc = UpdateAddHtlc(ByteVector32.Zeroes, 0, MilliSatoshi(Long.MaxValue), randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
alice2bob.forward(bob, htlc)
|
||||
val error = bob2alice.expectMsgType[Error]
|
||||
assert(new String(error.data.toArray) == InsufficientFunds(channelId(bob), amount = MilliSatoshi(Long.MaxValue), missing = 9223372036083735L sat, reserve = 20000 sat, fees = 8960 sat).getMessage)
|
||||
@ -679,9 +679,9 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
test("recv UpdateAddHtlc (insufficient funds w/ pending htlcs) (anchor outputs)", Tag(ChannelStateTestsTags.AnchorOutputs)) { f =>
|
||||
import f._
|
||||
val tx = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localCommit.commitTxAndRemoteSig.commitTx.tx
|
||||
alice2bob.forward(bob, UpdateAddHtlc(ByteVector32.Zeroes, 0, 400000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0))
|
||||
alice2bob.forward(bob, UpdateAddHtlc(ByteVector32.Zeroes, 1, 300000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0))
|
||||
alice2bob.forward(bob, UpdateAddHtlc(ByteVector32.Zeroes, 2, 100000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0))
|
||||
alice2bob.forward(bob, UpdateAddHtlc(ByteVector32.Zeroes, 0, 400000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None))
|
||||
alice2bob.forward(bob, UpdateAddHtlc(ByteVector32.Zeroes, 1, 300000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None))
|
||||
alice2bob.forward(bob, UpdateAddHtlc(ByteVector32.Zeroes, 2, 100000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None))
|
||||
val error = bob2alice.expectMsgType[Error]
|
||||
assert(new String(error.data.toArray) == InsufficientFunds(channelId(bob), amount = 100000000 msat, missing = 24760 sat, reserve = 20000 sat, fees = 4760 sat).getMessage)
|
||||
awaitCond(bob.stateName == CLOSING)
|
||||
@ -693,10 +693,10 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
test("recv UpdateAddHtlc (insufficient funds w/ pending htlcs 1/2)") { f =>
|
||||
import f._
|
||||
val tx = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localCommit.commitTxAndRemoteSig.commitTx.tx
|
||||
alice2bob.forward(bob, UpdateAddHtlc(ByteVector32.Zeroes, 0, 400000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0))
|
||||
alice2bob.forward(bob, UpdateAddHtlc(ByteVector32.Zeroes, 1, 200000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0))
|
||||
alice2bob.forward(bob, UpdateAddHtlc(ByteVector32.Zeroes, 2, 167600000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0))
|
||||
alice2bob.forward(bob, UpdateAddHtlc(ByteVector32.Zeroes, 3, 10000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0))
|
||||
alice2bob.forward(bob, UpdateAddHtlc(ByteVector32.Zeroes, 0, 400000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None))
|
||||
alice2bob.forward(bob, UpdateAddHtlc(ByteVector32.Zeroes, 1, 200000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None))
|
||||
alice2bob.forward(bob, UpdateAddHtlc(ByteVector32.Zeroes, 2, 167600000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None))
|
||||
alice2bob.forward(bob, UpdateAddHtlc(ByteVector32.Zeroes, 3, 10000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None))
|
||||
val error = bob2alice.expectMsgType[Error]
|
||||
assert(new String(error.data.toArray) == InsufficientFunds(channelId(bob), amount = 10000000 msat, missing = 11720 sat, reserve = 20000 sat, fees = 14120 sat).getMessage)
|
||||
awaitCond(bob.stateName == CLOSING)
|
||||
@ -710,9 +710,9 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
test("recv UpdateAddHtlc (insufficient funds w/ pending htlcs 2/2)") { f =>
|
||||
import f._
|
||||
val tx = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localCommit.commitTxAndRemoteSig.commitTx.tx
|
||||
alice2bob.forward(bob, UpdateAddHtlc(ByteVector32.Zeroes, 0, 300000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0))
|
||||
alice2bob.forward(bob, UpdateAddHtlc(ByteVector32.Zeroes, 1, 300000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0))
|
||||
alice2bob.forward(bob, UpdateAddHtlc(ByteVector32.Zeroes, 2, 500000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0))
|
||||
alice2bob.forward(bob, UpdateAddHtlc(ByteVector32.Zeroes, 0, 300000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None))
|
||||
alice2bob.forward(bob, UpdateAddHtlc(ByteVector32.Zeroes, 1, 300000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None))
|
||||
alice2bob.forward(bob, UpdateAddHtlc(ByteVector32.Zeroes, 2, 500000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None))
|
||||
val error = bob2alice.expectMsgType[Error]
|
||||
assert(new String(error.data.toArray) == InsufficientFunds(channelId(bob), amount = 500000000 msat, missing = 332400 sat, reserve = 20000 sat, fees = 12400 sat).getMessage)
|
||||
awaitCond(bob.stateName == CLOSING)
|
||||
@ -726,7 +726,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
test("recv UpdateAddHtlc (over max inflight htlc value)", Tag(ChannelStateTestsTags.AliceLowMaxHtlcValueInFlight)) { f =>
|
||||
import f._
|
||||
val tx = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localCommit.commitTxAndRemoteSig.commitTx.tx
|
||||
alice2bob.forward(alice, UpdateAddHtlc(ByteVector32.Zeroes, 0, 151000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0))
|
||||
alice2bob.forward(alice, UpdateAddHtlc(ByteVector32.Zeroes, 0, 151000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None))
|
||||
val error = alice2bob.expectMsgType[Error]
|
||||
assert(new String(error.data.toArray) == HtlcValueTooHighInFlight(channelId(alice), maximum = 150000000 msat, actual = 151000000 msat).getMessage)
|
||||
awaitCond(alice.stateName == CLOSING)
|
||||
@ -742,9 +742,9 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
val tx = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localCommit.commitTxAndRemoteSig.commitTx.tx
|
||||
// Bob accepts a maximum of 30 htlcs
|
||||
for (i <- 0 until 30) {
|
||||
alice2bob.forward(bob, UpdateAddHtlc(ByteVector32.Zeroes, i, 1000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0))
|
||||
alice2bob.forward(bob, UpdateAddHtlc(ByteVector32.Zeroes, i, 1000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None))
|
||||
}
|
||||
alice2bob.forward(bob, UpdateAddHtlc(ByteVector32.Zeroes, 30, 1000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0))
|
||||
alice2bob.forward(bob, UpdateAddHtlc(ByteVector32.Zeroes, 30, 1000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None))
|
||||
val error = bob2alice.expectMsgType[Error]
|
||||
assert(new String(error.data.toArray) == TooManyAcceptedHtlcs(channelId(bob), maximum = 30).getMessage)
|
||||
awaitCond(bob.stateName == CLOSING)
|
||||
@ -767,7 +767,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
test("recv CMD_SIGN (two identical htlcs in each direction)") { f =>
|
||||
import f._
|
||||
val sender = TestProbe()
|
||||
val add = CMD_ADD_HTLC(sender.ref, 10000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
val add = CMD_ADD_HTLC(sender.ref, 10000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
alice ! add
|
||||
sender.expectMsgType[RES_SUCCESS[CMD_ADD_HTLC]]
|
||||
alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
@ -818,19 +818,19 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
assert(a2b_2 > aliceMinOffer && a2b_2 > bobMinReceive)
|
||||
assert(b2a_1 > aliceMinReceive && b2a_1 > bobMinOffer)
|
||||
assert(b2a_2 < aliceMinReceive && b2a_2 > bobMinOffer)
|
||||
alice ! CMD_ADD_HTLC(sender.ref, a2b_1.toMilliSatoshi, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
alice ! CMD_ADD_HTLC(sender.ref, a2b_1.toMilliSatoshi, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
sender.expectMsgType[RES_SUCCESS[CMD_ADD_HTLC]]
|
||||
alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
alice2bob.forward(bob)
|
||||
alice ! CMD_ADD_HTLC(sender.ref, a2b_2.toMilliSatoshi, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
alice ! CMD_ADD_HTLC(sender.ref, a2b_2.toMilliSatoshi, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
sender.expectMsgType[RES_SUCCESS[CMD_ADD_HTLC]]
|
||||
alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
alice2bob.forward(bob)
|
||||
bob ! CMD_ADD_HTLC(sender.ref, b2a_1.toMilliSatoshi, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
bob ! CMD_ADD_HTLC(sender.ref, b2a_1.toMilliSatoshi, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
sender.expectMsgType[RES_SUCCESS[CMD_ADD_HTLC]]
|
||||
bob2alice.expectMsgType[UpdateAddHtlc]
|
||||
bob2alice.forward(alice)
|
||||
bob ! CMD_ADD_HTLC(sender.ref, b2a_2.toMilliSatoshi, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
bob ! CMD_ADD_HTLC(sender.ref, b2a_2.toMilliSatoshi, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
sender.expectMsgType[RES_SUCCESS[CMD_ADD_HTLC]]
|
||||
bob2alice.expectMsgType[UpdateAddHtlc]
|
||||
bob2alice.forward(alice)
|
||||
@ -850,7 +850,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
test("recv CMD_SIGN (htlcs with same pubkeyScript but different amounts)") { f =>
|
||||
import f._
|
||||
val sender = TestProbe()
|
||||
val add = CMD_ADD_HTLC(sender.ref, 10000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
val add = CMD_ADD_HTLC(sender.ref, 10000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
val epsilons = List(3, 1, 5, 7, 6) // unordered on purpose
|
||||
val htlcCount = epsilons.size
|
||||
for (i <- epsilons) {
|
||||
@ -1184,12 +1184,12 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
val r = randomBytes32()
|
||||
val h = Crypto.sha256(r)
|
||||
|
||||
alice ! CMD_ADD_HTLC(sender.ref, 50000000 msat, h, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
alice ! CMD_ADD_HTLC(sender.ref, 50000000 msat, h, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
sender.expectMsgType[RES_SUCCESS[CMD_ADD_HTLC]]
|
||||
val htlc1 = alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
alice2bob.forward(bob)
|
||||
|
||||
alice ! CMD_ADD_HTLC(sender.ref, 50000000 msat, h, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
alice ! CMD_ADD_HTLC(sender.ref, 50000000 msat, h, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
sender.expectMsgType[RES_SUCCESS[CMD_ADD_HTLC]]
|
||||
val htlc2 = alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
alice2bob.forward(bob)
|
||||
@ -2271,7 +2271,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
bob2alice.expectNoMessage(250 millis) // we don't close because the commitment doesn't contain any HTLC
|
||||
|
||||
// when we try to add an HTLC, we still disagree on the feerate so we close
|
||||
alice2bob.send(bob, UpdateAddHtlc(ByteVector32.Zeroes, 0, 2500000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0))
|
||||
alice2bob.send(bob, UpdateAddHtlc(ByteVector32.Zeroes, 0, 2500000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None))
|
||||
val error = bob2alice.expectMsgType[Error]
|
||||
assert(new String(error.data.toArray).contains("local/remote feerates are too different"))
|
||||
awaitCond(bob.stateName == CLOSING)
|
||||
@ -2299,7 +2299,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
|
||||
val initialState = bob.stateData.asInstanceOf[DATA_NORMAL]
|
||||
assert(initialState.commitments.latest.localCommit.spec.commitTxFeerate == TestConstants.anchorOutputsFeeratePerKw)
|
||||
val add = UpdateAddHtlc(ByteVector32.Zeroes, 0, 2500000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val add = UpdateAddHtlc(ByteVector32.Zeroes, 0, 2500000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
alice2bob.send(bob, add)
|
||||
val fee = UpdateFee(initialState.channelId, FeeratePerKw(FeeratePerByte(2 sat)))
|
||||
alice2bob.send(bob, fee)
|
||||
@ -3067,7 +3067,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
bob2alice.expectNoMessage(250 millis) // we don't close because the commitment doesn't contain any HTLC
|
||||
|
||||
// when we try to add an HTLC, we still disagree on the feerate so we close
|
||||
alice2bob.send(bob, UpdateAddHtlc(ByteVector32.Zeroes, 0, 2500000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0))
|
||||
alice2bob.send(bob, UpdateAddHtlc(ByteVector32.Zeroes, 0, 2500000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None))
|
||||
bob2alice.expectMsgType[Error]
|
||||
bob2blockchain.expectMsgType[PublishTx] // commit tx
|
||||
bob2blockchain.expectMsgType[PublishTx] // main delayed
|
||||
@ -3315,7 +3315,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
// alice = 800 000
|
||||
// bob = 200 000
|
||||
|
||||
val add = CMD_ADD_HTLC(sender.ref, 10000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
val add = CMD_ADD_HTLC(sender.ref, 10000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
alice ! add
|
||||
sender.expectMsgType[RES_SUCCESS[CMD_ADD_HTLC]]
|
||||
alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
|
@ -72,6 +72,37 @@ class OfflineStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
val aliceInit = Init(TestConstants.Alice.nodeParams.features.initFeatures())
|
||||
val bobInit = Init(TestConstants.Bob.nodeParams.features.initFeatures())
|
||||
|
||||
test("reconnect after creating channel", Tag(IgnoreChannelUpdates)) { f =>
|
||||
import f._
|
||||
|
||||
disconnect(alice, bob)
|
||||
reconnect(alice, bob, alice2bob, bob2alice)
|
||||
alice2bob.expectMsgType[ChannelReestablish]
|
||||
alice2bob.forward(bob)
|
||||
bob2alice.expectMsgType[ChannelReestablish]
|
||||
bob2alice.forward(alice)
|
||||
|
||||
// This is a new channel: peers exchange channel_ready again.
|
||||
val channelId = alice2bob.expectMsgType[ChannelReady].channelId
|
||||
bob2alice.expectMsgType[ChannelReady]
|
||||
|
||||
// The channel is ready to process payments.
|
||||
alicePeer.fishForMessage() {
|
||||
case e: ChannelReadyForPayments =>
|
||||
assert(e.fundingTxIndex == 0)
|
||||
assert(e.channelId == channelId)
|
||||
true
|
||||
case _ => false
|
||||
}
|
||||
bobPeer.fishForMessage() {
|
||||
case e: ChannelReadyForPayments =>
|
||||
assert(e.fundingTxIndex == 0)
|
||||
assert(e.channelId == channelId)
|
||||
true
|
||||
case _ => false
|
||||
}
|
||||
}
|
||||
|
||||
test("re-send lost htlc and signature after first commitment", Tag(IgnoreChannelUpdates)) { f =>
|
||||
import f._
|
||||
// alice bob
|
||||
@ -80,7 +111,7 @@ class OfflineStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
// |--- sig --X |
|
||||
// | |
|
||||
val sender = TestProbe()
|
||||
alice ! CMD_ADD_HTLC(sender.ref, 1000000 msat, ByteVector32.Zeroes, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
alice ! CMD_ADD_HTLC(sender.ref, 1000000 msat, ByteVector32.Zeroes, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
val htlc = alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
// bob receives the htlc
|
||||
alice2bob.forward(bob)
|
||||
@ -134,7 +165,7 @@ class OfflineStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
// | X-- rev ---|
|
||||
// | X-- sig ---|
|
||||
val sender = TestProbe()
|
||||
alice ! CMD_ADD_HTLC(ActorRef.noSender, 1000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
alice ! CMD_ADD_HTLC(ActorRef.noSender, 1000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
val htlc = alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
// bob receives the htlc and the signature
|
||||
alice2bob.forward(bob, htlc)
|
||||
@ -178,7 +209,7 @@ class OfflineStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
// |<--- rev ---|
|
||||
// | X-- sig ---|
|
||||
val sender = TestProbe()
|
||||
alice ! CMD_ADD_HTLC(ActorRef.noSender, 1000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
alice ! CMD_ADD_HTLC(ActorRef.noSender, 1000000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
val htlc = alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
// bob receives the htlc and the signature
|
||||
alice2bob.forward(bob, htlc)
|
||||
@ -511,7 +542,7 @@ class OfflineStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
channelUpdateListener.expectNoMessage(300 millis)
|
||||
|
||||
// we attempt to send a payment
|
||||
alice ! CMD_ADD_HTLC(sender.ref, 4200 msat, randomBytes32(), CltvExpiry(123456), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
alice ! CMD_ADD_HTLC(sender.ref, 4200 msat, randomBytes32(), CltvExpiry(123456), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
sender.expectMsgType[RES_ADD_FAILED[ChannelUnavailable]]
|
||||
|
||||
// alice will broadcast a new disabled channel_update
|
||||
|
@ -141,7 +141,7 @@ class ShutdownStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wit
|
||||
test("recv CMD_ADD_HTLC") { f =>
|
||||
import f._
|
||||
val sender = TestProbe()
|
||||
val add = CMD_ADD_HTLC(sender.ref, 500000000 msat, r1, cltvExpiry = CltvExpiry(300000), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
val add = CMD_ADD_HTLC(sender.ref, 500000000 msat, r1, cltvExpiry = CltvExpiry(300000), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
alice ! add
|
||||
val error = ChannelUnavailable(channelId(alice))
|
||||
sender.expectMsg(RES_ADD_FAILED(add, error, None))
|
||||
|
@ -123,7 +123,7 @@ class NegotiatingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
||||
aliceClose(f)
|
||||
alice2bob.expectMsgType[ClosingSigned]
|
||||
val sender = TestProbe()
|
||||
val add = CMD_ADD_HTLC(sender.ref, 5000000000L msat, randomBytes32(), CltvExpiry(300000), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
val add = CMD_ADD_HTLC(sender.ref, 5000000000L msat, randomBytes32(), CltvExpiry(300000), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
alice ! add
|
||||
val error = ChannelUnavailable(channelId(alice))
|
||||
sender.expectMsg(RES_ADD_FAILED(add, error, None))
|
||||
|
@ -77,7 +77,7 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
val bobInit = Init(bobParams.initFeatures)
|
||||
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = false, TestConstants.feeratePerKw, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, Some(TestConstants.initiatorPushAmount), requireConfirmedInputs = false, requestFunding_opt = None, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped)
|
||||
alice2blockchain.expectMsgType[SetChannelId]
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, None, dualFunded = false, None, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, None, dualFunded = false, None, requireConfirmedInputs = false, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
|
||||
bob2blockchain.expectMsgType[SetChannelId]
|
||||
alice2bob.expectMsgType[OpenChannel]
|
||||
alice2bob.forward(bob)
|
||||
@ -267,7 +267,7 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
||||
|
||||
// actual test starts here
|
||||
val sender = TestProbe()
|
||||
val add = CMD_ADD_HTLC(sender.ref, 500000000 msat, ByteVector32(ByteVector.fill(32)(1)), cltvExpiry = CltvExpiry(300000), onion = TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref))
|
||||
val add = CMD_ADD_HTLC(sender.ref, 500000000 msat, ByteVector32(ByteVector.fill(32)(1)), cltvExpiry = CltvExpiry(300000), onion = TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
alice ! add
|
||||
val error = ChannelUnavailable(channelId(alice))
|
||||
sender.expectMsg(RES_ADD_FAILED(add, error, None))
|
||||
|
@ -16,13 +16,17 @@
|
||||
|
||||
package fr.acinq.eclair.db
|
||||
|
||||
import fr.acinq.bitcoin.scalacompat.{SatoshiLong, TxId}
|
||||
import fr.acinq.bitcoin.scalacompat.{Crypto, SatoshiLong, TxId}
|
||||
import fr.acinq.eclair.TestDatabases.forAllDbs
|
||||
import fr.acinq.eclair.channel.{ChannelLiquidityPurchased, LiquidityPurchase}
|
||||
import fr.acinq.eclair.wire.protocol.LiquidityAds
|
||||
import fr.acinq.eclair.{MilliSatoshiLong, randomBytes32, randomKey}
|
||||
import fr.acinq.eclair.channel.{ChannelLiquidityPurchased, LiquidityPurchase, Upstream}
|
||||
import fr.acinq.eclair.payment.relay.OnTheFlyFunding
|
||||
import fr.acinq.eclair.payment.relay.OnTheFlyFundingSpec.{createWillAdd, randomOnion}
|
||||
import fr.acinq.eclair.wire.protocol.{LiquidityAds, UpdateAddHtlc}
|
||||
import fr.acinq.eclair.{CltvExpiry, MilliSatoshiLong, TimestampMilli, randomBytes32, randomKey}
|
||||
import org.scalatest.funsuite.AnyFunSuite
|
||||
|
||||
import java.util.UUID
|
||||
|
||||
class LiquidityDbSpec extends AnyFunSuite {
|
||||
|
||||
test("add/list liquidity purchases") {
|
||||
@ -57,4 +61,128 @@ class LiquidityDbSpec extends AnyFunSuite {
|
||||
}
|
||||
}
|
||||
|
||||
test("add/list/remove pending on-the-fly funding proposals") {
|
||||
forAllDbs { dbs =>
|
||||
val db = dbs.liquidity
|
||||
|
||||
val alice = randomKey().publicKey
|
||||
val bob = randomKey().publicKey
|
||||
val paymentHash1 = randomBytes32()
|
||||
val paymentHash2 = randomBytes32()
|
||||
val upstream = Seq(
|
||||
Upstream.Hot.Channel(UpdateAddHtlc(randomBytes32(), 7, 25_000_000 msat, paymentHash1, CltvExpiry(750_000), randomOnion(), None, 1.0, None), TimestampMilli(0), randomKey().publicKey),
|
||||
Upstream.Hot.Channel(UpdateAddHtlc(randomBytes32(), 0, 1 msat, paymentHash1, CltvExpiry(750_000), randomOnion(), Some(randomKey().publicKey), 1.0, None), TimestampMilli.now(), randomKey().publicKey),
|
||||
Upstream.Hot.Channel(UpdateAddHtlc(randomBytes32(), 561, 100_000_000 msat, paymentHash2, CltvExpiry(799_999), randomOnion(), None, 1.0, None), TimestampMilli.now(), randomKey().publicKey),
|
||||
Upstream.Hot.Channel(UpdateAddHtlc(randomBytes32(), 1105, 100_000_000 msat, paymentHash2, CltvExpiry(799_999), randomOnion(), None, 1.0, None), TimestampMilli.now(), randomKey().publicKey),
|
||||
)
|
||||
val pendingAlice = Seq(
|
||||
OnTheFlyFunding.Pending(
|
||||
proposed = Seq(
|
||||
OnTheFlyFunding.Proposal(createWillAdd(20_000 msat, paymentHash1, CltvExpiry(500)), upstream(0)),
|
||||
OnTheFlyFunding.Proposal(createWillAdd(1 msat, paymentHash1, CltvExpiry(750), Some(randomKey().publicKey)), upstream(1)),
|
||||
),
|
||||
status = OnTheFlyFunding.Status.Funded(randomBytes32(), TxId(randomBytes32()), 7, 500 msat)
|
||||
),
|
||||
OnTheFlyFunding.Pending(
|
||||
proposed = Seq(
|
||||
OnTheFlyFunding.Proposal(createWillAdd(195_000_000 msat, paymentHash2, CltvExpiry(1000)), Upstream.Hot.Trampoline(upstream(2) :: upstream(3) :: Nil)),
|
||||
),
|
||||
status = OnTheFlyFunding.Status.Funded(randomBytes32(), TxId(randomBytes32()), 3, 0 msat)
|
||||
)
|
||||
)
|
||||
val pendingBob = Seq(
|
||||
OnTheFlyFunding.Pending(
|
||||
proposed = Seq(
|
||||
OnTheFlyFunding.Proposal(createWillAdd(20_000 msat, paymentHash1, CltvExpiry(42)), upstream(0)),
|
||||
),
|
||||
status = OnTheFlyFunding.Status.Funded(randomBytes32(), TxId(randomBytes32()), 11, 3_500 msat)
|
||||
),
|
||||
OnTheFlyFunding.Pending(
|
||||
proposed = Seq(
|
||||
OnTheFlyFunding.Proposal(createWillAdd(24_000_000 msat, paymentHash2, CltvExpiry(800_000), Some(randomKey().publicKey)), Upstream.Local(UUID.randomUUID())),
|
||||
),
|
||||
status = OnTheFlyFunding.Status.Funded(randomBytes32(), TxId(randomBytes32()), 0, 10_000 msat)
|
||||
)
|
||||
)
|
||||
|
||||
assert(db.listPendingOnTheFlyPayments().isEmpty)
|
||||
assert(db.listPendingOnTheFlyFunding(alice).isEmpty)
|
||||
assert(db.listPendingOnTheFlyFunding().isEmpty)
|
||||
db.removePendingOnTheFlyFunding(alice, paymentHash1) // no-op
|
||||
|
||||
// Add pending proposals for Alice.
|
||||
db.addPendingOnTheFlyFunding(alice, pendingAlice(0))
|
||||
assert(db.listPendingOnTheFlyFunding(alice) == Map(paymentHash1 -> pendingAlice(0)))
|
||||
assert(db.listPendingOnTheFlyFunding() == Map(alice -> Map(paymentHash1 -> pendingAlice(0))))
|
||||
db.addPendingOnTheFlyFunding(alice, pendingAlice(1).copy(status = OnTheFlyFunding.Status.Proposed(null)))
|
||||
assert(db.listPendingOnTheFlyFunding(alice) == Map(paymentHash1 -> pendingAlice(0)))
|
||||
assert(db.listPendingOnTheFlyFunding() == Map(alice -> Map(paymentHash1 -> pendingAlice(0))))
|
||||
db.addPendingOnTheFlyFunding(alice, pendingAlice(1))
|
||||
assert(db.listPendingOnTheFlyFunding(alice) == Map(paymentHash1 -> pendingAlice(0), paymentHash2 -> pendingAlice(1)))
|
||||
assert(db.listPendingOnTheFlyFunding() == Map(alice -> Map(paymentHash1 -> pendingAlice(0), paymentHash2 -> pendingAlice(1))))
|
||||
assert(db.listPendingOnTheFlyPayments() == Map(alice -> Set(paymentHash1, paymentHash2)))
|
||||
|
||||
// Add pending proposals for Bob.
|
||||
assert(db.listPendingOnTheFlyFunding(bob).isEmpty)
|
||||
db.addPendingOnTheFlyFunding(bob, pendingBob(0))
|
||||
db.addPendingOnTheFlyFunding(bob, pendingBob(1))
|
||||
assert(db.listPendingOnTheFlyFunding(alice) == Map(paymentHash1 -> pendingAlice(0), paymentHash2 -> pendingAlice(1)))
|
||||
assert(db.listPendingOnTheFlyFunding(bob) == Map(paymentHash1 -> pendingBob(0), paymentHash2 -> pendingBob(1)))
|
||||
assert(db.listPendingOnTheFlyFunding() == Map(
|
||||
alice -> Map(paymentHash1 -> pendingAlice(0), paymentHash2 -> pendingAlice(1)),
|
||||
bob -> Map(paymentHash1 -> pendingBob(0), paymentHash2 -> pendingBob(1))
|
||||
))
|
||||
assert(db.listPendingOnTheFlyPayments() == Map(alice -> Set(paymentHash1, paymentHash2), bob -> Set(paymentHash1, paymentHash2)))
|
||||
|
||||
// Remove pending proposals that are completed.
|
||||
db.removePendingOnTheFlyFunding(alice, paymentHash1)
|
||||
assert(db.listPendingOnTheFlyFunding(alice) == Map(paymentHash2 -> pendingAlice(1)))
|
||||
assert(db.listPendingOnTheFlyFunding(bob) == Map(paymentHash1 -> pendingBob(0), paymentHash2 -> pendingBob(1)))
|
||||
assert(db.listPendingOnTheFlyFunding() == Map(
|
||||
alice -> Map(paymentHash2 -> pendingAlice(1)),
|
||||
bob -> Map(paymentHash1 -> pendingBob(0), paymentHash2 -> pendingBob(1))
|
||||
))
|
||||
assert(db.listPendingOnTheFlyPayments() == Map(alice -> Set(paymentHash2), bob -> Set(paymentHash1, paymentHash2)))
|
||||
db.removePendingOnTheFlyFunding(alice, paymentHash1) // no-op
|
||||
db.removePendingOnTheFlyFunding(bob, randomBytes32()) // no-op
|
||||
assert(db.listPendingOnTheFlyFunding(alice) == Map(paymentHash2 -> pendingAlice(1)))
|
||||
assert(db.listPendingOnTheFlyFunding(bob) == Map(paymentHash1 -> pendingBob(0), paymentHash2 -> pendingBob(1)))
|
||||
assert(db.listPendingOnTheFlyFunding() == Map(
|
||||
alice -> Map(paymentHash2 -> pendingAlice(1)),
|
||||
bob -> Map(paymentHash1 -> pendingBob(0), paymentHash2 -> pendingBob(1))
|
||||
))
|
||||
assert(db.listPendingOnTheFlyPayments() == Map(alice -> Set(paymentHash2), bob -> Set(paymentHash1, paymentHash2)))
|
||||
db.removePendingOnTheFlyFunding(alice, paymentHash2)
|
||||
assert(db.listPendingOnTheFlyFunding(alice).isEmpty)
|
||||
assert(db.listPendingOnTheFlyFunding(bob) == Map(paymentHash1 -> pendingBob(0), paymentHash2 -> pendingBob(1)))
|
||||
assert(db.listPendingOnTheFlyFunding() == Map(bob -> Map(paymentHash1 -> pendingBob(0), paymentHash2 -> pendingBob(1))))
|
||||
assert(db.listPendingOnTheFlyPayments() == Map(bob -> Set(paymentHash1, paymentHash2)))
|
||||
db.removePendingOnTheFlyFunding(bob, paymentHash2)
|
||||
assert(db.listPendingOnTheFlyFunding(bob) == Map(paymentHash1 -> pendingBob(0)))
|
||||
assert(db.listPendingOnTheFlyFunding() == Map(bob -> Map(paymentHash1 -> pendingBob(0))))
|
||||
assert(db.listPendingOnTheFlyPayments() == Map(bob -> Set(paymentHash1)))
|
||||
db.removePendingOnTheFlyFunding(bob, paymentHash1)
|
||||
assert(db.listPendingOnTheFlyFunding(bob).isEmpty)
|
||||
assert(db.listPendingOnTheFlyFunding().isEmpty)
|
||||
assert(db.listPendingOnTheFlyPayments().isEmpty)
|
||||
}
|
||||
}
|
||||
|
||||
test("add/get on-the-fly-funding preimages") {
|
||||
forAllDbs { dbs =>
|
||||
val db = dbs.liquidity
|
||||
|
||||
val preimage1 = randomBytes32()
|
||||
val preimage2 = randomBytes32()
|
||||
|
||||
db.addOnTheFlyFundingPreimage(preimage1)
|
||||
db.addOnTheFlyFundingPreimage(preimage1) // no-op
|
||||
db.addOnTheFlyFundingPreimage(preimage2)
|
||||
|
||||
assert(db.getOnTheFlyFundingPreimage(Crypto.sha256(preimage1)).contains(preimage1))
|
||||
assert(db.getOnTheFlyFundingPreimage(Crypto.sha256(preimage2)).contains(preimage2))
|
||||
assert(db.getOnTheFlyFundingPreimage(randomBytes32()).isEmpty)
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -74,7 +74,7 @@ class RustyTestsSpec extends TestKitBaseClass with Matchers with FixtureAnyFunSu
|
||||
// alice and bob will both have 1 000 000 sat
|
||||
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, 2000000 sat, dualFunded = false, commitTxFeerate = feeratePerKw, fundingTxFeerate = feeratePerKw, fundingTxFeeBudget_opt = None, Some(1000000000 msat), requireConfirmedInputs = false, requestFunding_opt = None, Alice.channelParams, pipe, bobInit, ChannelFlags(announceChannel = false), channelConfig, channelType, replyTo = system.deadLetters)
|
||||
alice2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, None, dualFunded = false, None, Bob.channelParams, pipe, aliceInit, channelConfig, channelType)
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, None, dualFunded = false, None, requireConfirmedInputs = false, Bob.channelParams, pipe, aliceInit, channelConfig, channelType)
|
||||
bob2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
pipe ! (alice, bob)
|
||||
within(30 seconds) {
|
||||
|
@ -57,7 +57,7 @@ class SynchronizationPipe(latch: CountDownLatch) extends Actor with ActorLogging
|
||||
|
||||
(script: @unchecked) match {
|
||||
case offer(x, amount, rhash) :: rest =>
|
||||
resolve(x) ! CMD_ADD_HTLC(self, MilliSatoshi(amount.toInt), ByteVector32.fromValidHex(rhash), CltvExpiry(144), TestConstants.emptyOnionPacket, None, 1.0, Origin.Hot(self, Upstream.Local(UUID.randomUUID())))
|
||||
resolve(x) ! CMD_ADD_HTLC(self, MilliSatoshi(amount.toInt), ByteVector32.fromValidHex(rhash), CltvExpiry(144), TestConstants.emptyOnionPacket, None, 1.0, None, Origin.Hot(self, Upstream.Local(UUID.randomUUID())))
|
||||
exec(rest, a, b)
|
||||
case fulfill(x, id, r) :: rest =>
|
||||
resolve(x) ! CMD_FULFILL_HTLC(id.toInt, ByteVector32.fromValidHex(r))
|
||||
|
@ -112,7 +112,7 @@ class MessageRelaySpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app
|
||||
|
||||
val request = switchboard.expectMsgType[GetPeerInfo]
|
||||
assert(request.remoteNodeId == bobId)
|
||||
request.replyTo ! Peer.PeerInfo(peer.ref.toClassic, bobId, Peer.CONNECTED, None, Set.empty)
|
||||
request.replyTo ! Peer.PeerInfo(peer.ref.toClassic, bobId, Peer.CONNECTED, None, None, Set.empty)
|
||||
assert(peer.expectMessageType[Peer.RelayOnionMessage].msg == message)
|
||||
}
|
||||
|
||||
@ -148,7 +148,7 @@ class MessageRelaySpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app
|
||||
|
||||
val getPeerInfo = switchboard.expectMsgType[GetPeerInfo]
|
||||
assert(getPeerInfo.remoteNodeId == previousNodeId)
|
||||
getPeerInfo.replyTo ! PeerInfo(peer.ref.toClassic, previousNodeId, Peer.CONNECTED, None, Set.empty)
|
||||
getPeerInfo.replyTo ! PeerInfo(peer.ref.toClassic, previousNodeId, Peer.CONNECTED, None, None, Set.empty)
|
||||
|
||||
probe.expectMessage(AgainstPolicy(messageId, RelayChannelsOnly))
|
||||
peer.expectNoMessage(100 millis)
|
||||
@ -164,7 +164,7 @@ class MessageRelaySpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app
|
||||
|
||||
val getPeerInfo1 = switchboard.expectMsgType[GetPeerInfo]
|
||||
assert(getPeerInfo1.remoteNodeId == previousNodeId)
|
||||
getPeerInfo1.replyTo ! PeerInfo(peer.ref.toClassic, previousNodeId, Peer.CONNECTED, None, Set(TestProbe()(system.classicSystem).ref))
|
||||
getPeerInfo1.replyTo ! PeerInfo(peer.ref.toClassic, previousNodeId, Peer.CONNECTED, None, None, Set(TestProbe()(system.classicSystem).ref))
|
||||
|
||||
val getPeerInfo2 = switchboard.expectMsgType[GetPeerInfo]
|
||||
assert(getPeerInfo2.remoteNodeId == bobId)
|
||||
@ -184,11 +184,11 @@ class MessageRelaySpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app
|
||||
|
||||
val getPeerInfo1 = switchboard.expectMsgType[GetPeerInfo]
|
||||
assert(getPeerInfo1.remoteNodeId == previousNodeId)
|
||||
getPeerInfo1.replyTo ! PeerInfo(TestProbe()(system.classicSystem).ref, previousNodeId, Peer.CONNECTED, None, Set(TestProbe()(system.classicSystem).ref))
|
||||
getPeerInfo1.replyTo ! PeerInfo(TestProbe()(system.classicSystem).ref, previousNodeId, Peer.CONNECTED, None, None, Set(TestProbe()(system.classicSystem).ref))
|
||||
|
||||
val getPeerInfo2 = switchboard.expectMsgType[GetPeerInfo]
|
||||
assert(getPeerInfo2.remoteNodeId == bobId)
|
||||
getPeerInfo2.replyTo ! PeerInfo(peer.ref.toClassic, bobId, Peer.CONNECTED, None, Set(0, 1).map(_ => TestProbe()(system.classicSystem).ref))
|
||||
getPeerInfo2.replyTo ! PeerInfo(peer.ref.toClassic, bobId, Peer.CONNECTED, None, None, Set(0, 1).map(_ => TestProbe()(system.classicSystem).ref))
|
||||
|
||||
assert(peer.expectMessageType[Peer.RelayOnionMessage].msg == message)
|
||||
}
|
||||
|
@ -22,22 +22,25 @@ import akka.actor.typed.eventstream.EventStream
|
||||
import akka.actor.typed.scaladsl.adapter.TypedActorRefOps
|
||||
import com.softwaremill.quicklens.ModifyPimp
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import fr.acinq.bitcoin.scalacompat.{ByteVector32, Crypto, SatoshiLong}
|
||||
import fr.acinq.bitcoin.scalacompat.{ByteVector32, Crypto, OutPoint, SatoshiLong, Transaction, TxId, TxOut}
|
||||
import fr.acinq.eclair.FeatureSupport.{Mandatory, Optional}
|
||||
import fr.acinq.eclair.Features._
|
||||
import fr.acinq.eclair.blockchain.DummyOnChainWallet
|
||||
import fr.acinq.eclair.channel.ChannelTypes.UnsupportedChannelType
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.channel.fsm.Channel
|
||||
import fr.acinq.eclair.channel.states.ChannelStateTestsTags
|
||||
import fr.acinq.eclair.channel.{ChannelAborted, ChannelTypes}
|
||||
import fr.acinq.eclair.io.OpenChannelInterceptor.{DefaultParams, OpenChannelInitiator, OpenChannelNonInitiator}
|
||||
import fr.acinq.eclair.io.Peer.{OpenChannelResponse, OutgoingMessage, SpawnChannelNonInitiator}
|
||||
import fr.acinq.eclair.io.PeerSpec.createOpenChannelMessage
|
||||
import fr.acinq.eclair.io.PeerSpec.{createOpenChannelMessage, createOpenDualFundedChannelMessage}
|
||||
import fr.acinq.eclair.io.PendingChannelsRateLimiter.AddOrRejectChannel
|
||||
import fr.acinq.eclair.wire.protocol.{ChannelTlv, Error, IPAddress, LiquidityAds, NodeAddress, OpenChannel, OpenChannelTlv, TlvStream}
|
||||
import fr.acinq.eclair.{AcceptOpenChannel, CltvExpiryDelta, FeatureSupport, Features, InitFeature, InterceptOpenChannelCommand, InterceptOpenChannelPlugin, InterceptOpenChannelReceived, MilliSatoshiLong, RejectOpenChannel, TestConstants, UnknownFeature, randomBytes32, randomKey}
|
||||
import fr.acinq.eclair.transactions.Transactions.{ClosingTx, InputInfo}
|
||||
import fr.acinq.eclair.wire.internal.channel.ChannelCodecsSpec
|
||||
import fr.acinq.eclair.wire.protocol.{ChannelReestablish, ChannelTlv, Error, IPAddress, LiquidityAds, NodeAddress, OpenChannel, OpenChannelTlv, Shutdown, TlvStream}
|
||||
import fr.acinq.eclair.{AcceptOpenChannel, BlockHeight, CltvExpiryDelta, FeatureSupport, Features, InitFeature, InterceptOpenChannelCommand, InterceptOpenChannelPlugin, InterceptOpenChannelReceived, MilliSatoshiLong, RejectOpenChannel, TestConstants, UnknownFeature, randomBytes32, randomKey}
|
||||
import org.scalatest.funsuite.FixtureAnyFunSuiteLike
|
||||
import org.scalatest.{Outcome, Tag}
|
||||
import scodec.bits.ByteVector
|
||||
|
||||
import java.net.InetAddress
|
||||
import scala.concurrent.duration.DurationInt
|
||||
@ -47,10 +50,12 @@ class OpenChannelInterceptorSpec extends ScalaTestWithActorTestKit(ConfigFactory
|
||||
val defaultParams: DefaultParams = DefaultParams(100 sat, 100000 msat, 100 msat, CltvExpiryDelta(288), 10)
|
||||
val openChannel: OpenChannel = createOpenChannelMessage()
|
||||
val remoteAddress: NodeAddress = IPAddress(InetAddress.getLoopbackAddress, 19735)
|
||||
val acceptStaticRemoteKeyChannelsTag = "accept static_remote_key channels"
|
||||
val defaultFeatures: Features[InitFeature] = Features(Map[InitFeature, FeatureSupport](StaticRemoteKey -> Optional, AnchorOutputsZeroFeeHtlcTx -> Optional))
|
||||
val staticRemoteKeyFeatures: Features[InitFeature] = Features(Map[InitFeature, FeatureSupport](StaticRemoteKey -> Optional))
|
||||
|
||||
val acceptStaticRemoteKeyChannelsTag = "accept static_remote_key channels"
|
||||
val noPlugin = "no plugin"
|
||||
|
||||
override def withFixture(test: OneArgTest): Outcome = {
|
||||
val peer = TestProbe[Any]()
|
||||
val peerConnection = TestProbe[Any]()
|
||||
@ -58,12 +63,13 @@ class OpenChannelInterceptorSpec extends ScalaTestWithActorTestKit(ConfigFactory
|
||||
val wallet = new DummyOnChainWallet()
|
||||
val pendingChannelsRateLimiter = TestProbe[PendingChannelsRateLimiter.Command]()
|
||||
val plugin = new InterceptOpenChannelPlugin {
|
||||
// @formatter:off
|
||||
override def name: String = "OpenChannelInterceptorPlugin"
|
||||
|
||||
override def openChannelInterceptor: ActorRef[InterceptOpenChannelCommand] = pluginInterceptor.ref
|
||||
// @formatter:on
|
||||
}
|
||||
val pluginParams = TestConstants.Alice.nodeParams.pluginParams :+ plugin
|
||||
val nodeParams = TestConstants.Alice.nodeParams.copy(pluginParams = pluginParams)
|
||||
val nodeParams = TestConstants.Alice.nodeParams
|
||||
.modify(_.pluginParams).usingIf(!test.tags.contains(noPlugin))(_ :+ plugin)
|
||||
.modify(_.channelConf).usingIf(test.tags.contains(acceptStaticRemoteKeyChannelsTag))(_.copy(acceptIncomingStaticRemoteKeyChannels = true))
|
||||
|
||||
val eventListener = TestProbe[ChannelAborted]()
|
||||
@ -75,6 +81,11 @@ class OpenChannelInterceptorSpec extends ScalaTestWithActorTestKit(ConfigFactory
|
||||
|
||||
case class FixtureParam(openChannelInterceptor: ActorRef[OpenChannelInterceptor.Command], peer: TestProbe[Any], pluginInterceptor: TestProbe[InterceptOpenChannelCommand], pendingChannelsRateLimiter: TestProbe[PendingChannelsRateLimiter.Command], peerConnection: TestProbe[Any], eventListener: TestProbe[ChannelAborted], wallet: DummyOnChainWallet)
|
||||
|
||||
private def commitments(isOpener: Boolean = false): Commitments = {
|
||||
val commitments = CommitmentsSpec.makeCommitments(500_000 msat, 400_000 msat, TestConstants.Alice.nodeParams.nodeId, remoteNodeId, announceChannel = false)
|
||||
commitments.copy(params = commitments.params.copy(localParams = commitments.params.localParams.copy(isChannelOpener = isOpener, paysCommitTxFees = isOpener)))
|
||||
}
|
||||
|
||||
test("reject channel open if timeout waiting for plugin to respond") { f =>
|
||||
import f._
|
||||
|
||||
@ -112,6 +123,33 @@ class OpenChannelInterceptorSpec extends ScalaTestWithActorTestKit(ConfigFactory
|
||||
assert(peer.expectMessageType[SpawnChannelNonInitiator].addFunding_opt.contains(addFunding))
|
||||
}
|
||||
|
||||
test("add liquidity if on-the-fly funding is used", Tag(noPlugin)) { f =>
|
||||
import f._
|
||||
|
||||
val features = defaultFeatures.add(Features.SplicePrototype, FeatureSupport.Optional).add(Features.OnTheFlyFunding, FeatureSupport.Optional)
|
||||
val requestFunding = LiquidityAds.RequestFunding(250_000 sat, TestConstants.defaultLiquidityRates.fundingRates.head, LiquidityAds.PaymentDetails.FromChannelBalanceForFutureHtlc(randomBytes32() :: Nil))
|
||||
val open = createOpenDualFundedChannelMessage().copy(
|
||||
channelFlags = ChannelFlags(nonInitiatorPaysCommitFees = true, announceChannel = false),
|
||||
tlvStream = TlvStream(ChannelTlv.RequestFundingTlv(requestFunding))
|
||||
)
|
||||
val openChannelNonInitiator = OpenChannelNonInitiator(remoteNodeId, Right(open), features, features, peerConnection.ref, remoteAddress)
|
||||
openChannelInterceptor ! openChannelNonInitiator
|
||||
pendingChannelsRateLimiter.expectMessageType[AddOrRejectChannel].replyTo ! PendingChannelsRateLimiter.AcceptOpenChannel
|
||||
// We check that all existing channels (if any) are closing before accepting the request.
|
||||
val currentChannels = Seq(
|
||||
Peer.ChannelInfo(TestProbe().ref, SHUTDOWN, DATA_SHUTDOWN(commitments(isOpener = true), Shutdown(randomBytes32(), ByteVector.empty), Shutdown(randomBytes32(), ByteVector.empty), None)),
|
||||
Peer.ChannelInfo(TestProbe().ref, NEGOTIATING, DATA_NEGOTIATING(commitments(), Shutdown(randomBytes32(), ByteVector.empty), Shutdown(randomBytes32(), ByteVector.empty), List(Nil), None)),
|
||||
Peer.ChannelInfo(TestProbe().ref, CLOSING, DATA_CLOSING(commitments(), BlockHeight(0), ByteVector.empty, Nil, ClosingTx(InputInfo(OutPoint(TxId(randomBytes32()), 5), TxOut(100_000 sat, Nil), Nil), Transaction(2, Nil, Nil, 0), None) :: Nil)),
|
||||
Peer.ChannelInfo(TestProbe().ref, WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT, DATA_WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT(commitments(), ChannelReestablish(randomBytes32(), 0, 0, randomKey(), randomKey().publicKey))),
|
||||
)
|
||||
peer.expectMessageType[Peer.GetPeerChannels].replyTo ! Peer.PeerChannels(remoteNodeId, currentChannels)
|
||||
val result = peer.expectMessageType[SpawnChannelNonInitiator]
|
||||
assert(!result.localParams.isChannelOpener)
|
||||
assert(result.localParams.paysCommitTxFees)
|
||||
assert(result.addFunding_opt.map(_.fundingAmount).contains(250_000 sat))
|
||||
assert(result.addFunding_opt.flatMap(_.rates_opt).contains(TestConstants.defaultLiquidityRates))
|
||||
}
|
||||
|
||||
test("continue channel open if no interceptor plugin registered and pending channels rate limiter accepts it") { f =>
|
||||
import f._
|
||||
|
||||
@ -195,6 +233,29 @@ class OpenChannelInterceptorSpec extends ScalaTestWithActorTestKit(ConfigFactory
|
||||
peer.expectMessageType[SpawnChannelNonInitiator]
|
||||
}
|
||||
|
||||
test("reject on-the-fly channel if another channel exists", Tag(noPlugin)) { f =>
|
||||
import f._
|
||||
|
||||
val features = defaultFeatures.add(Features.SplicePrototype, FeatureSupport.Optional).add(Features.OnTheFlyFunding, FeatureSupport.Optional)
|
||||
val requestFunding = LiquidityAds.RequestFunding(250_000 sat, TestConstants.defaultLiquidityRates.fundingRates.head, LiquidityAds.PaymentDetails.FromChannelBalanceForFutureHtlc(randomBytes32() :: Nil))
|
||||
val open = createOpenDualFundedChannelMessage().copy(
|
||||
channelFlags = ChannelFlags(nonInitiatorPaysCommitFees = true, announceChannel = false),
|
||||
tlvStream = TlvStream(ChannelTlv.RequestFundingTlv(requestFunding))
|
||||
)
|
||||
val currentChannel = Seq(
|
||||
Peer.ChannelInfo(TestProbe().ref, NORMAL, ChannelCodecsSpec.normal),
|
||||
Peer.ChannelInfo(TestProbe().ref, OFFLINE, ChannelCodecsSpec.normal),
|
||||
Peer.ChannelInfo(TestProbe().ref, SYNCING, ChannelCodecsSpec.normal),
|
||||
)
|
||||
currentChannel.foreach(channel => {
|
||||
val openChannelNonInitiator = OpenChannelNonInitiator(remoteNodeId, Right(open), features, features, peerConnection.ref, remoteAddress)
|
||||
openChannelInterceptor ! openChannelNonInitiator
|
||||
pendingChannelsRateLimiter.expectMessageType[AddOrRejectChannel].replyTo ! PendingChannelsRateLimiter.AcceptOpenChannel
|
||||
peer.expectMessageType[Peer.GetPeerChannels].replyTo ! Peer.PeerChannels(remoteNodeId, Seq(channel))
|
||||
assert(peer.expectMessageType[OutgoingMessage].msg.asInstanceOf[Error].channelId == open.temporaryChannelId)
|
||||
})
|
||||
}
|
||||
|
||||
test("don't spawn a wumbo channel if wumbo feature isn't enabled", Tag(ChannelStateTestsTags.DisableWumbo)) { f =>
|
||||
import f._
|
||||
|
||||
|
@ -25,7 +25,7 @@ import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
|
||||
import fr.acinq.eclair.blockchain.CurrentBlockHeight
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.io.PeerReadyNotifier.{NotifyWhenPeerReady, PeerUnavailable}
|
||||
import fr.acinq.eclair.{BlockHeight, randomKey}
|
||||
import fr.acinq.eclair.{BlockHeight, FeatureSupport, Features, randomKey}
|
||||
import org.scalatest.Outcome
|
||||
import org.scalatest.funsuite.FixtureAnyFunSuiteLike
|
||||
|
||||
@ -33,6 +33,12 @@ import scala.concurrent.duration.DurationInt
|
||||
|
||||
class PeerReadyNotifierSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("application")) with FixtureAnyFunSuiteLike {
|
||||
|
||||
private val remoteFeatures = Features(
|
||||
Features.StaticRemoteKey -> FeatureSupport.Optional,
|
||||
Features.AnchorOutputsZeroFeeHtlcTx -> FeatureSupport.Optional,
|
||||
Features.RouteBlinding -> FeatureSupport.Optional,
|
||||
).initFeatures()
|
||||
|
||||
case class FixtureParam(remoteNodeId: PublicKey, peerReadyManager: TestProbe[PeerReadyManager.Register], switchboard: TestProbe[Switchboard.GetPeerInfo], peer: TestProbe[Peer.GetPeerChannels], probe: TestProbe[PeerReadyNotifier.Result])
|
||||
|
||||
override def withFixture(test: OneArgTest): Outcome = {
|
||||
@ -56,7 +62,6 @@ class PeerReadyNotifierSpec extends ScalaTestWithActorTestKit(ConfigFactory.load
|
||||
|
||||
val notifier = testKit.spawn(PeerReadyNotifier(remoteNodeId, timeout_opt = Some(Left(10 millis))))
|
||||
notifier ! NotifyWhenPeerReady(probe.ref)
|
||||
peerReadyManager.expectMessageType[PeerReadyManager.Register].replyTo ! PeerReadyManager.Registered(remoteNodeId, otherAttempts = 0)
|
||||
probe.expectMessage(PeerUnavailable(remoteNodeId))
|
||||
}
|
||||
|
||||
@ -84,8 +89,8 @@ class PeerReadyNotifierSpec extends ScalaTestWithActorTestKit(ConfigFactory.load
|
||||
notifier ! NotifyWhenPeerReady(probe.ref)
|
||||
peerReadyManager.expectMessageType[PeerReadyManager.Register].replyTo ! PeerReadyManager.Registered(remoteNodeId, otherAttempts = 0)
|
||||
val request = switchboard.expectMessageType[Switchboard.GetPeerInfo]
|
||||
request.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.CONNECTED, None, Set.empty)
|
||||
probe.expectMessage(PeerReadyNotifier.PeerReady(remoteNodeId, peer.ref.toClassic, Seq.empty))
|
||||
request.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.CONNECTED, Some(remoteFeatures), None, Set.empty)
|
||||
probe.expectMessage(PeerReadyNotifier.PeerReady(remoteNodeId, peer.ref.toClassic, remoteFeatures, Seq.empty))
|
||||
}
|
||||
|
||||
test("peer connected (with channels)") { f =>
|
||||
@ -95,7 +100,7 @@ class PeerReadyNotifierSpec extends ScalaTestWithActorTestKit(ConfigFactory.load
|
||||
notifier ! NotifyWhenPeerReady(probe.ref)
|
||||
peerReadyManager.expectMessageType[PeerReadyManager.Register].replyTo ! PeerReadyManager.Registered(remoteNodeId, otherAttempts = 0)
|
||||
val request1 = switchboard.expectMessageType[Switchboard.GetPeerInfo]
|
||||
request1.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.CONNECTED, None, Set(TestProbe().ref.toClassic, TestProbe().ref.toClassic))
|
||||
request1.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.CONNECTED, Some(remoteFeatures), None, Set(TestProbe().ref.toClassic, TestProbe().ref.toClassic))
|
||||
|
||||
// Channels are not ready yet.
|
||||
val channels2 = Seq(Peer.ChannelInfo(null, SYNCING, null), Peer.ChannelInfo(null, SYNCING, null))
|
||||
@ -113,7 +118,7 @@ class PeerReadyNotifierSpec extends ScalaTestWithActorTestKit(ConfigFactory.load
|
||||
val channels4 = Seq(Peer.ChannelInfo(null, NORMAL, null), Peer.ChannelInfo(null, SHUTDOWN, null))
|
||||
val request4 = peer.expectMessageType[Peer.GetPeerChannels]
|
||||
request4.replyTo ! Peer.PeerChannels(remoteNodeId, channels4)
|
||||
probe.expectMessage(PeerReadyNotifier.PeerReady(remoteNodeId, peer.ref.toClassic, channels4))
|
||||
probe.expectMessage(PeerReadyNotifier.PeerReady(remoteNodeId, peer.ref.toClassic, remoteFeatures, channels4))
|
||||
}
|
||||
|
||||
test("peer connects after initial request") { f =>
|
||||
@ -123,7 +128,7 @@ class PeerReadyNotifierSpec extends ScalaTestWithActorTestKit(ConfigFactory.load
|
||||
notifier ! NotifyWhenPeerReady(probe.ref)
|
||||
peerReadyManager.expectMessageType[PeerReadyManager.Register].replyTo ! PeerReadyManager.Registered(remoteNodeId, otherAttempts = 1)
|
||||
val request1 = switchboard.expectMessageType[Switchboard.GetPeerInfo]
|
||||
request1.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.DISCONNECTED, None, Set(TestProbe().ref.toClassic, TestProbe().ref.toClassic))
|
||||
request1.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.DISCONNECTED, None, None, Set(TestProbe().ref.toClassic, TestProbe().ref.toClassic))
|
||||
peer.expectNoMessage(100 millis)
|
||||
|
||||
// An unrelated peer connects.
|
||||
@ -133,10 +138,10 @@ class PeerReadyNotifierSpec extends ScalaTestWithActorTestKit(ConfigFactory.load
|
||||
// The target peer connects.
|
||||
system.eventStream ! EventStream.Publish(PeerConnected(TestProbe().ref.toClassic, remoteNodeId, null))
|
||||
val request2 = switchboard.expectMessageType[Switchboard.GetPeerInfo]
|
||||
request2.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.CONNECTED, None, Set(TestProbe().ref.toClassic, TestProbe().ref.toClassic))
|
||||
request2.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.CONNECTED, Some(remoteFeatures), None, Set(TestProbe().ref.toClassic, TestProbe().ref.toClassic))
|
||||
val channels = Seq(Peer.ChannelInfo(null, NEGOTIATING, null))
|
||||
peer.expectMessageType[Peer.GetPeerChannels].replyTo ! Peer.PeerChannels(remoteNodeId, channels)
|
||||
probe.expectMessage(PeerReadyNotifier.PeerReady(remoteNodeId, peer.ref.toClassic, channels))
|
||||
probe.expectMessage(PeerReadyNotifier.PeerReady(remoteNodeId, peer.ref.toClassic, remoteFeatures, channels))
|
||||
}
|
||||
|
||||
test("peer connects then disconnects") { f =>
|
||||
@ -152,16 +157,16 @@ class PeerReadyNotifierSpec extends ScalaTestWithActorTestKit(ConfigFactory.load
|
||||
// The target peer connects and instantly disconnects.
|
||||
system.eventStream ! EventStream.Publish(PeerConnected(TestProbe().ref.toClassic, remoteNodeId, null))
|
||||
val request2 = switchboard.expectMessageType[Switchboard.GetPeerInfo]
|
||||
request2.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.DISCONNECTED, None, Set(TestProbe().ref.toClassic))
|
||||
request2.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.DISCONNECTED, None, None, Set(TestProbe().ref.toClassic))
|
||||
peer.expectNoMessage(100 millis)
|
||||
|
||||
// The target peer reconnects and stays connected.
|
||||
system.eventStream ! EventStream.Publish(PeerConnected(TestProbe().ref.toClassic, remoteNodeId, null))
|
||||
val request3 = switchboard.expectMessageType[Switchboard.GetPeerInfo]
|
||||
request3.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.CONNECTED, None, Set(TestProbe().ref.toClassic))
|
||||
request3.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.CONNECTED, Some(remoteFeatures), None, Set(TestProbe().ref.toClassic))
|
||||
val channels = Seq(Peer.ChannelInfo(null, CLOSING, null))
|
||||
peer.expectMessageType[Peer.GetPeerChannels].replyTo ! Peer.PeerChannels(remoteNodeId, channels)
|
||||
probe.expectMessage(PeerReadyNotifier.PeerReady(remoteNodeId, peer.ref.toClassic, channels))
|
||||
probe.expectMessage(PeerReadyNotifier.PeerReady(remoteNodeId, peer.ref.toClassic, remoteFeatures, channels))
|
||||
}
|
||||
|
||||
test("peer connects then disconnects (while waiting for channel states)") { f =>
|
||||
@ -171,22 +176,23 @@ class PeerReadyNotifierSpec extends ScalaTestWithActorTestKit(ConfigFactory.load
|
||||
notifier ! NotifyWhenPeerReady(probe.ref)
|
||||
peerReadyManager.expectMessageType[PeerReadyManager.Register].replyTo ! PeerReadyManager.Registered(remoteNodeId, otherAttempts = 5)
|
||||
val request1 = switchboard.expectMessageType[Switchboard.GetPeerInfo]
|
||||
request1.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.DISCONNECTED, None, Set.empty)
|
||||
request1.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.DISCONNECTED, None, None, Set.empty)
|
||||
peer.expectNoMessage(100 millis)
|
||||
|
||||
// The target peer connects.
|
||||
system.eventStream ! EventStream.Publish(PeerConnected(TestProbe().ref.toClassic, remoteNodeId, null))
|
||||
val request2 = switchboard.expectMessageType[Switchboard.GetPeerInfo]
|
||||
request2.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.CONNECTED, None, Set(TestProbe().ref.toClassic))
|
||||
request2.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.CONNECTED, Some(remoteFeatures), None, Set(TestProbe().ref.toClassic))
|
||||
peer.expectMessageType[Peer.GetPeerChannels]
|
||||
|
||||
// The target peer disconnects, so we wait for them to connect again.
|
||||
system.eventStream ! EventStream.Publish(PeerDisconnected(TestProbe().ref.toClassic, remoteNodeId))
|
||||
val request3 = switchboard.expectMessageType[Switchboard.GetPeerInfo]
|
||||
request3.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.CONNECTED, None, Set(TestProbe().ref.toClassic))
|
||||
val remoteFeatures1 = remoteFeatures.add(Features.OnionMessages, FeatureSupport.Optional)
|
||||
request3.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.CONNECTED, Some(remoteFeatures1), None, Set(TestProbe().ref.toClassic))
|
||||
val channels = Seq(Peer.ChannelInfo(null, NORMAL, null))
|
||||
peer.expectMessageType[Peer.GetPeerChannels].replyTo ! Peer.PeerChannels(remoteNodeId, channels)
|
||||
probe.expectMessage(PeerReadyNotifier.PeerReady(remoteNodeId, peer.ref.toClassic, channels))
|
||||
probe.expectMessage(PeerReadyNotifier.PeerReady(remoteNodeId, peer.ref.toClassic, remoteFeatures1, channels))
|
||||
}
|
||||
|
||||
test("peer connected (duration timeout)") { f =>
|
||||
@ -196,7 +202,7 @@ class PeerReadyNotifierSpec extends ScalaTestWithActorTestKit(ConfigFactory.load
|
||||
notifier ! NotifyWhenPeerReady(probe.ref)
|
||||
peerReadyManager.expectMessageType[PeerReadyManager.Register].replyTo ! PeerReadyManager.Registered(remoteNodeId, otherAttempts = 0)
|
||||
val request = switchboard.expectMessageType[Switchboard.GetPeerInfo]
|
||||
request.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.CONNECTED, None, Set(TestProbe().ref.toClassic))
|
||||
request.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.CONNECTED, Some(Features.empty), None, Set(TestProbe().ref.toClassic))
|
||||
peer.expectMessageType[Peer.GetPeerChannels]
|
||||
probe.expectMessage(PeerUnavailable(remoteNodeId))
|
||||
}
|
||||
@ -208,7 +214,7 @@ class PeerReadyNotifierSpec extends ScalaTestWithActorTestKit(ConfigFactory.load
|
||||
notifier ! NotifyWhenPeerReady(probe.ref)
|
||||
peerReadyManager.expectMessageType[PeerReadyManager.Register].replyTo ! PeerReadyManager.Registered(remoteNodeId, otherAttempts = 2)
|
||||
val request = switchboard.expectMessageType[Switchboard.GetPeerInfo]
|
||||
request.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.CONNECTED, None, Set(TestProbe().ref.toClassic))
|
||||
request.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.CONNECTED, None, None, Set(TestProbe().ref.toClassic))
|
||||
peer.expectMessageType[Peer.GetPeerChannels]
|
||||
system.eventStream ! EventStream.Publish(CurrentBlockHeight(BlockHeight(100)))
|
||||
probe.expectMessage(PeerUnavailable(remoteNodeId))
|
||||
|
@ -109,7 +109,7 @@ class PeerSpec extends FixtureSpec {
|
||||
|
||||
def connect(remoteNodeId: PublicKey, peer: TestFSMRef[Peer.State, Peer.Data, Peer], peerConnection: TestProbe, switchboard: TestProbe, channels: Set[PersistentChannelData] = Set.empty, remoteInit: protocol.Init = protocol.Init(Bob.nodeParams.features.initFeatures()))(implicit system: ActorSystem): Unit = {
|
||||
// let's simulate a connection
|
||||
switchboard.send(peer, Peer.Init(channels))
|
||||
switchboard.send(peer, Peer.Init(channels, Map.empty))
|
||||
val localInit = protocol.Init(peer.underlyingActor.nodeParams.features.initFeatures())
|
||||
switchboard.send(peer, PeerConnection.ConnectionReady(peerConnection.ref, remoteNodeId, fakeIPAddress, outgoing = true, localInit, remoteInit))
|
||||
peerConnection.expectMsgType[RecommendedFeerates]
|
||||
@ -140,7 +140,7 @@ class PeerSpec extends FixtureSpec {
|
||||
import f._
|
||||
|
||||
val probe = TestProbe()
|
||||
probe.send(peer, Peer.Init(Set.empty))
|
||||
probe.send(peer, Peer.Init(Set.empty, Map.empty))
|
||||
probe.send(peer, Peer.Connect(remoteNodeId, address_opt = None, probe.ref, isPersistent = true))
|
||||
probe.expectMsg(PeerConnection.ConnectionResult.NoAddressFound)
|
||||
}
|
||||
@ -167,7 +167,7 @@ class PeerSpec extends FixtureSpec {
|
||||
val mockAddress_opt = NodeAddress.fromParts(serverAddress.getHostName, serverAddress.getPort).toOption
|
||||
|
||||
val probe = TestProbe()
|
||||
probe.send(peer, Peer.Init(Set.empty))
|
||||
probe.send(peer, Peer.Init(Set.empty, Map.empty))
|
||||
// we have auto-reconnect=false so we need to manually tell the peer to reconnect
|
||||
probe.send(peer, Peer.Connect(remoteNodeId, mockAddress_opt, probe.ref, isPersistent = true))
|
||||
|
||||
@ -188,7 +188,7 @@ class PeerSpec extends FixtureSpec {
|
||||
assert(invalidDnsHostname_opt.get == DnsHostname("eclair.invalid", 9735))
|
||||
|
||||
val probe = TestProbe()
|
||||
probe.send(peer, Peer.Init(Set.empty))
|
||||
probe.send(peer, Peer.Init(Set.empty, Map.empty))
|
||||
probe.send(peer, Peer.Connect(remoteNodeId, invalidDnsHostname_opt, probe.ref, isPersistent = true))
|
||||
probe.expectMsgType[PeerConnection.ConnectionResult.ConnectionFailed]
|
||||
}
|
||||
@ -207,7 +207,7 @@ class PeerSpec extends FixtureSpec {
|
||||
nodeParams.db.network.addNode(ann)
|
||||
|
||||
val probe = TestProbe()
|
||||
probe.send(peer, Peer.Init(Set(ChannelCodecsSpec.normal)))
|
||||
probe.send(peer, Peer.Init(Set(ChannelCodecsSpec.normal), Map.empty))
|
||||
|
||||
// assert our mock server got an incoming connection (the client was spawned with the address from node_announcement)
|
||||
eventually {
|
||||
@ -256,7 +256,7 @@ class PeerSpec extends FixtureSpec {
|
||||
import f._
|
||||
|
||||
val probe = TestProbe()
|
||||
switchboard.send(peer, Peer.Init(Set.empty))
|
||||
switchboard.send(peer, Peer.Init(Set.empty, Map.empty))
|
||||
|
||||
eventually {
|
||||
probe.send(peer, Peer.GetPeerInfo(None))
|
||||
@ -313,7 +313,7 @@ class PeerSpec extends FixtureSpec {
|
||||
monitor.expectMsg(FSM.CurrentState(reconnectionTask, ReconnectionTask.IDLE))
|
||||
|
||||
val probe = TestProbe()
|
||||
probe.send(peer, Peer.Init(Set(ChannelCodecsSpec.normal)))
|
||||
probe.send(peer, Peer.Init(Set(ChannelCodecsSpec.normal), Map.empty))
|
||||
|
||||
// the reconnection task will wait a little...
|
||||
monitor.expectMsg(FSM.Transition(reconnectionTask, ReconnectionTask.IDLE, ReconnectionTask.WAITING))
|
||||
@ -665,7 +665,7 @@ class PeerSpec extends FixtureSpec {
|
||||
import f._
|
||||
val probe = TestProbe()
|
||||
probe.watch(peer)
|
||||
switchboard.send(peer, Peer.Init(Set.empty))
|
||||
switchboard.send(peer, Peer.Init(Set.empty, Map.empty))
|
||||
eventually {
|
||||
probe.send(peer, Peer.GetPeerInfo(None))
|
||||
assert(probe.expectMsgType[Peer.PeerInfo].state == Peer.DISCONNECTED)
|
||||
|
@ -6,15 +6,17 @@ import akka.testkit.{TestActorRef, TestProbe}
|
||||
import fr.acinq.bitcoin.scalacompat.ByteVector64
|
||||
import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
|
||||
import fr.acinq.eclair.TestConstants._
|
||||
import fr.acinq.eclair.channel.{ChannelIdAssigned, PersistentChannelData}
|
||||
import fr.acinq.eclair.channel.{ChannelIdAssigned, PersistentChannelData, Upstream}
|
||||
import fr.acinq.eclair.io.Peer.PeerNotFound
|
||||
import fr.acinq.eclair.io.Switchboard._
|
||||
import fr.acinq.eclair.payment.relay.{OnTheFlyFunding, OnTheFlyFundingSpec}
|
||||
import fr.acinq.eclair.wire.internal.channel.ChannelCodecsSpec
|
||||
import fr.acinq.eclair.wire.protocol._
|
||||
import fr.acinq.eclair.{Features, InitFeature, NodeParams, TestKitBaseClass, TimestampSecondLong, randomBytes32, randomKey}
|
||||
import fr.acinq.eclair.{CltvExpiry, Features, InitFeature, MilliSatoshiLong, NodeParams, TestKitBaseClass, TimestampSecondLong, randomBytes32, randomKey}
|
||||
import org.scalatest.funsuite.AnyFunSuiteLike
|
||||
import scodec.bits._
|
||||
|
||||
import java.util.UUID
|
||||
import scala.concurrent.duration.DurationInt
|
||||
|
||||
class SwitchboardSpec extends TestKitBaseClass with AnyFunSuiteLike {
|
||||
@ -24,13 +26,47 @@ class SwitchboardSpec extends TestKitBaseClass with AnyFunSuiteLike {
|
||||
test("on initialization create peers") {
|
||||
val nodeParams = Alice.nodeParams
|
||||
val (probe, peer) = (TestProbe(), TestProbe())
|
||||
val remoteNodeId = ChannelCodecsSpec.normal.commitments.remoteNodeId
|
||||
val remoteNodeId = ChannelCodecsSpec.normal.remoteNodeId
|
||||
// If we have a channel with that remote peer, we will automatically reconnect.
|
||||
|
||||
val switchboard = TestActorRef(new Switchboard(nodeParams, FakePeerFactory(probe, peer)))
|
||||
switchboard ! Switchboard.Init(List(ChannelCodecsSpec.normal))
|
||||
probe.expectMsg(remoteNodeId)
|
||||
peer.expectMsg(Peer.Init(Set(ChannelCodecsSpec.normal)))
|
||||
peer.expectMsg(Peer.Init(Set(ChannelCodecsSpec.normal), Map.empty))
|
||||
}
|
||||
|
||||
test("on initialization create peers with pending on-the-fly funding proposals") {
|
||||
val nodeParams = Alice.nodeParams
|
||||
|
||||
// We have a channel with one of our peer, and a pending on-the-fly funding with them as well.
|
||||
val channel = ChannelCodecsSpec.normal
|
||||
val remoteNodeId1 = channel.remoteNodeId
|
||||
val paymentHash1 = randomBytes32()
|
||||
val pendingOnTheFly1 = OnTheFlyFunding.Pending(
|
||||
proposed = Seq(OnTheFlyFunding.Proposal(OnTheFlyFundingSpec.createWillAdd(10_000_000 msat, paymentHash1, CltvExpiry(600)), Upstream.Local(UUID.randomUUID()))),
|
||||
status = OnTheFlyFundingSpec.createStatus()
|
||||
)
|
||||
nodeParams.db.liquidity.addPendingOnTheFlyFunding(remoteNodeId1, pendingOnTheFly1)
|
||||
|
||||
// We don't have channels yet with another of our peers, but we have a pending on-the-fly funding proposal.
|
||||
val remoteNodeId2 = randomKey().publicKey
|
||||
val paymentHash2 = randomBytes32()
|
||||
val pendingOnTheFly2 = OnTheFlyFunding.Pending(
|
||||
proposed = Seq(OnTheFlyFunding.Proposal(OnTheFlyFundingSpec.createWillAdd(5_000_000 msat, paymentHash2, CltvExpiry(600)), Upstream.Local(UUID.randomUUID()))),
|
||||
status = OnTheFlyFundingSpec.createStatus()
|
||||
)
|
||||
nodeParams.db.liquidity.addPendingOnTheFlyFunding(remoteNodeId2, pendingOnTheFly2)
|
||||
|
||||
val (probe, peer) = (TestProbe(), TestProbe())
|
||||
val switchboard = TestActorRef(new Switchboard(nodeParams, FakePeerFactory(probe, peer)))
|
||||
switchboard ! Switchboard.Init(List(channel))
|
||||
probe.expectMsgAllOf(remoteNodeId1, remoteNodeId2)
|
||||
probe.expectNoMessage(100 millis)
|
||||
peer.expectMsgAllOf(
|
||||
Peer.Init(Set(channel), Map(paymentHash1 -> pendingOnTheFly1)),
|
||||
Peer.Init(Set.empty, Map(paymentHash2 -> pendingOnTheFly2)),
|
||||
)
|
||||
peer.expectNoMessage(100 millis)
|
||||
}
|
||||
|
||||
test("when connecting to a new peer forward Peer.Connect to it") {
|
||||
@ -44,7 +80,7 @@ class SwitchboardSpec extends TestKitBaseClass with AnyFunSuiteLike {
|
||||
switchboard ! Switchboard.Init(Nil)
|
||||
probe.send(switchboard, Peer.Connect(remoteNodeId, None, probe.ref, isPersistent = true))
|
||||
probe.expectMsg(remoteNodeId)
|
||||
peer.expectMsg(Peer.Init(Set.empty))
|
||||
peer.expectMsg(Peer.Init(Set.empty, Map.empty))
|
||||
val connect = peer.expectMsgType[Peer.Connect]
|
||||
assert(connect.nodeId == remoteNodeId)
|
||||
assert(connect.address_opt.isEmpty)
|
||||
@ -58,7 +94,7 @@ class SwitchboardSpec extends TestKitBaseClass with AnyFunSuiteLike {
|
||||
switchboard ! Switchboard.Init(Nil)
|
||||
probe.send(switchboard, Peer.Connect(remoteNodeId, None, probe.ref, isPersistent = true))
|
||||
probe.expectMsg(remoteNodeId)
|
||||
peer.expectMsg(Peer.Init(Set.empty))
|
||||
peer.expectMsg(Peer.Init(Set.empty, Map.empty))
|
||||
peer.expectMsgType[Peer.Connect]
|
||||
|
||||
val unknownNodeId = randomKey().publicKey
|
||||
|
@ -110,7 +110,7 @@ class JsonSerializersSpec extends TestKitBaseClass with AnyFunSuiteLike with Mat
|
||||
}
|
||||
|
||||
test("PeerInfo serialization") {
|
||||
val peerInfo = PeerInfo(ActorRef.noSender, PublicKey(hex"0270685ca81a8e4d4d01beec5781f4cc924684072ae52c507f8ebe9daf0caaab7b"), Peer.CONNECTED, None, Set(ActorRef.noSender))
|
||||
val peerInfo = PeerInfo(ActorRef.noSender, PublicKey(hex"0270685ca81a8e4d4d01beec5781f4cc924684072ae52c507f8ebe9daf0caaab7b"), Peer.CONNECTED, None, None, Set(ActorRef.noSender))
|
||||
val expected = """{"nodeId":"0270685ca81a8e4d4d01beec5781f4cc924684072ae52c507f8ebe9daf0caaab7b","state":"CONNECTED","channels":1}"""
|
||||
JsonSerializers.serialization.write(peerInfo)(JsonSerializers.formats) shouldBe expected
|
||||
}
|
||||
@ -185,6 +185,7 @@ class JsonSerializersSpec extends TestKitBaseClass with AnyFunSuiteLike with Mat
|
||||
| "initFeatures": { "activated": {}, "unknown": [] }
|
||||
| },
|
||||
| "channelFlags": {
|
||||
| "nonInitiatorPaysCommitFees": false,
|
||||
| "announceChannel": true
|
||||
| }
|
||||
| },
|
||||
@ -255,6 +256,7 @@ class JsonSerializersSpec extends TestKitBaseClass with AnyFunSuiteLike with Mat
|
||||
),
|
||||
blinding_opt = None,
|
||||
confidence = 0.7,
|
||||
fundingFee_opt = None,
|
||||
)
|
||||
|
||||
val expectedIn = """{"direction":"IN","id":926,"amountMsat":12365,"paymentHash":"9fcd45bbaa09c60c991ac0425704163c3f3d2d683c789fa409455b9c97792692","cltvExpiry":621500}"""
|
||||
|
@ -96,7 +96,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
||||
}
|
||||
|
||||
def createBlindedPacket(amount: MilliSatoshi, paymentHash: ByteVector32, expiry: CltvExpiry, finalExpiry: CltvExpiry, pathId: ByteVector): IncomingPaymentPacket.FinalPacket = {
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 0, amount, paymentHash, expiry, TestConstants.emptyOnionPacket, Some(randomKey().publicKey), 1.0)
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 0, amount, paymentHash, expiry, TestConstants.emptyOnionPacket, Some(randomKey().publicKey), 1.0, None)
|
||||
val payload = FinalPayload.Blinded(TlvStream(AmountToForward(amount), TotalAmount(amount), OutgoingCltv(finalExpiry), EncryptedRecipientData(hex"deadbeef")), TlvStream(PathId(pathId), PaymentConstraints(CltvExpiry(500_000), 1 msat)))
|
||||
IncomingPaymentPacket.FinalPacket(add, payload)
|
||||
}
|
||||
@ -115,7 +115,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
||||
assert(!incoming.get.invoice.isExpired())
|
||||
assert(Crypto.sha256(incoming.get.paymentPreimage) == invoice.paymentHash)
|
||||
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 1, amountMsat, invoice.paymentHash, defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 1, amountMsat, invoice.paymentHash, defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
sender.send(handlerWithoutMpp, IncomingPaymentPacket.FinalPacket(add, FinalPayload.Standard.createPayload(add.amountMsat, add.amountMsat, add.cltvExpiry, invoice.paymentSecret, invoice.paymentMetadata)))
|
||||
assert(register.expectMsgType[Register.Forward[CMD_FULFILL_HTLC]].message.id == add.id)
|
||||
|
||||
@ -131,7 +131,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
||||
sender.send(handlerWithoutMpp, ReceiveStandardPayment(sender.ref, Some(50_000 msat), Left("1 coffee with extra fees and expiry")))
|
||||
val invoice = sender.expectMsgType[Bolt11Invoice]
|
||||
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 1, 75_000 msat, invoice.paymentHash, defaultExpiry + CltvExpiryDelta(12), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 1, 75_000 msat, invoice.paymentHash, defaultExpiry + CltvExpiryDelta(12), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
sender.send(handlerWithoutMpp, IncomingPaymentPacket.FinalPacket(add, FinalPayload.Standard.createPayload(70_000 msat, 70_000 msat, defaultExpiry, invoice.paymentSecret, invoice.paymentMetadata)))
|
||||
assert(register.expectMsgType[Register.Forward[CMD_FULFILL_HTLC]].message.id == add.id)
|
||||
|
||||
@ -149,7 +149,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
||||
assert(invoice.features.hasFeature(BasicMultiPartPayment))
|
||||
assert(nodeParams.db.payments.getIncomingPayment(invoice.paymentHash).get.status == IncomingPaymentStatus.Pending)
|
||||
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 2, amountMsat, invoice.paymentHash, defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 2, amountMsat, invoice.paymentHash, defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
sender.send(handlerWithMpp, IncomingPaymentPacket.FinalPacket(add, FinalPayload.Standard.createPayload(add.amountMsat, add.amountMsat, add.cltvExpiry, invoice.paymentSecret, invoice.paymentMetadata)))
|
||||
assert(register.expectMsgType[Register.Forward[CMD_FULFILL_HTLC]].message.id == add.id)
|
||||
|
||||
@ -196,7 +196,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
||||
val invoice = sender.expectMsgType[Bolt11Invoice]
|
||||
assert(nodeParams.db.payments.getIncomingPayment(invoice.paymentHash).get.status == IncomingPaymentStatus.Pending)
|
||||
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 0, amountMsat, invoice.paymentHash, CltvExpiryDelta(3).toCltvExpiry(nodeParams.currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 0, amountMsat, invoice.paymentHash, CltvExpiryDelta(3).toCltvExpiry(nodeParams.currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
sender.send(handlerWithMpp, IncomingPaymentPacket.FinalPacket(add, FinalPayload.Standard.createPayload(add.amountMsat, add.amountMsat, add.cltvExpiry, invoice.paymentSecret, invoice.paymentMetadata)))
|
||||
val cmd = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]].message
|
||||
assert(cmd.reason == Right(IncorrectOrUnknownPaymentDetails(amountMsat, nodeParams.currentBlockHeight)))
|
||||
@ -356,7 +356,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
||||
assert(!invoice.features.hasFeature(BasicMultiPartPayment))
|
||||
assert(invoice.isExpired())
|
||||
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 0, 1000 msat, invoice.paymentHash, defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 0, 1000 msat, invoice.paymentHash, defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
sender.send(handlerWithoutMpp, IncomingPaymentPacket.FinalPacket(add, FinalPayload.Standard.createPayload(add.amountMsat, add.amountMsat, add.cltvExpiry, invoice.paymentSecret, invoice.paymentMetadata)))
|
||||
register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]]
|
||||
val Some(incoming) = nodeParams.db.payments.getIncomingPayment(invoice.paymentHash)
|
||||
@ -371,7 +371,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
||||
assert(invoice.features.hasFeature(BasicMultiPartPayment))
|
||||
assert(invoice.isExpired())
|
||||
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 0, 800 msat, invoice.paymentHash, defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 0, 800 msat, invoice.paymentHash, defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
sender.send(handlerWithMpp, IncomingPaymentPacket.FinalPacket(add, FinalPayload.Standard.createPayload(add.amountMsat, 1000 msat, add.cltvExpiry, invoice.paymentSecret, invoice.paymentMetadata)))
|
||||
val cmd = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]].message
|
||||
assert(cmd.reason == Right(IncorrectOrUnknownPaymentDetails(1000 msat, nodeParams.currentBlockHeight)))
|
||||
@ -386,7 +386,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
||||
val invoice = sender.expectMsgType[Bolt11Invoice]
|
||||
assert(!invoice.features.hasFeature(BasicMultiPartPayment))
|
||||
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 0, 800 msat, invoice.paymentHash, defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 0, 800 msat, invoice.paymentHash, defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
sender.send(handlerWithoutMpp, IncomingPaymentPacket.FinalPacket(add, FinalPayload.Standard.createPayload(add.amountMsat, 1000 msat, add.cltvExpiry, invoice.paymentSecret, invoice.paymentMetadata)))
|
||||
val cmd = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]].message
|
||||
assert(cmd.reason == Right(IncorrectOrUnknownPaymentDetails(1000 msat, nodeParams.currentBlockHeight)))
|
||||
@ -401,7 +401,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
||||
assert(invoice.features.hasFeature(BasicMultiPartPayment))
|
||||
|
||||
val lowCltvExpiry = nodeParams.channelConf.fulfillSafetyBeforeTimeout.toCltvExpiry(nodeParams.currentBlockHeight)
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 0, 800 msat, invoice.paymentHash, lowCltvExpiry, TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 0, 800 msat, invoice.paymentHash, lowCltvExpiry, TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
sender.send(handlerWithMpp, IncomingPaymentPacket.FinalPacket(add, FinalPayload.Standard.createPayload(add.amountMsat, 1000 msat, add.cltvExpiry, invoice.paymentSecret, invoice.paymentMetadata)))
|
||||
val cmd = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]].message
|
||||
assert(cmd.reason == Right(IncorrectOrUnknownPaymentDetails(1000 msat, nodeParams.currentBlockHeight)))
|
||||
@ -415,7 +415,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
||||
val invoice = sender.expectMsgType[Bolt11Invoice]
|
||||
assert(invoice.features.hasFeature(BasicMultiPartPayment))
|
||||
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 0, 800 msat, invoice.paymentHash.reverse, defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 0, 800 msat, invoice.paymentHash.reverse, defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
sender.send(handlerWithMpp, IncomingPaymentPacket.FinalPacket(add, FinalPayload.Standard.createPayload(add.amountMsat, 1000 msat, add.cltvExpiry, invoice.paymentSecret, invoice.paymentMetadata)))
|
||||
val cmd = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]].message
|
||||
assert(cmd.reason == Right(IncorrectOrUnknownPaymentDetails(1000 msat, nodeParams.currentBlockHeight)))
|
||||
@ -429,7 +429,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
||||
val invoice = sender.expectMsgType[Bolt11Invoice]
|
||||
assert(invoice.features.hasFeature(BasicMultiPartPayment))
|
||||
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 0, 800 msat, invoice.paymentHash, defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 0, 800 msat, invoice.paymentHash, defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
sender.send(handlerWithMpp, IncomingPaymentPacket.FinalPacket(add, FinalPayload.Standard.createPayload(add.amountMsat, 999 msat, add.cltvExpiry, invoice.paymentSecret, invoice.paymentMetadata)))
|
||||
val cmd = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]].message
|
||||
assert(cmd.reason == Right(IncorrectOrUnknownPaymentDetails(999 msat, nodeParams.currentBlockHeight)))
|
||||
@ -443,7 +443,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
||||
val invoice = sender.expectMsgType[Bolt11Invoice]
|
||||
assert(invoice.features.hasFeature(BasicMultiPartPayment))
|
||||
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 0, 800 msat, invoice.paymentHash, defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 0, 800 msat, invoice.paymentHash, defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
sender.send(handlerWithMpp, IncomingPaymentPacket.FinalPacket(add, FinalPayload.Standard.createPayload(add.amountMsat, 2001 msat, add.cltvExpiry, invoice.paymentSecret, invoice.paymentMetadata)))
|
||||
val cmd = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]].message
|
||||
assert(cmd.reason == Right(IncorrectOrUnknownPaymentDetails(2001 msat, nodeParams.currentBlockHeight)))
|
||||
@ -458,7 +458,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
||||
assert(invoice.features.hasFeature(BasicMultiPartPayment))
|
||||
|
||||
// Invalid payment secret.
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 0, 800 msat, invoice.paymentHash, defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 0, 800 msat, invoice.paymentHash, defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
sender.send(handlerWithMpp, IncomingPaymentPacket.FinalPacket(add, FinalPayload.Standard.createPayload(add.amountMsat, 1000 msat, add.cltvExpiry, invoice.paymentSecret.reverse, invoice.paymentMetadata)))
|
||||
val cmd = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]].message
|
||||
assert(cmd.reason == Right(IncorrectOrUnknownPaymentDetails(1000 msat, nodeParams.currentBlockHeight)))
|
||||
@ -490,7 +490,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
||||
sender.send(handlerWithRouteBlinding, ReceiveOfferPayment(sender.ref, nodeKey, invoiceReq, createEmptyReceivingRoute(), TestProbe().ref, randomBytes32(), randomBytes32()))
|
||||
val invoice = sender.expectMsgType[CreateInvoiceActor.InvoiceCreated].invoice
|
||||
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 0, 5000 msat, invoice.paymentHash, defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 0, 5000 msat, invoice.paymentHash, defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
sender.send(handlerWithMpp, IncomingPaymentPacket.FinalPacket(add, FinalPayload.Standard.createPayload(add.amountMsat, add.amountMsat, add.cltvExpiry, randomBytes32(), None)))
|
||||
val cmd = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]].message
|
||||
assert(cmd.reason == Right(IncorrectOrUnknownPaymentDetails(5000 msat, nodeParams.currentBlockHeight)))
|
||||
@ -571,13 +571,13 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
||||
// Partial payment missing additional parts.
|
||||
f.sender.send(handler, ReceiveStandardPayment(f.sender.ref, Some(1000 msat), Left("1 slow coffee")))
|
||||
val pr1 = f.sender.expectMsgType[Bolt11Invoice]
|
||||
val add1 = UpdateAddHtlc(ByteVector32.One, 0, 800 msat, pr1.paymentHash, f.defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val add1 = UpdateAddHtlc(ByteVector32.One, 0, 800 msat, pr1.paymentHash, f.defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
f.sender.send(handler, IncomingPaymentPacket.FinalPacket(add1, FinalPayload.Standard.createPayload(add1.amountMsat, 1000 msat, add1.cltvExpiry, pr1.paymentSecret, pr1.paymentMetadata)))
|
||||
|
||||
// Partial payment exceeding the invoice amount, but incomplete because it promises to overpay.
|
||||
f.sender.send(handler, ReceiveStandardPayment(f.sender.ref, Some(1500 msat), Left("1 slow latte")))
|
||||
val pr2 = f.sender.expectMsgType[Bolt11Invoice]
|
||||
val add2 = UpdateAddHtlc(ByteVector32.One, 1, 1600 msat, pr2.paymentHash, f.defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val add2 = UpdateAddHtlc(ByteVector32.One, 1, 1600 msat, pr2.paymentHash, f.defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
f.sender.send(handler, IncomingPaymentPacket.FinalPacket(add2, FinalPayload.Standard.createPayload(add2.amountMsat, 2000 msat, add2.cltvExpiry, pr2.paymentSecret, pr2.paymentMetadata)))
|
||||
|
||||
awaitCond {
|
||||
@ -596,7 +596,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
||||
})
|
||||
|
||||
// Extraneous HTLCs should be failed.
|
||||
f.sender.send(handler, MultiPartPaymentFSM.ExtraPaymentReceived(pr1.paymentHash, HtlcPart(1000 msat, UpdateAddHtlc(ByteVector32.One, 42, 200 msat, pr1.paymentHash, add1.cltvExpiry, add1.onionRoutingPacket, None, 1.0)), Some(PaymentTimeout())))
|
||||
f.sender.send(handler, MultiPartPaymentFSM.ExtraPaymentReceived(pr1.paymentHash, HtlcPart(1000 msat, UpdateAddHtlc(ByteVector32.One, 42, 200 msat, pr1.paymentHash, add1.cltvExpiry, add1.onionRoutingPacket, None, 1.0, None)), Some(PaymentTimeout())))
|
||||
f.register.expectMsg(Register.Forward(null, ByteVector32.One, CMD_FAIL_HTLC(42, Right(PaymentTimeout()), commit = true)))
|
||||
|
||||
// The payment should still be pending in DB.
|
||||
@ -612,10 +612,10 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
||||
f.sender.send(handler, ReceiveStandardPayment(f.sender.ref, Some(1000 msat), Left("1 fast coffee"), paymentPreimage_opt = Some(preimage)))
|
||||
val invoice = f.sender.expectMsgType[Bolt11Invoice]
|
||||
|
||||
val add1 = UpdateAddHtlc(ByteVector32.One, 0, 800 msat, invoice.paymentHash, f.defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val add1 = UpdateAddHtlc(ByteVector32.One, 0, 800 msat, invoice.paymentHash, f.defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
f.sender.send(handler, IncomingPaymentPacket.FinalPacket(add1, FinalPayload.Standard.createPayload(add1.amountMsat, 1000 msat, add1.cltvExpiry, invoice.paymentSecret, invoice.paymentMetadata)))
|
||||
// Invalid payment secret -> should be rejected.
|
||||
val add2 = UpdateAddHtlc(ByteVector32.Zeroes, 42, 200 msat, invoice.paymentHash, f.defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val add2 = UpdateAddHtlc(ByteVector32.Zeroes, 42, 200 msat, invoice.paymentHash, f.defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
f.sender.send(handler, IncomingPaymentPacket.FinalPacket(add2, FinalPayload.Standard.createPayload(add2.amountMsat, 1000 msat, add2.cltvExpiry, invoice.paymentSecret.reverse, invoice.paymentMetadata)))
|
||||
val add3 = add2.copy(id = 43)
|
||||
f.sender.send(handler, IncomingPaymentPacket.FinalPacket(add3, FinalPayload.Standard.createPayload(add3.amountMsat, 1000 msat, add3.cltvExpiry, invoice.paymentSecret, invoice.paymentMetadata)))
|
||||
@ -637,7 +637,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
||||
})
|
||||
|
||||
// Extraneous HTLCs should be fulfilled.
|
||||
f.sender.send(handler, MultiPartPaymentFSM.ExtraPaymentReceived(invoice.paymentHash, HtlcPart(1000 msat, UpdateAddHtlc(ByteVector32.One, 44, 200 msat, invoice.paymentHash, add1.cltvExpiry, add1.onionRoutingPacket, None, 1.0)), None))
|
||||
f.sender.send(handler, MultiPartPaymentFSM.ExtraPaymentReceived(invoice.paymentHash, HtlcPart(1000 msat, UpdateAddHtlc(ByteVector32.One, 44, 200 msat, invoice.paymentHash, add1.cltvExpiry, add1.onionRoutingPacket, None, 1.0, None)), None))
|
||||
f.register.expectMsg(Register.Forward(null, ByteVector32.One, CMD_FULFILL_HTLC(44, preimage, commit = true)))
|
||||
assert(f.eventListener.expectMsgType[PaymentReceived].amount == 200.msat)
|
||||
val received2 = nodeParams.db.payments.getIncomingPayment(invoice.paymentHash)
|
||||
@ -655,9 +655,9 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
||||
f.sender.send(handler, ReceiveStandardPayment(f.sender.ref, Some(1000 msat), Left("1 coffee with tip please"), paymentPreimage_opt = Some(preimage)))
|
||||
val invoice = f.sender.expectMsgType[Bolt11Invoice]
|
||||
|
||||
val add1 = UpdateAddHtlc(randomBytes32(), 0, 1100 msat, invoice.paymentHash, f.defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val add1 = UpdateAddHtlc(randomBytes32(), 0, 1100 msat, invoice.paymentHash, f.defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
f.sender.send(handler, IncomingPaymentPacket.FinalPacket(add1, FinalPayload.Standard.createPayload(add1.amountMsat, 1500 msat, add1.cltvExpiry, invoice.paymentSecret, invoice.paymentMetadata)))
|
||||
val add2 = UpdateAddHtlc(randomBytes32(), 1, 500 msat, invoice.paymentHash, f.defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val add2 = UpdateAddHtlc(randomBytes32(), 1, 500 msat, invoice.paymentHash, f.defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
f.sender.send(handler, IncomingPaymentPacket.FinalPacket(add2, FinalPayload.Standard.createPayload(add2.amountMsat, 1500 msat, add2.cltvExpiry, invoice.paymentSecret, invoice.paymentMetadata)))
|
||||
|
||||
f.register.expectMsgAllOf(
|
||||
@ -682,7 +682,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
||||
assert(invoice.features.hasFeature(BasicMultiPartPayment))
|
||||
assert(invoice.paymentHash == Crypto.sha256(preimage))
|
||||
|
||||
val add1 = UpdateAddHtlc(ByteVector32.One, 0, 800 msat, invoice.paymentHash, f.defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val add1 = UpdateAddHtlc(ByteVector32.One, 0, 800 msat, invoice.paymentHash, f.defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
f.sender.send(handler, IncomingPaymentPacket.FinalPacket(add1, FinalPayload.Standard.createPayload(add1.amountMsat, 1000 msat, add1.cltvExpiry, invoice.paymentSecret, invoice.paymentMetadata)))
|
||||
f.register.expectMsg(Register.Forward(null, ByteVector32.One, CMD_FAIL_HTLC(0, Right(PaymentTimeout()), commit = true)))
|
||||
awaitCond({
|
||||
@ -690,9 +690,9 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
||||
f.sender.expectMsgType[PendingPayments].paymentHashes.isEmpty
|
||||
})
|
||||
|
||||
val add2 = UpdateAddHtlc(ByteVector32.One, 2, 300 msat, invoice.paymentHash, f.defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val add2 = UpdateAddHtlc(ByteVector32.One, 2, 300 msat, invoice.paymentHash, f.defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
f.sender.send(handler, IncomingPaymentPacket.FinalPacket(add2, FinalPayload.Standard.createPayload(add2.amountMsat, 1000 msat, add2.cltvExpiry, invoice.paymentSecret, invoice.paymentMetadata)))
|
||||
val add3 = UpdateAddHtlc(ByteVector32.Zeroes, 5, 700 msat, invoice.paymentHash, f.defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val add3 = UpdateAddHtlc(ByteVector32.Zeroes, 5, 700 msat, invoice.paymentHash, f.defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
f.sender.send(handler, IncomingPaymentPacket.FinalPacket(add3, FinalPayload.Standard.createPayload(add3.amountMsat, 1000 msat, add3.cltvExpiry, invoice.paymentSecret, invoice.paymentMetadata)))
|
||||
|
||||
// the fulfill are not necessarily in the same order as the commands
|
||||
@ -724,7 +724,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
||||
|
||||
assert(nodeParams.db.payments.getIncomingPayment(paymentHash).isEmpty)
|
||||
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 0, amountMsat, paymentHash, defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 0, amountMsat, paymentHash, defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
sender.send(handlerWithKeySend, IncomingPaymentPacket.FinalPacket(add, payload))
|
||||
register.expectMsgType[Register.Forward[CMD_FULFILL_HTLC]]
|
||||
|
||||
@ -745,7 +745,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
||||
|
||||
assert(nodeParams.db.payments.getIncomingPayment(paymentHash).isEmpty)
|
||||
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 0, amountMsat, paymentHash, defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 0, amountMsat, paymentHash, defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
sender.send(handlerWithKeySend, IncomingPaymentPacket.FinalPacket(add, payload))
|
||||
register.expectMsgType[Register.Forward[CMD_FULFILL_HTLC]]
|
||||
|
||||
@ -767,7 +767,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
||||
|
||||
assert(nodeParams.db.payments.getIncomingPayment(paymentHash).isEmpty)
|
||||
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 0, amountMsat, paymentHash, defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 0, amountMsat, paymentHash, defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
sender.send(handlerWithMpp, IncomingPaymentPacket.FinalPacket(add, payload))
|
||||
|
||||
f.register.expectMsg(Register.Forward(null, add.channelId, CMD_FAIL_HTLC(add.id, Right(IncorrectOrUnknownPaymentDetails(42000 msat, nodeParams.currentBlockHeight)), commit = true)))
|
||||
@ -781,7 +781,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
||||
val paymentSecret = randomBytes32()
|
||||
assert(nodeParams.db.payments.getIncomingPayment(paymentHash).isEmpty)
|
||||
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 0, 1000 msat, paymentHash, defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 0, 1000 msat, paymentHash, defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
sender.send(handlerWithoutMpp, IncomingPaymentPacket.FinalPacket(add, FinalPayload.Standard.createPayload(add.amountMsat, add.amountMsat, add.cltvExpiry, paymentSecret, None)))
|
||||
val cmd = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]].message
|
||||
assert(cmd.id == add.id)
|
||||
@ -795,7 +795,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
||||
val paymentSecret = randomBytes32()
|
||||
assert(nodeParams.db.payments.getIncomingPayment(paymentHash).isEmpty)
|
||||
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 0, 800 msat, paymentHash, defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 0, 800 msat, paymentHash, defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
sender.send(handlerWithMpp, IncomingPaymentPacket.FinalPacket(add, FinalPayload.Standard.createPayload(add.amountMsat, 1000 msat, add.cltvExpiry, paymentSecret, Some(hex"012345"))))
|
||||
val cmd = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]].message
|
||||
assert(cmd.id == add.id)
|
||||
@ -809,7 +809,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
||||
val paymentHash = Crypto.sha256(paymentPreimage)
|
||||
assert(nodeParams.db.payments.getIncomingPayment(paymentHash).isEmpty)
|
||||
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 0, 1000 msat, paymentHash, defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val add = UpdateAddHtlc(ByteVector32.One, 0, 1000 msat, paymentHash, defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
val invoice = Bolt11Invoice(Block.Testnet3GenesisBlock.hash, None, paymentHash, randomKey(), Left("dummy"), CltvExpiryDelta(12))
|
||||
val incomingPayment = IncomingStandardPayment(invoice, paymentPreimage, PaymentType.Standard, invoice.createdAt.toTimestampMilli, IncomingPaymentStatus.Pending)
|
||||
val fulfill = DoFulfill(incomingPayment, MultiPartPaymentFSM.MultiPartPaymentSucceeded(paymentHash, Queue(HtlcPart(1000 msat, add))))
|
||||
|
@ -232,7 +232,7 @@ object MultiPartPaymentFSMSpec {
|
||||
def htlcIdToChannelId(htlcId: Long) = ByteVector32(ByteVector.fromLong(htlcId).padLeft(32))
|
||||
|
||||
def createMultiPartHtlc(totalAmount: MilliSatoshi, htlcAmount: MilliSatoshi, htlcId: Long): HtlcPart = {
|
||||
val htlc = UpdateAddHtlc(htlcIdToChannelId(htlcId), htlcId, htlcAmount, paymentHash, CltvExpiry(42), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val htlc = UpdateAddHtlc(htlcIdToChannelId(htlcId), htlcId, htlcAmount, paymentHash, CltvExpiry(42), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
HtlcPart(totalAmount, htlc)
|
||||
}
|
||||
|
||||
|
@ -94,7 +94,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
||||
def addCompleted(result: HtlcResult) = {
|
||||
RES_ADD_SETTLED(
|
||||
origin = defaultOrigin,
|
||||
htlc = UpdateAddHtlc(ByteVector32.Zeroes, 0, defaultAmountMsat, defaultPaymentHash, defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0),
|
||||
htlc = UpdateAddHtlc(ByteVector32.Zeroes, 0, defaultAmountMsat, defaultPaymentHash, defaultExpiry, TestConstants.emptyOnionPacket, None, 1.0, None),
|
||||
result)
|
||||
}
|
||||
|
||||
|
@ -79,7 +79,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
|
||||
}
|
||||
|
||||
def testPeelOnion(packet_b: OnionRoutingPacket): Unit = {
|
||||
val add_b = UpdateAddHtlc(randomBytes32(), 0, amount_ab, paymentHash, expiry_ab, packet_b, None, 1.0)
|
||||
val add_b = UpdateAddHtlc(randomBytes32(), 0, amount_ab, paymentHash, expiry_ab, packet_b, None, 1.0, None)
|
||||
val Right(relay_b@ChannelRelayPacket(add_b2, payload_b, packet_c)) = decrypt(add_b, priv_b.privateKey, Features.empty)
|
||||
assert(add_b2 == add_b)
|
||||
assert(packet_c.payload.length == PaymentOnionCodecs.paymentOnionPayloadLength)
|
||||
@ -89,7 +89,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
|
||||
assert(relay_b.relayFeeMsat == fee_b)
|
||||
assert(relay_b.expiryDelta == channelUpdate_bc.cltvExpiryDelta)
|
||||
|
||||
val add_c = UpdateAddHtlc(randomBytes32(), 1, amount_bc, paymentHash, expiry_bc, packet_c, None, 1.0)
|
||||
val add_c = UpdateAddHtlc(randomBytes32(), 1, amount_bc, paymentHash, expiry_bc, packet_c, None, 1.0, None)
|
||||
val Right(relay_c@ChannelRelayPacket(add_c2, payload_c, packet_d)) = decrypt(add_c, priv_c.privateKey, Features.empty)
|
||||
assert(add_c2 == add_c)
|
||||
assert(packet_d.payload.length == PaymentOnionCodecs.paymentOnionPayloadLength)
|
||||
@ -99,7 +99,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
|
||||
assert(relay_c.relayFeeMsat == fee_c)
|
||||
assert(relay_c.expiryDelta == channelUpdate_cd.cltvExpiryDelta)
|
||||
|
||||
val add_d = UpdateAddHtlc(randomBytes32(), 2, amount_cd, paymentHash, expiry_cd, packet_d, None, 1.0)
|
||||
val add_d = UpdateAddHtlc(randomBytes32(), 2, amount_cd, paymentHash, expiry_cd, packet_d, None, 1.0, None)
|
||||
val Right(relay_d@ChannelRelayPacket(add_d2, payload_d, packet_e)) = decrypt(add_d, priv_d.privateKey, Features.empty)
|
||||
assert(add_d2 == add_d)
|
||||
assert(packet_e.payload.length == PaymentOnionCodecs.paymentOnionPayloadLength)
|
||||
@ -109,7 +109,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
|
||||
assert(relay_d.relayFeeMsat == fee_d)
|
||||
assert(relay_d.expiryDelta == channelUpdate_de.cltvExpiryDelta)
|
||||
|
||||
val add_e = UpdateAddHtlc(randomBytes32(), 2, amount_de, paymentHash, expiry_de, packet_e, None, 1.0)
|
||||
val add_e = UpdateAddHtlc(randomBytes32(), 2, amount_de, paymentHash, expiry_de, packet_e, None, 1.0, None)
|
||||
val Right(FinalPacket(add_e2, payload_e)) = decrypt(add_e, priv_e.privateKey, Features.empty)
|
||||
assert(add_e2 == add_e)
|
||||
assert(payload_e.isInstanceOf[FinalPayload.Standard])
|
||||
@ -133,7 +133,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
|
||||
assert(payment.cmd.onion.payload.length == PaymentOnionCodecs.paymentOnionPayloadLength)
|
||||
|
||||
// let's peel the onion
|
||||
val add_b = UpdateAddHtlc(randomBytes32(), 0, finalAmount, paymentHash, finalExpiry, payment.cmd.onion, None, 1.0)
|
||||
val add_b = UpdateAddHtlc(randomBytes32(), 0, finalAmount, paymentHash, finalExpiry, payment.cmd.onion, None, 1.0, None)
|
||||
val Right(FinalPacket(add_b2, payload_b)) = decrypt(add_b, priv_b.privateKey, Features.empty)
|
||||
assert(add_b2 == add_b)
|
||||
assert(payload_b.isInstanceOf[FinalPayload.Standard])
|
||||
@ -150,7 +150,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
|
||||
val Right(payment) = buildOutgoingPayment(TestConstants.emptyOrigin, paymentHash, route, recipient, 1.0)
|
||||
|
||||
// let's peel the onion
|
||||
val add_b = UpdateAddHtlc(randomBytes32(), 0, finalAmount + 100.msat, paymentHash, finalExpiry + CltvExpiryDelta(6), payment.cmd.onion, None, 1.0)
|
||||
val add_b = UpdateAddHtlc(randomBytes32(), 0, finalAmount + 100.msat, paymentHash, finalExpiry + CltvExpiryDelta(6), payment.cmd.onion, None, 1.0, None)
|
||||
val Right(FinalPacket(_, payload_b)) = decrypt(add_b, priv_b.privateKey, Features.empty)
|
||||
assert(payload_b.isInstanceOf[FinalPayload.Standard])
|
||||
assert(payload_b.amount == finalAmount)
|
||||
@ -170,7 +170,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
|
||||
assert(payment.cmd.cltvExpiry == expiry_ab)
|
||||
assert(payment.cmd.nextBlindingKey_opt.isEmpty)
|
||||
|
||||
val add_b = UpdateAddHtlc(randomBytes32(), 0, payment.cmd.amount, payment.cmd.paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, payment.cmd.nextBlindingKey_opt, 1.0)
|
||||
val add_b = UpdateAddHtlc(randomBytes32(), 0, payment.cmd.amount, payment.cmd.paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, payment.cmd.nextBlindingKey_opt, 1.0, payment.cmd.fundingFee_opt)
|
||||
val Right(relay_b@ChannelRelayPacket(_, payload_b, packet_c)) = decrypt(add_b, priv_b.privateKey, Features.empty)
|
||||
assert(packet_c.payload.length == PaymentOnionCodecs.paymentOnionPayloadLength)
|
||||
assert(relay_b.amountToForward >= amount_bc)
|
||||
@ -180,7 +180,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
|
||||
assert(relay_b.expiryDelta == channelUpdate_bc.cltvExpiryDelta)
|
||||
assert(payload_b.isInstanceOf[IntermediatePayload.ChannelRelay.Standard])
|
||||
|
||||
val add_c = UpdateAddHtlc(randomBytes32(), 1, relay_b.amountToForward, relay_b.add.paymentHash, relay_b.outgoingCltv, packet_c, None, 1.0)
|
||||
val add_c = UpdateAddHtlc(randomBytes32(), 1, relay_b.amountToForward, relay_b.add.paymentHash, relay_b.outgoingCltv, packet_c, None, 1.0, None)
|
||||
val Right(relay_c@ChannelRelayPacket(_, payload_c, packet_d)) = decrypt(add_c, priv_c.privateKey, Features(RouteBlinding -> Optional))
|
||||
assert(packet_d.payload.length == PaymentOnionCodecs.paymentOnionPayloadLength)
|
||||
assert(relay_c.amountToForward >= amount_cd)
|
||||
@ -191,7 +191,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
|
||||
assert(payload_c.isInstanceOf[IntermediatePayload.ChannelRelay.Blinded])
|
||||
val blinding_d = payload_c.asInstanceOf[IntermediatePayload.ChannelRelay.Blinded].nextBlinding
|
||||
|
||||
val add_d = UpdateAddHtlc(randomBytes32(), 2, relay_c.amountToForward, relay_c.add.paymentHash, relay_c.outgoingCltv, packet_d, Some(blinding_d), 1.0)
|
||||
val add_d = UpdateAddHtlc(randomBytes32(), 2, relay_c.amountToForward, relay_c.add.paymentHash, relay_c.outgoingCltv, packet_d, Some(blinding_d), 1.0, None)
|
||||
val Right(relay_d@ChannelRelayPacket(_, payload_d, packet_e)) = decrypt(add_d, priv_d.privateKey, Features(RouteBlinding -> Optional))
|
||||
assert(packet_e.payload.length == PaymentOnionCodecs.paymentOnionPayloadLength)
|
||||
assert(relay_d.amountToForward >= amount_de)
|
||||
@ -202,7 +202,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
|
||||
assert(payload_d.isInstanceOf[IntermediatePayload.ChannelRelay.Blinded])
|
||||
val blinding_e = payload_d.asInstanceOf[IntermediatePayload.ChannelRelay.Blinded].nextBlinding
|
||||
|
||||
val add_e = UpdateAddHtlc(randomBytes32(), 2, relay_d.amountToForward, relay_d.add.paymentHash, relay_d.outgoingCltv, packet_e, Some(blinding_e), 1.0)
|
||||
val add_e = UpdateAddHtlc(randomBytes32(), 2, relay_d.amountToForward, relay_d.add.paymentHash, relay_d.outgoingCltv, packet_e, Some(blinding_e), 1.0, None)
|
||||
val Right(FinalPacket(_, payload_e)) = decrypt(add_e, priv_e.privateKey, Features(RouteBlinding -> Optional))
|
||||
assert(payload_e.amount == finalAmount)
|
||||
assert(payload_e.totalAmount == finalAmount)
|
||||
@ -233,7 +233,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
|
||||
assert(payment.cmd.cltvExpiry == expiry_ab)
|
||||
assert(payment.cmd.nextBlindingKey_opt.isEmpty)
|
||||
|
||||
val add_b = UpdateAddHtlc(randomBytes32(), 0, payment.cmd.amount, payment.cmd.paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, payment.cmd.nextBlindingKey_opt, 1.0)
|
||||
val add_b = UpdateAddHtlc(randomBytes32(), 0, payment.cmd.amount, payment.cmd.paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, payment.cmd.nextBlindingKey_opt, 1.0, payment.cmd.fundingFee_opt)
|
||||
val Right(relay_b@ChannelRelayPacket(_, payload_b, packet_c)) = decrypt(add_b, priv_b.privateKey, Features.empty)
|
||||
assert(packet_c.payload.length == PaymentOnionCodecs.paymentOnionPayloadLength)
|
||||
assert(relay_b.amountToForward >= amount_bc)
|
||||
@ -243,7 +243,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
|
||||
assert(relay_b.expiryDelta == channelUpdate_bc.cltvExpiryDelta)
|
||||
assert(payload_b.isInstanceOf[IntermediatePayload.ChannelRelay.Standard])
|
||||
|
||||
val add_c = UpdateAddHtlc(randomBytes32(), 1, amount_bc, paymentHash, expiry_bc, packet_c, None, 1.0)
|
||||
val add_c = UpdateAddHtlc(randomBytes32(), 1, amount_bc, paymentHash, expiry_bc, packet_c, None, 1.0, None)
|
||||
val Right(FinalPacket(_, payload_c)) = decrypt(add_c, priv_c.privateKey, Features(RouteBlinding -> Optional))
|
||||
assert(payload_c.amount == amount_bc)
|
||||
assert(payload_c.totalAmount == amount_bc)
|
||||
@ -259,7 +259,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
|
||||
assert(payment.cmd.cltvExpiry == finalExpiry)
|
||||
assert(payment.cmd.nextBlindingKey_opt.nonEmpty)
|
||||
|
||||
val add_b = UpdateAddHtlc(randomBytes32(), 0, payment.cmd.amount, payment.cmd.paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, payment.cmd.nextBlindingKey_opt, 1.0)
|
||||
val add_b = UpdateAddHtlc(randomBytes32(), 0, payment.cmd.amount, payment.cmd.paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, payment.cmd.nextBlindingKey_opt, 1.0, payment.cmd.fundingFee_opt)
|
||||
val Right(FinalPacket(_, payload_b)) = decrypt(add_b, priv_b.privateKey, Features(RouteBlinding -> Optional))
|
||||
assert(payload_b.amount == finalAmount)
|
||||
assert(payload_b.totalAmount == finalAmount)
|
||||
@ -272,7 +272,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
|
||||
val Right(payment) = buildOutgoingBlindedPaymentAB(paymentHash)
|
||||
assert(payment.outgoingChannel == channelUpdate_ab.shortChannelId)
|
||||
|
||||
val add_b = UpdateAddHtlc(randomBytes32(), 0, payment.cmd.amount + 100.msat, payment.cmd.paymentHash, payment.cmd.cltvExpiry + CltvExpiryDelta(6), payment.cmd.onion, payment.cmd.nextBlindingKey_opt, 1.0)
|
||||
val add_b = UpdateAddHtlc(randomBytes32(), 0, payment.cmd.amount + 100.msat, payment.cmd.paymentHash, payment.cmd.cltvExpiry + CltvExpiryDelta(6), payment.cmd.onion, payment.cmd.nextBlindingKey_opt, 1.0, payment.cmd.fundingFee_opt)
|
||||
val Right(FinalPacket(_, payload_b)) = decrypt(add_b, priv_b.privateKey, Features(RouteBlinding -> Optional))
|
||||
assert(payload_b.amount == finalAmount)
|
||||
assert(payload_b.totalAmount == finalAmount)
|
||||
@ -293,12 +293,12 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
|
||||
assert(payment.cmd.amount == amount_ab)
|
||||
assert(payment.cmd.cltvExpiry == expiry_ab)
|
||||
|
||||
val add_b = UpdateAddHtlc(randomBytes32(), 1, payment.cmd.amount, paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, None, 1.0)
|
||||
val add_b = UpdateAddHtlc(randomBytes32(), 1, payment.cmd.amount, paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, None, 1.0, None)
|
||||
val Right(ChannelRelayPacket(add_b2, payload_b, packet_c)) = decrypt(add_b, priv_b.privateKey, Features.empty)
|
||||
assert(add_b2 == add_b)
|
||||
assert(payload_b == IntermediatePayload.ChannelRelay.Standard(channelUpdate_bc.shortChannelId, amount_bc, expiry_bc))
|
||||
|
||||
val add_c = UpdateAddHtlc(randomBytes32(), 2, amount_bc, paymentHash, expiry_bc, packet_c, None, 1.0)
|
||||
val add_c = UpdateAddHtlc(randomBytes32(), 2, amount_bc, paymentHash, expiry_bc, packet_c, None, 1.0, None)
|
||||
val Right(RelayToTrampolinePacket(add_c2, outer_c, inner_c, trampolinePacket_e)) = decrypt(add_c, priv_c.privateKey, Features.empty)
|
||||
assert(add_c2 == add_c)
|
||||
assert(outer_c.amount == amount_bc)
|
||||
@ -315,16 +315,16 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
|
||||
|
||||
// c forwards the trampoline payment to e through d.
|
||||
val recipient_e = ClearRecipient(e, Features.empty, inner_c.amountToForward, inner_c.outgoingCltv, randomBytes32(), nextTrampolineOnion_opt = Some(trampolinePacket_e))
|
||||
val Right(payment_e) = buildOutgoingPayment(Origin.Hot(ActorRef.noSender, Upstream.Hot.Trampoline(Seq(Upstream.Hot.Channel(add_c, TimestampMilli(1687345927000L), b)))), paymentHash, Route(inner_c.amountToForward, afterTrampolineChannelHops, None), recipient_e, 1.0)
|
||||
val Right(payment_e) = buildOutgoingPayment(Origin.Hot(ActorRef.noSender, Upstream.Hot.Trampoline(List(Upstream.Hot.Channel(add_c, TimestampMilli(1687345927000L), b)))), paymentHash, Route(inner_c.amountToForward, afterTrampolineChannelHops, None), recipient_e, 1.0)
|
||||
assert(payment_e.outgoingChannel == channelUpdate_cd.shortChannelId)
|
||||
assert(payment_e.cmd.amount == amount_cd)
|
||||
assert(payment_e.cmd.cltvExpiry == expiry_cd)
|
||||
val add_d = UpdateAddHtlc(randomBytes32(), 3, payment_e.cmd.amount, paymentHash, payment_e.cmd.cltvExpiry, payment_e.cmd.onion, None, 1.0)
|
||||
val add_d = UpdateAddHtlc(randomBytes32(), 3, payment_e.cmd.amount, paymentHash, payment_e.cmd.cltvExpiry, payment_e.cmd.onion, None, 1.0, None)
|
||||
val Right(ChannelRelayPacket(add_d2, payload_d, packet_e)) = decrypt(add_d, priv_d.privateKey, Features.empty)
|
||||
assert(add_d2 == add_d)
|
||||
assert(payload_d == IntermediatePayload.ChannelRelay.Standard(channelUpdate_de.shortChannelId, amount_de, expiry_de))
|
||||
|
||||
val add_e = UpdateAddHtlc(randomBytes32(), 4, amount_de, paymentHash, expiry_de, packet_e, None, 1.0)
|
||||
val add_e = UpdateAddHtlc(randomBytes32(), 4, amount_de, paymentHash, expiry_de, packet_e, None, 1.0, None)
|
||||
val Right(FinalPacket(add_e2, payload_e)) = decrypt(add_e, priv_e.privateKey, Features.empty)
|
||||
assert(add_e2 == add_e)
|
||||
assert(payload_e == FinalPayload.Standard(TlvStream(AmountToForward(finalAmount), OutgoingCltv(finalExpiry), PaymentData(paymentSecret, finalAmount), OnionPaymentPayloadTlv.PaymentMetadata(hex"010203"))))
|
||||
@ -346,10 +346,10 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
|
||||
assert(payment.cmd.amount == amount_ab)
|
||||
assert(payment.cmd.cltvExpiry == expiry_ab)
|
||||
|
||||
val add_b = UpdateAddHtlc(randomBytes32(), 1, payment.cmd.amount, paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, None, 1.0)
|
||||
val add_b = UpdateAddHtlc(randomBytes32(), 1, payment.cmd.amount, paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, None, 1.0, None)
|
||||
val Right(ChannelRelayPacket(_, _, packet_c)) = decrypt(add_b, priv_b.privateKey, Features.empty)
|
||||
|
||||
val add_c = UpdateAddHtlc(randomBytes32(), 2, amount_bc, paymentHash, expiry_bc, packet_c, None, 1.0)
|
||||
val add_c = UpdateAddHtlc(randomBytes32(), 2, amount_bc, paymentHash, expiry_bc, packet_c, None, 1.0, None)
|
||||
val Right(RelayToTrampolinePacket(_, outer_c, inner_c, _)) = decrypt(add_c, priv_c.privateKey, Features.empty)
|
||||
assert(outer_c.amount == amount_bc)
|
||||
assert(outer_c.totalAmount == amount_bc)
|
||||
@ -367,16 +367,16 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
|
||||
|
||||
// c forwards the trampoline payment to e through d.
|
||||
val recipient_e = ClearRecipient(e, Features.empty, inner_c.amountToForward, inner_c.outgoingCltv, inner_c.paymentSecret.get, invoice.extraEdges, inner_c.paymentMetadata)
|
||||
val Right(payment_e) = buildOutgoingPayment(Origin.Hot(ActorRef.noSender, Upstream.Hot.Trampoline(Seq(Upstream.Hot.Channel(add_c, TimestampMilli(1687345927000L), b)))), paymentHash, Route(inner_c.amountToForward, afterTrampolineChannelHops, None), recipient_e, 1.0)
|
||||
val Right(payment_e) = buildOutgoingPayment(Origin.Hot(ActorRef.noSender, Upstream.Hot.Trampoline(List(Upstream.Hot.Channel(add_c, TimestampMilli(1687345927000L), b)))), paymentHash, Route(inner_c.amountToForward, afterTrampolineChannelHops, None), recipient_e, 1.0)
|
||||
assert(payment_e.outgoingChannel == channelUpdate_cd.shortChannelId)
|
||||
assert(payment_e.cmd.amount == amount_cd)
|
||||
assert(payment_e.cmd.cltvExpiry == expiry_cd)
|
||||
val add_d = UpdateAddHtlc(randomBytes32(), 3, payment_e.cmd.amount, paymentHash, payment_e.cmd.cltvExpiry, payment_e.cmd.onion, None, 1.0)
|
||||
val add_d = UpdateAddHtlc(randomBytes32(), 3, payment_e.cmd.amount, paymentHash, payment_e.cmd.cltvExpiry, payment_e.cmd.onion, None, 1.0, None)
|
||||
val Right(ChannelRelayPacket(add_d2, payload_d, packet_e)) = decrypt(add_d, priv_d.privateKey, Features.empty)
|
||||
assert(add_d2 == add_d)
|
||||
assert(payload_d == IntermediatePayload.ChannelRelay.Standard(channelUpdate_de.shortChannelId, amount_de, expiry_de))
|
||||
|
||||
val add_e = UpdateAddHtlc(randomBytes32(), 4, amount_de, paymentHash, expiry_de, packet_e, None, 1.0)
|
||||
val add_e = UpdateAddHtlc(randomBytes32(), 4, amount_de, paymentHash, expiry_de, packet_e, None, 1.0, None)
|
||||
val Right(FinalPacket(add_e2, payload_e)) = decrypt(add_e, priv_e.privateKey, Features.empty)
|
||||
assert(add_e2 == add_e)
|
||||
assert(payload_e == FinalPayload.Standard(TlvStream(AmountToForward(finalAmount), OutgoingCltv(finalExpiry), PaymentData(invoice.paymentSecret, finalAmount), OnionPaymentPayloadTlv.PaymentMetadata(hex"010203"))))
|
||||
@ -396,10 +396,10 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
|
||||
val Right(payment) = buildOutgoingPayment(TestConstants.emptyOrigin, paymentHash, Route(recipient.trampolineAmount, trampolineChannelHops, Some(trampolineHop)), recipient, 1.0)
|
||||
assert(payment.outgoingChannel == channelUpdate_ab.shortChannelId)
|
||||
|
||||
val add_b = UpdateAddHtlc(randomBytes32(), 1, payment.cmd.amount, paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, None, 1.0)
|
||||
val add_b = UpdateAddHtlc(randomBytes32(), 1, payment.cmd.amount, paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, None, 1.0, None)
|
||||
val Right(ChannelRelayPacket(_, _, packet_c)) = decrypt(add_b, priv_b.privateKey, Features.empty)
|
||||
|
||||
val add_c = UpdateAddHtlc(randomBytes32(), 2, amount_bc, paymentHash, expiry_bc, packet_c, None, 1.0)
|
||||
val add_c = UpdateAddHtlc(randomBytes32(), 2, amount_bc, paymentHash, expiry_bc, packet_c, None, 1.0, None)
|
||||
val Right(RelayToTrampolinePacket(_, outer_c, inner_c, _)) = decrypt(add_c, priv_c.privateKey, Features.empty)
|
||||
assert(outer_c.records.get[OnionPaymentPayloadTlv.TrampolineOnion].get.packet.payload.size > 800)
|
||||
assert(inner_c.outgoingNodeId == e)
|
||||
@ -408,14 +408,14 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
|
||||
|
||||
// c forwards the trampoline payment to e through d.
|
||||
val recipient_e = ClearRecipient(e, Features.empty, inner_c.amountToForward, inner_c.outgoingCltv, inner_c.paymentSecret.get, invoice.extraEdges, inner_c.paymentMetadata)
|
||||
val Right(payment_e) = buildOutgoingPayment(Origin.Hot(ActorRef.noSender, Upstream.Hot.Trampoline(Seq(Upstream.Hot.Channel(add_c, TimestampMilli(1687345927000L), b)))), paymentHash, Route(inner_c.amountToForward, afterTrampolineChannelHops, None), recipient_e, 1.0)
|
||||
val Right(payment_e) = buildOutgoingPayment(Origin.Hot(ActorRef.noSender, Upstream.Hot.Trampoline(List(Upstream.Hot.Channel(add_c, TimestampMilli(1687345927000L), b)))), paymentHash, Route(inner_c.amountToForward, afterTrampolineChannelHops, None), recipient_e, 1.0)
|
||||
assert(payment_e.outgoingChannel == channelUpdate_cd.shortChannelId)
|
||||
val add_d = UpdateAddHtlc(randomBytes32(), 3, payment_e.cmd.amount, paymentHash, payment_e.cmd.cltvExpiry, payment_e.cmd.onion, None, 1.0)
|
||||
val add_d = UpdateAddHtlc(randomBytes32(), 3, payment_e.cmd.amount, paymentHash, payment_e.cmd.cltvExpiry, payment_e.cmd.onion, None, 1.0, None)
|
||||
val Right(ChannelRelayPacket(add_d2, payload_d, packet_e)) = decrypt(add_d, priv_d.privateKey, Features.empty)
|
||||
assert(add_d2 == add_d)
|
||||
assert(payload_d == IntermediatePayload.ChannelRelay.Standard(channelUpdate_de.shortChannelId, amount_de, expiry_de))
|
||||
|
||||
val add_e = UpdateAddHtlc(randomBytes32(), 4, amount_de, paymentHash, expiry_de, packet_e, None, 1.0)
|
||||
val add_e = UpdateAddHtlc(randomBytes32(), 4, amount_de, paymentHash, expiry_de, packet_e, None, 1.0, None)
|
||||
val Right(FinalPacket(add_e2, payload_e)) = decrypt(add_e, priv_e.privateKey, Features.empty)
|
||||
assert(add_e2 == add_e)
|
||||
assert(payload_e == FinalPayload.Standard(TlvStream(AmountToForward(finalAmount), OutgoingCltv(finalExpiry), PaymentData(invoice.paymentSecret, finalAmount), OnionPaymentPayloadTlv.PaymentMetadata(paymentMetadata))))
|
||||
@ -448,7 +448,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
|
||||
test("fail to decrypt when the onion is invalid") {
|
||||
val recipient = ClearRecipient(e, Features.empty, finalAmount, finalExpiry, paymentSecret)
|
||||
val Right(payment) = buildOutgoingPayment(TestConstants.emptyOrigin, paymentHash, Route(finalAmount, hops, None), recipient, 1.0)
|
||||
val add = UpdateAddHtlc(randomBytes32(), 1, payment.cmd.amount, paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion.copy(payload = payment.cmd.onion.payload.reverse), None, 1.0)
|
||||
val add = UpdateAddHtlc(randomBytes32(), 1, payment.cmd.amount, paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion.copy(payload = payment.cmd.onion.payload.reverse), None, 1.0, None)
|
||||
val Left(failure) = decrypt(add, priv_b.privateKey, Features.empty)
|
||||
assert(failure.isInstanceOf[InvalidOnionHmac])
|
||||
}
|
||||
@ -459,20 +459,20 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
|
||||
val recipient = TrampolineRecipient(invoice, finalAmount, finalExpiry, trampolineHop, randomBytes32())
|
||||
val Right(payment) = buildOutgoingPayment(TestConstants.emptyOrigin, paymentHash, Route(recipient.trampolineAmount, trampolineChannelHops, Some(trampolineHop)), recipient, 1.0)
|
||||
|
||||
val add_b = UpdateAddHtlc(randomBytes32(), 1, payment.cmd.amount, paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, None, 1.0)
|
||||
val add_b = UpdateAddHtlc(randomBytes32(), 1, payment.cmd.amount, paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, None, 1.0, None)
|
||||
val Right(ChannelRelayPacket(_, _, packet_c)) = decrypt(add_b, priv_b.privateKey, Features.empty)
|
||||
|
||||
val add_c = UpdateAddHtlc(randomBytes32(), 2, amount_bc, paymentHash, expiry_bc, packet_c, None, 1.0)
|
||||
val add_c = UpdateAddHtlc(randomBytes32(), 2, amount_bc, paymentHash, expiry_bc, packet_c, None, 1.0, None)
|
||||
val Right(RelayToTrampolinePacket(_, _, inner_c, trampolinePacket_e)) = decrypt(add_c, priv_c.privateKey, Features.empty)
|
||||
|
||||
// c forwards an invalid trampoline onion to e through d.
|
||||
val recipient_e = ClearRecipient(e, Features.empty, inner_c.amountToForward, inner_c.outgoingCltv, randomBytes32(), nextTrampolineOnion_opt = Some(trampolinePacket_e.copy(payload = trampolinePacket_e.payload.reverse)))
|
||||
val Right(payment_e) = buildOutgoingPayment(Origin.Hot(ActorRef.noSender, Upstream.Hot.Trampoline(Seq(Upstream.Hot.Channel(add_c, TimestampMilli(1687345927000L), b)))), paymentHash, Route(inner_c.amountToForward, afterTrampolineChannelHops, None), recipient_e, 1.0)
|
||||
val Right(payment_e) = buildOutgoingPayment(Origin.Hot(ActorRef.noSender, Upstream.Hot.Trampoline(List(Upstream.Hot.Channel(add_c, TimestampMilli(1687345927000L), b)))), paymentHash, Route(inner_c.amountToForward, afterTrampolineChannelHops, None), recipient_e, 1.0)
|
||||
assert(payment_e.outgoingChannel == channelUpdate_cd.shortChannelId)
|
||||
val add_d = UpdateAddHtlc(randomBytes32(), 3, payment_e.cmd.amount, paymentHash, payment_e.cmd.cltvExpiry, payment_e.cmd.onion, None, 1.0)
|
||||
val add_d = UpdateAddHtlc(randomBytes32(), 3, payment_e.cmd.amount, paymentHash, payment_e.cmd.cltvExpiry, payment_e.cmd.onion, None, 1.0, None)
|
||||
val Right(ChannelRelayPacket(_, _, packet_e)) = decrypt(add_d, priv_d.privateKey, Features.empty)
|
||||
|
||||
val add_e = UpdateAddHtlc(randomBytes32(), 4, amount_de, paymentHash, expiry_de, packet_e, None, 1.0)
|
||||
val add_e = UpdateAddHtlc(randomBytes32(), 4, amount_de, paymentHash, expiry_de, packet_e, None, 1.0, None)
|
||||
val Left(failure) = decrypt(add_e, priv_e.privateKey, Features.empty)
|
||||
assert(failure.isInstanceOf[InvalidOnionHmac])
|
||||
}
|
||||
@ -480,7 +480,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
|
||||
test("fail to decrypt when payment hash doesn't match associated data") {
|
||||
val recipient = ClearRecipient(e, Features.empty, finalAmount, finalExpiry, paymentSecret)
|
||||
val Right(payment) = buildOutgoingPayment(TestConstants.emptyOrigin, paymentHash.reverse, Route(finalAmount, hops, None), recipient, 1.0)
|
||||
val add = UpdateAddHtlc(randomBytes32(), 1, payment.cmd.amount, paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, None, 1.0)
|
||||
val add = UpdateAddHtlc(randomBytes32(), 1, payment.cmd.amount, paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, None, 1.0, None)
|
||||
val Left(failure) = decrypt(add, priv_b.privateKey, Features.empty)
|
||||
assert(failure.isInstanceOf[InvalidOnionHmac])
|
||||
}
|
||||
@ -507,7 +507,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
|
||||
assert(payment.outgoingChannel == channelUpdate_ab.shortChannelId)
|
||||
assert(payment.cmd.amount == amount_bc + fee_b)
|
||||
|
||||
val add_b = UpdateAddHtlc(randomBytes32(), 0, payment.cmd.amount, payment.cmd.paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, payment.cmd.nextBlindingKey_opt, 1.0)
|
||||
val add_b = UpdateAddHtlc(randomBytes32(), 0, payment.cmd.amount, payment.cmd.paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, payment.cmd.nextBlindingKey_opt, 1.0, payment.cmd.fundingFee_opt)
|
||||
val Left(failure) = decrypt(add_b, priv_b.privateKey, Features(RouteBlinding -> Optional))
|
||||
assert(failure.isInstanceOf[InvalidOnionBlinding])
|
||||
}
|
||||
@ -515,7 +515,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
|
||||
test("fail to decrypt blinded payment when route blinding is disabled") {
|
||||
val (route, recipient) = shortBlindedHops()
|
||||
val Right(payment) = buildOutgoingPayment(TestConstants.emptyOrigin, paymentHash, route, recipient, 1.0)
|
||||
val add_d = UpdateAddHtlc(randomBytes32(), 0, payment.cmd.amount, payment.cmd.paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, payment.cmd.nextBlindingKey_opt, 1.0)
|
||||
val add_d = UpdateAddHtlc(randomBytes32(), 0, payment.cmd.amount, payment.cmd.paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, payment.cmd.nextBlindingKey_opt, 1.0, payment.cmd.fundingFee_opt)
|
||||
val Left(failure) = decrypt(add_d, priv_d.privateKey, Features.empty) // d doesn't support route blinding
|
||||
assert(failure == InvalidOnionPayload(UInt64(10), 0))
|
||||
}
|
||||
@ -524,7 +524,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
|
||||
val recipient = ClearRecipient(b, Features.empty, finalAmount, finalExpiry, paymentSecret)
|
||||
val route = Route(finalAmount, hops.take(1), None)
|
||||
val Right(payment) = buildOutgoingPayment(TestConstants.emptyOrigin, paymentHash, route, recipient, 1.0)
|
||||
val add = UpdateAddHtlc(randomBytes32(), 1, payment.cmd.amount - 100.msat, paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, None, 1.0)
|
||||
val add = UpdateAddHtlc(randomBytes32(), 1, payment.cmd.amount - 100.msat, paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, None, 1.0, None)
|
||||
val Left(failure) = decrypt(add, priv_b.privateKey, Features.empty)
|
||||
assert(failure == FinalIncorrectHtlcAmount(payment.cmd.amount - 100.msat))
|
||||
}
|
||||
@ -533,7 +533,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
|
||||
val recipient = ClearRecipient(b, Features.empty, finalAmount, finalExpiry, paymentSecret)
|
||||
val route = Route(finalAmount, hops.take(1), None)
|
||||
val Right(payment) = buildOutgoingPayment(TestConstants.emptyOrigin, paymentHash, route, recipient, 1.0)
|
||||
val add = UpdateAddHtlc(randomBytes32(), 1, payment.cmd.amount, paymentHash, payment.cmd.cltvExpiry - CltvExpiryDelta(12), payment.cmd.onion, None, 1.0)
|
||||
val add = UpdateAddHtlc(randomBytes32(), 1, payment.cmd.amount, paymentHash, payment.cmd.cltvExpiry - CltvExpiryDelta(12), payment.cmd.onion, None, 1.0, None)
|
||||
val Left(failure) = decrypt(add, priv_b.privateKey, Features.empty)
|
||||
assert(failure == FinalIncorrectCltvExpiry(payment.cmd.cltvExpiry - CltvExpiryDelta(12)))
|
||||
}
|
||||
@ -545,7 +545,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
|
||||
assert(payment.cmd.amount == amount_cd)
|
||||
|
||||
// A smaller amount is sent to d, who doesn't know that it's invalid.
|
||||
val add_d = UpdateAddHtlc(randomBytes32(), 0, amount_de, paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, payment.cmd.nextBlindingKey_opt, 1.0)
|
||||
val add_d = UpdateAddHtlc(randomBytes32(), 0, amount_de, paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, payment.cmd.nextBlindingKey_opt, 1.0, payment.cmd.fundingFee_opt)
|
||||
val Right(relay_d@ChannelRelayPacket(_, payload_d, packet_e)) = decrypt(add_d, priv_d.privateKey, Features(RouteBlinding -> Optional))
|
||||
assert(payload_d.outgoing.contains(channelUpdate_de.shortChannelId))
|
||||
assert(relay_d.amountToForward < amount_de)
|
||||
@ -553,7 +553,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
|
||||
val blinding_e = payload_d.asInstanceOf[IntermediatePayload.ChannelRelay.Blinded].nextBlinding
|
||||
|
||||
// When e receives a smaller amount than expected, it rejects the payment.
|
||||
val add_e = UpdateAddHtlc(randomBytes32(), 0, relay_d.amountToForward, paymentHash, relay_d.outgoingCltv, packet_e, Some(blinding_e), 1.0)
|
||||
val add_e = UpdateAddHtlc(randomBytes32(), 0, relay_d.amountToForward, paymentHash, relay_d.outgoingCltv, packet_e, Some(blinding_e), 1.0, None)
|
||||
val Left(failure) = decrypt(add_e, priv_e.privateKey, Features(RouteBlinding -> Optional))
|
||||
assert(failure.isInstanceOf[InvalidOnionBlinding])
|
||||
}
|
||||
@ -567,7 +567,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
|
||||
// A smaller expiry is sent to d, who doesn't know that it's invalid.
|
||||
// Intermediate nodes can reduce the expiry by at most min_final_expiry_delta.
|
||||
val invalidExpiry = payment.cmd.cltvExpiry - Channel.MIN_CLTV_EXPIRY_DELTA - CltvExpiryDelta(1)
|
||||
val add_d = UpdateAddHtlc(randomBytes32(), 0, payment.cmd.amount, paymentHash, invalidExpiry, payment.cmd.onion, payment.cmd.nextBlindingKey_opt, 1.0)
|
||||
val add_d = UpdateAddHtlc(randomBytes32(), 0, payment.cmd.amount, paymentHash, invalidExpiry, payment.cmd.onion, payment.cmd.nextBlindingKey_opt, 1.0, payment.cmd.fundingFee_opt)
|
||||
val Right(relay_d@ChannelRelayPacket(_, payload_d, packet_e)) = decrypt(add_d, priv_d.privateKey, Features(RouteBlinding -> Optional))
|
||||
assert(payload_d.outgoing.contains(channelUpdate_de.shortChannelId))
|
||||
assert(relay_d.outgoingCltv < CltvExpiry(currentBlockCount))
|
||||
@ -575,7 +575,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
|
||||
val blinding_e = payload_d.asInstanceOf[IntermediatePayload.ChannelRelay.Blinded].nextBlinding
|
||||
|
||||
// When e receives a smaller expiry than expected, it rejects the payment.
|
||||
val add_e = UpdateAddHtlc(randomBytes32(), 0, relay_d.amountToForward, paymentHash, relay_d.outgoingCltv, packet_e, Some(blinding_e), 1.0)
|
||||
val add_e = UpdateAddHtlc(randomBytes32(), 0, relay_d.amountToForward, paymentHash, relay_d.outgoingCltv, packet_e, Some(blinding_e), 1.0, None)
|
||||
val Left(failure) = decrypt(add_e, priv_e.privateKey, Features(RouteBlinding -> Optional))
|
||||
assert(failure.isInstanceOf[InvalidOnionBlinding])
|
||||
}
|
||||
@ -587,7 +587,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
|
||||
assert(payment.outgoingChannel == channelUpdate_cd.shortChannelId)
|
||||
assert(payment.cmd.cltvExpiry > expiry_de)
|
||||
|
||||
val add_d = UpdateAddHtlc(randomBytes32(), 0, payment.cmd.amount, paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, payment.cmd.nextBlindingKey_opt, 1.0)
|
||||
val add_d = UpdateAddHtlc(randomBytes32(), 0, payment.cmd.amount, paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, payment.cmd.nextBlindingKey_opt, 1.0, payment.cmd.fundingFee_opt)
|
||||
val Left(failure) = decrypt(add_d, priv_d.privateKey, Features(RouteBlinding -> Optional))
|
||||
assert(failure.isInstanceOf[InvalidOnionBlinding])
|
||||
}
|
||||
@ -604,10 +604,10 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
|
||||
val recipient = TrampolineRecipient(invoice, finalAmount, finalExpiry, trampolineHop, randomBytes32())
|
||||
val Right(payment) = buildOutgoingPayment(TestConstants.emptyOrigin, paymentHash, Route(recipient.trampolineAmount, trampolineChannelHops, Some(trampolineHop)), recipient, 1.0)
|
||||
|
||||
val add_b = UpdateAddHtlc(randomBytes32(), 1, payment.cmd.amount, paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, None, 1.0)
|
||||
val add_b = UpdateAddHtlc(randomBytes32(), 1, payment.cmd.amount, paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, None, 1.0, None)
|
||||
val Right(ChannelRelayPacket(_, _, packet_c)) = decrypt(add_b, priv_b.privateKey, Features.empty)
|
||||
|
||||
UpdateAddHtlc(randomBytes32(), 2, amount_bc, paymentHash, expiry_bc, packet_c, None, 1.0)
|
||||
UpdateAddHtlc(randomBytes32(), 2, amount_bc, paymentHash, expiry_bc, packet_c, None, 1.0, None)
|
||||
}
|
||||
|
||||
test("fail to decrypt at the final trampoline node when amount has been decreased by next-to-last trampoline") {
|
||||
@ -617,11 +617,11 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
|
||||
// c forwards an invalid amount to e through (the outer total amount doesn't match the inner amount).
|
||||
val invalidTotalAmount = inner_c.amountToForward - 1.msat
|
||||
val recipient_e = ClearRecipient(e, Features.empty, invalidTotalAmount, inner_c.outgoingCltv, randomBytes32(), nextTrampolineOnion_opt = Some(trampolinePacket_e))
|
||||
val Right(payment_e) = buildOutgoingPayment(Origin.Hot(ActorRef.noSender, Upstream.Hot.Trampoline(Seq(Upstream.Hot.Channel(add_c, TimestampMilli(1687345927000L), b)))), paymentHash, Route(invalidTotalAmount, afterTrampolineChannelHops, None), recipient_e, 1.0)
|
||||
val add_d = UpdateAddHtlc(randomBytes32(), 3, payment_e.cmd.amount, paymentHash, payment_e.cmd.cltvExpiry, payment_e.cmd.onion, None, 1.0)
|
||||
val Right(payment_e) = buildOutgoingPayment(Origin.Hot(ActorRef.noSender, Upstream.Hot.Trampoline(List(Upstream.Hot.Channel(add_c, TimestampMilli(1687345927000L), b)))), paymentHash, Route(invalidTotalAmount, afterTrampolineChannelHops, None), recipient_e, 1.0)
|
||||
val add_d = UpdateAddHtlc(randomBytes32(), 3, payment_e.cmd.amount, paymentHash, payment_e.cmd.cltvExpiry, payment_e.cmd.onion, None, 1.0, None)
|
||||
val Right(ChannelRelayPacket(_, payload_d, packet_e)) = decrypt(add_d, priv_d.privateKey, Features.empty)
|
||||
|
||||
val add_e = UpdateAddHtlc(randomBytes32(), 4, payload_d.amountToForward(add_d.amountMsat), paymentHash, payload_d.outgoingCltv(add_d.cltvExpiry), packet_e, None, 1.0)
|
||||
val add_e = UpdateAddHtlc(randomBytes32(), 4, payload_d.amountToForward(add_d.amountMsat), paymentHash, payload_d.outgoingCltv(add_d.cltvExpiry), packet_e, None, 1.0, None)
|
||||
val Left(failure) = decrypt(add_e, priv_e.privateKey, Features.empty)
|
||||
assert(failure == FinalIncorrectHtlcAmount(invalidTotalAmount))
|
||||
}
|
||||
@ -633,11 +633,11 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
|
||||
// c forwards an invalid amount to e through (the outer expiry doesn't match the inner expiry).
|
||||
val invalidExpiry = inner_c.outgoingCltv - CltvExpiryDelta(12)
|
||||
val recipient_e = ClearRecipient(e, Features.empty, inner_c.amountToForward, invalidExpiry, randomBytes32(), nextTrampolineOnion_opt = Some(trampolinePacket_e))
|
||||
val Right(payment_e) = buildOutgoingPayment(Origin.Hot(ActorRef.noSender, Upstream.Hot.Trampoline(Seq(Upstream.Hot.Channel(add_c, TimestampMilli(1687345927000L), b)))), paymentHash, Route(inner_c.amountToForward, afterTrampolineChannelHops, None), recipient_e, 1.0)
|
||||
val add_d = UpdateAddHtlc(randomBytes32(), 3, payment_e.cmd.amount, paymentHash, payment_e.cmd.cltvExpiry, payment_e.cmd.onion, None, 1.0)
|
||||
val Right(payment_e) = buildOutgoingPayment(Origin.Hot(ActorRef.noSender, Upstream.Hot.Trampoline(List(Upstream.Hot.Channel(add_c, TimestampMilli(1687345927000L), b)))), paymentHash, Route(inner_c.amountToForward, afterTrampolineChannelHops, None), recipient_e, 1.0)
|
||||
val add_d = UpdateAddHtlc(randomBytes32(), 3, payment_e.cmd.amount, paymentHash, payment_e.cmd.cltvExpiry, payment_e.cmd.onion, None, 1.0, None)
|
||||
val Right(ChannelRelayPacket(_, payload_d, packet_e)) = decrypt(add_d, priv_d.privateKey, Features.empty)
|
||||
|
||||
val add_e = UpdateAddHtlc(randomBytes32(), 4, payload_d.amountToForward(add_d.amountMsat), paymentHash, payload_d.outgoingCltv(add_d.cltvExpiry), packet_e, None, 1.0)
|
||||
val add_e = UpdateAddHtlc(randomBytes32(), 4, payload_d.amountToForward(add_d.amountMsat), paymentHash, payload_d.outgoingCltv(add_d.cltvExpiry), packet_e, None, 1.0, None)
|
||||
val Left(failure) = decrypt(add_e, priv_e.privateKey, Features.empty)
|
||||
assert(failure == FinalIncorrectCltvExpiry(invalidExpiry))
|
||||
}
|
||||
@ -661,13 +661,13 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
|
||||
// a -> b -> c -> d -> e
|
||||
val recipient = ClearRecipient(e, Features.empty, finalAmount, finalExpiry, paymentSecret)
|
||||
val Right(payment) = buildOutgoingPayment(TestConstants.emptyOrigin, paymentHash, Route(finalAmount, hops, None), recipient, 1.0)
|
||||
val add_b = UpdateAddHtlc(randomBytes32(), 0, amount_ab, paymentHash, expiry_ab, payment.cmd.onion, None, 1.0)
|
||||
val add_b = UpdateAddHtlc(randomBytes32(), 0, amount_ab, paymentHash, expiry_ab, payment.cmd.onion, None, 1.0, None)
|
||||
val Right(ChannelRelayPacket(_, _, packet_c)) = decrypt(add_b, priv_b.privateKey, Features.empty)
|
||||
val add_c = UpdateAddHtlc(randomBytes32(), 1, amount_bc, paymentHash, expiry_bc, packet_c, None, 1.0)
|
||||
val add_c = UpdateAddHtlc(randomBytes32(), 1, amount_bc, paymentHash, expiry_bc, packet_c, None, 1.0, None)
|
||||
val Right(ChannelRelayPacket(_, _, packet_d)) = decrypt(add_c, priv_c.privateKey, Features.empty)
|
||||
val add_d = UpdateAddHtlc(randomBytes32(), 2, amount_cd, paymentHash, expiry_cd, packet_d, None, 1.0)
|
||||
val add_d = UpdateAddHtlc(randomBytes32(), 2, amount_cd, paymentHash, expiry_cd, packet_d, None, 1.0, None)
|
||||
val Right(ChannelRelayPacket(_, _, packet_e)) = decrypt(add_d, priv_d.privateKey, Features.empty)
|
||||
val add_e = UpdateAddHtlc(randomBytes32(), 3, amount_de, paymentHash, expiry_de, packet_e, None, 1.0)
|
||||
val add_e = UpdateAddHtlc(randomBytes32(), 3, amount_de, paymentHash, expiry_de, packet_e, None, 1.0, None)
|
||||
val Right(FinalPacket(_, payload_e)) = decrypt(add_e, priv_e.privateKey, Features.empty)
|
||||
assert(payload_e.isInstanceOf[FinalPayload.Standard])
|
||||
|
||||
@ -690,15 +690,15 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
|
||||
// a -> b -> c -> d -> e, blinded after c
|
||||
val (_, route, recipient) = longBlindedHops(hex"0451")
|
||||
val Right(payment) = buildOutgoingPayment(TestConstants.emptyOrigin, paymentHash, route, recipient, 1.0)
|
||||
val add_b = UpdateAddHtlc(randomBytes32(), 0, payment.cmd.amount, payment.cmd.paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, payment.cmd.nextBlindingKey_opt, 1.0)
|
||||
val add_b = UpdateAddHtlc(randomBytes32(), 0, payment.cmd.amount, payment.cmd.paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, payment.cmd.nextBlindingKey_opt, 1.0, payment.cmd.fundingFee_opt)
|
||||
val Right(ChannelRelayPacket(_, _, packet_c)) = decrypt(add_b, priv_b.privateKey, Features.empty)
|
||||
val add_c = UpdateAddHtlc(randomBytes32(), 1, amount_bc, paymentHash, expiry_bc, packet_c, None, 1.0)
|
||||
val add_c = UpdateAddHtlc(randomBytes32(), 1, amount_bc, paymentHash, expiry_bc, packet_c, None, 1.0, None)
|
||||
val Right(ChannelRelayPacket(_, payload_c, packet_d)) = decrypt(add_c, priv_c.privateKey, Features(RouteBlinding -> Optional))
|
||||
val blinding_d = payload_c.asInstanceOf[IntermediatePayload.ChannelRelay.Blinded].nextBlinding
|
||||
val add_d = UpdateAddHtlc(randomBytes32(), 2, amount_cd, paymentHash, expiry_cd, packet_d, Some(blinding_d), 1.0)
|
||||
val add_d = UpdateAddHtlc(randomBytes32(), 2, amount_cd, paymentHash, expiry_cd, packet_d, Some(blinding_d), 1.0, None)
|
||||
val Right(ChannelRelayPacket(_, payload_d, packet_e)) = decrypt(add_d, priv_d.privateKey, Features(RouteBlinding -> Optional))
|
||||
val blinding_e = payload_d.asInstanceOf[IntermediatePayload.ChannelRelay.Blinded].nextBlinding
|
||||
val add_e = UpdateAddHtlc(randomBytes32(), 3, amount_de, paymentHash, expiry_de, packet_e, Some(blinding_e), 1.0)
|
||||
val add_e = UpdateAddHtlc(randomBytes32(), 3, amount_de, paymentHash, expiry_de, packet_e, Some(blinding_e), 1.0, None)
|
||||
val Right(FinalPacket(_, payload_e)) = decrypt(add_e, priv_e.privateKey, Features(RouteBlinding -> Optional))
|
||||
assert(payload_e.isInstanceOf[FinalPayload.Blinded])
|
||||
|
||||
@ -796,7 +796,7 @@ object PaymentPacketSpec {
|
||||
val blindedRoute = BlindedRouteCreation.createBlindedRouteWithoutHops(b, hex"deadbeef", 1.msat, routeExpiry).route
|
||||
val finalPayload = NodePayload(blindedRoute.introductionNode.blindedPublicKey, OutgoingBlindedPerHopPayload.createFinalPayload(finalAmount, finalAmount, finalExpiry, blindedRoute.introductionNode.encryptedPayload))
|
||||
val onion = buildOnion(Seq(finalPayload), paymentHash, Some(PaymentOnionCodecs.paymentOnionPayloadLength)).toOption.get // BOLT 2 requires that associatedData == paymentHash
|
||||
val cmd = CMD_ADD_HTLC(ActorRef.noSender, finalAmount, paymentHash, finalExpiry, onion.packet, Some(blindedRoute.blindingKey), 1.0, TestConstants.emptyOrigin, commit = true)
|
||||
val cmd = CMD_ADD_HTLC(ActorRef.noSender, finalAmount, paymentHash, finalExpiry, onion.packet, Some(blindedRoute.blindingKey), 1.0, None, TestConstants.emptyOrigin, commit = true)
|
||||
Right(OutgoingPaymentPacket(cmd, channelUpdate_ab.shortChannelId, onion.sharedSecrets))
|
||||
}
|
||||
|
||||
|
@ -18,11 +18,10 @@ package fr.acinq.eclair.payment
|
||||
|
||||
import akka.Done
|
||||
import akka.actor.ActorRef
|
||||
import akka.actor.typed.scaladsl.adapter.ClassicActorRefOps
|
||||
import akka.event.LoggingAdapter
|
||||
import akka.testkit.TestProbe
|
||||
import com.softwaremill.quicklens.{ModifyPimp, QuicklensAt}
|
||||
import fr.acinq.bitcoin.scalacompat.{Block, ByteVector32, Crypto, OutPoint, SatoshiLong, Script, Transaction, TxIn, TxOut}
|
||||
import fr.acinq.bitcoin.scalacompat.{Block, ByteVector32, Crypto, OutPoint, SatoshiLong, Script, Transaction, TxId, TxIn, TxOut}
|
||||
import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher.WatchTxConfirmedTriggered
|
||||
import fr.acinq.eclair.channel.Helpers.Closing
|
||||
import fr.acinq.eclair.channel._
|
||||
@ -30,7 +29,8 @@ import fr.acinq.eclair.channel.states.ChannelStateTestsBase
|
||||
import fr.acinq.eclair.db.{OutgoingPayment, OutgoingPaymentStatus, PaymentType}
|
||||
import fr.acinq.eclair.payment.OutgoingPaymentPacket.buildOutgoingPayment
|
||||
import fr.acinq.eclair.payment.PaymentPacketSpec._
|
||||
import fr.acinq.eclair.payment.relay.{PostRestartHtlcCleaner, Relayer}
|
||||
import fr.acinq.eclair.payment.relay.OnTheFlyFundingSpec._
|
||||
import fr.acinq.eclair.payment.relay.{OnTheFlyFunding, PostRestartHtlcCleaner, Relayer}
|
||||
import fr.acinq.eclair.payment.send.SpontaneousRecipient
|
||||
import fr.acinq.eclair.router.BaseRouterSpec.channelHopFromUpdate
|
||||
import fr.acinq.eclair.router.Router.Route
|
||||
@ -153,6 +153,43 @@ class PostRestartHtlcCleanerSpec extends TestKitBaseClass with FixtureAnyFunSuit
|
||||
channel.expectNoMessage(100 millis)
|
||||
}
|
||||
|
||||
test("keep upstream HTLCs that weren't relayed downstream but use on-the-fly funding") { f =>
|
||||
import f._
|
||||
|
||||
val channelPaymentHash = randomBytes32()
|
||||
val trampolinePaymentHash = randomBytes32()
|
||||
|
||||
val htlc_ab_1 = Seq(
|
||||
buildHtlcIn(0, channelId_ab_1, channelPaymentHash),
|
||||
buildHtlcIn(1, channelId_ab_1, trampolinePaymentHash),
|
||||
)
|
||||
val htlc_ab_2 = Seq(
|
||||
buildHtlcIn(2, channelId_ab_2, trampolinePaymentHash),
|
||||
)
|
||||
|
||||
val channels = Seq(
|
||||
ChannelCodecsSpec.makeChannelDataNormal(htlc_ab_1, Map(0L -> Origin.Cold(Upstream.Local(UUID.randomUUID())), 1L -> Origin.Cold(Upstream.Local(UUID.randomUUID())))),
|
||||
ChannelCodecsSpec.makeChannelDataNormal(htlc_ab_2, Map(2L -> Origin.Cold(Upstream.Local(UUID.randomUUID()))))
|
||||
)
|
||||
|
||||
// The HTLCs were not relayed yet, but they match pending on-the-fly funding proposals.
|
||||
val upstreamChannel = Upstream.Hot.Channel(htlc_ab_1.head.add, TimestampMilli.now(), a)
|
||||
val downstreamChannel = OnTheFlyFunding.Pending(Seq(OnTheFlyFunding.Proposal(createWillAdd(100_000 msat, channelPaymentHash, CltvExpiry(500)), upstreamChannel)), createStatus())
|
||||
val upstreamTrampoline = Upstream.Hot.Trampoline(List(htlc_ab_1.last, htlc_ab_2.head).map(htlc => Upstream.Hot.Channel(htlc.add, TimestampMilli.now(), a)))
|
||||
val downstreamTrampoline = OnTheFlyFunding.Pending(Seq(OnTheFlyFunding.Proposal(createWillAdd(100_000 msat, trampolinePaymentHash, CltvExpiry(500)), upstreamTrampoline)), createStatus())
|
||||
nodeParams.db.liquidity.addPendingOnTheFlyFunding(randomKey().publicKey, downstreamChannel)
|
||||
nodeParams.db.liquidity.addPendingOnTheFlyFunding(randomKey().publicKey, downstreamTrampoline)
|
||||
|
||||
val channel = TestProbe()
|
||||
val (relayer, _) = f.createRelayer(nodeParams)
|
||||
relayer ! PostRestartHtlcCleaner.Init(channels)
|
||||
// Upstream channels go back to the NORMAL state, but HTLCs are kept because the on-the-fly proposal was funded.
|
||||
system.eventStream.publish(ChannelStateChanged(channel.ref, channels.head.commitments.channelId, system.deadLetters, a, OFFLINE, NORMAL, Some(channels(0).commitments)))
|
||||
channel.expectNoMessage(100 millis)
|
||||
system.eventStream.publish(ChannelStateChanged(channel.ref, channels(1).commitments.channelId, system.deadLetters, a, OFFLINE, NORMAL, Some(channels(1).commitments)))
|
||||
channel.expectNoMessage(100 millis)
|
||||
}
|
||||
|
||||
test("clean up upstream HTLCs for which we're the final recipient") { f =>
|
||||
import f._
|
||||
|
||||
@ -512,7 +549,7 @@ class PostRestartHtlcCleanerSpec extends TestKitBaseClass with FixtureAnyFunSuit
|
||||
import f._
|
||||
|
||||
val htlc_ab = buildHtlcIn(0, channelId_ab_1, paymentHash1, blinded = true)
|
||||
val upstream = Upstream.Cold.Channel(htlc_ab.add.channelId, htlc_ab.add.id, htlc_ab.add.amountMsat)
|
||||
val upstream = Upstream.Cold.Channel(htlc_ab.add)
|
||||
val htlc_bc = buildHtlcOut(6, channelId_bc_1, paymentHash1, blinded = true)
|
||||
val data_ab = ChannelCodecsSpec.makeChannelDataNormal(Seq(htlc_ab), Map.empty)
|
||||
val data_bc = ChannelCodecsSpec.makeChannelDataNormal(Seq(htlc_bc), Map(6L -> Origin.Cold(upstream)))
|
||||
@ -624,6 +661,54 @@ class PostRestartHtlcCleanerSpec extends TestKitBaseClass with FixtureAnyFunSuit
|
||||
eventListener.expectNoMessage(100 millis)
|
||||
}
|
||||
|
||||
test("ignore relayed htlc-fail for on-the-fly funding") { f =>
|
||||
import f._
|
||||
|
||||
// Upstream HTLCs.
|
||||
val htlc_ab = Seq(
|
||||
buildHtlcIn(0, channelId_ab_1, paymentHash1), // not relayed
|
||||
buildHtlcIn(1, channelId_ab_1, paymentHash1), // channel relayed
|
||||
buildHtlcIn(2, channelId_ab_1, paymentHash2), // trampoline relayed
|
||||
)
|
||||
// The first upstream HTLC was not relayed but has a pending on-the-fly funding proposal.
|
||||
val downstreamChannel = OnTheFlyFunding.Pending(Seq(OnTheFlyFunding.Proposal(createWillAdd(100_000 msat, paymentHash1, CltvExpiry(500)), Upstream.Hot.Channel(htlc_ab(0).add, TimestampMilli.now(), a))), createStatus())
|
||||
nodeParams.db.liquidity.addPendingOnTheFlyFunding(randomKey().publicKey, downstreamChannel)
|
||||
// The other two HTLCs were relayed after completing on-the-fly funding.
|
||||
val htlc_bc = Seq(
|
||||
buildHtlcOut(1, channelId_bc_1, paymentHash1).modify(_.add.tlvStream).setTo(TlvStream(UpdateAddHtlcTlv.FundingFeeTlv(LiquidityAds.FundingFee(2500 msat, TxId(randomBytes32()))))), // channel relayed
|
||||
buildHtlcOut(2, channelId_bc_1, paymentHash2).modify(_.add.tlvStream).setTo(TlvStream(UpdateAddHtlcTlv.FundingFeeTlv(LiquidityAds.FundingFee(1500 msat, TxId(randomBytes32()))))), // trampoline relayed
|
||||
)
|
||||
|
||||
val upstreamChannel = Upstream.Cold.Channel(htlc_ab(1).add)
|
||||
val upstreamTrampoline = Upstream.Cold.Trampoline(Upstream.Cold.Channel(htlc_ab(2).add) :: Nil)
|
||||
val data_ab = ChannelCodecsSpec.makeChannelDataNormal(htlc_ab, Map.empty)
|
||||
val data_bc = ChannelCodecsSpec.makeChannelDataNormal(htlc_bc, Map(1L -> Origin.Cold(upstreamChannel), 2L -> Origin.Cold(upstreamTrampoline)))
|
||||
|
||||
val (relayer, _) = f.createRelayer(nodeParams)
|
||||
relayer ! PostRestartHtlcCleaner.Init(Seq(data_ab, data_bc))
|
||||
|
||||
// HTLC failures are not relayed upstream, as we will retry until we reach the HTLC timeout.
|
||||
sender.send(relayer, buildForwardFail(htlc_bc(0).add, Upstream.Cold.Channel(htlc_ab(0).add)))
|
||||
sender.send(relayer, buildForwardFail(htlc_bc(0).add, upstreamChannel))
|
||||
sender.send(relayer, buildForwardOnChainFail(htlc_bc(0).add, upstreamChannel))
|
||||
sender.send(relayer, buildForwardFail(htlc_bc(1).add, upstreamTrampoline))
|
||||
sender.send(relayer, buildForwardOnChainFail(htlc_bc(1).add, upstreamTrampoline))
|
||||
register.expectNoMessage(100 millis)
|
||||
|
||||
// HTLC fulfills are relayed upstream as soon as available.
|
||||
sender.send(relayer, buildForwardFulfill(htlc_bc(0).add, upstreamChannel, preimage1))
|
||||
val fulfill1 = register.expectMsgType[Register.Forward[CMD_FULFILL_HTLC]]
|
||||
assert(fulfill1.channelId == channelId_ab_1)
|
||||
assert(fulfill1.message.id == 1)
|
||||
assert(fulfill1.message.r == preimage1)
|
||||
sender.send(relayer, buildForwardFulfill(htlc_bc(1).add, upstreamTrampoline, preimage2))
|
||||
val fulfill2 = register.expectMsgType[Register.Forward[CMD_FULFILL_HTLC]]
|
||||
assert(fulfill2.channelId == channelId_ab_1)
|
||||
assert(fulfill2.message.id == 2)
|
||||
assert(fulfill2.message.r == preimage2)
|
||||
register.expectNoMessage(100 millis)
|
||||
}
|
||||
|
||||
test("relayed standard->non-standard HTLC is retained") { f =>
|
||||
import f._
|
||||
|
||||
@ -702,7 +787,7 @@ object PostRestartHtlcCleanerSpec {
|
||||
val (route, recipient) = (Route(finalAmount, hops, None), SpontaneousRecipient(e, finalAmount, finalExpiry, randomBytes32()))
|
||||
buildOutgoingPayment(TestConstants.emptyOrigin, paymentHash, route, recipient, 1.0)
|
||||
}
|
||||
UpdateAddHtlc(channelId, htlcId, payment.cmd.amount, paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, payment.cmd.nextBlindingKey_opt, 1.0)
|
||||
UpdateAddHtlc(channelId, htlcId, payment.cmd.amount, paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, payment.cmd.nextBlindingKey_opt, 1.0, payment.cmd.fundingFee_opt)
|
||||
}
|
||||
|
||||
def buildHtlcIn(htlcId: Long, channelId: ByteVector32, paymentHash: ByteVector32, blinded: Boolean = false): DirectedHtlc = IncomingHtlc(buildHtlc(htlcId, channelId, paymentHash, blinded))
|
||||
@ -711,7 +796,7 @@ object PostRestartHtlcCleanerSpec {
|
||||
|
||||
def buildFinalHtlc(htlcId: Long, channelId: ByteVector32, paymentHash: ByteVector32): DirectedHtlc = {
|
||||
val Right(payment) = buildOutgoingPayment(TestConstants.emptyOrigin, paymentHash, Route(finalAmount, Seq(channelHopFromUpdate(a, b, channelUpdate_ab)), None), SpontaneousRecipient(b, finalAmount, finalExpiry, randomBytes32()), 1.0)
|
||||
IncomingHtlc(UpdateAddHtlc(channelId, htlcId, payment.cmd.amount, paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, None, 1.0))
|
||||
IncomingHtlc(UpdateAddHtlc(channelId, htlcId, payment.cmd.amount, paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, None, 1.0, None))
|
||||
}
|
||||
|
||||
def buildForwardFail(add: UpdateAddHtlc, upstream: Upstream.Cold): RES_ADD_SETTLED[Origin.Cold, HtlcResult.Fail] =
|
||||
@ -734,11 +819,11 @@ object PostRestartHtlcCleanerSpec {
|
||||
val parentId = UUID.randomUUID()
|
||||
val (id1, id2, id3) = (UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID())
|
||||
|
||||
val add1 = UpdateAddHtlc(channelId_bc_1, 72, 561 msat, paymentHash1, CltvExpiry(4200), onionRoutingPacket = TestConstants.emptyOnionPacket, blinding_opt = None, confidence = 1.0)
|
||||
val add1 = UpdateAddHtlc(channelId_bc_1, 72, 561 msat, paymentHash1, CltvExpiry(4200), onionRoutingPacket = TestConstants.emptyOnionPacket, blinding_opt = None, confidence = 1.0, fundingFee_opt = None)
|
||||
val origin1 = Origin.Cold(Upstream.Local(id1))
|
||||
val add2 = UpdateAddHtlc(channelId_bc_1, 75, 1105 msat, paymentHash2, CltvExpiry(4250), onionRoutingPacket = TestConstants.emptyOnionPacket, blinding_opt = None, confidence = 1.0)
|
||||
val add2 = UpdateAddHtlc(channelId_bc_1, 75, 1105 msat, paymentHash2, CltvExpiry(4250), onionRoutingPacket = TestConstants.emptyOnionPacket, blinding_opt = None, confidence = 1.0, fundingFee_opt = None)
|
||||
val origin2 = Origin.Cold(Upstream.Local(id2))
|
||||
val add3 = UpdateAddHtlc(channelId_bc_1, 78, 1729 msat, paymentHash2, CltvExpiry(4300), onionRoutingPacket = TestConstants.emptyOnionPacket, blinding_opt = None, confidence = 1.0)
|
||||
val add3 = UpdateAddHtlc(channelId_bc_1, 78, 1729 msat, paymentHash2, CltvExpiry(4300), onionRoutingPacket = TestConstants.emptyOnionPacket, blinding_opt = None, confidence = 1.0, fundingFee_opt = None)
|
||||
val origin3 = Origin.Cold(Upstream.Local(id3))
|
||||
|
||||
// Prepare channels and payment state before restart.
|
||||
@ -763,7 +848,7 @@ object PostRestartHtlcCleanerSpec {
|
||||
buildHtlcIn(0, channelId_ab_1, paymentHash1)
|
||||
)
|
||||
|
||||
val upstream_1 = Upstream.Cold.Channel(htlc_ab_1.head.add.channelId, htlc_ab_1.head.add.id, htlc_ab_1.head.add.amountMsat)
|
||||
val upstream_1 = Upstream.Cold.Channel(htlc_ab_1.head.add)
|
||||
|
||||
val htlc_bc_1 = Seq(
|
||||
buildHtlcOut(6, channelId_bc_1, paymentHash1)
|
||||
@ -847,10 +932,10 @@ object PostRestartHtlcCleanerSpec {
|
||||
|
||||
val notRelayed = Set((1L, channelId_bc_1), (0L, channelId_bc_2), (3L, channelId_bc_3), (5L, channelId_bc_3), (7L, channelId_bc_4))
|
||||
|
||||
val downstream_1_1 = UpdateAddHtlc(channelId_bc_1, 6L, finalAmount, paymentHash1, finalExpiry, TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val downstream_2_1 = UpdateAddHtlc(channelId_bc_1, 8L, finalAmount, paymentHash2, finalExpiry, TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val downstream_2_2 = UpdateAddHtlc(channelId_bc_2, 1L, finalAmount, paymentHash2, finalExpiry, TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val downstream_2_3 = UpdateAddHtlc(channelId_bc_3, 4L, finalAmount, paymentHash2, finalExpiry, TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val downstream_1_1 = UpdateAddHtlc(channelId_bc_1, 6L, finalAmount, paymentHash1, finalExpiry, TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
val downstream_2_1 = UpdateAddHtlc(channelId_bc_1, 8L, finalAmount, paymentHash2, finalExpiry, TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
val downstream_2_2 = UpdateAddHtlc(channelId_bc_2, 1L, finalAmount, paymentHash2, finalExpiry, TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
val downstream_2_3 = UpdateAddHtlc(channelId_bc_3, 4L, finalAmount, paymentHash2, finalExpiry, TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
|
||||
val data_ab_1 = ChannelCodecsSpec.makeChannelDataNormal(htlc_ab_1, Map.empty)
|
||||
val data_ab_2 = ChannelCodecsSpec.makeChannelDataNormal(htlc_ab_2, Map.empty)
|
||||
|
@ -129,7 +129,7 @@ class AsyncPaymentTriggererSpec extends ScalaTestWithActorTestKit(ConfigFactory.
|
||||
|
||||
triggerer ! Watch(probe.ref, remoteNodeId, paymentHash = ByteVector32.Zeroes, timeout = BlockHeight(100))
|
||||
val request1 = switchboard.expectMessageType[Switchboard.GetPeerInfo]
|
||||
request1.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.DISCONNECTED, None, Set(TestProbe().ref.toClassic))
|
||||
request1.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.DISCONNECTED, None, None, Set(TestProbe().ref.toClassic))
|
||||
|
||||
// An unrelated peer connects.
|
||||
system.eventStream ! EventStream.Publish(PeerConnected(peer.ref.toClassic, randomKey().publicKey, null))
|
||||
@ -138,7 +138,7 @@ class AsyncPaymentTriggererSpec extends ScalaTestWithActorTestKit(ConfigFactory.
|
||||
// The target peer connects.
|
||||
system.eventStream ! EventStream.Publish(PeerConnected(peer.ref.toClassic, remoteNodeId, null))
|
||||
val request2 = switchboard.expectMessageType[Switchboard.GetPeerInfo]
|
||||
request2.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.CONNECTED, None, Set(TestProbe().ref.toClassic))
|
||||
request2.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.CONNECTED, None, None, Set(TestProbe().ref.toClassic))
|
||||
peer.expectMessageType[Peer.GetPeerChannels].replyTo ! Peer.PeerChannels(remoteNodeId, Seq(Peer.ChannelInfo(null, NEGOTIATING, null)))
|
||||
probe.expectMessage(AsyncPaymentTriggered)
|
||||
|
||||
@ -152,7 +152,7 @@ class AsyncPaymentTriggererSpec extends ScalaTestWithActorTestKit(ConfigFactory.
|
||||
|
||||
triggerer ! Watch(probe.ref, remoteNodeId, paymentHash = ByteVector32.Zeroes, timeout = BlockHeight(100))
|
||||
val request1 = switchboard.expectMessageType[Switchboard.GetPeerInfo]
|
||||
request1.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.DISCONNECTED, None, Set(TestProbe().ref.toClassic))
|
||||
request1.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.DISCONNECTED, None, None, Set(TestProbe().ref.toClassic))
|
||||
|
||||
// Another async payment node relay watches the peer
|
||||
val probe2 = TestProbe[Result]()
|
||||
@ -165,7 +165,7 @@ class AsyncPaymentTriggererSpec extends ScalaTestWithActorTestKit(ConfigFactory.
|
||||
// Second watch succeeds
|
||||
system.eventStream ! EventStream.Publish(PeerConnected(peer.ref.toClassic, remoteNodeId, null))
|
||||
val request2 = switchboard.expectMessageType[Switchboard.GetPeerInfo]
|
||||
request2.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.CONNECTED, None, Set(TestProbe().ref.toClassic))
|
||||
request2.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.CONNECTED, None, None, Set(TestProbe().ref.toClassic))
|
||||
peer.expectMessageType[Peer.GetPeerChannels].replyTo ! Peer.PeerChannels(remoteNodeId, Seq(Peer.ChannelInfo(null, NEGOTIATING, null)))
|
||||
probe.expectNoMessage(100 millis)
|
||||
probe2.expectMessage(AsyncPaymentTriggered)
|
||||
@ -177,14 +177,14 @@ class AsyncPaymentTriggererSpec extends ScalaTestWithActorTestKit(ConfigFactory.
|
||||
// watch remote node
|
||||
triggerer ! Watch(probe.ref, remoteNodeId, paymentHash = ByteVector32.Zeroes, timeout = BlockHeight(100))
|
||||
val request1 = switchboard.expectMessageType[Switchboard.GetPeerInfo]
|
||||
request1.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.DISCONNECTED, None, Set(TestProbe().ref.toClassic))
|
||||
request1.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.DISCONNECTED, None, None, Set(TestProbe().ref.toClassic))
|
||||
|
||||
// watch another remote node
|
||||
val remoteNodeId2 = TestConstants.Bob.nodeParams.nodeId
|
||||
val probe2 = TestProbe[Result]()
|
||||
triggerer ! Watch(probe2.ref, remoteNodeId2, paymentHash = ByteVector32.Zeroes, timeout = BlockHeight(101))
|
||||
val request2 = switchboard.expectMessageType[Switchboard.GetPeerInfo]
|
||||
request2.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId2, Peer.DISCONNECTED, None, Set(TestProbe().ref.toClassic))
|
||||
request2.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId2, Peer.DISCONNECTED, None, None, Set(TestProbe().ref.toClassic))
|
||||
|
||||
// First remote node times out
|
||||
system.eventStream ! EventStream.Publish(CurrentBlockHeight(BlockHeight(100)))
|
||||
@ -194,7 +194,7 @@ class AsyncPaymentTriggererSpec extends ScalaTestWithActorTestKit(ConfigFactory.
|
||||
system.eventStream ! EventStream.Publish(PeerConnected(peer.ref.toClassic, remoteNodeId2, null))
|
||||
val request3 = switchboard.expectMessageType[Switchboard.GetPeerInfo]
|
||||
assert(request3.remoteNodeId == remoteNodeId2)
|
||||
request3.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId2, Peer.CONNECTED, None, Set(TestProbe().ref.toClassic))
|
||||
request3.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId2, Peer.CONNECTED, None, None, Set(TestProbe().ref.toClassic))
|
||||
peer.expectMessageType[Peer.GetPeerChannels].replyTo ! Peer.PeerChannels(remoteNodeId, Seq(Peer.ChannelInfo(null, NEGOTIATING, null)))
|
||||
probe.expectNoMessage(100 millis)
|
||||
probe2.expectMessage(AsyncPaymentTriggered)
|
||||
|
@ -28,6 +28,7 @@ import fr.acinq.bitcoin.scalacompat.{Block, ByteVector32, ByteVector64, Crypto,
|
||||
import fr.acinq.eclair.Features.ScidAlias
|
||||
import fr.acinq.eclair.TestConstants.emptyOnionPacket
|
||||
import fr.acinq.eclair.blockchain.fee.FeeratePerKw
|
||||
import fr.acinq.eclair.channel.Register.ForwardNodeId
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.crypto.Sphinx
|
||||
import fr.acinq.eclair.io.{Peer, PeerReadyManager, Switchboard}
|
||||
@ -53,14 +54,29 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a
|
||||
|
||||
val wakeUpEnabled = "wake_up_enabled"
|
||||
val wakeUpTimeout = "wake_up_timeout"
|
||||
val onTheFlyFunding = "on_the_fly_funding"
|
||||
|
||||
case class FixtureParam(nodeParams: NodeParams, channelRelayer: typed.ActorRef[ChannelRelayer.Command], register: TestProbe[Any])
|
||||
case class FixtureParam(nodeParams: NodeParams, channelRelayer: typed.ActorRef[ChannelRelayer.Command], register: TestProbe[Any]) {
|
||||
def createWakeUpActors(): (TestProbe[PeerReadyManager.Register], TestProbe[Switchboard.GetPeerInfo]) = {
|
||||
val peerReadyManager = TestProbe[PeerReadyManager.Register]()
|
||||
system.receptionist ! Receptionist.Register(PeerReadyManager.PeerReadyManagerServiceKey, peerReadyManager.ref)
|
||||
val switchboard = TestProbe[Switchboard.GetPeerInfo]()
|
||||
system.receptionist ! Receptionist.Register(Switchboard.SwitchboardServiceKey, switchboard.ref)
|
||||
(peerReadyManager, switchboard)
|
||||
}
|
||||
|
||||
def cleanUpWakeUpActors(peerReadyManager: TestProbe[PeerReadyManager.Register], switchboard: TestProbe[Switchboard.GetPeerInfo]): Unit = {
|
||||
system.receptionist ! Receptionist.Deregister(PeerReadyManager.PeerReadyManagerServiceKey, peerReadyManager.ref)
|
||||
system.receptionist ! Receptionist.Deregister(Switchboard.SwitchboardServiceKey, switchboard.ref)
|
||||
}
|
||||
}
|
||||
|
||||
override def withFixture(test: OneArgTest): Outcome = {
|
||||
// we are node B in the route A -> B -> C -> ....
|
||||
val nodeParams = TestConstants.Bob.nodeParams
|
||||
.modify(_.peerWakeUpConfig.enabled).setToIf(test.tags.contains(wakeUpEnabled))(true)
|
||||
.modify(_.peerWakeUpConfig.timeout).setToIf(test.tags.contains(wakeUpTimeout))(100 millis)
|
||||
.modify(_.features.activated).usingIf(test.tags.contains(onTheFlyFunding))(_ + (Features.OnTheFlyFunding -> FeatureSupport.Optional))
|
||||
val register = TestProbe[Any]("register")
|
||||
val channelRelayer = testKit.spawn(ChannelRelayer.apply(nodeParams, register.ref.toClassic))
|
||||
try {
|
||||
@ -178,11 +194,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a
|
||||
test("relay blinded payment (wake up wallet node)", Tag(wakeUpEnabled)) { f =>
|
||||
import f._
|
||||
|
||||
val peerReadyManager = TestProbe[PeerReadyManager.Register]()
|
||||
system.receptionist ! Receptionist.Register(PeerReadyManager.PeerReadyManagerServiceKey, peerReadyManager.ref)
|
||||
val switchboard = TestProbe[Switchboard.GetPeerInfo]()
|
||||
system.receptionist ! Receptionist.Register(Switchboard.SwitchboardServiceKey, switchboard.ref)
|
||||
|
||||
val (peerReadyManager, switchboard) = createWakeUpActors()
|
||||
val u = createLocalUpdate(channelId1, feeBaseMsat = 2500 msat, feeProportionalMillionths = 0)
|
||||
Seq(true, false).foreach(isIntroduction => {
|
||||
val payload = createBlindedPayload(Left(outgoingNodeId), u.channelUpdate, isIntroduction)
|
||||
@ -195,12 +207,118 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a
|
||||
peerReadyManager.expectMessageType[PeerReadyManager.Register].replyTo ! PeerReadyManager.Registered(outgoingNodeId, otherAttempts = 0)
|
||||
val wakeUp = switchboard.expectMessageType[Switchboard.GetPeerInfo]
|
||||
assert(wakeUp.remoteNodeId == outgoingNodeId)
|
||||
wakeUp.replyTo ! Peer.PeerInfo(TestProbe[Any]().ref.toClassic, outgoingNodeId, Peer.CONNECTED, None, Set.empty)
|
||||
wakeUp.replyTo ! Peer.PeerInfo(TestProbe[Any]().ref.toClassic, outgoingNodeId, Peer.CONNECTED, Some(nodeParams.features.initFeatures()), None, Set.empty)
|
||||
expectFwdAdd(register, channelIds(realScid1), outgoingAmount, outgoingExpiry, 7)
|
||||
})
|
||||
|
||||
system.receptionist ! Receptionist.Deregister(PeerReadyManager.PeerReadyManagerServiceKey, peerReadyManager.ref)
|
||||
system.receptionist ! Receptionist.Deregister(Switchboard.SwitchboardServiceKey, switchboard.ref)
|
||||
cleanUpWakeUpActors(peerReadyManager, switchboard)
|
||||
}
|
||||
|
||||
test("relay blinded payment (on-the-fly funding)", Tag(wakeUpEnabled), Tag(onTheFlyFunding)) { f =>
|
||||
import f._
|
||||
|
||||
val (peerReadyManager, switchboard) = createWakeUpActors()
|
||||
|
||||
val u = createLocalUpdate(channelId1, feeBaseMsat = 5000 msat, feeProportionalMillionths = 0)
|
||||
val payload = createBlindedPayload(Left(outgoingNodeId), u.channelUpdate, isIntroduction = false)
|
||||
val r = createValidIncomingPacket(payload, outgoingAmount + u.channelUpdate.feeBaseMsat, outgoingExpiry + u.channelUpdate.cltvExpiryDelta)
|
||||
|
||||
channelRelayer ! WrappedLocalChannelUpdate(u)
|
||||
channelRelayer ! Relay(r, TestConstants.Alice.nodeParams.nodeId)
|
||||
|
||||
// We try to wake-up the next node.
|
||||
val wakeUp = peerReadyManager.expectMessageType[PeerReadyManager.Register]
|
||||
assert(wakeUp.remoteNodeId == outgoingNodeId)
|
||||
wakeUp.replyTo ! PeerReadyManager.Registered(outgoingNodeId, otherAttempts = 0)
|
||||
val peerInfo = switchboard.expectMessageType[Switchboard.GetPeerInfo]
|
||||
assert(peerInfo.remoteNodeId == outgoingNodeId)
|
||||
peerInfo.replyTo ! Peer.PeerInfo(TestProbe[Any]().ref.toClassic, outgoingNodeId, Peer.CONNECTED, Some(nodeParams.features.initFeatures()), None, Set.empty)
|
||||
cleanUpWakeUpActors(peerReadyManager, switchboard)
|
||||
|
||||
// We try to use existing channels, but they don't have enough liquidity.
|
||||
val fwd = expectFwdAdd(register, channelIds(realScid1), outgoingAmount, outgoingExpiry, 7)
|
||||
fwd.message.replyTo ! RES_ADD_FAILED(fwd.message, InsufficientFunds(channelIds(realScid1), outgoingAmount, 100 sat, 0 sat, 0 sat), Some(u.channelUpdate))
|
||||
|
||||
val fwdNodeId = register.expectMessageType[ForwardNodeId[Peer.ProposeOnTheFlyFunding]]
|
||||
assert(fwdNodeId.nodeId == outgoingNodeId)
|
||||
assert(fwdNodeId.message.nextBlindingKey_opt.nonEmpty)
|
||||
assert(fwdNodeId.message.amount == outgoingAmount)
|
||||
assert(fwdNodeId.message.expiry == outgoingExpiry)
|
||||
}
|
||||
|
||||
test("relay blinded payment (on-the-fly funding not supported)", Tag(wakeUpEnabled), Tag(onTheFlyFunding)) { f =>
|
||||
import f._
|
||||
|
||||
val (peerReadyManager, switchboard) = createWakeUpActors()
|
||||
|
||||
val u = createLocalUpdate(channelId1, feeBaseMsat = 5000 msat, feeProportionalMillionths = 0)
|
||||
val payload = createBlindedPayload(Left(outgoingNodeId), u.channelUpdate, isIntroduction = false)
|
||||
val r = createValidIncomingPacket(payload, outgoingAmount + u.channelUpdate.feeBaseMsat, outgoingExpiry + u.channelUpdate.cltvExpiryDelta)
|
||||
|
||||
channelRelayer ! Relay(r, TestConstants.Alice.nodeParams.nodeId)
|
||||
|
||||
// We try to wake-up the next node.
|
||||
peerReadyManager.expectMessageType[PeerReadyManager.Register].replyTo ! PeerReadyManager.Registered(outgoingNodeId, otherAttempts = 1)
|
||||
val peerInfo = switchboard.expectMessageType[Switchboard.GetPeerInfo]
|
||||
assert(peerInfo.remoteNodeId == outgoingNodeId)
|
||||
// The next node doesn't support the on-the-fly funding feature.
|
||||
peerInfo.replyTo ! Peer.PeerInfo(TestProbe[Any]().ref.toClassic, outgoingNodeId, Peer.CONNECTED, Some(Features.empty), None, Set.empty)
|
||||
cleanUpWakeUpActors(peerReadyManager, switchboard)
|
||||
|
||||
// We fail without attempting on-the-fly funding.
|
||||
expectFwdFail(register, r.add.channelId, CMD_FAIL_MALFORMED_HTLC(r.add.id, Sphinx.hash(r.add.onionRoutingPacket), InvalidOnionBlinding(Sphinx.hash(r.add.onionRoutingPacket)).code, commit = true))
|
||||
}
|
||||
|
||||
test("relay blinded payment (on-the-fly funding failed)", Tag(wakeUpEnabled), Tag(onTheFlyFunding)) { f =>
|
||||
import f._
|
||||
|
||||
val (peerReadyManager, switchboard) = createWakeUpActors()
|
||||
|
||||
val u = createLocalUpdate(channelId1, feeBaseMsat = 5000 msat, feeProportionalMillionths = 0)
|
||||
val payload = createBlindedPayload(Left(outgoingNodeId), u.channelUpdate, isIntroduction = false)
|
||||
val r = createValidIncomingPacket(payload, outgoingAmount + u.channelUpdate.feeBaseMsat, outgoingExpiry + u.channelUpdate.cltvExpiryDelta)
|
||||
|
||||
channelRelayer ! Relay(r, TestConstants.Alice.nodeParams.nodeId)
|
||||
|
||||
// We try to wake-up the next node.
|
||||
peerReadyManager.expectMessageType[PeerReadyManager.Register].replyTo ! PeerReadyManager.Registered(outgoingNodeId, otherAttempts = 1)
|
||||
val peerInfo = switchboard.expectMessageType[Switchboard.GetPeerInfo]
|
||||
assert(peerInfo.remoteNodeId == outgoingNodeId)
|
||||
peerInfo.replyTo ! Peer.PeerInfo(TestProbe[Any]().ref.toClassic, outgoingNodeId, Peer.CONNECTED, Some(nodeParams.features.initFeatures()), None, Set.empty)
|
||||
cleanUpWakeUpActors(peerReadyManager, switchboard)
|
||||
|
||||
// We don't have any channel, so we attempt on-the-fly funding, but the peer is not available.
|
||||
val fwdNodeId = register.expectMessageType[ForwardNodeId[Peer.ProposeOnTheFlyFunding]]
|
||||
assert(fwdNodeId.nodeId == outgoingNodeId)
|
||||
fwdNodeId.replyTo ! Register.ForwardNodeIdFailure(fwdNodeId)
|
||||
expectFwdFail(register, r.add.channelId, CMD_FAIL_MALFORMED_HTLC(r.add.id, Sphinx.hash(r.add.onionRoutingPacket), InvalidOnionBlinding(Sphinx.hash(r.add.onionRoutingPacket)).code, commit = true))
|
||||
}
|
||||
|
||||
test("relay blinded payment (on-the-fly funding not attempted)", Tag(wakeUpEnabled), Tag(onTheFlyFunding)) { f =>
|
||||
import f._
|
||||
|
||||
val (peerReadyManager, switchboard) = createWakeUpActors()
|
||||
|
||||
val u = createLocalUpdate(channelId1, feeBaseMsat = 5000 msat, feeProportionalMillionths = 0)
|
||||
val payload = createBlindedPayload(Left(outgoingNodeId), u.channelUpdate, isIntroduction = false)
|
||||
val r = createValidIncomingPacket(payload, outgoingAmount + u.channelUpdate.feeBaseMsat, outgoingExpiry + u.channelUpdate.cltvExpiryDelta)
|
||||
|
||||
channelRelayer ! WrappedLocalChannelUpdate(u)
|
||||
channelRelayer ! Relay(r, TestConstants.Alice.nodeParams.nodeId)
|
||||
|
||||
// We try to wake-up the next node.
|
||||
peerReadyManager.expectMessageType[PeerReadyManager.Register].replyTo ! PeerReadyManager.Registered(outgoingNodeId, otherAttempts = 0)
|
||||
val peerInfo = switchboard.expectMessageType[Switchboard.GetPeerInfo]
|
||||
assert(peerInfo.remoteNodeId == outgoingNodeId)
|
||||
peerInfo.replyTo ! Peer.PeerInfo(TestProbe[Any]().ref.toClassic, outgoingNodeId, Peer.CONNECTED, Some(nodeParams.features.initFeatures()), None, Set.empty)
|
||||
cleanUpWakeUpActors(peerReadyManager, switchboard)
|
||||
|
||||
// We try to use existing channels, but they reject the payment for a reason that isn't tied to the liquidity.
|
||||
val fwd = expectFwdAdd(register, channelIds(realScid1), outgoingAmount, outgoingExpiry, 7)
|
||||
fwd.message.replyTo ! RES_ADD_FAILED(fwd.message, TooManyAcceptedHtlcs(channelIds(realScid1), 10), Some(u.channelUpdate))
|
||||
|
||||
// We fail without attempting on-the-fly funding.
|
||||
expectFwdFail(register, r.add.channelId, CMD_FAIL_MALFORMED_HTLC(r.add.id, Sphinx.hash(r.add.onionRoutingPacket), InvalidOnionBlinding(Sphinx.hash(r.add.onionRoutingPacket)).code, commit = true))
|
||||
}
|
||||
|
||||
test("relay with retries") { f =>
|
||||
@ -333,10 +451,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a
|
||||
test("fail to relay blinded payment (cannot wake up remote node)", Tag(wakeUpEnabled), Tag(wakeUpTimeout)) { f =>
|
||||
import f._
|
||||
|
||||
val peerReadyManager = TestProbe[PeerReadyManager.Register]()
|
||||
system.receptionist ! Receptionist.Register(PeerReadyManager.PeerReadyManagerServiceKey, peerReadyManager.ref)
|
||||
val switchboard = TestProbe[Switchboard.GetPeerInfo]()
|
||||
system.receptionist ! Receptionist.Register(Switchboard.SwitchboardServiceKey, switchboard.ref)
|
||||
val (peerReadyManager, switchboard) = createWakeUpActors()
|
||||
|
||||
val u = createLocalUpdate(channelId1, feeBaseMsat = 2500 msat, feeProportionalMillionths = 0)
|
||||
val payload = createBlindedPayload(Left(outgoingNodeId), u.channelUpdate, isIntroduction = true)
|
||||
@ -351,8 +466,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a
|
||||
val fail = register.expectMessageType[Register.Forward[CMD_FAIL_HTLC]]
|
||||
assert(fail.message.reason.contains(InvalidOnionBlinding(Sphinx.hash(r.add.onionRoutingPacket))))
|
||||
|
||||
system.receptionist ! Receptionist.Deregister(PeerReadyManager.PeerReadyManagerServiceKey, peerReadyManager.ref)
|
||||
system.receptionist ! Receptionist.Deregister(Switchboard.SwitchboardServiceKey, switchboard.ref)
|
||||
cleanUpWakeUpActors(peerReadyManager, switchboard)
|
||||
}
|
||||
|
||||
test("relay when expiry larger than our requirements") { f =>
|
||||
@ -543,7 +657,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a
|
||||
val payload = ChannelRelay.Standard(realScid1, outgoingAmount, outgoingExpiry)
|
||||
val r = createValidIncomingPacket(payload, outgoingAmount + u.channelUpdate.feeBaseMsat, outgoingExpiry + u.channelUpdate.cltvExpiryDelta)
|
||||
val u_disabled = createLocalUpdate(channelId1, enabled = false)
|
||||
val downstream_htlc = UpdateAddHtlc(channelId1, 7, outgoingAmount, paymentHash, outgoingExpiry, emptyOnionPacket, None, 1.0)
|
||||
val downstream_htlc = UpdateAddHtlc(channelId1, 7, outgoingAmount, paymentHash, outgoingExpiry, emptyOnionPacket, None, 1.0, None)
|
||||
|
||||
case class TestCase(result: HtlcResult, cmd: channel.HtlcSettlementCommand)
|
||||
|
||||
@ -569,7 +683,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a
|
||||
import f._
|
||||
|
||||
val u = createLocalUpdate(channelId1, feeBaseMsat = 5000 msat, feeProportionalMillionths = 0)
|
||||
val downstream = UpdateAddHtlc(channelId1, 7, outgoingAmount, paymentHash, outgoingExpiry, emptyOnionPacket, None, 0.0625)
|
||||
val downstream = UpdateAddHtlc(channelId1, 7, outgoingAmount, paymentHash, outgoingExpiry, emptyOnionPacket, None, 0.0625, None)
|
||||
|
||||
val testCases = Seq(
|
||||
HtlcResult.RemoteFail(UpdateFailHtlc(channelId1, downstream.id, hex"deadbeef")),
|
||||
@ -615,7 +729,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a
|
||||
val payload = ChannelRelay.Standard(realScid1, outgoingAmount, outgoingExpiry)
|
||||
val r = createValidIncomingPacket(payload, endorsementIn = 3)
|
||||
val u = createLocalUpdate(channelId1)
|
||||
val downstream_htlc = UpdateAddHtlc(channelId1, 7, outgoingAmount, paymentHash, outgoingExpiry, emptyOnionPacket, None, 0.4375)
|
||||
val downstream_htlc = UpdateAddHtlc(channelId1, 7, outgoingAmount, paymentHash, outgoingExpiry, emptyOnionPacket, None, 0.4375, None)
|
||||
|
||||
case class TestCase(result: HtlcResult)
|
||||
|
||||
|
@ -69,6 +69,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl
|
||||
|
||||
val wakeUpEnabled = "wake_up_enabled"
|
||||
val wakeUpTimeout = "wake_up_timeout"
|
||||
val onTheFlyFunding = "on_the_fly_funding"
|
||||
|
||||
case class FixtureParam(nodeParams: NodeParams, router: TestProbe[Any], register: TestProbe[Any], mockPayFSM: TestProbe[Any], eventListener: TestProbe[PaymentEvent]) {
|
||||
def createNodeRelay(packetIn: IncomingPaymentPacket.NodeRelayPacket, useRealPaymentFactory: Boolean = false): (ActorRef[NodeRelay.Command], TestProbe[NodeRelayer.Command]) = {
|
||||
@ -77,6 +78,19 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl
|
||||
val nodeRelay = testKit.spawn(NodeRelay(nodeParams, parent.ref, register.ref.toClassic, relayId, packetIn, outgoingPaymentFactory, router.ref.toClassic))
|
||||
(nodeRelay, parent)
|
||||
}
|
||||
|
||||
def createWakeUpActors(): (TestProbe[PeerReadyManager.Register], TestProbe[Switchboard.GetPeerInfo]) = {
|
||||
val peerReadyManager = TestProbe[PeerReadyManager.Register]()
|
||||
system.receptionist ! Receptionist.Register(PeerReadyManager.PeerReadyManagerServiceKey, peerReadyManager.ref)
|
||||
val switchboard = TestProbe[Switchboard.GetPeerInfo]()
|
||||
system.receptionist ! Receptionist.Register(Switchboard.SwitchboardServiceKey, switchboard.ref)
|
||||
(peerReadyManager, switchboard)
|
||||
}
|
||||
|
||||
def cleanUpWakeUpActors(peerReadyManager: TestProbe[PeerReadyManager.Register], switchboard: TestProbe[Switchboard.GetPeerInfo]): Unit = {
|
||||
system.receptionist ! Receptionist.Deregister(PeerReadyManager.PeerReadyManagerServiceKey, peerReadyManager.ref)
|
||||
system.receptionist ! Receptionist.Deregister(Switchboard.SwitchboardServiceKey, switchboard.ref)
|
||||
}
|
||||
}
|
||||
|
||||
case class FakeOutgoingPaymentFactory(f: FixtureParam) extends NodeRelay.OutgoingPaymentFactory {
|
||||
@ -100,6 +114,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl
|
||||
.modify(_.relayParams.asyncPaymentsParams.holdTimeoutBlocks).setToIf(test.tags.contains("long_hold_timeout"))(200000) // timeout after payment expires
|
||||
.modify(_.peerWakeUpConfig.enabled).setToIf(test.tags.contains(wakeUpEnabled))(true)
|
||||
.modify(_.peerWakeUpConfig.timeout).setToIf(test.tags.contains(wakeUpTimeout))(100 millis)
|
||||
.modify(_.features.activated).usingIf(test.tags.contains(onTheFlyFunding))(_ + (Features.OnTheFlyFunding -> FeatureSupport.Optional))
|
||||
val router = TestProbe[Any]("router")
|
||||
val register = TestProbe[Any]("register")
|
||||
val eventListener = TestProbe[PaymentEvent]("event-listener")
|
||||
@ -230,7 +245,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl
|
||||
incomingMultiPart.foreach(incoming => nodeRelayer ! NodeRelay.Relay(incoming, randomKey().publicKey))
|
||||
// and then one extra
|
||||
val extra = IncomingPaymentPacket.RelayToTrampolinePacket(
|
||||
UpdateAddHtlc(randomBytes32(), Random.nextInt(100), 1000 msat, paymentHash, CltvExpiry(499990), TestConstants.emptyOnionPacket, None, 1.0),
|
||||
UpdateAddHtlc(randomBytes32(), Random.nextInt(100), 1000 msat, paymentHash, CltvExpiry(499990), TestConstants.emptyOnionPacket, None, 1.0, None),
|
||||
FinalPayload.Standard.createPayload(1000 msat, incomingAmount, CltvExpiry(499990), incomingSecret, None),
|
||||
IntermediatePayload.NodeRelay.Standard(outgoingAmount, outgoingExpiry, outgoingNodeId),
|
||||
createTrampolinePacket(outgoingAmount, outgoingExpiry))
|
||||
@ -253,13 +268,13 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl
|
||||
incomingMultiPart.foreach(incoming => nodeRelayer ! NodeRelay.Relay(incoming, randomKey().publicKey))
|
||||
|
||||
val outgoingCfg = mockPayFSM.expectMessageType[SendPaymentConfig]
|
||||
validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingMultiPart.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey))), 5)
|
||||
validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingMultiPart.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey)).toList), 5)
|
||||
val outgoingPayment = mockPayFSM.expectMessageType[SendMultiPartPayment]
|
||||
validateOutgoingPayment(outgoingPayment)
|
||||
|
||||
// Receive new extraneous multi-part HTLC.
|
||||
val i1 = IncomingPaymentPacket.RelayToTrampolinePacket(
|
||||
UpdateAddHtlc(randomBytes32(), Random.nextInt(100), 1000 msat, paymentHash, CltvExpiry(499990), TestConstants.emptyOnionPacket, None, 1.0),
|
||||
UpdateAddHtlc(randomBytes32(), Random.nextInt(100), 1000 msat, paymentHash, CltvExpiry(499990), TestConstants.emptyOnionPacket, None, 1.0, None),
|
||||
FinalPayload.Standard.createPayload(1000 msat, incomingAmount, CltvExpiry(499990), incomingSecret, None),
|
||||
IntermediatePayload.NodeRelay.Standard(outgoingAmount, outgoingExpiry, outgoingNodeId),
|
||||
createTrampolinePacket(outgoingAmount, outgoingExpiry))
|
||||
@ -272,7 +287,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl
|
||||
|
||||
// Receive new HTLC with different details, but for the same payment hash.
|
||||
val i2 = IncomingPaymentPacket.RelayToTrampolinePacket(
|
||||
UpdateAddHtlc(randomBytes32(), Random.nextInt(100), 1500 msat, paymentHash, CltvExpiry(499990), TestConstants.emptyOnionPacket, None, 1.0),
|
||||
UpdateAddHtlc(randomBytes32(), Random.nextInt(100), 1500 msat, paymentHash, CltvExpiry(499990), TestConstants.emptyOnionPacket, None, 1.0, None),
|
||||
PaymentOnion.FinalPayload.Standard.createPayload(1500 msat, 1500 msat, CltvExpiry(499990), incomingSecret, None),
|
||||
IntermediatePayload.NodeRelay.Standard(1250 msat, outgoingExpiry, outgoingNodeId),
|
||||
createTrampolinePacket(outgoingAmount, outgoingExpiry))
|
||||
@ -350,7 +365,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl
|
||||
|
||||
// upstream payment relayed
|
||||
val outgoingCfg = mockPayFSM.expectMessageType[SendPaymentConfig]
|
||||
validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingAsyncPayment.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey))), 5)
|
||||
validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingAsyncPayment.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey)).toList), 5)
|
||||
val outgoingPayment = mockPayFSM.expectMessageType[SendMultiPartPayment]
|
||||
validateOutgoingPayment(outgoingPayment)
|
||||
// those are adapters for pay-fsm messages
|
||||
@ -558,7 +573,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl
|
||||
nodeRelayer ! NodeRelay.Relay(incomingMultiPart.last, randomKey().publicKey)
|
||||
|
||||
val outgoingCfg = mockPayFSM.expectMessageType[SendPaymentConfig]
|
||||
validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingMultiPart.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey))), 5)
|
||||
validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingMultiPart.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey)).toList), 5)
|
||||
val outgoingPayment = mockPayFSM.expectMessageType[SendMultiPartPayment]
|
||||
validateOutgoingPayment(outgoingPayment)
|
||||
// those are adapters for pay-fsm messages
|
||||
@ -615,6 +630,83 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl
|
||||
register.expectNoMessage(100 millis)
|
||||
}
|
||||
|
||||
// The two tests below are disabled by default, since there is no default mechanism to flag the next trampoline node
|
||||
// as being a wallet node. Feature branches that support wallet software should restore those tests and flag the
|
||||
// outgoing node_id as being a wallet node.
|
||||
ignore("relay incoming multi-part payment with on-the-fly funding", Tag(wakeUpEnabled), Tag(onTheFlyFunding)) { f =>
|
||||
import f._
|
||||
|
||||
val (peerReadyManager, switchboard) = createWakeUpActors()
|
||||
|
||||
// Receive an upstream multi-part payment.
|
||||
val (nodeRelayer, parent) = f.createNodeRelay(incomingMultiPart.head)
|
||||
incomingMultiPart.foreach(p => nodeRelayer ! NodeRelay.Relay(p, randomKey().publicKey))
|
||||
|
||||
// The remote node is a wallet node: we wake them up before relaying the payment.
|
||||
val wakeUp = peerReadyManager.expectMessageType[PeerReadyManager.Register]
|
||||
assert(wakeUp.remoteNodeId == outgoingNodeId)
|
||||
wakeUp.replyTo ! PeerReadyManager.Registered(outgoingNodeId, otherAttempts = 0)
|
||||
val peerInfo = switchboard.expectMessageType[Switchboard.GetPeerInfo]
|
||||
assert(peerInfo.remoteNodeId == outgoingNodeId)
|
||||
peerInfo.replyTo ! Peer.PeerInfo(TestProbe[Any]().ref.toClassic, outgoingNodeId, Peer.CONNECTED, Some(nodeParams.features.initFeatures()), None, Set.empty)
|
||||
cleanUpWakeUpActors(peerReadyManager, switchboard)
|
||||
|
||||
val outgoingCfg = mockPayFSM.expectMessageType[SendPaymentConfig]
|
||||
validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingMultiPart.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey)).toList), 5)
|
||||
val outgoingPayment = mockPayFSM.expectMessageType[SendMultiPartPayment]
|
||||
validateOutgoingPayment(outgoingPayment)
|
||||
|
||||
// The outgoing payment fails because we don't have enough balance: we trigger on-the-fly funding.
|
||||
outgoingPayment.replyTo ! PaymentFailed(relayId, paymentHash, LocalFailure(outgoingAmount, Nil, BalanceTooLow) :: Nil)
|
||||
val fwd = register.expectMessageType[Register.ForwardNodeId[Peer.ProposeOnTheFlyFunding]]
|
||||
assert(fwd.nodeId == outgoingNodeId)
|
||||
assert(fwd.message.nextBlindingKey_opt.isEmpty)
|
||||
assert(fwd.message.onion.payload.size == PaymentOnionCodecs.paymentOnionPayloadLength)
|
||||
// We verify that the next node is able to decrypt the onion that we will send in will_add_htlc.
|
||||
val dummyAdd = UpdateAddHtlc(randomBytes32(), 0, fwd.message.amount, fwd.message.paymentHash, fwd.message.expiry, fwd.message.onion, None, 1.0, None)
|
||||
val Right(incoming) = IncomingPaymentPacket.decrypt(dummyAdd, outgoingNodeKey, nodeParams.features)
|
||||
assert(incoming.isInstanceOf[IncomingPaymentPacket.FinalPacket])
|
||||
val finalPayload = incoming.asInstanceOf[IncomingPaymentPacket.FinalPacket].payload.asInstanceOf[FinalPayload.Standard]
|
||||
assert(finalPayload.amount == fwd.message.amount)
|
||||
assert(finalPayload.expiry == fwd.message.expiry)
|
||||
assert(finalPayload.paymentSecret == paymentSecret)
|
||||
|
||||
// Once on-the-fly funding has been proposed, the payment isn't our responsibility anymore.
|
||||
fwd.message.replyTo ! Peer.ProposeOnTheFlyFundingResponse.Proposed
|
||||
parent.expectMessageType[NodeRelayer.RelayComplete]
|
||||
}
|
||||
|
||||
ignore("relay incoming multi-part payment with on-the-fly funding (non-liquidity failure)", Tag(wakeUpEnabled), Tag(onTheFlyFunding)) { f =>
|
||||
import f._
|
||||
|
||||
val (peerReadyManager, switchboard) = createWakeUpActors()
|
||||
|
||||
// Receive an upstream multi-part payment.
|
||||
val (nodeRelayer, parent) = f.createNodeRelay(incomingMultiPart.head)
|
||||
incomingMultiPart.foreach(p => nodeRelayer ! NodeRelay.Relay(p, randomKey().publicKey))
|
||||
|
||||
// The remote node is a wallet node: we wake them up before relaying the payment.
|
||||
peerReadyManager.expectMessageType[PeerReadyManager.Register].replyTo ! PeerReadyManager.Registered(outgoingNodeId, otherAttempts = 0)
|
||||
val peerInfo = switchboard.expectMessageType[Switchboard.GetPeerInfo]
|
||||
assert(peerInfo.remoteNodeId == outgoingNodeId)
|
||||
peerInfo.replyTo ! Peer.PeerInfo(TestProbe[Any]().ref.toClassic, outgoingNodeId, Peer.CONNECTED, Some(nodeParams.features.initFeatures()), None, Set.empty)
|
||||
cleanUpWakeUpActors(peerReadyManager, switchboard)
|
||||
|
||||
val outgoingCfg = mockPayFSM.expectMessageType[SendPaymentConfig]
|
||||
validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingMultiPart.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey)).toList), 5)
|
||||
val outgoingPayment = mockPayFSM.expectMessageType[SendMultiPartPayment]
|
||||
validateOutgoingPayment(outgoingPayment)
|
||||
|
||||
// The outgoing payment fails, but it's not a liquidity issue.
|
||||
outgoingPayment.replyTo ! PaymentFailed(relayId, paymentHash, RemoteFailure(outgoingAmount, Nil, Sphinx.DecryptedFailurePacket(outgoingNodeId, TemporaryNodeFailure())) :: Nil)
|
||||
incomingMultiPart.foreach { p =>
|
||||
val fwd = register.expectMessageType[Register.Forward[CMD_FAIL_HTLC]]
|
||||
assert(fwd.channelId == p.add.channelId)
|
||||
assert(fwd.message == CMD_FAIL_HTLC(p.add.id, Right(TemporaryNodeFailure()), commit = true))
|
||||
}
|
||||
parent.expectMessageType[NodeRelayer.RelayComplete]
|
||||
}
|
||||
|
||||
test("relay to non-trampoline recipient supporting multi-part") { f =>
|
||||
import f._
|
||||
|
||||
@ -629,7 +721,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl
|
||||
incomingPayments.foreach(incoming => nodeRelayer ! NodeRelay.Relay(incoming, randomKey().publicKey))
|
||||
|
||||
val outgoingCfg = mockPayFSM.expectMessageType[SendPaymentConfig]
|
||||
validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingMultiPart.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey))), 5)
|
||||
validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingMultiPart.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey)).toList), 5)
|
||||
val outgoingPayment = mockPayFSM.expectMessageType[SendMultiPartPayment]
|
||||
assert(outgoingPayment.recipient.nodeId == outgoingNodeId)
|
||||
assert(outgoingPayment.recipient.totalAmount == outgoingAmount)
|
||||
@ -673,7 +765,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl
|
||||
incomingPayments.foreach(incoming => nodeRelayer ! NodeRelay.Relay(incoming, randomKey().publicKey))
|
||||
|
||||
val outgoingCfg = mockPayFSM.expectMessageType[SendPaymentConfig]
|
||||
validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingMultiPart.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey))), 5)
|
||||
validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingMultiPart.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey)).toList), 5)
|
||||
val outgoingPayment = mockPayFSM.expectMessageType[SendPaymentToNode]
|
||||
assert(outgoingPayment.recipient.nodeId == outgoingNodeId)
|
||||
assert(outgoingPayment.amount == outgoingAmount)
|
||||
@ -731,7 +823,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl
|
||||
incomingPayments.foreach(incoming => nodeRelayer ! NodeRelay.Relay(incoming, randomKey().publicKey))
|
||||
|
||||
val outgoingCfg = mockPayFSM.expectMessageType[SendPaymentConfig]
|
||||
validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingPayments.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey))), 5, ignoreNodeId = true)
|
||||
validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingPayments.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey)).toList), 5, ignoreNodeId = true)
|
||||
val outgoingPayment = mockPayFSM.expectMessageType[SendPaymentToNode]
|
||||
assert(outgoingPayment.amount == outgoingAmount)
|
||||
assert(outgoingPayment.recipient.expiry == outgoingExpiry)
|
||||
@ -764,7 +856,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl
|
||||
incomingPayments.foreach(incoming => nodeRelayer ! NodeRelay.Relay(incoming, randomKey().publicKey))
|
||||
|
||||
val outgoingCfg = mockPayFSM.expectMessageType[SendPaymentConfig]
|
||||
validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingPayments.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey))), 5, ignoreNodeId = true)
|
||||
validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingPayments.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey)).toList), 5, ignoreNodeId = true)
|
||||
val outgoingPayment = mockPayFSM.expectMessageType[SendMultiPartPayment]
|
||||
assert(outgoingPayment.recipient.totalAmount == outgoingAmount)
|
||||
assert(outgoingPayment.recipient.expiry == outgoingExpiry)
|
||||
@ -792,10 +884,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl
|
||||
test("relay to blinded path with wake-up", Tag(wakeUpEnabled)) { f =>
|
||||
import f._
|
||||
|
||||
val peerReadyManager = TestProbe[PeerReadyManager.Register]()
|
||||
system.receptionist ! Receptionist.Register(PeerReadyManager.PeerReadyManagerServiceKey, peerReadyManager.ref)
|
||||
val switchboard = TestProbe[Switchboard.GetPeerInfo]()
|
||||
system.receptionist ! Receptionist.Register(Switchboard.SwitchboardServiceKey, switchboard.ref)
|
||||
val (peerReadyManager, switchboard) = createWakeUpActors()
|
||||
|
||||
val incomingPayments = createIncomingPaymentsToWalletBlindedPath(nodeParams)
|
||||
val (nodeRelayer, parent) = f.createNodeRelay(incomingPayments.head)
|
||||
@ -805,12 +894,11 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl
|
||||
peerReadyManager.expectMessageType[PeerReadyManager.Register].replyTo ! PeerReadyManager.Registered(outgoingNodeId, otherAttempts = 0)
|
||||
val wakeUp = switchboard.expectMessageType[Switchboard.GetPeerInfo]
|
||||
assert(wakeUp.remoteNodeId == outgoingNodeId)
|
||||
wakeUp.replyTo ! Peer.PeerInfo(TestProbe[Any]().ref.toClassic, outgoingNodeId, Peer.CONNECTED, None, Set.empty)
|
||||
system.receptionist ! Receptionist.Deregister(PeerReadyManager.PeerReadyManagerServiceKey, peerReadyManager.ref)
|
||||
system.receptionist ! Receptionist.Deregister(Switchboard.SwitchboardServiceKey, switchboard.ref)
|
||||
wakeUp.replyTo ! Peer.PeerInfo(TestProbe[Any]().ref.toClassic, outgoingNodeId, Peer.CONNECTED, Some(nodeParams.features.initFeatures()), None, Set.empty)
|
||||
cleanUpWakeUpActors(peerReadyManager, switchboard)
|
||||
|
||||
val outgoingCfg = mockPayFSM.expectMessageType[SendPaymentConfig]
|
||||
validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingPayments.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey))), 5, ignoreNodeId = true)
|
||||
validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingPayments.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey)).toList), 5, ignoreNodeId = true)
|
||||
val outgoingPayment = mockPayFSM.expectMessageType[SendMultiPartPayment]
|
||||
assert(outgoingPayment.recipient.totalAmount == outgoingAmount)
|
||||
assert(outgoingPayment.recipient.expiry == outgoingExpiry)
|
||||
@ -838,20 +926,16 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl
|
||||
test("fail to relay to blinded path when wake-up fails", Tag(wakeUpEnabled), Tag(wakeUpTimeout)) { f =>
|
||||
import f._
|
||||
|
||||
val peerReadyManager = TestProbe[PeerReadyManager.Register]()
|
||||
system.receptionist ! Receptionist.Register(PeerReadyManager.PeerReadyManagerServiceKey, peerReadyManager.ref)
|
||||
val switchboard = TestProbe[Switchboard.GetPeerInfo]()
|
||||
system.receptionist ! Receptionist.Register(Switchboard.SwitchboardServiceKey, switchboard.ref)
|
||||
val (peerReadyManager, switchboard) = createWakeUpActors()
|
||||
|
||||
val incomingPayments = createIncomingPaymentsToWalletBlindedPath(nodeParams)
|
||||
val (nodeRelayer, _) = f.createNodeRelay(incomingPayments.head)
|
||||
incomingPayments.foreach(incoming => nodeRelayer ! NodeRelay.Relay(incoming, randomKey().publicKey))
|
||||
|
||||
// The remote node is a wallet node: we try to wake them up before relaying the payment, but it times out.
|
||||
peerReadyManager.expectMessageType[PeerReadyManager.Register].replyTo ! PeerReadyManager.Registered(outgoingNodeId, otherAttempts = 0)
|
||||
peerReadyManager.expectMessageType[PeerReadyManager.Register].replyTo ! PeerReadyManager.Registered(outgoingNodeId, otherAttempts = 3)
|
||||
assert(switchboard.expectMessageType[Switchboard.GetPeerInfo].remoteNodeId == outgoingNodeId)
|
||||
system.receptionist ! Receptionist.Deregister(PeerReadyManager.PeerReadyManagerServiceKey, peerReadyManager.ref)
|
||||
system.receptionist ! Receptionist.Deregister(Switchboard.SwitchboardServiceKey, switchboard.ref)
|
||||
cleanUpWakeUpActors(peerReadyManager, switchboard)
|
||||
mockPayFSM.expectNoMessage(100 millis)
|
||||
|
||||
incomingPayments.foreach { p =>
|
||||
@ -861,6 +945,76 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl
|
||||
}
|
||||
}
|
||||
|
||||
test("relay to blinded path with on-the-fly funding", Tag(wakeUpEnabled), Tag(onTheFlyFunding)) { f =>
|
||||
import f._
|
||||
|
||||
val (peerReadyManager, switchboard) = createWakeUpActors()
|
||||
|
||||
val incomingPayments = createIncomingPaymentsToWalletBlindedPath(nodeParams)
|
||||
val (nodeRelayer, parent) = f.createNodeRelay(incomingPayments.head)
|
||||
incomingPayments.foreach(incoming => nodeRelayer ! NodeRelay.Relay(incoming, randomKey().publicKey))
|
||||
|
||||
// The remote node is a wallet node: we wake them up before relaying the payment.
|
||||
peerReadyManager.expectMessageType[PeerReadyManager.Register].replyTo ! PeerReadyManager.Registered(outgoingNodeId, otherAttempts = 1)
|
||||
val peerInfo = switchboard.expectMessageType[Switchboard.GetPeerInfo]
|
||||
peerInfo.replyTo ! Peer.PeerInfo(TestProbe[Any]().ref.toClassic, outgoingNodeId, Peer.CONNECTED, Some(nodeParams.features.initFeatures()), None, Set.empty)
|
||||
cleanUpWakeUpActors(peerReadyManager, switchboard)
|
||||
|
||||
val outgoingCfg = mockPayFSM.expectMessageType[SendPaymentConfig]
|
||||
validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingPayments.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey)).toList), 5, ignoreNodeId = true)
|
||||
val outgoingPayment = mockPayFSM.expectMessageType[SendMultiPartPayment]
|
||||
|
||||
// The outgoing payment fails because we don't have enough balance: we trigger on-the-fly funding.
|
||||
outgoingPayment.replyTo ! PaymentFailed(relayId, paymentHash, LocalFailure(outgoingAmount, Nil, BalanceTooLow) :: Nil)
|
||||
val fwd = register.expectMessageType[Register.ForwardNodeId[Peer.ProposeOnTheFlyFunding]]
|
||||
assert(fwd.nodeId == outgoingNodeId)
|
||||
assert(fwd.message.nextBlindingKey_opt.nonEmpty)
|
||||
assert(fwd.message.onion.payload.size == PaymentOnionCodecs.paymentOnionPayloadLength)
|
||||
// We verify that the next node is able to decrypt the onion that we will send in will_add_htlc.
|
||||
val dummyAdd = UpdateAddHtlc(randomBytes32(), 0, fwd.message.amount, fwd.message.paymentHash, fwd.message.expiry, fwd.message.onion, fwd.message.nextBlindingKey_opt, 1.0, None)
|
||||
val Right(incoming) = IncomingPaymentPacket.decrypt(dummyAdd, outgoingNodeKey, nodeParams.features)
|
||||
assert(incoming.isInstanceOf[IncomingPaymentPacket.FinalPacket])
|
||||
val finalPayload = incoming.asInstanceOf[IncomingPaymentPacket.FinalPacket].payload.asInstanceOf[FinalPayload.Blinded]
|
||||
assert(finalPayload.amount == fwd.message.amount)
|
||||
assert(finalPayload.expiry == fwd.message.expiry)
|
||||
assert(finalPayload.pathId == hex"deadbeef")
|
||||
|
||||
// Once on-the-fly funding has been proposed, the payment isn't our responsibility anymore.
|
||||
fwd.message.replyTo ! Peer.ProposeOnTheFlyFundingResponse.Proposed
|
||||
parent.expectMessageType[NodeRelayer.RelayComplete]
|
||||
}
|
||||
|
||||
test("relay to blinded path with on-the-fly funding failure", Tag(wakeUpEnabled), Tag(onTheFlyFunding)) { f =>
|
||||
import f._
|
||||
|
||||
val (peerReadyManager, switchboard) = createWakeUpActors()
|
||||
|
||||
val incomingPayments = createIncomingPaymentsToWalletBlindedPath(nodeParams)
|
||||
val (nodeRelayer, _) = f.createNodeRelay(incomingPayments.head)
|
||||
incomingPayments.foreach(incoming => nodeRelayer ! NodeRelay.Relay(incoming, randomKey().publicKey))
|
||||
|
||||
// The remote node is a wallet node: we wake them up before relaying the payment.
|
||||
peerReadyManager.expectMessageType[PeerReadyManager.Register].replyTo ! PeerReadyManager.Registered(outgoingNodeId, otherAttempts = 0)
|
||||
val peerInfo = switchboard.expectMessageType[Switchboard.GetPeerInfo]
|
||||
peerInfo.replyTo ! Peer.PeerInfo(TestProbe[Any]().ref.toClassic, outgoingNodeId, Peer.CONNECTED, Some(nodeParams.features.initFeatures()), None, Set.empty)
|
||||
cleanUpWakeUpActors(peerReadyManager, switchboard)
|
||||
|
||||
val outgoingCfg = mockPayFSM.expectMessageType[SendPaymentConfig]
|
||||
validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingPayments.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey)).toList), 5, ignoreNodeId = true)
|
||||
val outgoingPayment = mockPayFSM.expectMessageType[SendMultiPartPayment]
|
||||
|
||||
// The outgoing payment fails because we don't have enough balance: we trigger on-the-fly funding, but can't reach our peer.
|
||||
outgoingPayment.replyTo ! PaymentFailed(relayId, paymentHash, LocalFailure(outgoingAmount, Nil, BalanceTooLow) :: Nil)
|
||||
val fwd = register.expectMessageType[Register.ForwardNodeId[Peer.ProposeOnTheFlyFunding]]
|
||||
fwd.message.replyTo ! Peer.ProposeOnTheFlyFundingResponse.NotAvailable("peer disconnected")
|
||||
// We fail the payments immediately since the recipient isn't available.
|
||||
incomingPayments.foreach { p =>
|
||||
val fwd = register.expectMessageType[Register.Forward[CMD_FAIL_HTLC]]
|
||||
assert(fwd.channelId == p.add.channelId)
|
||||
assert(fwd.message == CMD_FAIL_HTLC(p.add.id, Right(UnknownNextPeer()), commit = true))
|
||||
}
|
||||
}
|
||||
|
||||
test("relay to compact blinded paths") { f =>
|
||||
import f._
|
||||
|
||||
@ -875,7 +1029,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl
|
||||
getNodeId.replyTo ! Some(outgoingNodeId)
|
||||
|
||||
val outgoingCfg = mockPayFSM.expectMessageType[SendPaymentConfig]
|
||||
validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingPayments.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey))), 5, ignoreNodeId = true)
|
||||
validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingPayments.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey)).toList), 5, ignoreNodeId = true)
|
||||
val outgoingPayment = mockPayFSM.expectMessageType[SendPaymentToNode]
|
||||
assert(outgoingPayment.amount == outgoingAmount)
|
||||
assert(outgoingPayment.recipient.expiry == outgoingExpiry)
|
||||
@ -1008,7 +1162,7 @@ object NodeRelayerSpec {
|
||||
val (expiryIn, expiryOut) = (CltvExpiry(500000), CltvExpiry(490000))
|
||||
val amountIn = incomingAmount / 2
|
||||
RelayToTrampolinePacket(
|
||||
UpdateAddHtlc(randomBytes32(), Random.nextInt(100), amountIn, paymentHash, expiryIn, TestConstants.emptyOnionPacket, None, 1.0),
|
||||
UpdateAddHtlc(randomBytes32(), Random.nextInt(100), amountIn, paymentHash, expiryIn, TestConstants.emptyOnionPacket, None, 1.0, None),
|
||||
FinalPayload.Standard.createPayload(amountIn, incomingAmount, expiryIn, paymentSecret, None),
|
||||
IntermediatePayload.NodeRelay.Standard(outgoingAmount, expiryOut, outgoingNodeId),
|
||||
createTrampolinePacket(outgoingAmount, expiryOut))
|
||||
|
@ -0,0 +1,882 @@
|
||||
/*
|
||||
* Copyright 2024 ACINQ SAS
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package fr.acinq.eclair.payment.relay
|
||||
|
||||
import akka.actor.typed.scaladsl.adapter._
|
||||
import akka.actor.{ActorContext, ActorRef}
|
||||
import akka.testkit.{TestFSMRef, TestProbe}
|
||||
import com.softwaremill.quicklens.ModifyPimp
|
||||
import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
|
||||
import fr.acinq.bitcoin.scalacompat.{Block, ByteVector32, Crypto, Satoshi, SatoshiLong, TxId}
|
||||
import fr.acinq.eclair.blockchain.{CurrentBlockHeight, DummyOnChainWallet}
|
||||
import fr.acinq.eclair.channel.Upstream.Hot
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.crypto.Sphinx
|
||||
import fr.acinq.eclair.io.Peer._
|
||||
import fr.acinq.eclair.io.PendingChannelsRateLimiter.AddOrRejectChannel
|
||||
import fr.acinq.eclair.io.{Peer, PeerConnection, PendingChannelsRateLimiter}
|
||||
import fr.acinq.eclair.wire.protocol
|
||||
import fr.acinq.eclair.wire.protocol._
|
||||
import fr.acinq.eclair.{Alias, BlockHeight, CltvExpiry, CltvExpiryDelta, FeatureSupport, Features, MilliSatoshi, MilliSatoshiLong, NodeParams, TestConstants, TestKitBaseClass, TimestampMilli, ToMilliSatoshiConversion, UInt64, randomBytes, randomBytes32, randomKey, randomLong}
|
||||
import org.scalatest.Outcome
|
||||
import org.scalatest.funsuite.FixtureAnyFunSuiteLike
|
||||
|
||||
import java.util.UUID
|
||||
import scala.concurrent.duration.DurationInt
|
||||
|
||||
class OnTheFlyFundingSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike {
|
||||
|
||||
import OnTheFlyFundingSpec._
|
||||
|
||||
val remoteFeatures = Features(
|
||||
Features.StaticRemoteKey -> FeatureSupport.Optional,
|
||||
Features.AnchorOutputsZeroFeeHtlcTx -> FeatureSupport.Optional,
|
||||
Features.DualFunding -> FeatureSupport.Optional,
|
||||
Features.SplicePrototype -> FeatureSupport.Optional,
|
||||
Features.OnTheFlyFunding -> FeatureSupport.Optional,
|
||||
)
|
||||
|
||||
case class FixtureParam(nodeParams: NodeParams, remoteNodeId: PublicKey, peer: TestFSMRef[Peer.State, Peer.Data, Peer], peerConnection: TestProbe, channel: TestProbe, register: TestProbe, rateLimiter: TestProbe, probe: TestProbe) {
|
||||
def connect(peer: TestFSMRef[Peer.State, Peer.Data, Peer], remoteInit: protocol.Init = protocol.Init(remoteFeatures.initFeatures()), channelCount: Int = 0): Unit = {
|
||||
val localInit = protocol.Init(nodeParams.features.initFeatures())
|
||||
val address = NodeAddress.fromParts("0.0.0.0", 9735).get
|
||||
peer ! PeerConnection.ConnectionReady(peerConnection.ref, remoteNodeId, address, outgoing = true, localInit, remoteInit)
|
||||
peerConnection.expectMsgType[RecommendedFeerates]
|
||||
(0 until channelCount).foreach(_ => channel.expectMsgType[INPUT_RECONNECTED])
|
||||
probe.send(peer, Peer.GetPeerInfo(Some(probe.ref.toTyped)))
|
||||
val peerInfo = probe.expectMsgType[Peer.PeerInfo]
|
||||
assert(peerInfo.nodeId == remoteNodeId)
|
||||
assert(peerInfo.state == Peer.CONNECTED)
|
||||
}
|
||||
|
||||
def openChannel(fundingAmount: Satoshi): ByteVector32 = {
|
||||
peer ! Peer.OpenChannel(remoteNodeId, fundingAmount, None, None, None, None, None, None, None)
|
||||
val temporaryChannelId = channel.expectMsgType[INPUT_INIT_CHANNEL_INITIATOR].temporaryChannelId
|
||||
val channelId = randomBytes32()
|
||||
peer ! ChannelIdAssigned(channel.ref, remoteNodeId, temporaryChannelId, channelId)
|
||||
peerConnection.expectMsgType[PeerConnection.DoSync]
|
||||
channelId
|
||||
}
|
||||
|
||||
def disconnect(channelCount: Int = 0): Unit = {
|
||||
peer ! Peer.ConnectionDown(peerConnection.ref)
|
||||
(0 until channelCount).foreach(_ => channel.expectMsg(INPUT_DISCONNECTED))
|
||||
}
|
||||
|
||||
def createProposal(amount: MilliSatoshi, expiry: CltvExpiry, paymentHash: ByteVector32 = randomBytes32(), upstream: Upstream.Hot = Upstream.Local(UUID.randomUUID())): ProposeOnTheFlyFunding = {
|
||||
val blindingKey = upstream match {
|
||||
case u: Upstream.Hot.Channel if u.add.blinding_opt.nonEmpty => Some(randomKey().publicKey)
|
||||
case u: Upstream.Hot.Trampoline if u.received.exists(_.add.blinding_opt.nonEmpty) => Some(randomKey().publicKey)
|
||||
case _ => None
|
||||
}
|
||||
ProposeOnTheFlyFunding(probe.ref, amount, paymentHash, expiry, TestConstants.emptyOnionPacket, blindingKey, upstream)
|
||||
}
|
||||
|
||||
def proposeFunding(amount: MilliSatoshi, expiry: CltvExpiry, paymentHash: ByteVector32 = randomBytes32(), upstream: Upstream.Hot = Upstream.Local(UUID.randomUUID())): WillAddHtlc = {
|
||||
val proposal = createProposal(amount, expiry, paymentHash, upstream)
|
||||
peer ! proposal
|
||||
val willAdd = peerConnection.expectMsgType[WillAddHtlc]
|
||||
assert(willAdd.amount == amount)
|
||||
assert(willAdd.expiry == expiry)
|
||||
assert(willAdd.paymentHash == paymentHash)
|
||||
probe.expectMsg(ProposeOnTheFlyFundingResponse.Proposed)
|
||||
willAdd
|
||||
}
|
||||
|
||||
/** This should be used when the sender is buggy and keeps adding HTLCs after the funding proposal has been accepted. */
|
||||
def proposeExtraFunding(amount: MilliSatoshi, expiry: CltvExpiry, paymentHash: ByteVector32 = randomBytes32(), upstream: Upstream.Hot = Upstream.Local(UUID.randomUUID())): Unit = {
|
||||
val proposal = createProposal(amount, expiry, paymentHash, upstream)
|
||||
peer ! proposal
|
||||
probe.expectMsg(ProposeOnTheFlyFundingResponse.Proposed)
|
||||
peerConnection.expectNoMessage(100 millis)
|
||||
}
|
||||
|
||||
def signLiquidityPurchase(amount: Satoshi,
|
||||
paymentDetails: LiquidityAds.PaymentDetails,
|
||||
channelId: ByteVector32 = randomBytes32(),
|
||||
fees: LiquidityAds.Fees = LiquidityAds.Fees(0 sat, 0 sat),
|
||||
fundingTxIndex: Long = 0,
|
||||
htlcMinimum: MilliSatoshi = 1 msat): LiquidityPurchaseSigned = {
|
||||
val purchase = LiquidityAds.Purchase.Standard(amount, fees, paymentDetails)
|
||||
val event = LiquidityPurchaseSigned(channelId, TxId(randomBytes32()), fundingTxIndex, htlcMinimum, purchase)
|
||||
peer ! event
|
||||
event
|
||||
}
|
||||
|
||||
def makeChannelData(htlcMinimum: MilliSatoshi = 1 msat, localChanges: LocalChanges = LocalChanges(Nil, Nil, Nil)): DATA_NORMAL = {
|
||||
val commitments = CommitmentsSpec.makeCommitments(500_000_000 msat, 500_000_000 msat, nodeParams.nodeId, remoteNodeId, announceChannel = false)
|
||||
.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)
|
||||
}
|
||||
}
|
||||
|
||||
case class FakeChannelFactory(remoteNodeId: PublicKey, channel: TestProbe) extends ChannelFactory {
|
||||
override def spawn(context: ActorContext, remoteNodeId: PublicKey): ActorRef = {
|
||||
assert(remoteNodeId == remoteNodeId)
|
||||
channel.ref
|
||||
}
|
||||
}
|
||||
|
||||
override protected def withFixture(test: OneArgTest): Outcome = {
|
||||
val nodeParams = TestConstants.Alice.nodeParams
|
||||
.modify(_.features.activated).using(_ + (Features.AnchorOutputsZeroFeeHtlcTx -> FeatureSupport.Optional))
|
||||
.modify(_.features.activated).using(_ + (Features.DualFunding -> FeatureSupport.Optional))
|
||||
.modify(_.features.activated).using(_ + (Features.SplicePrototype -> FeatureSupport.Optional))
|
||||
.modify(_.features.activated).using(_ + (Features.OnTheFlyFunding -> FeatureSupport.Optional))
|
||||
val remoteNodeId = randomKey().publicKey
|
||||
val register = TestProbe()
|
||||
val channel = TestProbe()
|
||||
val peerConnection = TestProbe()
|
||||
val rateLimiter = TestProbe()
|
||||
val probe = TestProbe()
|
||||
val peer = TestFSMRef(new Peer(nodeParams, remoteNodeId, new DummyOnChainWallet(), FakeChannelFactory(remoteNodeId, channel), TestProbe().ref, register.ref, TestProbe().ref, rateLimiter.ref))
|
||||
peer ! Peer.Init(Set.empty, Map.empty)
|
||||
withFixture(test.toNoArgTest(FixtureParam(nodeParams, remoteNodeId, peer, peerConnection, channel, register, rateLimiter, probe)))
|
||||
}
|
||||
|
||||
test("ignore requests when peer doesn't support on-the-fly funding") { f =>
|
||||
import f._
|
||||
|
||||
connect(peer, remoteInit = protocol.Init(Features.empty))
|
||||
peer ! createProposal(100_000_000 msat, CltvExpiry(561))
|
||||
probe.expectMsgType[ProposeOnTheFlyFundingResponse.NotAvailable]
|
||||
}
|
||||
|
||||
test("ignore requests when disconnected") { f =>
|
||||
import f._
|
||||
|
||||
peer ! createProposal(100_000_000 msat, CltvExpiry(561))
|
||||
probe.expectMsgType[ProposeOnTheFlyFundingResponse.NotAvailable]
|
||||
}
|
||||
|
||||
test("receive remote failure") { f =>
|
||||
import f._
|
||||
|
||||
connect(peer)
|
||||
|
||||
val paymentHash = randomBytes32()
|
||||
val upstream1 = upstreamChannel(75_000_000 msat, CltvExpiry(561), paymentHash)
|
||||
val willAdd1 = proposeFunding(70_000_000 msat, CltvExpiry(550), paymentHash, upstream1)
|
||||
val upstream2 = upstreamChannel(80_000_000 msat, CltvExpiry(561), paymentHash, blinded = true)
|
||||
val willAdd2 = proposeFunding(75_000_000 msat, CltvExpiry(550), paymentHash, upstream2)
|
||||
val upstream3 = upstreamChannel(50_000_000 msat, CltvExpiry(561), paymentHash)
|
||||
val willAdd3 = proposeFunding(50_000_000 msat, CltvExpiry(550), paymentHash, upstream3)
|
||||
val upstream4 = Upstream.Hot.Trampoline(List(
|
||||
upstreamChannel(50_000_000 msat, CltvExpiry(561), paymentHash),
|
||||
upstreamChannel(50_000_000 msat, CltvExpiry(561), paymentHash),
|
||||
))
|
||||
val willAdd4 = proposeFunding(100_000_000 msat, CltvExpiry(550), paymentHash, upstream4)
|
||||
val upstream5 = Upstream.Hot.Trampoline(List(
|
||||
upstreamChannel(50_000_000 msat, CltvExpiry(561), paymentHash, blinded = true),
|
||||
upstreamChannel(50_000_000 msat, CltvExpiry(561), paymentHash, blinded = true),
|
||||
))
|
||||
val willAdd5 = proposeFunding(100_000_000 msat, CltvExpiry(550), paymentHash, upstream5)
|
||||
|
||||
val fail1 = WillFailHtlc(willAdd1.id, paymentHash, randomBytes(42))
|
||||
peerConnection.send(peer, fail1)
|
||||
val fwd1 = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]]
|
||||
assert(fwd1.channelId == upstream1.add.channelId)
|
||||
assert(fwd1.message.id == upstream1.add.id)
|
||||
assert(fwd1.message.reason == Left(fail1.reason))
|
||||
register.expectNoMessage(100 millis)
|
||||
|
||||
val fail2 = WillFailHtlc(willAdd2.id, paymentHash, randomBytes(50))
|
||||
peerConnection.send(peer, fail2)
|
||||
val fwd2 = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]]
|
||||
assert(fwd2.channelId == upstream2.add.channelId)
|
||||
assert(fwd2.message.id == upstream2.add.id)
|
||||
assert(fwd2.message.reason == Right(InvalidOnionBlinding(Sphinx.hash(upstream2.add.onionRoutingPacket))))
|
||||
|
||||
val fail3 = WillFailMalformedHtlc(willAdd3.id, paymentHash, randomBytes32(), InvalidOnionHmac(randomBytes32()).code)
|
||||
peerConnection.send(peer, fail3)
|
||||
val fwd3 = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]]
|
||||
assert(fwd3.channelId == upstream3.add.channelId)
|
||||
assert(fwd3.message.id == upstream3.add.id)
|
||||
assert(fwd3.message.reason == Right(InvalidOnionHmac(fail3.onionHash)))
|
||||
|
||||
val fail4 = WillFailHtlc(willAdd4.id, paymentHash, randomBytes(75))
|
||||
peerConnection.send(peer, fail4)
|
||||
upstream4.received.map(_.add).foreach(add => {
|
||||
val fwd = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]]
|
||||
assert(fwd.channelId == add.channelId)
|
||||
assert(fwd.message.id == add.id)
|
||||
assert(fwd.message.reason == Right(TemporaryNodeFailure()))
|
||||
})
|
||||
|
||||
val fail5 = WillFailHtlc(willAdd5.id, paymentHash, randomBytes(75))
|
||||
peerConnection.send(peer, fail5)
|
||||
upstream5.received.map(_.add).foreach(add => {
|
||||
val fwd = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]]
|
||||
assert(fwd.channelId == add.channelId)
|
||||
assert(fwd.message.id == add.id)
|
||||
assert(fwd.message.reason == Right(TemporaryNodeFailure()))
|
||||
})
|
||||
}
|
||||
|
||||
test("proposed on-the-fly funding timeout") { f =>
|
||||
import f._
|
||||
|
||||
connect(peer)
|
||||
|
||||
// A first funding is proposed coming from two upstream channels.
|
||||
val paymentHash1 = randomBytes32()
|
||||
val upstream1 = Seq(
|
||||
upstreamChannel(60_000_000 msat, CltvExpiry(561), paymentHash1, blinded = true),
|
||||
upstreamChannel(45_000_000 msat, CltvExpiry(561), paymentHash1, blinded = true),
|
||||
)
|
||||
proposeFunding(50_000_000 msat, CltvExpiry(550), paymentHash1, upstream1.head)
|
||||
proposeFunding(40_000_000 msat, CltvExpiry(550), paymentHash1, upstream1.last)
|
||||
|
||||
// A second funding is proposed coming from a trampoline payment.
|
||||
val paymentHash2 = randomBytes32()
|
||||
val upstream2 = Upstream.Hot.Trampoline(List(
|
||||
upstreamChannel(60_000_000 msat, CltvExpiry(561), paymentHash2),
|
||||
upstreamChannel(45_000_000 msat, CltvExpiry(561), paymentHash2),
|
||||
))
|
||||
proposeFunding(100_000_000 msat, CltvExpiry(550), paymentHash2, upstream2)
|
||||
|
||||
// A third funding is signed coming from a trampoline payment.
|
||||
val paymentHash3 = randomBytes32()
|
||||
val upstream3 = Upstream.Hot.Trampoline(List(
|
||||
upstreamChannel(60_000_000 msat, CltvExpiry(561), paymentHash3),
|
||||
upstreamChannel(45_000_000 msat, CltvExpiry(561), paymentHash3),
|
||||
))
|
||||
proposeFunding(100_000_000 msat, CltvExpiry(550), paymentHash3, upstream3)
|
||||
signLiquidityPurchase(100_000 sat, LiquidityAds.PaymentDetails.FromChannelBalanceForFutureHtlc(paymentHash3 :: Nil))
|
||||
|
||||
// The funding timeout is reached, unsigned proposals are failed upstream.
|
||||
peer ! OnTheFlyFundingTimeout(paymentHash1)
|
||||
upstream1.foreach(u => {
|
||||
val fwd = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]]
|
||||
assert(fwd.channelId == u.add.channelId)
|
||||
assert(fwd.message.id == u.add.id)
|
||||
assert(fwd.message.reason == Right(InvalidOnionBlinding(Sphinx.hash(u.add.onionRoutingPacket))))
|
||||
assert(fwd.message.commit)
|
||||
})
|
||||
peerConnection.expectMsgType[Warning]
|
||||
|
||||
peer ! OnTheFlyFundingTimeout(paymentHash2)
|
||||
upstream2.received.foreach(u => {
|
||||
val fwd = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]]
|
||||
assert(fwd.channelId == u.add.channelId)
|
||||
assert(fwd.message.id == u.add.id)
|
||||
assert(fwd.message.reason == Right(UnknownNextPeer()))
|
||||
assert(fwd.message.commit)
|
||||
})
|
||||
peerConnection.expectMsgType[Warning]
|
||||
|
||||
peer ! OnTheFlyFundingTimeout(paymentHash3)
|
||||
register.expectNoMessage(100 millis)
|
||||
peerConnection.expectNoMessage(100 millis)
|
||||
}
|
||||
|
||||
test("proposed on-the-fly funding HTLC timeout") { f =>
|
||||
import f._
|
||||
|
||||
connect(peer)
|
||||
|
||||
// A first funding is proposed coming from two upstream channels.
|
||||
val paymentHash1 = randomBytes32()
|
||||
val upstream1 = Seq(
|
||||
upstreamChannel(60_000_000 msat, CltvExpiry(560), paymentHash1),
|
||||
upstreamChannel(45_000_000 msat, CltvExpiry(550), paymentHash1),
|
||||
)
|
||||
proposeFunding(50_000_000 msat, CltvExpiry(520), paymentHash1, upstream1.head)
|
||||
proposeFunding(40_000_000 msat, CltvExpiry(510), paymentHash1, upstream1.last)
|
||||
|
||||
// A second funding is signed coming from two upstream channels, one of them received after signing.
|
||||
val paymentHash2 = randomBytes32()
|
||||
val upstream2 = Seq(
|
||||
upstreamChannel(45_000_000 msat, CltvExpiry(550), paymentHash2),
|
||||
upstreamChannel(60_000_000 msat, CltvExpiry(560), paymentHash2),
|
||||
)
|
||||
proposeFunding(40_000_000 msat, CltvExpiry(515), paymentHash2, upstream2.head)
|
||||
signLiquidityPurchase(100_000 sat, LiquidityAds.PaymentDetails.FromFutureHtlc(paymentHash2 :: Nil))
|
||||
proposeExtraFunding(50_000_000 msat, CltvExpiry(525), paymentHash2, upstream2.last)
|
||||
|
||||
// A third funding is signed coming from a trampoline payment.
|
||||
val paymentHash3 = randomBytes32()
|
||||
val upstream3 = Upstream.Hot.Trampoline(List(
|
||||
upstreamChannel(60_000_000 msat, CltvExpiry(560), paymentHash3),
|
||||
upstreamChannel(45_000_000 msat, CltvExpiry(560), paymentHash3),
|
||||
))
|
||||
proposeFunding(100_000_000 msat, CltvExpiry(512), paymentHash3, upstream3)
|
||||
signLiquidityPurchase(100_000 sat, LiquidityAds.PaymentDetails.FromChannelBalanceForFutureHtlc(paymentHash3 :: Nil))
|
||||
|
||||
// A fourth funding is proposed coming from a trampoline payment.
|
||||
val paymentHash4 = randomBytes32()
|
||||
val upstream4 = Upstream.Hot.Trampoline(List(upstreamChannel(60_000_000 msat, CltvExpiry(560), paymentHash4)))
|
||||
proposeFunding(50_000_000 msat, CltvExpiry(516), paymentHash4, upstream4)
|
||||
|
||||
// The first three proposals reach their CLTV expiry.
|
||||
peer ! CurrentBlockHeight(BlockHeight(515))
|
||||
val fwds = (0 until 6).map(_ => register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]])
|
||||
register.expectNoMessage(100 millis)
|
||||
fwds.foreach(fwd => {
|
||||
assert(fwd.message.reason == Right(UnknownNextPeer()))
|
||||
assert(fwd.message.commit)
|
||||
})
|
||||
assert(fwds.map(_.channelId).toSet == (upstream1 ++ upstream2 ++ upstream3.received).map(_.add.channelId).toSet)
|
||||
assert(fwds.map(_.message.id).toSet == (upstream1 ++ upstream2 ++ upstream3.received).map(_.add.id).toSet)
|
||||
awaitCond(nodeParams.db.liquidity.listPendingOnTheFlyFunding(remoteNodeId).isEmpty, interval = 100 millis)
|
||||
}
|
||||
|
||||
test("signed on-the-fly funding HTLC timeout after disconnection") { f =>
|
||||
import f._
|
||||
|
||||
connect(peer)
|
||||
probe.watch(peer.ref)
|
||||
|
||||
// A first funding proposal is signed.
|
||||
val upstream1 = upstreamChannel(60_000_000 msat, CltvExpiry(560))
|
||||
proposeFunding(50_000_000 msat, CltvExpiry(520), upstream1.add.paymentHash, upstream1)
|
||||
signLiquidityPurchase(75_000 sat, LiquidityAds.PaymentDetails.FromFutureHtlc(upstream1.add.paymentHash :: Nil))
|
||||
|
||||
// A second funding proposal is signed.
|
||||
val upstream2 = upstreamChannel(60_000_000 msat, CltvExpiry(560))
|
||||
proposeFunding(50_000_000 msat, CltvExpiry(525), upstream2.add.paymentHash, upstream2)
|
||||
signLiquidityPurchase(80_000 sat, LiquidityAds.PaymentDetails.FromFutureHtlc(upstream2.add.paymentHash :: Nil))
|
||||
|
||||
// We don't fail signed proposals on disconnection.
|
||||
disconnect()
|
||||
register.expectNoMessage(100 millis)
|
||||
|
||||
// But if a funding proposal reaches its CLTV expiry, we fail it.
|
||||
peer ! CurrentBlockHeight(BlockHeight(522))
|
||||
val fwd1 = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]]
|
||||
assert(fwd1.channelId == upstream1.add.channelId)
|
||||
assert(fwd1.message.id == upstream1.add.id)
|
||||
register.expectNoMessage(100 millis)
|
||||
// We still have one pending proposal, so we don't stop.
|
||||
probe.expectNoMessage(100 millis)
|
||||
|
||||
// When restarting, we watch for pending proposals.
|
||||
val peerAfterRestart = TestFSMRef(new Peer(nodeParams, remoteNodeId, new DummyOnChainWallet(), FakeChannelFactory(remoteNodeId, channel), TestProbe().ref, register.ref, TestProbe().ref, TestProbe().ref))
|
||||
peerAfterRestart ! Peer.Init(Set.empty, nodeParams.db.liquidity.listPendingOnTheFlyFunding(remoteNodeId))
|
||||
probe.watch(peerAfterRestart.ref)
|
||||
|
||||
// The last funding proposal reaches its CLTV expiry.
|
||||
peerAfterRestart ! CurrentBlockHeight(BlockHeight(525))
|
||||
val fwd2 = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]]
|
||||
assert(fwd2.channelId == upstream2.add.channelId)
|
||||
assert(fwd2.message.id == upstream2.add.id)
|
||||
register.expectNoMessage(100 millis)
|
||||
probe.expectTerminated(peerAfterRestart.ref)
|
||||
}
|
||||
|
||||
test("receive open_channel2") { f =>
|
||||
import f._
|
||||
|
||||
connect(peer)
|
||||
|
||||
val upstream = upstreamChannel(60_000_000 msat, CltvExpiry(560), paymentHash)
|
||||
proposeFunding(50_000_000 msat, CltvExpiry(520), paymentHash, upstream)
|
||||
|
||||
val requestFunding = LiquidityAds.RequestFunding(
|
||||
100_000 sat,
|
||||
LiquidityAds.FundingRate(10_000 sat, 500_000 sat, 0, 100, 1000 sat, 1000 sat),
|
||||
LiquidityAds.PaymentDetails.FromFutureHtlcWithPreimage(preimage :: Nil)
|
||||
)
|
||||
val open = createOpenChannelMessage(requestFunding)
|
||||
peerConnection.send(peer, open)
|
||||
rateLimiter.expectMsgType[AddOrRejectChannel].replyTo ! PendingChannelsRateLimiter.AcceptOpenChannel
|
||||
val init = channel.expectMsgType[INPUT_INIT_CHANNEL_NON_INITIATOR]
|
||||
assert(!init.localParams.isChannelOpener)
|
||||
assert(init.localParams.paysCommitTxFees)
|
||||
assert(init.fundingContribution_opt.contains(LiquidityAds.AddFunding(requestFunding.requestedAmount, nodeParams.willFundRates_opt)))
|
||||
|
||||
// The preimage was provided, so we fulfill upstream HTLCs.
|
||||
val fwd = register.expectMsgType[Register.Forward[CMD_FULFILL_HTLC]]
|
||||
assert(fwd.channelId == upstream.add.channelId)
|
||||
assert(fwd.message.id == upstream.add.id)
|
||||
assert(fwd.message.r == preimage)
|
||||
}
|
||||
|
||||
test("receive splice_init") { f =>
|
||||
import f._
|
||||
|
||||
connect(peer)
|
||||
val channelId = openChannel(200_000 sat)
|
||||
|
||||
val upstream = upstreamChannel(60_000_000 msat, CltvExpiry(560), paymentHash)
|
||||
proposeFunding(50_000_000 msat, CltvExpiry(520), paymentHash, upstream)
|
||||
|
||||
val requestFunding = LiquidityAds.RequestFunding(
|
||||
100_000 sat,
|
||||
LiquidityAds.FundingRate(10_000 sat, 500_000 sat, 0, 100, 1000 sat, 1000 sat),
|
||||
LiquidityAds.PaymentDetails.FromFutureHtlcWithPreimage(preimage :: Nil)
|
||||
)
|
||||
val splice = createSpliceMessage(channelId, requestFunding)
|
||||
peerConnection.send(peer, splice)
|
||||
channel.expectMsg(splice)
|
||||
channel.expectNoMessage(100 millis)
|
||||
|
||||
// The preimage was provided, so we fulfill upstream HTLCs.
|
||||
val fwd = register.expectMsgType[Register.Forward[CMD_FULFILL_HTLC]]
|
||||
assert(fwd.channelId == upstream.add.channelId)
|
||||
assert(fwd.message.id == upstream.add.id)
|
||||
assert(fwd.message.r == preimage)
|
||||
}
|
||||
|
||||
test("reject invalid open_channel2") { f =>
|
||||
import f._
|
||||
|
||||
connect(peer)
|
||||
|
||||
val requestFunding = LiquidityAds.RequestFunding(
|
||||
100_000 sat,
|
||||
LiquidityAds.FundingRate(10_000 sat, 500_000 sat, 0, 0, 5_000 sat, 5_000 sat),
|
||||
LiquidityAds.PaymentDetails.FromFutureHtlc(paymentHash :: Nil)
|
||||
)
|
||||
val open = createOpenChannelMessage(requestFunding, htlcMinimum = 1_000_000 msat)
|
||||
|
||||
// No matching will_add_htlc to pay fees.
|
||||
peerConnection.send(peer, open)
|
||||
rateLimiter.expectMsgType[AddOrRejectChannel].replyTo ! PendingChannelsRateLimiter.AcceptOpenChannel
|
||||
assert(peerConnection.expectMsgType[CancelOnTheFlyFunding].channelId == open.temporaryChannelId)
|
||||
channel.expectNoMessage(100 millis)
|
||||
|
||||
// Requested amount is too low.
|
||||
val bigUpstream = upstreamChannel(200_000_000 msat, expiryIn, paymentHash)
|
||||
proposeFunding(150_000_000 msat, expiryOut, paymentHash, bigUpstream)
|
||||
peerConnection.send(peer, open)
|
||||
rateLimiter.expectMsgType[AddOrRejectChannel].replyTo ! PendingChannelsRateLimiter.AcceptOpenChannel
|
||||
assert(peerConnection.expectMsgType[CancelOnTheFlyFunding].channelId == open.temporaryChannelId)
|
||||
assert(register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]].channelId == bigUpstream.add.channelId)
|
||||
channel.expectNoMessage(100 millis)
|
||||
|
||||
// Not enough funds to pay fees.
|
||||
val upstream = upstreamChannel(11_000_000 msat, expiryIn, paymentHash)
|
||||
proposeFunding(10_999_999 msat, expiryOut, paymentHash, upstream)
|
||||
peerConnection.send(peer, open)
|
||||
rateLimiter.expectMsgType[AddOrRejectChannel].replyTo ! PendingChannelsRateLimiter.AcceptOpenChannel
|
||||
assert(peerConnection.expectMsgType[CancelOnTheFlyFunding].channelId == open.temporaryChannelId)
|
||||
assert(register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]].channelId == upstream.add.channelId)
|
||||
channel.expectNoMessage(100 millis)
|
||||
|
||||
// Proposal already funded.
|
||||
proposeFunding(11_000_000 msat, expiryOut, paymentHash, upstream)
|
||||
signLiquidityPurchase(requestFunding.requestedAmount, requestFunding.paymentDetails, fees = requestFunding.fees(TestConstants.feeratePerKw, isChannelCreation = true))
|
||||
peerConnection.send(peer, open)
|
||||
rateLimiter.expectMsgType[AddOrRejectChannel].replyTo ! PendingChannelsRateLimiter.AcceptOpenChannel
|
||||
assert(peerConnection.expectMsgType[CancelOnTheFlyFunding].channelId == open.temporaryChannelId)
|
||||
register.expectNoMessage(100 millis)
|
||||
channel.expectNoMessage(100 millis)
|
||||
}
|
||||
|
||||
test("reject invalid splice_init") { f =>
|
||||
import f._
|
||||
|
||||
connect(peer)
|
||||
val channelId = openChannel(500_000 sat)
|
||||
|
||||
val requestFunding = LiquidityAds.RequestFunding(
|
||||
100_000 sat,
|
||||
LiquidityAds.FundingRate(10_000 sat, 500_000 sat, 0, 0, 10_000 sat, 5_000 sat),
|
||||
LiquidityAds.PaymentDetails.FromFutureHtlcWithPreimage(preimage :: Nil)
|
||||
)
|
||||
val splice = createSpliceMessage(channelId, requestFunding)
|
||||
|
||||
// No matching will_add_htlc to pay fees.
|
||||
peerConnection.send(peer, splice)
|
||||
assert(peerConnection.expectMsgType[CancelOnTheFlyFunding].channelId == channelId)
|
||||
channel.expectNoMessage(100 millis)
|
||||
|
||||
// Requested amount is too low.
|
||||
val bigUpstream = upstreamChannel(200_000_000 msat, expiryIn, paymentHash)
|
||||
proposeFunding(150_000_000 msat, expiryOut, paymentHash, bigUpstream)
|
||||
peerConnection.send(peer, splice)
|
||||
assert(peerConnection.expectMsgType[CancelOnTheFlyFunding].channelId == channelId)
|
||||
assert(register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]].channelId == bigUpstream.add.channelId)
|
||||
channel.expectNoMessage(100 millis)
|
||||
|
||||
// Not enough funds to pay fees.
|
||||
val upstream = upstreamChannel(11_000_000 msat, expiryIn, paymentHash)
|
||||
proposeFunding(9_000_000 msat, expiryOut, paymentHash, upstream)
|
||||
peerConnection.send(peer, splice)
|
||||
assert(peerConnection.expectMsgType[CancelOnTheFlyFunding].channelId == channelId)
|
||||
assert(register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]].channelId == upstream.add.channelId)
|
||||
channel.expectNoMessage(100 millis)
|
||||
|
||||
// Proposal already funded.
|
||||
proposeFunding(11_000_000 msat, expiryOut, paymentHash, upstream)
|
||||
signLiquidityPurchase(requestFunding.requestedAmount, requestFunding.paymentDetails, fees = requestFunding.fees(TestConstants.feeratePerKw, isChannelCreation = false), fundingTxIndex = 1)
|
||||
peerConnection.send(peer, splice)
|
||||
assert(peerConnection.expectMsgType[CancelOnTheFlyFunding].channelId == channelId)
|
||||
register.expectNoMessage(100 millis)
|
||||
channel.expectNoMessage(100 millis)
|
||||
}
|
||||
|
||||
test("successfully relay HTLCs to on-the-fly funded channel") { f =>
|
||||
import f._
|
||||
|
||||
connect(peer)
|
||||
|
||||
val preimage1 = randomBytes32()
|
||||
val paymentHash1 = Crypto.sha256(preimage1)
|
||||
val preimage2 = randomBytes32()
|
||||
val paymentHash2 = Crypto.sha256(preimage2)
|
||||
|
||||
val upstream1 = upstreamChannel(11_000_000 msat, expiryIn, paymentHash1)
|
||||
proposeFunding(10_000_000 msat, expiryOut, paymentHash1, upstream1)
|
||||
val upstream2 = upstreamChannel(16_000_000 msat, expiryIn, paymentHash2)
|
||||
proposeFunding(15_000_000 msat, expiryOut, paymentHash2, upstream2)
|
||||
|
||||
val htlcMinimum = 1_500_000 msat
|
||||
val fees = LiquidityAds.Fees(10_000 sat, 5_000 sat)
|
||||
val purchase = signLiquidityPurchase(200_000 sat, LiquidityAds.PaymentDetails.FromFutureHtlc(List(paymentHash1, paymentHash2)), fees = fees, htlcMinimum = htlcMinimum)
|
||||
|
||||
// Once the channel is ready to relay payments, we forward HTLCs matching the proposed will_add_htlc.
|
||||
// We have two distinct payment hashes that are relayed independently.
|
||||
val channelData = makeChannelData(htlcMinimum)
|
||||
peer ! ChannelReadyForPayments(channel.ref, remoteNodeId, purchase.channelId, fundingTxIndex = 0)
|
||||
val channelInfo = Seq(
|
||||
channel.expectMsgType[CMD_GET_CHANNEL_INFO],
|
||||
channel.expectMsgType[CMD_GET_CHANNEL_INFO],
|
||||
)
|
||||
|
||||
// We relay the first payment.
|
||||
channelInfo.head.replyTo ! RES_GET_CHANNEL_INFO(remoteNodeId, purchase.channelId, channel.ref, NORMAL, channelData)
|
||||
val cmd1 = channel.expectMsgType[CMD_ADD_HTLC]
|
||||
cmd1.replyTo ! RES_SUCCESS(cmd1, purchase.channelId)
|
||||
channel.expectNoMessage(100 millis)
|
||||
|
||||
// We relay the second payment.
|
||||
channelInfo.last.replyTo ! RES_GET_CHANNEL_INFO(remoteNodeId, purchase.channelId, channel.ref, NORMAL, channelData)
|
||||
val cmd2 = channel.expectMsgType[CMD_ADD_HTLC]
|
||||
cmd2.replyTo ! RES_SUCCESS(cmd1, purchase.channelId)
|
||||
channel.expectNoMessage(100 millis)
|
||||
|
||||
// The fee is split across outgoing payments.
|
||||
assert(Set(cmd1.paymentHash, cmd2.paymentHash) == Set(paymentHash1, paymentHash2))
|
||||
val fundingFees = Seq(cmd1, cmd2).map(cmd => {
|
||||
assert(cmd.amount >= htlcMinimum)
|
||||
assert(cmd.cltvExpiry == expiryOut)
|
||||
assert(cmd.commit)
|
||||
assert(cmd.fundingFee_opt.nonEmpty)
|
||||
assert(cmd.fundingFee_opt.get.fundingTxId == purchase.txId)
|
||||
assert(cmd.fundingFee_opt.get.amount > 0.msat)
|
||||
cmd.fundingFee_opt.get
|
||||
})
|
||||
val feesPaid = fundingFees.map(_.amount).sum
|
||||
assert(feesPaid == fees.total.toMilliSatoshi)
|
||||
assert(cmd1.amount + cmd2.amount + feesPaid == 25_000_000.msat)
|
||||
|
||||
// The payments are fulfilled.
|
||||
val (add1, add2) = if (cmd1.paymentHash == paymentHash1) (cmd1, cmd2) else (cmd2, cmd1)
|
||||
val outgoing = Seq(add1, add2).map(add => UpdateAddHtlc(purchase.channelId, randomHtlcId(), add.amount, add.paymentHash, add.cltvExpiry, add.onion, add.nextBlindingKey_opt, add.confidence, add.fundingFee_opt))
|
||||
add1.replyTo ! RES_ADD_SETTLED(add1.origin, outgoing.head, HtlcResult.RemoteFulfill(UpdateFulfillHtlc(purchase.channelId, outgoing.head.id, preimage1)))
|
||||
val fwd1 = register.expectMsgType[Register.Forward[CMD_FULFILL_HTLC]]
|
||||
assert(fwd1.channelId == upstream1.add.channelId)
|
||||
assert(fwd1.message.id == upstream1.add.id)
|
||||
assert(fwd1.message.r == preimage1)
|
||||
add2.replyTo ! RES_ADD_SETTLED(add2.origin, outgoing.last, HtlcResult.OnChainFulfill(preimage2))
|
||||
val fwd2 = register.expectMsgType[Register.Forward[CMD_FULFILL_HTLC]]
|
||||
assert(fwd2.channelId == upstream2.add.channelId)
|
||||
assert(fwd2.message.id == upstream2.add.id)
|
||||
assert(fwd2.message.r == preimage2)
|
||||
awaitCond(nodeParams.db.liquidity.listPendingOnTheFlyFunding(remoteNodeId).isEmpty, interval = 100 millis)
|
||||
register.expectNoMessage(100 millis)
|
||||
}
|
||||
|
||||
test("successfully relay HTLCs to on-the-fly spliced channel") { f =>
|
||||
import f._
|
||||
|
||||
// We create a channel, that can later be spliced.
|
||||
connect(peer)
|
||||
val channelId = openChannel(250_000 sat)
|
||||
|
||||
val htlcMinimum = 1_000_000 msat
|
||||
val fees = LiquidityAds.Fees(1000 sat, 4000 sat)
|
||||
val upstream = Seq(
|
||||
upstreamChannel(50_000_000 msat, expiryIn, paymentHash),
|
||||
upstreamChannel(60_000_000 msat, expiryIn, paymentHash),
|
||||
Upstream.Hot.Trampoline(upstreamChannel(50_000_000 msat, expiryIn, paymentHash) :: Nil)
|
||||
)
|
||||
proposeFunding(50_000_000 msat, expiryOut, paymentHash, upstream(0))
|
||||
proposeFunding(60_000_000 msat, expiryOut, paymentHash, upstream(1))
|
||||
val purchase = signLiquidityPurchase(200_000 sat, LiquidityAds.PaymentDetails.FromFutureHtlcWithPreimage(preimage :: Nil), channelId, fees, fundingTxIndex = 5, htlcMinimum)
|
||||
// We receive the last payment *after* signing the funding transaction.
|
||||
proposeExtraFunding(50_000_000 msat, expiryOut, paymentHash, upstream(2))
|
||||
|
||||
// Once the splice with the right funding index is locked, we forward HTLCs matching the proposed will_add_htlc.
|
||||
val channelData = makeChannelData(htlcMinimum)
|
||||
peer ! ChannelReadyForPayments(channel.ref, remoteNodeId, channelId, fundingTxIndex = 4)
|
||||
channel.expectNoMessage(100 millis)
|
||||
peer ! ChannelReadyForPayments(channel.ref, remoteNodeId, channelId, fundingTxIndex = 5)
|
||||
channel.expectMsgType[CMD_GET_CHANNEL_INFO].replyTo ! RES_GET_CHANNEL_INFO(remoteNodeId, channelId, channel.ref, NORMAL, channelData)
|
||||
val adds1 = Seq(
|
||||
channel.expectMsgType[CMD_ADD_HTLC],
|
||||
channel.expectMsgType[CMD_ADD_HTLC],
|
||||
channel.expectMsgType[CMD_ADD_HTLC],
|
||||
)
|
||||
adds1.foreach(add => {
|
||||
assert(add.paymentHash == paymentHash)
|
||||
assert(add.fundingFee_opt.nonEmpty)
|
||||
assert(add.fundingFee_opt.get.fundingTxId == purchase.txId)
|
||||
})
|
||||
adds1.take(2).foreach(add => assert(!add.commit))
|
||||
assert(adds1.last.commit)
|
||||
assert(adds1.map(_.fundingFee_opt.get.amount).sum == fees.total.toMilliSatoshi)
|
||||
assert(adds1.map(add => add.amount + add.fundingFee_opt.get.amount).sum == 160_000_000.msat)
|
||||
channel.expectNoMessage(100 millis)
|
||||
|
||||
// The recipient fails the payments: we don't relay the failure upstream and will retry.
|
||||
adds1.take(2).foreach(add => {
|
||||
val htlc = UpdateAddHtlc(channelId, randomHtlcId(), add.amount, paymentHash, add.cltvExpiry, add.onion, add.nextBlindingKey_opt, add.confidence, add.fundingFee_opt)
|
||||
val fail = UpdateFailHtlc(channelId, htlc.id, randomBytes(50))
|
||||
add.replyTo ! RES_SUCCESS(add, purchase.channelId)
|
||||
add.replyTo ! RES_ADD_SETTLED(add.origin, htlc, HtlcResult.RemoteFail(fail))
|
||||
})
|
||||
adds1.last.replyTo ! RES_ADD_FAILED(adds1.last, TooManyAcceptedHtlcs(channelId, 5), None)
|
||||
register.expectNoMessage(100 millis)
|
||||
|
||||
// When the next splice completes, we retry the payment.
|
||||
peer ! ChannelReadyForPayments(channel.ref, remoteNodeId, channelId, fundingTxIndex = 6)
|
||||
channel.expectMsgType[CMD_GET_CHANNEL_INFO].replyTo ! RES_GET_CHANNEL_INFO(remoteNodeId, channelId, channel.ref, NORMAL, channelData)
|
||||
val adds2 = Seq(
|
||||
channel.expectMsgType[CMD_ADD_HTLC],
|
||||
channel.expectMsgType[CMD_ADD_HTLC],
|
||||
channel.expectMsgType[CMD_ADD_HTLC],
|
||||
)
|
||||
adds2.foreach(add => add.replyTo ! RES_SUCCESS(add, purchase.channelId))
|
||||
channel.expectNoMessage(100 millis)
|
||||
|
||||
// The payment succeeds.
|
||||
adds2.foreach(add => {
|
||||
val htlc = UpdateAddHtlc(channelId, randomHtlcId(), add.amount, paymentHash, add.cltvExpiry, add.onion, add.nextBlindingKey_opt, add.confidence, add.fundingFee_opt)
|
||||
add.replyTo ! RES_ADD_SETTLED(add.origin, htlc, HtlcResult.OnChainFulfill(preimage))
|
||||
})
|
||||
val fwds = Seq(
|
||||
register.expectMsgType[Register.Forward[CMD_FULFILL_HTLC]],
|
||||
register.expectMsgType[Register.Forward[CMD_FULFILL_HTLC]],
|
||||
register.expectMsgType[Register.Forward[CMD_FULFILL_HTLC]],
|
||||
)
|
||||
val (channelsIn, htlcsIn) = upstream.flatMap {
|
||||
case u: Hot.Channel => Seq(u)
|
||||
case u: Hot.Trampoline => u.received
|
||||
case _: Upstream.Local => Nil
|
||||
}.map(c => (c.add.channelId, c.add.id)).toSet.unzip
|
||||
assert(fwds.map(_.channelId).toSet == channelsIn)
|
||||
assert(fwds.map(_.message.id).toSet == htlcsIn)
|
||||
fwds.foreach(fwd => assert(fwd.message.r == preimage))
|
||||
awaitCond(nodeParams.db.liquidity.listPendingOnTheFlyFunding(remoteNodeId).isEmpty, interval = 100 millis)
|
||||
|
||||
// We don't retry anymore.
|
||||
peer ! ChannelReadyForPayments(channel.ref, remoteNodeId, channelId, fundingTxIndex = 7)
|
||||
channel.expectNoMessage(100 millis)
|
||||
}
|
||||
|
||||
test("successfully relay HTLCs after restart") { f =>
|
||||
import f._
|
||||
|
||||
// We create a channel, that can later be spliced.
|
||||
connect(peer)
|
||||
val channelId = openChannel(250_000 sat)
|
||||
|
||||
// We relay an on-the-fly payment.
|
||||
val upstream = upstreamChannel(50_000_000 msat, expiryIn, paymentHash)
|
||||
proposeFunding(50_000_000 msat, expiryOut, paymentHash, upstream)
|
||||
val fees = LiquidityAds.Fees(1000 sat, 1000 sat)
|
||||
val purchase = signLiquidityPurchase(200_000 sat, LiquidityAds.PaymentDetails.FromChannelBalanceForFutureHtlc(paymentHash :: Nil), channelId, fees, fundingTxIndex = 1)
|
||||
peer ! ChannelReadyForPayments(channel.ref, remoteNodeId, channelId, fundingTxIndex = 1)
|
||||
val channelData1 = makeChannelData()
|
||||
channel.expectMsgType[CMD_GET_CHANNEL_INFO].replyTo ! RES_GET_CHANNEL_INFO(remoteNodeId, channelId, channel.ref, NORMAL, channelData1)
|
||||
// We don't collect additional fees if they were paid from our peer's channel balance already.
|
||||
val cmd1 = channel.expectMsgType[CMD_ADD_HTLC]
|
||||
cmd1.replyTo ! RES_SUCCESS(cmd1, purchase.channelId)
|
||||
val htlc = UpdateAddHtlc(channelId, 0, cmd1.amount, paymentHash, cmd1.cltvExpiry, cmd1.onion, cmd1.nextBlindingKey_opt, cmd1.confidence, cmd1.fundingFee_opt)
|
||||
assert(cmd1.fundingFee_opt.contains(LiquidityAds.FundingFee(0 msat, purchase.txId)))
|
||||
channel.expectNoMessage(100 millis)
|
||||
|
||||
// We disconnect: on reconnection, we don't attempt the payment again since it's already pending.
|
||||
disconnect(channelCount = 1)
|
||||
connect(peer, channelCount = 1)
|
||||
peer ! ChannelReadyForPayments(channel.ref, remoteNodeId, channelId, fundingTxIndex = 1)
|
||||
channel.expectNoMessage(100 millis)
|
||||
register.expectNoMessage(100 millis)
|
||||
|
||||
// On restart, we don't attempt the payment again: it's already pending.
|
||||
val peerAfterRestart = TestFSMRef(new Peer(nodeParams, remoteNodeId, new DummyOnChainWallet(), FakeChannelFactory(remoteNodeId, channel), TestProbe().ref, register.ref, TestProbe().ref, TestProbe().ref))
|
||||
peerAfterRestart ! Peer.Init(Set.empty, nodeParams.db.liquidity.listPendingOnTheFlyFunding(remoteNodeId))
|
||||
connect(peerAfterRestart)
|
||||
peerAfterRestart ! ChannelReadyForPayments(channel.ref, remoteNodeId, channelId, fundingTxIndex = 1)
|
||||
val channelData2 = makeChannelData(localChanges = LocalChanges(Nil, htlc :: Nil, Nil))
|
||||
channel.expectMsgType[CMD_GET_CHANNEL_INFO].replyTo ! RES_GET_CHANNEL_INFO(remoteNodeId, channelId, channel.ref, NORMAL, channelData2)
|
||||
channel.expectNoMessage(100 millis)
|
||||
|
||||
// The payment is failed by our peer but we don't see it (it's a cold origin): we attempt it again.
|
||||
val channelData3 = makeChannelData()
|
||||
peerAfterRestart ! ChannelReadyForPayments(channel.ref, remoteNodeId, channelId, fundingTxIndex = 1)
|
||||
channel.expectMsgType[CMD_GET_CHANNEL_INFO].replyTo ! RES_GET_CHANNEL_INFO(remoteNodeId, channelId, channel.ref, NORMAL, channelData3)
|
||||
val cmd2 = channel.expectMsgType[CMD_ADD_HTLC]
|
||||
cmd2.replyTo ! RES_SUCCESS(cmd2, purchase.channelId)
|
||||
assert(cmd2.paymentHash == paymentHash)
|
||||
assert(cmd2.amount == cmd1.amount)
|
||||
channel.expectNoMessage(100 millis)
|
||||
|
||||
// The payment is fulfilled by our peer.
|
||||
cmd2.replyTo ! RES_ADD_SETTLED(cmd2.origin, htlc, HtlcResult.OnChainFulfill(preimage))
|
||||
assert(register.expectMsgType[Register.Forward[CMD_FULFILL_HTLC]].channelId == upstream.add.channelId)
|
||||
nodeParams.db.liquidity.addOnTheFlyFundingPreimage(preimage)
|
||||
register.expectNoMessage(100 millis)
|
||||
awaitCond(nodeParams.db.liquidity.listPendingOnTheFlyFunding(remoteNodeId).isEmpty, interval = 100 millis)
|
||||
}
|
||||
|
||||
test("don't relay payments too close to expiry") { f =>
|
||||
import f._
|
||||
|
||||
connect(peer)
|
||||
|
||||
val upstream = upstreamChannel(100_000_000 msat, expiryIn, paymentHash)
|
||||
proposeFunding(100_000_000 msat, CltvExpiry(TestConstants.defaultBlockHeight), paymentHash, upstream)
|
||||
val purchase = signLiquidityPurchase(200_000 sat, LiquidityAds.PaymentDetails.FromFutureHtlc(List(paymentHash)))
|
||||
|
||||
// We're too close the HTLC expiry to relay it.
|
||||
peer ! ChannelReadyForPayments(channel.ref, remoteNodeId, purchase.channelId, fundingTxIndex = 0)
|
||||
channel.expectNoMessage(100 millis)
|
||||
peer ! CurrentBlockHeight(BlockHeight(TestConstants.defaultBlockHeight))
|
||||
val fwd = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]]
|
||||
assert(fwd.channelId == upstream.add.channelId)
|
||||
assert(fwd.message.id == upstream.add.id)
|
||||
awaitCond(nodeParams.db.liquidity.listPendingOnTheFlyFunding(remoteNodeId).isEmpty, interval = 100 millis)
|
||||
}
|
||||
|
||||
test("don't relay payments for known preimage") { f =>
|
||||
import f._
|
||||
|
||||
connect(peer)
|
||||
|
||||
val upstream = upstreamChannel(100_000_000 msat, expiryIn, paymentHash)
|
||||
proposeFunding(100_000_000 msat, expiryOut, paymentHash, upstream)
|
||||
val purchase = signLiquidityPurchase(200_000 sat, LiquidityAds.PaymentDetails.FromFutureHtlc(List(paymentHash)))
|
||||
|
||||
// We've already relayed that payment and have the matching preimage in our DB.
|
||||
// We don't relay it again to avoid paying our peer twice.
|
||||
nodeParams.db.liquidity.addOnTheFlyFundingPreimage(preimage)
|
||||
peer ! ChannelReadyForPayments(channel.ref, remoteNodeId, purchase.channelId, fundingTxIndex = 0)
|
||||
channel.expectMsgType[CMD_GET_CHANNEL_INFO].replyTo ! RES_GET_CHANNEL_INFO(remoteNodeId, purchase.channelId, channel.ref, NORMAL, makeChannelData())
|
||||
channel.expectNoMessage(100 millis)
|
||||
|
||||
val fwd = register.expectMsgType[Register.Forward[CMD_FULFILL_HTLC]]
|
||||
assert(fwd.channelId == upstream.add.channelId)
|
||||
assert(fwd.message.id == upstream.add.id)
|
||||
assert(fwd.message.r == preimage)
|
||||
register.expectNoMessage(100 millis)
|
||||
}
|
||||
|
||||
test("stop when disconnecting without pending proposals") { f =>
|
||||
import f._
|
||||
|
||||
connect(peer)
|
||||
probe.watch(peer.ref)
|
||||
disconnect()
|
||||
probe.expectTerminated(peer.ref)
|
||||
}
|
||||
|
||||
test("stop when disconnecting with non-funded pending proposals") { f =>
|
||||
import f._
|
||||
|
||||
connect(peer)
|
||||
probe.watch(peer.ref)
|
||||
|
||||
// We have two distinct pending funding proposals.
|
||||
val paymentHash1 = randomBytes32()
|
||||
val upstream1 = upstreamChannel(300_000_000 msat, CltvExpiry(1200), paymentHash1)
|
||||
proposeFunding(250_000_000 msat, CltvExpiry(1105), paymentHash1, upstream1)
|
||||
val paymentHash2 = randomBytes32()
|
||||
val upstream2 = Upstream.Hot.Trampoline(List(
|
||||
upstreamChannel(100_000_000 msat, CltvExpiry(1250), paymentHash2),
|
||||
upstreamChannel(150_000_000 msat, CltvExpiry(1240), paymentHash2),
|
||||
))
|
||||
proposeFunding(225_000_000 msat, CltvExpiry(1105), paymentHash2, upstream2)
|
||||
|
||||
// All incoming HTLCs are failed on disconnection.
|
||||
disconnect()
|
||||
(upstream1.add +: upstream2.received.map(_.add)).foreach(add => {
|
||||
val fwd = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]]
|
||||
assert(fwd.channelId == add.channelId)
|
||||
assert(fwd.message.id == add.id)
|
||||
assert(fwd.message.reason == Right(UnknownNextPeer()))
|
||||
assert(fwd.message.commit)
|
||||
})
|
||||
register.expectNoMessage(100 millis)
|
||||
probe.expectTerminated(peer.ref)
|
||||
}
|
||||
|
||||
test("don't stop when disconnecting with funded pending proposals") { f =>
|
||||
import f._
|
||||
|
||||
connect(peer)
|
||||
probe.watch(peer.ref)
|
||||
|
||||
// We have one funded proposal and one that was not funded yet.
|
||||
val upstream1 = upstreamChannel(300_000_000 msat, CltvExpiry(1200))
|
||||
proposeFunding(250_000_000 msat, CltvExpiry(1105), upstream1.add.paymentHash, upstream1)
|
||||
val upstream2 = upstreamChannel(250_000_000 msat, CltvExpiry(1000))
|
||||
proposeFunding(220_000_000 msat, CltvExpiry(1105), upstream2.add.paymentHash, upstream2)
|
||||
signLiquidityPurchase(500_000 sat, LiquidityAds.PaymentDetails.FromFutureHtlc(upstream2.add.paymentHash :: Nil))
|
||||
|
||||
// Only non-funded proposals are failed on disconnection, and we don't stop before the funded proposal completes.
|
||||
disconnect()
|
||||
val fwd = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]]
|
||||
assert(fwd.channelId == upstream1.add.channelId)
|
||||
assert(fwd.message.id == upstream1.add.id)
|
||||
assert(fwd.message.reason == Right(UnknownNextPeer()))
|
||||
assert(fwd.message.commit)
|
||||
register.expectNoMessage(100 millis)
|
||||
probe.expectNoMessage(100 millis)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
object OnTheFlyFundingSpec {
|
||||
|
||||
val expiryIn = CltvExpiry(TestConstants.defaultBlockHeight + 750)
|
||||
val expiryOut = CltvExpiry(TestConstants.defaultBlockHeight + 500)
|
||||
|
||||
val preimage = randomBytes32()
|
||||
val paymentHash = Crypto.sha256(preimage)
|
||||
|
||||
def randomOnion(): OnionRoutingPacket = OnionRoutingPacket(0, randomKey().publicKey.value, randomBytes(1300), randomBytes32())
|
||||
|
||||
def randomHtlcId(): Long = Math.abs(randomLong()) % 50_000
|
||||
|
||||
def upstreamChannel(amountIn: MilliSatoshi, expiryIn: CltvExpiry, paymentHash: ByteVector32 = randomBytes32(), blinded: Boolean = false): Upstream.Hot.Channel = {
|
||||
val blindingKey = if (blinded) Some(randomKey().publicKey) else None
|
||||
val add = UpdateAddHtlc(randomBytes32(), randomHtlcId(), amountIn, paymentHash, expiryIn, TestConstants.emptyOnionPacket, blindingKey, 1.0, None)
|
||||
Upstream.Hot.Channel(add, TimestampMilli.now(), randomKey().publicKey)
|
||||
}
|
||||
|
||||
def createWillAdd(amount: MilliSatoshi, paymentHash: ByteVector32, expiry: CltvExpiry, blinding_opt: Option[PublicKey] = None): WillAddHtlc = {
|
||||
WillAddHtlc(Block.RegtestGenesisBlock.hash, randomBytes32(), amount, paymentHash, expiry, randomOnion(), blinding_opt)
|
||||
}
|
||||
|
||||
def createStatus(): OnTheFlyFunding.Status = OnTheFlyFunding.Status.Funded(randomBytes32(), TxId(randomBytes32()), 0, 2500 msat)
|
||||
|
||||
def createOpenChannelMessage(requestFunding: LiquidityAds.RequestFunding, fundingAmount: Satoshi = 250_000 sat, htlcMinimum: MilliSatoshi = 1 msat): OpenDualFundedChannel = {
|
||||
val channelFlags = ChannelFlags(nonInitiatorPaysCommitFees = true, announceChannel = false)
|
||||
val tlvs = TlvStream[OpenDualFundedChannelTlv](ChannelTlv.ChannelTypeTlv(ChannelTypes.AnchorOutputsZeroFeeHtlcTx()), ChannelTlv.RequestFundingTlv(requestFunding))
|
||||
OpenDualFundedChannel(Block.RegtestGenesisBlock.hash, randomBytes32(), TestConstants.feeratePerKw, TestConstants.anchorOutputsFeeratePerKw, fundingAmount, 483 sat, UInt64(100), htlcMinimum, CltvExpiryDelta(144), 10, 0, randomKey().publicKey, randomKey().publicKey, randomKey().publicKey, randomKey().publicKey, randomKey().publicKey, randomKey().publicKey, randomKey().publicKey, channelFlags, tlvs)
|
||||
}
|
||||
|
||||
def createSpliceMessage(channelId: ByteVector32, requestFunding: LiquidityAds.RequestFunding): SpliceInit = {
|
||||
SpliceInit(channelId, 0 sat, 0, TestConstants.feeratePerKw, randomKey().publicKey, 0 msat, requireConfirmedInputs = false, Some(requestFunding))
|
||||
}
|
||||
|
||||
}
|
@ -20,8 +20,9 @@ import akka.actor.testkit.typed.scaladsl.{ScalaTestWithActorTestKit, TestProbe}
|
||||
import akka.actor.typed.eventstream.EventStream
|
||||
import akka.actor.typed.scaladsl.Behaviors
|
||||
import akka.actor.typed.scaladsl.adapter.{TypedActorContextOps, TypedActorRefOps}
|
||||
import akka.testkit.TestKit.awaitCond
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import fr.acinq.bitcoin.scalacompat.{Block, ByteVector32}
|
||||
import fr.acinq.bitcoin.scalacompat.{Block, ByteVector32, Crypto, TxId}
|
||||
import fr.acinq.eclair.FeatureSupport.{Mandatory, Optional}
|
||||
import fr.acinq.eclair.Features._
|
||||
import fr.acinq.eclair._
|
||||
@ -91,7 +92,7 @@ class RelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat
|
||||
// we use this to build a valid onion
|
||||
val Right(payment) = buildOutgoingPayment(TestConstants.emptyOrigin, paymentHash, Route(finalAmount, hops, None), ClearRecipient(e, Features.empty, finalAmount, finalExpiry, paymentSecret), 1.0)
|
||||
// and then manually build an htlc
|
||||
val add_ab = UpdateAddHtlc(randomBytes32(), 123456, payment.cmd.amount, payment.cmd.paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, None, 1.0)
|
||||
val add_ab = UpdateAddHtlc(randomBytes32(), 123456, payment.cmd.amount, payment.cmd.paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, None, 1.0, None)
|
||||
relayer ! RelayForward(add_ab, priv_a.publicKey)
|
||||
register.expectMessageType[Register.Forward[CMD_ADD_HTLC]]
|
||||
}
|
||||
@ -100,7 +101,7 @@ class RelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat
|
||||
import f._
|
||||
|
||||
val Right(payment) = buildOutgoingPayment(TestConstants.emptyOrigin, paymentHash, Route(finalAmount, hops.take(1), None), ClearRecipient(b, Features.empty, finalAmount, finalExpiry, paymentSecret), 1.0)
|
||||
val add_ab = UpdateAddHtlc(channelId_ab, 123456, payment.cmd.amount, payment.cmd.paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, None, 1.0)
|
||||
val add_ab = UpdateAddHtlc(channelId_ab, 123456, payment.cmd.amount, payment.cmd.paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, None, 1.0, None)
|
||||
relayer ! RelayForward(add_ab, priv_a.publicKey)
|
||||
|
||||
val fp = paymentHandler.expectMessageType[FinalPacket]
|
||||
@ -121,7 +122,7 @@ class RelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat
|
||||
val Right(payment) = buildOutgoingPayment(TestConstants.emptyOrigin, paymentHash, Route(finalAmount, Seq(channelHopFromUpdate(priv_a.publicKey, b, channelUpdate_ab)), None), recipient, 1.0)
|
||||
assert(payment.cmd.amount == finalAmount)
|
||||
assert(payment.cmd.cltvExpiry == finalExpiry)
|
||||
val add_ab = UpdateAddHtlc(channelId_ab, 123456, payment.cmd.amount, payment.cmd.paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, None, 1.0)
|
||||
val add_ab = UpdateAddHtlc(channelId_ab, 123456, payment.cmd.amount, payment.cmd.paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, None, 1.0, None)
|
||||
relayer ! RelayForward(add_ab, priv_a.publicKey)
|
||||
|
||||
val fp = paymentHandler.expectMessageType[FinalPacket]
|
||||
@ -141,7 +142,7 @@ class RelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat
|
||||
// we use this to build a valid onion
|
||||
val Right(payment) = buildOutgoingPayment(TestConstants.emptyOrigin, paymentHash, Route(finalAmount, hops, None), ClearRecipient(e, Features.empty, finalAmount, finalExpiry, paymentSecret), 1.0)
|
||||
// and then manually build an htlc with an invalid onion (hmac)
|
||||
val add_ab = UpdateAddHtlc(channelId_ab, 123456, payment.cmd.amount, payment.cmd.paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion.copy(hmac = payment.cmd.onion.hmac.reverse), None, 1.0)
|
||||
val add_ab = UpdateAddHtlc(channelId_ab, 123456, payment.cmd.amount, payment.cmd.paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion.copy(hmac = payment.cmd.onion.hmac.reverse), None, 1.0, None)
|
||||
relayer ! RelayForward(add_ab, priv_a.publicKey)
|
||||
|
||||
val fail = register.expectMessageType[Register.Forward[CMD_FAIL_MALFORMED_HTLC]].message
|
||||
@ -160,7 +161,7 @@ class RelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat
|
||||
val (_, blindedHop, recipient) = blindedRouteFromHops(finalAmount, finalExpiry, Seq(channelHopFromUpdate(b, c, channelUpdate_bc)), routeExpiry, paymentPreimage, hex"deadbeef")
|
||||
val route = Route(finalAmount, Seq(channelHopFromUpdate(priv_a.publicKey, b, channelUpdate_ab)), Some(blindedHop))
|
||||
val Right(payment) = buildOutgoingPayment(TestConstants.emptyOrigin, paymentHash, route, recipient, 1.0)
|
||||
val add_ab = UpdateAddHtlc(channelId_ab, 0, payment.cmd.amount, payment.cmd.paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, payment.cmd.nextBlindingKey_opt, 1.0)
|
||||
val add_ab = UpdateAddHtlc(channelId_ab, 0, payment.cmd.amount, payment.cmd.paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, payment.cmd.nextBlindingKey_opt, 1.0, payment.cmd.fundingFee_opt)
|
||||
relayer ! RelayForward(add_ab, priv_a.publicKey)
|
||||
|
||||
val fail = register.expectMessageType[Register.Forward[CMD_FAIL_HTLC]].message
|
||||
@ -176,7 +177,7 @@ class RelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat
|
||||
|
||||
// we use an expired blinded route.
|
||||
val Right(payment) = buildOutgoingBlindedPaymentAB(paymentHash, routeExpiry = CltvExpiry(nodeParams.currentBlockHeight - 1))
|
||||
val add_ab = UpdateAddHtlc(channelId_ab, 0, payment.cmd.amount, payment.cmd.paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, payment.cmd.nextBlindingKey_opt, 1.0)
|
||||
val add_ab = UpdateAddHtlc(channelId_ab, 0, payment.cmd.amount, payment.cmd.paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, payment.cmd.nextBlindingKey_opt, 1.0, payment.cmd.fundingFee_opt)
|
||||
relayer ! RelayForward(add_ab, priv_a.publicKey)
|
||||
|
||||
val fail = register.expectMessageType[Register.Forward[CMD_FAIL_MALFORMED_HTLC]].message
|
||||
@ -198,7 +199,7 @@ class RelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat
|
||||
val Right(payment) = buildOutgoingPayment(TestConstants.emptyOrigin, paymentHash, Route(recipient.trampolineAmount, Seq(channelHopFromUpdate(priv_a.publicKey, b, channelUpdate_ab)), Some(trampolineHop)), recipient, 1.0)
|
||||
|
||||
// and then manually build an htlc
|
||||
val add_ab = UpdateAddHtlc(channelId_ab, 123456, payment.cmd.amount, payment.cmd.paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, None, 1.0)
|
||||
val add_ab = UpdateAddHtlc(channelId_ab, 123456, payment.cmd.amount, payment.cmd.paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, None, 1.0, None)
|
||||
relayer ! RelayForward(add_ab, priv_a.publicKey)
|
||||
|
||||
val fail = register.expectMessageType[Register.Forward[CMD_FAIL_HTLC]].message
|
||||
@ -212,10 +213,10 @@ class RelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat
|
||||
import f._
|
||||
|
||||
val replyTo = TestProbe[Any]()
|
||||
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 42, amountMsat = 11000000 msat, paymentHash = ByteVector32.Zeroes, CltvExpiry(4200), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val add_bc = UpdateAddHtlc(channelId_bc, 72, 1000 msat, paymentHash, CltvExpiry(1), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val add_ab = UpdateAddHtlc(channelId_ab, 42, 11000000 msat, ByteVector32.Zeroes, CltvExpiry(4200), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
val add_bc = UpdateAddHtlc(channelId_bc, 72, 1000 msat, paymentHash, CltvExpiry(1), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
val channelOrigin = Origin.Hot(replyTo.ref.toClassic, Upstream.Hot.Channel(add_ab, TimestampMilli.now(), priv_a.publicKey))
|
||||
val trampolineOrigin = Origin.Hot(replyTo.ref.toClassic, Upstream.Hot.Trampoline(Seq(Upstream.Hot.Channel(add_ab, TimestampMilli.now(), priv_a.publicKey))))
|
||||
val trampolineOrigin = Origin.Hot(replyTo.ref.toClassic, Upstream.Hot.Trampoline(List(Upstream.Hot.Channel(add_ab, TimestampMilli.now(), priv_a.publicKey))))
|
||||
|
||||
val addSettled = Seq(
|
||||
RES_ADD_SETTLED(channelOrigin, add_bc, HtlcResult.OnChainFulfill(randomBytes32())),
|
||||
@ -234,4 +235,29 @@ class RelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat
|
||||
}
|
||||
}
|
||||
|
||||
test("store on-the-fly funding preimage") { f =>
|
||||
import f._
|
||||
|
||||
val replyTo = TestProbe[Any]()
|
||||
val add_ab = UpdateAddHtlc(channelId_ab, 17, 50_000 msat, paymentHash, CltvExpiry(800_000), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
val add_bc = UpdateAddHtlc(channelId_bc, 21, 45_000 msat, paymentHash, CltvExpiry(799_000), TestConstants.emptyOnionPacket, None, 1.0, Some(LiquidityAds.FundingFee(1000 msat, TxId(randomBytes32()))))
|
||||
val originHot = Origin.Hot(replyTo.ref.toClassic, Upstream.Hot.Channel(add_ab, TimestampMilli.now(), randomKey().publicKey))
|
||||
val originCold = Origin.Cold(originHot)
|
||||
|
||||
val addFulfilled = Seq(
|
||||
RES_ADD_SETTLED(originHot, add_bc, HtlcResult.OnChainFulfill(randomBytes32())),
|
||||
RES_ADD_SETTLED(originHot, add_bc, HtlcResult.RemoteFulfill(UpdateFulfillHtlc(add_bc.channelId, add_bc.id, randomBytes32()))),
|
||||
RES_ADD_SETTLED(originCold, add_bc, HtlcResult.OnChainFulfill(randomBytes32())),
|
||||
RES_ADD_SETTLED(originCold, add_bc, HtlcResult.RemoteFulfill(UpdateFulfillHtlc(add_bc.channelId, add_bc.id, randomBytes32()))),
|
||||
)
|
||||
|
||||
for (res <- addFulfilled) {
|
||||
val preimage = res.result.paymentPreimage
|
||||
val paymentHash = Crypto.sha256(preimage)
|
||||
assert(nodeParams.db.liquidity.getOnTheFlyFundingPreimage(paymentHash).isEmpty)
|
||||
relayer ! res
|
||||
awaitCond(nodeParams.db.liquidity.getOnTheFlyFundingPreimage(paymentHash).contains(preimage), 10 seconds)
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -29,11 +29,11 @@ class CommitmentSpecSpec extends AnyFunSuite {
|
||||
val R = randomBytes32()
|
||||
val H = Crypto.sha256(R)
|
||||
|
||||
val add1 = UpdateAddHtlc(ByteVector32.Zeroes, 1, (2000 * 1000) msat, H, CltvExpiry(400), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val add1 = UpdateAddHtlc(ByteVector32.Zeroes, 1, (2000 * 1000) msat, H, CltvExpiry(400), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
val spec1 = CommitmentSpec.reduce(spec, add1 :: Nil, Nil)
|
||||
assert(spec1 == spec.copy(htlcs = Set(OutgoingHtlc(add1)), toLocal = 3000000 msat))
|
||||
|
||||
val add2 = UpdateAddHtlc(ByteVector32.Zeroes, 2, (1000 * 1000) msat, H, CltvExpiry(400), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val add2 = UpdateAddHtlc(ByteVector32.Zeroes, 2, (1000 * 1000) msat, H, CltvExpiry(400), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
val spec2 = CommitmentSpec.reduce(spec1, add2 :: Nil, Nil)
|
||||
assert(spec2 == spec1.copy(htlcs = Set(OutgoingHtlc(add1), OutgoingHtlc(add2)), toLocal = 2000000 msat))
|
||||
|
||||
@ -51,11 +51,11 @@ class CommitmentSpecSpec extends AnyFunSuite {
|
||||
val R = randomBytes32()
|
||||
val H = Crypto.sha256(R)
|
||||
|
||||
val add1 = UpdateAddHtlc(ByteVector32.Zeroes, 1, (2000 * 1000) msat, H, CltvExpiry(400), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val add1 = UpdateAddHtlc(ByteVector32.Zeroes, 1, (2000 * 1000) msat, H, CltvExpiry(400), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
val spec1 = CommitmentSpec.reduce(spec, Nil, add1 :: Nil)
|
||||
assert(spec1 == spec.copy(htlcs = Set(IncomingHtlc(add1)), toRemote = 3000 * 1000 msat))
|
||||
|
||||
val add2 = UpdateAddHtlc(ByteVector32.Zeroes, 2, (1000 * 1000) msat, H, CltvExpiry(400), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val add2 = UpdateAddHtlc(ByteVector32.Zeroes, 2, (1000 * 1000) msat, H, CltvExpiry(400), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
val spec2 = CommitmentSpec.reduce(spec1, Nil, add2 :: Nil)
|
||||
assert(spec2 == spec1.copy(htlcs = Set(IncomingHtlc(add1), IncomingHtlc(add2)), toRemote = (2000 * 1000) msat))
|
||||
|
||||
@ -76,7 +76,7 @@ class CommitmentSpecSpec extends AnyFunSuite {
|
||||
}
|
||||
|
||||
def createHtlc(amount: MilliSatoshi): UpdateAddHtlc = {
|
||||
UpdateAddHtlc(ByteVector32.Zeroes, 0, amount, randomBytes32(), CltvExpiry(500), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
UpdateAddHtlc(ByteVector32.Zeroes, 0, amount, randomBytes32(), CltvExpiry(500), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -153,13 +153,13 @@ trait TestVectorsSpec extends AnyFunSuite with Logging {
|
||||
)
|
||||
|
||||
val htlcs = Seq[DirectedHtlc](
|
||||
IncomingHtlc(UpdateAddHtlc(ByteVector32.Zeroes, 0, 1000000 msat, Crypto.sha256(paymentPreimages(0)), CltvExpiry(500), TestConstants.emptyOnionPacket, None, 1.0)),
|
||||
IncomingHtlc(UpdateAddHtlc(ByteVector32.Zeroes, 1, 2000000 msat, Crypto.sha256(paymentPreimages(1)), CltvExpiry(501), TestConstants.emptyOnionPacket, None, 1.0)),
|
||||
OutgoingHtlc(UpdateAddHtlc(ByteVector32.Zeroes, 0, 2000000 msat, Crypto.sha256(paymentPreimages(2)), CltvExpiry(502), TestConstants.emptyOnionPacket, None, 1.0)),
|
||||
OutgoingHtlc(UpdateAddHtlc(ByteVector32.Zeroes, 1, 3000000 msat, Crypto.sha256(paymentPreimages(3)), CltvExpiry(503), TestConstants.emptyOnionPacket, None, 1.0)),
|
||||
IncomingHtlc(UpdateAddHtlc(ByteVector32.Zeroes, 2, 4000000 msat, Crypto.sha256(paymentPreimages(4)), CltvExpiry(504), TestConstants.emptyOnionPacket, None, 1.0)),
|
||||
OutgoingHtlc(UpdateAddHtlc(ByteVector32.Zeroes, 2, 5000001.msat, Crypto.sha256(paymentPreimages(5)), CltvExpiry(505), TestConstants.emptyOnionPacket, None, 1.0)),
|
||||
OutgoingHtlc(UpdateAddHtlc(ByteVector32.Zeroes, 3, 5000000.msat, Crypto.sha256(paymentPreimages(5)), CltvExpiry(506), TestConstants.emptyOnionPacket, None, 1.0))
|
||||
IncomingHtlc(UpdateAddHtlc(ByteVector32.Zeroes, 0, 1000000 msat, Crypto.sha256(paymentPreimages(0)), CltvExpiry(500), TestConstants.emptyOnionPacket, None, 1.0, None)),
|
||||
IncomingHtlc(UpdateAddHtlc(ByteVector32.Zeroes, 1, 2000000 msat, Crypto.sha256(paymentPreimages(1)), CltvExpiry(501), TestConstants.emptyOnionPacket, None, 1.0, None)),
|
||||
OutgoingHtlc(UpdateAddHtlc(ByteVector32.Zeroes, 0, 2000000 msat, Crypto.sha256(paymentPreimages(2)), CltvExpiry(502), TestConstants.emptyOnionPacket, None, 1.0, None)),
|
||||
OutgoingHtlc(UpdateAddHtlc(ByteVector32.Zeroes, 1, 3000000 msat, Crypto.sha256(paymentPreimages(3)), CltvExpiry(503), TestConstants.emptyOnionPacket, None, 1.0, None)),
|
||||
IncomingHtlc(UpdateAddHtlc(ByteVector32.Zeroes, 2, 4000000 msat, Crypto.sha256(paymentPreimages(4)), CltvExpiry(504), TestConstants.emptyOnionPacket, None, 1.0, None)),
|
||||
OutgoingHtlc(UpdateAddHtlc(ByteVector32.Zeroes, 2, 5000001.msat, Crypto.sha256(paymentPreimages(5)), CltvExpiry(505), TestConstants.emptyOnionPacket, None, 1.0, None)),
|
||||
OutgoingHtlc(UpdateAddHtlc(ByteVector32.Zeroes, 3, 5000000.msat, Crypto.sha256(paymentPreimages(5)), CltvExpiry(506), TestConstants.emptyOnionPacket, None, 1.0, None))
|
||||
)
|
||||
val htlcScripts = htlcs.map {
|
||||
case OutgoingHtlc(add) => Scripts.htlcOffered(Local.htlc_privkey.publicKey, Remote.htlc_privkey.publicKey, Local.revocation_pubkey, Crypto.ripemd160(add.paymentHash), commitmentFormat)
|
||||
|
@ -100,10 +100,10 @@ class TransactionsSpec extends AnyFunSuite with Logging {
|
||||
test("compute fees") {
|
||||
// see BOLT #3 specs
|
||||
val htlcs = Set[DirectedHtlc](
|
||||
OutgoingHtlc(UpdateAddHtlc(ByteVector32.Zeroes, 0, 5000000 msat, ByteVector32.Zeroes, CltvExpiry(552), TestConstants.emptyOnionPacket, None, 1.0)),
|
||||
OutgoingHtlc(UpdateAddHtlc(ByteVector32.Zeroes, 0, 1000000 msat, ByteVector32.Zeroes, CltvExpiry(553), TestConstants.emptyOnionPacket, None, 1.0)),
|
||||
IncomingHtlc(UpdateAddHtlc(ByteVector32.Zeroes, 0, 7000000 msat, ByteVector32.Zeroes, CltvExpiry(550), TestConstants.emptyOnionPacket, None, 1.0)),
|
||||
IncomingHtlc(UpdateAddHtlc(ByteVector32.Zeroes, 0, 800000 msat, ByteVector32.Zeroes, CltvExpiry(551), TestConstants.emptyOnionPacket, None, 1.0))
|
||||
OutgoingHtlc(UpdateAddHtlc(ByteVector32.Zeroes, 0, 5000000 msat, ByteVector32.Zeroes, CltvExpiry(552), TestConstants.emptyOnionPacket, None, 1.0, None)),
|
||||
OutgoingHtlc(UpdateAddHtlc(ByteVector32.Zeroes, 0, 1000000 msat, ByteVector32.Zeroes, CltvExpiry(553), TestConstants.emptyOnionPacket, None, 1.0, None)),
|
||||
IncomingHtlc(UpdateAddHtlc(ByteVector32.Zeroes, 0, 7000000 msat, ByteVector32.Zeroes, CltvExpiry(550), TestConstants.emptyOnionPacket, None, 1.0, None)),
|
||||
IncomingHtlc(UpdateAddHtlc(ByteVector32.Zeroes, 0, 800000 msat, ByteVector32.Zeroes, CltvExpiry(551), TestConstants.emptyOnionPacket, None, 1.0, None))
|
||||
)
|
||||
val spec = CommitmentSpec(htlcs, FeeratePerKw(5000 sat), toLocal = 0 msat, toRemote = 0 msat)
|
||||
val fee = commitTxFeeMsat(546 sat, spec, DefaultCommitmentFormat)
|
||||
@ -154,7 +154,7 @@ class TransactionsSpec extends AnyFunSuite with Logging {
|
||||
// HtlcPenaltyTx
|
||||
// first we create a fake commitTx tx, containing only the output that will be spent by the ClaimHtlcSuccessTx
|
||||
val paymentPreimage = randomBytes32()
|
||||
val htlc = UpdateAddHtlc(ByteVector32.Zeroes, 0, (20000 * 1000) msat, sha256(paymentPreimage), CltvExpiryDelta(144).toCltvExpiry(blockHeight), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val htlc = UpdateAddHtlc(ByteVector32.Zeroes, 0, (20000 * 1000) msat, sha256(paymentPreimage), CltvExpiryDelta(144).toCltvExpiry(blockHeight), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
val redeemScript = htlcReceived(localHtlcPriv.publicKey, remoteHtlcPriv.publicKey, localRevocationPriv.publicKey, ripemd160(htlc.paymentHash), htlc.cltvExpiry, DefaultCommitmentFormat)
|
||||
val pubKeyScript = write(pay2wsh(redeemScript))
|
||||
val commitTx = Transaction(version = 2, txIn = Nil, txOut = TxOut(htlc.amountMsat.truncateToSatoshi, pubKeyScript) :: Nil, lockTime = 0)
|
||||
@ -168,7 +168,7 @@ class TransactionsSpec extends AnyFunSuite with Logging {
|
||||
// ClaimHtlcSuccessTx
|
||||
// first we create a fake commitTx tx, containing only the output that will be spent by the ClaimHtlcSuccessTx
|
||||
val paymentPreimage = randomBytes32()
|
||||
val htlc = UpdateAddHtlc(ByteVector32.Zeroes, 0, (20000 * 1000) msat, sha256(paymentPreimage), CltvExpiryDelta(144).toCltvExpiry(blockHeight), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val htlc = UpdateAddHtlc(ByteVector32.Zeroes, 0, (20000 * 1000) msat, sha256(paymentPreimage), CltvExpiryDelta(144).toCltvExpiry(blockHeight), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
val spec = CommitmentSpec(Set(OutgoingHtlc(htlc)), feeratePerKw, toLocal = 0 msat, toRemote = 0 msat)
|
||||
val outputs = makeCommitTxOutputs(localPaysCommitTxFees = true, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, remotePaymentPriv.publicKey, localHtlcPriv.publicKey, remoteHtlcPriv.publicKey, localFundingPriv.publicKey, remoteFundingPriv.publicKey, spec, DefaultCommitmentFormat)
|
||||
val pubKeyScript = write(pay2wsh(htlcOffered(localHtlcPriv.publicKey, remoteHtlcPriv.publicKey, localRevocationPriv.publicKey, ripemd160(htlc.paymentHash), DefaultCommitmentFormat)))
|
||||
@ -183,7 +183,7 @@ class TransactionsSpec extends AnyFunSuite with Logging {
|
||||
// ClaimHtlcTimeoutTx
|
||||
// first we create a fake commitTx tx, containing only the output that will be spent by the ClaimHtlcTimeoutTx
|
||||
val paymentPreimage = randomBytes32()
|
||||
val htlc = UpdateAddHtlc(ByteVector32.Zeroes, 0, (20000 * 1000) msat, sha256(paymentPreimage), toLocalDelay.toCltvExpiry(blockHeight), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val htlc = UpdateAddHtlc(ByteVector32.Zeroes, 0, (20000 * 1000) msat, sha256(paymentPreimage), toLocalDelay.toCltvExpiry(blockHeight), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
val spec = CommitmentSpec(Set(IncomingHtlc(htlc)), feeratePerKw, toLocal = 0 msat, toRemote = 0 msat)
|
||||
val outputs = makeCommitTxOutputs(localPaysCommitTxFees = true, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, remotePaymentPriv.publicKey, localHtlcPriv.publicKey, remoteHtlcPriv.publicKey, localFundingPriv.publicKey, remoteFundingPriv.publicKey, spec, DefaultCommitmentFormat)
|
||||
val pubKeyScript = write(pay2wsh(htlcReceived(localHtlcPriv.publicKey, remoteHtlcPriv.publicKey, localRevocationPriv.publicKey, ripemd160(htlc.paymentHash), htlc.cltvExpiry, DefaultCommitmentFormat)))
|
||||
@ -261,17 +261,17 @@ class TransactionsSpec extends AnyFunSuite with Logging {
|
||||
|
||||
// htlc1 and htlc2 are regular IN/OUT htlcs
|
||||
val paymentPreimage1 = randomBytes32()
|
||||
val htlc1 = UpdateAddHtlc(ByteVector32.Zeroes, 0, MilliBtc(100).toMilliSatoshi, sha256(paymentPreimage1), CltvExpiry(300), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val htlc1 = UpdateAddHtlc(ByteVector32.Zeroes, 0, MilliBtc(100).toMilliSatoshi, sha256(paymentPreimage1), CltvExpiry(300), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
val paymentPreimage2 = randomBytes32()
|
||||
val htlc2 = UpdateAddHtlc(ByteVector32.Zeroes, 1, MilliBtc(200).toMilliSatoshi, sha256(paymentPreimage2), CltvExpiry(310), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val htlc2 = UpdateAddHtlc(ByteVector32.Zeroes, 1, MilliBtc(200).toMilliSatoshi, sha256(paymentPreimage2), CltvExpiry(310), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
// htlc3 and htlc4 are dust IN/OUT htlcs, with an amount large enough to be included in the commit tx, but too small to be claimed at 2nd stage
|
||||
val paymentPreimage3 = randomBytes32()
|
||||
val htlc3 = UpdateAddHtlc(ByteVector32.Zeroes, 2, (localDustLimit + weight2fee(feeratePerKw, DefaultCommitmentFormat.htlcTimeoutWeight)).toMilliSatoshi, sha256(paymentPreimage3), CltvExpiry(295), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val htlc3 = UpdateAddHtlc(ByteVector32.Zeroes, 2, (localDustLimit + weight2fee(feeratePerKw, DefaultCommitmentFormat.htlcTimeoutWeight)).toMilliSatoshi, sha256(paymentPreimage3), CltvExpiry(295), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
val paymentPreimage4 = randomBytes32()
|
||||
val htlc4 = UpdateAddHtlc(ByteVector32.Zeroes, 3, (localDustLimit + weight2fee(feeratePerKw, DefaultCommitmentFormat.htlcSuccessWeight)).toMilliSatoshi, sha256(paymentPreimage4), CltvExpiry(300), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val htlc4 = UpdateAddHtlc(ByteVector32.Zeroes, 3, (localDustLimit + weight2fee(feeratePerKw, DefaultCommitmentFormat.htlcSuccessWeight)).toMilliSatoshi, sha256(paymentPreimage4), CltvExpiry(300), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
// htlc5 and htlc6 are dust IN/OUT htlcs
|
||||
val htlc5 = UpdateAddHtlc(ByteVector32.Zeroes, 4, (localDustLimit * 0.9).toMilliSatoshi, sha256(randomBytes32()), CltvExpiry(295), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val htlc6 = UpdateAddHtlc(ByteVector32.Zeroes, 5, (localDustLimit * 0.9).toMilliSatoshi, sha256(randomBytes32()), CltvExpiry(305), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val htlc5 = UpdateAddHtlc(ByteVector32.Zeroes, 4, (localDustLimit * 0.9).toMilliSatoshi, sha256(randomBytes32()), CltvExpiry(295), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
val htlc6 = UpdateAddHtlc(ByteVector32.Zeroes, 5, (localDustLimit * 0.9).toMilliSatoshi, sha256(randomBytes32()), CltvExpiry(305), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
val spec = CommitmentSpec(
|
||||
htlcs = Set(
|
||||
OutgoingHtlc(htlc1),
|
||||
@ -492,21 +492,21 @@ class TransactionsSpec extends AnyFunSuite with Logging {
|
||||
|
||||
// htlc1, htlc2a and htlc2b are regular IN/OUT htlcs
|
||||
val paymentPreimage1 = randomBytes32()
|
||||
val htlc1 = UpdateAddHtlc(ByteVector32.Zeroes, 0, MilliBtc(100).toMilliSatoshi, sha256(paymentPreimage1), CltvExpiry(300), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val htlc1 = UpdateAddHtlc(ByteVector32.Zeroes, 0, MilliBtc(100).toMilliSatoshi, sha256(paymentPreimage1), CltvExpiry(300), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
val paymentPreimage2 = randomBytes32()
|
||||
val htlc2a = UpdateAddHtlc(ByteVector32.Zeroes, 1, MilliBtc(50).toMilliSatoshi, sha256(paymentPreimage2), CltvExpiry(310), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val htlc2b = UpdateAddHtlc(ByteVector32.Zeroes, 2, MilliBtc(150).toMilliSatoshi, sha256(paymentPreimage2), CltvExpiry(310), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val htlc2a = UpdateAddHtlc(ByteVector32.Zeroes, 1, MilliBtc(50).toMilliSatoshi, sha256(paymentPreimage2), CltvExpiry(310), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
val htlc2b = UpdateAddHtlc(ByteVector32.Zeroes, 2, MilliBtc(150).toMilliSatoshi, sha256(paymentPreimage2), CltvExpiry(310), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
// htlc3 and htlc4 are dust IN/OUT htlcs, with an amount large enough to be included in the commit tx, but too small to be claimed at 2nd stage
|
||||
val paymentPreimage3 = randomBytes32()
|
||||
val htlc3 = UpdateAddHtlc(ByteVector32.Zeroes, 3, (localDustLimit + weight2fee(feeratePerKw, UnsafeLegacyAnchorOutputsCommitmentFormat.htlcTimeoutWeight)).toMilliSatoshi, sha256(paymentPreimage3), CltvExpiry(295), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val htlc3 = UpdateAddHtlc(ByteVector32.Zeroes, 3, (localDustLimit + weight2fee(feeratePerKw, UnsafeLegacyAnchorOutputsCommitmentFormat.htlcTimeoutWeight)).toMilliSatoshi, sha256(paymentPreimage3), CltvExpiry(295), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
val paymentPreimage4 = randomBytes32()
|
||||
val htlc4 = UpdateAddHtlc(ByteVector32.Zeroes, 4, (localDustLimit + weight2fee(feeratePerKw, UnsafeLegacyAnchorOutputsCommitmentFormat.htlcSuccessWeight)).toMilliSatoshi, sha256(paymentPreimage4), CltvExpiry(300), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val htlc4 = UpdateAddHtlc(ByteVector32.Zeroes, 4, (localDustLimit + weight2fee(feeratePerKw, UnsafeLegacyAnchorOutputsCommitmentFormat.htlcSuccessWeight)).toMilliSatoshi, sha256(paymentPreimage4), CltvExpiry(300), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
// htlc5 and htlc6 are dust IN/OUT htlcs
|
||||
val htlc5 = UpdateAddHtlc(ByteVector32.Zeroes, 5, (localDustLimit * 0.9).toMilliSatoshi, sha256(randomBytes32()), CltvExpiry(295), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val htlc6 = UpdateAddHtlc(ByteVector32.Zeroes, 6, (localDustLimit * 0.9).toMilliSatoshi, sha256(randomBytes32()), CltvExpiry(305), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val htlc5 = UpdateAddHtlc(ByteVector32.Zeroes, 5, (localDustLimit * 0.9).toMilliSatoshi, sha256(randomBytes32()), CltvExpiry(295), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
val htlc6 = UpdateAddHtlc(ByteVector32.Zeroes, 6, (localDustLimit * 0.9).toMilliSatoshi, sha256(randomBytes32()), CltvExpiry(305), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
// htlc7 and htlc8 are at the dust limit when we ignore 2nd-stage tx fees
|
||||
val htlc7 = UpdateAddHtlc(ByteVector32.Zeroes, 7, localDustLimit.toMilliSatoshi, sha256(randomBytes32()), CltvExpiry(300), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val htlc8 = UpdateAddHtlc(ByteVector32.Zeroes, 8, localDustLimit.toMilliSatoshi, sha256(randomBytes32()), CltvExpiry(302), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val htlc7 = UpdateAddHtlc(ByteVector32.Zeroes, 7, localDustLimit.toMilliSatoshi, sha256(randomBytes32()), CltvExpiry(300), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
val htlc8 = UpdateAddHtlc(ByteVector32.Zeroes, 8, localDustLimit.toMilliSatoshi, sha256(randomBytes32()), CltvExpiry(302), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
val spec = CommitmentSpec(
|
||||
htlcs = Set(
|
||||
OutgoingHtlc(htlc1),
|
||||
@ -760,11 +760,11 @@ class TransactionsSpec extends AnyFunSuite with Logging {
|
||||
val paymentPreimage1 = ByteVector32(hex"1111111111111111111111111111111111111111111111111111111111111111")
|
||||
val paymentPreimage2 = ByteVector32(hex"2222222222222222222222222222222222222222222222222222222222222222")
|
||||
val paymentPreimage3 = ByteVector32(hex"3333333333333333333333333333333333333333333333333333333333333333")
|
||||
val htlc1 = UpdateAddHtlc(randomBytes32(), 1, millibtc2satoshi(MilliBtc(100)).toMilliSatoshi, sha256(paymentPreimage1), CltvExpiry(300), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val htlc2 = UpdateAddHtlc(randomBytes32(), 2, millibtc2satoshi(MilliBtc(200)).toMilliSatoshi, sha256(paymentPreimage2), CltvExpiry(300), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val htlc3 = UpdateAddHtlc(randomBytes32(), 3, millibtc2satoshi(MilliBtc(200)).toMilliSatoshi, sha256(paymentPreimage3), CltvExpiry(300), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val htlc4 = UpdateAddHtlc(randomBytes32(), 4, millibtc2satoshi(MilliBtc(200)).toMilliSatoshi, sha256(paymentPreimage3), CltvExpiry(300), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val htlc5 = UpdateAddHtlc(randomBytes32(), 5, millibtc2satoshi(MilliBtc(200)).toMilliSatoshi, sha256(paymentPreimage3), CltvExpiry(301), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val htlc1 = UpdateAddHtlc(randomBytes32(), 1, millibtc2satoshi(MilliBtc(100)).toMilliSatoshi, sha256(paymentPreimage1), CltvExpiry(300), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
val htlc2 = UpdateAddHtlc(randomBytes32(), 2, millibtc2satoshi(MilliBtc(200)).toMilliSatoshi, sha256(paymentPreimage2), CltvExpiry(300), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
val htlc3 = UpdateAddHtlc(randomBytes32(), 3, millibtc2satoshi(MilliBtc(200)).toMilliSatoshi, sha256(paymentPreimage3), CltvExpiry(300), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
val htlc4 = UpdateAddHtlc(randomBytes32(), 4, millibtc2satoshi(MilliBtc(200)).toMilliSatoshi, sha256(paymentPreimage3), CltvExpiry(300), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
val htlc5 = UpdateAddHtlc(randomBytes32(), 5, millibtc2satoshi(MilliBtc(200)).toMilliSatoshi, sha256(paymentPreimage3), CltvExpiry(301), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
|
||||
val spec = CommitmentSpec(
|
||||
htlcs = Set(
|
||||
@ -877,9 +877,9 @@ class TransactionsSpec extends AnyFunSuite with Logging {
|
||||
formatted
|
||||
}
|
||||
|
||||
def htlcIn(amount: Satoshi): DirectedHtlc = IncomingHtlc(UpdateAddHtlc(ByteVector32.Zeroes, 0, amount.toMilliSatoshi, ByteVector32.Zeroes, CltvExpiry(144), TestConstants.emptyOnionPacket, None, 1.0))
|
||||
def htlcIn(amount: Satoshi): DirectedHtlc = IncomingHtlc(UpdateAddHtlc(ByteVector32.Zeroes, 0, amount.toMilliSatoshi, ByteVector32.Zeroes, CltvExpiry(144), TestConstants.emptyOnionPacket, None, 1.0, None))
|
||||
|
||||
def htlcOut(amount: Satoshi): DirectedHtlc = OutgoingHtlc(UpdateAddHtlc(ByteVector32.Zeroes, 0, amount.toMilliSatoshi, ByteVector32.Zeroes, CltvExpiry(144), TestConstants.emptyOnionPacket, None, 1.0))
|
||||
def htlcOut(amount: Satoshi): DirectedHtlc = OutgoingHtlc(UpdateAddHtlc(ByteVector32.Zeroes, 0, amount.toMilliSatoshi, ByteVector32.Zeroes, CltvExpiry(144), TestConstants.emptyOnionPacket, None, 1.0, None))
|
||||
|
||||
case class TestVector(name: String, spec: CommitmentSpec, expectedFee: Satoshi)
|
||||
|
||||
|
@ -291,11 +291,11 @@ object ChannelCodecsSpec {
|
||||
)
|
||||
|
||||
val htlcs: Seq[DirectedHtlc] = Seq[DirectedHtlc](
|
||||
IncomingHtlc(UpdateAddHtlc(ByteVector32.Zeroes, 0, 1000000 msat, Crypto.sha256(paymentPreimages(0)), CltvExpiry(500), TestConstants.emptyOnionPacket, None, 1.0)),
|
||||
IncomingHtlc(UpdateAddHtlc(ByteVector32.Zeroes, 1, 2000000 msat, Crypto.sha256(paymentPreimages(1)), CltvExpiry(501), TestConstants.emptyOnionPacket, None, 1.0)),
|
||||
OutgoingHtlc(UpdateAddHtlc(ByteVector32.Zeroes, 30, 2000000 msat, Crypto.sha256(paymentPreimages(2)), CltvExpiry(502), TestConstants.emptyOnionPacket, None, 1.0)),
|
||||
OutgoingHtlc(UpdateAddHtlc(ByteVector32.Zeroes, 31, 3000000 msat, Crypto.sha256(paymentPreimages(3)), CltvExpiry(503), TestConstants.emptyOnionPacket, None, 1.0)),
|
||||
IncomingHtlc(UpdateAddHtlc(ByteVector32.Zeroes, 2, 4000000 msat, Crypto.sha256(paymentPreimages(4)), CltvExpiry(504), TestConstants.emptyOnionPacket, None, 1.0))
|
||||
IncomingHtlc(UpdateAddHtlc(ByteVector32.Zeroes, 0, 1000000 msat, Crypto.sha256(paymentPreimages(0)), CltvExpiry(500), TestConstants.emptyOnionPacket, None, 1.0, None)),
|
||||
IncomingHtlc(UpdateAddHtlc(ByteVector32.Zeroes, 1, 2000000 msat, Crypto.sha256(paymentPreimages(1)), CltvExpiry(501), TestConstants.emptyOnionPacket, None, 1.0, None)),
|
||||
OutgoingHtlc(UpdateAddHtlc(ByteVector32.Zeroes, 30, 2000000 msat, Crypto.sha256(paymentPreimages(2)), CltvExpiry(502), TestConstants.emptyOnionPacket, None, 1.0, None)),
|
||||
OutgoingHtlc(UpdateAddHtlc(ByteVector32.Zeroes, 31, 3000000 msat, Crypto.sha256(paymentPreimages(3)), CltvExpiry(503), TestConstants.emptyOnionPacket, None, 1.0, None)),
|
||||
IncomingHtlc(UpdateAddHtlc(ByteVector32.Zeroes, 2, 4000000 msat, Crypto.sha256(paymentPreimages(4)), CltvExpiry(504), TestConstants.emptyOnionPacket, None, 1.0, None))
|
||||
)
|
||||
|
||||
val normal: DATA_NORMAL = {
|
||||
|
@ -58,6 +58,7 @@ class ChannelCodecs0Spec extends AnyFunSuite {
|
||||
hex"1e3a0e7c55b9f74b209e3704695e38874dd0950c54089a2be675bbf1a83679f6ff",
|
||||
hex"2db02ba44906455d5ba19cf75979889cc23ecd86b88728478133ccf180ee407abf882bd86f6f318d8e993dda100dcd9641e56c270aaee5810d9dcc0c85677387232c4f90ef4c54afb9ed9c0077db8a7516f41af552364609df16a25fc3534087c821af9a6013dbff96ba980b9f6a8b59da95fd5177c4d8bfe924c05dbf3a9d6e62a83cc1c91fa35cbc676094c2868df62dc1399b3797120ffd3f850eeafd525014068c4533d2a144e983b8a7f75d18843ccfafbc6ae13db41e2379a82f81e42accfd171995c206ba05024295e4b7ed202056301cba96ae647a0556b9dcba6cd368600a73a05dfeaa6287e10b9ae1ca8516f5e64c483154ecc9aad87fa5380145f114d4bdd2d0be8b6c30588ba925642e16125c96b12248f79ffd04c4770cc6f7d85239943b8e53eae8fb085d9be5f849d5f2b8a4597d7d35083cf9ff06fce2b6d13e166cd725450a10eac6d2c574850c756dbe25dd2715b4dcd5cf2bf169f7d035bd48f19553133fda1ad99bef442e76d365a7fe372790581189b4c7684da5f2922421332fd1dcb0618bffc76c192e8715c2a43452adce7534c1e2db8274bccacb209c097ecd9db47b6d27f8f418d5a9efb9196fe3dea8a4f822b136f86b9cb641cfe47415620f480df4e8e86bfe1012d4ea675156feba6c61ab841922c2203f2458ec0f292fc01c794c579c06765760cbd42e678a16b40c925a568ce2b024007e5350ea96bb82c92105127cf7cecaa18b1b31d02aadc9bbe414489e6c77847cead92a44866ba1dd99a7b40d522c3898e55c7b275fd205500dd5ba42cfa2b8099e6051f8c3a10877a4e1fae05458b5f11356b78f3452908f229f1ba81353732152c72fb208d870b953021f6f8f658c29238ce4c84af4c037cffd188f50b36ad5e8395e0d7cfd439b6a80e33f87784c06ceb6f3fa6d4de52220876f1b53e30da5403e2413a1b22a11d1d7d99c13fae5047a182cca85eda0b3f50e4bb3ae3344a64513911ef45234d77c03eb63f07984465ae2defbf8d4207f70c6faeb35572735544f19ffc094c9e8284ac82261004ed7dcfa7d8c5c7f10f071c7043e1bae2666f2e5bf3282c1db853997372c6188353d8f932997de4a034c21c386d09cd2fbe461fadede20a4d9288dd060f43f6fc93119beff9154659141240c227b048f555c85c728581ffa523acf06fa591046390b104cceb05d943a4acc26989dc2603bd1c2c6fe128cf68e7747c6a73249100917a6964db98dede8e8ea36f58b775a8d18c9db455d57fcd5242a149f556284453af2698944884e8830a1a1bd5cbb700560528086be739d550bc5a7a44d2a21103564d24d862ce90f54271a71739eca1eb3e154170852e8f24e3840139bcc3cb8b184d7f142b5750d0d35f07283d8292e5b276d5c94dd9ecb084702a14c290fad7a729b681421ae21fa5a0cb0a1ca5d4ca6d4e9b1128e890443839c927fd97e40e1331c09aa4c726a9118526be5a75fda9a1f8e8e5807cca5f251cd431ef0052087e433eca5b325c208a5229352f1cb8cc180103fcd42f3b7cc5b96b2fe769c92f8c093604abf1e60dc963192f86739304e157f0d49d635f27629b101ddb440776774b6dc6227a1c007f1cabe7a88d7a9b9b4d0d66af9415be3fc4a720ecf481fe10d524b1a09833608e8911555f58643e10fa57a1b81c0ad5b3eb6b5f4be7b05787e31667bd2088a52c6ffda0b0f3ed7a881e66380c011ba7185f7045845f88403d2ff3df3f86a300f808bbd9c28fa33fa034d0c098796d6bc1b6369a3d7c70ece00420cfb2840df7b93da67583e93b0ff2c396ba89e9100bcabf0c6f947bc9d93bb2d3289",
|
||||
ByteVector32(hex"dac3bc8b2e16fdf2db3e627483bc395c701c1fc96ace53e4ebc54150e807921d")),
|
||||
TlvStream.empty
|
||||
)
|
||||
val remaining = bin"0000000" // 7 bits remainder because the direction is encoded with 1 bit and we are dealing with bytes
|
||||
|
||||
|
@ -76,7 +76,8 @@ class ChannelCodecs1Spec extends AnyFunSuite {
|
||||
paymentHash = randomBytes32(),
|
||||
onionRoutingPacket = TestConstants.emptyOnionPacket,
|
||||
blinding_opt = None,
|
||||
confidence = 1.0)
|
||||
confidence = 1.0,
|
||||
fundingFee_opt = None)
|
||||
val htlc1 = IncomingHtlc(add)
|
||||
val htlc2 = OutgoingHtlc(add)
|
||||
assert(htlcCodec.decodeValue(htlcCodec.encode(htlc1).require).require == htlc1)
|
||||
@ -92,7 +93,8 @@ class ChannelCodecs1Spec extends AnyFunSuite {
|
||||
paymentHash = randomBytes32(),
|
||||
onionRoutingPacket = TestConstants.emptyOnionPacket,
|
||||
blinding_opt = None,
|
||||
confidence = 1.0)
|
||||
confidence = 1.0,
|
||||
fundingFee_opt = None)
|
||||
val add2 = UpdateAddHtlc(
|
||||
channelId = randomBytes32(),
|
||||
id = Random.nextInt(Int.MaxValue),
|
||||
@ -101,7 +103,8 @@ class ChannelCodecs1Spec extends AnyFunSuite {
|
||||
paymentHash = randomBytes32(),
|
||||
onionRoutingPacket = TestConstants.emptyOnionPacket,
|
||||
blinding_opt = None,
|
||||
confidence = 1.0)
|
||||
confidence = 1.0,
|
||||
fundingFee_opt = None)
|
||||
val htlc1 = IncomingHtlc(add1)
|
||||
val htlc2 = OutgoingHtlc(add2)
|
||||
val htlcs = Set[DirectedHtlc](htlc1, htlc2)
|
||||
@ -125,18 +128,18 @@ class ChannelCodecs1Spec extends AnyFunSuite {
|
||||
assert(originCodec.decodeValue(originCodec.encode(localHot).require).require == localCold)
|
||||
assert(originCodec.decodeValue(originCodec.encode(localCold).require).require == localCold)
|
||||
|
||||
val add = UpdateAddHtlc(randomBytes32(), 4324, 11000000 msat, randomBytes32(), CltvExpiry(400000), TestConstants.emptyOnionPacket, None, 1.0)
|
||||
val add = UpdateAddHtlc(randomBytes32(), 4324, 11000000 msat, randomBytes32(), CltvExpiry(400000), TestConstants.emptyOnionPacket, None, 1.0, None)
|
||||
val relayedHot = Origin.Hot(replyTo, Upstream.Hot.Channel(add, TimestampMilli(0), randomKey().publicKey))
|
||||
val relayedCold = Origin.Cold(Upstream.Cold.Channel(add.channelId, add.id, add.amountMsat))
|
||||
assert(originCodec.decodeValue(originCodec.encode(relayedHot).require).require == relayedCold)
|
||||
assert(originCodec.decodeValue(originCodec.encode(relayedCold).require).require == relayedCold)
|
||||
|
||||
val adds = Seq(
|
||||
UpdateAddHtlc(randomBytes32(), 1L, 1000 msat, randomBytes32(), CltvExpiry(400000), TestConstants.emptyOnionPacket, None, 1.0),
|
||||
UpdateAddHtlc(randomBytes32(), 1L, 2000 msat, randomBytes32(), CltvExpiry(400000), TestConstants.emptyOnionPacket, None, 1.0),
|
||||
UpdateAddHtlc(randomBytes32(), 2L, 3000 msat, randomBytes32(), CltvExpiry(400000), TestConstants.emptyOnionPacket, None, 1.0),
|
||||
UpdateAddHtlc(randomBytes32(), 1L, 1000 msat, randomBytes32(), CltvExpiry(400000), TestConstants.emptyOnionPacket, None, 1.0, None),
|
||||
UpdateAddHtlc(randomBytes32(), 1L, 2000 msat, randomBytes32(), CltvExpiry(400000), TestConstants.emptyOnionPacket, None, 1.0, None),
|
||||
UpdateAddHtlc(randomBytes32(), 2L, 3000 msat, randomBytes32(), CltvExpiry(400000), TestConstants.emptyOnionPacket, None, 1.0, None),
|
||||
)
|
||||
val trampolineRelayedHot = Origin.Hot(replyTo, Upstream.Hot.Trampoline(adds.map(add => Upstream.Hot.Channel(add, TimestampMilli(0), randomKey().publicKey))))
|
||||
val trampolineRelayedHot = Origin.Hot(replyTo, Upstream.Hot.Trampoline(adds.map(add => Upstream.Hot.Channel(add, TimestampMilli(0), randomKey().publicKey)).toList))
|
||||
// We didn't encode the incoming HTLC amount.
|
||||
val trampolineRelayed = Origin.Cold(Upstream.Cold.Trampoline(adds.map(add => Upstream.Cold.Channel(add.channelId, add.id, 0 msat)).toList))
|
||||
assert(originCodec.decodeValue(originCodec.encode(trampolineRelayedHot).require).require == trampolineRelayed)
|
||||
|
@ -140,8 +140,10 @@ class CommonCodecsSpec extends AnyFunSuite {
|
||||
|
||||
test("encode/decode channel flags") {
|
||||
val testCases = Map(
|
||||
bin"00000000" -> ChannelFlags(announceChannel = false),
|
||||
bin"00000001" -> ChannelFlags(announceChannel = true),
|
||||
bin"00000000" -> ChannelFlags(nonInitiatorPaysCommitFees = false, announceChannel = false),
|
||||
bin"00000001" -> ChannelFlags(nonInitiatorPaysCommitFees = false, announceChannel = true),
|
||||
bin"00000010" -> ChannelFlags(nonInitiatorPaysCommitFees = true, announceChannel = false),
|
||||
bin"00000011" -> ChannelFlags(nonInitiatorPaysCommitFees = true, announceChannel = true),
|
||||
)
|
||||
testCases.foreach { case (bin, obj) =>
|
||||
assert(channelflags.decode(bin).require == DecodeResult(obj, BitVector.empty))
|
||||
@ -149,8 +151,9 @@ class CommonCodecsSpec extends AnyFunSuite {
|
||||
}
|
||||
|
||||
// BOLT 2: The receiving node MUST [...] ignore undefined bits in channel_flags.
|
||||
assert(channelflags.decode(bin"11111111").require == DecodeResult(ChannelFlags(announceChannel = true), BitVector.empty))
|
||||
assert(channelflags.decode(bin"11111110").require == DecodeResult(ChannelFlags(announceChannel = false), BitVector.empty))
|
||||
assert(channelflags.decode(bin"11111111").require == DecodeResult(ChannelFlags(nonInitiatorPaysCommitFees = true, announceChannel = true), BitVector.empty))
|
||||
assert(channelflags.decode(bin"11111110").require == DecodeResult(ChannelFlags(nonInitiatorPaysCommitFees = true, announceChannel = false), BitVector.empty))
|
||||
assert(channelflags.decode(bin"11111100").require == DecodeResult(ChannelFlags(nonInitiatorPaysCommitFees = false, announceChannel = false), BitVector.empty))
|
||||
}
|
||||
|
||||
test("encode/decode with rgb codec") {
|
||||
|
@ -55,43 +55,59 @@ class LightningMessageCodecsSpec extends AnyFunSuite {
|
||||
def publicKey(fill: Byte) = PrivateKey(ByteVector.fill(32)(fill)).publicKey
|
||||
|
||||
test("encode/decode init message") {
|
||||
case class TestCase(encoded: ByteVector, rawFeatures: ByteVector, networks: List[BlockHash], address: Option[IPAddress], valid: Boolean, reEncoded: Option[ByteVector] = None)
|
||||
case class TestCase(encoded: ByteVector, expected: Option[Init], reEncoded: Option[ByteVector] = None)
|
||||
val chainHash1 = BlockHash(ByteVector32(hex"0101010101010101010101010101010101010101010101010101010101010101"))
|
||||
val chainHash2 = BlockHash(ByteVector32(hex"0202020202020202020202020202020202020202020202020202020202020202"))
|
||||
val remoteAddress1 = IPv4(InetAddress.getByAddress(Array[Byte](140.toByte, 82.toByte, 121.toByte, 3.toByte)).asInstanceOf[Inet4Address], 9735)
|
||||
val remoteAddress2 = IPv6(InetAddress.getByAddress(hex"b643 8bb1 c1f9 0556 487c 0acb 2ba3 3cc2".toArray).asInstanceOf[Inet6Address], 9736)
|
||||
val fundingRates1 = LiquidityAds.WillFundRates(
|
||||
fundingRates = List(LiquidityAds.FundingRate(100_000 sat, 500_000 sat, 550, 100, 5_000 sat, 1_000 sat)),
|
||||
paymentTypes = Set(LiquidityAds.PaymentType.FromChannelBalance)
|
||||
)
|
||||
val fundingRates2 = LiquidityAds.WillFundRates(
|
||||
fundingRates = List(
|
||||
LiquidityAds.FundingRate(100_000 sat, 500_000 sat, 550, 100, 5_000 sat, 1_000 sat),
|
||||
LiquidityAds.FundingRate(500_000 sat, 5_000_000 sat, 1100, 75, 0 sat, 1_500 sat)
|
||||
),
|
||||
paymentTypes = Set(
|
||||
LiquidityAds.PaymentType.FromChannelBalance,
|
||||
LiquidityAds.PaymentType.FromFutureHtlc,
|
||||
LiquidityAds.PaymentType.FromFutureHtlcWithPreimage,
|
||||
LiquidityAds.PaymentType.FromChannelBalanceForFutureHtlc,
|
||||
LiquidityAds.PaymentType.Unknown(211)
|
||||
)
|
||||
)
|
||||
val testCases = Seq(
|
||||
TestCase(hex"0000 0000", hex"", Nil, None, valid = true), // no features
|
||||
TestCase(hex"0000 0002088a", hex"088a", Nil, None, valid = true), // no global features
|
||||
TestCase(hex"00020200 0000", hex"0200", Nil, None, valid = true, Some(hex"0000 00020200")), // no local features
|
||||
TestCase(hex"00020200 0002088a", hex"0a8a", Nil, None, valid = true, Some(hex"0000 00020a8a")), // local and global - no conflict - same size
|
||||
TestCase(hex"00020200 0003020002", hex"020202", Nil, None, valid = true, Some(hex"0000 0003020202")), // local and global - no conflict - different sizes
|
||||
TestCase(hex"00020a02 0002088a", hex"0a8a", Nil, None, valid = true, Some(hex"0000 00020a8a")), // local and global - conflict - same size
|
||||
TestCase(hex"00022200 000302aaa2", hex"02aaa2", Nil, None, valid = true, Some(hex"0000 000302aaa2")), // local and global - conflict - different sizes
|
||||
TestCase(hex"0000 0002088a 03012a05022aa2", hex"088a", Nil, None, valid = true), // unknown odd records
|
||||
TestCase(hex"0000 0002088a 03012a04022aa2", hex"088a", Nil, None, valid = false), // unknown even records
|
||||
TestCase(hex"0000 0002088a 0120010101010101010101010101010101010101010101010101010101010101", hex"088a", Nil, None, valid = false), // invalid tlv stream
|
||||
TestCase(hex"0000 0002088a 01200101010101010101010101010101010101010101010101010101010101010101", hex"088a", List(chainHash1), None, valid = true), // single network
|
||||
TestCase(hex"0000 0002088a 01200101010101010101010101010101010101010101010101010101010101010101 0307018c5279032607", hex"088a", List(chainHash1), Some(remoteAddress1), valid = true), // single network and IPv4 address
|
||||
TestCase(hex"0000 0002088a 01200101010101010101010101010101010101010101010101010101010101010101 031302b6438bb1c1f90556487c0acb2ba33cc22608", hex"088a", List(chainHash1), Some(remoteAddress2), valid = true), // single network and IPv6 address
|
||||
TestCase(hex"0000 0002088a 014001010101010101010101010101010101010101010101010101010101010101010202020202020202020202020202020202020202020202020202020202020202", hex"088a", List(chainHash1, chainHash2), None, valid = true), // multiple networks
|
||||
TestCase(hex"0000 0002088a 01200101010101010101010101010101010101010101010101010101010101010101 c9012a", hex"088a", List(chainHash1), None, valid = true), // network and unknown odd records
|
||||
TestCase(hex"0000 0002088a 01200101010101010101010101010101010101010101010101010101010101010101 02012a", hex"088a", Nil, None, valid = false), // network and unknown even records
|
||||
TestCase(hex"0000 0002088a fd053b190001000186a00007a1200226006400001388000003e8000101", hex"088a", Nil, None, valid = true), // one liquidity ads with the default payment type
|
||||
TestCase(hex"0000 0002088a fd053b470002000186a00007a1200226006400001388000003e80007a120004c4b40044c004b00000000000005dc001b080000000000000000000300000000000000000000000000000001", hex"088a", Nil, None, valid = true) // two liquidity ads with multiple payment types
|
||||
TestCase(hex"0000 0000", Some(Init(Features.empty))), // no features
|
||||
TestCase(hex"0000 0002088a", Some(Init(Features(hex"088a").initFeatures()))), // no global features
|
||||
TestCase(hex"00020200 0000", Some(Init(Features(hex"0200").initFeatures())), Some(hex"0000 00020200")), // no local features
|
||||
TestCase(hex"00020200 0002088a", Some(Init(Features(hex"0a8a").initFeatures())), Some(hex"0000 00020a8a")), // local and global - no conflict - same size
|
||||
TestCase(hex"00020200 0003020002", Some(Init(Features(hex"020202").initFeatures())), Some(hex"0000 0003020202")), // local and global - no conflict - different sizes
|
||||
TestCase(hex"00020a02 0002088a", Some(Init(Features(hex"0a8a").initFeatures())), Some(hex"0000 00020a8a")), // local and global - conflict - same size
|
||||
TestCase(hex"00022200 000302aaa2", Some(Init(Features(hex"02aaa2").initFeatures())), Some(hex"0000 000302aaa2")), // local and global - conflict - different sizes
|
||||
TestCase(hex"0000 0002088a 03012a05022aa2", Some(Init(Features(hex"088a").initFeatures(), TlvStream(Set.empty[InitTlv], Set(GenericTlv(UInt64(3), hex"2a"), GenericTlv(UInt64(5), hex"2aa2")))))), // unknown odd records
|
||||
TestCase(hex"0000 0002088a 03012a04022aa2", None), // unknown even records
|
||||
TestCase(hex"0000 0002088a 0120010101010101010101010101010101010101010101010101010101010101", None), // invalid tlv stream
|
||||
TestCase(hex"0000 0002088a 01200101010101010101010101010101010101010101010101010101010101010101", Some(Init(Features(hex"088a").initFeatures(), TlvStream(InitTlv.Networks(List(chainHash1)))))), // single network
|
||||
TestCase(hex"0000 0002088a 01200101010101010101010101010101010101010101010101010101010101010101 0307018c5279032607", Some(Init(Features(hex"088a").initFeatures(), TlvStream(InitTlv.Networks(List(chainHash1)), InitTlv.RemoteAddress(remoteAddress1))))), // single network and IPv4 address
|
||||
TestCase(hex"0000 0002088a 01200101010101010101010101010101010101010101010101010101010101010101 031302b6438bb1c1f90556487c0acb2ba33cc22608", Some(Init(Features(hex"088a").initFeatures(), TlvStream(InitTlv.Networks(List(chainHash1)), InitTlv.RemoteAddress(remoteAddress2))))), // single network and IPv6 address
|
||||
TestCase(hex"0000 0002088a 014001010101010101010101010101010101010101010101010101010101010101010202020202020202020202020202020202020202020202020202020202020202", Some(Init(Features(hex"088a").initFeatures(), TlvStream(InitTlv.Networks(List(chainHash1, chainHash2)))))), // multiple networks
|
||||
TestCase(hex"0000 0002088a 01200101010101010101010101010101010101010101010101010101010101010101 c9012a", Some(Init(Features(hex"088a").initFeatures(), TlvStream(Set[InitTlv](InitTlv.Networks(List(chainHash1))), Set(GenericTlv(UInt64(201), hex"2a")))))), // network and unknown odd records
|
||||
TestCase(hex"0000 0002088a 01200101010101010101010101010101010101010101010101010101010101010101 02012a", None), // network and unknown even records
|
||||
TestCase(hex"0000 0002088a fd053b190001000186a00007a1200226006400001388000003e8000101", Some(Init(Features(hex"088a").initFeatures(), TlvStream(InitTlv.OptionWillFund(fundingRates1))))), // one liquidity ads with the default payment type
|
||||
TestCase(hex"0000 0002088a fd053b470002000186a00007a1200226006400001388000003e80007a120004c4b40044c004b00000000000005dc001b080000000000000000000700000000000000000000000000000001", Some(Init(Features(hex"088a").initFeatures(), TlvStream(InitTlv.OptionWillFund(fundingRates2))))) // two liquidity ads with multiple payment types
|
||||
)
|
||||
|
||||
for (testCase <- testCases) {
|
||||
if (testCase.valid) {
|
||||
val init = initCodec.decode(testCase.encoded.bits).require.value
|
||||
assert(init.features.toByteVector == testCase.rawFeatures)
|
||||
assert(init.networks == testCase.networks)
|
||||
assert(init.remoteAddress_opt == testCase.address)
|
||||
val encoded = initCodec.encode(init).require
|
||||
assert(encoded.bytes == testCase.reEncoded.getOrElse(testCase.encoded))
|
||||
assert(initCodec.decode(encoded).require.value == init)
|
||||
} else {
|
||||
assert(initCodec.decode(testCase.encoded.bits).isFailure)
|
||||
testCase.expected match {
|
||||
case Some(expected) =>
|
||||
val init = initCodec.decode(testCase.encoded.bits).require.value
|
||||
assert(init == expected)
|
||||
val encoded = initCodec.encode(init).require
|
||||
assert(encoded.bytes == testCase.reEncoded.getOrElse(testCase.encoded))
|
||||
assert(initCodec.decode(encoded).require.value == init)
|
||||
case None =>
|
||||
assert(initCodec.decode(testCase.encoded.bits).isFailure)
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -313,9 +329,9 @@ class LightningMessageCodecsSpec extends AnyFunSuite {
|
||||
val defaultOpen = OpenDualFundedChannel(BlockHash(ByteVector32.Zeroes), ByteVector32.One, FeeratePerKw(5000 sat), FeeratePerKw(4000 sat), 250_000 sat, 500 sat, UInt64(50_000), 15 msat, CltvExpiryDelta(144), 483, 650_000, publicKey(1), publicKey(2), publicKey(3), publicKey(4), publicKey(5), publicKey(6), publicKey(7), ChannelFlags(true))
|
||||
val defaultEncodedWithoutFlags = hex"0040 0000000000000000000000000000000000000000000000000000000000000000 0100000000000000000000000000000000000000000000000000000000000000 00001388 00000fa0 000000000003d090 00000000000001f4 000000000000c350 000000000000000f 0090 01e3 0009eb10 031b84c5567b126440995d3ed5aaba0565d71e1834604819ff9c17f5e9d5dd078f 024d4b6cd1361032ca9bd2aeb9d900aa4d45d9ead80ac9423374c451a7254d0766 02531fe6068134503d2723133227c867ac8fa6c83c537e9a44c3c5bdbdcb1fe337 03462779ad4aad39514614751a71085f2f10e1c7a593e4e030efb5b8721ce55b0b 0362c0a046dacce86ddd0343c6d3c7c79c2208ba0d9c9cf24a6d046d21d21f90f7 03f006a18d5653c4edf5391ff23a61f03ff83d237e880ee61187fa9f379a028e0a 02989c0b76cb563971fdc9bef31ec06c3560f3249d6ee9e5d83c57625596e05f6f"
|
||||
val testCases = Seq(
|
||||
defaultEncodedWithoutFlags ++ hex"00" -> ChannelFlags(false),
|
||||
defaultEncodedWithoutFlags ++ hex"a2" -> ChannelFlags(false),
|
||||
defaultEncodedWithoutFlags ++ hex"ff" -> ChannelFlags(true),
|
||||
defaultEncodedWithoutFlags ++ hex"00" -> ChannelFlags(nonInitiatorPaysCommitFees = false, announceChannel = false),
|
||||
defaultEncodedWithoutFlags ++ hex"a2" -> ChannelFlags(nonInitiatorPaysCommitFees = true, announceChannel = false),
|
||||
defaultEncodedWithoutFlags ++ hex"ff" -> ChannelFlags(nonInitiatorPaysCommitFees = true, announceChannel = true),
|
||||
)
|
||||
testCases.foreach { case (bin, flags) =>
|
||||
val decoded = lightningMessageCodec.decode(bin.bits).require.value
|
||||
@ -442,14 +458,49 @@ class LightningMessageCodecsSpec extends AnyFunSuite {
|
||||
assert(lightningMessageCodec.encode(defaultAccept).require.bytes == defaultAcceptBin)
|
||||
val fundingScript = Script.write(Script.pay2wsh(Scripts.multiSig2of2(defaultOpen.fundingPubkey, defaultAccept.fundingPubkey)))
|
||||
|
||||
val Some(request) = LiquidityAds.requestFunding(750_000 sat, LiquidityAds.PaymentDetails.FromChannelBalance, willFundRates)
|
||||
val open = defaultOpen.copy(tlvStream = TlvStream(ChannelTlv.RequestFundingTlv(request)))
|
||||
val openBin = hex"fd053b 1e 00000000000b71b0 0007a120004c4b40044c004b00000000000005dc 0000"
|
||||
assert(lightningMessageCodec.encode(open).require.bytes == defaultOpenBin ++ openBin)
|
||||
val Right(willFund) = willFundRates.validateRequest(nodeKey, randomBytes32(), fundingScript, defaultOpen.fundingFeerate, request, isChannelCreation = true).map(_.willFund)
|
||||
val accept = defaultAccept.copy(tlvStream = TlvStream(ChannelTlv.ProvideFundingTlv(willFund)))
|
||||
val acceptBin = hex"fd053b 78 0007a120004c4b40044c004b00000000000005dc 002200202ec38203f4cf37a3b377d9a55c7ae0153c643046dbdbe2ffccfb11b74420103c c57cf393f6bd534472ec08cbfbbc7268501b32f563a21cdf02a99127c4f25168249acd6509f96b2e93843c3b838ee4808c75d0a15ff71ba886fda980b8ca954f"
|
||||
assert(lightningMessageCodec.encode(accept).require.bytes == defaultAcceptBin ++ acceptBin)
|
||||
{
|
||||
// Request funds from channel balance.
|
||||
val Some(request) = LiquidityAds.requestFunding(750_000 sat, LiquidityAds.PaymentDetails.FromChannelBalance, willFundRates)
|
||||
val open = defaultOpen.copy(tlvStream = TlvStream(ChannelTlv.RequestFundingTlv(request)))
|
||||
val openBin = hex"fd053b 1e 00000000000b71b0 0007a120004c4b40044c004b00000000000005dc 0000"
|
||||
assert(lightningMessageCodec.encode(open).require.bytes == defaultOpenBin ++ openBin)
|
||||
val Right(willFund) = willFundRates.validateRequest(nodeKey, randomBytes32(), fundingScript, defaultOpen.fundingFeerate, request, isChannelCreation = true).map(_.willFund)
|
||||
val accept = defaultAccept.copy(tlvStream = TlvStream(ChannelTlv.ProvideFundingTlv(willFund)))
|
||||
val acceptBin = hex"fd053b 78 0007a120004c4b40044c004b00000000000005dc 002200202ec38203f4cf37a3b377d9a55c7ae0153c643046dbdbe2ffccfb11b74420103c c57cf393f6bd534472ec08cbfbbc7268501b32f563a21cdf02a99127c4f25168249acd6509f96b2e93843c3b838ee4808c75d0a15ff71ba886fda980b8ca954f"
|
||||
assert(lightningMessageCodec.encode(accept).require.bytes == defaultAcceptBin ++ acceptBin)
|
||||
}
|
||||
{
|
||||
// Request funds from future HTLCs.
|
||||
val paymentHashes = List(
|
||||
ByteVector32.fromValidHex("80417c0c91deb72606958425ea1552a045a55a250e91870231b486dcb2106734"),
|
||||
ByteVector32.fromValidHex("d662b36d54c6d1c2a0227cdc114d12c578c25ab6ec664eebaa440d7e493eba47"),
|
||||
)
|
||||
val willFundRates1 = willFundRates.copy(paymentTypes = Set(LiquidityAds.PaymentType.FromFutureHtlc))
|
||||
val Some(request) = LiquidityAds.requestFunding(500_000 sat, LiquidityAds.PaymentDetails.FromFutureHtlc(paymentHashes), willFundRates1)
|
||||
val open = defaultOpen.copy(tlvStream = TlvStream(ChannelTlv.RequestFundingTlv(request)))
|
||||
val openBin = hex"fd053b 5e 000000000007a120 000186a00007a1200226006400001388000003e8 804080417c0c91deb72606958425ea1552a045a55a250e91870231b486dcb2106734d662b36d54c6d1c2a0227cdc114d12c578c25ab6ec664eebaa440d7e493eba47"
|
||||
assert(lightningMessageCodec.encode(open).require.bytes == defaultOpenBin ++ openBin)
|
||||
val Right(willFund) = willFundRates1.validateRequest(nodeKey, randomBytes32(), fundingScript, defaultOpen.fundingFeerate, request, isChannelCreation = true).map(_.willFund)
|
||||
val accept = defaultAccept.copy(tlvStream = TlvStream(ChannelTlv.ProvideFundingTlv(willFund)))
|
||||
val acceptBin = hex"fd053b 78 000186a00007a1200226006400001388000003e8 002200202ec38203f4cf37a3b377d9a55c7ae0153c643046dbdbe2ffccfb11b74420103c 035875ad2279190f6bfcc75a8bdccafeddfc2700a03587e3621114bf43b60d2c0de977ba0337b163d320471720a683ae211bea07742a2c4204dd5eb0bda75135"
|
||||
assert(lightningMessageCodec.encode(accept).require.bytes == defaultAcceptBin ++ acceptBin)
|
||||
}
|
||||
{
|
||||
// Request funds from channel balance for future HTLCs.
|
||||
val paymentHashes = List(
|
||||
ByteVector32.fromValidHex("80417c0c91deb72606958425ea1552a045a55a250e91870231b486dcb2106734"),
|
||||
ByteVector32.fromValidHex("d662b36d54c6d1c2a0227cdc114d12c578c25ab6ec664eebaa440d7e493eba47"),
|
||||
)
|
||||
val willFundRates1 = willFundRates.copy(paymentTypes = Set(LiquidityAds.PaymentType.FromChannelBalanceForFutureHtlc))
|
||||
val Some(request) = LiquidityAds.requestFunding(500_000 sat, LiquidityAds.PaymentDetails.FromChannelBalanceForFutureHtlc(paymentHashes), willFundRates1)
|
||||
val open = defaultOpen.copy(tlvStream = TlvStream(ChannelTlv.RequestFundingTlv(request)))
|
||||
val openBin = hex"fd053b 5e 000000000007a120 000186a00007a1200226006400001388000003e8 824080417c0c91deb72606958425ea1552a045a55a250e91870231b486dcb2106734d662b36d54c6d1c2a0227cdc114d12c578c25ab6ec664eebaa440d7e493eba47"
|
||||
assert(lightningMessageCodec.encode(open).require.bytes == defaultOpenBin ++ openBin)
|
||||
val Right(willFund) = willFundRates1.validateRequest(nodeKey, randomBytes32(), fundingScript, defaultOpen.fundingFeerate, request, isChannelCreation = true).map(_.willFund)
|
||||
val accept = defaultAccept.copy(tlvStream = TlvStream(ChannelTlv.ProvideFundingTlv(willFund)))
|
||||
val acceptBin = hex"fd053b 78 000186a00007a1200226006400001388000003e8 002200202ec38203f4cf37a3b377d9a55c7ae0153c643046dbdbe2ffccfb11b74420103c 035875ad2279190f6bfcc75a8bdccafeddfc2700a03587e3621114bf43b60d2c0de977ba0337b163d320471720a683ae211bea07742a2c4204dd5eb0bda75135"
|
||||
assert(lightningMessageCodec.encode(accept).require.bytes == defaultAcceptBin ++ acceptBin)
|
||||
}
|
||||
}
|
||||
|
||||
test("decode unknown liquidity ads payment types") {
|
||||
@ -476,7 +527,7 @@ class LightningMessageCodecsSpec extends AnyFunSuite {
|
||||
UpdateFee(randomBytes32(), FeeratePerKw(2 sat)),
|
||||
Shutdown(randomBytes32(), bin(47, 0)),
|
||||
ClosingSigned(randomBytes32(), 2 sat, randomBytes64()),
|
||||
UpdateAddHtlc(randomBytes32(), 2, 3 msat, bin32(0), CltvExpiry(4), TestConstants.emptyOnionPacket, None, 1.0),
|
||||
UpdateAddHtlc(randomBytes32(), 2, 3 msat, bin32(0), CltvExpiry(4), TestConstants.emptyOnionPacket, None, 1.0, None),
|
||||
UpdateFulfillHtlc(randomBytes32(), 2, bin32(0)),
|
||||
UpdateFailHtlc(randomBytes32(), 2, bin(154, 0)),
|
||||
UpdateFailMalformedHtlc(randomBytes32(), 2, randomBytes32(), 1111),
|
||||
@ -532,6 +583,31 @@ class LightningMessageCodecsSpec extends AnyFunSuite {
|
||||
}
|
||||
}
|
||||
|
||||
test("encode/decode on-the-fly funding messages") {
|
||||
val channelId = ByteVector32(hex"c11b8fbd682b3c6ee11f9d7268e22bb5887cd4d3bf3338bfcc340583f685733c")
|
||||
val paymentId = ByteVector32(hex"3118a7954088c27b19923894ed27923c297f88ec3734f90b2b4aafcb11238503")
|
||||
val blinding = PublicKey(hex"0296d5c32655a5eaa8be086479d7bcff967b6e9ca8319b69565747ae16ff20fad6")
|
||||
val paymentHash1 = ByteVector32(hex"80417c0c91deb72606958425ea1552a045a55a250e91870231b486dcb2106734")
|
||||
val paymentHash2 = ByteVector32(hex"3213a810a0bfc54566d9be09da1484538b5d19229e928dfa8b692966a8df6785")
|
||||
val fundingFee = LiquidityAds.FundingFee(5_000_100 msat, TxId(TxHash(ByteVector32(hex"24e1b2c94c4e734dd5b9c5f3c910fbb6b3b436ced6382c7186056a5a23f14566"))))
|
||||
val testCases = Seq(
|
||||
UpdateAddHtlc(channelId, 7, 75_000_000 msat, paymentHash1, CltvExpiry(840_000), TestConstants.emptyOnionPacket, blinding_opt = None, confidence = 0, fundingFee_opt = Some(fundingFee)) -> hex"0080 c11b8fbd682b3c6ee11f9d7268e22bb5887cd4d3bf3338bfcc340583f685733c 0000000000000007 00000000047868c0 80417c0c91deb72606958425ea1552a045a55a250e91870231b486dcb2106734 000cd140 00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000 fda0512800000000004c4ba424e1b2c94c4e734dd5b9c5f3c910fbb6b3b436ced6382c7186056a5a23f14566 fe0001a1470100",
|
||||
WillAddHtlc(Block.RegtestGenesisBlock.hash, paymentId, 50_000_000 msat, paymentHash1, CltvExpiry(840_000), TestConstants.emptyOnionPacket, blinding_opt = None) -> hex"a051 06226e46111a0b59caaf126043eb5bbf28c34f3a5e332a1fc7b2b73cf188910f 3118a7954088c27b19923894ed27923c297f88ec3734f90b2b4aafcb11238503 0000000002faf080 80417c0c91deb72606958425ea1552a045a55a250e91870231b486dcb2106734 000cd140 00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000",
|
||||
WillAddHtlc(Block.RegtestGenesisBlock.hash, paymentId, 50_000_000 msat, paymentHash1, CltvExpiry(840_000), TestConstants.emptyOnionPacket, blinding_opt = Some(blinding)) -> hex"a051 06226e46111a0b59caaf126043eb5bbf28c34f3a5e332a1fc7b2b73cf188910f 3118a7954088c27b19923894ed27923c297f88ec3734f90b2b4aafcb11238503 0000000002faf080 80417c0c91deb72606958425ea1552a045a55a250e91870231b486dcb2106734 000cd140 00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000 00210296d5c32655a5eaa8be086479d7bcff967b6e9ca8319b69565747ae16ff20fad6",
|
||||
WillFailHtlc(paymentId, paymentHash1, hex"deadbeef") -> hex"a052 3118a7954088c27b19923894ed27923c297f88ec3734f90b2b4aafcb11238503 80417c0c91deb72606958425ea1552a045a55a250e91870231b486dcb2106734 0004 deadbeef",
|
||||
WillFailMalformedHtlc(paymentId, paymentHash1, ByteVector32(hex"9d60e5791eee0799ce7b00009f56f56c6b988f6129b6a88494cce2cf2fa8b319"), 49157) -> hex"a053 3118a7954088c27b19923894ed27923c297f88ec3734f90b2b4aafcb11238503 80417c0c91deb72606958425ea1552a045a55a250e91870231b486dcb2106734 9d60e5791eee0799ce7b00009f56f56c6b988f6129b6a88494cce2cf2fa8b319 c005",
|
||||
CancelOnTheFlyFunding(channelId, Nil, hex"deadbeef") -> hex"a054 c11b8fbd682b3c6ee11f9d7268e22bb5887cd4d3bf3338bfcc340583f685733c 0000 0004 deadbeef",
|
||||
CancelOnTheFlyFunding(channelId, List(paymentHash1), hex"deadbeef") -> hex"a054 c11b8fbd682b3c6ee11f9d7268e22bb5887cd4d3bf3338bfcc340583f685733c 0001 80417c0c91deb72606958425ea1552a045a55a250e91870231b486dcb2106734 0004 deadbeef",
|
||||
CancelOnTheFlyFunding(channelId, List(paymentHash1, paymentHash2), hex"deadbeef") -> hex"a054 c11b8fbd682b3c6ee11f9d7268e22bb5887cd4d3bf3338bfcc340583f685733c 0002 80417c0c91deb72606958425ea1552a045a55a250e91870231b486dcb21067343213a810a0bfc54566d9be09da1484538b5d19229e928dfa8b692966a8df6785 0004 deadbeef",
|
||||
)
|
||||
for ((expected, encoded) <- testCases) {
|
||||
val decoded = lightningMessageCodec.decode(encoded.bits).require.value
|
||||
assert(decoded == expected)
|
||||
val reEncoded = lightningMessageCodec.encode(decoded).require.bytes
|
||||
assert(reEncoded == encoded)
|
||||
}
|
||||
}
|
||||
|
||||
test("unknown messages") {
|
||||
// Non-standard tag number so this message can only be handled by a codec with a fallback
|
||||
val unknown = UnknownMessage(tag = 47282, data = ByteVector32.Zeroes.bytes)
|
||||
|
@ -168,12 +168,14 @@ class ApiServiceSpec extends AnyFunSuite with ScalatestRouteTest with IdiomaticM
|
||||
ActorRef.noSender,
|
||||
nodeId = aliceNodeId,
|
||||
state = Peer.CONNECTED,
|
||||
features = Some(Features(Features.ChannelRangeQueries -> FeatureSupport.Optional).initFeatures()),
|
||||
address = Some(NodeAddress.fromParts("127.0.0.1", 9731).get),
|
||||
channels = Set(ActorRef.noSender)),
|
||||
PeerInfo(
|
||||
ActorRef.noSender,
|
||||
nodeId = bobNodeId,
|
||||
state = Peer.DISCONNECTED,
|
||||
features = None,
|
||||
address = None,
|
||||
channels = Set(ActorRef.noSender))))
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user