mirror of
https://github.com/ACINQ/eclair.git
synced 2025-01-19 05:33:59 +01:00
Assisted route handling (#279)
This commit is contained in:
parent
7d357662f3
commit
63b772cf2a
@ -116,15 +116,15 @@ trait Service extends Logging {
|
||||
case JsonRPCBody(_, _, "send", JString(paymentRequest) :: rest) =>
|
||||
for {
|
||||
req <- Future(PaymentRequest.read(paymentRequest))
|
||||
amount = (req.amount, rest) match {
|
||||
amountMsat = (req.amount, rest) match {
|
||||
case (Some(_), JInt(amt) :: Nil) => amt.toLong // overriding payment request amount with the one provided
|
||||
case (Some(amt), _) => amt.amount
|
||||
case (None, JInt(amt) :: Nil) => amt.toLong // amount wasn't specified in request, using custom one
|
||||
case (None, _) => throw new RuntimeException("you need to manually specify an amount for this payment request")
|
||||
}
|
||||
sendPayment = req.minFinalCltvExpiry match {
|
||||
case None => SendPayment(amount, req.paymentHash, req.nodeId)
|
||||
case Some(value) => SendPayment(amount, req.paymentHash, req.nodeId, value)
|
||||
case None => SendPayment(amountMsat, req.paymentHash, req.nodeId, req.routingInfo())
|
||||
case Some(minFinalCltvExpiry) => SendPayment(amountMsat, req.paymentHash, req.nodeId, req.routingInfo(), minFinalCltvExpiry = minFinalCltvExpiry)
|
||||
}
|
||||
res <- (paymentInitiator ? sendPayment).mapTo[PaymentResult]
|
||||
} yield res
|
||||
|
@ -1,50 +0,0 @@
|
||||
package fr.acinq.eclair.payment
|
||||
|
||||
import fr.acinq.bitcoin.Crypto.PublicKey
|
||||
import fr.acinq.eclair.payment.PaymentRequest.ExtraHop
|
||||
import fr.acinq.eclair.wire.ChannelUpdate
|
||||
|
||||
|
||||
object PaymentHop {
|
||||
/**
|
||||
*
|
||||
* @param baseMsat fixed fee
|
||||
* @param proportional proportional fee
|
||||
* @param msat amount in millisatoshi
|
||||
* @return the fee (in msat) that a node should be paid to forward an HTLC of 'amount' millisatoshis
|
||||
*/
|
||||
def nodeFee(baseMsat: Long, proportional: Long, msat: Long): Long = baseMsat + (proportional * msat) / 1000000
|
||||
|
||||
/**
|
||||
*
|
||||
* @param reversePath sequence of Hops from recipient to a start of assisted path
|
||||
* @param msat an amount to send to a payment recipient
|
||||
* @return a sequence of extra hops with a pre-calculated fee for a given msat amount
|
||||
*/
|
||||
def buildExtra(reversePath: Seq[Hop], msat: Long): Seq[ExtraHop] = reversePath.foldLeft(List.empty[ExtraHop]) {
|
||||
case (Nil, hop) => ExtraHop(hop.nodeId, hop.shortChannelId, hop.feeBaseMsat, hop.feeProportionalMillionths, hop.cltvExpiryDelta) :: Nil
|
||||
case (head :: rest, hop) => ExtraHop(hop.nodeId, hop.shortChannelId, hop.feeBaseMsat, hop.feeProportionalMillionths, hop.cltvExpiryDelta) :: head :: rest
|
||||
}
|
||||
}
|
||||
|
||||
trait PaymentHop {
|
||||
def nextFee(msat: Long): Long
|
||||
|
||||
def shortChannelId: Long
|
||||
|
||||
def cltvExpiryDelta: Int
|
||||
|
||||
def nodeId: PublicKey
|
||||
}
|
||||
|
||||
case class Hop(nodeId: PublicKey, nextNodeId: PublicKey, lastUpdate: ChannelUpdate) extends PaymentHop {
|
||||
def nextFee(msat: Long): Long = PaymentHop.nodeFee(lastUpdate.feeBaseMsat, lastUpdate.feeProportionalMillionths, msat)
|
||||
|
||||
def feeBaseMsat: Long = lastUpdate.feeBaseMsat
|
||||
|
||||
def feeProportionalMillionths: Long = lastUpdate.feeProportionalMillionths
|
||||
|
||||
def cltvExpiryDelta: Int = lastUpdate.cltvExpiryDelta
|
||||
|
||||
def shortChannelId: Long = lastUpdate.shortChannelId
|
||||
}
|
@ -7,6 +7,7 @@ import fr.acinq.eclair._
|
||||
import fr.acinq.eclair.channel.{CMD_ADD_HTLC, Register}
|
||||
import fr.acinq.eclair.crypto.Sphinx
|
||||
import fr.acinq.eclair.crypto.Sphinx.{ErrorPacket, Packet}
|
||||
import fr.acinq.eclair.payment.PaymentRequest.ExtraHop
|
||||
import fr.acinq.eclair.router._
|
||||
import fr.acinq.eclair.wire._
|
||||
import scodec.Attempt
|
||||
@ -15,7 +16,7 @@ import scala.util.{Failure, Success}
|
||||
|
||||
// @formatter:off
|
||||
case class ReceivePayment(amountMsat_opt: Option[MilliSatoshi], description: String)
|
||||
case class SendPayment(amountMsat: Long, paymentHash: BinaryData, targetNodeId: PublicKey, minFinalCltvExpiry: Long = PaymentLifecycle.defaultMinFinalCltvExpiry, maxAttempts: Int = 5)
|
||||
case class SendPayment(amountMsat: Long, paymentHash: BinaryData, targetNodeId: PublicKey, assistedRoutes: Seq[Seq[ExtraHop]] = Nil, minFinalCltvExpiry: Long = PaymentLifecycle.defaultMinFinalCltvExpiry, maxAttempts: Int = 5)
|
||||
|
||||
sealed trait PaymentResult
|
||||
case class PaymentSucceeded(route: Seq[Hop], paymentPreimage: BinaryData) extends PaymentResult
|
||||
@ -48,7 +49,7 @@ class PaymentLifecycle(sourceNodeId: PublicKey, router: ActorRef, register: Acto
|
||||
|
||||
when(WAITING_FOR_REQUEST) {
|
||||
case Event(c: SendPayment, WaitingForRequest) =>
|
||||
router ! RouteRequest(sourceNodeId, c.targetNodeId)
|
||||
router ! RouteRequest(sourceNodeId, c.targetNodeId, c.assistedRoutes)
|
||||
goto(WAITING_FOR_ROUTE) using WaitingForRoute(sender, c, failures = Nil)
|
||||
}
|
||||
|
||||
@ -81,7 +82,7 @@ class PaymentLifecycle(sourceNodeId: PublicKey, router: ActorRef, register: Acto
|
||||
// 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, ignoreNodes ++ blacklist, ignoreChannels)
|
||||
router ! RouteRequest(sourceNodeId, c.targetNodeId, c.assistedRoutes, ignoreNodes ++ blacklist, ignoreChannels)
|
||||
goto(WAITING_FOR_ROUTE) using WaitingForRoute(s, c, failures :+ UnreadableRemoteFailure(hops))
|
||||
case Success(e@ErrorPacket(nodeId, failureMessage)) if nodeId == c.targetNodeId =>
|
||||
log.warning(s"received an error message from target nodeId=$nodeId, failing the payment (failure=$failureMessage)")
|
||||
@ -95,7 +96,7 @@ class PaymentLifecycle(sourceNodeId: PublicKey, router: ActorRef, register: Acto
|
||||
case Success(e@ErrorPacket(nodeId, failureMessage: Node)) =>
|
||||
log.info(s"received an 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, ignoreNodes + nodeId, ignoreChannels)
|
||||
router ! RouteRequest(sourceNodeId, c.targetNodeId, c.assistedRoutes, ignoreNodes + nodeId, ignoreChannels)
|
||||
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)")
|
||||
@ -117,18 +118,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, ignoreNodes, ignoreChannels)
|
||||
router ! RouteRequest(sourceNodeId, c.targetNodeId, c.assistedRoutes, ignoreNodes, ignoreChannels)
|
||||
} 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, ignoreNodes + nodeId, ignoreChannels)
|
||||
router ! RouteRequest(sourceNodeId, c.targetNodeId, c.assistedRoutes, ignoreNodes + nodeId, ignoreChannels)
|
||||
}
|
||||
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(_.lastUpdate.shortChannelId)
|
||||
router ! RouteRequest(sourceNodeId, c.targetNodeId, ignoreNodes, ignoreChannels ++ faultyChannel.toSet)
|
||||
router ! RouteRequest(sourceNodeId, c.targetNodeId, c.assistedRoutes, ignoreNodes, ignoreChannels ++ faultyChannel.toSet)
|
||||
goto(WAITING_FOR_ROUTE) using WaitingForRoute(s, c, failures :+ RemoteFailure(hops, e))
|
||||
}
|
||||
|
||||
@ -146,7 +147,7 @@ class PaymentLifecycle(sourceNodeId: PublicKey, router: ActorRef, register: Acto
|
||||
stop(FSM.Normal)
|
||||
} else {
|
||||
log.info(s"received an error message from local, trying to use a different channel (failure=${t.getMessage})")
|
||||
router ! RouteRequest(sourceNodeId, c.targetNodeId, ignoreNodes, ignoreChannels + hops.head.lastUpdate.shortChannelId)
|
||||
router ! RouteRequest(sourceNodeId, c.targetNodeId, c.assistedRoutes, ignoreNodes, ignoreChannels + hops.head.lastUpdate.shortChannelId)
|
||||
goto(WAITING_FOR_ROUTE) using WaitingForRoute(s, c, failures :+ LocalFailure(t))
|
||||
}
|
||||
|
||||
@ -159,6 +160,15 @@ object PaymentLifecycle {
|
||||
|
||||
def props(sourceNodeId: PublicKey, router: ActorRef, register: ActorRef) = Props(classOf[PaymentLifecycle], sourceNodeId, router, register)
|
||||
|
||||
/**
|
||||
*
|
||||
* @param baseMsat fixed fee
|
||||
* @param proportional proportional fee
|
||||
* @param msat amount in millisatoshi
|
||||
* @return the fee (in msat) that a node should be paid to forward an HTLC of 'amount' millisatoshis
|
||||
*/
|
||||
def nodeFee(baseMsat: Long, proportional: Long, msat: Long): Long = baseMsat + (proportional * msat) / 1000000
|
||||
|
||||
def buildOnion(nodes: Seq[PublicKey], payloads: Seq[PerHopPayload], associatedData: BinaryData): Sphinx.PacketAndSecrets = {
|
||||
require(nodes.size == payloads.size)
|
||||
val sessionKey = randomKey
|
||||
@ -181,10 +191,11 @@ object PaymentLifecycle {
|
||||
* - firstExpiry is the cltv expiry for the first htlc in the route
|
||||
* - a sequence of payloads that will be used to build the onion
|
||||
*/
|
||||
def buildPayloads(finalAmountMsat: Long, finalExpiry: Long, hops: Seq[PaymentHop]): (Long, Long, Seq[PerHopPayload]) =
|
||||
def buildPayloads(finalAmountMsat: Long, finalExpiry: Long, hops: Seq[Hop]): (Long, Long, Seq[PerHopPayload]) =
|
||||
hops.reverse.foldLeft((finalAmountMsat, finalExpiry, PerHopPayload(0L, finalAmountMsat, finalExpiry) :: Nil)) {
|
||||
case ((msat, expiry, payloads), hop) =>
|
||||
(msat + hop.nextFee(msat), expiry + hop.cltvExpiryDelta, PerHopPayload(hop.shortChannelId, msat, expiry) +: payloads)
|
||||
val nextFee = nodeFee(hop.lastUpdate.feeBaseMsat, hop.lastUpdate.feeProportionalMillionths, msat)
|
||||
(msat + nextFee, expiry + hop.lastUpdate.cltvExpiryDelta, PerHopPayload(hop.lastUpdate.shortChannelId, msat, expiry) +: payloads)
|
||||
}
|
||||
|
||||
// this is defined in BOLT 11
|
||||
|
@ -8,7 +8,7 @@ import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey}
|
||||
import fr.acinq.bitcoin.{BinaryData, MilliSatoshi, _}
|
||||
import fr.acinq.eclair.crypto.BitStream
|
||||
import fr.acinq.eclair.crypto.BitStream.Bit
|
||||
import fr.acinq.eclair.payment.PaymentRequest.{Amount, RoutingInfoTag, Timestamp}
|
||||
import fr.acinq.eclair.payment.PaymentRequest.{Amount, ExtraHop, RoutingInfoTag, Timestamp}
|
||||
|
||||
import scala.annotation.tailrec
|
||||
import scala.util.Try
|
||||
@ -58,7 +58,7 @@ case class PaymentRequest(prefix: String, amount: Option[MilliSatoshi], timestam
|
||||
case PaymentRequest.FallbackAddressTag(version, hash) if prefix == "lntb" => Bech32.encodeWitnessAddress("tb", version, hash)
|
||||
}
|
||||
|
||||
def routingInfo(): Seq[RoutingInfoTag] = tags.collect { case t: RoutingInfoTag => t }
|
||||
def routingInfo(): Seq[Seq[ExtraHop]] = tags.collect { case t: RoutingInfoTag => t.path }
|
||||
|
||||
def expiry: Option[Long] = tags.collectFirst {
|
||||
case PaymentRequest.ExpiryTag(seconds) => seconds
|
||||
@ -215,18 +215,15 @@ object PaymentRequest {
|
||||
/**
|
||||
* Extra hop contained in RoutingInfoTag
|
||||
*
|
||||
* @param nodeId node id
|
||||
* @param nodeId start of the channel
|
||||
* @param shortChannelId channel id
|
||||
* @param feeBaseMast node fixed fee
|
||||
* @param feeBaseMsat node fixed fee
|
||||
* @param feeProportionalMillionths node proportional fee
|
||||
* @param cltvExpiryDelta node cltv expiry delta
|
||||
*/
|
||||
case class ExtraHop(nodeId: PublicKey, shortChannelId: Long, feeBaseMast: Long, feeProportionalMillionths: Long, cltvExpiryDelta: Int) extends PaymentHop {
|
||||
case class ExtraHop(nodeId: PublicKey, shortChannelId: Long, feeBaseMsat: Long, feeProportionalMillionths: Long, cltvExpiryDelta: Int) {
|
||||
def pack: Seq[Byte] = nodeId.toBin ++ Protocol.writeUInt64(shortChannelId, ByteOrder.BIG_ENDIAN) ++
|
||||
Protocol.writeUInt32(feeBaseMast, ByteOrder.BIG_ENDIAN) ++ Protocol.writeUInt32(feeProportionalMillionths, ByteOrder.BIG_ENDIAN) ++ Protocol.writeUInt16(cltvExpiryDelta, ByteOrder.BIG_ENDIAN)
|
||||
|
||||
// Fee is already pre-calculated for extra hops
|
||||
def nextFee(msat: Long): Long = PaymentHop.nodeFee(feeBaseMast, feeProportionalMillionths, msat)
|
||||
Protocol.writeUInt32(feeBaseMsat, ByteOrder.BIG_ENDIAN) ++ Protocol.writeUInt32(feeProportionalMillionths, ByteOrder.BIG_ENDIAN) ++ Protocol.writeUInt16(cltvExpiryDelta, ByteOrder.BIG_ENDIAN)
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -11,7 +11,7 @@ import fr.acinq.eclair._
|
||||
import fr.acinq.eclair.blockchain._
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.io.Peer
|
||||
import fr.acinq.eclair.payment.Hop
|
||||
import fr.acinq.eclair.payment.PaymentRequest.ExtraHop
|
||||
import fr.acinq.eclair.transactions.Scripts
|
||||
import fr.acinq.eclair.wire._
|
||||
import org.jgrapht.alg.shortestpath.DijkstraShortestPath
|
||||
@ -27,7 +27,8 @@ import scala.util.{Random, Success, Try}
|
||||
// @formatter:off
|
||||
|
||||
case class ChannelDesc(id: Long, a: PublicKey, b: PublicKey)
|
||||
case class RouteRequest(source: PublicKey, target: PublicKey, ignoreNodes: Set[PublicKey] = Set.empty, ignoreChannels: Set[Long] = Set.empty)
|
||||
case class Hop(nodeId: PublicKey, nextNodeId: PublicKey, lastUpdate: ChannelUpdate)
|
||||
case class RouteRequest(source: PublicKey, target: PublicKey, assistedRoutes: Seq[Seq[ExtraHop]] = Nil, ignoreNodes: Set[PublicKey] = Set.empty, ignoreChannels: Set[Long] = Set.empty)
|
||||
case class RouteResponse(hops: Seq[Hop], ignoreNodes: Set[PublicKey], ignoreChannels: Set[Long]) { require(hops.size > 0, "route cannot be empty") }
|
||||
case class ExcludeChannel(desc: ChannelDesc) // this is used when we get a TemporaryChannelFailure, to give time for the channel to recover (note that exclusions are directed)
|
||||
case class LiftChannelExclusion(desc: ChannelDesc)
|
||||
@ -324,9 +325,12 @@ class Router(nodeParams: NodeParams, watcher: ActorRef) extends FSM[State, Data]
|
||||
graph2dot(d.nodes, d.channels) pipeTo sender
|
||||
stay
|
||||
|
||||
case Event(RouteRequest(start, end, ignoreNodes, ignoreChannels), d) =>
|
||||
case Event(RouteRequest(start, end, assistedRoutes, ignoreNodes, ignoreChannels), d) =>
|
||||
// we convert extra routing info provided in the payment request to fake channel_update
|
||||
// it has precedence over all other channel_updates we know
|
||||
val assistedUpdates = assistedRoutes.flatMap(toFakeUpdates(_, end))
|
||||
// we start with channel_updates of local channels
|
||||
val updates0 = d.localUpdates.values.toMap
|
||||
val updates0 = d.localUpdates.values ++ assistedUpdates
|
||||
// we add them to the publicly-announced updates (channel_updates for announced channels will be deduped)
|
||||
val updates1 = d.updates ++ updates0
|
||||
// we then filter out the currently excluded channels
|
||||
@ -352,6 +356,19 @@ object Router {
|
||||
|
||||
def props(nodeParams: NodeParams, watcher: ActorRef) = Props(new Router(nodeParams, watcher))
|
||||
|
||||
def toFakeUpdate(extraHop: ExtraHop): ChannelUpdate =
|
||||
// the `direction` bit in flags will not be accurate but it doesn't matter because it is not used
|
||||
// what matters is that the `disable` bit is 0 so that this update doesn't get filtered out
|
||||
ChannelUpdate(signature = "", chainHash = "", extraHop.shortChannelId, Platform.currentTime / 1000, flags = BinaryData("0000"), extraHop.cltvExpiryDelta, htlcMinimumMsat = 0L, extraHop.feeBaseMsat, extraHop.feeProportionalMillionths)
|
||||
|
||||
def toFakeUpdates(extraRoute: Seq[ExtraHop], targetNodeId: PublicKey): Map[ChannelDesc, ChannelUpdate] = {
|
||||
// BOLT 11: "For each entry, the pubkey is the node ID of the start of the channel", and the last node is the destination
|
||||
val nextNodeIds = extraRoute.map(_.nodeId).drop(1) :+ targetNodeId
|
||||
extraRoute.zip(nextNodeIds).map {
|
||||
case (extraHop: ExtraHop, nextNodeId) => (ChannelDesc(extraHop.shortChannelId, extraHop.nodeId, nextNodeId) -> toFakeUpdate(extraHop))
|
||||
}.toMap
|
||||
}
|
||||
|
||||
def getDesc(u: ChannelUpdate, channel: ChannelAnnouncement): ChannelDesc = {
|
||||
require(u.flags.data.size == 2, s"invalid flags length ${u.flags.data.size} != 2")
|
||||
// the least significant bit tells us if it is node1 or node2
|
||||
|
@ -11,6 +11,7 @@ import fr.acinq.eclair._
|
||||
import fr.acinq.eclair.blockchain._
|
||||
import fr.acinq.eclair.channel.states.StateTestsHelperMethods
|
||||
import fr.acinq.eclair.payment._
|
||||
import fr.acinq.eclair.router.Hop
|
||||
import fr.acinq.eclair.wire._
|
||||
import grizzled.slf4j.Logging
|
||||
import org.junit.runner.RunWith
|
||||
|
@ -7,7 +7,8 @@ import fr.acinq.eclair.blockchain._
|
||||
import fr.acinq.eclair.blockchain.fee.FeeratesPerKw
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.crypto.Sphinx
|
||||
import fr.acinq.eclair.payment.{Hop, PaymentLifecycle}
|
||||
import fr.acinq.eclair.payment.PaymentLifecycle
|
||||
import fr.acinq.eclair.router.Hop
|
||||
import fr.acinq.eclair.wire._
|
||||
import fr.acinq.eclair.{Globals, NodeParams, TestConstants}
|
||||
|
||||
|
@ -9,6 +9,7 @@ import fr.acinq.eclair.blockchain.fee.FeeratesPerKw
|
||||
import fr.acinq.eclair.channel.states.StateTestsHelperMethods
|
||||
import fr.acinq.eclair.channel.{Data, State, _}
|
||||
import fr.acinq.eclair.payment.{ForwardAdd, Local, PaymentLifecycle, _}
|
||||
import fr.acinq.eclair.router.Hop
|
||||
import fr.acinq.eclair.wire.{ChannelUpdate, CommitSig, Error, FailureMessageCodecs, PermanentChannelFailure, RevokeAndAck, Shutdown, UpdateAddHtlc, UpdateFailHtlc, UpdateFailMalformedHtlc, UpdateFee, UpdateFulfillHtlc}
|
||||
import fr.acinq.eclair.{Globals, TestConstants, TestkitBaseClass}
|
||||
import org.junit.runner.RunWith
|
||||
|
@ -3,9 +3,9 @@ package fr.acinq.eclair.payment
|
||||
import fr.acinq.bitcoin.{BinaryData, Block, Crypto}
|
||||
import fr.acinq.eclair.crypto.Sphinx
|
||||
import fr.acinq.eclair.crypto.Sphinx.{PacketAndSecrets, ParsedPacket}
|
||||
import fr.acinq.eclair.payment.PaymentHop.nodeFee
|
||||
import fr.acinq.eclair.payment.PaymentLifecycle._
|
||||
import fr.acinq.eclair.randomKey
|
||||
import fr.acinq.eclair.router.Hop
|
||||
import fr.acinq.eclair.wire.{ChannelUpdate, LightningMessageCodecs, PerHopPayload}
|
||||
import org.junit.runner.RunWith
|
||||
import org.scalatest.FunSuite
|
||||
|
@ -77,7 +77,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, ignoreNodes = Set.empty, ignoreChannels = Set.empty))
|
||||
routerForwarder.expectMsg(RouteRequest(a, d, 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
|
||||
@ -86,7 +86,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
||||
sender.send(paymentFSM, UpdateFailMalformedHtlc("00" * 32, 0, "42" * 32, FailureMessageCodecs.BADONION))
|
||||
|
||||
// then the payment lifecycle will ask for a new route excluding the channel
|
||||
routerForwarder.expectMsg(RouteRequest(a, d, ignoreNodes = Set.empty, ignoreChannels = Set(channelId_ab)))
|
||||
routerForwarder.expectMsg(RouteRequest(a, d, assistedRoutes = Nil, ignoreNodes = Set.empty, ignoreChannels = Set(channelId_ab)))
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE)
|
||||
|
||||
}
|
||||
@ -105,7 +105,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, ignoreNodes = Set.empty, ignoreChannels = Set.empty))
|
||||
routerForwarder.expectMsg(RouteRequest(a, d, 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
|
||||
@ -121,7 +121,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
||||
// payment lifecycle forwards the embedded channelUpdate to the router
|
||||
routerForwarder.expectMsg(channelUpdate_bc)
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE)
|
||||
routerForwarder.expectMsg(RouteRequest(a, d, ignoreNodes = Set.empty, ignoreChannels = Set.empty))
|
||||
routerForwarder.expectMsg(RouteRequest(a, d, 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))
|
||||
@ -141,7 +141,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, ignoreNodes = Set.empty, ignoreChannels = Set.empty))
|
||||
routerForwarder.expectMsg(RouteRequest(a, d, 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
|
||||
@ -153,7 +153,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
||||
|
||||
// payment lifecycle forwards the embedded channelUpdate to the router
|
||||
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE)
|
||||
routerForwarder.expectMsg(RouteRequest(a, d, ignoreNodes = Set.empty, ignoreChannels = Set(channelId_bc)))
|
||||
routerForwarder.expectMsg(RouteRequest(a, d, assistedRoutes = Nil, ignoreNodes = Set.empty, ignoreChannels = Set(channelId_bc)))
|
||||
routerForwarder.forward(router)
|
||||
// we allow 2 tries, so we send a 2nd request to the router, which won't find another route
|
||||
|
||||
|
@ -106,10 +106,10 @@ class PaymentRequestSpec extends FunSuite {
|
||||
assert(pr.nodeId == PublicKey(BinaryData("03e7156ae33b0a208d0744199163177e909e80176e55d97a2f221ede0f934dd9ad")))
|
||||
assert(pr.description == Right(Crypto.sha256("One piece of chocolate cake, one icecream cone, one pickle, one slice of swiss cheese, one slice of salami, one lollypop, one piece of cherry pie, one sausage, one cupcake, and one slice of watermelon".getBytes)))
|
||||
assert(pr.fallbackAddress === Some("1RustyRX2oai4EYYDpQGWvEL62BBGqN9T"))
|
||||
assert(pr.routingInfo() === List(RoutingInfoTag(List(
|
||||
assert(pr.routingInfo() === List(List(
|
||||
ExtraHop(PublicKey("029e03a901b85534ff1e92c43c74431f7ce72046060fcf7a95c37e148f78c77255"), 72623859790382856L, 1, 20, 3),
|
||||
ExtraHop(PublicKey("039e03a901b85534ff1e92c43c74431f7ce72046060fcf7a95c37e148f78c77255"), 217304205466536202L, 2, 30, 4)
|
||||
))))
|
||||
)))
|
||||
assert(BinaryData(Protocol.writeUInt64(0x0102030405060708L, ByteOrder.BIG_ENDIAN)) == BinaryData("0102030405060708"))
|
||||
assert(BinaryData(Protocol.writeUInt64(0x030405060708090aL, ByteOrder.BIG_ENDIAN)) == BinaryData("030405060708090a"))
|
||||
assert(pr.tags.size == 4)
|
||||
|
@ -177,51 +177,6 @@ class RouteCalculationSpec extends FunSuite {
|
||||
assert(hops === Hop(a, b, uab) :: Hop(b, c, ubc) :: Hop(c, d, ucd) :: Hop(d, e, ude) :: Nil)
|
||||
}
|
||||
|
||||
test("calculate route with extra hops") {
|
||||
// E (sender) -> D - public -> C - private -> B - private -> A (receiver)
|
||||
|
||||
val amount = MilliSatoshi(100000000L)
|
||||
val paymentPreimage = BinaryData("0" * 32)
|
||||
val paymentHash = Crypto.sha256(paymentPreimage)
|
||||
val privateKey = PrivateKey("bb77027e3b6ef55f3b16eb6973d124f68e0c2afc16accc00a44ec6b3d1e58cc601")
|
||||
|
||||
// Ask router for a route from 02f0b230e53723ccc331db140edc518be1ee5ab29a508104a4be2f5be922c928e8 (node C)
|
||||
// to 0299439d988cbf31388d59e3d6f9e184e7a0739b8b8fcdc298957216833935f9d3 (node A)
|
||||
val hopCB = Hop(PublicKey("02f0b230e53723ccc331db140edc518be1ee5ab29a508104a4be2f5be922c928e8"),
|
||||
PublicKey("032b4af42b5e8089a7a06005ead9ac4667527390ee39c998b7b0307f0d81d7f4ac"),
|
||||
ChannelUpdate("3044022075bc283539935b1bc126035ef98d0f9bcd5dd7b0832b0a6175dc14a5ee12d47102203d141a4da4f83fca9d65bddfb9ee6ea5cdfcdb364de062d1370500f511b8370701",
|
||||
"06226e46111a0b59caaf126043eb5bbf28c34f3a5e332a1fc7b2b73cf188910f", 24412456671576064L, 1509366313, BinaryData("0000"), 144, 1000, 546000, 10))
|
||||
|
||||
val hopBA = Hop(PublicKey("032b4af42b5e8089a7a06005ead9ac4667527390ee39c998b7b0307f0d81d7f4ac"),
|
||||
PublicKey("0299439d988cbf31388d59e3d6f9e184e7a0739b8b8fcdc298957216833935f9d3"),
|
||||
ChannelUpdate("304402205e9b28e26add5417ad97f6eb161229dd7db0d7848e146a1856a8841238bc627902203cc59996ca490375fd76a3327adfb7c5150ee3288ad1663b8c4fbe8908eb489a01",
|
||||
"06226e46111a0b59caaf126043eb5bbf28c34f3a5e332a1fc7b2b73cf188910f", 23366821113626624L, 1509455356, BinaryData("0001"), 144, 1000, 546000, 10))
|
||||
|
||||
val reverseRoute = List(hopBA, hopCB)
|
||||
val extraRoute = PaymentHop.buildExtra(reverseRoute, amount.amount)
|
||||
|
||||
assert(extraRoute === List(
|
||||
ExtraHop(PublicKey("02f0b230e53723ccc331db140edc518be1ee5ab29a508104a4be2f5be922c928e8"), 24412456671576064L, 546000, 10, 144),
|
||||
ExtraHop(PublicKey("032b4af42b5e8089a7a06005ead9ac4667527390ee39c998b7b0307f0d81d7f4ac"), 23366821113626624L, 546000, 10, 144))
|
||||
)
|
||||
|
||||
// Sender side
|
||||
|
||||
// Ask router for a route D -> C
|
||||
val hopDC = Hop(PublicKey("03c1b07dbe10e178216150b49646ded556466ed15368857fa721cf1acd9d9a6f24"),
|
||||
PublicKey("02f0b230e53723ccc331db140edc518be1ee5ab29a508104a4be2f5be922c928e8"),
|
||||
ChannelUpdate("3044022060c1034092d4e41d75271eb619ef0a0f00d0b5a61c4245e0f14eeac91a3c823202200da9c8b8067e73c32aea41cb9eec050ce49cb944877d9abb3b08be2dea92497301",
|
||||
"06226e46111a0b59caaf126043eb5bbf28c34f3a5e332a1fc7b2b73cf188910f", 24403660578553856L, 1509456040, BinaryData("0001"), 144, 1000, 546000, 10))
|
||||
|
||||
val (amt, expiry, payloads) = PaymentLifecycle.buildPayloads(amount.amount, 10, Seq(hopDC) ++ extraRoute)
|
||||
|
||||
assert(payloads === List(PerHopPayload(24403660578553856L, 101094005L, 298),
|
||||
PerHopPayload(24412456671576064L, 100547000L, 154), PerHopPayload(23366821113626624L, 100000000L, 10), PerHopPayload(0L, 100000000L, 10)))
|
||||
|
||||
assert(amt == 101641015L)
|
||||
assert(expiry == 442)
|
||||
}
|
||||
|
||||
test("stale channels pruning") {
|
||||
// set current block height
|
||||
Globals.blockCount.set(500000)
|
||||
@ -265,4 +220,29 @@ class RouteCalculationSpec extends FunSuite {
|
||||
|
||||
}
|
||||
|
||||
test("convert extra hops to channel_update") {
|
||||
val a = randomKey.publicKey
|
||||
val b = randomKey.publicKey
|
||||
val c = randomKey.publicKey
|
||||
val d = randomKey.publicKey
|
||||
val e = randomKey.publicKey
|
||||
|
||||
val extraHop1 = ExtraHop(a, 1, 10, 11, 12)
|
||||
val extraHop2 = ExtraHop(b, 2, 20, 21, 22)
|
||||
val extraHop3 = ExtraHop(c, 3, 30, 31, 32)
|
||||
val extraHop4 = ExtraHop(d, 4, 40, 41, 42)
|
||||
|
||||
val extraHops = extraHop1 :: extraHop2 :: extraHop3 :: extraHop4 :: Nil
|
||||
|
||||
val fakeUpdates = Router.toFakeUpdates(extraHops, e)
|
||||
|
||||
assert(fakeUpdates == Map(
|
||||
ChannelDesc(extraHop1.shortChannelId, a, b) -> Router.toFakeUpdate(extraHop1),
|
||||
ChannelDesc(extraHop2.shortChannelId, b, c) -> Router.toFakeUpdate(extraHop2),
|
||||
ChannelDesc(extraHop3.shortChannelId, c, d) -> Router.toFakeUpdate(extraHop3),
|
||||
ChannelDesc(extraHop4.shortChannelId, d, e) -> Router.toFakeUpdate(extraHop4)
|
||||
))
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -6,6 +6,7 @@ import fr.acinq.bitcoin.Script.{pay2wsh, write}
|
||||
import fr.acinq.bitcoin.{Block, Satoshi, Transaction, TxOut}
|
||||
import fr.acinq.eclair.blockchain._
|
||||
import fr.acinq.eclair.channel.BITCOIN_FUNDING_EXTERNAL_CHANNEL_SPENT
|
||||
import fr.acinq.eclair.payment.PaymentRequest.ExtraHop
|
||||
import fr.acinq.eclair.router.Announcements.makeChannelUpdate
|
||||
import fr.acinq.eclair.transactions.Scripts
|
||||
import fr.acinq.eclair.wire.{ChannelAnnouncement, ChannelUpdate, Error, NodeAnnouncement}
|
||||
@ -131,6 +132,20 @@ class RouterSpec extends BaseRouterSpec {
|
||||
assert(res.hops.last.nextNodeId === d)
|
||||
}
|
||||
|
||||
test("route found (with extra routing info)") { case (router, _) =>
|
||||
val sender = TestProbe()
|
||||
val x = randomKey.publicKey
|
||||
val y = randomKey.publicKey
|
||||
val z = randomKey.publicKey
|
||||
val extraHop_cx = ExtraHop(c, 1, 10, 11, 12)
|
||||
val extraHop_xy = ExtraHop(x, 1, 10, 11, 12)
|
||||
val extraHop_yz = ExtraHop(y, 2, 20, 21, 22)
|
||||
sender.send(router, RouteRequest(a, z, assistedRoutes = Seq(extraHop_cx :: extraHop_xy :: extraHop_yz :: Nil)))
|
||||
val res = sender.expectMsgType[RouteResponse]
|
||||
assert(res.hops.map(_.nodeId).toList === a :: b :: c :: x :: y :: Nil)
|
||||
assert(res.hops.last.nextNodeId === z)
|
||||
}
|
||||
|
||||
test("route not found (channel disabled)") { case (router, _) =>
|
||||
val sender = TestProbe()
|
||||
sender.send(router, RouteRequest(a, d))
|
||||
|
@ -57,15 +57,19 @@ class Handlers(fKit: Future[Kit])(implicit ec: ExecutionContext = ExecutionConte
|
||||
}
|
||||
}
|
||||
|
||||
def send(nodeId: PublicKey, paymentHash: BinaryData, amountMsat: Long, minFinalCltvExpiry: Option[Long]) = {
|
||||
logger.info(s"sending $amountMsat to $paymentHash @ $nodeId")
|
||||
val request = minFinalCltvExpiry match {
|
||||
case None => SendPayment(amountMsat, paymentHash, nodeId)
|
||||
case Some(value) => SendPayment(amountMsat, paymentHash, nodeId, value)
|
||||
def send(overrideAmountMsat_opt: Option[Long], req: PaymentRequest) = {
|
||||
val amountMsat = overrideAmountMsat_opt
|
||||
.orElse(req.amount.map(_.amount))
|
||||
.getOrElse(throw new RuntimeException("you need to manually specify an amount for this payment request"))
|
||||
|
||||
logger.info(s"sending $amountMsat to ${req.paymentHash} @ ${req.nodeId}")
|
||||
val sendPayment = req.minFinalCltvExpiry match {
|
||||
case None => SendPayment(amountMsat, req.paymentHash, req.nodeId, req.routingInfo())
|
||||
case Some(minFinalCltvExpiry) => SendPayment(amountMsat, req.paymentHash, req.nodeId, req.routingInfo(), minFinalCltvExpiry = minFinalCltvExpiry)
|
||||
}
|
||||
(for {
|
||||
kit <- fKit
|
||||
res <- (kit.paymentInitiator ? request).mapTo[PaymentResult]
|
||||
res <- (kit.paymentInitiator ? sendPayment).mapTo[PaymentResult]
|
||||
} yield res)
|
||||
.onComplete {
|
||||
case Success(_: PaymentSucceeded) =>
|
||||
|
@ -68,14 +68,15 @@ class SendPaymentController(val handlers: Handlers, val stage: Stage) extends Lo
|
||||
}
|
||||
|
||||
@FXML def handleSend(event: ActionEvent) = {
|
||||
(Try(MilliSatoshi(amountField.getText().toLong)), Try(PaymentRequest.read(paymentRequest.getText))) match {
|
||||
(Try(amountField.getText().toLong), Try(PaymentRequest.read(paymentRequest.getText))) match {
|
||||
case (Success(amountMsat), Success(pr)) =>
|
||||
Try(handlers.send(pr.nodeId, pr.paymentHash, amountMsat.amount, pr.minFinalCltvExpiry)) match {
|
||||
case Success(s) => stage.close
|
||||
// we always override the payment request amount with the one from the UI
|
||||
Try(handlers.send(Some(amountMsat), pr)) match {
|
||||
case Success(_) => stage.close
|
||||
case Failure(f) => paymentRequestError.setText(s"Invalid Payment Request: ${f.getMessage}")
|
||||
}
|
||||
case (_, Success(_)) => amountFieldError.setText("Invalid amount")
|
||||
case (_, Failure(f)) => paymentRequestError.setText("Could not read this payment request")
|
||||
case (_, Failure(_)) => paymentRequestError.setText("Could not read this payment request")
|
||||
}
|
||||
}
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user