1
0
mirror of https://github.com/ACINQ/eclair.git synced 2024-11-20 10:39:19 +01:00

Reject received payments where PaymentRequest Expiry Tag time has been exceeded. (#749)

This fixes #748.

Also renamed htlc `expiry` to `cltvExpiry`. This removes confusion between different types of expiries and is how the spec is written.
This commit is contained in:
n1bor 2018-11-15 13:08:27 +00:00 committed by Pierre-Marie Padiou
parent 93464aa7fb
commit 738b4fa8e9
17 changed files with 129 additions and 84 deletions

View File

@ -617,8 +617,8 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
val trimmedHtlcs = Transactions.trimOfferedHtlcs(Satoshi(d.commitments.remoteParams.dustLimitSatoshis), nextRemoteCommit.spec) ++ Transactions.trimReceivedHtlcs(Satoshi(d.commitments.remoteParams.dustLimitSatoshis), nextRemoteCommit.spec)
trimmedHtlcs collect {
case DirectedHtlc(_, u) =>
log.info(s"adding paymentHash=${u.paymentHash} cltvExpiry=${u.expiry} to htlcs db for commitNumber=$nextCommitNumber")
nodeParams.channelsDb.addOrUpdateHtlcInfo(d.channelId, nextCommitNumber, u.paymentHash, u.expiry)
log.info(s"adding paymentHash=${u.paymentHash} cltvExpiry=${u.cltvExpiry} to htlcs db for commitNumber=$nextCommitNumber")
nodeParams.channelsDb.addOrUpdateHtlcInfo(d.channelId, nextCommitNumber, u.paymentHash, u.cltvExpiry)
}
context.system.eventStream.publish(ChannelSignatureSent(self, commitments1))
context.system.eventStream.publish(AvailableBalanceChanged(self, d.channelId, d.shortChannelId, nextRemoteCommit.spec.toRemoteMsat)) // note that remoteCommit.toRemote == toLocal

View File

@ -105,7 +105,7 @@ case class BITCOIN_PARENT_TX_CONFIRMED(childTx: Transaction) extends BitcoinEven
*/
sealed trait Command
final case class CMD_ADD_HTLC(amountMsat: Long, paymentHash: BinaryData, expiry: Long, onion: BinaryData = Sphinx.LAST_PACKET.serialize, upstream_opt: Option[UpdateAddHtlc] = None, commit: Boolean = false, redirected: Boolean = false) extends Command
final case class CMD_ADD_HTLC(amountMsat: Long, paymentHash: BinaryData, cltvExpiry: Long, onion: BinaryData = Sphinx.LAST_PACKET.serialize, upstream_opt: Option[UpdateAddHtlc] = None, commit: Boolean = false, redirected: Boolean = false) extends Command
final case class CMD_FULFILL_HTLC(id: Long, r: BinaryData, commit: Boolean = false) extends Command
final case class CMD_FAIL_HTLC(id: Long, reason: Either[BinaryData, FailureMessage], commit: Boolean = false) extends Command
final case class CMD_FAIL_MALFORMED_HTLC(id: Long, onionHash: BinaryData, failureCode: Int, commit: Boolean = false) extends Command

View File

@ -62,9 +62,9 @@ case class Commitments(localParams: LocalParams, remoteParams: RemoteParams,
def hasNoPendingHtlcs: Boolean = localCommit.spec.htlcs.isEmpty && remoteCommit.spec.htlcs.isEmpty && remoteNextCommitInfo.isRight
def hasTimedoutOutgoingHtlcs(blockheight: Long): Boolean =
localCommit.spec.htlcs.exists(htlc => htlc.direction == OUT && blockheight >= htlc.add.expiry) ||
remoteCommit.spec.htlcs.exists(htlc => htlc.direction == IN && blockheight >= htlc.add.expiry) ||
remoteNextCommitInfo.left.toOption.map(_.nextRemoteCommit.spec.htlcs.exists(htlc => htlc.direction == IN && blockheight >= htlc.add.expiry)).getOrElse(false)
localCommit.spec.htlcs.exists(htlc => htlc.direction == OUT && blockheight >= htlc.add.cltvExpiry) ||
remoteCommit.spec.htlcs.exists(htlc => htlc.direction == IN && blockheight >= htlc.add.cltvExpiry) ||
remoteNextCommitInfo.left.toOption.map(_.nextRemoteCommit.spec.htlcs.exists(htlc => htlc.direction == IN && blockheight >= htlc.add.cltvExpiry)).getOrElse(false)
def addLocalProposal(proposal: UpdateMessage): Commitments = Commitments.addLocalProposal(this, proposal)
@ -102,13 +102,13 @@ object Commitments {
val blockCount = Globals.blockCount.get()
// our counterparty needs a reasonable amount of time to pull the funds from downstream before we can get refunded (see BOLT 2 and BOLT 11 for a calculation and rationale)
val minExpiry = blockCount + Channel.MIN_CLTV_EXPIRY
if (cmd.expiry < minExpiry) {
return Left(ExpiryTooSmall(commitments.channelId, minimum = minExpiry, actual = cmd.expiry, blockCount = blockCount))
if (cmd.cltvExpiry < minExpiry) {
return Left(ExpiryTooSmall(commitments.channelId, minimum = minExpiry, actual = cmd.cltvExpiry, blockCount = blockCount))
}
val maxExpiry = blockCount + Channel.MAX_CLTV_EXPIRY
// we don't want to use too high a refund timeout, because our funds will be locked during that time if the payment is never fulfilled
if (cmd.expiry >= maxExpiry) {
return Left(ExpiryTooBig(commitments.channelId, maximum = maxExpiry, actual = cmd.expiry, blockCount = blockCount))
if (cmd.cltvExpiry >= maxExpiry) {
return Left(ExpiryTooBig(commitments.channelId, maximum = maxExpiry, actual = cmd.cltvExpiry, blockCount = blockCount))
}
if (cmd.amountMsat < commitments.remoteParams.htlcMinimumMsat) {
@ -116,7 +116,7 @@ object Commitments {
}
// let's compute the current commitment *as seen by them* with this change taken into account
val add = UpdateAddHtlc(commitments.channelId, commitments.localNextHtlcId, cmd.amountMsat, cmd.paymentHash, cmd.expiry, cmd.onion)
val add = UpdateAddHtlc(commitments.channelId, commitments.localNextHtlcId, cmd.amountMsat, cmd.paymentHash, cmd.cltvExpiry, cmd.onion)
// we increment the local htlc index and add an entry to the origins map
val commitments1 = addLocalProposal(commitments, add).copy(localNextHtlcId = commitments.localNextHtlcId + 1, originChannels = commitments.originChannels + (add.id -> origin))
// we need to base the next current commitment on the last sig we sent, even if we didn't yet receive their revocation
@ -540,17 +540,17 @@ object Commitments {
| toLocal: ${commitments.localCommit.spec.toLocalMsat}
| toRemote: ${commitments.localCommit.spec.toRemoteMsat}
| htlcs:
|${commitments.localCommit.spec.htlcs.map(h => s" ${h.direction} ${h.add.id} ${h.add.expiry}").mkString("\n")}
|${commitments.localCommit.spec.htlcs.map(h => s" ${h.direction} ${h.add.id} ${h.add.cltvExpiry}").mkString("\n")}
|remotecommit:
| toLocal: ${commitments.remoteCommit.spec.toLocalMsat}
| toRemote: ${commitments.remoteCommit.spec.toRemoteMsat}
| htlcs:
|${commitments.remoteCommit.spec.htlcs.map(h => s" ${h.direction} ${h.add.id} ${h.add.expiry}").mkString("\n")}
|${commitments.remoteCommit.spec.htlcs.map(h => s" ${h.direction} ${h.add.id} ${h.add.cltvExpiry}").mkString("\n")}
|next remotecommit:
| toLocal: ${commitments.remoteNextCommitInfo.left.toOption.map(_.nextRemoteCommit.spec.toLocalMsat).getOrElse("N/A")}
| toRemote: ${commitments.remoteNextCommitInfo.left.toOption.map(_.nextRemoteCommit.spec.toRemoteMsat).getOrElse("N/A")}
| htlcs:
|${commitments.remoteNextCommitInfo.left.toOption.map(_.nextRemoteCommit.spec.htlcs.map(h => s" ${h.direction} ${h.add.id} ${h.add.expiry}").mkString("\n")).getOrElse("N/A")}""".stripMargin
|${commitments.remoteNextCommitInfo.left.toOption.map(_.nextRemoteCommit.spec.htlcs.map(h => s" ${h.direction} ${h.add.id} ${h.add.cltvExpiry}").mkString("\n")).getOrElse("N/A")}""".stripMargin
}
}

View File

@ -30,23 +30,25 @@ import scala.concurrent.duration._
import scala.util.Try
/**
* Simple payment handler that generates payment requests and fulfills incoming htlcs.
*
* Note that unfulfilled payment requests are kept forever if they don't have an expiry!
*
* Created by PM on 17/06/2016.
*/
class LocalPaymentHandler(nodeParams: NodeParams) extends Actor with ActorLogging {
implicit val ec: ExecutionContext = context.system.dispatcher
import LocalPaymentHandler._
context.system.scheduler.schedule(10 minutes, 10 minutes)(self ! Platform.currentTime / 1000)
implicit val ec: ExecutionContext = context.system.dispatcher
context.system.scheduler.schedule(10 minutes, 10 minutes)(self ! PurgeExpiredRequests)
override def receive: Receive = run(Map.empty)
def run(hash2preimage: Map[BinaryData, (BinaryData, PaymentRequest)]): Receive = {
def run(hash2preimage: Map[BinaryData, PendingPaymentRequest]): Receive = {
case currentSeconds: Long =>
context.become(run(hash2preimage.collect {
case e@(_, (_, pr)) if pr.expiry.isEmpty => e // requests that don't expire are kept forever
case e@(_, (_, pr)) if pr.timestamp + pr.expiry.get > currentSeconds => e // clean up expired requests
}))
case PurgeExpiredRequests =>
context.become(run(hash2preimage.filterNot { case (_, pr) => hasExpired(pr) }))
case ReceivePayment(amount_opt, desc, expirySeconds_opt, extraHops) =>
Try {
@ -59,7 +61,7 @@ class LocalPaymentHandler(nodeParams: NodeParams) extends Actor with ActorLoggin
val paymentRequest = PaymentRequest(nodeParams.chainHash, amount_opt, paymentHash, nodeParams.privateKey, desc, fallbackAddress = None, expirySeconds = Some(expirySeconds), extraHops = extraHops)
log.debug(s"generated payment request=${PaymentRequest.write(paymentRequest)} from amount=$amount_opt")
sender ! paymentRequest
context.become(run(hash2preimage + (paymentHash -> (paymentPreimage, paymentRequest))))
context.become(run(hash2preimage + (paymentHash -> PendingPaymentRequest(paymentPreimage, paymentRequest))))
} recover { case t => sender ! Status.Failure(t) }
case CheckPayment(paymentHash) =>
@ -69,14 +71,17 @@ class LocalPaymentHandler(nodeParams: NodeParams) extends Actor with ActorLoggin
}
case htlc: UpdateAddHtlc =>
hash2preimage.get(htlc.paymentHash) match {
case Some((paymentPreimage, paymentRequest)) =>
hash2preimage
.get(htlc.paymentHash) // we retrieve the request
.filterNot(hasExpired) // and filter it out if it is expired (it will be purged independently)
match {
case Some(PendingPaymentRequest(paymentPreimage, paymentRequest)) =>
val minFinalExpiry = Globals.blockCount.get() + paymentRequest.minFinalCltvExpiry.getOrElse(Channel.MIN_CLTV_EXPIRY)
// The htlc amount must be equal or greater than the requested amount. A slight overpaying is permitted, however
// it must not be greater than two times the requested amount.
// see https://github.com/lightningnetwork/lightning-rfc/blob/master/04-onion-routing.md#failure-messages
paymentRequest.amount match {
case _ if htlc.expiry < minFinalExpiry =>
case _ if htlc.cltvExpiry < minFinalExpiry =>
sender ! CMD_FAIL_HTLC(htlc.id, Right(FinalExpiryTooSoon), commit = true)
case Some(amount) if MilliSatoshi(htlc.amountMsat) < amount =>
log.warning(s"received payment with amount too small for paymentHash=${htlc.paymentHash} amountMsat=${htlc.amountMsat}")
@ -95,9 +100,25 @@ class LocalPaymentHandler(nodeParams: NodeParams) extends Actor with ActorLoggin
case None =>
sender ! CMD_FAIL_HTLC(htlc.id, Right(UnknownPaymentHash), commit = true)
}
case 'requests =>
// this is just for testing
sender ! hash2preimage
}
}
object LocalPaymentHandler {
def props(nodeParams: NodeParams): Props = Props(new LocalPaymentHandler(nodeParams))
case object PurgeExpiredRequests
case class PendingPaymentRequest(preimage: BinaryData, paymentRequest: PaymentRequest)
def hasExpired(pr: PendingPaymentRequest): Boolean = pr.paymentRequest.expiry match {
case Some(expiry) => pr.paymentRequest.timestamp + expiry <= Platform.currentTime / 1000
case None => false // this request will never expire
}
}

View File

@ -200,7 +200,7 @@ object Relayer {
case class FinalPayload(add: UpdateAddHtlc, payload: PerHopPayload) extends NextPayload
case class RelayPayload(add: UpdateAddHtlc, payload: PerHopPayload, nextPacket: Sphinx.Packet) extends NextPayload {
val relayFeeSatoshi = add.amountMsat - payload.amtToForward
val expiryDelta = add.expiry - payload.outgoingCltvValue
val expiryDelta = add.cltvExpiry - payload.outgoingCltvValue
}
// @formatter:on
@ -240,8 +240,8 @@ object Relayer {
finalPayload.payload match {
case PerHopPayload(_, finalAmountToForward, _) if finalAmountToForward > add.amountMsat =>
Left(CMD_FAIL_HTLC(add.id, Right(FinalIncorrectHtlcAmount(add.amountMsat)), commit = true))
case PerHopPayload(_, _, finalOutgoingCltvValue) if finalOutgoingCltvValue != add.expiry =>
Left(CMD_FAIL_HTLC(add.id, Right(FinalIncorrectCltvExpiry(add.expiry)), commit = true))
case PerHopPayload(_, _, finalOutgoingCltvValue) if finalOutgoingCltvValue != add.cltvExpiry =>
Left(CMD_FAIL_HTLC(add.id, Right(FinalIncorrectCltvExpiry(add.cltvExpiry)), commit = true))
case _ =>
Right(add)
}
@ -265,7 +265,7 @@ object Relayer {
case Some(channelUpdate) if payload.amtToForward < channelUpdate.htlcMinimumMsat =>
Left(CMD_FAIL_HTLC(add.id, Right(AmountBelowMinimum(add.amountMsat, channelUpdate)), commit = true))
case Some(channelUpdate) if relayPayload.expiryDelta != channelUpdate.cltvExpiryDelta =>
Left(CMD_FAIL_HTLC(add.id, Right(IncorrectCltvExpiry(add.expiry, channelUpdate)), commit = true))
Left(CMD_FAIL_HTLC(add.id, Right(IncorrectCltvExpiry(add.cltvExpiry, channelUpdate)), commit = true))
case Some(channelUpdate) if relayPayload.relayFeeSatoshi < nodeFee(channelUpdate.feeBaseMsat, channelUpdate.feeProportionalMillionths, payload.amtToForward) =>
Left(CMD_FAIL_HTLC(add.id, Right(FeeInsufficient(add.amountMsat, channelUpdate)), commit = true))
case Some(channelUpdate) =>

View File

@ -200,7 +200,7 @@ object Transactions {
val htlcOfferedOutputs = trimOfferedHtlcs(localDustLimit, spec)
.map(htlc => TxOut(MilliSatoshi(htlc.add.amountMsat), pay2wsh(htlcOffered(localHtlcPubkey, remoteHtlcPubkey, localRevocationPubkey, ripemd160(htlc.add.paymentHash)))))
val htlcReceivedOutputs = trimReceivedHtlcs(localDustLimit, spec)
.map(htlc => TxOut(MilliSatoshi(htlc.add.amountMsat), pay2wsh(htlcReceived(localHtlcPubkey, remoteHtlcPubkey, localRevocationPubkey, ripemd160(htlc.add.paymentHash), htlc.add.expiry))))
.map(htlc => TxOut(MilliSatoshi(htlc.add.amountMsat), pay2wsh(htlcReceived(localHtlcPubkey, remoteHtlcPubkey, localRevocationPubkey, ripemd160(htlc.add.paymentHash), htlc.add.cltvExpiry))))
val txnumber = obscuredCommitTxNumber(commitTxNumber, localIsFunder, localPaymentBasePoint, remotePaymentBasePoint)
val (sequence, locktime) = encodeTxNumber(txnumber)
@ -227,12 +227,12 @@ object Transactions {
version = 2,
txIn = TxIn(input.outPoint, Array.emptyByteArray, 0x00000000L) :: Nil,
txOut = TxOut(amount, pay2wsh(toLocalDelayed(localRevocationPubkey, toLocalDelay, localDelayedPaymentPubkey))) :: Nil,
lockTime = htlc.expiry))
lockTime = htlc.cltvExpiry))
}
def makeHtlcSuccessTx(commitTx: Transaction, outputsAlreadyUsed: Set[Int], localDustLimit: Satoshi, localRevocationPubkey: PublicKey, toLocalDelay: Int, localDelayedPaymentPubkey: PublicKey, localHtlcPubkey: PublicKey, remoteHtlcPubkey: PublicKey, feeratePerKw: Long, htlc: UpdateAddHtlc): HtlcSuccessTx = {
val fee = weight2fee(feeratePerKw, htlcSuccessWeight)
val redeemScript = htlcReceived(localHtlcPubkey, remoteHtlcPubkey, localRevocationPubkey, ripemd160(htlc.paymentHash), htlc.expiry)
val redeemScript = htlcReceived(localHtlcPubkey, remoteHtlcPubkey, localRevocationPubkey, ripemd160(htlc.paymentHash), htlc.cltvExpiry)
val pubkeyScript = write(pay2wsh(redeemScript))
val outputIndex = findPubKeyScriptIndex(commitTx, pubkeyScript, outputsAlreadyUsed, amount_opt = Some(Satoshi(htlc.amountMsat / 1000)))
val amount = MilliSatoshi(htlc.amountMsat) - fee
@ -286,7 +286,7 @@ object Transactions {
}
def makeClaimHtlcTimeoutTx(commitTx: Transaction, outputsAlreadyUsed: Set[Int], localDustLimit: Satoshi, localHtlcPubkey: PublicKey, remoteHtlcPubkey: PublicKey, remoteRevocationPubkey: PublicKey, localFinalScriptPubKey: BinaryData, htlc: UpdateAddHtlc, feeratePerKw: Long): ClaimHtlcTimeoutTx = {
val redeemScript = htlcReceived(remoteHtlcPubkey, localHtlcPubkey, remoteRevocationPubkey, ripemd160(htlc.paymentHash), htlc.expiry)
val redeemScript = htlcReceived(remoteHtlcPubkey, localHtlcPubkey, remoteRevocationPubkey, ripemd160(htlc.paymentHash), htlc.cltvExpiry)
val pubkeyScript = write(pay2wsh(redeemScript))
val outputIndex = findPubKeyScriptIndex(commitTx, pubkeyScript, outputsAlreadyUsed, amount_opt = Some(Satoshi(htlc.amountMsat / 1000)))
val input = InputInfo(OutPoint(commitTx, outputIndex), commitTx.txOut(outputIndex), write(redeemScript))
@ -296,7 +296,7 @@ object Transactions {
version = 2,
txIn = TxIn(input.outPoint, Array.emptyByteArray, 0x00000000L) :: Nil,
txOut = TxOut(Satoshi(0), localFinalScriptPubKey) :: Nil,
lockTime = htlc.expiry)
lockTime = htlc.cltvExpiry)
val weight = Transactions.addSigs(ClaimHtlcTimeoutTx(input, tx), BinaryData("00" * 73)).tx.weight()
val fee = weight2fee(feeratePerKw, weight)

View File

@ -112,7 +112,7 @@ case class UpdateAddHtlc(channelId: BinaryData,
id: Long,
amountMsat: Long,
paymentHash: BinaryData,
expiry: Long,
cltvExpiry: Long,
onionRoutingPacket: BinaryData) extends HtlcMessage with UpdateMessage with HasChannelId
case class UpdateFulfillHtlc(channelId: BinaryData,

View File

@ -90,8 +90,8 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
val sender = TestProbe()
val h = BinaryData("42" * 32)
val originHtlc = UpdateAddHtlc(channelId = "42" * 32, id = 5656, amountMsat = 50000000, expiry = 400144, paymentHash = h, onionRoutingPacket = "00" * 1254)
val cmd = CMD_ADD_HTLC(originHtlc.amountMsat - 10000, h, originHtlc.expiry - 7, upstream_opt = Some(originHtlc))
val originHtlc = UpdateAddHtlc(channelId = "42" * 32, id = 5656, amountMsat = 50000000, cltvExpiry = 400144, paymentHash = h, onionRoutingPacket = "00" * 1254)
val cmd = CMD_ADD_HTLC(originHtlc.amountMsat - 10000, h, originHtlc.cltvExpiry - 7, upstream_opt = Some(originHtlc))
sender.send(alice, cmd)
sender.expectMsg("ok")
val htlc = alice2bob.expectMsgType[UpdateAddHtlc]
@ -108,7 +108,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
import f._
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
val sender = TestProbe()
val add = CMD_ADD_HTLC(500000000, "11" * 42, expiry = 400144)
val add = CMD_ADD_HTLC(500000000, "11" * 42, cltvExpiry = 400144)
sender.send(alice, add)
val error = InvalidPaymentHash(channelId(alice))
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), add.paymentHash, error, Local(Some(sender.ref)), Some(initialState.channelUpdate), Some(add))))
@ -121,7 +121,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
val currentBlockCount = Globals.blockCount.get
val expiryTooSmall = currentBlockCount + 3
val add = CMD_ADD_HTLC(500000000, "11" * 32, expiry = expiryTooSmall)
val add = CMD_ADD_HTLC(500000000, "11" * 32, cltvExpiry = expiryTooSmall)
sender.send(alice, add)
val error = ExpiryTooSmall(channelId(alice), currentBlockCount + Channel.MIN_CLTV_EXPIRY, expiryTooSmall, currentBlockCount)
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), add.paymentHash, error, Local(Some(sender.ref)), Some(initialState.channelUpdate), Some(add))))
@ -134,7 +134,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
val currentBlockCount = Globals.blockCount.get
val expiryTooBig = currentBlockCount + Channel.MAX_CLTV_EXPIRY + 1
val add = CMD_ADD_HTLC(500000000, "11" * 32, expiry = expiryTooBig)
val add = CMD_ADD_HTLC(500000000, "11" * 32, cltvExpiry = expiryTooBig)
sender.send(alice, add)
val error = ExpiryTooBig(channelId(alice), maximum = currentBlockCount + Channel.MAX_CLTV_EXPIRY, actual = expiryTooBig, blockCount = currentBlockCount)
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), add.paymentHash, error, Local(Some(sender.ref)), Some(initialState.channelUpdate), Some(add))))
@ -257,7 +257,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].localShutdown.isDefined && !alice.stateData.asInstanceOf[DATA_NORMAL].remoteShutdown.isDefined)
// actual test starts here
val add = CMD_ADD_HTLC(500000000, "11" * 32, expiry = 400144)
val add = CMD_ADD_HTLC(500000000, "11" * 32, cltvExpiry = 400144)
sender.send(alice, add)
val error = NoMoreHtlcsClosingInProgress(channelId(alice))
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), add.paymentHash, error, Local(Some(sender.ref)), Some(initialState.channelUpdate), Some(add))))
@ -269,14 +269,14 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val sender = TestProbe()
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
// let's make alice send an htlc
val add1 = CMD_ADD_HTLC(500000000, "11" * 32, expiry = 400144)
val add1 = CMD_ADD_HTLC(500000000, "11" * 32, cltvExpiry = 400144)
sender.send(alice, add1)
sender.expectMsg("ok")
// at the same time bob initiates a closing
sender.send(bob, CMD_CLOSE(None))
sender.expectMsg("ok")
// this command will be received by alice right after having received the shutdown
val add2 = CMD_ADD_HTLC(100000000, "22" * 32, expiry = 300000)
val add2 = CMD_ADD_HTLC(100000000, "22" * 32, cltvExpiry = 300000)
// messages cross
alice2bob.expectMsgType[UpdateAddHtlc]
alice2bob.forward(bob)
@ -328,7 +328,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
test("recv UpdateAddHtlc (value too small)") { f =>
import f._
val tx = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.localCommit.publishableTxs.commitTx.tx
val htlc = UpdateAddHtlc("00" * 32, 0, 150, BinaryData("42" * 32), expiry = 400144, defaultOnion)
val htlc = UpdateAddHtlc("00" * 32, 0, 150, BinaryData("42" * 32), cltvExpiry = 400144, defaultOnion)
alice2bob.forward(bob, htlc)
val error = bob2alice.expectMsgType[Error]
assert(new String(error.data) === HtlcValueTooSmall(channelId(bob), minimum = 1000, actual = 150).getMessage)

View File

@ -97,7 +97,7 @@ class ShutdownStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
test("recv CMD_ADD_HTLC") { f =>
import f._
val sender = TestProbe()
val add = CMD_ADD_HTLC(500000000, "11" * 32, expiry = 300000)
val add = CMD_ADD_HTLC(500000000, "11" * 32, cltvExpiry = 300000)
sender.send(alice, add)
val error = ChannelUnavailable(channelId(alice))
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), add.paymentHash, error, Local(Some(sender.ref)), None, Some(add))))

