1
0
Fork 0
mirror of https://github.com/ACINQ/eclair.git synced 2025-02-22 06:21:42 +01:00

Rework responses to channel open and rbf (#2608)

Main behavior changes (see commit messages for details):
- channel opening errors are returned with a 200/OK status from the api
- we return a success in the case of dual-funding or rbf, if the interactive tx has completed, even if the publish fails
- for rbf, we send the success response later in the flow: only when the rbf flow is successful, as opposed to when we initiate it

This is a prerequisite to splices, but also a first step towards reworking the channel request/response mechanism.

Co-authored-by: Bastien Teinturier <31281497+t-bast@users.noreply.github.com>
This commit is contained in:
Pierre-Marie Padiou 2023-03-31 14:44:29 +02:00 committed by GitHub
parent df0e7121ef
commit dcedeccb05
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
35 changed files with 345 additions and 295 deletions

View file

@ -36,7 +36,7 @@ import fr.acinq.eclair.channel._
import fr.acinq.eclair.crypto.Sphinx
import fr.acinq.eclair.db.AuditDb.{NetworkFee, Stats}
import fr.acinq.eclair.db.{IncomingPayment, OutgoingPayment, OutgoingPaymentStatus}
import fr.acinq.eclair.io.Peer.{GetPeerInfo, PeerInfo}
import fr.acinq.eclair.io.Peer.{GetPeerInfo, OpenChannelResponse, PeerInfo}
import fr.acinq.eclair.io._
import fr.acinq.eclair.message.{OnionMessages, Postman}
import fr.acinq.eclair.payment._
@ -87,7 +87,7 @@ trait Eclair {
def disconnect(nodeId: PublicKey)(implicit timeout: Timeout): Future[String]
def open(nodeId: PublicKey, fundingAmount: Satoshi, pushAmount_opt: Option[MilliSatoshi], channelType_opt: Option[SupportedChannelType], fundingFeeratePerByte_opt: Option[FeeratePerByte], announceChannel_opt: Option[Boolean], openTimeout_opt: Option[Timeout])(implicit timeout: Timeout): Future[ChannelOpenResponse]
def open(nodeId: PublicKey, fundingAmount: Satoshi, pushAmount_opt: Option[MilliSatoshi], channelType_opt: Option[SupportedChannelType], fundingFeeratePerByte_opt: Option[FeeratePerByte], announceChannel_opt: Option[Boolean], openTimeout_opt: Option[Timeout])(implicit timeout: Timeout): Future[OpenChannelResponse]
def rbfOpen(channelId: ByteVector32, targetFeerate: FeeratePerKw, lockTime_opt: Option[Long])(implicit timeout: Timeout): Future[CommandResponse[CMD_BUMP_FUNDING_FEE]]
@ -189,7 +189,7 @@ class EclairImpl(appKit: Kit) extends Eclair with Logging {
(appKit.switchboard ? Peer.Disconnect(nodeId)).mapTo[Peer.DisconnectResponse].map(_.toString)
}
override def open(nodeId: PublicKey, fundingAmount: Satoshi, pushAmount_opt: Option[MilliSatoshi], channelType_opt: Option[SupportedChannelType], fundingFeeratePerByte_opt: Option[FeeratePerByte], announceChannel_opt: Option[Boolean], openTimeout_opt: Option[Timeout])(implicit timeout: Timeout): Future[ChannelOpenResponse] = {
override def open(nodeId: PublicKey, fundingAmount: Satoshi, pushAmount_opt: Option[MilliSatoshi], channelType_opt: Option[SupportedChannelType], fundingFeeratePerByte_opt: Option[FeeratePerByte], announceChannel_opt: Option[Boolean], openTimeout_opt: Option[Timeout])(implicit timeout: Timeout): Future[OpenChannelResponse] = {
// we want the open timeout to expire *before* the default ask timeout, otherwise user will get a generic response
val openTimeout = openTimeout_opt.getOrElse(Timeout(20 seconds))
for {
@ -202,13 +202,12 @@ class EclairImpl(appKit: Kit) extends Eclair with Logging {
fundingTxFeerate_opt = fundingFeeratePerByte_opt.map(FeeratePerKw(_)),
channelFlags_opt = announceChannel_opt.map(announceChannel => ChannelFlags(announceChannel = announceChannel)),
timeout_opt = Some(openTimeout))
res <- (appKit.switchboard ? open).mapTo[ChannelOpenResponse]
res <- (appKit.switchboard ? open).mapTo[OpenChannelResponse]
} yield res
}
override def rbfOpen(channelId: ByteVector32, targetFeerate: FeeratePerKw, lockTime_opt: Option[Long])(implicit timeout: Timeout): Future[CommandResponse[CMD_BUMP_FUNDING_FEE]] = {
val cmd = CMD_BUMP_FUNDING_FEE(ActorRef.noSender, targetFeerate, lockTime_opt.getOrElse(appKit.nodeParams.currentBlockHeight.toLong))
sendToChannel(Left(channelId), cmd)
sendToChannelTyped(Left(channelId), CMD_BUMP_FUNDING_FEE(_, targetFeerate, lockTime_opt.getOrElse(appKit.nodeParams.currentBlockHeight.toLong)))
}
override def close(channels: List[ApiTypes.ChannelIdentifier], scriptPubKey_opt: Option[ByteVector], closingFeerates_opt: Option[ClosingFeerates])(implicit timeout: Timeout): Future[Map[ApiTypes.ChannelIdentifier, Either[Throwable, CommandResponse[CMD_CLOSE]]]] = {
@ -483,6 +482,19 @@ class EclairImpl(appKit: Kit) extends Eclair with Logging {
case t: Register.ForwardShortIdFailure[C]@unchecked => throw ChannelNotFound(Right(t.fwd.shortChannelId))
}
private def sendToChannelTyped[C <: Command, R <: CommandResponse[C]](channel: ApiTypes.ChannelIdentifier, cmdBuilder: akka.actor.typed.ActorRef[Any] => C)(implicit timeout: Timeout): Future[R] =
appKit.register.toTyped.ask[Any] { replyTo =>
val cmd = cmdBuilder(replyTo)
channel match {
case Left(channelId) => Register.Forward(replyTo, channelId, cmd)
case Right(shortChannelId) => Register.ForwardShortId(replyTo, shortChannelId, cmd)
}
}.map {
case t: R@unchecked => t
case t: Register.ForwardFailure[C]@unchecked => throw ChannelNotFound(Left(t.fwd.channelId))
case t: Register.ForwardShortIdFailure[C]@unchecked => throw ChannelNotFound(Right(t.fwd.shortChannelId))
}
/**
* Send a request to multiple channels and expect responses.
*

View file

@ -23,6 +23,7 @@ import fr.acinq.eclair.blockchain.fee.FeeratePerKw
import fr.acinq.eclair.channel.LocalFundingStatus.DualFundedUnconfirmedFundingTx
import fr.acinq.eclair.channel.fund.InteractiveTxBuilder
import fr.acinq.eclair.channel.fund.InteractiveTxBuilder._
import fr.acinq.eclair.io.Peer
import fr.acinq.eclair.payment.OutgoingPaymentPacket.Upstream
import fr.acinq.eclair.transactions.CommitmentSpec
import fr.acinq.eclair.transactions.Transactions._
@ -102,7 +103,8 @@ case class INPUT_INIT_CHANNEL_INITIATOR(temporaryChannelId: ByteVector32,
channelFlags: ChannelFlags,
channelConfig: ChannelConfig,
channelType: SupportedChannelType,
channelOrigin: ChannelOrigin = ChannelOrigin.Default) {
channelOrigin: ChannelOrigin = ChannelOrigin.Default,
replyTo: akka.actor.typed.ActorRef[Peer.OpenChannelResponse]) {
require(!(channelType.features.contains(Features.ScidAlias) && channelFlags.announceChannel), "option_scid_alias is not compatible with public channels")
}
case class INPUT_INIT_CHANNEL_NON_INITIATOR(temporaryChannelId: ByteVector32,
@ -200,7 +202,7 @@ sealed trait CloseCommand extends HasReplyToCommand
final case class CMD_CLOSE(replyTo: ActorRef, scriptPubKey: Option[ByteVector], feerates: Option[ClosingFeerates]) extends CloseCommand
final case class CMD_FORCECLOSE(replyTo: ActorRef) extends CloseCommand
final case class CMD_BUMP_FUNDING_FEE(replyTo: ActorRef, targetFeerate: FeeratePerKw, lockTime: Long) extends HasReplyToCommand
final case class CMD_BUMP_FUNDING_FEE(replyTo: akka.actor.typed.ActorRef[CommandResponse[CMD_BUMP_FUNDING_FEE]], targetFeerate: FeeratePerKw, lockTime: Long) extends Command
final case class CMD_UPDATE_RELAY_FEE(replyTo: ActorRef, feeBase: MilliSatoshi, feeProportionalMillionths: Long, cltvExpiryDelta_opt: Option[CltvExpiryDelta]) extends HasReplyToCommand
final case class CMD_GET_CHANNEL_STATE(replyTo: ActorRef) extends HasReplyToCommand
final case class CMD_GET_CHANNEL_DATA(replyTo: ActorRef) extends HasReplyToCommand
@ -248,22 +250,11 @@ object HtlcResult {
final case class RES_ADD_SETTLED[+O <: Origin, +R <: HtlcResult](origin: O, htlc: UpdateAddHtlc, result: R) extends CommandSuccess[CMD_ADD_HTLC]
/** other specific responses */
final case class RES_BUMP_FUNDING_FEE(rbfIndex: Int, fundingTxId: ByteVector32, fee: Satoshi) extends CommandSuccess[CMD_BUMP_FUNDING_FEE]
final case class RES_GET_CHANNEL_STATE(state: ChannelState) extends CommandSuccess[CMD_GET_CHANNEL_STATE]
final case class RES_GET_CHANNEL_DATA[+D <: ChannelData](data: D) extends CommandSuccess[CMD_GET_CHANNEL_DATA]
final case class RES_GET_CHANNEL_INFO(nodeId: PublicKey, channelId: ByteVector32, state: ChannelState, data: ChannelData) extends CommandSuccess[CMD_GET_CHANNEL_INFO]
/**
* Those are not response to [[Command]], but to [[fr.acinq.eclair.io.Peer.OpenChannel]]
*
* If actor A sends a [[fr.acinq.eclair.io.Peer.OpenChannel]] and actor B sends a [[CMD_CLOSE]], then A will receive a
* [[ChannelOpenResponse.ChannelClosed]] whereas B will receive a [[RES_SUCCESS]]
*/
sealed trait ChannelOpenResponse
object ChannelOpenResponse {
case class ChannelOpened(channelId: ByteVector32) extends ChannelOpenResponse { override def toString = s"created channel $channelId" }
case class ChannelClosed(channelId: ByteVector32) extends ChannelOpenResponse { override def toString = s"closed channel $channelId" }
}
/*
8888888b. d8888 88888888888 d8888
888 "Y88b d88888 888 d88888
@ -438,7 +429,7 @@ sealed trait RbfStatus
object RbfStatus {
case object NoRbf extends RbfStatus
case class RbfRequested(cmd: CMD_BUMP_FUNDING_FEE) extends RbfStatus
case class RbfInProgress(rbf: typed.ActorRef[InteractiveTxBuilder.Command]) extends RbfStatus
case class RbfInProgress(cmd_opt: Option[CMD_BUMP_FUNDING_FEE], rbf: typed.ActorRef[InteractiveTxBuilder.Command]) extends RbfStatus
case object RbfAborted extends RbfStatus
}
@ -467,7 +458,8 @@ final case class DATA_WAIT_FOR_FUNDING_INTERNAL(params: ChannelParams,
fundingAmount: Satoshi,
pushAmount: MilliSatoshi,
commitTxFeerate: FeeratePerKw,
remoteFirstPerCommitmentPoint: PublicKey) extends TransientChannelData {
remoteFirstPerCommitmentPoint: PublicKey,
replyTo: akka.actor.typed.ActorRef[Peer.OpenChannelResponse]) extends TransientChannelData {
val channelId: ByteVector32 = params.channelId
}
final case class DATA_WAIT_FOR_FUNDING_CREATED(params: ChannelParams,
@ -483,7 +475,8 @@ final case class DATA_WAIT_FOR_FUNDING_SIGNED(params: ChannelParams,
localSpec: CommitmentSpec,
localCommitTx: CommitTx,
remoteCommit: RemoteCommit,
lastSent: FundingCreated) extends TransientChannelData {
lastSent: FundingCreated,
replyTo: akka.actor.typed.ActorRef[Peer.OpenChannelResponse]) extends TransientChannelData {
val channelId: ByteVector32 = params.channelId
}
final case class DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments: Commitments,
@ -507,7 +500,8 @@ final case class DATA_WAIT_FOR_DUAL_FUNDING_CREATED(channelId: ByteVector32,
localPushAmount: MilliSatoshi,
remotePushAmount: MilliSatoshi,
txBuilder: typed.ActorRef[InteractiveTxBuilder.Command],
deferred: Option[ChannelReady]) extends TransientChannelData
deferred: Option[ChannelReady],
replyTo_opt: Option[akka.actor.typed.ActorRef[Peer.OpenChannelResponse]]) extends TransientChannelData
final case class DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments: Commitments,
localPushAmount: MilliSatoshi,
remotePushAmount: MilliSatoshi,

View file

@ -21,6 +21,7 @@ import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
import fr.acinq.bitcoin.scalacompat.{ByteVector32, Satoshi, Transaction}
import fr.acinq.eclair.blockchain.fee.FeeratePerKw
import fr.acinq.eclair.channel.Helpers.Closing.ClosingType
import fr.acinq.eclair.io.Peer.OpenChannelResponse
import fr.acinq.eclair.wire.protocol.{ChannelAnnouncement, ChannelUpdate}
import fr.acinq.eclair.{BlockHeight, Features, ShortChannelId}
@ -78,7 +79,7 @@ case class ChannelSignatureSent(channel: ActorRef, commitments: Commitments) ext
case class ChannelSignatureReceived(channel: ActorRef, commitments: Commitments) extends ChannelEvent
case class ChannelErrorOccurred(channel: ActorRef, channelId: ByteVector32, remoteNodeId: PublicKey, error: ChannelOpenError, isFatal: Boolean) extends ChannelEvent
case class ChannelErrorOccurred(channel: ActorRef, channelId: ByteVector32, remoteNodeId: PublicKey, error: ChannelError, isFatal: Boolean) extends ChannelEvent
// NB: the fee should be set to 0 when we're not paying it.
case class TransactionPublished(channelId: ByteVector32, remoteNodeId: PublicKey, tx: Transaction, miningFee: Satoshi, desc: String) extends ChannelEvent

View file

@ -17,9 +17,10 @@
package fr.acinq.eclair.channel
import fr.acinq.bitcoin.scalacompat.Crypto.PrivateKey
import fr.acinq.bitcoin.scalacompat.{ByteVector32, Satoshi, Transaction}
import fr.acinq.bitcoin.scalacompat.{ByteVector32, Satoshi}
import fr.acinq.eclair.blockchain.fee.FeeratePerKw
import fr.acinq.eclair.wire.protocol.{AnnouncementSignatures, Error, InteractiveTxMessage, UpdateAddHtlc}
import fr.acinq.eclair.wire.protocol
import fr.acinq.eclair.wire.protocol.{AnnouncementSignatures, InteractiveTxMessage, UpdateAddHtlc}
import fr.acinq.eclair.{BlockHeight, CltvExpiry, CltvExpiryDelta, MilliSatoshi, UInt64}
/**
@ -27,9 +28,9 @@ import fr.acinq.eclair.{BlockHeight, CltvExpiry, CltvExpiryDelta, MilliSatoshi,
*/
// @formatter:off
sealed trait ChannelOpenError
case class LocalError(t: Throwable) extends ChannelOpenError
case class RemoteError(e: Error) extends ChannelOpenError
sealed trait ChannelError
case class LocalError(t: Throwable) extends ChannelError
case class RemoteError(e: protocol.Error) extends ChannelError
// @formatter:on
class ChannelException(val channelId: ByteVector32, message: String) extends RuntimeException(message)

View file

@ -103,8 +103,8 @@ object Channel {
}
}
def props(nodeParams: NodeParams, wallet: OnChainChannelFunder with OnchainPubkeyCache, remoteNodeId: PublicKey, blockchain: typed.ActorRef[ZmqWatcher.Command], relayer: ActorRef, txPublisherFactory: TxPublisherFactory, origin_opt: Option[ActorRef]): Props =
Props(new Channel(nodeParams, wallet, remoteNodeId, blockchain, relayer, txPublisherFactory, origin_opt))
def props(nodeParams: NodeParams, wallet: OnChainChannelFunder with OnchainPubkeyCache, remoteNodeId: PublicKey, blockchain: typed.ActorRef[ZmqWatcher.Command], relayer: ActorRef, txPublisherFactory: TxPublisherFactory): Props =
Props(new Channel(nodeParams, wallet, remoteNodeId, blockchain, relayer, txPublisherFactory))
// see https://github.com/lightningnetwork/lightning-rfc/blob/master/07-routing-gossip.md#requirements
val ANNOUNCEMENTS_MINCONF = 6
@ -172,7 +172,7 @@ object Channel {
}
class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with OnchainPubkeyCache, val remoteNodeId: PublicKey, val blockchain: typed.ActorRef[ZmqWatcher.Command], val relayer: ActorRef, val txPublisherFactory: Channel.TxPublisherFactory, val origin_opt: Option[ActorRef] = None)(implicit val ec: ExecutionContext = ExecutionContext.Implicits.global)
class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with OnchainPubkeyCache, val remoteNodeId: PublicKey, val blockchain: typed.ActorRef[ZmqWatcher.Command], val relayer: ActorRef, val txPublisherFactory: Channel.TxPublisherFactory)(implicit val ec: ExecutionContext = ExecutionContext.Implicits.global)
extends FSM[ChannelState, ChannelData]
with FSMDiagnosticActorLogging[ChannelState, ChannelData]
with ChannelOpenSingleFunded
@ -1533,6 +1533,10 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
replyTo ! RES_SUCCESS(c, d.channelId)
stay()
case Event(c: CMD_BUMP_FUNDING_FEE, d) =>
c.replyTo ! RES_FAILURE(c, CommandUnavailableInThisState(d.channelId, "rbf", stateName))
stay()
// at restore, if the configuration has changed, the channel will send a command to itself to update the relay fees
case Event(RES_SUCCESS(_: CMD_UPDATE_RELAY_FEE, channelId), d: DATA_NORMAL) if channelId == d.channelId => stay()
@ -1834,10 +1838,9 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
val replyTo_opt = c match {
case hasOptionalReplyTo: HasOptionalReplyToCommand => hasOptionalReplyTo.replyTo_opt
case hasReplyTo: HasReplyToCommand => if (hasReplyTo.replyTo == ActorRef.noSender) Some(sender()) else Some(hasReplyTo.replyTo)
case _ => None
}
replyTo_opt.foreach { replyTo =>
replyTo ! RES_SUCCESS(c, newData.channelId)
}
replyTo_opt.foreach(_ ! RES_SUCCESS(c, newData.channelId))
stay() using newData
}
@ -1854,6 +1857,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
val replyTo_opt = c match {
case hasOptionalReplyTo: HasOptionalReplyToCommand => hasOptionalReplyTo.replyTo_opt
case hasReplyTo: HasReplyToCommand => if (hasReplyTo.replyTo == ActorRef.noSender) Some(sender()) else Some(hasReplyTo.replyTo)
case _ => None
}
replyTo_opt.foreach(replyTo => replyTo ! RES_FAILURE(c, cause))
context.system.eventStream.publish(ChannelErrorOccurred(self, stateData.channelId, remoteNodeId, LocalError(cause), isFatal = false))

