diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/db/PaymentsDb.scala b/eclair-core/src/main/scala/fr/acinq/eclair/db/PaymentsDb.scala index 325382558..3b520f225 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/db/PaymentsDb.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/db/PaymentsDb.scala @@ -199,7 +199,7 @@ case class HopSummary(nodeId: PublicKey, nextNodeId: PublicKey, shortChannelId: object HopSummary { def apply(h: Hop): HopSummary = { val shortChannelId = h match { - case ChannelHop(_, _, channelUpdate) => Some(channelUpdate.shortChannelId) + case ch: ChannelHop => Some(ch.shortChannelId) case _: NodeHop => None } HopSummary(h.nodeId, h.nextNodeId, shortChannelId) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/json/JsonSerializers.scala b/eclair-core/src/main/scala/fr/acinq/eclair/json/JsonSerializers.scala index b696715fe..d6941bfb8 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/json/JsonSerializers.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/json/JsonSerializers.scala @@ -31,12 +31,12 @@ import fr.acinq.eclair.io.Peer import fr.acinq.eclair.message.OnionMessages import fr.acinq.eclair.payment.PaymentFailure.PaymentFailedSummary import fr.acinq.eclair.payment._ -import fr.acinq.eclair.router.Router.{ChannelHop, Route} +import fr.acinq.eclair.router.Router.{ChannelHop, ChannelRelayParams, Route} import fr.acinq.eclair.transactions.DirectedHtlc import fr.acinq.eclair.transactions.Transactions._ import fr.acinq.eclair.wire.protocol.MessageOnionCodecs.blindedRouteCodec import fr.acinq.eclair.wire.protocol._ -import fr.acinq.eclair.{CltvExpiry, CltvExpiryDelta, FeatureSupport, Feature, MilliSatoshi, ShortChannelId, TimestampMilli, TimestampSecond, UInt64, UnknownFeature} +import fr.acinq.eclair.{CltvExpiry, CltvExpiryDelta, Feature, FeatureSupport, MilliSatoshi, ShortChannelId, TimestampMilli, TimestampSecond, UInt64, UnknownFeature} import org.json4s import org.json4s.JsonAST._ import org.json4s.jackson.Serialization @@ -290,8 +290,9 @@ object ColorSerializer extends MinimalSerializer({ }) // @formatter:off -private case class RouteFullJson(amount: MilliSatoshi, hops: Seq[ChannelHop]) -object RouteFullSerializer extends ConvertClassSerializer[Route](route => RouteFullJson(route.amount, route.hops)) +private case class ChannelHopJson(nodeId: PublicKey, nextNodeId: PublicKey, source: ChannelRelayParams) +private case class RouteFullJson(amount: MilliSatoshi, hops: Seq[ChannelHopJson]) +object RouteFullSerializer extends ConvertClassSerializer[Route](route => RouteFullJson(route.amount, route.hops.map(h => ChannelHopJson(h.nodeId, h.nextNodeId, h.params)))) private case class RouteNodeIdsJson(amount: MilliSatoshi, nodeIds: Seq[PublicKey]) object RouteNodeIdsSerializer extends ConvertClassSerializer[Route](route => { @@ -303,7 +304,7 @@ object RouteNodeIdsSerializer extends ConvertClassSerializer[Route](route => { }) private case class RouteShortChannelIdsJson(amount: MilliSatoshi, shortChannelIds: Seq[ShortChannelId]) -object RouteShortChannelIdsSerializer extends ConvertClassSerializer[Route](route => RouteShortChannelIdsJson(route.amount, route.hops.map(_.lastUpdate.shortChannelId))) +object RouteShortChannelIdsSerializer extends ConvertClassSerializer[Route](route => RouteShortChannelIdsJson(route.amount, route.hops.map(_.shortChannelId))) // @formatter:on // @formatter:off @@ -481,6 +482,11 @@ object CustomTypeHints { classOf[MessageReceivedJson] -> "onion-message-received" )) + val channelSources: CustomTypeHints = CustomTypeHints(Map( + classOf[ChannelRelayParams.FromAnnouncement] -> "announcement", + classOf[ChannelRelayParams.FromHint] -> "hint" + )) + val channelStates: ShortTypeHints = ShortTypeHints( List( classOf[Nothing], @@ -508,6 +514,7 @@ object JsonSerializers { CustomTypeHints.outgoingPaymentStatus + CustomTypeHints.paymentEvent + CustomTypeHints.onionMessageEvent + + CustomTypeHints.channelSources + CustomTypeHints.channelStates + ByteVectorSerializer + ByteVector32Serializer + diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/package.scala b/eclair-core/src/main/scala/fr/acinq/eclair/package.scala index 4a2cbb349..cf8779953 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/package.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/package.scala @@ -75,8 +75,6 @@ package object eclair { def nodeFee(relayFees: RelayFees, paymentAmount: MilliSatoshi): MilliSatoshi = nodeFee(relayFees.feeBase, relayFees.feeProportionalMillionths, paymentAmount) - def nodeFee(channelUpdate: ChannelUpdate, paymentAmount: MilliSatoshi): MilliSatoshi = nodeFee(channelUpdate.feeBaseMsat, channelUpdate.feeProportionalMillionths, paymentAmount) - /** * @param address base58 of bech32 address * @param chainHash hash of the chain we're on, which will be checked against the input address diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/Bolt11Invoice.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/Bolt11Invoice.scala index 2bcab282c..153c2b72d 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/Bolt11Invoice.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/Bolt11Invoice.scala @@ -19,6 +19,7 @@ package fr.acinq.eclair.payment import fr.acinq.bitcoin.scalacompat.Crypto.{PrivateKey, PublicKey} import fr.acinq.bitcoin.scalacompat.{Block, ByteVector32, ByteVector64, Crypto} import fr.acinq.bitcoin.{Base58, Base58Check, Bech32} +import fr.acinq.eclair.payment.relay.Relayer import fr.acinq.eclair.{CltvExpiryDelta, Feature, FeatureSupport, Features, InvoiceFeature, MilliSatoshi, MilliSatoshiLong, ShortChannelId, TimestampSecond, randomBytes32} import scodec.bits.{BitVector, ByteOrdering, ByteVector} import scodec.codecs.{list, ubyte} @@ -332,7 +333,9 @@ object Bolt11Invoice { * @param feeProportionalMillionths node proportional fee * @param cltvExpiryDelta node cltv expiry delta */ - case class ExtraHop(nodeId: PublicKey, shortChannelId: ShortChannelId, feeBase: MilliSatoshi, feeProportionalMillionths: Long, cltvExpiryDelta: CltvExpiryDelta) + case class ExtraHop(nodeId: PublicKey, shortChannelId: ShortChannelId, feeBase: MilliSatoshi, feeProportionalMillionths: Long, cltvExpiryDelta: CltvExpiryDelta) { + def relayFees: Relayer.RelayFees = Relayer.RelayFees(feeBase = feeBase, feeProportionalMillionths = feeProportionalMillionths) + } /** diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentEvents.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentEvents.scala index 64ec8c6a8..d63872276 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentEvents.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentEvents.scala @@ -180,7 +180,7 @@ object PaymentFailure { /** Ignore the channel outgoing from the given nodeId in the given route. */ private def ignoreNodeOutgoingChannel(nodeId: PublicKey, hops: Seq[Hop], ignore: Ignore): Ignore = { hops.collectFirst { - case hop: ChannelHop if hop.nodeId == nodeId => ChannelDesc(hop.lastUpdate.shortChannelId, hop.nodeId, hop.nextNodeId) + case hop: ChannelHop if hop.nodeId == nodeId => ChannelDesc(hop.shortChannelId, hop.nodeId, hop.nextNodeId) } match { case Some(faultyChannel) => ignore + faultyChannel case None => ignore @@ -219,7 +219,7 @@ object PaymentFailure { ignore ++ blacklist case LocalFailure(_, hops, _) => hops.headOption match { case Some(hop: ChannelHop) => - val faultyChannel = ChannelDesc(hop.lastUpdate.shortChannelId, hop.nodeId, hop.nextNodeId) + val faultyChannel = ChannelDesc(hop.shortChannelId, hop.nodeId, hop.nextNodeId) ignore + faultyChannel case _ => ignore } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentPacket.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentPacket.scala index 38094ab88..36ba5769a 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentPacket.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentPacket.scala @@ -170,7 +170,7 @@ object OutgoingPaymentPacket { hops.reverse.foldLeft((finalPayload.amount, finalPayload.expiry, Seq[PaymentOnion.PerHopPayload](finalPayload))) { case ((amount, expiry, payloads), hop) => val payload = hop match { - case hop: ChannelHop => PaymentOnion.ChannelRelayTlvPayload(hop.lastUpdate.shortChannelId, amount, expiry) + case hop: ChannelHop => PaymentOnion.ChannelRelayTlvPayload(hop.shortChannelId, amount, expiry) case hop: NodeHop => PaymentOnion.createNodeRelayPayload(amount, expiry, hop.nextNodeId) } (amount + hop.fee(amount), expiry + hop.cltvExpiryDelta, payload +: payloads) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/ChannelRelay.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/ChannelRelay.scala index 11c3e9dd1..818ab5b80 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/ChannelRelay.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/ChannelRelay.scala @@ -261,10 +261,10 @@ class ChannelRelay private(nodeParams: NodeParams, RelayFailure(CMD_FAIL_HTLC(add.id, Right(AmountBelowMinimum(payload.amountToForward, channelUpdate)), commit = true)) case Some(channelUpdate) if r.expiryDelta < channelUpdate.cltvExpiryDelta => RelayFailure(CMD_FAIL_HTLC(add.id, Right(IncorrectCltvExpiry(payload.outgoingCltv, channelUpdate)), commit = true)) - case Some(channelUpdate) if r.relayFeeMsat < nodeFee(channelUpdate, payload.amountToForward) && + case Some(channelUpdate) if r.relayFeeMsat < nodeFee(channelUpdate.relayFees, payload.amountToForward) && // fees also do not satisfy the previous channel update for `enforcementDelay` seconds after current update (TimestampSecond.now() - channelUpdate.timestamp > nodeParams.relayParams.enforcementDelay || - outgoingChannel_opt.flatMap(_.prevChannelUpdate).forall(c => r.relayFeeMsat < nodeFee(c, payload.amountToForward))) => + outgoingChannel_opt.flatMap(_.prevChannelUpdate).forall(c => r.relayFeeMsat < nodeFee(c.relayFees, payload.amountToForward))) => RelayFailure(CMD_FAIL_HTLC(add.id, Right(FeeInsufficient(add.amountMsat, channelUpdate)), commit = true)) case Some(channelUpdate) => val origin = Origin.ChannelRelayedHot(addResponseAdapter.toClassic, add, payload.amountToForward) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/PaymentLifecycle.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/PaymentLifecycle.scala index 249b2e961..b2256c42a 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/PaymentLifecycle.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/PaymentLifecycle.scala @@ -78,7 +78,7 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A log.info(s"route found: attempt=${failures.size + 1}/${c.maxAttempts} route=${route.printNodes()} channels=${route.printChannels()}") OutgoingPaymentPacket.buildCommand(self, cfg.upstream, paymentHash, route.hops, c.finalPayload) match { case Success((cmd, sharedSecrets)) => - register ! Register.ForwardShortId(self, route.hops.head.lastUpdate.shortChannelId, cmd) + register ! Register.ForwardShortId(self, route.hops.head.shortChannelId, cmd) goto(WAITING_FOR_PAYMENT_COMPLETE) using WaitingForComplete(c, cmd, failures, sharedSecrets, ignore, route) case Failure(t) => log.warning("cannot send outgoing payment: {}", t.getMessage) @@ -242,6 +242,7 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A * @return updated routing hints if applicable. */ private def handleUpdate(nodeId: PublicKey, failure: Update, data: WaitingForComplete): Seq[Seq[ExtraHop]] = { + // TODO: properly handle updates to channels provided as routing hints in the invoice data.route.getChannelUpdateForNode(nodeId) match { case Some(u) if u.shortChannelId != failure.update.shortChannelId => // it is possible that nodes in the route prefer using a different channel (to the same N+1 node) than the one we requested, that's fine diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/router/Graph.scala b/eclair-core/src/main/scala/fr/acinq/eclair/router/Graph.scala index 65e52fded..2cf45c35e 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/router/Graph.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/router/Graph.scala @@ -17,7 +17,7 @@ package fr.acinq.eclair.router import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey -import fr.acinq.bitcoin.scalacompat.{Btc, ByteVector32, MilliBtc, Satoshi} +import fr.acinq.bitcoin.scalacompat.{Btc, MilliBtc, Satoshi, SatoshiLong} import fr.acinq.eclair._ import fr.acinq.eclair.payment.relay.Relayer.RelayFees import fr.acinq.eclair.router.Graph.GraphStructure.{DirectedGraph, GraphEdge} @@ -248,8 +248,8 @@ object Graph { val neighbor = edge.desc.a if (current.weight.amount <= edge.capacity && edge.balance_opt.forall(current.weight.amount <= _) && - edge.update.htlcMaximumMsat.forall(current.weight.amount <= _) && - current.weight.amount >= edge.update.htlcMinimumMsat && + edge.params.htlcMaximum_opt.forall(current.weight.amount <= _) && + current.weight.amount >= edge.params.htlcMinimum && !ignoredEdges.contains(edge.desc) && !ignoredVertices.contains(neighbor)) { // NB: this contains the amount (including fees) that will need to be sent to `neighbor`, but the amount that @@ -302,7 +302,7 @@ object Graph { val totalAmount = if (edge.desc.a == sender && !includeLocalChannelCost) prev.amount else addEdgeFees(edge, prev.amount) val fee = totalAmount - prev.amount val totalFees = prev.fees + fee - val cltv = if (edge.desc.a == sender && !includeLocalChannelCost) CltvExpiryDelta(0) else edge.update.cltvExpiryDelta + val cltv = if (edge.desc.a == sender && !includeLocalChannelCost) CltvExpiryDelta(0) else edge.params.cltvExpiryDelta val totalCltv = prev.cltv + cltv weightRatios match { case Left(weightRatios) => @@ -320,7 +320,7 @@ object Graph { else 1 - normalize(edgeMaxCapacity.toLong.toDouble, CAPACITY_CHANNEL_LOW.toLong.toDouble, CAPACITY_CHANNEL_HIGH.toLong.toDouble) // Every edge is weighted by its cltv-delta value, normalized - val cltvFactor = normalize(edge.update.cltvExpiryDelta.toInt, CLTV_LOW, CLTV_HIGH) + val cltvFactor = normalize(edge.params.cltvExpiryDelta.toInt, CLTV_LOW, CLTV_HIGH) // NB we're guaranteed to have weightRatios and factors > 0 val factor = weightRatios.baseFactor + (cltvFactor * weightRatios.cltvDeltaFactor) + (ageFactor * weightRatios.ageFactor) + (capFactor * weightRatios.capacityFactor) @@ -329,9 +329,8 @@ object Graph { case Right(heuristicsConstants) => val hopCost = nodeFee(heuristicsConstants.hopCost, prev.amount) val totalHopsCost = prev.virtualFees + hopCost - // If the edge was added by the invoice, it is assumed that it can route the payment. // If we know the balance of the channel, then we will check separately that it can relay the payment. - val successProbability = if (edge.update.chainHash == ByteVector32.Zeroes || edge.balance_opt.nonEmpty) 1.0 else 1.0 - prev.amount.toLong.toDouble / edge.capacity.toMilliSatoshi.toLong.toDouble + val successProbability = if (edge.balance_opt.nonEmpty) 1.0 else 1.0 - prev.amount.toLong.toDouble / edge.capacity.toMilliSatoshi.toLong.toDouble if (successProbability < 0) { throw NegativeProbability(edge, prev, heuristicsConstants) } @@ -358,7 +357,7 @@ object Graph { * @return the new amount updated with the necessary fees for this edge */ private def addEdgeFees(edge: GraphEdge, amountToForward: MilliSatoshi): MilliSatoshi = { - amountToForward + nodeFee(edge.update, amountToForward) + amountToForward + edge.params.fee(amountToForward) } /** Validate that all edges along the path can relay the amount with fees. */ @@ -370,8 +369,8 @@ object Graph { case Some(edge) => val canRelayAmount = amount <= edge.capacity && edge.balance_opt.forall(amount <= _) && - edge.update.htlcMaximumMsat.forall(amount <= _) && - edge.update.htlcMinimumMsat <= amount + edge.params.htlcMaximum_opt.forall(amount <= _) && + edge.params.htlcMinimum <= amount if (canRelayAmount) validateReversePath(path.tail, addEdgeFees(edge, amount)) else false } @@ -423,27 +422,49 @@ object Graph { * Representation of an edge of the graph * * @param desc channel description - * @param update channel info + * @param params source of the channel parameters: can be a channel_update or hints from an invoice * @param capacity channel capacity * @param balance_opt (optional) available balance that can be sent through this edge */ - case class GraphEdge(desc: ChannelDesc, update: ChannelUpdate, capacity: Satoshi, balance_opt: Option[MilliSatoshi]) { + case class GraphEdge private(desc: ChannelDesc, params: ChannelRelayParams, capacity: Satoshi, balance_opt: Option[MilliSatoshi]) { def maxHtlcAmount(reservedCapacity: MilliSatoshi): MilliSatoshi = Seq( balance_opt.map(balance => balance - reservedCapacity), - update.htlcMaximumMsat, + params.htlcMaximum_opt, Some(capacity.toMilliSatoshi - reservedCapacity) ).flatten.min.max(0 msat) - def fee(amount: MilliSatoshi): MilliSatoshi = nodeFee(update, amount) + def fee(amount: MilliSatoshi): MilliSatoshi = params.fee(amount) + } + object GraphEdge { + def apply(u: ChannelUpdate, pc: PublicChannel): GraphEdge = GraphEdge( + desc = ChannelDesc(u, pc.ann), + params = ChannelRelayParams.FromAnnouncement(u), + capacity = pc.capacity, + balance_opt = pc.getBalanceSameSideAs(u) + ) + + def apply(u: ChannelUpdate, pc: PrivateChannel): GraphEdge = GraphEdge( + desc = ChannelDesc(u, pc), + params = ChannelRelayParams.FromAnnouncement(u), + capacity = pc.capacity, + balance_opt = pc.getBalanceSameSideAs(u) + ) + + def apply(ac: AssistedChannel): GraphEdge = GraphEdge( + desc = ChannelDesc(ac.shortChannelId, ac.nodeId, ac.nextNodeId), + params = ac.params, + // Bolt 11 routing hints don't include the channel's capacity, so we round up the maximum htlc amount + capacity = ac.params.htlcMaximum.truncateToSatoshi + 1.sat, + // we assume channels provided as hints have enough balance to handle the payment + balance_opt = Some(ac.params.htlcMaximum) + ) } /** A graph data structure that uses an adjacency list, stores the incoming edges of the neighbors */ case class DirectedGraph(private val vertices: Map[PublicKey, List[GraphEdge]]) { - def addEdge(d: ChannelDesc, u: ChannelUpdate, capacity: Satoshi, balance_opt: Option[MilliSatoshi] = None): DirectedGraph = addEdge(GraphEdge(d, u, capacity, balance_opt)) - def addEdges(edges: Iterable[GraphEdge]): DirectedGraph = edges.foldLeft(this)((acc, edge) => acc.addEdge(edge)) /** @@ -595,27 +616,21 @@ object Graph { // add all the vertices and edges in one go channels.values.foreach { channel => - channel.update_1_opt.foreach { u1 => - val desc1 = Router.getDesc(u1, channel.ann) - addDescToMap(desc1, u1, channel.capacity, channel.meta_opt.map(_.balance1)) - } - channel.update_2_opt.foreach { u2 => - val desc2 = Router.getDesc(u2, channel.ann) - addDescToMap(desc2, u2, channel.capacity, channel.meta_opt.map(_.balance2)) - } + channel.update_1_opt.foreach(u1 => addToMap(GraphEdge(u1, channel))) + channel.update_2_opt.foreach(u2 => addToMap(GraphEdge(u2, channel))) } - def addDescToMap(desc: ChannelDesc, u: ChannelUpdate, capacity: Satoshi, balance_opt: Option[MilliSatoshi]): Unit = { - mutableMap.put(desc.b, GraphEdge(desc, u, capacity, balance_opt) +: mutableMap.getOrElse(desc.b, List.empty[GraphEdge])) - if (!mutableMap.contains(desc.a)) { - mutableMap += desc.a -> List.empty[GraphEdge] + def addToMap(edge: GraphEdge): Unit = { + mutableMap.put(edge.desc.b, edge +: mutableMap.getOrElse(edge.desc.b, List.empty[GraphEdge])) + if (!mutableMap.contains(edge.desc.a)) { + mutableMap += edge.desc.a -> List.empty[GraphEdge] } } new DirectedGraph(mutableMap.toMap) } - def graphEdgeToHop(graphEdge: GraphEdge): ChannelHop = ChannelHop(graphEdge.desc.a, graphEdge.desc.b, graphEdge.update) + def graphEdgeToHop(graphEdge: GraphEdge): ChannelHop = ChannelHop(graphEdge.desc.shortChannelId, graphEdge.desc.a, graphEdge.desc.b, graphEdge.params) } } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/router/RouteCalculation.scala b/eclair-core/src/main/scala/fr/acinq/eclair/router/RouteCalculation.scala index 34ab89493..cf51a2cc9 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/router/RouteCalculation.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/router/RouteCalculation.scala @@ -47,9 +47,7 @@ object RouteCalculation { implicit val sender: ActorRef = ctx.self // necessary to preserve origin when sending messages to other actors val assistedChannels: Map[ShortChannelId, AssistedChannel] = fr.assistedRoutes.flatMap(toAssistedChannels(_, fr.route.targetNodeId, fr.amount)).toMap - val extraEdges = assistedChannels.values.map(ac => - GraphEdge(ChannelDesc(ac.extraHop.shortChannelId, ac.extraHop.nodeId, ac.nextNodeId), toFakeUpdate(ac.extraHop, ac.htlcMaximum), htlcMaxToCapacity(ac.htlcMaximum), Some(ac.htlcMaximum)) - ).toSet + val extraEdges = assistedChannels.values.map(ac => GraphEdge(ac)).toSet val g = extraEdges.foldLeft(d.graph) { case (g: DirectedGraph, e: GraphEdge) => g.addEdge(e) } fr.route match { @@ -59,33 +57,33 @@ object RouteCalculation { case edges if edges.nonEmpty && edges.forall(_.nonEmpty) => // select the largest edge (using balance when available, otherwise capacity). val selectedEdges = edges.map(es => es.maxBy(e => e.balance_opt.getOrElse(e.capacity.toMilliSatoshi))) - val hops = selectedEdges.map(d => ChannelHop(d.desc.a, d.desc.b, d.update)) + val hops = selectedEdges.map(d => ChannelHop(d.desc.shortChannelId, d.desc.a, d.desc.b, d.params)) ctx.sender() ! RouteResponse(Route(fr.amount, hops) :: Nil) case _ => // some nodes in the supplied route aren't connected in our graph ctx.sender() ! Status.Failure(new IllegalArgumentException("Not all the nodes in the supplied route are connected with public channels")) } - case PredefinedChannelRoute(targetNodeId, channels) => - val (end, hops) = channels.foldLeft((localNodeId, Seq.empty[ChannelHop])) { - case ((start, current), shortChannelId) => - val channelDesc_opt = d.channels.get(shortChannelId).flatMap(c => start match { + case PredefinedChannelRoute(targetNodeId, shortChannelIds) => + val (end, hops) = shortChannelIds.foldLeft((localNodeId, Seq.empty[ChannelHop])) { + case ((currentNode, previousHops), shortChannelId) => + val channelDesc_opt = d.channels.get(shortChannelId).flatMap(c => currentNode match { case c.ann.nodeId1 => Some(ChannelDesc(shortChannelId, c.ann.nodeId1, c.ann.nodeId2)) case c.ann.nodeId2 => Some(ChannelDesc(shortChannelId, c.ann.nodeId2, c.ann.nodeId1)) case _ => None - }).orElse(d.privateChannels.get(shortChannelId).flatMap(c => start match { + }).orElse(d.privateChannels.get(shortChannelId).flatMap(c => currentNode match { case c.nodeId1 => Some(ChannelDesc(shortChannelId, c.nodeId1, c.nodeId2)) case c.nodeId2 => Some(ChannelDesc(shortChannelId, c.nodeId2, c.nodeId1)) case _ => None - })).orElse(assistedChannels.get(shortChannelId).flatMap(c => start match { - case c.extraHop.nodeId => Some(ChannelDesc(shortChannelId, c.extraHop.nodeId, c.nextNodeId)) + })).orElse(assistedChannels.get(shortChannelId).flatMap(c => currentNode match { + case c.nodeId => Some(ChannelDesc(shortChannelId, c.nodeId, c.nextNodeId)) case _ => None })) channelDesc_opt.flatMap(c => g.getEdge(c)) match { - case Some(edge) => (edge.desc.b, current :+ ChannelHop(edge.desc.a, edge.desc.b, edge.update)) - case None => (start, current) + case Some(edge) => (edge.desc.b, previousHops :+ ChannelHop(edge.desc.shortChannelId, edge.desc.a, edge.desc.b, edge.params)) + case None => (currentNode, previousHops) } } - if (end != targetNodeId || hops.length != channels.length) { + if (end != targetNodeId || hops.length != shortChannelIds.length) { ctx.sender() ! Status.Failure(new IllegalArgumentException("The sequence of channels provided cannot be used to build a route to the target node")) } else { ctx.sender() ! RouteResponse(Route(fr.amount, hops) :: Nil) @@ -107,11 +105,9 @@ object RouteCalculation { // we convert extra routing info provided in the invoice to fake channel_update // it takes precedence over all other channel_updates we know val assistedChannels: Map[ShortChannelId, AssistedChannel] = r.assistedRoutes.flatMap(toAssistedChannels(_, r.target, r.amount)) - .filterNot { case (_, ac) => ac.extraHop.nodeId == r.source } // we ignore routing hints for our own channels, we have more accurate information + .filterNot { case (_, ac) => ac.nodeId == r.source } // we ignore routing hints for our own channels, we have more accurate information .toMap - val extraEdges = assistedChannels.values.map(ac => - GraphEdge(ChannelDesc(ac.extraHop.shortChannelId, ac.extraHop.nodeId, ac.nextNodeId), toFakeUpdate(ac.extraHop, ac.htlcMaximum), htlcMaxToCapacity(ac.htlcMaximum), Some(ac.htlcMaximum)) - ).toSet + val extraEdges = assistedChannels.values.map(ac => GraphEdge(ac)).toSet val ignoredEdges = r.ignore.channels ++ d.excludedChannels val params = r.routeParams val routesToFind = if (params.randomize) DEFAULT_ROUTES_COUNT else 1 @@ -149,24 +145,17 @@ object RouteCalculation { } } - private def toFakeUpdate(extraHop: ExtraHop, htlcMaximum: MilliSatoshi): 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 = ByteVector64.Zeroes, chainHash = ByteVector32.Zeroes, extraHop.shortChannelId, TimestampSecond.now(), channelFlags = ChannelUpdate.ChannelFlags(isNode1 = true, isEnabled = true), extraHop.cltvExpiryDelta, htlcMinimumMsat = 0 msat, extraHop.feeBase, extraHop.feeProportionalMillionths, Some(htlcMaximum)) - } - def toAssistedChannels(extraRoute: Seq[ExtraHop], targetNodeId: PublicKey, amount: MilliSatoshi): Map[ShortChannelId, AssistedChannel] = { // BOLT 11: "For each entry, the pubkey is the node ID of the start of the channel", and the last node is the destination // The invoice doesn't explicitly specify the channel's htlcMaximumMsat, but we can safely assume that the channel // should be able to route the payment, so we'll compute an htlcMaximumMsat accordingly. - // We could also get the channel capacity from the blockchain (since we have the shortChannelId) but that's more expensive. // We also need to make sure the channel isn't excluded by our heuristics. val lastChannelCapacity = amount.max(RoutingHeuristics.CAPACITY_CHANNEL_LOW) val nextNodeIds = extraRoute.map(_.nodeId).drop(1) :+ targetNodeId extraRoute.zip(nextNodeIds).reverse.foldLeft((lastChannelCapacity, Map.empty[ShortChannelId, AssistedChannel])) { case ((amount, acs), (extraHop: ExtraHop, nextNodeId)) => - val nextAmount = amount + nodeFee(extraHop.feeBase, extraHop.feeProportionalMillionths, amount) - (nextAmount, acs + (extraHop.shortChannelId -> AssistedChannel(extraHop, nextNodeId, nextAmount))) + val nextAmount = amount + nodeFee(extraHop.relayFees, amount) + (nextAmount, acs + (extraHop.shortChannelId -> AssistedChannel(nextNodeId, Router.ChannelRelayParams.FromHint(extraHop, nextAmount)))) }._2 } @@ -175,9 +164,6 @@ object RouteCalculation { extraRoute.zip(nextNodeIds).map { case (hop, nextNodeId) => ChannelDesc(hop.shortChannelId, hop.nodeId, nextNodeId) } } - /** Bolt 11 routing hints don't include the channel's capacity, so we round up the maximum htlc amount. */ - private def htlcMaxToCapacity(htlcMaximum: MilliSatoshi): Satoshi = htlcMaximum.truncateToSatoshi + 1.sat - /** This method is used after a payment failed, and we want to exclude some nodes that we know are failing */ def getIgnoredChannelDesc(channels: Map[ShortChannelId, PublicChannel], ignoreNodes: Set[PublicKey]): Iterable[ChannelDesc] = { val desc = if (ignoreNodes.isEmpty) { @@ -334,7 +320,7 @@ object RouteCalculation { val directChannels = g.getEdgesBetween(localNodeId, targetNodeId).collect { // We should always have balance information available for local channels. // NB: htlcMinimumMsat is set by our peer and may be 0 msat (even though it's not recommended). - case GraphEdge(_, update, _, Some(balance)) => DirectChannel(balance, balance <= 0.msat || balance < update.htlcMinimumMsat) + case GraphEdge(_, params, _, Some(balance)) => DirectChannel(balance, balance <= 0.msat || balance < params.htlcMinimum) } // If we have direct channels to the target, we can use them all. // We also count empty channels, which allows replacing them with a non-direct route (multiple hops). @@ -388,12 +374,12 @@ object RouteCalculation { /** Compute the maximum amount that we can send through the given route. */ private def computeRouteMaxAmount(route: Seq[GraphEdge], usedCapacity: mutable.Map[ShortChannelId, MilliSatoshi]): Route = { - val firstHopMaxAmount = route.head.maxHtlcAmount(usedCapacity.getOrElse(route.head.update.shortChannelId, 0 msat)) + val firstHopMaxAmount = route.head.maxHtlcAmount(usedCapacity.getOrElse(route.head.desc.shortChannelId, 0 msat)) val amount = route.drop(1).foldLeft(firstHopMaxAmount) { case (amount, edge) => // We compute fees going forward instead of backwards. That means we will slightly overestimate the fees of some // edges, but we will always stay inside the capacity bounds we computed. val amountMinusFees = amount - edge.fee(amount) - val edgeMaxAmount = edge.maxHtlcAmount(usedCapacity.getOrElse(edge.update.shortChannelId, 0 msat)) + val edgeMaxAmount = edge.maxHtlcAmount(usedCapacity.getOrElse(edge.desc.shortChannelId, 0 msat)) amountMinusFees.min(edgeMaxAmount) } Route(amount.max(0 msat), route.map(graphEdgeToHop)) @@ -411,7 +397,7 @@ object RouteCalculation { /** Update used capacity by taking into account an HTLC sent to the given route. */ private def updateUsedCapacity(route: Route, usedCapacity: mutable.Map[ShortChannelId, MilliSatoshi]): Unit = { route.hops.reverse.foldLeft(route.amount) { case (amount, hop) => - usedCapacity.updateWith(hop.lastUpdate.shortChannelId)(previous => Some(amount + previous.getOrElse(0 msat))) + usedCapacity.updateWith(hop.shortChannelId)(previous => Some(amount + previous.getOrElse(0 msat))) amount + hop.fee(amount) } } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/router/Router.scala b/eclair-core/src/main/scala/fr/acinq/eclair/router/Router.scala index 7a6f1fcc4..1e002d087 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/router/Router.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/router/Router.scala @@ -31,7 +31,9 @@ import fr.acinq.eclair.channel._ import fr.acinq.eclair.crypto.TransportHandler import fr.acinq.eclair.db.NetworkDb import fr.acinq.eclair.io.Peer.PeerRoutingMessage +import fr.acinq.eclair.payment.Bolt11Invoice import fr.acinq.eclair.payment.Bolt11Invoice.ExtraHop +import fr.acinq.eclair.payment.relay.Relayer import fr.acinq.eclair.remote.EclairInternalsSerializer.RemoteTypes import fr.acinq.eclair.router.Graph.GraphStructure.DirectedGraph import fr.acinq.eclair.router.Graph.{HeuristicsConstants, WeightRatios} @@ -310,7 +312,17 @@ object Router { } // @formatter:off - case class ChannelDesc(shortChannelId: ShortChannelId, a: PublicKey, b: PublicKey) + case class ChannelDesc private(shortChannelId: ShortChannelId, a: PublicKey, b: PublicKey) + object ChannelDesc { + def apply(u: ChannelUpdate, ann: ChannelAnnouncement): ChannelDesc = { + // the least significant bit tells us if it is node1 or node2 + if (u.channelFlags.isNode1) ChannelDesc(ann.shortChannelId, ann.nodeId1, ann.nodeId2) else ChannelDesc(ann.shortChannelId, ann.nodeId2, ann.nodeId1) + } + def apply(u: ChannelUpdate, pc: PrivateChannel): ChannelDesc = { + // the least significant bit tells us if it is node1 or node2 + if (u.channelFlags.isNode1) ChannelDesc(u.shortChannelId, pc.nodeId1, pc.nodeId2) else ChannelDesc(u.shortChannelId, pc.nodeId2, pc.nodeId1) + } + } case class ChannelMeta(balance1: MilliSatoshi, balance2: MilliSatoshi) sealed trait ChannelDetails { val capacity: Satoshi @@ -377,7 +389,10 @@ object Router { } // @formatter:on - case class AssistedChannel(extraHop: ExtraHop, nextNodeId: PublicKey, htlcMaximum: MilliSatoshi) + case class AssistedChannel(nextNodeId: PublicKey, params: ChannelRelayParams.FromHint) { + val nodeId: PublicKey = params.extraHop.nodeId + val shortChannelId: ShortChannelId = params.extraHop.shortChannelId + } trait Hop { /** @return the id of the start node. */ @@ -396,17 +411,47 @@ object Router { def cltvExpiryDelta: CltvExpiryDelta } + // @formatter:off + /** Channel routing parameters */ + sealed trait ChannelRelayParams { + def cltvExpiryDelta: CltvExpiryDelta + def relayFees: Relayer.RelayFees + final def fee(amount: MilliSatoshi): MilliSatoshi = nodeFee(relayFees, amount) + def htlcMinimum: MilliSatoshi + def htlcMaximum_opt: Option[MilliSatoshi] + } + + object ChannelRelayParams { + /** We learnt about this channel from a channel_update */ + case class FromAnnouncement(channelUpdate: ChannelUpdate) extends ChannelRelayParams { + override def cltvExpiryDelta: CltvExpiryDelta = channelUpdate.cltvExpiryDelta + override def relayFees: Relayer.RelayFees = channelUpdate.relayFees + override def htlcMinimum: MilliSatoshi = channelUpdate.htlcMinimumMsat + override def htlcMaximum_opt: Option[MilliSatoshi] = channelUpdate.htlcMaximumMsat + } + /** We learnt about this channel from hints in an invoice */ + case class FromHint(extraHop: Bolt11Invoice.ExtraHop, htlcMaximum: MilliSatoshi) extends ChannelRelayParams { + override def cltvExpiryDelta: CltvExpiryDelta = extraHop.cltvExpiryDelta + override def relayFees: Relayer.RelayFees = extraHop.relayFees + override def htlcMinimum: MilliSatoshi = 0 msat + override def htlcMaximum_opt: Option[MilliSatoshi] = Some(htlcMaximum) + } + } + // @formatter:on + /** * A directed hop between two connected nodes using a specific channel. * - * @param nodeId id of the start node. - * @param nextNodeId id of the end node. - * @param lastUpdate last update of the channel used for the hop. + * @param nodeId id of the start node. + * @param nextNodeId id of the end node. + * @param shortChannelId scid that will be used to build the payment onion. + * @param params source for the channel parameters. */ - case class ChannelHop(nodeId: PublicKey, nextNodeId: PublicKey, lastUpdate: ChannelUpdate) extends Hop { - override lazy val cltvExpiryDelta: CltvExpiryDelta = lastUpdate.cltvExpiryDelta - - override def fee(amount: MilliSatoshi): MilliSatoshi = nodeFee(lastUpdate, amount) + case class ChannelHop(shortChannelId: ShortChannelId, nodeId: PublicKey, nextNodeId: PublicKey, params: ChannelRelayParams) extends Hop { + // @formatter:off + override def cltvExpiryDelta: CltvExpiryDelta = params.cltvExpiryDelta + override def fee(amount: MilliSatoshi): MilliSatoshi = params.fee(amount) + // @formatter:on } /** @@ -484,11 +529,11 @@ object Router { } /** This method retrieves the channel update that we used when we built the route. */ - def getChannelUpdateForNode(nodeId: PublicKey): Option[ChannelUpdate] = hops.find(_.nodeId == nodeId).map(_.lastUpdate) + def getChannelUpdateForNode(nodeId: PublicKey): Option[ChannelUpdate] = hops.find(_.nodeId == nodeId).map(_.params).collect { case s: ChannelRelayParams.FromAnnouncement => s.channelUpdate } def printNodes(): String = hops.map(_.nextNodeId).mkString("->") - def printChannels(): String = hops.map(_.lastUpdate.shortChannelId).mkString("->") + def printChannels(): String = hops.map(_.shortChannelId).mkString("->") } @@ -590,16 +635,6 @@ object Router { case object TickPruneStaleChannels // @formatter:on - def getDesc(u: ChannelUpdate, channel: ChannelAnnouncement): ChannelDesc = { - // the least significant bit tells us if it is node1 or node2 - if (u.channelFlags.isNode1) ChannelDesc(u.shortChannelId, channel.nodeId1, channel.nodeId2) else ChannelDesc(u.shortChannelId, channel.nodeId2, channel.nodeId1) - } - - def getDesc(u: ChannelUpdate, pc: PrivateChannel): ChannelDesc = { - // the least significant bit tells us if it is node1 or node2 - if (u.channelFlags.isNode1) ChannelDesc(u.shortChannelId, pc.nodeId1, pc.nodeId2) else ChannelDesc(u.shortChannelId, pc.nodeId2, pc.nodeId1) - } - def isRelatedTo(c: ChannelAnnouncement, nodeId: PublicKey) = nodeId == c.nodeId1 || nodeId == c.nodeId2 def hasChannels(nodeId: PublicKey, channels: Iterable[PublicChannel]): Boolean = channels.exists(c => isRelatedTo(c.ann, nodeId)) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/router/Validation.scala b/eclair-core/src/main/scala/fr/acinq/eclair/router/Validation.scala index 5f4d66c73..ed0cce791 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/router/Validation.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/router/Validation.scala @@ -26,6 +26,7 @@ import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher.{UtxoStatus, ValidateReque import fr.acinq.eclair.channel.{AvailableBalanceChanged, LocalChannelDown, LocalChannelUpdate} import fr.acinq.eclair.crypto.TransportHandler import fr.acinq.eclair.db.NetworkDb +import fr.acinq.eclair.router.Graph.GraphStructure.GraphEdge import fr.acinq.eclair.router.Monitoring.Metrics import fr.acinq.eclair.router.Router._ import fr.acinq.eclair.transactions.Scripts @@ -263,14 +264,13 @@ object Validation { // related channel is already known (note: this means no related channel_update is in the stash) val publicChannel = true val pc = d.channels(u.shortChannelId) - val desc = getDesc(u, pc.ann) if (d.rebroadcast.updates.contains(u)) { log.debug("ignoring {} (pending rebroadcast)", u) sendDecision(origins, GossipDecision.Accepted(u)) val origins1 = d.rebroadcast.updates(u) ++ origins // NB: we update the channels because the balances may have changed even if the channel_update is the same. val pc1 = pc.applyChannelUpdate(update) - val graph1 = d.graph.addEdge(desc, u, pc1.capacity, pc1.getBalanceSameSideAs(u)) + val graph1 = d.graph.addEdge(GraphEdge(u, pc1)) d.copy(rebroadcast = d.rebroadcast.copy(updates = d.rebroadcast.updates + (u -> origins1)), channels = d.channels + (u.shortChannelId -> pc1), graph = graph1) } else if (StaleChannels.isStale(u)) { log.debug("ignoring {} (stale)", u) @@ -283,7 +283,7 @@ object Validation { case Left(_) => // NB: we update the graph because the balances may have changed even if the channel_update is the same. val pc1 = pc.applyChannelUpdate(update) - val graph1 = d.graph.addEdge(desc, u, pc1.capacity, pc1.getBalanceSameSideAs(u)) + val graph1 = d.graph.addEdge(GraphEdge(u, pc1)) d.copy(channels = d.channels + (u.shortChannelId -> pc1), graph = graph1) case Right(_) => d } @@ -301,10 +301,10 @@ object Validation { val pc1 = pc.applyChannelUpdate(update) val graph1 = if (u.channelFlags.isEnabled) { update.left.foreach(_ => log.info("added local shortChannelId={} public={} to the network graph", u.shortChannelId, publicChannel)) - d.graph.addEdge(desc, u, pc1.capacity, pc1.getBalanceSameSideAs(u)) + d.graph.addEdge(GraphEdge(u, pc1)) } else { update.left.foreach(_ => log.info("removed local shortChannelId={} public={} from the network graph", u.shortChannelId, publicChannel)) - d.graph.removeEdge(desc) + d.graph.removeEdge(ChannelDesc(u, pc.ann)) } d.copy(channels = d.channels + (u.shortChannelId -> pc1), rebroadcast = d.rebroadcast.copy(updates = d.rebroadcast.updates + (u -> origins)), graph = graph1) } else { @@ -314,7 +314,7 @@ object Validation { db.updateChannel(u) // we also need to update the graph val pc1 = pc.applyChannelUpdate(update) - val graph1 = d.graph.addEdge(desc, u, pc1.capacity, pc1.getBalanceSameSideAs(u)) + val graph1 = d.graph.addEdge(GraphEdge(u, pc1)) update.left.foreach(_ => log.info("added local shortChannelId={} public={} to the network graph", u.shortChannelId, publicChannel)) d.copy(channels = d.channels + (u.shortChannelId -> pc1), privateChannels = d.privateChannels - u.shortChannelId, rebroadcast = d.rebroadcast.copy(updates = d.rebroadcast.updates + (u -> origins)), graph = graph1) } @@ -331,7 +331,6 @@ object Validation { } else if (d.privateChannels.contains(u.shortChannelId)) { val publicChannel = false val pc = d.privateChannels(u.shortChannelId) - val desc = getDesc(u, pc) if (StaleChannels.isStale(u)) { log.debug("ignoring {} (stale)", u) sendDecision(origins, GossipDecision.Stale(u)) @@ -340,7 +339,7 @@ object Validation { log.debug("ignoring {} (already know same or newer)", u) sendDecision(origins, GossipDecision.Duplicate(u)) d - } else if (!Announcements.checkSig(u, desc.a)) { + } else if (!Announcements.checkSig(u, pc.getNodeIdSameSideAs(u))) { log.warning("bad signature for announcement shortChannelId={} {}", u.shortChannelId, u) sendDecision(origins, GossipDecision.InvalidSignature(u)) d @@ -353,10 +352,10 @@ object Validation { val pc1 = pc.applyChannelUpdate(update) val graph1 = if (u.channelFlags.isEnabled) { update.left.foreach(_ => log.info("added local shortChannelId={} public={} to the network graph", u.shortChannelId, publicChannel)) - d.graph.addEdge(desc, u, pc1.capacity, pc1.getBalanceSameSideAs(u)) + d.graph.addEdge(GraphEdge(u, pc1)) } else { update.left.foreach(_ => log.info("removed local shortChannelId={} public={} from the network graph", u.shortChannelId, publicChannel)) - d.graph.removeEdge(desc) + d.graph.removeEdge(ChannelDesc(u, pc1)) } d.copy(privateChannels = d.privateChannels + (u.shortChannelId -> pc1), graph = graph1) } else { @@ -365,7 +364,7 @@ object Validation { ctx.system.eventStream.publish(ChannelUpdatesReceived(u :: Nil)) // we also need to update the graph val pc1 = pc.applyChannelUpdate(update) - val graph1 = d.graph.addEdge(desc, u, pc1.capacity, pc1.getBalanceSameSideAs(u)) + val graph1 = d.graph.addEdge(GraphEdge(u, pc1)) update.left.foreach(_ => log.info("added local shortChannelId={} public={} to the network graph", u.shortChannelId, publicChannel)) d.copy(privateChannels = d.privateChannels + (u.shortChannelId -> pc1), graph = graph1) } @@ -462,14 +461,13 @@ object Validation { } def handleAvailableBalanceChanged(d: Data, e: AvailableBalanceChanged)(implicit log: LoggingAdapter): Data = { - val desc = ChannelDesc(e.shortChannelId, e.commitments.localNodeId, e.commitments.remoteNodeId) val (publicChannels1, graph1) = d.channels.get(e.shortChannelId) match { case Some(pc) => val pc1 = pc.updateBalances(e.commitments) log.debug("public channel balance updated: {}", pc1) val update_opt = if (e.commitments.localNodeId == pc1.ann.nodeId1) pc1.update_1_opt else pc1.update_2_opt - val graph1 = update_opt.map(u => d.graph.addEdge(desc, u, pc1.capacity, pc1.getBalanceSameSideAs(u))).getOrElse(d.graph) - (d.channels + (e.shortChannelId -> pc1), graph1) + val graph1 = update_opt.map(u => d.graph.addEdge(GraphEdge(u, pc1))).getOrElse(d.graph) + (d.channels + (pc.ann.shortChannelId -> pc1), graph1) case None => (d.channels, d.graph) } @@ -478,7 +476,7 @@ object Validation { val pc1 = pc.updateBalances(e.commitments) log.debug("private channel balance updated: {}", pc1) val update_opt = if (e.commitments.localNodeId == pc1.nodeId1) pc1.update_1_opt else pc1.update_2_opt - val graph2 = update_opt.map(u => graph1.addEdge(desc, u, pc1.capacity, pc1.getBalanceSameSideAs(u))).getOrElse(graph1) + val graph2 = update_opt.map(u => graph1.addEdge(GraphEdge(u, pc1))).getOrElse(graph1) (d.privateChannels + (e.shortChannelId -> pc1), graph2) case None => (d.privateChannels, graph1) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/LightningMessageTypes.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/LightningMessageTypes.scala index 6510657c4..c91e65964 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/LightningMessageTypes.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/LightningMessageTypes.scala @@ -22,6 +22,7 @@ import fr.acinq.bitcoin.scalacompat.Crypto.{PrivateKey, PublicKey} import fr.acinq.bitcoin.scalacompat.{ByteVector32, ByteVector64, Satoshi, ScriptWitness, Transaction} import fr.acinq.eclair.blockchain.fee.FeeratePerKw import fr.acinq.eclair.channel.{ChannelFlags, ChannelType} +import fr.acinq.eclair.payment.relay.Relayer import fr.acinq.eclair.{BlockHeight, CltvExpiry, CltvExpiryDelta, Feature, Features, InitFeature, MilliSatoshi, ShortChannelId, TimestampSecond, UInt64} import scodec.bits.ByteVector @@ -374,6 +375,8 @@ case class ChannelUpdate(signature: ByteVector64, def messageFlags: Byte = if (htlcMaximumMsat.isDefined) 1 else 0 def toStringShort: String = s"cltvExpiryDelta=$cltvExpiryDelta,feeBase=$feeBaseMsat,feeProportionalMillionths=$feeProportionalMillionths" + + def relayFees: Relayer.RelayFees = Relayer.RelayFees(feeBase = feeBaseMsat, feeProportionalMillionths = feeProportionalMillionths) } object ChannelUpdate { diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/FuzzySpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/FuzzySpec.scala index fa4daca90..2ca68f370 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/FuzzySpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/FuzzySpec.scala @@ -123,7 +123,7 @@ class FuzzySpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with Channe // allow overpaying (no more than 2 times the required amount) val amount = requiredAmount + Random.nextInt(requiredAmount.toLong.toInt).msat val expiry = (Channel.MIN_CLTV_EXPIRY_DELTA + 1).toCltvExpiry(currentBlockHeight = BlockHeight(400000)) - OutgoingPaymentPacket.buildCommand(self, Upstream.Local(UUID.randomUUID()), paymentHash, ChannelHop(null, dest, null) :: Nil, PaymentOnion.createSinglePartPayload(amount, expiry, paymentSecret, None)).get._1 + OutgoingPaymentPacket.buildCommand(self, Upstream.Local(UUID.randomUUID()), paymentHash, ChannelHop(null, null, dest, null) :: Nil, PaymentOnion.createSinglePartPayload(amount, expiry, paymentSecret, None)).get._1 } def initiatePaymentOrStop(remaining: Int): Unit = diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala index 31444f0ca..fd1703295 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala @@ -250,7 +250,7 @@ trait ChannelStateTestsHelperMethods extends TestKitBase { def makeCmdAdd(amount: MilliSatoshi, cltvExpiryDelta: CltvExpiryDelta, destination: PublicKey, paymentPreimage: ByteVector32, currentBlockHeight: BlockHeight, upstream: Upstream, replyTo: ActorRef = TestProbe().ref): (ByteVector32, CMD_ADD_HTLC) = { val paymentHash: ByteVector32 = Crypto.sha256(paymentPreimage) val expiry = cltvExpiryDelta.toCltvExpiry(currentBlockHeight) - val cmd = OutgoingPaymentPacket.buildCommand(replyTo, upstream, paymentHash, ChannelHop(null, destination, null) :: Nil, PaymentOnion.createSinglePartPayload(amount, expiry, randomBytes32(), None)).get._1.copy(commit = false) + val cmd = OutgoingPaymentPacket.buildCommand(replyTo, upstream, paymentHash, ChannelHop(null, null, destination, null) :: Nil, PaymentOnion.createSinglePartPayload(amount, expiry, randomBytes32(), None)).get._1.copy(commit = false) (paymentPreimage, cmd) } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/f/ShutdownStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/f/ShutdownStateSpec.scala index 5843eb05c..1c7afd267 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/f/ShutdownStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/f/ShutdownStateSpec.scala @@ -60,7 +60,7 @@ class ShutdownStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wit val h1 = Crypto.sha256(r1) val amount1 = 300000000 msat val expiry1 = CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight) - val cmd1 = OutgoingPaymentPacket.buildCommand(sender.ref, Upstream.Local(UUID.randomUUID), h1, ChannelHop(null, TestConstants.Bob.nodeParams.nodeId, null) :: Nil, PaymentOnion.createSinglePartPayload(amount1, expiry1, randomBytes32(), None)).get._1.copy(commit = false) + val cmd1 = OutgoingPaymentPacket.buildCommand(sender.ref, Upstream.Local(UUID.randomUUID), h1, ChannelHop(null, null, TestConstants.Bob.nodeParams.nodeId, null) :: Nil, PaymentOnion.createSinglePartPayload(amount1, expiry1, randomBytes32(), None)).get._1.copy(commit = false) alice ! cmd1 sender.expectMsgType[RES_SUCCESS[CMD_ADD_HTLC]] val htlc1 = alice2bob.expectMsgType[UpdateAddHtlc] @@ -70,7 +70,7 @@ class ShutdownStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wit val h2 = Crypto.sha256(r2) val amount2 = 200000000 msat val expiry2 = CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight) - val cmd2 = OutgoingPaymentPacket.buildCommand(sender.ref, Upstream.Local(UUID.randomUUID), h2, ChannelHop(null, TestConstants.Bob.nodeParams.nodeId, null) :: Nil, PaymentOnion.createSinglePartPayload(amount2, expiry2, randomBytes32(), None)).get._1.copy(commit = false) + val cmd2 = OutgoingPaymentPacket.buildCommand(sender.ref, Upstream.Local(UUID.randomUUID), h2, ChannelHop(null, null, TestConstants.Bob.nodeParams.nodeId, null) :: Nil, PaymentOnion.createSinglePartPayload(amount2, expiry2, randomBytes32(), None)).get._1.copy(commit = false) alice ! cmd2 sender.expectMsgType[RES_SUCCESS[CMD_ADD_HTLC]] val htlc2 = alice2bob.expectMsgType[UpdateAddHtlc] diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/db/PaymentsDbSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/db/PaymentsDbSpec.scala index 8294d15fe..7e3a93b85 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/db/PaymentsDbSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/db/PaymentsDbSpec.scala @@ -24,7 +24,7 @@ import fr.acinq.eclair.db.jdbc.JdbcUtils.{setVersion, using} import fr.acinq.eclair.db.pg.PgPaymentsDb import fr.acinq.eclair.db.sqlite.SqlitePaymentsDb import fr.acinq.eclair.payment._ -import fr.acinq.eclair.router.Router.{ChannelHop, NodeHop} +import fr.acinq.eclair.router.Router.{ChannelHop, ChannelRelayParams, NodeHop} import fr.acinq.eclair.wire.protocol.{ChannelUpdate, UnknownNextPeer} import fr.acinq.eclair.{CltvExpiryDelta, MilliSatoshiLong, ShortChannelId, TestDatabases, TimestampMilli, TimestampMilliLong, TimestampSecond, TimestampSecondLong, randomBytes32, randomBytes64, randomKey} import org.scalatest.funsuite.AnyFunSuite @@ -593,7 +593,7 @@ class PaymentsDbSpec extends AnyFunSuite { object PaymentsDbSpec { val (alicePriv, bobPriv, carolPriv, davePriv) = (randomKey(), randomKey(), randomKey(), randomKey()) val (alice, bob, carol, dave) = (alicePriv.publicKey, bobPriv.publicKey, carolPriv.publicKey, davePriv.publicKey) - val hop_ab = ChannelHop(alice, bob, ChannelUpdate(randomBytes64(), randomBytes32(), ShortChannelId(42), 1 unixsec, ChannelUpdate.ChannelFlags.DUMMY, CltvExpiryDelta(12), 1 msat, 1 msat, 1, None)) + val hop_ab = ChannelHop(ShortChannelId(42), alice, bob, ChannelRelayParams.FromAnnouncement(ChannelUpdate(randomBytes64(), randomBytes32(), ShortChannelId(42), 1 unixsec, ChannelUpdate.ChannelFlags.DUMMY, CltvExpiryDelta(12), 1 msat, 1 msat, 1, None))) val hop_bc = NodeHop(bob, carol, CltvExpiryDelta(14), 1 msat) val (preimage1, preimage2, preimage3, preimage4) = (randomBytes32(), randomBytes32(), randomBytes32(), randomBytes32()) val (paymentHash1, paymentHash2, paymentHash3, paymentHash4) = (Crypto.sha256(preimage1), Crypto.sha256(preimage2), Crypto.sha256(preimage3), Crypto.sha256(preimage4)) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartPaymentLifecycleSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartPaymentLifecycleSpec.scala index 75f7dea40..634f7e42a 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartPaymentLifecycleSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartPaymentLifecycleSpec.scala @@ -31,6 +31,7 @@ import fr.acinq.eclair.payment.send.PaymentError.RetryExhausted import fr.acinq.eclair.payment.send.PaymentInitiator.SendPaymentConfig import fr.acinq.eclair.payment.send.PaymentLifecycle.SendPaymentToRoute import fr.acinq.eclair.payment.send.{MultiPartPaymentLifecycle, PaymentInitiator} +import fr.acinq.eclair.router.BaseRouterSpec.channelHopFromUpdate import fr.acinq.eclair.router.Graph.WeightRatios import fr.acinq.eclair.router.Router._ import fr.acinq.eclair.router.{Announcements, RouteNotFound} @@ -317,7 +318,7 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS import f._ // The B -> E channel is private and provided in the invoice routing hints. - val routingHint = ExtraHop(b, hop_be.lastUpdate.shortChannelId, hop_be.lastUpdate.feeBaseMsat, hop_be.lastUpdate.feeProportionalMillionths, hop_be.lastUpdate.cltvExpiryDelta) + val routingHint = ExtraHop(b, hop_be.shortChannelId, hop_be.params.relayFees.feeBase, hop_be.params.relayFees.feeProportionalMillionths, hop_be.params.cltvExpiryDelta) val payment = SendMultiPartPayment(sender.ref, randomBytes32(), e, finalAmount, expiry, 3, None, routeParams = routeParams, assistedRoutes = List(List(routingHint))) sender.send(payFsm, payment) assert(router.expectMsgType[RouteRequest].assistedRoutes.head.head === routingHint) @@ -327,7 +328,7 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS childPayFsm.expectNoMessage(100 millis) // B changed his fees and expiry after the invoice was issued. - val channelUpdate = hop_be.lastUpdate.copy(feeBaseMsat = 250 msat, feeProportionalMillionths = 150, cltvExpiryDelta = CltvExpiryDelta(24)) + val channelUpdate = hop_be.params.asInstanceOf[ChannelRelayParams.FromAnnouncement].channelUpdate.copy(feeBaseMsat = 250 msat, feeProportionalMillionths = 150, cltvExpiryDelta = CltvExpiryDelta(24)) val childId = payFsm.stateData.asInstanceOf[PaymentProgress].pending.keys.head childPayFsm.send(payFsm, PaymentFailed(childId, paymentHash, Seq(RemoteFailure(route.amount, route.hops, Sphinx.DecryptedFailurePacket(b, FeeInsufficient(finalAmount, channelUpdate)))))) // We update the routing hints accordingly before requesting a new route. @@ -338,7 +339,7 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS import f._ // The B -> E channel is private and provided in the invoice routing hints. - val routingHint = ExtraHop(b, hop_be.lastUpdate.shortChannelId, hop_be.lastUpdate.feeBaseMsat, hop_be.lastUpdate.feeProportionalMillionths, hop_be.lastUpdate.cltvExpiryDelta) + val routingHint = ExtraHop(b, hop_be.shortChannelId, hop_be.params.relayFees.feeBase, hop_be.params.relayFees.feeProportionalMillionths, hop_be.params.cltvExpiryDelta) val payment = SendMultiPartPayment(sender.ref, randomBytes32(), e, finalAmount, expiry, 3, None, routeParams = routeParams, assistedRoutes = List(List(routingHint))) sender.send(payFsm, payment) assert(router.expectMsgType[RouteRequest].assistedRoutes.head.head === routingHint) @@ -349,13 +350,14 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS // B doesn't have enough liquidity on this channel. // NB: we need a channel update with a valid signature, otherwise we'll ignore the node instead of this specific channel. - val channelUpdate = Announcements.makeChannelUpdate(hop_be.lastUpdate.chainHash, priv_b, e, hop_be.lastUpdate.shortChannelId, hop_be.lastUpdate.cltvExpiryDelta, hop_be.lastUpdate.htlcMinimumMsat, hop_be.lastUpdate.feeBaseMsat, hop_be.lastUpdate.feeProportionalMillionths, hop_be.lastUpdate.htlcMaximumMsat.get) + val channelUpdateBE = hop_be.params.asInstanceOf[ChannelRelayParams.FromAnnouncement].channelUpdate + val channelUpdateBE1 = Announcements.makeChannelUpdate(channelUpdateBE.chainHash, priv_b, e, channelUpdateBE.shortChannelId, channelUpdateBE.cltvExpiryDelta, channelUpdateBE.htlcMinimumMsat, channelUpdateBE.feeBaseMsat, channelUpdateBE.feeProportionalMillionths, channelUpdateBE.htlcMaximumMsat.get) val childId = payFsm.stateData.asInstanceOf[PaymentProgress].pending.keys.head - childPayFsm.send(payFsm, PaymentFailed(childId, paymentHash, Seq(RemoteFailure(route.amount, route.hops, Sphinx.DecryptedFailurePacket(b, TemporaryChannelFailure(channelUpdate)))))) + childPayFsm.send(payFsm, PaymentFailed(childId, paymentHash, Seq(RemoteFailure(route.amount, route.hops, Sphinx.DecryptedFailurePacket(b, TemporaryChannelFailure(channelUpdateBE1)))))) // We update the routing hints accordingly before requesting a new route and ignore the channel. val routeRequest = router.expectMsgType[RouteRequest] assert(routeRequest.assistedRoutes.head.head === routingHint) - assert(routeRequest.ignore.channels.map(_.shortChannelId) === Set(channelUpdate.shortChannelId)) + assert(routeRequest.ignore.channels.map(_.shortChannelId) === Set(channelUpdateBE1.shortChannelId)) } test("update routing hints") { _ => @@ -698,13 +700,13 @@ object MultiPartPaymentLifecycleSpec { val channelUpdate_ad = defaultChannelUpdate.copy(shortChannelId = channelId_ad) val channelUpdate_de = defaultChannelUpdate.copy(shortChannelId = channelId_de) - val hop_ab_1 = ChannelHop(a, b, channelUpdate_ab_1) - val hop_ab_2 = ChannelHop(a, b, channelUpdate_ab_2) - val hop_be = ChannelHop(b, e, channelUpdate_be) - val hop_ac_1 = ChannelHop(a, c, channelUpdate_ac_1) - val hop_ac_2 = ChannelHop(a, c, channelUpdate_ac_2) - val hop_ce = ChannelHop(c, e, channelUpdate_ce) - val hop_ad = ChannelHop(a, d, channelUpdate_ad) - val hop_de = ChannelHop(d, e, channelUpdate_de) + val hop_ab_1 = channelHopFromUpdate(a, b, channelUpdate_ab_1) + val hop_ab_2 = channelHopFromUpdate(a, b, channelUpdate_ab_2) + val hop_be = channelHopFromUpdate(b, e, channelUpdate_be) + val hop_ac_1 = channelHopFromUpdate(a, c, channelUpdate_ac_1) + val hop_ac_2 = channelHopFromUpdate(a, c, channelUpdate_ac_2) + val hop_ce = channelHopFromUpdate(c, e, channelUpdate_ce) + val hop_ad = channelHopFromUpdate(a, d, channelUpdate_ad) + val hop_de = channelHopFromUpdate(d, e, channelUpdate_de) } \ No newline at end of file diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentLifecycleSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentLifecycleSpec.scala index 1ae69f923..144368adb 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentLifecycleSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentLifecycleSpec.scala @@ -38,7 +38,7 @@ import fr.acinq.eclair.payment.send.PaymentInitiator.SendPaymentConfig import fr.acinq.eclair.payment.send.PaymentLifecycle import fr.acinq.eclair.payment.send.PaymentLifecycle._ import fr.acinq.eclair.router.Announcements.makeChannelUpdate -import fr.acinq.eclair.router.BaseRouterSpec.channelAnnouncement +import fr.acinq.eclair.router.BaseRouterSpec.{channelAnnouncement, channelHopFromUpdate} import fr.acinq.eclair.router.Graph.WeightRatios import fr.acinq.eclair.router.Router._ import fr.acinq.eclair.router._ @@ -103,7 +103,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec { import cfg._ // pre-computed route going from A to D - val route = Route(defaultAmountMsat, ChannelHop(a, b, update_ab) :: ChannelHop(b, c, update_bc) :: ChannelHop(c, d, update_cd) :: Nil) + val route = Route(defaultAmountMsat, channelHopFromUpdate(a, b, update_ab) :: channelHopFromUpdate(b, c, update_bc) :: channelHopFromUpdate(c, d, update_cd) :: Nil) val request = SendPaymentToRoute(sender.ref, Right(route), PaymentOnion.createSinglePartPayload(defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret.get, defaultInvoice.paymentMetadata)) sender.send(paymentFSM, request) routerForwarder.expectNoMessage(100 millis) // we don't need the router, we have the pre-computed route @@ -693,7 +693,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec { val chan_bh = channelAnnouncement(channelId_bh, priv_b, priv_h, priv_funding_b, priv_funding_h) val channelUpdate_bh = makeChannelUpdate(Block.RegtestGenesisBlock.hash, priv_b, h, channelId_bh, CltvExpiryDelta(9), htlcMinimumMsat = 0 msat, feeBaseMsat = 0 msat, feeProportionalMillionths = 0, htlcMaximumMsat = 500000000 msat) val channelUpdate_hb = makeChannelUpdate(Block.RegtestGenesisBlock.hash, priv_h, b, channelId_bh, CltvExpiryDelta(9), htlcMinimumMsat = 0 msat, feeBaseMsat = 10 msat, feeProportionalMillionths = 8, htlcMaximumMsat = 500000000 msat) - assert(Router.getDesc(channelUpdate_bh, chan_bh) === ChannelDesc(channelId_bh, priv_b.publicKey, priv_h.publicKey)) + assert(ChannelDesc(channelUpdate_bh, chan_bh) === ChannelDesc(channelId_bh, priv_b.publicKey, priv_h.publicKey)) val peerConnection = TestProbe() router ! PeerRoutingMessage(peerConnection.ref, remoteNodeId, chan_bh) router ! PeerRoutingMessage(peerConnection.ref, remoteNodeId, channelUpdate_bh) @@ -736,21 +736,21 @@ class PaymentLifecycleSpec extends BaseRouterSpec { test("filter errors properly") { _ => val failures = Seq( LocalFailure(defaultAmountMsat, Nil, RouteNotFound), - RemoteFailure(defaultAmountMsat, ChannelHop(a, b, update_ab) :: Nil, Sphinx.DecryptedFailurePacket(a, TemporaryNodeFailure)), - LocalFailure(defaultAmountMsat, ChannelHop(a, b, update_ab) :: Nil, ChannelUnavailable(ByteVector32.Zeroes)), + RemoteFailure(defaultAmountMsat, channelHopFromUpdate(a, b, update_ab) :: Nil, Sphinx.DecryptedFailurePacket(a, TemporaryNodeFailure)), + LocalFailure(defaultAmountMsat, channelHopFromUpdate(a, b, update_ab) :: Nil, ChannelUnavailable(ByteVector32.Zeroes)), LocalFailure(defaultAmountMsat, Nil, RouteNotFound) ) val filtered = PaymentFailure.transformForUser(failures) val expected = Seq( LocalFailure(defaultAmountMsat, Nil, RouteNotFound), - RemoteFailure(defaultAmountMsat, ChannelHop(a, b, update_ab) :: Nil, Sphinx.DecryptedFailurePacket(a, TemporaryNodeFailure)), - LocalFailure(defaultAmountMsat, ChannelHop(a, b, update_ab) :: Nil, ChannelUnavailable(ByteVector32.Zeroes)) + RemoteFailure(defaultAmountMsat, channelHopFromUpdate(a, b, update_ab) :: Nil, Sphinx.DecryptedFailurePacket(a, TemporaryNodeFailure)), + LocalFailure(defaultAmountMsat, channelHopFromUpdate(a, b, update_ab) :: Nil, ChannelUnavailable(ByteVector32.Zeroes)) ) assert(filtered === expected) } test("ignore failed nodes/channels") { _ => - val route_abcd = ChannelHop(a, b, update_ab) :: ChannelHop(b, c, update_bc) :: ChannelHop(c, d, update_cd) :: Nil + val route_abcd = channelHopFromUpdate(a, b, update_ab) :: channelHopFromUpdate(b, c, update_bc) :: channelHopFromUpdate(c, d, update_cd) :: Nil val testCases = Seq( // local failures -> ignore first channel if there is one (LocalFailure(defaultAmountMsat, Nil, RouteNotFound), Set.empty, Set.empty), @@ -765,8 +765,8 @@ class PaymentLifecycleSpec extends BaseRouterSpec { (RemoteFailure(defaultAmountMsat, route_abcd, Sphinx.DecryptedFailurePacket(c, UnknownNextPeer)), Set.empty, Set(ChannelDesc(channelId_cd, c, d))), (RemoteFailure(defaultAmountMsat, route_abcd, Sphinx.DecryptedFailurePacket(b, FeeInsufficient(100 msat, update_bc))), Set.empty, Set.empty), // unreadable remote failures -> blacklist all nodes except our direct peer and the final recipient - (UnreadableRemoteFailure(defaultAmountMsat, ChannelHop(a, b, update_ab) :: Nil), Set.empty, Set.empty), - (UnreadableRemoteFailure(defaultAmountMsat, ChannelHop(a, b, update_ab) :: ChannelHop(b, c, update_bc) :: ChannelHop(c, d, update_cd) :: ChannelHop(d, e, null) :: Nil), Set(c, d), Set.empty) + (UnreadableRemoteFailure(defaultAmountMsat, channelHopFromUpdate(a, b, update_ab) :: Nil), Set.empty, Set.empty), + (UnreadableRemoteFailure(defaultAmountMsat, channelHopFromUpdate(a, b, update_ab) :: channelHopFromUpdate(b, c, update_bc) :: channelHopFromUpdate(c, d, update_cd) :: ChannelHop(ShortChannelId(5656986L), d, e, null) :: Nil), Set(c, d), Set.empty) ) for ((failure, expectedNodes, expectedChannels) <- testCases) { @@ -810,7 +810,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec { import cfg._ // pre-computed route going from A to D - val route = Route(defaultAmountMsat, ChannelHop(a, b, update_ab) :: ChannelHop(b, c, update_bc) :: ChannelHop(c, d, update_cd) :: Nil) + val route = Route(defaultAmountMsat, channelHopFromUpdate(a, b, update_ab) :: channelHopFromUpdate(b, c, update_bc) :: channelHopFromUpdate(c, d, update_cd) :: Nil) val request = SendPaymentToRoute(sender.ref, Right(route), PaymentOnion.createSinglePartPayload(defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret.get, defaultInvoice.paymentMetadata)) sender.send(paymentFSM, request) routerForwarder.expectNoMessage(100 millis) // we don't need the router, we have the pre-computed route diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentPacketSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentPacketSpec.scala index 41cf05855..2e041a6d8 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentPacketSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentPacketSpec.scala @@ -27,6 +27,7 @@ import fr.acinq.eclair.channel.fsm.Channel import fr.acinq.eclair.crypto.Sphinx import fr.acinq.eclair.payment.IncomingPaymentPacket.{ChannelRelayPacket, FinalPacket, NodeRelayPacket, decrypt} import fr.acinq.eclair.payment.OutgoingPaymentPacket._ +import fr.acinq.eclair.router.BaseRouterSpec.channelHopFromUpdate import fr.acinq.eclair.router.Router.{ChannelHop, NodeHop} import fr.acinq.eclair.transactions.Transactions.InputInfo import fr.acinq.eclair.wire.protocol.OnionPaymentPayloadTlv.{AmountToForward, OutgoingCltv, PaymentData} @@ -179,7 +180,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll { assert(inner_c.paymentMetadata === None) // c forwards the trampoline payment to d. - val Success((amount_d, expiry_d, onion_d)) = buildPaymentPacket(paymentHash, ChannelHop(c, d, channelUpdate_cd) :: Nil, PaymentOnion.createTrampolinePayload(amount_cd, amount_cd, expiry_cd, randomBytes32(), packet_d)) + val Success((amount_d, expiry_d, onion_d)) = buildPaymentPacket(paymentHash, channelHopFromUpdate(c, d, channelUpdate_cd) :: Nil, PaymentOnion.createTrampolinePayload(amount_cd, amount_cd, expiry_cd, randomBytes32(), packet_d)) assert(amount_d === amount_cd) assert(expiry_d === expiry_cd) val add_d = UpdateAddHtlc(randomBytes32(), 3, amount_d, paymentHash, expiry_d, onion_d.packet) @@ -197,7 +198,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll { assert(inner_d.paymentMetadata === None) // d forwards the trampoline payment to e. - val Success((amount_e, expiry_e, onion_e)) = buildPaymentPacket(paymentHash, ChannelHop(d, e, channelUpdate_de) :: Nil, PaymentOnion.createTrampolinePayload(amount_de, amount_de, expiry_de, randomBytes32(), packet_e)) + val Success((amount_e, expiry_e, onion_e)) = buildPaymentPacket(paymentHash, channelHopFromUpdate(d, e, channelUpdate_de) :: Nil, PaymentOnion.createTrampolinePayload(amount_de, amount_de, expiry_de, randomBytes32(), packet_e)) assert(amount_e === amount_de) assert(expiry_e === expiry_de) val add_e = UpdateAddHtlc(randomBytes32(), 4, amount_e, paymentHash, expiry_e, onion_e.packet) @@ -240,7 +241,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll { assert(inner_c.paymentSecret === None) // c forwards the trampoline payment to d. - val Success((amount_d, expiry_d, onion_d)) = buildPaymentPacket(paymentHash, ChannelHop(c, d, channelUpdate_cd) :: Nil, PaymentOnion.createTrampolinePayload(amount_cd, amount_cd, expiry_cd, randomBytes32(), packet_d)) + val Success((amount_d, expiry_d, onion_d)) = buildPaymentPacket(paymentHash, channelHopFromUpdate(c, d, channelUpdate_cd) :: Nil, PaymentOnion.createTrampolinePayload(amount_cd, amount_cd, expiry_cd, randomBytes32(), packet_d)) assert(amount_d === amount_cd) assert(expiry_d === expiry_cd) val add_d = UpdateAddHtlc(randomBytes32(), 3, amount_d, paymentHash, expiry_d, onion_d.packet) @@ -309,11 +310,11 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll { val Right(ChannelRelayPacket(_, _, packet_c)) = decrypt(UpdateAddHtlc(randomBytes32(), 1, firstAmount, paymentHash, firstExpiry, onion.packet), priv_b.privateKey) val Right(NodeRelayPacket(_, _, _, packet_d)) = decrypt(UpdateAddHtlc(randomBytes32(), 2, amount_bc, paymentHash, expiry_bc, packet_c), priv_c.privateKey) // c forwards the trampoline payment to d. - val Success((amount_d, expiry_d, onion_d)) = buildPaymentPacket(paymentHash, ChannelHop(c, d, channelUpdate_cd) :: Nil, PaymentOnion.createTrampolinePayload(amount_cd, amount_cd, expiry_cd, randomBytes32(), packet_d)) + val Success((amount_d, expiry_d, onion_d)) = buildPaymentPacket(paymentHash, channelHopFromUpdate(c, d, channelUpdate_cd) :: Nil, PaymentOnion.createTrampolinePayload(amount_cd, amount_cd, expiry_cd, randomBytes32(), packet_d)) val Right(NodeRelayPacket(_, _, _, packet_e)) = decrypt(UpdateAddHtlc(randomBytes32(), 3, amount_d, paymentHash, expiry_d, onion_d.packet), priv_d.privateKey) // d forwards an invalid amount to e (the outer total amount doesn't match the inner amount). val invalidTotalAmount = amount_de + 100.msat - val Success((amount_e, expiry_e, onion_e)) = buildPaymentPacket(paymentHash, ChannelHop(d, e, channelUpdate_de) :: Nil, PaymentOnion.createTrampolinePayload(amount_de, invalidTotalAmount, expiry_de, randomBytes32(), packet_e)) + val Success((amount_e, expiry_e, onion_e)) = buildPaymentPacket(paymentHash, channelHopFromUpdate(d, e, channelUpdate_de) :: Nil, PaymentOnion.createTrampolinePayload(amount_de, invalidTotalAmount, expiry_de, randomBytes32(), packet_e)) val Left(failure) = decrypt(UpdateAddHtlc(randomBytes32(), 4, amount_e, paymentHash, expiry_e, onion_e.packet), priv_e.privateKey) assert(failure === FinalIncorrectHtlcAmount(invalidTotalAmount)) } @@ -324,11 +325,11 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll { val Right(ChannelRelayPacket(_, _, packet_c)) = decrypt(UpdateAddHtlc(randomBytes32(), 1, firstAmount, paymentHash, firstExpiry, onion.packet), priv_b.privateKey) val Right(NodeRelayPacket(_, _, _, packet_d)) = decrypt(UpdateAddHtlc(randomBytes32(), 2, amount_bc, paymentHash, expiry_bc, packet_c), priv_c.privateKey) // c forwards the trampoline payment to d. - val Success((amount_d, expiry_d, onion_d)) = buildPaymentPacket(paymentHash, ChannelHop(c, d, channelUpdate_cd) :: Nil, PaymentOnion.createTrampolinePayload(amount_cd, amount_cd, expiry_cd, randomBytes32(), packet_d)) + val Success((amount_d, expiry_d, onion_d)) = buildPaymentPacket(paymentHash, channelHopFromUpdate(c, d, channelUpdate_cd) :: Nil, PaymentOnion.createTrampolinePayload(amount_cd, amount_cd, expiry_cd, randomBytes32(), packet_d)) val Right(NodeRelayPacket(_, _, _, packet_e)) = decrypt(UpdateAddHtlc(randomBytes32(), 3, amount_d, paymentHash, expiry_d, onion_d.packet), priv_d.privateKey) // d forwards an invalid expiry to e (the outer expiry doesn't match the inner expiry). val invalidExpiry = expiry_de - CltvExpiryDelta(12) - val Success((amount_e, expiry_e, onion_e)) = buildPaymentPacket(paymentHash, ChannelHop(d, e, channelUpdate_de) :: Nil, PaymentOnion.createTrampolinePayload(amount_de, amount_de, invalidExpiry, randomBytes32(), packet_e)) + val Success((amount_e, expiry_e, onion_e)) = buildPaymentPacket(paymentHash, channelHopFromUpdate(d, e, channelUpdate_de) :: Nil, PaymentOnion.createTrampolinePayload(amount_de, amount_de, invalidExpiry, randomBytes32(), packet_e)) val Left(failure) = decrypt(UpdateAddHtlc(randomBytes32(), 4, amount_e, paymentHash, expiry_e, onion_e.packet), priv_e.privateKey) assert(failure === FinalIncorrectCltvExpiry(invalidExpiry)) } @@ -392,10 +393,10 @@ object PaymentPacketSpec { // simple route a -> b -> c -> d -> e val hops = - ChannelHop(a, b, channelUpdate_ab) :: - ChannelHop(b, c, channelUpdate_bc) :: - ChannelHop(c, d, channelUpdate_cd) :: - ChannelHop(d, e, channelUpdate_de) :: Nil + channelHopFromUpdate(a, b, channelUpdate_ab) :: + channelHopFromUpdate(b, c, channelUpdate_bc) :: + channelHopFromUpdate(c, d, channelUpdate_cd) :: + channelHopFromUpdate(d, e, channelUpdate_de) :: Nil val finalAmount = 42000000 msat val currentBlockCount = 400000 @@ -407,15 +408,15 @@ object PaymentPacketSpec { val expiry_de = finalExpiry val amount_de = finalAmount - val fee_d = nodeFee(channelUpdate_de, amount_de) + val fee_d = nodeFee(channelUpdate_de.relayFees, amount_de) val expiry_cd = expiry_de + channelUpdate_de.cltvExpiryDelta val amount_cd = amount_de + fee_d - val fee_c = nodeFee(channelUpdate_cd, amount_cd) + val fee_c = nodeFee(channelUpdate_cd.relayFees, amount_cd) val expiry_bc = expiry_cd + channelUpdate_cd.cltvExpiryDelta val amount_bc = amount_cd + fee_c - val fee_b = nodeFee(channelUpdate_bc, amount_bc) + val fee_b = nodeFee(channelUpdate_bc.relayFees, amount_bc) val expiry_ab = expiry_bc + channelUpdate_bc.cltvExpiryDelta val amount_ab = amount_bc + fee_b @@ -431,7 +432,7 @@ object PaymentPacketSpec { NodeHop(d, e, channelUpdate_de.cltvExpiryDelta, fee_d) :: Nil val trampolineChannelHops = - ChannelHop(a, b, channelUpdate_ab) :: - ChannelHop(b, c, channelUpdate_bc) :: Nil + channelHopFromUpdate(a, b, channelUpdate_ab) :: + channelHopFromUpdate(b, c, channelUpdate_bc) :: Nil } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/PostRestartHtlcCleanerSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PostRestartHtlcCleanerSpec.scala index 47d7cb023..611eadc84 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/PostRestartHtlcCleanerSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PostRestartHtlcCleanerSpec.scala @@ -30,6 +30,7 @@ import fr.acinq.eclair.db.{OutgoingPayment, OutgoingPaymentStatus, PaymentType} import fr.acinq.eclair.payment.OutgoingPaymentPacket.{Upstream, buildCommand} import fr.acinq.eclair.payment.PaymentPacketSpec._ import fr.acinq.eclair.payment.relay.{PostRestartHtlcCleaner, Relayer} +import fr.acinq.eclair.router.BaseRouterSpec.channelHopFromUpdate import fr.acinq.eclair.router.Router.ChannelHop import fr.acinq.eclair.transactions.Transactions.{ClaimRemoteDelayedOutputTx, InputInfo} import fr.acinq.eclair.transactions.{DirectedHtlc, IncomingHtlc, OutgoingHtlc} @@ -729,7 +730,7 @@ object PostRestartHtlcCleanerSpec { def buildHtlcOut(htlcId: Long, channelId: ByteVector32, paymentHash: ByteVector32): DirectedHtlc = OutgoingHtlc(buildHtlc(htlcId, channelId, paymentHash)) def buildFinalHtlc(htlcId: Long, channelId: ByteVector32, paymentHash: ByteVector32): DirectedHtlc = { - val Success((cmd, _)) = buildCommand(ActorRef.noSender, Upstream.Local(UUID.randomUUID()), paymentHash, ChannelHop(a, TestConstants.Bob.nodeParams.nodeId, channelUpdate_ab) :: Nil, PaymentOnion.createSinglePartPayload(finalAmount, finalExpiry, randomBytes32(), None)) + val Success((cmd, _)) = buildCommand(ActorRef.noSender, Upstream.Local(UUID.randomUUID()), paymentHash, channelHopFromUpdate(a, TestConstants.Bob.nodeParams.nodeId, channelUpdate_ab) :: Nil, PaymentOnion.createSinglePartPayload(finalAmount, finalExpiry, randomBytes32(), None)) IncomingHtlc(UpdateAddHtlc(channelId, htlcId, cmd.amount, cmd.paymentHash, cmd.cltvExpiry, cmd.onion)) } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/RelayerSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/RelayerSpec.scala index 66cb6852e..29ad5083f 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/RelayerSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/RelayerSpec.scala @@ -30,6 +30,7 @@ import fr.acinq.eclair.payment.OutgoingPaymentPacket.{Upstream, buildCommand} import fr.acinq.eclair.payment.PaymentPacketSpec._ import fr.acinq.eclair.payment.relay.Relayer._ import fr.acinq.eclair.payment.{OutgoingPaymentPacket, PaymentPacketSpec} +import fr.acinq.eclair.router.BaseRouterSpec.channelHopFromUpdate import fr.acinq.eclair.router.Router.{ChannelHop, NodeHop} import fr.acinq.eclair.wire.protocol._ import fr.acinq.eclair.{NodeParams, TestConstants, randomBytes32, _} @@ -118,7 +119,7 @@ class RelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat val Success((trampolineAmount, trampolineExpiry, trampolineOnion)) = OutgoingPaymentPacket.buildTrampolinePacket(paymentHash, trampolineHops, PaymentOnion.createMultiPartPayload(finalAmount, totalAmount, finalExpiry, paymentSecret, None)) assert(trampolineAmount === finalAmount) assert(trampolineExpiry === finalExpiry) - val Success((cmd, _)) = buildCommand(ActorRef.noSender, Upstream.Local(UUID.randomUUID()), paymentHash, ChannelHop(a, b, channelUpdate_ab) :: Nil, PaymentOnion.createTrampolinePayload(trampolineAmount, trampolineAmount, trampolineExpiry, randomBytes32(), trampolineOnion.packet)) + val Success((cmd, _)) = buildCommand(ActorRef.noSender, Upstream.Local(UUID.randomUUID()), paymentHash, channelHopFromUpdate(a, b, channelUpdate_ab) :: Nil, PaymentOnion.createTrampolinePayload(trampolineAmount, trampolineAmount, trampolineExpiry, randomBytes32(), trampolineOnion.packet)) assert(cmd.amount === finalAmount) assert(cmd.cltvExpiry === finalExpiry) val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amount, cmd.paymentHash, cmd.cltvExpiry, cmd.onion) @@ -161,7 +162,7 @@ class RelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat // we use this to build a valid trampoline onion inside a normal onion val trampolineHops = NodeHop(a, b, channelUpdate_ab.cltvExpiryDelta, 0 msat) :: NodeHop(b, c, channelUpdate_bc.cltvExpiryDelta, fee_b) :: Nil val Success((trampolineAmount, trampolineExpiry, trampolineOnion)) = OutgoingPaymentPacket.buildTrampolinePacket(paymentHash, trampolineHops, PaymentOnion.createSinglePartPayload(finalAmount, finalExpiry, paymentSecret, None)) - val Success((cmd, _)) = buildCommand(ActorRef.noSender, Upstream.Local(UUID.randomUUID()), paymentHash, ChannelHop(a, b, channelUpdate_ab) :: Nil, PaymentOnion.createTrampolinePayload(trampolineAmount, trampolineAmount, trampolineExpiry, randomBytes32(), trampolineOnion.packet)) + val Success((cmd, _)) = buildCommand(ActorRef.noSender, Upstream.Local(UUID.randomUUID()), paymentHash, channelHopFromUpdate(a, b, channelUpdate_ab) :: Nil, PaymentOnion.createTrampolinePayload(trampolineAmount, trampolineAmount, trampolineExpiry, randomBytes32(), trampolineOnion.packet)) // and then manually build an htlc val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amount, cmd.paymentHash, cmd.cltvExpiry, cmd.onion) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/router/BaseRouterSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/router/BaseRouterSpec.scala index 90b8afac4..a2ce74b18 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/router/BaseRouterSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/router/BaseRouterSpec.scala @@ -19,7 +19,7 @@ package fr.acinq.eclair.router import akka.actor.ActorRef import akka.actor.typed.scaladsl.adapter.actorRefAdapter import akka.testkit.TestProbe -import fr.acinq.bitcoin.scalacompat.Crypto.PrivateKey +import fr.acinq.bitcoin.scalacompat.Crypto.{PrivateKey, PublicKey} import fr.acinq.bitcoin.scalacompat.Script.{pay2wsh, write} import fr.acinq.bitcoin.scalacompat.{Block, ByteVector32, SatoshiLong, Transaction, TxOut} import fr.acinq.eclair.TestConstants.Alice @@ -106,13 +106,13 @@ abstract class BaseRouterSpec extends TestKitBaseClass with FixtureAnyFunSuiteLi // and e --(4)--> f (we are a) within(30 seconds) { // first we make sure that we correctly resolve channelId+direction to nodeId - assert(Router.getDesc(update_ab, chan_ab) === ChannelDesc(chan_ab.shortChannelId, a, b)) - assert(Router.getDesc(update_bc, chan_bc) === ChannelDesc(chan_bc.shortChannelId, b, c)) - assert(Router.getDesc(update_cd, chan_cd) === ChannelDesc(chan_cd.shortChannelId, c, d)) - assert(Router.getDesc(update_ef, chan_ef) === ChannelDesc(chan_ef.shortChannelId, e, f)) - assert(Router.getDesc(update_ag_private, PrivateChannel(a, g, None, None, ChannelMeta(1000 msat, 2000 msat))) === ChannelDesc(channelId_ag_private, a, g)) - assert(Router.getDesc(update_ag_private, PrivateChannel(g, a, None, None, ChannelMeta(2000 msat, 1000 msat))) === ChannelDesc(channelId_ag_private, a, g)) - assert(Router.getDesc(update_gh, chan_gh) === ChannelDesc(chan_gh.shortChannelId, g, h)) + assert(ChannelDesc(update_ab, chan_ab) === ChannelDesc(chan_ab.shortChannelId, a, b)) + assert(ChannelDesc(update_bc, chan_bc) === ChannelDesc(chan_bc.shortChannelId, b, c)) + assert(ChannelDesc(update_cd, chan_cd) === ChannelDesc(chan_cd.shortChannelId, c, d)) + assert(ChannelDesc(update_ef, chan_ef) === ChannelDesc(chan_ef.shortChannelId, e, f)) + assert(ChannelDesc(update_ag_private, PrivateChannel(a, g, None, None, ChannelMeta(1000 msat, 2000 msat))) === ChannelDesc(channelId_ag_private, a, g)) + assert(ChannelDesc(update_ag_private, PrivateChannel(g, a, None, None, ChannelMeta(2000 msat, 1000 msat))) === ChannelDesc(channelId_ag_private, a, g)) + assert(ChannelDesc(update_gh, chan_gh) === ChannelDesc(chan_gh.shortChannelId, g, h)) // let's set up the router val sender = TestProbe() @@ -221,4 +221,7 @@ object BaseRouterSpec { val funding2_sig = Announcements.signChannelAnnouncement(witness, funding2_priv) makeChannelAnnouncement(Block.RegtestGenesisBlock.hash, channelId, node1_priv.publicKey, node2_priv.publicKey, funding1_priv.publicKey, funding2_priv.publicKey, node1_sig, node2_sig, funding1_sig, funding2_sig) } + + def channelHopFromUpdate(nodeId: PublicKey, nextNodeId: PublicKey, channelUpdate: ChannelUpdate): ChannelHop = + ChannelHop(channelUpdate.shortChannelId, nodeId, nextNodeId, ChannelRelayParams.FromAnnouncement(channelUpdate)) } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/router/GraphSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/router/GraphSpec.scala index 375f14423..9cac18440 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/router/GraphSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/router/GraphSpec.scala @@ -193,7 +193,7 @@ class GraphSpec extends AnyFunSuite { assert(mutatedGraph2.edgesOf(a).size == 3) // A --> B , A --> B , A --> D assert(mutatedGraph2.getEdgesBetween(a, b).size === 2) - assert(mutatedGraph2.getEdge(edgeForTheSameChannel).get.update.feeBaseMsat === 30.msat) + assert(mutatedGraph2.getEdge(edgeForTheSameChannel).get.params.relayFees.feeBase === 30.msat) } test("remove a vertex with incoming edges and check those edges are removed too") { diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/router/RouteCalculationSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/router/RouteCalculationSpec.scala index 6ecb23ed5..43816d207 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/router/RouteCalculationSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/router/RouteCalculationSpec.scala @@ -21,6 +21,7 @@ import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey import fr.acinq.bitcoin.scalacompat.{Block, ByteVector32, ByteVector64, Satoshi, SatoshiLong} import fr.acinq.eclair.payment.Bolt11Invoice.ExtraHop import fr.acinq.eclair.payment.relay.Relayer.RelayFees +import fr.acinq.eclair.router.BaseRouterSpec.channelHopFromUpdate import fr.acinq.eclair.router.Graph.GraphStructure.DirectedGraph.graphEdgeToHop import fr.acinq.eclair.router.Graph.GraphStructure.{DirectedGraph, GraphEdge} import fr.acinq.eclair.router.Graph.{HeuristicsConstants, RichWeight, WeightRatios} @@ -427,19 +428,19 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution { val ued = ChannelUpdate(DUMMY_SIG, Block.RegtestGenesisBlock.hash, ShortChannelId(4L), 1 unixsec, ChannelUpdate.ChannelFlags(isNode1 = false, isEnabled = false), CltvExpiryDelta(1), 49 msat, 2507 msat, 147, None) val edges = Seq( - GraphEdge(ChannelDesc(ShortChannelId(1L), a, b), uab, DEFAULT_CAPACITY, None), - GraphEdge(ChannelDesc(ShortChannelId(1L), b, a), uba, DEFAULT_CAPACITY, None), - GraphEdge(ChannelDesc(ShortChannelId(2L), b, c), ubc, DEFAULT_CAPACITY, None), - GraphEdge(ChannelDesc(ShortChannelId(2L), c, b), ucb, DEFAULT_CAPACITY, None), - GraphEdge(ChannelDesc(ShortChannelId(3L), c, d), ucd, DEFAULT_CAPACITY, None), - GraphEdge(ChannelDesc(ShortChannelId(3L), d, c), udc, DEFAULT_CAPACITY, None), - GraphEdge(ChannelDesc(ShortChannelId(4L), d, e), ude, DEFAULT_CAPACITY, None), - GraphEdge(ChannelDesc(ShortChannelId(4L), e, d), ued, DEFAULT_CAPACITY, None) + GraphEdge(ChannelDesc(ShortChannelId(1L), a, b), ChannelRelayParams.FromAnnouncement(uab), DEFAULT_CAPACITY, None), + GraphEdge(ChannelDesc(ShortChannelId(1L), b, a), ChannelRelayParams.FromAnnouncement(uba), DEFAULT_CAPACITY, None), + GraphEdge(ChannelDesc(ShortChannelId(2L), b, c), ChannelRelayParams.FromAnnouncement(ubc), DEFAULT_CAPACITY, None), + GraphEdge(ChannelDesc(ShortChannelId(2L), c, b), ChannelRelayParams.FromAnnouncement(ucb), DEFAULT_CAPACITY, None), + GraphEdge(ChannelDesc(ShortChannelId(3L), c, d), ChannelRelayParams.FromAnnouncement(ucd), DEFAULT_CAPACITY, None), + GraphEdge(ChannelDesc(ShortChannelId(3L), d, c), ChannelRelayParams.FromAnnouncement(udc), DEFAULT_CAPACITY, None), + GraphEdge(ChannelDesc(ShortChannelId(4L), d, e), ChannelRelayParams.FromAnnouncement(ude), DEFAULT_CAPACITY, None), + GraphEdge(ChannelDesc(ShortChannelId(4L), e, d), ChannelRelayParams.FromAnnouncement(ued), DEFAULT_CAPACITY, None) ) val g = DirectedGraph(edges) val Success(route :: Nil) = findRoute(g, a, e, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = BlockHeight(400000)) - assert(route.hops === ChannelHop(a, b, uab) :: ChannelHop(b, c, ubc) :: ChannelHop(c, d, ucd) :: ChannelHop(d, e, ude) :: Nil) + assert(route.hops === channelHopFromUpdate(a, b, uab) :: channelHopFromUpdate(b, c, ubc) :: channelHopFromUpdate(c, d, ucd) :: channelHopFromUpdate(d, e, ude) :: Nil) } test("convert extra hops to assisted channels") { @@ -458,10 +459,10 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution { val amount = 90000 sat // below RoutingHeuristics.CAPACITY_CHANNEL_LOW val assistedChannels = toAssistedChannels(extraHops, e, amount.toMilliSatoshi) - assert(assistedChannels(extraHop4.shortChannelId) === AssistedChannel(extraHop4, e, 100050.sat.toMilliSatoshi)) - assert(assistedChannels(extraHop3.shortChannelId) === AssistedChannel(extraHop3, d, 100200.sat.toMilliSatoshi)) - assert(assistedChannels(extraHop2.shortChannelId) === AssistedChannel(extraHop2, c, 100400.sat.toMilliSatoshi)) - assert(assistedChannels(extraHop1.shortChannelId) === AssistedChannel(extraHop1, b, 101416.sat.toMilliSatoshi)) + assert(assistedChannels(extraHop4.shortChannelId) === AssistedChannel(e, ChannelRelayParams.FromHint(extraHop4, 100050.sat.toMilliSatoshi))) + assert(assistedChannels(extraHop3.shortChannelId) === AssistedChannel(d, ChannelRelayParams.FromHint(extraHop3, 100200.sat.toMilliSatoshi))) + assert(assistedChannels(extraHop2.shortChannelId) === AssistedChannel(c, ChannelRelayParams.FromHint(extraHop2, 100400.sat.toMilliSatoshi))) + assert(assistedChannels(extraHop1.shortChannelId) === AssistedChannel(b, ChannelRelayParams.FromHint(extraHop1, 101416.sat.toMilliSatoshi))) } test("blacklist routes") { @@ -526,12 +527,12 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution { val Success(route1 :: Nil) = findRoute(g, a, e, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = BlockHeight(400000)) assert(route2Ids(route1) === 1 :: 2 :: 3 :: 4 :: Nil) - assert(route1.hops(1).lastUpdate.feeBaseMsat === 10.msat) + assert(route1.hops(1).params.relayFees.feeBase === 10.msat) val extraGraphEdges = Set(makeEdge(2L, b, c, 5 msat, 5)) val Success(route2 :: Nil) = findRoute(g, a, e, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, numRoutes = 1, extraEdges = extraGraphEdges, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = BlockHeight(400000)) assert(route2Ids(route2) === 1 :: 2 :: 3 :: 4 :: Nil) - assert(route2.hops(1).lastUpdate.feeBaseMsat === 5.msat) + assert(route2.hops(1).params.relayFees.feeBase === 5.msat) } test("compute ignored channels") { @@ -565,7 +566,7 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution { ) val publicChannels = channels.map { case (shortChannelId, announcement) => - val update = edges.find(_.desc.shortChannelId == shortChannelId).get.update + val ChannelRelayParams.FromAnnouncement(update) = edges.find(_.desc.shortChannelId == shortChannelId).get.params val (update_1_opt, update_2_opt) = if (update.channelFlags.isNode1) (Some(update), None) else (None, Some(update)) val pc = PublicChannel(announcement, ByteVector32.Zeroes, Satoshi(1000), update_1_opt, update_2_opt, None) (shortChannelId, pc) @@ -1094,7 +1095,7 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution { val amount = 50000 msat // These pending HTLCs will have already been taken into account in the edge's `balance_opt` field: findMultiPartRoute // should ignore this information. - val pendingHtlcs = Seq(Route(10000 msat, ChannelHop(a, b, edge_ab_1.update) :: Nil), Route(5000 msat, ChannelHop(a, b, edge_ab_2.update) :: Nil)) + val pendingHtlcs = Seq(Route(10000 msat, graphEdgeToHop(edge_ab_1) :: Nil), Route(5000 msat, graphEdgeToHop(edge_ab_2) :: Nil)) val Success(routes) = findMultiPartRoute(g, a, b, amount, 1 msat, pendingHtlcs = pendingHtlcs, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = BlockHeight(400000)) assert(routes.forall(_.length == 1), routes) checkRouteAmounts(routes, amount, 0 msat) @@ -1566,7 +1567,7 @@ class RouteCalculationSpec extends AnyFunSuite with ParallelTestExecution { makeEdge(6L, d, e, 50 msat, 0, minHtlc = 100 msat, capacity = 25 sat), )) - val pendingHtlcs = Seq(Route(5000 msat, ChannelHop(a, b, edge_ab.update) :: ChannelHop(b, e, edge_be.update) :: Nil)) + val pendingHtlcs = Seq(Route(5000 msat, graphEdgeToHop(edge_ab) :: graphEdgeToHop(edge_be) :: Nil)) val Success(routes) = findMultiPartRoute(g, a, e, amount, maxFee, pendingHtlcs = pendingHtlcs, routeParams = DEFAULT_ROUTE_PARAMS, currentBlockHeight = BlockHeight(400000)) assert(routes.forall(_.length == 2), routes) checkRouteAmounts(routes, amount, maxFee) @@ -1961,7 +1962,7 @@ object RouteCalculationSpec { capacity: Satoshi = DEFAULT_CAPACITY, balance_opt: Option[MilliSatoshi] = None): GraphEdge = { val update = makeUpdateShort(ShortChannelId(shortChannelId), nodeId1, nodeId2, feeBase, feeProportionalMillionth, minHtlc, maxHtlc, cltvDelta) - GraphEdge(ChannelDesc(ShortChannelId(shortChannelId), nodeId1, nodeId2), update, capacity, balance_opt) + GraphEdge(ChannelDesc(ShortChannelId(shortChannelId), nodeId1, nodeId2), ChannelRelayParams.FromAnnouncement(update), capacity, balance_opt) } def makeUpdateShort(shortChannelId: ShortChannelId, nodeId1: PublicKey, nodeId2: PublicKey, feeBase: MilliSatoshi, feeProportionalMillionth: Int, minHtlc: MilliSatoshi = DEFAULT_AMOUNT_MSAT, maxHtlc: Option[MilliSatoshi] = None, cltvDelta: CltvExpiryDelta = CltvExpiryDelta(0), timestamp: TimestampSecond = 0 unixsec): ChannelUpdate = @@ -1978,19 +1979,19 @@ object RouteCalculationSpec { htlcMaximumMsat = maxHtlc ) - def hops2Ids(hops: Seq[ChannelHop]): Seq[Long] = hops.map(hop => hop.lastUpdate.shortChannelId.toLong) + def hops2Ids(hops: Seq[ChannelHop]): Seq[Long] = hops.map(hop => hop.shortChannelId.toLong) def route2Ids(route: Route): Seq[Long] = hops2Ids(route.hops) def routes2Ids(routes: Seq[Route]): Set[Seq[Long]] = routes.map(route2Ids).toSet - def route2Edges(route: Route): Seq[GraphEdge] = route.hops.map(hop => GraphEdge(ChannelDesc(hop.lastUpdate.shortChannelId, hop.nodeId, hop.nextNodeId), hop.lastUpdate, 0 sat, None)) + def route2Edges(route: Route): Seq[GraphEdge] = route.hops.map(hop => GraphEdge(ChannelDesc(hop.shortChannelId, hop.nodeId, hop.nextNodeId), hop.params, 0 sat, None)) def route2Nodes(route: Route): Seq[(PublicKey, PublicKey)] = route.hops.map(hop => (hop.nodeId, hop.nextNodeId)) def checkIgnoredChannels(routes: Seq[Route], shortChannelIds: Long*): Unit = { shortChannelIds.foreach(shortChannelId => routes.foreach(route => { - assert(route.hops.forall(_.lastUpdate.shortChannelId.toLong != shortChannelId), route) + assert(route.hops.forall(_.shortChannelId.toLong != shortChannelId), route) })) } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/router/RouterSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/router/RouterSpec.scala index a5fc1bb36..b67eb961a 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/router/RouterSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/router/RouterSpec.scala @@ -29,11 +29,12 @@ import fr.acinq.eclair.io.Peer.PeerRoutingMessage import fr.acinq.eclair.payment.Bolt11Invoice.ExtraHop import fr.acinq.eclair.router.Announcements.{makeChannelUpdate, makeNodeAnnouncement} import fr.acinq.eclair.router.BaseRouterSpec.channelAnnouncement +import fr.acinq.eclair.router.Graph.RoutingHeuristics import fr.acinq.eclair.router.RouteCalculationSpec.{DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE, DEFAULT_ROUTE_PARAMS} import fr.acinq.eclair.router.Router._ import fr.acinq.eclair.transactions.Scripts import fr.acinq.eclair.wire.protocol._ -import fr.acinq.eclair.{BlockHeight, CltvExpiryDelta, Features, MilliSatoshi, MilliSatoshiLong, ShortChannelId, TestConstants, TimestampSecond, randomKey} +import fr.acinq.eclair.{BlockHeight, CltvExpiryDelta, Features, MilliSatoshi, MilliSatoshiLong, ShortChannelId, TestConstants, TimestampSecond, nodeFee, randomKey} import scodec.bits._ import scala.concurrent.duration._ @@ -479,7 +480,7 @@ class RouterSpec extends BaseRouterSpec { // the route hasn't changed (nodes are the same) assert(response.routes.head.hops.map(_.nodeId) === preComputedRoute.nodes.dropRight(1)) assert(response.routes.head.hops.map(_.nextNodeId) === preComputedRoute.nodes.drop(1)) - assert(response.routes.head.hops.map(_.lastUpdate) === Seq(update_ab, update_bc, update_cd)) + assert(response.routes.head.hops.map(_.params) === Seq(ChannelRelayParams.FromAnnouncement(update_ab), ChannelRelayParams.FromAnnouncement(update_bc), ChannelRelayParams.FromAnnouncement(update_cd))) } test("given a pre-defined channels route add the proper channel updates") { fixture => @@ -493,7 +494,7 @@ class RouterSpec extends BaseRouterSpec { // the route hasn't changed (nodes are the same) assert(response.routes.head.hops.map(_.nodeId) === Seq(a, b, c)) assert(response.routes.head.hops.map(_.nextNodeId) === Seq(b, c, d)) - assert(response.routes.head.hops.map(_.lastUpdate) === Seq(update_ab, update_bc, update_cd)) + assert(response.routes.head.hops.map(_.params) === Seq(ChannelRelayParams.FromAnnouncement(update_ab), ChannelRelayParams.FromAnnouncement(update_bc), ChannelRelayParams.FromAnnouncement(update_cd))) } test("given a pre-defined private channels route add the proper channel updates") { fixture => @@ -506,7 +507,7 @@ class RouterSpec extends BaseRouterSpec { val response = sender.expectMsgType[RouteResponse] assert(response.routes.length === 1) val route = response.routes.head - assert(route.hops.map(_.lastUpdate) === Seq(update_ag_private)) + assert(route.hops.map(_.params) === Seq(ChannelRelayParams.FromAnnouncement(update_ag_private))) assert(route.hops.head.nodeId === a) assert(route.hops.head.nextNodeId === g) } @@ -518,7 +519,7 @@ class RouterSpec extends BaseRouterSpec { val route = response.routes.head assert(route.hops.map(_.nodeId) === Seq(a, g)) assert(route.hops.map(_.nextNodeId) === Seq(g, h)) - assert(route.hops.map(_.lastUpdate) === Seq(update_ag_private, update_gh)) + assert(route.hops.map(_.params) === Seq(ChannelRelayParams.FromAnnouncement(update_ag_private), ChannelRelayParams.FromAnnouncement(update_gh))) } } @@ -530,32 +531,32 @@ class RouterSpec extends BaseRouterSpec { { val invoiceRoutingHint = ExtraHop(b, ShortChannelId(BlockHeight(420000), 516, 1105), 10 msat, 150, CltvExpiryDelta(96)) val preComputedRoute = PredefinedChannelRoute(targetNodeId, Seq(channelId_ab, invoiceRoutingHint.shortChannelId)) - sender.send(router, FinalizeRoute(10000 msat, preComputedRoute, assistedRoutes = Seq(Seq(invoiceRoutingHint)))) + val amount = 10_000.msat + // the amount affects the way we estimate the channel capacity of the hinted channel + assert(amount < RoutingHeuristics.CAPACITY_CHANNEL_LOW) + sender.send(router, FinalizeRoute(amount, preComputedRoute, assistedRoutes = Seq(Seq(invoiceRoutingHint)))) val response = sender.expectMsgType[RouteResponse] assert(response.routes.length === 1) val route = response.routes.head assert(route.hops.map(_.nodeId) === Seq(a, b)) assert(route.hops.map(_.nextNodeId) === Seq(b, targetNodeId)) - assert(route.hops.head.lastUpdate === update_ab) - assert(route.hops.last.lastUpdate.shortChannelId === invoiceRoutingHint.shortChannelId) - assert(route.hops.last.lastUpdate.feeBaseMsat === invoiceRoutingHint.feeBase) - assert(route.hops.last.lastUpdate.feeProportionalMillionths === invoiceRoutingHint.feeProportionalMillionths) - assert(route.hops.last.lastUpdate.cltvExpiryDelta === invoiceRoutingHint.cltvExpiryDelta) + assert(route.hops.head.params === ChannelRelayParams.FromAnnouncement(update_ab)) + assert(route.hops.last.params === ChannelRelayParams.FromHint(invoiceRoutingHint, RoutingHeuristics.CAPACITY_CHANNEL_LOW + nodeFee(invoiceRoutingHint.feeBase, invoiceRoutingHint.feeProportionalMillionths, RoutingHeuristics.CAPACITY_CHANNEL_LOW))) } { val invoiceRoutingHint = ExtraHop(h, ShortChannelId(BlockHeight(420000), 516, 1105), 10 msat, 150, CltvExpiryDelta(96)) val preComputedRoute = PredefinedChannelRoute(targetNodeId, Seq(channelId_ag_private, channelId_gh, invoiceRoutingHint.shortChannelId)) - sender.send(router, FinalizeRoute(10000 msat, preComputedRoute, assistedRoutes = Seq(Seq(invoiceRoutingHint)))) + val amount = RoutingHeuristics.CAPACITY_CHANNEL_LOW * 2 + // the amount affects the way we estimate the channel capacity of the hinted channel + assert(amount > RoutingHeuristics.CAPACITY_CHANNEL_LOW) + sender.send(router, FinalizeRoute(amount, preComputedRoute, assistedRoutes = Seq(Seq(invoiceRoutingHint)))) val response = sender.expectMsgType[RouteResponse] assert(response.routes.length === 1) val route = response.routes.head assert(route.hops.map(_.nodeId) === Seq(a, g, h)) assert(route.hops.map(_.nextNodeId) === Seq(g, h, targetNodeId)) - assert(route.hops.map(_.lastUpdate).dropRight(1) === Seq(update_ag_private, update_gh)) - assert(route.hops.last.lastUpdate.shortChannelId === invoiceRoutingHint.shortChannelId) - assert(route.hops.last.lastUpdate.feeBaseMsat === invoiceRoutingHint.feeBase) - assert(route.hops.last.lastUpdate.feeProportionalMillionths === invoiceRoutingHint.feeProportionalMillionths) - assert(route.hops.last.lastUpdate.cltvExpiryDelta === invoiceRoutingHint.cltvExpiryDelta) + assert(route.hops.map(_.params).dropRight(1) === Seq(ChannelRelayParams.FromAnnouncement(update_ag_private), ChannelRelayParams.FromAnnouncement(update_gh))) + assert(route.hops.last.params === ChannelRelayParams.FromHint(invoiceRoutingHint, amount + nodeFee(invoiceRoutingHint.feeBase, invoiceRoutingHint.feeProportionalMillionths, amount))) } } diff --git a/eclair-node/src/test/resources/api/findroute-full b/eclair-node/src/test/resources/api/findroute-full index d41d006dd..dff011caa 100644 --- a/eclair-node/src/test/resources/api/findroute-full +++ b/eclair-node/src/test/resources/api/findroute-full @@ -1 +1 @@ -{"routes":[{"amount":456,"hops":[{"nodeId":"03007e67dc5a8fd2b2ef21cb310ab6359ddb51f3f86a8b79b8b1e23bc3a6ea150a","nextNodeId":"026105f6cb4862810be989385d16f04b0f748f6f2a14040338b1a534d45b4be1c1","lastUpdate":{"signature":"92cf3f12e161391986eb2cd7106ddab41a23c734f8f1ed120fb64f4b91f98f690ecf930388e62965f8aefbf1adafcd25a572669a125396dcfb83615208754679","chainHash":"024b7b3626554c44dcc2454ee3812458bfa68d9fced466edfab470844cb7ffe2","shortChannelId":"1x2x3","timestamp":{"iso":"1970-01-01T00:00:00Z","unix":0},"channelFlags":{"isEnabled":true,"isNode1":true},"cltvExpiryDelta":0,"htlcMinimumMsat":1,"feeBaseMsat":1,"feeProportionalMillionths":1,"tlvStream":{"records":[],"unknown":[]}}},{"nodeId":"026105f6cb4862810be989385d16f04b0f748f6f2a14040338b1a534d45b4be1c1","nextNodeId":"038cfa2b5857843ee90cff91b06f692c0d8fe201921ee6387aee901d64f43699f0","lastUpdate":{"signature":"92cf3f12e161391986eb2cd7106ddab41a23c734f8f1ed120fb64f4b91f98f690ecf930388e62965f8aefbf1adafcd25a572669a125396dcfb83615208754679","chainHash":"024b7b3626554c44dcc2454ee3812458bfa68d9fced466edfab470844cb7ffe2","shortChannelId":"1x2x4","timestamp":{"iso":"1970-01-01T00:00:00Z","unix":0},"channelFlags":{"isEnabled":true,"isNode1":true},"cltvExpiryDelta":0,"htlcMinimumMsat":1,"feeBaseMsat":1,"feeProportionalMillionths":1,"tlvStream":{"records":[],"unknown":[]}}},{"nodeId":"038cfa2b5857843ee90cff91b06f692c0d8fe201921ee6387aee901d64f43699f0","nextNodeId":"02be60276e294c6921240daae33a361d214d02578656df0e74c61a09c3196e51df","lastUpdate":{"signature":"92cf3f12e161391986eb2cd7106ddab41a23c734f8f1ed120fb64f4b91f98f690ecf930388e62965f8aefbf1adafcd25a572669a125396dcfb83615208754679","chainHash":"024b7b3626554c44dcc2454ee3812458bfa68d9fced466edfab470844cb7ffe2","shortChannelId":"1x2x5","timestamp":{"iso":"1970-01-01T00:00:00Z","unix":0},"channelFlags":{"isEnabled":true,"isNode1":true},"cltvExpiryDelta":0,"htlcMinimumMsat":1,"feeBaseMsat":1,"feeProportionalMillionths":1,"tlvStream":{"records":[],"unknown":[]}}}]}]} \ No newline at end of file +{"routes":[{"amount":456,"hops":[{"nodeId":"03007e67dc5a8fd2b2ef21cb310ab6359ddb51f3f86a8b79b8b1e23bc3a6ea150a","nextNodeId":"026105f6cb4862810be989385d16f04b0f748f6f2a14040338b1a534d45b4be1c1","source":{"type":"announcement","channelUpdate":{"signature":"92cf3f12e161391986eb2cd7106ddab41a23c734f8f1ed120fb64f4b91f98f690ecf930388e62965f8aefbf1adafcd25a572669a125396dcfb83615208754679","chainHash":"024b7b3626554c44dcc2454ee3812458bfa68d9fced466edfab470844cb7ffe2","shortChannelId":"1x2x3","timestamp":{"iso":"1970-01-01T00:00:00Z","unix":0},"channelFlags":{"isEnabled":true,"isNode1":true},"cltvExpiryDelta":0,"htlcMinimumMsat":1,"feeBaseMsat":1,"feeProportionalMillionths":1,"tlvStream":{"records":[],"unknown":[]}}}},{"nodeId":"026105f6cb4862810be989385d16f04b0f748f6f2a14040338b1a534d45b4be1c1","nextNodeId":"038cfa2b5857843ee90cff91b06f692c0d8fe201921ee6387aee901d64f43699f0","source":{"type":"announcement","channelUpdate":{"signature":"92cf3f12e161391986eb2cd7106ddab41a23c734f8f1ed120fb64f4b91f98f690ecf930388e62965f8aefbf1adafcd25a572669a125396dcfb83615208754679","chainHash":"024b7b3626554c44dcc2454ee3812458bfa68d9fced466edfab470844cb7ffe2","shortChannelId":"1x2x4","timestamp":{"iso":"1970-01-01T00:00:00Z","unix":0},"channelFlags":{"isEnabled":true,"isNode1":true},"cltvExpiryDelta":0,"htlcMinimumMsat":1,"feeBaseMsat":1,"feeProportionalMillionths":1,"tlvStream":{"records":[],"unknown":[]}}}},{"nodeId":"038cfa2b5857843ee90cff91b06f692c0d8fe201921ee6387aee901d64f43699f0","nextNodeId":"02be60276e294c6921240daae33a361d214d02578656df0e74c61a09c3196e51df","source":{"type":"announcement","channelUpdate":{"signature":"92cf3f12e161391986eb2cd7106ddab41a23c734f8f1ed120fb64f4b91f98f690ecf930388e62965f8aefbf1adafcd25a572669a125396dcfb83615208754679","chainHash":"024b7b3626554c44dcc2454ee3812458bfa68d9fced466edfab470844cb7ffe2","shortChannelId":"1x2x5","timestamp":{"iso":"1970-01-01T00:00:00Z","unix":0},"channelFlags":{"isEnabled":true,"isNode1":true},"cltvExpiryDelta":0,"htlcMinimumMsat":1,"feeBaseMsat":1,"feeProportionalMillionths":1,"tlvStream":{"records":[],"unknown":[]}}}}]}]} \ No newline at end of file diff --git a/eclair-node/src/test/scala/fr/acinq/eclair/api/ApiServiceSpec.scala b/eclair-node/src/test/scala/fr/acinq/eclair/api/ApiServiceSpec.scala index be381b571..e54c2944b 100644 --- a/eclair-node/src/test/scala/fr/acinq/eclair/api/ApiServiceSpec.scala +++ b/eclair-node/src/test/scala/fr/acinq/eclair/api/ApiServiceSpec.scala @@ -46,7 +46,7 @@ import fr.acinq.eclair.payment.relay.Relayer.ChannelBalance import fr.acinq.eclair.payment.send.MultiPartPaymentLifecycle.PreimageReceived import fr.acinq.eclair.payment.send.PaymentInitiator.SendPaymentToRouteResponse import fr.acinq.eclair.router.Router -import fr.acinq.eclair.router.Router.PredefinedNodeRoute +import fr.acinq.eclair.router.Router.{ChannelRelayParams, PredefinedNodeRoute} import fr.acinq.eclair.wire.protocol._ import org.json4s.{Formats, Serialization} import org.mockito.scalatest.IdiomaticMockito @@ -1003,10 +1003,12 @@ class ApiServiceSpec extends AnyFunSuite with ScalatestRouteTest with IdiomaticM feeProportionalMillionths = 1, htlcMaximumMsat = None ) + val mockChannelUpdate2 = mockChannelUpdate1.copy(shortChannelId = ShortChannelId(BlockHeight(1), 2, 4)) + val mockChannelUpdate3 = mockChannelUpdate1.copy(shortChannelId = ShortChannelId(BlockHeight(1), 2, 5)) - val mockHop1 = Router.ChannelHop(PublicKey.fromBin(ByteVector.fromValidHex("03007e67dc5a8fd2b2ef21cb310ab6359ddb51f3f86a8b79b8b1e23bc3a6ea150a")), PublicKey.fromBin(ByteVector.fromValidHex("026105f6cb4862810be989385d16f04b0f748f6f2a14040338b1a534d45b4be1c1")), mockChannelUpdate1) - val mockHop2 = Router.ChannelHop(mockHop1.nextNodeId, PublicKey.fromBin(ByteVector.fromValidHex("038cfa2b5857843ee90cff91b06f692c0d8fe201921ee6387aee901d64f43699f0")), mockChannelUpdate1.copy(shortChannelId = ShortChannelId(BlockHeight(1), 2, 4))) - val mockHop3 = Router.ChannelHop(mockHop2.nextNodeId, PublicKey.fromBin(ByteVector.fromValidHex("02be60276e294c6921240daae33a361d214d02578656df0e74c61a09c3196e51df")), mockChannelUpdate1.copy(shortChannelId = ShortChannelId(BlockHeight(1), 2, 5))) + val mockHop1 = Router.ChannelHop(mockChannelUpdate1.shortChannelId, PublicKey.fromBin(ByteVector.fromValidHex("03007e67dc5a8fd2b2ef21cb310ab6359ddb51f3f86a8b79b8b1e23bc3a6ea150a")), PublicKey.fromBin(ByteVector.fromValidHex("026105f6cb4862810be989385d16f04b0f748f6f2a14040338b1a534d45b4be1c1")), ChannelRelayParams.FromAnnouncement(mockChannelUpdate1)) + val mockHop2 = Router.ChannelHop(mockChannelUpdate2.shortChannelId, mockHop1.nextNodeId, PublicKey.fromBin(ByteVector.fromValidHex("038cfa2b5857843ee90cff91b06f692c0d8fe201921ee6387aee901d64f43699f0")), ChannelRelayParams.FromAnnouncement(mockChannelUpdate2)) + val mockHop3 = Router.ChannelHop(mockChannelUpdate3.shortChannelId, mockHop2.nextNodeId, PublicKey.fromBin(ByteVector.fromValidHex("02be60276e294c6921240daae33a361d214d02578656df0e74c61a09c3196e51df")), ChannelRelayParams.FromAnnouncement(mockChannelUpdate3)) val mockHops = Seq(mockHop1, mockHop2, mockHop3) val eclair = mock[Eclair]