1
0
mirror of https://github.com/ACINQ/eclair.git synced 2024-11-20 02:27:32 +01:00

Channel sends typed responses (#1321)

Instead of sending strings, channel now sends typed responses.
This is more future-proof when we want to add data to those responses.
This commit is contained in:
Bastien Teinturier 2020-02-25 10:18:51 +01:00 committed by GitHub
parent 4012db5828
commit bb930cd8b0
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
26 changed files with 220 additions and 203 deletions

View File

@ -66,13 +66,13 @@ trait Eclair {
def disconnect(nodeId: PublicKey)(implicit timeout: Timeout): Future[String]
def open(nodeId: PublicKey, fundingAmount: Satoshi, pushAmount_opt: Option[MilliSatoshi], fundingFeerateSatByte_opt: Option[Long], flags_opt: Option[Int], openTimeout_opt: Option[Timeout])(implicit timeout: Timeout): Future[String]
def open(nodeId: PublicKey, fundingAmount: Satoshi, pushAmount_opt: Option[MilliSatoshi], fundingFeerateSatByte_opt: Option[Long], flags_opt: Option[Int], openTimeout_opt: Option[Timeout])(implicit timeout: Timeout): Future[ChannelCommandResponse]
def close(channelIdentifier: Either[ByteVector32, ShortChannelId], scriptPubKey_opt: Option[ByteVector])(implicit timeout: Timeout): Future[String]
def close(channelIdentifier: Either[ByteVector32, ShortChannelId], scriptPubKey_opt: Option[ByteVector])(implicit timeout: Timeout): Future[ChannelCommandResponse]
def forceClose(channelIdentifier: Either[ByteVector32, ShortChannelId])(implicit timeout: Timeout): Future[String]
def forceClose(channelIdentifier: Either[ByteVector32, ShortChannelId])(implicit timeout: Timeout): Future[ChannelCommandResponse]
def updateRelayFee(channelIdentifier: Either[ByteVector32, ShortChannelId], feeBase: MilliSatoshi, feeProportionalMillionths: Long)(implicit timeout: Timeout): Future[String]
def updateRelayFee(channelIdentifier: Either[ByteVector32, ShortChannelId], feeBase: MilliSatoshi, feeProportionalMillionths: Long)(implicit timeout: Timeout): Future[ChannelCommandResponse]
def channelsInfo(toRemoteNode_opt: Option[PublicKey])(implicit timeout: Timeout): Future[Iterable[RES_GETINFO]]
@ -135,7 +135,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], fundingFeerateSatByte_opt: Option[Long], flags_opt: Option[Int], openTimeout_opt: Option[Timeout])(implicit timeout: Timeout): Future[String] = {
override def open(nodeId: PublicKey, fundingAmount: Satoshi, pushAmount_opt: Option[MilliSatoshi], fundingFeerateSatByte_opt: Option[Long], flags_opt: Option[Int], openTimeout_opt: Option[Timeout])(implicit timeout: Timeout): Future[ChannelCommandResponse] = {
// 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(
@ -144,19 +144,19 @@ class EclairImpl(appKit: Kit) extends Eclair {
pushMsat = pushAmount_opt.getOrElse(0 msat),
fundingTxFeeratePerKw_opt = fundingFeerateSatByte_opt.map(feerateByte2Kw),
channelFlags = flags_opt.map(_.toByte),
timeout_opt = Some(openTimeout))).mapTo[String]
timeout_opt = Some(openTimeout))).mapTo[ChannelCommandResponse]
}
override def close(channelIdentifier: Either[ByteVector32, ShortChannelId], scriptPubKey_opt: Option[ByteVector])(implicit timeout: Timeout): Future[String] = {
sendToChannel(channelIdentifier, CMD_CLOSE(scriptPubKey_opt)).mapTo[String]
override def close(channelIdentifier: Either[ByteVector32, ShortChannelId], scriptPubKey_opt: Option[ByteVector])(implicit timeout: Timeout): Future[ChannelCommandResponse] = {
sendToChannel(channelIdentifier, CMD_CLOSE(scriptPubKey_opt)).mapTo[ChannelCommandResponse]
}
override def forceClose(channelIdentifier: Either[ByteVector32, ShortChannelId])(implicit timeout: Timeout): Future[String] = {
sendToChannel(channelIdentifier, CMD_FORCECLOSE).mapTo[String]
override def forceClose(channelIdentifier: Either[ByteVector32, ShortChannelId])(implicit timeout: Timeout): Future[ChannelCommandResponse] = {
sendToChannel(channelIdentifier, CMD_FORCECLOSE).mapTo[ChannelCommandResponse]
}
override def updateRelayFee(channelIdentifier: Either[ByteVector32, ShortChannelId], feeBaseMsat: MilliSatoshi, feeProportionalMillionths: Long)(implicit timeout: Timeout): Future[String] = {
sendToChannel(channelIdentifier, CMD_UPDATE_RELAY_FEE(feeBaseMsat, feeProportionalMillionths)).mapTo[String]
override def updateRelayFee(channelIdentifier: Either[ByteVector32, ShortChannelId], feeBaseMsat: MilliSatoshi, feeProportionalMillionths: Long)(implicit timeout: Timeout): Future[ChannelCommandResponse] = {
sendToChannel(channelIdentifier, CMD_UPDATE_RELAY_FEE(feeBaseMsat, feeProportionalMillionths)).mapTo[ChannelCommandResponse]
}
override def peersInfo()(implicit timeout: Timeout): Future[Iterable[PeerInfo]] = for {

View File

@ -31,7 +31,7 @@ import fr.acinq.eclair.payment._
import fr.acinq.eclair.payment.relay.{CommandBuffer, Origin, Relayer}
import fr.acinq.eclair.router.Announcements
import fr.acinq.eclair.transactions._
import fr.acinq.eclair.wire.{ChannelReestablish, _}
import fr.acinq.eclair.wire._
import scala.collection.immutable.Queue
import scala.compat.Platform
@ -39,7 +39,6 @@ import scala.concurrent.ExecutionContext
import scala.concurrent.duration._
import scala.util.{Failure, Success, Try}
/**
* Created by PM on 20/08/2015.
*/
@ -84,15 +83,9 @@ object Channel {
case object TickChannelOpenTimeout
// we will receive this message when we waited too long for a revocation for that commit number (NB: we explicitely specify the peer to allow for testing)
// we will receive this message when we waited too long for a revocation for that commit number (NB: we explicitly specify the peer to allow for testing)
case class RevocationTimeout(remoteCommitNumber: Long, peer: ActorRef)
// @formatter:off
sealed trait ChannelError
case class LocalError(t: Throwable) extends ChannelError
case class RemoteError(e: Error) extends ChannelError
// @formatter:on
def ackPendingFailsAndFulfills(updates: List[UpdateMessage], relayer: ActorRef): Unit = updates.collect {
case u: UpdateFailMalformedHtlc => relayer ! CommandBuffer.CommandAck(u.channelId, u.id)
case u: UpdateFulfillHtlc => relayer ! CommandBuffer.CommandAck(u.channelId, u.id)
@ -267,10 +260,10 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
goto(OFFLINE) using data
}
case Event(CMD_CLOSE(_), _) => goto(CLOSED) replying "ok"
case Event(CMD_CLOSE(_), _) => goto(CLOSED) replying ChannelCommandResponse.Ok
case Event(TickChannelOpenTimeout, _) =>
replyToUser(Left(LocalError(new RuntimeException("open channel cancelled, took too long"))))
channelOpenReplyToUser(Left(LocalError(new RuntimeException("open channel cancelled, took too long"))))
goto(CLOSED)
})
@ -318,7 +311,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
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
}
case Event(CMD_CLOSE(_), _) => goto(CLOSED) replying "ok"
case Event(CMD_CLOSE(_), _) => goto(CLOSED) replying ChannelCommandResponse.Ok
case Event(e: Error, d: DATA_WAIT_FOR_OPEN_CHANNEL) => handleRemoteError(e, d)
@ -353,20 +346,20 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
goto(WAIT_FOR_FUNDING_INTERNAL) using DATA_WAIT_FOR_FUNDING_INTERNAL(temporaryChannelId, localParams, remoteParams, fundingSatoshis, pushMsat, initialFeeratePerKw, accept.firstPerCommitmentPoint, channelVersion, open)
}
case Event(CMD_CLOSE(_), _) =>
replyToUser(Right("closed"))
goto(CLOSED) replying "ok"
case Event(CMD_CLOSE(_), d: DATA_WAIT_FOR_ACCEPT_CHANNEL) =>
channelOpenReplyToUser(Right(ChannelCommandResponse.ChannelClosed(d.lastSent.temporaryChannelId)))
goto(CLOSED) replying ChannelCommandResponse.Ok
case Event(e: Error, d: DATA_WAIT_FOR_ACCEPT_CHANNEL) =>
replyToUser(Left(RemoteError(e)))
channelOpenReplyToUser(Left(RemoteError(e)))
handleRemoteError(e, d)
case Event(INPUT_DISCONNECTED, _) =>
replyToUser(Left(LocalError(new RuntimeException("disconnected"))))
channelOpenReplyToUser(Left(LocalError(new RuntimeException("disconnected"))))
goto(CLOSED)
case Event(TickChannelOpenTimeout, _) =>
replyToUser(Left(LocalError(new RuntimeException("open channel cancelled, took too long"))))
channelOpenReplyToUser(Left(LocalError(new RuntimeException("open channel cancelled, took too long"))))
goto(CLOSED)
})
@ -391,23 +384,23 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
case Event(Status.Failure(t), d: DATA_WAIT_FOR_FUNDING_INTERNAL) =>
log.error(t, s"wallet returned error: ")
replyToUser(Left(LocalError(t)))
channelOpenReplyToUser(Left(LocalError(t)))
handleLocalError(ChannelFundingError(d.temporaryChannelId), d, None) // we use a generic exception and don't send the internal error to the peer
case Event(CMD_CLOSE(_), _) =>
replyToUser(Right("closed"))
goto(CLOSED) replying "ok"
case Event(CMD_CLOSE(_), d: DATA_WAIT_FOR_FUNDING_INTERNAL) =>
channelOpenReplyToUser(Right(ChannelCommandResponse.ChannelClosed(d.temporaryChannelId)))
goto(CLOSED) replying ChannelCommandResponse.Ok
case Event(e: Error, d: DATA_WAIT_FOR_FUNDING_INTERNAL) =>
replyToUser(Left(RemoteError(e)))
channelOpenReplyToUser(Left(RemoteError(e)))
handleRemoteError(e, d)
case Event(INPUT_DISCONNECTED, _) =>
replyToUser(Left(LocalError(new RuntimeException("disconnected"))))
channelOpenReplyToUser(Left(LocalError(new RuntimeException("disconnected"))))
goto(CLOSED)
case Event(TickChannelOpenTimeout, _) =>
replyToUser(Left(LocalError(new RuntimeException("open channel cancelled, took too long"))))
channelOpenReplyToUser(Left(LocalError(new RuntimeException("open channel cancelled, took too long"))))
goto(CLOSED)
})
@ -449,7 +442,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
goto(WAIT_FOR_FUNDING_CONFIRMED) using DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments, None, now, None, Right(fundingSigned)) storing() sending fundingSigned
}
case Event(CMD_CLOSE(_), _) => goto(CLOSED) replying "ok"
case Event(CMD_CLOSE(_), _) => goto(CLOSED) replying ChannelCommandResponse.Ok
case Event(e: Error, d: DATA_WAIT_FOR_FUNDING_CREATED) => handleRemoteError(e, d)
@ -466,7 +459,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
case Failure(cause) =>
// we rollback the funding tx, it will never be published
wallet.rollback(fundingTx)
replyToUser(Left(LocalError(cause)))
channelOpenReplyToUser(Left(LocalError(cause)))
handleLocalError(InvalidCommitmentSignature(channelId, signedLocalCommitTx.tx), d, Some(msg))
case Success(_) =>
val commitInput = localCommitTx.input
@ -490,12 +483,12 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
case Success(true) =>
// NB: funding tx isn't confirmed at this point, so technically we didn't really pay the network fee yet, so this is a (fair) approximation
feePaid(fundingTxFee, fundingTx, "funding", commitments.channelId)
replyToUser(Right(s"created channel $channelId"))
channelOpenReplyToUser(Right(ChannelCommandResponse.ChannelOpened(channelId)))
case Success(false) =>
replyToUser(Left(LocalError(new RuntimeException("couldn't publish funding tx"))))
channelOpenReplyToUser(Left(LocalError(new RuntimeException("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) =>
replyToUser(Left(LocalError(t)))
channelOpenReplyToUser(Left(LocalError(t)))
log.error(t, s"error while committing funding tx: ") // tx may still have been published, can't fail-fast
}
}
@ -505,25 +498,25 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
case Event(CMD_CLOSE(_) | CMD_FORCECLOSE, d: DATA_WAIT_FOR_FUNDING_SIGNED) =>
// we rollback the funding tx, it will never be published
wallet.rollback(d.fundingTx)
replyToUser(Right("closed"))
goto(CLOSED) replying "ok"
channelOpenReplyToUser(Right(ChannelCommandResponse.ChannelClosed(d.channelId)))
goto(CLOSED) replying ChannelCommandResponse.Ok
case Event(e: Error, d: DATA_WAIT_FOR_FUNDING_SIGNED) =>
// we rollback the funding tx, it will never be published
wallet.rollback(d.fundingTx)
replyToUser(Left(RemoteError(e)))
channelOpenReplyToUser(Left(RemoteError(e)))
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)
replyToUser(Left(LocalError(new RuntimeException("disconnected"))))
channelOpenReplyToUser(Left(LocalError(new RuntimeException("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)
replyToUser(Left(LocalError(new RuntimeException("open channel cancelled, took too long"))))
channelOpenReplyToUser(Left(LocalError(new RuntimeException("open channel cancelled, took too long"))))
goto(CLOSED)
})
@ -918,7 +911,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
log.info(s"updating relay fees: prevFeeBaseMsat={} nextFeeBaseMsat={} prevFeeProportionalMillionths={} nextFeeProportionalMillionths={}", d.channelUpdate.feeBaseMsat, feeBaseMsat, d.channelUpdate.feeProportionalMillionths, feeProportionalMillionths)
val channelUpdate = Announcements.makeChannelUpdate(nodeParams.chainHash, nodeParams.privateKey, remoteNodeId, d.shortChannelId, d.channelUpdate.cltvExpiryDelta, d.channelUpdate.htlcMinimumMsat, feeBaseMsat, feeProportionalMillionths, d.commitments.localCommit.spec.totalFunds, enable = Helpers.aboveReserve(d.commitments))
// we use GOTO instead of stay because we want to fire transitions
goto(NORMAL) using d.copy(channelUpdate = channelUpdate) storing() replying "ok"
goto(NORMAL) using d.copy(channelUpdate = channelUpdate) storing() replying ChannelCommandResponse.Ok
case Event(BroadcastChannelUpdate(reason), d: DATA_NORMAL) =>
val age = Platform.currentTime.milliseconds - d.channelUpdate.timestamp.seconds
@ -1417,7 +1410,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
log.info(s"updating relay fees: prevFeeBaseMsat={} nextFeeBaseMsat={} prevFeeProportionalMillionths={} nextFeeProportionalMillionths={}", d.channelUpdate.feeBaseMsat, feeBaseMsat, d.channelUpdate.feeProportionalMillionths, feeProportionalMillionths)
val channelUpdate = Announcements.makeChannelUpdate(nodeParams.chainHash, nodeParams.privateKey, remoteNodeId, d.shortChannelId, d.channelUpdate.cltvExpiryDelta, d.channelUpdate.htlcMinimumMsat, feeBaseMsat, feeProportionalMillionths, d.commitments.localCommit.spec.totalFunds, enable = false)
// we're in OFFLINE state, we don't broadcast the new update right away, we will do that when next time we go to NORMAL state
stay using d.copy(channelUpdate = channelUpdate) storing() replying "ok"
stay using d.copy(channelUpdate = channelUpdate) storing() replying ChannelCommandResponse.Ok
case Event(getTxResponse: GetTxWithMetaResponse, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) if getTxResponse.txid == d.commitments.commitInput.outPoint.txid => handleGetFundingTx(getTxResponse, d.waitingSince, d.fundingTx)
@ -1614,7 +1607,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
case Event(c@CMD_FORCECLOSE, d) =>
d match {
case data: HasCommitments => handleLocalError(ForcedLocalCommit(data.channelId), data, Some(c)) replying "ok"
case data: HasCommitments => handleLocalError(ForcedLocalCommit(data.channelId), data, Some(c)) replying ChannelCommandResponse.Ok
case _ => handleCommandError(CommandUnavailableInThisState(Helpers.getChannelId(d), "forceclose", stateName), c)
}
@ -1633,7 +1626,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
case Event(_: BroadcastChannelUpdate, _) => stay
// we receive this when we send command to ourselves
case Event("ok", _) => stay
case Event(ChannelCommandResponse.Ok, _) => stay
// we receive this when we tell the peer to disconnect
case Event("disconnecting", _) => stay
@ -1723,7 +1716,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
/**
* This function is used to return feedback to user at channel opening
*/
def replyToUser(message: Either[Channel.ChannelError, String]): Unit = {
def channelOpenReplyToUser(message: Either[ChannelOpenError, ChannelCommandResponse]): 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}"))
@ -1770,7 +1763,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
}
def handleCommandSuccess(sender: ActorRef, newData: Data) = {
stay using newData replying "ok"
stay using newData replying ChannelCommandResponse.Ok
}
def handleCommandError(cause: Throwable, cmd: Command) = {

View File

@ -19,14 +19,13 @@ package fr.acinq.eclair.channel
import akka.actor.ActorRef
import fr.acinq.bitcoin.Crypto.PublicKey
import fr.acinq.bitcoin.{ByteVector32, Satoshi, Transaction}
import fr.acinq.eclair.{MilliSatoshi, ShortChannelId}
import fr.acinq.eclair.channel.Channel.ChannelError
import fr.acinq.eclair.ShortChannelId
import fr.acinq.eclair.channel.Helpers.Closing.ClosingType
import fr.acinq.eclair.wire.{ChannelAnnouncement, ChannelUpdate}
/**
* Created by PM on 17/08/2016.
*/
* Created by PM on 17/08/2016.
*/
trait ChannelEvent
@ -48,7 +47,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, data: Data, error: ChannelError, isFatal: Boolean) extends ChannelEvent
case class ChannelErrorOccurred(channel: ActorRef, channelId: ByteVector32, remoteNodeId: PublicKey, data: Data, error: ChannelOpenError, isFatal: Boolean) extends ChannelEvent
case class NetworkFeePaid(channel: ActorRef, remoteNodeId: PublicKey, channelId: ByteVector32, tx: Transaction, fee: Satoshi, txType: String) extends ChannelEvent

View File

@ -19,13 +19,19 @@ package fr.acinq.eclair.channel
import fr.acinq.bitcoin.Crypto.PrivateKey
import fr.acinq.bitcoin.{ByteVector32, Satoshi, Transaction}
import fr.acinq.eclair.payment.relay.Origin
import fr.acinq.eclair.wire.{ChannelUpdate, UpdateAddHtlc}
import fr.acinq.eclair.wire.{ChannelUpdate, Error, UpdateAddHtlc}
import fr.acinq.eclair.{CltvExpiry, CltvExpiryDelta, MilliSatoshi, UInt64}
/**
* Created by PM on 11/04/2017.
*/
// @formatter:off
sealed trait ChannelOpenError
case class LocalError(t: Throwable) extends ChannelOpenError
case class RemoteError(e: Error) extends ChannelOpenError
// @formatter:on
class ChannelException(val channelId: ByteVector32, message: String) extends RuntimeException(message)
// @formatter:off

View File

@ -133,6 +133,24 @@ case object CMD_GETSTATEDATA extends Command
case object CMD_GETINFO extends Command
final case class RES_GETINFO(nodeId: PublicKey, channelId: ByteVector32, state: State, data: Data)
/*
88888888b. 8888888888 .d8888b. 88888888b. ,ad8888ba, 888b 88 .d8888b. 8888888888 .d8888b.
88 "8b 88 d88P Y88b 88 "8b d8"' `"8b 8888b 88 d88P Y88b 88 d88P Y88b
88 ,8P 88 Y88b. 88 ,8P d8' `8b 88 `8b 88 Y88b. 88 Y88b.
888888888P' 888888 "Y888b. 888888888P' 88 88 88 `8b 88 "Y888b. 888888 "Y888b.
88 88' 88 "Y88b. 88 88 88 88 `8b 88 "Y88b. 88 "Y88b.
88 `8b 88 "888 88 Y8, ,8P 88 `8b 88 "888 88 "888
88 `8b 88 Y88b d88P 88 Y8a. .a8P 88 `8888 Y88b d88P 88 Y88b d88P
88 `8b 8888888888 "Y8888P" 88 `"Y8888Y"' 88 `888 "Y8888P" 8888888888 "Y8888P"
*/
sealed trait ChannelCommandResponse
object ChannelCommandResponse {
case object Ok extends ChannelCommandResponse { override def toString = "ok" }
case class ChannelOpened(channelId: ByteVector32) extends ChannelCommandResponse { override def toString = s"created channel $channelId" }
case class ChannelClosed(channelId: ByteVector32) extends ChannelCommandResponse { override def toString = s"closed channel $channelId" }
}
/*
8888888b. d8888 88888888888 d8888
888 "Y88b d88888 888 d88888

View File

@ -21,7 +21,7 @@ import java.util.UUID
import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey}
import fr.acinq.bitcoin.{ByteVector32, Satoshi}
import fr.acinq.eclair.channel.{Channel, ChannelErrorOccurred, NetworkFeePaid}
import fr.acinq.eclair.channel.{ChannelErrorOccurred, LocalError, NetworkFeePaid, RemoteError}
import fr.acinq.eclair.db._
import fr.acinq.eclair.payment._
import fr.acinq.eclair.wire.ChannelCodecs
@ -96,7 +96,7 @@ class SqliteAuditDb(sqlite: Connection) extends AuditDb with Logging {
statement.executeUpdate("CREATE TABLE IF NOT EXISTS network_fees (channel_id BLOB NOT NULL, node_id BLOB NOT NULL, tx_id BLOB NOT NULL, fee_sat INTEGER NOT NULL, tx_type TEXT NOT NULL, timestamp INTEGER NOT NULL)")
statement.executeUpdate("CREATE TABLE IF NOT EXISTS channel_events (channel_id BLOB NOT NULL, node_id BLOB NOT NULL, capacity_sat INTEGER NOT NULL, is_funder BOOLEAN NOT NULL, is_private BOOLEAN NOT NULL, event TEXT NOT NULL, timestamp INTEGER NOT NULL)")
statement.executeUpdate("CREATE TABLE IF NOT EXISTS channel_errors (channel_id BLOB NOT NULL, node_id BLOB NOT NULL, error_name TEXT NOT NULL, error_message TEXT NOT NULL, is_fatal INTEGER NOT NULL, timestamp INTEGER NOT NULL)")
statement.executeUpdate("CREATE INDEX IF NOT EXISTS sent_timestamp_idx ON sent(timestamp)")
statement.executeUpdate("CREATE INDEX IF NOT EXISTS received_timestamp_idx ON received(timestamp)")
statement.executeUpdate("CREATE INDEX IF NOT EXISTS relayed_timestamp_idx ON relayed(timestamp)")
@ -186,8 +186,8 @@ class SqliteAuditDb(sqlite: Connection) extends AuditDb with Logging {
override def add(e: ChannelErrorOccurred): Unit =
using(sqlite.prepareStatement("INSERT INTO channel_errors VALUES (?, ?, ?, ?, ?, ?)")) { statement =>
val (errorName, errorMessage) = e.error match {
case Channel.LocalError(t) => (t.getClass.getSimpleName, t.getMessage)
case Channel.RemoteError(error) => ("remote", error.toAscii)
case LocalError(t) => (t.getClass.getSimpleName, t.getMessage)
case RemoteError(error) => ("remote", error.toAscii)
}
statement.setBytes(1, e.channelId.toArray)
statement.setBytes(2, e.remoteNodeId.value.toArray)

View File

@ -18,7 +18,6 @@ package fr.acinq.eclair.payment
import akka.actor.{Actor, ActorLogging, Props}
import fr.acinq.eclair.NodeParams
import fr.acinq.eclair.channel.Channel.{LocalError, RemoteError}
import fr.acinq.eclair.channel.Helpers.Closing._
import fr.acinq.eclair.channel._
import fr.acinq.eclair.db.ChannelLifecycleEvent

View File

@ -20,7 +20,7 @@ import akka.actor.Actor.Receive
import akka.actor.{ActorContext, ActorRef, PoisonPill, Status}
import akka.event.{DiagnosticLoggingAdapter, LoggingAdapter}
import fr.acinq.bitcoin.{ByteVector32, Crypto}
import fr.acinq.eclair.channel.{CMD_FAIL_HTLC, CMD_FULFILL_HTLC, Channel}
import fr.acinq.eclair.channel.{CMD_FAIL_HTLC, CMD_FULFILL_HTLC, Channel, ChannelCommandResponse}
import fr.acinq.eclair.db.{IncomingPayment, IncomingPaymentStatus, IncomingPaymentsDb, PaymentType}
import fr.acinq.eclair.payment.PaymentRequest.ExtraHop
import fr.acinq.eclair.payment.relay.CommandBuffer
@ -142,7 +142,7 @@ class MultiPartHandler(nodeParams: NodeParams, db: IncomingPaymentsDb, commandBu
case ack: CommandBuffer.CommandAck => commandBuffer forward ack
case "ok" => // ignoring responses from channels
case ChannelCommandResponse.Ok => // ignoring responses from channels
}
}

View File

@ -19,6 +19,7 @@ package fr.acinq.eclair.payment.receive
import akka.actor.{ActorRef, Props}
import akka.event.Logging.MDC
import fr.acinq.bitcoin.ByteVector32
import fr.acinq.eclair.channel.ChannelCommandResponse
import fr.acinq.eclair.payment.PaymentReceived.PartialPayment
import fr.acinq.eclair.wire.{FailureMessage, IncorrectOrUnknownPaymentDetails, UpdateAddHtlc}
import fr.acinq.eclair.{FSMDiagnosticActorLogging, Logs, MilliSatoshi, NodeParams, wire}
@ -84,7 +85,7 @@ class MultiPartPaymentFSM(nodeParams: NodeParams, paymentHash: ByteVector32, tot
}
whenUnhandled {
case Event("ok", _) => stay
case Event(ChannelCommandResponse.Ok, _) => stay
}
onTransition {

View File

@ -74,7 +74,7 @@ class ChannelRelayer(nodeParams: NodeParams, relayer: ActorRef, register: ActorR
case ack: CommandBuffer.CommandAck => commandBuffer forward ack
case "ok" => // ignoring responses from channels
case ChannelCommandResponse.Ok => // ignoring responses from channels
}
override def mdc(currentMessage: Any): MDC = {

View File

@ -103,7 +103,7 @@ class PostRestartHtlcCleaner(nodeParams: NodeParams, commandBuffer: ActorRef, in
case ack: CommandBuffer.CommandAck => commandBuffer forward ack
case "ok" => // ignoring responses from channels
case ChannelCommandResponse.Ok => // ignoring responses from channels
}
private def handleDownstreamFulfill(brokenHtlcs: BrokenHtlcs, origin: Origin, fulfilledHtlc: UpdateAddHtlc, paymentPreimage: ByteVector32): Unit =

View File

@ -191,7 +191,7 @@ class Relayer(nodeParams: NodeParams, router: ActorRef, register: ActorRef, comm
case ack: CommandBuffer.CommandAck => commandBuffer forward ack
case "ok" => () // ignoring responses from channels
case ChannelCommandResponse.Ok => () // ignoring responses from channels
}
override def mdc(currentMessage: Any): MDC = {

View File

@ -21,7 +21,7 @@ import akka.event.Logging.MDC
import fr.acinq.bitcoin.ByteVector32
import fr.acinq.bitcoin.Crypto.PublicKey
import fr.acinq.eclair._
import fr.acinq.eclair.channel.{CMD_ADD_HTLC, Register}
import fr.acinq.eclair.channel.{ChannelCommandResponse, CMD_ADD_HTLC, Register}
import fr.acinq.eclair.crypto.{Sphinx, TransportHandler}
import fr.acinq.eclair.db.{OutgoingPayment, OutgoingPaymentStatus, PaymentType}
import fr.acinq.eclair.payment.PaymentRequest.ExtraHop
@ -113,7 +113,7 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A
}
when(WAITING_FOR_PAYMENT_COMPLETE) {
case Event("ok", _) => stay
case Event(ChannelCommandResponse.Ok, _) => stay
case Event(fulfill: UpdateFulfillHtlc, WaitingForComplete(s, c, cmd, _, _, _, _, route)) =>
val p = PartialPayment(id, c.finalPayload.amount, cmd.amount - c.finalPayload.amount, fulfill.channelId, Some(cfg.fullRoute(route)))

View File

@ -186,7 +186,7 @@ class FuzzySpec extends TestkitBaseClass with StateTestsHelperMethods with Loggi
val sender = TestProbe()
awaitCond({
sender.send(alice, CMD_CLOSE(None))
sender.expectMsgAnyClassOf(classOf[String], classOf[Status.Failure]) == "ok"
sender.expectMsgAnyClassOf(classOf[ChannelCommandResponse], classOf[Status.Failure]) == ChannelCommandResponse.Ok
}, interval = 1 second, max = 30 seconds)
awaitCond(alice.stateName == CLOSING, interval = 1 second, max = 3 minutes)
awaitCond(bob.stateName == CLOSING, interval = 1 second, max = 3 minutes)
@ -203,11 +203,11 @@ class FuzzySpec extends TestkitBaseClass with StateTestsHelperMethods with Loggi
val sender = TestProbe()
awaitCond({
sender.send(alice, CMD_CLOSE(None))
val resa = sender.expectMsgAnyClassOf(classOf[String], classOf[Status.Failure])
val resa = sender.expectMsgAnyClassOf(classOf[ChannelCommandResponse], classOf[Status.Failure])
sender.send(bob, CMD_CLOSE(None))
val resb = sender.expectMsgAnyClassOf(classOf[String], classOf[Status.Failure])
val resb = sender.expectMsgAnyClassOf(classOf[ChannelCommandResponse], classOf[Status.Failure])
// we only need that one of them succeeds
resa == "ok" || resb == "ok"
resa == ChannelCommandResponse.Ok || resb == ChannelCommandResponse.Ok
}, interval = 1 second, max = 30 seconds)
awaitCond(alice.stateName == CLOSING, interval = 1 second, max = 3 minutes)
awaitCond(bob.stateName == CLOSING, interval = 1 second, max = 3 minutes)

View File

@ -12,7 +12,6 @@ import fr.acinq.eclair.transactions.Transactions.{ClaimP2WPKHOutputTx, InputInfo
import fr.acinq.eclair.wire.{ChannelReestablish, CommitSig, Error, Init, RevokeAndAck}
import fr.acinq.eclair.{TestConstants, TestkitBaseClass, _}
import org.scalatest.Outcome
import scodec.bits.ByteVector
import scala.concurrent.duration._
@ -47,7 +46,7 @@ class RecoverySpec extends TestkitBaseClass with StateTestsHelperMethods {
// then we add an htlc and sign it
addHtlc(250000000 msat, alice, bob, alice2bob, bob2alice)
sender.send(alice, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[CommitSig]
alice2bob.forward(bob)
// alice will receive neither the revocation nor the commit sig

View File

@ -123,7 +123,7 @@ trait StateTestsHelperMethods extends TestKitBase with fixture.TestSuite with Pa
val currentBlockHeight = s.underlyingActor.nodeParams.currentBlockHeight
val (payment_preimage, cmd) = makeCmdAdd(amount, r.underlyingActor.nodeParams.nodeId, currentBlockHeight)
sender.send(s, cmd)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
val htlc = s2r.expectMsgType[UpdateAddHtlc]
s2r.forward(r)
awaitCond(r.stateData.asInstanceOf[HasCommitments].commitments.remoteChanges.proposed.contains(htlc))
@ -133,7 +133,7 @@ trait StateTestsHelperMethods extends TestKitBase with fixture.TestSuite with Pa
def fulfillHtlc(id: Long, R: ByteVector32, s: TestFSMRef[State, Data, Channel], r: TestFSMRef[State, Data, Channel], s2r: TestProbe, r2s: TestProbe): Unit = {
val sender = TestProbe()
sender.send(s, CMD_FULFILL_HTLC(id, R))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
val fulfill = s2r.expectMsgType[UpdateFulfillHtlc]
s2r.forward(r)
awaitCond(r.stateData.asInstanceOf[HasCommitments].commitments.remoteChanges.proposed.contains(fulfill))
@ -145,7 +145,7 @@ trait StateTestsHelperMethods extends TestKitBase with fixture.TestSuite with Pa
val rCommitIndex = r.stateData.asInstanceOf[HasCommitments].commitments.localCommit.index
val rHasChanges = Commitments.localHasChanges(r.stateData.asInstanceOf[HasCommitments].commitments)
sender.send(s, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
s2r.expectMsgType[CommitSig]
s2r.forward(r)
r2s.expectMsgType[RevokeAndAck]

View File

@ -74,7 +74,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val h = randomBytes32
val add = CMD_ADD_HTLC(50000000 msat, h, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, Upstream.Local(UUID.randomUUID()))
sender.send(alice, add)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
val e = listener.expectMsgType[AvailableBalanceChanged]
assert(e.commitments.availableBalanceForSend < initialState.commitments.availableBalanceForSend)
val htlc = alice2bob.expectMsgType[UpdateAddHtlc]
@ -93,7 +93,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val h = randomBytes32
for (i <- 0 until 10) {
sender.send(alice, CMD_ADD_HTLC(50000000 msat, h, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, Upstream.Local(UUID.randomUUID())))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
val htlc = alice2bob.expectMsgType[UpdateAddHtlc]
assert(htlc.id == i && htlc.paymentHash == h)
}
@ -107,7 +107,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val originHtlc = UpdateAddHtlc(channelId = randomBytes32, id = 5656, amountMsat = 50000000 msat, cltvExpiry = CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), paymentHash = h, onionRoutingPacket = TestConstants.emptyOnionPacket)
val cmd = CMD_ADD_HTLC(originHtlc.amountMsat - 10000.msat, h, originHtlc.cltvExpiry - CltvExpiryDelta(7), TestConstants.emptyOnionPacket, Upstream.Relayed(originHtlc))
sender.send(alice, cmd)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
val htlc = alice2bob.expectMsgType[UpdateAddHtlc]
assert(htlc.id == 0 && htlc.paymentHash == h)
awaitCond(alice.stateData == initialState.copy(
@ -128,7 +128,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val upstream = Upstream.TrampolineRelayed(originHtlc1 :: originHtlc2 :: Nil)
val cmd = CMD_ADD_HTLC(originHtlc1.amountMsat + originHtlc2.amountMsat - 10000.msat, h, originHtlc2.cltvExpiry - CltvExpiryDelta(7), TestConstants.emptyOnionPacket, upstream)
sender.send(alice, cmd)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
val htlc = alice2bob.expectMsgType[UpdateAddHtlc]
assert(htlc.id == 0 && htlc.paymentHash == h)
awaitCond(alice.stateData == initialState.copy(
@ -181,7 +181,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val h = randomBytes32
val add = CMD_ADD_HTLC(50000000 msat, h, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, Upstream.Local(UUID.randomUUID()))
sender.send(alice, add)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
}
test("recv CMD_ADD_HTLC (insufficient funds)") { f =>
@ -227,13 +227,13 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val sender = TestProbe()
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
sender.send(alice, CMD_ADD_HTLC(500000000 msat, randomBytes32, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, Upstream.Local(UUID.randomUUID())))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[UpdateAddHtlc]
sender.send(alice, CMD_ADD_HTLC(200000000 msat, randomBytes32, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, Upstream.Local(UUID.randomUUID())))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[UpdateAddHtlc]
sender.send(alice, CMD_ADD_HTLC(67600000 msat, randomBytes32, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, Upstream.Local(UUID.randomUUID())))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[UpdateAddHtlc]
val add = CMD_ADD_HTLC(1000000 msat, randomBytes32, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, Upstream.Local(UUID.randomUUID()))
sender.send(alice, add)
@ -247,10 +247,10 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val sender = TestProbe()
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
sender.send(alice, CMD_ADD_HTLC(300000000 msat, randomBytes32, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, Upstream.Local(UUID.randomUUID())))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[UpdateAddHtlc]
sender.send(alice, CMD_ADD_HTLC(300000000 msat, randomBytes32, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, Upstream.Local(UUID.randomUUID())))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[UpdateAddHtlc]
val add = CMD_ADD_HTLC(500000000 msat, randomBytes32, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, Upstream.Local(UUID.randomUUID()))
sender.send(alice, add)
@ -276,7 +276,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val initialState = bob.stateData.asInstanceOf[DATA_NORMAL]
val add = CMD_ADD_HTLC(75500000 msat, randomBytes32, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, Upstream.Local(UUID.randomUUID()))
sender.send(bob, add)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
bob2alice.expectMsgType[UpdateAddHtlc]
val add1 = CMD_ADD_HTLC(75500000 msat, randomBytes32, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, Upstream.Local(UUID.randomUUID()))
sender.send(bob, add1)
@ -292,7 +292,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
// Bob accepts a maximum of 30 htlcs
for (i <- 0 until 30) {
sender.send(alice, CMD_ADD_HTLC(10000000 msat, randomBytes32, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, Upstream.Local(UUID.randomUUID())))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[UpdateAddHtlc]
}
val add = CMD_ADD_HTLC(10000000 msat, randomBytes32, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, Upstream.Local(UUID.randomUUID()))
@ -308,10 +308,10 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
val add1 = CMD_ADD_HTLC(TestConstants.fundingSatoshis.toMilliSatoshi * 2 / 3, randomBytes32, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, Upstream.Local(UUID.randomUUID()))
sender.send(alice, add1)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[UpdateAddHtlc]
sender.send(alice, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[CommitSig]
// this is over channel-capacity
val add2 = CMD_ADD_HTLC(TestConstants.fundingSatoshis.toMilliSatoshi * 2 / 3, randomBytes32, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, Upstream.Local(UUID.randomUUID()))
@ -326,7 +326,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val sender = TestProbe()
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
sender.send(alice, CMD_CLOSE(None))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[Shutdown]
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].localShutdown.isDefined && alice.stateData.asInstanceOf[DATA_NORMAL].remoteShutdown.isEmpty)
@ -345,10 +345,10 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
// let's make alice send an htlc
val add1 = CMD_ADD_HTLC(500000000 msat, randomBytes32, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, Upstream.Local(UUID.randomUUID()))
sender.send(alice, add1)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
// at the same time bob initiates a closing
sender.send(bob, CMD_CLOSE(None))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
// this command will be received by alice right after having received the shutdown
val add2 = CMD_ADD_HTLC(100000000 msat, randomBytes32, CltvExpiry(300000), TestConstants.emptyOnionPacket, Upstream.Local(UUID.randomUUID()))
// messages cross
@ -488,7 +488,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val sender = TestProbe()
val (r, htlc) = addHtlc(50000000 msat, alice, bob, alice2bob, bob2alice)
sender.send(alice, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
val commitSig = alice2bob.expectMsgType[CommitSig]
assert(commitSig.htlcSignatures.size == 1)
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.remoteNextCommitInfo.isLeft)
@ -499,28 +499,28 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val sender = TestProbe()
val add = CMD_ADD_HTLC(10000000 msat, randomBytes32, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, Upstream.Local(UUID.randomUUID()))
sender.send(alice, add)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[UpdateAddHtlc]
alice2bob.forward(bob)
sender.send(alice, add)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[UpdateAddHtlc]
alice2bob.forward(bob)
crossSign(alice, bob, alice2bob, bob2alice)
sender.send(bob, add)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
bob2alice.expectMsgType[UpdateAddHtlc]
bob2alice.forward(alice)
sender.send(bob, add)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
bob2alice.expectMsgType[UpdateAddHtlc]
bob2alice.forward(alice)
// actual test starts here
sender.send(bob, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
val commitSig = bob2alice.expectMsgType[CommitSig]
assert(commitSig.htlcSignatures.toSet.size == 4)
}
@ -545,19 +545,19 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
assert(b2a_1 > aliceMinReceive && b2a_1 > bobMinOffer)
assert(b2a_2 < aliceMinReceive && b2a_2 > bobMinOffer)
sender.send(alice, CMD_ADD_HTLC(a2b_1.toMilliSatoshi, randomBytes32, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, Upstream.Local(UUID.randomUUID())))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[UpdateAddHtlc]
alice2bob.forward(bob)
sender.send(alice, CMD_ADD_HTLC(a2b_2.toMilliSatoshi, randomBytes32, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, Upstream.Local(UUID.randomUUID())))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[UpdateAddHtlc]
alice2bob.forward(bob)
sender.send(bob, CMD_ADD_HTLC(b2a_1.toMilliSatoshi, randomBytes32, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, Upstream.Local(UUID.randomUUID())))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
bob2alice.expectMsgType[UpdateAddHtlc]
bob2alice.forward(alice)
sender.send(bob, CMD_ADD_HTLC(b2a_2.toMilliSatoshi, randomBytes32, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, Upstream.Local(UUID.randomUUID())))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
bob2alice.expectMsgType[UpdateAddHtlc]
bob2alice.forward(alice)
@ -581,13 +581,13 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val htlcCount = epsilons.size
for (i <- epsilons) {
sender.send(alice, add.copy(amount = add.amount + (i * 1000).msat))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[UpdateAddHtlc]
alice2bob.forward(bob)
}
// actual test starts here
sender.send(alice, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
val commitSig = alice2bob.expectMsgType[CommitSig]
assert(commitSig.htlcSignatures.toSet.size == htlcCount)
alice2bob.forward(bob)
@ -611,7 +611,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val (r, htlc) = addHtlc(50000000 msat, alice, bob, alice2bob, bob2alice)
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.remoteNextCommitInfo.isRight)
sender.send(alice, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[CommitSig]
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.remoteNextCommitInfo.isLeft)
val waitForRevocation = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.remoteNextCommitInfo.left.toOption.get
@ -629,7 +629,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val (r1, htlc1) = addHtlc(50000000 msat, alice, bob, alice2bob, bob2alice)
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.remoteNextCommitInfo.isRight)
sender.send(alice, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[CommitSig]
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.remoteNextCommitInfo.isLeft)
val waitForRevocation = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.remoteNextCommitInfo.left.toOption.get
@ -651,7 +651,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val (r, htlc) = addHtlc(50000000 msat, alice, bob, alice2bob, bob2alice)
crossSign(alice, bob, alice2bob, bob2alice)
sender.send(bob, CMD_FULFILL_HTLC(htlc.id, r))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
bob2alice.expectMsgType[UpdateFulfillHtlc]
// we listen to channel_update events
val listener = TestProbe()
@ -660,7 +660,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
// actual test starts here
// when signing the fulfill, bob will have its main output go above reserve in alice's commitment tx
sender.send(bob, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
bob2alice.expectMsgType[CommitSig]
// it should update its channel_update
awaitCond(Announcements.isEnabled(bob.stateData.asInstanceOf[DATA_NORMAL].channelUpdate.channelFlags))
@ -675,7 +675,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val listener = TestProbe()
system.eventStream.subscribe(listener.ref, classOf[AvailableBalanceChanged])
sender.send(alice, CMD_UPDATE_FEE(654564))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[UpdateFee]
sender.send(alice, CMD_SIGN)
listener.expectMsgType[AvailableBalanceChanged]
@ -689,7 +689,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val initialState = bob.stateData.asInstanceOf[DATA_NORMAL]
sender.send(alice, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
// actual test begins
alice2bob.expectMsgType[CommitSig]
@ -714,7 +714,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val initialState = bob.stateData.asInstanceOf[DATA_NORMAL]
sender.send(alice, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[CommitSig]
alice2bob.forward(bob)
bob2alice.expectMsgType[RevokeAndAck]
@ -748,7 +748,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val (r7, htlc7) = addHtlc(4000000 msat, bob, alice, bob2alice, alice2bob) // b->a (regular)
sender.send(alice, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[CommitSig]
alice2bob.forward(bob)
bob2alice.expectMsgType[RevokeAndAck]
@ -767,9 +767,9 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val sender = TestProbe()
sender.send(alice, CMD_UPDATE_FEE(TestConstants.feeratePerKw + 1000, commit = false))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
sender.send(alice, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
// actual test begins (note that channel sends a CMD_SIGN to itself when it receives RevokeAndAck and there are changes)
val updateFee = alice2bob.expectMsgType[UpdateFee]
@ -788,12 +788,12 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val h = Crypto.sha256(r)
sender.send(alice, CMD_ADD_HTLC(50000000 msat, h, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, Upstream.Local(UUID.randomUUID())))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
val htlc1 = alice2bob.expectMsgType[UpdateAddHtlc]
alice2bob.forward(bob)
sender.send(alice, CMD_ADD_HTLC(50000000 msat, h, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, Upstream.Local(UUID.randomUUID())))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
val htlc2 = alice2bob.expectMsgType[UpdateAddHtlc]
alice2bob.forward(bob)
@ -847,7 +847,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val tx = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.localCommit.publishableTxs.commitTx.tx
sender.send(alice, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
val commitSig = alice2bob.expectMsgType[CommitSig]
// actual test begins
@ -868,7 +868,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val tx = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.localCommit.publishableTxs.commitTx.tx
sender.send(alice, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
val commitSig = alice2bob.expectMsgType[CommitSig]
// actual test begins
@ -887,7 +887,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val (r, htlc) = addHtlc(50000000 msat, alice, bob, alice2bob, bob2alice)
sender.send(alice, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[CommitSig]
alice2bob.forward(bob)
@ -905,7 +905,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val (_, htlc) = addHtlc(50000000 msat, alice, bob, alice2bob, bob2alice)
sender.send(alice, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[CommitSig]
alice2bob.forward(bob)
bob2alice.expectMsgType[RevokeAndAck]
@ -945,7 +945,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val (r7, htlc7) = addHtlc(4000000 msat, bob, alice, bob2alice, alice2bob) // b->a (regular)
sender.send(alice, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[CommitSig]
alice2bob.forward(bob)
bob2alice.expectMsgType[RevokeAndAck]
@ -970,7 +970,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val (r1, htlc1) = addHtlc(50000000 msat, alice, bob, alice2bob, bob2alice)
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.remoteNextCommitInfo.isRight)
sender.send(alice, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[CommitSig]
alice2bob.forward(bob)
val (r2, htlc2) = addHtlc(50000000 msat, alice, bob, alice2bob, bob2alice)
@ -991,7 +991,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val (r, htlc) = addHtlc(50000000 msat, alice, bob, alice2bob, bob2alice)
sender.send(alice, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[CommitSig]
alice2bob.forward(bob)
@ -1028,11 +1028,11 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val (_, htlc) = addHtlc(50000000 msat, alice, bob, alice2bob, bob2alice)
crossSign(alice, bob, alice2bob, bob2alice)
sender.send(bob, CMD_FAIL_HTLC(htlc.id, Right(PermanentChannelFailure)))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
val fail = bob2alice.expectMsgType[UpdateFailHtlc]
bob2alice.forward(alice)
sender.send(bob, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
bob2alice.expectMsgType[CommitSig]
bob2alice.forward(alice)
alice2bob.expectMsgType[RevokeAndAck]
@ -1057,11 +1057,11 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val (_, htlc) = addHtlc(50000000 msat, alice, bob, alice2bob, bob2alice)
crossSign(alice, bob, alice2bob, bob2alice)
sender.send(bob, CMD_FAIL_MALFORMED_HTLC(htlc.id, Sphinx.PaymentPacket.hash(htlc.onionRoutingPacket), FailureMessageCodecs.BADONION))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
val fail = bob2alice.expectMsgType[UpdateFailMalformedHtlc]
bob2alice.forward(alice)
sender.send(bob, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
bob2alice.expectMsgType[CommitSig]
bob2alice.forward(alice)
alice2bob.expectMsgType[RevokeAndAck]
@ -1086,7 +1086,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val (r, htlc) = addHtlc(50000000 msat, alice, bob, alice2bob, bob2alice)
sender.send(alice, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[CommitSig]
alice2bob.forward(bob)
@ -1106,7 +1106,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
// actual test begins
val initialState = bob.stateData.asInstanceOf[DATA_NORMAL]
sender.send(bob, CMD_FULFILL_HTLC(htlc.id, r))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
val fulfill = bob2alice.expectMsgType[UpdateFulfillHtlc]
awaitCond(bob.stateData == initialState.copy(
commitments = initialState.commitments.copy(
@ -1153,7 +1153,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val (r, htlc) = addHtlc(50000000 msat, alice, bob, alice2bob, bob2alice)
crossSign(alice, bob, alice2bob, bob2alice)
sender.send(bob, CMD_FULFILL_HTLC(htlc.id, r))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
val fulfill = bob2alice.expectMsgType[UpdateFulfillHtlc]
// actual test begins
@ -1172,7 +1172,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val sender = TestProbe()
val (r, htlc) = addHtlc(50000000 msat, alice, bob, alice2bob, bob2alice)
sender.send(alice, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[CommitSig]
// actual test begins
@ -1231,7 +1231,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
// actual test begins
val initialState = bob.stateData.asInstanceOf[DATA_NORMAL]
sender.send(bob, CMD_FAIL_HTLC(htlc.id, Right(PermanentChannelFailure)))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
val fail = bob2alice.expectMsgType[UpdateFailHtlc]
awaitCond(bob.stateData == initialState.copy(
commitments = initialState.commitments.copy(
@ -1269,7 +1269,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
// actual test begins
val initialState = bob.stateData.asInstanceOf[DATA_NORMAL]
sender.send(bob, CMD_FAIL_MALFORMED_HTLC(htlc.id, Sphinx.PaymentPacket.hash(htlc.onionRoutingPacket), FailureMessageCodecs.BADONION))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
val fail = bob2alice.expectMsgType[UpdateFailMalformedHtlc]
awaitCond(bob.stateData == initialState.copy(
commitments = initialState.commitments.copy(
@ -1311,7 +1311,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val (_, htlc) = addHtlc(50000000 msat, alice, bob, alice2bob, bob2alice)
crossSign(alice, bob, alice2bob, bob2alice)
sender.send(bob, CMD_FAIL_HTLC(htlc.id, Right(PermanentChannelFailure)))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
val fail = bob2alice.expectMsgType[UpdateFailHtlc]
// actual test begins
@ -1333,7 +1333,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
// Bob fails the HTLC because he cannot parse it
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
sender.send(bob, CMD_FAIL_MALFORMED_HTLC(htlc.id, Sphinx.PaymentPacket.hash(htlc.onionRoutingPacket), FailureMessageCodecs.BADONION))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
val fail = bob2alice.expectMsgType[UpdateFailMalformedHtlc]
bob2alice.forward(alice)
@ -1377,7 +1377,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val sender = TestProbe()
val (r, htlc) = addHtlc(50000000 msat, alice, bob, alice2bob, bob2alice)
sender.send(alice, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[CommitSig]
// actual test begins
@ -1413,7 +1413,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
crossSign(alice, bob, alice2bob, bob2alice)
// Bob receives a failure with a completely invalid onion error (missing mac)
sender.send(bob, CMD_FAIL_HTLC(htlc.id, Left(ByteVector.fill(260)(42))))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
val fail = bob2alice.expectMsgType[UpdateFailHtlc]
assert(fail.id === htlc.id)
// We should rectify the packet length before forwarding upstream.
@ -1425,7 +1425,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val sender = TestProbe()
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
sender.send(alice, CMD_UPDATE_FEE(20000))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
val fee = alice2bob.expectMsgType[UpdateFee]
awaitCond(alice.stateData == initialState.copy(
commitments = initialState.commitments.copy(
@ -1437,10 +1437,10 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val sender = TestProbe()
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
sender.send(alice, CMD_UPDATE_FEE(20000))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
val fee1 = alice2bob.expectMsgType[UpdateFee]
sender.send(alice, CMD_UPDATE_FEE(30000))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
val fee2 = alice2bob.expectMsgType[UpdateFee]
awaitCond(alice.stateData == initialState.copy(
commitments = initialState.commitments.copy(
@ -1550,7 +1550,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val newFeeBaseMsat = TestConstants.Alice.nodeParams.feeBase * 2
val newFeeProportionalMillionth = TestConstants.Alice.nodeParams.feeProportionalMillionth * 2
sender.send(alice, CMD_UPDATE_RELAY_FEE(newFeeBaseMsat, newFeeProportionalMillionth))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
val localUpdate = channelUpdateListener.expectMsgType[LocalChannelUpdate]
assert(localUpdate.channelUpdate.feeBaseMsat == newFeeBaseMsat)
@ -1563,7 +1563,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val sender = TestProbe()
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].localShutdown.isEmpty)
sender.send(alice, CMD_CLOSE(None))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[Shutdown]
awaitCond(alice.stateName == NORMAL)
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].localShutdown.isDefined)
@ -1590,7 +1590,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val (r, htlc) = addHtlc(50000000 msat, alice, bob, alice2bob, bob2alice)
crossSign(alice, bob, alice2bob, bob2alice)
sender.send(alice, CMD_CLOSE(None))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[Shutdown]
awaitCond(alice.stateName == NORMAL)
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].localShutdown.isDefined)
@ -1601,7 +1601,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val sender = TestProbe()
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].localShutdown.isEmpty)
sender.send(alice, CMD_CLOSE(None))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[Shutdown]
awaitCond(alice.stateName == NORMAL)
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].localShutdown.isDefined)
@ -1614,11 +1614,11 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val sender = TestProbe()
val (r, htlc) = addHtlc(50000000 msat, alice, bob, alice2bob, bob2alice)
sender.send(alice, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[CommitSig]
// actual test begins
sender.send(alice, CMD_CLOSE(None))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[Shutdown]
awaitCond(alice.stateName == NORMAL)
}
@ -1712,7 +1712,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val sender = TestProbe()
val (r, htlc) = addHtlc(50000000 msat, alice, bob, alice2bob, bob2alice)
sender.send(alice, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[CommitSig]
sender.send(bob, CMD_CLOSE(None))
bob2alice.expectMsgType[Shutdown]
@ -1732,7 +1732,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val (r, htlc) = addHtlc(50000000 msat, alice, bob, alice2bob, bob2alice)
// now we can sign
sender.send(alice, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[CommitSig]
alice2bob.forward(bob)
// adding an outgoing pending htlc
@ -1813,7 +1813,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val HtlcSuccessTx(_, htlcSuccessTx, _) = initialState.commitments.localCommit.publishableTxs.htlcTxsAndSigs.head.txinfo
sender.send(bob, CMD_FULFILL_HTLC(htlc.id, r, commit = true))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
bob2alice.expectMsgType[UpdateFulfillHtlc]
sender.send(bob, CurrentBlockCount((htlc.cltvExpiry - Bob.nodeParams.fulfillSafetyBeforeTimeoutBlocks).toLong))
@ -1848,7 +1848,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val HtlcSuccessTx(_, htlcSuccessTx, _) = initialState.commitments.localCommit.publishableTxs.htlcTxsAndSigs.head.txinfo
sender.send(bob, CMD_FULFILL_HTLC(htlc.id, r, commit = false))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
bob2alice.expectMsgType[UpdateFulfillHtlc]
sender.send(bob, CurrentBlockCount((htlc.cltvExpiry - Bob.nodeParams.fulfillSafetyBeforeTimeoutBlocks).toLong))
@ -1883,7 +1883,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val HtlcSuccessTx(_, htlcSuccessTx, _) = initialState.commitments.localCommit.publishableTxs.htlcTxsAndSigs.head.txinfo
sender.send(bob, CMD_FULFILL_HTLC(htlc.id, r, commit = true))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
bob2alice.expectMsgType[UpdateFulfillHtlc]
bob2alice.forward(alice)
bob2alice.expectMsgType[CommitSig]
@ -2016,7 +2016,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
fulfillHtlc(0, rb1, alice, bob, alice2bob, bob2alice)
// alice sign but we intercept bob's revocation
sender.send(alice, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[CommitSig]
alice2bob.forward(bob)
bob2alice.expectMsgType[RevokeAndAck]
@ -2132,11 +2132,11 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val add = CMD_ADD_HTLC(10000000 msat, randomBytes32, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, Upstream.Local(UUID.randomUUID()))
sender.send(alice, add)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[UpdateAddHtlc]
alice2bob.forward(bob)
sender.send(alice, add)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[UpdateAddHtlc]
alice2bob.forward(bob)
@ -2145,7 +2145,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val revokedTx = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.localCommit.publishableTxs.commitTx.tx
sender.send(alice, add)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[UpdateAddHtlc]
alice2bob.forward(bob)

View File

@ -25,7 +25,6 @@ import fr.acinq.bitcoin.{ByteVector32, ScriptFlags, Transaction}
import fr.acinq.eclair.TestConstants.Alice
import fr.acinq.eclair.blockchain._
import fr.acinq.eclair.blockchain.fee.FeeratesPerKw
import fr.acinq.eclair.channel.Channel.LocalError
import fr.acinq.eclair.channel._
import fr.acinq.eclair.channel.states.StateTestsHelperMethods
import fr.acinq.eclair.payment.relay.CommandBuffer
@ -34,7 +33,6 @@ import fr.acinq.eclair.transactions.Transactions.HtlcSuccessTx
import fr.acinq.eclair.wire._
import fr.acinq.eclair.{CltvExpiry, CltvExpiryDelta, LongToBtcAmount, TestConstants, TestkitBaseClass, randomBytes32}
import org.scalatest.{Outcome, Tag}
import scodec.bits.ByteVector
import scala.concurrent.duration._
@ -276,7 +274,7 @@ class OfflineStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
// then we add an htlc and sign it
addHtlc(250000000 msat, alice, bob, alice2bob, bob2alice)
sender.send(alice, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[CommitSig]
alice2bob.forward(bob)
// alice will receive neither the revocation nor the commit sig
@ -361,7 +359,7 @@ class OfflineStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
// we make alice update here relay fee
sender.send(alice, CMD_UPDATE_RELAY_FEE(4200 msat, 123456))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
// alice doesn't broadcast the new channel_update yet
channelUpdateListener.expectNoMsg(300 millis)

View File

@ -61,7 +61,7 @@ class ShutdownStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val expiry1 = CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight)
val cmd1 = OutgoingPacket.buildCommand(Upstream.Local(UUID.randomUUID), h1, ChannelHop(null, TestConstants.Bob.nodeParams.nodeId, null) :: Nil, FinalLegacyPayload(amount1, expiry1))._1.copy(commit = false)
sender.send(alice, cmd1)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
val htlc1 = alice2bob.expectMsgType[UpdateAddHtlc]
alice2bob.forward(bob)
awaitCond(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.remoteChanges.proposed == htlc1 :: Nil)
@ -71,13 +71,13 @@ class ShutdownStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val expiry2 = CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight)
val cmd2 = OutgoingPacket.buildCommand(Upstream.Local(UUID.randomUUID), h2, ChannelHop(null, TestConstants.Bob.nodeParams.nodeId, null) :: Nil, FinalLegacyPayload(amount2, expiry2))._1.copy(commit = false)
sender.send(alice, cmd2)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
val htlc2 = alice2bob.expectMsgType[UpdateAddHtlc]
alice2bob.forward(bob)
awaitCond(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.remoteChanges.proposed == htlc1 :: htlc2 :: Nil)
// alice signs
sender.send(alice, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[CommitSig]
alice2bob.forward(bob)
bob2alice.expectMsgType[RevokeAndAck]
@ -118,7 +118,7 @@ class ShutdownStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val sender = TestProbe()
val initialState = bob.stateData.asInstanceOf[DATA_SHUTDOWN]
sender.send(bob, CMD_FULFILL_HTLC(0, r1))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
val fulfill = bob2alice.expectMsgType[UpdateFulfillHtlc]
awaitCond(bob.stateData == initialState.copy(
commitments = initialState.commitments.copy(
@ -200,7 +200,7 @@ class ShutdownStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val sender = TestProbe()
val initialState = bob.stateData.asInstanceOf[DATA_SHUTDOWN]
sender.send(bob, CMD_FAIL_HTLC(1, Right(PermanentChannelFailure)))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
val fail = bob2alice.expectMsgType[UpdateFailHtlc]
awaitCond(bob.stateData == initialState.copy(
commitments = initialState.commitments.copy(
@ -230,7 +230,7 @@ class ShutdownStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val sender = TestProbe()
val initialState = bob.stateData.asInstanceOf[DATA_SHUTDOWN]
sender.send(bob, CMD_FAIL_MALFORMED_HTLC(1, Crypto.sha256(ByteVector.empty), FailureMessageCodecs.BADONION))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
val fail = bob2alice.expectMsgType[UpdateFailMalformedHtlc]
awaitCond(bob.stateData == initialState.copy(
commitments = initialState.commitments.copy(
@ -329,7 +329,7 @@ class ShutdownStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
alice2bob.expectMsgType[RevokeAndAck]
alice2bob.forward(bob)
sender.send(alice, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[CommitSig]
awaitCond(alice.stateData.asInstanceOf[DATA_SHUTDOWN].commitments.remoteNextCommitInfo.isLeft)
}
@ -346,10 +346,10 @@ class ShutdownStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
import f._
val sender = TestProbe()
sender.send(bob, CMD_FULFILL_HTLC(0, r1))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
bob2alice.expectMsgType[UpdateFulfillHtlc]
sender.send(bob, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
bob2alice.expectMsgType[CommitSig]
awaitCond(bob.stateData.asInstanceOf[DATA_SHUTDOWN].commitments.remoteNextCommitInfo.isLeft)
val waitForRevocation = bob.stateData.asInstanceOf[DATA_SHUTDOWN].commitments.remoteNextCommitInfo.left.toOption.get
@ -365,11 +365,11 @@ class ShutdownStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
import f._
val sender = TestProbe()
sender.send(bob, CMD_FULFILL_HTLC(0, r1))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
bob2alice.expectMsgType[UpdateFulfillHtlc]
bob2alice.forward(alice)
sender.send(bob, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
bob2alice.expectMsgType[CommitSig]
bob2alice.forward(alice)
alice2bob.expectMsgType[RevokeAndAck]
@ -417,7 +417,7 @@ class ShutdownStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
fulfillHtlc(1, r2, bob, alice, bob2alice, alice2bob)
val sender = TestProbe()
sender.send(bob, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
bob2alice.expectMsgType[CommitSig]
bob2alice.forward(alice)
alice2bob.expectMsgType[RevokeAndAck]
@ -442,11 +442,11 @@ class ShutdownStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val tx = bob.stateData.asInstanceOf[DATA_SHUTDOWN].commitments.localCommit.publishableTxs.commitTx.tx
val sender = TestProbe()
sender.send(bob, CMD_FULFILL_HTLC(0, r1))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
bob2alice.expectMsgType[UpdateFulfillHtlc]
bob2alice.forward(alice)
sender.send(bob, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
bob2alice.expectMsgType[CommitSig]
bob2alice.forward(alice)
alice2bob.expectMsgType[RevokeAndAck]
@ -482,11 +482,11 @@ class ShutdownStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
import f._
val sender = TestProbe()
sender.send(bob, CMD_FAIL_HTLC(1, Right(PermanentChannelFailure)))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
val fail = bob2alice.expectMsgType[UpdateFailHtlc]
bob2alice.forward(alice)
sender.send(bob, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
bob2alice.expectMsgType[CommitSig]
bob2alice.forward(alice)
alice2bob.expectMsgType[RevokeAndAck]
@ -508,11 +508,11 @@ class ShutdownStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
import f._
val sender = TestProbe()
sender.send(bob, CMD_FAIL_MALFORMED_HTLC(1, Crypto.sha256(ByteVector.view("should be htlc.onionRoutingPacket".getBytes())), FailureMessageCodecs.BADONION))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
val fail = bob2alice.expectMsgType[UpdateFailMalformedHtlc]
bob2alice.forward(alice)
sender.send(bob, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
bob2alice.expectMsgType[CommitSig]
bob2alice.forward(alice)
alice2bob.expectMsgType[RevokeAndAck]
@ -535,7 +535,7 @@ class ShutdownStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val sender = TestProbe()
val initialState = alice.stateData.asInstanceOf[DATA_SHUTDOWN]
sender.send(alice, CMD_UPDATE_FEE(20000))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
val fee = alice2bob.expectMsgType[UpdateFee]
awaitCond(alice.stateData == initialState.copy(
commitments = initialState.commitments.copy(
@ -716,7 +716,7 @@ class ShutdownStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
// then signs
val sender = TestProbe()
sender.send(bob, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
bob2alice.expectMsgType[CommitSig]
bob2alice.forward(alice)
alice2bob.expectMsgType[RevokeAndAck]

View File

@ -27,7 +27,7 @@ import fr.acinq.eclair.blockchain._
import fr.acinq.eclair.blockchain.fee.FeeratesPerKw
import fr.acinq.eclair.channel.Helpers.Closing
import fr.acinq.eclair.channel.states.StateTestsHelperMethods
import fr.acinq.eclair.channel.{Data, State, _}
import fr.acinq.eclair.channel._
import fr.acinq.eclair.payment._
import fr.acinq.eclair.payment.relay.Relayer._
import fr.acinq.eclair.payment.relay.{CommandBuffer, Origin}
@ -457,7 +457,7 @@ class ClosingStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val (_, htlc) = addHtlc(4200000 msat, alice, bob, alice2bob, bob2alice)
// and signs it (but bob doesn't sign it)
sender.send(alice, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[CommitSig]
// note that bob doesn't receive the new sig!
// then we make alice unilaterally close the channel
@ -488,7 +488,7 @@ class ClosingStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val (_, htlc) = addHtlc(4200000 msat, alice, bob, alice2bob, bob2alice)
// and signs it (but bob doesn't sign it)
sender.send(alice, CMD_SIGN)
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
alice2bob.expectMsgType[CommitSig]
// then we make alice believe bob unilaterally close the channel
alice ! WatchEventSpent(BITCOIN_FUNDING_SPENT, bobCommitTx)

View File

@ -21,19 +21,17 @@ import java.util.UUID
import fr.acinq.bitcoin.Crypto.PrivateKey
import fr.acinq.bitcoin.{ByteVector32, Transaction}
import fr.acinq.eclair._
import fr.acinq.eclair.channel.Channel.{LocalError, RemoteError}
import fr.acinq.eclair.channel.{AvailableBalanceChanged, ChannelErrorOccurred, NetworkFeePaid}
import fr.acinq.eclair.channel.{ChannelErrorOccurred, LocalError, NetworkFeePaid, RemoteError}
import fr.acinq.eclair.db.sqlite.SqliteAuditDb
import fr.acinq.eclair.db.sqlite.SqliteUtils.{getVersion, using}
import fr.acinq.eclair.payment._
import fr.acinq.eclair.wire.{ChannelCodecs, ChannelCodecsSpec}
import fr.acinq.eclair.wire.ChannelCodecs
import org.scalatest.{FunSuite, Tag}
import scala.compat.Platform
import scala.concurrent.duration._
import scala.util.Random
class SqliteAuditDbSpec extends FunSuite {
test("init sqlite 2 times in a row") {

View File

@ -179,7 +179,7 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
fundingTxFeeratePerKw_opt = None,
channelFlags = None,
timeout_opt = None))
assert(sender.expectMsgType[String](10 seconds).startsWith("created channel"))
sender.expectMsgType[ChannelCommandResponse.ChannelOpened](10 seconds)
}
test("connect nodes") {
@ -795,7 +795,7 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
}, max = 20 seconds, interval = 1 second)
// we then have C unilateral close the channel (which will make F redeem the htlc onchain)
sender.send(nodes("C").register, Forward(htlc.channelId, CMD_FORCECLOSE))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
// we then wait for F to detect the unilateral close and go to CLOSING state
awaitCond({
sender.send(nodes("F1").register, Forward(htlc.channelId, CMD_GETSTATE))
@ -878,7 +878,7 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
}, max = 20 seconds, interval = 1 second)
// then we have F unilateral close the channel
sender.send(nodes("F2").register, Forward(htlc.channelId, CMD_FORCECLOSE))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
// we then fulfill the htlc (it won't be sent to C, and will be used to pull funds on-chain)
sender.send(nodes("F2").register, Forward(htlc.channelId, CMD_FULFILL_HTLC(htlc.id, preimage)))
// we then generate one block so that the htlc success tx gets written to the blockchain
@ -998,7 +998,7 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
val previouslyReceivedByC = res.filter(_ \ "address" == JString(finalAddressC)).flatMap(_ \ "txids" \\ classOf[JString])
// then we ask F to unilaterally close the channel
sender.send(nodes("F4").register, Forward(htlc.channelId, CMD_FORCECLOSE))
sender.expectMsg("ok")
sender.expectMsg(ChannelCommandResponse.Ok)
// we then generate enough blocks to make the htlc timeout
sender.send(bitcoincli, BitcoinReq("getnewaddress"))
val JString(address) = sender.expectMsgType[JValue]

View File

@ -104,7 +104,7 @@ class SynchronizationPipe(latch: CountDownLatch) extends Actor with ActorLogging
import scala.io.Source
val script = Source.fromFile(file).getLines().filterNot(_.startsWith("#")).toList
exec(script, a, b)
case "ok" => {}
case ChannelCommandResponse.Ok => {}
case msg if sender() == a =>
log.info(s"a -> b $msg")
b forward msg
@ -115,7 +115,7 @@ class SynchronizationPipe(latch: CountDownLatch) extends Actor with ActorLogging
}
def wait(a: ActorRef, b: ActorRef, script: List[String]): Receive = {
case "ok" => {}
case ChannelCommandResponse.Ok => {}
case msg if sender() == a && script.head.startsWith("B:recv") =>
log.info(s"a -> b $msg")
b forward msg

View File

@ -23,7 +23,7 @@ import com.google.common.net.HostAndPort
import de.heikoseeberger.akkahttpjson4s.Json4sSupport
import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey}
import fr.acinq.bitcoin.{ByteVector32, ByteVector64, OutPoint, Satoshi, Transaction}
import fr.acinq.eclair.channel.{ChannelVersion, State}
import fr.acinq.eclair.channel.{ChannelCommandResponse, ChannelVersion, State}
import fr.acinq.eclair.crypto.ShaChain
import fr.acinq.eclair.db.{IncomingPaymentStatus, OutgoingPaymentStatus}
import fr.acinq.eclair.payment._
@ -124,6 +124,12 @@ class ChannelVersionSerializer extends CustomSerializer[ChannelVersion](_ => ( {
case x: ChannelVersion => JString(x.bits.toBin)
}))
class ChannelCommandResponseSerializer extends CustomSerializer[ChannelCommandResponse](_ => ( {
null
}, {
case x: ChannelCommandResponse => JString(x.toString)
}))
class TransactionSerializer extends CustomSerializer[TransactionWithInputInfo](_ => ( {
null
}, {
@ -295,6 +301,7 @@ object JsonSupport extends Json4sSupport {
new OutPointSerializer +
new OutPointKeySerializer +
new ChannelVersionSerializer +
new ChannelCommandResponseSerializer +
new InputInfoSerializer +
new ColorSerializer +
new RouteResponseSerializer +

View File

@ -1 +1 @@
"03af0ed6052cf28d670665549bc86f4b721c9fdb309d40c58f5811f63966e005d0"
"closed channel 56d7d6eda04d80138270c49709f1eadb5ab4939e5061309ccdacdb98ce637d0e"

View File

@ -30,6 +30,7 @@ import de.heikoseeberger.akkahttpjson4s.Json4sSupport
import fr.acinq.bitcoin.Crypto.PublicKey
import fr.acinq.bitcoin.{Block, ByteVector32}
import fr.acinq.eclair._
import fr.acinq.eclair.channel.ChannelCommandResponse
import fr.acinq.eclair.db._
import fr.acinq.eclair.io.NodeURI
import fr.acinq.eclair.io.Peer.PeerInfo
@ -192,7 +193,7 @@ class ApiServiceSpec extends FunSuite with ScalatestRouteTest with IdiomaticMock
val channelId = "56d7d6eda04d80138270c49709f1eadb5ab4939e5061309ccdacdb98ce637d0e"
val eclair = mock[Eclair]
eclair.close(any, any)(any[Timeout]) returns Future.successful(aliceNodeId.toString())
eclair.close(any, any)(any[Timeout]) returns Future.successful(ChannelCommandResponse.ChannelClosed(ByteVector32.fromValidHex(channelId)))
val mockService = new MockService(eclair)
Post("/close", FormData("shortChannelId" -> shortChannelIdSerialized).toEntity) ~>
@ -203,7 +204,6 @@ class ApiServiceSpec extends FunSuite with ScalatestRouteTest with IdiomaticMock
assert(handled)
assert(status == OK)
val resp = entityAs[String]
assert(resp.contains(aliceNodeId.toString))
eclair.close(Right(ShortChannelId(shortChannelIdSerialized)), None)(any[Timeout]).wasCalled(once)
matchTestJson("close", resp)
}
@ -216,7 +216,6 @@ class ApiServiceSpec extends FunSuite with ScalatestRouteTest with IdiomaticMock
assert(handled)
assert(status == OK)
val resp = entityAs[String]
assert(resp.contains(aliceNodeId.toString))
eclair.close(Left(ByteVector32.fromValidHex(channelId)), None)(any[Timeout]).wasCalled(once)
matchTestJson("close", resp)
}