View file

@ -17,7 +17,6 @@
package fr.acinq.eclair.channel.fsm
import akka.actor.typed.scaladsl.adapter.{ClassicActorContextOps, actorRefAdapter}
import akka.actor.{ActorRef, Status}
import com.softwaremill.quicklens.{ModifyPimp, QuicklensAt}
import fr.acinq.bitcoin.scalacompat.{SatoshiLong, Script}
import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher._
@ -28,6 +27,7 @@ import fr.acinq.eclair.channel.fund.InteractiveTxBuilder
import fr.acinq.eclair.channel.fund.InteractiveTxBuilder.{FullySignedSharedTransaction, InteractiveTxParams, PartiallySignedSharedTransaction, RequireConfirmedInputs}
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.transactions.Scripts
import fr.acinq.eclair.wire.protocol._
import fr.acinq.eclair.{RealShortChannelId, ToMilliSatoshiConversion, UInt64}
@ -220,7 +220,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
localPushAmount = accept.pushAmount, remotePushAmount = open.pushAmount,
wallet))
txBuilder ! InteractiveTxBuilder.Start(self)
goto(WAIT_FOR_DUAL_FUNDING_CREATED) using DATA_WAIT_FOR_DUAL_FUNDING_CREATED(channelId, channelParams, open.secondPerCommitmentPoint, accept.pushAmount, open.pushAmount, txBuilder, None) sending accept
goto(WAIT_FOR_DUAL_FUNDING_CREATED) using DATA_WAIT_FOR_DUAL_FUNDING_CREATED(channelId, channelParams, open.secondPerCommitmentPoint, accept.pushAmount, open.pushAmount, txBuilder, deferred = None, replyTo_opt = None) sending accept
}
case Event(c: CloseCommand, d) => handleFastClose(c, d.channelId)
@ -235,7 +235,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
import d.init.{localParams, remoteInit}
Helpers.validateParamsDualFundedInitiator(nodeParams, d.init.channelType, localParams.initFeatures, remoteInit.features, d.lastSent, accept) match {
case Left(t) =>
channelOpenReplyToUser(Left(LocalError(t)))
d.init.replyTo ! OpenChannelResponse.Rejected(t.getMessage)
handleLocalError(t, d, Some(accept))
case Right((channelFeatures, remoteShutdownScript)) =>
// We've exchanged open_channel2 and accept_channel2, we now know the final channelId.
@ -287,23 +287,23 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
localPushAmount = d.lastSent.pushAmount, remotePushAmount = accept.pushAmount,
wallet))
txBuilder ! InteractiveTxBuilder.Start(self)
goto(WAIT_FOR_DUAL_FUNDING_CREATED) using DATA_WAIT_FOR_DUAL_FUNDING_CREATED(channelId, channelParams, accept.secondPerCommitmentPoint, d.lastSent.pushAmount, accept.pushAmount, txBuilder, None)
goto(WAIT_FOR_DUAL_FUNDING_CREATED) using DATA_WAIT_FOR_DUAL_FUNDING_CREATED(channelId, channelParams, accept.secondPerCommitmentPoint, d.lastSent.pushAmount, accept.pushAmount, txBuilder, deferred = None, replyTo_opt = Some(d.init.replyTo))
}
case Event(c: CloseCommand, d: DATA_WAIT_FOR_ACCEPT_DUAL_FUNDED_CHANNEL) =>
channelOpenReplyToUser(Right(ChannelOpenResponse.ChannelClosed(d.channelId)))
d.init.replyTo ! OpenChannelResponse.Cancelled
handleFastClose(c, d.channelId)
case Event(e: Error, d: DATA_WAIT_FOR_ACCEPT_DUAL_FUNDED_CHANNEL) =>
channelOpenReplyToUser(Left(RemoteError(e)))
d.init.replyTo ! OpenChannelResponse.RemoteError(e.toAscii)
handleRemoteError(e, d)
case Event(INPUT_DISCONNECTED, _) =>
channelOpenReplyToUser(Left(LocalError(new RuntimeException("disconnected"))))
case Event(INPUT_DISCONNECTED, d: DATA_WAIT_FOR_ACCEPT_DUAL_FUNDED_CHANNEL) =>
d.init.replyTo ! OpenChannelResponse.Disconnected
goto(CLOSED)
case Event(TickChannelOpenTimeout, _) =>
channelOpenReplyToUser(Left(LocalError(new RuntimeException("open channel cancelled, took too long"))))
case Event(TickChannelOpenTimeout, d: DATA_WAIT_FOR_ACCEPT_DUAL_FUNDED_CHANNEL) =>
d.init.replyTo ! OpenChannelResponse.TimedOut
goto(CLOSED)
})
@ -319,7 +319,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
case msg: TxAbort =>
log.info("our peer aborted the dual funding flow: ascii='{}' bin={}", msg.toAscii, msg.data)
d.txBuilder ! InteractiveTxBuilder.Abort
channelOpenReplyToUser(Left(LocalError(DualFundingAborted(d.channelId))))
d.replyTo_opt.foreach(_ ! OpenChannelResponse.RemoteError(msg.toAscii))
goto(CLOSED) sending TxAbort(d.channelId, DualFundingAborted(d.channelId).getMessage)
case _: TxInitRbf =>
log.info("ignoring unexpected tx_init_rbf message")
@ -342,6 +342,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
case InteractiveTxBuilder.Succeeded(fundingTx, commitment) =>
d.deferred.foreach(self ! _)
watchFundingConfirmed(fundingTx.sharedTx.txId, fundingTx.fundingParams.minDepth_opt)
d.replyTo_opt.foreach(_ ! OpenChannelResponse.Created(d.channelId, commitment.fundingTxId, fundingTx.sharedTx.tx.localFees))
val commitments = Commitments(
params = d.channelParams,
changes = CommitmentChanges.init(),
@ -356,28 +357,28 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
case sharedTx: FullySignedSharedTransaction => goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED) using d1 storing() sending sharedTx.localSigs calling publishFundingTx(fundingTx)
}
case f: InteractiveTxBuilder.Failed =>
channelOpenReplyToUser(Left(LocalError(f.cause)))
d.replyTo_opt.foreach(_ ! OpenChannelResponse.Rejected(f.cause.getMessage))
goto(CLOSED) sending TxAbort(d.channelId, f.cause.getMessage)
}
case Event(c: CloseCommand, d: DATA_WAIT_FOR_DUAL_FUNDING_CREATED) =>
d.txBuilder ! InteractiveTxBuilder.Abort
channelOpenReplyToUser(Right(ChannelOpenResponse.ChannelClosed(d.channelId)))
d.replyTo_opt.foreach(_ ! OpenChannelResponse.Cancelled)
handleFastClose(c, d.channelId)
case Event(e: Error, d: DATA_WAIT_FOR_DUAL_FUNDING_CREATED) =>
d.txBuilder ! InteractiveTxBuilder.Abort
channelOpenReplyToUser(Left(RemoteError(e)))
d.replyTo_opt.foreach(_ ! OpenChannelResponse.RemoteError(e.toAscii))
handleRemoteError(e, d)
case Event(INPUT_DISCONNECTED, d: DATA_WAIT_FOR_DUAL_FUNDING_CREATED) =>
d.txBuilder ! InteractiveTxBuilder.Abort
channelOpenReplyToUser(Left(LocalError(new RuntimeException("disconnected"))))
d.replyTo_opt.foreach(_ ! OpenChannelResponse.Disconnected)
goto(CLOSED)
case Event(TickChannelOpenTimeout, d: DATA_WAIT_FOR_DUAL_FUNDING_CREATED) =>
d.txBuilder ! InteractiveTxBuilder.Abort
channelOpenReplyToUser(Left(LocalError(new RuntimeException("open channel cancelled, took too long"))))
d.replyTo_opt.foreach(_ ! OpenChannelResponse.TimedOut)
goto(CLOSED)
})
@ -399,7 +400,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
}
case _: FullySignedSharedTransaction =>
d.rbfStatus match {
case RbfStatus.RbfInProgress(txBuilder) =>
case RbfStatus.RbfInProgress(_, txBuilder) =>
txBuilder ! InteractiveTxBuilder.ReceiveTxSigs(txSigs)
stay()
case _ =>
@ -410,27 +411,26 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
}
case Event(cmd: CMD_BUMP_FUNDING_FEE, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) =>
val replyTo = if (cmd.replyTo == ActorRef.noSender) sender() else cmd.replyTo
val zeroConf = d.latestFundingTx.fundingParams.minDepth_opt.isEmpty
if (!d.latestFundingTx.fundingParams.isInitiator) {
replyTo ! Status.Failure(InvalidRbfNonInitiator(d.channelId))
cmd.replyTo ! RES_FAILURE(cmd, InvalidRbfNonInitiator(d.channelId))
stay()
} else if (zeroConf) {
replyTo ! Status.Failure(InvalidRbfZeroConf(d.channelId))
cmd.replyTo ! RES_FAILURE(cmd, InvalidRbfZeroConf(d.channelId))
stay()
} else {
d.rbfStatus match {
case RbfStatus.NoRbf =>
val minNextFeerate = d.latestFundingTx.fundingParams.minNextFeerate
if (cmd.targetFeerate < minNextFeerate) {
replyTo ! Status.Failure(InvalidRbfFeerate(d.channelId, cmd.targetFeerate, minNextFeerate))
cmd.replyTo ! RES_FAILURE(cmd, InvalidRbfFeerate(d.channelId, cmd.targetFeerate, minNextFeerate))
stay()
} else {
stay() using d.copy(rbfStatus = RbfStatus.RbfRequested(cmd.copy(replyTo = replyTo))) sending TxInitRbf(d.channelId, cmd.lockTime, cmd.targetFeerate, d.latestFundingTx.fundingParams.localAmount)
stay() using d.copy(rbfStatus = RbfStatus.RbfRequested(cmd)) sending TxInitRbf(d.channelId, cmd.lockTime, cmd.targetFeerate, d.latestFundingTx.fundingParams.localAmount)
}
case _ =>
log.warning("cannot initiate rbf, another one is already in progress")
replyTo ! Status.Failure(InvalidRbfAlreadyInProgress(d.channelId))
cmd.replyTo ! RES_FAILURE(cmd, InvalidRbfAlreadyInProgress(d.channelId))
stay()
}
}
@ -480,7 +480,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
Some(TxAckRbf(d.channelId, fundingParams.localAmount)),
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(txBuilder)) sending toSend
stay() using d.copy(rbfStatus = RbfStatus.RbfInProgress(cmd_opt = None, txBuilder)) sending toSend
}
case RbfStatus.RbfAborted =>
log.info("rejecting rbf attempt: our previous tx_abort was not acked")
@ -500,7 +500,6 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
stay() using d.copy(rbfStatus = RbfStatus.RbfAborted) sending TxAbort(d.channelId, error.getMessage)
case RbfStatus.RbfRequested(cmd) =>
log.info("our peer accepted our rbf attempt and will contribute {} to the funding transaction", msg.fundingContribution)
cmd.replyTo ! RES_SUCCESS(cmd, d.channelId)
val fundingParams = d.latestFundingTx.fundingParams.copy(
// we don't change our funding contribution
remoteAmount = msg.fundingContribution,
@ -516,7 +515,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
localPushAmount = d.localPushAmount, remotePushAmount = d.remotePushAmount,
wallet))
txBuilder ! InteractiveTxBuilder.Start(self)
stay() using d.copy(rbfStatus = RbfStatus.RbfInProgress(txBuilder))
stay() using d.copy(rbfStatus = RbfStatus.RbfInProgress(cmd_opt = Some(cmd), txBuilder))
case _ =>
log.info("ignoring unexpected tx_ack_rbf")
stay() sending Warning(d.channelId, UnexpectedInteractiveTxMessage(d.channelId, msg).getMessage)
@ -524,7 +523,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
case Event(msg: InteractiveTxConstructionMessage, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) =>
d.rbfStatus match {
case RbfStatus.RbfInProgress(txBuilder) =>
case RbfStatus.RbfInProgress(_, txBuilder) =>
txBuilder ! InteractiveTxBuilder.ReceiveTxMessage(msg)
stay()
case _ =>
@ -534,7 +533,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
case Event(commitSig: CommitSig, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) =>
d.rbfStatus match {
case RbfStatus.RbfInProgress(txBuilder) =>
case RbfStatus.RbfInProgress(_, txBuilder) =>
txBuilder ! InteractiveTxBuilder.ReceiveCommitSig(commitSig)
stay()
case _ =>
@ -544,13 +543,14 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
case Event(msg: TxAbort, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) =>
d.rbfStatus match {
case RbfStatus.RbfInProgress(txBuilder) =>
case RbfStatus.RbfInProgress(cmd_opt, txBuilder) =>
log.info("our peer aborted the rbf attempt: ascii='{}' bin={}", msg.toAscii, msg.data)
cmd_opt.foreach(cmd => cmd.replyTo ! RES_FAILURE(cmd, RbfAttemptAborted(d.channelId)))
txBuilder ! InteractiveTxBuilder.Abort
stay() using d.copy(rbfStatus = RbfStatus.NoRbf) sending TxAbort(d.channelId, RbfAttemptAborted(d.channelId).getMessage)
case RbfStatus.RbfRequested(cmd) =>
log.info("our peer rejected our rbf attempt: ascii='{}' bin={}", msg.toAscii, msg.data)
cmd.replyTo ! Status.Failure(new RuntimeException(s"rbf attempt rejected by our peer: ${msg.toAscii}"))
cmd.replyTo ! RES_FAILURE(cmd, new RuntimeException(s"rbf attempt rejected by our peer: ${msg.toAscii}"))
stay() using d.copy(rbfStatus = RbfStatus.NoRbf) sending TxAbort(d.channelId, RbfAttemptAborted(d.channelId).getMessage)
case RbfStatus.RbfAborted =>
log.debug("our peer acked our previous tx_abort")
@ -561,21 +561,31 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
stay() sending TxAbort(d.channelId, DualFundingAborted(d.channelId).getMessage)
}
case Event(msg: InteractiveTxBuilder.Response, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => msg match {
case InteractiveTxBuilder.SendMessage(msg) => stay() sending msg
case InteractiveTxBuilder.Succeeded(fundingTx, commitment) =>
watchFundingConfirmed(fundingTx.sharedTx.txId, fundingTx.fundingParams.minDepth_opt)
// we add the latest commitments to the list
val commitments1 = d.commitments.add(commitment)
val d1 = DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments1, d.localPushAmount, d.remotePushAmount, d.waitingSince, d.lastChecked, RbfStatus.NoRbf, d.deferred)
fundingTx.sharedTx match {
case sharedTx: PartiallySignedSharedTransaction => stay() using d1 storing() sending sharedTx.localSigs
case sharedTx: FullySignedSharedTransaction => stay() using d1 storing() sending sharedTx.localSigs calling publishFundingTx(fundingTx)
}
case f: InteractiveTxBuilder.Failed =>
log.info("rbf attempt failed: {}", f.cause.getMessage)
stay() using d.copy(rbfStatus = RbfStatus.RbfAborted) sending TxAbort(d.channelId, f.cause.getMessage)
}
case Event(msg: InteractiveTxBuilder.Response, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) =>
d.rbfStatus match {
case RbfStatus.RbfInProgress(cmd_opt, _) =>
msg match {
case InteractiveTxBuilder.SendMessage(msg) => stay() sending msg
case InteractiveTxBuilder.Succeeded(fundingTx, commitment) =>
cmd_opt.foreach(cmd => cmd.replyTo ! RES_BUMP_FUNDING_FEE(rbfIndex = d.previousFundingTxs.length, commitment.fundingTxId, fundingTx.sharedTx.tx.fees))
watchFundingConfirmed(fundingTx.sharedTx.txId, fundingTx.fundingParams.minDepth_opt)
// we add the latest commitments to the list
val commitments1 = d.commitments.add(commitment)
val d1 = DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments1, d.localPushAmount, d.remotePushAmount, d.waitingSince, d.lastChecked, RbfStatus.NoRbf, d.deferred)
fundingTx.sharedTx match {
case sharedTx: PartiallySignedSharedTransaction => stay() using d1 storing() sending sharedTx.localSigs
case sharedTx: FullySignedSharedTransaction => stay() using d1 storing() sending sharedTx.localSigs calling publishFundingTx(fundingTx)
}
case f: InteractiveTxBuilder.Failed =>
log.info("rbf attempt failed: {}", f.cause.getMessage)
cmd_opt.foreach(cmd => cmd.replyTo ! RES_FAILURE(cmd, f.cause))
stay() using d.copy(rbfStatus = RbfStatus.RbfAborted) sending TxAbort(d.channelId, f.cause.getMessage)
}
case _ =>
// This can happen if we received a tx_abort right before receiving the interactive-tx result.
log.warning("ignoring interactive-tx result with rbfStatus={}", d.rbfStatus.getClass.getSimpleName)
stay()
}
case Event(w: WatchPublishedTriggered, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) =>
log.info("funding txid={} was successfully published for zero-conf channelId={}", w.tx.txid, d.channelId)
@ -599,11 +609,12 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
val shortIds = createShortIds(d.channelId, realScidStatus)
val channelReady = createChannelReady(shortIds, d.commitments.params)
val toSend = d.rbfStatus match {
case RbfStatus.RbfInProgress(txBuilder) =>
case RbfStatus.RbfInProgress(cmd_opt, txBuilder) =>
txBuilder ! InteractiveTxBuilder.Abort
cmd_opt.foreach(cmd => cmd.replyTo ! RES_FAILURE(cmd, InvalidRbfTxConfirmed(d.channelId)))
Seq(TxAbort(d.channelId, InvalidRbfTxConfirmed(d.channelId).getMessage), channelReady)
case RbfStatus.RbfRequested(cmd) =>
cmd.replyTo ! Status.Failure(InvalidRbfTxConfirmed(d.channelId))
cmd.replyTo ! RES_FAILURE(cmd, InvalidRbfTxConfirmed(d.channelId))
Seq(TxAbort(d.channelId, InvalidRbfTxConfirmed(d.channelId).getMessage), channelReady)
case RbfStatus.NoRbf | RbfStatus.RbfAborted =>
Seq(channelReady)
@ -615,7 +626,15 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
case Event(ProcessCurrentBlockHeight(c), d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => handleNewBlockDualFundingUnconfirmed(c, d)
case Event(e: BITCOIN_FUNDING_DOUBLE_SPENT, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => handleDualFundingDoubleSpent(e, d)
case Event(e: BITCOIN_FUNDING_DOUBLE_SPENT, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) =>
d.rbfStatus match {
case RbfStatus.RbfRequested(cmd) => cmd.replyTo ! RES_FAILURE(cmd, FundingTxDoubleSpent(d.channelId))
case RbfStatus.RbfInProgress(cmd_opt, txBuilder) =>
txBuilder ! InteractiveTxBuilder.Abort
cmd_opt.foreach(cmd => cmd.replyTo ! RES_FAILURE(cmd, FundingTxDoubleSpent(d.channelId)))
case _ => ()
}
handleDualFundingDoubleSpent(e, d)
case Event(remoteChannelReady: ChannelReady, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) =>
if (switchToZeroConf(remoteChannelReady, d)) {
@ -631,14 +650,24 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
case Event(INPUT_DISCONNECTED, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) =>
d.rbfStatus match {
case RbfStatus.RbfInProgress(txBuilder) => txBuilder ! InteractiveTxBuilder.Abort
case RbfStatus.RbfRequested(cmd) => cmd.replyTo ! Status.Failure(new RuntimeException("rbf attempt failed: disconnected"))
case RbfStatus.RbfRequested(cmd) => cmd.replyTo ! RES_FAILURE(cmd, new RuntimeException("rbf attempt failed: disconnected"))
case RbfStatus.RbfInProgress(cmd_opt, txBuilder) =>
cmd_opt.foreach(cmd => cmd.replyTo ! RES_FAILURE(cmd, new RuntimeException("rbf attempt failed: disconnected")))
txBuilder ! InteractiveTxBuilder.Abort
case RbfStatus.RbfAborted => // nothing to do
case RbfStatus.NoRbf => // nothing to do
}
goto(OFFLINE) using d.copy(rbfStatus = RbfStatus.NoRbf)
case Event(e: Error, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => handleRemoteError(e, d)
case Event(e: Error, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) =>
d.rbfStatus match {
case RbfStatus.RbfRequested(cmd) => cmd.replyTo ! RES_FAILURE(cmd, new RuntimeException(s"remote error: ${e.toAscii}"))
case RbfStatus.RbfInProgress(cmd_opt, txBuilder) =>
txBuilder ! InteractiveTxBuilder.Abort
cmd_opt.foreach(cmd => cmd.replyTo ! RES_FAILURE(cmd, new RuntimeException(s"remote error: ${e.toAscii}")))
case _ => ()
}
handleRemoteError(e, d)
})
when(WAIT_FOR_DUAL_FUNDING_READY)(handleExceptions {

View file

@ -28,6 +28,7 @@ import fr.acinq.eclair.channel._
import fr.acinq.eclair.channel.fsm.Channel._
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.transactions.Transactions.TxOwner
import fr.acinq.eclair.transactions.{Scripts, Transactions}
import fr.acinq.eclair.wire.protocol.{AcceptChannel, AnnouncementSignatures, ChannelReady, ChannelTlv, Error, FundingCreated, FundingSigned, OpenChannel, TlvStream}
@ -165,7 +166,7 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers {
case Event(accept: AcceptChannel, d@DATA_WAIT_FOR_ACCEPT_CHANNEL(init, open)) =>
Helpers.validateParamsSingleFundedFunder(nodeParams, init.channelType, init.localParams.initFeatures, init.remoteInit.features, open, accept) match {
case Left(t) =>
channelOpenReplyToUser(Left(LocalError(t)))
d.initFunder.replyTo ! OpenChannelResponse.Rejected(t.getMessage)
handleLocalError(t, d, Some(accept))
case Right((channelFeatures, remoteShutdownScript)) =>
val remoteParams = RemoteParams(
@ -189,28 +190,28 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers {
val fundingPubkeyScript = Script.write(Script.pay2wsh(Scripts.multiSig2of2(localFundingPubkey.publicKey, remoteParams.fundingPubKey)))
wallet.makeFundingTx(fundingPubkeyScript, init.fundingAmount, init.fundingTxFeerate).pipeTo(self)
val params = ChannelParams(init.temporaryChannelId, init.channelConfig, channelFeatures, init.localParams, remoteParams, open.channelFlags)
goto(WAIT_FOR_FUNDING_INTERNAL) using DATA_WAIT_FOR_FUNDING_INTERNAL(params, init.fundingAmount, init.pushAmount_opt.getOrElse(0 msat), init.commitTxFeerate, accept.firstPerCommitmentPoint)
goto(WAIT_FOR_FUNDING_INTERNAL) using DATA_WAIT_FOR_FUNDING_INTERNAL(params, init.fundingAmount, init.pushAmount_opt.getOrElse(0 msat), init.commitTxFeerate, accept.firstPerCommitmentPoint, d.initFunder.replyTo)
}
case Event(c: CloseCommand, d: DATA_WAIT_FOR_ACCEPT_CHANNEL) =>
channelOpenReplyToUser(Right(ChannelOpenResponse.ChannelClosed(d.lastSent.temporaryChannelId)))
d.initFunder.replyTo ! OpenChannelResponse.Cancelled
handleFastClose(c, d.lastSent.temporaryChannelId)
case Event(e: Error, d: DATA_WAIT_FOR_ACCEPT_CHANNEL) =>
channelOpenReplyToUser(Left(RemoteError(e)))
d.initFunder.replyTo ! OpenChannelResponse.RemoteError(e.toAscii)
handleRemoteError(e, d)
case Event(INPUT_DISCONNECTED, _) =>
channelOpenReplyToUser(Left(LocalError(new RuntimeException("disconnected"))))
case Event(INPUT_DISCONNECTED, d: DATA_WAIT_FOR_ACCEPT_CHANNEL) =>
d.initFunder.replyTo ! OpenChannelResponse.Disconnected
goto(CLOSED)
case Event(TickChannelOpenTimeout, _) =>
channelOpenReplyToUser(Left(LocalError(new RuntimeException("open channel cancelled, took too long"))))
case Event(TickChannelOpenTimeout, d: DATA_WAIT_FOR_ACCEPT_CHANNEL) =>
d.initFunder.replyTo ! OpenChannelResponse.TimedOut
goto(CLOSED)
})
when(WAIT_FOR_FUNDING_INTERNAL)(handleExceptions {
case Event(MakeFundingTxResponse(fundingTx, fundingTxOutputIndex, fundingTxFee), d@DATA_WAIT_FOR_FUNDING_INTERNAL(params, fundingAmount, pushMsat, commitTxFeerate, remoteFirstPerCommitmentPoint)) =>
case Event(MakeFundingTxResponse(fundingTx, fundingTxOutputIndex, fundingTxFee), d@DATA_WAIT_FOR_FUNDING_INTERNAL(params, fundingAmount, pushMsat, commitTxFeerate, remoteFirstPerCommitmentPoint, replyTo)) =>
val temporaryChannelId = params.channelId
// let's create the first commitment tx that spends the yet uncommitted funding tx
Funding.makeFirstCommitTxs(keyManager, params, localFundingAmount = fundingAmount, remoteFundingAmount = 0 sat, localPushAmount = pushMsat, remotePushAmount = 0 msat, commitTxFeerate, fundingTx.hash, fundingTxOutputIndex, remoteFirstPerCommitmentPoint) match {
@ -231,28 +232,28 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers {
txPublisher ! SetChannelId(remoteNodeId, channelId)
context.system.eventStream.publish(ChannelIdAssigned(self, remoteNodeId, temporaryChannelId, channelId))
// NB: we don't send a ChannelSignatureSent for the first commit
goto(WAIT_FOR_FUNDING_SIGNED) using DATA_WAIT_FOR_FUNDING_SIGNED(params1, fundingTx, fundingTxFee, localSpec, localCommitTx, RemoteCommit(0, remoteSpec, remoteCommitTx.tx.txid, remoteFirstPerCommitmentPoint), fundingCreated) sending fundingCreated
goto(WAIT_FOR_FUNDING_SIGNED) using DATA_WAIT_FOR_FUNDING_SIGNED(params1, fundingTx, fundingTxFee, localSpec, localCommitTx, RemoteCommit(0, remoteSpec, remoteCommitTx.tx.txid, remoteFirstPerCommitmentPoint), fundingCreated, replyTo) sending fundingCreated
}
case Event(Status.Failure(t), d: DATA_WAIT_FOR_FUNDING_INTERNAL) =>
log.error(t, s"wallet returned error: ")
channelOpenReplyToUser(Left(LocalError(t)))
d.replyTo ! OpenChannelResponse.Rejected(s"wallet error: ${t.getMessage}")
handleLocalError(ChannelFundingError(d.channelId), d, None) // we use a generic exception and don't send the internal error to the peer
case Event(c: CloseCommand, d: DATA_WAIT_FOR_FUNDING_INTERNAL) =>
channelOpenReplyToUser(Right(ChannelOpenResponse.ChannelClosed(d.channelId)))
d.replyTo ! OpenChannelResponse.Cancelled
handleFastClose(c, d.channelId)
case Event(e: Error, d: DATA_WAIT_FOR_FUNDING_INTERNAL) =>
channelOpenReplyToUser(Left(RemoteError(e)))
d.replyTo ! OpenChannelResponse.RemoteError(e.toAscii)
handleRemoteError(e, d)
case Event(INPUT_DISCONNECTED, _) =>
channelOpenReplyToUser(Left(LocalError(new RuntimeException("disconnected"))))
case Event(INPUT_DISCONNECTED, d: DATA_WAIT_FOR_FUNDING_INTERNAL) =>
d.replyTo ! OpenChannelResponse.Disconnected
goto(CLOSED)
case Event(TickChannelOpenTimeout, _) =>
channelOpenReplyToUser(Left(LocalError(new RuntimeException("open channel cancelled, took too long"))))
case Event(TickChannelOpenTimeout, d: DATA_WAIT_FOR_FUNDING_INTERNAL) =>
d.replyTo ! OpenChannelResponse.TimedOut
goto(CLOSED)
})
@ -300,9 +301,7 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers {
}
}
case Event(c: CloseCommand, d: DATA_WAIT_FOR_FUNDING_CREATED) =>
channelOpenReplyToUser(Right(ChannelOpenResponse.ChannelClosed(d.channelId)))
handleFastClose(c, d.channelId)
case Event(c: CloseCommand, d: DATA_WAIT_FOR_FUNDING_CREATED) => handleFastClose(c, d.channelId)
case Event(e: Error, d: DATA_WAIT_FOR_FUNDING_CREATED) => handleRemoteError(e, d)
@ -310,7 +309,7 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers {
})
when(WAIT_FOR_FUNDING_SIGNED)(handleExceptions {
case Event(msg@FundingSigned(_, remoteSig, _), d@DATA_WAIT_FOR_FUNDING_SIGNED(params, fundingTx, fundingTxFee, localSpec, localCommitTx, remoteCommit, fundingCreated)) =>
case Event(msg@FundingSigned(_, remoteSig, _), d@DATA_WAIT_FOR_FUNDING_SIGNED(params, fundingTx, fundingTxFee, localSpec, localCommitTx, remoteCommit, fundingCreated, _)) =>
// we make sure that their sig checks out and that our first commit tx is spendable
val fundingPubKey = keyManager.fundingPublicKey(params.localParams.fundingKeyPath)
val localSigOfLocalTx = keyManager.sign(localCommitTx, fundingPubKey, TxOwner.Local, params.channelFeatures.commitmentFormat)
@ -319,7 +318,7 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers {
case Failure(cause) =>
// we rollback the funding tx, it will never be published
wallet.rollback(fundingTx)
channelOpenReplyToUser(Left(LocalError(cause)))
d.replyTo ! OpenChannelResponse.Rejected(cause.getMessage)
handleLocalError(InvalidCommitmentSignature(d.channelId, signedLocalCommitTx.tx.txid), d, Some(msg))
case Success(_) =>
val commitment = Commitment(
@ -342,31 +341,31 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers {
watchFundingConfirmed(commitment.fundingTxId, Funding.minDepthFunder(params.localParams.initFeatures))
// we will publish the funding tx only after the channel state has been written to disk because we want to
// make sure we first persist the commitment that returns back the funds to us in case of problem
goto(WAIT_FOR_FUNDING_CONFIRMED) using DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments, blockHeight, None, Left(fundingCreated)) storing() calling publishFundingTx(d.channelId, fundingTx, fundingTxFee)
goto(WAIT_FOR_FUNDING_CONFIRMED) using DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments, blockHeight, None, Left(fundingCreated)) storing() calling publishFundingTx(d.channelId, fundingTx, fundingTxFee, d.replyTo)
}
case Event(c: CloseCommand, d: DATA_WAIT_FOR_FUNDING_SIGNED) =>
// we rollback the funding tx, it will never be published
wallet.rollback(d.fundingTx)
channelOpenReplyToUser(Right(ChannelOpenResponse.ChannelClosed(d.channelId)))
d.replyTo ! OpenChannelResponse.Cancelled
handleFastClose(c, d.channelId)
case Event(e: Error, d: DATA_WAIT_FOR_FUNDING_SIGNED) =>
// we rollback the funding tx, it will never be published
wallet.rollback(d.fundingTx)
channelOpenReplyToUser(Left(RemoteError(e)))
d.replyTo ! OpenChannelResponse.RemoteError(e.toAscii)
handleRemoteError(e, d)
case Event(INPUT_DISCONNECTED, d: DATA_WAIT_FOR_FUNDING_SIGNED) =>
// we rollback the funding tx, it will never be published
wallet.rollback(d.fundingTx)
channelOpenReplyToUser(Left(LocalError(new RuntimeException("disconnected"))))
d.replyTo ! OpenChannelResponse.Disconnected
goto(CLOSED)
case Event(TickChannelOpenTimeout, d: DATA_WAIT_FOR_FUNDING_SIGNED) =>
// we rollback the funding tx, it will never be published
wallet.rollback(d.fundingTx)
channelOpenReplyToUser(Left(LocalError(new RuntimeException("open channel cancelled, took too long"))))
d.replyTo ! OpenChannelResponse.TimedOut
goto(CLOSED)
})

View file

@ -16,7 +16,7 @@
package fr.acinq.eclair.channel.fsm
import akka.actor.{FSM, Status}
import akka.actor.{ActorRef, FSM, Status}
import fr.acinq.bitcoin.scalacompat.{ByteVector32, Script}
import fr.acinq.eclair.channel._
import fr.acinq.eclair.db.PendingCommandsDb
@ -37,18 +37,6 @@ trait CommonHandlers {
this: Channel =>
/**
* This function is used to return feedback to user at channel opening
*/
def channelOpenReplyToUser(message: Either[ChannelOpenError, ChannelOpenResponse]): Unit = {
val m = message match {
case Left(LocalError(t)) => Status.Failure(t)
case Left(RemoteError(e)) => Status.Failure(new RuntimeException(s"peer sent error: ascii='${e.toAscii}' bin=${e.data.toHex}"))
case Right(s) => s
}
origin_opt.foreach(_ ! m)
}
def send(msg: LightningMessage): Unit = {
peer ! Peer.OutgoingMessage(msg, activeConnection)
}

View file

@ -16,6 +16,7 @@
package fr.acinq.eclair.channel.fsm
import akka.actor.{ActorRef, Status}
import fr.acinq.bitcoin.scalacompat.{Transaction, TxIn}
import fr.acinq.eclair.NotificationsLogger
import fr.acinq.eclair.NotificationsLogger.NotifyNodeOperator
@ -49,12 +50,8 @@ trait DualFundingHandlers extends CommonFundingHandlers {
// Note that we don't use wallet.commit because we don't want to rollback on failure, since our peer may be able
// to publish and we may be able to RBF.
wallet.publishTransaction(fundingTx.signedTx).onComplete {
case Success(_) =>
context.system.eventStream.publish(TransactionPublished(dualFundedTx.fundingParams.channelId, remoteNodeId, fundingTx.signedTx, fundingTx.tx.localFees, "funding"))
channelOpenReplyToUser(Right(ChannelOpenResponse.ChannelOpened(dualFundedTx.fundingParams.channelId)))
case Failure(t) =>
channelOpenReplyToUser(Left(LocalError(t)))
log.warning("error while publishing funding tx: {}", t.getMessage) // tx may be published by our peer, we can't fail-fast
case Success(_) => context.system.eventStream.publish(TransactionPublished(dualFundedTx.fundingParams.channelId, remoteNodeId, fundingTx.signedTx, fundingTx.tx.localFees, "funding"))
case Failure(t) => log.warning("error while publishing funding tx: {}", t.getMessage) // tx may be published by our peer, we can't fail-fast
}
}
}
@ -105,7 +102,6 @@ trait DualFundingHandlers extends CommonFundingHandlers {
if (fundingTxIds.subsetOf(e.fundingTxIds)) {
log.warning("{} funding attempts have been double-spent, forgetting channel", fundingTxIds.size)
d.allFundingTxs.map(_.sharedTx.tx.buildUnsignedTx()).foreach(tx => wallet.rollback(tx))
channelOpenReplyToUser(Left(LocalError(FundingTxDoubleSpent(d.channelId))))
goto(CLOSED) sending Error(d.channelId, FundingTxDoubleSpent(d.channelId).getMessage)
} else {
// Not all funding attempts have been double-spent, the channel may still confirm.

View file

@ -23,6 +23,7 @@ import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher.{GetTxWithMeta, GetTxWithM
import fr.acinq.eclair.channel._
import fr.acinq.eclair.channel.fsm.Channel.{BITCOIN_FUNDING_PUBLISH_FAILED, BITCOIN_FUNDING_TIMEOUT, FUNDING_TIMEOUT_FUNDEE}
import fr.acinq.eclair.channel.publish.TxPublisher.PublishFinalTx
import fr.acinq.eclair.io.Peer.OpenChannelResponse
import fr.acinq.eclair.wire.protocol.Error
import scala.concurrent.duration.DurationInt
@ -39,16 +40,16 @@ trait SingleFundingHandlers extends CommonFundingHandlers {
this: Channel =>
def publishFundingTx(channelId: ByteVector32, fundingTx: Transaction, fundingTxFee: Satoshi): Unit = {
def publishFundingTx(channelId: ByteVector32, fundingTx: Transaction, fundingTxFee: Satoshi, replyTo: akka.actor.typed.ActorRef[OpenChannelResponse]): Unit = {
wallet.commit(fundingTx).onComplete {
case Success(true) =>
context.system.eventStream.publish(TransactionPublished(channelId, remoteNodeId, fundingTx, fundingTxFee, "funding"))
channelOpenReplyToUser(Right(ChannelOpenResponse.ChannelOpened(channelId)))
replyTo ! OpenChannelResponse.Created(channelId, fundingTx.txid, fundingTxFee)
case Success(false) =>
channelOpenReplyToUser(Left(LocalError(new RuntimeException("couldn't publish funding tx"))))
replyTo ! OpenChannelResponse.Rejected("couldn't publish funding tx")
self ! BITCOIN_FUNDING_PUBLISH_FAILED // fail-fast: this should be returned only when we are really sure the tx has *not* been published
case Failure(t) =>
channelOpenReplyToUser(Left(LocalError(t)))
replyTo ! OpenChannelResponse.Rejected(s"error while committing funding tx: ${t.getMessage}")
log.error(t, "error while committing funding tx: ") // tx may still have been published, can't fail-fast
}
}

View file

@ -17,7 +17,6 @@
package fr.acinq.eclair.io
import akka.actor
import akka.actor.Status
import akka.actor.typed.eventstream.EventStream.Publish
import akka.actor.typed.scaladsl.adapter.TypedActorRefOps
import akka.actor.typed.scaladsl.{ActorContext, Behaviors}
@ -28,7 +27,7 @@ import fr.acinq.eclair.Features.Wumbo
import fr.acinq.eclair.blockchain.OnchainPubkeyCache
import fr.acinq.eclair.channel._
import fr.acinq.eclair.channel.fsm.Channel
import fr.acinq.eclair.io.Peer.SpawnChannelNonInitiator
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
@ -59,7 +58,7 @@ object OpenChannelInterceptor {
val channelFlags: ChannelFlags = open.fold(_.channelFlags, _.channelFlags)
val channelType_opt: Option[ChannelType] = open.fold(_.channelType_opt, _.channelType_opt)
}
case class OpenChannelInitiator(replyTo: ActorRef[Any], remoteNodeId: PublicKey, open: Peer.OpenChannel, localFeatures: Features[InitFeature], remoteFeatures: Features[InitFeature]) extends WaitForRequestCommands
case class OpenChannelInitiator(replyTo: ActorRef[OpenChannelResponse], remoteNodeId: PublicKey, open: Peer.OpenChannel, localFeatures: Features[InitFeature], remoteFeatures: Features[InitFeature]) extends WaitForRequestCommands
private sealed trait CheckRateLimitsCommands extends Command
private case class PendingChannelsRateLimiterResponse(response: PendingChannelsRateLimiter.Response) extends CheckRateLimitsCommands
@ -79,7 +78,7 @@ object OpenChannelInterceptor {
def apply(peer: ActorRef[Any], nodeParams: NodeParams, remoteNodeId: PublicKey, wallet: OnchainPubkeyCache, pendingChannelsRateLimiter: ActorRef[PendingChannelsRateLimiter.Command], pluginTimeout: FiniteDuration = 1 minute): Behavior[Command] =
Behaviors.setup { context =>
Behaviors.withMdc(Logs.mdc(remoteNodeId_opt = Some(remoteNodeId))) {
new OpenChannelInterceptor(peer, pendingChannelsRateLimiter, pluginTimeout, nodeParams, wallet, context).waitForRequest()
new OpenChannelInterceptor(peer, pendingChannelsRateLimiter, pluginTimeout, nodeParams, wallet, context).waitForRequest()
}
}
@ -129,13 +128,13 @@ private class OpenChannelInterceptor(peer: ActorRef[Any],
private def sanityCheckInitiator(request: OpenChannelInitiator): Behavior[Command] = {
if (request.open.fundingAmount >= Channel.MAX_FUNDING && !request.localFeatures.hasFeature(Wumbo)) {
request.replyTo ! Status.Failure(new RuntimeException(s"fundingAmount=${request.open.fundingAmount} is too big, you must enable large channels support in 'eclair.features' to use funding above ${Channel.MAX_FUNDING} (see eclair.conf)"))
request.replyTo ! OpenChannelResponse.Rejected(s"fundingAmount=${request.open.fundingAmount} is too big, you must enable large channels support in 'eclair.features' to use funding above ${Channel.MAX_FUNDING} (see eclair.conf)")
waitForRequest()
} else if (request.open.fundingAmount >= Channel.MAX_FUNDING && !request.remoteFeatures.hasFeature(Wumbo)) {
request.replyTo ! Status.Failure(new RuntimeException(s"fundingAmount=${request.open.fundingAmount} is too big, the remote peer doesn't support wumbo"))
request.replyTo ! OpenChannelResponse.Rejected(s"fundingAmount=${request.open.fundingAmount} is too big, the remote peer doesn't support wumbo")
waitForRequest()
} else if (request.open.fundingAmount > nodeParams.channelConf.maxFundingSatoshis) {
request.replyTo ! Status.Failure(new RuntimeException(s"fundingAmount=${request.open.fundingAmount} is too big for the current settings, increase 'eclair.max-funding-satoshis' (see eclair.conf)"))
request.replyTo ! OpenChannelResponse.Rejected(s"fundingAmount=${request.open.fundingAmount} is too big for the current settings, increase 'eclair.max-funding-satoshis' (see eclair.conf)")
waitForRequest()
} else {
// If a channel type was provided, we directly use it instead of computing it based on local and remote features.
@ -144,7 +143,7 @@ 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, isInitiator = true, dualFunded = dualFunded, request.open.fundingAmount, request.open.disableMaxHtlcValueInFlight)
peer ! Peer.SpawnChannelInitiator(request.open, ChannelConfig.standard, channelType, localParams, request.replyTo.toClassic)
peer ! Peer.SpawnChannelInitiator(request.replyTo, request.open, ChannelConfig.standard, channelType, localParams)
waitForRequest()
}
}
@ -213,7 +212,7 @@ private class OpenChannelInterceptor(peer: ActorRef[Any],
Behaviors.receiveMessage {
case m: B => f(m)
case o: OpenChannelInitiator =>
o.replyTo ! Status.Failure(new RuntimeException("concurrent request rejected"))
o.replyTo ! OpenChannelResponse.Rejected("concurrent request rejected")
Behaviors.same
case o: OpenChannelNonInitiator =>
context.log.warn(s"ignoring remote channel open: concurrent request rejected")

View file

@ -61,7 +61,7 @@ class Peer(val nodeParams: NodeParams, remoteNodeId: PublicKey, wallet: OnchainP
when(INSTANTIATING) {
case Event(Init(storedChannels), _) =>
val channels = storedChannels.map { state =>
val channel = spawnChannel(origin_opt = None)
val channel = spawnChannel()
channel ! INPUT_RESTORED(state)
FinalChannelId(state.channelId) -> channel
}.toMap
@ -108,7 +108,7 @@ class Peer(val nodeParams: NodeParams, remoteNodeId: PublicKey, wallet: OnchainP
stay() using d.copy(channels = d.channels + (FinalChannelId(channelId) -> channel))
case Event(e: SpawnChannelInitiator, _) =>
e.origin ! Status.Failure(new RuntimeException("channel creation failed: disconnected"))
e.replyTo ! OpenChannelResponse.Disconnected
stay()
case Event(_: SpawnChannelNonInitiator, _) => stay() // we got disconnected before creating the channel actor
@ -152,8 +152,8 @@ class Peer(val nodeParams: NodeParams, remoteNodeId: PublicKey, wallet: OnchainP
openChannelInterceptor ! OpenChannelInitiator(sender().toTyped, remoteNodeId, c, d.localFeatures, d.remoteFeatures)
stay()
case Event(SpawnChannelInitiator(c, channelConfig, channelType, localParams, origin), d: ConnectedData) =>
val channel = spawnChannel(Some(origin))
case Event(SpawnChannelInitiator(replyTo, c, channelConfig, channelType, localParams), d: ConnectedData) =>
val channel = spawnChannel()
c.timeout_opt.map(openTimeout => context.system.scheduler.scheduleOnce(openTimeout.duration, channel, Channel.TickChannelOpenTimeout)(context.dispatcher))
val dualFunded = Features.canUseFeature(d.localFeatures, d.remoteFeatures, Features.DualFunding)
val requireConfirmedInputs = c.requireConfirmedInputsOverride_opt.getOrElse(nodeParams.channelConf.requireConfirmedInputsForDualFunding)
@ -165,7 +165,7 @@ class Peer(val nodeParams: NodeParams, remoteNodeId: PublicKey, wallet: OnchainP
val fundingTxFeerate = c.fundingTxFeerate_opt.getOrElse(nodeParams.onChainFeeConf.feeEstimator.getFeeratePerKw(target = nodeParams.onChainFeeConf.feeTargets.fundingBlockTarget))
val commitTxFeerate = nodeParams.onChainFeeConf.getCommitmentFeerate(remoteNodeId, channelType, c.fundingAmount, None)
log.info(s"requesting a new channel with type=$channelType fundingAmount=${c.fundingAmount} dualFunded=$dualFunded pushAmount=${c.pushAmount_opt} fundingFeerate=$fundingTxFeerate temporaryChannelId=$temporaryChannelId localParams=$localParams")
channel ! INPUT_INIT_CHANNEL_INITIATOR(temporaryChannelId, c.fundingAmount, dualFunded, commitTxFeerate, fundingTxFeerate, c.pushAmount_opt, requireConfirmedInputs, localParams, d.peerConnection, d.remoteInit, c.channelFlags_opt.getOrElse(nodeParams.channelConf.channelFlags), channelConfig, channelType, c.channelOrigin)
channel ! INPUT_INIT_CHANNEL_INITIATOR(temporaryChannelId, c.fundingAmount, dualFunded, commitTxFeerate, fundingTxFeerate, c.pushAmount_opt, requireConfirmedInputs, localParams, d.peerConnection, d.remoteInit, c.channelFlags_opt.getOrElse(nodeParams.channelConf.channelFlags), channelConfig, channelType, c.channelOrigin, replyTo)
stay() using d.copy(channels = d.channels + (TemporaryChannelId(temporaryChannelId) -> channel))
case Event(open: protocol.OpenChannel, d: ConnectedData) =>
@ -195,7 +195,7 @@ class Peer(val nodeParams: NodeParams, remoteNodeId: PublicKey, wallet: OnchainP
case Event(SpawnChannelNonInitiator(open, channelConfig, channelType, localParams, peerConnection), d: ConnectedData) =>
val temporaryChannelId = open.fold(_.temporaryChannelId, _.temporaryChannelId)
if (peerConnection == d.peerConnection) {
val channel = spawnChannel(None)
val channel = spawnChannel()
log.info(s"accepting a new channel with type=$channelType temporaryChannelId=$temporaryChannelId localParams=$localParams")
open match {
case Left(open) =>
@ -385,8 +385,8 @@ class Peer(val nodeParams: NodeParams, remoteNodeId: PublicKey, wallet: OnchainP
s(e)
}
def spawnChannel(origin_opt: Option[ActorRef]): ActorRef = {
val channel = channelFactory.spawn(context, remoteNodeId, origin_opt)
def spawnChannel(): ActorRef = {
val channel = channelFactory.spawn(context, remoteNodeId)
context watch channel
channel
}
@ -440,12 +440,12 @@ object Peer {
val CHANNELID_ZERO: ByteVector32 = ByteVector32.Zeroes
trait ChannelFactory {
def spawn(context: ActorContext, remoteNodeId: PublicKey, origin_opt: Option[ActorRef]): ActorRef
def spawn(context: ActorContext, remoteNodeId: PublicKey): ActorRef
}
case class SimpleChannelFactory(nodeParams: NodeParams, watcher: typed.ActorRef[ZmqWatcher.Command], relayer: ActorRef, wallet: OnChainChannelFunder with OnchainPubkeyCache, txPublisherFactory: Channel.TxPublisherFactory) extends ChannelFactory {
override def spawn(context: ActorContext, remoteNodeId: PublicKey, origin_opt: Option[ActorRef]): ActorRef =
context.actorOf(Channel.props(nodeParams, wallet, remoteNodeId, watcher, relayer, txPublisherFactory, origin_opt))
override def spawn(context: ActorContext, remoteNodeId: PublicKey): ActorRef =
context.actorOf(Channel.props(nodeParams, wallet, remoteNodeId, watcher, relayer, txPublisherFactory))
}
def props(nodeParams: NodeParams, remoteNodeId: PublicKey, wallet: OnchainPubkeyCache, channelFactory: ChannelFactory, switchboard: ActorRef, pendingChannelsRateLimiter: typed.ActorRef[PendingChannelsRateLimiter.Command]): Props = Props(new Peer(nodeParams, remoteNodeId, wallet, channelFactory, switchboard, pendingChannelsRateLimiter))
@ -509,13 +509,30 @@ object Peer {
fundingTxFeerate_opt.foreach(feerate => require(feerate >= FeeratePerKw.MinimumFeeratePerKw, s"fee rate $feerate is below minimum ${FeeratePerKw.MinimumFeeratePerKw}"))
}
case class SpawnChannelInitiator(cmd: Peer.OpenChannel, channelConfig: ChannelConfig, channelType: SupportedChannelType, localParams: LocalParams, origin: ActorRef)
sealed trait OpenChannelResponse
object OpenChannelResponse {
/**
* This response doesn't fully guarantee that the channel will actually be opened, because our peer may potentially
* double-spend the funding transaction. Callers must wait for on-chain confirmations if they want guarantees that
* the channel has been opened.
*/
case class Created(channelId: ByteVector32, fundingTxId: ByteVector32, fee: Satoshi) extends OpenChannelResponse { override def toString = s"created channel $channelId with fundingTxId=$fundingTxId and fees=$fee" }
case class Rejected(reason: String) extends OpenChannelResponse { override def toString = reason }
case object Cancelled extends OpenChannelResponse { override def toString = "channel creation cancelled" }
case object Disconnected extends OpenChannelResponse { override def toString = "disconnected" }
case object TimedOut extends OpenChannelResponse { override def toString = "open channel cancelled, took too long" }
case class RemoteError(ascii: String) extends OpenChannelResponse { override def toString = s"peer aborted the channel funding flow: '$ascii'" }
}
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, localParams: LocalParams, peerConnection: ActorRef)
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 PeerNotFound(nodeId: PublicKey) extends PeerInfoResponse with DisconnectResponse { override def toString: String = s"peer $nodeId not found" }

View file

@ -28,6 +28,7 @@ import fr.acinq.eclair.crypto.{ShaChain, Sphinx}
import fr.acinq.eclair.db.FailureType.FailureType
import fr.acinq.eclair.db.{IncomingPaymentStatus, OutgoingPaymentStatus}
import fr.acinq.eclair.io.Peer
import fr.acinq.eclair.io.Peer.OpenChannelResponse
import fr.acinq.eclair.message.OnionMessages
import fr.acinq.eclair.payment.PaymentFailure.PaymentFailedSummary
import fr.acinq.eclair.payment._
@ -198,8 +199,8 @@ object ChannelFeaturesSerializer extends MinimalSerializer({
case channelFeatures: ChannelFeatures => JArray(channelFeatures.features.map(f => JString(f.rfcName)).toList)
})
object ChannelOpenResponseSerializer extends MinimalSerializer({
case x: ChannelOpenResponse => JString(x.toString)
object OpenChannelResponseSerializer extends MinimalSerializer({
case x: OpenChannelResponse => JString(x.toString)
})
object CommandResponseSerializer extends MinimalSerializer({
@ -636,7 +637,7 @@ object JsonSerializers {
UnknownFeatureSerializer +
ChannelConfigSerializer +
ChannelFeaturesSerializer +
ChannelOpenResponseSerializer +
OpenChannelResponseSerializer +
CommandResponseSerializer +
InputInfoSerializer +
ColorSerializer +

View file

@ -78,7 +78,7 @@ class FuzzySpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with Channe
aliceRegister ! alice
bobRegister ! bob
// no announcements
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = false, TestConstants.feeratePerKw, TestConstants.feeratePerKw, Some(TestConstants.initiatorPushAmount), requireConfirmedInputs = false, Alice.channelParams, pipe, bobInit, channelFlags = ChannelFlags.Private, ChannelConfig.standard, ChannelTypes.Standard())
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = false, TestConstants.feeratePerKw, TestConstants.feeratePerKw, Some(TestConstants.initiatorPushAmount), requireConfirmedInputs = false, Alice.channelParams, pipe, bobInit, channelFlags = ChannelFlags.Private, 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())
bob2blockchain.expectMsgType[TxPublisher.SetChannelId]

View file

@ -16,7 +16,7 @@
package fr.acinq.eclair.channel.states
import akka.actor.typed.scaladsl.adapter.actorRefAdapter
import akka.actor.typed.scaladsl.adapter.{ClassicActorRefOps, actorRefAdapter}
import akka.actor.{ActorContext, ActorRef, ActorSystem}
import akka.testkit.{TestFSMRef, TestKit, TestProbe}
import com.softwaremill.quicklens.ModifyPimp
@ -94,7 +94,7 @@ trait ChannelStateTestsBase extends Assertions with Eventually {
case class SetupFixture(alice: TestFSMRef[ChannelState, ChannelData, Channel],
bob: TestFSMRef[ChannelState, ChannelData, Channel],
aliceOrigin: TestProbe,
aliceOpenReplyTo: TestProbe,
alice2bob: TestProbe,
bob2alice: TestProbe,
alice2blockchain: TestProbe,
@ -124,7 +124,7 @@ trait ChannelStateTestsBase extends Assertions with Eventually {
system.registerOnTermination(TestKit.shutdownActorSystem(systemB))
def init(nodeParamsA: NodeParams = TestConstants.Alice.nodeParams, nodeParamsB: NodeParams = TestConstants.Bob.nodeParams, wallet_opt: Option[OnChainWallet with OnchainPubkeyCache] = None, tags: Set[String] = Set.empty): SetupFixture = {
val aliceOrigin = TestProbe()
val aliceOpenReplyTo = TestProbe()
val alice2bob = TestProbe()
val bob2alice = TestProbe()
val alicePeer = TestProbe()
@ -161,13 +161,13 @@ trait ChannelStateTestsBase extends Assertions with Eventually {
}
val alice: TestFSMRef[ChannelState, ChannelData, Channel] = {
implicit val system: ActorSystem = systemA
TestFSMRef(new Channel(finalNodeParamsA, wallet, finalNodeParamsB.nodeId, alice2blockchain.ref, alice2relayer.ref, FakeTxPublisherFactory(alice2blockchain), origin_opt = Some(aliceOrigin.ref)), alicePeer.ref)
TestFSMRef(new Channel(finalNodeParamsA, wallet, finalNodeParamsB.nodeId, alice2blockchain.ref, alice2relayer.ref, FakeTxPublisherFactory(alice2blockchain)), alicePeer.ref)
}
val bob: TestFSMRef[ChannelState, ChannelData, Channel] = {
implicit val system: ActorSystem = systemB
TestFSMRef(new Channel(finalNodeParamsB, wallet, finalNodeParamsA.nodeId, bob2blockchain.ref, bob2relayer.ref, FakeTxPublisherFactory(bob2blockchain)), bobPeer.ref)
}
SetupFixture(alice, bob, aliceOrigin, alice2bob, bob2alice, alice2blockchain, bob2blockchain, router, alice2relayer, bob2relayer, channelUpdateListener, wallet, alicePeer, bobPeer)
SetupFixture(alice, bob, aliceOpenReplyTo, alice2bob, bob2alice, alice2blockchain, bob2blockchain, router, alice2relayer, bob2relayer, channelUpdateListener, wallet, alicePeer, bobPeer)
}
def computeFeatures(setup: SetupFixture, tags: Set[String], channelFlags: ChannelFlags): (LocalParams, LocalParams, SupportedChannelType) = {
@ -245,7 +245,7 @@ trait ChannelStateTestsBase extends Assertions with Eventually {
val aliceInit = Init(aliceParams.initFeatures)
val bobInit = Init(bobParams.initFeatures)
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, fundingAmount, dualFunded, commitTxFeerate, TestConstants.feeratePerKw, initiatorPushAmount, requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType)
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, fundingAmount, dualFunded, commitTxFeerate, TestConstants.feeratePerKw, initiatorPushAmount, requireConfirmedInputs = false, 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, nonInitiatorFundingAmount, dualFunded, nonInitiatorPushAmount, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
assert(bob2blockchain.expectMsgType[TxPublisher.SetChannelId].channelId == ByteVector32.Zeroes)

View file

@ -16,7 +16,7 @@
package fr.acinq.eclair.channel.states.a
import akka.actor.Status
import akka.actor.typed.scaladsl.adapter.ClassicActorRefOps
import akka.testkit.{TestFSMRef, TestProbe}
import fr.acinq.bitcoin.scalacompat.{Block, Btc, ByteVector32, SatoshiLong}
import fr.acinq.eclair.TestConstants.{Alice, Bob}
@ -25,6 +25,7 @@ import fr.acinq.eclair.channel._
import fr.acinq.eclair.channel.fsm.Channel
import fr.acinq.eclair.channel.fsm.Channel.TickChannelOpenTimeout
import fr.acinq.eclair.channel.states.{ChannelStateTestsBase, ChannelStateTestsTags}
import fr.acinq.eclair.io.Peer.OpenChannelResponse
import fr.acinq.eclair.wire.protocol.{AcceptChannel, ChannelTlv, Error, Init, OpenChannel, TlvStream}
import fr.acinq.eclair.{CltvExpiryDelta, FeatureSupport, Features, TestConstants, TestKitBaseClass}
import org.scalatest.funsuite.FixtureAnyFunSuiteLike
@ -39,7 +40,7 @@ import scala.concurrent.duration._
class WaitForAcceptChannelStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with ChannelStateTestsBase {
case class FixtureParam(alice: TestFSMRef[ChannelState, ChannelData, Channel], bob: TestFSMRef[ChannelState, ChannelData, Channel], aliceOrigin: TestProbe, alice2bob: TestProbe, bob2alice: TestProbe, alice2blockchain: TestProbe, listener: TestProbe)
case class FixtureParam(alice: TestFSMRef[ChannelState, ChannelData, Channel], bob: TestFSMRef[ChannelState, ChannelData, Channel], aliceOpenReplyTo: TestProbe, alice2bob: TestProbe, bob2alice: TestProbe, alice2blockchain: TestProbe, listener: TestProbe)
override def withFixture(test: OneArgTest): Outcome = {
import com.softwaremill.quicklens._
@ -66,12 +67,12 @@ class WaitForAcceptChannelStateSpec extends TestKitBaseClass with FixtureAnyFunS
within(30 seconds) {
alice.underlying.system.eventStream.subscribe(listener.ref, classOf[ChannelAborted])
val fundingAmount = if (test.tags.contains(ChannelStateTestsTags.Wumbo)) Btc(5).toSatoshi else TestConstants.fundingSatoshis
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, fundingAmount, dualFunded = false, commitTxFeerate, TestConstants.feeratePerKw, Some(TestConstants.initiatorPushAmount), requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType)
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, fundingAmount, dualFunded = false, commitTxFeerate, TestConstants.feeratePerKw, Some(TestConstants.initiatorPushAmount), requireConfirmedInputs = false, 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)
alice2bob.expectMsgType[OpenChannel]
alice2bob.forward(bob)
awaitCond(alice.stateName == WAIT_FOR_ACCEPT_CHANNEL)
withFixture(test.toNoArgTest(FixtureParam(alice, bob, aliceOrigin, alice2bob, bob2alice, alice2blockchain, listener)))
withFixture(test.toNoArgTest(FixtureParam(alice, bob, aliceOpenReplyTo, alice2bob, bob2alice, alice2blockchain, listener)))
}
}
@ -83,7 +84,7 @@ class WaitForAcceptChannelStateSpec extends TestKitBaseClass with FixtureAnyFunS
assert(accept.channelType_opt.contains(ChannelTypes.Standard()))
bob2alice.forward(alice)
awaitCond(alice.stateName == WAIT_FOR_FUNDING_INTERNAL)
aliceOrigin.expectNoMessage()
aliceOpenReplyTo.expectNoMessage()
}
test("recv AcceptChannel (anchor outputs)", Tag(ChannelStateTestsTags.AnchorOutputs)) { f =>
@ -93,7 +94,7 @@ class WaitForAcceptChannelStateSpec extends TestKitBaseClass with FixtureAnyFunS
bob2alice.forward(alice)
awaitCond(alice.stateName == WAIT_FOR_FUNDING_INTERNAL)
assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_FUNDING_INTERNAL].params.channelFeatures.channelType == ChannelTypes.AnchorOutputs())
aliceOrigin.expectNoMessage()
aliceOpenReplyTo.expectNoMessage()
}
test("recv AcceptChannel (anchor outputs zero fee htlc txs)", Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>
@ -103,7 +104,7 @@ class WaitForAcceptChannelStateSpec extends TestKitBaseClass with FixtureAnyFunS
bob2alice.forward(alice)
awaitCond(alice.stateName == WAIT_FOR_FUNDING_INTERNAL)
assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_FUNDING_INTERNAL].params.channelFeatures.channelType == ChannelTypes.AnchorOutputsZeroFeeHtlcTx())
aliceOrigin.expectNoMessage()
aliceOpenReplyTo.expectNoMessage()
}
test("recv AcceptChannel (anchor outputs zero fee htlc txs and scid alias)", Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs), Tag(ChannelStateTestsTags.ScidAlias)) { f =>
@ -113,7 +114,7 @@ class WaitForAcceptChannelStateSpec extends TestKitBaseClass with FixtureAnyFunS
bob2alice.forward(alice)
awaitCond(alice.stateName == WAIT_FOR_FUNDING_INTERNAL)
assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_FUNDING_INTERNAL].params.channelFeatures.channelType == ChannelTypes.AnchorOutputsZeroFeeHtlcTx(scidAlias = true))
aliceOrigin.expectNoMessage()
aliceOpenReplyTo.expectNoMessage()
}
test("recv AcceptChannel (channel type not set)", Tag(ChannelStateTestsTags.AnchorOutputs)) { f =>
@ -125,7 +126,7 @@ class WaitForAcceptChannelStateSpec extends TestKitBaseClass with FixtureAnyFunS
bob2alice.forward(alice, accept.copy(tlvStream = TlvStream(ChannelTlv.UpfrontShutdownScriptTlv(ByteVector.empty))))
awaitCond(alice.stateName == WAIT_FOR_FUNDING_INTERNAL)
assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_FUNDING_INTERNAL].params.channelFeatures.channelType == ChannelTypes.AnchorOutputs())
aliceOrigin.expectNoMessage()
aliceOpenReplyTo.expectNoMessage()
}
test("recv AcceptChannel (channel type not set but feature bit set)", Tag(ChannelStateTestsTags.ChannelType), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>
@ -136,7 +137,7 @@ class WaitForAcceptChannelStateSpec extends TestKitBaseClass with FixtureAnyFunS
alice2bob.expectMsg(Error(accept.temporaryChannelId, "option_channel_type was negotiated but channel_type is missing"))
listener.expectMsgType[ChannelAborted]
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[Status.Failure]
aliceOpenReplyTo.expectMsgType[OpenChannelResponse.Rejected]
}
test("recv AcceptChannel (non-default channel type)", Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs), Tag("standard-channel-type")) { f =>
@ -147,7 +148,7 @@ class WaitForAcceptChannelStateSpec extends TestKitBaseClass with FixtureAnyFunS
bob2alice.forward(alice, accept)
awaitCond(alice.stateName == WAIT_FOR_FUNDING_INTERNAL)
assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_FUNDING_INTERNAL].params.channelFeatures.channelType == ChannelTypes.Standard())
aliceOrigin.expectNoMessage()
aliceOpenReplyTo.expectNoMessage()
}
test("recv AcceptChannel (non-default channel type not set)", Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs), Tag("standard-channel-type")) { f =>
@ -160,7 +161,7 @@ class WaitForAcceptChannelStateSpec extends TestKitBaseClass with FixtureAnyFunS
alice2bob.expectMsg(Error(accept.temporaryChannelId, "invalid channel_type=anchor_outputs_zero_fee_htlc_tx, expected channel_type=standard"))
listener.expectMsgType[ChannelAborted]
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[Status.Failure]
aliceOpenReplyTo.expectMsgType[OpenChannelResponse.Rejected]
}
test("recv AcceptChannel (anchor outputs channel type without enabling the feature)") { () =>
@ -171,7 +172,7 @@ class WaitForAcceptChannelStateSpec extends TestKitBaseClass with FixtureAnyFunS
// Bob advertises support for anchor outputs, but Alice doesn't.
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, Some(TestConstants.initiatorPushAmount), requireConfirmedInputs = false, aliceParams, alice2bob.ref, Init(bobParams.initFeatures), ChannelFlags.Private, channelConfig, ChannelTypes.AnchorOutputs())
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = false, TestConstants.anchorOutputsFeeratePerKw, TestConstants.feeratePerKw, Some(TestConstants.initiatorPushAmount), requireConfirmedInputs = false, aliceParams, alice2bob.ref, Init(bobParams.initFeatures), ChannelFlags.Private, 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())
val open = alice2bob.expectMsgType[OpenChannel]
assert(open.channelType_opt.contains(ChannelTypes.AnchorOutputs()))
@ -181,7 +182,7 @@ class WaitForAcceptChannelStateSpec extends TestKitBaseClass with FixtureAnyFunS
bob2alice.forward(alice, accept)
awaitCond(alice.stateName == WAIT_FOR_FUNDING_INTERNAL)
assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_FUNDING_INTERNAL].params.channelFeatures.channelType == ChannelTypes.AnchorOutputs())
aliceOrigin.expectNoMessage()
aliceOpenReplyTo.expectNoMessage()
}
test("recv AcceptChannel (invalid channel type)") { f =>
@ -193,7 +194,7 @@ class WaitForAcceptChannelStateSpec extends TestKitBaseClass with FixtureAnyFunS
alice2bob.expectMsg(Error(accept.temporaryChannelId, "invalid channel_type=anchor_outputs, expected channel_type=standard"))
listener.expectMsgType[ChannelAborted]
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[Status.Failure]
aliceOpenReplyTo.expectMsgType[OpenChannelResponse.Rejected]
}
test("recv AcceptChannel (invalid max accepted htlcs)") { f =>
@ -206,7 +207,7 @@ class WaitForAcceptChannelStateSpec extends TestKitBaseClass with FixtureAnyFunS
assert(error == Error(accept.temporaryChannelId, InvalidMaxAcceptedHtlcs(accept.temporaryChannelId, invalidMaxAcceptedHtlcs, Channel.MAX_ACCEPTED_HTLCS).getMessage))
listener.expectMsgType[ChannelAborted]
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[Status.Failure]
aliceOpenReplyTo.expectMsgType[OpenChannelResponse.Rejected]
}
test("recv AcceptChannel (dust limit too low)", Tag("mainnet")) { f =>
@ -219,7 +220,7 @@ class WaitForAcceptChannelStateSpec extends TestKitBaseClass with FixtureAnyFunS
assert(error == Error(accept.temporaryChannelId, DustLimitTooSmall(accept.temporaryChannelId, lowDustLimitSatoshis, Channel.MIN_DUST_LIMIT).getMessage))
listener.expectMsgType[ChannelAborted]
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[Status.Failure]
aliceOpenReplyTo.expectMsgType[OpenChannelResponse.Rejected]
}
test("recv AcceptChannel (dust limit too high)") { f =>
@ -231,7 +232,7 @@ class WaitForAcceptChannelStateSpec extends TestKitBaseClass with FixtureAnyFunS
assert(error == Error(accept.temporaryChannelId, DustLimitTooLarge(accept.temporaryChannelId, highDustLimitSatoshis, Alice.nodeParams.channelConf.maxRemoteDustLimit).getMessage))
listener.expectMsgType[ChannelAborted]
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[Status.Failure]
aliceOpenReplyTo.expectMsgType[OpenChannelResponse.Rejected]
}
test("recv AcceptChannel (to_self_delay too high)") { f =>
@ -243,7 +244,7 @@ class WaitForAcceptChannelStateSpec extends TestKitBaseClass with FixtureAnyFunS
assert(error == Error(accept.temporaryChannelId, ToSelfDelayTooHigh(accept.temporaryChannelId, delayTooHigh, Alice.nodeParams.channelConf.maxToLocalDelay).getMessage))
listener.expectMsgType[ChannelAborted]
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[Status.Failure]
aliceOpenReplyTo.expectMsgType[OpenChannelResponse.Rejected]
}
test("recv AcceptChannel (reserve too high)") { f =>
@ -256,7 +257,7 @@ class WaitForAcceptChannelStateSpec extends TestKitBaseClass with FixtureAnyFunS
assert(error == Error(accept.temporaryChannelId, ChannelReserveTooHigh(accept.temporaryChannelId, reserveTooHigh, 0.3, 0.05).getMessage))
listener.expectMsgType[ChannelAborted]
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[Status.Failure]
aliceOpenReplyTo.expectMsgType[OpenChannelResponse.Rejected]
}
test("recv AcceptChannel (reserve below dust limit)") { f =>
@ -268,7 +269,7 @@ class WaitForAcceptChannelStateSpec extends TestKitBaseClass with FixtureAnyFunS
assert(error == Error(accept.temporaryChannelId, DustLimitTooLarge(accept.temporaryChannelId, accept.dustLimitSatoshis, reserveTooSmall).getMessage))
listener.expectMsgType[ChannelAborted]
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[Status.Failure]
aliceOpenReplyTo.expectMsgType[OpenChannelResponse.Rejected]
}
test("recv AcceptChannel (reserve below our dust limit)") { f =>
@ -281,7 +282,7 @@ class WaitForAcceptChannelStateSpec extends TestKitBaseClass with FixtureAnyFunS
assert(error == Error(accept.temporaryChannelId, ChannelReserveBelowOurDustLimit(accept.temporaryChannelId, reserveTooSmall, open.dustLimitSatoshis).getMessage))
listener.expectMsgType[ChannelAborted]
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[Status.Failure]
aliceOpenReplyTo.expectMsgType[OpenChannelResponse.Rejected]
}
test("recv AcceptChannel (dust limit above our reserve)", Tag("high-remote-dust-limit")) { f =>
@ -294,7 +295,7 @@ class WaitForAcceptChannelStateSpec extends TestKitBaseClass with FixtureAnyFunS
assert(error == Error(accept.temporaryChannelId, DustLimitAboveOurChannelReserve(accept.temporaryChannelId, dustTooBig, open.channelReserveSatoshis).getMessage))
listener.expectMsgType[ChannelAborted]
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[Status.Failure]
aliceOpenReplyTo.expectMsgType[OpenChannelResponse.Rejected]
}
test("recv AcceptChannel (wumbo size channel)", Tag(ChannelStateTestsTags.Wumbo), Tag("high-max-funding-size")) { f =>
@ -303,7 +304,7 @@ class WaitForAcceptChannelStateSpec extends TestKitBaseClass with FixtureAnyFunS
assert(accept.minimumDepth == 13) // with wumbo tag we use fundingSatoshis=5BTC
bob2alice.forward(alice, accept)
awaitCond(alice.stateName == WAIT_FOR_FUNDING_INTERNAL)
aliceOrigin.expectNoMessage()
aliceOpenReplyTo.expectNoMessage()
}
test("recv AcceptChannel (upfront shutdown script)", Tag(ChannelStateTestsTags.UpfrontShutdownScript)) { f =>
@ -313,7 +314,7 @@ class WaitForAcceptChannelStateSpec extends TestKitBaseClass with FixtureAnyFunS
bob2alice.forward(alice, accept)
awaitCond(alice.stateName == WAIT_FOR_FUNDING_INTERNAL)
assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_FUNDING_INTERNAL].params.remoteParams.upfrontShutdownScript_opt == accept.upfrontShutdownScript_opt)
aliceOrigin.expectNoMessage()
aliceOpenReplyTo.expectNoMessage()
}
test("recv AcceptChannel (empty upfront shutdown script)", Tag(ChannelStateTestsTags.UpfrontShutdownScript)) { f =>
@ -324,7 +325,7 @@ class WaitForAcceptChannelStateSpec extends TestKitBaseClass with FixtureAnyFunS
bob2alice.forward(alice, accept1)
awaitCond(alice.stateName == WAIT_FOR_FUNDING_INTERNAL)
assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_FUNDING_INTERNAL].params.remoteParams.upfrontShutdownScript_opt.isEmpty)
aliceOrigin.expectNoMessage()
aliceOpenReplyTo.expectNoMessage()
}
test("recv AcceptChannel (invalid upfront shutdown script)", Tag(ChannelStateTestsTags.UpfrontShutdownScript)) { f =>
@ -334,7 +335,7 @@ class WaitForAcceptChannelStateSpec extends TestKitBaseClass with FixtureAnyFunS
bob2alice.forward(alice, accept1)
listener.expectMsgType[ChannelAborted]
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[Status.Failure]
aliceOpenReplyTo.expectMsgType[OpenChannelResponse.Rejected]
}
test("recv Error") { f =>
@ -342,7 +343,7 @@ class WaitForAcceptChannelStateSpec extends TestKitBaseClass with FixtureAnyFunS
alice ! Error(ByteVector32.Zeroes, "oops")
listener.expectMsgType[ChannelAborted]
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[Status.Failure]
aliceOpenReplyTo.expectMsgType[OpenChannelResponse.RemoteError]
}
test("recv CMD_CLOSE") { f =>
@ -353,7 +354,7 @@ class WaitForAcceptChannelStateSpec extends TestKitBaseClass with FixtureAnyFunS
sender.expectMsg(RES_SUCCESS(c, ByteVector32.Zeroes))
listener.expectMsgType[ChannelAborted]
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[ChannelOpenResponse.ChannelClosed]
aliceOpenReplyTo.expectMsg(OpenChannelResponse.Cancelled)
}
test("recv INPUT_DISCONNECTED") { f =>
@ -361,7 +362,7 @@ class WaitForAcceptChannelStateSpec extends TestKitBaseClass with FixtureAnyFunS
alice ! INPUT_DISCONNECTED
listener.expectMsgType[ChannelAborted]
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[Status.Failure]
aliceOpenReplyTo.expectMsg(OpenChannelResponse.Disconnected)
}
test("recv TickChannelOpenTimeout") { f =>
@ -369,7 +370,7 @@ class WaitForAcceptChannelStateSpec extends TestKitBaseClass with FixtureAnyFunS
alice ! TickChannelOpenTimeout
listener.expectMsgType[ChannelAborted]
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[Status.Failure]
aliceOpenReplyTo.expectMsg(OpenChannelResponse.TimedOut)
}
}

View file

@ -16,7 +16,7 @@
package fr.acinq.eclair.channel.states.a
import akka.actor.Status
import akka.actor.typed.scaladsl.adapter.ClassicActorRefOps
import akka.testkit.{TestFSMRef, TestProbe}
import fr.acinq.bitcoin.scalacompat.{ByteVector32, SatoshiLong}
import fr.acinq.eclair.TestConstants.Alice
@ -24,6 +24,7 @@ import fr.acinq.eclair.channel._
import fr.acinq.eclair.channel.fsm.Channel
import fr.acinq.eclair.channel.fsm.Channel.TickChannelOpenTimeout
import fr.acinq.eclair.channel.states.{ChannelStateTestsBase, ChannelStateTestsTags}
import fr.acinq.eclair.io.Peer.OpenChannelResponse
import fr.acinq.eclair.wire.protocol.{AcceptDualFundedChannel, Error, Init, OpenDualFundedChannel}
import fr.acinq.eclair.{MilliSatoshiLong, TestConstants, TestKitBaseClass}
import org.scalatest.funsuite.FixtureAnyFunSuiteLike
@ -36,7 +37,7 @@ class WaitForAcceptDualFundedChannelStateSpec extends TestKitBaseClass with Fixt
val bobRequiresConfirmedInputs = "bob_requires_confirmed_inputs"
val dualFundingContribution = "dual_funding_contribution"
case class FixtureParam(alice: TestFSMRef[ChannelState, ChannelData, Channel], bob: TestFSMRef[ChannelState, ChannelData, Channel], open: OpenDualFundedChannel, aliceOrigin: TestProbe, alice2bob: TestProbe, bob2alice: TestProbe, listener: TestProbe)
case class FixtureParam(alice: TestFSMRef[ChannelState, ChannelData, Channel], bob: TestFSMRef[ChannelState, ChannelData, Channel], open: OpenDualFundedChannel, aliceOpenReplyTo: TestProbe, alice2bob: TestProbe, bob2alice: TestProbe, listener: TestProbe)
override def withFixture(test: OneArgTest): Outcome = {
val bobNodeParams = if (test.tags.contains(bobRequiresConfirmedInputs)) {
@ -58,12 +59,12 @@ class WaitForAcceptDualFundedChannelStateSpec extends TestKitBaseClass with Fixt
val listener = TestProbe()
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, None, requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, ChannelFlags.Private, channelConfig, channelType)
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = true, TestConstants.anchorOutputsFeeratePerKw, TestConstants.feeratePerKw, None, requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, ChannelFlags.Private, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped)
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, nonInitiatorContribution, dualFunded = true, nonInitiatorPushAmount, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
val open = alice2bob.expectMsgType[OpenDualFundedChannel]
alice2bob.forward(bob, open)
awaitCond(alice.stateName == WAIT_FOR_ACCEPT_DUAL_FUNDED_CHANNEL)
withFixture(test.toNoArgTest(FixtureParam(alice, bob, open, aliceOrigin, alice2bob, bob2alice, listener)))
withFixture(test.toNoArgTest(FixtureParam(alice, bob, open, aliceOpenReplyTo, alice2bob, bob2alice, listener)))
}
}
@ -83,7 +84,7 @@ class WaitForAcceptDualFundedChannelStateSpec extends TestKitBaseClass with Fixt
assert(listener.expectMsgType[ChannelIdAssigned].channelId == Helpers.computeChannelId(open, accept))
awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_CREATED)
aliceOrigin.expectNoMessage()
aliceOpenReplyTo.expectNoMessage()
}
test("recv AcceptDualFundedChannel (with non-initiator contribution)", Tag(ChannelStateTestsTags.DualFunding), Tag(dualFundingContribution), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>
@ -129,7 +130,7 @@ class WaitForAcceptDualFundedChannelStateSpec extends TestKitBaseClass with Fixt
val error = alice2bob.expectMsgType[Error]
assert(error == Error(accept.temporaryChannelId, InvalidFundingAmount(accept.temporaryChannelId, -1 sat, 0 sat, Alice.nodeParams.channelConf.maxFundingSatoshis).getMessage))
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[Status.Failure]
aliceOpenReplyTo.expectMsgType[OpenChannelResponse.Rejected]
}
test("recv AcceptDualFundedChannel (invalid push amount)", Tag(ChannelStateTestsTags.DualFunding), Tag(dualFundingContribution), Tag(ChannelStateTestsTags.NonInitiatorPushAmount), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>
@ -141,7 +142,7 @@ class WaitForAcceptDualFundedChannelStateSpec extends TestKitBaseClass with Fixt
assert(error == Error(accept.temporaryChannelId, InvalidPushAmount(accept.temporaryChannelId, TestConstants.nonInitiatorPushAmount, 25_000_000 msat).getMessage))
listener.expectMsgType[ChannelAborted]
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[Status.Failure]
aliceOpenReplyTo.expectMsgType[OpenChannelResponse.Rejected]
}
test("recv AcceptDualFundedChannel (invalid max accepted htlcs)", Tag(ChannelStateTestsTags.DualFunding), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>
@ -153,7 +154,7 @@ class WaitForAcceptDualFundedChannelStateSpec extends TestKitBaseClass with Fixt
assert(error == Error(accept.temporaryChannelId, InvalidMaxAcceptedHtlcs(accept.temporaryChannelId, invalidMaxAcceptedHtlcs, Channel.MAX_ACCEPTED_HTLCS).getMessage))
listener.expectMsgType[ChannelAborted]
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[Status.Failure]
aliceOpenReplyTo.expectMsgType[OpenChannelResponse.Rejected]
}
test("recv AcceptDualFundedChannel (dust limit too low)", Tag(ChannelStateTestsTags.DualFunding), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>
@ -165,7 +166,7 @@ class WaitForAcceptDualFundedChannelStateSpec extends TestKitBaseClass with Fixt
assert(error == Error(accept.temporaryChannelId, DustLimitTooSmall(accept.temporaryChannelId, lowDustLimit, Channel.MIN_DUST_LIMIT).getMessage))
listener.expectMsgType[ChannelAborted]
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[Status.Failure]
aliceOpenReplyTo.expectMsgType[OpenChannelResponse.Rejected]
}
test("recv AcceptDualFundedChannel (dust limit too high)", Tag(ChannelStateTestsTags.DualFunding), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>
@ -177,7 +178,7 @@ class WaitForAcceptDualFundedChannelStateSpec extends TestKitBaseClass with Fixt
assert(error == Error(accept.temporaryChannelId, DustLimitTooLarge(accept.temporaryChannelId, highDustLimit, Alice.nodeParams.channelConf.maxRemoteDustLimit).getMessage))
listener.expectMsgType[ChannelAborted]
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[Status.Failure]
aliceOpenReplyTo.expectMsgType[OpenChannelResponse.Rejected]
}
test("recv AcceptDualFundedChannel (to_self_delay too high)", Tag(ChannelStateTestsTags.DualFunding), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>
@ -189,7 +190,7 @@ class WaitForAcceptDualFundedChannelStateSpec extends TestKitBaseClass with Fixt
assert(error == Error(accept.temporaryChannelId, ToSelfDelayTooHigh(accept.temporaryChannelId, delayTooHigh, Alice.nodeParams.channelConf.maxToLocalDelay).getMessage))
listener.expectMsgType[ChannelAborted]
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[Status.Failure]
aliceOpenReplyTo.expectMsgType[OpenChannelResponse.Rejected]
}
test("recv Error", Tag(ChannelStateTestsTags.DualFunding), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>
@ -197,7 +198,7 @@ class WaitForAcceptDualFundedChannelStateSpec extends TestKitBaseClass with Fixt
alice ! Error(ByteVector32.Zeroes, "dual funding not supported")
listener.expectMsgType[ChannelAborted]
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[Status.Failure]
aliceOpenReplyTo.expectMsgType[OpenChannelResponse.RemoteError]
}
test("recv CMD_CLOSE", Tag(ChannelStateTestsTags.DualFunding), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>
@ -208,7 +209,7 @@ class WaitForAcceptDualFundedChannelStateSpec extends TestKitBaseClass with Fixt
sender.expectMsg(RES_SUCCESS(c, ByteVector32.Zeroes))
listener.expectMsgType[ChannelAborted]
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[ChannelOpenResponse.ChannelClosed]
aliceOpenReplyTo.expectMsg(OpenChannelResponse.Cancelled)
}
test("recv INPUT_DISCONNECTED", Tag(ChannelStateTestsTags.DualFunding), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>
@ -216,7 +217,7 @@ class WaitForAcceptDualFundedChannelStateSpec extends TestKitBaseClass with Fixt
alice ! INPUT_DISCONNECTED
listener.expectMsgType[ChannelAborted]
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[Status.Failure]
aliceOpenReplyTo.expectMsg(OpenChannelResponse.Disconnected)
}
test("recv TickChannelOpenTimeout", Tag(ChannelStateTestsTags.DualFunding), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>
@ -224,7 +225,7 @@ class WaitForAcceptDualFundedChannelStateSpec extends TestKitBaseClass with Fixt
alice ! TickChannelOpenTimeout
listener.expectMsgType[ChannelAborted]
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[Status.Failure]
aliceOpenReplyTo.expectMsg(OpenChannelResponse.TimedOut)
}
}

View file

@ -16,6 +16,7 @@
package fr.acinq.eclair.channel.states.a
import akka.actor.typed.scaladsl.adapter.ClassicActorRefOps
import akka.testkit.{TestFSMRef, TestProbe}
import fr.acinq.bitcoin.scalacompat.{Block, Btc, ByteVector32, SatoshiLong}
import fr.acinq.eclair.TestConstants.{Alice, Bob}
@ -59,7 +60,7 @@ class WaitForOpenChannelStateSpec extends TestKitBaseClass with FixtureAnyFunSui
val listener = TestProbe()
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, Some(TestConstants.initiatorPushAmount), requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType)
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = false, commitTxFeerate, TestConstants.feeratePerKw, Some(TestConstants.initiatorPushAmount), requireConfirmedInputs = false, 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)
awaitCond(bob.stateName == WAIT_FOR_OPEN_CHANNEL)
withFixture(test.toNoArgTest(FixtureParam(alice, bob, alice2bob, bob2alice, bob2blockchain, listener)))

