Skip to content

Commit

Permalink
Find route for messages (#2656)
Browse files Browse the repository at this point in the history
When sending a message, the postman can now ask the router to find a route using channels only.
The same route is also used as a reply path when applicable.

The graph data structure has been updated to include both active and disabled edges.
The graph now contains features for vertices.
  • Loading branch information
thomash-acinq committed Jun 19, 2023
1 parent 210b919 commit 194f5dd
Show file tree
Hide file tree
Showing 27 changed files with 989 additions and 401 deletions.
17 changes: 16 additions & 1 deletion eclair-core/src/main/resources/reference.conf
Original file line number Diff line number Diff line change
Expand Up @@ -307,6 +307,18 @@ eclair {
channel-query-chunk-size = 100 // max number of short_channel_ids in query_short_channel_ids *do not change this unless you know what you are doing*
}

message-path-finding {
max-route-length = 6
ratios {
// The next three weights must sum to one.
base = 0.6 // when computing the weight for a channel, proportion that stays the same for all channels
channel-age = 0.1 // when computing the weight for a channel, consider its AGE in this proportion
channel-capacity = 0.3 // when computing the weight for a channel, consider its CAPACITY in this proportion

disabled-multiplier = 2.5 // How much we prefer relaying a message along an active channel instead of a disabled one.
}
}

path-finding {
default {
randomize-route-selection = true // when computing a route for a payment we randomize the final selection
Expand Down Expand Up @@ -478,8 +490,11 @@ eclair {

max-per-peer-per-second = 10

# Minimum number of hops before our node to hide it in the reply paths that we build
min-intermediate-hops = 6

# Consider a message to be lost if we haven't received a reply after that amount of time
reply-timeout = 5 seconds
reply-timeout = 15 seconds

# If we expect a reply but do not get one, retry until we reach this number of attempts
max-attempts = 3
Expand Down
35 changes: 19 additions & 16 deletions eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala
Original file line number Diff line number Diff line change
Expand Up @@ -172,11 +172,11 @@ trait Eclair {

def verifyMessage(message: ByteVector, recoverableSignature: ByteVector): VerifiedMessage

def sendOnionMessage(intermediateNodes: Seq[PublicKey], destination: Either[PublicKey, Sphinx.RouteBlinding.BlindedRoute], replyPath: Option[Seq[PublicKey]], userCustomContent: ByteVector)(implicit timeout: Timeout): Future[SendOnionMessageResponse]
def sendOnionMessage(intermediateNodes_opt: Option[Seq[PublicKey]], destination: Either[PublicKey, Sphinx.RouteBlinding.BlindedRoute], expectsReply: Boolean, userCustomContent: ByteVector)(implicit timeout: Timeout): Future[SendOnionMessageResponse]

def payOffer(offer: Offer, amount: MilliSatoshi, quantity: Long, externalId_opt: Option[String] = None, maxAttempts_opt: Option[Int] = None, maxFeeFlat_opt: Option[Satoshi] = None, maxFeePct_opt: Option[Double] = None, pathFindingExperimentName_opt: Option[String] = None)(implicit timeout: Timeout): Future[UUID]
def payOffer(offer: Offer, amount: MilliSatoshi, quantity: Long, externalId_opt: Option[String] = None, maxAttempts_opt: Option[Int] = None, maxFeeFlat_opt: Option[Satoshi] = None, maxFeePct_opt: Option[Double] = None, pathFindingExperimentName_opt: Option[String] = None, connectDirectly: Boolean = false)(implicit timeout: Timeout): Future[UUID]

def payOfferBlocking(offer: Offer, amount: MilliSatoshi, quantity: Long, externalId_opt: Option[String] = None, maxAttempts_opt: Option[Int] = None, maxFeeFlat_opt: Option[Satoshi] = None, maxFeePct_opt: Option[Double] = None, pathFindingExperimentName_opt: Option[String] = None)(implicit timeout: Timeout): Future[PaymentEvent]
def payOfferBlocking(offer: Offer, amount: MilliSatoshi, quantity: Long, externalId_opt: Option[String] = None, maxAttempts_opt: Option[Int] = None, maxFeeFlat_opt: Option[Satoshi] = None, maxFeePct_opt: Option[Double] = None, pathFindingExperimentName_opt: Option[String] = None, connectDirectly: Boolean = false)(implicit timeout: Timeout): Future[PaymentEvent]

def stop(): Future[Unit]
}
Expand Down Expand Up @@ -621,22 +621,22 @@ class EclairImpl(appKit: Kit) extends Eclair with Logging {
}
}

override def sendOnionMessage(intermediateNodes: Seq[PublicKey],
override def sendOnionMessage(intermediateNodes_opt: Option[Seq[PublicKey]],
recipient: Either[PublicKey, Sphinx.RouteBlinding.BlindedRoute],
replyPath: Option[Seq[PublicKey]],
expectsReply: Boolean,
userCustomContent: ByteVector)(implicit timeout: Timeout): Future[SendOnionMessageResponse] = {
if (replyPath.nonEmpty && (replyPath.get.isEmpty || replyPath.get.last != appKit.nodeParams.nodeId)) {
return Future.failed(new Exception("Reply path must end at our node."))
}
TlvCodecs.tlvStream(MessageOnionCodecs.onionTlvCodec).decode(userCustomContent.bits) match {
case Attempt.Successful(DecodeResult(userTlvs, _)) =>
val destination = recipient match {
case Left(key) => OnionMessages.Recipient(key, None)
case Right(route) => OnionMessages.BlindedPath(route)
}
appKit.postman.ask(ref => Postman.SendMessage(intermediateNodes, destination, replyPath, userTlvs, ref, appKit.nodeParams.onionMessageConfig.timeout)).map {
case Postman.Response(payload) =>
SendOnionMessageResponse(sent = true, None, Some(SendOnionMessageResponsePayload(payload.records)))
val routingStrategy = intermediateNodes_opt match {
case Some(intermediateNodes) => OnionMessages.RoutingStrategy.UseRoute(intermediateNodes)
case None => OnionMessages.RoutingStrategy.FindRoute
}
appKit.postman.ask(ref => Postman.SendMessage(destination, routingStrategy, userTlvs, expectsReply, ref)).map {
case Postman.Response(payload) => SendOnionMessageResponse(sent = true, None, Some(SendOnionMessageResponsePayload(payload.records)))
case Postman.NoReply => SendOnionMessageResponse(sent = true, Some("No response"), None)
case Postman.MessageSent => SendOnionMessageResponse(sent = true, None, None)
case Postman.MessageFailed(failure: String) => SendOnionMessageResponse(sent = false, Some(failure), None)
Expand All @@ -653,6 +653,7 @@ class EclairImpl(appKit: Kit) extends Eclair with Logging {
maxFeeFlat_opt: Option[Satoshi],
maxFeePct_opt: Option[Double],
pathFindingExperimentName_opt: Option[String],
connectDirectly: Boolean,
blocking: Boolean)(implicit timeout: Timeout): Future[Any] = {
if (externalId_opt.exists(_.length > externalIdMaxLength)) {
return Future.failed(new IllegalArgumentException(s"externalId is too long: cannot exceed $externalIdMaxLength characters"))
Expand All @@ -664,7 +665,7 @@ class EclairImpl(appKit: Kit) extends Eclair with Logging {
.modify(_.boundaries.maxFeeFlat).setToIfDefined(maxFeeFlat_opt.map(_.toMilliSatoshi))
case Left(t) => return Future.failed(t)
}
val sendPaymentConfig = OfferPayment.SendPaymentConfig(externalId_opt, maxAttempts_opt.getOrElse(appKit.nodeParams.maxPaymentAttempts), routeParams, blocking)
val sendPaymentConfig = OfferPayment.SendPaymentConfig(externalId_opt, connectDirectly, maxAttempts_opt.getOrElse(appKit.nodeParams.maxPaymentAttempts), routeParams, blocking)
val offerPayment = appKit.system.spawnAnonymous(OfferPayment(appKit.nodeParams, appKit.postman, appKit.paymentInitiator))
offerPayment.ask((ref: typed.ActorRef[Any]) => OfferPayment.PayOffer(ref.toClassic, offer, amount, quantity, sendPaymentConfig)).flatMap {
case f: OfferPayment.Failure => Future.failed(new Exception(f.toString))
Expand All @@ -679,8 +680,9 @@ class EclairImpl(appKit: Kit) extends Eclair with Logging {
maxAttempts_opt: Option[Int],
maxFeeFlat_opt: Option[Satoshi],
maxFeePct_opt: Option[Double],
pathFindingExperimentName_opt: Option[String])(implicit timeout: Timeout): Future[UUID] = {
payOfferInternal(offer, amount, quantity, externalId_opt, maxAttempts_opt, maxFeeFlat_opt, maxFeePct_opt, pathFindingExperimentName_opt, blocking = false).mapTo[UUID]
pathFindingExperimentName_opt: Option[String],
connectDirectly: Boolean)(implicit timeout: Timeout): Future[UUID] = {
payOfferInternal(offer, amount, quantity, externalId_opt, maxAttempts_opt, maxFeeFlat_opt, maxFeePct_opt, pathFindingExperimentName_opt, connectDirectly, blocking = false).mapTo[UUID]
}

override def payOfferBlocking(offer: Offer,
Expand All @@ -690,8 +692,9 @@ class EclairImpl(appKit: Kit) extends Eclair with Logging {
maxAttempts_opt: Option[Int],
maxFeeFlat_opt: Option[Satoshi],
maxFeePct_opt: Option[Double],
pathFindingExperimentName_opt: Option[String])(implicit timeout: Timeout): Future[PaymentEvent] = {
payOfferInternal(offer, amount, quantity, externalId_opt, maxAttempts_opt, maxFeeFlat_opt, maxFeePct_opt, pathFindingExperimentName_opt, blocking = true).mapTo[PaymentEvent]
pathFindingExperimentName_opt: Option[String],
connectDirectly: Boolean)(implicit timeout: Timeout): Future[PaymentEvent] = {
payOfferInternal(offer, amount, quantity, externalId_opt, maxAttempts_opt, maxFeeFlat_opt, maxFeePct_opt, pathFindingExperimentName_opt, connectDirectly, blocking = true).mapTo[PaymentEvent]
}

override def stop(): Future[Unit] = {
Expand Down
15 changes: 13 additions & 2 deletions eclair-core/src/main/scala/fr/acinq/eclair/NodeParams.scala
Original file line number Diff line number Diff line change
Expand Up @@ -33,8 +33,8 @@ import fr.acinq.eclair.message.OnionMessages.OnionMessageConfig
import fr.acinq.eclair.payment.relay.Relayer.{AsyncPaymentsParams, RelayFees, RelayParams}
import fr.acinq.eclair.router.Announcements.AddressException
import fr.acinq.eclair.router.Graph.{HeuristicsConstants, WeightRatios}
import fr.acinq.eclair.router.PathFindingExperimentConf
import fr.acinq.eclair.router.Router.{MultiPartParams, PathFindingConf, RouterConf, SearchBoundaries}
import fr.acinq.eclair.router.{Graph, PathFindingExperimentConf}
import fr.acinq.eclair.router.Router.{MessageRouteParams, MultiPartParams, PathFindingConf, RouterConf, SearchBoundaries}
import fr.acinq.eclair.tor.Socks5ProxyParams
import fr.acinq.eclair.wire.protocol._
import grizzled.slf4j.Logging
Expand Down Expand Up @@ -422,6 +422,15 @@ object NodeParams extends Logging {
PathFindingExperimentConf(experiments.toMap)
}

def getMessageRouteParams(config: Config): MessageRouteParams = {
val maxRouteLength = config.getInt("max-route-length")
val ratioBase = config.getDouble("ratios.base")
val ratioAge = config.getDouble("ratios.channel-age")
val ratioCapacity = config.getDouble("ratios.channel-capacity")
val disabledMultiplier = config.getDouble("ratios.disabled-multiplier")
MessageRouteParams(maxRouteLength, Graph.MessagePath.WeightRatios(ratioBase, ratioAge, ratioCapacity, disabledMultiplier))
}

val unhandledExceptionStrategy = config.getString("channel.unhandled-exception-strategy") match {
case "local-close" => UnhandledExceptionStrategy.LocalClose
case "stop" => UnhandledExceptionStrategy.Stop
Expand Down Expand Up @@ -557,6 +566,7 @@ object NodeParams extends Logging {
channelRangeChunkSize = config.getInt("router.sync.channel-range-chunk-size"),
channelQueryChunkSize = config.getInt("router.sync.channel-query-chunk-size"),
pathFindingExperimentConf = getPathFindingExperimentConf(config.getConfig("router.path-finding.experiments")),
messageRouteParams = getMessageRouteParams(config.getConfig("router.message-path-finding")),
balanceEstimateHalfLife = FiniteDuration(config.getDuration("router.balance-estimate-half-life").getSeconds, TimeUnit.SECONDS),
),
socksProxy_opt = socksProxy_opt,
Expand All @@ -568,6 +578,7 @@ object NodeParams extends Logging {
blockchainWatchdogSources = config.getStringList("blockchain-watchdog.sources").asScala.toSeq,
onionMessageConfig = OnionMessageConfig(
relayPolicy = onionMessageRelayPolicy,
minIntermediateHops = config.getInt("onion-messages.min-intermediate-hops"),
timeout = FiniteDuration(config.getDuration("onion-messages.reply-timeout").getSeconds, TimeUnit.SECONDS),
maxAttempts = config.getInt("onion-messages.max-attempts"),
),
Expand Down
2 changes: 1 addition & 1 deletion eclair-core/src/main/scala/fr/acinq/eclair/Setup.scala
Original file line number Diff line number Diff line change
Expand Up @@ -368,7 +368,7 @@ class Setup(val datadir: File,

balanceActor = system.spawn(BalanceActor(nodeParams.db, bitcoinClient, channelsListener, nodeParams.balanceCheckInterval), name = "balance-actor")

postman = system.spawn(Behaviors.supervise(Postman(nodeParams, switchboard.toTyped, offerManager)).onFailure(typed.SupervisorStrategy.restart), name = "postman")
postman = system.spawn(Behaviors.supervise(Postman(nodeParams, switchboard.toTyped, router.toTyped, offerManager)).onFailure(typed.SupervisorStrategy.restart), name = "postman")

kit = Kit(
nodeParams = nodeParams,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,15 @@ import scala.concurrent.duration.FiniteDuration

object OnionMessages {

/**
* @param relayPolicy When to relay onion messages (always, never, only along existing channels).
* @param minIntermediateHops For routes we build to us, minimum number of hops before our node. Dummy hops are added
* if needed to hide our position in the network.
* @param timeout Time after which we consider that the message has been lost and stop waiting for a reply.
* @param maxAttempts Maximum number of attempts for sending a message.
*/
case class OnionMessageConfig(relayPolicy: RelayPolicy,
minIntermediateHops: Int,
timeout: FiniteDuration,
maxAttempts: Int)

Expand All @@ -43,6 +51,18 @@ object OnionMessages {
case class Recipient(nodeId: PublicKey, pathId: Option[ByteVector], padding: Option[ByteVector] = None, customTlvs: Set[GenericTlv] = Set.empty) extends Destination
// @formatter:on

// @formatter:off
sealed trait RoutingStrategy
object RoutingStrategy {
/** Use the provided route to reach the recipient or the blinded path's introduction node. */
case class UseRoute(intermediateNodes: Seq[PublicKey]) extends RoutingStrategy
/** Directly connect to the recipient or the blinded path's introduction node. */
val connectDirectly: UseRoute = UseRoute(Nil)
/** Use path-finding to find a route to reach the recipient or the blinded path's introduction node. */
case object FindRoute extends RoutingStrategy
}
// @formatter:on

private def buildIntermediatePayloads(intermediateNodes: Seq[IntermediateNode], nextTlvs: Set[RouteBlindingEncryptedDataTlv]): Seq[ByteVector] = {
if (intermediateNodes.isEmpty) {
Nil
Expand All @@ -63,9 +83,9 @@ object OnionMessages {
}

private[message] def buildRouteFrom(originKey: PrivateKey,
blindingSecret: PrivateKey,
intermediateNodes: Seq[IntermediateNode],
destination: Destination): Option[Sphinx.RouteBlinding.BlindedRoute] = {
blindingSecret: PrivateKey,
intermediateNodes: Seq[IntermediateNode],
destination: Destination): Option[Sphinx.RouteBlinding.BlindedRoute] = {
destination match {
case recipient: Recipient => Some(buildRoute(blindingSecret, intermediateNodes, recipient))
case BlindedPath(route) if route.introductionNodeId == originKey.publicKey =>
Expand Down
Loading

0 comments on commit 194f5dd

Please sign in to comment.