Skip to content

Commit

Permalink
Allow unknown failure messages.
Browse files Browse the repository at this point in the history
Extract failure code to test PERM or NODE bits.
  • Loading branch information
t-bast committed Aug 29, 2019
1 parent 09cf327 commit 87b1b63
Show file tree
Hide file tree
Showing 6 changed files with 124 additions and 70 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -112,7 +112,7 @@ class PaymentLifecycle(nodeParams: NodeParams, id: UUID, router: ActorRef, regis
log.warning(s"blacklisting intermediate nodes=${blacklist.mkString(",")}")
router ! RouteRequest(nodeParams.nodeId, c.targetNodeId, c.amount, c.assistedRoutes, ignoreNodes ++ blacklist, ignoreChannels, c.routeParams)
goto(WAITING_FOR_ROUTE) using WaitingForRoute(s, c, failures :+ UnreadableRemoteFailure(hops))
case Success(e@Sphinx.DecryptedFailurePacket(nodeId, failureMessage: Node)) =>
case Success(e@Sphinx.DecryptedFailurePacket(nodeId, failureMessage: FailureMessage)) if failureMessage.temporary =>
log.info(s"received 'Node' type error message from nodeId=$nodeId, trying to route around it (failure=$failureMessage)")
// let's try to route around this node
router ! RouteRequest(nodeParams.nodeId, c.targetNodeId, c.amount, c.assistedRoutes, ignoreNodes + nodeId, ignoreChannels, c.routeParams)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -120,7 +120,7 @@ class Relayer(nodeParams: NodeParams, register: ActorRef, paymentHandler: ActorR
}
case Left(badOnion) =>
log.warning(s"couldn't parse onion: reason=${badOnion.message}")
val cmdFail = CMD_FAIL_MALFORMED_HTLC(add.id, badOnion.onionHash, FailureMessageCodecs.failureCode(badOnion), commit = true)
val cmdFail = CMD_FAIL_MALFORMED_HTLC(add.id, badOnion.onionHash, badOnion.code, commit = true)
log.info(s"rejecting htlc #${add.id} paymentHash=${add.paymentHash} from channelId=${add.channelId} reason=malformed onionHash=${cmdFail.onionHash} failureCode=${cmdFail.failureCode}")
commandBuffer ! CommandBuffer.CommandSend(add.channelId, add.id, cmdFail)
}
Expand Down
109 changes: 59 additions & 50 deletions eclair-core/src/main/scala/fr/acinq/eclair/wire/FailureMessage.scala
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,8 @@ package fr.acinq.eclair.wire

import fr.acinq.bitcoin.ByteVector32
import fr.acinq.eclair.crypto.Mac32
import fr.acinq.eclair.wire.CommonCodecs.{cltvExpiry, millisatoshi, sha256}
import fr.acinq.eclair.wire.CommonCodecs.{cltvExpiry, discriminatorWithDefault, millisatoshi, sha256}
import fr.acinq.eclair.wire.FailureMessageCodecs.failureMessageCodec
import fr.acinq.eclair.wire.LightningMessageCodecs.{channelUpdateCodec, lightningMessageCodec}
import fr.acinq.eclair.{CltvExpiry, LongToBtcAmount, MilliSatoshi}
import scodec.codecs._
Expand All @@ -30,37 +31,48 @@ import scodec.{Attempt, Codec}
*/

// @formatter:off
sealed trait FailureMessage { def message: String }
sealed trait FailureMessage {
def message: String
// We actually encode the failure message, which is a bit clunky and not particularly efficient.
// It would be nice to be able to get that value from the discriminated codec directly.
lazy val code: Int = failureMessageCodec.encode(this).flatMap(uint16.decode).require.value
lazy val permanent: Boolean = (code & FailureMessageCodecs.PERM) != 0
lazy val temporary: Boolean = (code & FailureMessageCodecs.NODE) != 0
}
sealed trait BadOnion extends FailureMessage { def onionHash: ByteVector32 }
sealed trait Perm extends FailureMessage
sealed trait Node extends FailureMessage
sealed trait Update extends FailureMessage { def update: ChannelUpdate }