View file

@ -16,6 +16,7 @@
package fr.acinq.eclair.channel.states.a
import akka.actor.typed.scaladsl.adapter.ClassicActorRefOps
import akka.testkit.{TestFSMRef, TestProbe}
import fr.acinq.bitcoin.scalacompat.{Block, ByteVector32, SatoshiLong}
import fr.acinq.eclair.TestConstants.{Alice, Bob}
@ -56,7 +57,7 @@ class WaitForOpenDualFundedChannelStateSpec extends TestKitBaseClass with Fixtur
val bobInit = Init(bobParams.initFeatures)
val requireConfirmedInputs = test.tags.contains(aliceRequiresConfirmedInputs)
within(30 seconds) {
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = true, TestConstants.anchorOutputsFeeratePerKw, TestConstants.feeratePerKw, pushAmount, requireConfirmedInputs, aliceParams, alice2bob.ref, bobInit, ChannelFlags.Private, channelConfig, channelType)
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = true, TestConstants.anchorOutputsFeeratePerKw, TestConstants.feeratePerKw, pushAmount, requireConfirmedInputs, aliceParams, alice2bob.ref, bobInit, ChannelFlags.Private, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped)
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, None, dualFunded = true, None, 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)))

View file

@ -16,7 +16,7 @@
package fr.acinq.eclair.channel.states.b
import akka.actor.Status
import akka.actor.typed.scaladsl.adapter.ClassicActorRefOps
import akka.testkit.{TestFSMRef, TestProbe}
import fr.acinq.bitcoin.scalacompat.{ByteVector32, ByteVector64, SatoshiLong, Script}
import fr.acinq.eclair.blockchain.SingleKeyOnChainWallet
@ -28,6 +28,7 @@ import fr.acinq.eclair.channel.fsm.Channel.TickChannelOpenTimeout
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.wire.protocol.{AcceptDualFundedChannel, CommitSig, Error, Init, OpenDualFundedChannel, TxAbort, TxAckRbf, TxAddInput, TxAddOutput, TxComplete, TxInitRbf, TxSignatures, Warning}
import fr.acinq.eclair.{Features, MilliSatoshiLong, TestConstants, TestKitBaseClass, ToMilliSatoshiConversion, UInt64, randomBytes32, randomKey}
import org.scalatest.funsuite.FixtureAnyFunSuiteLike
@ -38,7 +39,7 @@ import scala.concurrent.duration.DurationInt
class WaitForDualFundingCreatedStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with ChannelStateTestsBase {
case class FixtureParam(alice: TestFSMRef[ChannelState, ChannelData, Channel], bob: TestFSMRef[ChannelState, ChannelData, Channel], aliceOrigin: TestProbe, 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], aliceOpenReplyTo: 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()
@ -56,7 +57,7 @@ class WaitForDualFundingCreatedStateSpec extends TestKitBaseClass with FixtureAn
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, TestConstants.feeratePerKw, TestConstants.feeratePerKw, initiatorPushAmount, requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType)
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = true, TestConstants.feeratePerKw, TestConstants.feeratePerKw, initiatorPushAmount, requireConfirmedInputs = false, 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)
alice2blockchain.expectMsgType[TxPublisher.SetChannelId] // temporary channel id
bob2blockchain.expectMsgType[TxPublisher.SetChannelId] // temporary channel id
@ -68,7 +69,7 @@ class WaitForDualFundingCreatedStateSpec extends TestKitBaseClass with FixtureAn
bob2blockchain.expectMsgType[TxPublisher.SetChannelId] // final channel id
awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_CREATED)
awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_CREATED)
withFixture(test.toNoArgTest(FixtureParam(alice, bob, aliceOrigin, alice2bob, bob2alice, alice2blockchain, bob2blockchain, wallet, aliceListener, bobListener)))
withFixture(test.toNoArgTest(FixtureParam(alice, bob, aliceOpenReplyTo, alice2bob, bob2alice, alice2blockchain, bob2blockchain, wallet, aliceListener, bobListener)))
}
}
@ -235,7 +236,7 @@ class WaitForDualFundingCreatedStateSpec extends TestKitBaseClass with FixtureAn
awaitCond(wallet.rolledback.length == 2)
aliceListener.expectMsgType[ChannelAborted]
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[Status.Failure]
aliceOpenReplyTo.expectMsgType[OpenChannelResponse.Rejected]
}
test("recv invalid CommitSig", Tag(ChannelStateTestsTags.DualFunding)) { f =>
@ -263,7 +264,7 @@ class WaitForDualFundingCreatedStateSpec extends TestKitBaseClass with FixtureAn
awaitCond(wallet.rolledback.length == 1)
aliceListener.expectMsgType[ChannelAborted]
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[Status.Failure]
aliceOpenReplyTo.expectMsgType[OpenChannelResponse.Rejected]
alice2bob.forward(bob, aliceCommitSig.copy(signature = ByteVector64.Zeroes))
bob2alice.expectMsgType[TxAbort]
@ -301,7 +302,7 @@ class WaitForDualFundingCreatedStateSpec extends TestKitBaseClass with FixtureAn
awaitCond(wallet.rolledback.size == 1)
aliceListener.expectMsgType[ChannelAborted]
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[Status.Failure]
aliceOpenReplyTo.expectMsgType[OpenChannelResponse.Rejected]
// Bob has sent his signatures already, so he cannot close the channel yet.
alice2bob.forward(bob, TxSignatures(channelId(alice), randomBytes32(), Nil))
@ -324,7 +325,7 @@ class WaitForDualFundingCreatedStateSpec extends TestKitBaseClass with FixtureAn
awaitCond(wallet.rolledback.size == 2)
aliceListener.expectMsgType[ChannelAborted]
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[Status.Failure]
aliceOpenReplyTo.expectMsgType[OpenChannelResponse.RemoteError]
}
test("recv TxInitRbf", Tag(ChannelStateTestsTags.DualFunding)) { f =>
@ -338,7 +339,7 @@ class WaitForDualFundingCreatedStateSpec extends TestKitBaseClass with FixtureAn
bob2alice.forward(alice, TxInitRbf(channelId(bob), 0, FeeratePerKw(15_000 sat)))
alice2bob.expectMsgType[Warning]
assert(alice.stateName == WAIT_FOR_DUAL_FUNDING_CREATED)
aliceOrigin.expectNoMessage(100 millis)
aliceOpenReplyTo.expectNoMessage(100 millis)
assert(wallet.rolledback.isEmpty)
}
@ -353,7 +354,7 @@ class WaitForDualFundingCreatedStateSpec extends TestKitBaseClass with FixtureAn
bob2alice.forward(alice, TxAckRbf(channelId(bob)))
alice2bob.expectMsgType[Warning]
assert(alice.stateName == WAIT_FOR_DUAL_FUNDING_CREATED)
aliceOrigin.expectNoMessage(100 millis)
aliceOpenReplyTo.expectNoMessage(100 millis)
assert(wallet.rolledback.isEmpty)
}
@ -365,7 +366,7 @@ class WaitForDualFundingCreatedStateSpec extends TestKitBaseClass with FixtureAn
awaitCond(wallet.rolledback.size == 1)
aliceListener.expectMsgType[ChannelAborted]
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[Status.Failure]
aliceOpenReplyTo.expectMsgType[OpenChannelResponse.RemoteError]
bob ! Error(finalChannelId, "oops")
awaitCond(wallet.rolledback.size == 2)
@ -385,7 +386,7 @@ class WaitForDualFundingCreatedStateSpec extends TestKitBaseClass with FixtureAn
awaitCond(wallet.rolledback.size == 1)
aliceListener.expectMsgType[ChannelAborted]
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[ChannelOpenResponse.ChannelClosed]
aliceOpenReplyTo.expectMsg(OpenChannelResponse.Cancelled)
bob ! c
sender.expectMsg(RES_SUCCESS(c, finalChannelId))
@ -401,7 +402,7 @@ class WaitForDualFundingCreatedStateSpec extends TestKitBaseClass with FixtureAn
awaitCond(wallet.rolledback.size == 1)
aliceListener.expectMsgType[ChannelAborted]
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[Status.Failure]
aliceOpenReplyTo.expectMsg(OpenChannelResponse.Disconnected)
bob ! INPUT_DISCONNECTED
awaitCond(wallet.rolledback.size == 2)
@ -415,7 +416,7 @@ class WaitForDualFundingCreatedStateSpec extends TestKitBaseClass with FixtureAn
awaitCond(wallet.rolledback.size == 1)
aliceListener.expectMsgType[ChannelAborted]
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[Status.Failure]
aliceOpenReplyTo.expectMsg(OpenChannelResponse.TimedOut)
}
}

