From 159722b55a42bfead8d87781713118eba7f50e9d Mon Sep 17 00:00:00 2001 From: t-bast Date: Fri, 9 Jan 2026 10:51:53 +0100 Subject: [PATCH 1/3] Add duration information to payment events We now record start and end time for payment events, and for remote failures, which lets us easily identify slower paths and faulty nodes. We properly track the settlement of each individual payment now. We also clean-up field names and harmonize those events. Note that we don't update the DB schema at that point: this will be done in another PR that batches all DB schema changes. --- .../fr/acinq/eclair/channel/fsm/Channel.scala | 2 +- .../acinq/eclair/crypto/WeakEntropyPool.scala | 2 +- .../scala/fr/acinq/eclair/db/PaymentsDb.scala | 4 +- .../fr/acinq/eclair/db/pg/PgAuditDb.scala | 17 +-- .../fr/acinq/eclair/db/pg/PgPaymentsDb.scala | 4 +- .../eclair/db/sqlite/SqliteAuditDb.scala | 17 +-- .../eclair/db/sqlite/SqlitePaymentsDb.scala | 4 +- .../acinq/eclair/json/JsonSerializers.scala | 4 +- .../fr/acinq/eclair/payment/Monitoring.scala | 2 +- .../acinq/eclair/payment/PaymentEvents.scala | 68 +++++----- .../payment/receive/MultiPartHandler.scala | 10 +- .../eclair/payment/relay/ChannelRelay.scala | 9 +- .../eclair/payment/relay/NodeRelay.scala | 16 +-- .../relay/PostRestartHtlcCleaner.scala | 14 +- .../acinq/eclair/payment/send/Autoprobe.scala | 2 +- .../send/MultiPartPaymentLifecycle.scala | 16 +-- .../payment/send/PaymentInitiator.scala | 18 +-- .../payment/send/PaymentLifecycle.scala | 39 +++--- .../send/TrampolinePaymentLifecycle.scala | 21 +-- .../fr/acinq/eclair/db/AuditDbSpec.scala | 12 +- .../fr/acinq/eclair/db/PaymentsDbSpec.scala | 18 +-- .../integration/PaymentIntegrationSpec.scala | 8 +- .../ZeroConfAliasIntegrationSpec.scala | 6 +- .../eclair/json/JsonSerializersSpec.scala | 6 +- .../eclair/payment/MultiPartHandlerSpec.scala | 59 ++++----- .../MultiPartPaymentLifecycleSpec.scala | 108 ++++++++-------- .../eclair/payment/PaymentInitiatorSpec.scala | 14 +- .../eclair/payment/PaymentLifecycleSpec.scala | 122 +++++++++++------- .../payment/relay/ChannelRelayerSpec.scala | 25 ++-- .../payment/relay/NodeRelayerSpec.scala | 54 ++++---- 30 files changed, 375 insertions(+), 326 deletions(-) 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 f3e10e09dd..c6e138a759 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 @@ -2276,7 +2276,7 @@ class Channel(val nodeParams: NodeParams, val channelKeys: ChannelKeys, val wall Closing .onChainOutgoingHtlcs(d.commitments.latest.localCommit, d.commitments.latest.remoteCommit, d.commitments.latest.nextRemoteCommit_opt, tx) .map(add => (add, d.commitments.originChannels.get(add.id).map(_.upstream).collect { case Upstream.Local(id) => id })) // we resolve the payment id if this was a local payment - .collect { case (add, Some(id)) => context.system.eventStream.publish(PaymentSettlingOnChain(id, amount = add.amountMsat, add.paymentHash)) } + .collect { case (add, Some(id)) => context.system.eventStream.publish(PaymentSettlingOnChain(id, d.channelId, add.amountMsat, add.paymentHash)) } // finally, if one of the unilateral closes is done, we move to CLOSED state, otherwise we stay() Closing.isClosed(d1, Some(tx)) match { case Some(closingType) => diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/crypto/WeakEntropyPool.scala b/eclair-core/src/main/scala/fr/acinq/eclair/crypto/WeakEntropyPool.scala index cd34acc589..303b3efe70 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/crypto/WeakEntropyPool.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/crypto/WeakEntropyPool.scala @@ -57,7 +57,7 @@ object WeakEntropyPool { def apply(collector: EntropyCollector): Behavior[Command] = { Behaviors.setup { context => context.system.eventStream ! EventStream.Subscribe(context.messageAdapter[NewBlock](e => WrappedNewBlock(e.blockId))) - context.system.eventStream ! EventStream.Subscribe(context.messageAdapter[ChannelPaymentRelayed](e => WrappedPaymentRelayed(e.paymentHash, e.timestamp))) + context.system.eventStream ! EventStream.Subscribe(context.messageAdapter[ChannelPaymentRelayed](e => WrappedPaymentRelayed(e.paymentHash, e.settledAt))) context.system.eventStream ! EventStream.Subscribe(context.messageAdapter[PeerConnected](e => WrappedPeerConnected(e.nodeId))) context.system.eventStream ! EventStream.Subscribe(context.messageAdapter[NodeUpdated](e => WrappedNodeUpdated(e.ann.signature))) context.system.eventStream ! EventStream.Subscribe(context.messageAdapter[ChannelSignatureReceived](e => WrappedChannelSignature(e.commitments.latest.localCommit.remoteSig))) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/db/PaymentsDb.scala b/eclair-core/src/main/scala/fr/acinq/eclair/db/PaymentsDb.scala index 53ce18f826..6e0747ba5d 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/db/PaymentsDb.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/db/PaymentsDb.scala @@ -249,8 +249,8 @@ object FailureType extends Enumeration { object FailureSummary { def apply(f: PaymentFailure): FailureSummary = f match { case LocalFailure(_, route, t) => FailureSummary(FailureType.LOCAL, t.getMessage, route.map(h => HopSummary(h)).toList, route.headOption.map(_.nodeId)) - case RemoteFailure(_, route, e) => FailureSummary(FailureType.REMOTE, e.failureMessage.message, route.map(h => HopSummary(h)).toList, Some(e.originNode)) - case UnreadableRemoteFailure(_, route, _, _) => FailureSummary(FailureType.UNREADABLE_REMOTE, "could not decrypt failure onion", route.map(h => HopSummary(h)).toList, None) + case RemoteFailure(_, route, e, _, _) => FailureSummary(FailureType.REMOTE, e.failureMessage.message, route.map(h => HopSummary(h)).toList, Some(e.originNode)) + case UnreadableRemoteFailure(_, route, _, _, _, _) => FailureSummary(FailureType.UNREADABLE_REMOTE, "could not decrypt failure onion", route.map(h => HopSummary(h)).toList, None) } } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/db/pg/PgAuditDb.scala b/eclair-core/src/main/scala/fr/acinq/eclair/db/pg/PgAuditDb.scala index 8eceb188d5..89277fd1fd 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/db/pg/PgAuditDb.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/db/pg/PgAuditDb.scala @@ -215,7 +215,7 @@ class PgAuditDb(implicit ds: DataSource) extends AuditDb with Logging { statement.setString(7, e.paymentPreimage.toHex) statement.setString(8, e.recipientNodeId.value.toHex) statement.setString(9, p.toChannelId.toHex) - statement.setTimestamp(10, p.timestamp.toSqlTimestamp) + statement.setTimestamp(10, p.settledAt.toSqlTimestamp) statement.addBatch() }) statement.executeBatch() @@ -230,7 +230,7 @@ class PgAuditDb(implicit ds: DataSource) extends AuditDb with Logging { statement.setLong(1, p.amount.toLong) statement.setString(2, e.paymentHash.toHex) statement.setString(3, p.fromChannelId.toHex) - statement.setTimestamp(4, p.timestamp.toSqlTimestamp) + statement.setTimestamp(4, p.receivedAt.toSqlTimestamp) statement.addBatch() }) statement.executeBatch() @@ -251,7 +251,7 @@ class PgAuditDb(implicit ds: DataSource) extends AuditDb with Logging { statement.setString(1, e.paymentHash.toHex) statement.setLong(2, nextTrampolineAmount.toLong) statement.setString(3, nextTrampolineNodeId.value.toHex) - statement.setTimestamp(4, e.timestamp.toSqlTimestamp) + statement.setTimestamp(4, e.settledAt.toSqlTimestamp) statement.executeUpdate() } // trampoline relayed payments do MPP aggregation and may have M inputs and N outputs @@ -362,12 +362,13 @@ class PgAuditDb(implicit ds: DataSource) extends AuditDb with Logging { MilliSatoshi(rs.getLong("fees_msat")), rs.getByteVector32FromHex("to_channel_id"), None, // we don't store the route in the audit DB - TimestampMilli.fromSqlTimestamp(rs.getTimestamp("timestamp"))) + // TODO: store startedAt when updating the DB schema instead of duplicating settledAt. + startedAt = TimestampMilli.fromSqlTimestamp(rs.getTimestamp("timestamp")), + settledAt = TimestampMilli.fromSqlTimestamp(rs.getTimestamp("timestamp"))) val sent = sentByParentId.get(parentId) match { case Some(s) => s.copy(parts = s.parts :+ part) case None => PaymentSent( parentId, - rs.getByteVector32FromHex("payment_hash"), rs.getByteVector32FromHex("payment_preimage"), MilliSatoshi(rs.getLong("recipient_amount_msat")), PublicKey(rs.getByteVectorFromHex("recipient_node_id")), @@ -375,7 +376,7 @@ class PgAuditDb(implicit ds: DataSource) extends AuditDb with Logging { None) } sentByParentId + (parentId -> sent) - }.values.toSeq.sortBy(_.timestamp) + }.values.toSeq.sortBy(_.settledAt) paginated_opt match { case Some(paginated) => result.slice(paginated.skip, paginated.skip + paginated.count) case None => result @@ -400,7 +401,7 @@ class PgAuditDb(implicit ds: DataSource) extends AuditDb with Logging { case None => PaymentReceived(paymentHash, Seq(part)) } receivedByHash + (paymentHash -> received) - }.values.toSeq.sortBy(_.timestamp) + }.values.toSeq.sortBy(_.settledAt) paginated_opt match { case Some(paginated) => result.slice(paginated.skip, paginated.skip + paginated.count) case None => result @@ -454,7 +455,7 @@ class PgAuditDb(implicit ds: DataSource) extends AuditDb with Logging { Seq(OnTheFlyFundingPaymentRelayed(paymentHash, incoming, outgoing)) case _ => Nil } - }.toSeq.sortBy(_.timestamp) + }.toSeq.sortBy(_.settledAt) paginated_opt match { case Some(paginated) => result.slice(paginated.skip, paginated.skip + paginated.count) case None => result diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/db/pg/PgPaymentsDb.scala b/eclair-core/src/main/scala/fr/acinq/eclair/db/pg/PgPaymentsDb.scala index 27af66e55b..2be4ddc1c5 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/db/pg/PgPaymentsDb.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/db/pg/PgPaymentsDb.scala @@ -140,7 +140,7 @@ class PgPaymentsDb(implicit ds: DataSource, lock: PgLock) extends PaymentsDb wit withLock { pg => using(pg.prepareStatement("UPDATE payments.sent SET (completed_at, payment_preimage, fees_msat, payment_route) = (?, ?, ?, ?) WHERE id = ? AND completed_at IS NULL")) { statement => paymentResult.parts.foreach(p => { - statement.setTimestamp(1, p.timestamp.toSqlTimestamp) + statement.setTimestamp(1, p.settledAt.toSqlTimestamp) statement.setString(2, paymentResult.paymentPreimage.toHex) statement.setLong(3, p.feesPaid.toLong) statement.setBytes(4, encodeRoute(p.route.getOrElse(Nil).map(h => HopSummary(h)).toList)) @@ -155,7 +155,7 @@ class PgPaymentsDb(implicit ds: DataSource, lock: PgLock) extends PaymentsDb wit override def updateOutgoingPayment(paymentResult: PaymentFailed): Unit = withMetrics("payments/update-outgoing-failed", DbBackends.Postgres) { withLock { pg => using(pg.prepareStatement("UPDATE payments.sent SET (completed_at, failures) = (?, ?) WHERE id = ? AND completed_at IS NULL")) { statement => - statement.setTimestamp(1, paymentResult.timestamp.toSqlTimestamp) + statement.setTimestamp(1, paymentResult.settledAt.toSqlTimestamp) statement.setBytes(2, encodeFailures(paymentResult.failures.map(f => FailureSummary(f)).toList)) statement.setString(3, paymentResult.id.toString) if (statement.executeUpdate() == 0) throw new IllegalArgumentException(s"Tried to mark an outgoing payment as failed but already in final status (id=${paymentResult.id})") diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteAuditDb.scala b/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteAuditDb.scala index e5cce5be67..dc2bd6e917 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteAuditDb.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteAuditDb.scala @@ -207,7 +207,7 @@ class SqliteAuditDb(val sqlite: Connection) extends AuditDb with Logging { statement.setBytes(7, e.paymentPreimage.toArray) statement.setBytes(8, e.recipientNodeId.value.toArray) statement.setBytes(9, p.toChannelId.toArray) - statement.setLong(10, p.timestamp.toLong) + statement.setLong(10, p.settledAt.toLong) statement.addBatch() }) statement.executeBatch() @@ -220,7 +220,7 @@ class SqliteAuditDb(val sqlite: Connection) extends AuditDb with Logging { statement.setLong(1, p.amount.toLong) statement.setBytes(2, e.paymentHash.toArray) statement.setBytes(3, p.fromChannelId.toArray) - statement.setLong(4, p.timestamp.toLong) + statement.setLong(4, p.receivedAt.toLong) statement.addBatch() }) statement.executeBatch() @@ -239,7 +239,7 @@ class SqliteAuditDb(val sqlite: Connection) extends AuditDb with Logging { statement.setBytes(1, e.paymentHash.toArray) statement.setLong(2, nextTrampolineAmount.toLong) statement.setBytes(3, nextTrampolineNodeId.value.toArray) - statement.setLong(4, e.timestamp.toLong) + statement.setLong(4, e.settledAt.toLong) statement.executeUpdate() } // trampoline relayed payments do MPP aggregation and may have M inputs and N outputs @@ -336,12 +336,13 @@ class SqliteAuditDb(val sqlite: Connection) extends AuditDb with Logging { MilliSatoshi(rs.getLong("fees_msat")), rs.getByteVector32("to_channel_id"), None, // we don't store the route in the audit DB - TimestampMilli(rs.getLong("timestamp"))) + // TODO: store startedAt when updating the DB schema instead of duplicating settledAt. + startedAt = TimestampMilli(rs.getLong("timestamp")), + settledAt = TimestampMilli(rs.getLong("timestamp"))) val sent = sentByParentId.get(parentId) match { case Some(s) => s.copy(parts = s.parts :+ part) case None => PaymentSent( parentId, - rs.getByteVector32("payment_hash"), rs.getByteVector32("payment_preimage"), MilliSatoshi(rs.getLong("recipient_amount_msat")), PublicKey(rs.getByteVector("recipient_node_id")), @@ -349,7 +350,7 @@ class SqliteAuditDb(val sqlite: Connection) extends AuditDb with Logging { None) } sentByParentId + (parentId -> sent) - }.values.toSeq.sortBy(_.timestamp) + }.values.toSeq.sortBy(_.settledAt) paginated_opt match { case Some(paginated) => result.slice(paginated.skip, paginated.skip + paginated.count) case None => result @@ -372,7 +373,7 @@ class SqliteAuditDb(val sqlite: Connection) extends AuditDb with Logging { case None => PaymentReceived(paymentHash, Seq(part)) } receivedByHash + (paymentHash -> received) - }.values.toSeq.sortBy(_.timestamp) + }.values.toSeq.sortBy(_.settledAt) paginated_opt match { case Some(paginated) => result.slice(paginated.skip, paginated.skip + paginated.count) case None => result @@ -425,7 +426,7 @@ class SqliteAuditDb(val sqlite: Connection) extends AuditDb with Logging { Seq(OnTheFlyFundingPaymentRelayed(paymentHash, incoming, outgoing)) case _ => Nil } - }.toSeq.sortBy(_.timestamp) + }.toSeq.sortBy(_.settledAt) paginated_opt match { case Some(paginated) => result.slice(paginated.skip, paginated.skip + paginated.count) case None => result diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqlitePaymentsDb.scala b/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqlitePaymentsDb.scala index d08008388a..27c0587b45 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqlitePaymentsDb.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqlitePaymentsDb.scala @@ -168,7 +168,7 @@ class SqlitePaymentsDb(val sqlite: Connection) extends PaymentsDb with Logging { override def updateOutgoingPayment(paymentResult: PaymentSent): Unit = withMetrics("payments/update-outgoing-sent", DbBackends.Sqlite) { using(sqlite.prepareStatement("UPDATE sent_payments SET (completed_at, payment_preimage, fees_msat, payment_route) = (?, ?, ?, ?) WHERE id = ? AND completed_at IS NULL")) { statement => paymentResult.parts.foreach(p => { - statement.setLong(1, p.timestamp.toLong) + statement.setLong(1, p.settledAt.toLong) statement.setBytes(2, paymentResult.paymentPreimage.toArray) statement.setLong(3, p.feesPaid.toLong) statement.setBytes(4, encodeRoute(p.route.getOrElse(Nil).map(h => HopSummary(h)).toList)) @@ -181,7 +181,7 @@ class SqlitePaymentsDb(val sqlite: Connection) extends PaymentsDb with Logging { override def updateOutgoingPayment(paymentResult: PaymentFailed): Unit = withMetrics("payments/update-outgoing-failed", DbBackends.Sqlite) { using(sqlite.prepareStatement("UPDATE sent_payments SET (completed_at, failures) = (?, ?) WHERE id = ? AND completed_at IS NULL")) { statement => - statement.setLong(1, paymentResult.timestamp.toLong) + statement.setLong(1, paymentResult.settledAt.toLong) statement.setBytes(2, encodeFailures(paymentResult.failures.map(f => FailureSummary(f)).toList)) statement.setString(3, paymentResult.id.toString) if (statement.executeUpdate() == 0) throw new IllegalArgumentException(s"Tried to mark an outgoing payment as failed but already in final status (id=${paymentResult.id})") diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/json/JsonSerializers.scala b/eclair-core/src/main/scala/fr/acinq/eclair/json/JsonSerializers.scala index f37d66e595..30a02ac999 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/json/JsonSerializers.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/json/JsonSerializers.scala @@ -375,7 +375,7 @@ object PaymentFailedSummarySerializer extends ConvertClassSerializer[PaymentFail val route = f.route.map(_.nodeId) ++ f.route.lastOption.map(_.nextNodeId) val message = f match { case LocalFailure(_, _, t) => t.getMessage - case RemoteFailure(_, _, Sphinx.DecryptedFailurePacket(origin, _, failureMessage)) => s"$origin returned: ${failureMessage.message}" + case RemoteFailure(_, _, Sphinx.DecryptedFailurePacket(origin, _, failureMessage), _, _) => s"$origin returned: ${failureMessage.message}" case _: UnreadableRemoteFailure => "unreadable remote failure" } PaymentFailureSummaryJson(f.amount, route, message) @@ -682,8 +682,8 @@ object CustomTypeHints { classOf[PaymentSent] -> "payment-sent", classOf[ChannelPaymentRelayed] -> "payment-relayed", classOf[TrampolinePaymentRelayed] -> "trampoline-payment-relayed", + classOf[OnTheFlyFundingPaymentRelayed] -> "on-the-fly-funding-payment-relayed", classOf[PaymentReceived] -> "payment-received", - classOf[PaymentSettlingOnChain] -> "payment-settling-onchain", classOf[PaymentFailed] -> "payment-failed", )) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/Monitoring.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/Monitoring.scala index 3ffa7e0adf..5ccba392c9 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/Monitoring.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/Monitoring.scala @@ -135,7 +135,7 @@ object Monitoring { def apply(pf: PaymentFailure): String = pf match { case LocalFailure(_, _, t) => t.getClass.getSimpleName - case RemoteFailure(_, _, e) => e.failureMessage.getClass.getSimpleName + case RemoteFailure(_, _, e, _, _) => e.failureMessage.getClass.getSimpleName case _: UnreadableRemoteFailure => "UnreadableRemoteFailure" } } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentEvents.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentEvents.scala index eec22e5aa5..0b462f651f 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentEvents.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentEvents.scala @@ -16,8 +16,8 @@ package fr.acinq.eclair.payment -import fr.acinq.bitcoin.scalacompat.ByteVector32 import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey +import fr.acinq.bitcoin.scalacompat.{ByteVector32, Crypto} import fr.acinq.eclair.crypto.Sphinx import fr.acinq.eclair.crypto.Sphinx.HoldTime import fr.acinq.eclair.payment.Invoice.ExtraEdge @@ -39,8 +39,12 @@ import scala.util.{Failure, Success, Try} */ sealed trait PaymentEvent { - val paymentHash: ByteVector32 - val timestamp: TimestampMilli + // @formatter:off + def paymentHash: ByteVector32 + def startedAt: TimestampMilli + def settledAt: TimestampMilli + def duration: FiniteDuration = settledAt - startedAt + // @formatter:on } /** @@ -48,18 +52,19 @@ sealed trait PaymentEvent { * * @param id id of the whole payment attempt (if using multi-part, there will be multiple parts, * each with a different id). - * @param paymentHash payment hash. * @param paymentPreimage payment preimage (proof of payment). * @param recipientAmount amount that has been received by the final recipient. * @param recipientNodeId id of the final recipient. * @param parts child payments (actual outgoing HTLCs). * @param remainingAttribution_opt for relayed trampoline payments, the attribution data that needs to be sent upstream */ -case class PaymentSent(id: UUID, paymentHash: ByteVector32, paymentPreimage: ByteVector32, recipientAmount: MilliSatoshi, recipientNodeId: PublicKey, parts: Seq[PaymentSent.PartialPayment], remainingAttribution_opt: Option[ByteVector]) extends PaymentEvent { +case class PaymentSent(id: UUID, paymentPreimage: ByteVector32, recipientAmount: MilliSatoshi, recipientNodeId: PublicKey, parts: Seq[PaymentSent.PartialPayment], remainingAttribution_opt: Option[ByteVector]) extends PaymentEvent { require(parts.nonEmpty, "must have at least one payment part") + val paymentHash: ByteVector32 = Crypto.sha256(paymentPreimage) val amountWithFees: MilliSatoshi = parts.map(_.amountWithFees).sum val feesPaid: MilliSatoshi = amountWithFees - recipientAmount // overall fees for this payment - val timestamp: TimestampMilli = parts.map(_.timestamp).min // we use min here because we receive the proof of payment as soon as the first partial payment is fulfilled + val startedAt: TimestampMilli = parts.map(_.startedAt).min + val settledAt: TimestampMilli = parts.map(_.settledAt).max } object PaymentSent { @@ -72,42 +77,40 @@ object PaymentSent { * @param feesPaid fees paid to route to the target node. * @param toChannelId id of the channel used. * @param route payment route used. - * @param timestamp absolute time in milli-seconds since UNIX epoch when the payment was fulfilled. + * @param startedAt absolute time in milliseconds since UNIX epoch when the payment was started. + * @param settledAt absolute time in milliseconds since UNIX epoch when the payment was fulfilled. */ - case class PartialPayment(id: UUID, amount: MilliSatoshi, feesPaid: MilliSatoshi, toChannelId: ByteVector32, route: Option[Seq[Hop]], timestamp: TimestampMilli = TimestampMilli.now()) { + case class PartialPayment(id: UUID, amount: MilliSatoshi, feesPaid: MilliSatoshi, toChannelId: ByteVector32, route: Option[Seq[Hop]], startedAt: TimestampMilli, settledAt: TimestampMilli) { require(route.isEmpty || route.get.nonEmpty, "route must be None or contain at least one hop") val amountWithFees: MilliSatoshi = amount + feesPaid + val duration: FiniteDuration = settledAt - startedAt } } -case class PaymentFailed(id: UUID, paymentHash: ByteVector32, failures: Seq[PaymentFailure], timestamp: TimestampMilli = TimestampMilli.now()) extends PaymentEvent +case class PaymentFailed(id: UUID, paymentHash: ByteVector32, failures: Seq[PaymentFailure], startedAt: TimestampMilli, settledAt: TimestampMilli = TimestampMilli.now()) extends PaymentEvent sealed trait PaymentRelayed extends PaymentEvent { val amountIn: MilliSatoshi val amountOut: MilliSatoshi - val receivedAt: TimestampMilli + val startedAt: TimestampMilli val settledAt: TimestampMilli } -case class ChannelPaymentRelayed(amountIn: MilliSatoshi, amountOut: MilliSatoshi, paymentHash: ByteVector32, fromChannelId: ByteVector32, toChannelId: ByteVector32, receivedAt: TimestampMilli, settledAt: TimestampMilli) extends PaymentRelayed { - override val timestamp: TimestampMilli = settledAt -} +case class ChannelPaymentRelayed(amountIn: MilliSatoshi, amountOut: MilliSatoshi, paymentHash: ByteVector32, fromChannelId: ByteVector32, toChannelId: ByteVector32, startedAt: TimestampMilli, settledAt: TimestampMilli) extends PaymentRelayed case class TrampolinePaymentRelayed(paymentHash: ByteVector32, incoming: PaymentRelayed.Incoming, outgoing: PaymentRelayed.Outgoing, nextTrampolineNodeId: PublicKey, nextTrampolineAmount: MilliSatoshi) extends PaymentRelayed { override val amountIn: MilliSatoshi = incoming.map(_.amount).sum override val amountOut: MilliSatoshi = outgoing.map(_.amount).sum - override val receivedAt: TimestampMilli = incoming.map(_.receivedAt).minOption.getOrElse(TimestampMilli.now()) + override val startedAt: TimestampMilli = incoming.map(_.receivedAt).minOption.getOrElse(TimestampMilli.now()) override val settledAt: TimestampMilli = outgoing.map(_.settledAt).maxOption.getOrElse(TimestampMilli.now()) - override val timestamp: TimestampMilli = settledAt } case class OnTheFlyFundingPaymentRelayed(paymentHash: ByteVector32, incoming: PaymentRelayed.Incoming, outgoing: PaymentRelayed.Outgoing) extends PaymentRelayed { override val amountIn: MilliSatoshi = incoming.map(_.amount).sum override val amountOut: MilliSatoshi = outgoing.map(_.amount).sum - override val receivedAt: TimestampMilli = incoming.map(_.receivedAt).minOption.getOrElse(TimestampMilli.now()) + override val startedAt: TimestampMilli = incoming.map(_.receivedAt).minOption.getOrElse(TimestampMilli.now()) override val settledAt: TimestampMilli = outgoing.map(_.settledAt).maxOption.getOrElse(TimestampMilli.now()) - override val timestamp: TimestampMilli = settledAt } object PaymentRelayed { @@ -123,20 +126,19 @@ object PaymentRelayed { case class PaymentReceived(paymentHash: ByteVector32, parts: Seq[PaymentReceived.PartialPayment]) extends PaymentEvent { require(parts.nonEmpty, "must have at least one payment part") val amount: MilliSatoshi = parts.map(_.amount).sum - val timestamp: TimestampMilli = parts.map(_.timestamp).max // we use max here because we fulfill the payment only once we received all the parts + val startedAt: TimestampMilli = parts.map(_.receivedAt).min // we use min here (when we receive the first payment part) + val settledAt: TimestampMilli = parts.map(_.receivedAt).max // we use max here because we fulfill the payment only once we received all the parts } object PaymentReceived { - case class PartialPayment(amount: MilliSatoshi, fromChannelId: ByteVector32, timestamp: TimestampMilli = TimestampMilli.now()) + case class PartialPayment(amount: MilliSatoshi, fromChannelId: ByteVector32, receivedAt: TimestampMilli = TimestampMilli.now()) } case class PaymentMetadataReceived(paymentHash: ByteVector32, paymentMetadata: ByteVector) -case class PaymentSettlingOnChain(id: UUID, amount: MilliSatoshi, paymentHash: ByteVector32, timestamp: TimestampMilli = TimestampMilli.now()) extends PaymentEvent - -case class WaitingToRelayPayment(remoteNodeId: PublicKey, paymentHash: ByteVector32, timestamp: TimestampMilli = TimestampMilli.now()) extends PaymentEvent +case class PaymentSettlingOnChain(id: UUID, channelId: ByteVector32, amount: MilliSatoshi, paymentHash: ByteVector32, timestamp: TimestampMilli = TimestampMilli.now()) sealed trait PaymentFailure { // @formatter:off @@ -149,10 +151,14 @@ sealed trait PaymentFailure { case class LocalFailure(amount: MilliSatoshi, route: Seq[Hop], t: Throwable) extends PaymentFailure /** A remote node failed the payment and we were able to decrypt the onion failure packet. */ -case class RemoteFailure(amount: MilliSatoshi, route: Seq[Hop], e: Sphinx.DecryptedFailurePacket) extends PaymentFailure +case class RemoteFailure(amount: MilliSatoshi, route: Seq[Hop], e: Sphinx.DecryptedFailurePacket, startedAt: TimestampMilli, failedAt: TimestampMilli) extends PaymentFailure { + val duration: FiniteDuration = failedAt - startedAt +} /** A remote node failed the payment but we couldn't decrypt the failure (e.g. a malicious node tampered with the message). */ -case class UnreadableRemoteFailure(amount: MilliSatoshi, route: Seq[Hop], e: Sphinx.CannotDecryptFailurePacket, holdTimes: Seq[HoldTime]) extends PaymentFailure +case class UnreadableRemoteFailure(amount: MilliSatoshi, route: Seq[Hop], e: Sphinx.CannotDecryptFailurePacket, startedAt: TimestampMilli, failedAt: TimestampMilli, holdTimes: Seq[HoldTime]) extends PaymentFailure { + val duration: FiniteDuration = failedAt - startedAt +} object PaymentFailure { @@ -194,7 +200,7 @@ object PaymentFailure { */ def hasAlreadyFailedOnce(nodeId: PublicKey, failures: Seq[PaymentFailure]): Boolean = failures - .collectFirst { case RemoteFailure(_, _, Sphinx.DecryptedFailurePacket(origin, _, u: Update)) if origin == nodeId => u.update_opt } + .collectFirst { case RemoteFailure(_, _, Sphinx.DecryptedFailurePacket(origin, _, u: Update), _, _) if origin == nodeId => u.update_opt } .isDefined /** Ignore the channel outgoing from the given nodeId in the given route. */ @@ -213,12 +219,12 @@ object PaymentFailure { /** Update the set of nodes and channels to ignore in retries depending on the failure we received. */ def updateIgnored(failure: PaymentFailure, ignore: Ignore): Ignore = failure match { - case RemoteFailure(_, hops, Sphinx.DecryptedFailurePacket(nodeId, _, _)) if nodeId == hops.last.nextNodeId => + case RemoteFailure(_, hops, Sphinx.DecryptedFailurePacket(nodeId, _, _), _, _) if nodeId == hops.last.nextNodeId => // The failure came from the final recipient: the payment should be aborted without penalizing anyone in the route. ignore - case RemoteFailure(_, _, Sphinx.DecryptedFailurePacket(nodeId, _, _: Node)) => + case RemoteFailure(_, _, Sphinx.DecryptedFailurePacket(nodeId, _, _: Node), _, _) => ignore + nodeId - case RemoteFailure(_, hops, Sphinx.DecryptedFailurePacket(nodeId, _, failureMessage: Update)) => + case RemoteFailure(_, hops, Sphinx.DecryptedFailurePacket(nodeId, _, failureMessage: Update), _, _) => if (failureMessage.update_opt.forall(update => Announcements.checkSig(update, nodeId))) { val shouldIgnore = failureMessage match { case _: TemporaryChannelFailure => true @@ -235,9 +241,9 @@ object PaymentFailure { // This node is fishy, it gave us a bad channel update signature, so let's filter it out. ignore + nodeId } - case RemoteFailure(_, hops, Sphinx.DecryptedFailurePacket(nodeId, _, _)) => + case RemoteFailure(_, hops, Sphinx.DecryptedFailurePacket(nodeId, _, _), _, _) => ignoreNodeOutgoingEdge(nodeId, hops, ignore) - case UnreadableRemoteFailure(_, hops, _, holdTimes) => + case UnreadableRemoteFailure(_, hops, _, _, _, holdTimes) => // TODO: Once everyone supports attributable errors, we should only exclude two nodes: the last for which we have attribution data and the next one. // We don't know which node is sending garbage, let's blacklist all nodes except: // - the nodes that returned attribution data (except the last one) @@ -267,7 +273,7 @@ object PaymentFailure { // We're only interested in the last channel update received per channel. val updates = failures.foldLeft(Map.empty[ShortChannelId, ChannelUpdate]) { case (current, failure) => failure match { - case RemoteFailure(_, _, Sphinx.DecryptedFailurePacket(_, _, f: Update)) => f.update_opt match { + case RemoteFailure(_, _, Sphinx.DecryptedFailurePacket(_, _, f: Update), _, _) => f.update_opt match { case Some(update) => current.updated(update.shortChannelId, update) case None => current } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/receive/MultiPartHandler.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/receive/MultiPartHandler.scala index 6822aa721a..b837838d29 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/receive/MultiPartHandler.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/receive/MultiPartHandler.scala @@ -187,8 +187,8 @@ class MultiPartHandler(nodeParams: NodeParams, register: ActorRef, db: IncomingP // NB: this case shouldn't happen unless the sender violated the spec, so it's ok that we take a slightly more // expensive code path by fetching the preimage from DB. case p: MultiPartPaymentFSM.HtlcPart => db.getIncomingPayment(paymentHash).foreach(record => { - val received = PaymentReceived(paymentHash, PaymentReceived.PartialPayment(p.amount, p.htlc.channelId) :: Nil) - if (db.receiveIncomingPayment(paymentHash, p.amount, received.timestamp)) { + val received = PaymentReceived(paymentHash, PaymentReceived.PartialPayment(p.amount, p.htlc.channelId, p.receivedAt) :: Nil) + if (db.receiveIncomingPayment(paymentHash, p.amount, received.settledAt)) { val attribution = FulfillAttributionData(htlcReceivedAt = p.receivedAt, trampolineReceivedAt_opt = None, downstreamAttribution_opt = None) PendingCommandsDb.safeSend(register, nodeParams.db.pendingCommands, p.htlc.channelId, CMD_FULFILL_HTLC(p.htlc.id, record.paymentPreimage, Some(attribution), commit = true)) ctx.system.eventStream.publish(received) @@ -207,17 +207,17 @@ class MultiPartHandler(nodeParams: NodeParams, register: ActorRef, db: IncomingP Logs.withMdc(log)(Logs.mdc(paymentHash_opt = Some(paymentHash))) { log.debug("fulfilling payment for amount={}", parts.map(_.amount).sum) val received = PaymentReceived(paymentHash, parts.flatMap { - case p: MultiPartPaymentFSM.HtlcPart => Some(PaymentReceived.PartialPayment(p.amount, p.htlc.channelId)) + case p: MultiPartPaymentFSM.HtlcPart => Some(PaymentReceived.PartialPayment(p.amount, p.htlc.channelId, p.receivedAt)) case _: MultiPartPaymentFSM.RecipientBlindedPathFeePart => None }) val recordedInDb = payment match { // Incoming offer payments are not stored in the database until they have been paid. case IncomingBlindedPayment(invoice, preimage, paymentType, _, _) => - db.receiveIncomingOfferPayment(invoice, preimage, received.amount, received.timestamp, paymentType) + db.receiveIncomingOfferPayment(invoice, preimage, received.amount, received.settledAt, paymentType) true // Incoming standard payments are already stored and need to be marked as received. case _: IncomingStandardPayment => - db.receiveIncomingPayment(paymentHash, received.amount, received.timestamp) + db.receiveIncomingPayment(paymentHash, received.amount, received.settledAt) } if (recordedInDb) { parts.collect { 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 bdae4a298a..167f4773d0 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 @@ -245,7 +245,8 @@ class ChannelRelay private(nodeParams: NodeParams, private def waitForAddSettled(): Behavior[Command] = Behaviors.receiveMessagePartial { case WrappedAddResponse(RES_ADD_SETTLED(_, htlc, fulfill: HtlcResult.Fulfill)) => - context.log.info("relaying fulfill to upstream, receivedAt={}, endedAt={}, confidence={}, originNode={}, outgoingChannel={}", upstream.receivedAt, r.receivedAt, reputationScore.outgoingConfidence, upstream.receivedFrom, htlc.channelId) + val now = TimestampMilli.now() + context.log.info("relaying fulfill to upstream, receivedAt={}, endedAt={}, confidence={}, originNode={}, outgoingChannel={}", upstream.receivedAt, now, reputationScore.outgoingConfidence, upstream.receivedFrom, htlc.channelId) Metrics.relayFulfill(reputationScore.outgoingConfidence) val downstreamAttribution_opt = fulfill match { case HtlcResult.RemoteFulfill(fulfill) => fulfill.attribution_opt @@ -253,12 +254,12 @@ class ChannelRelay private(nodeParams: NodeParams, } val attribution = FulfillAttributionData(htlcReceivedAt = upstream.receivedAt, trampolineReceivedAt_opt = None, downstreamAttribution_opt = downstreamAttribution_opt) val cmd = CMD_FULFILL_HTLC(upstream.add.id, fulfill.paymentPreimage, Some(attribution), commit = true) - context.system.eventStream ! EventStream.Publish(ChannelPaymentRelayed(upstream.amountIn, htlc.amountMsat, htlc.paymentHash, upstream.add.channelId, htlc.channelId, upstream.receivedAt, r.receivedAt)) + context.system.eventStream ! EventStream.Publish(ChannelPaymentRelayed(upstream.amountIn, htlc.amountMsat, htlc.paymentHash, upstream.add.channelId, htlc.channelId, upstream.receivedAt, now)) recordRelayDuration(isSuccess = true) safeSendAndStop(upstream.add.channelId, cmd) - case WrappedAddResponse(RES_ADD_SETTLED(_, htlc, fail: HtlcResult.Fail)) => - context.log.info("relaying fail to upstream, receivedAt={}, endedAt={}, confidence={}, originNode={}, outgoingChannel={}", upstream.receivedAt, r.receivedAt, reputationScore.outgoingConfidence, upstream.receivedFrom, htlc.channelId) + val now = TimestampMilli.now() + context.log.info("relaying fail to upstream, receivedAt={}, endedAt={}, confidence={}, originNode={}, outgoingChannel={}", upstream.receivedAt, now, reputationScore.outgoingConfidence, upstream.receivedFrom, htlc.channelId) Metrics.relayFail(reputationScore.outgoingConfidence) Metrics.recordPaymentRelayFailed(Tags.FailureType.Remote, Tags.RelayType.Channel) val cmd = translateRelayFailure(upstream.add.id, fail, Some(upstream.receivedAt)) 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 3b8c451b26..5e121622a3 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 @@ -184,12 +184,12 @@ object NodeRelay { case _ => // Otherwise, we try to find a downstream error that we could decrypt. val outgoingNodeFailure = nextPayload match { - case nextPayload: IntermediatePayload.NodeRelay.Standard => failures.collectFirst { case RemoteFailure(_, _, e) if e.originNode == nextPayload.outgoingNodeId => e.failureMessage } - case nextPayload: IntermediatePayload.NodeRelay.ToNonTrampoline => failures.collectFirst { case RemoteFailure(_, _, e) if e.originNode == nextPayload.outgoingNodeId => e.failureMessage } + case nextPayload: IntermediatePayload.NodeRelay.Standard => failures.collectFirst { case f: RemoteFailure if f.e.originNode == nextPayload.outgoingNodeId => f.e.failureMessage } + case nextPayload: IntermediatePayload.NodeRelay.ToNonTrampoline => failures.collectFirst { case f: RemoteFailure if f.e.originNode == nextPayload.outgoingNodeId => f.e.failureMessage } // When using blinded paths, we will never get a failure from the final node (for privacy reasons). case _: IntermediatePayload.NodeRelay.ToBlindedPaths => None } - val otherNodeFailure = failures.collectFirst { case RemoteFailure(_, _, e) => e.failureMessage } + val otherNodeFailure = failures.collectFirst { case f: RemoteFailure => f.e.failureMessage } val failure = outgoingNodeFailure.getOrElse(otherNodeFailure.getOrElse(TemporaryNodeFailure())) Some(failure) } @@ -405,13 +405,13 @@ class NodeRelay private(nodeParams: NodeParams, context.log.warn("trampoline payment failed downstream but was fulfilled upstream") recordRelayDuration(startedAt, isSuccess = true) stopping() - case WrappedPaymentFailed(PaymentFailed(_, _, failures, _)) => + case WrappedPaymentFailed(e) => walletNodeId_opt match { - case Some(walletNodeId) if shouldAttemptOnTheFlyFunding(nodeParams, recipientFeatures_opt, failures)(context) => + case Some(walletNodeId) if shouldAttemptOnTheFlyFunding(nodeParams, recipientFeatures_opt, e.failures)(context) => context.log.info("trampoline payment failed, attempting on-the-fly funding") - attemptOnTheFlyFunding(upstream, walletNodeId, recipient, nextPayload, failures, startedAt) + attemptOnTheFlyFunding(upstream, walletNodeId, recipient, nextPayload, e.failures, startedAt) case _ => - rejectPayment(upstream, translateError(nodeParams, failures, upstream, nextPayload)) + rejectPayment(upstream, translateError(nodeParams, e.failures, upstream, nextPayload)) recordRelayDuration(startedAt, isSuccess = false) stopping() } @@ -516,7 +516,7 @@ class NodeRelay private(nodeParams: NodeParams, fulfillPayment(upstream, paymentSent.paymentPreimage, paymentSent.remainingAttribution_opt) } val incoming = upstream.received.map(r => PaymentRelayed.IncomingPart(r.add.amountMsat, r.add.channelId, r.receivedAt)) - val outgoing = paymentSent.parts.map(part => PaymentRelayed.OutgoingPart(part.amountWithFees, part.toChannelId, part.timestamp)) + val outgoing = paymentSent.parts.map(part => PaymentRelayed.OutgoingPart(part.amountWithFees, part.toChannelId, part.settledAt)) context.system.eventStream ! EventStream.Publish(TrampolinePaymentRelayed(paymentHash, incoming, outgoing, paymentSent.recipientNodeId, paymentSent.recipientAmount)) } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/PostRestartHtlcCleaner.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/PostRestartHtlcCleaner.scala index 8c51b41677..607e81e68c 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/PostRestartHtlcCleaner.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/PostRestartHtlcCleaner.scala @@ -178,15 +178,15 @@ class PostRestartHtlcCleaner(nodeParams: NodeParams, register: ActorRef, initial val feesPaid = 0.msat // fees are unknown since we lost the reference to the payment nodeParams.db.payments.getOutgoingPayment(id) match { case Some(p) => - nodeParams.db.payments.updateOutgoingPayment(PaymentSent(p.parentId, fulfilledHtlc.paymentHash, paymentPreimage, p.recipientAmount, p.recipientNodeId, PaymentSent.PartialPayment(id, fulfilledHtlc.amountMsat, feesPaid, fulfilledHtlc.channelId, None) :: Nil, None)) + nodeParams.db.payments.updateOutgoingPayment(PaymentSent(p.parentId, paymentPreimage, p.recipientAmount, p.recipientNodeId, PaymentSent.PartialPayment(id, fulfilledHtlc.amountMsat, feesPaid, fulfilledHtlc.channelId, None, startedAt = p.createdAt, settledAt = TimestampMilli.now()) :: Nil, None)) // If all downstream HTLCs are now resolved, we can emit the payment event. val payments = nodeParams.db.payments.listOutgoingPayments(p.parentId) if (!payments.exists(p => p.status == OutgoingPaymentStatus.Pending)) { val succeeded = payments.collect { - case OutgoingPayment(id, _, _, _, _, amount, _, _, _, _, _, OutgoingPaymentStatus.Succeeded(_, feesPaid, _, completedAt)) => - PaymentSent.PartialPayment(id, amount, feesPaid, ByteVector32.Zeroes, None, completedAt) + case OutgoingPayment(id, _, _, _, _, amount, _, _, createdAt, _, _, OutgoingPaymentStatus.Succeeded(_, feesPaid, _, completedAt)) => + PaymentSent.PartialPayment(id, amount, feesPaid, ByteVector32.Zeroes, None, createdAt, completedAt) } - val sent = PaymentSent(p.parentId, fulfilledHtlc.paymentHash, paymentPreimage, p.recipientAmount, p.recipientNodeId, succeeded, None) + val sent = PaymentSent(p.parentId, paymentPreimage, p.recipientAmount, p.recipientNodeId, succeeded, None) log.info(s"payment id=${sent.id} paymentHash=${sent.paymentHash} successfully sent (amount=${sent.recipientAmount})") context.system.eventStream.publish(sent) } @@ -197,7 +197,7 @@ class PostRestartHtlcCleaner(nodeParams: NodeParams, register: ActorRef, initial val dummyFinalAmount = fulfilledHtlc.amountMsat val dummyNodeId = nodeParams.nodeId nodeParams.db.payments.addOutgoingPayment(OutgoingPayment(id, id, None, fulfilledHtlc.paymentHash, PaymentType.Standard, fulfilledHtlc.amountMsat, dummyFinalAmount, dummyNodeId, TimestampMilli.now(), None, None, OutgoingPaymentStatus.Pending)) - nodeParams.db.payments.updateOutgoingPayment(PaymentSent(id, fulfilledHtlc.paymentHash, paymentPreimage, dummyFinalAmount, dummyNodeId, PaymentSent.PartialPayment(id, fulfilledHtlc.amountMsat, feesPaid, fulfilledHtlc.channelId, None) :: Nil, None)) + nodeParams.db.payments.updateOutgoingPayment(PaymentSent(id, paymentPreimage, dummyFinalAmount, dummyNodeId, PaymentSent.PartialPayment(id, fulfilledHtlc.amountMsat, feesPaid, fulfilledHtlc.channelId, None, startedAt = TimestampMilli.now(), settledAt = TimestampMilli.now()) :: Nil, None)) } // There can never be more than one pending downstream HTLC for a given local origin (a multi-part payment is // instead spread across multiple local origins) so we can now forget this origin. @@ -243,7 +243,7 @@ class PostRestartHtlcCleaner(nodeParams: NodeParams, register: ActorRef, initial case Some(relayedOut) => // If this is a local payment, we need to update the DB: origin.upstream match { - case Upstream.Local(id) => nodeParams.db.payments.updateOutgoingPayment(PaymentFailed(id, failedHtlc.paymentHash, Nil)) + case Upstream.Local(id) => nodeParams.db.payments.updateOutgoingPayment(PaymentFailed(id, failedHtlc.paymentHash, Nil, startedAt = TimestampMilli.now(), settledAt = TimestampMilli.now())) case _ => } val relayedOut1 = relayedOut diff Set((failedHtlc.channelId, failedHtlc.id)) @@ -256,7 +256,7 @@ class PostRestartHtlcCleaner(nodeParams: NodeParams, register: ActorRef, initial val payments = nodeParams.db.payments.listOutgoingPayments(p.parentId) if (payments.forall(_.status.isInstanceOf[OutgoingPaymentStatus.Failed])) { log.warning(s"payment failed for paymentHash=${failedHtlc.paymentHash}") - context.system.eventStream.publish(PaymentFailed(p.parentId, failedHtlc.paymentHash, Nil)) + context.system.eventStream.publish(PaymentFailed(p.parentId, failedHtlc.paymentHash, Nil, p.createdAt, settledAt = TimestampMilli.now())) } }) case Upstream.Cold.Channel(originChannelId, originHtlcId, _) => diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/Autoprobe.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/Autoprobe.scala index d45e650d57..e7f0e21588 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/Autoprobe.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/Autoprobe.scala @@ -76,7 +76,7 @@ class Autoprobe(nodeParams: NodeParams, router: ActorRef, paymentInitiator: Acto case paymentResult: PaymentEvent => paymentResult match { - case PaymentFailed(_, _, _ :+ RemoteFailure(_, _, DecryptedFailurePacket(targetNodeId, _, _: IncorrectOrUnknownPaymentDetails)), _) => + case PaymentFailed(_, _, _ :+ RemoteFailure(_, _, DecryptedFailurePacket(targetNodeId, _, _: IncorrectOrUnknownPaymentDetails), _, _), _, _) => log.info(s"payment probe successful to node=$targetNodeId") case _ => log.info(s"payment probe failed with paymentResult=$paymentResult") diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/MultiPartPaymentLifecycle.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/MultiPartPaymentLifecycle.scala index b486e22f2c..c90b1be5a7 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/MultiPartPaymentLifecycle.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/MultiPartPaymentLifecycle.scala @@ -17,7 +17,7 @@ package fr.acinq.eclair.payment.send import akka.actor.typed.scaladsl.adapter.ClassicActorRefOps -import akka.actor.{ActorRef, FSM, Props, Status} +import akka.actor.{ActorRef, FSM, Props} import akka.event.Logging.MDC import fr.acinq.bitcoin.scalacompat.ByteVector32 import fr.acinq.eclair.channel.{HtlcOverriddenByLocalCommit, HtlcsTimedoutDownstream, HtlcsWillTimeoutUpstream, Upstream} @@ -49,9 +49,9 @@ class MultiPartPaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, require(cfg.id == cfg.parentId, "multi-part payment cannot have a parent payment") - val id = cfg.id - val paymentHash = cfg.paymentHash - val start = TimestampMilli.now() + val id: UUID = cfg.id + val paymentHash: ByteVector32 = cfg.paymentHash + val start: TimestampMilli = TimestampMilli.now() private var retriedFailedChannels = false startWith(WAIT_FOR_PAYMENT_REQUEST, WaitingForRequest) @@ -101,7 +101,7 @@ class MultiPartPaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, // payment, which may be confusing for users. val dummyPayment = OutgoingPayment(id, cfg.parentId, cfg.externalId, paymentHash, cfg.paymentType, d.request.recipient.totalAmount, d.request.recipient.totalAmount, d.request.recipient.nodeId, TimestampMilli.now(), cfg.invoice, cfg.payerKey_opt, OutgoingPaymentStatus.Pending) nodeParams.db.payments.addOutgoingPayment(dummyPayment) - nodeParams.db.payments.updateOutgoingPayment(PaymentFailed(id, paymentHash, failure :: Nil)) + nodeParams.db.payments.updateOutgoingPayment(PaymentFailed(id, paymentHash, failure :: Nil, startedAt = start, settledAt = TimestampMilli.now())) } gotoAbortedOrStop(PaymentAborted(d.request, d.failures :+ failure, d.pending.keySet)) } @@ -154,7 +154,7 @@ class MultiPartPaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, val failures = d.failures ++ pf.failures val pending = d.pending - pf.id if (pending.isEmpty) { - myStop(d.request, Left(PaymentFailed(id, paymentHash, failures))) + myStop(d.request, Left(PaymentFailed(id, paymentHash, failures, startedAt = start, settledAt = TimestampMilli.now()))) } else { stay() using d.copy(failures = failures, pending = pending) } @@ -207,7 +207,7 @@ class MultiPartPaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, private def gotoAbortedOrStop(d: PaymentAborted): State = { if (d.pending.isEmpty) { - myStop(d.request, Left(PaymentFailed(id, paymentHash, d.failures))) + myStop(d.request, Left(PaymentFailed(id, paymentHash, d.failures, startedAt = start, settledAt = TimestampMilli.now()))) } else goto(PAYMENT_ABORTED) using d } @@ -255,7 +255,7 @@ class MultiPartPaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, // in case of a relayed payment, we need to take into account the fee of the first channels paymentSent.parts.collect { // NB: the route attribute will always be defined here - case p@PartialPayment(_, _, _, _, Some(route), _) => route.head.fee(p.amountWithFees) + case p@PartialPayment(_, _, _, _, Some(route), _, _) => route.head.fee(p.amountWithFees) }.sum } paymentSent.feesPaid + localFees diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/PaymentInitiator.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/PaymentInitiator.scala index 3a7170e682..ee5c565c9d 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/PaymentInitiator.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/PaymentInitiator.scala @@ -28,7 +28,7 @@ import fr.acinq.eclair.payment.send.BlindedPathsResolver.ResolvedPath import fr.acinq.eclair.payment.send.PaymentError._ import fr.acinq.eclair.router.Router._ import fr.acinq.eclair.wire.protocol._ -import fr.acinq.eclair.{CltvExpiry, CltvExpiryDelta, Features, MilliSatoshi, MilliSatoshiLong, NodeParams} +import fr.acinq.eclair.{CltvExpiry, CltvExpiryDelta, Features, MilliSatoshi, MilliSatoshiLong, NodeParams, TimestampMilli} import scodec.bits.ByteVector import java.util.UUID @@ -57,7 +57,7 @@ class PaymentInitiator(nodeParams: NodeParams, outgoingPaymentFactory: PaymentIn case invoice: Bolt12Invoice => BlindedRecipient(invoice, r.resolvedPaths, r.recipientAmount, finalExpiry, r.userCustomTlvs) } if (!nodeParams.features.invoiceFeatures().areSupported(recipient.features)) { - replyTo ! PaymentFailed(paymentId, r.paymentHash, LocalFailure(r.recipientAmount, Nil, UnsupportedFeatures(recipient.features)) :: Nil) + replyTo ! PaymentFailed(paymentId, r.paymentHash, LocalFailure(r.recipientAmount, Nil, UnsupportedFeatures(recipient.features)) :: Nil, startedAt = TimestampMilli.now(), settledAt = TimestampMilli.now()) } else if (Features.canUseFeature(nodeParams.features.invoiceFeatures(), recipient.features, Features.BasicMultiPartPayment)) { val fsm = outgoingPaymentFactory.spawnOutgoingMultiPartPayment(context, paymentCfg, publishPreimage = !r.blockUntilComplete) fsm ! MultiPartPaymentLifecycle.SendMultiPartPayment(self, recipient, r.maxAttempts, r.routeParams) @@ -84,7 +84,7 @@ class PaymentInitiator(nodeParams: NodeParams, outgoingPaymentFactory: PaymentIn r.replyTo ! paymentId } if (r.invoice.amount_opt.isEmpty) { - r.replyTo ! PaymentFailed(paymentId, r.paymentHash, LocalFailure(r.recipientAmount, Nil, new IllegalArgumentException("test trampoline payments must not use amount-less invoices")) :: Nil) + r.replyTo ! PaymentFailed(paymentId, r.paymentHash, LocalFailure(r.recipientAmount, Nil, new IllegalArgumentException("test trampoline payments must not use amount-less invoices")) :: Nil, startedAt = TimestampMilli.now(), settledAt = TimestampMilli.now()) } else { log.info(s"sending trampoline payment with trampolineNodeId=${r.trampolineNodeId} and invoice=${r.invoice.toString}") val fsm = outgoingPaymentFactory.spawnOutgoingTrampolinePayment(context) @@ -96,7 +96,7 @@ class PaymentInitiator(nodeParams: NodeParams, outgoingPaymentFactory: PaymentIn val paymentId = UUID.randomUUID() val parentPaymentId = r.parentId.getOrElse(UUID.randomUUID()) if (!nodeParams.features.invoiceFeatures().areSupported(r.invoice.features)) { - sender() ! PaymentFailed(paymentId, r.paymentHash, LocalFailure(r.recipientAmount, Nil, UnsupportedFeatures(r.invoice.features)) :: Nil) + sender() ! PaymentFailed(paymentId, r.paymentHash, LocalFailure(r.recipientAmount, Nil, UnsupportedFeatures(r.invoice.features)) :: Nil, startedAt = TimestampMilli.now(), settledAt = TimestampMilli.now()) } else { sender() ! SendPaymentToRouteResponse(paymentId, parentPaymentId) val paymentCfg = SendPaymentConfig(paymentId, parentPaymentId, r.externalId, r.paymentHash, r.recipientNodeId, Upstream.Local(paymentId), Some(r.invoice), None, storeInDb = true, publishEvent = true, recordPathFindingMetrics = false, accountable = false) @@ -227,7 +227,7 @@ object PaymentInitiator { trampolineNodeId: PublicKey, routeParams: RouteParams, blockUntilComplete: Boolean = false) extends SendRequestedPayment { - override val recipientAmount = invoice.amount_opt.getOrElse(0 msat) + override val recipientAmount: MilliSatoshi = invoice.amount_opt.getOrElse(0 msat) } /** @@ -270,7 +270,7 @@ object PaymentInitiator { routeParams: RouteParams, userCustomTlvs: Set[GenericTlv] = Set.empty, recordPathFindingMetrics: Boolean = false) { - val paymentHash = Crypto.sha256(paymentPreimage) + val paymentHash: ByteVector32 = Crypto.sha256(paymentPreimage) } /** @@ -331,12 +331,14 @@ object PaymentInitiator { recordPathFindingMetrics: Boolean, accountable: Boolean) { val paymentContext: PaymentContext = PaymentContext(id, parentId, paymentHash) - val paymentType = invoice match { + val paymentType: String = invoice match { case Some(_: Bolt12Invoice) => PaymentType.Blinded case _ => PaymentType.Standard } - def createPaymentSent(recipient: Recipient, preimage: ByteVector32, parts: Seq[PaymentSent.PartialPayment], remainingAttribution_opt: Option[ByteVector]) = PaymentSent(parentId, paymentHash, preimage, recipient.totalAmount, recipient.nodeId, parts, remainingAttribution_opt) + def createPaymentSent(recipient: Recipient, preimage: ByteVector32, parts: Seq[PaymentSent.PartialPayment], remainingAttribution_opt: Option[ByteVector]): PaymentSent = { + PaymentSent(parentId, preimage, recipient.totalAmount, recipient.nodeId, parts, remainingAttribution_opt) + } } } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/PaymentLifecycle.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/PaymentLifecycle.scala index 2de7489c22..32d4a06d16 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/PaymentLifecycle.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/PaymentLifecycle.scala @@ -86,7 +86,7 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A case Event(PaymentRouteNotFound(t), WaitingForRoute(request, failures, _)) => Metrics.PaymentError.withTag(Tags.Failure, Tags.FailureType(LocalFailure(request.amount, Nil, t))).increment() - myStop(request, Left(PaymentFailed(id, paymentHash, failures :+ LocalFailure(request.amount, Nil, t)))) + myStop(request, Left(PaymentFailed(id, paymentHash, failures :+ LocalFailure(request.amount, Nil, t), startedAt = start, settledAt = TimestampMilli.now()))) } when(WAITING_FOR_CONFIDENCE) { @@ -94,11 +94,11 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A OutgoingPaymentPacket.buildOutgoingPayment(Origin.Hot(self, cfg.upstream), paymentHash, route, request.recipient, score) match { case Right(payment) => register ! Register.ForwardShortId(self.toTyped[Register.ForwardShortIdFailure[CMD_ADD_HTLC]], payment.outgoingChannel, payment.cmd) - goto(WAITING_FOR_PAYMENT_COMPLETE) using WaitingForComplete(request, payment.cmd, failures, payment.sharedSecrets, ignore, route) + goto(WAITING_FOR_PAYMENT_COMPLETE) using WaitingForComplete(request, payment.cmd, failures, payment.sharedSecrets, ignore, route, sentAt = TimestampMilli.now()) case Left(error) => log.warning("cannot send outgoing payment: {}", error.getMessage) Metrics.PaymentError.withTag(Tags.Failure, Tags.FailureType(LocalFailure(request.amount, route.fullRoute, error))).increment() - myStop(request, Left(PaymentFailed(id, paymentHash, failures :+ LocalFailure(request.amount, route.fullRoute, error)))) + myStop(request, Left(PaymentFailed(id, paymentHash, failures :+ LocalFailure(request.amount, route.fullRoute, error), startedAt = start, settledAt = TimestampMilli.now()))) } } @@ -114,7 +114,7 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A case Event(RES_ADD_SETTLED(_, htlc, fulfill: HtlcResult.Fulfill), d: WaitingForComplete) => router ! Router.RouteDidRelay(d.route) Metrics.PaymentAttempt.withTag(Tags.MultiPart, value = false).record(d.failures.size + 1) - val p = PartialPayment(id, d.request.amount, d.cmd.amount - d.request.amount, htlc.channelId, Some(d.route.fullRoute)) + val p = PartialPayment(id, d.request.amount, d.cmd.amount - d.request.amount, htlc.channelId, Some(d.route.fullRoute), startedAt = d.sentAt, settledAt = TimestampMilli.now()) val remainingAttribution_opt = fulfill match { case HtlcResult.RemoteFulfill(updateFulfill) => updateFulfill.attribution_opt match { @@ -184,22 +184,23 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A log.info(s"received an error message from local, trying to use a different channel (failure=${t.getMessage})") retry(localFailure, d) } else { - myStop(d.request, Left(PaymentFailed(id, paymentHash, d.failures :+ localFailure))) + myStop(d.request, Left(PaymentFailed(id, paymentHash, d.failures :+ localFailure, startedAt = start, settledAt = TimestampMilli.now()))) } } private def handleRemoteFail(d: WaitingForComplete, fail: UpdateFailHtlc) = { import d._ + val now = TimestampMilli.now() val htlcFailure = Sphinx.FailurePacket.decrypt(fail.reason, fail.attribution_opt, sharedSecrets) if (htlcFailure.holdTimes.nonEmpty) { context.system.eventStream.publish(Router.ReportedHoldTimes(htlcFailure.holdTimes)) } ((htlcFailure.failure match { case success@Right(e) => - Metrics.PaymentError.withTag(Tags.Failure, Tags.FailureType(RemoteFailure(request.amount, Nil, e))).increment() + Metrics.PaymentError.withTag(Tags.Failure, Tags.FailureType(RemoteFailure(request.amount, Nil, e, startedAt = d.sentAt, failedAt = now))).increment() success case failure@Left(e) => - Metrics.PaymentError.withTag(Tags.Failure, Tags.FailureType(UnreadableRemoteFailure(request.amount, Nil, e, htlcFailure.holdTimes))).increment() + Metrics.PaymentError.withTag(Tags.Failure, Tags.FailureType(UnreadableRemoteFailure(request.amount, Nil, e, startedAt = d.sentAt, failedAt = now, htlcFailure.holdTimes))).increment() failure }) match { case res@Right(Sphinx.DecryptedFailurePacket(_, index, failureMessage)) => @@ -227,11 +228,11 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A case Right(e@Sphinx.DecryptedFailurePacket(nodeId, _, failureMessage)) if nodeId == recipient.nodeId => // if destination node returns an error, we fail the payment immediately log.warning(s"received an error message from target nodeId=$nodeId, failing the payment (failure=$failureMessage)") - myStop(request, Left(PaymentFailed(id, paymentHash, failures :+ RemoteFailure(request.amount, route.fullRoute, e)))) + myStop(request, Left(PaymentFailed(id, paymentHash, failures :+ RemoteFailure(request.amount, route.fullRoute, e, startedAt = d.sentAt, failedAt = now), startedAt = start, settledAt = now))) case Right(e@Sphinx.DecryptedFailurePacket(nodeId, _, failureMessage)) if route.finalHop_opt.collect { case h: NodeHop if h.nodeId == nodeId => h }.nonEmpty => // if trampoline node returns an error, we fail the payment immediately log.warning(s"received an error message from trampoline nodeId=$nodeId, failing the payment (failure=$failureMessage)") - myStop(request, Left(PaymentFailed(id, paymentHash, failures :+ RemoteFailure(request.amount, route.fullRoute, e)))) + myStop(request, Left(PaymentFailed(id, paymentHash, failures :+ RemoteFailure(request.amount, route.fullRoute, e, startedAt = d.sentAt, failedAt = now), startedAt = start, settledAt = now))) case res if failures.size + 1 >= request.maxAttempts => // otherwise we never try more than maxAttempts, no matter the kind of error returned val failure = res match { @@ -241,24 +242,24 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A case failureMessage: Update => handleUpdate(nodeId, failureMessage, d) case _ => } - RemoteFailure(request.amount, route.fullRoute, e) + RemoteFailure(request.amount, route.fullRoute, e, startedAt = d.sentAt, failedAt = now) case Left(e@Sphinx.CannotDecryptFailurePacket(unwrapped, _)) => log.warning(s"cannot parse returned error ${fail.reason.toHex} with sharedSecrets=$sharedSecrets: unwrapped=$unwrapped") - UnreadableRemoteFailure(request.amount, route.fullRoute, e, htlcFailure.holdTimes) + UnreadableRemoteFailure(request.amount, route.fullRoute, e, startedAt = d.sentAt, failedAt = now, htlcFailure.holdTimes) } log.warning(s"too many failed attempts, failing the payment") - myStop(request, Left(PaymentFailed(id, paymentHash, failures :+ failure))) + myStop(request, Left(PaymentFailed(id, paymentHash, failures :+ failure, startedAt = start, settledAt = now))) case Left(e@Sphinx.CannotDecryptFailurePacket(unwrapped, _)) => log.warning(s"cannot parse returned error: unwrapped=$unwrapped, route=${route.printNodes()}") - val failure = UnreadableRemoteFailure(request.amount, route.fullRoute, e, htlcFailure.holdTimes) + val failure = UnreadableRemoteFailure(request.amount, route.fullRoute, e, startedAt = d.sentAt, failedAt = now, htlcFailure.holdTimes) retry(failure, d) case Right(e@Sphinx.DecryptedFailurePacket(nodeId, _, failureMessage: Node)) => log.info(s"received 'Node' type error message from nodeId=$nodeId, trying to route around it (failure=$failureMessage)") - val failure = RemoteFailure(request.amount, route.fullRoute, e) + val failure = RemoteFailure(request.amount, route.fullRoute, e, startedAt = d.sentAt, failedAt = now) retry(failure, d) case Right(e@Sphinx.DecryptedFailurePacket(nodeId, _, failureMessage: Update)) => log.info(s"received 'Update' type error message from nodeId=$nodeId, retrying payment (failure=$failureMessage)") - val failure = RemoteFailure(request.amount, route.fullRoute, e) + val failure = RemoteFailure(request.amount, route.fullRoute, e, startedAt = d.sentAt, failedAt = now) if (failureMessage.update_opt.forall(update => Announcements.checkSig(update, nodeId))) { val recipient1 = handleUpdate(nodeId, failureMessage, d) val ignore1 = PaymentFailure.updateIgnored(failure, ignore) @@ -286,7 +287,7 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A case Right(e@Sphinx.DecryptedFailurePacket(nodeId, _, _: InvalidOnionBlinding)) => // there was a failure inside the blinded route we used: we cannot know why it failed, so let's ignore it. log.info(s"received an error coming from nodeId=$nodeId inside the blinded route, retrying with different blinded routes") - val failure = RemoteFailure(request.amount, route.fullRoute, e) + val failure = RemoteFailure(request.amount, route.fullRoute, e, startedAt = d.sentAt, failedAt = now) val ignore1 = PaymentFailure.updateIgnored(failure, ignore) request match { case _: SendPaymentToRoute => @@ -298,7 +299,7 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A } case Right(e@Sphinx.DecryptedFailurePacket(nodeId, _, failureMessage)) => log.info(s"received an error message from nodeId=$nodeId, trying to use a different channel (failure=$failureMessage)") - val failure = RemoteFailure(request.amount, route.fullRoute, e) + val failure = RemoteFailure(request.amount, route.fullRoute, e, startedAt = d.sentAt, failedAt = now) retry(failure, d) } } @@ -417,7 +418,7 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A // in case of a relayed payment, we need to take into account the fee of the first channels paymentSent.parts.collect { // NB: the route attribute will always be defined here - case p@PartialPayment(_, _, _, _, Some(route), _) => route.head.fee(p.amountWithFees) + case p@PartialPayment(_, _, _, _, Some(route), _, _) => route.head.fee(p.amountWithFees) }.sum } paymentSent.feesPaid + localFees @@ -504,7 +505,7 @@ object PaymentLifecycle { case object WaitingForRequest extends Data case class WaitingForRoute(request: SendPayment, failures: Seq[PaymentFailure], ignore: Ignore) extends Data case class WaitingForConfidence(request: SendPayment, failures: Seq[PaymentFailure], ignore: Ignore, route: Route) extends Data - case class WaitingForComplete(request: SendPayment, cmd: CMD_ADD_HTLC, failures: Seq[PaymentFailure], sharedSecrets: Seq[Sphinx.SharedSecret], ignore: Ignore, route: Route) extends Data { + case class WaitingForComplete(request: SendPayment, cmd: CMD_ADD_HTLC, failures: Seq[PaymentFailure], sharedSecrets: Seq[Sphinx.SharedSecret], ignore: Ignore, route: Route, sentAt: TimestampMilli) extends Data { val recipient: Recipient = request.recipient } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/TrampolinePaymentLifecycle.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/TrampolinePaymentLifecycle.scala index 465b6119f8..0dcdbcb7a0 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/TrampolinePaymentLifecycle.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/TrampolinePaymentLifecycle.scala @@ -33,7 +33,7 @@ import fr.acinq.eclair.reputation.Reputation import fr.acinq.eclair.router.Router import fr.acinq.eclair.router.Router.RouteParams import fr.acinq.eclair.wire.protocol.{PaymentOnion, PaymentOnionCodecs} -import fr.acinq.eclair.{CltvExpiry, CltvExpiryDelta, Features, Logs, MilliSatoshi, NodeParams, randomBytes32} +import fr.acinq.eclair.{CltvExpiry, CltvExpiryDelta, Features, Logs, MilliSatoshi, NodeParams, TimestampMilli, randomBytes32} import java.util.UUID @@ -113,7 +113,7 @@ object TrampolinePaymentLifecycle { val add = CMD_ADD_HTLC(addHtlcAdapter.toClassic, outgoing.trampolineAmount, paymentHash, outgoing.trampolineExpiry, outgoing.onion.packet, None, Reputation.Score.max(accountable = false), None, origin, commit = true) channelInfo.channel ! add val channelId = channelInfo.data.asInstanceOf[DATA_NORMAL].channelId - val part = PartialPayment(cmd.paymentId, amount, computeFees(amount, attemptNumber), channelId, None) + val part = PartialPayment(cmd.paymentId, amount, computeFees(amount, attemptNumber), channelId, None, startedAt = TimestampMilli.now(), settledAt = TimestampMilli.now()) // we will update settledAt below waitForSettlement(part, outgoing.onion.sharedSecrets, outgoing.trampolineOnion.sharedSecrets) } @@ -137,7 +137,7 @@ object TrampolinePaymentLifecycle { } case _: HtlcResult.OnChainFulfill => Nil } - parent ! HtlcSettled(fulfill, part, holdTimes) + parent ! HtlcSettled(fulfill, part.copy(settledAt = TimestampMilli.now()), holdTimes) Behaviors.stopped case fail: HtlcResult.Fail => val holdTimes = fail match { @@ -145,7 +145,7 @@ object TrampolinePaymentLifecycle { Sphinx.FailurePacket.decrypt(updateFail.reason, updateFail.attribution_opt, outerOnionSecrets).holdTimes case _ => Nil } - parent ! HtlcSettled(fail, part, holdTimes) + parent ! HtlcSettled(fail, part.copy(settledAt = TimestampMilli.now()), holdTimes) Behaviors.stopped } } @@ -161,6 +161,7 @@ class TrampolinePaymentLifecycle private(nodeParams: NodeParams, import TrampolinePayment._ import TrampolinePaymentLifecycle._ + private val startedAt = TimestampMilli.now() private val paymentHash = cmd.invoice.paymentHash private val totalAmount = cmd.invoice.amount_opt.get @@ -174,7 +175,7 @@ class TrampolinePaymentLifecycle private(nodeParams: NodeParams, Behaviors.receiveMessagePartial { case TrampolinePeerNotFound(nodeId) => context.log.warn("could not send trampoline payment: we don't have channels with trampoline node {}", nodeId) - cmd.replyTo ! PaymentFailed(cmd.paymentId, paymentHash, LocalFailure(totalAmount, Nil, new RuntimeException("no channels with trampoline node")) :: Nil) + cmd.replyTo ! PaymentFailed(cmd.paymentId, paymentHash, LocalFailure(totalAmount, Nil, new RuntimeException("no channels with trampoline node")) :: Nil, startedAt, settledAt = TimestampMilli.now()) Behaviors.stopped case WrappedPeerChannels(channels) => sendPayment(channels, attemptNumber) @@ -194,7 +195,7 @@ class TrampolinePaymentLifecycle private(nodeParams: NodeParams, val expiry = CltvExpiry(nodeParams.currentBlockHeight) + CltvExpiryDelta(36) if (filtered.isEmpty) { context.log.warn("no usable channel with trampoline node {}", cmd.trampolineNodeId) - cmd.replyTo ! PaymentFailed(cmd.paymentId, paymentHash, LocalFailure(totalAmount, Nil, new RuntimeException("no usable channel with trampoline node")) :: Nil) + cmd.replyTo ! PaymentFailed(cmd.paymentId, paymentHash, LocalFailure(totalAmount, Nil, new RuntimeException("no usable channel with trampoline node")) :: Nil, startedAt, settledAt = TimestampMilli.now()) Behaviors.stopped } else { val amount1 = totalAmount / 2 @@ -220,7 +221,7 @@ class TrampolinePaymentLifecycle private(nodeParams: NodeParams, waitForSettlement(remaining - 1, attemptNumber, fulfilledParts) } else { context.log.warn("trampoline payment failed") - cmd.replyTo ! PaymentFailed(cmd.paymentId, paymentHash, LocalFailure(totalAmount, Nil, failure) :: Nil) + cmd.replyTo ! PaymentFailed(cmd.paymentId, paymentHash, LocalFailure(totalAmount, Nil, failure) :: Nil, startedAt, settledAt = TimestampMilli.now()) Behaviors.stopped } case HtlcSettled(result: HtlcResult, part, holdTimes) => @@ -235,7 +236,7 @@ class TrampolinePaymentLifecycle private(nodeParams: NodeParams, waitForSettlement(remaining - 1, attemptNumber, part +: fulfilledParts) } else { context.log.info("trampoline payment succeeded") - cmd.replyTo ! PaymentSent(cmd.paymentId, paymentHash, fulfill.paymentPreimage, totalAmount, cmd.invoice.nodeId, part +: fulfilledParts, None) + cmd.replyTo ! PaymentSent(cmd.paymentId, fulfill.paymentPreimage, totalAmount, cmd.invoice.nodeId, part +: fulfilledParts, None) Behaviors.stopped } case fail: HtlcResult.Fail => @@ -254,11 +255,11 @@ class TrampolinePaymentLifecycle private(nodeParams: NodeParams, val nextFees = computeFees(totalAmount, attemptNumber) if (attemptNumber > 3) { context.log.warn("cannot retry trampoline payment: retries exceeded") - cmd.replyTo ! PaymentFailed(cmd.paymentId, paymentHash, LocalFailure(totalAmount, Nil, new RuntimeException("maximum trampoline retries exceeded")) :: Nil) + cmd.replyTo ! PaymentFailed(cmd.paymentId, paymentHash, LocalFailure(totalAmount, Nil, new RuntimeException("maximum trampoline retries exceeded")) :: Nil, startedAt, settledAt = TimestampMilli.now()) Behaviors.stopped } else if (cmd.routeParams.getMaxFee(totalAmount) < nextFees) { context.log.warn("cannot retry trampoline payment: maximum fees exceeded ({} > {})", nextFees, cmd.routeParams.getMaxFee(totalAmount)) - cmd.replyTo ! PaymentFailed(cmd.paymentId, paymentHash, LocalFailure(totalAmount, Nil, new RuntimeException("maximum trampoline fees exceeded")) :: Nil) + cmd.replyTo ! PaymentFailed(cmd.paymentId, paymentHash, LocalFailure(totalAmount, Nil, new RuntimeException("maximum trampoline fees exceeded")) :: Nil, startedAt, settledAt = TimestampMilli.now()) Behaviors.stopped } else { context.log.info("retrying trampoline payment with fees={}", nextFees) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/db/AuditDbSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/db/AuditDbSpec.scala index 94a095ec8a..d67e2ca698 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/db/AuditDbSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/db/AuditDbSpec.scala @@ -63,7 +63,7 @@ class AuditDbSpec extends AnyFunSuite { val db = dbs.audit val now = TimestampMilli.now() - val e1 = PaymentSent(ZERO_UUID, randomBytes32(), randomBytes32(), 40000 msat, randomKey().publicKey, PaymentSent.PartialPayment(ZERO_UUID, 42000 msat, 1000 msat, randomBytes32(), None) :: Nil, None) + val e1 = PaymentSent(ZERO_UUID, randomBytes32(), 40000 msat, randomKey().publicKey, PaymentSent.PartialPayment(ZERO_UUID, 42000 msat, 1000 msat, randomBytes32(), None, now, now) :: Nil, None) val pp2a = PaymentReceived.PartialPayment(42000 msat, randomBytes32()) val pp2b = PaymentReceived.PartialPayment(42100 msat, randomBytes32()) val e2 = PaymentReceived(randomBytes32(), pp2a :: pp2b :: Nil) @@ -71,11 +71,11 @@ class AuditDbSpec extends AnyFunSuite { val e4a = TransactionPublished(randomBytes32(), randomKey().publicKey, Transaction(0, Seq.empty, Seq.empty, 0), 42 sat, "mutual") val e4b = TransactionConfirmed(e4a.channelId, e4a.remoteNodeId, e4a.tx) val e4c = TransactionConfirmed(randomBytes32(), randomKey().publicKey, Transaction(2, Nil, TxOut(500 sat, hex"1234") :: Nil, 0)) - val pp5a = PaymentSent.PartialPayment(UUID.randomUUID(), 42000 msat, 1000 msat, randomBytes32(), None, timestamp = 0 unixms) - val pp5b = PaymentSent.PartialPayment(UUID.randomUUID(), 42100 msat, 900 msat, randomBytes32(), None, timestamp = 1 unixms) - val e5 = PaymentSent(UUID.randomUUID(), randomBytes32(), randomBytes32(), 84100 msat, randomKey().publicKey, pp5a :: pp5b :: Nil, None) - val pp6 = PaymentSent.PartialPayment(UUID.randomUUID(), 42000 msat, 1000 msat, randomBytes32(), None, timestamp = now + 10.minutes) - val e6 = PaymentSent(UUID.randomUUID(), randomBytes32(), randomBytes32(), 42000 msat, randomKey().publicKey, pp6 :: Nil, None) + val pp5a = PaymentSent.PartialPayment(UUID.randomUUID(), 42000 msat, 1000 msat, randomBytes32(), None, startedAt = 0 unixms, settledAt = 0 unixms) + val pp5b = PaymentSent.PartialPayment(UUID.randomUUID(), 42100 msat, 900 msat, randomBytes32(), None, startedAt = 1 unixms, settledAt = 1 unixms) + val e5 = PaymentSent(UUID.randomUUID(), randomBytes32(), 84100 msat, randomKey().publicKey, pp5a :: pp5b :: Nil, None) + val pp6 = PaymentSent.PartialPayment(UUID.randomUUID(), 42000 msat, 1000 msat, randomBytes32(), None, startedAt = now + 10.minutes, settledAt = now + 10.minutes) + val e6 = PaymentSent(UUID.randomUUID(), randomBytes32(), 42000 msat, randomKey().publicKey, pp6 :: Nil, None) val e7 = ChannelEvent(randomBytes32(), randomKey().publicKey, randomTxId(), 456123000 sat, isChannelOpener = true, isPrivate = false, ChannelEvent.EventType.Closed(MutualClose(null))) val e10 = TrampolinePaymentRelayed(randomBytes32(), Seq( diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/db/PaymentsDbSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/db/PaymentsDbSpec.scala index ba41fe669a..c146f57bda 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/db/PaymentsDbSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/db/PaymentsDbSpec.scala @@ -200,9 +200,9 @@ class PaymentsDbSpec extends AnyFunSuite { val ps6 = OutgoingPayment(UUID.randomUUID(), UUID.randomUUID(), Some("3"), randomBytes32(), PaymentType.Standard, 789 msat, 789 msat, bob, 1250 unixms, None, None, OutgoingPaymentStatus.Failed(Nil, 1300 unixms)) db.addOutgoingPayment(ps4) db.addOutgoingPayment(ps5.copy(status = OutgoingPaymentStatus.Pending)) - db.updateOutgoingPayment(PaymentSent(ps5.parentId, ps5.paymentHash, preimage1, ps5.amount, ps5.recipientNodeId, Seq(PaymentSent.PartialPayment(ps5.id, ps5.amount, 42 msat, randomBytes32(), None, 1180 unixms)), None)) + db.updateOutgoingPayment(PaymentSent(ps5.parentId, preimage1, ps5.amount, ps5.recipientNodeId, Seq(PaymentSent.PartialPayment(ps5.id, ps5.amount, 42 msat, randomBytes32(), None, 1000 unixms, 1180 unixms)), None)) db.addOutgoingPayment(ps6.copy(status = OutgoingPaymentStatus.Pending)) - db.updateOutgoingPayment(PaymentFailed(ps6.id, ps6.paymentHash, Nil, 1300 unixms)) + db.updateOutgoingPayment(PaymentFailed(ps6.id, ps6.paymentHash, Nil, 1100 unixms, 1300 unixms)) assert(db.listOutgoingPayments(1 unixms, 2000 unixms) == Seq(ps1, ps2, ps3, ps4, ps5, ps6)) assert(db.listIncomingPayments(1 unixms, TimestampMilli.now(), None) == Seq(pr1, pr2, pr3)) @@ -764,19 +764,19 @@ class PaymentsDbSpec extends AnyFunSuite { db.addOutgoingPayment(s3) db.addOutgoingPayment(s4) - db.updateOutgoingPayment(PaymentFailed(s3.id, s3.paymentHash, Nil, 310 unixms)) + db.updateOutgoingPayment(PaymentFailed(s3.id, s3.paymentHash, Nil, 10 unixms, 310 unixms)) val ss3 = s3.copy(status = OutgoingPaymentStatus.Failed(Nil, 310 unixms)) assert(db.getOutgoingPayment(s3.id).contains(ss3)) - db.updateOutgoingPayment(PaymentFailed(s4.id, s4.paymentHash, Seq(LocalFailure(s4.amount, Seq(hop_ab), new RuntimeException("woops")), RemoteFailure(s4.amount, Seq(hop_ab, hop_bc), Sphinx.DecryptedFailurePacket(carol, 2, UnknownNextPeer()))), 320 unixms)) + db.updateOutgoingPayment(PaymentFailed(s4.id, s4.paymentHash, Seq(LocalFailure(s4.amount, Seq(hop_ab), new RuntimeException("woops")), RemoteFailure(s4.amount, Seq(hop_ab, hop_bc), Sphinx.DecryptedFailurePacket(carol, 2, UnknownNextPeer()), startedAt = 10 unixms, failedAt = 320 unixms)), startedAt = 100 unixms, settledAt = 320 unixms)) val ss4 = s4.copy(status = OutgoingPaymentStatus.Failed(Seq(FailureSummary(FailureType.LOCAL, "woops", List(HopSummary(alice, bob, Some(ShortChannelId(42)))), Some(alice)), FailureSummary(FailureType.REMOTE, "processing node does not know the next peer in the route", List(HopSummary(alice, bob, Some(ShortChannelId(42))), HopSummary(bob, carol, None)), Some(carol))), 320 unixms)) assert(db.getOutgoingPayment(s4.id).contains(ss4)) // can't update again once it's in a final state - assertThrows[IllegalArgumentException](db.updateOutgoingPayment(PaymentSent(parentId, s3.paymentHash, preimage1, s3.recipientAmount, s3.recipientNodeId, Seq(PaymentSent.PartialPayment(s3.id, s3.amount, 42 msat, randomBytes32(), None)), None))) + assertThrows[IllegalArgumentException](db.updateOutgoingPayment(PaymentSent(parentId, preimage1, s3.recipientAmount, s3.recipientNodeId, Seq(PaymentSent.PartialPayment(s3.id, s3.amount, 42 msat, randomBytes32(), None, startedAt = 100 unixms, settledAt = 500 unixms)), None))) - val paymentSent = PaymentSent(parentId, paymentHash1, preimage1, 600 msat, carol, Seq( - PaymentSent.PartialPayment(s1.id, s1.amount, 15 msat, randomBytes32(), None, 400 unixms), - PaymentSent.PartialPayment(s2.id, s2.amount, 20 msat, randomBytes32(), Some(Seq(hop_ab, hop_bc)), 410 unixms) + val paymentSent = PaymentSent(parentId, preimage1, 600 msat, carol, Seq( + PaymentSent.PartialPayment(s1.id, s1.amount, 15 msat, randomBytes32(), None, startedAt = 200 unixms, settledAt = 400 unixms), + PaymentSent.PartialPayment(s2.id, s2.amount, 20 msat, randomBytes32(), Some(Seq(hop_ab, hop_bc)), startedAt = 210 unixms, settledAt = 410 unixms) ), None) val ss1 = s1.copy(status = OutgoingPaymentStatus.Succeeded(preimage1, 15 msat, Nil, 400 unixms)) val ss2 = s2.copy(status = OutgoingPaymentStatus.Succeeded(preimage1, 20 msat, Seq(HopSummary(alice, bob, Some(ShortChannelId(42))), HopSummary(bob, carol, None)), 410 unixms)) @@ -786,7 +786,7 @@ class PaymentsDbSpec extends AnyFunSuite { assert(db.listOutgoingPayments(parentId) == Seq(ss1, ss2, ss3, ss4)) // can't update again once it's in a final state - assertThrows[IllegalArgumentException](db.updateOutgoingPayment(PaymentFailed(s1.id, s1.paymentHash, Nil))) + assertThrows[IllegalArgumentException](db.updateOutgoingPayment(PaymentFailed(s1.id, s1.paymentHash, Nil, startedAt = 200 unixms, settledAt = 400 unixms))) } } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala index 0d8e890a95..a53019b46c 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala @@ -380,7 +380,13 @@ class PaymentIntegrationSpec extends IntegrationSpec { awaitCond(nodes("B").nodeParams.db.audit.listSent(start, TimestampMilli.now()).nonEmpty) val sent = nodes("B").nodeParams.db.audit.listSent(start, TimestampMilli.now()) assert(sent.length == 1, sent) - assert(sent.head.copy(parts = sent.head.parts.sortBy(_.timestamp)) == paymentSent.copy(parts = paymentSent.parts.map(_.copy(route = None)).sortBy(_.timestamp), remainingAttribution_opt = None), sent) + val paymentSent1 = paymentSent.copy( + // We don't store the route in the DB, and don't store the startedAt timestamp yet (we set it to the same value as settledAt). + parts = paymentSent.parts.map(p => p.copy(route = None, startedAt = p.settledAt)).sortBy(_.settledAt), + // We don't store attribution data in the DB. + remainingAttribution_opt = None, + ) + assert(sent.head.copy(parts = sent.head.parts.sortBy(_.settledAt)) == paymentSent1) awaitCond(nodes("D").nodeParams.db.payments.getIncomingPayment(invoice.paymentHash).exists(_.status.isInstanceOf[IncomingPaymentStatus.Received])) val Some(IncomingStandardPayment(_, _, _, _, IncomingPaymentStatus.Received(receivedAmount, _))) = nodes("D").nodeParams.db.payments.getIncomingPayment(invoice.paymentHash) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/zeroconf/ZeroConfAliasIntegrationSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/zeroconf/ZeroConfAliasIntegrationSpec.scala index 43e337d0eb..bf256981b0 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/zeroconf/ZeroConfAliasIntegrationSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/zeroconf/ZeroConfAliasIntegrationSpec.scala @@ -88,8 +88,8 @@ class ZeroConfAliasIntegrationSpec extends FixtureSpec with IntegrationPatience val result = sendPayment(alice, carol, 100_000 msat, hints = List(createBobToCarolTestHint(f, useHint, overrideHintScid_opt))) match { case Left(paymentFailed) => Left(PaymentFailure.transformForUser(paymentFailed.failures).last match { - case LocalFailure(_, _, t) => Left(t) - case RemoteFailure(_, _, e) => Right(e.failureMessage) + case f: LocalFailure => Left(f.t) + case f: RemoteFailure => Right(f.e.failureMessage) case _: UnreadableRemoteFailure => fail("received unreadable remote failure") }) case Right(_) => Right(Ok) @@ -277,7 +277,7 @@ class ZeroConfAliasIntegrationSpec extends FixtureSpec with IntegrationPatience // The channel update returned in failures doesn't leak the real scid. val failure = sendFailingPayment(alice, carol, 40_000_000 msat, hints = List(List(carolHint))) - val failureWithChannelUpdate = failure.failures.collect { case RemoteFailure(_, _, Sphinx.DecryptedFailurePacket(_, _, f: Update)) => f } + val failureWithChannelUpdate = failure.failures.collect { case RemoteFailure(_, _, Sphinx.DecryptedFailurePacket(_, _, f: Update), _, _) => f } assert(failureWithChannelUpdate.length == 1) assert(failureWithChannelUpdate.head.update_opt.map(_.shortChannelId).contains(bobAlias)) } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/json/JsonSerializersSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/json/JsonSerializersSpec.scala index eab686c1b7..857eb2933a 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/json/JsonSerializersSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/json/JsonSerializersSpec.scala @@ -30,7 +30,7 @@ import fr.acinq.eclair.crypto.{ShaChain, Sphinx} import fr.acinq.eclair.db.OfferData import fr.acinq.eclair.io.Peer import fr.acinq.eclair.io.Peer.PeerInfo -import fr.acinq.eclair.payment.{Invoice, PaymentSettlingOnChain} +import fr.acinq.eclair.payment.{ChannelPaymentRelayed, Invoice} import fr.acinq.eclair.transactions.Transactions._ import fr.acinq.eclair.transactions.{CommitmentSpec, IncomingHtlc, OutgoingHtlc, Transactions} import fr.acinq.eclair.wire.internal.channel.ChannelCodecs @@ -404,8 +404,8 @@ class JsonSerializersSpec extends TestKitBaseClass with AnyFunSuiteLike with Mat } test("type hints") { - val e1 = PaymentSettlingOnChain(UUID.randomUUID, 42 msat, randomBytes32()) - assert(JsonSerializers.serialization.writePretty(e1)(JsonSerializers.formats).contains("\"type\" : \"payment-settling-onchain\"")) + val e1 = ChannelPaymentRelayed(110 msat, 100 msat, randomBytes32(), randomBytes32(), randomBytes32(), 100 unixms, 150 unixms) + assert(JsonSerializers.serialization.writePretty(e1)(JsonSerializers.formats).contains("\"type\" : \"payment-relayed\"")) } test("transaction serializer") { diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartHandlerSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartHandlerSpec.scala index 3d435b8971..0557333ade 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartHandlerSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartHandlerSpec.scala @@ -22,7 +22,7 @@ import fr.acinq.bitcoin.scalacompat.{Block, ByteVector32, Crypto} import fr.acinq.eclair.FeatureSupport.{Mandatory, Optional} import fr.acinq.eclair.Features.{KeySend, _} import fr.acinq.eclair.TestConstants.Alice -import fr.acinq.eclair.channel.{CMD_FAIL_HTLC, CMD_FULFILL_HTLC, FailureAttributionData, FulfillAttributionData, Register} +import fr.acinq.eclair.channel._ import fr.acinq.eclair.db.{IncomingBlindedPayment, IncomingPaymentStatus, IncomingStandardPayment, PaymentType} import fr.acinq.eclair.payment.Bolt11Invoice.ExtraHop import fr.acinq.eclair.payment.PaymentReceived.PartialPayment @@ -30,8 +30,6 @@ import fr.acinq.eclair.payment.offer.OfferManager import fr.acinq.eclair.payment.receive.MultiPartHandler._ import fr.acinq.eclair.payment.receive.MultiPartPaymentFSM.HtlcPart import fr.acinq.eclair.payment.receive.{MultiPartPaymentFSM, PaymentHandler} -import fr.acinq.eclair.payment.relay.Relayer.RelayFees -import fr.acinq.eclair.reputation.Reputation import fr.acinq.eclair.router.BlindedRouteCreation.aggregatePaymentInfo import fr.acinq.eclair.router.Router import fr.acinq.eclair.router.Router.ChannelHop @@ -40,7 +38,7 @@ import fr.acinq.eclair.wire.protocol.OnionPaymentPayloadTlv._ import fr.acinq.eclair.wire.protocol.PaymentOnion.FinalPayload import fr.acinq.eclair.wire.protocol.RouteBlindingEncryptedDataTlv.{PathId, PaymentConstraints} import fr.acinq.eclair.wire.protocol._ -import fr.acinq.eclair.{CltvExpiry, CltvExpiryDelta, EncodedNodeId, Feature, Features, MilliSatoshi, MilliSatoshiLong, NodeParams, ShortChannelId, TestConstants, TestKitBaseClass, TimestampMilli, TimestampMilliLong, randomBytes, randomBytes32, randomKey} +import fr.acinq.eclair.{CltvExpiry, CltvExpiryDelta, EncodedNodeId, Feature, Features, MilliSatoshi, MilliSatoshiLong, NodeParams, ShortChannelId, TestConstants, TestKitBaseClass, TimestampMilli, randomBytes, randomBytes32, randomKey} import org.scalatest.Outcome import org.scalatest.funsuite.FixtureAnyFunSuiteLike import scodec.bits.{ByteVector, HexStringSyntax} @@ -97,16 +95,17 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike } } - def createBlindedPacket(amount: MilliSatoshi, paymentHash: ByteVector32, expiry: CltvExpiry, finalExpiry: CltvExpiry, pathId: ByteVector): IncomingPaymentPacket.FinalPacket = { + def createBlindedPacket(amount: MilliSatoshi, paymentHash: ByteVector32, expiry: CltvExpiry, finalExpiry: CltvExpiry, pathId: ByteVector, receivedAt: TimestampMilli = TimestampMilli.now()): IncomingPaymentPacket.FinalPacket = { val add = UpdateAddHtlc(ByteVector32.One, 0, amount, paymentHash, expiry, TestConstants.emptyOnionPacket, Some(randomKey().publicKey), accountable = false, None) val payload = FinalPayload.Blinded(TlvStream(AmountToForward(amount), TotalAmount(amount), OutgoingCltv(finalExpiry), EncryptedRecipientData(hex"deadbeef")), TlvStream(PathId(pathId), PaymentConstraints(CltvExpiry(500_000), 1 msat))) - IncomingPaymentPacket.FinalPacket(add, payload, TimestampMilli.now()) + IncomingPaymentPacket.FinalPacket(add, payload, receivedAt) } test("PaymentHandler should reply with a fulfill/fail, emit a PaymentReceived and add payment in DB") { f => import f._ val amountMsat = 42000 msat + val receivedAt = TimestampMilli.now() { sender.send(handlerWithoutMpp, ReceiveStandardPayment(sender.ref, Some(amountMsat), Left("1 coffee"))) @@ -118,14 +117,14 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike assert(Crypto.sha256(incoming.get.paymentPreimage) == invoice.paymentHash) val add = UpdateAddHtlc(ByteVector32.One, 1, amountMsat, invoice.paymentHash, defaultExpiry, TestConstants.emptyOnionPacket, None, accountable = false, None) - sender.send(handlerWithoutMpp, IncomingPaymentPacket.FinalPacket(add, FinalPayload.Standard.createPayload(add.amountMsat, add.amountMsat, add.cltvExpiry, invoice.paymentSecret, invoice.paymentMetadata, upgradeAccountability = false), TimestampMilli.now())) + sender.send(handlerWithoutMpp, IncomingPaymentPacket.FinalPacket(add, FinalPayload.Standard.createPayload(add.amountMsat, add.amountMsat, add.cltvExpiry, invoice.paymentSecret, invoice.paymentMetadata, upgradeAccountability = false), receivedAt)) assert(register.expectMsgType[Register.Forward[CMD_FULFILL_HTLC]].message.id == add.id) val paymentReceived = eventListener.expectMsgType[PaymentReceived] - assert(paymentReceived.copy(parts = paymentReceived.parts.map(_.copy(timestamp = 0 unixms))) == PaymentReceived(add.paymentHash, PartialPayment(amountMsat, add.channelId, timestamp = 0 unixms) :: Nil)) + assert(paymentReceived == PaymentReceived(add.paymentHash, PartialPayment(amountMsat, add.channelId, receivedAt) :: Nil)) val received = nodeParams.db.payments.getIncomingPayment(invoice.paymentHash) assert(received.isDefined && received.get.status.isInstanceOf[IncomingPaymentStatus.Received]) - assert(received.get.status.asInstanceOf[IncomingPaymentStatus.Received].copy(receivedAt = 0 unixms) == IncomingPaymentStatus.Received(amountMsat, 0 unixms)) + assert(received.get.status.asInstanceOf[IncomingPaymentStatus.Received] == IncomingPaymentStatus.Received(amountMsat, receivedAt)) sender.expectNoMessage(50 millis) } @@ -134,14 +133,14 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike val invoice = sender.expectMsgType[Bolt11Invoice] val add = UpdateAddHtlc(ByteVector32.One, 1, 75_000 msat, invoice.paymentHash, defaultExpiry + CltvExpiryDelta(12), TestConstants.emptyOnionPacket, None, accountable = false, None) - sender.send(handlerWithoutMpp, IncomingPaymentPacket.FinalPacket(add, FinalPayload.Standard.createPayload(70_000 msat, 70_000 msat, defaultExpiry, invoice.paymentSecret, invoice.paymentMetadata, upgradeAccountability = false), TimestampMilli.now())) + sender.send(handlerWithoutMpp, IncomingPaymentPacket.FinalPacket(add, FinalPayload.Standard.createPayload(70_000 msat, 70_000 msat, defaultExpiry, invoice.paymentSecret, invoice.paymentMetadata, upgradeAccountability = false), receivedAt)) assert(register.expectMsgType[Register.Forward[CMD_FULFILL_HTLC]].message.id == add.id) val paymentReceived = eventListener.expectMsgType[PaymentReceived] - assert(paymentReceived.copy(parts = paymentReceived.parts.map(_.copy(timestamp = 0 unixms))) == PaymentReceived(add.paymentHash, PartialPayment(add.amountMsat, add.channelId, timestamp = 0 unixms) :: Nil)) + assert(paymentReceived == PaymentReceived(add.paymentHash, PartialPayment(add.amountMsat, add.channelId, receivedAt) :: Nil)) val received = nodeParams.db.payments.getIncomingPayment(invoice.paymentHash) assert(received.isDefined && received.get.status.isInstanceOf[IncomingPaymentStatus.Received]) - assert(received.get.status.asInstanceOf[IncomingPaymentStatus.Received].copy(receivedAt = 0 unixms) == IncomingPaymentStatus.Received(add.amountMsat, 0 unixms)) + assert(received.get.status.asInstanceOf[IncomingPaymentStatus.Received] == IncomingPaymentStatus.Received(add.amountMsat, receivedAt)) sender.expectNoMessage(50 millis) } @@ -152,14 +151,14 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike assert(nodeParams.db.payments.getIncomingPayment(invoice.paymentHash).get.status == IncomingPaymentStatus.Pending) val add = UpdateAddHtlc(ByteVector32.One, 2, amountMsat, invoice.paymentHash, defaultExpiry, TestConstants.emptyOnionPacket, None, accountable = false, None) - sender.send(handlerWithMpp, IncomingPaymentPacket.FinalPacket(add, FinalPayload.Standard.createPayload(add.amountMsat, add.amountMsat, add.cltvExpiry, invoice.paymentSecret, invoice.paymentMetadata, upgradeAccountability = false), TimestampMilli.now())) + sender.send(handlerWithMpp, IncomingPaymentPacket.FinalPacket(add, FinalPayload.Standard.createPayload(add.amountMsat, add.amountMsat, add.cltvExpiry, invoice.paymentSecret, invoice.paymentMetadata, upgradeAccountability = false), receivedAt)) assert(register.expectMsgType[Register.Forward[CMD_FULFILL_HTLC]].message.id == add.id) val paymentReceived = eventListener.expectMsgType[PaymentReceived] - assert(paymentReceived.copy(parts = paymentReceived.parts.map(_.copy(timestamp = 0 unixms))) == PaymentReceived(add.paymentHash, PartialPayment(amountMsat, add.channelId, timestamp = 0 unixms) :: Nil)) + assert(paymentReceived == PaymentReceived(add.paymentHash, PartialPayment(amountMsat, add.channelId, receivedAt) :: Nil)) val received = nodeParams.db.payments.getIncomingPayment(invoice.paymentHash) assert(received.isDefined && received.get.status.isInstanceOf[IncomingPaymentStatus.Received]) - assert(received.get.status.asInstanceOf[IncomingPaymentStatus.Received].copy(receivedAt = 0 unixms) == IncomingPaymentStatus.Received(amountMsat, 0 unixms)) + assert(received.get.status.asInstanceOf[IncomingPaymentStatus.Received] == IncomingPaymentStatus.Received(amountMsat, receivedAt)) sender.expectNoMessage(50 millis) } @@ -176,20 +175,20 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike // Offer invoices shouldn't be stored in the DB until we receive a payment for it. assert(nodeParams.db.payments.getIncomingPayment(invoice.paymentHash).isEmpty) - val finalPacket = createBlindedPacket(amountMsat, invoice.paymentHash, defaultExpiry, CltvExpiry(nodeParams.currentBlockHeight), pathId) + val finalPacket = createBlindedPacket(amountMsat, invoice.paymentHash, defaultExpiry, CltvExpiry(nodeParams.currentBlockHeight), pathId, receivedAt) sender.send(handlerWithRouteBlinding, finalPacket) val receivePayment = offerManager.expectMsgType[OfferManager.ReceivePayment] assert(receivePayment.paymentHash == invoice.paymentHash) assert(receivePayment.payload.pathId == pathId.bytes) - val payment = IncomingBlindedPayment(MinimalBolt12Invoice(invoice.records), preimage, PaymentType.Blinded, TimestampMilli.now(), IncomingPaymentStatus.Pending) + val payment = IncomingBlindedPayment(MinimalBolt12Invoice(invoice.records), preimage, PaymentType.Blinded, receivedAt, IncomingPaymentStatus.Pending) receivePayment.replyTo ! GetIncomingPaymentActor.ProcessPayment(payment, 0 msat) assert(register.expectMsgType[Register.Forward[CMD_FULFILL_HTLC]].message.id == finalPacket.add.id) val paymentReceived = eventListener.expectMsgType[PaymentReceived] - assert(paymentReceived.copy(parts = paymentReceived.parts.map(_.copy(timestamp = 0 unixms))) == PaymentReceived(finalPacket.add.paymentHash, PartialPayment(amountMsat, finalPacket.add.channelId, timestamp = 0 unixms) :: Nil)) + assert(paymentReceived == PaymentReceived(finalPacket.add.paymentHash, PartialPayment(amountMsat, finalPacket.add.channelId, receivedAt) :: Nil)) val received = nodeParams.db.payments.getIncomingPayment(invoice.paymentHash) assert(received.isDefined && received.get.status.isInstanceOf[IncomingPaymentStatus.Received]) - assert(received.get.status.asInstanceOf[IncomingPaymentStatus.Received].copy(receivedAt = 0 unixms) == IncomingPaymentStatus.Received(amountMsat, 0 unixms)) + assert(received.get.status.asInstanceOf[IncomingPaymentStatus.Received] == IncomingPaymentStatus.Received(amountMsat, receivedAt)) sender.expectNoMessage(50 millis) } @@ -199,7 +198,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike assert(nodeParams.db.payments.getIncomingPayment(invoice.paymentHash).get.status == IncomingPaymentStatus.Pending) val add = UpdateAddHtlc(ByteVector32.One, 0, amountMsat, invoice.paymentHash, CltvExpiryDelta(3).toCltvExpiry(nodeParams.currentBlockHeight), TestConstants.emptyOnionPacket, None, accountable = false, None) - sender.send(handlerWithMpp, IncomingPaymentPacket.FinalPacket(add, FinalPayload.Standard.createPayload(add.amountMsat, add.amountMsat, add.cltvExpiry, invoice.paymentSecret, invoice.paymentMetadata, upgradeAccountability = false), TimestampMilli.now())) + sender.send(handlerWithMpp, IncomingPaymentPacket.FinalPacket(add, FinalPayload.Standard.createPayload(add.amountMsat, add.amountMsat, add.cltvExpiry, invoice.paymentSecret, invoice.paymentMetadata, upgradeAccountability = false), receivedAt)) val cmd = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]].message assert(cmd.reason == FailureReason.LocalFailure(IncorrectOrUnknownPaymentDetails(amountMsat, nodeParams.currentBlockHeight))) assert(nodeParams.db.payments.getIncomingPayment(invoice.paymentHash).get.status == IncomingPaymentStatus.Pending) @@ -608,7 +607,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike ) val paymentReceived = f.eventListener.expectMsgType[PaymentReceived] - assert(paymentReceived.parts.map(_.copy(timestamp = 0 unixms)).toSet == Set(PartialPayment(800 msat, ByteVector32.One, 0 unixms), PartialPayment(200 msat, ByteVector32.Zeroes, 0 unixms))) + assert(paymentReceived.parts.toSet == Set(PartialPayment(800 msat, ByteVector32.One, receivedAt1), PartialPayment(200 msat, ByteVector32.Zeroes, receivedAt3))) val received = nodeParams.db.payments.getIncomingPayment(invoice.paymentHash) assert(received.isDefined && received.get.status.isInstanceOf[IncomingPaymentStatus.Received]) assert(received.get.status.asInstanceOf[IncomingPaymentStatus.Received].amount == 1000.msat) @@ -650,7 +649,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike ) val paymentReceived = f.eventListener.expectMsgType[PaymentReceived] - assert(paymentReceived.parts.map(_.copy(timestamp = 0 unixms)).toSet == Set(PartialPayment(1100 msat, add1.channelId, 0 unixms), PartialPayment(500 msat, add2.channelId, 0 unixms))) + assert(paymentReceived.parts.toSet == Set(PartialPayment(1100 msat, add1.channelId, receivedAt1), PartialPayment(500 msat, add2.channelId, receivedAt2))) val received = nodeParams.db.payments.getIncomingPayment(invoice.paymentHash) assert(received.isDefined && received.get.status.isInstanceOf[IncomingPaymentStatus.Received]) assert(received.get.status.asInstanceOf[IncomingPaymentStatus.Received].amount == 1600.msat) @@ -690,7 +689,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike val paymentReceived = f.eventListener.expectMsgType[PaymentReceived] assert(paymentReceived.paymentHash == invoice.paymentHash) - assert(paymentReceived.parts.map(_.copy(timestamp = 0 unixms)).toSet == Set(PartialPayment(300 msat, ByteVector32.One, 0 unixms), PartialPayment(700 msat, ByteVector32.Zeroes, 0 unixms))) + assert(paymentReceived.parts.toSet == Set(PartialPayment(300 msat, ByteVector32.One, receivedAt2), PartialPayment(700 msat, ByteVector32.Zeroes, receivedAt3))) val received = nodeParams.db.payments.getIncomingPayment(invoice.paymentHash) assert(received.isDefined && received.get.status.isInstanceOf[IncomingPaymentStatus.Received]) assert(received.get.status.asInstanceOf[IncomingPaymentStatus.Received].amount == 1000.msat) @@ -712,14 +711,15 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike assert(nodeParams.db.payments.getIncomingPayment(paymentHash).isEmpty) val add = UpdateAddHtlc(ByteVector32.One, 0, amountMsat, paymentHash, defaultExpiry, TestConstants.emptyOnionPacket, None, accountable = false, None) - sender.send(handlerWithKeySend, IncomingPaymentPacket.FinalPacket(add, payload, TimestampMilli.now())) + val receivedAt = TimestampMilli.now() + sender.send(handlerWithKeySend, IncomingPaymentPacket.FinalPacket(add, payload, receivedAt)) register.expectMsgType[Register.Forward[CMD_FULFILL_HTLC]] val paymentReceived = eventListener.expectMsgType[PaymentReceived] - assert(paymentReceived.copy(parts = paymentReceived.parts.map(_.copy(timestamp = 0 unixms))) == PaymentReceived(add.paymentHash, PartialPayment(amountMsat, add.channelId, timestamp = 0 unixms) :: Nil)) + assert(paymentReceived == PaymentReceived(add.paymentHash, PartialPayment(amountMsat, add.channelId, receivedAt) :: Nil)) val received = nodeParams.db.payments.getIncomingPayment(paymentHash) assert(received.isDefined && received.get.status.isInstanceOf[IncomingPaymentStatus.Received]) - assert(received.get.status.asInstanceOf[IncomingPaymentStatus.Received].copy(receivedAt = 0 unixms) == IncomingPaymentStatus.Received(amountMsat, 0 unixms)) + assert(received.get.status.asInstanceOf[IncomingPaymentStatus.Received] == IncomingPaymentStatus.Received(amountMsat, receivedAt)) } test("PaymentHandler should handle single-part KeySend payment without payment secret") { f => @@ -733,14 +733,15 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike assert(nodeParams.db.payments.getIncomingPayment(paymentHash).isEmpty) val add = UpdateAddHtlc(ByteVector32.One, 0, amountMsat, paymentHash, defaultExpiry, TestConstants.emptyOnionPacket, None, accountable = false, None) - sender.send(handlerWithKeySend, IncomingPaymentPacket.FinalPacket(add, payload, TimestampMilli.now())) + val receivedAt = TimestampMilli.now() + sender.send(handlerWithKeySend, IncomingPaymentPacket.FinalPacket(add, payload, receivedAt)) register.expectMsgType[Register.Forward[CMD_FULFILL_HTLC]] val paymentReceived = eventListener.expectMsgType[PaymentReceived] - assert(paymentReceived.copy(parts = paymentReceived.parts.map(_.copy(timestamp = 0 unixms))) == PaymentReceived(add.paymentHash, PartialPayment(amountMsat, add.channelId, timestamp = 0 unixms) :: Nil)) + assert(paymentReceived == PaymentReceived(add.paymentHash, PartialPayment(amountMsat, add.channelId, receivedAt) :: Nil)) val received = nodeParams.db.payments.getIncomingPayment(paymentHash) assert(received.isDefined && received.get.status.isInstanceOf[IncomingPaymentStatus.Received]) - assert(received.get.status.asInstanceOf[IncomingPaymentStatus.Received].copy(receivedAt = 0 unixms) == IncomingPaymentStatus.Received(amountMsat, 0 unixms)) + assert(received.get.status.asInstanceOf[IncomingPaymentStatus.Received] == IncomingPaymentStatus.Received(amountMsat, receivedAt)) } test("PaymentHandler should reject KeySend payment when feature is disabled") { f => diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartPaymentLifecycleSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartPaymentLifecycleSpec.scala index f19487b1d2..72aa268672 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartPaymentLifecycleSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartPaymentLifecycleSpec.scala @@ -194,7 +194,7 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS childPayFsm.expectNoMessage(100 millis) val childId = payFsm.stateData.asInstanceOf[PaymentProgress].pending.keys.head - childPayFsm.send(payFsm, PaymentFailed(childId, paymentHash, Seq(RemoteFailure(failingRoute.amount, failingRoute.fullRoute, Sphinx.DecryptedFailurePacket(b, 1, PermanentChannelFailure()))))) + childPayFsm.send(payFsm, PaymentFailed(childId, paymentHash, Seq(RemoteFailure(failingRoute.amount, failingRoute.fullRoute, Sphinx.DecryptedFailurePacket(b, 1, PermanentChannelFailure()), 100 unixms, 200 unixms)), 50 unixms, 200 unixms)) // We retry ignoring the failing channel. expectRouteRequest(router, nodeParams.nodeId, clearRecipient, routeParams.copy(randomize = true), allowMultiPart = true, ignore = Ignore(Set.empty, Set(ChannelDesc(channelId_be, b, e))), paymentContext = Some(cfg.paymentContext)) router.send(payFsm, RouteResponse(Seq(Route(400_000 msat, hop_ac_1 :: hop_ce :: Nil, None), Route(600_000 msat, hop_ad :: hop_de :: Nil, None)))) @@ -225,12 +225,12 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS childPayFsm.expectNoMessage(100 millis) val (failedId1, failedRoute1) :: (failedId2, failedRoute2) :: Nil = payFsm.stateData.asInstanceOf[PaymentProgress].pending.toSeq - childPayFsm.send(payFsm, PaymentFailed(failedId1, paymentHash, Seq(RemoteFailure(failedRoute1.amount, failedRoute1.hops, Sphinx.DecryptedFailurePacket(b, 1, TemporaryNodeFailure()))))) + childPayFsm.send(payFsm, PaymentFailed(failedId1, paymentHash, Seq(RemoteFailure(failedRoute1.amount, failedRoute1.hops, Sphinx.DecryptedFailurePacket(b, 1, TemporaryNodeFailure()), 500 unixms, 550 unixms)), 500 unixms, 600 unixms)) // When we retry, we ignore the failing node and we let the router know about the remaining pending route. expectRouteRequest(router, nodeParams.nodeId, clearRecipient, routeParams.copy(randomize = true), Ignore(Set(b), Set.empty), pendingPayments = Seq(failedRoute2), allowMultiPart = true, paymentContext = Some(cfg.paymentContext)) // The second part fails while we're still waiting for new routes. - childPayFsm.send(payFsm, PaymentFailed(failedId2, paymentHash, Seq(RemoteFailure(failedRoute2.amount, failedRoute2.hops, Sphinx.DecryptedFailurePacket(b, 1, TemporaryNodeFailure()))))) + childPayFsm.send(payFsm, PaymentFailed(failedId2, paymentHash, Seq(RemoteFailure(failedRoute2.amount, failedRoute2.hops, Sphinx.DecryptedFailurePacket(b, 1, TemporaryNodeFailure()), 300 unixms, 300 unixms)), 100 unixms, 300 unixms)) // We receive a response to our first request, but it's now obsolete: we re-sent a new route request that takes into // account the latest failures. router.send(payFsm, RouteResponse(Seq(Route(failedRoute1.amount, hop_ac_1 :: hop_ce :: Nil, None)))) @@ -265,7 +265,7 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS childPayFsm.expectNoMessage(100 millis) val (failedId, failedRoute) :: Nil = payFsm.stateData.asInstanceOf[PaymentProgress].pending.toSeq - childPayFsm.send(payFsm, PaymentFailed(failedId, paymentHash, Seq(LocalFailure(failedRoute.amount, failedRoute.fullRoute, RemoteCannotAffordFeesForNewHtlc(randomBytes32(), finalAmount, 15 sat, 0 sat, 15 sat))))) + childPayFsm.send(payFsm, PaymentFailed(failedId, paymentHash, Seq(LocalFailure(failedRoute.amount, failedRoute.fullRoute, RemoteCannotAffordFeesForNewHtlc(randomBytes32(), finalAmount, 15 sat, 0 sat, 15 sat))), 0 unixms, 750 unixms)) // We retry without the failing channel. expectRouteRequest(router, @@ -290,7 +290,7 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS childPayFsm.expectNoMessage(100 millis) val (failedId, failedRoute) :: (_, pendingRoute) :: Nil = payFsm.stateData.asInstanceOf[PaymentProgress].pending.toSeq - childPayFsm.send(payFsm, PaymentFailed(failedId, paymentHash, Seq(LocalFailure(failedRoute.amount, failedRoute.fullRoute, ChannelUnavailable(randomBytes32()))))) + childPayFsm.send(payFsm, PaymentFailed(failedId, paymentHash, Seq(LocalFailure(failedRoute.amount, failedRoute.fullRoute, ChannelUnavailable(randomBytes32()))), 100 unixms, 250 unixms)) // If the router doesn't find routes, we will retry without ignoring the channel: it may work with a different split // of the amount to send. @@ -342,7 +342,7 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS // B changed his fees and expiry after the invoice was issued. val channelUpdate = channelUpdate_be.copy(feeBaseMsat = 250 msat, feeProportionalMillionths = 150, cltvExpiryDelta = CltvExpiryDelta(24)) val childId = payFsm.stateData.asInstanceOf[PaymentProgress].pending.keys.head - childPayFsm.send(payFsm, PaymentFailed(childId, paymentHash, Seq(RemoteFailure(route.amount, route.fullRoute, Sphinx.DecryptedFailurePacket(b, 1, FeeInsufficient(finalAmount, Some(channelUpdate))))))) + childPayFsm.send(payFsm, PaymentFailed(childId, paymentHash, Seq(RemoteFailure(route.amount, route.fullRoute, Sphinx.DecryptedFailurePacket(b, 1, FeeInsufficient(finalAmount, Some(channelUpdate))), 100 unixms, 250 unixms)), 100 unixms, 250 unixms)) // We update the routing hints accordingly before requesting a new route. val extraEdge1 = extraEdge.copy(feeBase = 250 msat, feeProportionalMillionths = 150, cltvExpiryDelta = CltvExpiryDelta(24)) assert(router.expectMsgType[RouteRequest].target.extraEdges == Seq(extraEdge1)) @@ -366,7 +366,7 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS // NB: we need a channel update with a valid signature, otherwise we'll ignore the node instead of this specific channel. val channelUpdate = Announcements.makeChannelUpdate(channelUpdate_be.chainHash, priv_b, e, channelUpdate_be.shortChannelId, channelUpdate_be.cltvExpiryDelta, channelUpdate_be.htlcMinimumMsat, channelUpdate_be.feeBaseMsat, channelUpdate_be.feeProportionalMillionths, channelUpdate_be.htlcMaximumMsat) val childId = payFsm.stateData.asInstanceOf[PaymentProgress].pending.keys.head - childPayFsm.send(payFsm, PaymentFailed(childId, paymentHash, Seq(RemoteFailure(route.amount, route.fullRoute, Sphinx.DecryptedFailurePacket(b, 1, TemporaryChannelFailure(Some(channelUpdate))))))) + childPayFsm.send(payFsm, PaymentFailed(childId, paymentHash, Seq(RemoteFailure(route.amount, route.fullRoute, Sphinx.DecryptedFailurePacket(b, 1, TemporaryChannelFailure(Some(channelUpdate))), 100 unixms, 250 unixms)), 100 unixms, 250 unixms)) // We update the routing hints accordingly before requesting a new route and ignore the channel. val routeRequest = router.expectMsgType[RouteRequest] assert(routeRequest.target.extraEdges == Seq(extraEdge)) @@ -387,7 +387,7 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS // The blinded route fails to relay the payment. val childId = payFsm.stateData.asInstanceOf[PaymentProgress].pending.keys.head - childPayFsm.send(payFsm, PaymentFailed(childId, paymentHash, Seq(RemoteFailure(route.amount, route.fullRoute, Sphinx.DecryptedFailurePacket(b, 1, InvalidOnionBlinding(randomBytes32())))))) + childPayFsm.send(payFsm, PaymentFailed(childId, paymentHash, Seq(RemoteFailure(route.amount, route.fullRoute, Sphinx.DecryptedFailurePacket(b, 1, InvalidOnionBlinding(randomBytes32())), 100 unixms, 250 unixms)), 100 unixms, 250 unixms)) // We retry and ignore that blinded route. val routeRequest = router.expectMsgType[RouteRequest] assert(routeRequest.ignore.channels.map(_.shortChannelId) == Set(hop_be.dummyId)) @@ -407,8 +407,8 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS { val failures = Seq( LocalFailure(finalAmount, Nil, ChannelUnavailable(randomBytes32())), - RemoteFailure(finalAmount, Nil, Sphinx.DecryptedFailurePacket(b, 2, FeeInsufficient(100 msat, Some(makeChannelUpdate(ShortChannelId(2), 15 msat, 150, CltvExpiryDelta(48)))))), - UnreadableRemoteFailure(finalAmount, Nil, Sphinx.CannotDecryptFailurePacket(randomBytes(292), None), Nil) + RemoteFailure(finalAmount, Nil, Sphinx.DecryptedFailurePacket(b, 2, FeeInsufficient(100 msat, Some(makeChannelUpdate(ShortChannelId(2), 15 msat, 150, CltvExpiryDelta(48))))), 100 unixms, 250 unixms), + UnreadableRemoteFailure(finalAmount, Nil, Sphinx.CannotDecryptFailurePacket(randomBytes(292), None), 100 unixms, 250 unixms, Nil) ) val extraEdges1 = Seq( ExtraEdge(a, b, ShortChannelId(1), 10 msat, 0, CltvExpiryDelta(12), 1 msat, None), @@ -419,10 +419,10 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS } { val failures = Seq( - RemoteFailure(finalAmount, Nil, Sphinx.DecryptedFailurePacket(a, 1, FeeInsufficient(100 msat, Some(makeChannelUpdate(ShortChannelId(1), 20 msat, 20, CltvExpiryDelta(20)))))), - RemoteFailure(finalAmount, Nil, Sphinx.DecryptedFailurePacket(b, 2, FeeInsufficient(100 msat, Some(makeChannelUpdate(ShortChannelId(2), 21 msat, 21, CltvExpiryDelta(21)))))), - RemoteFailure(finalAmount, Nil, Sphinx.DecryptedFailurePacket(a, 1, FeeInsufficient(100 msat, Some(makeChannelUpdate(ShortChannelId(3), 22 msat, 22, CltvExpiryDelta(22)))))), - RemoteFailure(finalAmount, Nil, Sphinx.DecryptedFailurePacket(a, 1, FeeInsufficient(100 msat, Some(makeChannelUpdate(ShortChannelId(1), 23 msat, 23, CltvExpiryDelta(23)))))), + RemoteFailure(finalAmount, Nil, Sphinx.DecryptedFailurePacket(a, 1, FeeInsufficient(100 msat, Some(makeChannelUpdate(ShortChannelId(1), 20 msat, 20, CltvExpiryDelta(20))))), 100 unixms, 250 unixms), + RemoteFailure(finalAmount, Nil, Sphinx.DecryptedFailurePacket(b, 2, FeeInsufficient(100 msat, Some(makeChannelUpdate(ShortChannelId(2), 21 msat, 21, CltvExpiryDelta(21))))), 100 unixms, 250 unixms), + RemoteFailure(finalAmount, Nil, Sphinx.DecryptedFailurePacket(a, 1, FeeInsufficient(100 msat, Some(makeChannelUpdate(ShortChannelId(3), 22 msat, 22, CltvExpiryDelta(22))))), 100 unixms, 250 unixms), + RemoteFailure(finalAmount, Nil, Sphinx.DecryptedFailurePacket(a, 1, FeeInsufficient(100 msat, Some(makeChannelUpdate(ShortChannelId(1), 23 msat, 23, CltvExpiryDelta(23))))), 100 unixms, 250 unixms), ) val extraEdges1 = Seq( ExtraEdge(a, b, ShortChannelId(1), 23 msat, 23, CltvExpiryDelta(23), defaultChannelUpdate.htlcMinimumMsat, Some(defaultChannelUpdate.htlcMaximumMsat)), @@ -444,14 +444,14 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS childPayFsm.expectMsgType[SendPaymentToRoute] val (failedId1, failedRoute1) = payFsm.stateData.asInstanceOf[PaymentProgress].pending.head - childPayFsm.send(payFsm, PaymentFailed(failedId1, paymentHash, Seq(UnreadableRemoteFailure(failedRoute1.amount, failedRoute1.hops, Sphinx.CannotDecryptFailurePacket(randomBytes(292), None), Nil)))) + childPayFsm.send(payFsm, PaymentFailed(failedId1, paymentHash, Seq(UnreadableRemoteFailure(failedRoute1.amount, failedRoute1.hops, Sphinx.CannotDecryptFailurePacket(randomBytes(292), None), 100 unixms, 250 unixms, Nil)), 100 unixms, 250 unixms)) router.expectMsgType[RouteRequest] router.send(payFsm, RouteResponse(Seq(Route(500_000 msat, hop_ad :: hop_de :: Nil, None)))) childPayFsm.expectMsgType[SendPaymentToRoute] assert(!payFsm.stateData.asInstanceOf[PaymentProgress].pending.contains(failedId1)) val (failedId2, failedRoute2) = payFsm.stateData.asInstanceOf[PaymentProgress].pending.head - val result = abortAfterFailure(f, PaymentFailed(failedId2, paymentHash, Seq(UnreadableRemoteFailure(failedRoute2.amount, failedRoute2.hops, Sphinx.CannotDecryptFailurePacket(randomBytes(292), None), Nil)))) + val result = abortAfterFailure(f, PaymentFailed(failedId2, paymentHash, Seq(UnreadableRemoteFailure(failedRoute2.amount, failedRoute2.hops, Sphinx.CannotDecryptFailurePacket(randomBytes(292), None), 100 unixms, 250 unixms, Nil)), 100 unixms, 250 unixms)) assert(result.failures.length >= 3) assert(result.failures.contains(LocalFailure(finalAmount, Nil, RetryExhausted))) @@ -503,7 +503,7 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS childPayFsm.expectMsgType[SendPaymentToRoute] val (failedId, failedRoute) = payFsm.stateData.asInstanceOf[PaymentProgress].pending.head - val result = abortAfterFailure(f, PaymentFailed(failedId, paymentHash, Seq(RemoteFailure(failedRoute.amount, failedRoute.fullRoute, Sphinx.DecryptedFailurePacket(e, 2, IncorrectOrUnknownPaymentDetails(600_000 msat, BlockHeight(0))))))) + val result = abortAfterFailure(f, PaymentFailed(failedId, paymentHash, Seq(RemoteFailure(failedRoute.amount, failedRoute.fullRoute, Sphinx.DecryptedFailurePacket(e, 2, IncorrectOrUnknownPaymentDetails(600_000 msat, BlockHeight(0))), 100 unixms, 250 unixms)), 100 unixms, 250 unixms)) assert(result.failures.length == 1) val metrics = metricsListener.expectMsgType[PathFindingExperimentMetrics] @@ -524,7 +524,7 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS childPayFsm.expectMsgType[SendPaymentToRoute] val (failedId, failedRoute) = payFsm.stateData.asInstanceOf[PaymentProgress].pending.head - val result = abortAfterFailure(f, PaymentFailed(failedId, paymentHash, Seq(LocalFailure(failedRoute.amount, failedRoute.fullRoute, HtlcsTimedoutDownstream(channelId = ByteVector32.One, htlcs = Set.empty))))) + val result = abortAfterFailure(f, PaymentFailed(failedId, paymentHash, Seq(LocalFailure(failedRoute.amount, failedRoute.fullRoute, HtlcsTimedoutDownstream(channelId = ByteVector32.One, htlcs = Set.empty))), 100 unixms, 250 unixms)) assert(result.failures.length == 1) } @@ -539,10 +539,10 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS childPayFsm.expectMsgType[SendPaymentToRoute] val (failedId1, failedRoute1) :: (failedId2, failedRoute2) :: Nil = payFsm.stateData.asInstanceOf[PaymentProgress].pending.toSeq - childPayFsm.send(payFsm, PaymentFailed(failedId1, paymentHash, Seq(UnreadableRemoteFailure(failedRoute1.amount, failedRoute1.hops, Sphinx.CannotDecryptFailurePacket(randomBytes(292), None), Nil)))) + childPayFsm.send(payFsm, PaymentFailed(failedId1, paymentHash, Seq(UnreadableRemoteFailure(failedRoute1.amount, failedRoute1.hops, Sphinx.CannotDecryptFailurePacket(randomBytes(292), None), 100 unixms, 250 unixms, Nil)), 100 unixms, 250 unixms)) router.expectMsgType[RouteRequest] - val result = abortAfterFailure(f, PaymentFailed(failedId2, paymentHash, Seq(RemoteFailure(failedRoute2.amount, failedRoute2.hops, Sphinx.DecryptedFailurePacket(e, 2, PaymentTimeout()))))) + val result = abortAfterFailure(f, PaymentFailed(failedId2, paymentHash, Seq(RemoteFailure(failedRoute2.amount, failedRoute2.hops, Sphinx.DecryptedFailurePacket(e, 2, PaymentTimeout()), 100 unixms, 250 unixms)), 100 unixms, 250 unixms)) assert(result.failures.length == 2) } @@ -556,8 +556,8 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS childPayFsm.expectMsgType[SendPaymentToRoute] childPayFsm.expectMsgType[SendPaymentToRoute] - val (failedId, failedRoute) :: (successId, successRoute) :: Nil = payFsm.stateData.asInstanceOf[PaymentProgress].pending.toSeq - childPayFsm.send(payFsm, PaymentFailed(failedId, paymentHash, Seq(UnreadableRemoteFailure(failedRoute.amount, failedRoute.fullRoute, Sphinx.CannotDecryptFailurePacket(randomBytes(292), None), Nil)))) + val (failedId, failedRoute) :: (successId, _) :: Nil = payFsm.stateData.asInstanceOf[PaymentProgress].pending.toSeq + childPayFsm.send(payFsm, PaymentFailed(failedId, paymentHash, Seq(UnreadableRemoteFailure(failedRoute.amount, failedRoute.fullRoute, Sphinx.CannotDecryptFailurePacket(randomBytes(292), None), 100 unixms, 250 unixms, Nil)), 100 unixms, 250 unixms)) router.expectMsgType[RouteRequest] val result = fulfillPendingPayments(f, 1, e, finalAmount) @@ -576,11 +576,11 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS childPayFsm.expectMsgType[SendPaymentToRoute] val (failedId, failedRoute) :: (successId, successRoute) :: Nil = payFsm.stateData.asInstanceOf[PaymentProgress].pending.toSeq - childPayFsm.send(payFsm, PaymentFailed(failedId, paymentHash, Seq(RemoteFailure(failedRoute.amount, failedRoute.fullRoute, Sphinx.DecryptedFailurePacket(e, 2, PaymentTimeout()))))) + childPayFsm.send(payFsm, PaymentFailed(failedId, paymentHash, Seq(RemoteFailure(failedRoute.amount, failedRoute.fullRoute, Sphinx.DecryptedFailurePacket(e, 2, PaymentTimeout()), 100 unixms, 250 unixms)), 100 unixms, 250 unixms)) awaitCond(payFsm.stateName == PAYMENT_ABORTED) sender.watch(payFsm) - childPayFsm.send(payFsm, PaymentSent(cfg.id, paymentHash, paymentPreimage, finalAmount, e, Seq(PaymentSent.PartialPayment(successId, successRoute.amount, successRoute.channelFee(false), randomBytes32(), Some(successRoute.fullRoute))), None)) + childPayFsm.send(payFsm, PaymentSent(cfg.id, paymentPreimage, finalAmount, e, Seq(PaymentSent.PartialPayment(successId, successRoute.amount, successRoute.channelFee(false), randomBytes32(), Some(successRoute.fullRoute), 100 unixms, 250 unixms)), None)) sender.expectMsg(PreimageReceived(paymentHash, paymentPreimage, None)) val result = sender.expectMsgType[PaymentSent] assert(result.id == cfg.id) @@ -608,12 +608,12 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS childPayFsm.expectMsgType[SendPaymentToRoute] val (childId, route) :: (failedId, failedRoute) :: Nil = payFsm.stateData.asInstanceOf[PaymentProgress].pending.toSeq - childPayFsm.send(payFsm, PaymentSent(cfg.id, paymentHash, paymentPreimage, finalAmount, e, Seq(PaymentSent.PartialPayment(childId, route.amount, route.channelFee(false), randomBytes32(), Some(route.fullRoute))), None)) + childPayFsm.send(payFsm, PaymentSent(cfg.id, paymentPreimage, finalAmount, e, Seq(PaymentSent.PartialPayment(childId, route.amount, route.channelFee(false), randomBytes32(), Some(route.fullRoute), 100 unixms, 250 unixms)), None)) sender.expectMsg(PreimageReceived(paymentHash, paymentPreimage, None)) awaitCond(payFsm.stateName == PAYMENT_SUCCEEDED) sender.watch(payFsm) - childPayFsm.send(payFsm, PaymentFailed(failedId, paymentHash, Seq(RemoteFailure(failedRoute.amount, failedRoute.fullRoute, Sphinx.DecryptedFailurePacket(e, 2, PaymentTimeout()))))) + childPayFsm.send(payFsm, PaymentFailed(failedId, paymentHash, Seq(RemoteFailure(failedRoute.amount, failedRoute.fullRoute, Sphinx.DecryptedFailurePacket(e, 2, PaymentTimeout()), 100 unixms, 250 unixms)), 100 unixms, 250 unixms)) val result = sender.expectMsgType[PaymentSent] assert(result.parts.length == 1 && result.parts.head.id == childId) assert(result.amountWithFees < finalAmount) // we got the preimage without paying the full amount @@ -632,9 +632,9 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS assert(pending.size == childCount) val partialPayments = pending.map { - case (childId, route) => PaymentSent.PartialPayment(childId, route.amount, route.channelFee(false) + route.blindedFee, randomBytes32(), Some(route.fullRoute)) + case (childId, route) => PaymentSent.PartialPayment(childId, route.amount, route.channelFee(false) + route.blindedFee, randomBytes32(), Some(route.fullRoute), 100 unixms, 250 unixms) } - partialPayments.foreach(pp => childPayFsm.send(payFsm, PaymentSent(cfg.id, paymentHash, paymentPreimage, finalAmount, e, Seq(pp), None))) + partialPayments.foreach(pp => childPayFsm.send(payFsm, PaymentSent(cfg.id, paymentPreimage, finalAmount, e, Seq(pp), None))) sender.expectMsg(PreimageReceived(paymentHash, paymentPreimage, None)) val result = sender.expectMsgType[PaymentSent] assert(result.id == cfg.id) @@ -663,7 +663,7 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS assert(payFsm.stateData.asInstanceOf[PaymentAborted].pending.size == pending.size - 1) // Fail all remaining child payments. payFsm.stateData.asInstanceOf[PaymentAborted].pending.foreach(childId => - childPayFsm.send(payFsm, PaymentFailed(childId, paymentHash, Seq(RemoteFailure(pending(childId).amount, hop_ab_1 :: hop_be :: Nil, Sphinx.DecryptedFailurePacket(e, 2, PaymentTimeout()))))) + childPayFsm.send(payFsm, PaymentFailed(childId, paymentHash, Seq(RemoteFailure(pending(childId).amount, hop_ab_1 :: hop_be :: Nil, Sphinx.DecryptedFailurePacket(e, 2, PaymentTimeout()), 100 unixms, 250 unixms)), 100 unixms, 250 unixms)) ) } @@ -684,12 +684,12 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS object MultiPartPaymentLifecycleSpec { - val paymentPreimage = randomBytes32() - val paymentHash = Crypto.sha256(paymentPreimage) - val expiry = CltvExpiry(1105) - val blindedRouteExpiry = CltvExpiry(500_000) - val finalAmount = 1_000_000 msat - val routeParams = PathFindingConf( + val paymentPreimage: ByteVector32 = randomBytes32() + val paymentHash: ByteVector32 = Crypto.sha256(paymentPreimage) + val expiry: CltvExpiry = CltvExpiry(1105) + val blindedRouteExpiry: CltvExpiry = CltvExpiry(500_000) + val finalAmount: MilliSatoshi = 1_000_000 msat + val routeParams: RouteParams = PathFindingConf( randomize = false, boundaries = SearchBoundaries( 15_000 msat, @@ -723,25 +723,25 @@ object MultiPartPaymentLifecycleSpec { val channelId_ad = ShortChannelId(21) val channelId_de = ShortChannelId(22) val defaultChannelUpdate = ChannelUpdate(randomBytes64(), Block.RegtestGenesisBlock.hash, ShortChannelId(0), 0 unixsec, ChannelUpdate.MessageFlags(dontForward = false), ChannelUpdate.ChannelFlags.DUMMY, CltvExpiryDelta(12), 1 msat, 100 msat, 0, 2_000_000 msat) - val channelUpdate_ab_1 = defaultChannelUpdate.copy(shortChannelId = channelId_ab_1) - val channelUpdate_ab_2 = defaultChannelUpdate.copy(shortChannelId = channelId_ab_2) - val channelUpdate_be = defaultChannelUpdate.copy(shortChannelId = channelId_be) - val channelUpdate_ac_1 = defaultChannelUpdate.copy(shortChannelId = channelId_ac_1) - val channelUpdate_ac_2 = defaultChannelUpdate.copy(shortChannelId = channelId_ac_2) - val channelUpdate_ce = defaultChannelUpdate.copy(shortChannelId = channelId_ce) - val channelUpdate_ad = defaultChannelUpdate.copy(shortChannelId = channelId_ad) - val channelUpdate_de = defaultChannelUpdate.copy(shortChannelId = channelId_de) - - val hop_ab_1 = channelHopFromUpdate(a, b, channelUpdate_ab_1) - val hop_ab_2 = channelHopFromUpdate(a, b, channelUpdate_ab_2) - val hop_be = channelHopFromUpdate(b, e, channelUpdate_be) - val hop_ac_1 = channelHopFromUpdate(a, c, channelUpdate_ac_1) - val hop_ac_2 = channelHopFromUpdate(a, c, channelUpdate_ac_2) - val hop_ce = channelHopFromUpdate(c, e, channelUpdate_ce) - val hop_ad = channelHopFromUpdate(a, d, channelUpdate_ad) - val hop_de = channelHopFromUpdate(d, e, channelUpdate_de) - - val recipientFeatures = Features( + val channelUpdate_ab_1: ChannelUpdate = defaultChannelUpdate.copy(shortChannelId = channelId_ab_1) + val channelUpdate_ab_2: ChannelUpdate = defaultChannelUpdate.copy(shortChannelId = channelId_ab_2) + val channelUpdate_be: ChannelUpdate = defaultChannelUpdate.copy(shortChannelId = channelId_be) + val channelUpdate_ac_1: ChannelUpdate = defaultChannelUpdate.copy(shortChannelId = channelId_ac_1) + val channelUpdate_ac_2: ChannelUpdate = defaultChannelUpdate.copy(shortChannelId = channelId_ac_2) + val channelUpdate_ce: ChannelUpdate = defaultChannelUpdate.copy(shortChannelId = channelId_ce) + val channelUpdate_ad: ChannelUpdate = defaultChannelUpdate.copy(shortChannelId = channelId_ad) + val channelUpdate_de: ChannelUpdate = defaultChannelUpdate.copy(shortChannelId = channelId_de) + + val hop_ab_1: ChannelHop = channelHopFromUpdate(a, b, channelUpdate_ab_1) + val hop_ab_2: ChannelHop = channelHopFromUpdate(a, b, channelUpdate_ab_2) + val hop_be: ChannelHop = channelHopFromUpdate(b, e, channelUpdate_be) + val hop_ac_1: ChannelHop = channelHopFromUpdate(a, c, channelUpdate_ac_1) + val hop_ac_2: ChannelHop = channelHopFromUpdate(a, c, channelUpdate_ac_2) + val hop_ce: ChannelHop = channelHopFromUpdate(c, e, channelUpdate_ce) + val hop_ad: ChannelHop = channelHopFromUpdate(a, d, channelUpdate_ad) + val hop_de: ChannelHop = channelHopFromUpdate(d, e, channelUpdate_de) + + val recipientFeatures: Features[InvoiceFeature] = Features( Features.VariableLengthOnion -> FeatureSupport.Mandatory, Features.PaymentSecret -> FeatureSupport.Mandatory, Features.BasicMultiPartPayment -> FeatureSupport.Optional, diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentInitiatorSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentInitiatorSpec.scala index 93cddfa036..61808558a9 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentInitiatorSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentInitiatorSpec.scala @@ -38,7 +38,7 @@ import fr.acinq.eclair.router.BlindedRouteCreation import fr.acinq.eclair.router.Router._ import fr.acinq.eclair.wire.protocol.OfferTypes.{InvoiceRequest, Offer} import fr.acinq.eclair.wire.protocol._ -import fr.acinq.eclair.{Bolt11Feature, Bolt12Feature, CltvExpiry, CltvExpiryDelta, EncodedNodeId, Feature, Features, MilliSatoshiLong, NodeParams, PaymentFinalExpiryConf, TestConstants, TestKitBaseClass, TimestampSecond, UnknownFeature, randomBytes32, randomKey} +import fr.acinq.eclair.{Bolt11Feature, Bolt12Feature, CltvExpiry, CltvExpiryDelta, EncodedNodeId, Feature, Features, MilliSatoshiLong, NodeParams, PaymentFinalExpiryConf, TestConstants, TestKitBaseClass, TimestampMilliLong, TimestampSecond, UnknownFeature, randomBytes32, randomKey} import org.scalatest.funsuite.FixtureAnyFunSuiteLike import org.scalatest.{Outcome, Tag} import scodec.bits.{ByteVector, HexStringSyntax} @@ -183,7 +183,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike sender.send(initiator, GetPayment(PaymentIdentifier.PaymentHash(invoice.paymentHash))) sender.expectMsg(PaymentIsPending(payment.paymentId, invoice.paymentHash, PendingPaymentToRoute(sender.ref, request))) - val pf = PaymentFailed(payment.paymentId, invoice.paymentHash, Nil) + val pf = PaymentFailed(payment.paymentId, invoice.paymentHash, Nil, 100 unixms, 200 unixms) payFsm.send(initiator, pf) sender.expectMsg(pf) eventListener.expectNoMessage(100 millis) @@ -208,7 +208,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike sender.send(initiator, GetPayment(PaymentIdentifier.PaymentHash(invoice.paymentHash))) sender.expectMsg(PaymentIsPending(id, invoice.paymentHash, PendingPaymentToNode(sender.ref, req))) - val pf = PaymentFailed(id, invoice.paymentHash, Nil) + val pf = PaymentFailed(id, invoice.paymentHash, Nil, 0 unixms, 150 unixms) payFsm.send(initiator, pf) sender.expectMsg(pf) eventListener.expectNoMessage(100 millis) @@ -231,7 +231,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike sender.send(initiator, GetPayment(PaymentIdentifier.PaymentHash(invoice.paymentHash))) sender.expectMsg(PaymentIsPending(id, invoice.paymentHash, PendingPaymentToNode(sender.ref, req))) - val ps = PaymentSent(id, invoice.paymentHash, randomBytes32(), finalAmount, priv_c.publicKey, Seq(PartialPayment(UUID.randomUUID(), finalAmount, 0 msat, randomBytes32(), None)), None) + val ps = PaymentSent(id, paymentPreimage, finalAmount, priv_c.publicKey, Seq(PartialPayment(UUID.randomUUID(), finalAmount, 0 msat, randomBytes32(), None, 100 unixms, 200 unixms)), None) payFsm.send(initiator, ps) sender.expectMsg(ps) eventListener.expectNoMessage(100 millis) @@ -275,7 +275,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike sender.send(initiator, GetPayment(PaymentIdentifier.PaymentHash(invoice.paymentHash))) sender.expectMsg(PaymentIsPending(payment.paymentId, invoice.paymentHash, PendingPaymentToRoute(sender.ref, req))) - val pf = PaymentFailed(payment.paymentId, invoice.paymentHash, Nil) + val pf = PaymentFailed(payment.paymentId, invoice.paymentHash, Nil, 100 unixms, 200 unixms) payFsm.send(initiator, pf) sender.expectMsg(pf) eventListener.expectNoMessage(100 millis) @@ -317,7 +317,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike sender.send(initiator, GetPayment(PaymentIdentifier.PaymentHash(invoice.paymentHash))) sender.expectMsg(PaymentIsPending(id, invoice.paymentHash, PendingPaymentToNode(sender.ref, req))) - val pf = PaymentFailed(id, invoice.paymentHash, Nil) + val pf = PaymentFailed(id, invoice.paymentHash, Nil, 100 unixms, 200 unixms) payFsm.send(initiator, pf) sender.expectMsg(pf) eventListener.expectNoMessage(100 millis) @@ -350,7 +350,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike sender.send(initiator, GetPayment(PaymentIdentifier.PaymentHash(invoice.paymentHash))) sender.expectMsg(PaymentIsPending(id, invoice.paymentHash, PendingPaymentToNode(sender.ref, req))) - val ps = PaymentSent(id, invoice.paymentHash, paymentPreimage, finalAmount, invoice.nodeId, Seq(PartialPayment(UUID.randomUUID(), finalAmount, 0 msat, randomBytes32(), None)), None) + val ps = PaymentSent(id, paymentPreimage, finalAmount, invoice.nodeId, Seq(PartialPayment(UUID.randomUUID(), finalAmount, 0 msat, randomBytes32(), None, 100 unixms, 200 unixms)), None) payFsm.send(initiator, ps) sender.expectMsg(ps) eventListener.expectNoMessage(100 millis) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentLifecycleSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentLifecycleSpec.scala index 9afcae009b..5da7c28a1a 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentLifecycleSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentLifecycleSpec.scala @@ -59,16 +59,16 @@ import scala.concurrent.duration._ class PaymentLifecycleSpec extends BaseRouterSpec { - val defaultAmountMsat = 142_000_000 msat - val defaultExpiry = Channel.MIN_CLTV_EXPIRY_DELTA.toCltvExpiry(BlockHeight(40_000)) - val defaultRouteExpiry = CltvExpiry(100_000) - val defaultPaymentPreimage = randomBytes32() - val defaultPaymentHash = Crypto.sha256(defaultPaymentPreimage) - val defaultOrigin = Origin.Cold(Upstream.Local(UUID.randomUUID())) - val defaultExternalId = UUID.randomUUID().toString - val defaultInvoice = Bolt11Invoice(Block.RegtestGenesisBlock.hash, None, defaultPaymentHash, priv_d, Left("test"), Channel.MIN_CLTV_EXPIRY_DELTA) - val defaultRecipient = ClearRecipient(defaultInvoice, defaultAmountMsat, defaultExpiry, Set.empty) - val defaultRouteParams = TestConstants.Alice.nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams + val defaultAmountMsat: MilliSatoshi = 142_000_000 msat + val defaultExpiry: CltvExpiry = Channel.MIN_CLTV_EXPIRY_DELTA.toCltvExpiry(BlockHeight(40_000)) + val defaultRouteExpiry: CltvExpiry = CltvExpiry(100_000) + val defaultPaymentPreimage: ByteVector32 = randomBytes32() + val defaultPaymentHash: ByteVector32 = Crypto.sha256(defaultPaymentPreimage) + val defaultOrigin: Origin = Origin.Cold(Upstream.Local(UUID.randomUUID())) + val defaultExternalId: String = UUID.randomUUID().toString + val defaultInvoice: Bolt11Invoice = Bolt11Invoice(Block.RegtestGenesisBlock.hash, None, defaultPaymentHash, priv_d, Left("test"), Channel.MIN_CLTV_EXPIRY_DELTA) + val defaultRecipient: Recipient = ClearRecipient(defaultInvoice, defaultAmountMsat, defaultExpiry, Set.empty) + val defaultRouteParams: RouteParams = TestConstants.Alice.nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams def expectRouteRequest(probe: TestProbe, source: PublicKey, cfg: SendPaymentConfig, target: Recipient = defaultRecipient, ignore: Ignore = Ignore.empty): Unit = { val request = probe.expectMsgType[RouteRequest] @@ -106,7 +106,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec { PaymentFixture(cfg, nodeParams, paymentFSM, routerForwarder, register, sender, monitor, eventListener, metricsListener) } - def addCompleted(result: HtlcResult) = { + def addCompleted(result: HtlcResult): RES_ADD_SETTLED[Origin, HtlcResult] = { RES_ADD_SETTLED( origin = defaultOrigin, htlc = UpdateAddHtlc(ByteVector32.Zeroes, 0, defaultAmountMsat, defaultPaymentHash, defaultExpiry, TestConstants.emptyOnionPacket, None, accountable = false, None), @@ -357,7 +357,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec { val WaitingForRoute(_, Nil, _) = paymentFSM.stateData routerForwarder.forward(routerFixture.router) awaitCond(paymentFSM.stateName == WAITING_FOR_PAYMENT_COMPLETE) - val WaitingForComplete(_, cmd1, Nil, _, ignore1, route) = paymentFSM.stateData + val WaitingForComplete(_, cmd1, Nil, _, ignore1, route, _) = paymentFSM.stateData assert(ignore1.nodes.isEmpty) register.expectMsg(ForwardShortId(paymentFSM.toTyped, scid_ab, cmd1)) @@ -369,7 +369,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec { // let's simulate a response by the router with another route sender.send(paymentFSM, RouteResponse(route :: Nil)) awaitCond(paymentFSM.stateName == WAITING_FOR_PAYMENT_COMPLETE) - val WaitingForComplete(_, cmd2, _, _, ignore2, _) = paymentFSM.stateData + val WaitingForComplete(_, cmd2, _, _, ignore2, _, _) = paymentFSM.stateData assert(ignore2.nodes == Set(c)) // and reply a 2nd time with an unparsable failure register.expectMsg(ForwardShortId(paymentFSM.toTyped, scid_ab, cmd2)) @@ -408,7 +408,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec { routerForwarder.forward(routerFixture.router) awaitCond(paymentFSM.stateName == WAITING_FOR_PAYMENT_COMPLETE) - val WaitingForComplete(_, cmd1, Nil, _, _, _) = paymentFSM.stateData + val WaitingForComplete(_, cmd1, Nil, _, _, _, _) = paymentFSM.stateData register.expectMsg(ForwardShortId(paymentFSM.toTyped, scid_ab, cmd1)) sender.send(paymentFSM, RES_ADD_FAILED(cmd1, ChannelUnavailable(ByteVector32.Zeroes), None)) @@ -450,7 +450,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec { expectRouteRequest(routerForwarder, a, cfg) routerForwarder.forward(routerFixture.router) awaitCond(paymentFSM.stateName == WAITING_FOR_PAYMENT_COMPLETE) - val WaitingForComplete(_, cmd1, Nil, _, _, _) = paymentFSM.stateData + val WaitingForComplete(_, cmd1, Nil, _, _, _, _) = paymentFSM.stateData register.expectMsg(ForwardShortId(paymentFSM.toTyped, scid_ab, cmd1)) sender.send(paymentFSM, addCompleted(HtlcResult.RemoteFailMalformed(UpdateFailMalformedHtlc(ByteVector32.Zeroes, 0, randomBytes32(), FailureMessageCodecs.BADONION)))) @@ -473,7 +473,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec { expectRouteRequest(routerForwarder, a, cfg) routerForwarder.forward(routerFixture.router) awaitCond(paymentFSM.stateName == WAITING_FOR_PAYMENT_COMPLETE) - val WaitingForComplete(_, cmd1, Nil, _, _, _) = paymentFSM.stateData + val WaitingForComplete(_, cmd1, Nil, _, _, _, _) = paymentFSM.stateData register.expectMsg(ForwardShortId(paymentFSM.toTyped, scid_ab, cmd1)) sender.send(paymentFSM, addCompleted(HtlcResult.OnChainFail(HtlcsTimedoutDownstream(randomBytes32(), Set.empty)))) @@ -496,7 +496,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec { expectRouteRequest(routerForwarder, a, cfg) routerForwarder.forward(routerFixture.router) awaitCond(paymentFSM.stateName == WAITING_FOR_PAYMENT_COMPLETE) - val WaitingForComplete(_, cmd1, Nil, _, _, _) = paymentFSM.stateData + val WaitingForComplete(_, cmd1, Nil, _, _, _, _) = paymentFSM.stateData register.expectMsg(ForwardShortId(paymentFSM.toTyped, scid_ab, cmd1)) val update_bc_disabled = update_bc.copy(channelFlags = ChannelUpdate.ChannelFlags(isNode1 = true, isEnabled = false)) @@ -518,8 +518,9 @@ class PaymentLifecycleSpec extends BaseRouterSpec { expectRouteRequest(routerForwarder, a, cfg) routerForwarder.forward(routerFixture.router) awaitCond(paymentFSM.stateName == WAITING_FOR_PAYMENT_COMPLETE) - val WaitingForComplete(_, cmd1, Nil, sharedSecrets1, _, route) = paymentFSM.stateData + val WaitingForComplete(_, cmd1, Nil, sharedSecrets1, _, route, sentAt) = paymentFSM.stateData + val failedAt = TimestampMilli.now() register.expectMsg(ForwardShortId(paymentFSM.toTyped, scid_ab, cmd1)) val failure = TemporaryChannelFailure(Some(update_bc)) sender.send(paymentFSM, addCompleted(HtlcResult.RemoteFail(UpdateFailHtlc(ByteVector32.Zeroes, 0, SphinxSpec.createAndWrap(sharedSecrets1.head.secret, failure))))) @@ -532,7 +533,13 @@ class PaymentLifecycleSpec extends BaseRouterSpec { expectRouteRequest(routerForwarder, a, cfg, ignore = Ignore(Set.empty, Set(ChannelDesc(update_bc.shortChannelId, b, c)))) routerForwarder.forward(routerFixture.router) // we allow 2 tries, so we send a 2nd request to the router - assert(sender.expectMsgType[PaymentFailed].failures == RemoteFailure(route.amount, route.hops, Sphinx.DecryptedFailurePacket(b, 1, failure)) :: LocalFailure(defaultAmountMsat, Nil, RouteNotFound) :: Nil) + val pf = sender.expectMsgType[PaymentFailed] + assert(pf.failures.size == 2) + assert(pf.failures.head.isInstanceOf[RemoteFailure]) + assert(pf.failures.head.asInstanceOf[RemoteFailure].failedAt >= failedAt) + assert(pf.failures.head.asInstanceOf[RemoteFailure].copy(failedAt = failedAt) == RemoteFailure(route.amount, route.hops, Sphinx.DecryptedFailurePacket(b, 1, failure), sentAt, failedAt)) + assert(pf.failures.last == LocalFailure(defaultAmountMsat, Nil, RouteNotFound)) + assert(pf.settledAt >= failedAt) } test("payment failed (Update)") { routerFixture => @@ -548,12 +555,13 @@ class PaymentLifecycleSpec extends BaseRouterSpec { expectRouteRequest(routerForwarder, a, cfg) routerForwarder.forward(routerFixture.router) awaitCond(paymentFSM.stateName == WAITING_FOR_PAYMENT_COMPLETE) - val WaitingForComplete(_, cmd1, Nil, sharedSecrets1, _, route1) = paymentFSM.stateData + val WaitingForComplete(_, cmd1, Nil, sharedSecrets1, _, route1, sentAt1) = paymentFSM.stateData register.expectMsg(ForwardShortId(paymentFSM.toTyped, scid_ab, cmd1)) // we change the cltv expiry val channelUpdate_bc_modified = makeChannelUpdate(Block.RegtestGenesisBlock.hash, priv_b, c, scid_bc, CltvExpiryDelta(42), htlcMinimumMsat = update_bc.htlcMinimumMsat, feeBaseMsat = update_bc.feeBaseMsat, feeProportionalMillionths = update_bc.feeProportionalMillionths, htlcMaximumMsat = update_bc.htlcMaximumMsat) val failure = IncorrectCltvExpiry(CltvExpiry(5), Some(channelUpdate_bc_modified)) + val failedAt1 = TimestampMilli.now() // and node replies with a failure containing a new channel update sender.send(paymentFSM, addCompleted(HtlcResult.RemoteFail(UpdateFailHtlc(ByteVector32.Zeroes, 0, SphinxSpec.createAndWrap(sharedSecrets1.head.secret, failure))))) @@ -563,12 +571,13 @@ class PaymentLifecycleSpec extends BaseRouterSpec { // router answers with a new route, taking into account the new update awaitCond(paymentFSM.stateName == WAITING_FOR_PAYMENT_COMPLETE) - val WaitingForComplete(_, cmd2, _, sharedSecrets2, _, route2) = paymentFSM.stateData + val WaitingForComplete(_, cmd2, _, sharedSecrets2, _, route2, sentAt2) = paymentFSM.stateData register.expectMsg(ForwardShortId(paymentFSM.toTyped, scid_ab, cmd2)) // we change the cltv expiry one more time val channelUpdate_bc_modified_2 = makeChannelUpdate(Block.RegtestGenesisBlock.hash, priv_b, c, scid_bc, CltvExpiryDelta(43), htlcMinimumMsat = update_bc.htlcMinimumMsat, feeBaseMsat = update_bc.feeBaseMsat, feeProportionalMillionths = update_bc.feeProportionalMillionths, htlcMaximumMsat = update_bc.htlcMaximumMsat) val failure2 = IncorrectCltvExpiry(CltvExpiry(5), Some(channelUpdate_bc_modified_2)) + val failedAt2 = TimestampMilli.now() // and node replies with a failure containing a new channel update sender.send(paymentFSM, addCompleted(HtlcResult.RemoteFail(UpdateFailHtlc(ByteVector32.Zeroes, 0, SphinxSpec.createAndWrap(sharedSecrets2.head.secret, failure2))))) @@ -580,7 +589,12 @@ class PaymentLifecycleSpec extends BaseRouterSpec { routerForwarder.forward(routerFixture.router) // this time the router can't find a route: game over - assert(sender.expectMsgType[PaymentFailed].failures == RemoteFailure(route1.amount, route1.hops, Sphinx.DecryptedFailurePacket(b, 1, failure)) :: RemoteFailure(route2.amount, route2.hops, Sphinx.DecryptedFailurePacket(b, 1, failure2)) :: LocalFailure(defaultAmountMsat, Nil, RouteNotFound) :: Nil) + val pf = sender.expectMsgType[PaymentFailed] + assert(pf.failures.size == 3) + assert(pf.failures.take(2).forall(_.isInstanceOf[RemoteFailure])) + assert(pf.failures(0).asInstanceOf[RemoteFailure].copy(failedAt = failedAt1) == RemoteFailure(route1.amount, route1.hops, Sphinx.DecryptedFailurePacket(b, 1, failure), sentAt1, failedAt1)) + assert(pf.failures(1).asInstanceOf[RemoteFailure].copy(failedAt = failedAt2) == RemoteFailure(route2.amount, route2.hops, Sphinx.DecryptedFailurePacket(b, 1, failure2), sentAt2, failedAt2)) + assert(pf.failures.last == LocalFailure(defaultAmountMsat, Nil, RouteNotFound)) awaitCond(nodeParams.db.payments.getOutgoingPayment(id).exists(_.status.isInstanceOf[OutgoingPaymentStatus.Failed])) routerForwarder.expectNoMessage(100 millis) @@ -595,7 +609,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec { expectRouteRequest(routerForwarder, a, cfg) routerForwarder.forward(routerFixture.router) awaitCond(paymentFSM.stateName == WAITING_FOR_PAYMENT_COMPLETE) - val WaitingForComplete(_, cmd1, Nil, sharedSecrets1, _, _) = paymentFSM.stateData + val WaitingForComplete(_, cmd1, Nil, sharedSecrets1, _, _, _) = paymentFSM.stateData register.expectMsg(ForwardShortId(paymentFSM.toTyped, scid_ab, cmd1)) // the node replies with a temporary failure containing the same update as the one we already have (likely a balance issue) @@ -627,7 +641,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec { expectRouteRequest(routerForwarder, a, cfg, target = recipient) routerForwarder.forward(routerFixture.router) awaitCond(paymentFSM.stateName == WAITING_FOR_PAYMENT_COMPLETE) - val WaitingForComplete(_, cmd1, Nil, sharedSecrets1, _, _) = paymentFSM.stateData + val WaitingForComplete(_, cmd1, Nil, sharedSecrets1, _, _, _) = paymentFSM.stateData register.expectMsg(ForwardShortId(paymentFSM.toTyped, scid_ab, cmd1)) // we change the cltv expiry @@ -646,7 +660,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec { // router answers with a new route, taking into account the new update awaitCond(paymentFSM.stateName == WAITING_FOR_PAYMENT_COMPLETE) - val WaitingForComplete(_, cmd2, _, _, _, _) = paymentFSM.stateData + val WaitingForComplete(_, cmd2, _, _, _, _, _) = paymentFSM.stateData register.expectMsg(ForwardShortId(paymentFSM.toTyped, scid_ab, cmd2)) assert(cmd2.cltvExpiry > cmd1.cltvExpiry) } @@ -667,7 +681,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec { expectRouteRequest(routerForwarder, a, cfg, target = recipient) routerForwarder.forward(routerFixture.router) awaitCond(paymentFSM.stateName == WAITING_FOR_PAYMENT_COMPLETE) - val WaitingForComplete(_, cmd1, Nil, sharedSecrets1, _, _) = paymentFSM.stateData + val WaitingForComplete(_, cmd1, Nil, sharedSecrets1, _, _, _) = paymentFSM.stateData register.expectMsg(ForwardShortId(paymentFSM.toTyped, scid_ab, cmd1)) // we disable the channel @@ -693,9 +707,10 @@ class PaymentLifecycleSpec extends BaseRouterSpec { expectRouteRequest(routerForwarder, a, cfg) routerForwarder.forward(router) awaitCond(paymentFSM.stateName == WAITING_FOR_PAYMENT_COMPLETE) - val WaitingForComplete(_, cmd1, Nil, sharedSecrets1, _, route1) = paymentFSM.stateData + val WaitingForComplete(_, cmd1, Nil, sharedSecrets1, _, route1, sentAt1) = paymentFSM.stateData register.expectMsg(ForwardShortId(paymentFSM.toTyped, scid_ab, cmd1)) + val failedAt = TimestampMilli.now() sender.send(paymentFSM, addCompleted(HtlcResult.RemoteFail(UpdateFailHtlc(ByteVector32.Zeroes, 0, SphinxSpec.createAndWrap(sharedSecrets1.head.secret, failure))))) // payment lifecycle forwards the embedded channelUpdate to the router @@ -704,7 +719,11 @@ class PaymentLifecycleSpec extends BaseRouterSpec { routerForwarder.forward(router) // we allow 2 tries, so we send a 2nd request to the router, which won't find another route - assert(sender.expectMsgType[PaymentFailed].failures == RemoteFailure(route1.amount, route1.hops, Sphinx.DecryptedFailurePacket(b, 1, failure)) :: LocalFailure(defaultAmountMsat, Nil, RouteNotFound) :: Nil) + val pf = sender.expectMsgType[PaymentFailed] + assert(pf.failures.size == 2) + assert(pf.failures.head.isInstanceOf[RemoteFailure]) + assert(pf.failures.head.asInstanceOf[RemoteFailure].copy(failedAt = failedAt) == RemoteFailure(route1.amount, route1.hops, Sphinx.DecryptedFailurePacket(b, 1, failure), sentAt1, failedAt)) + assert(pf.failures.last == LocalFailure(defaultAmountMsat, Nil, RouteNotFound)) awaitCond(nodeParams.db.payments.getOutgoingPayment(id).exists(_.status.isInstanceOf[OutgoingPaymentStatus.Failed])) } @@ -730,7 +749,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec { routerForwarder.expectMsgType[RouteRequest] routerForwarder.forward(routerFixture.router) awaitCond(paymentFSM.stateName == WAITING_FOR_PAYMENT_COMPLETE) - val WaitingForComplete(_, cmd1, Nil, sharedSecrets, _, route) = paymentFSM.stateData + val WaitingForComplete(_, cmd1, Nil, sharedSecrets, _, route, sentAt) = paymentFSM.stateData register.expectMsg(ForwardShortId(paymentFSM.toTyped, scid_ab, cmd1)) // The payment fails inside the blinded route: the introduction node sends back an error. @@ -746,7 +765,10 @@ class PaymentLifecycleSpec extends BaseRouterSpec { // Without the blinded route, the router cannot find a route to the recipient. val failed = sender.expectMsgType[PaymentFailed] - assert(failed.failures == Seq(RemoteFailure(defaultAmountMsat, route.hops ++ Seq(blindedHop), Sphinx.DecryptedFailurePacket(b, 1, failure)), LocalFailure(defaultAmountMsat, Nil, RouteNotFound))) + assert(failed.failures.size == 2) + assert(failed.failures.head.isInstanceOf[RemoteFailure]) + assert(failed.failures.head.asInstanceOf[RemoteFailure].copy(failedAt = 0 unixms) == RemoteFailure(defaultAmountMsat, route.hops ++ Seq(blindedHop), Sphinx.DecryptedFailurePacket(b, 1, failure), sentAt, 0 unixms)) + assert(failed.failures.last == LocalFailure(defaultAmountMsat, Nil, RouteNotFound)) awaitCond(nodeParams.db.payments.getOutgoingPayment(cfg.id).exists(_.status.isInstanceOf[OutgoingPaymentStatus.Failed])) } @@ -824,7 +846,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec { sender.send(paymentFSM, addCompleted(HtlcResult.OnChainFulfill(defaultPaymentPreimage))) val paymentOK = sender.expectMsgType[PaymentSent] - val PaymentSent(_, _, paymentOK.paymentPreimage, finalAmount, _, PartialPayment(_, partAmount, fee, ByteVector32.Zeroes, _, _) :: Nil, _) = eventListener.expectMsgType[PaymentSent] + val PaymentSent(_, paymentOK.paymentPreimage, finalAmount, _, PartialPayment(_, partAmount, fee, ByteVector32.Zeroes, _, _, _) :: Nil, _) = eventListener.expectMsgType[PaymentSent] assert(partAmount == request.amount) assert(finalAmount == defaultAmountMsat) @@ -878,14 +900,14 @@ class PaymentLifecycleSpec extends BaseRouterSpec { test("filter errors properly") { () => val failures = Seq( LocalFailure(defaultAmountMsat, Nil, RouteNotFound), - RemoteFailure(defaultAmountMsat, channelHopFromUpdate(a, b, update_ab) :: Nil, Sphinx.DecryptedFailurePacket(b, 1, TemporaryNodeFailure())), + RemoteFailure(defaultAmountMsat, channelHopFromUpdate(a, b, update_ab) :: Nil, Sphinx.DecryptedFailurePacket(b, 1, TemporaryNodeFailure()), 150 unixms, 300 unixms), LocalFailure(defaultAmountMsat, channelHopFromUpdate(a, b, update_ab) :: Nil, ChannelUnavailable(ByteVector32.Zeroes)), LocalFailure(defaultAmountMsat, Nil, RouteNotFound) ) val filtered = PaymentFailure.transformForUser(failures) val expected = Seq( LocalFailure(defaultAmountMsat, Nil, RouteNotFound), - RemoteFailure(defaultAmountMsat, channelHopFromUpdate(a, b, update_ab) :: Nil, Sphinx.DecryptedFailurePacket(b, 1, TemporaryNodeFailure())), + RemoteFailure(defaultAmountMsat, channelHopFromUpdate(a, b, update_ab) :: Nil, Sphinx.DecryptedFailurePacket(b, 1, TemporaryNodeFailure()), 150 unixms, 300 unixms), LocalFailure(defaultAmountMsat, channelHopFromUpdate(a, b, update_ab) :: Nil, ChannelUnavailable(ByteVector32.Zeroes)) ) assert(filtered == expected) @@ -903,22 +925,22 @@ class PaymentLifecycleSpec extends BaseRouterSpec { (LocalFailure(defaultAmountMsat, NodeHop(a, b, CltvExpiryDelta(144), 0 msat) :: NodeHop(b, c, CltvExpiryDelta(144), 0 msat) :: Nil, RouteNotFound), Set.empty, Set.empty), (LocalFailure(defaultAmountMsat, route_abcd, new RuntimeException("fatal")), Set.empty, Set(ChannelDesc(scid_ab, a, b))), // remote failure from final recipient -> all intermediate nodes behaved correctly - (RemoteFailure(defaultAmountMsat, route_abcd, Sphinx.DecryptedFailurePacket(d, 3, IncorrectOrUnknownPaymentDetails(100 msat, BlockHeight(42)))), Set.empty, Set.empty), + (RemoteFailure(defaultAmountMsat, route_abcd, Sphinx.DecryptedFailurePacket(d, 3, IncorrectOrUnknownPaymentDetails(100 msat, BlockHeight(42))), 150 unixms, 300 unixms), Set.empty, Set.empty), // remote failures from intermediate nodes -> depending on the failure, ignore either the failing node or its outgoing channel - (RemoteFailure(defaultAmountMsat, route_abcd, Sphinx.DecryptedFailurePacket(b, 1, PermanentNodeFailure())), Set(b), Set.empty), - (RemoteFailure(defaultAmountMsat, route_abcd, Sphinx.DecryptedFailurePacket(c, 2, TemporaryNodeFailure())), Set(c), Set.empty), - (RemoteFailure(defaultAmountMsat, route_abcd, Sphinx.DecryptedFailurePacket(b, 1, PermanentChannelFailure())), Set.empty, Set(ChannelDesc(scid_bc, b, c))), - (RemoteFailure(defaultAmountMsat, route_abcd, Sphinx.DecryptedFailurePacket(c, 2, UnknownNextPeer())), Set.empty, Set(ChannelDesc(scid_cd, c, d))), - (RemoteFailure(defaultAmountMsat, route_abcd, Sphinx.DecryptedFailurePacket(b, 1, FeeInsufficient(100 msat, Some(update_bc)))), Set.empty, Set.empty), - (RemoteFailure(defaultAmountMsat, blindedRoute_abc, Sphinx.DecryptedFailurePacket(b, 1, InvalidOnionBlinding(randomBytes32()))), Set.empty, Set(ChannelDesc(blindedHop_bc.dummyId, blindedHop_bc.nodeId, blindedHop_bc.nextNodeId))), - (RemoteFailure(defaultAmountMsat, blindedRoute_abc, Sphinx.DecryptedFailurePacket(blindedHop_bc.resolved.route.blindedNodeIds(1), 2, InvalidOnionBlinding(randomBytes32()))), Set.empty, Set(ChannelDesc(blindedHop_bc.dummyId, blindedHop_bc.nodeId, blindedHop_bc.nextNodeId))), + (RemoteFailure(defaultAmountMsat, route_abcd, Sphinx.DecryptedFailurePacket(b, 1, PermanentNodeFailure()), 150 unixms, 300 unixms), Set(b), Set.empty), + (RemoteFailure(defaultAmountMsat, route_abcd, Sphinx.DecryptedFailurePacket(c, 2, TemporaryNodeFailure()), 150 unixms, 300 unixms), Set(c), Set.empty), + (RemoteFailure(defaultAmountMsat, route_abcd, Sphinx.DecryptedFailurePacket(b, 1, PermanentChannelFailure()), 150 unixms, 300 unixms), Set.empty, Set(ChannelDesc(scid_bc, b, c))), + (RemoteFailure(defaultAmountMsat, route_abcd, Sphinx.DecryptedFailurePacket(c, 2, UnknownNextPeer()), 150 unixms, 300 unixms), Set.empty, Set(ChannelDesc(scid_cd, c, d))), + (RemoteFailure(defaultAmountMsat, route_abcd, Sphinx.DecryptedFailurePacket(b, 1, FeeInsufficient(100 msat, Some(update_bc))), 150 unixms, 300 unixms), Set.empty, Set.empty), + (RemoteFailure(defaultAmountMsat, blindedRoute_abc, Sphinx.DecryptedFailurePacket(b, 1, InvalidOnionBlinding(randomBytes32())), 150 unixms, 300 unixms), Set.empty, Set(ChannelDesc(blindedHop_bc.dummyId, blindedHop_bc.nodeId, blindedHop_bc.nextNodeId))), + (RemoteFailure(defaultAmountMsat, blindedRoute_abc, Sphinx.DecryptedFailurePacket(blindedHop_bc.resolved.route.blindedNodeIds(1), 2, InvalidOnionBlinding(randomBytes32())), 150 unixms, 300 unixms), Set.empty, Set(ChannelDesc(blindedHop_bc.dummyId, blindedHop_bc.nodeId, blindedHop_bc.nextNodeId))), // unreadable remote failures -> blacklist all nodes except our direct peer, the final recipient, the last hop or nodes relaying attribution data - (UnreadableRemoteFailure(defaultAmountMsat, channelHopFromUpdate(a, b, update_ab) :: Nil, Sphinx.CannotDecryptFailurePacket(ByteVector.empty, None), Nil), Set.empty, Set.empty), - (UnreadableRemoteFailure(defaultAmountMsat, channelHopFromUpdate(a, b, update_ab) :: channelHopFromUpdate(b, c, update_bc) :: channelHopFromUpdate(c, d, update_cd) :: Nil, Sphinx.CannotDecryptFailurePacket(ByteVector.empty, None), Nil), Set(c), Set.empty), - (UnreadableRemoteFailure(defaultAmountMsat, channelHopFromUpdate(a, b, update_ab) :: channelHopFromUpdate(b, c, update_bc) :: channelHopFromUpdate(c, d, update_cd) :: channelHopFromUpdate(d, e, update_de) :: Nil, Sphinx.CannotDecryptFailurePacket(ByteVector.empty, None), Nil), Set(c, d), Set.empty), - (UnreadableRemoteFailure(defaultAmountMsat, channelHopFromUpdate(a, b, update_ab) :: channelHopFromUpdate(b, c, update_bc) :: channelHopFromUpdate(c, d, update_cd) :: channelHopFromUpdate(d, e, update_de) :: Nil, Sphinx.CannotDecryptFailurePacket(ByteVector.empty, None), Seq(HoldTime(100 millis, b), HoldTime(90 millis, c), HoldTime(80 millis, d))), Set(d), Set.empty), - (UnreadableRemoteFailure(defaultAmountMsat, channelHopFromUpdate(a, b, update_ab) :: channelHopFromUpdate(b, c, update_bc) :: channelHopFromUpdate(c, d, update_cd) :: NodeHop(d, e, CltvExpiryDelta(24), 0 msat) :: Nil, Sphinx.CannotDecryptFailurePacket(ByteVector.empty, None), Nil), Set(c), Set.empty), - (UnreadableRemoteFailure(defaultAmountMsat, channelHopFromUpdate(a, b, update_ab) :: channelHopFromUpdate(b, c, update_bc) :: channelHopFromUpdate(c, d, update_cd) :: blindedHop_de :: Nil, Sphinx.CannotDecryptFailurePacket(ByteVector.empty, None), Nil), Set(c), Set.empty), + (UnreadableRemoteFailure(defaultAmountMsat, channelHopFromUpdate(a, b, update_ab) :: Nil, Sphinx.CannotDecryptFailurePacket(ByteVector.empty, None), 150 unixms, 300 unixms, Nil), Set.empty, Set.empty), + (UnreadableRemoteFailure(defaultAmountMsat, channelHopFromUpdate(a, b, update_ab) :: channelHopFromUpdate(b, c, update_bc) :: channelHopFromUpdate(c, d, update_cd) :: Nil, Sphinx.CannotDecryptFailurePacket(ByteVector.empty, None), 150 unixms, 300 unixms, Nil), Set(c), Set.empty), + (UnreadableRemoteFailure(defaultAmountMsat, channelHopFromUpdate(a, b, update_ab) :: channelHopFromUpdate(b, c, update_bc) :: channelHopFromUpdate(c, d, update_cd) :: channelHopFromUpdate(d, e, update_de) :: Nil, Sphinx.CannotDecryptFailurePacket(ByteVector.empty, None), 150 unixms, 300 unixms, Nil), Set(c, d), Set.empty), + (UnreadableRemoteFailure(defaultAmountMsat, channelHopFromUpdate(a, b, update_ab) :: channelHopFromUpdate(b, c, update_bc) :: channelHopFromUpdate(c, d, update_cd) :: channelHopFromUpdate(d, e, update_de) :: Nil, Sphinx.CannotDecryptFailurePacket(ByteVector.empty, None), 150 unixms, 300 unixms, Seq(HoldTime(100 millis, b), HoldTime(90 millis, c), HoldTime(80 millis, d))), Set(d), Set.empty), + (UnreadableRemoteFailure(defaultAmountMsat, channelHopFromUpdate(a, b, update_ab) :: channelHopFromUpdate(b, c, update_bc) :: channelHopFromUpdate(c, d, update_cd) :: NodeHop(d, e, CltvExpiryDelta(24), 0 msat) :: Nil, Sphinx.CannotDecryptFailurePacket(ByteVector.empty, None), 150 unixms, 300 unixms, Nil), Set(c), Set.empty), + (UnreadableRemoteFailure(defaultAmountMsat, channelHopFromUpdate(a, b, update_ab) :: channelHopFromUpdate(b, c, update_bc) :: channelHopFromUpdate(c, d, update_cd) :: blindedHop_de :: Nil, Sphinx.CannotDecryptFailurePacket(ByteVector.empty, None), 150 unixms, 300 unixms, Nil), Set(c), Set.empty), ) for ((failure, expectedNodes, expectedChannels) <- testCases) { @@ -928,8 +950,8 @@ class PaymentLifecycleSpec extends BaseRouterSpec { } val failures = Seq( - RemoteFailure(defaultAmountMsat, route_abcd, Sphinx.DecryptedFailurePacket(c, 2, TemporaryNodeFailure())), - RemoteFailure(defaultAmountMsat, route_abcd, Sphinx.DecryptedFailurePacket(b, 1, UnknownNextPeer())), + RemoteFailure(defaultAmountMsat, route_abcd, Sphinx.DecryptedFailurePacket(c, 2, TemporaryNodeFailure()), 150 unixms, 300 unixms), + RemoteFailure(defaultAmountMsat, route_abcd, Sphinx.DecryptedFailurePacket(b, 1, UnknownNextPeer()), 150 unixms, 300 unixms), LocalFailure(defaultAmountMsat, route_abcd, new RuntimeException("fatal")) ) val ignore = PaymentFailure.updateIgnored(failures, Ignore.empty) @@ -973,7 +995,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec { val Transition(_, WAITING_FOR_ROUTE, WAITING_FOR_CONFIDENCE) = monitor.expectMsgClass(classOf[Transition[_]]) val Transition(_, WAITING_FOR_CONFIDENCE, WAITING_FOR_PAYMENT_COMPLETE) = monitor.expectMsgClass(classOf[Transition[_]]) - val WaitingForComplete(_, _, Nil, sharedSecrets1, _, _) = paymentFSM.stateData + val WaitingForComplete(_, _, Nil, sharedSecrets1, _, _, _) = paymentFSM.stateData awaitCond(nodeParams.db.payments.getOutgoingPayment(id).exists(_.status == OutgoingPaymentStatus.Pending)) val Some(outgoing) = nodeParams.db.payments.getOutgoingPayment(id) assert(outgoing.copy(createdAt = 0 unixms) == OutgoingPayment(id, parentId, Some(defaultExternalId), defaultPaymentHash, PaymentType.Standard, defaultAmountMsat, defaultAmountMsat, d, 0 unixms, Some(defaultInvoice), None, OutgoingPaymentStatus.Pending)) 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 898c02bd7b..1603964e70 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 @@ -595,7 +595,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a { val payload = ChannelRelay.Standard(ShortChannelId(12345), 998900 msat, CltvExpiry(60), upgradeAccountability = false) - val r = createValidIncomingPacket(payload, 1000000 msat, CltvExpiry(70), accountableIn = false) + val r = createValidIncomingPacket(payload, 1000000 msat, CltvExpiry(70)) channelRelayer ! Relay(r, TestConstants.Alice.nodeParams.nodeId, 0.1) receiveConfidence(Reputation.Score(1.0, accountable = false)) // select the channel to the same node, with the lowest capacity and balance but still high enough to handle the payment @@ -616,7 +616,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a { // higher amount payment (have to increased incoming htlc amount for fees to be sufficient) val payload = ChannelRelay.Standard(ShortChannelId(12345), 50000000 msat, CltvExpiry(60), upgradeAccountability = false) - val r = createValidIncomingPacket(payload, 60000000 msat, CltvExpiry(70), accountableIn = false) + val r = createValidIncomingPacket(payload, 60000000 msat, CltvExpiry(70)) channelRelayer ! Relay(r, TestConstants.Alice.nodeParams.nodeId, 0.1) receiveConfidence(Reputation.Score(1.0, accountable = false)) expectFwdAdd(register, channelUpdates(ShortChannelId(11111)).channelId, r.amountToForward, r.outgoingCltv, outAccountable = false).message @@ -624,7 +624,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a { // lower amount payment val payload = ChannelRelay.Standard(ShortChannelId(12345), 1000 msat, CltvExpiry(60), upgradeAccountability = false) - val r = createValidIncomingPacket(payload, 60000000 msat, CltvExpiry(70), accountableIn = false) + val r = createValidIncomingPacket(payload, 60000000 msat, CltvExpiry(70)) channelRelayer ! Relay(r, TestConstants.Alice.nodeParams.nodeId, 0.1) receiveConfidence(Reputation.Score(1.0, accountable = false)) expectFwdAdd(register, channelUpdates(ShortChannelId(33333)).channelId, r.amountToForward, r.outgoingCltv, outAccountable = false).message @@ -701,7 +701,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a Seq(true, false).foreach { isIntroduction => testCases.foreach { htlcResult => - val r = createValidIncomingPacket(createBlindedPayload(Right(u.channelUpdate.shortChannelId), u.channelUpdate, isIntroduction), outgoingAmount + u.channelUpdate.feeBaseMsat, outgoingExpiry + u.channelUpdate.cltvExpiryDelta, accountableIn = false) + val r = createValidIncomingPacket(createBlindedPayload(Right(u.channelUpdate.shortChannelId), u.channelUpdate, isIntroduction), outgoingAmount + u.channelUpdate.feeBaseMsat, outgoingExpiry + u.channelUpdate.cltvExpiryDelta) channelRelayer ! WrappedLocalChannelUpdate(u) channelRelayer ! Relay(r, TestConstants.Alice.nodeParams.nodeId, 0.1) receiveConfidence(Reputation.Score(1.0, accountable = false)) @@ -734,7 +734,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a val channelId1 = channelIds(realScid1) val payload = ChannelRelay.Standard(realScid1, outgoingAmount, outgoingExpiry, upgradeAccountability = false) - val r = createValidIncomingPacket(payload, accountableIn = false) + val r = createValidIncomingPacket(payload) val u = createLocalUpdate(channelId1) val downstream_htlc = UpdateAddHtlc(channelId1, 7, outgoingAmount, paymentHash, outgoingExpiry, emptyOnionPacket, None, accountable = false, None) @@ -746,6 +746,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a ) testCases.foreach { testCase => + val now = TimestampMilli.now() channelRelayer ! WrappedLocalChannelUpdate(u) channelRelayer ! Relay(r, TestConstants.Alice.nodeParams.nodeId, 0.1) receiveConfidence(Reputation.Score(1.0, accountable = false)) @@ -760,7 +761,13 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a assert(fwd2.message.r == paymentPreimage) val paymentRelayed = eventListener.expectMessageType[ChannelPaymentRelayed] - assert(paymentRelayed.copy(receivedAt = 0 unixms, settledAt = 0 unixms) == ChannelPaymentRelayed(r.add.amountMsat, r.amountToForward, r.add.paymentHash, r.add.channelId, channelId1, receivedAt = 0 unixms, settledAt = 0 unixms)) + assert(paymentRelayed.paymentHash == r.add.paymentHash) + assert(paymentRelayed.amountIn == r.add.amountMsat) + assert(paymentRelayed.fromChannelId == r.add.channelId) + assert(paymentRelayed.amountOut == r.amountToForward) + assert(paymentRelayed.toChannelId == channelId1) + assert(paymentRelayed.startedAt == r.receivedAt) + assert(paymentRelayed.settledAt >= now) } } @@ -772,7 +779,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a channelRelayer ! WrappedLocalChannelUpdate(u) val payload = ChannelRelay.Standard(realScid1, outgoingAmount, outgoingExpiry, upgradeAccountability = false) - val r = createValidIncomingPacket(payload, accountableIn = false) + val r = createValidIncomingPacket(payload) channelRelayer ! Relay(r, TestConstants.Alice.nodeParams.nodeId, 0.1) receiveConfidence(Reputation.Score(1.0, accountable = false)) val fwd1 = expectFwdAdd(register, channelIds(realScid1), outgoingAmount, outgoingExpiry, outAccountable = false) @@ -886,14 +893,14 @@ object ChannelRelayerSpec { ChannelRelay.Blinded(tlvs, PaymentRelayData(blindedTlvs), randomKey().publicKey) } - def createValidIncomingPacket(payload: IntermediatePayload.ChannelRelay, amountIn: MilliSatoshi = 11_000_000 msat, expiryIn: CltvExpiry = CltvExpiry(400_100), accountableIn: Boolean = false): IncomingPaymentPacket.ChannelRelayPacket = { + def createValidIncomingPacket(payload: IntermediatePayload.ChannelRelay, amountIn: MilliSatoshi = 11_000_000 msat, expiryIn: CltvExpiry = CltvExpiry(400_100), accountableIn: Boolean = false, receivedAt: TimestampMilli = TimestampMilli.now()): IncomingPaymentPacket.ChannelRelayPacket = { val nextPathKey_opt = payload match { case p: ChannelRelay.Blinded => Some(UpdateAddHtlcTlv.PathKey(p.nextPathKey)) case _: ChannelRelay.Standard => None } val tlvs = TlvStream(Set[Option[UpdateAddHtlcTlv]](nextPathKey_opt, if (accountableIn) Some(UpdateAddHtlcTlv.Accountable()) else None).flatten) val add_ab = UpdateAddHtlc(channelId = randomBytes32(), id = 123456, amountIn, paymentHash, expiryIn, emptyOnionPacket, tlvs) - ChannelRelayPacket(add_ab, payload, emptyOnionPacket, TimestampMilli.now()) + ChannelRelayPacket(add_ab, payload, emptyOnionPacket, receivedAt) } def createAliases(channelId: ByteVector32): ShortIdAliases = { 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 ebca25ce83..7a01f66349 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 @@ -49,7 +49,7 @@ import fr.acinq.eclair.wire.protocol.PaymentOnion.{FinalPayload, IntermediatePay import fr.acinq.eclair.wire.protocol.RouteBlindingEncryptedDataCodecs.blindedRouteDataCodec import fr.acinq.eclair.wire.protocol.RouteBlindingEncryptedDataTlv.{AllowedFeatures, PathId, PaymentConstraints} import fr.acinq.eclair.wire.protocol._ -import fr.acinq.eclair.{Alias, BlockHeight, Bolt11Feature, Bolt12Feature, CltvExpiry, CltvExpiryDelta, EncodedNodeId, FeatureSupport, Features, MilliSatoshi, MilliSatoshiLong, NodeParams, RealShortChannelId, ShortChannelId, TestConstants, TimestampMilli, UInt64, randomBytes32, randomKey} +import fr.acinq.eclair.{Alias, BlockHeight, Bolt11Feature, Bolt12Feature, CltvExpiry, CltvExpiryDelta, EncodedNodeId, FeatureSupport, Features, MilliSatoshi, MilliSatoshiLong, NodeParams, RealShortChannelId, ShortChannelId, TestConstants, TimestampMilli, TimestampMilliLong, UInt64, randomBytes32, randomKey} import org.scalatest.funsuite.FixtureAnyFunSuiteLike import org.scalatest.{Outcome, Tag} import scodec.bits.{ByteVector, HexStringSyntax} @@ -485,7 +485,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl val nodeRelayerAdapters = mockPayFSM.expectMessageType[SendMultiPartPayment].replyTo // The proposed fees are low, so we ask the sender to raise them. - nodeRelayerAdapters ! PaymentFailed(relayId, paymentHash, LocalFailure(outgoingAmount, Nil, BalanceTooLow) :: Nil) + nodeRelayerAdapters ! PaymentFailed(relayId, paymentHash, LocalFailure(outgoingAmount, Nil, BalanceTooLow) :: Nil, 100 unixms, 150 unixms) incomingMultiPart.foreach { p => val fwd = register.expectMessageType[Register.Forward[CMD_FAIL_HTLC]] assert(fwd.channelId == p.add.channelId) @@ -537,8 +537,8 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl router.expectMessageType[RouteRequest] // If we're having a hard time finding routes, raising the fee/cltv will likely help. - val failures = LocalFailure(outgoingAmount, Nil, RouteNotFound) :: RemoteFailure(outgoingAmount, Nil, Sphinx.DecryptedFailurePacket(outgoingNodeId, 1, PermanentNodeFailure())) :: LocalFailure(outgoingAmount, Nil, RouteNotFound) :: Nil - payFSM ! PaymentFailed(relayId, incomingMultiPart.head.add.paymentHash, failures) + val failures = LocalFailure(outgoingAmount, Nil, RouteNotFound) :: RemoteFailure(outgoingAmount, Nil, Sphinx.DecryptedFailurePacket(outgoingNodeId, 1, PermanentNodeFailure()), 100 unixms, 150 unixms) :: LocalFailure(outgoingAmount, Nil, RouteNotFound) :: Nil + payFSM ! PaymentFailed(relayId, incomingMultiPart.head.add.paymentHash, failures, 100 unixms, 150 unixms) incomingMultiPart.foreach { p => val fwd = register.expectMessageType[Register.Forward[CMD_FAIL_HTLC]] @@ -562,8 +562,8 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl val payFSM = mockPayFSM.expectMessageType[akka.actor.ActorRef] router.expectMessageType[RouteRequest] - val failures = RemoteFailure(outgoingAmount, Nil, Sphinx.DecryptedFailurePacket(outgoingNodeId, 1, FinalIncorrectHtlcAmount(42 msat))) :: UnreadableRemoteFailure(outgoingAmount, Nil, Sphinx.CannotDecryptFailurePacket(ByteVector.empty, None), Nil) :: Nil - payFSM ! PaymentFailed(relayId, incomingMultiPart.head.add.paymentHash, failures) + val failures = RemoteFailure(outgoingAmount, Nil, Sphinx.DecryptedFailurePacket(outgoingNodeId, 1, FinalIncorrectHtlcAmount(42 msat)), 100 unixms, 150 unixms) :: UnreadableRemoteFailure(outgoingAmount, Nil, Sphinx.CannotDecryptFailurePacket(ByteVector.empty, None), 100 unixms, 150 unixms, Nil) :: Nil + payFSM ! PaymentFailed(relayId, incomingMultiPart.head.add.paymentHash, failures, 100 unixms, 150 unixms) incomingMultiPart.foreach { p => val fwd = register.expectMessageType[Register.Forward[CMD_FAIL_HTLC]] @@ -668,7 +668,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl // The corresponding commands conflict with the previous fulfill and are ignored. val downstreamHtlc = UpdateAddHtlc(randomBytes32(), 7, outgoingAmount, paymentHash, outgoingExpiry, TestConstants.emptyOnionPacket, None, accountable = false, None) val failure = LocalFailure(outgoingAmount, Nil, HtlcOverriddenByLocalCommit(randomBytes32(), downstreamHtlc)) - nodeRelayerAdapters ! PaymentFailed(relayId, incomingMultiPart.head.add.paymentHash, Seq(failure)) + nodeRelayerAdapters ! PaymentFailed(relayId, incomingMultiPart.head.add.paymentHash, Seq(failure), 100 unixms, 150 unixms) eventListener.expectNoMessage(100 millis) // the payment didn't succeed, but didn't fail either, so we just ignore it parent.expectMessageType[NodeRelayer.RelayComplete] register.expectNoMessage(100 millis) @@ -738,7 +738,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl validateOutgoingPayment(outgoingPayment) // The outgoing payment fails because we don't have enough balance: we trigger on-the-fly funding. - outgoingPayment.replyTo ! PaymentFailed(relayId, paymentHash, LocalFailure(outgoingAmount, Nil, BalanceTooLow) :: Nil) + outgoingPayment.replyTo ! PaymentFailed(relayId, paymentHash, LocalFailure(outgoingAmount, Nil, BalanceTooLow) :: Nil, 100 unixms, 150 unixms) val fwd = register.expectMessageType[Register.ForwardNodeId[Peer.ProposeOnTheFlyFunding]] assert(fwd.nodeId == outgoingNodeId) assert(fwd.message.nextPathKey_opt.isEmpty) @@ -784,7 +784,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl validateOutgoingPayment(outgoingPayment) // The outgoing payment fails, but it's not a liquidity issue. - outgoingPayment.replyTo ! PaymentFailed(relayId, paymentHash, RemoteFailure(outgoingAmount, Nil, Sphinx.DecryptedFailurePacket(outgoingNodeId, 1, TemporaryNodeFailure())) :: Nil) + outgoingPayment.replyTo ! PaymentFailed(relayId, paymentHash, RemoteFailure(outgoingAmount, Nil, Sphinx.DecryptedFailurePacket(outgoingNodeId, 1, TemporaryNodeFailure()), 100 unixms, 150 unixms) :: Nil, 100 unixms, 150 unixms) incomingMultiPart.foreach { p => val fwd = register.expectMessageType[Register.Forward[CMD_FAIL_HTLC]] assert(fwd.channelId == p.add.channelId) @@ -1039,7 +1039,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl val outgoingPayment = mockPayFSM.expectMessageType[SendMultiPartPayment] // The outgoing payment fails because we don't have enough balance: we trigger on-the-fly funding. - outgoingPayment.replyTo ! PaymentFailed(relayId, paymentHash, LocalFailure(outgoingAmount, Nil, BalanceTooLow) :: Nil) + outgoingPayment.replyTo ! PaymentFailed(relayId, paymentHash, LocalFailure(outgoingAmount, Nil, BalanceTooLow) :: Nil, 100 unixms, 150 unixms) val fwd = register.expectMessageType[Register.ForwardNodeId[Peer.ProposeOnTheFlyFunding]] assert(fwd.nodeId == outgoingNodeId) assert(fwd.message.nextPathKey_opt.nonEmpty) @@ -1078,7 +1078,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl val outgoingPayment = mockPayFSM.expectMessageType[SendMultiPartPayment] // The outgoing payment fails because we don't have enough balance: we trigger on-the-fly funding, but can't reach our peer. - outgoingPayment.replyTo ! PaymentFailed(relayId, paymentHash, LocalFailure(outgoingAmount, Nil, BalanceTooLow) :: Nil) + outgoingPayment.replyTo ! PaymentFailed(relayId, paymentHash, LocalFailure(outgoingAmount, Nil, BalanceTooLow) :: Nil, 100 unixms, 110 unixms) val fwd = register.expectMessageType[Register.ForwardNodeId[Peer.ProposeOnTheFlyFunding]] fwd.message.replyTo ! Peer.ProposeOnTheFlyFundingResponse.NotAvailable("peer disconnected") // We fail the payments immediately since the recipient isn't available. @@ -1185,25 +1185,25 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl object NodeRelayerSpec { - val relayId = UUID.randomUUID() + val relayId: UUID = UUID.randomUUID() - val paymentPreimage = randomBytes32() - val paymentHash = Crypto.sha256(paymentPreimage) - val paymentSecret = randomBytes32() + val paymentPreimage: ByteVector32 = randomBytes32() + val paymentHash: ByteVector32 = Crypto.sha256(paymentPreimage) + val paymentSecret: ByteVector32 = randomBytes32() - val outgoingAmount = 40_000_000 msat - val outgoingExpiry = CltvExpiry(490000) - val outgoingNodeKey = randomKey() - val outgoingNodeId = outgoingNodeKey.publicKey + val outgoingAmount: MilliSatoshi = 40_000_000 msat + val outgoingExpiry: CltvExpiry = CltvExpiry(490000) + val outgoingNodeKey: PrivateKey = randomKey() + val outgoingNodeId: PublicKey = outgoingNodeKey.publicKey - val incomingAmount = 41_000_000 msat - val incomingSecret = randomBytes32() - val incomingMultiPart = Seq( - createValidIncomingPacket(15_000_000 msat, incomingAmount, CltvExpiry(500000), outgoingAmount, outgoingExpiry, TimestampMilli(1000), accountableIn = false), - createValidIncomingPacket(15_000_000 msat, incomingAmount, CltvExpiry(499999), outgoingAmount, outgoingExpiry, TimestampMilli(2000), accountableIn = false), - createValidIncomingPacket(11_000_000 msat, incomingAmount, CltvExpiry(499999), outgoingAmount, outgoingExpiry, TimestampMilli(3000), accountableIn = false) + val incomingAmount: MilliSatoshi = 41_000_000 msat + val incomingSecret: ByteVector32 = randomBytes32() + val incomingMultiPart: Seq[RelayToTrampolinePacket] = Seq( + createValidIncomingPacket(15_000_000 msat, incomingAmount, CltvExpiry(500000), outgoingAmount, outgoingExpiry, TimestampMilli(1000)), + createValidIncomingPacket(15_000_000 msat, incomingAmount, CltvExpiry(499999), outgoingAmount, outgoingExpiry, TimestampMilli(2000)), + createValidIncomingPacket(11_000_000 msat, incomingAmount, CltvExpiry(499999), outgoingAmount, outgoingExpiry, TimestampMilli(3000)) ) - val incomingSinglePart = createValidIncomingPacket(incomingAmount, incomingAmount, CltvExpiry(500000), outgoingAmount, outgoingExpiry, TimestampMilli(5000)) + val incomingSinglePart: RelayToTrampolinePacket = createValidIncomingPacket(incomingAmount, incomingAmount, CltvExpiry(500000), outgoingAmount, outgoingExpiry, TimestampMilli(5000)) val incomingAsyncPayment: Seq[RelayToTrampolinePacket] = incomingMultiPart.map(p => p.copy(innerPayload = IntermediatePayload.NodeRelay.Standard.createNodeRelayForAsyncPayment(p.innerPayload.amountToForward, p.innerPayload.outgoingCltv, outgoingNodeId, upgradeAccountability = false))) def asyncTimeoutHeight(nodeParams: NodeParams): BlockHeight = @@ -1213,7 +1213,7 @@ object NodeRelayerSpec { (paymentPackets.map(_.outerPayload.expiry).min - nodeParams.relayParams.asyncPaymentsParams.cancelSafetyBeforeTimeout).blockHeight def createSuccessEvent(): PaymentSent = - PaymentSent(relayId, paymentHash, paymentPreimage, outgoingAmount, outgoingNodeId, Seq(PaymentSent.PartialPayment(UUID.randomUUID(), outgoingAmount, 10 msat, randomBytes32(), None)), None) + PaymentSent(relayId, paymentPreimage, outgoingAmount, outgoingNodeId, Seq(PaymentSent.PartialPayment(UUID.randomUUID(), outgoingAmount, 10 msat, randomBytes32(), None, 0 unixms, 50 unixms)), None) def createTrampolinePacket(amount: MilliSatoshi, expiry: CltvExpiry): OnionRoutingPacket = { val payload = NodePayload(outgoingNodeId, FinalPayload.Standard.createPayload(amount, amount, expiry, paymentSecret, upgradeAccountability = false)) From 98a13c566b923029ce761506fc8188efaafee31d Mon Sep 17 00:00:00 2001 From: t-bast Date: Fri, 9 Jan 2026 11:25:34 +0100 Subject: [PATCH 2/3] Fix API unit tests --- .../acinq/eclair/json/JsonSerializers.scala | 5 ++++ .../fr/acinq/eclair/api/ApiServiceSpec.scala | 27 +++++++------------ 2 files changed, 15 insertions(+), 17 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/json/JsonSerializers.scala b/eclair-core/src/main/scala/fr/acinq/eclair/json/JsonSerializers.scala index 30a02ac999..c0f7af0240 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/json/JsonSerializers.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/json/JsonSerializers.scala @@ -383,6 +383,9 @@ object PaymentFailedSummarySerializer extends ConvertClassSerializer[PaymentFail )) // @formatter:on +private case class PaymentSentJson(id: UUID, paymentHash: ByteVector32, paymentPreimage: ByteVector32, recipientAmount: MilliSatoshi, recipientNodeId: PublicKey, parts: Seq[PaymentSent.PartialPayment]) +object PaymentSentSerializer extends ConvertClassSerializer[PaymentSent](p => PaymentSentJson(p.id, p.paymentHash, p.paymentPreimage, p.recipientAmount, p.recipientNodeId, p.parts)) + object ThrowableSerializer extends MinimalSerializer({ case t: Throwable if t.getMessage != null => JString(t.getMessage) case t: Throwable => JString(t.getClass.getSimpleName) @@ -680,6 +683,7 @@ object CustomTypeHints { val paymentEvent: CustomTypeHints = CustomTypeHints(Map( classOf[PaymentSent] -> "payment-sent", + classOf[PaymentSentJson] -> "payment-sent", classOf[ChannelPaymentRelayed] -> "payment-relayed", classOf[TrampolinePaymentRelayed] -> "trampoline-payment-relayed", classOf[OnTheFlyFundingPaymentRelayed] -> "on-the-fly-funding-payment-relayed", @@ -795,6 +799,7 @@ object JsonSerializers { GlobalBalanceSerializer + PeerInfoSerializer + PaymentFailedSummarySerializer + + PaymentSentSerializer + OnionMessageReceivedSerializer + ShortIdAliasesSerializer + FundingTxStatusSerializer + diff --git a/eclair-node/src/test/scala/fr/acinq/eclair/api/ApiServiceSpec.scala b/eclair-node/src/test/scala/fr/acinq/eclair/api/ApiServiceSpec.scala index f71e03e1d4..33f1466b54 100644 --- a/eclair-node/src/test/scala/fr/acinq/eclair/api/ApiServiceSpec.scala +++ b/eclair-node/src/test/scala/fr/acinq/eclair/api/ApiServiceSpec.scala @@ -623,7 +623,7 @@ class ApiServiceSpec extends AnyFunSuite with ScalatestRouteTest with IdiomaticM val mockService = new MockService(eclair) val uuid = UUID.fromString("487da196-a4dc-4b1e-92b4-3e5e905e9f3f") - val paymentSent = PaymentSent(uuid, ByteVector32.Zeroes, ByteVector32.One, 25 msat, aliceNodeId, Seq(PaymentSent.PartialPayment(uuid, 21 msat, 1 msat, ByteVector32.Zeroes, None, TimestampMilli(1553784337711L))), None) + val paymentSent = PaymentSent(uuid, ByteVector32.One, 25 msat, aliceNodeId, Seq(PaymentSent.PartialPayment(uuid, 21 msat, 1 msat, ByteVector32.Zeroes, None, TimestampMilli(1553784337650L), TimestampMilli(1553784337711L))), None) eclair.sendBlocking(any, any, any, any, any, any, any)(any[Timeout]).returns(Future.successful(paymentSent)) Post("/payinvoice", FormData("invoice" -> invoice, "blocking" -> "true").toEntity) ~> addCredentials(BasicHttpCredentials("", mockApi().password)) ~> @@ -632,11 +632,11 @@ class ApiServiceSpec extends AnyFunSuite with ScalatestRouteTest with IdiomaticM assert(handled) assert(status == OK) val response = entityAs[String] - val expected = """{"type":"payment-sent","id":"487da196-a4dc-4b1e-92b4-3e5e905e9f3f","paymentHash":"0000000000000000000000000000000000000000000000000000000000000000","paymentPreimage":"0100000000000000000000000000000000000000000000000000000000000000","recipientAmount":25,"recipientNodeId":"03af0ed6052cf28d670665549bc86f4b721c9fdb309d40c58f5811f63966e005d0","parts":[{"id":"487da196-a4dc-4b1e-92b4-3e5e905e9f3f","amount":21,"feesPaid":1,"toChannelId":"0000000000000000000000000000000000000000000000000000000000000000","timestamp":{"iso":"2019-03-28T14:45:37.711Z","unix":1553784337}}]}""" + val expected = """{"type":"payment-sent","id":"487da196-a4dc-4b1e-92b4-3e5e905e9f3f","paymentHash":"01d0fabd251fcbbe2b93b4b927b26ad2a1a99077152e45ded1e678afa45dbec5","paymentPreimage":"0100000000000000000000000000000000000000000000000000000000000000","recipientAmount":25,"recipientNodeId":"03af0ed6052cf28d670665549bc86f4b721c9fdb309d40c58f5811f63966e005d0","parts":[{"id":"487da196-a4dc-4b1e-92b4-3e5e905e9f3f","amount":21,"feesPaid":1,"toChannelId":"0000000000000000000000000000000000000000000000000000000000000000","startedAt":{"iso":"2019-03-28T14:45:37.650Z","unix":1553784337},"settledAt":{"iso":"2019-03-28T14:45:37.711Z","unix":1553784337}}]}""" assert(response == expected) } - val paymentFailed = PaymentFailed(uuid, ByteVector32.Zeroes, failures = Seq.empty, timestamp = TimestampMilli(1553784963659L)) + val paymentFailed = PaymentFailed(uuid, ByteVector32.Zeroes, failures = Seq.empty, startedAt = TimestampMilli(1553784963507L), settledAt = TimestampMilli(1553784963659L)) eclair.sendBlocking(any, any, any, any, any, any, any)(any[Timeout]).returns(Future.successful(paymentFailed)) Post("/payinvoice", FormData("invoice" -> invoice, "blocking" -> "true").toEntity) ~> addCredentials(BasicHttpCredentials("", mockApi().password)) ~> @@ -645,7 +645,7 @@ class ApiServiceSpec extends AnyFunSuite with ScalatestRouteTest with IdiomaticM assert(handled) assert(status == OK) val response = entityAs[String] - val expected = """{"type":"payment-failed","id":"487da196-a4dc-4b1e-92b4-3e5e905e9f3f","paymentHash":"0000000000000000000000000000000000000000000000000000000000000000","failures":[],"timestamp":{"iso":"2019-03-28T14:56:03.659Z","unix":1553784963}}""" + val expected = """{"type":"payment-failed","id":"487da196-a4dc-4b1e-92b4-3e5e905e9f3f","paymentHash":"0000000000000000000000000000000000000000000000000000000000000000","failures":[],"startedAt":{"iso":"2019-03-28T14:56:03.507Z","unix":1553784963},"settledAt":{"iso":"2019-03-28T14:56:03.659Z","unix":1553784963}}""" assert(response == expected) } } @@ -1109,7 +1109,6 @@ class ApiServiceSpec extends AnyFunSuite with ScalatestRouteTest with IdiomaticM assert(status == OK) eclair.audit(123456 unixsec, 654321 unixsec, Some(Paginated(count = 1, skip = 2)))(any[Timeout]).wasCalled(once) } - } test("the websocket should return typed objects") { @@ -1122,20 +1121,20 @@ class ApiServiceSpec extends AnyFunSuite with ScalatestRouteTest with IdiomaticM addCredentials(BasicHttpCredentials("", mockApi().password)) ~> mockService.webSocket ~> check { - val pf = PaymentFailed(fixedUUID, ByteVector32.Zeroes, failures = Seq.empty, timestamp = TimestampMilli(1553784963659L)) - val expectedSerializedPf = """{"type":"payment-failed","id":"487da196-a4dc-4b1e-92b4-3e5e905e9f3f","paymentHash":"0000000000000000000000000000000000000000000000000000000000000000","failures":[],"timestamp":{"iso":"2019-03-28T14:56:03.659Z","unix":1553784963}}""" + val pf = PaymentFailed(fixedUUID, ByteVector32.Zeroes, failures = Seq.empty, startedAt = TimestampMilli(1553784963221L), settledAt = TimestampMilli(1553784963659L)) + val expectedSerializedPf = """{"type":"payment-failed","id":"487da196-a4dc-4b1e-92b4-3e5e905e9f3f","paymentHash":"0000000000000000000000000000000000000000000000000000000000000000","failures":[],"startedAt":{"iso":"2019-03-28T14:56:03.221Z","unix":1553784963},"settledAt":{"iso":"2019-03-28T14:56:03.659Z","unix":1553784963}}""" assert(serialization.write(pf) == expectedSerializedPf) system.eventStream.publish(pf) wsClient.expectMessage(expectedSerializedPf) - val ps = PaymentSent(fixedUUID, ByteVector32.Zeroes, ByteVector32.One, 25 msat, aliceNodeId, Seq(PaymentSent.PartialPayment(fixedUUID, 21 msat, 1 msat, ByteVector32.Zeroes, None, TimestampMilli(1553784337711L))), None) - val expectedSerializedPs = """{"type":"payment-sent","id":"487da196-a4dc-4b1e-92b4-3e5e905e9f3f","paymentHash":"0000000000000000000000000000000000000000000000000000000000000000","paymentPreimage":"0100000000000000000000000000000000000000000000000000000000000000","recipientAmount":25,"recipientNodeId":"03af0ed6052cf28d670665549bc86f4b721c9fdb309d40c58f5811f63966e005d0","parts":[{"id":"487da196-a4dc-4b1e-92b4-3e5e905e9f3f","amount":21,"feesPaid":1,"toChannelId":"0000000000000000000000000000000000000000000000000000000000000000","timestamp":{"iso":"2019-03-28T14:45:37.711Z","unix":1553784337}}]}""" + val ps = PaymentSent(fixedUUID, ByteVector32.One, 25 msat, aliceNodeId, Seq(PaymentSent.PartialPayment(fixedUUID, 21 msat, 1 msat, ByteVector32.Zeroes, None, startedAt = TimestampMilli(1553784337539L), settledAt = TimestampMilli(1553784337711L))), None) + val expectedSerializedPs = """{"type":"payment-sent","id":"487da196-a4dc-4b1e-92b4-3e5e905e9f3f","paymentHash":"01d0fabd251fcbbe2b93b4b927b26ad2a1a99077152e45ded1e678afa45dbec5","paymentPreimage":"0100000000000000000000000000000000000000000000000000000000000000","recipientAmount":25,"recipientNodeId":"03af0ed6052cf28d670665549bc86f4b721c9fdb309d40c58f5811f63966e005d0","parts":[{"id":"487da196-a4dc-4b1e-92b4-3e5e905e9f3f","amount":21,"feesPaid":1,"toChannelId":"0000000000000000000000000000000000000000000000000000000000000000","startedAt":{"iso":"2019-03-28T14:45:37.539Z","unix":1553784337},"settledAt":{"iso":"2019-03-28T14:45:37.711Z","unix":1553784337}}]}""" assert(serialization.write(ps) == expectedSerializedPs) system.eventStream.publish(ps) wsClient.expectMessage(expectedSerializedPs) val prel = ChannelPaymentRelayed(21 msat, 20 msat, ByteVector32.Zeroes, ByteVector32.Zeroes, ByteVector32.One, TimestampMilli(1553784961048L), TimestampMilli(1553784963659L)) - val expectedSerializedPrel = """{"type":"payment-relayed","amountIn":21,"amountOut":20,"paymentHash":"0000000000000000000000000000000000000000000000000000000000000000","fromChannelId":"0000000000000000000000000000000000000000000000000000000000000000","toChannelId":"0100000000000000000000000000000000000000000000000000000000000000","receivedAt":{"iso":"2019-03-28T14:56:01.048Z","unix":1553784961},"settledAt":{"iso":"2019-03-28T14:56:03.659Z","unix":1553784963}}""" + val expectedSerializedPrel = """{"type":"payment-relayed","amountIn":21,"amountOut":20,"paymentHash":"0000000000000000000000000000000000000000000000000000000000000000","fromChannelId":"0000000000000000000000000000000000000000000000000000000000000000","toChannelId":"0100000000000000000000000000000000000000000000000000000000000000","startedAt":{"iso":"2019-03-28T14:56:01.048Z","unix":1553784961},"settledAt":{"iso":"2019-03-28T14:56:03.659Z","unix":1553784963}}""" assert(serialization.write(prel) == expectedSerializedPrel) system.eventStream.publish(prel) wsClient.expectMessage(expectedSerializedPrel) @@ -1147,17 +1146,11 @@ class ApiServiceSpec extends AnyFunSuite with ScalatestRouteTest with IdiomaticM wsClient.expectMessage(expectedSerializedPtrel) val precv = PaymentReceived(ByteVector32.Zeroes, Seq(PaymentReceived.PartialPayment(21 msat, ByteVector32.Zeroes, TimestampMilli(1553784963659L)))) - val expectedSerializedPrecv = """{"type":"payment-received","paymentHash":"0000000000000000000000000000000000000000000000000000000000000000","parts":[{"amount":21,"fromChannelId":"0000000000000000000000000000000000000000000000000000000000000000","timestamp":{"iso":"2019-03-28T14:56:03.659Z","unix":1553784963}}]}""" + val expectedSerializedPrecv = """{"type":"payment-received","paymentHash":"0000000000000000000000000000000000000000000000000000000000000000","parts":[{"amount":21,"fromChannelId":"0000000000000000000000000000000000000000000000000000000000000000","receivedAt":{"iso":"2019-03-28T14:56:03.659Z","unix":1553784963}}]}""" assert(serialization.write(precv) == expectedSerializedPrecv) system.eventStream.publish(precv) wsClient.expectMessage(expectedSerializedPrecv) - val pset = PaymentSettlingOnChain(fixedUUID, 21 msat, ByteVector32.One, timestamp = TimestampMilli(1553785442676L)) - val expectedSerializedPset = """{"type":"payment-settling-onchain","id":"487da196-a4dc-4b1e-92b4-3e5e905e9f3f","amount":21,"paymentHash":"0100000000000000000000000000000000000000000000000000000000000000","timestamp":{"iso":"2019-03-28T15:04:02.676Z","unix":1553785442}}""" - assert(serialization.write(pset) == expectedSerializedPset) - system.eventStream.publish(pset) - wsClient.expectMessage(expectedSerializedPset) - val chcr = ChannelCreated(system.deadLetters, system.deadLetters, bobNodeId, isOpener = true, ByteVector32.One, FeeratePerKw(25 sat), Some(FeeratePerKw(20 sat))) val expectedSerializedChcr = """{"type":"channel-created","remoteNodeId":"039dc0e0b1d25905e44fdf6f8e89755a5e219685840d0bc1d28d3308f9628a3585","isOpener":true,"temporaryChannelId":"0100000000000000000000000000000000000000000000000000000000000000","commitTxFeeratePerKw":25,"fundingTxFeeratePerKw":20}""" assert(serialization.write(chcr) == expectedSerializedChcr) From 5be0537e1d8670838823308fd5dd52b724bb6edc Mon Sep 17 00:00:00 2001 From: t-bast Date: Fri, 9 Jan 2026 13:45:15 +0100 Subject: [PATCH 3/3] Take path-finding duration into account in `PaymentSent` We add a `startedAt` field to `PaymentSent` to take into account the duration of path-finding, which is excluded from individual parts' start time. --- .../src/main/scala/fr/acinq/eclair/db/pg/PgAuditDb.scala | 3 ++- .../scala/fr/acinq/eclair/db/sqlite/SqliteAuditDb.scala | 3 ++- .../scala/fr/acinq/eclair/json/JsonSerializers.scala | 4 ++-- .../scala/fr/acinq/eclair/payment/PaymentEvents.scala | 3 +-- .../eclair/payment/relay/PostRestartHtlcCleaner.scala | 9 +++++---- .../eclair/payment/send/MultiPartPaymentLifecycle.scala | 6 +++--- .../fr/acinq/eclair/payment/send/PaymentInitiator.scala | 4 ++-- .../fr/acinq/eclair/payment/send/PaymentLifecycle.scala | 2 +- .../eclair/payment/send/TrampolinePaymentLifecycle.scala | 2 +- .../src/test/scala/fr/acinq/eclair/EclairImplSpec.scala | 7 ++++--- .../src/test/scala/fr/acinq/eclair/db/AuditDbSpec.scala | 6 +++--- .../test/scala/fr/acinq/eclair/db/PaymentsDbSpec.scala | 6 +++--- .../eclair/integration/PaymentIntegrationSpec.scala | 6 ++++-- .../eclair/payment/MultiPartPaymentLifecycleSpec.scala | 6 +++--- .../fr/acinq/eclair/payment/PaymentInitiatorSpec.scala | 4 ++-- .../fr/acinq/eclair/payment/PaymentLifecycleSpec.scala | 2 +- .../fr/acinq/eclair/payment/relay/NodeRelayerSpec.scala | 2 +- .../test/scala/fr/acinq/eclair/api/ApiServiceSpec.scala | 8 ++++---- 18 files changed, 44 insertions(+), 39 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/db/pg/PgAuditDb.scala b/eclair-core/src/main/scala/fr/acinq/eclair/db/pg/PgAuditDb.scala index 89277fd1fd..3623a0cdb7 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/db/pg/PgAuditDb.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/db/pg/PgAuditDb.scala @@ -373,7 +373,8 @@ class PgAuditDb(implicit ds: DataSource) extends AuditDb with Logging { MilliSatoshi(rs.getLong("recipient_amount_msat")), PublicKey(rs.getByteVectorFromHex("recipient_node_id")), Seq(part), - None) + None, + part.startedAt) } sentByParentId + (parentId -> sent) }.values.toSeq.sortBy(_.settledAt) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteAuditDb.scala b/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteAuditDb.scala index dc2bd6e917..b69d2d96fb 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteAuditDb.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteAuditDb.scala @@ -347,7 +347,8 @@ class SqliteAuditDb(val sqlite: Connection) extends AuditDb with Logging { MilliSatoshi(rs.getLong("recipient_amount_msat")), PublicKey(rs.getByteVector("recipient_node_id")), Seq(part), - None) + None, + part.startedAt) } sentByParentId + (parentId -> sent) }.values.toSeq.sortBy(_.settledAt) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/json/JsonSerializers.scala b/eclair-core/src/main/scala/fr/acinq/eclair/json/JsonSerializers.scala index c0f7af0240..d81e9b4e39 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/json/JsonSerializers.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/json/JsonSerializers.scala @@ -383,8 +383,8 @@ object PaymentFailedSummarySerializer extends ConvertClassSerializer[PaymentFail )) // @formatter:on -private case class PaymentSentJson(id: UUID, paymentHash: ByteVector32, paymentPreimage: ByteVector32, recipientAmount: MilliSatoshi, recipientNodeId: PublicKey, parts: Seq[PaymentSent.PartialPayment]) -object PaymentSentSerializer extends ConvertClassSerializer[PaymentSent](p => PaymentSentJson(p.id, p.paymentHash, p.paymentPreimage, p.recipientAmount, p.recipientNodeId, p.parts)) +private case class PaymentSentJson(id: UUID, paymentHash: ByteVector32, paymentPreimage: ByteVector32, recipientAmount: MilliSatoshi, recipientNodeId: PublicKey, parts: Seq[PaymentSent.PartialPayment], fees: MilliSatoshi, startedAt: TimestampMilli, settledAt: TimestampMilli) +object PaymentSentSerializer extends ConvertClassSerializer[PaymentSent](p => PaymentSentJson(p.id, p.paymentHash, p.paymentPreimage, p.recipientAmount, p.recipientNodeId, p.parts, p.feesPaid, p.startedAt, p.settledAt)) object ThrowableSerializer extends MinimalSerializer({ case t: Throwable if t.getMessage != null => JString(t.getMessage) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentEvents.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentEvents.scala index 0b462f651f..e1a9477dda 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentEvents.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentEvents.scala @@ -58,12 +58,11 @@ sealed trait PaymentEvent { * @param parts child payments (actual outgoing HTLCs). * @param remainingAttribution_opt for relayed trampoline payments, the attribution data that needs to be sent upstream */ -case class PaymentSent(id: UUID, paymentPreimage: ByteVector32, recipientAmount: MilliSatoshi, recipientNodeId: PublicKey, parts: Seq[PaymentSent.PartialPayment], remainingAttribution_opt: Option[ByteVector]) extends PaymentEvent { +case class PaymentSent(id: UUID, paymentPreimage: ByteVector32, recipientAmount: MilliSatoshi, recipientNodeId: PublicKey, parts: Seq[PaymentSent.PartialPayment], remainingAttribution_opt: Option[ByteVector], startedAt: TimestampMilli) extends PaymentEvent { require(parts.nonEmpty, "must have at least one payment part") val paymentHash: ByteVector32 = Crypto.sha256(paymentPreimage) val amountWithFees: MilliSatoshi = parts.map(_.amountWithFees).sum val feesPaid: MilliSatoshi = amountWithFees - recipientAmount // overall fees for this payment - val startedAt: TimestampMilli = parts.map(_.startedAt).min val settledAt: TimestampMilli = parts.map(_.settledAt).max } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/PostRestartHtlcCleaner.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/PostRestartHtlcCleaner.scala index 607e81e68c..cdc8fb1252 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/PostRestartHtlcCleaner.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/PostRestartHtlcCleaner.scala @@ -178,7 +178,7 @@ class PostRestartHtlcCleaner(nodeParams: NodeParams, register: ActorRef, initial val feesPaid = 0.msat // fees are unknown since we lost the reference to the payment nodeParams.db.payments.getOutgoingPayment(id) match { case Some(p) => - nodeParams.db.payments.updateOutgoingPayment(PaymentSent(p.parentId, paymentPreimage, p.recipientAmount, p.recipientNodeId, PaymentSent.PartialPayment(id, fulfilledHtlc.amountMsat, feesPaid, fulfilledHtlc.channelId, None, startedAt = p.createdAt, settledAt = TimestampMilli.now()) :: Nil, None)) + nodeParams.db.payments.updateOutgoingPayment(PaymentSent(p.parentId, paymentPreimage, p.recipientAmount, p.recipientNodeId, PaymentSent.PartialPayment(id, fulfilledHtlc.amountMsat, feesPaid, fulfilledHtlc.channelId, None, startedAt = p.createdAt, settledAt = TimestampMilli.now()) :: Nil, None, p.createdAt)) // If all downstream HTLCs are now resolved, we can emit the payment event. val payments = nodeParams.db.payments.listOutgoingPayments(p.parentId) if (!payments.exists(p => p.status == OutgoingPaymentStatus.Pending)) { @@ -186,7 +186,7 @@ class PostRestartHtlcCleaner(nodeParams: NodeParams, register: ActorRef, initial case OutgoingPayment(id, _, _, _, _, amount, _, _, createdAt, _, _, OutgoingPaymentStatus.Succeeded(_, feesPaid, _, completedAt)) => PaymentSent.PartialPayment(id, amount, feesPaid, ByteVector32.Zeroes, None, createdAt, completedAt) } - val sent = PaymentSent(p.parentId, paymentPreimage, p.recipientAmount, p.recipientNodeId, succeeded, None) + val sent = PaymentSent(p.parentId, paymentPreimage, p.recipientAmount, p.recipientNodeId, succeeded, None, p.createdAt) log.info(s"payment id=${sent.id} paymentHash=${sent.paymentHash} successfully sent (amount=${sent.recipientAmount})") context.system.eventStream.publish(sent) } @@ -196,8 +196,9 @@ class PostRestartHtlcCleaner(nodeParams: NodeParams, register: ActorRef, initial // dummy values in the DB (to make sure we store the preimage) but we don't emit an event. val dummyFinalAmount = fulfilledHtlc.amountMsat val dummyNodeId = nodeParams.nodeId - nodeParams.db.payments.addOutgoingPayment(OutgoingPayment(id, id, None, fulfilledHtlc.paymentHash, PaymentType.Standard, fulfilledHtlc.amountMsat, dummyFinalAmount, dummyNodeId, TimestampMilli.now(), None, None, OutgoingPaymentStatus.Pending)) - nodeParams.db.payments.updateOutgoingPayment(PaymentSent(id, paymentPreimage, dummyFinalAmount, dummyNodeId, PaymentSent.PartialPayment(id, fulfilledHtlc.amountMsat, feesPaid, fulfilledHtlc.channelId, None, startedAt = TimestampMilli.now(), settledAt = TimestampMilli.now()) :: Nil, None)) + val now = TimestampMilli.now() + nodeParams.db.payments.addOutgoingPayment(OutgoingPayment(id, id, None, fulfilledHtlc.paymentHash, PaymentType.Standard, fulfilledHtlc.amountMsat, dummyFinalAmount, dummyNodeId, now, None, None, OutgoingPaymentStatus.Pending)) + nodeParams.db.payments.updateOutgoingPayment(PaymentSent(id, paymentPreimage, dummyFinalAmount, dummyNodeId, PaymentSent.PartialPayment(id, fulfilledHtlc.amountMsat, feesPaid, fulfilledHtlc.channelId, None, startedAt = now, settledAt = now) :: Nil, None, startedAt = now)) } // There can never be more than one pending downstream HTLC for a given local origin (a multi-part payment is // instead spread across multiple local origins) so we can now forget this origin. diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/MultiPartPaymentLifecycle.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/MultiPartPaymentLifecycle.scala index c90b1be5a7..a25cf1b462 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/MultiPartPaymentLifecycle.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/MultiPartPaymentLifecycle.scala @@ -176,7 +176,7 @@ class MultiPartPaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, val parts = d.parts ++ ps.parts val pending = d.pending - ps.parts.head.id if (pending.isEmpty) { - myStop(d.request, Right(cfg.createPaymentSent(d.request.recipient, d.preimage, parts, d.remainingAttribution_opt))) + myStop(d.request, Right(cfg.createPaymentSent(d.request.recipient, d.preimage, parts, d.remainingAttribution_opt, start))) } else { stay() using d.copy(parts = parts, pending = pending) } @@ -187,7 +187,7 @@ class MultiPartPaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, log.warning(s"payment succeeded but partial payment failed (id=${pf.id})") val pending = d.pending - pf.id if (pending.isEmpty) { - myStop(d.request, Right(cfg.createPaymentSent(d.request.recipient, d.preimage, d.parts, d.remainingAttribution_opt))) + myStop(d.request, Right(cfg.createPaymentSent(d.request.recipient, d.preimage, d.parts, d.remainingAttribution_opt, start))) } else { stay() using d.copy(pending = pending) } @@ -217,7 +217,7 @@ class MultiPartPaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, d.request.replyTo ! PreimageReceived(paymentHash, d.preimage, d.remainingAttribution_opt) } if (d.pending.isEmpty) { - myStop(d.request, Right(cfg.createPaymentSent(d.request.recipient, d.preimage, d.parts, d.remainingAttribution_opt))) + myStop(d.request, Right(cfg.createPaymentSent(d.request.recipient, d.preimage, d.parts, d.remainingAttribution_opt, start))) } else goto(PAYMENT_SUCCEEDED) using d } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/PaymentInitiator.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/PaymentInitiator.scala index ee5c565c9d..6fdb12d147 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/PaymentInitiator.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/PaymentInitiator.scala @@ -336,8 +336,8 @@ object PaymentInitiator { case _ => PaymentType.Standard } - def createPaymentSent(recipient: Recipient, preimage: ByteVector32, parts: Seq[PaymentSent.PartialPayment], remainingAttribution_opt: Option[ByteVector]): PaymentSent = { - PaymentSent(parentId, preimage, recipient.totalAmount, recipient.nodeId, parts, remainingAttribution_opt) + def createPaymentSent(recipient: Recipient, preimage: ByteVector32, parts: Seq[PaymentSent.PartialPayment], remainingAttribution_opt: Option[ByteVector], startedAt: TimestampMilli): PaymentSent = { + PaymentSent(parentId, preimage, recipient.totalAmount, recipient.nodeId, parts, remainingAttribution_opt, startedAt) } } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/PaymentLifecycle.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/PaymentLifecycle.scala index 32d4a06d16..f571a0165a 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/PaymentLifecycle.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/PaymentLifecycle.scala @@ -128,7 +128,7 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A } case _: HtlcResult.OnChainFulfill => None } - myStop(d.request, Right(cfg.createPaymentSent(d.recipient, fulfill.paymentPreimage, p :: Nil, remainingAttribution_opt))) + myStop(d.request, Right(cfg.createPaymentSent(d.recipient, fulfill.paymentPreimage, p :: Nil, remainingAttribution_opt, start))) case Event(RES_ADD_SETTLED(_, _, fail: HtlcResult.Fail), d: WaitingForComplete) => fail match { diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/TrampolinePaymentLifecycle.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/TrampolinePaymentLifecycle.scala index 0dcdbcb7a0..22cc576798 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/TrampolinePaymentLifecycle.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/TrampolinePaymentLifecycle.scala @@ -236,7 +236,7 @@ class TrampolinePaymentLifecycle private(nodeParams: NodeParams, waitForSettlement(remaining - 1, attemptNumber, part +: fulfilledParts) } else { context.log.info("trampoline payment succeeded") - cmd.replyTo ! PaymentSent(cmd.paymentId, fulfill.paymentPreimage, totalAmount, cmd.invoice.nodeId, part +: fulfilledParts, None) + cmd.replyTo ! PaymentSent(cmd.paymentId, fulfill.paymentPreimage, totalAmount, cmd.invoice.nodeId, part +: fulfilledParts, None, startedAt) Behaviors.stopped } case fail: HtlcResult.Fail => diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/EclairImplSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/EclairImplSpec.scala index 11de170020..918e44d051 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/EclairImplSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/EclairImplSpec.scala @@ -581,9 +581,10 @@ class EclairImplSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with I import f._ val eclair = new EclairImpl(kit) + val startedAt = TimestampMilli.now() // A first payment has been sent out and is currently pending. - val pendingPayment1 = OutgoingPayment(UUID.randomUUID(), UUID.randomUUID(), None, randomBytes32(), "test", 500 msat, 750 msat, randomKey().publicKey, TimestampMilli.now(), None, None, OutgoingPaymentStatus.Pending) + val pendingPayment1 = OutgoingPayment(UUID.randomUUID(), UUID.randomUUID(), None, randomBytes32(), "test", 500 msat, 750 msat, randomKey().publicKey, startedAt, None, None, OutgoingPaymentStatus.Pending) kit.nodeParams.db.payments.addOutgoingPayment(pendingPayment1) eclair.sentInfo(PaymentIdentifier.PaymentUUID(pendingPayment1.parentId)).pipeTo(sender.ref) sender.expectMsg(Seq(pendingPayment1)) @@ -611,9 +612,9 @@ class EclairImplSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with I // A third payment is fully settled in the DB and not being retried. val failedAt = TimestampMilli.now() - val failedPayment = OutgoingPayment(UUID.randomUUID(), UUID.randomUUID(), None, spontaneousPayment.paymentHash, "test", 700 msat, 900 msat, randomKey().publicKey, TimestampMilli.now(), None, None, OutgoingPaymentStatus.Failed(Nil, failedAt)) + val failedPayment = OutgoingPayment(UUID.randomUUID(), UUID.randomUUID(), None, spontaneousPayment.paymentHash, "test", 700 msat, 900 msat, randomKey().publicKey, startedAt, None, None, OutgoingPaymentStatus.Failed(Nil, failedAt)) kit.nodeParams.db.payments.addOutgoingPayment(failedPayment.copy(status = OutgoingPaymentStatus.Pending)) - kit.nodeParams.db.payments.updateOutgoingPayment(PaymentFailed(failedPayment.id, failedPayment.paymentHash, Nil, failedAt)) + kit.nodeParams.db.payments.updateOutgoingPayment(PaymentFailed(failedPayment.id, failedPayment.paymentHash, Nil, startedAt, failedAt)) eclair.sentInfo(PaymentIdentifier.PaymentUUID(failedPayment.parentId)).pipeTo(sender.ref) paymentInitiator.expectMsg(GetPayment(PaymentIdentifier.PaymentUUID(failedPayment.parentId))) paymentInitiator.reply(NoPendingPayment(PaymentIdentifier.PaymentUUID(failedPayment.parentId))) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/db/AuditDbSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/db/AuditDbSpec.scala index d67e2ca698..88bdd5364d 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/db/AuditDbSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/db/AuditDbSpec.scala @@ -63,7 +63,7 @@ class AuditDbSpec extends AnyFunSuite { val db = dbs.audit val now = TimestampMilli.now() - val e1 = PaymentSent(ZERO_UUID, randomBytes32(), 40000 msat, randomKey().publicKey, PaymentSent.PartialPayment(ZERO_UUID, 42000 msat, 1000 msat, randomBytes32(), None, now, now) :: Nil, None) + val e1 = PaymentSent(ZERO_UUID, randomBytes32(), 40000 msat, randomKey().publicKey, PaymentSent.PartialPayment(ZERO_UUID, 42000 msat, 1000 msat, randomBytes32(), None, now, now) :: Nil, None, now) val pp2a = PaymentReceived.PartialPayment(42000 msat, randomBytes32()) val pp2b = PaymentReceived.PartialPayment(42100 msat, randomBytes32()) val e2 = PaymentReceived(randomBytes32(), pp2a :: pp2b :: Nil) @@ -73,9 +73,9 @@ class AuditDbSpec extends AnyFunSuite { val e4c = TransactionConfirmed(randomBytes32(), randomKey().publicKey, Transaction(2, Nil, TxOut(500 sat, hex"1234") :: Nil, 0)) val pp5a = PaymentSent.PartialPayment(UUID.randomUUID(), 42000 msat, 1000 msat, randomBytes32(), None, startedAt = 0 unixms, settledAt = 0 unixms) val pp5b = PaymentSent.PartialPayment(UUID.randomUUID(), 42100 msat, 900 msat, randomBytes32(), None, startedAt = 1 unixms, settledAt = 1 unixms) - val e5 = PaymentSent(UUID.randomUUID(), randomBytes32(), 84100 msat, randomKey().publicKey, pp5a :: pp5b :: Nil, None) + val e5 = PaymentSent(UUID.randomUUID(), randomBytes32(), 84100 msat, randomKey().publicKey, pp5a :: pp5b :: Nil, None, startedAt = 0 unixms) val pp6 = PaymentSent.PartialPayment(UUID.randomUUID(), 42000 msat, 1000 msat, randomBytes32(), None, startedAt = now + 10.minutes, settledAt = now + 10.minutes) - val e6 = PaymentSent(UUID.randomUUID(), randomBytes32(), 42000 msat, randomKey().publicKey, pp6 :: Nil, None) + val e6 = PaymentSent(UUID.randomUUID(), randomBytes32(), 42000 msat, randomKey().publicKey, pp6 :: Nil, None, startedAt = now + 10.minutes) val e7 = ChannelEvent(randomBytes32(), randomKey().publicKey, randomTxId(), 456123000 sat, isChannelOpener = true, isPrivate = false, ChannelEvent.EventType.Closed(MutualClose(null))) val e10 = TrampolinePaymentRelayed(randomBytes32(), Seq( diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/db/PaymentsDbSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/db/PaymentsDbSpec.scala index c146f57bda..238e6a2fae 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/db/PaymentsDbSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/db/PaymentsDbSpec.scala @@ -200,7 +200,7 @@ class PaymentsDbSpec extends AnyFunSuite { val ps6 = OutgoingPayment(UUID.randomUUID(), UUID.randomUUID(), Some("3"), randomBytes32(), PaymentType.Standard, 789 msat, 789 msat, bob, 1250 unixms, None, None, OutgoingPaymentStatus.Failed(Nil, 1300 unixms)) db.addOutgoingPayment(ps4) db.addOutgoingPayment(ps5.copy(status = OutgoingPaymentStatus.Pending)) - db.updateOutgoingPayment(PaymentSent(ps5.parentId, preimage1, ps5.amount, ps5.recipientNodeId, Seq(PaymentSent.PartialPayment(ps5.id, ps5.amount, 42 msat, randomBytes32(), None, 1000 unixms, 1180 unixms)), None)) + db.updateOutgoingPayment(PaymentSent(ps5.parentId, preimage1, ps5.amount, ps5.recipientNodeId, Seq(PaymentSent.PartialPayment(ps5.id, ps5.amount, 42 msat, randomBytes32(), None, 1000 unixms, 1180 unixms)), None, 900 unixms)) db.addOutgoingPayment(ps6.copy(status = OutgoingPaymentStatus.Pending)) db.updateOutgoingPayment(PaymentFailed(ps6.id, ps6.paymentHash, Nil, 1100 unixms, 1300 unixms)) @@ -772,12 +772,12 @@ class PaymentsDbSpec extends AnyFunSuite { assert(db.getOutgoingPayment(s4.id).contains(ss4)) // can't update again once it's in a final state - assertThrows[IllegalArgumentException](db.updateOutgoingPayment(PaymentSent(parentId, preimage1, s3.recipientAmount, s3.recipientNodeId, Seq(PaymentSent.PartialPayment(s3.id, s3.amount, 42 msat, randomBytes32(), None, startedAt = 100 unixms, settledAt = 500 unixms)), None))) + assertThrows[IllegalArgumentException](db.updateOutgoingPayment(PaymentSent(parentId, preimage1, s3.recipientAmount, s3.recipientNodeId, Seq(PaymentSent.PartialPayment(s3.id, s3.amount, 42 msat, randomBytes32(), None, startedAt = 100 unixms, settledAt = 500 unixms)), None, 100 unixms))) val paymentSent = PaymentSent(parentId, preimage1, 600 msat, carol, Seq( PaymentSent.PartialPayment(s1.id, s1.amount, 15 msat, randomBytes32(), None, startedAt = 200 unixms, settledAt = 400 unixms), PaymentSent.PartialPayment(s2.id, s2.amount, 20 msat, randomBytes32(), Some(Seq(hop_ab, hop_bc)), startedAt = 210 unixms, settledAt = 410 unixms) - ), None) + ), None, startedAt = 100 unixms) val ss1 = s1.copy(status = OutgoingPaymentStatus.Succeeded(preimage1, 15 msat, Nil, 400 unixms)) val ss2 = s2.copy(status = OutgoingPaymentStatus.Succeeded(preimage1, 20 msat, Seq(HopSummary(alice, bob, Some(ShortChannelId(42))), HopSummary(bob, carol, None)), 410 unixms)) db.updateOutgoingPayment(paymentSent) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala index a53019b46c..cad805fc12 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala @@ -48,7 +48,7 @@ import fr.acinq.eclair.router.Router.{ChannelHop, GossipDecision, PublicChannel} import fr.acinq.eclair.router.{Announcements, AnnouncementsBatchValidationSpec, Router} import fr.acinq.eclair.wire.protocol.OfferTypes.{Offer, OfferPaths} import fr.acinq.eclair.wire.protocol.{ChannelAnnouncement, ChannelUpdate, IncorrectOrUnknownPaymentDetails} -import fr.acinq.eclair.{CltvExpiryDelta, EclairImpl, EncodedNodeId, Features, Kit, MilliSatoshiLong, ShortChannelId, TimestampMilli, randomBytes32, randomKey} +import fr.acinq.eclair.{CltvExpiryDelta, EclairImpl, EncodedNodeId, Features, Kit, MilliSatoshiLong, ShortChannelId, TimestampMilli,TimestampMilliLong, randomBytes32, randomKey} import org.json4s.JsonAST.{JString, JValue} import scodec.bits.{ByteVector, HexStringSyntax} @@ -369,6 +369,7 @@ class PaymentIntegrationSpec extends IntegrationSpec { assert(paymentSent.feesPaid > 0.msat, paymentSent) assert(paymentSent.parts.forall(p => p.id != paymentSent.id), paymentSent) assert(paymentSent.parts.forall(p => p.route.isDefined), paymentSent) + assert(paymentSent.startedAt >= start) val paymentParts = nodes("B").nodeParams.db.payments.listOutgoingPayments(paymentId).filter(_.status.isInstanceOf[OutgoingPaymentStatus.Succeeded]) assert(paymentParts.length == paymentSent.parts.length, paymentParts) @@ -385,8 +386,9 @@ class PaymentIntegrationSpec extends IntegrationSpec { parts = paymentSent.parts.map(p => p.copy(route = None, startedAt = p.settledAt)).sortBy(_.settledAt), // We don't store attribution data in the DB. remainingAttribution_opt = None, + startedAt = 0 unixms, ) - assert(sent.head.copy(parts = sent.head.parts.sortBy(_.settledAt)) == paymentSent1) + assert(sent.head.copy(parts = sent.head.parts.sortBy(_.settledAt), startedAt = 0 unixms) == paymentSent1) awaitCond(nodes("D").nodeParams.db.payments.getIncomingPayment(invoice.paymentHash).exists(_.status.isInstanceOf[IncomingPaymentStatus.Received])) val Some(IncomingStandardPayment(_, _, _, _, IncomingPaymentStatus.Received(receivedAmount, _))) = nodes("D").nodeParams.db.payments.getIncomingPayment(invoice.paymentHash) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartPaymentLifecycleSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartPaymentLifecycleSpec.scala index 72aa268672..d7b636d2c2 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartPaymentLifecycleSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartPaymentLifecycleSpec.scala @@ -580,7 +580,7 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS awaitCond(payFsm.stateName == PAYMENT_ABORTED) sender.watch(payFsm) - childPayFsm.send(payFsm, PaymentSent(cfg.id, paymentPreimage, finalAmount, e, Seq(PaymentSent.PartialPayment(successId, successRoute.amount, successRoute.channelFee(false), randomBytes32(), Some(successRoute.fullRoute), 100 unixms, 250 unixms)), None)) + childPayFsm.send(payFsm, PaymentSent(cfg.id, paymentPreimage, finalAmount, e, Seq(PaymentSent.PartialPayment(successId, successRoute.amount, successRoute.channelFee(false), randomBytes32(), Some(successRoute.fullRoute), 100 unixms, 250 unixms)), None, 75 unixms)) sender.expectMsg(PreimageReceived(paymentHash, paymentPreimage, None)) val result = sender.expectMsgType[PaymentSent] assert(result.id == cfg.id) @@ -608,7 +608,7 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS childPayFsm.expectMsgType[SendPaymentToRoute] val (childId, route) :: (failedId, failedRoute) :: Nil = payFsm.stateData.asInstanceOf[PaymentProgress].pending.toSeq - childPayFsm.send(payFsm, PaymentSent(cfg.id, paymentPreimage, finalAmount, e, Seq(PaymentSent.PartialPayment(childId, route.amount, route.channelFee(false), randomBytes32(), Some(route.fullRoute), 100 unixms, 250 unixms)), None)) + childPayFsm.send(payFsm, PaymentSent(cfg.id, paymentPreimage, finalAmount, e, Seq(PaymentSent.PartialPayment(childId, route.amount, route.channelFee(false), randomBytes32(), Some(route.fullRoute), 100 unixms, 250 unixms)), None, 75 unixms)) sender.expectMsg(PreimageReceived(paymentHash, paymentPreimage, None)) awaitCond(payFsm.stateName == PAYMENT_SUCCEEDED) @@ -634,7 +634,7 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS val partialPayments = pending.map { case (childId, route) => PaymentSent.PartialPayment(childId, route.amount, route.channelFee(false) + route.blindedFee, randomBytes32(), Some(route.fullRoute), 100 unixms, 250 unixms) } - partialPayments.foreach(pp => childPayFsm.send(payFsm, PaymentSent(cfg.id, paymentPreimage, finalAmount, e, Seq(pp), None))) + partialPayments.foreach(pp => childPayFsm.send(payFsm, PaymentSent(cfg.id, paymentPreimage, finalAmount, e, Seq(pp), None, 100 unixms))) sender.expectMsg(PreimageReceived(paymentHash, paymentPreimage, None)) val result = sender.expectMsgType[PaymentSent] assert(result.id == cfg.id) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentInitiatorSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentInitiatorSpec.scala index 61808558a9..4e43828e77 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentInitiatorSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentInitiatorSpec.scala @@ -231,7 +231,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike sender.send(initiator, GetPayment(PaymentIdentifier.PaymentHash(invoice.paymentHash))) sender.expectMsg(PaymentIsPending(id, invoice.paymentHash, PendingPaymentToNode(sender.ref, req))) - val ps = PaymentSent(id, paymentPreimage, finalAmount, priv_c.publicKey, Seq(PartialPayment(UUID.randomUUID(), finalAmount, 0 msat, randomBytes32(), None, 100 unixms, 200 unixms)), None) + val ps = PaymentSent(id, paymentPreimage, finalAmount, priv_c.publicKey, Seq(PartialPayment(UUID.randomUUID(), finalAmount, 0 msat, randomBytes32(), None, 100 unixms, 200 unixms)), None, 80 unixms) payFsm.send(initiator, ps) sender.expectMsg(ps) eventListener.expectNoMessage(100 millis) @@ -350,7 +350,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike sender.send(initiator, GetPayment(PaymentIdentifier.PaymentHash(invoice.paymentHash))) sender.expectMsg(PaymentIsPending(id, invoice.paymentHash, PendingPaymentToNode(sender.ref, req))) - val ps = PaymentSent(id, paymentPreimage, finalAmount, invoice.nodeId, Seq(PartialPayment(UUID.randomUUID(), finalAmount, 0 msat, randomBytes32(), None, 100 unixms, 200 unixms)), None) + val ps = PaymentSent(id, paymentPreimage, finalAmount, invoice.nodeId, Seq(PartialPayment(UUID.randomUUID(), finalAmount, 0 msat, randomBytes32(), None, 100 unixms, 200 unixms)), None, 100 unixms) payFsm.send(initiator, ps) sender.expectMsg(ps) eventListener.expectNoMessage(100 millis) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentLifecycleSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentLifecycleSpec.scala index 5da7c28a1a..c3003ead4d 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentLifecycleSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentLifecycleSpec.scala @@ -846,7 +846,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec { sender.send(paymentFSM, addCompleted(HtlcResult.OnChainFulfill(defaultPaymentPreimage))) val paymentOK = sender.expectMsgType[PaymentSent] - val PaymentSent(_, paymentOK.paymentPreimage, finalAmount, _, PartialPayment(_, partAmount, fee, ByteVector32.Zeroes, _, _, _) :: Nil, _) = eventListener.expectMsgType[PaymentSent] + val PaymentSent(_, paymentOK.paymentPreimage, finalAmount, _, PartialPayment(_, partAmount, fee, ByteVector32.Zeroes, _, _, _) :: Nil, _, _) = eventListener.expectMsgType[PaymentSent] assert(partAmount == request.amount) assert(finalAmount == defaultAmountMsat) 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 7a01f66349..9d37bd3310 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 @@ -1213,7 +1213,7 @@ object NodeRelayerSpec { (paymentPackets.map(_.outerPayload.expiry).min - nodeParams.relayParams.asyncPaymentsParams.cancelSafetyBeforeTimeout).blockHeight def createSuccessEvent(): PaymentSent = - PaymentSent(relayId, paymentPreimage, outgoingAmount, outgoingNodeId, Seq(PaymentSent.PartialPayment(UUID.randomUUID(), outgoingAmount, 10 msat, randomBytes32(), None, 0 unixms, 50 unixms)), None) + PaymentSent(relayId, paymentPreimage, outgoingAmount, outgoingNodeId, Seq(PaymentSent.PartialPayment(UUID.randomUUID(), outgoingAmount, 10 msat, randomBytes32(), None, 0 unixms, 50 unixms)), None, 0 unixms) def createTrampolinePacket(amount: MilliSatoshi, expiry: CltvExpiry): OnionRoutingPacket = { val payload = NodePayload(outgoingNodeId, FinalPayload.Standard.createPayload(amount, amount, expiry, paymentSecret, upgradeAccountability = false)) diff --git a/eclair-node/src/test/scala/fr/acinq/eclair/api/ApiServiceSpec.scala b/eclair-node/src/test/scala/fr/acinq/eclair/api/ApiServiceSpec.scala index 33f1466b54..50e8cc53b6 100644 --- a/eclair-node/src/test/scala/fr/acinq/eclair/api/ApiServiceSpec.scala +++ b/eclair-node/src/test/scala/fr/acinq/eclair/api/ApiServiceSpec.scala @@ -623,7 +623,7 @@ class ApiServiceSpec extends AnyFunSuite with ScalatestRouteTest with IdiomaticM val mockService = new MockService(eclair) val uuid = UUID.fromString("487da196-a4dc-4b1e-92b4-3e5e905e9f3f") - val paymentSent = PaymentSent(uuid, ByteVector32.One, 25 msat, aliceNodeId, Seq(PaymentSent.PartialPayment(uuid, 21 msat, 1 msat, ByteVector32.Zeroes, None, TimestampMilli(1553784337650L), TimestampMilli(1553784337711L))), None) + val paymentSent = PaymentSent(uuid, ByteVector32.One, 25 msat, aliceNodeId, Seq(PaymentSent.PartialPayment(uuid, 28 msat, 1 msat, ByteVector32.Zeroes, None, TimestampMilli(1553784337650L), TimestampMilli(1553784337711L))), None, TimestampMilli(1553784337120L)) eclair.sendBlocking(any, any, any, any, any, any, any)(any[Timeout]).returns(Future.successful(paymentSent)) Post("/payinvoice", FormData("invoice" -> invoice, "blocking" -> "true").toEntity) ~> addCredentials(BasicHttpCredentials("", mockApi().password)) ~> @@ -632,7 +632,7 @@ class ApiServiceSpec extends AnyFunSuite with ScalatestRouteTest with IdiomaticM assert(handled) assert(status == OK) val response = entityAs[String] - val expected = """{"type":"payment-sent","id":"487da196-a4dc-4b1e-92b4-3e5e905e9f3f","paymentHash":"01d0fabd251fcbbe2b93b4b927b26ad2a1a99077152e45ded1e678afa45dbec5","paymentPreimage":"0100000000000000000000000000000000000000000000000000000000000000","recipientAmount":25,"recipientNodeId":"03af0ed6052cf28d670665549bc86f4b721c9fdb309d40c58f5811f63966e005d0","parts":[{"id":"487da196-a4dc-4b1e-92b4-3e5e905e9f3f","amount":21,"feesPaid":1,"toChannelId":"0000000000000000000000000000000000000000000000000000000000000000","startedAt":{"iso":"2019-03-28T14:45:37.650Z","unix":1553784337},"settledAt":{"iso":"2019-03-28T14:45:37.711Z","unix":1553784337}}]}""" + val expected = """{"type":"payment-sent","id":"487da196-a4dc-4b1e-92b4-3e5e905e9f3f","paymentHash":"01d0fabd251fcbbe2b93b4b927b26ad2a1a99077152e45ded1e678afa45dbec5","paymentPreimage":"0100000000000000000000000000000000000000000000000000000000000000","recipientAmount":25,"recipientNodeId":"03af0ed6052cf28d670665549bc86f4b721c9fdb309d40c58f5811f63966e005d0","parts":[{"id":"487da196-a4dc-4b1e-92b4-3e5e905e9f3f","amount":28,"feesPaid":1,"toChannelId":"0000000000000000000000000000000000000000000000000000000000000000","startedAt":{"iso":"2019-03-28T14:45:37.650Z","unix":1553784337},"settledAt":{"iso":"2019-03-28T14:45:37.711Z","unix":1553784337}}],"fees":4,"startedAt":{"iso":"2019-03-28T14:45:37.120Z","unix":1553784337},"settledAt":{"iso":"2019-03-28T14:45:37.711Z","unix":1553784337}}""" assert(response == expected) } @@ -1127,8 +1127,8 @@ class ApiServiceSpec extends AnyFunSuite with ScalatestRouteTest with IdiomaticM system.eventStream.publish(pf) wsClient.expectMessage(expectedSerializedPf) - val ps = PaymentSent(fixedUUID, ByteVector32.One, 25 msat, aliceNodeId, Seq(PaymentSent.PartialPayment(fixedUUID, 21 msat, 1 msat, ByteVector32.Zeroes, None, startedAt = TimestampMilli(1553784337539L), settledAt = TimestampMilli(1553784337711L))), None) - val expectedSerializedPs = """{"type":"payment-sent","id":"487da196-a4dc-4b1e-92b4-3e5e905e9f3f","paymentHash":"01d0fabd251fcbbe2b93b4b927b26ad2a1a99077152e45ded1e678afa45dbec5","paymentPreimage":"0100000000000000000000000000000000000000000000000000000000000000","recipientAmount":25,"recipientNodeId":"03af0ed6052cf28d670665549bc86f4b721c9fdb309d40c58f5811f63966e005d0","parts":[{"id":"487da196-a4dc-4b1e-92b4-3e5e905e9f3f","amount":21,"feesPaid":1,"toChannelId":"0000000000000000000000000000000000000000000000000000000000000000","startedAt":{"iso":"2019-03-28T14:45:37.539Z","unix":1553784337},"settledAt":{"iso":"2019-03-28T14:45:37.711Z","unix":1553784337}}]}""" + val ps = PaymentSent(fixedUUID, ByteVector32.One, 25 msat, aliceNodeId, Seq(PaymentSent.PartialPayment(fixedUUID, 28 msat, 1 msat, ByteVector32.Zeroes, None, startedAt = TimestampMilli(1553784337539L), settledAt = TimestampMilli(1553784337711L))), None, startedAt = TimestampMilli(1553784337073L)) + val expectedSerializedPs = """{"type":"payment-sent","id":"487da196-a4dc-4b1e-92b4-3e5e905e9f3f","paymentHash":"01d0fabd251fcbbe2b93b4b927b26ad2a1a99077152e45ded1e678afa45dbec5","paymentPreimage":"0100000000000000000000000000000000000000000000000000000000000000","recipientAmount":25,"recipientNodeId":"03af0ed6052cf28d670665549bc86f4b721c9fdb309d40c58f5811f63966e005d0","parts":[{"id":"487da196-a4dc-4b1e-92b4-3e5e905e9f3f","amount":28,"feesPaid":1,"toChannelId":"0000000000000000000000000000000000000000000000000000000000000000","startedAt":{"iso":"2019-03-28T14:45:37.539Z","unix":1553784337},"settledAt":{"iso":"2019-03-28T14:45:37.711Z","unix":1553784337}}],"fees":4,"startedAt":{"iso":"2019-03-28T14:45:37.073Z","unix":1553784337},"settledAt":{"iso":"2019-03-28T14:45:37.711Z","unix":1553784337}}""" assert(serialization.write(ps) == expectedSerializedPs) system.eventStream.publish(ps) wsClient.expectMessage(expectedSerializedPs)