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

Added a timeout for channel open request (#928)

Until now, if the peer is unresponsive (typically doesn't respond to
`open_channel` or `funding_created`), we waited indefinitely, or until the
connection closed.

It translated to an API timeout for users, and uncertainty about the
state of the channel.

This PR:
- adds an optional `--openTimeoutSeconds` timeout to the `open` endpoint, that will
actively cancel the channel opening if it takes too long before reaching
state `WAIT_FOR_FUNDING_CONFIRMED`.
- makes the `ask` timeout configurable per request with a new `--timeoutSeconds`
- makes the akka http timeout slightly greater than the `ask` timeout

Ask timeout is set to 30s by default.
This commit is contained in:
Pierre-Marie Padiou 2019-04-18 18:30:51 +02:00 committed by GitHub
parent 6afe28d147
commit 84c0fab433
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
13 changed files with 305 additions and 258 deletions

View File

@ -40,90 +40,92 @@ case class AuditResponse(sent: Seq[PaymentSent], received: Seq[PaymentReceived],
trait Eclair {
def connect(uri: String): Future[String]
def connect(uri: String)(implicit timeout: Timeout): Future[String]
def open(nodeId: PublicKey, fundingSatoshis: Long, pushMsat: Option[Long], fundingFeerateSatByte: Option[Long], flags: Option[Int]): Future[String]
def open(nodeId: PublicKey, fundingSatoshis: Long, pushMsat: Option[Long], fundingFeerateSatByte: Option[Long], flags: Option[Int], openTimeout_opt: Option[Timeout])(implicit timeout: Timeout): Future[String]
def close(channelIdentifier: Either[ByteVector32, ShortChannelId], scriptPubKey: Option[ByteVector]): Future[String]
def close(channelIdentifier: Either[ByteVector32, ShortChannelId], scriptPubKey: Option[ByteVector])(implicit timeout: Timeout): Future[String]
def forceClose(channelIdentifier: Either[ByteVector32, ShortChannelId]): Future[String]
def forceClose(channelIdentifier: Either[ByteVector32, ShortChannelId])(implicit timeout: Timeout): Future[String]
def updateRelayFee(channelId: String, feeBaseMsat: Long, feeProportionalMillionths: Long): Future[String]
def updateRelayFee(channelId: String, feeBaseMsat: Long, feeProportionalMillionths: Long)(implicit timeout: Timeout): Future[String]
def channelsInfo(toRemoteNode: Option[PublicKey]): Future[Iterable[RES_GETINFO]]
def channelsInfo(toRemoteNode: Option[PublicKey])(implicit timeout: Timeout): Future[Iterable[RES_GETINFO]]
def channelInfo(channelId: ByteVector32): Future[RES_GETINFO]
def channelInfo(channelId: ByteVector32)(implicit timeout: Timeout): Future[RES_GETINFO]
def peersInfo(): Future[Iterable[PeerInfo]]
def peersInfo()(implicit timeout: Timeout): Future[Iterable[PeerInfo]]
def receive(description: String, amountMsat: Option[Long], expire: Option[Long], fallbackAddress: Option[String]): Future[PaymentRequest]
def receive(description: String, amountMsat: Option[Long], expire: Option[Long], fallbackAddress: Option[String])(implicit timeout: Timeout): Future[PaymentRequest]
def receivedInfo(paymentHash: ByteVector32): Future[Option[IncomingPayment]]
def receivedInfo(paymentHash: ByteVector32)(implicit timeout: Timeout): Future[Option[IncomingPayment]]
def send(recipientNodeId: PublicKey, amountMsat: Long, paymentHash: ByteVector32, assistedRoutes: Seq[Seq[PaymentRequest.ExtraHop]] = Seq.empty, minFinalCltvExpiry: Option[Long] = None, maxAttempts: Option[Int] = None): Future[UUID]
def send(recipientNodeId: PublicKey, amountMsat: Long, paymentHash: ByteVector32, assistedRoutes: Seq[Seq[PaymentRequest.ExtraHop]] = Seq.empty, minFinalCltvExpiry: Option[Long] = None, maxAttempts: Option[Int] = None)(implicit timeout: Timeout): Future[UUID]
def sentInfo(id: Either[UUID, ByteVector32]): Future[Seq[OutgoingPayment]]
def sentInfo(id: Either[UUID, ByteVector32])(implicit timeout: Timeout): Future[Seq[OutgoingPayment]]
def findRoute(targetNodeId: PublicKey, amountMsat: Long, assistedRoutes: Seq[Seq[PaymentRequest.ExtraHop]] = Seq.empty): Future[RouteResponse]
def findRoute(targetNodeId: PublicKey, amountMsat: Long, assistedRoutes: Seq[Seq[PaymentRequest.ExtraHop]] = Seq.empty)(implicit timeout: Timeout): Future[RouteResponse]
def audit(from_opt: Option[Long], to_opt: Option[Long]): Future[AuditResponse]
def audit(from_opt: Option[Long], to_opt: Option[Long])(implicit timeout: Timeout): Future[AuditResponse]
def networkFees(from_opt: Option[Long], to_opt: Option[Long]): Future[Seq[NetworkFee]]
def networkFees(from_opt: Option[Long], to_opt: Option[Long])(implicit timeout: Timeout): Future[Seq[NetworkFee]]
def channelStats(): Future[Seq[Stats]]
def channelStats()(implicit timeout: Timeout): Future[Seq[Stats]]
def getInvoice(paymentHash: ByteVector32): Future[Option[PaymentRequest]]
def getInvoice(paymentHash: ByteVector32)(implicit timeout: Timeout): Future[Option[PaymentRequest]]
def pendingInvoices(from_opt: Option[Long], to_opt: Option[Long]): Future[Seq[PaymentRequest]]
def pendingInvoices(from_opt: Option[Long], to_opt: Option[Long])(implicit timeout: Timeout): Future[Seq[PaymentRequest]]
def allInvoices(from_opt: Option[Long], to_opt: Option[Long]): Future[Seq[PaymentRequest]]
def allInvoices(from_opt: Option[Long], to_opt: Option[Long])(implicit timeout: Timeout): Future[Seq[PaymentRequest]]
def allNodes(): Future[Iterable[NodeAnnouncement]]
def allNodes()(implicit timeout: Timeout): Future[Iterable[NodeAnnouncement]]
def allChannels(): Future[Iterable[ChannelDesc]]
def allChannels()(implicit timeout: Timeout): Future[Iterable[ChannelDesc]]
def allUpdates(nodeId: Option[PublicKey]): Future[Iterable[ChannelUpdate]]
def allUpdates(nodeId: Option[PublicKey])(implicit timeout: Timeout): Future[Iterable[ChannelUpdate]]
def getInfoResponse(): Future[GetInfoResponse]
def getInfoResponse()(implicit timeout: Timeout): Future[GetInfoResponse]
}
class EclairImpl(appKit: Kit) extends Eclair {
implicit val ec = appKit.system.dispatcher
implicit val timeout = Timeout(60 seconds) // used by akka ask
override def connect(uri: String): Future[String] = {
override def connect(uri: String)(implicit timeout: Timeout): Future[String] = {
(appKit.switchboard ? Peer.Connect(NodeURI.parse(uri))).mapTo[String]
}
override def open(nodeId: PublicKey, fundingSatoshis: Long, pushMsat: Option[Long], fundingFeerateSatByte: Option[Long], flags: Option[Int]): Future[String] = {
override def open(nodeId: PublicKey, fundingSatoshis: Long, pushMsat: Option[Long], fundingFeerateSatByte: Option[Long], flags: Option[Int], openTimeout_opt: Option[Timeout])(implicit timeout: Timeout): Future[String] = {
// 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(
remoteNodeId = nodeId,
fundingSatoshis = Satoshi(fundingSatoshis),
pushMsat = pushMsat.map(MilliSatoshi).getOrElse(MilliSatoshi(0)),
fundingTxFeeratePerKw_opt = fundingFeerateSatByte.map(feerateByte2Kw),
channelFlags = flags.map(_.toByte))).mapTo[String]
channelFlags = flags.map(_.toByte),
timeout_opt = Some(openTimeout))).mapTo[String]
}
override def close(channelIdentifier: Either[ByteVector32, ShortChannelId], scriptPubKey: Option[ByteVector]): Future[String] = {
override def close(channelIdentifier: Either[ByteVector32, ShortChannelId], scriptPubKey: Option[ByteVector])(implicit timeout: Timeout): Future[String] = {
sendToChannel(channelIdentifier.fold[String](_.toString(), _.toString()), CMD_CLOSE(scriptPubKey)).mapTo[String]
}
override def forceClose(channelIdentifier: Either[ByteVector32, ShortChannelId]): Future[String] = {
override def forceClose(channelIdentifier: Either[ByteVector32, ShortChannelId])(implicit timeout: Timeout): Future[String] = {
sendToChannel(channelIdentifier.fold[String](_.toString(), _.toString()), CMD_FORCECLOSE).mapTo[String]
}
override def updateRelayFee(channelId: String, feeBaseMsat: Long, feeProportionalMillionths: Long): Future[String] = {
override def updateRelayFee(channelId: String, feeBaseMsat: Long, feeProportionalMillionths: Long)(implicit timeout: Timeout): Future[String] = {
sendToChannel(channelId, CMD_UPDATE_RELAY_FEE(feeBaseMsat, feeProportionalMillionths)).mapTo[String]
}
override def peersInfo(): Future[Iterable[PeerInfo]] = for {
override def peersInfo()(implicit timeout: Timeout): Future[Iterable[PeerInfo]] = for {
peers <- (appKit.switchboard ? 'peers).mapTo[Iterable[ActorRef]]
peerinfos <- Future.sequence(peers.map(peer => (peer ? GetPeerInfo).mapTo[PeerInfo]))
} yield peerinfos
override def channelsInfo(toRemoteNode: Option[PublicKey]): Future[Iterable[RES_GETINFO]] = toRemoteNode match {
override def channelsInfo(toRemoteNode: Option[PublicKey])(implicit timeout: Timeout): Future[Iterable[RES_GETINFO]] = toRemoteNode match {
case Some(pk) => for {
channelsId <- (appKit.register ? 'channelsTo).mapTo[Map[ByteVector32, PublicKey]].map(_.filter(_._2 == pk).keys)
channels <- Future.sequence(channelsId.map(channelId => sendToChannel(channelId.toString(), CMD_GETINFO).mapTo[RES_GETINFO]))
@ -134,31 +136,31 @@ class EclairImpl(appKit: Kit) extends Eclair {
} yield channels
}
override def channelInfo(channelId: ByteVector32): Future[RES_GETINFO] = {
override def channelInfo(channelId: ByteVector32)(implicit timeout: Timeout): Future[RES_GETINFO] = {
sendToChannel(channelId.toString(), CMD_GETINFO).mapTo[RES_GETINFO]
}
override def allNodes(): Future[Iterable[NodeAnnouncement]] = (appKit.router ? 'nodes).mapTo[Iterable[NodeAnnouncement]]
override def allNodes()(implicit timeout: Timeout): Future[Iterable[NodeAnnouncement]] = (appKit.router ? 'nodes).mapTo[Iterable[NodeAnnouncement]]
override def allChannels(): Future[Iterable[ChannelDesc]] = {
override def allChannels()(implicit timeout: Timeout): Future[Iterable[ChannelDesc]] = {
(appKit.router ? 'channels).mapTo[Iterable[ChannelAnnouncement]].map(_.map(c => ChannelDesc(c.shortChannelId, c.nodeId1, c.nodeId2)))
}
override def allUpdates(nodeId: Option[PublicKey]): Future[Iterable[ChannelUpdate]] = nodeId match {
override def allUpdates(nodeId: Option[PublicKey])(implicit timeout: Timeout): Future[Iterable[ChannelUpdate]] = nodeId match {
case None => (appKit.router ? 'updates).mapTo[Iterable[ChannelUpdate]]
case Some(pk) => (appKit.router ? 'updatesMap).mapTo[Map[ChannelDesc, ChannelUpdate]].map(_.filter(e => e._1.a == pk || e._1.b == pk).values)
}
override def receive(description: String, amountMsat: Option[Long], expire: Option[Long], fallbackAddress: Option[String]): Future[PaymentRequest] = {
override def receive(description: String, amountMsat: Option[Long], expire: Option[Long], fallbackAddress: Option[String])(implicit timeout: Timeout): Future[PaymentRequest] = {
fallbackAddress.map { fa => fr.acinq.eclair.addressToPublicKeyScript(fa, appKit.nodeParams.chainHash) } // if it's not a bitcoin address throws an exception
(appKit.paymentHandler ? ReceivePayment(description = description, amountMsat_opt = amountMsat.map(MilliSatoshi), expirySeconds_opt = expire, fallbackAddress = fallbackAddress)).mapTo[PaymentRequest]
}
override def findRoute(targetNodeId: PublicKey, amountMsat: Long, assistedRoutes: Seq[Seq[PaymentRequest.ExtraHop]] = Seq.empty): Future[RouteResponse] = {
override def findRoute(targetNodeId: PublicKey, amountMsat: Long, assistedRoutes: Seq[Seq[PaymentRequest.ExtraHop]] = Seq.empty)(implicit timeout: Timeout): Future[RouteResponse] = {
(appKit.router ? RouteRequest(appKit.nodeParams.nodeId, targetNodeId, amountMsat, assistedRoutes)).mapTo[RouteResponse]
}
override def send(recipientNodeId: PublicKey, amountMsat: Long, paymentHash: ByteVector32, assistedRoutes: Seq[Seq[PaymentRequest.ExtraHop]] = Seq.empty, minFinalCltvExpiry_opt: Option[Long] = None, maxAttempts_opt: Option[Int] = None): Future[UUID] = {
override def send(recipientNodeId: PublicKey, amountMsat: Long, paymentHash: ByteVector32, assistedRoutes: Seq[Seq[PaymentRequest.ExtraHop]] = Seq.empty, minFinalCltvExpiry_opt: Option[Long] = None, maxAttempts_opt: Option[Int] = None)(implicit timeout: Timeout): Future[UUID] = {
val maxAttempts = maxAttempts_opt.getOrElse(appKit.nodeParams.maxPaymentAttempts)
val sendPayment = minFinalCltvExpiry_opt match {
case Some(minCltv) => SendPayment(amountMsat, paymentHash, recipientNodeId, assistedRoutes, finalCltvExpiry = minCltv, maxAttempts = maxAttempts)
@ -167,18 +169,18 @@ class EclairImpl(appKit: Kit) extends Eclair {
(appKit.paymentInitiator ? sendPayment).mapTo[UUID]
}
override def sentInfo(id: Either[UUID, ByteVector32]): Future[Seq[OutgoingPayment]] = Future {
override def sentInfo(id: Either[UUID, ByteVector32])(implicit timeout: Timeout): Future[Seq[OutgoingPayment]] = Future {
id match {
case Left(uuid) => appKit.nodeParams.db.payments.getOutgoingPayment(uuid).toSeq
case Right(paymentHash) => appKit.nodeParams.db.payments.getOutgoingPayments(paymentHash)
}
}
override def receivedInfo(paymentHash: ByteVector32): Future[Option[IncomingPayment]] = Future {
override def receivedInfo(paymentHash: ByteVector32)(implicit timeout: Timeout): Future[Option[IncomingPayment]] = Future {
appKit.nodeParams.db.payments.getIncomingPayment(paymentHash)
}
override def audit(from_opt: Option[Long], to_opt: Option[Long]): Future[AuditResponse] = {
override def audit(from_opt: Option[Long], to_opt: Option[Long])(implicit timeout: Timeout): Future[AuditResponse] = {
val from = from_opt.getOrElse(0L)
val to = to_opt.getOrElse(Long.MaxValue)
@ -189,30 +191,30 @@ class EclairImpl(appKit: Kit) extends Eclair {
))
}
override def networkFees(from_opt: Option[Long], to_opt: Option[Long]): Future[Seq[NetworkFee]] = {
override def networkFees(from_opt: Option[Long], to_opt: Option[Long])(implicit timeout: Timeout): Future[Seq[NetworkFee]] = {
val from = from_opt.getOrElse(0L)
val to = to_opt.getOrElse(Long.MaxValue)
Future(appKit.nodeParams.db.audit.listNetworkFees(from, to))
}
override def channelStats(): Future[Seq[Stats]] = Future(appKit.nodeParams.db.audit.stats)
override def channelStats()(implicit timeout: Timeout): Future[Seq[Stats]] = Future(appKit.nodeParams.db.audit.stats)
override def allInvoices(from_opt: Option[Long], to_opt: Option[Long]): Future[Seq[PaymentRequest]] = Future {
override def allInvoices(from_opt: Option[Long], to_opt: Option[Long])(implicit timeout: Timeout): Future[Seq[PaymentRequest]] = Future {
val from = from_opt.getOrElse(0L)
val to = to_opt.getOrElse(Long.MaxValue)
appKit.nodeParams.db.payments.listPaymentRequests(from, to)
}
override def pendingInvoices(from_opt: Option[Long], to_opt: Option[Long]): Future[Seq[PaymentRequest]] = Future {
override def pendingInvoices(from_opt: Option[Long], to_opt: Option[Long])(implicit timeout: Timeout): Future[Seq[PaymentRequest]] = Future {
val from = from_opt.getOrElse(0L)
val to = to_opt.getOrElse(Long.MaxValue)
appKit.nodeParams.db.payments.listPendingPaymentRequests(from, to)
}
override def getInvoice(paymentHash: ByteVector32): Future[Option[PaymentRequest]] = Future {
override def getInvoice(paymentHash: ByteVector32)(implicit timeout: Timeout): Future[Option[PaymentRequest]] = Future {
appKit.nodeParams.db.payments.getPaymentRequest(paymentHash)
}
@ -223,7 +225,7 @@ class EclairImpl(appKit: Kit) extends Eclair {
* @param request
* @return
*/
def sendToChannel(channelIdentifier: String, request: Any): Future[Any] =
def sendToChannel(channelIdentifier: String, request: Any)(implicit timeout: Timeout): Future[Any] =
for {
fwdReq <- Future(Register.ForwardShortId(ShortChannelId(channelIdentifier), request))
.recoverWith { case _ => Future(Register.Forward(ByteVector32.fromValidHex(channelIdentifier), request)) }
@ -231,7 +233,7 @@ class EclairImpl(appKit: Kit) extends Eclair {
res <- appKit.register ? fwdReq
} yield res
override def getInfoResponse: Future[GetInfoResponse] = Future.successful(
override def getInfoResponse()(implicit timeout: Timeout): Future[GetInfoResponse] = Future.successful(
GetInfoResponse(nodeId = appKit.nodeParams.nodeId,
alias = appKit.nodeParams.alias,
chainHash = appKit.nodeParams.chainHash,

View File

@ -19,11 +19,13 @@ package fr.acinq.eclair.api
import java.util.UUID
import akka.http.scaladsl.unmarshalling.Unmarshaller
import akka.util.Timeout
import fr.acinq.bitcoin.ByteVector32
import fr.acinq.bitcoin.Crypto.PublicKey
import fr.acinq.eclair.ShortChannelId
import fr.acinq.eclair.payment.PaymentRequest
import scodec.bits.ByteVector
import scala.concurrent.duration._
object FormParamExtractors {
@ -51,4 +53,8 @@ object FormParamExtractors {
UUID.fromString(str)
}
implicit val timeoutSecondsUnmarshaller: Unmarshaller[String, Timeout] = Unmarshaller.strict { str =>
Timeout(str.toInt.seconds)
}
}

View File

@ -156,13 +156,13 @@ trait OldService extends Logging {
}
case "open" => req.params match {
case JString(nodeId) :: JInt(fundingSatoshis) :: Nil =>
completeRpcFuture(req.id, (switchboard ? Peer.OpenChannel(PublicKey(ByteVector.fromValidHex(nodeId)), Satoshi(fundingSatoshis.toLong), MilliSatoshi(0), fundingTxFeeratePerKw_opt = None, channelFlags = None)).mapTo[String])
completeRpcFuture(req.id, (switchboard ? Peer.OpenChannel(PublicKey(ByteVector.fromValidHex(nodeId)), Satoshi(fundingSatoshis.toLong), MilliSatoshi(0), fundingTxFeeratePerKw_opt = None, channelFlags = None, timeout_opt = None)).mapTo[String])
case JString(nodeId) :: JInt(fundingSatoshis) :: JInt(pushMsat) :: Nil =>
completeRpcFuture(req.id, (switchboard ? Peer.OpenChannel(PublicKey(ByteVector.fromValidHex(nodeId)), Satoshi(fundingSatoshis.toLong), MilliSatoshi(pushMsat.toLong), channelFlags = None, fundingTxFeeratePerKw_opt = None)).mapTo[String])
completeRpcFuture(req.id, (switchboard ? Peer.OpenChannel(PublicKey(ByteVector.fromValidHex(nodeId)), Satoshi(fundingSatoshis.toLong), MilliSatoshi(pushMsat.toLong), channelFlags = None, fundingTxFeeratePerKw_opt = None, timeout_opt = None)).mapTo[String])
case JString(nodeId) :: JInt(fundingSatoshis) :: JInt(pushMsat) :: JInt(fundingFeerateSatPerByte) :: Nil =>
completeRpcFuture(req.id, (switchboard ? Peer.OpenChannel(PublicKey(ByteVector.fromValidHex(nodeId)), Satoshi(fundingSatoshis.toLong), MilliSatoshi(pushMsat.toLong), fundingTxFeeratePerKw_opt = Some(feerateByte2Kw(fundingFeerateSatPerByte.toLong)), channelFlags = None)).mapTo[String])
completeRpcFuture(req.id, (switchboard ? Peer.OpenChannel(PublicKey(ByteVector.fromValidHex(nodeId)), Satoshi(fundingSatoshis.toLong), MilliSatoshi(pushMsat.toLong), fundingTxFeeratePerKw_opt = Some(feerateByte2Kw(fundingFeerateSatPerByte.toLong)), channelFlags = None, timeout_opt = None)).mapTo[String])
case JString(nodeId) :: JInt(fundingSatoshis) :: JInt(pushMsat) :: JInt(fundingFeerateSatPerByte) :: JInt(flags) :: Nil =>
completeRpcFuture(req.id, (switchboard ? Peer.OpenChannel(PublicKey(ByteVector.fromValidHex(nodeId)), Satoshi(fundingSatoshis.toLong), MilliSatoshi(pushMsat.toLong), fundingTxFeeratePerKw_opt = Some(feerateByte2Kw(fundingFeerateSatPerByte.toLong)), channelFlags = Some(flags.toByte))).mapTo[String])
completeRpcFuture(req.id, (switchboard ? Peer.OpenChannel(PublicKey(ByteVector.fromValidHex(nodeId)), Satoshi(fundingSatoshis.toLong), MilliSatoshi(pushMsat.toLong), fundingTxFeeratePerKw_opt = Some(feerateByte2Kw(fundingFeerateSatPerByte.toLong)), channelFlags = Some(flags.toByte), timeout_opt = None)).mapTo[String])
case _ => reject(UnknownParamsRejection(req.id, s"[nodeId, fundingSatoshis], [nodeId, fundingSatoshis, pushMsat], [nodeId, fundingSatoshis, pushMsat, feerateSatPerByte] or [nodeId, fundingSatoshis, pushMsat, feerateSatPerByte, flag]"))
}
case "close" => req.params match {

View File

@ -16,11 +16,6 @@
package fr.acinq.eclair.api
import akka.http.scaladsl.server._
import fr.acinq.bitcoin.Crypto.PublicKey
import fr.acinq.bitcoin.{ByteVector32, MilliSatoshi, Satoshi}
import fr.acinq.eclair.{Eclair, Kit, NodeParams, ShortChannelId}
import FormParamExtractors._
import java.util.UUID
import akka.NotUsed
@ -34,17 +29,19 @@ import akka.http.scaladsl.server._
import akka.http.scaladsl.server.directives.Credentials
import akka.stream.scaladsl.{BroadcastHub, Flow, Keep, Source}
import akka.stream.{ActorMaterializer, OverflowStrategy}
import akka.util.Timeout
import fr.acinq.bitcoin.ByteVector32
import fr.acinq.bitcoin.Crypto.PublicKey
import fr.acinq.eclair.api.FormParamExtractors._
import fr.acinq.eclair.api.JsonSupport.CustomTypeHints
import fr.acinq.eclair.io.NodeURI
import fr.acinq.eclair.payment.PaymentLifecycle.PaymentFailed
import fr.acinq.eclair.payment._
import fr.acinq.eclair.payment.{PaymentReceived, PaymentRequest, _}
import fr.acinq.eclair.{Eclair, ShortChannelId}
import grizzled.slf4j.Logging
import org.json4s.jackson.Serialization
import scodec.bits.ByteVector
import scala.concurrent.Future
import scala.concurrent.duration._
@ -72,7 +69,7 @@ trait Service extends ExtraDirectives with Logging {
implicit val actorSystem: ActorSystem
implicit val mat: ActorMaterializer
// a named and typed URL parameter used across several routes, 32-bytes hex-encoded
// named and typed URL parameters used across several routes
val channelId = "channelId".as[ByteVector32](sha256HashUnmarshaller)
val nodeId = "nodeId".as[PublicKey]
val shortChannelId = "shortChannelId".as[ShortChannelId](shortChannelIdUnmarshaller)
@ -137,149 +134,156 @@ trait Service extends ExtraDirectives with Logging {
respondWithDefaultHeaders(customHeaders) {
handleExceptions(apiExceptionHandler) {
handleRejections(apiRejectionHandler) {
withRequestTimeoutResponse(timeoutResponse) {
authenticateBasicAsync(realm = "Access restricted", userPassAuthenticator) { _ =>
post {
path("getinfo") {
complete(eclairApi.getInfoResponse())
} ~
path("connect") {
formFields("uri".as[String]) { uri =>
complete(eclairApi.connect(uri))
} ~ formFields(nodeId, "host".as[String], "port".as[Int].?) { (nodeId, host, port_opt) =>
complete(eclairApi.connect(s"$nodeId@$host:${port_opt.getOrElse(NodeURI.DEFAULT_PORT)}"))
formFields("timeoutSeconds".as[Timeout].?) { tm_opt =>
// this is the akka timeout
implicit val timeout = tm_opt.getOrElse(Timeout(30 seconds))
// we ensure that http timeout is greater than akka timeout
withRequestTimeout(timeout.duration + 2.seconds) {
withRequestTimeoutResponse(timeoutResponse) {
authenticateBasicAsync(realm = "Access restricted", userPassAuthenticator) { _ =>
post {
path("getinfo") {
complete(eclairApi.getInfoResponse())
} ~
path("connect") {
formFields("uri".as[String]) { uri =>
complete(eclairApi.connect(uri))
} ~ formFields(nodeId, "host".as[String], "port".as[Int].?) { (nodeId, host, port_opt) =>
complete(eclairApi.connect(s"$nodeId@$host:${port_opt.getOrElse(NodeURI.DEFAULT_PORT)}"))
}
} ~
path("open") {
formFields(nodeId, "fundingSatoshis".as[Long], "pushMsat".as[Long].?, "fundingFeerateSatByte".as[Long].?, "channelFlags".as[Int].?, "openTimeoutSeconds".as[Timeout].?) {
(nodeId, fundingSatoshis, pushMsat, fundingFeerateSatByte, channelFlags, openTimeout_opt) =>
complete(eclairApi.open(nodeId, fundingSatoshis, pushMsat, fundingFeerateSatByte, channelFlags, openTimeout_opt))
}
} ~
path("updaterelayfee") {
formFields(channelId, "feeBaseMsat".as[Long], "feeProportionalMillionths".as[Long]) { (channelId, feeBase, feeProportional) =>
complete(eclairApi.updateRelayFee(channelId.toString, feeBase, feeProportional))
}
} ~
path("close") {
formFields(channelId, "scriptPubKey".as[ByteVector](binaryDataUnmarshaller).?) { (channelId, scriptPubKey_opt) =>
complete(eclairApi.close(Left(channelId), scriptPubKey_opt))
} ~ formFields(shortChannelId, "scriptPubKey".as[ByteVector](binaryDataUnmarshaller).?) { (shortChannelId, scriptPubKey_opt) =>
complete(eclairApi.close(Right(shortChannelId), scriptPubKey_opt))
}
} ~
path("forceclose") {
formFields(channelId) { channelId =>
complete(eclairApi.forceClose(Left(channelId)))
} ~ formFields(shortChannelId) { shortChannelId =>
complete(eclairApi.forceClose(Right(shortChannelId)))
}
} ~
path("peers") {
complete(eclairApi.peersInfo())
} ~
path("channels") {
formFields(nodeId.?) { toRemoteNodeId_opt =>
complete(eclairApi.channelsInfo(toRemoteNodeId_opt))
}
} ~
path("channel") {
formFields(channelId) { channelId =>
complete(eclairApi.channelInfo(channelId))
}
} ~
path("allnodes") {
complete(eclairApi.allNodes())
} ~
path("allchannels") {
complete(eclairApi.allChannels())
} ~
path("allupdates") {
formFields(nodeId.?) { nodeId_opt =>
complete(eclairApi.allUpdates(nodeId_opt))
}
} ~
path("findroute") {
formFields(invoice, amountMsat.?) {
case (invoice@PaymentRequest(_, Some(amount), _, nodeId, _, _), None) => complete(eclairApi.findRoute(nodeId, amount.toLong, invoice.routingInfo))
case (invoice, Some(overrideAmount)) => complete(eclairApi.findRoute(invoice.nodeId, overrideAmount, invoice.routingInfo))
case _ => reject(MalformedFormFieldRejection("invoice", "The invoice must have an amount or you need to specify one using 'amountMsat'"))
}
} ~
path("findroutetonode") {
formFields(nodeId, amountMsat) { (nodeId, amount) =>
complete(eclairApi.findRoute(nodeId, amount))
}
} ~
path("parseinvoice") {
formFields(invoice) { invoice =>
complete(invoice)
}
} ~
path("payinvoice") {
formFields(invoice, amountMsat.?, "maxAttempts".as[Int].?) {
case (invoice@PaymentRequest(_, Some(amount), _, nodeId, _, _), None, maxAttempts) =>
complete(eclairApi.send(nodeId, amount.toLong, invoice.paymentHash, invoice.routingInfo, invoice.minFinalCltvExpiry, maxAttempts))
case (invoice, Some(overrideAmount), maxAttempts) =>
complete(eclairApi.send(invoice.nodeId, overrideAmount, invoice.paymentHash, invoice.routingInfo, invoice.minFinalCltvExpiry, maxAttempts))
case _ => reject(MalformedFormFieldRejection("invoice", "The invoice must have an amount or you need to specify one using the field 'amountMsat'"))
}
} ~
path("sendtonode") {
formFields(amountMsat, paymentHash, nodeId, "maxAttempts".as[Int].?) { (amountMsat, paymentHash, nodeId, maxAttempts) =>
complete(eclairApi.send(nodeId, amountMsat, paymentHash, maxAttempts = maxAttempts))
}
} ~
path("getsentinfo") {
formFields("id".as[UUID]) { id =>
complete(eclairApi.sentInfo(Left(id)))
} ~ formFields(paymentHash) { paymentHash =>
complete(eclairApi.sentInfo(Right(paymentHash)))
}
} ~
path("createinvoice") {
formFields("description".as[String], amountMsat.?, "expireIn".as[Long].?, "fallbackAddress".as[String].?) { (desc, amountMsat, expire, fallBackAddress) =>
complete(eclairApi.receive(desc, amountMsat, expire, fallBackAddress))
}
} ~
path("getinvoice") {
formFields(paymentHash) { paymentHash =>
completeOrNotFound(eclairApi.getInvoice(paymentHash))
}
} ~
path("listinvoices") {
formFields(from.?, to.?) { (from_opt, to_opt) =>
complete(eclairApi.allInvoices(from_opt, to_opt))
}
} ~
path("listpendinginvoices") {
formFields(from.?, to.?) { (from_opt, to_opt) =>
complete(eclairApi.pendingInvoices(from_opt, to_opt))
}
} ~
path("getreceivedinfo") {
formFields(paymentHash) { paymentHash =>
completeOrNotFound(eclairApi.receivedInfo(paymentHash))
} ~ formFields(invoice) { invoice =>
completeOrNotFound(eclairApi.receivedInfo(invoice.paymentHash))
}
} ~
path("audit") {
formFields(from.?, to.?) { (from_opt, to_opt) =>
complete(eclairApi.audit(from_opt, to_opt))
}
} ~
path("networkfees") {
formFields(from.?, to.?) { (from_opt, to_opt) =>
complete(eclairApi.networkFees(from_opt, to_opt))
}
} ~
path("channelstats") {
complete(eclairApi.channelStats())
}
} ~ get {
path("ws") {
handleWebSocketMessages(makeSocketHandler)
}
} ~
path("open") {
formFields(nodeId, "fundingSatoshis".as[Long], "pushMsat".as[Long].?, "fundingFeerateSatByte".as[Long].?, "channelFlags".as[Int].?) {
(nodeId, fundingSatoshis, pushMsat, fundingFeerateSatByte, channelFlags) =>
complete(eclairApi.open(nodeId, fundingSatoshis, pushMsat, fundingFeerateSatByte, channelFlags))
}
} ~
path("updaterelayfee") {
formFields(channelId, "feeBaseMsat".as[Long], "feeProportionalMillionths".as[Long]) { (channelId, feeBase, feeProportional) =>
complete(eclairApi.updateRelayFee(channelId.toString, feeBase, feeProportional))
}
} ~
path("close") {
formFields(channelId, "scriptPubKey".as[ByteVector](binaryDataUnmarshaller).?) { (channelId, scriptPubKey_opt) =>
complete(eclairApi.close(Left(channelId), scriptPubKey_opt))
} ~ formFields(shortChannelId, "scriptPubKey".as[ByteVector](binaryDataUnmarshaller).?) { (shortChannelId, scriptPubKey_opt) =>
complete(eclairApi.close(Right(shortChannelId), scriptPubKey_opt))
}
} ~
path("forceclose") {
formFields(channelId) { channelId =>
complete(eclairApi.forceClose(Left(channelId)))
} ~ formFields(shortChannelId) { shortChannelId =>
complete(eclairApi.forceClose(Right(shortChannelId)))
}
} ~
path("peers") {
complete(eclairApi.peersInfo())
} ~
path("channels") {
formFields(nodeId.?) { toRemoteNodeId_opt =>
complete(eclairApi.channelsInfo(toRemoteNodeId_opt))
}
} ~
path("channel") {
formFields(channelId) { channelId =>
complete(eclairApi.channelInfo(channelId))
}
} ~
path("allnodes") {
complete(eclairApi.allNodes())
} ~
path("allchannels") {
complete(eclairApi.allChannels())
} ~
path("allupdates") {
formFields(nodeId.?) { nodeId_opt =>
complete(eclairApi.allUpdates(nodeId_opt))
}
} ~
path("findroute") {
formFields(invoice, amountMsat.?) {
case (invoice@PaymentRequest(_, Some(amount), _, nodeId, _, _), None) => complete(eclairApi.findRoute(nodeId, amount.toLong, invoice.routingInfo))
case (invoice, Some(overrideAmount)) => complete(eclairApi.findRoute(invoice.nodeId, overrideAmount, invoice.routingInfo))
case _ => reject(MalformedFormFieldRejection("invoice", "The invoice must have an amount or you need to specify one using 'amountMsat'"))
}
} ~
path("findroutetonode") {
formFields(nodeId, amountMsat) { (nodeId, amount) =>
complete(eclairApi.findRoute(nodeId, amount))
}
} ~
path("parseinvoice") {
formFields(invoice) { invoice =>
complete(invoice)
}
} ~
path("payinvoice") {
formFields(invoice, amountMsat.?, "maxAttempts".as[Int].?) {
case (invoice@PaymentRequest(_, Some(amount), _, nodeId, _, _), None, maxAttempts) =>
complete(eclairApi.send(nodeId, amount.toLong, invoice.paymentHash, invoice.routingInfo, invoice.minFinalCltvExpiry, maxAttempts))
case (invoice, Some(overrideAmount), maxAttempts) =>
complete(eclairApi.send(invoice.nodeId, overrideAmount, invoice.paymentHash, invoice.routingInfo, invoice.minFinalCltvExpiry, maxAttempts))
case _ => reject(MalformedFormFieldRejection("invoice", "The invoice must have an amount or you need to specify one using the field 'amountMsat'"))
}
} ~
path("sendtonode") {
formFields(amountMsat, paymentHash, nodeId, "maxAttempts".as[Int].?) { (amountMsat, paymentHash, nodeId, maxAttempts) =>
complete(eclairApi.send(nodeId, amountMsat, paymentHash, maxAttempts = maxAttempts))
}
} ~
path("getsentinfo") {
formFields("id".as[UUID]) { id =>
complete(eclairApi.sentInfo(Left(id)))
} ~ formFields(paymentHash) { paymentHash =>
complete(eclairApi.sentInfo(Right(paymentHash)))
}
} ~
path("createinvoice") {
formFields("description".as[String], amountMsat.?, "expireIn".as[Long].?, "fallbackAddress".as[String].?) { (desc, amountMsat, expire, fallBackAddress) =>
complete(eclairApi.receive(desc, amountMsat, expire, fallBackAddress))
}
} ~
path("getinvoice") {
formFields(paymentHash) { paymentHash =>
completeOrNotFound(eclairApi.getInvoice(paymentHash))
}
} ~
path("listinvoices") {
formFields(from.?, to.?) { (from_opt, to_opt) =>
complete(eclairApi.allInvoices(from_opt, to_opt))
}
} ~
path("listpendinginvoices") {
formFields(from.?, to.?) { (from_opt, to_opt) =>
complete(eclairApi.pendingInvoices(from_opt, to_opt))
}
} ~
path("getreceivedinfo") {
formFields(paymentHash) { paymentHash =>
completeOrNotFound(eclairApi.receivedInfo(paymentHash))
} ~ formFields(invoice) { invoice =>
completeOrNotFound(eclairApi.receivedInfo(invoice.paymentHash))
}
} ~
path("audit") {
formFields(from.?, to.?) { (from_opt, to_opt) =>
complete(eclairApi.audit(from_opt, to_opt))
}
} ~
path("networkfees") {
formFields(from.?, to.?) { (from_opt, to_opt) =>
complete(eclairApi.networkFees(from_opt, to_opt))
}
} ~
path("channelstats") {
complete(eclairApi.channelStats())
}
} ~ get {
path("ws") {
handleWebSocketMessages(makeSocketHandler)
}
}
}
@ -288,6 +292,4 @@ trait Service extends ExtraDirectives with Logging {
}
}
}
}

View File

@ -73,6 +73,8 @@ object Channel {
case object TickRefreshChannelUpdate
case object TickChannelOpenTimeout
case class RevocationTimeout(remoteCommitNumber: Long, peer: ActorRef) // 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)
sealed trait ChannelError
@ -239,6 +241,10 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
}
case Event(CMD_CLOSE(_), _) => goto(CLOSED) replying "ok"
case Event(TickChannelOpenTimeout, _) =>
replyToUser(Left(LocalError(new RuntimeException("open channel cancelled, took too long"))))
goto(CLOSED)
})
when(WAIT_FOR_OPEN_CHANNEL)(handleExceptions {
@ -328,6 +334,10 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
case Event(INPUT_DISCONNECTED, _) =>
replyToUser(Left(LocalError(new RuntimeException("disconnected"))))
goto(CLOSED)
case Event(TickChannelOpenTimeout, _) =>
replyToUser(Left(LocalError(new RuntimeException("open channel cancelled, took too long"))))
goto(CLOSED)
})
when(WAIT_FOR_FUNDING_INTERNAL)(handleExceptions {
@ -365,6 +375,10 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
case Event(INPUT_DISCONNECTED, _) =>
replyToUser(Left(LocalError(new RuntimeException("disconnected"))))
goto(CLOSED)
case Event(TickChannelOpenTimeout, _) =>
replyToUser(Left(LocalError(new RuntimeException("open channel cancelled, took too long"))))
goto(CLOSED)
})
when(WAIT_FOR_FUNDING_CREATED)(handleExceptions {
@ -472,6 +486,12 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
wallet.rollback(d.fundingTx)
replyToUser(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"))))
goto(CLOSED)
})
when(WAIT_FOR_FUNDING_CONFIRMED)(handleExceptions {

View File

@ -22,6 +22,7 @@ import java.nio.ByteOrder
import akka.actor.{ActorRef, FSM, OneForOneStrategy, PoisonPill, Props, Status, SupervisorStrategy, Terminated}
import akka.event.Logging.MDC
import akka.util.Timeout
import fr.acinq.bitcoin.Crypto.PublicKey
import fr.acinq.bitcoin.{ByteVector32, DeterministicWallet, MilliSatoshi, Protocol, Satoshi}
import fr.acinq.eclair.blockchain.EclairWallet
@ -162,12 +163,6 @@ class Peer(nodeParams: NodeParams, remoteNodeId: PublicKey, authenticator: Actor
connection ! PoisonPill
stay
case Event(o: Peer.OpenChannel, _) =>
// we're almost there, just wait a little
import scala.concurrent.ExecutionContext.Implicits.global
context.system.scheduler.scheduleOnce(100 milliseconds, self, o)
stay
case Event(Terminated(actor), d: InitializingData) if actor == d.transport =>
log.warning(s"lost connection to $remoteNodeId")
goto(DISCONNECTED) using DisconnectedData(d.address_opt, d.channels)
@ -262,6 +257,7 @@ class Peer(nodeParams: NodeParams, remoteNodeId: PublicKey, authenticator: Actor
case Event(c: Peer.OpenChannel, d: ConnectedData) =>
log.info(s"requesting a new channel to $remoteNodeId with fundingSatoshis=${c.fundingSatoshis}, pushMsat=${c.pushMsat} and fundingFeeratePerByte=${c.fundingTxFeeratePerKw_opt}")
val (channel, localParams) = createNewChannel(nodeParams, funder = true, c.fundingSatoshis.toLong, origin_opt = Some(sender))
c.timeout_opt.map(openTimeout => context.system.scheduler.scheduleOnce(openTimeout.duration, channel, Channel.TickChannelOpenTimeout)(context.dispatcher))
val temporaryChannelId = randomBytes32
val channelFeeratePerKw = Globals.feeratesPerKw.get.blocks_2
val fundingTxFeeratePerKw = c.fundingTxFeeratePerKw_opt.getOrElse(Globals.feeratesPerKw.get.blocks_6)
@ -557,7 +553,7 @@ object Peer {
case object Reconnect
case object Disconnect
case object ResumeAnnouncements
case class OpenChannel(remoteNodeId: PublicKey, fundingSatoshis: Satoshi, pushMsat: MilliSatoshi, fundingTxFeeratePerKw_opt: Option[Long], channelFlags: Option[Byte]) {
case class OpenChannel(remoteNodeId: PublicKey, fundingSatoshis: Satoshi, pushMsat: MilliSatoshi, fundingTxFeeratePerKw_opt: Option[Long], channelFlags: Option[Byte], timeout_opt: Option[Timeout]) {
require(fundingSatoshis.amount < Channel.MAX_FUNDING_SATOSHIS, s"fundingSatoshis must be less than ${Channel.MAX_FUNDING_SATOSHIS}")
require(pushMsat.amount <= 1000 * fundingSatoshis.amount, s"pushMsat must be less or equal to fundingSatoshis")
require(fundingSatoshis.amount >= 0, s"fundingSatoshis must be positive")

View File

@ -72,13 +72,13 @@ class Switchboard(nodeParams: NodeParams, authenticator: ActorRef, watcher: Acto
case Peer.Connect(NodeURI(publicKey, _)) if publicKey == nodeParams.nodeId =>
sender ! Status.Failure(new RuntimeException("cannot open connection with oneself"))
case c@Peer.Connect(NodeURI(remoteNodeId, _)) =>
case c: Peer.Connect =>
// we create a peer if it doesn't exist
val peer = createOrGetPeer(remoteNodeId, previousKnownAddress = None, offlineChannels = Set.empty)
val peer = createOrGetPeer(c.uri.nodeId, previousKnownAddress = None, offlineChannels = Set.empty)
peer forward c
case o@Peer.OpenChannel(remoteNodeId, _, _, _, _) =>
getPeer(remoteNodeId) match {
case o: Peer.OpenChannel =>
getPeer(o.remoteNodeId) match {
case Some(peer) => peer forward o
case None => sender ! Status.Failure(new RuntimeException("no connection to peer"))
}

View File

@ -2,12 +2,15 @@ package fr.acinq.eclair
import akka.actor.ActorSystem
import akka.testkit.{TestKit, TestProbe}
import akka.util.Timeout
import fr.acinq.bitcoin.Crypto.PublicKey
import fr.acinq.eclair.blockchain.TestWallet
import fr.acinq.eclair.io.Peer.OpenChannel
import org.scalatest.FunSuiteLike
import scodec.bits._
import scala.concurrent.duration._
class EclairImplSpec extends TestKit(ActorSystem("mySystem")) with FunSuiteLike {
test("convert fee rate properly") {
val watcher = TestProbe()
@ -34,13 +37,15 @@ class EclairImplSpec extends TestKit(ActorSystem("mySystem")) with FunSuiteLike
val eclair = new EclairImpl(kit)
val nodeId = PublicKey(hex"030bb6a5e0c6b203c7e2180fb78c7ba4bdce46126761d8201b91ddac089cdecc87")
implicit val timeout = Timeout(30 seconds)
// standard conversion
eclair.open(nodeId, fundingSatoshis = 10000000L, pushMsat = None, fundingFeerateSatByte = Some(5), flags = None)
eclair.open(nodeId, fundingSatoshis = 10000000L, pushMsat = None, fundingFeerateSatByte = Some(5), flags = None, openTimeout_opt = None)
val open = switchboard.expectMsgType[OpenChannel]
assert(open.fundingTxFeeratePerKw_opt == Some(1250))
// check that minimum fee rate of 253 sat/bw is used
eclair.open(nodeId, fundingSatoshis = 10000000L, pushMsat = None, fundingFeerateSatByte = Some(1), flags = None)
eclair.open(nodeId, fundingSatoshis = 10000000L, pushMsat = None, fundingFeerateSatByte = Some(1), flags = None, openTimeout_opt = None)
val open1 = switchboard.expectMsgType[OpenChannel]
assert(open1.fundingTxFeeratePerKw_opt == Some(MinimumFeeratePerKw))
}

View File

@ -18,25 +18,26 @@ package fr.acinq.eclair.api
import java.util.UUID
import akka.actor.{Actor, ActorSystem, Props, Scheduler}
import org.scalatest.FunSuite
import akka.actor.ActorSystem
import akka.http.scaladsl.model.FormData
import akka.http.scaladsl.model.StatusCodes._
import akka.http.scaladsl.testkit.{RouteTestTimeout, ScalatestRouteTest, WSProbe}
import fr.acinq.eclair._
import fr.acinq.eclair.io.Peer.{GetPeerInfo, PeerInfo}
import TestConstants._
import akka.http.scaladsl.model.headers.BasicHttpCredentials
import akka.http.scaladsl.server.{Directives, Route}
import akka.http.scaladsl.server.Route
import akka.http.scaladsl.testkit.{RouteTestTimeout, ScalatestRouteTest, WSProbe}
import akka.stream.ActorMaterializer
import akka.http.scaladsl.model.{ContentTypes, FormData, MediaTypes, Multipart}
import akka.util.Timeout
import fr.acinq.bitcoin.{ByteVector32, Crypto, MilliSatoshi}
import fr.acinq.eclair.TestConstants._
import fr.acinq.eclair._
import fr.acinq.eclair.channel.RES_GETINFO
import fr.acinq.eclair.db.{NetworkFee, IncomingPayment, OutgoingPayment, Stats}
import fr.acinq.eclair.payment.PaymentLifecycle.{PaymentFailed, ReceivePayment}
import fr.acinq.eclair.db.{IncomingPayment, NetworkFee, OutgoingPayment, Stats}
import fr.acinq.eclair.io.Peer.PeerInfo
import fr.acinq.eclair.payment.PaymentLifecycle.PaymentFailed
import fr.acinq.eclair.payment._
import fr.acinq.eclair.router.{ChannelDesc, RouteResponse}
import fr.acinq.eclair.wire.{ChannelUpdate, NodeAddress, NodeAnnouncement}
import org.json4s.jackson.Serialization
import org.scalatest.FunSuite
import scodec.bits.ByteVector
import scala.concurrent.Future
@ -48,52 +49,51 @@ import scala.util.Try
class ApiServiceSpec extends FunSuite with ScalatestRouteTest {
trait EclairMock extends Eclair {
override def connect(uri: String): Future[String] = ???
override def connect(uri: String)(implicit timeout: Timeout): Future[String] = ???
override def open(nodeId: Crypto.PublicKey, fundingSatoshis: Long, pushMsat: Option[Long], fundingFeerateSatByte: Option[Long], flags: Option[Int]): Future[String] = ???
override def open(nodeId: Crypto.PublicKey, fundingSatoshis: Long, pushMsat: Option[Long], fundingFeerateSatByte: Option[Long], flags: Option[Int], timeout_opt: Option[Timeout])(implicit timeout: Timeout): Future[String] = ???
override def close(channelIdentifier: Either[ByteVector32, ShortChannelId], scriptPubKey: Option[ByteVector]): Future[String] = ???
override def close(channelIdentifier: Either[ByteVector32, ShortChannelId], scriptPubKey: Option[ByteVector])(implicit timeout: Timeout): Future[String] = ???
override def forceClose(channelIdentifier: Either[ByteVector32, ShortChannelId]): Future[String] = ???
override def forceClose(channelIdentifier: Either[ByteVector32, ShortChannelId])(implicit timeout: Timeout): Future[String] = ???
override def updateRelayFee(channelId: String, feeBaseMsat: Long, feeProportionalMillionths: Long): Future[String] = ???
override def updateRelayFee(channelId: String, feeBaseMsat: Long, feeProportionalMillionths: Long)(implicit timeout: Timeout): Future[String] = ???
override def peersInfo(): Future[Iterable[PeerInfo]] = ???
override def channelsInfo(toRemoteNode: Option[Crypto.PublicKey])(implicit timeout: Timeout): Future[Iterable[RES_GETINFO]] = ???
override def channelsInfo(toRemoteNode: Option[Crypto.PublicKey]): Future[Iterable[RES_GETINFO]] = ???
override def channelInfo(channelId: ByteVector32)(implicit timeout: Timeout): Future[RES_GETINFO] = ???
override def channelInfo(channelId: ByteVector32): Future[RES_GETINFO] = ???
override def peersInfo()(implicit timeout: Timeout): Future[Iterable[PeerInfo]] = ???
override def allNodes(): Future[Iterable[NodeAnnouncement]] = ???
override def receive(description: String, amountMsat: Option[Long], expire: Option[Long], fallbackAddress: Option[String])(implicit timeout: Timeout): Future[PaymentRequest] = ???
override def allChannels(): Future[Iterable[ChannelDesc]] = ???
override def receivedInfo(paymentHash: ByteVector32)(implicit timeout: Timeout): Future[Option[IncomingPayment]] = ???
override def allUpdates(nodeId: Option[Crypto.PublicKey]): Future[Iterable[ChannelUpdate]] = ???
override def send(recipientNodeId: Crypto.PublicKey, amountMsat: Long, paymentHash: ByteVector32, assistedRoutes: Seq[Seq[PaymentRequest.ExtraHop]], minFinalCltvExpiry: Option[Long], maxAttempts: Option[Int])(implicit timeout: Timeout): Future[UUID] = ???
override def receive(description: String, amountMsat: Option[Long], expire: Option[Long], fallbackAddress: Option[String]): Future[PaymentRequest] = ???
override def sentInfo(id: Either[UUID, ByteVector32])(implicit timeout: Timeout): Future[Seq[OutgoingPayment]] = ???
override def findRoute(targetNodeId: Crypto.PublicKey, amountMsat: Long, assistedRoutes: Seq[Seq[PaymentRequest.ExtraHop]]): Future[RouteResponse] = ???
override def findRoute(targetNodeId: Crypto.PublicKey, amountMsat: Long, assistedRoutes: Seq[Seq[PaymentRequest.ExtraHop]])(implicit timeout: Timeout): Future[RouteResponse] = ???
override def send(recipientNodeId: Crypto.PublicKey, amountMsat: Long, paymentHash: ByteVector32, assistedRoutes: Seq[Seq[PaymentRequest.ExtraHop]], minFinalCltvExpiry: Option[Long], maxAttempts: Option[Int] = None): Future[UUID] = ???
override def audit(from_opt: Option[Long], to_opt: Option[Long])(implicit timeout: Timeout): Future[AuditResponse] = ???
override def receivedInfo(paymentHash: ByteVector32): Future[Option[IncomingPayment]] = ???
override def networkFees(from_opt: Option[Long], to_opt: Option[Long])(implicit timeout: Timeout): Future[Seq[NetworkFee]] = ???
override def audit(from_opt: Option[Long], to_opt: Option[Long]): Future[AuditResponse] = ???
override def channelStats()(implicit timeout: Timeout): Future[Seq[Stats]] = ???
override def networkFees(from_opt: Option[Long], to_opt: Option[Long]): Future[Seq[NetworkFee]] = ???
override def getInvoice(paymentHash: ByteVector32)(implicit timeout: Timeout): Future[Option[PaymentRequest]] = ???
override def channelStats(): Future[Seq[Stats]] = ???
override def pendingInvoices(from_opt: Option[Long], to_opt: Option[Long])(implicit timeout: Timeout): Future[Seq[PaymentRequest]] = ???
override def getInfoResponse(): Future[GetInfoResponse] = ???
override def allInvoices(from_opt: Option[Long], to_opt: Option[Long])(implicit timeout: Timeout): Future[Seq[PaymentRequest]] = ???
override def sentInfo(id: Either[UUID, ByteVector32]): Future[Seq[OutgoingPayment]] = ???
override def allNodes()(implicit timeout: Timeout): Future[Iterable[NodeAnnouncement]] = ???
override def allInvoices(from_opt: Option[Long], to_opt: Option[Long]): Future[Seq[PaymentRequest]] = ???
override def allChannels()(implicit timeout: Timeout): Future[Iterable[ChannelDesc]] = ???
override def getInvoice(paymentHash: ByteVector32): Future[Option[PaymentRequest]] = ???
override def pendingInvoices(from_opt: Option[Long], to_opt: Option[Long]): Future[Seq[PaymentRequest]] = ???
override def allUpdates(nodeId: Option[Crypto.PublicKey])(implicit timeout: Timeout): Future[Iterable[ChannelUpdate]] = ???
override def getInfoResponse()(implicit timeout: Timeout): Future[GetInfoResponse] = ???
}
implicit val formats = JsonSupport.formats
@ -166,7 +166,7 @@ class ApiServiceSpec extends FunSuite with ScalatestRouteTest {
test("'peers' should ask the switchboard for current known peers") {
val mockService = new MockService(new EclairMock {
override def peersInfo(): Future[Iterable[PeerInfo]] = Future.successful(List(
override def peersInfo()(implicit timeout: Timeout): Future[Iterable[PeerInfo]] = Future.successful(List(
PeerInfo(
nodeId = Alice.nodeParams.nodeId,
state = "CONNECTED",
@ -193,7 +193,7 @@ class ApiServiceSpec extends FunSuite with ScalatestRouteTest {
test("'getinfo' response should include this node ID") {
val mockService = new MockService(new EclairMock {
override def getInfoResponse(): Future[GetInfoResponse] = Future.successful(GetInfoResponse(
override def getInfoResponse()(implicit timeout: Timeout): Future[GetInfoResponse] = Future.successful(GetInfoResponse(
nodeId = Alice.nodeParams.nodeId,
alias = Alice.nodeParams.alias,
chainHash = Alice.nodeParams.chainHash,
@ -219,7 +219,7 @@ class ApiServiceSpec extends FunSuite with ScalatestRouteTest {
val shortChannelIdSerialized = "42000x27x3"
val mockService = new MockService(new EclairMock {
override def close(channelIdentifier: Either[ByteVector32, ShortChannelId], scriptPubKey: Option[ByteVector]): Future[String] = {
override def close(channelIdentifier: Either[ByteVector32, ShortChannelId], scriptPubKey: Option[ByteVector])(implicit timeout: Timeout): Future[String] = {
Future.successful(Alice.nodeParams.nodeId.toString())
}
})
@ -244,7 +244,7 @@ class ApiServiceSpec extends FunSuite with ScalatestRouteTest {
val remoteUri = "030bb6a5e0c6b203c7e2180fb78c7ba4bdce46126761d8201b91ddac089cdecc87@93.137.102.239:9735"
val mockService = new MockService(new EclairMock {
override def connect(uri: String): Future[String] = Future.successful("connected")
override def connect(uri: String)(implicit timeout: Timeout): Future[String] = Future.successful("connected")
})
Post("/connect", FormData("nodeId" -> remoteNodeId, "host" -> remoteHost).toEntity) ~>
@ -272,7 +272,7 @@ class ApiServiceSpec extends FunSuite with ScalatestRouteTest {
val invoice = "lnbc12580n1pw2ywztpp554ganw404sh4yjkwnysgn3wjcxfcq7gtx53gxczkjr9nlpc3hzvqdq2wpskwctddyxqr4rqrzjqwryaup9lh50kkranzgcdnn2fgvx390wgj5jd07rwr3vxeje0glc7z9rtvqqwngqqqqqqqlgqqqqqeqqjqrrt8smgjvfj7sg38dwtr9kc9gg3era9k3t2hvq3cup0jvsrtrxuplevqgfhd3rzvhulgcxj97yjuj8gdx8mllwj4wzjd8gdjhpz3lpqqvk2plh"
val mockService = new MockService(new EclairMock {
override def send(recipientNodeId: Crypto.PublicKey, amountMsat: Long, paymentHash: ByteVector32, assistedRoutes: Seq[Seq[PaymentRequest.ExtraHop]], minFinalCltvExpiry: Option[Long], maxAttempts: Option[Int] = None): Future[UUID] = Future.successful(
override def send(recipientNodeId: Crypto.PublicKey, amountMsat: Long, paymentHash: ByteVector32, assistedRoutes: Seq[Seq[PaymentRequest.ExtraHop]], minFinalCltvExpiry: Option[Long], maxAttempts: Option[Int] = None)(implicit timeout: Timeout): Future[UUID] = Future.successful(
id
)
})
@ -290,7 +290,7 @@ class ApiServiceSpec extends FunSuite with ScalatestRouteTest {
test("'receivedinfo' method should respond HTTP 404 with a JSON encoded response if the element is not found") {
val mockService = new MockService(new EclairMock {
override def receivedInfo(paymentHash: ByteVector32): Future[Option[IncomingPayment]] = Future.successful(None) // element not found
override def receivedInfo(paymentHash: ByteVector32)(implicit timeout: Timeout): Future[Option[IncomingPayment]] = Future.successful(None) // element not found
})
Post("/getreceivedinfo", FormData("paymentHash" -> ByteVector32.Zeroes.toHex).toEntity) ~>

View File

@ -20,6 +20,7 @@ import akka.testkit.{TestFSMRef, TestProbe}
import fr.acinq.bitcoin.{Block, ByteVector32, Satoshi}
import fr.acinq.eclair.TestConstants.{Alice, Bob}
import fr.acinq.eclair.blockchain.{MakeFundingTxResponse, TestWallet}
import fr.acinq.eclair.channel.Channel.TickChannelOpenTimeout
import fr.acinq.eclair.channel.states.StateTestsHelperMethods
import fr.acinq.eclair.channel.{WAIT_FOR_FUNDING_INTERNAL, _}
import fr.acinq.eclair.wire.{AcceptChannel, Error, Init, OpenChannel}
@ -154,4 +155,10 @@ class WaitForAcceptChannelStateSpec extends TestkitBaseClass with StateTestsHelp
awaitCond(alice.stateName == CLOSED)
}
test("recv TickChannelOpenTimeout") { f =>
import f._
alice ! TickChannelOpenTimeout
awaitCond(alice.stateName == CLOSED)
}
}

View File

@ -20,6 +20,7 @@ import akka.testkit.{TestFSMRef, TestProbe}
import fr.acinq.bitcoin.ByteVector32
import fr.acinq.eclair.TestConstants.{Alice, Bob}
import fr.acinq.eclair.blockchain._
import fr.acinq.eclair.channel.Channel.TickChannelOpenTimeout
import fr.acinq.eclair.channel._
import fr.acinq.eclair.channel.states.StateTestsHelperMethods
import fr.acinq.eclair.wire.{AcceptChannel, Error, FundingCreated, FundingSigned, Init, OpenChannel}
@ -94,4 +95,10 @@ class WaitForFundingSignedStateSpec extends TestkitBaseClass with StateTestsHelp
assert(alice.underlyingActor.wallet.asInstanceOf[TestWallet].rolledback.contains(fundingTx))
}
test("recv TickChannelOpenTimeout") { f =>
import f._
alice ! TickChannelOpenTimeout
awaitCond(alice.stateName == CLOSED)
}
}

View File

@ -168,7 +168,8 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
fundingSatoshis = Satoshi(fundingSatoshis),
pushMsat = MilliSatoshi(pushMsat),
fundingTxFeeratePerKw_opt = None,
channelFlags = None))
channelFlags = None,
timeout_opt = None))
assert(sender.expectMsgType[String](10 seconds).startsWith("created channel"))
}

View File

@ -32,6 +32,7 @@ import javafx.fxml.FXML
import javafx.scene.control._
import javafx.stage.Stage
import scala.concurrent.duration._
import scala.util.{Failure, Success, Try}
/**
@ -109,7 +110,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(fr.acinq.eclair.feerateByte2Kw), Some(channelFlags))))
handlers.open(nodeUri, Some(Peer.OpenChannel(nodeUri.nodeId, capacitySat, MilliSatoshi(pushMsat), feeratePerByte_opt.map(fr.acinq.eclair.feerateByte2Kw), Some(channelFlags), Some(30 seconds))))
stage.close()
case (Failure(t), _, _) =>
logger.error(s"could not parse capacity with cause=${t.getLocalizedMessage}")