Skip to content

Commit

Permalink
Add a txOut field to our InteractiveTxBuilder.Input interface (#2791)
Browse files Browse the repository at this point in the history
* Add a txOut field to our InteractiveTxBuilder.Input interface

This will help us add support for taproot inputs: to create taproot signatures we need all prevouts (and not just prevouts for tx inputs that spend taproot outputs).
  • Loading branch information
sstone committed Dec 4, 2023
1 parent e73c1cf commit f0cb58a
Show file tree
Hide file tree
Showing 4 changed files with 77 additions and 15 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -216,11 +216,13 @@ object InteractiveTxBuilder {
def serialId: UInt64
def outPoint: OutPoint
def sequence: Long
def txOut: TxOut
}
object Input {
/** A local-only input that funds the interactive transaction. */
case class Local(serialId: UInt64, previousTx: Transaction, previousTxOutput: Long, sequence: Long) extends Input with Outgoing {
override val outPoint: OutPoint = OutPoint(previousTx, previousTxOutput.toInt)
override def txOut: TxOut = previousTx.txOut(previousTxOutput.toInt)
}

/**
Expand All @@ -230,7 +232,9 @@ object InteractiveTxBuilder {
case class Remote(serialId: UInt64, outPoint: OutPoint, txOut: TxOut, sequence: Long) extends Input with Incoming

/** The shared input can be added by us or by our peer, depending on who initiated the protocol. */
case class Shared(serialId: UInt64, outPoint: OutPoint, sequence: Long, localAmount: MilliSatoshi, remoteAmount: MilliSatoshi, htlcAmount: MilliSatoshi) extends Input with Incoming with Outgoing
case class Shared(serialId: UInt64, outPoint: OutPoint, publicKeyScript: ByteVector, sequence: Long, localAmount: MilliSatoshi, remoteAmount: MilliSatoshi, htlcAmount: MilliSatoshi) extends Input with Incoming with Outgoing {
override def txOut: TxOut = TxOut((localAmount + remoteAmount + htlcAmount).truncateToSatoshi, publicKeyScript)
}
}

sealed trait Output {
Expand Down Expand Up @@ -282,7 +286,7 @@ object InteractiveTxBuilder {
localInputs: List[Input.Local], remoteInputs: List[Input.Remote],
localOutputs: List[Output.Local], remoteOutputs: List[Output.Remote],
lockTime: Long) {
val localAmountIn: MilliSatoshi = sharedInput_opt.map(_.localAmount).getOrElse(0 msat) + localInputs.map(i => i.previousTx.txOut(i.previousTxOutput.toInt).amount).sum
val localAmountIn: MilliSatoshi = sharedInput_opt.map(_.localAmount).getOrElse(0 msat) + localInputs.map(i => i.txOut.amount).sum
val remoteAmountIn: MilliSatoshi = sharedInput_opt.map(_.remoteAmount).getOrElse(0 msat) + remoteInputs.map(_.txOut.amount).sum
val localAmountOut: MilliSatoshi = sharedOutput.localAmount + localOutputs.map(_.amount).sum
val remoteAmountOut: MilliSatoshi = sharedOutput.remoteAmount + remoteOutputs.map(_.amount).sum
Expand Down Expand Up @@ -489,7 +493,7 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
case None =>
(addInput.sharedInput_opt, fundingParams.sharedInput_opt) match {
case (Some(outPoint), Some(sharedInput)) if outPoint == sharedInput.info.outPoint =>
Input.Shared(addInput.serialId, outPoint, addInput.sequence, purpose.previousLocalBalance, purpose.previousRemoteBalance, purpose.htlcBalance)
Input.Shared(addInput.serialId, outPoint, sharedInput.info.txOut.publicKeyScript, addInput.sequence, purpose.previousLocalBalance, purpose.previousRemoteBalance, purpose.htlcBalance)
case _ =>
return Left(PreviousTxMissing(fundingParams.channelId, addInput.serialId))
}
Expand Down Expand Up @@ -810,7 +814,7 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
// Partially signed PSBT must include spent amounts for all inputs that were signed, and we can "trust" these amounts because they are included
// in the hash that we signed (see BIP143). If our bitcoin node lied about them, then our signatures are invalid.
val actualLocalAmountIn = ourWalletInputs.map(i => kmp2scala(response.psbt.getInput(i).getWitnessUtxo.amount)).sum
val expectedLocalAmountIn = unsignedTx.localInputs.map(i => i.previousTx.txOut(i.previousTxOutput.toInt).amount).sum
val expectedLocalAmountIn = unsignedTx.localInputs.map(i => i.txOut.amount).sum
require(actualLocalAmountIn == expectedLocalAmountIn, s"local spent amount $actualLocalAmountIn does not match what we expect ($expectedLocalAmountIn): bitcoin core may be malicious")
val actualLocalAmountOut = ourWalletOutputs.map(i => partiallySignedTx.txOut(i).amount).sum
val expectedLocalAmountOut = unsignedTx.localOutputs.map {
Expand Down Expand Up @@ -884,9 +888,9 @@ object InteractiveTxSigningSession {
private def shouldSignFirst(isInitiator: Boolean, channelParams: ChannelParams, tx: SharedTransaction): Boolean = {
val sharedAmountIn = tx.sharedInput_opt.map(i => i.localAmount + i.remoteAmount + i.htlcAmount).getOrElse(0 msat).truncateToSatoshi
val (localAmountIn, remoteAmountIn) = if (isInitiator) {
(sharedAmountIn + tx.localInputs.map(i => i.previousTx.txOut(i.previousTxOutput.toInt).amount).sum, tx.remoteInputs.map(i => i.txOut.amount).sum)
(sharedAmountIn + tx.localInputs.map(i => i.txOut.amount).sum, tx.remoteInputs.map(i => i.txOut.amount).sum)
} else {
(tx.localInputs.map(i => i.previousTx.txOut(i.previousTxOutput.toInt).amount).sum, sharedAmountIn + tx.remoteInputs.map(i => i.txOut.amount).sum)
(tx.localInputs.map(i => i.txOut.amount).sum, sharedAmountIn + tx.remoteInputs.map(i => i.txOut.amount).sum)
}
if (localAmountIn == remoteAmountIn) {
// When both peers contribute the same amount, the peer with the lowest pubkey must transmit its `tx_signatures` first.
Expand Down Expand Up @@ -928,7 +932,7 @@ object InteractiveTxSigningSession {
}
val previousOutputs = {
val sharedOutput = fundingParams.sharedInput_opt.map(sharedInput => sharedInput.info.outPoint -> sharedInput.info.txOut).toMap
val localOutputs = txWithSigs.tx.localInputs.map(i => i.outPoint -> i.previousTx.txOut(i.previousTxOutput.toInt)).toMap
val localOutputs = txWithSigs.tx.localInputs.map(i => i.outPoint -> i.txOut).toMap
val remoteOutputs = txWithSigs.tx.remoteInputs.map(i => i.outPoint -> i.txOut).toMap
sharedOutput ++ localOutputs ++ remoteOutputs
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -157,7 +157,7 @@ private class InteractiveTxFunder(replyTo: ActorRef[InteractiveTxFunder.Response
// force us to add wallet inputs. The caller may manually decrease the output amounts if it wants to actually
// contribute to the RBF attempt.
if (fundingParams.isInitiator) {
val sharedInput = fundingParams.sharedInput_opt.toSeq.map(sharedInput => Input.Shared(UInt64(0), sharedInput.info.outPoint, 0xfffffffdL, purpose.previousLocalBalance, purpose.previousRemoteBalance, purpose.htlcBalance))
val sharedInput = fundingParams.sharedInput_opt.toSeq.map(sharedInput => Input.Shared(UInt64(0), sharedInput.info.outPoint, sharedInput.info.txOut.publicKeyScript, 0xfffffffdL, purpose.previousLocalBalance, purpose.previousRemoteBalance, purpose.htlcBalance))
val sharedOutput = Output.Shared(UInt64(0), fundingPubkeyScript, purpose.previousLocalBalance + fundingParams.localContribution, purpose.previousRemoteBalance + fundingParams.remoteContribution, purpose.htlcBalance)
val nonChangeOutputs = fundingParams.localOutputs.map(txOut => Output.Local.NonChange(UInt64(0), txOut.amount, txOut.publicKeyScript))
val fundingContributions = sortFundingContributions(fundingParams, sharedInput ++ previousWalletInputs, sharedOutput +: nonChangeOutputs)
Expand Down Expand Up @@ -292,7 +292,7 @@ private class InteractiveTxFunder(replyTo: ActorRef[InteractiveTxFunder.Response
case None => fundingParams.sharedInput_opt match {
case Some(sharedInput) if sharedInput.info.outPoint == txIn.outPoint =>
// We don't need to validate the shared input, it comes from a valid lightning channel.
Future.successful(Right(Input.Shared(UInt64(0), sharedInput.info.outPoint, txIn.sequence, purpose.previousLocalBalance, purpose.previousRemoteBalance, purpose.htlcBalance)))
Future.successful(Right(Input.Shared(UInt64(0), sharedInput.info.outPoint, sharedInput.info.txOut.publicKeyScript, txIn.sequence, purpose.previousLocalBalance, purpose.previousRemoteBalance, purpose.htlcBalance)))
case _ =>
for {
previousTx <- wallet.getTransaction(txIn.outPoint.txid)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@ import fr.acinq.eclair.channel.fund.InteractiveTxSigningSession.UnsignedLocalCom
import fr.acinq.eclair.channel.fund.{InteractiveTxBuilder, InteractiveTxSigningSession}
import fr.acinq.eclair.crypto.ShaChain
import fr.acinq.eclair.MilliSatoshiLong
import fr.acinq.eclair.channel.fund.InteractiveTxBuilder.{FullySignedSharedTransaction, PartiallySignedSharedTransaction}
import fr.acinq.eclair.transactions.Transactions._
import fr.acinq.eclair.transactions.{CommitmentSpec, DirectedHtlc, IncomingHtlc, OutgoingHtlc}
import fr.acinq.eclair.wire.protocol.CommonCodecs._
Expand Down Expand Up @@ -252,6 +253,7 @@ private[channel] object ChannelCodecs4 {
private val sharedInteractiveTxInputWithoutHtlcsCodec: Codec[InteractiveTxBuilder.Input.Shared] = (
("serialId" | uint64) ::
("outPoint" | outPointCodec) ::
("publicKeyScript" | provide(ByteVector.empty)) ::
("sequence" | uint32) ::
("localAmount" | millisatoshi) ::
("remoteAmount" | millisatoshi) ::
Expand All @@ -260,12 +262,23 @@ private[channel] object ChannelCodecs4 {
private val sharedInteractiveTxInputWithHtlcsCodec: Codec[InteractiveTxBuilder.Input.Shared] = (
("serialId" | uint64) ::
("outPoint" | outPointCodec) ::
("publicKeyScript" | provide(ByteVector.empty)) ::
("sequence" | uint32) ::
("localAmount" | millisatoshi) ::
("remoteAmount" | millisatoshi) ::
("htlcAmount" | millisatoshi)).as[InteractiveTxBuilder.Input.Shared]

private val sharedInteractiveTxInputWithHtlcsAndPubkeyScriptCodec: Codec[InteractiveTxBuilder.Input.Shared] = (
("serialId" | uint64) ::
("outPoint" | outPointCodec) ::
("publicKeyScript" | lengthDelimited(bytes)) ::
("sequence" | uint32) ::
("localAmount" | millisatoshi) ::
("remoteAmount" | millisatoshi) ::
("htlcAmount" | millisatoshi)).as[InteractiveTxBuilder.Input.Shared]

private val sharedInteractiveTxInputCodec: Codec[InteractiveTxBuilder.Input.Shared] = discriminated[InteractiveTxBuilder.Input.Shared].by(byte)
.typecase(0x03, sharedInteractiveTxInputWithHtlcsAndPubkeyScriptCodec)
.typecase(0x02, sharedInteractiveTxInputWithHtlcsCodec)
.typecase(0x01, sharedInteractiveTxInputWithoutHtlcsCodec)

Expand Down Expand Up @@ -355,7 +368,20 @@ private[channel] object ChannelCodecs4 {
private val dualFundedUnconfirmedFundingTxCodec: Codec[DualFundedUnconfirmedFundingTx] = (
("sharedTx" | signedSharedTransactionCodec) ::
("createdAt" | blockHeight) ::
("fundingParams" | fundingParamsCodec)).as[DualFundedUnconfirmedFundingTx]
("fundingParams" | fundingParamsCodec)).as[DualFundedUnconfirmedFundingTx].xmap(
dfu => (dfu.sharedTx.tx.sharedInput_opt, dfu.fundingParams.sharedInput_opt) match {
case (Some(sharedTxInput), Some(sharedFundingParamsInput)) if sharedTxInput.publicKeyScript.isEmpty =>
val sharedTxInput1 = sharedTxInput.copy(publicKeyScript = sharedFundingParamsInput.info.txOut.publicKeyScript)
val sharedTx1 = dfu.sharedTx.tx.copy(sharedInput_opt = Some(sharedTxInput1))
val dfu1 = dfu.sharedTx match {
case pt: PartiallySignedSharedTransaction => dfu.copy(sharedTx = pt.copy(tx = sharedTx1))
case ft: FullySignedSharedTransaction => dfu.copy(sharedTx = ft.copy(tx = sharedTx1))
}
dfu1
case _ => dfu
},
dfu => dfu
)

val fundingTxStatusCodec: Codec[LocalFundingStatus] = discriminated[LocalFundingStatus].by(uint8)
.typecase(0x01, optional(bool8, txCodec).as[SingleFundedUnconfirmedFundingTx])
Expand Down
Original file line number Diff line number Diff line change
@@ -1,22 +1,24 @@
package fr.acinq.eclair.wire.internal.channel.version4

import com.softwaremill.quicklens.ModifyPimp
import fr.acinq.bitcoin.scalacompat.{DeterministicWallet, OutPoint, Satoshi, SatoshiLong, Script, Transaction, TxIn, TxOut}
import fr.acinq.bitcoin.scalacompat.Crypto.PrivateKey
import fr.acinq.bitcoin.scalacompat.{ByteVector32, DeterministicWallet, OutPoint, Satoshi, SatoshiLong, Script, Transaction, TxId, TxIn, TxOut}
import fr.acinq.eclair.FeatureSupport.{Mandatory, Optional}
import fr.acinq.eclair.Features.{ChannelRangeQueries, PaymentSecret, VariableLengthOnion}
import fr.acinq.eclair.TestUtils.randomTxId
import fr.acinq.eclair.blockchain.fee.FeeratePerKw
import fr.acinq.eclair.blockchain.fee.{FeeratePerByte, FeeratePerKw}
import fr.acinq.eclair.channel.LocalFundingStatus.DualFundedUnconfirmedFundingTx
import fr.acinq.eclair.channel._
import fr.acinq.eclair.channel.fund.InteractiveTxBuilder.{InteractiveTxParams, PartiallySignedSharedTransaction, RequireConfirmedInputs, SharedTransaction}
import fr.acinq.eclair.channel.fund.InteractiveTxSigningSession.UnsignedLocalCommit
import fr.acinq.eclair.channel.fund.{InteractiveTxBuilder, InteractiveTxSigningSession}
import fr.acinq.eclair.transactions.CommitmentSpec
import fr.acinq.eclair.transactions.Transactions.{CommitTx, InputInfo}
import fr.acinq.eclair.transactions.{CommitmentSpec, Scripts}
import fr.acinq.eclair.wire.internal.channel.ChannelCodecsSpec.normal
import fr.acinq.eclair.wire.internal.channel.version4.ChannelCodecs4.Codecs.{channelConfigCodec, localParamsCodec, rbfStatusCodec, remoteParamsCodec}
import fr.acinq.eclair.wire.internal.channel.version4.ChannelCodecs4.Codecs._
import fr.acinq.eclair.wire.internal.channel.version4.ChannelCodecs4.channelDataCodec
import fr.acinq.eclair.wire.protocol.TxSignatures
import fr.acinq.eclair.{CltvExpiryDelta, Features, MilliSatoshi, MilliSatoshiLong, UInt64, randomBytes32, randomKey}
import fr.acinq.eclair.{BlockHeight, CltvExpiryDelta, Features, MilliSatoshi, MilliSatoshiLong, UInt64, randomBytes32, randomKey}
import org.scalatest.funsuite.AnyFunSuite
import scodec.bits._

Expand Down Expand Up @@ -154,4 +156,34 @@ class ChannelCodecs4Spec extends AnyFunSuite {
}
}

test("decode unconfirmed dual funded") {
// data encoded with the previous version of eclair, when Shared.Input did not include a pubkey script
val raw = ByteVector.fromValidHex("0x020001ff02000000000000002a2400000000000000000000000000000000000000000000000000000000000000000000000000003039000000000000006400000000000000c8000000000000012c02000000000000002b04deadbeef000000000000006400000000000000c8000000000000012c00000000000000000000000042000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000003e80000000000000000000000000000000000000000000000000000000000000000ff000000000000006400000000000000c8ff0001240000000000000000000000000000000000000000000000000000000000000000000000002be803000000000000220020eb72e573a9513d982a01f0e6a6b53e92764db81a0c26d2be94c5fc5b69a0db7d475221024d4b6cd1361032ca9bd2aeb9d900aa4d45d9ead80ac9423374c451a7254d076621031b84c5567b126440995d3ed5aaba0565d71e1834604819ff9c17f5e9d5dd078f52ae00000000024d4b6cd1361032ca9bd2aeb9d900aa4d45d9ead80ac9423374c451a7254d0766031b84c5567b126440995d3ed5aaba0565d71e1834604819ff9c17f5e9d5dd078f000000000000000000000000014a000002ee0000")
val decoded = fundingTxStatusCodec.decode(raw.bits).require.value.asInstanceOf[LocalFundingStatus.DualFundedUnconfirmedFundingTx]

// check that our codec will set the pubkeyscript using the one from the funding params
val channelId = ByteVector32.Zeroes
val script = Scripts.multiSig2of2(PrivateKey(ByteVector.fromValidHex("01" * 32)).publicKey, PrivateKey(ByteVector.fromValidHex("02" * 32)).publicKey)
val dualFundedUnconfirmedFundingTx = DualFundedUnconfirmedFundingTx(
PartiallySignedSharedTransaction(
SharedTransaction(
// we include the correct pubkey script here
Some(InteractiveTxBuilder.Input.Shared(UInt64(42), OutPoint(TxId(ByteVector32.Zeroes), 0), Script.write(Script.pay2wsh(script)), 12345L, MilliSatoshi(100), MilliSatoshi(200), MilliSatoshi(300))),
sharedOutput = InteractiveTxBuilder.Output.Shared(UInt64(43), ByteVector.fromValidHex("deadbeef"), MilliSatoshi(100), MilliSatoshi(200), MilliSatoshi(300)),
localInputs = Nil, remoteInputs = Nil, localOutputs = Nil, remoteOutputs = Nil, lockTime = 0
),
localSigs = TxSignatures(channelId, TxId(ByteVector32.Zeroes), Nil)
),
createdAt = BlockHeight(1000),
fundingParams = InteractiveTxParams(channelId = channelId, isInitiator = true, localContribution = 100.sat, remoteContribution = 200.sat,
sharedInput_opt = Some(InteractiveTxBuilder.Multisig2of2Input(
InputInfo(OutPoint(TxId(ByteVector32.Zeroes), 0), TxOut(1000.sat, Script.pay2wsh(script)), script),
0,
PrivateKey(ByteVector.fromValidHex("02" * 32)).publicKey
)),
remoteFundingPubKey = PrivateKey(ByteVector.fromValidHex("01" * 32)).publicKey,
localOutputs = Nil, lockTime = 0, dustLimit = 330.sat, targetFeerate = FeeratePerKw(FeeratePerByte(3.sat)), requireConfirmedInputs = RequireConfirmedInputs(false, false))
)
assert(decoded == dualFundedUnconfirmedFundingTx)
}
}

0 comments on commit f0cb58a

Please sign in to comment.