1
0
Fork 0
mirror of https://github.com/ACINQ/eclair.git synced 2025-03-13 19:37:35 +01:00

simplify pay-to-open message

We remove the `feeThresholdSatoshis`/`feeProportionalMillionths` fields
and only keep the `feeAmount` field, which is renamed to `payToOpenFee`.

Fee aggregation is easier, we just sum the `payToOpenFee`.
This commit is contained in:
pm47 2021-01-25 15:19:42 +01:00
parent 189e67ed67
commit f678dfa7b8
No known key found for this signature in database
GPG key ID: E434ED292E85643A
5 changed files with 63 additions and 48 deletions

View file

@ -168,7 +168,7 @@ class MultiPartHandler(nodeParams: NodeParams, register: ActorRef, db: IncomingP
// and we do as if we had received only that pay-to-open request (this is what will be written to db)
val parts1 = parts.collect { case h: MultiPartPaymentFSM.HtlcPart => h } :+ MultiPartPaymentFSM.PayToOpenPart(parts.head.totalAmount, summarizedPayToOpenRequest, payToOpenParts.head.peer)
log.info(s"received pay-to-open payment for amount=${summarizedPayToOpenRequest.amountMsat}")
if (summarizedPayToOpenRequest.feeSatoshis == 0.sat) {
if (summarizedPayToOpenRequest.payToOpenFee == 0.sat) {
// we always say ok when fee is zero, without asking the user
ctx.self ! DoFulfill(preimage, s)
} else {
@ -183,7 +183,7 @@ class MultiPartHandler(nodeParams: NodeParams, register: ActorRef, db: IncomingP
.foreach {
case true =>
// user said yes
log.info(s"user said ok to pay-to-open request for amount=${summarizedPayToOpenRequest.amountMsat} fee=${summarizedPayToOpenRequest.feeSatoshis}")
log.info(s"user said ok to pay-to-open request for amount=${summarizedPayToOpenRequest.amountMsat} fee=${summarizedPayToOpenRequest.payToOpenFee}")
ctx.self ! DoFulfill(preimage, MultiPartPaymentFSM.MultiPartPaymentSucceeded(paymentHash, parts1))
case false =>
// user said no or didn't answer
@ -223,7 +223,7 @@ class MultiPartHandler(nodeParams: NodeParams, register: ActorRef, db: IncomingP
log.info("fulfilling payment for amount={}", parts.map(_.amount).sum)
val received = PaymentReceived(paymentHash, parts.map {
case p: MultiPartPaymentFSM.HtlcPart => PaymentReceived.PartialPayment(p.amount, p.htlc.channelId)
case p: MultiPartPaymentFSM.PayToOpenPart => PaymentReceived.PartialPayment(p.amount - p.payToOpen.feeSatoshis, ByteVector32.Zeroes)
case p: MultiPartPaymentFSM.PayToOpenPart => PaymentReceived.PartialPayment(p.amount - p.payToOpen.payToOpenFee, ByteVector32.Zeroes)
})
// The first thing we do is store the payment. This allows us to reconcile pending HTLCs after a restart.
db.receiveIncomingPayment(paymentHash, received.amount, received.timestamp)

View file

@ -16,13 +16,13 @@
package fr.acinq.eclair.wire
import fr.acinq.bitcoin.Satoshi
import fr.acinq.eclair.wire.CommonCodecs._
import fr.acinq.eclair.wire.Monitoring.{Metrics, Tags}
import fr.acinq.eclair.{Features, KamonExt, wire}
import scodec.bits.{BitVector, ByteVector}
import scodec.codecs._
import scodec.{Attempt, Codec}
import scodec.bits._
/**
@ -300,8 +300,8 @@ object LightningMessageCodecs {
("pushMsat" | millisatoshi) ::
("feeSatoshis" | satoshi) ::
("paymentHash" | bytes32) ::
("feeThresholdSatoshis" | satoshi) ::
("feeProportionalMillionths" | uint32) ::
("feeThresholdSatoshis" | satoshi.unit(Satoshi(0))) ::
("feeProportionalMillionths" | uint32.unit(0)) ::
("expireAt" | uint32) ::
("htlc_opt" | optional(bool(8), updateAddHtlcCodec))).as[PayToOpenRequest]

View file

@ -300,10 +300,8 @@ case class GossipTimestampFilter(chainHash: ByteVector32,
case class PayToOpenRequest(chainHash: ByteVector32,
fundingSatoshis: Satoshi,
amountMsat: MilliSatoshi,
feeSatoshis: Satoshi,
payToOpenFee: Satoshi,
paymentHash: ByteVector32,
feeThresholdSatoshis: Satoshi,
feeProportionalMillionths: Long,
expireAt: Long,
htlc_opt: Option[UpdateAddHtlc]
) extends LightningMessage with HasChainHash {
@ -319,16 +317,6 @@ case class PayToOpenRequest(chainHash: ByteVector32,
object PayToOpenRequest {
def computeFee(amount: MilliSatoshi, feeThresholdSatoshis: Satoshi, feeProportionalMillionths: Long) = {
if (amount.truncateToSatoshi < feeThresholdSatoshis) {
// for tiny amounts there is no fee
0.sat
} else {
// NB: this fee is proportional, which allow us to sum them in case of multi-parts payments
amount.truncateToSatoshi * feeProportionalMillionths / 1000000
}
}
def computeFunding(amount: MilliSatoshi, fee: Satoshi) = 100000.sat + (amount * 1.5D).truncateToSatoshi + fee
/**
@ -341,15 +329,13 @@ object PayToOpenRequest {
require(requests.nonEmpty, "there needs to be at least one pay-to-open request")
require(requests.map(_.chainHash).toSet.size == 1, "all pay-to-open chain hash must be equal")
require(requests.map(_.paymentHash).toSet.size == 1, "all pay-to-open payment hash must be equal")
require(requests.map(_.feeThresholdSatoshis).toSet.size == 1, "all pay-to-open fee rates must be equal")
require(requests.map(_.feeProportionalMillionths).toSet.size == 1, "all pay-to-open fee rates must be equal")
val chainHash = requests.head.chainHash
val paymentHash = requests.head.paymentHash
val totalAmount = requests.map(_.amountMsat).sum
val feeAmount = PayToOpenRequest.computeFee(totalAmount, requests.head.feeThresholdSatoshis, requests.head.feeProportionalMillionths)
val fundingAmount = PayToOpenRequest.computeFunding(totalAmount, feeAmount)
val payToOpenFees = requests.map(_.payToOpenFee).sum
val fundingAmount = PayToOpenRequest.computeFunding(totalAmount, payToOpenFees)
val expireAt = requests.map(_.expireAt).min // the aggregate request expires when the first of the underlying request expires
PayToOpenRequest(chainHash, fundingAmount, totalAmount, feeAmount, paymentHash, requests.head.feeThresholdSatoshis, requests.head.feeProportionalMillionths, expireAt, None)
PayToOpenRequest(chainHash, fundingAmount, totalAmount, payToOpenFees, paymentHash, expireAt, None)
}
}

View file

@ -18,7 +18,7 @@ package fr.acinq.eclair.payment
import akka.actor.Status.Failure
import akka.testkit.{TestActorRef, TestProbe}
import fr.acinq.bitcoin.{Block, ByteVector32, Crypto}
import fr.acinq.bitcoin.{Block, ByteVector32, Crypto, Satoshi}
import fr.acinq.eclair.FeatureSupport.Optional
import fr.acinq.eclair.Features._
import fr.acinq.eclair.TestConstants.Alice
@ -505,7 +505,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
f.sender.send(handler, ReceivePayment(Some(1000 msat), "1 fast coffee"))
val pr = f.sender.expectMsgType[PaymentRequest]
val p1 = PayToOpenRequest(Block.RegtestGenesisBlock.hash, 1 mbtc, 1000 msat, 0 sat, pr.paymentHash, payToOpenFeeThresholdSatoshis, payToOpenFeeProportionalMillionths, secondsFromNow(60), None)
val p1 = PayToOpenRequest(Block.RegtestGenesisBlock.hash, 1 mbtc, 1000 msat, 0 sat, pr.paymentHash, secondsFromNow(60), None)
f.sender.send(handler, p1)
val r1 = f.sender.expectMsgType[PayToOpenResponse]
@ -522,7 +522,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
})
// Extraneous pay-to-opens will be ignored
val pExtra = PayToOpenRequest(Block.RegtestGenesisBlock.hash, 1 mbtc, 200 msat, 0 sat, pr.paymentHash, payToOpenFeeThresholdSatoshis, payToOpenFeeProportionalMillionths, secondsFromNow(60), None)
val pExtra = PayToOpenRequest(Block.RegtestGenesisBlock.hash, 1 mbtc, 200 msat, 0 sat, pr.paymentHash, secondsFromNow(60), None)
f.sender.send(handler, MultiPartPaymentFSM.ExtraPaymentReceived(pr.paymentHash, PayToOpenPart(1000 msat, pExtra, f.sender.ref), None))
f.sender.expectNoMsg()
@ -537,13 +537,13 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
system.eventStream.subscribe(eventListener.ref, classOf[PayToOpenRequestEvent])
val amount = 20000000 msat
val fee = PayToOpenRequest.computeFee(amount, 10000 sat, 1000)
val fee = 1000 sat
val funding = PayToOpenRequest.computeFunding(amount, fee)
f.sender.send(handler, ReceivePayment(Some(20000000 msat), "1 fast coffee"))
val pr = f.sender.expectMsgType[PaymentRequest]
val p1 = PayToOpenRequest(Block.RegtestGenesisBlock.hash, funding, amount, fee, pr.paymentHash, payToOpenFeeThresholdSatoshis, payToOpenFeeProportionalMillionths, secondsFromNow(60), None)
val p1 = PayToOpenRequest(Block.RegtestGenesisBlock.hash, funding, amount, fee, pr.paymentHash, secondsFromNow(60), None)
f.sender.send(handler, p1)
val e1 = eventListener.expectMsgType[PayToOpenRequestEvent]
@ -575,13 +575,13 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
system.eventStream.subscribe(eventListener.ref, classOf[PayToOpenRequestEvent])
val amount = 20000000 msat
val fee = PayToOpenRequest.computeFee(amount, 10000 sat, 1000)
val fee = 1200 sat
val funding = PayToOpenRequest.computeFunding(amount, fee)
f.sender.send(handler, ReceivePayment(Some(20000000 msat), "1 fast coffee"))
val pr = f.sender.expectMsgType[PaymentRequest]
val p1 = PayToOpenRequest(Block.RegtestGenesisBlock.hash, funding, amount, fee, pr.paymentHash, payToOpenFeeThresholdSatoshis, payToOpenFeeProportionalMillionths, secondsFromNow(60), None)
val p1 = PayToOpenRequest(Block.RegtestGenesisBlock.hash, funding, amount, fee, pr.paymentHash, secondsFromNow(60), None)
f.sender.send(handler, p1)
val e1 = eventListener.expectMsgType[PayToOpenRequestEvent]
@ -603,13 +603,13 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
system.eventStream.subscribe(eventListener.ref, classOf[PayToOpenRequestEvent])
val amount = 20000000 msat
val fee = PayToOpenRequest.computeFee(amount, 10000 sat, 1000)
val fee = 1500 sat
val funding = PayToOpenRequest.computeFunding(amount, fee)
f.sender.send(handler, ReceivePayment(Some(20000000 msat), "1 fast coffee"))
val pr = f.sender.expectMsgType[PaymentRequest]
val p1 = PayToOpenRequest(Block.RegtestGenesisBlock.hash, funding, amount, fee, pr.paymentHash, payToOpenFeeThresholdSatoshis, payToOpenFeeProportionalMillionths, secondsFromNow(2), None)
val p1 = PayToOpenRequest(Block.RegtestGenesisBlock.hash, funding, amount, fee, pr.paymentHash, secondsFromNow(2), None)
f.sender.send(handler, p1)
val e1 = eventListener.expectMsgType[PayToOpenRequestEvent]
@ -640,34 +640,34 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
f.sender.send(handler, IncomingPacket.FinalPacket(add2, Onion.createMultiPartPayload(add2.amountMsat, 100000000 msat, add2.cltvExpiry, pr.paymentSecret.get)))
val amount1 = 20000000 msat
val fee1 = PayToOpenRequest.computeFee(amount1, 10000 sat, 1000)
val fee1 = 1000 sat
val funding1 = PayToOpenRequest.computeFunding(amount1, fee1)
val payload1 = Onion.createMultiPartPayload(amount1, 100000000 msat, CltvExpiry(420000), pr.paymentSecret.get)
val onion1 = buildOnion(Sphinx.PaymentPacket)(nodeParams.nodeId :: Nil, payload1 :: Nil, pr.paymentHash).packet
val htlc1 = UpdateAddHtlc(ByteVector32.Zeroes, 0, payload1.amount, pr.paymentHash, payload1.expiry, onion1)
val p1 = PayToOpenRequest(Block.RegtestGenesisBlock.hash, funding1, amount1, fee1, pr.paymentHash, payToOpenFeeThresholdSatoshis, payToOpenFeeProportionalMillionths, secondsFromNow(45), Some(htlc1))
val p1 = PayToOpenRequest(Block.RegtestGenesisBlock.hash, funding1, amount1, fee1, pr.paymentHash, secondsFromNow(45), Some(htlc1))
f.sender.send(handler, p1)
val amount2 = 20000000 msat
val fee2 = PayToOpenRequest.computeFee(amount2, 10000 sat, 1000)
val fee2 = 1000 sat
val funding2 = PayToOpenRequest.computeFunding(amount2, fee1)
val payload2 = Onion.createMultiPartPayload(amount2, 100000000 msat, CltvExpiry(420000), pr.paymentSecret.get)
val onion2 = buildOnion(Sphinx.PaymentPacket)(nodeParams.nodeId :: Nil, payload2 :: Nil, pr.paymentHash).packet
val htlc2 = UpdateAddHtlc(ByteVector32.Zeroes, 0, payload2.amount, pr.paymentHash, payload2.expiry, onion2)
val p2 = PayToOpenRequest(Block.RegtestGenesisBlock.hash, funding2, amount2, fee2, pr.paymentHash, payToOpenFeeThresholdSatoshis, payToOpenFeeProportionalMillionths, secondsFromNow(50), Some(htlc2))
val p2 = PayToOpenRequest(Block.RegtestGenesisBlock.hash, funding2, amount2, fee2, pr.paymentHash, secondsFromNow(50), Some(htlc2))
f.sender.send(handler, p2)
val amount3 = 10000000 msat
val fee3 = PayToOpenRequest.computeFee(amount3, 10000 sat, 1000)
val fee3 = 10000 sat
val funding3 = PayToOpenRequest.computeFunding(amount1, fee1)
val payload3 = Onion.createMultiPartPayload(amount3, 100000000 msat, CltvExpiry(420000), pr.paymentSecret.get)
val onion3 = buildOnion(Sphinx.PaymentPacket)(nodeParams.nodeId :: Nil, payload3 :: Nil, pr.paymentHash).packet
val htlc3 = UpdateAddHtlc(ByteVector32.Zeroes, 0, payload3.amount, pr.paymentHash, payload3.expiry, onion3)
val p3 = PayToOpenRequest(Block.RegtestGenesisBlock.hash, funding3, amount3, fee3, pr.paymentHash, payToOpenFeeThresholdSatoshis, payToOpenFeeProportionalMillionths, secondsFromNow(60), Some(htlc3))
val p3 = PayToOpenRequest(Block.RegtestGenesisBlock.hash, funding3, amount3, fee3, pr.paymentHash, secondsFromNow(60), Some(htlc3))
f.sender.send(handler, p3)
val payToOpenAmount = amount1 + amount2 + amount3
val payToOpenFee = PayToOpenRequest.computeFee(payToOpenAmount, 10000 sat, 1000)
val payToOpenFee = fee1 + fee2 + fee3
val payToOpenFunding = PayToOpenRequest.computeFunding(payToOpenAmount, payToOpenFee)
val e1 = eventListener.expectMsgType[PayToOpenRequestEvent]
@ -675,10 +675,8 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
chainHash = p1.chainHash,
fundingSatoshis = payToOpenFunding,
amountMsat = payToOpenAmount,
feeSatoshis = payToOpenFee,
payToOpenFee = payToOpenFee,
paymentHash = p1.paymentHash,
feeThresholdSatoshis = payToOpenFeeThresholdSatoshis,
feeProportionalMillionths = payToOpenFeeProportionalMillionths,
expireAt = p1.expireAt,
htlc_opt = None
))
@ -723,6 +721,4 @@ object MultiPartHandlerSpec {
*/
def secondsFromNow(s: Int): Long = (Platform.currentTime.milliseconds + s.seconds).toSeconds
val payToOpenFeeThresholdSatoshis = 10000 sat
val payToOpenFeeProportionalMillionths = 1000
}

View file

@ -431,14 +431,47 @@ class LightningMessageCodecsSpec extends AnyFunSuite {
("chainHash" | bytes32) ::
("fundingSatoshis" | satoshi) ::
("pushMsat" | millisatoshi) ::
("feeSatoshis" | satoshi) ::
("payToOpenFee" | satoshi) ::
("paymentHash" | bytes32)).as[OldPayToOpenRequest]
val p = PayToOpenRequest(randomBytes32, 12 mbtc, 12345 msat, 7 sat, randomBytes32, 10000 sat, 1000, 1234567890L, Some(UpdateAddHtlc(randomBytes32, 42, 12345 msat, randomBytes32, CltvExpiry(420), TestConstants.emptyOnionPacket)))
val p = PayToOpenRequest(randomBytes32, 12 mbtc, 12345 msat, 7 sat, randomBytes32, 1234567890L, Some(UpdateAddHtlc(randomBytes32, 42, 12345 msat, randomBytes32, CltvExpiry(420), TestConstants.emptyOnionPacket)))
val bits = payToOpenRequestCodec.encode(p).require
val DecodeResult(oldp, remainder) = oldPayToOpenRequestCodec.decode(bits).require
assert(oldp === OldPayToOpenRequest(p.chainHash, p.fundingSatoshis, p.amountMsat, p.feeSatoshis, p.paymentHash))
assert(oldp === OldPayToOpenRequest(p.chainHash, p.fundingSatoshis, p.amountMsat, p.payToOpenFee, p.paymentHash))
assert(remainder.nonEmpty)
}
test("non-reg pay-to-open 2") {
// we just need to make sure that old phoenix can decode new pay-to-open requests
case class OldPayToOpenRequest(chainHash: ByteVector32,
fundingSatoshis: Satoshi,
amountMsat: MilliSatoshi,
feeSatoshis: Satoshi,
paymentHash: ByteVector32,
feeThresholdSatoshis: Satoshi,
feeProportionalMillionths: Long,
expireAt: Long,
htlc_opt: Option[UpdateAddHtlc]
)
import fr.acinq.eclair.wire.CommonCodecs._
import scodec.codecs._
val oldPayToOpenRequestCodec: Codec[OldPayToOpenRequest] = (
("chainHash" | bytes32) ::
("fundingSatoshis" | satoshi) ::
("pushMsat" | millisatoshi) ::
("feeSatoshis" | satoshi) ::
("paymentHash" | bytes32) ::
("feeThresholdSatoshis" | satoshi) ::
("feeProportionalMillionths" | uint32) ::
("expireAt" | uint32) ::
("htlc_opt" | optional(bool(8), updateAddHtlcCodec))).as[OldPayToOpenRequest]
val p = OldPayToOpenRequest(randomBytes32, 12 mbtc, 12345 msat, 7 sat, randomBytes32, 10000 sat, 1000, 1234567890L, Some(UpdateAddHtlc(randomBytes32, 42, 12345 msat, randomBytes32, CltvExpiry(420), TestConstants.emptyOnionPacket)))
val bits = oldPayToOpenRequestCodec.encode(p).require
val DecodeResult(newp, remainder) = payToOpenRequestCodec.decode(bits).require
assert(newp === PayToOpenRequest(p.chainHash, p.fundingSatoshis, p.amountMsat, p.feeSatoshis, p.paymentHash, p.expireAt, p.htlc_opt))
assert(remainder.isEmpty)
}
}