Skip to content

Commit

Permalink
Return remote features from PeerReadyNotifier
Browse files Browse the repository at this point in the history
This is useful to know what features we can use to relay to nodes that
we just woke up.
  • Loading branch information
t-bast committed Sep 11, 2024
1 parent 9f1ace1 commit ec90ec1
Show file tree
Hide file tree
Showing 9 changed files with 63 additions and 53 deletions.
11 changes: 6 additions & 5 deletions eclair-core/src/main/scala/fr/acinq/eclair/io/Peer.scala
Original file line number Diff line number Diff line change
Expand Up @@ -475,10 +475,11 @@ class Peer(val nodeParams: NodeParams,

case Event(r: GetPeerInfo, d) =>
val replyTo = r.replyTo.getOrElse(sender().toTyped)
replyTo ! PeerInfo(self, remoteNodeId, stateName, d match {
case c: ConnectedData => Some(c.address)
case _ => None
}, d.channels.values.toSet)
val peerInfo = d match {
case c: ConnectedData => PeerInfo(self, remoteNodeId, stateName, Some(c.remoteFeatures), Some(c.address), c.channels.values.toSet)
case _ => PeerInfo(self, remoteNodeId, stateName, None, None, d.channels.values.toSet)
}
replyTo ! peerInfo
stay()

case Event(r: GetPeerChannels, d) =>
Expand Down Expand Up @@ -867,7 +868,7 @@ object Peer {

case class GetPeerInfo(replyTo: Option[typed.ActorRef[PeerInfoResponse]])
sealed trait PeerInfoResponse { def nodeId: PublicKey }
case class PeerInfo(peer: ActorRef, nodeId: PublicKey, state: State, address: Option[NodeAddress], channels: Set[ActorRef]) extends PeerInfoResponse
case class PeerInfo(peer: ActorRef, nodeId: PublicKey, state: State, features: Option[Features[InitFeature]], address: Option[NodeAddress], channels: Set[ActorRef]) extends PeerInfoResponse
case class PeerNotFound(nodeId: PublicKey) extends PeerInfoResponse with DisconnectResponse { override def toString: String = s"peer $nodeId not found" }

/** Return the peer's current channels: note that the data may change concurrently, never assume it is fully up-to-date. */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ import akka.actor.typed.scaladsl.{ActorContext, Behaviors, TimerScheduler}
import akka.actor.typed.{ActorRef, Behavior, SupervisorStrategy}
import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
import fr.acinq.eclair.blockchain.CurrentBlockHeight
import fr.acinq.eclair.{BlockHeight, Logs, channel}
import fr.acinq.eclair.{BlockHeight, Features, InitFeature, Logs, channel}

import scala.concurrent.duration.{DurationInt, FiniteDuration}

Expand Down Expand Up @@ -107,15 +107,15 @@ object PeerReadyNotifier {
private case object PeerNotConnected extends Command
private case object PeerConnected extends Command
private case object PeerDisconnected extends Command
private case class WrappedPeerInfo(peer: ActorRef[Peer.GetPeerChannels], channelCount: Int) extends Command
private case class WrappedPeerInfo(peer: ActorRef[Peer.GetPeerChannels], remoteFeatures: Features[InitFeature], channelCount: Int) extends Command
private case class NewBlockNotTimedOut(currentBlockHeight: BlockHeight) extends Command
private case object CheckChannelsReady extends Command
private case class WrappedPeerChannels(wrapped: Peer.PeerChannels) extends Command
private case object Timeout extends Command
private case object ToBeIgnored extends Command

sealed trait Result { def remoteNodeId: PublicKey }
case class PeerReady(remoteNodeId: PublicKey, peer: akka.actor.ActorRef, channelInfos: Seq[Peer.ChannelInfo]) extends Result { val channelsCount: Int = channelInfos.size }
case class PeerReady(remoteNodeId: PublicKey, peer: akka.actor.ActorRef, remoteFeatures: Features[InitFeature], channelInfos: Seq[Peer.ChannelInfo]) extends Result { val channelsCount: Int = channelInfos.size }
case class PeerUnavailable(remoteNodeId: PublicKey) extends Result

private case object ChannelsReadyTimerKey
Expand Down Expand Up @@ -243,7 +243,7 @@ private class PeerReadyNotifier(replyTo: ActorRef[PeerReadyNotifier.Result],
// In that case we still want to wait for a connection, because we may want to open a channel to them.
case _: Peer.PeerNotFound => PeerNotConnected
case info: Peer.PeerInfo if info.state != Peer.CONNECTED => PeerNotConnected
case info: Peer.PeerInfo => WrappedPeerInfo(info.peer.toTyped, info.channels.size)
case info: Peer.PeerInfo => WrappedPeerInfo(info.peer.toTyped, info.features.getOrElse(Features.empty), info.channels.size)
}
// We check whether the peer is already connected.
switchboard ! Switchboard.GetPeerInfo(peerInfoAdapter, remoteNodeId)
Expand All @@ -256,14 +256,14 @@ private class PeerReadyNotifier(replyTo: ActorRef[PeerReadyNotifier.Result],
Behaviors.same
case PeerDisconnected =>
Behaviors.same
case WrappedPeerInfo(peer, channelCount) =>
case WrappedPeerInfo(peer, remoteFeatures, channelCount) =>
if (channelCount == 0) {
log.info("peer is ready with no channels")
replyTo ! PeerReady(remoteNodeId, peer.toClassic, Seq.empty)
replyTo ! PeerReady(remoteNodeId, peer.toClassic, remoteFeatures, Seq.empty)
Behaviors.stopped
} else {
log.debug("peer is connected with {} channels", channelCount)
waitForChannelsReady(peer, switchboard)
waitForChannelsReady(peer, switchboard, remoteFeatures)
}
case NewBlockNotTimedOut(currentBlockHeight) =>
log.debug("waiting for peer to connect at block {}", currentBlockHeight)
Expand All @@ -277,7 +277,7 @@ private class PeerReadyNotifier(replyTo: ActorRef[PeerReadyNotifier.Result],
}
}

private def waitForChannelsReady(peer: ActorRef[Peer.GetPeerChannels], switchboard: ActorRef[Switchboard.GetPeerInfo]): Behavior[Command] = {
private def waitForChannelsReady(peer: ActorRef[Peer.GetPeerChannels], switchboard: ActorRef[Switchboard.GetPeerInfo], remoteFeatures: Features[InitFeature]): Behavior[Command] = {
timers.startTimerWithFixedDelay(ChannelsReadyTimerKey, CheckChannelsReady, initialDelay = 50 millis, delay = 1 second)
Behaviors.receiveMessagePartial {
case CheckChannelsReady =>
Expand All @@ -286,7 +286,7 @@ private class PeerReadyNotifier(replyTo: ActorRef[PeerReadyNotifier.Result],
Behaviors.same
case WrappedPeerChannels(peerChannels) =>
if (peerChannels.channels.map(_.state).forall(isChannelReady)) {
replyTo ! PeerReady(remoteNodeId, peer.toClassic, peerChannels.channels)
replyTo ! PeerReady(remoteNodeId, peer.toClassic, remoteFeatures, peerChannels.channels)
Behaviors.stopped
} else {
log.debug("peer has {} channels that are not ready", peerChannels.channels.count(s => !isChannelReady(s.state)))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -112,7 +112,7 @@ class MessageRelaySpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app

val request = switchboard.expectMsgType[GetPeerInfo]
assert(request.remoteNodeId == bobId)
request.replyTo ! Peer.PeerInfo(peer.ref.toClassic, bobId, Peer.CONNECTED, None, Set.empty)
request.replyTo ! Peer.PeerInfo(peer.ref.toClassic, bobId, Peer.CONNECTED, None, None, Set.empty)
assert(peer.expectMessageType[Peer.RelayOnionMessage].msg == message)
}

Expand Down Expand Up @@ -148,7 +148,7 @@ class MessageRelaySpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app

val getPeerInfo = switchboard.expectMsgType[GetPeerInfo]
assert(getPeerInfo.remoteNodeId == previousNodeId)
getPeerInfo.replyTo ! PeerInfo(peer.ref.toClassic, previousNodeId, Peer.CONNECTED, None, Set.empty)
getPeerInfo.replyTo ! PeerInfo(peer.ref.toClassic, previousNodeId, Peer.CONNECTED, None, None, Set.empty)

probe.expectMessage(AgainstPolicy(messageId, RelayChannelsOnly))
peer.expectNoMessage(100 millis)
Expand All @@ -164,7 +164,7 @@ class MessageRelaySpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app

val getPeerInfo1 = switchboard.expectMsgType[GetPeerInfo]
assert(getPeerInfo1.remoteNodeId == previousNodeId)
getPeerInfo1.replyTo ! PeerInfo(peer.ref.toClassic, previousNodeId, Peer.CONNECTED, None, Set(TestProbe()(system.classicSystem).ref))
getPeerInfo1.replyTo ! PeerInfo(peer.ref.toClassic, previousNodeId, Peer.CONNECTED, None, None, Set(TestProbe()(system.classicSystem).ref))

val getPeerInfo2 = switchboard.expectMsgType[GetPeerInfo]
assert(getPeerInfo2.remoteNodeId == bobId)
Expand All @@ -184,11 +184,11 @@ class MessageRelaySpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app

val getPeerInfo1 = switchboard.expectMsgType[GetPeerInfo]
assert(getPeerInfo1.remoteNodeId == previousNodeId)
getPeerInfo1.replyTo ! PeerInfo(TestProbe()(system.classicSystem).ref, previousNodeId, Peer.CONNECTED, None, Set(TestProbe()(system.classicSystem).ref))
getPeerInfo1.replyTo ! PeerInfo(TestProbe()(system.classicSystem).ref, previousNodeId, Peer.CONNECTED, None, None, Set(TestProbe()(system.classicSystem).ref))

val getPeerInfo2 = switchboard.expectMsgType[GetPeerInfo]
assert(getPeerInfo2.remoteNodeId == bobId)
getPeerInfo2.replyTo ! PeerInfo(peer.ref.toClassic, bobId, Peer.CONNECTED, None, Set(0, 1).map(_ => TestProbe()(system.classicSystem).ref))
getPeerInfo2.replyTo ! PeerInfo(peer.ref.toClassic, bobId, Peer.CONNECTED, None, None, Set(0, 1).map(_ => TestProbe()(system.classicSystem).ref))

assert(peer.expectMessageType[Peer.RelayOnionMessage].msg == message)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,14 +25,20 @@ import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
import fr.acinq.eclair.blockchain.CurrentBlockHeight
import fr.acinq.eclair.channel._
import fr.acinq.eclair.io.PeerReadyNotifier.{NotifyWhenPeerReady, PeerUnavailable}
import fr.acinq.eclair.{BlockHeight, randomKey}
import fr.acinq.eclair.{BlockHeight, FeatureSupport, Features, randomKey}
import org.scalatest.Outcome
import org.scalatest.funsuite.FixtureAnyFunSuiteLike

import scala.concurrent.duration.DurationInt

class PeerReadyNotifierSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("application")) with FixtureAnyFunSuiteLike {

private val remoteFeatures = Features(
Features.StaticRemoteKey -> FeatureSupport.Optional,
Features.AnchorOutputsZeroFeeHtlcTx -> FeatureSupport.Optional,
Features.RouteBlinding -> FeatureSupport.Optional,
).initFeatures()

case class FixtureParam(remoteNodeId: PublicKey, peerReadyManager: TestProbe[PeerReadyManager.Register], switchboard: TestProbe[Switchboard.GetPeerInfo], peer: TestProbe[Peer.GetPeerChannels], probe: TestProbe[PeerReadyNotifier.Result])

override def withFixture(test: OneArgTest): Outcome = {
Expand Down Expand Up @@ -84,8 +90,8 @@ class PeerReadyNotifierSpec extends ScalaTestWithActorTestKit(ConfigFactory.load
notifier ! NotifyWhenPeerReady(probe.ref)
peerReadyManager.expectMessageType[PeerReadyManager.Register].replyTo ! PeerReadyManager.Registered(remoteNodeId, otherAttempts = 0)
val request = switchboard.expectMessageType[Switchboard.GetPeerInfo]
request.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.CONNECTED, None, Set.empty)
probe.expectMessage(PeerReadyNotifier.PeerReady(remoteNodeId, peer.ref.toClassic, Seq.empty))
request.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.CONNECTED, Some(remoteFeatures), None, Set.empty)
probe.expectMessage(PeerReadyNotifier.PeerReady(remoteNodeId, peer.ref.toClassic, remoteFeatures, Seq.empty))
}

test("peer connected (with channels)") { f =>
Expand All @@ -95,7 +101,7 @@ class PeerReadyNotifierSpec extends ScalaTestWithActorTestKit(ConfigFactory.load
notifier ! NotifyWhenPeerReady(probe.ref)
peerReadyManager.expectMessageType[PeerReadyManager.Register].replyTo ! PeerReadyManager.Registered(remoteNodeId, otherAttempts = 0)
val request1 = switchboard.expectMessageType[Switchboard.GetPeerInfo]
request1.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.CONNECTED, None, Set(TestProbe().ref.toClassic, TestProbe().ref.toClassic))
request1.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.CONNECTED, Some(remoteFeatures), None, Set(TestProbe().ref.toClassic, TestProbe().ref.toClassic))

// Channels are not ready yet.
val channels2 = Seq(Peer.ChannelInfo(null, SYNCING, null), Peer.ChannelInfo(null, SYNCING, null))
Expand All @@ -113,7 +119,7 @@ class PeerReadyNotifierSpec extends ScalaTestWithActorTestKit(ConfigFactory.load
val channels4 = Seq(Peer.ChannelInfo(null, NORMAL, null), Peer.ChannelInfo(null, SHUTDOWN, null))
val request4 = peer.expectMessageType[Peer.GetPeerChannels]
request4.replyTo ! Peer.PeerChannels(remoteNodeId, channels4)
probe.expectMessage(PeerReadyNotifier.PeerReady(remoteNodeId, peer.ref.toClassic, channels4))
probe.expectMessage(PeerReadyNotifier.PeerReady(remoteNodeId, peer.ref.toClassic, remoteFeatures, channels4))
}

test("peer connects after initial request") { f =>
Expand All @@ -123,7 +129,7 @@ class PeerReadyNotifierSpec extends ScalaTestWithActorTestKit(ConfigFactory.load
notifier ! NotifyWhenPeerReady(probe.ref)
peerReadyManager.expectMessageType[PeerReadyManager.Register].replyTo ! PeerReadyManager.Registered(remoteNodeId, otherAttempts = 1)
val request1 = switchboard.expectMessageType[Switchboard.GetPeerInfo]
request1.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.DISCONNECTED, None, Set(TestProbe().ref.toClassic, TestProbe().ref.toClassic))
request1.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.DISCONNECTED, None, None, Set(TestProbe().ref.toClassic, TestProbe().ref.toClassic))
peer.expectNoMessage(100 millis)

// An unrelated peer connects.
Expand All @@ -133,10 +139,10 @@ class PeerReadyNotifierSpec extends ScalaTestWithActorTestKit(ConfigFactory.load
// The target peer connects.
system.eventStream ! EventStream.Publish(PeerConnected(TestProbe().ref.toClassic, remoteNodeId, null))
val request2 = switchboard.expectMessageType[Switchboard.GetPeerInfo]
request2.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.CONNECTED, None, Set(TestProbe().ref.toClassic, TestProbe().ref.toClassic))
request2.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.CONNECTED, Some(remoteFeatures), None, Set(TestProbe().ref.toClassic, TestProbe().ref.toClassic))
val channels = Seq(Peer.ChannelInfo(null, NEGOTIATING, null))
peer.expectMessageType[Peer.GetPeerChannels].replyTo ! Peer.PeerChannels(remoteNodeId, channels)
probe.expectMessage(PeerReadyNotifier.PeerReady(remoteNodeId, peer.ref.toClassic, channels))
probe.expectMessage(PeerReadyNotifier.PeerReady(remoteNodeId, peer.ref.toClassic, remoteFeatures, channels))
}

test("peer connects then disconnects") { f =>
Expand All @@ -152,16 +158,16 @@ class PeerReadyNotifierSpec extends ScalaTestWithActorTestKit(ConfigFactory.load
// The target peer connects and instantly disconnects.
system.eventStream ! EventStream.Publish(PeerConnected(TestProbe().ref.toClassic, remoteNodeId, null))
val request2 = switchboard.expectMessageType[Switchboard.GetPeerInfo]
request2.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.DISCONNECTED, None, Set(TestProbe().ref.toClassic))
request2.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.DISCONNECTED, None, None, Set(TestProbe().ref.toClassic))
peer.expectNoMessage(100 millis)

// The target peer reconnects and stays connected.
system.eventStream ! EventStream.Publish(PeerConnected(TestProbe().ref.toClassic, remoteNodeId, null))
val request3 = switchboard.expectMessageType[Switchboard.GetPeerInfo]
request3.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.CONNECTED, None, Set(TestProbe().ref.toClassic))
request3.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.CONNECTED, Some(remoteFeatures), None, Set(TestProbe().ref.toClassic))
val channels = Seq(Peer.ChannelInfo(null, CLOSING, null))
peer.expectMessageType[Peer.GetPeerChannels].replyTo ! Peer.PeerChannels(remoteNodeId, channels)
probe.expectMessage(PeerReadyNotifier.PeerReady(remoteNodeId, peer.ref.toClassic, channels))
probe.expectMessage(PeerReadyNotifier.PeerReady(remoteNodeId, peer.ref.toClassic, remoteFeatures, channels))
}

test("peer connects then disconnects (while waiting for channel states)") { f =>
Expand All @@ -171,22 +177,23 @@ class PeerReadyNotifierSpec extends ScalaTestWithActorTestKit(ConfigFactory.load
notifier ! NotifyWhenPeerReady(probe.ref)
peerReadyManager.expectMessageType[PeerReadyManager.Register].replyTo ! PeerReadyManager.Registered(remoteNodeId, otherAttempts = 5)
val request1 = switchboard.expectMessageType[Switchboard.GetPeerInfo]
request1.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.DISCONNECTED, None, Set.empty)
request1.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.DISCONNECTED, None, None, Set.empty)
peer.expectNoMessage(100 millis)

// The target peer connects.
system.eventStream ! EventStream.Publish(PeerConnected(TestProbe().ref.toClassic, remoteNodeId, null))
val request2 = switchboard.expectMessageType[Switchboard.GetPeerInfo]
request2.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.CONNECTED, None, Set(TestProbe().ref.toClassic))
request2.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.CONNECTED, Some(remoteFeatures), None, Set(TestProbe().ref.toClassic))
peer.expectMessageType[Peer.GetPeerChannels]

