From dcbd78de582b32988da820025baa1ebd1f5149a7 Mon Sep 17 00:00:00 2001 From: staudtMarius Date: Fri, 12 Sep 2025 16:43:08 +0200 Subject: [PATCH 1/6] Added external result provider --- CHANGELOG.md | 1 + build.gradle | 2 +- .../ie3/simona/agent/EnvironmentRefs.scala | 4 + .../edu/ie3/simona/agent/em/EmAgent.scala | 30 +- .../edu/ie3/simona/agent/grid/GridAgent.scala | 10 +- .../simona/agent/grid/GridAgentBuilder.scala | 22 +- .../ie3/simona/agent/grid/GridAgentData.scala | 18 +- .../participant/ParticipantAgentInit.scala | 9 +- .../ParticipantResultHandler.scala | 40 +-- .../edu/ie3/simona/event/ResultEvent.scala | 8 +- .../event/listener/DelayedStopHelper.scala | 2 +- ...entListener.scala => ResultListener.scala} | 185 ++++------- .../ontology/messages/ResultMessage.scala | 41 +++ .../service/results/ExtResultProvider.scala | 149 +++++++++ .../service/results/ResultServiceProxy.scala | 287 ++++++++++++++++++ .../results}/Transformer3wResultSupport.scala | 6 +- .../scala/edu/ie3/simona/sim/SimonaSim.scala | 47 +-- .../ie3/simona/sim/setup/ExtSimSetup.scala | 52 +++- .../simona/sim/setup/ExtSimSetupData.scala | 101 +++--- .../ie3/simona/sim/setup/SimonaSetup.scala | 116 ++++--- .../sim/setup/SimonaStandaloneSetup.scala | 41 ++- .../edu/ie3/simona/util/CollectionUtils.scala | 10 + .../edu/ie3/simona/agent/em/EmAgentIT.scala | 79 ++--- .../edu/ie3/simona/agent/em/EmAgentSpec.scala | 51 ++-- .../agent/grid/DBFSAlgorithmCenGridSpec.scala | 28 +- .../DBFSAlgorithmFailedPowerFlowSpec.scala | 58 +++- .../grid/DBFSAlgorithmParticipantSpec.scala | 7 +- .../agent/grid/DBFSAlgorithmSupGridSpec.scala | 14 +- .../agent/grid/GridAgentSetupSpec.scala | 2 - .../ie3/simona/agent/grid/ThermalGridIT.scala | 146 ++++----- .../congestion/CongestionTestBaseData.scala | 10 +- .../grid/congestion/DCMAlgorithmSpec.scala | 2 +- .../ParticipantAgentInitSpec.scala | 33 +- .../participant/ParticipantAgentSpec.scala | 193 ++++++------ ...nerSpec.scala => ResultListenerSpec.scala} | 131 ++------ .../io/result/ResultEntityKafkaSpec.scala | 20 +- .../model/participant/evcs/EvcsModelIT.scala | 31 +- .../ThreeWindingResultHandlingSpec.scala | 7 +- .../results}/ThreeWindingResultTestData.scala | 2 +- .../edu/ie3/simona/sim/SimonaSimSpec.scala | 48 +-- .../sim/setup/ExtSimSetupDataSpec.scala | 111 +++---- .../simona/sim/setup/ExtSimSetupSpec.scala | 2 +- .../simona/sim/setup/SimonaSetupSpec.scala | 20 +- .../test/common/model/grid/DbfsTestGrid.scala | 9 + .../common/result/PowerFlowResultData.scala | 9 +- 45 files changed, 1319 insertions(+), 875 deletions(-) rename src/main/scala/edu/ie3/simona/event/listener/{ResultEventListener.scala => ResultListener.scala} (58%) create mode 100644 src/main/scala/edu/ie3/simona/ontology/messages/ResultMessage.scala create mode 100644 src/main/scala/edu/ie3/simona/service/results/ExtResultProvider.scala create mode 100644 src/main/scala/edu/ie3/simona/service/results/ResultServiceProxy.scala rename src/main/scala/edu/ie3/simona/{event/listener => service/results}/Transformer3wResultSupport.scala (97%) rename src/test/scala/edu/ie3/simona/event/listener/{ResultEventListenerSpec.scala => ResultListenerSpec.scala} (75%) rename src/test/scala/edu/ie3/simona/{event/listener => service/results}/ThreeWindingResultHandlingSpec.scala (95%) rename src/test/scala/edu/ie3/simona/{event/listener => service/results}/ThreeWindingResultTestData.scala (97%) diff --git a/CHANGELOG.md b/CHANGELOG.md index 2eb067eeeb..f8bd1952a1 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -17,6 +17,7 @@ and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0 - Considering primary data that start before simulation [#1034](https://github.com/ie3-institute/simona/issues/1034) - Implement time series retrieval for WeatherSource [#1511](https://github.com/ie3-institute/simona/issues/1511) - Implement weather forecast provision by WeatherService [#1512](https://github.com/ie3-institute/simona/issues/1512) +- Added external result provider [#1530](https://github.com/ie3-institute/simona/issues/1530) ### Changed - Upgraded `scala2` to `scala3` [#53](https://github.com/ie3-institute/simona/issues/53) diff --git a/build.gradle b/build.gradle index 5b53c94ca3..5d0534267b 100644 --- a/build.gradle +++ b/build.gradle @@ -94,7 +94,7 @@ dependencies { exclude group: 'edu.ie3' } - implementation('com.github.ie3-institute:simonaAPI:0.10-SNAPSHOT') { + implementation('com.github.ie3-institute:simonaAPI:0.10.0') { exclude group: 'org.apache.logging.log4j' exclude group: 'org.slf4j' /* Exclude our own nested dependencies */ diff --git a/src/main/scala/edu/ie3/simona/agent/EnvironmentRefs.scala b/src/main/scala/edu/ie3/simona/agent/EnvironmentRefs.scala index 5b372bd87e..eeeb8a3dc5 100644 --- a/src/main/scala/edu/ie3/simona/agent/EnvironmentRefs.scala +++ b/src/main/scala/edu/ie3/simona/agent/EnvironmentRefs.scala @@ -8,6 +8,7 @@ package edu.ie3.simona.agent import edu.ie3.simona.event.RuntimeEvent import edu.ie3.simona.ontology.messages.{SchedulerMessage, ServiceMessage} +import edu.ie3.simona.service.results.ResultServiceProxy import org.apache.pekko.actor.typed.ActorRef /** Container class, that gather together reference to relevant entities, that @@ -19,6 +20,8 @@ import org.apache.pekko.actor.typed.ActorRef * Reference to the runtime event listener. * @param primaryServiceProxy * Reference to the primary service proxy. + * @param resultProxy + * Reference to the result service proxy. * @param weather * Reference to the service, that provides weather information. * @param loadProfiles @@ -30,6 +33,7 @@ final case class EnvironmentRefs( scheduler: ActorRef[SchedulerMessage], runtimeEventListener: ActorRef[RuntimeEvent], primaryServiceProxy: ActorRef[ServiceMessage], + resultProxy: ActorRef[ResultServiceProxy.Message], weather: ActorRef[ServiceMessage], loadProfiles: ActorRef[ServiceMessage], evDataService: Option[ActorRef[ServiceMessage]], diff --git a/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala b/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala index 2de62f427e..82d9e36ccc 100644 --- a/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala +++ b/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala @@ -60,7 +60,7 @@ object EmAgent { * agent is em-controlled, or a [[Left]] with a reference to the scheduler * that is activating this agent. * @param listener - * A collection of result event listeners. + * A listener for result events. */ def apply( inputModel: EmInput, @@ -69,7 +69,7 @@ object EmAgent { modelStrategy: String, simulationStartDate: ZonedDateTime, parent: Either[ActorRef[SchedulerMessage], ActorRef[FlexResponse]], - listener: Iterable[ActorRef[ResultEvent]], + listener: ActorRef[ResultEvent], ): Behavior[Message] = Behaviors.setup[Message] { ctx => parent.map { @@ -213,9 +213,7 @@ object EmAgent { ) ) - emData.listener.foreach { - _ ! FlexOptionsResultEvent(flexResult) - } + emData.listener ! FlexOptionsResultEvent(flexResult) } emData.parent match { @@ -365,17 +363,15 @@ object EmAgent { } maybeResult.foreach { result => - emData.listener.foreach { - _ ! ParticipantResultEvent( - new EmResult( - lastActiveTick - .toDateTime(using emData.simulationStartDate), - modelShell.uuid, - result.p.toMegawatts.asMegaWatt, - result.q.toMegavars.asMegaVar, - ) + emData.listener ! ParticipantResultEvent( + new EmResult( + lastActiveTick + .toDateTime(using emData.simulationStartDate), + modelShell.uuid, + result.p.toMegawatts.asMegaWatt, + result.q.toMegavars.asMegaVar, ) - } + ) emData.parent.foreach { _ ! FlexResult(modelShell.uuid, result) @@ -406,13 +402,13 @@ object EmAgent { * agent is em-controlled, or a [[Left]] with a reference to the scheduler * that is activating this agent. * @param listener - * A collection of result event listeners. + * A listener for result events. */ private final case class EmData( outputConfig: NotifierConfig, simulationStartDate: ZonedDateTime, parent: Either[ActorRef[SchedulerMessage], ActorRef[FlexResponse]], - listener: Iterable[ActorRef[ResultEvent]], + listener: ActorRef[ResultEvent], ) /** The existence of this data object indicates that the corresponding agent diff --git a/src/main/scala/edu/ie3/simona/agent/grid/GridAgent.scala b/src/main/scala/edu/ie3/simona/agent/grid/GridAgent.scala index bd6e9b462e..e5ea501e32 100644 --- a/src/main/scala/edu/ie3/simona/agent/grid/GridAgent.scala +++ b/src/main/scala/edu/ie3/simona/agent/grid/GridAgent.scala @@ -32,6 +32,7 @@ import edu.ie3.simona.ontology.messages.SchedulerMessage.{ Completion, ScheduleActivation, } +import edu.ie3.simona.service.results.ResultServiceProxy.ExpectResult import edu.ie3.simona.util.TickUtil.TickLong import org.apache.pekko.actor.typed.scaladsl.AskPattern.Askable import org.apache.pekko.actor.typed.scaladsl.{ @@ -68,7 +69,6 @@ object GridAgent extends DBFSAlgorithm with DCMAlgorithm { def apply( environmentRefs: EnvironmentRefs, simonaConfig: SimonaConfig, - listener: Iterable[ActorRef[ResultEvent]], ): Behavior[Message] = Behaviors.withStash(100) { buffer => val cfg = simonaConfig.simona @@ -83,7 +83,6 @@ object GridAgent extends DBFSAlgorithm with DCMAlgorithm { val agentValues = GridAgentConstantData( environmentRefs, simonaConfig, - listener, resolution, simStartTime, simEndTime, @@ -229,6 +228,13 @@ object GridAgent extends DBFSAlgorithm with DCMAlgorithm { ctx.self, Some(activation.tick), ) + + // inform the result proxy that this grid agent will send new results + constantData.environmentRefs.resultProxy ! ExpectResult( + gridAgentBaseData.assets, + activation.tick, + ) + buffer.unstashAll(simulateGrid(gridAgentBaseData, activation.tick)) case (_, msg: Message) => diff --git a/src/main/scala/edu/ie3/simona/agent/grid/GridAgentBuilder.scala b/src/main/scala/edu/ie3/simona/agent/grid/GridAgentBuilder.scala index 129d355fb7..2a15855501 100644 --- a/src/main/scala/edu/ie3/simona/agent/grid/GridAgentBuilder.scala +++ b/src/main/scala/edu/ie3/simona/agent/grid/GridAgentBuilder.scala @@ -291,9 +291,9 @@ object GridAgentBuilder { given ParticipantRefs = ParticipantRefs( gridAgentContext.self, - constantData.environmentRefs.primaryServiceProxy, + environmentRefs.primaryServiceProxy, + environmentRefs.resultProxy, serviceMap, - constantData.listener, ) given SimulationParameters = SimulationParameters( @@ -316,7 +316,7 @@ object GridAgentBuilder { constantData.outputConfigUtil.getOrDefault( NotifierIdentifier.FixedFeedIn ), - constantData.environmentRefs.scheduler, + environmentRefs.scheduler, maybeControllingEm, ) case input: LoadInput => @@ -326,7 +326,7 @@ object GridAgentBuilder { input.getUuid ), constantData.outputConfigUtil.getOrDefault(NotifierIdentifier.Load), - constantData.environmentRefs.scheduler, + environmentRefs.scheduler, maybeControllingEm, ) case input: PvInput => @@ -338,7 +338,7 @@ object GridAgentBuilder { constantData.outputConfigUtil.getOrDefault( NotifierIdentifier.PvPlant ), - constantData.environmentRefs.scheduler, + environmentRefs.scheduler, maybeControllingEm, ) case input: BmInput => @@ -350,7 +350,7 @@ object GridAgentBuilder { constantData.outputConfigUtil.getOrDefault( NotifierIdentifier.BioMassPlant ), - constantData.environmentRefs.scheduler, + environmentRefs.scheduler, maybeControllingEm, ) case input: WecInput => @@ -360,7 +360,7 @@ object GridAgentBuilder { input.getUuid ), constantData.outputConfigUtil.getOrDefault(NotifierIdentifier.Wec), - constantData.environmentRefs.scheduler, + environmentRefs.scheduler, maybeControllingEm, ) case input: EvcsInput => @@ -370,7 +370,7 @@ object GridAgentBuilder { input.getUuid ), constantData.outputConfigUtil.getOrDefault(NotifierIdentifier.Evcs), - constantData.environmentRefs.scheduler, + environmentRefs.scheduler, maybeControllingEm, ) case input: HpInput => @@ -382,7 +382,7 @@ object GridAgentBuilder { input.getUuid ), constantData.outputConfigUtil.getOrDefault(NotifierIdentifier.Hp), - constantData.environmentRefs.scheduler, + environmentRefs.scheduler, maybeControllingEm, ) case None => @@ -399,7 +399,7 @@ object GridAgentBuilder { constantData.outputConfigUtil.getOrDefault( NotifierIdentifier.Storage ), - constantData.environmentRefs.scheduler, + environmentRefs.scheduler, maybeControllingEm, ) case input: SystemParticipantInput => @@ -468,7 +468,7 @@ object GridAgentBuilder { maybeControllingEm.toRight( constantData.environmentRefs.scheduler ), - constantData.listener, + constantData.environmentRefs.resultProxy, ), actorName(classOf[EmAgent.type], emInput.getId), ) diff --git a/src/main/scala/edu/ie3/simona/agent/grid/GridAgentData.scala b/src/main/scala/edu/ie3/simona/agent/grid/GridAgentData.scala index b8ed1645ee..abe97f2d5d 100644 --- a/src/main/scala/edu/ie3/simona/agent/grid/GridAgentData.scala +++ b/src/main/scala/edu/ie3/simona/agent/grid/GridAgentData.scala @@ -46,9 +46,6 @@ object GridAgentData { * the grid agent. * @param simonaConfig * Configuration of SIMONA, that is used for. - * @param listener - * A sequence of listeners, that will receive the results from the grid - * agent. * @param resolution * That is used for the power flow. If no power flow should be carried out, * this value is set to [[Long.MaxValue]]. @@ -60,14 +57,12 @@ object GridAgentData { final case class GridAgentConstantData( environmentRefs: EnvironmentRefs, simonaConfig: SimonaConfig, - listener: Iterable[ActorRef[ResultEvent]], resolution: Long, simStartTime: ZonedDateTime, simEndTime: ZonedDateTime, ) { - def notifyListeners(event: ResultEvent): Unit = { - listener.foreach(_ ! event) - } + def notifyListeners(event: ResultEvent): Unit = + environmentRefs.resultProxy ! event val participantConfigUtil: ParticipantConfigUtil = ConfigUtil.ParticipantConfigUtil(simonaConfig.simona.runtime.participant) @@ -323,6 +318,15 @@ object GridAgentData { ) extends GridAgentData with GridAgentDataHelper { + val assets: Seq[UUID] = { + val components = gridEnv.gridModel.gridComponents + components.nodes.map(_.uuid) ++ components.lines.map( + _.uuid + ) ++ components.switches.map(_.uuid) ++ components.transformers.map( + _.uuid + ) ++ components.transformers3w.map(_.uuid) + } + override protected val subgridGates: Vector[SubGridGate] = gridEnv.subgridGateToActorRef.keys.toVector override protected val subgridId: Int = gridEnv.gridModel.subnetNo diff --git a/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgentInit.scala b/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgentInit.scala index 938179e4ac..d874a19d84 100644 --- a/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgentInit.scala +++ b/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgentInit.scala @@ -35,6 +35,7 @@ import edu.ie3.simona.ontology.messages.flex.FlexibilityMessage.* import edu.ie3.simona.ontology.messages.{SchedulerMessage, ServiceMessage} import edu.ie3.simona.scheduler.ScheduleLock.ScheduleKey import edu.ie3.simona.service.ServiceType +import edu.ie3.simona.service.results.ResultServiceProxy.ExpectResult import edu.ie3.simona.service.weather.WeatherDataType import edu.ie3.simona.service.weather.WeatherService.WeatherRegistrationData import edu.ie3.simona.util.Coordinate @@ -59,16 +60,16 @@ object ParticipantAgentInit { * Reference to the grid agent. * @param primaryServiceProxy * Reference to the primary service proxy. + * @param resultServiceProxy + * Reference to the result service proxy. * @param services * References to services by service type. - * @param resultListener - * Reference to the result listeners. */ final case class ParticipantRefs( gridAgent: ActorRef[GridAgent.Message], primaryServiceProxy: ActorRef[ServiceMessage], + resultServiceProxy: ActorRef[ResultEvent | ExpectResult], services: Map[ServiceType, ActorRef[ServiceMessage]], - resultListener: Iterable[ActorRef[ResultEvent]], ) /** Container class that holds parameters related to the simulation. @@ -421,7 +422,7 @@ object ParticipantAgentInit { simulationParams.requestVoltageDeviationTolerance, ), ParticipantResultHandler( - participantRefs.resultListener, + participantRefs.resultServiceProxy, notifierConfig, ), ) diff --git a/src/main/scala/edu/ie3/simona/agent/participant/ParticipantResultHandler.scala b/src/main/scala/edu/ie3/simona/agent/participant/ParticipantResultHandler.scala index ce95d1d299..f3803131e9 100644 --- a/src/main/scala/edu/ie3/simona/agent/participant/ParticipantResultHandler.scala +++ b/src/main/scala/edu/ie3/simona/agent/participant/ParticipantResultHandler.scala @@ -20,18 +20,21 @@ import edu.ie3.simona.event.ResultEvent.{ } import edu.ie3.simona.event.notifier.NotifierConfig import edu.ie3.simona.exceptions.CriticalFailureException +import edu.ie3.simona.service.results.ResultServiceProxy.ExpectResult import org.apache.pekko.actor.typed.ActorRef +import java.util.UUID + /** Handles all kind of results stemming from the participant by sending them to - * the result listener, if applicable. + * the result proxy, if applicable. * - * @param listener - * The actor reference to the result listener. + * @param resultProxy + * The actor reference to the result resultProxy. * @param config * The result configuration. */ final case class ParticipantResultHandler( - private val listener: Iterable[ActorRef[ResultEvent]], + private val resultProxy: ActorRef[ResultEvent | ExpectResult], private val config: NotifierConfig, ) { @@ -42,18 +45,16 @@ final case class ParticipantResultHandler( */ def maybeSend(result: ResultEntity): Unit = if config.simulationResultInfo then { - listener.foreach(actor => - result match { - case thermalResult: ThermalUnitResult => - actor ! ThermalResultEvent(thermalResult) - case participantResult: SystemParticipantResult => - actor ! ParticipantResultEvent(participantResult) - case unsupported => - throw new CriticalFailureException( - s"Results of class '${unsupported.getClass.getSimpleName}' are currently not supported." - ) - } - ) + result match { + case thermalResult: ThermalUnitResult => + resultProxy ! ThermalResultEvent(thermalResult) + case participantResult: SystemParticipantResult => + resultProxy ! ParticipantResultEvent(participantResult) + case unsupported => + throw new CriticalFailureException( + s"Results of class '${unsupported.getClass.getSimpleName}' are currently not supported." + ) + } } /** Send the flex options result to all listeners, if enabled. @@ -63,9 +64,10 @@ final case class ParticipantResultHandler( */ def maybeSend(result: FlexOptionsResult): Unit = if config.flexResult then { - listener.foreach( - _ ! FlexOptionsResultEvent(result) - ) + resultProxy ! FlexOptionsResultEvent(result) } + def informProxy(uuid: UUID, tick: Long): Unit = + resultProxy ! ExpectResult(uuid, tick) + } diff --git a/src/main/scala/edu/ie3/simona/event/ResultEvent.scala b/src/main/scala/edu/ie3/simona/event/ResultEvent.scala index 49b345fa15..c3264c6a2f 100644 --- a/src/main/scala/edu/ie3/simona/event/ResultEvent.scala +++ b/src/main/scala/edu/ie3/simona/event/ResultEvent.scala @@ -6,14 +6,15 @@ package edu.ie3.simona.event -import edu.ie3.datamodel.models.result.{CongestionResult, NodeResult} import edu.ie3.datamodel.models.result.connector.{ LineResult, SwitchResult, Transformer2WResult, } import edu.ie3.datamodel.models.result.system.{ + EmResult, FlexOptionsResult, + HpResult, SystemParticipantResult, } import edu.ie3.datamodel.models.result.thermal.{ @@ -21,16 +22,15 @@ import edu.ie3.datamodel.models.result.thermal.{ ThermalHouseResult, ThermalUnitResult, } -import edu.ie3.datamodel.models.result.system.{EmResult, HpResult} +import edu.ie3.datamodel.models.result.{CongestionResult, NodeResult} import edu.ie3.simona.agent.grid.GridResultsSupport.PartialTransformer3wResult -import edu.ie3.simona.event.listener.ResultEventListener import tech.units.indriya.ComparableQuantity import java.time.ZonedDateTime import java.util.UUID import javax.measure.quantity.{Energy, Power, Temperature} -sealed trait ResultEvent extends Event with ResultEventListener.Request +sealed trait ResultEvent extends Event /** Calculation result events */ diff --git a/src/main/scala/edu/ie3/simona/event/listener/DelayedStopHelper.scala b/src/main/scala/edu/ie3/simona/event/listener/DelayedStopHelper.scala index c76924af63..7b4ccbaf94 100644 --- a/src/main/scala/edu/ie3/simona/event/listener/DelayedStopHelper.scala +++ b/src/main/scala/edu/ie3/simona/event/listener/DelayedStopHelper.scala @@ -21,7 +21,7 @@ object DelayedStopHelper { * functionality */ sealed trait StoppingMsg - extends ResultEventListener.Request + extends ResultListener.Request with RuntimeEventListener.Request /** Message indicating that [[RuntimeEventListener]] should stop. Instead of diff --git a/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala b/src/main/scala/edu/ie3/simona/event/listener/ResultListener.scala similarity index 58% rename from src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala rename to src/main/scala/edu/ie3/simona/event/listener/ResultListener.scala index 28866b54f6..73cc708223 100644 --- a/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala +++ b/src/main/scala/edu/ie3/simona/event/listener/ResultListener.scala @@ -6,53 +6,49 @@ package edu.ie3.simona.event.listener -import org.apache.pekko.actor.typed.scaladsl.Behaviors -import org.apache.pekko.actor.typed.{Behavior, PostStop} import edu.ie3.datamodel.io.processor.result.ResultEntityProcessor import edu.ie3.datamodel.models.result.{NodeResult, ResultEntity} -import edu.ie3.simona.agent.grid.GridResultsSupport.PartialTransformer3wResult -import edu.ie3.simona.event.ResultEvent.{ - FlexOptionsResultEvent, - ParticipantResultEvent, - PowerFlowResultEvent, - ThermalResultEvent, -} +import edu.ie3.simona.api.data.connection.ExtResultListener +import edu.ie3.simona.api.ontology.results.ProvideResultEntities +import edu.ie3.simona.event.ResultEvent import edu.ie3.simona.exceptions.{ FileHierarchyException, ProcessResultEventException, } import edu.ie3.simona.io.result.* +import edu.ie3.simona.ontology.messages.ResultMessage +import edu.ie3.simona.ontology.messages.ResultMessage.ResultResponse import edu.ie3.simona.util.ResultFileHierarchy +import org.apache.pekko.actor.typed.scaladsl.Behaviors +import org.apache.pekko.actor.typed.{Behavior, PostStop} import org.slf4j.Logger +import edu.ie3.simona.util.CollectionUtils.asJava import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.duration.DurationInt import scala.concurrent.{Await, Future} import scala.util.{Failure, Success, Try} -object ResultEventListener extends Transformer3wResultSupport { +object ResultListener { trait Request + type Message = Request | ResultMessage.Response + private final case class SinkResponse( response: Map[Class[?], ResultEntitySink] ) extends Request private final case class InitFailed(ex: Exception) extends Request - /** [[ResultEventListener]] base data containing all information the listener - * needs + /** [[ResultListener]] base data containing all information the listener needs * * @param classToSink * a map containing the sink for each class that should be processed by the * listener */ private final case class BaseData( - classToSink: Map[Class[?], ResultEntitySink], - threeWindingResults: Map[ - Transformer3wKey, - AggregatedTransformer3wResult, - ] = Map.empty, + classToSink: Map[Class[?], ResultEntitySink] ) /** Initialize the sinks for this listener based on the provided collection @@ -147,73 +143,19 @@ object ResultEventListener extends Transformer3wResultSupport { } } - /** Handle the given result and possibly update the state data + /** Handle the given results. * - * @param resultEntity - * Result entity to handle + * @param resultEntities + * Results entity to handle. * @param baseData - * Base data - * @return - * The possibly update base data + * Base data. */ - private def handleResult( - resultEntity: ResultEntity, + private def handleResults( + resultEntities: Iterable[ResultEntity], baseData: BaseData, log: Logger, - ): BaseData = { - handOverToSink(resultEntity, baseData.classToSink, log) - baseData - } - - /** Handle a partial three winding result properly by adding it to an - * [[AggregatedTransformer3wResult]] and flushing then possibly completed - * results. Finally, the base data are updated. - * - * @param result - * Result entity to handle - * @param baseData - * Base data - * @return - * The possibly update base data - */ - private def handlePartialTransformer3wResult( - result: PartialTransformer3wResult, - baseData: BaseData, - log: Logger, - ): BaseData = { - val key = Transformer3wKey(result.input, result.time) - // retrieve existing partial result or use empty one - val partialResult = - baseData.threeWindingResults.getOrElse( - key, - AggregatedTransformer3wResult.EMPTY, - ) - // add partial result - val updatedResults = partialResult.add(result).map { updatedResult => - if updatedResult.ready then { - // if result is complete, we can write it out - updatedResult.consolidate.foreach { - handOverToSink(_, baseData.classToSink, log) - } - // also remove partial result from map - baseData.threeWindingResults.removed(key) - } else { - // if result is not complete yet, just update it - baseData.threeWindingResults + (key -> updatedResult) - } - } match { - case Success(results) => results - case Failure(exception) => - log.warn( - "Failure when handling partial Transformer3w result", - exception, - ) - // on failure, we just continue with previous results - baseData.threeWindingResults - } - - baseData.copy(threeWindingResults = updatedResults) - } + ): Unit = + resultEntities.foreach(handOverToSink(_, baseData.classToSink, log)) /** Handing over the given result entity to the sink, that might be apparent * in the map @@ -236,9 +178,29 @@ object ResultEventListener extends Transformer3wResultSupport { log.error("Error while writing result event: ", exception) } + /** Method to create an external result listener. + * + * @param connection + * Result listener data connection. + * @return + * The behavior of the listener. + */ + def external(connection: ExtResultListener): Behavior[Message] = + Behaviors.receivePartial[Message] { + case (_, ResultResponse(results)) => + connection.queueExtResponseMsg( + new ProvideResultEntities(results.asJava) + ) + + Behaviors.same + + case (ctx, msg: DelayedStopHelper.StoppingMsg) => + DelayedStopHelper.handleMsg((ctx, msg)) + } + def apply( resultFileHierarchy: ResultFileHierarchy - ): Behavior[Request] = Behaviors.setup[Request] { ctx => + ): Behavior[Message] = Behaviors.setup[Message] { ctx => ctx.log.debug("Starting initialization!") resultFileHierarchy.resultSinkType match { case _: ResultSinkType.Kafka => @@ -254,18 +216,18 @@ object ResultEventListener extends Transformer3wResultSupport { ctx.pipeToSelf( Future.sequence( - ResultEventListener.initializeSinks(resultFileHierarchy) + ResultListener.initializeSinks(resultFileHierarchy) ) ) { case Failure(exception: Exception) => InitFailed(exception) case Success(result) => SinkResponse(result.toMap) } - init() + init } - private def init(): Behavior[Request] = Behaviors.withStash(200) { buffer => - Behaviors.receive[Request] { + private def init: Behavior[Message] = Behaviors.withStash(200) { buffer => + Behaviors.receive[Message] { case (ctx, SinkResponse(response)) => ctx.log.debug("Initialization complete!") buffer.unstashAll(idle(BaseData(response))) @@ -281,47 +243,12 @@ object ResultEventListener extends Transformer3wResultSupport { } } - private def idle(baseData: BaseData): Behavior[Request] = Behaviors - .receivePartial[Request] { - case (ctx, ParticipantResultEvent(participantResult)) => - val updatedBaseData = handleResult(participantResult, baseData, ctx.log) - idle(updatedBaseData) - - case (ctx, ThermalResultEvent(thermalResult)) => - val updatedBaseData = handleResult(thermalResult, baseData, ctx.log) - idle(updatedBaseData) - - case ( - ctx, - PowerFlowResultEvent( - nodeResults, - switchResults, - lineResults, - transformer2wResults, - transformer3wResults, - congestionResults, - ), - ) => - val updatedBaseData = - (nodeResults ++ switchResults ++ lineResults ++ transformer2wResults ++ transformer3wResults ++ congestionResults) - .foldLeft(baseData) { - case (currentBaseData, resultEntity: ResultEntity) => - handleResult(resultEntity, currentBaseData, ctx.log) - case ( - currentBaseData, - partialTransformerResult: PartialTransformer3wResult, - ) => - handlePartialTransformer3wResult( - partialTransformerResult, - currentBaseData, - ctx.log, - ) - } - idle(updatedBaseData) + private def idle(baseData: BaseData): Behavior[Message] = Behaviors + .receivePartial[Message] { + case (ctx, ResultResponse(results)) => + handleResults(results.values.flatten, baseData, ctx.log) - case (ctx, FlexOptionsResultEvent(flexOptionsResult)) => - val updatedBaseData = handleResult(flexOptionsResult, baseData, ctx.log) - idle(updatedBaseData) + Behaviors.same case (ctx, msg: DelayedStopHelper.StoppingMsg) => DelayedStopHelper.handleMsg((ctx, msg)) @@ -329,15 +256,7 @@ object ResultEventListener extends Transformer3wResultSupport { } .receiveSignal { case (ctx, PostStop) => // wait until all I/O has finished - ctx.log.debug( - "Shutdown initiated.\n\tThe following three winding results are not comprehensive and are not " + - "handled in sinks:{}\n\tWaiting until writing result data is completed ...", - baseData.threeWindingResults.keys - .map { case Transformer3wKey(model, zdt) => - s"model '$model' at $zdt" - } - .mkString("\n\t\t"), - ) + ctx.log.debug("Shutdown initiated.") // close sinks concurrently to speed up closing (closing calls might be blocking) Await.ready( diff --git a/src/main/scala/edu/ie3/simona/ontology/messages/ResultMessage.scala b/src/main/scala/edu/ie3/simona/ontology/messages/ResultMessage.scala new file mode 100644 index 0000000000..dd2efb9d4f --- /dev/null +++ b/src/main/scala/edu/ie3/simona/ontology/messages/ResultMessage.scala @@ -0,0 +1,41 @@ +/* + * © 2025. TU Dortmund University, + * Institute of Energy Systems, Energy Efficiency and Energy Economics, + * Research group Distribution grid planning and operation + */ + +package edu.ie3.simona.ontology.messages + +import edu.ie3.datamodel.models.result.ResultEntity +import org.apache.pekko.actor.typed.ActorRef + +import java.util.UUID + +object ResultMessage { + + /** Message to request results. + * + * @param requestedResults + * The uuids of the input models. + * @param tick + * For which results are requested. + * @param replyTo + * The actor that should receive the results. + */ + final case class RequestResult( + requestedResults: Seq[UUID], + tick: Long, + replyTo: ActorRef[Response], + ) + + /** Trait that is extended by all responses to a [[RequestResult]]. + */ + sealed trait Response + + /** Response message that contains the requested results. + * @param results + * Map: uuid to results. + */ + final case class ResultResponse(results: Map[UUID, Iterable[ResultEntity]]) + extends Response +} diff --git a/src/main/scala/edu/ie3/simona/service/results/ExtResultProvider.scala b/src/main/scala/edu/ie3/simona/service/results/ExtResultProvider.scala new file mode 100644 index 0000000000..920e5385be --- /dev/null +++ b/src/main/scala/edu/ie3/simona/service/results/ExtResultProvider.scala @@ -0,0 +1,149 @@ +/* + * © 2025. TU Dortmund University, + * Institute of Energy Systems, Energy Efficiency and Energy Economics, + * Research group Distribution grid planning and operation + */ + +package edu.ie3.simona.service.results + +import edu.ie3.simona.api.data.connection.ExtResultDataConnection +import edu.ie3.simona.api.ontology.DataMessageFromExt +import edu.ie3.simona.api.ontology.results.{ + ProvideResultEntities, + RequestResultEntities, + ResultDataMessageFromExt, +} +import edu.ie3.simona.event.listener.DelayedStopHelper +import edu.ie3.simona.exceptions.CriticalFailureException +import edu.ie3.simona.ontology.messages.SchedulerMessage.{ + Completion, + ScheduleActivation, +} +import edu.ie3.simona.ontology.messages.ServiceMessage.ScheduleServiceActivation +import edu.ie3.simona.ontology.messages.{Activation, SchedulerMessage} +import edu.ie3.simona.ontology.messages.ResultMessage +import edu.ie3.simona.ontology.messages.ResultMessage.{ + RequestResult, + ResultResponse, +} +import org.apache.pekko.actor.typed.scaladsl.Behaviors +import org.apache.pekko.actor.typed.{ActorRef, Behavior} + +import java.util +import edu.ie3.simona.util.CollectionUtils.asJava + +import scala.jdk.CollectionConverters.* + +object ExtResultProvider { + + type Message = ResultMessage.Response | DelayedStopHelper.StoppingMsg + + /** State data for a result [[provider]]. + * + * @param scheduler + * Reference to the scheduler. + * @param resultProxy + * The result service proxy. + * @param connection + * Result data connection to the external simulation. + * @param extMessage + * Option for the current message from the external simulation. + */ + private final case class ProviderState( + scheduler: ActorRef[SchedulerMessage], + resultProxy: ActorRef[RequestResult], + connection: ExtResultDataConnection, + extMessage: Option[ResultDataMessageFromExt] = None, + ) + + /** Method to create an external result provider. In contrast to the listener, + * the result provider will only provide those result that were requested. + * + * @param connection + * Result data connection to the external simulation. + * @param scheduler + * Reference to the scheduler. + * @param resultProxy + * The result service proxy. + * @return + * The behavior of the result provider. + */ + def apply( + connection: ExtResultDataConnection, + scheduler: ActorRef[SchedulerMessage], + resultProxy: ActorRef[RequestResult], + ): Behavior[Message | DataMessageFromExt | Activation] = { + val stateData = ProviderState(scheduler, resultProxy, connection) + + provider(stateData) + } + + /** Definition of the behavior of the result provider. + * + * @param stateData + * The state data of the provider. + * @return + * The behavior of the result provider. + */ + private def provider( + stateData: ProviderState + ): Behavior[Message | DataMessageFromExt | Activation] = + Behaviors.receivePartial[Message | DataMessageFromExt | Activation] { + case (ctx, ResultResponse(results)) => + ctx.log.warn(s"Sending results to ext. Results: $results") + + // send result to external simulation + stateData.connection.queueExtResponseMsg( + new ProvideResultEntities(results.asJava) + ) + + stateData.scheduler ! Completion(ctx.self) + + Behaviors.same + + case (_, messageFromExt: ResultDataMessageFromExt) => + // save ext message + provider(stateData.copy(extMessage = Some(messageFromExt))) + + case (ctx, ScheduleServiceActivation(tick, unlockKey)) => + stateData.scheduler ! ScheduleActivation( + ctx.self, + tick, + Some(unlockKey), + ) + + Behaviors.same + + case (ctx, Activation(tick)) => + // handle ext message + + val extMsg = stateData.extMessage.getOrElse( + // this should not be possible because the external simulation schedules this provider + throw CriticalFailureException( + "ExtResultDataService was triggered without ResultDataMessageFromExt available" + ) + ) + + extMsg match { + case requestResultEntities: RequestResultEntities => + val requestedResults = + new util.ArrayList(requestResultEntities.requestedResults) + + // request results from result proxy + stateData.resultProxy ! RequestResult( + requestedResults.asScala.toSeq, + tick, + ctx.self, + ) + + Behaviors.same + case other => + ctx.log.warn(s"Cannot handle external result message: $other") + Behaviors.same + } + + case (ctx, msg: DelayedStopHelper.StoppingMsg) => + DelayedStopHelper.handleMsg((ctx, msg)) + + } +} diff --git a/src/main/scala/edu/ie3/simona/service/results/ResultServiceProxy.scala b/src/main/scala/edu/ie3/simona/service/results/ResultServiceProxy.scala new file mode 100644 index 0000000000..245d7e2da2 --- /dev/null +++ b/src/main/scala/edu/ie3/simona/service/results/ResultServiceProxy.scala @@ -0,0 +1,287 @@ +/* + * © 2025. TU Dortmund University, + * Institute of Energy Systems, Energy Efficiency and Energy Economics, + * Research group Distribution grid planning and operation + */ + +package edu.ie3.simona.service.results + +import edu.ie3.datamodel.models.result.ResultEntity +import edu.ie3.datamodel.models.result.connector.Transformer3WResult +import edu.ie3.simona.agent.grid.GridResultsSupport.PartialTransformer3wResult +import edu.ie3.simona.event.ResultEvent +import edu.ie3.simona.event.ResultEvent.* +import edu.ie3.simona.event.listener.DelayedStopHelper +import edu.ie3.simona.ontology.messages.ResultMessage.{ + RequestResult, + ResultResponse, +} +import edu.ie3.simona.service.ServiceStateData.ServiceBaseStateData +import edu.ie3.simona.service.results.Transformer3wResultSupport.{ + AggregatedTransformer3wResult, + Transformer3wKey, +} +import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK +import edu.ie3.simona.util.TickUtil.RichZonedDateTime +import org.apache.pekko.actor.typed.scaladsl.{Behaviors, StashBuffer} +import org.apache.pekko.actor.typed.{ActorRef, Behavior, PostStop} +import org.slf4j.Logger + +import java.time.ZonedDateTime +import java.util.UUID +import scala.util.{Failure, Success} + +object ResultServiceProxy { + + type Message = ResultEvent | RequestResult | ExpectResult | + DelayedStopHelper.StoppingMsg + + final case class ExpectResult(assets: UUID | Seq[UUID], tick: Long) + + private final case class ResultServiceStateData( + listeners: Seq[ActorRef[ResultResponse]], + simStartTime: ZonedDateTime, + currentTick: Long = INIT_SIM_TICK, + threeWindingResults: Map[ + Transformer3wKey, + AggregatedTransformer3wResult, + ] = Map.empty, + gridResults: Map[UUID, Iterable[ResultEntity]] = Map.empty, + results: Map[UUID, Iterable[ResultEntity]] = Map.empty, + waitingForResults: Map[UUID, Long] = Map.empty, + ) extends ServiceBaseStateData { + def notifyListener(results: Map[UUID, Iterable[ResultEntity]]): Unit = + listeners.foreach(_ ! ResultResponse(results)) + + def notifyListener(result: ResultEntity): Unit = + listeners.foreach( + _ ! ResultResponse(Map(result.getInputModel -> List(result))) + ) + + def isWaiting(uuids: Iterable[UUID], tick: Long): Boolean = { + uuids.exists { uuid => + waitingForResults.get(uuid) match { + case Some(nextTick) if nextTick <= tick => true + case _ => false + } + } + } + + def updateTick(tick: Long): ResultServiceStateData = + copy(currentTick = tick) + + def waitForResult(expectResult: ExpectResult): ResultServiceStateData = + expectResult.assets match { + case uuid: UUID => + copy(waitingForResults = + waitingForResults.updated(uuid, expectResult.tick) + ) + case uuids: Seq[UUID] => + val tick = expectResult.tick + + copy(waitingForResults = + waitingForResults ++ uuids.map(uuid => uuid -> tick).toMap + ) + } + + def addResult(result: ResultEntity): ResultServiceStateData = { + val uuid = result.getInputModel + val tick = result.getTime.toTick(using simStartTime) + + val updatedWaitingForResults = + if waitingForResults.get(uuid).contains(tick) then { + waitingForResults.removed(uuid) + } else waitingForResults + + val updatedResults = results.get(uuid) match { + case Some(values) => + val updatedValues = values + .map { value => value.getClass -> value } + .toMap + .updated(result.getClass, result) + .values + + results.updated(uuid, updatedValues) + + case None => + results.updated(uuid, Iterable(result)) + } + + copy( + results = updatedResults, + waitingForResults = updatedWaitingForResults, + ) + } + + def getResults(uuids: Seq[UUID]): Map[UUID, Iterable[ResultEntity]] = { + uuids.flatMap { uuid => + gridResults.get(uuid) match { + case Some(values) => + Some(uuid -> values) + case None => + results.get(uuid).map { res => uuid -> res } + } + }.toMap + } + + } + + def apply( + listeners: Seq[ActorRef[ResultResponse]], + simStartTime: ZonedDateTime, + bufferSize: Int = 10000, + ): Behavior[Message] = Behaviors.withStash(bufferSize) { buffer => + idle(ResultServiceStateData(listeners, simStartTime))(using buffer) + } + + private def idle( + stateData: ResultServiceStateData + )(using + buffer: StashBuffer[Message] + ): Behavior[Message] = Behaviors + .receivePartial[Message] { + case (_, expectResult: ExpectResult) => + idle(stateData.waitForResult(expectResult)) + + case (ctx, resultEvent: ResultEvent) => + // ctx.log.warn(s"Received results: $resultEvent") + + // handles the event and updates the state data + val updatedStateData = + handleResultEvent(resultEvent, stateData)(using ctx.log) + + // un-stash received requests + buffer.unstashAll(idle(updatedStateData)) + case (ctx, requestResultMessage: RequestResult) => + val requestedResults = requestResultMessage.requestedResults + val tick = requestResultMessage.tick + + if stateData.isWaiting(requestedResults, tick) then { + // ctx.log.warn(s"Cannot answer request: $requestedResults") + + buffer.stash(requestResultMessage) + } else { + + requestResultMessage.replyTo ! ResultResponse( + stateData.getResults(requestedResults) + ) + } + + Behaviors.same + + case (ctx, msg: DelayedStopHelper.StoppingMsg) => + DelayedStopHelper.handleMsg((ctx, msg)) + } + .receiveSignal { case (ctx, PostStop) => + ctx.log.debug( + "Shutdown initiated.\n\tThe following three winding results are not comprehensive and are not " + + "handled in sinks:{}\n\tWaiting until writing result data is completed ...", + stateData.threeWindingResults.keys + .map { case Transformer3wKey(model, zdt) => + s"model '$model' at $zdt" + } + .mkString("\n\t\t"), + ) + + Behaviors.same + } + + private def handleResultEvent( + resultEvent: ResultEvent, + stateData: ResultServiceStateData, + )(using log: Logger): ResultServiceStateData = resultEvent match { + case PowerFlowResultEvent( + nodeResults, + switchResults, + lineResults, + transformer2wResults, + partialTransformer3wResults, + congestionResults, + ) => + // handling of three winding transformers + val (updatedResults, transformer3wResults) = + handleThreeWindingTransformers( + partialTransformer3wResults, + stateData.threeWindingResults, + ) + + val gridResults = + (transformer3wResults ++ nodeResults ++ switchResults ++ lineResults ++ transformer2wResults ++ congestionResults) + .groupBy(_.getInputModel) + + // notify listener + stateData.notifyListener(gridResults) + + stateData.copy( + gridResults = stateData.gridResults ++ gridResults, + threeWindingResults = updatedResults, + waitingForResults = + stateData.waitingForResults.removedAll(gridResults.keys), + ) + + case ParticipantResultEvent(systemParticipantResult) => + // notify listener + stateData.notifyListener(systemParticipantResult) + + stateData.addResult(systemParticipantResult) + + case ThermalResultEvent(thermalResult) => + // notify listener + stateData.notifyListener(thermalResult) + + stateData.addResult(thermalResult) + + case FlexOptionsResultEvent(flexOptionsResult) => + // notify listener + stateData.notifyListener(flexOptionsResult) + + stateData.addResult(flexOptionsResult) + } + + private def handleThreeWindingTransformers( + transformer3wResults: Iterable[PartialTransformer3wResult], + threeWindingResults: Map[Transformer3wKey, AggregatedTransformer3wResult], + )(using log: Logger) = transformer3wResults.foldLeft( + threeWindingResults, + Seq.empty[Transformer3WResult], + ) { case ((allPartialResults, allResults), result) => + val key = Transformer3wKey(result.input, result.time) + // retrieve existing partial result or use empty one + val partialResult = + allPartialResults.getOrElse( + key, + AggregatedTransformer3wResult.EMPTY, + ) + // add partial result + partialResult.add(result).map { updatedResult => + if updatedResult.ready then { + // if result is complete, we can write it out + updatedResult.consolidate match { + case Failure(exception) => + log.warn( + "Failure when handling partial Transformer3w result", + exception, + ) + // on failure, we just continue with previous results + (allPartialResults, allResults) + case Success(res) => + (allPartialResults.removed(key), allResults.appended(res)) + } + + } else { + // if result is not complete yet, just update it + (allPartialResults + (key -> updatedResult), allResults) + } + } match { + case Success(results) => results + case Failure(exception) => + log.warn( + "Failure when handling partial Transformer3w result", + exception, + ) + // on failure, we just continue with previous results + (allPartialResults, allResults) + } + } + +} diff --git a/src/main/scala/edu/ie3/simona/event/listener/Transformer3wResultSupport.scala b/src/main/scala/edu/ie3/simona/service/results/Transformer3wResultSupport.scala similarity index 97% rename from src/main/scala/edu/ie3/simona/event/listener/Transformer3wResultSupport.scala rename to src/main/scala/edu/ie3/simona/service/results/Transformer3wResultSupport.scala index 1065475d06..c6f9787823 100644 --- a/src/main/scala/edu/ie3/simona/event/listener/Transformer3wResultSupport.scala +++ b/src/main/scala/edu/ie3/simona/service/results/Transformer3wResultSupport.scala @@ -4,19 +4,19 @@ * Research group Distribution grid planning and operation */ -package edu.ie3.simona.event.listener +package edu.ie3.simona.service.results import edu.ie3.datamodel.models.result.connector.Transformer3WResult import edu.ie3.simona.agent.grid.GridResultsSupport.PartialTransformer3wResult -import tech.units.indriya.quantity.Quantities import edu.ie3.util.quantities.PowerSystemUnits +import tech.units.indriya.quantity.Quantities import tech.units.indriya.unit.Units import java.time.ZonedDateTime import java.util.UUID import scala.util.{Failure, Success, Try} -private[listener] trait Transformer3wResultSupport { +private[results] object Transformer3wResultSupport { /** Case class to serve as a map key for unfulfilled three winding results * diff --git a/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala b/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala index 9438cdd5b1..7987d2106e 100644 --- a/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala +++ b/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala @@ -26,8 +26,8 @@ import java.nio.file.Path * overall simulation has been successful or not. For specific status * information, the user needs to pass in and subscribe to the corresponding * listener e.g. [[edu.ie3.simona.event.listener.RuntimeEventListener]] for - * simulation status or [[edu.ie3.simona.event.listener.ResultEventListener]] - * for result events + * simulation status or [[edu.ie3.simona.event.listener.ResultListener]] for + * result events * * @since 01.07.20 */ @@ -70,8 +70,15 @@ object SimonaSim { ): Behavior[Request] = Behaviors .receivePartial[Request] { case (ctx, Start(_)) => - val resultEventListeners = simonaSetup.resultEventListener(ctx) val runtimeEventListener = simonaSetup.runtimeEventListener(ctx) + val resultEventListeners = simonaSetup.resultEventListener(ctx) + + // result proxy + val resultProxy = simonaSetup.resultServiceProxy( + ctx, + resultEventListeners, + simonaSetup.simonaConfig.simona.time.simStartTime, + ) val timeAdvancer = simonaSetup.timeAdvancer(ctx, ctx.self, runtimeEventListener) @@ -86,7 +93,11 @@ object SimonaSim { simonaSetup.simonaConfig.simona.input.extSimDir.map(Path.of(_)) val extSimulationData = - simonaSetup.extSimulations(ctx, scheduler, extSimDir) + simonaSetup.extSimulations(ctx, scheduler, resultProxy, extSimDir) + + val allResultEventListeners = + resultEventListeners ++ extSimulationData.resultListeners + val resultProviders = extSimulationData.resultProviders /* start services */ // primary service proxy @@ -104,17 +115,14 @@ object SimonaSim { scheduler, runtimeEventListener, primaryServiceProxy, + resultProxy, weatherService, loadProfileService, extSimulationData.evDataService, ) /* start grid agents */ - val gridAgents = simonaSetup.gridAgents( - ctx, - environmentRefs, - resultEventListeners, - ) + val gridAgents = simonaSetup.gridAgents(ctx, environmentRefs) val otherActors = Iterable[ActorRef[?]]( timeAdvancer, @@ -123,16 +131,15 @@ object SimonaSim { weatherService, ) ++ gridAgents ++ - extSimulationData.extDataServices.map(_._2) + extSimulationData.allServiceRefs /* watch all actors */ - resultEventListeners.foreach(ctx.watch) + allResultEventListeners.foreach(ctx.watch) + resultProviders.foreach(ctx.watch) ctx.watch(runtimeEventListener) - extSimulationData.extResultListeners.foreach { case (_, ref) => - ctx.watch(ref) - } - extSimulationData.extSimAdapters.foreach(ctx.watch) + ctx.watch(resultProxy) otherActors.foreach(ctx.watch) + extSimulationData.extSimAdapters.foreach(ctx.watch) // End pre-initialization phase preInitKey.unlock() @@ -140,11 +147,11 @@ object SimonaSim { // Start simulation timeAdvancer ! TimeAdvancer.Start - val delayedActors = resultEventListeners.appended(runtimeEventListener) - - extSimulationData.extResultListeners.foreach(ref => - delayedActors.appended(ref) - ) + val delayedActors = + allResultEventListeners + .appendedAll(resultProviders) + .appended(runtimeEventListener) + .appended(resultProxy) idle( ActorData( diff --git a/src/main/scala/edu/ie3/simona/sim/setup/ExtSimSetup.scala b/src/main/scala/edu/ie3/simona/sim/setup/ExtSimSetup.scala index 4563b5e88e..ed80751e6d 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/ExtSimSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/ExtSimSetup.scala @@ -6,26 +6,26 @@ package edu.ie3.simona.sim.setup -import edu.ie3.simona.api.data.connection.{ - ExtEvDataConnection, - ExtInputDataConnection, - ExtPrimaryDataConnection, -} +import edu.ie3.simona.api.data.connection.* import edu.ie3.simona.api.ontology.DataMessageFromExt import edu.ie3.simona.api.ontology.simulation.ControlResponseMessageFromExt import edu.ie3.simona.api.simulation.{ExtSimAdapterData, ExtSimulation} import edu.ie3.simona.api.{ExtLinkInterface, ExtSimAdapter} +import edu.ie3.simona.event.listener.ResultListener import edu.ie3.simona.exceptions.ServiceException import edu.ie3.simona.ontology.messages.{SchedulerMessage, ServiceMessage} +import edu.ie3.simona.ontology.messages.ResultMessage.RequestResult import edu.ie3.simona.scheduler.ScheduleLock import edu.ie3.simona.service.ServiceStateData.InitializeServiceStateData import edu.ie3.simona.service.ev.ExtEvDataService import edu.ie3.simona.service.ev.ExtEvDataService.InitExtEvData +import edu.ie3.simona.service.results.ExtResultProvider import edu.ie3.simona.util.SimonaConstants.PRE_INIT_TICK import org.apache.pekko.actor.typed.ActorRef import org.apache.pekko.actor.typed.scaladsl.ActorContext import org.slf4j.{Logger, LoggerFactory} +import java.time.ZonedDateTime import java.util.UUID import scala.jdk.CollectionConverters.{ListHasAsScala, SetHasAsScala} import scala.util.{Failure, Success, Try} @@ -44,6 +44,10 @@ object ExtSimSetup { * The actor context of this actor system. * @param scheduler * The scheduler of simona. + * @param resultProxy + * The result service proxy. + * @param startTime + * The start time of the simulation. * @return * An [[ExtSimSetupData]] that holds information regarding the external * data connections as well as the actor references of the created @@ -55,6 +59,8 @@ object ExtSimSetup { )(using context: ActorContext[?], scheduler: ActorRef[SchedulerMessage], + resultProxy: ActorRef[RequestResult], + startTime: ZonedDateTime, ): ExtSimSetupData = extLinks.zipWithIndex.foldLeft(ExtSimSetupData.apply) { case (extSimSetupData, (extLink, index)) => // external simulation always needs at least an ExtSimAdapter @@ -79,14 +85,14 @@ object ExtSimSetup { ) // setup data services that belong to this external simulation - val updatedSetupData = connect(extSimulation, extSimSetupData) + val updatedSetupData = connect(extSimulation, extSimSetupData, index) // starting external simulation new Thread(extSimulation, s"External simulation $index") .start() // updating the data with newly connected external simulation - updatedSetupData.update(extSimAdapter) + updatedSetupData.updateAdapter(extSimAdapter) } match { case Failure(exception) => log.warn( @@ -104,6 +110,8 @@ object ExtSimSetup { * To connect. * @param extSimSetupData * That contains information about all external simulations. + * @param index + * Index of the external link interface. * @param context * The actor context of this actor system. * @param scheduler @@ -116,10 +124,13 @@ object ExtSimSetup { private[setup] def connect( extSimulation: ExtSimulation, extSimSetupData: ExtSimSetupData, + index: Int, )(using context: ActorContext[?], scheduler: ActorRef[SchedulerMessage], extSimAdapterData: ExtSimAdapterData, + resultProxy: ActorRef[RequestResult], + startTime: ZonedDateTime, ): ExtSimSetupData = { given extSimAdapter: ActorRef[ControlResponseMessageFromExt] = extSimAdapterData.getAdapter @@ -135,7 +146,7 @@ object ExtSimSetup { case (setupData, connection) => connection match { case extEvDataConnection: ExtEvDataConnection => - if setupData.evDataConnection.nonEmpty then { + if setupData.evDataService.nonEmpty then { throw ServiceException( s"Trying to connect another EvDataConnection. Currently only one is allowed." ) @@ -154,6 +165,31 @@ object ExtSimSetup { extSimSetupData.update(extEvDataConnection, serviceRef) + case extResultDataConnection: ExtResultDataConnection => + val extResultProvider = context.spawn( + ExtResultProvider( + extResultDataConnection, + scheduler, + resultProxy, + ), + s"ExtResultProvider", + ) + + extResultDataConnection.setActorRefs( + extResultProvider, + extSimAdapter, + ) + + extSimSetupData.update(extResultDataConnection, extResultProvider) + + case extResultListener: ExtResultListener => + val extResultEventListener = context.spawn( + ResultListener.external(extResultListener), + s"ExtResultListener_$index", + ) + + extSimSetupData.update(extResultListener, extResultEventListener) + case otherConnection => log.warn( s"There is currently no implementation for the connection: $otherConnection." 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 a08f71d1ab..0db9e5ef1b 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/ExtSimSetupData.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/ExtSimSetupData.scala @@ -8,7 +8,10 @@ package edu.ie3.simona.sim.setup import edu.ie3.simona.api.ExtSimAdapter import edu.ie3.simona.api.data.connection.* +import edu.ie3.simona.event.listener.ResultListener import edu.ie3.simona.ontology.messages.ServiceMessage +import edu.ie3.simona.service.ev.ExtEvDataService +import edu.ie3.simona.service.results.ExtResultProvider import org.apache.pekko.actor.typed.ActorRef /** Case class that holds information regarding the external data connections as @@ -16,82 +19,73 @@ import org.apache.pekko.actor.typed.ActorRef * * @param extSimAdapters * All adapters to external simulations. - * @param extPrimaryDataServices + * @param primaryDataServices * Seq: external primary data connections to service references. - * @param extDataServices - * Seq: external input data connection to service references. - * @param extResultListeners - * Map: external result data connections to result data providers. + * @param evDataService + * Option for an external ev data service. + * @param resultListeners + * Seq: external result listeners. + * @param resultProviders + * Seq: external result providers. */ final case class ExtSimSetupData( extSimAdapters: Iterable[ActorRef[ExtSimAdapter.Request]], - extPrimaryDataServices: Seq[ + primaryDataServices: Seq[ (ExtPrimaryDataConnection, ActorRef[ServiceMessage]) ], - extDataServices: Seq[ - (? <: ExtInputDataConnection[?], ActorRef[ServiceMessage]) - ], - extResultListeners: Seq[(ExtResultDataConnection, ActorRef[ServiceMessage])], + evDataService: Option[ActorRef[ExtEvDataService.Message]], + resultListeners: Seq[ActorRef[ResultListener.Message]], + resultProviders: Seq[ActorRef[ExtResultProvider.Message]], ) { private[setup] def update( connection: ExtPrimaryDataConnection, ref: ActorRef[ServiceMessage], ): ExtSimSetupData = - copy(extPrimaryDataServices = - extPrimaryDataServices ++ Seq((connection, ref)) - ) + copy(primaryDataServices = primaryDataServices ++ Seq((connection, ref))) private[setup] def update( - connection: ExtInputDataConnection[?], - ref: ActorRef[ServiceMessage], - ): ExtSimSetupData = connection match { - case primaryConnection: ExtPrimaryDataConnection => - update(primaryConnection, ref) - case _ => - copy(extDataServices = extDataServices ++ Seq((connection, ref))) + connection: ExtDataConnection, + ref: ActorRef[?], + ): ExtSimSetupData = (connection, ref) match { + case ( + primaryConnection: ExtPrimaryDataConnection, + serviceRef: ActorRef[ServiceMessage], + ) => + update(primaryConnection, serviceRef) + case ( + _: ExtEvDataConnection, + serviceRef: ActorRef[ExtEvDataService.Message], + ) => + copy(evDataService = Some(serviceRef)) + case ( + _: ExtResultDataConnection, + providerRef: ActorRef[ExtResultProvider.Message], + ) => + copy(resultProviders = resultProviders ++ Seq(providerRef)) + case ( + _: ExtResultListener, + listenerRef: ActorRef[ResultListener.Message], + ) => + copy(resultListeners = resultListeners ++ Seq(listenerRef)) + case (_, _) => + this } - private[setup] def update( - connection: ExtResultDataConnection, - ref: ActorRef[ServiceMessage], - ): ExtSimSetupData = - copy(extResultListeners = extResultListeners ++ Seq((connection, ref))) - - private[setup] def update( + private[setup] def updateAdapter( extSimAdapter: ActorRef[ExtSimAdapter.Request] ): ExtSimSetupData = copy(extSimAdapters = extSimAdapters ++ Set(extSimAdapter)) - def evDataService: Option[ActorRef[ServiceMessage]] = - extDataServices.collectFirst { - case (_: ExtEvDataConnection, ref: ActorRef[ServiceMessage]) => ref - } - - def emDataService: Option[ActorRef[ServiceMessage]] = - extDataServices.collectFirst { - case (_: ExtEmDataConnection, ref: ActorRef[ServiceMessage]) => ref - } - - def evDataConnection: Option[ExtEvDataConnection] = - extDataServices.collectFirst { case (connection: ExtEvDataConnection, _) => - connection - } - - def emDataConnection: Option[ExtEmDataConnection] = - extDataServices.collectFirst { case (connection: ExtEmDataConnection, _) => - connection - } - def primaryDataConnections: Seq[ExtPrimaryDataConnection] = - extPrimaryDataServices.map { - case (connection: ExtPrimaryDataConnection, _) => connection - } - - def resultDataConnections: Seq[ExtResultDataConnection] = - extResultListeners.map { case (connection: ExtResultDataConnection, _) => + primaryDataServices.map { case (connection: ExtPrimaryDataConnection, _) => connection } + + def allServiceRefs: Iterable[ActorRef[?]] = + primaryDataServices.map(_._2) ++ Seq( + evDataService + ).flatten ++ resultListeners ++ resultProviders } object ExtSimSetupData { @@ -101,6 +95,7 @@ object ExtSimSetupData { def apply: ExtSimSetupData = ExtSimSetupData( Iterable.empty, Seq.empty, + None, Seq.empty, Seq.empty, ) diff --git a/src/main/scala/edu/ie3/simona/sim/setup/SimonaSetup.scala b/src/main/scala/edu/ie3/simona/sim/setup/SimonaSetup.scala index 8398ba9533..0b4696ca3a 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaSetup.scala @@ -11,17 +11,23 @@ import edu.ie3.datamodel.models.input.connector.Transformer3WInput import edu.ie3.simona.agent.EnvironmentRefs import edu.ie3.simona.agent.grid.GridAgent import edu.ie3.simona.config.SimonaConfig -import edu.ie3.simona.event.listener.{ResultEventListener, RuntimeEventListener} +import edu.ie3.simona.event.listener.{ResultListener, RuntimeEventListener} import edu.ie3.simona.event.{ResultEvent, RuntimeEvent} import edu.ie3.simona.ontology.messages.{SchedulerMessage, ServiceMessage} +import edu.ie3.simona.ontology.messages.ResultMessage.{ + RequestResult, + ResultResponse, +} import edu.ie3.simona.scheduler.TimeAdvancer import edu.ie3.simona.scheduler.core.Core.CoreFactory import edu.ie3.simona.scheduler.core.RegularSchedulerCore +import edu.ie3.simona.service.results.ResultServiceProxy import edu.ie3.simona.sim.SimonaSim import org.apache.pekko.actor.typed.ActorRef import org.apache.pekko.actor.typed.scaladsl.ActorContext import java.nio.file.Path +import java.time.ZonedDateTime /** Trait that can be used to set up a customized simona simulation by providing * implementations for all setup information required by a @@ -38,7 +44,7 @@ trait SimonaSetup { /** Main arguments of the executable. May be used to pass additional * configuration parameters to the setup e.g. for external simulation - * configuration + * configuration. */ val args: Array[String] @@ -46,41 +52,41 @@ trait SimonaSetup { */ def logOutputDir: Path - /** Creates the runtime event listener + /** Creates the runtime event listener. * * @param context - * Actor context to use + * Actor context to use. * @return - * An actor reference to the runtime event listener + * An actor reference to the runtime event listener. */ def runtimeEventListener( context: ActorContext[?] ): ActorRef[RuntimeEventListener.Request] - /** Creates a sequence of result event listeners + /** Creates a sequence of result event listeners. * * @param context - * Actor context to use + * Actor context to use. * @return - * A sequence of actor references to result event listeners + * A sequence of actor references to result event listeners. */ def resultEventListener( context: ActorContext[?] - ): Seq[ActorRef[ResultEventListener.Request]] + ): Seq[ActorRef[ResultListener.Message]] /** Creates a primary service proxy. The proxy is the first instance to ask * for primary data. If necessary, it delegates the registration request to * it's subordinate workers. * * @param context - * Actor context to use + * Actor context to use. * @param scheduler - * Actor reference to it's according scheduler to use + * Actor reference to it's according scheduler to use. * @param extSimSetupData - * that can contain external + * that can contain external. * [[edu.ie3.simona.api.data.primarydata.ExtPrimaryDataConnection]] * @return - * An actor reference to the service + * An actor reference to the service. */ def primaryServiceProxy( context: ActorContext[?], @@ -88,63 +94,88 @@ trait SimonaSetup { extSimSetupData: ExtSimSetupData, ): ActorRef[ServiceMessage] - /** Creates a weather service + /** Creates a result service proxy. The proxy will receive information about + * the result that should be expected for the current tick and all result + * events that are send by the agents. The proxy is responsible for + * processing the result events and passing the processed data to the + * different result listeners and providers. + * + * @param context + * Actor context to use. + * @param listeners + * The internal result event listeners. + * @param simStartTime + * The start time of the simulation. + * @return + * An actor reference to the service. + */ + def resultServiceProxy( + context: ActorContext[?], + listeners: Seq[ActorRef[ResultResponse]], + simStartTime: ZonedDateTime, + ): ActorRef[ResultServiceProxy.Message] + + /** Creates a weather service. * * @param context - * Actor context to use + * Actor context to use. * @param scheduler - * Actor reference to it's according scheduler to use + * Actor reference to it's according scheduler to use. * @return * An actor reference to the service as well as matching data to initialize - * the service + * the service. */ def weatherService( context: ActorContext[?], scheduler: ActorRef[SchedulerMessage], ): ActorRef[ServiceMessage] - /** Creates a load profile service + /** Creates a load profile service. * * @param context - * Actor context to use + * Actor context to use. * @param scheduler - * Actor reference to it's according scheduler to use + * Actor reference to it's according scheduler to use. * @return * An actor reference to the service as well as matching data to initialize - * the service + * the service. */ def loadProfileService( context: ActorContext[?], scheduler: ActorRef[SchedulerMessage], ): ActorRef[ServiceMessage] - /** Loads external simulations and provides corresponding actors and init data + /** Loads external simulations and provides corresponding actors and init + * data. * * @param context - * Actor context to use + * Actor context to use. * @param scheduler - * Actor reference to the scheduler to use + * Actor reference to the scheduler to use. + * @param resultProxy + * Actor reference to the result provider. * @param extSimPath - * option for a directory with external simulations + * Option for a directory with external simulations. * @return - * External simulations and their init data + * External simulations and their init data. */ def extSimulations( context: ActorContext[?], scheduler: ActorRef[SchedulerMessage], + resultProxy: ActorRef[RequestResult], extSimPath: Option[Path], ): ExtSimSetupData - /** Creates the time advancer + /** Creates the time advancer. * * @param context - * Actor context to use + * Actor context to use. * @param simulation - * The simulation root actor ([[edu.ie3.simona.sim.SimonaSim]]) + * The simulation root actor ([[edu.ie3.simona.sim.SimonaSim]]). * @param runtimeEventListener - * Runtime event listener + * Runtime event listener. * @return - * An actor reference to the time advancer + * An actor reference to the time advancer. */ def timeAdvancer( context: ActorContext[?], @@ -152,17 +183,17 @@ trait SimonaSetup { runtimeEventListener: ActorRef[RuntimeEvent], ): ActorRef[TimeAdvancer.Request] - /** Creates a scheduler service + /** Creates a scheduler service. * * @param context - * Actor context to use + * Actor context to use. * @param parent - * The parent scheduler, which could be a time advancer + * The parent scheduler, which could be a time advancer. * @param coreFactory * The factory creating a scheduler core that determines the scheduler's - * behavior, defaulting to a regular scheduler + * behavior, defaulting to a regular scheduler. * @return - * An actor reference to the scheduler + * An actor reference to the scheduler. */ def scheduler( context: ActorContext[?], @@ -170,27 +201,24 @@ trait SimonaSetup { coreFactory: CoreFactory = RegularSchedulerCore, ): ActorRef[SchedulerMessage] - /** Creates all the needed grid agents + /** Creates all the needed grid agents. * * @param context - * Actor context to use + * Actor context to use. * @param environmentRefs - * EnvironmentRefs to use - * @param resultEventListeners - * Listeners that await events from system participants + * EnvironmentRefs to use. * @return * A mapping from actor reference to it's according initialization data to - * be used when setting up the agents + * be used when setting up the agents. */ def gridAgents( context: ActorContext[?], environmentRefs: EnvironmentRefs, - resultEventListeners: Seq[ActorRef[ResultEvent]], ): Iterable[ActorRef[GridAgent.Message]] /** SIMONA links sub grids connected by a three winding transformer a bit * different. Therefore, the internal node has to be set as superior node. - * All other gates are left unchanged + * All other gates are left unchanged. */ protected val modifySubGridGateForThreeWindingSupport : SubGridGate => SubGridGate = 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 854094e44d..3eb44caf66 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala @@ -15,11 +15,15 @@ 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.config.{GridConfigParser, SimonaConfig} -import edu.ie3.simona.event.listener.{ResultEventListener, RuntimeEventListener} +import edu.ie3.simona.event.listener.{ResultListener, RuntimeEventListener} import edu.ie3.simona.event.{ResultEvent, RuntimeEvent} import edu.ie3.simona.exceptions.agent.GridAgentInitializationException import edu.ie3.simona.io.grid.GridProvider import edu.ie3.simona.ontology.messages.{SchedulerMessage, ServiceMessage} +import edu.ie3.simona.ontology.messages.ResultMessage.{ + RequestResult, + ResultResponse, +} import edu.ie3.simona.scheduler.core.Core.CoreFactory import edu.ie3.simona.scheduler.core.RegularSchedulerCore import edu.ie3.simona.scheduler.{ScheduleLock, Scheduler, TimeAdvancer} @@ -27,6 +31,7 @@ import edu.ie3.simona.service.load.LoadProfileService import edu.ie3.simona.service.load.LoadProfileService.InitLoadProfileServiceStateData import edu.ie3.simona.service.primary.PrimaryServiceProxy import edu.ie3.simona.service.primary.PrimaryServiceProxy.InitPrimaryServiceProxyStateData +import edu.ie3.simona.service.results.ResultServiceProxy import edu.ie3.simona.service.weather.WeatherService import edu.ie3.simona.service.weather.WeatherService.InitWeatherServiceStateData import edu.ie3.simona.sim.SimonaSim @@ -39,6 +44,7 @@ import org.apache.pekko.actor.typed.ActorRef import org.apache.pekko.actor.typed.scaladsl.ActorContext import java.nio.file.Path +import java.time.ZonedDateTime import java.util.UUID import java.util.concurrent.LinkedBlockingQueue import scala.jdk.CollectionConverters.* @@ -62,7 +68,6 @@ class SimonaStandaloneSetup( override def gridAgents( context: ActorContext[?], environmentRefs: EnvironmentRefs, - resultEventListeners: Seq[ActorRef[ResultEvent]], ): Iterable[ActorRef[GridAgent.Message]] = { /* get the grid */ @@ -85,7 +90,6 @@ class SimonaStandaloneSetup( subGridTopologyGraph, context, environmentRefs, - resultEventListeners, ) val keys = ScheduleLock.multiKey( @@ -164,6 +168,16 @@ class SimonaStandaloneSetup( primaryServiceProxy } + override def resultServiceProxy( + context: ActorContext[?], + listeners: Seq[ActorRef[ResultResponse]], + simStartTime: ZonedDateTime, + ): ActorRef[ResultServiceProxy.Message] = + context.spawn( + ResultServiceProxy(listeners, simStartTime), + "resultEventProxyAgent", + ) + override def weatherService( context: ActorContext[?], scheduler: ActorRef[SchedulerMessage], @@ -211,6 +225,7 @@ class SimonaStandaloneSetup( override def extSimulations( context: ActorContext[?], scheduler: ActorRef[SchedulerMessage], + resultProxy: ActorRef[RequestResult], extSimPath: Option[Path], ): ExtSimSetupData = { val jars = ExtSimLoader.scanInputFolder(extSimPath) @@ -219,6 +234,8 @@ class SimonaStandaloneSetup( setupExtSim(extLinks, args)(using context, scheduler, + resultProxy, + simonaConfig.simona.time.simStartTime, ) } @@ -227,12 +244,8 @@ class SimonaStandaloneSetup( simulation: ActorRef[SimonaSim.SimulationEnded.type], runtimeEventListener: ActorRef[RuntimeEvent], ): ActorRef[TimeAdvancer.Request] = { - val startDateTime = TimeUtil.withDefaults.toZonedDateTime( - simonaConfig.simona.time.startDateTime - ) - val endDateTime = TimeUtil.withDefaults.toZonedDateTime( - simonaConfig.simona.time.endDateTime - ) + val startDateTime = simonaConfig.simona.time.simStartTime + val endDateTime = simonaConfig.simona.time.simEndTime context.spawn( TimeAdvancer( @@ -271,15 +284,13 @@ class SimonaStandaloneSetup( override def resultEventListener( context: ActorContext[?] - ): Seq[ActorRef[ResultEventListener.Request]] = { + ): Seq[ActorRef[ResultListener.Message]] = { // append ResultEventListener as well to write raw output files Seq( context .spawn( - ResultEventListener( - resultFileHierarchy - ), - ResultEventListener.getClass.getSimpleName, + ResultListener(resultFileHierarchy), + ResultListener.getClass.getSimpleName, ) ) } @@ -288,7 +299,6 @@ class SimonaStandaloneSetup( subGridTopologyGraph: SubGridTopologyGraph, context: ActorContext[?], environmentRefs: EnvironmentRefs, - resultEventListeners: Seq[ActorRef[ResultEvent]], ): Map[Int, ActorRef[GridAgent.Message]] = { subGridTopologyGraph .vertexSet() @@ -299,7 +309,6 @@ class SimonaStandaloneSetup( GridAgent( environmentRefs, simonaConfig, - resultEventListeners, ), subGridContainer.getSubnet.toString, ) diff --git a/src/main/scala/edu/ie3/simona/util/CollectionUtils.scala b/src/main/scala/edu/ie3/simona/util/CollectionUtils.scala index e84e14efa1..31a8b585a9 100644 --- a/src/main/scala/edu/ie3/simona/util/CollectionUtils.scala +++ b/src/main/scala/edu/ie3/simona/util/CollectionUtils.scala @@ -10,9 +10,19 @@ import squants.Quantity import scala.annotation.tailrec import scala.collection.immutable.HashSet +import scala.math.Ordering.Double +import scala.jdk.CollectionConverters.{SeqHasAsJava, MapHasAsJava} object CollectionUtils { + extension [K, V](scalaMap: Map[K, Iterable[V]]) { + def asJava: java.util.Map[K, java.util.List[V]] = { + scalaMap.map { case (key, value) => + key -> value.toList.asJava + }.asJava + } + } + /** fast implementation to test if a list contains duplicates. See * https://stackoverflow.com/questions/3871491/functional-programming-does-a-list-only-contain-unique-items * for details diff --git a/src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala b/src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala index 2b1f679014..76b0d73ebc 100644 --- a/src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala +++ b/src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala @@ -31,10 +31,12 @@ import edu.ie3.simona.ontology.messages.ServiceMessage.{ SecondaryServiceRegistrationMessage, } import edu.ie3.simona.ontology.messages.{Activation, SchedulerMessage} +import edu.ie3.simona.ontology.messages.ResultMessage.RequestResult import edu.ie3.simona.scheduler.ScheduleLock import edu.ie3.simona.service.Data.SecondaryData.WeatherData import edu.ie3.simona.service.ServiceType import edu.ie3.simona.service.primary.PrimaryServiceProxy +import edu.ie3.simona.service.results.ResultServiceProxy.ExpectResult import edu.ie3.simona.service.weather.WeatherService.WeatherRegistrationData import edu.ie3.simona.service.weather.{WeatherDataType, WeatherService} import edu.ie3.simona.test.common.TestSpawnerTyped @@ -105,7 +107,8 @@ class EmAgentIT "having load, pv and storage agents connected" should { "be initialized correctly and run through some activations" in { val gridAgent = TestProbe[GridAgent.Message]("GridAgent") - val resultListener = TestProbe[ResultEvent]("ResultListener") + val resultServiceProxy = + TestProbe[ResultEvent | ExpectResult]("ResultServiceProxy") val primaryServiceProxy = TestProbe[PrimaryServiceProxy.Message]("PrimaryServiceProxy") val weatherService = TestProbe[WeatherService.Message]("WeatherService") @@ -114,8 +117,8 @@ class EmAgentIT given ParticipantRefs = ParticipantRefs( gridAgent = gridAgent.ref, primaryServiceProxy = primaryServiceProxy.ref, + resultServiceProxy = resultServiceProxy.ref, services = Map(ServiceType.WeatherService -> weatherService.ref), - resultListener = Iterable(resultListener.ref), ) val keys = ScheduleLock @@ -133,7 +136,7 @@ class EmAgentIT "PRIORITIZED", simulationStartDate, parent = Left(scheduler.ref), - listener = Iterable(resultListener.ref), + listener = resultServiceProxy.ref, ), "EmAgent", ) @@ -244,14 +247,14 @@ class EmAgentIT Some(7200), ) - resultListener.expectMessageType[ParticipantResultEvent] match { + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid emResult.getTime shouldBe 0L.toDateTime emResult.getP should equalWithTolerance(-0.00057340027.asMegaWatt) emResult.getQ should equalWithTolerance(-0.0018318880807.asMegaVar) } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(emAgentActivation, Some(7200))) /* TICK 7200 @@ -275,14 +278,14 @@ class EmAgentIT Some(14400), ) - resultListener.expectMessageType[ParticipantResultEvent] match { + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid emResult.getTime shouldBe 7200.toDateTime emResult.getP should equalWithTolerance(0.asMegaWatt) emResult.getQ should equalWithTolerance(-0.00113292701968.asMegaVar) } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(emAgentActivation, Some(13246))) /* TICK 13246 @@ -294,14 +297,14 @@ class EmAgentIT */ emAgentActivation ! Activation(13246) - resultListener.expectMessageType[ParticipantResultEvent] match { + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid emResult.getTime shouldBe 13246.toDateTime emResult.getP should equalWithTolerance(-0.00344685673.asMegaWatt) emResult.getQ should equalWithTolerance(-0.001132927.asMegaVar) } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(emAgentActivation, Some(14400))) /* TICK 14400 @@ -328,14 +331,14 @@ class EmAgentIT emAgentActivation ! Activation(14400) - resultListener.expectMessageType[ParticipantResultEvent] match { + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid emResult.getTime shouldBe 14400.toDateTime emResult.getP should equalWithTolerance(0.asMegaWatt) emResult.getQ should equalWithTolerance(0.000065375.asMegaVar) } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(emAgentActivation, Some(21600))) } } @@ -343,7 +346,8 @@ class EmAgentIT "having load, pv and heat pump agents connected" should { "be initialized correctly and run through some activations" in { val gridAgent = TestProbe[GridAgent.Message]("GridAgent") - val resultListener = TestProbe[ResultEvent]("ResultListener") + val resultServiceProxy = + TestProbe[ResultEvent | ExpectResult]("ResultServiceProxy") val primaryServiceProxy = TestProbe[PrimaryServiceProxy.Message]("PrimaryServiceProxy") val weatherService = TestProbe[WeatherService.Message]("WeatherService") @@ -352,8 +356,8 @@ class EmAgentIT given ParticipantRefs = ParticipantRefs( gridAgent = gridAgent.ref, primaryServiceProxy = primaryServiceProxy.ref, + resultServiceProxy = resultServiceProxy.ref, services = Map(ServiceType.WeatherService -> weatherService.ref), - resultListener = Iterable(resultListener.ref), ) val keys = ScheduleLock @@ -371,7 +375,7 @@ class EmAgentIT "PRIORITIZED", simulationStartDate, parent = Left(scheduler.ref), - listener = Iterable(resultListener.ref), + listener = resultServiceProxy.ref, ), "EmAgent1", ) @@ -502,14 +506,14 @@ class EmAgentIT ) } - resultListener.expectMessageType[ParticipantResultEvent] match { + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid emResult.getTime shouldBe 0.toDateTime emResult.getP should equalWithTolerance(-0.0055734002706.asMegaWatt) emResult.getQ should equalWithTolerance(-0.0018318880807.asMegaVar) } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(emAgentActivation, Some(7200))) /* TICK 7200 @@ -535,14 +539,14 @@ class EmAgentIT ) } - resultListener.expectMessageType[ParticipantResultEvent] match { + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid emResult.getTime shouldBe 7200.toDateTime emResult.getP should equalWithTolerance(0.001403143271.asMegaWatt) emResult.getQ should equalWithTolerance(-0.00014809252.asMegaVar) } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(emAgentActivation, Some(10800))) /* TICK 10800 @@ -569,14 +573,14 @@ class EmAgentIT ) } - resultListener.expectMessageType[ParticipantResultEvent] match { + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid emResult.getTime shouldBe 10800.toDateTime emResult.getP should equalWithTolerance(0.0011098586291.asMegaWatt) emResult.getQ should equalWithTolerance(-0.000244490516.asMegaVar) } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(emAgentActivation, Some(11000))) /* TICK 11000 @@ -603,14 +607,14 @@ class EmAgentIT ) } - resultListener.expectMessageType[ParticipantResultEvent] match { + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid emResult.getTime shouldBe 11000.toDateTime emResult.getP should equalWithTolerance(0.00021037894.asMegaWatt) emResult.getQ should equalWithTolerance(0.0000691482.asMegaVar) } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(emAgentActivation, Some(11500))) /* TICK 11500 @@ -637,14 +641,14 @@ class EmAgentIT ) } - resultListener.expectMessageType[ParticipantResultEvent] match { + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid emResult.getTime shouldBe 11500.toDateTime emResult.getP should equalWithTolerance(0.00013505248.asMegaWatt) emResult.getQ should equalWithTolerance(0.000044389603878.asMegaVar) } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(emAgentActivation, Some(28800))) } } @@ -652,7 +656,8 @@ class EmAgentIT "having a pv and a load agent connected" should { "have correct values also for agents with limited operation time" in { val gridAgent = TestProbe[GridAgent.Message]("GridAgent") - val resultListener = TestProbe[ResultEvent]("ResultListener") + val resultServiceProxy = + TestProbe[ResultEvent | ExpectResult]("ResultServiceProxy") val primaryServiceProxy = TestProbe[PrimaryServiceProxy.Message]("PrimaryServiceProxy") val weatherService = TestProbe[WeatherService.Message]("WeatherService") @@ -661,8 +666,8 @@ class EmAgentIT given ParticipantRefs = ParticipantRefs( gridAgent = gridAgent.ref, primaryServiceProxy = primaryServiceProxy.ref, + resultServiceProxy = resultServiceProxy.ref, services = Map(ServiceType.WeatherService -> weatherService.ref), - resultListener = Iterable(resultListener.ref), ) val keys = ScheduleLock @@ -680,7 +685,7 @@ class EmAgentIT "PRIORITIZED", simulationStartDate, parent = Left(scheduler.ref), - listener = Iterable(resultListener.ref), + listener = resultServiceProxy.ref, ), "EmAgentReactivePower", ) @@ -782,14 +787,14 @@ class EmAgentIT ) } - resultListener.expectMessageType[ParticipantResultEvent] match { + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid emResult.getTime shouldBe 0.toDateTime emResult.getP should equalWithTolerance(0.000268603.asMegaWatt) emResult.getQ should equalWithTolerance(0.0000882855367.asMegaVar) } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(emAgentActivation, Some(3600))) /* TICK 3600 @@ -813,14 +818,14 @@ class EmAgentIT ) } - resultListener.expectMessageType[ParticipantResultEvent] match { + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid emResult.getTime shouldBe 3600.toDateTime emResult.getP should equalWithTolerance(0.000268603.asMegaWatt) emResult.getQ should equalWithTolerance(0.0000882855367.asMegaVar) } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(emAgentActivation, Some(7200))) /* TICK 7200 @@ -844,14 +849,14 @@ class EmAgentIT emAgentActivation ! Activation(7200) - resultListener.expectMessageType[ParticipantResultEvent] match { + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid emResult.getTime shouldBe 7200.toDateTime emResult.getP should equalWithTolerance(-0.008423564.asMegaWatt) emResult.getQ should equalWithTolerance(-0.0027686916118.asMegaVar) } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(emAgentActivation, Some(10800))) /* TICK 10800 @@ -860,14 +865,14 @@ class EmAgentIT -> expect P and Q values of PV */ emAgentActivation ! Activation(10800) - resultListener.expectMessageType[ParticipantResultEvent] match { + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid emResult.getTime shouldBe 10800.toDateTime emResult.getP should equalWithTolerance(-0.008692167.asMegaWatt) emResult.getQ should equalWithTolerance(-0.00285697715.asMegaVar) } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(emAgentActivation, Some(14400))) /* TICK 14400 @@ -876,14 +881,14 @@ class EmAgentIT -> expect P: 0 W Q: 0 var */ emAgentActivation ! Activation(14400) - resultListener.expectMessageType[ParticipantResultEvent] match { + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid emResult.getTime shouldBe 14400.toDateTime emResult.getP should equalWithTolerance(0.asMegaWatt) emResult.getQ should equalWithTolerance(0.asMegaVar) } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(emAgentActivation, None)) } } diff --git a/src/test/scala/edu/ie3/simona/agent/em/EmAgentSpec.scala b/src/test/scala/edu/ie3/simona/agent/em/EmAgentSpec.scala index 80bd7813cf..ec9e43640e 100644 --- a/src/test/scala/edu/ie3/simona/agent/em/EmAgentSpec.scala +++ b/src/test/scala/edu/ie3/simona/agent/em/EmAgentSpec.scala @@ -67,7 +67,7 @@ class EmAgentSpec "A self-optimizing EM agent" should { "be initialized correctly and run through some activations" in { - val resultListener = TestProbe[ResultEvent]("ResultListener") + val resultProxy = TestProbe[ResultEvent]("ResultProxy") val scheduler = TestProbe[SchedulerMessage]("Scheduler") val emAgent = spawn( @@ -78,7 +78,7 @@ class EmAgentSpec "PRIORITIZED", simulationStart, parent = Left(scheduler.ref), - listener = Iterable(resultListener.ref), + listener = resultProxy.ref, ) ) @@ -116,7 +116,7 @@ class EmAgentSpec ) // expect no results for init - resultListener.expectNoMessage() + resultProxy.expectNoMessage() // expect completion from EmAgent scheduler.expectMessage( Completion(emAgent, Some(0)) @@ -178,7 +178,7 @@ class EmAgentSpec ) // expect correct results - resultListener.expectMessageType[FlexOptionsResultEvent] match { + resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(flexResult) => flexResult.getInputModel shouldBe emInput.getUuid flexResult.getTime shouldBe 0.toDateTime @@ -187,7 +187,7 @@ class EmAgentSpec flexResult.getpMax() should equalWithTolerance(.006.asMegaWatt) } - resultListener.expectMessageType[ParticipantResultEvent] match { + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid emResult.getTime shouldBe simulationStart @@ -232,7 +232,7 @@ class EmAgentSpec emAgent ! FlexCompletion(modelUuid = evcsInput.getUuid) // expect correct results - resultListener.expectMessageType[FlexOptionsResultEvent] match { + resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(flexResult) => flexResult.getInputModel shouldBe emInput.getUuid flexResult.getTime shouldBe 300.toDateTime @@ -241,7 +241,7 @@ class EmAgentSpec flexResult.getpMax() should equalWithTolerance(-.005.asMegaWatt) } - resultListener.expectMessageType[ParticipantResultEvent] match { + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid emResult.getTime shouldBe 300.toDateTime @@ -255,7 +255,7 @@ class EmAgentSpec } "revoke triggers correctly" in { - val resultListener = TestProbe[ResultEvent]("ResultListener") + val resultProxy = TestProbe[ResultEvent]("ResultProxy") val scheduler = TestProbe[SchedulerMessage]("Scheduler") val emAgent = spawn( @@ -266,7 +266,7 @@ class EmAgentSpec "PRIORITIZED", simulationStart, parent = Left(scheduler.ref), - listener = Iterable(resultListener.ref), + listener = resultProxy.ref, ) ) @@ -344,7 +344,7 @@ class EmAgentSpec ) // expect correct results - resultListener.expectMessageType[FlexOptionsResultEvent] match { + resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(flexResult) => flexResult.getInputModel shouldBe emInput.getUuid flexResult.getTime shouldBe 0.toDateTime @@ -353,7 +353,7 @@ class EmAgentSpec flexResult.getpMax() should equalWithTolerance(.006.asMegaWatt) } - resultListener.expectMessageType[ParticipantResultEvent] match { + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid emResult.getTime shouldBe simulationStart @@ -414,7 +414,7 @@ class EmAgentSpec ) // expect correct results - resultListener.expectMessageType[FlexOptionsResultEvent] match { + resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(flexResult) => flexResult.getInputModel shouldBe emInput.getUuid flexResult.getTime shouldBe 300.toDateTime @@ -423,7 +423,7 @@ class EmAgentSpec flexResult.getpMax() should equalWithTolerance(.008.asMegaWatt) } - resultListener.expectMessageType[ParticipantResultEvent] match { + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid emResult.getTime shouldBe 300.toDateTime @@ -438,7 +438,7 @@ class EmAgentSpec } "handle ChangingFlexOptions indicator correctly" in { - val resultListener = TestProbe[ResultEvent]("ResultListener") + val resultProxy = TestProbe[ResultEvent]("ResultProxy") val scheduler = TestProbe[SchedulerMessage]("Scheduler") val emAgent = spawn( @@ -449,7 +449,7 @@ class EmAgentSpec "PRIORITIZED", simulationStart, parent = Left(scheduler.ref), - listener = Iterable(resultListener.ref), + listener = resultProxy.ref, ) ) @@ -528,7 +528,7 @@ class EmAgentSpec requestAtTick = Some(600), ) - resultListener.expectMessageType[FlexOptionsResultEvent] match { + resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(flexResult) => flexResult.getInputModel shouldBe emInput.getUuid flexResult.getTime shouldBe 0.toDateTime @@ -537,7 +537,7 @@ class EmAgentSpec flexResult.getpMax() should equalWithTolerance(.006.asMegaWatt) } - resultListener.expectMessageType[ParticipantResultEvent] match { + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid emResult.getTime shouldBe 0.toDateTime @@ -607,7 +607,7 @@ class EmAgentSpec ) // expect correct results - resultListener.expectMessageType[FlexOptionsResultEvent] match { + resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(flexResult) => flexResult.getInputModel shouldBe emInput.getUuid flexResult.getTime shouldBe 300.toDateTime @@ -616,7 +616,7 @@ class EmAgentSpec flexResult.getpMax() should equalWithTolerance(.008.asMegaWatt) } - resultListener.expectMessageType[ParticipantResultEvent] match { + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid emResult.getTime shouldBe 300.toDateTime @@ -632,8 +632,7 @@ class EmAgentSpec "An EM-controlled EM agent" should { "be initialized correctly and run through some activations" in { - val resultListener = TestProbe[ResultEvent]("ResultListener") - + val resultProxy = TestProbe[ResultEvent]("ResultProxy") val parentEmAgent = TestProbe[FlexResponse]("ParentEmAgent") val emAgent = spawn( @@ -644,7 +643,7 @@ class EmAgentSpec "PRIORITIZED", simulationStart, parent = Right(parentEmAgent.ref), - listener = Iterable(resultListener.ref), + listener = resultProxy.ref, ) ) @@ -685,7 +684,7 @@ class EmAgentSpec ) // expect no results for init - resultListener.expectNoMessage() + resultProxy.expectNoMessage() // expect completion from EmAgent parentEmAgent.expectMessage( FlexCompletion( @@ -723,7 +722,7 @@ class EmAgentSpec ), ) - resultListener.expectMessageType[FlexOptionsResultEvent] match { + resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(flexResult) => flexResult.getInputModel shouldBe emInput.getUuid flexResult.getTime shouldBe 0.toDateTime @@ -780,7 +779,7 @@ class EmAgentSpec ) // expect correct results - resultListener.expectMessageType[ParticipantResultEvent] match { + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid emResult.getTime shouldBe 0.toDateTime @@ -830,7 +829,7 @@ class EmAgentSpec ) // expect correct results - resultListener.expectMessageType[ParticipantResultEvent] match { + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid emResult.getTime shouldBe 150.toDateTime diff --git a/src/test/scala/edu/ie3/simona/agent/grid/DBFSAlgorithmCenGridSpec.scala b/src/test/scala/edu/ie3/simona/agent/grid/DBFSAlgorithmCenGridSpec.scala index a633cb5c4c..cd93c5eea5 100644 --- a/src/test/scala/edu/ie3/simona/agent/grid/DBFSAlgorithmCenGridSpec.scala +++ b/src/test/scala/edu/ie3/simona/agent/grid/DBFSAlgorithmCenGridSpec.scala @@ -25,6 +25,8 @@ import edu.ie3.simona.ontology.messages.{Activation, SchedulerMessage} import edu.ie3.simona.scheduler.ScheduleLock import edu.ie3.simona.service.load.LoadProfileService import edu.ie3.simona.service.primary.PrimaryServiceProxy +import edu.ie3.simona.service.results.ResultServiceProxy +import edu.ie3.simona.service.results.ResultServiceProxy.ExpectResult import edu.ie3.simona.service.weather.WeatherService import edu.ie3.simona.test.common.model.grid.DbfsTestGrid import edu.ie3.simona.test.common.{ConfigTestData, TestSpawnerTyped} @@ -37,6 +39,7 @@ import org.apache.pekko.actor.testkit.typed.scaladsl.{ import squants.electro.Kilovolts import squants.energy.Megawatts +import java.util.UUID import scala.language.postfixOps /** Test to ensure the functions that a [[GridAgent]] in center position should @@ -60,6 +63,7 @@ class DBFSAlgorithmCenGridSpec ) private val primaryService = TestProbe[PrimaryServiceProxy.Message]("primaryService") + private val resultProxy = TestProbe[ResultServiceProxy.Message]("resultProxy") private val weatherService = TestProbe[WeatherService.Message]("weatherService") private val loadProfileService = @@ -85,23 +89,16 @@ class DBFSAlgorithmCenGridSpec scheduler = scheduler.ref, runtimeEventListener = runtimeEvents.ref, primaryServiceProxy = primaryService.ref, + resultProxy = resultProxy.ref, weather = weatherService.ref, loadProfiles = loadProfileService.ref, evDataService = None, ) - val resultListener: TestProbe[ResultEvent] = TestProbe("resultListener") - "A GridAgent actor in center position with async test" should { val centerGridAgent = - testKit.spawn( - GridAgent( - environmentRefs, - simonaConfig, - listener = Iterable(resultListener.ref), - ) - ) + testKit.spawn(GridAgent(environmentRefs, simonaConfig)) s"initialize itself when it receives an init activation" in { @@ -156,6 +153,17 @@ class DBFSAlgorithmCenGridSpec // send the start grid simulation trigger centerGridAgent ! Activation(3600) + resultProxy.expectMessageType[ExpectResult] match { + case ExpectResult(assets, tick) => + assets match { + case uuids: Seq[UUID] => + uuids.toSet shouldBe assetsHv.toSet + case uuid: UUID => + fail(s"Received uuid $uuid, but expected grid asset uuids.") + } + tick shouldBe 3600 + } + /* We expect one grid power request message per inferior grid */ val firstPowerRequestSender11 = inferiorGrid11.expectGridPowerRequest() @@ -441,7 +449,7 @@ class DBFSAlgorithmCenGridSpec // after all grids have received a FinishGridSimulationTrigger, the scheduler should receive a Completion scheduler.expectMessageType[Completion].newTick shouldBe Some(7200) - val resultMessage = resultListener.expectMessageType[ResultEvent] + val resultMessage = resultProxy.expectMessageType[ResultEvent] resultMessage match { case powerFlowResultEvent: PowerFlowResultEvent => // we expect results for 4 nodes, 5 lines and 2 transformer2ws diff --git a/src/test/scala/edu/ie3/simona/agent/grid/DBFSAlgorithmFailedPowerFlowSpec.scala b/src/test/scala/edu/ie3/simona/agent/grid/DBFSAlgorithmFailedPowerFlowSpec.scala index ad6e11f740..fd3cea9f00 100644 --- a/src/test/scala/edu/ie3/simona/agent/grid/DBFSAlgorithmFailedPowerFlowSpec.scala +++ b/src/test/scala/edu/ie3/simona/agent/grid/DBFSAlgorithmFailedPowerFlowSpec.scala @@ -14,7 +14,7 @@ import edu.ie3.simona.agent.grid.GridAgentMessages.Responses.{ ExchangePower, ExchangeVoltage, } -import edu.ie3.simona.event.{ResultEvent, RuntimeEvent} +import edu.ie3.simona.event.RuntimeEvent import edu.ie3.simona.model.grid.{RefSystem, VoltageLimits} import edu.ie3.simona.ontology.messages.SchedulerMessage.{ Completion, @@ -24,6 +24,8 @@ import edu.ie3.simona.ontology.messages.{Activation, SchedulerMessage} import edu.ie3.simona.scheduler.ScheduleLock import edu.ie3.simona.service.load.LoadProfileService import edu.ie3.simona.service.primary.PrimaryServiceProxy +import edu.ie3.simona.service.results.ResultServiceProxy +import edu.ie3.simona.service.results.ResultServiceProxy.ExpectResult import edu.ie3.simona.service.weather.WeatherService import edu.ie3.simona.test.common.model.grid.DbfsTestGrid import edu.ie3.simona.test.common.{ConfigTestData, TestSpawnerTyped} @@ -38,6 +40,7 @@ import org.apache.pekko.actor.typed.scaladsl.adapter.TypedActorRefOps import squants.electro.Kilovolts import squants.energy.Megawatts +import java.util.UUID import scala.concurrent.duration.DurationInt import scala.language.postfixOps @@ -54,6 +57,7 @@ class DBFSAlgorithmFailedPowerFlowSpec ) private val primaryService = TestProbe[PrimaryServiceProxy.Message]("primaryService") + private val resultProxy = TestProbe[ResultServiceProxy.Message]("resultProxy") private val weatherService = TestProbe[WeatherService.Message]("weatherService") private val loadProfileService = @@ -71,26 +75,19 @@ class DBFSAlgorithmFailedPowerFlowSpec scheduler = scheduler.ref, runtimeEventListener = runtimeEvents.ref, primaryServiceProxy = primaryService.ref, + resultProxy = resultProxy.ref, weather = weatherService.ref, loadProfiles = loadProfileService.ref, evDataService = None, ) - val resultListener: TestProbe[ResultEvent] = TestProbe("resultListener") - "A GridAgent actor in center position with async test" should { // since the grid agent is stopped after a failed power flow // we need to initialize the agent for each test def initAndGoToSimulateGrid: ActorRef[GridAgent.Message] = { val centerGridAgent = - testKit.spawn( - GridAgent( - environmentRefs, - simonaConfig, - listener = Iterable(resultListener.ref), - ) - ) + testKit.spawn(GridAgent(environmentRefs, simonaConfig)) // this subnet has 1 superior grid (ehv) and 3 inferior grids (mv). Map the gates to test probes accordingly val subGridGateToActorRef = hvSubGridGatesPF.map { @@ -143,6 +140,18 @@ class DBFSAlgorithmFailedPowerFlowSpec // send the start grid simulation trigger centerGridAgent ! Activation(3600) + + resultProxy.expectMessageType[ExpectResult] match { + case ExpectResult(assets, tick) => + assets match { + case uuids: Seq[UUID] => + uuids.toSet shouldBe assetsHvPF.toSet + case uuid: UUID => + fail(s"Received uuid $uuid, but expected grid asset uuids.") + } + tick shouldBe 3600 + } + // we expect a request for grid power values here for sweepNo $sweepNo val powerRequestSender = inferiorGridAgent.expectGridPowerRequest() @@ -213,7 +222,7 @@ class DBFSAlgorithmFailedPowerFlowSpec // after all grids have received a FinishGridSimulationTrigger, the scheduler should receive a Completion scheduler.expectMessageType[Completion].newTick shouldBe Some(7200) - resultListener.expectNoMessage() + resultProxy.expectNoMessage() // PowerFlowFailed events are only sent by the slack subgrid runtimeEvents.expectNoMessage() @@ -227,6 +236,17 @@ class DBFSAlgorithmFailedPowerFlowSpec // send the start grid simulation trigger centerGridAgent ! Activation(3600) + resultProxy.expectMessageType[ExpectResult] match { + case ExpectResult(assets, tick) => + assets match { + case uuids: Seq[UUID] => + uuids.toSet shouldBe assetsHvPF.toSet + case uuid: UUID => + fail(s"Received uuid $uuid, but expected grid asset uuids.") + } + tick shouldBe 3600 + } + // we expect a request for grid power values here for sweepNo 0 val powerRequestSender = inferiorGridAgent.expectGridPowerRequest() @@ -285,7 +305,7 @@ class DBFSAlgorithmFailedPowerFlowSpec // after all grids have received a FinishGridSimulationTrigger, the scheduler should receive a Completion scheduler.expectMessageType[Completion].newTick shouldBe Some(7200) - resultListener.expectNoMessage() + resultProxy.expectNoMessage() // PowerFlowFailed events are only sent by the slack subgrid runtimeEvents.expectNoMessage() @@ -302,7 +322,6 @@ class DBFSAlgorithmFailedPowerFlowSpec GridAgent( environmentRefs, simonaConfig, // stopOnFailure is enabled - listener = Iterable(resultListener.ref), ) ) @@ -341,6 +360,17 @@ class DBFSAlgorithmFailedPowerFlowSpec // send the start grid simulation trigger slackGridAgent ! Activation(3600) + resultProxy.expectMessageType[ExpectResult] match { + case ExpectResult(assets, tick) => + assets match { + case uuids: Seq[UUID] => + uuids.toSet shouldBe assetsEhv.toSet + case uuid: UUID => + fail(s"Received uuid $uuid, but expected grid asset uuids.") + } + tick shouldBe 3600 + } + val powerRequestSender = hvGridAgent.expectGridPowerRequest() // normally the inferior grid agents ask for the slack voltage as well to run their power flow calculation @@ -376,7 +406,7 @@ class DBFSAlgorithmFailedPowerFlowSpec hvGridAgent.gaProbe.expectNoMessage() scheduler.expectNoMessage() - resultListener.expectNoMessage() + resultProxy.expectNoMessage() } } diff --git a/src/test/scala/edu/ie3/simona/agent/grid/DBFSAlgorithmParticipantSpec.scala b/src/test/scala/edu/ie3/simona/agent/grid/DBFSAlgorithmParticipantSpec.scala index cb6c89a22e..ebe77c7411 100644 --- a/src/test/scala/edu/ie3/simona/agent/grid/DBFSAlgorithmParticipantSpec.scala +++ b/src/test/scala/edu/ie3/simona/agent/grid/DBFSAlgorithmParticipantSpec.scala @@ -26,6 +26,7 @@ import edu.ie3.simona.ontology.messages.{Activation, SchedulerMessage} import edu.ie3.simona.scheduler.ScheduleLock import edu.ie3.simona.service.load.LoadProfileService import edu.ie3.simona.service.primary.PrimaryServiceProxy +import edu.ie3.simona.service.results.ResultServiceProxy import edu.ie3.simona.service.weather.WeatherService import edu.ie3.simona.test.common.model.grid.DbfsTestGridWithParticipants import edu.ie3.simona.test.common.{ConfigTestData, TestSpawnerTyped} @@ -53,6 +54,7 @@ class DBFSAlgorithmParticipantSpec TestProbe("runtimeEvents") private val primaryService = TestProbe[PrimaryServiceProxy.Message]("primaryService") + private val resultProxy = TestProbe[ResultServiceProxy.Message]("resultProxy") private val weatherService = TestProbe[WeatherService.Message]("weatherService") private val loadProfileService = @@ -62,14 +64,12 @@ class DBFSAlgorithmParticipantSpec scheduler = scheduler.ref, runtimeEventListener = runtimeEvents.ref, primaryServiceProxy = primaryService.ref, + resultProxy = resultProxy.ref, weather = weatherService.ref, loadProfiles = loadProfileService.ref, evDataService = None, ) - protected val resultListener: TestProbe[ResultEvent] = - TestProbe("resultListener") - private val superiorGridAgent = SuperiorGA( TestProbe("superiorGridAgent_1000"), Seq(supNodeA.getUuid), @@ -80,7 +80,6 @@ class DBFSAlgorithmParticipantSpec GridAgent( environmentRefs, simonaConfig, - Iterable(resultListener.ref), ) ) diff --git a/src/test/scala/edu/ie3/simona/agent/grid/DBFSAlgorithmSupGridSpec.scala b/src/test/scala/edu/ie3/simona/agent/grid/DBFSAlgorithmSupGridSpec.scala index e24b668a95..8d94e25d11 100644 --- a/src/test/scala/edu/ie3/simona/agent/grid/DBFSAlgorithmSupGridSpec.scala +++ b/src/test/scala/edu/ie3/simona/agent/grid/DBFSAlgorithmSupGridSpec.scala @@ -23,6 +23,8 @@ import edu.ie3.simona.ontology.messages.{Activation, SchedulerMessage} import edu.ie3.simona.scheduler.ScheduleLock import edu.ie3.simona.service.load.LoadProfileService import edu.ie3.simona.service.primary.PrimaryServiceProxy +import edu.ie3.simona.service.results.ResultServiceProxy +import edu.ie3.simona.service.results.ResultServiceProxy.ExpectResult import edu.ie3.simona.service.weather.WeatherService import edu.ie3.simona.test.common.model.grid.DbfsTestGrid import edu.ie3.simona.test.common.{ConfigTestData, TestSpawnerTyped, UnitSpec} @@ -55,6 +57,7 @@ class DBFSAlgorithmSupGridSpec private val runtimeEvents = TestProbe[RuntimeEvent]("runtimeEvents") private val primaryService = TestProbe[PrimaryServiceProxy.Message]("primaryService") + private val resultProxy = TestProbe[ResultServiceProxy.Message]("resultProxy") private val weatherService = TestProbe[WeatherService.Message]("weatherService") private val loadProfileService = @@ -65,19 +68,17 @@ class DBFSAlgorithmSupGridSpec scheduler = scheduler.ref, runtimeEventListener = runtimeEvents.ref, primaryServiceProxy = primaryService.ref, + resultProxy = resultProxy.ref, weather = weatherService.ref, loadProfiles = loadProfileService.ref, evDataService = None, ) - val resultListener: TestProbe[ResultEvent] = TestProbe("resultListener") - "A GridAgent actor in superior position with async test" should { val superiorGridAgentFSM: ActorRef[GridAgent.Message] = testKit.spawn( GridAgent( environmentRefs, simonaConfig, - listener = Iterable(resultListener.ref), ) ) @@ -161,8 +162,9 @@ class DBFSAlgorithmSupGridSpec case Completion(_, Some(3600)) => // we expect another completion message when the agent is in SimulateGrid again case Completion(_, Some(7200)) => + resultProxy.expectMessageType[ExpectResult] // agent should be in Idle again and listener should contain power flow result data - val resultMessage = resultListener.expectMessageType[ResultEvent] + val resultMessage = resultProxy.expectMessageType[ResultEvent] resultMessage match { case powerFlowResultEvent: PowerFlowResultEvent => @@ -281,9 +283,9 @@ class DBFSAlgorithmSupGridSpec // when we received a FinishGridSimulationTrigger (as inferior grid agent) // we expect another completion message then as well (scheduler view) case Completion(_, Some(7200)) => + resultProxy.expectMessageType[ExpectResult] // after doing cleanup stuff, our agent should go back to idle again and listener should contain power flow result data - val resultMessage = - resultListener.expectMessageType[ResultEvent] + val resultMessage = resultProxy.expectMessageType[ResultEvent] resultMessage match { case powerFlowResultEvent: PowerFlowResultEvent => diff --git a/src/test/scala/edu/ie3/simona/agent/grid/GridAgentSetupSpec.scala b/src/test/scala/edu/ie3/simona/agent/grid/GridAgentSetupSpec.scala index 39d0b02811..1c9f2657f8 100644 --- a/src/test/scala/edu/ie3/simona/agent/grid/GridAgentSetupSpec.scala +++ b/src/test/scala/edu/ie3/simona/agent/grid/GridAgentSetupSpec.scala @@ -39,7 +39,6 @@ class GridAgentSetupSpec gridContainer.getSubGridTopologyGraph, ctx, mock[EnvironmentRefs], - Seq.empty, ) Behaviors.stopped @@ -66,7 +65,6 @@ class GridAgentSetupSpec threeWindingTestGrid.getSubGridTopologyGraph, ctx, mock[EnvironmentRefs], - Seq.empty, ) Behaviors.stopped diff --git a/src/test/scala/edu/ie3/simona/agent/grid/ThermalGridIT.scala b/src/test/scala/edu/ie3/simona/agent/grid/ThermalGridIT.scala index 51148bfda6..f5348044c5 100644 --- a/src/test/scala/edu/ie3/simona/agent/grid/ThermalGridIT.scala +++ b/src/test/scala/edu/ie3/simona/agent/grid/ThermalGridIT.scala @@ -31,10 +31,12 @@ import edu.ie3.simona.ontology.messages.ServiceMessage.{ SecondaryServiceRegistrationMessage, } import edu.ie3.simona.ontology.messages.{Activation, SchedulerMessage} +import edu.ie3.simona.ontology.messages.ResultMessage.RequestResult import edu.ie3.simona.scheduler.ScheduleLock import edu.ie3.simona.service.Data.SecondaryData.WeatherData import edu.ie3.simona.service.ServiceType import edu.ie3.simona.service.primary.PrimaryServiceProxy +import edu.ie3.simona.service.results.ResultServiceProxy.ExpectResult import edu.ie3.simona.service.weather.WeatherService.WeatherRegistrationData import edu.ie3.simona.service.weather.{WeatherDataType, WeatherService} import edu.ie3.simona.test.common.input.EmInputTestData @@ -110,7 +112,8 @@ class ThermalGridIT ) val gridAgent = TestProbe[GridAgent.Message]("GridAgent") - val resultListener = TestProbe[ResultEvent]("ResultListener") + val resultServiceProxy = + TestProbe[ResultEvent | ExpectResult]("ResultProxy") val scheduler: TestProbe[SchedulerMessage] = TestProbe("scheduler") val primaryServiceProxy = TestProbe[PrimaryServiceProxy.Message]("PrimaryServiceProxy") @@ -119,8 +122,8 @@ class ThermalGridIT given ParticipantRefs = ParticipantRefs( gridAgent = gridAgent.ref, primaryServiceProxy = primaryServiceProxy.ref, + resultServiceProxy = resultServiceProxy.ref, services = Map(ServiceType.WeatherService -> weatherService.ref), - resultListener = Iterable(resultListener.ref), ) val key = ScheduleLock.singleKey(TSpawner, scheduler.ref, PRE_INIT_TICK) @@ -179,7 +182,7 @@ class ThermalGridIT val weatherDependentAgents = Seq(hpAgent) scheduler.expectMessage(Completion(heatPumpAgent, Some(0))) - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() /* TICK 0 Start of Simulation @@ -205,7 +208,7 @@ class ThermalGridIT Range(0, 3) .map { _ => - resultListener.expectMessageType[ResultEvent] + resultServiceProxy.expectMessageType[ResultEvent] } .foreach { case ParticipantResultEvent(participantResult) => @@ -240,7 +243,7 @@ class ThermalGridIT energy should equalWithTolerance(0.asMegaWattHour) } } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(heatPumpAgent, Some(3416))) /* TICK 3416 @@ -253,7 +256,7 @@ class ThermalGridIT Range(0, 3) .map { _ => - resultListener.expectMessageType[ResultEvent] + resultServiceProxy.expectMessageType[ResultEvent] } .foreach { case ParticipantResultEvent(participantResult) => @@ -290,7 +293,7 @@ class ThermalGridIT energy should equalWithTolerance(0.01044.asMegaWattHour) } } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(heatPumpAgent, Some(3600))) /* TICK 3600 @@ -315,7 +318,7 @@ class ThermalGridIT ) } - resultListener.expectMessageType[ParticipantResultEvent] match { + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(hpResult) => hpResult.getInputModel shouldBe typicalHpInputModel.getUuid hpResult.getTime shouldBe 3600.toDateTime @@ -328,7 +331,7 @@ class ThermalGridIT // Since this activation is caused by new weather data, we don't expect any // message for house or heat storage since there is no change of their operating // point nor one of it reached any boundary. - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(heatPumpAgent, Some(4412))) /* TICK 4412 @@ -341,7 +344,7 @@ class ThermalGridIT Range(0, 2) .map { _ => - resultListener.expectMessageType[ResultEvent] + resultServiceProxy.expectMessageType[ResultEvent] } .foreach { case ParticipantResultEvent(participantResult) => @@ -368,7 +371,7 @@ class ThermalGridIT ) } } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(heatPumpAgent, Some(21600))) /* TICK 21600 @@ -394,7 +397,7 @@ class ThermalGridIT ) } - resultListener.expectMessageType[ParticipantResultEvent] match { + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(hpResult) => hpResult.getInputModel shouldBe typicalHpInputModel.getUuid hpResult.getTime shouldBe 21600.toDateTime @@ -405,7 +408,7 @@ class ThermalGridIT // Since this activation is caused by new weather data, we don't expect any // message for house or heat storage since there is no change of their operating // point nor one of it reached any boundary. - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(heatPumpAgent, Some(23288))) /* TICK 23288 @@ -418,7 +421,7 @@ class ThermalGridIT Range(0, 3) .map { _ => - resultListener.expectMessageType[ResultEvent] + resultServiceProxy.expectMessageType[ResultEvent] } .foreach { case ParticipantResultEvent(participantResult) => @@ -455,7 +458,7 @@ class ThermalGridIT energy should equalWithTolerance(0.01044.asMegaWattHour) } } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(heatPumpAgent, Some(25000))) /* TICK 25000 @@ -481,7 +484,7 @@ class ThermalGridIT ) } - resultListener.expectMessageType[ParticipantResultEvent] match { + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(hpResult) => hpResult.getInputModel shouldBe typicalHpInputModel.getUuid hpResult.getTime shouldBe 25000.toDateTime @@ -492,7 +495,7 @@ class ThermalGridIT // Since this activation is caused by new weather data, we don't expect any // message for house or heat storage since there is no change of their operating // point nor one of it reached any boundary. - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(heatPumpAgent, Some(26887))) /* TICK 26887 @@ -505,7 +508,7 @@ class ThermalGridIT Range(0, 3) .map { _ => - resultListener.expectMessageType[ResultEvent] + resultServiceProxy.expectMessageType[ResultEvent] } .foreach { case ParticipantResultEvent(participantResult) => @@ -542,7 +545,7 @@ class ThermalGridIT energy should equalWithTolerance(0.asMegaWattHour) } } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(heatPumpAgent, Some(28000))) /* TICK 28000 @@ -566,7 +569,7 @@ class ThermalGridIT Some(151200), ) } - resultListener.expectMessageType[ParticipantResultEvent] match { + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(hpResult) => hpResult.getInputModel shouldBe typicalHpInputModel.getUuid hpResult.getTime shouldBe 28000.toDateTime @@ -577,7 +580,7 @@ class ThermalGridIT // Since this activation is caused by new weather data, we don't expect any // message for house or storage since there is no change of their operating // point nor one of it reached any boundary. - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(heatPumpAgent, Some(32043))) /* TICK 32043 @@ -590,7 +593,7 @@ class ThermalGridIT Range(0, 3) .map { _ => - resultListener.expectMessageType[ResultEvent] + resultServiceProxy.expectMessageType[ResultEvent] } .foreach { case ParticipantResultEvent(participantResult) => @@ -627,7 +630,7 @@ class ThermalGridIT energy should equalWithTolerance(0.asMegaWattHour) } } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(heatPumpAgent, Some(35459))) /* TICK 35459 @@ -640,7 +643,7 @@ class ThermalGridIT Range(0, 3) .map { _ => - resultListener.expectMessageType[ResultEvent] + resultServiceProxy.expectMessageType[ResultEvent] } .foreach { case ParticipantResultEvent(participantResult) => @@ -677,7 +680,7 @@ class ThermalGridIT energy should equalWithTolerance(0.01044.asMegaWattHour) } } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(heatPumpAgent, Some(35995))) /* TICK 35995 @@ -690,7 +693,7 @@ class ThermalGridIT Range(0, 2) .map { _ => - resultListener.expectMessageType[ResultEvent] + resultServiceProxy.expectMessageType[ResultEvent] } .foreach { case ParticipantResultEvent(participantResult) => @@ -715,7 +718,7 @@ class ThermalGridIT indoorTemperature should equalWithTolerance(20.asDegreeCelsius) } } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(heatPumpAgent, Some(74629))) } } @@ -735,7 +738,8 @@ class ThermalGridIT ) val gridAgent = TestProbe[GridAgent.Message]("GridAgent") - val resultListener: TestProbe[ResultEvent] = TestProbe("resultListener") + val resultServiceProxy: TestProbe[ResultEvent | ExpectResult] = + TestProbe("resultServiceProxy") val scheduler: TestProbe[SchedulerMessage] = TestProbe("scheduler") val primaryServiceProxy = TestProbe[PrimaryServiceProxy.Message]("PrimaryServiceProxy") @@ -744,8 +748,8 @@ class ThermalGridIT given ParticipantRefs = ParticipantRefs( gridAgent = gridAgent.ref, primaryServiceProxy = primaryServiceProxy.ref, + resultServiceProxy = resultServiceProxy.ref, services = Map(ServiceType.WeatherService -> weatherService.ref), - resultListener = Iterable(resultListener.ref), ) val keys = ScheduleLock @@ -763,7 +767,7 @@ class ThermalGridIT "PRIORITIZED", simulationStartWithPv, parent = Left(scheduler.ref), - listener = Iterable(resultListener.ref), + listener = resultServiceProxy.ref, ), "EmAgent", ) @@ -860,7 +864,7 @@ class ThermalGridIT weatherService.ref, 0L, ) - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(emAgentActivation, Some(0))) val weatherDependentAgents = Seq(hpAgent.toClassic, pvAgent.toClassic) @@ -890,7 +894,7 @@ class ThermalGridIT Range(0, 4) .map { _ => - resultListener.expectMessageType[ResultEvent] + resultServiceProxy.expectMessageType[ResultEvent] } .foreach { case ParticipantResultEvent(participantResult) => @@ -930,7 +934,7 @@ class ThermalGridIT energy should equalWithTolerance(0.asMegaWattHour) } } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(emAgentActivation, Some(1800))) /* TICK 1800 @@ -958,7 +962,7 @@ class ThermalGridIT Range(0, 3) .map { _ => - resultListener.expectMessageType[ResultEvent] + resultServiceProxy.expectMessageType[ResultEvent] } .foreach { case ParticipantResultEvent(participantResult) => @@ -993,7 +997,7 @@ class ThermalGridIT energy should equalWithTolerance(0.asMegaWattHour) } } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(emAgentActivation, Some(5216))) /* TICK 5216 @@ -1007,7 +1011,7 @@ class ThermalGridIT Range(0, 4) .map { _ => - resultListener.expectMessageType[ResultEvent] + resultServiceProxy.expectMessageType[ResultEvent] } .foreach { case ParticipantResultEvent(participantResult) => @@ -1050,7 +1054,7 @@ class ThermalGridIT energy should equalWithTolerance(0.01044.asMegaWattHour) } } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(emAgentActivation, Some(5400))) /* TICK 5400 @@ -1078,7 +1082,7 @@ class ThermalGridIT Range(0, 4) .map { _ => - resultListener.expectMessageType[ResultEvent] + resultServiceProxy.expectMessageType[ResultEvent] } .foreach { case ParticipantResultEvent(participantResult) => @@ -1120,7 +1124,7 @@ class ThermalGridIT energy should equalWithTolerance(0.01044.asMegaWattHour) } } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(emAgentActivation, Some(6824))) /* TICK 6824 @@ -1134,7 +1138,7 @@ class ThermalGridIT Range(0, 4) .map { _ => - resultListener.expectMessageType[ResultEvent] + resultServiceProxy.expectMessageType[ResultEvent] } .foreach { case ParticipantResultEvent(participantResult) => @@ -1176,7 +1180,7 @@ class ThermalGridIT energy should equalWithTolerance(0.0063104.asMegaWattHour) } } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(emAgentActivation, Some(9200))) /* TICK 9200 @@ -1205,7 +1209,7 @@ class ThermalGridIT Range(0, 3) .map { _ => - resultListener.expectMessageType[ResultEvent] + resultServiceProxy.expectMessageType[ResultEvent] } .foreach { case ParticipantResultEvent(participantResult) => @@ -1242,7 +1246,7 @@ class ThermalGridIT ) } } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(emAgentActivation, Some(10551))) /* TICK 10551 @@ -1257,7 +1261,7 @@ class ThermalGridIT Range(0, 4) .map { _ => - resultListener.expectMessageType[ResultEvent] + resultServiceProxy.expectMessageType[ResultEvent] } .foreach { case ParticipantResultEvent(participantResult) => @@ -1299,7 +1303,7 @@ class ThermalGridIT energy should equalWithTolerance(0.01044.asMegaWattHour) } } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(emAgentActivation, Some(11638))) /* TICK 11638 @@ -1313,7 +1317,7 @@ class ThermalGridIT Range(0, 3) .map { _ => - resultListener.expectMessageType[ResultEvent] + resultServiceProxy.expectMessageType[ResultEvent] } .foreach { case ParticipantResultEvent(participantResult) => @@ -1345,7 +1349,7 @@ class ThermalGridIT ) } } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(emAgentActivation, Some(12000))) /* TICK 12000 @@ -1374,7 +1378,7 @@ class ThermalGridIT Range(0, 3) .map { _ => - resultListener.expectMessageType[ResultEvent] + resultServiceProxy.expectMessageType[ResultEvent] } .foreach { case ParticipantResultEvent(participantResult) => @@ -1406,7 +1410,7 @@ class ThermalGridIT ) } } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(emAgentActivation, Some(12139))) /* TICK 12139 @@ -1420,7 +1424,7 @@ class ThermalGridIT Range(0, 3) .map { _ => - resultListener.expectMessageType[ResultEvent] + resultServiceProxy.expectMessageType[ResultEvent] } .foreach { case ParticipantResultEvent(participantResult) => @@ -1450,7 +1454,7 @@ class ThermalGridIT indoorTemperature should equalWithTolerance(20.asDegreeCelsius) } } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(emAgentActivation, Some(12500))) /* TICK 12500 @@ -1479,7 +1483,7 @@ class ThermalGridIT Range(0, 2) .map { _ => - resultListener.expectMessageType[ResultEvent] + resultServiceProxy.expectMessageType[ResultEvent] } .foreach { case ParticipantResultEvent(participantResult) => participantResult match { @@ -1499,7 +1503,7 @@ class ThermalGridIT ) } } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(emAgentActivation, Some(24413))) /* TICK 24413 @@ -1513,7 +1517,7 @@ class ThermalGridIT Range(0, 4) .map { _ => - resultListener.expectMessageType[ResultEvent] + resultServiceProxy.expectMessageType[ResultEvent] } .foreach { case ParticipantResultEvent(participantResult) => @@ -1553,7 +1557,7 @@ class ThermalGridIT energy should equalWithTolerance(0.01044.asMegaWattHour) } } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(emAgentActivation, Some(25200))) /* TICK 25200 @@ -1581,7 +1585,7 @@ class ThermalGridIT Range(0, 4) .map { _ => - resultListener.expectMessageType[ResultEvent] + resultServiceProxy.expectMessageType[ResultEvent] } .foreach { case ParticipantResultEvent(participantResult) => @@ -1629,7 +1633,7 @@ class ThermalGridIT ) } } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(emAgentActivation, Some(27500))) /* TICK 27500 @@ -1657,7 +1661,7 @@ class ThermalGridIT Range(0, 2) .map { _ => - resultListener.expectMessageType[ResultEvent] + resultServiceProxy.expectMessageType[ResultEvent] } .foreach { case ParticipantResultEvent(participantResult) => participantResult match { @@ -1677,7 +1681,7 @@ class ThermalGridIT ) } } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(emAgentActivation, Some(30923))) /* TICK 30923 @@ -1691,7 +1695,7 @@ class ThermalGridIT Range(0, 4) .map { _ => - resultListener.expectMessageType[ResultEvent] + resultServiceProxy.expectMessageType[ResultEvent] } .foreach { case ParticipantResultEvent(participantResult) => @@ -1738,7 +1742,7 @@ class ThermalGridIT ) } } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(emAgentActivation, Some(31000))) /* TICK 31000 @@ -1766,7 +1770,7 @@ class ThermalGridIT Range(0, 3) .map { _ => - resultListener.expectMessageType[ResultEvent] + resultServiceProxy.expectMessageType[ResultEvent] } .foreach { case ParticipantResultEvent(participantResult) => @@ -1802,7 +1806,7 @@ class ThermalGridIT ) } } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(emAgentActivation, Some(40964))) /* TICK 40964 @@ -1816,7 +1820,7 @@ class ThermalGridIT Range(0, 4) .map { _ => - resultListener.expectMessageType[ResultEvent] + resultServiceProxy.expectMessageType[ResultEvent] } .foreach { case ParticipantResultEvent(participantResult) => @@ -1858,7 +1862,7 @@ class ThermalGridIT ) } } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(emAgentActivation, Some(43858))) /* TICK 43858 @@ -1876,7 +1880,7 @@ class ThermalGridIT Range(0, 4) .map { _ => - resultListener.expectMessageType[ResultEvent] + resultServiceProxy.expectMessageType[ResultEvent] } .foreach { case ParticipantResultEvent(participantResult) => @@ -1919,7 +1923,7 @@ class ThermalGridIT energy should equalWithTolerance(0.asMegaWattHour) } } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(emAgentActivation, Some(46635))) /* TICK 46635 @@ -1933,7 +1937,7 @@ class ThermalGridIT Range(0, 3) .map { _ => - resultListener.expectMessageType[ResultEvent] + resultServiceProxy.expectMessageType[ResultEvent] } .foreach { case ParticipantResultEvent(participantResult) => @@ -1963,7 +1967,7 @@ class ThermalGridIT indoorTemperature should equalWithTolerance(18.asDegreeCelsius) } } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(emAgentActivation, Some(56278))) /* TICK 56278 @@ -1977,7 +1981,7 @@ class ThermalGridIT Range(0, 3) .map { _ => - resultListener.expectMessageType[ResultEvent] + resultServiceProxy.expectMessageType[ResultEvent] } .foreach { case ParticipantResultEvent(participantResult) => @@ -2007,7 +2011,7 @@ class ThermalGridIT indoorTemperature should equalWithTolerance(20.asDegreeCelsius) } } - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(emAgentActivation, Some(66279))) } } diff --git a/src/test/scala/edu/ie3/simona/agent/grid/congestion/CongestionTestBaseData.scala b/src/test/scala/edu/ie3/simona/agent/grid/congestion/CongestionTestBaseData.scala index c6e8383ef1..37d370c915 100644 --- a/src/test/scala/edu/ie3/simona/agent/grid/congestion/CongestionTestBaseData.scala +++ b/src/test/scala/edu/ie3/simona/agent/grid/congestion/CongestionTestBaseData.scala @@ -19,6 +19,7 @@ import edu.ie3.simona.model.grid.RefSystem import edu.ie3.simona.ontology.messages.SchedulerMessage import edu.ie3.simona.service.load.LoadProfileService import edu.ie3.simona.service.primary.PrimaryServiceProxy +import edu.ie3.simona.service.results.ResultServiceProxy import edu.ie3.simona.service.weather.WeatherService import edu.ie3.simona.test.common.result.CongestedComponentsTestData import edu.ie3.simona.test.common.{ConfigTestData, TestSpawnerTyped} @@ -66,6 +67,9 @@ trait CongestionTestBaseData TestProbe( "primaryService" ) + protected val resultProxy: TestProbe[ResultServiceProxy.Message] = TestProbe( + "resultServiceProxy" + ) protected val weatherService: TestProbe[WeatherService.Message] = TestProbe( "weatherService" ) @@ -78,20 +82,16 @@ trait CongestionTestBaseData scheduler = scheduler.ref, runtimeEventListener = runtimeEvents.ref, primaryServiceProxy = primaryService.ref, + resultProxy = resultProxy.ref, weather = weatherService.ref, loadProfiles = loadProfileService.ref, evDataService = None, ) - protected val resultListener: TestProbe[ResultEvent] = TestProbe( - "resultListener" - ) - protected implicit val constantData: GridAgentConstantData = GridAgentConstantData( environmentRefs, simonaConfig, - Iterable(resultListener.ref), 3600, startTime, endTime, diff --git a/src/test/scala/edu/ie3/simona/agent/grid/congestion/DCMAlgorithmSpec.scala b/src/test/scala/edu/ie3/simona/agent/grid/congestion/DCMAlgorithmSpec.scala index fde0c5b5ca..9e925102ee 100644 --- a/src/test/scala/edu/ie3/simona/agent/grid/congestion/DCMAlgorithmSpec.scala +++ b/src/test/scala/edu/ie3/simona/agent/grid/congestion/DCMAlgorithmSpec.scala @@ -88,7 +88,7 @@ class DCMAlgorithmSpec ) // we should receive an empty result event - resultListener.expectMessageType[PowerFlowResultEvent] match { + resultProxy.expectMessageType[PowerFlowResultEvent] match { case PowerFlowResultEvent( nodeResults, _, diff --git a/src/test/scala/edu/ie3/simona/agent/participant/ParticipantAgentInitSpec.scala b/src/test/scala/edu/ie3/simona/agent/participant/ParticipantAgentInitSpec.scala index 489f7b2693..f90e3feb4a 100644 --- a/src/test/scala/edu/ie3/simona/agent/participant/ParticipantAgentInitSpec.scala +++ b/src/test/scala/edu/ie3/simona/agent/participant/ParticipantAgentInitSpec.scala @@ -36,6 +36,7 @@ import edu.ie3.simona.scheduler.ScheduleLock import edu.ie3.simona.service.Data.PrimaryData.ActivePowerExtra import edu.ie3.simona.service.ServiceType import edu.ie3.simona.service.primary.PrimaryServiceProxy +import edu.ie3.simona.service.results.ResultServiceProxy.ExpectResult import edu.ie3.simona.service.weather.WeatherDataType import edu.ie3.simona.service.weather.WeatherService.WeatherRegistrationData import edu.ie3.simona.test.common.input.{LoadInputTestData, PvInputTestData} @@ -94,13 +95,13 @@ class ParticipantAgentInitSpec val gridAgent = createTestProbe[GridAgent.Message]() val primaryService = createTestProbe[PrimaryServiceProxy.Message]() - val resultListener = createTestProbe[ResultEvent]() + val resultServiceProxy = createTestProbe[ResultEvent | ExpectResult]() given ParticipantRefs = ParticipantRefs( gridAgent = gridAgent.ref, primaryServiceProxy = primaryService.ref, + resultServiceProxy = resultServiceProxy.ref, services = Map.empty, - resultListener = Iterable(resultListener.ref), ) val key = ScheduleLock.singleKey(TSpawner, scheduler.ref, PRE_INIT_TICK) @@ -143,13 +144,13 @@ class ParticipantAgentInitSpec val gridAgent = createTestProbe[GridAgent.Message]() val primaryService = createTestProbe[Any]() - val resultListener = createTestProbe[ResultEvent]() + val resultServiceProxy = createTestProbe[ResultEvent | ExpectResult]() given ParticipantRefs = ParticipantRefs( gridAgent = gridAgent.ref, primaryServiceProxy = primaryService.ref, + resultServiceProxy = resultServiceProxy.ref, services = Map.empty, - resultListener = Iterable(resultListener.ref), ) val key = ScheduleLock.singleKey(TSpawner, scheduler.ref, PRE_INIT_TICK) @@ -201,13 +202,13 @@ class ParticipantAgentInitSpec val gridAgent = createTestProbe[GridAgent.Message]() val primaryService = createTestProbe[Any]() - val resultListener = createTestProbe[ResultEvent]() + val resultServiceProxy = createTestProbe[ResultEvent | ExpectResult]() given ParticipantRefs = ParticipantRefs( gridAgent = gridAgent.ref, primaryServiceProxy = primaryService.ref, + resultServiceProxy = resultServiceProxy.ref, services = Map.empty, - resultListener = Iterable(resultListener.ref), ) val key = ScheduleLock.singleKey(TSpawner, scheduler.ref, PRE_INIT_TICK) @@ -264,13 +265,13 @@ class ParticipantAgentInitSpec val gridAgent = createTestProbe[GridAgent.Message]() val primaryService = createTestProbe[Any]() - val resultListener = createTestProbe[ResultEvent]() + val resultServiceProxy = createTestProbe[ResultEvent | ExpectResult]() given ParticipantRefs = ParticipantRefs( gridAgent = gridAgent.ref, primaryServiceProxy = primaryService.ref, + resultServiceProxy = resultServiceProxy.ref, services = Map.empty, - resultListener = Iterable(resultListener.ref), ) val key = ScheduleLock.singleKey(TSpawner, scheduler.ref, PRE_INIT_TICK) @@ -351,14 +352,14 @@ class ParticipantAgentInitSpec val gridAgent = createTestProbe[GridAgent.Message]() val primaryService = createTestProbe[Any]() - val resultListener = createTestProbe[ResultEvent]() + val resultServiceProxy = createTestProbe[ResultEvent | ExpectResult]() val service = createTestProbe[Any]() given ParticipantRefs = ParticipantRefs( gridAgent = gridAgent.ref, primaryServiceProxy = primaryService.ref, + resultServiceProxy = resultServiceProxy.ref, services = Map(ServiceType.WeatherService -> service.ref), - resultListener = Iterable(resultListener.ref), ) val key = ScheduleLock.singleKey(TSpawner, scheduler.ref, PRE_INIT_TICK) @@ -419,14 +420,14 @@ class ParticipantAgentInitSpec val gridAgent = createTestProbe[GridAgent.Message]() val primaryService = createTestProbe[Any]() - val resultListener = createTestProbe[ResultEvent]() + val resultServiceProxy = createTestProbe[ResultEvent | ExpectResult]() val service = createTestProbe[Any]() given ParticipantRefs = ParticipantRefs( gridAgent = gridAgent.ref, primaryServiceProxy = primaryService.ref, + resultServiceProxy = resultServiceProxy.ref, services = Map(ServiceType.WeatherService -> service.ref), - resultListener = Iterable(resultListener.ref), ) val key = ScheduleLock.singleKey(TSpawner, scheduler.ref, PRE_INIT_TICK) @@ -482,14 +483,14 @@ class ParticipantAgentInitSpec val gridAgent = createTestProbe[GridAgent.Message]() val primaryService = createTestProbe[Any]() - val resultListener = createTestProbe[ResultEvent]() + val resultServiceProxy = createTestProbe[ResultEvent | ExpectResult]() val service = createTestProbe[Any]() given ParticipantRefs = ParticipantRefs( gridAgent = gridAgent.ref, primaryServiceProxy = primaryService.ref, + resultServiceProxy = resultServiceProxy.ref, services = Map(ServiceType.WeatherService -> service.ref), - resultListener = Iterable(resultListener.ref), ) val key = ScheduleLock.singleKey(TSpawner, scheduler.ref, PRE_INIT_TICK) @@ -563,14 +564,14 @@ class ParticipantAgentInitSpec val gridAgent = createTestProbe[GridAgent.Message]() val primaryService = createTestProbe[Any]() - val resultListener = createTestProbe[ResultEvent]() + val resultServiceProxy = createTestProbe[ResultEvent | ExpectResult]() val service = createTestProbe[Any]() given ParticipantRefs = ParticipantRefs( gridAgent = gridAgent.ref, primaryServiceProxy = primaryService.ref, + resultServiceProxy = resultServiceProxy.ref, services = Map(ServiceType.WeatherService -> service.ref), - resultListener = Iterable(resultListener.ref), ) val key = ScheduleLock.singleKey(TSpawner, scheduler.ref, PRE_INIT_TICK) diff --git a/src/test/scala/edu/ie3/simona/agent/participant/ParticipantAgentSpec.scala b/src/test/scala/edu/ie3/simona/agent/participant/ParticipantAgentSpec.scala index f511824c62..a2d9161d63 100644 --- a/src/test/scala/edu/ie3/simona/agent/participant/ParticipantAgentSpec.scala +++ b/src/test/scala/edu/ie3/simona/agent/participant/ParticipantAgentSpec.scala @@ -44,6 +44,7 @@ import edu.ie3.simona.ontology.messages.{ ServiceMessage, } import edu.ie3.simona.service.Data.PrimaryData.{ActivePower, ActivePowerExtra} +import edu.ie3.simona.service.results.ResultServiceProxy.ExpectResult import edu.ie3.simona.test.common.UnitSpec import edu.ie3.simona.util.TickUtil.TickLong import edu.ie3.util.TimeUtil @@ -95,7 +96,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { val scheduler = createTestProbe[SchedulerMessage]() val gridAgent = createTestProbe[GridAgent.Message]() - val resultListener = createTestProbe[ResultEvent]() + val resultProxy = createTestProbe[ResultEvent | ExpectResult]() val responseReceiver = createTestProbe[MockResponseMessage]() // no additional activation ticks @@ -117,10 +118,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { expectedRequestTick = 12 * 3600, requestVoltageDeviationTolerance = Each(1e-14), ), - ParticipantResultHandler( - Iterable(resultListener.ref), - notifierConfig, - ), + ParticipantResultHandler(resultProxy.ref, notifierConfig), )(using Left(scheduler.ref)) ) @@ -134,7 +132,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! Activation(8 * 3600) - resultListener.expectMessageType[ParticipantResultEvent] match { + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe (8 * 3600).toDateTime @@ -209,7 +207,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! Activation(20 * 3600) - resultListener.expectMessageType[ParticipantResultEvent] match { + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe (20 * 3600).toDateTime @@ -245,7 +243,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { val scheduler = createTestProbe[SchedulerMessage]() val gridAgent = createTestProbe[GridAgent.Message]() - val resultListener = createTestProbe[ResultEvent]() + val resultProxy = createTestProbe[ResultEvent | ExpectResult]() val responseReceiver = createTestProbe[MockResponseMessage]() // with additional activation ticks @@ -273,10 +271,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { expectedRequestTick = 12 * 3600, requestVoltageDeviationTolerance = Each(1e-14), ), - ParticipantResultHandler( - Iterable(resultListener.ref), - notifierConfig, - ), + ParticipantResultHandler(resultProxy.ref, notifierConfig), )(using Left(scheduler.ref)) ) @@ -290,7 +285,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! Activation(8 * 3600) - resultListener.expectMessageType[ParticipantResultEvent] match { + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe (8 * 3600).toDateTime @@ -323,13 +318,13 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { q should approximate(Kilovars(0.968644209676)) } - resultListener.expectNoMessage() + resultProxy.expectNoMessage() scheduler.expectNoMessage() participantAgent ! GridSimulationFinished(12 * 3600, 24 * 3600) // calculation should start now - resultListener.expectMessageType[ParticipantResultEvent] match { + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe (12 * 3600).toDateTime @@ -353,7 +348,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! Activation(20 * 3600) - resultListener.expectMessageType[ParticipantResultEvent] match { + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe (20 * 3600).toDateTime @@ -393,7 +388,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { val scheduler = createTestProbe[SchedulerMessage]() val gridAgent = createTestProbe[GridAgent.Message]() - val resultListener = createTestProbe[ResultEvent]() + val resultProxy = createTestProbe[ResultEvent | ExpectResult]() val responseReceiver = createTestProbe[MockResponseMessage]() val service = createTestProbe[ServiceMessage]() @@ -422,10 +417,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { expectedRequestTick = 12 * 3600, requestVoltageDeviationTolerance = Each(1e-14), ), - ParticipantResultHandler( - Iterable(resultListener.ref), - notifierConfig, - ), + ParticipantResultHandler(resultProxy.ref, notifierConfig), )(using Left(scheduler.ref) ) @@ -439,7 +431,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! Activation(0) // nothing should happen, still waiting for secondary data... - resultListener.expectNoMessage() + resultProxy.expectNoMessage() scheduler.expectNoMessage() participantAgent ! DataProvision( @@ -450,7 +442,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { ) // outside of operation interval, 0 MW - resultListener.expectMessageType[ParticipantResultEvent] match { + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe simulationStartDate @@ -476,7 +468,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { Some(12 * 3600), ) - resultListener.expectNoMessage() + resultProxy.expectNoMessage() scheduler.expectNoMessage() // TICK 8 * 3600: Start of operation interval @@ -489,7 +481,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! Activation(8 * 3600) - resultListener.expectMessageType[ParticipantResultEvent] match { + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe (8 * 3600).toDateTime @@ -526,7 +518,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! GridSimulationFinished(12 * 3600, 24 * 3600) // nothing should happen, still waiting for secondary data... - resultListener.expectNoMessage() + resultProxy.expectNoMessage() scheduler.expectNoMessage() participantAgent ! DataProvision( @@ -537,7 +529,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { ) // calculation should start now - resultListener.expectMessageType[ParticipantResultEvent] match { + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe (12 * 3600).toDateTime @@ -567,7 +559,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { ) // no-op activation, thus no result expected - resultListener.expectNoMessage() + resultProxy.expectNoMessage() // new data is expected at 18 hours scheduler.expectMessage( @@ -582,7 +574,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! Activation(18 * 3600) // nothing should happen, still waiting for secondary data... - resultListener.expectNoMessage() + resultProxy.expectNoMessage() scheduler.expectNoMessage() participantAgent ! DataProvision( @@ -593,7 +585,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { ) // calculation should start now - resultListener.expectMessageType[ParticipantResultEvent] match { + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe (18 * 3600).toDateTime @@ -612,7 +604,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! Activation(20 * 3600) - resultListener.expectMessageType[ParticipantResultEvent] match { + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe (20 * 3600).toDateTime @@ -644,7 +636,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! GridSimulationFinished(24 * 3600, 36 * 3600) - resultListener.expectNoMessage() + resultProxy.expectNoMessage() scheduler.expectNoMessage() } @@ -657,7 +649,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { val scheduler = createTestProbe[SchedulerMessage]() val gridAgent = createTestProbe[GridAgent.Message]() - val resultListener = createTestProbe[ResultEvent]() + val resultProxy = createTestProbe[ResultEvent | ExpectResult]() val service = createTestProbe[ServiceMessage]() // no additional activation ticks @@ -684,10 +676,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { expectedRequestTick = 12 * 3600, requestVoltageDeviationTolerance = Each(1e-14), ), - ParticipantResultHandler( - Iterable(resultListener.ref), - notifierConfig, - ), + ParticipantResultHandler(resultProxy.ref, notifierConfig), )(using Left(scheduler.ref) ) @@ -698,7 +687,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! Activation(0) // nothing should happen, still waiting for primary data... - resultListener.expectNoMessage() + resultProxy.expectNoMessage() scheduler.expectNoMessage() participantAgent ! DataProvision( @@ -709,7 +698,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { ) // outside of operation interval, 0 MW - resultListener.expectMessageType[ParticipantResultEvent] match { + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe simulationStartDate @@ -732,14 +721,14 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { Some(12 * 3600), ) - resultListener.expectNoMessage() + resultProxy.expectNoMessage() scheduler.expectNoMessage() // TICK 8 * 3600: Start of operation interval participantAgent ! Activation(8 * 3600) - resultListener.expectMessageType[ParticipantResultEvent] match { + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe (8 * 3600).toDateTime @@ -773,7 +762,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! GridSimulationFinished(12 * 3600, 24 * 3600) // nothing should happen, still waiting for primary data... - resultListener.expectNoMessage() + resultProxy.expectNoMessage() scheduler.expectNoMessage() participantAgent ! DataProvision( @@ -784,7 +773,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { ) // calculation should start now - resultListener.expectMessageType[ParticipantResultEvent] match { + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe (12 * 3600).toDateTime @@ -802,7 +791,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! Activation(18 * 3600) // nothing should happen, still waiting for primary data... - resultListener.expectNoMessage() + resultProxy.expectNoMessage() scheduler.expectNoMessage() participantAgent ! DataProvision( @@ -813,7 +802,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { ) // calculation should start now - resultListener.expectMessageType[ParticipantResultEvent] match { + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe (18 * 3600).toDateTime @@ -829,7 +818,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! Activation(20 * 3600) - resultListener.expectMessageType[ParticipantResultEvent] match { + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe (20 * 3600).toDateTime @@ -858,7 +847,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! GridSimulationFinished(24 * 3600, 36 * 3600) - resultListener.expectNoMessage() + resultProxy.expectNoMessage() scheduler.expectNoMessage() } @@ -875,7 +864,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { val em = createTestProbe[FlexResponse]() val gridAgent = createTestProbe[GridAgent.Message]() - val resultListener = createTestProbe[ResultEvent]() + val resultProxy = createTestProbe[ResultEvent | ExpectResult]() val responseReceiver = createTestProbe[MockResponseMessage]() // no additional activation ticks @@ -897,10 +886,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { expectedRequestTick = 12 * 3600, requestVoltageDeviationTolerance = Each(1e-14), ), - ParticipantResultHandler( - Iterable(resultListener.ref), - notifierConfig, - ), + ParticipantResultHandler(resultProxy.ref, notifierConfig), )(using Right(em.ref) ) @@ -927,7 +913,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(3)) } - resultListener.expectMessageType[FlexOptionsResultEvent] match { + resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe (8 * 3600).toDateTime @@ -938,7 +924,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssuePowerControl(8 * 3600, Kilowatts(3)) - resultListener.expectMessageType[ParticipantResultEvent] match { + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe (8 * 3600).toDateTime @@ -999,7 +985,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(0)) } - resultListener.expectMessageType[FlexOptionsResultEvent] match { + resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe (20 * 3600).toDateTime @@ -1010,7 +996,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssueNoControl(20 * 3600) - resultListener.expectMessageType[ParticipantResultEvent] match { + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe (20 * 3600).toDateTime @@ -1051,7 +1037,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { val em = createTestProbe[FlexResponse]() val gridAgent = createTestProbe[GridAgent.Message]() - val resultListener = createTestProbe[ResultEvent]() + val resultProxy = createTestProbe[ResultEvent | ExpectResult]() val responseReceiver = createTestProbe[MockResponseMessage]() // with additional activation ticks @@ -1084,10 +1070,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { expectedRequestTick = 12 * 3600, requestVoltageDeviationTolerance = Each(1e-14), ), - ParticipantResultHandler( - Iterable(resultListener.ref), - notifierConfig, - ), + ParticipantResultHandler(resultProxy.ref, notifierConfig), )(using Right(em.ref) ) @@ -1108,7 +1091,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(3)) } - resultListener.expectMessageType[FlexOptionsResultEvent] match { + resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe (8 * 3600).toDateTime @@ -1119,7 +1102,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssuePowerControl(8 * 3600, Kilowatts(3)) - resultListener.expectMessageType[ParticipantResultEvent] match { + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe (8 * 3600).toDateTime @@ -1161,7 +1144,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { q should approximate(Kilovars(0.48432210483)) } - resultListener.expectNoMessage() + resultProxy.expectNoMessage() em.expectNoMessage() participantAgent ! GridSimulationFinished(12 * 3600, 24 * 3600) @@ -1178,7 +1161,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(3)) } - resultListener.expectMessageType[FlexOptionsResultEvent] match { + resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe (12 * 3600).toDateTime @@ -1189,7 +1172,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssueNoControl(12 * 3600) - resultListener.expectMessageType[ParticipantResultEvent] match { + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe (12 * 3600).toDateTime @@ -1227,7 +1210,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(0)) } - resultListener.expectMessageType[FlexOptionsResultEvent] match { + resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe (20 * 3600).toDateTime @@ -1238,7 +1221,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssueNoControl(20 * 3600) - resultListener.expectMessageType[ParticipantResultEvent] match { + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe (20 * 3600).toDateTime @@ -1286,7 +1269,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { val em = createTestProbe[FlexResponse]() val gridAgent = createTestProbe[GridAgent.Message]() - val resultListener = createTestProbe[ResultEvent]() + val resultProxy = createTestProbe[ResultEvent | ExpectResult]() val responseReceiver = createTestProbe[MockResponseMessage]() val service = createTestProbe[ServiceMessage]() @@ -1320,10 +1303,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { expectedRequestTick = 12 * 3600, requestVoltageDeviationTolerance = Each(1e-14), ), - ParticipantResultHandler( - Iterable(resultListener.ref), - notifierConfig, - ), + ParticipantResultHandler(resultProxy.ref, notifierConfig), )(using Right(em.ref)) ) @@ -1335,7 +1315,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! FlexActivation(0) // nothing should happen, still waiting for secondary data... - resultListener.expectNoMessage() + resultProxy.expectNoMessage() em.expectNoMessage() participantAgent ! DataProvision( @@ -1356,7 +1336,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(0)) } - resultListener.expectMessageType[FlexOptionsResultEvent] match { + resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe simulationStartDate @@ -1368,7 +1348,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssueNoControl(0) // outside of operation interval, 0 MW - resultListener.expectMessageType[ParticipantResultEvent] match { + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe simulationStartDate @@ -1404,7 +1384,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { Some(12 * 3600), ) - resultListener.expectNoMessage() + resultProxy.expectNoMessage() em.expectNoMessage() // TICK 8 * 3600: Start of operation interval @@ -1428,7 +1408,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(4)) } - resultListener.expectMessageType[FlexOptionsResultEvent] match { + resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe (8 * 3600).toDateTime @@ -1439,7 +1419,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssuePowerControl(8 * 3600, Kilowatts(3)) - resultListener.expectMessageType[ParticipantResultEvent] match { + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe (8 * 3600).toDateTime @@ -1486,7 +1466,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! GridSimulationFinished(12 * 3600, 24 * 3600) // nothing should happen, still waiting for secondary data... - resultListener.expectNoMessage() + resultProxy.expectNoMessage() em.expectNoMessage() participantAgent ! DataProvision( @@ -1508,7 +1488,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(5)) } - resultListener.expectMessageType[FlexOptionsResultEvent] match { + resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe (12 * 3600).toDateTime @@ -1519,7 +1499,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssueNoControl(12 * 3600) - resultListener.expectMessageType[ParticipantResultEvent] match { + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe (12 * 3600).toDateTime @@ -1549,7 +1529,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! FlexActivation(18 * 3600) // nothing should happen, still waiting for secondary data... - resultListener.expectNoMessage() + resultProxy.expectNoMessage() em.expectNoMessage() participantAgent ! DataProvision( @@ -1571,7 +1551,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(8)) } - resultListener.expectMessageType[FlexOptionsResultEvent] match { + resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe (18 * 3600).toDateTime @@ -1582,7 +1562,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssueNoControl(18 * 3600) - resultListener.expectMessageType[ParticipantResultEvent] match { + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe (18 * 3600).toDateTime @@ -1623,7 +1603,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(0)) } - resultListener.expectMessageType[FlexOptionsResultEvent] match { + resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe (20 * 3600).toDateTime @@ -1634,7 +1614,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssueNoControl(20 * 3600) - resultListener.expectMessageType[ParticipantResultEvent] match { + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe (20 * 3600).toDateTime @@ -1673,7 +1653,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! GridSimulationFinished(24 * 3600, 36 * 3600) - resultListener.expectNoMessage() + resultProxy.expectNoMessage() em.expectNoMessage() } @@ -1686,7 +1666,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { val em = createTestProbe[FlexResponse]() val gridAgent = createTestProbe[GridAgent.Message]() - val resultListener = createTestProbe[ResultEvent]() + val resultProxy = createTestProbe[ResultEvent | ExpectResult]() val service = createTestProbe[ServiceMessage]() // no additional activation ticks @@ -1713,10 +1693,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { expectedRequestTick = 12 * 3600, requestVoltageDeviationTolerance = Each(1e-14), ), - ParticipantResultHandler( - Iterable(resultListener.ref), - notifierConfig, - ), + ParticipantResultHandler(resultProxy.ref, notifierConfig), )(using Right(em.ref) ) @@ -1727,7 +1704,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! FlexActivation(0) // nothing should happen, still waiting for primary data... - resultListener.expectNoMessage() + resultProxy.expectNoMessage() em.expectNoMessage() participantAgent ! DataProvision( @@ -1748,7 +1725,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(0)) } - resultListener.expectMessageType[FlexOptionsResultEvent] match { + resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe simulationStartDate @@ -1760,7 +1737,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssueNoControl(0) // outside of operation interval, 0 MW - resultListener.expectMessageType[ParticipantResultEvent] match { + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe simulationStartDate @@ -1793,7 +1770,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { Some(12 * 3600), ) - resultListener.expectNoMessage() + resultProxy.expectNoMessage() em.expectNoMessage() // TICK 8 * 3600: Start of operation interval @@ -1811,7 +1788,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(3)) } - resultListener.expectMessageType[FlexOptionsResultEvent] match { + resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe (8 * 3600).toDateTime @@ -1822,7 +1799,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssuePowerControl(8 * 3600, Kilowatts(3)) - resultListener.expectMessageType[ParticipantResultEvent] match { + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe (8 * 3600).toDateTime @@ -1866,7 +1843,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! GridSimulationFinished(12 * 3600, 24 * 3600) // nothing should happen, still waiting for primary data... - resultListener.expectNoMessage() + resultProxy.expectNoMessage() em.expectNoMessage() participantAgent ! DataProvision( @@ -1888,7 +1865,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(6)) } - resultListener.expectMessageType[FlexOptionsResultEvent] match { + resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe (12 * 3600).toDateTime @@ -1899,7 +1876,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssueNoControl(12 * 3600) - resultListener.expectMessageType[ParticipantResultEvent] match { + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe (12 * 3600).toDateTime @@ -1926,7 +1903,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! FlexActivation(18 * 3600) // nothing should happen, still waiting for primary data... - resultListener.expectNoMessage() + resultProxy.expectNoMessage() em.expectNoMessage() participantAgent ! DataProvision( @@ -1948,7 +1925,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(3)) } - resultListener.expectMessageType[FlexOptionsResultEvent] match { + resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe (18 * 3600).toDateTime @@ -1959,7 +1936,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssueNoControl(18 * 3600) - resultListener.expectMessageType[ParticipantResultEvent] match { + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe (18 * 3600).toDateTime @@ -1996,7 +1973,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(0)) } - resultListener.expectMessageType[FlexOptionsResultEvent] match { + resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe (20 * 3600).toDateTime @@ -2007,7 +1984,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssueNoControl(20 * 3600) - resultListener.expectMessageType[ParticipantResultEvent] match { + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid result.getTime shouldBe (20 * 3600).toDateTime @@ -2043,7 +2020,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! GridSimulationFinished(24 * 3600, 36 * 3600) - resultListener.expectNoMessage() + resultProxy.expectNoMessage() em.expectNoMessage() } diff --git a/src/test/scala/edu/ie3/simona/event/listener/ResultEventListenerSpec.scala b/src/test/scala/edu/ie3/simona/event/listener/ResultListenerSpec.scala similarity index 75% rename from src/test/scala/edu/ie3/simona/event/listener/ResultEventListenerSpec.scala rename to src/test/scala/edu/ie3/simona/event/listener/ResultListenerSpec.scala index 742c3b4d5d..6a3704d2a1 100644 --- a/src/test/scala/edu/ie3/simona/event/listener/ResultEventListenerSpec.scala +++ b/src/test/scala/edu/ie3/simona/event/listener/ResultListenerSpec.scala @@ -23,6 +23,10 @@ import edu.ie3.simona.event.ResultEvent.{ import edu.ie3.simona.io.result.ResultSinkType.Csv import edu.ie3.simona.io.result.{ResultEntitySink, ResultSinkType} import edu.ie3.simona.logging.LogbackConfiguration +import edu.ie3.simona.service.results.{ + ThreeWindingResultTestData, + Transformer3wResultSupport, +} import edu.ie3.simona.test.common.result.PowerFlowResultData import edu.ie3.simona.test.common.{IOTestCommons, UnitSpec} import edu.ie3.simona.util.ResultFileHierarchy @@ -34,6 +38,7 @@ import org.apache.pekko.actor.testkit.typed.scaladsl.{ ScalaTestWithActorTestKit, } import org.apache.pekko.testkit.TestKit.awaitCond +import edu.ie3.simona.ontology.messages.ResultMessage.ResultResponse import java.io.{File, FileInputStream} import java.util.UUID @@ -44,7 +49,7 @@ import scala.concurrent.{Await, Future} import scala.io.Source import scala.language.postfixOps -class ResultEventListenerSpec +class ResultListenerSpec extends ScalaTestWithActorTestKit( ActorTestKit.ApplicationTestConfig.withValue( "org.apache.pekko.actor.testkit.typed.filter-leeway", @@ -54,8 +59,7 @@ class ResultEventListenerSpec with UnitSpec with IOTestCommons with PowerFlowResultData - with ThreeWindingResultTestData - with Transformer3wResultSupport { + with ThreeWindingResultTestData { val simulationName = "testSim" val resultEntitiesToBeWritten: Set[Class[? <: ResultEntity]] = Set( classOf[PvResult], @@ -97,7 +101,7 @@ class ResultEventListenerSpec Symbol("initializeSinks") ) - ResultEventListener invokePrivate initializeSinks(resultFileHierarchy) + ResultListener invokePrivate initializeSinks(resultFileHierarchy) } private def getFileLinesLength(file: File) = { @@ -141,7 +145,7 @@ class ResultEventListenerSpec resultFileHierarchy(2, ".ttt", Set(classOf[Transformer3WResult])) val deathWatch = createTestProbe("deathWatch") val listener = spawn( - ResultEventListener( + ResultListener( fileHierarchy ) ) @@ -152,16 +156,16 @@ class ResultEventListenerSpec } "handling ordinary results" should { - "process a valid participants result correctly" in { + "process participants results correctly" in { val specificOutputFileHierarchy = resultFileHierarchy(3, ".csv") val listenerRef = spawn( - ResultEventListener( + ResultListener( specificOutputFileHierarchy ) ) - listenerRef ! ParticipantResultEvent(dummyPvResult) + listenerRef ! ResultResponse(dummyPvResult) val outputFile = specificOutputFileHierarchy.rawOutputDataFilePaths .getOrElse( @@ -203,22 +207,20 @@ class ResultEventListenerSpec resultFileSource.close() } - "process a valid power flow result correctly" in { + "process grid results correctly" in { val specificOutputFileHierarchy = resultFileHierarchy(4, ".csv") - val listenerRef = spawn( - ResultEventListener( - specificOutputFileHierarchy + val listenerRef = + spawn(ResultListener(specificOutputFileHierarchy)) + + listenerRef ! ResultResponse( + Iterable( + dummyNodeResult, + dummySwitchResult, + dummyLineResult, + dummyTrafo2wResult, ) ) - listenerRef ! PowerFlowResultEvent( - Iterable(dummyNodeResult), - Iterable(dummySwitchResult), - Iterable(dummyLineResult), - Iterable(dummyTrafo2wResult), - Iterable.empty[PartialTransformer3wResult], - ) - val outputFiles = Map( dummyNodeResultString -> specificOutputFileHierarchy.rawOutputDataFilePaths .getOrElse( @@ -286,103 +288,18 @@ class ResultEventListenerSpec } } - "handling three winding transformer results" should { - def powerflow3wResult( - partialResult: PartialTransformer3wResult - ): PowerFlowResultEvent = - PowerFlowResultEvent( - Iterable.empty[NodeResult], - Iterable.empty[SwitchResult], - Iterable.empty[LineResult], - Iterable.empty[Transformer2WResult], - Iterable(partialResult), - ) - - "correctly reacts on received results" in { - val fileHierarchy = - resultFileHierarchy(5, ".csv", Set(classOf[Transformer3WResult])) - val listener = spawn( - ResultEventListener( - fileHierarchy - ) - ) - - val outputFile = fileHierarchy.rawOutputDataFilePaths - .getOrElse( - classOf[Transformer3WResult], - fail( - s"Cannot get filepath for raw result file of class '${classOf[Transformer3WResult].getSimpleName}' from outputFileHierarchy!'" - ), - ) - .toFile - - /* The result file is created at start up and only contains a headline. */ - awaitCond( - outputFile.exists(), - interval = 500.millis, - max = timeoutDuration, - ) - getFileLinesLength(outputFile) shouldBe 1 - - /* Face the listener with data, as long as they are not comprehensive */ - listener ! powerflow3wResult(resultA) - - listener ! powerflow3wResult(resultC) - - /* Also add unrelated result for different input model */ - val otherResultA = resultA.copy(input = UUID.randomUUID()) - listener ! powerflow3wResult(otherResultA) - - /* Add result A again, which should lead to a failure internally, - but everything should still continue normally - */ - listener ! powerflow3wResult(resultA) - - /* Make sure, that there still is no content in file */ - getFileLinesLength(outputFile) shouldBe 1 - - /* Complete awaited result */ - listener ! powerflow3wResult(resultB) - - // stop listener so that result is flushed out - listener ! DelayedStopHelper.FlushAndStop - - /* Await that the result is written */ - awaitCond( - getFileLinesLength(outputFile) == 2, - interval = 500.millis, - max = timeoutDuration, - ) - /* Check the result */ - val resultFileSource = Source.fromFile(outputFile) - val resultFileLines = resultFileSource.getLines().toSeq - - resultFileLines.size shouldBe 2 - val resultLine = resultFileLines.lastOption.getOrElse( - fail( - "Cannot get csv row that should have been written out by the listener!" - ) - ) - - resultLine shouldBe "2.0,1.0,4.0,3.0,6.0,5.0,40d02538-d8dd-421c-8e68-400f1da170c7,-5," + TimeUtil.withDefaults - .toString(time) - - resultFileSource.close() - } - } - "shutting down" should { "shutdown and compress the data when requested to do so without any errors" in { val specificOutputFileHierarchy = resultFileHierarchy(6, ".csv.gz", compressResults = true) val listenerRef = spawn( - ResultEventListener( + ResultListener( specificOutputFileHierarchy ) ) ResultSinkType.Csv(fileFormat = ".csv.gz", delimiter = ",") - listenerRef ! ParticipantResultEvent(dummyPvResult) + listenerRef ! ResultResponse(dummyPvResult) val outputFile = new File( ".gz$".r.replaceAllIn( diff --git a/src/test/scala/edu/ie3/simona/io/result/ResultEntityKafkaSpec.scala b/src/test/scala/edu/ie3/simona/io/result/ResultEntityKafkaSpec.scala index bc6edb13dd..7e186d8379 100644 --- a/src/test/scala/edu/ie3/simona/io/result/ResultEntityKafkaSpec.scala +++ b/src/test/scala/edu/ie3/simona/io/result/ResultEntityKafkaSpec.scala @@ -6,10 +6,11 @@ package edu.ie3.simona.io.result -import org.apache.pekko.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit import edu.ie3.datamodel.models.result.NodeResult +import edu.ie3.simona.api.ontology.results.ProvideResultEntities import edu.ie3.simona.event.ResultEvent.PowerFlowResultEvent -import edu.ie3.simona.event.listener.ResultEventListener +import edu.ie3.simona.ontology.messages.ResultMessage.ResultResponse +import edu.ie3.simona.event.listener.ResultListener import edu.ie3.simona.io.result.plain.PlainResult.PlainNodeResult import edu.ie3.simona.io.result.plain.PlainWriter import edu.ie3.simona.logging.LogbackConfiguration @@ -23,6 +24,7 @@ import org.apache.kafka.clients.consumer.KafkaConsumer import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.serialization.{Deserializer, Serdes} import org.apache.kafka.common.utils.Bytes +import org.apache.pekko.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit import org.scalatest.GivenWhenThen import org.scalatest.concurrent.Eventually import org.scalatest.wordspec.AnyWordSpecLike @@ -91,7 +93,7 @@ class ResultEntityKafkaSpec // build the listener val listenerRef = spawn( - ResultEventListener( + ResultListener( ResultFileHierarchy( "out", "simName", @@ -132,12 +134,12 @@ class ResultEntityKafkaSpec ) When("receiving the NodeResults") - listenerRef ! PowerFlowResultEvent( - Iterable(nodeRes1, nodeRes2, nodeRes3), - Iterable.empty, - Iterable.empty, - Iterable.empty, - Iterable.empty, + listenerRef ! ResultResponse( + Map( + nodeRes1.getInputModel -> Iterable(nodeRes1), + nodeRes2.getInputModel -> Iterable(nodeRes2), + nodeRes3.getInputModel -> Iterable(nodeRes3), + ) ) Then("records can be fetched from Kafka") diff --git a/src/test/scala/edu/ie3/simona/model/participant/evcs/EvcsModelIT.scala b/src/test/scala/edu/ie3/simona/model/participant/evcs/EvcsModelIT.scala index 24c8e0e6a1..4cc9b3e53d 100644 --- a/src/test/scala/edu/ie3/simona/model/participant/evcs/EvcsModelIT.scala +++ b/src/test/scala/edu/ie3/simona/model/participant/evcs/EvcsModelIT.scala @@ -16,8 +16,8 @@ import edu.ie3.simona.agent.participant.ParticipantAgentInit.{ } import edu.ie3.simona.api.data.connection.ExtEvDataConnection import edu.ie3.simona.api.data.model.ev.EvModel -import edu.ie3.simona.api.ontology.ev.* import edu.ie3.simona.api.ontology.ScheduleDataServiceMessage +import edu.ie3.simona.api.ontology.ev.* import edu.ie3.simona.config.RuntimeConfig.EvcsRuntimeConfig import edu.ie3.simona.event.ResultEvent import edu.ie3.simona.event.ResultEvent.ParticipantResultEvent @@ -31,11 +31,13 @@ import edu.ie3.simona.ontology.messages.ServiceMessage.{ PrimaryServiceRegistrationMessage, } import edu.ie3.simona.ontology.messages.{Activation, SchedulerMessage} +import edu.ie3.simona.ontology.messages.ResultMessage.RequestResult import edu.ie3.simona.scheduler.ScheduleLock import edu.ie3.simona.service.ServiceType import edu.ie3.simona.service.ev.ExtEvDataService import edu.ie3.simona.service.ev.ExtEvDataService.InitExtEvData import edu.ie3.simona.service.primary.PrimaryServiceProxy +import edu.ie3.simona.service.results.ResultServiceProxy.ExpectResult import edu.ie3.simona.test.common.input.EvcsInputTestData import edu.ie3.simona.test.common.{TestSpawnerTyped, UnitSpec} import edu.ie3.simona.util.SimonaConstants.{INIT_SIM_TICK, PRE_INIT_TICK} @@ -88,7 +90,8 @@ class EvcsModelIT "handle a few requests and arrivals as expected" in { val gridAgent = TestProbe[GridAgent.Message]("GridAgent") - val resultListener = TestProbe[ResultEvent]("ResultListener") + val resultProxy = + TestProbe[ResultEvent | ExpectResult]("ResultServiceProxy") val primaryServiceProxy = TestProbe[PrimaryServiceProxy.Message]("PrimaryServiceProxy") val scheduler = TestProbe[SchedulerMessage]("Scheduler") @@ -124,8 +127,8 @@ class EvcsModelIT given ParticipantRefs = ParticipantRefs( gridAgent = gridAgent.ref, primaryServiceProxy = primaryServiceProxy.ref, + resultServiceProxy = resultProxy.ref, services = Map(ServiceType.EvMovementService -> evService), - resultListener = Iterable(resultListener.ref), ) val evcsKey = @@ -207,7 +210,7 @@ class EvcsModelIT scheduler.expectMessage(Completion(evService, None)) - resultListener.expectNoMessage() + resultProxy.expectNoMessage() // Send arrivals extEvData.provideArrivingEvs( @@ -227,7 +230,7 @@ class EvcsModelIT evcsActivation ! Activation(0) - resultListener + resultProxy .receiveMessages(3) .map { case ParticipantResultEvent(result) => result @@ -271,12 +274,12 @@ class EvcsModelIT scheduler.expectMessage(Completion(evService, None)) - resultListener.expectNoMessage() + resultProxy.expectNoMessage() // EVCS activation without arrivals evcsActivation ! Activation(1800) - resultListener + resultProxy .receiveMessages(2) .map { case ParticipantResultEvent(result) => result @@ -303,7 +306,7 @@ class EvcsModelIT evcsActivation ! Activation(3600) - resultListener + resultProxy .receiveMessages(2) .map { case ParticipantResultEvent(result) => result @@ -374,7 +377,7 @@ class EvcsModelIT evcsActivation ! Activation(9000) - resultListener.expectNoMessage() + resultProxy.expectNoMessage() // Next data at 10800 scheduler.expectMessage(Completion(evcsActivation, Some(10800))) @@ -394,7 +397,7 @@ class EvcsModelIT scheduler.expectMessage(Completion(evService, None)) - resultListener.expectNoMessage() + resultProxy.expectNoMessage() // Send arrivals extEvData.provideArrivingEvs( @@ -414,7 +417,7 @@ class EvcsModelIT evcsActivation ! Activation(10800) - resultListener + resultProxy .receiveMessages(2) .map { case ParticipantResultEvent(result) => result @@ -456,7 +459,7 @@ class EvcsModelIT // EVCS activation evcsActivation ! Activation(12600) - resultListener + resultProxy .receiveMessages(2) .map { case ParticipantResultEvent(result) => result @@ -527,7 +530,7 @@ class EvcsModelIT evcsActivation ! Activation(14400) - resultListener.expectNoMessage() + resultProxy.expectNoMessage() // evB is departing at 18000 scheduler.expectMessage(Completion(evcsActivation, Some(18000))) @@ -580,7 +583,7 @@ class EvcsModelIT evcsActivation ! Activation(18000) - resultListener.expectNoMessage() + resultProxy.expectNoMessage() // No future arrivals planned, next activation: end of simulation scheduler.expectMessage(Completion(evcsActivation, Some(48 * 3600))) diff --git a/src/test/scala/edu/ie3/simona/event/listener/ThreeWindingResultHandlingSpec.scala b/src/test/scala/edu/ie3/simona/service/results/ThreeWindingResultHandlingSpec.scala similarity index 95% rename from src/test/scala/edu/ie3/simona/event/listener/ThreeWindingResultHandlingSpec.scala rename to src/test/scala/edu/ie3/simona/service/results/ThreeWindingResultHandlingSpec.scala index 227b4ce9f5..1615462104 100644 --- a/src/test/scala/edu/ie3/simona/event/listener/ThreeWindingResultHandlingSpec.scala +++ b/src/test/scala/edu/ie3/simona/service/results/ThreeWindingResultHandlingSpec.scala @@ -4,9 +4,11 @@ * Research group Distribution grid planning and operation */ -package edu.ie3.simona.event.listener +package edu.ie3.simona.service.results import edu.ie3.simona.agent.grid.GridResultsSupport.PartialTransformer3wResult +import edu.ie3.simona.service.results.Transformer3wResultSupport +import edu.ie3.simona.service.results.Transformer3wResultSupport.AggregatedTransformer3wResult import edu.ie3.simona.test.common.UnitSpec import edu.ie3.util.TimeUtil import org.scalatest.prop.TableDrivenPropertyChecks @@ -19,8 +21,7 @@ import scala.util.{Failure, Success} class ThreeWindingResultHandlingSpec extends UnitSpec with TableDrivenPropertyChecks - with ThreeWindingResultTestData - with Transformer3wResultSupport { + with ThreeWindingResultTestData { "Handling three winding results" when { "assembling joint values" should { val mockAResult = PartialTransformer3wResult.PortA( diff --git a/src/test/scala/edu/ie3/simona/event/listener/ThreeWindingResultTestData.scala b/src/test/scala/edu/ie3/simona/service/results/ThreeWindingResultTestData.scala similarity index 97% rename from src/test/scala/edu/ie3/simona/event/listener/ThreeWindingResultTestData.scala rename to src/test/scala/edu/ie3/simona/service/results/ThreeWindingResultTestData.scala index de3ae98ec7..92011d19eb 100644 --- a/src/test/scala/edu/ie3/simona/event/listener/ThreeWindingResultTestData.scala +++ b/src/test/scala/edu/ie3/simona/service/results/ThreeWindingResultTestData.scala @@ -4,7 +4,7 @@ * Research group Distribution grid planning and operation */ -package edu.ie3.simona.event.listener +package edu.ie3.simona.service.results import edu.ie3.datamodel.models.StandardUnits import edu.ie3.datamodel.models.result.connector.Transformer3WResult diff --git a/src/test/scala/edu/ie3/simona/sim/SimonaSimSpec.scala b/src/test/scala/edu/ie3/simona/sim/SimonaSimSpec.scala index 9eb3ba7b71..68abb1100e 100644 --- a/src/test/scala/edu/ie3/simona/sim/SimonaSimSpec.scala +++ b/src/test/scala/edu/ie3/simona/sim/SimonaSimSpec.scala @@ -12,15 +12,20 @@ import edu.ie3.simona.api.ExtSimAdapter import edu.ie3.simona.config.SimonaConfig import edu.ie3.simona.event.listener.{ DelayedStopHelper, - ResultEventListener, + ResultListener, RuntimeEventListener, } import edu.ie3.simona.event.{ResultEvent, RuntimeEvent} import edu.ie3.simona.main.RunSimona.SimonaEnded +import edu.ie3.simona.ontology.messages.ResultMessage.{ + RequestResult, + ResultResponse, +} import edu.ie3.simona.ontology.messages.{SchedulerMessage, ServiceMessage} import edu.ie3.simona.scheduler.TimeAdvancer import edu.ie3.simona.scheduler.core.Core.CoreFactory import edu.ie3.simona.scheduler.core.RegularSchedulerCore +import edu.ie3.simona.service.results.ResultServiceProxy import edu.ie3.simona.sim.SimonaSim.SimulationEnded import edu.ie3.simona.sim.SimonaSimSpec.* import edu.ie3.simona.sim.setup.{ExtSimSetupData, SimonaSetup} @@ -29,11 +34,11 @@ import org.apache.pekko.actor.testkit.typed.scaladsl.{ ScalaTestWithActorTestKit, TestProbe, } -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} import java.nio.file.Path +import java.time.ZonedDateTime import java.util.UUID class SimonaSimSpec extends ScalaTestWithActorTestKit with UnitSpec { @@ -47,9 +52,9 @@ class SimonaSimSpec extends ScalaTestWithActorTestKit with UnitSpec { val runtimeListener = TestProbe[RuntimeEventListener.Request]("runtimeEventListener") val resultListener = - TestProbe[ResultEventListener.Request]("resultEventListener") + TestProbe[ResultListener.Message]("resultEventListener") val timeAdvancer = TestProbe[TimeAdvancer.Request]("timeAdvancer") - val extSimAdapter = TestProbe[ExtSimAdapter.Stop]("extSimAdapter") + val extSimAdapter = TestProbe[ExtSimAdapter.Request]("extSimAdapter") val simonaSim = spawn( SimonaSim( @@ -61,6 +66,7 @@ class SimonaSimSpec extends ScalaTestWithActorTestKit with UnitSpec { override def extSimulations( context: ActorContext[?], scheduler: ActorRef[SchedulerMessage], + resultProxy: ActorRef[RequestResult], extSimPath: Option[Path], ): ExtSimSetupData = { // We cannot return a TestProbe ref here, @@ -70,8 +76,9 @@ class SimonaSimSpec extends ScalaTestWithActorTestKit with UnitSpec { uniqueName("extSimAdapterForwarder"), ) ExtSimSetupData( - Iterable(extSim.toClassic), + Iterable(extSim), Seq.empty, + None, Seq.empty, Seq.empty, ) @@ -113,7 +120,7 @@ class SimonaSimSpec extends ScalaTestWithActorTestKit with UnitSpec { val runtimeListener = TestProbe[RuntimeEventListener.Request]("runtimeEventListener") val resultListener = - TestProbe[ResultEventListener.Request]("resultEventListener") + TestProbe[ResultListener.Message]("resultEventListener") val timeAdvancer = TestProbe[TimeAdvancer.Request]("timeAdvancer") val receiveThrowingActor = @@ -179,7 +186,7 @@ class SimonaSimSpec extends ScalaTestWithActorTestKit with UnitSpec { val runtimeListener = TestProbe[RuntimeEventListener.Request]("runtimeEventListener") val resultListener = - TestProbe[ResultEventListener.Request]("resultEventListener") + TestProbe[ResultListener.Message]("resultEventListener") val timeAdvancer = TestProbe[TimeAdvancer.Request]("timeAdvancer") val receiveStoppingActor = @@ -242,7 +249,7 @@ class SimonaSimSpec extends ScalaTestWithActorTestKit with UnitSpec { "RuntimeEventListener stops unexpectedly" in { val starter = TestProbe[SimonaEnded]("starter") val resultListener = - TestProbe[ResultEventListener.Request]("resultEventListener") + TestProbe[ResultListener.Message]("resultEventListener") val timeAdvancer = TestProbe[TimeAdvancer.Request]("timeAdvancer") val receiveThrowingActor = @@ -309,7 +316,7 @@ class SimonaSimSpec extends ScalaTestWithActorTestKit with UnitSpec { override def resultEventListener( context: ActorContext[?] - ): Seq[ActorRef[ResultEventListener.Request]] = + ): Seq[ActorRef[ResultListener.Message]] = throwTestException() } ), @@ -380,25 +387,25 @@ object SimonaSimSpec { "This is an exception for test purposes. It is expected to be thrown." ) - /** Makes the given actor name unique by appending a random UUID */ + /** Makes the given actor name unique by appending a random UUID. */ def uniqueName(name: String): String = s"${name}_${UUID.randomUUID()}" - /** Mock implementation of [[SimonaSetup]] + /** Mock implementation of [[SimonaSetup]]. * * @param runtimeEventProbe * Optional ActorRef that messages received by RuntimeEventListener are - * forwarded to + * forwarded to. * @param resultEventProbe * Optional ActorRef that messages received by ResultEventListener are - * forwarded to + * forwarded to. * @param timeAdvancerProbe * Optional ActorRef that messages received by TimeAdvancer are forwarded - * to + * to. */ class MockSetup( runtimeEventProbe: Option[ActorRef[RuntimeEventListener.Request]] = None, - resultEventProbe: Option[ActorRef[ResultEventListener.Request]] = None, + resultEventProbe: Option[ActorRef[ResultListener.Message]] = None, timeAdvancerProbe: Option[ActorRef[TimeAdvancer.Request]] = None, ) extends SimonaSetup with ConfigTestData { @@ -417,7 +424,7 @@ object SimonaSimSpec { override def resultEventListener( context: ActorContext[?] - ): Seq[ActorRef[ResultEventListener.Request]] = Seq( + ): Seq[ActorRef[ResultListener.Message]] = Seq( context.spawn( stoppableForwardMessage(resultEventProbe), uniqueName("resultEventForwarder"), @@ -431,6 +438,13 @@ object SimonaSimSpec { ): ActorRef[ServiceMessage] = context.spawn(empty, uniqueName("primaryService")) + override def resultServiceProxy( + context: ActorContext[?], + listeners: Seq[ActorRef[ResultResponse]], + simStartTime: ZonedDateTime, + ): ActorRef[ResultServiceProxy.Message] = + context.spawn(stoppableForwardMessage(None), uniqueName("resultService")) + override def weatherService( context: ActorContext[?], scheduler: ActorRef[SchedulerMessage], @@ -463,12 +477,12 @@ object SimonaSimSpec { override def gridAgents( context: ActorContext[?], environmentRefs: EnvironmentRefs, - resultEventListeners: Seq[ActorRef[ResultEvent]], ): Iterable[ActorRef[GridAgent.Message]] = Iterable.empty override def extSimulations( context: ActorContext[?], scheduler: ActorRef[SchedulerMessage], + resultProxy: ActorRef[RequestResult], extSimPath: Option[Path], ): ExtSimSetupData = ExtSimSetupData.apply diff --git a/src/test/scala/edu/ie3/simona/sim/setup/ExtSimSetupDataSpec.scala b/src/test/scala/edu/ie3/simona/sim/setup/ExtSimSetupDataSpec.scala index 2899223cfa..ff7492978e 100644 --- a/src/test/scala/edu/ie3/simona/sim/setup/ExtSimSetupDataSpec.scala +++ b/src/test/scala/edu/ie3/simona/sim/setup/ExtSimSetupDataSpec.scala @@ -30,7 +30,7 @@ class ExtSimSetupDataSpec extends ScalaTestWithActorTestKit with UnitSpec { "An ExtSimSetupData" should { - val emptyMapInput = Map.empty[UUID, Class[Value]].asJava + val emptyMapInput = Map.empty[UUID, Class[? <: Value]].asJava val emptyUuidList = List.empty[UUID].asJava "be updated with an ExtPrimaryDataConnection correctly" in { @@ -46,9 +46,10 @@ class ExtSimSetupDataSpec extends ScalaTestWithActorTestKit with UnitSpec { ) updated.extSimAdapters shouldBe empty - updated.extPrimaryDataServices shouldBe Seq((connection, primaryRef)) - updated.extDataServices shouldBe empty - updated.extResultListeners shouldBe empty + updated.primaryDataServices shouldBe Seq((connection, primaryRef)) + updated.evDataService shouldBe None + updated.resultListeners shouldBe empty + updated.resultProviders shouldBe empty } "be updated with multiple ExtPrimaryDataConnection correctly" in { @@ -67,12 +68,13 @@ class ExtSimSetupDataSpec extends ScalaTestWithActorTestKit with UnitSpec { .update(connection2, primaryRef2) updated.extSimAdapters shouldBe empty - updated.extPrimaryDataServices shouldBe Seq( + updated.primaryDataServices shouldBe Seq( (connection1, primaryRef1), (connection2, primaryRef2), ) - updated.extDataServices shouldBe empty - updated.extResultListeners shouldBe empty + updated.evDataService shouldBe None + updated.resultListeners shouldBe empty + updated.resultProviders shouldBe empty } "be updated with an ExtInputDataConnection correctly" in { @@ -85,27 +87,19 @@ class ExtSimSetupDataSpec extends ScalaTestWithActorTestKit with UnitSpec { val evConnection = new ExtEvDataConnection() val evRef = TestProbe[ExtEvDataService.Message]("ev_service").ref - val emConnection = new ExtEmDataConnection(emptyUuidList, EmMode.BASE) - val emRef = TestProbe[ServiceMessage]("em_service").ref - val cases = Table( ("connection", "serviceRef", "expected"), ( primaryConnection, primaryRef, - extSimSetupData.copy(extPrimaryDataServices = + extSimSetupData.copy(primaryDataServices = Seq((primaryConnection, primaryRef)) ), ), ( evConnection, evRef, - extSimSetupData.copy(extDataServices = Seq((evConnection, evRef))), - ), - ( - emConnection, - emRef, - extSimSetupData.copy(extDataServices = Seq((emConnection, emRef))), + extSimSetupData.copy(evDataService = Some(evRef)), ), ) @@ -113,25 +107,28 @@ class ExtSimSetupDataSpec extends ScalaTestWithActorTestKit with UnitSpec { val updated = extSimSetupData.update(connection, serviceRef) updated.extSimAdapters shouldBe expected.extSimAdapters - updated.extPrimaryDataServices shouldBe expected.extPrimaryDataServices - updated.extDataServices shouldBe expected.extDataServices - updated.extResultListeners shouldBe expected.extResultListeners + updated.primaryDataServices shouldBe expected.primaryDataServices + updated.evDataService shouldBe expected.evDataService + updated.resultListeners shouldBe expected.resultListeners + updated.resultProviders shouldBe expected.resultProviders } } "be updated with an ExtResultDataConnection correctly" in { val extSimSetupData = ExtSimSetupData.apply - val resultConnection = - new ExtResultDataConnection(emptyUuidList, emptyUuidList, emptyUuidList) - val resultRef = TestProbe[ServiceMessage]("result_service").ref + val resultConnection = new ExtResultDataConnection(emptyUuidList) + val resultServiceProxyRef = + TestProbe[ServiceMessage]("resultServiceProxy").ref - val updated = extSimSetupData.update(resultConnection, resultRef) + val updated = + extSimSetupData.update(resultConnection, resultServiceProxyRef) updated.extSimAdapters shouldBe empty - updated.extPrimaryDataServices shouldBe empty - updated.extDataServices shouldBe empty - updated.extResultListeners shouldBe Seq((resultConnection, resultRef)) + updated.primaryDataServices shouldBe empty + updated.evDataService shouldBe None + updated.resultListeners shouldBe empty + updated.resultProviders shouldBe Seq(resultServiceProxyRef) } "be updated with multiple different connections" in { @@ -144,31 +141,20 @@ class ExtSimSetupDataSpec extends ScalaTestWithActorTestKit with UnitSpec { val evConnection = new ExtEvDataConnection() val evRef = TestProbe[ExtEvDataService.Message]("ev_service").ref - val emConnection = new ExtEmDataConnection(emptyUuidList, EmMode.BASE) - val emRef = TestProbe[ServiceMessage]("em_service").ref - - val resultConnection = - new ExtResultDataConnection(emptyUuidList, emptyUuidList, emptyUuidList) - val resultRef = TestProbe[ServiceMessage]("result_service").ref + val resultConnection = new ExtResultDataConnection(emptyUuidList) + val resultServiceProxyRef = + TestProbe[ServiceMessage]("resultServiceProxy").ref val updated = extSimSetupData .update(primaryConnection, primaryRef) .update(evConnection, evRef) - .update(emConnection, emRef) - .update(resultConnection, resultRef) + .update(resultConnection, resultServiceProxyRef) updated.extSimAdapters shouldBe empty - updated.extPrimaryDataServices shouldBe Seq( - ( - primaryConnection, - primaryRef, - ) - ) - updated.extDataServices shouldBe Seq( - (evConnection, evRef), - (emConnection, emRef), - ) - updated.extResultListeners shouldBe Seq((resultConnection, resultRef)) + updated.primaryDataServices shouldBe Seq((primaryConnection, primaryRef)) + updated.evDataService shouldBe Some(evRef) + updated.resultListeners shouldBe empty + updated.resultProviders shouldBe Seq(resultServiceProxyRef) } "return evDataService correctly" in { @@ -176,41 +162,16 @@ class ExtSimSetupDataSpec extends ScalaTestWithActorTestKit with UnitSpec { val evRef = TestProbe[ExtEvDataService.Message]("ev_service").ref val cases = Table( - ("extSimSetupData", "expectedConnection", "expectedService"), - ( - ExtSimSetupData.apply.update(evConnection, evRef), - Some(evConnection), - Some(evRef), - ), - (ExtSimSetupData.apply, None, None), + ("extSimSetupData", "expectedService"), + (ExtSimSetupData.apply.update(evConnection, evRef), Some(evRef)), + (ExtSimSetupData.apply, None), ) - forAll(cases) { (extSimSetupData, expectedConnection, expectedService) => - extSimSetupData.evDataConnection shouldBe expectedConnection + forAll(cases) { (extSimSetupData, expectedService) => extSimSetupData.evDataService shouldBe expectedService } } - "return emDataService correctly" in { - val emConnection = new ExtEmDataConnection(emptyUuidList, EmMode.BASE) - val emRef = TestProbe[ServiceMessage]("em_service").ref - - val cases = Table( - ("extSimSetupData", "expectedConnection", "expectedService"), - ( - ExtSimSetupData.apply.update(emConnection, emRef), - Some(emConnection), - Some(emRef), - ), - (ExtSimSetupData.apply, None, None), - ) - - forAll(cases) { (extSimSetupData, expectedConnection, expectedService) => - extSimSetupData.emDataConnection shouldBe expectedConnection - extSimSetupData.emDataService shouldBe expectedService - } - } - } } diff --git a/src/test/scala/edu/ie3/simona/sim/setup/ExtSimSetupSpec.scala b/src/test/scala/edu/ie3/simona/sim/setup/ExtSimSetupSpec.scala index 6a7bbc4d74..9fa01a4912 100644 --- a/src/test/scala/edu/ie3/simona/sim/setup/ExtSimSetupSpec.scala +++ b/src/test/scala/edu/ie3/simona/sim/setup/ExtSimSetupSpec.scala @@ -25,7 +25,7 @@ class ExtSimSetupSpec extends UnitSpec { val uuid5 = UUID.fromString("ebcefed4-a3e6-4a2a-b4a5-74226d548546") val uuid6 = UUID.fromString("4a9c8e14-c0ee-425b-af40-9552b9075414") - def toMap(uuids: Set[UUID]): java.util.Map[UUID, Class[Value]] = uuids + def toMap(uuids: Set[UUID]): java.util.Map[UUID, Class[? <: Value]] = uuids .map(uuid => uuid -> classOf[Value]) .toMap .asJava diff --git a/src/test/scala/edu/ie3/simona/sim/setup/SimonaSetupSpec.scala b/src/test/scala/edu/ie3/simona/sim/setup/SimonaSetupSpec.scala index 6819e6adc5..f9a01eb4f8 100644 --- a/src/test/scala/edu/ie3/simona/sim/setup/SimonaSetupSpec.scala +++ b/src/test/scala/edu/ie3/simona/sim/setup/SimonaSetupSpec.scala @@ -14,12 +14,17 @@ import edu.ie3.datamodel.models.input.connector.{ import edu.ie3.simona.agent.EnvironmentRefs import edu.ie3.simona.agent.grid.GridAgent import edu.ie3.simona.config.SimonaConfig -import edu.ie3.simona.event.listener.{ResultEventListener, RuntimeEventListener} +import edu.ie3.simona.event.listener.{ResultListener, RuntimeEventListener} import edu.ie3.simona.event.{ResultEvent, RuntimeEvent} import edu.ie3.simona.ontology.messages.{SchedulerMessage, ServiceMessage} +import edu.ie3.simona.ontology.messages.ResultMessage.{ + RequestResult, + ResultResponse, +} import edu.ie3.simona.scheduler.TimeAdvancer import edu.ie3.simona.scheduler.core.Core.CoreFactory import edu.ie3.simona.scheduler.core.RegularSchedulerCore +import edu.ie3.simona.service.results.ResultServiceProxy import edu.ie3.simona.sim.SimonaSim import edu.ie3.simona.test.common.model.grid.SubGridGateMokka import edu.ie3.simona.test.common.{ConfigTestData, UnitSpec} @@ -27,6 +32,7 @@ import org.apache.pekko.actor.typed.ActorRef import org.apache.pekko.actor.typed.scaladsl.ActorContext import java.nio.file.Path +import java.time.ZonedDateTime import java.util.UUID class SimonaSetupSpec @@ -49,7 +55,7 @@ class SimonaSetupSpec override def resultEventListener( context: ActorContext[?] - ): Seq[ActorRef[ResultEventListener.Request]] = + ): Seq[ActorRef[ResultListener.Message]] = throw new NotImplementedException("This is a dummy setup") override def primaryServiceProxy( @@ -60,6 +66,14 @@ class SimonaSetupSpec "This is a dummy setup" ) + override def resultServiceProxy( + context: ActorContext[?], + listeners: Seq[ActorRef[ResultResponse]], + simStartTime: ZonedDateTime, + ): ActorRef[ResultServiceProxy.Message] = throw new NotImplementedException( + "This is a dummy setup" + ) + override def weatherService( context: ActorContext[?], scheduler: ActorRef[SchedulerMessage], @@ -77,6 +91,7 @@ class SimonaSetupSpec override def extSimulations( context: ActorContext[?], scheduler: ActorRef[SchedulerMessage], + resultProxy: ActorRef[RequestResult], extSimPath: Option[Path], ): ExtSimSetupData = throw new NotImplementedException( "This is a dummy setup" @@ -101,7 +116,6 @@ class SimonaSetupSpec override def gridAgents( context: ActorContext[?], environmentRefs: EnvironmentRefs, - resultEventListeners: Seq[ActorRef[ResultEvent]], ): Iterable[ActorRef[GridAgent.Message]] = throw new NotImplementedException("This is a dummy setup") diff --git a/src/test/scala/edu/ie3/simona/test/common/model/grid/DbfsTestGrid.scala b/src/test/scala/edu/ie3/simona/test/common/model/grid/DbfsTestGrid.scala index 48d46928a3..c40233138f 100644 --- a/src/test/scala/edu/ie3/simona/test/common/model/grid/DbfsTestGrid.scala +++ b/src/test/scala/edu/ie3/simona/test/common/model/grid/DbfsTestGrid.scala @@ -413,4 +413,13 @@ trait DbfsTestGrid extends SubGridGateMokka { subGridGates, ) } + + protected val assetsHv: Seq[UUID] = + hvGridContainer.allEntitiesAsList.asScala.map(_.getUuid).toSeq + protected val assetsHvPF: Seq[UUID] = + hvGridContainerPF.allEntitiesAsList.asScala.map(_.getUuid).toSeq + + protected val assetsEhv: Seq[UUID] = + ehvGridContainer.allEntitiesAsList.asScala.map(_.getUuid).toSeq + } diff --git a/src/test/scala/edu/ie3/simona/test/common/result/PowerFlowResultData.scala b/src/test/scala/edu/ie3/simona/test/common/result/PowerFlowResultData.scala index f4ea226071..08fcd183da 100644 --- a/src/test/scala/edu/ie3/simona/test/common/result/PowerFlowResultData.scala +++ b/src/test/scala/edu/ie3/simona/test/common/result/PowerFlowResultData.scala @@ -7,9 +7,8 @@ package edu.ie3.simona.test.common.result import java.util.UUID - import edu.ie3.datamodel.models.StandardUnits -import edu.ie3.datamodel.models.result.NodeResult +import edu.ie3.datamodel.models.result.{NodeResult, ResultEntity} import edu.ie3.datamodel.models.result.connector.{ LineResult, SwitchResult, @@ -28,6 +27,12 @@ trait PowerFlowResultData { private val dummyInputModel = UUID.fromString("e5ac84d3-c7a5-4870-a42d-837920aec9bb") + given Conversion[ResultEntity, Map[UUID, Iterable[ResultEntity]]] = + (res: ResultEntity) => Map(res.getInputModel -> Iterable(res)) + + given Conversion[Iterable[ResultEntity], Map[UUID, Iterable[ResultEntity]]] = + (res: Iterable[ResultEntity]) => res.groupBy(_.getInputModel) + val dummyPvResult = new PvResult( dummyTime, dummyInputModel, From 147c85847282b9a14b8bf4a7996b8f8857382837 Mon Sep 17 00:00:00 2001 From: staudtMarius Date: Tue, 16 Sep 2025 08:31:34 +0200 Subject: [PATCH 2/6] Adding test for `ResultServiceProxy` and `ExtResultProvider`. --- .../service/results/ResultServiceProxy.scala | 9 +- .../results/ExtResultProviderSpec.scala | 95 ++++++ .../results/ResultServiceProxySpec.scala | 270 ++++++++++++++++++ .../simona/test/common/ConfigTestData.scala | 3 + .../result/CongestedComponentsTestData.scala | 5 - .../common/result/PowerFlowResultData.scala | 2 +- 6 files changed, 371 insertions(+), 13 deletions(-) create mode 100644 src/test/scala/edu/ie3/simona/service/results/ExtResultProviderSpec.scala create mode 100644 src/test/scala/edu/ie3/simona/service/results/ResultServiceProxySpec.scala diff --git a/src/main/scala/edu/ie3/simona/service/results/ResultServiceProxy.scala b/src/main/scala/edu/ie3/simona/service/results/ResultServiceProxy.scala index 245d7e2da2..69a325adf1 100644 --- a/src/main/scala/edu/ie3/simona/service/results/ResultServiceProxy.scala +++ b/src/main/scala/edu/ie3/simona/service/results/ResultServiceProxy.scala @@ -21,7 +21,6 @@ import edu.ie3.simona.service.results.Transformer3wResultSupport.{ AggregatedTransformer3wResult, Transformer3wKey, } -import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK import edu.ie3.simona.util.TickUtil.RichZonedDateTime import org.apache.pekko.actor.typed.scaladsl.{Behaviors, StashBuffer} import org.apache.pekko.actor.typed.{ActorRef, Behavior, PostStop} @@ -41,7 +40,6 @@ object ResultServiceProxy { private final case class ResultServiceStateData( listeners: Seq[ActorRef[ResultResponse]], simStartTime: ZonedDateTime, - currentTick: Long = INIT_SIM_TICK, threeWindingResults: Map[ Transformer3wKey, AggregatedTransformer3wResult, @@ -51,7 +49,7 @@ object ResultServiceProxy { waitingForResults: Map[UUID, Long] = Map.empty, ) extends ServiceBaseStateData { def notifyListener(results: Map[UUID, Iterable[ResultEntity]]): Unit = - listeners.foreach(_ ! ResultResponse(results)) + if results.nonEmpty then listeners.foreach(_ ! ResultResponse(results)) def notifyListener(result: ResultEntity): Unit = listeners.foreach( @@ -67,9 +65,6 @@ object ResultServiceProxy { } } - def updateTick(tick: Long): ResultServiceStateData = - copy(currentTick = tick) - def waitForResult(expectResult: ExpectResult): ResultServiceStateData = expectResult.assets match { case uuid: UUID => @@ -152,7 +147,7 @@ object ResultServiceProxy { // un-stash received requests buffer.unstashAll(idle(updatedStateData)) - case (ctx, requestResultMessage: RequestResult) => + case (_, requestResultMessage: RequestResult) => val requestedResults = requestResultMessage.requestedResults val tick = requestResultMessage.tick diff --git a/src/test/scala/edu/ie3/simona/service/results/ExtResultProviderSpec.scala b/src/test/scala/edu/ie3/simona/service/results/ExtResultProviderSpec.scala new file mode 100644 index 0000000000..bbca74d124 --- /dev/null +++ b/src/test/scala/edu/ie3/simona/service/results/ExtResultProviderSpec.scala @@ -0,0 +1,95 @@ +/* + * © 2025. TU Dortmund University, + * Institute of Energy Systems, Energy Efficiency and Energy Economics, + * Research group Distribution grid planning and operation + */ + +package edu.ie3.simona.service.results + +import edu.ie3.simona.api.data.connection.ExtResultDataConnection +import edu.ie3.simona.api.ontology.ScheduleDataServiceMessage +import edu.ie3.simona.api.ontology.results.{ + ProvideResultEntities, + RequestResultEntities, +} +import edu.ie3.simona.api.ontology.simulation.ControlResponseMessageFromExt +import edu.ie3.simona.ontology.messages.ResultMessage.{ + RequestResult, + ResultResponse, +} +import edu.ie3.simona.ontology.messages.SchedulerMessage.Completion +import edu.ie3.simona.ontology.messages.{Activation, SchedulerMessage} +import edu.ie3.simona.test.common.UnitSpec +import edu.ie3.simona.test.common.result.PowerFlowResultData +import org.apache.pekko.actor.testkit.typed.scaladsl.{ + ScalaTestWithActorTestKit, + TestProbe, +} + +import scala.jdk.CollectionConverters.{ + ListHasAsScala, + SeqHasAsJava, + SetHasAsScala, +} + +class ExtResultProviderSpec + extends ScalaTestWithActorTestKit + with UnitSpec + with PowerFlowResultData { + + private val scheduler = TestProbe[SchedulerMessage]("scheduler") + private val resultProxy = TestProbe[RequestResult]("resultProxy") + + "The ExtResultProvider" should { + + "handle result responses correctly" in { + val connection = new ExtResultDataConnection(List(dummyInputModel).asJava) + val extSimAdapter = + TestProbe[ControlResponseMessageFromExt]("extSimAdapter") + val provider = + spawn(ExtResultProvider(connection, scheduler.ref, resultProxy.ref)) + connection.setActorRefs(provider.ref, extSimAdapter.ref) + + provider ! ResultResponse( + Map(dummyInputModel -> List(dummyNodeResult, dummyPvResult)) + ) + + val results = + connection.receiveWithType(classOf[ProvideResultEntities]).results + results.keySet.asScala shouldBe Set(dummyInputModel) + results.get(dummyInputModel).asScala shouldBe List( + dummyNodeResult, + dummyPvResult, + ) + + scheduler.expectMessage(Completion(provider.ref)) + } + + "handle result data message from external" in { + val connection = new ExtResultDataConnection(List(dummyInputModel).asJava) + val extSimAdapter = + TestProbe[ControlResponseMessageFromExt]("extSimAdapter") + val provider = + spawn(ExtResultProvider(connection, scheduler.ref, resultProxy.ref)) + connection.setActorRefs(provider.ref, extSimAdapter.ref) + + // requesting results from the result provider + connection.sendExtMsg( + new RequestResultEntities(3600L, List(dummyInputModel).asJava) + ) + + extSimAdapter.expectMessage(new ScheduleDataServiceMessage(provider.ref)) + provider ! Activation(3600L) + + resultProxy.expectMessage( + RequestResult(Seq(dummyInputModel), 3600L, provider.ref) + ) + + provider ! ResultResponse(Map(dummyInputModel -> List(dummyPvResult))) + + scheduler.expectMessage(Completion(provider.ref)) + } + + } + +} diff --git a/src/test/scala/edu/ie3/simona/service/results/ResultServiceProxySpec.scala b/src/test/scala/edu/ie3/simona/service/results/ResultServiceProxySpec.scala new file mode 100644 index 0000000000..17eca14dbd --- /dev/null +++ b/src/test/scala/edu/ie3/simona/service/results/ResultServiceProxySpec.scala @@ -0,0 +1,270 @@ +/* + * © 2025. TU Dortmund University, + * Institute of Energy Systems, Energy Efficiency and Energy Economics, + * Research group Distribution grid planning and operation + */ + +package edu.ie3.simona.service.results + +import edu.ie3.simona.event.ResultEvent.{ + ParticipantResultEvent, + PowerFlowResultEvent, +} +import edu.ie3.simona.ontology.messages.ResultMessage +import edu.ie3.simona.ontology.messages.ResultMessage.{ + RequestResult, + ResultResponse, +} +import edu.ie3.simona.service.results.ResultServiceProxy.ExpectResult +import edu.ie3.simona.test.common.result.PowerFlowResultData +import edu.ie3.simona.test.common.{ConfigTestData, UnitSpec} +import org.apache.pekko.actor.testkit.typed.scaladsl.{ + ScalaTestWithActorTestKit, + TestProbe, +} + +class ResultServiceProxySpec + extends ScalaTestWithActorTestKit + with UnitSpec + with ConfigTestData + with PowerFlowResultData + with ThreeWindingResultTestData { + + "The ResultServiceProxy" should { + + "answer request for results correctly without waiting for results" in { + val resultProvider = TestProbe[ResultMessage.Response]("listener") + + val resultProxy = spawn(ResultServiceProxy(Seq.empty, startTime, 10)) + + resultProxy ! RequestResult( + Seq(dummyInputModel, inputModel), + 3600L, + resultProvider.ref, + ) + + // no results, since the result proxy received not waiting for result information + resultProvider + .expectMessageType[ResultResponse] + .results shouldBe Map.empty + } + + "answer request for results correctly with waiting for some results" in { + val resultProvider = TestProbe[ResultMessage.Response]("listener") + + val resultProxy = spawn(ResultServiceProxy(Seq.empty, startTime, 10)) + + // tells the proxy to wait for the results of dummyInputModel for tick 3600L + resultProxy ! ExpectResult(Seq(dummyInputModel), 3600L) + + resultProxy ! RequestResult( + Seq(dummyInputModel, inputModel), + 3600L, + resultProvider.ref, + ) + + // still waiting for results + resultProvider.expectNoMessage() + + resultProxy ! PowerFlowResultEvent( + Seq(dummyNodeResult), + Seq(dummySwitchResult), + Seq(dummyLineResult), + Seq(dummyTrafo2wResult), + Seq(resultA), + ) + + // no results for three winding transformers, because the proxy is not told to wait and the results was not received beforehand + resultProvider.expectMessageType[ResultResponse].results shouldBe Map( + dummyInputModel -> List( + dummyNodeResult, + dummySwitchResult, + dummyLineResult, + dummyTrafo2wResult, + ) + ) + } + + "answer request for results correctly with waiting for some results with different receive order" in { + val resultProvider = TestProbe[ResultMessage.Response]("listener") + + val resultProxy = spawn(ResultServiceProxy(Seq.empty, startTime, 10)) + + // tells the proxy to wait for the results with dumyInputModel for tick 3600L + resultProxy ! ExpectResult(Seq(dummyInputModel), 3600L) + + resultProxy ! RequestResult( + Seq(dummyInputModel, inputModel), + 3600L, + resultProvider.ref, + ) + + // receiving three winding results for port B and C beforehand + resultProxy ! PowerFlowResultEvent( + Seq.empty, + Seq.empty, + Seq.empty, + Seq.empty, + Seq(resultB, resultC), + ) + + // still waiting for results + resultProvider.expectNoMessage() + + resultProxy ! PowerFlowResultEvent( + Seq(dummyNodeResult), + Seq(dummySwitchResult), + Seq(dummyLineResult), + Seq(dummyTrafo2wResult), + Seq(resultA), + ) + + // receives three winding result, because all partial results are present + resultProvider.expectMessageType[ResultResponse].results shouldBe Map( + dummyInputModel -> List( + dummyNodeResult, + dummySwitchResult, + dummyLineResult, + dummyTrafo2wResult, + ), + inputModel -> List(expected), + ) + } + + "answer request for results correctly with waiting for all results" in { + val resultProvider = TestProbe[ResultMessage.Response]("listener") + + val resultProxy = spawn(ResultServiceProxy(Seq.empty, startTime, 10)) + + // tells the proxy to wait for the results of dumyInputModel for tick 3600L + resultProxy ! ExpectResult(Seq(dummyInputModel), 3600L) + + // tells the proxy to also wait for the results of inputModel for tick 3600L + resultProxy ! ExpectResult(Seq(inputModel), 3600L) + + resultProxy ! RequestResult( + Seq(dummyInputModel, inputModel), + 3600L, + resultProvider.ref, + ) + + // still waiting for results + resultProvider.expectNoMessage() + + resultProxy ! PowerFlowResultEvent( + Seq(dummyNodeResult), + Seq(dummySwitchResult), + Seq(dummyLineResult), + Seq(dummyTrafo2wResult), + Seq(resultA), + ) + + // still waiting for results + resultProvider.expectNoMessage() + + // receiving three winding results for port B and C + resultProxy ! PowerFlowResultEvent( + Seq.empty, + Seq.empty, + Seq.empty, + Seq.empty, + Seq(resultB, resultC), + ) + + // no results for three winding transformers, because the proxy is not told to wait and the results was not received beforehand + resultProvider.expectMessageType[ResultResponse].results shouldBe Map( + dummyInputModel -> List( + dummyNodeResult, + dummySwitchResult, + dummyLineResult, + dummyTrafo2wResult, + ), + inputModel -> List(expected), + ) + } + + "correctly handle grid result events" in { + val listener = TestProbe[ResultResponse]("listener") + + val resultProxy = + spawn(ResultServiceProxy(Seq(listener.ref), startTime, 10)) + + resultProxy ! PowerFlowResultEvent( + Seq(dummyNodeResult), + Seq(dummySwitchResult), + Seq(dummyLineResult), + Seq(dummyTrafo2wResult), + Seq.empty, + ) + + // all results have the same uuid, therefore, all result a grouped to this uuid + listener.expectMessageType[ResultResponse].results shouldBe Map( + dummyInputModel -> List( + dummyNodeResult, + dummySwitchResult, + dummyLineResult, + dummyTrafo2wResult, + ) + ) + } + + "correctly handle three winding transformer result events" in { + val listener = TestProbe[ResultResponse]("listener") + + val resultProxy = + spawn(ResultServiceProxy(Seq(listener.ref), startTime, 10)) + + // sending result for port A + resultProxy ! PowerFlowResultEvent( + Seq.empty, + Seq.empty, + Seq.empty, + Seq.empty, + Seq(resultA), + ) + + // no message, because the three winding result is not complete + listener.expectNoMessage() + + // sending result for port C + resultProxy ! PowerFlowResultEvent( + Seq.empty, + Seq.empty, + Seq.empty, + Seq.empty, + Seq(resultC), + ) + + // no message, because the three winding result is not complete + listener.expectNoMessage() + + // sending result for port B + resultProxy ! PowerFlowResultEvent( + Seq.empty, + Seq.empty, + Seq.empty, + Seq.empty, + Seq(resultB), + ) + + listener.expectMessageType[ResultResponse].results shouldBe Map( + inputModel -> List(expected) + ) + } + + "correctly handle participant result events" in { + val listener = TestProbe[ResultResponse]("listener") + + val resultProxy = + spawn(ResultServiceProxy(Seq(listener.ref), startTime, 10)) + + resultProxy ! ParticipantResultEvent(dummyPvResult) + + listener.expectMessageType[ResultResponse].results shouldBe Map( + dummyPvResult.getInputModel -> List(dummyPvResult) + ) + } + + } + +} diff --git a/src/test/scala/edu/ie3/simona/test/common/ConfigTestData.scala b/src/test/scala/edu/ie3/simona/test/common/ConfigTestData.scala index 246548e7d2..c77fe63717 100644 --- a/src/test/scala/edu/ie3/simona/test/common/ConfigTestData.scala +++ b/src/test/scala/edu/ie3/simona/test/common/ConfigTestData.scala @@ -9,6 +9,8 @@ package edu.ie3.simona.test.common import com.typesafe.config.{Config, ConfigFactory} import edu.ie3.simona.config.SimonaConfig +import java.time.ZonedDateTime + /** Simple (empty) configuration data. Furthermore, it would make sense to * implement another class which reads a config and provides config based * values in the future. @@ -151,4 +153,5 @@ trait ConfigTestData { ) protected val simonaConfig: SimonaConfig = SimonaConfig(typesafeConfig) + protected val startTime: ZonedDateTime = simonaConfig.simona.time.simStartTime } diff --git a/src/test/scala/edu/ie3/simona/test/common/result/CongestedComponentsTestData.scala b/src/test/scala/edu/ie3/simona/test/common/result/CongestedComponentsTestData.scala index 1f938f4da3..fceb3acf1e 100644 --- a/src/test/scala/edu/ie3/simona/test/common/result/CongestedComponentsTestData.scala +++ b/src/test/scala/edu/ie3/simona/test/common/result/CongestedComponentsTestData.scala @@ -20,7 +20,6 @@ import edu.ie3.simona.model.grid.* import edu.ie3.simona.test.common.ConfigTestData import edu.ie3.simona.test.common.input.NodeInputTestData import edu.ie3.simona.test.common.model.grid.DbfsTestGrid -import edu.ie3.util.TimeUtil import edu.ie3.util.quantities.QuantityUtils.* import squants.electro.Kilovolts import squants.energy.Megawatts @@ -34,10 +33,6 @@ trait CongestedComponentsTestData with NodeInputTestData with DbfsTestGrid { - val startTime: ZonedDateTime = TimeUtil.withDefaults.toZonedDateTime( - simonaConfig.simona.time.startDateTime - ) - val endTime: ZonedDateTime = startTime.plusHours(2) val trafoType3W = new Transformer3WTypeInput( diff --git a/src/test/scala/edu/ie3/simona/test/common/result/PowerFlowResultData.scala b/src/test/scala/edu/ie3/simona/test/common/result/PowerFlowResultData.scala index 08fcd183da..b9a71382e6 100644 --- a/src/test/scala/edu/ie3/simona/test/common/result/PowerFlowResultData.scala +++ b/src/test/scala/edu/ie3/simona/test/common/result/PowerFlowResultData.scala @@ -24,7 +24,7 @@ trait PowerFlowResultData { private val dummyTime = TimeUtil.withDefaults.toZonedDateTime("2020-01-30T17:26:44Z") - private val dummyInputModel = + protected val dummyInputModel: UUID = UUID.fromString("e5ac84d3-c7a5-4870-a42d-837920aec9bb") given Conversion[ResultEntity, Map[UUID, Iterable[ResultEntity]]] = From 55a590996a53d75af8ac9663463dc387fd7b9341 Mon Sep 17 00:00:00 2001 From: staudtMarius Date: Tue, 28 Oct 2025 15:33:49 +0100 Subject: [PATCH 3/6] Fixing issues after merging dev. --- .../edu/ie3/simona/agent/em/EmAgent.scala | 4 +-- .../simona/sim/setup/ExtSimSetupData.scala | 20 ++++++++----- .../agent/em/EmAgentWithServiceSpec.scala | 30 +++++++++---------- .../sim/setup/ExtSimSetupDataSpec.scala | 4 +-- 4 files changed, 32 insertions(+), 26 deletions(-) diff --git a/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala b/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala index 16ce240386..310842570c 100644 --- a/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala +++ b/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala @@ -63,7 +63,7 @@ object EmAgent { * agent is em-controlled, or a [[Left]] with a reference to the scheduler * that is activating this agent. * @param listener - * A listener for result events. + * A listener for result events. * @param emDataService * An energy management service. */ @@ -74,7 +74,7 @@ object EmAgent { modelStrategy: String, simulationStartDate: ZonedDateTime, parent: Either[ActorRef[SchedulerMessage], ActorRef[FlexResponse]], - listener: Iterable[ActorRef[ResultEvent]], + listener: ActorRef[ResultEvent], emDataService: Option[ActorRef[ExtEmDataService.Message]] = None, ): Behavior[Message] = Behaviors.setup[Message] { ctx => 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 916defb7ee..c6583440d7 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/ExtSimSetupData.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/ExtSimSetupData.scala @@ -26,10 +26,10 @@ import org.apache.pekko.actor.typed.ActorRef * Option for an external em data service. * @param evDataService * Option for an external ev data service. - * @param resultListeners - * Seq: external result listeners. - * @param resultProviders - * Seq: external result providers. + * @param resultListeners + * Seq: external result listeners. + * @param resultProviders + * Seq: external result providers. */ final case class ExtSimSetupData( extSimAdapters: Iterable[ActorRef[ExtSimAdapter.Request]], @@ -67,8 +67,13 @@ final case class ExtSimSetupData( serviceRef: ActorRef[ExtEvDataService.Message], ) => copy(evDataService = Some(serviceRef)) - case (_: ExtResultDataConnection, serviceRef: ActorRef[ServiceMessage]) => - copy(extResultListeners = extResultListeners ++ Seq(serviceRef)) + case (_: ExtResultListener, serviceRef: ActorRef[ResultListener.Message]) => + copy(resultListeners = resultListeners ++ Seq(serviceRef)) + case ( + _: ExtResultDataConnection, + serviceRef: ActorRef[ExtResultProvider.Message], + ) => + copy(resultProviders = resultProviders ++ Seq(serviceRef)) case (_, _) => this } @@ -87,7 +92,8 @@ final case class ExtSimSetupData( Seq( emDataService, evDataService, - ).flatten ++ extResultListeners ++ resultProviders ++ primaryDataServices.map(_._2) + ).flatten ++ resultListeners ++ resultProviders ++ primaryDataServices + .map(_._2) } object ExtSimSetupData { diff --git a/src/test/scala/edu/ie3/simona/agent/em/EmAgentWithServiceSpec.scala b/src/test/scala/edu/ie3/simona/agent/em/EmAgentWithServiceSpec.scala index 23c085f585..7e46b7efd4 100644 --- a/src/test/scala/edu/ie3/simona/agent/em/EmAgentWithServiceSpec.scala +++ b/src/test/scala/edu/ie3/simona/agent/em/EmAgentWithServiceSpec.scala @@ -73,7 +73,7 @@ class EmAgentWithServiceSpec "An EM-controlled EM agent with em service" should { "be initialized correctly and run through some activations" in { - val resultListener = TestProbe[ResultEvent]("ResultListener") + val resultServiceProxy = TestProbe[ResultEvent]("ResultListener") val parentEmAgent = TestProbe[EmAgent.Message]("ParentEmAgent") @@ -88,7 +88,7 @@ class EmAgentWithServiceSpec "PRIORITIZED", simulationStartDate, parent = Right(parentEmAgent.ref), - listener = Iterable(resultListener.ref), + listener = resultServiceProxy.ref, Some(serviceRef), ) ) @@ -152,7 +152,7 @@ class EmAgentWithServiceSpec ) // expect no results for init - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() // expect completion from EmAgent service.expectMessage( EmFlexMessage( @@ -193,7 +193,7 @@ class EmAgentWithServiceSpec ), ) - resultListener.expectMessageType[FlexOptionsResultEvent] match { + resultServiceProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(flexResult) => flexResult.getInputModel shouldBe emInput.getUuid flexResult.getTime shouldBe 0.toDateTime @@ -255,7 +255,7 @@ class EmAgentWithServiceSpec ) // expect correct results - resultListener.expectMessageType[ParticipantResultEvent] match { + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid emResult.getTime shouldBe 0.toDateTime @@ -313,7 +313,7 @@ class EmAgentWithServiceSpec ) // expect correct results - resultListener.expectMessageType[ParticipantResultEvent] match { + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid emResult.getTime shouldBe 150.toDateTime @@ -345,7 +345,7 @@ class EmAgentWithServiceSpec } "communicate with parent em through em service" in { - val resultListener = TestProbe[ResultEvent]("ResultListener") + val resultServiceProxy = TestProbe[ResultEvent]("ResultListener") val scheduler = TestProbe[SchedulerMessage]("Scheduler") val service = TestProbe[ExtEmDataService.Message]("emService") @@ -367,7 +367,7 @@ class EmAgentWithServiceSpec "PROPORTIONAL", simulationStartDate, parent = Left(scheduler.ref), - listener = Iterable(resultListener.ref), + listener = resultServiceProxy.ref, Some(serviceRef), ) ) @@ -393,7 +393,7 @@ class EmAgentWithServiceSpec "PRIORITIZED", simulationStartDate, parent = Right(parentEmAgent), - listener = Iterable(resultListener.ref), + listener = resultServiceProxy.ref, Some(serviceRef), ) ) @@ -471,7 +471,7 @@ class EmAgentWithServiceSpec ) // expect no results for init - resultListener.expectNoMessage() + resultServiceProxy.expectNoMessage() // expect completion from EmAgent service.expectMessage( EmFlexMessage( @@ -536,7 +536,7 @@ class EmAgentWithServiceSpec ), ) - resultListener.expectMessageType[FlexOptionsResultEvent] match { + resultServiceProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(flexResult) => flexResult.getInputModel shouldBe updatedEmInput.getUuid flexResult.getTime shouldBe 0.toDateTime @@ -636,14 +636,14 @@ class EmAgentWithServiceSpec ) // expect correct results - resultListener.expectMessageType[FlexOptionsResultEvent] match { + resultServiceProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(flexOptionsResult) => flexOptionsResult.getpRef should equalWithTolerance(0.asMegaWatt) flexOptionsResult.getpMin should equalWithTolerance(-0.016.asMegaWatt) flexOptionsResult.getpMax should equalWithTolerance(0.006.asMegaWatt) } - resultListener.expectMessageType[ParticipantResultEvent] match { + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe updatedEmInput.getUuid emResult.getTime shouldBe 0.toDateTime @@ -696,7 +696,7 @@ class EmAgentWithServiceSpec modelUuid shouldBe parentEmInput.getUuid } - resultListener.expectMessageType[ParticipantResultEvent] match { + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe parentEmInput.getUuid emResult.getTime shouldBe 0.toDateTime @@ -746,7 +746,7 @@ class EmAgentWithServiceSpec ) // expect correct results - resultListener.expectMessageType[ParticipantResultEvent] match { + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe updatedEmInput.getUuid emResult.getTime shouldBe 150.toDateTime diff --git a/src/test/scala/edu/ie3/simona/sim/setup/ExtSimSetupDataSpec.scala b/src/test/scala/edu/ie3/simona/sim/setup/ExtSimSetupDataSpec.scala index 329c4402aa..89155bff17 100644 --- a/src/test/scala/edu/ie3/simona/sim/setup/ExtSimSetupDataSpec.scala +++ b/src/test/scala/edu/ie3/simona/sim/setup/ExtSimSetupDataSpec.scala @@ -141,7 +141,7 @@ class ExtSimSetupDataSpec extends ScalaTestWithActorTestKit with UnitSpec { updated.emDataService shouldBe None updated.evDataService shouldBe None updated.resultListeners shouldBe empty - updated.resultProviders shouldBe Seq(resultRef) + updated.resultProviders shouldBe Seq(resultServiceProxyRef) } "be updated with multiple different connections correctly" in { @@ -177,7 +177,7 @@ class ExtSimSetupDataSpec extends ScalaTestWithActorTestKit with UnitSpec { updated.emDataService shouldBe Some(emRef) updated.evDataService shouldBe Some(evRef) updated.resultListeners shouldBe empty - updated.resultProviders shouldBe Seq(resultRef) + updated.resultProviders shouldBe Seq(resultServiceProxyRef) } "return evDataService correctly" in { From ee6e0ddd9a3322666b3580d2fd6765d2b751b59c Mon Sep 17 00:00:00 2001 From: staudtMarius Date: Wed, 26 Nov 2025 17:11:40 +0100 Subject: [PATCH 4/6] Enhancing `ExpectResult` message. --- .../agent/participant/ParticipantAgent.scala | 14 ++ .../ParticipantResultHandler.scala | 8 +- .../service/results/ResultServiceProxy.scala | 34 ++++- .../edu/ie3/simona/agent/em/EmAgentIT.scala | 127 +++++++++++++++++- .../agent/grid/DBFSAlgorithmCenGridSpec.scala | 3 +- .../DBFSAlgorithmFailedPowerFlowSpec.scala | 9 +- .../participant/ParticipantAgentSpec.scala | 82 ++++++++--- 7 files changed, 246 insertions(+), 31 deletions(-) diff --git a/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgent.scala b/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgent.scala index 110a220887..01cc941723 100644 --- a/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgent.scala +++ b/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgent.scala @@ -344,6 +344,20 @@ object ParticipantAgent { ) ) + // determines, if we need to wait for a set point + // we only wait if we received a flex activation + val waitForSetPoint = activation match { + case _: FlexActivation => true + case _ => false + } + + // inform the result proxy that this grid agent will send new results + resultHandler.informProxy( + modelShell.uuid, + activation.tick, + waitForSetPoint, + ) + val (updatedShell, updatedGridAdapter) = Scope(modelShell) .map( _.updateInputData( diff --git a/src/main/scala/edu/ie3/simona/agent/participant/ParticipantResultHandler.scala b/src/main/scala/edu/ie3/simona/agent/participant/ParticipantResultHandler.scala index f3803131e9..e9b604a7e4 100644 --- a/src/main/scala/edu/ie3/simona/agent/participant/ParticipantResultHandler.scala +++ b/src/main/scala/edu/ie3/simona/agent/participant/ParticipantResultHandler.scala @@ -67,7 +67,11 @@ final case class ParticipantResultHandler( resultProxy ! FlexOptionsResultEvent(result) } - def informProxy(uuid: UUID, tick: Long): Unit = - resultProxy ! ExpectResult(uuid, tick) + def informProxy( + uuid: UUID, + tick: Long, + waitForSetPoint: Boolean = false, + ): Unit = + resultProxy ! ExpectResult(uuid, tick, waitForSetPoint) } diff --git a/src/main/scala/edu/ie3/simona/service/results/ResultServiceProxy.scala b/src/main/scala/edu/ie3/simona/service/results/ResultServiceProxy.scala index 69a325adf1..2988a8c62e 100644 --- a/src/main/scala/edu/ie3/simona/service/results/ResultServiceProxy.scala +++ b/src/main/scala/edu/ie3/simona/service/results/ResultServiceProxy.scala @@ -21,6 +21,7 @@ import edu.ie3.simona.service.results.Transformer3wResultSupport.{ AggregatedTransformer3wResult, Transformer3wKey, } +import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK import edu.ie3.simona.util.TickUtil.RichZonedDateTime import org.apache.pekko.actor.typed.scaladsl.{Behaviors, StashBuffer} import org.apache.pekko.actor.typed.{ActorRef, Behavior, PostStop} @@ -35,11 +36,16 @@ object ResultServiceProxy { type Message = ResultEvent | RequestResult | ExpectResult | DelayedStopHelper.StoppingMsg - final case class ExpectResult(assets: UUID | Seq[UUID], tick: Long) + final case class ExpectResult( + assets: UUID | Seq[UUID], + tick: Long, + waitForSetPoint: Boolean = false, + ) private final case class ResultServiceStateData( listeners: Seq[ActorRef[ResultResponse]], simStartTime: ZonedDateTime, + currentTick: Long = INIT_SIM_TICK, threeWindingResults: Map[ Transformer3wKey, AggregatedTransformer3wResult, @@ -47,6 +53,7 @@ object ResultServiceProxy { gridResults: Map[UUID, Iterable[ResultEntity]] = Map.empty, results: Map[UUID, Iterable[ResultEntity]] = Map.empty, waitingForResults: Map[UUID, Long] = Map.empty, + requiresSetPoint: Set[UUID] = Set.empty, ) extends ServiceBaseStateData { def notifyListener(results: Map[UUID, Iterable[ResultEntity]]): Unit = if results.nonEmpty then listeners.foreach(_ ! ResultResponse(results)) @@ -59,25 +66,39 @@ object ResultServiceProxy { def isWaiting(uuids: Iterable[UUID], tick: Long): Boolean = { uuids.exists { uuid => waitingForResults.get(uuid) match { - case Some(nextTick) if nextTick <= tick => true - case _ => false + case Some(nextTick) => + nextTick <= tick && !requiresSetPoint.contains(uuid) + case _ => false } } } - def waitForResult(expectResult: ExpectResult): ResultServiceStateData = + def updateTick(tick: Long): ResultServiceStateData = + copy(currentTick = tick) + + def waitForResult(expectResult: ExpectResult): ResultServiceStateData = { expectResult.assets match { case uuid: UUID => - copy(waitingForResults = + val updated = copy(waitingForResults = waitingForResults.updated(uuid, expectResult.tick) ) + + if expectResult.waitForSetPoint then { + updated.copy(requiresSetPoint = requiresSetPoint + uuid) + } else updated + case uuids: Seq[UUID] => val tick = expectResult.tick - copy(waitingForResults = + val updated = copy(waitingForResults = waitingForResults ++ uuids.map(uuid => uuid -> tick).toMap ) + + if expectResult.waitForSetPoint then { + updated.copy(requiresSetPoint = requiresSetPoint ++ uuids) + } else updated } + } def addResult(result: ResultEntity): ResultServiceStateData = { val uuid = result.getInputModel @@ -105,6 +126,7 @@ object ResultServiceProxy { copy( results = updatedResults, waitingForResults = updatedWaitingForResults, + requiresSetPoint = requiresSetPoint.excl(uuid), ) } diff --git a/src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala b/src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala index 777ec38d51..3386ee5e45 100644 --- a/src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala +++ b/src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala @@ -31,7 +31,6 @@ import edu.ie3.simona.ontology.messages.ServiceMessage.{ SecondaryServiceRegistrationMessage, } import edu.ie3.simona.ontology.messages.{Activation, SchedulerMessage} -import edu.ie3.simona.ontology.messages.ResultMessage.RequestResult import edu.ie3.simona.scheduler.ScheduleLock import edu.ie3.simona.service.Data.SecondaryData.WeatherData import edu.ie3.simona.service.ServiceType @@ -235,6 +234,12 @@ class EmAgentIT */ emAgentActivation ! Activation(0) + // we receive a message for each agent that is not waiting for secondary data + resultServiceProxy.receiveMessages(2) should contain allOf ( + ExpectResult(storageInput.getUuid, 0, true), + ExpectResult(loadInput.getUuid, 0, true) + ) + pvAgent ! DataProvision( 0, weatherService.ref, @@ -247,6 +252,16 @@ class EmAgentIT Some(7200), ) + // we receive a message, since new data arrived + resultServiceProxy.expectMessage(ExpectResult(pvInput.getUuid, 0, true)) + + // we receive update messages, since new set points were provided + resultServiceProxy.receiveMessages(3) should contain allOf ( + ExpectResult(pvInput.getUuid, 0), + ExpectResult(storageInput.getUuid, 0), + ExpectResult(loadInput.getUuid, 0) + ) + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid @@ -266,6 +281,11 @@ class EmAgentIT */ emAgentActivation ! Activation(7200) + // the result proxy will receive ExpectResult messages + resultServiceProxy.expectMessage( + ExpectResult(storageInput.getUuid, 7200, true) + ) + pvAgent ! DataProvision( 7200, weatherService.ref, @@ -278,6 +298,17 @@ class EmAgentIT Some(14400), ) + // we receive a message, since new data arrived + resultServiceProxy.expectMessage( + ExpectResult(pvInput.getUuid, 7200, true) + ) + + // we receive update messages, since new set points were provided + resultServiceProxy.receiveMessages(2) should contain allOf ( + ExpectResult(pvInput.getUuid, 7200), + ExpectResult(storageInput.getUuid, 7200) + ) + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid @@ -297,6 +328,16 @@ class EmAgentIT */ emAgentActivation ! Activation(13246) + // the result proxy will receive ExpectResult messages + resultServiceProxy.expectMessage( + ExpectResult(storageInput.getUuid, 13246, true) + ) + + // we receive an update message, since a new set point were provided + resultServiceProxy.expectMessage( + ExpectResult(storageInput.getUuid, 13246) + ) + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid @@ -331,6 +372,13 @@ class EmAgentIT emAgentActivation ! Activation(14400) + // we receive update messages, since we received an activation and a new set point was provided + resultServiceProxy.receiveMessages(3) should contain allOf ( + ExpectResult(pvInput.getUuid, 14400, true), + ExpectResult(pvInput.getUuid, 14400), + ExpectResult(storageInput.getUuid, 14400) + ) + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid @@ -492,6 +540,11 @@ class EmAgentIT */ emAgentActivation ! Activation(0) + // the result proxy will receive ExpectResult messages + resultServiceProxy.expectMessage( + ExpectResult(loadInput.getUuid, 0, true) + ) + weatherDependentAgents.foreach { _ ! DataProvision( 0, @@ -506,6 +559,19 @@ class EmAgentIT ) } + // we receive a message, since new data arrived + resultServiceProxy.receiveMessages(2) should contain allOf ( + ExpectResult(pvInput.getUuid, 0, true), + ExpectResult(hpInputModel.getUuid, 0, true) + ) + + // we receive update messages, since a new set point was provided + resultServiceProxy.receiveMessages(3) should contain allOf ( + ExpectResult(pvInput.getUuid, 0), + ExpectResult(hpInputModel.getUuid, 0), + ExpectResult(loadInput.getUuid, 0) + ) + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid @@ -539,6 +605,18 @@ class EmAgentIT ) } + // we receive a message, since new data arrived + resultServiceProxy.receiveMessages(2) should contain allOf ( + ExpectResult(pvInput.getUuid, 7200, true), + ExpectResult(hpInputModel.getUuid, 7200, true) + ) + + // we receive update messages, since a new set point was provided + resultServiceProxy.receiveMessages(2) should contain allOf ( + ExpectResult(pvInput.getUuid, 7200), + ExpectResult(hpInputModel.getUuid, 7200) + ) + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid @@ -572,6 +650,18 @@ class EmAgentIT ) } + // we receive a message, since new data arrived + resultServiceProxy.receiveMessages(2) should contain allOf ( + ExpectResult(pvInput.getUuid, 10800, true), + ExpectResult(hpInputModel.getUuid, 10800, true) + ) + + // we receive update messages, since a new set point was provided + resultServiceProxy.receiveMessages(2) should contain allOf ( + ExpectResult(pvInput.getUuid, 10800), + ExpectResult(hpInputModel.getUuid, 10800) + ) + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid @@ -607,6 +697,18 @@ class EmAgentIT ) } + // we receive a message, since new data arrived + resultServiceProxy.receiveMessages(2) should contain allOf ( + ExpectResult(pvInput.getUuid, 11000, true), + ExpectResult(hpInputModel.getUuid, 11000, true) + ) + + // we receive update messages, since a new set point was provided + resultServiceProxy.receiveMessages(2) should contain allOf ( + ExpectResult(pvInput.getUuid, 11000), + ExpectResult(hpInputModel.getUuid, 11000) + ) + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid @@ -642,6 +744,18 @@ class EmAgentIT ) } + // we receive a message, since new data arrived + resultServiceProxy.receiveMessages(2) should contain allOf ( + ExpectResult(pvInput.getUuid, 11500, true), + ExpectResult(hpInputModel.getUuid, 11500, true) + ) + + // we receive update messages, since a new set point was provided + resultServiceProxy.receiveMessages(2) should contain allOf ( + ExpectResult(pvInput.getUuid, 11500), + ExpectResult(hpInputModel.getUuid, 11500) + ) + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid @@ -788,6 +902,9 @@ class EmAgentIT ) } + // the result proxy will receive ExpectResult messages + resultServiceProxy.receiveMessages(3) + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid @@ -819,6 +936,9 @@ class EmAgentIT ) } + // the result proxy will receive ExpectResult messages + resultServiceProxy.receiveMessages(3) + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid @@ -850,6 +970,7 @@ class EmAgentIT emAgentActivation ! Activation(7200) + resultServiceProxy.receiveMessages(3) resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid @@ -866,6 +987,8 @@ class EmAgentIT -> expect P and Q values of PV */ emAgentActivation ! Activation(10800) + + resultServiceProxy.receiveMessages(2) resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid @@ -882,6 +1005,8 @@ class EmAgentIT -> expect P: 0 W Q: 0 var */ emAgentActivation ! Activation(14400) + + resultServiceProxy.receiveMessages(2) resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid diff --git a/src/test/scala/edu/ie3/simona/agent/grid/DBFSAlgorithmCenGridSpec.scala b/src/test/scala/edu/ie3/simona/agent/grid/DBFSAlgorithmCenGridSpec.scala index 35bf9e3b4b..27fdc605ec 100644 --- a/src/test/scala/edu/ie3/simona/agent/grid/DBFSAlgorithmCenGridSpec.scala +++ b/src/test/scala/edu/ie3/simona/agent/grid/DBFSAlgorithmCenGridSpec.scala @@ -155,10 +155,11 @@ class DBFSAlgorithmCenGridSpec centerGridAgent ! Activation(3600) resultProxy.expectMessageType[ExpectResult] match { - case ExpectResult(assets, tick) => + case ExpectResult(assets, tick, waitForSetPoint) => assets match { case uuids: Seq[UUID] => uuids.toSet shouldBe assetsHv.toSet + waitForSetPoint shouldBe false case uuid: UUID => fail(s"Received uuid $uuid, but expected grid asset uuids.") } diff --git a/src/test/scala/edu/ie3/simona/agent/grid/DBFSAlgorithmFailedPowerFlowSpec.scala b/src/test/scala/edu/ie3/simona/agent/grid/DBFSAlgorithmFailedPowerFlowSpec.scala index 2b07656af0..c2c9c808c6 100644 --- a/src/test/scala/edu/ie3/simona/agent/grid/DBFSAlgorithmFailedPowerFlowSpec.scala +++ b/src/test/scala/edu/ie3/simona/agent/grid/DBFSAlgorithmFailedPowerFlowSpec.scala @@ -143,10 +143,11 @@ class DBFSAlgorithmFailedPowerFlowSpec centerGridAgent ! Activation(3600) resultProxy.expectMessageType[ExpectResult] match { - case ExpectResult(assets, tick) => + case ExpectResult(assets, tick, waitForSetPoint) => assets match { case uuids: Seq[UUID] => uuids.toSet shouldBe assetsHvPF.toSet + waitForSetPoint shouldBe false case uuid: UUID => fail(s"Received uuid $uuid, but expected grid asset uuids.") } @@ -238,10 +239,11 @@ class DBFSAlgorithmFailedPowerFlowSpec centerGridAgent ! Activation(3600) resultProxy.expectMessageType[ExpectResult] match { - case ExpectResult(assets, tick) => + case ExpectResult(assets, tick, waitForSetPoint) => assets match { case uuids: Seq[UUID] => uuids.toSet shouldBe assetsHvPF.toSet + waitForSetPoint shouldBe false case uuid: UUID => fail(s"Received uuid $uuid, but expected grid asset uuids.") } @@ -362,10 +364,11 @@ class DBFSAlgorithmFailedPowerFlowSpec slackGridAgent ! Activation(3600) resultProxy.expectMessageType[ExpectResult] match { - case ExpectResult(assets, tick) => + case ExpectResult(assets, tick, waitForSetPoint) => assets match { case uuids: Seq[UUID] => uuids.toSet shouldBe assetsEhv.toSet + waitForSetPoint shouldBe false case uuid: UUID => fail(s"Received uuid $uuid, but expected grid asset uuids.") } diff --git a/src/test/scala/edu/ie3/simona/agent/participant/ParticipantAgentSpec.scala b/src/test/scala/edu/ie3/simona/agent/participant/ParticipantAgentSpec.scala index a2d9161d63..919c681cbf 100644 --- a/src/test/scala/edu/ie3/simona/agent/participant/ParticipantAgentSpec.scala +++ b/src/test/scala/edu/ie3/simona/agent/participant/ParticipantAgentSpec.scala @@ -132,6 +132,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! Activation(8 * 3600) + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -207,6 +208,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! Activation(20 * 3600) + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -285,6 +287,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! Activation(8 * 3600) + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -318,7 +321,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { q should approximate(Kilovars(0.968644209676)) } - resultProxy.expectNoMessage() + resultProxy.expectMessageType[ExpectResult] scheduler.expectNoMessage() participantAgent ! GridSimulationFinished(12 * 3600, 24 * 3600) @@ -348,6 +351,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! Activation(20 * 3600) + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -430,6 +434,8 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! Activation(0) + resultProxy.expectMessageType[ExpectResult] + // nothing should happen, still waiting for secondary data... resultProxy.expectNoMessage() scheduler.expectNoMessage() @@ -442,6 +448,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { ) // outside of operation interval, 0 MW + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -468,7 +475,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { Some(12 * 3600), ) - resultProxy.expectNoMessage() + resultProxy.expectMessageType[ExpectResult] scheduler.expectNoMessage() // TICK 8 * 3600: Start of operation interval @@ -481,6 +488,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! Activation(8 * 3600) + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -518,7 +526,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! GridSimulationFinished(12 * 3600, 24 * 3600) // nothing should happen, still waiting for secondary data... - resultProxy.expectNoMessage() + resultProxy.expectMessageType[ExpectResult] scheduler.expectNoMessage() participantAgent ! DataProvision( @@ -529,6 +537,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { ) // calculation should start now + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -550,6 +559,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! Activation(15 * 3600) // nothing should happen, still waiting for secondary data... + resultProxy.expectMessageType[ExpectResult] scheduler.expectNoMessage() participantAgent ! NoDataProvision( @@ -559,7 +569,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { ) // no-op activation, thus no result expected - resultProxy.expectNoMessage() + resultProxy.expectMessageType[ExpectResult] // new data is expected at 18 hours scheduler.expectMessage( @@ -574,7 +584,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! Activation(18 * 3600) // nothing should happen, still waiting for secondary data... - resultProxy.expectNoMessage() + resultProxy.expectMessageType[ExpectResult] scheduler.expectNoMessage() participantAgent ! DataProvision( @@ -585,6 +595,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { ) // calculation should start now + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -604,6 +615,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! Activation(20 * 3600) + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -687,7 +699,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! Activation(0) // nothing should happen, still waiting for primary data... - resultProxy.expectNoMessage() + resultProxy.expectMessageType[ExpectResult] scheduler.expectNoMessage() participantAgent ! DataProvision( @@ -698,6 +710,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { ) // outside of operation interval, 0 MW + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -721,13 +734,14 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { Some(12 * 3600), ) - resultProxy.expectNoMessage() + resultProxy.expectMessageType[ExpectResult] scheduler.expectNoMessage() // TICK 8 * 3600: Start of operation interval participantAgent ! Activation(8 * 3600) + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -762,7 +776,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! GridSimulationFinished(12 * 3600, 24 * 3600) // nothing should happen, still waiting for primary data... - resultProxy.expectNoMessage() + resultProxy.expectMessageType[ExpectResult] scheduler.expectNoMessage() participantAgent ! DataProvision( @@ -773,6 +787,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { ) // calculation should start now + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -791,7 +806,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! Activation(18 * 3600) // nothing should happen, still waiting for primary data... - resultProxy.expectNoMessage() + resultProxy.expectMessageType[ExpectResult] scheduler.expectNoMessage() participantAgent ! DataProvision( @@ -802,6 +817,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { ) // calculation should start now + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -818,6 +834,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! Activation(20 * 3600) + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -913,6 +930,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(3)) } + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -924,6 +942,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssuePowerControl(8 * 3600, Kilowatts(3)) + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -985,6 +1004,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(0)) } + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -996,6 +1016,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssueNoControl(20 * 3600) + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1091,6 +1112,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(3)) } + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1102,6 +1124,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssuePowerControl(8 * 3600, Kilowatts(3)) + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1144,7 +1167,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { q should approximate(Kilovars(0.48432210483)) } - resultProxy.expectNoMessage() + resultProxy.expectMessageType[ExpectResult] em.expectNoMessage() participantAgent ! GridSimulationFinished(12 * 3600, 24 * 3600) @@ -1172,6 +1195,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssueNoControl(12 * 3600) + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1210,6 +1234,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(0)) } + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1221,6 +1246,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssueNoControl(20 * 3600) + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1315,7 +1341,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! FlexActivation(0) // nothing should happen, still waiting for secondary data... - resultProxy.expectNoMessage() + resultProxy.expectMessageType[ExpectResult] em.expectNoMessage() participantAgent ! DataProvision( @@ -1336,6 +1362,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(0)) } + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1348,6 +1375,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssueNoControl(0) // outside of operation interval, 0 MW + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1384,7 +1412,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { Some(12 * 3600), ) - resultProxy.expectNoMessage() + resultProxy.expectMessageType[ExpectResult] em.expectNoMessage() // TICK 8 * 3600: Start of operation interval @@ -1408,6 +1436,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(4)) } + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1419,6 +1448,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssuePowerControl(8 * 3600, Kilowatts(3)) + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1466,7 +1496,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! GridSimulationFinished(12 * 3600, 24 * 3600) // nothing should happen, still waiting for secondary data... - resultProxy.expectNoMessage() + resultProxy.expectMessageType[ExpectResult] em.expectNoMessage() participantAgent ! DataProvision( @@ -1488,6 +1518,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(5)) } + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1499,6 +1530,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssueNoControl(12 * 3600) + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1529,7 +1561,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! FlexActivation(18 * 3600) // nothing should happen, still waiting for secondary data... - resultProxy.expectNoMessage() + resultProxy.expectMessageType[ExpectResult] em.expectNoMessage() participantAgent ! DataProvision( @@ -1551,6 +1583,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(8)) } + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1562,6 +1595,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssueNoControl(18 * 3600) + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1603,6 +1637,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(0)) } + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1614,6 +1649,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssueNoControl(20 * 3600) + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1704,7 +1740,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! FlexActivation(0) // nothing should happen, still waiting for primary data... - resultProxy.expectNoMessage() + resultProxy.expectMessageType[ExpectResult] em.expectNoMessage() participantAgent ! DataProvision( @@ -1725,6 +1761,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(0)) } + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1737,6 +1774,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssueNoControl(0) // outside of operation interval, 0 MW + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1770,7 +1808,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { Some(12 * 3600), ) - resultProxy.expectNoMessage() + resultProxy.expectMessageType[ExpectResult] em.expectNoMessage() // TICK 8 * 3600: Start of operation interval @@ -1788,6 +1826,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(3)) } + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1799,6 +1838,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssuePowerControl(8 * 3600, Kilowatts(3)) + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1843,7 +1883,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! GridSimulationFinished(12 * 3600, 24 * 3600) // nothing should happen, still waiting for primary data... - resultProxy.expectNoMessage() + resultProxy.expectMessageType[ExpectResult] em.expectNoMessage() participantAgent ! DataProvision( @@ -1865,6 +1905,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(6)) } + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1876,6 +1917,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssueNoControl(12 * 3600) + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1903,7 +1945,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! FlexActivation(18 * 3600) // nothing should happen, still waiting for primary data... - resultProxy.expectNoMessage() + resultProxy.expectMessageType[ExpectResult] em.expectNoMessage() participantAgent ! DataProvision( @@ -1925,6 +1967,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(3)) } + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1936,6 +1979,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssueNoControl(18 * 3600) + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1973,6 +2017,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(0)) } + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1984,6 +2029,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssueNoControl(20 * 3600) + resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid From 50c1d4ec07c282ff538b9725eaa7f2d6ff2f9175 Mon Sep 17 00:00:00 2001 From: staudtMarius Date: Thu, 27 Nov 2025 10:16:38 +0100 Subject: [PATCH 5/6] Fixing failing tests. --- .../agent/participant/ParticipantAgent.scala | 2 +- .../edu/ie3/simona/agent/em/EmAgentIT.scala | 55 ++- .../ie3/simona/agent/grid/ThermalGridIT.scala | 268 ++++++++++++++- .../participant/ParticipantAgentSpec.scala | 314 ++++++++++++++---- .../model/participant/evcs/EvcsModelIT.scala | 26 +- 5 files changed, 584 insertions(+), 81 deletions(-) diff --git a/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgent.scala b/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgent.scala index 01cc941723..eff37a6776 100644 --- a/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgent.scala +++ b/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgent.scala @@ -351,7 +351,7 @@ object ParticipantAgent { case _ => false } - // inform the result proxy that this grid agent will send new results + // inform the result proxy that this participant agent will send new results resultHandler.informProxy( modelShell.uuid, activation.tick, diff --git a/src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala b/src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala index 3386ee5e45..0f5bcb6759 100644 --- a/src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala +++ b/src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala @@ -888,6 +888,11 @@ class EmAgentIT */ emAgentActivation ! Activation(0) + // the result proxy will receive ExpectResult messages + resultServiceProxy.expectMessage( + ExpectResult(loadInputWithLimitedOperationTime.getUuid, 0, true) + ) + weatherDependentAgents.foreach { _ ! DataProvision( 0, @@ -902,8 +907,10 @@ class EmAgentIT ) } - // the result proxy will receive ExpectResult messages - resultServiceProxy.receiveMessages(3) + // we receive update messages, since a new set point was provided + resultServiceProxy.expectMessage( + ExpectResult(loadInputWithLimitedOperationTime.getUuid, 0) + ) resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => @@ -936,8 +943,15 @@ class EmAgentIT ) } - // the result proxy will receive ExpectResult messages - resultServiceProxy.receiveMessages(3) + // we receive a message, since new data arrived + resultServiceProxy.expectMessage( + ExpectResult(pvInputLimitedOperationTime.getUuid, 3600, true) + ) + + // we receive an update message, since a new set point was provided + resultServiceProxy.expectMessage( + ExpectResult(pvInputLimitedOperationTime.getUuid, 3600) + ) resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => @@ -970,7 +984,16 @@ class EmAgentIT emAgentActivation ! Activation(7200) - resultServiceProxy.receiveMessages(3) + // we receive a message, since new data arrived + resultServiceProxy.expectMessage( + ExpectResult(pvInputLimitedOperationTime.getUuid, 7200, true) + ) + + // we receive an update message, since a new set point was provided + resultServiceProxy.expectMessage( + ExpectResult(pvInputLimitedOperationTime.getUuid, 7200) + ) + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid @@ -988,7 +1011,16 @@ class EmAgentIT */ emAgentActivation ! Activation(10800) - resultServiceProxy.receiveMessages(2) + // we receive a message, since new data arrived + resultServiceProxy.expectMessage( + ExpectResult(loadInputWithLimitedOperationTime.getUuid, 10800, true) + ) + + // we receive an update message, since a new set point was provided + resultServiceProxy.expectMessage( + ExpectResult(loadInputWithLimitedOperationTime.getUuid, 10800) + ) + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid @@ -1006,7 +1038,16 @@ class EmAgentIT */ emAgentActivation ! Activation(14400) - resultServiceProxy.receiveMessages(2) + // we receive a message, since new data arrived + resultServiceProxy.expectMessage( + ExpectResult(pvInputLimitedOperationTime.getUuid, 14400, true) + ) + + // we receive an update message, since a new set point was provided + resultServiceProxy.expectMessage( + ExpectResult(pvInputLimitedOperationTime.getUuid, 14400) + ) + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid diff --git a/src/test/scala/edu/ie3/simona/agent/grid/ThermalGridIT.scala b/src/test/scala/edu/ie3/simona/agent/grid/ThermalGridIT.scala index 73b5c4d135..3ac2a3749d 100644 --- a/src/test/scala/edu/ie3/simona/agent/grid/ThermalGridIT.scala +++ b/src/test/scala/edu/ie3/simona/agent/grid/ThermalGridIT.scala @@ -35,7 +35,6 @@ import edu.ie3.simona.ontology.messages.ServiceMessage.{ SecondaryServiceRegistrationMessage, } import edu.ie3.simona.ontology.messages.{Activation, SchedulerMessage} -import edu.ie3.simona.ontology.messages.ResultMessage.RequestResult import edu.ie3.simona.scheduler.ScheduleLock import edu.ie3.simona.service.Data.SecondaryData.WeatherData import edu.ie3.simona.service.ServiceType @@ -59,7 +58,6 @@ import org.apache.pekko.actor.testkit.typed.scaladsl.{ ScalaTestWithActorTestKit, TestProbe, } -import org.apache.pekko.actor.typed.scaladsl.adapter.TypedActorRefOps import org.scalatest.OptionValues.convertOptionToValuable import org.scalatest.matchers.should import org.scalatest.wordspec.AnyWordSpecLike @@ -199,6 +197,9 @@ class ThermalGridIT */ heatPumpAgent ! Activation(0) + // no message, since we are still waiting for secondary data + resultServiceProxy.expectNoMessage() + weatherDependentAgents.foreach { _ ! DataProvision( 0, @@ -213,6 +214,10 @@ class ThermalGridIT ) } + resultServiceProxy.expectMessage( + ExpectResult(typicalHpInputModel.getUuid, 0) + ) + Range(0, 3) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -260,6 +265,10 @@ class ThermalGridIT */ heatPumpAgent ! Activation(3416) + resultServiceProxy.expectMessage( + ExpectResult(typicalHpInputModel.getUuid, 3416) + ) + Range(0, 3) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -308,6 +317,9 @@ class ThermalGridIT */ heatPumpAgent ! Activation(3600) + // no message, since we are still waiting for secondary data + resultServiceProxy.expectNoMessage() + weatherDependentAgents.foreach { _ ! DataProvision( 3600, @@ -322,6 +334,10 @@ class ThermalGridIT ) } + resultServiceProxy.expectMessage( + ExpectResult(typicalHpInputModel.getUuid, 3600) + ) + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(hpResult) => hpResult.getInputModel shouldBe typicalHpInputModel.getUuid @@ -346,6 +362,10 @@ class ThermalGridIT */ heatPumpAgent ! Activation(4412) + resultServiceProxy.expectMessage( + ExpectResult(typicalHpInputModel.getUuid, 4412) + ) + Range(0, 2) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -385,6 +405,9 @@ class ThermalGridIT */ heatPumpAgent ! Activation(21600) + // no message, since we are still waiting for secondary data + resultServiceProxy.expectNoMessage() + weatherDependentAgents.foreach { _ ! DataProvision( 21600, @@ -399,6 +422,10 @@ class ThermalGridIT ) } + resultServiceProxy.expectMessage( + ExpectResult(typicalHpInputModel.getUuid, 21600) + ) + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(hpResult) => hpResult.getInputModel shouldBe typicalHpInputModel.getUuid @@ -421,6 +448,10 @@ class ThermalGridIT */ heatPumpAgent ! Activation(23288) + resultServiceProxy.expectMessage( + ExpectResult(typicalHpInputModel.getUuid, 23288) + ) + Range(0, 3) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -470,6 +501,9 @@ class ThermalGridIT */ heatPumpAgent ! Activation(25000) + // no message, since we are still waiting for secondary data + resultServiceProxy.expectNoMessage() + weatherDependentAgents.foreach { _ ! DataProvision( 25000, @@ -484,6 +518,10 @@ class ThermalGridIT ) } + resultServiceProxy.expectMessage( + ExpectResult(typicalHpInputModel.getUuid, 25000) + ) + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(hpResult) => hpResult.getInputModel shouldBe typicalHpInputModel.getUuid @@ -506,6 +544,10 @@ class ThermalGridIT */ heatPumpAgent ! Activation(26704) + resultServiceProxy.expectMessage( + ExpectResult(typicalHpInputModel.getUuid, 26704) + ) + Range(0, 2) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -544,6 +586,9 @@ class ThermalGridIT */ heatPumpAgent ! Activation(28000) + // no message, since we are still waiting for secondary data + resultServiceProxy.expectNoMessage() + weatherDependentAgents.foreach { _ ! DataProvision( 28000, @@ -557,6 +602,11 @@ class ThermalGridIT Some(151200), ) } + + resultServiceProxy.expectMessage( + ExpectResult(typicalHpInputModel.getUuid, 28000) + ) + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(hpResult) => hpResult.getInputModel shouldBe typicalHpInputModel.getUuid @@ -579,6 +629,10 @@ class ThermalGridIT */ heatPumpAgent ! Activation(31837) + resultServiceProxy.expectMessage( + ExpectResult(typicalHpInputModel.getUuid, 31837) + ) + Range(0, 3) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -627,6 +681,10 @@ class ThermalGridIT */ heatPumpAgent ! Activation(35253) + resultServiceProxy.expectMessage( + ExpectResult(typicalHpInputModel.getUuid, 35253) + ) + Range(0, 3) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -675,6 +733,10 @@ class ThermalGridIT */ heatPumpAgent ! Activation(35788) + resultServiceProxy.expectMessage( + ExpectResult(typicalHpInputModel.getUuid, 35788) + ) + Range(0, 2) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -852,7 +914,7 @@ class ThermalGridIT resultServiceProxy.expectNoMessage() scheduler.expectMessage(Completion(emAgentActivation, Some(0))) - val weatherDependentAgents = Seq(hpAgent.toClassic, pvAgent.toClassic) + val weatherDependentAgents = Seq(hpAgent, pvAgent) /* TICK 0 Start of Simulation, No sun at the moment. @@ -863,6 +925,9 @@ class ThermalGridIT */ emAgentActivation ! Activation(0) + // no message, since we are still waiting for secondary data + resultServiceProxy.expectNoMessage() + weatherDependentAgents.foreach { _ ! DataProvision( 0, @@ -877,6 +942,15 @@ class ThermalGridIT ) } + resultServiceProxy.receiveMessages(4) should contain allOf ( + // expect messages due to flex activation + ExpectResult(typicalHpInputModel.getUuid, 0, true), + ExpectResult(pvInput.getUuid, 0, true), + // expect messages due to new set point + ExpectResult(typicalHpInputModel.getUuid, 0), + ExpectResult(pvInput.getUuid, 0) + ) + Range(0, 4) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -930,6 +1004,9 @@ class ThermalGridIT */ emAgentActivation ! Activation(1800) + // no message, since we are still waiting for secondary data + resultServiceProxy.expectNoMessage() + weatherDependentAgents.foreach { _ ! DataProvision( 1800, @@ -944,6 +1021,15 @@ class ThermalGridIT ) } + resultServiceProxy.receiveMessages(4) should contain allOf ( + // expect messages due to flex activation + ExpectResult(typicalHpInputModel.getUuid, 1800, true), + ExpectResult(pvInput.getUuid, 1800, true), + // expect messages due to new set point + ExpectResult(typicalHpInputModel.getUuid, 1800), + ExpectResult(pvInput.getUuid, 1800) + ) + Range(0, 3) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -992,6 +1078,13 @@ class ThermalGridIT */ emAgentActivation ! Activation(5216) + resultServiceProxy.receiveMessages(2) should contain allOf ( + // expect messages due to flex activation + ExpectResult(typicalHpInputModel.getUuid, 5216, true), + // expect messages due to new set point + ExpectResult(typicalHpInputModel.getUuid, 5216) + ) + Range(0, 4) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -1046,6 +1139,9 @@ class ThermalGridIT */ emAgentActivation ! Activation(5400) + // no message, since we are still waiting for secondary data + resultServiceProxy.expectNoMessage() + weatherDependentAgents.foreach { _ ! DataProvision( 5400, @@ -1060,6 +1156,18 @@ class ThermalGridIT ) } + // expect messages due to flex activation + resultServiceProxy.receiveMessages(2) should contain allOf ( + ExpectResult(typicalHpInputModel.getUuid, 5400, true), + ExpectResult(pvInput.getUuid, 5400, true) + ) + + // expect messages due to new set point + resultServiceProxy.receiveMessages(2) should contain allOf ( + ExpectResult(typicalHpInputModel.getUuid, 5400), + ExpectResult(pvInput.getUuid, 5400) + ) + Range(0, 3) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -1104,6 +1212,13 @@ class ThermalGridIT */ emAgentActivation ! Activation(6731) + resultServiceProxy.receiveMessages(2) should contain allOf ( + // expect messages due to flex activation + ExpectResult(typicalHpInputModel.getUuid, 6731, true), + // expect messages due to new set point + ExpectResult(typicalHpInputModel.getUuid, 6731) + ) + Range(0, 4) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -1158,6 +1273,9 @@ class ThermalGridIT */ emAgentActivation ! Activation(9200) + // no message, since we are still waiting for secondary data + resultServiceProxy.expectNoMessage() + weatherDependentAgents.foreach { _ ! DataProvision( 9200, @@ -1172,6 +1290,15 @@ class ThermalGridIT ) } + resultServiceProxy.receiveMessages(4) should contain allOf ( + // expect messages due to flex activation + ExpectResult(typicalHpInputModel.getUuid, 9200, true), + ExpectResult(pvInput.getUuid, 9200, true), + // expect messages due to new set point + ExpectResult(typicalHpInputModel.getUuid, 9200), + ExpectResult(pvInput.getUuid, 9200) + ) + Range(0, 3) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -1216,6 +1343,13 @@ class ThermalGridIT */ emAgentActivation ! Activation(10531) + resultServiceProxy.receiveMessages(2) should contain allOf ( + // expect messages due to flex activation + ExpectResult(typicalHpInputModel.getUuid, 10531, true), + // expect messages due to new set point + ExpectResult(typicalHpInputModel.getUuid, 10531) + ) + Range(0, 4) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -1270,6 +1404,13 @@ class ThermalGridIT */ emAgentActivation ! Activation(11638) + resultServiceProxy.receiveMessages(2) should contain allOf ( + // expect messages due to flex activation + ExpectResult(typicalHpInputModel.getUuid, 11638, true), + // expect messages due to new set point + ExpectResult(typicalHpInputModel.getUuid, 11638) + ) + Range(0, 3) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -1315,6 +1456,9 @@ class ThermalGridIT */ emAgentActivation ! Activation(12000) + // no message, since we are still waiting for secondary data + resultServiceProxy.expectNoMessage() + weatherDependentAgents.foreach { _ ! DataProvision( 12000, @@ -1329,6 +1473,15 @@ class ThermalGridIT ) } + resultServiceProxy.receiveMessages(4) should contain allOf ( + // expect messages due to flex activation + ExpectResult(typicalHpInputModel.getUuid, 12000, true), + ExpectResult(pvInput.getUuid, 12000, true), + // expect messages due to new set point + ExpectResult(typicalHpInputModel.getUuid, 12000), + ExpectResult(pvInput.getUuid, 12000) + ) + Range(0, 3) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -1373,6 +1526,13 @@ class ThermalGridIT */ emAgentActivation ! Activation(12139) + resultServiceProxy.receiveMessages(2) should contain allOf ( + // expect messages due to flex activation + ExpectResult(typicalHpInputModel.getUuid, 12139, true), + // expect messages due to new set point + ExpectResult(typicalHpInputModel.getUuid, 12139) + ) + Range(0, 3) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -1418,6 +1578,9 @@ class ThermalGridIT */ emAgentActivation ! Activation(12500) + // no message, since we are still waiting for secondary data + resultServiceProxy.expectNoMessage() + weatherDependentAgents.foreach { _ ! DataProvision( 12500, @@ -1432,6 +1595,18 @@ class ThermalGridIT ) } + // expect messages due to flex activation + resultServiceProxy.receiveMessages(2) should contain allOf ( + ExpectResult(typicalHpInputModel.getUuid, 12500, true), + ExpectResult(pvInput.getUuid, 12500, true) + ) + + // expect messages due to new set point + resultServiceProxy.receiveMessages(2) should contain allOf ( + ExpectResult(typicalHpInputModel.getUuid, 12500), + ExpectResult(pvInput.getUuid, 12500) + ) + Range(0, 2) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -1462,6 +1637,13 @@ class ThermalGridIT */ emAgentActivation ! Activation(24412) + resultServiceProxy.receiveMessages(2) should contain allOf ( + // expect messages due to flex activation + ExpectResult(typicalHpInputModel.getUuid, 24412, true), + // expect messages due to new set point + ExpectResult(typicalHpInputModel.getUuid, 24412) + ) + Range(0, 4) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -1516,6 +1698,9 @@ class ThermalGridIT */ emAgentActivation ! Activation(25200) + // no message, since we are still waiting for secondary data + resultServiceProxy.expectNoMessage() + weatherDependentAgents.foreach { _ ! DataProvision( 25200, @@ -1530,6 +1715,18 @@ class ThermalGridIT ) } + // expect messages due to flex activation + resultServiceProxy.receiveMessages(2) should contain allOf ( + ExpectResult(typicalHpInputModel.getUuid, 25200, true), + ExpectResult(pvInput.getUuid, 25200, true) + ) + + // expect messages due to new set point + resultServiceProxy.receiveMessages(2) should contain allOf ( + ExpectResult(typicalHpInputModel.getUuid, 25200), + ExpectResult(pvInput.getUuid, 25200) + ) + Range(0, 3) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -1574,6 +1771,9 @@ class ThermalGridIT */ emAgentActivation ! Activation(27500) + // no message, since we are still waiting for secondary data + resultServiceProxy.expectNoMessage() + weatherDependentAgents.foreach { _ ! DataProvision( 27500, @@ -1588,6 +1788,18 @@ class ThermalGridIT ) } + // expect messages due to flex activation + resultServiceProxy.receiveMessages(2) should contain allOf ( + ExpectResult(typicalHpInputModel.getUuid, 27500, true), + ExpectResult(pvInput.getUuid, 27500, true) + ) + + // expect messages due to new set point + resultServiceProxy.receiveMessages(2) should contain allOf ( + ExpectResult(typicalHpInputModel.getUuid, 27500), + ExpectResult(pvInput.getUuid, 27500) + ) + Range(0, 2) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -1618,6 +1830,13 @@ class ThermalGridIT */ emAgentActivation ! Activation(30872) + resultServiceProxy.receiveMessages(2) should contain allOf ( + // expect messages due to flex activation + ExpectResult(typicalHpInputModel.getUuid, 30872, true), + // expect messages due to new set point + ExpectResult(typicalHpInputModel.getUuid, 30872) + ) + Range(0, 4) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -1674,6 +1893,9 @@ class ThermalGridIT */ emAgentActivation ! Activation(31000) + // no message, since we are still waiting for secondary data + resultServiceProxy.expectNoMessage() + weatherDependentAgents.foreach { _ ! DataProvision( 31000, @@ -1688,6 +1910,18 @@ class ThermalGridIT ) } + // expect messages due to flex activation + resultServiceProxy.receiveMessages(2) should contain allOf ( + ExpectResult(typicalHpInputModel.getUuid, 31000, true), + ExpectResult(pvInput.getUuid, 31000, true) + ) + + // expect messages due to new set point + resultServiceProxy.receiveMessages(2) should contain allOf ( + ExpectResult(typicalHpInputModel.getUuid, 31000), + ExpectResult(pvInput.getUuid, 31000) + ) + Range(0, 3) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -1732,6 +1966,13 @@ class ThermalGridIT */ emAgentActivation ! Activation(40942) + resultServiceProxy.receiveMessages(2) should contain allOf ( + // expect messages due to flex activation + ExpectResult(typicalHpInputModel.getUuid, 40942, true), + // expect messages due to new set point + ExpectResult(typicalHpInputModel.getUuid, 40942) + ) + Range(0, 4) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -1790,6 +2031,13 @@ class ThermalGridIT */ emAgentActivation ! Activation(43698) + resultServiceProxy.receiveMessages(2) should contain allOf ( + // expect messages due to flex activation + ExpectResult(typicalHpInputModel.getUuid, 43698, true), + // expect messages due to new set point + ExpectResult(typicalHpInputModel.getUuid, 43698) + ) + Range(0, 4) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -1844,6 +2092,13 @@ class ThermalGridIT */ emAgentActivation ! Activation(46631) + resultServiceProxy.receiveMessages(2) should contain allOf ( + // expect messages due to flex activation + ExpectResult(typicalHpInputModel.getUuid, 46631, true), + // expect messages due to new set point + ExpectResult(typicalHpInputModel.getUuid, 46631) + ) + Range(0, 3) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -1888,6 +2143,13 @@ class ThermalGridIT */ emAgentActivation ! Activation(56274) + resultServiceProxy.receiveMessages(2) should contain allOf ( + // expect messages due to flex activation + ExpectResult(typicalHpInputModel.getUuid, 56274, true), + // expect messages due to new set point + ExpectResult(typicalHpInputModel.getUuid, 56274) + ) + Range(0, 3) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] diff --git a/src/test/scala/edu/ie3/simona/agent/participant/ParticipantAgentSpec.scala b/src/test/scala/edu/ie3/simona/agent/participant/ParticipantAgentSpec.scala index 919c681cbf..e4e4838e3e 100644 --- a/src/test/scala/edu/ie3/simona/agent/participant/ParticipantAgentSpec.scala +++ b/src/test/scala/edu/ie3/simona/agent/participant/ParticipantAgentSpec.scala @@ -132,7 +132,11 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! Activation(8 * 3600) - resultProxy.expectMessageType[ExpectResult] + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 8 * 3600) + ) + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -208,7 +212,11 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! Activation(20 * 3600) - resultProxy.expectMessageType[ExpectResult] + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 20 * 3600) + ) + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -287,7 +295,11 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! Activation(8 * 3600) - resultProxy.expectMessageType[ExpectResult] + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 8 * 3600) + ) + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -321,11 +333,15 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { q should approximate(Kilovars(0.968644209676)) } - resultProxy.expectMessageType[ExpectResult] scheduler.expectNoMessage() participantAgent ! GridSimulationFinished(12 * 3600, 24 * 3600) + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 12 * 3600) + ) + // calculation should start now resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => @@ -351,7 +367,11 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! Activation(20 * 3600) - resultProxy.expectMessageType[ExpectResult] + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 20 * 3600) + ) + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -434,8 +454,6 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! Activation(0) - resultProxy.expectMessageType[ExpectResult] - // nothing should happen, still waiting for secondary data... resultProxy.expectNoMessage() scheduler.expectNoMessage() @@ -447,8 +465,10 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { Some(6 * 3600), ) + // the result proxy is informed that a result will be provided + resultProxy.expectMessage(ExpectResult(MockParticipantModel.uuid, 0)) + // outside of operation interval, 0 MW - resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -475,7 +495,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { Some(12 * 3600), ) - resultProxy.expectMessageType[ExpectResult] + resultProxy.expectNoMessage() scheduler.expectNoMessage() // TICK 8 * 3600: Start of operation interval @@ -488,7 +508,11 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! Activation(8 * 3600) - resultProxy.expectMessageType[ExpectResult] + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 8 * 3600) + ) + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -526,7 +550,6 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! GridSimulationFinished(12 * 3600, 24 * 3600) // nothing should happen, still waiting for secondary data... - resultProxy.expectMessageType[ExpectResult] scheduler.expectNoMessage() participantAgent ! DataProvision( @@ -536,8 +559,12 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { Some(15 * 3600), ) + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 12 * 3600) + ) + // calculation should start now - resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -559,7 +586,6 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! Activation(15 * 3600) // nothing should happen, still waiting for secondary data... - resultProxy.expectMessageType[ExpectResult] scheduler.expectNoMessage() participantAgent ! NoDataProvision( @@ -568,8 +594,10 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { Some(18 * 3600), ) - // no-op activation, thus no result expected - resultProxy.expectMessageType[ExpectResult] + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 15 * 3600) + ) // new data is expected at 18 hours scheduler.expectMessage( @@ -584,7 +612,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! Activation(18 * 3600) // nothing should happen, still waiting for secondary data... - resultProxy.expectMessageType[ExpectResult] + resultProxy.expectNoMessage() scheduler.expectNoMessage() participantAgent ! DataProvision( @@ -594,8 +622,12 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { Some(24 * 3600), ) + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 18 * 3600) + ) + // calculation should start now - resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -615,7 +647,11 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! Activation(20 * 3600) - resultProxy.expectMessageType[ExpectResult] + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 20 * 3600) + ) + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -699,7 +735,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! Activation(0) // nothing should happen, still waiting for primary data... - resultProxy.expectMessageType[ExpectResult] + resultProxy.expectNoMessage() scheduler.expectNoMessage() participantAgent ! DataProvision( @@ -709,8 +745,10 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { Some(6 * 3600), ) + // the result proxy is informed that a result will be provided + resultProxy.expectMessage(ExpectResult(MockParticipantModel.uuid, 0)) + // outside of operation interval, 0 MW - resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -734,14 +772,20 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { Some(12 * 3600), ) - resultProxy.expectMessageType[ExpectResult] scheduler.expectNoMessage() + // no message, since we received no activation + resultProxy.expectNoMessage() + // TICK 8 * 3600: Start of operation interval participantAgent ! Activation(8 * 3600) - resultProxy.expectMessageType[ExpectResult] + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 8 * 3600) + ) + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -776,7 +820,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! GridSimulationFinished(12 * 3600, 24 * 3600) // nothing should happen, still waiting for primary data... - resultProxy.expectMessageType[ExpectResult] + resultProxy.expectNoMessage() scheduler.expectNoMessage() participantAgent ! DataProvision( @@ -786,8 +830,12 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { Some(18 * 3600), ) + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 12 * 3600) + ) + // calculation should start now - resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -806,7 +854,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! Activation(18 * 3600) // nothing should happen, still waiting for primary data... - resultProxy.expectMessageType[ExpectResult] + resultProxy.expectNoMessage() scheduler.expectNoMessage() participantAgent ! DataProvision( @@ -816,8 +864,12 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { Some(24 * 3600), ) + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 18 * 3600) + ) + // calculation should start now - resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -834,7 +886,11 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! Activation(20 * 3600) - resultProxy.expectMessageType[ExpectResult] + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 20 * 3600) + ) + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -919,6 +975,11 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! FlexActivation(8 * 3600) + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 8 * 3600, true) + ) + em.expectMessageType[ProvideFlexOptions] match { case ProvideFlexOptions( modelUuid, @@ -930,7 +991,6 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(3)) } - resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -942,7 +1002,11 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssuePowerControl(8 * 3600, Kilowatts(3)) - resultProxy.expectMessageType[ExpectResult] + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 8 * 3600) + ) + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -993,6 +1057,11 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! FlexActivation(20 * 3600) + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 20 * 3600, true) + ) + em.expectMessageType[ProvideFlexOptions] match { case ProvideFlexOptions( modelUuid, @@ -1004,7 +1073,6 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(0)) } - resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1016,7 +1084,11 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssueNoControl(20 * 3600) - resultProxy.expectMessageType[ExpectResult] + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 20 * 3600) + ) + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1101,6 +1173,11 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! FlexActivation(8 * 3600) + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 8 * 3600, true) + ) + em.expectMessageType[ProvideFlexOptions] match { case ProvideFlexOptions( modelUuid, @@ -1112,7 +1189,6 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(3)) } - resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1124,7 +1200,11 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssuePowerControl(8 * 3600, Kilowatts(3)) - resultProxy.expectMessageType[ExpectResult] + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 8 * 3600) + ) + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1154,6 +1234,9 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! FlexActivation(12 * 3600) + // no message, since we are still waiting for the grid + resultProxy.expectNoMessage() + participantAgent ! RequestAssetPowerMessage( 12 * 3600, Each(1), @@ -1167,11 +1250,16 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { q should approximate(Kilovars(0.48432210483)) } - resultProxy.expectMessageType[ExpectResult] + resultProxy.expectNoMessage() em.expectNoMessage() participantAgent ! GridSimulationFinished(12 * 3600, 24 * 3600) + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 12 * 3600, true) + ) + // calculation should start now em.expectMessageType[ProvideFlexOptions] match { case ProvideFlexOptions( @@ -1195,7 +1283,11 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssueNoControl(12 * 3600) - resultProxy.expectMessageType[ExpectResult] + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 12 * 3600) + ) + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1223,6 +1315,11 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! FlexActivation(20 * 3600) + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 20 * 3600, true) + ) + em.expectMessageType[ProvideFlexOptions] match { case ProvideFlexOptions( modelUuid, @@ -1234,7 +1331,6 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(0)) } - resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1246,7 +1342,11 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssueNoControl(20 * 3600) - resultProxy.expectMessageType[ExpectResult] + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 20 * 3600) + ) + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1341,7 +1441,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! FlexActivation(0) // nothing should happen, still waiting for secondary data... - resultProxy.expectMessageType[ExpectResult] + resultProxy.expectNoMessage() em.expectNoMessage() participantAgent ! DataProvision( @@ -1351,6 +1451,11 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { Some(6 * 3600), ) + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 0, true) + ) + em.expectMessageType[ProvideFlexOptions] match { case ProvideFlexOptions( modelUuid, @@ -1362,7 +1467,6 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(0)) } - resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1374,8 +1478,10 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssueNoControl(0) + // the result proxy is informed that a result will be provided + resultProxy.expectMessage(ExpectResult(MockParticipantModel.uuid, 0)) + // outside of operation interval, 0 MW - resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1412,7 +1518,8 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { Some(12 * 3600), ) - resultProxy.expectMessageType[ExpectResult] + // no message, since we are still waiting for an activation + resultProxy.expectNoMessage() em.expectNoMessage() // TICK 8 * 3600: Start of operation interval @@ -1425,6 +1532,11 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! FlexActivation(8 * 3600) + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 8 * 3600, true) + ) + em.expectMessageType[ProvideFlexOptions] match { case ProvideFlexOptions( modelUuid, @@ -1436,7 +1548,6 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(4)) } - resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1448,7 +1559,11 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssuePowerControl(8 * 3600, Kilowatts(3)) - resultProxy.expectMessageType[ExpectResult] + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 8 * 3600) + ) + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1479,6 +1594,9 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! FlexActivation(12 * 3600) + // no message, since we are still waiting for the grid agent + resultProxy.expectNoMessage() + participantAgent ! RequestAssetPowerMessage( 12 * 3600, Each(1), @@ -1496,7 +1614,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! GridSimulationFinished(12 * 3600, 24 * 3600) // nothing should happen, still waiting for secondary data... - resultProxy.expectMessageType[ExpectResult] + resultProxy.expectNoMessage() em.expectNoMessage() participantAgent ! DataProvision( @@ -1506,6 +1624,11 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { Some(18 * 3600), ) + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 12 * 3600, true) + ) + // calculation should start now em.expectMessageType[ProvideFlexOptions] match { case ProvideFlexOptions( @@ -1518,7 +1641,6 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(5)) } - resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1530,7 +1652,11 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssueNoControl(12 * 3600) - resultProxy.expectMessageType[ExpectResult] + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 12 * 3600) + ) + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1561,7 +1687,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! FlexActivation(18 * 3600) // nothing should happen, still waiting for secondary data... - resultProxy.expectMessageType[ExpectResult] + resultProxy.expectNoMessage() em.expectNoMessage() participantAgent ! DataProvision( @@ -1571,6 +1697,11 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { Some(24 * 3600), ) + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 18 * 3600, true) + ) + // calculation should start now em.expectMessageType[ProvideFlexOptions] match { case ProvideFlexOptions( @@ -1583,7 +1714,6 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(8)) } - resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1595,7 +1725,11 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssueNoControl(18 * 3600) - resultProxy.expectMessageType[ExpectResult] + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 18 * 3600) + ) + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1626,6 +1760,11 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! FlexActivation(20 * 3600) + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 20 * 3600, true) + ) + em.expectMessageType[ProvideFlexOptions] match { case ProvideFlexOptions( modelUuid, @@ -1637,7 +1776,6 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(0)) } - resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1649,7 +1787,11 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssueNoControl(20 * 3600) - resultProxy.expectMessageType[ExpectResult] + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 20 * 3600) + ) + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1740,7 +1882,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! FlexActivation(0) // nothing should happen, still waiting for primary data... - resultProxy.expectMessageType[ExpectResult] + resultProxy.expectNoMessage() em.expectNoMessage() participantAgent ! DataProvision( @@ -1750,6 +1892,11 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { Some(6 * 3600), ) + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 0, true) + ) + em.expectMessageType[ProvideFlexOptions] match { case ProvideFlexOptions( modelUuid, @@ -1761,7 +1908,6 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(0)) } - resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1773,8 +1919,10 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssueNoControl(0) + // the result proxy is informed that a result will be provided + resultProxy.expectMessage(ExpectResult(MockParticipantModel.uuid, 0)) + // outside of operation interval, 0 MW - resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1808,13 +1956,19 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { Some(12 * 3600), ) - resultProxy.expectMessageType[ExpectResult] + // no message, since we are still waiting for an activation + resultProxy.expectNoMessage() em.expectNoMessage() // TICK 8 * 3600: Start of operation interval participantAgent ! FlexActivation(8 * 3600) + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 8 * 3600, true) + ) + em.expectMessageType[ProvideFlexOptions] match { case ProvideFlexOptions( modelUuid, @@ -1826,7 +1980,6 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(3)) } - resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1838,7 +1991,11 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssuePowerControl(8 * 3600, Kilowatts(3)) - resultProxy.expectMessageType[ExpectResult] + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 8 * 3600) + ) + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1866,6 +2023,9 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! FlexActivation(12 * 3600) + // no message, since we are still waiting for the grid agent + resultProxy.expectNoMessage() + participantAgent ! RequestAssetPowerMessage( 12 * 3600, Each(1), @@ -1883,7 +2043,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! GridSimulationFinished(12 * 3600, 24 * 3600) // nothing should happen, still waiting for primary data... - resultProxy.expectMessageType[ExpectResult] + resultProxy.expectNoMessage() em.expectNoMessage() participantAgent ! DataProvision( @@ -1893,6 +2053,11 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { Some(18 * 3600), ) + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 12 * 3600, true) + ) + // calculation should start now em.expectMessageType[ProvideFlexOptions] match { case ProvideFlexOptions( @@ -1905,7 +2070,6 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(6)) } - resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1917,7 +2081,11 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssueNoControl(12 * 3600) - resultProxy.expectMessageType[ExpectResult] + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 12 * 3600) + ) + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1945,7 +2113,7 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! FlexActivation(18 * 3600) // nothing should happen, still waiting for primary data... - resultProxy.expectMessageType[ExpectResult] + resultProxy.expectNoMessage() em.expectNoMessage() participantAgent ! DataProvision( @@ -1955,6 +2123,11 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { Some(24 * 3600), ) + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 18 * 3600, true) + ) + // calculation should start now em.expectMessageType[ProvideFlexOptions] match { case ProvideFlexOptions( @@ -1967,7 +2140,6 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(3)) } - resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -1979,7 +2151,11 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssueNoControl(18 * 3600) - resultProxy.expectMessageType[ExpectResult] + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 18 * 3600) + ) + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -2006,6 +2182,11 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! FlexActivation(20 * 3600) + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 20 * 3600, true) + ) + em.expectMessageType[ProvideFlexOptions] match { case ProvideFlexOptions( modelUuid, @@ -2017,7 +2198,6 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { max should approximate(Kilowatts(0)) } - resultProxy.expectMessageType[ExpectResult] resultProxy.expectMessageType[FlexOptionsResultEvent] match { case FlexOptionsResultEvent(result: FlexOptionsResult) => result.getInputModel shouldBe MockParticipantModel.uuid @@ -2029,7 +2209,11 @@ class ParticipantAgentSpec extends ScalaTestWithActorTestKit with UnitSpec { participantAgent ! IssueNoControl(20 * 3600) - resultProxy.expectMessageType[ExpectResult] + // the result proxy is informed that a result will be provided + resultProxy.expectMessage( + ExpectResult(MockParticipantModel.uuid, 20 * 3600) + ) + resultProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(result: MockResult) => result.getInputModel shouldBe MockParticipantModel.uuid diff --git a/src/test/scala/edu/ie3/simona/model/participant/evcs/EvcsModelIT.scala b/src/test/scala/edu/ie3/simona/model/participant/evcs/EvcsModelIT.scala index 4cc9b3e53d..f8118bb35b 100644 --- a/src/test/scala/edu/ie3/simona/model/participant/evcs/EvcsModelIT.scala +++ b/src/test/scala/edu/ie3/simona/model/participant/evcs/EvcsModelIT.scala @@ -230,6 +230,9 @@ class EvcsModelIT evcsActivation ! Activation(0) + // the result proxy is informed that a result will be provided + resultProxy.expectMessage(ExpectResult(evcsInputModel.getUuid, 0)) + resultProxy .receiveMessages(3) .map { case ParticipantResultEvent(result) => @@ -279,6 +282,9 @@ class EvcsModelIT // EVCS activation without arrivals evcsActivation ! Activation(1800) + // the result proxy is informed that a result will be provided + resultProxy.expectMessage(ExpectResult(evcsInputModel.getUuid, 1800)) + resultProxy .receiveMessages(2) .map { case ParticipantResultEvent(result) => @@ -306,6 +312,9 @@ class EvcsModelIT evcsActivation ! Activation(3600) + // the result proxy is informed that a result will be provided + resultProxy.expectMessage(ExpectResult(evcsInputModel.getUuid, 3600)) + resultProxy .receiveMessages(2) .map { case ParticipantResultEvent(result) => @@ -377,7 +386,8 @@ class EvcsModelIT evcsActivation ! Activation(9000) - resultProxy.expectNoMessage() + // the result proxy is informed that a result will be provided + resultProxy.expectMessage(ExpectResult(evcsInputModel.getUuid, 9000)) // Next data at 10800 scheduler.expectMessage(Completion(evcsActivation, Some(10800))) @@ -397,8 +407,6 @@ class EvcsModelIT scheduler.expectMessage(Completion(evService, None)) - resultProxy.expectNoMessage() - // Send arrivals extEvData.provideArrivingEvs( Map( @@ -417,6 +425,9 @@ class EvcsModelIT evcsActivation ! Activation(10800) + // the result proxy is informed that a result will be provided + resultProxy.expectMessage(ExpectResult(evcsInputModel.getUuid, 10800)) + resultProxy .receiveMessages(2) .map { case ParticipantResultEvent(result) => @@ -459,6 +470,9 @@ class EvcsModelIT // EVCS activation evcsActivation ! Activation(12600) + // the result proxy is informed that a result will be provided + resultProxy.expectMessage(ExpectResult(evcsInputModel.getUuid, 12600)) + resultProxy .receiveMessages(2) .map { case ParticipantResultEvent(result) => @@ -530,7 +544,8 @@ class EvcsModelIT evcsActivation ! Activation(14400) - resultProxy.expectNoMessage() + // the result proxy is informed that a result will be provided + resultProxy.expectMessage(ExpectResult(evcsInputModel.getUuid, 14400)) // evB is departing at 18000 scheduler.expectMessage(Completion(evcsActivation, Some(18000))) @@ -583,7 +598,8 @@ class EvcsModelIT evcsActivation ! Activation(18000) - resultProxy.expectNoMessage() + // the result proxy is informed that a result will be provided + resultProxy.expectMessage(ExpectResult(evcsInputModel.getUuid, 18000)) // No future arrivals planned, next activation: end of simulation scheduler.expectMessage(Completion(evcsActivation, Some(48 * 3600))) From c2b06760150efbc1af24da78ed87277762bd89d9 Mon Sep 17 00:00:00 2001 From: staudtMarius Date: Wed, 10 Dec 2025 15:10:51 +0100 Subject: [PATCH 6/6] Fixing failing tests. --- .../edu/ie3/simona/agent/em/EmAgentIT.scala | 48 +++++ .../ie3/simona/agent/grid/ThermalGridIT.scala | 199 +++++++++++++++--- 2 files changed, 223 insertions(+), 24 deletions(-) diff --git a/src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala b/src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala index 87f3659300..111b4d0360 100644 --- a/src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala +++ b/src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala @@ -592,6 +592,13 @@ class EmAgentIT */ emAgentActivation ! Activation(75) + resultServiceProxy.receiveMessages(2) should contain allOf ( + // we receive a message, since new data arrived + ExpectResult(hpInputModel.getUuid, 75, true), + // we receive update messages, since a new set point was provided + ExpectResult(hpInputModel.getUuid, 75) + ) + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid @@ -612,6 +619,12 @@ class EmAgentIT */ emAgentActivation ! Activation(3600) + resultServiceProxy.receiveMessages(2) should contain allOf ( + // we receive a message, since new data arrived + ExpectResult(hpInputModel.getUuid, 3600, true), + // we receive update messages, since a new set point was provided + ExpectResult(hpInputModel.getUuid, 3600) + ) resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid @@ -632,6 +645,13 @@ class EmAgentIT */ emAgentActivation ! Activation(3675) + resultServiceProxy.receiveMessages(2) should contain allOf ( + // we receive a message, since new data arrived + ExpectResult(hpInputModel.getUuid, 3675, true), + // we receive update messages, since a new set point was provided + ExpectResult(hpInputModel.getUuid, 3675) + ) + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid @@ -656,6 +676,13 @@ class EmAgentIT */ emAgentActivation ! Activation(6056) + resultServiceProxy.receiveMessages(2) should contain allOf ( + // we receive a message, since new data arrived + ExpectResult(hpInputModel.getUuid, 6056, true), + // we receive update messages, since a new set point was provided + ExpectResult(hpInputModel.getUuid, 6056) + ) + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid @@ -721,6 +748,13 @@ class EmAgentIT */ emAgentActivation ! Activation(7278) + resultServiceProxy.receiveMessages(2) should contain allOf ( + // we receive a message, since new data arrived + ExpectResult(hpInputModel.getUuid, 7278, true), + // we receive update messages, since a new set point was provided + ExpectResult(hpInputModel.getUuid, 7278) + ) + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid @@ -741,6 +775,13 @@ class EmAgentIT */ emAgentActivation ! Activation(7981) + resultServiceProxy.receiveMessages(2) should contain allOf ( + // we receive a message, since new data arrived + ExpectResult(hpInputModel.getUuid, 7981, true), + // we receive update messages, since a new set point was provided + ExpectResult(hpInputModel.getUuid, 7981) + ) + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid @@ -806,6 +847,13 @@ class EmAgentIT */ emAgentActivation ! Activation(10879) + resultServiceProxy.receiveMessages(2) should contain allOf ( + // we receive a message, since new data arrived + ExpectResult(hpInputModel.getUuid, 10879, true), + // we receive update messages, since a new set point was provided + ExpectResult(hpInputModel.getUuid, 10879) + ) + resultServiceProxy.expectMessageType[ParticipantResultEvent] match { case ParticipantResultEvent(emResult: EmResult) => emResult.getInputModel shouldBe emInput.getUuid diff --git a/src/test/scala/edu/ie3/simona/agent/grid/ThermalGridIT.scala b/src/test/scala/edu/ie3/simona/agent/grid/ThermalGridIT.scala index 8d6943f86b..6432961a18 100644 --- a/src/test/scala/edu/ie3/simona/agent/grid/ThermalGridIT.scala +++ b/src/test/scala/edu/ie3/simona/agent/grid/ThermalGridIT.scala @@ -162,6 +162,10 @@ class ThermalGridIT tickPairs.foreach { case (currentTick, nextTick) => activationActor ! Activation(currentTick) + resultServiceProxy.expectMessage( + ExpectResult(typicalHpInputModel.getUuid, currentTick) + ) + Range(0, 2) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] } .foreach { @@ -300,6 +304,11 @@ class ThermalGridIT */ heatPumpAgent ! Activation(45) + // we receive update messages, since a new set point was provided + resultServiceProxy.expectMessage( + ExpectResult(typicalHpInputModel.getUuid, 45) + ) + Range(0, 2) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -455,6 +464,10 @@ class ThermalGridIT */ heatPumpAgent ! Activation(3625) + resultServiceProxy.expectMessage( + ExpectResult(typicalHpInputModel.getUuid, 3625) + ) + Range(0, 2) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -615,6 +628,11 @@ class ThermalGridIT */ heatPumpAgent ! Activation(21659) + // we receive update messages, since a new set point was provided + resultServiceProxy.expectMessage( + ExpectResult(typicalHpInputModel.getUuid, 21659) + ) + Range(0, 2) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -750,6 +768,11 @@ class ThermalGridIT */ heatPumpAgent ! Activation(25200) + // we receive update messages, since a new set point was provided + resultServiceProxy.expectMessage( + ExpectResult(typicalHpInputModel.getUuid, 25200) + ) + Range(0, 2) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -791,6 +814,11 @@ class ThermalGridIT */ heatPumpAgent ! Activation(25316) + // we receive update messages, since a new set point was provided + resultServiceProxy.expectMessage( + ExpectResult(typicalHpInputModel.getUuid, 25316) + ) + Range(0, 2) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -921,6 +949,11 @@ class ThermalGridIT */ heatPumpAgent ! Activation(28800) + // we receive update messages, since a new set point was provided + resultServiceProxy.expectMessage( + ExpectResult(typicalHpInputModel.getUuid, 28800) + ) + Range(0, 2) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -964,6 +997,11 @@ class ThermalGridIT */ heatPumpAgent ! Activation(28852) + // we receive update messages, since a new set point was provided + resultServiceProxy.expectMessage( + ExpectResult(typicalHpInputModel.getUuid, 28852) + ) + Range(0, 3) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -1015,6 +1053,11 @@ class ThermalGridIT */ heatPumpAgent ! Activation(29193) + // we receive update messages, since a new set point was provided + resultServiceProxy.expectMessage( + ExpectResult(typicalHpInputModel.getUuid, 29193) + ) + Range(0, 3) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -1066,8 +1109,9 @@ class ThermalGridIT */ heatPumpAgent ! Activation(32032) + // we receive update messages, since a new set point was provided resultServiceProxy.expectMessage( - ExpectResult(typicalHpInputModel.getUuid, 31837) + ExpectResult(typicalHpInputModel.getUuid, 32032) ) Range(0, 3) @@ -1120,6 +1164,11 @@ class ThermalGridIT */ heatPumpAgent ! Activation(32400) + // we receive update messages, since a new set point was provided + resultServiceProxy.expectMessage( + ExpectResult(typicalHpInputModel.getUuid, 32400) + ) + Range(0, 2) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -1161,6 +1210,11 @@ class ThermalGridIT */ heatPumpAgent ! Activation(32541) + // we receive update messages, since a new set point was provided + resultServiceProxy.expectMessage( + ExpectResult(typicalHpInputModel.getUuid, 32541) + ) + Range(0, 2) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -1202,8 +1256,9 @@ class ThermalGridIT */ heatPumpAgent ! Activation(35448) + // we receive update messages, since a new set point was provided resultServiceProxy.expectMessage( - ExpectResult(typicalHpInputModel.getUuid, 35253) + ExpectResult(typicalHpInputModel.getUuid, 35448) ) Range(0, 3) @@ -1256,8 +1311,9 @@ class ThermalGridIT */ heatPumpAgent ! Activation(35983) + // we receive update messages, since a new set point was provided resultServiceProxy.expectMessage( - ExpectResult(typicalHpInputModel.getUuid, 35788) + ExpectResult(typicalHpInputModel.getUuid, 35983) ) Range(0, 2) @@ -1379,6 +1435,13 @@ class ThermalGridIT tickPairs.foreach { case (currentTick, nextTick) => activationActor ! Activation(currentTick) + resultServiceProxy.receiveMessages(2) should contain allOf ( + // we receive a message, since new data arrived + ExpectResult(typicalHpInputModel.getUuid, currentTick, true), + // we receive update messages, since a new set point was provided + ExpectResult(typicalHpInputModel.getUuid, currentTick) + ) + Range(0, 3) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] } .foreach { @@ -1561,6 +1624,13 @@ class ThermalGridIT */ emAgentActivation ! Activation(150) + resultServiceProxy.receiveMessages(2) should contain allOf ( + // we receive a message, since new data arrived + ExpectResult(typicalHpInputModel.getUuid, 150, true), + // we receive update messages, since a new set point was provided + ExpectResult(typicalHpInputModel.getUuid, 150) + ) + Range(0, 3) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -1630,10 +1700,8 @@ class ThermalGridIT resultServiceProxy.receiveMessages(4) should contain allOf ( // expect messages due to flex activation ExpectResult(typicalHpInputModel.getUuid, 1800, true), - ExpectResult(pvInput.getUuid, 1800, true), // expect messages due to new set point - ExpectResult(typicalHpInputModel.getUuid, 1800), - ExpectResult(pvInput.getUuid, 1800) + ExpectResult(typicalHpInputModel.getUuid, 1800) ) Range(0, 3) @@ -1686,6 +1754,13 @@ class ThermalGridIT */ emAgentActivation ! Activation(3600) + resultServiceProxy.receiveMessages(2) should contain allOf ( + // we receive a message, since new data arrived + ExpectResult(typicalHpInputModel.getUuid, 3600, true), + // we receive update messages, since a new set point was provided + ExpectResult(typicalHpInputModel.getUuid, 3600) + ) + Range(0, 3) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -1740,6 +1815,13 @@ class ThermalGridIT */ emAgentActivation ! Activation(3750) + resultServiceProxy.receiveMessages(2) should contain allOf ( + // we receive a message, since new data arrived + ExpectResult(typicalHpInputModel.getUuid, 3750, true), + // we receive update messages, since a new set point was provided + ExpectResult(typicalHpInputModel.getUuid, 3750) + ) + Range(0, 3) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -1998,6 +2080,13 @@ class ThermalGridIT */ emAgentActivation ! Activation(7200) + resultServiceProxy.receiveMessages(2) should contain allOf ( + // we receive a message, since new data arrived + ExpectResult(typicalHpInputModel.getUuid, 7200, true), + // we receive update messages, since a new set point was provided + ExpectResult(typicalHpInputModel.getUuid, 7200) + ) + Range(0, 3) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -2044,6 +2133,13 @@ class ThermalGridIT */ emAgentActivation ! Activation(7355) + resultServiceProxy.receiveMessages(2) should contain allOf ( + // we receive a message, since new data arrived + ExpectResult(typicalHpInputModel.getUuid, 7355, true), + // we receive update messages, since a new set point was provided + ExpectResult(typicalHpInputModel.getUuid, 7355) + ) + Range(0, 3) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -2225,6 +2321,13 @@ class ThermalGridIT */ emAgentActivation ! Activation(10800) + resultServiceProxy.receiveMessages(2) should contain allOf ( + // we receive a message, since new data arrived + ExpectResult(typicalHpInputModel.getUuid, 10800, true), + // we receive update messages, since a new set point was provided + ExpectResult(typicalHpInputModel.getUuid, 10800) + ) + Range(0, 3) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -2275,6 +2378,13 @@ class ThermalGridIT */ emAgentActivation ! Activation(10958) + resultServiceProxy.receiveMessages(2) should contain allOf ( + // we receive a message, since new data arrived + ExpectResult(typicalHpInputModel.getUuid, 10958, true), + // we receive update messages, since a new set point was provided + ExpectResult(typicalHpInputModel.getUuid, 10958) + ) + Range(0, 3) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -2523,14 +2633,11 @@ class ThermalGridIT ) } - // expect messages due to flex activation - resultServiceProxy.receiveMessages(2) should contain allOf ( + resultServiceProxy.receiveMessages(4) should contain allOf ( + // expect messages due to flex activation ExpectResult(typicalHpInputModel.getUuid, 12500, true), - ExpectResult(pvInput.getUuid, 12500, true) - ) - - // expect messages due to new set point - resultServiceProxy.receiveMessages(2) should contain allOf ( + ExpectResult(pvInput.getUuid, 12500, true), + // expect messages due to new set point ExpectResult(typicalHpInputModel.getUuid, 12500), ExpectResult(pvInput.getUuid, 12500) ) @@ -2729,6 +2836,13 @@ class ThermalGridIT */ emAgentActivation ! Activation(25230) + resultServiceProxy.receiveMessages(2) should contain allOf ( + // we receive a message, since new data arrived + ExpectResult(typicalHpInputModel.getUuid, 25230, true), + // we receive update messages, since a new set point was provided + ExpectResult(typicalHpInputModel.getUuid, 25230) + ) + Range(0, 4) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -2785,6 +2899,13 @@ class ThermalGridIT */ emAgentActivation ! Activation(26210) + resultServiceProxy.receiveMessages(2) should contain allOf ( + // we receive a message, since new data arrived + ExpectResult(typicalHpInputModel.getUuid, 26210, true), + // we receive update messages, since a new set point was provided + ExpectResult(typicalHpInputModel.getUuid, 26210) + ) + Range(0, 4) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -2904,9 +3025,9 @@ class ThermalGridIT resultServiceProxy.receiveMessages(2) should contain allOf ( // expect messages due to flex activation - ExpectResult(typicalHpInputModel.getUuid, 30872, true), + ExpectResult(typicalHpInputModel.getUuid, 28800, true), // expect messages due to new set point - ExpectResult(typicalHpInputModel.getUuid, 30872) + ExpectResult(typicalHpInputModel.getUuid, 28800) ) Range(0, 3) @@ -2955,6 +3076,13 @@ class ThermalGridIT */ emAgentActivation ! Activation(28941) + resultServiceProxy.receiveMessages(2) should contain allOf ( + // expect messages due to flex activation + ExpectResult(typicalHpInputModel.getUuid, 28941, true), + // expect messages due to new set point + ExpectResult(typicalHpInputModel.getUuid, 28941) + ) + Range(0, 3) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -3078,6 +3206,13 @@ class ThermalGridIT */ emAgentActivation ! Activation(31762) + resultServiceProxy.receiveMessages(2) should contain allOf ( + // expect messages due to flex activation + ExpectResult(typicalHpInputModel.getUuid, 31762, true), + // expect messages due to new set point + ExpectResult(typicalHpInputModel.getUuid, 31762) + ) + Range(0, 4) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -3154,9 +3289,9 @@ class ThermalGridIT resultServiceProxy.receiveMessages(2) should contain allOf ( // expect messages due to flex activation - ExpectResult(typicalHpInputModel.getUuid, 40942, true), + ExpectResult(typicalHpInputModel.getUuid, 41762, true), // expect messages due to new set point - ExpectResult(typicalHpInputModel.getUuid, 40942) + ExpectResult(typicalHpInputModel.getUuid, 41762) ) Range(0, 4) @@ -3205,7 +3340,7 @@ class ThermalGridIT scheduler.expectMessage(Completion(emAgentActivation, Some(43200))) /* We'll jump through a bunch of activations caused from DomesticHotWaterStorage being active. -The results are checked implicitly through the state of stored energy at the next result check. + The results are checked implicitly through the state of stored energy at the next result check. */ val thirdActivationTicksBlock = Seq(43200L, 43311L) @@ -3230,6 +3365,13 @@ The results are checked implicitly through the state of stored energy at the nex */ emAgentActivation ! Activation(43311) + resultServiceProxy.receiveMessages(2) should contain allOf ( + // expect messages due to flex activation + ExpectResult(typicalHpInputModel.getUuid, 43311, true), + // expect messages due to new set point + ExpectResult(typicalHpInputModel.getUuid, 43311) + ) + Range(0, 3) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -3281,9 +3423,9 @@ The results are checked implicitly through the state of stored energy at the nex resultServiceProxy.receiveMessages(2) should contain allOf ( // expect messages due to flex activation - ExpectResult(typicalHpInputModel.getUuid, 43698, true), + ExpectResult(typicalHpInputModel.getUuid, 43628, true), // expect messages due to new set point - ExpectResult(typicalHpInputModel.getUuid, 43698) + ExpectResult(typicalHpInputModel.getUuid, 43628) ) Range(0, 4) @@ -3344,9 +3486,9 @@ The results are checked implicitly through the state of stored energy at the nex resultServiceProxy.receiveMessages(2) should contain allOf ( // expect messages due to flex activation - ExpectResult(typicalHpInputModel.getUuid, 46631, true), + ExpectResult(typicalHpInputModel.getUuid, 45620, true), // expect messages due to new set point - ExpectResult(typicalHpInputModel.getUuid, 46631) + ExpectResult(typicalHpInputModel.getUuid, 45620) ) Range(0, 3) @@ -3409,6 +3551,15 @@ The results are checked implicitly through the state of stored energy at the nex ) } + resultServiceProxy.receiveMessages(4) should contain allOf ( + // expect messages due to flex activation + ExpectResult(typicalHpInputModel.getUuid, 46800, true), + ExpectResult(pvInput.getUuid, 46800, true), + // expect messages due to new set point + ExpectResult(typicalHpInputModel.getUuid, 46800), + ExpectResult(pvInput.getUuid, 46800) + ) + Range(0, 3) .map { _ => resultServiceProxy.expectMessageType[ResultEvent] @@ -3474,9 +3625,9 @@ The results are checked implicitly through the state of stored energy at the nex resultServiceProxy.receiveMessages(2) should contain allOf ( // expect messages due to flex activation - ExpectResult(typicalHpInputModel.getUuid, 56274, true), + ExpectResult(typicalHpInputModel.getUuid, 55263, true), // expect messages due to new set point - ExpectResult(typicalHpInputModel.getUuid, 56274) + ExpectResult(typicalHpInputModel.getUuid, 55263) ) Range(0, 3)