mirror of
https://github.com/ACINQ/eclair.git
synced 2025-02-22 14:22:39 +01:00
Add a proper payments database (#885)
There is no unique identifier for payments in LN protocol. Critically, we can't use `payment_hash` as a unique id because there is no way to ensure unicity at the protocol level. Also, the general case for a "payment" is to be associated to multiple `update_add_htlc`s, because of automated retries. We also routinely retry payments, which means that the same `payment_hash` will be conceptually linked to a list of lists of `update_add_htlc`s. In order to address this, we introduce a payment id, which uniquely identifies a payment, as in a set of sequential `update_add_htlc` managed by a single `PaymentLifecycle` that ends with a `PaymentSent` or `PaymentFailed` outcome. We can then query the api using either `payment_id` or `payment_hash`. The former will return a single payment status, the latter will return a set of payment statuses, each identified by their `payment_id`. * Add a payment identifier * Remove InvalidPaymentHash channel exception * Remove unused 'close' from paymentsDb * Introduce sent_payments in PaymentDB, bump db version * Return the UUID of the ongoing payment in /send API * Add api to query payments by ID * Add 'fallbackAddress' in /receive API * Expose /paymentinfo by paymentHash * Add id column to audit.sent table, add test for db migration * Add invoices to payment DB * Add license header to ExtraDirective.scala * Respond with HTTP 404 if the corresponding invoice/paymentHash was not found. * Left-pad numeric bolt11 tagged fields to have a number of bits multiple of five (bech32 encoding). * Add invoices API * Remove CheckPayment message * GUI: consume UUID reply from payment initiator * API: reply with JSON encoded response if the queried element wasn't found * Return a payment request object in /receive * Remove limit of pending payment requests! * Avoid printing "null" fields when serializing an invoice to json * Add index on paymentDb.sent_payments.payment_hash * Order results in descending order in listPaymentRequest
This commit is contained in:
parent
70d7db7f96
commit
9032da5326
54 changed files with 1265 additions and 569 deletions
|
@ -30,8 +30,8 @@ and COMMAND is one of:
|
|||
getinfo, connect, open, close, forceclose, updaterelayfee,
|
||||
peers, channels, channel, allnodes, allchannels, allupdates,
|
||||
receive, parseinvoice, findroute, findroutetonode,
|
||||
send, sendtonode, checkpayment,
|
||||
audit, networkfees, channelstats
|
||||
payinvoice, sendtonode, getreceivedinfo, audit, networkfees,
|
||||
channelstats, getsentinfo, getinvoice, allinvoice, listpendinginvoices
|
||||
|
||||
Examples
|
||||
--------
|
||||
|
|
|
@ -87,7 +87,6 @@ eclair {
|
|||
|
||||
payment-handler = "local"
|
||||
payment-request-expiry = 1 hour // default expiry for payment requests generated by this node
|
||||
max-pending-payment-requests = 10000000
|
||||
min-funding-satoshis = 100000
|
||||
max-payment-attempts = 5
|
||||
|
||||
|
|
|
@ -1,22 +1,38 @@
|
|||
/*
|
||||
* Copyright 2018 ACINQ SAS
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package fr.acinq.eclair
|
||||
|
||||
import java.util.UUID
|
||||
import akka.actor.ActorRef
|
||||
import akka.pattern._
|
||||
import akka.util.Timeout
|
||||
import fr.acinq.bitcoin.Crypto.PublicKey
|
||||
import fr.acinq.bitcoin.{ByteVector32, MilliSatoshi, Satoshi}
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.db.{NetworkFee, Stats}
|
||||
import fr.acinq.eclair.db.{NetworkFee, IncomingPayment, OutgoingPayment, Stats}
|
||||
import fr.acinq.eclair.io.Peer.{GetPeerInfo, PeerInfo}
|
||||
import fr.acinq.eclair.io.{NodeURI, Peer}
|
||||
import fr.acinq.eclair.payment.PaymentLifecycle._
|
||||
import fr.acinq.eclair.payment.{PaymentLifecycle, PaymentReceived, PaymentRelayed, PaymentRequest, PaymentSent}
|
||||
import fr.acinq.eclair.router.{ChannelDesc, RouteRequest, RouteResponse}
|
||||
import fr.acinq.eclair.wire.{ChannelAnnouncement, ChannelUpdate, NodeAddress, NodeAnnouncement}
|
||||
import scodec.bits.ByteVector
|
||||
|
||||
import scala.concurrent.Future
|
||||
import scala.concurrent.duration._
|
||||
import fr.acinq.eclair.payment.{PaymentReceived, PaymentRelayed, PaymentRequest, PaymentSent}
|
||||
import fr.acinq.eclair.wire.{ChannelAnnouncement, ChannelUpdate, NodeAddress, NodeAnnouncement}
|
||||
|
||||
case class GetInfoResponse(nodeId: PublicKey, alias: String, chainHash: ByteVector32, blockHeight: Int, publicAddresses: Seq[NodeAddress])
|
||||
|
||||
|
@ -34,32 +50,40 @@ trait Eclair {
|
|||
|
||||
def updateRelayFee(channelId: String, feeBaseMsat: Long, feeProportionalMillionths: Long): Future[String]
|
||||
|
||||
def peersInfo(): Future[Iterable[PeerInfo]]
|
||||
|
||||
def channelsInfo(toRemoteNode: Option[PublicKey]): Future[Iterable[RES_GETINFO]]
|
||||
|
||||
def channelInfo(channelId: ByteVector32): Future[RES_GETINFO]
|
||||
|
||||
def allnodes(): Future[Iterable[NodeAnnouncement]]
|
||||
def peersInfo(): Future[Iterable[PeerInfo]]
|
||||
|
||||
def allchannels(): Future[Iterable[ChannelDesc]]
|
||||
def receive(description: String, amountMsat: Option[Long], expire: Option[Long], fallbackAddress: Option[String]): Future[PaymentRequest]
|
||||
|
||||
def allupdates(nodeId: Option[PublicKey]): Future[Iterable[ChannelUpdate]]
|
||||
def receivedInfo(paymentHash: ByteVector32): Future[Option[IncomingPayment]]
|
||||
|
||||
def receive(description: String, amountMsat: Option[Long], expire: Option[Long]): Future[String]
|
||||
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 sentInfo(id: Either[UUID, ByteVector32]): Future[Seq[OutgoingPayment]]
|
||||
|
||||
def findRoute(targetNodeId: PublicKey, amountMsat: Long, assistedRoutes: Seq[Seq[PaymentRequest.ExtraHop]] = Seq.empty): Future[RouteResponse]
|
||||
|
||||
def send(recipientNodeId: PublicKey, amountMsat: Long, paymentHash: ByteVector32, assistedRoutes: Seq[Seq[PaymentRequest.ExtraHop]] = Seq.empty, minFinalCltvExpiry: Option[Long] = None, maxAttempts: Option[Int] = None): Future[PaymentResult]
|
||||
|
||||
def checkpayment(paymentHash: ByteVector32): Future[Boolean]
|
||||
|
||||
def audit(from_opt: Option[Long], to_opt: Option[Long]): Future[AuditResponse]
|
||||
|
||||
def networkFees(from_opt: Option[Long], to_opt: Option[Long]): Future[Seq[NetworkFee]]
|
||||
|
||||
def channelStats(): Future[Seq[Stats]]
|
||||
|
||||
def getInvoice(paymentHash: ByteVector32): Future[Option[PaymentRequest]]
|
||||
|
||||
def pendingInvoices(from_opt: Option[Long], to_opt: Option[Long]): Future[Seq[PaymentRequest]]
|
||||
|
||||
def allInvoices(from_opt: Option[Long], to_opt: Option[Long]): Future[Seq[PaymentRequest]]
|
||||
|
||||
def allNodes(): Future[Iterable[NodeAnnouncement]]
|
||||
|
||||
def allChannels(): Future[Iterable[ChannelDesc]]
|
||||
|
||||
def allUpdates(nodeId: Option[PublicKey]): Future[Iterable[ChannelUpdate]]
|
||||
|
||||
def getInfoResponse(): Future[GetInfoResponse]
|
||||
|
||||
}
|
||||
|
@ -114,41 +138,44 @@ class EclairImpl(appKit: Kit) extends Eclair {
|
|||
sendToChannel(channelId.toString(), CMD_GETINFO).mapTo[RES_GETINFO]
|
||||
}
|
||||
|
||||
override def allnodes(): Future[Iterable[NodeAnnouncement]] = (appKit.router ? 'nodes).mapTo[Iterable[NodeAnnouncement]]
|
||||
override def allNodes(): Future[Iterable[NodeAnnouncement]] = (appKit.router ? 'nodes).mapTo[Iterable[NodeAnnouncement]]
|
||||
|
||||
override def allchannels(): Future[Iterable[ChannelDesc]] = {
|
||||
override def allChannels(): 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]): 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]): Future[String] = {
|
||||
(appKit.paymentHandler ? ReceivePayment(description = description, amountMsat_opt = amountMsat.map(MilliSatoshi), expirySeconds_opt = expire)).mapTo[PaymentRequest].map { pr =>
|
||||
PaymentRequest.write(pr)
|
||||
}
|
||||
override def receive(description: String, amountMsat: Option[Long], expire: Option[Long], fallbackAddress: Option[String]): 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] = {
|
||||
(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[PaymentResult] = {
|
||||
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] = {
|
||||
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)
|
||||
case None => SendPayment(amountMsat, paymentHash, recipientNodeId, assistedRoutes, maxAttempts = maxAttempts)
|
||||
case None => SendPayment(amountMsat, paymentHash, recipientNodeId, assistedRoutes, maxAttempts = maxAttempts)
|
||||
}
|
||||
(appKit.paymentInitiator ? sendPayment).mapTo[PaymentResult].map {
|
||||
case s: PaymentSucceeded => s
|
||||
case f: PaymentFailed => f.copy(failures = PaymentLifecycle.transformForUser(f.failures))
|
||||
(appKit.paymentInitiator ? sendPayment).mapTo[UUID]
|
||||
}
|
||||
|
||||
override def sentInfo(id: Either[UUID, ByteVector32]): 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 checkpayment(paymentHash: ByteVector32): Future[Boolean] = {
|
||||
(appKit.paymentHandler ? CheckPayment(paymentHash)).mapTo[Boolean]
|
||||
override def receivedInfo(paymentHash: ByteVector32): Future[Option[IncomingPayment]] = Future {
|
||||
appKit.nodeParams.db.payments.getIncomingPayment(paymentHash)
|
||||
}
|
||||
|
||||
override def audit(from_opt: Option[Long], to_opt: Option[Long]): Future[AuditResponse] = {
|
||||
|
@ -171,6 +198,24 @@ class EclairImpl(appKit: Kit) extends Eclair {
|
|||
|
||||
override def channelStats(): Future[Seq[Stats]] = Future(appKit.nodeParams.db.audit.stats)
|
||||
|
||||
override def allInvoices(from_opt: Option[Long], to_opt: Option[Long]): 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 {
|
||||
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 {
|
||||
appKit.nodeParams.db.payments.getPaymentRequest(paymentHash)
|
||||
}
|
||||
|
||||
/**
|
||||
* Sends a request to a channel and expects a response
|
||||
*
|
||||
|
@ -188,10 +233,10 @@ class EclairImpl(appKit: Kit) extends Eclair {
|
|||
|
||||
override def getInfoResponse: Future[GetInfoResponse] = Future.successful(
|
||||
GetInfoResponse(nodeId = appKit.nodeParams.nodeId,
|
||||
alias = appKit.nodeParams.alias,
|
||||
chainHash = appKit.nodeParams.chainHash,
|
||||
blockHeight = Globals.blockCount.intValue(),
|
||||
publicAddresses = appKit.nodeParams.publicAddresses)
|
||||
alias = appKit.nodeParams.alias,
|
||||
chainHash = appKit.nodeParams.chainHash,
|
||||
blockHeight = Globals.blockCount.intValue(),
|
||||
publicAddresses = appKit.nodeParams.publicAddresses)
|
||||
)
|
||||
|
||||
}
|
||||
|
|
|
@ -73,7 +73,6 @@ case class NodeParams(keyManager: KeyManager,
|
|||
channelFlags: Byte,
|
||||
watcherType: WatcherType,
|
||||
paymentRequestExpiry: FiniteDuration,
|
||||
maxPendingPaymentRequests: Int,
|
||||
minFundingSatoshis: Long,
|
||||
routerConf: RouterConf,
|
||||
socksProxy_opt: Option[Socks5ProxyParams],
|
||||
|
@ -210,7 +209,6 @@ object NodeParams {
|
|||
channelFlags = config.getInt("channel-flags").toByte,
|
||||
watcherType = watcherType,
|
||||
paymentRequestExpiry = FiniteDuration(config.getDuration("payment-request-expiry").getSeconds, TimeUnit.SECONDS),
|
||||
maxPendingPaymentRequests = config.getInt("max-pending-payment-requests"),
|
||||
minFundingSatoshis = config.getLong("min-funding-satoshis"),
|
||||
routerConf = RouterConf(
|
||||
channelExcludeDuration = FiniteDuration(config.getDuration("router.channel-exclude-duration").getSeconds, TimeUnit.SECONDS),
|
||||
|
|
|
@ -245,7 +245,7 @@ class Setup(datadir: File,
|
|||
authenticator = system.actorOf(SimpleSupervisor.props(Authenticator.props(nodeParams), "authenticator", SupervisorStrategy.Resume))
|
||||
switchboard = system.actorOf(SimpleSupervisor.props(Switchboard.props(nodeParams, authenticator, watcher, router, relayer, wallet), "switchboard", SupervisorStrategy.Resume))
|
||||
server = system.actorOf(SimpleSupervisor.props(Server.props(nodeParams, authenticator, serverBindingAddress, Some(tcpBound)), "server", SupervisorStrategy.Restart))
|
||||
paymentInitiator = system.actorOf(SimpleSupervisor.props(PaymentInitiator.props(nodeParams.nodeId, router, register), "payment-initiator", SupervisorStrategy.Restart))
|
||||
paymentInitiator = system.actorOf(SimpleSupervisor.props(PaymentInitiator.props(nodeParams, router, register), "payment-initiator", SupervisorStrategy.Restart))
|
||||
_ = for (i <- 0 until config.getInt("autoprobe-count")) yield system.actorOf(SimpleSupervisor.props(Autoprobe.props(nodeParams, router, paymentInitiator), s"payment-autoprobe-$i", SupervisorStrategy.Restart))
|
||||
|
||||
kit = Kit(
|
||||
|
|
|
@ -0,0 +1,37 @@
|
|||
/*
|
||||
* Copyright 2018 ACINQ SAS
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package fr.acinq.eclair.api
|
||||
|
||||
import akka.http.scaladsl.marshalling.ToResponseMarshaller
|
||||
import akka.http.scaladsl.model.{ContentTypes, HttpResponse}
|
||||
import akka.http.scaladsl.model.StatusCodes._
|
||||
import akka.http.scaladsl.server.{Directives, Route}
|
||||
import fr.acinq.eclair.api.JsonSupport._
|
||||
import scala.concurrent.{Future}
|
||||
import scala.util.{Failure, Success}
|
||||
|
||||
trait ExtraDirectives extends Directives {
|
||||
|
||||
// custom directive to fail with HTTP 404 (and JSON response) if the element was not found
|
||||
def completeOrNotFound[T](fut: Future[Option[T]])(implicit marshaller: ToResponseMarshaller[T]): Route = onComplete(fut) {
|
||||
case Success(Some(t)) => complete(t)
|
||||
case Success(None) =>
|
||||
complete(HttpResponse(NotFound).withEntity(ContentTypes.`application/json`, serialization.writePretty(ErrorResponse("Not found"))))
|
||||
case Failure(_) => reject
|
||||
}
|
||||
|
||||
}
|
|
@ -1,5 +1,23 @@
|
|||
/*
|
||||
* Copyright 2018 ACINQ SAS
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package fr.acinq.eclair.api
|
||||
|
||||
import java.util.UUID
|
||||
|
||||
import akka.http.scaladsl.unmarshalling.Unmarshaller
|
||||
import fr.acinq.bitcoin.ByteVector32
|
||||
import fr.acinq.bitcoin.Crypto.PublicKey
|
||||
|
@ -29,4 +47,8 @@ object FormParamExtractors {
|
|||
ShortChannelId(str)
|
||||
}
|
||||
|
||||
implicit val javaUUIDUnmarshaller: Unmarshaller[String, UUID] = Unmarshaller.strict { str =>
|
||||
UUID.fromString(str)
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -17,9 +17,7 @@
|
|||
package fr.acinq.eclair.api
|
||||
|
||||
import java.net.InetSocketAddress
|
||||
|
||||
import akka.http.scaladsl.model.MediaType
|
||||
import akka.http.scaladsl.model.MediaTypes._
|
||||
import java.util.UUID
|
||||
import com.google.common.net.HostAndPort
|
||||
import de.heikoseeberger.akkahttpjson4s.Json4sSupport
|
||||
import de.heikoseeberger.akkahttpjson4s.Json4sSupport.ShouldWritePretty
|
||||
|
@ -27,6 +25,7 @@ import fr.acinq.bitcoin.Crypto.{Point, PrivateKey, PublicKey, Scalar}
|
|||
import fr.acinq.bitcoin.{ByteVector32, MilliSatoshi, OutPoint, Transaction}
|
||||
import fr.acinq.eclair.channel.State
|
||||
import fr.acinq.eclair.crypto.ShaChain
|
||||
import fr.acinq.eclair.db.OutgoingPaymentStatus
|
||||
import fr.acinq.eclair.payment.PaymentRequest
|
||||
import fr.acinq.eclair.router.RouteResponse
|
||||
import fr.acinq.eclair.transactions.Direction
|
||||
|
@ -139,19 +138,37 @@ class DirectionSerializer extends CustomSerializer[Direction](format => ({ null
|
|||
case d: Direction => JString(d.toString)
|
||||
}))
|
||||
|
||||
class PaymentRequestSerializer extends CustomSerializer[PaymentRequest](format => ({ null },{
|
||||
case p: PaymentRequest => JObject(JField("prefix", JString(p.prefix)) ::
|
||||
JField("amount", if (p.amount.isDefined) JLong(p.amount.get.toLong) else JNull) ::
|
||||
JField("timestamp", JLong(p.timestamp)) ::
|
||||
JField("nodeId", JString(p.nodeId.toString())) ::
|
||||
JField("description", JString(p.description match {
|
||||
case Left(l) => l.toString()
|
||||
case Right(r) => r.toString()
|
||||
})) ::
|
||||
JField("paymentHash", JString(p.paymentHash.toString())) ::
|
||||
JField("expiry", if (p.expiry.isDefined) JLong(p.expiry.get) else JNull) ::
|
||||
JField("minFinalCltvExpiry", if (p.minFinalCltvExpiry.isDefined) JLong(p.minFinalCltvExpiry.get) else JNull) ::
|
||||
Nil)
|
||||
class PaymentRequestSerializer extends CustomSerializer[PaymentRequest](format => ( {
|
||||
null
|
||||
}, {
|
||||
case p: PaymentRequest => {
|
||||
val expiry = p.expiry.map(ex => JField("expiry", JLong(ex))).toSeq
|
||||
val minFinalCltvExpiry = p.minFinalCltvExpiry.map(mfce => JField("minFinalCltvExpiry", JLong(mfce))).toSeq
|
||||
val amount = p.amount.map(msat => JField("amount", JLong(msat.toLong))).toSeq
|
||||
|
||||
val fieldList = List(JField("prefix", JString(p.prefix)),
|
||||
JField("timestamp", JLong(p.timestamp)),
|
||||
JField("nodeId", JString(p.nodeId.toString())),
|
||||
JField("serialized", JString(PaymentRequest.write(p))),
|
||||
JField("description", JString(p.description match {
|
||||
case Left(l) => l.toString()
|
||||
case Right(r) => r.toString()
|
||||
})),
|
||||
JField("paymentHash", JString(p.paymentHash.toString()))) ++
|
||||
expiry ++
|
||||
minFinalCltvExpiry ++
|
||||
amount
|
||||
|
||||
JObject(fieldList)
|
||||
}
|
||||
}))
|
||||
|
||||
class JavaUUIDSerializer extends CustomSerializer[UUID](format => ({ null }, {
|
||||
case id: UUID => JString(id.toString)
|
||||
}))
|
||||
|
||||
class OutgoingPaymentStatusSerializer extends CustomSerializer[OutgoingPaymentStatus.Value](format => ({ null }, {
|
||||
case el: OutgoingPaymentStatus.Value => JString(el.toString)
|
||||
}))
|
||||
|
||||
object JsonSupport extends Json4sSupport {
|
||||
|
@ -182,7 +199,9 @@ object JsonSupport extends Json4sSupport {
|
|||
new FailureMessageSerializer +
|
||||
new NodeAddressSerializer +
|
||||
new DirectionSerializer +
|
||||
new PaymentRequestSerializer
|
||||
new PaymentRequestSerializer +
|
||||
new JavaUUIDSerializer +
|
||||
new OutgoingPaymentStatusSerializer
|
||||
|
||||
implicit val shouldWritePretty: ShouldWritePretty = ShouldWritePretty.True
|
||||
|
||||
|
|
|
@ -307,7 +307,7 @@ trait OldService extends Logging {
|
|||
case _ => Future.failed(new IllegalArgumentException("payment identifier must be a payment request or a payment hash"))
|
||||
}
|
||||
}
|
||||
found <- (paymentHandler ? CheckPayment(ByteVector32.fromValidHex(identifier))).map(found => new JBool(found.asInstanceOf[Boolean]))
|
||||
found <- Future(appKit.nodeParams.db.payments.getIncomingPayment(ByteVector32.fromValidHex(identifier)).map(_ => JBool(true)).getOrElse(JBool(false)))
|
||||
} yield found)
|
||||
case _ => reject(UnknownParamsRejection(req.id, "[paymentHash] or [paymentRequest]"))
|
||||
}
|
||||
|
|
|
@ -21,36 +21,39 @@ 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
|
||||
import akka.actor.{Actor, ActorRef, ActorSystem, Props}
|
||||
import akka.actor.{Actor, ActorSystem, Props}
|
||||
import akka.http.scaladsl.model.HttpMethods.POST
|
||||
import akka.http.scaladsl.model._
|
||||
import akka.http.scaladsl.model.headers.CacheDirectives.{`max-age`, `no-store`, public}
|
||||
import akka.http.scaladsl.model.headers.{`Access-Control-Allow-Headers`, `Access-Control-Allow-Methods`, `Cache-Control`}
|
||||
import akka.http.scaladsl.model.ws.{Message, TextMessage}
|
||||
import akka.http.scaladsl.server.directives.{Credentials, LoggingMagnet}
|
||||
import akka.stream.{ActorMaterializer, OverflowStrategy}
|
||||
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 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.{Eclair, ShortChannelId}
|
||||
import grizzled.slf4j.Logging
|
||||
import org.json4s.{ShortTypeHints, TypeHints}
|
||||
import org.json4s.jackson.Serialization
|
||||
import scodec.bits.ByteVector
|
||||
|
||||
import scala.concurrent.{ExecutionContext, Future}
|
||||
import scala.concurrent.Future
|
||||
import scala.concurrent.duration._
|
||||
|
||||
case class ErrorResponse(error: String)
|
||||
|
||||
trait Service extends Directives with Logging {
|
||||
trait Service extends ExtraDirectives with Logging {
|
||||
|
||||
// important! Must NOT import the unmarshaller as it is too generic...see https://github.com/akka/akka-http/issues/541
|
||||
import JsonSupport.marshaller
|
||||
import JsonSupport.formats
|
||||
import JsonSupport.serialization
|
||||
import JsonSupport.{formats, marshaller, serialization}
|
||||
|
||||
// used to send typed messages over the websocket
|
||||
val formatsWithTypeHint = formats.withTypeHintFieldName("type") +
|
||||
|
@ -73,6 +76,11 @@ trait Service extends Directives with Logging {
|
|||
val channelId = "channelId".as[ByteVector32](sha256HashUnmarshaller)
|
||||
val nodeId = "nodeId".as[PublicKey]
|
||||
val shortChannelId = "shortChannelId".as[ShortChannelId](shortChannelIdUnmarshaller)
|
||||
val paymentHash = "paymentHash".as[ByteVector32](sha256HashUnmarshaller)
|
||||
val from = "from".as[Long]
|
||||
val to = "to".as[Long]
|
||||
val amountMsat = "amountMsat".as[Long]
|
||||
val invoice = "invoice".as[PaymentRequest]
|
||||
|
||||
val apiExceptionHandler = ExceptionHandler {
|
||||
case t: Throwable =>
|
||||
|
@ -148,6 +156,11 @@ trait Service extends Directives with Logging {
|
|||
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))
|
||||
|
@ -162,11 +175,6 @@ trait Service extends Directives with Logging {
|
|||
complete(eclairApi.forceClose(Right(shortChannelId)))
|
||||
}
|
||||
} ~
|
||||
path("updaterelayfee") {
|
||||
formFields(channelId, "feeBaseMsat".as[Long], "feeProportionalMillionths".as[Long]) { (channelId, feeBase, feeProportional) =>
|
||||
complete(eclairApi.updateRelayFee(channelId.toString, feeBase, feeProportional))
|
||||
}
|
||||
} ~
|
||||
path("peers") {
|
||||
complete(eclairApi.peersInfo())
|
||||
} ~
|
||||
|
@ -181,39 +189,35 @@ trait Service extends Directives with Logging {
|
|||
}
|
||||
} ~
|
||||
path("allnodes") {
|
||||
complete(eclairApi.allnodes())
|
||||
complete(eclairApi.allNodes())
|
||||
} ~
|
||||
path("allchannels") {
|
||||
complete(eclairApi.allchannels())
|
||||
complete(eclairApi.allChannels())
|
||||
} ~
|
||||
path("allupdates") {
|
||||
formFields(nodeId.?) { nodeId_opt =>
|
||||
complete(eclairApi.allupdates(nodeId_opt))
|
||||
}
|
||||
} ~
|
||||
path("receive") {
|
||||
formFields("description".as[String], "amountMsat".as[Long].?, "expireIn".as[Long].?) { (desc, amountMsat, expire) =>
|
||||
complete(eclairApi.receive(desc, amountMsat, expire))
|
||||
}
|
||||
} ~
|
||||
path("parseinvoice") {
|
||||
formFields("invoice".as[PaymentRequest]) { invoice =>
|
||||
complete(invoice)
|
||||
complete(eclairApi.allUpdates(nodeId_opt))
|
||||
}
|
||||
} ~
|
||||
path("findroute") {
|
||||
formFields("invoice".as[PaymentRequest], "amountMsat".as[Long].?) {
|
||||
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".as[Long]) { (nodeId, amount) =>
|
||||
complete(eclairApi.findRoute(nodeId, amount))
|
||||
}
|
||||
} ~
|
||||
path("send") {
|
||||
formFields("invoice".as[PaymentRequest], "amountMsat".as[Long].?, "maxAttempts".as[Int].?) {
|
||||
} ~
|
||||
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) =>
|
||||
|
@ -222,25 +226,52 @@ trait Service extends Directives with Logging {
|
|||
}
|
||||
} ~
|
||||
path("sendtonode") {
|
||||
formFields("amountMsat".as[Long], "paymentHash".as[ByteVector32](sha256HashUnmarshaller), "nodeId".as[PublicKey], "maxAttempts".as[Int].?) { (amountMsat, paymentHash, nodeId, maxAttempts) =>
|
||||
formFields(amountMsat, paymentHash, nodeId, "maxAttempts".as[Int].?) { (amountMsat, paymentHash, nodeId, maxAttempts) =>
|
||||
complete(eclairApi.send(nodeId, amountMsat, paymentHash, maxAttempts = maxAttempts))
|
||||
}
|
||||
} ~
|
||||
path("checkpayment") {
|
||||
formFields("paymentHash".as[ByteVector32](sha256HashUnmarshaller)) { paymentHash =>
|
||||
complete(eclairApi.checkpayment(paymentHash))
|
||||
} ~ formFields("invoice".as[PaymentRequest]) { invoice =>
|
||||
complete(eclairApi.checkpayment(invoice.paymentHash))
|
||||
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".as[Long].?, "to".as[Long].?) { (from, to) =>
|
||||
complete(eclairApi.audit(from, to))
|
||||
formFields(from.?, to.?) { (from_opt, to_opt) =>
|
||||
complete(eclairApi.audit(from_opt, to_opt))
|
||||
}
|
||||
} ~
|
||||
path("networkfees") {
|
||||
formFields("from".as[Long].?, "to".as[Long].?) { (from, to) =>
|
||||
complete(eclairApi.networkFees(from, to))
|
||||
formFields(from.?, to.?) { (from_opt, to_opt) =>
|
||||
complete(eclairApi.networkFees(from_opt, to_opt))
|
||||
}
|
||||
} ~
|
||||
path("channelstats") {
|
||||
|
|
|
@ -1256,8 +1256,8 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
|
|||
// for our outgoing payments, let's send events if we know that they will settle on chain
|
||||
Closing
|
||||
.onchainOutgoingHtlcs(d.commitments.localCommit, d.commitments.remoteCommit, d.commitments.remoteNextCommitInfo.left.toOption.map(_.nextRemoteCommit), tx)
|
||||
.filter(add => Closing.isSentByLocal(add.id, d.commitments.originChannels)) // we only care about htlcs for which we were the original sender here
|
||||
.foreach(add => context.system.eventStream.publish(PaymentSettlingOnChain(amount = MilliSatoshi(add.amountMsat), add.paymentHash)))
|
||||
.map(add => (add, d.commitments.originChannels.get(add.id).collect { case Local(id, _) => id })) // we resolve the payment id if this was a local payment
|
||||
.collect { case (add, Some(id)) => context.system.eventStream.publish(PaymentSettlingOnChain(id, amount = MilliSatoshi(add.amountMsat), add.paymentHash)) }
|
||||
// then let's see if any of the possible close scenarii can be considered done
|
||||
val mutualCloseDone = d.mutualClosePublished.exists(_.txid == tx.txid) // this case is trivial, in a mutual close scenario we only need to make sure that one of the closing txes is confirmed
|
||||
val localCommitDone = localCommitPublished1.map(Closing.isLocalCommitDone(_)).getOrElse(false)
|
||||
|
@ -2052,9 +2052,9 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
|
|||
}
|
||||
}
|
||||
|
||||
def origin(c: CMD_ADD_HTLC): Origin = c.upstream_opt match {
|
||||
case None => Local(Some(sender))
|
||||
case Some(u) => Relayed(u.channelId, u.id, u.amountMsat, c.amountMsat)
|
||||
def origin(c: CMD_ADD_HTLC): Origin = c.upstream match {
|
||||
case Left(id) => Local(id, Some(sender)) // we were the origin of the payment
|
||||
case Right(u) => Relayed(u.channelId, u.id, u.amountMsat, c.amountMsat) // this is a relayed payment
|
||||
}
|
||||
|
||||
def feePaid(fee: Satoshi, tx: Transaction, desc: String, channelId: ByteVector32) = {
|
||||
|
|
|
@ -16,6 +16,8 @@
|
|||
|
||||
package fr.acinq.eclair.channel
|
||||
|
||||
import java.util.UUID
|
||||
|
||||
import akka.actor.ActorRef
|
||||
import fr.acinq.bitcoin.Crypto.{Point, PublicKey}
|
||||
import fr.acinq.bitcoin.{ByteVector32, DeterministicWallet, OutPoint, Satoshi, Transaction}
|
||||
|
@ -106,7 +108,7 @@ case class BITCOIN_PARENT_TX_CONFIRMED(childTx: Transaction) extends BitcoinEven
|
|||
*/
|
||||
|
||||
sealed trait Command
|
||||
final case class CMD_ADD_HTLC(amountMsat: Long, paymentHash: ByteVector32, cltvExpiry: Long, onion: ByteVector = Sphinx.LAST_PACKET.serialize, upstream_opt: Option[UpdateAddHtlc] = None, commit: Boolean = false, redirected: Boolean = false) extends Command
|
||||
final case class CMD_ADD_HTLC(amountMsat: Long, paymentHash: ByteVector32, cltvExpiry: Long, onion: ByteVector = Sphinx.LAST_PACKET.serialize, upstream: Either[UUID, UpdateAddHtlc], commit: Boolean = false, redirected: Boolean = false) extends Command
|
||||
final case class CMD_FULFILL_HTLC(id: Long, r: ByteVector32, commit: Boolean = false) extends Command
|
||||
final case class CMD_FAIL_HTLC(id: Long, reason: Either[ByteVector, FailureMessage], commit: Boolean = false) extends Command
|
||||
final case class CMD_FAIL_MALFORMED_HTLC(id: Long, onionHash: ByteVector32, failureCode: Int, commit: Boolean = false) extends Command
|
||||
|
|
|
@ -811,7 +811,7 @@ object Helpers {
|
|||
* @return
|
||||
*/
|
||||
def isSentByLocal(htlcId: Long, originChannels: Map[Long, Origin]) = originChannels.get(htlcId) match {
|
||||
case Some(Local(_)) => true
|
||||
case Some(Local(_, _)) => true
|
||||
case _ => false
|
||||
}
|
||||
|
||||
|
|
|
@ -16,40 +16,70 @@
|
|||
|
||||
package fr.acinq.eclair.db
|
||||
|
||||
import java.util.UUID
|
||||
import fr.acinq.bitcoin.ByteVector32
|
||||
import fr.acinq.eclair.payment.PaymentRequest
|
||||
|
||||
/**
|
||||
* Store the Lightning payments received by the node. Sent and relayed payments are not persisted.
|
||||
* <p>
|
||||
* A payment is a [[Payment]] object. In the local context of a LN node, it is safe to consider that
|
||||
* a payment is uniquely identified by its payment hash. As such, implementations of this database can use the payment
|
||||
* hash as a unique key and index.
|
||||
* <p>
|
||||
* Basic operations on this DB are:
|
||||
* <ul>
|
||||
* <li>insertion
|
||||
* <li>find by payment hash
|
||||
* <li>list all
|
||||
* </ul>
|
||||
* Payments should not be updated nor deleted.
|
||||
*/
|
||||
trait PaymentsDb {
|
||||
|
||||
def addPayment(payment: Payment)
|
||||
// creates a record for a non yet finalized outgoing payment
|
||||
def addOutgoingPayment(outgoingPayment: OutgoingPayment)
|
||||
|
||||
def findByPaymentHash(paymentHash: ByteVector32): Option[Payment]
|
||||
// updates the status of the payment, if the newStatus is SUCCEEDED you must supply a preimage
|
||||
def updateOutgoingPayment(id: UUID, newStatus: OutgoingPaymentStatus.Value, preimage: Option[ByteVector32] = None)
|
||||
|
||||
def listPayments(): Seq[Payment]
|
||||
def getOutgoingPayment(id: UUID): Option[OutgoingPayment]
|
||||
|
||||
def close: Unit
|
||||
// all the outgoing payment (attempts) to pay the given paymentHash
|
||||
def getOutgoingPayments(paymentHash: ByteVector32): Seq[OutgoingPayment]
|
||||
|
||||
def listOutgoingPayments(): Seq[OutgoingPayment]
|
||||
|
||||
def addPaymentRequest(pr: PaymentRequest, preimage: ByteVector32)
|
||||
|
||||
def getPaymentRequest(paymentHash: ByteVector32): Option[PaymentRequest]
|
||||
|
||||
def getPendingPaymentRequestAndPreimage(paymentHash: ByteVector32): Option[(ByteVector32, PaymentRequest)]
|
||||
|
||||
def listPaymentRequests(from: Long, to: Long): Seq[PaymentRequest]
|
||||
|
||||
// returns non paid, non expired payment requests
|
||||
def listPendingPaymentRequests(from: Long, to: Long): Seq[PaymentRequest]
|
||||
|
||||
// assumes there is already a payment request for it (the record for the given payment hash)
|
||||
def addIncomingPayment(payment: IncomingPayment)
|
||||
|
||||
def getIncomingPayment(paymentHash: ByteVector32): Option[IncomingPayment]
|
||||
|
||||
def listIncomingPayments(): Seq[IncomingPayment]
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Payment object stored in DB.
|
||||
* Incoming payment object stored in DB.
|
||||
*
|
||||
* @param payment_hash identifier of the payment
|
||||
* @param amount_msat amount of the payment, in milli-satoshis
|
||||
* @param timestamp absolute time in seconds since UNIX epoch when the payment was created.
|
||||
* @param paymentHash identifier of the payment
|
||||
* @param amountMsat amount of the payment, in milli-satoshis
|
||||
* @param receivedAt absolute time in seconds since UNIX epoch when the payment was received.
|
||||
*/
|
||||
case class Payment(payment_hash: ByteVector32, amount_msat: Long, timestamp: Long)
|
||||
case class IncomingPayment(paymentHash: ByteVector32, amountMsat: Long, receivedAt: Long)
|
||||
|
||||
/**
|
||||
* Sent payment is every payment that is sent by this node, they may not be finalized and
|
||||
* when is final it can be failed or successful.
|
||||
*
|
||||
* @param id internal payment identifier
|
||||
* @param paymentHash payment_hash
|
||||
* @param preimage the preimage of the payment_hash, known if the outgoing payment was successful
|
||||
* @param amountMsat amount of the payment, in milli-satoshis
|
||||
* @param createdAt absolute time in seconds since UNIX epoch when the payment was created.
|
||||
* @param completedAt absolute time in seconds since UNIX epoch when the payment succeeded.
|
||||
* @param status current status of the payment.
|
||||
*/
|
||||
case class OutgoingPayment(id: UUID, paymentHash: ByteVector32, preimage:Option[ByteVector32], amountMsat: Long, createdAt: Long, completedAt: Option[Long], status: OutgoingPaymentStatus.Value)
|
||||
|
||||
object OutgoingPaymentStatus extends Enumeration {
|
||||
val PENDING = Value(1, "PENDING")
|
||||
val SUCCEEDED = Value(2, "SUCCEEDED")
|
||||
val FAILED = Value(3, "FAILED")
|
||||
}
|
|
@ -17,39 +17,67 @@
|
|||
package fr.acinq.eclair.db.sqlite
|
||||
|
||||
import java.sql.Connection
|
||||
import java.util.UUID
|
||||
|
||||
import fr.acinq.bitcoin.Crypto.PublicKey
|
||||
import fr.acinq.bitcoin.MilliSatoshi
|
||||
import fr.acinq.eclair.channel.{AvailableBalanceChanged, NetworkFeePaid}
|
||||
import fr.acinq.eclair.db.{AuditDb, ChannelLifecycleEvent, NetworkFee, Stats}
|
||||
import fr.acinq.eclair.payment.{PaymentReceived, PaymentRelayed, PaymentSent}
|
||||
import fr.acinq.eclair.wire.ChannelCodecs
|
||||
import grizzled.slf4j.Logging
|
||||
|
||||
import scala.collection.immutable.Queue
|
||||
import scala.compat.Platform
|
||||
|
||||
class SqliteAuditDb(sqlite: Connection) extends AuditDb {
|
||||
class SqliteAuditDb(sqlite: Connection) extends AuditDb with Logging {
|
||||
|
||||
import SqliteUtils._
|
||||
import ExtendedResultSet._
|
||||
|
||||
val DB_NAME = "audit"
|
||||
val CURRENT_VERSION = 1
|
||||
val CURRENT_VERSION = 2
|
||||
|
||||
using(sqlite.createStatement()) { statement =>
|
||||
require(getVersion(statement, DB_NAME, CURRENT_VERSION) == CURRENT_VERSION) // there is only one version currently deployed
|
||||
statement.executeUpdate("CREATE TABLE IF NOT EXISTS balance_updated (channel_id BLOB NOT NULL, node_id BLOB NOT NULL, amount_msat INTEGER NOT NULL, capacity_sat INTEGER NOT NULL, reserve_sat INTEGER NOT NULL, timestamp INTEGER NOT NULL)")
|
||||
statement.executeUpdate("CREATE TABLE IF NOT EXISTS sent (amount_msat INTEGER NOT NULL, fees_msat INTEGER NOT NULL, payment_hash BLOB NOT NULL, payment_preimage BLOB NOT NULL, to_channel_id BLOB NOT NULL, timestamp INTEGER NOT NULL)")
|
||||
statement.executeUpdate("CREATE TABLE IF NOT EXISTS received (amount_msat INTEGER NOT NULL, payment_hash BLOB NOT NULL, from_channel_id BLOB NOT NULL, timestamp INTEGER NOT NULL)")
|
||||
statement.executeUpdate("CREATE TABLE IF NOT EXISTS relayed (amount_in_msat INTEGER NOT NULL, amount_out_msat INTEGER NOT NULL, payment_hash BLOB NOT NULL, from_channel_id BLOB NOT NULL, to_channel_id BLOB NOT NULL, timestamp INTEGER NOT NULL)")
|
||||
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 STRING NOT NULL, timestamp INTEGER NOT NULL)")
|
||||
getVersion(statement, DB_NAME, CURRENT_VERSION) match {
|
||||
case 1 => // previous version let's migrate
|
||||
logger.warn(s"Performing db migration for DB $DB_NAME, found version=1 current=$CURRENT_VERSION")
|
||||
statement.executeUpdate("CREATE TABLE IF NOT EXISTS balance_updated (channel_id BLOB NOT NULL, node_id BLOB NOT NULL, amount_msat INTEGER NOT NULL, capacity_sat INTEGER NOT NULL, reserve_sat INTEGER NOT NULL, timestamp INTEGER NOT NULL)")
|
||||
statement.executeUpdate("CREATE TABLE IF NOT EXISTS sent (amount_msat INTEGER NOT NULL, fees_msat INTEGER NOT NULL, payment_hash BLOB NOT NULL, payment_preimage BLOB NOT NULL, to_channel_id BLOB NOT NULL, timestamp INTEGER NOT NULL)")
|
||||
statement.executeUpdate("CREATE TABLE IF NOT EXISTS received (amount_msat INTEGER NOT NULL, payment_hash BLOB NOT NULL, from_channel_id BLOB NOT NULL, timestamp INTEGER NOT NULL)")
|
||||
statement.executeUpdate("CREATE TABLE IF NOT EXISTS relayed (amount_in_msat INTEGER NOT NULL, amount_out_msat INTEGER NOT NULL, payment_hash BLOB NOT NULL, from_channel_id BLOB NOT NULL, to_channel_id BLOB NOT NULL, timestamp INTEGER NOT NULL)")
|
||||
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 STRING NOT NULL, timestamp INTEGER NOT NULL)")
|
||||
|
||||
statement.executeUpdate("CREATE INDEX IF NOT EXISTS balance_updated_idx ON balance_updated(timestamp)")
|
||||
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)")
|
||||
statement.executeUpdate("CREATE INDEX IF NOT EXISTS network_fees_timestamp_idx ON network_fees(timestamp)")
|
||||
statement.executeUpdate("CREATE INDEX IF NOT EXISTS channel_events_timestamp_idx ON channel_events(timestamp)")
|
||||
// add id
|
||||
statement.executeUpdate(s"ALTER TABLE sent ADD id BLOB DEFAULT '${ChannelCodecs.UNKNOWN_UUID.toString}' NOT NULL")
|
||||
|
||||
statement.executeUpdate("CREATE INDEX IF NOT EXISTS balance_updated_idx ON balance_updated(timestamp)")
|
||||
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)")
|
||||
statement.executeUpdate("CREATE INDEX IF NOT EXISTS network_fees_timestamp_idx ON network_fees(timestamp)")
|
||||
statement.executeUpdate("CREATE INDEX IF NOT EXISTS channel_events_timestamp_idx ON channel_events(timestamp)")
|
||||
|
||||
// update version
|
||||
setVersion(statement, DB_NAME, CURRENT_VERSION)
|
||||
case CURRENT_VERSION =>
|
||||
statement.executeUpdate("CREATE TABLE IF NOT EXISTS balance_updated (channel_id BLOB NOT NULL, node_id BLOB NOT NULL, amount_msat INTEGER NOT NULL, capacity_sat INTEGER NOT NULL, reserve_sat INTEGER NOT NULL, timestamp INTEGER NOT NULL)")
|
||||
statement.executeUpdate("CREATE TABLE IF NOT EXISTS sent (amount_msat INTEGER NOT NULL, fees_msat INTEGER NOT NULL, payment_hash BLOB NOT NULL, payment_preimage BLOB NOT NULL, to_channel_id BLOB NOT NULL, timestamp INTEGER NOT NULL, id BLOB NOT NULL)")
|
||||
statement.executeUpdate("CREATE TABLE IF NOT EXISTS received (amount_msat INTEGER NOT NULL, payment_hash BLOB NOT NULL, from_channel_id BLOB NOT NULL, timestamp INTEGER NOT NULL)")
|
||||
statement.executeUpdate("CREATE TABLE IF NOT EXISTS relayed (amount_in_msat INTEGER NOT NULL, amount_out_msat INTEGER NOT NULL, payment_hash BLOB NOT NULL, from_channel_id BLOB NOT NULL, to_channel_id BLOB NOT NULL, timestamp INTEGER NOT NULL)")
|
||||
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 STRING NOT NULL, timestamp INTEGER NOT NULL)")
|
||||
|
||||
statement.executeUpdate("CREATE INDEX IF NOT EXISTS balance_updated_idx ON balance_updated(timestamp)")
|
||||
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)")
|
||||
statement.executeUpdate("CREATE INDEX IF NOT EXISTS network_fees_timestamp_idx ON network_fees(timestamp)")
|
||||
statement.executeUpdate("CREATE INDEX IF NOT EXISTS channel_events_timestamp_idx ON channel_events(timestamp)")
|
||||
|
||||
case unknownVersion => throw new RuntimeException(s"Unknown version of DB $DB_NAME found, version=$unknownVersion")
|
||||
}
|
||||
}
|
||||
|
||||
override def add(e: AvailableBalanceChanged): Unit =
|
||||
|
@ -76,13 +104,15 @@ class SqliteAuditDb(sqlite: Connection) extends AuditDb {
|
|||
}
|
||||
|
||||
override def add(e: PaymentSent): Unit =
|
||||
using(sqlite.prepareStatement("INSERT INTO sent VALUES (?, ?, ?, ?, ?, ?)")) { statement =>
|
||||
using(sqlite.prepareStatement("INSERT INTO sent VALUES (?, ?, ?, ?, ?, ?, ?)")) { statement =>
|
||||
statement.setLong(1, e.amount.toLong)
|
||||
statement.setLong(2, e.feesPaid.toLong)
|
||||
statement.setBytes(3, e.paymentHash.toArray)
|
||||
statement.setBytes(4, e.paymentPreimage.toArray)
|
||||
statement.setBytes(5, e.toChannelId.toArray)
|
||||
statement.setLong(6, e.timestamp)
|
||||
statement.setBytes(7, e.id.toString.getBytes)
|
||||
|
||||
statement.executeUpdate()
|
||||
}
|
||||
|
||||
|
@ -125,6 +155,7 @@ class SqliteAuditDb(sqlite: Connection) extends AuditDb {
|
|||
var q: Queue[PaymentSent] = Queue()
|
||||
while (rs.next()) {
|
||||
q = q :+ PaymentSent(
|
||||
id = UUID.fromString(rs.getString("id")),
|
||||
amount = MilliSatoshi(rs.getLong("amount_msat")),
|
||||
feesPaid = MilliSatoshi(rs.getLong("fees_msat")),
|
||||
paymentHash = rs.getByteVector32("payment_hash"),
|
||||
|
|
|
@ -17,68 +17,211 @@
|
|||
package fr.acinq.eclair.db.sqlite
|
||||
|
||||
import java.sql.Connection
|
||||
|
||||
import java.time.Instant
|
||||
import java.util.UUID
|
||||
import fr.acinq.bitcoin.ByteVector32
|
||||
import fr.acinq.eclair.db.sqlite.SqliteUtils.{getVersion, using}
|
||||
import fr.acinq.eclair.db.{Payment, PaymentsDb}
|
||||
import fr.acinq.eclair.db.sqlite.SqliteUtils._
|
||||
import fr.acinq.eclair.db.{IncomingPayment, OutgoingPayment, OutgoingPaymentStatus, PaymentsDb}
|
||||
import fr.acinq.eclair.payment.PaymentRequest
|
||||
import grizzled.slf4j.Logging
|
||||
|
||||
import scala.collection.immutable.Queue
|
||||
import OutgoingPaymentStatus._
|
||||
|
||||
/**
|
||||
* Payments are stored in the `payments` table.
|
||||
* The primary key in this DB is the `payment_hash` column. Columns are not nullable.
|
||||
* <p>
|
||||
* Types:
|
||||
* <ul>
|
||||
* <li>`payment_hash`: BLOB
|
||||
* <li>`amount_msat`: INTEGER
|
||||
* <li>`timestamp`: INTEGER (unix timestamp)
|
||||
*/
|
||||
class SqlitePaymentsDb(sqlite: Connection) extends PaymentsDb with Logging {
|
||||
|
||||
import SqliteUtils.ExtendedResultSet._
|
||||
|
||||
val DB_NAME = "payments"
|
||||
val CURRENT_VERSION = 1
|
||||
val CURRENT_VERSION = 2
|
||||
|
||||
using(sqlite.createStatement()) { statement =>
|
||||
require(getVersion(statement, DB_NAME, CURRENT_VERSION) == CURRENT_VERSION) // there is only one version currently deployed
|
||||
statement.executeUpdate("CREATE TABLE IF NOT EXISTS payments (payment_hash BLOB NOT NULL PRIMARY KEY, amount_msat INTEGER NOT NULL, timestamp INTEGER NOT NULL)")
|
||||
require(getVersion(statement, DB_NAME, CURRENT_VERSION) <= CURRENT_VERSION) // version 2 is "backward compatible" in the sense that it uses separate tables from version 1. There is no migration though
|
||||
statement.executeUpdate("CREATE TABLE IF NOT EXISTS received_payments (payment_hash BLOB NOT NULL PRIMARY KEY, preimage BLOB NOT NULL, payment_request TEXT NOT NULL, received_msat INTEGER, created_at INTEGER NOT NULL, expire_at INTEGER, received_at INTEGER)")
|
||||
statement.executeUpdate("CREATE TABLE IF NOT EXISTS sent_payments (id TEXT NOT NULL PRIMARY KEY, payment_hash BLOB NOT NULL, preimage BLOB, amount_msat INTEGER NOT NULL, created_at INTEGER NOT NULL, completed_at INTEGER, status VARCHAR NOT NULL)")
|
||||
statement.executeUpdate("CREATE INDEX IF NOT EXISTS payment_hash_idx ON sent_payments(payment_hash)")
|
||||
setVersion(statement, DB_NAME, CURRENT_VERSION)
|
||||
}
|
||||
|
||||
override def addPayment(payment: Payment): Unit = {
|
||||
using(sqlite.prepareStatement("INSERT INTO payments VALUES (?, ?, ?)")) { statement =>
|
||||
statement.setBytes(1, payment.payment_hash.toArray)
|
||||
statement.setLong(2, payment.amount_msat)
|
||||
statement.setLong(3, payment.timestamp)
|
||||
override def addOutgoingPayment(sent: OutgoingPayment): Unit = {
|
||||
using(sqlite.prepareStatement("INSERT INTO sent_payments (id, payment_hash, amount_msat, created_at, status) VALUES (?, ?, ?, ?, ?)")) { statement =>
|
||||
statement.setString(1, sent.id.toString)
|
||||
statement.setBytes(2, sent.paymentHash.toArray)
|
||||
statement.setLong(3, sent.amountMsat)
|
||||
statement.setLong(4, sent.createdAt)
|
||||
statement.setString(5, sent.status.toString)
|
||||
val res = statement.executeUpdate()
|
||||
logger.debug(s"inserted $res payment=${payment} in DB")
|
||||
logger.debug(s"inserted $res payment=${sent.paymentHash} into payment DB")
|
||||
}
|
||||
}
|
||||
|
||||
override def findByPaymentHash(paymentHash: ByteVector32): Option[Payment] = {
|
||||
using(sqlite.prepareStatement("SELECT payment_hash, amount_msat, timestamp FROM payments WHERE payment_hash = ?")) { statement =>
|
||||
statement.setBytes(1, paymentHash.toArray)
|
||||
override def updateOutgoingPayment(id: UUID, newStatus: OutgoingPaymentStatus.Value, preimage: Option[ByteVector32] = None) = {
|
||||
require((newStatus == SUCCEEDED && preimage.isDefined) || (newStatus == FAILED && preimage.isEmpty), "Wrong combination of state/preimage")
|
||||
|
||||
using(sqlite.prepareStatement("UPDATE sent_payments SET (completed_at, preimage, status) = (?, ?, ?) WHERE id = ? AND completed_at IS NULL")) { statement =>
|
||||
statement.setLong(1, Instant.now().getEpochSecond)
|
||||
statement.setBytes(2, if (preimage.isEmpty) null else preimage.get.toArray)
|
||||
statement.setString(3, newStatus.toString)
|
||||
statement.setString(4, id.toString)
|
||||
if (statement.executeUpdate() == 0) throw new IllegalArgumentException(s"Tried to update an outgoing payment (id=$id) already in final status with=$newStatus")
|
||||
}
|
||||
}
|
||||
|
||||
override def getOutgoingPayment(id: UUID): Option[OutgoingPayment] = {
|
||||
using(sqlite.prepareStatement("SELECT id, payment_hash, preimage, amount_msat, created_at, completed_at, status FROM sent_payments WHERE id = ?")) { statement =>
|
||||
statement.setString(1, id.toString)
|
||||
val rs = statement.executeQuery()
|
||||
if (rs.next()) {
|
||||
Some(Payment(rs.getByteVector32("payment_hash"), rs.getLong("amount_msat"), rs.getLong("timestamp")))
|
||||
Some(OutgoingPayment(
|
||||
UUID.fromString(rs.getString("id")),
|
||||
rs.getByteVector32("payment_hash"),
|
||||
rs.getByteVector32Nullable("preimage"),
|
||||
rs.getLong("amount_msat"),
|
||||
rs.getLong("created_at"),
|
||||
getNullableLong(rs, "completed_at"),
|
||||
OutgoingPaymentStatus.withName(rs.getString("status"))
|
||||
))
|
||||
} else {
|
||||
None
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
override def listPayments(): Seq[Payment] = {
|
||||
using(sqlite.createStatement()) { statement =>
|
||||
val rs = statement.executeQuery("SELECT payment_hash, amount_msat, timestamp FROM payments")
|
||||
var q: Queue[Payment] = Queue()
|
||||
override def getOutgoingPayments(paymentHash: ByteVector32): Seq[OutgoingPayment] = {
|
||||
using(sqlite.prepareStatement("SELECT id, payment_hash, preimage, amount_msat, created_at, completed_at, status FROM sent_payments WHERE payment_hash = ?")) { statement =>
|
||||
statement.setBytes(1, paymentHash.toArray)
|
||||
val rs = statement.executeQuery()
|
||||
var q: Queue[OutgoingPayment] = Queue()
|
||||
while (rs.next()) {
|
||||
q = q :+ Payment(rs.getByteVector32("payment_hash"), rs.getLong("amount_msat"), rs.getLong("timestamp"))
|
||||
q = q :+ OutgoingPayment(
|
||||
UUID.fromString(rs.getString("id")),
|
||||
rs.getByteVector32("payment_hash"),
|
||||
rs.getByteVector32Nullable("preimage"),
|
||||
rs.getLong("amount_msat"),
|
||||
rs.getLong("created_at"),
|
||||
getNullableLong(rs, "completed_at"),
|
||||
OutgoingPaymentStatus.withName(rs.getString("status"))
|
||||
)
|
||||
}
|
||||
q
|
||||
}
|
||||
}
|
||||
|
||||
override def close(): Unit = sqlite.close()
|
||||
}
|
||||
override def listOutgoingPayments(): Seq[OutgoingPayment] = {
|
||||
using(sqlite.createStatement()) { statement =>
|
||||
val rs = statement.executeQuery("SELECT id, payment_hash, preimage, amount_msat, created_at, completed_at, status FROM sent_payments")
|
||||
var q: Queue[OutgoingPayment] = Queue()
|
||||
while (rs.next()) {
|
||||
q = q :+ OutgoingPayment(
|
||||
UUID.fromString(rs.getString("id")),
|
||||
rs.getByteVector32("payment_hash"),
|
||||
rs.getByteVector32Nullable("preimage"),
|
||||
rs.getLong("amount_msat"),
|
||||
rs.getLong("created_at"),
|
||||
getNullableLong(rs, "completed_at"),
|
||||
OutgoingPaymentStatus.withName(rs.getString("status"))
|
||||
)
|
||||
}
|
||||
q
|
||||
}
|
||||
}
|
||||
|
||||
override def addPaymentRequest(pr: PaymentRequest, preimage: ByteVector32): Unit = {
|
||||
val insertStmt = pr.expiry match {
|
||||
case Some(_) => "INSERT INTO received_payments (payment_hash, preimage, payment_request, created_at, expire_at) VALUES (?, ?, ?, ?, ?)"
|
||||
case None => "INSERT INTO received_payments (payment_hash, preimage, payment_request, created_at) VALUES (?, ?, ?, ?)"
|
||||
}
|
||||
|
||||
using(sqlite.prepareStatement(insertStmt)) { statement =>
|
||||
statement.setBytes(1, pr.paymentHash.toArray)
|
||||
statement.setBytes(2, preimage.toArray)
|
||||
statement.setString(3, PaymentRequest.write(pr))
|
||||
statement.setLong(4, pr.timestamp)
|
||||
pr.expiry.foreach { ex => statement.setLong(5, pr.timestamp + ex) } // we store "when" the invoice will expire
|
||||
statement.executeUpdate()
|
||||
}
|
||||
}
|
||||
|
||||
override def getPaymentRequest(paymentHash: ByteVector32): Option[PaymentRequest] = {
|
||||
using(sqlite.prepareStatement("SELECT payment_request FROM received_payments WHERE payment_hash = ?")) { statement =>
|
||||
statement.setBytes(1, paymentHash.toArray)
|
||||
val rs = statement.executeQuery()
|
||||
if (rs.next()) {
|
||||
Some(PaymentRequest.read(rs.getString("payment_request")))
|
||||
} else {
|
||||
None
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
override def getPendingPaymentRequestAndPreimage(paymentHash: ByteVector32): Option[(ByteVector32, PaymentRequest)] = {
|
||||
using(sqlite.prepareStatement("SELECT payment_request, preimage FROM received_payments WHERE payment_hash = ? AND received_at IS NULL")) { statement =>
|
||||
statement.setBytes(1, paymentHash.toArray)
|
||||
val rs = statement.executeQuery()
|
||||
if (rs.next()) {
|
||||
val preimage = rs.getByteVector32("preimage")
|
||||
val pr = PaymentRequest.read(rs.getString("payment_request"))
|
||||
Some(preimage, pr)
|
||||
} else {
|
||||
None
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
override def listPaymentRequests(from: Long, to: Long): Seq[PaymentRequest] = listPaymentRequests(from, to, pendingOnly = false)
|
||||
|
||||
override def listPendingPaymentRequests(from: Long, to: Long): Seq[PaymentRequest] = listPaymentRequests(from, to, pendingOnly = true)
|
||||
|
||||
def listPaymentRequests(from: Long, to: Long, pendingOnly: Boolean): Seq[PaymentRequest] = {
|
||||
val queryStmt = pendingOnly match {
|
||||
case true => "SELECT payment_request FROM received_payments WHERE created_at > ? AND created_at < ? AND (expire_at > ? OR expire_at IS NULL) AND received_msat IS NULL ORDER BY created_at DESC"
|
||||
case false => "SELECT payment_request FROM received_payments WHERE created_at > ? AND created_at < ? ORDER BY created_at DESC"
|
||||
}
|
||||
|
||||
using(sqlite.prepareStatement(queryStmt)) { statement =>
|
||||
statement.setLong(1, from)
|
||||
statement.setLong(2, to)
|
||||
if (pendingOnly) statement.setLong(3, Instant.now().getEpochSecond)
|
||||
|
||||
val rs = statement.executeQuery()
|
||||
var q: Queue[PaymentRequest] = Queue()
|
||||
while (rs.next()) {
|
||||
q = q :+ PaymentRequest.read(rs.getString("payment_request"))
|
||||
}
|
||||
q
|
||||
}
|
||||
}
|
||||
|
||||
override def addIncomingPayment(payment: IncomingPayment): Unit = {
|
||||
using(sqlite.prepareStatement("UPDATE received_payments SET (received_msat, received_at) = (?, ?) WHERE payment_hash = ?")) { statement =>
|
||||
statement.setLong(1, payment.amountMsat)
|
||||
statement.setLong(2, payment.receivedAt)
|
||||
statement.setBytes(3, payment.paymentHash.toArray)
|
||||
val res = statement.executeUpdate()
|
||||
if (res == 0) throw new IllegalArgumentException("Inserted a received payment without having an invoice")
|
||||
}
|
||||
}
|
||||
|
||||
override def getIncomingPayment(paymentHash: ByteVector32): Option[IncomingPayment] = {
|
||||
using(sqlite.prepareStatement("SELECT payment_hash, received_msat, received_at FROM received_payments WHERE payment_hash = ? AND received_msat > 0")) { statement =>
|
||||
statement.setBytes(1, paymentHash.toArray)
|
||||
val rs = statement.executeQuery()
|
||||
if (rs.next()) {
|
||||
Some(IncomingPayment(rs.getByteVector32("payment_hash"), rs.getLong("received_msat"), rs.getLong("received_at")))
|
||||
} else {
|
||||
None
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
override def listIncomingPayments(): Seq[IncomingPayment] = {
|
||||
using(sqlite.createStatement()) { statement =>
|
||||
val rs = statement.executeQuery("SELECT payment_hash, received_msat, received_at FROM received_payments WHERE received_msat > 0")
|
||||
var q: Queue[IncomingPayment] = Queue()
|
||||
while (rs.next()) {
|
||||
q = q :+ IncomingPayment(rs.getByteVector32("payment_hash"), rs.getLong("received_msat"), rs.getLong("received_at"))
|
||||
}
|
||||
q
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -44,7 +44,8 @@ object SqliteUtils {
|
|||
|
||||
/**
|
||||
* Several logical databases (channels, network, peers) may be stored in the same physical sqlite database.
|
||||
* We keep track of their respective version using a dedicated table.
|
||||
* We keep track of their respective version using a dedicated table. The version entry will be created if
|
||||
* there is none but will never be updated here (use setVersion to do that).
|
||||
*
|
||||
* @param statement
|
||||
* @param db_name
|
||||
|
@ -60,6 +61,19 @@ object SqliteUtils {
|
|||
res.getInt("version")
|
||||
}
|
||||
|
||||
/**
|
||||
* Updates the version for a particular logical database, it will overwrite the previous version.
|
||||
* @param statement
|
||||
* @param db_name
|
||||
* @param newVersion
|
||||
* @return
|
||||
*/
|
||||
def setVersion(statement: Statement, db_name: String, newVersion: Int) = {
|
||||
statement.executeUpdate("CREATE TABLE IF NOT EXISTS versions (db_name TEXT NOT NULL PRIMARY KEY, version INTEGER NOT NULL)")
|
||||
// overwrite the existing version
|
||||
statement.executeUpdate(s"UPDATE versions SET version=$newVersion WHERE db_name='$db_name'")
|
||||
}
|
||||
|
||||
/**
|
||||
* This helper assumes that there is a "data" column available, decodable with the provided codec
|
||||
*
|
||||
|
@ -78,6 +92,19 @@ object SqliteUtils {
|
|||
q
|
||||
}
|
||||
|
||||
/**
|
||||
* This helper retrieves the value from a nullable integer column and interprets it as an option. This is needed
|
||||
* because `rs.getLong` would return `0` for a null value.
|
||||
* It is used on Android only
|
||||
*
|
||||
* @param label
|
||||
* @return
|
||||
*/
|
||||
def getNullableLong(rs: ResultSet, label: String) : Option[Long] = {
|
||||
val result = rs.getLong(label)
|
||||
if (rs.wasNull()) None else Some(result)
|
||||
}
|
||||
|
||||
/**
|
||||
* Obtain an exclusive lock on a sqlite database. This is useful when we want to make sure that only one process
|
||||
* accesses the database file (see https://www.sqlite.org/pragma.html).
|
||||
|
@ -99,6 +126,11 @@ object SqliteUtils {
|
|||
def getByteVector(columnLabel: String): ByteVector = ByteVector(rs.getBytes(columnLabel))
|
||||
|
||||
def getByteVector32(columnLabel: String): ByteVector32 = ByteVector32(ByteVector(rs.getBytes(columnLabel)))
|
||||
|
||||
def getByteVector32Nullable(columnLabel: String): Option[ByteVector32] = {
|
||||
val bytes = rs.getBytes(columnLabel)
|
||||
if(rs.wasNull()) None else Some(ByteVector32(ByteVector(bytes)))
|
||||
}
|
||||
}
|
||||
|
||||
object ExtendedResultSet {
|
||||
|
|
|
@ -62,7 +62,7 @@ class Autoprobe(nodeParams: NodeParams, router: ActorRef, paymentInitiator: Acto
|
|||
|
||||
case paymentResult: PaymentResult =>
|
||||
paymentResult match {
|
||||
case PaymentFailed(_, _ :+ RemoteFailure(_, ErrorPacket(targetNodeId, UnknownPaymentHash))) =>
|
||||
case PaymentFailed(_, _, _ :+ RemoteFailure(_, ErrorPacket(targetNodeId, UnknownPaymentHash))) =>
|
||||
log.info(s"payment probe successful to node=$targetNodeId")
|
||||
case _ =>
|
||||
log.info(s"payment probe failed with paymentResult=$paymentResult")
|
||||
|
|
|
@ -19,15 +19,14 @@ package fr.acinq.eclair.payment
|
|||
import akka.actor.{Actor, ActorLogging, Props, Status}
|
||||
import fr.acinq.bitcoin.{ByteVector32, Crypto, MilliSatoshi}
|
||||
import fr.acinq.eclair.channel.{CMD_FAIL_HTLC, CMD_FULFILL_HTLC, Channel}
|
||||
import fr.acinq.eclair.db.Payment
|
||||
import fr.acinq.eclair.payment.PaymentLifecycle.{CheckPayment, ReceivePayment}
|
||||
import fr.acinq.eclair.db.IncomingPayment
|
||||
import fr.acinq.eclair.payment.PaymentLifecycle.ReceivePayment
|
||||
import fr.acinq.eclair.wire._
|
||||
import fr.acinq.eclair.{Globals, NodeParams, randomBytes32}
|
||||
|
||||
import scala.compat.Platform
|
||||
import scala.concurrent.ExecutionContext
|
||||
import scala.concurrent.duration._
|
||||
import scala.util.Try
|
||||
import scala.util.{Failure, Success, Try}
|
||||
|
||||
/**
|
||||
* Simple payment handler that generates payment requests and fulfills incoming htlcs.
|
||||
|
@ -38,44 +37,28 @@ import scala.util.Try
|
|||
*/
|
||||
class LocalPaymentHandler(nodeParams: NodeParams) extends Actor with ActorLogging {
|
||||
|
||||
import LocalPaymentHandler._
|
||||
|
||||
implicit val ec: ExecutionContext = context.system.dispatcher
|
||||
context.system.scheduler.schedule(10 minutes, 10 minutes)(self ! PurgeExpiredRequests)
|
||||
val paymentDb = nodeParams.db.payments
|
||||
|
||||
override def receive: Receive = run(Map.empty)
|
||||
|
||||
def run(hash2preimage: Map[ByteVector32, PendingPaymentRequest]): Receive = {
|
||||
|
||||
case PurgeExpiredRequests =>
|
||||
context.become(run(hash2preimage.filterNot { case (_, pr) => hasExpired(pr) }))
|
||||
override def receive: Receive = {
|
||||
|
||||
case ReceivePayment(amount_opt, desc, expirySeconds_opt, extraHops, fallbackAddress_opt) =>
|
||||
Try {
|
||||
if (hash2preimage.size > nodeParams.maxPendingPaymentRequests) {
|
||||
throw new RuntimeException(s"too many pending payment requests (max=${nodeParams.maxPendingPaymentRequests})")
|
||||
}
|
||||
val paymentPreimage = randomBytes32
|
||||
val paymentHash = Crypto.sha256(paymentPreimage)
|
||||
val expirySeconds = expirySeconds_opt.getOrElse(nodeParams.paymentRequestExpiry.toSeconds)
|
||||
val paymentRequest = PaymentRequest(nodeParams.chainHash, amount_opt, paymentHash, nodeParams.privateKey, desc, fallbackAddress_opt, expirySeconds = Some(expirySeconds), extraHops = extraHops)
|
||||
log.debug(s"generated payment request=${PaymentRequest.write(paymentRequest)} from amount=$amount_opt")
|
||||
sender ! paymentRequest
|
||||
context.become(run(hash2preimage + (paymentHash -> PendingPaymentRequest(paymentPreimage, paymentRequest))))
|
||||
} recover { case t => sender ! Status.Failure(t) }
|
||||
|
||||
case CheckPayment(paymentHash) =>
|
||||
nodeParams.db.payments.findByPaymentHash(paymentHash) match {
|
||||
case Some(_) => sender ! true
|
||||
case _ => sender ! false
|
||||
log.debug(s"generated payment request={} from amount={}", PaymentRequest.write(paymentRequest), amount_opt)
|
||||
paymentDb.addPaymentRequest(paymentRequest, paymentPreimage)
|
||||
paymentRequest
|
||||
} match {
|
||||
case Success(paymentRequest) => sender ! paymentRequest
|
||||
case Failure(exception) => sender ! Status.Failure(exception)
|
||||
}
|
||||
|
||||
case htlc: UpdateAddHtlc =>
|
||||
hash2preimage
|
||||
.get(htlc.paymentHash) // we retrieve the request
|
||||
.filterNot(hasExpired) // and filter it out if it is expired (it will be purged independently)
|
||||
match {
|
||||
case Some(PendingPaymentRequest(paymentPreimage, paymentRequest)) =>
|
||||
paymentDb.getPendingPaymentRequestAndPreimage(htlc.paymentHash) match {
|
||||
case Some((paymentPreimage, paymentRequest)) =>
|
||||
val minFinalExpiry = Globals.blockCount.get() + paymentRequest.minFinalCltvExpiry.getOrElse(Channel.MIN_CLTV_EXPIRY)
|
||||
// The htlc amount must be equal or greater than the requested amount. A slight overpaying is permitted, however
|
||||
// it must not be greater than two times the requested amount.
|
||||
|
@ -92,18 +75,13 @@ class LocalPaymentHandler(nodeParams: NodeParams) extends Actor with ActorLoggin
|
|||
case _ =>
|
||||
log.info(s"received payment for paymentHash=${htlc.paymentHash} amountMsat=${htlc.amountMsat}")
|
||||
// amount is correct or was not specified in the payment request
|
||||
nodeParams.db.payments.addPayment(Payment(htlc.paymentHash, htlc.amountMsat, Platform.currentTime / 1000))
|
||||
nodeParams.db.payments.addIncomingPayment(IncomingPayment(htlc.paymentHash, htlc.amountMsat, Platform.currentTime / 1000))
|
||||
sender ! CMD_FULFILL_HTLC(htlc.id, paymentPreimage, commit = true)
|
||||
context.system.eventStream.publish(PaymentReceived(MilliSatoshi(htlc.amountMsat), htlc.paymentHash, htlc.channelId))
|
||||
context.become(run(hash2preimage - htlc.paymentHash))
|
||||
}
|
||||
case None =>
|
||||
sender ! CMD_FAIL_HTLC(htlc.id, Right(UnknownPaymentHash), commit = true)
|
||||
}
|
||||
|
||||
case 'requests =>
|
||||
// this is just for testing
|
||||
sender ! hash2preimage
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -112,13 +90,4 @@ object LocalPaymentHandler {
|
|||
|
||||
def props(nodeParams: NodeParams): Props = Props(new LocalPaymentHandler(nodeParams))
|
||||
|
||||
case object PurgeExpiredRequests
|
||||
|
||||
case class PendingPaymentRequest(preimage: ByteVector32, paymentRequest: PaymentRequest)
|
||||
|
||||
def hasExpired(pr: PendingPaymentRequest): Boolean = pr.paymentRequest.expiry match {
|
||||
case Some(expiry) => pr.paymentRequest.timestamp + expiry <= Platform.currentTime / 1000
|
||||
case None => false // this request will never expire
|
||||
}
|
||||
|
||||
}
|
|
@ -16,10 +16,10 @@
|
|||
|
||||
package fr.acinq.eclair.payment
|
||||
|
||||
import java.time.Instant
|
||||
import java.util.UUID
|
||||
import fr.acinq.bitcoin.{ByteVector32, MilliSatoshi}
|
||||
|
||||
import scala.compat.Platform
|
||||
|
||||
/**
|
||||
* Created by PM on 01/02/2017.
|
||||
*/
|
||||
|
@ -27,10 +27,10 @@ sealed trait PaymentEvent {
|
|||
val paymentHash: ByteVector32
|
||||
}
|
||||
|
||||
case class PaymentSent(amount: MilliSatoshi, feesPaid: MilliSatoshi, paymentHash: ByteVector32, paymentPreimage: ByteVector32, toChannelId: ByteVector32, timestamp: Long = Platform.currentTime) extends PaymentEvent
|
||||
case class PaymentSent(id: UUID, amount: MilliSatoshi, feesPaid: MilliSatoshi, paymentHash: ByteVector32, paymentPreimage: ByteVector32, toChannelId: ByteVector32, timestamp: Long = Instant.now().getEpochSecond) extends PaymentEvent
|
||||
|
||||
case class PaymentRelayed(amountIn: MilliSatoshi, amountOut: MilliSatoshi, paymentHash: ByteVector32, fromChannelId: ByteVector32, toChannelId: ByteVector32, timestamp: Long = Platform.currentTime) extends PaymentEvent
|
||||
case class PaymentRelayed(amountIn: MilliSatoshi, amountOut: MilliSatoshi, paymentHash: ByteVector32, fromChannelId: ByteVector32, toChannelId: ByteVector32, timestamp: Long = Instant.now().getEpochSecond) extends PaymentEvent
|
||||
|
||||
case class PaymentReceived(amount: MilliSatoshi, paymentHash: ByteVector32, fromChannelId: ByteVector32, timestamp: Long = Platform.currentTime) extends PaymentEvent
|
||||
case class PaymentReceived(amount: MilliSatoshi, paymentHash: ByteVector32, fromChannelId: ByteVector32, timestamp: Long = Instant.now().getEpochSecond) extends PaymentEvent
|
||||
|
||||
case class PaymentSettlingOnChain(amount: MilliSatoshi, paymentHash: ByteVector32, timestamp: Long = Platform.currentTime) extends PaymentEvent
|
||||
case class PaymentSettlingOnChain(id: UUID, amount: MilliSatoshi, paymentHash: ByteVector32, timestamp: Long = Instant.now().getEpochSecond) extends PaymentEvent
|
||||
|
|
|
@ -16,23 +16,26 @@
|
|||
|
||||
package fr.acinq.eclair.payment
|
||||
|
||||
import java.util.UUID
|
||||
import akka.actor.{Actor, ActorLogging, ActorRef, Props}
|
||||
import fr.acinq.bitcoin.Crypto.PublicKey
|
||||
import fr.acinq.eclair.NodeParams
|
||||
import fr.acinq.eclair.payment.PaymentLifecycle.SendPayment
|
||||
|
||||
/**
|
||||
* Created by PM on 29/08/2016.
|
||||
*/
|
||||
class PaymentInitiator(sourceNodeId: PublicKey, router: ActorRef, register: ActorRef) extends Actor with ActorLogging {
|
||||
class PaymentInitiator(nodeParams: NodeParams, router: ActorRef, register: ActorRef) extends Actor with ActorLogging {
|
||||
|
||||
override def receive: Receive = {
|
||||
case c: SendPayment =>
|
||||
val payFsm = context.actorOf(PaymentLifecycle.props(sourceNodeId, router, register))
|
||||
val paymentId = UUID.randomUUID()
|
||||
val payFsm = context.actorOf(PaymentLifecycle.props(nodeParams, paymentId, router, register))
|
||||
payFsm forward c
|
||||
sender ! paymentId
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
object PaymentInitiator {
|
||||
def props(sourceNodeId: PublicKey, router: ActorRef, register: ActorRef) = Props(classOf[PaymentInitiator], sourceNodeId, router, register)
|
||||
def props(nodeParams: NodeParams, router: ActorRef, register: ActorRef) = Props(classOf[PaymentInitiator], nodeParams, router, register)
|
||||
}
|
||||
|
|
|
@ -16,6 +16,9 @@
|
|||
|
||||
package fr.acinq.eclair.payment
|
||||
|
||||
import java.time.Instant
|
||||
import java.util.UUID
|
||||
|
||||
import akka.actor.{ActorRef, FSM, Props, Status}
|
||||
import fr.acinq.bitcoin.Crypto.PublicKey
|
||||
import fr.acinq.bitcoin.{ByteVector32, MilliSatoshi}
|
||||
|
@ -23,25 +26,28 @@ import fr.acinq.eclair._
|
|||
import fr.acinq.eclair.channel.{AddHtlcFailed, CMD_ADD_HTLC, Channel, Register}
|
||||
import fr.acinq.eclair.crypto.Sphinx.{ErrorPacket, Packet}
|
||||
import fr.acinq.eclair.crypto.{Sphinx, TransportHandler}
|
||||
import fr.acinq.eclair.db.{OutgoingPayment, OutgoingPaymentStatus}
|
||||
import fr.acinq.eclair.payment.PaymentLifecycle._
|
||||
import fr.acinq.eclair.payment.PaymentRequest.ExtraHop
|
||||
import fr.acinq.eclair.router._
|
||||
import fr.acinq.eclair.wire._
|
||||
import scodec.Attempt
|
||||
import scodec.bits.ByteVector
|
||||
|
||||
import scala.util.{Failure, Success}
|
||||
|
||||
/**
|
||||
* Created by PM on 26/08/2016.
|
||||
*/
|
||||
class PaymentLifecycle(sourceNodeId: PublicKey, router: ActorRef, register: ActorRef) extends FSM[PaymentLifecycle.State, PaymentLifecycle.Data] {
|
||||
class PaymentLifecycle(nodeParams: NodeParams, id: UUID, router: ActorRef, register: ActorRef) extends FSM[PaymentLifecycle.State, PaymentLifecycle.Data] {
|
||||
|
||||
val paymentsDb = nodeParams.db.payments
|
||||
|
||||
startWith(WAITING_FOR_REQUEST, WaitingForRequest)
|
||||
|
||||
when(WAITING_FOR_REQUEST) {
|
||||
case Event(c: SendPayment, WaitingForRequest) =>
|
||||
router ! RouteRequest(sourceNodeId, c.targetNodeId, c.amountMsat, c.assistedRoutes, routeParams = c.routeParams)
|
||||
router ! RouteRequest(nodeParams.nodeId, c.targetNodeId, c.amountMsat, c.assistedRoutes, routeParams = c.routeParams)
|
||||
paymentsDb.addOutgoingPayment(OutgoingPayment(id, c.paymentHash, None, c.amountMsat, Instant.now().getEpochSecond, None, OutgoingPaymentStatus.PENDING))
|
||||
goto(WAITING_FOR_ROUTE) using WaitingForRoute(sender, c, failures = Nil)
|
||||
}
|
||||
|
||||
|
@ -52,12 +58,13 @@ class PaymentLifecycle(sourceNodeId: PublicKey, router: ActorRef, register: Acto
|
|||
// we add one block in order to not have our htlc fail when a new block has just been found
|
||||
val finalExpiry = Globals.blockCount.get().toInt + c.finalCltvExpiry.toInt + 1
|
||||
|
||||
val (cmd, sharedSecrets) = buildCommand(c.amountMsat, finalExpiry, c.paymentHash, hops)
|
||||
val (cmd, sharedSecrets) = buildCommand(id, c.amountMsat, finalExpiry, c.paymentHash, hops)
|
||||
register ! Register.ForwardShortId(firstHop.lastUpdate.shortChannelId, cmd)
|
||||
goto(WAITING_FOR_PAYMENT_COMPLETE) using WaitingForComplete(s, c, cmd, failures, sharedSecrets, ignoreNodes, ignoreChannels, hops)
|
||||
|
||||
case Event(Status.Failure(t), WaitingForRoute(s, c, failures)) =>
|
||||
reply(s, PaymentFailed(c.paymentHash, failures = failures :+ LocalFailure(t)))
|
||||
reply(s, PaymentFailed(id, c.paymentHash, failures = failures :+ LocalFailure(t)))
|
||||
paymentsDb.updateOutgoingPayment(id, OutgoingPaymentStatus.FAILED)
|
||||
stop(FSM.Normal)
|
||||
}
|
||||
|
||||
|
@ -65,8 +72,9 @@ class PaymentLifecycle(sourceNodeId: PublicKey, router: ActorRef, register: Acto
|
|||
case Event("ok", _) => stay()
|
||||
|
||||
case Event(fulfill: UpdateFulfillHtlc, WaitingForComplete(s, c, cmd, _, _, _, _, hops)) =>
|
||||
reply(s, PaymentSucceeded(cmd.amountMsat, c.paymentHash, fulfill.paymentPreimage, hops))
|
||||
context.system.eventStream.publish(PaymentSent(MilliSatoshi(c.amountMsat), MilliSatoshi(cmd.amountMsat - c.amountMsat), cmd.paymentHash, fulfill.paymentPreimage, fulfill.channelId))
|
||||
paymentsDb.updateOutgoingPayment(id, OutgoingPaymentStatus.SUCCEEDED, preimage = Some(fulfill.paymentPreimage))
|
||||
reply(s, PaymentSucceeded(id, cmd.amountMsat, c.paymentHash, fulfill.paymentPreimage, hops))
|
||||
context.system.eventStream.publish(PaymentSent(id, MilliSatoshi(c.amountMsat), MilliSatoshi(cmd.amountMsat - c.amountMsat), cmd.paymentHash, fulfill.paymentPreimage, fulfill.channelId))
|
||||
stop(FSM.Normal)
|
||||
|
||||
case Event(fail: UpdateFailHtlc, WaitingForComplete(s, c, _, failures, sharedSecrets, ignoreNodes, ignoreChannels, hops)) =>
|
||||
|
@ -74,7 +82,8 @@ class PaymentLifecycle(sourceNodeId: PublicKey, router: ActorRef, register: Acto
|
|||
case Success(e@ErrorPacket(nodeId, failureMessage)) if nodeId == c.targetNodeId =>
|
||||
// if destination node returns an error, we fail the payment immediately
|
||||
log.warning(s"received an error message from target nodeId=$nodeId, failing the payment (failure=$failureMessage)")
|
||||
reply(s, PaymentFailed(c.paymentHash, failures = failures :+ RemoteFailure(hops, e)))
|
||||
reply(s, PaymentFailed(id, c.paymentHash, failures = failures :+ RemoteFailure(hops, e)))
|
||||
paymentsDb.updateOutgoingPayment(id, OutgoingPaymentStatus.FAILED)
|
||||
stop(FSM.Normal)
|
||||
case res if failures.size + 1 >= c.maxAttempts =>
|
||||
// otherwise we never try more than maxAttempts, no matter the kind of error returned
|
||||
|
@ -87,19 +96,20 @@ class PaymentLifecycle(sourceNodeId: PublicKey, router: ActorRef, register: Acto
|
|||
UnreadableRemoteFailure(hops)
|
||||
}
|
||||
log.warning(s"too many failed attempts, failing the payment")
|
||||
reply(s, PaymentFailed(c.paymentHash, failures = failures :+ failure))
|
||||
reply(s, PaymentFailed(id, c.paymentHash, failures = failures :+ failure))
|
||||
paymentsDb.updateOutgoingPayment(id, OutgoingPaymentStatus.FAILED)
|
||||
stop(FSM.Normal)
|
||||
case Failure(t) =>
|
||||
log.warning(s"cannot parse returned error: ${t.getMessage}")
|
||||
// in that case we don't know which node is sending garbage, let's try to blacklist all nodes except the one we are directly connected to and the destination node
|
||||
val blacklist = hops.map(_.nextNodeId).drop(1).dropRight(1)
|
||||
log.warning(s"blacklisting intermediate nodes=${blacklist.mkString(",")}")
|
||||
router ! RouteRequest(sourceNodeId, c.targetNodeId, c.amountMsat, c.assistedRoutes, ignoreNodes ++ blacklist, ignoreChannels, c.routeParams)
|
||||
router ! RouteRequest(nodeParams.nodeId, c.targetNodeId, c.amountMsat, c.assistedRoutes, ignoreNodes ++ blacklist, ignoreChannels, c.routeParams)
|
||||
goto(WAITING_FOR_ROUTE) using WaitingForRoute(s, c, failures :+ UnreadableRemoteFailure(hops))
|
||||
case Success(e@ErrorPacket(nodeId, failureMessage: Node)) =>
|
||||
log.info(s"received 'Node' type error message from nodeId=$nodeId, trying to route around it (failure=$failureMessage)")
|
||||
// let's try to route around this node
|
||||
router ! RouteRequest(sourceNodeId, c.targetNodeId, c.amountMsat, c.assistedRoutes, ignoreNodes + nodeId, ignoreChannels, c.routeParams)
|
||||
router ! RouteRequest(nodeParams.nodeId, c.targetNodeId, c.amountMsat, c.assistedRoutes, ignoreNodes + nodeId, ignoreChannels, c.routeParams)
|
||||
goto(WAITING_FOR_ROUTE) using WaitingForRoute(s, c, failures :+ RemoteFailure(hops, e))
|
||||
case Success(e@ErrorPacket(nodeId, failureMessage: Update)) =>
|
||||
log.info(s"received 'Update' type error message from nodeId=$nodeId, retrying payment (failure=$failureMessage)")
|
||||
|
@ -127,18 +137,18 @@ class PaymentLifecycle(sourceNodeId: PublicKey, router: ActorRef, register: Acto
|
|||
// in any case, we forward the update to the router
|
||||
router ! failureMessage.update
|
||||
// let's try again, router will have updated its state
|
||||
router ! RouteRequest(sourceNodeId, c.targetNodeId, c.amountMsat, c.assistedRoutes, ignoreNodes, ignoreChannels, c.routeParams)
|
||||
router ! RouteRequest(nodeParams.nodeId, c.targetNodeId, c.amountMsat, c.assistedRoutes, ignoreNodes, ignoreChannels, c.routeParams)
|
||||
} else {
|
||||
// this node is fishy, it gave us a bad sig!! let's filter it out
|
||||
log.warning(s"got bad signature from node=$nodeId update=${failureMessage.update}")
|
||||
router ! RouteRequest(sourceNodeId, c.targetNodeId, c.amountMsat, c.assistedRoutes, ignoreNodes + nodeId, ignoreChannels, c.routeParams)
|
||||
router ! RouteRequest(nodeParams.nodeId, c.targetNodeId, c.amountMsat, c.assistedRoutes, ignoreNodes + nodeId, ignoreChannels, c.routeParams)
|
||||
}
|
||||
goto(WAITING_FOR_ROUTE) using WaitingForRoute(s, c, failures :+ RemoteFailure(hops, e))
|
||||
case Success(e@ErrorPacket(nodeId, failureMessage)) =>
|
||||
log.info(s"received an error message from nodeId=$nodeId, trying to use a different channel (failure=$failureMessage)")
|
||||
// let's try again without the channel outgoing from nodeId
|
||||
val faultyChannel = hops.find(_.nodeId == nodeId).map(hop => ChannelDesc(hop.lastUpdate.shortChannelId, hop.nodeId, hop.nextNodeId))
|
||||
router ! RouteRequest(sourceNodeId, c.targetNodeId, c.amountMsat, c.assistedRoutes, ignoreNodes, ignoreChannels ++ faultyChannel.toSet, c.routeParams)
|
||||
router ! RouteRequest(nodeParams.nodeId, c.targetNodeId, c.amountMsat, c.assistedRoutes, ignoreNodes, ignoreChannels ++ faultyChannel.toSet, c.routeParams)
|
||||
goto(WAITING_FOR_ROUTE) using WaitingForRoute(s, c, failures :+ RemoteFailure(hops, e))
|
||||
}
|
||||
|
||||
|
@ -152,12 +162,13 @@ class PaymentLifecycle(sourceNodeId: PublicKey, router: ActorRef, register: Acto
|
|||
|
||||
case Event(Status.Failure(t), WaitingForComplete(s, c, _, failures, _, ignoreNodes, ignoreChannels, hops)) =>
|
||||
if (failures.size + 1 >= c.maxAttempts) {
|
||||
reply(s, PaymentFailed(c.paymentHash, failures :+ LocalFailure(t)))
|
||||
paymentsDb.updateOutgoingPayment(id, OutgoingPaymentStatus.FAILED)
|
||||
reply(s, PaymentFailed(id, c.paymentHash, failures :+ LocalFailure(t)))
|
||||
stop(FSM.Normal)
|
||||
} else {
|
||||
log.info(s"received an error message from local, trying to use a different channel (failure=${t.getMessage})")
|
||||
val faultyChannel = ChannelDesc(hops.head.lastUpdate.shortChannelId, hops.head.nodeId, hops.head.nextNodeId)
|
||||
router ! RouteRequest(sourceNodeId, c.targetNodeId, c.amountMsat, c.assistedRoutes, ignoreNodes, ignoreChannels + faultyChannel, c.routeParams)
|
||||
router ! RouteRequest(nodeParams.nodeId, c.targetNodeId, c.amountMsat, c.assistedRoutes, ignoreNodes, ignoreChannels + faultyChannel, c.routeParams)
|
||||
goto(WAITING_FOR_ROUTE) using WaitingForRoute(s, c, failures :+ LocalFailure(t))
|
||||
}
|
||||
|
||||
|
@ -177,7 +188,7 @@ class PaymentLifecycle(sourceNodeId: PublicKey, router: ActorRef, register: Acto
|
|||
|
||||
object PaymentLifecycle {
|
||||
|
||||
def props(sourceNodeId: PublicKey, router: ActorRef, register: ActorRef) = Props(classOf[PaymentLifecycle], sourceNodeId, router, register)
|
||||
def props(nodeParams: NodeParams, id: UUID, router: ActorRef, register: ActorRef) = Props(classOf[PaymentLifecycle], nodeParams, id, router, register)
|
||||
|
||||
// @formatter:off
|
||||
case class ReceivePayment(amountMsat_opt: Option[MilliSatoshi], description: String, expirySeconds_opt: Option[Long] = None, extraHops: List[List[ExtraHop]] = Nil, fallbackAddress: Option[String] = None)
|
||||
|
@ -190,15 +201,14 @@ object PaymentLifecycle {
|
|||
routeParams: Option[RouteParams] = None) {
|
||||
require(amountMsat > 0, s"amountMsat must be > 0")
|
||||
}
|
||||
case class CheckPayment(paymentHash: ByteVector32)
|
||||
|
||||
sealed trait PaymentResult
|
||||
case class PaymentSucceeded(amountMsat: Long, paymentHash: ByteVector32, paymentPreimage: ByteVector32, route: Seq[Hop]) extends PaymentResult // note: the amount includes fees
|
||||
case class PaymentSucceeded(id: UUID, amountMsat: Long, paymentHash: ByteVector32, paymentPreimage: ByteVector32, route: Seq[Hop]) extends PaymentResult // note: the amount includes fees
|
||||
sealed trait PaymentFailure
|
||||
case class LocalFailure(t: Throwable) extends PaymentFailure
|
||||
case class RemoteFailure(route: Seq[Hop], e: ErrorPacket) extends PaymentFailure
|
||||
case class UnreadableRemoteFailure(route: Seq[Hop]) extends PaymentFailure
|
||||
case class PaymentFailed(paymentHash: ByteVector32, failures: Seq[PaymentFailure]) extends PaymentResult
|
||||
case class PaymentFailed(id: UUID, paymentHash: ByteVector32, failures: Seq[PaymentFailure]) extends PaymentResult
|
||||
|
||||
sealed trait Data
|
||||
case object WaitingForRequest extends Data
|
||||
|
@ -242,12 +252,12 @@ object PaymentLifecycle {
|
|||
(msat + nextFee, expiry + hop.lastUpdate.cltvExpiryDelta, PerHopPayload(hop.lastUpdate.shortChannelId, msat, expiry) +: payloads)
|
||||
}
|
||||
|
||||
def buildCommand(finalAmountMsat: Long, finalExpiry: Long, paymentHash: ByteVector32, hops: Seq[Hop]): (CMD_ADD_HTLC, Seq[(ByteVector32, PublicKey)]) = {
|
||||
def buildCommand(id: UUID, finalAmountMsat: Long, finalExpiry: Long, paymentHash: ByteVector32, hops: Seq[Hop]): (CMD_ADD_HTLC, Seq[(ByteVector32, PublicKey)]) = {
|
||||
val (firstAmountMsat, firstExpiry, payloads) = buildPayloads(finalAmountMsat, finalExpiry, hops.drop(1))
|
||||
val nodes = hops.map(_.nextNodeId)
|
||||
// BOLT 2 requires that associatedData == paymentHash
|
||||
val onion = buildOnion(nodes, payloads, paymentHash)
|
||||
CMD_ADD_HTLC(firstAmountMsat, paymentHash, firstExpiry, Packet.write(onion.packet), upstream_opt = None, commit = true) -> onion.sharedSecrets
|
||||
CMD_ADD_HTLC(firstAmountMsat, paymentHash, firstExpiry, Packet.write(onion.packet), upstream = Left(id), commit = true) -> onion.sharedSecrets
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -238,7 +238,8 @@ object PaymentRequest {
|
|||
}
|
||||
|
||||
/**
|
||||
* This returns a bitvector with the minimum size necessary to encode the long
|
||||
* This returns a bitvector with the minimum size necessary to encode the long, left padded
|
||||
* to have a length (in bits) multiples of 5
|
||||
* @param l
|
||||
*/
|
||||
def long2bits(l: Long) = {
|
||||
|
@ -247,7 +248,11 @@ object PaymentRequest {
|
|||
for (i <- 0 until bin.size.toInt) {
|
||||
if (highest == -1 && bin(i)) highest = i
|
||||
}
|
||||
if (highest == -1) BitVector.empty else bin.drop(highest)
|
||||
val nonPadded = if (highest == -1) BitVector.empty else bin.drop(highest)
|
||||
nonPadded.size % 5 match {
|
||||
case 0 => nonPadded
|
||||
case remaining => BitVector.fill(5 - remaining)(false) ++ nonPadded
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -16,12 +16,15 @@
|
|||
|
||||
package fr.acinq.eclair.payment
|
||||
|
||||
import java.util.UUID
|
||||
|
||||
import akka.actor.{Actor, ActorLogging, ActorRef, Props, Status}
|
||||
import akka.event.LoggingAdapter
|
||||
import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey}
|
||||
import fr.acinq.bitcoin.{ByteVector32, Crypto, MilliSatoshi}
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.crypto.Sphinx
|
||||
import fr.acinq.eclair.db.OutgoingPaymentStatus
|
||||
import fr.acinq.eclair.payment.PaymentLifecycle.{PaymentFailed, PaymentSucceeded}
|
||||
import fr.acinq.eclair.router.Announcements
|
||||
import fr.acinq.eclair.wire._
|
||||
|
@ -35,7 +38,7 @@ import scala.util.{Failure, Success, Try}
|
|||
// @formatter:off
|
||||
|
||||
sealed trait Origin
|
||||
case class Local(sender: Option[ActorRef]) extends Origin // we don't persist reference to local actors
|
||||
case class Local(id: UUID, sender: Option[ActorRef]) extends Origin // we don't persist reference to local actors
|
||||
case class Relayed(originChannelId: ByteVector32, originHtlcId: Long, amountMsatIn: Long, amountMsatOut: Long) extends Origin
|
||||
|
||||
sealed trait ForwardMessage
|
||||
|
@ -111,24 +114,26 @@ class Relayer(nodeParams: NodeParams, register: ActorRef, paymentHandler: ActorR
|
|||
commandBuffer ! CommandBuffer.CommandSend(add.channelId, add.id, cmdFail)
|
||||
}
|
||||
|
||||
case Status.Failure(Register.ForwardShortIdFailure(Register.ForwardShortId(shortChannelId, CMD_ADD_HTLC(_, _, _, _, Some(add), _, _)))) =>
|
||||
case Status.Failure(Register.ForwardShortIdFailure(Register.ForwardShortId(shortChannelId, CMD_ADD_HTLC(_, _, _, _, Right(add), _, _)))) =>
|
||||
log.warning(s"couldn't resolve downstream channel $shortChannelId, failing htlc #${add.id}")
|
||||
val cmdFail = CMD_FAIL_HTLC(add.id, Right(UnknownNextPeer), commit = true)
|
||||
commandBuffer ! CommandBuffer.CommandSend(add.channelId, add.id, cmdFail)
|
||||
|
||||
case Status.Failure(AddHtlcFailed(_, paymentHash, _, Local(None), _, _)) =>
|
||||
// we sent the payment, but we probably restarted and the reference to the original sender was lost, we just publish the failure on the event stream
|
||||
context.system.eventStream.publish(PaymentFailed(paymentHash, Nil))
|
||||
case Status.Failure(AddHtlcFailed(_, paymentHash, _, Local(id, None), _, _)) =>
|
||||
// we sent the payment, but we probably restarted and the reference to the original sender was lost,
|
||||
// we publish the failure on the event stream and update the status in paymentDb
|
||||
nodeParams.db.payments.updateOutgoingPayment(id, OutgoingPaymentStatus.FAILED)
|
||||
context.system.eventStream.publish(PaymentFailed(id, paymentHash, Nil))
|
||||
|
||||
case Status.Failure(AddHtlcFailed(_, _, error, Local(Some(sender)), _, _)) =>
|
||||
case Status.Failure(AddHtlcFailed(_, _, error, Local(_, Some(sender)), _, _)) =>
|
||||
sender ! Status.Failure(error)
|
||||
|
||||
case Status.Failure(AddHtlcFailed(_, paymentHash, error, Relayed(originChannelId, originHtlcId, _, _), channelUpdate_opt, originalCommand_opt)) =>
|
||||
originalCommand_opt match {
|
||||
case Some(cmd) if cmd.redirected && cmd.upstream_opt.isDefined => // cmd.upstream_opt.isDefined always true since origin = relayed
|
||||
case Some(cmd) if cmd.redirected && cmd.upstream.isRight => // cmd.upstream_opt.isDefined always true since origin = relayed
|
||||
// if it was redirected, we give it one more try with the original requested channel (meaning that the error returned will always be for the requested channel)
|
||||
log.info(s"retrying htlc #$originHtlcId paymentHash=$paymentHash from channelId=$originChannelId")
|
||||
self ! ForwardAdd(cmd.upstream_opt.get, canRedirect = false)
|
||||
self ! ForwardAdd(cmd.upstream.right.get, canRedirect = false)
|
||||
case _ =>
|
||||
// otherwise we just return a failure
|
||||
val failure = (error, channelUpdate_opt) match {
|
||||
|
@ -146,13 +151,15 @@ class Relayer(nodeParams: NodeParams, register: ActorRef, paymentHandler: ActorR
|
|||
commandBuffer ! CommandBuffer.CommandSend(originChannelId, originHtlcId, cmdFail)
|
||||
}
|
||||
|
||||
case ForwardFulfill(fulfill, Local(None), add) =>
|
||||
case ForwardFulfill(fulfill, Local(id, None), add) =>
|
||||
val feesPaid = MilliSatoshi(0)
|
||||
context.system.eventStream.publish(PaymentSent(MilliSatoshi(add.amountMsat), feesPaid, add.paymentHash, fulfill.paymentPreimage, fulfill.channelId))
|
||||
// we sent the payment, but we probably restarted and the reference to the original sender was lost, we just publish the success on the event stream
|
||||
context.system.eventStream.publish(PaymentSucceeded(add.amountMsat, add.paymentHash, fulfill.paymentPreimage, Nil)) //
|
||||
context.system.eventStream.publish(PaymentSent(id, MilliSatoshi(add.amountMsat), feesPaid, add.paymentHash, fulfill.paymentPreimage, fulfill.channelId))
|
||||
// we sent the payment, but we probably restarted and the reference to the original sender was lost,
|
||||
// we publish the failure on the event stream and update the status in paymentDb
|
||||
nodeParams.db.payments.updateOutgoingPayment(id, OutgoingPaymentStatus.SUCCEEDED, Some(fulfill.paymentPreimage))
|
||||
context.system.eventStream.publish(PaymentSucceeded(id, add.amountMsat, add.paymentHash, fulfill.paymentPreimage, Nil)) //
|
||||
|
||||
case ForwardFulfill(fulfill, Local(Some(sender)), _) =>
|
||||
case ForwardFulfill(fulfill, Local(_, Some(sender)), _) =>
|
||||
sender ! fulfill
|
||||
|
||||
case ForwardFulfill(fulfill, Relayed(originChannelId, originHtlcId, amountMsatIn, amountMsatOut), add) =>
|
||||
|
@ -160,22 +167,26 @@ class Relayer(nodeParams: NodeParams, register: ActorRef, paymentHandler: ActorR
|
|||
commandBuffer ! CommandBuffer.CommandSend(originChannelId, originHtlcId, cmd)
|
||||
context.system.eventStream.publish(PaymentRelayed(MilliSatoshi(amountMsatIn), MilliSatoshi(amountMsatOut), add.paymentHash, fromChannelId = originChannelId, toChannelId = fulfill.channelId))
|
||||
|
||||
case ForwardFail(_, Local(None), add) =>
|
||||
// we sent the payment, but we probably restarted and the reference to the original sender was lost, we just publish the failure on the event stream
|
||||
context.system.eventStream.publish(PaymentFailed(add.paymentHash, Nil))
|
||||
case ForwardFail(_, Local(id, None), add) =>
|
||||
// we sent the payment, but we probably restarted and the reference to the original sender was lost
|
||||
// we publish the failure on the event stream and update the status in paymentDb
|
||||
nodeParams.db.payments.updateOutgoingPayment(id, OutgoingPaymentStatus.FAILED)
|
||||
context.system.eventStream.publish(PaymentFailed(id, add.paymentHash, Nil))
|
||||
|
||||
case ForwardFail(fail, Local(Some(sender)), _) =>
|
||||
case ForwardFail(fail, Local(_, Some(sender)), _) =>
|
||||
sender ! fail
|
||||
|
||||
case ForwardFail(fail, Relayed(originChannelId, originHtlcId, _, _), _) =>
|
||||
val cmd = CMD_FAIL_HTLC(originHtlcId, Left(fail.reason), commit = true)
|
||||
commandBuffer ! CommandBuffer.CommandSend(originChannelId, originHtlcId, cmd)
|
||||
|
||||
case ForwardFailMalformed(_, Local(None), add) =>
|
||||
// we sent the payment, but we probably restarted and the reference to the original sender was lost, we just publish the failure on the event stream
|
||||
context.system.eventStream.publish(PaymentFailed(add.paymentHash, Nil))
|
||||
case ForwardFailMalformed(_, Local(id, None), add) =>
|
||||
// we sent the payment, but we probably restarted and the reference to the original sender was lost
|
||||
// we publish the failure on the event stream and update the status in paymentDb
|
||||
nodeParams.db.payments.updateOutgoingPayment(id, OutgoingPaymentStatus.FAILED)
|
||||
context.system.eventStream.publish(PaymentFailed(id, add.paymentHash, Nil))
|
||||
|
||||
case ForwardFailMalformed(fail, Local(Some(sender)), _) =>
|
||||
case ForwardFailMalformed(fail, Local(_, Some(sender)), _) =>
|
||||
sender ! fail
|
||||
|
||||
case ForwardFailMalformed(fail, Relayed(originChannelId, originHtlcId, _, _), _) =>
|
||||
|
@ -269,7 +280,7 @@ object Relayer {
|
|||
Left(CMD_FAIL_HTLC(add.id, Right(FeeInsufficient(add.amountMsat, channelUpdate)), commit = true))
|
||||
case Some(channelUpdate) =>
|
||||
val isRedirected = (channelUpdate.shortChannelId != payload.shortChannelId) // we may decide to use another channel (to the same node) from the one requested
|
||||
Right(CMD_ADD_HTLC(payload.amtToForward, add.paymentHash, payload.outgoingCltvValue, nextPacket.serialize, upstream_opt = Some(add), commit = true, redirected = isRedirected))
|
||||
Right(CMD_ADD_HTLC(payload.amtToForward, add.paymentHash, payload.outgoingCltvValue, nextPacket.serialize, upstream = Right(add), commit = true, redirected = isRedirected))
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -16,6 +16,9 @@
|
|||
|
||||
package fr.acinq.eclair.wire
|
||||
|
||||
import java.util.UUID
|
||||
|
||||
import akka.actor.ActorRef
|
||||
import fr.acinq.bitcoin.DeterministicWallet.{ExtendedPrivateKey, KeyPath}
|
||||
import fr.acinq.bitcoin.{ByteVector32, OutPoint, Transaction, TxOut}
|
||||
import fr.acinq.eclair.channel._
|
||||
|
@ -154,14 +157,23 @@ object ChannelCodecs extends Logging {
|
|||
("sentAfterLocalCommitIndex" | uint64) ::
|
||||
("reSignAsap" | bool)).as[WaitingForRevocation]
|
||||
|
||||
val localCodec: Codec[Local] = (
|
||||
("id" | uuid) ::
|
||||
("sender" | provide(Option.empty[ActorRef]))
|
||||
).as[Local]
|
||||
|
||||
val relayedCodec: Codec[Relayed] = (
|
||||
("originChannelId" | bytes32) ::
|
||||
("originHtlcId" | int64) ::
|
||||
("amountMsatIn" | uint64) ::
|
||||
("amountMsatOut" | uint64)).as[Relayed]
|
||||
|
||||
// this is for backward compatibility to handle legacy payments that didn't have identifiers
|
||||
val UNKNOWN_UUID = UUID.fromString("00000000-0000-0000-0000-000000000000")
|
||||
|
||||
val originCodec: Codec[Origin] = discriminated[Origin].by(uint16)
|
||||
.typecase(0x01, provide(Local(None)))
|
||||
.typecase(0x03, localCodec) // backward compatible
|
||||
.typecase(0x01, provide(Local(UNKNOWN_UUID, None)))
|
||||
.typecase(0x02, relayedCodec)
|
||||
|
||||
val originsListCodec: Codec[List[(Long, Origin)]] = listOfN(uint16, int64 ~ originCodec)
|
||||
|
|
|
@ -81,7 +81,6 @@ object TestConstants {
|
|||
channelFlags = 1,
|
||||
watcherType = BITCOIND,
|
||||
paymentRequestExpiry = 1 hour,
|
||||
maxPendingPaymentRequests = 10000000,
|
||||
minFundingSatoshis = 1000L,
|
||||
routerConf = RouterConf(
|
||||
randomizeRouteSelection = false,
|
||||
|
@ -146,7 +145,6 @@ object TestConstants {
|
|||
channelFlags = 1,
|
||||
watcherType = BITCOIND,
|
||||
paymentRequestExpiry = 1 hour,
|
||||
maxPendingPaymentRequests = 10000000,
|
||||
minFundingSatoshis = 1000L,
|
||||
routerConf = RouterConf(
|
||||
randomizeRouteSelection = false,
|
||||
|
|
|
@ -16,6 +16,8 @@
|
|||
|
||||
package fr.acinq.eclair.api
|
||||
|
||||
import java.util.UUID
|
||||
|
||||
import akka.actor.{Actor, ActorSystem, Props, Scheduler}
|
||||
import org.scalatest.FunSuite
|
||||
import akka.http.scaladsl.model.StatusCodes._
|
||||
|
@ -29,13 +31,14 @@ import akka.stream.ActorMaterializer
|
|||
import akka.http.scaladsl.model.{ContentTypes, FormData, MediaTypes, Multipart}
|
||||
import fr.acinq.bitcoin.{ByteVector32, Crypto, MilliSatoshi}
|
||||
import fr.acinq.eclair.channel.RES_GETINFO
|
||||
import fr.acinq.eclair.db.{NetworkFee, Stats}
|
||||
import fr.acinq.eclair.payment.PaymentLifecycle.PaymentFailed
|
||||
import fr.acinq.eclair.db.{NetworkFee, IncomingPayment, OutgoingPayment, Stats}
|
||||
import fr.acinq.eclair.payment.PaymentLifecycle.{PaymentFailed, ReceivePayment}
|
||||
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 scodec.bits.ByteVector
|
||||
|
||||
import scala.concurrent.Future
|
||||
import scala.concurrent.duration._
|
||||
import scala.io.Source
|
||||
|
@ -61,19 +64,19 @@ class ApiServiceSpec extends FunSuite with ScalatestRouteTest {
|
|||
|
||||
override def channelInfo(channelId: ByteVector32): Future[RES_GETINFO] = ???
|
||||
|
||||
override def allnodes(): Future[Iterable[NodeAnnouncement]] = ???
|
||||
override def allNodes(): Future[Iterable[NodeAnnouncement]] = ???
|
||||
|
||||
override def allchannels(): Future[Iterable[ChannelDesc]] = ???
|
||||
override def allChannels(): Future[Iterable[ChannelDesc]] = ???
|
||||
|
||||
override def allupdates(nodeId: Option[Crypto.PublicKey]): Future[Iterable[ChannelUpdate]] = ???
|
||||
override def allUpdates(nodeId: Option[Crypto.PublicKey]): Future[Iterable[ChannelUpdate]] = ???
|
||||
|
||||
override def receive(description: String, amountMsat: Option[Long], expire: Option[Long]): Future[String] = ???
|
||||
override def receive(description: String, amountMsat: Option[Long], expire: Option[Long], fallbackAddress: Option[String]): Future[PaymentRequest] = ???
|
||||
|
||||
override def findRoute(targetNodeId: Crypto.PublicKey, amountMsat: Long, assistedRoutes: Seq[Seq[PaymentRequest.ExtraHop]]): 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[PaymentLifecycle.PaymentResult] = ???
|
||||
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 checkpayment(paymentHash: ByteVector32): Future[Boolean] = ???
|
||||
override def receivedInfo(paymentHash: ByteVector32): Future[Option[IncomingPayment]] = ???
|
||||
|
||||
override def audit(from_opt: Option[Long], to_opt: Option[Long]): Future[AuditResponse] = ???
|
||||
|
||||
|
@ -82,6 +85,15 @@ class ApiServiceSpec extends FunSuite with ScalatestRouteTest {
|
|||
override def channelStats(): Future[Seq[Stats]] = ???
|
||||
|
||||
override def getInfoResponse(): Future[GetInfoResponse] = ???
|
||||
|
||||
override def sentInfo(id: Either[UUID, ByteVector32]): Future[Seq[OutgoingPayment]] = ???
|
||||
|
||||
override def allInvoices(from_opt: Option[Long], to_opt: Option[Long]): Future[Seq[PaymentRequest]] = ???
|
||||
|
||||
override def getInvoice(paymentHash: ByteVector32): Future[Option[PaymentRequest]] = ???
|
||||
|
||||
override def pendingInvoices(from_opt: Option[Long], to_opt: Option[Long]): Future[Seq[PaymentRequest]] = ???
|
||||
|
||||
}
|
||||
|
||||
implicit val formats = JsonSupport.formats
|
||||
|
@ -137,7 +149,6 @@ class ApiServiceSpec extends FunSuite with ScalatestRouteTest {
|
|||
assert(handled)
|
||||
assert(status == BadRequest)
|
||||
val resp = entityAs[ErrorResponse](JsonSupport.unmarshaller, ClassTag(classOf[ErrorResponse]))
|
||||
println(resp.error)
|
||||
assert(resp.error == "The form field 'channelId' was malformed:\nInvalid hexadecimal character 'h' at index 0")
|
||||
}
|
||||
|
||||
|
@ -251,14 +262,53 @@ class ApiServiceSpec extends FunSuite with ScalatestRouteTest {
|
|||
check {
|
||||
assert(handled)
|
||||
assert(status == OK)
|
||||
println(entityAs[String])
|
||||
assert(entityAs[String] == "\"connected\"")
|
||||
}
|
||||
}
|
||||
|
||||
test("'send' method should return the UUID of the outgoing payment") {
|
||||
|
||||
val id = UUID.randomUUID()
|
||||
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(
|
||||
id
|
||||
)
|
||||
})
|
||||
|
||||
Post("/payinvoice", FormData("invoice" -> invoice).toEntity) ~>
|
||||
addCredentials(BasicHttpCredentials("", mockService.password)) ~>
|
||||
Route.seal(mockService.route) ~>
|
||||
check {
|
||||
assert(handled)
|
||||
assert(status == OK)
|
||||
assert(entityAs[String] == "\""+id.toString+"\"")
|
||||
}
|
||||
}
|
||||
|
||||
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
|
||||
})
|
||||
|
||||
Post("/getreceivedinfo", FormData("paymentHash" -> ByteVector32.Zeroes.toHex).toEntity) ~>
|
||||
addCredentials(BasicHttpCredentials("", mockService.password)) ~>
|
||||
Route.seal(mockService.route) ~>
|
||||
check {
|
||||
assert(handled)
|
||||
assert(status == NotFound)
|
||||
val resp = entityAs[ErrorResponse](JsonSupport.unmarshaller, ClassTag(classOf[ErrorResponse]))
|
||||
assert(resp == ErrorResponse("Not found"))
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
test("the websocket should return typed objects") {
|
||||
|
||||
val mockService = new MockService(new EclairMock {})
|
||||
val fixedUUID = UUID.fromString("487da196-a4dc-4b1e-92b4-3e5e905e9f3f")
|
||||
|
||||
val wsClient = WSProbe()
|
||||
|
||||
|
@ -267,14 +317,14 @@ class ApiServiceSpec extends FunSuite with ScalatestRouteTest {
|
|||
mockService.route ~>
|
||||
check {
|
||||
|
||||
val pf = PaymentFailed(ByteVector32.Zeroes, failures = Seq.empty)
|
||||
val expectedSerializedPf = """{"type":"payment-failed","paymentHash":"0000000000000000000000000000000000000000000000000000000000000000","failures":[]}"""
|
||||
val pf = PaymentFailed(fixedUUID, ByteVector32.Zeroes, failures = Seq.empty)
|
||||
val expectedSerializedPf = """{"type":"payment-failed","id":"487da196-a4dc-4b1e-92b4-3e5e905e9f3f","paymentHash":"0000000000000000000000000000000000000000000000000000000000000000","failures":[]}"""
|
||||
Serialization.write(pf)(mockService.formatsWithTypeHint) === expectedSerializedPf
|
||||
system.eventStream.publish(pf)
|
||||
wsClient.expectMessage(expectedSerializedPf)
|
||||
|
||||
val ps = PaymentSent(amount = MilliSatoshi(21), feesPaid = MilliSatoshi(1), paymentHash = ByteVector32.Zeroes, paymentPreimage = ByteVector32.One, toChannelId = ByteVector32.Zeroes, timestamp = 1553784337711L)
|
||||
val expectedSerializedPs = """{"type":"payment-sent","amount":21,"feesPaid":1,"paymentHash":"0000000000000000000000000000000000000000000000000000000000000000","paymentPreimage":"0100000000000000000000000000000000000000000000000000000000000000","toChannelId":"0000000000000000000000000000000000000000000000000000000000000000","timestamp":1553784337711}"""
|
||||
val ps = PaymentSent(fixedUUID, amount = MilliSatoshi(21), feesPaid = MilliSatoshi(1), paymentHash = ByteVector32.Zeroes, paymentPreimage = ByteVector32.One, toChannelId = ByteVector32.Zeroes, timestamp = 1553784337711L)
|
||||
val expectedSerializedPs = """{"type":"payment-sent","id":"487da196-a4dc-4b1e-92b4-3e5e905e9f3f","amount":21,"feesPaid":1,"paymentHash":"0000000000000000000000000000000000000000000000000000000000000000","paymentPreimage":"0100000000000000000000000000000000000000000000000000000000000000","toChannelId":"0000000000000000000000000000000000000000000000000000000000000000","timestamp":1553784337711}"""
|
||||
Serialization.write(ps)(mockService.formatsWithTypeHint) === expectedSerializedPs
|
||||
system.eventStream.publish(ps)
|
||||
wsClient.expectMessage(expectedSerializedPs)
|
||||
|
@ -291,14 +341,13 @@ class ApiServiceSpec extends FunSuite with ScalatestRouteTest {
|
|||
system.eventStream.publish(precv)
|
||||
wsClient.expectMessage(expectedSerializedPrecv)
|
||||
|
||||
val pset = PaymentSettlingOnChain(amount = MilliSatoshi(21), paymentHash = ByteVector32.One, timestamp = 1553785442676L)
|
||||
val expectedSerializedPset = """{"type":"payment-settling-onchain","amount":21,"paymentHash":"0100000000000000000000000000000000000000000000000000000000000000","timestamp":1553785442676}"""
|
||||
val pset = PaymentSettlingOnChain(fixedUUID, amount = MilliSatoshi(21), paymentHash = ByteVector32.One, timestamp = 1553785442676L)
|
||||
val expectedSerializedPset = """{"type":"payment-settling-onchain","id":"487da196-a4dc-4b1e-92b4-3e5e905e9f3f","amount":21,"paymentHash":"0100000000000000000000000000000000000000000000000000000000000000","timestamp":1553785442676}"""
|
||||
Serialization.write(pset)(mockService.formatsWithTypeHint) === expectedSerializedPset
|
||||
system.eventStream.publish(pset)
|
||||
wsClient.expectMessage(expectedSerializedPset)
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
||||
private def matchTestJson(apiName: String, response: String) = {
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
package fr.acinq.eclair.api
|
||||
|
||||
import java.net.InetAddress
|
||||
import java.util.UUID
|
||||
|
||||
import fr.acinq.bitcoin.{MilliSatoshi, OutPoint}
|
||||
import fr.acinq.eclair._
|
||||
|
@ -27,7 +28,6 @@ import fr.acinq.eclair.payment.PaymentRequest
|
|||
import fr.acinq.eclair.transactions.{IN, OUT}
|
||||
import fr.acinq.eclair.wire.{NodeAddress, Tor2, Tor3}
|
||||
import org.json4s.jackson.Serialization
|
||||
import org.json4s.{DefaultFormats, ShortTypeHints}
|
||||
import org.scalatest.{FunSuite, Matchers}
|
||||
import scodec.bits._
|
||||
|
||||
|
@ -74,12 +74,12 @@ class JsonSerializersSpec extends FunSuite with Matchers {
|
|||
test("Payment Request") {
|
||||
val ref = "lnbc2500u1pvjluezpp5qqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqypqdq5xysxxatsyp3k7enxv4jsxqzpuaztrnwngzn3kdzw5hydlzf03qdgm2hdq27cqv3agm2awhz5se903vruatfhq77w3ls4evs3ch9zw97j25emudupq63nyw24cg27h2rspfj9srp"
|
||||
val pr = PaymentRequest.read(ref)
|
||||
Serialization.write(pr)(org.json4s.DefaultFormats + new PaymentRequestSerializer) shouldBe """{"prefix":"lnbc","amount":250000000,"timestamp":1496314658,"nodeId":"03e7156ae33b0a208d0744199163177e909e80176e55d97a2f221ede0f934dd9ad","description":"1 cup coffee","paymentHash":"0001020304050607080900010203040506070809000102030405060708090102","expiry":60,"minFinalCltvExpiry":null}"""
|
||||
JsonSupport.serialization.write(pr)(JsonSupport.formats) shouldBe """{"prefix":"lnbc","timestamp":1496314658,"nodeId":"03e7156ae33b0a208d0744199163177e909e80176e55d97a2f221ede0f934dd9ad","serialized":"lnbc2500u1pvjluezpp5qqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqypqdq5xysxxatsyp3k7enxv4jsxqzpuaztrnwngzn3kdzw5hydlzf03qdgm2hdq27cqv3agm2awhz5se903vruatfhq77w3ls4evs3ch9zw97j25emudupq63nyw24cg27h2rspfj9srp","description":"1 cup coffee","paymentHash":"0001020304050607080900010203040506070809000102030405060708090102","expiry":60,"amount":250000000}"""
|
||||
}
|
||||
|
||||
test("type hints") {
|
||||
implicit val formats = DefaultFormats.withTypeHintFieldName("type") + CustomTypeHints(Map(classOf[PaymentSettlingOnChain] -> "payment-settling-onchain")) + new MilliSatoshiSerializer
|
||||
val e1 = PaymentSettlingOnChain(MilliSatoshi(42), randomBytes32)
|
||||
implicit val formats = JsonSupport.formats.withTypeHintFieldName("type") + CustomTypeHints(Map(classOf[PaymentSettlingOnChain] -> "payment-settling-onchain")) + new MilliSatoshiSerializer
|
||||
val e1 = PaymentSettlingOnChain(UUID.randomUUID, MilliSatoshi(42), randomBytes32)
|
||||
assert(Serialization.writePretty(e1).contains("\"type\" : \"payment-settling-onchain\""))
|
||||
}
|
||||
}
|
||||
|
|
|
@ -16,9 +16,8 @@
|
|||
|
||||
package fr.acinq.eclair.blockchain.electrum.db.sqlite
|
||||
|
||||
import java.sql.DriverManager
|
||||
|
||||
import fr.acinq.bitcoin.{Block, BlockHeader, OutPoint, Satoshi, Transaction, TxIn, TxOut}
|
||||
import fr.acinq.eclair.TestConstants
|
||||
import fr.acinq.eclair.blockchain.electrum.ElectrumClient
|
||||
import fr.acinq.eclair.blockchain.electrum.ElectrumClient.GetMerkleResponse
|
||||
import fr.acinq.eclair.blockchain.electrum.ElectrumWallet.PersistentData
|
||||
|
@ -29,8 +28,6 @@ import scala.util.Random
|
|||
class SqliteWalletDbSpec extends FunSuite {
|
||||
val random = new Random()
|
||||
|
||||
def inmem = DriverManager.getConnection("jdbc:sqlite::memory:")
|
||||
|
||||
def makeChildHeader(header: BlockHeader): BlockHeader = header.copy(hashPreviousBlock = header.hash, nonce = random.nextLong() & 0xffffffffL)
|
||||
|
||||
def makeHeaders(n: Int, acc: Seq[BlockHeader] = Seq(Block.RegtestGenesisBlock.header)): Seq[BlockHeader] = {
|
||||
|
@ -38,7 +35,7 @@ class SqliteWalletDbSpec extends FunSuite {
|
|||
}
|
||||
|
||||
test("add/get/list headers") {
|
||||
val db = new SqliteWalletDb(inmem)
|
||||
val db = new SqliteWalletDb(TestConstants.sqliteInMemory())
|
||||
val headers = makeHeaders(100)
|
||||
db.addHeaders(2016, headers)
|
||||
|
||||
|
@ -61,7 +58,7 @@ class SqliteWalletDbSpec extends FunSuite {
|
|||
}
|
||||
|
||||
test("serialize persistent data") {
|
||||
val db = new SqliteWalletDb(inmem)
|
||||
val db = new SqliteWalletDb(TestConstants.sqliteInMemory())
|
||||
|
||||
import fr.acinq.eclair.{randomBytes, randomBytes32}
|
||||
|
||||
|
|
|
@ -16,6 +16,7 @@
|
|||
|
||||
package fr.acinq.eclair.channel
|
||||
|
||||
import java.util.UUID
|
||||
import java.util.concurrent.CountDownLatch
|
||||
|
||||
import akka.actor.{Actor, ActorLogging, ActorRef, Props, Status}
|
||||
|
@ -92,7 +93,7 @@ class FuzzySpec extends TestkitBaseClass with StateTestsHelperMethods with Loggi
|
|||
// allow overpaying (no more than 2 times the required amount)
|
||||
val amount = requiredAmount + Random.nextInt(requiredAmount)
|
||||
val expiry = Globals.blockCount.get().toInt + Channel.MIN_CLTV_EXPIRY + 1
|
||||
PaymentLifecycle.buildCommand(amount, expiry, paymentHash, Hop(null, dest, null) :: Nil)._1
|
||||
PaymentLifecycle.buildCommand(UUID.randomUUID(), amount, expiry, paymentHash, Hop(null, dest, null) :: Nil)._1
|
||||
}
|
||||
|
||||
def initiatePayment(stopping: Boolean) =
|
||||
|
|
|
@ -16,6 +16,7 @@
|
|||
|
||||
package fr.acinq.eclair.channel
|
||||
|
||||
import java.util.UUID
|
||||
import java.util.concurrent.CountDownLatch
|
||||
import java.util.concurrent.atomic.AtomicLong
|
||||
|
||||
|
@ -50,7 +51,7 @@ class ThroughputSpec extends FunSuite {
|
|||
case ('add, tgt: ActorRef) =>
|
||||
val r = randomBytes32
|
||||
val h = Crypto.sha256(r)
|
||||
tgt ! CMD_ADD_HTLC(1, h, 1)
|
||||
tgt ! CMD_ADD_HTLC(1, h, 1, upstream = Left(UUID.randomUUID()))
|
||||
context.become(run(h2r + (h -> r)))
|
||||
|
||||
case ('sig, tgt: ActorRef) => tgt ! CMD_SIGN
|
||||
|
|
|
@ -16,6 +16,9 @@
|
|||
|
||||
package fr.acinq.eclair.channel.states
|
||||
|
||||
import java.util.UUID
|
||||
|
||||
import akka.actor.Actor
|
||||
import akka.testkit.{TestFSMRef, TestKitBase, TestProbe}
|
||||
import fr.acinq.bitcoin.{ByteVector32, Crypto}
|
||||
import fr.acinq.eclair.TestConstants.{Alice, Bob}
|
||||
|
@ -112,7 +115,7 @@ trait StateTestsHelperMethods extends TestKitBase {
|
|||
val sender = TestProbe()
|
||||
val receiverPubkey = r.underlyingActor.nodeParams.nodeId
|
||||
val expiry = 400144
|
||||
val cmd = PaymentLifecycle.buildCommand(amountMsat, expiry, H, Hop(null, receiverPubkey, null) :: Nil)._1.copy(commit = false)
|
||||
val cmd = PaymentLifecycle.buildCommand(UUID.randomUUID, amountMsat, expiry, H, Hop(null, receiverPubkey, null) :: Nil)._1.copy(commit = false)
|
||||
sender.send(s, cmd)
|
||||
sender.expectMsg("ok")
|
||||
val htlc = s2r.expectMsgType[UpdateAddHtlc]
|
||||
|
|
|
@ -16,6 +16,8 @@
|
|||
|
||||
package fr.acinq.eclair.channel.states.e
|
||||
|
||||
import java.util.UUID
|
||||
|
||||
import akka.actor.Status
|
||||
import akka.actor.Status.Failure
|
||||
import akka.testkit.TestProbe
|
||||
|
@ -64,7 +66,8 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
|
|||
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
|
||||
val sender = TestProbe()
|
||||
val h = randomBytes32
|
||||
sender.send(alice, CMD_ADD_HTLC(50000000, h, 400144))
|
||||
val add = CMD_ADD_HTLC(50000000, h, 400144, upstream = Left(UUID.randomUUID()))
|
||||
sender.send(alice, add)
|
||||
sender.expectMsg("ok")
|
||||
val htlc = alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
assert(htlc.id == 0 && htlc.paymentHash == h)
|
||||
|
@ -72,7 +75,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
|
|||
commitments = initialState.commitments.copy(
|
||||
localNextHtlcId = 1,
|
||||
localChanges = initialState.commitments.localChanges.copy(proposed = htlc :: Nil),
|
||||
originChannels = Map(0L -> Local(Some(sender.ref)))
|
||||
originChannels = Map(0L -> Local(add.upstream.left.get, Some(sender.ref)))
|
||||
)))
|
||||
}
|
||||
|
||||
|
@ -81,7 +84,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
|
|||
val sender = TestProbe()
|
||||
val h = randomBytes32
|
||||
for (i <- 0 until 10) {
|
||||
sender.send(alice, CMD_ADD_HTLC(50000000, h, 400144))
|
||||
sender.send(alice, CMD_ADD_HTLC(50000000, h, 400144, upstream = Left(UUID.randomUUID())))
|
||||
sender.expectMsg("ok")
|
||||
val htlc = alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
assert(htlc.id == i && htlc.paymentHash == h)
|
||||
|
@ -94,7 +97,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
|
|||
val sender = TestProbe()
|
||||
val h = randomBytes32
|
||||
val originHtlc = UpdateAddHtlc(channelId = randomBytes32, id = 5656, amountMsat = 50000000, cltvExpiry = 400144, paymentHash = h, onionRoutingPacket = ByteVector.fill(1254)(0))
|
||||
val cmd = CMD_ADD_HTLC(originHtlc.amountMsat - 10000, h, originHtlc.cltvExpiry - 7, upstream_opt = Some(originHtlc))
|
||||
val cmd = CMD_ADD_HTLC(originHtlc.amountMsat - 10000, h, originHtlc.cltvExpiry - 7, upstream = Right(originHtlc))
|
||||
sender.send(alice, cmd)
|
||||
sender.expectMsg("ok")
|
||||
val htlc = alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
|
@ -113,10 +116,10 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
|
|||
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
|
||||
val currentBlockCount = Globals.blockCount.get
|
||||
val expiryTooSmall = currentBlockCount + 3
|
||||
val add = CMD_ADD_HTLC(500000000, randomBytes32, cltvExpiry = expiryTooSmall)
|
||||
val add = CMD_ADD_HTLC(500000000, randomBytes32, cltvExpiry = expiryTooSmall, upstream = Left(UUID.randomUUID()))
|
||||
sender.send(alice, add)
|
||||
val error = ExpiryTooSmall(channelId(alice), currentBlockCount + Channel.MIN_CLTV_EXPIRY, expiryTooSmall, currentBlockCount)
|
||||
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), add.paymentHash, error, Local(Some(sender.ref)), Some(initialState.channelUpdate), Some(add))))
|
||||
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), add.paymentHash, error, Local(add.upstream.left.get, Some(sender.ref)), Some(initialState.channelUpdate), Some(add))))
|
||||
alice2bob.expectNoMsg(200 millis)
|
||||
}
|
||||
|
||||
|
@ -126,10 +129,10 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
|
|||
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
|
||||
val currentBlockCount = Globals.blockCount.get
|
||||
val expiryTooBig = currentBlockCount + Channel.MAX_CLTV_EXPIRY + 1
|
||||
val add = CMD_ADD_HTLC(500000000, randomBytes32, cltvExpiry = expiryTooBig)
|
||||
val add = CMD_ADD_HTLC(500000000, randomBytes32, cltvExpiry = expiryTooBig, upstream = Left(UUID.randomUUID()))
|
||||
sender.send(alice, add)
|
||||
val error = ExpiryTooBig(channelId(alice), maximum = currentBlockCount + Channel.MAX_CLTV_EXPIRY, actual = expiryTooBig, blockCount = currentBlockCount)
|
||||
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), add.paymentHash, error, Local(Some(sender.ref)), Some(initialState.channelUpdate), Some(add))))
|
||||
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), add.paymentHash, error, Local(add.upstream.left.get, Some(sender.ref)), Some(initialState.channelUpdate), Some(add))))
|
||||
alice2bob.expectNoMsg(200 millis)
|
||||
}
|
||||
|
||||
|
@ -137,10 +140,10 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
|
|||
import f._
|
||||
val sender = TestProbe()
|
||||
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
|
||||
val add = CMD_ADD_HTLC(50, randomBytes32, 400144)
|
||||
val add = CMD_ADD_HTLC(50, randomBytes32, 400144, upstream = Left(UUID.randomUUID()))
|
||||
sender.send(alice, add)
|
||||
val error = HtlcValueTooSmall(channelId(alice), 1000, 50)
|
||||
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), add.paymentHash, error, Local(Some(sender.ref)), Some(initialState.channelUpdate), Some(add))))
|
||||
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), add.paymentHash, error, Local(add.upstream.left.get, Some(sender.ref)), Some(initialState.channelUpdate), Some(add))))
|
||||
alice2bob.expectNoMsg(200 millis)
|
||||
}
|
||||
|
||||
|
@ -148,10 +151,10 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
|
|||
import f._
|
||||
val sender = TestProbe()
|
||||
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
|
||||
val add = CMD_ADD_HTLC(Int.MaxValue, randomBytes32, 400144)
|
||||
val add = CMD_ADD_HTLC(Int.MaxValue, randomBytes32, 400144, upstream = Left(UUID.randomUUID()))
|
||||
sender.send(alice, add)
|
||||
val error = InsufficientFunds(channelId(alice), amountMsat = Int.MaxValue, missingSatoshis = 1376443, reserveSatoshis = 20000, feesSatoshis = 8960)
|
||||
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), add.paymentHash, error, Local(Some(sender.ref)), Some(initialState.channelUpdate), Some(add))))
|
||||
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), add.paymentHash, error, Local(add.upstream.left.get, Some(sender.ref)), Some(initialState.channelUpdate), Some(add))))
|
||||
alice2bob.expectNoMsg(200 millis)
|
||||
}
|
||||
|
||||
|
@ -159,19 +162,19 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
|
|||
import f._
|
||||
val sender = TestProbe()
|
||||
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
|
||||
sender.send(alice, CMD_ADD_HTLC(500000000, randomBytes32, 400144))
|
||||
sender.send(alice, CMD_ADD_HTLC(500000000, randomBytes32, 400144, upstream = Left(UUID.randomUUID())))
|
||||
sender.expectMsg("ok")
|
||||
alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
sender.send(alice, CMD_ADD_HTLC(200000000, randomBytes32, 400144))
|
||||
sender.send(alice, CMD_ADD_HTLC(200000000, randomBytes32, 400144, upstream = Left(UUID.randomUUID())))
|
||||
sender.expectMsg("ok")
|
||||
alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
sender.send(alice, CMD_ADD_HTLC(67600000, randomBytes32, 400144))
|
||||
sender.send(alice, CMD_ADD_HTLC(67600000, randomBytes32, 400144, upstream = Left(UUID.randomUUID())))
|
||||
sender.expectMsg("ok")
|
||||
alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
val add = CMD_ADD_HTLC(1000000, randomBytes32, 400144)
|
||||
val add = CMD_ADD_HTLC(1000000, randomBytes32, 400144, upstream = Left(UUID.randomUUID()))
|
||||
sender.send(alice, add)
|
||||
val error = InsufficientFunds(channelId(alice), amountMsat = 1000000, missingSatoshis = 1000, reserveSatoshis = 20000, feesSatoshis = 12400)
|
||||
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), add.paymentHash, error, Local(Some(sender.ref)), Some(initialState.channelUpdate), Some(add))))
|
||||
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), add.paymentHash, error, Local(add.upstream.left.get, Some(sender.ref)), Some(initialState.channelUpdate), Some(add))))
|
||||
alice2bob.expectNoMsg(200 millis)
|
||||
}
|
||||
|
||||
|
@ -179,16 +182,16 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
|
|||
import f._
|
||||
val sender = TestProbe()
|
||||
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
|
||||
sender.send(alice, CMD_ADD_HTLC(300000000, randomBytes32, 400144))
|
||||
sender.send(alice, CMD_ADD_HTLC(300000000, randomBytes32, 400144, upstream = Left(UUID.randomUUID())))
|
||||
sender.expectMsg("ok")
|
||||
alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
sender.send(alice, CMD_ADD_HTLC(300000000, randomBytes32, 400144))
|
||||
sender.send(alice, CMD_ADD_HTLC(300000000, randomBytes32, 400144, upstream = Left(UUID.randomUUID())))
|
||||
sender.expectMsg("ok")
|
||||
alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
val add = CMD_ADD_HTLC(500000000, randomBytes32, 400144)
|
||||
val add = CMD_ADD_HTLC(500000000, randomBytes32, 400144, upstream = Left(UUID.randomUUID()))
|
||||
sender.send(alice, add)
|
||||
val error = InsufficientFunds(channelId(alice), amountMsat = 500000000, missingSatoshis = 332400, reserveSatoshis = 20000, feesSatoshis = 12400)
|
||||
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), add.paymentHash, error, Local(Some(sender.ref)), Some(initialState.channelUpdate), Some(add))))
|
||||
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), add.paymentHash, error, Local(add.upstream.left.get, Some(sender.ref)), Some(initialState.channelUpdate), Some(add))))
|
||||
alice2bob.expectNoMsg(200 millis)
|
||||
}
|
||||
|
||||
|
@ -196,10 +199,10 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
|
|||
import f._
|
||||
val sender = TestProbe()
|
||||
val initialState = bob.stateData.asInstanceOf[DATA_NORMAL]
|
||||
val add = CMD_ADD_HTLC(151000000, randomBytes32, 400144)
|
||||
val add = CMD_ADD_HTLC(151000000, randomBytes32, 400144, upstream = Left(UUID.randomUUID()))
|
||||
sender.send(bob, add)
|
||||
val error = HtlcValueTooHighInFlight(channelId(bob), maximum = 150000000, actual = 151000000)
|
||||
sender.expectMsg(Failure(AddHtlcFailed(channelId(bob), add.paymentHash, error, Local(Some(sender.ref)), Some(initialState.channelUpdate), Some(add))))
|
||||
sender.expectMsg(Failure(AddHtlcFailed(channelId(bob), add.paymentHash, error, Local(add.upstream.left.get, Some(sender.ref)), Some(initialState.channelUpdate), Some(add))))
|
||||
bob2alice.expectNoMsg(200 millis)
|
||||
}
|
||||
|
||||
|
@ -209,14 +212,14 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
|
|||
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
|
||||
// Bob accepts a maximum of 30 htlcs
|
||||
for (i <- 0 until 30) {
|
||||
sender.send(alice, CMD_ADD_HTLC(10000000, randomBytes32, 400144))
|
||||
sender.send(alice, CMD_ADD_HTLC(10000000, randomBytes32, 400144, upstream = Left(UUID.randomUUID())))
|
||||
sender.expectMsg("ok")
|
||||
alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
}
|
||||
val add = CMD_ADD_HTLC(10000000, randomBytes32, 400144)
|
||||
val add = CMD_ADD_HTLC(10000000, randomBytes32, 400144, upstream = Left(UUID.randomUUID()))
|
||||
sender.send(alice, add)
|
||||
val error = TooManyAcceptedHtlcs(channelId(alice), maximum = 30)
|
||||
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), add.paymentHash, error, Local(Some(sender.ref)), Some(initialState.channelUpdate), Some(add))))
|
||||
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), add.paymentHash, error, Local(add.upstream.left.get, Some(sender.ref)), Some(initialState.channelUpdate), Some(add))))
|
||||
alice2bob.expectNoMsg(200 millis)
|
||||
}
|
||||
|
||||
|
@ -224,7 +227,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
|
|||
import f._
|
||||
val sender = TestProbe()
|
||||
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
|
||||
val add1 = CMD_ADD_HTLC(TestConstants.fundingSatoshis * 2 / 3 * 1000, randomBytes32, 400144)
|
||||
val add1 = CMD_ADD_HTLC(TestConstants.fundingSatoshis * 2 / 3 * 1000, randomBytes32, 400144, upstream = Left(UUID.randomUUID()))
|
||||
sender.send(alice, add1)
|
||||
sender.expectMsg("ok")
|
||||
alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
|
@ -232,10 +235,10 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
|
|||
sender.expectMsg("ok")
|
||||
alice2bob.expectMsgType[CommitSig]
|
||||
// this is over channel-capacity
|
||||
val add2 = CMD_ADD_HTLC(TestConstants.fundingSatoshis * 2 / 3 * 1000, randomBytes32, 400144)
|
||||
val add2 = CMD_ADD_HTLC(TestConstants.fundingSatoshis * 2 / 3 * 1000, randomBytes32, 400144, upstream = Left(UUID.randomUUID()))
|
||||
sender.send(alice, add2)
|
||||
val error = InsufficientFunds(channelId(alice), add2.amountMsat, 564012, 20000, 10680)
|
||||
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), add2.paymentHash, error, Local(Some(sender.ref)), Some(initialState.channelUpdate), Some(add2))))
|
||||
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), add2.paymentHash, error, Local(add2.upstream.left.get, Some(sender.ref)), Some(initialState.channelUpdate), Some(add2))))
|
||||
alice2bob.expectNoMsg(200 millis)
|
||||
}
|
||||
|
||||
|
@ -249,10 +252,10 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
|
|||
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].localShutdown.isDefined && !alice.stateData.asInstanceOf[DATA_NORMAL].remoteShutdown.isDefined)
|
||||
|
||||
// actual test starts here
|
||||
val add = CMD_ADD_HTLC(500000000, randomBytes32, cltvExpiry = 400144)
|
||||
val add = CMD_ADD_HTLC(500000000, randomBytes32, cltvExpiry = 400144, upstream = Left(UUID.randomUUID()))
|
||||
sender.send(alice, add)
|
||||
val error = NoMoreHtlcsClosingInProgress(channelId(alice))
|
||||
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), add.paymentHash, error, Local(Some(sender.ref)), Some(initialState.channelUpdate), Some(add))))
|
||||
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), add.paymentHash, error, Local(add.upstream.left.get, Some(sender.ref)), Some(initialState.channelUpdate), Some(add))))
|
||||
alice2bob.expectNoMsg(200 millis)
|
||||
}
|
||||
|
||||
|
@ -261,14 +264,14 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
|
|||
val sender = TestProbe()
|
||||
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
|
||||
// let's make alice send an htlc
|
||||
val add1 = CMD_ADD_HTLC(500000000, randomBytes32, cltvExpiry = 400144)
|
||||
val add1 = CMD_ADD_HTLC(500000000, randomBytes32, cltvExpiry = 400144, upstream = Left(UUID.randomUUID()))
|
||||
sender.send(alice, add1)
|
||||
sender.expectMsg("ok")
|
||||
// at the same time bob initiates a closing
|
||||
sender.send(bob, CMD_CLOSE(None))
|
||||
sender.expectMsg("ok")
|
||||
// this command will be received by alice right after having received the shutdown
|
||||
val add2 = CMD_ADD_HTLC(100000000, randomBytes32, cltvExpiry = 300000)
|
||||
val add2 = CMD_ADD_HTLC(100000000, randomBytes32, cltvExpiry = 300000, upstream = Left(UUID.randomUUID()))
|
||||
// messages cross
|
||||
alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
alice2bob.forward(bob)
|
||||
|
@ -276,7 +279,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
|
|||
bob2alice.forward(alice)
|
||||
sender.send(alice, add2)
|
||||
val error = NoMoreHtlcsClosingInProgress(channelId(alice))
|
||||
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), add2.paymentHash, error, Local(Some(sender.ref)), Some(initialState.channelUpdate), Some(add2))))
|
||||
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), add2.paymentHash, error, Local(add2.upstream.left.get, Some(sender.ref)), Some(initialState.channelUpdate), Some(add2))))
|
||||
}
|
||||
|
||||
test("recv UpdateAddHtlc") { f =>
|
||||
|
@ -415,7 +418,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
|
|||
test("recv CMD_SIGN (two identical htlcs in each direction)") { f =>
|
||||
import f._
|
||||
val sender = TestProbe()
|
||||
val add = CMD_ADD_HTLC(10000000, randomBytes32, 400144)
|
||||
val add = CMD_ADD_HTLC(10000000, randomBytes32, 400144, upstream = Left(UUID.randomUUID()))
|
||||
sender.send(alice, add)
|
||||
sender.expectMsg("ok")
|
||||
alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
|
@ -462,19 +465,19 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
|
|||
assert(a2b_2 > aliceMinOffer && a2b_2 > bobMinReceive)
|
||||
assert(b2a_1 > aliceMinReceive && b2a_1 > bobMinOffer)
|
||||
assert(b2a_2 < aliceMinReceive && b2a_2 > bobMinOffer)
|
||||
sender.send(alice, CMD_ADD_HTLC(a2b_1 * 1000, randomBytes32, 400144))
|
||||
sender.send(alice, CMD_ADD_HTLC(a2b_1 * 1000, randomBytes32, 400144, upstream = Left(UUID.randomUUID())))
|
||||
sender.expectMsg("ok")
|
||||
alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
alice2bob.forward(bob)
|
||||
sender.send(alice, CMD_ADD_HTLC(a2b_2 * 1000, randomBytes32, 400144))
|
||||
sender.send(alice, CMD_ADD_HTLC(a2b_2 * 1000, randomBytes32, 400144, upstream = Left(UUID.randomUUID())))
|
||||
sender.expectMsg("ok")
|
||||
alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
alice2bob.forward(bob)
|
||||
sender.send(bob, CMD_ADD_HTLC(b2a_1 * 1000, randomBytes32, 400144))
|
||||
sender.send(bob, CMD_ADD_HTLC(b2a_1 * 1000, randomBytes32, 400144, upstream = Left(UUID.randomUUID())))
|
||||
sender.expectMsg("ok")
|
||||
bob2alice.expectMsgType[UpdateAddHtlc]
|
||||
bob2alice.forward(alice)
|
||||
sender.send(bob, CMD_ADD_HTLC(b2a_2 * 1000, randomBytes32, 400144))
|
||||
sender.send(bob, CMD_ADD_HTLC(b2a_2 * 1000, randomBytes32, 400144, upstream = Left(UUID.randomUUID())))
|
||||
sender.expectMsg("ok")
|
||||
bob2alice.expectMsgType[UpdateAddHtlc]
|
||||
bob2alice.forward(alice)
|
||||
|
@ -494,7 +497,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
|
|||
test("recv CMD_SIGN (htlcs with same pubkeyScript but different amounts)") { f =>
|
||||
import f._
|
||||
val sender = TestProbe()
|
||||
val add = CMD_ADD_HTLC(10000000, randomBytes32, 400144)
|
||||
val add = CMD_ADD_HTLC(10000000, randomBytes32, 400144, upstream = Left(UUID.randomUUID()))
|
||||
val epsilons = List(3, 1, 5, 7, 6) // unordered on purpose
|
||||
val htlcCount = epsilons.size
|
||||
for (i <- epsilons) {
|
||||
|
@ -691,12 +694,12 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
|
|||
val r = randomBytes32
|
||||
val h = Crypto.sha256(r)
|
||||
|
||||
sender.send(alice, CMD_ADD_HTLC(50000000, h, 400144))
|
||||
sender.send(alice, CMD_ADD_HTLC(50000000, h, 400144, upstream = Left(UUID.randomUUID())))
|
||||
sender.expectMsg("ok")
|
||||
val htlc1 = alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
alice2bob.forward(bob)
|
||||
|
||||
sender.send(alice, CMD_ADD_HTLC(50000000, h, 400144))
|
||||
sender.send(alice, CMD_ADD_HTLC(50000000, h, 400144, upstream = Left(UUID.randomUUID())))
|
||||
sender.expectMsg("ok")
|
||||
val htlc2 = alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
alice2bob.forward(bob)
|
||||
|
@ -1864,7 +1867,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
|
|||
// alice = 800 000
|
||||
// bob = 200 000
|
||||
|
||||
val add = CMD_ADD_HTLC(10000000, randomBytes32, 400144)
|
||||
val add = CMD_ADD_HTLC(10000000, randomBytes32, 400144, upstream = Left(UUID.randomUUID()))
|
||||
sender.send(alice, add)
|
||||
sender.expectMsg("ok")
|
||||
alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
|
|
|
@ -16,6 +16,8 @@
|
|||
|
||||
package fr.acinq.eclair.channel.states.e
|
||||
|
||||
import java.util.UUID
|
||||
|
||||
import akka.testkit.TestProbe
|
||||
import fr.acinq.bitcoin.Crypto.Scalar
|
||||
import fr.acinq.bitcoin.{ByteVector32, ScriptFlags, Transaction}
|
||||
|
@ -58,7 +60,7 @@ class OfflineStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
|
|||
import f._
|
||||
val sender = TestProbe()
|
||||
|
||||
sender.send(alice, CMD_ADD_HTLC(1000000, ByteVector32.Zeroes, 400144))
|
||||
sender.send(alice, CMD_ADD_HTLC(1000000, ByteVector32.Zeroes, 400144, upstream = Left(UUID.randomUUID())))
|
||||
val ab_add_0 = alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
// add ->b
|
||||
alice2bob.forward(bob)
|
||||
|
@ -135,7 +137,7 @@ class OfflineStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
|
|||
import f._
|
||||
val sender = TestProbe()
|
||||
|
||||
sender.send(alice, CMD_ADD_HTLC(1000000, randomBytes32, 400144))
|
||||
sender.send(alice, CMD_ADD_HTLC(1000000, randomBytes32, 400144, upstream = Left(UUID.randomUUID())))
|
||||
val ab_add_0 = alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
// add ->b
|
||||
alice2bob.forward(bob, ab_add_0)
|
||||
|
|
|
@ -16,6 +16,8 @@
|
|||
|
||||
package fr.acinq.eclair.channel.states.f
|
||||
|
||||
import java.util.UUID
|
||||
|
||||
import akka.actor.Status.Failure
|
||||
import akka.testkit.TestProbe
|
||||
import fr.acinq.bitcoin.Crypto.Scalar
|
||||
|
@ -54,7 +56,7 @@ class ShutdownStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
|
|||
val h1 = Crypto.sha256(r1)
|
||||
val amount1 = 300000000
|
||||
val expiry1 = 400144
|
||||
val cmd1 = PaymentLifecycle.buildCommand(amount1, expiry1, h1, Hop(null, TestConstants.Bob.nodeParams.nodeId, null) :: Nil)._1.copy(commit = false)
|
||||
val cmd1 = PaymentLifecycle.buildCommand(UUID.randomUUID, amount1, expiry1, h1, Hop(null, TestConstants.Bob.nodeParams.nodeId, null) :: Nil)._1.copy(commit = false)
|
||||
sender.send(alice, cmd1)
|
||||
sender.expectMsg("ok")
|
||||
val htlc1 = alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
|
@ -64,7 +66,7 @@ class ShutdownStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
|
|||
val h2 = Crypto.sha256(r2)
|
||||
val amount2 = 200000000
|
||||
val expiry2 = 400144
|
||||
val cmd2 = PaymentLifecycle.buildCommand(amount2, expiry2, h2, Hop(null, TestConstants.Bob.nodeParams.nodeId, null) :: Nil)._1.copy(commit = false)
|
||||
val cmd2 = PaymentLifecycle.buildCommand(UUID.randomUUID, amount2, expiry2, h2, Hop(null, TestConstants.Bob.nodeParams.nodeId, null) :: Nil)._1.copy(commit = false)
|
||||
sender.send(alice, cmd2)
|
||||
sender.expectMsg("ok")
|
||||
val htlc2 = alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
|
@ -101,10 +103,10 @@ class ShutdownStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
|
|||
test("recv CMD_ADD_HTLC") { f =>
|
||||
import f._
|
||||
val sender = TestProbe()
|
||||
val add = CMD_ADD_HTLC(500000000, r1, cltvExpiry = 300000)
|
||||
val add = CMD_ADD_HTLC(500000000, r1, cltvExpiry = 300000, upstream = Left(UUID.randomUUID()))
|
||||
sender.send(alice, add)
|
||||
val error = ChannelUnavailable(channelId(alice))
|
||||
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), add.paymentHash, error, Local(Some(sender.ref)), None, Some(add))))
|
||||
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), add.paymentHash, error, Local(add.upstream.left.get, Some(sender.ref)), None, Some(add))))
|
||||
alice2bob.expectNoMsg(200 millis)
|
||||
}
|
||||
|
||||
|
|
|
@ -16,6 +16,8 @@
|
|||
|
||||
package fr.acinq.eclair.channel.states.g
|
||||
|
||||
import java.util.UUID
|
||||
|
||||
import akka.actor.Status.Failure
|
||||
import akka.event.LoggingAdapter
|
||||
import akka.testkit.TestProbe
|
||||
|
@ -70,10 +72,10 @@ class NegotiatingStateSpec extends TestkitBaseClass with StateTestsHelperMethods
|
|||
import f._
|
||||
alice2bob.expectMsgType[ClosingSigned]
|
||||
val sender = TestProbe()
|
||||
val add = CMD_ADD_HTLC(500000000, ByteVector32(ByteVector.fill(32)(1)), cltvExpiry = 300000)
|
||||
val add = CMD_ADD_HTLC(500000000, ByteVector32(ByteVector.fill(32)(1)), cltvExpiry = 300000, upstream = Left(UUID.randomUUID()))
|
||||
sender.send(alice, add)
|
||||
val error = ChannelUnavailable(channelId(alice))
|
||||
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), add.paymentHash, error, Local(Some(sender.ref)), None, Some(add))))
|
||||
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), add.paymentHash, error, Local(add.upstream.left.get, Some(sender.ref)), None, Some(add))))
|
||||
alice2bob.expectNoMsg(200 millis)
|
||||
}
|
||||
|
||||
|
|
|
@ -16,6 +16,8 @@
|
|||
|
||||
package fr.acinq.eclair.channel.states.h
|
||||
|
||||
import java.util.UUID
|
||||
|
||||
import akka.actor.Status
|
||||
import akka.actor.Status.Failure
|
||||
import akka.testkit.{TestFSMRef, TestProbe}
|
||||
|
@ -113,10 +115,10 @@ class ClosingStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
|
|||
|
||||
// actual test starts here
|
||||
val sender = TestProbe()
|
||||
val add = CMD_ADD_HTLC(500000000, ByteVector32(ByteVector.fill(32)(1)), cltvExpiry = 300000)
|
||||
val add = CMD_ADD_HTLC(500000000, ByteVector32(ByteVector.fill(32)(1)), cltvExpiry = 300000, upstream = Left(UUID.randomUUID()))
|
||||
sender.send(alice, add)
|
||||
val error = ChannelUnavailable(channelId(alice))
|
||||
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), add.paymentHash, error, Local(Some(sender.ref)), None, Some(add))))
|
||||
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), add.paymentHash, error, Local(add.upstream.left.get, Some(sender.ref)), None, Some(add))))
|
||||
alice2bob.expectNoMsg(200 millis)
|
||||
}
|
||||
|
||||
|
|
|
@ -16,6 +16,8 @@
|
|||
|
||||
package fr.acinq.eclair.db
|
||||
|
||||
import java.util.UUID
|
||||
|
||||
import fr.acinq.bitcoin.Crypto.{PrivateKey, Scalar}
|
||||
import fr.acinq.bitcoin.{Block, ByteVector32, Crypto, DeterministicWallet, MilliSatoshi, Satoshi, Transaction}
|
||||
import fr.acinq.eclair.channel.Helpers.Funding
|
||||
|
@ -104,7 +106,7 @@ object ChannelStateSpec {
|
|||
val commitments = Commitments(localParams, remoteParams, channelFlags = 0x01.toByte, localCommit, remoteCommit, LocalChanges(Nil, Nil, Nil), RemoteChanges(Nil, Nil, Nil),
|
||||
localNextHtlcId = 32L,
|
||||
remoteNextHtlcId = 4L,
|
||||
originChannels = Map(42L -> Local(None), 15000L -> Relayed(ByteVector32(ByteVector.fill(32)(42)), 43, 11000000L, 10000000L)),
|
||||
originChannels = Map(42L -> Local(UUID.randomUUID, None), 15000L -> Relayed(ByteVector32(ByteVector.fill(32)(42)), 43, 11000000L, 10000000L)),
|
||||
remoteNextCommitInfo = Right(randomKey.publicKey),
|
||||
commitInput = commitmentInput, remotePerCommitmentSecrets = ShaChain.init, channelId = ByteVector32.Zeroes)
|
||||
|
||||
|
|
|
@ -16,38 +16,37 @@
|
|||
|
||||
package fr.acinq.eclair.db
|
||||
|
||||
import java.sql.DriverManager
|
||||
import java.util.UUID
|
||||
|
||||
import fr.acinq.bitcoin.{MilliSatoshi, Satoshi, Transaction}
|
||||
import fr.acinq.eclair.channel.{AvailableBalanceChanged, NetworkFeePaid}
|
||||
import fr.acinq.eclair.db.sqlite.SqliteAuditDb
|
||||
import fr.acinq.eclair.db.sqlite.SqliteUtils.{ExtendedResultSet, getVersion, using}
|
||||
import fr.acinq.eclair.payment.{PaymentReceived, PaymentRelayed, PaymentSent}
|
||||
import fr.acinq.eclair.{ShortChannelId, randomBytes32, randomKey}
|
||||
import fr.acinq.eclair.wire.ChannelCodecs
|
||||
import fr.acinq.eclair.{ShortChannelId, TestConstants, randomBytes32, randomKey}
|
||||
import org.scalatest.FunSuite
|
||||
|
||||
import scala.compat.Platform
|
||||
|
||||
|
||||
class SqliteAuditDbSpec extends FunSuite {
|
||||
|
||||
def inmem = DriverManager.getConnection("jdbc:sqlite::memory:")
|
||||
|
||||
test("init sqlite 2 times in a row") {
|
||||
val sqlite = inmem
|
||||
val sqlite = TestConstants.sqliteInMemory()
|
||||
val db1 = new SqliteAuditDb(sqlite)
|
||||
val db2 = new SqliteAuditDb(sqlite)
|
||||
}
|
||||
|
||||
test("add/list events") {
|
||||
val sqlite = inmem
|
||||
val sqlite = TestConstants.sqliteInMemory()
|
||||
val db = new SqliteAuditDb(sqlite)
|
||||
|
||||
val e1 = PaymentSent(MilliSatoshi(42000), MilliSatoshi(1000), randomBytes32, randomBytes32, randomBytes32)
|
||||
val e1 = PaymentSent(ChannelCodecs.UNKNOWN_UUID, MilliSatoshi(42000), MilliSatoshi(1000), randomBytes32, randomBytes32, randomBytes32)
|
||||
val e2 = PaymentReceived(MilliSatoshi(42000), randomBytes32, randomBytes32)
|
||||
val e3 = PaymentRelayed(MilliSatoshi(42000), MilliSatoshi(1000), randomBytes32, randomBytes32, randomBytes32)
|
||||
val e4 = NetworkFeePaid(null, randomKey.publicKey, randomBytes32, Transaction(0, Seq.empty, Seq.empty, 0), Satoshi(42), "mutual")
|
||||
val e5 = PaymentSent(MilliSatoshi(42000), MilliSatoshi(1000), randomBytes32, randomBytes32, randomBytes32, timestamp = 0)
|
||||
val e6 = PaymentSent(MilliSatoshi(42000), MilliSatoshi(1000), randomBytes32, randomBytes32, randomBytes32, timestamp = Platform.currentTime * 2)
|
||||
val e5 = PaymentSent(ChannelCodecs.UNKNOWN_UUID, MilliSatoshi(42000), MilliSatoshi(1000), randomBytes32, randomBytes32, randomBytes32, timestamp = 0)
|
||||
val e6 = PaymentSent(ChannelCodecs.UNKNOWN_UUID, MilliSatoshi(42000), MilliSatoshi(1000), randomBytes32, randomBytes32, randomBytes32, timestamp = Platform.currentTime * 2)
|
||||
val e7 = AvailableBalanceChanged(null, randomBytes32, ShortChannelId(500000, 42, 1), 456123000, ChannelStateSpec.commitments)
|
||||
val e8 = ChannelLifecycleEvent(randomBytes32, randomKey.publicKey, 456123000, true, false, "mutual")
|
||||
|
||||
|
@ -69,7 +68,7 @@ class SqliteAuditDbSpec extends FunSuite {
|
|||
}
|
||||
|
||||
test("stats") {
|
||||
val sqlite = inmem
|
||||
val sqlite = TestConstants.sqliteInMemory()
|
||||
val db = new SqliteAuditDb(sqlite)
|
||||
|
||||
val n1 = randomKey.publicKey
|
||||
|
@ -93,7 +92,72 @@ class SqliteAuditDbSpec extends FunSuite {
|
|||
assert(db.stats.toSet === Set(
|
||||
Stats(channelId = c1, avgPaymentAmountSatoshi = 42, paymentCount = 3, relayFeeSatoshi = 4, networkFeeSatoshi = 100),
|
||||
Stats(channelId = c2, avgPaymentAmountSatoshi = 40, paymentCount = 1, relayFeeSatoshi = 2, networkFeeSatoshi = 500),
|
||||
Stats(channelId = c3, avgPaymentAmountSatoshi = 0, paymentCount = 0, relayFeeSatoshi = 0, networkFeeSatoshi = 400)
|
||||
Stats(channelId = c3, avgPaymentAmountSatoshi = 0, paymentCount = 0, relayFeeSatoshi = 0, networkFeeSatoshi = 400)
|
||||
))
|
||||
}
|
||||
|
||||
test("handle migration version 1 -> 2") {
|
||||
|
||||
import ExtendedResultSet._
|
||||
val connection = TestConstants.sqliteInMemory()
|
||||
|
||||
// simulate existing previous version db
|
||||
using(connection.createStatement()) { statement =>
|
||||
getVersion(statement, "audit", 1)
|
||||
statement.executeUpdate("CREATE TABLE IF NOT EXISTS balance_updated (channel_id BLOB NOT NULL, node_id BLOB NOT NULL, amount_msat INTEGER NOT NULL, capacity_sat INTEGER NOT NULL, reserve_sat INTEGER NOT NULL, timestamp INTEGER NOT NULL)")
|
||||
statement.executeUpdate("CREATE TABLE IF NOT EXISTS sent (amount_msat INTEGER NOT NULL, fees_msat INTEGER NOT NULL, payment_hash BLOB NOT NULL, payment_preimage BLOB NOT NULL, to_channel_id BLOB NOT NULL, timestamp INTEGER NOT NULL)")
|
||||
statement.executeUpdate("CREATE TABLE IF NOT EXISTS received (amount_msat INTEGER NOT NULL, payment_hash BLOB NOT NULL, from_channel_id BLOB NOT NULL, timestamp INTEGER NOT NULL)")
|
||||
statement.executeUpdate("CREATE TABLE IF NOT EXISTS relayed (amount_in_msat INTEGER NOT NULL, amount_out_msat INTEGER NOT NULL, payment_hash BLOB NOT NULL, from_channel_id BLOB NOT NULL, to_channel_id BLOB NOT NULL, timestamp INTEGER NOT NULL)")
|
||||
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 STRING NOT NULL, timestamp INTEGER NOT NULL)")
|
||||
|
||||
statement.executeUpdate("CREATE INDEX IF NOT EXISTS balance_updated_idx ON balance_updated(timestamp)")
|
||||
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)")
|
||||
statement.executeUpdate("CREATE INDEX IF NOT EXISTS network_fees_timestamp_idx ON network_fees(timestamp)")
|
||||
statement.executeUpdate("CREATE INDEX IF NOT EXISTS channel_events_timestamp_idx ON channel_events(timestamp)")
|
||||
}
|
||||
|
||||
using(connection.createStatement()) { statement =>
|
||||
assert(getVersion(statement, "audit", 1) == 1) // version 1 is deployed now
|
||||
}
|
||||
|
||||
val ps = PaymentSent(UUID.randomUUID(), MilliSatoshi(42000), MilliSatoshi(1000), randomBytes32, randomBytes32, randomBytes32)
|
||||
val ps1 = PaymentSent(UUID.randomUUID(), MilliSatoshi(42001), MilliSatoshi(1001), randomBytes32, randomBytes32, randomBytes32)
|
||||
val ps2 = PaymentSent(UUID.randomUUID(), MilliSatoshi(42002), MilliSatoshi(1002), randomBytes32, randomBytes32, randomBytes32)
|
||||
|
||||
// add a row (no ID on sent)
|
||||
using(connection.prepareStatement("INSERT INTO sent VALUES (?, ?, ?, ?, ?, ?)")) { statement =>
|
||||
statement.setLong(1, ps.amount.toLong)
|
||||
statement.setLong(2, ps.feesPaid.toLong)
|
||||
statement.setBytes(3, ps.paymentHash.toArray)
|
||||
statement.setBytes(4, ps.paymentPreimage.toArray)
|
||||
statement.setBytes(5, ps.toChannelId.toArray)
|
||||
statement.setLong(6, ps.timestamp)
|
||||
statement.executeUpdate()
|
||||
}
|
||||
|
||||
val migratedDb = new SqliteAuditDb(connection)
|
||||
|
||||
using(connection.createStatement()) { statement =>
|
||||
assert(getVersion(statement, "audit", 1) == 2) // version changed from 1 -> 2
|
||||
}
|
||||
|
||||
// existing rows will use 00000000-0000-0000-0000-000000000000 as default
|
||||
assert(migratedDb.listSent(0, Long.MaxValue) == Seq(ps.copy(id = ChannelCodecs.UNKNOWN_UUID)))
|
||||
|
||||
val postMigrationDb = new SqliteAuditDb(connection)
|
||||
|
||||
using(connection.createStatement()) { statement =>
|
||||
assert(getVersion(statement, "audit", 2) == 2) // version 2
|
||||
}
|
||||
|
||||
postMigrationDb.add(ps1)
|
||||
postMigrationDb.add(ps2)
|
||||
|
||||
// the old record will have the UNKNOWN_UUID but the new ones will have their actual id
|
||||
assert(postMigrationDb.listSent(0, Long.MaxValue) == Seq(ps.copy(id = ChannelCodecs.UNKNOWN_UUID), ps1, ps2))
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -16,9 +16,8 @@
|
|||
|
||||
package fr.acinq.eclair.db
|
||||
|
||||
import java.sql.DriverManager
|
||||
|
||||
import fr.acinq.bitcoin.ByteVector32
|
||||
import fr.acinq.eclair.TestConstants
|
||||
import fr.acinq.eclair.db.sqlite.{SqliteChannelsDb, SqlitePendingRelayDb}
|
||||
import org.scalatest.FunSuite
|
||||
import org.sqlite.SQLiteException
|
||||
|
@ -27,16 +26,14 @@ import scodec.bits.ByteVector
|
|||
|
||||
class SqliteChannelsDbSpec extends FunSuite {
|
||||
|
||||
def inmem = DriverManager.getConnection("jdbc:sqlite::memory:")
|
||||
|
||||
test("init sqlite 2 times in a row") {
|
||||
val sqlite = inmem
|
||||
val sqlite = TestConstants.sqliteInMemory()
|
||||
val db1 = new SqliteChannelsDb(sqlite)
|
||||
val db2 = new SqliteChannelsDb(sqlite)
|
||||
}
|
||||
|
||||
test("add/remove/list channels") {
|
||||
val sqlite = inmem
|
||||
val sqlite = TestConstants.sqliteInMemory()
|
||||
val db = new SqliteChannelsDb(sqlite)
|
||||
new SqlitePendingRelayDb(sqlite) // needed by db.removeChannel
|
||||
|
||||
|
|
|
@ -16,31 +16,27 @@
|
|||
|
||||
package fr.acinq.eclair.db
|
||||
|
||||
import java.sql.DriverManager
|
||||
|
||||
import fr.acinq.bitcoin.{Block, Crypto, Satoshi}
|
||||
import fr.acinq.eclair.db.sqlite.SqliteNetworkDb
|
||||
import fr.acinq.eclair.router.Announcements
|
||||
import fr.acinq.eclair.wire.{Color, NodeAddress, Tor2}
|
||||
import fr.acinq.eclair.{ShortChannelId, randomBytes32, randomKey}
|
||||
import fr.acinq.eclair.{ShortChannelId, TestConstants, randomBytes32, randomKey}
|
||||
import org.scalatest.FunSuite
|
||||
import org.sqlite.SQLiteException
|
||||
|
||||
|
||||
class SqliteNetworkDbSpec extends FunSuite {
|
||||
|
||||
def inmem = DriverManager.getConnection("jdbc:sqlite::memory:")
|
||||
|
||||
val shortChannelIds = (42 to (5000 + 42)).map(i => ShortChannelId(i))
|
||||
|
||||
test("init sqlite 2 times in a row") {
|
||||
val sqlite = inmem
|
||||
val sqlite = TestConstants.sqliteInMemory()
|
||||
val db1 = new SqliteNetworkDb(sqlite)
|
||||
val db2 = new SqliteNetworkDb(sqlite)
|
||||
}
|
||||
|
||||
test("add/remove/list nodes") {
|
||||
val sqlite = inmem
|
||||
val sqlite = TestConstants.sqliteInMemory()
|
||||
val db = new SqliteNetworkDb(sqlite)
|
||||
|
||||
val node_1 = Announcements.makeNodeAnnouncement(randomKey, "node-alice", Color(100.toByte, 200.toByte, 300.toByte), NodeAddress.fromParts("192.168.1.42", 42000).get :: Nil)
|
||||
|
@ -64,7 +60,7 @@ class SqliteNetworkDbSpec extends FunSuite {
|
|||
}
|
||||
|
||||
test("add/remove/list channels and channel_updates") {
|
||||
val sqlite = inmem
|
||||
val sqlite = TestConstants.sqliteInMemory()
|
||||
val db = new SqliteNetworkDb(sqlite)
|
||||
|
||||
def sig = Crypto.encodeSignature(Crypto.sign(randomBytes32, randomKey)) :+ 1.toByte
|
||||
|
@ -105,7 +101,7 @@ class SqliteNetworkDbSpec extends FunSuite {
|
|||
}
|
||||
|
||||
test("remove many channels") {
|
||||
val sqlite = inmem
|
||||
val sqlite = TestConstants.sqliteInMemory()
|
||||
val db = new SqliteNetworkDb(sqlite)
|
||||
val sig = Crypto.encodeSignature(Crypto.sign(randomBytes32, randomKey)) :+ 1.toByte
|
||||
val priv = randomKey
|
||||
|
@ -128,7 +124,7 @@ class SqliteNetworkDbSpec extends FunSuite {
|
|||
}
|
||||
|
||||
test("prune many channels") {
|
||||
val sqlite = inmem
|
||||
val sqlite = TestConstants.sqliteInMemory()
|
||||
val db = new SqliteNetworkDb(sqlite)
|
||||
|
||||
db.addToPruned(shortChannelIds)
|
||||
|
|
|
@ -16,34 +16,172 @@
|
|||
|
||||
package fr.acinq.eclair.db
|
||||
|
||||
import java.sql.DriverManager
|
||||
import java.time.Instant
|
||||
import java.util.UUID
|
||||
|
||||
import fr.acinq.bitcoin.ByteVector32
|
||||
import fr.acinq.eclair.db.sqlite.SqliteUtils._
|
||||
import fr.acinq.bitcoin.{Block, ByteVector32, MilliSatoshi}
|
||||
import fr.acinq.eclair.TestConstants.Bob
|
||||
import fr.acinq.eclair.{TestConstants, payment}
|
||||
import fr.acinq.eclair.db.sqlite.SqlitePaymentsDb
|
||||
import fr.acinq.eclair.payment.PaymentRequest
|
||||
import org.scalatest.FunSuite
|
||||
import scodec.bits._
|
||||
import fr.acinq.eclair.randomBytes32
|
||||
|
||||
import scala.compat.Platform
|
||||
import OutgoingPaymentStatus._
|
||||
|
||||
class SqlitePaymentsDbSpec extends FunSuite {
|
||||
|
||||
def inmem = DriverManager.getConnection("jdbc:sqlite::memory:")
|
||||
|
||||
test("init sqlite 2 times in a row") {
|
||||
val sqlite = inmem
|
||||
val sqlite = TestConstants.sqliteInMemory()
|
||||
val db1 = new SqlitePaymentsDb(sqlite)
|
||||
val db2 = new SqlitePaymentsDb(sqlite)
|
||||
}
|
||||
|
||||
test("add/list payments/find 1 payment that exists/find 1 payment that does not exist") {
|
||||
val sqlite = inmem
|
||||
test("handle version migration 1->2") {
|
||||
|
||||
val connection = TestConstants.sqliteInMemory()
|
||||
|
||||
using(connection.createStatement()) { statement =>
|
||||
getVersion(statement, "payments", 1)
|
||||
statement.executeUpdate("CREATE TABLE IF NOT EXISTS payments (payment_hash BLOB NOT NULL PRIMARY KEY, amount_msat INTEGER NOT NULL, timestamp INTEGER NOT NULL)")
|
||||
}
|
||||
|
||||
using(connection.createStatement()) { statement =>
|
||||
assert(getVersion(statement, "payments", 1) == 1) // version 1 is deployed now
|
||||
}
|
||||
|
||||
val oldReceivedPayment = IncomingPayment(ByteVector32(hex"0f059ef9b55bb70cc09069ee4df854bf0fab650eee6f2b87ba26d1ad08ab114f"), 123, 1233322)
|
||||
|
||||
// insert old type record
|
||||
using(connection.prepareStatement("INSERT INTO payments VALUES (?, ?, ?)")) { statement =>
|
||||
statement.setBytes(1, oldReceivedPayment.paymentHash.toArray)
|
||||
statement.setLong(2, oldReceivedPayment.amountMsat)
|
||||
statement.setLong(3, oldReceivedPayment.receivedAt)
|
||||
statement.executeUpdate()
|
||||
}
|
||||
|
||||
val preMigrationDb = new SqlitePaymentsDb(connection)
|
||||
|
||||
using(connection.createStatement()) { statement =>
|
||||
assert(getVersion(statement, "payments", 1) == 2) // version has changed from 1 to 2!
|
||||
}
|
||||
|
||||
// the existing received payment can NOT be queried anymore
|
||||
assert(preMigrationDb.getIncomingPayment(oldReceivedPayment.paymentHash).isEmpty)
|
||||
|
||||
// add a few rows
|
||||
val ps1 = OutgoingPayment(id = UUID.randomUUID(), paymentHash = ByteVector32(hex"0f059ef9b55bb70cc09069ee4df854bf0fab650eee6f2b87ba26d1ad08ab114f"), None, amountMsat = 12345, createdAt = 12345, None, PENDING)
|
||||
val i1 = PaymentRequest.read("lnbc10u1pw2t4phpp5ezwm2gdccydhnphfyepklc0wjkxhz0r4tctg9paunh2lxgeqhcmsdqlxycrqvpqwdshgueqvfjhggr0dcsry7qcqzpgfa4ecv7447p9t5hkujy9qgrxvkkf396p9zar9p87rv2htmeuunkhydl40r64n5s2k0u7uelzc8twxmp37nkcch6m0wg5tvvx69yjz8qpk94qf3")
|
||||
val pr1 = IncomingPayment(i1.paymentHash, 12345678, 1513871928275L)
|
||||
|
||||
preMigrationDb.addPaymentRequest(i1, ByteVector32.Zeroes)
|
||||
preMigrationDb.addIncomingPayment(pr1)
|
||||
preMigrationDb.addOutgoingPayment(ps1)
|
||||
|
||||
assert(preMigrationDb.listIncomingPayments() == Seq(pr1))
|
||||
assert(preMigrationDb.listOutgoingPayments() == Seq(ps1))
|
||||
assert(preMigrationDb.listPaymentRequests(0, Long.MaxValue) == Seq(i1))
|
||||
|
||||
val postMigrationDb = new SqlitePaymentsDb(connection)
|
||||
|
||||
using(connection.createStatement()) { statement =>
|
||||
assert(getVersion(statement, "payments", 2) == 2) // version still to 2
|
||||
}
|
||||
|
||||
assert(postMigrationDb.listIncomingPayments() == Seq(pr1))
|
||||
assert(postMigrationDb.listOutgoingPayments() == Seq(ps1))
|
||||
assert(preMigrationDb.listPaymentRequests(0, Long.MaxValue) == Seq(i1))
|
||||
}
|
||||
|
||||
test("add/list received payments/find 1 payment that exists/find 1 payment that does not exist") {
|
||||
val sqlite = TestConstants.sqliteInMemory()
|
||||
val db = new SqlitePaymentsDb(sqlite)
|
||||
|
||||
val p1 = Payment(ByteVector32(hex"08d47d5f7164d4b696e8f6b62a03094d4f1c65f16e9d7b11c4a98854707e55cf"), 12345678, 1513871928275L)
|
||||
val p2 = Payment(ByteVector32(hex"0f059ef9b55bb70cc09069ee4df854bf0fab650eee6f2b87ba26d1ad08ab114f"), 12345678, 1513871928275L)
|
||||
assert(db.listPayments() === Nil)
|
||||
db.addPayment(p1)
|
||||
db.addPayment(p2)
|
||||
assert(db.listPayments().toList === List(p1, p2))
|
||||
assert(db.findByPaymentHash(p1.payment_hash) === Some(p1))
|
||||
assert(db.findByPaymentHash(ByteVector32(hex"6e7e8018f05e169cf1d99e77dc22cb372d09f10b6a81f1eae410718c56cad187")) === None)
|
||||
// can't receive a payment without an invoice associated with it
|
||||
assertThrows[IllegalArgumentException](db.addIncomingPayment(IncomingPayment(ByteVector32(hex"6e7e8018f05e169cf1d99e77dc22cb372d09f10b6a81f1eae410718c56cad188"), 12345678, 1513871928275L)))
|
||||
|
||||
val i1 = PaymentRequest.read("lnbc5450n1pw2t4qdpp5vcrf6ylgpettyng4ac3vujsk0zpc25cj0q3zp7l7w44zvxmpzh8qdzz2pshjmt9de6zqen0wgsr2dp4ypcxj7r9d3ejqct5ypekzar0wd5xjuewwpkxzcm99cxqzjccqp2rzjqtspxelp67qc5l56p6999wkatsexzhs826xmupyhk6j8lxl038t27z9tsqqqgpgqqqqqqqlgqqqqqzsqpcz8z8hmy8g3ecunle4n3edn3zg2rly8g4klsk5md736vaqqy3ktxs30ht34rkfkqaffzxmjphvd0637dk2lp6skah2hq09z6lrjna3xqp3d4vyd")
|
||||
val i2 = PaymentRequest.read("lnbc10u1pw2t4phpp5ezwm2gdccydhnphfyepklc0wjkxhz0r4tctg9paunh2lxgeqhcmsdqlxycrqvpqwdshgueqvfjhggr0dcsry7qcqzpgfa4ecv7447p9t5hkujy9qgrxvkkf396p9zar9p87rv2htmeuunkhydl40r64n5s2k0u7uelzc8twxmp37nkcch6m0wg5tvvx69yjz8qpk94qf3")
|
||||
|
||||
db.addPaymentRequest(i1, ByteVector32.Zeroes)
|
||||
db.addPaymentRequest(i2, ByteVector32.Zeroes)
|
||||
|
||||
val p1 = IncomingPayment(i1.paymentHash, 12345678, 1513871928275L)
|
||||
val p2 = IncomingPayment(i2.paymentHash, 12345678, 1513871928275L)
|
||||
assert(db.listIncomingPayments() === Nil)
|
||||
db.addIncomingPayment(p1)
|
||||
db.addIncomingPayment(p2)
|
||||
assert(db.listIncomingPayments().toList === List(p1, p2))
|
||||
assert(db.getIncomingPayment(p1.paymentHash) === Some(p1))
|
||||
assert(db.getIncomingPayment(ByteVector32(hex"6e7e8018f05e169cf1d99e77dc22cb372d09f10b6a81f1eae410718c56cad187")) === None)
|
||||
}
|
||||
|
||||
test("add/retrieve/update sent payments") {
|
||||
|
||||
val db = new SqlitePaymentsDb(TestConstants.sqliteInMemory())
|
||||
|
||||
val s1 = OutgoingPayment(id = UUID.randomUUID(), paymentHash = ByteVector32(hex"0f059ef9b55bb70cc09069ee4df854bf0fab650eee6f2b87ba26d1ad08ab114f"), None, amountMsat = 12345, createdAt = 12345, None, PENDING)
|
||||
val s2 = OutgoingPayment(id = UUID.randomUUID(), paymentHash = ByteVector32(hex"08d47d5f7164d4b696e8f6b62a03094d4f1c65f16e9d7b11c4a98854707e55cf"), None, amountMsat = 12345, createdAt = 12345, None, PENDING)
|
||||
|
||||
assert(db.listOutgoingPayments().isEmpty)
|
||||
db.addOutgoingPayment(s1)
|
||||
db.addOutgoingPayment(s2)
|
||||
|
||||
assert(db.listOutgoingPayments().toList == Seq(s1, s2))
|
||||
assert(db.getOutgoingPayment(s1.id) === Some(s1))
|
||||
assert(db.getOutgoingPayment(s1.id).get.completedAt.isEmpty)
|
||||
assert(db.getOutgoingPayment(UUID.randomUUID()) === None)
|
||||
assert(db.getOutgoingPayments(s2.paymentHash) === Seq(s2))
|
||||
assert(db.getOutgoingPayments(ByteVector32.Zeroes) === Seq.empty)
|
||||
|
||||
val s3 = s2.copy(id = UUID.randomUUID(), amountMsat = 88776655)
|
||||
db.addOutgoingPayment(s3)
|
||||
|
||||
db.updateOutgoingPayment(s3.id, FAILED)
|
||||
assert(db.getOutgoingPayment(s3.id).get.status == FAILED)
|
||||
assert(db.getOutgoingPayment(s3.id).get.preimage.isEmpty) // failed sent payments don't have a preimage
|
||||
assert(db.getOutgoingPayment(s3.id).get.completedAt.isDefined)
|
||||
|
||||
// can't update again once it's in a final state
|
||||
assertThrows[IllegalArgumentException](db.updateOutgoingPayment(s3.id, SUCCEEDED))
|
||||
|
||||
db.updateOutgoingPayment(s1.id, SUCCEEDED, Some(ByteVector32.One))
|
||||
assert(db.getOutgoingPayment(s1.id).get.preimage.isDefined)
|
||||
assert(db.getOutgoingPayment(s1.id).get.completedAt.isDefined)
|
||||
}
|
||||
|
||||
test("add/retrieve payment requests") {
|
||||
|
||||
val someTimestamp = 12345
|
||||
val db = new SqlitePaymentsDb(TestConstants.sqliteInMemory())
|
||||
|
||||
val bob = Bob.keyManager
|
||||
|
||||
val (paymentHash1, paymentHash2) = (randomBytes32, randomBytes32)
|
||||
|
||||
val i1 = PaymentRequest(chainHash = Block.TestnetGenesisBlock.hash, amount = Some(MilliSatoshi(123)), paymentHash = paymentHash1, privateKey = bob.nodeKey.privateKey, description = "Some invoice", expirySeconds = None, timestamp = someTimestamp)
|
||||
val i2 = PaymentRequest(chainHash = Block.TestnetGenesisBlock.hash, amount = None, paymentHash = paymentHash2, privateKey = bob.nodeKey.privateKey, description = "Some invoice", expirySeconds = Some(123456), timestamp = Instant.now().getEpochSecond)
|
||||
|
||||
// i2 doesn't expire
|
||||
assert(i1.expiry.isEmpty && i2.expiry.isDefined)
|
||||
assert(i1.amount.isDefined && i2.amount.isEmpty)
|
||||
|
||||
db.addPaymentRequest(i1, ByteVector32.Zeroes)
|
||||
db.addPaymentRequest(i2, ByteVector32.One)
|
||||
|
||||
// order matters, i2 has a more recent timestamp than i1
|
||||
assert(db.listPaymentRequests(0, Long.MaxValue) == Seq(i2, i1))
|
||||
assert(db.getPaymentRequest(i1.paymentHash) == Some(i1))
|
||||
assert(db.getPaymentRequest(i2.paymentHash) == Some(i2))
|
||||
|
||||
assert(db.listPendingPaymentRequests(0, Long.MaxValue) == Seq(i2, i1))
|
||||
assert(db.getPendingPaymentRequestAndPreimage(paymentHash1) == Some((ByteVector32.Zeroes, i1)))
|
||||
assert(db.getPendingPaymentRequestAndPreimage(paymentHash2) == Some((ByteVector32.One, i2)))
|
||||
|
||||
assert(db.listPaymentRequests(someTimestamp - 100, someTimestamp + 100) == Seq(i1))
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -16,27 +16,23 @@
|
|||
|
||||
package fr.acinq.eclair.db
|
||||
|
||||
import java.sql.DriverManager
|
||||
|
||||
import fr.acinq.eclair.channel.{CMD_FAIL_HTLC, CMD_FAIL_MALFORMED_HTLC, CMD_FULFILL_HTLC}
|
||||
import fr.acinq.eclair.db.sqlite.SqlitePendingRelayDb
|
||||
import fr.acinq.eclair.randomBytes32
|
||||
import fr.acinq.eclair.{TestConstants, randomBytes32}
|
||||
import fr.acinq.eclair.wire.FailureMessageCodecs
|
||||
import org.scalatest.FunSuite
|
||||
|
||||
|
||||
class SqlitePendingRelayDbSpec extends FunSuite {
|
||||
|
||||
def inmem = DriverManager.getConnection("jdbc:sqlite::memory:")
|
||||
|
||||
test("init sqlite 2 times in a row") {
|
||||
val sqlite = inmem
|
||||
val sqlite = TestConstants.sqliteInMemory()
|
||||
val db1 = new SqlitePendingRelayDb(sqlite)
|
||||
val db2 = new SqlitePendingRelayDb(sqlite)
|
||||
}
|
||||
|
||||
test("add/remove/list messages") {
|
||||
val sqlite = inmem
|
||||
val sqlite = TestConstants.sqliteInMemory()
|
||||
val db = new SqlitePendingRelayDb(sqlite)
|
||||
|
||||
val channelId1 = randomBytes32
|
||||
|
|
|
@ -17,7 +17,7 @@
|
|||
package fr.acinq.eclair.integration
|
||||
|
||||
import java.io.{File, PrintWriter}
|
||||
import java.util.Properties
|
||||
import java.util.{Properties, UUID}
|
||||
|
||||
import akka.actor.{ActorRef, ActorSystem}
|
||||
import akka.testkit.{TestKit, TestProbe}
|
||||
|
@ -262,7 +262,7 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
|
|||
// then we make the actual payment
|
||||
sender.send(nodes("A").paymentInitiator,
|
||||
SendPayment(amountMsat.amount, pr.paymentHash, nodes("D").nodeParams.nodeId, routeParams = integrationTestRouteParams, maxAttempts = 1))
|
||||
sender.expectMsgType[PaymentSucceeded]
|
||||
sender.expectMsgType[UUID]
|
||||
}
|
||||
|
||||
test("send an HTLC A->D with an invalid expiry delta for B") {
|
||||
|
@ -286,7 +286,9 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
|
|||
val sendReq = SendPayment(amountMsat.amount, pr.paymentHash, nodes("D").nodeParams.nodeId, routeParams = integrationTestRouteParams, maxAttempts = 5)
|
||||
sender.send(nodes("A").paymentInitiator, sendReq)
|
||||
// A will receive an error from B that include the updated channel update, then will retry the payment
|
||||
sender.expectMsgType[PaymentSucceeded](5 seconds)
|
||||
val paymentId = sender.expectMsgType[UUID](5 seconds)
|
||||
val ps = sender.expectMsgType[PaymentSucceeded](5 seconds)
|
||||
assert(ps.id == paymentId)
|
||||
|
||||
awaitCond({
|
||||
// in the meantime, the router will have updated its state
|
||||
|
@ -322,7 +324,7 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
|
|||
val sendReq = SendPayment(amountMsat.amount, pr.paymentHash, nodes("D").nodeParams.nodeId, routeParams = integrationTestRouteParams, maxAttempts = 5)
|
||||
sender.send(nodes("A").paymentInitiator, sendReq)
|
||||
// A will first receive an error from C, then retry and route around C: A->B->E->C->D
|
||||
sender.expectMsgType[PaymentSucceeded](5 seconds)
|
||||
sender.expectMsgType[UUID](5 seconds)
|
||||
}
|
||||
|
||||
test("send an HTLC A->D with an unknown payment hash") {
|
||||
|
@ -331,7 +333,9 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
|
|||
sender.send(nodes("A").paymentInitiator, pr)
|
||||
|
||||
// A will receive an error from D and won't retry
|
||||
val paymentId = sender.expectMsgType[UUID]
|
||||
val failed = sender.expectMsgType[PaymentFailed]
|
||||
assert(failed.id == paymentId)
|
||||
assert(failed.paymentHash === pr.paymentHash)
|
||||
assert(failed.failures.size === 1)
|
||||
assert(failed.failures.head.asInstanceOf[RemoteFailure].e === ErrorPacket(nodes("D").nodeParams.nodeId, UnknownPaymentHash))
|
||||
|
@ -349,7 +353,9 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
|
|||
sender.send(nodes("A").paymentInitiator, sendReq)
|
||||
|
||||
// A will first receive an IncorrectPaymentAmount error from D
|
||||
val paymentId = sender.expectMsgType[UUID]
|
||||
val failed = sender.expectMsgType[PaymentFailed]
|
||||
assert(failed.id == paymentId)
|
||||
assert(failed.paymentHash === pr.paymentHash)
|
||||
assert(failed.failures.size === 1)
|
||||
assert(failed.failures.head.asInstanceOf[RemoteFailure].e === ErrorPacket(nodes("D").nodeParams.nodeId, IncorrectPaymentAmount))
|
||||
|
@ -367,7 +373,9 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
|
|||
sender.send(nodes("A").paymentInitiator, sendReq)
|
||||
|
||||
// A will first receive an IncorrectPaymentAmount error from D
|
||||
val paymentId = sender.expectMsgType[UUID]
|
||||
val failed = sender.expectMsgType[PaymentFailed]
|
||||
assert(paymentId == failed.id)
|
||||
assert(failed.paymentHash === pr.paymentHash)
|
||||
assert(failed.failures.size === 1)
|
||||
assert(failed.failures.head.asInstanceOf[RemoteFailure].e === ErrorPacket(nodes("D").nodeParams.nodeId, IncorrectPaymentAmount))
|
||||
|
@ -383,7 +391,7 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
|
|||
// A send payment of 3 mBTC, more than asked but it should still be accepted
|
||||
val sendReq = SendPayment(300000000L, pr.paymentHash, nodes("D").nodeParams.nodeId, routeParams = integrationTestRouteParams, maxAttempts = 5)
|
||||
sender.send(nodes("A").paymentInitiator, sendReq)
|
||||
sender.expectMsgType[PaymentSucceeded]
|
||||
sender.expectMsgType[UUID]
|
||||
}
|
||||
|
||||
test("send multiple HTLCs A->D with a failover when a channel gets exhausted") {
|
||||
|
@ -396,7 +404,8 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
|
|||
|
||||
val sendReq = SendPayment(amountMsat.amount, pr.paymentHash, nodes("D").nodeParams.nodeId, routeParams = integrationTestRouteParams, maxAttempts = 5)
|
||||
sender.send(nodes("A").paymentInitiator, sendReq)
|
||||
sender.expectMsgType[PaymentSucceeded]
|
||||
sender.expectMsgType[UUID]
|
||||
sender.expectMsgType[PaymentSucceeded] // the payment FSM will also reply to the sender after the payment is completed
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -411,10 +420,11 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
|
|||
sender.send(nodes("A").paymentInitiator,
|
||||
SendPayment(amountMsat.amount, pr.paymentHash, nodes("C").nodeParams.nodeId, maxAttempts = 1, routeParams = integrationTestRouteParams.map(_.copy(ratios = Some(WeightRatios(0, 0, 1))))))
|
||||
|
||||
sender.expectMsgType[UUID](max = 60 seconds)
|
||||
awaitCond({
|
||||
sender.expectMsgType[PaymentResult](10 seconds) match {
|
||||
case PaymentFailed(_, failures) => failures == Seq.empty // if something went wrong fail with a hint
|
||||
case PaymentSucceeded(_, _, _, route) => route.exists(_.nodeId == nodes("G").nodeParams.nodeId)
|
||||
case PaymentFailed(_, _, failures) => failures == Seq.empty // if something went wrong fail with a hint
|
||||
case PaymentSucceeded(_, _, _, _, route) => route.exists(_.nodeId == nodes("G").nodeParams.nodeId)
|
||||
}
|
||||
}, max = 30 seconds, interval = 10 seconds)
|
||||
}
|
||||
|
@ -455,6 +465,7 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
|
|||
val paymentReq = SendPayment(100000000L, paymentHash, nodes("F1").nodeParams.nodeId, maxAttempts = 1, routeParams = integrationTestRouteParams)
|
||||
val paymentSender = TestProbe()
|
||||
paymentSender.send(nodes("A").paymentInitiator, paymentReq)
|
||||
paymentSender.expectMsgType[UUID](30 seconds)
|
||||
// F gets the htlc
|
||||
val htlc = htlcReceiver.expectMsgType[UpdateAddHtlc]
|
||||
// now that we have the channel id, we retrieve channels default final addresses
|
||||
|
@ -534,6 +545,8 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
|
|||
val paymentReq = SendPayment(100000000L, paymentHash, nodes("F2").nodeParams.nodeId, maxAttempts = 1, routeParams = integrationTestRouteParams)
|
||||
val paymentSender = TestProbe()
|
||||
paymentSender.send(nodes("A").paymentInitiator, paymentReq)
|
||||
paymentSender.expectMsgType[UUID](30 seconds)
|
||||
|
||||
// F gets the htlc
|
||||
val htlc = htlcReceiver.expectMsgType[UpdateAddHtlc]
|
||||
// now that we have the channel id, we retrieve channels default final addresses
|
||||
|
@ -609,6 +622,7 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
|
|||
val paymentReq = SendPayment(100000000L, paymentHash, nodes("F3").nodeParams.nodeId, maxAttempts = 1, routeParams = integrationTestRouteParams)
|
||||
val paymentSender = TestProbe()
|
||||
paymentSender.send(nodes("A").paymentInitiator, paymentReq)
|
||||
val paymentId = paymentSender.expectMsgType[UUID]
|
||||
// F gets the htlc
|
||||
val htlc = htlcReceiver.expectMsgType[UpdateAddHtlc]
|
||||
// now that we have the channel id, we retrieve channels default final addresses
|
||||
|
@ -629,6 +643,7 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
|
|||
}, max = 30 seconds, interval = 1 second)
|
||||
// this will fail the htlc
|
||||
val failed = paymentSender.expectMsgType[PaymentFailed](30 seconds)
|
||||
assert(failed.id == paymentId)
|
||||
assert(failed.paymentHash === paymentHash)
|
||||
assert(failed.failures.size === 1)
|
||||
assert(failed.failures.head.asInstanceOf[RemoteFailure].e === ErrorPacket(nodes("C").nodeParams.nodeId, PermanentChannelFailure))
|
||||
|
@ -669,6 +684,8 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
|
|||
val paymentReq = SendPayment(100000000L, paymentHash, nodes("F4").nodeParams.nodeId, maxAttempts = 1, routeParams = integrationTestRouteParams)
|
||||
val paymentSender = TestProbe()
|
||||
paymentSender.send(nodes("A").paymentInitiator, paymentReq)
|
||||
val paymentId = paymentSender.expectMsgType[UUID](30 seconds)
|
||||
|
||||
// F gets the htlc
|
||||
val htlc = htlcReceiver.expectMsgType[UpdateAddHtlc]
|
||||
// now that we have the channel id, we retrieve channels default final addresses
|
||||
|
@ -692,6 +709,7 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
|
|||
}, max = 30 seconds, interval = 1 second)
|
||||
// this will fail the htlc
|
||||
val failed = paymentSender.expectMsgType[PaymentFailed](30 seconds)
|
||||
assert(failed.id == paymentId)
|
||||
assert(failed.paymentHash === paymentHash)
|
||||
assert(failed.failures.size === 1)
|
||||
assert(failed.failures.head.asInstanceOf[RemoteFailure].e === ErrorPacket(nodes("C").nodeParams.nodeId, PermanentChannelFailure))
|
||||
|
@ -739,12 +757,13 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
|
|||
val pr = sender.expectMsgType[PaymentRequest]
|
||||
val sendReq = SendPayment(300000000L, pr.paymentHash, pr.nodeId, routeParams = integrationTestRouteParams, maxAttempts = 1)
|
||||
sender.send(nodes("A").paymentInitiator, sendReq)
|
||||
val paymentId = sender.expectMsgType[UUID]
|
||||
// we forward the htlc to the payment handler
|
||||
forwardHandlerF.expectMsgType[UpdateAddHtlc]
|
||||
forwardHandlerF.forward(paymentHandlerF)
|
||||
sigListener.expectMsgType[ChannelSignatureReceived]
|
||||
sigListener.expectMsgType[ChannelSignatureReceived]
|
||||
sender.expectMsgType[PaymentSucceeded]
|
||||
sender.expectMsgType[PaymentSucceeded].id === paymentId
|
||||
|
||||
// we now send a few htlcs C->F and F->C in order to obtain a commitments with multiple htlcs
|
||||
def send(amountMsat: Long, paymentHandler: ActorRef, paymentInitiator: ActorRef) = {
|
||||
|
@ -752,7 +771,7 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
|
|||
val pr = sender.expectMsgType[PaymentRequest]
|
||||
val sendReq = SendPayment(amountMsat, pr.paymentHash, pr.nodeId, routeParams = integrationTestRouteParams, maxAttempts = 1)
|
||||
sender.send(paymentInitiator, sendReq)
|
||||
sender.expectNoMsg()
|
||||
sender.expectMsgType[UUID]
|
||||
}
|
||||
|
||||
val buffer = TestProbe()
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
package fr.acinq.eclair.interop.rustytests
|
||||
|
||||
import java.io.{BufferedWriter, File, FileWriter}
|
||||
import java.util.UUID
|
||||
import java.util.concurrent.CountDownLatch
|
||||
|
||||
import akka.actor.{Actor, ActorLogging, ActorRef, Stash}
|
||||
|
@ -56,7 +57,7 @@ class SynchronizationPipe(latch: CountDownLatch) extends Actor with ActorLogging
|
|||
|
||||
script match {
|
||||
case offer(x, amount, rhash) :: rest =>
|
||||
resolve(x) ! CMD_ADD_HTLC(amount.toInt, ByteVector32.fromValidHex(rhash), 144)
|
||||
resolve(x) ! CMD_ADD_HTLC(amount.toInt, ByteVector32.fromValidHex(rhash), 144, upstream = Left(UUID.randomUUID()))
|
||||
exec(rest, a, b)
|
||||
case fulfill(x, id, r) :: rest =>
|
||||
resolve(x) ! CMD_FULFILL_HTLC(id.toInt, ByteVector32.fromValidHex(r))
|
||||
|
|
|
@ -49,9 +49,9 @@ class ChannelSelectionSpec extends FunSuite {
|
|||
implicit val log = akka.event.NoLogging
|
||||
|
||||
// nominal case
|
||||
assert(Relayer.handleRelay(relayPayload, Some(channelUpdate)) === Right(CMD_ADD_HTLC(relayPayload.payload.amtToForward, relayPayload.add.paymentHash, relayPayload.payload.outgoingCltvValue, relayPayload.nextPacket.serialize, upstream_opt = Some(relayPayload.add), commit = true, redirected = false)))
|
||||
assert(Relayer.handleRelay(relayPayload, Some(channelUpdate)) === Right(CMD_ADD_HTLC(relayPayload.payload.amtToForward, relayPayload.add.paymentHash, relayPayload.payload.outgoingCltvValue, relayPayload.nextPacket.serialize, upstream = Right(relayPayload.add), commit = true, redirected = false)))
|
||||
// redirected to preferred channel
|
||||
assert(Relayer.handleRelay(relayPayload, Some(channelUpdate.copy(shortChannelId = ShortChannelId(1111)))) === Right(CMD_ADD_HTLC(relayPayload.payload.amtToForward, relayPayload.add.paymentHash, relayPayload.payload.outgoingCltvValue, relayPayload.nextPacket.serialize, upstream_opt = Some(relayPayload.add), commit = true, redirected = true)))
|
||||
assert(Relayer.handleRelay(relayPayload, Some(channelUpdate.copy(shortChannelId = ShortChannelId(1111)))) === Right(CMD_ADD_HTLC(relayPayload.payload.amtToForward, relayPayload.add.paymentHash, relayPayload.payload.outgoingCltvValue, relayPayload.nextPacket.serialize, upstream = Right(relayPayload.add), commit = true, redirected = true)))
|
||||
// no channel_update
|
||||
assert(Relayer.handleRelay(relayPayload, channelUpdate_opt = None) === Left(CMD_FAIL_HTLC(relayPayload.add.id, Right(UnknownNextPeer), commit = true)))
|
||||
// channel disabled
|
||||
|
@ -68,7 +68,7 @@ class ChannelSelectionSpec extends FunSuite {
|
|||
assert(Relayer.handleRelay(relayPayload_insufficientfee, Some(channelUpdate)) === Left(CMD_FAIL_HTLC(relayPayload.add.id, Right(FeeInsufficient(relayPayload_insufficientfee.add.amountMsat, channelUpdate)), commit = true)))
|
||||
// note that a generous fee is ok!
|
||||
val relayPayload_highfee = relayPayload.copy(payload = relayPayload.payload.copy(amtToForward = 900000))
|
||||
assert(Relayer.handleRelay(relayPayload_highfee, Some(channelUpdate)) === Right(CMD_ADD_HTLC(relayPayload_highfee.payload.amtToForward, relayPayload_highfee.add.paymentHash, relayPayload_highfee.payload.outgoingCltvValue, relayPayload_highfee.nextPacket.serialize, upstream_opt = Some(relayPayload.add), commit = true, redirected = false)))
|
||||
assert(Relayer.handleRelay(relayPayload_highfee, Some(channelUpdate)) === Right(CMD_ADD_HTLC(relayPayload_highfee.payload.amtToForward, relayPayload_highfee.add.paymentHash, relayPayload_highfee.payload.outgoingCltvValue, relayPayload_highfee.nextPacket.serialize, upstream = Right(relayPayload.add), commit = true, redirected = false)))
|
||||
}
|
||||
|
||||
test("relay channel selection") {
|
||||
|
|
|
@ -16,6 +16,8 @@
|
|||
|
||||
package fr.acinq.eclair.payment
|
||||
|
||||
import java.util.UUID
|
||||
|
||||
import fr.acinq.bitcoin.DeterministicWallet.ExtendedPrivateKey
|
||||
import fr.acinq.bitcoin.{Block, Crypto, DeterministicWallet}
|
||||
import fr.acinq.eclair.channel.Channel
|
||||
|
@ -96,7 +98,7 @@ class HtlcGenerationSpec extends FunSuite {
|
|||
|
||||
test("build a command including the onion") {
|
||||
|
||||
val (add, _) = buildCommand(finalAmountMsat, finalExpiry, paymentHash, hops)
|
||||
val (add, _) = buildCommand(UUID.randomUUID, finalAmountMsat, finalExpiry, paymentHash, hops)
|
||||
|
||||
assert(add.amountMsat > finalAmountMsat)
|
||||
assert(add.cltvExpiry === finalExpiry + channelUpdate_de.cltvExpiryDelta + channelUpdate_cd.cltvExpiryDelta + channelUpdate_bc.cltvExpiryDelta)
|
||||
|
@ -130,7 +132,7 @@ class HtlcGenerationSpec extends FunSuite {
|
|||
}
|
||||
|
||||
test("build a command with no hops") {
|
||||
val (add, _) = buildCommand(finalAmountMsat, finalExpiry, paymentHash, hops.take(1))
|
||||
val (add, _) = buildCommand(UUID.randomUUID(), finalAmountMsat, finalExpiry, paymentHash, hops.take(1))
|
||||
|
||||
assert(add.amountMsat === finalAmountMsat)
|
||||
assert(add.cltvExpiry === finalExpiry)
|
||||
|
|
|
@ -22,8 +22,7 @@ import akka.testkit.{TestActorRef, TestKit, TestProbe}
|
|||
import fr.acinq.bitcoin.{ByteVector32, MilliSatoshi, Satoshi}
|
||||
import fr.acinq.eclair.TestConstants.Alice
|
||||
import fr.acinq.eclair.channel.{CMD_FAIL_HTLC, CMD_FULFILL_HTLC}
|
||||
import fr.acinq.eclair.payment.LocalPaymentHandler.PendingPaymentRequest
|
||||
import fr.acinq.eclair.payment.PaymentLifecycle.{CheckPayment, ReceivePayment}
|
||||
import fr.acinq.eclair.payment.PaymentLifecycle.{ReceivePayment}
|
||||
import fr.acinq.eclair.payment.PaymentRequest.ExtraHop
|
||||
import fr.acinq.eclair.wire.{FinalExpiryTooSoon, UnknownPaymentHash, UpdateAddHtlc}
|
||||
import fr.acinq.eclair.{Globals, ShortChannelId, randomKey}
|
||||
|
@ -51,65 +50,41 @@ class PaymentHandlerSpec extends TestKit(ActorSystem("test")) with FunSuiteLike
|
|||
{
|
||||
sender.send(handler, ReceivePayment(Some(amountMsat), "1 coffee"))
|
||||
val pr = sender.expectMsgType[PaymentRequest]
|
||||
sender.send(handler, CheckPayment(pr.paymentHash))
|
||||
assert(sender.expectMsgType[Boolean] === false)
|
||||
assert(nodeParams.db.payments.getIncomingPayment(pr.paymentHash).isEmpty)
|
||||
assert(nodeParams.db.payments.getPendingPaymentRequestAndPreimage(pr.paymentHash).isDefined)
|
||||
|
||||
val add = UpdateAddHtlc(ByteVector32(ByteVector.fill(32)(1)), 0, amountMsat.amount, pr.paymentHash, expiry, ByteVector.empty)
|
||||
sender.send(handler, add)
|
||||
sender.expectMsgType[CMD_FULFILL_HTLC]
|
||||
|
||||
val paymentRelayed = eventListener.expectMsgType[PaymentReceived]
|
||||
assert(paymentRelayed.copy(timestamp = 0) === PaymentReceived(amountMsat, add.paymentHash, add.channelId, timestamp = 0))
|
||||
sender.send(handler, CheckPayment(pr.paymentHash))
|
||||
assert(sender.expectMsgType[Boolean] === true)
|
||||
assert(nodeParams.db.payments.getIncomingPayment(pr.paymentHash).exists(_.paymentHash == pr.paymentHash))
|
||||
}
|
||||
|
||||
{
|
||||
sender.send(handler, ReceivePayment(Some(amountMsat), "another coffee"))
|
||||
val pr = sender.expectMsgType[PaymentRequest]
|
||||
sender.send(handler, CheckPayment(pr.paymentHash))
|
||||
assert(sender.expectMsgType[Boolean] === false)
|
||||
assert(nodeParams.db.payments.getIncomingPayment(pr.paymentHash).isEmpty)
|
||||
|
||||
val add = UpdateAddHtlc(ByteVector32(ByteVector.fill(32)(1)), 0, amountMsat.amount, pr.paymentHash, expiry, ByteVector.empty)
|
||||
sender.send(handler, add)
|
||||
sender.expectMsgType[CMD_FULFILL_HTLC]
|
||||
val paymentRelayed = eventListener.expectMsgType[PaymentReceived]
|
||||
assert(paymentRelayed.copy(timestamp = 0) === PaymentReceived(amountMsat, add.paymentHash, add.channelId, timestamp = 0))
|
||||
sender.send(handler, CheckPayment(pr.paymentHash))
|
||||
assert(sender.expectMsgType[Boolean] === true)
|
||||
assert(nodeParams.db.payments.getIncomingPayment(pr.paymentHash).exists(_.paymentHash == pr.paymentHash))
|
||||
}
|
||||
|
||||
{
|
||||
sender.send(handler, ReceivePayment(Some(amountMsat), "bad expiry"))
|
||||
val pr = sender.expectMsgType[PaymentRequest]
|
||||
sender.send(handler, CheckPayment(pr.paymentHash))
|
||||
assert(sender.expectMsgType[Boolean] === false)
|
||||
assert(nodeParams.db.payments.getIncomingPayment(pr.paymentHash).isEmpty)
|
||||
|
||||
val add = UpdateAddHtlc(ByteVector32(ByteVector.fill(32)(1)), 0, amountMsat.amount, pr.paymentHash, cltvExpiry = Globals.blockCount.get() + 3, ByteVector.empty)
|
||||
sender.send(handler, add)
|
||||
assert(sender.expectMsgType[CMD_FAIL_HTLC].reason == Right(FinalExpiryTooSoon))
|
||||
eventListener.expectNoMsg(300 milliseconds)
|
||||
sender.send(handler, CheckPayment(pr.paymentHash))
|
||||
assert(sender.expectMsgType[Boolean] === false)
|
||||
}
|
||||
{
|
||||
sender.send(handler, ReceivePayment(Some(amountMsat), "timeout expired", Some(1L)))
|
||||
//allow request to timeout
|
||||
Thread.sleep(1001)
|
||||
val pr = sender.expectMsgType[PaymentRequest]
|
||||
sender.send(handler, CheckPayment(pr.paymentHash))
|
||||
assert(sender.expectMsgType[Boolean] === false)
|
||||
val add = UpdateAddHtlc(ByteVector32(ByteVector.fill(32)(1)), 0, amountMsat.amount, pr.paymentHash, expiry, ByteVector.empty)
|
||||
sender.send(handler, add)
|
||||
assert(sender.expectMsgType[CMD_FAIL_HTLC].reason == Right(UnknownPaymentHash))
|
||||
// We chose UnknownPaymentHash on purpose. So if you have expired by 1 second or 1 hour you get the same error message.
|
||||
eventListener.expectNoMsg(300 milliseconds)
|
||||
sender.send(handler, CheckPayment(pr.paymentHash))
|
||||
assert(sender.expectMsgType[Boolean] === false)
|
||||
// make sure that the request is indeed pruned
|
||||
sender.send(handler, 'requests)
|
||||
sender.expectMsgType[Map[ByteVector, PendingPaymentRequest]].contains(pr.paymentHash)
|
||||
sender.send(handler, LocalPaymentHandler.PurgeExpiredRequests)
|
||||
awaitCond({
|
||||
sender.send(handler, 'requests)
|
||||
sender.expectMsgType[Map[ByteVector32, PendingPaymentRequest]].contains(pr.paymentHash) == false
|
||||
})
|
||||
assert(nodeParams.db.payments.getIncomingPayment(pr.paymentHash).isEmpty)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -141,21 +116,6 @@ class PaymentHandlerSpec extends TestKit(ActorSystem("test")) with FunSuiteLike
|
|||
assert(pr.amount.contains(MilliSatoshi(100000000L)) && pr.nodeId.toString == nodeParams.nodeId.toString)
|
||||
}
|
||||
|
||||
test("Payment request generation should fail when there are too many pending requests") {
|
||||
val nodeParams = Alice.nodeParams.copy(maxPendingPaymentRequests = 42)
|
||||
val handler = system.actorOf(LocalPaymentHandler.props(nodeParams))
|
||||
val sender = TestProbe()
|
||||
|
||||
for (i <- 0 to nodeParams.maxPendingPaymentRequests) {
|
||||
sender.send(handler, ReceivePayment(None, s"Request #$i"))
|
||||
sender.expectMsgType[PaymentRequest]
|
||||
}
|
||||
|
||||
// over limit
|
||||
sender.send(handler, ReceivePayment(None, "This one should fail"))
|
||||
assert(sender.expectMsgType[Status.Failure].cause.getMessage === s"too many pending payment requests (max=${nodeParams.maxPendingPaymentRequests})")
|
||||
}
|
||||
|
||||
test("Payment request generation should succeed when the amount is not set") {
|
||||
val handler = system.actorOf(LocalPaymentHandler.props(Alice.nodeParams))
|
||||
val sender = TestProbe()
|
||||
|
|
|
@ -16,6 +16,8 @@
|
|||
|
||||
package fr.acinq.eclair.payment
|
||||
|
||||
import java.util.UUID
|
||||
|
||||
import akka.actor.FSM.{CurrentState, SubscribeTransitionCallBack, Transition}
|
||||
import akka.actor.Status
|
||||
import akka.testkit.{TestFSMRef, TestProbe}
|
||||
|
@ -24,15 +26,16 @@ import fr.acinq.bitcoin.{Block, ByteVector32, MilliSatoshi, Satoshi, Transaction
|
|||
import fr.acinq.eclair.blockchain.{UtxoStatus, ValidateRequest, ValidateResult, WatchSpentBasic}
|
||||
import fr.acinq.eclair.channel.Register.ForwardShortId
|
||||
import fr.acinq.eclair.channel.{AddHtlcFailed, ChannelUnavailable}
|
||||
import fr.acinq.eclair.crypto.Sphinx
|
||||
import fr.acinq.eclair.crypto.{KeyManager, Sphinx}
|
||||
import fr.acinq.eclair.crypto.Sphinx.ErrorPacket
|
||||
import fr.acinq.eclair.db.OutgoingPaymentStatus
|
||||
import fr.acinq.eclair.io.Peer.PeerRoutingMessage
|
||||
import fr.acinq.eclair.payment.PaymentLifecycle._
|
||||
import fr.acinq.eclair.router.Announcements.{makeChannelUpdate, makeNodeAnnouncement}
|
||||
import fr.acinq.eclair.router._
|
||||
import fr.acinq.eclair.transactions.Scripts
|
||||
import fr.acinq.eclair.wire._
|
||||
import fr.acinq.eclair.{Globals, ShortChannelId, randomBytes32}
|
||||
import fr.acinq.eclair._
|
||||
|
||||
/**
|
||||
* Created by PM on 29/08/2016.
|
||||
|
@ -46,9 +49,13 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
val defaultAmountMsat = 142000000L
|
||||
val defaultPaymentHash = randomBytes32
|
||||
|
||||
|
||||
test("payment failed (route not found)") { fixture =>
|
||||
import fixture._
|
||||
val paymentFSM = system.actorOf(PaymentLifecycle.props(a, router, TestProbe().ref))
|
||||
val nodeParams = TestConstants.Alice.nodeParams
|
||||
val paymentDb = nodeParams.db.payments
|
||||
val id = UUID.randomUUID()
|
||||
val paymentFSM = TestFSMRef(new PaymentLifecycle(nodeParams, id, router, TestProbe().ref))
|
||||
val monitor = TestProbe()
|
||||
val sender = TestProbe()
|
||||
|
||||
|
@ -57,14 +64,19 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
|
||||
val request = SendPayment(defaultAmountMsat, defaultPaymentHash, f, maxAttempts = 5)
|
||||
sender.send(paymentFSM, request)
|
||||
awaitCond(paymentDb.getOutgoingPayment(id).exists(_.status == OutgoingPaymentStatus.PENDING))
|
||||
val Transition(_, WAITING_FOR_REQUEST, WAITING_FOR_ROUTE) = monitor.expectMsgClass(classOf[Transition[_]])
|
||||
|
||||
sender.expectMsg(PaymentFailed(request.paymentHash, LocalFailure(RouteNotFound) :: Nil))
|
||||
sender.expectMsg(PaymentFailed(id, request.paymentHash, LocalFailure(RouteNotFound) :: Nil))
|
||||
awaitCond(paymentDb.getOutgoingPayment(id).exists(_.status == OutgoingPaymentStatus.FAILED))
|
||||
}
|
||||
|
||||
test("payment failed (route too expensive)") { fixture =>
|
||||
import fixture._
|
||||
val paymentFSM = system.actorOf(PaymentLifecycle.props(a, router, TestProbe().ref))
|
||||
val nodeParams = TestConstants.Alice.nodeParams.copy(keyManager = testKeyManager)
|
||||
val paymentDb = nodeParams.db.payments
|
||||
val id = UUID.randomUUID()
|
||||
val paymentFSM = system.actorOf(PaymentLifecycle.props(nodeParams, id, router, TestProbe().ref))
|
||||
val monitor = TestProbe()
|
||||
val sender = TestProbe()
|
||||
|
||||
|
@ -76,13 +88,17 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
val Transition(_, WAITING_FOR_REQUEST, WAITING_FOR_ROUTE) = monitor.expectMsgClass(classOf[Transition[_]])
|
||||
|
||||
val Seq(LocalFailure(RouteNotFound)) = sender.expectMsgType[PaymentFailed].failures
|
||||
awaitCond(paymentDb.getOutgoingPayment(id).exists(_.status == OutgoingPaymentStatus.FAILED))
|
||||
}
|
||||
|
||||
test("payment failed (unparsable failure)") { fixture =>
|
||||
import fixture._
|
||||
val nodeParams = TestConstants.Alice.nodeParams.copy(keyManager = testKeyManager)
|
||||
val paymentDb = nodeParams.db.payments
|
||||
val relayer = TestProbe()
|
||||
val routerForwarder = TestProbe()
|
||||
val paymentFSM = TestFSMRef(new PaymentLifecycle(a, routerForwarder.ref, relayer.ref))
|
||||
val id = UUID.randomUUID()
|
||||
val paymentFSM = TestFSMRef(new PaymentLifecycle(nodeParams, id, routerForwarder.ref, relayer.ref))
|
||||
val monitor = TestProbe()
|
||||
val sender = TestProbe()
|
||||
|
||||
|
@ -91,7 +107,8 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
|
||||
val request = SendPayment(defaultAmountMsat, defaultPaymentHash, d, maxAttempts = 2)
|
||||
sender.send(paymentFSM, request)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && paymentDb.getOutgoingPayment(id).exists(_.status == OutgoingPaymentStatus.PENDING))
|
||||
|
||||
val WaitingForRoute(_, _, Nil) = paymentFSM.stateData
|
||||
routerForwarder.expectMsg(RouteRequest(a, d, defaultAmountMsat, ignoreNodes = Set.empty, ignoreChannels = Set.empty))
|
||||
routerForwarder.forward(router)
|
||||
|
@ -102,7 +119,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
sender.send(paymentFSM, UpdateFailHtlc(ByteVector32.Zeroes, 0, defaultPaymentHash)) // unparsable message
|
||||
|
||||
// then the payment lifecycle will ask for a new route excluding all intermediate nodes
|
||||
routerForwarder.expectMsg(RouteRequest(a, d, defaultAmountMsat, ignoreNodes = Set(c), ignoreChannels = Set.empty))
|
||||
routerForwarder.expectMsg(RouteRequest(nodeParams.nodeId, d, defaultAmountMsat, ignoreNodes = Set(c), ignoreChannels = Set.empty))
|
||||
|
||||
// let's simulate a response by the router with another route
|
||||
sender.send(paymentFSM, RouteResponse(hops, Set(c), Set.empty))
|
||||
|
@ -113,14 +130,18 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
sender.send(paymentFSM, UpdateFailHtlc(ByteVector32.Zeroes, 0, defaultPaymentHash)) // unparsable message
|
||||
|
||||
// we allow 2 tries, so we send a 2nd request to the router
|
||||
sender.expectMsg(PaymentFailed(request.paymentHash, UnreadableRemoteFailure(hops) :: UnreadableRemoteFailure(hops) :: Nil))
|
||||
sender.expectMsg(PaymentFailed(id, request.paymentHash, UnreadableRemoteFailure(hops) :: UnreadableRemoteFailure(hops) :: Nil))
|
||||
awaitCond(paymentDb.getOutgoingPayment(id).exists(_.status == OutgoingPaymentStatus.FAILED)) // after last attempt the payment is failed
|
||||
}
|
||||
|
||||
test("payment failed (local error)") { fixture =>
|
||||
import fixture._
|
||||
val nodeParams = TestConstants.Alice.nodeParams.copy(keyManager = testKeyManager)
|
||||
val paymentDb = nodeParams.db.payments
|
||||
val relayer = TestProbe()
|
||||
val routerForwarder = TestProbe()
|
||||
val paymentFSM = TestFSMRef(new PaymentLifecycle(a, routerForwarder.ref, relayer.ref))
|
||||
val id = UUID.randomUUID()
|
||||
val paymentFSM = TestFSMRef(new PaymentLifecycle(nodeParams, id, routerForwarder.ref, relayer.ref))
|
||||
val monitor = TestProbe()
|
||||
val sender = TestProbe()
|
||||
|
||||
|
@ -129,26 +150,30 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
|
||||
val request = SendPayment(defaultAmountMsat, defaultPaymentHash, d, maxAttempts = 2)
|
||||
sender.send(paymentFSM, request)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && paymentDb.getOutgoingPayment(id).exists(_.status == OutgoingPaymentStatus.PENDING))
|
||||
|
||||
val WaitingForRoute(_, _, Nil) = paymentFSM.stateData
|
||||
routerForwarder.expectMsg(RouteRequest(a, d, defaultAmountMsat, assistedRoutes = Nil, ignoreNodes = Set.empty, ignoreChannels = Set.empty))
|
||||
routerForwarder.expectMsg(RouteRequest(nodeParams.nodeId, d, defaultAmountMsat, assistedRoutes = Nil, ignoreNodes = Set.empty, ignoreChannels = Set.empty))
|
||||
routerForwarder.forward(router)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_PAYMENT_COMPLETE)
|
||||
val WaitingForComplete(_, _, cmd1, Nil, _, _, _, hops) = paymentFSM.stateData
|
||||
|
||||
relayer.expectMsg(ForwardShortId(channelId_ab, cmd1))
|
||||
sender.send(paymentFSM, Status.Failure(AddHtlcFailed(ByteVector32.Zeroes, request.paymentHash, ChannelUnavailable(ByteVector32.Zeroes), Local(Some(paymentFSM.underlying.self)), None, None)))
|
||||
sender.send(paymentFSM, Status.Failure(AddHtlcFailed(ByteVector32.Zeroes, request.paymentHash, ChannelUnavailable(ByteVector32.Zeroes), Local(id, Some(paymentFSM.underlying.self)), None, None)))
|
||||
|
||||
// then the payment lifecycle will ask for a new route excluding the channel
|
||||
routerForwarder.expectMsg(RouteRequest(a, d, defaultAmountMsat, assistedRoutes = Nil, ignoreNodes = Set.empty, ignoreChannels = Set(ChannelDesc(channelId_ab, a, b))))
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE)
|
||||
routerForwarder.expectMsg(RouteRequest(nodeParams.nodeId, d, defaultAmountMsat, assistedRoutes = Nil, ignoreNodes = Set.empty, ignoreChannels = Set(ChannelDesc(channelId_ab, a, b))))
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && paymentDb.getOutgoingPayment(id).exists(_.status == OutgoingPaymentStatus.PENDING)) // payment is still pending because the error is recoverable
|
||||
}
|
||||
|
||||
test("payment failed (first hop returns an UpdateFailMalformedHtlc)") { fixture =>
|
||||
import fixture._
|
||||
val nodeParams = TestConstants.Alice.nodeParams.copy(keyManager = testKeyManager)
|
||||
val paymentDb = nodeParams.db.payments
|
||||
val relayer = TestProbe()
|
||||
val routerForwarder = TestProbe()
|
||||
val paymentFSM = TestFSMRef(new PaymentLifecycle(a, routerForwarder.ref, relayer.ref))
|
||||
val id = UUID.randomUUID()
|
||||
val paymentFSM = TestFSMRef(new PaymentLifecycle(nodeParams, id, routerForwarder.ref, relayer.ref))
|
||||
val monitor = TestProbe()
|
||||
val sender = TestProbe()
|
||||
|
||||
|
@ -157,9 +182,10 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
|
||||
val request = SendPayment(defaultAmountMsat, defaultPaymentHash, d, maxAttempts = 2)
|
||||
sender.send(paymentFSM, request)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && paymentDb.getOutgoingPayment(id).exists(_.status == OutgoingPaymentStatus.PENDING))
|
||||
|
||||
val WaitingForRoute(_, _, Nil) = paymentFSM.stateData
|
||||
routerForwarder.expectMsg(RouteRequest(a, d, defaultAmountMsat, assistedRoutes = Nil, ignoreNodes = Set.empty, ignoreChannels = Set.empty))
|
||||
routerForwarder.expectMsg(RouteRequest(nodeParams.nodeId, d, defaultAmountMsat, assistedRoutes = Nil, ignoreNodes = Set.empty, ignoreChannels = Set.empty))
|
||||
routerForwarder.forward(router)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_PAYMENT_COMPLETE)
|
||||
val WaitingForComplete(_, _, cmd1, Nil, _, _, _, hops) = paymentFSM.stateData
|
||||
|
@ -169,14 +195,16 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
|
||||
// then the payment lifecycle will ask for a new route excluding the channel
|
||||
routerForwarder.expectMsg(RouteRequest(a, d, defaultAmountMsat, assistedRoutes = Nil, ignoreNodes = Set.empty, ignoreChannels = Set(ChannelDesc(channelId_ab, a, b))))
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && paymentDb.getOutgoingPayment(id).exists(_.status == OutgoingPaymentStatus.PENDING))
|
||||
}
|
||||
|
||||
test("payment failed (TemporaryChannelFailure)") { fixture =>
|
||||
import fixture._
|
||||
val nodeParams = TestConstants.Alice.nodeParams.copy(keyManager = testKeyManager)
|
||||
val relayer = TestProbe()
|
||||
val routerForwarder = TestProbe()
|
||||
val paymentFSM = TestFSMRef(new PaymentLifecycle(a, routerForwarder.ref, relayer.ref))
|
||||
val id = UUID.randomUUID()
|
||||
val paymentFSM = TestFSMRef(new PaymentLifecycle(nodeParams, id, routerForwarder.ref, relayer.ref))
|
||||
val monitor = TestProbe()
|
||||
val sender = TestProbe()
|
||||
|
||||
|
@ -187,7 +215,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
sender.send(paymentFSM, request)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE)
|
||||
val WaitingForRoute(_, _, Nil) = paymentFSM.stateData
|
||||
routerForwarder.expectMsg(RouteRequest(a, d, defaultAmountMsat, assistedRoutes = Nil, ignoreNodes = Set.empty, ignoreChannels = Set.empty))
|
||||
routerForwarder.expectMsg(RouteRequest(nodeParams.nodeId, d, defaultAmountMsat, assistedRoutes = Nil, ignoreNodes = Set.empty, ignoreChannels = Set.empty))
|
||||
routerForwarder.forward(router)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_PAYMENT_COMPLETE)
|
||||
val WaitingForComplete(_, _, cmd1, Nil, sharedSecrets1, _, _, hops) = paymentFSM.stateData
|
||||
|
@ -206,14 +234,17 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
routerForwarder.expectMsg(RouteRequest(a, d, defaultAmountMsat, assistedRoutes = Nil, ignoreNodes = Set.empty, ignoreChannels = Set.empty))
|
||||
routerForwarder.forward(router)
|
||||
// we allow 2 tries, so we send a 2nd request to the router
|
||||
sender.expectMsg(PaymentFailed(request.paymentHash, RemoteFailure(hops, ErrorPacket(b, failure)) :: LocalFailure(RouteNotFound) :: Nil))
|
||||
sender.expectMsg(PaymentFailed(id, request.paymentHash, RemoteFailure(hops, ErrorPacket(b, failure)) :: LocalFailure(RouteNotFound) :: Nil))
|
||||
}
|
||||
|
||||
test("payment failed (Update)") { fixture =>
|
||||
import fixture._
|
||||
val nodeParams = TestConstants.Alice.nodeParams.copy(keyManager = testKeyManager)
|
||||
val paymentDb = nodeParams.db.payments
|
||||
val relayer = TestProbe()
|
||||
val routerForwarder = TestProbe()
|
||||
val paymentFSM = TestFSMRef(new PaymentLifecycle(a, routerForwarder.ref, relayer.ref))
|
||||
val id = UUID.randomUUID()
|
||||
val paymentFSM = TestFSMRef(new PaymentLifecycle(nodeParams, id, routerForwarder.ref, relayer.ref))
|
||||
val monitor = TestProbe()
|
||||
val sender = TestProbe()
|
||||
|
||||
|
@ -222,9 +253,10 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
|
||||
val request = SendPayment(defaultAmountMsat, defaultPaymentHash, d, maxAttempts = 5)
|
||||
sender.send(paymentFSM, request)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && paymentDb.getOutgoingPayment(id).exists(_.status == OutgoingPaymentStatus.PENDING))
|
||||
|
||||
val WaitingForRoute(_, _, Nil) = paymentFSM.stateData
|
||||
routerForwarder.expectMsg(RouteRequest(a, d, defaultAmountMsat, assistedRoutes = Nil, ignoreNodes = Set.empty, ignoreChannels = Set.empty))
|
||||
routerForwarder.expectMsg(RouteRequest(nodeParams.nodeId, d, defaultAmountMsat, assistedRoutes = Nil, ignoreNodes = Set.empty, ignoreChannels = Set.empty))
|
||||
routerForwarder.forward(router)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_PAYMENT_COMPLETE)
|
||||
val WaitingForComplete(_, _, cmd1, Nil, sharedSecrets1, _, _, hops) = paymentFSM.stateData
|
||||
|
@ -238,8 +270,8 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
|
||||
// payment lifecycle forwards the embedded channelUpdate to the router
|
||||
routerForwarder.expectMsg(channelUpdate_bc_modified)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE)
|
||||
routerForwarder.expectMsg(RouteRequest(a, d, defaultAmountMsat, assistedRoutes = Nil, ignoreNodes = Set.empty, ignoreChannels = Set.empty))
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && paymentDb.getOutgoingPayment(id).exists(_.status == OutgoingPaymentStatus.PENDING)) // 1 failure but not final, the payment is still PENDING
|
||||
routerForwarder.expectMsg(RouteRequest(nodeParams.nodeId, d, defaultAmountMsat, assistedRoutes = Nil, ignoreNodes = Set.empty, ignoreChannels = Set.empty))
|
||||
routerForwarder.forward(router)
|
||||
|
||||
// router answers with a new route, taking into account the new update
|
||||
|
@ -259,18 +291,22 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
// but it will still forward the embedded channelUpdate to the router
|
||||
routerForwarder.expectMsg(channelUpdate_bc_modified_2)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE)
|
||||
routerForwarder.expectMsg(RouteRequest(a, d, defaultAmountMsat, assistedRoutes = Nil, ignoreNodes = Set.empty, ignoreChannels = Set.empty))
|
||||
routerForwarder.expectMsg(RouteRequest(nodeParams.nodeId, d, defaultAmountMsat, assistedRoutes = Nil, ignoreNodes = Set.empty, ignoreChannels = Set.empty))
|
||||
routerForwarder.forward(router)
|
||||
|
||||
// this time the router can't find a route: game over
|
||||
sender.expectMsg(PaymentFailed(request.paymentHash, RemoteFailure(hops, ErrorPacket(b, failure)) :: RemoteFailure(hops2, ErrorPacket(b, failure2)) :: LocalFailure(RouteNotFound) :: Nil))
|
||||
sender.expectMsg(PaymentFailed(id, request.paymentHash, RemoteFailure(hops, ErrorPacket(b, failure)) :: RemoteFailure(hops2, ErrorPacket(b, failure2)) :: LocalFailure(RouteNotFound) :: Nil))
|
||||
awaitCond(paymentDb.getOutgoingPayment(id).exists(_.status == OutgoingPaymentStatus.FAILED))
|
||||
}
|
||||
|
||||
test("payment failed (PermanentChannelFailure)") { fixture =>
|
||||
import fixture._
|
||||
val nodeParams = TestConstants.Alice.nodeParams.copy(keyManager = testKeyManager)
|
||||
val paymentDb = nodeParams.db.payments
|
||||
val relayer = TestProbe()
|
||||
val routerForwarder = TestProbe()
|
||||
val paymentFSM = TestFSMRef(new PaymentLifecycle(a, routerForwarder.ref, relayer.ref))
|
||||
val id = UUID.randomUUID()
|
||||
val paymentFSM = TestFSMRef(new PaymentLifecycle(nodeParams, id, routerForwarder.ref, relayer.ref))
|
||||
val monitor = TestProbe()
|
||||
val sender = TestProbe()
|
||||
|
||||
|
@ -279,9 +315,10 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
|
||||
val request = SendPayment(defaultAmountMsat, defaultPaymentHash, d, maxAttempts = 2)
|
||||
sender.send(paymentFSM, request)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && paymentDb.getOutgoingPayment(id).exists(_.status == OutgoingPaymentStatus.PENDING))
|
||||
|
||||
val WaitingForRoute(_, _, Nil) = paymentFSM.stateData
|
||||
routerForwarder.expectMsg(RouteRequest(a, d, defaultAmountMsat, assistedRoutes = Nil, ignoreNodes = Set.empty, ignoreChannels = Set.empty))
|
||||
routerForwarder.expectMsg(RouteRequest(nodeParams.nodeId, d, defaultAmountMsat, assistedRoutes = Nil, ignoreNodes = Set.empty, ignoreChannels = Set.empty))
|
||||
routerForwarder.forward(router)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_PAYMENT_COMPLETE)
|
||||
val WaitingForComplete(_, _, cmd1, Nil, sharedSecrets1, _, _, hops) = paymentFSM.stateData
|
||||
|
@ -293,16 +330,20 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
|
||||
// payment lifecycle forwards the embedded channelUpdate to the router
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE)
|
||||
routerForwarder.expectMsg(RouteRequest(a, d, defaultAmountMsat, assistedRoutes = Nil, ignoreNodes = Set.empty, ignoreChannels = Set(ChannelDesc(channelId_bc, b, c))))
|
||||
routerForwarder.expectMsg(RouteRequest(nodeParams.nodeId, d, defaultAmountMsat, assistedRoutes = Nil, ignoreNodes = Set.empty, ignoreChannels = Set(ChannelDesc(channelId_bc, b, c))))
|
||||
routerForwarder.forward(router)
|
||||
// we allow 2 tries, so we send a 2nd request to the router, which won't find another route
|
||||
|
||||
sender.expectMsg(PaymentFailed(request.paymentHash, RemoteFailure(hops, ErrorPacket(b, failure)) :: LocalFailure(RouteNotFound) :: Nil))
|
||||
sender.expectMsg(PaymentFailed(id, request.paymentHash, RemoteFailure(hops, ErrorPacket(b, failure)) :: LocalFailure(RouteNotFound) :: Nil))
|
||||
awaitCond(paymentDb.getOutgoingPayment(id).exists(_.status == OutgoingPaymentStatus.FAILED))
|
||||
}
|
||||
|
||||
test("payment succeeded") { fixture =>
|
||||
import fixture._
|
||||
val paymentFSM = system.actorOf(PaymentLifecycle.props(a, router, TestProbe().ref))
|
||||
val nodeParams = TestConstants.Alice.nodeParams.copy(keyManager = testKeyManager)
|
||||
val paymentDb = nodeParams.db.payments
|
||||
val id = UUID.randomUUID()
|
||||
val paymentFSM = system.actorOf(PaymentLifecycle.props(nodeParams, id, router, TestProbe().ref))
|
||||
val monitor = TestProbe()
|
||||
val sender = TestProbe()
|
||||
val eventListener = TestProbe()
|
||||
|
@ -315,19 +356,20 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
sender.send(paymentFSM, request)
|
||||
val Transition(_, WAITING_FOR_REQUEST, WAITING_FOR_ROUTE) = monitor.expectMsgClass(classOf[Transition[_]])
|
||||
val Transition(_, WAITING_FOR_ROUTE, WAITING_FOR_PAYMENT_COMPLETE) = monitor.expectMsgClass(classOf[Transition[_]])
|
||||
|
||||
awaitCond(paymentDb.getOutgoingPayment(id).exists(_.status == OutgoingPaymentStatus.PENDING))
|
||||
sender.send(paymentFSM, UpdateFulfillHtlc(ByteVector32.Zeroes, 0, defaultPaymentHash))
|
||||
|
||||
val paymentOK = sender.expectMsgType[PaymentSucceeded]
|
||||
val PaymentSent(MilliSatoshi(request.amountMsat), fee, request.paymentHash, paymentOK.paymentPreimage, _, _) = eventListener.expectMsgType[PaymentSent]
|
||||
val PaymentSent(_, MilliSatoshi(request.amountMsat), fee, request.paymentHash, paymentOK.paymentPreimage, _, _) = eventListener.expectMsgType[PaymentSent]
|
||||
assert(fee > MilliSatoshi(0))
|
||||
assert(fee === MilliSatoshi(paymentOK.amountMsat - request.amountMsat))
|
||||
awaitCond(paymentDb.getOutgoingPayment(id).exists(_.status == OutgoingPaymentStatus.SUCCEEDED))
|
||||
}
|
||||
|
||||
test("payment succeeded to a channel with fees=0") { fixture =>
|
||||
import fixture._
|
||||
import fr.acinq.eclair.randomKey
|
||||
|
||||
val nodeParams = TestConstants.Alice.nodeParams.copy(keyManager = testKeyManager)
|
||||
// the network will be a --(1)--> b ---(2)--> c --(3)--> d and e --(4)--> f (we are a) and b -> g has fees=0
|
||||
// \
|
||||
// \--(5)--> g
|
||||
|
@ -349,7 +391,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
watcher.expectMsgType[WatchSpentBasic]
|
||||
|
||||
// actual test begins
|
||||
val paymentFSM = system.actorOf(PaymentLifecycle.props(a, router, TestProbe().ref))
|
||||
val paymentFSM = system.actorOf(PaymentLifecycle.props(nodeParams, UUID.randomUUID(), router, TestProbe().ref))
|
||||
val monitor = TestProbe()
|
||||
val sender = TestProbe()
|
||||
val eventListener = TestProbe()
|
||||
|
@ -369,7 +411,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
sender.send(paymentFSM, UpdateFulfillHtlc(ByteVector32.Zeroes, 0, defaultPaymentHash))
|
||||
|
||||
val paymentOK = sender.expectMsgType[PaymentSucceeded]
|
||||
val PaymentSent(MilliSatoshi(request.amountMsat), fee, request.paymentHash, paymentOK.paymentPreimage, _, _) = eventListener.expectMsgType[PaymentSent]
|
||||
val PaymentSent(_, MilliSatoshi(request.amountMsat), fee, request.paymentHash, paymentOK.paymentPreimage, _, _) = eventListener.expectMsgType[PaymentSent]
|
||||
|
||||
// during the route computation the fees were treated as if they were 1msat but when sending the onion we actually put zero
|
||||
// NB: A -> B doesn't pay fees because it's our direct neighbor
|
||||
|
@ -378,8 +420,8 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
assert(fee === MilliSatoshi(paymentOK.amountMsat - request.amountMsat))
|
||||
}
|
||||
|
||||
test("filter errors properly") { fixture =>
|
||||
val failures = LocalFailure(RouteNotFound) :: RemoteFailure(Hop(a, b, channelUpdate_ab) :: Nil, ErrorPacket(a, TemporaryNodeFailure)) :: LocalFailure(AddHtlcFailed(ByteVector32.Zeroes, ByteVector32.Zeroes, ChannelUnavailable(ByteVector32.Zeroes), Local(None), None, None)) :: LocalFailure(RouteNotFound) :: Nil
|
||||
test("filter errors properly") { _ =>
|
||||
val failures = LocalFailure(RouteNotFound) :: RemoteFailure(Hop(a, b, channelUpdate_ab) :: Nil, ErrorPacket(a, TemporaryNodeFailure)) :: LocalFailure(AddHtlcFailed(ByteVector32.Zeroes, ByteVector32.Zeroes, ChannelUnavailable(ByteVector32.Zeroes), Local(UUID.randomUUID(), None), None, None)) :: LocalFailure(RouteNotFound) :: Nil
|
||||
val filtered = PaymentLifecycle.transformForUser(failures)
|
||||
assert(filtered == LocalFailure(RouteNotFound) :: RemoteFailure(Hop(a, b, channelUpdate_ab) :: Nil, ErrorPacket(a, TemporaryNodeFailure)) :: LocalFailure(ChannelUnavailable(ByteVector32.Zeroes)) :: Nil)
|
||||
}
|
||||
|
|
|
@ -63,14 +63,14 @@ class PaymentRequestSpec extends FunSuite {
|
|||
assert(string2Bits("pz") === bin"0000100010")
|
||||
}
|
||||
|
||||
test("minimal length long") {
|
||||
test("minimal length long, left-padded to be multiple of 5") {
|
||||
import scodec.bits._
|
||||
assert(long2bits(0) == bin"")
|
||||
assert(long2bits(1) == bin"1")
|
||||
assert(long2bits(42) == bin"101010")
|
||||
assert(long2bits(255) == bin"11111111")
|
||||
assert(long2bits(256) == bin"100000000")
|
||||
assert(long2bits(3600) == bin"111000010000")
|
||||
assert(long2bits(1) == bin"00001")
|
||||
assert(long2bits(42) == bin"0000101010")
|
||||
assert(long2bits(255) == bin"0011111111")
|
||||
assert(long2bits(256) == bin"0100000000")
|
||||
assert(long2bits(3600) == bin"000111000010000")
|
||||
}
|
||||
|
||||
test("verify that padding is zero") {
|
||||
|
@ -218,13 +218,24 @@ class PaymentRequestSpec extends FunSuite {
|
|||
assert(PaymentRequest.write(pr.sign(priv)) == ref)
|
||||
}
|
||||
|
||||
test("expiry is a variable-length unsigned value") {
|
||||
val pr = PaymentRequest(Block.RegtestGenesisBlock.hash, Some(MilliSatoshi(100000L)), ByteVector32(hex"0001020304050607080900010203040506070809000102030405060708090102"),
|
||||
priv, "test", fallbackAddress = None, expirySeconds = Some(21600), timestamp = System.currentTimeMillis() / 1000L)
|
||||
test("correctly serialize/deserialize variable-length tagged fields") {
|
||||
val number = 123456
|
||||
|
||||
val serialized = PaymentRequest write pr
|
||||
val pr1 = PaymentRequest read serialized
|
||||
assert(pr.expiry === Some(21600))
|
||||
val codec = PaymentRequest.Codecs.dataCodec(scodec.codecs.bits).as[PaymentRequest.Expiry]
|
||||
val field = PaymentRequest.Expiry(number)
|
||||
|
||||
assert(field.toLong == number)
|
||||
|
||||
val serializedExpiry = codec.encode(field).require
|
||||
val field1 = codec.decodeValue(serializedExpiry).require
|
||||
assert(field1 == field)
|
||||
|
||||
// Now with a payment request
|
||||
val pr = PaymentRequest(chainHash = Block.LivenetGenesisBlock.hash, amount = Some(MilliSatoshi(123)), paymentHash = ByteVector32(ByteVector.fill(32)(1)), privateKey = priv, description = "Some invoice", expirySeconds = Some(123456), timestamp = 12345)
|
||||
|
||||
val serialized = PaymentRequest.write(pr)
|
||||
val pr1 = PaymentRequest.read(serialized)
|
||||
assert(pr == pr1)
|
||||
}
|
||||
|
||||
test("ignore unknown tags") {
|
||||
|
|
|
@ -16,6 +16,8 @@
|
|||
|
||||
package fr.acinq.eclair.payment
|
||||
|
||||
import java.util.UUID
|
||||
|
||||
import akka.actor.{ActorRef, Status}
|
||||
import akka.testkit.TestProbe
|
||||
import fr.acinq.bitcoin.{ByteVector32, Crypto, MilliSatoshi}
|
||||
|
@ -67,7 +69,7 @@ class RelayerSpec extends TestkitBaseClass {
|
|||
val sender = TestProbe()
|
||||
|
||||
// we use this to build a valid onion
|
||||
val (cmd, _) = buildCommand(finalAmountMsat, finalExpiry, paymentHash, hops)
|
||||
val (cmd, _) = buildCommand(UUID.randomUUID(), finalAmountMsat, finalExpiry, paymentHash, hops)
|
||||
// and then manually build an htlc
|
||||
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.cltvExpiry, cmd.onion)
|
||||
relayer ! LocalChannelUpdate(null, channelId_bc, channelUpdate_bc.shortChannelId, c, None, channelUpdate_bc, makeCommitments(channelId_bc))
|
||||
|
@ -76,7 +78,7 @@ class RelayerSpec extends TestkitBaseClass {
|
|||
|
||||
val fwd = register.expectMsgType[Register.ForwardShortId[CMD_ADD_HTLC]]
|
||||
assert(fwd.shortChannelId === channelUpdate_bc.shortChannelId)
|
||||
assert(fwd.message.upstream_opt === Some(add_ab))
|
||||
assert(fwd.message.upstream === Right(add_ab))
|
||||
|
||||
sender.expectNoMsg(100 millis)
|
||||
paymentHandler.expectNoMsg(100 millis)
|
||||
|
@ -87,7 +89,7 @@ class RelayerSpec extends TestkitBaseClass {
|
|||
val sender = TestProbe()
|
||||
|
||||
// we use this to build a valid onion
|
||||
val (cmd, _) = buildCommand(finalAmountMsat, finalExpiry, paymentHash, hops)
|
||||
val (cmd, _) = buildCommand(UUID.randomUUID(), finalAmountMsat, finalExpiry, paymentHash, hops)
|
||||
// and then manually build an htlc
|
||||
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.cltvExpiry, cmd.onion)
|
||||
|
||||
|
@ -107,7 +109,7 @@ class RelayerSpec extends TestkitBaseClass {
|
|||
val sender = TestProbe()
|
||||
|
||||
// we use this to build a valid onion
|
||||
val (cmd, _) = buildCommand(finalAmountMsat, finalExpiry, paymentHash, hops)
|
||||
val (cmd, _) = buildCommand(UUID.randomUUID(), finalAmountMsat, finalExpiry, paymentHash, hops)
|
||||
// and then manually build an htlc
|
||||
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.cltvExpiry, cmd.onion)
|
||||
relayer ! LocalChannelUpdate(null, channelId_bc, channelUpdate_bc.shortChannelId, c, None, channelUpdate_bc, makeCommitments(channelId_bc))
|
||||
|
@ -116,7 +118,7 @@ class RelayerSpec extends TestkitBaseClass {
|
|||
|
||||
val fwd1 = register.expectMsgType[Register.ForwardShortId[CMD_ADD_HTLC]]
|
||||
assert(fwd1.shortChannelId === channelUpdate_bc.shortChannelId)
|
||||
assert(fwd1.message.upstream_opt === Some(add_ab))
|
||||
assert(fwd1.message.upstream === Right(add_ab))
|
||||
|
||||
sender.send(relayer, Status.Failure(Register.ForwardShortIdFailure(fwd1)))
|
||||
|
||||
|
@ -134,7 +136,7 @@ class RelayerSpec extends TestkitBaseClass {
|
|||
val sender = TestProbe()
|
||||
|
||||
// check that payments are sent properly
|
||||
val (cmd, _) = buildCommand(finalAmountMsat, finalExpiry, paymentHash, hops)
|
||||
val (cmd, _) = buildCommand(UUID.randomUUID(), finalAmountMsat, finalExpiry, paymentHash, hops)
|
||||
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.cltvExpiry, cmd.onion)
|
||||
relayer ! LocalChannelUpdate(null, channelId_bc, channelUpdate_bc.shortChannelId, c, None, channelUpdate_bc, makeCommitments(channelId_bc))
|
||||
|
||||
|
@ -142,7 +144,7 @@ class RelayerSpec extends TestkitBaseClass {
|
|||
|
||||
val fwd = register.expectMsgType[Register.ForwardShortId[CMD_ADD_HTLC]]
|
||||
assert(fwd.shortChannelId === channelUpdate_bc.shortChannelId)
|
||||
assert(fwd.message.upstream_opt === Some(add_ab))
|
||||
assert(fwd.message.upstream === Right(add_ab))
|
||||
|
||||
sender.expectNoMsg(100 millis)
|
||||
paymentHandler.expectNoMsg(100 millis)
|
||||
|
@ -150,7 +152,7 @@ class RelayerSpec extends TestkitBaseClass {
|
|||
// now tell the relayer that the channel is down and try again
|
||||
relayer ! LocalChannelDown(sender.ref, channelId = channelId_bc, shortChannelId = channelUpdate_bc.shortChannelId, remoteNodeId = TestConstants.Bob.nodeParams.nodeId)
|
||||
|
||||
val (cmd1, _) = buildCommand(finalAmountMsat, finalExpiry, randomBytes32, hops)
|
||||
val (cmd1, _) = buildCommand(UUID.randomUUID(), finalAmountMsat, finalExpiry, randomBytes32, hops)
|
||||
val add_ab1 = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd1.amountMsat, cmd1.paymentHash, cmd1.cltvExpiry, cmd1.onion)
|
||||
sender.send(relayer, ForwardAdd(add_ab))
|
||||
|
||||
|
@ -167,7 +169,7 @@ class RelayerSpec extends TestkitBaseClass {
|
|||
val sender = TestProbe()
|
||||
|
||||
// we use this to build a valid onion
|
||||
val (cmd, _) = buildCommand(finalAmountMsat, finalExpiry, paymentHash, hops)
|
||||
val (cmd, _) = buildCommand(UUID.randomUUID(), finalAmountMsat, finalExpiry, paymentHash, hops)
|
||||
// and then manually build an htlc
|
||||
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.cltvExpiry, cmd.onion)
|
||||
val channelUpdate_bc_disabled = channelUpdate_bc.copy(channelFlags = Announcements.makeChannelFlags(Announcements.isNode1(channelUpdate_bc.channelFlags), enable = false))
|
||||
|
@ -188,7 +190,7 @@ class RelayerSpec extends TestkitBaseClass {
|
|||
val sender = TestProbe()
|
||||
|
||||
// we use this to build a valid onion
|
||||
val (cmd, _) = buildCommand(finalAmountMsat, finalExpiry, paymentHash, hops)
|
||||
val (cmd, _) = buildCommand(UUID.randomUUID(), finalAmountMsat, finalExpiry, paymentHash, hops)
|
||||
// and then manually build an htlc
|
||||
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.cltvExpiry, ByteVector.fill(Sphinx.PacketLength)(0))
|
||||
relayer ! LocalChannelUpdate(null, channelId_bc, channelUpdate_bc.shortChannelId, c, None, channelUpdate_bc, makeCommitments(channelId_bc))
|
||||
|
@ -208,7 +210,7 @@ class RelayerSpec extends TestkitBaseClass {
|
|||
val sender = TestProbe()
|
||||
|
||||
// we use this to build a valid onion
|
||||
val (cmd, _) = buildCommand(channelUpdate_bc.htlcMinimumMsat - 1, finalExpiry, paymentHash, hops.map(hop => hop.copy(lastUpdate = hop.lastUpdate.copy(feeBaseMsat = 0, feeProportionalMillionths = 0))))
|
||||
val (cmd, _) = buildCommand(UUID.randomUUID(), channelUpdate_bc.htlcMinimumMsat - 1, finalExpiry, paymentHash, hops.map(hop => hop.copy(lastUpdate = hop.lastUpdate.copy(feeBaseMsat = 0, feeProportionalMillionths = 0))))
|
||||
// and then manually build an htlc
|
||||
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.cltvExpiry, cmd.onion)
|
||||
relayer ! LocalChannelUpdate(null, channelId_bc, channelUpdate_bc.shortChannelId, c, None, channelUpdate_bc, makeCommitments(channelId_bc))
|
||||
|
@ -228,7 +230,7 @@ class RelayerSpec extends TestkitBaseClass {
|
|||
val sender = TestProbe()
|
||||
|
||||
val hops1 = hops.updated(1, hops(1).copy(lastUpdate = hops(1).lastUpdate.copy(cltvExpiryDelta = 0)))
|
||||
val (cmd, _) = buildCommand(finalAmountMsat, finalExpiry, paymentHash, hops1)
|
||||
val (cmd, _) = buildCommand(UUID.randomUUID(), finalAmountMsat, finalExpiry, paymentHash, hops1)
|
||||
// and then manually build an htlc
|
||||
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.cltvExpiry, cmd.onion)
|
||||
relayer ! LocalChannelUpdate(null, channelId_bc, channelUpdate_bc.shortChannelId, c, None, channelUpdate_bc, makeCommitments(channelId_bc))
|
||||
|
@ -248,7 +250,7 @@ class RelayerSpec extends TestkitBaseClass {
|
|||
val sender = TestProbe()
|
||||
|
||||
val hops1 = hops.updated(1, hops(1).copy(lastUpdate = hops(1).lastUpdate.copy(feeBaseMsat = hops(1).lastUpdate.feeBaseMsat / 2)))
|
||||
val (cmd, _) = buildCommand(finalAmountMsat, finalExpiry, paymentHash, hops1)
|
||||
val (cmd, _) = buildCommand(UUID.randomUUID(), finalAmountMsat, finalExpiry, paymentHash, hops1)
|
||||
// and then manually build an htlc
|
||||
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.cltvExpiry, cmd.onion)
|
||||
relayer ! LocalChannelUpdate(null, channelId_bc, channelUpdate_bc.shortChannelId, c, None, channelUpdate_bc, makeCommitments(channelId_bc))
|
||||
|
@ -269,7 +271,7 @@ class RelayerSpec extends TestkitBaseClass {
|
|||
|
||||
// to simulate this we use a zero-hop route A->B where A is the 'attacker'
|
||||
val hops1 = hops.head :: Nil
|
||||
val (cmd, _) = buildCommand(finalAmountMsat, finalExpiry, paymentHash, hops1)
|
||||
val (cmd, _) = buildCommand(UUID.randomUUID(), finalAmountMsat, finalExpiry, paymentHash, hops1)
|
||||
// and then manually build an htlc with a wrong expiry
|
||||
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat - 1, cmd.paymentHash, cmd.cltvExpiry, cmd.onion)
|
||||
relayer ! LocalChannelUpdate(null, channelId_bc, channelUpdate_bc.shortChannelId, c, None, channelUpdate_bc, makeCommitments(channelId_bc))
|
||||
|
@ -290,7 +292,7 @@ class RelayerSpec extends TestkitBaseClass {
|
|||
|
||||
// to simulate this we use a zero-hop route A->B where A is the 'attacker'
|
||||
val hops1 = hops.head :: Nil
|
||||
val (cmd, _) = buildCommand(finalAmountMsat, finalExpiry, paymentHash, hops1)
|
||||
val (cmd, _) = buildCommand(UUID.randomUUID(), finalAmountMsat, finalExpiry, paymentHash, hops1)
|
||||
// and then manually build an htlc with a wrong expiry
|
||||
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.cltvExpiry - 1, cmd.onion)
|
||||
relayer ! LocalChannelUpdate(null, channelId_bc, channelUpdate_bc.shortChannelId, c, None, channelUpdate_bc, makeCommitments(channelId_bc))
|
||||
|
|
|
@ -23,6 +23,7 @@ import fr.acinq.bitcoin.Script.{pay2wsh, write}
|
|||
import fr.acinq.bitcoin.{Block, ByteVector32, Satoshi, Transaction, TxOut}
|
||||
import fr.acinq.eclair.TestConstants.Alice
|
||||
import fr.acinq.eclair.blockchain.{UtxoStatus, ValidateRequest, ValidateResult, WatchSpentBasic}
|
||||
import fr.acinq.eclair.crypto.LocalKeyManager
|
||||
import fr.acinq.eclair.io.Peer.PeerRoutingMessage
|
||||
import fr.acinq.eclair.router.Announcements._
|
||||
import fr.acinq.eclair.transactions.Scripts
|
||||
|
@ -45,14 +46,15 @@ abstract class BaseRouterSpec extends TestkitBaseClass {
|
|||
|
||||
val remoteNodeId = PrivateKey(ByteVector32(ByteVector.fill(32)(1)), compressed = true).publicKey
|
||||
|
||||
val (priv_a, priv_b, priv_c, priv_d, priv_e, priv_f) = (randomKey, randomKey, randomKey, randomKey, randomKey, randomKey)
|
||||
val (a, b, c, d, e, f) = (priv_a.publicKey, priv_b.publicKey, priv_c.publicKey, priv_d.publicKey, priv_e.publicKey, priv_f.publicKey)
|
||||
val seed = ByteVector32(ByteVector.fill(32)(2))
|
||||
val testKeyManager = new LocalKeyManager(seed, Block.RegtestGenesisBlock.hash)
|
||||
|
||||
val (priv_a, priv_b, priv_c, priv_d, priv_e, priv_f) = (testKeyManager.nodeKey.privateKey, randomKey, randomKey, randomKey, randomKey, randomKey)
|
||||
val (a, b, c, d, e, f) = (testKeyManager.nodeId, priv_b.publicKey, priv_c.publicKey, priv_d.publicKey, priv_e.publicKey, priv_f.publicKey)
|
||||
|
||||
val (priv_funding_a, priv_funding_b, priv_funding_c, priv_funding_d, priv_funding_e, priv_funding_f) = (randomKey, randomKey, randomKey, randomKey, randomKey, randomKey)
|
||||
val (funding_a, funding_b, funding_c, funding_d, funding_e, funding_f) = (priv_funding_a.publicKey, priv_funding_b.publicKey, priv_funding_c.publicKey, priv_funding_d.publicKey, priv_funding_e.publicKey, priv_funding_f.publicKey)
|
||||
|
||||
//val DUMMY_SIG = hex"3045022100e0a180fdd0fe38037cc878c03832861b40a29d32bd7b40b10c9e1efc8c1468a002205ae06d1624896d0d29f4b31e32772ea3cb1b4d7ed4e077e5da28dcc33c0e781201"
|
||||
|
||||
val ann_a = makeNodeAnnouncement(priv_a, "node-A", Color(15, 10, -70), Nil)
|
||||
val ann_b = makeNodeAnnouncement(priv_b, "node-B", Color(50, 99, -80), Nil)
|
||||
val ann_c = makeNodeAnnouncement(priv_c, "node-C", Color(123, 100, -40), Nil)
|
||||
|
|
|
@ -16,6 +16,8 @@
|
|||
|
||||
package fr.acinq.eclair.wire
|
||||
|
||||
import java.util.UUID
|
||||
import akka.actor.ActorSystem
|
||||
import fr.acinq.bitcoin.DeterministicWallet.KeyPath
|
||||
import fr.acinq.bitcoin.{DeterministicWallet, OutPoint}
|
||||
import fr.acinq.eclair.channel._
|
||||
|
@ -26,7 +28,6 @@ import fr.acinq.eclair.wire.ChannelCodecs._
|
|||
import fr.acinq.eclair.{UInt64, randomBytes, randomBytes32, randomKey}
|
||||
import org.scalatest.FunSuite
|
||||
import scodec.bits._
|
||||
|
||||
import scala.compat.Platform
|
||||
import scala.util.Random
|
||||
|
||||
|
@ -140,19 +141,21 @@ class ChannelCodecsSpec extends FunSuite {
|
|||
}
|
||||
|
||||
test("encode/decode origin") {
|
||||
assert(originCodec.decodeValue(originCodec.encode(Local(None)).require).require === Local(None))
|
||||
val id = UUID.randomUUID()
|
||||
assert(originCodec.decodeValue(originCodec.encode(Local(id, Some(ActorSystem("system").deadLetters))).require).require === Local(id, None))
|
||||
// TODO: add backward compatibility check
|
||||
val relayed = Relayed(randomBytes32, 4324, 12000000L, 11000000L)
|
||||
assert(originCodec.decodeValue(originCodec.encode(relayed).require).require === relayed)
|
||||
}
|
||||
|
||||
test("encode/decode map of origins") {
|
||||
val map = Map(
|
||||
1L -> Local(None),
|
||||
1L -> Local(UUID.randomUUID(), None),
|
||||
42L -> Relayed(randomBytes32, 4324, 12000000L, 11000000L),
|
||||
130L -> Relayed(randomBytes32, -45, 13000000L, 12000000L),
|
||||
1000L -> Relayed(randomBytes32, 10, 14000000L, 13000000L),
|
||||
-32L -> Relayed(randomBytes32, 54, 15000000L, 14000000L),
|
||||
-4L -> Local(None))
|
||||
-4L -> Local(UUID.randomUUID(), None))
|
||||
assert(originsMapCodec.decodeValue(originsMapCodec.encode(map).require).require === map)
|
||||
}
|
||||
|
||||
|
|
|
@ -16,6 +16,8 @@
|
|||
|
||||
package fr.acinq.eclair.gui
|
||||
|
||||
import java.util.UUID
|
||||
|
||||
import akka.pattern.{AskTimeoutException, ask}
|
||||
import akka.util.Timeout
|
||||
import fr.acinq.bitcoin.MilliSatoshi
|
||||
|
@ -89,7 +91,7 @@ class Handlers(fKit: Future[Kit])(implicit ec: ExecutionContext = ExecutionConte
|
|||
case None => SendPayment(amountMsat, req.paymentHash, req.nodeId, req.routingInfo, maxAttempts = kit.nodeParams.maxPaymentAttempts)
|
||||
case Some(minFinalCltvExpiry) => SendPayment(amountMsat, req.paymentHash, req.nodeId, req.routingInfo, finalCltvExpiry = minFinalCltvExpiry, maxAttempts = kit.nodeParams.maxPaymentAttempts)
|
||||
}
|
||||
res <- (kit.paymentInitiator ? sendPayment).mapTo[PaymentResult]
|
||||
res <- (kit.paymentInitiator ? sendPayment).mapTo[UUID]
|
||||
} yield res).recover {
|
||||
// completed payment will be handled by the GUIUpdater by listening to PaymentSucceeded/PaymentFailed events
|
||||
case _: AskTimeoutException =>
|
||||
|
|
Loading…
Add table
Reference in a new issue