// The target peer disconnects, so we wait for them to connect again.
system.eventStream ! EventStream.Publish(PeerDisconnected(TestProbe().ref.toClassic, remoteNodeId))
val request3 = switchboard.expectMessageType[Switchboard.GetPeerInfo]
request3.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.CONNECTED, None, Set(TestProbe().ref.toClassic))
val remoteFeatures1 = remoteFeatures.add(Features.OnionMessages, FeatureSupport.Optional)
request3.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.CONNECTED, Some(remoteFeatures1), None, Set(TestProbe().ref.toClassic))
val channels = Seq(Peer.ChannelInfo(null, NORMAL, null))
peer.expectMessageType[Peer.GetPeerChannels].replyTo ! Peer.PeerChannels(remoteNodeId, channels)
probe.expectMessage(PeerReadyNotifier.PeerReady(remoteNodeId, peer.ref.toClassic, channels))
probe.expectMessage(PeerReadyNotifier.PeerReady(remoteNodeId, peer.ref.toClassic, remoteFeatures1, channels))
}

test("peer connected (duration timeout)") { f =>
Expand All @@ -196,7 +203,7 @@ class PeerReadyNotifierSpec extends ScalaTestWithActorTestKit(ConfigFactory.load
notifier ! NotifyWhenPeerReady(probe.ref)
peerReadyManager.expectMessageType[PeerReadyManager.Register].replyTo ! PeerReadyManager.Registered(remoteNodeId, otherAttempts = 0)
val request = switchboard.expectMessageType[Switchboard.GetPeerInfo]
request.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.CONNECTED, None, Set(TestProbe().ref.toClassic))
request.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.CONNECTED, Some(Features.empty), None, Set(TestProbe().ref.toClassic))
peer.expectMessageType[Peer.GetPeerChannels]
probe.expectMessage(PeerUnavailable(remoteNodeId))
}
Expand All @@ -208,7 +215,7 @@ class PeerReadyNotifierSpec extends ScalaTestWithActorTestKit(ConfigFactory.load
notifier ! NotifyWhenPeerReady(probe.ref)
peerReadyManager.expectMessageType[PeerReadyManager.Register].replyTo ! PeerReadyManager.Registered(remoteNodeId, otherAttempts = 2)
val request = switchboard.expectMessageType[Switchboard.GetPeerInfo]
request.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.CONNECTED, None, Set(TestProbe().ref.toClassic))
request.replyTo ! Peer.PeerInfo(peer.ref.toClassic, remoteNodeId, Peer.CONNECTED, None, None, Set(TestProbe().ref.toClassic))
peer.expectMessageType[Peer.GetPeerChannels]
system.eventStream ! EventStream.Publish(CurrentBlockHeight(BlockHeight(100)))
probe.expectMessage(PeerUnavailable(remoteNodeId))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -110,7 +110,7 @@ class JsonSerializersSpec extends TestKitBaseClass with AnyFunSuiteLike with Mat
}

test("PeerInfo serialization") {
val peerInfo = PeerInfo(ActorRef.noSender, PublicKey(hex"0270685ca81a8e4d4d01beec5781f4cc924684072ae52c507f8ebe9daf0caaab7b"), Peer.CONNECTED, None, Set(ActorRef.noSender))
val peerInfo = PeerInfo(ActorRef.noSender, PublicKey(hex"0270685ca81a8e4d4d01beec5781f4cc924684072ae52c507f8ebe9daf0caaab7b"), Peer.CONNECTED, None, None, Set(ActorRef.noSender))
val expected = """{"nodeId":"0270685ca81a8e4d4d01beec5781f4cc924684072ae52c507f8ebe9daf0caaab7b","state":"CONNECTED","channels":1}"""
JsonSerializers.serialization.write(peerInfo)(JsonSerializers.formats) shouldBe expected
}
Expand Down
Loading

0 comments on commit ec90ec1

Please sign in to comment.