From da1b1a6c87d0db0328b1d472f7375bda5964d263 Mon Sep 17 00:00:00 2001 From: Thomas HUET Date: Mon, 12 Jun 2023 16:25:15 +0200 Subject: [PATCH 1/2] Add local reputation --- docs/release-notes/eclair-vnext.md | 27 ++++++ eclair-core/src/main/resources/reference.conf | 17 ++++ .../scala/fr/acinq/eclair/NodeParams.scala | 9 +- .../main/scala/fr/acinq/eclair/Setup.scala | 8 +- .../eclair/channel/ChannelExceptions.scala | 2 + .../fr/acinq/eclair/channel/Commitments.scala | 39 +++++++- .../fr/acinq/eclair/channel/Monitoring.scala | 6 ++ .../eclair/payment/relay/ChannelRelay.scala | 28 +++++- .../eclair/payment/relay/ChannelRelayer.scala | 12 ++- .../eclair/payment/relay/NodeRelay.scala | 62 ++++++++---- .../eclair/payment/relay/NodeRelayer.scala | 14 ++- .../acinq/eclair/payment/relay/Relayer.scala | 14 +-- .../acinq/eclair/reputation/Reputation.scala | 95 ++++++++++++++++++ .../reputation/ReputationRecorder.scala | 90 +++++++++++++++++ .../scala/fr/acinq/eclair/TestConstants.scala | 9 +- .../fr/acinq/eclair/channel/FuzzySpec.scala | 4 +- .../basic/fixtures/MinimalNodeFixture.scala | 3 +- .../payment/PostRestartHtlcCleanerSpec.scala | 2 +- .../payment/relay/ChannelRelayerSpec.scala | 91 ++++++++++++++---- .../payment/relay/NodeRelayerSpec.scala | 78 +++++++++++---- .../eclair/payment/relay/RelayerSpec.scala | 17 +++- .../reputation/ReputationRecorderSpec.scala | 96 +++++++++++++++++++ .../eclair/reputation/ReputationSpec.scala | 81 ++++++++++++++++ 23 files changed, 712 insertions(+), 92 deletions(-) create mode 100644 eclair-core/src/main/scala/fr/acinq/eclair/reputation/Reputation.scala create mode 100644 eclair-core/src/main/scala/fr/acinq/eclair/reputation/ReputationRecorder.scala create mode 100644 eclair-core/src/test/scala/fr/acinq/eclair/reputation/ReputationRecorderSpec.scala create mode 100644 eclair-core/src/test/scala/fr/acinq/eclair/reputation/ReputationSpec.scala diff --git a/docs/release-notes/eclair-vnext.md b/docs/release-notes/eclair-vnext.md index bd7013d146..a99a617e9c 100644 --- a/docs/release-notes/eclair-vnext.md +++ b/docs/release-notes/eclair-vnext.md @@ -33,6 +33,33 @@ Existing `static_remote_key` channels will continue to work. You can override th Eclair will not allow remote peers to open new obsolete channels that do not support `option_static_remotekey`. +### Local reputation and HTLC endorsement + +To protect against jamming attacks, eclair gives a reputation to its neighbors and uses it to decide if a HTLC should be relayed given how congested the outgoing channel is. +The reputation is basically how much this node paid us in fees divided by how much they should have paid us for the liquidity and slots that they blocked. +The reputation is per incoming node and endorsement level. +The confidence that the HTLC will be fulfilled is transmitted to the next node using the endorsement TLV of the `update_add_htlc` message. +Note that HTLCs that are considered dangerous are still relayed: this is the first phase of a network-wide experimentation aimed at collecting data. + +To configure, edit `eclair.conf`: + +```eclair.conf +// We assign reputations to our peers to prioritize payments during congestion. +// The reputation is computed as fees paid divided by what should have been paid if all payments were successful. +eclair.relay.peer-reputation { + // Set this parameter to false to disable the reputation algorithm and simply relay the incoming endorsement + // value, as described by https://github.com/lightning/blips/blob/master/blip-0004.md, + enabled = true + // Reputation decays with the following half life to emphasize recent behavior. + half-life = 7 days + // Payments that stay pending for longer than this get penalized + max-relay-duration = 12 seconds + // Pending payments are counted as failed, and because they could potentially stay pending for a very long time, + // the following multiplier is applied. + pending-multiplier = 1000 // A pending payment counts as a thousand failed ones. +} +``` + ### API changes - `channelstats` now takes optional parameters `--count` and `--skip` to control pagination. By default, it will return first 10 entries. (#2890) diff --git a/eclair-core/src/main/resources/reference.conf b/eclair-core/src/main/resources/reference.conf index 77dd6f058c..3ba42a966f 100644 --- a/eclair-core/src/main/resources/reference.conf +++ b/eclair-core/src/main/resources/reference.conf @@ -238,6 +238,23 @@ eclair { // Number of blocks before the incoming HTLC expires that an async payment must be triggered by the receiver cancel-safety-before-timeout-blocks = 144 } + + // We assign reputation to our peers to prioritize payments during congestion. + // The reputation is computed as fees paid divided by what should have been paid if all payments were successful. + peer-reputation { + // Set this parameter to false to disable the reputation algorithm and simply relay the incoming endorsement + // value, as described by https://github.com/lightning/blips/blob/master/blip-0004.md, + enabled = true + // Reputation decays with the following half life to emphasize recent behavior. + half-life = 15 days + // Payments that stay pending for longer than this get penalized. + max-relay-duration = 12 seconds + // Pending payments are counted as failed, and because they could potentially stay pending for a very long time, + // the following multiplier is applied. We want it to be as close as possible to the true cost of a worst case + // HTLC (max-cltv-delta / max-relay-duration, around 100000 with default parameters) while still being comparable + // to the number of HTLCs received per peer during twice the half life. + pending-multiplier = 200 // A pending payment counts as two hundred failed ones. + } } on-chain-fees { diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/NodeParams.scala b/eclair-core/src/main/scala/fr/acinq/eclair/NodeParams.scala index 55a0b6fc8b..e89737a45e 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/NodeParams.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/NodeParams.scala @@ -32,6 +32,7 @@ import fr.acinq.eclair.io.{PeerConnection, PeerReadyNotifier} import fr.acinq.eclair.message.OnionMessages.OnionMessageConfig import fr.acinq.eclair.payment.relay.OnTheFlyFunding import fr.acinq.eclair.payment.relay.Relayer.{AsyncPaymentsParams, RelayFees, RelayParams} +import fr.acinq.eclair.reputation.Reputation import fr.acinq.eclair.router.Announcements.AddressException import fr.acinq.eclair.router.Graph.{HeuristicsConstants, WeightRatios} import fr.acinq.eclair.router.Router._ @@ -619,7 +620,13 @@ object NodeParams extends Logging { privateChannelFees = getRelayFees(config.getConfig("relay.fees.private-channels")), minTrampolineFees = getRelayFees(config.getConfig("relay.fees.min-trampoline")), enforcementDelay = FiniteDuration(config.getDuration("relay.fees.enforcement-delay").getSeconds, TimeUnit.SECONDS), - asyncPaymentsParams = AsyncPaymentsParams(asyncPaymentHoldTimeoutBlocks, asyncPaymentCancelSafetyBeforeTimeoutBlocks) + asyncPaymentsParams = AsyncPaymentsParams(asyncPaymentHoldTimeoutBlocks, asyncPaymentCancelSafetyBeforeTimeoutBlocks), + peerReputationConfig = Reputation.Config( + enabled = config.getBoolean("relay.peer-reputation.enabled"), + halfLife = FiniteDuration(config.getDuration("relay.peer-reputation.half-life").getSeconds, TimeUnit.SECONDS), + maxRelayDuration = FiniteDuration(config.getDuration("relay.peer-reputation.max-relay-duration").getSeconds, TimeUnit.SECONDS), + pendingMultiplier = config.getDouble("relay.peer-reputation.pending-multiplier"), + ), ), db = database, autoReconnect = config.getBoolean("auto-reconnect"), diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/Setup.scala b/eclair-core/src/main/scala/fr/acinq/eclair/Setup.scala index ce74a08deb..b39ca56a9d 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/Setup.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/Setup.scala @@ -44,6 +44,7 @@ import fr.acinq.eclair.payment.offer.OfferManager import fr.acinq.eclair.payment.receive.PaymentHandler import fr.acinq.eclair.payment.relay.{AsyncPaymentTriggerer, PostRestartHtlcCleaner, Relayer} import fr.acinq.eclair.payment.send.{Autoprobe, PaymentInitiator} +import fr.acinq.eclair.reputation.ReputationRecorder import fr.acinq.eclair.router._ import fr.acinq.eclair.tor.{Controller, TorProtocolHandler} import fr.acinq.eclair.wire.protocol.NodeAddress @@ -362,7 +363,12 @@ class Setup(val datadir: File, paymentHandler = system.actorOf(SimpleSupervisor.props(PaymentHandler.props(nodeParams, register, offerManager), "payment-handler", SupervisorStrategy.Resume)) triggerer = system.spawn(Behaviors.supervise(AsyncPaymentTriggerer()).onFailure(typed.SupervisorStrategy.resume), name = "async-payment-triggerer") peerReadyManager = system.spawn(Behaviors.supervise(PeerReadyManager()).onFailure(typed.SupervisorStrategy.restart), name = "peer-ready-manager") - relayer = system.actorOf(SimpleSupervisor.props(Relayer.props(nodeParams, router, register, paymentHandler, Some(postRestartCleanUpInitialized)), "relayer", SupervisorStrategy.Resume)) + reputationRecorder_opt = if (nodeParams.relayParams.peerReputationConfig.enabled) { + Some(system.spawn(Behaviors.supervise(ReputationRecorder(nodeParams.relayParams.peerReputationConfig, Map.empty)).onFailure(typed.SupervisorStrategy.resume), name = "reputation-recorder")) + } else { + None + } + relayer = system.actorOf(SimpleSupervisor.props(Relayer.props(nodeParams, router, register, paymentHandler, reputationRecorder_opt, Some(postRestartCleanUpInitialized)), "relayer", SupervisorStrategy.Resume)) _ = relayer ! PostRestartHtlcCleaner.Init(channels) // Before initializing the switchboard (which re-connects us to the network) and the user-facing parts of the system, // we want to make sure the handler for post-restart broken HTLCs has finished initializing. diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelExceptions.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelExceptions.scala index 01cc44a8e6..8cf49c33d0 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelExceptions.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelExceptions.scala @@ -123,6 +123,8 @@ case class ExpiryTooBig (override val channelId: Byte case class HtlcValueTooSmall (override val channelId: ByteVector32, minimum: MilliSatoshi, actual: MilliSatoshi) extends ChannelException(channelId, s"htlc value too small: minimum=$minimum actual=$actual") case class HtlcValueTooHighInFlight (override val channelId: ByteVector32, maximum: MilliSatoshi, actual: MilliSatoshi) extends ChannelException(channelId, s"in-flight htlcs hold too much value: maximum=$maximum actual=$actual") case class TooManyAcceptedHtlcs (override val channelId: ByteVector32, maximum: Long) extends ChannelException(channelId, s"too many accepted htlcs: maximum=$maximum") +case class TooManySmallHtlcs (override val channelId: ByteVector32, number: Long, below: MilliSatoshi) extends ChannelException(channelId, s"too many small htlcs: $number HTLCs below $below") +case class ConfidenceTooLow (override val channelId: ByteVector32, confidence: Double, occupancy: Double) extends ChannelException(channelId, s"confidence too low: confidence=$confidence occupancy=$occupancy") case class LocalDustHtlcExposureTooHigh (override val channelId: ByteVector32, maximum: Satoshi, actual: MilliSatoshi) extends ChannelException(channelId, s"dust htlcs hold too much value: maximum=$maximum actual=$actual") case class RemoteDustHtlcExposureTooHigh (override val channelId: ByteVector32, maximum: Satoshi, actual: MilliSatoshi) extends ChannelException(channelId, s"dust htlcs hold too much value: maximum=$maximum actual=$actual") case class InsufficientFunds (override val channelId: ByteVector32, amount: MilliSatoshi, missing: Satoshi, reserve: Satoshi, fees: Satoshi) extends ChannelException(channelId, s"insufficient funds: missing=$missing reserve=$reserve fees=$fees") diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/Commitments.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/Commitments.scala index b5a2fecfd7..5c71cae468 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/Commitments.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/Commitments.scala @@ -429,7 +429,7 @@ case class Commitment(fundingTxIndex: Long, localCommit.spec.htlcs.collect(DirectedHtlc.incoming).filter(nearlyExpired) } - def canSendAdd(amount: MilliSatoshi, params: ChannelParams, changes: CommitmentChanges, feerates: FeeratesPerKw, feeConf: OnChainFeeConf): Either[ChannelException, Unit] = { + def canSendAdd(amount: MilliSatoshi, params: ChannelParams, changes: CommitmentChanges, feerates: FeeratesPerKw, feeConf: OnChainFeeConf, confidence: Double): Either[ChannelException, Unit] = { // we allowed mismatches between our feerates and our remote's as long as commitments didn't contain any HTLC at risk // we need to verify that we're not disagreeing on feerates anymore before offering new HTLCs // NB: there may be a pending update_fee that hasn't been applied yet that needs to be taken into account @@ -488,7 +488,8 @@ case class Commitment(fundingTxIndex: Long, if (allowedHtlcValueInFlight < htlcValueInFlight) { return Left(HtlcValueTooHighInFlight(params.channelId, maximum = allowedHtlcValueInFlight, actual = htlcValueInFlight)) } - if (Seq(params.localParams.maxAcceptedHtlcs, params.remoteParams.maxAcceptedHtlcs).min < outgoingHtlcs.size) { + val maxAcceptedHtlcs = params.localParams.maxAcceptedHtlcs.min(params.remoteParams.maxAcceptedHtlcs) + if (maxAcceptedHtlcs < outgoingHtlcs.size) { return Left(TooManyAcceptedHtlcs(params.channelId, maximum = Seq(params.localParams.maxAcceptedHtlcs, params.remoteParams.maxAcceptedHtlcs).min)) } @@ -505,6 +506,18 @@ case class Commitment(fundingTxIndex: Long, return Left(RemoteDustHtlcExposureTooHigh(params.channelId, maxDustExposure, remoteDustExposureAfterAdd)) } + // Jamming protection + // Must be the last checks so that they can be ignored for shadow deployment. + for ((amountMsat, i) <- outgoingHtlcs.toSeq.map(_.amountMsat).sorted.zipWithIndex) { + if ((amountMsat.toLong < 1) || (math.log(amountMsat.toLong.toDouble) * maxAcceptedHtlcs / math.log(params.localParams.maxHtlcValueInFlightMsat.toLong.toDouble / maxAcceptedHtlcs) < i)) { + return Left(TooManySmallHtlcs(params.channelId, number = i + 1, below = amountMsat)) + } + } + val occupancy = (outgoingHtlcs.size.toDouble / maxAcceptedHtlcs).max(htlcValueInFlight.toLong.toDouble / allowedHtlcValueInFlight.toLong.toDouble) + if (confidence + 0.05 < occupancy) { + return Left(ConfidenceTooLow(params.channelId, confidence, occupancy)) + } + Right(()) } @@ -835,7 +848,7 @@ case class Commitments(params: ChannelParams, * @param cmd add HTLC command * @return either Left(failure, error message) where failure is a failure message (see BOLT #4 and the Failure Message class) or Right(new commitments, updateAddHtlc) */ - def sendAdd(cmd: CMD_ADD_HTLC, currentHeight: BlockHeight, channelConf: ChannelConf, feerates: FeeratesPerKw, feeConf: OnChainFeeConf): Either[ChannelException, (Commitments, UpdateAddHtlc)] = { + def sendAdd(cmd: CMD_ADD_HTLC, currentHeight: BlockHeight, channelConf: ChannelConf, feerates: FeeratesPerKw, feeConf: OnChainFeeConf)(implicit log: LoggingAdapter): Either[ChannelException, (Commitments, UpdateAddHtlc)] = { // we must ensure we're not relaying htlcs that are already expired, otherwise the downstream channel will instantly close // NB: we add a 3 blocks safety to reduce the probability of running into this when our bitcoin node is slightly outdated val minExpiry = CltvExpiry(currentHeight + 3) @@ -859,8 +872,24 @@ case class Commitments(params: ChannelParams, val changes1 = changes.addLocalProposal(add).copy(localNextHtlcId = changes.localNextHtlcId + 1) val originChannels1 = originChannels + (add.id -> cmd.origin) // we verify that this htlc is allowed in every active commitment - active.map(_.canSendAdd(add.amountMsat, params, changes1, feerates, feeConf)) - .collectFirst { case Left(f) => Left(f) } + val canSendAdds = active.map(_.canSendAdd(add.amountMsat, params, changes1, feerates, feeConf, cmd.confidence)) + // Log only for jamming protection. + canSendAdds.collectFirst { + case Left(f: TooManySmallHtlcs) => + log.info("TooManySmallHtlcs: {} outgoing HTLCs are below {}}", f.number, f.below) + Metrics.dropHtlc(f, Tags.Directions.Outgoing) + case Left(f: ConfidenceTooLow) => + log.info("ConfidenceTooLow: confidence is {}% while channel is {}% full", (100 * f.confidence).toInt, (100 * f.occupancy).toInt) + Metrics.dropHtlc(f, Tags.Directions.Outgoing) + } + canSendAdds.flatMap { // TODO: We ignore jamming protection, delete this flatMap to activate jamming protection. + case Left(_: TooManySmallHtlcs) | Left(_: ConfidenceTooLow) => None + case x => Some(x) + } + .collectFirst { case Left(f) => + Metrics.dropHtlc(f, Tags.Directions.Outgoing) + Left(f) + } .getOrElse(Right(copy(changes = changes1, originChannels = originChannels1), add)) } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/Monitoring.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/Monitoring.scala index 748751968c..648a0ce927 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/Monitoring.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/Monitoring.scala @@ -35,6 +35,7 @@ object Monitoring { val RemoteFeeratePerByte = Kamon.histogram("channels.remote-feerate-per-byte") val Splices = Kamon.histogram("channels.splices", "Splices") val ProcessMessage = Kamon.timer("channels.messages-processed") + val HtlcDropped = Kamon.counter("channels.htlc-dropped") def recordHtlcsInFlight(remoteSpec: CommitmentSpec, previousRemoteSpec: CommitmentSpec): Unit = { for (direction <- Tags.Directions.Incoming :: Tags.Directions.Outgoing :: Nil) { @@ -75,6 +76,10 @@ object Monitoring { Metrics.Splices.withTag(Tags.Origin, Tags.Origins.Remote).withTag(Tags.SpliceType, Tags.SpliceTypes.SpliceCpfp).record(Math.abs(fundingParams.remoteContribution.toLong)) } } + + def dropHtlc(reason: ChannelException, direction: String): Unit = { + HtlcDropped.withTag(Tags.Reason, reason.getClass.getSimpleName).withTag(Tags.Direction, direction).increment() + } } object Tags { @@ -85,6 +90,7 @@ object Monitoring { val State = "state" val CommitmentFormat = "commitment-format" val SpliceType = "splice-type" + val Reason = "reason" object Events { val Created = "created" 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 e6159d5ec2..cb3a9b7a94 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 @@ -17,6 +17,7 @@ package fr.acinq.eclair.payment.relay import akka.actor.ActorRef +import akka.actor.typed import akka.actor.typed.Behavior import akka.actor.typed.eventstream.EventStream import akka.actor.typed.scaladsl.adapter.TypedActorRefOps @@ -31,6 +32,8 @@ import fr.acinq.eclair.io.{Peer, PeerReadyNotifier} import fr.acinq.eclair.payment.Monitoring.{Metrics, Tags} import fr.acinq.eclair.payment.relay.Relayer.{OutgoingChannel, OutgoingChannelParams} import fr.acinq.eclair.payment.{ChannelPaymentRelayed, IncomingPaymentPacket} +import fr.acinq.eclair.reputation.ReputationRecorder +import fr.acinq.eclair.reputation.ReputationRecorder.{CancelRelay, GetConfidence, RecordResult} import fr.acinq.eclair.wire.protocol.FailureMessageCodecs.createBadOnionFailure import fr.acinq.eclair.wire.protocol.PaymentOnion.IntermediatePayload import fr.acinq.eclair.wire.protocol._ @@ -47,6 +50,7 @@ object ChannelRelay { sealed trait Command private case object DoRelay extends Command private case class WrappedPeerReadyResult(result: PeerReadyNotifier.Result) extends Command + private case class WrappedConfidence(confidence: Double) extends Command private case class WrappedForwardFailure(failure: Register.ForwardFailure[CMD_ADD_HTLC]) extends Command private case class WrappedAddResponse(res: CommandResponse[CMD_ADD_HTLC]) extends Command private case class WrappedOnTheFlyFundingResponse(result: Peer.ProposeOnTheFlyFundingResponse) extends Command @@ -61,6 +65,7 @@ object ChannelRelay { def apply(nodeParams: NodeParams, register: ActorRef, + reputationRecorder_opt: Option[typed.ActorRef[ReputationRecorder.ChannelRelayCommand]], channels: Map[ByteVector32, Relayer.OutgoingChannel], originNode: PublicKey, relayId: UUID, @@ -72,8 +77,17 @@ object ChannelRelay { paymentHash_opt = Some(r.add.paymentHash), nodeAlias_opt = Some(nodeParams.alias))) { val upstream = Upstream.Hot.Channel(r.add.removeUnknownTlvs(), TimestampMilli.now(), originNode) - val confidence = (r.add.endorsement + 0.5) / 8 - new ChannelRelay(nodeParams, register, channels, r, upstream, confidence, context).start() + reputationRecorder_opt match { + case Some(reputationRecorder) => + reputationRecorder ! GetConfidence(context.messageAdapter[ReputationRecorder.Confidence](confidence => WrappedConfidence(confidence.value)), originNode, r.add.endorsement, relayId, r.relayFeeMsat) + case None => + val confidence = (r.add.endorsement + 0.5) / 8 + context.self ! WrappedConfidence(confidence) + } + Behaviors.receiveMessagePartial { + case WrappedConfidence(confidence) => + new ChannelRelay(nodeParams, register, reputationRecorder_opt, channels, r, upstream, confidence, context, relayId).start() + } } } @@ -114,11 +128,13 @@ object ChannelRelay { */ class ChannelRelay private(nodeParams: NodeParams, register: ActorRef, + reputationRecorder_opt: Option[typed.ActorRef[ReputationRecorder.ChannelRelayCommand]], channels: Map[ByteVector32, Relayer.OutgoingChannel], r: IncomingPaymentPacket.ChannelRelayPacket, upstream: Upstream.Hot.Channel, confidence: Double, - context: ActorContext[ChannelRelay.Command]) { + context: ActorContext[ChannelRelay.Command], + relayId: UUID) { import ChannelRelay._ @@ -184,6 +200,7 @@ class ChannelRelay private(nodeParams: NodeParams, case RelayFailure(cmdFail) => Metrics.recordPaymentRelayFailed(Tags.FailureType(cmdFail), Tags.RelayType.Channel) context.log.info("rejecting htlc reason={}", cmdFail.reason) + reputationRecorder_opt.foreach(_ ! CancelRelay(upstream.receivedFrom, r.add.endorsement, relayId)) safeSendAndStop(r.add.channelId, cmdFail) case RelayNeedsFunding(nextNodeId, cmdFail) => val cmd = Peer.ProposeOnTheFlyFunding(onTheFlyFundingResponseAdapter, r.amountToForward, r.add.paymentHash, r.outgoingCltv, r.nextPacket, nextBlindingKey_opt, upstream) @@ -203,6 +220,7 @@ class ChannelRelay private(nodeParams: NodeParams, context.log.warn(s"couldn't resolve downstream channel $channelId, failing htlc #${upstream.add.id}") val cmdFail = CMD_FAIL_HTLC(upstream.add.id, Right(UnknownNextPeer()), commit = true) Metrics.recordPaymentRelayFailed(Tags.FailureType(cmdFail), Tags.RelayType.Channel) + reputationRecorder_opt.foreach(_ ! CancelRelay(upstream.receivedFrom, r.add.endorsement, relayId)) safeSendAndStop(upstream.add.channelId, cmdFail) case WrappedAddResponse(addFailed: RES_ADD_FAILED[_]) => @@ -413,9 +431,11 @@ class ChannelRelay private(nodeParams: NodeParams, featureOk && liquidityIssue && relayParamsOk } - private def recordRelayDuration(isSuccess: Boolean): Unit = + private def recordRelayDuration(isSuccess: Boolean): Unit = { + reputationRecorder_opt.foreach(_ ! RecordResult(upstream.receivedFrom, r.add.endorsement, relayId, isSuccess)) Metrics.RelayedPaymentDuration .withTag(Tags.Relay, Tags.RelayType.Channel) .withTag(Tags.Success, isSuccess) .record((TimestampMilli.now() - upstream.receivedAt).toMillis, TimeUnit.MILLISECONDS) + } } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/ChannelRelayer.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/ChannelRelayer.scala index 8c635df706..590ca16e37 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/ChannelRelayer.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/ChannelRelayer.scala @@ -16,14 +16,15 @@ package fr.acinq.eclair.payment.relay -import akka.actor.ActorRef import akka.actor.typed.Behavior import akka.actor.typed.eventstream.EventStream import akka.actor.typed.scaladsl.Behaviors +import akka.actor.{ActorRef, typed} import fr.acinq.bitcoin.scalacompat.ByteVector32 import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey import fr.acinq.eclair.channel._ import fr.acinq.eclair.payment.IncomingPaymentPacket +import fr.acinq.eclair.reputation.ReputationRecorder import fr.acinq.eclair.{Logs, NodeParams, ShortChannelId, SubscriptionsComplete} import java.util.UUID @@ -58,6 +59,7 @@ object ChannelRelayer { def apply(nodeParams: NodeParams, register: ActorRef, + reputationRecorder_opt: Option[typed.ActorRef[ReputationRecorder.ChannelRelayCommand]], channels: Map[ByteVector32, Relayer.OutgoingChannel] = Map.empty, scid2channels: Map[ShortChannelId, ByteVector32] = Map.empty, node2channels: mutable.MultiDict[PublicKey, ByteVector32] = mutable.MultiDict.empty): Behavior[Command] = @@ -82,7 +84,7 @@ object ChannelRelayer { case None => Map.empty } context.log.debug(s"spawning a new handler with relayId=$relayId to nextNodeId={} with channels={}", nextNodeId_opt.getOrElse(""), nextChannels.keys.mkString(",")) - context.spawn(ChannelRelay.apply(nodeParams, register, nextChannels, originNode, relayId, channelRelayPacket), name = relayId.toString) + context.spawn(ChannelRelay.apply(nodeParams, register, reputationRecorder_opt, nextChannels, originNode, relayId, channelRelayPacket), name = relayId.toString) Behaviors.same case GetOutgoingChannels(replyTo, Relayer.GetOutgoingChannels(enabledOnly)) => @@ -103,14 +105,14 @@ object ChannelRelayer { context.log.debug("adding mappings={} to channelId={}", mappings.keys.mkString(","), channelId) val scid2channels1 = scid2channels ++ mappings val node2channels1 = node2channels.addOne(remoteNodeId, channelId) - apply(nodeParams, register, channels1, scid2channels1, node2channels1) + apply(nodeParams, register, reputationRecorder_opt, channels1, scid2channels1, node2channels1) case WrappedLocalChannelDown(LocalChannelDown(_, channelId, shortIds, remoteNodeId)) => context.log.debug(s"removed local channel info for channelId=$channelId localAlias=${shortIds.localAlias}") val channels1 = channels - channelId val scid2Channels1 = scid2channels - shortIds.localAlias -- shortIds.real.toOption val node2channels1 = node2channels.subtractOne(remoteNodeId, channelId) - apply(nodeParams, register, channels1, scid2Channels1, node2channels1) + apply(nodeParams, register, reputationRecorder_opt, channels1, scid2Channels1, node2channels1) case WrappedAvailableBalanceChanged(AvailableBalanceChanged(_, channelId, shortIds, commitments)) => val channels1 = channels.get(channelId) match { @@ -119,7 +121,7 @@ object ChannelRelayer { channels + (channelId -> c.copy(commitments = commitments)) case None => channels // we only consider the balance if we have the channel_update } - apply(nodeParams, register, channels1, scid2channels, node2channels) + apply(nodeParams, register, reputationRecorder_opt, channels1, scid2channels, node2channels) } } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/NodeRelay.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/NodeRelay.scala index d3623a5177..472ad705dc 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/NodeRelay.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/NodeRelay.scala @@ -39,6 +39,8 @@ import fr.acinq.eclair.payment.send.PaymentInitiator.SendPaymentConfig import fr.acinq.eclair.payment.send.PaymentLifecycle.SendPaymentToNode import fr.acinq.eclair.payment.send._ import fr.acinq.eclair.router.Router.{ChannelHop, HopRelayParams, Route, RouteParams} +import fr.acinq.eclair.reputation.ReputationRecorder +import fr.acinq.eclair.router.Router.RouteParams import fr.acinq.eclair.router.{BalanceTooLow, RouteNotFound} import fr.acinq.eclair.wire.protocol.PaymentOnion.IntermediatePayload import fr.acinq.eclair.wire.protocol._ @@ -67,6 +69,7 @@ object NodeRelay { private case class WrappedPeerReadyResult(result: PeerReadyNotifier.Result) extends Command private case class WrappedResolvedPaths(resolved: Seq[ResolvedPath]) extends Command private case class WrappedOnTheFlyFundingResponse(result: Peer.ProposeOnTheFlyFundingResponse) extends Command + private case class WrappedConfidence(confidence: Double) extends Command // @formatter:on trait OutgoingPaymentFactory { @@ -88,6 +91,7 @@ object NodeRelay { def apply(nodeParams: NodeParams, parent: typed.ActorRef[NodeRelayer.Command], register: ActorRef, + reputationRecorder_opt: Option[typed.ActorRef[ReputationRecorder.TrampolineRelayCommand]], relayId: UUID, nodeRelayPacket: NodeRelayPacket, outgoingPaymentFactory: OutgoingPaymentFactory, @@ -110,7 +114,7 @@ object NodeRelay { case IncomingPaymentPacket.RelayToTrampolinePacket(_, _, _, nextPacket) => Some(nextPacket) case _: IncomingPaymentPacket.RelayToBlindedPathsPacket => None } - new NodeRelay(nodeParams, parent, register, relayId, paymentHash, nodeRelayPacket.outerPayload.paymentSecret, context, outgoingPaymentFactory, router) + new NodeRelay(nodeParams, parent, register, reputationRecorder_opt, relayId, paymentHash, nodeRelayPacket.outerPayload.paymentSecret, context, outgoingPaymentFactory, router) .receiving(Queue.empty, nodeRelayPacket.innerPayload, nextPacket_opt, incomingPaymentHandler) } } @@ -211,6 +215,7 @@ object NodeRelay { class NodeRelay private(nodeParams: NodeParams, parent: akka.actor.typed.ActorRef[NodeRelayer.Command], register: ActorRef, + reputationRecorder_opt: Option[typed.ActorRef[ReputationRecorder.TrampolineRelayCommand]], relayId: UUID, paymentHash: ByteVector32, paymentSecret: ByteVector32, @@ -328,26 +333,38 @@ class NodeRelay private(nodeParams: NodeParams, /** Relay the payment to the next identified node: this is similar to sending an outgoing payment. */ private def relay(upstream: Upstream.Hot.Trampoline, recipient: Recipient, walletNodeId_opt: Option[PublicKey], recipientFeatures_opt: Option[Features[InitFeature]], payloadOut: IntermediatePayload.NodeRelay, packetOut_opt: Option[OnionRoutingPacket]): Behavior[Command] = { context.log.debug("relaying trampoline payment (amountIn={} expiryIn={} amountOut={} expiryOut={})", upstream.amountIn, upstream.expiryIn, payloadOut.amountToForward, payloadOut.outgoingCltv) - val confidence = (upstream.received.map(_.add.endorsement).min + 0.5) / 8 // We only make one try when it's a direct payment to a wallet. val maxPaymentAttempts = if (walletNodeId_opt.isDefined) 1 else nodeParams.maxPaymentAttempts - val paymentCfg = SendPaymentConfig(relayId, relayId, None, paymentHash, recipient.nodeId, upstream, None, None, storeInDb = false, publishEvent = false, recordPathFindingMetrics = true, confidence) - val routeParams = computeRouteParams(nodeParams, upstream.amountIn, upstream.expiryIn, payloadOut.amountToForward, payloadOut.outgoingCltv) - // If the next node is using trampoline, we assume that they support MPP. - val useMultiPart = recipient.features.hasFeature(Features.BasicMultiPartPayment) || packetOut_opt.nonEmpty - val payFsmAdapters = { - context.messageAdapter[PreimageReceived](WrappedPreimageReceived) - context.messageAdapter[PaymentSent](WrappedPaymentSent) - context.messageAdapter[PaymentFailed](WrappedPaymentFailed) - }.toClassic - val payment = if (useMultiPart) { - SendMultiPartPayment(payFsmAdapters, recipient, maxPaymentAttempts, routeParams) - } else { - SendPaymentToNode(payFsmAdapters, recipient, maxPaymentAttempts, routeParams) + val totalFee = upstream.amountIn - payloadOut.amountToForward + val fees = upstream.received.foldLeft(Map.empty[ReputationRecorder.PeerEndorsement, MilliSatoshi])((fees, r) => + fees.updatedWith(ReputationRecorder.PeerEndorsement(r.receivedFrom, r.add.endorsement))(fee => + Some(fee.getOrElse(MilliSatoshi(0)) + r.add.amountMsat * totalFee.toLong / upstream.amountIn.toLong))) + reputationRecorder_opt match { + case Some(reputationRecorder) => reputationRecorder ! ReputationRecorder.GetTrampolineConfidence(context.messageAdapter[ReputationRecorder.Confidence](confidence => WrappedConfidence(confidence.value)), fees, relayId) + case None => context.self ! WrappedConfidence((upstream.received.map(_.add.endorsement).min + 0.5) / 8) + } + Behaviors.receiveMessagePartial { + rejectExtraHtlcPartialFunction orElse { + case WrappedConfidence(confidence) => + val paymentCfg = SendPaymentConfig(relayId, relayId, None, paymentHash, recipient.nodeId, upstream, None, None, storeInDb = false, publishEvent = false, recordPathFindingMetrics = true, confidence) + val routeParams = computeRouteParams(nodeParams, upstream.amountIn, upstream.expiryIn, payloadOut.amountToForward, payloadOut.outgoingCltv) + // If the next node is using trampoline, we assume that they support MPP. + val useMultiPart = recipient.features.hasFeature(Features.BasicMultiPartPayment) || packetOut_opt.nonEmpty + val payFsmAdapters = { + context.messageAdapter[PreimageReceived](WrappedPreimageReceived) + context.messageAdapter[PaymentSent](WrappedPaymentSent) + context.messageAdapter[PaymentFailed](WrappedPaymentFailed) + }.toClassic + val payment = if (useMultiPart) { + SendMultiPartPayment(payFsmAdapters, recipient, maxPaymentAttempts, routeParams) + } else { + SendPaymentToNode(payFsmAdapters, recipient, maxPaymentAttempts, routeParams) + } + val payFSM = outgoingPaymentFactory.spawnOutgoingPayFSM(context, paymentCfg, useMultiPart) + payFSM ! payment + sending(upstream, recipient, recipientFeatures_opt, payloadOut, TimestampMilli.now(), fulfilledUpstream = false) + } } - val payFSM = outgoingPaymentFactory.spawnOutgoingPayFSM(context, paymentCfg, useMultiPart) - payFSM ! payment - sending(upstream, recipient, recipientFeatures_opt, payloadOut, TimestampMilli.now(), fulfilledUpstream = false) } /** @@ -379,10 +396,16 @@ class NodeRelay private(nodeParams: NodeParams, case WrappedPaymentSent(paymentSent) => context.log.debug("trampoline payment fully resolved downstream") success(upstream, fulfilledUpstream, paymentSent) + val totalFee = upstream.amountIn - paymentSent.amountWithFees + val fees = upstream.received.foldLeft(Map.empty[ReputationRecorder.PeerEndorsement, MilliSatoshi])((fees, r) => + fees.updatedWith(ReputationRecorder.PeerEndorsement(r.receivedFrom, r.add.endorsement))(fee => + Some(fee.getOrElse(MilliSatoshi(0)) + r.add.amountMsat * totalFee.toLong / upstream.amountIn.toLong))) + reputationRecorder_opt.foreach(_ ! ReputationRecorder.RecordTrampolineSuccess(fees, relayId)) recordRelayDuration(startedAt, isSuccess = true) stopping() case _: WrappedPaymentFailed if fulfilledUpstream => context.log.warn("trampoline payment failed downstream but was fulfilled upstream") + reputationRecorder_opt.foreach(_ ! ReputationRecorder.RecordTrampolineFailure(upstream.received.map(r => ReputationRecorder.PeerEndorsement(r.receivedFrom, r.add.endorsement)).toSet, relayId)) recordRelayDuration(startedAt, isSuccess = true) stopping() case WrappedPaymentFailed(PaymentFailed(_, _, failures, _)) => @@ -392,6 +415,7 @@ class NodeRelay private(nodeParams: NodeParams, attemptOnTheFlyFunding(upstream, walletNodeId, recipient, nextPayload, failures, startedAt) case _ => rejectPayment(upstream, translateError(nodeParams, failures, upstream, nextPayload)) + reputationRecorder_opt.foreach(_ ! ReputationRecorder.RecordTrampolineFailure(upstream.received.map(r => ReputationRecorder.PeerEndorsement(r.receivedFrom, r.add.endorsement)).toSet, relayId)) recordRelayDuration(startedAt, isSuccess = false) stopping() } @@ -497,7 +521,7 @@ class NodeRelay private(nodeParams: NodeParams, context.system.eventStream ! EventStream.Publish(TrampolinePaymentRelayed(paymentHash, incoming, outgoing, paymentSent.recipientNodeId, paymentSent.recipientAmount)) } - private def recordRelayDuration(startedAt: TimestampMilli, isSuccess: Boolean): Unit = + private def recordRelayDuration(startedAt: TimestampMilli, isSuccess: Boolean): Unit = // TODO: always record reputation and duration before stopping, everything in one function Metrics.RelayedPaymentDuration .withTag(Tags.Relay, Tags.RelayType.Trampoline) .withTag(Tags.Success, isSuccess) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/NodeRelayer.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/NodeRelayer.scala index 75bb545c89..4bff50e476 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/NodeRelayer.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/NodeRelayer.scala @@ -21,6 +21,7 @@ import akka.actor.typed.{ActorRef, Behavior} import fr.acinq.bitcoin.scalacompat.ByteVector32 import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey import fr.acinq.eclair.payment._ +import fr.acinq.eclair.reputation.ReputationRecorder import fr.acinq.eclair.{Logs, NodeParams} import java.util.UUID @@ -57,7 +58,12 @@ object NodeRelayer { * NB: the payment secret used here is different from the invoice's payment secret and ensures we can * group together HTLCs that the previous trampoline node sent in the same MPP. */ - def apply(nodeParams: NodeParams, register: akka.actor.ActorRef, outgoingPaymentFactory: NodeRelay.OutgoingPaymentFactory, router: akka.actor.ActorRef, children: Map[PaymentKey, ActorRef[NodeRelay.Command]] = Map.empty): Behavior[Command] = + def apply(nodeParams: NodeParams, + register: akka.actor.ActorRef, + reputationRecorder_opt: Option[ActorRef[ReputationRecorder.TrampolineRelayCommand]], + outgoingPaymentFactory: NodeRelay.OutgoingPaymentFactory, + router: akka.actor.ActorRef, + children: Map[PaymentKey, ActorRef[NodeRelay.Command]] = Map.empty): Behavior[Command] = Behaviors.setup { context => Behaviors.withMdc(Logs.mdc(category_opt = Some(Logs.LogCategory.PAYMENT)), mdc) { Behaviors.receiveMessage { @@ -72,15 +78,15 @@ object NodeRelayer { case None => val relayId = UUID.randomUUID() context.log.debug(s"spawning a new handler with relayId=$relayId") - val handler = context.spawn(NodeRelay.apply(nodeParams, context.self, register, relayId, nodeRelayPacket, outgoingPaymentFactory, router), relayId.toString) + val handler = context.spawn(NodeRelay.apply(nodeParams, context.self, register, reputationRecorder_opt, relayId, nodeRelayPacket, outgoingPaymentFactory, router), relayId.toString) context.log.debug("forwarding incoming htlc #{} from channel {} to new handler", htlcIn.id, htlcIn.channelId) handler ! NodeRelay.Relay(nodeRelayPacket, originNode) - apply(nodeParams, register, outgoingPaymentFactory, router, children + (childKey -> handler)) + apply(nodeParams, register, reputationRecorder_opt, outgoingPaymentFactory, router, children + (childKey -> handler)) } case RelayComplete(childHandler, paymentHash, paymentSecret) => // we do a back-and-forth between parent and child before stopping the child to prevent a race condition childHandler ! NodeRelay.Stop - apply(nodeParams, register, outgoingPaymentFactory, router, children - PaymentKey(paymentHash, paymentSecret)) + apply(nodeParams, register, reputationRecorder_opt, outgoingPaymentFactory, router, children - PaymentKey(paymentHash, paymentSecret)) case GetPendingPayments(replyTo) => replyTo ! children Behaviors.same diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/Relayer.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/Relayer.scala index 1600e185cd..8bdc451f91 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/Relayer.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/Relayer.scala @@ -28,6 +28,7 @@ import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey import fr.acinq.eclair.channel._ import fr.acinq.eclair.db.PendingCommandsDb import fr.acinq.eclair.payment._ +import fr.acinq.eclair.reputation.{Reputation, ReputationRecorder} import fr.acinq.eclair.wire.protocol._ import fr.acinq.eclair.{CltvExpiryDelta, Logs, MilliSatoshi, NodeParams} import grizzled.slf4j.Logging @@ -49,7 +50,7 @@ import scala.util.Random * It also receives channel HTLC events (fulfill / failed) and relays those to the appropriate handlers. * It also maintains an up-to-date view of local channel balances. */ -class Relayer(nodeParams: NodeParams, router: ActorRef, register: ActorRef, paymentHandler: ActorRef, initialized: Option[Promise[Done]] = None) extends Actor with DiagnosticActorLogging { +class Relayer(nodeParams: NodeParams, router: ActorRef, register: ActorRef, paymentHandler: ActorRef, reputationRecorder_opt: Option[typed.ActorRef[ReputationRecorder.Command]], initialized: Option[Promise[Done]] = None) extends Actor with DiagnosticActorLogging { import Relayer._ @@ -57,8 +58,8 @@ class Relayer(nodeParams: NodeParams, router: ActorRef, register: ActorRef, paym implicit def implicitLog: LoggingAdapter = log private val postRestartCleaner = context.actorOf(PostRestartHtlcCleaner.props(nodeParams, register, initialized), "post-restart-htlc-cleaner") - private val channelRelayer = context.spawn(Behaviors.supervise(ChannelRelayer(nodeParams, register)).onFailure(SupervisorStrategy.resume), "channel-relayer") - private val nodeRelayer = context.spawn(Behaviors.supervise(NodeRelayer(nodeParams, register, NodeRelay.SimpleOutgoingPaymentFactory(nodeParams, router, register), router)).onFailure(SupervisorStrategy.resume), name = "node-relayer") + private val channelRelayer = context.spawn(Behaviors.supervise(ChannelRelayer(nodeParams, register, reputationRecorder_opt)).onFailure(SupervisorStrategy.resume), "channel-relayer") + private val nodeRelayer = context.spawn(Behaviors.supervise(NodeRelayer(nodeParams, register, reputationRecorder_opt, NodeRelay.SimpleOutgoingPaymentFactory(nodeParams, router, register), router)).onFailure(SupervisorStrategy.resume), name = "node-relayer") def receive: Receive = { case init: PostRestartHtlcCleaner.Init => postRestartCleaner forward init @@ -125,8 +126,8 @@ class Relayer(nodeParams: NodeParams, router: ActorRef, register: ActorRef, paym object Relayer extends Logging { - def props(nodeParams: NodeParams, router: ActorRef, register: ActorRef, paymentHandler: ActorRef, initialized: Option[Promise[Done]] = None): Props = - Props(new Relayer(nodeParams, router, register, paymentHandler, initialized)) + def props(nodeParams: NodeParams, router: ActorRef, register: ActorRef, paymentHandler: ActorRef, reputationRecorder_opt: Option[typed.ActorRef[ReputationRecorder.Command]], initialized: Option[Promise[Done]] = None): Props = + Props(new Relayer(nodeParams, router, register, paymentHandler, reputationRecorder_opt, initialized)) // @formatter:off case class RelayFees(feeBase: MilliSatoshi, feeProportionalMillionths: Long) { @@ -140,7 +141,8 @@ object Relayer extends Logging { privateChannelFees: RelayFees, minTrampolineFees: RelayFees, enforcementDelay: FiniteDuration, - asyncPaymentsParams: AsyncPaymentsParams) { + asyncPaymentsParams: AsyncPaymentsParams, + peerReputationConfig: Reputation.Config) { def defaultFees(announceChannel: Boolean): RelayFees = { if (announceChannel) { publicChannelFees diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/reputation/Reputation.scala b/eclair-core/src/main/scala/fr/acinq/eclair/reputation/Reputation.scala new file mode 100644 index 0000000000..f74c0aca67 --- /dev/null +++ b/eclair-core/src/main/scala/fr/acinq/eclair/reputation/Reputation.scala @@ -0,0 +1,95 @@ +/* + * Copyright 2023 ACINQ SAS + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package fr.acinq.eclair.reputation + +import fr.acinq.eclair.{MilliSatoshi, TimestampMilli} + +import java.util.UUID +import scala.concurrent.duration.FiniteDuration + +/** + * Created by thomash on 21/07/2023. + */ + +/** + * Local reputation for a given incoming node, that should be track for each incoming endorsement level. + * + * @param pastWeight How much fees we would have collected in the past if all payments had succeeded (exponential moving average). + * @param pastScore How much fees we have collected in the past (exponential moving average). + * @param lastSettlementAt Timestamp of the last recorded payment settlement. + * @param pending Set of pending payments (payments may contain multiple HTLCs when using trampoline). + * @param halfLife Half life for the exponential moving average. + * @param maxRelayDuration Duration after which payments are penalized for staying pending too long. + * @param pendingMultiplier How much to penalize pending payments. + */ +case class Reputation(pastWeight: Double, pastScore: Double, lastSettlementAt: TimestampMilli, pending: Map[UUID, Reputation.PendingPayment], halfLife: FiniteDuration, maxRelayDuration: FiniteDuration, pendingMultiplier: Double) { + private def decay(now: TimestampMilli): Double = scala.math.pow(0.5, (now - lastSettlementAt) / halfLife) + + private def pendingWeight(now: TimestampMilli): Double = pending.values.map(_.weight(now, maxRelayDuration, pendingMultiplier)).sum + + /** + * Register a payment to relay and estimate the confidence that it will succeed. + * + * @return (updated reputation, confidence) + */ + def attempt(relayId: UUID, fee: MilliSatoshi, now: TimestampMilli = TimestampMilli.now()): (Reputation, Double) = { + val d = decay(now) + val newReputation = copy(pending = pending + (relayId -> Reputation.PendingPayment(fee, now))) + val confidence = d * pastScore / (d * pastWeight + newReputation.pendingWeight(now)) + (newReputation, confidence) + } + + /** + * Mark a previously registered payment as failed without trying to relay it (usually because its confidence was too low). + * + * @return updated reputation + */ + def cancel(relayId: UUID): Reputation = copy(pending = pending - relayId) + + /** + * When a payment is settled, we record whether it succeeded and how long it took. + * + * @param feeOverride When relaying trampoline payments, the actual fee is only known when the payment succeeds. This + * is used instead of the fee upper bound that was known when first attempting the relay. + * @return updated reputation + */ + def record(relayId: UUID, isSuccess: Boolean, feeOverride: Option[MilliSatoshi] = None, now: TimestampMilli = TimestampMilli.now()): Reputation = { + pending.get(relayId) match { + case Some(p) => + val d = decay(now) + val p1 = p.copy(fee = feeOverride.getOrElse(p.fee)) + val newWeight = d * pastWeight + p1.weight(now, maxRelayDuration, 1.0) + val newScore = d * pastScore + (if (isSuccess) p1.fee.toLong.toDouble else 0) + Reputation(newWeight, newScore, now, pending - relayId, halfLife, maxRelayDuration, pendingMultiplier) + case None => this + } + } +} + +object Reputation { + /** We're relaying that payment and are waiting for it to settle. */ + case class PendingPayment(fee: MilliSatoshi, startedAt: TimestampMilli) { + def weight(now: TimestampMilli, minDuration: FiniteDuration, multiplier: Double): Double = { + val duration = now - startedAt + fee.toLong.toDouble * (duration / minDuration).max(multiplier) + } + } + + case class Config(enabled: Boolean, halfLife: FiniteDuration, maxRelayDuration: FiniteDuration, pendingMultiplier: Double) + + def init(config: Config): Reputation = Reputation(0.0, 0.0, TimestampMilli.min, Map.empty, config.halfLife, config.maxRelayDuration, config.pendingMultiplier) +} diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/reputation/ReputationRecorder.scala b/eclair-core/src/main/scala/fr/acinq/eclair/reputation/ReputationRecorder.scala new file mode 100644 index 0000000000..77982ebfbe --- /dev/null +++ b/eclair-core/src/main/scala/fr/acinq/eclair/reputation/ReputationRecorder.scala @@ -0,0 +1,90 @@ +/* + * Copyright 2023 ACINQ SAS + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package fr.acinq.eclair.reputation + +import akka.actor.typed.scaladsl.Behaviors +import akka.actor.typed.{ActorRef, Behavior} +import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey +import fr.acinq.eclair.MilliSatoshi + +import java.util.UUID + +/** + * Created by thomash on 21/07/2023. + */ + +object ReputationRecorder { + // @formatter:off + sealed trait Command + + sealed trait ChannelRelayCommand extends Command + case class GetConfidence(replyTo: ActorRef[Confidence], originNode: PublicKey, endorsement: Int, relayId: UUID, fee: MilliSatoshi) extends ChannelRelayCommand + case class CancelRelay(originNode: PublicKey, endorsement: Int, relayId: UUID) extends ChannelRelayCommand + case class RecordResult(originNode: PublicKey, endorsement: Int, relayId: UUID, isSuccess: Boolean) extends ChannelRelayCommand + + sealed trait TrampolineRelayCommand extends Command + case class GetTrampolineConfidence(replyTo: ActorRef[Confidence], fees: Map[PeerEndorsement, MilliSatoshi], relayId: UUID) extends TrampolineRelayCommand + case class RecordTrampolineFailure(upstream: Set[PeerEndorsement], relayId: UUID) extends TrampolineRelayCommand + case class RecordTrampolineSuccess(fees: Map[PeerEndorsement, MilliSatoshi], relayId: UUID) extends TrampolineRelayCommand + // @formatter:on + + /** + * @param nodeId nodeId of the upstream peer. + * @param endorsement endorsement value set by the upstream peer in the HTLC we received. + */ + case class PeerEndorsement(nodeId: PublicKey, endorsement: Int) + + /** Confidence that the outgoing HTLC will succeed. */ + case class Confidence(value: Double) + + def apply(reputationConfig: Reputation.Config, reputations: Map[PeerEndorsement, Reputation]): Behavior[Command] = { + Behaviors.receiveMessage { + case GetConfidence(replyTo, originNode, endorsement, relayId, fee) => + val (updatedReputation, confidence) = reputations.getOrElse(PeerEndorsement(originNode, endorsement), Reputation.init(reputationConfig)).attempt(relayId, fee) + replyTo ! Confidence(confidence) + ReputationRecorder(reputationConfig, reputations.updated(PeerEndorsement(originNode, endorsement), updatedReputation)) + case CancelRelay(originNode, endorsement, relayId) => + val updatedReputation = reputations.getOrElse(PeerEndorsement(originNode, endorsement), Reputation.init(reputationConfig)).cancel(relayId) + ReputationRecorder(reputationConfig, reputations.updated(PeerEndorsement(originNode, endorsement), updatedReputation)) + case RecordResult(originNode, endorsement, relayId, isSuccess) => + val updatedReputation = reputations.getOrElse(PeerEndorsement(originNode, endorsement), Reputation.init(reputationConfig)).record(relayId, isSuccess) + ReputationRecorder(reputationConfig, reputations.updated(PeerEndorsement(originNode, endorsement), updatedReputation)) + case GetTrampolineConfidence(replyTo, fees, relayId) => + val (confidence, updatedReputations) = fees.foldLeft((1.0, reputations)) { + case ((c, r), (peerEndorsement, fee)) => + val (updatedReputation, confidence) = reputations.getOrElse(peerEndorsement, Reputation.init(reputationConfig)).attempt(relayId, fee) + (c.min(confidence), r.updated(peerEndorsement, updatedReputation)) + } + replyTo ! Confidence(confidence) + ReputationRecorder(reputationConfig, updatedReputations) + case RecordTrampolineFailure(keys, relayId) => + val updatedReputations = keys.foldLeft(reputations) { + case (r, peerEndorsement) => + val updatedReputation = reputations.getOrElse(peerEndorsement, Reputation.init(reputationConfig)).record(relayId, isSuccess = false) + r.updated(peerEndorsement, updatedReputation) + } + ReputationRecorder(reputationConfig, updatedReputations) + case RecordTrampolineSuccess(fees, relayId) => + val updatedReputations = fees.foldLeft(reputations) { + case (r, (peerEndorsement, fee)) => + val updatedReputation = reputations.getOrElse(peerEndorsement, Reputation.init(reputationConfig)).record(relayId, isSuccess = true, Some(fee)) + r.updated(peerEndorsement, updatedReputation) + } + ReputationRecorder(reputationConfig, updatedReputations) + } + } +} diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/TestConstants.scala b/eclair-core/src/test/scala/fr/acinq/eclair/TestConstants.scala index b48c937552..9667b537cd 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/TestConstants.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/TestConstants.scala @@ -28,6 +28,7 @@ import fr.acinq.eclair.io.{OpenChannelInterceptor, PeerConnection, PeerReadyNoti import fr.acinq.eclair.message.OnionMessages.OnionMessageConfig import fr.acinq.eclair.payment.relay.OnTheFlyFunding import fr.acinq.eclair.payment.relay.Relayer.{AsyncPaymentsParams, RelayFees, RelayParams} +import fr.acinq.eclair.reputation.Reputation import fr.acinq.eclair.router.Graph.{MessagePath, WeightRatios} import fr.acinq.eclair.router.PathFindingExperimentConf import fr.acinq.eclair.router.Router._ @@ -168,7 +169,9 @@ object TestConstants { feeBase = 548000 msat, feeProportionalMillionths = 30), enforcementDelay = 10 minutes, - asyncPaymentsParams = AsyncPaymentsParams(1008, CltvExpiryDelta(144))), + asyncPaymentsParams = AsyncPaymentsParams(1008, CltvExpiryDelta(144)), + peerReputationConfig = Reputation.Config(enabled = false, 1 day, 10 seconds, 100), + ), db = TestDatabases.inMemoryDb(), autoReconnect = false, initialRandomReconnectDelay = 5 seconds, @@ -341,7 +344,9 @@ object TestConstants { feeBase = 548000 msat, feeProportionalMillionths = 30), enforcementDelay = 10 minutes, - asyncPaymentsParams = AsyncPaymentsParams(1008, CltvExpiryDelta(144))), + asyncPaymentsParams = AsyncPaymentsParams(1008, CltvExpiryDelta(144)), + peerReputationConfig = Reputation.Config(enabled = false, 2 day, 20 seconds, 200), + ), db = TestDatabases.inMemoryDb(), autoReconnect = false, initialRandomReconnectDelay = 5 seconds, 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 506f8ca31a..45a29cd80b 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 @@ -66,8 +66,8 @@ class FuzzySpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with Channe val bobRegister = system.actorOf(Props(new TestRegister())) val alicePaymentHandler = system.actorOf(Props(new PaymentHandler(aliceParams, aliceRegister, TestProbe().ref))) val bobPaymentHandler = system.actorOf(Props(new PaymentHandler(bobParams, bobRegister, TestProbe().ref))) - val aliceRelayer = system.actorOf(Relayer.props(aliceParams, TestProbe().ref, aliceRegister, alicePaymentHandler)) - val bobRelayer = system.actorOf(Relayer.props(bobParams, TestProbe().ref, bobRegister, bobPaymentHandler)) + val aliceRelayer = system.actorOf(Relayer.props(aliceParams, TestProbe().ref, aliceRegister, alicePaymentHandler, None)) + val bobRelayer = system.actorOf(Relayer.props(bobParams, TestProbe().ref, bobRegister, bobPaymentHandler, None)) val wallet = new DummyOnChainWallet() val alice: TestFSMRef[ChannelState, ChannelData, Channel] = TestFSMRef(new Channel(aliceParams, wallet, bobParams.nodeId, alice2blockchain.ref, aliceRelayer, FakeTxPublisherFactory(alice2blockchain)), alicePeer.ref) val bob: TestFSMRef[ChannelState, ChannelData, Channel] = TestFSMRef(new Channel(bobParams, wallet, aliceParams.nodeId, bob2blockchain.ref, bobRelayer, FakeTxPublisherFactory(bob2blockchain)), bobPeer.ref) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/fixtures/MinimalNodeFixture.scala b/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/fixtures/MinimalNodeFixture.scala index abbfb5e3f2..2773c54292 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/fixtures/MinimalNodeFixture.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/fixtures/MinimalNodeFixture.scala @@ -28,6 +28,7 @@ import fr.acinq.eclair.payment.offer.OfferManager import fr.acinq.eclair.payment.receive.{MultiPartHandler, PaymentHandler} import fr.acinq.eclair.payment.relay.{ChannelRelayer, PostRestartHtlcCleaner, Relayer} import fr.acinq.eclair.payment.send.PaymentInitiator +import fr.acinq.eclair.reputation.ReputationRecorder import fr.acinq.eclair.router.Router import fr.acinq.eclair.wire.protocol.IPAddress import fr.acinq.eclair.{BlockHeight, MilliSatoshi, NodeParams, SubscriptionsComplete, TestBitcoinCoreClient, TestDatabases} @@ -95,7 +96,7 @@ object MinimalNodeFixture extends Assertions with Eventually with IntegrationPat val router = system.actorOf(Router.props(nodeParams, watcherTyped), "router") val offerManager = system.spawn(OfferManager(nodeParams, router, 1 minute), "offer-manager") val paymentHandler = system.actorOf(PaymentHandler.props(nodeParams, register, offerManager), "payment-handler") - val relayer = system.actorOf(Relayer.props(nodeParams, router, register, paymentHandler), "relayer") + val relayer = system.actorOf(Relayer.props(nodeParams, router, register, paymentHandler, None), "relayer") val txPublisherFactory = Channel.SimpleTxPublisherFactory(nodeParams, bitcoinClient) val channelFactory = Peer.SimpleChannelFactory(nodeParams, watcherTyped, relayer, wallet, txPublisherFactory) val pendingChannelsRateLimiter = system.spawnAnonymous(Behaviors.supervise(PendingChannelsRateLimiter(nodeParams, router.toTyped, Seq())).onFailure(typed.SupervisorStrategy.resume)) 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 f2e4d12e90..42cd0486ab 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 @@ -57,7 +57,7 @@ class PostRestartHtlcCleanerSpec extends TestKitBaseClass with FixtureAnyFunSuit case class FixtureParam(nodeParams: NodeParams, register: TestProbe, sender: TestProbe, eventListener: TestProbe) { def createRelayer(nodeParams1: NodeParams): (ActorRef, ActorRef) = { - val relayer = system.actorOf(Relayer.props(nodeParams1, TestProbe().ref, register.ref, TestProbe().ref)) + val relayer = system.actorOf(Relayer.props(nodeParams1, TestProbe().ref, register.ref, TestProbe().ref, None)) // we need ensure the post-htlc-restart child actor is initialized sender.send(relayer, Relayer.GetChildActors(sender.ref)) (relayer, sender.expectMsgType[Relayer.ChildActors].postRestartCleaner) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/ChannelRelayerSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/ChannelRelayerSpec.scala index ea2d35b05a..34cbe20f55 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/ChannelRelayerSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/ChannelRelayerSpec.scala @@ -35,6 +35,7 @@ import fr.acinq.eclair.io.{Peer, PeerReadyManager, Switchboard} import fr.acinq.eclair.payment.IncomingPaymentPacket.ChannelRelayPacket import fr.acinq.eclair.payment.relay.ChannelRelayer._ import fr.acinq.eclair.payment.{ChannelPaymentRelayed, IncomingPaymentPacket, PaymentPacketSpec} +import fr.acinq.eclair.reputation.ReputationRecorder import fr.acinq.eclair.router.Announcements import fr.acinq.eclair.wire.protocol.BlindedRouteData.PaymentRelayData import fr.acinq.eclair.wire.protocol.PaymentOnion.IntermediatePayload @@ -56,7 +57,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a val wakeUpTimeout = "wake_up_timeout" val onTheFlyFunding = "on_the_fly_funding" - case class FixtureParam(nodeParams: NodeParams, channelRelayer: typed.ActorRef[ChannelRelayer.Command], register: TestProbe[Any]) { + case class FixtureParam(nodeParams: NodeParams, channelRelayer: typed.ActorRef[ChannelRelayer.Command], register: TestProbe[Any], reputationRecorder: TestProbe[ReputationRecorder.Command]) { def createWakeUpActors(): (TestProbe[PeerReadyManager.Register], TestProbe[Switchboard.GetPeerInfo]) = { val peerReadyManager = TestProbe[PeerReadyManager.Register]() system.receptionist ! Receptionist.Register(PeerReadyManager.PeerReadyManagerServiceKey, peerReadyManager.ref) @@ -78,18 +79,20 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a .modify(_.peerWakeUpConfig.timeout).setToIf(test.tags.contains(wakeUpTimeout))(100 millis) .modify(_.features.activated).usingIf(test.tags.contains(onTheFlyFunding))(_ + (Features.OnTheFlyFunding -> FeatureSupport.Optional)) val register = TestProbe[Any]("register") - val channelRelayer = testKit.spawn(ChannelRelayer.apply(nodeParams, register.ref.toClassic)) + val reputationRecorder = TestProbe[ReputationRecorder.Command]("reputation-recorder") + val channelRelayer = testKit.spawn(ChannelRelayer.apply(nodeParams, register.ref.toClassic, Some(reputationRecorder.ref))) try { - withFixture(test.toNoArgTest(FixtureParam(nodeParams, channelRelayer, register))) + withFixture(test.toNoArgTest(FixtureParam(nodeParams, channelRelayer, register, reputationRecorder))) } finally { testKit.stop(channelRelayer) } } - def expectFwdFail(register: TestProbe[Any], channelId: ByteVector32, cmd: channel.Command): Register.Forward[channel.Command] = { + def expectFwdFail(register: TestProbe[Any], channelId: ByteVector32, cmd: channel.Command, reputationRecorder: TestProbe[ReputationRecorder.Command]): Register.Forward[channel.Command] = { val fwd = register.expectMessageType[Register.Forward[channel.Command]] assert(fwd.message == cmd) assert(fwd.channelId == channelId) + reputationRecorder.expectMessageType[ReputationRecorder.CancelRelay] fwd } @@ -104,6 +107,14 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a fwd } + def setConfidence(f: FixtureParam)(value: Double): Unit = { + import f._ + + val getConfidence = reputationRecorder.expectMessageType[ReputationRecorder.GetConfidence] + assert(getConfidence.originNode == TestConstants.Alice.nodeParams.nodeId) + getConfidence.replyTo ! ReputationRecorder.Confidence(value) + } + def basicRelayTest(f: FixtureParam)(relayPayloadScid: ShortChannelId, lcu: LocalChannelUpdate, success: Boolean): Unit = { import f._ @@ -112,11 +123,12 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a channelRelayer ! WrappedLocalChannelUpdate(lcu) channelRelayer ! Relay(r, TestConstants.Alice.nodeParams.nodeId) + setConfidence(f)(1.0) if (success) { expectFwdAdd(register, lcu.channelId, outgoingAmount, outgoingExpiry, 7) } else { - expectFwdFail(register, r.add.channelId, CMD_FAIL_HTLC(r.add.id, Right(UnknownNextPeer()), commit = true)) + expectFwdFail(register, r.add.channelId, CMD_FAIL_HTLC(r.add.id, Right(UnknownNextPeer()), commit = true), reputationRecorder) } } @@ -161,6 +173,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a val r1 = createValidIncomingPacket(payload1) channelRelayer ! WrappedLocalChannelUpdate(lcu1) channelRelayer ! Relay(r1, TestConstants.Alice.nodeParams.nodeId) + setConfidence(f)(1.0) expectFwdAdd(register, lcu1.channelId, outgoingAmount, outgoingExpiry, 7) // reorg happens @@ -172,9 +185,11 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a // both old and new real scids work channelRelayer ! Relay(r1, TestConstants.Alice.nodeParams.nodeId) + setConfidence(f)(1.0) expectFwdAdd(register, lcu1.channelId, outgoingAmount, outgoingExpiry, 7) // new real scid works channelRelayer ! Relay(r2, TestConstants.Alice.nodeParams.nodeId) + setConfidence(f)(1.0) expectFwdAdd(register, lcu2.channelId, outgoingAmount, outgoingExpiry, 7) } @@ -187,6 +202,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a channelRelayer ! WrappedLocalChannelUpdate(u) channelRelayer ! Relay(r, TestConstants.Alice.nodeParams.nodeId) + setConfidence(f)(1.0) expectFwdAdd(register, channelIds(realScid1), outgoingAmount, outgoingExpiry, 7) } @@ -202,6 +218,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a channelRelayer ! WrappedLocalChannelUpdate(u) channelRelayer ! Relay(r, TestConstants.Alice.nodeParams.nodeId) + setConfidence(f)(1.0) // We try to wake-up the next node. peerReadyManager.expectMessageType[PeerReadyManager.Register].replyTo ! PeerReadyManager.Registered(outgoingNodeId, otherAttempts = 0) @@ -266,7 +283,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a cleanUpWakeUpActors(peerReadyManager, switchboard) // We fail without attempting on-the-fly funding. - expectFwdFail(register, r.add.channelId, CMD_FAIL_MALFORMED_HTLC(r.add.id, Sphinx.hash(r.add.onionRoutingPacket), InvalidOnionBlinding(Sphinx.hash(r.add.onionRoutingPacket)).code, commit = true)) + expectFwdFail(register, r.add.channelId, CMD_FAIL_MALFORMED_HTLC(r.add.id, Sphinx.hash(r.add.onionRoutingPacket), InvalidOnionBlinding(Sphinx.hash(r.add.onionRoutingPacket)).code, commit = true), reputationRecorder) } test("relay blinded payment (on-the-fly funding failed)", Tag(wakeUpEnabled), Tag(onTheFlyFunding)) { f => @@ -291,7 +308,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a val fwdNodeId = register.expectMessageType[ForwardNodeId[Peer.ProposeOnTheFlyFunding]] assert(fwdNodeId.nodeId == outgoingNodeId) fwdNodeId.replyTo ! Register.ForwardNodeIdFailure(fwdNodeId) - expectFwdFail(register, r.add.channelId, CMD_FAIL_MALFORMED_HTLC(r.add.id, Sphinx.hash(r.add.onionRoutingPacket), InvalidOnionBlinding(Sphinx.hash(r.add.onionRoutingPacket)).code, commit = true)) + expectFwdFail(register, r.add.channelId, CMD_FAIL_MALFORMED_HTLC(r.add.id, Sphinx.hash(r.add.onionRoutingPacket), InvalidOnionBlinding(Sphinx.hash(r.add.onionRoutingPacket)).code, commit = true), reputationRecorder) } test("relay blinded payment (on-the-fly funding not attempted)", Tag(wakeUpEnabled), Tag(onTheFlyFunding)) { f => @@ -318,7 +335,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a fwd.message.replyTo ! RES_ADD_FAILED(fwd.message, TooManyAcceptedHtlcs(channelIds(realScid1), 10), Some(u.channelUpdate)) // We fail without attempting on-the-fly funding. - expectFwdFail(register, r.add.channelId, CMD_FAIL_MALFORMED_HTLC(r.add.id, Sphinx.hash(r.add.onionRoutingPacket), InvalidOnionBlinding(Sphinx.hash(r.add.onionRoutingPacket)).code, commit = true)) + expectFwdFail(register, r.add.channelId, CMD_FAIL_MALFORMED_HTLC(r.add.id, Sphinx.hash(r.add.onionRoutingPacket), InvalidOnionBlinding(Sphinx.hash(r.add.onionRoutingPacket)).code, commit = true), reputationRecorder) } test("relay with retries") { f => @@ -336,6 +353,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a channelRelayer ! WrappedLocalChannelUpdate(u2) channelRelayer ! Relay(r, TestConstants.Alice.nodeParams.nodeId) + setConfidence(f)(1.0) // first try val fwd1 = expectFwdAdd(register, channelIds(realScid2), outgoingAmount, outgoingExpiry, 7) @@ -348,7 +366,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a fwd1.message.replyTo ! RES_ADD_FAILED(fwd2.message, HtlcValueTooHighInFlight(channelIds(realScid1), 1000000000 msat, 1516977616 msat), Some(u1.channelUpdate)) // the relayer should give up - expectFwdFail(register, r.add.channelId, CMD_FAIL_HTLC(r.add.id, Right(TemporaryChannelFailure(Some(u1.channelUpdate))), commit = true)) + expectFwdFail(register, r.add.channelId, CMD_FAIL_HTLC(r.add.id, Right(TemporaryChannelFailure(Some(u1.channelUpdate))), commit = true), reputationRecorder) } test("fail to relay when we have no channel_update for the next channel") { f => @@ -358,8 +376,9 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a val r = createValidIncomingPacket(payload) channelRelayer ! Relay(r, TestConstants.Alice.nodeParams.nodeId) + setConfidence(f)(1.0) - expectFwdFail(register, r.add.channelId, CMD_FAIL_HTLC(r.add.id, Right(UnknownNextPeer()), commit = true)) + expectFwdFail(register, r.add.channelId, CMD_FAIL_HTLC(r.add.id, Right(UnknownNextPeer()), commit = true), reputationRecorder) } test("fail to relay when register returns an error") { f => @@ -371,11 +390,12 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a channelRelayer ! WrappedLocalChannelUpdate(u) channelRelayer ! Relay(r, TestConstants.Alice.nodeParams.nodeId) + setConfidence(f)(1.0) val fwd = expectFwdAdd(register, channelIds(realScid1), outgoingAmount, outgoingExpiry, 7) fwd.replyTo ! Register.ForwardFailure(fwd) - expectFwdFail(register, r.add.channelId, CMD_FAIL_HTLC(r.add.id, Right(UnknownNextPeer()), commit = true)) + expectFwdFail(register, r.add.channelId, CMD_FAIL_HTLC(r.add.id, Right(UnknownNextPeer()), commit = true), reputationRecorder) } test("fail to relay when the channel is advertised as unusable (down)") { f => @@ -389,8 +409,9 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a channelRelayer ! WrappedLocalChannelUpdate(u) channelRelayer ! WrappedLocalChannelDown(d) channelRelayer ! Relay(r, TestConstants.Alice.nodeParams.nodeId) + setConfidence(f)(1.0) - expectFwdFail(register, r.add.channelId, CMD_FAIL_HTLC(r.add.id, Right(UnknownNextPeer()), commit = true)) + expectFwdFail(register, r.add.channelId, CMD_FAIL_HTLC(r.add.id, Right(UnknownNextPeer()), commit = true), reputationRecorder) } test("fail to relay when channel is disabled") { f => @@ -402,8 +423,9 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a channelRelayer ! WrappedLocalChannelUpdate(u) channelRelayer ! Relay(r, TestConstants.Alice.nodeParams.nodeId) + setConfidence(f)(1.0) - expectFwdFail(register, r.add.channelId, CMD_FAIL_HTLC(r.add.id, Right(ChannelDisabled(u.channelUpdate.messageFlags, u.channelUpdate.channelFlags, Some(u.channelUpdate))), commit = true)) + expectFwdFail(register, r.add.channelId, CMD_FAIL_HTLC(r.add.id, Right(ChannelDisabled(u.channelUpdate.messageFlags, u.channelUpdate.channelFlags, Some(u.channelUpdate))), commit = true), reputationRecorder) } test("fail to relay when amount is below minimum") { f => @@ -415,8 +437,9 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a channelRelayer ! WrappedLocalChannelUpdate(u) channelRelayer ! Relay(r, TestConstants.Alice.nodeParams.nodeId) + setConfidence(f)(1.0) - expectFwdFail(register, r.add.channelId, CMD_FAIL_HTLC(r.add.id, Right(AmountBelowMinimum(outgoingAmount, Some(u.channelUpdate))), commit = true)) + expectFwdFail(register, r.add.channelId, CMD_FAIL_HTLC(r.add.id, Right(AmountBelowMinimum(outgoingAmount, Some(u.channelUpdate))), commit = true), reputationRecorder) } test("fail to relay blinded payment") { f => @@ -429,6 +452,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a channelRelayer ! WrappedLocalChannelUpdate(u) channelRelayer ! Relay(r, TestConstants.Alice.nodeParams.nodeId) + setConfidence(f)(1.0) val cmd = register.expectMessageType[Register.Forward[channel.Command]] assert(cmd.channelId == r.add.channelId) @@ -445,6 +469,8 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a assert(fail.onionHash == Sphinx.hash(r.add.onionRoutingPacket)) assert(fail.failureCode == InvalidOnionBlinding(Sphinx.hash(r.add.onionRoutingPacket)).code) } + + reputationRecorder.expectMessageType[ReputationRecorder.CancelRelay] } } @@ -459,6 +485,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a channelRelayer ! WrappedLocalChannelUpdate(u) channelRelayer ! Relay(r, TestConstants.Alice.nodeParams.nodeId) + setConfidence(f)(1.0) // We try to wake-up the next node, but we timeout before they connect. peerReadyManager.expectMessageType[PeerReadyManager.Register].replyTo ! PeerReadyManager.Registered(outgoingNodeId, otherAttempts = 0) @@ -478,6 +505,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a channelRelayer ! WrappedLocalChannelUpdate(u) channelRelayer ! Relay(r, TestConstants.Alice.nodeParams.nodeId) + setConfidence(f)(1.0) expectFwdAdd(register, channelIds(realScid1), r.amountToForward, r.outgoingCltv, 7).message } @@ -491,8 +519,9 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a channelRelayer ! WrappedLocalChannelUpdate(u) channelRelayer ! Relay(r, TestConstants.Alice.nodeParams.nodeId) + setConfidence(f)(1.0) - expectFwdFail(register, r.add.channelId, CMD_FAIL_HTLC(r.add.id, Right(IncorrectCltvExpiry(r.outgoingCltv, Some(u.channelUpdate))), commit = true)) + expectFwdFail(register, r.add.channelId, CMD_FAIL_HTLC(r.add.id, Right(IncorrectCltvExpiry(r.outgoingCltv, Some(u.channelUpdate))), commit = true), reputationRecorder) } test("fail to relay when fee is insufficient") { f => @@ -504,8 +533,9 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a channelRelayer ! WrappedLocalChannelUpdate(u) channelRelayer ! Relay(r, TestConstants.Alice.nodeParams.nodeId) + setConfidence(f)(1.0) - expectFwdFail(register, r.add.channelId, CMD_FAIL_HTLC(r.add.id, Right(FeeInsufficient(r.add.amountMsat, Some(u.channelUpdate))), commit = true)) + expectFwdFail(register, r.add.channelId, CMD_FAIL_HTLC(r.add.id, Right(FeeInsufficient(r.add.amountMsat, Some(u.channelUpdate))), commit = true), reputationRecorder) } test("relay that would fail (fee insufficient) with a recent channel update but succeed with the previous update") { f => @@ -517,6 +547,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a channelRelayer ! WrappedLocalChannelUpdate(u1) channelRelayer ! Relay(r, TestConstants.Alice.nodeParams.nodeId) + setConfidence(f)(1.0) // relay succeeds with current channel update (u1) with lower fees expectFwdAdd(register, channelIds(realScid1), outgoingAmount, outgoingExpiry, 7) @@ -525,6 +556,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a channelRelayer ! WrappedLocalChannelUpdate(u2) channelRelayer ! Relay(r, TestConstants.Alice.nodeParams.nodeId) + setConfidence(f)(1.0) // relay succeeds because the current update (u2) with higher fees occurred less than 10 minutes ago expectFwdAdd(register, channelIds(realScid1), outgoingAmount, outgoingExpiry, 7) @@ -534,9 +566,10 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a channelRelayer ! WrappedLocalChannelUpdate(u1) channelRelayer ! WrappedLocalChannelUpdate(u3) channelRelayer ! Relay(r, TestConstants.Alice.nodeParams.nodeId) + setConfidence(f)(1.0) // relay fails because the current update (u3) with higher fees occurred more than 10 minutes ago - expectFwdFail(register, r.add.channelId, CMD_FAIL_HTLC(r.add.id, Right(FeeInsufficient(r.add.amountMsat, Some(u3.channelUpdate))), commit = true)) + expectFwdFail(register, r.add.channelId, CMD_FAIL_HTLC(r.add.id, Right(FeeInsufficient(r.add.amountMsat, Some(u3.channelUpdate))), commit = true), reputationRecorder) } test("fail to relay when there is a local error") { f => @@ -563,9 +596,10 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a testCases.foreach { testCase => channelRelayer ! WrappedLocalChannelUpdate(u) channelRelayer ! Relay(r, TestConstants.Alice.nodeParams.nodeId) + setConfidence(f)(1.0) val fwd = expectFwdAdd(register, channelIds(realScid1), outgoingAmount, outgoingExpiry, 7) fwd.message.replyTo ! RES_ADD_FAILED(fwd.message, testCase.exc, Some(testCase.update)) - expectFwdFail(register, r.add.channelId, CMD_FAIL_HTLC(r.add.id, Right(testCase.failure), commit = true)) + expectFwdFail(register, r.add.channelId, CMD_FAIL_HTLC(r.add.id, Right(testCase.failure), commit = true), reputationRecorder) } } @@ -598,6 +632,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a val payload = ChannelRelay.Standard(ShortChannelId(12345), 998900 msat, CltvExpiry(60)) val r = createValidIncomingPacket(payload, 1000000 msat, CltvExpiry(70), endorsementIn = 5) channelRelayer ! Relay(r, TestConstants.Alice.nodeParams.nodeId) + setConfidence(f)(5.5 / 8) // select the channel to the same node, with the lowest capacity and balance but still high enough to handle the payment val cmd1 = expectFwdAdd(register, channelUpdates(ShortChannelId(22223)).channelId, r.amountToForward, r.outgoingCltv, 5).message cmd1.replyTo ! RES_ADD_FAILED(cmd1, ChannelUnavailable(randomBytes32()), None) @@ -611,13 +646,14 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a val cmd4 = expectFwdAdd(register, channelUpdates(ShortChannelId(11111)).channelId, r.amountToForward, r.outgoingCltv, 5).message cmd4.replyTo ! RES_ADD_FAILED(cmd4, HtlcValueTooHighInFlight(randomBytes32(), 100000000 msat, 100000000 msat), Some(channelUpdates(ShortChannelId(11111)).channelUpdate)) // all the suitable channels have been tried - expectFwdFail(register, r.add.channelId, CMD_FAIL_HTLC(r.add.id, Right(TemporaryChannelFailure(Some(channelUpdates(ShortChannelId(12345)).channelUpdate))), commit = true)) + expectFwdFail(register, r.add.channelId, CMD_FAIL_HTLC(r.add.id, Right(TemporaryChannelFailure(Some(channelUpdates(ShortChannelId(12345)).channelUpdate))), commit = true), reputationRecorder) } { // higher amount payment (have to increased incoming htlc amount for fees to be sufficient) val payload = ChannelRelay.Standard(ShortChannelId(12345), 50000000 msat, CltvExpiry(60)) val r = createValidIncomingPacket(payload, 60000000 msat, CltvExpiry(70), endorsementIn = 0) channelRelayer ! Relay(r, TestConstants.Alice.nodeParams.nodeId) + setConfidence(f)(0) expectFwdAdd(register, channelUpdates(ShortChannelId(11111)).channelId, r.amountToForward, r.outgoingCltv, 0).message } { @@ -625,6 +661,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a val payload = ChannelRelay.Standard(ShortChannelId(12345), 1000 msat, CltvExpiry(60)) val r = createValidIncomingPacket(payload, 60000000 msat, CltvExpiry(70), endorsementIn = 6) channelRelayer ! Relay(r, TestConstants.Alice.nodeParams.nodeId) + setConfidence(f)(6.5 / 8) expectFwdAdd(register, channelUpdates(ShortChannelId(33333)).channelId, r.amountToForward, r.outgoingCltv, 6).message } { @@ -632,6 +669,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a val payload = ChannelRelay.Standard(ShortChannelId(12345), 1000000000 msat, CltvExpiry(60)) val r = createValidIncomingPacket(payload, 1010000000 msat, CltvExpiry(70)) channelRelayer ! Relay(r, TestConstants.Alice.nodeParams.nodeId) + setConfidence(f)(7.5 / 8) expectFwdAdd(register, channelUpdates(ShortChannelId(12345)).channelId, r.amountToForward, r.outgoingCltv, 7).message } { @@ -639,6 +677,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a val payload = ChannelRelay.Standard(ShortChannelId(12345), 998900 msat, CltvExpiry(50)) val r = createValidIncomingPacket(payload, 1000000 msat, CltvExpiry(70)) channelRelayer ! Relay(r, TestConstants.Alice.nodeParams.nodeId) + setConfidence(f)(7.5 / 8) expectFwdAdd(register, channelUpdates(ShortChannelId(22223)).channelId, r.amountToForward, r.outgoingCltv, 7).message } { @@ -646,7 +685,8 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a val payload = ChannelRelay.Standard(ShortChannelId(12345), 998900 msat, CltvExpiry(61)) val r = createValidIncomingPacket(payload, 1000000 msat, CltvExpiry(70)) channelRelayer ! Relay(r, TestConstants.Alice.nodeParams.nodeId) - expectFwdFail(register, r.add.channelId, CMD_FAIL_HTLC(r.add.id, Right(IncorrectCltvExpiry(CltvExpiry(61), Some(channelUpdates(ShortChannelId(12345)).channelUpdate))), commit = true)) + setConfidence(f)(0.2) + expectFwdFail(register, r.add.channelId, CMD_FAIL_HTLC(r.add.id, Right(IncorrectCltvExpiry(CltvExpiry(61), Some(channelUpdates(ShortChannelId(12345)).channelUpdate))), commit = true), reputationRecorder) } } @@ -672,10 +712,14 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a testCases.foreach { testCase => channelRelayer ! WrappedLocalChannelUpdate(u) channelRelayer ! Relay(r, TestConstants.Alice.nodeParams.nodeId) + setConfidence(f)(1.0) val fwd = expectFwdAdd(register, channelIds(realScid1), outgoingAmount, outgoingExpiry, 7) fwd.message.replyTo ! RES_SUCCESS(fwd.message, channelId1) fwd.message.origin.replyTo ! RES_ADD_SETTLED(fwd.message.origin, downstream_htlc, testCase.result) - expectFwdFail(register, r.add.channelId, testCase.cmd) + val fwdFail = register.expectMessageType[Register.Forward[channel.Command]] + assert(fwdFail.message == testCase.cmd) + assert(fwdFail.channelId == r.add.channelId) + assert(!reputationRecorder.expectMessageType[ReputationRecorder.RecordResult].isSuccess) } } @@ -698,6 +742,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a val r = createValidIncomingPacket(createBlindedPayload(Right(u.channelUpdate.shortChannelId), u.channelUpdate, isIntroduction), outgoingAmount + u.channelUpdate.feeBaseMsat, outgoingExpiry + u.channelUpdate.cltvExpiryDelta, endorsementIn = 0) channelRelayer ! WrappedLocalChannelUpdate(u) channelRelayer ! Relay(r, TestConstants.Alice.nodeParams.nodeId) + setConfidence(f)(0) val fwd = expectFwdAdd(register, channelId1, outgoingAmount, outgoingExpiry, 0) fwd.message.replyTo ! RES_SUCCESS(fwd.message, channelId1) fwd.message.origin.replyTo ! RES_ADD_SETTLED(fwd.message.origin, downstream, htlcResult) @@ -716,6 +761,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a assert(fail.onionHash == Sphinx.hash(r.add.onionRoutingPacket)) assert(fail.failureCode == InvalidOnionBlinding(Sphinx.hash(r.add.onionRoutingPacket)).code) } + assert(!reputationRecorder.expectMessageType[ReputationRecorder.RecordResult].isSuccess) } } } @@ -741,6 +787,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a testCases.foreach { testCase => channelRelayer ! WrappedLocalChannelUpdate(u) channelRelayer ! Relay(r, TestConstants.Alice.nodeParams.nodeId) + setConfidence(f)(3.5 / 8) val fwd1 = expectFwdAdd(register, channelIds(realScid1), outgoingAmount, outgoingExpiry, 3) fwd1.message.replyTo ! RES_SUCCESS(fwd1.message, channelId1) @@ -753,6 +800,8 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a val paymentRelayed = eventListener.expectMessageType[ChannelPaymentRelayed] assert(paymentRelayed.copy(startedAt = 0 unixms, settledAt = 0 unixms) == ChannelPaymentRelayed(r.add.amountMsat, r.amountToForward, r.add.paymentHash, r.add.channelId, channelId1, startedAt = 0 unixms, settledAt = 0 unixms)) + + assert(reputationRecorder.expectMessageType[ReputationRecorder.RecordResult].isSuccess) } } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/NodeRelayerSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/NodeRelayerSpec.scala index 122f4e5560..978c87ddb0 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/NodeRelayerSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/NodeRelayerSpec.scala @@ -45,6 +45,8 @@ import fr.acinq.eclair.payment.send.PaymentLifecycle.SendPaymentToNode import fr.acinq.eclair.payment.send.{BlindedRecipient, ClearRecipient} import fr.acinq.eclair.router.Router.{ChannelHop, HopRelayParams, RouteRequest} import fr.acinq.eclair.router.{BalanceTooLow, BlindedRouteCreation, RouteNotFound, Router} +import fr.acinq.eclair.reputation.ReputationRecorder +import fr.acinq.eclair.reputation.ReputationRecorder.{Confidence, GetTrampolineConfidence, RecordTrampolineFailure, RecordTrampolineSuccess} import fr.acinq.eclair.wire.protocol.OfferTypes._ import fr.acinq.eclair.wire.protocol.PaymentOnion.{FinalPayload, IntermediatePayload} import fr.acinq.eclair.wire.protocol.RouteBlindingEncryptedDataCodecs.blindedRouteDataCodec @@ -71,11 +73,11 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl val wakeUpTimeout = "wake_up_timeout" val onTheFlyFunding = "on_the_fly_funding" - case class FixtureParam(nodeParams: NodeParams, router: TestProbe[Any], register: TestProbe[Any], mockPayFSM: TestProbe[Any], eventListener: TestProbe[PaymentEvent]) { + case class FixtureParam(nodeParams: NodeParams, router: TestProbe[Any], register: TestProbe[Any], reputationRecorder: TestProbe[ReputationRecorder.Command], mockPayFSM: TestProbe[Any], eventListener: TestProbe[PaymentEvent]) { def createNodeRelay(packetIn: IncomingPaymentPacket.NodeRelayPacket, useRealPaymentFactory: Boolean = false): (ActorRef[NodeRelay.Command], TestProbe[NodeRelayer.Command]) = { val parent = TestProbe[NodeRelayer.Command]("parent-relayer") val outgoingPaymentFactory = if (useRealPaymentFactory) RealOutgoingPaymentFactory(this) else FakeOutgoingPaymentFactory(this) - val nodeRelay = testKit.spawn(NodeRelay(nodeParams, parent.ref, register.ref.toClassic, relayId, packetIn, outgoingPaymentFactory, router.ref.toClassic)) + val nodeRelay = testKit.spawn(NodeRelay(nodeParams, parent.ref, register.ref.toClassic, Some(reputationRecorder.ref), relayId, packetIn, outgoingPaymentFactory, router.ref.toClassic)) (nodeRelay, parent) } @@ -117,16 +119,17 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl .modify(_.features.activated).usingIf(test.tags.contains(onTheFlyFunding))(_ + (Features.OnTheFlyFunding -> FeatureSupport.Optional)) val router = TestProbe[Any]("router") val register = TestProbe[Any]("register") + val reputationRecorder = TestProbe[ReputationRecorder.Command]("reputation-recorder") val eventListener = TestProbe[PaymentEvent]("event-listener") system.eventStream ! EventStream.Subscribe(eventListener.ref) val mockPayFSM = TestProbe[Any]("pay-fsm") - withFixture(test.toNoArgTest(FixtureParam(nodeParams, router, register, mockPayFSM, eventListener))) + withFixture(test.toNoArgTest(FixtureParam(nodeParams, router, register, reputationRecorder, mockPayFSM, eventListener))) } test("create child handlers for new payments") { f => import f._ val probe = TestProbe[Any]() - val parentRelayer = testKit.spawn(NodeRelayer(nodeParams, register.ref.toClassic, FakeOutgoingPaymentFactory(f), router.ref.toClassic)) + val parentRelayer = testKit.spawn(NodeRelayer(nodeParams, register.ref.toClassic, Some(reputationRecorder.ref), FakeOutgoingPaymentFactory(f), router.ref.toClassic)) parentRelayer ! NodeRelayer.GetPendingPayments(probe.ref.toClassic) probe.expectMessage(Map.empty) @@ -165,7 +168,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl val outgoingPaymentFactory = FakeOutgoingPaymentFactory(f) { - val parentRelayer = testKit.spawn(NodeRelayer(nodeParams, register.ref.toClassic, outgoingPaymentFactory, router.ref.toClassic)) + val parentRelayer = testKit.spawn(NodeRelayer(nodeParams, register.ref.toClassic, Some(reputationRecorder.ref), outgoingPaymentFactory, router.ref.toClassic)) parentRelayer ! NodeRelayer.GetPendingPayments(probe.ref.toClassic) probe.expectMessage(Map.empty) } @@ -173,7 +176,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl val (paymentHash1, paymentSecret1, child1) = (randomBytes32(), randomBytes32(), TestProbe[NodeRelay.Command]()) val (paymentHash2, paymentSecret2, child2) = (randomBytes32(), randomBytes32(), TestProbe[NodeRelay.Command]()) val children = Map(PaymentKey(paymentHash1, paymentSecret1) -> child1.ref, PaymentKey(paymentHash2, paymentSecret2) -> child2.ref) - val parentRelayer = testKit.spawn(NodeRelayer(nodeParams, register.ref.toClassic, outgoingPaymentFactory, router.ref.toClassic, children)) + val parentRelayer = testKit.spawn(NodeRelayer(nodeParams, register.ref.toClassic, Some(reputationRecorder.ref), outgoingPaymentFactory, router.ref.toClassic, children)) parentRelayer ! NodeRelayer.GetPendingPayments(probe.ref.toClassic) probe.expectMessage(children) @@ -189,7 +192,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl val (paymentSecret1, child1) = (randomBytes32(), TestProbe[NodeRelay.Command]()) val (paymentSecret2, child2) = (randomBytes32(), TestProbe[NodeRelay.Command]()) val children = Map(PaymentKey(paymentHash, paymentSecret1) -> child1.ref, PaymentKey(paymentHash, paymentSecret2) -> child2.ref) - val parentRelayer = testKit.spawn(NodeRelayer(nodeParams, register.ref.toClassic, outgoingPaymentFactory, router.ref.toClassic, children)) + val parentRelayer = testKit.spawn(NodeRelayer(nodeParams, register.ref.toClassic, Some(reputationRecorder.ref), outgoingPaymentFactory, router.ref.toClassic, children)) parentRelayer ! NodeRelayer.GetPendingPayments(probe.ref.toClassic) probe.expectMessage(children) @@ -199,7 +202,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl probe.expectMessage(Map(PaymentKey(paymentHash, paymentSecret2) -> child2.ref)) } { - val parentRelayer = testKit.spawn(NodeRelayer(nodeParams, register.ref.toClassic, outgoingPaymentFactory, router.ref.toClassic)) + val parentRelayer = testKit.spawn(NodeRelayer(nodeParams, register.ref.toClassic, Some(reputationRecorder.ref), outgoingPaymentFactory, router.ref.toClassic)) parentRelayer ! NodeRelayer.Relay(incomingMultiPart.head, randomKey().publicKey) parentRelayer ! NodeRelayer.GetPendingPayments(probe.ref.toClassic) val pending1 = probe.expectMessageType[Map[PaymentKey, ActorRef[NodeRelay.Command]]] @@ -251,6 +254,8 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl createTrampolinePacket(outgoingAmount, outgoingExpiry)) nodeRelayer ! NodeRelay.Relay(extra, randomKey().publicKey) + reputationRecorder.expectMessageType[GetTrampolineConfidence].replyTo ! Confidence(1.0) + // the extra payment will be rejected val fwd = register.expectMessageType[Register.Forward[CMD_FAIL_HTLC]] assert(fwd.channelId == extra.add.channelId) @@ -267,8 +272,10 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl // Receive a complete upstream multi-part payment, which we relay out. incomingMultiPart.foreach(incoming => nodeRelayer ! NodeRelay.Relay(incoming, randomKey().publicKey)) + reputationRecorder.expectMessageType[GetTrampolineConfidence].replyTo ! Confidence(3.5 / 8) + val outgoingCfg = mockPayFSM.expectMessageType[SendPaymentConfig] - validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingMultiPart.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey)).toList), 5) + validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingMultiPart.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey)).toList), 3) val outgoingPayment = mockPayFSM.expectMessageType[SendMultiPartPayment] validateOutgoingPayment(outgoingPayment) @@ -363,9 +370,11 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl val (nodeRelayer, parent) = createNodeRelay(incomingAsyncPayment.head) incomingAsyncPayment.foreach(p => nodeRelayer ! NodeRelay.Relay(p, randomKey().publicKey)) + reputationRecorder.expectMessageType[GetTrampolineConfidence].replyTo ! Confidence(6.5 / 8) + // upstream payment relayed val outgoingCfg = mockPayFSM.expectMessageType[SendPaymentConfig] - validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingAsyncPayment.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey)).toList), 5) + validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingAsyncPayment.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey)).toList), 6) val outgoingPayment = mockPayFSM.expectMessageType[SendMultiPartPayment] validateOutgoingPayment(outgoingPayment) // those are adapters for pay-fsm messages @@ -386,6 +395,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl assert(relayEvent.incoming.map(p => (p.amount, p.channelId)).toSet == incomingAsyncPayment.map(i => (i.add.amountMsat, i.add.channelId)).toSet) assert(relayEvent.outgoing.nonEmpty) parent.expectMessageType[NodeRelayer.RelayComplete] + reputationRecorder.expectMessageType[RecordTrampolineSuccess] register.expectNoMessage(100 millis) } @@ -462,6 +472,8 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl val (nodeRelayer, _) = f.createNodeRelay(incomingMultiPart.head) incomingMultiPart.foreach(p => nodeRelayer ! NodeRelay.Relay(p, randomKey().publicKey)) + reputationRecorder.expectMessageType[GetTrampolineConfidence].replyTo ! Confidence(1.0) + mockPayFSM.expectMessageType[SendPaymentConfig] // those are adapters for pay-fsm messages val nodeRelayerAdapters = mockPayFSM.expectMessageType[SendMultiPartPayment].replyTo @@ -474,6 +486,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl assert(fwd.message == CMD_FAIL_HTLC(p.add.id, Right(TrampolineFeeInsufficient()), commit = true)) } + reputationRecorder.expectMessageType[RecordTrampolineFailure] register.expectNoMessage(100 millis) eventListener.expectNoMessage(100 millis) } @@ -489,6 +502,8 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl val (nodeRelayer, _) = f.createNodeRelay(incoming.head, useRealPaymentFactory = true) incoming.foreach(p => nodeRelayer ! NodeRelay.Relay(p, randomKey().publicKey)) + reputationRecorder.expectMessageType[GetTrampolineConfidence].replyTo ! Confidence(1.0) + val payFSM = mockPayFSM.expectMessageType[akka.actor.ActorRef] router.expectMessageType[RouteRequest] payFSM ! Status.Failure(BalanceTooLow) @@ -499,6 +514,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl assert(fwd.message == CMD_FAIL_HTLC(p.add.id, Right(TemporaryNodeFailure()), commit = true)) } + reputationRecorder.expectMessageType[RecordTrampolineFailure] register.expectNoMessage(100 millis) } @@ -509,6 +525,8 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl val (nodeRelayer, _) = f.createNodeRelay(incomingMultiPart.head, useRealPaymentFactory = true) incomingMultiPart.foreach(p => nodeRelayer ! NodeRelay.Relay(p, randomKey().publicKey)) + reputationRecorder.expectMessageType[GetTrampolineConfidence].replyTo ! Confidence(1.0) + val payFSM = mockPayFSM.expectMessageType[akka.actor.ActorRef] router.expectMessageType[RouteRequest] @@ -522,6 +540,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl assert(fwd.message == CMD_FAIL_HTLC(p.add.id, Right(TrampolineFeeInsufficient()), commit = true)) } + reputationRecorder.expectMessageType[RecordTrampolineFailure] register.expectNoMessage(100 millis) } @@ -532,6 +551,8 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl val (nodeRelayer, _) = f.createNodeRelay(incomingMultiPart.head, useRealPaymentFactory = true) incomingMultiPart.foreach(p => nodeRelayer ! NodeRelay.Relay(p, randomKey().publicKey)) + reputationRecorder.expectMessageType[GetTrampolineConfidence].replyTo ! Confidence(1.0) + val payFSM = mockPayFSM.expectMessageType[akka.actor.ActorRef] router.expectMessageType[RouteRequest] @@ -544,6 +565,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl assert(fwd.message == CMD_FAIL_HTLC(p.add.id, Right(FinalIncorrectHtlcAmount(42 msat)), commit = true)) } + reputationRecorder.expectMessageType[RecordTrampolineFailure] register.expectNoMessage(100 millis) } @@ -554,6 +576,8 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl val (nodeRelayer, _) = f.createNodeRelay(incomingSinglePart, useRealPaymentFactory = true) nodeRelayer ! NodeRelay.Relay(incomingSinglePart, randomKey().publicKey) + reputationRecorder.expectMessageType[GetTrampolineConfidence].replyTo ! Confidence(1.0) + val routeRequest = router.expectMessageType[RouteRequest] val routeParams = routeRequest.routeParams assert(routeParams.boundaries.maxFeeProportional == 0) // should be disabled @@ -572,8 +596,10 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl nodeRelayer ! NodeRelay.Relay(incomingMultiPart.last, randomKey().publicKey) + reputationRecorder.expectMessageType[GetTrampolineConfidence].replyTo ! Confidence(3.5 / 8) + val outgoingCfg = mockPayFSM.expectMessageType[SendPaymentConfig] - validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingMultiPart.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey)).toList), 5) + validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingMultiPart.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey)).toList), 3) val outgoingPayment = mockPayFSM.expectMessageType[SendMultiPartPayment] validateOutgoingPayment(outgoingPayment) // those are adapters for pay-fsm messages @@ -597,6 +623,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl validateRelayEvent(relayEvent) assert(relayEvent.incoming.map(p => (p.amount, p.channelId)).toSet == incomingMultiPart.map(i => (i.add.amountMsat, i.add.channelId)).toSet) assert(relayEvent.outgoing.nonEmpty) + reputationRecorder.expectMessageType[RecordTrampolineSuccess] parent.expectMessageType[NodeRelayer.RelayComplete] register.expectNoMessage(100 millis) } @@ -608,6 +635,8 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl val (nodeRelayer, parent) = f.createNodeRelay(incomingSinglePart) nodeRelayer ! NodeRelay.Relay(incomingSinglePart, randomKey().publicKey) + reputationRecorder.expectMessageType[GetTrampolineConfidence].replyTo ! Confidence(1.0) + val outgoingCfg = mockPayFSM.expectMessageType[SendPaymentConfig] validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(Upstream.Hot.Channel(incomingSinglePart.add, TimestampMilli.now(), randomKey().publicKey) :: Nil), 7) val outgoingPayment = mockPayFSM.expectMessageType[SendMultiPartPayment] @@ -626,6 +655,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl validateRelayEvent(relayEvent) assert(relayEvent.incoming.map(p => (p.amount, p.channelId)) == Seq((incomingSinglePart.add.amountMsat, incomingSinglePart.add.channelId))) assert(relayEvent.outgoing.nonEmpty) + reputationRecorder.expectMessageType[RecordTrampolineSuccess] parent.expectMessageType[NodeRelayer.RelayComplete] register.expectNoMessage(100 millis) } @@ -720,8 +750,10 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl val (nodeRelayer, parent) = f.createNodeRelay(incomingPayments.head) incomingPayments.foreach(incoming => nodeRelayer ! NodeRelay.Relay(incoming, randomKey().publicKey)) + reputationRecorder.expectMessageType[GetTrampolineConfidence].replyTo ! Confidence(2.5 / 8) + val outgoingCfg = mockPayFSM.expectMessageType[SendPaymentConfig] - validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingMultiPart.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey)).toList), 5) + validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingMultiPart.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey)).toList), 2) val outgoingPayment = mockPayFSM.expectMessageType[SendMultiPartPayment] assert(outgoingPayment.recipient.nodeId == outgoingNodeId) assert(outgoingPayment.recipient.totalAmount == outgoingAmount) @@ -747,6 +779,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl validateRelayEvent(relayEvent) assert(relayEvent.incoming.map(p => (p.amount, p.channelId)) == incomingMultiPart.map(i => (i.add.amountMsat, i.add.channelId))) assert(relayEvent.outgoing.nonEmpty) + reputationRecorder.expectMessageType[RecordTrampolineSuccess] parent.expectMessageType[NodeRelayer.RelayComplete] register.expectNoMessage(100 millis) } @@ -764,8 +797,10 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl val (nodeRelayer, parent) = f.createNodeRelay(incomingPayments.head) incomingPayments.foreach(incoming => nodeRelayer ! NodeRelay.Relay(incoming, randomKey().publicKey)) + reputationRecorder.expectMessageType[GetTrampolineConfidence].replyTo ! Confidence(0) + val outgoingCfg = mockPayFSM.expectMessageType[SendPaymentConfig] - validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingMultiPart.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey)).toList), 5) + validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingMultiPart.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey)).toList), 0) val outgoingPayment = mockPayFSM.expectMessageType[SendPaymentToNode] assert(outgoingPayment.recipient.nodeId == outgoingNodeId) assert(outgoingPayment.amount == outgoingAmount) @@ -791,6 +826,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl validateRelayEvent(relayEvent) assert(relayEvent.incoming.map(p => (p.amount, p.channelId)) == incomingMultiPart.map(i => (i.add.amountMsat, i.add.channelId))) assert(relayEvent.outgoing.length == 1) + reputationRecorder.expectMessageType[RecordTrampolineSuccess] parent.expectMessageType[NodeRelayer.RelayComplete] register.expectNoMessage(100 millis) } @@ -822,8 +858,10 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl val (nodeRelayer, parent) = f.createNodeRelay(incomingPayments.head) incomingPayments.foreach(incoming => nodeRelayer ! NodeRelay.Relay(incoming, randomKey().publicKey)) + reputationRecorder.expectMessageType[GetTrampolineConfidence].replyTo ! Confidence(1.0) + val outgoingCfg = mockPayFSM.expectMessageType[SendPaymentConfig] - validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingPayments.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey)).toList), 5, ignoreNodeId = true) + validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingPayments.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey)).toList), 7, ignoreNodeId = true) val outgoingPayment = mockPayFSM.expectMessageType[SendPaymentToNode] assert(outgoingPayment.amount == outgoingAmount) assert(outgoingPayment.recipient.expiry == outgoingExpiry) @@ -855,8 +893,10 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl val (nodeRelayer, parent) = f.createNodeRelay(incomingPayments.head) incomingPayments.foreach(incoming => nodeRelayer ! NodeRelay.Relay(incoming, randomKey().publicKey)) + reputationRecorder.expectMessageType[GetTrampolineConfidence].replyTo ! Confidence(1.5 / 8) + val outgoingCfg = mockPayFSM.expectMessageType[SendPaymentConfig] - validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingPayments.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey)).toList), 5, ignoreNodeId = true) + validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingPayments.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey)).toList), 1, ignoreNodeId = true) val outgoingPayment = mockPayFSM.expectMessageType[SendMultiPartPayment] assert(outgoingPayment.recipient.totalAmount == outgoingAmount) assert(outgoingPayment.recipient.expiry == outgoingExpiry) @@ -897,8 +937,10 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl wakeUp.replyTo ! Peer.PeerInfo(TestProbe[Any]().ref.toClassic, outgoingNodeId, Peer.CONNECTED, Some(nodeParams.features.initFeatures()), None, Set.empty) cleanUpWakeUpActors(peerReadyManager, switchboard) + reputationRecorder.expectMessageType[GetTrampolineConfidence].replyTo ! Confidence(3.5 / 8) + val outgoingCfg = mockPayFSM.expectMessageType[SendPaymentConfig] - validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingPayments.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey)).toList), 5, ignoreNodeId = true) + validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingPayments.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey)).toList), 3, ignoreNodeId = true) val outgoingPayment = mockPayFSM.expectMessageType[SendMultiPartPayment] assert(outgoingPayment.recipient.totalAmount == outgoingAmount) assert(outgoingPayment.recipient.expiry == outgoingExpiry) @@ -1028,8 +1070,10 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl assert(getNodeId.shortChannelId == scidDir.scid) getNodeId.replyTo ! Some(outgoingNodeId) + reputationRecorder.expectMessageType[GetTrampolineConfidence].replyTo ! Confidence(4.5 / 8) + val outgoingCfg = mockPayFSM.expectMessageType[SendPaymentConfig] - validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingPayments.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey)).toList), 5, ignoreNodeId = true) + validateOutgoingCfg(outgoingCfg, Upstream.Hot.Trampoline(incomingPayments.map(p => Upstream.Hot.Channel(p.add, TimestampMilli.now(), randomKey().publicKey)).toList), 4, ignoreNodeId = true) val outgoingPayment = mockPayFSM.expectMessageType[SendPaymentToNode] assert(outgoingPayment.amount == outgoingAmount) assert(outgoingPayment.recipient.expiry == outgoingExpiry) 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 2c22c743e6..4b374fa59a 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 @@ -34,6 +34,7 @@ import fr.acinq.eclair.payment.OutgoingPaymentPacket.{NodePayload, buildOnion, b import fr.acinq.eclair.payment.PaymentPacketSpec._ import fr.acinq.eclair.payment.relay.Relayer._ import fr.acinq.eclair.payment.send.{ClearRecipient, TrampolineRecipient} +import fr.acinq.eclair.reputation.ReputationRecorder import fr.acinq.eclair.router.BaseRouterSpec.{blindedRouteFromHops, channelHopFromUpdate} import fr.acinq.eclair.router.Router.{NodeHop, Route} import fr.acinq.eclair.wire.protocol.PaymentOnion.FinalPayload @@ -47,7 +48,7 @@ import scala.concurrent.duration.DurationInt class RelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("application")) with FixtureAnyFunSuiteLike { - case class FixtureParam(nodeParams: NodeParams, relayer: akka.actor.ActorRef, router: TestProbe[Any], register: TestProbe[Any], childActors: ChildActors, paymentHandler: TestProbe[Any]) + case class FixtureParam(nodeParams: NodeParams, relayer: akka.actor.ActorRef, router: TestProbe[Any], register: TestProbe[Any], childActors: ChildActors, paymentHandler: TestProbe[Any], reputationRecorder: TestProbe[ReputationRecorder.Command]) override def withFixture(test: OneArgTest): Outcome = { // we are node B in the route A -> B -> C -> .... @@ -56,17 +57,26 @@ class RelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat val router = TestProbe[Any]("router") val register = TestProbe[Any]("register") val paymentHandler = TestProbe[Any]("payment-handler") + val reputationRecorder = TestProbe[ReputationRecorder.Command]("reputation-recorder") val probe = TestProbe[Any]() // we can't spawn top-level actors with akka typed testKit.spawn(Behaviors.setup[Any] { context => - val relayer = context.toClassic.actorOf(Relayer.props(nodeParams, router.ref.toClassic, register.ref.toClassic, paymentHandler.ref.toClassic)) + val relayer = context.toClassic.actorOf(Relayer.props(nodeParams, router.ref.toClassic, register.ref.toClassic, paymentHandler.ref.toClassic, Some(reputationRecorder.ref))) probe.ref ! relayer Behaviors.empty[Any] }) val relayer = probe.expectMessageType[akka.actor.ActorRef] relayer ! GetChildActors(probe.ref.toClassic) val childActors = probe.expectMessageType[ChildActors] - withFixture(test.toNoArgTest(FixtureParam(nodeParams, relayer, router, register, childActors, paymentHandler))) + withFixture(test.toNoArgTest(FixtureParam(nodeParams, relayer, router, register, childActors, paymentHandler, reputationRecorder))) + } + + def setConfidence(f: FixtureParam)(value: Double): Unit = { + import f._ + + val getConfidence = reputationRecorder.expectMessageType[ReputationRecorder.GetConfidence] + assert(getConfidence.originNode == TestConstants.Alice.nodeParams.nodeId) + getConfidence.replyTo ! ReputationRecorder.Confidence(value) } val channelId_ab = randomBytes32() @@ -94,6 +104,7 @@ class RelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat // and then manually build an htlc val add_ab = UpdateAddHtlc(randomBytes32(), 123456, payment.cmd.amount, payment.cmd.paymentHash, payment.cmd.cltvExpiry, payment.cmd.onion, None, 1.0, None) relayer ! RelayForward(add_ab, priv_a.publicKey) + setConfidence(f)(1.0) register.expectMessageType[Register.Forward[CMD_ADD_HTLC]] } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/reputation/ReputationRecorderSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/reputation/ReputationRecorderSpec.scala new file mode 100644 index 0000000000..7228b3372d --- /dev/null +++ b/eclair-core/src/test/scala/fr/acinq/eclair/reputation/ReputationRecorderSpec.scala @@ -0,0 +1,96 @@ +/* + * Copyright 2023 ACINQ SAS + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package fr.acinq.eclair.reputation + +import akka.actor.testkit.typed.scaladsl.{ScalaTestWithActorTestKit, TestProbe} +import akka.actor.typed.ActorRef +import com.typesafe.config.ConfigFactory +import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey +import fr.acinq.eclair.reputation.ReputationRecorder._ +import fr.acinq.eclair.{MilliSatoshiLong, randomKey} +import org.scalatest.Outcome +import org.scalatest.funsuite.FixtureAnyFunSuiteLike + +import java.util.UUID +import scala.concurrent.duration.DurationInt + +class ReputationRecorderSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("application")) with FixtureAnyFunSuiteLike { + val (uuid1, uuid2, uuid3, uuid4, uuid5, uuid6, uuid7, uuid8) = (UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID()) + val originNode: PublicKey = randomKey().publicKey + + case class FixtureParam(config: Reputation.Config, reputationRecorder: ActorRef[Command], replyTo: TestProbe[Confidence]) + + override def withFixture(test: OneArgTest): Outcome = { + val config = Reputation.Config(enabled = true, 1 day, 10 seconds, 2) + val replyTo = TestProbe[Confidence]("confidence") + val reputationRecorder = testKit.spawn(ReputationRecorder(config, Map.empty)) + withFixture(test.toNoArgTest(FixtureParam(config, reputationRecorder.ref, replyTo))) + } + + test("channel relay") { f => + import f._ + + reputationRecorder ! GetConfidence(replyTo.ref, originNode, 7, uuid1, 2000 msat) + assert(replyTo.expectMessageType[Confidence].value == 0) + reputationRecorder ! RecordResult(originNode, 7, uuid1, isSuccess = true) + reputationRecorder ! GetConfidence(replyTo.ref, originNode, 7, uuid2, 1000 msat) + assert(replyTo.expectMessageType[Confidence].value === (2.0 / 4) +- 0.001) + reputationRecorder ! GetConfidence(replyTo.ref, originNode, 7, uuid3, 3000 msat) + assert(replyTo.expectMessageType[Confidence].value === (2.0 / 10) +- 0.001) + reputationRecorder ! CancelRelay(originNode, 7, uuid3) + reputationRecorder ! GetConfidence(replyTo.ref, originNode, 7, uuid4, 1000 msat) + assert(replyTo.expectMessageType[Confidence].value === (2.0 / 6) +- 0.001) + reputationRecorder ! RecordResult(originNode, 7, uuid4, isSuccess = true) + reputationRecorder ! RecordResult(originNode, 7, uuid2, isSuccess = false) + // Not endorsed + reputationRecorder ! GetConfidence(replyTo.ref, originNode, 0, uuid5, 1000 msat) + assert(replyTo.expectMessageType[Confidence].value == 0) + // Different origin node + reputationRecorder ! GetConfidence(replyTo.ref, randomKey().publicKey, 7, uuid6, 1000 msat) + assert(replyTo.expectMessageType[Confidence].value == 0) + // Very large HTLC + reputationRecorder ! GetConfidence(replyTo.ref, originNode, 7, uuid5, 100000000 msat) + assert(replyTo.expectMessageType[Confidence].value === 0.0 +- 0.001) + } + + test("trampoline relay") { f => + import f._ + + val (a, b, c) = (randomKey().publicKey, randomKey().publicKey, randomKey().publicKey) + + reputationRecorder ! GetTrampolineConfidence(replyTo.ref, Map(PeerEndorsement(a, 7) -> 2000.msat, PeerEndorsement(b, 7) -> 4000.msat, PeerEndorsement(c, 0) -> 6000.msat), uuid1) + assert(replyTo.expectMessageType[Confidence].value == 0) + reputationRecorder ! RecordTrampolineSuccess(Map(PeerEndorsement(a, 7) -> 1000.msat, PeerEndorsement(b, 7) -> 2000.msat, PeerEndorsement(c, 0) -> 3000.msat), uuid1) + reputationRecorder ! GetTrampolineConfidence(replyTo.ref, Map(PeerEndorsement(a, 7) -> 1000.msat, PeerEndorsement(c, 0) -> 1000.msat), uuid2) + assert(replyTo.expectMessageType[Confidence].value === (1.0 / 3) +- 0.001) + reputationRecorder ! GetTrampolineConfidence(replyTo.ref, Map(PeerEndorsement(a, 0) -> 1000.msat, PeerEndorsement(b, 7) -> 2000.msat), uuid3) + assert(replyTo.expectMessageType[Confidence].value == 0) + reputationRecorder ! RecordTrampolineFailure(Set(PeerEndorsement(a, 7), PeerEndorsement(c, 0)), uuid2) + reputationRecorder ! RecordTrampolineSuccess(Map(PeerEndorsement(a, 0) -> 1000.msat, PeerEndorsement(b, 7) -> 2000.msat), uuid3) + + reputationRecorder ! GetConfidence(replyTo.ref, a, 7, uuid4, 1000 msat) + assert(replyTo.expectMessageType[Confidence].value === (1.0 / 4) +- 0.001) + reputationRecorder ! GetConfidence(replyTo.ref, a, 0, uuid5, 1000 msat) + assert(replyTo.expectMessageType[Confidence].value === (1.0 / 3) +- 0.001) + reputationRecorder ! GetConfidence(replyTo.ref, b, 7, uuid6, 1000 msat) + assert(replyTo.expectMessageType[Confidence].value === (4.0 / 6) +- 0.001) + reputationRecorder ! GetConfidence(replyTo.ref, b, 0, uuid7, 1000 msat) + assert(replyTo.expectMessageType[Confidence].value == 0.0) + reputationRecorder ! GetConfidence(replyTo.ref, c, 0, uuid8, 1000 msat) + assert(replyTo.expectMessageType[Confidence].value === (3.0 / 6) +- 0.001) + } +} \ No newline at end of file diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/reputation/ReputationSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/reputation/ReputationSpec.scala new file mode 100644 index 0000000000..4ee8042ad6 --- /dev/null +++ b/eclair-core/src/test/scala/fr/acinq/eclair/reputation/ReputationSpec.scala @@ -0,0 +1,81 @@ +/* + * Copyright 2023 ACINQ SAS + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package fr.acinq.eclair.reputation + +import fr.acinq.eclair.reputation.Reputation._ +import fr.acinq.eclair.{MilliSatoshiLong, TimestampMilli} +import org.scalactic.Tolerance.convertNumericToPlusOrMinusWrapper +import org.scalatest.funsuite.AnyFunSuite + +import java.util.UUID +import scala.concurrent.duration.DurationInt + +class ReputationSpec extends AnyFunSuite { + val (uuid1, uuid2, uuid3, uuid4, uuid5, uuid6, uuid7) = (UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID()) + + test("basic") { + val r0 = Reputation.init(Config(enabled = true, 1 day, 1 second, 2)) + val (r1, c1) = r0.attempt(uuid1, 10000 msat) + assert(c1 == 0) + val r2 = r1.record(uuid1, isSuccess = true) + val (r3, c3) = r2.attempt(uuid2, 10000 msat) + assert(c3 === (1.0 / 3) +- 0.001) + val (r4, c4) = r3.attempt(uuid3, 10000 msat) + assert(c4 === (1.0 / 5) +- 0.001) + val r5 = r4.record(uuid2, isSuccess = true) + val r6 = r5.record(uuid3, isSuccess = true) + val (r7, c7) = r6.attempt(uuid4, 1 msat) + assert(c7 === 1.0 +- 0.001) + val (r8, c8) = r7.attempt(uuid5, 40000 msat) + assert(c8 === (3.0 / 11) +- 0.001) + val (r9, c9) = r8.attempt(uuid6, 10000 msat) + assert(c9 === (3.0 / 13) +- 0.001) + val r10 = r9.cancel(uuid5) + val r11 = r10.record(uuid6, isSuccess = false) + val (_, c12) = r11.attempt(uuid7, 10000 msat) + assert(c12 === (3.0 / 6) +- 0.001) + } + + test("long HTLC") { + val r0 = Reputation.init(Config(enabled = true, 1000 day, 1 second, 10)) + val (r1, c1) = r0.attempt(uuid1, 100000 msat, TimestampMilli(0)) + assert(c1 == 0) + val r2 = r1.record(uuid1, isSuccess = true, now = TimestampMilli(0)) + val (r3, c3) = r2.attempt(uuid2, 1000 msat, TimestampMilli(0)) + assert(c3 === (10.0 / 11) +- 0.001) + val r4 = r3.record(uuid2, isSuccess = false, now = TimestampMilli(0) + 100.seconds) + val (_, c5) = r4.attempt(uuid3, 0 msat, now = TimestampMilli(0) + 100.seconds) + assert(c5 === 0.5 +- 0.001) + } + + test("exponential decay") { + val r0 = Reputation.init(Config(enabled = true, 100 seconds, 1 second, 1)) + val (r1, _) = r0.attempt(uuid1, 1000 msat, TimestampMilli(0)) + val r2 = r1.record(uuid1, isSuccess = true, now = TimestampMilli(0)) + val (r3, c3) = r2.attempt(uuid2, 1000 msat, TimestampMilli(0)) + assert(c3 == 1.0 / 2) + val r4 = r3.record(uuid2, isSuccess = true, now = TimestampMilli(0)) + val (r5, c5) = r4.attempt(uuid3, 1000 msat, TimestampMilli(0)) + assert(c5 == 2.0 / 3) + val r6 = r5.record(uuid3, isSuccess = true, now = TimestampMilli(0)) + val (r7, c7) = r6.attempt(uuid4, 1000 msat, TimestampMilli(0) + 100.seconds) + assert(c7 == 1.5 / 2.5) + val r8 = r7.cancel(uuid4) + val (_, c9) = r8.attempt(uuid5, 1000 msat, TimestampMilli(0) + 1.hour) + assert(c9 < 0.000001) + } +} \ No newline at end of file From 9973d1ecaf8736ea9f989bbb58b3b4e7bc24b42a Mon Sep 17 00:00:00 2001 From: Thomas HUET Date: Thu, 8 Aug 2024 10:42:58 +0200 Subject: [PATCH 2/2] Reputation is recorded from channel events --- .../main/scala/fr/acinq/eclair/Setup.scala | 2 +- .../fr/acinq/eclair/channel/ChannelData.scala | 6 +- .../fr/acinq/eclair/channel/fsm/Channel.scala | 4 + .../eclair/payment/relay/ChannelRelay.scala | 18 +-- .../eclair/payment/relay/ChannelRelayer.scala | 2 +- .../eclair/payment/relay/NodeRelay.scala | 17 +-- .../eclair/payment/relay/NodeRelayer.scala | 2 +- .../acinq/eclair/reputation/Reputation.scala | 37 +++-- .../reputation/ReputationRecorder.scala | 133 ++++++++++++------ .../payment/relay/ChannelRelayerSpec.scala | 9 +- .../payment/relay/NodeRelayerSpec.scala | 11 +- .../eclair/payment/relay/RelayerSpec.scala | 2 +- .../reputation/ReputationRecorderSpec.scala | 4 +- 13 files changed, 136 insertions(+), 111 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/Setup.scala b/eclair-core/src/main/scala/fr/acinq/eclair/Setup.scala index b39ca56a9d..3d5ec1b325 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/Setup.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/Setup.scala @@ -364,7 +364,7 @@ class Setup(val datadir: File, triggerer = system.spawn(Behaviors.supervise(AsyncPaymentTriggerer()).onFailure(typed.SupervisorStrategy.resume), name = "async-payment-triggerer") peerReadyManager = system.spawn(Behaviors.supervise(PeerReadyManager()).onFailure(typed.SupervisorStrategy.restart), name = "peer-ready-manager") reputationRecorder_opt = if (nodeParams.relayParams.peerReputationConfig.enabled) { - Some(system.spawn(Behaviors.supervise(ReputationRecorder(nodeParams.relayParams.peerReputationConfig, Map.empty)).onFailure(typed.SupervisorStrategy.resume), name = "reputation-recorder")) + Some(system.spawn(Behaviors.supervise(ReputationRecorder(nodeParams.relayParams.peerReputationConfig)).onFailure(typed.SupervisorStrategy.resume), name = "reputation-recorder")) } else { None } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala index 982403d640..3d4428192a 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala @@ -26,7 +26,7 @@ import fr.acinq.eclair.channel.fund.{InteractiveTxBuilder, InteractiveTxSigningS import fr.acinq.eclair.io.Peer import fr.acinq.eclair.transactions.CommitmentSpec import fr.acinq.eclair.transactions.Transactions._ -import fr.acinq.eclair.wire.protocol.{ChannelAnnouncement, ChannelReady, ChannelReestablish, ChannelUpdate, ClosingSigned, CommitSig, FailureMessage, FundingCreated, FundingSigned, Init, LiquidityAds, OnionRoutingPacket, OpenChannel, OpenDualFundedChannel, Shutdown, SpliceInit, Stfu, TxSignatures, UpdateAddHtlc, UpdateFailHtlc, UpdateFailMalformedHtlc, UpdateFulfillHtlc} +import fr.acinq.eclair.wire.protocol.{ChannelAnnouncement, ChannelReady, ChannelReestablish, ChannelUpdate, ClosingSigned, CommitSig, FailureMessage, FundingCreated, FundingSigned, HtlcFailureMessage, Init, LiquidityAds, OnionRoutingPacket, OpenChannel, OpenDualFundedChannel, Shutdown, SpliceInit, Stfu, TxSignatures, UpdateAddHtlc, UpdateFailHtlc, UpdateFailMalformedHtlc, UpdateFulfillHtlc} import fr.acinq.eclair.{Alias, BlockHeight, CltvExpiry, CltvExpiryDelta, Features, InitFeature, MilliSatoshi, MilliSatoshiLong, RealShortChannelId, TimestampMilli, UInt64} import scodec.bits.ByteVector @@ -244,6 +244,10 @@ final case class CMD_GET_CHANNEL_STATE(replyTo: ActorRef) extends HasReplyToComm final case class CMD_GET_CHANNEL_DATA(replyTo: ActorRef) extends HasReplyToCommand final case class CMD_GET_CHANNEL_INFO(replyTo: akka.actor.typed.ActorRef[RES_GET_CHANNEL_INFO]) extends Command +case class OutgoingHtlcAdded(add: UpdateAddHtlc, upstream: Upstream.Hot, fee: MilliSatoshi) +case class OutgoingHtlcFailed(fail: HtlcFailureMessage) +case class OutgoingHtlcFulfilled(fulfill: UpdateFulfillHtlc) + /* 88888888b. 8888888888 .d8888b. 88888888b. ,ad8888ba, 888b 88 .d8888b. 8888888888 .d8888b. 88 "8b 88 d88P Y88b 88 "8b d8"' `"8b 8888b 88 d88P Y88b 88 d88P Y88b diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala index 056cbbed46..d47614abcb 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala @@ -440,6 +440,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with case Right((commitments1, add)) => if (c.commit) self ! CMD_SIGN() context.system.eventStream.publish(AvailableBalanceChanged(self, d.channelId, d.shortIds, commitments1)) + context.system.eventStream.publish(OutgoingHtlcAdded(add, c.origin.upstream, nodeFee(d.channelUpdate.relayFees, add.amountMsat))) handleCommandSuccess(c, d.copy(commitments = commitments1)) sending add case Left(cause) => handleAddHtlcCommandError(c, cause, Some(d.channelUpdate)) } @@ -466,6 +467,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with case Right((commitments1, origin, htlc)) => // we forward preimages as soon as possible to the upstream channel because it allows us to pull funds relayer ! RES_ADD_SETTLED(origin, htlc, HtlcResult.RemoteFulfill(fulfill)) + context.system.eventStream.publish(OutgoingHtlcFulfilled(fulfill)) stay() using d.copy(commitments = commitments1) case Left(cause) => handleLocalError(cause, d, Some(fulfill)) } @@ -499,12 +501,14 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with } case Event(fail: UpdateFailHtlc, d: DATA_NORMAL) => + context.system.eventStream.publish(OutgoingHtlcFailed(fail)) d.commitments.receiveFail(fail) match { case Right((commitments1, _, _)) => stay() using d.copy(commitments = commitments1) case Left(cause) => handleLocalError(cause, d, Some(fail)) } case Event(fail: UpdateFailMalformedHtlc, d: DATA_NORMAL) => + context.system.eventStream.publish(OutgoingHtlcFailed(fail)) d.commitments.receiveFailMalformed(fail) match { case Right((commitments1, _, _)) => stay() using d.copy(commitments = commitments1) case Left(cause) => handleLocalError(cause, d, Some(fail)) 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 cb3a9b7a94..bfd686ecf3 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 @@ -33,7 +33,7 @@ import fr.acinq.eclair.payment.Monitoring.{Metrics, Tags} import fr.acinq.eclair.payment.relay.Relayer.{OutgoingChannel, OutgoingChannelParams} import fr.acinq.eclair.payment.{ChannelPaymentRelayed, IncomingPaymentPacket} import fr.acinq.eclair.reputation.ReputationRecorder -import fr.acinq.eclair.reputation.ReputationRecorder.{CancelRelay, GetConfidence, RecordResult} +import fr.acinq.eclair.reputation.ReputationRecorder.GetConfidence import fr.acinq.eclair.wire.protocol.FailureMessageCodecs.createBadOnionFailure import fr.acinq.eclair.wire.protocol.PaymentOnion.IntermediatePayload import fr.acinq.eclair.wire.protocol._ @@ -65,7 +65,7 @@ object ChannelRelay { def apply(nodeParams: NodeParams, register: ActorRef, - reputationRecorder_opt: Option[typed.ActorRef[ReputationRecorder.ChannelRelayCommand]], + reputationRecorder_opt: Option[typed.ActorRef[GetConfidence]], channels: Map[ByteVector32, Relayer.OutgoingChannel], originNode: PublicKey, relayId: UUID, @@ -79,14 +79,14 @@ object ChannelRelay { val upstream = Upstream.Hot.Channel(r.add.removeUnknownTlvs(), TimestampMilli.now(), originNode) reputationRecorder_opt match { case Some(reputationRecorder) => - reputationRecorder ! GetConfidence(context.messageAdapter[ReputationRecorder.Confidence](confidence => WrappedConfidence(confidence.value)), originNode, r.add.endorsement, relayId, r.relayFeeMsat) + reputationRecorder ! GetConfidence(context.messageAdapter[ReputationRecorder.Confidence](confidence => WrappedConfidence(confidence.value)), upstream, r.relayFeeMsat) case None => val confidence = (r.add.endorsement + 0.5) / 8 context.self ! WrappedConfidence(confidence) } Behaviors.receiveMessagePartial { case WrappedConfidence(confidence) => - new ChannelRelay(nodeParams, register, reputationRecorder_opt, channels, r, upstream, confidence, context, relayId).start() + new ChannelRelay(nodeParams, register, channels, r, upstream, confidence, context).start() } } } @@ -128,13 +128,11 @@ object ChannelRelay { */ class ChannelRelay private(nodeParams: NodeParams, register: ActorRef, - reputationRecorder_opt: Option[typed.ActorRef[ReputationRecorder.ChannelRelayCommand]], channels: Map[ByteVector32, Relayer.OutgoingChannel], r: IncomingPaymentPacket.ChannelRelayPacket, upstream: Upstream.Hot.Channel, confidence: Double, - context: ActorContext[ChannelRelay.Command], - relayId: UUID) { + context: ActorContext[ChannelRelay.Command]) { import ChannelRelay._ @@ -200,7 +198,6 @@ class ChannelRelay private(nodeParams: NodeParams, case RelayFailure(cmdFail) => Metrics.recordPaymentRelayFailed(Tags.FailureType(cmdFail), Tags.RelayType.Channel) context.log.info("rejecting htlc reason={}", cmdFail.reason) - reputationRecorder_opt.foreach(_ ! CancelRelay(upstream.receivedFrom, r.add.endorsement, relayId)) safeSendAndStop(r.add.channelId, cmdFail) case RelayNeedsFunding(nextNodeId, cmdFail) => val cmd = Peer.ProposeOnTheFlyFunding(onTheFlyFundingResponseAdapter, r.amountToForward, r.add.paymentHash, r.outgoingCltv, r.nextPacket, nextBlindingKey_opt, upstream) @@ -220,7 +217,6 @@ class ChannelRelay private(nodeParams: NodeParams, context.log.warn(s"couldn't resolve downstream channel $channelId, failing htlc #${upstream.add.id}") val cmdFail = CMD_FAIL_HTLC(upstream.add.id, Right(UnknownNextPeer()), commit = true) Metrics.recordPaymentRelayFailed(Tags.FailureType(cmdFail), Tags.RelayType.Channel) - reputationRecorder_opt.foreach(_ ! CancelRelay(upstream.receivedFrom, r.add.endorsement, relayId)) safeSendAndStop(upstream.add.channelId, cmdFail) case WrappedAddResponse(addFailed: RES_ADD_FAILED[_]) => @@ -431,11 +427,9 @@ class ChannelRelay private(nodeParams: NodeParams, featureOk && liquidityIssue && relayParamsOk } - private def recordRelayDuration(isSuccess: Boolean): Unit = { - reputationRecorder_opt.foreach(_ ! RecordResult(upstream.receivedFrom, r.add.endorsement, relayId, isSuccess)) + private def recordRelayDuration(isSuccess: Boolean): Unit = Metrics.RelayedPaymentDuration .withTag(Tags.Relay, Tags.RelayType.Channel) .withTag(Tags.Success, isSuccess) .record((TimestampMilli.now() - upstream.receivedAt).toMillis, TimeUnit.MILLISECONDS) - } } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/ChannelRelayer.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/ChannelRelayer.scala index 590ca16e37..7cf29cc977 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/ChannelRelayer.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/ChannelRelayer.scala @@ -59,7 +59,7 @@ object ChannelRelayer { def apply(nodeParams: NodeParams, register: ActorRef, - reputationRecorder_opt: Option[typed.ActorRef[ReputationRecorder.ChannelRelayCommand]], + reputationRecorder_opt: Option[typed.ActorRef[ReputationRecorder.GetConfidence]], channels: Map[ByteVector32, Relayer.OutgoingChannel] = Map.empty, scid2channels: Map[ShortChannelId, ByteVector32] = Map.empty, node2channels: mutable.MultiDict[PublicKey, ByteVector32] = mutable.MultiDict.empty): Behavior[Command] = diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/NodeRelay.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/NodeRelay.scala index 472ad705dc..a0d26f78d1 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/NodeRelay.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/NodeRelay.scala @@ -40,6 +40,7 @@ import fr.acinq.eclair.payment.send.PaymentLifecycle.SendPaymentToNode import fr.acinq.eclair.payment.send._ import fr.acinq.eclair.router.Router.{ChannelHop, HopRelayParams, Route, RouteParams} import fr.acinq.eclair.reputation.ReputationRecorder +import fr.acinq.eclair.reputation.ReputationRecorder.GetTrampolineConfidence import fr.acinq.eclair.router.Router.RouteParams import fr.acinq.eclair.router.{BalanceTooLow, RouteNotFound} import fr.acinq.eclair.wire.protocol.PaymentOnion.IntermediatePayload @@ -91,7 +92,7 @@ object NodeRelay { def apply(nodeParams: NodeParams, parent: typed.ActorRef[NodeRelayer.Command], register: ActorRef, - reputationRecorder_opt: Option[typed.ActorRef[ReputationRecorder.TrampolineRelayCommand]], + reputationRecorder_opt: Option[typed.ActorRef[GetTrampolineConfidence]], relayId: UUID, nodeRelayPacket: NodeRelayPacket, outgoingPaymentFactory: OutgoingPaymentFactory, @@ -215,7 +216,7 @@ object NodeRelay { class NodeRelay private(nodeParams: NodeParams, parent: akka.actor.typed.ActorRef[NodeRelayer.Command], register: ActorRef, - reputationRecorder_opt: Option[typed.ActorRef[ReputationRecorder.TrampolineRelayCommand]], + reputationRecorder_opt: Option[typed.ActorRef[GetTrampolineConfidence]], relayId: UUID, paymentHash: ByteVector32, paymentSecret: ByteVector32, @@ -336,11 +337,8 @@ class NodeRelay private(nodeParams: NodeParams, // We only make one try when it's a direct payment to a wallet. val maxPaymentAttempts = if (walletNodeId_opt.isDefined) 1 else nodeParams.maxPaymentAttempts val totalFee = upstream.amountIn - payloadOut.amountToForward - val fees = upstream.received.foldLeft(Map.empty[ReputationRecorder.PeerEndorsement, MilliSatoshi])((fees, r) => - fees.updatedWith(ReputationRecorder.PeerEndorsement(r.receivedFrom, r.add.endorsement))(fee => - Some(fee.getOrElse(MilliSatoshi(0)) + r.add.amountMsat * totalFee.toLong / upstream.amountIn.toLong))) reputationRecorder_opt match { - case Some(reputationRecorder) => reputationRecorder ! ReputationRecorder.GetTrampolineConfidence(context.messageAdapter[ReputationRecorder.Confidence](confidence => WrappedConfidence(confidence.value)), fees, relayId) + case Some(reputationRecorder) => reputationRecorder ! GetTrampolineConfidence(context.messageAdapter(confidence => WrappedConfidence(confidence.value)), upstream, totalFee) case None => context.self ! WrappedConfidence((upstream.received.map(_.add.endorsement).min + 0.5) / 8) } Behaviors.receiveMessagePartial { @@ -396,16 +394,10 @@ class NodeRelay private(nodeParams: NodeParams, case WrappedPaymentSent(paymentSent) => context.log.debug("trampoline payment fully resolved downstream") success(upstream, fulfilledUpstream, paymentSent) - val totalFee = upstream.amountIn - paymentSent.amountWithFees - val fees = upstream.received.foldLeft(Map.empty[ReputationRecorder.PeerEndorsement, MilliSatoshi])((fees, r) => - fees.updatedWith(ReputationRecorder.PeerEndorsement(r.receivedFrom, r.add.endorsement))(fee => - Some(fee.getOrElse(MilliSatoshi(0)) + r.add.amountMsat * totalFee.toLong / upstream.amountIn.toLong))) - reputationRecorder_opt.foreach(_ ! ReputationRecorder.RecordTrampolineSuccess(fees, relayId)) recordRelayDuration(startedAt, isSuccess = true) stopping() case _: WrappedPaymentFailed if fulfilledUpstream => context.log.warn("trampoline payment failed downstream but was fulfilled upstream") - reputationRecorder_opt.foreach(_ ! ReputationRecorder.RecordTrampolineFailure(upstream.received.map(r => ReputationRecorder.PeerEndorsement(r.receivedFrom, r.add.endorsement)).toSet, relayId)) recordRelayDuration(startedAt, isSuccess = true) stopping() case WrappedPaymentFailed(PaymentFailed(_, _, failures, _)) => @@ -415,7 +407,6 @@ class NodeRelay private(nodeParams: NodeParams, attemptOnTheFlyFunding(upstream, walletNodeId, recipient, nextPayload, failures, startedAt) case _ => rejectPayment(upstream, translateError(nodeParams, failures, upstream, nextPayload)) - reputationRecorder_opt.foreach(_ ! ReputationRecorder.RecordTrampolineFailure(upstream.received.map(r => ReputationRecorder.PeerEndorsement(r.receivedFrom, r.add.endorsement)).toSet, relayId)) recordRelayDuration(startedAt, isSuccess = false) stopping() } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/NodeRelayer.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/NodeRelayer.scala index 4bff50e476..17cadc734d 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/NodeRelayer.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/NodeRelayer.scala @@ -60,7 +60,7 @@ object NodeRelayer { */ def apply(nodeParams: NodeParams, register: akka.actor.ActorRef, - reputationRecorder_opt: Option[ActorRef[ReputationRecorder.TrampolineRelayCommand]], + reputationRecorder_opt: Option[ActorRef[ReputationRecorder.GetTrampolineConfidence]], outgoingPaymentFactory: NodeRelay.OutgoingPaymentFactory, router: akka.actor.ActorRef, children: Map[PaymentKey, ActorRef[NodeRelay.Command]] = Map.empty): Behavior[Command] = diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/reputation/Reputation.scala b/eclair-core/src/main/scala/fr/acinq/eclair/reputation/Reputation.scala index f74c0aca67..fda3ea5d54 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/reputation/Reputation.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/reputation/Reputation.scala @@ -16,9 +16,10 @@ package fr.acinq.eclair.reputation +import fr.acinq.bitcoin.scalacompat.ByteVector32 +import fr.acinq.eclair.reputation.Reputation.HtlcId import fr.acinq.eclair.{MilliSatoshi, TimestampMilli} -import java.util.UUID import scala.concurrent.duration.FiniteDuration /** @@ -26,7 +27,7 @@ import scala.concurrent.duration.FiniteDuration */ /** - * Local reputation for a given incoming node, that should be track for each incoming endorsement level. + * Local reputation for a given incoming node, that should be tracked for each incoming endorsement level. * * @param pastWeight How much fees we would have collected in the past if all payments had succeeded (exponential moving average). * @param pastScore How much fees we have collected in the past (exponential moving average). @@ -36,51 +37,47 @@ import scala.concurrent.duration.FiniteDuration * @param maxRelayDuration Duration after which payments are penalized for staying pending too long. * @param pendingMultiplier How much to penalize pending payments. */ -case class Reputation(pastWeight: Double, pastScore: Double, lastSettlementAt: TimestampMilli, pending: Map[UUID, Reputation.PendingPayment], halfLife: FiniteDuration, maxRelayDuration: FiniteDuration, pendingMultiplier: Double) { +case class Reputation(pastWeight: Double, pastScore: Double, lastSettlementAt: TimestampMilli, pending: Map[HtlcId, Reputation.PendingPayment], halfLife: FiniteDuration, maxRelayDuration: FiniteDuration, pendingMultiplier: Double) { private def decay(now: TimestampMilli): Double = scala.math.pow(0.5, (now - lastSettlementAt) / halfLife) private def pendingWeight(now: TimestampMilli): Double = pending.values.map(_.weight(now, maxRelayDuration, pendingMultiplier)).sum /** - * Register a payment to relay and estimate the confidence that it will succeed. - * - * @return (updated reputation, confidence) + * Estimate the confidence that a payment will succeed. */ - def attempt(relayId: UUID, fee: MilliSatoshi, now: TimestampMilli = TimestampMilli.now()): (Reputation, Double) = { + def getConfidence(fee: MilliSatoshi, now: TimestampMilli = TimestampMilli.now()): Double = { val d = decay(now) - val newReputation = copy(pending = pending + (relayId -> Reputation.PendingPayment(fee, now))) - val confidence = d * pastScore / (d * pastWeight + newReputation.pendingWeight(now)) - (newReputation, confidence) + d * pastScore / (d * pastWeight + pendingWeight(now) + fee.toLong.toDouble * pendingMultiplier) } /** - * Mark a previously registered payment as failed without trying to relay it (usually because its confidence was too low). + * Register a pending relay. * * @return updated reputation */ - def cancel(relayId: UUID): Reputation = copy(pending = pending - relayId) + def attempt(htlcId: HtlcId, fee: MilliSatoshi, now: TimestampMilli = TimestampMilli.now()): Reputation = + copy(pending = pending + (htlcId -> Reputation.PendingPayment(fee, now))) /** * When a payment is settled, we record whether it succeeded and how long it took. * - * @param feeOverride When relaying trampoline payments, the actual fee is only known when the payment succeeds. This - * is used instead of the fee upper bound that was known when first attempting the relay. * @return updated reputation */ - def record(relayId: UUID, isSuccess: Boolean, feeOverride: Option[MilliSatoshi] = None, now: TimestampMilli = TimestampMilli.now()): Reputation = { - pending.get(relayId) match { + def record(htlcId: HtlcId, isSuccess: Boolean, now: TimestampMilli = TimestampMilli.now()): Reputation = { + pending.get(htlcId) match { case Some(p) => val d = decay(now) - val p1 = p.copy(fee = feeOverride.getOrElse(p.fee)) - val newWeight = d * pastWeight + p1.weight(now, maxRelayDuration, 1.0) - val newScore = d * pastScore + (if (isSuccess) p1.fee.toLong.toDouble else 0) - Reputation(newWeight, newScore, now, pending - relayId, halfLife, maxRelayDuration, pendingMultiplier) + val newWeight = d * pastWeight + p.weight(now, maxRelayDuration, if (isSuccess) 1.0 else 0.0) + val newScore = d * pastScore + (if (isSuccess) p.fee.toLong.toDouble else 0) + Reputation(newWeight, newScore, now, pending - htlcId, halfLife, maxRelayDuration, pendingMultiplier) case None => this } } } object Reputation { + case class HtlcId(channelId: ByteVector32, id: Long) + /** We're relaying that payment and are waiting for it to settle. */ case class PendingPayment(fee: MilliSatoshi, startedAt: TimestampMilli) { def weight(now: TimestampMilli, minDuration: FiniteDuration, multiplier: Double): Double = { diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/reputation/ReputationRecorder.scala b/eclair-core/src/main/scala/fr/acinq/eclair/reputation/ReputationRecorder.scala index 77982ebfbe..3cb7509e28 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/reputation/ReputationRecorder.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/reputation/ReputationRecorder.scala @@ -16,12 +16,19 @@ package fr.acinq.eclair.reputation -import akka.actor.typed.scaladsl.Behaviors +import akka.actor.typed.eventstream.EventStream +import akka.actor.typed.scaladsl.{ActorContext, Behaviors} import akka.actor.typed.{ActorRef, Behavior} import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey import fr.acinq.eclair.MilliSatoshi +import fr.acinq.eclair.channel.Upstream.Hot +import fr.acinq.eclair.channel.{OutgoingHtlcAdded, OutgoingHtlcFailed, OutgoingHtlcFulfilled, Upstream} +import fr.acinq.eclair.reputation.Reputation.HtlcId +import fr.acinq.eclair.wire.protocol.{UpdateFailHtlc, UpdateFailMalformedHtlc} +import ReputationRecorder._ + +import scala.collection.mutable -import java.util.UUID /** * Created by thomash on 21/07/2023. @@ -30,16 +37,11 @@ import java.util.UUID object ReputationRecorder { // @formatter:off sealed trait Command - - sealed trait ChannelRelayCommand extends Command - case class GetConfidence(replyTo: ActorRef[Confidence], originNode: PublicKey, endorsement: Int, relayId: UUID, fee: MilliSatoshi) extends ChannelRelayCommand - case class CancelRelay(originNode: PublicKey, endorsement: Int, relayId: UUID) extends ChannelRelayCommand - case class RecordResult(originNode: PublicKey, endorsement: Int, relayId: UUID, isSuccess: Boolean) extends ChannelRelayCommand - - sealed trait TrampolineRelayCommand extends Command - case class GetTrampolineConfidence(replyTo: ActorRef[Confidence], fees: Map[PeerEndorsement, MilliSatoshi], relayId: UUID) extends TrampolineRelayCommand - case class RecordTrampolineFailure(upstream: Set[PeerEndorsement], relayId: UUID) extends TrampolineRelayCommand - case class RecordTrampolineSuccess(fees: Map[PeerEndorsement, MilliSatoshi], relayId: UUID) extends TrampolineRelayCommand + case class GetConfidence(replyTo: ActorRef[Confidence], upstream: Upstream.Hot.Channel, fee: MilliSatoshi) extends Command + case class GetTrampolineConfidence(replyTo: ActorRef[Confidence], upstream: Upstream.Hot.Trampoline, fee: MilliSatoshi) extends Command + private case class WrappedOutgoingHtlcAdded(added: OutgoingHtlcAdded) extends Command + private case class WrappedOutgoingHtlcFailed(failed: OutgoingHtlcFailed) extends Command + private case class WrappedOutgoingHtlcFulfilled(fulfilled: OutgoingHtlcFulfilled) extends Command // @formatter:on /** @@ -48,43 +50,90 @@ object ReputationRecorder { */ case class PeerEndorsement(nodeId: PublicKey, endorsement: Int) + object PeerEndorsement { + def apply(channel: Upstream.Hot.Channel): PeerEndorsement = PeerEndorsement(channel.receivedFrom, channel.add.endorsement) + } + /** Confidence that the outgoing HTLC will succeed. */ case class Confidence(value: Double) - def apply(reputationConfig: Reputation.Config, reputations: Map[PeerEndorsement, Reputation]): Behavior[Command] = { + def apply(config: Reputation.Config): Behavior[Command] = + Behaviors.setup(context => { + context.system.eventStream ! EventStream.Subscribe(context.messageAdapter(WrappedOutgoingHtlcAdded)) + context.system.eventStream ! EventStream.Subscribe(context.messageAdapter(WrappedOutgoingHtlcFailed)) + context.system.eventStream ! EventStream.Subscribe(context.messageAdapter(WrappedOutgoingHtlcFulfilled)) + new ReputationRecorder(config, context).run() + }) +} + +class ReputationRecorder(config: Reputation.Config, context: ActorContext[ReputationRecorder.Command]) { + private val reputations: mutable.Map[PeerEndorsement, Reputation] = mutable.HashMap.empty.withDefaultValue(Reputation.init(config)) + private val pending: mutable.Map[HtlcId, Upstream.Hot] = mutable.HashMap.empty + + def run(): Behavior[Command] = Behaviors.receiveMessage { - case GetConfidence(replyTo, originNode, endorsement, relayId, fee) => - val (updatedReputation, confidence) = reputations.getOrElse(PeerEndorsement(originNode, endorsement), Reputation.init(reputationConfig)).attempt(relayId, fee) + case GetConfidence(replyTo, upstream, fee) => + val confidence = reputations(PeerEndorsement(upstream)).getConfidence(fee) replyTo ! Confidence(confidence) - ReputationRecorder(reputationConfig, reputations.updated(PeerEndorsement(originNode, endorsement), updatedReputation)) - case CancelRelay(originNode, endorsement, relayId) => - val updatedReputation = reputations.getOrElse(PeerEndorsement(originNode, endorsement), Reputation.init(reputationConfig)).cancel(relayId) - ReputationRecorder(reputationConfig, reputations.updated(PeerEndorsement(originNode, endorsement), updatedReputation)) - case RecordResult(originNode, endorsement, relayId, isSuccess) => - val updatedReputation = reputations.getOrElse(PeerEndorsement(originNode, endorsement), Reputation.init(reputationConfig)).record(relayId, isSuccess) - ReputationRecorder(reputationConfig, reputations.updated(PeerEndorsement(originNode, endorsement), updatedReputation)) - case GetTrampolineConfidence(replyTo, fees, relayId) => - val (confidence, updatedReputations) = fees.foldLeft((1.0, reputations)) { - case ((c, r), (peerEndorsement, fee)) => - val (updatedReputation, confidence) = reputations.getOrElse(peerEndorsement, Reputation.init(reputationConfig)).attempt(relayId, fee) - (c.min(confidence), r.updated(peerEndorsement, updatedReputation)) - } + Behaviors.same + + case GetTrampolineConfidence(replyTo, upstream, totalFee) => + val confidence = + upstream.received + .groupMapReduce(r => PeerEndorsement(r.receivedFrom, r.add.endorsement))(_.add.amountMsat)(_ + _) + .map { + case (peerEndorsement, amount) => + val fee = amount * totalFee.toLong / upstream.amountIn.toLong + reputations(peerEndorsement).getConfidence(fee) + } + .min replyTo ! Confidence(confidence) - ReputationRecorder(reputationConfig, updatedReputations) - case RecordTrampolineFailure(keys, relayId) => - val updatedReputations = keys.foldLeft(reputations) { - case (r, peerEndorsement) => - val updatedReputation = reputations.getOrElse(peerEndorsement, Reputation.init(reputationConfig)).record(relayId, isSuccess = false) - r.updated(peerEndorsement, updatedReputation) + Behaviors.same + + case WrappedOutgoingHtlcAdded(OutgoingHtlcAdded(add, upstream, fee)) => + val htlcId = HtlcId(add.channelId, add.id) + upstream match { + case channel: Hot.Channel => + reputations(PeerEndorsement(channel)) = reputations(PeerEndorsement(channel)).attempt(htlcId, fee) + pending += (htlcId -> upstream) + case trampoline: Hot.Trampoline => + trampoline.received.foreach(channel => + reputations(PeerEndorsement(channel)) = reputations(PeerEndorsement(channel)).attempt(htlcId, fee * channel.amountIn.toLong / trampoline.amountIn.toLong) + ) + pending += (htlcId -> upstream) + case _: Upstream.Local => () + } + Behaviors.same + + case WrappedOutgoingHtlcFailed(OutgoingHtlcFailed(fail)) => + val htlcId = fail match { + case UpdateFailHtlc(channelId, id, _, _) => HtlcId(channelId, id) + case UpdateFailMalformedHtlc(channelId, id, _, _, _) => HtlcId(channelId, id) + } + pending.get(htlcId) match { + case Some(channel: Hot.Channel) => + reputations(PeerEndorsement(channel)) = reputations(PeerEndorsement(channel)).record(htlcId, isSuccess = false) + case Some(trampoline: Hot.Trampoline) => + trampoline.received.foreach(channel => + reputations(PeerEndorsement(channel)) = reputations(PeerEndorsement(channel)).record(htlcId, isSuccess = false) + ) + case _ => () } - ReputationRecorder(reputationConfig, updatedReputations) - case RecordTrampolineSuccess(fees, relayId) => - val updatedReputations = fees.foldLeft(reputations) { - case (r, (peerEndorsement, fee)) => - val updatedReputation = reputations.getOrElse(peerEndorsement, Reputation.init(reputationConfig)).record(relayId, isSuccess = true, Some(fee)) - r.updated(peerEndorsement, updatedReputation) + pending -= htlcId + Behaviors.same + + case WrappedOutgoingHtlcFulfilled(OutgoingHtlcFulfilled(fulfill)) => + val htlcId = HtlcId(fulfill.channelId, fulfill.id) + pending.get(htlcId) match { + case Some(channel: Hot.Channel) => + reputations(PeerEndorsement(channel)) = reputations(PeerEndorsement(channel)).record(htlcId, isSuccess = true) + case Some(trampoline: Hot.Trampoline) => + trampoline.received.foreach(channel => + reputations(PeerEndorsement(channel)) = reputations(PeerEndorsement(channel)).record(htlcId, isSuccess = true) + ) + case _ => () } - ReputationRecorder(reputationConfig, updatedReputations) + pending -= htlcId + Behaviors.same } - } } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/ChannelRelayerSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/ChannelRelayerSpec.scala index 34cbe20f55..47a32e6b2b 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/ChannelRelayerSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/ChannelRelayerSpec.scala @@ -92,7 +92,6 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a val fwd = register.expectMessageType[Register.Forward[channel.Command]] assert(fwd.message == cmd) assert(fwd.channelId == channelId) - reputationRecorder.expectMessageType[ReputationRecorder.CancelRelay] fwd } @@ -111,7 +110,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a import f._ val getConfidence = reputationRecorder.expectMessageType[ReputationRecorder.GetConfidence] - assert(getConfidence.originNode == TestConstants.Alice.nodeParams.nodeId) + assert(getConfidence.upstream.receivedFrom == TestConstants.Alice.nodeParams.nodeId) getConfidence.replyTo ! ReputationRecorder.Confidence(value) } @@ -469,8 +468,6 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a assert(fail.onionHash == Sphinx.hash(r.add.onionRoutingPacket)) assert(fail.failureCode == InvalidOnionBlinding(Sphinx.hash(r.add.onionRoutingPacket)).code) } - - reputationRecorder.expectMessageType[ReputationRecorder.CancelRelay] } } @@ -719,7 +716,6 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a val fwdFail = register.expectMessageType[Register.Forward[channel.Command]] assert(fwdFail.message == testCase.cmd) assert(fwdFail.channelId == r.add.channelId) - assert(!reputationRecorder.expectMessageType[ReputationRecorder.RecordResult].isSuccess) } } @@ -761,7 +757,6 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a assert(fail.onionHash == Sphinx.hash(r.add.onionRoutingPacket)) assert(fail.failureCode == InvalidOnionBlinding(Sphinx.hash(r.add.onionRoutingPacket)).code) } - assert(!reputationRecorder.expectMessageType[ReputationRecorder.RecordResult].isSuccess) } } } @@ -800,8 +795,6 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a val paymentRelayed = eventListener.expectMessageType[ChannelPaymentRelayed] assert(paymentRelayed.copy(startedAt = 0 unixms, settledAt = 0 unixms) == ChannelPaymentRelayed(r.add.amountMsat, r.amountToForward, r.add.paymentHash, r.add.channelId, channelId1, startedAt = 0 unixms, settledAt = 0 unixms)) - - assert(reputationRecorder.expectMessageType[ReputationRecorder.RecordResult].isSuccess) } } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/NodeRelayerSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/NodeRelayerSpec.scala index 978c87ddb0..8f4cdc1d78 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/NodeRelayerSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/NodeRelayerSpec.scala @@ -46,7 +46,7 @@ import fr.acinq.eclair.payment.send.{BlindedRecipient, ClearRecipient} import fr.acinq.eclair.router.Router.{ChannelHop, HopRelayParams, RouteRequest} import fr.acinq.eclair.router.{BalanceTooLow, BlindedRouteCreation, RouteNotFound, Router} import fr.acinq.eclair.reputation.ReputationRecorder -import fr.acinq.eclair.reputation.ReputationRecorder.{Confidence, GetTrampolineConfidence, RecordTrampolineFailure, RecordTrampolineSuccess} +import fr.acinq.eclair.reputation.ReputationRecorder.{Confidence, GetTrampolineConfidence} import fr.acinq.eclair.wire.protocol.OfferTypes._ import fr.acinq.eclair.wire.protocol.PaymentOnion.{FinalPayload, IntermediatePayload} import fr.acinq.eclair.wire.protocol.RouteBlindingEncryptedDataCodecs.blindedRouteDataCodec @@ -395,7 +395,6 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl assert(relayEvent.incoming.map(p => (p.amount, p.channelId)).toSet == incomingAsyncPayment.map(i => (i.add.amountMsat, i.add.channelId)).toSet) assert(relayEvent.outgoing.nonEmpty) parent.expectMessageType[NodeRelayer.RelayComplete] - reputationRecorder.expectMessageType[RecordTrampolineSuccess] register.expectNoMessage(100 millis) } @@ -486,7 +485,6 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl assert(fwd.message == CMD_FAIL_HTLC(p.add.id, Right(TrampolineFeeInsufficient()), commit = true)) } - reputationRecorder.expectMessageType[RecordTrampolineFailure] register.expectNoMessage(100 millis) eventListener.expectNoMessage(100 millis) } @@ -514,7 +512,6 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl assert(fwd.message == CMD_FAIL_HTLC(p.add.id, Right(TemporaryNodeFailure()), commit = true)) } - reputationRecorder.expectMessageType[RecordTrampolineFailure] register.expectNoMessage(100 millis) } @@ -540,7 +537,6 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl assert(fwd.message == CMD_FAIL_HTLC(p.add.id, Right(TrampolineFeeInsufficient()), commit = true)) } - reputationRecorder.expectMessageType[RecordTrampolineFailure] register.expectNoMessage(100 millis) } @@ -565,7 +561,6 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl assert(fwd.message == CMD_FAIL_HTLC(p.add.id, Right(FinalIncorrectHtlcAmount(42 msat)), commit = true)) } - reputationRecorder.expectMessageType[RecordTrampolineFailure] register.expectNoMessage(100 millis) } @@ -623,7 +618,6 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl validateRelayEvent(relayEvent) assert(relayEvent.incoming.map(p => (p.amount, p.channelId)).toSet == incomingMultiPart.map(i => (i.add.amountMsat, i.add.channelId)).toSet) assert(relayEvent.outgoing.nonEmpty) - reputationRecorder.expectMessageType[RecordTrampolineSuccess] parent.expectMessageType[NodeRelayer.RelayComplete] register.expectNoMessage(100 millis) } @@ -655,7 +649,6 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl validateRelayEvent(relayEvent) assert(relayEvent.incoming.map(p => (p.amount, p.channelId)) == Seq((incomingSinglePart.add.amountMsat, incomingSinglePart.add.channelId))) assert(relayEvent.outgoing.nonEmpty) - reputationRecorder.expectMessageType[RecordTrampolineSuccess] parent.expectMessageType[NodeRelayer.RelayComplete] register.expectNoMessage(100 millis) } @@ -779,7 +772,6 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl validateRelayEvent(relayEvent) assert(relayEvent.incoming.map(p => (p.amount, p.channelId)) == incomingMultiPart.map(i => (i.add.amountMsat, i.add.channelId))) assert(relayEvent.outgoing.nonEmpty) - reputationRecorder.expectMessageType[RecordTrampolineSuccess] parent.expectMessageType[NodeRelayer.RelayComplete] register.expectNoMessage(100 millis) } @@ -826,7 +818,6 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl validateRelayEvent(relayEvent) assert(relayEvent.incoming.map(p => (p.amount, p.channelId)) == incomingMultiPart.map(i => (i.add.amountMsat, i.add.channelId))) assert(relayEvent.outgoing.length == 1) - reputationRecorder.expectMessageType[RecordTrampolineSuccess] parent.expectMessageType[NodeRelayer.RelayComplete] register.expectNoMessage(100 millis) } 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 4b374fa59a..406b6bd566 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 @@ -75,7 +75,7 @@ class RelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat import f._ val getConfidence = reputationRecorder.expectMessageType[ReputationRecorder.GetConfidence] - assert(getConfidence.originNode == TestConstants.Alice.nodeParams.nodeId) + assert(getConfidence.upstream.receivedFrom == TestConstants.Alice.nodeParams.nodeId) getConfidence.replyTo ! ReputationRecorder.Confidence(value) } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/reputation/ReputationRecorderSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/reputation/ReputationRecorderSpec.scala index 7228b3372d..ec47d15444 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/reputation/ReputationRecorderSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/reputation/ReputationRecorderSpec.scala @@ -20,7 +20,9 @@ import akka.actor.testkit.typed.scaladsl.{ScalaTestWithActorTestKit, TestProbe} import akka.actor.typed.ActorRef import com.typesafe.config.ConfigFactory import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey +import fr.acinq.eclair.channel.Upstream import fr.acinq.eclair.reputation.ReputationRecorder._ +import fr.acinq.eclair.wire.protocol.UpdateAddHtlc import fr.acinq.eclair.{MilliSatoshiLong, randomKey} import org.scalatest.Outcome import org.scalatest.funsuite.FixtureAnyFunSuiteLike @@ -37,7 +39,7 @@ class ReputationRecorderSpec extends ScalaTestWithActorTestKit(ConfigFactory.loa override def withFixture(test: OneArgTest): Outcome = { val config = Reputation.Config(enabled = true, 1 day, 10 seconds, 2) val replyTo = TestProbe[Confidence]("confidence") - val reputationRecorder = testKit.spawn(ReputationRecorder(config, Map.empty)) + val reputationRecorder = testKit.spawn(ReputationRecorder(config)) withFixture(test.toNoArgTest(FixtureParam(config, reputationRecorder.ref, replyTo))) }