mirror of
https://github.com/ACINQ/eclair.git
synced 2025-02-23 22:46:44 +01:00
Send to route (#952)
* Add /sendtoroute API and functionality * Do not use extractor pattern in PaymentLifecycle::SendPaymentToRoute * /sendtoroute: support route parameter as comma separated list * Add test for 'sendtoroute' in EclairImplSpec
This commit is contained in:
parent
cfaa5508eb
commit
101bcd7c50
10 changed files with 132 additions and 18 deletions
|
@ -81,6 +81,8 @@ trait Eclair {
|
|||
|
||||
def findRoute(targetNodeId: PublicKey, amountMsat: Long, assistedRoutes: Seq[Seq[PaymentRequest.ExtraHop]] = Seq.empty)(implicit timeout: Timeout): Future[RouteResponse]
|
||||
|
||||
def sendToRoute(route: Seq[PublicKey], amountMsat: Long, paymentHash: ByteVector32, finalCltvExpiry: Long)(implicit timeout: Timeout): Future[UUID]
|
||||
|
||||
def audit(from_opt: Option[Long], to_opt: Option[Long])(implicit timeout: Timeout): Future[AuditResponse]
|
||||
|
||||
def networkFees(from_opt: Option[Long], to_opt: Option[Long])(implicit timeout: Timeout): Future[Seq[NetworkFee]]
|
||||
|
@ -175,6 +177,10 @@ class EclairImpl(appKit: Kit) extends Eclair {
|
|||
(appKit.router ? RouteRequest(appKit.nodeParams.nodeId, targetNodeId, amountMsat, assistedRoutes)).mapTo[RouteResponse]
|
||||
}
|
||||
|
||||
override def sendToRoute(route: Seq[PublicKey], amountMsat: Long, paymentHash: ByteVector32, finalCltvExpiry: Long)(implicit timeout: Timeout): Future[UUID] = {
|
||||
(appKit.paymentInitiator ? SendPaymentToRoute(amountMsat, paymentHash, route, finalCltvExpiry)).mapTo[UUID]
|
||||
}
|
||||
|
||||
override def send(recipientNodeId: PublicKey, amountMsat: Long, paymentHash: ByteVector32, assistedRoutes: Seq[Seq[PaymentRequest.ExtraHop]] = Seq.empty, minFinalCltvExpiry_opt: Option[Long] = None, maxAttempts_opt: Option[Int] = None)(implicit timeout: Timeout): Future[UUID] = {
|
||||
val maxAttempts = maxAttempts_opt.getOrElse(appKit.nodeParams.maxPaymentAttempts)
|
||||
val sendPayment = minFinalCltvExpiry_opt match {
|
||||
|
|
|
@ -18,6 +18,7 @@ package fr.acinq.eclair.api
|
|||
|
||||
import java.util.UUID
|
||||
|
||||
import JsonSupport._
|
||||
import akka.http.scaladsl.unmarshalling.Unmarshaller
|
||||
import akka.util.Timeout
|
||||
import fr.acinq.bitcoin.ByteVector32
|
||||
|
@ -25,7 +26,9 @@ import fr.acinq.bitcoin.Crypto.PublicKey
|
|||
import fr.acinq.eclair.ShortChannelId
|
||||
import fr.acinq.eclair.payment.PaymentRequest
|
||||
import scodec.bits.ByteVector
|
||||
|
||||
import scala.concurrent.duration._
|
||||
import scala.util.{Failure, Success, Try}
|
||||
|
||||
object FormParamExtractors {
|
||||
|
||||
|
@ -57,4 +60,15 @@ object FormParamExtractors {
|
|||
Timeout(str.toInt.seconds)
|
||||
}
|
||||
|
||||
implicit val pubkeyListUnmarshaller: Unmarshaller[String, List[PublicKey]] = Unmarshaller.strict { str =>
|
||||
Try(serialization.read[List[String]](str).map { el =>
|
||||
PublicKey(ByteVector.fromValidHex(el), checkValid = false)
|
||||
}).recoverWith {
|
||||
case error => Try(str.split(",").toList.map(pk => PublicKey(ByteVector.fromValidHex(pk))))
|
||||
} match {
|
||||
case Success(list) => list
|
||||
case Failure(_) => throw new IllegalArgumentException(s"PublicKey list must be either json-encoded or comma separated list")
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -221,6 +221,11 @@ trait Service extends ExtraDirectives with Logging {
|
|||
complete(eclairApi.send(nodeId, amountMsat, paymentHash, maxAttempts = maxAttempts))
|
||||
}
|
||||
} ~
|
||||
path("sendtoroute") {
|
||||
formFields(amountMsatFormParam, paymentHashFormParam, "finalCltvExpiry".as[Long], "route".as[List[PublicKey]](pubkeyListUnmarshaller)) { (amountMsat, paymentHash, finalCltvExpiry, route) =>
|
||||
complete(eclairApi.sendToRoute(route, amountMsat, paymentHash, finalCltvExpiry))
|
||||
}
|
||||
} ~
|
||||
path("getsentinfo") {
|
||||
formFields("id".as[UUID]) { id =>
|
||||
complete(eclairApi.sentInfo(Left(id)))
|
||||
|
|
|
@ -19,7 +19,7 @@ package fr.acinq.eclair.payment
|
|||
import java.util.UUID
|
||||
import akka.actor.{Actor, ActorLogging, ActorRef, Props}
|
||||
import fr.acinq.eclair.NodeParams
|
||||
import fr.acinq.eclair.payment.PaymentLifecycle.SendPayment
|
||||
import fr.acinq.eclair.payment.PaymentLifecycle.GenericSendPayment
|
||||
|
||||
/**
|
||||
* Created by PM on 29/08/2016.
|
||||
|
@ -27,7 +27,7 @@ import fr.acinq.eclair.payment.PaymentLifecycle.SendPayment
|
|||
class PaymentInitiator(nodeParams: NodeParams, router: ActorRef, register: ActorRef) extends Actor with ActorLogging {
|
||||
|
||||
override def receive: Receive = {
|
||||
case c: SendPayment =>
|
||||
case c: GenericSendPayment =>
|
||||
val paymentId = UUID.randomUUID()
|
||||
val payFsm = context.actorOf(PaymentLifecycle.props(nodeParams, paymentId, router, register))
|
||||
payFsm forward c
|
||||
|
|
|
@ -45,6 +45,12 @@ class PaymentLifecycle(nodeParams: NodeParams, id: UUID, router: ActorRef, regis
|
|||
startWith(WAITING_FOR_REQUEST, WaitingForRequest)
|
||||
|
||||
when(WAITING_FOR_REQUEST) {
|
||||
case Event(c: SendPaymentToRoute, WaitingForRequest) =>
|
||||
val send = SendPayment(c.amountMsat, c.paymentHash, c.hops.last, finalCltvExpiry = c.finalCltvExpiry, maxAttempts = 1)
|
||||
paymentsDb.addOutgoingPayment(OutgoingPayment(id, c.paymentHash, None, c.amountMsat, Platform.currentTime, None, OutgoingPaymentStatus.PENDING))
|
||||
router ! FinalizeRoute(c.hops)
|
||||
goto(WAITING_FOR_ROUTE) using WaitingForRoute(sender, send, failures = Nil)
|
||||
|
||||
case Event(c: SendPayment, WaitingForRequest) =>
|
||||
router ! RouteRequest(nodeParams.nodeId, c.targetNodeId, c.amountMsat, c.assistedRoutes, routeParams = c.routeParams)
|
||||
paymentsDb.addOutgoingPayment(OutgoingPayment(id, c.paymentHash, None, c.amountMsat, Platform.currentTime, None, OutgoingPaymentStatus.PENDING))
|
||||
|
@ -192,13 +198,15 @@ object PaymentLifecycle {
|
|||
|
||||
// @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, paymentPreimage: Option[ByteVector32] = None)
|
||||
sealed trait GenericSendPayment
|
||||
case class SendPaymentToRoute(amountMsat: Long, paymentHash: ByteVector32, hops: Seq[PublicKey], finalCltvExpiry: Long = Channel.MIN_CLTV_EXPIRY) extends GenericSendPayment
|
||||
case class SendPayment(amountMsat: Long,
|
||||
paymentHash: ByteVector32,
|
||||
targetNodeId: PublicKey,
|
||||
assistedRoutes: Seq[Seq[ExtraHop]] = Nil,
|
||||
finalCltvExpiry: Long = Channel.MIN_CLTV_EXPIRY,
|
||||
maxAttempts: Int,
|
||||
routeParams: Option[RouteParams] = None) {
|
||||
routeParams: Option[RouteParams] = None) extends GenericSendPayment {
|
||||
require(amountMsat > 0, s"amountMsat must be > 0")
|
||||
}
|
||||
|
||||
|
|
|
@ -67,6 +67,7 @@ case class RouteRequest(source: PublicKey,
|
|||
ignoreChannels: Set[ChannelDesc] = Set.empty,
|
||||
routeParams: Option[RouteParams] = None)
|
||||
|
||||
case class FinalizeRoute(hops:Seq[PublicKey])
|
||||
case class RouteResponse(hops: Seq[Hop], ignoreNodes: Set[PublicKey], ignoreChannels: Set[ChannelDesc]) {
|
||||
require(hops.size > 0, "route cannot be empty")
|
||||
}
|
||||
|
@ -410,6 +411,13 @@ class Router(nodeParams: NodeParams, watcher: ActorRef, initialized: Option[Prom
|
|||
sender ! d
|
||||
stay
|
||||
|
||||
case Event(FinalizeRoute(partialHops), d) =>
|
||||
// split into sublists [(a,b),(b,c), ...] then get the edges between each of those pairs, then select the largest edge between them
|
||||
val edges = partialHops.sliding(2).map { case List(v1, v2) => d.graph.getEdgesBetween(v1, v2).maxBy(_.update.htlcMaximumMsat) }
|
||||
val hops = edges.map(d => Hop(d.desc.a, d.desc.b, d.update)).toSeq
|
||||
sender ! RouteResponse(hops, Set.empty, Set.empty)
|
||||
stay
|
||||
|
||||
case Event(RouteRequest(start, end, amount, assistedRoutes, ignoreNodes, ignoreChannels, params_opt), d) =>
|
||||
// we convert extra routing info provided in the payment request to fake channel_update
|
||||
// it takes precedence over all other channel_updates we know
|
||||
|
|
|
@ -26,11 +26,9 @@ import akka.util.Timeout
|
|||
import fr.acinq.bitcoin.Crypto.PublicKey
|
||||
import fr.acinq.eclair.blockchain.TestWallet
|
||||
import fr.acinq.eclair.io.Peer.OpenChannel
|
||||
import fr.acinq.eclair.payment.PaymentLifecycle.SendPayment
|
||||
import fr.acinq.eclair.payment.PaymentRequest.ExtraHop
|
||||
import fr.acinq.eclair.payment.PaymentLifecycle.{ReceivePayment, SendPayment, SendPaymentToRoute}
|
||||
import org.scalatest.{Outcome, fixture}
|
||||
import fr.acinq.eclair.payment.PaymentLifecycle.{SendPayment}
|
||||
import fr.acinq.eclair.payment.PaymentLifecycle.{ReceivePayment, SendPayment}
|
||||
import fr.acinq.eclair.payment.PaymentLifecycle.SendPayment
|
||||
import fr.acinq.eclair.payment.PaymentRequest.ExtraHop
|
||||
import org.scalatest.{Matchers, Outcome, fixture}
|
||||
import scodec.bits._
|
||||
|
@ -239,4 +237,19 @@ class EclairImplSpec extends TestKit(ActorSystem("mySystem")) with fixture.FunSu
|
|||
Await.result(eclair.allInvoices(None, None), 10 seconds)
|
||||
paymentDb.listPaymentRequests(0, MaxEpochSeconds).wasCalled(once) // assert the call was made only once and with the specified params
|
||||
}
|
||||
|
||||
test("sendtoroute should pass the parameters correctly") { f =>
|
||||
import f._
|
||||
|
||||
val route = Seq(PublicKey(hex"030bb6a5e0c6b203c7e2180fb78c7ba4bdce46126761d8201b91ddac089cdecc87"))
|
||||
val eclair = new EclairImpl(kit)
|
||||
eclair.sendToRoute(route, 1234, ByteVector32.One, 123)
|
||||
|
||||
val send = paymentInitiator.expectMsgType[SendPaymentToRoute]
|
||||
|
||||
assert(send.hops == route)
|
||||
assert(send.amountMsat == 1234)
|
||||
assert(send.finalCltvExpiry == 123)
|
||||
assert(send.paymentHash == ByteVector32.One)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -26,20 +26,18 @@ import akka.http.scaladsl.server.Route
|
|||
import akka.http.scaladsl.testkit.{RouteTestTimeout, ScalatestRouteTest, WSProbe}
|
||||
import akka.stream.ActorMaterializer
|
||||
import akka.util.Timeout
|
||||
import fr.acinq.bitcoin.Crypto.PublicKey
|
||||
import fr.acinq.bitcoin.{ByteVector32, Crypto, MilliSatoshi}
|
||||
import fr.acinq.eclair.TestConstants._
|
||||
import fr.acinq.eclair._
|
||||
import fr.acinq.eclair.channel.RES_GETINFO
|
||||
import fr.acinq.eclair.db.{IncomingPayment, NetworkFee, OutgoingPayment, Stats}
|
||||
import fr.acinq.eclair.io.Peer.PeerInfo
|
||||
import fr.acinq.eclair.payment.PaymentLifecycle.PaymentFailed
|
||||
import fr.acinq.eclair.payment._
|
||||
import fr.acinq.eclair.router.{ChannelDesc, RouteResponse}
|
||||
import fr.acinq.eclair.wire.{ChannelUpdate, NodeAddress, NodeAnnouncement}
|
||||
import org.json4s.jackson.Serialization
|
||||
import org.mockito.scalatest.IdiomaticMockito
|
||||
import org.scalatest.{FunSuite, Matchers}
|
||||
import scodec.bits.ByteVector
|
||||
import scodec.bits._
|
||||
import scala.concurrent.Future
|
||||
import scala.concurrent.duration._
|
||||
import scala.io.Source
|
||||
|
@ -282,19 +280,39 @@ class ApiServiceSpec extends FunSuite with ScalatestRouteTest with IdiomaticMock
|
|||
}
|
||||
}
|
||||
|
||||
test("'updaterelayfee' should use named parameters") {
|
||||
test("'sendtoroute' method should accept a both a json-encoded AND comma separaterd list of pubkeys") {
|
||||
|
||||
val rawUUID = "487da196-a4dc-4b1e-92b4-3e5e905e9f3f"
|
||||
val paymentUUID = UUID.fromString(rawUUID)
|
||||
val expectedRoute = List(PublicKey(hex"0217eb8243c95f5a3b7d4c5682d10de354b7007eb59b6807ae407823963c7547a9"), PublicKey(hex"0242a4ae0c5bef18048fbecf995094b74bfb0f7391418d71ed394784373f41e4f3"), PublicKey(hex"026ac9fcd64fb1aa1c491fc490634dc33da41d4a17b554e0adf1b32fee88ee9f28"))
|
||||
val csvNodes = "0217eb8243c95f5a3b7d4c5682d10de354b7007eb59b6807ae407823963c7547a9, 0242a4ae0c5bef18048fbecf995094b74bfb0f7391418d71ed394784373f41e4f3, 026ac9fcd64fb1aa1c491fc490634dc33da41d4a17b554e0adf1b32fee88ee9f28"
|
||||
val jsonNodes = serialization.write(expectedRoute)
|
||||
|
||||
val eclair = mock[Eclair]
|
||||
eclair.updateRelayFee(any, any[Long], any[Long])(timeout = any[Timeout]) returns Future.successful("done")
|
||||
val service = new MockService(eclair)
|
||||
eclair.sendToRoute(any[List[PublicKey]], anyLong, any[ByteVector32], anyLong)(any[Timeout]) returns Future.successful(paymentUUID)
|
||||
val mockService = new MockService(eclair)
|
||||
|
||||
Post("/updaterelayfee", FormData("channelId" -> ByteVector32.Zeroes.toHex, "feeBaseMsat" -> "123", "feeProportionalMillionths" -> "456").toEntity) ~>
|
||||
addCredentials(BasicHttpCredentials("", service.password)) ~>
|
||||
Route.seal(service.route) ~>
|
||||
Post("/sendtoroute", FormData("route" -> jsonNodes, "amountMsat" -> "1234", "paymentHash" -> ByteVector32.Zeroes.toHex, "finalCltvExpiry" -> "190").toEntity) ~>
|
||||
addCredentials(BasicHttpCredentials("", mockService.password)) ~>
|
||||
addHeader("Content-Type", "application/json") ~>
|
||||
Route.seal(mockService.route) ~>
|
||||
check {
|
||||
assert(handled)
|
||||
assert(status == OK)
|
||||
eclair.updateRelayFee(Left(ByteVector32.Zeroes), 123, 456)(any[Timeout]).wasCalled(once)
|
||||
assert(entityAs[String] == "\""+rawUUID+"\"")
|
||||
eclair.sendToRoute(expectedRoute, 1234, ByteVector32.Zeroes, 190)(any[Timeout]).wasCalled(once)
|
||||
}
|
||||
|
||||
// this test uses CSV encoded route
|
||||
Post("/sendtoroute", FormData("route" -> csvNodes, "amountMsat" -> "1234", "paymentHash" -> ByteVector32.One.toHex, "finalCltvExpiry" -> "190").toEntity) ~>
|
||||
addCredentials(BasicHttpCredentials("", mockService.password)) ~>
|
||||
addHeader("Content-Type", "application/json") ~>
|
||||
Route.seal(mockService.route) ~>
|
||||
check {
|
||||
assert(handled)
|
||||
assert(status == OK)
|
||||
assert(entityAs[String] == "\""+rawUUID+"\"")
|
||||
eclair.sendToRoute(expectedRoute, 1234, ByteVector32.One, 190)(any[Timeout]).wasCalled(once)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -45,6 +45,34 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
|||
|
||||
val defaultAmountMsat = 142000000L
|
||||
|
||||
test("send to route") { fixture =>
|
||||
import fixture._
|
||||
val defaultPaymentHash = randomBytes32
|
||||
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()
|
||||
system.eventStream.subscribe(eventListener.ref, classOf[PaymentEvent])
|
||||
|
||||
paymentFSM ! SubscribeTransitionCallBack(monitor.ref)
|
||||
val CurrentState(_, WAITING_FOR_REQUEST) = monitor.expectMsgClass(classOf[CurrentState[_]])
|
||||
|
||||
// pre-computed route going from A to D
|
||||
val request = SendPaymentToRoute(defaultAmountMsat, defaultPaymentHash, Seq(a,b,c,d))
|
||||
|
||||
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))
|
||||
|
||||
sender.expectMsgType[PaymentSucceeded]
|
||||
awaitCond(paymentDb.getOutgoingPayment(id).exists(_.status == OutgoingPaymentStatus.SUCCEEDED))
|
||||
}
|
||||
|
||||
test("payment failed (route not found)") { fixture =>
|
||||
import fixture._
|
||||
val defaultPaymentHash = randomBytes32
|
||||
|
|
|
@ -237,6 +237,20 @@ class RouterSpec extends BaseRouterSpec {
|
|||
assert(state.updates.size == 8)
|
||||
}
|
||||
|
||||
test("given a pre-computed route add the proper channel updates") { fixture =>
|
||||
import fixture._
|
||||
|
||||
val sender = TestProbe()
|
||||
val preComputedRoute = Seq(a, b, c, d)
|
||||
sender.send(router, FinalizeRoute(preComputedRoute))
|
||||
|
||||
val response = sender.expectMsgType[RouteResponse]
|
||||
// the route hasn't changed (nodes are the same)
|
||||
assert(response.hops.map(_.nodeId).toList == preComputedRoute.dropRight(1).toList)
|
||||
assert(response.hops.last.nextNodeId == preComputedRoute.last)
|
||||
assert(response.hops.map(_.lastUpdate).toList == List(channelUpdate_ab, channelUpdate_bc, channelUpdate_cd))
|
||||
}
|
||||
|
||||
test("ask for channels that we marked as stale for which we receive a new update") { fixture =>
|
||||
import fixture._
|
||||
val blockHeight = Globals.blockCount.get().toInt - 2020
|
||||
|
|
Loading…
Add table
Reference in a new issue