1
0
Fork 0
mirror of https://github.com/ACINQ/eclair.git synced 2025-02-23 06:35:11 +01:00

Minor fixes (#452)

* removed unused `WAITING_FOR_VALIDATION` state

* unwatch connection when peer is dequeued from `initial_routing_sync` waitlist

* Updated testnet electrum servers list

* Added error log in supervisor when throwable is caught

The default error logging does not log the stack.

* filter updates in one pass

* filter stale channels in one pass

* added `strategy` to supervisor logs

* add the downstream htlc to `Forward*` messages
This commit is contained in:
Pierre-Marie Padiou 2018-02-20 18:02:28 +01:00 committed by GitHub
parent 6b837ecbae
commit 226aeb56d8
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
8 changed files with 71 additions and 54 deletions

View file

@ -3,8 +3,13 @@
"t": "51001",
"s": "51002"
},
"electrum.akinbo.org": {
"testnet.hsmiths.com": {
"t": "53011",
"s": "53012"
},
"testnet.qtornado.com": {
"t": "51001",
"s": "51002"
}
}

View file

@ -19,7 +19,12 @@ class SimpleSupervisor(childProps: Props, childName: String, strategy: Superviso
}
// we allow at most <maxNrOfRetries> within <withinTimeRange>, otherwise the child actor is not restarted (this avoids restart loops)
override val supervisorStrategy = OneForOneStrategy(loggingEnabled = true, maxNrOfRetries = 100, withinTimeRange = 1 minute) { case _ => strategy }
override val supervisorStrategy = OneForOneStrategy(loggingEnabled = false, maxNrOfRetries = 100, withinTimeRange = 1 minute) {
case t =>
// log this as silent errors are dangerous
log.error(t, s"supervisor caught error for child=$childName strategy=$strategy ")
strategy
}
}
object SimpleSupervisor {

View file

@ -500,8 +500,8 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
case Event(fulfill: UpdateFulfillHtlc, d: DATA_NORMAL) =>
Try(Commitments.receiveFulfill(d.commitments, fulfill)) match {
case Success(Right((commitments1, origin))) =>
relayer ! ForwardFulfill(fulfill, origin)
case Success(Right((commitments1, origin, htlc))) =>
relayer ! ForwardFulfill(fulfill, origin, htlc)
stay using d.copy(commitments = commitments1)
case Success(Left(_)) => stay
case Failure(cause) => handleLocalError(cause, d, Some(fulfill))
@ -525,8 +525,8 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
case Event(fail: UpdateFailHtlc, d: DATA_NORMAL) =>
Try(Commitments.receiveFail(d.commitments, fail)) match {
case Success(Right((commitments1, origin))) =>
relayer ! ForwardFail(fail, origin)
case Success(Right((commitments1, origin, htlc))) =>
relayer ! ForwardFail(fail, origin, htlc)
stay using d.copy(commitments = commitments1)
case Success(Left(_)) => stay
case Failure(cause) => handleLocalError(cause, d, Some(fail))
@ -534,8 +534,8 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
case Event(fail: UpdateFailMalformedHtlc, d: DATA_NORMAL) =>
Try(Commitments.receiveFailMalformed(d.commitments, fail)) match {
case Success(Right((commitments1, origin))) =>
relayer ! ForwardFailMalformed(fail, origin)
case Success(Right((commitments1, origin, htlc))) =>
relayer ! ForwardFailMalformed(fail, origin, htlc)
stay using d.copy(commitments = commitments1)
case Success(Left(_)) => stay
case Failure(cause) => handleLocalError(cause, d, Some(fail))
@ -801,8 +801,8 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
case Event(fulfill: UpdateFulfillHtlc, d: DATA_SHUTDOWN) =>
Try(Commitments.receiveFulfill(d.commitments, fulfill)) match {
case Success(Right((commitments1, origin))) =>
relayer ! ForwardFulfill(fulfill, origin)
case Success(Right((commitments1, origin, htlc))) =>
relayer ! ForwardFulfill(fulfill, origin, htlc)
stay using d.copy(commitments = commitments1)
case Success(Left(_)) => stay
case Failure(cause) => handleLocalError(cause, d, Some(fulfill))
@ -826,8 +826,8 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
case Event(fail: UpdateFailHtlc, d: DATA_SHUTDOWN) =>
Try(Commitments.receiveFail(d.commitments, fail)) match {
case Success(Right((commitments1, origin))) =>
relayer ! ForwardFail(fail, origin)
case Success(Right((commitments1, origin, htlc))) =>
relayer ! ForwardFail(fail, origin, htlc)
stay using d.copy(commitments = commitments1)
case Success(Left(_)) => stay
case Failure(cause) => handleLocalError(cause, d, Some(fail))
@ -835,8 +835,8 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
case Event(fail: UpdateFailMalformedHtlc, d: DATA_SHUTDOWN) =>
Try(Commitments.receiveFailMalformed(d.commitments, fail)) match {
case Success(Right((commitments1, origin))) =>
relayer ! ForwardFailMalformed(fail, origin)
case Success(Right((commitments1, origin, htlc))) =>
relayer ! ForwardFailMalformed(fail, origin, htlc)
stay using d.copy(commitments = commitments1)
case Success(Left(_)) => stay
case Failure(cause) => handleLocalError(cause, d, Some(fail))
@ -1042,11 +1042,11 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
// we watch it in order to extract payment preimage if funds are pulled by the counterparty
// we can then use these preimages to fulfill origin htlcs
log.warning(s"processing BITCOIN_OUTPUT_SPENT with txid=${tx.txid} tx=$tx")
val fulfills = Closing.extractPreimages(d.commitments.localCommit, tx)
fulfills map { fulfill =>
val extracted = Closing.extractPreimages(d.commitments.localCommit, tx)
extracted map { case (htlc, fulfill) =>
val origin = d.commitments.originChannels(fulfill.id)
log.warning(s"fulfilling htlc #${fulfill.id} paymentHash=${sha256(fulfill.paymentPreimage)} origin=$origin")
relayer ! ForwardFulfill(fulfill, origin)
relayer ! ForwardFulfill(fulfill, origin, htlc)
}
stay

View file

@ -195,9 +195,9 @@ object Commitments {
case None => throw UnknownHtlcId(commitments.channelId, cmd.id)
}
def receiveFulfill(commitments: Commitments, fulfill: UpdateFulfillHtlc): Either[Commitments, (Commitments, Origin)] =
def receiveFulfill(commitments: Commitments, fulfill: UpdateFulfillHtlc): Either[Commitments, (Commitments, Origin, UpdateAddHtlc)] =
getHtlcCrossSigned(commitments, OUT, fulfill.id) match {
case Some(htlc) if htlc.paymentHash == sha256(fulfill.paymentPreimage) => Right((addRemoteProposal(commitments, fulfill), commitments.originChannels(fulfill.id)))
case Some(htlc) if htlc.paymentHash == sha256(fulfill.paymentPreimage) => Right((addRemoteProposal(commitments, fulfill), commitments.originChannels(fulfill.id), htlc))
case Some(htlc) => throw InvalidHtlcPreimage(commitments.channelId, fulfill.id)
case None => throw UnknownHtlcId(commitments.channelId, fulfill.id)
}
@ -250,20 +250,20 @@ object Commitments {
}
}
def receiveFail(commitments: Commitments, fail: UpdateFailHtlc): Either[Commitments, (Commitments, Origin)] =
def receiveFail(commitments: Commitments, fail: UpdateFailHtlc): Either[Commitments, (Commitments, Origin, UpdateAddHtlc)] =
getHtlcCrossSigned(commitments, OUT, fail.id) match {
case Some(htlc) => Right((addRemoteProposal(commitments, fail), commitments.originChannels(fail.id)))
case Some(htlc) => Right((addRemoteProposal(commitments, fail), commitments.originChannels(fail.id), htlc))
case None => throw UnknownHtlcId(commitments.channelId, fail.id)
}
def receiveFailMalformed(commitments: Commitments, fail: UpdateFailMalformedHtlc): Either[Commitments, (Commitments, Origin)] = {
def receiveFailMalformed(commitments: Commitments, fail: UpdateFailMalformedHtlc): Either[Commitments, (Commitments, Origin, UpdateAddHtlc)] = {
// A receiving node MUST fail the channel if the BADONION bit in failure_code is not set for update_fail_malformed_htlc.
if ((fail.failureCode & FailureMessageCodecs.BADONION) == 0) {
throw InvalidFailureCode(commitments.channelId)
}
getHtlcCrossSigned(commitments, OUT, fail.id) match {
case Some(htlc) => Right((addRemoteProposal(commitments, fail), commitments.originChannels(fail.id)))
case Some(htlc) => Right((addRemoteProposal(commitments, fail), commitments.originChannels(fail.id), htlc))
case None => throw UnknownHtlcId(commitments.channelId, fail.id)
}
}

View file

@ -438,9 +438,11 @@ object Helpers {
*
* @param localCommit
* @param tx
* @return a set of fulfills that need to be sent upstream if extraction was successful
* @return a set of pairs (add, fulfills) if extraction was successful:
* - add is the htlc in the downstream channel from which we extracted the preimage
* - fulfill needs to be sent to the upstream channel
*/
def extractPreimages(localCommit: LocalCommit, tx: Transaction)(implicit log: LoggingAdapter): Set[UpdateFulfillHtlc] = {
def extractPreimages(localCommit: LocalCommit, tx: Transaction)(implicit log: LoggingAdapter): Set[(UpdateAddHtlc, UpdateFulfillHtlc)] = {
val paymentPreimages = tx.txIn.map(_.witness match {
case ScriptWitness(Seq(localSig, paymentPreimage, htlcOfferedScript)) if paymentPreimage.size == 32 =>
log.info(s"extracted paymentPreimage=$paymentPreimage from tx=$tx (claim-htlc-success)")
@ -459,7 +461,7 @@ object Helpers {
outgoingHtlcs.collect {
case add if add.paymentHash == sha256(paymentPreimage) =>
// let's just pretend we received the preimage from the counterparty and build a fulfill message
UpdateFulfillHtlc(add.channelId, add.id, paymentPreimage)
(add, UpdateFulfillHtlc(add.channelId, add.id, paymentPreimage))
}
// TODO: should we handle local htlcs here as well? currently timed out htlcs that we sent will never have an answer
}

View file

@ -19,9 +19,9 @@ case class Local(sender: Option[ActorRef]) extends Origin // we don't persist re
case class Relayed(originChannelId: BinaryData, originHtlcId: Long, amountMsatIn: Long, amountMsatOut: Long) extends Origin
case class ForwardAdd(add: UpdateAddHtlc)
case class ForwardFulfill(fulfill: UpdateFulfillHtlc, to: Origin)
case class ForwardFail(fail: UpdateFailHtlc, to: Origin)
case class ForwardFailMalformed(fail: UpdateFailMalformedHtlc, to: Origin)
case class ForwardFulfill(fulfill: UpdateFulfillHtlc, to: Origin, htlc: UpdateAddHtlc)
case class ForwardFail(fail: UpdateFailHtlc, to: Origin, htlc: UpdateAddHtlc)
case class ForwardFailMalformed(fail: UpdateFailMalformedHtlc, to: Origin, htlc: UpdateAddHtlc)
// @formatter:on
@ -130,25 +130,25 @@ class Relayer(nodeParams: NodeParams, register: ActorRef, paymentHandler: ActorR
log.info(s"rejecting htlc #$originHtlcId paymentHash=$paymentHash from channelId=$originChannelId reason=${cmdFail.reason}")
commandBuffer ! CommandBuffer.CommandSend(originChannelId, originHtlcId, cmdFail)
case ForwardFulfill(fulfill, Local(Some(sender))) =>
case ForwardFulfill(fulfill, Local(Some(sender)), _) =>
sender ! fulfill
case ForwardFulfill(fulfill, Relayed(originChannelId, originHtlcId, amountMsatIn, amountMsatOut)) =>
case ForwardFulfill(fulfill, Relayed(originChannelId, originHtlcId, amountMsatIn, amountMsatOut), _) =>
val cmd = CMD_FULFILL_HTLC(originHtlcId, fulfill.paymentPreimage, commit = true)
commandBuffer ! CommandBuffer.CommandSend(originChannelId, originHtlcId, cmd)
context.system.eventStream.publish(PaymentRelayed(MilliSatoshi(amountMsatIn), MilliSatoshi(amountMsatOut), Crypto.sha256(fulfill.paymentPreimage)))
case ForwardFail(fail, Local(Some(sender))) =>
case ForwardFail(fail, Local(Some(sender)), _) =>
sender ! fail
case ForwardFail(fail, Relayed(originChannelId, originHtlcId, _, _)) =>
case ForwardFail(fail, Relayed(originChannelId, originHtlcId, _, _), _) =>
val cmd = CMD_FAIL_HTLC(originHtlcId, Left(fail.reason), commit = true)
commandBuffer ! CommandBuffer.CommandSend(originChannelId, originHtlcId, cmd)
case ForwardFailMalformed(fail, Local(Some(sender))) =>
case ForwardFailMalformed(fail, Local(Some(sender)), _) =>
sender ! fail
case ForwardFailMalformed(fail, Relayed(originChannelId, originHtlcId, _, _)) =>
case ForwardFailMalformed(fail, Relayed(originChannelId, originHtlcId, _, _), _) =>
val cmd = CMD_FAIL_MALFORMED_HTLC(originHtlcId, fail.onionHash, fail.failureCode, commit = true)
commandBuffer ! CommandBuffer.CommandSend(originChannelId, originHtlcId, cmd)

View file

@ -52,7 +52,6 @@ case class Data(nodes: Map[PublicKey, NodeAnnouncement],
sealed trait State
case object NORMAL extends State
case object WAITING_FOR_VALIDATION extends State
case object TickBroadcast
case object TickPruneStaleChannels
@ -159,14 +158,16 @@ class Router(nodeParams: NodeParams, watcher: ActorRef) extends FSM[State, Data]
log.info("done sending announcements to a peer, freeing slot (waiting={})", d.sendStateWaitlist.size)
val d1 = d.copy(sendingState = d.sendingState - actor)
d.sendStateWaitlist.dequeueOption match {
case Some((remote, sendStateWaitlist1)) => stay using handleSendState(remote, d1.copy(sendStateWaitlist = sendStateWaitlist1))
case Some((remote, sendStateWaitlist1)) =>
context unwatch remote
stay using handleSendState(remote, d1.copy(sendStateWaitlist = sendStateWaitlist1))
case None => stay using d1
}
case Event(Terminated(actor), d: Data) if d.sendStateWaitlist.contains(actor) =>
// note: 'contains' and 'filter' operations are expensive on a queue, but its size should be very small (maybe even capped?)
log.info("peer={} died, removing from wait list (waiting={})", actor, d.sendStateWaitlist.size - 1)
stay using d.copy(sendStateWaitlist = d.sendStateWaitlist filterNot(_ == actor))
stay using d.copy(sendStateWaitlist = d.sendStateWaitlist filterNot (_ == actor))
case Event(c: ChannelAnnouncement, d) =>
log.debug("received channel announcement for shortChannelId={} nodeId1={} nodeId2={} from {}", c.shortChannelId.toHexString, c.nodeId1, c.nodeId2, sender)
@ -369,7 +370,7 @@ class Router(nodeParams: NodeParams, watcher: ActorRef) extends FSM[State, Data]
// we add them to the publicly-announced updates (order matters!! local/assisted channel_updates will override channel_updates received by the network)
val updates1 = d.updates ++ updates0
// we then filter out the currently excluded channels
val updates2 = updates1.filterKeys(!d.excludedChannels.contains(_))
val updates2 = updates1 -- d.excludedChannels
// we also filter out disabled channels, and channels/nodes that are blacklisted for this particular request
val updates3 = filterUpdates(updates2, ignoreNodes, ignoreChannels)
log.info("finding a route {}->{} with ignoreNodes={} ignoreChannels={}", start, end, ignoreNodes.map(_.toBin).mkString(","), ignoreChannels.map(_.toHexString).mkString(","))
@ -429,7 +430,7 @@ class Router(nodeParams: NodeParams, watcher: ActorRef) extends FSM[State, Data]
log.debug("ignoring {} (old timestamp or duplicate)", u)
d
} else if (!Announcements.checkSig(u, desc.a)) {
log.warning("bad signature for announcement shortChannelId={} {}", u, u.shortChannelId.toHexString)
log.warning("bad signature for announcement shortChannelId={} {}", u.shortChannelId.toHexString, u)
origin ! Error(Peer.CHANNELID_ZERO, "bad announcement sig!!!".getBytes())
d
} else if (d.updates.contains(desc)) {
@ -534,9 +535,10 @@ object Router {
val staleThresholdSeconds = Platform.currentTime / 1000 - 1209600
val staleThresholdBlocks = Globals.blockCount.get() - 2016
val staleChannels = channels
.filter(c => fromShortId(c.shortChannelId)._1 < staleThresholdBlocks) // consider only channels older than 2 weeks
.filter(c => updates.exists(_.shortChannelId == c.shortChannelId)) // channel must have updates
.filter(c => updates.filter(_.shortChannelId == c.shortChannelId).map(_.timestamp).max < staleThresholdSeconds) // updates are all older than 2 weeks (can have 1 or 2)
.filter(c =>
fromShortId(c.shortChannelId)._1 < staleThresholdBlocks // consider only channels older than 2 weeks
&& updates.exists(_.shortChannelId == c.shortChannelId) // channel must have updates
&& updates.filter(_.shortChannelId == c.shortChannelId).map(_.timestamp).max < staleThresholdSeconds) // updates are all older than 2 weeks (can have 1 or 2)
staleChannels.map(_.shortChannelId)
}
@ -544,10 +546,11 @@ object Router {
* This method is used after a payment failed, and we want to exclude some nodes/channels that we know are failing
*/
def filterUpdates(updates: Map[ChannelDesc, ChannelUpdate], ignoreNodes: Set[PublicKey], ignoreChannels: Set[Long]) =
updates
.filterNot(u => ignoreNodes.map(_.toBin).contains(u._1.a) || ignoreNodes.map(_.toBin).contains(u._1.b))
.filterNot(u => ignoreChannels.contains(u._1.id))
.filter(u => Announcements.isEnabled(u._2.flags))
updates.filter { case (desc, u) =>
!ignoreNodes.contains(desc.a) && !ignoreNodes.contains(desc.b) &&
!ignoreChannels.contains(desc.id) &&
Announcements.isEnabled(u.flags)
}
def findRouteDijkstra(localNodeId: PublicKey, targetNodeId: PublicKey, channels: Iterable[ChannelDesc]): Seq[ChannelDesc] = {
if (localNodeId == targetNodeId) throw CannotRouteToSelf

View file

@ -352,25 +352,27 @@ class RelayerSpec extends TestkitBaseClass {
system.eventStream.subscribe(eventListener.ref, classOf[PaymentEvent])
// there isn't any corresponding htlc, it does not matter here
val fulfill_cb = UpdateFulfillHtlc(channelId = channelId_bc, id = 42, paymentPreimage = "00" * 32)
// we build a fake htlc for the downstream channel
val add_bc = UpdateAddHtlc(channelId = channelId_bc, id = 72, amountMsat = 10000000L, paymentHash = "00" * 32, expiry = 4200, onionRoutingPacket = "")
val fulfill_ba = UpdateFulfillHtlc(channelId = channelId_bc, id = 42, paymentPreimage = "00" * 32)
val origin = Relayed(channelId_ab, 150, 11000000L, 10000000L)
sender.send(relayer, ForwardFulfill(fulfill_cb, origin))
sender.send(relayer, ForwardFulfill(fulfill_ba, origin, add_bc))
val fwd = register.expectMsgType[Register.Forward[CMD_FULFILL_HTLC]]
assert(fwd.channelId === origin.originChannelId)
assert(fwd.message.id === origin.originHtlcId)
eventListener.expectMsg(PaymentRelayed(MilliSatoshi(origin.amountMsatIn), MilliSatoshi(origin.amountMsatOut), Crypto.sha256(fulfill_cb.paymentPreimage)))
eventListener.expectMsg(PaymentRelayed(MilliSatoshi(origin.amountMsatIn), MilliSatoshi(origin.amountMsatOut), Crypto.sha256(fulfill_ba.paymentPreimage)))
}
test("relay an htlc-fail") { case (relayer, register, _) =>
val sender = TestProbe()
// there isn't any corresponding htlc, it does not matter here
val fail_cb = UpdateFailHtlc(channelId = channelId_bc, id = 42, reason = Sphinx.createErrorPacket(BinaryData("01" * 32), TemporaryChannelFailure(channelUpdate_cd)))
// we build a fake htlc for the downstream channel
val add_bc = UpdateAddHtlc(channelId = channelId_bc, id = 72, amountMsat = 10000000L, paymentHash = "00" * 32, expiry = 4200, onionRoutingPacket = "")
val fail_ba = UpdateFailHtlc(channelId = channelId_bc, id = 42, reason = Sphinx.createErrorPacket(BinaryData("01" * 32), TemporaryChannelFailure(channelUpdate_cd)))
val origin = Relayed(channelId_ab, 150, 11000000L, 10000000L)
sender.send(relayer, ForwardFail(fail_cb, origin))
sender.send(relayer, ForwardFail(fail_ba, origin, add_bc))
val fwd = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]]
assert(fwd.channelId === origin.originChannelId)