View file

@ -17,6 +17,7 @@
package fr.acinq.eclair.channel.states.b
import akka.actor.ActorRef
import akka.actor.typed.scaladsl.adapter.ClassicActorRefOps
import akka.testkit.{TestFSMRef, TestProbe}
import fr.acinq.bitcoin.scalacompat.{Btc, ByteVector32, SatoshiLong}
import fr.acinq.eclair.TestConstants.{Alice, Bob}
@ -67,7 +68,7 @@ class WaitForFundingCreatedStateSpec extends TestKitBaseClass with FixtureAnyFun
val listener = TestProbe()
within(30 seconds) {
bob.underlying.system.eventStream.subscribe(listener.ref, classOf[ChannelAborted])
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, fundingSatoshis, dualFunded = false, TestConstants.feeratePerKw, TestConstants.feeratePerKw, Some(pushMsat), requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType)
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, fundingSatoshis, dualFunded = false, TestConstants.feeratePerKw, TestConstants.feeratePerKw, Some(pushMsat), requireConfirmedInputs = false, 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)
bob2blockchain.expectMsgType[TxPublisher.SetChannelId]

View file

@ -17,6 +17,7 @@
package fr.acinq.eclair.channel.states.b
import akka.actor.Status
import akka.actor.typed.scaladsl.adapter.ClassicActorRefOps
import akka.testkit.{TestFSMRef, TestProbe}
import fr.acinq.bitcoin.scalacompat.ByteVector32
import fr.acinq.eclair.blockchain.NoOpOnChainWallet
@ -24,6 +25,7 @@ import fr.acinq.eclair.channel._
import fr.acinq.eclair.channel.fsm.Channel
import fr.acinq.eclair.channel.fsm.Channel.TickChannelOpenTimeout
import fr.acinq.eclair.channel.states.ChannelStateTestsBase
import fr.acinq.eclair.io.Peer.OpenChannelResponse
import fr.acinq.eclair.wire.protocol._
import fr.acinq.eclair.{TestConstants, TestKitBaseClass}
import org.scalatest.Outcome
@ -37,7 +39,7 @@ import scala.concurrent.duration._
class WaitForFundingInternalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with ChannelStateTestsBase {
case class FixtureParam(alice: TestFSMRef[ChannelState, ChannelData, Channel], aliceOrigin: TestProbe, alice2bob: TestProbe, bob2alice: TestProbe, alice2blockchain: TestProbe, listener: TestProbe)
case class FixtureParam(alice: TestFSMRef[ChannelState, ChannelData, Channel], aliceOpenReplyTo: TestProbe, alice2bob: TestProbe, bob2alice: TestProbe, alice2blockchain: TestProbe, listener: TestProbe)
override def withFixture(test: OneArgTest): Outcome = {
val setup = init(wallet_opt = Some(new NoOpOnChainWallet()), tags = test.tags)
@ -50,14 +52,14 @@ class WaitForFundingInternalStateSpec extends TestKitBaseClass with FixtureAnyFu
val listener = TestProbe()
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, Some(TestConstants.initiatorPushAmount), requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType)
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = false, TestConstants.feeratePerKw, TestConstants.feeratePerKw, Some(TestConstants.initiatorPushAmount), requireConfirmedInputs = false, 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)
alice2bob.expectMsgType[OpenChannel]
alice2bob.forward(bob)
bob2alice.expectMsgType[AcceptChannel]
bob2alice.forward(alice)
awaitCond(alice.stateName == WAIT_FOR_FUNDING_INTERNAL)
withFixture(test.toNoArgTest(FixtureParam(alice, aliceOrigin, alice2bob, bob2alice, alice2blockchain, listener)))
withFixture(test.toNoArgTest(FixtureParam(alice, aliceOpenReplyTo, alice2bob, bob2alice, alice2blockchain, listener)))
}
}
@ -66,7 +68,7 @@ class WaitForFundingInternalStateSpec extends TestKitBaseClass with FixtureAnyFu
alice ! Status.Failure(new RuntimeException("insufficient funds"))
listener.expectMsgType[ChannelAborted]
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[Status.Failure]
aliceOpenReplyTo.expectMsgType[OpenChannelResponse.Rejected]
}
test("recv Error") { f =>
@ -74,7 +76,7 @@ class WaitForFundingInternalStateSpec extends TestKitBaseClass with FixtureAnyFu
alice ! Error(ByteVector32.Zeroes, "oops")
listener.expectMsgType[ChannelAborted]
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[Status.Failure]
aliceOpenReplyTo.expectMsgType[OpenChannelResponse.RemoteError]
}
test("recv CMD_CLOSE") { f =>
@ -85,7 +87,7 @@ class WaitForFundingInternalStateSpec extends TestKitBaseClass with FixtureAnyFu
sender.expectMsg(RES_SUCCESS(c, ByteVector32.Zeroes))
listener.expectMsgType[ChannelAborted]
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[ChannelOpenResponse.ChannelClosed]
aliceOpenReplyTo.expectMsg(OpenChannelResponse.Cancelled)
}
test("recv INPUT_DISCONNECTED") { f =>
@ -93,7 +95,7 @@ class WaitForFundingInternalStateSpec extends TestKitBaseClass with FixtureAnyFu
alice ! INPUT_DISCONNECTED
listener.expectMsgType[ChannelAborted]
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[Status.Failure]
aliceOpenReplyTo.expectMsg(OpenChannelResponse.Disconnected)
}
test("recv TickChannelOpenTimeout") { f =>
@ -101,7 +103,7 @@ class WaitForFundingInternalStateSpec extends TestKitBaseClass with FixtureAnyFu
alice ! TickChannelOpenTimeout
listener.expectMsgType[ChannelAborted]
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[Status.Failure]
aliceOpenReplyTo.expectMsg(OpenChannelResponse.TimedOut)
}
}

