mirror of
https://github.com/ACINQ/eclair.git
synced 2024-11-19 01:43:22 +01:00
Allow sending message to route that starts with us (#2585)
In case we are the introduction node of a message blinded route, we would not be able to send a message to that route. We now unwrap the first hop in case the route starts with us.
This commit is contained in:
parent
f901aeac04
commit
198fd934f3
@ -371,7 +371,7 @@ class Setup(val datadir: File,
|
||||
_ = triggerer ! AsyncPaymentTriggerer.Start(switchboard.toTyped)
|
||||
balanceActor = system.spawn(BalanceActor(nodeParams.db, bitcoinClient, channelsListener, nodeParams.balanceCheckInterval), name = "balance-actor")
|
||||
|
||||
postman = system.spawn(Behaviors.supervise(Postman(switchboard.toTyped)).onFailure(typed.SupervisorStrategy.restart), name = "postman")
|
||||
postman = system.spawn(Behaviors.supervise(Postman(nodeParams, switchboard.toTyped)).onFailure(typed.SupervisorStrategy.restart), name = "postman")
|
||||
|
||||
kit = Kit(
|
||||
nodeParams = nodeParams,
|
||||
|
@ -28,7 +28,6 @@ import scodec.{Attempt, DecodeResult}
|
||||
|
||||
import scala.annotation.tailrec
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
import scala.util.Try
|
||||
|
||||
object OnionMessages {
|
||||
|
||||
@ -42,35 +41,55 @@ object OnionMessages {
|
||||
case class Recipient(nodeId: PublicKey, pathId: Option[ByteVector], padding: Option[ByteVector] = None) extends Destination
|
||||
// @formatter:on
|
||||
|
||||
def buildRoute(blindingSecret: PrivateKey,
|
||||
intermediateNodes: Seq[IntermediateNode],
|
||||
destination: Destination): Sphinx.RouteBlinding.BlindedRoute = {
|
||||
val last = destination match {
|
||||
case Recipient(nodeId, _, _) => OutgoingNodeId(nodeId) :: Nil
|
||||
case BlindedPath(Sphinx.RouteBlinding.BlindedRoute(nodeId, blindingKey, _)) => OutgoingNodeId(nodeId) :: NextBlinding(blindingKey) :: Nil
|
||||
}
|
||||
val intermediatePayloads = if (intermediateNodes.isEmpty) {
|
||||
private def buildIntermediatePayloads(intermediateNodes: Seq[IntermediateNode], nextTlvs: Set[RouteBlindingEncryptedDataTlv]): Seq[ByteVector] = {
|
||||
if (intermediateNodes.isEmpty) {
|
||||
Nil
|
||||
} else {
|
||||
(intermediateNodes.tail.map(node => Set(OutgoingNodeId(node.nodeId))) :+ last)
|
||||
(intermediateNodes.tail.map(node => Set(OutgoingNodeId(node.nodeId))) :+ nextTlvs)
|
||||
.zip(intermediateNodes).map { case (tlvs, hop) => hop.padding.map(Padding).toSet[RouteBlindingEncryptedDataTlv] ++ tlvs }
|
||||
.map(tlvs => RouteBlindingEncryptedDataCodecs.blindedRouteDataCodec.encode(TlvStream(tlvs)).require.bytes)
|
||||
}
|
||||
destination match {
|
||||
case Recipient(nodeId, pathId, padding) =>
|
||||
val tlvs: Set[RouteBlindingEncryptedDataTlv] = Set(padding.map(Padding), pathId.map(PathId)).flatten
|
||||
}
|
||||
|
||||
def buildRoute(blindingSecret: PrivateKey,
|
||||
intermediateNodes: Seq[IntermediateNode],
|
||||
recipient: Recipient): Sphinx.RouteBlinding.BlindedRoute = {
|
||||
val intermediatePayloads = buildIntermediatePayloads(intermediateNodes, Set(OutgoingNodeId(recipient.nodeId)))
|
||||
val tlvs: Set[RouteBlindingEncryptedDataTlv] = Set(recipient.padding.map(Padding), recipient.pathId.map(PathId)).flatten
|
||||
val lastPayload = RouteBlindingEncryptedDataCodecs.blindedRouteDataCodec.encode(TlvStream(tlvs)).require.bytes
|
||||
Sphinx.RouteBlinding.create(blindingSecret, intermediateNodes.map(_.nodeId) :+ nodeId, intermediatePayloads :+ lastPayload).route
|
||||
Sphinx.RouteBlinding.create(blindingSecret, intermediateNodes.map(_.nodeId) :+ recipient.nodeId, intermediatePayloads :+ lastPayload).route
|
||||
}
|
||||
|
||||
private def buildRouteFrom(originKey: PrivateKey,
|
||||
blindingSecret: PrivateKey,
|
||||
intermediateNodes: Seq[IntermediateNode],
|
||||
destination: Destination): Option[Sphinx.RouteBlinding.BlindedRoute] = {
|
||||
destination match {
|
||||
case recipient: Recipient => Some(buildRoute(blindingSecret, intermediateNodes, recipient))
|
||||
case BlindedPath(route) if route.introductionNodeId == originKey.publicKey =>
|
||||
RouteBlindingEncryptedDataCodecs.decode(originKey, route.blindingKey, route.blindedNodes.head.encryptedPayload) match {
|
||||
case Left(_) => None
|
||||
case Right(decoded) =>
|
||||
decoded.tlvs.get[RouteBlindingEncryptedDataTlv.OutgoingNodeId] match {
|
||||
case None => None
|
||||
case Some(RouteBlindingEncryptedDataTlv.OutgoingNodeId(nextNodeId)) =>
|
||||
Some(Sphinx.RouteBlinding.BlindedRoute(nextNodeId, decoded.nextBlinding, route.blindedNodes.tail))
|
||||
}
|
||||
}
|
||||
case BlindedPath(route) if intermediateNodes.isEmpty => Some(route)
|
||||
case BlindedPath(route) =>
|
||||
if (intermediateNodes.isEmpty) {
|
||||
route
|
||||
} else {
|
||||
val intermediatePayloads = buildIntermediatePayloads(intermediateNodes, Set(OutgoingNodeId(route.introductionNodeId), NextBlinding(route.blindingKey)))
|
||||
val routePrefix = Sphinx.RouteBlinding.create(blindingSecret, intermediateNodes.map(_.nodeId), intermediatePayloads).route
|
||||
Sphinx.RouteBlinding.BlindedRoute(routePrefix.introductionNodeId, routePrefix.blindingKey, routePrefix.blindedNodes ++ route.blindedNodes)
|
||||
}
|
||||
Some(Sphinx.RouteBlinding.BlindedRoute(routePrefix.introductionNodeId, routePrefix.blindingKey, routePrefix.blindedNodes ++ route.blindedNodes))
|
||||
}
|
||||
}
|
||||
|
||||
// @formatter:off
|
||||
sealed trait BuildMessageError
|
||||
case class MessageTooLarge(payloadSize: Long) extends BuildMessageError
|
||||
case class InvalidDestination(destination: Destination) extends BuildMessageError
|
||||
// @formatter:on
|
||||
|
||||
/**
|
||||
* Builds an encrypted onion containing a message that should be relayed to the destination.
|
||||
*
|
||||
@ -81,12 +100,15 @@ object OnionMessages {
|
||||
* @param content List of TLVs to send to the recipient of the message
|
||||
* @return The node id to send the onion to and the onion containing the message
|
||||
*/
|
||||
def buildMessage(sessionKey: PrivateKey,
|
||||
def buildMessage(nodeKey: PrivateKey,
|
||||
sessionKey: PrivateKey,
|
||||
blindingSecret: PrivateKey,
|
||||
intermediateNodes: Seq[IntermediateNode],
|
||||
destination: Destination,
|
||||
content: TlvStream[OnionMessagePayloadTlv]): Try[(PublicKey, OnionMessage)] = Try{
|
||||
val route = buildRoute(blindingSecret, intermediateNodes, destination)
|
||||
content: TlvStream[OnionMessagePayloadTlv]): Either[BuildMessageError, (PublicKey, OnionMessage)] = {
|
||||
buildRouteFrom(nodeKey, blindingSecret, intermediateNodes, destination) match {
|
||||
case None => Left(InvalidDestination(destination))
|
||||
case Some(route) =>
|
||||
val lastPayload = MessageOnionCodecs.perHopPayloadCodec.encode(TlvStream(content.records + EncryptedData(route.encryptedPayloads.last), content.unknown)).require.bytes
|
||||
val payloads = route.encryptedPayloads.dropRight(1).map(encTlv => MessageOnionCodecs.perHopPayloadCodec.encode(TlvStream(EncryptedData(encTlv))).require.bytes) :+ lastPayload
|
||||
val payloadSize = payloads.map(_.length + Sphinx.MacLength).sum
|
||||
@ -96,13 +118,14 @@ object OnionMessages {
|
||||
32768
|
||||
} else if (payloadSize > 65432) {
|
||||
// A payload of size 65432 corresponds to a total lightning message size of 65535.
|
||||
throw new Exception(s"Message is too large: payloadSize=$payloadSize")
|
||||
return Left(MessageTooLarge(payloadSize))
|
||||
} else {
|
||||
payloadSize.toInt
|
||||
}
|
||||
// Since we are setting the packet size based on the payload, the onion creation should never fail (hence the `.get`).
|
||||
val Sphinx.PacketAndSecrets(packet, _) = Sphinx.create(sessionKey, packetSize, route.blindedNodes.map(_.blindedPublicKey), payloads, None).get
|
||||
(route.introductionNodeId, OnionMessage(route.blindingKey, packet))
|
||||
Right((route.introductionNodeId, OnionMessage(route.blindingKey, packet)))
|
||||
}
|
||||
}
|
||||
|
||||
// @formatter:off
|
||||
|
@ -22,14 +22,13 @@ import akka.actor.typed.{ActorRef, Behavior}
|
||||
import fr.acinq.bitcoin.scalacompat.ByteVector32
|
||||
import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
|
||||
import fr.acinq.eclair.io.{MessageRelay, Switchboard}
|
||||
import fr.acinq.eclair.message.OnionMessages.{Destination, ReceiveMessage}
|
||||
import fr.acinq.eclair.message.OnionMessages.Destination
|
||||
import fr.acinq.eclair.wire.protocol.MessageOnion.FinalPayload
|
||||
import fr.acinq.eclair.wire.protocol.{OnionMessagePayloadTlv, TlvStream}
|
||||
import fr.acinq.eclair.{randomBytes32, randomKey}
|
||||
import fr.acinq.eclair.{NodeParams, randomBytes32, randomKey}
|
||||
|
||||
import scala.collection.mutable
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
import scala.util.{Failure, Success}
|
||||
|
||||
object Postman {
|
||||
// @formatter:off
|
||||
@ -62,9 +61,9 @@ object Postman {
|
||||
case class MessageFailed(reason: String) extends MessageStatus
|
||||
// @formatter:on
|
||||
|
||||
def apply(switchboard: ActorRef[Switchboard.RelayMessage]): Behavior[Command] = {
|
||||
def apply(nodeParams: NodeParams, switchboard: ActorRef[Switchboard.RelayMessage]): Behavior[Command] = {
|
||||
Behaviors.setup(context => {
|
||||
context.system.eventStream ! EventStream.Subscribe(context.messageAdapter[ReceiveMessage](r => WrappedMessage(r.finalPayload)))
|
||||
context.system.eventStream ! EventStream.Subscribe(context.messageAdapter[OnionMessages.ReceiveMessage](r => WrappedMessage(r.finalPayload)))
|
||||
|
||||
val relayMessageStatusAdapter = context.messageAdapter[MessageRelay.Status](SendingStatus)
|
||||
|
||||
@ -94,14 +93,15 @@ object Postman {
|
||||
OnionMessages.buildRoute(randomKey(), intermediateHops, lastHop)
|
||||
})
|
||||
OnionMessages.buildMessage(
|
||||
nodeParams.privateKey,
|
||||
randomKey(),
|
||||
randomKey(),
|
||||
intermediateNodes.map(OnionMessages.IntermediateNode(_)),
|
||||
destination,
|
||||
TlvStream(replyRoute.map(OnionMessagePayloadTlv.ReplyPath).toSet ++ messageContent.records, messageContent.unknown)) match {
|
||||
case Failure(f) =>
|
||||
replyTo ! MessageFailed(f.getMessage)
|
||||
case Success((nextNodeId, message)) =>
|
||||
case Left(failure) =>
|
||||
replyTo ! MessageFailed(failure.toString)
|
||||
case Right((nextNodeId, message)) =>
|
||||
if (replyPath.isEmpty) { // not expecting reply
|
||||
sendStatusTo += (messageId -> replyTo)
|
||||
} else { // expecting reply
|
||||
|
@ -31,10 +31,11 @@ import fr.acinq.eclair.blockchain.bitcoind.rpc.BitcoinCoreClient
|
||||
import fr.acinq.eclair.channel.{CMD_CLOSE, RES_SUCCESS}
|
||||
import fr.acinq.eclair.io.Switchboard
|
||||
import fr.acinq.eclair.message.OnionMessages
|
||||
import fr.acinq.eclair.message.OnionMessages.{IntermediateNode, Recipient, buildRoute}
|
||||
import fr.acinq.eclair.router.Router
|
||||
import fr.acinq.eclair.wire.protocol.TlvCodecs.genericTlv
|
||||
import fr.acinq.eclair.wire.protocol.{GenericTlv, NodeAnnouncement}
|
||||
import fr.acinq.eclair.{EclairImpl, Features, MilliSatoshi, SendOnionMessageResponse, UInt64, randomBytes}
|
||||
import fr.acinq.eclair.{EclairImpl, Features, MilliSatoshi, SendOnionMessageResponse, UInt64, randomBytes, randomKey}
|
||||
import scodec.bits.{ByteVector, HexStringSyntax}
|
||||
|
||||
import scala.concurrent.ExecutionContext.Implicits.global
|
||||
@ -73,6 +74,21 @@ class MessageIntegrationSpec extends IntegrationSpec {
|
||||
eventListener.expectMsgType[OnionMessages.ReceiveMessage](max = 60 seconds)
|
||||
}
|
||||
|
||||
test("send to route that starts at ourselves") {
|
||||
val alice = new EclairImpl(nodes("A"))
|
||||
|
||||
val probe = TestProbe()
|
||||
val eventListener = TestProbe()
|
||||
nodes("B").system.eventStream.subscribe(eventListener.ref, classOf[OnionMessages.ReceiveMessage])
|
||||
|
||||
val blindedRoute = buildRoute(randomKey(), Seq(IntermediateNode(nodes("A").nodeParams.nodeId), IntermediateNode(nodes("B").nodeParams.nodeId), IntermediateNode(nodes("B").nodeParams.nodeId)), Recipient(nodes("B").nodeParams.nodeId, None))
|
||||
assert(blindedRoute.introductionNodeId == nodes("A").nodeParams.nodeId)
|
||||
|
||||
alice.sendOnionMessage(Nil, Right(blindedRoute), None, ByteVector.empty).pipeTo(probe.ref)
|
||||
assert(probe.expectMsgType[SendOnionMessageResponse].sent)
|
||||
eventListener.expectMsgType[OnionMessages.ReceiveMessage](max = 60 seconds)
|
||||
}
|
||||
|
||||
test("expect reply") {
|
||||
val alice = new EclairImpl(nodes("A"))
|
||||
val bob = new EclairImpl(nodes("B"))
|
||||
@ -140,7 +156,7 @@ class MessageIntegrationSpec extends IntegrationSpec {
|
||||
assert(probe.expectMsgType[SendOnionMessageResponse].sent)
|
||||
|
||||
val r = eventListener.expectMsgType[OnionMessages.ReceiveMessage](max = 60 seconds)
|
||||
assert(r.finalPayload.records.unknown.toSet == Set(GenericTlv(UInt64(113), hex"010203"), GenericTlv(UInt64(117), hex"0102")))
|
||||
assert(r.finalPayload.records.unknown == Set(GenericTlv(UInt64(113), hex"010203"), GenericTlv(UInt64(117), hex"0102")))
|
||||
}
|
||||
|
||||
test("send very large message with hop") {
|
||||
@ -157,7 +173,7 @@ class MessageIntegrationSpec extends IntegrationSpec {
|
||||
assert(probe.expectMsgType[SendOnionMessageResponse].sent)
|
||||
|
||||
val r = eventListener.expectMsgType[OnionMessages.ReceiveMessage](max = 60 seconds)
|
||||
assert(r.finalPayload.records.unknown.toSet == Set(GenericTlv(UInt64(135), bytes)))
|
||||
assert(r.finalPayload.records.unknown == Set(GenericTlv(UInt64(135), bytes)))
|
||||
}
|
||||
|
||||
test("send too large message with hop") {
|
||||
@ -266,7 +282,7 @@ class MessageIntegrationSpec extends IntegrationSpec {
|
||||
assert(probe.expectMsgType[SendOnionMessageResponse].sent)
|
||||
|
||||
val r = eventListener.expectMsgType[OnionMessages.ReceiveMessage](max = 60 seconds)
|
||||
assert(r.finalPayload.records.unknown.toSet == Set(GenericTlv(UInt64(113), hex"010203"), GenericTlv(UInt64(117), hex"0102")))
|
||||
assert(r.finalPayload.records.unknown == Set(GenericTlv(UInt64(113), hex"010203"), GenericTlv(UInt64(117), hex"0102")))
|
||||
}
|
||||
|
||||
test("channel relay with no-relay") {
|
||||
@ -344,7 +360,7 @@ class MessageIntegrationSpec extends IntegrationSpec {
|
||||
|
||||
val r = eventListener.expectMsgType[OnionMessages.ReceiveMessage](max = 60 seconds)
|
||||
assert(r.finalPayload.pathId_opt.isEmpty)
|
||||
assert(r.finalPayload.records.unknown.toSet == Set(GenericTlv(UInt64(115), hex"")))
|
||||
assert(r.finalPayload.records.unknown == Set(GenericTlv(UInt64(115), hex"")))
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -58,7 +58,7 @@ class MessageRelaySpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app
|
||||
test("relay with new connection") { f =>
|
||||
import f._
|
||||
|
||||
val Success((_, message)) = OnionMessages.buildMessage(randomKey(), randomKey(), Seq(IntermediateNode(aliceId)), Recipient(bobId, None), TlvStream.empty)
|
||||
val Right((_, message)) = OnionMessages.buildMessage(randomKey(), randomKey(), randomKey(), Seq(IntermediateNode(aliceId)), Recipient(bobId, None), TlvStream.empty)
|
||||
val messageId = randomBytes32()
|
||||
relay ! RelayMessage(messageId, switchboard.ref, randomKey().publicKey, bobId, message, RelayAll, None)
|
||||
|
||||
@ -71,7 +71,7 @@ class MessageRelaySpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app
|
||||
test("relay with existing peer") { f =>
|
||||
import f._
|
||||
|
||||
val Success((_, message)) = OnionMessages.buildMessage(randomKey(), randomKey(), Seq(IntermediateNode(aliceId)), Recipient(bobId, None), TlvStream.empty)
|
||||
val Right((_, message)) = OnionMessages.buildMessage(randomKey(), randomKey(), randomKey(), Seq(IntermediateNode(aliceId)), Recipient(bobId, None), TlvStream.empty)
|
||||
val messageId = randomBytes32()
|
||||
relay ! RelayMessage(messageId, switchboard.ref, randomKey().publicKey, bobId, message, RelayAll, None)
|
||||
|
||||
@ -84,7 +84,7 @@ class MessageRelaySpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app
|
||||
test("can't open new connection") { f =>
|
||||
import f._
|
||||
|
||||
val Success((_, message)) = OnionMessages.buildMessage(randomKey(), randomKey(), Seq(IntermediateNode(aliceId)), Recipient(bobId, None), TlvStream.empty)
|
||||
val Right((_, message)) = OnionMessages.buildMessage(randomKey(), randomKey(), randomKey(), Seq(IntermediateNode(aliceId)), Recipient(bobId, None), TlvStream.empty)
|
||||
val messageId = randomBytes32()
|
||||
relay ! RelayMessage(messageId, switchboard.ref, randomKey().publicKey, bobId, message, RelayAll, Some(probe.ref))
|
||||
|
||||
@ -97,7 +97,7 @@ class MessageRelaySpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app
|
||||
test("no channel with previous node") { f =>
|
||||
import f._
|
||||
|
||||
val Success((_, message)) = OnionMessages.buildMessage(randomKey(), randomKey(), Seq(IntermediateNode(aliceId)), Recipient(bobId, None), TlvStream.empty)
|
||||
val Right((_, message)) = OnionMessages.buildMessage(randomKey(), randomKey(), randomKey(), Seq(IntermediateNode(aliceId)), Recipient(bobId, None), TlvStream.empty)
|
||||
val messageId = randomBytes32()
|
||||
val previousNodeId = randomKey().publicKey
|
||||
relay ! RelayMessage(messageId, switchboard.ref, previousNodeId, bobId, message, RelayChannelsOnly, Some(probe.ref))
|
||||
@ -113,7 +113,7 @@ class MessageRelaySpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app
|
||||
test("no channel with next node") { f =>
|
||||
import f._
|
||||
|
||||
val Success((_, message)) = OnionMessages.buildMessage(randomKey(), randomKey(), Seq(IntermediateNode(aliceId)), Recipient(bobId, None), TlvStream.empty)
|
||||
val Right((_, message)) = OnionMessages.buildMessage(randomKey(), randomKey(), randomKey(), Seq(IntermediateNode(aliceId)), Recipient(bobId, None), TlvStream.empty)
|
||||
val messageId = randomBytes32()
|
||||
val previousNodeId = randomKey().publicKey
|
||||
relay ! RelayMessage(messageId, switchboard.ref, previousNodeId, bobId, message, RelayChannelsOnly, Some(probe.ref))
|
||||
@ -133,7 +133,7 @@ class MessageRelaySpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app
|
||||
test("channels on both ends") { f =>
|
||||
import f._
|
||||
|
||||
val Success((_, message)) = OnionMessages.buildMessage(randomKey(), randomKey(), Seq(IntermediateNode(aliceId)), Recipient(bobId, None), TlvStream.empty)
|
||||
val Right((_, message)) = OnionMessages.buildMessage(randomKey(), randomKey(), randomKey(), Seq(IntermediateNode(aliceId)), Recipient(bobId, None), TlvStream.empty)
|
||||
val messageId = randomBytes32()
|
||||
val previousNodeId = randomKey().publicKey
|
||||
relay ! RelayMessage(messageId, switchboard.ref, previousNodeId, bobId, message, RelayChannelsOnly, None)
|
||||
@ -152,7 +152,7 @@ class MessageRelaySpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app
|
||||
test("no relay") { f =>
|
||||
import f._
|
||||
|
||||
val Success((_, message)) = OnionMessages.buildMessage(randomKey(), randomKey(), Seq(IntermediateNode(aliceId)), Recipient(bobId, None), TlvStream.empty)
|
||||
val Right((_, message)) = OnionMessages.buildMessage(randomKey(), randomKey(), randomKey(), Seq(IntermediateNode(aliceId)), Recipient(bobId, None), TlvStream.empty)
|
||||
val messageId = randomBytes32()
|
||||
val previousNodeId = randomKey().publicKey
|
||||
relay ! RelayMessage(messageId, switchboard.ref, previousNodeId, bobId, message, NoRelay, Some(probe.ref))
|
||||
|
@ -392,7 +392,7 @@ class PeerConnectionSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wi
|
||||
import f._
|
||||
connect(nodeParams, remoteNodeId, switchboard, router, connection, transport, peerConnection, peer, isPersistent = false)
|
||||
val probe = TestProbe()
|
||||
val Success((_, message)) = buildMessage(randomKey(), randomKey(), Nil, Recipient(remoteNodeId, None), TlvStream.empty)
|
||||
val Right((_, message)) = buildMessage(nodeParams.privateKey, randomKey(), randomKey(), Nil, Recipient(remoteNodeId, None), TlvStream.empty)
|
||||
probe.send(peerConnection, message)
|
||||
probe watch peerConnection
|
||||
probe.expectTerminated(peerConnection, max = 1500 millis)
|
||||
@ -408,7 +408,7 @@ class PeerConnectionSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wi
|
||||
import f._
|
||||
connect(nodeParams, remoteNodeId, switchboard, router, connection, transport, peerConnection, peer, isPersistent = false)
|
||||
val probe = TestProbe()
|
||||
val Success((_, message)) = buildMessage(randomKey(), randomKey(), Nil, Recipient(remoteNodeId, None), TlvStream.empty)
|
||||
val Right((_, message)) = buildMessage(nodeParams.privateKey, randomKey(), randomKey(), Nil, Recipient(remoteNodeId, None), TlvStream.empty)
|
||||
probe watch peerConnection
|
||||
probe.send(peerConnection, message)
|
||||
// The connection is still open for a short while.
|
||||
@ -421,7 +421,7 @@ class PeerConnectionSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wi
|
||||
import f._
|
||||
connect(nodeParams, remoteNodeId, switchboard, router, connection, transport, peerConnection, peer, isPersistent = false)
|
||||
val probe = TestProbe()
|
||||
val Success((_, message)) = buildMessage(randomKey(), randomKey(), Nil, Recipient(remoteNodeId, None), TlvStream.empty)
|
||||
val Right((_, message)) = buildMessage(nodeParams.privateKey, randomKey(), randomKey(), Nil, Recipient(remoteNodeId, None), TlvStream.empty)
|
||||
probe.send(peerConnection, message)
|
||||
assert(peerConnection.stateName == PeerConnection.CONNECTED)
|
||||
probe.send(peerConnection, ChannelReady(ByteVector32(hex"0000000000000000000000000000000000000000000000000000000000000000"), randomKey().publicKey))
|
||||
@ -434,7 +434,7 @@ class PeerConnectionSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wi
|
||||
test("incoming rate limiting") { f =>
|
||||
import f._
|
||||
connect(nodeParams, remoteNodeId, switchboard, router, connection, transport, peerConnection, peer, isPersistent = true)
|
||||
val Success((_, message)) = buildMessage(randomKey(), randomKey(), Nil, Recipient(nodeParams.nodeId, None), TlvStream.empty)
|
||||
val Right((_, message)) = buildMessage(nodeParams.privateKey, randomKey(), randomKey(), Nil, Recipient(nodeParams.nodeId, None), TlvStream.empty)
|
||||
for (_ <- 1 to 30) {
|
||||
transport.send(peerConnection, message)
|
||||
}
|
||||
@ -453,7 +453,7 @@ class PeerConnectionSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wi
|
||||
test("outgoing rate limiting") { f =>
|
||||
import f._
|
||||
connect(nodeParams, remoteNodeId, switchboard, router, connection, transport, peerConnection, peer, isPersistent = true)
|
||||
val Success((_, message)) = buildMessage(randomKey(), randomKey(), Nil, Recipient(remoteNodeId, None), TlvStream.empty)
|
||||
val Right((_, message)) = buildMessage(nodeParams.privateKey, randomKey(), randomKey(), Nil, Recipient(remoteNodeId, None), TlvStream.empty)
|
||||
for (_ <- 1 to 30) {
|
||||
peer.send(peerConnection, message)
|
||||
}
|
||||
|
@ -642,7 +642,7 @@ class PeerSpec extends FixtureSpec {
|
||||
test("reply to relay request") { f =>
|
||||
import f._
|
||||
connect(remoteNodeId, peer, peerConnection, switchboard, channels = Set(ChannelCodecsSpec.normal))
|
||||
val Success((_, msg)) = buildMessage(randomKey(), randomKey(), Nil, Recipient(remoteNodeId, None), TlvStream.empty)
|
||||
val Right((_, msg)) = buildMessage(nodeParams.privateKey, randomKey(), randomKey(), Nil, Recipient(remoteNodeId, None), TlvStream.empty)
|
||||
val messageId = randomBytes32()
|
||||
val probe = TestProbe()
|
||||
peer ! RelayOnionMessage(messageId, msg, Some(probe.ref.toTyped))
|
||||
@ -651,7 +651,7 @@ class PeerSpec extends FixtureSpec {
|
||||
|
||||
test("reply to relay request disconnected") { f =>
|
||||
import f._
|
||||
val Success((_, msg)) = buildMessage(randomKey(), randomKey(), Nil, Recipient(remoteNodeId, None), TlvStream.empty)
|
||||
val Right((_, msg)) = buildMessage(nodeParams.privateKey, randomKey(), randomKey(), Nil, Recipient(remoteNodeId, None), TlvStream.empty)
|
||||
val messageId = randomBytes32()
|
||||
val probe = TestProbe()
|
||||
peer ! RelayOnionMessage(messageId, msg, Some(probe.ref.toTyped))
|
||||
|
@ -39,10 +39,11 @@ import scala.util.Success
|
||||
class OnionMessagesSpec extends AnyFunSuite {
|
||||
|
||||
test("single-hop onion message without path_id") {
|
||||
val nodeKey = randomKey()
|
||||
val sessionKey = randomKey()
|
||||
val blindingSecret = randomKey()
|
||||
val destination = randomKey()
|
||||
val Success((nextNodeId, message)) = buildMessage(sessionKey, blindingSecret, Nil, Recipient(destination.publicKey, None), TlvStream.empty)
|
||||
val Right((nextNodeId, message)) = buildMessage(nodeKey, sessionKey, blindingSecret, Nil, Recipient(destination.publicKey, None), TlvStream.empty)
|
||||
assert(nextNodeId == destination.publicKey)
|
||||
|
||||
process(destination, message) match {
|
||||
@ -108,7 +109,7 @@ class OnionMessagesSpec extends AnyFunSuite {
|
||||
// Building the onion with functions from `OnionMessages`
|
||||
val replyPath = buildRoute(blindingOverride, IntermediateNode(carol.publicKey, padding = Some(hex"0000000000000000000000000000000000000000000000000000000000000000000000")) :: Nil, Recipient(dave.publicKey, pathId = Some(hex"01234567")))
|
||||
assert(replyPath == routeFromCarol)
|
||||
val Success((_, message)) = buildMessage(sessionKey, blindingSecret, IntermediateNode(alice.publicKey) :: IntermediateNode(bob.publicKey) :: Nil, BlindedPath(replyPath), TlvStream.empty)
|
||||
val Right((_, message)) = buildMessage(randomKey(), sessionKey, blindingSecret, IntermediateNode(alice.publicKey) :: IntermediateNode(bob.publicKey) :: Nil, BlindedPath(replyPath), TlvStream.empty)
|
||||
assert(message == onionForAlice)
|
||||
|
||||
// Checking that the onion is relayed properly
|
||||
@ -202,6 +203,7 @@ class OnionMessagesSpec extends AnyFunSuite {
|
||||
}
|
||||
|
||||
test("build message with existing route") {
|
||||
val nodeKey = randomKey()
|
||||
val sessionKey = randomKey()
|
||||
val blindingSecret = randomKey()
|
||||
val blindingOverride = randomKey()
|
||||
@ -209,7 +211,7 @@ class OnionMessagesSpec extends AnyFunSuite {
|
||||
val replyPath = buildRoute(blindingOverride, IntermediateNode(destination.publicKey) :: Nil, Recipient(destination.publicKey, pathId = Some(hex"01234567")))
|
||||
assert(replyPath.blindingKey == blindingOverride.publicKey)
|
||||
assert(replyPath.introductionNodeId == destination.publicKey)
|
||||
val Success((nextNodeId, message)) = buildMessage(sessionKey, blindingSecret, Nil, BlindedPath(replyPath), TlvStream.empty)
|
||||
val Right((nextNodeId, message)) = buildMessage(nodeKey, sessionKey, blindingSecret, Nil, BlindedPath(replyPath), TlvStream.empty)
|
||||
assert(nextNodeId == destination.publicKey)
|
||||
assert(message.blindingKey == blindingOverride.publicKey) // blindingSecret was not used as the replyPath was used as is
|
||||
|
||||
@ -220,13 +222,14 @@ class OnionMessagesSpec extends AnyFunSuite {
|
||||
}
|
||||
|
||||
test("very large multi-hop onion message") {
|
||||
val nodeKey = randomKey()
|
||||
val alice = randomKey()
|
||||
val bob = randomKey()
|
||||
val carol = randomKey()
|
||||
val sessionKey = randomKey()
|
||||
val blindingSecret = randomKey()
|
||||
val pathId = randomBytes(65201)
|
||||
val Success((_, messageForAlice)) = buildMessage(sessionKey, blindingSecret, IntermediateNode(alice.publicKey) :: IntermediateNode(bob.publicKey) :: Nil, Recipient(carol.publicKey, Some(pathId)), TlvStream.empty)
|
||||
val Right((_, messageForAlice)) = buildMessage(nodeKey, sessionKey, blindingSecret, IntermediateNode(alice.publicKey) :: IntermediateNode(bob.publicKey) :: Nil, Recipient(carol.publicKey, Some(pathId)), TlvStream.empty)
|
||||
|
||||
// Checking that the onion is relayed properly
|
||||
process(alice, messageForAlice) match {
|
||||
@ -246,6 +249,7 @@ class OnionMessagesSpec extends AnyFunSuite {
|
||||
}
|
||||
|
||||
test("too large multi-hop onion message") {
|
||||
val nodeKey = randomKey()
|
||||
val alice = randomKey()
|
||||
val bob = randomKey()
|
||||
val carol = randomKey()
|
||||
@ -254,6 +258,6 @@ class OnionMessagesSpec extends AnyFunSuite {
|
||||
|
||||
val pathId = randomBytes(65202)
|
||||
|
||||
assert(buildMessage(sessionKey, blindingSecret, IntermediateNode(alice.publicKey) :: IntermediateNode(bob.publicKey) :: Nil, Recipient(carol.publicKey, Some(pathId)), TlvStream.empty).isFailure)
|
||||
assert(buildMessage(nodeKey, sessionKey, blindingSecret, IntermediateNode(alice.publicKey) :: IntermediateNode(bob.publicKey) :: Nil, Recipient(carol.publicKey, Some(pathId)), TlvStream.empty) == Left(MessageTooLarge(65433)))
|
||||
}
|
||||
}
|
||||
|
@ -22,10 +22,10 @@ import akka.actor.typed.eventstream.EventStream
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import fr.acinq.eclair.io.MessageRelay.{Disconnected, Sent}
|
||||
import fr.acinq.eclair.io.Switchboard.RelayMessage
|
||||
import fr.acinq.eclair.message.OnionMessages.{BlindedPath, ReceiveMessage, Recipient, buildMessage}
|
||||
import fr.acinq.eclair.message.OnionMessages.{BlindedPath, IntermediateNode, ReceiveMessage, Recipient, buildMessage, buildRoute}
|
||||
import fr.acinq.eclair.message.Postman._
|
||||
import fr.acinq.eclair.wire.protocol.{GenericTlv, OnionMessagePayloadTlv, TlvStream}
|
||||
import fr.acinq.eclair.{UInt64, randomKey}
|
||||
import fr.acinq.eclair.{NodeParams, TestConstants, UInt64, randomKey}
|
||||
import org.scalatest.Outcome
|
||||
import org.scalatest.funsuite.FixtureAnyFunSuiteLike
|
||||
import scodec.bits.HexStringSyntax
|
||||
@ -35,14 +35,15 @@ import scala.util.Success
|
||||
|
||||
class PostmanSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("application")) with FixtureAnyFunSuiteLike {
|
||||
|
||||
case class FixtureParam(postman: ActorRef[Command], messageRecipient: TestProbe[OnionMessageResponse], switchboard: TestProbe[RelayMessage])
|
||||
case class FixtureParam(postman: ActorRef[Command], nodeParams: NodeParams, messageRecipient: TestProbe[OnionMessageResponse], switchboard: TestProbe[RelayMessage])
|
||||
|
||||
override def withFixture(test: OneArgTest): Outcome = {
|
||||
val nodeParams = TestConstants.Alice.nodeParams
|
||||
val messageRecipient = TestProbe[OnionMessageResponse]("messageRecipient")
|
||||
val switchboard = TestProbe[RelayMessage]("switchboard")
|
||||
val postman = testKit.spawn(Postman(switchboard.ref))
|
||||
val postman = testKit.spawn(Postman(nodeParams, switchboard.ref))
|
||||
try {
|
||||
withFixture(test.toNoArgTest(FixtureParam(postman, messageRecipient, switchboard)))
|
||||
withFixture(test.toNoArgTest(FixtureParam(postman, nodeParams, messageRecipient, switchboard)))
|
||||
} finally {
|
||||
testKit.stop(postman)
|
||||
}
|
||||
@ -63,7 +64,7 @@ class PostmanSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat
|
||||
assert(finalPayload.records.unknown == Set(GenericTlv(UInt64(33), hex"abcd")))
|
||||
|
||||
val replyPath = finalPayload.replyPath_opt.get
|
||||
val Success((_, reply)) = buildMessage(randomKey(), randomKey(), Nil, BlindedPath(replyPath), TlvStream(Set.empty[OnionMessagePayloadTlv], Set(GenericTlv(UInt64(55), hex"1234"))))
|
||||
val Right((_, reply)) = buildMessage(recipientKey, randomKey(), randomKey(), Nil, BlindedPath(replyPath), TlvStream(Set.empty[OnionMessagePayloadTlv], Set(GenericTlv(UInt64(55), hex"1234"))))
|
||||
val ReceiveMessage(replyPayload) = OnionMessages.process(ourKey, reply)
|
||||
|
||||
testKit.system.eventStream ! EventStream.Publish(ReceiveMessage(replyPayload))
|
||||
@ -106,7 +107,7 @@ class PostmanSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat
|
||||
messageRecipient.expectMessage(NoReply)
|
||||
|
||||
val replyPath = finalPayload.replyPath_opt.get
|
||||
val Success((_, reply)) = buildMessage(randomKey(), randomKey(), Nil, BlindedPath(replyPath), TlvStream(Set.empty[OnionMessagePayloadTlv], Set(GenericTlv(UInt64(55), hex"1234"))))
|
||||
val Right((_, reply)) = buildMessage(recipientKey, randomKey(), randomKey(), Nil, BlindedPath(replyPath), TlvStream(Set.empty[OnionMessagePayloadTlv], Set(GenericTlv(UInt64(55), hex"1234"))))
|
||||
val ReceiveMessage(replyPayload) = OnionMessages.process(ourKey, reply)
|
||||
testKit.system.eventStream ! EventStream.Publish(ReceiveMessage(replyPayload))
|
||||
|
||||
@ -130,4 +131,23 @@ class PostmanSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat
|
||||
messageRecipient.expectMessage(MessageSent)
|
||||
messageRecipient.expectNoMessage()
|
||||
}
|
||||
|
||||
test("send to route that starts at ourselves") {f =>
|
||||
import f._
|
||||
|
||||
val recipientKey = randomKey()
|
||||
|
||||
val blindedRoute = buildRoute(randomKey(), Seq(IntermediateNode(nodeParams.nodeId)), Recipient(recipientKey.publicKey, None))
|
||||
postman ! SendMessage(Nil, BlindedPath(blindedRoute), None, TlvStream(Set.empty[OnionMessagePayloadTlv], Set(GenericTlv(UInt64(33), hex"abcd"))), messageRecipient.ref, 100 millis)
|
||||
|
||||
val RelayMessage(messageId, _, nextNodeId, message, _, _) = switchboard.expectMessageType[RelayMessage]
|
||||
assert(nextNodeId == recipientKey.publicKey)
|
||||
postman ! SendingStatus(Sent(messageId))
|
||||
val ReceiveMessage(finalPayload) = OnionMessages.process(recipientKey, message)
|
||||
assert(finalPayload.records.unknown == Set(GenericTlv(UInt64(33), hex"abcd")))
|
||||
assert(finalPayload.replyPath_opt.isEmpty)
|
||||
|
||||
messageRecipient.expectMessage(MessageSent)
|
||||
messageRecipient.expectNoMessage()
|
||||
}
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user