1
0
Fork 0
mirror of https://github.com/ACINQ/eclair.git synced 2025-02-23 06:35:11 +01:00

Set initial relay fees during channel open (#1610)

It can be useful to override the default relay fees when opening channels
to specific nodes.

Note that these initial relay fees are not persisted in the DB. That means
that if your node reboots before the funding transaction confirms, the
channel will be opened with the default relay fees, not the overridden values.

Fixes #1507
This commit is contained in:
Bastien Teinturier 2020-12-08 09:51:00 +01:00 committed by GitHub
parent 040e6d08dc
commit a4d184569b
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
24 changed files with 159 additions and 56 deletions

View file

@ -84,7 +84,7 @@ trait Eclair {
def disconnect(nodeId: PublicKey)(implicit timeout: Timeout): Future[String]
def open(nodeId: PublicKey, fundingAmount: Satoshi, pushAmount_opt: Option[MilliSatoshi], fundingFeeratePerByte_opt: Option[FeeratePerByte], flags_opt: Option[Int], openTimeout_opt: Option[Timeout])(implicit timeout: Timeout): Future[ChannelOpenResponse]
def open(nodeId: PublicKey, fundingAmount: Satoshi, pushAmount_opt: Option[MilliSatoshi], fundingFeeratePerByte_opt: Option[FeeratePerByte], initialRelayFees_opt: Option[(MilliSatoshi, Int)], flags_opt: Option[Int], openTimeout_opt: Option[Timeout])(implicit timeout: Timeout): Future[ChannelOpenResponse]
def close(channels: List[ApiTypes.ChannelIdentifier], scriptPubKey_opt: Option[ByteVector])(implicit timeout: Timeout): Future[Map[ApiTypes.ChannelIdentifier, Either[Throwable, CommandResponse[CMD_CLOSE]]]]
@ -165,7 +165,7 @@ class EclairImpl(appKit: Kit) extends Eclair {
(appKit.switchboard ? Peer.Disconnect(nodeId)).mapTo[String]
}
override def open(nodeId: PublicKey, fundingAmount: Satoshi, pushAmount_opt: Option[MilliSatoshi], fundingFeeratePerByte_opt: Option[FeeratePerByte], flags_opt: Option[Int], openTimeout_opt: Option[Timeout])(implicit timeout: Timeout): Future[ChannelOpenResponse] = {
override def open(nodeId: PublicKey, fundingAmount: Satoshi, pushAmount_opt: Option[MilliSatoshi], fundingFeeratePerByte_opt: Option[FeeratePerByte], initialRelayFees_opt: Option[(MilliSatoshi, Int)], flags_opt: Option[Int], openTimeout_opt: Option[Timeout])(implicit timeout: Timeout): Future[ChannelOpenResponse] = {
// we want the open timeout to expire *before* the default ask timeout, otherwise user won't get a generic response
val openTimeout = openTimeout_opt.getOrElse(Timeout(10 seconds))
(appKit.switchboard ? Peer.OpenChannel(
@ -173,6 +173,7 @@ class EclairImpl(appKit: Kit) extends Eclair {
fundingSatoshis = fundingAmount,
pushMsat = pushAmount_opt.getOrElse(0 msat),
fundingTxFeeratePerKw_opt = fundingFeeratePerByte_opt.map(FeeratePerKw(_)),
initialRelayFees_opt = initialRelayFees_opt,
channelFlags = flags_opt.map(_.toByte),
timeout_opt = Some(openTimeout))).mapTo[ChannelOpenResponse]
}

View file

@ -161,7 +161,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
startWith(WAIT_FOR_INIT_INTERNAL, Nothing)
when(WAIT_FOR_INIT_INTERNAL)(handleExceptions {
case Event(initFunder@INPUT_INIT_FUNDER(temporaryChannelId, fundingSatoshis, pushMsat, initialFeeratePerKw, fundingTxFeeratePerKw, localParams, remote, _, channelFlags, channelVersion), Nothing) =>
case Event(initFunder@INPUT_INIT_FUNDER(temporaryChannelId, fundingSatoshis, pushMsat, initialFeeratePerKw, fundingTxFeeratePerKw, initialRelayFees_opt, localParams, remote, _, channelFlags, channelVersion), Nothing) =>
context.system.eventStream.publish(ChannelCreated(self, peer, remoteNodeId, isFunder = true, temporaryChannelId, initialFeeratePerKw, Some(fundingTxFeeratePerKw)))
activeConnection = remote
val fundingPubKey = keyManager.fundingPublicKey(localParams.fundingKeyPath).publicKey
@ -330,7 +330,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
htlcBasepoint = open.htlcBasepoint,
features = remoteInit.features)
log.debug("remote params: {}", remoteParams)
goto(WAIT_FOR_FUNDING_CREATED) using DATA_WAIT_FOR_FUNDING_CREATED(open.temporaryChannelId, localParams, remoteParams, open.fundingSatoshis, open.pushMsat, open.feeratePerKw, open.firstPerCommitmentPoint, open.channelFlags, channelVersion, accept) sending accept
goto(WAIT_FOR_FUNDING_CREATED) using DATA_WAIT_FOR_FUNDING_CREATED(open.temporaryChannelId, localParams, remoteParams, open.fundingSatoshis, open.pushMsat, open.feeratePerKw, None, open.firstPerCommitmentPoint, open.channelFlags, channelVersion, accept) sending accept
}
case Event(c: CloseCommand, d) => handleFastClose(c, d.channelId)
@ -341,7 +341,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
})
when(WAIT_FOR_ACCEPT_CHANNEL)(handleExceptions {
case Event(accept: AcceptChannel, d@DATA_WAIT_FOR_ACCEPT_CHANNEL(INPUT_INIT_FUNDER(temporaryChannelId, fundingSatoshis, pushMsat, initialFeeratePerKw, fundingTxFeeratePerKw, localParams, _, remoteInit, _, channelVersion), open)) =>
case Event(accept: AcceptChannel, d@DATA_WAIT_FOR_ACCEPT_CHANNEL(INPUT_INIT_FUNDER(temporaryChannelId, fundingSatoshis, pushMsat, initialFeeratePerKw, fundingTxFeeratePerKw, initialRelayFees_opt, localParams, _, remoteInit, _, channelVersion), open)) =>
log.info(s"received AcceptChannel=$accept")
Try(Helpers.validateParamsFunder(nodeParams, open, accept)) match {
case Failure(t) => handleLocalError(t, d, Some(accept))
@ -365,7 +365,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
val localFundingPubkey = keyManager.fundingPublicKey(localParams.fundingKeyPath)
val fundingPubkeyScript = Script.write(Script.pay2wsh(Scripts.multiSig2of2(localFundingPubkey.publicKey, remoteParams.fundingPubKey)))
wallet.makeFundingTx(fundingPubkeyScript, fundingSatoshis, fundingTxFeeratePerKw).pipeTo(self)
goto(WAIT_FOR_FUNDING_INTERNAL) using DATA_WAIT_FOR_FUNDING_INTERNAL(temporaryChannelId, localParams, remoteParams, fundingSatoshis, pushMsat, initialFeeratePerKw, accept.firstPerCommitmentPoint, channelVersion, open)
goto(WAIT_FOR_FUNDING_INTERNAL) using DATA_WAIT_FOR_FUNDING_INTERNAL(temporaryChannelId, localParams, remoteParams, fundingSatoshis, pushMsat, initialFeeratePerKw, initialRelayFees_opt, accept.firstPerCommitmentPoint, channelVersion, open)
}
case Event(c: CloseCommand, d: DATA_WAIT_FOR_ACCEPT_CHANNEL) =>
@ -386,7 +386,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
})
when(WAIT_FOR_FUNDING_INTERNAL)(handleExceptions {
case Event(MakeFundingTxResponse(fundingTx, fundingTxOutputIndex, fundingTxFee), DATA_WAIT_FOR_FUNDING_INTERNAL(temporaryChannelId, localParams, remoteParams, fundingAmount, pushMsat, initialFeeratePerKw, remoteFirstPerCommitmentPoint, channelVersion, open)) =>
case Event(MakeFundingTxResponse(fundingTx, fundingTxOutputIndex, fundingTxFee), DATA_WAIT_FOR_FUNDING_INTERNAL(temporaryChannelId, localParams, remoteParams, fundingAmount, pushMsat, initialFeeratePerKw, initialRelayFees_opt, remoteFirstPerCommitmentPoint, channelVersion, open)) =>
// let's create the first commitment tx that spends the yet uncommitted funding tx
val (localSpec, localCommitTx, remoteSpec, remoteCommitTx) = Funding.makeFirstCommitTxs(keyManager, channelVersion, temporaryChannelId, localParams, remoteParams, fundingAmount, pushMsat, initialFeeratePerKw, fundingTx.hash, fundingTxOutputIndex, remoteFirstPerCommitmentPoint)
require(fundingTx.txOut(fundingTxOutputIndex).publicKeyScript == localCommitTx.input.txOut.publicKeyScript, s"pubkey script mismatch!")
@ -402,7 +402,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
peer ! ChannelIdAssigned(self, remoteNodeId, temporaryChannelId, channelId) // we notify the peer asap so it knows how to route messages
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(channelId, localParams, remoteParams, fundingTx, fundingTxFee, localSpec, localCommitTx, RemoteCommit(0, remoteSpec, remoteCommitTx.tx.txid, remoteFirstPerCommitmentPoint), open.channelFlags, channelVersion, fundingCreated) sending fundingCreated
goto(WAIT_FOR_FUNDING_SIGNED) using DATA_WAIT_FOR_FUNDING_SIGNED(channelId, localParams, remoteParams, fundingTx, fundingTxFee, initialRelayFees_opt, localSpec, localCommitTx, RemoteCommit(0, remoteSpec, remoteCommitTx.tx.txid, remoteFirstPerCommitmentPoint), open.channelFlags, channelVersion, fundingCreated) sending fundingCreated
case Event(Status.Failure(t), d: DATA_WAIT_FOR_FUNDING_INTERNAL) =>
log.error(t, s"wallet returned error: ")
@ -427,7 +427,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
})
when(WAIT_FOR_FUNDING_CREATED)(handleExceptions {
case Event(FundingCreated(_, fundingTxHash, fundingTxOutputIndex, remoteSig), d@DATA_WAIT_FOR_FUNDING_CREATED(temporaryChannelId, localParams, remoteParams, fundingAmount, pushMsat, initialFeeratePerKw, remoteFirstPerCommitmentPoint, channelFlags, channelVersion, _)) =>
case Event(FundingCreated(_, fundingTxHash, fundingTxOutputIndex, remoteSig), d@DATA_WAIT_FOR_FUNDING_CREATED(temporaryChannelId, localParams, remoteParams, fundingAmount, pushMsat, initialFeeratePerKw, initialRelayFees_opt, remoteFirstPerCommitmentPoint, channelFlags, channelVersion, _)) =>
// they fund the channel with their funding tx, so the money is theirs (but we are paid pushMsat)
val (localSpec, localCommitTx, remoteSpec, remoteCommitTx) = Funding.makeFirstCommitTxs(keyManager, channelVersion, temporaryChannelId, localParams, remoteParams, fundingAmount, pushMsat, initialFeeratePerKw, fundingTxHash, fundingTxOutputIndex, remoteFirstPerCommitmentPoint)
@ -462,7 +462,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
blockchain ! WatchSpent(self, commitInput.outPoint.txid, commitInput.outPoint.index.toInt, commitInput.txOut.publicKeyScript, BITCOIN_FUNDING_SPENT) // TODO: should we wait for an acknowledgment from the watcher?
blockchain ! WatchConfirmed(self, commitInput.outPoint.txid, commitInput.txOut.publicKeyScript, fundingMinDepth, BITCOIN_FUNDING_DEPTHOK)
context.system.scheduler.scheduleOnce(FUNDING_TIMEOUT_FUNDEE, self, BITCOIN_FUNDING_TIMEOUT)
goto(WAIT_FOR_FUNDING_CONFIRMED) using DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments, None, now, None, Right(fundingSigned)) storing() sending fundingSigned
goto(WAIT_FOR_FUNDING_CONFIRMED) using DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments, None, initialRelayFees_opt, now, None, Right(fundingSigned)) storing() sending fundingSigned
}
case Event(c: CloseCommand, d: DATA_WAIT_FOR_FUNDING_CREATED) =>
@ -475,7 +475,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
})
when(WAIT_FOR_FUNDING_SIGNED)(handleExceptions {
case Event(msg@FundingSigned(_, remoteSig), d@DATA_WAIT_FOR_FUNDING_SIGNED(channelId, localParams, remoteParams, fundingTx, fundingTxFee, localSpec, localCommitTx, remoteCommit, channelFlags, channelVersion, fundingCreated)) =>
case Event(msg@FundingSigned(_, remoteSig), d@DATA_WAIT_FOR_FUNDING_SIGNED(channelId, localParams, remoteParams, fundingTx, fundingTxFee, initialRelayFees_opt, localSpec, localCommitTx, remoteCommit, channelFlags, channelVersion, fundingCreated)) =>
// we make sure that their sig checks out and that our first commit tx is spendable
val fundingPubKey = keyManager.fundingPublicKey(localParams.fundingKeyPath)
val localSigOfLocalTx = keyManager.sign(localCommitTx, fundingPubKey, TxOwner.Local, channelVersion.commitmentFormat)
@ -519,7 +519,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
}
}
goto(WAIT_FOR_FUNDING_CONFIRMED) using DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments, Some(fundingTx), now, None, Left(fundingCreated)) storing() calling (publishFundingTx)
goto(WAIT_FOR_FUNDING_CONFIRMED) using DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments, Some(fundingTx), initialRelayFees_opt, now, None, Left(fundingCreated)) storing() calling publishFundingTx()
}
case Event(c: CloseCommand, d: DATA_WAIT_FOR_FUNDING_SIGNED) =>
@ -552,7 +552,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
log.info(s"received their FundingLocked, deferring message")
stay using d.copy(deferred = Some(msg)) // no need to store, they will re-send if we get disconnected
case Event(WatchEventConfirmed(BITCOIN_FUNDING_DEPTHOK, blockHeight, txIndex, fundingTx), d@DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments, _, _, deferred, _)) =>
case Event(WatchEventConfirmed(BITCOIN_FUNDING_DEPTHOK, blockHeight, txIndex, fundingTx), d@DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments, _, initialRelayFees_opt, _, deferred, _)) =>
Try(Transaction.correctlySpends(commitments.localCommit.publishableTxs.commitTx.tx, Seq(fundingTx), ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)) match {
case Success(_) =>
log.info(s"channelId=${commitments.channelId} was confirmed at blockHeight=$blockHeight txIndex=$txIndex")
@ -565,7 +565,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
// as soon as it reaches NORMAL state, and before it is announced on the network
// (this id might be updated when the funding tx gets deeply buried, if there was a reorg in the meantime)
val shortChannelId = ShortChannelId(blockHeight, txIndex, commitments.commitInput.outPoint.index.toInt)
goto(WAIT_FOR_FUNDING_LOCKED) using DATA_WAIT_FOR_FUNDING_LOCKED(commitments, shortChannelId, fundingLocked) storing() sending fundingLocked
goto(WAIT_FOR_FUNDING_LOCKED) using DATA_WAIT_FOR_FUNDING_LOCKED(commitments, shortChannelId, fundingLocked, initialRelayFees_opt) storing() sending fundingLocked
case Failure(t) =>
log.error(t, s"rejecting channel with invalid funding tx: ${fundingTx.bin}")
goto(CLOSED)
@ -592,12 +592,13 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
})
when(WAIT_FOR_FUNDING_LOCKED)(handleExceptions {
case Event(FundingLocked(_, nextPerCommitmentPoint), d@DATA_WAIT_FOR_FUNDING_LOCKED(commitments, shortChannelId, _)) =>
case Event(FundingLocked(_, nextPerCommitmentPoint), d@DATA_WAIT_FOR_FUNDING_LOCKED(commitments, shortChannelId, _, initialRelayFees_opt)) =>
// used to get the final shortChannelId, used in announcements (if minDepth >= ANNOUNCEMENTS_MINCONF this event will fire instantly)
blockchain ! WatchConfirmed(self, commitments.commitInput.outPoint.txid, commitments.commitInput.txOut.publicKeyScript, ANNOUNCEMENTS_MINCONF, BITCOIN_FUNDING_DEEPLYBURIED)
context.system.eventStream.publish(ShortChannelIdAssigned(self, commitments.channelId, shortChannelId, None))
// we create a channel_update early so that we can use it to send payments through this channel, but it won't be propagated to other nodes since the channel is not yet announced
val initialChannelUpdate = Announcements.makeChannelUpdate(nodeParams.chainHash, nodeParams.privateKey, remoteNodeId, shortChannelId, nodeParams.expiryDelta, d.commitments.remoteParams.htlcMinimum, nodeParams.feeBase, nodeParams.feeProportionalMillionth, commitments.capacity.toMilliSatoshi, enable = Helpers.aboveReserve(d.commitments))
val (feeBase, feeProportionalMillionths) = initialRelayFees_opt.getOrElse((nodeParams.feeBase, nodeParams.feeProportionalMillionth))
val initialChannelUpdate = Announcements.makeChannelUpdate(nodeParams.chainHash, nodeParams.privateKey, remoteNodeId, shortChannelId, nodeParams.expiryDelta, d.commitments.remoteParams.htlcMinimum, feeBase, feeProportionalMillionths, commitments.capacity.toMilliSatoshi, enable = Helpers.aboveReserve(d.commitments))
// we need to periodically re-send channel updates, otherwise channel will be considered stale and get pruned by network
context.system.scheduler.schedule(initialDelay = REFRESH_CHANNEL_UPDATE_INTERVAL, interval = REFRESH_CHANNEL_UPDATE_INTERVAL, receiver = self, message = BroadcastChannelUpdate(PeriodicRefresh))
goto(NORMAL) using DATA_NORMAL(commitments.copy(remoteNextCommitInfo = Right(nextPerCommitmentPoint)), shortChannelId, buried = false, None, initialChannelUpdate, None, None) storing()

View file

@ -77,7 +77,17 @@ case object ERR_INFORMATION_LEAK extends State
8888888888 Y8P 8888888888 888 Y888 888 "Y8888P"
*/
case class INPUT_INIT_FUNDER(temporaryChannelId: ByteVector32, fundingAmount: Satoshi, pushAmount: MilliSatoshi, initialFeeratePerKw: FeeratePerKw, fundingTxFeeratePerKw: FeeratePerKw, localParams: LocalParams, remote: ActorRef, remoteInit: Init, channelFlags: Byte, channelVersion: ChannelVersion)
case class INPUT_INIT_FUNDER(temporaryChannelId: ByteVector32,
fundingAmount: Satoshi,
pushAmount: MilliSatoshi,
initialFeeratePerKw: FeeratePerKw,
fundingTxFeeratePerKw: FeeratePerKw,
initialRelayFees_opt: Option[(MilliSatoshi, Int)],
localParams: LocalParams,
remote: ActorRef,
remoteInit: Init,
channelFlags: Byte,
channelVersion: ChannelVersion)
case class INPUT_INIT_FUNDEE(temporaryChannelId: ByteVector32, localParams: LocalParams, remote: ActorRef, remoteInit: Init, channelVersion: ChannelVersion)
case object INPUT_CLOSE_COMPLETE_TIMEOUT // when requesting a mutual close, we wait for as much as this timeout, then unilateral close
case object INPUT_DISCONNECTED
@ -269,19 +279,50 @@ final case class DATA_WAIT_FOR_OPEN_CHANNEL(initFundee: INPUT_INIT_FUNDEE) exten
final case class DATA_WAIT_FOR_ACCEPT_CHANNEL(initFunder: INPUT_INIT_FUNDER, lastSent: OpenChannel) extends Data {
val channelId: ByteVector32 = initFunder.temporaryChannelId
}
final case class DATA_WAIT_FOR_FUNDING_INTERNAL(temporaryChannelId: ByteVector32, localParams: LocalParams, remoteParams: RemoteParams, fundingAmount: Satoshi, pushAmount: MilliSatoshi, initialFeeratePerKw: FeeratePerKw, remoteFirstPerCommitmentPoint: PublicKey, channelVersion: ChannelVersion, lastSent: OpenChannel) extends Data {
final case class DATA_WAIT_FOR_FUNDING_INTERNAL(temporaryChannelId: ByteVector32,
localParams: LocalParams,
remoteParams: RemoteParams,
fundingAmount: Satoshi,
pushAmount: MilliSatoshi,
initialFeeratePerKw: FeeratePerKw,
initialRelayFees_opt: Option[(MilliSatoshi, Int)],
remoteFirstPerCommitmentPoint: PublicKey,
channelVersion: ChannelVersion,
lastSent: OpenChannel) extends Data {
val channelId: ByteVector32 = temporaryChannelId
}
final case class DATA_WAIT_FOR_FUNDING_CREATED(temporaryChannelId: ByteVector32, localParams: LocalParams, remoteParams: RemoteParams, fundingAmount: Satoshi, pushAmount: MilliSatoshi, initialFeeratePerKw: FeeratePerKw, remoteFirstPerCommitmentPoint: PublicKey, channelFlags: Byte, channelVersion: ChannelVersion, lastSent: AcceptChannel) extends Data {
final case class DATA_WAIT_FOR_FUNDING_CREATED(temporaryChannelId: ByteVector32,
localParams: LocalParams,
remoteParams: RemoteParams,
fundingAmount: Satoshi,
pushAmount: MilliSatoshi,
initialFeeratePerKw: FeeratePerKw,
initialRelayFees_opt: Option[(MilliSatoshi, Int)],
remoteFirstPerCommitmentPoint: PublicKey,
channelFlags: Byte,
channelVersion: ChannelVersion,
lastSent: AcceptChannel) extends Data {
val channelId: ByteVector32 = temporaryChannelId
}
final case class DATA_WAIT_FOR_FUNDING_SIGNED(channelId: ByteVector32, localParams: LocalParams, remoteParams: RemoteParams, fundingTx: Transaction, fundingTxFee: Satoshi, localSpec: CommitmentSpec, localCommitTx: CommitTx, remoteCommit: RemoteCommit, channelFlags: Byte, channelVersion: ChannelVersion, lastSent: FundingCreated) extends Data
final case class DATA_WAIT_FOR_FUNDING_SIGNED(channelId: ByteVector32,
localParams: LocalParams,
remoteParams: RemoteParams,
fundingTx: Transaction,
fundingTxFee: Satoshi,
initialRelayFees_opt: Option[(MilliSatoshi, Int)],
localSpec: CommitmentSpec,
localCommitTx: CommitTx,
remoteCommit: RemoteCommit,
channelFlags: Byte,
channelVersion: ChannelVersion,
lastSent: FundingCreated) extends Data
final case class DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments: Commitments,
fundingTx: Option[Transaction],
initialRelayFees_opt: Option[(MilliSatoshi, Int)],
waitingSince: Long, // how long have we been waiting for the funding tx to confirm
deferred: Option[FundingLocked],
lastSent: Either[FundingCreated, FundingSigned]) extends Data with HasCommitments
final case class DATA_WAIT_FOR_FUNDING_LOCKED(commitments: Commitments, shortChannelId: ShortChannelId, lastSent: FundingLocked) extends Data with HasCommitments
final case class DATA_WAIT_FOR_FUNDING_LOCKED(commitments: Commitments, shortChannelId: ShortChannelId, lastSent: FundingLocked, initialRelayFees_opt: Option[(MilliSatoshi, Int)]) extends Data with HasCommitments
final case class DATA_NORMAL(commitments: Commitments,
shortChannelId: ShortChannelId,
buried: Boolean,

View file

@ -127,7 +127,7 @@ class Peer(val nodeParams: NodeParams, remoteNodeId: PublicKey, watcher: ActorRe
val channelFeeratePerKw = nodeParams.onChainFeeConf.feeEstimator.getFeeratePerKw(target = nodeParams.onChainFeeConf.feeTargets.commitmentBlockTarget)
val fundingTxFeeratePerKw = c.fundingTxFeeratePerKw_opt.getOrElse(nodeParams.onChainFeeConf.feeEstimator.getFeeratePerKw(target = nodeParams.onChainFeeConf.feeTargets.fundingBlockTarget))
log.info(s"requesting a new channel with fundingSatoshis=${c.fundingSatoshis}, pushMsat=${c.pushMsat} and fundingFeeratePerByte=${c.fundingTxFeeratePerKw_opt} temporaryChannelId=$temporaryChannelId localParams=$localParams")
channel ! INPUT_INIT_FUNDER(temporaryChannelId, c.fundingSatoshis, c.pushMsat, channelFeeratePerKw, fundingTxFeeratePerKw, localParams, d.peerConnection, d.remoteInit, c.channelFlags.getOrElse(nodeParams.channelFlags), channelVersion)
channel ! INPUT_INIT_FUNDER(temporaryChannelId, c.fundingSatoshis, c.pushMsat, channelFeeratePerKw, fundingTxFeeratePerKw, c.initialRelayFees_opt, localParams, d.peerConnection, d.remoteInit, c.channelFlags.getOrElse(nodeParams.channelFlags), channelVersion)
stay using d.copy(channels = d.channels + (TemporaryChannelId(temporaryChannelId) -> channel))
}
@ -381,7 +381,7 @@ object Peer {
}
case class Disconnect(nodeId: PublicKey)
case class OpenChannel(remoteNodeId: PublicKey, fundingSatoshis: Satoshi, pushMsat: MilliSatoshi, fundingTxFeeratePerKw_opt: Option[FeeratePerKw], channelFlags: Option[Byte], timeout_opt: Option[Timeout]) {
case class OpenChannel(remoteNodeId: PublicKey, fundingSatoshis: Satoshi, pushMsat: MilliSatoshi, fundingTxFeeratePerKw_opt: Option[FeeratePerKw], initialRelayFees_opt: Option[(MilliSatoshi, Int)], channelFlags: Option[Byte], timeout_opt: Option[Timeout]) {
require(pushMsat <= fundingSatoshis, s"pushMsat must be less or equal to fundingSatoshis")
require(fundingSatoshis >= 0.sat, s"fundingSatoshis must be positive")
require(pushMsat >= 0.msat, s"pushMsat must be positive")

View file

@ -18,6 +18,7 @@ package fr.acinq.eclair.wire
import fr.acinq.bitcoin.DeterministicWallet.{ExtendedPrivateKey, KeyPath}
import fr.acinq.bitcoin.{ByteVector32, OutPoint, Transaction, TxOut}
import fr.acinq.eclair.MilliSatoshi
import fr.acinq.eclair.channel._
import fr.acinq.eclair.crypto.ShaChain
import fr.acinq.eclair.transactions.Transactions._
@ -227,6 +228,7 @@ object ChannelCodecs extends Logging {
val DATA_WAIT_FOR_FUNDING_CONFIRMED_Codec: Codec[DATA_WAIT_FOR_FUNDING_CONFIRMED] = (
("commitments" | commitmentsCodec) ::
("fundingTx" | optional(bool8, txCodec)) ::
("initialRelayFees" | provide(Option.empty[(MilliSatoshi, Int)])) ::
("waitingSince" | int64) ::
("deferred" | optional(bool8, lengthDelimited(fundingLockedCodec))) ::
("lastSent" | either(bool8, lengthDelimited(fundingCreatedCodec), lengthDelimited(fundingSignedCodec)))).as[DATA_WAIT_FOR_FUNDING_CONFIRMED]
@ -234,7 +236,8 @@ object ChannelCodecs extends Logging {
val DATA_WAIT_FOR_FUNDING_LOCKED_Codec: Codec[DATA_WAIT_FOR_FUNDING_LOCKED] = (
("commitments" | commitmentsCodec) ::
("shortChannelId" | shortchannelid) ::
("lastSent" | lengthDelimited(fundingLockedCodec))).as[DATA_WAIT_FOR_FUNDING_LOCKED]
("lastSent" | lengthDelimited(fundingLockedCodec)) ::
("initialRelayFees" | provide(Option.empty[(MilliSatoshi, Int)]))).as[DATA_WAIT_FOR_FUNDING_LOCKED]
val DATA_NORMAL_Codec: Codec[DATA_NORMAL] = (
("commitments" | commitmentsCodec) ::

View file

@ -20,6 +20,7 @@ import java.util.UUID
import fr.acinq.bitcoin.DeterministicWallet.{ExtendedPrivateKey, KeyPath}
import fr.acinq.bitcoin.{ByteVector32, ByteVector64, Crypto, OutPoint, Transaction, TxOut}
import fr.acinq.eclair.MilliSatoshi
import fr.acinq.eclair.channel._
import fr.acinq.eclair.crypto.ShaChain
import fr.acinq.eclair.transactions.Transactions._
@ -260,6 +261,7 @@ private[wire] object LegacyChannelCodecs extends Logging {
val DATA_WAIT_FOR_FUNDING_CONFIRMED_COMPAT_01_Codec: Codec[DATA_WAIT_FOR_FUNDING_CONFIRMED] = (
("commitments" | commitmentsCodec) ::
("fundingTx" | provide[Option[Transaction]](None)) ::
("initialRelayFees" | provide(Option.empty[(MilliSatoshi, Int)])) ::
("waitingSince" | provide(System.currentTimeMillis.milliseconds.toSeconds)) ::
("deferred" | optional(bool, fundingLockedCodec)) ::
("lastSent" | either(bool, fundingCreatedCodec, fundingSignedCodec))).as[DATA_WAIT_FOR_FUNDING_CONFIRMED].decodeOnly
@ -267,6 +269,7 @@ private[wire] object LegacyChannelCodecs extends Logging {
val DATA_WAIT_FOR_FUNDING_CONFIRMED_Codec: Codec[DATA_WAIT_FOR_FUNDING_CONFIRMED] = (
("commitments" | commitmentsCodec) ::
("fundingTx" | optional(bool, txCodec)) ::
("initialRelayFees" | provide(Option.empty[(MilliSatoshi, Int)])) ::
("waitingSince" | int64) ::
("deferred" | optional(bool, fundingLockedCodec)) ::
("lastSent" | either(bool, fundingCreatedCodec, fundingSignedCodec))).as[DATA_WAIT_FOR_FUNDING_CONFIRMED].decodeOnly
@ -274,7 +277,8 @@ private[wire] object LegacyChannelCodecs extends Logging {
val DATA_WAIT_FOR_FUNDING_LOCKED_Codec: Codec[DATA_WAIT_FOR_FUNDING_LOCKED] = (
("commitments" | commitmentsCodec) ::
("shortChannelId" | shortchannelid) ::
("lastSent" | fundingLockedCodec)).as[DATA_WAIT_FOR_FUNDING_LOCKED].decodeOnly
("lastSent" | fundingLockedCodec) ::
("initialRelayFees" | provide(Option.empty[(MilliSatoshi, Int)]))).as[DATA_WAIT_FOR_FUNDING_LOCKED].decodeOnly
// All channel_announcement's written prior to supporting unknown trailing fields had the same fixed size, because
// those are the announcements that *we* created and we always used an empty features field, which was the only

View file

@ -86,12 +86,12 @@ class EclairImplSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with I
val nodeId = PublicKey(hex"030bb6a5e0c6b203c7e2180fb78c7ba4bdce46126761d8201b91ddac089cdecc87")
// standard conversion
eclair.open(nodeId, fundingAmount = 10000000L sat, pushAmount_opt = None, fundingFeeratePerByte_opt = Some(FeeratePerByte(5 sat)), flags_opt = None, openTimeout_opt = None)
eclair.open(nodeId, fundingAmount = 10000000L sat, pushAmount_opt = None, fundingFeeratePerByte_opt = Some(FeeratePerByte(5 sat)), initialRelayFees_opt = None, flags_opt = None, openTimeout_opt = None)
val open = switchboard.expectMsgType[OpenChannel]
assert(open.fundingTxFeeratePerKw_opt === Some(FeeratePerKw(1250 sat)))
// check that minimum fee rate of 253 sat/bw is used
eclair.open(nodeId, fundingAmount = 10000000L sat, pushAmount_opt = None, fundingFeeratePerByte_opt = Some(FeeratePerByte(1 sat)), flags_opt = None, openTimeout_opt = None)
eclair.open(nodeId, fundingAmount = 10000000L sat, pushAmount_opt = None, fundingFeeratePerByte_opt = Some(FeeratePerByte(1 sat)), initialRelayFees_opt = None, flags_opt = None, openTimeout_opt = None)
val open1 = switchboard.expectMsgType[OpenChannel]
assert(open1.fundingTxFeeratePerKw_opt === Some(FeeratePerKw.MinimumFeeratePerKw))
}

View file

@ -77,7 +77,7 @@ class FuzzySpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with StateT
registerA ! alice
registerB ! bob
// no announcements
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, TestConstants.fundingSatoshis, TestConstants.pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, Alice.channelParams, pipe, bobInit, channelFlags = 0x00.toByte, ChannelVersion.STANDARD)
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, TestConstants.fundingSatoshis, TestConstants.pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, None, Alice.channelParams, pipe, bobInit, channelFlags = 0x00.toByte, ChannelVersion.STANDARD)
bob ! INPUT_INIT_FUNDEE(ByteVector32.Zeroes, Bob.channelParams, pipe, aliceInit, ChannelVersion.STANDARD)
pipe ! (alice, bob)
alice2blockchain.expectMsgType[WatchSpent]

View file

@ -41,7 +41,6 @@ class ThroughputSpec extends AnyFunSuite {
val blockCount = new AtomicLong()
val blockchain = system.actorOf(ZmqWatcher.props(randomBytes32, blockCount, new TestBitcoinClient()), "blockchain")
val paymentHandler = system.actorOf(Props(new Actor() {
val random = new Random()
context.become(run(Map()))
@ -71,7 +70,7 @@ class ThroughputSpec extends AnyFunSuite {
val bob = system.actorOf(Channel.props(Bob.nodeParams, wallet, Alice.nodeParams.nodeId, blockchain, relayerB, None), "b")
val aliceInit = Init(Alice.channelParams.features)
val bobInit = Init(Bob.channelParams.features)
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, TestConstants.fundingSatoshis, TestConstants.pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, Alice.channelParams, pipe, bobInit, ChannelFlags.Empty, ChannelVersion.STANDARD)
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, TestConstants.fundingSatoshis, TestConstants.pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, None, Alice.channelParams, pipe, bobInit, ChannelFlags.Empty, ChannelVersion.STANDARD)
bob ! INPUT_INIT_FUNDEE(ByteVector32.Zeroes, Bob.channelParams, pipe, aliceInit, ChannelVersion.STANDARD)
val latch = new CountDownLatch(2)
@ -85,7 +84,6 @@ class ThroughputSpec extends AnyFunSuite {
pipe ! (alice, bob)
latch.await()
var i = new AtomicLong(0)
val random = new Random()
def msg = random.nextInt(100) % 5 match {

View file

@ -94,7 +94,7 @@ trait StateTestsHelperMethods extends TestKitBase with FixtureTestSuite with Par
val aliceInit = Init(aliceParams.features)
val bobInit = Init(bobParams.features)
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, TestConstants.fundingSatoshis, pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, aliceParams, alice2bob.ref, bobInit, channelFlags, channelVersion)
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, TestConstants.fundingSatoshis, pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, None, aliceParams, alice2bob.ref, bobInit, channelFlags, channelVersion)
bob ! INPUT_INIT_FUNDEE(ByteVector32.Zeroes, bobParams, bob2alice.ref, aliceInit, channelVersion)
alice2bob.expectMsgType[OpenChannel]
alice2bob.forward(bob)

View file

@ -69,7 +69,7 @@ class WaitForAcceptChannelStateSpec extends TestKitBaseClass with FixtureAnyFunS
val bobInit = Init(bobParams.features)
within(30 seconds) {
val fundingAmount = if (test.tags.contains("wumbo")) Btc(5).toSatoshi else TestConstants.fundingSatoshis
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, fundingAmount, TestConstants.pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, aliceParams, alice2bob.ref, bobInit, ChannelFlags.Empty, channelVersion)
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, fundingAmount, TestConstants.pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, None, aliceParams, alice2bob.ref, bobInit, ChannelFlags.Empty, channelVersion)
bob ! INPUT_INIT_FUNDEE(ByteVector32.Zeroes, bobParams, bob2alice.ref, aliceInit, channelVersion)
alice2bob.expectMsgType[OpenChannel]
alice2bob.forward(bob)

View file

@ -56,7 +56,7 @@ class WaitForOpenChannelStateSpec extends TestKitBaseClass with FixtureAnyFunSui
val aliceInit = Init(aliceParams.features)
val bobInit = Init(bobParams.features)
within(30 seconds) {
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, TestConstants.fundingSatoshis, TestConstants.pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, aliceParams, alice2bob.ref, bobInit, ChannelFlags.Empty, channelVersion)
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, TestConstants.fundingSatoshis, TestConstants.pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, None, aliceParams, alice2bob.ref, bobInit, ChannelFlags.Empty, channelVersion)
bob ! INPUT_INIT_FUNDEE(ByteVector32.Zeroes, bobParams, bob2alice.ref, aliceInit, channelVersion)
awaitCond(bob.stateName == WAIT_FOR_OPEN_CHANNEL)
withFixture(test.toNoArgTest(FixtureParam(bob, alice2bob, bob2alice, bob2blockchain)))

View file

@ -49,7 +49,7 @@ class WaitForFundingCreatedInternalStateSpec extends TestKitBaseClass with Fixtu
val aliceInit = Init(Alice.channelParams.features)
val bobInit = Init(Bob.channelParams.features)
within(30 seconds) {
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, TestConstants.fundingSatoshis, TestConstants.pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, Alice.channelParams, alice2bob.ref, bobInit, ChannelFlags.Empty, ChannelVersion.STANDARD)
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, TestConstants.fundingSatoshis, TestConstants.pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, None, Alice.channelParams, alice2bob.ref, bobInit, ChannelFlags.Empty, ChannelVersion.STANDARD)
bob ! INPUT_INIT_FUNDEE(ByteVector32.Zeroes, Bob.channelParams, bob2alice.ref, aliceInit, ChannelVersion.STANDARD)
alice2bob.expectMsgType[OpenChannel]
alice2bob.forward(bob)

View file

@ -50,7 +50,7 @@ class WaitForFundingCreatedStateSpec extends TestKitBaseClass with FixtureAnyFun
val aliceInit = Init(Alice.channelParams.features)
val bobInit = Init(Bob.channelParams.features)
within(30 seconds) {
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, fundingSatoshis, pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, Alice.channelParams, alice2bob.ref, bobInit, ChannelFlags.Empty, ChannelVersion.STANDARD)
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, fundingSatoshis, pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, None, Alice.channelParams, alice2bob.ref, bobInit, ChannelFlags.Empty, ChannelVersion.STANDARD)
bob ! INPUT_INIT_FUNDEE(ByteVector32.Zeroes, Bob.channelParams, bob2alice.ref, aliceInit, ChannelVersion.STANDARD)
alice2bob.expectMsgType[OpenChannel]
alice2bob.forward(bob)

View file

@ -44,7 +44,7 @@ class WaitForFundingSignedStateSpec extends TestKitBaseClass with FixtureAnyFunS
val aliceInit = Init(Alice.channelParams.features)
val bobInit = Init(Bob.channelParams.features)
within(30 seconds) {
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, TestConstants.fundingSatoshis, TestConstants.pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, Alice.channelParams, alice2bob.ref, bobInit, ChannelFlags.Empty, ChannelVersion.STANDARD)
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, TestConstants.fundingSatoshis, TestConstants.pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, None, Alice.channelParams, alice2bob.ref, bobInit, ChannelFlags.Empty, ChannelVersion.STANDARD)
bob ! INPUT_INIT_FUNDEE(ByteVector32.Zeroes, Bob.channelParams, bob2alice.ref, aliceInit, ChannelVersion.STANDARD)
alice2bob.expectMsgType[OpenChannel]
alice2bob.forward(bob)

View file

@ -45,7 +45,7 @@ class WaitForFundingConfirmedStateSpec extends TestKitBaseClass with FixtureAnyF
val aliceInit = Init(Alice.channelParams.features)
val bobInit = Init(Bob.channelParams.features)
within(30 seconds) {
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, TestConstants.fundingSatoshis, TestConstants.pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, Alice.channelParams, alice2bob.ref, bobInit, ChannelFlags.Empty, ChannelVersion.STANDARD)
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, TestConstants.fundingSatoshis, TestConstants.pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, None, Alice.channelParams, alice2bob.ref, bobInit, ChannelFlags.Empty, ChannelVersion.STANDARD)
bob ! INPUT_INIT_FUNDEE(ByteVector32.Zeroes, Bob.channelParams, bob2alice.ref, aliceInit, ChannelVersion.STANDARD)
alice2bob.expectMsgType[OpenChannel]
alice2bob.forward(bob)

View file

@ -23,7 +23,7 @@ import fr.acinq.eclair.blockchain._
import fr.acinq.eclair.channel._
import fr.acinq.eclair.channel.states.StateTestsHelperMethods
import fr.acinq.eclair.wire._
import fr.acinq.eclair.{TestConstants, TestKitBaseClass}
import fr.acinq.eclair.{LongToBtcAmount, TestConstants, TestKitBaseClass}
import org.scalatest.Outcome
import org.scalatest.funsuite.FixtureAnyFunSuiteLike
@ -35,6 +35,8 @@ import scala.concurrent.duration._
class WaitForFundingLockedStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with StateTestsHelperMethods {
val initialRelayFees = (1000 msat, 100)
case class FixtureParam(alice: TestFSMRef[State, Data, Channel], bob: TestFSMRef[State, Data, Channel], alice2bob: TestProbe, bob2alice: TestProbe, alice2blockchain: TestProbe, router: TestProbe)
override def withFixture(test: OneArgTest): Outcome = {
@ -43,7 +45,7 @@ class WaitForFundingLockedStateSpec extends TestKitBaseClass with FixtureAnyFunS
val aliceInit = Init(Alice.channelParams.features)
val bobInit = Init(Bob.channelParams.features)
within(30 seconds) {
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, TestConstants.fundingSatoshis, TestConstants.pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, Alice.channelParams, alice2bob.ref, bobInit, ChannelFlags.Empty, ChannelVersion.STANDARD)
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, TestConstants.fundingSatoshis, TestConstants.pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, Some(initialRelayFees), Alice.channelParams, alice2bob.ref, bobInit, ChannelFlags.Empty, ChannelVersion.STANDARD)
bob ! INPUT_INIT_FUNDEE(ByteVector32.Zeroes, Bob.channelParams, bob2alice.ref, aliceInit, ChannelVersion.STANDARD)
alice2bob.expectMsgType[OpenChannel]
alice2bob.forward(bob)
@ -75,6 +77,9 @@ class WaitForFundingLockedStateSpec extends TestKitBaseClass with FixtureAnyFunS
bob2alice.expectMsgType[FundingLocked]
bob2alice.forward(alice)
awaitCond(alice.stateName == NORMAL)
val initialChannelUpdate = alice.stateData.asInstanceOf[DATA_NORMAL].channelUpdate
assert(initialChannelUpdate.feeBaseMsat === initialRelayFees._1)
assert(initialChannelUpdate.feeProportionalMillionths === initialRelayFees._2)
bob2alice.expectNoMsg(200 millis)
}

View file

@ -63,7 +63,7 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
within(30 seconds) {
val aliceInit = Init(Alice.channelParams.features)
val bobInit = Init(Bob.channelParams.features)
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, TestConstants.fundingSatoshis, TestConstants.pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, Alice.channelParams, alice2bob.ref, bobInit, ChannelFlags.Empty, ChannelVersion.STANDARD)
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, TestConstants.fundingSatoshis, TestConstants.pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, None, Alice.channelParams, alice2bob.ref, bobInit, ChannelFlags.Empty, ChannelVersion.STANDARD)
bob ! INPUT_INIT_FUNDEE(ByteVector32.Zeroes, Bob.channelParams, bob2alice.ref, aliceInit, ChannelVersion.STANDARD)
alice2bob.expectMsgType[OpenChannel]
alice2bob.forward(bob)

View file

@ -161,6 +161,7 @@ abstract class IntegrationSpec extends TestKitBaseClass with BitcoindService wit
fundingSatoshis = fundingSatoshis,
pushMsat = pushMsat,
fundingTxFeeratePerKw_opt = None,
initialRelayFees_opt = None,
channelFlags = None,
timeout_opt = None))
sender.expectMsgType[ChannelOpenResponse.ChannelOpened](10 seconds)

View file

@ -67,7 +67,7 @@ class RustyTestsSpec extends TestKitBaseClass with Matchers with FixtureAnyFunSu
val bobInit = Init(Bob.channelParams.features)
// alice and bob will both have 1 000 000 sat
feeEstimator.setFeerate(FeeratesPerKw.single(FeeratePerKw(10000 sat)))
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, 2000000 sat, 1000000000 msat, feeEstimator.getFeeratePerKw(target = 2), feeEstimator.getFeeratePerKw(target = 6), Alice.channelParams, pipe, bobInit, ChannelFlags.Empty, ChannelVersion.STANDARD)
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, 2000000 sat, 1000000000 msat, feeEstimator.getFeeratePerKw(target = 2), feeEstimator.getFeeratePerKw(target = 6), None, Alice.channelParams, pipe, bobInit, ChannelFlags.Empty, ChannelVersion.STANDARD)
bob ! INPUT_INIT_FUNDEE(ByteVector32.Zeroes, Bob.channelParams, pipe, aliceInit, ChannelVersion.STANDARD)
pipe ! (alice, bob)
within(30 seconds) {

View file

@ -19,8 +19,8 @@ package fr.acinq.eclair.io
import java.net.{InetAddress, ServerSocket, Socket}
import java.util.concurrent.Executors
import akka.actor.FSM
import akka.actor.Status.Failure
import akka.actor.{ActorRef, FSM}
import akka.testkit.{TestFSMRef, TestProbe}
import com.google.common.net.HostAndPort
import fr.acinq.bitcoin.Crypto.PublicKey
@ -30,8 +30,8 @@ import fr.acinq.eclair.Features.{StaticRemoteKey, Wumbo}
import fr.acinq.eclair.TestConstants._
import fr.acinq.eclair._
import fr.acinq.eclair.blockchain.{EclairWallet, TestWallet}
import fr.acinq.eclair.channel.states.StateTestsHelperMethods
import fr.acinq.eclair.channel._
import fr.acinq.eclair.channel.states.StateTestsHelperMethods
import fr.acinq.eclair.io.Peer._
import fr.acinq.eclair.wire._
import org.scalatest.funsuite.FixtureAnyFunSuiteLike
@ -249,7 +249,7 @@ class PeerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with StateTe
connect(remoteNodeId, peer, peerConnection)
assert(peer.stateData.channels.isEmpty)
probe.send(peer, Peer.OpenChannel(remoteNodeId, fundingAmountBig, 0 msat, None, None, None))
probe.send(peer, Peer.OpenChannel(remoteNodeId, fundingAmountBig, 0 msat, None, None, None, None))
assert(probe.expectMsgType[Failure].cause.getMessage == s"fundingSatoshis=$fundingAmountBig is too big, you must enable large channels support in 'eclair.features' to use funding above ${Channel.MAX_FUNDING} (see eclair.conf)")
}
@ -263,7 +263,7 @@ class PeerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with StateTe
connect(remoteNodeId, peer, peerConnection) // Bob doesn't support wumbo, Alice does
assert(peer.stateData.channels.isEmpty)
probe.send(peer, Peer.OpenChannel(remoteNodeId, fundingAmountBig, 0 msat, None, None, None))
probe.send(peer, Peer.OpenChannel(remoteNodeId, fundingAmountBig, 0 msat, None, None, None, None))
assert(probe.expectMsgType[Failure].cause.getMessage == s"fundingSatoshis=$fundingAmountBig is too big, the remote peer doesn't support wumbo")
}
@ -277,7 +277,7 @@ class PeerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with StateTe
connect(remoteNodeId, peer, peerConnection, remoteInit = wire.Init(Features(Set(ActivatedFeature(Wumbo, Optional))))) // Bob supports wumbo
assert(peer.stateData.channels.isEmpty)
probe.send(peer, Peer.OpenChannel(remoteNodeId, fundingAmountBig, 0 msat, None, None, None))
probe.send(peer, Peer.OpenChannel(remoteNodeId, fundingAmountBig, 0 msat, None, None, None, None))
assert(probe.expectMsgType[Failure].cause.getMessage == s"fundingSatoshis=$fundingAmountBig is too big for the current settings, increase 'eclair.max-funding-satoshis' (see eclair.conf)")
}
@ -288,14 +288,23 @@ class PeerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with StateTe
val probe = TestProbe()
system.eventStream.subscribe(probe.ref, classOf[ChannelCreated])
connect(remoteNodeId, peer, peerConnection)
assert(peer.stateData.channels.isEmpty)
probe.send(peer, Peer.OpenChannel(remoteNodeId, 12300 sat, 0 msat, None, None, None))
val relayFees = Some(100 msat, 1000)
probe.send(peer, Peer.OpenChannel(remoteNodeId, 12300 sat, 0 msat, None, relayFees, None, None))
awaitCond(peer.stateData.channels.nonEmpty)
val channelCreated = probe.expectMsgType[ChannelCreated]
assert(channelCreated.initialFeeratePerKw == peer.feeEstimator.getFeeratePerKw(peer.feeTargets.commitmentBlockTarget))
assert(channelCreated.fundingTxFeeratePerKw.get == peer.feeEstimator.getFeeratePerKw(peer.feeTargets.fundingBlockTarget))
peer.stateData.channels.foreach { case (_, channelRef) =>
probe.send(channelRef, CMD_GETINFO(probe.ref))
val info = probe.expectMsgType[RES_GETINFO]
assert(info.state == WAIT_FOR_ACCEPT_CHANNEL)
val inputInit = info.data.asInstanceOf[DATA_WAIT_FOR_ACCEPT_CHANNEL].initFunder
assert(inputInit.initialRelayFees_opt === relayFees)
}
}
test("use correct final script if option_static_remotekey is negotiated", Tag("static_remotekey")) { f =>
@ -303,7 +312,7 @@ class PeerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with StateTe
val probe = TestProbe()
connect(remoteNodeId, peer, peerConnection, remoteInit = wire.Init(Features(Set(ActivatedFeature(StaticRemoteKey, Optional))))) // Bob supports option_static_remotekey
probe.send(peer, Peer.OpenChannel(remoteNodeId, 24000 sat, 0 msat, None, None, None))
probe.send(peer, Peer.OpenChannel(remoteNodeId, 24000 sat, 0 msat, None, None, None, None))
awaitCond(peer.stateData.channels.nonEmpty)
peer.stateData.channels.foreach { case (_, channelRef) =>
probe.send(channelRef, CMD_GETINFO(probe.ref))

View file

@ -118,7 +118,7 @@ class OpenChannelController(val handlers: Handlers, val stage: Stage) extends Lo
feerateError.setText("Fee rate must be greater than 0")
case (Success(capacitySat), Success(pushMsat), Success(feeratePerByte_opt)) =>
val channelFlags = if (publicChannel.isSelected) ChannelFlags.AnnounceChannel else ChannelFlags.Empty
handlers.open(nodeUri, Some(Peer.OpenChannel(nodeUri.nodeId, capacitySat, MilliSatoshi(pushMsat), feeratePerByte_opt.map(f => FeeratePerKw(FeeratePerByte(f.sat))), Some(channelFlags), Some(30 seconds))))
handlers.open(nodeUri, Some(Peer.OpenChannel(nodeUri.nodeId, capacitySat, MilliSatoshi(pushMsat), feeratePerByte_opt.map(f => FeeratePerKw(FeeratePerByte(f.sat))), None, Some(channelFlags), Some(30 seconds))))
stage.close()
case (Failure(t), _, _) =>
logger.error(s"could not parse capacity with cause=${t.getLocalizedMessage}")

View file

@ -157,9 +157,17 @@ trait Service extends ExtraDirectives with Logging {
}
} ~
path("open") {
formFields(nodeIdFormParam, "fundingSatoshis".as[Satoshi], "pushMsat".as[MilliSatoshi].?, "fundingFeerateSatByte".as[FeeratePerByte].?, "channelFlags".as[Int].?, "openTimeoutSeconds".as[Timeout].?) {
(nodeId, fundingSatoshis, pushMsat, fundingFeerateSatByte, channelFlags, openTimeout_opt) =>
complete(eclairApi.open(nodeId, fundingSatoshis, pushMsat, fundingFeerateSatByte, channelFlags, openTimeout_opt))
formFields(nodeIdFormParam, "fundingSatoshis".as[Satoshi], "pushMsat".as[MilliSatoshi].?, "fundingFeerateSatByte".as[FeeratePerByte].?, "feeBaseMsat".as[MilliSatoshi].?, "feeProportionalMillionths".as[Int].?, "channelFlags".as[Int].?, "openTimeoutSeconds".as[Timeout].?) {
(nodeId, fundingSatoshis, pushMsat, fundingFeerateSatByte, feeBase, feeProportional, channelFlags, openTimeout_opt) =>
if (feeBase.nonEmpty && feeProportional.isEmpty || feeBase.isEmpty && feeProportional.nonEmpty) {
reject(MalformedFormFieldRejection("feeBaseMsat/feeProportionalMillionths", "All relay fees parameters (feeBaseMsat/feeProportionalMillionths) must be specified to override node defaults"))
} else {
val initialRelayFees = (feeBase, feeProportional) match {
case (Some(feeBase), Some(feeProportional)) => Some(feeBase, feeProportional)
case _ => None
}
complete(eclairApi.open(nodeId, fundingSatoshis, pushMsat, fundingFeerateSatByte, initialRelayFees, channelFlags, openTimeout_opt))
}
}
} ~
path("updaterelayfee") {

View file

@ -34,6 +34,7 @@ import fr.acinq.eclair.FeatureSupport.{Mandatory, Optional}
import fr.acinq.eclair.Features.{ChannelRangeQueriesExtended, OptionDataLossProtect}
import fr.acinq.eclair._
import fr.acinq.eclair.blockchain.fee.FeeratePerKw
import fr.acinq.eclair.channel.ChannelOpenResponse.ChannelOpened
import fr.acinq.eclair.channel.Helpers.Closing
import fr.acinq.eclair.channel._
import fr.acinq.eclair.db._
@ -200,6 +201,37 @@ class ApiServiceSpec extends AnyFunSuite with ScalatestRouteTest with IdiomaticM
}
}
test("'open' channels") {
val nodeId = PublicKey(hex"030bb6a5e0c6b203c7e2180fb78c7ba4bdce46126761d8201b91ddac089cdecc87")
val channelId = ByteVector32(hex"56d7d6eda04d80138270c49709f1eadb5ab4939e5061309ccdacdb98ce637d0e")
val eclair = mock[Eclair]
eclair.open(any, any, any, any, any, any, any)(any[Timeout]) returns Future.successful(ChannelOpened(channelId))
val mockService = new MockService(eclair)
Post("/open", FormData("nodeId" -> nodeId.toString(), "fundingSatoshis" -> "100000").toEntity) ~>
addCredentials(BasicHttpCredentials("", mockService.password)) ~>
addHeader("Content-Type", "application/json") ~>
Route.seal(mockService.route) ~>
check {
assert(handled)
assert(status == OK)
assert(entityAs[String] == "\"created channel 56d7d6eda04d80138270c49709f1eadb5ab4939e5061309ccdacdb98ce637d0e\"")
eclair.open(nodeId, 100000 sat, None, None, None, None, None)(any[Timeout]).wasCalled(once)
}
Post("/open", FormData("nodeId" -> nodeId.toString(), "fundingSatoshis" -> "50000", "feeBaseMsat" -> "100", "feeProportionalMillionths" -> "10").toEntity) ~>
addCredentials(BasicHttpCredentials("", mockService.password)) ~>
addHeader("Content-Type", "application/json") ~>
Route.seal(mockService.route) ~>
check {
assert(handled)
assert(status == OK)
assert(entityAs[String] == "\"created channel 56d7d6eda04d80138270c49709f1eadb5ab4939e5061309ccdacdb98ce637d0e\"")
eclair.open(nodeId, 50000 sat, None, None, Some(100 msat, 10), None, None)(any[Timeout]).wasCalled(once)
}
}
test("'close' method should accept channelIds and shortChannelIds") {
val shortChannelIdSerialized = "42000x27x3"
val channelId = ByteVector32(hex"56d7d6eda04d80138270c49709f1eadb5ab4939e5061309ccdacdb98ce637d0e")