View file

@ -16,7 +16,7 @@
package fr.acinq.eclair.channel.states.b
import akka.actor.Status
import akka.actor.typed.scaladsl.adapter.ClassicActorRefOps
import akka.testkit.{TestFSMRef, TestProbe}
import fr.acinq.bitcoin.scalacompat.{Btc, ByteVector32, ByteVector64, SatoshiLong}
import fr.acinq.eclair.TestConstants.{Alice, Bob}
@ -27,6 +27,7 @@ import fr.acinq.eclair.channel.fsm.Channel
import fr.acinq.eclair.channel.fsm.Channel.TickChannelOpenTimeout
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.wire.protocol.{AcceptChannel, Error, FundingCreated, FundingSigned, Init, OpenChannel}
import fr.acinq.eclair.{TestConstants, TestKitBaseClass}
import org.scalatest.funsuite.FixtureAnyFunSuiteLike
@ -40,7 +41,7 @@ import scala.concurrent.duration._
class WaitForFundingSignedStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with ChannelStateTestsBase {
case class FixtureParam(alice: TestFSMRef[ChannelState, ChannelData, Channel], aliceOrigin: TestProbe, alice2bob: TestProbe, bob2alice: TestProbe, alice2blockchain: TestProbe, listener: TestProbe)
case class FixtureParam(alice: TestFSMRef[ChannelState, ChannelData, Channel], aliceOpenReplyTo: TestProbe, alice2bob: TestProbe, bob2alice: TestProbe, alice2blockchain: TestProbe, listener: TestProbe)
override def withFixture(test: OneArgTest): Outcome = {
import com.softwaremill.quicklens._
@ -66,7 +67,7 @@ class WaitForFundingSignedStateSpec extends TestKitBaseClass with FixtureAnyFunS
val listener = TestProbe()
within(30 seconds) {
alice.underlying.system.eventStream.subscribe(listener.ref, classOf[ChannelAborted])
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, fundingSatoshis, dualFunded = false, TestConstants.feeratePerKw, TestConstants.feeratePerKw, Some(pushMsat), requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType)
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, fundingSatoshis, dualFunded = false, TestConstants.feeratePerKw, TestConstants.feeratePerKw, Some(pushMsat), requireConfirmedInputs = false, 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)
bob2blockchain.expectMsgType[TxPublisher.SetChannelId]
@ -78,7 +79,7 @@ class WaitForFundingSignedStateSpec extends TestKitBaseClass with FixtureAnyFunS
alice2bob.forward(bob)
alice2blockchain.expectMsgType[TxPublisher.SetChannelId]
awaitCond(alice.stateName == WAIT_FOR_FUNDING_SIGNED)
withFixture(test.toNoArgTest(FixtureParam(alice, aliceOrigin, alice2bob, bob2alice, alice2blockchain, listener)))
withFixture(test.toNoArgTest(FixtureParam(alice, aliceOpenReplyTo, alice2bob, bob2alice, alice2blockchain, listener)))
}
}
@ -95,7 +96,7 @@ class WaitForFundingSignedStateSpec extends TestKitBaseClass with FixtureAnyFunS
val txPublished = listener.expectMsgType[TransactionPublished]
assert(txPublished.tx.txid == fundingTxId)
assert(txPublished.miningFee > 0.sat)
aliceOrigin.expectMsgType[ChannelOpenResponse.ChannelOpened]
aliceOpenReplyTo.expectMsgType[OpenChannelResponse.Created]
}
test("recv FundingSigned with valid signature (zero-conf)", Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs), Tag(ChannelStateTestsTags.ZeroConf)) { f =>
@ -106,7 +107,7 @@ class WaitForFundingSignedStateSpec extends TestKitBaseClass with FixtureAnyFunS
// alice doesn't watch for the funding tx to confirm, she only waits for the transaction to be published
alice2blockchain.expectMsgType[WatchPublished]
alice2blockchain.expectNoMessage(100 millis)
aliceOrigin.expectMsgType[ChannelOpenResponse.ChannelOpened]
aliceOpenReplyTo.expectMsgType[OpenChannelResponse.Created]
}
test("recv FundingSigned with valid signature (wumbo)", Tag(ChannelStateTestsTags.Wumbo)) { f =>
@ -116,7 +117,7 @@ class WaitForFundingSignedStateSpec extends TestKitBaseClass with FixtureAnyFunS
awaitCond(alice.stateName == WAIT_FOR_FUNDING_CONFIRMED)
val watchConfirmed = alice2blockchain.expectMsgType[WatchFundingConfirmed]
assert(watchConfirmed.minDepth == 1) // when funder we trust ourselves so we never wait more than 1 block
aliceOrigin.expectMsgType[ChannelOpenResponse.ChannelOpened]
aliceOpenReplyTo.expectMsgType[OpenChannelResponse.Created]
}
test("recv FundingSigned with invalid signature") { f =>
@ -125,7 +126,7 @@ class WaitForFundingSignedStateSpec extends TestKitBaseClass with FixtureAnyFunS
alice ! FundingSigned(ByteVector32.Zeroes, ByteVector64.Zeroes)
awaitCond(alice.stateName == CLOSED)
alice2bob.expectMsgType[Error]
aliceOrigin.expectMsgType[Status.Failure]
aliceOpenReplyTo.expectMsgType[OpenChannelResponse.Rejected]
listener.expectMsgType[ChannelAborted]
}
@ -136,7 +137,7 @@ class WaitForFundingSignedStateSpec extends TestKitBaseClass with FixtureAnyFunS
alice ! c
sender.expectMsg(RES_SUCCESS(c, alice.stateData.asInstanceOf[DATA_WAIT_FOR_FUNDING_SIGNED].channelId))
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[ChannelOpenResponse.ChannelClosed]
aliceOpenReplyTo.expectMsg(OpenChannelResponse.Cancelled)
listener.expectMsgType[ChannelAborted]
}
@ -145,7 +146,7 @@ class WaitForFundingSignedStateSpec extends TestKitBaseClass with FixtureAnyFunS
val sender = TestProbe()
alice ! CMD_FORCECLOSE(sender.ref)
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[ChannelOpenResponse.ChannelClosed]
aliceOpenReplyTo.expectMsg(OpenChannelResponse.Cancelled)
listener.expectMsgType[ChannelAborted]
}
@ -156,7 +157,7 @@ class WaitForFundingSignedStateSpec extends TestKitBaseClass with FixtureAnyFunS
alice ! INPUT_DISCONNECTED
awaitCond(alice.stateName == CLOSED)
assert(alice.underlyingActor.wallet.asInstanceOf[DummyOnChainWallet].rolledback.contains(fundingTx))
aliceOrigin.expectMsgType[Status.Failure]
aliceOpenReplyTo.expectMsg(OpenChannelResponse.Disconnected)
listener.expectMsgType[ChannelAborted]
}
@ -164,7 +165,7 @@ class WaitForFundingSignedStateSpec extends TestKitBaseClass with FixtureAnyFunS
import f._
alice ! TickChannelOpenTimeout
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[Status.Failure]
aliceOpenReplyTo.expectMsg(OpenChannelResponse.TimedOut)
listener.expectMsgType[ChannelAborted]
}

