Skip to content

Commit

Permalink
Dynamic funding pubkeys (#2634)
Browse files Browse the repository at this point in the history
Funding pubkeys are now dynamic and change for each splice.

Main changes:
- `remoteFundingPubkey` has been moved from `RemoteParams` to `Commitment`
- `localFundingPubkey` is computed by appending the `fundingTxIndex` to a new dedicated `fundingKeyPath`. As a nice side-effect, the resulting funding pubkey is constant across rbf attempts. Also, there is no change in the data model, since the base `fundingKeyPath` is constant and still belongs to `LocalParams`.

---------

Co-authored-by: Bastien Teinturier <31281497+t-bast@users.noreply.github.com>
  • Loading branch information
pm47 and t-bast committed Apr 13, 2023
1 parent a8471df commit 71568ca
Show file tree
Hide file tree
Showing 36 changed files with 398 additions and 363 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -22,15 +22,13 @@ import fr.acinq.bitcoin.scalacompat.{ByteVector32, DeterministicWallet, OutPoint
import fr.acinq.eclair.blockchain.fee.FeeratePerKw
import fr.acinq.eclair.channel.LocalFundingStatus.DualFundedUnconfirmedFundingTx
import fr.acinq.eclair.channel.fund.InteractiveTxBuilder._
import fr.acinq.eclair.io.Peer
import fr.acinq.eclair.channel.fund.{InteractiveTxBuilder, InteractiveTxSigningSession}
import fr.acinq.eclair.io.Peer
import fr.acinq.eclair.payment.OutgoingPaymentPacket.Upstream
import fr.acinq.eclair.transactions.CommitmentSpec
import fr.acinq.eclair.transactions.Transactions._
import fr.acinq.eclair.wire.protocol.{ChannelAnnouncement, ChannelReady, ChannelReestablish, ChannelUpdate, ClosingSigned, FailureMessage, FundingCreated, FundingSigned, Init, OnionRoutingPacket, OpenChannel, OpenDualFundedChannel, Shutdown, SpliceInit, UpdateAddHtlc, UpdateFailHtlc, UpdateFailMalformedHtlc, UpdateFulfillHtlc}
import fr.acinq.eclair.wire.protocol.{ChannelAnnouncement, ChannelReady, ChannelReestablish, ChannelUpdate, ClosingSigned, CommitSig, FailureMessage, FundingCreated, FundingSigned, Init, OnionRoutingPacket, OpenChannel, OpenDualFundedChannel, Shutdown, SpliceInit, UpdateAddHtlc, UpdateFailHtlc, UpdateFailMalformedHtlc, UpdateFulfillHtlc}
import fr.acinq.eclair.{Alias, BlockHeight, CltvExpiry, CltvExpiryDelta, Features, InitFeature, MilliSatoshi, MilliSatoshiLong, RealShortChannelId, UInt64}
import fr.acinq.eclair.wire.protocol.{ChannelAnnouncement, ChannelReady, ChannelReestablish, ChannelUpdate, ClosingSigned, CommitSig, FailureMessage, FundingCreated, FundingSigned, Init, OnionRoutingPacket, OpenChannel, OpenDualFundedChannel, Shutdown, UpdateAddHtlc, UpdateFailHtlc, UpdateFailMalformedHtlc, UpdateFulfillHtlc}
import fr.acinq.eclair.{Alias, BlockHeight, CltvExpiry, CltvExpiryDelta, Features, InitFeature, MilliSatoshi, RealShortChannelId, UInt64}
import scodec.bits.ByteVector

import java.util.UUID
Expand Down Expand Up @@ -494,6 +492,7 @@ final case class DATA_WAIT_FOR_FUNDING_INTERNAL(params: ChannelParams,
fundingAmount: Satoshi,
pushAmount: MilliSatoshi,
commitTxFeerate: FeeratePerKw,
remoteFundingPubKey: PublicKey,
remoteFirstPerCommitmentPoint: PublicKey,
replyTo: akka.actor.typed.ActorRef[Peer.OpenChannelResponse]) extends TransientChannelData {
val channelId: ByteVector32 = params.channelId
Expand All @@ -502,10 +501,12 @@ final case class DATA_WAIT_FOR_FUNDING_CREATED(params: ChannelParams,
fundingAmount: Satoshi,
pushAmount: MilliSatoshi,
commitTxFeerate: FeeratePerKw,
remoteFundingPubKey: PublicKey,
remoteFirstPerCommitmentPoint: PublicKey) extends TransientChannelData {
val channelId: ByteVector32 = params.channelId
}
final case class DATA_WAIT_FOR_FUNDING_SIGNED(params: ChannelParams,
remoteFundingPubKey: PublicKey,
fundingTx: Transaction,
fundingTxFee: Satoshi,
localSpec: CommitmentSpec,
Expand Down Expand Up @@ -616,7 +617,6 @@ case class RemoteParams(nodeId: PublicKey,
htlcMinimum: MilliSatoshi,
toSelfDelay: CltvExpiryDelta,
maxAcceptedHtlcs: Int,
fundingPubKey: PublicKey,
revocationBasepoint: PublicKey,
paymentBasepoint: PublicKey,
delayedPaymentBasepoint: PublicKey,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -202,9 +202,9 @@ case class LocalCommit(index: Long, spec: CommitmentSpec, commitTxAndRemoteSig:

/** The remote commitment maps to a commitment transaction that only our peer can sign and broadcast. */
case class RemoteCommit(index: Long, spec: CommitmentSpec, txid: ByteVector32, remotePerCommitmentPoint: PublicKey) {
def sign(keyManager: ChannelKeyManager, params: ChannelParams, commitInput: InputInfo): CommitSig = {
val (remoteCommitTx, htlcTxs) = Commitment.makeRemoteTxs(keyManager, params.channelConfig, params.channelFeatures, index, params.localParams, params.remoteParams, commitInput, remotePerCommitmentPoint, spec)
val sig = keyManager.sign(remoteCommitTx, keyManager.fundingPublicKey(params.localParams.fundingKeyPath), TxOwner.Remote, params.commitmentFormat)
def sign(keyManager: ChannelKeyManager, params: ChannelParams, fundingTxIndex: Long, remoteFundingPubKey: PublicKey, commitInput: InputInfo): CommitSig = {
val (remoteCommitTx, htlcTxs) = Commitment.makeRemoteTxs(keyManager, params.channelConfig, params.channelFeatures, index, params.localParams, params.remoteParams, fundingTxIndex, remoteFundingPubKey, commitInput, remotePerCommitmentPoint, spec)
val sig = keyManager.sign(remoteCommitTx, keyManager.fundingPublicKey(params.localParams.fundingKeyPath, fundingTxIndex), TxOwner.Remote, params.commitmentFormat)
val channelKeyPath = keyManager.keyPath(params.localParams, params.channelConfig)
val sortedHtlcTxs = htlcTxs.sortBy(_.input.outPoint.index)
val htlcSigs = sortedHtlcTxs.map(keyManager.sign(_, keyManager.htlcPoint(channelKeyPath), remotePerCommitmentPoint, TxOwner.Remote, params.commitmentFormat))
Expand All @@ -224,6 +224,7 @@ case class NextRemoteCommit(sig: CommitSig, commit: RemoteCommit)
* - commitments that share the same index are rbfed
*/
case class Commitment(fundingTxIndex: Long,
remoteFundingPubKey: PublicKey,
localFundingStatus: LocalFundingStatus, remoteFundingStatus: RemoteFundingStatus,
localCommit: LocalCommit, remoteCommit: RemoteCommit, nextRemoteCommit_opt: Option[NextRemoteCommit]) {
val commitInput: InputInfo = localCommit.commitTxAndRemoteSig.commitTx.input
Expand Down Expand Up @@ -564,8 +565,8 @@ case class Commitment(fundingTxIndex: Long,
def sendCommit(keyManager: ChannelKeyManager, params: ChannelParams, changes: CommitmentChanges, remoteNextPerCommitmentPoint: PublicKey)(implicit log: LoggingAdapter): (Commitment, CommitSig) = {
// remote commitment will include all local proposed changes + remote acked changes
val spec = CommitmentSpec.reduce(remoteCommit.spec, changes.remoteChanges.acked, changes.localChanges.proposed)
val (remoteCommitTx, htlcTxs) = Commitment.makeRemoteTxs(keyManager, params.channelConfig, params.channelFeatures, remoteCommit.index + 1, params.localParams, params.remoteParams, commitInput, remoteNextPerCommitmentPoint, spec)
val sig = keyManager.sign(remoteCommitTx, keyManager.fundingPublicKey(params.localParams.fundingKeyPath), TxOwner.Remote, params.commitmentFormat)
val (remoteCommitTx, htlcTxs) = Commitment.makeRemoteTxs(keyManager, params.channelConfig, params.channelFeatures, remoteCommit.index + 1, params.localParams, params.remoteParams, fundingTxIndex, remoteFundingPubKey, commitInput, remoteNextPerCommitmentPoint, spec)
val sig = keyManager.sign(remoteCommitTx, keyManager.fundingPublicKey(params.localParams.fundingKeyPath, fundingTxIndex), TxOwner.Remote, params.commitmentFormat)

val sortedHtlcTxs: Seq[TransactionWithInputInfo] = htlcTxs.sortBy(_.input.outPoint.index)
val channelKeyPath = keyManager.keyPath(params.localParams, params.channelConfig)
Expand All @@ -590,9 +591,9 @@ case class Commitment(fundingTxIndex: Long,
// we will reply to this sig with our old revocation hash preimage (at index) and our next revocation hash (at index + 1)
// and will increment our index
val spec = CommitmentSpec.reduce(localCommit.spec, changes.localChanges.acked, changes.remoteChanges.proposed)
val (localCommitTx, htlcTxs) = Commitment.makeLocalTxs(keyManager, params.channelConfig, params.channelFeatures, localCommit.index + 1, params.localParams, params.remoteParams, commitInput, localPerCommitmentPoint, spec)
val (localCommitTx, htlcTxs) = Commitment.makeLocalTxs(keyManager, params.channelConfig, params.channelFeatures, localCommit.index + 1, params.localParams, params.remoteParams, fundingTxIndex, remoteFundingPubKey, commitInput, localPerCommitmentPoint, spec)
log.info(s"built local commit number=${localCommit.index + 1} toLocalMsat=${spec.toLocal.toLong} toRemoteMsat=${spec.toRemote.toLong} htlc_in={} htlc_out={} feeratePerKw=${spec.commitTxFeerate} txid=${localCommitTx.tx.txid} fundingTxId=$fundingTxId", spec.htlcs.collect(DirectedHtlc.incoming).map(_.id).mkString(","), spec.htlcs.collect(DirectedHtlc.outgoing).map(_.id).mkString(","))
if (!checkSig(localCommitTx, commit.signature, params.remoteParams.fundingPubKey, TxOwner.Remote, params.commitmentFormat)) {
if (!checkSig(localCommitTx, commit.signature, remoteFundingPubKey, TxOwner.Remote, params.commitmentFormat)) {
return Left(InvalidCommitmentSignature(params.channelId, localCommitTx.tx.txid))
}
val sortedHtlcTxs: Seq[HtlcTx] = htlcTxs.sortBy(_.input.outPoint.index)
Expand All @@ -615,9 +616,9 @@ case class Commitment(fundingTxIndex: Long,
/** Return a fully signed commit tx, that can be published as-is. */
def fullySignedLocalCommitTx(params: ChannelParams, keyManager: ChannelKeyManager): CommitTx = {
val unsignedCommitTx = localCommit.commitTxAndRemoteSig.commitTx
val localSig = keyManager.sign(unsignedCommitTx, keyManager.fundingPublicKey(params.localParams.fundingKeyPath), TxOwner.Local, params.commitmentFormat)
val localSig = keyManager.sign(unsignedCommitTx, keyManager.fundingPublicKey(params.localParams.fundingKeyPath, fundingTxIndex), TxOwner.Local, params.commitmentFormat)
val remoteSig = localCommit.commitTxAndRemoteSig.remoteSig
val commitTx = addSigs(unsignedCommitTx, keyManager.fundingPublicKey(params.localParams.fundingKeyPath).publicKey, params.remoteParams.fundingPubKey, localSig, remoteSig)
val commitTx = addSigs(unsignedCommitTx, keyManager.fundingPublicKey(params.localParams.fundingKeyPath, fundingTxIndex).publicKey, remoteFundingPubKey, localSig, remoteSig)
// We verify the remote signature when receiving their commit_sig, so this check should always pass.
require(checkSpendable(commitTx).isSuccess, "commit signatures are invalid")
commitTx
Expand All @@ -632,11 +633,13 @@ object Commitment {
commitTxNumber: Long,
localParams: LocalParams,
remoteParams: RemoteParams,
fundingTxIndex: Long,
remoteFundingPubKey: PublicKey,
commitmentInput: InputInfo,
localPerCommitmentPoint: PublicKey,
spec: CommitmentSpec): (CommitTx, Seq[HtlcTx]) = {
val channelKeyPath = keyManager.keyPath(localParams, channelConfig)
val localFundingPubkey = keyManager.fundingPublicKey(localParams.fundingKeyPath).publicKey
val localFundingPubkey = keyManager.fundingPublicKey(localParams.fundingKeyPath, fundingTxIndex).publicKey
val localDelayedPaymentPubkey = Generators.derivePubKey(keyManager.delayedPaymentPoint(channelKeyPath).publicKey, localPerCommitmentPoint)
val localHtlcPubkey = Generators.derivePubKey(keyManager.htlcPoint(channelKeyPath).publicKey, localPerCommitmentPoint)
val remotePaymentPubkey = if (channelFeatures.hasFeature(Features.StaticRemoteKey)) {
Expand All @@ -647,7 +650,7 @@ object Commitment {
val remoteHtlcPubkey = Generators.derivePubKey(remoteParams.htlcBasepoint, localPerCommitmentPoint)
val localRevocationPubkey = Generators.revocationPubKey(remoteParams.revocationBasepoint, localPerCommitmentPoint)
val localPaymentBasepoint = localParams.walletStaticPaymentBasepoint.getOrElse(keyManager.paymentPoint(channelKeyPath).publicKey)
val outputs = makeCommitTxOutputs(localParams.isInitiator, localParams.dustLimit, localRevocationPubkey, remoteParams.toSelfDelay, localDelayedPaymentPubkey, remotePaymentPubkey, localHtlcPubkey, remoteHtlcPubkey, localFundingPubkey, remoteParams.fundingPubKey, spec, channelFeatures.commitmentFormat)
val outputs = makeCommitTxOutputs(localParams.isInitiator, localParams.dustLimit, localRevocationPubkey, remoteParams.toSelfDelay, localDelayedPaymentPubkey, remotePaymentPubkey, localHtlcPubkey, remoteHtlcPubkey, localFundingPubkey, remoteFundingPubKey, spec, channelFeatures.commitmentFormat)
val commitTx = makeCommitTx(commitmentInput, commitTxNumber, localPaymentBasepoint, remoteParams.paymentBasepoint, localParams.isInitiator, outputs)
val htlcTxs = makeHtlcTxs(commitTx.tx, localParams.dustLimit, localRevocationPubkey, remoteParams.toSelfDelay, localDelayedPaymentPubkey, spec.htlcTxFeerate(channelFeatures.commitmentFormat), outputs, channelFeatures.commitmentFormat)
(commitTx, htlcTxs)
Expand All @@ -659,11 +662,13 @@ object Commitment {
commitTxNumber: Long,
localParams: LocalParams,
remoteParams: RemoteParams,
fundingTxIndex: Long,
remoteFundingPubKey: PublicKey,
commitmentInput: InputInfo,
remotePerCommitmentPoint: PublicKey,
spec: CommitmentSpec): (CommitTx, Seq[HtlcTx]) = {
val channelKeyPath = keyManager.keyPath(localParams, channelConfig)
val localFundingPubkey = keyManager.fundingPublicKey(localParams.fundingKeyPath).publicKey
val localFundingPubkey = keyManager.fundingPublicKey(localParams.fundingKeyPath, fundingTxIndex).publicKey
val localPaymentBasepoint = localParams.walletStaticPaymentBasepoint.getOrElse(keyManager.paymentPoint(channelKeyPath).publicKey)
val localPaymentPubkey = if (channelFeatures.hasFeature(Features.StaticRemoteKey)) {
localPaymentBasepoint
Expand All @@ -674,7 +679,7 @@ object Commitment {
val remoteDelayedPaymentPubkey = Generators.derivePubKey(remoteParams.delayedPaymentBasepoint, remotePerCommitmentPoint)
val remoteHtlcPubkey = Generators.derivePubKey(remoteParams.htlcBasepoint, remotePerCommitmentPoint)
val remoteRevocationPubkey = Generators.revocationPubKey(keyManager.revocationPoint(channelKeyPath).publicKey, remotePerCommitmentPoint)
val outputs = makeCommitTxOutputs(!localParams.isInitiator, remoteParams.dustLimit, remoteRevocationPubkey, localParams.toSelfDelay, remoteDelayedPaymentPubkey, localPaymentPubkey, remoteHtlcPubkey, localHtlcPubkey, remoteParams.fundingPubKey, localFundingPubkey, spec, channelFeatures.commitmentFormat)
val outputs = makeCommitTxOutputs(!localParams.isInitiator, remoteParams.dustLimit, remoteRevocationPubkey, localParams.toSelfDelay, remoteDelayedPaymentPubkey, localPaymentPubkey, remoteHtlcPubkey, localHtlcPubkey, remoteFundingPubKey, localFundingPubkey, spec, channelFeatures.commitmentFormat)
val commitTx = makeCommitTx(commitmentInput, commitTxNumber, remoteParams.paymentBasepoint, localPaymentBasepoint, !localParams.isInitiator, outputs)
val htlcTxs = makeHtlcTxs(commitTx.tx, remoteParams.dustLimit, remoteRevocationPubkey, localParams.toSelfDelay, remoteDelayedPaymentPubkey, spec.htlcTxFeerate(channelFeatures.commitmentFormat), outputs, channelFeatures.commitmentFormat)
(commitTx, htlcTxs)
Expand All @@ -684,6 +689,7 @@ object Commitment {
/** Subset of Commitments when we want to work with a single, specific commitment. */
case class FullCommitment(params: ChannelParams, changes: CommitmentChanges,
fundingTxIndex: Long,
remoteFundingPubKey: PublicKey,
localFundingStatus: LocalFundingStatus, remoteFundingStatus: RemoteFundingStatus,
localCommit: LocalCommit, remoteCommit: RemoteCommit, nextRemoteCommit_opt: Option[NextRemoteCommit]) {
val channelId = params.channelId
Expand All @@ -692,7 +698,7 @@ case class FullCommitment(params: ChannelParams, changes: CommitmentChanges,
val commitInput = localCommit.commitTxAndRemoteSig.commitTx.input
val fundingTxId = commitInput.outPoint.txid
val capacity = commitInput.txOut.amount
val commitment = Commitment(fundingTxIndex, localFundingStatus, remoteFundingStatus, localCommit, remoteCommit, nextRemoteCommit_opt)
val commitment = Commitment(fundingTxIndex, remoteFundingPubKey, localFundingStatus, remoteFundingStatus, localCommit, remoteCommit, nextRemoteCommit_opt)

def localChannelReserve: Satoshi = commitment.localChannelReserve(params)

Expand Down Expand Up @@ -756,7 +762,7 @@ case class Commitments(params: ChannelParams,
lazy val availableBalanceForReceive: MilliSatoshi = active.map(_.availableBalanceForReceive(params, changes)).min

// We always use the last commitment that was created, to make sure we never go back in time.
val latest = FullCommitment(params, changes, active.head.fundingTxIndex, active.head.localFundingStatus, active.head.remoteFundingStatus, active.head.localCommit, active.head.remoteCommit, active.head.nextRemoteCommit_opt)
val latest = FullCommitment(params, changes, active.head.fundingTxIndex, active.head.remoteFundingPubKey, active.head.localFundingStatus, active.head.remoteFundingStatus, active.head.localCommit, active.head.remoteCommit, active.head.nextRemoteCommit_opt)

val all: Seq[Commitment] = active ++ inactive

Expand Down Expand Up @@ -1082,10 +1088,12 @@ case class Commitments(params: ChannelParams,
}

def validateSeed(keyManager: ChannelKeyManager): Boolean = {
val localFundingKey = keyManager.fundingPublicKey(params.localParams.fundingKeyPath).publicKey
val remoteFundingKey = params.remoteParams.fundingPubKey
val fundingScript = Script.write(Scripts.multiSig2of2(localFundingKey, remoteFundingKey))
active.forall(_.commitInput.redeemScript == fundingScript)
active.forall { commitment =>
val localFundingKey = keyManager.fundingPublicKey(params.localParams.fundingKeyPath, commitment.fundingTxIndex).publicKey
val remoteFundingKey = commitment.remoteFundingPubKey
val fundingScript = Script.write(Scripts.multiSig2of2(localFundingKey, remoteFundingKey))
commitment.commitInput.redeemScript == fundingScript
}
}

/**
Expand Down
Loading

0 comments on commit 71568ca

Please sign in to comment.