1
0
Fork 0
mirror of https://github.com/ACINQ/eclair.git synced 2025-02-22 14:22:39 +01:00

Send an event when an error occurs in a channel (#663)

This allows e.g. the mobile app to know why a channel got closed.
Depending on whether the error is local or remote, a
`Throwable`/`wire.Error` will be attached to the event.
This commit is contained in:
Pierre-Marie Padiou 2018-07-18 14:45:12 +02:00 committed by GitHub
parent fe82641e92
commit 633deb5c0b
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
2 changed files with 40 additions and 53 deletions

View file

@ -64,6 +64,10 @@ object Channel {
case object TickRefreshChannelUpdate
sealed trait ChannelError
case class LocalError(t: Throwable) extends ChannelError
case class RemoteError(e: Error) extends ChannelError
}
class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: PublicKey, blockchain: ActorRef, router: ActorRef, relayer: ActorRef, origin_opt: Option[ActorRef] = None)(implicit ec: ExecutionContext = ExecutionContext.Implicits.global) extends FSM[State, Data] with FSMDiagnosticActorLogging[State, Data] {
@ -196,12 +200,9 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
})
when(WAIT_FOR_OPEN_CHANNEL)(handleExceptions {
case Event(open: OpenChannel, DATA_WAIT_FOR_OPEN_CHANNEL(INPUT_INIT_FUNDEE(temporaryChannelId, localParams, _, remoteInit))) =>
case Event(open: OpenChannel, d@DATA_WAIT_FOR_OPEN_CHANNEL(INPUT_INIT_FUNDEE(_, localParams, _, remoteInit))) =>
Try(Helpers.validateParamsFundee(nodeParams, open)) match {
case Failure(t) =>
log.warning(t.getMessage)
val error = Error(open.temporaryChannelId, t.getMessage.getBytes)
goto(CLOSED) sending error
case Failure(t) => handleLocalError(t, d, Some(open))
case Success(_) =>
context.system.eventStream.publish(ChannelCreated(self, context.parent, remoteNodeId, false, open.temporaryChannelId))
// TODO: maybe also check uniqueness of temporary channel id
@ -247,13 +248,9 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
})
when(WAIT_FOR_ACCEPT_CHANNEL)(handleExceptions {
case Event(accept: AcceptChannel, DATA_WAIT_FOR_ACCEPT_CHANNEL(INPUT_INIT_FUNDER(temporaryChannelId, fundingSatoshis, pushMsat, initialFeeratePerKw, fundingTxFeeratePerKw, localParams, _, remoteInit, _), open)) =>
case Event(accept: AcceptChannel, d@DATA_WAIT_FOR_ACCEPT_CHANNEL(INPUT_INIT_FUNDER(temporaryChannelId, fundingSatoshis, pushMsat, initialFeeratePerKw, fundingTxFeeratePerKw, localParams, _, remoteInit, _), open)) =>
Try(Helpers.validateParamsFunder(nodeParams, open, accept)) match {
case Failure(t) =>
log.warning(t.getMessage)
val error = Error(temporaryChannelId, t.getMessage.getBytes)
replyToUser(Left(Left(t)))
goto(CLOSED) sending error
case Failure(t) => handleLocalError(t, d, Some(accept))
case _ =>
// TODO: check equality of temporaryChannelId? or should be done upstream
val remoteParams = RemoteParams(
@ -283,11 +280,11 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
goto(CLOSED) replying "ok"
case Event(e: Error, d: DATA_WAIT_FOR_ACCEPT_CHANNEL) =>
replyToUser(Left(Right(e)))
replyToUser(Left(RemoteError(e)))
handleRemoteError(e, d)
case Event(INPUT_DISCONNECTED, _) =>
replyToUser(Left(Left(new RuntimeException("disconnected"))))
replyToUser(Left(LocalError(new RuntimeException("disconnected"))))
goto(CLOSED)
})
@ -312,26 +309,24 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
case Event(Status.Failure(t), d: DATA_WAIT_FOR_FUNDING_INTERNAL) =>
log.error(t, s"wallet returned error: ")
val exc = ChannelFundingError(d.temporaryChannelId)
val error = Error(d.temporaryChannelId, exc.getMessage.getBytes)
replyToUser(Left(Left(t)))
goto(CLOSED) sending error
replyToUser(Left(LocalError(t)))
handleLocalError(ChannelFundingError(d.temporaryChannelId), d, None) // we use a generic exception and don't send the internal error to the peer
case Event(CMD_CLOSE(_), _) =>
replyToUser(Right("closed"))
goto(CLOSED) replying "ok"
case Event(e: Error, d: DATA_WAIT_FOR_FUNDING_INTERNAL) =>
replyToUser(Left(Right(e)))
replyToUser(Left(RemoteError(e)))
handleRemoteError(e, d)
case Event(INPUT_DISCONNECTED, _) =>
replyToUser(Left(Left(new RuntimeException("disconnected"))))
replyToUser(Left(LocalError(new RuntimeException("disconnected"))))
goto(CLOSED)
})
when(WAIT_FOR_FUNDING_CREATED)(handleExceptions {
case Event(FundingCreated(_, fundingTxHash, fundingTxOutputIndex, remoteSig), DATA_WAIT_FOR_FUNDING_CREATED(temporaryChannelId, localParams, remoteParams, fundingSatoshis, pushMsat, initialFeeratePerKw, remoteFirstPerCommitmentPoint, channelFlags, _)) =>
case Event(FundingCreated(_, fundingTxHash, fundingTxOutputIndex, remoteSig), d@DATA_WAIT_FOR_FUNDING_CREATED(temporaryChannelId, localParams, remoteParams, fundingSatoshis, pushMsat, initialFeeratePerKw, remoteFirstPerCommitmentPoint, channelFlags, _)) =>
// they fund the channel with their funding tx, so the money is theirs (but we are paid pushMsat)
val (localSpec, localCommitTx, remoteSpec, remoteCommitTx) = Funding.makeFirstCommitTxs(keyManager, temporaryChannelId, localParams, remoteParams, fundingSatoshis: Long, pushMsat, initialFeeratePerKw, fundingTxHash, fundingTxOutputIndex, remoteFirstPerCommitmentPoint, nodeParams.maxFeerateMismatch)
@ -339,12 +334,7 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
val localSigOfLocalTx = keyManager.sign(localCommitTx, keyManager.fundingPublicKey(localParams.channelKeyPath))
val signedLocalCommitTx = Transactions.addSigs(localCommitTx, keyManager.fundingPublicKey(localParams.channelKeyPath).publicKey, remoteParams.fundingPubKey, localSigOfLocalTx, remoteSig)
Transactions.checkSpendable(signedLocalCommitTx) match {
case Failure(cause) =>
log.error(cause, "their FundingCreated message contains an invalid signature")
val exc = InvalidCommitmentSignature(temporaryChannelId, signedLocalCommitTx.tx)
val error = Error(temporaryChannelId, exc.getMessage.getBytes)
// we haven't anything at stake yet, we can just stop
goto(CLOSED) sending error
case Failure(cause) => handleLocalError(InvalidCommitmentSignature(temporaryChannelId, signedLocalCommitTx.tx), d, None)
case Success(_) =>
val localSigOfRemoteTx = keyManager.sign(remoteCommitTx, keyManager.fundingPublicKey(localParams.channelKeyPath))
val channelId = toLongId(fundingTxHash, fundingTxOutputIndex)
@ -379,20 +369,16 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
})
when(WAIT_FOR_FUNDING_SIGNED)(handleExceptions {
case Event(FundingSigned(_, remoteSig), DATA_WAIT_FOR_FUNDING_SIGNED(channelId, localParams, remoteParams, fundingTx, localSpec, localCommitTx, remoteCommit, channelFlags, fundingCreated)) =>
case Event(msg@FundingSigned(_, remoteSig), d@DATA_WAIT_FOR_FUNDING_SIGNED(channelId, localParams, remoteParams, fundingTx, localSpec, localCommitTx, remoteCommit, channelFlags, fundingCreated)) =>
// we make sure that their sig checks out and that our first commit tx is spendable
val localSigOfLocalTx = keyManager.sign(localCommitTx, keyManager.fundingPublicKey(localParams.channelKeyPath))
val signedLocalCommitTx = Transactions.addSigs(localCommitTx, keyManager.fundingPublicKey(localParams.channelKeyPath).publicKey, remoteParams.fundingPubKey, localSigOfLocalTx, remoteSig)
Transactions.checkSpendable(signedLocalCommitTx) match {
case Failure(cause) =>
log.error(cause, "their FundingSigned message contains an invalid signature")
val exc = InvalidCommitmentSignature(channelId, signedLocalCommitTx.tx)
val error = Error(channelId, exc.getMessage.getBytes)
// we rollback the funding tx, it will never be published
wallet.rollback(fundingTx)
replyToUser(Left(Left(cause)))
// we haven't published anything yet, we can just stop
goto(CLOSED) sending error
replyToUser(Left(LocalError(cause)))
handleLocalError(InvalidCommitmentSignature(channelId, signedLocalCommitTx.tx), d, Some(msg))
case Success(_) =>
val commitInput = localCommitTx.input
val commitments = Commitments(localParams, remoteParams, channelFlags,
@ -413,10 +399,10 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
case Success(true) =>
replyToUser(Right(s"created channel $channelId"))
case Success(false) =>
replyToUser(Left(Left(new RuntimeException("couldn't publish funding tx"))))
replyToUser(Left(LocalError(new RuntimeException("couldn't publish funding tx"))))
self ! BITCOIN_FUNDING_PUBLISH_FAILED // fail-fast: this should be returned only when we are really sure the tx has *not* been published
case Failure(t) =>
replyToUser(Left(Left(t)))
replyToUser(Left(LocalError(t)))
log.error(t, s"error while committing funding tx: ") // tx may still have been published, can't fail-fast
}
goto(WAIT_FOR_FUNDING_CONFIRMED) using nextState
@ -431,7 +417,7 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
case Event(e: Error, d: DATA_WAIT_FOR_FUNDING_SIGNED) =>
// we rollback the funding tx, it will never be published
wallet.rollback(d.fundingTx)
replyToUser(Left(Right(e)))
replyToUser(Left(RemoteError(e)))
handleRemoteError(e, d)
})
@ -456,13 +442,16 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
log.error(s"failed to publish funding tx")
val exc = ChannelFundingError(d.channelId)
val error = Error(d.channelId, exc.getMessage.getBytes)
// note: implementation guarantees that the tx will not ever be published, so we can close the channel right away
// NB: we don't use the handleLocalError handler because it would result in the commit tx being published, which we don't want:
// implementation *guarantees* that in case of BITCOIN_FUNDING_PUBLISH_FAILED, the funding tx hasn't and will never be published, so we can close the channel right away
context.system.eventStream.publish(ChannelFailed(self, Helpers.getChannelId(stateData), remoteNodeId, stateData, LocalError(exc)))
goto(CLOSED) sending error
// TODO: not implemented, users will have to manually close
case Event(BITCOIN_FUNDING_TIMEOUT, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) =>
val exc = FundingTxTimedout(d.channelId)
val error = Error(d.channelId, exc.getMessage.getBytes)
context.system.eventStream.publish(ChannelFailed(self, Helpers.getChannelId(stateData), remoteNodeId, stateData, LocalError(exc)))
goto(ERR_FUNDING_TIMEOUT) sending error
case Event(remoteAnnSigs: AnnouncementSignatures, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) if d.commitments.announceChannel =>
@ -1499,10 +1488,10 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
/**
* This function is used to return feedback to user at channel opening
*/
def replyToUser(message: Either[Either[Throwable, Error], String]) = {
def replyToUser(message: Either[Channel.ChannelError, String]) = {
val m = message match {
case Left(Left(t)) => Status.Failure(t)
case Left(Right(e)) => Status.Failure(new RuntimeException(s"peer sent error: '${if (isAsciiPrintable(e.data)) new String(e.data, StandardCharsets.US_ASCII) else e.data.toString()}'"))
case Left(LocalError(t)) => Status.Failure(t)
case Left(RemoteError(e)) => Status.Failure(new RuntimeException(s"peer sent error: '${if (isAsciiPrintable(e.data)) new String(e.data, StandardCharsets.US_ASCII) else e.data.toString()}'"))
case Right(s) => s
}
origin_opt.map(_ ! m)
@ -1521,7 +1510,7 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
stay replying Status.Failure(cause)
}
def handleLocalError(cause: Throwable, d: HasCommitments, msg: Option[Any]) = {
def handleLocalError(cause: Throwable, d: Data, msg: Option[Any]) = {
cause match {
case _: ForcedLocalCommit => log.warning(s"force-closing channel at user request")
case _ => log.error(s"${cause.getMessage} while processing msg=${msg.getOrElse("n/a").getClass.getSimpleName} in state=$stateName")
@ -1530,22 +1519,24 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
case _: ChannelException => ()
case _ => log.error(cause, s"msg=${msg.getOrElse("n/a")} stateData=$stateData ")
}
val error = Error(d.channelId, cause.getMessage.getBytes)
val error = Error(Helpers.getChannelId(d), cause.getMessage.getBytes)
context.system.eventStream.publish(ChannelFailed(self, Helpers.getChannelId(stateData), remoteNodeId, stateData, LocalError(cause)))
d match {
case negotiating@DATA_NEGOTIATING(_, _, _, _, Some(bestUnpublishedClosingTx)) =>
log.info(s"we have a valid closing tx, publishing it instead of our commitment: closingTxId=${bestUnpublishedClosingTx.txid}")
// 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
case dd: HasCommitments => spendLocalCurrent(dd) sending error // otherwise we use our current commitment
case _ => goto(CLOSED) sending error // when there is no commitment yet, we just send an error to our peer and go to CLOSED state
}
}
def handleRemoteError(e: Error, d: Data) = {
// see BOLT 1: only print out data verbatim if is composed of printable ASCII characters
log.error(s"peer sent error: ascii='${if (isAsciiPrintable(e.data)) new String(e.data, StandardCharsets.US_ASCII) else "n/a"}' bin=${e.data}")
context.system.eventStream.publish(ChannelFailed(self, Helpers.getChannelId(stateData), remoteNodeId, stateData, RemoteError(e)))
d match {
case _: DATA_CLOSING => stay // nothing to do, there is already a spending tx published
case negotiating@DATA_NEGOTIATING(_, _, _, _, Some(bestUnpublishedClosingTx)) =>
@ -1854,14 +1845,7 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
try {
s(event)
} catch {
case t: Throwable => event.stateData match {
case d: HasCommitments =>
handleLocalError(t, d, None)
case d: Data =>
log.error(t, "")
val error = Error(Helpers.getChannelId(d), t.getMessage.getBytes)
goto(CLOSED) sending error
}
case t: Throwable => handleLocalError(t, event.stateData, None)
}
}

View file

@ -20,6 +20,7 @@ import akka.actor.ActorRef
import fr.acinq.bitcoin.BinaryData
import fr.acinq.bitcoin.Crypto.PublicKey
import fr.acinq.eclair.ShortChannelId
import fr.acinq.eclair.channel.Channel.ChannelError
import fr.acinq.eclair.wire.{ChannelAnnouncement, ChannelUpdate}
/**
@ -48,3 +49,5 @@ case class ChannelSignatureReceived(channel: ActorRef, commitments: Commitments)
// NB: this event is only sent when the channel is available
case class AvailableBalanceChanged(channel: ActorRef, channelId: BinaryData, shortChannelId: ShortChannelId, localBalanceMsat: Long) extends ChannelEvent
case class ChannelFailed(channel: ActorRef, channelId: BinaryData, remoteNodeId: PublicKey, data: Data, error: ChannelError) extends ChannelEvent