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

Correctly translates and relay channels errors (#230)

* return `UnknownNextPeer` when next channel is unknown

* correctly relay channels errors (fixes #227)

* keeping `channel_update` in DATA_NORMAL

* moved class `ForwardLocalFail`->`AddHtlcFailed`

* made `AddHtlcFailed` a `ChannelException`

* unannounced channels now return `channel_update` in errors

* use remote `htlc_minimum_msat` in `channel_update`

also set default `htlc-minimum-msat`=`10000`

This fixes #248.

* Announce short channel id when funding depth ok

* added logs to router

* channel do not send `node_announcement` anymore
This commit is contained in:
Pierre-Marie Padiou 2017-12-06 13:32:06 +01:00 committed by GitHub
parent 0794fb8d5a
commit f47ea72369
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
15 changed files with 357 additions and 240 deletions

View File

@ -52,7 +52,7 @@ eclair {
default-feerate-per-kb = 20000 // default bitcoin core value
max-htlc-value-in-flight-msat = 100000000000 // 1 BTC ~= unlimited
htlc-minimum-msat = 1000000
htlc-minimum-msat = 10000
max-accepted-htlcs = 30
reserve-to-funding-ratio = 0.01 // recommended by BOLT #2
@ -63,7 +63,7 @@ eclair {
expiry-delta-blocks = 144
fee-base-msat = 546000
fee-proportional-millionth = 10
fee-proportional-millionths = 10
// maximum local vs remote feerate mismatch; 1.0 means 100%
// actual check is abs((local feerate - remote fee rate) / (local fee rate + remote fee rate)/2) > fee rate mismatch

View File

@ -53,7 +53,9 @@ case class NodeParams(extendedPrivateKey: ExtendedPrivateKey,
chainHash: BinaryData,
channelFlags: Byte,
channelExcludeDuration: FiniteDuration,
watcherType: WatcherType)
watcherType: WatcherType) {
val nodeId = privateKey.publicKey
}
object NodeParams {
@ -140,7 +142,7 @@ object NodeParams {
minDepthBlocks = config.getInt("mindepth-blocks"),
smartfeeNBlocks = 3,
feeBaseMsat = config.getInt("fee-base-msat"),
feeProportionalMillionth = config.getInt("fee-proportional-millionth"),
feeProportionalMillionth = config.getInt("fee-proportional-millionths"),
reserveToFundingRatio = config.getDouble("reserve-to-funding-ratio"),
maxReserveToFundingRatio = config.getDouble("max-reserve-to-funding-ratio"),
channelsDb = channelsDb,

View File

@ -11,7 +11,7 @@ import fr.acinq.eclair.blockchain._
import fr.acinq.eclair.channel.Helpers.{Closing, Funding}
import fr.acinq.eclair.crypto.{Generators, ShaChain, Sphinx}
import fr.acinq.eclair.payment._
import fr.acinq.eclair.router.{Announcements, TickBroadcast}
import fr.acinq.eclair.router.Announcements
import fr.acinq.eclair.transactions._
import fr.acinq.eclair.wire.{ChannelReestablish, _}
import org.bitcoinj.script.{Script => BitcoinjScript}
@ -139,15 +139,19 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
// no need to go OFFLINE, we can directly switch to CLOSING
goto(CLOSING) using closing
case d: HasCommitments =>
d match {
case DATA_NORMAL(_, Some(shortChannelId), _, _, _) =>
context.system.eventStream.publish(ShortChannelIdAssigned(self, d.channelId, shortChannelId))
val channelUpdate = Announcements.makeChannelUpdate(nodeParams.chainHash, nodeParams.privateKey, remoteNodeId, shortChannelId, nodeParams.expiryDeltaBlocks, nodeParams.htlcMinimumMsat, nodeParams.feeBaseMsat, nodeParams.feeProportionalMillionth, enable = false)
relayer ! channelUpdate
case _ => ()
case normal: DATA_NORMAL =>
context.system.eventStream.publish(ShortChannelIdAssigned(self, normal.channelId, normal.channelUpdate.shortChannelId))
// we rebuild a channel_update for two reasons:
// - we want to reload values from configuration
// - if eclair was previously killed, it might not have had time to publish a channel_update with enable=false
val channelUpdate = Announcements.makeChannelUpdate(nodeParams.chainHash, nodeParams.privateKey, remoteNodeId, normal.channelUpdate.shortChannelId, nodeParams.expiryDeltaBlocks, normal.commitments.remoteParams.htlcMinimumMsat, nodeParams.feeBaseMsat, nodeParams.feeProportionalMillionth, enable = false)
if (normal.channelAnnouncement.isDefined) {
router ! channelUpdate
}
goto(OFFLINE) using d
relayer ! channelUpdate
goto(OFFLINE) using normal.copy(channelUpdate = channelUpdate)
case _ => goto(OFFLINE) using data
}
})
@ -378,7 +382,11 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
val nextPerCommitmentPoint = Generators.perCommitPoint(commitments.localParams.shaSeed, 1)
val fundingLocked = FundingLocked(commitments.channelId, nextPerCommitmentPoint)
deferred.map(self ! _)
goto(WAIT_FOR_FUNDING_LOCKED) using store(DATA_WAIT_FOR_FUNDING_LOCKED(commitments, fundingLocked)) sending fundingLocked
// this is the temporary channel id that we will use in our channel_update message, the goal is to be able to use our channel
// as soon as it reaches NORMAL state, and before it is announced on the network
// (this id might be updated when the funding tx gets deeply buried, if there was a reorg in the meantime)
val shortChannelId = toShortId(blockHeight, txIndex, commitments.commitInput.outPoint.index.toInt)
goto(WAIT_FOR_FUNDING_LOCKED) using store(DATA_WAIT_FOR_FUNDING_LOCKED(commitments, shortChannelId, fundingLocked)) sending fundingLocked
case Event(BITCOIN_FUNDING_PUBLISH_FAILED, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) =>
val error = Error(d.channelId, "Funding tx publish failure".getBytes)
@ -390,7 +398,7 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
goto(ERR_FUNDING_TIMEOUT) sending error
case Event(remoteAnnSigs: AnnouncementSignatures, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) if d.commitments.announceChannel =>
log.info(s"received remote announcement signatures, delaying")
log.debug(s"received remote announcement signatures, delaying")
// we may receive their announcement sigs before our watcher notifies us that the channel has reached min_conf (especially during testing when blocks are generated in bulk)
// note: no need to persist their message, in case of disconnection they will resend it
context.system.scheduler.scheduleOnce(2 seconds, self, remoteAnnSigs)
@ -406,21 +414,24 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
})
when(WAIT_FOR_FUNDING_LOCKED)(handleExceptions {
case Event(FundingLocked(_, nextPerCommitmentPoint), d@DATA_WAIT_FOR_FUNDING_LOCKED(commitments, _)) =>
if (d.commitments.announceChannel) {
// used for announcement of channel (if minDepth >= ANNOUNCEMENTS_MINCONF this event will fire instantly)
blockchain ! WatchConfirmed(self, commitments.commitInput.outPoint.txid, commitments.commitInput.txOut.publicKeyScript, ANNOUNCEMENTS_MINCONF, BITCOIN_FUNDING_DEEPLYBURIED)
} else if (d.commitments.announceChannel && nodeParams.watcherType == BITCOINJ && d.commitments.localParams.isFunder && System.getProperty("spvtest") != null) {
case Event(FundingLocked(_, nextPerCommitmentPoint), d@DATA_WAIT_FOR_FUNDING_LOCKED(commitments, shortChannelId, _)) =>
if (d.commitments.announceChannel && nodeParams.watcherType == BITCOINJ && d.commitments.localParams.isFunder && System.getProperty("spvtest") != null) {
// bitcoinj-based watcher currently can't get the tx index in block (which is used to calculate the short id)
// instead, we rely on a hack by trusting the index the counterparty sends us
// but in testing when connecting to bitcoinj impl together we make the funder choose some random data
log.warning("using hardcoded short id for testing with bitcoinj!!!!!")
context.system.scheduler.scheduleOnce(5 seconds, self, WatchEventConfirmed(BITCOIN_FUNDING_DEEPLYBURIED, Random.nextInt(100), Random.nextInt(100)))
} else {
// used to get the final shortChannelId, used in announcements (if minDepth >= ANNOUNCEMENTS_MINCONF this event will fire instantly)
blockchain ! WatchConfirmed(self, commitments.commitInput.outPoint.txid, commitments.commitInput.txOut.publicKeyScript, ANNOUNCEMENTS_MINCONF, BITCOIN_FUNDING_DEEPLYBURIED)
}
goto(NORMAL) using store(DATA_NORMAL(commitments.copy(remoteNextCommitInfo = Right(nextPerCommitmentPoint)), None, None, None, None))
context.system.eventStream.publish(ShortChannelIdAssigned(self, commitments.channelId, shortChannelId))
val initialChannelUpdate = Announcements.makeChannelUpdate(nodeParams.chainHash, nodeParams.privateKey, remoteNodeId, shortChannelId, nodeParams.expiryDeltaBlocks, d.commitments.remoteParams.htlcMinimumMsat, nodeParams.feeBaseMsat, nodeParams.feeProportionalMillionth, enable = true)
relayer ! initialChannelUpdate
goto(NORMAL) using store(DATA_NORMAL(commitments.copy(remoteNextCommitInfo = Right(nextPerCommitmentPoint)), shortChannelId, None, initialChannelUpdate, None, None, None))
case Event(remoteAnnSigs: AnnouncementSignatures, d: DATA_WAIT_FOR_FUNDING_LOCKED) if d.commitments.announceChannel =>
log.info(s"received remote announcement signatures, delaying")
log.debug(s"received remote announcement signatures, delaying")
// we may receive their announcement sigs before our watcher notifies us that the channel has reached min_conf (especially during testing when blocks are generated in bulk)
// note: no need to persist their message, in case of disconnection they will resend it
context.system.scheduler.scheduleOnce(2 seconds, self, remoteAnnSigs)
@ -451,15 +462,15 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
// note: spec would allow us to keep sending new htlcs after having received their shutdown (and not sent ours)
// but we want to converge as fast as possible and they would probably not route them anyway
val error = ClosingInProgress(d.channelId)
handleCommandAddError(error, origin(c))
handleCommandError(AddHtlcFailed(d.channelId, error, origin(c), Some(d.channelUpdate)))
case Event(c: CMD_ADD_HTLC, d: DATA_NORMAL) =>
Try(Commitments.sendAdd(d.commitments, c, origin(c))) match {
case Success(Right((commitments1, add))) =>
if (c.commit) self ! CMD_SIGN
handleCommandSuccess(sender, d.copy(commitments = commitments1)) sending add
case Success(Left(error)) => handleCommandAddError(error, origin(c))
case Failure(cause) => handleCommandError(cause)
case Success(Left(error)) => handleCommandError(AddHtlcFailed(d.channelId, error, origin(c), Some(d.channelUpdate)))
case Failure(cause) => handleCommandError(AddHtlcFailed(d.channelId, cause, origin(c), Some(d.channelUpdate)))
}
case Event(add: UpdateAddHtlc, d: DATA_NORMAL) =>
@ -677,43 +688,21 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
case _ => stay
}
case Event(WatchEventConfirmed(BITCOIN_FUNDING_DEEPLYBURIED, blockHeight, txIndex), d: DATA_NORMAL) if d.commitments.announceChannel && d.shortChannelId.isEmpty =>
case Event(WatchEventConfirmed(BITCOIN_FUNDING_DEEPLYBURIED, blockHeight, txIndex), d: DATA_NORMAL) if d.channelAnnouncement.isEmpty =>
val shortChannelId = toShortId(blockHeight, txIndex, d.commitments.commitInput.outPoint.index.toInt)
log.info(s"funding tx is deeply buried at blockHeight=$blockHeight txIndex=$txIndex, sending announcements")
val annSignatures = Helpers.makeAnnouncementSignatures(nodeParams, d.commitments, shortChannelId)
stay using store(d.copy(localAnnouncementSignatures = Some(annSignatures))) sending annSignatures
log.info(s"funding tx is deeply buried at blockHeight=$blockHeight txIndex=$txIndex shortChannelId=$shortChannelId")
// we re-announce this shortChannelId, because it might be different from the one we were using before if there was a reorg
context.system.eventStream.publish(ShortChannelIdAssigned(self, d.channelId, shortChannelId))
val annSignatures_opt = if (d.commitments.announceChannel) Some(Helpers.makeAnnouncementSignatures(nodeParams, d.commitments, shortChannelId)) else None
stay using store(d.copy(shortChannelId = shortChannelId, localAnnouncementSignatures = annSignatures_opt)) sending annSignatures_opt.toSeq
case Event(remoteAnnSigs: AnnouncementSignatures, d: DATA_NORMAL) if d.commitments.announceChannel =>
// channels are publicly announced if both parties want it (defined as feature bit)
d.localAnnouncementSignatures match {
case Some(localAnnSigs) if d.shortChannelId.isDefined =>
// this can happen if our announcement_signatures was lost during a disconnection
// specs says that we "MUST respond to the first announcement_signatures message after reconnection with its own announcement_signatures message"
// current implementation always replies to announcement_signatures, not only the first time
log.info(s"re-sending our announcement sigs")
stay sending localAnnSigs
case Some(localAnnSigs) =>
require(localAnnSigs.shortChannelId == remoteAnnSigs.shortChannelId, s"shortChannelId mismatch: local=${localAnnSigs.shortChannelId} remote=${remoteAnnSigs.shortChannelId}")
log.info(s"announcing channelId=${d.channelId} on the network with shortId=${localAnnSigs.shortChannelId}")
import d.commitments.{localParams, remoteParams}
val channelAnn = Announcements.makeChannelAnnouncement(nodeParams.chainHash, localAnnSigs.shortChannelId, localParams.nodeId, remoteParams.nodeId, localParams.fundingPrivKey.publicKey, remoteParams.fundingPubKey, localAnnSigs.nodeSignature, remoteAnnSigs.nodeSignature, localAnnSigs.bitcoinSignature, remoteAnnSigs.bitcoinSignature)
val nodeAnn = Announcements.makeNodeAnnouncement(nodeParams.privateKey, nodeParams.alias, nodeParams.color, nodeParams.publicAddresses)
val channelUpdate = Announcements.makeChannelUpdate(nodeParams.chainHash, nodeParams.privateKey, remoteNodeId, localAnnSigs.shortChannelId, nodeParams.expiryDeltaBlocks, nodeParams.htlcMinimumMsat, nodeParams.feeBaseMsat, nodeParams.feeProportionalMillionth)
router ! channelAnn
router ! nodeAnn
router ! channelUpdate
relayer ! channelUpdate
// TODO: remove this later when we use testnet/mainnet
// let's trigger the broadcast immediately so that we don't wait for 60 seconds to announce our newly created channel
// we give 3 seconds for the router-watcher roundtrip
context.system.scheduler.scheduleOnce(3 seconds, router, TickBroadcast)
context.system.eventStream.publish(ShortChannelIdAssigned(self, d.channelId, localAnnSigs.shortChannelId))
// we acknowledge our AnnouncementSignatures message
stay using store(d.copy(shortChannelId = Some(localAnnSigs.shortChannelId))) // note: we don't clear our announcement sigs because we may need to re-send them
case None =>
log.info(s"received remote announcement signatures, delaying")
(d.localAnnouncementSignatures, d.channelAnnouncement) match {
case (None, _) =>
// our watcher didn't notify yet that the tx has reached ANNOUNCEMENTS_MINCONF confirmations, let's delay remote's message
// note: no need to persist their message, in case of disconnection they will resend it
log.debug(s"received remote announcement signatures, delaying")
context.system.scheduler.scheduleOnce(5 seconds, self, remoteAnnSigs)
if (nodeParams.watcherType == BITCOINJ) {
log.warning(s"HACK: since we cannot get the tx index with bitcoinj, we copy the value sent by remote")
@ -721,16 +710,33 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
self ! WatchEventConfirmed(BITCOIN_FUNDING_DEEPLYBURIED, blockHeight, txIndex)
}
stay
case (Some(localAnnSigs), None) =>
require(localAnnSigs.shortChannelId == remoteAnnSigs.shortChannelId, s"shortChannelId mismatch: local=${localAnnSigs.shortChannelId} remote=${remoteAnnSigs.shortChannelId}")
log.info(s"announcing channelId=${d.channelId} on the network with shortId=${localAnnSigs.shortChannelId}")
import d.commitments.{localParams, remoteParams}
val channelAnn = Announcements.makeChannelAnnouncement(nodeParams.chainHash, localAnnSigs.shortChannelId, localParams.nodeId, remoteParams.nodeId, localParams.fundingPrivKey.publicKey, remoteParams.fundingPubKey, localAnnSigs.nodeSignature, remoteAnnSigs.nodeSignature, localAnnSigs.bitcoinSignature, remoteAnnSigs.bitcoinSignature)
val channelUpdate = Announcements.makeChannelUpdate(nodeParams.chainHash, nodeParams.privateKey, remoteNodeId, d.shortChannelId, d.channelUpdate.cltvExpiryDelta, d.channelUpdate.htlcMinimumMsat, d.channelUpdate.feeBaseMsat, d.channelUpdate.feeProportionalMillionths, enable = true)
router ! channelAnn
router ! channelUpdate
relayer ! channelUpdate
stay using store(d.copy(channelAnnouncement = Some(channelAnn), channelUpdate = channelUpdate)) // note: we don't clear our announcement sigs because we may need to re-send them
case (Some(localAnnSigs), Some(_)) =>
// they send their announcement sigs, but we already have a valid channel annoucement
// this can happen if our announcement_signatures was lost during a disconnection
// specs says that we "MUST respond to the first announcement_signatures message after reconnection with its own announcement_signatures message"
// current implementation always replies to announcement_signatures, not only the first time
log.info(s"re-sending our announcement sigs")
stay sending localAnnSigs
}
case Event(TickRefreshChannelUpdate, d: DATA_NORMAL) if d.shortChannelId.isDefined =>
d.shortChannelId match {
case Some(shortChannelId) => // periodic refresh is used as a keep alive
val channelUpdate = Announcements.makeChannelUpdate(nodeParams.chainHash, nodeParams.privateKey, remoteNodeId, shortChannelId, nodeParams.expiryDeltaBlocks, nodeParams.htlcMinimumMsat, nodeParams.feeBaseMsat, nodeParams.feeProportionalMillionth)
router ! channelUpdate
stay
case None => stay // channel is not announced
case Event(TickRefreshChannelUpdate, d: DATA_NORMAL) =>
// periodic refresh is used as a keep alive
val channelUpdate = Announcements.makeChannelUpdate(nodeParams.chainHash, nodeParams.privateKey, remoteNodeId, d.shortChannelId, d.channelUpdate.cltvExpiryDelta, d.channelUpdate.htlcMinimumMsat, d.channelUpdate.feeBaseMsat, d.channelUpdate.feeProportionalMillionths, enable = true)
if (d.channelAnnouncement.isDefined) {
router ! channelUpdate
}
relayer ! channelUpdate
stay using d.copy(channelUpdate = channelUpdate)
case Event(WatchEventSpent(BITCOIN_FUNDING_SPENT, tx: Transaction), d: DATA_NORMAL) if tx.txid == d.commitments.remoteCommit.txid => handleRemoteSpentCurrent(tx, d)
@ -739,18 +745,17 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
case Event(WatchEventSpent(BITCOIN_FUNDING_SPENT, tx: Transaction), d: DATA_NORMAL) => handleRemoteSpentOther(tx, d)
case Event(INPUT_DISCONNECTED, d: DATA_NORMAL) =>
// we disable the channel
log.info(s"disabling the channel (disconnected)")
val channelUpdate = Announcements.makeChannelUpdate(nodeParams.chainHash, nodeParams.privateKey, remoteNodeId, d.shortChannelId, d.channelUpdate.cltvExpiryDelta, d.channelUpdate.htlcMinimumMsat, d.channelUpdate.feeBaseMsat, d.channelUpdate.feeProportionalMillionths, enable = false)
if (d.channelAnnouncement.isDefined) {
router ! channelUpdate
}
relayer ! channelUpdate
d.commitments.localChanges.proposed.collect {
case add: UpdateAddHtlc => relayer ! ForwardLocalFail(ChannelUnavailable(d.channelId), d.commitments.originChannels(add.id))
case add: UpdateAddHtlc => relayer ! AddHtlcFailed(d.channelId, ChannelUnavailable(d.channelId), d.commitments.originChannels(add.id), Some(channelUpdate))
}
d.shortChannelId match {
case Some(shortChannelId) =>
// if channel has be announced, we disable it
log.info(s"disabling the channel (disconnected)")
val channelUpdate = Announcements.makeChannelUpdate(nodeParams.chainHash, nodeParams.privateKey, remoteNodeId, shortChannelId, nodeParams.expiryDeltaBlocks, nodeParams.htlcMinimumMsat, nodeParams.feeBaseMsat, nodeParams.feeProportionalMillionth, enable = false)
router ! channelUpdate
case None => {}
}
goto(OFFLINE)
goto(OFFLINE) using d.copy(channelUpdate = channelUpdate)
case Event(e: Error, d: DATA_NORMAL) => handleRemoteError(e, d)
@ -1048,7 +1053,7 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
case add if ripemd160(add.paymentHash) == extracted =>
val origin = d.commitments.originChannels(add.id)
log.warning(s"found a match between paymentHash160=$extracted and origin=$origin: htlc timed out")
relayer ! ForwardLocalFail(HtlcTimedout(d.channelId), origin)
relayer ! AddHtlcFailed(d.channelId, HtlcTimedout(d.channelId), origin, None)
}
// TODO: should we handle local htlcs here as well? currently timed out htlcs that we sent will never have an answer
// TODO: we do not handle the case where htlcs transactions end up being unconfirmed this can happen if an htlc-success tx is published right before a htlc timed out
@ -1159,26 +1164,27 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
forwarder ! localShutdown
}
// even if we were just disconnected/reconnected, we need to put back the watch because the event may have been
// fired while we were in OFFLINE (if not, the operation is idempotent anyway)
// NB: in BITCOINJ mode we currently can't get the tx index in block (which is used to calculate the short id)
// instead, we rely on a hack by trusting the index the counterparty sends us
if (d.commitments.announceChannel && d.localAnnouncementSignatures.isEmpty && nodeParams.watcherType != BITCOINJ) {
blockchain ! WatchConfirmed(self, d.commitments.commitInput.outPoint.txid, d.commitments.commitInput.txOut.publicKeyScript, ANNOUNCEMENTS_MINCONF, BITCOIN_FUNDING_DEEPLYBURIED)
}
// rfc: a node SHOULD retransmit the announcement_signatures message if it has not received an announcement_signatures message
if (d.localAnnouncementSignatures.isDefined && d.shortChannelId.isEmpty) {
forwarder ! d.localAnnouncementSignatures.get
}
d.shortChannelId.map {
case shortChannelId =>
// we re-enable the channel
log.info(s"enabling the channel (reconnected)")
val channelUpdate = Announcements.makeChannelUpdate(nodeParams.chainHash, nodeParams.privateKey, remoteNodeId, shortChannelId, nodeParams.expiryDeltaBlocks, nodeParams.htlcMinimumMsat, nodeParams.feeBaseMsat, nodeParams.feeProportionalMillionth, enable = true)
val channelUpdate = Announcements.makeChannelUpdate(nodeParams.chainHash, nodeParams.privateKey, remoteNodeId, d.shortChannelId, nodeParams.expiryDeltaBlocks, d.commitments.remoteParams.htlcMinimumMsat, nodeParams.feeBaseMsat, nodeParams.feeProportionalMillionth, enable = true)
(d.localAnnouncementSignatures, d.channelAnnouncement) match {
case (None, None) if nodeParams.watcherType != BITCOINJ =>
// even if we were just disconnected/reconnected, we need to put back the watch because the event may have been
// fired while we were in OFFLINE (if not, the operation is idempotent anyway)
blockchain ! WatchConfirmed(self, d.commitments.commitInput.outPoint.txid, d.commitments.commitInput.txOut.publicKeyScript, ANNOUNCEMENTS_MINCONF, BITCOIN_FUNDING_DEEPLYBURIED)
case (None, None) if nodeParams.watcherType == BITCOINJ =>
// NB: in BITCOINJ mode we currently can't get the tx index in block (which is used to calculate the short id)
// instead, we rely on a hack by trusting the index the counterparty sends us)
case (Some(localAnnSigs), None) =>
// BOLT 7: a node SHOULD retransmit the announcement_signatures message if it has not received an announcement_signatures message
forwarder ! localAnnSigs
case (_, Some(channelAnn)) =>
// we might have been down for a long time (more than 2 weeks) and other nodes in the network might have forgotten the channel
log.info(s"re-announcing channelId=${d.channelId} on the network with shortId=${channelAnn.shortChannelId}")
router ! channelAnn
router ! channelUpdate
}
goto(NORMAL) using d.copy(commitments = commitments1)
relayer ! channelUpdate
goto(NORMAL) using d.copy(commitments = commitments1, channelUpdate = channelUpdate)
case Event(channelReestablish: ChannelReestablish, d: DATA_SHUTDOWN) =>
val commitments1 = handleSync(channelReestablish, d)
@ -1239,7 +1245,10 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
case Event(c: CMD_ADD_HTLC, d: HasCommitments) =>
log.info(s"rejecting htlc request in state=$stateName")
val error = ChannelUnavailable(d.channelId)
handleCommandAddError(error, origin(c))
d match {
case normal: DATA_NORMAL => handleCommandError(AddHtlcFailed(d.channelId, error, origin(c), Some(normal.channelUpdate))) // can happen if we are in OFFLINE or SYNCING state (channelUpdate will have enable=false)
case _ => handleCommandError(AddHtlcFailed(d.channelId, error, origin(c), None))
}
// we only care about this event in NORMAL and SHUTDOWN state, and we never unregister to the event stream
case Event(CurrentBlockCount(_), _) => stay
@ -1281,15 +1290,6 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
stay using newData replying "ok"
}
def handleCommandAddError(cause: Throwable, origin: Origin) = {
relayer ! ForwardLocalFail(cause, origin)
cause match {
case _: ChannelException => log.error(s"$cause")
case _ => log.error(cause, "")
}
stay
}
def handleCommandError(cause: Throwable) = {
cause match {
case _: ChannelException => log.error(s"$cause")

View File

@ -2,47 +2,50 @@ package fr.acinq.eclair.channel
import fr.acinq.bitcoin.BinaryData
import fr.acinq.eclair.UInt64
import fr.acinq.eclair.payment.Origin
import fr.acinq.eclair.wire.ChannelUpdate
/**
* Created by PM on 11/04/2017.
*/
class ChannelException(channelId: BinaryData, message: String) extends RuntimeException(message)
class ChannelException(val channelId: BinaryData, message: String) extends RuntimeException(message)
// @formatter:off
case class DebugTriggeredException (channelId: BinaryData) extends ChannelException(channelId, "debug-mode triggered failure")
case class InvalidChainHash (channelId: BinaryData, local: BinaryData, remote: BinaryData) extends ChannelException(channelId, s"invalid chain_hash (local=$local remote=$remote)")
case class InvalidFundingAmount (channelId: BinaryData, fundingSatoshis: Long, min: Long, max: Long) extends ChannelException(channelId, s"invalid funding_satoshis=$fundingSatoshis (min=$min max=$max)")
case class InvalidPushAmount (channelId: BinaryData, pushMsat: Long, max: Long) extends ChannelException(channelId, s"invalid push_msat=$pushMsat (max=$max)")
case class InvalidMaxAcceptedHtlcs (channelId: BinaryData, maxAcceptedHtlcs: Int, max: Int) extends ChannelException(channelId, s"invalid max_accepted_htlcs=$maxAcceptedHtlcs (max=$max)")
case class InvalidDustLimit (channelId: BinaryData, dustLimitSatoshis: Long, min: Long) extends ChannelException(channelId, s"invalid dust_limit_satoshis=$dustLimitSatoshis (min=$min)")
case class ChannelReserveTooHigh (channelId: BinaryData, channelReserveSatoshis: Long, reserveToFundingRatio: Double, maxReserveToFundingRatio: Double) extends ChannelException(channelId, s"channelReserveSatoshis too high: reserve=$channelReserveSatoshis fundingRatio=$reserveToFundingRatio maxFundingRatio=$maxReserveToFundingRatio")
case class ClosingInProgress (channelId: BinaryData) extends ChannelException(channelId, "cannot send new htlcs, closing in progress")
case class ClosingAlreadyInProgress (channelId: BinaryData) extends ChannelException(channelId, "closing already in progress")
case class CannotCloseWithUnsignedOutgoingHtlcs(channelId: BinaryData) extends ChannelException(channelId, "cannot close when there are unsigned outgoing htlcs")
case class ChannelUnavailable (channelId: BinaryData) extends ChannelException(channelId, "channel is unavailable (offline or closing)")
case class InvalidFinalScript (channelId: BinaryData) extends ChannelException(channelId, "invalid final script")
case class HtlcTimedout (channelId: BinaryData) extends ChannelException(channelId, s"one or more htlcs timed out")
case class FeerateTooDifferent (channelId: BinaryData, localFeeratePerKw: Long, remoteFeeratePerKw: Long) extends ChannelException(channelId, s"local/remote feerates are too different: remoteFeeratePerKw=$remoteFeeratePerKw localFeeratePerKw=$localFeeratePerKw")
case class InvalidCloseSignature (channelId: BinaryData) extends ChannelException(channelId, "cannot verify their close signature")
case class InvalidCommitmentSignature (channelId: BinaryData) extends ChannelException(channelId, "invalid commitment signature")
case class ForcedLocalCommit (channelId: BinaryData, reason: String) extends ChannelException(channelId, s"forced local commit: reason")
case class UnexpectedHtlcId (channelId: BinaryData, expected: Long, actual: Long) extends ChannelException(channelId, s"unexpected htlc id: expected=$expected actual=$actual")
case class InvalidPaymentHash (channelId: BinaryData) extends ChannelException(channelId, "invalid payment hash")
case class ExpiryTooSmall (channelId: BinaryData, minimum: Long, actual: Long, blockCount: Long) extends ChannelException(channelId, s"expiry too small: required=$minimum actual=$actual blockCount=$blockCount")
case class ExpiryCannotBeInThePast (channelId: BinaryData, expiry: Long, blockCount: Long) extends ChannelException(channelId, s"expiry can't be in the past: expiry=$expiry blockCount=$blockCount")
case class HtlcValueTooSmall (channelId: BinaryData, minimum: Long, actual: Long) extends ChannelException(channelId, s"htlc value too small: minimum=$minimum actual=$actual")
case class HtlcValueTooHighInFlight (channelId: BinaryData, maximum: UInt64, actual: UInt64) extends ChannelException(channelId, s"in-flight htlcs hold too much value: maximum=$maximum actual=$actual")
case class TooManyAcceptedHtlcs (channelId: BinaryData, maximum: Long) extends ChannelException(channelId, s"too many accepted htlcs: maximum=$maximum")
case class InsufficientFunds (channelId: BinaryData, amountMsat: Long, missingSatoshis: Long, reserveSatoshis: Long, feesSatoshis: Long) extends ChannelException(channelId, s"insufficient funds: missingSatoshis=$missingSatoshis reserveSatoshis=$reserveSatoshis fees=$feesSatoshis")
case class InvalidHtlcPreimage (channelId: BinaryData, id: Long) extends ChannelException(channelId, s"invalid htlc preimage for htlc id=$id")
case class UnknownHtlcId (channelId: BinaryData, id: Long) extends ChannelException(channelId, s"unknown htlc id=$id")
case class FundeeCannotSendUpdateFee (channelId: BinaryData) extends ChannelException(channelId, s"only the funder should send update_fee messages")
case class CannotAffordFees (channelId: BinaryData, missingSatoshis: Long, reserveSatoshis: Long, feesSatoshis: Long) extends ChannelException(channelId, s"can't pay the fee: missingSatoshis=$missingSatoshis reserveSatoshis=$reserveSatoshis feesSatoshis=$feesSatoshis")
case class CannotSignWithoutChanges (channelId: BinaryData) extends ChannelException(channelId, "cannot sign when there are no changes")
case class CannotSignBeforeRevocation (channelId: BinaryData) extends ChannelException(channelId, "cannot sign until next revocation hash is received")
case class UnexpectedRevocation (channelId: BinaryData) extends ChannelException(channelId, "received unexpected RevokeAndAck message")
case class InvalidRevocation (channelId: BinaryData) extends ChannelException(channelId, "invalid revocation")
case class CommitmentSyncError (channelId: BinaryData) extends ChannelException(channelId, "commitment sync error")
case class RevocationSyncError (channelId: BinaryData) extends ChannelException(channelId, "revocation sync error")
case class InvalidFailureCode (channelId: BinaryData) extends ChannelException(channelId, "UpdateFailMalformedHtlc message doesn't have BADONION bit set")
// @formatter:on
case class DebugTriggeredException (override val channelId: BinaryData) extends ChannelException(channelId, "debug-mode triggered failure")
case class InvalidChainHash (override val channelId: BinaryData, local: BinaryData, remote: BinaryData) extends ChannelException(channelId, s"invalid chain_hash (local=$local remote=$remote)")
case class InvalidFundingAmount (override val channelId: BinaryData, fundingSatoshis: Long, min: Long, max: Long) extends ChannelException(channelId, s"invalid funding_satoshis=$fundingSatoshis (min=$min max=$max)")
case class InvalidPushAmount (override val channelId: BinaryData, pushMsat: Long, max: Long) extends ChannelException(channelId, s"invalid push_msat=$pushMsat (max=$max)")
case class InvalidMaxAcceptedHtlcs (override val channelId: BinaryData, maxAcceptedHtlcs: Int, max: Int) extends ChannelException(channelId, s"invalid max_accepted_htlcs=$maxAcceptedHtlcs (max=$max)")
case class InvalidDustLimit (override val channelId: BinaryData, dustLimitSatoshis: Long, min: Long) extends ChannelException(channelId, s"invalid dust_limit_satoshis=$dustLimitSatoshis (min=$min)")
case class ChannelReserveTooHigh (override val channelId: BinaryData, channelReserveSatoshis: Long, reserveToFundingRatio: Double, maxReserveToFundingRatio: Double) extends ChannelException(channelId, s"channelReserveSatoshis too high: reserve=$channelReserveSatoshis fundingRatio=$reserveToFundingRatio maxFundingRatio=$maxReserveToFundingRatio")
case class ClosingInProgress (override val channelId: BinaryData) extends ChannelException(channelId, "cannot send new htlcs, closing in progress")
case class ClosingAlreadyInProgress (override val channelId: BinaryData) extends ChannelException(channelId, "closing already in progress")
case class CannotCloseWithUnsignedOutgoingHtlcs(override val channelId: BinaryData) extends ChannelException(channelId, "cannot close when there are unsigned outgoing htlcs")
case class ChannelUnavailable (override val channelId: BinaryData) extends ChannelException(channelId, "channel is unavailable (offline or closing)")
case class InvalidFinalScript (override val channelId: BinaryData) extends ChannelException(channelId, "invalid final script")
case class HtlcTimedout (override val channelId: BinaryData) extends ChannelException(channelId, s"one or more htlcs timed out")
case class FeerateTooDifferent (override val channelId: BinaryData, localFeeratePerKw: Long, remoteFeeratePerKw: Long) extends ChannelException(channelId, s"local/remote feerates are too different: remoteFeeratePerKw=$remoteFeeratePerKw localFeeratePerKw=$localFeeratePerKw")
case class InvalidCloseSignature (override val channelId: BinaryData) extends ChannelException(channelId, "cannot verify their close signature")
case class InvalidCommitmentSignature (override val channelId: BinaryData) extends ChannelException(channelId, "invalid commitment signature")
case class ForcedLocalCommit (override val channelId: BinaryData, reason: String) extends ChannelException(channelId, s"forced local commit: reason")
case class UnexpectedHtlcId (override val channelId: BinaryData, expected: Long, actual: Long) extends ChannelException(channelId, s"unexpected htlc id: expected=$expected actual=$actual")
case class InvalidPaymentHash (override val channelId: BinaryData) extends ChannelException(channelId, "invalid payment hash")
case class ExpiryTooSmall (override val channelId: BinaryData, minimum: Long, actual: Long, blockCount: Long) extends ChannelException(channelId, s"expiry too small: required=$minimum actual=$actual blockCount=$blockCount")
case class ExpiryCannotBeInThePast (override val channelId: BinaryData, expiry: Long, blockCount: Long) extends ChannelException(channelId, s"expiry can't be in the past: expiry=$expiry blockCount=$blockCount")
case class HtlcValueTooSmall (override val channelId: BinaryData, minimum: Long, actual: Long) extends ChannelException(channelId, s"htlc value too small: minimum=$minimum actual=$actual")
case class HtlcValueTooHighInFlight (override val channelId: BinaryData, maximum: UInt64, actual: UInt64) extends ChannelException(channelId, s"in-flight htlcs hold too much value: maximum=$maximum actual=$actual")
case class TooManyAcceptedHtlcs (override val channelId: BinaryData, maximum: Long) extends ChannelException(channelId, s"too many accepted htlcs: maximum=$maximum")
case class InsufficientFunds (override val channelId: BinaryData, amountMsat: Long, missingSatoshis: Long, reserveSatoshis: Long, feesSatoshis: Long) extends ChannelException(channelId, s"insufficient funds: missingSatoshis=$missingSatoshis reserveSatoshis=$reserveSatoshis fees=$feesSatoshis")
case class InvalidHtlcPreimage (override val channelId: BinaryData, id: Long) extends ChannelException(channelId, s"invalid htlc preimage for htlc id=$id")
case class UnknownHtlcId (override val channelId: BinaryData, id: Long) extends ChannelException(channelId, s"unknown htlc id=$id")
case class FundeeCannotSendUpdateFee (override val channelId: BinaryData) extends ChannelException(channelId, s"only the funder should send update_fee messages")
case class CannotAffordFees (override val channelId: BinaryData, missingSatoshis: Long, reserveSatoshis: Long, feesSatoshis: Long) extends ChannelException(channelId, s"can't pay the fee: missingSatoshis=$missingSatoshis reserveSatoshis=$reserveSatoshis feesSatoshis=$feesSatoshis")
case class CannotSignWithoutChanges (override val channelId: BinaryData) extends ChannelException(channelId, "cannot sign when there are no changes")
case class CannotSignBeforeRevocation (override val channelId: BinaryData) extends ChannelException(channelId, "cannot sign until next revocation hash is received")
case class UnexpectedRevocation (override val channelId: BinaryData) extends ChannelException(channelId, "received unexpected RevokeAndAck message")
case class InvalidRevocation (override val channelId: BinaryData) extends ChannelException(channelId, "invalid revocation")
case class CommitmentSyncError (override val channelId: BinaryData) extends ChannelException(channelId, "commitment sync error")
case class RevocationSyncError (override val channelId: BinaryData) extends ChannelException(channelId, "revocation sync error")
case class InvalidFailureCode (override val channelId: BinaryData) extends ChannelException(channelId, "UpdateFailMalformedHtlc message doesn't have BADONION bit set")
case class AddHtlcFailed (override val channelId: BinaryData, t: Throwable, origin: Origin, channelUpdate: Option[ChannelUpdate]) extends ChannelException(channelId, s"cannot add htlc with origin=$origin reason=${t.getMessage}")
// @formatter:on

View File

@ -7,7 +7,7 @@ import fr.acinq.eclair.UInt64
import fr.acinq.eclair.crypto.Sphinx
import fr.acinq.eclair.transactions.CommitmentSpec
import fr.acinq.eclair.transactions.Transactions.CommitTx
import fr.acinq.eclair.wire.{AcceptChannel, AnnouncementSignatures, ClosingSigned, FailureMessage, FundingCreated, FundingLocked, FundingSigned, Init, OpenChannel, Shutdown, UpdateAddHtlc}
import fr.acinq.eclair.wire.{AcceptChannel, AnnouncementSignatures, ChannelAnnouncement, ChannelUpdate, ClosingSigned, FailureMessage, FundingCreated, FundingLocked, FundingSigned, Init, OpenChannel, Shutdown, UpdateAddHtlc}
/**
@ -133,9 +133,11 @@ final case class DATA_WAIT_FOR_FUNDING_INTERNAL(temporaryChannelId: BinaryData,
final case class DATA_WAIT_FOR_FUNDING_CREATED(temporaryChannelId: BinaryData, localParams: LocalParams, remoteParams: RemoteParams, fundingSatoshis: Long, pushMsat: Long, initialFeeratePerKw: Long, remoteFirstPerCommitmentPoint: Point, channelFlags: Byte, lastSent: AcceptChannel) extends Data
final case class DATA_WAIT_FOR_FUNDING_SIGNED(channelId: BinaryData, localParams: LocalParams, remoteParams: RemoteParams, fundingTx: Transaction, localSpec: CommitmentSpec, localCommitTx: CommitTx, remoteCommit: RemoteCommit, channelFlags: Byte, lastSent: FundingCreated) extends Data
final case class DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments: Commitments, deferred: Option[FundingLocked], lastSent: Either[FundingCreated, FundingSigned]) extends Data with HasCommitments
final case class DATA_WAIT_FOR_FUNDING_LOCKED(commitments: Commitments, lastSent: FundingLocked) extends Data with HasCommitments
final case class DATA_WAIT_FOR_FUNDING_LOCKED(commitments: Commitments, shortChannelId: Long, lastSent: FundingLocked) extends Data with HasCommitments
final case class DATA_NORMAL(commitments: Commitments,
shortChannelId: Option[Long],
shortChannelId: Long,
channelAnnouncement: Option[ChannelAnnouncement],
channelUpdate: ChannelUpdate,
localAnnouncementSignatures: Option[AnnouncementSignatures],
localShutdown: Option[Shutdown],
remoteShutdown: Option[Shutdown]) extends Data with HasCommitments

View File

@ -56,12 +56,7 @@ class PaymentLifecycle(sourceNodeId: PublicKey, router: ActorRef, register: Acto
val firstHop = hops.head
val finalExpiry = Globals.blockCount.get().toInt + c.minFinalCltvExpiry.toInt
val (cmd, sharedSecrets) = buildCommand(c.amountMsat, finalExpiry, c.paymentHash, hops)
// TODO: HACK!!!! see Router.scala (we actually store the first node id in the sig)
if (firstHop.lastUpdate.signature.size == 32) {
register ! Register.Forward(firstHop.lastUpdate.signature, cmd)
} else {
register ! Register.ForwardShortId(firstHop.lastUpdate.shortChannelId, cmd)
}
register ! Register.ForwardShortId(firstHop.lastUpdate.shortChannelId, cmd)
goto(WAITING_FOR_PAYMENT_COMPLETE) using WaitingForComplete(s, c, cmd, failures, sharedSecrets, ignoreNodes, ignoreChannels, hops)
case Event(Status.Failure(t), WaitingForRoute(s, c, failures)) =>

View File

@ -20,7 +20,6 @@ case class Relayed(originChannelId: BinaryData, originHtlcId: Long, amountMsatIn
case class ForwardAdd(add: UpdateAddHtlc)
case class ForwardFulfill(fulfill: UpdateFulfillHtlc, to: Origin)
case class ForwardLocalFail(error: Throwable, to: Origin) // happens when the failure happened in a local channel (and not in some downstream channel)
case class ForwardFail(fail: UpdateFailHtlc, to: Origin)
case class ForwardFailMalformed(fail: UpdateFailMalformedHtlc, to: Origin)
@ -45,7 +44,7 @@ class Relayer(nodeParams: NodeParams, register: ActorRef, paymentHandler: ActorR
case ChannelStateChanged(channel, _, _, _, NORMAL | SHUTDOWN | CLOSING, d: HasCommitments) =>
import d.channelId
preimagesDb.listPreimages(channelId) match {
case Nil => {}
case Nil => ()
case preimages =>
log.info(s"re-sending ${preimages.size} unacked fulfills to channel $channelId")
preimages.map(p => CMD_FULFILL_HTLC(p._2, p._3, commit = false)).foreach(channel ! _)
@ -75,11 +74,11 @@ class Relayer(nodeParams: NodeParams, register: ActorRef, paymentHandler: ActorR
paymentHandler forward add
}
case Success((Attempt.Successful(DecodeResult(perHopPayload, _)), nextPacket, _)) =>
val channelUpdate_opt = channelUpdates.get(perHopPayload.channel_id)
channelUpdate_opt match {
channelUpdates.get(perHopPayload.channel_id) match {
case None =>
// TODO: clarify what we're supposed to do in the specs
sender ! CMD_FAIL_HTLC(add.id, Right(TemporaryNodeFailure), commit = true)
// if we don't (yet?) have a channel_update for the next channel, we consider the channel doesn't exist
// TODO: use a different channel to the same peer instead?
sender ! CMD_FAIL_HTLC(add.id, Right(UnknownNextPeer), commit = true)
case Some(channelUpdate) if !Announcements.isEnabled(channelUpdate.flags) =>
sender ! CMD_FAIL_HTLC(add.id, Right(ChannelDisabled(channelUpdate.flags, channelUpdate)), commit = true)
case Some(channelUpdate) if add.amountMsat < channelUpdate.htlcMinimumMsat =>
@ -90,7 +89,7 @@ class Relayer(nodeParams: NodeParams, register: ActorRef, paymentHandler: ActorR
sender ! CMD_FAIL_HTLC(add.id, Right(ExpiryTooSoon(channelUpdate)), commit = true)
case _ =>
log.info(s"forwarding htlc #${add.id} to shortChannelId=${perHopPayload.channel_id}")
register forward Register.ForwardShortId(perHopPayload.channel_id, CMD_ADD_HTLC(perHopPayload.amtToForward, add.paymentHash, perHopPayload.outgoingCltvValue, nextPacket.serialize, upstream_opt = Some(add), commit = true))
register ! Register.ForwardShortId(perHopPayload.channel_id, CMD_ADD_HTLC(perHopPayload.amtToForward, add.paymentHash, perHopPayload.outgoingCltvValue, nextPacket.serialize, upstream_opt = Some(add), commit = true))
}
case Success((Attempt.Failure(cause), _, _)) =>
log.error(s"couldn't parse payload: $cause")
@ -105,6 +104,20 @@ class Relayer(nodeParams: NodeParams, register: ActorRef, paymentHandler: ActorR
log.warning(s"couldn't resolve downstream channel $shortChannelId, failing htlc #${add.id}")
register ! Register.Forward(add.channelId, CMD_FAIL_HTLC(add.id, Right(UnknownNextPeer), commit = true))
case AddHtlcFailed(_, error, Local(Some(sender)), _) =>
sender ! Status.Failure(error)
case AddHtlcFailed(_, error, Relayed(originChannelId, originHtlcId, _, _), channelUpdate_opt) =>
val failure = (error, channelUpdate_opt) match {
case (_: ChannelUnavailable, Some(channelUpdate)) if !Announcements.isEnabled(channelUpdate.flags) => ChannelDisabled(channelUpdate.flags, channelUpdate)
case (_: InsufficientFunds, Some(channelUpdate)) => TemporaryChannelFailure(channelUpdate)
case (_: TooManyAcceptedHtlcs, Some(channelUpdate)) => TemporaryChannelFailure(channelUpdate)
case (_: HtlcTimedout, _) => PermanentChannelFailure
case _ => TemporaryNodeFailure
}
val cmd = CMD_FAIL_HTLC(originHtlcId, Right(failure), commit = true)
register ! Register.Forward(originChannelId, cmd)
case ForwardFulfill(fulfill, Local(Some(sender))) =>
sender ! fulfill
@ -119,18 +132,6 @@ class Relayer(nodeParams: NodeParams, register: ActorRef, paymentHandler: ActorR
log.debug(s"fulfill acked for channelId=$channelId htlcId=$htlcId")
preimagesDb.removePreimage(channelId, htlcId)
case ForwardLocalFail(error, Local(Some(sender))) =>
sender ! Status.Failure(error)
case ForwardLocalFail(error, Relayed(originChannelId, originHtlcId, _, _)) =>
// TODO: clarify what we're supposed to do in the specs depending on the error
val failure = error match {
case HtlcTimedout(_) => PermanentChannelFailure
case _ => TemporaryNodeFailure
}
val cmd = CMD_FAIL_HTLC(originHtlcId, Right(failure), commit = true)
register ! Register.Forward(originChannelId, cmd)
case ForwardFail(fail, Local(Some(sender))) =>
sender ! fail

View File

@ -41,7 +41,7 @@ case class Data(nodes: Map[PublicKey, NodeAnnouncement],
stash: Seq[RoutingMessage],
awaiting: Seq[ChannelAnnouncement],
origins: Map[RoutingMessage, ActorRef],
localChannels: Map[BinaryData, PublicKey],
localUpdates: Map[BinaryData, (ChannelDesc, ChannelUpdate)],
excludedChannels: Set[ChannelDesc]) // those channels are temporarily excluded from route calculation, because their node returned a TemporaryChannelFailure
sealed trait State
@ -79,10 +79,6 @@ class Router(nodeParams: NodeParams, watcher: ActorRef) extends FSM[State, Data]
db.listChannels().map(self ! _)
db.listNodes().map(self ! _)
db.listChannelUpdates().map(self ! _)
if (db.listChannels().size > 0) {
val nodeAnn = Announcements.makeNodeAnnouncement(nodeParams.privateKey, nodeParams.alias, nodeParams.color, nodeParams.publicAddresses, Platform.currentTime / 1000)
self ! nodeAnn
}
log.info(s"starting state machine")
startWith(NORMAL, Data(Map.empty, Map.empty, Map.empty, Nil, Nil, Nil, Map.empty, Map.empty, Set.empty))
@ -107,7 +103,7 @@ class Router(nodeParams: NodeParams, watcher: ActorRef) extends FSM[State, Data]
when(WAITING_FOR_VALIDATION) {
case Event(ParallelGetResponse(results), d) =>
val validated = results.map {
val validated = results.flatMap {
case IndividualResult(c, Some(tx), true) =>
// TODO: blacklisting
val (_, _, outputIndex) = fromShortId(c.shortChannelId)
@ -137,8 +133,18 @@ class Router(nodeParams: NodeParams, watcher: ActorRef) extends FSM[State, Data]
// TODO: blacklist?
log.warning(s"could not retrieve tx for shortChannelId=${c.shortChannelId}")
None
}.flatten
// we reprocess node and channel-update announcements that may have been validated
}
// in case we just validated our first local channel, we announce the local node
// note that this will also make sure we always update our node announcement on restart (eg: alias, color), because
// even if we had stored a previous announcement, it would be overriden by this more recent one
if (!d.nodes.contains(nodeParams.nodeId) && validated.exists(isRelatedTo(_, nodeParams.nodeId))) {
log.info(s"first local channel validated, announcing local node")
val nodeAnn = Announcements.makeNodeAnnouncement(nodeParams.privateKey, nodeParams.alias, nodeParams.color, nodeParams.publicAddresses)
self ! nodeAnn
}
// we also reprocess node and channel_update announcements related to channels that were processed
val (resend, stash1) = d.stash.partition {
case n: NodeAnnouncement => results.exists(r => isRelatedTo(r.c, n.nodeId))
case u: ChannelUpdate => results.exists(r => r.c.shortChannelId == u.shortChannelId)
@ -150,10 +156,13 @@ class Router(nodeParams: NodeParams, watcher: ActorRef) extends FSM[State, Data]
whenUnhandled {
case Event(ChannelStateChanged(_, _, _, _, channel.NORMAL, d: DATA_NORMAL), d1) =>
stay using d1.copy(localChannels = d1.localChannels + (d.commitments.channelId -> d.commitments.remoteParams.nodeId))
val channelDesc = ChannelDesc(d.channelUpdate.shortChannelId, d.commitments.localParams.nodeId, d.commitments.remoteParams.nodeId)
log.debug(s"added local channel_update for channelId=${d.channelId} update=${d.channelUpdate}")
stay using d1.copy(localUpdates = d1.localUpdates + (d.channelId -> (channelDesc, d.channelUpdate)))
case Event(ChannelStateChanged(_, _, _, channel.NORMAL, _, d: DATA_NEGOTIATING), d1) =>
stay using d1.copy(localChannels = d1.localChannels - d.commitments.channelId)
case Event(ChannelStateChanged(_, _, _, channel.NORMAL, _, d: HasCommitments), d1) =>
log.debug(s"removed local channel_update for channelId=${d.channelId}")
stay using d1.copy(localUpdates = d1.localUpdates - d.channelId)
case Event(_: ChannelStateChanged, _) => stay
@ -316,21 +325,13 @@ class Router(nodeParams: NodeParams, watcher: ActorRef) extends FSM[State, Data]
stay
case Event(RouteRequest(start, end, ignoreNodes, ignoreChannels), d) =>
val localNodeId = nodeParams.privateKey.publicKey
// TODO: HACK!!!!! the following is a workaround to make our routing work with private/not-yet-announced channels, that do not have a channelUpdate
val fakeUpdates = d.localChannels.map { case (channelId, remoteNodeId) =>
// note that this id is deterministic, otherwise filterUpdates would not work
val fakeShortId = BigInt(channelId.take(7).toArray).toLong
val channelDesc = ChannelDesc(fakeShortId, localNodeId, remoteNodeId)
// note that we store the channelId in the sig, other values are not used because if it is selected this will be the first channel in the route
val channelUpdate = ChannelUpdate(signature = channelId, chainHash = nodeParams.chainHash, fakeShortId, 0, "0000", 0, 0, 0, 0)
(channelDesc -> channelUpdate)
}
// we replace local channelUpdates (we have them for regular public already-announced channels) by the ones we just generated
val updates1 = d.updates.filterKeys(_.a != localNodeId) ++ fakeUpdates
// we start with channel_updates of local channels
val updates0 = d.localUpdates.values.toMap
// we add them to the publicly-announced updates (channel_updates for announced channels will be deduped)
val updates1 = d.updates ++ updates0
// we then filter out the currently excluded channels
val updates2 = updates1.filterKeys(!d.excludedChannels.contains(_))
// we also filter out excluded channels
// we also filter out disabled channels, and channels/nodes that are blacklisted for this particular request
val updates3 = filterUpdates(updates2, ignoreNodes, ignoreChannels)
log.info(s"finding a route $start->$end with ignoreNodes=${ignoreNodes.map(_.toBin).mkString(",")} ignoreChannels=${ignoreChannels.map(_.toHexString).mkString(",")}")
findRoute(start, end, updates3).map(r => RouteResponse(r, ignoreNodes, ignoreChannels)) pipeTo sender
@ -383,7 +384,7 @@ object Router {
updates
.filterNot(u => ignoreNodes.map(_.toBin).contains(u._1.a) || ignoreNodes.map(_.toBin).contains(u._1.b))
.filterNot(u => ignoreChannels.contains(u._1.id))
.filterNot(u => !Announcements.isEnabled(u._2.flags))
.filter(u => Announcements.isEnabled(u._2.flags))
def findRouteDijkstra(localNodeId: PublicKey, targetNodeId: PublicKey, channels: Iterable[ChannelDesc]): Seq[ChannelDesc] = {
if (localNodeId == targetNodeId) throw CannotRouteToSelf

View File

@ -196,11 +196,14 @@ object ChannelCodecs {
val DATA_WAIT_FOR_FUNDING_LOCKED_Codec: Codec[DATA_WAIT_FOR_FUNDING_LOCKED] = (
("commitments" | commitmentsCodec) ::
("shortChannelId" | uint64) ::
("lastSent" | fundingLockedCodec)).as[DATA_WAIT_FOR_FUNDING_LOCKED]
val DATA_NORMAL_Codec: Codec[DATA_NORMAL] = (
("commitments" | commitmentsCodec) ::
("shortChannelId" | optional(bool, uint64)) ::
("shortChannelId" | uint64) ::
("channelAnnouncement" | optional(bool, channelAnnouncementCodec)) ::
("channelUpdate" | channelUpdateCodec) ::
("localAnnouncementSignatures" | optional(bool, announcementSignaturesCodec)) ::
("localShutdown" | optional(bool, shutdownCodec)) ::
("remoteShutdown" | optional(bool, shutdownCodec))).as[DATA_NORMAL]

View File

@ -77,6 +77,8 @@ trait StateTestsHelperMethods extends TestKitBase {
alice2bob.forward(bob)
bob2alice.expectMsgType[FundingLocked]
bob2alice.forward(alice)
alice2blockchain.expectMsgType[WatchConfirmed] // deeply buried
bob2blockchain.expectMsgType[WatchConfirmed] // deeply buried
awaitCond(alice.stateName == NORMAL)
awaitCond(bob.stateName == NORMAL)
}

View File

@ -4,15 +4,16 @@ import akka.actor.Status.Failure
import akka.testkit.{TestFSMRef, TestProbe}
import fr.acinq.bitcoin.Crypto.Scalar
import fr.acinq.bitcoin.{BinaryData, Crypto, Satoshi, ScriptFlags, Transaction}
import fr.acinq.eclair.TestConstants.Bob
import fr.acinq.eclair.TestConstants.{Alice, Bob}
import fr.acinq.eclair.UInt64.Conversions._
import fr.acinq.eclair.blockchain._
import fr.acinq.eclair.blockchain.fee.FeeratesPerKw
import fr.acinq.eclair.channel.states.StateTestsHelperMethods
import fr.acinq.eclair.channel.{Data, State, _}
import fr.acinq.eclair.payment._
import fr.acinq.eclair.router.Announcements
import fr.acinq.eclair.transactions.{IN, OUT}
import fr.acinq.eclair.wire.{AnnouncementSignatures, ClosingSigned, CommitSig, Error, FailureMessageCodecs, PermanentChannelFailure, RevokeAndAck, Shutdown, UpdateAddHtlc, UpdateFailHtlc, UpdateFailMalformedHtlc, UpdateFee, UpdateFulfillHtlc}
import fr.acinq.eclair.wire.{AnnouncementSignatures, ChannelUpdate, ClosingSigned, CommitSig, Error, FailureMessageCodecs, PermanentChannelFailure, RevokeAndAck, Shutdown, UpdateAddHtlc, UpdateFailHtlc, UpdateFailMalformedHtlc, UpdateFee, UpdateFulfillHtlc}
import fr.acinq.eclair.{Globals, TestConstants, TestkitBaseClass}
import org.junit.runner.RunWith
import org.scalatest.Tag
@ -33,6 +34,8 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
import setup._
within(30 seconds) {
reachNormal(alice, bob, alice2bob, bob2alice, alice2blockchain, bob2blockchain, test.tags)
relayer.expectMsgType[ChannelUpdate]
relayer.expectMsgType[ChannelUpdate]
awaitCond(alice.stateName == NORMAL)
awaitCond(bob.stateName == NORMAL)
}
@ -90,14 +93,14 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
}
}
test("recv CMD_ADD_HTLC (invalid payment hash)") { case (alice, _, alice2bob, _, _, _, relayer) =>
test("recv CMD_ADD_HTLC (invalid payment hash)") { case (alice, _, alice2bob, _, _, _, _) =>
within(30 seconds) {
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
val sender = TestProbe()
val add = CMD_ADD_HTLC(500000000, "11" * 42, expiry = 400144)
sender.send(alice, add)
val error = InvalidPaymentHash(channelId(alice))
//sender.expectMsg(Failure(InvalidPaymentHash(channelId(alice))))
relayer.expectMsg(ForwardLocalFail(error, Local(Some(sender.ref))))
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), error, Local(Some(sender.ref)), Some(initialState.channelUpdate))))
alice2bob.expectNoMsg(200 millis)
}
}
@ -105,11 +108,11 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
test("recv CMD_ADD_HTLC (expiry too small)") { case (alice, _, alice2bob, _, _, _, relayer) =>
within(30 seconds) {
val sender = TestProbe()
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
val add = CMD_ADD_HTLC(500000000, "11" * 32, expiry = 300000)
sender.send(alice, add)
val error = ExpiryCannotBeInThePast(channelId(alice), 300000, 400000)
//sender.expectMsg(Failure(error))
relayer.expectMsg(ForwardLocalFail(error, Local(Some(sender.ref))))
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), error, Local(Some(sender.ref)), Some(initialState.channelUpdate))))
alice2bob.expectNoMsg(200 millis)
}
}
@ -117,11 +120,11 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
test("recv CMD_ADD_HTLC (value too small)") { case (alice, _, alice2bob, _, _, _, relayer) =>
within(30 seconds) {
val sender = TestProbe()
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
val add = CMD_ADD_HTLC(50, "11" * 32, 400144)
sender.send(alice, add)
val error = HtlcValueTooSmall(channelId(alice), 1000, 50)
//sender.expectMsg(Failure(error))
relayer.expectMsg(ForwardLocalFail(error, Local(Some(sender.ref))))
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), error, Local(Some(sender.ref)), Some(initialState.channelUpdate))))
alice2bob.expectNoMsg(200 millis)
}
}
@ -129,11 +132,11 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
test("recv CMD_ADD_HTLC (insufficient funds)") { case (alice, _, alice2bob, _, _, _, relayer) =>
within(30 seconds) {
val sender = TestProbe()
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
val add = CMD_ADD_HTLC(Int.MaxValue, "11" * 32, 400144)
sender.send(alice, add)
val error = InsufficientFunds(channelId(alice), amountMsat = Int.MaxValue, missingSatoshis = 1376443, reserveSatoshis = 20000, feesSatoshis = 8960)
//sender.expectMsg(Failure(error))
relayer.expectMsg(ForwardLocalFail(error, Local(Some(sender.ref))))
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), error, Local(Some(sender.ref)), Some(initialState.channelUpdate))))
alice2bob.expectNoMsg(200 millis)
}
}
@ -141,6 +144,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
test("recv CMD_ADD_HTLC (insufficient funds w/ pending htlcs and 0 balance)") { case (alice, _, alice2bob, _, _, _, relayer) =>
within(30 seconds) {
val sender = TestProbe()
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
sender.send(alice, CMD_ADD_HTLC(500000000, "11" * 32, 400144))
sender.expectMsg("ok")
alice2bob.expectMsgType[UpdateAddHtlc]
@ -153,8 +157,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val add = CMD_ADD_HTLC(1000000, "44" * 32, 400144)
sender.send(alice, add)
val error = InsufficientFunds(channelId(alice), amountMsat = 1000000, missingSatoshis = 1000, reserveSatoshis = 20000, feesSatoshis = 12400)
//sender.expectMsg(Failure(error))
relayer.expectMsg(ForwardLocalFail(error, Local(Some(sender.ref))))
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), error, Local(Some(sender.ref)), Some(initialState.channelUpdate))))
alice2bob.expectNoMsg(200 millis)
}
}
@ -162,6 +165,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
test("recv CMD_ADD_HTLC (insufficient funds w/ pending htlcs 2/2)") { case (alice, _, alice2bob, _, _, _, relayer) =>
within(30 seconds) {
val sender = TestProbe()
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
sender.send(alice, CMD_ADD_HTLC(300000000, "11" * 32, 400144))
sender.expectMsg("ok")
alice2bob.expectMsgType[UpdateAddHtlc]
@ -171,8 +175,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val add = CMD_ADD_HTLC(500000000, "33" * 32, 400144)
sender.send(alice, add)
val error = InsufficientFunds(channelId(alice), amountMsat = 500000000, missingSatoshis = 332400, reserveSatoshis = 20000, feesSatoshis = 12400)
//sender.expectMsg(Failure(error))
relayer.expectMsg(ForwardLocalFail(error, Local(Some(sender.ref))))
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), error, Local(Some(sender.ref)), Some(initialState.channelUpdate))))
alice2bob.expectNoMsg(200 millis)
}
}
@ -180,11 +183,11 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
test("recv CMD_ADD_HTLC (over max inflight htlc value)") { case (_, bob, _, bob2alice, _, _, relayer) =>
within(30 seconds) {
val sender = TestProbe()
val initialState = bob.stateData.asInstanceOf[DATA_NORMAL]
val add = CMD_ADD_HTLC(151000000, "11" * 32, 400144)
sender.send(bob, add)
val error = HtlcValueTooHighInFlight(channelId(bob), maximum = 150000000, actual = 151000000)
//sender.expectMsg(Failure(error))
relayer.expectMsg(ForwardLocalFail(error, Local(Some(sender.ref))))
sender.expectMsg(Failure(AddHtlcFailed(channelId(bob), error, Local(Some(sender.ref)), Some(initialState.channelUpdate))))
bob2alice.expectNoMsg(200 millis)
}
}
@ -192,6 +195,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
test("recv CMD_ADD_HTLC (over max accepted htlcs)") { case (alice, _, alice2bob, _, _, _, relayer) =>
within(30 seconds) {
val sender = TestProbe()
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
// Bob accepts a maximum of 30 htlcs
for (i <- 0 until 30) {
sender.send(alice, CMD_ADD_HTLC(10000000, "11" * 32, 400144))
@ -201,15 +205,15 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val add = CMD_ADD_HTLC(10000000, "33" * 32, 400144)
sender.send(alice, add)
val error = TooManyAcceptedHtlcs(channelId(alice), maximum = 30)
//sender.expectMsg(Failure(error))
relayer.expectMsg(ForwardLocalFail(error, Local(Some(sender.ref))))
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), error, Local(Some(sender.ref)), Some(initialState.channelUpdate))))
alice2bob.expectNoMsg(200 millis)
}
}
test("recv CMD_ADD_HTLC (while waiting for a revocation)") { case (alice, _, alice2bob, _, _, _, relayer) =>
test("recv CMD_ADD_HTLC (over capacity)") { case (alice, _, alice2bob, _, _, _, relayer) =>
within(30 seconds) {
val sender = TestProbe()
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
val add1 = CMD_ADD_HTLC(TestConstants.fundingSatoshis * 2 / 3 * 1000, "11" * 32, 400144)
sender.send(alice, add1)
sender.expectMsg("ok")
@ -220,8 +224,8 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
// this is over channel-capacity
val add2 = CMD_ADD_HTLC(TestConstants.fundingSatoshis * 2 / 3 * 1000, "22" * 32, 400144)
sender.send(alice, add2)
//sender.expectMsgType[Failure]
relayer.expectMsgType[ForwardLocalFail]
val error = InsufficientFunds(channelId(alice), add2.amountMsat, 564012, 20000, 10680)
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), error, Local(Some(sender.ref)), Some(initialState.channelUpdate))))
alice2bob.expectNoMsg(200 millis)
}
}
@ -229,6 +233,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
test("recv CMD_ADD_HTLC (after having sent Shutdown)") { case (alice, _, alice2bob, _, _, _, relayer) =>
within(30 seconds) {
val sender = TestProbe()
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
sender.send(alice, CMD_CLOSE(None))
sender.expectMsg("ok")
alice2bob.expectMsgType[Shutdown]
@ -238,8 +243,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val add = CMD_ADD_HTLC(500000000, "11" * 32, expiry = 400144)
sender.send(alice, add)
val error = ClosingInProgress(channelId(alice))
//sender.expectMsg(Failure(error))
relayer.expectMsg(ForwardLocalFail(error, Local(Some(sender.ref))))
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), error, Local(Some(sender.ref)), Some(initialState.channelUpdate))))
alice2bob.expectNoMsg(200 millis)
}
}
@ -247,6 +251,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
test("recv CMD_ADD_HTLC (after having received Shutdown)") { case (alice, bob, alice2bob, bob2alice, _, _, relayer) =>
within(30 seconds) {
val sender = TestProbe()
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
// let's make alice send an htlc
val add1 = CMD_ADD_HTLC(500000000, "11" * 32, expiry = 400144)
sender.send(alice, add1)
@ -263,8 +268,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
bob2alice.forward(alice)
sender.send(alice, add2)
val error = ClosingInProgress(channelId(alice))
//sender.expectMsg(Failure(error))
relayer.expectMsg(ForwardLocalFail(error, Local(Some(sender.ref))))
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), error, Local(Some(sender.ref)), Some(initialState.channelUpdate))))
}
}
@ -1665,7 +1669,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val sender = TestProbe()
sender.send(alice, WatchEventConfirmed(BITCOIN_FUNDING_DEEPLYBURIED, 42, 10))
val annSigs = alice2bob.expectMsgType[AnnouncementSignatures]
assert(alice.stateData.asInstanceOf[DATA_NORMAL] === initialState.copy(localAnnouncementSignatures = Some(annSigs)))
assert(alice.stateData.asInstanceOf[DATA_NORMAL] === initialState.copy(shortChannelId = annSigs.shortChannelId, localAnnouncementSignatures = Some(annSigs)))
}
}
@ -1677,9 +1681,13 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val annSigsA = alice2bob.expectMsgType[AnnouncementSignatures]
sender.send(bob, WatchEventConfirmed(BITCOIN_FUNDING_DEEPLYBURIED, 42, 10))
val annSigsB = bob2alice.expectMsgType[AnnouncementSignatures]
import initialState.commitments.localParams
import initialState.commitments.remoteParams
val channelAnn = Announcements.makeChannelAnnouncement(Alice.nodeParams.chainHash, annSigsA.shortChannelId, localParams.nodeId, remoteParams.nodeId, localParams.fundingPrivKey.publicKey, remoteParams.fundingPubKey, annSigsA.nodeSignature, annSigsB.nodeSignature, annSigsA.bitcoinSignature, annSigsB.bitcoinSignature)
val channelUpdate = Announcements.makeChannelUpdate(Alice.nodeParams.chainHash, Alice.nodeParams.privateKey, remoteParams.nodeId, annSigsA.shortChannelId, Alice.nodeParams.expiryDeltaBlocks, Bob.nodeParams.htlcMinimumMsat, Alice.nodeParams.feeBaseMsat, Alice.nodeParams.feeProportionalMillionth)
// actual test starts here
bob2alice.forward(alice)
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL] == initialState.copy(shortChannelId = Some(annSigsB.shortChannelId), localAnnouncementSignatures = Some(annSigsA)))
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL] == initialState.copy(shortChannelId = annSigsA.shortChannelId, channelAnnouncement = Some(channelAnn), channelUpdate = channelUpdate, localAnnouncementSignatures = Some(annSigsA)))
}
}
@ -1691,8 +1699,12 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
val annSigsA = alice2bob.expectMsgType[AnnouncementSignatures]
sender.send(bob, WatchEventConfirmed(BITCOIN_FUNDING_DEEPLYBURIED, 42, 10))
val annSigsB = bob2alice.expectMsgType[AnnouncementSignatures]
import initialState.commitments.localParams
import initialState.commitments.remoteParams
val channelAnn = Announcements.makeChannelAnnouncement(Alice.nodeParams.chainHash, annSigsA.shortChannelId, localParams.nodeId, remoteParams.nodeId, localParams.fundingPrivKey.publicKey, remoteParams.fundingPubKey, annSigsA.nodeSignature, annSigsB.nodeSignature, annSigsA.bitcoinSignature, annSigsB.bitcoinSignature)
val channelUpdate = Announcements.makeChannelUpdate(Alice.nodeParams.chainHash, Alice.nodeParams.privateKey, remoteParams.nodeId, annSigsA.shortChannelId, Alice.nodeParams.expiryDeltaBlocks, Bob.nodeParams.htlcMinimumMsat, Alice.nodeParams.feeBaseMsat, Alice.nodeParams.feeProportionalMillionth)
bob2alice.forward(alice)
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL] == initialState.copy(shortChannelId = Some(annSigsB.shortChannelId), localAnnouncementSignatures = Some(annSigsA)))
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL] == initialState.copy(shortChannelId = annSigsA.shortChannelId, channelAnnouncement = Some(channelAnn), channelUpdate = channelUpdate, localAnnouncementSignatures = Some(annSigsA)))
// actual test starts here
// simulate bob re-sending its sigs

View File

@ -8,8 +8,8 @@ import fr.acinq.eclair.blockchain._
import fr.acinq.eclair.blockchain.fee.FeeratesPerKw
import fr.acinq.eclair.channel.states.StateTestsHelperMethods
import fr.acinq.eclair.channel.{Data, State, _}
import fr.acinq.eclair.payment.{ForwardAdd, ForwardLocalFail, Local, PaymentLifecycle, _}
import fr.acinq.eclair.wire.{CommitSig, Error, FailureMessageCodecs, PermanentChannelFailure, RevokeAndAck, Shutdown, UpdateAddHtlc, UpdateFailHtlc, UpdateFailMalformedHtlc, UpdateFee, UpdateFulfillHtlc}
import fr.acinq.eclair.payment.{ForwardAdd, Local, PaymentLifecycle, _}
import fr.acinq.eclair.wire.{ChannelUpdate, CommitSig, Error, FailureMessageCodecs, PermanentChannelFailure, RevokeAndAck, Shutdown, UpdateAddHtlc, UpdateFailHtlc, UpdateFailMalformedHtlc, UpdateFee, UpdateFulfillHtlc}
import fr.acinq.eclair.{Globals, TestConstants, TestkitBaseClass}
import org.junit.runner.RunWith
import org.scalatest.junit.JUnitRunner
@ -29,6 +29,8 @@ class ShutdownStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
import setup._
within(30 seconds) {
reachNormal(alice, bob, alice2bob, bob2alice, alice2blockchain, bob2blockchain)
relayer.expectMsgType[ChannelUpdate]
relayer.expectMsgType[ChannelUpdate]
val sender = TestProbe()
// alice sends an HTLC to bob
val r1: BinaryData = "11" * 32
@ -78,14 +80,13 @@ class ShutdownStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
}
}
test("recv CMD_ADD_HTLC") { case (alice, _, alice2bob, _, _, _, relayer) =>
test("recv CMD_ADD_HTLC") { case (alice, _, alice2bob, _, _, _, _) =>
within(30 seconds) {
val sender = TestProbe()
val add = CMD_ADD_HTLC(500000000, "11" * 32, expiry = 300000)
sender.send(alice, add)
val error = ChannelUnavailable(channelId(alice))
//sender.expectMsg(Failure(error))
relayer.expectMsg(ForwardLocalFail(error, Local(Some(sender.ref))))
sender.expectMsg(Failure(AddHtlcFailed(channelId(alice), error, Local(Some(sender.ref)), None)))
alice2bob.expectNoMsg(200 millis)
}
}

View File

@ -28,7 +28,8 @@ class ClosingStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
import setup._
within(30 seconds) {
reachNormal(alice, bob, alice2bob, bob2alice, alice2blockchain, bob2blockchain)
relayer.expectMsgType[ChannelUpdate]
relayer.expectMsgType[ChannelUpdate]
val bobCommitTxes: List[Transaction] = (for (amt <- List(100000000, 200000000, 300000000)) yield {
val (r, htlc) = addHtlc(amt, alice, bob, alice2bob, bob2alice)
crossSign(alice, bob, alice2bob, bob2alice)

View File

@ -6,9 +6,10 @@ import fr.acinq.eclair.channel.Helpers.Funding
import fr.acinq.eclair.channel._
import fr.acinq.eclair.crypto.{ShaChain, Sphinx}
import fr.acinq.eclair.payment.{Local, Relayed}
import fr.acinq.eclair.router.Announcements
import fr.acinq.eclair.transactions.Transactions.CommitTx
import fr.acinq.eclair.transactions._
import fr.acinq.eclair.wire.{ChannelCodecs, UpdateAddHtlc}
import fr.acinq.eclair.wire.{ChannelCodecs, ChannelUpdate, UpdateAddHtlc}
import fr.acinq.eclair.{UInt64, randomKey}
import org.junit.runner.RunWith
import org.scalatest.FunSuite
@ -96,5 +97,7 @@ object ChannelStateSpec {
remoteNextCommitInfo = Right(randomKey.publicKey),
commitInput = commitmentInput, remotePerCommitmentSecrets = ShaChain.init, channelId = "00" * 32)
val normal = DATA_NORMAL(commitments, Some(42), None, None, None)
val channelUpdate = Announcements.makeChannelUpdate("11" * 32, randomKey, randomKey.publicKey, 142553, 42, 15, 575, 53)
val normal = DATA_NORMAL(commitments, 42, None, channelUpdate, None, None, None)
}

View File

@ -7,8 +7,9 @@ import fr.acinq.bitcoin.{BinaryData, Crypto, MilliSatoshi}
import fr.acinq.eclair.channel._
import fr.acinq.eclair.crypto.Sphinx
import fr.acinq.eclair.payment.PaymentLifecycle.buildCommand
import fr.acinq.eclair.router.Announcements
import fr.acinq.eclair.wire._
import fr.acinq.eclair.{TestConstants, TestkitBaseClass}
import fr.acinq.eclair.{Globals, TestConstants, TestkitBaseClass}
import org.junit.runner.RunWith
import org.scalatest.junit.JUnitRunner
@ -64,7 +65,7 @@ class RelayerSpec extends TestkitBaseClass {
paymentHandler.expectNoMsg(500 millis)
}
test("fail to relay an htlc-add when there is no available upstream channel") { case (relayer, register, paymentHandler) =>
test("fail to relay an htlc-add when we have no channel_update for the next channel") { case (relayer, register, paymentHandler) =>
val sender = TestProbe()
// we use this to build a valid onion
@ -76,15 +77,33 @@ class RelayerSpec extends TestkitBaseClass {
val fail = sender.expectMsgType[CMD_FAIL_HTLC]
assert(fail.id === add_ab.id)
assert(fail.reason == Right(UnknownNextPeer))
register.expectNoMsg(500 millis)
paymentHandler.expectNoMsg(500 millis)
}
test("fail to relay an htlc-add when the requested channel is disabled") { case (relayer, register, paymentHandler) =>
val sender = TestProbe()
// we use this to build a valid onion
val (cmd, _) = buildCommand(finalAmountMsat, finalExpiry, paymentHash, hops)
// and then manually build an htlc
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.expiry, cmd.onion)
val channelUpdate_bc_disabled = channelUpdate_bc.copy(flags = Announcements.makeFlags(Announcements.isNode1(channelUpdate_bc.flags), enable = false))
relayer ! channelUpdate_bc_disabled
sender.send(relayer, ForwardAdd(add_ab))
val fail = sender.expectMsgType[CMD_FAIL_HTLC]
assert(fail.id === add_ab.id)
assert(fail.reason == Right(ChannelDisabled(channelUpdate_bc_disabled.flags, channelUpdate_bc_disabled)))
register.expectNoMsg(500 millis)
paymentHandler.expectNoMsg(500 millis)
}
test("fail to relay an htlc-add when the onion is malformed") { case (relayer, register, paymentHandler) =>
// TODO: we should use the new update_fail_malformed_htlc message (see BOLT 2)
val sender = TestProbe()
// we use this to build a valid onion
@ -107,7 +126,7 @@ class RelayerSpec extends TestkitBaseClass {
val sender = TestProbe()
// we use this to build a valid onion
val (cmd, secrets) = buildCommand(channelUpdate_bc.htlcMinimumMsat - 1, finalExpiry, paymentHash, hops.map(hop => hop.copy(lastUpdate = hop.lastUpdate.copy(feeBaseMsat = 0, feeProportionalMillionths = 0))))
val (cmd, _) = buildCommand(channelUpdate_bc.htlcMinimumMsat - 1, finalExpiry, paymentHash, hops.map(hop => hop.copy(lastUpdate = hop.lastUpdate.copy(feeBaseMsat = 0, feeProportionalMillionths = 0))))
// and then manually build an htlc
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.expiry, cmd.onion)
relayer ! channelUpdate_bc
@ -126,7 +145,7 @@ class RelayerSpec extends TestkitBaseClass {
val sender = TestProbe()
val hops1 = hops.updated(1, hops(1).copy(lastUpdate = hops(1).lastUpdate.copy(cltvExpiryDelta = 0)))
val (cmd, secrets) = buildCommand(finalAmountMsat, finalExpiry, paymentHash, hops1)
val (cmd, _) = buildCommand(finalAmountMsat, finalExpiry, paymentHash, hops1)
// and then manually build an htlc
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.expiry, cmd.onion)
relayer ! channelUpdate_bc
@ -144,7 +163,7 @@ class RelayerSpec extends TestkitBaseClass {
test("fail to relay an htlc-add when expiry is too soon") { case (relayer, register, paymentHandler) =>
val sender = TestProbe()
val (cmd, secrets) = buildCommand(finalAmountMsat, 0, paymentHash, hops)
val (cmd, _) = buildCommand(finalAmountMsat, 0, paymentHash, hops)
// and then manually build an htlc
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.expiry, cmd.onion)
relayer ! channelUpdate_bc
@ -164,7 +183,7 @@ class RelayerSpec extends TestkitBaseClass {
// to simulate this we use a zero-hop route A->B where A is the 'attacker'
val hops1 = hops.head :: Nil
val (cmd, secrets) = buildCommand(finalAmountMsat, finalExpiry, paymentHash, hops1)
val (cmd, _) = buildCommand(finalAmountMsat, finalExpiry, paymentHash, hops1)
// and then manually build an htlc with a wrong expiry
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat - 1, cmd.paymentHash, cmd.expiry, cmd.onion)
relayer ! channelUpdate_bc
@ -184,7 +203,7 @@ class RelayerSpec extends TestkitBaseClass {
// to simulate this we use a zero-hop route A->B where A is the 'attacker'
val hops1 = hops.head :: Nil
val (cmd, secrets) = buildCommand(finalAmountMsat, finalExpiry, paymentHash, hops1)
val (cmd, _) = buildCommand(finalAmountMsat, finalExpiry, paymentHash, hops1)
// and then manually build an htlc with a wrong expiry
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.expiry - 1, cmd.onion)
relayer ! channelUpdate_bc
@ -199,6 +218,78 @@ class RelayerSpec extends TestkitBaseClass {
paymentHandler.expectNoMsg(500 millis)
}
test("fail to relay an htlc-add when next channel's balance is too low") { case (relayer, register, paymentHandler) =>
val sender = TestProbe()
val (cmd, _) = buildCommand(finalAmountMsat, Globals.blockCount.get().toInt + 10, paymentHash, hops)
// and then manually build an htlc
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.expiry, cmd.onion)
relayer ! channelUpdate_bc
sender.send(relayer, ForwardAdd(add_ab))
val fwd = register.expectMsgType[Register.ForwardShortId[CMD_ADD_HTLC]]
assert(fwd.shortChannelId === channelUpdate_bc.shortChannelId)
assert(fwd.message.upstream_opt === Some(add_ab))
sender.send(relayer, AddHtlcFailed(channelId_bc, new InsufficientFunds(channelId_bc, cmd.amountMsat, 100, 0, 0), Relayed(add_ab.channelId, add_ab.id, add_ab.amountMsat, cmd.amountMsat), Some(channelUpdate_bc)))
val fail = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]].message
assert(fail.id === add_ab.id)
assert(fail.reason == Right(TemporaryChannelFailure(channelUpdate_bc)))
register.expectNoMsg(500 millis)
paymentHandler.expectNoMsg(500 millis)
}
test("fail to relay an htlc-add when next channel has too many inflight htlcs") { case (relayer, register, paymentHandler) =>
val sender = TestProbe()
val (cmd, _) = buildCommand(finalAmountMsat, Globals.blockCount.get().toInt + 10, paymentHash, hops)
// and then manually build an htlc
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.expiry, cmd.onion)
relayer ! channelUpdate_bc
sender.send(relayer, ForwardAdd(add_ab))
val fwd = register.expectMsgType[Register.ForwardShortId[CMD_ADD_HTLC]]
assert(fwd.shortChannelId === channelUpdate_bc.shortChannelId)
assert(fwd.message.upstream_opt === Some(add_ab))
sender.send(relayer, AddHtlcFailed(channelId_bc, new TooManyAcceptedHtlcs(channelId_bc, 30), Relayed(add_ab.channelId, add_ab.id, add_ab.amountMsat, cmd.amountMsat), Some(channelUpdate_bc)))
val fail = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]].message
assert(fail.id === add_ab.id)
assert(fail.reason == Right(TemporaryChannelFailure(channelUpdate_bc)))
register.expectNoMsg(500 millis)
paymentHandler.expectNoMsg(500 millis)
}
test("fail to relay an htlc-add when next channel has a timed out htlc (and is thus closing)") { case (relayer, register, paymentHandler) =>
val sender = TestProbe()
val (cmd, _) = buildCommand(finalAmountMsat, Globals.blockCount.get().toInt + 10, paymentHash, hops)
// and then manually build an htlc
val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.expiry, cmd.onion)
relayer ! channelUpdate_bc
sender.send(relayer, ForwardAdd(add_ab))
val fwd = register.expectMsgType[Register.ForwardShortId[CMD_ADD_HTLC]]
assert(fwd.shortChannelId === channelUpdate_bc.shortChannelId)
assert(fwd.message.upstream_opt === Some(add_ab))
sender.send(relayer, AddHtlcFailed(channelId_bc, new HtlcTimedout(channelId_bc), Relayed(add_ab.channelId, add_ab.id, add_ab.amountMsat, cmd.amountMsat), Some(channelUpdate_bc)))
val fail = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]].message
assert(fail.id === add_ab.id)
assert(fail.reason == Right(PermanentChannelFailure))
register.expectNoMsg(500 millis)
paymentHandler.expectNoMsg(500 millis)
}
test("relay an htlc-fulfill") { case (relayer, register, _) =>
val sender = TestProbe()
val eventListener = TestProbe()