View file

@ -16,6 +16,7 @@
package fr.acinq.eclair.channel.states.c
import akka.actor.typed.scaladsl.adapter.ClassicActorRefOps
import akka.testkit.{TestFSMRef, TestProbe}
import com.softwaremill.quicklens.ModifyPimp
import fr.acinq.bitcoin.scalacompat.{ByteVector32, Transaction}
@ -59,7 +60,7 @@ class WaitForChannelReadyStateSpec extends TestKitBaseClass with FixtureAnyFunSu
alice.underlying.system.eventStream.subscribe(aliceListener.ref, classOf[ChannelAborted])
bob.underlying.system.eventStream.subscribe(bobListener.ref, classOf[ChannelAborted])
alice.underlyingActor.nodeParams.db.peers.addOrUpdateRelayFees(bobParams.nodeId, relayFees)
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = false, TestConstants.feeratePerKw, TestConstants.feeratePerKw, pushMsat, requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType)
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = false, TestConstants.feeratePerKw, TestConstants.feeratePerKw, pushMsat, requireConfirmedInputs = false, 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)
bob2blockchain.expectMsgType[TxPublisher.SetChannelId]

View file

@ -16,8 +16,7 @@
package fr.acinq.eclair.channel.states.c
import akka.actor.Status
import akka.actor.typed.scaladsl.adapter.actorRefAdapter
import akka.actor.typed.scaladsl.adapter.{ClassicActorRefOps, actorRefAdapter}
import akka.testkit.{TestFSMRef, TestProbe}
import com.softwaremill.quicklens.{ModifyPimp, QuicklensAt}
import fr.acinq.bitcoin.scalacompat.{ByteVector32, SatoshiLong, Transaction}
@ -68,7 +67,7 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture
val bobContribution = if (test.tags.contains("no-funding-contribution")) None else Some(TestConstants.nonInitiatorFundingSatoshis)
val (initiatorPushAmount, nonInitiatorPushAmount) = if (test.tags.contains("both_push_amount")) (Some(TestConstants.initiatorPushAmount), Some(TestConstants.nonInitiatorPushAmount)) else (None, None)
within(30 seconds) {
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = true, TestConstants.feeratePerKw, TestConstants.feeratePerKw, initiatorPushAmount, requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType)
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = true, TestConstants.feeratePerKw, TestConstants.feeratePerKw, initiatorPushAmount, requireConfirmedInputs = false, 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)
alice2blockchain.expectMsgType[SetChannelId] // temporary channel id
bob2blockchain.expectMsgType[SetChannelId] // temporary channel id
@ -251,7 +250,7 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture
assert(alice2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTx.txid)
alice2bob.expectMsgType[TxAbort]
alice2bob.expectMsgType[ChannelReady]
probe.expectMsg(Status.Failure(InvalidRbfTxConfirmed(channelId(alice))))
assert(probe.expectMsgType[CommandFailure[_, _]].t == InvalidRbfTxConfirmed(channelId(alice)))
awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_READY)
}
@ -316,7 +315,6 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture
alice2bob.forward(bob)
assert(bob2alice.expectMsgType[TxAckRbf].fundingContribution == TestConstants.nonInitiatorFundingSatoshis)
bob2alice.forward(alice)
probe.expectMsgType[RES_SUCCESS[CMD_BUMP_FUNDING_FEE]]
// Alice and Bob build a new version of the funding transaction, with one new input every time.
val inputCount = previousFundingTxs.length + 2
@ -345,6 +343,8 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture
alice2bob.expectMsgType[TxSignatures]
alice2bob.forward(bob)
probe.expectMsgType[RES_BUMP_FUNDING_FEE]
val nextFundingTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].latestFundingTx.sharedTx.asInstanceOf[FullySignedSharedTransaction]
assert(aliceListener.expectMsgType[TransactionPublished].tx.txid == nextFundingTx.signedTx.txid)
assert(alice2blockchain.expectMsgType[WatchFundingConfirmed].txId == nextFundingTx.signedTx.txid)