View File

@ -69,7 +69,7 @@ class NegotiatingStateSpec extends TestkitBaseClass with StateTestsHelperMethods
import f._
alice2bob.expectMsgType[ClosingSigned]
val sender = TestProbe()
val add = CMD_ADD_HTLC(500000000, "11" * 32, expiry = 300000)
val add = CMD_ADD_HTLC(500000000, "11" * 32, cltvExpiry = 300000)
sender.send(alice, add)
val error = ChannelUnavailable(channelId(alice))
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), add.paymentHash, error, Local(Some(sender.ref)), None, Some(add))))

View File

@ -110,7 +110,7 @@ class ClosingStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
// actual test starts here
val sender = TestProbe()
val add = CMD_ADD_HTLC(500000000, "11" * 32, expiry = 300000)
val add = CMD_ADD_HTLC(500000000, "11" * 32, cltvExpiry = 300000)
sender.send(alice, add)
val error = ChannelUnavailable(channelId(alice))
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), add.paymentHash, error, Local(Some(sender.ref)), None, Some(add))))

View File

@ -99,7 +99,7 @@ class HtlcGenerationSpec extends FunSuite {
val (add, _) = buildCommand(finalAmountMsat, finalExpiry, paymentHash, hops)
assert(add.amountMsat > finalAmountMsat)
assert(add.expiry === finalExpiry + channelUpdate_de.cltvExpiryDelta + channelUpdate_cd.cltvExpiryDelta + channelUpdate_bc.cltvExpiryDelta)
assert(add.cltvExpiry === finalExpiry + channelUpdate_de.cltvExpiryDelta + channelUpdate_cd.cltvExpiryDelta + channelUpdate_bc.cltvExpiryDelta)
assert(add.paymentHash === paymentHash)
assert(add.onion.length === Sphinx.PacketLength)
@ -133,7 +133,7 @@ class HtlcGenerationSpec extends FunSuite {
val (add, _) = buildCommand(finalAmountMsat, finalExpiry, paymentHash, hops.take(1))
assert(add.amountMsat === finalAmountMsat)
assert(add.expiry === finalExpiry)
assert(add.cltvExpiry === finalExpiry)
assert(add.paymentHash === paymentHash)
assert(add.onion.size === Sphinx.PacketLength)

View File

@ -18,13 +18,14 @@ package fr.acinq.eclair.payment
import akka.actor.Status.Failure
import akka.actor.{ActorSystem, Status}
import akka.testkit.{TestKit, TestProbe}
import fr.acinq.bitcoin.{MilliSatoshi, Satoshi}
import akka.testkit.{TestActorRef, TestKit, TestProbe}
import fr.acinq.bitcoin.{BinaryData, MilliSatoshi, Satoshi}
import fr.acinq.eclair.TestConstants.Alice
import fr.acinq.eclair.channel.{CMD_FAIL_HTLC, CMD_FULFILL_HTLC}
import fr.acinq.eclair.payment.LocalPaymentHandler.PendingPaymentRequest
import fr.acinq.eclair.payment.PaymentLifecycle.{CheckPayment, ReceivePayment}
import fr.acinq.eclair.payment.PaymentRequest.ExtraHop
import fr.acinq.eclair.wire.{FinalExpiryTooSoon, UpdateAddHtlc}
import fr.acinq.eclair.wire.{FinalExpiryTooSoon, UnknownPaymentHash, UpdateAddHtlc}
import fr.acinq.eclair.{Globals, ShortChannelId, randomKey}
import org.scalatest.FunSuiteLike
@ -38,7 +39,7 @@ class PaymentHandlerSpec extends TestKit(ActorSystem("test")) with FunSuiteLike
test("LocalPaymentHandler should reply with a fulfill/fail, emit a PaymentReceived and adds payment in DB") {
val nodeParams = Alice.nodeParams
val handler = system.actorOf(LocalPaymentHandler.props(nodeParams))
val handler = TestActorRef[LocalPaymentHandler](LocalPaymentHandler.props(nodeParams))
val sender = TestProbe()
val eventListener = TestProbe()
system.eventStream.subscribe(eventListener.ref, classOf[PaymentReceived])
@ -55,7 +56,7 @@ class PaymentHandlerSpec extends TestKit(ActorSystem("test")) with FunSuiteLike
sender.send(handler, add)
sender.expectMsgType[CMD_FULFILL_HTLC]
val paymentRelayed = eventListener.expectMsgType[PaymentReceived]
assert(paymentRelayed.copy(timestamp = 0) === PaymentReceived(amountMsat,add.paymentHash, add.channelId, timestamp = 0))
assert(paymentRelayed.copy(timestamp = 0) === PaymentReceived(amountMsat, add.paymentHash, add.channelId, timestamp = 0))
sender.send(handler, CheckPayment(pr.paymentHash))
assert(sender.expectMsgType[Boolean] === true)
}
@ -69,7 +70,7 @@ class PaymentHandlerSpec extends TestKit(ActorSystem("test")) with FunSuiteLike
sender.send(handler, add)
sender.expectMsgType[CMD_FULFILL_HTLC]
val paymentRelayed = eventListener.expectMsgType[PaymentReceived]
assert(paymentRelayed.copy(timestamp = 0) === PaymentReceived(amountMsat,add.paymentHash, add.channelId, timestamp = 0))
assert(paymentRelayed.copy(timestamp = 0) === PaymentReceived(amountMsat, add.paymentHash, add.channelId, timestamp = 0))
sender.send(handler, CheckPayment(pr.paymentHash))
assert(sender.expectMsgType[Boolean] === true)
}
@ -79,13 +80,36 @@ class PaymentHandlerSpec extends TestKit(ActorSystem("test")) with FunSuiteLike
val pr = sender.expectMsgType[PaymentRequest]
sender.send(handler, CheckPayment(pr.paymentHash))
assert(sender.expectMsgType[Boolean] === false)
val add = UpdateAddHtlc("11" * 32, 0, amountMsat.amount, pr.paymentHash, expiry = Globals.blockCount.get() + 3, "")
val add = UpdateAddHtlc("11" * 32, 0, amountMsat.amount, pr.paymentHash, cltvExpiry = Globals.blockCount.get() + 3, "")
sender.send(handler, add)
assert(sender.expectMsgType[CMD_FAIL_HTLC].reason == Right(FinalExpiryTooSoon))
eventListener.expectNoMsg(300 milliseconds)
sender.send(handler, CheckPayment(pr.paymentHash))
assert(sender.expectMsgType[Boolean] === false)
}
{
sender.send(handler, ReceivePayment(Some(amountMsat), "timeout expired", Some(1L)))
//allow request to timeout
Thread.sleep(1001)
val pr = sender.expectMsgType[PaymentRequest]
sender.send(handler, CheckPayment(pr.paymentHash))
assert(sender.expectMsgType[Boolean] === false)
val add = UpdateAddHtlc("11" * 32, 0, amountMsat.amount, pr.paymentHash, expiry, "")
sender.send(handler, add)
assert(sender.expectMsgType[CMD_FAIL_HTLC].reason == Right(UnknownPaymentHash))
// We chose UnknownPaymentHash on purpose. So if you have expired by 1 second or 1 hour you get the same error message.
eventListener.expectNoMsg(300 milliseconds)
sender.send(handler, CheckPayment(pr.paymentHash))
assert(sender.expectMsgType[Boolean] === false)
// make sure that the request is indeed pruned
sender.send(handler, 'requests)
sender.expectMsgType[Map[BinaryData, PendingPaymentRequest]].contains(pr.paymentHash)
sender.send(handler, LocalPaymentHandler.PurgeExpiredRequests)
awaitCond({
sender.send(handler, 'requests)
sender.expectMsgType[Map[BinaryData, PendingPaymentRequest]].contains(pr.paymentHash) == false
})
}
}
test("Payment request generation should fail when the amount asked in not valid") {

View File

@ -66,7 +66,7 @@ class RelayerSpec extends TestkitBaseClass {
// we use this to build a valid onion
val (cmd, _) = buildCommand(finalAmountMsat, finalExpiry, paymentHash, hops)
// and then manually build an htlc
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.expiry, cmd.onion)
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.cltvExpiry, cmd.onion)
relayer ! LocalChannelUpdate(null, channelId_bc, channelUpdate_bc.shortChannelId, c, None, channelUpdate_bc, makeCommitments(channelId_bc))
sender.send(relayer, ForwardAdd(add_ab))
@ -86,7 +86,7 @@ class RelayerSpec extends TestkitBaseClass {
// we use this to build a valid onion
val (cmd, _) = buildCommand(finalAmountMsat, finalExpiry, paymentHash, hops)
// and then manually build an htlc
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.expiry, cmd.onion)
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.cltvExpiry, cmd.onion)
sender.send(relayer, ForwardAdd(add_ab))
@ -106,7 +106,7 @@ class RelayerSpec extends TestkitBaseClass {
// we use this to build a valid onion
val (cmd, _) = buildCommand(finalAmountMsat, finalExpiry, paymentHash, hops)
// and then manually build an htlc
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.expiry, cmd.onion)
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.cltvExpiry, cmd.onion)
relayer ! LocalChannelUpdate(null, channelId_bc, channelUpdate_bc.shortChannelId, c, None, channelUpdate_bc, makeCommitments(channelId_bc))
sender.send(relayer, ForwardAdd(add_ab))
@ -132,7 +132,7 @@ class RelayerSpec extends TestkitBaseClass {
// check that payments are sent properly
val (cmd, _) = buildCommand(finalAmountMsat, finalExpiry, paymentHash, hops)
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.expiry, cmd.onion)
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.cltvExpiry, cmd.onion)
relayer ! LocalChannelUpdate(null, channelId_bc, channelUpdate_bc.shortChannelId, c, None, channelUpdate_bc, makeCommitments(channelId_bc))
sender.send(relayer, ForwardAdd(add_ab))
@ -148,7 +148,7 @@ class RelayerSpec extends TestkitBaseClass {
relayer ! LocalChannelDown(sender.ref, channelId = channelId_bc, shortChannelId = channelUpdate_bc.shortChannelId, remoteNodeId = TestConstants.Bob.nodeParams.nodeId)
val (cmd1, _) = buildCommand(finalAmountMsat, finalExpiry, "02" * 32, hops)
val add_ab1 = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd1.amountMsat, cmd1.paymentHash, cmd1.expiry, cmd1.onion)
val add_ab1 = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd1.amountMsat, cmd1.paymentHash, cmd1.cltvExpiry, cmd1.onion)
sender.send(relayer, ForwardAdd(add_ab))
val fail = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]].message
@ -166,7 +166,7 @@ class RelayerSpec extends TestkitBaseClass {
// we use this to build a valid onion
val (cmd, _) = buildCommand(finalAmountMsat, finalExpiry, paymentHash, hops)
// and then manually build an htlc
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.expiry, cmd.onion)
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.cltvExpiry, cmd.onion)
val channelUpdate_bc_disabled = channelUpdate_bc.copy(channelFlags = Announcements.makeChannelFlags(Announcements.isNode1(channelUpdate_bc.channelFlags), enable = false))
relayer ! LocalChannelUpdate(null, channelId_bc, channelUpdate_bc.shortChannelId, c, None, channelUpdate_bc_disabled, makeCommitments(channelId_bc))
@ -187,7 +187,7 @@ class RelayerSpec extends TestkitBaseClass {
// we use this to build a valid onion
val (cmd, _) = buildCommand(finalAmountMsat, finalExpiry, paymentHash, hops)
// and then manually build an htlc
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.expiry, "00" * Sphinx.PacketLength)
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.cltvExpiry, "00" * Sphinx.PacketLength)
relayer ! LocalChannelUpdate(null, channelId_bc, channelUpdate_bc.shortChannelId, c, None, channelUpdate_bc, makeCommitments(channelId_bc))
sender.send(relayer, ForwardAdd(add_ab))
@ -207,7 +207,7 @@ class RelayerSpec extends TestkitBaseClass {
// we use this to build a valid onion
val (cmd, _) = buildCommand(channelUpdate_bc.htlcMinimumMsat - 1, finalExpiry, paymentHash, hops.map(hop => hop.copy(lastUpdate = hop.lastUpdate.copy(feeBaseMsat = 0, feeProportionalMillionths = 0))))
// and then manually build an htlc
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.expiry, cmd.onion)
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.cltvExpiry, cmd.onion)
relayer ! LocalChannelUpdate(null, channelId_bc, channelUpdate_bc.shortChannelId, c, None, channelUpdate_bc, makeCommitments(channelId_bc))
sender.send(relayer, ForwardAdd(add_ab))
@ -227,14 +227,14 @@ class RelayerSpec extends TestkitBaseClass {
val hops1 = hops.updated(1, hops(1).copy(lastUpdate = hops(1).lastUpdate.copy(cltvExpiryDelta = 0)))
val (cmd, _) = buildCommand(finalAmountMsat, finalExpiry, paymentHash, hops1)
// and then manually build an htlc
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.expiry, cmd.onion)
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.cltvExpiry, cmd.onion)
relayer ! LocalChannelUpdate(null, channelId_bc, channelUpdate_bc.shortChannelId, c, None, channelUpdate_bc, makeCommitments(channelId_bc))
sender.send(relayer, ForwardAdd(add_ab))
val fail = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]].message
assert(fail.id === add_ab.id)
assert(fail.reason == Right(IncorrectCltvExpiry(cmd.expiry, channelUpdate_bc)))
assert(fail.reason == Right(IncorrectCltvExpiry(cmd.cltvExpiry, channelUpdate_bc)))
register.expectNoMsg(100 millis)
paymentHandler.expectNoMsg(100 millis)
@ -247,7 +247,7 @@ class RelayerSpec extends TestkitBaseClass {
val hops1 = hops.updated(1, hops(1).copy(lastUpdate = hops(1).lastUpdate.copy(feeBaseMsat = hops(1).lastUpdate.feeBaseMsat / 2)))
val (cmd, _) = buildCommand(finalAmountMsat, finalExpiry, paymentHash, hops1)
// and then manually build an htlc
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.expiry, cmd.onion)
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.cltvExpiry, cmd.onion)
relayer ! LocalChannelUpdate(null, channelId_bc, channelUpdate_bc.shortChannelId, c, None, channelUpdate_bc, makeCommitments(channelId_bc))
sender.send(relayer, ForwardAdd(add_ab))
@ -268,7 +268,7 @@ class RelayerSpec extends TestkitBaseClass {
val hops1 = hops.head :: Nil
val (cmd, _) = buildCommand(finalAmountMsat, finalExpiry, paymentHash, hops1)
// and then manually build an htlc with a wrong expiry
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat - 1, cmd.paymentHash, cmd.expiry, cmd.onion)
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat - 1, cmd.paymentHash, cmd.cltvExpiry, cmd.onion)
relayer ! LocalChannelUpdate(null, channelId_bc, channelUpdate_bc.shortChannelId, c, None, channelUpdate_bc, makeCommitments(channelId_bc))
sender.send(relayer, ForwardAdd(add_ab))
@ -289,14 +289,14 @@ class RelayerSpec extends TestkitBaseClass {
val hops1 = hops.head :: Nil
val (cmd, _) = buildCommand(finalAmountMsat, finalExpiry, paymentHash, hops1)
// and then manually build an htlc with a wrong expiry
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.expiry - 1, cmd.onion)
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.cltvExpiry - 1, cmd.onion)
relayer ! LocalChannelUpdate(null, channelId_bc, channelUpdate_bc.shortChannelId, c, None, channelUpdate_bc, makeCommitments(channelId_bc))
sender.send(relayer, ForwardAdd(add_ab))
val fail = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]].message
assert(fail.id === add_ab.id)
assert(fail.reason == Right(FinalIncorrectCltvExpiry(add_ab.expiry)))
assert(fail.reason == Right(FinalIncorrectCltvExpiry(add_ab.cltvExpiry)))
register.expectNoMsg(100 millis)
paymentHandler.expectNoMsg(100 millis)
@ -340,7 +340,7 @@ class RelayerSpec extends TestkitBaseClass {
system.eventStream.subscribe(eventListener.ref, classOf[PaymentEvent])
// we build a fake htlc for the downstream channel
val add_bc = UpdateAddHtlc(channelId = channelId_bc, id = 72, amountMsat = 10000000L, paymentHash = "00" * 32, expiry = 4200, onionRoutingPacket = "")
val add_bc = UpdateAddHtlc(channelId = channelId_bc, id = 72, amountMsat = 10000000L, paymentHash = "00" * 32, cltvExpiry = 4200, onionRoutingPacket = "")
val fulfill_ba = UpdateFulfillHtlc(channelId = channelId_bc, id = 42, paymentPreimage = "00" * 32)
val origin = Relayed(channelId_ab, 150, 11000000L, 10000000L)
sender.send(relayer, ForwardFulfill(fulfill_ba, origin, add_bc))
@ -358,7 +358,7 @@ class RelayerSpec extends TestkitBaseClass {
val sender = TestProbe()
// we build a fake htlc for the downstream channel
val add_bc = UpdateAddHtlc(channelId = channelId_bc, id = 72, amountMsat = 10000000L, paymentHash = "00" * 32, expiry = 4200, onionRoutingPacket = "")
val add_bc = UpdateAddHtlc(channelId = channelId_bc, id = 72, amountMsat = 10000000L, paymentHash = "00" * 32, cltvExpiry = 4200, onionRoutingPacket = "")
val fail_ba = UpdateFailHtlc(channelId = channelId_bc, id = 42, reason = Sphinx.createErrorPacket(BinaryData("01" * 32), TemporaryChannelFailure(channelUpdate_cd)))
val origin = Relayed(channelId_ab, 150, 11000000L, 10000000L)
sender.send(relayer, ForwardFail(fail_ba, origin, add_bc))

View File

@ -160,7 +160,7 @@ class TestVectorsSpec extends FunSuite with Logging {
)
val htlcScripts = htlcs.map(htlc => htlc.direction match {
case OUT => Scripts.htlcOffered(Local.payment_privkey.publicKey, Remote.payment_privkey.publicKey, Local.revocation_pubkey, Crypto.ripemd160(htlc.add.paymentHash))
case IN => Scripts.htlcReceived(Local.payment_privkey.publicKey, Remote.payment_privkey.publicKey, Local.revocation_pubkey, Crypto.ripemd160(htlc.add.paymentHash), htlc.add.expiry)
case IN => Scripts.htlcReceived(Local.payment_privkey.publicKey, Remote.payment_privkey.publicKey, Local.revocation_pubkey, Crypto.ripemd160(htlc.add.paymentHash), htlc.add.cltvExpiry)
})
def dir2string(dir: Direction) = dir match {
@ -171,7 +171,7 @@ class TestVectorsSpec extends FunSuite with Logging {
for (i <- 0 until htlcs.length) {
logger.info(s"htlc $i direction: ${dir2string(htlcs(i).direction)}")
logger.info(s"htlc $i amount_msat: ${htlcs(i).add.amountMsat}")
logger.info(s"htlc $i expiry: ${htlcs(i).add.expiry}")
logger.info(s"htlc $i expiry: ${htlcs(i).add.cltvExpiry}")
logger.info(s"htlc $i payment_preimage: ${paymentPreimages(i)}")
}

View File

@ -124,8 +124,8 @@ class TransactionsSpec extends FunSuite with Logging {
// HtlcPenaltyTx
// first we create a fake commitTx tx, containing only the output that will be spent by the ClaimHtlcSuccessTx
val paymentPreimage = BinaryData("42" * 32)
val htlc = UpdateAddHtlc("00" * 32, 0, Satoshi(20000).amount * 1000, sha256(paymentPreimage), expiry = 400144, BinaryData.empty)
val redeemScript = htlcReceived(localHtlcPriv.publicKey, remoteHtlcPriv.publicKey, localRevocationPriv.publicKey, ripemd160(htlc.paymentHash), htlc.expiry)
val htlc = UpdateAddHtlc("00" * 32, 0, Satoshi(20000).amount * 1000, sha256(paymentPreimage), cltvExpiry = 400144, BinaryData.empty)
val redeemScript = htlcReceived(localHtlcPriv.publicKey, remoteHtlcPriv.publicKey, localRevocationPriv.publicKey, ripemd160(htlc.paymentHash), htlc.cltvExpiry)
val pubKeyScript = write(pay2wsh(redeemScript))
val commitTx = Transaction(version = 0, txIn = Nil, txOut = TxOut(Satoshi(htlc.amountMsat / 1000), pubKeyScript) :: Nil, lockTime = 0)
val htlcPenaltyTx = makeHtlcPenaltyTx(commitTx, outputsAlreadyUsed = Set.empty, Script.write(redeemScript), localDustLimit, finalPubKeyScript, feeratePerKw)
@ -139,7 +139,7 @@ class TransactionsSpec extends FunSuite with Logging {
// ClaimHtlcSuccessTx
// first we create a fake commitTx tx, containing only the output that will be spent by the ClaimHtlcSuccessTx
val paymentPreimage = BinaryData("42" * 32)
val htlc = UpdateAddHtlc("00" * 32, 0, Satoshi(20000).amount * 1000, sha256(paymentPreimage), expiry = 400144, BinaryData.empty)
val htlc = UpdateAddHtlc("00" * 32, 0, Satoshi(20000).amount * 1000, sha256(paymentPreimage), cltvExpiry = 400144, BinaryData.empty)
val pubKeyScript = write(pay2wsh(htlcOffered(localHtlcPriv.publicKey, remoteHtlcPriv.publicKey, localRevocationPriv.publicKey, ripemd160(htlc.paymentHash))))
val commitTx = Transaction(version = 0, txIn = Nil, txOut = TxOut(Satoshi(htlc.amountMsat / 1000), pubKeyScript) :: Nil, lockTime = 0)
val claimHtlcSuccessTx = makeClaimHtlcSuccessTx(commitTx, outputsAlreadyUsed = Set.empty, localDustLimit, remoteHtlcPriv.publicKey, localHtlcPriv.publicKey, localRevocationPriv.publicKey, finalPubKeyScript, htlc, feeratePerKw)
@ -153,8 +153,8 @@ class TransactionsSpec extends FunSuite with Logging {
// ClaimHtlcTimeoutTx
// first we create a fake commitTx tx, containing only the output that will be spent by the ClaimHtlcSuccessTx
val paymentPreimage = BinaryData("42" * 32)
val htlc = UpdateAddHtlc("00" * 32, 0, Satoshi(20000).amount * 1000, sha256(paymentPreimage), expiry = 400144, BinaryData.empty)
val pubKeyScript = write(pay2wsh(htlcReceived(localHtlcPriv.publicKey, remoteHtlcPriv.publicKey, localRevocationPriv.publicKey, ripemd160(htlc.paymentHash), htlc.expiry)))
val htlc = UpdateAddHtlc("00" * 32, 0, Satoshi(20000).amount * 1000, sha256(paymentPreimage), cltvExpiry = 400144, BinaryData.empty)
val pubKeyScript = write(pay2wsh(htlcReceived(localHtlcPriv.publicKey, remoteHtlcPriv.publicKey, localRevocationPriv.publicKey, ripemd160(htlc.paymentHash), htlc.cltvExpiry)))
val commitTx = Transaction(version = 0, txIn = Nil, txOut = TxOut(Satoshi(htlc.amountMsat / 1000), pubKeyScript) :: Nil, lockTime = 0)
val claimClaimHtlcTimeoutTx = makeClaimHtlcTimeoutTx(commitTx, outputsAlreadyUsed = Set.empty, localDustLimit, remoteHtlcPriv.publicKey, localHtlcPriv.publicKey, localRevocationPriv.publicKey, finalPubKeyScript, htlc, feeratePerKw)
// we use dummy signatures to compute the weight
@ -304,7 +304,7 @@ class TransactionsSpec extends FunSuite with Logging {
{
// remote spends received HTLC output with revocation key
val script = Script.write(Scripts.htlcReceived(localHtlcPriv.publicKey, remoteHtlcPriv.publicKey, localRevocationPriv.publicKey, Crypto.ripemd160(htlc2.paymentHash), htlc2.expiry))
val script = Script.write(Scripts.htlcReceived(localHtlcPriv.publicKey, remoteHtlcPriv.publicKey, localRevocationPriv.publicKey, Crypto.ripemd160(htlc2.paymentHash), htlc2.cltvExpiry))
val htlcPenaltyTx = makeHtlcPenaltyTx(commitTx.tx, outputsAlreadyUsed = Set.empty, script, localDustLimit, finalPubKeyScript, feeratePerKw)
val sig = sign(htlcPenaltyTx, localRevocationPriv)
val signed = addSigs(htlcPenaltyTx, sig, localRevocationPriv.publicKey)

View File

@ -104,7 +104,7 @@ class ChannelCodecsSpec extends FunSuite {
channelId = randomBytes(32),
id = Random.nextInt(Int.MaxValue),
amountMsat = Random.nextInt(Int.MaxValue),
expiry = Random.nextInt(Int.MaxValue),
cltvExpiry = Random.nextInt(Int.MaxValue),
paymentHash = randomBytes(32),
onionRoutingPacket = randomBytes(Sphinx.PacketLength))
val htlc1 = DirectedHtlc(direction = IN, add = add)
@ -118,14 +118,14 @@ class ChannelCodecsSpec extends FunSuite {
channelId = randomBytes(32),
id = Random.nextInt(Int.MaxValue),
amountMsat = Random.nextInt(Int.MaxValue),
expiry = Random.nextInt(Int.MaxValue),
cltvExpiry = Random.nextInt(Int.MaxValue),
paymentHash = randomBytes(32),
onionRoutingPacket = randomBytes(Sphinx.PacketLength))
val add2 = UpdateAddHtlc(
channelId = randomBytes(32),
id = Random.nextInt(Int.MaxValue),
amountMsat = Random.nextInt(Int.MaxValue),
expiry = Random.nextInt(Int.MaxValue),
cltvExpiry = Random.nextInt(Int.MaxValue),
paymentHash = randomBytes(32),
onionRoutingPacket = randomBytes(Sphinx.PacketLength))
val htlc1 = DirectedHtlc(direction = IN, add = add1)