1
0
Fork 0
mirror of https://github.com/ACINQ/eclair.git synced 2025-03-04 01:53:49 +01:00

Trampoline/MPP DB changes (#1287)

With MPP and Trampoline (and particularly the combination of the two),
we need to keep track of multiple amounts, recipients and fees.
There's a trampoline fee and a fee to reach the first trampoline node.
The trampoline nodes must appear in the route, but not as payment recipients.

Adding new fields to payment events and DB structs lets us distinguish those.

We also relax the spec requirement about feature graph dependencies.
The requirement to include `var_onion_optin` in invoice feature bits
was added after the first Phoenix release.
Phoenix users will thus have non spec-compliant invoices in their
payment history.
We accept invoices that don't set this field; this is a harmless
spec violation (as long as we set it in new invoices).
This commit is contained in:
Bastien Teinturier 2020-01-29 14:21:41 +01:00 committed by GitHub
parent 16456bbce0
commit 453a7c66b1
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
35 changed files with 1038 additions and 537 deletions

View file

@ -90,7 +90,9 @@ object Features {
// Features may depend on other features, as specified in Bolt 9.
private val featuresDependency = Map(
ChannelRangeQueriesExtended -> (ChannelRangeQueries :: Nil),
PaymentSecret -> (VariableLengthOnion :: Nil),
// This dependency requirement was added to the spec after the Phoenix release, which means Phoenix users have "invalid"
// invoices in their payment history. We choose to treat such invoices as valid; this is a harmless spec violation.
// PaymentSecret -> (VariableLengthOnion :: Nil),
BasicMultiPartPayment -> (PaymentSecret :: Nil),
TrampolinePayment -> (PaymentSecret :: Nil)
)

View file

@ -22,7 +22,7 @@ import java.util.UUID
import fr.acinq.bitcoin.ByteVector32
import fr.acinq.bitcoin.Crypto.PublicKey
import fr.acinq.eclair.payment._
import fr.acinq.eclair.router.ChannelHop
import fr.acinq.eclair.router.{ChannelHop, Hop, NodeHop}
import fr.acinq.eclair.{MilliSatoshi, ShortChannelId}
import scala.compat.Platform
@ -31,7 +31,7 @@ trait PaymentsDb extends IncomingPaymentsDb with OutgoingPaymentsDb with Payment
trait IncomingPaymentsDb {
/** Add a new expected incoming payment (not yet received). */
def addIncomingPayment(pr: PaymentRequest, preimage: ByteVector32): Unit
def addIncomingPayment(pr: PaymentRequest, preimage: ByteVector32, paymentType: String = PaymentType.Standard): Unit
/**
* Mark an incoming payment as received (paid). The received amount may exceed the payment request amount.
@ -80,6 +80,12 @@ trait OutgoingPaymentsDb {
}
case object PaymentType {
val Standard = "Standard"
val SwapIn = "SwapIn"
val SwapOut = "SwapOut"
}
/**
* An incoming payment received by this node.
* At first it is in a pending state once the payment request has been generated, then will become either a success (if
@ -87,11 +93,13 @@ trait OutgoingPaymentsDb {
*
* @param paymentRequest Bolt 11 payment request.
* @param paymentPreimage pre-image associated with the payment request's payment_hash.
* @param paymentType distinguish different payment types (standard, swaps, etc).
* @param createdAt absolute time in milli-seconds since UNIX epoch when the payment request was generated.
* @param status current status of the payment.
*/
case class IncomingPayment(paymentRequest: PaymentRequest,
paymentPreimage: ByteVector32,
paymentType: String,
createdAt: Long,
status: IncomingPaymentStatus)
@ -123,8 +131,10 @@ object IncomingPaymentStatus {
* @param parentId internal identifier of a parent payment, or [[id]] if single-part payment.
* @param externalId external payment identifier: lets lightning applications reconcile payments with their own db.
* @param paymentHash payment_hash.
* @param amount amount of the payment, in milli-satoshis.
* @param targetNodeId node ID of the payment recipient.
* @param paymentType distinguish different payment types (standard, swaps, etc).
* @param amount amount that will be received by the target node, will be different from recipientAmount for trampoline payments.
* @param recipientAmount amount that will be received by the final recipient.
* @param recipientNodeId id of the final recipient.
* @param createdAt absolute time in milli-seconds since UNIX epoch when the payment was created.
* @param paymentRequest Bolt 11 payment request (if paying from an invoice).
* @param status current status of the payment.
@ -133,8 +143,10 @@ case class OutgoingPayment(id: UUID,
parentId: UUID,
externalId: Option[String],
paymentHash: ByteVector32,
paymentType: String,
amount: MilliSatoshi,
targetNodeId: PublicKey,
recipientAmount: MilliSatoshi,
recipientNodeId: PublicKey,
createdAt: Long,
paymentRequest: Option[PaymentRequest],
status: OutgoingPaymentStatus)
@ -151,8 +163,9 @@ object OutgoingPaymentStatus {
* We now have a valid proof-of-payment.
*
* @param paymentPreimage the preimage of the payment_hash.
* @param feesPaid total amount of fees paid to intermediate routing nodes.
* @param route payment route.
* @param feesPaid fees paid to route to the target node (which not necessarily the final recipient, e.g. when
* trampoline is used).
* @param route payment route used.
* @param completedAt absolute time in milli-seconds since UNIX epoch when the payment was completed.
*/
case class Succeeded(paymentPreimage: ByteVector32, feesPaid: MilliSatoshi, route: Seq[HopSummary], completedAt: Long) extends OutgoingPaymentStatus
@ -176,7 +189,13 @@ case class HopSummary(nodeId: PublicKey, nextNodeId: PublicKey, shortChannelId:
}
object HopSummary {
def apply(h: ChannelHop): HopSummary = HopSummary(h.nodeId, h.nextNodeId, Some(h.lastUpdate.shortChannelId))
def apply(h: Hop): HopSummary = {
val shortChannelId = h match {
case ChannelHop(_, _, channelUpdate) => Some(channelUpdate.shortChannelId)
case _: NodeHop => None
}
HopSummary(h.nodeId, h.nextNodeId, shortChannelId)
}
}
/** A minimal representation of a payment failure (suitable to store in a database). */
@ -210,6 +229,7 @@ trait PaymentsOverviewDb {
*/
sealed trait PlainPayment {
val paymentHash: ByteVector32
val paymentType: String
val paymentRequest: Option[String]
val finalAmount: Option[MilliSatoshi]
val createdAt: Long
@ -217,6 +237,7 @@ sealed trait PlainPayment {
}
case class PlainIncomingPayment(paymentHash: ByteVector32,
paymentType: String,
finalAmount: Option[MilliSatoshi],
paymentRequest: Option[String],
status: IncomingPaymentStatus,
@ -227,6 +248,7 @@ case class PlainIncomingPayment(paymentHash: ByteVector32,
case class PlainOutgoingPayment(parentId: Option[UUID],
externalId: Option[String],
paymentHash: ByteVector32,
paymentType: String,
finalAmount: Option[MilliSatoshi],
paymentRequest: Option[String],
status: OutgoingPaymentStatus,

View file

@ -19,13 +19,13 @@ package fr.acinq.eclair.db.sqlite
import java.sql.{Connection, Statement}
import java.util.UUID
import fr.acinq.bitcoin.Crypto.PublicKey
import fr.acinq.bitcoin.Satoshi
import fr.acinq.eclair.MilliSatoshi
import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey}
import fr.acinq.bitcoin.{ByteVector32, Satoshi}
import fr.acinq.eclair.channel.{AvailableBalanceChanged, Channel, ChannelErrorOccurred, NetworkFeePaid}
import fr.acinq.eclair.db._
import fr.acinq.eclair.payment._
import fr.acinq.eclair.wire.ChannelCodecs
import fr.acinq.eclair.{LongToBtcAmount, MilliSatoshi}
import grizzled.slf4j.Logging
import scala.collection.immutable.Queue
@ -37,34 +37,63 @@ class SqliteAuditDb(sqlite: Connection) extends AuditDb with Logging {
import ExtendedResultSet._
val DB_NAME = "audit"
val CURRENT_VERSION = 3
val CURRENT_VERSION = 4
case class RelayedPart(channelId: ByteVector32, amount: MilliSatoshi, direction: String, relayType: String, timestamp: Long)
using(sqlite.createStatement(), inTransaction = true) { statement =>
def migration12(statement: Statement) = {
def migration12(statement: Statement): Int = {
statement.executeUpdate(s"ALTER TABLE sent ADD id BLOB DEFAULT '${ChannelCodecs.UNKNOWN_UUID.toString}' NOT NULL")
}
def migration23(statement: Statement) = {
def migration23(statement: Statement): Int = {
statement.executeUpdate("CREATE TABLE IF NOT EXISTS channel_errors (channel_id BLOB NOT NULL, node_id BLOB NOT NULL, error_name TEXT NOT NULL, error_message TEXT NOT NULL, is_fatal INTEGER NOT NULL, timestamp INTEGER NOT NULL)")
statement.executeUpdate("CREATE INDEX IF NOT EXISTS channel_errors_timestamp_idx ON channel_errors(timestamp)")
}
def migration34(statement: Statement): Int = {
statement.executeUpdate("DROP index sent_timestamp_idx")
statement.executeUpdate("ALTER TABLE sent RENAME TO _sent_old")
statement.executeUpdate("CREATE TABLE sent (amount_msat INTEGER NOT NULL, fees_msat INTEGER NOT NULL, recipient_amount_msat INTEGER NOT NULL, payment_id TEXT NOT NULL, parent_payment_id TEXT NOT NULL, payment_hash BLOB NOT NULL, payment_preimage BLOB NOT NULL, recipient_node_id BLOB NOT NULL, to_channel_id BLOB NOT NULL, timestamp INTEGER NOT NULL)")
// Old rows will be missing a recipient node id, so we use an easy-to-spot default value.
val defaultRecipientNodeId = PrivateKey(ByteVector32.One).publicKey
statement.executeUpdate(s"INSERT INTO sent (amount_msat, fees_msat, recipient_amount_msat, payment_id, parent_payment_id, payment_hash, payment_preimage, recipient_node_id, to_channel_id, timestamp) SELECT amount_msat, fees_msat, amount_msat, id, id, payment_hash, payment_preimage, X'${defaultRecipientNodeId.toString}', to_channel_id, timestamp FROM _sent_old")
statement.executeUpdate("DROP table _sent_old")
statement.executeUpdate("DROP INDEX relayed_timestamp_idx")
statement.executeUpdate("ALTER TABLE relayed RENAME TO _relayed_old")
statement.executeUpdate("CREATE TABLE relayed (payment_hash BLOB NOT NULL, amount_msat INTEGER NOT NULL, channel_id BLOB NOT NULL, direction TEXT NOT NULL, relay_type TEXT NOT NULL, timestamp INTEGER NOT NULL)")
statement.executeUpdate("INSERT INTO relayed (payment_hash, amount_msat, channel_id, direction, relay_type, timestamp) SELECT payment_hash, amount_in_msat, from_channel_id, 'IN', 'channel', timestamp FROM _relayed_old")
statement.executeUpdate("INSERT INTO relayed (payment_hash, amount_msat, channel_id, direction, relay_type, timestamp) SELECT payment_hash, amount_out_msat, to_channel_id, 'OUT', 'channel', timestamp FROM _relayed_old")
statement.executeUpdate("DROP table _relayed_old")
statement.executeUpdate("CREATE INDEX IF NOT EXISTS sent_timestamp_idx ON sent(timestamp)")
statement.executeUpdate("CREATE INDEX IF NOT EXISTS relayed_timestamp_idx ON relayed(timestamp)")
statement.executeUpdate("CREATE INDEX IF NOT EXISTS relayed_payment_hash_idx ON relayed(payment_hash)")
}
getVersion(statement, DB_NAME, CURRENT_VERSION) match {
case 1 => // previous version let's migrate
logger.warn(s"migrating db $DB_NAME, found version=1 current=$CURRENT_VERSION")
migration12(statement)
migration23(statement)
migration34(statement)
setVersion(statement, DB_NAME, CURRENT_VERSION)
case 2 =>
logger.warn(s"migrating db $DB_NAME, found version=2 current=$CURRENT_VERSION")
migration23(statement)
migration34(statement)
setVersion(statement, DB_NAME, CURRENT_VERSION)
case 3 =>
logger.warn(s"migrating db $DB_NAME, found version=3 current=$CURRENT_VERSION")
migration34(statement)
setVersion(statement, DB_NAME, CURRENT_VERSION)
case CURRENT_VERSION =>
statement.executeUpdate("CREATE TABLE IF NOT EXISTS balance_updated (channel_id BLOB NOT NULL, node_id BLOB NOT NULL, amount_msat INTEGER NOT NULL, capacity_sat INTEGER NOT NULL, reserve_sat INTEGER NOT NULL, timestamp INTEGER NOT NULL)")
statement.executeUpdate("CREATE TABLE IF NOT EXISTS sent (amount_msat INTEGER NOT NULL, fees_msat INTEGER NOT NULL, payment_hash BLOB NOT NULL, payment_preimage BLOB NOT NULL, to_channel_id BLOB NOT NULL, timestamp INTEGER NOT NULL, id BLOB NOT NULL)")
statement.executeUpdate("CREATE TABLE IF NOT EXISTS sent (amount_msat INTEGER NOT NULL, fees_msat INTEGER NOT NULL, recipient_amount_msat INTEGER NOT NULL, payment_id TEXT NOT NULL, parent_payment_id TEXT NOT NULL, payment_hash BLOB NOT NULL, payment_preimage BLOB NOT NULL, recipient_node_id BLOB NOT NULL, to_channel_id BLOB NOT NULL, timestamp INTEGER NOT NULL)")
statement.executeUpdate("CREATE TABLE IF NOT EXISTS received (amount_msat INTEGER NOT NULL, payment_hash BLOB NOT NULL, from_channel_id BLOB NOT NULL, timestamp INTEGER NOT NULL)")
statement.executeUpdate("CREATE TABLE IF NOT EXISTS relayed (amount_in_msat INTEGER NOT NULL, amount_out_msat INTEGER NOT NULL, payment_hash BLOB NOT NULL, from_channel_id BLOB NOT NULL, to_channel_id BLOB NOT NULL, timestamp INTEGER NOT NULL)")
statement.executeUpdate("CREATE TABLE IF NOT EXISTS relayed (payment_hash BLOB NOT NULL, amount_msat INTEGER NOT NULL, channel_id BLOB NOT NULL, direction TEXT NOT NULL, relay_type TEXT NOT NULL, timestamp INTEGER NOT NULL)")
statement.executeUpdate("CREATE TABLE IF NOT EXISTS network_fees (channel_id BLOB NOT NULL, node_id BLOB NOT NULL, tx_id BLOB NOT NULL, fee_sat INTEGER NOT NULL, tx_type TEXT NOT NULL, timestamp INTEGER NOT NULL)")
statement.executeUpdate("CREATE TABLE IF NOT EXISTS channel_events (channel_id BLOB NOT NULL, node_id BLOB NOT NULL, capacity_sat INTEGER NOT NULL, is_funder BOOLEAN NOT NULL, is_private BOOLEAN NOT NULL, event TEXT NOT NULL, timestamp INTEGER NOT NULL)")
statement.executeUpdate("CREATE TABLE IF NOT EXISTS channel_errors (channel_id BLOB NOT NULL, node_id BLOB NOT NULL, error_name TEXT NOT NULL, error_message TEXT NOT NULL, is_fatal INTEGER NOT NULL, timestamp INTEGER NOT NULL)")
@ -73,6 +102,7 @@ class SqliteAuditDb(sqlite: Connection) extends AuditDb with Logging {
statement.executeUpdate("CREATE INDEX IF NOT EXISTS sent_timestamp_idx ON sent(timestamp)")
statement.executeUpdate("CREATE INDEX IF NOT EXISTS received_timestamp_idx ON received(timestamp)")
statement.executeUpdate("CREATE INDEX IF NOT EXISTS relayed_timestamp_idx ON relayed(timestamp)")
statement.executeUpdate("CREATE INDEX IF NOT EXISTS relayed_payment_hash_idx ON relayed(payment_hash)")
statement.executeUpdate("CREATE INDEX IF NOT EXISTS network_fees_timestamp_idx ON network_fees(timestamp)")
statement.executeUpdate("CREATE INDEX IF NOT EXISTS channel_events_timestamp_idx ON channel_events(timestamp)")
statement.executeUpdate("CREATE INDEX IF NOT EXISTS channel_errors_timestamp_idx ON channel_errors(timestamp)")
@ -104,15 +134,18 @@ class SqliteAuditDb(sqlite: Connection) extends AuditDb with Logging {
}
override def add(e: PaymentSent): Unit =
using(sqlite.prepareStatement("INSERT INTO sent VALUES (?, ?, ?, ?, ?, ?, ?)")) { statement =>
using(sqlite.prepareStatement("INSERT INTO sent VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?)")) { statement =>
e.parts.foreach(p => {
statement.setLong(1, p.amount.toLong)
statement.setLong(2, p.feesPaid.toLong)
statement.setBytes(3, e.paymentHash.toArray)
statement.setBytes(4, e.paymentPreimage.toArray)
statement.setBytes(5, p.toChannelId.toArray)
statement.setLong(6, p.timestamp)
statement.setBytes(7, p.id.toString.getBytes)
statement.setLong(3, e.recipientAmount.toLong)
statement.setString(4, p.id.toString)
statement.setString(5, e.id.toString)
statement.setBytes(6, e.paymentHash.toArray)
statement.setBytes(7, e.paymentPreimage.toArray)
statement.setBytes(8, e.recipientNodeId.value.toArray)
statement.setBytes(9, p.toChannelId.toArray)
statement.setLong(10, p.timestamp)
statement.addBatch()
})
statement.executeBatch()
@ -130,23 +163,27 @@ class SqliteAuditDb(sqlite: Connection) extends AuditDb with Logging {
statement.executeBatch()
}
override def add(e: PaymentRelayed): Unit =
using(sqlite.prepareStatement("INSERT INTO relayed VALUES (?, ?, ?, ?, ?, ?)")) { statement =>
statement.setLong(1, e.amountIn.toLong)
statement.setLong(2, e.amountOut.toLong)
statement.setBytes(3, e.paymentHash.toArray)
e match {
case ChannelPaymentRelayed(_, _, _, fromChannelId, toChannelId, _) =>
statement.setBytes(4, fromChannelId.toArray)
statement.setBytes(5, toChannelId.toArray)
case TrampolinePaymentRelayed(_, _, _, _, fromChannelIds, toChannelIds, _) =>
// TODO: @t-bast: we should change the DB schema to allow accurate Trampoline reporting
statement.setBytes(4, fromChannelIds.head.toArray)
statement.setBytes(5, toChannelIds.head.toArray)
override def add(e: PaymentRelayed): Unit = {
val payments = e match {
case ChannelPaymentRelayed(amountIn, amountOut, _, fromChannelId, toChannelId, ts) =>
// non-trampoline relayed payments have one input and one output
Seq(RelayedPart(fromChannelId, amountIn, "IN", "channel", ts), RelayedPart(toChannelId, amountOut, "OUT", "channel", ts))
case TrampolinePaymentRelayed(_, incoming, outgoing, ts) =>
// trampoline relayed payments do MPP aggregation and may have M inputs and N outputs
incoming.map(i => RelayedPart(i.channelId, i.amount, "IN", "trampoline", ts)) ++ outgoing.map(o => RelayedPart(o.channelId, o.amount, "OUT", "trampoline", ts))
}
for (p <- payments) {
using(sqlite.prepareStatement("INSERT INTO relayed VALUES (?, ?, ?, ?, ?, ?)")) { statement =>
statement.setBytes(1, e.paymentHash.toArray)
statement.setLong(2, p.amount.toLong)
statement.setBytes(3, p.channelId.toArray)
statement.setString(4, p.direction)
statement.setString(5, p.relayType)
statement.setLong(6, e.timestamp)
statement.executeUpdate()
}
}
}
override def add(e: NetworkFeePaid): Unit =
using(sqlite.prepareStatement("INSERT INTO network_fees VALUES (?, ?, ?, ?, ?, ?)")) { statement =>
@ -175,61 +212,84 @@ class SqliteAuditDb(sqlite: Connection) extends AuditDb with Logging {
}
override def listSent(from: Long, to: Long): Seq[PaymentSent] =
using(sqlite.prepareStatement("SELECT * FROM sent WHERE timestamp >= ? AND timestamp < ? ORDER BY timestamp")) { statement =>
using(sqlite.prepareStatement("SELECT * FROM sent WHERE timestamp >= ? AND timestamp < ?")) { statement =>
statement.setLong(1, from)
statement.setLong(2, to)
val rs = statement.executeQuery()
var q: Queue[PaymentSent] = Queue()
var sentByParentId = Map.empty[UUID, PaymentSent]
while (rs.next()) {
q = q :+ PaymentSent(
UUID.fromString(rs.getString("id")),
rs.getByteVector32("payment_hash"),
rs.getByteVector32("payment_preimage"),
Seq(PaymentSent.PartialPayment(
UUID.fromString(rs.getString("id")),
val parentId = UUID.fromString(rs.getString("parent_payment_id"))
val part = PaymentSent.PartialPayment(
UUID.fromString(rs.getString("payment_id")),
MilliSatoshi(rs.getLong("amount_msat")),
MilliSatoshi(rs.getLong("fees_msat")),
rs.getByteVector32("to_channel_id"),
None, // we don't store the route
rs.getLong("timestamp"))))
None, // we don't store the route in the audit DB
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")),
Seq(part))
}
q
sentByParentId = sentByParentId + (parentId -> sent)
}
sentByParentId.values.toSeq.sortBy(_.timestamp)
}
override def listReceived(from: Long, to: Long): Seq[PaymentReceived] =
using(sqlite.prepareStatement("SELECT * FROM received WHERE timestamp >= ? AND timestamp < ? ORDER BY timestamp")) { statement =>
using(sqlite.prepareStatement("SELECT * FROM received WHERE timestamp >= ? AND timestamp < ?")) { statement =>
statement.setLong(1, from)
statement.setLong(2, to)
val rs = statement.executeQuery()
var q: Queue[PaymentReceived] = Queue()
var receivedByHash = Map.empty[ByteVector32, PaymentReceived]
while (rs.next()) {
q = q :+ PaymentReceived(
rs.getByteVector32("payment_hash"),
Seq(PaymentReceived.PartialPayment(
val paymentHash = rs.getByteVector32("payment_hash")
val part = PaymentReceived.PartialPayment(
MilliSatoshi(rs.getLong("amount_msat")),
rs.getByteVector32("from_channel_id"),
rs.getLong("timestamp")
)))
rs.getLong("timestamp"))
val received = receivedByHash.get(paymentHash) match {
case Some(r) => r.copy(parts = r.parts :+ part)
case None => PaymentReceived(paymentHash, Seq(part))
}
q
receivedByHash = receivedByHash + (paymentHash -> received)
}
receivedByHash.values.toSeq.sortBy(_.timestamp)
}
override def listRelayed(from: Long, to: Long): Seq[PaymentRelayed] =
using(sqlite.prepareStatement("SELECT * FROM relayed WHERE timestamp >= ? AND timestamp < ? ORDER BY timestamp")) { statement =>
using(sqlite.prepareStatement("SELECT * FROM relayed WHERE timestamp >= ? AND timestamp < ?")) { statement =>
statement.setLong(1, from)
statement.setLong(2, to)
val rs = statement.executeQuery()
var q: Queue[PaymentRelayed] = Queue()
var relayedByHash = Map.empty[ByteVector32, Seq[RelayedPart]]
while (rs.next()) {
q = q :+ ChannelPaymentRelayed(
amountIn = MilliSatoshi(rs.getLong("amount_in_msat")),
amountOut = MilliSatoshi(rs.getLong("amount_out_msat")),
paymentHash = rs.getByteVector32("payment_hash"),
fromChannelId = rs.getByteVector32("from_channel_id"),
toChannelId = rs.getByteVector32("to_channel_id"),
timestamp = rs.getLong("timestamp"))
val paymentHash = rs.getByteVector32("payment_hash")
val part = RelayedPart(
rs.getByteVector32("channel_id"),
MilliSatoshi(rs.getLong("amount_msat")),
rs.getString("direction"),
rs.getString("relay_type"),
rs.getLong("timestamp"))
relayedByHash = relayedByHash + (paymentHash -> (relayedByHash.getOrElse(paymentHash, Nil) :+ part))
}
q
relayedByHash.map {
case (paymentHash, parts) => parts.head.relayType match {
case "channel" => parts.foldLeft(ChannelPaymentRelayed(0 msat, 0 msat, paymentHash, ByteVector32.Zeroes, ByteVector32.Zeroes, parts.head.timestamp)) {
case (e, part) if part.direction == "IN" => e.copy(amountIn = part.amount, fromChannelId = part.channelId)
case (e, part) if part.direction == "OUT" => e.copy(amountOut = part.amount, toChannelId = part.channelId)
}
case "trampoline" => parts.foldLeft(TrampolinePaymentRelayed(paymentHash, Nil, Nil, parts.head.timestamp)) {
case (e, part) if part.direction == "IN" => e.copy(incoming = e.incoming :+ PaymentRelayed.Part(part.amount, part.channelId))
case (e, part) if part.direction == "OUT" => e.copy(outgoing = e.outgoing :+ PaymentRelayed.Part(part.amount, part.channelId))
}
}
}.toSeq.sortBy(_.timestamp)
}
override def listNetworkFees(from: Long, to: Long): Seq[NetworkFee] =
@ -250,47 +310,33 @@ class SqliteAuditDb(sqlite: Connection) extends AuditDb with Logging {
q
}
override def stats: Seq[Stats] =
using(sqlite.createStatement()) { statement =>
val rs = statement.executeQuery(
"""
|SELECT
| channel_id,
| sum(avg_payment_amount_sat) AS avg_payment_amount_sat,
| sum(payment_count) AS payment_count,
| sum(relay_fee_sat) AS relay_fee_sat,
| sum(network_fee_sat) AS network_fee_sat
|FROM (
| SELECT
| to_channel_id AS channel_id,
| avg(amount_out_msat) / 1000 AS avg_payment_amount_sat,
| count(*) AS payment_count,
| sum(amount_in_msat - amount_out_msat) / 1000 AS relay_fee_sat,
| 0 AS network_fee_sat
| FROM relayed
| GROUP BY 1
| UNION
| SELECT
| channel_id,
| 0 AS avg_payment_amount_sat,
| 0 AS payment_count,
| 0 AS relay_fee_sat,
| sum(fee_sat) AS network_fee_sat
| FROM network_fees
| GROUP BY 1
|)
|GROUP BY 1
""".stripMargin)
var q: Queue[Stats] = Queue()
while (rs.next()) {
q = q :+ Stats(
channelId = rs.getByteVector32("channel_id"),
avgPaymentAmount = Satoshi(rs.getLong("avg_payment_amount_sat")),
paymentCount = rs.getInt("payment_count"),
relayFee = Satoshi(rs.getLong("relay_fee_sat")),
networkFee = Satoshi(rs.getLong("network_fee_sat")))
override def stats: Seq[Stats] = {
val networkFees = listNetworkFees(0, Platform.currentTime + 1).foldLeft(Map.empty[ByteVector32, Satoshi]) { case (feeByChannelId, f) =>
feeByChannelId + (f.channelId -> (feeByChannelId.getOrElse(f.channelId, 0 sat) + f.fee))
}
q
val relayed = listRelayed(0, Platform.currentTime + 1).foldLeft(Map.empty[ByteVector32, Seq[PaymentRelayed]]) { case (relayedByChannelId, e) =>
val relayedTo = e match {
case c: ChannelPaymentRelayed => Set(c.toChannelId)
case t: TrampolinePaymentRelayed => t.outgoing.map(_.channelId).toSet
}
val updated = relayedTo.map(channelId => (channelId, relayedByChannelId.getOrElse(channelId, Nil) :+ e)).toMap
relayedByChannelId ++ updated
}
networkFees.map {
case (channelId, networkFee) =>
val r = relayed.getOrElse(channelId, Nil)
val paymentCount = r.length
if (paymentCount == 0) {
Stats(channelId, 0 sat, 0, 0 sat, networkFee)
} else {
val avgPaymentAmount = r.map(_.amountOut).sum / paymentCount
val relayFee = r.map {
case c: ChannelPaymentRelayed => c.amountIn - c.amountOut
case t: TrampolinePaymentRelayed => (t.amountIn - t.amountOut) * t.outgoing.count(_.channelId == channelId) / t.outgoing.length
}.sum
Stats(channelId, avgPaymentAmount.truncateToSatoshi, paymentCount, relayFee.truncateToSatoshi, networkFee)
}
}.toSeq
}
// used by mobile apps

View file

@ -37,28 +37,22 @@ import scala.concurrent.duration._
class SqlitePaymentsDb(sqlite: Connection) extends PaymentsDb with Logging {
import SqlitePaymentsDb._
import SqliteUtils.ExtendedResultSet._
val DB_NAME = "payments"
val CURRENT_VERSION = 3
private val hopSummaryCodec = (("node_id" | CommonCodecs.publicKey) :: ("next_node_id" | CommonCodecs.publicKey) :: ("short_channel_id" | optional(bool, CommonCodecs.shortchannelid))).as[HopSummary]
private val paymentRouteCodec = discriminated[List[HopSummary]].by(byte)
.typecase(0x01, listOfN(uint8, hopSummaryCodec))
private val failureSummaryCodec = (("type" | enumerated(uint8, FailureType)) :: ("message" | ascii32) :: paymentRouteCodec).as[FailureSummary]
private val paymentFailuresCodec = discriminated[List[FailureSummary]].by(byte)
.typecase(0x01, listOfN(uint8, failureSummaryCodec))
val CURRENT_VERSION = 4
using(sqlite.createStatement(), inTransaction = true) { statement =>
def migration12(statement: Statement) = {
def migration12(statement: Statement): Int = {
// Version 2 is "backwards compatible" in the sense that it uses separate tables from version 1 (which used a single "payments" table).
statement.executeUpdate("CREATE TABLE IF NOT EXISTS received_payments (payment_hash BLOB NOT NULL PRIMARY KEY, preimage BLOB NOT NULL, payment_request TEXT NOT NULL, received_msat INTEGER, created_at INTEGER NOT NULL, expire_at INTEGER, received_at INTEGER)")
statement.executeUpdate("CREATE TABLE IF NOT EXISTS sent_payments (id TEXT NOT NULL PRIMARY KEY, payment_hash BLOB NOT NULL, preimage BLOB, amount_msat INTEGER NOT NULL, created_at INTEGER NOT NULL, completed_at INTEGER, status VARCHAR NOT NULL)")
statement.executeUpdate("CREATE INDEX IF NOT EXISTS payment_hash_idx ON sent_payments(payment_hash)")
}
def migration23(statement: Statement) = {
def migration23(statement: Statement): Int = {
// We add many more columns to the sent_payments table.
statement.executeUpdate("DROP index payment_hash_idx")
statement.executeUpdate("ALTER TABLE sent_payments RENAME TO _sent_payments_old")
@ -82,19 +76,47 @@ class SqlitePaymentsDb(sqlite: Connection) extends PaymentsDb with Logging {
statement.executeUpdate("CREATE INDEX IF NOT EXISTS received_created_idx ON received_payments(created_at)")
}
def migration34(statement: Statement): Int = {
// We add a recipient_amount_msat and payment_type columns, rename some columns and change column order.
statement.executeUpdate("DROP index sent_parent_id_idx")
statement.executeUpdate("DROP index sent_payment_hash_idx")
statement.executeUpdate("DROP index sent_created_idx")
statement.executeUpdate("ALTER TABLE sent_payments RENAME TO _sent_payments_old")
statement.executeUpdate("CREATE TABLE sent_payments (id TEXT NOT NULL PRIMARY KEY, parent_id TEXT NOT NULL, external_id TEXT, payment_hash BLOB NOT NULL, payment_preimage BLOB, payment_type TEXT NOT NULL, amount_msat INTEGER NOT NULL, fees_msat INTEGER, recipient_amount_msat INTEGER NOT NULL, recipient_node_id BLOB NOT NULL, payment_request TEXT, payment_route BLOB, failures BLOB, created_at INTEGER NOT NULL, completed_at INTEGER)")
statement.executeUpdate("INSERT INTO sent_payments (id, parent_id, external_id, payment_hash, payment_preimage, payment_type, amount_msat, fees_msat, recipient_amount_msat, recipient_node_id, payment_request, payment_route, failures, created_at, completed_at) SELECT id, parent_id, external_id, payment_hash, payment_preimage, 'Standard', amount_msat, fees_msat, amount_msat, target_node_id, payment_request, payment_route, failures, created_at, completed_at FROM _sent_payments_old")
statement.executeUpdate("DROP table _sent_payments_old")
statement.executeUpdate("CREATE INDEX IF NOT EXISTS sent_parent_id_idx ON sent_payments(parent_id)")
statement.executeUpdate("CREATE INDEX IF NOT EXISTS sent_payment_hash_idx ON sent_payments(payment_hash)")
statement.executeUpdate("CREATE INDEX IF NOT EXISTS sent_created_idx ON sent_payments(created_at)")
// We add payment_type column.
statement.executeUpdate("DROP index received_created_idx")
statement.executeUpdate("ALTER TABLE received_payments RENAME TO _received_payments_old")
statement.executeUpdate("CREATE TABLE received_payments (payment_hash BLOB NOT NULL PRIMARY KEY, payment_type TEXT NOT NULL, payment_preimage BLOB NOT NULL, payment_request TEXT NOT NULL, received_msat INTEGER, created_at INTEGER NOT NULL, expire_at INTEGER NOT NULL, received_at INTEGER)")
statement.executeUpdate("INSERT INTO received_payments (payment_hash, payment_type, payment_preimage, payment_request, received_msat, created_at, expire_at, received_at) SELECT payment_hash, 'Standard', payment_preimage, payment_request, received_msat, created_at, expire_at, received_at FROM _received_payments_old")
statement.executeUpdate("DROP table _received_payments_old")
statement.executeUpdate("CREATE INDEX IF NOT EXISTS received_created_idx ON received_payments(created_at)")
}
getVersion(statement, DB_NAME, CURRENT_VERSION) match {
case 1 =>
logger.warn(s"migrating db $DB_NAME, found version=1 current=$CURRENT_VERSION")
migration12(statement)
migration23(statement)
migration34(statement)
setVersion(statement, DB_NAME, CURRENT_VERSION)
case 2 =>
logger.warn(s"migrating db $DB_NAME, found version=2 current=$CURRENT_VERSION")
migration23(statement)
migration34(statement)
setVersion(statement, DB_NAME, CURRENT_VERSION)
case 3 =>
logger.warn(s"migrating db $DB_NAME, found version=3 current=$CURRENT_VERSION")
migration34(statement)
setVersion(statement, DB_NAME, CURRENT_VERSION)
case CURRENT_VERSION =>
statement.executeUpdate("CREATE TABLE IF NOT EXISTS received_payments (payment_hash BLOB NOT NULL PRIMARY KEY, payment_preimage BLOB NOT NULL, payment_request TEXT NOT NULL, received_msat INTEGER, created_at INTEGER NOT NULL, expire_at INTEGER NOT NULL, received_at INTEGER)")
statement.executeUpdate("CREATE TABLE IF NOT EXISTS sent_payments (id TEXT NOT NULL PRIMARY KEY, parent_id TEXT NOT NULL, external_id TEXT, payment_hash BLOB NOT NULL, amount_msat INTEGER NOT NULL, target_node_id BLOB NOT NULL, created_at INTEGER NOT NULL, payment_request TEXT, completed_at INTEGER, payment_preimage BLOB, fees_msat INTEGER, payment_route BLOB, failures BLOB)")
statement.executeUpdate("CREATE TABLE IF NOT EXISTS received_payments (payment_hash BLOB NOT NULL PRIMARY KEY, payment_type TEXT NOT NULL, payment_preimage BLOB NOT NULL, payment_request TEXT NOT NULL, received_msat INTEGER, created_at INTEGER NOT NULL, expire_at INTEGER NOT NULL, received_at INTEGER)")
statement.executeUpdate("CREATE TABLE IF NOT EXISTS sent_payments (id TEXT NOT NULL PRIMARY KEY, parent_id TEXT NOT NULL, external_id TEXT, payment_hash BLOB NOT NULL, payment_preimage BLOB, payment_type TEXT NOT NULL, amount_msat INTEGER NOT NULL, fees_msat INTEGER, recipient_amount_msat INTEGER NOT NULL, recipient_node_id BLOB NOT NULL, payment_request TEXT, payment_route BLOB, failures BLOB, created_at INTEGER NOT NULL, completed_at INTEGER)")
statement.executeUpdate("CREATE INDEX IF NOT EXISTS sent_parent_id_idx ON sent_payments(parent_id)")
statement.executeUpdate("CREATE INDEX IF NOT EXISTS sent_payment_hash_idx ON sent_payments(payment_hash)")
@ -107,15 +129,17 @@ class SqlitePaymentsDb(sqlite: Connection) extends PaymentsDb with Logging {
override def addOutgoingPayment(sent: OutgoingPayment): Unit = {
require(sent.status == OutgoingPaymentStatus.Pending, s"outgoing payment isn't pending (${sent.status.getClass.getSimpleName})")
using(sqlite.prepareStatement("INSERT INTO sent_payments (id, parent_id, external_id, payment_hash, amount_msat, target_node_id, created_at, payment_request) VALUES (?, ?, ?, ?, ?, ?, ?, ?)")) { statement =>
using(sqlite.prepareStatement("INSERT INTO sent_payments (id, parent_id, external_id, payment_hash, payment_type, amount_msat, recipient_amount_msat, recipient_node_id, created_at, payment_request) VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?)")) { statement =>
statement.setString(1, sent.id.toString)
statement.setString(2, sent.parentId.toString)
statement.setString(3, sent.externalId.orNull)
statement.setBytes(4, sent.paymentHash.toArray)
statement.setLong(5, sent.amount.toLong)
statement.setBytes(6, sent.targetNodeId.value.toArray)
statement.setLong(7, sent.createdAt)
statement.setString(8, sent.paymentRequest.map(PaymentRequest.write).orNull)
statement.setString(5, sent.paymentType)
statement.setLong(6, sent.amount.toLong)
statement.setLong(7, sent.recipientAmount.toLong)
statement.setBytes(8, sent.recipientNodeId.value.toArray)
statement.setLong(9, sent.createdAt)
statement.setString(10, sent.paymentRequest.map(PaymentRequest.write).orNull)
statement.executeUpdate()
}
}
@ -154,8 +178,10 @@ class SqlitePaymentsDb(sqlite: Connection) extends PaymentsDb with Logging {
UUID.fromString(rs.getString("parent_id")),
rs.getStringNullable("external_id"),
rs.getByteVector32("payment_hash"),
rs.getString("payment_type"),
MilliSatoshi(rs.getLong("amount_msat")),
PublicKey(rs.getByteVector("target_node_id")),
MilliSatoshi(rs.getLong("recipient_amount_msat")),
PublicKey(rs.getByteVector("recipient_node_id")),
rs.getLong("created_at"),
rs.getStringNullable("payment_request").map(PaymentRequest.read),
status
@ -232,13 +258,14 @@ class SqlitePaymentsDb(sqlite: Connection) extends PaymentsDb with Logging {
q
}
override def addIncomingPayment(pr: PaymentRequest, preimage: ByteVector32): Unit =
using(sqlite.prepareStatement("INSERT INTO received_payments (payment_hash, payment_preimage, payment_request, created_at, expire_at) VALUES (?, ?, ?, ?, ?)")) { statement =>
override def addIncomingPayment(pr: PaymentRequest, preimage: ByteVector32, paymentType: String): Unit =
using(sqlite.prepareStatement("INSERT INTO received_payments (payment_hash, payment_preimage, payment_type, payment_request, created_at, expire_at) VALUES (?, ?, ?, ?, ?, ?)")) { statement =>
statement.setBytes(1, pr.paymentHash.toArray)
statement.setBytes(2, preimage.toArray)
statement.setString(3, PaymentRequest.write(pr))
statement.setLong(4, pr.timestamp.seconds.toMillis) // BOLT11 timestamp is in seconds
statement.setLong(5, (pr.timestamp + pr.expiry.getOrElse(PaymentRequest.DEFAULT_EXPIRY_SECONDS.toLong)).seconds.toMillis)
statement.setString(3, paymentType)
statement.setString(4, PaymentRequest.write(pr))
statement.setLong(5, pr.timestamp.seconds.toMillis) // BOLT11 timestamp is in seconds
statement.setLong(6, (pr.timestamp + pr.expiry.getOrElse(PaymentRequest.DEFAULT_EXPIRY_SECONDS.toLong)).seconds.toMillis)
statement.executeUpdate()
}
@ -255,8 +282,10 @@ class SqlitePaymentsDb(sqlite: Connection) extends PaymentsDb with Logging {
private def parseIncomingPayment(rs: ResultSet): IncomingPayment = {
val paymentRequest = rs.getString("payment_request")
IncomingPayment(PaymentRequest.read(paymentRequest),
IncomingPayment(
PaymentRequest.read(paymentRequest),
rs.getByteVector32("payment_preimage"),
rs.getString("payment_type"),
rs.getLong("created_at"),
buildIncomingPaymentStatus(rs.getMilliSatoshiNullable("received_msat"), Some(paymentRequest), rs.getLongNullable("received_at")))
}
@ -344,9 +373,9 @@ class SqlitePaymentsDb(sqlite: Connection) extends PaymentsDb with Logging {
| NULL as external_id,
| payment_hash,
| payment_preimage,
| payment_type,
| received_msat as final_amount,
| payment_request,
| NULL as target_node_id,
| created_at,
| received_at as completed_at,
| expire_at,
@ -359,9 +388,9 @@ class SqlitePaymentsDb(sqlite: Connection) extends PaymentsDb with Logging {
| external_id,
| payment_hash,
| payment_preimage,
| payment_type,
| sum(amount_msat + fees_msat) as final_amount,
| payment_request,
| target_node_id,
| created_at,
| completed_at,
| NULL as expire_at,
@ -380,6 +409,7 @@ class SqlitePaymentsDb(sqlite: Connection) extends PaymentsDb with Logging {
val parentId = rs.getUUIDNullable("parent_id")
val externalId_opt = rs.getStringNullable("external_id")
val paymentHash = rs.getByteVector32("payment_hash")
val paymentType = rs.getString("payment_type")
val paymentRequest_opt = rs.getStringNullable("payment_request")
val amount_opt = rs.getMilliSatoshiNullable("final_amount")
val createdAt = rs.getLong("created_at")
@ -388,12 +418,12 @@ class SqlitePaymentsDb(sqlite: Connection) extends PaymentsDb with Logging {
val p = if (rs.getString("type") == "received") {
val status: IncomingPaymentStatus = buildIncomingPaymentStatus(amount_opt, paymentRequest_opt, completedAt_opt)
PlainIncomingPayment(paymentHash, amount_opt, paymentRequest_opt, status, createdAt, completedAt_opt, expireAt_opt)
PlainIncomingPayment(paymentHash, paymentType, amount_opt, paymentRequest_opt, status, createdAt, completedAt_opt, expireAt_opt)
} else {
val preimage_opt = rs.getByteVector32Nullable("payment_preimage")
// note that the resulting status will not contain any details (routes, failures...)
val status: OutgoingPaymentStatus = buildOutgoingPaymentStatus(preimage_opt, None, None, completedAt_opt, None)
PlainOutgoingPayment(parentId, externalId_opt, paymentHash, amount_opt, paymentRequest_opt, status, createdAt, completedAt_opt)
PlainOutgoingPayment(parentId, externalId_opt, paymentHash, paymentType, amount_opt, paymentRequest_opt, status, createdAt, completedAt_opt)
}
q = q :+ p
}
@ -403,4 +433,16 @@ class SqlitePaymentsDb(sqlite: Connection) extends PaymentsDb with Logging {
// used by mobile apps
override def close(): Unit = sqlite.close()
}
object SqlitePaymentsDb {
private val hopSummaryCodec = (("node_id" | CommonCodecs.publicKey) :: ("next_node_id" | CommonCodecs.publicKey) :: ("short_channel_id" | optional(bool, CommonCodecs.shortchannelid))).as[HopSummary]
val paymentRouteCodec = discriminated[List[HopSummary]].by(byte)
.typecase(0x01, listOfN(uint8, hopSummaryCodec))
private val failureSummaryCodec = (("type" | enumerated(uint8, FailureType)) :: ("message" | ascii32) :: paymentRouteCodec).as[FailureSummary]
val paymentFailuresCodec = discriminated[List[FailureSummary]].by(byte)
.typecase(0x01, listOfN(uint8, failureSummaryCodec))
}

View file

@ -48,7 +48,7 @@ class Auditor(nodeParams: NodeParams) extends Actor with ActorLogging {
.histogram("payment.hist")
.withTag("direction", "sent")
.withTag("type", "amount")
.record(e.amount.truncateToSatoshi.toLong)
.record(e.recipientAmount.truncateToSatoshi.toLong)
Kamon
.histogram("payment.hist")
.withTag("direction", "sent")

View file

@ -22,7 +22,7 @@ import fr.acinq.bitcoin.ByteVector32
import fr.acinq.bitcoin.Crypto.PublicKey
import fr.acinq.eclair.MilliSatoshi
import fr.acinq.eclair.crypto.Sphinx
import fr.acinq.eclair.router.ChannelHop
import fr.acinq.eclair.router.Hop
import scala.compat.Platform
@ -35,19 +35,41 @@ sealed trait PaymentEvent {
val timestamp: Long
}
case class PaymentSent(id: UUID, paymentHash: ByteVector32, paymentPreimage: ByteVector32, parts: Seq[PaymentSent.PartialPayment]) extends PaymentEvent {
require(parts.nonEmpty, "must have at least one subpayment")
val amount: MilliSatoshi = parts.map(_.amount).sum
val feesPaid: MilliSatoshi = parts.map(_.feesPaid).sum
/**
* A payment was successfully sent and fulfilled.
*
* @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).
*/
case class PaymentSent(id: UUID, paymentHash: ByteVector32, paymentPreimage: ByteVector32, recipientAmount: MilliSatoshi, recipientNodeId: PublicKey, parts: Seq[PaymentSent.PartialPayment]) extends PaymentEvent {
require(parts.nonEmpty, "must have at least one payment part")
val amountWithFees: MilliSatoshi = parts.map(_.amountWithFees).sum
val feesPaid: MilliSatoshi = amountWithFees - recipientAmount // overall fees for this payment (routing + trampoline)
val trampolineFees: MilliSatoshi = parts.map(_.amount).sum - recipientAmount
val nonTrampolineFees: MilliSatoshi = feesPaid - trampolineFees // routing fees to reach the first trampoline node, or the recipient if not using trampoline
val timestamp: Long = parts.map(_.timestamp).min // we use min here because we receive the proof of payment as soon as the first partial payment is fulfilled
}
// TODO: @t-bast: the route fields should be a Seq[Hop], not Seq[ChannelHop]
object PaymentSent {
case class PartialPayment(id: UUID, amount: MilliSatoshi, feesPaid: MilliSatoshi, toChannelId: ByteVector32, route: Option[Seq[ChannelHop]], timestamp: Long = Platform.currentTime) {
/**
* A successfully sent partial payment (single outgoing HTLC).
*
* @param id id of the outgoing payment.
* @param amount amount received by the target node.
* @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.
*/
case class PartialPayment(id: UUID, amount: MilliSatoshi, feesPaid: MilliSatoshi, toChannelId: ByteVector32, route: Option[Seq[Hop]], timestamp: Long = Platform.currentTime) {
require(route.isEmpty || route.get.nonEmpty, "route must be None or contain at least one hop")
val amountWithFees: MilliSatoshi = amount + feesPaid
}
}
@ -57,14 +79,27 @@ case class PaymentFailed(id: UUID, paymentHash: ByteVector32, failures: Seq[Paym
sealed trait PaymentRelayed extends PaymentEvent {
val amountIn: MilliSatoshi
val amountOut: MilliSatoshi
val timestamp: Long
}
case class ChannelPaymentRelayed(amountIn: MilliSatoshi, amountOut: MilliSatoshi, paymentHash: ByteVector32, fromChannelId: ByteVector32, toChannelId: ByteVector32, timestamp: Long = Platform.currentTime) extends PaymentRelayed
case class TrampolinePaymentRelayed(amountIn: MilliSatoshi, amountOut: MilliSatoshi, paymentHash: ByteVector32, toNodeId: PublicKey, fromChannelIds: Seq[ByteVector32], toChannelIds: Seq[ByteVector32], timestamp: Long = Platform.currentTime) extends PaymentRelayed
case class TrampolinePaymentRelayed(paymentHash: ByteVector32, incoming: PaymentRelayed.Incoming, outgoing: PaymentRelayed.Outgoing, timestamp: Long = Platform.currentTime) extends PaymentRelayed {
override val amountIn: MilliSatoshi = incoming.map(_.amount).sum
override val amountOut: MilliSatoshi = outgoing.map(_.amount).sum
}
object PaymentRelayed {
case class Part(amount: MilliSatoshi, channelId: ByteVector32)
type Incoming = Seq[Part]
type Outgoing = Seq[Part]
}
case class PaymentReceived(paymentHash: ByteVector32, parts: Seq[PaymentReceived.PartialPayment]) extends PaymentEvent {
require(parts.nonEmpty, "must have at least one subpayment")
require(parts.nonEmpty, "must have at least one payment part")
val amount: MilliSatoshi = parts.map(_.amount).sum
val timestamp: Long = parts.map(_.timestamp).max // we use max here because we fulfill the payment only once we received all the parts
}
@ -83,14 +118,13 @@ sealed trait PaymentFailure
case class LocalFailure(t: Throwable) extends PaymentFailure
/** A remote node failed the payment and we were able to decrypt the onion failure packet. */
case class RemoteFailure(route: Seq[ChannelHop], e: Sphinx.DecryptedFailurePacket) extends PaymentFailure
case class RemoteFailure(route: Seq[Hop], e: Sphinx.DecryptedFailurePacket) extends PaymentFailure
/** 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(route: Seq[ChannelHop]) extends PaymentFailure
case class UnreadableRemoteFailure(route: Seq[Hop]) extends PaymentFailure
object PaymentFailure {
import fr.acinq.bitcoin.Crypto.PublicKey
import fr.acinq.eclair.channel.AddHtlcFailed
import fr.acinq.eclair.router.RouteNotFound
import fr.acinq.eclair.wire.Update

View file

@ -495,15 +495,14 @@ object PaymentRequest {
timestamp = bolt11Data.timestamp,
nodeId = pub,
tags = bolt11Data.taggedFields,
signature = bolt11Data.signature
)
signature = bolt11Data.signature)
}
private def readBoltData(input: String): Bolt11Data = {
val lowercaseInput = input.toLowerCase
val separatorIndex = lowercaseInput.lastIndexOf('1')
val hrp = lowercaseInput.take(separatorIndex)
val prefix: String = prefixes.values.find(prefix => hrp.startsWith(prefix)).getOrElse(throw new RuntimeException("unknown prefix"))
if (!prefixes.values.exists(prefix => hrp.startsWith(prefix))) throw new RuntimeException("unknown prefix")
val data = string2Bits(lowercaseInput.slice(separatorIndex + 1, lowercaseInput.length - 6)) // 6 == checksum size
Codecs.bolt11DataCodec.decode(data).require.value
}

View file

@ -100,14 +100,14 @@ class NodeRelayer(nodeParams: NodeParams, relayer: ActorRef, router: ActorRef, c
case None => throw new RuntimeException(s"could not find pending incoming payment (paymentHash=$paymentHash)")
}
case PaymentSent(id, paymentHash, paymentPreimage, parts) =>
case PaymentSent(id, paymentHash, paymentPreimage, _, _, parts) =>
log.debug("trampoline payment successfully relayed")
pendingOutgoing.get(id).foreach {
case PendingResult(upstream, nextPayload) =>
case PendingResult(upstream, _) =>
fulfillPayment(upstream, paymentPreimage)
val fromChannelIds = upstream.adds.map(_.channelId)
val toChannelIds = parts.map(_.toChannelId)
context.system.eventStream.publish(TrampolinePaymentRelayed(upstream.amountIn, nextPayload.amountToForward, paymentHash, nextPayload.outgoingNodeId, fromChannelIds, toChannelIds))
val incoming = upstream.adds.map(add => PaymentRelayed.Part(add.amountMsat, add.channelId))
val outgoing = parts.map(part => PaymentRelayed.Part(part.amountWithFees, part.toChannelId))
context.system.eventStream.publish(TrampolinePaymentRelayed(paymentHash, incoming, outgoing))
}
context become main(pendingIncoming, pendingOutgoing - id)
@ -130,7 +130,7 @@ class NodeRelayer(nodeParams: NodeParams, relayer: ActorRef, router: ActorRef, c
private def relay(paymentHash: ByteVector32, upstream: Upstream.TrampolineRelayed, payloadOut: Onion.NodeRelayPayload, packetOut: OnionRoutingPacket): UUID = {
val paymentId = UUID.randomUUID()
val paymentCfg = SendPaymentConfig(paymentId, paymentId, None, paymentHash, payloadOut.outgoingNodeId, upstream, None, storeInDb = false, publishEvent = false)
val paymentCfg = SendPaymentConfig(paymentId, paymentId, None, paymentHash, payloadOut.amountToForward, payloadOut.outgoingNodeId, upstream, None, storeInDb = false, publishEvent = false, Nil)
val routeParams = computeRouteParams(nodeParams, upstream.amountIn, upstream.expiryIn, payloadOut.amountToForward, payloadOut.outgoingCltv)
payloadOut.invoiceFeatures match {
case Some(_) =>
@ -139,13 +139,13 @@ class NodeRelayer(nodeParams: NodeParams, relayer: ActorRef, router: ActorRef, c
// TODO: @t-bast: MPP is disabled for trampoline to non-trampoline payments until we improve the splitting algorithm for nodes with a lot of channels.
val payFSM = spawnOutgoingPayFSM(paymentCfg, multiPart = false)
val finalPayload = Onion.createSinglePartPayload(payloadOut.amountToForward, payloadOut.outgoingCltv, payloadOut.paymentSecret)
val payment = SendPayment(paymentHash, payloadOut.outgoingNodeId, finalPayload, nodeParams.maxPaymentAttempts, routingHints, Some(routeParams))
val payment = SendPayment(payloadOut.outgoingNodeId, finalPayload, nodeParams.maxPaymentAttempts, routingHints, Some(routeParams))
payFSM ! payment
case None =>
log.debug("relaying trampoline payment to next trampoline node")
val payFSM = spawnOutgoingPayFSM(paymentCfg, multiPart = true)
val paymentSecret = randomBytes32 // we generate a new secret to protect against probing attacks
val payment = SendMultiPartPayment(paymentHash, paymentSecret, payloadOut.outgoingNodeId, payloadOut.amountToForward, payloadOut.outgoingCltv, nodeParams.maxPaymentAttempts, routeParams = Some(routeParams), additionalTlvs = Seq(OnionTlv.TrampolineOnion(packetOut)))
val payment = SendMultiPartPayment(paymentSecret, payloadOut.outgoingNodeId, payloadOut.amountToForward, payloadOut.outgoingCltv, nodeParams.maxPaymentAttempts, routeParams = Some(routeParams), additionalTlvs = Seq(OnionTlv.TrampolineOnion(packetOut)))
payFSM ! payment
}
paymentId
@ -166,12 +166,12 @@ class NodeRelayer(nodeParams: NodeParams, relayer: ActorRef, router: ActorRef, c
override def mdc(currentMessage: Any): MDC = {
val paymentHash_opt = currentMessage match {
case IncomingPacket.NodeRelayPacket(add, _, _, _) => Some(add.paymentHash)
case MultiPartPaymentFSM.MultiPartHtlcFailed(paymentHash, _, _) => Some(paymentHash)
case MultiPartPaymentFSM.MultiPartHtlcSucceeded(paymentHash, _) => Some(paymentHash)
case MultiPartPaymentFSM.ExtraHtlcReceived(paymentHash, _, _) => Some(paymentHash)
case PaymentFailed(_, paymentHash, _, _) => Some(paymentHash)
case PaymentSent(_, paymentHash, _, _) => Some(paymentHash)
case m: IncomingPacket.NodeRelayPacket => Some(m.add.paymentHash)
case m: MultiPartPaymentFSM.MultiPartHtlcFailed => Some(m.paymentHash)
case m: MultiPartPaymentFSM.MultiPartHtlcSucceeded => Some(m.paymentHash)
case m: MultiPartPaymentFSM.ExtraHtlcReceived => Some(m.paymentHash)
case m: PaymentFailed => Some(m.paymentHash)
case m: PaymentSent => Some(m.paymentHash)
case _ => None
}
Logs.mdc(category_opt = Some(Logs.LogCategory.PAYMENT), paymentHash_opt = paymentHash_opt)

View file

@ -30,6 +30,7 @@ import fr.acinq.eclair.wire.{TemporaryNodeFailure, UpdateAddHtlc}
import fr.acinq.eclair.{LongToBtcAmount, NodeParams}
import scodec.bits.ByteVector
import scala.compat.Platform
import scala.concurrent.Promise
import scala.util.Try
@ -110,20 +111,29 @@ class PostRestartHtlcCleaner(nodeParams: NodeParams, commandBuffer: ActorRef, in
case Some(relayedOut) => origin match {
case Origin.Local(id, _) =>
val feesPaid = 0.msat // fees are unknown since we lost the reference to the payment
nodeParams.db.payments.updateOutgoingPayment(PaymentSent(id, fulfilledHtlc.paymentHash, paymentPreimage, PaymentSent.PartialPayment(id, fulfilledHtlc.amountMsat, feesPaid, fulfilledHtlc.channelId, None) :: Nil))
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))
// If all downstream HTLCs are now resolved, we can emit the payment event.
nodeParams.db.payments.getOutgoingPayment(id).foreach(p => {
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)) =>
case OutgoingPayment(id, _, _, _, _, amount, _, _, _, _, OutgoingPaymentStatus.Succeeded(_, feesPaid, _, completedAt)) =>
PaymentSent.PartialPayment(id, amount, feesPaid, ByteVector32.Zeroes, None, completedAt)
}
val sent = PaymentSent(p.parentId, fulfilledHtlc.paymentHash, paymentPreimage, succeeded)
log.info(s"payment id=${sent.id} paymentHash=${sent.paymentHash} successfully sent (amount=${sent.amount})")
val sent = PaymentSent(p.parentId, fulfilledHtlc.paymentHash, paymentPreimage, p.recipientAmount, p.recipientNodeId, succeeded)
log.info(s"payment id=${sent.id} paymentHash=${sent.paymentHash} successfully sent (amount=${sent.recipientAmount})")
context.system.eventStream.publish(sent)
}
})
case None =>
log.warning(s"database inconsistency detected: payment $id is fulfilled but doesn't have a corresponding database entry")
// Since we don't have a matching DB entry, we've lost the payment recipient and total amount, so we put
// 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, Platform.currentTime, None, OutgoingPaymentStatus.Pending))
nodeParams.db.payments.updateOutgoingPayment(PaymentSent(id, fulfilledHtlc.paymentHash, paymentPreimage, dummyFinalAmount, dummyNodeId, PaymentSent.PartialPayment(id, fulfilledHtlc.amountMsat, feesPaid, fulfilledHtlc.channelId, None) :: Nil))
}
// 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.
context become main(brokenHtlcs.copy(relayedOut = brokenHtlcs.relayedOut - origin))
@ -229,7 +239,7 @@ object PostRestartHtlcCleaner {
*/
private def shouldFulfill(finalPacket: IncomingPacket.FinalPacket, paymentsDb: IncomingPaymentsDb): Option[ByteVector32] =
paymentsDb.getIncomingPayment(finalPacket.add.paymentHash) match {
case Some(IncomingPayment(_, preimage, _, IncomingPaymentStatus.Received(_, _))) => Some(preimage)
case Some(IncomingPayment(_, preimage, _, _, IncomingPaymentStatus.Received(_, _))) => Some(preimage)
case _ => None
}

View file

@ -55,11 +55,13 @@ 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
private val span = Kamon.spanBuilder("multi-part-payment")
.tag("parentPaymentId", cfg.parentId.toString)
.tag("paymentHash", cfg.paymentHash.toHex)
.tag("targetNodeId", cfg.targetNodeId.toString())
.tag("paymentHash", paymentHash.toHex)
.tag("recipientNodeId", cfg.recipientNodeId.toString())
.tag("recipientAmount", cfg.recipientAmount.toLong)
.start()
startWith(WAIT_FOR_PAYMENT_REQUEST, WaitingForRequest)
@ -94,8 +96,7 @@ class MultiPartPaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig,
val pending = setFees(d.request.routeParams, payments, payments.size)
Kamon.runWithContextEntry(parentPaymentIdKey, cfg.parentId) {
Kamon.runWithSpan(span, finishSpan = true) {
pending.headOption.foreach { case (childId, payment) => spawnChildPaymentFsm(childId, includeTrampolineFees = true) ! payment }
pending.tail.foreach { case (childId, payment) => spawnChildPaymentFsm(childId, includeTrampolineFees = false) ! payment }
pending.foreach { case (childId, payment) => spawnChildPaymentFsm(childId) ! payment }
}
}
goto(PAYMENT_IN_PROGRESS) using PaymentProgress(d.sender, d.request, d.networkStats, channels.length, 0 msat, d.request.maxAttempts - 1, pending, Set.empty, Nil)
@ -138,7 +139,7 @@ class MultiPartPaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig,
case Event(ps: PaymentSent, d: PaymentProgress) =>
require(ps.parts.length == 1, "child payment must contain only one part")
// As soon as we get the preimage we can consider that the whole payment succeeded (we have a proof of payment).
goto(PAYMENT_SUCCEEDED) using PaymentSucceeded(d.sender, d.request, ps.paymentPreimage, ps.parts, d.pending.keySet - ps.id)
goto(PAYMENT_SUCCEEDED) using PaymentSucceeded(d.sender, d.request, ps.paymentPreimage, ps.parts, d.pending.keySet - ps.parts.head.id)
}
when(RETRY_WITH_UPDATED_BALANCES) {
@ -151,7 +152,7 @@ class MultiPartPaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig,
goto(PAYMENT_ABORTED) using PaymentAborted(d.sender, d.request, d.failures :+ LocalFailure(BalanceTooLow), d.pending.keySet)
} else {
val pending = setFees(d.request.routeParams, payments, payments.size + d.pending.size)
pending.foreach { case (childId, payment) => spawnChildPaymentFsm(childId, includeTrampolineFees = false) ! payment }
pending.foreach { case (childId, payment) => spawnChildPaymentFsm(childId) ! payment }
goto(PAYMENT_IN_PROGRESS) using d.copy(toSend = 0 msat, remainingAttempts = d.remainingAttempts - 1, pending = d.pending ++ pending, channelsCount = channels.length)
}
@ -167,7 +168,7 @@ class MultiPartPaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig,
case Event(ps: PaymentSent, d: PaymentProgress) =>
require(ps.parts.length == 1, "child payment must contain only one part")
// As soon as we get the preimage we can consider that the whole payment succeeded (we have a proof of payment).
goto(PAYMENT_SUCCEEDED) using PaymentSucceeded(d.sender, d.request, ps.paymentPreimage, ps.parts, d.pending.keySet - ps.id)
goto(PAYMENT_SUCCEEDED) using PaymentSucceeded(d.sender, d.request, ps.paymentPreimage, ps.parts, d.pending.keySet - ps.parts.head.id)
}
when(PAYMENT_ABORTED) {
@ -175,7 +176,7 @@ class MultiPartPaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig,
val failures = d.failures ++ pf.failures
val pending = d.pending - pf.id
if (pending.isEmpty) {
myStop(d.sender, Left(PaymentFailed(id, d.request.paymentHash, failures)))
myStop(d.sender, Left(PaymentFailed(id, paymentHash, failures)))
} else {
stay using d.copy(failures = failures, pending = pending)
}
@ -184,17 +185,17 @@ class MultiPartPaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig,
// This is a spec violation and is too bad for them, we obtained a proof of payment without paying the full amount.
case Event(ps: PaymentSent, d: PaymentAborted) =>
require(ps.parts.length == 1, "child payment must contain only one part")
log.warning(s"payment recipient fulfilled incomplete multi-part payment (id=${ps.id})")
goto(PAYMENT_SUCCEEDED) using PaymentSucceeded(d.sender, d.request, ps.paymentPreimage, ps.parts, d.pending - ps.id)
log.warning(s"payment recipient fulfilled incomplete multi-part payment (id=${ps.parts.head.id})")
goto(PAYMENT_SUCCEEDED) using PaymentSucceeded(d.sender, d.request, ps.paymentPreimage, ps.parts, d.pending - ps.parts.head.id)
}
when(PAYMENT_SUCCEEDED) {
case Event(ps: PaymentSent, d: PaymentSucceeded) =>
require(ps.parts.length == 1, "child payment must contain only one part")
val parts = d.parts ++ ps.parts
val pending = d.pending - ps.id
val pending = d.pending - ps.parts.head.id
if (pending.isEmpty) {
myStop(d.sender, Right(PaymentSent(id, d.request.paymentHash, d.preimage, parts)))
myStop(d.sender, Right(cfg.createPaymentSent(d.preimage, parts)))
} else {
stay using d.copy(parts = parts, pending = pending)
}
@ -205,7 +206,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.sender, Right(PaymentSent(id, d.request.paymentHash, d.preimage, d.parts)))
myStop(d.sender, Right(cfg.createPaymentSent(d.preimage, d.parts)))
} else {
stay using d.copy(pending = pending)
}
@ -214,28 +215,23 @@ class MultiPartPaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig,
onTransition {
case _ -> PAYMENT_ABORTED => nextStateData match {
case d: PaymentAborted if d.pending.isEmpty =>
myStop(d.sender, Left(PaymentFailed(id, d.request.paymentHash, d.failures)))
myStop(d.sender, Left(PaymentFailed(id, paymentHash, d.failures)))
case _ =>
}
case _ -> PAYMENT_SUCCEEDED => nextStateData match {
case d: PaymentSucceeded if d.pending.isEmpty =>
myStop(d.sender, Right(PaymentSent(id, d.request.paymentHash, d.preimage, d.parts)))
myStop(d.sender, Right(cfg.createPaymentSent(d.preimage, d.parts)))
case _ =>
}
}
def spawnChildPaymentFsm(childId: UUID, includeTrampolineFees: Boolean): ActorRef = {
def spawnChildPaymentFsm(childId: UUID): ActorRef = {
val upstream = cfg.upstream match {
case Upstream.Local(_) => Upstream.Local(childId)
case _ => cfg.upstream
}
// We attach the trampoline fees to the first child in order to account for them in the DB.
// This is hackish and won't work if the first child payment fails and is retried, but it's okay-ish for an MVP.
// We will update the DB schema to contain accurate Trampoline reporting, which will fix that in the future.
// TODO: @t-bast: fix that once the DB schema is updated
val trampolineData = if (includeTrampolineFees) cfg.trampolineData else cfg.trampolineData.map(_.copy(trampolineAttempts = Nil))
val childCfg = cfg.copy(id = childId, publishEvent = false, upstream = upstream, trampolineData = trampolineData)
val childCfg = cfg.copy(id = childId, publishEvent = false, upstream = upstream)
context.actorOf(PaymentLifecycle.props(nodeParams, childCfg, router, register))
}
@ -271,7 +267,7 @@ class MultiPartPaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig,
}
override def mdc(currentMessage: Any): MDC = {
Logs.mdc(category_opt = Some(Logs.LogCategory.PAYMENT), parentPaymentId_opt = Some(cfg.parentId), paymentId_opt = Some(id), paymentHash_opt = Some(cfg.paymentHash))
Logs.mdc(category_opt = Some(Logs.LogCategory.PAYMENT), parentPaymentId_opt = Some(cfg.parentId), paymentId_opt = Some(id), paymentHash_opt = Some(paymentHash))
}
initialize()
@ -284,11 +280,24 @@ object MultiPartPaymentLifecycle {
def props(nodeParams: NodeParams, cfg: SendPaymentConfig, relayer: ActorRef, router: ActorRef, register: ActorRef) = Props(new MultiPartPaymentLifecycle(nodeParams, cfg, relayer, router, register))
case class SendMultiPartPayment(paymentHash: ByteVector32,
paymentSecret: ByteVector32,
/**
* Send a payment to a given node. The payment may be split into multiple child payments, for which a path-finding
* algorithm will run to find suitable payment routes.
*
* @param paymentSecret payment secret to protect against probing (usually from a Bolt 11 invoice).
* @param targetNodeId target node (may be the final recipient when using source-routing, or the first trampoline
* node when using trampoline).
* @param totalAmount total amount to send to the target node.
* @param targetExpiry expiry at the target node (CLTV for the target node's received HTLCs).
* @param maxAttempts maximum number of retries.
* @param assistedRoutes routing hints (usually from a Bolt 11 invoice).
* @param routeParams parameters to fine-tune the routing algorithm.
* @param additionalTlvs when provided, additional tlvs that will be added to the onion sent to the target node.
*/
case class SendMultiPartPayment(paymentSecret: ByteVector32,
targetNodeId: PublicKey,
totalAmount: MilliSatoshi,
finalExpiry: CltvExpiry,
targetExpiry: CltvExpiry,
maxAttempts: Int,
assistedRoutes: Seq[Seq[ExtraHop]] = Nil,
routeParams: Option[RouteParams] = None,
@ -400,9 +409,8 @@ object MultiPartPaymentLifecycle {
private def createChildPayment(nodeParams: NodeParams, request: SendMultiPartPayment, childAmount: MilliSatoshi, channel: OutgoingChannel): SendPayment = {
SendPayment(
request.paymentHash,
request.targetNodeId,
Onion.createMultiPartPayload(childAmount, request.totalAmount, request.finalExpiry, request.paymentSecret, request.additionalTlvs),
Onion.createMultiPartPayload(childAmount, request.totalAmount, request.targetExpiry, request.paymentSecret, request.additionalTlvs),
request.maxAttempts,
request.assistedRoutes,
request.routeParams,

View file

@ -27,7 +27,7 @@ import fr.acinq.eclair.payment.PaymentRequest.ExtraHop
import fr.acinq.eclair.payment._
import fr.acinq.eclair.payment.send.MultiPartPaymentLifecycle.SendMultiPartPayment
import fr.acinq.eclair.payment.send.PaymentLifecycle.{SendPayment, SendPaymentToRoute}
import fr.acinq.eclair.router.{NodeHop, RouteParams}
import fr.acinq.eclair.router.{ChannelHop, Hop, NodeHop, RouteParams}
import fr.acinq.eclair.wire.Onion.FinalLegacyPayload
import fr.acinq.eclair.wire.{Onion, OnionTlv, TrampolineExpiryTooSoon, TrampolineFeeInsufficient}
import fr.acinq.eclair.{CltvExpiryDelta, Features, LongToBtcAmount, MilliSatoshi, NodeParams, randomBytes32}
@ -45,7 +45,7 @@ class PaymentInitiator(nodeParams: NodeParams, router: ActorRef, relayer: ActorR
case r: SendPaymentRequest =>
val paymentId = UUID.randomUUID()
sender ! paymentId
val paymentCfg = SendPaymentConfig(paymentId, paymentId, r.externalId, r.paymentHash, r.targetNodeId, Upstream.Local(paymentId), r.paymentRequest, storeInDb = true, publishEvent = true)
val paymentCfg = SendPaymentConfig(paymentId, paymentId, r.externalId, r.paymentHash, r.recipientAmount, r.recipientNodeId, Upstream.Local(paymentId), r.paymentRequest, storeInDb = true, publishEvent = true, Nil)
val finalExpiry = r.finalExpiry(nodeParams.currentBlockHeight)
r.paymentRequest match {
case Some(invoice) if !invoice.features.supported =>
@ -53,8 +53,8 @@ class PaymentInitiator(nodeParams: NodeParams, router: ActorRef, relayer: ActorR
case Some(invoice) if invoice.features.allowMultiPart && Features.hasFeature(nodeParams.features, Features.BasicMultiPartPayment) =>
invoice.paymentSecret match {
case Some(paymentSecret) => r.predefinedRoute match {
case Nil => spawnMultiPartPaymentFsm(paymentCfg) forward SendMultiPartPayment(r.paymentHash, paymentSecret, r.targetNodeId, r.amount, finalExpiry, r.maxAttempts, r.assistedRoutes, r.routeParams)
case hops => spawnPaymentFsm(paymentCfg) forward SendPaymentToRoute(r.paymentHash, hops, Onion.createMultiPartPayload(r.amount, invoice.amount.getOrElse(r.amount), finalExpiry, paymentSecret))
case Nil => spawnMultiPartPaymentFsm(paymentCfg) forward SendMultiPartPayment(paymentSecret, r.recipientNodeId, r.recipientAmount, finalExpiry, r.maxAttempts, r.assistedRoutes, r.routeParams)
case hops => spawnPaymentFsm(paymentCfg) forward SendPaymentToRoute(hops, Onion.createMultiPartPayload(r.recipientAmount, invoice.amount.getOrElse(r.recipientAmount), finalExpiry, paymentSecret))
}
case None => sender ! PaymentFailed(paymentId, r.paymentHash, LocalFailure(InvalidInvoice("multi-part invoice is missing a payment secret")) :: Nil)
}
@ -62,8 +62,8 @@ class PaymentInitiator(nodeParams: NodeParams, router: ActorRef, relayer: ActorR
val payFsm = spawnPaymentFsm(paymentCfg)
// NB: we only generate legacy payment onions for now for maximum compatibility.
r.predefinedRoute match {
case Nil => payFsm forward SendPayment(r.paymentHash, r.targetNodeId, FinalLegacyPayload(r.amount, finalExpiry), r.maxAttempts, r.assistedRoutes, r.routeParams)
case hops => payFsm forward SendPaymentToRoute(r.paymentHash, hops, FinalLegacyPayload(r.amount, finalExpiry))
case Nil => payFsm forward SendPayment(r.recipientNodeId, FinalLegacyPayload(r.recipientAmount, finalExpiry), r.maxAttempts, r.assistedRoutes, r.routeParams)
case hops => payFsm forward SendPaymentToRoute(hops, FinalLegacyPayload(r.recipientAmount, finalExpiry))
}
}
@ -72,9 +72,9 @@ class PaymentInitiator(nodeParams: NodeParams, router: ActorRef, relayer: ActorR
sender ! paymentId
r.trampolineAttempts match {
case Nil =>
sender ! PaymentFailed(paymentId, r.paymentRequest.paymentHash, LocalFailure(TrampolineFeesMissing) :: Nil)
sender ! PaymentFailed(paymentId, r.paymentHash, LocalFailure(TrampolineFeesMissing) :: Nil)
case _ if !r.paymentRequest.features.allowTrampoline && r.paymentRequest.amount.isEmpty =>
sender ! PaymentFailed(paymentId, r.paymentRequest.paymentHash, LocalFailure(TrampolineLegacyAmountLessInvoice) :: Nil)
sender ! PaymentFailed(paymentId, r.paymentHash, LocalFailure(TrampolineLegacyAmountLessInvoice) :: Nil)
case (trampolineFees, trampolineExpiryDelta) :: remainingAttempts =>
log.info(s"sending trampoline payment with trampoline fees=$trampolineFees and expiry delta=$trampolineExpiryDelta")
sendTrampolinePayment(paymentId, r, trampolineFees, trampolineExpiryDelta)
@ -108,25 +108,25 @@ class PaymentInitiator(nodeParams: NodeParams, router: ActorRef, relayer: ActorR
def spawnMultiPartPaymentFsm(paymentCfg: SendPaymentConfig): ActorRef = context.actorOf(MultiPartPaymentLifecycle.props(nodeParams, paymentCfg, relayer, router, register))
private def sendTrampolinePayment(paymentId: UUID, r: SendTrampolinePaymentRequest, trampolineFees: MilliSatoshi, trampolineExpiryDelta: CltvExpiryDelta): Unit = {
val paymentCfg = SendPaymentConfig(paymentId, paymentId, None, r.paymentRequest.paymentHash, r.trampolineNodeId, Upstream.Local(paymentId), Some(r.paymentRequest), storeInDb = true, publishEvent = false, Some(r.copy(trampolineAttempts = Seq((trampolineFees, trampolineExpiryDelta)))))
val finalPayload = if (r.paymentRequest.features.allowMultiPart) {
Onion.createMultiPartPayload(r.finalAmount, r.finalAmount, r.finalExpiry(nodeParams.currentBlockHeight), r.paymentRequest.paymentSecret.get)
} else {
Onion.createSinglePartPayload(r.finalAmount, r.finalExpiry(nodeParams.currentBlockHeight), r.paymentRequest.paymentSecret)
}
val trampolineRoute = Seq(
NodeHop(nodeParams.nodeId, r.trampolineNodeId, nodeParams.expiryDeltaBlocks, 0 msat),
NodeHop(r.trampolineNodeId, r.paymentRequest.nodeId, trampolineExpiryDelta, trampolineFees) // for now we only use a single trampoline hop
NodeHop(r.trampolineNodeId, r.recipientNodeId, trampolineExpiryDelta, trampolineFees) // for now we only use a single trampoline hop
)
val paymentCfg = SendPaymentConfig(paymentId, paymentId, None, r.paymentHash, r.recipientAmount, r.recipientNodeId, Upstream.Local(paymentId), Some(r.paymentRequest), storeInDb = true, publishEvent = false, trampolineRoute.tail)
val finalPayload = if (r.paymentRequest.features.allowMultiPart) {
Onion.createMultiPartPayload(r.recipientAmount, r.recipientAmount, r.finalExpiry(nodeParams.currentBlockHeight), r.paymentRequest.paymentSecret.get)
} else {
Onion.createSinglePartPayload(r.recipientAmount, r.finalExpiry(nodeParams.currentBlockHeight), r.paymentRequest.paymentSecret)
}
// We assume that the trampoline node supports multi-part payments (it should).
val (trampolineAmount, trampolineExpiry, trampolineOnion) = if (r.paymentRequest.features.allowTrampoline) {
OutgoingPacket.buildPacket(Sphinx.TrampolinePacket)(r.paymentRequest.paymentHash, trampolineRoute, finalPayload)
OutgoingPacket.buildPacket(Sphinx.TrampolinePacket)(r.paymentHash, trampolineRoute, finalPayload)
} else {
OutgoingPacket.buildTrampolineToLegacyPacket(r.paymentRequest, trampolineRoute, finalPayload)
}
// We generate a random secret for this payment to avoid leaking the invoice secret to the first trampoline node.
val trampolineSecret = randomBytes32
spawnMultiPartPaymentFsm(paymentCfg) ! SendMultiPartPayment(r.paymentRequest.paymentHash, trampolineSecret, r.trampolineNodeId, trampolineAmount, trampolineExpiry, 1, r.paymentRequest.routingInfo, r.routeParams, Seq(OnionTlv.TrampolineOnion(trampolineOnion.packet)))
spawnMultiPartPaymentFsm(paymentCfg) ! SendMultiPartPayment(trampolineSecret, r.trampolineNodeId, trampolineAmount, trampolineExpiry, 1, r.paymentRequest.routingInfo, r.routeParams, Seq(OnionTlv.TrampolineOnion(trampolineOnion.packet)))
}
}
@ -142,7 +142,7 @@ object PaymentInitiator {
* Once we have trampoline fee estimation built into the router, the decision to use Trampoline or not should be done
* automatically by the router instead of the caller.
*
* @param finalAmount amount that should be received by the final recipient (usually from a Bolt 11 invoice).
* @param recipientAmount amount that should be received by the final recipient (usually from a Bolt 11 invoice).
* @param paymentRequest Bolt 11 invoice.
* @param trampolineNodeId id of the trampoline node.
* @param trampolineAttempts fees and expiry delta for the trampoline node. If this list contains multiple entries,
@ -152,19 +152,34 @@ object PaymentInitiator {
* @param finalExpiryDelta expiry delta for the final recipient.
* @param routeParams (optional) parameters to fine-tune the routing algorithm.
*/
case class SendTrampolinePaymentRequest(finalAmount: MilliSatoshi,
case class SendTrampolinePaymentRequest(recipientAmount: MilliSatoshi,
paymentRequest: PaymentRequest,
trampolineNodeId: PublicKey,
trampolineAttempts: Seq[(MilliSatoshi, CltvExpiryDelta)],
finalExpiryDelta: CltvExpiryDelta = Channel.MIN_CLTV_EXPIRY_DELTA,
routeParams: Option[RouteParams] = None) {
val recipientNodeId = paymentRequest.nodeId
val paymentHash = paymentRequest.paymentHash
// We add one block in order to not have our htlcs fail when a new block has just been found.
def finalExpiry(currentBlockHeight: Long) = finalExpiryDelta.toCltvExpiry(currentBlockHeight + 1)
}
case class SendPaymentRequest(amount: MilliSatoshi,
/**
* @param recipientAmount amount that should be received by the final recipient (usually from a Bolt 11 invoice).
* @param paymentHash payment hash.
* @param recipientNodeId id of the final recipient.
* @param maxAttempts maximum number of retries.
* @param finalExpiryDelta expiry delta for the final recipient.
* @param paymentRequest (optional) Bolt 11 invoice.
* @param externalId (optional) externally-controlled identifier (to reconcile between application DB and eclair DB).
* @param predefinedRoute (optional) route to use for the payment.
* @param assistedRoutes (optional) routing hints (usually from a Bolt 11 invoice).
* @param routeParams (optional) parameters to fine-tune the routing algorithm.
*/
case class SendPaymentRequest(recipientAmount: MilliSatoshi,
paymentHash: ByteVector32,
targetNodeId: PublicKey,
recipientNodeId: PublicKey,
maxAttempts: Int,
finalExpiryDelta: CltvExpiryDelta = Channel.MIN_CLTV_EXPIRY_DELTA,
paymentRequest: Option[PaymentRequest] = None,
@ -176,17 +191,40 @@ object PaymentInitiator {
def finalExpiry(currentBlockHeight: Long) = finalExpiryDelta.toCltvExpiry(currentBlockHeight + 1)
}
/**
* Configuration for an instance of a payment state machine.
*
* @param id id of the outgoing payment (mapped to a single outgoing HTLC).
* @param parentId id of the whole payment (if using multi-part, there will be N associated child payments,
* each with a different id).
* @param externalId externally-controlled identifier (to reconcile between application DB and eclair DB).
* @param paymentHash payment hash.
* @param recipientAmount amount that should be received by the final recipient (usually from a Bolt 11 invoice).
* @param recipientNodeId id of the final recipient.
* @param upstream information about the payment origin (to link upstream to downstream when relaying a payment).
* @param paymentRequest Bolt 11 invoice.
* @param storeInDb whether to store data in the payments DB (e.g. when we're relaying a trampoline payment, we
* don't want to store in the DB).
* @param publishEvent whether to publish a [[fr.acinq.eclair.payment.PaymentEvent]] on success/failure (e.g. for
* multi-part child payments, we don't want to emit events for each child, only for the whole payment).
* @param additionalHops additional hops that the payment state machine isn't aware of (e.g. when using trampoline, hops
* that occur after the first trampoline node).
*/
case class SendPaymentConfig(id: UUID,
parentId: UUID,
externalId: Option[String],
paymentHash: ByteVector32,
targetNodeId: PublicKey,
recipientAmount: MilliSatoshi,
recipientNodeId: PublicKey,
upstream: Upstream,
paymentRequest: Option[PaymentRequest],
storeInDb: Boolean, // e.g. for trampoline we don't want to store in the DB when we're relaying payments
publishEvent: Boolean,
// TODO: @t-bast: this is a very awkward work-around to get accurate data in the DB: fix this once we update the DB schema
trampolineData: Option[SendTrampolinePaymentRequest] = None)
additionalHops: Seq[NodeHop]) {
def fullRoute(hops: Seq[ChannelHop]): Seq[Hop] = hops ++ additionalHops
def createPaymentSent(preimage: ByteVector32, parts: Seq[PaymentSent.PartialPayment]) = PaymentSent(parentId, paymentHash, preimage, recipientAmount, recipientNodeId, parts)
}
// @formatter:off
case class InvalidInvoice(message: String) extends IllegalArgumentException(s"can't send payment: $message")

View file

@ -23,7 +23,7 @@ import fr.acinq.bitcoin.Crypto.PublicKey
import fr.acinq.eclair._
import fr.acinq.eclair.channel.{CMD_ADD_HTLC, Register}
import fr.acinq.eclair.crypto.{Sphinx, TransportHandler}
import fr.acinq.eclair.db.{OutgoingPayment, OutgoingPaymentStatus}
import fr.acinq.eclair.db.{OutgoingPayment, OutgoingPaymentStatus, PaymentType}
import fr.acinq.eclair.payment.PaymentRequest.ExtraHop
import fr.acinq.eclair.payment.PaymentSent.PartialPayment
import fr.acinq.eclair.payment._
@ -45,6 +45,7 @@ import scala.util.{Failure, Success}
class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: ActorRef, register: ActorRef) extends FSMDiagnosticActorLogging[PaymentLifecycle.State, PaymentLifecycle.Data] {
val id = cfg.id
val paymentHash = cfg.paymentHash
val paymentsDb = nodeParams.db.payments
private val span = Kamon.runWithContextEntry(MultiPartPaymentLifecycle.parentPaymentIdKey, cfg.parentId) {
@ -55,8 +56,9 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A
}
spanBuilder
.tag("paymentId", cfg.id.toString)
.tag("paymentHash", cfg.paymentHash.toHex)
.tag("targetNodeId", cfg.targetNodeId.toString())
.tag("paymentHash", paymentHash.toHex)
.tag("recipientNodeId", cfg.recipientNodeId.toString())
.tag("recipientAmount", cfg.recipientAmount.toLong)
.start()
}
@ -64,20 +66,20 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A
when(WAITING_FOR_REQUEST) {
case Event(c: SendPaymentToRoute, WaitingForRequest) =>
span.tag("targetNodeId", c.targetNodeId.toString())
span.tag("amount", c.finalPayload.amount.toLong)
span.tag("totalAmount", c.finalPayload.totalAmount.toLong)
span.tag("expiry", c.finalPayload.expiry.toLong)
log.debug("sending {} to route {}", c.finalPayload.amount, c.hops.mkString("->"))
val send = SendPayment(c.paymentHash, c.hops.last, c.finalPayload, maxAttempts = 1)
val send = SendPayment(c.hops.last, c.finalPayload, maxAttempts = 1)
router ! FinalizeRoute(c.hops)
if (cfg.storeInDb) {
val targetNodeId = cfg.trampolineData.map(_.paymentRequest.nodeId).getOrElse(cfg.targetNodeId)
val finalAmount = c.finalPayload.amount - cfg.trampolineData.map(_.trampolineAttempts.headOption.map(_._1).getOrElse(0 msat)).getOrElse(0 msat)
paymentsDb.addOutgoingPayment(OutgoingPayment(id, cfg.parentId, cfg.externalId, cfg.paymentHash, finalAmount, targetNodeId, Platform.currentTime, cfg.paymentRequest, OutgoingPaymentStatus.Pending))
paymentsDb.addOutgoingPayment(OutgoingPayment(id, cfg.parentId, cfg.externalId, paymentHash, PaymentType.Standard, c.finalPayload.amount, cfg.recipientAmount, cfg.recipientNodeId, Platform.currentTime, cfg.paymentRequest, OutgoingPaymentStatus.Pending))
}
goto(WAITING_FOR_ROUTE) using WaitingForRoute(sender, send, failures = Nil)
case Event(c: SendPayment, WaitingForRequest) =>
span.tag("targetNodeId", c.targetNodeId.toString())
span.tag("amount", c.finalPayload.amount.toLong)
span.tag("totalAmount", c.finalPayload.totalAmount.toLong)
span.tag("expiry", c.finalPayload.expiry.toLong)
@ -91,9 +93,7 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A
router ! RouteRequest(c.getRouteRequestStart(nodeParams), c.targetNodeId, c.finalPayload.amount, c.assistedRoutes, routeParams = c.routeParams, ignoreNodes = ignoredNodes)
}
if (cfg.storeInDb) {
val targetNodeId = cfg.trampolineData.map(_.paymentRequest.nodeId).getOrElse(cfg.targetNodeId)
val finalAmount = c.finalPayload.amount - cfg.trampolineData.map(_.trampolineAttempts.headOption.map(_._1).getOrElse(0 msat)).getOrElse(0 msat)
paymentsDb.addOutgoingPayment(OutgoingPayment(id, cfg.parentId, cfg.externalId, cfg.paymentHash, finalAmount, targetNodeId, Platform.currentTime, cfg.paymentRequest, OutgoingPaymentStatus.Pending))
paymentsDb.addOutgoingPayment(OutgoingPayment(id, cfg.parentId, cfg.externalId, paymentHash, PaymentType.Standard, c.finalPayload.amount, cfg.recipientAmount, cfg.recipientNodeId, Platform.currentTime, cfg.paymentRequest, OutgoingPaymentStatus.Pending))
}
goto(WAITING_FOR_ROUTE) using WaitingForRoute(sender, c, failures = Nil)
}
@ -103,12 +103,12 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A
val hops = c.routePrefix ++ routeHops
log.info(s"route found: attempt=${failures.size + 1}/${c.maxAttempts} route=${hops.map(_.nextNodeId).mkString("->")} channels=${hops.map(_.lastUpdate.shortChannelId).mkString("->")}")
val firstHop = hops.head
val (cmd, sharedSecrets) = OutgoingPacket.buildCommand(cfg.upstream, c.paymentHash, hops, c.finalPayload)
val (cmd, sharedSecrets) = OutgoingPacket.buildCommand(cfg.upstream, paymentHash, hops, c.finalPayload)
register ! Register.ForwardShortId(firstHop.lastUpdate.shortChannelId, cmd)
goto(WAITING_FOR_PAYMENT_COMPLETE) using WaitingForComplete(s, c, cmd, failures, sharedSecrets, ignoreNodes, ignoreChannels, hops)
case Event(Status.Failure(t), WaitingForRoute(s, c, failures)) =>
onFailure(s, PaymentFailed(id, c.paymentHash, failures :+ LocalFailure(t)))
case Event(Status.Failure(t), WaitingForRoute(s, _, failures)) =>
onFailure(s, PaymentFailed(id, paymentHash, failures :+ LocalFailure(t)))
myStop()
}
@ -116,9 +116,8 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A
case Event("ok", _) => stay
case Event(fulfill: UpdateFulfillHtlc, WaitingForComplete(s, c, cmd, _, _, _, _, route)) =>
val trampolineFees = cfg.trampolineData.map(_.trampolineAttempts.headOption.map(_._1).getOrElse(0 msat)).getOrElse(0 msat)
val p = PartialPayment(id, c.finalPayload.amount - trampolineFees, cmd.amount - c.finalPayload.amount + trampolineFees, fulfill.channelId, Some(route))
onSuccess(s, PaymentSent(id, c.paymentHash, fulfill.paymentPreimage, p :: Nil))
val p = PartialPayment(id, c.finalPayload.amount, cmd.amount - c.finalPayload.amount, fulfill.channelId, Some(cfg.fullRoute(route)))
onSuccess(s, cfg.createPaymentSent(fulfill.paymentPreimage, p :: Nil))
myStop()
case Event(fail: UpdateFailHtlc, WaitingForComplete(s, c, _, failures, sharedSecrets, ignoreNodes, ignoreChannels, hops)) =>
@ -126,20 +125,20 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A
case Success(e@Sphinx.DecryptedFailurePacket(nodeId, failureMessage)) if nodeId == c.targetNodeId =>
// 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)")
onFailure(s, PaymentFailed(id, c.paymentHash, failures :+ RemoteFailure(hops, e)))
onFailure(s, PaymentFailed(id, paymentHash, failures :+ RemoteFailure(cfg.fullRoute(hops), e)))
myStop()
case res if failures.size + 1 >= c.maxAttempts =>
// otherwise we never try more than maxAttempts, no matter the kind of error returned
val failure = res match {
case Success(e@Sphinx.DecryptedFailurePacket(nodeId, failureMessage)) =>
log.info(s"received an error message from nodeId=$nodeId (failure=$failureMessage)")
RemoteFailure(hops, e)
RemoteFailure(cfg.fullRoute(hops), e)
case Failure(t) =>
log.warning(s"cannot parse returned error: ${t.getMessage}")
UnreadableRemoteFailure(hops)
UnreadableRemoteFailure(cfg.fullRoute(hops))
}
log.warning(s"too many failed attempts, failing the payment")
onFailure(s, PaymentFailed(id, c.paymentHash, failures :+ failure))
onFailure(s, PaymentFailed(id, paymentHash, failures :+ failure))
myStop()
case Failure(t) =>
log.warning(s"cannot parse returned error: ${t.getMessage}")
@ -147,12 +146,12 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A
val blacklist = hops.map(_.nextNodeId).drop(1).dropRight(1)
log.warning(s"blacklisting intermediate nodes=${blacklist.mkString(",")}")
router ! RouteRequest(c.getRouteRequestStart(nodeParams), c.targetNodeId, c.finalPayload.amount, c.assistedRoutes, ignoreNodes ++ blacklist, ignoreChannels, c.routeParams)
goto(WAITING_FOR_ROUTE) using WaitingForRoute(s, c, failures :+ UnreadableRemoteFailure(hops))
goto(WAITING_FOR_ROUTE) using WaitingForRoute(s, c, failures :+ UnreadableRemoteFailure(cfg.fullRoute(hops)))
case Success(e@Sphinx.DecryptedFailurePacket(nodeId, failureMessage: Node)) =>
log.info(s"received 'Node' type error message from nodeId=$nodeId, trying to route around it (failure=$failureMessage)")
// let's try to route around this node
router ! RouteRequest(c.getRouteRequestStart(nodeParams), c.targetNodeId, c.finalPayload.amount, c.assistedRoutes, ignoreNodes + nodeId, ignoreChannels, c.routeParams)
goto(WAITING_FOR_ROUTE) using WaitingForRoute(s, c, failures :+ RemoteFailure(hops, e))
goto(WAITING_FOR_ROUTE) using WaitingForRoute(s, c, failures :+ RemoteFailure(cfg.fullRoute(hops), e))
case Success(e@Sphinx.DecryptedFailurePacket(nodeId, failureMessage: Update)) =>
log.info(s"received 'Update' type error message from nodeId=$nodeId, retrying payment (failure=$failureMessage)")
if (Announcements.checkSig(failureMessage.update, nodeId)) {
@ -194,13 +193,13 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A
log.warning(s"got bad signature from node=$nodeId update=${failureMessage.update}")
router ! RouteRequest(c.getRouteRequestStart(nodeParams), c.targetNodeId, c.finalPayload.amount, c.assistedRoutes, ignoreNodes + nodeId, ignoreChannels, c.routeParams)
}
goto(WAITING_FOR_ROUTE) using WaitingForRoute(s, c, failures :+ RemoteFailure(hops, e))
goto(WAITING_FOR_ROUTE) using WaitingForRoute(s, c, failures :+ RemoteFailure(cfg.fullRoute(hops), e))
case Success(e@Sphinx.DecryptedFailurePacket(nodeId, failureMessage)) =>
log.info(s"received an error message from nodeId=$nodeId, trying to use a different channel (failure=$failureMessage)")
// let's try again without the channel outgoing from nodeId
val faultyChannel = hops.find(_.nodeId == nodeId).map(hop => ChannelDesc(hop.lastUpdate.shortChannelId, hop.nodeId, hop.nextNodeId))
router ! RouteRequest(c.getRouteRequestStart(nodeParams), c.targetNodeId, c.finalPayload.amount, c.assistedRoutes, ignoreNodes, ignoreChannels ++ faultyChannel.toSet, c.routeParams)
goto(WAITING_FOR_ROUTE) using WaitingForRoute(s, c, failures :+ RemoteFailure(hops, e))
goto(WAITING_FOR_ROUTE) using WaitingForRoute(s, c, failures :+ RemoteFailure(cfg.fullRoute(hops), e))
}
case Event(fail: UpdateFailMalformedHtlc, _) =>
@ -215,7 +214,7 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A
// If the first hop was selected by the sender (in routePrefix) and it failed, it doesn't make sense to retry (we
// will end up retrying over that same faulty channel).
if (failures.size + 1 >= c.maxAttempts || c.routePrefix.nonEmpty) {
onFailure(s, PaymentFailed(id, c.paymentHash, failures :+ LocalFailure(t)))
onFailure(s, PaymentFailed(id, paymentHash, failures :+ LocalFailure(t)))
myStop()
} else {
log.info(s"received an error message from local, trying to use a different channel (failure=${t.getMessage})")
@ -236,7 +235,7 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A
// this means that previous state was WAITING_FOR_COMPLETE
d.failures.lastOption.foreach(failure => stateSpan.foreach(span => KamonExt.failSpan(span, failure)))
case d: WaitingForComplete =>
stateSpanBuilder.tag("route", s"${d.hops.map(_.nextNodeId).mkString("->")}")
stateSpanBuilder.tag("route", s"${cfg.fullRoute(d.hops).map(_.nextNodeId).mkString("->")}")
case _ => ()
}
stateSpan.foreach(_.finish())
@ -267,7 +266,7 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A
}
override def mdc(currentMessage: Any): MDC = {
Logs.mdc(category_opt = Some(Logs.LogCategory.PAYMENT), parentPaymentId_opt = Some(cfg.parentId), paymentId_opt = Some(id), paymentHash_opt = Some(cfg.paymentHash))
Logs.mdc(category_opt = Some(Logs.LogCategory.PAYMENT), parentPaymentId_opt = Some(cfg.parentId), paymentId_opt = Some(id), paymentHash_opt = Some(paymentHash))
}
initialize()
@ -280,26 +279,27 @@ object PaymentLifecycle {
/**
* Send a payment to a pre-defined route without running the path-finding algorithm.
*
* @param paymentHash payment hash.
* @param hops payment route to use.
* @param finalPayload payload for the target node.
* @param finalPayload onion payload for the target node.
*/
case class SendPaymentToRoute(paymentHash: ByteVector32, hops: Seq[PublicKey], finalPayload: FinalPayload)
case class SendPaymentToRoute(hops: Seq[PublicKey], finalPayload: FinalPayload) {
require(hops.nonEmpty, s"payment route must not be empty")
val targetNodeId = hops.last
}
/**
* Send a payment to a given node. A path-finding algorithm will run to find a suitable payment route.
*
* @param paymentHash payment hash.
* @param targetNodeId target node (payment recipient).
* @param finalPayload payload for the target node.
* @param targetNodeId target node (may be the final recipient when using source-routing, or the first trampoline
* node when using trampoline).
* @param finalPayload onion payload for the target node.
* @param maxAttempts maximum number of retries.
* @param assistedRoutes routing hints for the last part of the route (provided in the Bolt 11 invoice).
* @param routeParams parameters to tweak the path-finding algorithm.
* @param assistedRoutes routing hints (usually from a Bolt 11 invoice).
* @param routeParams parameters to fine-tune the routing algorithm.
* @param routePrefix when provided, the payment route will start with these hops. Path-finding will run only to
* find how to route from the last node of the route prefix to the target node.
*/
case class SendPayment(paymentHash: ByteVector32,
targetNodeId: PublicKey,
case class SendPayment(targetNodeId: PublicKey,
finalPayload: FinalPayload,
maxAttempts: Int,
assistedRoutes: Seq[Seq[ExtraHop]] = Nil,

View file

@ -128,7 +128,6 @@ case class ChannelHop(nodeId: PublicKey, nextNodeId: PublicKey, lastUpdate: Chan
* A directed hop between two trampoline nodes.
* These nodes need not be connected and we don't need to know a route between them.
* The start node will compute the route to the end node itself when it receives our payment.
* TODO: @t-bast: once the NodeUpdate message is implemented, we should use that instead of inline cltv and fee.
*
* @param nodeId id of the start node.
* @param nextNodeId id of the end node.

View file

@ -101,8 +101,8 @@ class EclairImplSpec extends TestKit(ActorSystem("test")) with fixture.FunSuiteL
eclair.send(None, nodeId, 123 msat, ByteVector32.Zeroes, invoice_opt = None)
val send = paymentInitiator.expectMsgType[SendPaymentRequest]
assert(send.externalId === None)
assert(send.targetNodeId === nodeId)
assert(send.amount === 123.msat)
assert(send.recipientNodeId === nodeId)
assert(send.recipientAmount === 123.msat)
assert(send.paymentHash === ByteVector32.Zeroes)
assert(send.paymentRequest === None)
assert(send.assistedRoutes === Seq.empty)
@ -114,8 +114,8 @@ class EclairImplSpec extends TestKit(ActorSystem("test")) with fixture.FunSuiteL
eclair.send(Some(externalId1), nodeId, 123 msat, ByteVector32.Zeroes, invoice_opt = Some(invoice1))
val send1 = paymentInitiator.expectMsgType[SendPaymentRequest]
assert(send1.externalId === Some(externalId1))
assert(send1.targetNodeId === nodeId)
assert(send1.amount === 123.msat)
assert(send1.recipientNodeId === nodeId)
assert(send1.recipientAmount === 123.msat)
assert(send1.paymentHash === ByteVector32.Zeroes)
assert(send1.paymentRequest === Some(invoice1))
assert(send1.assistedRoutes === hints)
@ -126,8 +126,8 @@ class EclairImplSpec extends TestKit(ActorSystem("test")) with fixture.FunSuiteL
eclair.send(Some(externalId2), nodeId, 123 msat, ByteVector32.Zeroes, invoice_opt = Some(invoice2))
val send2 = paymentInitiator.expectMsgType[SendPaymentRequest]
assert(send2.externalId === Some(externalId2))
assert(send2.targetNodeId === nodeId)
assert(send2.amount === 123.msat)
assert(send2.recipientNodeId === nodeId)
assert(send2.recipientAmount === 123.msat)
assert(send2.paymentHash === ByteVector32.Zeroes)
assert(send2.paymentRequest === Some(invoice2))
assert(send2.finalExpiryDelta === CltvExpiryDelta(96))
@ -136,8 +136,8 @@ class EclairImplSpec extends TestKit(ActorSystem("test")) with fixture.FunSuiteL
eclair.send(None, nodeId, 123 msat, ByteVector32.Zeroes, invoice_opt = None, feeThreshold_opt = Some(123 sat), maxFeePct_opt = Some(4.20))
val send3 = paymentInitiator.expectMsgType[SendPaymentRequest]
assert(send3.externalId === None)
assert(send3.targetNodeId === nodeId)
assert(send3.amount === 123.msat)
assert(send3.recipientNodeId === nodeId)
assert(send3.recipientAmount === 123.msat)
assert(send3.paymentHash === ByteVector32.Zeroes)
assert(send3.routeParams.get.maxFeeBase === 123000.msat) // conversion sat -> msat
assert(send3.routeParams.get.maxFeePct === 4.20)
@ -321,7 +321,7 @@ class EclairImplSpec extends TestKit(ActorSystem("test")) with fixture.FunSuiteL
val send = paymentInitiator.expectMsgType[SendPaymentRequest]
assert(send.externalId === Some("42"))
assert(send.predefinedRoute === route)
assert(send.amount === 1234.msat)
assert(send.recipientAmount === 1234.msat)
assert(send.finalExpiryDelta === CltvExpiryDelta(123))
assert(send.paymentHash === ByteVector32.One)
assert(send.paymentRequest === Some(pr))

View file

@ -61,15 +61,15 @@ class FeaturesSpec extends FunSuite {
bin"000000000000010000000000" -> false,
bin"000000000000100010000000" -> true,
bin"000000000000100001000000" -> true,
// payment_secret depends on var_onion_optin
bin"000000001000000000000000" -> false,
bin"000000000100000000000000" -> false,
// payment_secret depends on var_onion_optin, but we allow not setting it to be compatible with Phoenix
bin"000000001000000000000000" -> true,
bin"000000000100000000000000" -> true,
bin"000000000100001000000000" -> true,
// basic_mpp depends on payment_secret
bin"000000100000000000000000" -> false,
bin"000000010000000000000000" -> false,
bin"000000101000000000000000" -> false,
bin"000000011000000000000000" -> false,
bin"000000101000000000000000" -> true, // we allow not setting var_onion_optin
bin"000000011000000000000000" -> true, // we allow not setting var_onion_optin
bin"000000011000001000000000" -> true,
bin"000000100100000100000000" -> true
)

View file

@ -75,8 +75,10 @@ class StartupSpec extends FunSuite {
test("NodeParams should fail if features are inconsistent") {
val legalFeaturesConf = ConfigFactory.parseString("features = \"028a8a\"")
val illegalFeaturesConf = ConfigFactory.parseString("features = \"028000\"") // basic_mpp without var_onion_optin
val illegalButAllowedFeaturesConf = ConfigFactory.parseString("features = \"028000\"") // basic_mpp without var_onion_optin
val illegalFeaturesConf = ConfigFactory.parseString("features = \"020000\"") // basic_mpp without payment_secret
assert(Try(makeNodeParamsWithDefaults(legalFeaturesConf.withFallback(defaultConf))).isSuccess)
assert(Try(makeNodeParamsWithDefaults(illegalButAllowedFeaturesConf.withFallback(defaultConf))).isSuccess)
assert(Try(makeNodeParamsWithDefaults(illegalFeaturesConf.withFallback(defaultConf))).isFailure)
}

View file

@ -18,7 +18,8 @@ package fr.acinq.eclair.db
import java.util.UUID
import fr.acinq.bitcoin.Transaction
import fr.acinq.bitcoin.Crypto.PrivateKey
import fr.acinq.bitcoin.{ByteVector32, Transaction}
import fr.acinq.eclair._
import fr.acinq.eclair.channel.Channel.{LocalError, RemoteError}
import fr.acinq.eclair.channel.{AvailableBalanceChanged, ChannelErrorOccurred, NetworkFeePaid}
@ -26,10 +27,11 @@ import fr.acinq.eclair.db.sqlite.SqliteAuditDb
import fr.acinq.eclair.db.sqlite.SqliteUtils.{getVersion, using}
import fr.acinq.eclair.payment._
import fr.acinq.eclair.wire.{ChannelCodecs, ChannelCodecsSpec}
import org.scalatest.FunSuite
import org.scalatest.{FunSuite, Tag}
import scala.compat.Platform
import scala.concurrent.duration._
import scala.util.Random
class SqliteAuditDbSpec extends FunSuite {
@ -44,7 +46,7 @@ class SqliteAuditDbSpec extends FunSuite {
val sqlite = TestConstants.sqliteInMemory()
val db = new SqliteAuditDb(sqlite)
val e1 = PaymentSent(ChannelCodecs.UNKNOWN_UUID, randomBytes32, randomBytes32, PaymentSent.PartialPayment(ChannelCodecs.UNKNOWN_UUID, 42000 msat, 1000 msat, randomBytes32, None) :: Nil)
val e1 = PaymentSent(ChannelCodecs.UNKNOWN_UUID, randomBytes32, randomBytes32, 40000 msat, randomKey.publicKey, PaymentSent.PartialPayment(ChannelCodecs.UNKNOWN_UUID, 42000 msat, 1000 msat, randomBytes32, None) :: Nil)
val pp2a = PaymentReceived.PartialPayment(42000 msat, randomBytes32)
val pp2b = PaymentReceived.PartialPayment(42100 msat, randomBytes32)
val e2 = PaymentReceived(randomBytes32, pp2a :: pp2b :: Nil)
@ -52,16 +54,14 @@ class SqliteAuditDbSpec extends FunSuite {
val e4 = NetworkFeePaid(null, randomKey.publicKey, randomBytes32, Transaction(0, Seq.empty, Seq.empty, 0), 42 sat, "mutual")
val pp5a = PaymentSent.PartialPayment(UUID.randomUUID(), 42000 msat, 1000 msat, randomBytes32, None, timestamp = 0)
val pp5b = PaymentSent.PartialPayment(UUID.randomUUID(), 42100 msat, 900 msat, randomBytes32, None, timestamp = 1)
val e5 = PaymentSent(ChannelCodecs.UNKNOWN_UUID, randomBytes32, randomBytes32, pp5a :: pp5b :: Nil)
val e5 = PaymentSent(UUID.randomUUID(), randomBytes32, randomBytes32, 84100 msat, randomKey.publicKey, pp5a :: pp5b :: Nil)
val pp6 = PaymentSent.PartialPayment(UUID.randomUUID(), 42000 msat, 1000 msat, randomBytes32, None, timestamp = (Platform.currentTime.milliseconds + 10.minutes).toMillis)
val e6 = PaymentSent(ChannelCodecs.UNKNOWN_UUID, randomBytes32, randomBytes32, pp6 :: Nil)
val e6 = PaymentSent(UUID.randomUUID(), randomBytes32, randomBytes32, 42000 msat, randomKey.publicKey, pp6 :: Nil)
val e7 = AvailableBalanceChanged(null, randomBytes32, ShortChannelId(500000, 42, 1), 456123000 msat, ChannelCodecsSpec.normal.commitments)
val e8 = ChannelLifecycleEvent(randomBytes32, randomKey.publicKey, 456123000 sat, isFunder = true, isPrivate = false, "mutual")
val e9 = ChannelErrorOccurred(null, randomBytes32, randomKey.publicKey, null, LocalError(new RuntimeException("oops")), isFatal = true)
val e10 = ChannelErrorOccurred(null, randomBytes32, randomKey.publicKey, null, RemoteError(wire.Error(randomBytes32, "remote oops")), isFatal = true)
val e11 = TrampolinePaymentRelayed(42000 msat, 40000 msat, randomBytes32, randomKey.publicKey, Seq(randomBytes32), Seq(randomBytes32))
// TrampolinePaymentRelayed events are converted to ChannelPaymentRelayed events for now. We need to udpate the DB schema to fix this.
val e11bis = ChannelPaymentRelayed(42000 msat, 40000 msat, e11.paymentHash, e11.fromChannelIds.head, e11.toChannelIds.head, e11.timestamp)
val e11 = TrampolinePaymentRelayed(randomBytes32, Seq(PaymentRelayed.Part(20000 msat, randomBytes32), PaymentRelayed.Part(22000 msat, randomBytes32)), Seq(PaymentRelayed.Part(20000 msat, randomBytes32), PaymentRelayed.Part(10000 msat, randomBytes32), PaymentRelayed.Part(10000 msat, randomBytes32)))
db.add(e1)
db.add(e2)
@ -75,10 +75,10 @@ class SqliteAuditDbSpec extends FunSuite {
db.add(e10)
db.add(e11)
assert(db.listSent(from = 0L, to = (Platform.currentTime.milliseconds + 15.minute).toMillis).toSet === Set(e1, e5.copy(id = pp5a.id, parts = pp5a :: Nil), e5.copy(id = pp5b.id, parts = pp5b :: Nil), e6.copy(id = pp6.id)))
assert(db.listSent(from = 0L, to = (Platform.currentTime.milliseconds + 15.minute).toMillis).toSet === Set(e1, e5, e6))
assert(db.listSent(from = 100000L, to = (Platform.currentTime.milliseconds + 1.minute).toMillis).toList === List(e1))
assert(db.listReceived(from = 0L, to = (Platform.currentTime.milliseconds + 1.minute).toMillis).toList === List(e2.copy(parts = pp2a :: Nil), e2.copy(parts = pp2b :: Nil)))
assert(db.listRelayed(from = 0L, to = (Platform.currentTime.milliseconds + 1.minute).toMillis).toList === List(e3, e11bis))
assert(db.listReceived(from = 0L, to = (Platform.currentTime.milliseconds + 1.minute).toMillis).toList === List(e2))
assert(db.listRelayed(from = 0L, to = (Platform.currentTime.milliseconds + 1.minute).toMillis).toList === List(e3, e11))
assert(db.listNetworkFees(from = 0L, to = (Platform.currentTime.milliseconds + 1.minute).toMillis).size === 1)
assert(db.listNetworkFees(from = 0L, to = (Platform.currentTime.milliseconds + 1.minute).toMillis).head.txType === "mutual")
}
@ -90,30 +90,68 @@ class SqliteAuditDbSpec extends FunSuite {
val n1 = randomKey.publicKey
val n2 = randomKey.publicKey
val n3 = randomKey.publicKey
val n4 = randomKey.publicKey
val c1 = randomBytes32
val c2 = randomBytes32
val c3 = randomBytes32
val c4 = randomBytes32
db.add(ChannelPaymentRelayed(46000 msat, 44000 msat, randomBytes32, randomBytes32, c1))
db.add(ChannelPaymentRelayed(41000 msat, 40000 msat, randomBytes32, randomBytes32, c1))
db.add(ChannelPaymentRelayed(43000 msat, 42000 msat, randomBytes32, randomBytes32, c1))
db.add(ChannelPaymentRelayed(42000 msat, 40000 msat, randomBytes32, randomBytes32, c2))
db.add(TrampolinePaymentRelayed(randomBytes32, Seq(PaymentRelayed.Part(25000 msat, randomBytes32)), Seq(PaymentRelayed.Part(20000 msat, c4))))
db.add(TrampolinePaymentRelayed(randomBytes32, Seq(PaymentRelayed.Part(46000 msat, randomBytes32)), Seq(PaymentRelayed.Part(16000 msat, c2), PaymentRelayed.Part(10000 msat, c4), PaymentRelayed.Part(14000 msat, c4))))
db.add(NetworkFeePaid(null, n1, c1, Transaction(0, Seq.empty, Seq.empty, 0), 100 sat, "funding"))
db.add(NetworkFeePaid(null, n2, c2, Transaction(0, Seq.empty, Seq.empty, 0), 200 sat, "funding"))
db.add(NetworkFeePaid(null, n2, c2, Transaction(0, Seq.empty, Seq.empty, 0), 300 sat, "mutual"))
db.add(NetworkFeePaid(null, n3, c3, Transaction(0, Seq.empty, Seq.empty, 0), 400 sat, "funding"))
db.add(NetworkFeePaid(null, n4, c4, Transaction(0, Seq.empty, Seq.empty, 0), 500 sat, "funding"))
assert(db.stats.toSet === Set(
Stats(channelId = c1, avgPaymentAmount = 42 sat, paymentCount = 3, relayFee = 4 sat, networkFee = 100 sat),
Stats(channelId = c2, avgPaymentAmount = 40 sat, paymentCount = 1, relayFee = 2 sat, networkFee = 500 sat),
Stats(channelId = c3, avgPaymentAmount = 0 sat, paymentCount = 0, relayFee = 0 sat, networkFee = 400 sat)
Stats(channelId = c2, avgPaymentAmount = 40 sat, paymentCount = 2, relayFee = 4 sat, networkFee = 500 sat),
Stats(channelId = c3, avgPaymentAmount = 0 sat, paymentCount = 0, relayFee = 0 sat, networkFee = 400 sat),
Stats(channelId = c4, avgPaymentAmount = 30 sat, paymentCount = 2, relayFee = 9 sat, networkFee = 500 sat)
))
}
test("handle migration version 1 -> 3") {
ignore("relay stats performance", Tag("perf")) {
val sqlite = TestConstants.sqliteInMemory()
val db = new SqliteAuditDb(sqlite)
val nodeCount = 100
val channelCount = 1000
val eventCount = 100000
val nodeIds = (1 to nodeCount).map(_ => randomKey.publicKey)
val channelIds = (1 to channelCount).map(_ => randomBytes32)
// Fund channels.
channelIds.foreach(channelId => {
val nodeId = nodeIds(Random.nextInt(nodeCount))
db.add(NetworkFeePaid(null, nodeId, channelId, Transaction(0, Seq.empty, Seq.empty, 0), 100 sat, "funding"))
})
// Add relay events.
(1 to eventCount).foreach(_ => {
// 25% trampoline relays.
if (Random.nextInt(4) == 0) {
val outgoingCount = 1 + Random.nextInt(4)
val incoming = Seq(PaymentRelayed.Part(10000 msat, randomBytes32))
val outgoing = (1 to outgoingCount).map(_ => PaymentRelayed.Part(Random.nextInt(2000).msat, channelIds(Random.nextInt(channelCount))))
db.add(TrampolinePaymentRelayed(randomBytes32, incoming, outgoing))
} else {
val toChannelId = channelIds(Random.nextInt(channelCount))
db.add(ChannelPaymentRelayed(10000 msat, Random.nextInt(10000).msat, randomBytes32, randomBytes32, toChannelId))
}
})
// Test starts here.
val start = Platform.currentTime
assert(db.stats.nonEmpty)
val end = Platform.currentTime
fail(s"took ${end - start}ms")
}
test("handle migration version 1 -> 4") {
val connection = TestConstants.sqliteInMemory()
// simulate existing previous version db
@ -135,19 +173,19 @@ class SqliteAuditDbSpec extends FunSuite {
}
using(connection.createStatement()) { statement =>
assert(getVersion(statement, "audit", 3) == 1) // we expect version 1
assert(getVersion(statement, "audit", 4) == 1) // we expect version 1
}
val ps = PaymentSent(UUID.randomUUID(), randomBytes32, randomBytes32, PaymentSent.PartialPayment(UUID.randomUUID(), 42000 msat, 1000 msat, randomBytes32, None) :: Nil)
val ps = PaymentSent(UUID.randomUUID(), randomBytes32, randomBytes32, 42000 msat, PrivateKey(ByteVector32.One).publicKey, PaymentSent.PartialPayment(UUID.randomUUID(), 42000 msat, 1000 msat, randomBytes32, None) :: Nil)
val pp1 = PaymentSent.PartialPayment(UUID.randomUUID(), 42001 msat, 1001 msat, randomBytes32, None)
val pp2 = PaymentSent.PartialPayment(UUID.randomUUID(), 42002 msat, 1002 msat, randomBytes32, None)
val ps1 = PaymentSent(UUID.randomUUID(), randomBytes32, randomBytes32, pp1 :: pp2 :: Nil)
val ps1 = PaymentSent(UUID.randomUUID(), randomBytes32, randomBytes32, 84003 msat, PrivateKey(ByteVector32.One).publicKey, pp1 :: pp2 :: Nil)
val e1 = ChannelErrorOccurred(null, randomBytes32, randomKey.publicKey, null, LocalError(new RuntimeException("oops")), isFatal = true)
val e2 = ChannelErrorOccurred(null, randomBytes32, randomKey.publicKey, null, RemoteError(wire.Error(randomBytes32, "remote oops")), isFatal = true)
// add a row (no ID on sent)
using(connection.prepareStatement("INSERT INTO sent VALUES (?, ?, ?, ?, ?, ?)")) { statement =>
statement.setLong(1, ps.amount.toLong)
statement.setLong(1, ps.recipientAmount.toLong)
statement.setLong(2, ps.feesPaid.toLong)
statement.setBytes(3, ps.paymentHash.toArray)
statement.setBytes(4, ps.paymentPreimage.toArray)
@ -159,7 +197,7 @@ class SqliteAuditDbSpec extends FunSuite {
val migratedDb = new SqliteAuditDb(connection)
using(connection.createStatement()) { statement =>
assert(getVersion(statement, "audit", 3) == 3) // version changed from 1 -> 3
assert(getVersion(statement, "audit", 4) == 4) // version changed from 1 -> 4
}
// existing rows in the 'sent' table will use id=00000000-0000-0000-0000-000000000000 as default
@ -168,7 +206,7 @@ class SqliteAuditDbSpec extends FunSuite {
val postMigrationDb = new SqliteAuditDb(connection)
using(connection.createStatement()) { statement =>
assert(getVersion(statement, "audit", 3) == 3) // version 3
assert(getVersion(statement, "audit", 4) == 4) // version 4
}
postMigrationDb.add(ps1)
@ -176,14 +214,11 @@ class SqliteAuditDbSpec extends FunSuite {
postMigrationDb.add(e2)
// the old record will have the UNKNOWN_UUID but the new ones will have their actual id
assert(postMigrationDb.listSent(0, (Platform.currentTime.milliseconds + 1.minute).toMillis) === Seq(
ps.copy(id = ChannelCodecs.UNKNOWN_UUID, parts = Seq(ps.parts.head.copy(id = ChannelCodecs.UNKNOWN_UUID))),
ps1.copy(id = pp1.id, parts = pp1 :: Nil),
ps1.copy(id = pp2.id, parts = pp2 :: Nil)))
val expected = Seq(ps.copy(id = ChannelCodecs.UNKNOWN_UUID, parts = Seq(ps.parts.head.copy(id = ChannelCodecs.UNKNOWN_UUID))), ps1)
assert(postMigrationDb.listSent(0, (Platform.currentTime.milliseconds + 1.minute).toMillis) === expected)
}
test("handle migration version 2 -> 3") {
test("handle migration version 2 -> 4") {
val connection = TestConstants.sqliteInMemory()
// simulate existing previous version db
@ -205,7 +240,7 @@ class SqliteAuditDbSpec extends FunSuite {
}
using(connection.createStatement()) { statement =>
assert(getVersion(statement, "audit", 3) == 2) // version 2 is deployed now
assert(getVersion(statement, "audit", 4) == 2) // version 2 is deployed now
}
val e1 = ChannelErrorOccurred(null, randomBytes32, randomKey.publicKey, null, LocalError(new RuntimeException("oops")), isFatal = true)
@ -214,7 +249,7 @@ class SqliteAuditDbSpec extends FunSuite {
val migratedDb = new SqliteAuditDb(connection)
using(connection.createStatement()) { statement =>
assert(getVersion(statement, "audit", 3) == 3) // version changed from 2 -> 3
assert(getVersion(statement, "audit", 4) == 4) // version changed from 2 -> 4
}
migratedDb.add(e1)
@ -222,10 +257,142 @@ class SqliteAuditDbSpec extends FunSuite {
val postMigrationDb = new SqliteAuditDb(connection)
using(connection.createStatement()) { statement =>
assert(getVersion(statement, "audit", 3) == 3) // version 3
assert(getVersion(statement, "audit", 4) == 4) // version 4
}
postMigrationDb.add(e2)
}
test("handle migration version 3 -> 4") {
val connection = TestConstants.sqliteInMemory()
// simulate existing previous version db
using(connection.createStatement()) { statement =>
getVersion(statement, "audit", 3)
statement.executeUpdate("CREATE TABLE IF NOT EXISTS balance_updated (channel_id BLOB NOT NULL, node_id BLOB NOT NULL, amount_msat INTEGER NOT NULL, capacity_sat INTEGER NOT NULL, reserve_sat INTEGER NOT NULL, timestamp INTEGER NOT NULL)")
statement.executeUpdate("CREATE TABLE IF NOT EXISTS sent (amount_msat INTEGER NOT NULL, fees_msat INTEGER NOT NULL, payment_hash BLOB NOT NULL, payment_preimage BLOB NOT NULL, to_channel_id BLOB NOT NULL, timestamp INTEGER NOT NULL, id BLOB NOT NULL)")
statement.executeUpdate("CREATE TABLE IF NOT EXISTS received (amount_msat INTEGER NOT NULL, payment_hash BLOB NOT NULL, from_channel_id BLOB NOT NULL, timestamp INTEGER NOT NULL)")
statement.executeUpdate("CREATE TABLE IF NOT EXISTS relayed (amount_in_msat INTEGER NOT NULL, amount_out_msat INTEGER NOT NULL, payment_hash BLOB NOT NULL, from_channel_id BLOB NOT NULL, to_channel_id BLOB NOT NULL, timestamp INTEGER NOT NULL)")
statement.executeUpdate("CREATE TABLE IF NOT EXISTS network_fees (channel_id BLOB NOT NULL, node_id BLOB NOT NULL, tx_id BLOB NOT NULL, fee_sat INTEGER NOT NULL, tx_type TEXT NOT NULL, timestamp INTEGER NOT NULL)")
statement.executeUpdate("CREATE TABLE IF NOT EXISTS channel_events (channel_id BLOB NOT NULL, node_id BLOB NOT NULL, capacity_sat INTEGER NOT NULL, is_funder BOOLEAN NOT NULL, is_private BOOLEAN NOT NULL, event TEXT NOT NULL, timestamp INTEGER NOT NULL)")
statement.executeUpdate("CREATE TABLE IF NOT EXISTS channel_errors (channel_id BLOB NOT NULL, node_id BLOB NOT NULL, error_name TEXT NOT NULL, error_message TEXT NOT NULL, is_fatal INTEGER NOT NULL, timestamp INTEGER NOT NULL)")
statement.executeUpdate("CREATE INDEX IF NOT EXISTS balance_updated_idx ON balance_updated(timestamp)")
statement.executeUpdate("CREATE INDEX IF NOT EXISTS sent_timestamp_idx ON sent(timestamp)")
statement.executeUpdate("CREATE INDEX IF NOT EXISTS received_timestamp_idx ON received(timestamp)")
statement.executeUpdate("CREATE INDEX IF NOT EXISTS relayed_timestamp_idx ON relayed(timestamp)")
statement.executeUpdate("CREATE INDEX IF NOT EXISTS network_fees_timestamp_idx ON network_fees(timestamp)")
statement.executeUpdate("CREATE INDEX IF NOT EXISTS channel_events_timestamp_idx ON channel_events(timestamp)")
statement.executeUpdate("CREATE INDEX IF NOT EXISTS channel_errors_timestamp_idx ON channel_errors(timestamp)")
}
using(connection.createStatement()) { statement =>
assert(getVersion(statement, "audit", 4) == 3) // version 3 is deployed now
}
val pp1 = PaymentSent.PartialPayment(UUID.randomUUID(), 500 msat, 10 msat, randomBytes32, None, 100)
val pp2 = PaymentSent.PartialPayment(UUID.randomUUID(), 600 msat, 5 msat, randomBytes32, None, 110)
val ps1 = PaymentSent(UUID.randomUUID(), randomBytes32, randomBytes32, 1100 msat, PrivateKey(ByteVector32.One).publicKey, pp1 :: pp2 :: Nil)
for (pp <- Seq(pp1, pp2)) {
using(connection.prepareStatement("INSERT INTO sent (amount_msat, fees_msat, payment_hash, payment_preimage, to_channel_id, timestamp, id) VALUES (?, ?, ?, ?, ?, ?, ?)")) { statement =>
statement.setLong(1, pp.amount.toLong)
statement.setLong(2, pp.feesPaid.toLong)
statement.setBytes(3, ps1.paymentHash.toArray)
statement.setBytes(4, ps1.paymentPreimage.toArray)
statement.setBytes(5, pp.toChannelId.toArray)
statement.setLong(6, pp.timestamp)
statement.setBytes(7, pp.id.toString.getBytes)
statement.executeUpdate()
}
}
val relayed1 = ChannelPaymentRelayed(600 msat, 500 msat, randomBytes32, randomBytes32, randomBytes32, 105)
val relayed2 = ChannelPaymentRelayed(650 msat, 500 msat, randomBytes32, randomBytes32, randomBytes32, 115)
for (relayed <- Seq(relayed1, relayed2)) {
using(connection.prepareStatement("INSERT INTO relayed (amount_in_msat, amount_out_msat, payment_hash, from_channel_id, to_channel_id, timestamp) VALUES (?, ?, ?, ?, ?, ?)")) { statement =>
statement.setLong(1, relayed.amountIn.toLong)
statement.setLong(2, relayed.amountOut.toLong)
statement.setBytes(3, relayed.paymentHash.toArray)
statement.setBytes(4, relayed.fromChannelId.toArray)
statement.setBytes(5, relayed.toChannelId.toArray)
statement.setLong(6, relayed.timestamp)
statement.executeUpdate()
}
}
val migratedDb = new SqliteAuditDb(connection)
using(connection.createStatement()) { statement =>
assert(getVersion(statement, "audit", 4) == 4) // version changed from 3 -> 4
}
assert(migratedDb.listSent(50, 150).toSet === Set(
ps1.copy(id = pp1.id, recipientAmount = pp1.amount, parts = pp1 :: Nil),
ps1.copy(id = pp2.id, recipientAmount = pp2.amount, parts = pp2 :: Nil)
))
assert(migratedDb.listRelayed(100, 120) === Seq(relayed1, relayed2))
val postMigrationDb = new SqliteAuditDb(connection)
using(connection.createStatement()) { statement =>
assert(getVersion(statement, "audit", 4) == 4) // version 4
}
val ps2 = PaymentSent(UUID.randomUUID(), randomBytes32, randomBytes32, 1100 msat, randomKey.publicKey, Seq(
PaymentSent.PartialPayment(UUID.randomUUID(), 500 msat, 10 msat, randomBytes32, None, 160),
PaymentSent.PartialPayment(UUID.randomUUID(), 600 msat, 5 msat, randomBytes32, None, 165)
))
val relayed3 = TrampolinePaymentRelayed(randomBytes32, Seq(PaymentRelayed.Part(500 msat, randomBytes32), PaymentRelayed.Part(450 msat, randomBytes32)), Seq(PaymentRelayed.Part(800 msat, randomBytes32)), 150)
postMigrationDb.add(ps2)
assert(postMigrationDb.listSent(155, 200) === Seq(ps2))
postMigrationDb.add(relayed3)
assert(postMigrationDb.listRelayed(100, 160) === Seq(relayed1, relayed2, relayed3))
}
test("fails if the DB contains invalid values") {
val sqlite = TestConstants.sqliteInMemory()
val db = new SqliteAuditDb(sqlite)
using(sqlite.prepareStatement("INSERT INTO relayed (payment_hash, amount_msat, channel_id, direction, relay_type, timestamp) VALUES (?, ?, ?, ?, ?, ?)")) { statement =>
statement.setBytes(1, randomBytes32.toArray)
statement.setLong(2, 42)
statement.setBytes(3, randomBytes32.toArray)
statement.setString(4, "IN")
statement.setString(5, "unknown") // invalid relay type
statement.setLong(6, 10)
statement.executeUpdate()
}
assertThrows[MatchError](db.listRelayed(5, 15))
using(sqlite.prepareStatement("INSERT INTO relayed (payment_hash, amount_msat, channel_id, direction, relay_type, timestamp) VALUES (?, ?, ?, ?, ?, ?)")) { statement =>
statement.setBytes(1, randomBytes32.toArray)
statement.setLong(2, 51)
statement.setBytes(3, randomBytes32.toArray)
statement.setString(4, "UP") // invalid direction
statement.setString(5, "channel")
statement.setLong(6, 20)
statement.executeUpdate()
}
assertThrows[MatchError](db.listRelayed(15, 25))
val paymentHash = randomBytes32
val channelId = randomBytes32
using(sqlite.prepareStatement("INSERT INTO relayed (payment_hash, amount_msat, channel_id, direction, relay_type, timestamp) VALUES (?, ?, ?, ?, ?, ?)")) { statement =>
statement.setBytes(1, paymentHash.toArray)
statement.setLong(2, 65)
statement.setBytes(3, channelId.toArray)
statement.setString(4, "IN")
statement.setString(5, "channel")
statement.setLong(6, 30)
statement.executeUpdate()
}
assert(db.listRelayed(25, 35) === Seq(ChannelPaymentRelayed(65 msat, 0 msat, paymentHash, channelId, ByteVector32.Zeroes, 30)))
}
}

View file

@ -24,9 +24,9 @@ import fr.acinq.eclair.crypto.Sphinx
import fr.acinq.eclair.db.sqlite.SqlitePaymentsDb
import fr.acinq.eclair.db.sqlite.SqliteUtils._
import fr.acinq.eclair.payment._
import fr.acinq.eclair.router.ChannelHop
import fr.acinq.eclair.router.{ChannelHop, NodeHop}
import fr.acinq.eclair.wire.{ChannelUpdate, UnknownNextPeer}
import fr.acinq.eclair.{CltvExpiryDelta, LongToBtcAmount, ShortChannelId, TestConstants, db, randomBytes32, randomBytes64, randomKey}
import fr.acinq.eclair.{CltvExpiryDelta, LongToBtcAmount, ShortChannelId, TestConstants, randomBytes32, randomBytes64, randomKey}
import org.scalatest.FunSuite
import scala.compat.Platform
@ -42,7 +42,7 @@ class SqlitePaymentsDbSpec extends FunSuite {
val db2 = new SqlitePaymentsDb(sqlite)
}
test("handle version migration 1->3") {
test("handle version migration 1->4") {
val connection = TestConstants.sqliteInMemory()
using(connection.createStatement()) { statement =>
@ -67,16 +67,16 @@ class SqlitePaymentsDbSpec extends FunSuite {
val preMigrationDb = new SqlitePaymentsDb(connection)
using(connection.createStatement()) { statement =>
assert(getVersion(statement, "payments", 1) == 3) // version changed from 1 -> 3
assert(getVersion(statement, "payments", 1) == 4) // version changed from 1 -> 4
}
// the existing received payment can NOT be queried anymore
assert(preMigrationDb.getIncomingPayment(paymentHash1).isEmpty)
// add a few rows
val ps1 = OutgoingPayment(UUID.randomUUID(), UUID.randomUUID(), None, paymentHash1, 12345 msat, alice, 1000, None, OutgoingPaymentStatus.Pending)
val ps1 = OutgoingPayment(UUID.randomUUID(), UUID.randomUUID(), None, paymentHash1, PaymentType.Standard, 12345 msat, 12345 msat, alice, 1000, None, OutgoingPaymentStatus.Pending)
val i1 = PaymentRequest(Block.TestnetGenesisBlock.hash, Some(500 msat), paymentHash1, davePriv, "Some invoice", expirySeconds = None, timestamp = 1)
val pr1 = IncomingPayment(i1, preimage1, i1.timestamp.seconds.toMillis, IncomingPaymentStatus.Received(550 msat, 1100))
val pr1 = IncomingPayment(i1, preimage1, PaymentType.Standard, i1.timestamp.seconds.toMillis, IncomingPaymentStatus.Received(550 msat, 1100))
preMigrationDb.addOutgoingPayment(ps1)
preMigrationDb.addIncomingPayment(i1, preimage1)
@ -88,14 +88,14 @@ class SqlitePaymentsDbSpec extends FunSuite {
val postMigrationDb = new SqlitePaymentsDb(connection)
using(connection.createStatement()) { statement =>
assert(getVersion(statement, "payments", 3) == 3) // version still to 3
assert(getVersion(statement, "payments", 4) == 4) // version still to 4
}
assert(postMigrationDb.listIncomingPayments(1, 1500) === Seq(pr1))
assert(postMigrationDb.listOutgoingPayments(1, 1500) === Seq(ps1))
}
test("handle version migration 2->3") {
test("handle version migration 2->4") {
val connection = TestConstants.sqliteInMemory()
using(connection.createStatement()) { statement =>
@ -113,13 +113,13 @@ class SqlitePaymentsDbSpec extends FunSuite {
val id1 = UUID.randomUUID()
val id2 = UUID.randomUUID()
val id3 = UUID.randomUUID()
val ps1 = OutgoingPayment(id1, id1, None, randomBytes32, 561 msat, PrivateKey(ByteVector32.One).publicKey, 1000, None, OutgoingPaymentStatus.Pending)
val ps2 = OutgoingPayment(id2, id2, None, randomBytes32, 1105 msat, PrivateKey(ByteVector32.One).publicKey, 1010, None, OutgoingPaymentStatus.Failed(Nil, 1050))
val ps3 = OutgoingPayment(id3, id3, None, paymentHash1, 1729 msat, PrivateKey(ByteVector32.One).publicKey, 1040, None, OutgoingPaymentStatus.Succeeded(preimage1, 0 msat, Nil, 1060))
val ps1 = OutgoingPayment(id1, id1, None, randomBytes32, PaymentType.Standard, 561 msat, 561 msat, PrivateKey(ByteVector32.One).publicKey, 1000, None, OutgoingPaymentStatus.Pending)
val ps2 = OutgoingPayment(id2, id2, None, randomBytes32, PaymentType.Standard, 1105 msat, 1105 msat, PrivateKey(ByteVector32.One).publicKey, 1010, None, OutgoingPaymentStatus.Failed(Nil, 1050))
val ps3 = OutgoingPayment(id3, id3, None, paymentHash1, PaymentType.Standard, 1729 msat, 1729 msat, PrivateKey(ByteVector32.One).publicKey, 1040, None, OutgoingPaymentStatus.Succeeded(preimage1, 0 msat, Nil, 1060))
val i1 = PaymentRequest(Block.TestnetGenesisBlock.hash, Some(12345678 msat), paymentHash1, davePriv, "Some invoice", expirySeconds = None, timestamp = 1)
val pr1 = IncomingPayment(i1, preimage1, i1.timestamp.seconds.toMillis, IncomingPaymentStatus.Received(12345678 msat, 1090))
val pr1 = IncomingPayment(i1, preimage1, PaymentType.Standard, i1.timestamp.seconds.toMillis, IncomingPaymentStatus.Received(12345678 msat, 1090))
val i2 = PaymentRequest(Block.TestnetGenesisBlock.hash, Some(12345678 msat), paymentHash2, carolPriv, "Another invoice", expirySeconds = Some(30), timestamp = 1)
val pr2 = IncomingPayment(i2, preimage2, i2.timestamp.seconds.toMillis, IncomingPaymentStatus.Expired)
val pr2 = IncomingPayment(i2, preimage2, PaymentType.Standard, i2.timestamp.seconds.toMillis, IncomingPaymentStatus.Expired)
// Changes between version 2 and 3 to sent_payments:
// - removed the status column
@ -185,7 +185,7 @@ class SqlitePaymentsDbSpec extends FunSuite {
val preMigrationDb = new SqlitePaymentsDb(connection)
using(connection.createStatement()) { statement =>
assert(getVersion(statement, "payments", 2) == 3) // version changed from 2 -> 3
assert(getVersion(statement, "payments", 2) == 4) // version changed from 2 -> 4
}
assert(preMigrationDb.getIncomingPayment(i1.paymentHash) === Some(pr1))
@ -195,19 +195,19 @@ class SqlitePaymentsDbSpec extends FunSuite {
val postMigrationDb = new SqlitePaymentsDb(connection)
using(connection.createStatement()) { statement =>
assert(getVersion(statement, "payments", 3) == 3) // version still to 3
assert(getVersion(statement, "payments", 4) == 4) // version still to 4
}
val i3 = PaymentRequest(Block.TestnetGenesisBlock.hash, Some(561 msat), paymentHash3, alicePriv, "invoice #3", expirySeconds = Some(30))
val pr3 = IncomingPayment(i3, preimage3, i3.timestamp.seconds.toMillis, IncomingPaymentStatus.Pending)
val pr3 = IncomingPayment(i3, preimage3, PaymentType.Standard, i3.timestamp.seconds.toMillis, IncomingPaymentStatus.Pending)
postMigrationDb.addIncomingPayment(i3, pr3.paymentPreimage)
val ps4 = OutgoingPayment(UUID.randomUUID(), UUID.randomUUID(), Some("1"), randomBytes32, 123 msat, alice, 1100, Some(i3), OutgoingPaymentStatus.Pending)
val ps5 = OutgoingPayment(UUID.randomUUID(), UUID.randomUUID(), Some("2"), randomBytes32, 456 msat, bob, 1150, Some(i2), OutgoingPaymentStatus.Succeeded(preimage1, 42 msat, Nil, 1180))
val ps6 = OutgoingPayment(UUID.randomUUID(), UUID.randomUUID(), Some("3"), randomBytes32, 789 msat, bob, 1250, None, OutgoingPaymentStatus.Failed(Nil, 1300))
val ps4 = OutgoingPayment(UUID.randomUUID(), UUID.randomUUID(), Some("1"), randomBytes32, PaymentType.Standard, 123 msat, 123 msat, alice, 1100, Some(i3), OutgoingPaymentStatus.Pending)
val ps5 = OutgoingPayment(UUID.randomUUID(), UUID.randomUUID(), Some("2"), randomBytes32, PaymentType.Standard, 456 msat, 456 msat, bob, 1150, Some(i2), OutgoingPaymentStatus.Succeeded(preimage1, 42 msat, Nil, 1180))
val ps6 = OutgoingPayment(UUID.randomUUID(), UUID.randomUUID(), Some("3"), randomBytes32, PaymentType.Standard, 789 msat, 789 msat, bob, 1250, None, OutgoingPaymentStatus.Failed(Nil, 1300))
postMigrationDb.addOutgoingPayment(ps4)
postMigrationDb.addOutgoingPayment(ps5.copy(status = OutgoingPaymentStatus.Pending))
postMigrationDb.updateOutgoingPayment(PaymentSent(ps5.parentId, ps5.paymentHash, preimage1, Seq(PaymentSent.PartialPayment(ps5.id, ps5.amount, 42 msat, randomBytes32, None, 1180))))
postMigrationDb.updateOutgoingPayment(PaymentSent(ps5.parentId, ps5.paymentHash, preimage1, ps5.amount, ps5.recipientNodeId, Seq(PaymentSent.PartialPayment(ps5.id, ps5.amount, 42 msat, randomBytes32, None, 1180))))
postMigrationDb.addOutgoingPayment(ps6.copy(status = OutgoingPaymentStatus.Pending))
postMigrationDb.updateOutgoingPayment(PaymentFailed(ps6.id, ps6.paymentHash, Nil, 1300))
@ -216,6 +216,99 @@ class SqlitePaymentsDbSpec extends FunSuite {
assert(postMigrationDb.listExpiredIncomingPayments(1, 2000) === Seq(pr2))
}
test("handle version migration 3->4") {
val connection = TestConstants.sqliteInMemory()
using(connection.createStatement()) { statement =>
getVersion(statement, "payments", 3)
statement.executeUpdate("CREATE TABLE IF NOT EXISTS received_payments (payment_hash BLOB NOT NULL PRIMARY KEY, payment_preimage BLOB NOT NULL, payment_request TEXT NOT NULL, received_msat INTEGER, created_at INTEGER NOT NULL, expire_at INTEGER NOT NULL, received_at INTEGER)")
statement.executeUpdate("CREATE TABLE IF NOT EXISTS sent_payments (id TEXT NOT NULL PRIMARY KEY, parent_id TEXT NOT NULL, external_id TEXT, payment_hash BLOB NOT NULL, amount_msat INTEGER NOT NULL, target_node_id BLOB NOT NULL, created_at INTEGER NOT NULL, payment_request TEXT, completed_at INTEGER, payment_preimage BLOB, fees_msat INTEGER, payment_route BLOB, failures BLOB)")
statement.executeUpdate("CREATE INDEX IF NOT EXISTS sent_parent_id_idx ON sent_payments(parent_id)")
statement.executeUpdate("CREATE INDEX IF NOT EXISTS sent_payment_hash_idx ON sent_payments(payment_hash)")
statement.executeUpdate("CREATE INDEX IF NOT EXISTS sent_created_idx ON sent_payments(created_at)")
statement.executeUpdate("CREATE INDEX IF NOT EXISTS received_created_idx ON received_payments(created_at)")
}
using(connection.createStatement()) { statement =>
assert(getVersion(statement, "payments", 3) == 3) // version 3 is deployed now
}
// Insert a bunch of old version 3 rows.
val (id1, id2, id3) = (UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID())
val parentId = UUID.randomUUID()
val invoice1 = PaymentRequest(Block.TestnetGenesisBlock.hash, Some(2834 msat), paymentHash1, bobPriv, "invoice #1", expirySeconds = Some(30))
val ps1 = OutgoingPayment(id1, id1, Some("42"), randomBytes32, PaymentType.Standard, 561 msat, 561 msat, alice, 1000, None, OutgoingPaymentStatus.Failed(Seq(FailureSummary(FailureType.REMOTE, "no candy for you", List(HopSummary(hop_ab), HopSummary(hop_bc)))), 1020))
val ps2 = OutgoingPayment(id2, parentId, Some("42"), paymentHash1, PaymentType.Standard, 1105 msat, 1105 msat, bob, 1010, Some(invoice1), OutgoingPaymentStatus.Pending)
val ps3 = OutgoingPayment(id3, parentId, None, paymentHash1, PaymentType.Standard, 1729 msat, 1729 msat, bob, 1040, None, OutgoingPaymentStatus.Succeeded(preimage1, 10 msat, Seq(HopSummary(hop_ab), HopSummary(hop_bc)), 1060))
using(connection.prepareStatement("INSERT INTO sent_payments (id, parent_id, external_id, payment_hash, amount_msat, target_node_id, created_at, completed_at, failures) VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?)")) { statement =>
statement.setString(1, ps1.id.toString)
statement.setString(2, ps1.parentId.toString)
statement.setString(3, ps1.externalId.get.toString)
statement.setBytes(4, ps1.paymentHash.toArray)
statement.setLong(5, ps1.amount.toLong)
statement.setBytes(6, ps1.recipientNodeId.value.toArray)
statement.setLong(7, ps1.createdAt)
statement.setLong(8, ps1.status.asInstanceOf[OutgoingPaymentStatus.Failed].completedAt)
statement.setBytes(9, SqlitePaymentsDb.paymentFailuresCodec.encode(ps1.status.asInstanceOf[OutgoingPaymentStatus.Failed].failures.toList).require.toByteArray)
statement.executeUpdate()
}
using(connection.prepareStatement("INSERT INTO sent_payments (id, parent_id, external_id, payment_hash, amount_msat, target_node_id, created_at, payment_request) VALUES (?, ?, ?, ?, ?, ?, ?, ?)")) { statement =>
statement.setString(1, ps2.id.toString)
statement.setString(2, ps2.parentId.toString)
statement.setString(3, ps2.externalId.get.toString)
statement.setBytes(4, ps2.paymentHash.toArray)
statement.setLong(5, ps2.amount.toLong)
statement.setBytes(6, ps2.recipientNodeId.value.toArray)
statement.setLong(7, ps2.createdAt)
statement.setString(8, PaymentRequest.write(invoice1))
statement.executeUpdate()
}
using(connection.prepareStatement("INSERT INTO sent_payments (id, parent_id, payment_hash, amount_msat, target_node_id, created_at, completed_at, payment_preimage, fees_msat, payment_route) VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?)")) { statement =>
statement.setString(1, ps3.id.toString)
statement.setString(2, ps3.parentId.toString)
statement.setBytes(3, ps3.paymentHash.toArray)
statement.setLong(4, ps3.amount.toLong)
statement.setBytes(5, ps3.recipientNodeId.value.toArray)
statement.setLong(6, ps3.createdAt)
statement.setLong(7, ps3.status.asInstanceOf[OutgoingPaymentStatus.Succeeded].completedAt)
statement.setBytes(8, ps3.status.asInstanceOf[OutgoingPaymentStatus.Succeeded].paymentPreimage.toArray)
statement.setLong(9, ps3.status.asInstanceOf[OutgoingPaymentStatus.Succeeded].feesPaid.toLong)
statement.setBytes(10, SqlitePaymentsDb.paymentRouteCodec.encode(ps3.status.asInstanceOf[OutgoingPaymentStatus.Succeeded].route.toList).require.toByteArray)
statement.executeUpdate()
}
// Changes between version 3 and 4 to sent_payments:
// - added final amount column
// - added payment type column, with a default to "Standard"
// - renamed target_node_id -> recipient_node_id
// - re-ordered columns
val preMigrationDb = new SqlitePaymentsDb(connection)
using(connection.createStatement()) { statement =>
assert(getVersion(statement, "payments", 3) == 4) // version changed from 3 -> 4
}
assert(preMigrationDb.getOutgoingPayment(id1) === Some(ps1))
assert(preMigrationDb.listOutgoingPayments(parentId) === Seq(ps2, ps3))
val postMigrationDb = new SqlitePaymentsDb(connection)
using(connection.createStatement()) { statement =>
assert(getVersion(statement, "payments", 4) == 4) // version still to 4
}
val ps4 = OutgoingPayment(UUID.randomUUID(), UUID.randomUUID(), None, randomBytes32, PaymentType.SwapOut, 50 msat, 100 msat, carol, 1100, Some(invoice1), OutgoingPaymentStatus.Pending)
postMigrationDb.addOutgoingPayment(ps4)
postMigrationDb.updateOutgoingPayment(PaymentSent(parentId, paymentHash1, preimage1, ps2.recipientAmount, ps2.recipientNodeId, Seq(PaymentSent.PartialPayment(id2, ps2.amount, 15 msat, randomBytes32, Some(Seq(hop_ab)), 1105))))
assert(postMigrationDb.listOutgoingPayments(1, 2000) === Seq(ps1, ps2.copy(status = OutgoingPaymentStatus.Succeeded(preimage1, 15 msat, Seq(HopSummary(hop_ab)), 1105)), ps3, ps4))
}
test("add/retrieve/update incoming payments") {
val sqlite = TestConstants.sqliteInMemory()
val db = new SqlitePaymentsDb(sqlite)
@ -225,23 +318,23 @@ class SqlitePaymentsDbSpec extends FunSuite {
val expiredInvoice1 = PaymentRequest(Block.TestnetGenesisBlock.hash, Some(561 msat), randomBytes32, alicePriv, "invoice #1", timestamp = 1)
val expiredInvoice2 = PaymentRequest(Block.TestnetGenesisBlock.hash, Some(1105 msat), randomBytes32, bobPriv, "invoice #2", timestamp = 2, expirySeconds = Some(30))
val expiredPayment1 = IncomingPayment(expiredInvoice1, randomBytes32, expiredInvoice1.timestamp.seconds.toMillis, IncomingPaymentStatus.Expired)
val expiredPayment2 = IncomingPayment(expiredInvoice2, randomBytes32, expiredInvoice2.timestamp.seconds.toMillis, IncomingPaymentStatus.Expired)
val expiredPayment1 = IncomingPayment(expiredInvoice1, randomBytes32, PaymentType.Standard, expiredInvoice1.timestamp.seconds.toMillis, IncomingPaymentStatus.Expired)
val expiredPayment2 = IncomingPayment(expiredInvoice2, randomBytes32, PaymentType.Standard, expiredInvoice2.timestamp.seconds.toMillis, IncomingPaymentStatus.Expired)
val pendingInvoice1 = PaymentRequest(Block.TestnetGenesisBlock.hash, Some(561 msat), randomBytes32, alicePriv, "invoice #3")
val pendingInvoice2 = PaymentRequest(Block.TestnetGenesisBlock.hash, Some(1105 msat), randomBytes32, bobPriv, "invoice #4", expirySeconds = Some(30))
val pendingPayment1 = IncomingPayment(pendingInvoice1, randomBytes32, pendingInvoice1.timestamp.seconds.toMillis, IncomingPaymentStatus.Pending)
val pendingPayment2 = IncomingPayment(pendingInvoice2, randomBytes32, pendingInvoice2.timestamp.seconds.toMillis, IncomingPaymentStatus.Pending)
val pendingPayment1 = IncomingPayment(pendingInvoice1, randomBytes32, PaymentType.Standard, pendingInvoice1.timestamp.seconds.toMillis, IncomingPaymentStatus.Pending)
val pendingPayment2 = IncomingPayment(pendingInvoice2, randomBytes32, PaymentType.SwapIn, pendingInvoice2.timestamp.seconds.toMillis, IncomingPaymentStatus.Pending)
val paidInvoice1 = PaymentRequest(Block.TestnetGenesisBlock.hash, Some(561 msat), randomBytes32, alicePriv, "invoice #5")
val paidInvoice2 = PaymentRequest(Block.TestnetGenesisBlock.hash, Some(1105 msat), randomBytes32, bobPriv, "invoice #6", expirySeconds = Some(60))
val receivedAt1 = Platform.currentTime + 1
val receivedAt2 = Platform.currentTime + 2
val payment1 = IncomingPayment(paidInvoice1, randomBytes32, paidInvoice1.timestamp.seconds.toMillis, IncomingPaymentStatus.Received(561 msat, receivedAt2))
val payment2 = IncomingPayment(paidInvoice2, randomBytes32, paidInvoice2.timestamp.seconds.toMillis, IncomingPaymentStatus.Received(1111 msat, receivedAt2))
val payment1 = IncomingPayment(paidInvoice1, randomBytes32, PaymentType.Standard, paidInvoice1.timestamp.seconds.toMillis, IncomingPaymentStatus.Received(561 msat, receivedAt2))
val payment2 = IncomingPayment(paidInvoice2, randomBytes32, PaymentType.Standard, paidInvoice2.timestamp.seconds.toMillis, IncomingPaymentStatus.Received(1111 msat, receivedAt2))
db.addIncomingPayment(pendingInvoice1, pendingPayment1.paymentPreimage)
db.addIncomingPayment(pendingInvoice2, pendingPayment2.paymentPreimage)
db.addIncomingPayment(pendingInvoice2, pendingPayment2.paymentPreimage, PaymentType.SwapIn)
db.addIncomingPayment(expiredInvoice1, expiredPayment1.paymentPreimage)
db.addIncomingPayment(expiredInvoice2, expiredPayment2.paymentPreimage)
db.addIncomingPayment(paidInvoice1, payment1.paymentPreimage)
@ -273,8 +366,8 @@ class SqlitePaymentsDbSpec extends FunSuite {
val parentId = UUID.randomUUID()
val i1 = PaymentRequest(Block.TestnetGenesisBlock.hash, Some(123 msat), paymentHash1, davePriv, "Some invoice", expirySeconds = None, timestamp = 0)
val s1 = OutgoingPayment(UUID.randomUUID(), parentId, None, paymentHash1, 123 msat, alice, 100, Some(i1), OutgoingPaymentStatus.Pending)
val s2 = OutgoingPayment(UUID.randomUUID(), parentId, Some("1"), paymentHash1, 456 msat, bob, 200, None, OutgoingPaymentStatus.Pending)
val s1 = OutgoingPayment(UUID.randomUUID(), parentId, None, paymentHash1, PaymentType.Standard, 123 msat, 600 msat, dave, 100, Some(i1), OutgoingPaymentStatus.Pending)
val s2 = OutgoingPayment(UUID.randomUUID(), parentId, Some("1"), paymentHash1, PaymentType.SwapOut, 456 msat, 600 msat, dave, 200, None, OutgoingPaymentStatus.Pending)
assert(db.listOutgoingPayments(0, Platform.currentTime).isEmpty)
db.addOutgoingPayment(s1)
@ -294,7 +387,7 @@ class SqlitePaymentsDbSpec extends FunSuite {
assert(db.listOutgoingPayments(ByteVector32.Zeroes) === Nil)
val s3 = s2.copy(id = UUID.randomUUID(), amount = 789 msat, createdAt = 300)
val s4 = s2.copy(id = UUID.randomUUID(), createdAt = 300)
val s4 = s2.copy(id = UUID.randomUUID(), paymentType = PaymentType.Standard, createdAt = 300)
db.addOutgoingPayment(s3)
db.addOutgoingPayment(s4)
@ -302,18 +395,18 @@ class SqlitePaymentsDbSpec extends FunSuite {
val ss3 = s3.copy(status = OutgoingPaymentStatus.Failed(Nil, 310))
assert(db.getOutgoingPayment(s3.id) === Some(ss3))
db.updateOutgoingPayment(PaymentFailed(s4.id, s4.paymentHash, Seq(LocalFailure(new RuntimeException("woops")), RemoteFailure(Seq(hop_ab, hop_bc), Sphinx.DecryptedFailurePacket(carol, UnknownNextPeer))), 320))
val ss4 = s4.copy(status = OutgoingPaymentStatus.Failed(Seq(FailureSummary(FailureType.LOCAL, "woops", Nil), FailureSummary(FailureType.REMOTE, "processing node does not know the next peer in the route", List(HopSummary(alice, bob, Some(ShortChannelId(42))), HopSummary(bob, carol, Some(ShortChannelId(43)))))), 320))
val ss4 = s4.copy(status = OutgoingPaymentStatus.Failed(Seq(FailureSummary(FailureType.LOCAL, "woops", Nil), 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)))), 320))
assert(db.getOutgoingPayment(s4.id) === Some(ss4))
// can't update again once it's in a final state
assertThrows[IllegalArgumentException](db.updateOutgoingPayment(PaymentSent(parentId, s3.paymentHash, preimage1, Seq(PaymentSent.PartialPayment(s3.id, s3.amount, 42 msat, randomBytes32, None)))))
assertThrows[IllegalArgumentException](db.updateOutgoingPayment(PaymentSent(parentId, s3.paymentHash, preimage1, s3.recipientAmount, s3.recipientNodeId, Seq(PaymentSent.PartialPayment(s3.id, s3.amount, 42 msat, randomBytes32, None)))))
val paymentSent = PaymentSent(parentId, paymentHash1, preimage1, Seq(
val paymentSent = PaymentSent(parentId, paymentHash1, preimage1, 600 msat, carol, Seq(
PaymentSent.PartialPayment(s1.id, s1.amount, 15 msat, randomBytes32, None, 400),
PaymentSent.PartialPayment(s2.id, s2.amount, 20 msat, randomBytes32, Some(Seq(hop_ab, hop_bc)), 410)
))
val ss1 = s1.copy(status = OutgoingPaymentStatus.Succeeded(preimage1, 15 msat, Nil, 400))
val ss2 = s2.copy(status = OutgoingPaymentStatus.Succeeded(preimage1, 20 msat, Seq(HopSummary(alice, bob, Some(ShortChannelId(42))), HopSummary(bob, carol, Some(ShortChannelId(43)))), 410))
val ss2 = s2.copy(status = OutgoingPaymentStatus.Succeeded(preimage1, 20 msat, Seq(HopSummary(alice, bob, Some(ShortChannelId(42))), HopSummary(bob, carol, None)), 410))
db.updateOutgoingPayment(paymentSent)
assert(db.getOutgoingPayment(s1.id) === Some(ss1))
assert(db.getOutgoingPayment(s2.id) === Some(ss2))
@ -328,15 +421,15 @@ class SqlitePaymentsDbSpec extends FunSuite {
// -- feed db with incoming payments
val expiredInvoice = PaymentRequest(Block.TestnetGenesisBlock.hash, Some(123 msat), randomBytes32, alicePriv, "incoming #1", timestamp = 1)
val expiredPayment = IncomingPayment(expiredInvoice, randomBytes32, 100, IncomingPaymentStatus.Expired)
val expiredPayment = IncomingPayment(expiredInvoice, randomBytes32, PaymentType.Standard, 100, IncomingPaymentStatus.Expired)
val pendingInvoice = PaymentRequest(Block.TestnetGenesisBlock.hash, Some(456 msat), randomBytes32, alicePriv, "incoming #2")
val pendingPayment = IncomingPayment(pendingInvoice, randomBytes32, 120, IncomingPaymentStatus.Pending)
val pendingPayment = IncomingPayment(pendingInvoice, randomBytes32, PaymentType.Standard, 120, IncomingPaymentStatus.Pending)
val paidInvoice1 = PaymentRequest(Block.TestnetGenesisBlock.hash, Some(789 msat), randomBytes32, alicePriv, "incoming #3")
val receivedAt1 = 150
val receivedPayment1 = IncomingPayment(paidInvoice1, randomBytes32, 130, IncomingPaymentStatus.Received(561 msat, receivedAt1))
val receivedPayment1 = IncomingPayment(paidInvoice1, randomBytes32, PaymentType.Standard, 130, IncomingPaymentStatus.Received(561 msat, receivedAt1))
val paidInvoice2 = PaymentRequest(Block.TestnetGenesisBlock.hash, Some(888 msat), randomBytes32, alicePriv, "incoming #4")
val receivedAt2 = 160
val receivedPayment2 = IncomingPayment(paidInvoice2, randomBytes32, paidInvoice2.timestamp.seconds.toMillis, IncomingPaymentStatus.Received(889 msat, receivedAt2))
val receivedPayment2 = IncomingPayment(paidInvoice2, randomBytes32, PaymentType.Standard, paidInvoice2.timestamp.seconds.toMillis, IncomingPaymentStatus.Received(889 msat, receivedAt2))
db.addIncomingPayment(pendingInvoice, pendingPayment.paymentPreimage)
db.addIncomingPayment(expiredInvoice, expiredPayment.paymentPreimage)
db.addIncomingPayment(paidInvoice1, receivedPayment1.paymentPreimage)
@ -350,11 +443,11 @@ class SqlitePaymentsDbSpec extends FunSuite {
val invoice = PaymentRequest(Block.TestnetGenesisBlock.hash, Some(1337 msat), paymentHash1, davePriv, "outgoing #1", expirySeconds = None, timestamp = 0)
// 1st attempt, pending -> failed
val outgoing1 = OutgoingPayment(UUID.randomUUID(), parentId1, None, paymentHash1, 123 msat, alice, 200, Some(invoice), OutgoingPaymentStatus.Pending)
val outgoing1 = OutgoingPayment(UUID.randomUUID(), parentId1, None, paymentHash1, PaymentType.Standard, 123 msat, 123 msat, alice, 200, Some(invoice), OutgoingPaymentStatus.Pending)
db.addOutgoingPayment(outgoing1)
db.updateOutgoingPayment(PaymentFailed(outgoing1.id, outgoing1.paymentHash, Nil, 210))
// 2nd attempt: pending
val outgoing2 = OutgoingPayment(UUID.randomUUID(), parentId1, None, paymentHash1, 123 msat, alice, 211, Some(invoice), OutgoingPaymentStatus.Pending)
val outgoing2 = OutgoingPayment(UUID.randomUUID(), parentId1, None, paymentHash1, PaymentType.Standard, 123 msat, 123 msat, alice, 211, Some(invoice), OutgoingPaymentStatus.Pending)
db.addOutgoingPayment(outgoing2)
// -- 1st check: result contains 2 incoming PAID, 1 outgoing PENDING. Outgoing1 must not be overridden by Outgoing2
@ -366,12 +459,12 @@ class SqlitePaymentsDbSpec extends FunSuite {
// failed #2 and add a successful payment (made of 2 partial payments)
db.updateOutgoingPayment(PaymentFailed(outgoing2.id, outgoing2.paymentHash, Nil, 250))
val outgoing3 = OutgoingPayment(UUID.randomUUID(), parentId2, None, paymentHash1, 200 msat, bob, 300, Some(invoice), OutgoingPaymentStatus.Pending)
val outgoing4 = OutgoingPayment(UUID.randomUUID(), parentId2, None, paymentHash1, 300 msat, bob, 310, Some(invoice), OutgoingPaymentStatus.Pending)
val outgoing3 = OutgoingPayment(UUID.randomUUID(), parentId2, None, paymentHash1, PaymentType.Standard, 200 msat, 500 msat, bob, 300, Some(invoice), OutgoingPaymentStatus.Pending)
val outgoing4 = OutgoingPayment(UUID.randomUUID(), parentId2, None, paymentHash1, PaymentType.Standard, 300 msat, 500 msat, bob, 310, Some(invoice), OutgoingPaymentStatus.Pending)
db.addOutgoingPayment(outgoing3)
db.addOutgoingPayment(outgoing4)
// complete #2 and #3 partial payments
val sent = PaymentSent(parentId2, paymentHash1, preimage1, Seq(
val sent = PaymentSent(parentId2, paymentHash1, preimage1, outgoing3.recipientAmount, outgoing3.recipientNodeId, Seq(
PaymentSent.PartialPayment(outgoing3.id, outgoing3.amount, 15 msat, randomBytes32, None, 400),
PaymentSent.PartialPayment(outgoing4.id, outgoing4.amount, 20 msat, randomBytes32, None, 410)
))
@ -403,7 +496,7 @@ object SqlitePaymentsDbSpec {
val (alicePriv, bobPriv, carolPriv, davePriv) = (randomKey, randomKey, randomKey, randomKey)
val (alice, bob, carol, dave) = (alicePriv.publicKey, bobPriv.publicKey, carolPriv.publicKey, davePriv.publicKey)
val hop_ab = ChannelHop(alice, bob, ChannelUpdate(randomBytes64, randomBytes32, ShortChannelId(42), 1, 0, 0, CltvExpiryDelta(12), 1 msat, 1 msat, 1, None))
val hop_bc = ChannelHop(bob, carol, ChannelUpdate(randomBytes64, randomBytes32, ShortChannelId(43), 1, 0, 0, CltvExpiryDelta(12), 1 msat, 1 msat, 1, None))
val hop_bc = NodeHop(bob, carol, CltvExpiryDelta(14), 1 msat)
val (preimage1, preimage2, preimage3, preimage4) = (randomBytes32, randomBytes32, randomBytes32, randomBytes32)
val (paymentHash1, paymentHash2, paymentHash3, paymentHash4) = (Crypto.sha256(preimage1), Crypto.sha256(preimage2), Crypto.sha256(preimage3), Crypto.sha256(preimage4))
}

View file

@ -32,7 +32,7 @@ import fr.acinq.eclair.channel.Channel.{BroadcastChannelUpdate, PeriodicRefresh}
import fr.acinq.eclair.channel.Register.{Forward, ForwardShortId}
import fr.acinq.eclair.channel._
import fr.acinq.eclair.crypto.Sphinx.DecryptedFailurePacket
import fr.acinq.eclair.db.{IncomingPayment, IncomingPaymentStatus, OutgoingPaymentStatus}
import fr.acinq.eclair.db._
import fr.acinq.eclair.io.Peer
import fr.acinq.eclair.io.Peer.{Disconnect, PeerRoutingMessage}
import fr.acinq.eclair.payment.PaymentRequest.ExtraHop
@ -434,13 +434,14 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
awaitCond({
sender.expectMsgType[PaymentEvent](10 seconds) match {
case PaymentFailed(_, _, failures, _) => failures == Seq.empty // if something went wrong fail with a hint
case PaymentSent(_, _, _, part :: Nil) => part.route.get.exists(_.nodeId == nodes("G").nodeParams.nodeId)
case PaymentSent(_, _, _, _, _, part :: Nil) => part.route.getOrElse(Nil).exists(_.nodeId == nodes("G").nodeParams.nodeId)
case _ => false
}
}, max = 30 seconds, interval = 10 seconds)
}
test("send a multi-part payment B->D") {
val start = Platform.currentTime
val sender = TestProbe()
val amount = 1000000000L.msat
sender.send(nodes("D").paymentHandler, ReceivePayment(Some(amount), "split the restaurant bill"))
@ -453,7 +454,8 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
assert(paymentSent.id === paymentId)
assert(paymentSent.paymentHash === pr.paymentHash)
assert(paymentSent.parts.length > 1)
assert(paymentSent.amount === amount)
assert(paymentSent.recipientNodeId === nodes("D").nodeParams.nodeId)
assert(paymentSent.recipientAmount === amount)
assert(paymentSent.feesPaid > 0.msat)
assert(paymentSent.parts.forall(p => p.id != paymentSent.id))
assert(paymentSent.parts.forall(p => p.route.isDefined))
@ -465,8 +467,11 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
assert(paymentParts.forall(p => p.parentId != p.id))
assert(paymentParts.forall(p => p.status.asInstanceOf[OutgoingPaymentStatus.Succeeded].feesPaid > 0.msat))
awaitCond(nodes("B").nodeParams.db.audit.listSent(start, Platform.currentTime).nonEmpty)
assert(nodes("B").nodeParams.db.audit.listSent(start, Platform.currentTime) === Seq(paymentSent.copy(parts = paymentSent.parts.map(_.copy(route = None)))))
awaitCond(nodes("D").nodeParams.db.payments.getIncomingPayment(pr.paymentHash).exists(_.status.isInstanceOf[IncomingPaymentStatus.Received]))
val Some(IncomingPayment(_, _, _, IncomingPaymentStatus.Received(receivedAmount, _))) = nodes("D").nodeParams.db.payments.getIncomingPayment(pr.paymentHash)
val Some(IncomingPayment(_, _, _, _, IncomingPaymentStatus.Received(receivedAmount, _))) = nodes("D").nodeParams.db.payments.getIncomingPayment(pr.paymentHash)
assert(receivedAmount === amount)
}
@ -513,7 +518,7 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
assert(paymentSent.id === paymentId)
assert(paymentSent.paymentHash === pr.paymentHash)
assert(paymentSent.parts.length > 1)
assert(paymentSent.amount === amount)
assert(paymentSent.recipientAmount === amount)
assert(paymentSent.feesPaid === 0.msat) // no fees when using direct channels
val paymentParts = nodes("D").nodeParams.db.payments.listOutgoingPayments(paymentId).filter(_.status.isInstanceOf[OutgoingPaymentStatus.Succeeded])
@ -523,7 +528,7 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
assert(paymentParts.forall(p => p.status.asInstanceOf[OutgoingPaymentStatus.Succeeded].feesPaid == 0.msat))
awaitCond(nodes("C").nodeParams.db.payments.getIncomingPayment(pr.paymentHash).exists(_.status.isInstanceOf[IncomingPaymentStatus.Received]))
val Some(IncomingPayment(_, _, _, IncomingPaymentStatus.Received(receivedAmount, _))) = nodes("C").nodeParams.db.payments.getIncomingPayment(pr.paymentHash)
val Some(IncomingPayment(_, _, _, _, IncomingPaymentStatus.Received(receivedAmount, _))) = nodes("C").nodeParams.db.payments.getIncomingPayment(pr.paymentHash)
assert(receivedAmount === amount)
}
@ -570,22 +575,26 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
val paymentSent = sender.expectMsgType[PaymentSent](30 seconds)
assert(paymentSent.id === paymentId)
assert(paymentSent.paymentHash === pr.paymentHash)
assert(paymentSent.amount === amount)
assert(paymentSent.recipientNodeId === nodes("F3").nodeParams.nodeId)
assert(paymentSent.recipientAmount === amount)
assert(paymentSent.feesPaid === 1000000.msat)
assert(paymentSent.nonTrampolineFees === 0.msat)
awaitCond(nodes("F3").nodeParams.db.payments.getIncomingPayment(pr.paymentHash).exists(_.status.isInstanceOf[IncomingPaymentStatus.Received]))
val Some(IncomingPayment(_, _, _, IncomingPaymentStatus.Received(receivedAmount, _))) = nodes("F3").nodeParams.db.payments.getIncomingPayment(pr.paymentHash)
val Some(IncomingPayment(_, _, _, _, IncomingPaymentStatus.Received(receivedAmount, _))) = nodes("F3").nodeParams.db.payments.getIncomingPayment(pr.paymentHash)
assert(receivedAmount === amount)
awaitCond(nodes("G").nodeParams.db.audit.listRelayed(start, Platform.currentTime).exists(_.paymentHash == pr.paymentHash))
val relayed = nodes("G").nodeParams.db.audit.listRelayed(start, Platform.currentTime).filter(_.paymentHash == pr.paymentHash).head
assert(relayed.amountIn - relayed.amountOut === 1000000.msat)
assert(relayed.amountIn - relayed.amountOut > 0.msat)
assert(relayed.amountIn - relayed.amountOut < 1000000.msat)
// TODO: @t-bast: validate trampoline route data once implemented
val outgoingSuccess = nodes("B").nodeParams.db.payments.listOutgoingPayments(paymentId).filter(p => p.status.isInstanceOf[OutgoingPaymentStatus.Succeeded])
assert(outgoingSuccess.forall(p => p.targetNodeId == nodes("F3").nodeParams.nodeId))
assert(outgoingSuccess.map(_.amount).sum === amount)
assert(outgoingSuccess.map(_.status.asInstanceOf[OutgoingPaymentStatus.Succeeded].feesPaid).sum === 1000000.msat)
outgoingSuccess.foreach { case OutgoingPayment(_, _, _, _, _, _, _, recipientNodeId, _, _, OutgoingPaymentStatus.Succeeded(_, _, route, _)) =>
assert(recipientNodeId === nodes("F3").nodeParams.nodeId)
assert(route.lastOption === Some(HopSummary(nodes("G").nodeParams.nodeId, nodes("F3").nodeParams.nodeId)))
}
assert(outgoingSuccess.map(_.amount).sum === amount + 1000000.msat)
}
test("send a trampoline payment D->B (via trampoline C)") {
@ -603,22 +612,28 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
val paymentSent = sender.expectMsgType[PaymentSent](30 seconds)
assert(paymentSent.id === paymentId)
assert(paymentSent.paymentHash === pr.paymentHash)
assert(paymentSent.amount === amount)
assert(paymentSent.recipientAmount === amount)
assert(paymentSent.feesPaid === 300000.msat)
assert(paymentSent.nonTrampolineFees === 0.msat)
awaitCond(nodes("B").nodeParams.db.payments.getIncomingPayment(pr.paymentHash).exists(_.status.isInstanceOf[IncomingPaymentStatus.Received]))
val Some(IncomingPayment(_, _, _, IncomingPaymentStatus.Received(receivedAmount, _))) = nodes("B").nodeParams.db.payments.getIncomingPayment(pr.paymentHash)
val Some(IncomingPayment(_, _, _, _, IncomingPaymentStatus.Received(receivedAmount, _))) = nodes("B").nodeParams.db.payments.getIncomingPayment(pr.paymentHash)
assert(receivedAmount === amount)
awaitCond(nodes("C").nodeParams.db.audit.listRelayed(start, Platform.currentTime).exists(_.paymentHash == pr.paymentHash))
val relayed = nodes("C").nodeParams.db.audit.listRelayed(start, Platform.currentTime).filter(_.paymentHash == pr.paymentHash).head
assert(relayed.amountIn - relayed.amountOut === 300000.msat)
assert(relayed.amountIn - relayed.amountOut > 0.msat)
assert(relayed.amountIn - relayed.amountOut < 300000.msat)
// TODO: @t-bast: validate trampoline route data once implemented
val outgoingSuccess = nodes("D").nodeParams.db.payments.listOutgoingPayments(paymentId).filter(p => p.status.isInstanceOf[OutgoingPaymentStatus.Succeeded])
assert(outgoingSuccess.forall(p => p.targetNodeId == nodes("B").nodeParams.nodeId))
assert(outgoingSuccess.map(_.amount).sum === amount)
assert(outgoingSuccess.map(_.status.asInstanceOf[OutgoingPaymentStatus.Succeeded].feesPaid).sum === 300000.msat)
outgoingSuccess.foreach { case OutgoingPayment(_, _, _, _, _, _, _, recipientNodeId, _, _, OutgoingPaymentStatus.Succeeded(_, _, route, _)) =>
assert(recipientNodeId === nodes("B").nodeParams.nodeId)
assert(route.lastOption === Some(HopSummary(nodes("C").nodeParams.nodeId, nodes("B").nodeParams.nodeId)))
}
assert(outgoingSuccess.map(_.amount).sum === amount + 300000.msat)
awaitCond(nodes("D").nodeParams.db.audit.listSent(start, Platform.currentTime).nonEmpty)
assert(nodes("D").nodeParams.db.audit.listSent(start, Platform.currentTime) === Seq(paymentSent.copy(parts = paymentSent.parts.map(_.copy(route = None)))))
}
test("send a trampoline payment F3->A (via trampoline C, non-trampoline recipient)") {
@ -641,22 +656,24 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
val paymentSent = sender.expectMsgType[PaymentSent](30 seconds)
assert(paymentSent.id === paymentId)
assert(paymentSent.paymentHash === pr.paymentHash)
assert(paymentSent.amount === amount)
assert(paymentSent.feesPaid === 1000000.msat)
assert(paymentSent.recipientAmount === amount)
assert(paymentSent.trampolineFees === 1000000.msat)
awaitCond(nodes("A").nodeParams.db.payments.getIncomingPayment(pr.paymentHash).exists(_.status.isInstanceOf[IncomingPaymentStatus.Received]))
val Some(IncomingPayment(_, _, _, IncomingPaymentStatus.Received(receivedAmount, _))) = nodes("A").nodeParams.db.payments.getIncomingPayment(pr.paymentHash)
val Some(IncomingPayment(_, _, _, _, IncomingPaymentStatus.Received(receivedAmount, _))) = nodes("A").nodeParams.db.payments.getIncomingPayment(pr.paymentHash)
assert(receivedAmount === amount)
awaitCond(nodes("C").nodeParams.db.audit.listRelayed(start, Platform.currentTime).exists(_.paymentHash == pr.paymentHash))
val relayed = nodes("C").nodeParams.db.audit.listRelayed(start, Platform.currentTime).filter(_.paymentHash == pr.paymentHash).head
assert(relayed.amountIn - relayed.amountOut === 1000000.msat)
assert(relayed.amountIn - relayed.amountOut > 0.msat)
assert(relayed.amountIn - relayed.amountOut < 1000000.msat)
// TODO: @t-bast: validate trampoline route data once implemented
val outgoingSuccess = nodes("F3").nodeParams.db.payments.listOutgoingPayments(paymentId).filter(p => p.status.isInstanceOf[OutgoingPaymentStatus.Succeeded])
assert(outgoingSuccess.forall(p => p.targetNodeId == nodes("A").nodeParams.nodeId))
assert(outgoingSuccess.map(_.amount).sum === amount)
assert(outgoingSuccess.map(_.status.asInstanceOf[OutgoingPaymentStatus.Succeeded].feesPaid).sum === 1000000.msat)
outgoingSuccess.foreach { case OutgoingPayment(_, _, _, _, _, _, _, recipientNodeId, _, _, OutgoingPaymentStatus.Succeeded(_, _, route, _)) =>
assert(recipientNodeId === nodes("A").nodeParams.nodeId)
assert(route.lastOption === Some(HopSummary(nodes("C").nodeParams.nodeId, nodes("A").nodeParams.nodeId)))
}
assert(outgoingSuccess.map(_.amount).sum === amount + 1000000.msat)
}
test("send a trampoline payment B->D (temporary local failure at trampoline)") {

View file

@ -20,8 +20,7 @@ import java.util.UUID
import akka.actor.{ActorRef, ActorSystem}
import akka.testkit.{TestFSMRef, TestKit, TestProbe}
import fr.acinq.bitcoin.Crypto.PrivateKey
import fr.acinq.bitcoin.{Block, Crypto, Satoshi}
import fr.acinq.bitcoin.{Block, Crypto, DeterministicWallet, Satoshi, Transaction}
import fr.acinq.eclair.TestConstants.TestFeeEstimator
import fr.acinq.eclair._
import fr.acinq.eclair.blockchain.fee.FeeratesPerKw
@ -60,12 +59,12 @@ class MultiPartPaymentLifecycleSpec extends TestKit(ActorSystem("test")) with fi
override def withFixture(test: OneArgTest): Outcome = {
val id = UUID.randomUUID()
val cfg = SendPaymentConfig(id, id, Some("42"), paymentHash, b, Upstream.Local(id), None, storeInDb = true, publishEvent = true)
val cfg = SendPaymentConfig(id, id, Some("42"), paymentHash, finalAmount, finalRecipient, Upstream.Local(id), None, storeInDb = true, publishEvent = true, Nil)
val nodeParams = TestConstants.Alice.nodeParams
nodeParams.onChainFeeConf.feeEstimator.asInstanceOf[TestFeeEstimator].setFeerate(FeeratesPerKw.single(500))
val (childPayFsm, router, relayer, sender, eventListener) = (TestProbe(), TestProbe(), TestProbe(), TestProbe(), TestProbe())
class TestMultiPartPaymentLifecycle extends MultiPartPaymentLifecycle(nodeParams, cfg, relayer.ref, router.ref, TestProbe().ref) {
override def spawnChildPaymentFsm(childId: UUID, includeTrampolineFees: Boolean): ActorRef = childPayFsm.ref
override def spawnChildPaymentFsm(childId: UUID): ActorRef = childPayFsm.ref
}
val paymentHandler = TestFSMRef(new TestMultiPartPaymentLifecycle().asInstanceOf[MultiPartPaymentLifecycle])
system.eventStream.subscribe(eventListener.ref, classOf[PaymentEvent])
@ -91,7 +90,7 @@ class MultiPartPaymentLifecycleSpec extends TestKit(ActorSystem("test")) with fi
import f._
assert(payFsm.stateName === WAIT_FOR_PAYMENT_REQUEST)
val payment = SendMultiPartPayment(paymentHash, randomBytes32, b, 1500 * 1000 msat, expiry, 1)
val payment = SendMultiPartPayment(randomBytes32, b, 1500 * 1000 msat, expiry, 1)
sender.send(payFsm, payment)
router.expectMsg(GetNetworkStats)
assert(payFsm.stateName === WAIT_FOR_NETWORK_STATS)
@ -105,7 +104,7 @@ class MultiPartPaymentLifecycleSpec extends TestKit(ActorSystem("test")) with fi
import f._
assert(payFsm.stateName === WAIT_FOR_PAYMENT_REQUEST)
val payment = SendMultiPartPayment(paymentHash, randomBytes32, b, 2500 * 1000 msat, expiry, 1)
val payment = SendMultiPartPayment(randomBytes32, b, 2500 * 1000 msat, expiry, 1)
sender.send(payFsm, payment)
router.expectMsg(GetNetworkStats)
assert(payFsm.stateName === WAIT_FOR_NETWORK_STATS)
@ -126,7 +125,7 @@ class MultiPartPaymentLifecycleSpec extends TestKit(ActorSystem("test")) with fi
test("send to peer node via multiple channels") { f =>
import f._
val payment = SendMultiPartPayment(paymentHash, randomBytes32, b, 2000 * 1000 msat, expiry, 3)
val payment = SendMultiPartPayment(randomBytes32, b, 2000 * 1000 msat, expiry, 3)
// When sending to a peer node, we should not filter out unannounced channels.
val channels = OutgoingChannels(Seq(
OutgoingChannel(c, channelUpdate_ac_2, makeCommitments(1000 * 1000 msat, 0)),
@ -140,8 +139,8 @@ class MultiPartPaymentLifecycleSpec extends TestKit(ActorSystem("test")) with fi
// The payment should be split in two, using direct channels with b.
// MaxAttempts should be set to 1 when using direct channels to the destination.
childPayFsm.expectMsgAllOf(
SendPayment(paymentHash, b, Onion.createMultiPartPayload(1000 * 1000 msat, payment.totalAmount, expiry, payment.paymentSecret), 1, routePrefix = Seq(ChannelHop(nodeParams.nodeId, b, channelUpdate_ab_1.copy(channelFlags = ChannelFlags.Empty)))),
SendPayment(paymentHash, b, Onion.createMultiPartPayload(1000 * 1000 msat, payment.totalAmount, expiry, payment.paymentSecret), 1, routePrefix = Seq(ChannelHop(nodeParams.nodeId, b, channelUpdate_ab_2.copy(channelFlags = ChannelFlags.Empty))))
SendPayment(b, Onion.createMultiPartPayload(1000 * 1000 msat, payment.totalAmount, expiry, payment.paymentSecret), 1, routePrefix = Seq(ChannelHop(nodeParams.nodeId, b, channelUpdate_ab_1.copy(channelFlags = ChannelFlags.Empty)))),
SendPayment(b, Onion.createMultiPartPayload(1000 * 1000 msat, payment.totalAmount, expiry, payment.paymentSecret), 1, routePrefix = Seq(ChannelHop(nodeParams.nodeId, b, channelUpdate_ab_2.copy(channelFlags = ChannelFlags.Empty))))
)
childPayFsm.expectNoMsg(50 millis)
val childIds = payFsm.stateData.asInstanceOf[PaymentProgress].pending.keys.toSeq
@ -149,26 +148,32 @@ class MultiPartPaymentLifecycleSpec extends TestKit(ActorSystem("test")) with fi
val pp1 = PartialPayment(childIds.head, 1000 * 1000 msat, 0 msat, randomBytes32, None)
val pp2 = PartialPayment(childIds(1), 1000 * 1000 msat, 0 msat, randomBytes32, None)
childPayFsm.send(payFsm, PaymentSent(childIds.head, paymentHash, paymentPreimage, Seq(pp1)))
childPayFsm.send(payFsm, PaymentSent(childIds(1), paymentHash, paymentPreimage, Seq(pp2)))
val expectedMsg = PaymentSent(paymentId, paymentHash, paymentPreimage, Seq(pp1, pp2))
childPayFsm.send(payFsm, PaymentSent(paymentId, paymentHash, paymentPreimage, finalAmount, b, Seq(pp1)))
childPayFsm.send(payFsm, PaymentSent(paymentId, paymentHash, paymentPreimage, finalAmount, b, Seq(pp2)))
val expectedMsg = PaymentSent(paymentId, paymentHash, paymentPreimage, finalAmount, finalRecipient, Seq(pp1, pp2))
sender.expectMsg(expectedMsg)
eventListener.expectMsg(expectedMsg)
assert(expectedMsg.recipientAmount === finalAmount)
assert(expectedMsg.amountWithFees === (2000 * 1000).msat)
assert(expectedMsg.trampolineFees === (1000 * 1000).msat)
assert(expectedMsg.nonTrampolineFees === 0.msat)
assert(expectedMsg.feesPaid === expectedMsg.trampolineFees)
}
test("send to peer node via single big channel") { f =>
import f._
val payment = SendMultiPartPayment(paymentHash, randomBytes32, b, 1000 * 1000 msat, expiry, 1)
val payment = SendMultiPartPayment(randomBytes32, b, 1000 * 1000 msat, expiry, 1)
// Network statistics should be ignored when sending to peer (otherwise we should have split into multiple payments).
initPayment(f, payment, emptyStats.copy(capacity = Stats(Seq(100), d => Satoshi(d.toLong))), localChannels(0))
childPayFsm.expectMsg(SendPayment(paymentHash, b, Onion.createMultiPartPayload(payment.totalAmount, payment.totalAmount, expiry, payment.paymentSecret), 1, routePrefix = Seq(ChannelHop(nodeParams.nodeId, b, channelUpdate_ab_1))))
childPayFsm.expectMsg(SendPayment(b, Onion.createMultiPartPayload(payment.totalAmount, payment.totalAmount, expiry, payment.paymentSecret), 1, routePrefix = Seq(ChannelHop(nodeParams.nodeId, b, channelUpdate_ab_1))))
childPayFsm.expectNoMsg(50 millis)
}
test("send to peer node via remote channels") { f =>
import f._
// d only has a single channel with capacity 1000 sat, we try to send more.
val payment = SendMultiPartPayment(paymentHash, randomBytes32, d, 2000 * 1000 msat, expiry, 1)
val payment = SendMultiPartPayment(randomBytes32, d, 2000 * 1000 msat, expiry, 1)
val testChannels = localChannels()
val balanceToTarget = testChannels.channels.filter(_.nextNodeId == d).map(_.commitments.availableBalanceForSend).sum
assert(balanceToTarget < (1000 * 1000).msat) // the commit tx fee prevents us from completely emptying our channel
@ -183,28 +188,27 @@ class MultiPartPaymentLifecycleSpec extends TestKit(ActorSystem("test")) with fi
test("send to remote node without splitting") { f =>
import f._
val payment = SendMultiPartPayment(paymentHash, randomBytes32, e, 300 * 1000 msat, expiry, 1)
val payment = SendMultiPartPayment(randomBytes32, e, 300 * 1000 msat, expiry, 1)
initPayment(f, payment, emptyStats.copy(capacity = Stats(Seq(1500), d => Satoshi(d.toLong))), localChannels())
waitUntilAmountSent(f, payment.totalAmount)
payFsm.stateData.asInstanceOf[PaymentProgress].pending.foreach {
case (id, payment) => childPayFsm.send(payFsm, PaymentSent(id, paymentHash, paymentPreimage, Seq(PartialPayment(id, payment.finalPayload.amount, 5 msat, randomBytes32, None))))
case (id, payment) => childPayFsm.send(payFsm, PaymentSent(paymentId, paymentHash, paymentPreimage, finalAmount, e, Seq(PartialPayment(id, payment.finalPayload.amount, 5 msat, randomBytes32, None))))
}
val result = sender.expectMsgType[PaymentSent]
assert(result.id === paymentId)
assert(result.amount === payment.totalAmount)
assert(result.amountWithFees === payment.totalAmount + result.nonTrampolineFees)
assert(result.parts.length === 1)
}
test("send to remote node via multiple channels") { f =>
import f._
val payment = SendMultiPartPayment(paymentHash, randomBytes32, e, 3200 * 1000 msat, expiry, 3)
val payment = SendMultiPartPayment(randomBytes32, e, 3200 * 1000 msat, expiry, 3)
// A network capacity of 1000 sat should split the payment in at least 3 parts.
initPayment(f, payment, emptyStats.copy(capacity = Stats(Seq(1000), d => Satoshi(d.toLong))), localChannels())
val payments = Iterator.iterate(0 msat)(sent => {
val child = childPayFsm.expectMsgType[SendPayment]
assert(child.paymentHash === paymentHash)
assert(child.targetNodeId === e)
assert(child.maxAttempts === 3)
assert(child.finalPayload.expiry === expiry)
@ -222,19 +226,21 @@ class MultiPartPaymentLifecycleSpec extends TestKit(ActorSystem("test")) with fi
val partialPayments = pending.map {
case (id, payment) => PartialPayment(id, payment.finalPayload.amount, 1 msat, randomBytes32, Some(hop_ac_1 :: hop_ab_2 :: Nil))
}
partialPayments.foreach(pp => childPayFsm.send(payFsm, PaymentSent(pp.id, paymentHash, paymentPreimage, Seq(pp))))
partialPayments.foreach(pp => childPayFsm.send(payFsm, PaymentSent(paymentId, paymentHash, paymentPreimage, finalAmount, e, Seq(pp))))
val result = sender.expectMsgType[PaymentSent]
assert(result.id === paymentId)
assert(result.paymentHash === paymentHash)
assert(result.paymentPreimage === paymentPreimage)
assert(result.parts === partialPayments)
assert(result.amount === (3200 * 1000).msat)
assert(result.feesPaid === partialPayments.map(_.feesPaid).sum)
assert(result.recipientAmount === finalAmount)
assert(result.amountWithFees > (3200 * 1000).msat)
assert(result.trampolineFees === (2200 * 1000).msat)
assert(result.nonTrampolineFees === partialPayments.map(_.feesPaid).sum)
}
test("send to remote node via single big channel") { f =>
import f._
val payment = SendMultiPartPayment(paymentHash, randomBytes32, e, 3500 * 1000 msat, expiry, 3)
val payment = SendMultiPartPayment(randomBytes32, e, 3500 * 1000 msat, expiry, 3)
// When splitting inside a channel, we need to take the fees of the commit tx into account (multiple outgoing HTLCs
// will increase the size of the commit tx and thus its fee.
val feeRatePerKw = 100
@ -249,20 +255,21 @@ class MultiPartPaymentLifecycleSpec extends TestKit(ActorSystem("test")) with fi
val partialPayments = pending.map {
case (id, payment) => PartialPayment(id, payment.finalPayload.amount, 1 msat, randomBytes32, None)
}
partialPayments.foreach(pp => childPayFsm.send(payFsm, PaymentSent(pp.id, paymentHash, paymentPreimage, Seq(pp))))
partialPayments.foreach(pp => childPayFsm.send(payFsm, PaymentSent(paymentId, paymentHash, paymentPreimage, payment.totalAmount, e, Seq(pp))))
val result = sender.expectMsgType[PaymentSent]
assert(result.id === paymentId)
assert(result.paymentHash === paymentHash)
assert(result.paymentPreimage === paymentPreimage)
assert(result.parts === partialPayments)
assert(result.amount === (3500 * 1000).msat)
assert(result.feesPaid === partialPayments.map(_.feesPaid).sum)
assert(result.amountWithFees - result.nonTrampolineFees === (3500 * 1000).msat)
assert(result.recipientNodeId === finalRecipient) // the recipient is obtained from the config, not from the request (which may be to the first trampoline node)
assert(result.nonTrampolineFees === partialPayments.map(_.feesPaid).sum)
}
test("send to remote trampoline node") { f =>
import f._
val trampolineTlv = OnionTlv.TrampolineOnion(OnionRoutingPacket(0, ByteVector.fill(33)(0), ByteVector.fill(400)(0), randomBytes32))
val payment = SendMultiPartPayment(paymentHash, randomBytes32, e, 3000 * 1000 msat, expiry, 3, additionalTlvs = Seq(trampolineTlv))
val payment = SendMultiPartPayment(randomBytes32, e, 3000 * 1000 msat, expiry, 3, additionalTlvs = Seq(trampolineTlv))
initPayment(f, payment, emptyStats.copy(capacity = Stats(Seq(1000), d => Satoshi(d.toLong))), localChannels())
waitUntilAmountSent(f, payment.totalAmount)
@ -275,7 +282,7 @@ class MultiPartPaymentLifecycleSpec extends TestKit(ActorSystem("test")) with fi
test("split fees between child payments") { f =>
import f._
val routeParams = RouteParams(randomize = false, 100 msat, 0.05, 20, CltvExpiryDelta(144), None)
val payment = SendMultiPartPayment(paymentHash, randomBytes32, e, 3000 * 1000 msat, expiry, 3, routeParams = Some(routeParams))
val payment = SendMultiPartPayment(randomBytes32, e, 3000 * 1000 msat, expiry, 3, routeParams = Some(routeParams))
initPayment(f, payment, emptyStats.copy(capacity = Stats(Seq(1000), d => Satoshi(d.toLong))), localChannels())
waitUntilAmountSent(f, 3000 * 1000 msat)
@ -290,7 +297,7 @@ class MultiPartPaymentLifecycleSpec extends TestKit(ActorSystem("test")) with fi
test("skip empty channels") { f =>
import f._
val payment = SendMultiPartPayment(paymentHash, randomBytes32, e, 3000 * 1000 msat, expiry, 3)
val payment = SendMultiPartPayment(randomBytes32, e, 3000 * 1000 msat, expiry, 3)
val testChannels = localChannels()
val testChannels1 = testChannels.copy(channels = testChannels.channels ++ Seq(
OutgoingChannel(b, channelUpdate_ab_1.copy(shortChannelId = ShortChannelId(42)), makeCommitments(0 msat, 10)),
@ -299,17 +306,17 @@ class MultiPartPaymentLifecycleSpec extends TestKit(ActorSystem("test")) with fi
initPayment(f, payment, emptyStats.copy(capacity = Stats(Seq(1000), d => Satoshi(d.toLong))), testChannels1)
waitUntilAmountSent(f, payment.totalAmount)
payFsm.stateData.asInstanceOf[PaymentProgress].pending.foreach {
case (id, payment) => childPayFsm.send(payFsm, PaymentSent(id, paymentHash, paymentPreimage, Seq(PartialPayment(id, payment.finalPayload.amount, 5 msat, randomBytes32, None))))
case (id, p) => childPayFsm.send(payFsm, PaymentSent(paymentId, paymentHash, paymentPreimage, payment.totalAmount, e, Seq(PartialPayment(id, p.finalPayload.amount, 5 msat, randomBytes32, None))))
}
val result = sender.expectMsgType[PaymentSent]
assert(result.id === paymentId)
assert(result.amount === payment.totalAmount)
assert(result.amountWithFees > payment.totalAmount)
}
test("retry after error") { f =>
import f._
val payment = SendMultiPartPayment(paymentHash, randomBytes32, e, 3000 * 1000 msat, expiry, 3)
val payment = SendMultiPartPayment(randomBytes32, e, 3000 * 1000 msat, expiry, 3)
val testChannels = localChannels()
// A network capacity of 1000 sat should split the payment in at least 3 parts.
initPayment(f, payment, emptyStats.copy(capacity = Stats(Seq(1000), d => Satoshi(d.toLong))), testChannels)
@ -342,7 +349,7 @@ class MultiPartPaymentLifecycleSpec extends TestKit(ActorSystem("test")) with fi
test("cannot send (not enough capacity on local channels)") { f =>
import f._
val payment = SendMultiPartPayment(paymentHash, randomBytes32, e, 3000 * 1000 msat, expiry, 3)
val payment = SendMultiPartPayment(randomBytes32, e, 3000 * 1000 msat, expiry, 3)
initPayment(f, payment, emptyStats.copy(capacity = Stats(Seq(1000), d => Satoshi(d.toLong))), OutgoingChannels(Seq(
OutgoingChannel(b, channelUpdate_ab_1, makeCommitments(1000 * 1000 msat, 10)),
OutgoingChannel(c, channelUpdate_ac_2, makeCommitments(1000 * 1000 msat, 10)),
@ -357,7 +364,7 @@ class MultiPartPaymentLifecycleSpec extends TestKit(ActorSystem("test")) with fi
test("cannot send (fee rate too high)") { f =>
import f._
val payment = SendMultiPartPayment(paymentHash, randomBytes32, e, 2500 * 1000 msat, expiry, 3)
val payment = SendMultiPartPayment(randomBytes32, e, 2500 * 1000 msat, expiry, 3)
initPayment(f, payment, emptyStats.copy(capacity = Stats(Seq(1000), d => Satoshi(d.toLong))), OutgoingChannels(Seq(
OutgoingChannel(b, channelUpdate_ab_1, makeCommitments(1500 * 1000 msat, 1000)),
OutgoingChannel(c, channelUpdate_ac_2, makeCommitments(1500 * 1000 msat, 1000)),
@ -372,7 +379,7 @@ class MultiPartPaymentLifecycleSpec extends TestKit(ActorSystem("test")) with fi
test("payment timeout") { f =>
import f._
val payment = SendMultiPartPayment(paymentHash, randomBytes32, e, 3000 * 1000 msat, expiry, 5)
val payment = SendMultiPartPayment(randomBytes32, e, 3000 * 1000 msat, expiry, 5)
initPayment(f, payment, emptyStats.copy(capacity = Stats(Seq(1000), d => Satoshi(d.toLong))), localChannels())
waitUntilAmountSent(f, payment.totalAmount)
val (childId1, _) = payFsm.stateData.asInstanceOf[PaymentProgress].pending.head
@ -385,7 +392,7 @@ class MultiPartPaymentLifecycleSpec extends TestKit(ActorSystem("test")) with fi
test("failure received from final recipient") { f =>
import f._
val payment = SendMultiPartPayment(paymentHash, randomBytes32, e, 3000 * 1000 msat, expiry, 5)
val payment = SendMultiPartPayment(randomBytes32, e, 3000 * 1000 msat, expiry, 5)
initPayment(f, payment, emptyStats.copy(capacity = Stats(Seq(1000), d => Satoshi(d.toLong))), localChannels())
waitUntilAmountSent(f, payment.totalAmount)
val (childId1, _) = payFsm.stateData.asInstanceOf[PaymentProgress].pending.head
@ -398,7 +405,7 @@ class MultiPartPaymentLifecycleSpec extends TestKit(ActorSystem("test")) with fi
test("fail after too many attempts") { f =>
import f._
val payment = SendMultiPartPayment(paymentHash, randomBytes32, e, 3000 * 1000 msat, expiry, 2)
val payment = SendMultiPartPayment(randomBytes32, e, 3000 * 1000 msat, expiry, 2)
initPayment(f, payment, emptyStats.copy(capacity = Stats(Seq(1000), d => Satoshi(d.toLong))), localChannels())
waitUntilAmountSent(f, payment.totalAmount)
val (childId1, childPayment1) = payFsm.stateData.asInstanceOf[PaymentProgress].pending.head
@ -428,14 +435,14 @@ class MultiPartPaymentLifecycleSpec extends TestKit(ActorSystem("test")) with fi
test("receive partial failure after success (recipient spec violation)") { f =>
import f._
val payment = SendMultiPartPayment(paymentHash, randomBytes32, e, 4000 * 1000 msat, expiry, 2)
val payment = SendMultiPartPayment(randomBytes32, e, 4000 * 1000 msat, expiry, 2)
initPayment(f, payment, emptyStats.copy(capacity = Stats(Seq(1500), d => Satoshi(d.toLong))), localChannels())
waitUntilAmountSent(f, payment.totalAmount)
val pending = payFsm.stateData.asInstanceOf[PaymentProgress].pending
// If one of the payments succeeds, the recipient MUST succeed them all: we can consider the whole payment succeeded.
val (id1, payment1) = pending.head
childPayFsm.send(payFsm, PaymentSent(id1, paymentHash, paymentPreimage, Seq(PartialPayment(id1, payment1.finalPayload.amount, 10 msat, randomBytes32, None))))
childPayFsm.send(payFsm, PaymentSent(paymentId, paymentHash, paymentPreimage, payment.totalAmount, e, Seq(PartialPayment(id1, payment1.finalPayload.amount, 0 msat, randomBytes32, None))))
awaitCond(payFsm.stateName === PAYMENT_SUCCEEDED)
// A partial failure should simply be ignored.
@ -443,16 +450,16 @@ class MultiPartPaymentLifecycleSpec extends TestKit(ActorSystem("test")) with fi
childPayFsm.send(payFsm, PaymentFailed(id2, paymentHash, Nil))
pending.tail.tail.foreach {
case (id, payment) => childPayFsm.send(payFsm, PaymentSent(id, paymentHash, paymentPreimage, Seq(PartialPayment(id, payment.finalPayload.amount, 10 msat, randomBytes32, None))))
case (id, p) => childPayFsm.send(payFsm, PaymentSent(paymentId, paymentHash, paymentPreimage, payment.totalAmount, e, Seq(PartialPayment(id, p.finalPayload.amount, 0 msat, randomBytes32, None))))
}
val result = sender.expectMsgType[PaymentSent]
assert(result.id === paymentId)
assert(result.amount === payment.totalAmount - payment2.finalPayload.amount)
assert(result.amountWithFees === payment.totalAmount - payment2.finalPayload.amount)
}
test("receive partial success after abort (recipient spec violation)") { f =>
import f._
val payment = SendMultiPartPayment(paymentHash, randomBytes32, e, 5000 * 1000 msat, expiry, 1)
val payment = SendMultiPartPayment(randomBytes32, e, 5000 * 1000 msat, expiry, 1)
initPayment(f, payment, emptyStats.copy(capacity = Stats(Seq(2000), d => Satoshi(d.toLong))), localChannels())
waitUntilAmountSent(f, payment.totalAmount)
val pending = payFsm.stateData.asInstanceOf[PaymentProgress].pending
@ -465,7 +472,7 @@ class MultiPartPaymentLifecycleSpec extends TestKit(ActorSystem("test")) with fi
// The in-flight HTLC set doesn't pay the full amount, so the recipient MUST not fulfill any of those.
// But if he does, it's too bad for him as we have obtained a cheaper proof of payment.
val (id2, payment2) = pending.tail.head
childPayFsm.send(payFsm, PaymentSent(id2, paymentHash, paymentPreimage, Seq(PartialPayment(id2, payment2.finalPayload.amount, 5 msat, randomBytes32, None))))
childPayFsm.send(payFsm, PaymentSent(paymentId, paymentHash, paymentPreimage, payment.totalAmount, e, Seq(PartialPayment(id2, payment2.finalPayload.amount, 5 msat, randomBytes32, None))))
awaitCond(payFsm.stateName === PAYMENT_SUCCEEDED)
// Even if all other child payments fail, we obtained the preimage so the payment is a success from our point of view.
@ -474,8 +481,8 @@ class MultiPartPaymentLifecycleSpec extends TestKit(ActorSystem("test")) with fi
}
val result = sender.expectMsgType[PaymentSent]
assert(result.id === paymentId)
assert(result.amount === payment2.finalPayload.amount)
assert(result.feesPaid === 5.msat)
assert(result.amountWithFees === payment2.finalPayload.amount + 5.msat)
assert(result.nonTrampolineFees === 5.msat)
}
test("split payment", Tag("fuzzy")) { f =>
@ -486,7 +493,7 @@ class MultiPartPaymentLifecycleSpec extends TestKit(ActorSystem("test")) with fi
val toSend = ((1 + Random.nextInt(3500)) * 1000).msat
val networkStats = emptyStats.copy(capacity = Stats(Seq(400 + Random.nextInt(1600)), d => Satoshi(d.toLong)))
val routeParams = RouteParams(randomize = true, Random.nextInt(1000).msat, Random.nextInt(10).toDouble / 100, 20, CltvExpiryDelta(144), None)
val request = SendMultiPartPayment(paymentHash, randomBytes32, e, toSend, CltvExpiry(561), 1, Nil, Some(routeParams))
val request = SendMultiPartPayment(randomBytes32, e, toSend, CltvExpiry(561), 1, Nil, Some(routeParams))
val fuzzParams = s"(sending $toSend with network capacity ${networkStats.capacity.percentile75.toMilliSatoshi}, fee base ${routeParams.maxFeeBase} and fee percentage ${routeParams.maxFeePct})"
val (remaining, payments) = splitPayment(f.nodeParams, toSend, testChannels.channels, Some(networkStats), request, randomize = true)
assert(remaining === 0.msat, fuzzParams)
@ -502,6 +509,8 @@ object MultiPartPaymentLifecycleSpec {
val paymentPreimage = randomBytes32
val paymentHash = Crypto.sha256(paymentPreimage)
val expiry = CltvExpiry(1105)
val finalAmount = 1000000 msat
val finalRecipient = randomKey.publicKey
/**
* We simulate a multi-part-friendly network:
@ -513,7 +522,7 @@ object MultiPartPaymentLifecycleSpec {
* where a has multiple channels with each of his peers.
*/
val a :: b :: c :: d :: e :: Nil = Seq.fill(5)(PrivateKey(randomBytes32).publicKey)
val a :: b :: c :: d :: e :: Nil = Seq.fill(5)(randomKey.publicKey)
val channelId_ab_1 = ShortChannelId(1)
val channelId_ab_2 = ShortChannelId(2)
val channelId_ac_1 = ShortChannelId(11)

View file

@ -252,8 +252,8 @@ class NodeRelayerSpec extends TestkitBaseClass {
incomingMultiPart.foreach(p => commandBuffer.expectMsg(CommandBuffer.CommandSend(p.add.channelId, CMD_FULFILL_HTLC(p.add.id, paymentPreimage, commit = true))))
val relayEvent = eventListener.expectMsgType[TrampolinePaymentRelayed]
validateRelayEvent(relayEvent)
assert(relayEvent.fromChannelIds.toSet === incomingMultiPart.map(_.add.channelId).toSet)
assert(relayEvent.toChannelIds.nonEmpty)
assert(relayEvent.incoming.toSet === incomingMultiPart.map(i => PaymentRelayed.Part(i.add.amountMsat, i.add.channelId)).toSet)
assert(relayEvent.outgoing.nonEmpty)
commandBuffer.expectNoMsg(100 millis)
}
@ -273,8 +273,8 @@ class NodeRelayerSpec extends TestkitBaseClass {
commandBuffer.expectMsg(CommandBuffer.CommandSend(incomingAdd.channelId, CMD_FULFILL_HTLC(incomingAdd.id, paymentPreimage, commit = true)))
val relayEvent = eventListener.expectMsgType[TrampolinePaymentRelayed]
validateRelayEvent(relayEvent)
assert(relayEvent.fromChannelIds === Seq(incomingSinglePart.add.channelId))
assert(relayEvent.toChannelIds.nonEmpty)
assert(relayEvent.incoming === Seq(PaymentRelayed.Part(incomingSinglePart.add.amountMsat, incomingSinglePart.add.channelId)))
assert(relayEvent.outgoing.nonEmpty)
commandBuffer.expectNoMsg(100 millis)
}
@ -293,10 +293,9 @@ class NodeRelayerSpec extends TestkitBaseClass {
val outgoingCfg = outgoingPayFSM.expectMsgType[SendPaymentConfig]
validateOutgoingCfg(outgoingCfg, Upstream.TrampolineRelayed(incomingMultiPart.map(_.add)))
val outgoingPayment = outgoingPayFSM.expectMsgType[SendMultiPartPayment]
assert(outgoingPayment.paymentHash === paymentHash)
assert(outgoingPayment.paymentSecret === pr.paymentSecret.get) // we should use the provided secret
assert(outgoingPayment.totalAmount === outgoingAmount)
assert(outgoingPayment.finalExpiry === outgoingExpiry)
assert(outgoingPayment.targetExpiry === outgoingExpiry)
assert(outgoingPayment.targetNodeId === outgoingNodeId)
assert(outgoingPayment.additionalTlvs === Nil)
assert(outgoingPayment.routeParams.isDefined)
@ -306,8 +305,8 @@ class NodeRelayerSpec extends TestkitBaseClass {
incomingMultiPart.foreach(p => commandBuffer.expectMsg(CommandBuffer.CommandSend(p.add.channelId, CMD_FULFILL_HTLC(p.add.id, paymentPreimage, commit = true))))
val relayEvent = eventListener.expectMsgType[TrampolinePaymentRelayed]
validateRelayEvent(relayEvent)
assert(relayEvent.fromChannelIds === incomingMultiPart.map(_.add.channelId))
assert(relayEvent.toChannelIds.nonEmpty)
assert(relayEvent.incoming === incomingMultiPart.map(i => PaymentRelayed.Part(i.add.amountMsat, i.add.channelId)))
assert(relayEvent.outgoing.nonEmpty)
commandBuffer.expectNoMsg(100 millis)
}
@ -324,7 +323,6 @@ class NodeRelayerSpec extends TestkitBaseClass {
val outgoingCfg = outgoingPayFSM.expectMsgType[SendPaymentConfig]
validateOutgoingCfg(outgoingCfg, Upstream.TrampolineRelayed(incomingMultiPart.map(_.add)))
val outgoingPayment = outgoingPayFSM.expectMsgType[SendPayment]
assert(outgoingPayment.paymentHash === paymentHash)
assert(outgoingPayment.routePrefix === Nil)
assert(outgoingPayment.finalPayload.amount === outgoingAmount)
assert(outgoingPayment.finalPayload.expiry === outgoingExpiry)
@ -336,8 +334,8 @@ class NodeRelayerSpec extends TestkitBaseClass {
incomingMultiPart.foreach(p => commandBuffer.expectMsg(CommandBuffer.CommandSend(p.add.channelId, CMD_FULFILL_HTLC(p.add.id, paymentPreimage, commit = true))))
val relayEvent = eventListener.expectMsgType[TrampolinePaymentRelayed]
validateRelayEvent(relayEvent)
assert(relayEvent.fromChannelIds === incomingMultiPart.map(_.add.channelId))
assert(relayEvent.toChannelIds.length === 1)
assert(relayEvent.incoming === incomingMultiPart.map(i => PaymentRelayed.Part(i.add.amountMsat, i.add.channelId)))
assert(relayEvent.outgoing.length === 1)
commandBuffer.expectNoMsg(100 millis)
}
@ -346,15 +344,15 @@ class NodeRelayerSpec extends TestkitBaseClass {
assert(!outgoingCfg.storeInDb)
assert(outgoingCfg.paymentHash === paymentHash)
assert(outgoingCfg.paymentRequest === None)
assert(outgoingCfg.targetNodeId === outgoingNodeId)
assert(outgoingCfg.recipientAmount === outgoingAmount)
assert(outgoingCfg.recipientNodeId === outgoingNodeId)
assert(outgoingCfg.upstream === upstream)
}
def validateOutgoingPayment(outgoingPayment: SendMultiPartPayment): Unit = {
assert(outgoingPayment.paymentHash === paymentHash)
assert(outgoingPayment.paymentSecret !== incomingSecret) // we should generate a new outgoing secret
assert(outgoingPayment.totalAmount === outgoingAmount)
assert(outgoingPayment.finalExpiry === outgoingExpiry)
assert(outgoingPayment.targetExpiry === outgoingExpiry)
assert(outgoingPayment.targetNodeId === outgoingNodeId)
assert(outgoingPayment.additionalTlvs === Seq(OnionTlv.TrampolineOnion(nextTrampolinePacket)))
assert(outgoingPayment.routeParams.isDefined)
@ -363,9 +361,8 @@ class NodeRelayerSpec extends TestkitBaseClass {
def validateRelayEvent(e: TrampolinePaymentRelayed): Unit = {
assert(e.amountIn === incomingAmount)
assert(e.amountOut === outgoingAmount)
assert(e.amountOut >= outgoingAmount) // outgoingAmount + routing fees
assert(e.paymentHash === paymentHash)
assert(e.toNodeId === outgoingNodeId)
}
}
@ -394,7 +391,7 @@ object NodeRelayerSpec {
createValidIncomingPacket(incomingAmount, incomingAmount, CltvExpiry(500000), outgoingAmount, outgoingExpiry)
def createSuccessEvent(id: UUID): PaymentSent =
PaymentSent(id, paymentHash, paymentPreimage, Seq(PaymentSent.PartialPayment(id, outgoingAmount, 10 msat, randomBytes32, None)))
PaymentSent(id, paymentHash, paymentPreimage, outgoingAmount, outgoingNodeId, Seq(PaymentSent.PartialPayment(id, outgoingAmount, 10 msat, randomBytes32, None)))
def createValidIncomingPacket(amountIn: MilliSatoshi, totalAmountIn: MilliSatoshi, expiryIn: CltvExpiry, amountOut: MilliSatoshi, expiryOut: CltvExpiry): IncomingPacket.NodeRelayPacket = {
val outerPayload = if (amountIn == totalAmountIn) {

View file

@ -85,8 +85,8 @@ class PaymentInitiatorSpec extends TestKit(ActorSystem("test")) with fixture.Fun
import f._
sender.send(initiator, SendPaymentRequest(finalAmount, paymentHash, c, 1, predefinedRoute = Seq(a, b, c)))
val paymentId = sender.expectMsgType[UUID]
payFsm.expectMsg(SendPaymentConfig(paymentId, paymentId, None, paymentHash, c, Upstream.Local(paymentId), None, storeInDb = true, publishEvent = true))
payFsm.expectMsg(SendPaymentToRoute(paymentHash, Seq(a, b, c), FinalLegacyPayload(finalAmount, Channel.MIN_CLTV_EXPIRY_DELTA.toCltvExpiry(nodeParams.currentBlockHeight + 1))))
payFsm.expectMsg(SendPaymentConfig(paymentId, paymentId, None, paymentHash, finalAmount, c, Upstream.Local(paymentId), None, storeInDb = true, publishEvent = true, Nil))
payFsm.expectMsg(SendPaymentToRoute(Seq(a, b, c), FinalLegacyPayload(finalAmount, Channel.MIN_CLTV_EXPIRY_DELTA.toCltvExpiry(nodeParams.currentBlockHeight + 1))))
}
test("forward legacy payment") { f =>
@ -95,13 +95,13 @@ class PaymentInitiatorSpec extends TestKit(ActorSystem("test")) with fixture.Fun
val routeParams = RouteParams(randomize = true, 15 msat, 1.5, 5, CltvExpiryDelta(561), None)
sender.send(initiator, SendPaymentRequest(finalAmount, paymentHash, c, 1, CltvExpiryDelta(42), assistedRoutes = hints, routeParams = Some(routeParams)))
val id1 = sender.expectMsgType[UUID]
payFsm.expectMsg(SendPaymentConfig(id1, id1, None, paymentHash, c, Upstream.Local(id1), None, storeInDb = true, publishEvent = true))
payFsm.expectMsg(SendPayment(paymentHash, c, FinalLegacyPayload(finalAmount, CltvExpiryDelta(42).toCltvExpiry(nodeParams.currentBlockHeight + 1)), 1, hints, Some(routeParams)))
payFsm.expectMsg(SendPaymentConfig(id1, id1, None, paymentHash, finalAmount, c, Upstream.Local(id1), None, storeInDb = true, publishEvent = true, Nil))
payFsm.expectMsg(SendPayment(c, FinalLegacyPayload(finalAmount, CltvExpiryDelta(42).toCltvExpiry(nodeParams.currentBlockHeight + 1)), 1, hints, Some(routeParams)))
sender.send(initiator, SendPaymentRequest(finalAmount, paymentHash, e, 3))
val id2 = sender.expectMsgType[UUID]
payFsm.expectMsg(SendPaymentConfig(id2, id2, None, paymentHash, e, Upstream.Local(id2), None, storeInDb = true, publishEvent = true))
payFsm.expectMsg(SendPayment(paymentHash, e, FinalLegacyPayload(finalAmount, Channel.MIN_CLTV_EXPIRY_DELTA.toCltvExpiry(nodeParams.currentBlockHeight + 1)), 3))
payFsm.expectMsg(SendPaymentConfig(id2, id2, None, paymentHash, finalAmount, e, Upstream.Local(id2), None, storeInDb = true, publishEvent = true, Nil))
payFsm.expectMsg(SendPayment(e, FinalLegacyPayload(finalAmount, Channel.MIN_CLTV_EXPIRY_DELTA.toCltvExpiry(nodeParams.currentBlockHeight + 1)), 3))
}
test("forward legacy payment when multi-part deactivated", Tag("mpp_disabled")) { f =>
@ -110,8 +110,8 @@ class PaymentInitiatorSpec extends TestKit(ActorSystem("test")) with fixture.Fun
val req = SendPaymentRequest(finalAmount, paymentHash, c, 1, CltvExpiryDelta(42), Some(pr))
sender.send(initiator, req)
val id = sender.expectMsgType[UUID]
payFsm.expectMsg(SendPaymentConfig(id, id, None, paymentHash, c, Upstream.Local(id), Some(pr), storeInDb = true, publishEvent = true))
payFsm.expectMsg(SendPayment(paymentHash, c, FinalLegacyPayload(finalAmount, req.finalExpiry(nodeParams.currentBlockHeight)), 1))
payFsm.expectMsg(SendPaymentConfig(id, id, None, paymentHash, finalAmount, c, Upstream.Local(id), Some(pr), storeInDb = true, publishEvent = true, Nil))
payFsm.expectMsg(SendPayment(c, FinalLegacyPayload(finalAmount, req.finalExpiry(nodeParams.currentBlockHeight)), 1))
}
test("forward multi-part payment") { f =>
@ -120,8 +120,8 @@ class PaymentInitiatorSpec extends TestKit(ActorSystem("test")) with fixture.Fun
val req = SendPaymentRequest(finalAmount + 100.msat, paymentHash, c, 1, CltvExpiryDelta(42), Some(pr))
sender.send(initiator, req)
val id = sender.expectMsgType[UUID]
multiPartPayFsm.expectMsg(SendPaymentConfig(id, id, None, paymentHash, c, Upstream.Local(id), Some(pr), storeInDb = true, publishEvent = true))
multiPartPayFsm.expectMsg(SendMultiPartPayment(paymentHash, pr.paymentSecret.get, c, finalAmount + 100.msat, req.finalExpiry(nodeParams.currentBlockHeight), 1))
multiPartPayFsm.expectMsg(SendPaymentConfig(id, id, None, paymentHash, finalAmount + 100.msat, c, Upstream.Local(id), Some(pr), storeInDb = true, publishEvent = true, Nil))
multiPartPayFsm.expectMsg(SendMultiPartPayment(pr.paymentSecret.get, c, finalAmount + 100.msat, req.finalExpiry(nodeParams.currentBlockHeight), 1))
}
test("forward multi-part payment with pre-defined route") { f =>
@ -130,9 +130,8 @@ class PaymentInitiatorSpec extends TestKit(ActorSystem("test")) with fixture.Fun
val req = SendPaymentRequest(finalAmount / 2, paymentHash, c, 1, paymentRequest = Some(pr), predefinedRoute = Seq(a, b, c))
sender.send(initiator, req)
val id = sender.expectMsgType[UUID]
payFsm.expectMsg(SendPaymentConfig(id, id, None, paymentHash, c, Upstream.Local(id), Some(pr), storeInDb = true, publishEvent = true))
payFsm.expectMsg(SendPaymentConfig(id, id, None, paymentHash, finalAmount / 2, c, Upstream.Local(id), Some(pr), storeInDb = true, publishEvent = true, Nil))
val msg = payFsm.expectMsgType[SendPaymentToRoute]
assert(msg.paymentHash === paymentHash)
assert(msg.hops === Seq(a, b, c))
assert(msg.finalPayload.amount === finalAmount / 2)
assert(msg.finalPayload.paymentSecret === pr.paymentSecret)
@ -151,10 +150,9 @@ class PaymentInitiatorSpec extends TestKit(ActorSystem("test")) with fixture.Fun
multiPartPayFsm.expectMsgType[SendPaymentConfig]
val msg = multiPartPayFsm.expectMsgType[SendMultiPartPayment]
assert(msg.paymentHash === pr.paymentHash)
assert(msg.paymentSecret !== pr.paymentSecret.get) // we should not leak the invoice secret to the trampoline node
assert(msg.targetNodeId === b)
assert(msg.finalExpiry.toLong === currentBlockCount + 9 + 12 + 1)
assert(msg.targetExpiry.toLong === currentBlockCount + 9 + 12 + 1)
assert(msg.totalAmount === finalAmount + trampolineFees)
assert(msg.additionalTlvs.head.isInstanceOf[OnionTlv.TrampolineOnion])
@ -191,10 +189,9 @@ class PaymentInitiatorSpec extends TestKit(ActorSystem("test")) with fixture.Fun
multiPartPayFsm.expectMsgType[SendPaymentConfig]
val msg = multiPartPayFsm.expectMsgType[SendMultiPartPayment]
assert(msg.paymentHash === pr.paymentHash)
assert(msg.paymentSecret !== pr.paymentSecret.get) // we should not leak the invoice secret to the trampoline node
assert(msg.targetNodeId === b)
assert(msg.finalExpiry.toLong === currentBlockCount + 9 + 12 + 1)
assert(msg.targetExpiry.toLong === currentBlockCount + 9 + 12 + 1)
assert(msg.totalAmount === finalAmount + trampolineFees)
assert(msg.additionalTlvs.head.isInstanceOf[OnionTlv.TrampolineOnion])
@ -251,7 +248,7 @@ class PaymentInitiatorSpec extends TestKit(ActorSystem("test")) with fixture.Fun
assert(msg2.totalAmount === finalAmount + 25000.msat)
// Simulate success which should publish the event and respond to the original sender.
val success = PaymentSent(cfg.parentId, pr.paymentHash, randomBytes32, Seq(PaymentSent.PartialPayment(UUID.randomUUID(), 1000 msat, 500 msat, randomBytes32, None)))
val success = PaymentSent(cfg.parentId, pr.paymentHash, randomBytes32, finalAmount, c, Seq(PaymentSent.PartialPayment(UUID.randomUUID(), 1000 msat, 500 msat, randomBytes32, None)))
multiPartPayFsm.send(initiator, success)
sender.expectMsg(success)
eventListener.expectMsg(success)

View file

@ -28,7 +28,7 @@ import fr.acinq.eclair.blockchain.{UtxoStatus, ValidateRequest, ValidateResult,
import fr.acinq.eclair.channel.Register.ForwardShortId
import fr.acinq.eclair.channel.{AddHtlcFailed, Channel, ChannelUnavailable, Upstream}
import fr.acinq.eclair.crypto.Sphinx
import fr.acinq.eclair.db.{OutgoingPayment, OutgoingPaymentStatus}
import fr.acinq.eclair.db.{OutgoingPayment, OutgoingPaymentStatus, PaymentType}
import fr.acinq.eclair.io.Peer.PeerRoutingMessage
import fr.acinq.eclair.payment.PaymentRequest.ExtraHop
import fr.acinq.eclair.payment.PaymentSent.PartialPayment
@ -59,6 +59,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
val defaultPaymentRequest = SendPaymentRequest(defaultAmountMsat, defaultPaymentHash, d, 1, externalId = Some(defaultExternalId))
case class PaymentFixture(id: UUID,
parentId: UUID,
nodeParams: NodeParams,
paymentFSM: TestFSMRef[PaymentLifecycle.State, PaymentLifecycle.Data, PaymentLifecycle],
routerForwarder: TestProbe,
@ -68,15 +69,15 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
eventListener: TestProbe)
def createPaymentLifecycle(storeInDb: Boolean = true, publishEvent: Boolean = true): PaymentFixture = {
val id = UUID.randomUUID()
val (id, parentId) = (UUID.randomUUID(), UUID.randomUUID())
val nodeParams = TestConstants.Alice.nodeParams.copy(keyManager = testKeyManager)
val cfg = SendPaymentConfig(id, id, Some(defaultExternalId), defaultPaymentHash, d, Upstream.Local(id), defaultPaymentRequest.paymentRequest, storeInDb, publishEvent)
val cfg = SendPaymentConfig(id, parentId, Some(defaultExternalId), defaultPaymentHash, defaultAmountMsat, d, Upstream.Local(id), defaultPaymentRequest.paymentRequest, storeInDb, publishEvent, Nil)
val (routerForwarder, register, sender, monitor, eventListener) = (TestProbe(), TestProbe(), TestProbe(), TestProbe(), TestProbe())
val paymentFSM = TestFSMRef(new PaymentLifecycle(nodeParams, cfg, routerForwarder.ref, register.ref))
paymentFSM ! SubscribeTransitionCallBack(monitor.ref)
val CurrentState(_, WAITING_FOR_REQUEST) = monitor.expectMsgClass(classOf[CurrentState[_]])
system.eventStream.subscribe(eventListener.ref, classOf[PaymentEvent])
PaymentFixture(id, nodeParams, paymentFSM, routerForwarder, register, sender, monitor, eventListener)
PaymentFixture(id, parentId, nodeParams, paymentFSM, routerForwarder, register, sender, monitor, eventListener)
}
test("send to route") { routerFixture =>
@ -84,7 +85,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
import payFixture._
// pre-computed route going from A to D
val request = SendPaymentToRoute(defaultPaymentHash, Seq(a, b, c, d), FinalLegacyPayload(defaultAmountMsat, defaultExpiry))
val request = SendPaymentToRoute(Seq(a, b, c, d), FinalLegacyPayload(defaultAmountMsat, defaultExpiry))
sender.send(paymentFSM, request)
routerForwarder.expectMsg(FinalizeRoute(Seq(a, b, c, d)))
@ -94,10 +95,11 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
val Transition(_, WAITING_FOR_ROUTE, WAITING_FOR_PAYMENT_COMPLETE) = monitor.expectMsgClass(classOf[Transition[_]])
awaitCond(nodeParams.db.payments.getOutgoingPayment(id).exists(_.status == OutgoingPaymentStatus.Pending))
val Some(outgoing) = nodeParams.db.payments.getOutgoingPayment(id)
assert(outgoing.copy(createdAt = 0) === OutgoingPayment(id, id, Some(defaultExternalId), defaultPaymentHash, defaultAmountMsat, d, 0, None, OutgoingPaymentStatus.Pending))
assert(outgoing.copy(createdAt = 0) === OutgoingPayment(id, parentId, Some(defaultExternalId), defaultPaymentHash, PaymentType.Standard, defaultAmountMsat, defaultAmountMsat, d, 0, None, OutgoingPaymentStatus.Pending))
sender.send(paymentFSM, UpdateFulfillHtlc(ByteVector32.Zeroes, 0, defaultPaymentHash))
sender.expectMsgType[PaymentSent]
val ps = sender.expectMsgType[PaymentSent]
assert(ps.id === parentId)
awaitCond(nodeParams.db.payments.getOutgoingPayment(id).exists(_.status.isInstanceOf[OutgoingPaymentStatus.Succeeded]))
}
@ -105,7 +107,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
val payFixture = createPaymentLifecycle()
import payFixture._
val brokenRoute = SendPaymentToRoute(randomBytes32, Seq(randomKey.publicKey, randomKey.publicKey, randomKey.publicKey), FinalLegacyPayload(defaultAmountMsat, defaultExpiry))
val brokenRoute = SendPaymentToRoute(Seq(randomKey.publicKey, randomKey.publicKey, randomKey.publicKey), FinalLegacyPayload(defaultAmountMsat, defaultExpiry))
sender.send(paymentFSM, brokenRoute)
routerForwarder.expectMsgType[FinalizeRoute]
routerForwarder.forward(routerFixture.router)
@ -118,7 +120,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
val payFixture = createPaymentLifecycle()
import payFixture._
val request = SendPayment(defaultPaymentHash, d, FinalLegacyPayload(defaultAmountMsat, defaultExpiry), 3, routePrefix = Seq(ChannelHop(a, b, channelUpdate_ab), ChannelHop(b, c, channelUpdate_bc)))
val request = SendPayment(d, FinalLegacyPayload(defaultAmountMsat, defaultExpiry), 3, routePrefix = Seq(ChannelHop(a, b, channelUpdate_ab), ChannelHop(b, c, channelUpdate_bc)))
sender.send(paymentFSM, request)
routerForwarder.expectMsg(RouteRequest(c, d, defaultAmountMsat, ignoreNodes = Set(a, b)))
val Transition(_, WAITING_FOR_REQUEST, WAITING_FOR_ROUTE) = monitor.expectMsgClass(classOf[Transition[_]])
@ -132,7 +134,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
val payFixture = createPaymentLifecycle()
import payFixture._
val request = SendPayment(defaultPaymentHash, c, FinalLegacyPayload(defaultAmountMsat, defaultExpiry), 3, routePrefix = Seq(ChannelHop(a, b, channelUpdate_ab), ChannelHop(b, c, channelUpdate_bc)))
val request = SendPayment(c, FinalLegacyPayload(defaultAmountMsat, defaultExpiry), 3, routePrefix = Seq(ChannelHop(a, b, channelUpdate_ab), ChannelHop(b, c, channelUpdate_bc)))
sender.send(paymentFSM, request)
routerForwarder.expectNoMsg(50 millis) // we don't need the router when we already have the whole route
val Transition(_, WAITING_FOR_REQUEST, WAITING_FOR_ROUTE) = monitor.expectMsgClass(classOf[Transition[_]])
@ -144,7 +146,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
val payFixture = createPaymentLifecycle()
import payFixture._
val request = SendPayment(defaultPaymentHash, d, FinalLegacyPayload(defaultAmountMsat, defaultExpiry), 3, routePrefix = Seq(ChannelHop(a, b, channelUpdate_ab), ChannelHop(b, c, channelUpdate_bc)))
val request = SendPayment(d, FinalLegacyPayload(defaultAmountMsat, defaultExpiry), 3, routePrefix = Seq(ChannelHop(a, b, channelUpdate_ab), ChannelHop(b, c, channelUpdate_bc)))
sender.send(paymentFSM, request)
routerForwarder.expectMsg(RouteRequest(c, d, defaultAmountMsat, ignoreNodes = Set(a, b)))
val Transition(_, WAITING_FOR_REQUEST, WAITING_FOR_ROUTE) = monitor.expectMsgClass(classOf[Transition[_]])
@ -163,7 +165,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
val payFixture = createPaymentLifecycle()
import payFixture._
val request = SendPayment(defaultPaymentHash, f, FinalLegacyPayload(defaultAmountMsat, defaultExpiry), 5)
val request = SendPayment(f, FinalLegacyPayload(defaultAmountMsat, defaultExpiry), 5)
sender.send(paymentFSM, request)
val Transition(_, WAITING_FOR_REQUEST, WAITING_FOR_ROUTE) = monitor.expectMsgClass(classOf[Transition[_]])
val routeRequest = routerForwarder.expectMsgType[RouteRequest]
@ -178,7 +180,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
val payFixture = createPaymentLifecycle()
import payFixture._
val request = SendPayment(defaultPaymentHash, d, FinalLegacyPayload(defaultAmountMsat, defaultExpiry), 5, routeParams = Some(RouteParams(randomize = false, maxFeeBase = 100 msat, maxFeePct = 0.0, routeMaxLength = 20, routeMaxCltv = CltvExpiryDelta(2016), ratios = None)))
val request = SendPayment(d, FinalLegacyPayload(defaultAmountMsat, defaultExpiry), 5, routeParams = Some(RouteParams(randomize = false, maxFeeBase = 100 msat, maxFeePct = 0.0, routeMaxLength = 20, routeMaxCltv = CltvExpiryDelta(2016), ratios = None)))
sender.send(paymentFSM, request)
val routeRequest = routerForwarder.expectMsgType[RouteRequest]
val Transition(_, WAITING_FOR_REQUEST, WAITING_FOR_ROUTE) = monitor.expectMsgClass(classOf[Transition[_]])
@ -192,7 +194,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
val payFixture = createPaymentLifecycle()
import payFixture._
val request = SendPayment(defaultPaymentHash, d, FinalLegacyPayload(defaultAmountMsat, defaultExpiry), 2)
val request = SendPayment(d, FinalLegacyPayload(defaultAmountMsat, defaultExpiry), 2)
sender.send(paymentFSM, request)
routerForwarder.expectMsg(RouteRequest(a, d, defaultAmountMsat, ignoreNodes = Set.empty, ignoreChannels = Set.empty))
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && nodeParams.db.payments.getOutgoingPayment(id).exists(_.status === OutgoingPaymentStatus.Pending))
@ -225,7 +227,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
val payFixture = createPaymentLifecycle()
import payFixture._
val request = SendPayment(defaultPaymentHash, d, FinalLegacyPayload(defaultAmountMsat, defaultExpiry), 2)
val request = SendPayment(d, FinalLegacyPayload(defaultAmountMsat, defaultExpiry), 2)
sender.send(paymentFSM, request)
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && nodeParams.db.payments.getOutgoingPayment(id).exists(_.status === OutgoingPaymentStatus.Pending))
@ -236,7 +238,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
val WaitingForComplete(_, _, cmd1, Nil, _, _, _, _) = paymentFSM.stateData
register.expectMsg(ForwardShortId(channelId_ab, cmd1))
sender.send(paymentFSM, Status.Failure(AddHtlcFailed(ByteVector32.Zeroes, request.paymentHash, ChannelUnavailable(ByteVector32.Zeroes), Local(id, Some(paymentFSM.underlying.self)), None, None)))
sender.send(paymentFSM, Status.Failure(AddHtlcFailed(ByteVector32.Zeroes, defaultPaymentHash, ChannelUnavailable(ByteVector32.Zeroes), Local(id, Some(paymentFSM.underlying.self)), None, None)))
// then the payment lifecycle will ask for a new route excluding the channel
routerForwarder.expectMsg(RouteRequest(nodeParams.nodeId, d, defaultAmountMsat, assistedRoutes = Nil, ignoreNodes = Set.empty, ignoreChannels = Set(ChannelDesc(channelId_ab, a, b))))
@ -247,7 +249,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
val payFixture = createPaymentLifecycle()
import payFixture._
val request = SendPayment(defaultPaymentHash, d, FinalLegacyPayload(defaultAmountMsat, defaultExpiry), 2)
val request = SendPayment(d, FinalLegacyPayload(defaultAmountMsat, defaultExpiry), 2)
sender.send(paymentFSM, request)
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && nodeParams.db.payments.getOutgoingPayment(id).exists(_.status === OutgoingPaymentStatus.Pending))
@ -269,7 +271,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
val payFixture = createPaymentLifecycle()
import payFixture._
val request = SendPayment(defaultPaymentHash, d, FinalLegacyPayload(defaultAmountMsat, defaultExpiry), 2)
val request = SendPayment(d, FinalLegacyPayload(defaultAmountMsat, defaultExpiry), 2)
sender.send(paymentFSM, request)
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE)
val WaitingForRoute(_, _, Nil) = paymentFSM.stateData
@ -298,7 +300,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
val payFixture = createPaymentLifecycle()
import payFixture._
val request = SendPayment(defaultPaymentHash, d, FinalLegacyPayload(defaultAmountMsat, defaultExpiry), 5)
val request = SendPayment(d, FinalLegacyPayload(defaultAmountMsat, defaultExpiry), 5)
sender.send(paymentFSM, request)
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && nodeParams.db.payments.getOutgoingPayment(id).exists(_.status === OutgoingPaymentStatus.Pending))
@ -356,7 +358,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
ExtraHop(c, channelId_cd, channelUpdate_cd.feeBaseMsat, channelUpdate_cd.feeProportionalMillionths, channelUpdate_cd.cltvExpiryDelta)
))
val request = SendPayment(defaultPaymentHash, d, FinalLegacyPayload(defaultAmountMsat, defaultExpiry), 5, assistedRoutes = assistedRoutes)
val request = SendPayment(d, FinalLegacyPayload(defaultAmountMsat, defaultExpiry), 5, assistedRoutes = assistedRoutes)
sender.send(paymentFSM, request)
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && nodeParams.db.payments.getOutgoingPayment(id).exists(_.status === OutgoingPaymentStatus.Pending))
@ -394,7 +396,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
val payFixture = createPaymentLifecycle()
import payFixture._
val request = SendPayment(defaultPaymentHash, d, FinalLegacyPayload(defaultAmountMsat, defaultExpiry), 2)
val request = SendPayment(d, FinalLegacyPayload(defaultAmountMsat, defaultExpiry), 2)
sender.send(paymentFSM, request)
awaitCond(paymentFSM.stateName == WAITING_FOR_ROUTE && nodeParams.db.payments.getOutgoingPayment(id).exists(_.status === OutgoingPaymentStatus.Pending))
@ -431,7 +433,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
val payFixture = createPaymentLifecycle()
import payFixture._
val request = SendPayment(defaultPaymentHash, d, FinalLegacyPayload(defaultAmountMsat, defaultExpiry), 5)
val request = SendPayment(d, FinalLegacyPayload(defaultAmountMsat, defaultExpiry), 5)
sender.send(paymentFSM, request)
routerForwarder.expectMsgType[RouteRequest]
val Transition(_, WAITING_FOR_REQUEST, WAITING_FOR_ROUTE) = monitor.expectMsgClass(classOf[Transition[_]])
@ -439,14 +441,16 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
val Transition(_, WAITING_FOR_ROUTE, WAITING_FOR_PAYMENT_COMPLETE) = monitor.expectMsgClass(classOf[Transition[_]])
awaitCond(nodeParams.db.payments.getOutgoingPayment(id).exists(_.status === OutgoingPaymentStatus.Pending))
val Some(outgoing) = nodeParams.db.payments.getOutgoingPayment(id)
assert(outgoing.copy(createdAt = 0) === OutgoingPayment(id, id, Some(defaultExternalId), defaultPaymentHash, defaultAmountMsat, d, 0, None, OutgoingPaymentStatus.Pending))
assert(outgoing.copy(createdAt = 0) === OutgoingPayment(id, parentId, Some(defaultExternalId), defaultPaymentHash, PaymentType.Standard, defaultAmountMsat, defaultAmountMsat, d, 0, None, OutgoingPaymentStatus.Pending))
sender.send(paymentFSM, UpdateFulfillHtlc(ByteVector32.Zeroes, 0, defaultPaymentPreimage))
val ps = eventListener.expectMsgType[PaymentSent]
assert(ps.id === parentId)
assert(ps.feesPaid > 0.msat)
assert(ps.amount === defaultAmountMsat)
assert(ps.recipientAmount === defaultAmountMsat)
assert(ps.paymentHash === defaultPaymentHash)
assert(ps.paymentPreimage === defaultPaymentPreimage)
assert(ps.parts.head.id === id)
awaitCond(nodeParams.db.payments.getOutgoingPayment(id).exists(_.status.isInstanceOf[OutgoingPaymentStatus.Succeeded]))
}
@ -477,7 +481,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
import payFixture._
// we send a payment to G
val request = SendPayment(defaultPaymentHash, g, FinalLegacyPayload(defaultAmountMsat, defaultExpiry), 5)
val request = SendPayment(g, FinalLegacyPayload(defaultAmountMsat, defaultExpiry), 5)
sender.send(paymentFSM, request)
routerForwarder.expectMsgType[RouteRequest]
@ -488,13 +492,14 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
sender.send(paymentFSM, UpdateFulfillHtlc(ByteVector32.Zeroes, 0, defaultPaymentHash))
val paymentOK = sender.expectMsgType[PaymentSent]
val PaymentSent(_, request.paymentHash, paymentOK.paymentPreimage, PartialPayment(_, request.finalPayload.amount, fee, ByteVector32.Zeroes, _, _) :: Nil) = eventListener.expectMsgType[PaymentSent]
val PaymentSent(_, _, paymentOK.paymentPreimage, finalAmount, _, PartialPayment(_, request.finalPayload.amount, fee, ByteVector32.Zeroes, _, _) :: Nil) = eventListener.expectMsgType[PaymentSent]
assert(finalAmount === defaultAmountMsat)
// during the route computation the fees were treated as if they were 1msat but when sending the onion we actually put zero
// NB: A -> B doesn't pay fees because it's our direct neighbor
// NB: B -> G doesn't asks for fees at all
assert(fee === 0.msat)
assert(paymentOK.amount === request.finalPayload.amount)
assert(paymentOK.recipientAmount === request.finalPayload.amount)
}
test("filter errors properly") { _ =>
@ -507,7 +512,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
val payFixture = createPaymentLifecycle(storeInDb = false, publishEvent = false)
import payFixture._
val request = SendPayment(defaultPaymentHash, d, FinalLegacyPayload(defaultAmountMsat, defaultExpiry), 3)
val request = SendPayment(d, FinalLegacyPayload(defaultAmountMsat, defaultExpiry), 3)
sender.send(paymentFSM, request)
routerForwarder.expectMsgType[RouteRequest]
val Transition(_, WAITING_FOR_REQUEST, WAITING_FOR_ROUTE) = monitor.expectMsgClass(classOf[Transition[_]])

View file

@ -22,7 +22,7 @@ import akka.actor.ActorRef
import akka.testkit.TestProbe
import fr.acinq.bitcoin.{Block, ByteVector32, Crypto}
import fr.acinq.eclair.channel._
import fr.acinq.eclair.db.{OutgoingPayment, OutgoingPaymentStatus}
import fr.acinq.eclair.db.{OutgoingPayment, OutgoingPaymentStatus, PaymentType}
import fr.acinq.eclair.payment.OutgoingPacket.buildCommand
import fr.acinq.eclair.payment.PaymentPacketSpec._
import fr.acinq.eclair.payment.relay.Relayer.{ForwardFail, ForwardFulfill}
@ -257,7 +257,8 @@ class PostRestartHtlcCleanerSpec extends TestkitBaseClass {
assert(e1.paymentPreimage === preimage2)
assert(e1.paymentHash === paymentHash2)
assert(e1.parts.length === 2)
assert(e1.amount === 2834.msat)
assert(e1.amountWithFees === 2834.msat)
assert(e1.recipientAmount === 2500.msat)
assert(nodeParams.db.payments.getOutgoingPayment(testCase.childIds(1)).get.status.isInstanceOf[OutgoingPaymentStatus.Succeeded])
assert(nodeParams.db.payments.getOutgoingPayment(testCase.childIds(2)).get.status.isInstanceOf[OutgoingPaymentStatus.Succeeded])
assert(nodeParams.db.payments.getOutgoingPayment(testCase.childIds.head).get.status === OutgoingPaymentStatus.Pending)
@ -268,7 +269,7 @@ class PostRestartHtlcCleanerSpec extends TestkitBaseClass {
assert(e2.paymentPreimage === preimage1)
assert(e2.paymentHash === paymentHash1)
assert(e2.parts.length === 1)
assert(e2.amount === 561.msat)
assert(e2.recipientAmount === 561.msat)
assert(nodeParams.db.payments.getOutgoingPayment(testCase.childIds.head).get.status.isInstanceOf[OutgoingPaymentStatus.Succeeded])
commandBuffer.expectNoMsg(100 millis)
@ -402,9 +403,9 @@ object PostRestartHtlcCleanerSpec {
val origin3 = Origin.Local(id3, None)
// Prepare channels and payment state before restart.
nodeParams.db.payments.addOutgoingPayment(OutgoingPayment(id1, id1, None, paymentHash1, add1.amountMsat, c, 0, None, OutgoingPaymentStatus.Pending))
nodeParams.db.payments.addOutgoingPayment(OutgoingPayment(id2, parentId, None, paymentHash2, add2.amountMsat, c, 0, None, OutgoingPaymentStatus.Pending))
nodeParams.db.payments.addOutgoingPayment(OutgoingPayment(id3, parentId, None, paymentHash2, add3.amountMsat, c, 0, None, OutgoingPaymentStatus.Pending))
nodeParams.db.payments.addOutgoingPayment(OutgoingPayment(id1, id1, None, paymentHash1, PaymentType.Standard, add1.amountMsat, add1.amountMsat, c, 0, None, OutgoingPaymentStatus.Pending))
nodeParams.db.payments.addOutgoingPayment(OutgoingPayment(id2, parentId, None, paymentHash2, PaymentType.Standard, add2.amountMsat, 2500 msat, c, 0, None, OutgoingPaymentStatus.Pending))
nodeParams.db.payments.addOutgoingPayment(OutgoingPayment(id3, parentId, None, paymentHash2, PaymentType.Standard, add3.amountMsat, 2500 msat, c, 0, None, OutgoingPaymentStatus.Pending))
nodeParams.db.channels.addOrUpdateChannel(ChannelCodecsSpec.makeChannelDataNormal(
Seq(add1, add2, add3).map(add => DirectedHtlc(OUT, add)),
Map(add1.id -> origin1, add2.id -> origin2, add3.id -> origin3))

View file

@ -210,8 +210,8 @@ class GUIUpdater(mainController: MainController) extends Actor with ActorLogging
mainController.handlers.notification("Payment Failed", message, NOTIFICATION_ERROR)
case p: PaymentSent =>
log.debug(s"payment sent with h=${p.paymentHash}, amount=${p.amount}, fees=${p.feesPaid}")
val message = CoinUtils.formatAmountInUnit(p.amount + p.feesPaid, FxApp.getUnit, withUnit = true)
log.debug(s"payment sent with h=${p.paymentHash}, amount=${p.recipientAmount}, fees=${p.feesPaid}")
val message = CoinUtils.formatAmountInUnit(p.amountWithFees, FxApp.getUnit, withUnit = true)
mainController.handlers.notification("Payment Sent", message, NOTIFICATION_SUCCESS)
runInGuiThread(() => mainController.paymentSentList.prepend(PaymentSentRecord(p, LocalDateTime.now())))

View file

@ -310,7 +310,7 @@ class MainController(val handlers: Handlers, val hostServices: HostServices) ext
override def onChanged(c: Change[_ <: PaymentSentRecord]) = updateTabHeader(paymentSentTab, "Sent", paymentSentList)
})
paymentSentAmountColumn.setCellValueFactory(new Callback[CellDataFeatures[PaymentSentRecord, String], ObservableValue[String]]() {
def call(record: CellDataFeatures[PaymentSentRecord, String]) = new SimpleStringProperty(CoinUtils.formatAmountInUnit(record.getValue.event.amount, FxApp.getUnit, withUnit = true))
def call(record: CellDataFeatures[PaymentSentRecord, String]) = new SimpleStringProperty(CoinUtils.formatAmountInUnit(record.getValue.event.recipientAmount, FxApp.getUnit, withUnit = true))
})
paymentSentFeesColumn.setCellValueFactory(new Callback[CellDataFeatures[PaymentSentRecord, String], ObservableValue[String]]() {
def call(record: CellDataFeatures[PaymentSentRecord, String]) = new SimpleStringProperty(CoinUtils.formatAmountInUnit(record.getValue.event.feesPaid, FxApp.getUnit, withUnit = true))

View file

@ -261,9 +261,11 @@ object CustomTypeHints {
classOf[OutgoingPaymentStatus.Succeeded] -> "sent"
))
// TODO: @t-bast: don't forget to update slate-doc
val paymentEvent = CustomTypeHints(Map(
classOf[PaymentSent] -> "payment-sent",
classOf[ChannelPaymentRelayed] -> "payment-relayed",
classOf[TrampolinePaymentRelayed] -> "trampoline-payment-relayed",
classOf[PaymentReceived] -> "payment-received",
classOf[PaymentSettlingOnChain] -> "payment-settling-onchain",
classOf[PaymentFailed] -> "payment-failed"

View file

@ -222,6 +222,11 @@ trait Service extends ExtraDirectives with Logging {
}
} ~
// TODO: @t-bast: remove this API once stabilized: should re-work the payment APIs to integrate Trampoline nicely
// - payinvoice: should stay the same, it's the easy flow where the node does its magic
// - sendtonode: exactly like payinvoice but without an invoice: let the node do its magic
// - sendtoroute: no path-finding, lets the user control exactly how to send (provide multiple routes, with trampoline or not, etc) -> maybe doesn't go through normal PayFSM (avoid retries)
// -> maybe somehow make one call per partial HTLC (allows easier failure reporting and out-of-node retry logic)?
// -> needs both trampolineRoute and routeToTrampoline arguments?
path("sendtotrampoline") {
formFields(invoiceFormParam, "trampolineId".as[PublicKey], "trampolineFeesMsat".as[MilliSatoshi], "trampolineExpiryDelta".as[Int]) {
(invoice, trampolineId, trampolineFees, trampolineExpiryDelta) =>

View file

@ -1 +1 @@
{"paymentRequest":{"prefix":"lnbc","timestamp":1496314658,"nodeId":"03e7156ae33b0a208d0744199163177e909e80176e55d97a2f221ede0f934dd9ad","serialized":"lnbc2500u1pvjluezpp5qqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqypqdq5xysxxatsyp3k7enxv4jsxqzpuaztrnwngzn3kdzw5hydlzf03qdgm2hdq27cqv3agm2awhz5se903vruatfhq77w3ls4evs3ch9zw97j25emudupq63nyw24cg27h2rspfj9srp","description":"1 cup coffee","paymentHash":"0001020304050607080900010203040506070809000102030405060708090102","expiry":60,"amount":250000000},"paymentPreimage":"0100000000000000000000000000000000000000000000000000000000000000","createdAt":42,"status":{"type":"expired"}}
{"paymentRequest":{"prefix":"lnbc","timestamp":1496314658,"nodeId":"03e7156ae33b0a208d0744199163177e909e80176e55d97a2f221ede0f934dd9ad","serialized":"lnbc2500u1pvjluezpp5qqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqypqdq5xysxxatsyp3k7enxv4jsxqzpuaztrnwngzn3kdzw5hydlzf03qdgm2hdq27cqv3agm2awhz5se903vruatfhq77w3ls4evs3ch9zw97j25emudupq63nyw24cg27h2rspfj9srp","description":"1 cup coffee","paymentHash":"0001020304050607080900010203040506070809000102030405060708090102","expiry":60,"amount":250000000},"paymentPreimage":"0100000000000000000000000000000000000000000000000000000000000000","paymentType":"Standard","createdAt":42,"status":{"type":"expired"}}

View file

@ -1 +1 @@
{"paymentRequest":{"prefix":"lnbc","timestamp":1496314658,"nodeId":"03e7156ae33b0a208d0744199163177e909e80176e55d97a2f221ede0f934dd9ad","serialized":"lnbc2500u1pvjluezpp5qqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqypqdq5xysxxatsyp3k7enxv4jsxqzpuaztrnwngzn3kdzw5hydlzf03qdgm2hdq27cqv3agm2awhz5se903vruatfhq77w3ls4evs3ch9zw97j25emudupq63nyw24cg27h2rspfj9srp","description":"1 cup coffee","paymentHash":"0001020304050607080900010203040506070809000102030405060708090102","expiry":60,"amount":250000000},"paymentPreimage":"0100000000000000000000000000000000000000000000000000000000000000","createdAt":42,"status":{"type":"pending"}}
{"paymentRequest":{"prefix":"lnbc","timestamp":1496314658,"nodeId":"03e7156ae33b0a208d0744199163177e909e80176e55d97a2f221ede0f934dd9ad","serialized":"lnbc2500u1pvjluezpp5qqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqypqdq5xysxxatsyp3k7enxv4jsxqzpuaztrnwngzn3kdzw5hydlzf03qdgm2hdq27cqv3agm2awhz5se903vruatfhq77w3ls4evs3ch9zw97j25emudupq63nyw24cg27h2rspfj9srp","description":"1 cup coffee","paymentHash":"0001020304050607080900010203040506070809000102030405060708090102","expiry":60,"amount":250000000},"paymentPreimage":"0100000000000000000000000000000000000000000000000000000000000000","paymentType":"Standard","createdAt":42,"status":{"type":"pending"}}

View file

@ -1 +1 @@
{"paymentRequest":{"prefix":"lnbc","timestamp":1496314658,"nodeId":"03e7156ae33b0a208d0744199163177e909e80176e55d97a2f221ede0f934dd9ad","serialized":"lnbc2500u1pvjluezpp5qqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqypqdq5xysxxatsyp3k7enxv4jsxqzpuaztrnwngzn3kdzw5hydlzf03qdgm2hdq27cqv3agm2awhz5se903vruatfhq77w3ls4evs3ch9zw97j25emudupq63nyw24cg27h2rspfj9srp","description":"1 cup coffee","paymentHash":"0001020304050607080900010203040506070809000102030405060708090102","expiry":60,"amount":250000000},"paymentPreimage":"0100000000000000000000000000000000000000000000000000000000000000","createdAt":42,"status":{"type":"received","amount":42,"receivedAt":45}}
{"paymentRequest":{"prefix":"lnbc","timestamp":1496314658,"nodeId":"03e7156ae33b0a208d0744199163177e909e80176e55d97a2f221ede0f934dd9ad","serialized":"lnbc2500u1pvjluezpp5qqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqypqdq5xysxxatsyp3k7enxv4jsxqzpuaztrnwngzn3kdzw5hydlzf03qdgm2hdq27cqv3agm2awhz5se903vruatfhq77w3ls4evs3ch9zw97j25emudupq63nyw24cg27h2rspfj9srp","description":"1 cup coffee","paymentHash":"0001020304050607080900010203040506070809000102030405060708090102","expiry":60,"amount":250000000},"paymentPreimage":"0100000000000000000000000000000000000000000000000000000000000000","paymentType":"Standard","createdAt":42,"status":{"type":"received","amount":42,"receivedAt":45}}

View file

@ -1 +1 @@
[{"id":"00000000-0000-0000-0000-000000000000","parentId":"11111111-1111-1111-1111-111111111111","paymentHash":"0000000000000000000000000000000000000000000000000000000000000000","amount":42,"targetNodeId":"03af0ed6052cf28d670665549bc86f4b721c9fdb309d40c58f5811f63966e005d0","createdAt":1,"status":{"type":"failed","failures":[],"completedAt":2}}]
[{"id":"00000000-0000-0000-0000-000000000000","parentId":"11111111-1111-1111-1111-111111111111","paymentHash":"0000000000000000000000000000000000000000000000000000000000000000","paymentType":"Standard","amount":42,"recipientAmount":50,"recipientNodeId":"03af0ed6052cf28d670665549bc86f4b721c9fdb309d40c58f5811f63966e005d0","createdAt":1,"status":{"type":"failed","failures":[],"completedAt":2}}]

View file

@ -1 +1 @@
[{"id":"00000000-0000-0000-0000-000000000000","parentId":"11111111-1111-1111-1111-111111111111","paymentHash":"0000000000000000000000000000000000000000000000000000000000000000","amount":42,"targetNodeId":"03af0ed6052cf28d670665549bc86f4b721c9fdb309d40c58f5811f63966e005d0","createdAt":1,"status":{"type":"pending"}}]
[{"id":"00000000-0000-0000-0000-000000000000","parentId":"11111111-1111-1111-1111-111111111111","paymentHash":"0000000000000000000000000000000000000000000000000000000000000000","paymentType":"Standard","amount":42,"recipientAmount":50,"recipientNodeId":"03af0ed6052cf28d670665549bc86f4b721c9fdb309d40c58f5811f63966e005d0","createdAt":1,"status":{"type":"pending"}}]

View file

@ -1 +1 @@
[{"id":"00000000-0000-0000-0000-000000000000","parentId":"11111111-1111-1111-1111-111111111111","paymentHash":"0000000000000000000000000000000000000000000000000000000000000000","amount":42,"targetNodeId":"03af0ed6052cf28d670665549bc86f4b721c9fdb309d40c58f5811f63966e005d0","createdAt":1,"status":{"type":"sent","paymentPreimage":"0100000000000000000000000000000000000000000000000000000000000000","feesPaid":5,"route":[],"completedAt":3}}]
[{"id":"00000000-0000-0000-0000-000000000000","parentId":"11111111-1111-1111-1111-111111111111","paymentHash":"0000000000000000000000000000000000000000000000000000000000000000","paymentType":"Standard","amount":42,"recipientAmount":50,"recipientNodeId":"03af0ed6052cf28d670665549bc86f4b721c9fdb309d40c58f5811f63966e005d0","createdAt":1,"status":{"type":"sent","paymentPreimage":"0100000000000000000000000000000000000000000000000000000000000000","feesPaid":5,"route":[],"completedAt":3}}]

View file

@ -30,7 +30,7 @@ import de.heikoseeberger.akkahttpjson4s.Json4sSupport
import fr.acinq.bitcoin.Crypto.PublicKey
import fr.acinq.bitcoin.{Block, ByteVector32}
import fr.acinq.eclair._
import fr.acinq.eclair.db.{IncomingPayment, IncomingPaymentStatus, OutgoingPayment, OutgoingPaymentStatus}
import fr.acinq.eclair.db.{IncomingPayment, IncomingPaymentStatus, OutgoingPayment, OutgoingPaymentStatus, PaymentType}
import fr.acinq.eclair.io.NodeURI
import fr.acinq.eclair.io.Peer.PeerInfo
import fr.acinq.eclair.payment.relay.Relayer.UsableBalance
@ -297,7 +297,7 @@ class ApiServiceSpec extends FunSuite with ScalatestRouteTest with IdiomaticMock
test("'getreceivedinfo'") {
val invoice = "lnbc2500u1pvjluezpp5qqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqypqdq5xysxxatsyp3k7enxv4jsxqzpuaztrnwngzn3kdzw5hydlzf03qdgm2hdq27cqv3agm2awhz5se903vruatfhq77w3ls4evs3ch9zw97j25emudupq63nyw24cg27h2rspfj9srp"
val defaultPayment = IncomingPayment(PaymentRequest.read(invoice), ByteVector32.One, 42, IncomingPaymentStatus.Pending)
val defaultPayment = IncomingPayment(PaymentRequest.read(invoice), ByteVector32.One, PaymentType.Standard, 42, IncomingPaymentStatus.Pending)
val eclair = mock[Eclair]
val notFound = randomBytes32
eclair.receivedInfo(notFound)(any) returns Future.successful(None)
@ -355,7 +355,7 @@ class ApiServiceSpec extends FunSuite with ScalatestRouteTest with IdiomaticMock
}
test("'getsentinfo'") {
val defaultPayment = OutgoingPayment(UUID.fromString("00000000-0000-0000-0000-000000000000"), UUID.fromString("11111111-1111-1111-1111-111111111111"), None, ByteVector32.Zeroes, 42 msat, aliceNodeId, 1, None, OutgoingPaymentStatus.Pending)
val defaultPayment = OutgoingPayment(UUID.fromString("00000000-0000-0000-0000-000000000000"), UUID.fromString("11111111-1111-1111-1111-111111111111"), None, ByteVector32.Zeroes, PaymentType.Standard, 42 msat, 50 msat, aliceNodeId, 1, None, OutgoingPaymentStatus.Pending)
val eclair = mock[Eclair]
val pending = UUID.randomUUID()
eclair.sentInfo(Left(pending))(any) returns Future.successful(Seq(defaultPayment))
@ -474,25 +474,31 @@ class ApiServiceSpec extends FunSuite with ScalatestRouteTest with IdiomaticMock
system.eventStream.publish(pf)
wsClient.expectMessage(expectedSerializedPf)
val ps = PaymentSent(fixedUUID, ByteVector32.Zeroes, ByteVector32.One, Seq(PaymentSent.PartialPayment(fixedUUID, 21 msat, 1 msat, ByteVector32.Zeroes, None, 1553784337711L)))
val expectedSerializedPs = """{"type":"payment-sent","id":"487da196-a4dc-4b1e-92b4-3e5e905e9f3f","paymentHash":"0000000000000000000000000000000000000000000000000000000000000000","paymentPreimage":"0100000000000000000000000000000000000000000000000000000000000000","parts":[{"id":"487da196-a4dc-4b1e-92b4-3e5e905e9f3f","amount":21,"feesPaid":1,"toChannelId":"0000000000000000000000000000000000000000000000000000000000000000","timestamp":1553784337711}]}"""
val ps = PaymentSent(fixedUUID, ByteVector32.Zeroes, ByteVector32.One, 25 msat, aliceNodeId, Seq(PaymentSent.PartialPayment(fixedUUID, 21 msat, 1 msat, ByteVector32.Zeroes, None, 1553784337711L)))
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":1553784337711}]}"""
assert(serialization.write(ps) === expectedSerializedPs)
system.eventStream.publish(ps)
wsClient.expectMessage(expectedSerializedPs)
val prel = ChannelPaymentRelayed(amountIn = 21 msat, amountOut = 20 msat, paymentHash = ByteVector32.Zeroes, fromChannelId = ByteVector32.Zeroes, ByteVector32.One, timestamp = 1553784963659L)
val prel = ChannelPaymentRelayed(21 msat, 20 msat, ByteVector32.Zeroes, ByteVector32.Zeroes, ByteVector32.One, 1553784963659L)
val expectedSerializedPrel = """{"type":"payment-relayed","amountIn":21,"amountOut":20,"paymentHash":"0000000000000000000000000000000000000000000000000000000000000000","fromChannelId":"0000000000000000000000000000000000000000000000000000000000000000","toChannelId":"0100000000000000000000000000000000000000000000000000000000000000","timestamp":1553784963659}"""
assert(serialization.write(prel) === expectedSerializedPrel)
system.eventStream.publish(prel)
wsClient.expectMessage(expectedSerializedPrel)
val ptrel = TrampolinePaymentRelayed(ByteVector32.Zeroes, Seq(PaymentRelayed.Part(21 msat, ByteVector32.Zeroes)), Seq(PaymentRelayed.Part(8 msat, ByteVector32.Zeroes), PaymentRelayed.Part(10 msat, ByteVector32.One)), 1553784963659L)
val expectedSerializedPtrel = """{"type":"trampoline-payment-relayed","paymentHash":"0000000000000000000000000000000000000000000000000000000000000000","incoming":[{"amount":21,"channelId":"0000000000000000000000000000000000000000000000000000000000000000"}],"outgoing":[{"amount":8,"channelId":"0000000000000000000000000000000000000000000000000000000000000000"},{"amount":10,"channelId":"0100000000000000000000000000000000000000000000000000000000000000"}],"timestamp":1553784963659}"""
assert(serialization.write(ptrel) === expectedSerializedPtrel)
system.eventStream.publish(ptrel)
wsClient.expectMessage(expectedSerializedPtrel)
val precv = PaymentReceived(ByteVector32.Zeroes, Seq(PaymentReceived.PartialPayment(21 msat, ByteVector32.Zeroes, 1553784963659L)))
val expectedSerializedPrecv = """{"type":"payment-received","paymentHash":"0000000000000000000000000000000000000000000000000000000000000000","parts":[{"amount":21,"fromChannelId":"0000000000000000000000000000000000000000000000000000000000000000","timestamp":1553784963659}]}"""
assert(serialization.write(precv) === expectedSerializedPrecv)
system.eventStream.publish(precv)
wsClient.expectMessage(expectedSerializedPrecv)
val pset = PaymentSettlingOnChain(fixedUUID, amount = 21 msat, paymentHash = ByteVector32.One, timestamp = 1553785442676L)
val pset = PaymentSettlingOnChain(fixedUUID, 21 msat, ByteVector32.One, timestamp = 1553785442676L)
val expectedSerializedPset = """{"type":"payment-settling-onchain","id":"487da196-a4dc-4b1e-92b4-3e5e905e9f3f","amount":21,"paymentHash":"0100000000000000000000000000000000000000000000000000000000000000","timestamp":1553785442676}"""
assert(serialization.write(pset) === expectedSerializedPset)
system.eventStream.publish(pset)