View file

@ -16,6 +16,7 @@
package fr.acinq.eclair.channel.states.c
import akka.actor.typed.scaladsl.adapter.ClassicActorRefOps
import akka.testkit.{TestFSMRef, TestProbe}
import fr.acinq.bitcoin.scalacompat.{ByteVector32, Transaction}
import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher._
@ -50,7 +51,7 @@ class WaitForDualFundingReadyStateSpec extends TestKitBaseClass with FixtureAnyF
val listener = TestProbe()
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, None, requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType)
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = true, TestConstants.anchorOutputsFeeratePerKw, TestConstants.feeratePerKw, None, requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped)
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, Some(TestConstants.nonInitiatorFundingSatoshis), dualFunded = true, None, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
alice2blockchain.expectMsgType[SetChannelId] // temporary channel id
bob2blockchain.expectMsgType[SetChannelId] // temporary channel id

View file

@ -16,6 +16,7 @@
package fr.acinq.eclair.channel.states.c
import akka.actor.typed.scaladsl.adapter.ClassicActorRefOps
import akka.testkit.{TestFSMRef, TestProbe}
import fr.acinq.bitcoin.scalacompat.{ByteVector32, SatoshiLong, Script, Transaction}
import fr.acinq.eclair.blockchain.CurrentBlockHeight
@ -59,7 +60,7 @@ class WaitForFundingConfirmedStateSpec extends TestKitBaseClass with FixtureAnyF
alice.underlying.system.eventStream.subscribe(listener.ref, classOf[ChannelClosed])
bob.underlying.system.eventStream.subscribe(listener.ref, classOf[ChannelAborted])
bob.underlying.system.eventStream.subscribe(listener.ref, classOf[ChannelClosed])
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = false, TestConstants.feeratePerKw, TestConstants.feeratePerKw, Some(pushMsat), requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType)
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = false, TestConstants.feeratePerKw, TestConstants.feeratePerKw, Some(pushMsat), requireConfirmedInputs = false, 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)
bob2blockchain.expectMsgType[TxPublisher.SetChannelId]

View file

@ -16,6 +16,7 @@
package fr.acinq.eclair.channel.states.h
import akka.actor.typed.scaladsl.adapter.ClassicActorRefOps
import akka.testkit.{TestFSMRef, TestProbe}
import fr.acinq.bitcoin.ScriptFlags
import fr.acinq.bitcoin.scalacompat.Crypto.PrivateKey
@ -71,7 +72,7 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
val (aliceParams, bobParams, channelType) = computeFeatures(setup, test.tags, channelFlags)
val aliceInit = Init(aliceParams.initFeatures)
val bobInit = Init(bobParams.initFeatures)
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = false, TestConstants.feeratePerKw, TestConstants.feeratePerKw, Some(TestConstants.initiatorPushAmount), requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType)
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = false, TestConstants.feeratePerKw, TestConstants.feeratePerKw, Some(TestConstants.initiatorPushAmount), requireConfirmedInputs = false, 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)
bob2blockchain.expectMsgType[SetChannelId]

View file

