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

Better closing tx management (#423)

* call `doPublish` in `handleMutualClose`

That's more consistent with other closing scenarii, and we weren't
putting a `WatchConfirmed` in case of unexpected closing tx.

* store the unsigned closing tx along with sent `closing_signed`

So that we can identify txes based on txid when they are published,
instead of on their (malleable) signature.

* don't log publish error when tx is already in blockchain

* store the last signed closing tx during negotiation

And use it instead of our local commitment if channel fails before end of
negotiation.

* fixed `SYNCING`->`NEGOTIATING` transition

There were three bugs:
(1) we weren't re-sending our `shutdown`
(2) we were re-sending all previous `closing_signed` instead of restarting
  the negotiation
(3) there was even a bug in (2) since
1aee6e8c21

On top of that, had to implement changes as per
https://github.com/lightningnetwork/lightning-rfc/pull/36.

Note that this is unfinished, as there is a corner case where fundee
reuses previous `closing_signed` to compute its next closing fee, even if
it was just disconnected for possibly a long time and network fees have
changed a lot in the meantime.

* watch for closing tx published in `OFFLINE` and `SYNCING`

* added a tx json serializer

* use a separate list for each attempted negotiation

This allow us to properly handle restarting a negotiation after a
reconnect.
This commit is contained in:
Pierre-Marie Padiou 2018-02-26 13:23:04 +01:00 committed by GitHub
parent a065ae757f
commit 7aca967e91
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 179 additions and 105 deletions

View File

@ -4,7 +4,7 @@ import java.net.InetSocketAddress
import com.google.common.net.HostAndPort
import fr.acinq.bitcoin.Crypto.{Point, PrivateKey, PublicKey, Scalar}
import fr.acinq.bitcoin.{BinaryData, OutPoint}
import fr.acinq.bitcoin.{BinaryData, OutPoint, Transaction}
import fr.acinq.eclair.channel.State
import fr.acinq.eclair.crypto.ShaChain
import fr.acinq.eclair.transactions.Transactions.TransactionWithInputInfo
@ -44,6 +44,10 @@ class ScalarSerializer extends CustomSerializer[Scalar](format => ({ null }, {
case x: Scalar => JString("XXX")
}))
class TransactionSerializer extends CustomSerializer[TransactionWithInputInfo](ser = format => ({ null }, {
case x: Transaction => JString(x.toString())
}))
class TransactionWithInputInfoSerializer extends CustomSerializer[TransactionWithInputInfo](ser = format => ({ null }, {
case x: TransactionWithInputInfo => JString(x.tx.toString())
}))

View File

@ -51,7 +51,7 @@ trait Service extends Logging {
implicit def ec: ExecutionContext = ExecutionContext.Implicits.global
implicit val serialization = jackson.Serialization
implicit val formats = org.json4s.DefaultFormats + new BinaryDataSerializer + new StateSerializer + new ShaChainSerializer + new PublicKeySerializer + new PrivateKeySerializer + new ScalarSerializer + new PointSerializer + new TransactionWithInputInfoSerializer + new InetSocketAddressSerializer + new OutPointKeySerializer + new ColorSerializer
implicit val formats = org.json4s.DefaultFormats + new BinaryDataSerializer + new StateSerializer + new ShaChainSerializer + new PublicKeySerializer + new PrivateKeySerializer + new ScalarSerializer + new PointSerializer + new TransactionSerializer + new TransactionWithInputInfoSerializer + new InetSocketAddressSerializer + new OutPointKeySerializer + new ColorSerializer
implicit val timeout = Timeout(30 seconds)
implicit val shouldWritePretty: ShouldWritePretty = ShouldWritePretty.True

View File

@ -169,12 +169,12 @@ class ZmqWatcher(client: ExtendedBitcoinClient)(implicit ec: ExecutionContext =
def publish(tx: Transaction, isRetry: Boolean = false): Unit = {
log.info(s"publishing tx (isRetry=$isRetry): txid=${tx.txid} tx=$tx")
client.publishTransaction(tx)(singleThreadExecutionContext).recover {
case t: Throwable if t.getMessage.contains("(code: -27)") => () // 'transaction already in block chain' isn't an error
case t: Throwable if t.getMessage.contains("(code: -25)") && !isRetry => // we retry only once
import akka.pattern.after
import scala.concurrent.duration._
after(3 seconds, context.system.scheduler)(Future.successful({})).map(x => publish(tx, isRetry = true))
case t: Throwable if t.getMessage.contains("(code: -27)") => () // "transaction already in block chain (code: -27)" ignore error
case t: Throwable => log.error(s"cannot publish tx: reason=${t.getMessage} txid=${tx.txid} tx=$tx")
}
}

View File

@ -39,6 +39,9 @@ object Channel {
// we don't want the counterparty to use a dust limit lower than that, because they wouldn't only hurt themselves we may need them to publish their commit tx in certain cases (backup/restore)
val MIN_DUSTLIMIT = 546
// we won't exchange more than this many signatures when negotiating the closing fee
val MAX_NEGOTIATION_ITERATIONS = 20
case object TickRefreshChannelUpdate
}
@ -681,8 +684,15 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
// are there pending signed htlcs on either side? we need to have received their last revocation!
if (d.commitments.hasNoPendingHtlcs) {
// there are no pending signed htlcs, let's go directly to NEGOTIATING
val closingSigned = Closing.makeFirstClosingTx(d.commitments, localShutdown.scriptPubKey, remoteShutdown.scriptPubKey)
goto(NEGOTIATING) using store(DATA_NEGOTIATING(d.commitments, localShutdown, remoteShutdown, closingSigned :: Nil)) sending sendList :+ closingSigned
if (d.commitments.localParams.isFunder) {
// we are funder, need to initiate the negotiation by sending the first closing_signed
val (closingTx, closingSigned) = Closing.makeFirstClosingTx(d.commitments, localShutdown.scriptPubKey, remoteShutdown.scriptPubKey)
goto(NEGOTIATING) using store(DATA_NEGOTIATING(d.commitments, localShutdown, remoteShutdown, List(List(ClosingTxProposed(closingTx.tx, closingSigned))), bestUnpublishedClosingTx_opt = None)) sending sendList :+ closingSigned
} else {
// we are fundee, will wait for their closing_signed
goto(NEGOTIATING) using store(DATA_NEGOTIATING(d.commitments, localShutdown, remoteShutdown, closingTxProposed = List(List()), bestUnpublishedClosingTx_opt = None)) sending sendList
}
} else {
// there are some pending signed htlcs, we need to fail/fulfill them
goto(SHUTDOWN) using store(DATA_SHUTDOWN(d.commitments, localShutdown, remoteShutdown)) sending sendList
@ -887,8 +897,14 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
(commitments1, revocation)
} match {
case Success((commitments1, revocation)) if commitments1.hasNoPendingHtlcs =>
val closingSigned = Closing.makeFirstClosingTx(commitments1, localShutdown.scriptPubKey, remoteShutdown.scriptPubKey)
goto(NEGOTIATING) using store(DATA_NEGOTIATING(commitments1, localShutdown, remoteShutdown, closingSigned :: Nil)) sending revocation :: closingSigned :: Nil
if (d.commitments.localParams.isFunder) {
// we are funder, need to initiate the negotiation by sending the first closing_signed
val (closingTx, closingSigned) = Closing.makeFirstClosingTx(commitments1, localShutdown.scriptPubKey, remoteShutdown.scriptPubKey)
goto(NEGOTIATING) using store(DATA_NEGOTIATING(commitments1, localShutdown, remoteShutdown, List(List(ClosingTxProposed(closingTx.tx, closingSigned))), bestUnpublishedClosingTx_opt = None)) sending revocation :: closingSigned :: Nil
} else {
// we are fundee, will wait for their closing_signed
goto(NEGOTIATING) using store(DATA_NEGOTIATING(commitments1, localShutdown, remoteShutdown, closingTxProposed = List(List()), bestUnpublishedClosingTx_opt = None)) sending revocation
}
case Success((commitments1, revocation)) =>
if (Commitments.localHasChanges(commitments1)) {
// if we have newly acknowledged changes let's sign them
@ -904,8 +920,14 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
Try(Commitments.receiveRevocation(commitments, revocation)) match {
case Success(commitments1) if commitments1.hasNoPendingHtlcs =>
log.debug(s"received a new rev, switching to NEGOTIATING spec:\n${Commitments.specs2String(commitments1)}")
val closingSigned = Closing.makeFirstClosingTx(commitments1, localShutdown.scriptPubKey, remoteShutdown.scriptPubKey)
goto(NEGOTIATING) using store(DATA_NEGOTIATING(commitments1, localShutdown, remoteShutdown, closingSigned :: Nil)) sending closingSigned
if (d.commitments.localParams.isFunder) {
// we are funder, need to initiate the negotiation by sending the first closing_signed
val (closingTx, closingSigned) = Closing.makeFirstClosingTx(commitments1, localShutdown.scriptPubKey, remoteShutdown.scriptPubKey)
goto(NEGOTIATING) using store(DATA_NEGOTIATING(commitments1, localShutdown, remoteShutdown, List(List(ClosingTxProposed(closingTx.tx, closingSigned))), bestUnpublishedClosingTx_opt = None)) sending closingSigned
} else {
// we are fundee, will wait for their closing_signed
goto(NEGOTIATING) using store(DATA_NEGOTIATING(commitments1, localShutdown, remoteShutdown, closingTxProposed = List(List()), bestUnpublishedClosingTx_opt = None))
}
case Success(commitments1) =>
// BOLT 2: A sending node SHOULD fail to route any HTLC added after it sent shutdown.
d.commitments.remoteChanges.signed.collect {
@ -951,23 +973,36 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
case Event(c@ClosingSigned(_, remoteClosingFee, remoteSig), d: DATA_NEGOTIATING) =>
log.info(s"received closingFeeSatoshis=$remoteClosingFee")
Closing.checkClosingSignature(d.commitments, d.localShutdown.scriptPubKey, d.remoteShutdown.scriptPubKey, Satoshi(remoteClosingFee), remoteSig) match {
case Success(signedClosingTx) if remoteClosingFee == d.localClosingSigned.last.feeSatoshis =>
doPublish(signedClosingTx)
handleMutualClose(signedClosingTx, Left(d))
case Success(signedClosingTx) if Some(remoteClosingFee) == d.closingTxProposed.last.lastOption.map(_.localClosingSigned.feeSatoshis) || d.closingTxProposed.flatten.size >= MAX_NEGOTIATION_ITERATIONS =>
// we close when we converge or when there were too many iterations
handleMutualClose(signedClosingTx, Left(d.copy(bestUnpublishedClosingTx_opt = Some(signedClosingTx))))
case Success(signedClosingTx) =>
val nextClosingFee = Closing.nextClosingFee(Satoshi(d.localClosingSigned.last.feeSatoshis), Satoshi(remoteClosingFee))
val (_, closingSigned) = Closing.makeClosingTx(d.commitments, d.localShutdown.scriptPubKey, d.remoteShutdown.scriptPubKey, nextClosingFee)
log.info(s"proposing closingFeeSatoshis=${closingSigned.feeSatoshis}")
if (nextClosingFee == Satoshi(remoteClosingFee)) {
doPublish(signedClosingTx)
handleMutualClose(signedClosingTx, Left(store(d))) sending closingSigned
// if we are fundee and we were waiting for them to send their first closing_signed, we don't have a lastLocalClosingFee, so we compute a firstClosingFee
val lastLocalClosingFee = d.closingTxProposed.last.lastOption.map(_.localClosingSigned.feeSatoshis).map(Satoshi)
val nextClosingFee = Closing.nextClosingFee(
localClosingFee = lastLocalClosingFee.getOrElse(Closing.firstClosingFee(d.commitments, d.localShutdown.scriptPubKey, d.remoteShutdown.scriptPubKey)),
remoteClosingFee = Satoshi(remoteClosingFee))
val (closingTx, closingSigned) = Closing.makeClosingTx(d.commitments, d.localShutdown.scriptPubKey, d.remoteShutdown.scriptPubKey, nextClosingFee)
if (Some(nextClosingFee) == lastLocalClosingFee) {
// next computed fee is the same than the one we previously sent (probably because of rounding), let's close now
handleMutualClose(signedClosingTx, Left(d.copy(bestUnpublishedClosingTx_opt = Some(signedClosingTx))))
} else if (nextClosingFee == Satoshi(remoteClosingFee)) {
// we have converged!
val closingTxProposed1 = d.closingTxProposed match {
case previousNegotiations :+ currentNegotiation => previousNegotiations :+ (currentNegotiation :+ ClosingTxProposed(closingTx.tx, closingSigned))
}
handleMutualClose(signedClosingTx, Left(store(d.copy(closingTxProposed = closingTxProposed1, bestUnpublishedClosingTx_opt = Some(signedClosingTx))))) sending closingSigned
} else {
stay using store(d.copy(localClosingSigned = d.localClosingSigned :+ closingSigned)) sending closingSigned
log.info(s"proposing closingFeeSatoshis=${closingSigned.feeSatoshis}")
val closingTxProposed1 = d.closingTxProposed match {
case previousNegotiations :+ currentNegotiation => previousNegotiations :+ (currentNegotiation :+ ClosingTxProposed(closingTx.tx, closingSigned))
}
stay using store(d.copy(closingTxProposed = closingTxProposed1, bestUnpublishedClosingTx_opt = Some(signedClosingTx))) sending closingSigned
}
case Failure(cause) => handleLocalError(cause, d, Some(c))
}
case Event(WatchEventSpent(BITCOIN_FUNDING_SPENT, tx), d: DATA_NEGOTIATING) if d.localClosingSigned.exists(closingSigned => tx.txIn.head.witness.stack.contains(closingSigned.signature)) =>
case Event(WatchEventSpent(BITCOIN_FUNDING_SPENT, tx), d: DATA_NEGOTIATING) if d.closingTxProposed.flatten.map(_.unsignedTx.txid).contains(tx.txid) =>
// they can publish a closing tx with any sig we sent them, even if we are not done negotiating
handleMutualClose(tx, Left(d))
@ -1011,7 +1046,7 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
}
case Event(WatchEventSpent(BITCOIN_FUNDING_SPENT, tx), d: DATA_CLOSING) =>
if (d.localClosingSigned.exists(closingSigned => tx.txIn.head.witness.stack.contains(closingSigned.signature))) {
if (d.mutualCloseProposed.map(_.txid).contains(tx.txid)) {
// at any time they can publish a closing tx with any sig we sent them
handleMutualClose(tx, Right(d))
} else if (Some(tx.txid) == d.localCommitPublished.map(_.commitTx.txid)) {
@ -1064,8 +1099,8 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
// we may need to fail some htlcs in case a commitment tx was published and they have reached the timeout threshold
val timedoutHtlcs =
Closing.timedoutHtlcs(d.commitments.localCommit, Satoshi(d.commitments.localParams.dustLimitSatoshis), tx) ++
Closing.timedoutHtlcs(d.commitments.remoteCommit, Satoshi(d.commitments.remoteParams.dustLimitSatoshis), tx) ++
d.commitments.remoteNextCommitInfo.left.toSeq.flatMap(r => Closing.timedoutHtlcs(r.nextRemoteCommit, Satoshi(d.commitments.remoteParams.dustLimitSatoshis), tx))
Closing.timedoutHtlcs(d.commitments.remoteCommit, Satoshi(d.commitments.remoteParams.dustLimitSatoshis), tx) ++
d.commitments.remoteNextCommitInfo.left.toSeq.flatMap(r => Closing.timedoutHtlcs(r.nextRemoteCommit, Satoshi(d.commitments.remoteParams.dustLimitSatoshis), tx))
timedoutHtlcs.foreach { add =>
val origin = d.commitments.originChannels(add.id)
log.warning(s"failing htlc #${add.id} paymentHash=${add.paymentHash} origin=$origin: htlc timed out")
@ -1169,6 +1204,8 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
// -> in CLOSING we either have mutual closed (so no more htlcs), or already have unilaterally closed (so no action required), and we can't be in OFFLINE state anyway
handleLocalError(HtlcTimedout(d.channelId), d, Some(c))
case Event(WatchEventSpent(BITCOIN_FUNDING_SPENT, tx), d: DATA_NEGOTIATING) if d.closingTxProposed.flatten.map(_.unsignedTx.txid).contains(tx.txid) => handleMutualClose(tx, Left(d))
case Event(WatchEventSpent(BITCOIN_FUNDING_SPENT, tx), d: HasCommitments) if tx.txid == d.commitments.remoteCommit.txid => handleRemoteSpentCurrent(tx, d)
case Event(WatchEventSpent(BITCOIN_FUNDING_SPENT, tx), d: HasCommitments) if d.commitments.remoteNextCommitInfo.left.toOption.map(_.nextRemoteCommit.txid).contains(tx.txid) => handleRemoteSpentNext(tx, d)
@ -1218,6 +1255,7 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
val commitments1 = handleSync(channelReestablish, d)
// BOLT 2: A node if it has sent a previous shutdown MUST retransmit shutdown.
d.localShutdown.foreach {
localShutdown =>
log.debug(s"re-sending localShutdown")
@ -1250,17 +1288,30 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
case Event(channelReestablish: ChannelReestablish, d: DATA_SHUTDOWN) =>
val commitments1 = handleSync(channelReestablish, d)
forwarder ! d.localShutdown
goto(SHUTDOWN) using d.copy(commitments = commitments1)
// BOLT 2: A node if it has sent a previous shutdown MUST retransmit shutdown.
goto(SHUTDOWN) using d.copy(commitments = commitments1) sending d.localShutdown
case Event(_: ChannelReestablish, d: DATA_NEGOTIATING) =>
forwarder ! d.localClosingSigned
goto(NEGOTIATING)
// BOLT 2: A node if it has sent a previous shutdown MUST retransmit shutdown.
// negotiation restarts from the beginning, and is initialized by the funder
// note: in any case we still need to keep all previously sent closing_signed, because they may publish one of them
if (d.commitments.localParams.isFunder) {
// we could use the last closing_signed we sent, but network fees may have changed while we were offline so it is better to restart from scratch
val (closingTx, closingSigned) = Closing.makeFirstClosingTx(d.commitments, d.localShutdown.scriptPubKey, d.remoteShutdown.scriptPubKey)
val closingTxProposed1 = d.closingTxProposed :+ List(ClosingTxProposed(closingTx.tx, closingSigned))
goto(NEGOTIATING) using store(d.copy(closingTxProposed = closingTxProposed1)) sending d.localShutdown :: closingSigned :: Nil
} else {
// we start a new round of negotiation
val closingTxProposed1 = if (d.closingTxProposed.last.isEmpty) d.closingTxProposed else d.closingTxProposed :+ List()
goto(NEGOTIATING) using d.copy(closingTxProposed = closingTxProposed1) sending d.localShutdown
}
case Event(c: CMD_CLOSE, d: HasCommitments) => handleLocalError(ForcedLocalCommit(d.channelId, "can't do a mutual close while syncing"), d, Some(c))
case Event(c@CurrentBlockCount(count), d: HasCommitments) if d.commitments.hasTimedoutOutgoingHtlcs(count) => handleLocalError(HtlcTimedout(d.channelId), d, Some(c))
case Event(WatchEventSpent(BITCOIN_FUNDING_SPENT, tx), d: DATA_NEGOTIATING) if d.closingTxProposed.flatten.map(_.unsignedTx.txid).contains(tx.txid) => handleMutualClose(tx, Left(d))
case Event(WatchEventSpent(BITCOIN_FUNDING_SPENT, tx), d: HasCommitments) if tx.txid == d.commitments.remoteCommit.txid => handleRemoteSpentCurrent(tx, d)
case Event(WatchEventSpent(BITCOIN_FUNDING_SPENT, tx), d: HasCommitments) if d.commitments.remoteNextCommitInfo.left.toOption.map(_.nextRemoteCommit.txid).contains(tx.txid) => handleRemoteSpentNext(tx, d)
@ -1397,7 +1448,15 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
case _ => log.error(cause, s"msg=${msg.getOrElse("n/a")} stateData=$stateData ")
}
val error = Error(d.channelId, cause.getMessage.getBytes)
spendLocalCurrent(d) sending error
d match {
case negotiating@DATA_NEGOTIATING(_, _, _, _, Some(bestUnpublishedClosingTx)) =>
// if we were in the process of closing and already received a closing sig from the counterparty, it's always better to use that
handleMutualClose(bestUnpublishedClosingTx, Left(negotiating))
case _ =>
// otherwise we use our latest commitment
spendLocalCurrent(d) sending error
}
}
def handleRemoteError(e: Error, d: Data) = {
@ -1405,26 +1464,21 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
log.error(s"peer sent error: ascii='${if (isAsciiPrintable(e.data)) new String(e.data, StandardCharsets.US_ASCII) else "n/a"}' bin=${e.data}")
d match {
case _: DATA_CLOSING => stay // nothing to do, there is already a spending tx published
//case negotiating: DATA_NEGOTIATING => stay TODO: (nitpick) would be nice to publish a closing tx instead if we have already received one of their sigs
case negotiating@DATA_NEGOTIATING(_, _, _, _, Some(bestUnpublishedClosingTx)) =>
// if we were in the process of closing and already received a closing sig from the counterparty, it's always better to use that
handleMutualClose(bestUnpublishedClosingTx, Left(negotiating))
case hasCommitments: HasCommitments => spendLocalCurrent(hasCommitments)
case _ => goto(CLOSED) // when there is no commitment yet, we just go to CLOSED state in case an error occurs
}
}
def handleMutualClose(closingTx: Transaction, d: Either[DATA_NEGOTIATING, DATA_CLOSING]) = {
log.info(s"a closing tx has been published: closingTxId=${closingTx.txid}")
log.info(s"closing tx published: closingTxId=${closingTx.txid}")
val closingSigned = d match {
case Left(negotiating) => negotiating.localClosingSigned
case Right(closing) => closing.localClosingSigned
}
val index = closingSigned.indexWhere(closingSigned => closingTx.txIn.head.witness.stack.contains(closingSigned.signature))
if (index != closingSigned.size - 1) {
log.warning(s"closing tx was published before end of negotiation: closingTxId=${closingTx.txid} index=$index signatures=${closingSigned.size}")
}
doPublish(closingTx)
val nextData = d match {
case Left(negotiating) => DATA_CLOSING(negotiating.commitments, negotiating.localClosingSigned, mutualClosePublished = closingTx :: Nil)
case Left(negotiating) => DATA_CLOSING(negotiating.commitments, negotiating.closingTxProposed.flatten.map(_.unsignedTx), mutualClosePublished = closingTx :: Nil)
case Right(closing) => closing.copy(mutualClosePublished = closing.mutualClosePublished :+ closingTx)
}
goto(CLOSING) using store(nextData)
@ -1452,11 +1506,11 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
val localCommitPublished = Helpers.Closing.claimCurrentLocalCommitTxOutputs(d.commitments, commitTx)
doPublish(localCommitPublished)
val nextData = d match {
case closing: DATA_CLOSING => closing.copy(localCommitPublished = Some(localCommitPublished))
case negotiating: DATA_NEGOTIATING => DATA_CLOSING(d.commitments, negotiating.localClosingSigned, localCommitPublished = Some(localCommitPublished))
case _ => DATA_CLOSING(d.commitments, localClosingSigned = Nil, localCommitPublished = Some(localCommitPublished))
}
val nextData = d match {
case closing: DATA_CLOSING => closing.copy(localCommitPublished = Some(localCommitPublished))
case negotiating: DATA_NEGOTIATING => DATA_CLOSING(d.commitments, negotiating.closingTxProposed.flatten.map(_.unsignedTx), localCommitPublished = Some(localCommitPublished))
case _ => DATA_CLOSING(d.commitments, mutualCloseProposed = Nil, localCommitPublished = Some(localCommitPublished))
}
goto(CLOSING) using store(nextData)
}
@ -1525,8 +1579,8 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
val nextData = d match {
case closing: DATA_CLOSING => closing.copy(remoteCommitPublished = Some(remoteCommitPublished))
case negotiating: DATA_NEGOTIATING => DATA_CLOSING(d.commitments, negotiating.localClosingSigned, remoteCommitPublished = Some(remoteCommitPublished))
case _ => DATA_CLOSING(d.commitments, localClosingSigned = Nil, remoteCommitPublished = Some(remoteCommitPublished))
case negotiating: DATA_NEGOTIATING => DATA_CLOSING(d.commitments, negotiating.closingTxProposed.flatten.map(_.unsignedTx), remoteCommitPublished = Some(remoteCommitPublished))
case _ => DATA_CLOSING(d.commitments, mutualCloseProposed = Nil, remoteCommitPublished = Some(remoteCommitPublished))
}
goto(CLOSING) using store(nextData)
@ -1554,8 +1608,8 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
val nextData = d match {
case closing: DATA_CLOSING => closing.copy(nextRemoteCommitPublished = Some(remoteCommitPublished))
case negotiating: DATA_NEGOTIATING => DATA_CLOSING(d.commitments, negotiating.localClosingSigned, nextRemoteCommitPublished = Some(remoteCommitPublished))
case _ => DATA_CLOSING(d.commitments, localClosingSigned = Nil, nextRemoteCommitPublished = Some(remoteCommitPublished))
case negotiating: DATA_NEGOTIATING => DATA_CLOSING(d.commitments, negotiating.closingTxProposed.flatten.map(_.unsignedTx), nextRemoteCommitPublished = Some(remoteCommitPublished))
case _ => DATA_CLOSING(d.commitments, mutualCloseProposed = Nil, nextRemoteCommitPublished = Some(remoteCommitPublished))
}
goto(CLOSING) using store(nextData)
@ -1591,8 +1645,8 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
val nextData = d match {
case closing: DATA_CLOSING => closing.copy(revokedCommitPublished = closing.revokedCommitPublished :+ revokedCommitPublished)
case negotiating: DATA_NEGOTIATING => DATA_CLOSING(d.commitments, negotiating.localClosingSigned, revokedCommitPublished = revokedCommitPublished :: Nil)
case _ => DATA_CLOSING(d.commitments, localClosingSigned = Nil, revokedCommitPublished = revokedCommitPublished :: Nil)
case negotiating: DATA_NEGOTIATING => DATA_CLOSING(d.commitments, negotiating.closingTxProposed.flatten.map(_.unsignedTx), revokedCommitPublished = revokedCommitPublished :: Nil)
case _ => DATA_CLOSING(d.commitments, mutualCloseProposed = Nil, revokedCommitPublished = revokedCommitPublished :: Nil)
}
goto(CLOSING) using store(nextData) sending error
case None =>

View File

@ -124,6 +124,8 @@ trait HasCommitments extends Data {
def channelId = commitments.channelId
}
case class ClosingTxProposed(unsignedTx: Transaction, localClosingSigned: ClosingSigned)
case class LocalCommitPublished(commitTx: Transaction, claimMainDelayedOutputTx: Option[Transaction], htlcSuccessTxs: List[Transaction], htlcTimeoutTxs: List[Transaction], claimHtlcDelayedTx: List[Transaction], irrevocablySpent: Map[OutPoint, BinaryData])
case class RemoteCommitPublished(commitTx: Transaction, claimMainOutputTx: Option[Transaction], claimHtlcSuccessTxs: List[Transaction], claimHtlcTimeoutTxs: List[Transaction], irrevocablySpent: Map[OutPoint, BinaryData])
case class RevokedCommitPublished(commitTx: Transaction, claimMainOutputTx: Option[Transaction], mainPenaltyTx: Option[Transaction], claimHtlcTimeoutTxs: List[Transaction], htlcTimeoutTxs: List[Transaction], htlcPenaltyTxs: List[Transaction], irrevocablySpent: Map[OutPoint, BinaryData])
@ -145,10 +147,14 @@ final case class DATA_NORMAL(commitments: Commitments,
final case class DATA_SHUTDOWN(commitments: Commitments,
localShutdown: Shutdown, remoteShutdown: Shutdown) extends Data with HasCommitments
final case class DATA_NEGOTIATING(commitments: Commitments,
localShutdown: Shutdown, remoteShutdown: Shutdown, localClosingSigned: List[ClosingSigned]) extends Data with HasCommitments
localShutdown: Shutdown, remoteShutdown: Shutdown,
closingTxProposed: List[List[ClosingTxProposed]], // one list for every negotiation (there can be several in case of disconnection)
bestUnpublishedClosingTx_opt: Option[Transaction]) extends Data with HasCommitments {
require(!closingTxProposed.isEmpty, "there must always be a list for the current negotiation")
require(!commitments.localParams.isFunder || closingTxProposed.forall(!_.isEmpty), "funder must have at least one closing signature for every negotation attempt because it initiates the closing")
}
final case class DATA_CLOSING(commitments: Commitments,
localClosingSigned: List[ClosingSigned],
mutualCloseProposed: List[Transaction], // all exchanged closing sigs are flattened, we use this only to keep track of what publishable tx they have
mutualClosePublished: List[Transaction] = Nil,
localCommitPublished: Option[LocalCommitPublished] = None,
remoteCommitPublished: Option[RemoteCommitPublished] = None,

View File

@ -169,33 +169,36 @@ object Helpers {
}
}
def makeFirstClosingTx(commitments: Commitments, localScriptPubkey: BinaryData, remoteScriptPubkey: BinaryData)(implicit log: LoggingAdapter): ClosingSigned = {
log.debug(s"making first closing tx with commitments:\n${Commitments.specs2String(commitments)}")
def firstClosingFee(commitments: Commitments, localScriptPubkey: BinaryData, remoteScriptPubkey: BinaryData)(implicit log: LoggingAdapter): Satoshi = {
import commitments._
val closingFee = {
// this is just to estimate the weight, it depends on size of the pubkey scripts
val dummyClosingTx = Transactions.makeClosingTx(commitInput, localScriptPubkey, remoteScriptPubkey, localParams.isFunder, Satoshi(0), Satoshi(0), localCommit.spec)
val closingWeight = Transaction.weight(Transactions.addSigs(dummyClosingTx, localParams.fundingPrivKey.publicKey, remoteParams.fundingPubKey, "aa" * 71, "bb" * 71).tx)
// no need to use a very high fee here, so we target 6 blocks; also, we "MUST set fee_satoshis less than or equal to the base fee of the final commitment transaction"
val feeratePerKw = Math.min(Globals.feeratesPerKw.get.blocks_6, commitments.localCommit.spec.feeratePerKw)
log.info(s"using feeratePerKw=$feeratePerKw for initial closing tx")
Transactions.weight2fee(feeratePerKw, closingWeight)
}
val (_, closingSigned) = makeClosingTx(commitments, localScriptPubkey, remoteScriptPubkey, closingFee)
log.info(s"proposing closingFeeSatoshis=${closingSigned.feeSatoshis}")
closingSigned
// this is just to estimate the weight, it depends on size of the pubkey scripts
val dummyClosingTx = Transactions.makeClosingTx(commitInput, localScriptPubkey, remoteScriptPubkey, localParams.isFunder, Satoshi(0), Satoshi(0), localCommit.spec)
val closingWeight = Transaction.weight(Transactions.addSigs(dummyClosingTx, localParams.fundingPrivKey.publicKey, remoteParams.fundingPubKey, "aa" * 71, "bb" * 71).tx)
// no need to use a very high fee here, so we target 6 blocks; also, we "MUST set fee_satoshis less than or equal to the base fee of the final commitment transaction"
val feeratePerKw = Math.min(Globals.feeratesPerKw.get.blocks_6, commitments.localCommit.spec.feeratePerKw)
log.info(s"using feeratePerKw=$feeratePerKw for initial closing tx")
Transactions.weight2fee(feeratePerKw, closingWeight)
}
def nextClosingFee(localClosingFee: Satoshi, remoteClosingFee: Satoshi): Satoshi = ((localClosingFee + remoteClosingFee) / 4) * 2
def makeFirstClosingTx(commitments: Commitments, localScriptPubkey: BinaryData, remoteScriptPubkey: BinaryData)(implicit log: LoggingAdapter): (ClosingTx, ClosingSigned) = {
val closingFee = firstClosingFee(commitments, localScriptPubkey, remoteScriptPubkey)
makeClosingTx(commitments, localScriptPubkey, remoteScriptPubkey, closingFee)
}
def makeClosingTx(commitments: Commitments, localScriptPubkey: BinaryData, remoteScriptPubkey: BinaryData, closingFee: Satoshi)(implicit log: LoggingAdapter): (ClosingTx, ClosingSigned) = {
import commitments._
require(isValidFinalScriptPubkey(localScriptPubkey), "invalid localScriptPubkey")
require(isValidFinalScriptPubkey(remoteScriptPubkey), "invalid remoteScriptPubkey")
log.debug(s"making closing tx with closingFee={} and commitments:\n{}", closingFee, Commitments.specs2String(commitments))
// TODO: check that
val dustLimitSatoshis = Satoshi(Math.max(localParams.dustLimitSatoshis, remoteParams.dustLimitSatoshis))
val closingTx = Transactions.makeClosingTx(commitInput, localScriptPubkey, remoteScriptPubkey, localParams.isFunder, dustLimitSatoshis, closingFee, localCommit.spec)
val localClosingSig = Transactions.sign(closingTx, commitments.localParams.fundingPrivKey)
val closingSigned = ClosingSigned(channelId, closingFee.amount, localClosingSig)
log.debug(s"closingTx=${closingTx.tx}}")
log.info(s"signed closing txid=${closingTx.tx.txid} with closingFeeSatoshis=${closingSigned.feeSatoshis}")
log.debug(s"closingTxid=${closingTx.tx.txid} closingTx=${closingTx.tx}}")
(closingTx, closingSigned)
}
@ -211,8 +214,6 @@ object Helpers {
Transactions.checkSpendable(signedClosingTx).map(x => signedClosingTx.tx).recover { case _ => throw InvalidCloseSignature(commitments.channelId, signedClosingTx.tx) }
}
def nextClosingFee(localClosingFee: Satoshi, remoteClosingFee: Satoshi): Satoshi = ((localClosingFee + remoteClosingFee) / 4) * 2
def generateTx(desc: String)(attempt: Try[TransactionWithInputInfo])(implicit log: LoggingAdapter): Option[TransactionWithInputInfo] = {
attempt match {
case Success(txinfo) =>

View File

@ -173,6 +173,10 @@ object ChannelCodecs extends Logging {
("remotePerCommitmentSecrets" | ShaChain.shaChainCodec) ::
("channelId" | binarydata(32))).as[Commitments]
val closingTxProposedCodec: Codec[ClosingTxProposed] = (
("unsignedTx" | txCodec) ::
("localClosingSigned" | closingSignedCodec)).as[ClosingTxProposed]
val localCommitPublishedCodec: Codec[LocalCommitPublished] = (
("commitTx" | txCodec) ::
("claimMainDelayedOutputTx" | optional(bool, txCodec)) ::
@ -225,11 +229,12 @@ object ChannelCodecs extends Logging {
("commitments" | commitmentsCodec) ::
("localShutdown" | shutdownCodec) ::
("remoteShutdown" | shutdownCodec) ::
("localClosingSigned" | listOfN(uint16, closingSignedCodec))).as[DATA_NEGOTIATING]
("closingTxProposed" | listOfN(uint16, listOfN(uint16, closingTxProposedCodec))) ::
("bestUnpublishedClosingTx_opt" | optional(bool, txCodec))).as[DATA_NEGOTIATING]
val DATA_CLOSING_Codec: Codec[DATA_CLOSING] = (
("commitments" | commitmentsCodec) ::
("localClosingSigned" | listOfN(uint16, closingSignedCodec)) ::
("mutualCloseProposed" | listOfN(uint16, txCodec)) ::
("mutualClosePublished" | listOfN(uint16, txCodec)) ::
("localCommitPublished" | optional(bool, localCommitPublishedCodec)) ::
("remoteCommitPublished" | optional(bool, remoteCommitPublishedCodec)) ::

View File

@ -34,16 +34,16 @@ class NegotiatingStateSpec extends TestkitBaseClass with StateTestsHelperMethods
val sender = TestProbe()
// alice initiates a closing
if (test.tags.contains("fee2")) Globals.feeratesPerKw.set(FeeratesPerKw.single(4319)) else Globals.feeratesPerKw.set(FeeratesPerKw.single(10000))
sender.send(alice, CMD_CLOSE(None))
alice2bob.expectMsgType[Shutdown]
alice2bob.forward(bob)
sender.send(bob, CMD_CLOSE(None))
bob2alice.expectMsgType[Shutdown]
// NB: at this point, bob has already computed and sent the first ClosingSigned message
bob2alice.forward(alice)
alice2bob.expectMsgType[Shutdown]
awaitCond(alice.stateName == NEGOTIATING)
// NB: at this point, alice has already computed and sent the first ClosingSigned message
// In order to force a fee negotiation, we will change the current fee before forwarding
// the Shutdown message to alice, so that alice computes a different initial closing fee.
if (test.tags.contains("fee2")) Globals.feeratesPerKw.set(FeeratesPerKw.single(4316)) else Globals.feeratesPerKw.set(FeeratesPerKw.single(5000))
bob2alice.forward(alice)
awaitCond(alice.stateName == NEGOTIATING)
alice2bob.forward(bob)
awaitCond(bob.stateName == NEGOTIATING)
test((alice, bob, alice2bob, bob2alice, alice2blockchain, bob2blockchain))
}
@ -61,18 +61,21 @@ class NegotiatingStateSpec extends TestkitBaseClass with StateTestsHelperMethods
}
}
test("recv ClosingSigned (theirCloseFee != ourCloseFee)") { case (alice, _, alice2bob, bob2alice, _, _) =>
test("recv ClosingSigned (theirCloseFee != ourCloseFee)") { case (alice, bob, alice2bob, bob2alice, _, _) =>
within(30 seconds) {
// alice initiates the negotiation
val aliceCloseSig1 = alice2bob.expectMsgType[ClosingSigned]
val bobCloseSig = bob2alice.expectMsgType[ClosingSigned]
assert(bobCloseSig.feeSatoshis == 2 * aliceCloseSig1.feeSatoshis)
alice2bob.forward(bob)
// bob answers with a counter proposition
val bobCloseSig1 = bob2alice.expectMsgType[ClosingSigned]
assert(aliceCloseSig1.feeSatoshis > bobCloseSig1.feeSatoshis)
// actual test starts here
val initialState = alice.stateData.asInstanceOf[DATA_NEGOTIATING]
bob2alice.forward(alice)
val aliceCloseSig2 = alice2bob.expectMsgType[ClosingSigned]
// BOLT 2: If the receiver [doesn't agree with the fee] it SHOULD propose a value strictly between the received fee-satoshis and its previously-sent fee-satoshis
assert(aliceCloseSig2.feeSatoshis > aliceCloseSig1.feeSatoshis && aliceCloseSig2.feeSatoshis < bobCloseSig.feeSatoshis)
awaitCond(alice.stateData.asInstanceOf[DATA_NEGOTIATING] == initialState.copy(localClosingSigned = initialState.localClosingSigned :+ aliceCloseSig2))
assert(aliceCloseSig2.feeSatoshis < aliceCloseSig1.feeSatoshis && aliceCloseSig2.feeSatoshis > bobCloseSig1.feeSatoshis)
awaitCond(alice.stateData.asInstanceOf[DATA_NEGOTIATING].closingTxProposed.last.map(_.localClosingSigned) == initialState.closingTxProposed.last.map(_.localClosingSigned) :+ aliceCloseSig2)
}
}
@ -87,16 +90,11 @@ class NegotiatingStateSpec extends TestkitBaseClass with StateTestsHelperMethods
do {
aliceCloseFee = alice2bob.expectMsgType[ClosingSigned].feeSatoshis
alice2bob.forward(bob)
bobCloseFee = bob2alice.expectMsgType[ClosingSigned].feeSatoshis
bob2alice.forward(alice)
} while (aliceCloseFee != bobCloseFee)
val closingTxA = alice2blockchain.expectMsgType[PublishAsap].tx
assert(alice2blockchain.expectMsgType[WatchConfirmed].event === BITCOIN_TX_CONFIRMED(closingTxA))
val closingTxB = bob2blockchain.expectMsgType[PublishAsap].tx
assert(bob2blockchain.expectMsgType[WatchConfirmed].event === BITCOIN_TX_CONFIRMED(closingTxB))
assert(closingTxA === closingTxB)
awaitCond(alice.stateName == CLOSING)
awaitCond(bob.stateName == CLOSING)
if (!bob2blockchain.msgAvailable) {
bobCloseFee = bob2alice.expectMsgType[ClosingSigned].feeSatoshis
bob2alice.forward(alice)
}
} while (!alice2blockchain.msgAvailable && !bob2blockchain.msgAvailable)
}
}
@ -108,12 +106,12 @@ class NegotiatingStateSpec extends TestkitBaseClass with StateTestsHelperMethods
testFeeConverge(alice, bob, alice2bob, bob2alice, alice2blockchain, bob2blockchain)
}
test("recv ClosingSigned (fee too high)") { case (alice, bob, alice2bob, bob2alice, alice2blockchain, bob2blockchain) =>
test("recv ClosingSigned (fee too high)") { case (_, bob, alice2bob, bob2alice, _, bob2blockchain) =>
within(30 seconds) {
val closingSigned = bob2alice.expectMsgType[ClosingSigned]
val aliceCloseSig = alice2bob.expectMsgType[ClosingSigned]
val sender = TestProbe()
val tx = bob.stateData.asInstanceOf[DATA_NEGOTIATING].commitments.localCommit.publishableTxs.commitTx.tx
sender.send(bob, closingSigned.copy(feeSatoshis = 99000)) // sig doesn't matter, it is checked later
sender.send(bob, aliceCloseSig.copy(feeSatoshis = 99000)) // sig doesn't matter, it is checked later
val error = bob2alice.expectMsgType[Error]
assert(new String(error.data).startsWith("invalid close fee: fee_satoshis=99000"))
bob2blockchain.expectMsg(PublishAsap(tx))
@ -122,12 +120,12 @@ class NegotiatingStateSpec extends TestkitBaseClass with StateTestsHelperMethods
}
}
test("recv ClosingSigned (invalid sig)") { case (alice, bob, alice2bob, bob2alice, alice2blockchain, bob2blockchain) =>
test("recv ClosingSigned (invalid sig)") { case (_, bob, alice2bob, bob2alice, _, bob2blockchain) =>
within(30 seconds) {
val closingSigned = bob2alice.expectMsgType[ClosingSigned]
val aliceCloseSig = alice2bob.expectMsgType[ClosingSigned]
val sender = TestProbe()
val tx = bob.stateData.asInstanceOf[DATA_NEGOTIATING].commitments.localCommit.publishableTxs.commitTx.tx
sender.send(bob, closingSigned.copy(signature = "00" * 64))
sender.send(bob, aliceCloseSig.copy(signature = "00" * 64))
val error = bob2alice.expectMsgType[Error]
assert(new String(error.data).startsWith("invalid close signature"))
bob2blockchain.expectMsg(PublishAsap(tx))
@ -142,11 +140,13 @@ class NegotiatingStateSpec extends TestkitBaseClass with StateTestsHelperMethods
do {
aliceCloseFee = alice2bob.expectMsgType[ClosingSigned].feeSatoshis
alice2bob.forward(bob)
bobCloseFee = bob2alice.expectMsgType[ClosingSigned].feeSatoshis
if (aliceCloseFee != bobCloseFee) {
bob2alice.forward(alice)
if (!bob2blockchain.msgAvailable) {
bobCloseFee = bob2alice.expectMsgType[ClosingSigned].feeSatoshis
if (aliceCloseFee != bobCloseFee) {
bob2alice.forward(alice)
}
}
} while (aliceCloseFee != bobCloseFee)
} while (!alice2blockchain.msgAvailable && !bob2blockchain.msgAvailable)
// at this point alice and bob have converged on closing fees, but alice has not yet received the final signature whereas bob has
// bob publishes the mutual close and alice is notified that the funding tx has been spent
// actual test starts here
@ -154,7 +154,9 @@ class NegotiatingStateSpec extends TestkitBaseClass with StateTestsHelperMethods
val mutualCloseTx = bob2blockchain.expectMsgType[PublishAsap].tx
assert(bob2blockchain.expectMsgType[WatchConfirmed].event === BITCOIN_TX_CONFIRMED(mutualCloseTx))
alice ! WatchEventSpent(BITCOIN_FUNDING_SPENT, mutualCloseTx)
alice2blockchain.expectNoMsg(1 second)
alice2blockchain.expectMsgType[PublishAsap]
alice2blockchain.expectMsgType[WatchConfirmed]
alice2blockchain.expectNoMsg(100 millis)
assert(alice.stateName == CLOSING)
}
}
@ -173,7 +175,9 @@ class NegotiatingStateSpec extends TestkitBaseClass with StateTestsHelperMethods
val Success(bobClosingTx) = Closing.checkClosingSignature(d.commitments, d.localShutdown.scriptPubKey, d.remoteShutdown.scriptPubKey, Satoshi(aliceClose1.feeSatoshis), aliceClose1.signature)
alice ! WatchEventSpent(BITCOIN_FUNDING_SPENT, bobClosingTx)
alice2blockchain.expectNoMsg(1 second)
alice2blockchain.expectMsgType[PublishAsap]
alice2blockchain.expectMsgType[WatchConfirmed]
alice2blockchain.expectNoMsg(100 millis)
assert(alice.stateName == CLOSING)
}
}