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

Update funding timeout fundee (#1692)

Our previous timeout was based on timestamps, mostly because blockCount
could be 0 on mobile using Electrum until a new block was received.
Now that we're diverging from the mobile wallet codebase, we can use block
heights instead which is more accurate.

See lightningnetwork/lightning-rfc#839
This commit is contained in:
Bastien Teinturier 2021-02-19 14:16:30 +01:00 committed by GitHub
parent ab89851cdf
commit 083515086f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
7 changed files with 114 additions and 71 deletions

View File

@ -66,10 +66,10 @@ import scala.util.{Failure, Success}
*
* Created by PM on 25/01/2016.
*
* @param datadir directory where eclair-core will write/read its data.
* @param pluginParams parameters for all configured plugins.
* @param seeds_opt optional seeds, if set eclair will use them instead of generating them and won't create a node_seed.dat and channel_seed.dat files.
* @param db optional databases to use, if not set eclair will create the necessary databases
* @param datadir directory where eclair-core will write/read its data.
* @param pluginParams parameters for all configured plugins.
* @param seeds_opt optional seeds, if set eclair will use them instead of generating them and won't create a node_seed.dat and channel_seed.dat files.
* @param db optional databases to use, if not set eclair will create the necessary databases
*/
class Setup(datadir: File,
pluginParams: Seq[PluginParams],
@ -185,6 +185,8 @@ class Setup(datadir: File,
assert(!initialBlockDownload, s"bitcoind should be synchronized (initialblockdownload=$initialBlockDownload)")
assert(progress > 0.999, s"bitcoind should be synchronized (progress=$progress)")
assert(headers - blocks <= 1, s"bitcoind should be synchronized (headers=$headers blocks=$blocks)")
logger.info(s"current blockchain height=$blocks")
blockCount.set(blocks)
Bitcoind(bitcoinClient)
case ELECTRUM =>
val addresses = config.hasPath("electrum") match {

View File

@ -69,8 +69,8 @@ object Channel {
// since BOLT 1.1, there is a max value for the refund delay of the main commitment tx
val MAX_TO_SELF_DELAY = CltvExpiryDelta(2016)
// as a fundee, we will wait that much time for the funding tx to confirm (funder will rely on the funding tx being double-spent)
val FUNDING_TIMEOUT_FUNDEE = 5 days
// as a fundee, we will wait that many blocks for the funding tx to confirm (funder will rely on the funding tx being double-spent)
val FUNDING_TIMEOUT_FUNDEE = 2016
// pruning occurs if no new update has been received in two weeks (BOLT 7)
val REFRESH_CHANNEL_UPDATE_INTERVAL = 10 days
@ -238,7 +238,12 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
}
}
// no need to go OFFLINE, we can directly switch to CLOSING
goto(CLOSING) using closing
if (closing.waitingSinceBlock > 1500000) {
// we were using timestamps instead of block heights when the channel was created: we reset it *and* we use block heights
goto(CLOSING) using closing.copy(waitingSinceBlock = nodeParams.currentBlockHeight) storing()
} else {
goto(CLOSING) using closing
}
case normal: DATA_NORMAL =>
// TODO: should we wait for an acknowledgment from the watcher?
@ -280,7 +285,12 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
blockchain ! WatchLost(self, data.commitments.commitInput.outPoint.txid, nodeParams.minDepthBlocks, BITCOIN_FUNDING_LOST)
// we make sure that the funding tx has been published
blockchain ! GetTxWithMeta(funding.commitments.commitInput.outPoint.txid)
goto(OFFLINE) using data
if (funding.waitingSinceBlock > 1500000) {
// we were using timestamps instead of block heights when the channel was created: we reset it *and* we use block heights
goto(OFFLINE) using funding.copy(waitingSinceBlock = nodeParams.currentBlockHeight) storing()
} else {
goto(OFFLINE) using funding
}
case _ =>
// TODO: should we wait for an acknowledgment from the watcher?
@ -474,8 +484,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
val fundingMinDepth = Helpers.minDepthForFunding(nodeParams, fundingAmount)
blockchain ! WatchSpent(self, commitInput.outPoint.txid, commitInput.outPoint.index.toInt, commitInput.txOut.publicKeyScript, BITCOIN_FUNDING_SPENT) // TODO: should we wait for an acknowledgment from the watcher?
blockchain ! WatchConfirmed(self, commitInput.outPoint.txid, commitInput.txOut.publicKeyScript, fundingMinDepth, BITCOIN_FUNDING_DEPTHOK)
context.system.scheduler.scheduleOnce(FUNDING_TIMEOUT_FUNDEE, self, BITCOIN_FUNDING_TIMEOUT)
goto(WAIT_FOR_FUNDING_CONFIRMED) using DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments, None, initialRelayFees_opt, now, None, Right(fundingSigned)) storing() sending fundingSigned
goto(WAIT_FOR_FUNDING_CONFIRMED) using DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments, None, initialRelayFees_opt, nodeParams.currentBlockHeight, None, Right(fundingSigned)) storing() sending fundingSigned
}
}
@ -592,10 +601,19 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
context.system.scheduler.scheduleOnce(2 seconds, self, remoteAnnSigs)
stay
case Event(getTxResponse: GetTxWithMetaResponse, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) if getTxResponse.txid == d.commitments.commitInput.outPoint.txid => handleGetFundingTx(getTxResponse, d.waitingSince, d.fundingTx)
case Event(getTxResponse: GetTxWithMetaResponse, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) if getTxResponse.txid == d.commitments.commitInput.outPoint.txid => handleGetFundingTx(getTxResponse, d.waitingSinceBlock, d.fundingTx)
case Event(BITCOIN_FUNDING_PUBLISH_FAILED, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) => handleFundingPublishFailed(d)
case Event(c: CurrentBlockCount, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) => d.fundingTx match {
case Some(_) => stay // we are funder, we're still waiting for the funding tx to be confirmed
case None if c.blockCount - d.waitingSinceBlock > FUNDING_TIMEOUT_FUNDEE =>
log.warning(s"funding tx hasn't been published in ${c.blockCount - d.waitingSinceBlock} blocks")
self ! BITCOIN_FUNDING_TIMEOUT
stay
case None => stay // let's wait longer
}
case Event(BITCOIN_FUNDING_TIMEOUT, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) => handleFundingTimeout(d)
case Event(WatchEventSpent(BITCOIN_FUNDING_SPENT, tx), d: DATA_WAIT_FOR_FUNDING_CONFIRMED) if tx.txid == d.commitments.remoteCommit.txid => handleRemoteSpentCurrent(tx, d)
@ -1254,11 +1272,15 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
nextRemoteCommitPublished1.foreach(doPublish)
}
stay using d.copy(commitments = commitments1, localCommitPublished = localCommitPublished1, remoteCommitPublished = remoteCommitPublished1, nextRemoteCommitPublished = nextRemoteCommitPublished1) storing() calling (republish)
stay using d.copy(commitments = commitments1, localCommitPublished = localCommitPublished1, remoteCommitPublished = remoteCommitPublished1, nextRemoteCommitPublished = nextRemoteCommitPublished1) storing() calling republish()
case Left(cause) => handleCommandError(cause, c)
}
case Event(getTxResponse: GetTxWithMetaResponse, d: DATA_CLOSING) if getTxResponse.txid == d.commitments.commitInput.outPoint.txid => handleGetFundingTx(getTxResponse, d.waitingSince, d.fundingTx)
case Event(getTxResponse: GetTxWithMetaResponse, d: DATA_CLOSING) if getTxResponse.txid == d.commitments.commitInput.outPoint.txid =>
// NB: waitingSinceBlock contains the block at which closing was initiated, not the block at which funding was initiated.
// That means we're lenient with our peer and give its funding tx more time to confirm, to avoid having to store two distinct
// waitingSinceBlock (e.g. closingWaitingSinceBlock and fundingWaitingSinceBlock).
handleGetFundingTx(getTxResponse, d.waitingSinceBlock, d.fundingTx)
case Event(BITCOIN_FUNDING_PUBLISH_FAILED, d: DATA_CLOSING) => handleFundingPublishFailed(d)
@ -1465,7 +1487,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
// we're in OFFLINE state, we don't broadcast the new update right away, we will do that when next time we go to NORMAL state
stay using d.copy(channelUpdate = channelUpdate) storing()
case Event(getTxResponse: GetTxWithMetaResponse, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) if getTxResponse.txid == d.commitments.commitInput.outPoint.txid => handleGetFundingTx(getTxResponse, d.waitingSince, d.fundingTx)
case Event(getTxResponse: GetTxWithMetaResponse, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) if getTxResponse.txid == d.commitments.commitInput.outPoint.txid => handleGetFundingTx(getTxResponse, d.waitingSinceBlock, d.fundingTx)
case Event(BITCOIN_FUNDING_PUBLISH_FAILED, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) => handleFundingPublishFailed(d)
@ -1635,7 +1657,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
case Event(c: CurrentFeerates, d: HasCommitments) =>
handleOfflineFeerate(c, d)
case Event(getTxResponse: GetTxWithMetaResponse, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) if getTxResponse.txid == d.commitments.commitInput.outPoint.txid => handleGetFundingTx(getTxResponse, d.waitingSince, d.fundingTx)
case Event(getTxResponse: GetTxWithMetaResponse, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) if getTxResponse.txid == d.commitments.commitInput.outPoint.txid => handleGetFundingTx(getTxResponse, d.waitingSinceBlock, d.fundingTx)
case Event(BITCOIN_FUNDING_PUBLISH_FAILED, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) => handleFundingPublishFailed(d)
@ -1937,7 +1959,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
}
}
def handleGetFundingTx(getTxResponse: GetTxWithMetaResponse, waitingSince: Long, fundingTx_opt: Option[Transaction]) = {
def handleGetFundingTx(getTxResponse: GetTxWithMetaResponse, waitingSinceBlock: Long, fundingTx_opt: Option[Transaction]) = {
import getTxResponse._
tx_opt match {
case Some(_) => () // the funding tx exists, nothing to do
@ -1951,14 +1973,13 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
// we also check if the funding tx has been double-spent
checkDoubleSpent(fundingTx)
context.system.scheduler.scheduleOnce(1 day, blockchain, GetTxWithMeta(txid))
case None if (now.seconds - waitingSince.seconds) > FUNDING_TIMEOUT_FUNDEE && (now.seconds - lastBlockTimestamp.seconds) < 1.hour =>
case None if (nodeParams.currentBlockHeight - waitingSinceBlock) > FUNDING_TIMEOUT_FUNDEE =>
// if we are fundee, we give up after some time
// NB: we want to be sure that the blockchain is in sync to prevent false negatives
log.warning(s"funding tx hasn't been published in ${(now.seconds - waitingSince.seconds).toDays} days and blockchain is fresh from ${(now.seconds - lastBlockTimestamp.seconds).toMinutes} minutes ago")
log.warning(s"funding tx hasn't been published in ${nodeParams.currentBlockHeight - waitingSinceBlock} blocks")
self ! BITCOIN_FUNDING_TIMEOUT
case None =>
// let's wait a little longer
log.info(s"funding tx still hasn't been published in ${(now.seconds - waitingSince.seconds).toDays} days, will wait ${(FUNDING_TIMEOUT_FUNDEE - now.seconds + waitingSince.seconds).toDays} more days...")
log.info(s"funding tx still hasn't been published in ${nodeParams.currentBlockHeight - waitingSinceBlock} blocks, will wait ${FUNDING_TIMEOUT_FUNDEE - nodeParams.currentBlockHeight + waitingSinceBlock} more blocks...")
context.system.scheduler.scheduleOnce(1 day, blockchain, GetTxWithMeta(txid))
}
}
@ -2082,7 +2103,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
def handleMutualClose(closingTx: Transaction, d: Either[DATA_NEGOTIATING, DATA_CLOSING]) = {
log.info(s"closing tx published: closingTxId=${closingTx.txid}")
val nextData = d match {
case Left(negotiating) => DATA_CLOSING(negotiating.commitments, fundingTx = None, waitingSince = now, negotiating.closingTxProposed.flatten.map(_.unsignedTx), mutualClosePublished = closingTx :: Nil)
case Left(negotiating) => DATA_CLOSING(negotiating.commitments, fundingTx = None, waitingSinceBlock = nodeParams.currentBlockHeight, negotiating.closingTxProposed.flatten.map(_.unsignedTx), mutualClosePublished = closingTx :: Nil)
case Right(closing) => closing.copy(mutualClosePublished = closing.mutualClosePublished :+ closingTx)
}
goto(CLOSING) using nextData storing() calling doPublish(closingTx)
@ -2107,9 +2128,9 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
val localCommitPublished = Helpers.Closing.claimCurrentLocalCommitTxOutputs(keyManager, d.commitments, commitTx, nodeParams.onChainFeeConf.feeEstimator, nodeParams.onChainFeeConf.feeTargets)
val nextData = d match {
case closing: DATA_CLOSING => closing.copy(localCommitPublished = Some(localCommitPublished))
case negotiating: DATA_NEGOTIATING => DATA_CLOSING(d.commitments, fundingTx = None, waitingSince = now, negotiating.closingTxProposed.flatten.map(_.unsignedTx), localCommitPublished = Some(localCommitPublished))
case waitForFundingConfirmed: DATA_WAIT_FOR_FUNDING_CONFIRMED => DATA_CLOSING(d.commitments, fundingTx = waitForFundingConfirmed.fundingTx, waitingSince = now, mutualCloseProposed = Nil, localCommitPublished = Some(localCommitPublished))
case _ => DATA_CLOSING(d.commitments, fundingTx = None, waitingSince = now, mutualCloseProposed = Nil, localCommitPublished = Some(localCommitPublished))
case negotiating: DATA_NEGOTIATING => DATA_CLOSING(d.commitments, fundingTx = None, waitingSinceBlock = nodeParams.currentBlockHeight, negotiating.closingTxProposed.flatten.map(_.unsignedTx), localCommitPublished = Some(localCommitPublished))
case waitForFundingConfirmed: DATA_WAIT_FOR_FUNDING_CONFIRMED => DATA_CLOSING(d.commitments, fundingTx = waitForFundingConfirmed.fundingTx, waitingSinceBlock = nodeParams.currentBlockHeight, mutualCloseProposed = Nil, localCommitPublished = Some(localCommitPublished))
case _ => DATA_CLOSING(d.commitments, fundingTx = None, waitingSinceBlock = nodeParams.currentBlockHeight, mutualCloseProposed = Nil, localCommitPublished = Some(localCommitPublished))
}
goto(CLOSING) using nextData storing() calling doPublish(localCommitPublished)
}
@ -2169,9 +2190,9 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
val remoteCommitPublished = Helpers.Closing.claimRemoteCommitTxOutputs(keyManager, d.commitments, d.commitments.remoteCommit, commitTx, nodeParams.onChainFeeConf.feeEstimator, nodeParams.onChainFeeConf.feeTargets)
val nextData = d match {
case closing: DATA_CLOSING => closing.copy(remoteCommitPublished = Some(remoteCommitPublished))
case negotiating: DATA_NEGOTIATING => DATA_CLOSING(d.commitments, fundingTx = None, waitingSince = now, negotiating.closingTxProposed.flatten.map(_.unsignedTx), remoteCommitPublished = Some(remoteCommitPublished))
case waitForFundingConfirmed: DATA_WAIT_FOR_FUNDING_CONFIRMED => DATA_CLOSING(d.commitments, fundingTx = waitForFundingConfirmed.fundingTx, waitingSince = now, mutualCloseProposed = Nil, remoteCommitPublished = Some(remoteCommitPublished))
case _ => DATA_CLOSING(d.commitments, fundingTx = None, waitingSince = now, mutualCloseProposed = Nil, remoteCommitPublished = Some(remoteCommitPublished))
case negotiating: DATA_NEGOTIATING => DATA_CLOSING(d.commitments, fundingTx = None, waitingSinceBlock = nodeParams.currentBlockHeight, negotiating.closingTxProposed.flatten.map(_.unsignedTx), remoteCommitPublished = Some(remoteCommitPublished))
case waitForFundingConfirmed: DATA_WAIT_FOR_FUNDING_CONFIRMED => DATA_CLOSING(d.commitments, fundingTx = waitForFundingConfirmed.fundingTx, waitingSinceBlock = nodeParams.currentBlockHeight, mutualCloseProposed = Nil, remoteCommitPublished = Some(remoteCommitPublished))
case _ => DATA_CLOSING(d.commitments, fundingTx = None, waitingSinceBlock = nodeParams.currentBlockHeight, mutualCloseProposed = Nil, remoteCommitPublished = Some(remoteCommitPublished))
}
goto(CLOSING) using nextData storing() calling doPublish(remoteCommitPublished)
}
@ -2181,12 +2202,12 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
d.commitments.channelVersion match {
case v if v.paysDirectlyToWallet =>
val remoteCommitPublished = RemoteCommitPublished(commitTx, None, List.empty, List.empty, Map.empty)
val nextData = DATA_CLOSING(d.commitments, fundingTx = None, waitingSince = now, Nil, futureRemoteCommitPublished = Some(remoteCommitPublished))
val nextData = DATA_CLOSING(d.commitments, fundingTx = None, waitingSinceBlock = nodeParams.currentBlockHeight, Nil, futureRemoteCommitPublished = Some(remoteCommitPublished))
goto(CLOSING) using nextData storing() // we don't need to claim our main output in the remote commit because it already spends to our wallet address
case _ =>
val remotePerCommitmentPoint = d.remoteChannelReestablish.myCurrentPerCommitmentPoint
val remoteCommitPublished = Helpers.Closing.claimRemoteCommitMainOutput(keyManager, d.commitments, remotePerCommitmentPoint, commitTx, nodeParams.onChainFeeConf.feeEstimator, nodeParams.onChainFeeConf.feeTargets)
val nextData = DATA_CLOSING(d.commitments, fundingTx = None, waitingSince = now, Nil, futureRemoteCommitPublished = Some(remoteCommitPublished))
val nextData = DATA_CLOSING(d.commitments, fundingTx = None, waitingSinceBlock = nodeParams.currentBlockHeight, Nil, futureRemoteCommitPublished = Some(remoteCommitPublished))
goto(CLOSING) using nextData storing() calling doPublish(remoteCommitPublished)
}
}
@ -2200,9 +2221,9 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
val remoteCommitPublished = Helpers.Closing.claimRemoteCommitTxOutputs(keyManager, d.commitments, remoteCommit, commitTx, nodeParams.onChainFeeConf.feeEstimator, nodeParams.onChainFeeConf.feeTargets)
val nextData = d match {
case closing: DATA_CLOSING => closing.copy(nextRemoteCommitPublished = Some(remoteCommitPublished))
case negotiating: DATA_NEGOTIATING => DATA_CLOSING(d.commitments, fundingTx = None, waitingSince = now, negotiating.closingTxProposed.flatten.map(_.unsignedTx), nextRemoteCommitPublished = Some(remoteCommitPublished))
case negotiating: DATA_NEGOTIATING => DATA_CLOSING(d.commitments, fundingTx = None, waitingSinceBlock = nodeParams.currentBlockHeight, negotiating.closingTxProposed.flatten.map(_.unsignedTx), nextRemoteCommitPublished = Some(remoteCommitPublished))
// NB: if there is a next commitment, we can't be in DATA_WAIT_FOR_FUNDING_CONFIRMED so we don't have the case where fundingTx is defined
case _ => DATA_CLOSING(d.commitments, fundingTx = None, waitingSince = now, mutualCloseProposed = Nil, nextRemoteCommitPublished = Some(remoteCommitPublished))
case _ => DATA_CLOSING(d.commitments, fundingTx = None, waitingSinceBlock = nodeParams.currentBlockHeight, mutualCloseProposed = Nil, nextRemoteCommitPublished = Some(remoteCommitPublished))
}
goto(CLOSING) using nextData storing() calling doPublish(remoteCommitPublished)
}
@ -2234,9 +2255,9 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
val nextData = d match {
case closing: DATA_CLOSING => closing.copy(revokedCommitPublished = closing.revokedCommitPublished :+ revokedCommitPublished)
case negotiating: DATA_NEGOTIATING => DATA_CLOSING(d.commitments, fundingTx = None, waitingSince = now, negotiating.closingTxProposed.flatten.map(_.unsignedTx), revokedCommitPublished = revokedCommitPublished :: Nil)
case negotiating: DATA_NEGOTIATING => DATA_CLOSING(d.commitments, fundingTx = None, waitingSinceBlock = nodeParams.currentBlockHeight, negotiating.closingTxProposed.flatten.map(_.unsignedTx), revokedCommitPublished = revokedCommitPublished :: Nil)
// NB: if there is a revoked commitment, we can't be in DATA_WAIT_FOR_FUNDING_CONFIRMED so we don't have the case where fundingTx is defined
case _ => DATA_CLOSING(d.commitments, fundingTx = None, waitingSince = now, mutualCloseProposed = Nil, revokedCommitPublished = revokedCommitPublished :: Nil)
case _ => DATA_CLOSING(d.commitments, fundingTx = None, waitingSinceBlock = nodeParams.currentBlockHeight, mutualCloseProposed = Nil, revokedCommitPublished = revokedCommitPublished :: Nil)
}
goto(CLOSING) using nextData storing() calling doPublish(revokedCommitPublished) sending error
case None =>
@ -2424,12 +2445,10 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
}
def send(msg: LightningMessage) = {
def send(msg: LightningMessage): Unit = {
peer ! OutgoingMessage(msg, activeConnection)
}
def now = System.currentTimeMillis.milliseconds.toSeconds
override def mdc(currentMessage: Any): MDC = {
val category_opt = LogCategory(currentMessage)
val id = currentMessage match {

View File

@ -335,7 +335,7 @@ final case class DATA_WAIT_FOR_FUNDING_SIGNED(channelId: ByteVector32,
final case class DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments: Commitments,
fundingTx: Option[Transaction],
initialRelayFees_opt: Option[(MilliSatoshi, Int)],
waitingSince: Long, // how long have we been waiting for the funding tx to confirm
waitingSinceBlock: Long, // how long have we been waiting for the funding tx to confirm
deferred: Option[FundingLocked],
lastSent: Either[FundingCreated, FundingSigned]) extends Data with HasCommitments
final case class DATA_WAIT_FOR_FUNDING_LOCKED(commitments: Commitments, shortChannelId: ShortChannelId, lastSent: FundingLocked, initialRelayFees_opt: Option[(MilliSatoshi, Int)]) extends Data with HasCommitments
@ -353,11 +353,11 @@ final case class DATA_NEGOTIATING(commitments: Commitments,
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.nonEmpty, "there must always be a list for the current negotiation")
require(!commitments.localParams.isFunder || closingTxProposed.forall(_.nonEmpty), "funder must have at least one closing signature for every negotation attempt because it initiates the closing")
require(!commitments.localParams.isFunder || closingTxProposed.forall(_.nonEmpty), "funder must have at least one closing signature for every negotiation attempt because it initiates the closing")
}
final case class DATA_CLOSING(commitments: Commitments,
fundingTx: Option[Transaction], // this will be non-empty if we are funder and we got in closing while waiting for our own tx to be published
waitingSince: Long, // how long since we initiated the closing
waitingSinceBlock: Long, // how long since we initiated the closing
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,

View File

@ -60,7 +60,7 @@ class HelpersSpec extends AnyFunSuite {
DATA_CLOSING(
commitments = commitments,
fundingTx = None,
waitingSince = 0,
waitingSinceBlock = 0,
mutualCloseProposed = tx1 :: tx2 :: tx3 :: Nil,
mutualClosePublished = tx2 :: tx3 :: Nil,
localCommitPublished = None,
@ -75,7 +75,7 @@ class HelpersSpec extends AnyFunSuite {
DATA_CLOSING(
commitments = commitments,
fundingTx = None,
waitingSince = 0,
waitingSinceBlock = 0,
mutualCloseProposed = tx1 :: Nil,
mutualClosePublished = tx1 :: Nil,
localCommitPublished = Some(LocalCommitPublished(
@ -97,7 +97,7 @@ class HelpersSpec extends AnyFunSuite {
DATA_CLOSING(
commitments = commitments,
fundingTx = None,
waitingSince = 0,
waitingSinceBlock = 0,
mutualCloseProposed = tx1 :: Nil,
mutualClosePublished = tx1 :: Nil,
localCommitPublished = Some(LocalCommitPublished(
@ -121,7 +121,7 @@ class HelpersSpec extends AnyFunSuite {
DATA_CLOSING(
commitments = commitments,
fundingTx = None,
waitingSince = 0,
waitingSinceBlock = 0,
mutualCloseProposed = Nil,
mutualClosePublished = Nil,
localCommitPublished = Some(LocalCommitPublished(
@ -149,7 +149,7 @@ class HelpersSpec extends AnyFunSuite {
DATA_CLOSING(
commitments = commitments,
fundingTx = None,
waitingSince = 0,
waitingSinceBlock = 0,
mutualCloseProposed = tx1 :: Nil,
mutualClosePublished = tx1 :: Nil,
localCommitPublished = Some(LocalCommitPublished(
@ -179,7 +179,7 @@ class HelpersSpec extends AnyFunSuite {
DATA_CLOSING(
commitments = commitments.copy(remoteNextCommitInfo = Left(WaitingForRevocation(commitments.remoteCommit, null, 7L))),
fundingTx = None,
waitingSince = 0,
waitingSinceBlock = 0,
mutualCloseProposed = tx1 :: Nil,
mutualClosePublished = tx1 :: Nil,
localCommitPublished = Some(LocalCommitPublished(
@ -215,7 +215,7 @@ class HelpersSpec extends AnyFunSuite {
DATA_CLOSING(
commitments = commitments,
fundingTx = None,
waitingSince = 0,
waitingSinceBlock = 0,
mutualCloseProposed = Nil,
mutualClosePublished = Nil,
localCommitPublished = None,
@ -236,7 +236,7 @@ class HelpersSpec extends AnyFunSuite {
DATA_CLOSING(
commitments = commitments,
fundingTx = None,
waitingSince = 0,
waitingSinceBlock = 0,
mutualCloseProposed = Nil,
mutualClosePublished = Nil,
localCommitPublished = None,
@ -259,7 +259,7 @@ class HelpersSpec extends AnyFunSuite {
DATA_CLOSING(
commitments = commitments,
fundingTx = None,
waitingSince = 0,
waitingSinceBlock = 0,
mutualCloseProposed = Nil,
mutualClosePublished = Nil,
localCommitPublished = Some(LocalCommitPublished(
@ -306,7 +306,7 @@ class HelpersSpec extends AnyFunSuite {
DATA_CLOSING(
commitments = commitments,
fundingTx = None,
waitingSince = 0,
waitingSinceBlock = 0,
mutualCloseProposed = Nil,
mutualClosePublished = Nil,
localCommitPublished = Some(LocalCommitPublished(

View File

@ -106,13 +106,54 @@ class WaitForFundingConfirmedStateSpec extends TestKitBaseClass with FixtureAnyF
awaitCond(alice.stateName == CLOSED)
}
test("recv BITCOIN_FUNDING_TIMEOUT") { f =>
test("recv BITCOIN_FUNDING_TIMEOUT (funder)") { f =>
import f._
alice ! BITCOIN_FUNDING_TIMEOUT
alice2bob.expectMsgType[Error]
awaitCond(alice.stateName == CLOSED)
}
test("recv BITCOIN_FUNDING_TIMEOUT (fundee)") { f =>
import f._
bob ! BITCOIN_FUNDING_TIMEOUT
bob2alice.expectMsgType[Error]
awaitCond(bob.stateName == CLOSED)
}
test("recv CurrentBlockCount (funder)") { f =>
import f._
val initialState = alice.stateData.asInstanceOf[DATA_WAIT_FOR_FUNDING_CONFIRMED]
alice ! CurrentBlockCount(initialState.waitingSinceBlock + Channel.FUNDING_TIMEOUT_FUNDEE + 1)
alice2bob.expectNoMsg(100 millis)
}
test("recv CurrentBlockCount (funding timeout not reached)") { f =>
import f._
val initialState = bob.stateData.asInstanceOf[DATA_WAIT_FOR_FUNDING_CONFIRMED]
bob ! CurrentBlockCount(initialState.waitingSinceBlock + Channel.FUNDING_TIMEOUT_FUNDEE - 1)
bob2alice.expectNoMsg(100 millis)
}
test("recv CurrentBlockCount (funding timeout reached)") { f =>
import f._
val initialState = bob.stateData.asInstanceOf[DATA_WAIT_FOR_FUNDING_CONFIRMED]
bob ! CurrentBlockCount(initialState.waitingSinceBlock + Channel.FUNDING_TIMEOUT_FUNDEE + 1)
bob2alice.expectMsgType[Error]
awaitCond(bob.stateName == CLOSED)
}
test("migrate waitingSince to waitingSinceBlocks") { f =>
import f._
// Before version 0.5.1, eclair used an absolute timestamp instead of a block height for funding timeouts.
val beforeMigration = bob.stateData.asInstanceOf[DATA_WAIT_FOR_FUNDING_CONFIRMED].copy(waitingSinceBlock = System.currentTimeMillis().milliseconds.toSeconds)
bob.setState(WAIT_FOR_INIT_INTERNAL, Nothing)
bob ! INPUT_RESTORED(beforeMigration)
awaitCond(bob.stateName == OFFLINE)
// We reset the waiting period to the current block height when starting up after updating eclair.
val currentBlockHeight = bob.underlyingActor.nodeParams.currentBlockHeight
assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_FUNDING_CONFIRMED].waitingSinceBlock === currentBlockHeight)
}
test("recv BITCOIN_FUNDING_SPENT (remote commit)") { f =>
import f._
// bob publishes his commitment tx

View File

@ -243,32 +243,13 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
bob2blockchain.expectMsgType[WatchConfirmed] // claim-main-delayed
// test starts here
bob.setState(stateData = bob.stateData.asInstanceOf[DATA_CLOSING].copy(waitingSince = System.currentTimeMillis.milliseconds.toSeconds - 15.days.toSeconds))
bob.setState(stateData = bob.stateData.asInstanceOf[DATA_CLOSING].copy(waitingSinceBlock = bob.underlyingActor.nodeParams.currentBlockHeight - Channel.FUNDING_TIMEOUT_FUNDEE - 1))
bob ! GetTxWithMetaResponse(fundingTx.txid, None, System.currentTimeMillis.milliseconds.toSeconds)
bob2alice.expectMsgType[Error]
bob2blockchain.expectNoMsg(200 millis)
assert(bob.stateName == CLOSED)
}
test("recv GetTxResponse (fundee, tx not found, timeout, blockchain lags)", Tag("funding_unconfirmed")) { f =>
import f._
val sender = TestProbe()
val fundingTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_FUNDING_CONFIRMED].fundingTx.get
bob ! CMD_FORCECLOSE(sender.ref)
awaitCond(bob.stateName == CLOSING)
bob2alice.expectMsgType[Error]
bob2blockchain.expectMsgType[PublishAsap]
bob2blockchain.expectMsgType[PublishAsap] // claim-main-delayed
bob2blockchain.expectMsgType[WatchConfirmed] // commitment
bob2blockchain.expectMsgType[WatchConfirmed] // claim-main-delayed
// test starts here
bob ! GetTxWithMetaResponse(fundingTx.txid, None, System.currentTimeMillis.milliseconds.toSeconds - 3.hours.toSeconds)
bob2alice.expectNoMsg(200 millis)
bob2blockchain.expectNoMsg(200 millis)
assert(bob.stateName == CLOSING) // the above expectNoMsg will make us wait, so this checks that we are still in CLOSING
}
test("recv CMD_ADD_HTLC") { f =>
import f._
mutualClose(alice, bob, alice2bob, bob2alice, alice2blockchain, bob2blockchain)

View File

@ -335,7 +335,7 @@ class ChannelCodecsSpec extends AnyFunSuite {
// let's decode the old data (this will use the old codec that provides default values for new fields)
val data_new = stateDataCodec.decode(bin_old.toBitVector).require.value
assert(data_new.asInstanceOf[DATA_WAIT_FOR_FUNDING_CONFIRMED].fundingTx === None)
assert(System.currentTimeMillis.milliseconds.toSeconds - data_new.asInstanceOf[DATA_WAIT_FOR_FUNDING_CONFIRMED].waitingSince < 3600) // we just set this timestamp to current time
assert(System.currentTimeMillis.milliseconds.toSeconds - data_new.asInstanceOf[DATA_WAIT_FOR_FUNDING_CONFIRMED].waitingSinceBlock < 3600) // we just set this timestamp to current time
// and re-encode it with the new codec
val bin_new = ByteVector(stateDataCodec.encode(data_new).require.toByteVector.toArray)
// data should now be encoded under the new format