Skip to content

Commit f0cb58a

Browse files
authored
Add a txOut field to our InteractiveTxBuilder.Input interface (#2791)
* 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).
1 parent e73c1cf commit f0cb58a

File tree

4 files changed

+77
-15
lines changed

4 files changed

+77
-15
lines changed

eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxBuilder.scala

Lines changed: 11 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -216,11 +216,13 @@ object InteractiveTxBuilder {
216216
def serialId: UInt64
217217
def outPoint: OutPoint
218218
def sequence: Long
219+
def txOut: TxOut
219220
}
220221
object Input {
221222
/** A local-only input that funds the interactive transaction. */
222223
case class Local(serialId: UInt64, previousTx: Transaction, previousTxOutput: Long, sequence: Long) extends Input with Outgoing {
223224
override val outPoint: OutPoint = OutPoint(previousTx, previousTxOutput.toInt)
225+
override def txOut: TxOut = previousTx.txOut(previousTxOutput.toInt)
224226
}
225227

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

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

236240
sealed trait Output {
@@ -282,7 +286,7 @@ object InteractiveTxBuilder {
282286
localInputs: List[Input.Local], remoteInputs: List[Input.Remote],
283287
localOutputs: List[Output.Local], remoteOutputs: List[Output.Remote],
284288
lockTime: Long) {
285-
val localAmountIn: MilliSatoshi = sharedInput_opt.map(_.localAmount).getOrElse(0 msat) + localInputs.map(i => i.previousTx.txOut(i.previousTxOutput.toInt).amount).sum
289+
val localAmountIn: MilliSatoshi = sharedInput_opt.map(_.localAmount).getOrElse(0 msat) + localInputs.map(i => i.txOut.amount).sum
286290
val remoteAmountIn: MilliSatoshi = sharedInput_opt.map(_.remoteAmount).getOrElse(0 msat) + remoteInputs.map(_.txOut.amount).sum
287291
val localAmountOut: MilliSatoshi = sharedOutput.localAmount + localOutputs.map(_.amount).sum
288292
val remoteAmountOut: MilliSatoshi = sharedOutput.remoteAmount + remoteOutputs.map(_.amount).sum
@@ -489,7 +493,7 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
489493
case None =>
490494
(addInput.sharedInput_opt, fundingParams.sharedInput_opt) match {
491495
case (Some(outPoint), Some(sharedInput)) if outPoint == sharedInput.info.outPoint =>
492-
Input.Shared(addInput.serialId, outPoint, addInput.sequence, purpose.previousLocalBalance, purpose.previousRemoteBalance, purpose.htlcBalance)
496+
Input.Shared(addInput.serialId, outPoint, sharedInput.info.txOut.publicKeyScript, addInput.sequence, purpose.previousLocalBalance, purpose.previousRemoteBalance, purpose.htlcBalance)
493497
case _ =>
494498
return Left(PreviousTxMissing(fundingParams.channelId, addInput.serialId))
495499
}
@@ -810,7 +814,7 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
810814
// Partially signed PSBT must include spent amounts for all inputs that were signed, and we can "trust" these amounts because they are included
811815
// in the hash that we signed (see BIP143). If our bitcoin node lied about them, then our signatures are invalid.
812816
val actualLocalAmountIn = ourWalletInputs.map(i => kmp2scala(response.psbt.getInput(i).getWitnessUtxo.amount)).sum
813-
val expectedLocalAmountIn = unsignedTx.localInputs.map(i => i.previousTx.txOut(i.previousTxOutput.toInt).amount).sum
817+
val expectedLocalAmountIn = unsignedTx.localInputs.map(i => i.txOut.amount).sum
814818
require(actualLocalAmountIn == expectedLocalAmountIn, s"local spent amount $actualLocalAmountIn does not match what we expect ($expectedLocalAmountIn): bitcoin core may be malicious")
815819
val actualLocalAmountOut = ourWalletOutputs.map(i => partiallySignedTx.txOut(i).amount).sum
816820
val expectedLocalAmountOut = unsignedTx.localOutputs.map {
@@ -884,9 +888,9 @@ object InteractiveTxSigningSession {
884888
private def shouldSignFirst(isInitiator: Boolean, channelParams: ChannelParams, tx: SharedTransaction): Boolean = {
885889
val sharedAmountIn = tx.sharedInput_opt.map(i => i.localAmount + i.remoteAmount + i.htlcAmount).getOrElse(0 msat).truncateToSatoshi
886890
val (localAmountIn, remoteAmountIn) = if (isInitiator) {
887-
(sharedAmountIn + tx.localInputs.map(i => i.previousTx.txOut(i.previousTxOutput.toInt).amount).sum, tx.remoteInputs.map(i => i.txOut.amount).sum)
891+
(sharedAmountIn + tx.localInputs.map(i => i.txOut.amount).sum, tx.remoteInputs.map(i => i.txOut.amount).sum)
888892
} else {
889-
(tx.localInputs.map(i => i.previousTx.txOut(i.previousTxOutput.toInt).amount).sum, sharedAmountIn + tx.remoteInputs.map(i => i.txOut.amount).sum)
893+
(tx.localInputs.map(i => i.txOut.amount).sum, sharedAmountIn + tx.remoteInputs.map(i => i.txOut.amount).sum)
890894
}
891895
if (localAmountIn == remoteAmountIn) {
892896
// When both peers contribute the same amount, the peer with the lowest pubkey must transmit its `tx_signatures` first.
@@ -928,7 +932,7 @@ object InteractiveTxSigningSession {
928932
}
929933
val previousOutputs = {
930934
val sharedOutput = fundingParams.sharedInput_opt.map(sharedInput => sharedInput.info.outPoint -> sharedInput.info.txOut).toMap
931-
val localOutputs = txWithSigs.tx.localInputs.map(i => i.outPoint -> i.previousTx.txOut(i.previousTxOutput.toInt)).toMap
935+
val localOutputs = txWithSigs.tx.localInputs.map(i => i.outPoint -> i.txOut).toMap
932936
val remoteOutputs = txWithSigs.tx.remoteInputs.map(i => i.outPoint -> i.txOut).toMap
933937
sharedOutput ++ localOutputs ++ remoteOutputs
934938
}

eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxFunder.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -157,7 +157,7 @@ private class InteractiveTxFunder(replyTo: ActorRef[InteractiveTxFunder.Response
157157
// force us to add wallet inputs. The caller may manually decrease the output amounts if it wants to actually
158158
// contribute to the RBF attempt.
159159
if (fundingParams.isInitiator) {
160-
val sharedInput = fundingParams.sharedInput_opt.toSeq.map(sharedInput => Input.Shared(UInt64(0), sharedInput.info.outPoint, 0xfffffffdL, purpose.previousLocalBalance, purpose.previousRemoteBalance, purpose.htlcBalance))
160+
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))
161161
val sharedOutput = Output.Shared(UInt64(0), fundingPubkeyScript, purpose.previousLocalBalance + fundingParams.localContribution, purpose.previousRemoteBalance + fundingParams.remoteContribution, purpose.htlcBalance)
162162
val nonChangeOutputs = fundingParams.localOutputs.map(txOut => Output.Local.NonChange(UInt64(0), txOut.amount, txOut.publicKeyScript))
163163
val fundingContributions = sortFundingContributions(fundingParams, sharedInput ++ previousWalletInputs, sharedOutput +: nonChangeOutputs)
@@ -292,7 +292,7 @@ private class InteractiveTxFunder(replyTo: ActorRef[InteractiveTxFunder.Response
292292
case None => fundingParams.sharedInput_opt match {
293293
case Some(sharedInput) if sharedInput.info.outPoint == txIn.outPoint =>
294294
// We don't need to validate the shared input, it comes from a valid lightning channel.
295-
Future.successful(Right(Input.Shared(UInt64(0), sharedInput.info.outPoint, txIn.sequence, purpose.previousLocalBalance, purpose.previousRemoteBalance, purpose.htlcBalance)))
295+
Future.successful(Right(Input.Shared(UInt64(0), sharedInput.info.outPoint, sharedInput.info.txOut.publicKeyScript, txIn.sequence, purpose.previousLocalBalance, purpose.previousRemoteBalance, purpose.htlcBalance)))
296296
case _ =>
297297
for {
298298
previousTx <- wallet.getTransaction(txIn.outPoint.txid)

eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version4/ChannelCodecs4.scala

Lines changed: 27 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -10,6 +10,7 @@ import fr.acinq.eclair.channel.fund.InteractiveTxSigningSession.UnsignedLocalCom
1010
import fr.acinq.eclair.channel.fund.{InteractiveTxBuilder, InteractiveTxSigningSession}
1111
import fr.acinq.eclair.crypto.ShaChain
1212
import fr.acinq.eclair.MilliSatoshiLong
13+
import fr.acinq.eclair.channel.fund.InteractiveTxBuilder.{FullySignedSharedTransaction, PartiallySignedSharedTransaction}
1314
import fr.acinq.eclair.transactions.Transactions._
1415
import fr.acinq.eclair.transactions.{CommitmentSpec, DirectedHtlc, IncomingHtlc, OutgoingHtlc}
1516
import fr.acinq.eclair.wire.protocol.CommonCodecs._
@@ -252,6 +253,7 @@ private[channel] object ChannelCodecs4 {
252253
private val sharedInteractiveTxInputWithoutHtlcsCodec: Codec[InteractiveTxBuilder.Input.Shared] = (
253254
("serialId" | uint64) ::
254255
("outPoint" | outPointCodec) ::
256+
("publicKeyScript" | provide(ByteVector.empty)) ::
255257
("sequence" | uint32) ::
256258
("localAmount" | millisatoshi) ::
257259
("remoteAmount" | millisatoshi) ::
@@ -260,12 +262,23 @@ private[channel] object ChannelCodecs4 {
260262
private val sharedInteractiveTxInputWithHtlcsCodec: Codec[InteractiveTxBuilder.Input.Shared] = (
261263
("serialId" | uint64) ::
262264
("outPoint" | outPointCodec) ::
265+
("publicKeyScript" | provide(ByteVector.empty)) ::
266+
("sequence" | uint32) ::
267+
("localAmount" | millisatoshi) ::
268+
("remoteAmount" | millisatoshi) ::
269+
("htlcAmount" | millisatoshi)).as[InteractiveTxBuilder.Input.Shared]
270+
271+
private val sharedInteractiveTxInputWithHtlcsAndPubkeyScriptCodec: Codec[InteractiveTxBuilder.Input.Shared] = (
272+
("serialId" | uint64) ::
273+
("outPoint" | outPointCodec) ::
274+
("publicKeyScript" | lengthDelimited(bytes)) ::
263275
("sequence" | uint32) ::
264276
("localAmount" | millisatoshi) ::
265277
("remoteAmount" | millisatoshi) ::
266278
("htlcAmount" | millisatoshi)).as[InteractiveTxBuilder.Input.Shared]
267279

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

@@ -355,7 +368,20 @@ private[channel] object ChannelCodecs4 {
355368
private val dualFundedUnconfirmedFundingTxCodec: Codec[DualFundedUnconfirmedFundingTx] = (
356369
("sharedTx" | signedSharedTransactionCodec) ::
357370
("createdAt" | blockHeight) ::
358-
("fundingParams" | fundingParamsCodec)).as[DualFundedUnconfirmedFundingTx]
371+
("fundingParams" | fundingParamsCodec)).as[DualFundedUnconfirmedFundingTx].xmap(
372+
dfu => (dfu.sharedTx.tx.sharedInput_opt, dfu.fundingParams.sharedInput_opt) match {
373+
case (Some(sharedTxInput), Some(sharedFundingParamsInput)) if sharedTxInput.publicKeyScript.isEmpty =>
374+
val sharedTxInput1 = sharedTxInput.copy(publicKeyScript = sharedFundingParamsInput.info.txOut.publicKeyScript)
375+
val sharedTx1 = dfu.sharedTx.tx.copy(sharedInput_opt = Some(sharedTxInput1))
376+
val dfu1 = dfu.sharedTx match {
377+
case pt: PartiallySignedSharedTransaction => dfu.copy(sharedTx = pt.copy(tx = sharedTx1))
378+
case ft: FullySignedSharedTransaction => dfu.copy(sharedTx = ft.copy(tx = sharedTx1))
379+
}
380+
dfu1
381+
case _ => dfu
382+
},
383+
dfu => dfu
384+
)
359385

360386
val fundingTxStatusCodec: Codec[LocalFundingStatus] = discriminated[LocalFundingStatus].by(uint8)
361387
.typecase(0x01, optional(bool8, txCodec).as[SingleFundedUnconfirmedFundingTx])

eclair-core/src/test/scala/fr/acinq/eclair/wire/internal/channel/version4/ChannelCodecs4Spec.scala

Lines changed: 37 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -1,22 +1,24 @@
11
package fr.acinq.eclair.wire.internal.channel.version4
22

33
import com.softwaremill.quicklens.ModifyPimp
4-
import fr.acinq.bitcoin.scalacompat.{DeterministicWallet, OutPoint, Satoshi, SatoshiLong, Script, Transaction, TxIn, TxOut}
4+
import fr.acinq.bitcoin.scalacompat.Crypto.PrivateKey
5+
import fr.acinq.bitcoin.scalacompat.{ByteVector32, DeterministicWallet, OutPoint, Satoshi, SatoshiLong, Script, Transaction, TxId, TxIn, TxOut}
56
import fr.acinq.eclair.FeatureSupport.{Mandatory, Optional}
67
import fr.acinq.eclair.Features.{ChannelRangeQueries, PaymentSecret, VariableLengthOnion}
78
import fr.acinq.eclair.TestUtils.randomTxId
8-
import fr.acinq.eclair.blockchain.fee.FeeratePerKw
9+
import fr.acinq.eclair.blockchain.fee.{FeeratePerByte, FeeratePerKw}
10+
import fr.acinq.eclair.channel.LocalFundingStatus.DualFundedUnconfirmedFundingTx
911
import fr.acinq.eclair.channel._
1012
import fr.acinq.eclair.channel.fund.InteractiveTxBuilder.{InteractiveTxParams, PartiallySignedSharedTransaction, RequireConfirmedInputs, SharedTransaction}
1113
import fr.acinq.eclair.channel.fund.InteractiveTxSigningSession.UnsignedLocalCommit
1214
import fr.acinq.eclair.channel.fund.{InteractiveTxBuilder, InteractiveTxSigningSession}
13-
import fr.acinq.eclair.transactions.CommitmentSpec
1415
import fr.acinq.eclair.transactions.Transactions.{CommitTx, InputInfo}
16+
import fr.acinq.eclair.transactions.{CommitmentSpec, Scripts}
1517
import fr.acinq.eclair.wire.internal.channel.ChannelCodecsSpec.normal
16-
import fr.acinq.eclair.wire.internal.channel.version4.ChannelCodecs4.Codecs.{channelConfigCodec, localParamsCodec, rbfStatusCodec, remoteParamsCodec}
18+
import fr.acinq.eclair.wire.internal.channel.version4.ChannelCodecs4.Codecs._
1719
import fr.acinq.eclair.wire.internal.channel.version4.ChannelCodecs4.channelDataCodec
1820
import fr.acinq.eclair.wire.protocol.TxSignatures
19-
import fr.acinq.eclair.{CltvExpiryDelta, Features, MilliSatoshi, MilliSatoshiLong, UInt64, randomBytes32, randomKey}
21+
import fr.acinq.eclair.{BlockHeight, CltvExpiryDelta, Features, MilliSatoshi, MilliSatoshiLong, UInt64, randomBytes32, randomKey}
2022
import org.scalatest.funsuite.AnyFunSuite
2123
import scodec.bits._
2224

@@ -154,4 +156,34 @@ class ChannelCodecs4Spec extends AnyFunSuite {
154156
}
155157
}
156158

159+
test("decode unconfirmed dual funded") {
160+
// data encoded with the previous version of eclair, when Shared.Input did not include a pubkey script
161+
val raw = ByteVector.fromValidHex("0x020001ff02000000000000002a2400000000000000000000000000000000000000000000000000000000000000000000000000003039000000000000006400000000000000c8000000000000012c02000000000000002b04deadbeef000000000000006400000000000000c8000000000000012c00000000000000000000000042000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000003e80000000000000000000000000000000000000000000000000000000000000000ff000000000000006400000000000000c8ff0001240000000000000000000000000000000000000000000000000000000000000000000000002be803000000000000220020eb72e573a9513d982a01f0e6a6b53e92764db81a0c26d2be94c5fc5b69a0db7d475221024d4b6cd1361032ca9bd2aeb9d900aa4d45d9ead80ac9423374c451a7254d076621031b84c5567b126440995d3ed5aaba0565d71e1834604819ff9c17f5e9d5dd078f52ae00000000024d4b6cd1361032ca9bd2aeb9d900aa4d45d9ead80ac9423374c451a7254d0766031b84c5567b126440995d3ed5aaba0565d71e1834604819ff9c17f5e9d5dd078f000000000000000000000000014a000002ee0000")
162+
val decoded = fundingTxStatusCodec.decode(raw.bits).require.value.asInstanceOf[LocalFundingStatus.DualFundedUnconfirmedFundingTx]
163+
164+
// check that our codec will set the pubkeyscript using the one from the funding params
165+
val channelId = ByteVector32.Zeroes
166+
val script = Scripts.multiSig2of2(PrivateKey(ByteVector.fromValidHex("01" * 32)).publicKey, PrivateKey(ByteVector.fromValidHex("02" * 32)).publicKey)
167+
val dualFundedUnconfirmedFundingTx = DualFundedUnconfirmedFundingTx(
168+
PartiallySignedSharedTransaction(
169+
SharedTransaction(
170+
// we include the correct pubkey script here
171+
Some(InteractiveTxBuilder.Input.Shared(UInt64(42), OutPoint(TxId(ByteVector32.Zeroes), 0), Script.write(Script.pay2wsh(script)), 12345L, MilliSatoshi(100), MilliSatoshi(200), MilliSatoshi(300))),
172+
sharedOutput = InteractiveTxBuilder.Output.Shared(UInt64(43), ByteVector.fromValidHex("deadbeef"), MilliSatoshi(100), MilliSatoshi(200), MilliSatoshi(300)),
173+
localInputs = Nil, remoteInputs = Nil, localOutputs = Nil, remoteOutputs = Nil, lockTime = 0
174+
),
175+
localSigs = TxSignatures(channelId, TxId(ByteVector32.Zeroes), Nil)
176+
),
177+
createdAt = BlockHeight(1000),
178+
fundingParams = InteractiveTxParams(channelId = channelId, isInitiator = true, localContribution = 100.sat, remoteContribution = 200.sat,
179+
sharedInput_opt = Some(InteractiveTxBuilder.Multisig2of2Input(
180+
InputInfo(OutPoint(TxId(ByteVector32.Zeroes), 0), TxOut(1000.sat, Script.pay2wsh(script)), script),
181+
0,
182+
PrivateKey(ByteVector.fromValidHex("02" * 32)).publicKey
183+
)),
184+
remoteFundingPubKey = PrivateKey(ByteVector.fromValidHex("01" * 32)).publicKey,
185+
localOutputs = Nil, lockTime = 0, dustLimit = 330.sat, targetFeerate = FeeratePerKw(FeeratePerByte(3.sat)), requireConfirmedInputs = RequireConfirmedInputs(false, false))
186+
)
187+
assert(decoded == dualFundedUnconfirmedFundingTx)
188+
}
157189
}

0 commit comments

Comments
 (0)