mirror of
https://github.com/ACINQ/eclair.git
synced 2024-11-20 10:39:19 +01:00
Routing heuristics (#821)
* Add route-weight-ratios to SendPayment/RouteRequest * Update test channel_update with real world fee values * Add maxFeeBase and maxFeePct to SendCommand, use high fee tolerance in integration test * Expose randomized route selection feature in SendPayment, used in integration test too * Add maxCltv to SendPayment/RouteRequest * Implement boundaries for graph searching with cost, cltv, and size * Enable searching for routes with size/CLTV/fee limits * Expose RouteParams in RouteRequest and SendPayment * If we couldn't find a route on the first attempt, retry relaxing the restriction on the route size * Avoid returning an empty path, collapse the route not found cases into one * When retrying to search for a route, relax 'maxCltv' * Group search params configurations into a block * Use the returning edges in 'ignoredEdges' when looking for a spur path * Log path-finding params when receiving a route request * Enforce weight ratios to be between (0,1] * Make path-finding heuristics optional
This commit is contained in:
parent
32d0500ef9
commit
fe31f2d9d2
@ -103,6 +103,13 @@ eclair {
|
||||
max-cltv = 1008 // max acceptable cltv expiry for the payment (1008 ~ 1 week)
|
||||
fee-threshold-sat = 21 // if fee is below this value we skip the max-fee-pct check
|
||||
max-fee-pct = 0.03 // route will be discarded if fee is above this value (in percentage relative to the total payment amount); doesn't apply if fee < fee-threshold-sat
|
||||
|
||||
// channel 'weight' is computed with the following formula: channelFee * (cltvDelta * ratio-cltv + channelAge * ratio-channel-age + channelCapacity * ratio-channel-capacity)
|
||||
// the following parameters can be used to ask the router to use heuristics to find i.e: 'cltv-optimized' routes, **the sum of the three ratios must be > 0 and <= 1**
|
||||
heuristics-enable = true // if true uses heuristics for path-finding
|
||||
ratio-cltv = 0.15 // when computing the weight for a channel, consider its CLTV delta in this proportion
|
||||
ratio-channel-age = 0.35 // when computing the weight for a channel, consider its AGE in this proportion
|
||||
ratio-channel-capacity = 0.5 // when computing the weight for a channel, consider its CAPACITY in this proportion
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -246,7 +246,11 @@ object NodeParams {
|
||||
searchMaxRouteLength = config.getInt("router.path-finding.max-route-length"),
|
||||
searchMaxCltv = config.getInt("router.path-finding.max-cltv"),
|
||||
searchMaxFeeBaseSat = config.getLong("router.path-finding.fee-threshold-sat"),
|
||||
searchMaxFeePct = config.getDouble("router.path-finding.max-fee-pct")
|
||||
searchMaxFeePct = config.getDouble("router.path-finding.max-fee-pct"),
|
||||
searchHeuristicsEnabled = config.getBoolean("router.path-finding.heuristics-enable"),
|
||||
searchRatioCltv = config.getDouble("router.path-finding.ratio-cltv"),
|
||||
searchRatioChannelAge = config.getDouble("router.path-finding.ratio-channel-age"),
|
||||
searchRatioChannelCapacity = config.getDouble("router.path-finding.ratio-channel-capacity")
|
||||
),
|
||||
socksProxy_opt = socksProxy_opt
|
||||
)
|
||||
|
@ -54,7 +54,7 @@ class Autoprobe(nodeParams: NodeParams, router: ActorRef, paymentInitiator: Acto
|
||||
case Some(targetNodeId) =>
|
||||
val paymentHash = randomBytes(32) // we don't even know the preimage (this needs to be a secure random!)
|
||||
log.info(s"sending payment probe to node=$targetNodeId payment_hash=$paymentHash")
|
||||
paymentInitiator ! SendPayment(PAYMENT_AMOUNT_MSAT, paymentHash, targetNodeId, maxAttempts = 1)
|
||||
paymentInitiator ! SendPayment(PAYMENT_AMOUNT_MSAT, paymentHash, targetNodeId, maxAttempts = 1, randomize = Some(true))
|
||||
case None =>
|
||||
log.info(s"could not find a destination, re-scheduling")
|
||||
scheduleProbe()
|
||||
|
@ -19,14 +19,21 @@ package fr.acinq.eclair.router
|
||||
import fr.acinq.bitcoin.Crypto.PublicKey
|
||||
import scala.collection.mutable
|
||||
import fr.acinq.eclair._
|
||||
import fr.acinq.eclair.router.Graph.GraphStructure.{DirectedGraph, GraphEdge}
|
||||
import fr.acinq.eclair.wire.ChannelUpdate
|
||||
import Router._
|
||||
import fr.acinq.eclair.channel.Channel
|
||||
import fr.acinq.eclair.router.Graph.GraphStructure.{DirectedGraph, GraphEdge}
|
||||
|
||||
object Graph {
|
||||
|
||||
// @formatter:off
|
||||
case class RichWeight(cost: Long, length: Int, cltv: Int)
|
||||
// A compound weight for an edge, weight is obtained with (cost X factor),'cost' contains the actual amount+fees in millisatoshi, 'cltvCumulative' the total CLTV necessary to reach this edge
|
||||
case class RichWeight(cost: Long, length: Int, cltv: Int, weight: Double) extends Ordered[RichWeight] {
|
||||
override def compare(that: RichWeight): Int = this.weight.compareTo(that.weight)
|
||||
}
|
||||
case class WeightRatios(cltvDeltaFactor: Double, ageFactor: Double, capacityFactor: Double) {
|
||||
require(0 < cltvDeltaFactor + ageFactor + capacityFactor && cltvDeltaFactor + ageFactor + capacityFactor <= 1, "The sum of heuristics ratios must be between 0 and 1 (included)")
|
||||
}
|
||||
case class WeightedNode(key: PublicKey, weight: RichWeight)
|
||||
case class WeightedPath(path: Seq[GraphEdge], weight: RichWeight)
|
||||
// @formatter:on
|
||||
@ -37,14 +44,14 @@ object Graph {
|
||||
*/
|
||||
object QueueComparator extends Ordering[WeightedNode] {
|
||||
override def compare(x: WeightedNode, y: WeightedNode): Int = {
|
||||
val weightCmp = x.weight.cost.compareTo(y.weight.cost)
|
||||
val weightCmp = x.weight.compareTo(y.weight)
|
||||
if (weightCmp == 0) x.key.toString().compareTo(y.key.toString())
|
||||
else weightCmp
|
||||
}
|
||||
}
|
||||
|
||||
implicit object PathComparator extends Ordering[WeightedPath] {
|
||||
override def compare(x: WeightedPath, y: WeightedPath): Int = y.weight.cost.compareTo(x.weight.cost)
|
||||
override def compare(x: WeightedPath, y: WeightedPath): Int = y.weight.compare(x.weight)
|
||||
}
|
||||
|
||||
/**
|
||||
@ -56,9 +63,21 @@ object Graph {
|
||||
* @param targetNode
|
||||
* @param amountMsat
|
||||
* @param pathsToFind
|
||||
* @param wr an object containing the ratios used to 'weight' edges when searching for the shortest path
|
||||
* @param currentBlockHeight the height of the chain tip (latest block)
|
||||
* @param boundaries a predicate function that can be used to impose limits on the outcome of the search
|
||||
* @return
|
||||
*/
|
||||
def yenKshortestPaths(graph: DirectedGraph, sourceNode: PublicKey, targetNode: PublicKey, amountMsat: Long, ignoredEdges: Set[ChannelDesc], extraEdges: Set[GraphEdge], pathsToFind: Int, boundaries: RichWeight => Boolean): Seq[WeightedPath] = {
|
||||
def yenKshortestPaths(graph: DirectedGraph,
|
||||
sourceNode: PublicKey,
|
||||
targetNode: PublicKey,
|
||||
amountMsat: Long,
|
||||
ignoredEdges: Set[ChannelDesc],
|
||||
extraEdges: Set[GraphEdge],
|
||||
pathsToFind: Int,
|
||||
wr: Option[WeightRatios],
|
||||
currentBlockHeight: Long,
|
||||
boundaries: RichWeight => Boolean): Seq[WeightedPath] = {
|
||||
|
||||
var allSpurPathsFound = false
|
||||
|
||||
@ -68,8 +87,8 @@ object Graph {
|
||||
val candidates = new mutable.PriorityQueue[WeightedPath]
|
||||
|
||||
// find the shortest path, k = 0
|
||||
val shortestPath = dijkstraShortestPath(graph, sourceNode, targetNode, amountMsat, ignoredEdges, extraEdges, RichWeight(amountMsat, 0, 0), boundaries)
|
||||
shortestPaths += WeightedPath(shortestPath, pathWeight(shortestPath, amountMsat, isPartial = false))
|
||||
val shortestPath = dijkstraShortestPath(graph, sourceNode, targetNode, amountMsat, ignoredEdges, extraEdges, RichWeight(amountMsat, 0, 0, 0), boundaries, currentBlockHeight, wr)
|
||||
shortestPaths += WeightedPath(shortestPath, pathWeight(shortestPath, amountMsat, isPartial = false, currentBlockHeight, wr))
|
||||
|
||||
// avoid returning a list with an empty path
|
||||
if (shortestPath.isEmpty) return Seq.empty
|
||||
@ -89,7 +108,7 @@ object Graph {
|
||||
|
||||
// select the sub-path from the source to the spur node of the k-th previous shortest path
|
||||
val rootPathEdges = if (i == 0) prevShortestPath.head :: Nil else prevShortestPath.take(i)
|
||||
val rootPathWeight = pathWeight(rootPathEdges, amountMsat, isPartial = true)
|
||||
val rootPathWeight = pathWeight(rootPathEdges, amountMsat, isPartial = true, currentBlockHeight, wr)
|
||||
|
||||
// links to be removed that are part of the previous shortest path and which share the same root path
|
||||
val edgesToIgnore = shortestPaths.flatMap { weightedPath =>
|
||||
@ -104,7 +123,7 @@ object Graph {
|
||||
val returningEdges = rootPathEdges.lastOption.map(last => graph.getEdgesBetween(last.desc.b, last.desc.a)).toSeq.flatten.map(_.desc)
|
||||
|
||||
// find the "spur" path, a sub-path going from the spur edge to the target avoiding previously found sub-paths
|
||||
val spurPath = dijkstraShortestPath(graph, spurEdge.desc.a, targetNode, amountMsat, ignoredEdges ++ edgesToIgnore.toSet ++ returningEdges, extraEdges, rootPathWeight, boundaries)
|
||||
val spurPath = dijkstraShortestPath(graph, spurEdge.desc.a, targetNode, amountMsat, ignoredEdges ++ edgesToIgnore.toSet ++ returningEdges.toSet, extraEdges, rootPathWeight, boundaries, currentBlockHeight, wr)
|
||||
|
||||
// if there wasn't a path the spur will be empty
|
||||
if (spurPath.nonEmpty) {
|
||||
@ -115,7 +134,7 @@ object Graph {
|
||||
case false => rootPathEdges ++ spurPath
|
||||
}
|
||||
|
||||
val candidatePath = WeightedPath(totalPath, pathWeight(totalPath, amountMsat, isPartial = false))
|
||||
val candidatePath = WeightedPath(totalPath, pathWeight(totalPath, amountMsat, isPartial = false, currentBlockHeight, wr))
|
||||
|
||||
if (boundaries(candidatePath.weight) && !shortestPaths.contains(candidatePath) && !candidates.exists(_ == candidatePath)) {
|
||||
candidates.enqueue(candidatePath)
|
||||
@ -142,12 +161,15 @@ object Graph {
|
||||
* the shortest path from the target to the source (this is because we want to calculate the weight of the
|
||||
* edges correctly). The graph @param g is optimized for querying the incoming edges given a vertex.
|
||||
*
|
||||
* @param g the graph on which will be performed the search
|
||||
* @param sourceNode the starting node of the path we're looking for
|
||||
* @param targetNode the destination node of the path
|
||||
* @param amountMsat the amount (in millisatoshis) we want to transmit
|
||||
* @param ignoredEdges a list of edges we do not want to consider
|
||||
* @param extraEdges a list of extra edges we want to consider but are not currently in the graph
|
||||
* @param g the graph on which will be performed the search
|
||||
* @param sourceNode the starting node of the path we're looking for
|
||||
* @param targetNode the destination node of the path
|
||||
* @param amountMsat the amount (in millisatoshis) we want to transmit
|
||||
* @param ignoredEdges a list of edges we do not want to consider
|
||||
* @param extraEdges a list of extra edges we want to consider but are not currently in the graph
|
||||
* @param wr an object containing the ratios used to 'weight' edges when searching for the shortest path
|
||||
* @param currentBlockHeight the height of the chain tip (latest block)
|
||||
* @param boundaries a predicate function that can be used to impose limits on the outcome of the search
|
||||
* @return
|
||||
*/
|
||||
|
||||
@ -158,7 +180,9 @@ object Graph {
|
||||
ignoredEdges: Set[ChannelDesc],
|
||||
extraEdges: Set[GraphEdge],
|
||||
initialWeight: RichWeight,
|
||||
boundaries: RichWeight => Boolean): Seq[GraphEdge] = {
|
||||
boundaries: RichWeight => Boolean,
|
||||
currentBlockHeight: Long,
|
||||
wr: Option[WeightRatios]): Seq[GraphEdge] = {
|
||||
|
||||
// the graph does not contain source/destination nodes
|
||||
if (!g.containsVertex(sourceNode)) return Seq.empty
|
||||
@ -167,12 +191,12 @@ object Graph {
|
||||
val maxMapSize = 100 // conservative estimation to avoid over allocating memory
|
||||
|
||||
// this is not the actual optimal size for the maps, because we only put in there all the vertices in the worst case scenario.
|
||||
val cost = new java.util.HashMap[PublicKey, RichWeight](maxMapSize)
|
||||
val weight = new java.util.HashMap[PublicKey, RichWeight](maxMapSize)
|
||||
val prev = new java.util.HashMap[PublicKey, GraphEdge](maxMapSize)
|
||||
val vertexQueue = new org.jheaps.tree.SimpleFibonacciHeap[WeightedNode, Short](QueueComparator)
|
||||
|
||||
// initialize the queue and cost array with the initial weight
|
||||
cost.put(targetNode, initialWeight)
|
||||
weight.put(targetNode, initialWeight)
|
||||
vertexQueue.insert(WeightedNode(targetNode, initialWeight))
|
||||
|
||||
var targetFound = false
|
||||
@ -187,39 +211,38 @@ object Graph {
|
||||
// build the neighbors with optional extra edges
|
||||
val currentNeighbors = extraEdges.isEmpty match {
|
||||
case true => g.getIncomingEdgesOf(current.key)
|
||||
case false => g.getIncomingEdgesOf(current.key) ++ extraEdges.filter(_.desc.b == current.key)
|
||||
case false =>
|
||||
val extraNeighbors = extraEdges.filter(_.desc.b == current.key)
|
||||
// the resulting set must have only one element per shortChannelId
|
||||
g.getIncomingEdgesOf(current.key).filterNot(e => extraNeighbors.exists(_.desc.shortChannelId == e.desc.shortChannelId)) ++ extraNeighbors
|
||||
}
|
||||
|
||||
val currentWeight = cost.get(current.key)
|
||||
// note: there is always an entry for the current in the 'weight' map
|
||||
val currentWeight = weight.get(current.key)
|
||||
|
||||
// for each neighbor
|
||||
currentNeighbors.foreach { edge =>
|
||||
|
||||
val neighbor = edge.desc.a
|
||||
|
||||
// note: 'cost' contains the smallest known cumulative cost (amount + fees) necessary to reach 'current' so far
|
||||
// note: there is always an entry for the current in the 'cost' map
|
||||
val newMinimumKnownWeight = RichWeight(
|
||||
cost = edgeWeight(edge, currentWeight.cost, initialWeight.length == 0 && neighbor == sourceNode),
|
||||
length = currentWeight.length + 1,
|
||||
cltv = currentWeight.cltv + edge.update.cltvExpiryDelta
|
||||
)
|
||||
// note: 'newMinimumKnownWeight' contains the smallest known cumulative cost (amount + fees) necessary to reach 'current' so far
|
||||
val newMinimumKnownWeight = edgeWeight(edge, currentWeight, initialWeight.length == 0 && neighbor == sourceNode, currentBlockHeight, wr)
|
||||
|
||||
// test for ignored edges
|
||||
if (edge.update.htlcMaximumMsat.forall(newMinimumKnownWeight.cost <= _) &&
|
||||
newMinimumKnownWeight.cost >= edge.update.htlcMinimumMsat &&
|
||||
boundaries(newMinimumKnownWeight) && // ignore this edge if it violates the boundary checks
|
||||
if (edge.update.htlcMaximumMsat.forall(newMinimumKnownWeight.cost + amountMsat <= _) &&
|
||||
newMinimumKnownWeight.cost + amountMsat >= edge.update.htlcMinimumMsat &&
|
||||
boundaries(newMinimumKnownWeight) && // check if this neighbor edge would break off the 'boundaries'
|
||||
!ignoredEdges.contains(edge.desc)
|
||||
) {
|
||||
|
||||
// we call containsKey first because "getOrDefault" is not available in JDK7
|
||||
val neighborCost = cost.containsKey(neighbor) match {
|
||||
case false => RichWeight(Long.MaxValue, 0, 0)
|
||||
case true => cost.get(neighbor)
|
||||
val neighborCost = weight.containsKey(neighbor) match {
|
||||
case false => RichWeight(Long.MaxValue, Int.MaxValue, Int.MaxValue, Double.MaxValue)
|
||||
case true => weight.get(neighbor)
|
||||
}
|
||||
|
||||
// if this neighbor has a shorter distance than previously known
|
||||
if (newMinimumKnownWeight.cost < neighborCost.cost) {
|
||||
if (newMinimumKnownWeight.weight < neighborCost.weight) {
|
||||
|
||||
// update the visiting tree
|
||||
prev.put(neighbor, edge)
|
||||
@ -228,7 +251,7 @@ object Graph {
|
||||
vertexQueue.insert(WeightedNode(neighbor, newMinimumKnownWeight)) // O(1)
|
||||
|
||||
// update the minimum known distance array
|
||||
cost.put(neighbor, newMinimumKnownWeight)
|
||||
weight.put(neighbor, newMinimumKnownWeight)
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -254,26 +277,81 @@ object Graph {
|
||||
}
|
||||
}
|
||||
|
||||
// Computes the compound weight for the given @param edge, the weight is cumulative and must account for the previous edge's weight.
|
||||
private def edgeWeight(edge: GraphEdge, prev: RichWeight, isNeighborTarget: Boolean, currentBlockHeight: Long, weightRatios: Option[WeightRatios]): RichWeight = weightRatios match {
|
||||
case None =>
|
||||
val edgeCost = if (isNeighborTarget) prev.cost else edgeFeeCost(edge, prev.cost)
|
||||
RichWeight(cost = edgeCost, length = prev.length + 1, cltv = prev.cltv + edge.update.cltvExpiryDelta, weight = edgeCost)
|
||||
|
||||
case Some(wr) =>
|
||||
import RoutingHeuristics._
|
||||
|
||||
// Every edge is weighted by funding block height where older blocks add less weight, the window considered is 2 months.
|
||||
val channelBlockHeight = ShortChannelId.coordinates(edge.desc.shortChannelId).blockHeight
|
||||
val ageFactor = normalize(channelBlockHeight, min = currentBlockHeight - BLOCK_TIME_TWO_MONTHS, max = currentBlockHeight)
|
||||
|
||||
// Every edge is weighted by channel capacity, larger channels add less weight
|
||||
val edgeMaxCapacity = edge.update.htlcMaximumMsat.getOrElse(CAPACITY_CHANNEL_LOW_MSAT)
|
||||
val capFactor = 1 - normalize(edgeMaxCapacity, CAPACITY_CHANNEL_LOW_MSAT, CAPACITY_CHANNEL_HIGH_MSAT)
|
||||
|
||||
// Every edge is weighted by its clvt-delta value, normalized
|
||||
val channelCltvDelta = edge.update.cltvExpiryDelta
|
||||
val cltvFactor = normalize(channelCltvDelta, CLTV_LOW, CLTV_HIGH)
|
||||
|
||||
// NB 'edgeCost' includes the amount to be sent plus the fees that must be paid to traverse this @param edge
|
||||
val edgeCost = if (isNeighborTarget) prev.cost else edgeFeeCost(edge, prev.cost)
|
||||
|
||||
// NB we're guaranteed to have weightRatios and factors > 0
|
||||
val factor = (cltvFactor * wr.cltvDeltaFactor) + (ageFactor * wr.ageFactor) + (capFactor * wr.capacityFactor)
|
||||
val edgeWeight = if (isNeighborTarget) prev.weight else edgeCost * factor
|
||||
|
||||
RichWeight(cost = edgeCost, length = prev.length + 1, cltv = prev.cltv + channelCltvDelta, weight = edgeWeight)
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @param edge the edge for which we want to compute the weight
|
||||
* @param amountWithFees the value that this edge will have to carry along
|
||||
* @param isNeighborSource true if the receiving vertex of this edge is the target node (source in a reversed graph), which has cost 0
|
||||
* @param edge the edge for which we want to compute the weight
|
||||
* @param amountWithFees the value that this edge will have to carry along
|
||||
* @return the new amount updated with the necessary fees for this edge
|
||||
*/
|
||||
private def edgeWeight(edge: GraphEdge, amountWithFees: Long, isNeighborSource: Boolean): Long = isNeighborSource match {
|
||||
case false => amountWithFees + nodeFee(edge.update.feeBaseMsat, edge.update.feeProportionalMillionths, amountWithFees)
|
||||
case true => amountWithFees
|
||||
private def edgeFeeCost(edge: GraphEdge, amountWithFees: Long): Long = {
|
||||
amountWithFees + nodeFee(edge.update.feeBaseMsat, edge.update.feeProportionalMillionths, amountWithFees)
|
||||
}
|
||||
|
||||
// Calculates the total cost of a path (amount + fees), direct channels with the source will have a cost of 0 (pay no fees)
|
||||
def pathWeight(path: Seq[GraphEdge], amountMsat: Long, isPartial: Boolean): RichWeight = {
|
||||
path.drop(if (isPartial) 0 else 1).foldRight(RichWeight(amountMsat, 0, 0)) { (edge, prev) =>
|
||||
RichWeight(
|
||||
cost = edgeWeight(edge, prev.cost, isNeighborSource = false),
|
||||
cltv = prev.cltv + edge.update.cltvExpiryDelta,
|
||||
length = prev.length + 1
|
||||
)
|
||||
def pathWeight(path: Seq[GraphEdge], amountMsat: Long, isPartial: Boolean, currentBlockHeight: Long, wr: Option[WeightRatios]): RichWeight = {
|
||||
path.drop(if (isPartial) 0 else 1).foldRight(RichWeight(amountMsat, 0, 0, 0)) { (edge, prev) =>
|
||||
edgeWeight(edge, prev, false, currentBlockHeight, wr)
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
object RoutingHeuristics {
|
||||
|
||||
// Number of blocks in two months
|
||||
val BLOCK_TIME_TWO_MONTHS = 8640
|
||||
|
||||
// Low/High bound for channel capacity
|
||||
val CAPACITY_CHANNEL_LOW_MSAT = 1000 * 1000L // 1000 sat
|
||||
val CAPACITY_CHANNEL_HIGH_MSAT = Channel.MAX_FUNDING_SATOSHIS * 1000L
|
||||
|
||||
// Low/High bound for CLTV channel value
|
||||
val CLTV_LOW = 9
|
||||
val CLTV_HIGH = 2016
|
||||
|
||||
/**
|
||||
* Normalize the given value between (0, 1). If the @param value is outside the min/max window we flatten it to something very close to the
|
||||
* extremes but always bigger than zero so it's guaranteed to never return zero
|
||||
*
|
||||
* @param value
|
||||
* @param min
|
||||
* @param max
|
||||
* @return
|
||||
*/
|
||||
def normalize(value: Double, min: Double, max: Double) = {
|
||||
if (value <= min) 0.00001D
|
||||
else if (value > max) 0.99999D
|
||||
else (value - min) / (max - min)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -19,22 +19,19 @@ package fr.acinq.eclair.router
|
||||
import akka.Done
|
||||
import akka.actor.{ActorRef, Props, Status}
|
||||
import akka.event.Logging.MDC
|
||||
import akka.pattern.pipe
|
||||
import fr.acinq.bitcoin.BinaryData
|
||||
import fr.acinq.bitcoin.Crypto.PublicKey
|
||||
import fr.acinq.bitcoin.Script.{pay2wsh, write}
|
||||
import fr.acinq.eclair._
|
||||
import fr.acinq.eclair.blockchain._
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.crypto.TransportHandler
|
||||
import fr.acinq.eclair.io.Peer.{ChannelClosed, InvalidSignature, InvalidAnnouncement, PeerRoutingMessage}
|
||||
import fr.acinq.eclair.io.Peer.{ChannelClosed, InvalidAnnouncement, InvalidSignature, PeerRoutingMessage}
|
||||
import fr.acinq.eclair.payment.PaymentRequest.ExtraHop
|
||||
import fr.acinq.eclair.router.Graph.GraphStructure.DirectedGraph.graphEdgeToHop
|
||||
import fr.acinq.eclair.router.Graph.GraphStructure.{DirectedGraph, GraphEdge}
|
||||
import fr.acinq.eclair.router.Graph.{RichWeight, WeightedPath}
|
||||
import fr.acinq.eclair.router.Graph.{RichWeight, WeightRatios}
|
||||
import fr.acinq.eclair.transactions.Scripts
|
||||
import fr.acinq.eclair.wire._
|
||||
|
||||
import scala.collection.{SortedSet, mutable}
|
||||
import scala.collection.immutable.{SortedMap, TreeMap}
|
||||
import scala.compat.Platform
|
||||
@ -44,10 +41,21 @@ import scala.util.{Random, Try}
|
||||
|
||||
// @formatter:off
|
||||
|
||||
case class RouterConf(randomizeRouteSelection: Boolean, channelExcludeDuration: FiniteDuration, routerBroadcastInterval: FiniteDuration, searchMaxFeeBaseSat: Long, searchMaxFeePct: Double, searchMaxRouteLength: Int, searchMaxCltv: Int)
|
||||
case class RouterConf(randomizeRouteSelection: Boolean,
|
||||
channelExcludeDuration: FiniteDuration,
|
||||
routerBroadcastInterval: FiniteDuration,
|
||||
searchMaxFeeBaseSat: Long,
|
||||
searchMaxFeePct: Double,
|
||||
searchMaxRouteLength: Int,
|
||||
searchMaxCltv: Int,
|
||||
searchHeuristicsEnabled: Boolean,
|
||||
searchRatioCltv: Double,
|
||||
searchRatioChannelAge: Double,
|
||||
searchRatioChannelCapacity: Double)
|
||||
|
||||
case class ChannelDesc(shortChannelId: ShortChannelId, a: PublicKey, b: PublicKey)
|
||||
case class Hop(nodeId: PublicKey, nextNodeId: PublicKey, lastUpdate: ChannelUpdate)
|
||||
case class RouteParams(maxFeeBaseMsat: Long, maxFeePct: Double, routeMaxLength: Int, routeMaxCltv: Int)
|
||||
case class RouteParams(maxFeeBaseMsat: Long, maxFeePct: Double, routeMaxLength: Int, routeMaxCltv: Int, ratios: Option[WeightRatios])
|
||||
case class RouteRequest(source: PublicKey,
|
||||
target: PublicKey,
|
||||
amountMsat: Long,
|
||||
@ -56,6 +64,7 @@ case class RouteRequest(source: PublicKey,
|
||||
ignoreChannels: Set[ChannelDesc] = Set.empty,
|
||||
randomize: Option[Boolean] = None,
|
||||
routeParams: Option[RouteParams] = None)
|
||||
|
||||
case class RouteResponse(hops: Seq[Hop], ignoreNodes: Set[PublicKey], ignoreChannels: Set[ChannelDesc]) {
|
||||
require(hops.size > 0, "route cannot be empty")
|
||||
}
|
||||
@ -110,10 +119,18 @@ class Router(nodeParams: NodeParams, watcher: ActorRef, initialized: Option[Prom
|
||||
val SHORTID_WINDOW = 100
|
||||
|
||||
val defaultRouteParams = RouteParams(
|
||||
maxFeeBaseMsat = nodeParams.routerConf.searchMaxFeeBaseSat * 1000,
|
||||
maxFeeBaseMsat = nodeParams.routerConf.searchMaxFeeBaseSat * 1000, // converting sat -> msat
|
||||
maxFeePct = nodeParams.routerConf.searchMaxFeePct,
|
||||
routeMaxLength = nodeParams.routerConf.searchMaxRouteLength,
|
||||
routeMaxCltv = nodeParams.routerConf.searchMaxCltv
|
||||
routeMaxCltv = nodeParams.routerConf.searchMaxCltv,
|
||||
ratios = nodeParams.routerConf.searchHeuristicsEnabled match {
|
||||
case false => None
|
||||
case true => Some(WeightRatios(
|
||||
cltvDeltaFactor = nodeParams.routerConf.searchRatioCltv,
|
||||
ageFactor = nodeParams.routerConf.searchRatioChannelAge,
|
||||
capacityFactor = nodeParams.routerConf.searchRatioChannelCapacity
|
||||
))
|
||||
}
|
||||
)
|
||||
|
||||
val db = nodeParams.networkDb
|
||||
@ -397,11 +414,13 @@ class Router(nodeParams: NodeParams, watcher: ActorRef, initialized: Option[Prom
|
||||
// we also filter out updates corresponding to channels/nodes that are blacklisted for this particular request
|
||||
// TODO: in case of duplicates, d.updates will be overridden by assistedUpdates even if they are more recent!
|
||||
val ignoredUpdates = getIgnoredChannelDesc(d.updates ++ d.privateUpdates ++ assistedUpdates, ignoreNodes) ++ ignoreChannels ++ d.excludedChannels
|
||||
log.info(s"finding a route $start->$end with assistedChannels={} ignoreNodes={} ignoreChannels={} excludedChannels={}", assistedUpdates.keys.mkString(","), ignoreNodes.map(_.toBin).mkString(","), ignoreChannels.mkString(","), d.excludedChannels.mkString(","))
|
||||
val extraEdges = assistedUpdates.map { case (c, u) => GraphEdge(c, u) }.toSet
|
||||
// if we want to randomize we ask the router to make a random selection among the three best routes
|
||||
val routesToFind = if (randomize_opt.getOrElse(nodeParams.routerConf.randomizeRouteSelection)) DEFAULT_ROUTES_COUNT else 1
|
||||
findRoute(d.graph, start, end, amount, numRoutes = routesToFind, extraEdges = extraEdges, ignoredEdges = ignoredUpdates.toSet, routeParams = params_opt.getOrElse(defaultRouteParams))
|
||||
val params = params_opt.getOrElse(defaultRouteParams)
|
||||
|
||||
log.info(s"finding a route $start->$end with assistedChannels={} ignoreNodes={} ignoreChannels={} excludedChannels={}", assistedUpdates.keys.mkString(","), ignoreNodes.map(_.toBin).mkString(","), ignoreChannels.mkString(","), d.excludedChannels.mkString(","))
|
||||
log.info(s"finding a route with randomize={} params={}", routesToFind > 1, params)
|
||||
findRoute(d.graph, start, end, amount, numRoutes = routesToFind, extraEdges = extraEdges, ignoredEdges = ignoredUpdates.toSet, routeParams = params)
|
||||
.map(r => sender ! RouteResponse(r, ignoreNodes, ignoreChannels))
|
||||
.recover { case t => sender ! Status.Failure(t) }
|
||||
stay
|
||||
@ -821,6 +840,7 @@ object Router {
|
||||
* @param extraEdges a set of extra edges we want to CONSIDER during the search
|
||||
* @param ignoredEdges a set of extra edges we want to IGNORE during the search
|
||||
* @param routeParams a set of parameters that can restrict the route search
|
||||
* @param wr an object containing the ratios used to 'weight' edges when searching for the shortest path
|
||||
* @return the computed route to the destination @targetNodeId
|
||||
*/
|
||||
def findRoute(g: DirectedGraph,
|
||||
@ -834,13 +854,15 @@ object Router {
|
||||
|
||||
if (localNodeId == targetNodeId) throw CannotRouteToSelf
|
||||
|
||||
val currentBlockHeight = Globals.blockCount.get()
|
||||
|
||||
val boundaries: RichWeight => Boolean = { weight =>
|
||||
((weight.cost - amountMsat) < routeParams.maxFeeBaseMsat || (weight.cost - amountMsat) < (routeParams.maxFeePct * amountMsat)) &&
|
||||
weight.length <= routeParams.routeMaxLength && weight.length <= ROUTE_MAX_LENGTH &&
|
||||
weight.cltv <= routeParams.routeMaxCltv
|
||||
}
|
||||
|
||||
val foundRoutes = Graph.yenKshortestPaths(g, localNodeId, targetNodeId, amountMsat, ignoredEdges, extraEdges, numRoutes, boundaries).toList match {
|
||||
val foundRoutes = Graph.yenKshortestPaths(g, localNodeId, targetNodeId, amountMsat, ignoredEdges, extraEdges, numRoutes, routeParams.ratios, currentBlockHeight, boundaries).toList match {
|
||||
case Nil if routeParams.routeMaxLength < ROUTE_MAX_LENGTH => // if not found within the constraints we relax and repeat the search
|
||||
return findRoute(g, localNodeId, targetNodeId, amountMsat, numRoutes, extraEdges, ignoredEdges, routeParams.copy(routeMaxLength = ROUTE_MAX_LENGTH, routeMaxCltv = DEFAULT_ROUTE_MAX_CLTV))
|
||||
case Nil => throw RouteNotFound
|
||||
|
@ -93,7 +93,11 @@ object TestConstants {
|
||||
searchMaxFeeBaseSat = 21,
|
||||
searchMaxFeePct = 0.03,
|
||||
searchMaxCltv = 2016,
|
||||
searchMaxRouteLength = 20
|
||||
searchMaxRouteLength = 20,
|
||||
searchHeuristicsEnabled = false,
|
||||
searchRatioCltv = 0.0,
|
||||
searchRatioChannelAge = 0.0,
|
||||
searchRatioChannelCapacity = 0.0
|
||||
),
|
||||
socksProxy_opt = None
|
||||
)
|
||||
@ -160,7 +164,11 @@ object TestConstants {
|
||||
searchMaxFeeBaseSat = 21,
|
||||
searchMaxFeePct = 0.03,
|
||||
searchMaxCltv = 2016,
|
||||
searchMaxRouteLength = 20
|
||||
searchMaxRouteLength = 20,
|
||||
searchHeuristicsEnabled = false,
|
||||
searchRatioCltv = 0.0,
|
||||
searchRatioChannelAge = 0.0,
|
||||
searchRatioChannelCapacity = 0.0
|
||||
),
|
||||
socksProxy_opt = None
|
||||
)
|
||||
|
@ -18,6 +18,7 @@ package fr.acinq.eclair.api
|
||||
|
||||
|
||||
import java.io.{File, FileOutputStream}
|
||||
|
||||
import akka.actor.{Actor, ActorSystem, Props, Scheduler}
|
||||
import org.scalatest.FunSuite
|
||||
import akka.http.scaladsl.model.StatusCodes._
|
||||
@ -33,6 +34,7 @@ import akka.http.scaladsl.model.ws.{Message, TextMessage}
|
||||
import akka.http.scaladsl.server.Route
|
||||
import akka.stream.scaladsl.Flow
|
||||
import fr.acinq.eclair.channel.Register.ForwardShortId
|
||||
import fr.acinq.eclair.router.{Graph, Router}
|
||||
import org.json4s.Formats
|
||||
import org.json4s.JsonAST.{JInt, JString}
|
||||
import org.json4s.jackson.Serialization
|
||||
|
@ -18,9 +18,8 @@ package fr.acinq.eclair.integration
|
||||
|
||||
import java.io.{File, PrintWriter}
|
||||
import java.util.Properties
|
||||
|
||||
import collection.JavaConversions._
|
||||
import akka.actor.{ActorRef, ActorSystem, Terminated}
|
||||
import akka.pattern.pipe
|
||||
import akka.testkit.{TestKit, TestProbe}
|
||||
import com.google.common.net.HostAndPort
|
||||
import com.typesafe.config.{Config, ConfigFactory}
|
||||
@ -37,6 +36,7 @@ import fr.acinq.eclair.io.Peer.{Disconnect, PeerRoutingMessage}
|
||||
import fr.acinq.eclair.io.{NodeURI, Peer}
|
||||
import fr.acinq.eclair.payment.PaymentLifecycle.{State => _, _}
|
||||
import fr.acinq.eclair.payment.{LocalPaymentHandler, PaymentRequest}
|
||||
import fr.acinq.eclair.router.Graph.WeightRatios
|
||||
import fr.acinq.eclair.router.Router.ROUTE_MAX_LENGTH
|
||||
import fr.acinq.eclair.router.{Announcements, AnnouncementsBatchValidationSpec, ChannelDesc, RouteParams}
|
||||
import fr.acinq.eclair.transactions.Transactions
|
||||
@ -60,7 +60,20 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
|
||||
|
||||
var nodes: Map[String, Kit] = Map()
|
||||
|
||||
val integrationTestRouteParams = Some(RouteParams(maxFeeBaseMsat = Long.MaxValue, maxFeePct = Double.MaxValue, routeMaxCltv = Int.MaxValue, routeMaxLength = ROUTE_MAX_LENGTH))
|
||||
// we override the default because these test were designed to use cost-optimized routes
|
||||
val integrationTestRouteParams = Some(RouteParams(
|
||||
maxFeeBaseMsat = Long.MaxValue,
|
||||
maxFeePct = Double.MaxValue,
|
||||
routeMaxCltv = Int.MaxValue,
|
||||
routeMaxLength = ROUTE_MAX_LENGTH,
|
||||
ratios = Some(WeightRatios(
|
||||
cltvDeltaFactor = 0.1,
|
||||
ageFactor = 0,
|
||||
capacityFactor = 0
|
||||
))
|
||||
))
|
||||
|
||||
val commonConfig = ConfigFactory.parseMap(Map("eclair.chain" -> "regtest", "eclair.server.public-ips.1" -> "127.0.0.1", "eclair.bitcoind.port" -> 28333, "eclair.bitcoind.rpcport" -> 28332, "eclair.bitcoind.zmqblock" -> "tcp://127.0.0.1:28334", "eclair.bitcoind.zmqtx" -> "tcp://127.0.0.1:28335", "eclair.mindepth-blocks" -> 2, "eclair.max-htlc-value-in-flight-msat" -> 100000000000L, "eclair.router.broadcast-interval" -> "2 second", "eclair.auto-reconnect" -> false))
|
||||
|
||||
implicit val formats = DefaultFormats
|
||||
|
||||
@ -112,17 +125,16 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
|
||||
|
||||
test("starting eclair nodes") {
|
||||
import collection.JavaConversions._
|
||||
val commonConfig = ConfigFactory.parseMap(Map("eclair.chain" -> "regtest", "eclair.spv" -> false, "eclair.server.public-ips.1" -> "127.0.0.1", "eclair.bitcoind.port" -> 28333, "eclair.bitcoind.rpcport" -> 28332, "eclair.bitcoind.zmqblock" -> "tcp://127.0.0.1:28334", "eclair.bitcoind.zmqtx" -> "tcp://127.0.0.1:28335", "eclair.mindepth-blocks" -> 2, "eclair.max-htlc-value-in-flight-msat" -> 100000000000L, "eclair.router.broadcast-interval" -> "2 second", "eclair.auto-reconnect" -> false))
|
||||
instantiateEclairNode("A", ConfigFactory.parseMap(Map("eclair.node-alias" -> "A", "eclair.delay-blocks" -> 130, "eclair.server.port" -> 29730, "eclair.api.port" -> 28080, "eclair.channel-flags" -> 0)).withFallback(commonConfig)) // A's channels are private
|
||||
instantiateEclairNode("B", ConfigFactory.parseMap(Map("eclair.node-alias" -> "B", "eclair.delay-blocks" -> 131, "eclair.server.port" -> 29731, "eclair.api.port" -> 28081)).withFallback(commonConfig))
|
||||
instantiateEclairNode("C", ConfigFactory.parseMap(Map("eclair.node-alias" -> "C", "eclair.delay-blocks" -> 132, "eclair.server.port" -> 29732, "eclair.api.port" -> 28082, "eclair.payment-handler" -> "noop")).withFallback(commonConfig))
|
||||
instantiateEclairNode("D", ConfigFactory.parseMap(Map("eclair.node-alias" -> "D", "eclair.delay-blocks" -> 133, "eclair.server.port" -> 29733, "eclair.api.port" -> 28083)).withFallback(commonConfig))
|
||||
instantiateEclairNode("E", ConfigFactory.parseMap(Map("eclair.node-alias" -> "E", "eclair.delay-blocks" -> 134, "eclair.server.port" -> 29734, "eclair.api.port" -> 28084)).withFallback(commonConfig))
|
||||
instantiateEclairNode("F1", ConfigFactory.parseMap(Map("eclair.node-alias" -> "F1", "eclair.delay-blocks" -> 135, "eclair.server.port" -> 29735, "eclair.api.port" -> 28085, "eclair.payment-handler" -> "noop")).withFallback(commonConfig)) // NB: eclair.payment-handler = noop allows us to manually fulfill htlcs
|
||||
instantiateEclairNode("F2", ConfigFactory.parseMap(Map("eclair.node-alias" -> "F2", "eclair.delay-blocks" -> 136, "eclair.server.port" -> 29736, "eclair.api.port" -> 28086, "eclair.payment-handler" -> "noop")).withFallback(commonConfig))
|
||||
instantiateEclairNode("F3", ConfigFactory.parseMap(Map("eclair.node-alias" -> "F3", "eclair.delay-blocks" -> 137, "eclair.server.port" -> 29737, "eclair.api.port" -> 28087, "eclair.payment-handler" -> "noop")).withFallback(commonConfig))
|
||||
instantiateEclairNode("F4", ConfigFactory.parseMap(Map("eclair.node-alias" -> "F4", "eclair.delay-blocks" -> 138, "eclair.server.port" -> 29738, "eclair.api.port" -> 28088, "eclair.payment-handler" -> "noop")).withFallback(commonConfig))
|
||||
instantiateEclairNode("F5", ConfigFactory.parseMap(Map("eclair.node-alias" -> "F5", "eclair.delay-blocks" -> 139, "eclair.server.port" -> 29739, "eclair.api.port" -> 28089, "eclair.payment-handler" -> "noop")).withFallback(commonConfig))
|
||||
instantiateEclairNode("A", ConfigFactory.parseMap(Map("eclair.node-alias" -> "A", "eclair.expiry-delta-blocks" -> 130, "eclair.server.port" -> 29730, "eclair.api.port" -> 28080, "eclair.channel-flags" -> 0)).withFallback(commonConfig)) // A's channels are private
|
||||
instantiateEclairNode("B", ConfigFactory.parseMap(Map("eclair.node-alias" -> "B", "eclair.expiry-delta-blocks" -> 131, "eclair.server.port" -> 29731, "eclair.api.port" -> 28081)).withFallback(commonConfig))
|
||||
instantiateEclairNode("C", ConfigFactory.parseMap(Map("eclair.node-alias" -> "C", "eclair.expiry-delta-blocks" -> 132, "eclair.server.port" -> 29732, "eclair.api.port" -> 28082, "eclair.payment-handler" -> "noop")).withFallback(commonConfig))
|
||||
instantiateEclairNode("D", ConfigFactory.parseMap(Map("eclair.node-alias" -> "D", "eclair.expiry-delta-blocks" -> 133, "eclair.server.port" -> 29733, "eclair.api.port" -> 28083)).withFallback(commonConfig))
|
||||
instantiateEclairNode("E", ConfigFactory.parseMap(Map("eclair.node-alias" -> "E", "eclair.expiry-delta-blocks" -> 134, "eclair.server.port" -> 29734, "eclair.api.port" -> 28084)).withFallback(commonConfig))
|
||||
instantiateEclairNode("F1", ConfigFactory.parseMap(Map("eclair.node-alias" -> "F1", "eclair.expiry-delta-blocks" -> 135, "eclair.server.port" -> 29735, "eclair.api.port" -> 28085, "eclair.payment-handler" -> "noop")).withFallback(commonConfig)) // NB: eclair.payment-handler = noop allows us to manually fulfill htlcs
|
||||
instantiateEclairNode("F2", ConfigFactory.parseMap(Map("eclair.node-alias" -> "F2", "eclair.expiry-delta-blocks" -> 136, "eclair.server.port" -> 29736, "eclair.api.port" -> 28086, "eclair.payment-handler" -> "noop")).withFallback(commonConfig))
|
||||
instantiateEclairNode("F3", ConfigFactory.parseMap(Map("eclair.node-alias" -> "F3", "eclair.expiry-delta-blocks" -> 137, "eclair.server.port" -> 29737, "eclair.api.port" -> 28087, "eclair.payment-handler" -> "noop")).withFallback(commonConfig))
|
||||
instantiateEclairNode("F4", ConfigFactory.parseMap(Map("eclair.node-alias" -> "F4", "eclair.expiry-delta-blocks" -> 138, "eclair.server.port" -> 29738, "eclair.api.port" -> 28088, "eclair.payment-handler" -> "noop")).withFallback(commonConfig))
|
||||
instantiateEclairNode("F5", ConfigFactory.parseMap(Map("eclair.node-alias" -> "F5", "eclair.expiry-delta-blocks" -> 139, "eclair.server.port" -> 29739, "eclair.api.port" -> 28089, "eclair.payment-handler" -> "noop")).withFallback(commonConfig))
|
||||
|
||||
// by default C has a normal payment handler, but this can be overriden in tests
|
||||
val paymentHandlerC = nodes("C").system.actorOf(LocalPaymentHandler.props(nodes("C").nodeParams))
|
||||
@ -146,11 +158,11 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
|
||||
}
|
||||
|
||||
test("connect nodes") {
|
||||
//
|
||||
// A ---- B ---- C ==== D
|
||||
// | / \
|
||||
// --E--' F{1,2,3,4,5}
|
||||
//
|
||||
// ,--G--, // G is being added later in a test
|
||||
// / \
|
||||
// A---B ------- C ==== D
|
||||
// \ / \
|
||||
// '--E--' F{1,2,3,4,5}
|
||||
|
||||
val sender = TestProbe()
|
||||
val eventListener = TestProbe()
|
||||
@ -218,7 +230,7 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
|
||||
sender.send(bitcoincli, BitcoinReq("generate", 4))
|
||||
sender.expectMsgType[JValue]
|
||||
// A requires private channels, as a consequence:
|
||||
// - only A and B now about channel A-B
|
||||
// - only A and B know about channel A-B
|
||||
// - A is not announced
|
||||
awaitAnnouncements(nodes.filterKeys(key => List("A", "B").contains(key)), 9, 10, 22)
|
||||
awaitAnnouncements(nodes.filterKeys(key => !List("A", "B").contains(key)), 9, 10, 20)
|
||||
@ -371,6 +383,38 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
|
||||
}
|
||||
}
|
||||
|
||||
test("send an HTLC A->B->G->C using heuristics to select the route") {
|
||||
val sender = TestProbe()
|
||||
|
||||
// G has very large channels but slightly more expensive than the others
|
||||
instantiateEclairNode("G", ConfigFactory.parseMap(Map("eclair.node-alias" -> "G", "eclair.expiry-delta-blocks" -> 140, "eclair.server.port" -> 29740, "eclair.api.port" -> 28090, "eclair.fee-base-msat" -> 1010, "eclair.fee-proportional-millionths" -> 102)).withFallback(commonConfig))
|
||||
connect(nodes("B"), nodes("G"), 16000000, 0)
|
||||
connect(nodes("G"), nodes("C"), 16000000, 0)
|
||||
|
||||
sender.send(bitcoincli, BitcoinReq("generate", 10))
|
||||
sender.expectMsgType[JValue](10 seconds)
|
||||
|
||||
awaitCond({
|
||||
sender.send(nodes("A").router, 'channels)
|
||||
sender.expectMsgType[Iterable[ChannelAnnouncement]](5 seconds).exists(chanAnn => chanAnn.nodeId1 == nodes("G").nodeParams.nodeId || chanAnn.nodeId2 == nodes("G").nodeParams.nodeId)
|
||||
}, max = 60 seconds, interval = 3 seconds)
|
||||
|
||||
val amountMsat = MilliSatoshi(2000)
|
||||
// first we retrieve a payment hash from C
|
||||
sender.send(nodes("C").paymentHandler, ReceivePayment(Some(amountMsat), "Change from coffee"))
|
||||
val pr = sender.expectMsgType[PaymentRequest](30 seconds)
|
||||
|
||||
// the payment is requesting to use a capacity-optimized route which will select node G even though it's a bit more expensive
|
||||
sender.send(nodes("A").paymentInitiator,
|
||||
SendPayment(amountMsat.amount, pr.paymentHash, nodes("C").nodeParams.nodeId, randomize = Some(false), routeParams = integrationTestRouteParams.map(_.copy(ratios = Some(WeightRatios(0, 0, 1))))))
|
||||
|
||||
awaitCond({
|
||||
val route = sender.expectMsgType[PaymentSucceeded].route
|
||||
route.exists(_.nodeId == nodes("G").nodeParams.nodeId) // assert the used route is actually going through G
|
||||
}, max = 30 seconds, interval = 3 seconds)
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* We currently use p2pkh script Helpers.getFinalScriptPubKey
|
||||
*
|
||||
@ -463,7 +507,7 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
|
||||
sender.expectMsgType[JValue](10 seconds)
|
||||
// and we wait for C'channel to close
|
||||
awaitCond(stateListener.expectMsgType[ChannelStateChanged].currentState == CLOSED, max = 30 seconds)
|
||||
awaitAnnouncements(nodes.filter(_._1 == "A"), 8, 9, 20)
|
||||
awaitAnnouncements(nodes.filter(_._1 == "A"), 9, 11, 24)
|
||||
}
|
||||
|
||||
test("propagate a fulfill upstream when a downstream htlc is redeemed on-chain (remote commit)") {
|
||||
@ -538,7 +582,7 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
|
||||
sender.expectMsgType[JValue](10 seconds)
|
||||
// and we wait for C'channel to close
|
||||
awaitCond(stateListener.expectMsgType[ChannelStateChanged].currentState == CLOSED, max = 30 seconds)
|
||||
awaitAnnouncements(nodes.filter(_._1 == "A"), 7, 8, 18)
|
||||
awaitAnnouncements(nodes.filter(_._1 == "A"), 8, 10, 22)
|
||||
}
|
||||
|
||||
test("propagate a failure upstream when a downstream htlc times out (local commit)") {
|
||||
@ -598,7 +642,7 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
|
||||
sender.expectMsgType[JValue](10 seconds)
|
||||
// and we wait for C'channel to close
|
||||
awaitCond(stateListener.expectMsgType[ChannelStateChanged].currentState == CLOSED, max = 30 seconds)
|
||||
awaitAnnouncements(nodes.filter(_._1 == "A"), 6, 7, 16)
|
||||
awaitAnnouncements(nodes.filter(_._1 == "A"), 7, 9, 20)
|
||||
}
|
||||
|
||||
test("propagate a failure upstream when a downstream htlc times out (remote commit)") {
|
||||
@ -661,7 +705,7 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
|
||||
sender.expectMsgType[JValue](10 seconds)
|
||||
// and we wait for C'channel to close
|
||||
awaitCond(stateListener.expectMsgType[ChannelStateChanged].currentState == CLOSED, max = 30 seconds)
|
||||
awaitAnnouncements(nodes.filter(_._1 == "A"), 5, 6, 14)
|
||||
awaitAnnouncements(nodes.filter(_._1 == "A"), 6, 8, 18)
|
||||
}
|
||||
|
||||
test("punish a node that has published a revoked commit tx") {
|
||||
@ -786,7 +830,7 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
|
||||
// and we wait for C'channel to close
|
||||
awaitCond(stateListener.expectMsgType[ChannelStateChanged].currentState == CLOSED, max = 30 seconds)
|
||||
// this will remove the channel
|
||||
awaitAnnouncements(nodes.filter(_._1 == "A"), 4, 5, 12)
|
||||
awaitAnnouncements(nodes.filter(_._1 == "A"), 5, 7, 16)
|
||||
}
|
||||
|
||||
test("generate and validate lots of channels") {
|
||||
@ -813,9 +857,8 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
|
||||
announcements.foreach(ann => nodes("A").router ! PeerRoutingMessage(sender.ref, remoteNodeId, ann))
|
||||
awaitCond({
|
||||
sender.send(nodes("D").router, 'channels)
|
||||
sender.expectMsgType[Iterable[ChannelAnnouncement]](5 seconds).size == channels.size + 5 // 5 remaining channels because D->F{1-5} have disappeared
|
||||
sender.expectMsgType[Iterable[ChannelAnnouncement]](5 seconds).size == channels.size + 7 // 7 remaining channels because D->F{1-5} have disappeared
|
||||
}, max = 120 seconds, interval = 1 second)
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
@ -67,7 +67,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
|
||||
paymentFSM ! SubscribeTransitionCallBack(monitor.ref)
|
||||
val CurrentState(_, WAITING_FOR_REQUEST) = monitor.expectMsgClass(classOf[CurrentState[_]])
|
||||
|
||||
val request = SendPayment(defaultAmountMsat, defaultPaymentHash, d, routeParams = Some(RouteParams(maxFeeBaseMsat = 0, maxFeePct = 0.001, routeMaxLength = 20, routeMaxCltv = 2016)))
|
||||
val request = SendPayment(defaultAmountMsat, defaultPaymentHash, d, routeParams = Some(RouteParams(maxFeeBaseMsat = 100, maxFeePct = 0.0, routeMaxLength = 20, routeMaxCltv = 2016, ratios = None)))
|
||||
sender.send(paymentFSM, request)
|
||||
val Transition(_, WAITING_FOR_REQUEST, WAITING_FOR_ROUTE) = monitor.expectMsgClass(classOf[Transition[_]])
|
||||
|
||||
|
@ -75,14 +75,14 @@ abstract class BaseRouterSpec extends TestkitBaseClass {
|
||||
val chan_cd = channelAnnouncement(channelId_cd, priv_c, priv_d, priv_funding_c, priv_funding_d)
|
||||
val chan_ef = channelAnnouncement(channelId_ef, priv_e, priv_f, priv_funding_e, priv_funding_f)
|
||||
|
||||
val channelUpdate_ab = makeChannelUpdate(Block.RegtestGenesisBlock.hash, priv_a, b, channelId_ab, cltvExpiryDelta = 7, htlcMinimumMsat = 0, feeBaseMsat = 766000, feeProportionalMillionths = 10, htlcMaximumMsat = 500000000L)
|
||||
val channelUpdate_ba = makeChannelUpdate(Block.RegtestGenesisBlock.hash, priv_b, a, channelId_ab, cltvExpiryDelta = 7, htlcMinimumMsat = 0, feeBaseMsat = 766000, feeProportionalMillionths = 10, htlcMaximumMsat = 500000000L)
|
||||
val channelUpdate_bc = makeChannelUpdate(Block.RegtestGenesisBlock.hash, priv_b, c, channelId_bc, cltvExpiryDelta = 5, htlcMinimumMsat = 0, feeBaseMsat = 233000, feeProportionalMillionths = 1, htlcMaximumMsat = 500000000L)
|
||||
val channelUpdate_cb = makeChannelUpdate(Block.RegtestGenesisBlock.hash, priv_c, b, channelId_bc, cltvExpiryDelta = 5, htlcMinimumMsat = 0, feeBaseMsat = 233000, feeProportionalMillionths = 1, htlcMaximumMsat = 500000000L)
|
||||
val channelUpdate_cd = makeChannelUpdate(Block.RegtestGenesisBlock.hash, priv_c, d, channelId_cd, cltvExpiryDelta = 3, htlcMinimumMsat = 0, feeBaseMsat = 153000, feeProportionalMillionths = 4, htlcMaximumMsat = 500000000L)
|
||||
val channelUpdate_dc = makeChannelUpdate(Block.RegtestGenesisBlock.hash, priv_d, c, channelId_cd, cltvExpiryDelta = 3, htlcMinimumMsat = 0, feeBaseMsat = 153000, feeProportionalMillionths = 4, htlcMaximumMsat = 500000000L)
|
||||
val channelUpdate_ef = makeChannelUpdate(Block.RegtestGenesisBlock.hash, priv_e, f, channelId_ef, cltvExpiryDelta = 9, htlcMinimumMsat = 0, feeBaseMsat = 786000, feeProportionalMillionths = 8, htlcMaximumMsat = 500000000L)
|
||||
val channelUpdate_fe = makeChannelUpdate(Block.RegtestGenesisBlock.hash, priv_f, e, channelId_ef, cltvExpiryDelta = 9, htlcMinimumMsat = 0, feeBaseMsat = 786000, feeProportionalMillionths = 8, htlcMaximumMsat = 500000000L)
|
||||
val channelUpdate_ab = makeChannelUpdate(Block.RegtestGenesisBlock.hash, priv_a, b, channelId_ab, cltvExpiryDelta = 7, htlcMinimumMsat = 0, feeBaseMsat = 10, feeProportionalMillionths = 10, htlcMaximumMsat = 500000000L)
|
||||
val channelUpdate_ba = makeChannelUpdate(Block.RegtestGenesisBlock.hash, priv_b, a, channelId_ab, cltvExpiryDelta = 7, htlcMinimumMsat = 0, feeBaseMsat = 10, feeProportionalMillionths = 10, htlcMaximumMsat = 500000000L)
|
||||
val channelUpdate_bc = makeChannelUpdate(Block.RegtestGenesisBlock.hash, priv_b, c, channelId_bc, cltvExpiryDelta = 5, htlcMinimumMsat = 0, feeBaseMsat = 10, feeProportionalMillionths = 1, htlcMaximumMsat = 500000000L)
|
||||
val channelUpdate_cb = makeChannelUpdate(Block.RegtestGenesisBlock.hash, priv_c, b, channelId_bc, cltvExpiryDelta = 5, htlcMinimumMsat = 0, feeBaseMsat = 10, feeProportionalMillionths = 1, htlcMaximumMsat = 500000000L)
|
||||
val channelUpdate_cd = makeChannelUpdate(Block.RegtestGenesisBlock.hash, priv_c, d, channelId_cd, cltvExpiryDelta = 3, htlcMinimumMsat = 0, feeBaseMsat = 10, feeProportionalMillionths = 4, htlcMaximumMsat = 500000000L)
|
||||
val channelUpdate_dc = makeChannelUpdate(Block.RegtestGenesisBlock.hash, priv_d, c, channelId_cd, cltvExpiryDelta = 3, htlcMinimumMsat = 0, feeBaseMsat = 10, feeProportionalMillionths = 4, htlcMaximumMsat = 500000000L)
|
||||
val channelUpdate_ef = makeChannelUpdate(Block.RegtestGenesisBlock.hash, priv_e, f, channelId_ef, cltvExpiryDelta = 9, htlcMinimumMsat = 0, feeBaseMsat = 10, feeProportionalMillionths = 8, htlcMaximumMsat = 500000000L)
|
||||
val channelUpdate_fe = makeChannelUpdate(Block.RegtestGenesisBlock.hash, priv_f, e, channelId_ef, cltvExpiryDelta = 9, htlcMinimumMsat = 0, feeBaseMsat = 10, feeProportionalMillionths = 8, htlcMaximumMsat = 500000000L)
|
||||
|
||||
override def withFixture(test: OneArgTest): Outcome = {
|
||||
// the network will be a --(1)--> b ---(2)--> c --(3)--> d and e --(4)--> f (we are a)
|
||||
|
@ -21,9 +21,10 @@ import fr.acinq.bitcoin.{BinaryData, Block, Crypto}
|
||||
import fr.acinq.eclair.payment.PaymentRequest.ExtraHop
|
||||
import fr.acinq.eclair.router.Graph.GraphStructure.DirectedGraph.graphEdgeToHop
|
||||
import fr.acinq.eclair.router.Graph.GraphStructure.{DirectedGraph, GraphEdge}
|
||||
import fr.acinq.eclair.router.Graph.{RichWeight, WeightRatios}
|
||||
import fr.acinq.eclair.router.Graph.RichWeight
|
||||
import fr.acinq.eclair.wire._
|
||||
import fr.acinq.eclair.{ShortChannelId, randomKey}
|
||||
import fr.acinq.eclair.{Globals, ShortChannelId, randomKey}
|
||||
import org.scalatest.FunSuite
|
||||
|
||||
import scala.util.{Failure, Success}
|
||||
@ -36,15 +37,15 @@ class RouteCalculationSpec extends FunSuite {
|
||||
|
||||
import RouteCalculationSpec._
|
||||
|
||||
val (a, b, c, d, e) = (randomKey.publicKey, randomKey.publicKey, randomKey.publicKey, randomKey.publicKey, randomKey.publicKey)
|
||||
val (a, b, c, d, e, f) = (randomKey.publicKey, randomKey.publicKey, randomKey.publicKey, randomKey.publicKey, randomKey.publicKey, randomKey.publicKey)
|
||||
|
||||
test("calculate simple route") {
|
||||
|
||||
val updates = List(
|
||||
makeUpdate(1L, a, b, 0, 0),
|
||||
makeUpdate(2L, b, c, 0, 0),
|
||||
makeUpdate(3L, c, d, 0, 0),
|
||||
makeUpdate(4L, d, e, 0, 0)
|
||||
makeUpdate(1L, a, b, 1, 10, cltvDelta = 1),
|
||||
makeUpdate(2L, b, c, 1, 10, cltvDelta = 1),
|
||||
makeUpdate(3L, c, d, 1, 10, cltvDelta = 1),
|
||||
makeUpdate(4L, d, e, 1, 10, cltvDelta = 1)
|
||||
).toMap
|
||||
|
||||
val g = makeGraph(updates)
|
||||
@ -89,11 +90,13 @@ class RouteCalculationSpec extends FunSuite {
|
||||
|
||||
val Success(route) = Router.findRoute(graph, a, d, amountMsat, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS)
|
||||
|
||||
val totalCost = Graph.pathWeight(hops2Edges(route), amountMsat, false, 0, None).cost
|
||||
|
||||
assert(hops2Ids(route) === 4 :: 5 :: 6 :: Nil)
|
||||
assert(Graph.pathWeight(hops2Edges(route), amountMsat, isPartial = false).cost === expectedCost)
|
||||
assert(totalCost === expectedCost)
|
||||
|
||||
// now channel 5 could route the amount (10000) but not the amount + fees (10007)
|
||||
val (desc, update) = makeUpdate(5L, e, f, feeBaseMsat = 1, feeProportionalMillionth = 400, minHtlcMsat = 0, maxHtlcMsat = Some(10005))
|
||||
val (desc, update) = makeUpdate(5L, e, f, feeBaseMsat = 1, feeProportionalMillionth = 400, minHtlcMsat = 0, maxHtlcMsat = Some(10005L))
|
||||
val graph1 = graph.addEdge(desc, update)
|
||||
|
||||
val Success(route1) = Router.findRoute(graph1, a, d, amountMsat, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS)
|
||||
@ -245,79 +248,41 @@ class RouteCalculationSpec extends FunSuite {
|
||||
assert(route.map(hops2Ids) === Failure(RouteNotFound))
|
||||
}
|
||||
|
||||
test("route not found (source node not connected)") {
|
||||
test("route not found (source OR target node not connected)") {
|
||||
|
||||
val updates = List(
|
||||
makeUpdate(2L, b, c, 0, 0),
|
||||
makeUpdate(4L, d, e, 0, 0)
|
||||
makeUpdate(4L, c, d, 0, 0)
|
||||
).toMap
|
||||
|
||||
val g = makeGraph(updates).addVertex(a)
|
||||
val g = makeGraph(updates).addVertex(a).addVertex(e)
|
||||
|
||||
val route = Router.findRoute(g, a, e, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS)
|
||||
assert(route.map(hops2Ids) === Failure(RouteNotFound))
|
||||
assert(Router.findRoute(g, a, d, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS) === Failure(RouteNotFound))
|
||||
assert(Router.findRoute(g, b, e, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS) === Failure(RouteNotFound))
|
||||
}
|
||||
|
||||
test("route not found (target node not connected)") {
|
||||
|
||||
val updates = List(
|
||||
makeUpdate(1L, a, b, 0, 0),
|
||||
makeUpdate(2L, b, c, 0, 0),
|
||||
makeUpdate(3L, c, d, 0, 0)
|
||||
).toMap
|
||||
|
||||
val g = makeGraph(updates)
|
||||
|
||||
val route = Router.findRoute(g, a, e, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS)
|
||||
assert(route.map(hops2Ids) === Failure(RouteNotFound))
|
||||
}
|
||||
|
||||
test("route not found (unknown destination)") {
|
||||
|
||||
val updates = List(
|
||||
makeUpdate(1L, a, b, 0, 0),
|
||||
makeUpdate(2L, b, c, 0, 0),
|
||||
makeUpdate(3L, c, d, 0, 0)
|
||||
).toMap
|
||||
|
||||
val g = makeGraph(updates)
|
||||
|
||||
val route = Router.findRoute(g, a, e, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS)
|
||||
assert(route.map(hops2Ids) === Failure(RouteNotFound))
|
||||
}
|
||||
|
||||
test("route not found (amount too high)") {
|
||||
test("route not found (amount too high OR too low)") {
|
||||
|
||||
val highAmount = DEFAULT_AMOUNT_MSAT * 10
|
||||
val lowAmount = DEFAULT_AMOUNT_MSAT / 10
|
||||
|
||||
val updates = List(
|
||||
val updatesHi = List(
|
||||
makeUpdate(1L, a, b, 0, 0),
|
||||
makeUpdate(2L, b, c, 0, 0, maxHtlcMsat = Some(DEFAULT_AMOUNT_MSAT)),
|
||||
makeUpdate(3L, c, d, 0, 0)
|
||||
).toMap
|
||||
|
||||
val g = makeGraph(updates)
|
||||
|
||||
val route = Router.findRoute(g, a, d, highAmount, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS)
|
||||
assert(route.map(hops2Ids) === Failure(RouteNotFound))
|
||||
|
||||
}
|
||||
|
||||
test("route not found (amount too low)") {
|
||||
|
||||
val lowAmount = DEFAULT_AMOUNT_MSAT / 10
|
||||
|
||||
val updates = List(
|
||||
val updatesLo = List(
|
||||
makeUpdate(1L, a, b, 0, 0),
|
||||
makeUpdate(2L, b, c, 0, 0, minHtlcMsat = DEFAULT_AMOUNT_MSAT),
|
||||
makeUpdate(3L, c, d, 0, 0)
|
||||
).toMap
|
||||
|
||||
val g = makeGraph(updates)
|
||||
|
||||
val route = Router.findRoute(g, a, d, lowAmount, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS)
|
||||
assert(route.map(hops2Ids) === Failure(RouteNotFound))
|
||||
val g = makeGraph(updatesHi)
|
||||
val g1 = makeGraph(updatesLo)
|
||||
|
||||
assert(Router.findRoute(g, a, d, highAmount, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS) === Failure(RouteNotFound))
|
||||
assert(Router.findRoute(g1, a, d, lowAmount, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS) === Failure(RouteNotFound))
|
||||
}
|
||||
|
||||
test("route to self") {
|
||||
@ -580,12 +545,12 @@ class RouteCalculationSpec extends FunSuite {
|
||||
val f = randomKey.publicKey
|
||||
|
||||
val g = makeGraph(List(
|
||||
makeUpdate(1, a, b, feeBaseMsat = 1, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltv = 50),
|
||||
makeUpdate(2, b, c, feeBaseMsat = 1, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltv = 50),
|
||||
makeUpdate(3, c, d, feeBaseMsat = 1, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltv = 50),
|
||||
makeUpdate(4, a, e, feeBaseMsat = 1, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltv = 9),
|
||||
makeUpdate(5, e, f, feeBaseMsat = 5, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltv = 9),
|
||||
makeUpdate(6, f, d, feeBaseMsat = 5, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltv = 9)
|
||||
makeUpdate(1, a, b, feeBaseMsat = 1, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltvDelta = 50),
|
||||
makeUpdate(2, b, c, feeBaseMsat = 1, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltvDelta = 50),
|
||||
makeUpdate(3, c, d, feeBaseMsat = 1, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltvDelta = 50),
|
||||
makeUpdate(4, a, e, feeBaseMsat = 1, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltvDelta = 9),
|
||||
makeUpdate(5, e, f, feeBaseMsat = 5, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltvDelta = 9),
|
||||
makeUpdate(6, f, d, feeBaseMsat = 5, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltvDelta = 9)
|
||||
).toMap)
|
||||
|
||||
val route = Router.findRoute(g, a, d, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS.copy(routeMaxCltv = 28))
|
||||
@ -597,12 +562,12 @@ class RouteCalculationSpec extends FunSuite {
|
||||
val f = randomKey.publicKey
|
||||
|
||||
val g = makeGraph(List(
|
||||
makeUpdate(1, a, b, feeBaseMsat = 1, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltv = 9),
|
||||
makeUpdate(2, b, c, feeBaseMsat = 1, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltv = 9),
|
||||
makeUpdate(3, c, d, feeBaseMsat = 1, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltv = 9),
|
||||
makeUpdate(4, d, e, feeBaseMsat = 1, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltv = 9),
|
||||
makeUpdate(5, e, f, feeBaseMsat = 5, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltv = 9),
|
||||
makeUpdate(6, b, f, feeBaseMsat = 5, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltv = 9)
|
||||
makeUpdate(1, a, b, feeBaseMsat = 1, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltvDelta = 9),
|
||||
makeUpdate(2, b, c, feeBaseMsat = 1, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltvDelta = 9),
|
||||
makeUpdate(3, c, d, feeBaseMsat = 1, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltvDelta = 9),
|
||||
makeUpdate(4, d, e, feeBaseMsat = 1, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltvDelta = 9),
|
||||
makeUpdate(5, e, f, feeBaseMsat = 5, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltvDelta = 9),
|
||||
makeUpdate(6, b, f, feeBaseMsat = 5, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltvDelta = 9)
|
||||
).toMap)
|
||||
|
||||
val route = Router.findRoute(g, a, f, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS.copy(routeMaxLength = 3))
|
||||
@ -663,7 +628,7 @@ class RouteCalculationSpec extends FunSuite {
|
||||
|
||||
val graph = DirectedGraph.makeGraph(edges)
|
||||
|
||||
val fourShortestPaths = Graph.yenKshortestPaths(graph, d, f, DEFAULT_AMOUNT_MSAT, Set.empty, Set.empty, pathsToFind = 4, noopBoundaries)
|
||||
val fourShortestPaths = Graph.yenKshortestPaths(graph, d, f, DEFAULT_AMOUNT_MSAT, Set.empty, Set.empty, pathsToFind = 4, None, 0, noopBoundaries)
|
||||
|
||||
assert(fourShortestPaths.size === 4)
|
||||
assert(hops2Ids(fourShortestPaths(0).path.map(graphEdgeToHop)) === 2 :: 5 :: Nil) // D -> E -> F
|
||||
@ -697,7 +662,7 @@ class RouteCalculationSpec extends FunSuite {
|
||||
|
||||
val graph = DirectedGraph().addEdges(edges)
|
||||
|
||||
val twoShortestPaths = Graph.yenKshortestPaths(graph, c, h, DEFAULT_AMOUNT_MSAT, Set.empty, Set.empty, pathsToFind = 2, noopBoundaries)
|
||||
val twoShortestPaths = Graph.yenKshortestPaths(graph, c, h, DEFAULT_AMOUNT_MSAT, Set.empty, Set.empty, pathsToFind = 2, None, 0, noopBoundaries)
|
||||
|
||||
assert(twoShortestPaths.size === 2)
|
||||
val shortest = twoShortestPaths(0)
|
||||
@ -731,16 +696,16 @@ class RouteCalculationSpec extends FunSuite {
|
||||
val graph = DirectedGraph().addEdges(edges)
|
||||
|
||||
//we ask for 3 shortest paths but only 2 can be found
|
||||
val foundPaths = Graph.yenKshortestPaths(graph, a, f, DEFAULT_AMOUNT_MSAT, Set.empty, Set.empty, pathsToFind = 3, noopBoundaries)
|
||||
val foundPaths = Graph.yenKshortestPaths(graph, a, f, DEFAULT_AMOUNT_MSAT, Set.empty, Set.empty, pathsToFind = 3, None, 0, noopBoundaries)
|
||||
|
||||
assert(foundPaths.size === 2)
|
||||
assert(hops2Ids(foundPaths(0).path.map(graphEdgeToHop)) === 1 :: 2 :: 3 :: Nil) // A -> B -> C -> F
|
||||
assert(hops2Ids(foundPaths(1).path.map(graphEdgeToHop)) === 1 :: 2 :: 4 :: 5 :: 6 :: Nil) // A -> B -> C -> D -> E -> F
|
||||
}
|
||||
|
||||
test("select a random route below the allowed fee spread") {
|
||||
test("select a random route below the requested fee") {
|
||||
|
||||
val f = randomKey.publicKey
|
||||
val strictFeeParams = DEFAULT_ROUTE_PARAMS.copy(maxFeeBaseMsat = 7, maxFeePct = 0)
|
||||
|
||||
// A -> B -> C -> D has total cost of 10000005
|
||||
// A -> E -> C -> D has total cost of 11080003 !!
|
||||
@ -755,16 +720,123 @@ class RouteCalculationSpec extends FunSuite {
|
||||
makeUpdate(7L, e, c, feeBaseMsat = 9, 0)
|
||||
).toMap)
|
||||
|
||||
(for {_ <- 0 to 10} yield Router.findRoute(g, a, d, DEFAULT_AMOUNT_MSAT, numRoutes = 3, routeParams = DEFAULT_ROUTE_PARAMS.copy(maxFeeBaseMsat = 7, maxFeePct = 0))).map {
|
||||
case Failure(_) => assert(false)
|
||||
(for {_ <- 0 to 10} yield Router.findRoute(g, a, d, DEFAULT_AMOUNT_MSAT, numRoutes = 3, routeParams = strictFeeParams)).map {
|
||||
case Failure(thr) => assert(false, thr)
|
||||
case Success(someRoute) =>
|
||||
|
||||
val routeCost = Graph.pathWeight(hops2Edges(someRoute), DEFAULT_AMOUNT_MSAT, isPartial = false).cost - DEFAULT_AMOUNT_MSAT
|
||||
val routeCost = Graph.pathWeight(hops2Edges(someRoute), DEFAULT_AMOUNT_MSAT, isPartial = false, 0, None).cost - DEFAULT_AMOUNT_MSAT
|
||||
|
||||
// over the three routes we could only get the 2 cheapest because the third is too expensive (over 7msat of fees)
|
||||
assert(routeCost == 5 || routeCost == 6)
|
||||
}
|
||||
}
|
||||
|
||||
test("Use weight ratios to when computing the edge weight") {
|
||||
|
||||
val largeCapacity = 8000000000L
|
||||
|
||||
// A -> B -> C -> D is 'fee optimized', lower fees route (totFees = 2, totCltv = 4000)
|
||||
// A -> E -> F -> D is 'timeout optimized', lower CLTV route (totFees = 3, totCltv = 18)
|
||||
// A -> E -> C -> D is 'capacity optimized', more recent channel/larger capacity route
|
||||
val updates = List(
|
||||
makeUpdate(1L, a, b, feeBaseMsat = 0, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltvDelta = 13),
|
||||
makeUpdate(4L, a, e, feeBaseMsat = 0, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltvDelta = 12),
|
||||
makeUpdate(2L, b, c, feeBaseMsat = 1, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltvDelta = 500),
|
||||
makeUpdate(3L, c, d, feeBaseMsat = 1, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltvDelta = 500),
|
||||
makeUpdate(5L, e, f, feeBaseMsat = 2, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltvDelta = 9),
|
||||
makeUpdate(6L, f, d, feeBaseMsat = 2, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltvDelta = 9),
|
||||
makeUpdate(7L, e, c, feeBaseMsat = 2, 0, minHtlcMsat = 0, maxHtlcMsat = Some(largeCapacity), cltvDelta = 12)
|
||||
).toMap
|
||||
|
||||
val g = makeGraph(updates)
|
||||
|
||||
val Success(routeFeeOptimized) = Router.findRoute(g, a, d, DEFAULT_AMOUNT_MSAT, numRoutes = 0, routeParams = DEFAULT_ROUTE_PARAMS)
|
||||
assert(hops2Nodes(routeFeeOptimized) === (a, b) :: (b, c) :: (c, d) :: Nil)
|
||||
|
||||
val Success(routeCltvOptimized) = Router.findRoute(g, a, d, DEFAULT_AMOUNT_MSAT, numRoutes = 0, routeParams = DEFAULT_ROUTE_PARAMS.copy(ratios = Some(WeightRatios(
|
||||
cltvDeltaFactor = 1,
|
||||
ageFactor = 0,
|
||||
capacityFactor = 0
|
||||
))))
|
||||
|
||||
assert(hops2Nodes(routeCltvOptimized) === (a, e) :: (e, f) :: (f, d) :: Nil)
|
||||
|
||||
val Success(routeCapacityOptimized) = Router.findRoute(g, a, d, DEFAULT_AMOUNT_MSAT, numRoutes = 0, routeParams = DEFAULT_ROUTE_PARAMS.copy(ratios = Some(WeightRatios(
|
||||
cltvDeltaFactor = 0,
|
||||
ageFactor = 0,
|
||||
capacityFactor = 1
|
||||
))))
|
||||
|
||||
assert(hops2Nodes(routeCapacityOptimized) === (a, e) :: (e, c) :: (c, d) :: Nil)
|
||||
}
|
||||
|
||||
test("prefer going through an older channel if fees and CLTV are the same") {
|
||||
|
||||
val currentBlockHeight = 554000
|
||||
|
||||
val g = makeGraph(List(
|
||||
makeUpdateShort(ShortChannelId(s"${currentBlockHeight}x0x1"), a, b, feeBaseMsat = 1, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltvDelta = 144),
|
||||
makeUpdateShort(ShortChannelId(s"${currentBlockHeight}x0x4"), a, e, feeBaseMsat = 1, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltvDelta = 144),
|
||||
makeUpdateShort(ShortChannelId(s"${currentBlockHeight - 3000}x0x2"), b, c, feeBaseMsat = 1, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltvDelta = 144), // younger channel
|
||||
makeUpdateShort(ShortChannelId(s"${currentBlockHeight - 3000}x0x3"), c, d, feeBaseMsat = 1, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltvDelta = 144),
|
||||
makeUpdateShort(ShortChannelId(s"${currentBlockHeight}x0x5"), e, f, feeBaseMsat = 1, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltvDelta = 144),
|
||||
makeUpdateShort(ShortChannelId(s"${currentBlockHeight}x0x6"), f, d, feeBaseMsat = 1, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltvDelta = 144)
|
||||
).toMap)
|
||||
|
||||
Globals.blockCount.set(currentBlockHeight)
|
||||
|
||||
val Success(routeScoreOptimized) = Router.findRoute(g, a, d, DEFAULT_AMOUNT_MSAT / 2, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS.copy(ratios = Some(WeightRatios(
|
||||
ageFactor = 0.33,
|
||||
cltvDeltaFactor = 0.33,
|
||||
capacityFactor = 0.33
|
||||
))))
|
||||
|
||||
assert(hops2Nodes(routeScoreOptimized) === (a, b) :: (b, c) :: (c, d) :: Nil)
|
||||
}
|
||||
|
||||
test("prefer a route with a smaller total CLTV if fees and score are the same") {
|
||||
|
||||
val g = makeGraph(List(
|
||||
makeUpdateShort(ShortChannelId(s"0x0x1"), a, b, feeBaseMsat = 1, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltvDelta = 12),
|
||||
makeUpdateShort(ShortChannelId(s"0x0x4"), a, e, feeBaseMsat = 1, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltvDelta = 12),
|
||||
makeUpdateShort(ShortChannelId(s"0x0x2"), b, c, feeBaseMsat = 1, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltvDelta = 10), // smaller CLTV
|
||||
makeUpdateShort(ShortChannelId(s"0x0x3"), c, d, feeBaseMsat = 1, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltvDelta = 12),
|
||||
makeUpdateShort(ShortChannelId(s"0x0x5"), e, f, feeBaseMsat = 1, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltvDelta = 12),
|
||||
makeUpdateShort(ShortChannelId(s"0x0x6"), f, d, feeBaseMsat = 1, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltvDelta = 12)
|
||||
).toMap)
|
||||
|
||||
|
||||
val Success(routeScoreOptimized) = Router.findRoute(g, a, d, DEFAULT_AMOUNT_MSAT, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS.copy(ratios = Some(WeightRatios(
|
||||
ageFactor = 0.33,
|
||||
cltvDeltaFactor = 0.33,
|
||||
capacityFactor = 0.33
|
||||
))))
|
||||
|
||||
assert(hops2Nodes(routeScoreOptimized) === (a, b) :: (b, c) :: (c, d) :: Nil)
|
||||
}
|
||||
|
||||
|
||||
test("avoid a route that breaks off the max CLTV") {
|
||||
|
||||
// A -> B -> C -> D is cheaper but has a total CLTV > 2016!
|
||||
// A -> E -> F -> D is more expensive but has a total CLTV < 2016
|
||||
val g = makeGraph(List(
|
||||
makeUpdateShort(ShortChannelId(s"0x0x1"), a, b, feeBaseMsat = 1, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltvDelta = 144),
|
||||
makeUpdateShort(ShortChannelId(s"0x0x4"), a, e, feeBaseMsat = 1, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltvDelta = 144),
|
||||
makeUpdateShort(ShortChannelId(s"0x0x2"), b, c, feeBaseMsat = 1, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltvDelta = 1000),
|
||||
makeUpdateShort(ShortChannelId(s"0x0x3"), c, d, feeBaseMsat = 1, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltvDelta = 900),
|
||||
makeUpdateShort(ShortChannelId(s"0x0x5"), e, f, feeBaseMsat = 10, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltvDelta = 144),
|
||||
makeUpdateShort(ShortChannelId(s"0x0x6"), f, d, feeBaseMsat = 10, 0, minHtlcMsat = 0, maxHtlcMsat = None, cltvDelta = 144)
|
||||
).toMap)
|
||||
|
||||
val Success(routeScoreOptimized) = Router.findRoute(g, a, d, DEFAULT_AMOUNT_MSAT / 2, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS.copy(ratios = Some(WeightRatios(
|
||||
ageFactor = 0.33,
|
||||
cltvDeltaFactor = 0.33,
|
||||
capacityFactor = 0.33
|
||||
))))
|
||||
|
||||
assert(hops2Nodes(routeScoreOptimized) === (a, e) :: (e, f) :: (f, d) :: Nil)
|
||||
}
|
||||
}
|
||||
|
||||
object RouteCalculationSpec {
|
||||
@ -773,7 +845,7 @@ object RouteCalculationSpec {
|
||||
|
||||
val DEFAULT_AMOUNT_MSAT = 10000000
|
||||
|
||||
val DEFAULT_ROUTE_PARAMS = RouteParams(maxFeeBaseMsat = 21000, maxFeePct = 0.03, routeMaxCltv = 144, routeMaxLength = 6)
|
||||
val DEFAULT_ROUTE_PARAMS = RouteParams(maxFeeBaseMsat = 21000, maxFeePct = 0.03, routeMaxCltv = 2016, routeMaxLength = 6, ratios = None)
|
||||
|
||||
val DUMMY_SIG = BinaryData("3045022100e0a180fdd0fe38037cc878c03832861b40a29d32bd7b40b10c9e1efc8c1468a002205ae06d1624896d0d29f4b31e32772ea3cb1b4d7ed4e077e5da28dcc33c0e781201")
|
||||
|
||||
@ -782,18 +854,22 @@ object RouteCalculationSpec {
|
||||
ChannelAnnouncement(DUMMY_SIG, DUMMY_SIG, DUMMY_SIG, DUMMY_SIG, "", Block.RegtestGenesisBlock.hash, ShortChannelId(shortChannelId), nodeId1, nodeId2, randomKey.publicKey, randomKey.publicKey)
|
||||
}
|
||||
|
||||
def makeUpdate(shortChannelId: Long, nodeId1: PublicKey, nodeId2: PublicKey, feeBaseMsat: Int, feeProportionalMillionth: Int, minHtlcMsat: Long = DEFAULT_AMOUNT_MSAT, maxHtlcMsat: Option[Long] = None, cltv: Int = 0): (ChannelDesc, ChannelUpdate) =
|
||||
ChannelDesc(ShortChannelId(shortChannelId), nodeId1, nodeId2) -> ChannelUpdate(
|
||||
def makeUpdate(shortChannelId: Long, nodeId1: PublicKey, nodeId2: PublicKey, feeBaseMsat: Int, feeProportionalMillionth: Int, minHtlcMsat: Long = DEFAULT_AMOUNT_MSAT, maxHtlcMsat: Option[Long] = None, cltvDelta: Int = 0): (ChannelDesc, ChannelUpdate) = {
|
||||
makeUpdateShort(ShortChannelId(shortChannelId), nodeId1, nodeId2, feeBaseMsat, feeProportionalMillionth, minHtlcMsat, maxHtlcMsat, cltvDelta)
|
||||
}
|
||||
|
||||
def makeUpdateShort(shortChannelId: ShortChannelId, nodeId1: PublicKey, nodeId2: PublicKey, feeBaseMsat: Int, feeProportionalMillionth: Int, minHtlcMsat: Long = DEFAULT_AMOUNT_MSAT, maxHtlcMsat: Option[Long] = None, cltvDelta: Int = 0): (ChannelDesc, ChannelUpdate) =
|
||||
ChannelDesc(shortChannelId, nodeId1, nodeId2) -> ChannelUpdate(
|
||||
signature = DUMMY_SIG,
|
||||
chainHash = Block.RegtestGenesisBlock.hash,
|
||||
shortChannelId = ShortChannelId(shortChannelId),
|
||||
shortChannelId = shortChannelId,
|
||||
timestamp = 0L,
|
||||
messageFlags = maxHtlcMsat match {
|
||||
case Some(_) => 1
|
||||
case None => 0
|
||||
},
|
||||
channelFlags = 0,
|
||||
cltvExpiryDelta = cltv,
|
||||
cltvExpiryDelta = cltvDelta,
|
||||
htlcMinimumMsat = minHtlcMsat,
|
||||
feeBaseMsat = feeBaseMsat,
|
||||
feeProportionalMillionths = feeProportionalMillionth,
|
||||
@ -806,4 +882,8 @@ object RouteCalculationSpec {
|
||||
|
||||
def hops2Edges(route: Seq[Hop]) = route.map(hop => GraphEdge(ChannelDesc(hop.lastUpdate.shortChannelId, hop.nodeId, hop.nextNodeId), hop.lastUpdate))
|
||||
|
||||
def hops2ShortChannelIds(route: Seq[Hop]) = route.map(hop => hop.lastUpdate.shortChannelId.toString).toList
|
||||
|
||||
def hops2Nodes(route: Seq[Hop]) = route.map(hop => (hop.nodeId, hop.nextNodeId))
|
||||
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user