From 62f1540ab43bd41e0b99f862d441d0f0d4d98c11 Mon Sep 17 00:00:00 2001 From: staudtMarius Date: Wed, 8 Jan 2025 14:07:26 +0100 Subject: [PATCH] Converted `ExtSimAdapter` to typed. --- CHANGELOG.md | 1 + .../edu/ie3/simona/api/ExtSimAdapter.scala | 109 +++++++----- .../scala/edu/ie3/simona/sim/SimonaSim.scala | 6 +- .../simona/sim/setup/ExtSimSetupData.scala | 4 +- .../sim/setup/SimonaStandaloneSetup.scala | 19 +- .../ie3/simona/api/ExtSimAdapterSpec.scala | 162 +++++++++--------- 6 files changed, 171 insertions(+), 130 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 416c701666..92a1e698f4 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -114,6 +114,7 @@ and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0 - Prepare 'ChpModelSpec' and 'CylindricalThermalStorageSpec' for Storage without storageVolumeLvlMin [#1012](https://github.com/ie3-institute/simona/issues/1012) - Fixed SonarQube quality gate using the correct parameter '-Dsonar.qualitygate.wait=true' [#1072](https://github.com/ie3-institute/simona/issues/1072) - Updated `simonaAPI` to version `0.6.0` [#1080](https://github.com/ie3-institute/simona/issues/1080) +- Converted `ExtSimAdapter` to typed [#1094](https://github.com/ie3-institute/simona/issues/1094) ### Fixed - Fix rendering of references in documentation [#505](https://github.com/ie3-institute/simona/issues/505) diff --git a/src/main/scala/edu/ie3/simona/api/ExtSimAdapter.scala b/src/main/scala/edu/ie3/simona/api/ExtSimAdapter.scala index b71adf4a30..426305db48 100644 --- a/src/main/scala/edu/ie3/simona/api/ExtSimAdapter.scala +++ b/src/main/scala/edu/ie3/simona/api/ExtSimAdapter.scala @@ -6,36 +6,32 @@ package edu.ie3.simona.api -import org.apache.pekko.actor.typed.scaladsl.adapter.ClassicActorRefOps -import org.apache.pekko.actor.{Actor, ActorRef, PoisonPill, Props} -import edu.ie3.simona.api.ExtSimAdapter.{Create, ExtSimAdapterStateData, Stop} import edu.ie3.simona.api.data.ontology.ScheduleDataServiceMessage import edu.ie3.simona.api.simulation.ExtSimAdapterData import edu.ie3.simona.api.simulation.ontology.{ ActivationMessage, + ControlResponseMessageFromExt, TerminationCompleted, TerminationMessage, CompletionMessage => ExtCompletionMessage, } -import edu.ie3.simona.logging.SimonaActorLogging import edu.ie3.simona.ontology.messages.SchedulerMessage.{ Completion, ScheduleActivation, } import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.ScheduleServiceActivation -import edu.ie3.simona.ontology.messages.Activation +import edu.ie3.simona.ontology.messages.{Activation, SchedulerMessage} import edu.ie3.simona.scheduler.ScheduleLock import edu.ie3.simona.scheduler.ScheduleLock.ScheduleKey import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK +import org.apache.pekko.actor.typed.scaladsl.Behaviors +import org.apache.pekko.actor.typed.{ActorRef, Behavior} import scala.jdk.OptionConverters._ object ExtSimAdapter { - def props(scheduler: ActorRef): Props = - Props( - new ExtSimAdapter(scheduler) - ) + sealed trait AdapterMessage extends ControlResponseMessageFromExt /** The [[ExtSimAdapterData]] can only be constructed once the ExtSimAdapter * actor is created. Thus, we need an extra initialization message. @@ -44,80 +40,109 @@ object ExtSimAdapter { * The [[ExtSimAdapterData]] of the corresponding external simulation */ final case class Create(extSimData: ExtSimAdapterData, unlockKey: ScheduleKey) + extends AdapterMessage - final case class Stop(simulationSuccessful: Boolean) + final case class Stop(simulationSuccessful: Boolean) extends AdapterMessage final case class ExtSimAdapterStateData( extSimData: ExtSimAdapterData, currentTick: Option[Long] = None, - ) -} + ) extends AdapterMessage + + final case class WrappedActivation(activation: Activation) + extends AdapterMessage + final case class WrappedScheduleDataServiceMessage( + msg: ScheduleDataServiceMessage + ) extends AdapterMessage + + def adapter( + ref: ActorRef[ControlResponseMessageFromExt] + ): Behavior[ScheduleDataServiceMessage] = Behaviors.receiveMessagePartial { + extMsg => + ref ! WrappedScheduleDataServiceMessage(extMsg) + Behaviors.same + } -final case class ExtSimAdapter(scheduler: ActorRef) - extends Actor - with SimonaActorLogging { - override def receive: Receive = { case Create(extSimAdapterData, unlockKey) => - // triggering first time at init tick - scheduler ! ScheduleActivation( - self.toTyped, - INIT_SIM_TICK, - Some(unlockKey), - ) - context become receiveIdle( - ExtSimAdapterStateData(extSimAdapterData) - ) + def apply( + scheduler: ActorRef[SchedulerMessage] + ): Behavior[ControlResponseMessageFromExt] = Behaviors.setup { ctx => + val activationAdapter = ctx.messageAdapter(WrappedActivation) + initialize(scheduler, activationAdapter) } - private def receiveIdle(implicit - stateData: ExtSimAdapterStateData - ): Receive = { - case Activation(tick) => + private def initialize(implicit + scheduler: ActorRef[SchedulerMessage], + activationAdapter: ActorRef[Activation], + ): Behavior[ControlResponseMessageFromExt] = Behaviors.receiveMessage { + case Create(extSimData, unlockKey) => + // triggering first time at init tick + scheduler ! ScheduleActivation( + activationAdapter, + INIT_SIM_TICK, + Some(unlockKey), + ) + + receiveIdle(ExtSimAdapterStateData(extSimData)) + } + + private[api] def receiveIdle(stateData: ExtSimAdapterStateData)(implicit + scheduler: ActorRef[SchedulerMessage], + activationAdapter: ActorRef[Activation], + ): Behavior[ControlResponseMessageFromExt] = Behaviors.receive { + case (ctx, WrappedActivation(Activation(tick))) => stateData.extSimData.queueExtMsg( new ActivationMessage(tick) ) - log.debug( + ctx.log.debug( "Tick {} has been activated in external simulation", tick, ) - context become receiveIdle( - stateData.copy(currentTick = Some(tick)) - ) + receiveIdle(stateData.copy(currentTick = Some(tick))) - case extCompl: ExtCompletionMessage => + case (ctx, extCompl: ExtCompletionMessage) => // when multiple triggers have been sent, a completion message // always refers to the oldest tick val newTick = extCompl.nextActivation().toScala.map(Long2long) - scheduler ! Completion(self.toTyped, newTick) - log.debug( + scheduler ! Completion(activationAdapter, newTick) + ctx.log.debug( "Tick {} has been completed in external simulation", stateData.currentTick, ) - context become receiveIdle(stateData.copy(currentTick = None)) + receiveIdle(stateData.copy(currentTick = None)) - case scheduleDataService: ScheduleDataServiceMessage => + case ( + ctx, + WrappedScheduleDataServiceMessage( + scheduleDataService: ScheduleDataServiceMessage + ), + ) => val tick = stateData.currentTick.getOrElse( throw new RuntimeException("No tick has been triggered") ) - val key = ScheduleLock.singleKey(context, scheduler.toTyped, tick) + val key = ScheduleLock.singleKey(ctx, scheduler, tick) scheduleDataService.getDataService ! ScheduleServiceActivation( tick, key, ) - case Stop(simulationSuccessful) => + Behaviors.same + + case (_, Stop(simulationSuccessful)) => // let external sim know that we have terminated stateData.extSimData.queueExtMsg( new TerminationMessage(simulationSuccessful) ) - case _: TerminationCompleted => + Behaviors.same + + case (_, _: TerminationCompleted) => // external simulation has terminated as well, we can exit - self ! PoisonPill + Behaviors.stopped } } diff --git a/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala b/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala index c8196aaf30..a5e9f56f71 100644 --- a/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala +++ b/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala @@ -8,6 +8,7 @@ package edu.ie3.simona.sim import edu.ie3.simona.agent.EnvironmentRefs import edu.ie3.simona.api.ExtSimAdapter +import edu.ie3.simona.api.simulation.ontology.ControlResponseMessageFromExt import edu.ie3.simona.event.RuntimeEvent import edu.ie3.simona.event.listener.{DelayedStopHelper, RuntimeEventListener} import edu.ie3.simona.main.RunSimona.SimonaEnded @@ -17,7 +18,6 @@ import edu.ie3.util.scala.Scope import org.apache.pekko.actor.typed.scaladsl.adapter._ import org.apache.pekko.actor.typed.scaladsl.{ActorContext, Behaviors} import org.apache.pekko.actor.typed.{ActorRef, Behavior, PostStop, Terminated} -import org.apache.pekko.actor.{ActorRef => ClassicRef} /** Main entrance point to a simona simulation as the guardian actor. This actor * starts the initialization of all actors and waits for the simulation to end. @@ -118,7 +118,7 @@ object SimonaSim { /* watch all actors */ resultEventListeners.foreach(ctx.watch) ctx.watch(runtimeEventListener) - extSimulationData.extSimAdapters.map(_.toTyped).foreach(ctx.watch) + extSimulationData.extSimAdapters.foreach(ctx.watch) otherActors.foreach(ctx.watch) // Start simulation @@ -267,7 +267,7 @@ object SimonaSim { */ private final case class ActorData( starter: ActorRef[SimonaEnded], - extSimAdapters: Iterable[ClassicRef], + extSimAdapters: Iterable[ActorRef[ControlResponseMessageFromExt]], runtimeEventListener: ActorRef[RuntimeEventListener.Request], delayedStoppingActors: Seq[ActorRef[DelayedStopHelper.StoppingMsg]], otherActors: Iterable[ActorRef[_]], diff --git a/src/main/scala/edu/ie3/simona/sim/setup/ExtSimSetupData.scala b/src/main/scala/edu/ie3/simona/sim/setup/ExtSimSetupData.scala index 3209fb706d..0a9f4e5f7e 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/ExtSimSetupData.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/ExtSimSetupData.scala @@ -6,11 +6,13 @@ package edu.ie3.simona.sim.setup +import edu.ie3.simona.api.simulation.ontology.ControlResponseMessageFromExt import org.apache.pekko.actor.{ActorRef => ClassicRef} import edu.ie3.simona.service.ev.ExtEvDataService +import org.apache.pekko.actor.typed.ActorRef final case class ExtSimSetupData( - extSimAdapters: Iterable[ClassicRef], + extSimAdapters: Iterable[ActorRef[ControlResponseMessageFromExt]], extDataServices: Map[Class[_], ClassicRef], ) { diff --git a/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala b/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala index 3e62408033..148728bf3e 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala @@ -16,7 +16,6 @@ import edu.ie3.simona.agent.EnvironmentRefs import edu.ie3.simona.agent.grid.GridAgent import edu.ie3.simona.agent.grid.GridAgentMessages.CreateGridAgent import edu.ie3.simona.api.ExtSimAdapter -import edu.ie3.simona.api.data.ExtDataConnection import edu.ie3.simona.api.data.ev.ExtEvDataConnection import edu.ie3.simona.api.simulation.ExtSimAdapterData import edu.ie3.simona.config.{ArgsParser, RefSystemParser, SimonaConfig} @@ -206,11 +205,18 @@ class SimonaStandaloneSetup( val (extSimAdapters, extDataServices) = extLinks.zipWithIndex.map { case (extLink, index) => // external simulation always needs at least an ExtSimAdapter - val extSimAdapter = context.toClassic.simonaActorOf( - ExtSimAdapter.props(scheduler.toClassic), + val extSimAdapter = context.spawn( + ExtSimAdapter(scheduler), s"$index", ) - val extSimAdapterData = new ExtSimAdapterData(extSimAdapter, args) + + val extSimAdapterMsgAdapter = context.spawn( + ExtSimAdapter.adapter(extSimAdapter), + s"$index-message-adapter", + ) + + val extSimAdapterData = + new ExtSimAdapterData(extSimAdapterMsgAdapter.toClassic, args) // send init data right away, init activation is scheduled extSimAdapter ! ExtSimAdapter.Create( @@ -230,7 +236,10 @@ class SimonaStandaloneSetup( ExtEvDataService.props(scheduler.toClassic), s"$index-$dIndex", ) - evConnection.setActorRefs(extEvDataService, extSimAdapter) + evConnection.setActorRefs( + extEvDataService, + extSimAdapterMsgAdapter.toClassic, + ) extEvDataService ! SimonaService.Create( InitExtEvData(evConnection), diff --git a/src/test/scala/edu/ie3/simona/api/ExtSimAdapterSpec.scala b/src/test/scala/edu/ie3/simona/api/ExtSimAdapterSpec.scala index 0a4dab2d51..ba337863fe 100644 --- a/src/test/scala/edu/ie3/simona/api/ExtSimAdapterSpec.scala +++ b/src/test/scala/edu/ie3/simona/api/ExtSimAdapterSpec.scala @@ -6,8 +6,12 @@ package edu.ie3.simona.api -import com.typesafe.config.ConfigFactory -import edu.ie3.simona.api.ExtSimAdapter.Stop +import edu.ie3.simona.api.ExtSimAdapter.{ + ExtSimAdapterStateData, + Stop, + WrappedActivation, + WrappedScheduleDataServiceMessage, +} import edu.ie3.simona.api.data.ontology.ScheduleDataServiceMessage import edu.ie3.simona.api.simulation.ExtSimAdapterData import edu.ie3.simona.api.simulation.ontology.{ @@ -16,81 +20,82 @@ import edu.ie3.simona.api.simulation.ontology.{ TerminationMessage, CompletionMessage => ExtCompletionMessage, } -import edu.ie3.simona.ontology.messages.Activation import edu.ie3.simona.ontology.messages.SchedulerMessage.{ Completion, ScheduleActivation, } import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.ScheduleServiceActivation -import edu.ie3.simona.scheduler.ScheduleLock.ScheduleKey -import edu.ie3.simona.test.common.{TestKitWithShutdown, TestSpawnerClassic} +import edu.ie3.simona.ontology.messages.{Activation, SchedulerMessage} +import edu.ie3.simona.scheduler.ScheduleLock.{LockMsg, ScheduleKey} +import edu.ie3.simona.test.common.TestSpawnerTyped import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK -import org.apache.pekko.actor.typed.scaladsl.adapter.ClassicActorRefOps -import org.apache.pekko.actor.{ActorSystem, Terminated} -import org.apache.pekko.testkit.{TestActorRef, TestProbe} +import org.apache.pekko.actor.testkit.typed.Effect.Stopped +import org.apache.pekko.actor.testkit.typed.scaladsl.{ + BehaviorTestKit, + ScalaTestWithActorTestKit, + TestProbe, +} +import org.apache.pekko.actor.typed.scaladsl.adapter.TypedActorRefOps +import org.apache.pekko.testkit.TestKit.awaitCond +import org.scalatest.prop.TableDrivenPropertyChecks.forAll +import org.scalatest.prop.Tables.Table import org.scalatest.wordspec.AnyWordSpecLike import java.util.UUID import scala.concurrent.duration.DurationInt import scala.jdk.OptionConverters.RichOption +import scala.language.{existentials, implicitConversions} class ExtSimAdapterSpec - extends TestKitWithShutdown( - ActorSystem( - "ExtSimAdapterSpec", - ConfigFactory - .parseString(""" - |pekko.loggers = ["org.apache.pekko.testkit.TestEventListener"] - |pekko.loglevel = "INFO" - |""".stripMargin), - ) - ) + extends ScalaTestWithActorTestKit with AnyWordSpecLike - with TestSpawnerClassic { + with TestSpawnerTyped { - private val scheduler = TestProbe("scheduler") + private val scheduler = TestProbe[SchedulerMessage]("scheduler") private val mainArgs = Array.empty[String] + implicit def wrap(activation: Activation): WrappedActivation = + WrappedActivation(activation) + "An uninitialized ExtSimScheduler" must { "send correct completion message after initialisation" in { - val lock = TestProbe("lock") + val lock = TestProbe[LockMsg]("lock") - val extSimAdapter = TestActorRef( - new ExtSimAdapter(scheduler.ref) - ) + val extSimAdapter = testKit.spawn(ExtSimAdapter(scheduler.ref)) - val extData = new ExtSimAdapterData(extSimAdapter, mainArgs) + val extData = new ExtSimAdapterData(extSimAdapter.toClassic, mainArgs) - val key1 = ScheduleKey(lock.ref.toTyped, UUID.randomUUID()) - scheduler.send(extSimAdapter, ExtSimAdapter.Create(extData, key1)) - scheduler.expectMsg( - ScheduleActivation(extSimAdapter.toTyped, INIT_SIM_TICK, Some(key1)) - ) + val key1 = ScheduleKey(lock.ref, UUID.randomUUID()) + extSimAdapter ! ExtSimAdapter.Create(extData, key1) + + val activationMessage = scheduler.expectMessageType[ScheduleActivation] + activationMessage.tick shouldBe INIT_SIM_TICK + activationMessage.unlockKey shouldBe Some(key1) } } "An initialized ExtSimScheduler" must { "forward an activation trigger and a corresponding completion message properly" in { - val lock = TestProbe("lock") + val lock = TestProbe[LockMsg]("lock") - val extSimAdapter = TestActorRef( - new ExtSimAdapter(scheduler.ref) - ) + val extSimAdapter = testKit.spawn(ExtSimAdapter(scheduler.ref)) - val extData = new ExtSimAdapterData(extSimAdapter, mainArgs) + val extData = new ExtSimAdapterData(extSimAdapter.toClassic, mainArgs) - val key1 = ScheduleKey(lock.ref.toTyped, UUID.randomUUID()) - scheduler.send(extSimAdapter, ExtSimAdapter.Create(extData, key1)) - scheduler.expectMsg( - ScheduleActivation(extSimAdapter.toTyped, INIT_SIM_TICK, Some(key1)) - ) + val key1 = ScheduleKey(lock.ref, UUID.randomUUID()) - scheduler.send(extSimAdapter, Activation(INIT_SIM_TICK)) + extSimAdapter ! ExtSimAdapter.Create(extData, key1) + + val activationMessage = scheduler.expectMessageType[ScheduleActivation] + activationMessage.tick shouldBe INIT_SIM_TICK + activationMessage.unlockKey shouldBe Some(key1) + val activationAdapter = activationMessage.actor + + extSimAdapter ! Activation(INIT_SIM_TICK) awaitCond( !extData.receiveMessageQueue.isEmpty, max = 3.seconds, - message = "No message received", ) extData.receiveMessageQueue.size() shouldBe 1 extData.receiveMessageQueue.take() shouldBe new ActivationMessage( @@ -106,71 +111,72 @@ class ExtSimAdapterSpec ) ) - scheduler.expectMsg(Completion(extSimAdapter.toTyped, Some(nextTick))) + scheduler.expectMessage(Completion(activationAdapter, Some(nextTick))) } "schedule the data service when it is told to" in { - val lock = TestProbe("lock") + val lock = TestProbe[LockMsg]("lock") - val extSimAdapter = TestActorRef( - new ExtSimAdapter(scheduler.ref) + val extSimAdapter = testKit.spawn( + ExtSimAdapter(scheduler.ref) ) + val adapter = testKit.spawn(ExtSimAdapter.adapter(extSimAdapter)) + val extData = new ExtSimAdapterData(adapter.toClassic, mainArgs) + val dataService = TestProbe[ScheduleServiceActivation]("dataService") - val extData = new ExtSimAdapterData(extSimAdapter, mainArgs) - val dataService = TestProbe("dataService") + val key1 = ScheduleKey(lock.ref, UUID.randomUUID()) + extSimAdapter ! ExtSimAdapter.Create(extData, key1) - val key1 = ScheduleKey(lock.ref.toTyped, UUID.randomUUID()) - scheduler.send(extSimAdapter, ExtSimAdapter.Create(extData, key1)) - scheduler.expectMsg( - ScheduleActivation(extSimAdapter.toTyped, INIT_SIM_TICK, Some(key1)) - ) + val activationMessage = scheduler.expectMessageType[ScheduleActivation] + activationMessage.tick shouldBe INIT_SIM_TICK + activationMessage.unlockKey shouldBe Some(key1) - scheduler.send(extSimAdapter, Activation(INIT_SIM_TICK)) + extSimAdapter ! Activation(INIT_SIM_TICK) awaitCond( !extData.receiveMessageQueue.isEmpty, max = 3.seconds, - message = "No message received", ) extData.receiveMessageQueue.size() shouldBe 1 extData.receiveMessageQueue.take() - extSimAdapter ! new ScheduleDataServiceMessage( - dataService.ref + extSimAdapter ! WrappedScheduleDataServiceMessage( + new ScheduleDataServiceMessage( + dataService.ref.toClassic + ) ) - scheduler.expectMsgType[ScheduleActivation] // lock activation scheduled + scheduler + .expectMessageType[ScheduleActivation] // lock activation scheduled dataService - .expectMsgType[ScheduleServiceActivation] + .expectMessageType[ScheduleServiceActivation] .tick shouldBe INIT_SIM_TICK scheduler.expectNoMessage() } "terminate the external simulation and itself when told to" in { forAll(Table("simSuccessful", true, false)) { (simSuccessful: Boolean) => - val lock = TestProbe("lock") - - val extSimAdapter = TestActorRef( - new ExtSimAdapter(scheduler.ref) + val activationAdapter = TestProbe[Activation] + + val probe = TestProbe[ScheduleDataServiceMessage] + val extData = new ExtSimAdapterData(probe.ref.toClassic, mainArgs) + + val extSimAdapter = BehaviorTestKit( + ExtSimAdapter.receiveIdle( + ExtSimAdapterStateData( + extData, + None, + ) + )(scheduler.ref, activationAdapter.ref) ) - val extData = new ExtSimAdapterData(extSimAdapter, mainArgs) + extSimAdapter.isAlive shouldBe true - val key1 = ScheduleKey(lock.ref.toTyped, UUID.randomUUID()) - scheduler.send(extSimAdapter, ExtSimAdapter.Create(extData, key1)) - scheduler.expectMsg( - ScheduleActivation(extSimAdapter.toTyped, INIT_SIM_TICK, Some(key1)) - ) - - val stopWatcher = TestProbe() - stopWatcher.watch(extSimAdapter) - - extSimAdapter ! Stop(simSuccessful) + extSimAdapter.run(Stop(simSuccessful)) awaitCond( !extData.receiveMessageQueue.isEmpty, max = 3.seconds, - message = "No message received", ) extData.receiveMessageQueue.size() shouldBe 1 extData.receiveMessageQueue.take() shouldBe new TerminationMessage( @@ -178,12 +184,10 @@ class ExtSimAdapterSpec ) // up until now, extSimAdapter should still be running - stopWatcher.expectNoMessage() - - extSimAdapter ! new TerminationCompleted() + extSimAdapter.run(new TerminationCompleted()) // extSimAdapter should have terminated now - stopWatcher.expectMsgType[Terminated].actor shouldBe extSimAdapter + extSimAdapter.isAlive shouldBe false // scheduler is not involved in this scheduler.expectNoMessage()