From d4261f5f4f6cbfc15f462503c6a2459f64d8a7b4 Mon Sep 17 00:00:00 2001 From: Thomas HUET Date: Wed, 20 Nov 2024 12:09:05 +0100 Subject: [PATCH 1/7] Basic offer management without plugins --- eclair-core/src/main/resources/reference.conf | 8 + .../main/scala/fr/acinq/eclair/Eclair.scala | 28 +++- .../scala/fr/acinq/eclair/NodeParams.scala | 10 +- .../main/scala/fr/acinq/eclair/Setup.scala | 6 +- .../scala/fr/acinq/eclair/db/Databases.scala | 5 + .../fr/acinq/eclair/db/DualDatabases.scala | 26 ++++ .../scala/fr/acinq/eclair/db/OffersDb.scala | 51 +++++++ .../fr/acinq/eclair/db/pg/PgOffersDb.scala | 113 ++++++++++++++ .../eclair/db/sqlite/SqliteOffersDb.scala | 99 +++++++++++++ .../eclair/payment/offer/DefaultHandler.scala | 79 ++++++++++ .../eclair/payment/offer/OfferCreator.scala | 105 +++++++++++++ .../eclair/payment/offer/OfferManager.scala | 30 ++-- .../acinq/eclair/payment/relay/Relayer.scala | 5 +- .../payment/send/BlindedPathsResolver.scala | 5 +- .../fr/acinq/eclair/EclairImplSpec.scala | 2 + .../scala/fr/acinq/eclair/TestConstants.scala | 7 +- .../scala/fr/acinq/eclair/TestDatabases.scala | 1 + .../fr/acinq/eclair/db/OffersDbSpec.scala | 70 +++++++++ .../integration/PaymentIntegrationSpec.scala | 40 ++--- .../basic/fixtures/MinimalNodeFixture.scala | 9 +- .../basic/payment/OfferPaymentSpec.scala | 140 +++++++++++++----- .../payment/offer/OfferManagerSpec.scala | 11 +- 22 files changed, 759 insertions(+), 91 deletions(-) create mode 100644 eclair-core/src/main/scala/fr/acinq/eclair/db/OffersDb.scala create mode 100644 eclair-core/src/main/scala/fr/acinq/eclair/db/pg/PgOffersDb.scala create mode 100644 eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteOffersDb.scala create mode 100644 eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/DefaultHandler.scala create mode 100644 eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/OfferCreator.scala create mode 100644 eclair-core/src/test/scala/fr/acinq/eclair/db/OffersDbSpec.scala diff --git a/eclair-core/src/main/resources/reference.conf b/eclair-core/src/main/resources/reference.conf index 403c8d5e3f..6e32d5165c 100644 --- a/eclair-core/src/main/resources/reference.conf +++ b/eclair-core/src/main/resources/reference.conf @@ -636,6 +636,14 @@ eclair { // Frequency at which we clean our DB to remove peer storage from nodes with whom we don't have channels anymore. cleanup-frequency = 1 day } + + managed-offers { + message-path-min-length = 2 + + payment-path-count = 2 + payment-path-length = 4 + payment-path-expiry-delta = 500 + } } akka { diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala b/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala index 83f1e20669..708dc5c722 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala @@ -39,15 +39,17 @@ import fr.acinq.eclair.db.AuditDb.{NetworkFee, Stats} import fr.acinq.eclair.db.{IncomingPayment, OutgoingPayment, OutgoingPaymentStatus} import fr.acinq.eclair.io.Peer.{GetPeerInfo, OpenChannelResponse, PeerInfo} import fr.acinq.eclair.io._ +import fr.acinq.eclair.message.OnionMessages.{IntermediateNode, Recipient} import fr.acinq.eclair.message.{OnionMessages, Postman} import fr.acinq.eclair.payment._ +import fr.acinq.eclair.payment.offer.{OfferCreator, OfferManager} import fr.acinq.eclair.payment.receive.MultiPartHandler.ReceiveStandardPayment import fr.acinq.eclair.payment.relay.Relayer.{ChannelBalance, GetOutgoingChannels, OutgoingChannels, RelayFees} import fr.acinq.eclair.payment.send.PaymentInitiator._ import fr.acinq.eclair.payment.send.{ClearRecipient, OfferPayment, PaymentIdentifier} import fr.acinq.eclair.router.Router import fr.acinq.eclair.router.Router._ -import fr.acinq.eclair.wire.protocol.OfferTypes.Offer +import fr.acinq.eclair.wire.protocol.OfferTypes.{Offer, OfferAbsoluteExpiry, OfferIssuer, OfferQuantityMax, OfferTlv} import fr.acinq.eclair.wire.protocol._ import grizzled.slf4j.Logging import scodec.bits.ByteVector @@ -126,6 +128,12 @@ trait Eclair { def receive(description: Either[String, ByteVector32], amount_opt: Option[MilliSatoshi], expire_opt: Option[Long], fallbackAddress_opt: Option[String], paymentPreimage_opt: Option[ByteVector32], privateChannelIds_opt: Option[List[ByteVector32]])(implicit timeout: Timeout): Future[Bolt11Invoice] + def createOffer(description_opt: Option[String], amount_opt: Option[MilliSatoshi], expiry_opt: Option[TimestampSecond], issuer_opt: Option[String], firstNodeId_opt: Option[PublicKey])(implicit timeout: Timeout): Future[Offer] + + def disableOffer(offer: Offer)(implicit timeout: Timeout): Unit + + def listOffers(onlyActive: Boolean = true)(implicit timeout: Timeout): Future[Seq[Offer]] + def newAddress(): Future[String] def receivedInfo(paymentHash: ByteVector32)(implicit timeout: Timeout): Future[Option[IncomingPayment]] @@ -388,6 +396,24 @@ class EclairImpl(val appKit: Kit) extends Eclair with Logging with SpendFromChan } } + override def createOffer(description_opt: Option[String], amount_opt: Option[MilliSatoshi], expiry_opt: Option[TimestampSecond], issuer_opt: Option[String], firstNodeId_opt: Option[PublicKey])(implicit timeout: Timeout): Future[Offer] = { + val offerCreator = appKit.system.spawnAnonymous(OfferCreator(appKit.nodeParams, appKit.router, appKit.offerManager, appKit.defaultOfferHandler)) + offerCreator.ask[Either[String, Offer]](replyTo => OfferCreator.Create(replyTo, description_opt, amount_opt, expiry_opt, issuer_opt, firstNodeId_opt)) + .flatMap { + case Left(errorMessage) => Future.failed(new Exception(errorMessage)) + case Right(offer) => Future.successful(offer) + } + } + + override def disableOffer(offer: Offer)(implicit timeout: Timeout): Unit = { + appKit.offerManager ! OfferManager.DisableOffer(offer) + appKit.nodeParams.db.managedOffers.disableOffer(offer) + } + + override def listOffers(onlyActive: Boolean = true)(implicit timeout: Timeout): Future[Seq[Offer]] = Future { + appKit.nodeParams.db.managedOffers.listOffers(onlyActive).map(_.offer) + } + override def newAddress(): Future[String] = { appKit.wallet match { case w: BitcoinCoreClient => w.getReceiveAddress() diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/NodeParams.scala b/eclair-core/src/main/scala/fr/acinq/eclair/NodeParams.scala index 48b502c0b8..1d349054cf 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/NodeParams.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/NodeParams.scala @@ -30,6 +30,7 @@ import fr.acinq.eclair.db._ import fr.acinq.eclair.io.MessageRelay.{RelayAll, RelayChannelsOnly, RelayPolicy} import fr.acinq.eclair.io.{PeerConnection, PeerReadyNotifier} import fr.acinq.eclair.message.OnionMessages.OnionMessageConfig +import fr.acinq.eclair.payment.offer.OffersConfig import fr.acinq.eclair.payment.relay.OnTheFlyFunding import fr.acinq.eclair.payment.relay.Relayer.{AsyncPaymentsParams, RelayFees, RelayParams} import fr.acinq.eclair.router.Announcements.AddressException @@ -92,7 +93,8 @@ case class NodeParams(nodeKeyManager: NodeKeyManager, liquidityAdsConfig: LiquidityAds.Config, peerWakeUpConfig: PeerReadyNotifier.WakeUpConfig, onTheFlyFundingConfig: OnTheFlyFunding.Config, - peerStorageConfig: PeerStorageConfig) { + peerStorageConfig: PeerStorageConfig, + offersConfig: OffersConfig) { val privateKey: Crypto.PrivateKey = nodeKeyManager.nodeKey.privateKey val nodeId: PublicKey = nodeKeyManager.nodeId @@ -705,6 +707,12 @@ object NodeParams extends Logging { writeDelay = FiniteDuration(config.getDuration("peer-storage.write-delay").getSeconds, TimeUnit.SECONDS), removalDelay = FiniteDuration(config.getDuration("peer-storage.removal-delay").getSeconds, TimeUnit.SECONDS), cleanUpFrequency = FiniteDuration(config.getDuration("peer-storage.cleanup-frequency").getSeconds, TimeUnit.SECONDS), + ), + offersConfig = OffersConfig( + messagePathMinLength = config.getInt("managed-offers.message-path-min-length"), + paymentPathCount = config.getInt("managed-offers.payment-path-count"), + paymentPathLength = config.getInt("managed-offers.payment-path-length"), + paymentPathCltvExpiryDelta = CltvExpiryDelta(config.getInt("managed-offers.payment-path-expiry-delta")), ) ) } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/Setup.scala b/eclair-core/src/main/scala/fr/acinq/eclair/Setup.scala index d2affaea41..2c05b28e01 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/Setup.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/Setup.scala @@ -40,7 +40,7 @@ import fr.acinq.eclair.db.FileBackupHandler.FileBackupParams import fr.acinq.eclair.db.{Databases, DbEventHandler, FileBackupHandler, PeerStorageCleaner} import fr.acinq.eclair.io._ import fr.acinq.eclair.message.Postman -import fr.acinq.eclair.payment.offer.OfferManager +import fr.acinq.eclair.payment.offer.{DefaultHandler, OfferManager} import fr.acinq.eclair.payment.receive.PaymentHandler import fr.acinq.eclair.payment.relay.{AsyncPaymentTriggerer, PostRestartHtlcCleaner, Relayer} import fr.acinq.eclair.payment.send.{Autoprobe, PaymentInitiator} @@ -358,6 +358,8 @@ class Setup(val datadir: File, dbEventHandler = system.actorOf(SimpleSupervisor.props(DbEventHandler.props(nodeParams), "db-event-handler", SupervisorStrategy.Resume)) register = system.actorOf(SimpleSupervisor.props(Register.props(), "register", SupervisorStrategy.Resume)) offerManager = system.spawn(Behaviors.supervise(OfferManager(nodeParams, paymentTimeout = 1 minute)).onFailure(typed.SupervisorStrategy.resume), name = "offer-manager") + defaultOfferHandler = system.spawn(Behaviors.supervise(DefaultHandler(nodeParams, router)).onFailure(typed.SupervisorStrategy.resume), name = "default-offer-handler") + _ = for (offer <- nodeParams.db.managedOffers.listOffers(onlyActive = true)) offerManager ! OfferManager.RegisterOffer(offer.offer, None, offer.pathId_opt, defaultOfferHandler) paymentHandler = system.actorOf(SimpleSupervisor.props(PaymentHandler.props(nodeParams, register, offerManager), "payment-handler", SupervisorStrategy.Resume)) triggerer = system.spawn(Behaviors.supervise(AsyncPaymentTriggerer()).onFailure(typed.SupervisorStrategy.resume), name = "async-payment-triggerer") peerReadyManager = system.spawn(Behaviors.supervise(PeerReadyManager()).onFailure(typed.SupervisorStrategy.restart), name = "peer-ready-manager") @@ -399,6 +401,7 @@ class Setup(val datadir: File, balanceActor = balanceActor, postman = postman, offerManager = offerManager, + defaultOfferHandler = defaultOfferHandler, wallet = bitcoinClient) zmqBlockTimeout = after(5 seconds, using = system.scheduler)(Future.failed(BitcoinZMQConnectionTimeoutException)) @@ -468,6 +471,7 @@ case class Kit(nodeParams: NodeParams, balanceActor: typed.ActorRef[BalanceActor.Command], postman: typed.ActorRef[Postman.Command], offerManager: typed.ActorRef[OfferManager.Command], + defaultOfferHandler: typed.ActorRef[OfferManager.HandlerCommand], wallet: OnChainWallet with OnchainPubkeyCache) object Kit { diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/db/Databases.scala b/eclair-core/src/main/scala/fr/acinq/eclair/db/Databases.scala index bf0007a15f..4ef14b8dd8 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/db/Databases.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/db/Databases.scala @@ -43,6 +43,7 @@ trait Databases { def channels: ChannelsDb def peers: PeersDb def payments: PaymentsDb + def managedOffers: OffersDb def pendingCommands: PendingCommandsDb def liquidity: LiquidityDb //@formatter:on @@ -66,6 +67,7 @@ object Databases extends Logging { channels: SqliteChannelsDb, peers: SqlitePeersDb, payments: SqlitePaymentsDb, + managedOffers: SqliteOffersDb, pendingCommands: SqlitePendingCommandsDb, private val backupConnection: Connection) extends Databases with FileBackup { override def backup(backupFile: File): Unit = SqliteUtils.using(backupConnection.createStatement()) { @@ -85,6 +87,7 @@ object Databases extends Logging { channels = new SqliteChannelsDb(eclairJdbc), peers = new SqlitePeersDb(eclairJdbc), payments = new SqlitePaymentsDb(eclairJdbc), + managedOffers = new SqliteOffersDb(eclairJdbc), pendingCommands = new SqlitePendingCommandsDb(eclairJdbc), backupConnection = eclairJdbc ) @@ -97,6 +100,7 @@ object Databases extends Logging { channels: PgChannelsDb, peers: PgPeersDb, payments: PgPaymentsDb, + managedOffers: PgOffersDb, pendingCommands: PgPendingCommandsDb, dataSource: HikariDataSource, lock: PgLock) extends Databases with ExclusiveLock { @@ -157,6 +161,7 @@ object Databases extends Logging { channels = new PgChannelsDb, peers = new PgPeersDb, payments = new PgPaymentsDb, + managedOffers = new PgOffersDb, pendingCommands = new PgPendingCommandsDb, dataSource = ds, lock = lock) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/db/DualDatabases.scala b/eclair-core/src/main/scala/fr/acinq/eclair/db/DualDatabases.scala index c06c199906..c0a3a7eed4 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/db/DualDatabases.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/db/DualDatabases.scala @@ -36,6 +36,7 @@ case class DualDatabases(primary: Databases, secondary: Databases) extends Datab override val channels: ChannelsDb = DualChannelsDb(primary.channels, secondary.channels) override val peers: PeersDb = DualPeersDb(primary.peers, secondary.peers) override val payments: PaymentsDb = DualPaymentsDb(primary.payments, secondary.payments) + override val managedOffers: OffersDb = DualOffersDb(primary.managedOffers, secondary.managedOffers) override val pendingCommands: PendingCommandsDb = DualPendingCommandsDb(primary.pendingCommands, secondary.pendingCommands) override val liquidity: LiquidityDb = DualLiquidityDb(primary.liquidity, secondary.liquidity) @@ -405,6 +406,31 @@ case class DualPaymentsDb(primary: PaymentsDb, secondary: PaymentsDb) extends Pa } } +case class DualOffersDb(primary: OffersDb, secondary: OffersDb) extends OffersDb { + + private implicit val ec: ExecutionContext = ExecutionContext.fromExecutor(Executors.newSingleThreadExecutor(new ThreadFactoryBuilder().setNameFormat("db-offers").build())) + + override def addOffer(offer: OfferTypes.Offer, pathId_opt: Option[ByteVector32]): Unit = { + runAsync(secondary.addOffer(offer, pathId_opt)) + primary.addOffer(offer, pathId_opt) + } + + override def disableOffer(offer: OfferTypes.Offer): Unit = { + runAsync(secondary.disableOffer(offer)) + primary.disableOffer(offer) + } + + override def enableOffer(offer: OfferTypes.Offer): Unit = { + runAsync(secondary.enableOffer(offer)) + primary.enableOffer(offer) + } + + override def listOffers(onlyActive: Boolean): Seq[OfferData] = { + runAsync(secondary.listOffers(onlyActive)) + primary.listOffers(onlyActive) + } +} + case class DualPendingCommandsDb(primary: PendingCommandsDb, secondary: PendingCommandsDb) extends PendingCommandsDb { private implicit val ec: ExecutionContext = ExecutionContext.fromExecutor(Executors.newSingleThreadExecutor(new ThreadFactoryBuilder().setNameFormat("db-pending-commands").build())) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/db/OffersDb.scala b/eclair-core/src/main/scala/fr/acinq/eclair/db/OffersDb.scala new file mode 100644 index 0000000000..f7d1409f83 --- /dev/null +++ b/eclair-core/src/main/scala/fr/acinq/eclair/db/OffersDb.scala @@ -0,0 +1,51 @@ +/* + * Copyright 2025 ACINQ SAS + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package fr.acinq.eclair.db + +import fr.acinq.bitcoin.scalacompat.ByteVector32 +import fr.acinq.eclair.TimestampMilli +import fr.acinq.eclair.wire.protocol.OfferTypes.Offer + +/** + * Database for offers fully managed by eclair, as opposed to offers managed by a plugin. + */ +trait OffersDb { + /** + * Add an offer managed by eclair. + * @param pathId_opt If the offer uses a blinded path, this is the corresponding pathId. + */ + def addOffer(offer: Offer, pathId_opt: Option[ByteVector32]): Unit + + /** + * Disable an offer. The offer is still stored but new invoice requests and new payment attempts for already emitted + * invoices will be rejected. + */ + def disableOffer(offer: Offer): Unit + + /** + * Activate an offer that was previously disabled. + */ + def enableOffer(offer: Offer): Unit + + /** + * List offers managed by eclair. + * @param onlyActive Whether to return only active offers or also disabled ones. + */ + def listOffers(onlyActive: Boolean): Seq[OfferData] +} + +case class OfferData(offer: Offer, pathId_opt: Option[ByteVector32], createdAt: TimestampMilli, isActive: Boolean) \ No newline at end of file diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/db/pg/PgOffersDb.scala b/eclair-core/src/main/scala/fr/acinq/eclair/db/pg/PgOffersDb.scala new file mode 100644 index 0000000000..43cc5ff31a --- /dev/null +++ b/eclair-core/src/main/scala/fr/acinq/eclair/db/pg/PgOffersDb.scala @@ -0,0 +1,113 @@ +/* + * Copyright 2025 ACINQ SAS + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package fr.acinq.eclair.db.pg + +import fr.acinq.bitcoin.scalacompat.ByteVector32 +import fr.acinq.eclair.TimestampMilli +import fr.acinq.eclair.db.Monitoring.Metrics.withMetrics +import fr.acinq.eclair.db.Monitoring.Tags.DbBackends +import fr.acinq.eclair.db.{OfferData, OffersDb} +import fr.acinq.eclair.db.pg.PgUtils.PgLock +import fr.acinq.eclair.wire.protocol.OfferTypes +import fr.acinq.eclair.wire.protocol.OfferTypes.Offer +import grizzled.slf4j.Logging + +import java.sql.ResultSet +import javax.sql.DataSource + +object PgOffersDb { + val DB_NAME = "offers" + val CURRENT_VERSION = 1 +} + +class PgOffersDb(implicit ds: DataSource, lock: PgLock) extends OffersDb with Logging { + + import PgOffersDb._ + import PgUtils.ExtendedResultSet._ + import PgUtils._ + import lock._ + + inTransaction { pg => + using(pg.createStatement()) { statement => + getVersion(statement, DB_NAME) match { + case None => + statement.executeUpdate("CREATE SCHEMA offers") + statement.executeUpdate("CREATE TABLE offers.managed (offer_id TEXT NOT NULL PRIMARY KEY, offer TEXT NOT NULL, path_id TEXT, created_at TIMESTAMP WITH TIME ZONE NOT NULL, is_active BOOLEAN NOT NULL)") + statement.executeUpdate("CREATE INDEX offer_is_active_idx ON offers.managed(is_active)") + case Some(CURRENT_VERSION) => () // table is up-to-date, nothing to do + case Some(unknownVersion) => throw new RuntimeException(s"Unknown version of DB $DB_NAME found, version=$unknownVersion") + } + setVersion(statement, DB_NAME, CURRENT_VERSION) + } + } + + override def addOffer(offer: OfferTypes.Offer, pathId_opt: Option[ByteVector32]): Unit = withMetrics("offers/add", DbBackends.Postgres){ + withLock { pg => + using(pg.prepareStatement("INSERT INTO offers.managed (offer_id, offer, path_id, created_at, is_active) VALUES (?, ?, ?, ?, TRUE)")) { statement => + statement.setString(1, offer.offerId.toHex) + statement.setString(2, offer.toString) + pathId_opt match { + case Some(pathId) => statement.setString(3, pathId.toHex) + case None => statement.setNull(3, java.sql.Types.VARCHAR) + } + statement.setTimestamp(4, TimestampMilli.now().toSqlTimestamp) + statement.executeUpdate() + } + } + } + + override def disableOffer(offer: OfferTypes.Offer): Unit = withMetrics("offers/disable", DbBackends.Postgres){ + withLock { pg => + using(pg.prepareStatement("UPDATE offers.managed SET is_active = FALSE WHERE offer_id = ?")) { statement => + statement.setString(1, offer.offerId.toHex) + statement.executeUpdate() + } + } + } + + override def enableOffer(offer: OfferTypes.Offer): Unit = withMetrics("offers/enable", DbBackends.Postgres){ + withLock { pg => + using(pg.prepareStatement("UPDATE offers.managed SET is_active = TRUE WHERE offer_id = ?")) { statement => + statement.setString(1, offer.offerId.toHex) + statement.executeUpdate() + } + } + } + + private def parseOfferData(rs: ResultSet): OfferData = { + OfferData( + Offer.decode(rs.getString("offer")).get, + rs.getStringNullable("path_id").map(ByteVector32.fromValidHex), + TimestampMilli.fromSqlTimestamp(rs.getTimestamp("created_at")), + rs.getBoolean("is_active") + ) + } + + override def listOffers(onlyActive: Boolean): Seq[OfferData] = withMetrics("offers/list", DbBackends.Postgres){ + withLock { pg => + if (onlyActive) { + using(pg.prepareStatement("SELECT * FROM offers.managed WHERE is_active = TRUE")) { statement => + statement.executeQuery().map(parseOfferData).toSeq + } + } else { + using(pg.prepareStatement("SELECT * FROM offers.managed")) { statement => + statement.executeQuery().map(parseOfferData).toSeq + } + } + } + } +} diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteOffersDb.scala b/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteOffersDb.scala new file mode 100644 index 0000000000..e252881d60 --- /dev/null +++ b/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteOffersDb.scala @@ -0,0 +1,99 @@ +/* + * Copyright 2025 ACINQ SAS + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package fr.acinq.eclair.db.sqlite + +import fr.acinq.bitcoin.scalacompat.ByteVector32 +import fr.acinq.eclair.TimestampMilli +import fr.acinq.eclair.db.Monitoring.Metrics.withMetrics +import fr.acinq.eclair.db.Monitoring.Tags.DbBackends +import fr.acinq.eclair.db.sqlite.SqliteUtils.{getVersion, setVersion, using} +import fr.acinq.eclair.db.{OfferData, OffersDb} +import fr.acinq.eclair.wire.protocol.OfferTypes +import fr.acinq.eclair.wire.protocol.OfferTypes.Offer +import grizzled.slf4j.Logging + +import java.sql.{Connection, ResultSet} + +object SqliteOffersDb { + val DB_NAME = "offers" + val CURRENT_VERSION = 1 +} + +class SqliteOffersDb(val sqlite: Connection) extends OffersDb with Logging { + import SqliteOffersDb._ + import SqliteUtils.ExtendedResultSet._ + + using(sqlite.createStatement(), inTransaction = true) { statement => + getVersion(statement, DB_NAME) match { + case None => + statement.executeUpdate("CREATE TABLE managed_offers (offer_id BLOB NOT NULL PRIMARY KEY, offer TEXT NOT NULL, path_id BLOB, created_at INTEGER NOT NULL, is_active INTEGER NOT NULL)") + statement.executeUpdate("CREATE INDEX offer_is_active_idx ON managed_offers(is_active)") + case Some(CURRENT_VERSION) => () // table is up-to-date, nothing to do + case Some(unknownVersion) => throw new RuntimeException(s"Unknown version of DB $DB_NAME found, version=$unknownVersion") + } + setVersion(statement, DB_NAME, CURRENT_VERSION) + + } + + override def addOffer(offer: OfferTypes.Offer, pathId_opt: Option[ByteVector32]): Unit = withMetrics("offers/add", DbBackends.Sqlite) { + using(sqlite.prepareStatement("INSERT INTO managed_offers (offer_id, offer, path_id, created_at, is_active) VALUES (?, ?, ?, ?, TRUE)")) { statement => + statement.setBytes(1, offer.offerId.toArray) + statement.setString(2, offer.toString) + pathId_opt match { + case Some(pathId) => statement.setBytes(3, pathId.toArray) + case None => statement.setNull(3, java.sql.Types.VARBINARY) + } + statement.setLong(4, TimestampMilli.now().toLong) + statement.executeUpdate() + } + } + + override def disableOffer(offer: OfferTypes.Offer): Unit = withMetrics("offers/disable", DbBackends.Sqlite) { + using(sqlite.prepareStatement("UPDATE managed_offers SET is_active = FALSE WHERE offer_id = ?")) { statement => + statement.setBytes(1, offer.offerId.toArray) + statement.executeUpdate() + } + } + + override def enableOffer(offer: OfferTypes.Offer): Unit = withMetrics("offers/enable", DbBackends.Sqlite) { + using(sqlite.prepareStatement("UPDATE managed_offers SET is_active = TRUE WHERE offer_id = ?")) { statement => + statement.setBytes(1, offer.offerId.toArray) + statement.executeUpdate() + } + } + + private def parseOfferData(rs: ResultSet): OfferData = { + OfferData( + Offer.decode(rs.getString("offer")).get, + rs.getByteVector32Nullable("path_id"), + TimestampMilli(rs.getLong("created_at")), + rs.getBoolean("is_active") + ) + } + + override def listOffers(onlyActive: Boolean): Seq[OfferData] = withMetrics("offers/list", DbBackends.Sqlite) { + if (onlyActive) { + using(sqlite.prepareStatement("SELECT * FROM managed_offers WHERE is_active = TRUE")) { statement => + statement.executeQuery().map(parseOfferData).toSeq + } + } else { + using(sqlite.prepareStatement("SELECT * FROM managed_offers")) { statement => + statement.executeQuery().map(parseOfferData).toSeq + } + } + } +} diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/DefaultHandler.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/DefaultHandler.scala new file mode 100644 index 0000000000..d4bf390016 --- /dev/null +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/DefaultHandler.scala @@ -0,0 +1,79 @@ +/* + * Copyright 2025 ACINQ SAS + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package fr.acinq.eclair.payment.offer + +import akka.actor.typed.Behavior +import akka.actor.typed.scaladsl.Behaviors +import akka.actor.{ActorRef, typed} +import fr.acinq.eclair.crypto.Sphinx.RouteBlinding.BlindedRoute +import fr.acinq.eclair.payment.offer.OfferManager.InvoiceRequestActor +import fr.acinq.eclair.payment.relay.Relayer.RelayFees +import fr.acinq.eclair.router.Router +import fr.acinq.eclair.router.Router.{BlindedRouteRequest, ChannelHop} +import fr.acinq.eclair.wire.protocol.OfferTypes +import fr.acinq.eclair.{CltvExpiryDelta, EncodedNodeId, MilliSatoshi, MilliSatoshiLong, NodeParams} + +object DefaultHandler { + def apply(nodeParams: NodeParams, router: ActorRef): Behavior[OfferManager.HandlerCommand] = { + Behaviors.setup(context => + Behaviors.receiveMessage { + case OfferManager.HandleInvoiceRequest(replyTo, invoiceRequest) => + val amount = invoiceRequest.amount.getOrElse(10_000_000 msat) + invoiceRequest.offer.contactInfos.head match { + case OfferTypes.RecipientNodeId(_) => + val route = InvoiceRequestActor.Route(Nil, nodeParams.channelConf.maxExpiryDelta) + replyTo ! InvoiceRequestActor.ApproveRequest(amount, Seq(route)) + case OfferTypes.BlindedPath(BlindedRoute(firstNodeId: EncodedNodeId.WithPublicKey, _, _)) if firstNodeId.publicKey == nodeParams.nodeId => + replyTo ! InvoiceRequestActor.ApproveRequest(amount, makeRoutes(nodeParams, Seq(Nil))) + case OfferTypes.BlindedPath(BlindedRoute(firstNodeId: EncodedNodeId.WithPublicKey, _, _)) => + val baseParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams + val routeParams = baseParams.copy(boundaries = baseParams.boundaries.copy(maxRouteLength = nodeParams.offersConfig.paymentPathLength, maxCltv = nodeParams.offersConfig.paymentPathCltvExpiryDelta)) + router ! BlindedRouteRequest(context.spawnAnonymous(waitForRoute(nodeParams, replyTo, invoiceRequest.offer, amount)), firstNodeId.publicKey, nodeParams.nodeId, amount, routeParams, pathsToFind = 2) + case OfferTypes.BlindedPath(BlindedRoute(_: EncodedNodeId.ShortChannelIdDir, _, _)) => + context.log.error("unexpected managed offer with compact first node id") + replyTo ! InvoiceRequestActor.RejectRequest("internal error") + } + Behaviors.same + case OfferManager.HandlePayment(replyTo, _, _) => + replyTo ! OfferManager.PaymentActor.AcceptPayment() + Behaviors.same + } + ) + } + + def waitForRoute(nodeParams: NodeParams, replyTo: typed.ActorRef[InvoiceRequestActor.Command], offer: OfferTypes.Offer, amount: MilliSatoshi): Behavior[Router.PaymentRouteResponse] = { + Behaviors.receive { + case (_, Router.RouteResponse(routes)) => + replyTo ! InvoiceRequestActor.ApproveRequest(amount, makeRoutes(nodeParams, routes.map(_.hops))) + Behaviors.stopped + case (context, Router.PaymentRouteNotFound(error)) => + context.log.error("Couldn't find blinded route for creating invoice offer={} amount={} : {}", offer, amount, error.getMessage) + replyTo ! InvoiceRequestActor.RejectRequest("internal error") + Behaviors.stopped + } + } + + def makeRoutes(nodeParams: NodeParams, routes: Seq[Seq[Router.ChannelHop]]): Seq[InvoiceRequestActor.Route] = { + (0 until nodeParams.offersConfig.paymentPathCount).map(i => { + val hops = routes(i % routes.length) + val dummyHops = Seq.fill(nodeParams.offersConfig.paymentPathLength - hops.length)(ChannelHop.dummy(nodeParams.nodeId, 0 msat, 0, CltvExpiryDelta(0))) + InvoiceRequestActor.Route(hops ++ dummyHops, nodeParams.channelConf.maxExpiryDelta, feeOverride = Some(RelayFees.zero), cltvOverride = Some(nodeParams.offersConfig.paymentPathCltvExpiryDelta)) + }) + } +} + +case class OffersConfig(messagePathMinLength: Int, paymentPathCount: Int, paymentPathLength: Int, paymentPathCltvExpiryDelta: CltvExpiryDelta) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/OfferCreator.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/OfferCreator.scala new file mode 100644 index 0000000000..cd98b31545 --- /dev/null +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/OfferCreator.scala @@ -0,0 +1,105 @@ +/* + * Copyright 2025 ACINQ SAS + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package fr.acinq.eclair.payment.offer + +import akka.actor.{ActorRef, typed} +import akka.actor.typed.Behavior +import akka.actor.typed.scaladsl.{ActorContext, Behaviors} +import fr.acinq.bitcoin.scalacompat.Crypto.{PrivateKey, PublicKey} +import fr.acinq.bitcoin.scalacompat.{Block, ByteVector32} +import fr.acinq.eclair.message.OnionMessages +import fr.acinq.eclair.message.OnionMessages.{IntermediateNode, Recipient} +import fr.acinq.eclair.router.Router +import fr.acinq.eclair.wire.protocol.OfferTypes._ +import fr.acinq.eclair.wire.protocol.TlvStream +import fr.acinq.eclair.{MilliSatoshi, NodeParams, TimestampSecond, randomBytes32, randomKey} + +object OfferCreator { + sealed trait Command + + case class Create(replyTo: typed.ActorRef[Either[String, Offer]], + description_opt: Option[String], + amount_opt: Option[MilliSatoshi], + expiry_opt: Option[TimestampSecond], + issuer_opt: Option[String], + firstNodeId_opt: Option[PublicKey]) extends Command + + case class RouteResponseWrapper(response: Router.MessageRouteResponse) extends Command + + def apply(nodeParams: NodeParams, router: ActorRef, offerManager: typed.ActorRef[OfferManager.Command], defaultOfferHandler: typed.ActorRef[OfferManager.HandlerCommand]): Behavior[Command] = + Behaviors.receivePartial { + case (context, Create(replyTo, description_opt, amount_opt, expiry_opt, issuer_opt, firstNodeId_opt)) => + new OfferCreator(context, replyTo, nodeParams, router, offerManager, defaultOfferHandler).init(description_opt, amount_opt, expiry_opt, issuer_opt, firstNodeId_opt) + } +} + +private class OfferCreator(context: ActorContext[OfferCreator.Command], + replyTo: typed.ActorRef[Either[String, Offer]], + nodeParams: NodeParams, router: ActorRef, + offerManager: typed.ActorRef[OfferManager.Command], + defaultOfferHandler: typed.ActorRef[OfferManager.HandlerCommand]) { + + import OfferCreator._ + + private def init(description_opt: Option[String], + amount_opt: Option[MilliSatoshi], + expiry_opt: Option[TimestampSecond], + issuer_opt: Option[String], + firstNodeId_opt: Option[PublicKey]): Behavior[Command] = { + if (amount_opt.nonEmpty && description_opt.isEmpty) { + replyTo ! Left("Description is mandatory for offers with set amount.") + Behaviors.stopped + } else { + val tlvs: Set[OfferTlv] = Set( + if (nodeParams.chainHash != Block.LivenetGenesisBlock.hash) Some(OfferChains(Seq(nodeParams.chainHash))) else None, + amount_opt.map(OfferAmount), + description_opt.map(OfferDescription), + expiry_opt.map(OfferAbsoluteExpiry), + issuer_opt.map(OfferIssuer), + ).flatten + firstNodeId_opt match { + case Some(firstNodeId) => + router ! Router.MessageRouteRequest(context.messageAdapter(RouteResponseWrapper(_)), firstNodeId, nodeParams.nodeId, Set.empty) + waitForRoute(firstNodeId, tlvs) + case None => + val offer = Offer(TlvStream(tlvs + OfferNodeId(nodeParams.nodeId))) + registerOffer(offer, Some(nodeParams.privateKey), None) + } + } + } + + private def waitForRoute(firstNode: PublicKey, tlvs: Set[OfferTlv]): Behavior[Command] = { + Behaviors.receiveMessagePartial { + case RouteResponseWrapper(Router.MessageRoute(intermediateNodes, _)) => + val pathId = randomBytes32() + val nodes = firstNode +: (intermediateNodes ++ Seq.fill(nodeParams.offersConfig.messagePathMinLength - intermediateNodes.length - 1)(nodeParams.nodeId)) + val paths = Seq(OnionMessages.buildRoute(randomKey(), nodes.map(IntermediateNode(_)), Recipient(nodeParams.nodeId, Some(pathId))).route) + val offer = Offer(TlvStream(tlvs + OfferPaths(paths))) + registerOffer(offer, None, Some(pathId)) + case RouteResponseWrapper(Router.MessageRouteNotFound(_)) => + replyTo ! Left("No route found") + Behaviors.stopped + } + } + + private def registerOffer(offer: Offer, nodeKey: Option[PrivateKey], pathId_opt: Option[ByteVector32]): Behavior[Command] = { + nodeParams.db.managedOffers.addOffer(offer, pathId_opt) + offerManager ! OfferManager.RegisterOffer(offer, nodeKey, pathId_opt, defaultOfferHandler) + replyTo ! Right(offer) + Behaviors.stopped + } +} diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/OfferManager.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/OfferManager.scala index f15c1aa035..314b225469 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/OfferManager.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/OfferManager.scala @@ -84,12 +84,11 @@ object OfferManager { * When a payment is received for an offer invoice, a `HandlePayment` is sent to the handler registered for this offer. * The handler may receive several `HandlePayment` for the same payment, usually because of multi-part payments. * - * @param replyTo The handler must reply with either `PaymentActor.ApprovePayment` or `PaymentActor.RejectPayment`. - * @param offerId The id of the offer in case a single handler handles multiple offers. - * @param pluginData_opt If the plugin handler needs to associate data with a payment, it shouldn't store it to avoid - * DoS and should instead use that field to include that data in the blinded path. + * @param replyTo The handler must reply with either `PaymentActor.ApprovePayment` or `PaymentActor.RejectPayment`. + * @param offer The offer in case a single handler handles multiple offers. + * @param invoiceData Data from the invoice this payment is for (quantity, amount, creation time, etc.). */ - case class HandlePayment(replyTo: ActorRef[PaymentActor.Command], offerId: ByteVector32, pluginData_opt: Option[ByteVector] = None) extends HandlerCommand + case class HandlePayment(replyTo: ActorRef[PaymentActor.Command], offer: Offer, invoiceData: MinimalInvoiceData) extends HandlerCommand private case class RegisteredOffer(offer: Offer, nodeKey: Option[PrivateKey], pathId_opt: Option[ByteVector32], handler: ActorRef[HandlerCommand]) @@ -125,7 +124,7 @@ object OfferManager { MinimalInvoiceData.verify(nodeParams.nodeId, signed) match { case Some(metadata) if Crypto.sha256(metadata.preimage) == paymentHash => val child = context.spawnAnonymous(PaymentActor(nodeParams, replyTo, offer, metadata, amountReceived, paymentTimeout)) - handler ! HandlePayment(child, signed.offerId, metadata.pluginData_opt) + handler ! HandlePayment(child, offer, metadata) case Some(_) => replyTo ! MultiPartHandler.GetIncomingPaymentActor.RejectPayment(s"preimage does not match payment hash for offer ${signed.offerId.toHex}") case None => replyTo ! MultiPartHandler.GetIncomingPaymentActor.RejectPayment(s"invalid signature for metadata for offer ${signed.offerId.toHex}") } @@ -162,19 +161,14 @@ object OfferManager { /** * @param recipientPaysFees If true, fees for the blinded route will be hidden to the payer and paid by the recipient. */ - case class Route(hops: Seq[Router.ChannelHop], recipientPaysFees: Boolean, maxFinalExpiryDelta: CltvExpiryDelta, shortChannelIdDir_opt: Option[ShortChannelIdDir] = None) { + case class Route(hops: Seq[Router.ChannelHop], maxFinalExpiryDelta: CltvExpiryDelta, feeOverride: Option[RelayFees] = None, cltvOverride: Option[CltvExpiryDelta] = None, shortChannelIdDir_opt: Option[ShortChannelIdDir] = None) { def finalize(nodePriv: PrivateKey, preimage: ByteVector32, amount: MilliSatoshi, invoiceRequest: InvoiceRequest, minFinalExpiryDelta: CltvExpiryDelta, pluginData_opt: Option[ByteVector]): ReceivingRoute = { - val (paymentInfo, metadata) = if (recipientPaysFees) { - val realPaymentInfo = aggregatePaymentInfo(amount, hops, minFinalExpiryDelta) - val recipientFees = RelayFees(realPaymentInfo.feeBase, realPaymentInfo.feeProportionalMillionths) - val metadata = MinimalInvoiceData(preimage, invoiceRequest.payerId, TimestampSecond.now(), invoiceRequest.quantity, amount, recipientFees, pluginData_opt) - val paymentInfo = realPaymentInfo.copy(feeBase = 0 msat, feeProportionalMillionths = 0) - (paymentInfo, metadata) - } else { - val paymentInfo = aggregatePaymentInfo(amount, hops, minFinalExpiryDelta) - val metadata = MinimalInvoiceData(preimage, invoiceRequest.payerId, TimestampSecond.now(), invoiceRequest.quantity, amount, RelayFees.zero, pluginData_opt) - (paymentInfo, metadata) - } + val aggregatedPaymentInfo = aggregatePaymentInfo(amount, hops, minFinalExpiryDelta) + val fees = feeOverride.getOrElse(RelayFees(aggregatedPaymentInfo.feeBase, aggregatedPaymentInfo.feeProportionalMillionths)) + val cltvExpiryDelta = cltvOverride.getOrElse(aggregatedPaymentInfo.cltvExpiryDelta) + val paymentInfo = aggregatedPaymentInfo.copy(feeBase = fees.feeBase, feeProportionalMillionths = fees.feeProportionalMillionths, cltvExpiryDelta = cltvExpiryDelta) + val recipientFees = RelayFees(aggregatedPaymentInfo.feeBase - paymentInfo.feeBase, aggregatedPaymentInfo.feeProportionalMillionths - paymentInfo.feeProportionalMillionths) + val metadata = MinimalInvoiceData(preimage, invoiceRequest.payerId, TimestampSecond.now(), invoiceRequest.quantity, amount, recipientFees, pluginData_opt) val pathId = MinimalInvoiceData.encode(nodePriv, invoiceRequest.offer.offerId, metadata) ReceivingRoute(hops, pathId, maxFinalExpiryDelta, paymentInfo, shortChannelIdDir_opt) } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/Relayer.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/Relayer.scala index 0d69468b06..dd706ecf6e 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/Relayer.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/Relayer.scala @@ -129,10 +129,7 @@ object Relayer extends Logging { Props(new Relayer(nodeParams, router, register, paymentHandler, initialized)) // @formatter:off - case class RelayFees(feeBase: MilliSatoshi, feeProportionalMillionths: Long) { - require(feeBase.toLong >= 0.0, "feeBase must be nonnegative") - require(feeProportionalMillionths >= 0.0, "feeProportionalMillionths must be nonnegative") - } + case class RelayFees(feeBase: MilliSatoshi, feeProportionalMillionths: Long) object RelayFees { val zero: RelayFees = RelayFees(MilliSatoshi(0), 0) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/BlindedPathsResolver.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/BlindedPathsResolver.scala index 663c92e543..e5e2d47fd6 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/BlindedPathsResolver.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/BlindedPathsResolver.scala @@ -164,10 +164,7 @@ private class BlindedPathsResolver(nodeParams: NodeParams, val relayFees = getRelayFees(nodeParams, nextNodeId.publicKey, announceChannel = false) val shouldRelay = paymentRelayData.paymentRelay.feeBase >= relayFees.feeBase && paymentRelayData.paymentRelay.feeProportionalMillionths >= relayFees.feeProportionalMillionths && - paymentRelayData.paymentRelay.cltvExpiryDelta >= nodeParams.channelConf.expiryDelta && - nextPaymentInfo.feeBase >= 0.msat && - nextPaymentInfo.feeProportionalMillionths >= 0 && - nextPaymentInfo.cltvExpiryDelta.toInt >= 0 + paymentRelayData.paymentRelay.cltvExpiryDelta >= nodeParams.channelConf.expiryDelta if (shouldRelay) { context.log.debug("unwrapped blinded path starting at our node: next_node={}", nextNodeId.publicKey) val path = ResolvedPath(PartialBlindedRoute(nextNodeId, nextPathKey, nextBlindedNodes), nextPaymentInfo) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/EclairImplSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/EclairImplSpec.scala index 4bf5a594ed..b449d431e3 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/EclairImplSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/EclairImplSpec.scala @@ -73,6 +73,7 @@ class EclairImplSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with I val balanceActor = TestProbe() val postman = TestProbe() val offerManager = TestProbe() + val defaultOfferHandler = TestProbe() val kit = Kit( TestConstants.Alice.nodeParams, system, @@ -88,6 +89,7 @@ class EclairImplSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with I balanceActor.ref.toTyped, postman.ref.toTyped, offerManager.ref.toTyped, + defaultOfferHandler.ref.toTyped, new DummyOnChainWallet() ) withFixture(test.toNoArgTest(FixtureParam(register, relayer, router, paymentInitiator, switchboard, paymentHandler, TestProbe(), kit))) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/TestConstants.scala b/eclair-core/src/test/scala/fr/acinq/eclair/TestConstants.scala index 9084e843dc..4906ab3e06 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/TestConstants.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/TestConstants.scala @@ -26,6 +26,7 @@ import fr.acinq.eclair.db.RevokedHtlcInfoCleaner import fr.acinq.eclair.io.MessageRelay.RelayAll import fr.acinq.eclair.io.{OpenChannelInterceptor, PeerConnection, PeerReadyNotifier} import fr.acinq.eclair.message.OnionMessages.OnionMessageConfig +import fr.acinq.eclair.payment.offer.OffersConfig import fr.acinq.eclair.payment.relay.OnTheFlyFunding import fr.acinq.eclair.payment.relay.Relayer.{AsyncPaymentsParams, RelayFees, RelayParams} import fr.acinq.eclair.router.Graph.{MessageWeightRatios, PaymentWeightRatios} @@ -246,7 +247,8 @@ object TestConstants { liquidityAdsConfig = LiquidityAds.Config(Some(defaultLiquidityRates), lockUtxos = true), peerWakeUpConfig = PeerReadyNotifier.WakeUpConfig(enabled = false, timeout = 30 seconds), onTheFlyFundingConfig = OnTheFlyFunding.Config(proposalTimeout = 90 seconds), - peerStorageConfig = PeerStorageConfig(writeDelay = 5 seconds, removalDelay = 10 seconds, cleanUpFrequency = 1 hour) + peerStorageConfig = PeerStorageConfig(writeDelay = 5 seconds, removalDelay = 10 seconds, cleanUpFrequency = 1 hour), + offersConfig = OffersConfig(messagePathMinLength = 2, paymentPathCount = 2, paymentPathLength = 4, paymentPathCltvExpiryDelta = CltvExpiryDelta(500)), ) def channelParams: LocalParams = OpenChannelInterceptor.makeChannelParams( @@ -428,7 +430,8 @@ object TestConstants { liquidityAdsConfig = LiquidityAds.Config(Some(defaultLiquidityRates), lockUtxos = true), peerWakeUpConfig = PeerReadyNotifier.WakeUpConfig(enabled = false, timeout = 30 seconds), onTheFlyFundingConfig = OnTheFlyFunding.Config(proposalTimeout = 90 seconds), - peerStorageConfig = PeerStorageConfig(writeDelay = 5 seconds, removalDelay = 10 seconds, cleanUpFrequency = 1 hour) + peerStorageConfig = PeerStorageConfig(writeDelay = 5 seconds, removalDelay = 10 seconds, cleanUpFrequency = 1 hour), + offersConfig = OffersConfig(messagePathMinLength = 2, paymentPathCount = 2, paymentPathLength = 4, paymentPathCltvExpiryDelta = CltvExpiryDelta(500)), ) def channelParams: LocalParams = OpenChannelInterceptor.makeChannelParams( diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/TestDatabases.scala b/eclair-core/src/test/scala/fr/acinq/eclair/TestDatabases.scala index 1f76345423..a1a111d00f 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/TestDatabases.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/TestDatabases.scala @@ -33,6 +33,7 @@ sealed trait TestDatabases extends Databases { override def channels: ChannelsDb = db.channels override def peers: PeersDb = db.peers override def payments: PaymentsDb = db.payments + override def managedOffers: OffersDb = db.managedOffers override def pendingCommands: PendingCommandsDb = db.pendingCommands override def liquidity: LiquidityDb = db.liquidity def close(): Unit diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/db/OffersDbSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/db/OffersDbSpec.scala new file mode 100644 index 0000000000..ca14979871 --- /dev/null +++ b/eclair-core/src/test/scala/fr/acinq/eclair/db/OffersDbSpec.scala @@ -0,0 +1,70 @@ +/* + * Copyright 2025 ACINQ SAS + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package fr.acinq.eclair.db + +import fr.acinq.bitcoin.scalacompat.Block +import fr.acinq.eclair.TestDatabases.{TestPgDatabases, TestSqliteDatabases} +import fr.acinq.eclair._ +import fr.acinq.eclair.db.pg.PgOffersDb +import fr.acinq.eclair.db.sqlite.SqliteOffersDb +import fr.acinq.eclair.wire.protocol.OfferTypes.Offer +import org.scalatest.funsuite.AnyFunSuite + +class OffersDbSpec extends AnyFunSuite { + + import fr.acinq.eclair.TestDatabases.forAllDbs + + test("init database two times in a row") { + forAllDbs { + case sqlite: TestSqliteDatabases => + new SqliteOffersDb(sqlite.connection) + new SqliteOffersDb(sqlite.connection) + case pg: TestPgDatabases => + new PgOffersDb()(pg.datasource, pg.lock) + new PgOffersDb()(pg.datasource, pg.lock) + } + } + + test("add/disable/enable/list offers") { + forAllDbs { dbs => + val db = dbs.managedOffers + + assert(db.listOffers(onlyActive = false).isEmpty) + val offer1 = Offer(None, Some("test 1"), randomKey().publicKey, Features(), Block.LivenetGenesisBlock.hash) + db.addOffer(offer1, None) + val listed1 = db.listOffers(onlyActive = true) + assert(listed1.length == 1) + assert(listed1.head.offer == offer1) + assert(listed1.head.pathId_opt == None) + assert(listed1.head.isActive) + val offer2 = Offer(None, Some("test 2"), randomKey().publicKey, Features(), Block.LivenetGenesisBlock.hash) + val pathId = randomBytes32() + db.addOffer(offer2, Some(pathId)) + assert(db.listOffers(onlyActive = true).length == 2) + db.disableOffer(offer1) + assert(db.listOffers(onlyActive = false).length == 2) + val listed2 = db.listOffers(onlyActive = true) + assert(listed2.length == 1) + assert(listed2.head.offer == offer2) + assert(listed2.head.pathId_opt == Some(pathId)) + assert(listed2.head.isActive) + db.disableOffer(offer2) + db.enableOffer(offer1) + assert(db.listOffers(onlyActive = true) == listed1) + } + } +} diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala index 436a2d5e75..5493307146 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala @@ -634,15 +634,15 @@ class PaymentIntegrationSpec extends IntegrationSpec { val handleInvoiceRequest = offerHandler.expectMessageType[HandleInvoiceRequest] val receivingRoutes = Seq( - OfferManager.InvoiceRequestActor.Route(route1.hops, recipientPaysFees = false, CltvExpiryDelta(1000)), - OfferManager.InvoiceRequestActor.Route(route2.hops, recipientPaysFees = false, CltvExpiryDelta(1000)), - OfferManager.InvoiceRequestActor.Route(route3.hops, recipientPaysFees = false, CltvExpiryDelta(1000)), + OfferManager.InvoiceRequestActor.Route(route1.hops, CltvExpiryDelta(1000)), + OfferManager.InvoiceRequestActor.Route(route2.hops, CltvExpiryDelta(1000)), + OfferManager.InvoiceRequestActor.Route(route3.hops, CltvExpiryDelta(1000)), ) handleInvoiceRequest.replyTo ! InvoiceRequestActor.ApproveRequest(amount, receivingRoutes, pluginData_opt = Some(hex"abcd")) val handlePayment = offerHandler.expectMessageType[HandlePayment] - assert(handlePayment.offerId == offer.offerId) - assert(handlePayment.pluginData_opt.contains(hex"abcd")) + assert(handlePayment.offer == offer) + assert(handlePayment.invoiceData.pluginData_opt.contains(hex"abcd")) handlePayment.replyTo ! PaymentActor.AcceptPayment() val paymentSent = sender.expectMsgType[PaymentSent] @@ -668,14 +668,14 @@ class PaymentIntegrationSpec extends IntegrationSpec { val handleInvoiceRequest = offerHandler.expectMessageType[HandleInvoiceRequest] // C uses a 0-hop blinded route and signs the invoice with its public nodeId. val receivingRoutes = Seq( - OfferManager.InvoiceRequestActor.Route(Nil, recipientPaysFees = false, CltvExpiryDelta(1000)), - OfferManager.InvoiceRequestActor.Route(Nil, recipientPaysFees = false, CltvExpiryDelta(1000)), + OfferManager.InvoiceRequestActor.Route(Nil, CltvExpiryDelta(1000)), + OfferManager.InvoiceRequestActor.Route(Nil, CltvExpiryDelta(1000)), ) handleInvoiceRequest.replyTo ! InvoiceRequestActor.ApproveRequest(amount, receivingRoutes, pluginData_opt = Some(hex"0123")) val handlePayment = offerHandler.expectMessageType[HandlePayment] - assert(handlePayment.offerId == offer.offerId) - assert(handlePayment.pluginData_opt.contains(hex"0123")) + assert(handlePayment.offer == offer) + assert(handlePayment.invoiceData.pluginData_opt.contains(hex"0123")) handlePayment.replyTo ! PaymentActor.AcceptPayment() val paymentSent = sender.expectMsgType[PaymentSent] @@ -703,13 +703,13 @@ class PaymentIntegrationSpec extends IntegrationSpec { val handleInvoiceRequest = offerHandler.expectMessageType[HandleInvoiceRequest] val receivingRoutes = Seq( - OfferManager.InvoiceRequestActor.Route(Seq(ChannelHop.dummy(nodes("A").nodeParams.nodeId, 100 msat, 100, CltvExpiryDelta(48)), ChannelHop.dummy(nodes("A").nodeParams.nodeId, 150 msat, 50, CltvExpiryDelta(36))), recipientPaysFees = false, CltvExpiryDelta(1000)) + OfferManager.InvoiceRequestActor.Route(Seq(ChannelHop.dummy(nodes("A").nodeParams.nodeId, 100 msat, 100, CltvExpiryDelta(48)), ChannelHop.dummy(nodes("A").nodeParams.nodeId, 150 msat, 50, CltvExpiryDelta(36))), CltvExpiryDelta(1000)) ) handleInvoiceRequest.replyTo ! InvoiceRequestActor.ApproveRequest(amount, receivingRoutes) val handlePayment = offerHandler.expectMessageType[HandlePayment] - assert(handlePayment.offerId == offer.offerId) - assert(handlePayment.pluginData_opt.isEmpty) + assert(handlePayment.offer == offer) + assert(handlePayment.invoiceData.pluginData_opt.isEmpty) handlePayment.replyTo ! PaymentActor.AcceptPayment() val paymentSent = sender.expectMsgType[PaymentSent] @@ -740,13 +740,13 @@ class PaymentIntegrationSpec extends IntegrationSpec { val handleInvoiceRequest = offerHandler.expectMessageType[HandleInvoiceRequest] val receivingRoutes = Seq( - OfferManager.InvoiceRequestActor.Route(route.hops :+ ChannelHop.dummy(nodes("C").nodeParams.nodeId, 55 msat, 55, CltvExpiryDelta(55)), recipientPaysFees = false, CltvExpiryDelta(555)) + OfferManager.InvoiceRequestActor.Route(route.hops :+ ChannelHop.dummy(nodes("C").nodeParams.nodeId, 55 msat, 55, CltvExpiryDelta(55)), CltvExpiryDelta(555)) ) handleInvoiceRequest.replyTo ! InvoiceRequestActor.ApproveRequest(amount, receivingRoutes, pluginData_opt = Some(hex"eff0")) val handlePayment = offerHandler.expectMessageType[HandlePayment] - assert(handlePayment.offerId == offer.offerId) - assert(handlePayment.pluginData_opt.contains(hex"eff0")) + assert(handlePayment.offer == offer) + assert(handlePayment.invoiceData.pluginData_opt.contains(hex"eff0")) handlePayment.replyTo ! PaymentActor.AcceptPayment() val paymentSent = sender.expectMsgType[PaymentSent] @@ -773,12 +773,12 @@ class PaymentIntegrationSpec extends IntegrationSpec { val route = sender.expectMsgType[Router.RouteResponse].routes.head val handleInvoiceRequest = offerHandler.expectMessageType[HandleInvoiceRequest] - val receivingRoutes = Seq(OfferManager.InvoiceRequestActor.Route(route.hops, recipientPaysFees = false, CltvExpiryDelta(500))) + val receivingRoutes = Seq(OfferManager.InvoiceRequestActor.Route(route.hops, CltvExpiryDelta(500))) handleInvoiceRequest.replyTo ! InvoiceRequestActor.ApproveRequest(amount, receivingRoutes, pluginData_opt = Some(hex"0123")) val handlePayment = offerHandler.expectMessageType[HandlePayment] - assert(handlePayment.offerId == offer.offerId) - assert(handlePayment.pluginData_opt.contains(hex"0123")) + assert(handlePayment.offer == offer) + assert(handlePayment.invoiceData.pluginData_opt.contains(hex"0123")) handlePayment.replyTo ! PaymentActor.AcceptPayment() val paymentSent = sender.expectMsgType[PaymentSent] @@ -821,12 +821,12 @@ class PaymentIntegrationSpec extends IntegrationSpec { ShortChannelIdDir(channelBC.nodeId1 == nodes("B").nodeParams.nodeId, channelBC.shortChannelId) } val receivingRoutes = Seq( - OfferManager.InvoiceRequestActor.Route(route.hops :+ ChannelHop.dummy(nodes("C").nodeParams.nodeId, 55 msat, 55, CltvExpiryDelta(55)), recipientPaysFees = false, CltvExpiryDelta(555), Some(scidDirCB)) + OfferManager.InvoiceRequestActor.Route(route.hops :+ ChannelHop.dummy(nodes("C").nodeParams.nodeId, 55 msat, 55, CltvExpiryDelta(55)), CltvExpiryDelta(555), shortChannelIdDir_opt = Some(scidDirCB)) ) handleInvoiceRequest.replyTo ! InvoiceRequestActor.ApproveRequest(amount, receivingRoutes) val handlePayment = offerHandler.expectMessageType[HandlePayment] - assert(handlePayment.offerId == offer.offerId) + assert(handlePayment.offer == offer) handlePayment.replyTo ! PaymentActor.AcceptPayment() val paymentSent = sender.expectMsgType[PaymentSent] diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/fixtures/MinimalNodeFixture.scala b/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/fixtures/MinimalNodeFixture.scala index 30e9091349..3351257a51 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/fixtures/MinimalNodeFixture.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/fixtures/MinimalNodeFixture.scala @@ -24,13 +24,13 @@ import fr.acinq.eclair.io.{Peer, PeerConnection, PendingChannelsRateLimiter, Swi import fr.acinq.eclair.message.Postman import fr.acinq.eclair.payment.Bolt11Invoice.ExtraHop import fr.acinq.eclair.payment._ -import fr.acinq.eclair.payment.offer.OfferManager +import fr.acinq.eclair.payment.offer.{DefaultHandler, OfferManager} import fr.acinq.eclair.payment.receive.{MultiPartHandler, PaymentHandler} import fr.acinq.eclair.payment.relay.{ChannelRelayer, PostRestartHtlcCleaner, Relayer} import fr.acinq.eclair.payment.send.PaymentInitiator import fr.acinq.eclair.router.Router import fr.acinq.eclair.wire.protocol.IPAddress -import fr.acinq.eclair.{BlockHeight, MilliSatoshi, MilliSatoshiLong, NodeParams, RealShortChannelId, SubscriptionsComplete, TestBitcoinCoreClient, TestDatabases} +import fr.acinq.eclair.{BlockHeight, EclairImpl, Kit, MilliSatoshi, MilliSatoshiLong, NodeParams, RealShortChannelId, SubscriptionsComplete, TestBitcoinCoreClient, TestDatabases, nodeFee} import org.scalatest.concurrent.{Eventually, IntegrationPatience} import org.scalatest.{Assertions, EitherValues} @@ -54,12 +54,15 @@ case class MinimalNodeFixture private(nodeParams: NodeParams, paymentInitiator: ActorRef, paymentHandler: ActorRef, offerManager: typed.ActorRef[OfferManager.Command], + defaultOfferHandler: typed.ActorRef[OfferManager.HandlerCommand], postman: typed.ActorRef[Postman.Command], watcher: TestProbe, wallet: SingleKeyOnChainWallet, bitcoinClient: TestBitcoinCoreClient) { val nodeId = nodeParams.nodeId val routeParams = nodeParams.routerConf.pathFindingExperimentConf.experiments.values.head.getDefaultRouteParams + + val eclairImpl = new EclairImpl(Kit(nodeParams, system, watcher.ref.toTyped, paymentHandler, register, relayer, router, switchboard, paymentInitiator, TestProbe()(system).ref, TestProbe()(system).ref.toTyped, TestProbe()(system).ref.toTyped, postman, offerManager, defaultOfferHandler, wallet)) } object MinimalNodeFixture extends Assertions with Eventually with IntegrationPatience with EitherValues { @@ -94,6 +97,7 @@ object MinimalNodeFixture extends Assertions with Eventually with IntegrationPat val register = system.actorOf(Register.props(), "register") val router = system.actorOf(Router.props(nodeParams, watcherTyped), "router") val offerManager = system.spawn(OfferManager(nodeParams, 1 minute), "offer-manager") + val defaultOfferHandler = system.spawn(DefaultHandler(nodeParams, router), "default-offer-handler") val paymentHandler = system.actorOf(PaymentHandler.props(nodeParams, register, offerManager), "payment-handler") val relayer = system.actorOf(Relayer.props(nodeParams, router, register, paymentHandler), "relayer") val txPublisherFactory = Channel.SimpleTxPublisherFactory(nodeParams, bitcoinClient) @@ -122,6 +126,7 @@ object MinimalNodeFixture extends Assertions with Eventually with IntegrationPat paymentInitiator = paymentInitiator, paymentHandler = paymentHandler, offerManager = offerManager, + defaultOfferHandler = defaultOfferHandler, postman = postman, watcher = watcher, wallet = wallet, diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/payment/OfferPaymentSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/payment/OfferPaymentSpec.scala index 6d4e211e2f..0eead28973 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/payment/OfferPaymentSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/payment/OfferPaymentSpec.scala @@ -20,6 +20,7 @@ import akka.actor.typed.Behavior import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.adapter.{ClassicActorRefOps, ClassicActorSystemOps} import akka.testkit.TestProbe +import akka.util.Timeout import com.softwaremill.quicklens.ModifyPimp import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey import fr.acinq.bitcoin.scalacompat.{ByteVector32, SatoshiLong} @@ -34,12 +35,13 @@ import fr.acinq.eclair.message.OnionMessages.{IntermediateNode, Recipient, build import fr.acinq.eclair.payment._ import fr.acinq.eclair.payment.offer.OfferManager import fr.acinq.eclair.payment.offer.OfferManager.InvoiceRequestActor +import fr.acinq.eclair.payment.relay.Relayer.RelayFees import fr.acinq.eclair.payment.send.OfferPayment import fr.acinq.eclair.payment.send.PaymentInitiator.{SendPaymentToNode, SendSpontaneousPayment} import fr.acinq.eclair.router.Router import fr.acinq.eclair.router.Router.ChannelHop import fr.acinq.eclair.testutils.FixtureSpec -import fr.acinq.eclair.wire.protocol.OfferTypes.{Offer, OfferPaths} +import fr.acinq.eclair.wire.protocol.OfferTypes.{BlindedPath, Offer, OfferPaths} import fr.acinq.eclair.wire.protocol.{IncorrectOrUnknownPaymentDetails, InvalidOnionBlinding} import fr.acinq.eclair.{CltvExpiryDelta, EncodedNodeId, Features, MilliSatoshi, MilliSatoshiLong, ShortChannelId, randomBytes32, randomKey} import org.scalatest.concurrent.IntegrationPatience @@ -47,6 +49,7 @@ import org.scalatest.{Tag, TestData} import scodec.bits.HexStringSyntax import java.util.UUID +import scala.concurrent.Await import scala.concurrent.duration.DurationInt class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { @@ -231,7 +234,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { carol.router ! Router.FinalizeRoute(sender.ref.toTyped, Router.PredefinedNodeRoute(amount, Seq(bob.nodeId, carol.nodeId))) val route = sender.expectMsgType[Router.RouteResponse].routes.head - val routes = Seq(InvoiceRequestActor.Route(route.hops, recipientPaysFees = false, maxFinalExpiryDelta)) + val routes = Seq(InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta)) val (offer, result) = sendOfferPayment(f, alice, carol, amount, routes) val payment = verifyPaymentSuccess(offer, amount, result) assert(payment.parts.length == 1) @@ -247,7 +250,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { carol.router ! Router.FinalizeRoute(sender.ref.toTyped, Router.PredefinedNodeRoute(amount, Seq(bob.nodeId, carol.nodeId))) val route = sender.expectMsgType[Router.RouteResponse].routes.head - val routes = Seq(InvoiceRequestActor.Route(route.hops, recipientPaysFees = true, maxFinalExpiryDelta)) + val routes = Seq(InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta, feeOverride = Some(RelayFees.zero))) val (offer, result) = sendOfferPayment(f, alice, carol, amount, routes) val payment = verifyPaymentSuccess(offer, amount, result) assert(payment.parts.length == 1) @@ -264,8 +267,8 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { val route = sender.expectMsgType[Router.RouteResponse].routes.head val routes = Seq( - InvoiceRequestActor.Route(route.hops, recipientPaysFees = false, maxFinalExpiryDelta), - InvoiceRequestActor.Route(route.hops, recipientPaysFees = false, maxFinalExpiryDelta), + InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta), + InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta), ) val (offer, result) = sendOfferPayment(f, alice, carol, amount, routes, maxAttempts = 3) val payment = verifyPaymentSuccess(offer, amount, result) @@ -283,8 +286,8 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { val route = sender.expectMsgType[Router.RouteResponse].routes.head val routes = Seq( - InvoiceRequestActor.Route(route.hops, recipientPaysFees = true, maxFinalExpiryDelta), - InvoiceRequestActor.Route(route.hops, recipientPaysFees = true, maxFinalExpiryDelta), + InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta, feeOverride = Some(RelayFees.zero)), + InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta, feeOverride = Some(RelayFees.zero)), ) val (offer, result) = sendOfferPayment(f, alice, carol, amount, routes, maxAttempts = 3) val payment = verifyPaymentSuccess(offer, amount, result) @@ -300,7 +303,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { val route = sender.expectMsgType[Router.RouteResponse].routes.head // Carol advertises a single blinded path from Bob to herself. - val routes = Seq(InvoiceRequestActor.Route(route.hops, recipientPaysFees = false, maxFinalExpiryDelta)) + val routes = Seq(InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta)) // We make a first set of payments to ensure channels have less than 50 000 sat on Bob's side. Seq(50_000_000 msat, 50_000_000 msat).foreach(amount => { @@ -332,7 +335,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { carol.router ! Router.FinalizeRoute(sender.ref.toTyped, Router.PredefinedNodeRoute(10_000_000 msat, Seq(bob.nodeId, carol.nodeId))) val route = sender.expectMsgType[Router.RouteResponse].routes.head - val routes = Seq(InvoiceRequestActor.Route(route.hops, recipientPaysFees = false, maxFinalExpiryDelta)) + val routes = Seq(InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta)) val amount1 = 150_000_000 msat val (offer, result) = sendPrivateOfferPayment(f, alice, carol, amount1, routes, maxAttempts = 3) val payment = verifyPaymentSuccess(offer, amount1, result) @@ -348,8 +351,8 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { val amount = 125_000_000 msat val routes = Seq( - InvoiceRequestActor.Route(route.hops :+ ChannelHop.dummy(carol.nodeId, 150 msat, 0, CltvExpiryDelta(50)), recipientPaysFees = false, maxFinalExpiryDelta), - InvoiceRequestActor.Route(route.hops ++ Seq(ChannelHop.dummy(carol.nodeId, 50 msat, 0, CltvExpiryDelta(20)), ChannelHop.dummy(carol.nodeId, 100 msat, 0, CltvExpiryDelta(30))), recipientPaysFees = false, maxFinalExpiryDelta), + InvoiceRequestActor.Route(route.hops :+ ChannelHop.dummy(carol.nodeId, 150 msat, 0, CltvExpiryDelta(50)), maxFinalExpiryDelta), + InvoiceRequestActor.Route(route.hops ++ Seq(ChannelHop.dummy(carol.nodeId, 50 msat, 0, CltvExpiryDelta(20)), ChannelHop.dummy(carol.nodeId, 100 msat, 0, CltvExpiryDelta(30))), maxFinalExpiryDelta), ) val (offer, result) = sendOfferPayment(f, alice, carol, amount, routes) val payment = verifyPaymentSuccess(offer, amount, result) @@ -366,8 +369,8 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { val amount = 125_000_000 msat val routes = Seq( - InvoiceRequestActor.Route(route.hops :+ ChannelHop.dummy(carol.nodeId, 150 msat, 0, CltvExpiryDelta(50)), recipientPaysFees = true, maxFinalExpiryDelta), - InvoiceRequestActor.Route(route.hops ++ Seq(ChannelHop.dummy(carol.nodeId, 50 msat, 0, CltvExpiryDelta(20)), ChannelHop.dummy(carol.nodeId, 100 msat, 0, CltvExpiryDelta(30))), recipientPaysFees = true, maxFinalExpiryDelta), + InvoiceRequestActor.Route(route.hops :+ ChannelHop.dummy(carol.nodeId, 150 msat, 0, CltvExpiryDelta(50)), maxFinalExpiryDelta, feeOverride = Some(RelayFees.zero)), + InvoiceRequestActor.Route(route.hops ++ Seq(ChannelHop.dummy(carol.nodeId, 50 msat, 0, CltvExpiryDelta(20)), ChannelHop.dummy(carol.nodeId, 100 msat, 0, CltvExpiryDelta(30))), maxFinalExpiryDelta, feeOverride = Some(RelayFees.zero)), ) val (offer, result) = sendOfferPayment(f, alice, carol, amount, routes) val payment = verifyPaymentSuccess(offer, amount, result) @@ -384,7 +387,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { carol.router ! Router.FinalizeRoute(sender.ref.toTyped, Router.PredefinedNodeRoute(amount, Seq(bob.nodeId, carol.nodeId))) val route = sender.expectMsgType[Router.RouteResponse].routes.head - val routes = Seq(InvoiceRequestActor.Route(route.hops, recipientPaysFees = false, maxFinalExpiryDelta)) + val routes = Seq(InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta)) val (offer, result) = sendPrivateOfferPayment(f, alice, carol, amount, routes) val payment = verifyPaymentSuccess(offer, amount, result) assert(payment.parts.forall(_.feesPaid > 0.msat)) @@ -399,7 +402,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { carol.router ! Router.FinalizeRoute(sender.ref.toTyped, Router.PredefinedNodeRoute(amount, Seq(bob.nodeId, carol.nodeId))) val route = sender.expectMsgType[Router.RouteResponse].routes.head - val routes = Seq(InvoiceRequestActor.Route(route.hops, recipientPaysFees = true, maxFinalExpiryDelta)) + val routes = Seq(InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta, feeOverride = Some(RelayFees.zero))) val (offer, result) = sendPrivateOfferPayment(f, alice, carol, amount, routes) val payment = verifyPaymentSuccess(offer, amount, result) assert(payment.parts.forall(_.feesPaid == 0.msat)) @@ -409,7 +412,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { import f._ val amount = 75_000_000 msat - val routes = Seq(InvoiceRequestActor.Route(Nil, recipientPaysFees = false, maxFinalExpiryDelta)) + val routes = Seq(InvoiceRequestActor.Route(Nil, maxFinalExpiryDelta)) val (offer, result) = sendOfferPayment(f, alice, bob, amount, routes) val payment = verifyPaymentSuccess(offer, amount, result) assert(payment.parts.length == 1) @@ -419,7 +422,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { import f._ val amount = 250_000_000 msat - val routes = Seq(InvoiceRequestActor.Route(Seq(ChannelHop.dummy(bob.nodeId, 10 msat, 25, CltvExpiryDelta(24)), ChannelHop.dummy(bob.nodeId, 5 msat, 10, CltvExpiryDelta(36))), recipientPaysFees = false, maxFinalExpiryDelta)) + val routes = Seq(InvoiceRequestActor.Route(Seq(ChannelHop.dummy(bob.nodeId, 10 msat, 25, CltvExpiryDelta(24)), ChannelHop.dummy(bob.nodeId, 5 msat, 10, CltvExpiryDelta(36))), maxFinalExpiryDelta)) val (offer, result) = sendOfferPayment(f, alice, bob, amount, routes) val payment = verifyPaymentSuccess(offer, amount, result) assert(payment.parts.length == 1) @@ -430,7 +433,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { import f._ val amount = 250_000_000 msat - val routes = Seq(InvoiceRequestActor.Route(Seq(ChannelHop.dummy(bob.nodeId, 10 msat, 25, CltvExpiryDelta(24)), ChannelHop.dummy(bob.nodeId, 5 msat, 10, CltvExpiryDelta(36))), recipientPaysFees = true, maxFinalExpiryDelta)) + val routes = Seq(InvoiceRequestActor.Route(Seq(ChannelHop.dummy(bob.nodeId, 10 msat, 25, CltvExpiryDelta(24)), ChannelHop.dummy(bob.nodeId, 5 msat, 10, CltvExpiryDelta(36))), maxFinalExpiryDelta, feeOverride = Some(RelayFees.zero))) val (offer, result) = sendOfferPayment(f, alice, bob, amount, routes) val payment = verifyPaymentSuccess(offer, amount, result) assert(payment.parts.length == 1) @@ -446,7 +449,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { carol.router ! Router.FinalizeRoute(sender.ref.toTyped, Router.PredefinedNodeRoute(amount, Seq(bob.nodeId, carol.nodeId))) val route = sender.expectMsgType[Router.RouteResponse].routes.head - val routes = Seq(InvoiceRequestActor.Route(route.hops, recipientPaysFees = false, maxFinalExpiryDelta)) + val routes = Seq(InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta)) val (offer, result) = sendOfferPayment(f, bob, carol, amount, routes) val payment = verifyPaymentSuccess(offer, amount, result) assert(payment.parts.length == 1) @@ -464,7 +467,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { val route = sender.expectMsgType[Router.RouteResponse].routes.head // Carol creates a blinded path using that channel. - val routes = Seq(InvoiceRequestActor.Route(route.hops, recipientPaysFees = false, maxFinalExpiryDelta)) + val routes = Seq(InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta)) // We make a payment to ensure that the channel contains less than 150 000 sat on Bob's side. assert(sendPayment(bob, carol, 50_000_000 msat).isRight) @@ -489,7 +492,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { carol.router ! Router.FinalizeRoute(sender.ref.toTyped, Router.PredefinedNodeRoute(amount, Seq(bob.nodeId, carol.nodeId))) val route = sender.expectMsgType[Router.RouteResponse].routes.head - val routes = Seq(InvoiceRequestActor.Route(route.hops :+ ChannelHop.dummy(carol.nodeId, 25 msat, 250, CltvExpiryDelta(75)), recipientPaysFees = false, maxFinalExpiryDelta)) + val routes = Seq(InvoiceRequestActor.Route(route.hops :+ ChannelHop.dummy(carol.nodeId, 25 msat, 250, CltvExpiryDelta(75)), maxFinalExpiryDelta)) val (offer, result) = sendOfferPayment(f, bob, carol, amount, routes) val payment = verifyPaymentSuccess(offer, amount, result) assert(payment.parts.length == 1) @@ -515,7 +518,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { // Carol receives a first payment through those channels. { - val routes = Seq(InvoiceRequestActor.Route(route.hops, recipientPaysFees = false, maxFinalExpiryDelta)) + val routes = Seq(InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta)) val amount1 = 100_000_000 msat val (offer, result) = sendOfferPayment(f, alice, carol, amount1, routes) val payment = verifyPaymentSuccess(offer, amount1, result) @@ -531,7 +534,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { // Carol receives a second payment that requires using MPP. { - val routes = Seq(InvoiceRequestActor.Route(route.hops, recipientPaysFees = false, maxFinalExpiryDelta)) + val routes = Seq(InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta)) val amount2 = 200_000_000 msat val (offer, result) = sendOfferPayment(f, alice, carol, amount2, routes, maxAttempts = 3) val payment = verifyPaymentSuccess(offer, amount2, result) @@ -560,7 +563,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { val route = sender.expectMsgType[Router.RouteResponse].routes.head // Carol receives a payment that requires using MPP. - val routes = Seq(InvoiceRequestActor.Route(route.hops, recipientPaysFees = false, maxFinalExpiryDelta)) + val routes = Seq(InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta)) val amount = 300_000_000 msat val (offer, result) = sendOfferPayment(f, alice, carol, amount, routes, maxAttempts = 3) val payment = verifyPaymentSuccess(offer, amount, result) @@ -588,7 +591,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { val route = sender.expectMsgType[Router.RouteResponse].routes.head // Carol receives a payment that requires using MPP. - val routes = Seq(InvoiceRequestActor.Route(route.hops, recipientPaysFees = false, maxFinalExpiryDelta)) + val routes = Seq(InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta)) val amount = 200_000_000 msat val (offer, result) = sendOfferPayment(f, alice, carol, amount, routes, maxAttempts = 3) val payment = verifyPaymentSuccess(offer, amount, result) @@ -618,7 +621,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { sender.expectMsgType[PaymentSent] }) // Bob now doesn't have enough funds to relay the payment. - val routes = Seq(InvoiceRequestActor.Route(route.hops, recipientPaysFees = false, maxFinalExpiryDelta)) + val routes = Seq(InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta)) val (_, result) = sendOfferPayment(f, alice, carol, 75_000_000 msat, routes) verifyBlindedFailure(result, bob.nodeId) } @@ -630,7 +633,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { carol.router ! Router.FinalizeRoute(sender.ref.toTyped, Router.PredefinedNodeRoute(25_000_000 msat, Seq(bob.nodeId, carol.nodeId))) val route = sender.expectMsgType[Router.RouteResponse].routes.head - val routes = Seq(InvoiceRequestActor.Route(route.hops, recipientPaysFees = false, CltvExpiryDelta(-500))) + val routes = Seq(InvoiceRequestActor.Route(route.hops, CltvExpiryDelta(-500))) val (_, result) = sendOfferPayment(f, alice, carol, 25_000_000 msat, routes) verifyBlindedFailure(result, bob.nodeId) } @@ -645,7 +648,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { carol.router ! Router.FinalizeRoute(sender.ref.toTyped, Router.PredefinedNodeRoute(recipientAmount, Seq(bob.nodeId, carol.nodeId))) val route = sender.expectMsgType[Router.RouteResponse].routes.head - val routes = Seq(InvoiceRequestActor.Route(route.hops, recipientPaysFees = false, maxFinalExpiryDelta)) + val routes = Seq(InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta)) // The amount is below what Carol expects. val payment = sendOfferPaymentWithInvalidAmount(f, alice, carol, payerAmount, recipientAmount, routes) verifyBlindedFailure(payment, bob.nodeId) @@ -656,7 +659,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { val payerAmount = 25_000_000 msat val recipientAmount = 50_000_000 msat - val routes = Seq(InvoiceRequestActor.Route(Nil, recipientPaysFees = false, maxFinalExpiryDelta)) + val routes = Seq(InvoiceRequestActor.Route(Nil, maxFinalExpiryDelta)) // The amount is below what Bob expects: since he is both the introduction node and the final recipient, he sends // back a normal error. val payment = sendOfferPaymentWithInvalidAmount(f, alice, bob, payerAmount, recipientAmount, routes) @@ -672,7 +675,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { val payerAmount = 25_000_000 msat val recipientAmount = 50_000_000 msat - val routes = Seq(InvoiceRequestActor.Route(Seq(ChannelHop.dummy(bob.nodeId, 1 msat, 100, CltvExpiryDelta(48))), recipientPaysFees = false, maxFinalExpiryDelta)) + val routes = Seq(InvoiceRequestActor.Route(Seq(ChannelHop.dummy(bob.nodeId, 1 msat, 100, CltvExpiryDelta(48))), maxFinalExpiryDelta)) // The amount is below what Bob expects: since he is both the introduction node and the final recipient, he sends // back a normal error. val payment = sendOfferPaymentWithInvalidAmount(f, alice, bob, payerAmount, recipientAmount, routes) @@ -693,7 +696,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { carol.router ! Router.FinalizeRoute(sender.ref.toTyped, Router.PredefinedNodeRoute(recipientAmount, Seq(bob.nodeId, carol.nodeId))) val route = sender.expectMsgType[Router.RouteResponse].routes.head - val routes = Seq(InvoiceRequestActor.Route(route.hops, recipientPaysFees = false, maxFinalExpiryDelta)) + val routes = Seq(InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta)) // The amount is below what Carol expects. val payment = sendOfferPaymentWithInvalidAmount(f, bob, carol, payerAmount, recipientAmount, routes) assert(payment.failures.head.isInstanceOf[PaymentFailure]) @@ -718,7 +721,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { carol.router ! Router.FinalizeRoute(sender.ref.toTyped, Router.PredefinedNodeRoute(amount, Seq(bob.nodeId, carol.nodeId))) val route = sender.expectMsgType[Router.RouteResponse].routes.head - val receivingRoute = InvoiceRequestActor.Route(route.hops, recipientPaysFees = false, maxFinalExpiryDelta) + val receivingRoute = InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta) val handler = carol.system.spawnAnonymous(offerHandler(amount, Seq(receivingRoute))) carol.offerManager ! OfferManager.RegisterOffer(compactOffer, Some(recipientKey), Some(pathId), handler) val offerPayment = alice.system.spawnAnonymous(OfferPayment(alice.nodeParams, alice.postman, alice.router, alice.register, alice.paymentInitiator)) @@ -740,7 +743,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { val offerPaths = Seq(OnionMessages.buildRoute(randomKey(), Seq(IntermediateNode(bob.nodeId)), Recipient(carol.nodeId, Some(pathId))).route) val offer = Offer.withPaths(None, Some("implicit node id"), offerPaths, Features.empty, carol.nodeParams.chainHash) - val handler = carol.system.spawnAnonymous(offerHandler(amount, Seq(InvoiceRequestActor.Route(route.hops, recipientPaysFees = false, maxFinalExpiryDelta)))) + val handler = carol.system.spawnAnonymous(offerHandler(amount, Seq(InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta)))) carol.offerManager ! OfferManager.RegisterOffer(offer, None, Some(pathId), handler) val offerPayment = alice.system.spawnAnonymous(OfferPayment(alice.nodeParams, alice.postman, alice.router, alice.register, alice.paymentInitiator)) val sendPaymentConfig = OfferPayment.SendPaymentConfig(None, connectDirectly = false, maxAttempts = 1, alice.routeParams, blocking = true) @@ -749,4 +752,75 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { val payment = verifyPaymentSuccess(offer, amount, result) assert(payment.parts.length == 1) } + + test("basic offer") { f => + import f._ + + implicit val timeout: Timeout = 10 seconds + + val amount = 20_000_000 msat + val offer = Await.result(carol.eclairImpl.createOffer(description_opt = Some("test offer"), amount_opt = Some(amount), expiry_opt = None, issuer_opt = None, firstNodeId_opt = None), 10 seconds) + + assert(offer.nodeId == Some(carol.nodeId)) + assert(offer.description == Some("test offer")) + assert(offer.amount == Some(amount)) + + val payment = Await.result(alice.eclairImpl.payOfferBlocking(offer, amount, 1), 10 seconds) + assert(payment.isInstanceOf[PaymentSent]) + assert(payment.asInstanceOf[PaymentSent].feesPaid > 0.msat) + } + + test("offer without node id (dummy hops only)") { f => + import f._ + + implicit val timeout: Timeout = 10 seconds + + val amount = 20_000_000 msat + val offer = Await.result(carol.eclairImpl.createOffer(description_opt = Some("test offer"), amount_opt = Some(amount), expiry_opt = None, issuer_opt = None, firstNodeId_opt = Some(carol.nodeId)), 10 seconds) + + assert(offer.nodeId == None) + assert(offer.contactInfos.head.asInstanceOf[BlindedPath].route.firstNodeId == EncodedNodeId.WithPublicKey.Plain(carol.nodeId)) + assert(offer.description == Some("test offer")) + assert(offer.amount == Some(amount)) + + val payment = Await.result(alice.eclairImpl.payOfferBlocking(offer, amount, 1), 10 seconds) + assert(payment.isInstanceOf[PaymentSent]) + assert(payment.asInstanceOf[PaymentSent].feesPaid > 0.msat) + } + + test("offer without node id (real and dummy blinded hops)") { f => + import f._ + + implicit val timeout: Timeout = 10 seconds + + val amount = 20_000_000 msat + val offer = Await.result(carol.eclairImpl.createOffer(description_opt = Some("test offer"), amount_opt = Some(amount), expiry_opt = None, issuer_opt = None, firstNodeId_opt = Some(bob.nodeId)), 10 seconds) + + assert(offer.nodeId == None) + assert(offer.contactInfos.head.asInstanceOf[BlindedPath].route.firstNodeId == EncodedNodeId.WithPublicKey.Plain(bob.nodeId)) + assert(offer.description == Some("test offer")) + assert(offer.amount == Some(amount)) + + val payment = Await.result(alice.eclairImpl.payOfferBlocking(offer, amount, 1), 10 seconds) + assert(payment.isInstanceOf[PaymentSent]) + assert(payment.asInstanceOf[PaymentSent].feesPaid == 0.msat) + } + + test("offer without node id (payer is first node of blinded path)") { f => + import f._ + + implicit val timeout: Timeout = 10 seconds + + val amount = 20_000_000 msat + val offer = Await.result(carol.eclairImpl.createOffer(description_opt = Some("test offer"), amount_opt = Some(amount), expiry_opt = None, issuer_opt = None, firstNodeId_opt = Some(alice.nodeId)), 10 seconds) + + assert(offer.nodeId == None) + assert(offer.contactInfos.head.asInstanceOf[BlindedPath].route.firstNodeId == EncodedNodeId.WithPublicKey.Plain(alice.nodeId)) + assert(offer.description == Some("test offer")) + assert(offer.amount == Some(amount)) + + val payment = Await.result(alice.eclairImpl.payOfferBlocking(offer, amount, 1), 10 seconds) + assert(payment.isInstanceOf[PaymentSent]) + assert(payment.asInstanceOf[PaymentSent].feesPaid < 0.msat) + } } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/offer/OfferManagerSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/offer/OfferManagerSpec.scala index 0f0b8596df..c8c42636f4 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/offer/OfferManagerSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/offer/OfferManagerSpec.scala @@ -28,6 +28,7 @@ import fr.acinq.eclair.payment.Bolt12Invoice import fr.acinq.eclair.payment.offer.OfferManager._ import fr.acinq.eclair.payment.receive.MultiPartHandler import fr.acinq.eclair.payment.receive.MultiPartHandler.GetIncomingPaymentActor.{ProcessPayment, RejectPayment} +import fr.acinq.eclair.payment.relay.Relayer.RelayFees import fr.acinq.eclair.router.Router.ChannelHop import fr.acinq.eclair.wire.protocol.OfferTypes.{InvoiceRequest, Offer} import fr.acinq.eclair.wire.protocol.RouteBlindingEncryptedDataCodecs.RouteBlindingDecryptedData @@ -76,7 +77,7 @@ class OfferManagerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app val handleInvoiceRequest = handler.expectMessageType[HandleInvoiceRequest] assert(handleInvoiceRequest.invoiceRequest.isValid) assert(handleInvoiceRequest.invoiceRequest.payerId == payerKey.publicKey) - handleInvoiceRequest.replyTo ! InvoiceRequestActor.ApproveRequest(amount, Seq(InvoiceRequestActor.Route(hops, hideFees, CltvExpiryDelta(1000))), pluginData_opt) + handleInvoiceRequest.replyTo ! InvoiceRequestActor.ApproveRequest(amount, Seq(InvoiceRequestActor.Route(hops, CltvExpiryDelta(1000), feeOverride = if (hideFees) Some(RelayFees.zero) else None)), pluginData_opt) val invoiceMessage = postman.expectMessageType[Postman.SendMessage] val Right(invoice) = Bolt12Invoice.validate(invoiceMessage.message.get[OnionMessagePayloadTlv.Invoice].get.tlvs) assert(invoice.validateFor(handleInvoiceRequest.invoiceRequest, pathNodeId).isRight) @@ -126,8 +127,8 @@ class OfferManagerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app val paymentPayload = createPaymentPayload(f, invoice) offerManager ! ReceivePayment(paymentHandler.ref, invoice.paymentHash, paymentPayload, amount) val handlePayment = handler.expectMessageType[HandlePayment] - assert(handlePayment.offerId == offer.offerId) - assert(handlePayment.pluginData_opt.contains(hex"deadbeef")) + assert(handlePayment.offer == offer) + assert(handlePayment.invoiceData.pluginData_opt.contains(hex"deadbeef")) handlePayment.replyTo ! PaymentActor.AcceptPayment() val ProcessPayment(incomingPayment, _) = paymentHandler.expectMessageType[ProcessPayment] assert(Crypto.sha256(incomingPayment.paymentPreimage) == invoice.paymentHash) @@ -298,7 +299,7 @@ class OfferManagerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app offerManager ! ReceivePayment(paymentHandler.ref, invoice.paymentHash, paymentPayload, amount) val handlePayment = handler.expectMessageType[HandlePayment] - assert(handlePayment.offerId == offer.offerId) + assert(handlePayment.offer == offer) handlePayment.replyTo ! PaymentActor.AcceptPayment() val ProcessPayment(incomingPayment, maxRecipientPathFees) = paymentHandler.expectMessageType[ProcessPayment] assert(Crypto.sha256(incomingPayment.paymentPreimage) == invoice.paymentHash) @@ -324,7 +325,7 @@ class OfferManagerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app offerManager ! ReceivePayment(paymentHandler.ref, invoice.paymentHash, paymentPayload, amountReceived) val handlePayment = handler.expectMessageType[HandlePayment] - assert(handlePayment.offerId == offer.offerId) + assert(handlePayment.offer == offer) handlePayment.replyTo ! PaymentActor.AcceptPayment() val ProcessPayment(incomingPayment, maxRecipientPathFees) = paymentHandler.expectMessageType[ProcessPayment] assert(Crypto.sha256(incomingPayment.paymentPreimage) == invoice.paymentHash) From 242fa8c4f2f998dc4627cd19a230d3738b9988c8 Mon Sep 17 00:00:00 2001 From: Thomas HUET Date: Thu, 6 Mar 2025 15:41:18 +0100 Subject: [PATCH 2/7] comments t-bast --- docs/release-notes/eclair-vnext.md | 33 ++++++++++++ eclair-core/src/main/resources/reference.conf | 6 ++- .../main/scala/fr/acinq/eclair/Eclair.scala | 18 +++---- .../scala/fr/acinq/eclair/NodeParams.scala | 8 +-- .../main/scala/fr/acinq/eclair/Setup.scala | 6 +-- .../scala/fr/acinq/eclair/db/Databases.scala | 10 ++-- .../fr/acinq/eclair/db/DualDatabases.scala | 7 +-- .../scala/fr/acinq/eclair/db/OffersDb.scala | 5 -- .../fr/acinq/eclair/db/pg/PgOffersDb.scala | 28 ++++------- .../fr/acinq/eclair/db/pg/PgPaymentsDb.scala | 4 +- .../eclair/db/sqlite/SqliteOffersDb.scala | 24 ++++----- ...andler.scala => DefaultOfferHandler.scala} | 6 +-- .../eclair/payment/offer/OfferCreator.scala | 37 ++++++++------ .../eclair/payment/offer/OfferManager.scala | 13 ++++- .../scala/fr/acinq/eclair/TestDatabases.scala | 2 +- .../fr/acinq/eclair/db/OffersDbSpec.scala | 5 +- .../basic/fixtures/MinimalNodeFixture.scala | 4 +- .../basic/payment/OfferPaymentSpec.scala | 8 +-- .../scala/fr/acinq/eclair/api/Service.scala | 4 +- .../fr/acinq/eclair/api/handlers/Offer.scala | 50 +++++++++++++++++++ 20 files changed, 178 insertions(+), 100 deletions(-) rename eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/{DefaultHandler.scala => DefaultOfferHandler.scala} (99%) create mode 100644 eclair-node/src/main/scala/fr/acinq/eclair/api/handlers/Offer.scala diff --git a/docs/release-notes/eclair-vnext.md b/docs/release-notes/eclair-vnext.md index 133362df1b..76cd6f3e96 100644 --- a/docs/release-notes/eclair-vnext.md +++ b/docs/release-notes/eclair-vnext.md @@ -9,6 +9,39 @@ With this release, eclair requires using Bitcoin Core 28.1. Newer versions of Bitcoin Core may be used, but have not been extensively tested. +### Offers + +You can now create an offer with + +``` +./eclair-cli createoffer --description=coffee --amountMsat=20000 --expireInSeconds=3600 --issuer=me@example.com --blindedPathsFirstNodeId=03864ef025fde8fb587d989186ce6a4a186895ee44a926bfc370e2c366597a3f8f +``` + +All parameters are optional and omitting all of them will create a minimal offer with your public node id. +You can also disable offers and list offers with + +``` +./eclair-cli disableoffer --offer=lnoxxx +./eclair-cli listoffers +``` + +If you specify `--blindedPathsFirstNodeId`, your public node id will not be in the offer, you will instead be hidden behind a blinded path starting at the node that you have chosen. +You can configure the number and length of blinded paths used in `eclair.conf`: + +``` +offers { + // Minimum length of an offer blinded path + message-path-min-length = 2 + + // Number of payment paths to put in Bolt12 invoices + payment-path-count = 2 + // Length of payment paths to put in Bolt12 invoices + payment-path-length = 4 + // Expiry delta of payment paths to put in Bolt12 invoices + payment-path-expiry-delta = 500 +} +``` + ### Simplified mutual close This release includes support for the latest [mutual close protocol](https://github.com/lightning/bolts/pull/1205). diff --git a/eclair-core/src/main/resources/reference.conf b/eclair-core/src/main/resources/reference.conf index 6e32d5165c..3b903dafd9 100644 --- a/eclair-core/src/main/resources/reference.conf +++ b/eclair-core/src/main/resources/reference.conf @@ -637,11 +637,15 @@ eclair { cleanup-frequency = 1 day } - managed-offers { + offers { + // Minimum length of an offer blinded path when hiding our real node id message-path-min-length = 2 + // Number of payment paths to put in Bolt12 invoices when hiding our real node id payment-path-count = 2 + // Length of payment paths to put in Bolt12 invoices when hiding our real node id payment-path-length = 4 + // Expiry delta of payment paths to put in Bolt12 invoices when hiding our real node id payment-path-expiry-delta = 500 } } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala b/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala index 708dc5c722..fc7a6fb913 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala @@ -128,9 +128,9 @@ trait Eclair { def receive(description: Either[String, ByteVector32], amount_opt: Option[MilliSatoshi], expire_opt: Option[Long], fallbackAddress_opt: Option[String], paymentPreimage_opt: Option[ByteVector32], privateChannelIds_opt: Option[List[ByteVector32]])(implicit timeout: Timeout): Future[Bolt11Invoice] - def createOffer(description_opt: Option[String], amount_opt: Option[MilliSatoshi], expiry_opt: Option[TimestampSecond], issuer_opt: Option[String], firstNodeId_opt: Option[PublicKey])(implicit timeout: Timeout): Future[Offer] + def createOffer(description_opt: Option[String], amount_opt: Option[MilliSatoshi], expire_opt: Option[Long], issuer_opt: Option[String], blindedPathsFirstNodeId_opt: Option[PublicKey])(implicit timeout: Timeout): Future[Offer] - def disableOffer(offer: Offer)(implicit timeout: Timeout): Unit + def disableOffer(offer: Offer)(implicit timeout: Timeout): Future[Unit] def listOffers(onlyActive: Boolean = true)(implicit timeout: Timeout): Future[Seq[Offer]] @@ -396,22 +396,22 @@ class EclairImpl(val appKit: Kit) extends Eclair with Logging with SpendFromChan } } - override def createOffer(description_opt: Option[String], amount_opt: Option[MilliSatoshi], expiry_opt: Option[TimestampSecond], issuer_opt: Option[String], firstNodeId_opt: Option[PublicKey])(implicit timeout: Timeout): Future[Offer] = { + override def createOffer(description_opt: Option[String], amount_opt: Option[MilliSatoshi], expireInSeconds_opt: Option[Long], issuer_opt: Option[String], blindedPathsFirstNodeId_opt: Option[PublicKey])(implicit timeout: Timeout): Future[Offer] = { val offerCreator = appKit.system.spawnAnonymous(OfferCreator(appKit.nodeParams, appKit.router, appKit.offerManager, appKit.defaultOfferHandler)) - offerCreator.ask[Either[String, Offer]](replyTo => OfferCreator.Create(replyTo, description_opt, amount_opt, expiry_opt, issuer_opt, firstNodeId_opt)) + val expiry_opt = expireInSeconds_opt.map(TimestampSecond.now() + _) + offerCreator.ask[OfferCreator.CreateOfferResult](replyTo => OfferCreator.Create(replyTo, description_opt, amount_opt, expiry_opt, issuer_opt, blindedPathsFirstNodeId_opt)) .flatMap { - case Left(errorMessage) => Future.failed(new Exception(errorMessage)) - case Right(offer) => Future.successful(offer) + case OfferCreator.CreateOfferError(reason) => Future.failed(new Exception(reason)) + case OfferCreator.CreatedOffer(offer) => Future.successful(offer) } } - override def disableOffer(offer: Offer)(implicit timeout: Timeout): Unit = { + override def disableOffer(offer: Offer)(implicit timeout: Timeout): Future[Unit] = Future { appKit.offerManager ! OfferManager.DisableOffer(offer) - appKit.nodeParams.db.managedOffers.disableOffer(offer) } override def listOffers(onlyActive: Boolean = true)(implicit timeout: Timeout): Future[Seq[Offer]] = Future { - appKit.nodeParams.db.managedOffers.listOffers(onlyActive).map(_.offer) + appKit.nodeParams.db.offers.listOffers(onlyActive).map(_.offer) } override def newAddress(): Future[String] = { diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/NodeParams.scala b/eclair-core/src/main/scala/fr/acinq/eclair/NodeParams.scala index 1d349054cf..59d0dfca97 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/NodeParams.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/NodeParams.scala @@ -709,10 +709,10 @@ object NodeParams extends Logging { cleanUpFrequency = FiniteDuration(config.getDuration("peer-storage.cleanup-frequency").getSeconds, TimeUnit.SECONDS), ), offersConfig = OffersConfig( - messagePathMinLength = config.getInt("managed-offers.message-path-min-length"), - paymentPathCount = config.getInt("managed-offers.payment-path-count"), - paymentPathLength = config.getInt("managed-offers.payment-path-length"), - paymentPathCltvExpiryDelta = CltvExpiryDelta(config.getInt("managed-offers.payment-path-expiry-delta")), + messagePathMinLength = config.getInt("offers.message-path-min-length"), + paymentPathCount = config.getInt("offers.payment-path-count"), + paymentPathLength = config.getInt("offers.payment-path-length"), + paymentPathCltvExpiryDelta = CltvExpiryDelta(config.getInt("offers.payment-path-expiry-delta")), ) ) } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/Setup.scala b/eclair-core/src/main/scala/fr/acinq/eclair/Setup.scala index 2c05b28e01..673e87789f 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/Setup.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/Setup.scala @@ -40,7 +40,7 @@ import fr.acinq.eclair.db.FileBackupHandler.FileBackupParams import fr.acinq.eclair.db.{Databases, DbEventHandler, FileBackupHandler, PeerStorageCleaner} import fr.acinq.eclair.io._ import fr.acinq.eclair.message.Postman -import fr.acinq.eclair.payment.offer.{DefaultHandler, OfferManager} +import fr.acinq.eclair.payment.offer.{DefaultOfferHandler, OfferManager} import fr.acinq.eclair.payment.receive.PaymentHandler import fr.acinq.eclair.payment.relay.{AsyncPaymentTriggerer, PostRestartHtlcCleaner, Relayer} import fr.acinq.eclair.payment.send.{Autoprobe, PaymentInitiator} @@ -358,8 +358,8 @@ class Setup(val datadir: File, dbEventHandler = system.actorOf(SimpleSupervisor.props(DbEventHandler.props(nodeParams), "db-event-handler", SupervisorStrategy.Resume)) register = system.actorOf(SimpleSupervisor.props(Register.props(), "register", SupervisorStrategy.Resume)) offerManager = system.spawn(Behaviors.supervise(OfferManager(nodeParams, paymentTimeout = 1 minute)).onFailure(typed.SupervisorStrategy.resume), name = "offer-manager") - defaultOfferHandler = system.spawn(Behaviors.supervise(DefaultHandler(nodeParams, router)).onFailure(typed.SupervisorStrategy.resume), name = "default-offer-handler") - _ = for (offer <- nodeParams.db.managedOffers.listOffers(onlyActive = true)) offerManager ! OfferManager.RegisterOffer(offer.offer, None, offer.pathId_opt, defaultOfferHandler) + defaultOfferHandler = system.spawn(Behaviors.supervise(DefaultOfferHandler(nodeParams, router)).onFailure(typed.SupervisorStrategy.resume), name = "default-offer-handler") + _ = for (offer <- nodeParams.db.offers.listOffers(onlyActive = true)) offerManager ! OfferManager.RegisterOffer(offer.offer, None, offer.pathId_opt, defaultOfferHandler) paymentHandler = system.actorOf(SimpleSupervisor.props(PaymentHandler.props(nodeParams, register, offerManager), "payment-handler", SupervisorStrategy.Resume)) triggerer = system.spawn(Behaviors.supervise(AsyncPaymentTriggerer()).onFailure(typed.SupervisorStrategy.resume), name = "async-payment-triggerer") peerReadyManager = system.spawn(Behaviors.supervise(PeerReadyManager()).onFailure(typed.SupervisorStrategy.restart), name = "peer-ready-manager") diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/db/Databases.scala b/eclair-core/src/main/scala/fr/acinq/eclair/db/Databases.scala index 4ef14b8dd8..c7f929f572 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/db/Databases.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/db/Databases.scala @@ -43,7 +43,7 @@ trait Databases { def channels: ChannelsDb def peers: PeersDb def payments: PaymentsDb - def managedOffers: OffersDb + def offers: OffersDb def pendingCommands: PendingCommandsDb def liquidity: LiquidityDb //@formatter:on @@ -67,7 +67,7 @@ object Databases extends Logging { channels: SqliteChannelsDb, peers: SqlitePeersDb, payments: SqlitePaymentsDb, - managedOffers: SqliteOffersDb, + offers: SqliteOffersDb, pendingCommands: SqlitePendingCommandsDb, private val backupConnection: Connection) extends Databases with FileBackup { override def backup(backupFile: File): Unit = SqliteUtils.using(backupConnection.createStatement()) { @@ -87,7 +87,7 @@ object Databases extends Logging { channels = new SqliteChannelsDb(eclairJdbc), peers = new SqlitePeersDb(eclairJdbc), payments = new SqlitePaymentsDb(eclairJdbc), - managedOffers = new SqliteOffersDb(eclairJdbc), + offers = new SqliteOffersDb(eclairJdbc), pendingCommands = new SqlitePendingCommandsDb(eclairJdbc), backupConnection = eclairJdbc ) @@ -100,7 +100,7 @@ object Databases extends Logging { channels: PgChannelsDb, peers: PgPeersDb, payments: PgPaymentsDb, - managedOffers: PgOffersDb, + offers: PgOffersDb, pendingCommands: PgPendingCommandsDb, dataSource: HikariDataSource, lock: PgLock) extends Databases with ExclusiveLock { @@ -161,7 +161,7 @@ object Databases extends Logging { channels = new PgChannelsDb, peers = new PgPeersDb, payments = new PgPaymentsDb, - managedOffers = new PgOffersDb, + offers = new PgOffersDb, pendingCommands = new PgPendingCommandsDb, dataSource = ds, lock = lock) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/db/DualDatabases.scala b/eclair-core/src/main/scala/fr/acinq/eclair/db/DualDatabases.scala index c0a3a7eed4..84c26d6ef6 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/db/DualDatabases.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/db/DualDatabases.scala @@ -36,7 +36,7 @@ case class DualDatabases(primary: Databases, secondary: Databases) extends Datab override val channels: ChannelsDb = DualChannelsDb(primary.channels, secondary.channels) override val peers: PeersDb = DualPeersDb(primary.peers, secondary.peers) override val payments: PaymentsDb = DualPaymentsDb(primary.payments, secondary.payments) - override val managedOffers: OffersDb = DualOffersDb(primary.managedOffers, secondary.managedOffers) + override val offers: OffersDb = DualOffersDb(primary.offers, secondary.offers) override val pendingCommands: PendingCommandsDb = DualPendingCommandsDb(primary.pendingCommands, secondary.pendingCommands) override val liquidity: LiquidityDb = DualLiquidityDb(primary.liquidity, secondary.liquidity) @@ -420,11 +420,6 @@ case class DualOffersDb(primary: OffersDb, secondary: OffersDb) extends OffersDb primary.disableOffer(offer) } - override def enableOffer(offer: OfferTypes.Offer): Unit = { - runAsync(secondary.enableOffer(offer)) - primary.enableOffer(offer) - } - override def listOffers(onlyActive: Boolean): Seq[OfferData] = { runAsync(secondary.listOffers(onlyActive)) primary.listOffers(onlyActive) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/db/OffersDb.scala b/eclair-core/src/main/scala/fr/acinq/eclair/db/OffersDb.scala index f7d1409f83..8316138b8d 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/db/OffersDb.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/db/OffersDb.scala @@ -36,11 +36,6 @@ trait OffersDb { */ def disableOffer(offer: Offer): Unit - /** - * Activate an offer that was previously disabled. - */ - def enableOffer(offer: Offer): Unit - /** * List offers managed by eclair. * @param onlyActive Whether to return only active offers or also disabled ones. diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/db/pg/PgOffersDb.scala b/eclair-core/src/main/scala/fr/acinq/eclair/db/pg/PgOffersDb.scala index 43cc5ff31a..d0bec01ae0 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/db/pg/PgOffersDb.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/db/pg/PgOffersDb.scala @@ -45,9 +45,9 @@ class PgOffersDb(implicit ds: DataSource, lock: PgLock) extends OffersDb with Lo using(pg.createStatement()) { statement => getVersion(statement, DB_NAME) match { case None => - statement.executeUpdate("CREATE SCHEMA offers") - statement.executeUpdate("CREATE TABLE offers.managed (offer_id TEXT NOT NULL PRIMARY KEY, offer TEXT NOT NULL, path_id TEXT, created_at TIMESTAMP WITH TIME ZONE NOT NULL, is_active BOOLEAN NOT NULL)") - statement.executeUpdate("CREATE INDEX offer_is_active_idx ON offers.managed(is_active)") + statement.executeUpdate("CREATE SCHEMA IF NOT EXISTS payments") + statement.executeUpdate("CREATE TABLE payments.offers (offer_id TEXT NOT NULL PRIMARY KEY, offer TEXT NOT NULL, path_id TEXT, created_at TIMESTAMP WITH TIME ZONE NOT NULL, disabled_at TIMESTAMP WITH TIME ZONE)") + statement.executeUpdate("CREATE INDEX offer_disabled_at_idx ON payments.offers(disabled_at)") case Some(CURRENT_VERSION) => () // table is up-to-date, nothing to do case Some(unknownVersion) => throw new RuntimeException(s"Unknown version of DB $DB_NAME found, version=$unknownVersion") } @@ -57,7 +57,7 @@ class PgOffersDb(implicit ds: DataSource, lock: PgLock) extends OffersDb with Lo override def addOffer(offer: OfferTypes.Offer, pathId_opt: Option[ByteVector32]): Unit = withMetrics("offers/add", DbBackends.Postgres){ withLock { pg => - using(pg.prepareStatement("INSERT INTO offers.managed (offer_id, offer, path_id, created_at, is_active) VALUES (?, ?, ?, ?, TRUE)")) { statement => + using(pg.prepareStatement("INSERT INTO payments.offers (offer_id, offer, path_id, created_at, disabled_at) VALUES (?, ?, ?, ?, NULL)")) { statement => statement.setString(1, offer.offerId.toHex) statement.setString(2, offer.toString) pathId_opt match { @@ -72,17 +72,9 @@ class PgOffersDb(implicit ds: DataSource, lock: PgLock) extends OffersDb with Lo override def disableOffer(offer: OfferTypes.Offer): Unit = withMetrics("offers/disable", DbBackends.Postgres){ withLock { pg => - using(pg.prepareStatement("UPDATE offers.managed SET is_active = FALSE WHERE offer_id = ?")) { statement => - statement.setString(1, offer.offerId.toHex) - statement.executeUpdate() - } - } - } - - override def enableOffer(offer: OfferTypes.Offer): Unit = withMetrics("offers/enable", DbBackends.Postgres){ - withLock { pg => - using(pg.prepareStatement("UPDATE offers.managed SET is_active = TRUE WHERE offer_id = ?")) { statement => - statement.setString(1, offer.offerId.toHex) + using(pg.prepareStatement("UPDATE payments.offers SET disabled_at = ? WHERE offer_id = ?")) { statement => + statement.setTimestamp(1, TimestampMilli.now().toSqlTimestamp) + statement.setString(2, offer.offerId.toHex) statement.executeUpdate() } } @@ -93,18 +85,18 @@ class PgOffersDb(implicit ds: DataSource, lock: PgLock) extends OffersDb with Lo Offer.decode(rs.getString("offer")).get, rs.getStringNullable("path_id").map(ByteVector32.fromValidHex), TimestampMilli.fromSqlTimestamp(rs.getTimestamp("created_at")), - rs.getBoolean("is_active") + { rs.getTimestamp("disabled_at"); rs.wasNull() } ) } override def listOffers(onlyActive: Boolean): Seq[OfferData] = withMetrics("offers/list", DbBackends.Postgres){ withLock { pg => if (onlyActive) { - using(pg.prepareStatement("SELECT * FROM offers.managed WHERE is_active = TRUE")) { statement => + using(pg.prepareStatement("SELECT * FROM payments.offers WHERE disabled_at IS NULL")) { statement => statement.executeQuery().map(parseOfferData).toSeq } } else { - using(pg.prepareStatement("SELECT * FROM offers.managed")) { statement => + using(pg.prepareStatement("SELECT * FROM payments.offers")) { statement => statement.executeQuery().map(parseOfferData).toSeq } } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/db/pg/PgPaymentsDb.scala b/eclair-core/src/main/scala/fr/acinq/eclair/db/pg/PgPaymentsDb.scala index d78883a296..27af66e55b 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/db/pg/PgPaymentsDb.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/db/pg/PgPaymentsDb.scala @@ -50,7 +50,7 @@ class PgPaymentsDb(implicit ds: DataSource, lock: PgLock) extends PaymentsDb wit using(pg.createStatement()) { statement => def migration45(statement: Statement): Unit = { - statement.executeUpdate("CREATE SCHEMA payments") + statement.executeUpdate("CREATE SCHEMA IF NOT EXISTS payments") statement.executeUpdate("ALTER TABLE received_payments RENAME TO received") statement.executeUpdate("ALTER TABLE received SET SCHEMA payments") statement.executeUpdate("ALTER TABLE sent_payments RENAME TO sent") @@ -79,7 +79,7 @@ class PgPaymentsDb(implicit ds: DataSource, lock: PgLock) extends PaymentsDb wit getVersion(statement, DB_NAME) match { case None => - statement.executeUpdate("CREATE SCHEMA payments") + statement.executeUpdate("CREATE SCHEMA IF NOT EXISTS payments") statement.executeUpdate("CREATE TABLE payments.received (payment_hash TEXT NOT NULL PRIMARY KEY, payment_type TEXT NOT NULL, payment_preimage TEXT NOT NULL, path_ids BYTEA, payment_request TEXT NOT NULL, received_msat BIGINT, created_at TIMESTAMP WITH TIME ZONE NOT NULL, expire_at TIMESTAMP WITH TIME ZONE NOT NULL, received_at TIMESTAMP WITH TIME ZONE)") statement.executeUpdate("CREATE TABLE payments.sent (id TEXT NOT NULL PRIMARY KEY, parent_id TEXT NOT NULL, external_id TEXT, payment_hash TEXT NOT NULL, payment_preimage TEXT, payment_type TEXT NOT NULL, amount_msat BIGINT NOT NULL, fees_msat BIGINT, recipient_amount_msat BIGINT NOT NULL, recipient_node_id TEXT NOT NULL, payment_request TEXT, offer_id TEXT, payer_key TEXT, payment_route BYTEA, failures BYTEA, created_at TIMESTAMP WITH TIME ZONE NOT NULL, completed_at TIMESTAMP WITH TIME ZONE)") diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteOffersDb.scala b/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteOffersDb.scala index e252881d60..d93db830ee 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteOffersDb.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteOffersDb.scala @@ -40,8 +40,8 @@ class SqliteOffersDb(val sqlite: Connection) extends OffersDb with Logging { using(sqlite.createStatement(), inTransaction = true) { statement => getVersion(statement, DB_NAME) match { case None => - statement.executeUpdate("CREATE TABLE managed_offers (offer_id BLOB NOT NULL PRIMARY KEY, offer TEXT NOT NULL, path_id BLOB, created_at INTEGER NOT NULL, is_active INTEGER NOT NULL)") - statement.executeUpdate("CREATE INDEX offer_is_active_idx ON managed_offers(is_active)") + statement.executeUpdate("CREATE TABLE offers (offer_id BLOB NOT NULL PRIMARY KEY, offer TEXT NOT NULL, path_id BLOB, created_at INTEGER NOT NULL, disabled_at INTEGER)") + statement.executeUpdate("CREATE INDEX offer_disabled_at_idx ON offers(disabled_at)") case Some(CURRENT_VERSION) => () // table is up-to-date, nothing to do case Some(unknownVersion) => throw new RuntimeException(s"Unknown version of DB $DB_NAME found, version=$unknownVersion") } @@ -50,7 +50,7 @@ class SqliteOffersDb(val sqlite: Connection) extends OffersDb with Logging { } override def addOffer(offer: OfferTypes.Offer, pathId_opt: Option[ByteVector32]): Unit = withMetrics("offers/add", DbBackends.Sqlite) { - using(sqlite.prepareStatement("INSERT INTO managed_offers (offer_id, offer, path_id, created_at, is_active) VALUES (?, ?, ?, ?, TRUE)")) { statement => + using(sqlite.prepareStatement("INSERT INTO offers (offer_id, offer, path_id, created_at, disabled_at) VALUES (?, ?, ?, ?, NULL)")) { statement => statement.setBytes(1, offer.offerId.toArray) statement.setString(2, offer.toString) pathId_opt match { @@ -63,15 +63,9 @@ class SqliteOffersDb(val sqlite: Connection) extends OffersDb with Logging { } override def disableOffer(offer: OfferTypes.Offer): Unit = withMetrics("offers/disable", DbBackends.Sqlite) { - using(sqlite.prepareStatement("UPDATE managed_offers SET is_active = FALSE WHERE offer_id = ?")) { statement => - statement.setBytes(1, offer.offerId.toArray) - statement.executeUpdate() - } - } - - override def enableOffer(offer: OfferTypes.Offer): Unit = withMetrics("offers/enable", DbBackends.Sqlite) { - using(sqlite.prepareStatement("UPDATE managed_offers SET is_active = TRUE WHERE offer_id = ?")) { statement => - statement.setBytes(1, offer.offerId.toArray) + using(sqlite.prepareStatement("UPDATE offers SET disabled_at = ? WHERE offer_id = ?")) { statement => + statement.setLong(1, TimestampMilli.now().toLong) + statement.setBytes(2, offer.offerId.toArray) statement.executeUpdate() } } @@ -81,17 +75,17 @@ class SqliteOffersDb(val sqlite: Connection) extends OffersDb with Logging { Offer.decode(rs.getString("offer")).get, rs.getByteVector32Nullable("path_id"), TimestampMilli(rs.getLong("created_at")), - rs.getBoolean("is_active") + { rs.getLong("disabled_at"); rs.wasNull() } ) } override def listOffers(onlyActive: Boolean): Seq[OfferData] = withMetrics("offers/list", DbBackends.Sqlite) { if (onlyActive) { - using(sqlite.prepareStatement("SELECT * FROM managed_offers WHERE is_active = TRUE")) { statement => + using(sqlite.prepareStatement("SELECT * FROM offers WHERE disabled_at IS NULL")) { statement => statement.executeQuery().map(parseOfferData).toSeq } } else { - using(sqlite.prepareStatement("SELECT * FROM managed_offers")) { statement => + using(sqlite.prepareStatement("SELECT * FROM offers")) { statement => statement.executeQuery().map(parseOfferData).toSeq } } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/DefaultHandler.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/DefaultOfferHandler.scala similarity index 99% rename from eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/DefaultHandler.scala rename to eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/DefaultOfferHandler.scala index d4bf390016..a38c830dbc 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/DefaultHandler.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/DefaultOfferHandler.scala @@ -27,7 +27,9 @@ import fr.acinq.eclair.router.Router.{BlindedRouteRequest, ChannelHop} import fr.acinq.eclair.wire.protocol.OfferTypes import fr.acinq.eclair.{CltvExpiryDelta, EncodedNodeId, MilliSatoshi, MilliSatoshiLong, NodeParams} -object DefaultHandler { +case class OffersConfig(messagePathMinLength: Int, paymentPathCount: Int, paymentPathLength: Int, paymentPathCltvExpiryDelta: CltvExpiryDelta) + +object DefaultOfferHandler { def apply(nodeParams: NodeParams, router: ActorRef): Behavior[OfferManager.HandlerCommand] = { Behaviors.setup(context => Behaviors.receiveMessage { @@ -75,5 +77,3 @@ object DefaultHandler { }) } } - -case class OffersConfig(messagePathMinLength: Int, paymentPathCount: Int, paymentPathLength: Int, paymentPathCltvExpiryDelta: CltvExpiryDelta) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/OfferCreator.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/OfferCreator.scala index cd98b31545..3abf87161a 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/OfferCreator.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/OfferCreator.scala @@ -23,6 +23,7 @@ import fr.acinq.bitcoin.scalacompat.Crypto.{PrivateKey, PublicKey} import fr.acinq.bitcoin.scalacompat.{Block, ByteVector32} import fr.acinq.eclair.message.OnionMessages import fr.acinq.eclair.message.OnionMessages.{IntermediateNode, Recipient} +import fr.acinq.eclair.payment.offer.OfferCreator.CreateOfferResult import fr.acinq.eclair.router.Router import fr.acinq.eclair.wire.protocol.OfferTypes._ import fr.acinq.eclair.wire.protocol.TlvStream @@ -31,24 +32,30 @@ import fr.acinq.eclair.{MilliSatoshi, NodeParams, TimestampSecond, randomBytes32 object OfferCreator { sealed trait Command - case class Create(replyTo: typed.ActorRef[Either[String, Offer]], + case class Create(replyTo: typed.ActorRef[CreateOfferResult], description_opt: Option[String], amount_opt: Option[MilliSatoshi], expiry_opt: Option[TimestampSecond], issuer_opt: Option[String], - firstNodeId_opt: Option[PublicKey]) extends Command + blindedPathsFirstNodeId_opt: Option[PublicKey]) extends Command - case class RouteResponseWrapper(response: Router.MessageRouteResponse) extends Command + private case class WrappedRouterResponse(response: Router.MessageRouteResponse) extends Command + + sealed trait CreateOfferResult + + case class CreatedOffer(offer: Offer) extends CreateOfferResult + + case class CreateOfferError(reason: String) extends CreateOfferResult def apply(nodeParams: NodeParams, router: ActorRef, offerManager: typed.ActorRef[OfferManager.Command], defaultOfferHandler: typed.ActorRef[OfferManager.HandlerCommand]): Behavior[Command] = Behaviors.receivePartial { - case (context, Create(replyTo, description_opt, amount_opt, expiry_opt, issuer_opt, firstNodeId_opt)) => - new OfferCreator(context, replyTo, nodeParams, router, offerManager, defaultOfferHandler).init(description_opt, amount_opt, expiry_opt, issuer_opt, firstNodeId_opt) + case (context, Create(replyTo, description_opt, amount_opt, expiry_opt, issuer_opt, blindedPathsFirstNodeId_opt)) => + new OfferCreator(context, replyTo, nodeParams, router, offerManager, defaultOfferHandler).init(description_opt, amount_opt, expiry_opt, issuer_opt, blindedPathsFirstNodeId_opt) } } private class OfferCreator(context: ActorContext[OfferCreator.Command], - replyTo: typed.ActorRef[Either[String, Offer]], + replyTo: typed.ActorRef[CreateOfferResult], nodeParams: NodeParams, router: ActorRef, offerManager: typed.ActorRef[OfferManager.Command], defaultOfferHandler: typed.ActorRef[OfferManager.HandlerCommand]) { @@ -59,9 +66,9 @@ private class OfferCreator(context: ActorContext[OfferCreator.Command], amount_opt: Option[MilliSatoshi], expiry_opt: Option[TimestampSecond], issuer_opt: Option[String], - firstNodeId_opt: Option[PublicKey]): Behavior[Command] = { + blindedPathsFirstNodeId_opt: Option[PublicKey]): Behavior[Command] = { if (amount_opt.nonEmpty && description_opt.isEmpty) { - replyTo ! Left("Description is mandatory for offers with set amount.") + replyTo ! CreateOfferError("Description is mandatory for offers with set amount.") Behaviors.stopped } else { val tlvs: Set[OfferTlv] = Set( @@ -71,9 +78,9 @@ private class OfferCreator(context: ActorContext[OfferCreator.Command], expiry_opt.map(OfferAbsoluteExpiry), issuer_opt.map(OfferIssuer), ).flatten - firstNodeId_opt match { + blindedPathsFirstNodeId_opt match { case Some(firstNodeId) => - router ! Router.MessageRouteRequest(context.messageAdapter(RouteResponseWrapper(_)), firstNodeId, nodeParams.nodeId, Set.empty) + router ! Router.MessageRouteRequest(context.messageAdapter(WrappedRouterResponse(_)), firstNodeId, nodeParams.nodeId, Set.empty) waitForRoute(firstNodeId, tlvs) case None => val offer = Offer(TlvStream(tlvs + OfferNodeId(nodeParams.nodeId))) @@ -84,22 +91,22 @@ private class OfferCreator(context: ActorContext[OfferCreator.Command], private def waitForRoute(firstNode: PublicKey, tlvs: Set[OfferTlv]): Behavior[Command] = { Behaviors.receiveMessagePartial { - case RouteResponseWrapper(Router.MessageRoute(intermediateNodes, _)) => + case WrappedRouterResponse(Router.MessageRoute(intermediateNodes, _)) => val pathId = randomBytes32() val nodes = firstNode +: (intermediateNodes ++ Seq.fill(nodeParams.offersConfig.messagePathMinLength - intermediateNodes.length - 1)(nodeParams.nodeId)) val paths = Seq(OnionMessages.buildRoute(randomKey(), nodes.map(IntermediateNode(_)), Recipient(nodeParams.nodeId, Some(pathId))).route) val offer = Offer(TlvStream(tlvs + OfferPaths(paths))) registerOffer(offer, None, Some(pathId)) - case RouteResponseWrapper(Router.MessageRouteNotFound(_)) => - replyTo ! Left("No route found") + case WrappedRouterResponse(Router.MessageRouteNotFound(_)) => + replyTo ! CreateOfferError("No route found") Behaviors.stopped } } private def registerOffer(offer: Offer, nodeKey: Option[PrivateKey], pathId_opt: Option[ByteVector32]): Behavior[Command] = { - nodeParams.db.managedOffers.addOffer(offer, pathId_opt) + nodeParams.db.offers.addOffer(offer, pathId_opt) offerManager ! OfferManager.RegisterOffer(offer, nodeKey, pathId_opt, defaultOfferHandler) - replyTo ! Right(offer) + replyTo ! CreatedOffer(offer) Behaviors.stopped } } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/OfferManager.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/OfferManager.scala index 314b225469..86e772cf36 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/OfferManager.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/OfferManager.scala @@ -90,7 +90,15 @@ object OfferManager { */ case class HandlePayment(replyTo: ActorRef[PaymentActor.Command], offer: Offer, invoiceData: MinimalInvoiceData) extends HandlerCommand - private case class RegisteredOffer(offer: Offer, nodeKey: Option[PrivateKey], pathId_opt: Option[ByteVector32], handler: ActorRef[HandlerCommand]) + /** + * Register an offer so that we can respond to invoice requests for it. + * + * @param offer The offer to register. + * @param nodeKey_opt If the offer has a node id, this must be the associated private key. + * @param pathId_opt If this offer has a blinded path, this must be its path id. + * @param handler Handler for invoice requests and payments for this offer. + */ + private case class RegisteredOffer(offer: Offer, nodeKey_opt: Option[PrivateKey], pathId_opt: Option[ByteVector32], handler: ActorRef[HandlerCommand]) def apply(nodeParams: NodeParams, paymentTimeout: FiniteDuration): Behavior[Command] = { Behaviors.setup { context => @@ -106,12 +114,13 @@ object OfferManager { case RegisterOffer(offer, nodeKey, pathId_opt, handler) => normal(registeredOffers + (offer.offerId -> RegisteredOffer(offer, nodeKey, pathId_opt, handler))) case DisableOffer(offer) => + nodeParams.db.offers.disableOffer(offer) normal(registeredOffers - offer.offerId) case RequestInvoice(messagePayload, blindedKey, postman) => registeredOffers.get(messagePayload.invoiceRequest.offer.offerId) match { case Some(registered) if registered.pathId_opt.map(_.bytes) == messagePayload.pathId_opt && messagePayload.invoiceRequest.isValid => context.log.debug("received valid invoice request for offerId={}", messagePayload.invoiceRequest.offer.offerId) - val child = context.spawnAnonymous(InvoiceRequestActor(nodeParams, messagePayload.invoiceRequest, registered.handler, registered.nodeKey.getOrElse(blindedKey), messagePayload.replyPath, postman)) + val child = context.spawnAnonymous(InvoiceRequestActor(nodeParams, messagePayload.invoiceRequest, registered.handler, registered.nodeKey_opt.getOrElse(blindedKey), messagePayload.replyPath, postman)) child ! InvoiceRequestActor.RequestInvoice case _ => context.log.debug("offer {} is not registered or invoice request is invalid", messagePayload.invoiceRequest.offer.offerId) } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/TestDatabases.scala b/eclair-core/src/test/scala/fr/acinq/eclair/TestDatabases.scala index a1a111d00f..dbb4775563 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/TestDatabases.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/TestDatabases.scala @@ -33,7 +33,7 @@ sealed trait TestDatabases extends Databases { override def channels: ChannelsDb = db.channels override def peers: PeersDb = db.peers override def payments: PaymentsDb = db.payments - override def managedOffers: OffersDb = db.managedOffers + override def offers: OffersDb = db.offers override def pendingCommands: PendingCommandsDb = db.pendingCommands override def liquidity: LiquidityDb = db.liquidity def close(): Unit diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/db/OffersDbSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/db/OffersDbSpec.scala index ca14979871..0050412345 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/db/OffersDbSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/db/OffersDbSpec.scala @@ -41,7 +41,7 @@ class OffersDbSpec extends AnyFunSuite { test("add/disable/enable/list offers") { forAllDbs { dbs => - val db = dbs.managedOffers + val db = dbs.offers assert(db.listOffers(onlyActive = false).isEmpty) val offer1 = Offer(None, Some("test 1"), randomKey().publicKey, Features(), Block.LivenetGenesisBlock.hash) @@ -63,8 +63,7 @@ class OffersDbSpec extends AnyFunSuite { assert(listed2.head.pathId_opt == Some(pathId)) assert(listed2.head.isActive) db.disableOffer(offer2) - db.enableOffer(offer1) - assert(db.listOffers(onlyActive = true) == listed1) + assert(db.listOffers(onlyActive = true).isEmpty) } } } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/fixtures/MinimalNodeFixture.scala b/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/fixtures/MinimalNodeFixture.scala index 3351257a51..6a5d676f9a 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/fixtures/MinimalNodeFixture.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/fixtures/MinimalNodeFixture.scala @@ -24,7 +24,7 @@ import fr.acinq.eclair.io.{Peer, PeerConnection, PendingChannelsRateLimiter, Swi import fr.acinq.eclair.message.Postman import fr.acinq.eclair.payment.Bolt11Invoice.ExtraHop import fr.acinq.eclair.payment._ -import fr.acinq.eclair.payment.offer.{DefaultHandler, OfferManager} +import fr.acinq.eclair.payment.offer.{DefaultOfferHandler, OfferManager} import fr.acinq.eclair.payment.receive.{MultiPartHandler, PaymentHandler} import fr.acinq.eclair.payment.relay.{ChannelRelayer, PostRestartHtlcCleaner, Relayer} import fr.acinq.eclair.payment.send.PaymentInitiator @@ -97,7 +97,7 @@ object MinimalNodeFixture extends Assertions with Eventually with IntegrationPat val register = system.actorOf(Register.props(), "register") val router = system.actorOf(Router.props(nodeParams, watcherTyped), "router") val offerManager = system.spawn(OfferManager(nodeParams, 1 minute), "offer-manager") - val defaultOfferHandler = system.spawn(DefaultHandler(nodeParams, router), "default-offer-handler") + val defaultOfferHandler = system.spawn(DefaultOfferHandler(nodeParams, router), "default-offer-handler") val paymentHandler = system.actorOf(PaymentHandler.props(nodeParams, register, offerManager), "payment-handler") val relayer = system.actorOf(Relayer.props(nodeParams, router, register, paymentHandler), "relayer") val txPublisherFactory = Channel.SimpleTxPublisherFactory(nodeParams, bitcoinClient) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/payment/OfferPaymentSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/payment/OfferPaymentSpec.scala index 0eead28973..1ab0261340 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/payment/OfferPaymentSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/payment/OfferPaymentSpec.scala @@ -759,7 +759,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { implicit val timeout: Timeout = 10 seconds val amount = 20_000_000 msat - val offer = Await.result(carol.eclairImpl.createOffer(description_opt = Some("test offer"), amount_opt = Some(amount), expiry_opt = None, issuer_opt = None, firstNodeId_opt = None), 10 seconds) + val offer = Await.result(carol.eclairImpl.createOffer(description_opt = Some("test offer"), amount_opt = Some(amount), expireInSeconds_opt = None, issuer_opt = None, blindedPathsFirstNodeId_opt = None), 10 seconds) assert(offer.nodeId == Some(carol.nodeId)) assert(offer.description == Some("test offer")) @@ -776,7 +776,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { implicit val timeout: Timeout = 10 seconds val amount = 20_000_000 msat - val offer = Await.result(carol.eclairImpl.createOffer(description_opt = Some("test offer"), amount_opt = Some(amount), expiry_opt = None, issuer_opt = None, firstNodeId_opt = Some(carol.nodeId)), 10 seconds) + val offer = Await.result(carol.eclairImpl.createOffer(description_opt = Some("test offer"), amount_opt = Some(amount), expireInSeconds_opt = None, issuer_opt = None, blindedPathsFirstNodeId_opt = Some(carol.nodeId)), 10 seconds) assert(offer.nodeId == None) assert(offer.contactInfos.head.asInstanceOf[BlindedPath].route.firstNodeId == EncodedNodeId.WithPublicKey.Plain(carol.nodeId)) @@ -794,7 +794,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { implicit val timeout: Timeout = 10 seconds val amount = 20_000_000 msat - val offer = Await.result(carol.eclairImpl.createOffer(description_opt = Some("test offer"), amount_opt = Some(amount), expiry_opt = None, issuer_opt = None, firstNodeId_opt = Some(bob.nodeId)), 10 seconds) + val offer = Await.result(carol.eclairImpl.createOffer(description_opt = Some("test offer"), amount_opt = Some(amount), expireInSeconds_opt = None, issuer_opt = None, blindedPathsFirstNodeId_opt = Some(bob.nodeId)), 10 seconds) assert(offer.nodeId == None) assert(offer.contactInfos.head.asInstanceOf[BlindedPath].route.firstNodeId == EncodedNodeId.WithPublicKey.Plain(bob.nodeId)) @@ -812,7 +812,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { implicit val timeout: Timeout = 10 seconds val amount = 20_000_000 msat - val offer = Await.result(carol.eclairImpl.createOffer(description_opt = Some("test offer"), amount_opt = Some(amount), expiry_opt = None, issuer_opt = None, firstNodeId_opt = Some(alice.nodeId)), 10 seconds) + val offer = Await.result(carol.eclairImpl.createOffer(description_opt = Some("test offer"), amount_opt = Some(amount), expireInSeconds_opt = None, issuer_opt = None, blindedPathsFirstNodeId_opt = Some(alice.nodeId)), 10 seconds) assert(offer.nodeId == None) assert(offer.contactInfos.head.asInstanceOf[BlindedPath].route.firstNodeId == EncodedNodeId.WithPublicKey.Plain(alice.nodeId)) diff --git a/eclair-node/src/main/scala/fr/acinq/eclair/api/Service.scala b/eclair-node/src/main/scala/fr/acinq/eclair/api/Service.scala index 6e96a9c092..60bfcfce2d 100644 --- a/eclair-node/src/main/scala/fr/acinq/eclair/api/Service.scala +++ b/eclair-node/src/main/scala/fr/acinq/eclair/api/Service.scala @@ -23,7 +23,7 @@ import fr.acinq.eclair.api.directives.EclairDirectives import fr.acinq.eclair.api.handlers._ import grizzled.slf4j.Logging -trait Service extends EclairDirectives with WebSocket with Node with Control with Channel with Fees with PathFinding with Invoice with Payment with Message with OnChain with Logging { +trait Service extends EclairDirectives with WebSocket with Node with Control with Channel with Fees with PathFinding with Invoice with Offer with Payment with Message with OnChain with Logging { /** * Allows router access to the API password as configured in eclair.conf @@ -46,7 +46,7 @@ trait Service extends EclairDirectives with WebSocket with Node with Control wit * This is where we handle errors to ensure all routes are correctly tried before rejecting. */ def finalRoutes(extraRouteProviders: Seq[RouteProvider] = Nil): Route = securedHandler { - val baseRoutes = nodeRoutes ~ controlRoutes ~ channelRoutes ~ feeRoutes ~ pathFindingRoutes ~ invoiceRoutes ~ paymentRoutes ~ messageRoutes ~ onChainRoutes ~ webSocket + val baseRoutes = nodeRoutes ~ controlRoutes ~ channelRoutes ~ feeRoutes ~ pathFindingRoutes ~ invoiceRoutes ~ offerRoutes ~ paymentRoutes ~ messageRoutes ~ onChainRoutes ~ webSocket extraRouteProviders.map(_.route(this)).foldLeft(baseRoutes)(_ ~ _) } } diff --git a/eclair-node/src/main/scala/fr/acinq/eclair/api/handlers/Offer.scala b/eclair-node/src/main/scala/fr/acinq/eclair/api/handlers/Offer.scala new file mode 100644 index 0000000000..506f9c3a16 --- /dev/null +++ b/eclair-node/src/main/scala/fr/acinq/eclair/api/handlers/Offer.scala @@ -0,0 +1,50 @@ +/* + * Copyright 2025 ACINQ SAS + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package fr.acinq.eclair.api.handlers + +import akka.http.scaladsl.server.Route +import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey +import fr.acinq.eclair.api.Service +import fr.acinq.eclair.api.directives.EclairDirectives +import fr.acinq.eclair.api.serde.FormParamExtractors._ + +trait Offer { + this: Service with EclairDirectives => + + import fr.acinq.eclair.api.serde.JsonSupport.{formats, marshaller, serialization} + + val createOffer: Route = postRequest("createoffer") { implicit t => + formFields("description".?, amountMsatFormParam.?, "expireInSeconds".as[Long].?, "issuer".?, "blindedPathsFirstNodeId".as[PublicKey].?) { + (description_opt, amount_opt, expireInSeconds_opt, issuer_opt, blindedPathsFirstNodeId_opt) => complete(eclairApi.createOffer(description_opt, amount_opt, expireInSeconds_opt, issuer_opt, blindedPathsFirstNodeId_opt)) + } + } + + val disableOffer: Route = postRequest("disableoffer") { implicit t => + formFields(offerFormParam) { offer => + complete(eclairApi.disableOffer(offer)) + } + } + + val listoffers: Route = postRequest("listoffers") { implicit t => + formFields("activeOnly".as[Boolean].?) { onlyActive => + complete(eclairApi.listOffers(onlyActive.getOrElse(false))) + } + } + + val offerRoutes: Route = createOffer ~ disableOffer ~ listoffers + +} From 7491779749c2c5a8a7b578484fdf573bee49418d Mon Sep 17 00:00:00 2001 From: Thomas HUET Date: Fri, 7 Mar 2025 11:57:34 +0100 Subject: [PATCH 3/7] more comments --- .../main/scala/fr/acinq/eclair/Setup.scala | 2 +- .../fr/acinq/eclair/db/DualDatabases.scala | 12 ++++---- .../scala/fr/acinq/eclair/db/OffersDb.scala | 6 ++-- .../fr/acinq/eclair/db/pg/PgOffersDb.scala | 30 +++++++++---------- .../eclair/db/sqlite/SqliteOffersDb.scala | 30 +++++++++---------- .../payment/offer/DefaultOfferHandler.scala | 7 +++-- .../eclair/payment/offer/OfferManager.scala | 5 +++- .../fr/acinq/eclair/db/OffersDbSpec.scala | 4 +-- .../fr/acinq/eclair/api/handlers/Offer.scala | 2 +- 9 files changed, 51 insertions(+), 47 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/Setup.scala b/eclair-core/src/main/scala/fr/acinq/eclair/Setup.scala index 673e87789f..1caeab4e99 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/Setup.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/Setup.scala @@ -359,7 +359,7 @@ class Setup(val datadir: File, register = system.actorOf(SimpleSupervisor.props(Register.props(), "register", SupervisorStrategy.Resume)) offerManager = system.spawn(Behaviors.supervise(OfferManager(nodeParams, paymentTimeout = 1 minute)).onFailure(typed.SupervisorStrategy.resume), name = "offer-manager") defaultOfferHandler = system.spawn(Behaviors.supervise(DefaultOfferHandler(nodeParams, router)).onFailure(typed.SupervisorStrategy.resume), name = "default-offer-handler") - _ = for (offer <- nodeParams.db.offers.listOffers(onlyActive = true)) offerManager ! OfferManager.RegisterOffer(offer.offer, None, offer.pathId_opt, defaultOfferHandler) + _ = for (offer <- nodeParams.db.offers.listOffers(onlyActive = true)) offerManager ! OfferManager.RegisterOffer(offer.offer, if (offer.pathId_opt.isEmpty) Some(nodeParams.privateKey) else None, offer.pathId_opt, defaultOfferHandler) paymentHandler = system.actorOf(SimpleSupervisor.props(PaymentHandler.props(nodeParams, register, offerManager), "payment-handler", SupervisorStrategy.Resume)) triggerer = system.spawn(Behaviors.supervise(AsyncPaymentTriggerer()).onFailure(typed.SupervisorStrategy.resume), name = "async-payment-triggerer") peerReadyManager = system.spawn(Behaviors.supervise(PeerReadyManager()).onFailure(typed.SupervisorStrategy.restart), name = "peer-ready-manager") diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/db/DualDatabases.scala b/eclair-core/src/main/scala/fr/acinq/eclair/db/DualDatabases.scala index 84c26d6ef6..ee7ce61966 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/db/DualDatabases.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/db/DualDatabases.scala @@ -410,14 +410,14 @@ case class DualOffersDb(primary: OffersDb, secondary: OffersDb) extends OffersDb private implicit val ec: ExecutionContext = ExecutionContext.fromExecutor(Executors.newSingleThreadExecutor(new ThreadFactoryBuilder().setNameFormat("db-offers").build())) - override def addOffer(offer: OfferTypes.Offer, pathId_opt: Option[ByteVector32]): Unit = { - runAsync(secondary.addOffer(offer, pathId_opt)) - primary.addOffer(offer, pathId_opt) + override def addOffer(offer: OfferTypes.Offer, pathId_opt: Option[ByteVector32], createdAt: TimestampMilli = TimestampMilli.now()): Unit = { + runAsync(secondary.addOffer(offer, pathId_opt, createdAt)) + primary.addOffer(offer, pathId_opt, createdAt) } - override def disableOffer(offer: OfferTypes.Offer): Unit = { - runAsync(secondary.disableOffer(offer)) - primary.disableOffer(offer) + override def disableOffer(offer: OfferTypes.Offer, disabledAt: TimestampMilli = TimestampMilli.now()): Unit = { + runAsync(secondary.disableOffer(offer, disabledAt)) + primary.disableOffer(offer, disabledAt) } override def listOffers(onlyActive: Boolean): Seq[OfferData] = { diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/db/OffersDb.scala b/eclair-core/src/main/scala/fr/acinq/eclair/db/OffersDb.scala index 8316138b8d..ced1553162 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/db/OffersDb.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/db/OffersDb.scala @@ -28,13 +28,13 @@ trait OffersDb { * Add an offer managed by eclair. * @param pathId_opt If the offer uses a blinded path, this is the corresponding pathId. */ - def addOffer(offer: Offer, pathId_opt: Option[ByteVector32]): Unit + def addOffer(offer: Offer, pathId_opt: Option[ByteVector32], createdAt: TimestampMilli = TimestampMilli.now()): Unit /** * Disable an offer. The offer is still stored but new invoice requests and new payment attempts for already emitted * invoices will be rejected. */ - def disableOffer(offer: Offer): Unit + def disableOffer(offer: Offer, disabledAt: TimestampMilli = TimestampMilli.now()): Unit /** * List offers managed by eclair. @@ -43,4 +43,4 @@ trait OffersDb { def listOffers(onlyActive: Boolean): Seq[OfferData] } -case class OfferData(offer: Offer, pathId_opt: Option[ByteVector32], createdAt: TimestampMilli, isActive: Boolean) \ No newline at end of file +case class OfferData(offer: Offer, pathId_opt: Option[ByteVector32], createdAt: TimestampMilli, disabledAt: Option[TimestampMilli]) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/db/pg/PgOffersDb.scala b/eclair-core/src/main/scala/fr/acinq/eclair/db/pg/PgOffersDb.scala index d0bec01ae0..b5c4ab213c 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/db/pg/PgOffersDb.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/db/pg/PgOffersDb.scala @@ -46,8 +46,9 @@ class PgOffersDb(implicit ds: DataSource, lock: PgLock) extends OffersDb with Lo getVersion(statement, DB_NAME) match { case None => statement.executeUpdate("CREATE SCHEMA IF NOT EXISTS payments") - statement.executeUpdate("CREATE TABLE payments.offers (offer_id TEXT NOT NULL PRIMARY KEY, offer TEXT NOT NULL, path_id TEXT, created_at TIMESTAMP WITH TIME ZONE NOT NULL, disabled_at TIMESTAMP WITH TIME ZONE)") - statement.executeUpdate("CREATE INDEX offer_disabled_at_idx ON payments.offers(disabled_at)") + statement.executeUpdate("CREATE TABLE payments.offers (offer_id TEXT NOT NULL PRIMARY KEY, offer TEXT NOT NULL, path_id TEXT, created_at TIMESTAMP WITH TIME ZONE NOT NULL, is_active BOOLEAN NOT NULL, disabled_at TIMESTAMP WITH TIME ZONE)") + statement.executeUpdate("CREATE INDEX offer_created_at_idx ON payments.offers(created_at)") + statement.executeUpdate("CREATE INDEX offer_is_active_idx ON payments.offers(is_active)") case Some(CURRENT_VERSION) => () // table is up-to-date, nothing to do case Some(unknownVersion) => throw new RuntimeException(s"Unknown version of DB $DB_NAME found, version=$unknownVersion") } @@ -55,25 +56,22 @@ class PgOffersDb(implicit ds: DataSource, lock: PgLock) extends OffersDb with Lo } } - override def addOffer(offer: OfferTypes.Offer, pathId_opt: Option[ByteVector32]): Unit = withMetrics("offers/add", DbBackends.Postgres){ + override def addOffer(offer: OfferTypes.Offer, pathId_opt: Option[ByteVector32], createdAt: TimestampMilli = TimestampMilli.now()): Unit = withMetrics("offers/add", DbBackends.Postgres){ withLock { pg => - using(pg.prepareStatement("INSERT INTO payments.offers (offer_id, offer, path_id, created_at, disabled_at) VALUES (?, ?, ?, ?, NULL)")) { statement => + using(pg.prepareStatement("INSERT INTO payments.offers (offer_id, offer, path_id, created_at, is_active, disabled_at) VALUES (?, ?, ?, ?, TRUE, NULL)")) { statement => statement.setString(1, offer.offerId.toHex) statement.setString(2, offer.toString) - pathId_opt match { - case Some(pathId) => statement.setString(3, pathId.toHex) - case None => statement.setNull(3, java.sql.Types.VARCHAR) - } - statement.setTimestamp(4, TimestampMilli.now().toSqlTimestamp) + statement.setString(3, pathId_opt.map(_.toHex).orNull) + statement.setTimestamp(4, createdAt.toSqlTimestamp) statement.executeUpdate() } } } - override def disableOffer(offer: OfferTypes.Offer): Unit = withMetrics("offers/disable", DbBackends.Postgres){ + override def disableOffer(offer: OfferTypes.Offer, disabledAt: TimestampMilli = TimestampMilli.now()): Unit = withMetrics("offers/disable", DbBackends.Postgres){ withLock { pg => - using(pg.prepareStatement("UPDATE payments.offers SET disabled_at = ? WHERE offer_id = ?")) { statement => - statement.setTimestamp(1, TimestampMilli.now().toSqlTimestamp) + using(pg.prepareStatement("UPDATE payments.offers SET disabled_at = ?, is_active = FALSE WHERE offer_id = ?")) { statement => + statement.setTimestamp(1, disabledAt.toSqlTimestamp) statement.setString(2, offer.offerId.toHex) statement.executeUpdate() } @@ -81,22 +79,24 @@ class PgOffersDb(implicit ds: DataSource, lock: PgLock) extends OffersDb with Lo } private def parseOfferData(rs: ResultSet): OfferData = { + val disabledAt = rs.getTimestamp("disabled_at") + val disabledAt_opt = if (rs.wasNull()) None else Some(TimestampMilli.fromSqlTimestamp(disabledAt)) OfferData( Offer.decode(rs.getString("offer")).get, rs.getStringNullable("path_id").map(ByteVector32.fromValidHex), TimestampMilli.fromSqlTimestamp(rs.getTimestamp("created_at")), - { rs.getTimestamp("disabled_at"); rs.wasNull() } + disabledAt_opt ) } override def listOffers(onlyActive: Boolean): Seq[OfferData] = withMetrics("offers/list", DbBackends.Postgres){ withLock { pg => if (onlyActive) { - using(pg.prepareStatement("SELECT * FROM payments.offers WHERE disabled_at IS NULL")) { statement => + using(pg.prepareStatement("SELECT * FROM payments.offers WHERE is_active = TRUE ORDER BY created_at DESC")) { statement => statement.executeQuery().map(parseOfferData).toSeq } } else { - using(pg.prepareStatement("SELECT * FROM payments.offers")) { statement => + using(pg.prepareStatement("SELECT * FROM payments.offers ORDER BY created_at DESC")) { statement => statement.executeQuery().map(parseOfferData).toSeq } } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteOffersDb.scala b/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteOffersDb.scala index d93db830ee..6a5c9c3b5d 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteOffersDb.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteOffersDb.scala @@ -40,8 +40,9 @@ class SqliteOffersDb(val sqlite: Connection) extends OffersDb with Logging { using(sqlite.createStatement(), inTransaction = true) { statement => getVersion(statement, DB_NAME) match { case None => - statement.executeUpdate("CREATE TABLE offers (offer_id BLOB NOT NULL PRIMARY KEY, offer TEXT NOT NULL, path_id BLOB, created_at INTEGER NOT NULL, disabled_at INTEGER)") - statement.executeUpdate("CREATE INDEX offer_disabled_at_idx ON offers(disabled_at)") + statement.executeUpdate("CREATE TABLE offers (offer_id BLOB NOT NULL PRIMARY KEY, offer TEXT NOT NULL, path_id BLOB, created_at INTEGER NOT NULL, is_active INTEGER NOT NULL, disabled_at INTEGER)") + statement.executeUpdate("CREATE INDEX offer_created_at_idx ON offers(created_at)") + statement.executeUpdate("CREATE INDEX offer_is_active_idx ON offers(is_active)") case Some(CURRENT_VERSION) => () // table is up-to-date, nothing to do case Some(unknownVersion) => throw new RuntimeException(s"Unknown version of DB $DB_NAME found, version=$unknownVersion") } @@ -49,43 +50,42 @@ class SqliteOffersDb(val sqlite: Connection) extends OffersDb with Logging { } - override def addOffer(offer: OfferTypes.Offer, pathId_opt: Option[ByteVector32]): Unit = withMetrics("offers/add", DbBackends.Sqlite) { - using(sqlite.prepareStatement("INSERT INTO offers (offer_id, offer, path_id, created_at, disabled_at) VALUES (?, ?, ?, ?, NULL)")) { statement => + override def addOffer(offer: OfferTypes.Offer, pathId_opt: Option[ByteVector32], createdAt: TimestampMilli = TimestampMilli.now()): Unit = withMetrics("offers/add", DbBackends.Sqlite) { + using(sqlite.prepareStatement("INSERT INTO offers (offer_id, offer, path_id, created_at, is_active, disabled_at) VALUES (?, ?, ?, ?, TRUE, NULL)")) { statement => statement.setBytes(1, offer.offerId.toArray) statement.setString(2, offer.toString) - pathId_opt match { - case Some(pathId) => statement.setBytes(3, pathId.toArray) - case None => statement.setNull(3, java.sql.Types.VARBINARY) - } - statement.setLong(4, TimestampMilli.now().toLong) + statement.setBytes(3, pathId_opt.map(_.toArray).orNull) + statement.setLong(4, createdAt.toLong) statement.executeUpdate() } } - override def disableOffer(offer: OfferTypes.Offer): Unit = withMetrics("offers/disable", DbBackends.Sqlite) { - using(sqlite.prepareStatement("UPDATE offers SET disabled_at = ? WHERE offer_id = ?")) { statement => - statement.setLong(1, TimestampMilli.now().toLong) + override def disableOffer(offer: OfferTypes.Offer, disabledAt: TimestampMilli = TimestampMilli.now()): Unit = withMetrics("offers/disable", DbBackends.Sqlite) { + using(sqlite.prepareStatement("UPDATE offers SET disabled_at = ?, is_active = FALSE WHERE offer_id = ?")) { statement => + statement.setLong(1, disabledAt.toLong) statement.setBytes(2, offer.offerId.toArray) statement.executeUpdate() } } private def parseOfferData(rs: ResultSet): OfferData = { + val disabledAt = rs.getLong("disabled_at") + val disabledAt_opt = if (rs.wasNull()) None else Some(TimestampMilli(disabledAt)) OfferData( Offer.decode(rs.getString("offer")).get, rs.getByteVector32Nullable("path_id"), TimestampMilli(rs.getLong("created_at")), - { rs.getLong("disabled_at"); rs.wasNull() } + disabledAt_opt ) } override def listOffers(onlyActive: Boolean): Seq[OfferData] = withMetrics("offers/list", DbBackends.Sqlite) { if (onlyActive) { - using(sqlite.prepareStatement("SELECT * FROM offers WHERE disabled_at IS NULL")) { statement => + using(sqlite.prepareStatement("SELECT * FROM offers WHERE is_active = TRUE ORDER BY created_at DESC")) { statement => statement.executeQuery().map(parseOfferData).toSeq } } else { - using(sqlite.prepareStatement("SELECT * FROM offers")) { statement => + using(sqlite.prepareStatement("SELECT * FROM offers ORDER BY created_at DESC")) { statement => statement.executeQuery().map(parseOfferData).toSeq } } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/DefaultOfferHandler.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/DefaultOfferHandler.scala index a38c830dbc..f3db553100 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/DefaultOfferHandler.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/DefaultOfferHandler.scala @@ -44,7 +44,7 @@ object DefaultOfferHandler { case OfferTypes.BlindedPath(BlindedRoute(firstNodeId: EncodedNodeId.WithPublicKey, _, _)) => val baseParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams val routeParams = baseParams.copy(boundaries = baseParams.boundaries.copy(maxRouteLength = nodeParams.offersConfig.paymentPathLength, maxCltv = nodeParams.offersConfig.paymentPathCltvExpiryDelta)) - router ! BlindedRouteRequest(context.spawnAnonymous(waitForRoute(nodeParams, replyTo, invoiceRequest.offer, amount)), firstNodeId.publicKey, nodeParams.nodeId, amount, routeParams, pathsToFind = 2) + router ! BlindedRouteRequest(context.spawnAnonymous(waitForRoute(nodeParams, replyTo, invoiceRequest.offer, amount)), firstNodeId.publicKey, nodeParams.nodeId, amount, routeParams, nodeParams.offersConfig.paymentPathCount) case OfferTypes.BlindedPath(BlindedRoute(_: EncodedNodeId.ShortChannelIdDir, _, _)) => context.log.error("unexpected managed offer with compact first node id") replyTo ! InvoiceRequestActor.RejectRequest("internal error") @@ -63,13 +63,14 @@ object DefaultOfferHandler { replyTo ! InvoiceRequestActor.ApproveRequest(amount, makeRoutes(nodeParams, routes.map(_.hops))) Behaviors.stopped case (context, Router.PaymentRouteNotFound(error)) => - context.log.error("Couldn't find blinded route for creating invoice offer={} amount={} : {}", offer, amount, error.getMessage) + context.log.error("couldn't find blinded route for creating invoice offer={} amount={} : {}", offer, amount, error.getMessage) replyTo ! InvoiceRequestActor.RejectRequest("internal error") Behaviors.stopped } } - def makeRoutes(nodeParams: NodeParams, routes: Seq[Seq[Router.ChannelHop]]): Seq[InvoiceRequestActor.Route] = { + // Ensure that we don't leak routing information by always returning the same number of routes, with the same number of hops and the same fees and CLTV delta. + private def makeRoutes(nodeParams: NodeParams, routes: Seq[Seq[Router.ChannelHop]]): Seq[InvoiceRequestActor.Route] = { (0 until nodeParams.offersConfig.paymentPathCount).map(i => { val hops = routes(i % routes.length) val dummyHops = Seq.fill(nodeParams.offersConfig.paymentPathLength - hops.length)(ChannelHop.dummy(nodeParams.nodeId, 0 msat, 0, CltvExpiryDelta(0))) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/OfferManager.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/OfferManager.scala index 86e772cf36..c32e839097 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/OfferManager.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/OfferManager.scala @@ -168,7 +168,10 @@ object OfferManager { customTlvs: Set[GenericTlv] = Set.empty) extends Command /** - * @param recipientPaysFees If true, fees for the blinded route will be hidden to the payer and paid by the recipient. + * @param feeOverride fees that will be published for this route, the difference between these and the + * actual fees of the route will be paid by the recipient. + * @param cltvOverride Cltv expiry delta to publish for the route. + * @param shortChannelIdDir_opt short channel id and direction to use for the first node instead of its node id. */ case class Route(hops: Seq[Router.ChannelHop], maxFinalExpiryDelta: CltvExpiryDelta, feeOverride: Option[RelayFees] = None, cltvOverride: Option[CltvExpiryDelta] = None, shortChannelIdDir_opt: Option[ShortChannelIdDir] = None) { def finalize(nodePriv: PrivateKey, preimage: ByteVector32, amount: MilliSatoshi, invoiceRequest: InvoiceRequest, minFinalExpiryDelta: CltvExpiryDelta, pluginData_opt: Option[ByteVector]): ReceivingRoute = { diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/db/OffersDbSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/db/OffersDbSpec.scala index 0050412345..4e5e4b53c3 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/db/OffersDbSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/db/OffersDbSpec.scala @@ -50,7 +50,7 @@ class OffersDbSpec extends AnyFunSuite { assert(listed1.length == 1) assert(listed1.head.offer == offer1) assert(listed1.head.pathId_opt == None) - assert(listed1.head.isActive) + assert(listed1.head.disabledAt == None) val offer2 = Offer(None, Some("test 2"), randomKey().publicKey, Features(), Block.LivenetGenesisBlock.hash) val pathId = randomBytes32() db.addOffer(offer2, Some(pathId)) @@ -61,7 +61,7 @@ class OffersDbSpec extends AnyFunSuite { assert(listed2.length == 1) assert(listed2.head.offer == offer2) assert(listed2.head.pathId_opt == Some(pathId)) - assert(listed2.head.isActive) + assert(listed2.head.disabledAt == None) db.disableOffer(offer2) assert(db.listOffers(onlyActive = true).isEmpty) } diff --git a/eclair-node/src/main/scala/fr/acinq/eclair/api/handlers/Offer.scala b/eclair-node/src/main/scala/fr/acinq/eclair/api/handlers/Offer.scala index 506f9c3a16..96949c9d34 100644 --- a/eclair-node/src/main/scala/fr/acinq/eclair/api/handlers/Offer.scala +++ b/eclair-node/src/main/scala/fr/acinq/eclair/api/handlers/Offer.scala @@ -41,7 +41,7 @@ trait Offer { val listoffers: Route = postRequest("listoffers") { implicit t => formFields("activeOnly".as[Boolean].?) { onlyActive => - complete(eclairApi.listOffers(onlyActive.getOrElse(false))) + complete(eclairApi.listOffers(onlyActive.getOrElse(true))) } } From e378122fcf70d94371d5ab237274123e55112189 Mon Sep 17 00:00:00 2001 From: Thomas HUET Date: Fri, 7 Mar 2025 14:11:22 +0100 Subject: [PATCH 4/7] invoice requests must have an amount --- .../fr/acinq/eclair/payment/Bolt12Invoice.scala | 5 ++--- .../eclair/payment/offer/DefaultOfferHandler.scala | 2 +- .../eclair/payment/receive/MultiPartHandler.scala | 4 +--- .../fr/acinq/eclair/wire/protocol/OfferTypes.scala | 13 +++++-------- 4 files changed, 9 insertions(+), 15 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/Bolt12Invoice.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/Bolt12Invoice.scala index 538de2f626..82ad8d9c87 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/Bolt12Invoice.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/Bolt12Invoice.scala @@ -60,7 +60,7 @@ case class Bolt12Invoice(records: TlvStream[InvoiceTlv]) extends Invoice { Left("Wrong node id") } else if (isExpired()) { Left("Invoice expired") - } else if (!request.amount.forall(_ == amount)) { + } else if (request.amount != amount) { Left("Incompatible amount") } else if (!Features.areCompatible(request.features, features.bolt12Features())) { Left("Incompatible features") @@ -106,8 +106,7 @@ object Bolt12Invoice { paths: Seq[PaymentBlindedRoute], additionalTlvs: Set[InvoiceTlv] = Set.empty, customTlvs: Set[GenericTlv] = Set.empty): Bolt12Invoice = { - require(request.amount.nonEmpty || request.offer.amount.nonEmpty) - val amount = request.amount.orElse(request.offer.amount.map(_ * request.quantity)).get + val amount = request.amount val tlvs: Set[InvoiceTlv] = removeSignature(request.records).records ++ Set( Some(InvoicePaths(paths.map(_.route))), Some(InvoiceBlindedPay(paths.map(_.paymentInfo))), diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/DefaultOfferHandler.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/DefaultOfferHandler.scala index f3db553100..dce0c17c41 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/DefaultOfferHandler.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/DefaultOfferHandler.scala @@ -34,7 +34,7 @@ object DefaultOfferHandler { Behaviors.setup(context => Behaviors.receiveMessage { case OfferManager.HandleInvoiceRequest(replyTo, invoiceRequest) => - val amount = invoiceRequest.amount.getOrElse(10_000_000 msat) + val amount = invoiceRequest.amount invoiceRequest.offer.contactInfos.head match { case OfferTypes.RecipientNodeId(_) => val route = InvoiceRequestActor.Route(Nil, nodeParams.channelConf.maxExpiryDelta) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/receive/MultiPartHandler.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/receive/MultiPartHandler.scala index 3d7722037b..c2dc342bba 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/receive/MultiPartHandler.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/receive/MultiPartHandler.scala @@ -292,9 +292,7 @@ object MultiPartHandler { paymentPreimage: ByteVector32, additionalTlvs: Set[InvoiceTlv] = Set.empty, customTlvs: Set[GenericTlv] = Set.empty) extends ReceivePayment { - require(invoiceRequest.offer.amount.nonEmpty || invoiceRequest.amount.nonEmpty, "an amount must be specified in the offer or in the invoice request") - - val amount = invoiceRequest.amount.orElse(invoiceRequest.offer.amount.map(_ * invoiceRequest.quantity)).get + val amount = invoiceRequest.amount } object CreateInvoiceActor { diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/OfferTypes.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/OfferTypes.scala index 0428cadab2..519ae90e1b 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/OfferTypes.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/OfferTypes.scala @@ -347,22 +347,18 @@ object OfferTypes { val metadata: ByteVector = records.get[InvoiceRequestMetadata].get.data val chain: BlockHash = records.get[InvoiceRequestChain].map(_.hash).getOrElse(Block.LivenetGenesisBlock.hash) - val amount: Option[MilliSatoshi] = records.get[InvoiceRequestAmount].map(_.amount) + private val amount_opt: Option[MilliSatoshi] = records.get[InvoiceRequestAmount].map(_.amount) val features: Features[Bolt12Feature] = records.get[InvoiceRequestFeatures].map(_.features.bolt12Features()).getOrElse(Features.empty) val quantity_opt: Option[Long] = records.get[InvoiceRequestQuantity].map(_.quantity) val quantity: Long = quantity_opt.getOrElse(1) + private val baseInvoiceAmount_opt = offer.amount.map(_ * quantity) + val amount: MilliSatoshi = amount_opt.orElse(baseInvoiceAmount_opt).get val payerId: PublicKey = records.get[InvoiceRequestPayerId].get.publicKey val payerNote: Option[String] = records.get[InvoiceRequestPayerNote].map(_.note) private val signature: ByteVector64 = records.get[Signature].get.signature def isValid: Boolean = { - val amountOk = offer.amount match { - case Some(offerAmount) => - val baseInvoiceAmount = offerAmount * quantity - amount.forall(baseInvoiceAmount <= _) - case None => amount.nonEmpty - } - amountOk && + amount_opt.forall(a => baseInvoiceAmount_opt.forall(b => a >= b)) && offer.chains.contains(chain) && offer.quantityMax.forall(max => quantity_opt.nonEmpty && quantity <= max) && quantity_opt.forall(_ => offer.quantityMax.nonEmpty) && @@ -426,6 +422,7 @@ object OfferTypes { _ -> () ) if (records.get[InvoiceRequestMetadata].isEmpty) return Left(MissingRequiredTlv(UInt64(0))) + if (records.get[InvoiceRequestAmount].isEmpty && records.get[OfferAmount].isEmpty) return Left(MissingRequiredTlv(UInt64(82))) if (records.get[InvoiceRequestPayerId].isEmpty) return Left(MissingRequiredTlv(UInt64(88))) if (records.get[Signature].isEmpty) return Left(MissingRequiredTlv(UInt64(240))) if (records.unknown.exists(!isInvoiceRequestTlv(_))) return Left(ForbiddenTlv(records.unknown.find(!isInvoiceRequestTlv(_)).get.tag)) From e6ed8fd71cd2f0b3cc0a600cece0a105b34057e5 Mon Sep 17 00:00:00 2001 From: Thomas HUET Date: Fri, 7 Mar 2025 14:30:39 +0100 Subject: [PATCH 5/7] fix tests --- .../scala/fr/acinq/eclair/payment/Bolt12InvoiceSpec.scala | 1 + .../fr/acinq/eclair/wire/protocol/OfferTypesSpec.scala | 6 +++--- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/Bolt12InvoiceSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/Bolt12InvoiceSpec.scala index 10410d4d3a..15282f14f2 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/Bolt12InvoiceSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/Bolt12InvoiceSpec.scala @@ -185,6 +185,7 @@ class Bolt12InvoiceSpec extends AnyFunSuite { val tlvs = Set[InvoiceTlv]( InvoiceRequestMetadata(hex"012345"), OfferNodeId(nodeKey.publicKey), + InvoiceRequestAmount(1684 msat), InvoiceRequestPayerId(randomKey().publicKey), InvoicePaths(Seq(createPaymentBlindedRoute(randomKey().publicKey).route)), InvoiceBlindedPay(Seq(PaymentInfo(0 msat, 0, CltvExpiryDelta(0), 0 msat, 765432 msat, Features.empty))), diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/wire/protocol/OfferTypesSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/wire/protocol/OfferTypesSpec.scala index 988a22c391..72de49bb7e 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/wire/protocol/OfferTypesSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/wire/protocol/OfferTypesSpec.scala @@ -122,8 +122,7 @@ class OfferTypesSpec extends AnyFunSuite { val request = InvoiceRequest(offer, 500 msat, 1, Features.empty, payerKey, Block.LivenetGenesisBlock.hash) assert(request.isValid) assert(request.offer == offer) - val withoutAmount = signInvoiceRequest(request.copy(records = TlvStream(request.records.records.filter { case InvoiceRequestAmount(_) => false case _ => true })), payerKey) - assert(!withoutAmount.isValid) + assertThrows[Exception](signInvoiceRequest(request.copy(records = TlvStream(request.records.records.filter { case InvoiceRequestAmount(_) => false case _ => true })), payerKey)) } test("check that invoice request matches offer (chain compatibility)") { @@ -188,11 +187,12 @@ class OfferTypesSpec extends AnyFunSuite { InvoiceRequestMetadata(hex"abcdef"), OfferNodeId(nodeId), InvoiceRequestPayerId(payerKey.publicKey), + InvoiceRequestAmount(21000 msat) ) val signature = signSchnorr(InvoiceRequest.signatureTag, rootHash(TlvStream[InvoiceRequestTlv](tlvsWithoutSignature), OfferCodecs.invoiceRequestTlvCodec), payerKey) val tlvs = tlvsWithoutSignature + Signature(signature) val invoiceRequest = InvoiceRequest(TlvStream(tlvs)) - val encoded = "lnr1qqp6hn00zcssxr0juddeytv7nwawhk9nq9us0arnk8j8wnsq8r2e86vzgtfneupetqssynwewhp70gwlp4chhm53g90jt9fpnx7rpmrzla3zd0nvxymm8e0p7pq06rwacy8756zgl3hdnsyfepq573astyz94rgn9uhxlyqj4gdyk6q8q0yrv6al909v3435amuvjqvkuq6k8fyld78r8srdyx7wnmwsdu" + val encoded = "lnr1qqp6hn00zcssxr0juddeytv7nwawhk9nq9us0arnk8j8wnsq8r2e86vzgtfneupe2gp9yzzcyypymkt4c0n6rhcdw9a7ay2ptuje2gvehscwcchlvgntump3x7e7tc0sgp9k43qeu892gfnz2hrr7akh2x8erh7zm2tv52884vyl462dm5tfcahgtuzt7j0npy7getf4trv5d4g78a9fkwu3kke6hcxdr6t2n7vz" assert(InvoiceRequest.decode(encoded).get == invoiceRequest) assert(invoiceRequest.offer.amount.isEmpty) assert(invoiceRequest.offer.description.isEmpty) From eef26147ec6d9edcea0b64e8c75ec579028ac8bc Mon Sep 17 00:00:00 2001 From: t-bast Date: Fri, 7 Mar 2025 16:20:51 +0100 Subject: [PATCH 6/7] Clean-up and refactoring This commit doesn't contain any implementation change, it only focuses on refactoring, adding comments, logs, applying formatter suggestions and improving tests. --- .../src/main/scala/fr/acinq/eclair/Logs.scala | 2 + .../scala/fr/acinq/eclair/db/OffersDb.scala | 6 +- .../fr/acinq/eclair/db/pg/PgOffersDb.scala | 12 +- .../eclair/db/sqlite/SqliteOffersDb.scala | 7 +- .../payment/offer/DefaultOfferHandler.scala | 108 +++++++++++++----- .../eclair/payment/offer/OfferCreator.scala | 43 ++++--- .../eclair/payment/offer/OfferManager.scala | 39 ++++--- .../payment/send/BlindedPathsResolver.scala | 2 +- .../fr/acinq/eclair/db/OffersDbSpec.scala | 44 ++----- .../basic/payment/OfferPaymentSpec.scala | 14 +-- .../payment/offer/OfferManagerSpec.scala | 3 +- .../eclair/wire/protocol/OfferTypesSpec.scala | 6 +- 12 files changed, 167 insertions(+), 119 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/Logs.scala b/eclair-core/src/main/scala/fr/acinq/eclair/Logs.scala index a79dd51f16..878b6ba41d 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/Logs.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/Logs.scala @@ -49,6 +49,7 @@ object Logs { parentPaymentId_opt: Option[UUID] = None, paymentId_opt: Option[UUID] = None, paymentHash_opt: Option[ByteVector32] = None, + offerId_opt: Option[ByteVector32] = None, txPublishId_opt: Option[UUID] = None, messageId_opt: Option[ByteVector32] = None, nodeAlias_opt: Option[String] = None): Map[String, String] = @@ -60,6 +61,7 @@ object Logs { parentPaymentId_opt.map(p => "parentPaymentId" -> s" p:$p"), paymentId_opt.map(i => "paymentId" -> s" i:$i"), paymentHash_opt.map(h => "paymentHash" -> s" h:$h"), + offerId_opt.map(o => "offerId" -> s" o:$o"), txPublishId_opt.map(t => "txPublishId" -> s" t:$t"), messageId_opt.map(m => "messageId" -> s" m:$m"), nodeAlias_opt.map(a => "nodeAlias" -> s" a:$a"), diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/db/OffersDb.scala b/eclair-core/src/main/scala/fr/acinq/eclair/db/OffersDb.scala index ced1553162..8a21335d3f 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/db/OffersDb.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/db/OffersDb.scala @@ -26,6 +26,7 @@ import fr.acinq.eclair.wire.protocol.OfferTypes.Offer trait OffersDb { /** * Add an offer managed by eclair. + * * @param pathId_opt If the offer uses a blinded path, this is the corresponding pathId. */ def addOffer(offer: Offer, pathId_opt: Option[ByteVector32], createdAt: TimestampMilli = TimestampMilli.now()): Unit @@ -38,9 +39,12 @@ trait OffersDb { /** * List offers managed by eclair. + * * @param onlyActive Whether to return only active offers or also disabled ones. */ def listOffers(onlyActive: Boolean): Seq[OfferData] } -case class OfferData(offer: Offer, pathId_opt: Option[ByteVector32], createdAt: TimestampMilli, disabledAt: Option[TimestampMilli]) +case class OfferData(offer: Offer, pathId_opt: Option[ByteVector32], createdAt: TimestampMilli, disabledAt_opt: Option[TimestampMilli]) { + val disabled: Boolean = disabledAt_opt.nonEmpty +} diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/db/pg/PgOffersDb.scala b/eclair-core/src/main/scala/fr/acinq/eclair/db/pg/PgOffersDb.scala index b5c4ab213c..ca38e77860 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/db/pg/PgOffersDb.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/db/pg/PgOffersDb.scala @@ -20,8 +20,8 @@ import fr.acinq.bitcoin.scalacompat.ByteVector32 import fr.acinq.eclair.TimestampMilli import fr.acinq.eclair.db.Monitoring.Metrics.withMetrics import fr.acinq.eclair.db.Monitoring.Tags.DbBackends -import fr.acinq.eclair.db.{OfferData, OffersDb} import fr.acinq.eclair.db.pg.PgUtils.PgLock +import fr.acinq.eclair.db.{OfferData, OffersDb} import fr.acinq.eclair.wire.protocol.OfferTypes import fr.acinq.eclair.wire.protocol.OfferTypes.Offer import grizzled.slf4j.Logging @@ -56,7 +56,7 @@ class PgOffersDb(implicit ds: DataSource, lock: PgLock) extends OffersDb with Lo } } - override def addOffer(offer: OfferTypes.Offer, pathId_opt: Option[ByteVector32], createdAt: TimestampMilli = TimestampMilli.now()): Unit = withMetrics("offers/add", DbBackends.Postgres){ + override def addOffer(offer: OfferTypes.Offer, pathId_opt: Option[ByteVector32], createdAt: TimestampMilli = TimestampMilli.now()): Unit = withMetrics("offers/add", DbBackends.Postgres) { withLock { pg => using(pg.prepareStatement("INSERT INTO payments.offers (offer_id, offer, path_id, created_at, is_active, disabled_at) VALUES (?, ?, ?, ?, TRUE, NULL)")) { statement => statement.setString(1, offer.offerId.toHex) @@ -68,7 +68,7 @@ class PgOffersDb(implicit ds: DataSource, lock: PgLock) extends OffersDb with Lo } } - override def disableOffer(offer: OfferTypes.Offer, disabledAt: TimestampMilli = TimestampMilli.now()): Unit = withMetrics("offers/disable", DbBackends.Postgres){ + override def disableOffer(offer: OfferTypes.Offer, disabledAt: TimestampMilli = TimestampMilli.now()): Unit = withMetrics("offers/disable", DbBackends.Postgres) { withLock { pg => using(pg.prepareStatement("UPDATE payments.offers SET disabled_at = ?, is_active = FALSE WHERE offer_id = ?")) { statement => statement.setTimestamp(1, disabledAt.toSqlTimestamp) @@ -79,17 +79,15 @@ class PgOffersDb(implicit ds: DataSource, lock: PgLock) extends OffersDb with Lo } private def parseOfferData(rs: ResultSet): OfferData = { - val disabledAt = rs.getTimestamp("disabled_at") - val disabledAt_opt = if (rs.wasNull()) None else Some(TimestampMilli.fromSqlTimestamp(disabledAt)) OfferData( Offer.decode(rs.getString("offer")).get, rs.getStringNullable("path_id").map(ByteVector32.fromValidHex), TimestampMilli.fromSqlTimestamp(rs.getTimestamp("created_at")), - disabledAt_opt + rs.getTimestampNullable("disabled_at").map(TimestampMilli.fromSqlTimestamp) ) } - override def listOffers(onlyActive: Boolean): Seq[OfferData] = withMetrics("offers/list", DbBackends.Postgres){ + override def listOffers(onlyActive: Boolean): Seq[OfferData] = withMetrics("offers/list", DbBackends.Postgres) { withLock { pg => if (onlyActive) { using(pg.prepareStatement("SELECT * FROM payments.offers WHERE is_active = TRUE ORDER BY created_at DESC")) { statement => diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteOffersDb.scala b/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteOffersDb.scala index 6a5c9c3b5d..886ffc5065 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteOffersDb.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteOffersDb.scala @@ -34,13 +34,14 @@ object SqliteOffersDb { } class SqliteOffersDb(val sqlite: Connection) extends OffersDb with Logging { + import SqliteOffersDb._ import SqliteUtils.ExtendedResultSet._ using(sqlite.createStatement(), inTransaction = true) { statement => getVersion(statement, DB_NAME) match { case None => - statement.executeUpdate("CREATE TABLE offers (offer_id BLOB NOT NULL PRIMARY KEY, offer TEXT NOT NULL, path_id BLOB, created_at INTEGER NOT NULL, is_active INTEGER NOT NULL, disabled_at INTEGER)") + statement.executeUpdate("CREATE TABLE offers (offer_id BLOB NOT NULL PRIMARY KEY, offer TEXT NOT NULL, path_id BLOB, created_at INTEGER NOT NULL, is_active BOOLEAN NOT NULL, disabled_at INTEGER)") statement.executeUpdate("CREATE INDEX offer_created_at_idx ON offers(created_at)") statement.executeUpdate("CREATE INDEX offer_is_active_idx ON offers(is_active)") case Some(CURRENT_VERSION) => () // table is up-to-date, nothing to do @@ -69,13 +70,11 @@ class SqliteOffersDb(val sqlite: Connection) extends OffersDb with Logging { } private def parseOfferData(rs: ResultSet): OfferData = { - val disabledAt = rs.getLong("disabled_at") - val disabledAt_opt = if (rs.wasNull()) None else Some(TimestampMilli(disabledAt)) OfferData( Offer.decode(rs.getString("offer")).get, rs.getByteVector32Nullable("path_id"), TimestampMilli(rs.getLong("created_at")), - disabledAt_opt + rs.getLongNullable("disabled_at").map(TimestampMilli(_)) ) } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/DefaultOfferHandler.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/DefaultOfferHandler.scala index dce0c17c41..98ef2b1e90 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/DefaultOfferHandler.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/DefaultOfferHandler.scala @@ -17,37 +17,47 @@ package fr.acinq.eclair.payment.offer import akka.actor.typed.Behavior -import akka.actor.typed.scaladsl.Behaviors +import akka.actor.typed.scaladsl.{ActorContext, Behaviors} import akka.actor.{ActorRef, typed} -import fr.acinq.eclair.crypto.Sphinx.RouteBlinding.BlindedRoute +import com.softwaremill.quicklens.ModifyPimp +import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey +import fr.acinq.eclair.Logs.LogCategory import fr.acinq.eclair.payment.offer.OfferManager.InvoiceRequestActor import fr.acinq.eclair.payment.relay.Relayer.RelayFees import fr.acinq.eclair.router.Router import fr.acinq.eclair.router.Router.{BlindedRouteRequest, ChannelHop} import fr.acinq.eclair.wire.protocol.OfferTypes -import fr.acinq.eclair.{CltvExpiryDelta, EncodedNodeId, MilliSatoshi, MilliSatoshiLong, NodeParams} +import fr.acinq.eclair.wire.protocol.OfferTypes.InvoiceRequest +import fr.acinq.eclair.{CltvExpiryDelta, EncodedNodeId, Logs, MilliSatoshiLong, NodeParams} case class OffersConfig(messagePathMinLength: Int, paymentPathCount: Int, paymentPathLength: Int, paymentPathCltvExpiryDelta: CltvExpiryDelta) +/** + * This actor creates Bolt 12 invoices for offers that are managed by eclair. + * It creates payment blinded paths whenever the corresponding offer is using a (message) blinded path. + */ object DefaultOfferHandler { def apply(nodeParams: NodeParams, router: ActorRef): Behavior[OfferManager.HandlerCommand] = { Behaviors.setup(context => Behaviors.receiveMessage { case OfferManager.HandleInvoiceRequest(replyTo, invoiceRequest) => - val amount = invoiceRequest.amount invoiceRequest.offer.contactInfos.head match { case OfferTypes.RecipientNodeId(_) => val route = InvoiceRequestActor.Route(Nil, nodeParams.channelConf.maxExpiryDelta) - replyTo ! InvoiceRequestActor.ApproveRequest(amount, Seq(route)) - case OfferTypes.BlindedPath(BlindedRoute(firstNodeId: EncodedNodeId.WithPublicKey, _, _)) if firstNodeId.publicKey == nodeParams.nodeId => - replyTo ! InvoiceRequestActor.ApproveRequest(amount, makeRoutes(nodeParams, Seq(Nil))) - case OfferTypes.BlindedPath(BlindedRoute(firstNodeId: EncodedNodeId.WithPublicKey, _, _)) => - val baseParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams - val routeParams = baseParams.copy(boundaries = baseParams.boundaries.copy(maxRouteLength = nodeParams.offersConfig.paymentPathLength, maxCltv = nodeParams.offersConfig.paymentPathCltvExpiryDelta)) - router ! BlindedRouteRequest(context.spawnAnonymous(waitForRoute(nodeParams, replyTo, invoiceRequest.offer, amount)), firstNodeId.publicKey, nodeParams.nodeId, amount, routeParams, nodeParams.offersConfig.paymentPathCount) - case OfferTypes.BlindedPath(BlindedRoute(_: EncodedNodeId.ShortChannelIdDir, _, _)) => - context.log.error("unexpected managed offer with compact first node id") - replyTo ! InvoiceRequestActor.RejectRequest("internal error") + replyTo ! InvoiceRequestActor.ApproveRequest(invoiceRequest.amount, Seq(route)) + case OfferTypes.BlindedPath(path) => + path.firstNodeId match { + case firstNodeId: EncodedNodeId.WithPublicKey if firstNodeId.publicKey == nodeParams.nodeId => + // We're using a fake blinded path starting at ourselves: we only need to add dummy hops. + val paths = PaymentPathsBuilder.finalizeRoutes(nodeParams, Seq(Nil)) + replyTo ! InvoiceRequestActor.ApproveRequest(invoiceRequest.amount, paths) + case firstNodeId: EncodedNodeId.WithPublicKey => + val pathBuilder = context.spawnAnonymous(PaymentPathsBuilder(nodeParams, router, invoiceRequest)) + pathBuilder ! PaymentPathsBuilder.GetPaymentPaths(replyTo, firstNodeId.publicKey) + case _: EncodedNodeId.ShortChannelIdDir => + context.log.error("unexpected managed offer with compact first node id") + replyTo ! InvoiceRequestActor.RejectRequest("internal error") + } } Behaviors.same case OfferManager.HandlePayment(replyTo, _, _) => @@ -57,24 +67,60 @@ object DefaultOfferHandler { ) } - def waitForRoute(nodeParams: NodeParams, replyTo: typed.ActorRef[InvoiceRequestActor.Command], offer: OfferTypes.Offer, amount: MilliSatoshi): Behavior[Router.PaymentRouteResponse] = { - Behaviors.receive { - case (_, Router.RouteResponse(routes)) => - replyTo ! InvoiceRequestActor.ApproveRequest(amount, makeRoutes(nodeParams, routes.map(_.hops))) - Behaviors.stopped - case (context, Router.PaymentRouteNotFound(error)) => - context.log.error("couldn't find blinded route for creating invoice offer={} amount={} : {}", offer, amount, error.getMessage) - replyTo ! InvoiceRequestActor.RejectRequest("internal error") - Behaviors.stopped + /** + * Short-lived actor that creates payment blinded paths with help from the router. + */ + private object PaymentPathsBuilder { + + // @formatter:off + sealed trait Command + case class GetPaymentPaths(replyTo: typed.ActorRef[InvoiceRequestActor.Command], blindedPathFirstNodeId: PublicKey) extends Command + private case class WrappedRouteResponse(response: Router.PaymentRouteResponse) extends Command + // @formatter:on + + def apply(nodeParams: NodeParams, router: ActorRef, invoiceRequest: InvoiceRequest): Behavior[Command] = { + Behaviors.setup { context => + Behaviors.withMdc(Logs.mdc(category_opt = Some(LogCategory.PAYMENT), offerId_opt = Some(invoiceRequest.offer.offerId))) { + Behaviors.receiveMessagePartial { + case GetPaymentPaths(replyTo, blindedPathFirstNodeId) => + val routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams + .modify(_.boundaries.maxRouteLength).setTo(nodeParams.offersConfig.paymentPathLength) + .modify(_.boundaries.maxCltv).setTo(nodeParams.offersConfig.paymentPathCltvExpiryDelta) + router ! BlindedRouteRequest(context.messageAdapter(WrappedRouteResponse), blindedPathFirstNodeId, nodeParams.nodeId, invoiceRequest.amount, routeParams, nodeParams.offersConfig.paymentPathCount) + waitForRoute(nodeParams, replyTo, invoiceRequest, blindedPathFirstNodeId, context) + } + } + } } - } - // Ensure that we don't leak routing information by always returning the same number of routes, with the same number of hops and the same fees and CLTV delta. - private def makeRoutes(nodeParams: NodeParams, routes: Seq[Seq[Router.ChannelHop]]): Seq[InvoiceRequestActor.Route] = { - (0 until nodeParams.offersConfig.paymentPathCount).map(i => { - val hops = routes(i % routes.length) - val dummyHops = Seq.fill(nodeParams.offersConfig.paymentPathLength - hops.length)(ChannelHop.dummy(nodeParams.nodeId, 0 msat, 0, CltvExpiryDelta(0))) - InvoiceRequestActor.Route(hops ++ dummyHops, nodeParams.channelConf.maxExpiryDelta, feeOverride = Some(RelayFees.zero), cltvOverride = Some(nodeParams.offersConfig.paymentPathCltvExpiryDelta)) - }) + private def waitForRoute(nodeParams: NodeParams, replyTo: typed.ActorRef[InvoiceRequestActor.Command], invoiceRequest: InvoiceRequest, blindedPathFirstNodeId: PublicKey, context: ActorContext[Command]): Behavior[Command] = { + Behaviors.receiveMessagePartial { + case WrappedRouteResponse(Router.RouteResponse(routes)) => + context.log.debug("found {} blinded paths starting at {} (amount={})", routes.size, blindedPathFirstNodeId, invoiceRequest.amount) + replyTo ! InvoiceRequestActor.ApproveRequest(invoiceRequest.amount, finalizeRoutes(nodeParams, routes.map(_.hops))) + Behaviors.stopped + case WrappedRouteResponse(Router.PaymentRouteNotFound(error)) => + context.log.warn("couldn't find blinded paths to create invoice amount={} firstNodeId={}: {}", invoiceRequest.amount, blindedPathFirstNodeId, error.getMessage) + replyTo ! InvoiceRequestActor.RejectRequest("internal error") + Behaviors.stopped + } + } + + def finalizeRoutes(nodeParams: NodeParams, routes: Seq[Seq[Router.ChannelHop]]): Seq[InvoiceRequestActor.Route] = { + (0 until nodeParams.offersConfig.paymentPathCount).map(i => { + // We always return the number of routes configured, regardless of how many routes were actually found by the + // router: this ensures that we don't leak information about our graph data. + // However, payers may eagerly use MPP whereas we actually have a single route available, which could result in + // a lower payment success rate. + val hops = routes(i % routes.length) + // We always pad blinded paths to the configured length, using dummy hops if necessary. + val dummyHops = Seq.fill(nodeParams.offersConfig.paymentPathLength - hops.length)(ChannelHop.dummy(nodeParams.nodeId, 0 msat, 0, CltvExpiryDelta(0))) + // We always override the fees of the payment path: the payer shouldn't be paying for our privacy. + // Note that we told the router to only find paths with a lower cltv_expiry_delta than what we'll be using, + // which ensures that we won't reject payments because of their expiry. + InvoiceRequestActor.Route(hops ++ dummyHops, nodeParams.channelConf.maxExpiryDelta, feeOverride_opt = Some(RelayFees.zero), cltvOverride_opt = Some(nodeParams.offersConfig.paymentPathCltvExpiryDelta)) + }) + } } + } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/OfferCreator.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/OfferCreator.scala index 3abf87161a..8788602295 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/OfferCreator.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/OfferCreator.scala @@ -16,9 +16,9 @@ package fr.acinq.eclair.payment.offer -import akka.actor.{ActorRef, typed} import akka.actor.typed.Behavior import akka.actor.typed.scaladsl.{ActorContext, Behaviors} +import akka.actor.{ActorRef, typed} import fr.acinq.bitcoin.scalacompat.Crypto.{PrivateKey, PublicKey} import fr.acinq.bitcoin.scalacompat.{Block, ByteVector32} import fr.acinq.eclair.message.OnionMessages @@ -29,29 +29,38 @@ import fr.acinq.eclair.wire.protocol.OfferTypes._ import fr.acinq.eclair.wire.protocol.TlvStream import fr.acinq.eclair.{MilliSatoshi, NodeParams, TimestampSecond, randomBytes32, randomKey} +/** + * A short-lived actor that creates an offer based on the parameters provided. + * It will ask the router for a blinded path when [[OfferCreator.Create.blindedPathsFirstNodeId_opt]] is provided. + */ object OfferCreator { - sealed trait Command + // @formatter:off + sealed trait Command case class Create(replyTo: typed.ActorRef[CreateOfferResult], description_opt: Option[String], amount_opt: Option[MilliSatoshi], expiry_opt: Option[TimestampSecond], issuer_opt: Option[String], blindedPathsFirstNodeId_opt: Option[PublicKey]) extends Command - private case class WrappedRouterResponse(response: Router.MessageRouteResponse) extends Command + // @formatter:on + // @formatter:off sealed trait CreateOfferResult - case class CreatedOffer(offer: Offer) extends CreateOfferResult - case class CreateOfferError(reason: String) extends CreateOfferResult + // @formatter:on - def apply(nodeParams: NodeParams, router: ActorRef, offerManager: typed.ActorRef[OfferManager.Command], defaultOfferHandler: typed.ActorRef[OfferManager.HandlerCommand]): Behavior[Command] = - Behaviors.receivePartial { - case (context, Create(replyTo, description_opt, amount_opt, expiry_opt, issuer_opt, blindedPathsFirstNodeId_opt)) => - new OfferCreator(context, replyTo, nodeParams, router, offerManager, defaultOfferHandler).init(description_opt, amount_opt, expiry_opt, issuer_opt, blindedPathsFirstNodeId_opt) + def apply(nodeParams: NodeParams, router: ActorRef, offerManager: typed.ActorRef[OfferManager.Command], defaultOfferHandler: typed.ActorRef[OfferManager.HandlerCommand]): Behavior[Command] = { + Behaviors.setup { context => + Behaviors.receiveMessagePartial { + case Create(replyTo, description_opt, amount_opt, expiry_opt, issuer_opt, blindedPathsFirstNodeId_opt) => + val actor = new OfferCreator(context, replyTo, nodeParams, router, offerManager, defaultOfferHandler) + actor.createOffer(description_opt, amount_opt, expiry_opt, issuer_opt, blindedPathsFirstNodeId_opt) + } } + } } private class OfferCreator(context: ActorContext[OfferCreator.Command], @@ -62,11 +71,11 @@ private class OfferCreator(context: ActorContext[OfferCreator.Command], import OfferCreator._ - private def init(description_opt: Option[String], - amount_opt: Option[MilliSatoshi], - expiry_opt: Option[TimestampSecond], - issuer_opt: Option[String], - blindedPathsFirstNodeId_opt: Option[PublicKey]): Behavior[Command] = { + private def createOffer(description_opt: Option[String], + amount_opt: Option[MilliSatoshi], + expiry_opt: Option[TimestampSecond], + issuer_opt: Option[String], + blindedPathsFirstNodeId_opt: Option[PublicKey]): Behavior[Command] = { if (amount_opt.nonEmpty && description_opt.isEmpty) { replyTo ! CreateOfferError("Description is mandatory for offers with set amount.") Behaviors.stopped @@ -83,6 +92,7 @@ private class OfferCreator(context: ActorContext[OfferCreator.Command], router ! Router.MessageRouteRequest(context.messageAdapter(WrappedRouterResponse(_)), firstNodeId, nodeParams.nodeId, Set.empty) waitForRoute(firstNodeId, tlvs) case None => + // When not using a blinded path, we use our public nodeId for the offer (no privacy). val offer = Offer(TlvStream(tlvs + OfferNodeId(nodeParams.nodeId))) registerOffer(offer, Some(nodeParams.privateKey), None) } @@ -93,6 +103,7 @@ private class OfferCreator(context: ActorContext[OfferCreator.Command], Behaviors.receiveMessagePartial { case WrappedRouterResponse(Router.MessageRoute(intermediateNodes, _)) => val pathId = randomBytes32() + // We add dummy hops to the route if it is too short, which provides better privacy. val nodes = firstNode +: (intermediateNodes ++ Seq.fill(nodeParams.offersConfig.messagePathMinLength - intermediateNodes.length - 1)(nodeParams.nodeId)) val paths = Seq(OnionMessages.buildRoute(randomKey(), nodes.map(IntermediateNode(_)), Recipient(nodeParams.nodeId, Some(pathId))).route) val offer = Offer(TlvStream(tlvs + OfferPaths(paths))) @@ -103,9 +114,9 @@ private class OfferCreator(context: ActorContext[OfferCreator.Command], } } - private def registerOffer(offer: Offer, nodeKey: Option[PrivateKey], pathId_opt: Option[ByteVector32]): Behavior[Command] = { + private def registerOffer(offer: Offer, nodeKey_opt: Option[PrivateKey], pathId_opt: Option[ByteVector32]): Behavior[Command] = { nodeParams.db.offers.addOffer(offer, pathId_opt) - offerManager ! OfferManager.RegisterOffer(offer, nodeKey, pathId_opt, defaultOfferHandler) + offerManager ! OfferManager.RegisterOffer(offer, nodeKey_opt, pathId_opt, defaultOfferHandler) replyTo ! CreatedOffer(offer) Behaviors.stopped } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/OfferManager.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/OfferManager.scala index c32e839097..f4a6edb12c 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/OfferManager.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/OfferManager.scala @@ -34,7 +34,7 @@ import fr.acinq.eclair.router.Router import fr.acinq.eclair.wire.protocol.OfferTypes.{InvoiceRequest, InvoiceTlv, Offer} import fr.acinq.eclair.wire.protocol.PaymentOnion.FinalPayload import fr.acinq.eclair.wire.protocol._ -import fr.acinq.eclair.{CltvExpiryDelta, Logs, MilliSatoshi, MilliSatoshiLong, NodeParams, TimestampMilli, TimestampSecond, nodeFee, randomBytes32} +import fr.acinq.eclair.{CltvExpiryDelta, Logs, MilliSatoshi, NodeParams, TimestampMilli, TimestampSecond, nodeFee, randomBytes32} import scodec.bits.ByteVector import scala.concurrent.duration.FiniteDuration @@ -47,14 +47,17 @@ object OfferManager { sealed trait Command /** - * Register an offer and its handler. + * Register an offer so that we can respond to invoice requests for it using the handler provided. * - * @param offer The offer. - * @param nodeKey The private key corresponding to the node id used in the offer. - * @param pathId_opt If the offer uses a blinded path, the path id of this blinded path. - * @param handler An actor that will be in charge of accepting or rejecting invoice requests and payments for this offer. + * @param offer The offer to register. + * @param nodeKey_opt If the offer has a node id, this must be the associated private key. + * @param pathId_opt If the offer uses a blinded path, the path id of this blinded path. + * @param handler An actor that will be in charge of accepting or rejecting invoice requests and payments for this offer. */ - case class RegisterOffer(offer: Offer, nodeKey: Option[PrivateKey], pathId_opt: Option[ByteVector32], handler: ActorRef[HandlerCommand]) extends Command + case class RegisterOffer(offer: Offer, nodeKey_opt: Option[PrivateKey], pathId_opt: Option[ByteVector32], handler: ActorRef[HandlerCommand]) extends Command { + require(offer.nodeId.isEmpty || nodeKey_opt.nonEmpty, "offers including the node_id field must be registered with the corresponding private key") + require(!offer.contactInfos.exists(_.isInstanceOf[OfferTypes.BlindedPath]) || pathId_opt.nonEmpty, "offers including a blinded path must be registered with the corresponding path_id") + } /** * Forget about an offer. Invoice requests and payment attempts for this offer will be ignored. @@ -91,12 +94,8 @@ object OfferManager { case class HandlePayment(replyTo: ActorRef[PaymentActor.Command], offer: Offer, invoiceData: MinimalInvoiceData) extends HandlerCommand /** - * Register an offer so that we can respond to invoice requests for it. - * - * @param offer The offer to register. - * @param nodeKey_opt If the offer has a node id, this must be the associated private key. - * @param pathId_opt If this offer has a blinded path, this must be its path id. - * @param handler Handler for invoice requests and payments for this offer. + * An active offer, for which we handle invoice requests. + * See [[RegisterOffer]] for more details about the fields. */ private case class RegisteredOffer(offer: Offer, nodeKey_opt: Option[PrivateKey], pathId_opt: Option[ByteVector32], handler: ActorRef[HandlerCommand]) @@ -168,16 +167,20 @@ object OfferManager { customTlvs: Set[GenericTlv] = Set.empty) extends Command /** - * @param feeOverride fees that will be published for this route, the difference between these and the + * Route used in payment blinded paths: [[feeOverride_opt]] and [[cltvOverride_opt]] allow hiding the routing + * parameters of the route's intermediate hops, which provides better privacy. + * + * @param feeOverride_opt fees that will be published for this route, the difference between these and the * actual fees of the route will be paid by the recipient. - * @param cltvOverride Cltv expiry delta to publish for the route. + * @param cltvOverride_opt cltv_expiry_delta to publish for the route, which must be greater than the route's + * real cltv_expiry_delta. * @param shortChannelIdDir_opt short channel id and direction to use for the first node instead of its node id. */ - case class Route(hops: Seq[Router.ChannelHop], maxFinalExpiryDelta: CltvExpiryDelta, feeOverride: Option[RelayFees] = None, cltvOverride: Option[CltvExpiryDelta] = None, shortChannelIdDir_opt: Option[ShortChannelIdDir] = None) { + case class Route(hops: Seq[Router.ChannelHop], maxFinalExpiryDelta: CltvExpiryDelta, feeOverride_opt: Option[RelayFees] = None, cltvOverride_opt: Option[CltvExpiryDelta] = None, shortChannelIdDir_opt: Option[ShortChannelIdDir] = None) { def finalize(nodePriv: PrivateKey, preimage: ByteVector32, amount: MilliSatoshi, invoiceRequest: InvoiceRequest, minFinalExpiryDelta: CltvExpiryDelta, pluginData_opt: Option[ByteVector]): ReceivingRoute = { val aggregatedPaymentInfo = aggregatePaymentInfo(amount, hops, minFinalExpiryDelta) - val fees = feeOverride.getOrElse(RelayFees(aggregatedPaymentInfo.feeBase, aggregatedPaymentInfo.feeProportionalMillionths)) - val cltvExpiryDelta = cltvOverride.getOrElse(aggregatedPaymentInfo.cltvExpiryDelta) + val fees = feeOverride_opt.getOrElse(RelayFees(aggregatedPaymentInfo.feeBase, aggregatedPaymentInfo.feeProportionalMillionths)) + val cltvExpiryDelta = cltvOverride_opt.getOrElse(aggregatedPaymentInfo.cltvExpiryDelta) val paymentInfo = aggregatedPaymentInfo.copy(feeBase = fees.feeBase, feeProportionalMillionths = fees.feeProportionalMillionths, cltvExpiryDelta = cltvExpiryDelta) val recipientFees = RelayFees(aggregatedPaymentInfo.feeBase - paymentInfo.feeBase, aggregatedPaymentInfo.feeProportionalMillionths - paymentInfo.feeProportionalMillionths) val metadata = MinimalInvoiceData(preimage, invoiceRequest.payerId, TimestampSecond.now(), invoiceRequest.quantity, amount, recipientFees, pluginData_opt) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/BlindedPathsResolver.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/BlindedPathsResolver.scala index e5e2d47fd6..df284bf46f 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/BlindedPathsResolver.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/BlindedPathsResolver.scala @@ -14,7 +14,7 @@ import fr.acinq.eclair.router.Router import fr.acinq.eclair.wire.protocol.OfferTypes.PaymentInfo import fr.acinq.eclair.wire.protocol.RouteBlindingEncryptedDataCodecs.RouteBlindingDecryptedData import fr.acinq.eclair.wire.protocol.{BlindedRouteData, OfferTypes, RouteBlindingEncryptedDataCodecs} -import fr.acinq.eclair.{EncodedNodeId, Logs, MilliSatoshiLong, NodeParams, ShortChannelId} +import fr.acinq.eclair.{EncodedNodeId, Logs, NodeParams, ShortChannelId} import scodec.bits.ByteVector import scala.annotation.tailrec diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/db/OffersDbSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/db/OffersDbSpec.scala index 4e5e4b53c3..5b64c2e00d 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/db/OffersDbSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/db/OffersDbSpec.scala @@ -17,10 +17,7 @@ package fr.acinq.eclair.db import fr.acinq.bitcoin.scalacompat.Block -import fr.acinq.eclair.TestDatabases.{TestPgDatabases, TestSqliteDatabases} import fr.acinq.eclair._ -import fr.acinq.eclair.db.pg.PgOffersDb -import fr.acinq.eclair.db.sqlite.SqliteOffersDb import fr.acinq.eclair.wire.protocol.OfferTypes.Offer import org.scalatest.funsuite.AnyFunSuite @@ -28,42 +25,25 @@ class OffersDbSpec extends AnyFunSuite { import fr.acinq.eclair.TestDatabases.forAllDbs - test("init database two times in a row") { - forAllDbs { - case sqlite: TestSqliteDatabases => - new SqliteOffersDb(sqlite.connection) - new SqliteOffersDb(sqlite.connection) - case pg: TestPgDatabases => - new PgOffersDb()(pg.datasource, pg.lock) - new PgOffersDb()(pg.datasource, pg.lock) - } - } - test("add/disable/enable/list offers") { forAllDbs { dbs => val db = dbs.offers assert(db.listOffers(onlyActive = false).isEmpty) - val offer1 = Offer(None, Some("test 1"), randomKey().publicKey, Features(), Block.LivenetGenesisBlock.hash) - db.addOffer(offer1, None) - val listed1 = db.listOffers(onlyActive = true) - assert(listed1.length == 1) - assert(listed1.head.offer == offer1) - assert(listed1.head.pathId_opt == None) - assert(listed1.head.disabledAt == None) - val offer2 = Offer(None, Some("test 2"), randomKey().publicKey, Features(), Block.LivenetGenesisBlock.hash) + val offer1 = OfferData(Offer(None, Some("test 1"), randomKey().publicKey, Features(), Block.LivenetGenesisBlock.hash), None, TimestampMilli(100), None) + db.addOffer(offer1.offer, None, offer1.createdAt) + assert(db.listOffers(onlyActive = true) == Seq(offer1)) val pathId = randomBytes32() - db.addOffer(offer2, Some(pathId)) - assert(db.listOffers(onlyActive = true).length == 2) - db.disableOffer(offer1) - assert(db.listOffers(onlyActive = false).length == 2) - val listed2 = db.listOffers(onlyActive = true) - assert(listed2.length == 1) - assert(listed2.head.offer == offer2) - assert(listed2.head.pathId_opt == Some(pathId)) - assert(listed2.head.disabledAt == None) - db.disableOffer(offer2) + val offer2 = OfferData(Offer(Some(15_000 msat), Some("test 2"), randomKey().publicKey, Features(), Block.LivenetGenesisBlock.hash), Some(pathId), TimestampMilli(200), None) + db.addOffer(offer2.offer, Some(pathId), offer2.createdAt) + assert(db.listOffers(onlyActive = true) == Seq(offer2, offer1)) + db.disableOffer(offer1.offer, disabledAt = TimestampMilli(250)) + assert(db.listOffers(onlyActive = true) == Seq(offer2)) + assert(db.listOffers(onlyActive = false) == Seq(offer2, offer1.copy(disabledAt_opt = Some(TimestampMilli(250))))) + db.disableOffer(offer2.offer, disabledAt = TimestampMilli(300)) assert(db.listOffers(onlyActive = true).isEmpty) + assert(db.listOffers(onlyActive = false) == Seq(offer2.copy(disabledAt_opt = Some(TimestampMilli(300))), offer1.copy(disabledAt_opt = Some(TimestampMilli(250))))) } } + } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/payment/OfferPaymentSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/payment/OfferPaymentSpec.scala index 1ab0261340..d5271e9f97 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/payment/OfferPaymentSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/payment/OfferPaymentSpec.scala @@ -250,7 +250,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { carol.router ! Router.FinalizeRoute(sender.ref.toTyped, Router.PredefinedNodeRoute(amount, Seq(bob.nodeId, carol.nodeId))) val route = sender.expectMsgType[Router.RouteResponse].routes.head - val routes = Seq(InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta, feeOverride = Some(RelayFees.zero))) + val routes = Seq(InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta, feeOverride_opt = Some(RelayFees.zero))) val (offer, result) = sendOfferPayment(f, alice, carol, amount, routes) val payment = verifyPaymentSuccess(offer, amount, result) assert(payment.parts.length == 1) @@ -286,8 +286,8 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { val route = sender.expectMsgType[Router.RouteResponse].routes.head val routes = Seq( - InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta, feeOverride = Some(RelayFees.zero)), - InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta, feeOverride = Some(RelayFees.zero)), + InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta, feeOverride_opt = Some(RelayFees.zero)), + InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta, feeOverride_opt = Some(RelayFees.zero)), ) val (offer, result) = sendOfferPayment(f, alice, carol, amount, routes, maxAttempts = 3) val payment = verifyPaymentSuccess(offer, amount, result) @@ -369,8 +369,8 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { val amount = 125_000_000 msat val routes = Seq( - InvoiceRequestActor.Route(route.hops :+ ChannelHop.dummy(carol.nodeId, 150 msat, 0, CltvExpiryDelta(50)), maxFinalExpiryDelta, feeOverride = Some(RelayFees.zero)), - InvoiceRequestActor.Route(route.hops ++ Seq(ChannelHop.dummy(carol.nodeId, 50 msat, 0, CltvExpiryDelta(20)), ChannelHop.dummy(carol.nodeId, 100 msat, 0, CltvExpiryDelta(30))), maxFinalExpiryDelta, feeOverride = Some(RelayFees.zero)), + InvoiceRequestActor.Route(route.hops :+ ChannelHop.dummy(carol.nodeId, 150 msat, 0, CltvExpiryDelta(50)), maxFinalExpiryDelta, feeOverride_opt = Some(RelayFees.zero)), + InvoiceRequestActor.Route(route.hops ++ Seq(ChannelHop.dummy(carol.nodeId, 50 msat, 0, CltvExpiryDelta(20)), ChannelHop.dummy(carol.nodeId, 100 msat, 0, CltvExpiryDelta(30))), maxFinalExpiryDelta, feeOverride_opt = Some(RelayFees.zero)), ) val (offer, result) = sendOfferPayment(f, alice, carol, amount, routes) val payment = verifyPaymentSuccess(offer, amount, result) @@ -402,7 +402,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { carol.router ! Router.FinalizeRoute(sender.ref.toTyped, Router.PredefinedNodeRoute(amount, Seq(bob.nodeId, carol.nodeId))) val route = sender.expectMsgType[Router.RouteResponse].routes.head - val routes = Seq(InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta, feeOverride = Some(RelayFees.zero))) + val routes = Seq(InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta, feeOverride_opt = Some(RelayFees.zero))) val (offer, result) = sendPrivateOfferPayment(f, alice, carol, amount, routes) val payment = verifyPaymentSuccess(offer, amount, result) assert(payment.parts.forall(_.feesPaid == 0.msat)) @@ -433,7 +433,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { import f._ val amount = 250_000_000 msat - val routes = Seq(InvoiceRequestActor.Route(Seq(ChannelHop.dummy(bob.nodeId, 10 msat, 25, CltvExpiryDelta(24)), ChannelHop.dummy(bob.nodeId, 5 msat, 10, CltvExpiryDelta(36))), maxFinalExpiryDelta, feeOverride = Some(RelayFees.zero))) + val routes = Seq(InvoiceRequestActor.Route(Seq(ChannelHop.dummy(bob.nodeId, 10 msat, 25, CltvExpiryDelta(24)), ChannelHop.dummy(bob.nodeId, 5 msat, 10, CltvExpiryDelta(36))), maxFinalExpiryDelta, feeOverride_opt = Some(RelayFees.zero))) val (offer, result) = sendOfferPayment(f, alice, bob, amount, routes) val payment = verifyPaymentSuccess(offer, amount, result) assert(payment.parts.length == 1) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/offer/OfferManagerSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/offer/OfferManagerSpec.scala index c8c42636f4..16fad765a8 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/offer/OfferManagerSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/offer/OfferManagerSpec.scala @@ -77,7 +77,8 @@ class OfferManagerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app val handleInvoiceRequest = handler.expectMessageType[HandleInvoiceRequest] assert(handleInvoiceRequest.invoiceRequest.isValid) assert(handleInvoiceRequest.invoiceRequest.payerId == payerKey.publicKey) - handleInvoiceRequest.replyTo ! InvoiceRequestActor.ApproveRequest(amount, Seq(InvoiceRequestActor.Route(hops, CltvExpiryDelta(1000), feeOverride = if (hideFees) Some(RelayFees.zero) else None)), pluginData_opt) + val feeOverride_opt = if (hideFees) Some(RelayFees.zero) else None + handleInvoiceRequest.replyTo ! InvoiceRequestActor.ApproveRequest(amount, Seq(InvoiceRequestActor.Route(hops, CltvExpiryDelta(1000), feeOverride_opt)), pluginData_opt) val invoiceMessage = postman.expectMessageType[Postman.SendMessage] val Right(invoice) = Bolt12Invoice.validate(invoiceMessage.message.get[OnionMessagePayloadTlv.Invoice].get.tlvs) assert(invoice.validateFor(handleInvoiceRequest.invoiceRequest, pathNodeId).isRight) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/wire/protocol/OfferTypesSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/wire/protocol/OfferTypesSpec.scala index 72de49bb7e..b0a541ce7a 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/wire/protocol/OfferTypesSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/wire/protocol/OfferTypesSpec.scala @@ -122,7 +122,11 @@ class OfferTypesSpec extends AnyFunSuite { val request = InvoiceRequest(offer, 500 msat, 1, Features.empty, payerKey, Block.LivenetGenesisBlock.hash) assert(request.isValid) assert(request.offer == offer) - assertThrows[Exception](signInvoiceRequest(request.copy(records = TlvStream(request.records.records.filter { case InvoiceRequestAmount(_) => false case _ => true })), payerKey)) + // Since the offer doesn't contain an amount, the invoice_request must contain one to be valid. + assertThrows[Exception](request.copy(records = TlvStream(request.records.records.filter { + case InvoiceRequestAmount(_) => false + case _ => true + }))) } test("check that invoice request matches offer (chain compatibility)") { From 25df7bff3229ea16619900b7ecd637394ca97c91 Mon Sep 17 00:00:00 2001 From: Thomas HUET Date: Fri, 7 Mar 2025 19:10:49 +0100 Subject: [PATCH 7/7] No eclairImpl in tests --- .../basic/fixtures/MinimalNodeFixture.scala | 2 - .../basic/payment/OfferPaymentSpec.scala | 103 +++++++++--------- 2 files changed, 49 insertions(+), 56 deletions(-) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/fixtures/MinimalNodeFixture.scala b/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/fixtures/MinimalNodeFixture.scala index 6a5d676f9a..eaff03f4a6 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/fixtures/MinimalNodeFixture.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/fixtures/MinimalNodeFixture.scala @@ -61,8 +61,6 @@ case class MinimalNodeFixture private(nodeParams: NodeParams, bitcoinClient: TestBitcoinCoreClient) { val nodeId = nodeParams.nodeId val routeParams = nodeParams.routerConf.pathFindingExperimentConf.experiments.values.head.getDefaultRouteParams - - val eclairImpl = new EclairImpl(Kit(nodeParams, system, watcher.ref.toTyped, paymentHandler, register, relayer, router, switchboard, paymentInitiator, TestProbe()(system).ref, TestProbe()(system).ref.toTyped, TestProbe()(system).ref.toTyped, postman, offerManager, defaultOfferHandler, wallet)) } object MinimalNodeFixture extends Assertions with Eventually with IntegrationPatience with EitherValues { diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/payment/OfferPaymentSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/payment/OfferPaymentSpec.scala index d5271e9f97..b465fab95d 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/payment/OfferPaymentSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/payment/OfferPaymentSpec.scala @@ -33,7 +33,7 @@ import fr.acinq.eclair.integration.basic.fixtures.composite.ThreeNodesFixture import fr.acinq.eclair.message.OnionMessages import fr.acinq.eclair.message.OnionMessages.{IntermediateNode, Recipient, buildRoute} import fr.acinq.eclair.payment._ -import fr.acinq.eclair.payment.offer.OfferManager +import fr.acinq.eclair.payment.offer.{OfferCreator, OfferManager} import fr.acinq.eclair.payment.offer.OfferManager.InvoiceRequestActor import fr.acinq.eclair.payment.relay.Relayer.RelayFees import fr.acinq.eclair.payment.send.OfferPayment @@ -167,23 +167,29 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { } } - def sendOfferPayment(f: FixtureParam, payer: MinimalNodeFixture, recipient: MinimalNodeFixture, amount: MilliSatoshi, routes: Seq[InvoiceRequestActor.Route], maxAttempts: Int = 1): (Offer, PaymentEvent) = { - import f._ + def createOffer(recipient: MinimalNodeFixture, description_opt: Option[String], amount_opt: Option[MilliSatoshi], issuer_opt: Option[String], blindedPathsFirstNodeId_opt: Option[PublicKey]): Offer = { + val sender = TestProbe("sender")(recipient.system) + val offerCreator = recipient.system.spawnAnonymous(OfferCreator(recipient.nodeParams, recipient.router, recipient.offerManager, recipient.defaultOfferHandler)) + offerCreator ! OfferCreator.Create(sender.ref.toTyped, description_opt, amount_opt, None, issuer_opt, blindedPathsFirstNodeId_opt) + sender.expectMsgType[OfferCreator.CreatedOffer].offer + } - val sender = TestProbe("sender") - val offer = Offer(None, Some("test"), recipient.nodeId, Features.empty, recipient.nodeParams.chainHash) - val handler = recipient.system.spawnAnonymous(offerHandler(amount, routes)) - recipient.offerManager ! OfferManager.RegisterOffer(offer, Some(recipient.nodeParams.privateKey), None, handler) + def payOffer(payer: MinimalNodeFixture, offer: Offer, amount: MilliSatoshi, maxAttempts: Int = 1): PaymentEvent = { + val sender = TestProbe("sender")(payer.system) val offerPayment = payer.system.spawnAnonymous(OfferPayment(payer.nodeParams, payer.postman, payer.router, payer.register, payer.paymentInitiator)) val sendPaymentConfig = OfferPayment.SendPaymentConfig(None, connectDirectly = false, maxAttempts, payer.routeParams, blocking = true) offerPayment ! OfferPayment.PayOffer(sender.ref, offer, amount, 1, sendPaymentConfig) - (offer, sender.expectMsgType[PaymentEvent]) + sender.expectMsgType[PaymentEvent] } - def sendPrivateOfferPayment(f: FixtureParam, payer: MinimalNodeFixture, recipient: MinimalNodeFixture, amount: MilliSatoshi, routes: Seq[InvoiceRequestActor.Route], maxAttempts: Int = 1): (Offer, PaymentEvent) = { - import f._ + def sendOfferPayment(payer: MinimalNodeFixture, recipient: MinimalNodeFixture, amount: MilliSatoshi, routes: Seq[InvoiceRequestActor.Route], maxAttempts: Int = 1): (Offer, PaymentEvent) = { + val offer = Offer(None, Some("test"), recipient.nodeId, Features.empty, recipient.nodeParams.chainHash) + val handler = recipient.system.spawnAnonymous(offerHandler(amount, routes)) + recipient.offerManager ! OfferManager.RegisterOffer(offer, Some(recipient.nodeParams.privateKey), None, handler) + (offer, payOffer(payer, offer, amount, maxAttempts)) + } - val sender = TestProbe("sender") + def sendPrivateOfferPayment(payer: MinimalNodeFixture, recipient: MinimalNodeFixture, amount: MilliSatoshi, routes: Seq[InvoiceRequestActor.Route], maxAttempts: Int = 1): (Offer, PaymentEvent) = { val recipientKey = randomKey() val pathId = randomBytes32() val offerPaths = routes.map(route => { @@ -193,10 +199,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { val offer = Offer(None, Some("test"), recipientKey.publicKey, Features.empty, recipient.nodeParams.chainHash, additionalTlvs = Set(OfferPaths(offerPaths))) val handler = recipient.system.spawnAnonymous(offerHandler(amount, routes)) recipient.offerManager ! OfferManager.RegisterOffer(offer, Some(recipientKey), Some(pathId), handler) - val offerPayment = payer.system.spawnAnonymous(OfferPayment(payer.nodeParams, payer.postman, payer.router, payer.register, payer.paymentInitiator)) - val sendPaymentConfig = OfferPayment.SendPaymentConfig(None, connectDirectly = false, maxAttempts, payer.routeParams, blocking = true) - offerPayment ! OfferPayment.PayOffer(sender.ref, offer, amount, 1, sendPaymentConfig) - (offer, sender.expectMsgType[PaymentEvent]) + (offer, payOffer(payer, offer, amount, maxAttempts)) } def sendOfferPaymentWithInvalidAmount(f: FixtureParam, payer: MinimalNodeFixture, recipient: MinimalNodeFixture, payerAmount: MilliSatoshi, recipientAmount: MilliSatoshi, routes: Seq[InvoiceRequestActor.Route]): PaymentFailed = { @@ -235,7 +238,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { val route = sender.expectMsgType[Router.RouteResponse].routes.head val routes = Seq(InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta)) - val (offer, result) = sendOfferPayment(f, alice, carol, amount, routes) + val (offer, result) = sendOfferPayment(alice, carol, amount, routes) val payment = verifyPaymentSuccess(offer, amount, result) assert(payment.parts.length == 1) assert(payment.parts.head.feesPaid > 0.msat) @@ -251,7 +254,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { val route = sender.expectMsgType[Router.RouteResponse].routes.head val routes = Seq(InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta, feeOverride_opt = Some(RelayFees.zero))) - val (offer, result) = sendOfferPayment(f, alice, carol, amount, routes) + val (offer, result) = sendOfferPayment(alice, carol, amount, routes) val payment = verifyPaymentSuccess(offer, amount, result) assert(payment.parts.length == 1) assert(payment.parts.head.feesPaid == 0.msat) @@ -270,7 +273,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta), InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta), ) - val (offer, result) = sendOfferPayment(f, alice, carol, amount, routes, maxAttempts = 3) + val (offer, result) = sendOfferPayment(alice, carol, amount, routes, maxAttempts = 3) val payment = verifyPaymentSuccess(offer, amount, result) assert(payment.parts.length == 2) assert(payment.parts.forall(_.feesPaid > 0.msat)) @@ -289,7 +292,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta, feeOverride_opt = Some(RelayFees.zero)), InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta, feeOverride_opt = Some(RelayFees.zero)), ) - val (offer, result) = sendOfferPayment(f, alice, carol, amount, routes, maxAttempts = 3) + val (offer, result) = sendOfferPayment(alice, carol, amount, routes, maxAttempts = 3) val payment = verifyPaymentSuccess(offer, amount, result) assert(payment.parts.length == 2) assert(payment.parts.forall(_.feesPaid == 0.msat)) @@ -307,13 +310,13 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { // We make a first set of payments to ensure channels have less than 50 000 sat on Bob's side. Seq(50_000_000 msat, 50_000_000 msat).foreach(amount => { - val (offer, result) = sendPrivateOfferPayment(f, alice, carol, amount, routes) + val (offer, result) = sendPrivateOfferPayment(alice, carol, amount, routes) verifyPaymentSuccess(offer, amount, result) }) // None of the channels between Bob and Carol have enough balance for the payment: Alice needs to split it. val amount = 50_000_000 msat - val (offer, result) = sendPrivateOfferPayment(f, alice, carol, amount, routes, maxAttempts = 3) + val (offer, result) = sendPrivateOfferPayment(alice, carol, amount, routes, maxAttempts = 3) val payment = verifyPaymentSuccess(offer, amount, result) assert(payment.parts.length > 1) } @@ -337,7 +340,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { val routes = Seq(InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta)) val amount1 = 150_000_000 msat - val (offer, result) = sendPrivateOfferPayment(f, alice, carol, amount1, routes, maxAttempts = 3) + val (offer, result) = sendPrivateOfferPayment(alice, carol, amount1, routes, maxAttempts = 3) val payment = verifyPaymentSuccess(offer, amount1, result) assert(payment.parts.length > 1) } @@ -354,7 +357,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { InvoiceRequestActor.Route(route.hops :+ ChannelHop.dummy(carol.nodeId, 150 msat, 0, CltvExpiryDelta(50)), maxFinalExpiryDelta), InvoiceRequestActor.Route(route.hops ++ Seq(ChannelHop.dummy(carol.nodeId, 50 msat, 0, CltvExpiryDelta(20)), ChannelHop.dummy(carol.nodeId, 100 msat, 0, CltvExpiryDelta(30))), maxFinalExpiryDelta), ) - val (offer, result) = sendOfferPayment(f, alice, carol, amount, routes) + val (offer, result) = sendOfferPayment(alice, carol, amount, routes) val payment = verifyPaymentSuccess(offer, amount, result) assert(payment.parts.length == 2) assert(payment.parts.forall(_.feesPaid > 0.msat)) @@ -372,7 +375,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { InvoiceRequestActor.Route(route.hops :+ ChannelHop.dummy(carol.nodeId, 150 msat, 0, CltvExpiryDelta(50)), maxFinalExpiryDelta, feeOverride_opt = Some(RelayFees.zero)), InvoiceRequestActor.Route(route.hops ++ Seq(ChannelHop.dummy(carol.nodeId, 50 msat, 0, CltvExpiryDelta(20)), ChannelHop.dummy(carol.nodeId, 100 msat, 0, CltvExpiryDelta(30))), maxFinalExpiryDelta, feeOverride_opt = Some(RelayFees.zero)), ) - val (offer, result) = sendOfferPayment(f, alice, carol, amount, routes) + val (offer, result) = sendOfferPayment(alice, carol, amount, routes) val payment = verifyPaymentSuccess(offer, amount, result) assert(payment.parts.length == 2) assert(payment.parts.forall(_.feesPaid == 0.msat)) @@ -388,7 +391,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { val route = sender.expectMsgType[Router.RouteResponse].routes.head val routes = Seq(InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta)) - val (offer, result) = sendPrivateOfferPayment(f, alice, carol, amount, routes) + val (offer, result) = sendPrivateOfferPayment(alice, carol, amount, routes) val payment = verifyPaymentSuccess(offer, amount, result) assert(payment.parts.forall(_.feesPaid > 0.msat)) } @@ -403,7 +406,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { val route = sender.expectMsgType[Router.RouteResponse].routes.head val routes = Seq(InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta, feeOverride_opt = Some(RelayFees.zero))) - val (offer, result) = sendPrivateOfferPayment(f, alice, carol, amount, routes) + val (offer, result) = sendPrivateOfferPayment(alice, carol, amount, routes) val payment = verifyPaymentSuccess(offer, amount, result) assert(payment.parts.forall(_.feesPaid == 0.msat)) } @@ -413,7 +416,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { val amount = 75_000_000 msat val routes = Seq(InvoiceRequestActor.Route(Nil, maxFinalExpiryDelta)) - val (offer, result) = sendOfferPayment(f, alice, bob, amount, routes) + val (offer, result) = sendOfferPayment(alice, bob, amount, routes) val payment = verifyPaymentSuccess(offer, amount, result) assert(payment.parts.length == 1) } @@ -423,7 +426,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { val amount = 250_000_000 msat val routes = Seq(InvoiceRequestActor.Route(Seq(ChannelHop.dummy(bob.nodeId, 10 msat, 25, CltvExpiryDelta(24)), ChannelHop.dummy(bob.nodeId, 5 msat, 10, CltvExpiryDelta(36))), maxFinalExpiryDelta)) - val (offer, result) = sendOfferPayment(f, alice, bob, amount, routes) + val (offer, result) = sendOfferPayment(alice, bob, amount, routes) val payment = verifyPaymentSuccess(offer, amount, result) assert(payment.parts.length == 1) assert(payment.parts.forall(_.feesPaid > 0.msat)) @@ -434,7 +437,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { val amount = 250_000_000 msat val routes = Seq(InvoiceRequestActor.Route(Seq(ChannelHop.dummy(bob.nodeId, 10 msat, 25, CltvExpiryDelta(24)), ChannelHop.dummy(bob.nodeId, 5 msat, 10, CltvExpiryDelta(36))), maxFinalExpiryDelta, feeOverride_opt = Some(RelayFees.zero))) - val (offer, result) = sendOfferPayment(f, alice, bob, amount, routes) + val (offer, result) = sendOfferPayment(alice, bob, amount, routes) val payment = verifyPaymentSuccess(offer, amount, result) assert(payment.parts.length == 1) assert(payment.parts.forall(_.feesPaid == 0.msat)) @@ -450,7 +453,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { val route = sender.expectMsgType[Router.RouteResponse].routes.head val routes = Seq(InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta)) - val (offer, result) = sendOfferPayment(f, bob, carol, amount, routes) + val (offer, result) = sendOfferPayment(bob, carol, amount, routes) val payment = verifyPaymentSuccess(offer, amount, result) assert(payment.parts.length == 1) } @@ -478,7 +481,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { // None of the channels have enough balance for the payment: it must be split. val amount = 150_000_000 msat - val (offer, result) = sendOfferPayment(f, bob, carol, amount, routes, maxAttempts = 3) + val (offer, result) = sendOfferPayment(bob, carol, amount, routes, maxAttempts = 3) val payment = verifyPaymentSuccess(offer, amount, result) assert(payment.parts.length > 1) } @@ -493,7 +496,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { val route = sender.expectMsgType[Router.RouteResponse].routes.head val routes = Seq(InvoiceRequestActor.Route(route.hops :+ ChannelHop.dummy(carol.nodeId, 25 msat, 250, CltvExpiryDelta(75)), maxFinalExpiryDelta)) - val (offer, result) = sendOfferPayment(f, bob, carol, amount, routes) + val (offer, result) = sendOfferPayment(bob, carol, amount, routes) val payment = verifyPaymentSuccess(offer, amount, result) assert(payment.parts.length == 1) } @@ -520,7 +523,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { { val routes = Seq(InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta)) val amount1 = 100_000_000 msat - val (offer, result) = sendOfferPayment(f, alice, carol, amount1, routes) + val (offer, result) = sendOfferPayment(alice, carol, amount1, routes) val payment = verifyPaymentSuccess(offer, amount1, result) assert(payment.parts.length == 1) } @@ -536,7 +539,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { { val routes = Seq(InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta)) val amount2 = 200_000_000 msat - val (offer, result) = sendOfferPayment(f, alice, carol, amount2, routes, maxAttempts = 3) + val (offer, result) = sendOfferPayment(alice, carol, amount2, routes, maxAttempts = 3) val payment = verifyPaymentSuccess(offer, amount2, result) assert(payment.parts.length > 1) } @@ -565,7 +568,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { // Carol receives a payment that requires using MPP. val routes = Seq(InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta)) val amount = 300_000_000 msat - val (offer, result) = sendOfferPayment(f, alice, carol, amount, routes, maxAttempts = 3) + val (offer, result) = sendOfferPayment(alice, carol, amount, routes, maxAttempts = 3) val payment = verifyPaymentSuccess(offer, amount, result) assert(payment.parts.length > 1) } @@ -593,7 +596,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { // Carol receives a payment that requires using MPP. val routes = Seq(InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta)) val amount = 200_000_000 msat - val (offer, result) = sendOfferPayment(f, alice, carol, amount, routes, maxAttempts = 3) + val (offer, result) = sendOfferPayment(alice, carol, amount, routes, maxAttempts = 3) val payment = verifyPaymentSuccess(offer, amount, result) assert(payment.parts.length > 1) } @@ -622,7 +625,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { }) // Bob now doesn't have enough funds to relay the payment. val routes = Seq(InvoiceRequestActor.Route(route.hops, maxFinalExpiryDelta)) - val (_, result) = sendOfferPayment(f, alice, carol, 75_000_000 msat, routes) + val (_, result) = sendOfferPayment(alice, carol, 75_000_000 msat, routes) verifyBlindedFailure(result, bob.nodeId) } @@ -634,7 +637,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { val route = sender.expectMsgType[Router.RouteResponse].routes.head val routes = Seq(InvoiceRequestActor.Route(route.hops, CltvExpiryDelta(-500))) - val (_, result) = sendOfferPayment(f, alice, carol, 25_000_000 msat, routes) + val (_, result) = sendOfferPayment(alice, carol, 25_000_000 msat, routes) verifyBlindedFailure(result, bob.nodeId) } @@ -756,16 +759,14 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { test("basic offer") { f => import f._ - implicit val timeout: Timeout = 10 seconds - val amount = 20_000_000 msat - val offer = Await.result(carol.eclairImpl.createOffer(description_opt = Some("test offer"), amount_opt = Some(amount), expireInSeconds_opt = None, issuer_opt = None, blindedPathsFirstNodeId_opt = None), 10 seconds) + val offer = createOffer(carol, description_opt = Some("test offer"), amount_opt = Some(amount), issuer_opt = None, blindedPathsFirstNodeId_opt = None) assert(offer.nodeId == Some(carol.nodeId)) assert(offer.description == Some("test offer")) assert(offer.amount == Some(amount)) - val payment = Await.result(alice.eclairImpl.payOfferBlocking(offer, amount, 1), 10 seconds) + val payment = payOffer(alice, offer, amount) assert(payment.isInstanceOf[PaymentSent]) assert(payment.asInstanceOf[PaymentSent].feesPaid > 0.msat) } @@ -773,17 +774,15 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { test("offer without node id (dummy hops only)") { f => import f._ - implicit val timeout: Timeout = 10 seconds - val amount = 20_000_000 msat - val offer = Await.result(carol.eclairImpl.createOffer(description_opt = Some("test offer"), amount_opt = Some(amount), expireInSeconds_opt = None, issuer_opt = None, blindedPathsFirstNodeId_opt = Some(carol.nodeId)), 10 seconds) + val offer = createOffer(carol, description_opt = Some("test offer"), amount_opt = Some(amount), issuer_opt = None, blindedPathsFirstNodeId_opt = Some(carol.nodeId)) assert(offer.nodeId == None) assert(offer.contactInfos.head.asInstanceOf[BlindedPath].route.firstNodeId == EncodedNodeId.WithPublicKey.Plain(carol.nodeId)) assert(offer.description == Some("test offer")) assert(offer.amount == Some(amount)) - val payment = Await.result(alice.eclairImpl.payOfferBlocking(offer, amount, 1), 10 seconds) + val payment = payOffer(alice, offer, amount) assert(payment.isInstanceOf[PaymentSent]) assert(payment.asInstanceOf[PaymentSent].feesPaid > 0.msat) } @@ -791,17 +790,15 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { test("offer without node id (real and dummy blinded hops)") { f => import f._ - implicit val timeout: Timeout = 10 seconds - val amount = 20_000_000 msat - val offer = Await.result(carol.eclairImpl.createOffer(description_opt = Some("test offer"), amount_opt = Some(amount), expireInSeconds_opt = None, issuer_opt = None, blindedPathsFirstNodeId_opt = Some(bob.nodeId)), 10 seconds) + val offer = createOffer(carol, description_opt = Some("test offer"), amount_opt = Some(amount), issuer_opt = None, blindedPathsFirstNodeId_opt = Some(bob.nodeId)) assert(offer.nodeId == None) assert(offer.contactInfos.head.asInstanceOf[BlindedPath].route.firstNodeId == EncodedNodeId.WithPublicKey.Plain(bob.nodeId)) assert(offer.description == Some("test offer")) assert(offer.amount == Some(amount)) - val payment = Await.result(alice.eclairImpl.payOfferBlocking(offer, amount, 1), 10 seconds) + val payment = payOffer(alice, offer, amount) assert(payment.isInstanceOf[PaymentSent]) assert(payment.asInstanceOf[PaymentSent].feesPaid == 0.msat) } @@ -809,17 +806,15 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { test("offer without node id (payer is first node of blinded path)") { f => import f._ - implicit val timeout: Timeout = 10 seconds - val amount = 20_000_000 msat - val offer = Await.result(carol.eclairImpl.createOffer(description_opt = Some("test offer"), amount_opt = Some(amount), expireInSeconds_opt = None, issuer_opt = None, blindedPathsFirstNodeId_opt = Some(alice.nodeId)), 10 seconds) + val offer = createOffer(carol, description_opt = Some("test offer"), amount_opt = Some(amount), issuer_opt = None, blindedPathsFirstNodeId_opt = Some(alice.nodeId)) assert(offer.nodeId == None) assert(offer.contactInfos.head.asInstanceOf[BlindedPath].route.firstNodeId == EncodedNodeId.WithPublicKey.Plain(alice.nodeId)) assert(offer.description == Some("test offer")) assert(offer.amount == Some(amount)) - val payment = Await.result(alice.eclairImpl.payOfferBlocking(offer, amount, 1), 10 seconds) + val payment = payOffer(alice, offer, amount) assert(payment.isInstanceOf[PaymentSent]) assert(payment.asInstanceOf[PaymentSent].feesPaid < 0.msat) }