case object InvalidRealm extends Perm { def message = "realm was not understood by the processing node" }
case object TemporaryNodeFailure extends Node { def message = "general temporary failure of the processing node" }
case object PermanentNodeFailure extends Perm with Node { def message = "general permanent failure of the processing node" }
case object RequiredNodeFeatureMissing extends Perm with Node { def message = "processing node requires features that are missing from this onion" }
case class InvalidOnionVersion(onionHash: ByteVector32) extends BadOnion with Perm { def message = "onion version was not understood by the processing node" }
case class InvalidOnionHmac(onionHash: ByteVector32) extends BadOnion with Perm { def message = "onion HMAC was incorrect when it reached the processing node" }
case class InvalidOnionKey(onionHash: ByteVector32) extends BadOnion with Perm { def message = "ephemeral key was unparsable by the processing node" }
case class InvalidOnionPayload(onionHash: ByteVector32) extends BadOnion with Perm { def message = "onion per-hop payload could not be parsed" }
case object InvalidRealm extends FailureMessage { def message = "realm was not understood by the processing node" }
case object TemporaryNodeFailure extends FailureMessage { def message = "general temporary failure of the processing node" }
case object PermanentNodeFailure extends FailureMessage { def message = "general permanent failure of the processing node" }
case object RequiredNodeFeatureMissing extends FailureMessage { def message = "processing node requires features that are missing from this onion" }
case class InvalidOnionVersion(onionHash: ByteVector32) extends BadOnion { def message = "onion version was not understood by the processing node" }
case class InvalidOnionHmac(onionHash: ByteVector32) extends BadOnion { def message = "onion HMAC was incorrect when it reached the processing node" }
case class InvalidOnionKey(onionHash: ByteVector32) extends BadOnion { def message = "ephemeral key was unparsable by the processing node" }
case class InvalidOnionPayload(onionHash: ByteVector32) extends BadOnion { def message = "onion per-hop payload could not be parsed" }
case class TemporaryChannelFailure(update: ChannelUpdate) extends Update { def message = s"channel ${update.shortChannelId} is currently unavailable" }
case object PermanentChannelFailure extends Perm { def message = "channel is permanently unavailable" }
case object RequiredChannelFeatureMissing extends Perm { def message = "channel requires features not present in the onion" }
case object UnknownNextPeer extends Perm { def message = "processing node does not know the next peer in the route" }
case object PermanentChannelFailure extends FailureMessage { def message = "channel is permanently unavailable" }
case object RequiredChannelFeatureMissing extends FailureMessage { def message = "channel requires features not present in the onion" }
case object UnknownNextPeer extends FailureMessage { def message = "processing node does not know the next peer in the route" }
case class AmountBelowMinimum(amount: MilliSatoshi, update: ChannelUpdate) extends Update { def message = s"payment amount was below the minimum required by the channel" }
case class FeeInsufficient(amount: MilliSatoshi, update: ChannelUpdate) extends Update { def message = s"payment fee was below the minimum required by the channel" }
case class ChannelDisabled(messageFlags: Byte, channelFlags: Byte, update: ChannelUpdate) extends Update { def message = "channel is currently disabled" }
case class IncorrectCltvExpiry(expiry: CltvExpiry, update: ChannelUpdate) extends Update { def message = "payment expiry doesn't match the value in the onion" }
case class IncorrectOrUnknownPaymentDetails(amount: MilliSatoshi, height: Long) extends Perm { def message = "incorrect payment details or unknown payment hash" }
/** Deprecated: this failure code allows probing attacks: IncorrectOrUnknownPaymentDetails should be used instead. */
case object IncorrectPaymentAmount extends Perm { def message = "payment amount is incorrect" }
case class IncorrectOrUnknownPaymentDetails(amount: MilliSatoshi, height: Long) extends FailureMessage { def message = "incorrect payment details or unknown payment hash" }
case class ExpiryTooSoon(update: ChannelUpdate) extends Update { def message = "payment expiry is too close to the current block height for safe handling by the relaying node" }
/** Deprecated: this failure code allows probing attacks: IncorrectOrUnknownPaymentDetails should be used instead. */
case object FinalExpiryTooSoon extends FailureMessage { def message = "payment expiry is too close to the current block height for safe handling by the final node" }
case class FinalIncorrectCltvExpiry(expiry: CltvExpiry) extends FailureMessage { def message = "payment expiry doesn't match the value in the onion" }
case class FinalIncorrectHtlcAmount(amount: MilliSatoshi) extends FailureMessage { def message = "payment amount is incorrect in the final htlc" }
case object ExpiryTooFar extends FailureMessage { def message = "payment expiry is too far in the future" }