@ -22,7 +22,7 @@ import com.typesafe.config.{Config, ConfigFactory}
import fr.acinq.bitcoin.scalacompat.Satoshi
import fr.acinq.eclair.Features._
import fr.acinq.eclair.blockchain.bitcoind.BitcoindService
import fr.acinq.eclair.channel._
import fr.acinq.eclair.io.Peer.OpenChannelResponse
import fr.acinq.eclair.io.{Peer, PeerConnection}
import fr.acinq.eclair.payment.relay.Relayer.RelayFees
import fr.acinq.eclair.router.Graph.WeightRatios
@ -170,7 +170,7 @@ abstract class IntegrationSpec extends TestKitBaseClass with BitcoindService wit
sender.expectMsgType[PeerConnection.ConnectionResult.HasConnection](10 seconds)
}
def connect(node1: Kit, node2: Kit, fundingAmount: Satoshi, pushMsat: MilliSatoshi): ChannelOpenResponse.ChannelOpened = {
def connect(node1: Kit, node2: Kit, fundingAmount: Satoshi, pushMsat: MilliSatoshi): OpenChannelResponse.Created = {
val sender = TestProbe()
connect(node1, node2)
sender.send(node1.switchboard, Peer.OpenChannel(
@ -181,7 +181,7 @@ abstract class IntegrationSpec extends TestKitBaseClass with BitcoindService wit
fundingTxFeerate_opt = None,
channelFlags_opt = None,
timeout_opt = None))
sender.expectMsgType[ChannelOpenResponse.ChannelOpened](10 seconds)
sender.expectMsgType[OpenChannelResponse.Created](10 seconds)
}
def getBlockHeight(): BlockHeight = {

View file

@ -13,11 +13,11 @@ import fr.acinq.eclair.blockchain.DummyOnChainWallet
import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher
import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher.{WatchFundingConfirmed, WatchFundingConfirmedTriggered, WatchFundingDeeplyBuried, WatchFundingDeeplyBuriedTriggered}
import fr.acinq.eclair.blockchain.fee.FeeratePerKw
import fr.acinq.eclair.channel.ChannelOpenResponse.ChannelOpened
import fr.acinq.eclair.channel._
import fr.acinq.eclair.channel.fsm.Channel
import fr.acinq.eclair.crypto.TransportHandler
import fr.acinq.eclair.crypto.keymanager.{LocalChannelKeyManager, LocalNodeKeyManager}
import fr.acinq.eclair.io.Peer.OpenChannelResponse
import fr.acinq.eclair.io.PeerConnection.ConnectionResult
import fr.acinq.eclair.io.{Peer, PeerConnection, PendingChannelsRateLimiter, Switchboard}
import fr.acinq.eclair.message.Postman
@ -37,7 +37,7 @@ import java.net.InetAddress
import java.util.UUID
import java.util.concurrent.atomic.AtomicLong
import scala.concurrent.duration.DurationInt
import scala.util.{Random, Success, Try}
import scala.util.Random
/**
* A minimal node setup, with real actors.
@ -177,10 +177,10 @@ object MinimalNodeFixture extends Assertions with Eventually with IntegrationPat
sender.expectMsgType[ConnectionResult.Connected]
}
def openChannel(node1: MinimalNodeFixture, node2: MinimalNodeFixture, funding: Satoshi, channelType_opt: Option[SupportedChannelType] = None)(implicit system: ActorSystem): ChannelOpened = {
def openChannel(node1: MinimalNodeFixture, node2: MinimalNodeFixture, funding: Satoshi, channelType_opt: Option[SupportedChannelType] = None)(implicit system: ActorSystem): OpenChannelResponse.Created = {
val sender = TestProbe("sender")
sender.send(node1.switchboard, Peer.OpenChannel(node2.nodeParams.nodeId, funding, channelType_opt, None, None, None, None))
sender.expectMsgType[ChannelOpened]
sender.expectMsgType[OpenChannelResponse.Created]
}
def fundingTx(node: MinimalNodeFixture, channelId: ByteVector32)(implicit system: ActorSystem): Transaction = {

View file

@ -75,7 +75,7 @@ class RustyTestsSpec extends TestKitBaseClass with Matchers with FixtureAnyFunSu
val bobInit = Init(Bob.channelParams.initFeatures)
// alice and bob will both have 1 000 000 sat
feeEstimator.setFeerate(FeeratesPerKw.single(FeeratePerKw(10000 sat)))
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, 2000000 sat, dualFunded = false, feeEstimator.getFeeratePerKw(target = 2), feeEstimator.getFeeratePerKw(target = 6), Some(1000000000 msat), requireConfirmedInputs = false, Alice.channelParams, pipe, bobInit, ChannelFlags.Private, channelConfig, channelType)
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, 2000000 sat, dualFunded = false, feeEstimator.getFeeratePerKw(target = 2), feeEstimator.getFeeratePerKw(target = 6), Some(1000000000 msat), requireConfirmedInputs = false, Alice.channelParams, pipe, bobInit, ChannelFlags.Private, 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)
bob2blockchain.expectMsgType[TxPublisher.SetChannelId]

View file

@ -16,7 +16,6 @@
package fr.acinq.eclair.io
import akka.actor.Status
import akka.actor.testkit.typed.scaladsl.{ScalaTestWithActorTestKit, TestProbe}
import akka.actor.typed.ActorRef
import akka.actor.typed.eventstream.EventStream
@ -31,7 +30,7 @@ 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.{OutgoingMessage, SpawnChannelNonInitiator}
import fr.acinq.eclair.io.Peer.{OpenChannelResponse, OutgoingMessage, SpawnChannelNonInitiator}
import fr.acinq.eclair.io.PeerSpec.createOpenChannelMessage
import fr.acinq.eclair.io.PendingChannelsRateLimiter.AddOrRejectChannel
import fr.acinq.eclair.payment.Bolt11Invoice.defaultFeatures.initFeatures
@ -55,6 +54,7 @@ class OpenChannelInterceptorSpec extends ScalaTestWithActorTestKit(ConfigFactory
val pendingChannelsRateLimiter = TestProbe[PendingChannelsRateLimiter.Command]()
val plugin = new InterceptOpenChannelPlugin {
override def name: String = "OpenChannelInterceptorPlugin"
override def openChannelInterceptor: ActorRef[InterceptOpenChannelCommand] = pluginInterceptor.ref
}
val pluginParams = TestConstants.Alice.nodeParams.pluginParams :+ plugin
@ -157,7 +157,7 @@ class OpenChannelInterceptorSpec extends ScalaTestWithActorTestKit(ConfigFactory
openChannelInterceptor ! OpenChannelInitiator(probe.ref, remoteNodeId, Peer.OpenChannel(remoteNodeId, fundingAmountBig, None, None, None, None, None), Features.empty, Features.empty)
assert(probe.expectMessageType[Status.Failure].cause.getMessage == s"fundingAmount=$fundingAmountBig is too big, you must enable large channels support in 'eclair.features' to use funding above ${Channel.MAX_FUNDING} (see eclair.conf)")
assert(probe.expectMessageType[OpenChannelResponse.Rejected].reason == s"fundingAmount=$fundingAmountBig is too big, you must enable large channels support in 'eclair.features' to use funding above ${Channel.MAX_FUNDING} (see eclair.conf)")
}
test("don't spawn a wumbo channel if remote doesn't support wumbo", Tag(ChannelStateTestsTags.Wumbo)) { f =>
@ -168,7 +168,7 @@ class OpenChannelInterceptorSpec extends ScalaTestWithActorTestKit(ConfigFactory
openChannelInterceptor ! OpenChannelInitiator(probe.ref, remoteNodeId, Peer.OpenChannel(remoteNodeId, fundingAmountBig, None, None, None, None, None), initFeatures().add(Wumbo, Optional), Features.empty)
assert(probe.expectMessageType[Status.Failure].cause.getMessage == s"fundingAmount=$fundingAmountBig is too big, the remote peer doesn't support wumbo")
assert(probe.expectMessageType[OpenChannelResponse.Rejected].reason == s"fundingAmount=$fundingAmountBig is too big, the remote peer doesn't support wumbo")
}
test("don't spawn a channel if fundingSatoshis is greater than maxFundingSatoshis", Tag(ChannelStateTestsTags.Wumbo)) { f =>
@ -179,7 +179,7 @@ class OpenChannelInterceptorSpec extends ScalaTestWithActorTestKit(ConfigFactory
openChannelInterceptor ! OpenChannelInitiator(probe.ref, remoteNodeId, Peer.OpenChannel(remoteNodeId, fundingAmountBig, None, None, None, None, None), initFeatures().add(Wumbo, Optional), initFeatures().add(Wumbo, Optional))
assert(probe.expectMessageType[Status.Failure].cause.getMessage == s"fundingAmount=$fundingAmountBig is too big for the current settings, increase 'eclair.max-funding-satoshis' (see eclair.conf)")
assert(probe.expectMessageType[OpenChannelResponse.Rejected].reason == s"fundingAmount=$fundingAmountBig is too big for the current settings, increase 'eclair.max-funding-satoshis' (see eclair.conf)")
}
test("don't spawn a channel if we don't support their channel type") { f =>

View file

@ -38,7 +38,6 @@ import scodec.bits._
import java.net.{Inet4Address, InetSocketAddress}
import scala.collection.mutable
import scala.concurrent.duration._
import scala.util.Success
class PeerConnectionSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with ParallelTestExecution {

View file

@ -82,7 +82,7 @@ class PeerSpec extends FixtureSpec {
}
case class FakeChannelFactory(channel: TestProbe) extends ChannelFactory {
override def spawn(context: ActorContext, remoteNodeId: PublicKey, origin_opt: Option[ActorRef]): ActorRef = {
override def spawn(context: ActorContext, remoteNodeId: PublicKey): ActorRef = {
assert(remoteNodeId == Bob.nodeParams.nodeId)
channel.ref
}
@ -546,18 +546,10 @@ class PeerSpec extends FixtureSpec {
import f._
val probe = TestProbe()
val channelFactory = new ChannelFactory {
override def spawn(context: ActorContext, remoteNodeId: PublicKey, origin_opt: Option[ActorRef]): ActorRef = {
assert(origin_opt.contains(probe.ref))
channel.ref
}
}
val peer = TestFSMRef(new Peer(TestConstants.Alice.nodeParams, remoteNodeId, new DummyOnChainWallet(), channelFactory, switchboard.ref, mockLimiter.toTyped))
connect(remoteNodeId, peer, peerConnection, switchboard)
probe.send(peer, Peer.OpenChannel(remoteNodeId, 15000 sat, None, Some(100 msat), None, None, None))
val init = channel.expectMsgType[INPUT_INIT_CHANNEL_INITIATOR]
assert(init.fundingAmount == 15000.sat)
assert(init.pushAmount_opt.contains(100.msat))
assert(init.replyTo == probe.ref.toTyped[OpenChannelResponse])
}
test("handle final channelId assigned in state DISCONNECTED") { f =>

View file

@ -37,7 +37,7 @@ import fr.acinq.eclair.channel.Helpers.Closing
import fr.acinq.eclair.channel._
import fr.acinq.eclair.crypto.Sphinx
import fr.acinq.eclair.db._
import fr.acinq.eclair.io.Peer.PeerInfo
import fr.acinq.eclair.io.Peer.{OpenChannelResponse, PeerInfo}
import fr.acinq.eclair.io.{NodeURI, Peer}
import fr.acinq.eclair.message.OnionMessages
import fr.acinq.eclair.payment._
@ -274,9 +274,10 @@ class ApiServiceSpec extends AnyFunSuite with ScalatestRouteTest with IdiomaticM
test("'open' channels") {
val nodeId = PublicKey(hex"030bb6a5e0c6b203c7e2180fb78c7ba4bdce46126761d8201b91ddac089cdecc87")
val channelId = ByteVector32(hex"56d7d6eda04d80138270c49709f1eadb5ab4939e5061309ccdacdb98ce637d0e")
val fundingTxId = ByteVector32(hex"a86b3f93c1b2ea3f221159869d6f556cae1ba2622cc8c7eb71c7f4f64e0fbca4")
val eclair = mock[Eclair]
eclair.open(any, any, any, any, any, any, any)(any[Timeout]) returns Future.successful(ChannelOpenResponse.ChannelOpened(channelId))
eclair.open(any, any, any, any, any, any, any)(any[Timeout]) returns Future.successful(OpenChannelResponse.Created(channelId, fundingTxId, 100 sat))
val mockService = new MockService(eclair)
Post("/open", FormData("nodeId" -> nodeId.toString(), "fundingSatoshis" -> "100002").toEntity) ~>
@ -286,7 +287,7 @@ class ApiServiceSpec extends AnyFunSuite with ScalatestRouteTest with IdiomaticM
check {
assert(handled)
assert(status == OK)
assert(entityAs[String] == "\"created channel 56d7d6eda04d80138270c49709f1eadb5ab4939e5061309ccdacdb98ce637d0e\"")
assert(entityAs[String] == "\"created channel 56d7d6eda04d80138270c49709f1eadb5ab4939e5061309ccdacdb98ce637d0e with fundingTxId=a86b3f93c1b2ea3f221159869d6f556cae1ba2622cc8c7eb71c7f4f64e0fbca4 and fees=100 sat\"")
eclair.open(nodeId, 100002 sat, None, None, None, None, None)(any[Timeout]).wasCalled(once)
}
}
@ -310,9 +311,10 @@ class ApiServiceSpec extends AnyFunSuite with ScalatestRouteTest with IdiomaticM
test("'open' channels with standard channelType") {
val nodeId = PublicKey(hex"030bb6a5e0c6b203c7e2180fb78c7ba4bdce46126761d8201b91ddac089cdecc87")
val channelId = ByteVector32(hex"56d7d6eda04d80138270c49709f1eadb5ab4939e5061309ccdacdb98ce637d0e")
val fundingTxId = ByteVector32(hex"a86b3f93c1b2ea3f221159869d6f556cae1ba2622cc8c7eb71c7f4f64e0fbca4")
val eclair = mock[Eclair]
eclair.open(any, any, any, any, any, any, any)(any[Timeout]) returns Future.successful(ChannelOpenResponse.ChannelOpened(channelId))
eclair.open(any, any, any, any, any, any, any)(any[Timeout]) returns Future.successful(OpenChannelResponse.Created(channelId, fundingTxId, 0 sat))
val mockService = new MockService(eclair)
Post("/open", FormData("nodeId" -> nodeId.toString(), "fundingSatoshis" -> "25000", "channelType" -> "standard").toEntity) ~>
@ -322,7 +324,7 @@ class ApiServiceSpec extends AnyFunSuite with ScalatestRouteTest with IdiomaticM
check {
assert(handled)
assert(status == OK)
assert(entityAs[String] == "\"created channel 56d7d6eda04d80138270c49709f1eadb5ab4939e5061309ccdacdb98ce637d0e\"")
assert(entityAs[String] == "\"created channel 56d7d6eda04d80138270c49709f1eadb5ab4939e5061309ccdacdb98ce637d0e with fundingTxId=a86b3f93c1b2ea3f221159869d6f556cae1ba2622cc8c7eb71c7f4f64e0fbca4 and fees=0 sat\"")
eclair.open(nodeId, 25000 sat, None, Some(ChannelTypes.Standard()), None, None, None)(any[Timeout]).wasCalled(once)
}
}
@ -330,9 +332,10 @@ class ApiServiceSpec extends AnyFunSuite with ScalatestRouteTest with IdiomaticM
test("'open' channels with static_remotekey channelType") {
val nodeId = PublicKey(hex"030bb6a5e0c6b203c7e2180fb78c7ba4bdce46126761d8201b91ddac089cdecc87")
val channelId = ByteVector32(hex"56d7d6eda04d80138270c49709f1eadb5ab4939e5061309ccdacdb98ce637d0e")
val fundingTxId = ByteVector32(hex"a86b3f93c1b2ea3f221159869d6f556cae1ba2622cc8c7eb71c7f4f64e0fbca4")
val eclair = mock[Eclair]
eclair.open(any, any, any, any, any, any, any)(any[Timeout]) returns Future.successful(ChannelOpenResponse.ChannelOpened(channelId))
eclair.open(any, any, any, any, any, any, any)(any[Timeout]) returns Future.successful(OpenChannelResponse.Created(channelId, fundingTxId, 1 sat))
val mockService = new MockService(eclair)
Post("/open", FormData("nodeId" -> nodeId.toString(), "fundingSatoshis" -> "25000", "channelType" -> "static_remotekey").toEntity) ~>
@ -342,7 +345,7 @@ class ApiServiceSpec extends AnyFunSuite with ScalatestRouteTest with IdiomaticM
check {
assert(handled)
assert(status == OK)
assert(entityAs[String] == "\"created channel 56d7d6eda04d80138270c49709f1eadb5ab4939e5061309ccdacdb98ce637d0e\"")
assert(entityAs[String] == "\"created channel 56d7d6eda04d80138270c49709f1eadb5ab4939e5061309ccdacdb98ce637d0e with fundingTxId=a86b3f93c1b2ea3f221159869d6f556cae1ba2622cc8c7eb71c7f4f64e0fbca4 and fees=1 sat\"")
eclair.open(nodeId, 25000 sat, None, Some(ChannelTypes.StaticRemoteKey()), None, None, None)(any[Timeout]).wasCalled(once)
}
}
@ -350,9 +353,10 @@ class ApiServiceSpec extends AnyFunSuite with ScalatestRouteTest with IdiomaticM
test("'open' channels with anchor_outputs channelType") {
val nodeId = PublicKey(hex"030bb6a5e0c6b203c7e2180fb78c7ba4bdce46126761d8201b91ddac089cdecc87")
val channelId = ByteVector32(hex"56d7d6eda04d80138270c49709f1eadb5ab4939e5061309ccdacdb98ce637d0e")
val fundingTxId = ByteVector32(hex"a86b3f93c1b2ea3f221159869d6f556cae1ba2622cc8c7eb71c7f4f64e0fbca4")
val eclair = mock[Eclair]
eclair.open(any, any, any, any, any, any, any)(any[Timeout]) returns Future.successful(ChannelOpenResponse.ChannelOpened(channelId))
eclair.open(any, any, any, any, any, any, any)(any[Timeout]) returns Future.successful(OpenChannelResponse.Created(channelId, fundingTxId, 500 sat))
val mockService = new MockService(eclair)
Post("/open", FormData("nodeId" -> nodeId.toString(), "fundingSatoshis" -> "25000", "channelType" -> "anchor_outputs").toEntity) ~>
@ -362,7 +366,7 @@ class ApiServiceSpec extends AnyFunSuite with ScalatestRouteTest with IdiomaticM
check {
assert(handled)
assert(status == OK)
assert(entityAs[String] == "\"created channel 56d7d6eda04d80138270c49709f1eadb5ab4939e5061309ccdacdb98ce637d0e\"")
assert(entityAs[String] == "\"created channel 56d7d6eda04d80138270c49709f1eadb5ab4939e5061309ccdacdb98ce637d0e with fundingTxId=a86b3f93c1b2ea3f221159869d6f556cae1ba2622cc8c7eb71c7f4f64e0fbca4 and fees=500 sat\"")
eclair.open(nodeId, 25000 sat, None, Some(ChannelTypes.AnchorOutputs()), None, None, None)(any[Timeout]).wasCalled(once)
}
@ -373,7 +377,7 @@ class ApiServiceSpec extends AnyFunSuite with ScalatestRouteTest with IdiomaticM
check {
assert(handled)
assert(status == OK)
assert(entityAs[String] == "\"created channel 56d7d6eda04d80138270c49709f1eadb5ab4939e5061309ccdacdb98ce637d0e\"")
assert(entityAs[String] == "\"created channel 56d7d6eda04d80138270c49709f1eadb5ab4939e5061309ccdacdb98ce637d0e with fundingTxId=a86b3f93c1b2ea3f221159869d6f556cae1ba2622cc8c7eb71c7f4f64e0fbca4 and fees=500 sat\"")
eclair.open(nodeId, 25000 sat, None, Some(ChannelTypes.AnchorOutputsZeroFeeHtlcTx()), None, None, None)(any[Timeout]).wasCalled(once)
}
}