1
0
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:
Anton Kumaigorodski 2017-12-20 20:14:24 +02:00 committed by Pierre-Marie Padiou
parent 7d357662f3
commit 63b772cf2a
15 changed files with 119 additions and 141 deletions

View File

@ -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

View File

@ -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
}

View File

@ -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

View File

@ -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)
}
/**

View File

@ -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

View File

@ -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

View File

@ -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}

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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)

View File

@ -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)
))
}
}

View File

@ -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))

View File

@ -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) =>

View File

@ -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")
}
}