/**
* We allow remote nodes to send us unknown failure codes (e.g. deprecated failure codes).
* By reading the PERM and NODE bits we can still extract useful information for payment retry even without knowing how
* to decode the failure payload (but we can't extract a channel update or onion hash).
*/
case class UnknownFailureMessage(failureCode: Int) extends FailureMessage {
def message = "unknown failure message"
override lazy val code = failureCode
}
// @formatter:on

object FailureMessageCodecs {
Expand All @@ -75,36 +87,33 @@ object FailureMessageCodecs {
// this codec supports both versions for decoding, and will encode with the message type
val channelUpdateWithLengthCodec = variableSizeBytes(uint16, choice(channelUpdateCodecWithType, channelUpdateCodec))

val failureMessageCodec = discriminated[FailureMessage].by(uint16)
.typecase(PERM | 1, provide(InvalidRealm))
.typecase(NODE | 2, provide(TemporaryNodeFailure))
.typecase(PERM | 2, provide(PermanentNodeFailure))
.typecase(PERM | NODE | 3, provide(RequiredNodeFeatureMissing))
.typecase(BADONION | PERM, sha256.as[InvalidOnionPayload])
.typecase(BADONION | PERM | 4, sha256.as[InvalidOnionVersion])
.typecase(BADONION | PERM | 5, sha256.as[InvalidOnionHmac])
.typecase(BADONION | PERM | 6, sha256.as[InvalidOnionKey])
.typecase(UPDATE | 7, ("channelUpdate" | channelUpdateWithLengthCodec).as[TemporaryChannelFailure])
.typecase(PERM | 8, provide(PermanentChannelFailure))
.typecase(PERM | 9, provide(RequiredChannelFeatureMissing))
.typecase(PERM | 10, provide(UnknownNextPeer))
.typecase(UPDATE | 11, (("amountMsat" | millisatoshi) :: ("channelUpdate" | channelUpdateWithLengthCodec)).as[AmountBelowMinimum])
.typecase(UPDATE | 12, (("amountMsat" | millisatoshi) :: ("channelUpdate" | channelUpdateWithLengthCodec)).as[FeeInsufficient])
.typecase(UPDATE | 13, (("expiry" | cltvExpiry) :: ("channelUpdate" | channelUpdateWithLengthCodec)).as[IncorrectCltvExpiry])
.typecase(UPDATE | 14, ("channelUpdate" | channelUpdateWithLengthCodec).as[ExpiryTooSoon])
.typecase(UPDATE | 20, (("messageFlags" | byte) :: ("channelFlags" | byte) :: ("channelUpdate" | channelUpdateWithLengthCodec)).as[ChannelDisabled])
.typecase(PERM | 15, (("amountMsat" | withDefaultValue(optional(bitsRemaining, millisatoshi), 0 msat)) :: ("height" | withDefaultValue(optional(bitsRemaining, uint32), 0L))).as[IncorrectOrUnknownPaymentDetails])
.typecase(PERM | 16, provide(IncorrectPaymentAmount))
.typecase(17, provide(FinalExpiryTooSoon))
.typecase(18, ("expiry" | cltvExpiry).as[FinalIncorrectCltvExpiry])
.typecase(19, ("amountMsat" | millisatoshi).as[FinalIncorrectHtlcAmount])
.typecase(21, provide(ExpiryTooFar))

/**
* Return the failure code for a given failure message. This method actually encodes the failure message, which is a
* bit clunky and not particularly efficient. It shouldn't be used on the application's hot path.
*/
def failureCode(failure: FailureMessage): Int = failureMessageCodec.encode(failure).flatMap(uint16.decode).require.value
val failureMessageCodec = discriminatorWithDefault(
discriminated[FailureMessage].by(uint16)
.typecase(PERM | 1, provide(InvalidRealm))
.typecase(NODE | 2, provide(TemporaryNodeFailure))
.typecase(PERM | NODE | 2, provide(PermanentNodeFailure))
.typecase(PERM | NODE | 3, provide(RequiredNodeFeatureMissing))
.typecase(BADONION | PERM, sha256.as[InvalidOnionPayload])
.typecase(BADONION | PERM | 4, sha256.as[InvalidOnionVersion])
.typecase(BADONION | PERM | 5, sha256.as[InvalidOnionHmac])
.typecase(BADONION | PERM | 6, sha256.as[InvalidOnionKey])
.typecase(UPDATE | 7, ("channelUpdate" | channelUpdateWithLengthCodec).as[TemporaryChannelFailure])
.typecase(PERM | 8, provide(PermanentChannelFailure))
.typecase(PERM | 9, provide(RequiredChannelFeatureMissing))
.typecase(PERM | 10, provide(UnknownNextPeer))
.typecase(UPDATE | 11, (("amountMsat" | millisatoshi) :: ("channelUpdate" | channelUpdateWithLengthCodec)).as[AmountBelowMinimum])
.typecase(UPDATE | 12, (("amountMsat" | millisatoshi) :: ("channelUpdate" | channelUpdateWithLengthCodec)).as[FeeInsufficient])
.typecase(UPDATE | 13, (("expiry" | cltvExpiry) :: ("channelUpdate" | channelUpdateWithLengthCodec)).as[IncorrectCltvExpiry])
.typecase(UPDATE | 14, ("channelUpdate" | channelUpdateWithLengthCodec).as[ExpiryTooSoon])
.typecase(UPDATE | 20, (("messageFlags" | byte) :: ("channelFlags" | byte) :: ("channelUpdate" | channelUpdateWithLengthCodec)).as[ChannelDisabled])
.typecase(PERM | 15, (("amountMsat" | withDefaultValue(optional(bitsRemaining, millisatoshi), 0 msat)) :: ("height" | withDefaultValue(optional(bitsRemaining, uint32), 0L))).as[IncorrectOrUnknownPaymentDetails])
// PERM | 16 (incorrect_payment_amount) has been deprecated because it allowed probing attacks: IncorrectOrUnknownPaymentDetails should be used instead.
// PERM | 17 (final_expiry_too_soon) has been deprecated because it allowed probing attacks: IncorrectOrUnknownPaymentDetails should be used instead.
.typecase(18, ("expiry" | cltvExpiry).as[FinalIncorrectCltvExpiry])
.typecase(19, ("amountMsat" | millisatoshi).as[FinalIncorrectHtlcAmount])
.typecase(21, provide(ExpiryTooFar)),
uint16.xmap(UnknownFailureMessage(_).asInstanceOf[FailureMessage], (_: FailureMessage).code)
)

/**
* An onion-encrypted failure from an intermediate node:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ import fr.acinq.eclair.TestConstants.Alice
import fr.acinq.eclair.channel.{CMD_FAIL_HTLC, CMD_FULFILL_HTLC}
import fr.acinq.eclair.payment.PaymentLifecycle.ReceivePayment
import fr.acinq.eclair.payment.PaymentRequest.ExtraHop
import fr.acinq.eclair.wire.{FinalExpiryTooSoon, IncorrectOrUnknownPaymentDetails, UpdateAddHtlc}
import fr.acinq.eclair.wire.{IncorrectOrUnknownPaymentDetails, UpdateAddHtlc}
import fr.acinq.eclair.{CltvExpiryDelta, Globals, LongToBtcAmount, ShortChannelId, TestConstants, randomKey}
import org.scalatest.FunSuiteLike
import scodec.bits.ByteVector
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -327,7 +327,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
awaitCond(paymentDb.getOutgoingPayment(id).exists(_.status == OutgoingPaymentStatus.FAILED))
}

test("payment failed (PermanentChannelFailure)") { fixture =>
def testPermanentFailure(fixture: FixtureParam, failure: FailureMessage): Unit = {
import fixture._
val nodeParams = TestConstants.Alice.nodeParams.copy(keyManager = testKeyManager)
val paymentDb = nodeParams.db.payments
Expand All @@ -351,8 +351,6 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
awaitCond(paymentFSM.stateName == WAITING_FOR_PAYMENT_COMPLETE)
val WaitingForComplete(_, _, cmd1, Nil, sharedSecrets1, _, _, hops) = paymentFSM.stateData

val failure = PermanentChannelFailure

relayer.expectMsg(ForwardShortId(channelId_ab, cmd1))
sender.send(paymentFSM, UpdateFailHtlc(ByteVector32.Zeroes, 0, Sphinx.FailurePacket.create(sharedSecrets1.head._1, failure)))

Expand All @@ -366,6 +364,15 @@ class PaymentLifecycleSpec extends BaseRouterSpec {
awaitCond(paymentDb.getOutgoingPayment(id).exists(_.status == OutgoingPaymentStatus.FAILED))
}

test("payment failed (PermanentChannelFailure)") { fixture =>
testPermanentFailure(fixture, PermanentChannelFailure)
}

test("payment failed (deprecated permanent failure)") { fixture =>
// PERM | 17 (final_expiry_too_soon) has been deprecated but older nodes might still use it.
testPermanentFailure(fixture, UnknownFailureMessage(FailureMessageCodecs.PERM | 17))
}

test("payment succeeded") { fixture =>
import fixture._
val defaultPaymentHash = randomBytes32
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,19 +42,55 @@ class FailureMessageCodecsSpec extends FunSuite {
htlcMaximumMsat = None)

test("encode/decode all channel messages") {
val msgs: List[FailureMessage] =
InvalidRealm :: TemporaryNodeFailure :: PermanentNodeFailure :: RequiredNodeFeatureMissing ::
InvalidOnionVersion(randomBytes32) :: InvalidOnionHmac(randomBytes32) :: InvalidOnionKey(randomBytes32) :: InvalidOnionPayload(randomBytes32) ::
TemporaryChannelFailure(channelUpdate) :: PermanentChannelFailure :: RequiredChannelFeatureMissing :: UnknownNextPeer ::
AmountBelowMinimum(123456 msat, channelUpdate) :: FeeInsufficient(546463 msat, channelUpdate) :: IncorrectCltvExpiry(CltvExpiry(1211), channelUpdate) :: ExpiryTooSoon(channelUpdate) ::
IncorrectOrUnknownPaymentDetails(123456 msat, 1105) :: IncorrectPaymentAmount :: FinalExpiryTooSoon :: FinalIncorrectCltvExpiry(CltvExpiry(1234)) :: ChannelDisabled(0, 1, channelUpdate) :: ExpiryTooFar :: Nil

msgs.foreach {
msg => {
val encoded = failureMessageCodec.encode(msg).require
val decoded = failureMessageCodec.decode(encoded).require
assert(msg === decoded.value)
}
val msgs = Seq[(Boolean, Boolean, FailureMessage)](
(false, true, InvalidRealm),
(true, false, TemporaryNodeFailure),
(true, true, PermanentNodeFailure),
(true, true, RequiredNodeFeatureMissing),
(false, true, InvalidOnionVersion(randomBytes32)),
(false, true, InvalidOnionHmac(randomBytes32)),
(false, true, InvalidOnionKey(randomBytes32)),
(false, true, InvalidOnionPayload(randomBytes32)),
(false, false, TemporaryChannelFailure(channelUpdate)),
(false, true, PermanentChannelFailure),
(false, true, RequiredChannelFeatureMissing),
(false, true, UnknownNextPeer),
(false, false, AmountBelowMinimum(123456 msat, channelUpdate)),
(false, false, FeeInsufficient(546463 msat, channelUpdate)),
(false, false, IncorrectCltvExpiry(CltvExpiry(1211), channelUpdate)),
(false, false, ExpiryTooSoon(channelUpdate)),
(false, true, IncorrectOrUnknownPaymentDetails(123456 msat, 1105)),
(false, false, FinalIncorrectCltvExpiry(CltvExpiry(1234))),
(false, false, ChannelDisabled(0, 1, channelUpdate)),
(false, false, ExpiryTooFar)
)

for ((temporary, permanent, msg) <- msgs) {
val encoded = failureMessageCodec.encode(msg).require
val decoded = failureMessageCodec.decode(encoded).require.value
assert(msg === decoded)
assert(msg.temporary === temporary)
assert(msg.permanent === permanent)
}
}

test("decode unknown failure messages") {
val testCases = Seq(
// Deprecated incorrect_payment_amount.
(false, true, hex"4010"),
// Deprecated final_expiry_too_soon.
(false, true, hex"4011"),
// Unknown failure messages.
(false, false, hex"00ff 42"),
(true, false, hex"20ff 42"),
(true, true, hex"60ff 42")
)

for ((temporary, permanent, bin) <- testCases) {
val decoded = failureMessageCodec.decode(bin.bits).require.value
assert(decoded.isInstanceOf[UnknownFailureMessage])
assert(decoded.temporary === temporary)
assert(decoded.permanent === permanent)
}
}

Expand All @@ -67,7 +103,9 @@ class FailureMessageCodecsSpec extends FunSuite {
)

for ((code, message) <- msgs) {
assert(failureCode(message) === code)
assert(message.code === code)
assert(message.permanent)
assert(!message.temporary)
}
}

Expand Down

0 comments on commit 87b1b63

Please sign in to comment.