From 67b6d40620492616f129c985c85ec1acf195f4c7 Mon Sep 17 00:00:00 2001 From: Sebastian Peter Date: Thu, 8 Feb 2024 21:34:51 +0100 Subject: [PATCH 01/32] Introducing EmAgent (cherry-picked from all/#tbw-wHp-squants-2023-12) --- .gitattributes | 1 - .gitignore | 3 + CHANGELOG.md | 2 + build.gradle | 1 + input/samples/vn_simona/vn_simona.conf | 9 + .../resources/config/config-template.conf | 20 + .../edu/ie3/simona/agent/em/EmAgent.scala | 484 ++++++++++++ .../edu/ie3/simona/agent/em/EmDataCore.scala | 460 +++++++++++ .../agent/em/FlexCorrespondenceStore.scala | 143 ++++ .../edu/ie3/simona/agent/grid/GridAgent.scala | 1 + .../agent/grid/GridAgentController.scala | 380 +++++++-- .../edu/ie3/simona/config/SimonaConfig.scala | 183 +++++ .../ie3/simona/io/grid/CsvGridSource.scala | 22 +- .../ie3/simona/model/em/EmAggregateFlex.scala | 29 + .../simona/model/em/EmAggregateSelfOpt.scala | 60 ++ .../model/em/EmAggregateSimpleSum.scala | 37 + .../ie3/simona/model/em/EmModelShell.scala | 133 ++++ .../ie3/simona/model/em/EmModelStrat.scala | 60 ++ .../ie3/simona/model/em/FlexTimeSeries.scala | 107 +++ .../model/em/PrioritizedFlexStrat.scala | 204 +++++ .../model/em/ProportionalFlexStrat.scala | 138 ++++ .../messages/flex/FlexibilityMessage.scala | 3 +- .../ie3/simona/sim/setup/SetupHelper.scala | 6 +- .../edu/ie3/simona/util/ConfigUtil.scala | 27 +- .../ie3/simona/util/EntityMapperUtil.scala | 3 +- .../util/scala/io/FlexSignalFromExcel.scala | 176 +++++ .../edu/ie3/util/scala/io/flexSignal.xlsx | Bin 0 -> 47344 bytes .../edu/ie3/simona/agent/em/EmAgentIT.scala | 722 ++++++++++++++++++ .../edu/ie3/simona/agent/em/EmAgentSpec.scala | 696 +++++++++++++++++ .../edu/ie3/simona/event/NotifierSpec.scala | 2 + .../model/em/EmAggregateSelfOptSpec.scala | 171 +++++ .../model/em/EmAggregateSimpleSumSpec.scala | 51 ++ .../model/em/PrioritizedFlexStratSpec.scala | 339 ++++++++ .../model/em/ProportionalFlexStratSpec.scala | 157 ++++ .../simona/test/common/ConfigTestData.scala | 9 + .../simona/test/common/DefaultTestData.scala | 9 + .../test/common/input/EmInputTestData.scala | 163 ++++ .../edu/ie3/simona/util/ConfigUtilSpec.scala | 20 +- .../scala/io/FlexSignalFromExcelSpec.scala | 48 ++ 39 files changed, 5006 insertions(+), 73 deletions(-) create mode 100644 src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala create mode 100644 src/main/scala/edu/ie3/simona/agent/em/EmDataCore.scala create mode 100644 src/main/scala/edu/ie3/simona/agent/em/FlexCorrespondenceStore.scala create mode 100644 src/main/scala/edu/ie3/simona/model/em/EmAggregateFlex.scala create mode 100644 src/main/scala/edu/ie3/simona/model/em/EmAggregateSelfOpt.scala create mode 100644 src/main/scala/edu/ie3/simona/model/em/EmAggregateSimpleSum.scala create mode 100644 src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala create mode 100644 src/main/scala/edu/ie3/simona/model/em/EmModelStrat.scala create mode 100644 src/main/scala/edu/ie3/simona/model/em/FlexTimeSeries.scala create mode 100644 src/main/scala/edu/ie3/simona/model/em/PrioritizedFlexStrat.scala create mode 100644 src/main/scala/edu/ie3/simona/model/em/ProportionalFlexStrat.scala create mode 100644 src/main/scala/edu/ie3/util/scala/io/FlexSignalFromExcel.scala create mode 100644 src/test/resources/edu/ie3/util/scala/io/flexSignal.xlsx create mode 100644 src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala create mode 100644 src/test/scala/edu/ie3/simona/agent/em/EmAgentSpec.scala create mode 100644 src/test/scala/edu/ie3/simona/model/em/EmAggregateSelfOptSpec.scala create mode 100644 src/test/scala/edu/ie3/simona/model/em/EmAggregateSimpleSumSpec.scala create mode 100644 src/test/scala/edu/ie3/simona/model/em/PrioritizedFlexStratSpec.scala create mode 100644 src/test/scala/edu/ie3/simona/model/em/ProportionalFlexStratSpec.scala create mode 100644 src/test/scala/edu/ie3/simona/test/common/input/EmInputTestData.scala create mode 100644 src/test/scala/edu/ie3/util/scala/io/FlexSignalFromExcelSpec.scala diff --git a/.gitattributes b/.gitattributes index 00a51aff5e..022b84144a 100644 --- a/.gitattributes +++ b/.gitattributes @@ -3,4 +3,3 @@ # # These are explicitly windows files and should use crlf *.bat text eol=crlf - diff --git a/.gitignore b/.gitignore index 7b25571331..83d4c33fc9 100644 --- a/.gitignore +++ b/.gitignore @@ -232,3 +232,6 @@ deploy.properties # Ignore Gradle build output directory build + +# Remove unwanted input directories +input/ext_sim diff --git a/CHANGELOG.md b/CHANGELOG.md index 2513a80be2..878f757496 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -18,6 +18,8 @@ and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0 - Added capability of SystemParticipants to handle flexibility [#308](https://github.com/ie3-institute/simona/issues/308) - Added smart charging logic [#31](https://github.com/ie3-institute/simona/issues/31) and flex calculation in `EvcsAgent` [#332](https://github.com/ie3-institute/simona/issues/332) - Enhance output quotes of `RunSimona` [#743](https://github.com/ie3-institute/simona/issues/743) +- Output of accompanying thermal result models +- Implementation of Energy Management Agents [#204](https://github.com/ie3-institute/simona/issues/204) ### Changed - Adapted to changed data source in PSDM [#435](https://github.com/ie3-institute/simona/issues/435) diff --git a/build.gradle b/build.gradle index 84c534b02e..98b79d2b7e 100644 --- a/build.gradle +++ b/build.gradle @@ -153,6 +153,7 @@ dependencies { implementation 'org.scalanlp:breeze_2.13:2.1.0' // scientific calculations (http://www.scalanlp.org/) implementation 'de.lmu.ifi.dbs.elki:elki:0.7.5' // Statistics (for random load model) implementation 'org.jgrapht:jgrapht-core:1.5.2' + implementation 'org.apache.poi:poi-ooxml:5.2.3' // Reading xlsx files } tasks.withType(JavaCompile) { diff --git a/input/samples/vn_simona/vn_simona.conf b/input/samples/vn_simona/vn_simona.conf index fe2e05d439..5c1ac8f059 100644 --- a/input/samples/vn_simona/vn_simona.conf +++ b/input/samples/vn_simona/vn_simona.conf @@ -158,6 +158,15 @@ simona.runtime.participant.hp = { individualConfigs = [] } +simona.runtime.participant.em = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + ################################################################## # Event Configuration ################################################################## diff --git a/src/main/resources/config/config-template.conf b/src/main/resources/config/config-template.conf index 859d98e016..a656f0a365 100644 --- a/src/main/resources/config/config-template.conf +++ b/src/main/resources/config/config-template.conf @@ -54,6 +54,14 @@ EvcsRuntimeConfig { lowestEvSoc: Double | 0.2 # Defines the lowest possible state of charge (SoC) that an EV is allowed to uncharge in vehicle to grid (V2G) mode } +#@define extends BaseRuntimeConfig +EmRuntimeConfig { + baseRuntimeConfig: BaseRuntimeConfig # this entry is ignored by the config generator, + # but cannot removed bc otherwise EmRuntimeConfig is handled as String + pvFlex: Boolean | false + aggregateFlex: String | "SELF_OPT_EXCL_PV" +} + #@define extends BaseRuntimeConfig HpRuntimeConfig { baseRuntimeConfig: BaseRuntimeConfig # this entry is ignored by the config generator, @@ -304,6 +312,18 @@ simona.runtime.participant = { defaultConfig = HpRuntimeConfig # Mandatory default config (uuids are ignored, best provide "default") individualConfigs = [HpRuntimeConfig] } + em = { + defaultConfig = EmRuntimeConfig # Mandatory default config (uuids are ignored, best provide "default") + individualConfigs = [EmRuntimeConfig] + } +} + +#@optional +simona.runtime.rootEm = { + filePath: String + nodeId: String + timeSeriesType: String + threshold: Double } ################################################################## diff --git a/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala b/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala new file mode 100644 index 0000000000..f9b352f84c --- /dev/null +++ b/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala @@ -0,0 +1,484 @@ +/* + * © 2023. TU Dortmund University, + * Institute of Energy Systems, Energy Efficiency and Energy Economics, + * Research group Distribution grid planning and operation + */ + +package edu.ie3.simona.agent.em + +import edu.ie3.datamodel.models.input.system.EmInput +import edu.ie3.datamodel.models.result.system.{EmResult, FlexOptionsResult} +import edu.ie3.simona.agent.participant.data.Data.PrimaryData.ApparentPower +import edu.ie3.simona.agent.participant.statedata.BaseStateData.FlexControlledData +import edu.ie3.simona.config.SimonaConfig +import edu.ie3.simona.config.SimonaConfig.EmRuntimeConfig +import edu.ie3.simona.event.ResultEvent +import edu.ie3.simona.event.ResultEvent.{ + FlexOptionsResultEvent, + ParticipantResultEvent, +} +import edu.ie3.simona.event.notifier.NotifierConfig +import edu.ie3.simona.exceptions.CriticalFailureException +import edu.ie3.simona.model.em.{EmModelShell, EmTools, FlexTimeSeries} +import edu.ie3.simona.ontology.messages.SchedulerMessage.{ + Completion, + ScheduleActivation, +} +import edu.ie3.simona.ontology.messages.flex.FlexibilityMessage._ +import edu.ie3.simona.ontology.messages.flex.MinMaxFlexibilityMessage.ProvideMinMaxFlexOptions +import edu.ie3.simona.ontology.messages.{Activation, SchedulerMessage} +import edu.ie3.simona.util.TickUtil.TickLong +import edu.ie3.util.quantities.QuantityUtils.RichQuantityDouble +import edu.ie3.util.scala.quantities.Megavars +import org.apache.pekko.actor.typed.scaladsl.Behaviors +import org.apache.pekko.actor.typed.{ActorRef, Behavior} +import squants.energy.{Kilowatts, Megawatts} + +import java.time.ZonedDateTime + +/** Energy management agent that receives flex options from and issues control + * messages to connected agents + * ([[edu.ie3.simona.agent.participant.ParticipantAgent]]s and subordinate + * [[EmAgent]]s) + */ +object EmAgent { + + type Actor = ActorRef[FlexRequest] + + /** Extended by all messages that an [[EmAgent]] can receive + */ + trait EmMessage + + /** Extended by all requests that an [[EmAgent]] can receive, i.e. + * activations, flex requests and control messages + */ + private sealed trait EmRequest extends EmMessage { + val tick: Long + } + + /** Wrapper for an [[Activation]] for usage by an adapter. Activations can + * only be received if this EM agent is not EM-controlled. + * + * @param tick + * The tick to activate + */ + private final case class EmActivation(override val tick: Long) + extends EmRequest + + /** Wrapper for [[FlexRequest]] messages for usage by an adapter (if this + * [[EmAgent]] is EM-controlled itself) + * + * @param msg + * The wrapped flex request + */ + private final case class Flex(msg: FlexRequest) extends EmRequest { + override val tick: Long = msg.tick + } + + /** Creates the initial [[Behavior]] for an [[EmAgent]] in an inactive state + * + * @param inputModel + * Model for simulation + * @param modelConfig + * Configuration for this type of model + * @param modelStrategy + * The model strategy to use + * @param outputConfig + * Config for the output behaviour of simulation results + * @param simulationStartDate + * Date of the very first tick in the simulation + * @param parent + * Either a [[Right]] with a reference to the parent [[EmAgent]] if this + * agent is em-controlled, or a [[Left]] with a reference to the scheduler + * that is activating this agent + * @param maybeRootEmConfig + * Config for the root EM agent, if applicable + * @param listener + * A collection of result event listeners + */ + def apply( + inputModel: EmInput, + modelConfig: EmRuntimeConfig, + outputConfig: NotifierConfig, + modelStrategy: String, + simulationStartDate: ZonedDateTime, + parent: Either[ActorRef[SchedulerMessage], ActorRef[FlexResponse]], + maybeRootEmConfig: Option[SimonaConfig.Simona.Runtime.RootEm] = None, + listener: Iterable[ActorRef[ResultEvent]], + ): Behavior[EmMessage] = Behaviors.setup { ctx => + val constantData = EmData( + outputConfig, + simulationStartDate, + parent + .map { parentEm => + val flexAdapter = ctx.messageAdapter[FlexRequest](Flex) + + parentEm ! RegisterParticipant( + inputModel.getUuid, + flexAdapter, + inputModel, + ) + + FlexControlledData(parentEm, flexAdapter) + } + .left + .map { scheduler => + { + val activationAdapter = ctx.messageAdapter[Activation] { msg => + EmActivation(msg.tick) + } + SchedulerData(scheduler, activationAdapter) + } + }, + maybeRootEmConfig.map( + FlexTimeSeries(_)(simulationStartDate) + ), + listener, + ) + + val modelShell = EmModelShell( + inputModel.getUuid, + inputModel.getId, + modelStrategy, + modelConfig, + ) + + inactive( + constantData, + modelShell, + EmDataCore.create(simulationStartDate), + ) + } + + /** Behavior of an inactive [[EmAgent]], which waits for an activation or flex + * request to be activated. + */ + private def inactive( + emData: EmData, + modelShell: EmModelShell, + core: EmDataCore.Inactive, + ): Behavior[EmMessage] = Behaviors.receivePartial { + + case (_, RegisterParticipant(model, actor, spi)) => + val updatedModelShell = modelShell.addParticipant(model, spi) + val updatedCore = core.addParticipant(actor, model) + inactive(emData, updatedModelShell, updatedCore) + + case (_, ScheduleFlexRequest(participant, newTick, scheduleKey)) => + val (maybeSchedule, newCore) = core + .handleSchedule(participant, newTick) + + maybeSchedule match { + case Some(scheduleTick) => + // also potentially schedule with parent if the new earliest tick is + // different from the old earliest tick (including if nothing had + // been scheduled before) + emData.parentData.fold( + schedulerData => + schedulerData.scheduler ! ScheduleActivation( + schedulerData.activationAdapter, + scheduleTick, + scheduleKey, + ), + _.emAgent ! ScheduleFlexRequest( + modelShell.uuid, + scheduleTick, + scheduleKey, + ), + ) + case None => + // we don't need to escalate to the parent, this means that we can + // release the lock (if applicable) + scheduleKey.foreach { + _.unlock() + } + } + inactive(emData, modelShell, newCore) + + case (ctx, msg: EmRequest) => + val flexOptionsCore = core.activate(msg.tick) + + msg match { + case Flex(_: RequestFlexOptions) | EmActivation(_) => + val (toActivate, newCore) = flexOptionsCore.takeNewFlexRequests() + toActivate.foreach { + _ ! RequestFlexOptions(msg.tick) + } + + awaitingFlexOptions(emData, modelShell, newCore) + + case Flex(_: IssueFlexControl) => + // We got sent a flex control message instead of a flex request, + // this means that flex options must have not changed since + // they were last calculated + + // Thus, we just jump to the appropriate place and forward the + // control message there + ctx.self ! msg + + awaitingFlexCtrl(emData, modelShell, flexOptionsCore) + } + + } + + /** Behavior of an [[EmAgent]] waiting for flex options to be received in + * order to transition to the next behavior. + */ + private def awaitingFlexOptions( + emData: EmData, + modelShell: EmModelShell, + flexOptionsCore: EmDataCore.AwaitingFlexOptions, + ): Behavior[EmMessage] = Behaviors.receiveMessagePartial { + case flexOptions: ProvideFlexOptions => + val updatedCore = flexOptionsCore.handleFlexOptions(flexOptions) + + if (updatedCore.isComplete) { + + val allFlexOptions = updatedCore.getFlexOptions + + emData.parentData match { + case Right(flexStateData) => + // aggregate flex options and provide to parent + val (ref, min, max) = + modelShell.aggregateFlexOptions(allFlexOptions) + + if (emData.outputConfig.flexResult) { + val flexResult = new FlexOptionsResult( + flexOptionsCore.activeTick.toDateTime( + emData.simulationStartDate + ), + modelShell.uuid, + ref.toMegawatts.asMegaWatt, + min.toMegawatts.asMegaWatt, + max.toMegawatts.asMegaWatt, + ) + + emData.listener.foreach { + _ ! FlexOptionsResultEvent(flexResult) + } + } + + val flexMessage = ProvideMinMaxFlexOptions( + modelShell.uuid, + ref, + min, + max, + ) + + flexStateData.emAgent ! flexMessage + + val updatedEmData = emData.copy( + parentData = Right( + flexStateData.copy( + lastFlexOptions = Some(flexMessage) + ) + ) + ) + + awaitingFlexCtrl(updatedEmData, modelShell, updatedCore) + + case Left(_) => + // if we're not EM-controlled ourselves, we're determining the set points + // either via flex time series or as 0 kW + val setPower = emData.flexTimeSeries match { + case Some(_) => + throw new NotImplementedError( + "Flex time series are currently not implemented" + ) + + case None => Kilowatts(0) + } + + val flexControl = + modelShell.determineFlexControl(allFlexOptions, setPower) + + val (allFlexMsgs, newCore) = updatedCore + .handleFlexCtrl(flexControl) + .fillInMissingIssueCtrl() + .complete() + + allFlexMsgs.foreach { case (actor, msg) => + actor ! msg + } + + awaitingCompletions(emData, modelShell, newCore) + } + + } else { + // more flex options expected + awaitingFlexOptions( + emData, + modelShell, + updatedCore, + ) + } + + /* We do not need to handle ScheduleFlexRequests here, since active agents + can schedule themselves with there completions and inactive agents should + be sleeping right now + */ + } + + /** Behavior of an [[EmAgent]] waiting for a flex control message to be + * received in order to transition to the next behavior. This behavior should + * only be used by EmAgents that are themselves EM-controlled. + */ + private def awaitingFlexCtrl( + emData: EmData, + modelShell: EmModelShell, + flexOptionsCore: EmDataCore.AwaitingFlexOptions, + ): Behavior[EmMessage] = Behaviors.receiveMessagePartial { + case Flex(flexCtrl: IssueFlexControl) => + val flexData = emData.parentData.getOrElse( + throw new CriticalFailureException(s"EmAgent is not EM-controlled.") + ) + + // flex options calculated by this EmAgent + val ownFlexOptions = flexData.lastFlexOptions.getOrElse( + throw new CriticalFailureException( + s"Flex options have not been calculated by EmAgent." + ) + ) + + val setPointActivePower = EmTools + .determineFlexPower( + ownFlexOptions, + flexCtrl, + ) + + // flex options calculated by connected agents + val receivedFlexOptions = flexOptionsCore.getFlexOptions + + val ctrlSetPoints = + modelShell.determineFlexControl( + receivedFlexOptions, + setPointActivePower, + ) + + val (allFlexMsgs, newCore) = flexOptionsCore + .handleFlexCtrl(ctrlSetPoints) + .fillInMissingIssueCtrl() + .complete() + + allFlexMsgs.foreach { case (actor, msg) => + actor ! msg + } + + awaitingCompletions(emData, modelShell, newCore) + } + + /** Behavior of an [[EmAgent]] waiting for completions messages to be received + * in order to transition to the inactive behavior. + */ + private def awaitingCompletions( + emData: EmData, + modelShell: EmModelShell, + core: EmDataCore.AwaitingCompletions, + ): Behavior[EmMessage] = Behaviors.receiveMessagePartial { + // Completions and results + case completion: FlexCtrlCompletion => + val updatedCore = core.handleCompletion(completion) + + updatedCore + .maybeComplete() + .map { inactiveCore => + sendCompletionCommunication( + emData, + modelShell, + inactiveCore, + lastActiveTick = updatedCore.activeTick, + ) + inactive(emData, modelShell, inactiveCore) + } + .getOrElse { + // more flex options expected + awaitingCompletions( + emData, + modelShell, + updatedCore, + ) + } + + } + + private def sendCompletionCommunication( + emData: EmData, + modelShell: EmModelShell, + inactiveCore: EmDataCore.Inactive, + lastActiveTick: Long, + ): Unit = { + // calc result + val result = inactiveCore.getResults + .reduceOption { (power1, power2) => + ApparentPower(power1.p + power2.p, power1.q + power2.q) + } + .getOrElse( + ApparentPower( + Megawatts(0d), + Megavars(0d), + ) + ) + + emData.listener.foreach { + _ ! ParticipantResultEvent( + new EmResult( + lastActiveTick + .toDateTime(emData.simulationStartDate), + modelShell.uuid, + result.p.toMegawatts.asMegaWatt, + result.q.toMegavars.asMegaVar, + ) + ) + } + + emData.parentData.fold( + schedulerData => + schedulerData.scheduler ! Completion( + schedulerData.activationAdapter, + inactiveCore.nextActiveTick, + ), + _.emAgent ! FlexCtrlCompletion( + modelShell.uuid, + result, + inactiveCore.hasFlexWithNext, + inactiveCore.nextActiveTick, + ), + ) + } + + /** Data that is supposed to stay (mostly) constant during simulation + * + * @param outputConfig + * Config for the output behaviour of simulation results + * @param simulationStartDate + * Date of the very first tick in the simulation + * @param parentData + * Either a [[Right]] with [[FlexControlledData]] if this agent is + * em-controlled, or a [[Left]] with [[SchedulerData]] + * @param flexTimeSeries + * Flex time series if this is the root EM + * @param listener + * A collection of result event listeners + */ + private final case class EmData( + outputConfig: NotifierConfig, + simulationStartDate: ZonedDateTime, + parentData: Either[SchedulerData, FlexControlledData], + flexTimeSeries: Option[FlexTimeSeries], + listener: Iterable[ActorRef[ResultEvent]], + ) + + /** The existence of this data object indicates that the corresponding agent + * is not EM-controlled, but activated by a + * [[edu.ie3.simona.scheduler.Scheduler]] + * + * @param scheduler + * The scheduler that is activating this agent + * @param activationAdapter + * The activation adapter handling [[Activation]] messages + */ + final case class SchedulerData( + scheduler: ActorRef[SchedulerMessage], + activationAdapter: ActorRef[Activation], + ) +} diff --git a/src/main/scala/edu/ie3/simona/agent/em/EmDataCore.scala b/src/main/scala/edu/ie3/simona/agent/em/EmDataCore.scala new file mode 100644 index 0000000000..3b46b176ee --- /dev/null +++ b/src/main/scala/edu/ie3/simona/agent/em/EmDataCore.scala @@ -0,0 +1,460 @@ +/* + * © 2023. TU Dortmund University, + * Institute of Energy Systems, Energy Efficiency and Energy Economics, + * Research group Distribution grid planning and operation + */ + +package edu.ie3.simona.agent.em + +import edu.ie3.simona.exceptions.CriticalFailureException +import edu.ie3.simona.agent.participant.data.Data.PrimaryData.ApparentPower +import EmAgent.Actor +import FlexCorrespondenceStore.WithTime +import edu.ie3.simona.ontology.messages.flex.FlexibilityMessage._ +import edu.ie3.util.scala.collection.mutable.PriorityMultiBiSet +import squants.Power + +import java.time.ZonedDateTime +import java.util.UUID + +/** Data related to participant scheduling and flex correspondences within an + * [[EmAgent]]. Depending on the state of the EmAgent, different data is stored + * and retrieved. + */ +object EmDataCore { + + /** Creates a new instance of an (inactive) EmAgent data core. + * @param startDate + * The start date of the simulation + */ + def create(implicit startDate: ZonedDateTime): Inactive = + Inactive( + Map.empty, + PriorityMultiBiSet.empty, + Set.empty, + FlexCorrespondenceStore(), + None, + ) + + /** Data structure holding relevant data and providing methods that handle + * interactions with an inactive [[EmAgent]] + * + * @param modelToActor + * Map of model uuid to corresponding model actor + * @param activationQueue + * Queue of flex request activations per tick + * @param flexWithNext + * UUIDs of agents to be activated with next activation, whatever tick that + * is going to be (the next tick can be changed when agents are + * (re-)scheduled) + * @param correspondences + * The data structure storing received and sent flex messages with the + * corresponding tick + * @param lastActiveTick + * The last active tick, if applicable + */ + final case class Inactive private ( + private val modelToActor: Map[UUID, Actor], + private val activationQueue: PriorityMultiBiSet[Long, UUID], + private val flexWithNext: Set[UUID], + private val correspondences: FlexCorrespondenceStore, + private val lastActiveTick: Option[Long], + ) { + + /** Adds a connected agent, given its model UUID and actor reference + * @param actor + * The agent's [[org.apache.pekko.actor.typed.ActorRef]] + * @param model + * The agent's model UUID + * @return + * The adapted [[Inactive]] core + */ + def addParticipant(actor: Actor, model: UUID): Inactive = + copy( + modelToActor = modelToActor.updated(model, actor) + ) + + /** Tries to handle an activation of the EmAgent for given tick. If the + * activation for the tick is not valid, a [[CriticalFailureException]] is + * thrown. If successful, an [[AwaitingFlexOptions]] data core is returned + * with the active tick set to the earliest tick scheduled. + * + * @param newTick + * The tick that the scheduler is to be activated with + * @return + * The changed [[AwaitingFlexOptions]] that should be used for the + * activated EM agent + * @throws CriticalFailureException + * on critical error + */ + def activate(newTick: Long): AwaitingFlexOptions = { + activationQueue.headKeyOption.foreach { nextScheduledTick => + if (newTick > nextScheduledTick) + throw new CriticalFailureException( + s"Cannot activate with new tick $newTick because the next scheduled tick $nextScheduledTick needs to be activated first." + ) + } + + // schedule flex requests for those participants which + // want to be asked at the next active tick, whatever + // that tick is going to be + val updatedQueue = flexWithNext.foldLeft(activationQueue) { + case (currentQueue, model) => + currentQueue.set(newTick, model) + currentQueue + } + + AwaitingFlexOptions( + modelToActor, + updatedQueue, + correspondences, + activeTick = newTick, + ) + } + + /** Tries to handle the scheduling a flex request for a connected agent for + * given tick. If scheduling for the tick is not valid, a + * [[CriticalFailureException]] is thrown. If, on the other hand, the flex + * request scheduling is successful and makes a separate scheduling of the + * current [[EmAgent]] with its parent necessary, the tick that the EM + * agent needs to be scheduled for is returned. + * + * @param model + * The model UUID of the agent to be scheduled + * @param newTick + * The tick that the agent is scheduled for + * @return + * A tuple of the optional tick that the current EM agent should be + * scheduled for with its parent, and the changed [[Inactive]] core + * @throws CriticalFailureException + * on critical error + */ + def handleSchedule( + model: UUID, + newTick: Long, + ): (Option[Long], Inactive) = { + lastActiveTick.filter(newTick <= _).foreach { lastActive => + throw new CriticalFailureException( + s"Cannot schedule a flex request for $model at tick $newTick because the last active tick was $lastActive" + ) + } + + val oldEarliestTick = activationQueue.headKeyOption + + activationQueue.set(newTick, model) + val newEarliestTick = activationQueue.headKeyOption + + val maybeScheduleTick = + Option + .when(newEarliestTick != oldEarliestTick)(newEarliestTick) + .flatten + + (maybeScheduleTick, this) + } + + def hasFlexWithNext: Boolean = flexWithNext.nonEmpty + + /** Returns the tick that will be activated next (if applicable) at the + * current state. + */ + def nextActiveTick: Option[Long] = + activationQueue.headKeyOption + + /** Returns relevant results for all connected agents. + */ + def getResults: Iterable[ApparentPower] = + correspondences.store.values.flatMap(_.receivedResult.map(_.get)) + + } + + /** Data structure holding relevant data and providing methods that handle + * interactions with an [[EmAgent]] that is waiting to receive all relevant + * flex options and subsequently calculate flex control + * + * @param modelToActor + * Map of model uuid to corresponding model actor + * @param activationQueue + * Queue of flex request activations per tick + * @param correspondences + * The data structure storing received and sent flex messages with the + * corresponding tick + * @param awaitedFlexOptions + * The set of model uuids, from which flex options are still expected + * @param activeTick + * The currently active tick + */ + final case class AwaitingFlexOptions( + private val modelToActor: Map[UUID, Actor], + private val activationQueue: PriorityMultiBiSet[Long, UUID], + private val correspondences: FlexCorrespondenceStore, + private val awaitedFlexOptions: Set[UUID] = Set.empty, + activeTick: Long, + ) { + + /** Removes and returns flex requests scheduled for the current tick, which + * can be sent out at the current moment. + * + * @return + * A tuple of a collection of agents scheduled for the current tick, and + * the updated [[AwaitingFlexOptions]] core + * @throws CriticalFailureException + * on critical error + */ + def takeNewFlexRequests(): (Iterable[Actor], AwaitingFlexOptions) = { + val toActivate = activationQueue.getAndRemoveSet(activeTick) + val newFlexOptionsCore = + copy(awaitedFlexOptions = awaitedFlexOptions.concat(toActivate)) + + val actors = toActivate.map { modelUuid => + modelToActor + .getOrElse( + modelUuid, + throw new CriticalFailureException( + s"Could not find actor for model uuid $modelUuid" + ), + ) + } + + (actors, newFlexOptionsCore) + } + + /** Handles the retrieval of flex options sent by some connected agent for + * the currently active tick. + * + * @param flexOptions + * The received flex options + * @return + * The updated [[AwaitingFlexOptions]] core + */ + def handleFlexOptions( + flexOptions: ProvideFlexOptions + ): AwaitingFlexOptions = + copy( + correspondences = + correspondences.updateFlexOptions(flexOptions, activeTick), + awaitedFlexOptions = awaitedFlexOptions.excl(flexOptions.modelUuid), + ) + + /** Checks whether all awaited flex options have been received and we can + * continue by calculating flex control. This method does not change the + * state of the [[AwaitingFlexOptions]] data core. + * @return + * true if all awaited flex options have been received + */ + def isComplete: Boolean = awaitedFlexOptions.isEmpty + + /** Returns all flex options that are currently relevant, which can include + * flex options received at an earlier tick + * @return + * all relevant flex options + */ + def getFlexOptions: Iterable[(UUID, ProvideFlexOptions)] = + correspondences.store.flatMap { case (model, correspondence) => + correspondence.receivedFlexOptions.map(model -> _.get) + } + + /** Handles and stores the control messages created by this [[EmAgent]] + * + * @param ctrlMsgs + * The control messages created by this EM agent + * @return + * The updated [[AwaitingFlexOptions]] core + */ + def handleFlexCtrl( + ctrlMsgs: Iterable[(UUID, Power)] + ): AwaitingFlexOptions = { + val updatedStore = ctrlMsgs.foldLeft(correspondences) { + case (store, (model, power)) => + val ctrlMsg = IssuePowerControl(activeTick, power) + store.updateFlexControl(model, ctrlMsg, activeTick) + } + copy(correspondences = updatedStore) + } + + /** The model strategy might miss control messages when creating them in + * bulk. This method creates the missing messages, in particular for those + * agents that have been issued a flex request for the current tick and + * those that have received a control messages at an earlier tick. + * @return + * The updated [[AwaitingFlexOptions]] core + */ + def fillInMissingIssueCtrl(): AwaitingFlexOptions = { + val updatedStore = correspondences.store + .filter { case (_, correspondence) => + // let's get those correspondences that have not received a ctrl msg at this tick + correspondence.issuedCtrlMsg.forall(_.tick < activeTick) + } + .flatMap { case (participant, correspondence) => + // We still create a no-control-msg in its place, if... + + // ... a response is expected for this tick, since we've + // requested flex options at this tick + val currentlyRequested = + correspondence.receivedFlexOptions.forall(_.tick == activeTick) + + // ... or flex control has been issued for this participant + // at an earlier tick + val flexControlCancelled = correspondence.issuedCtrlMsg match { + case Some(WithTime(_: IssuePowerControl, tick)) + if tick < activeTick => + true + case _ => false + } + + Option.when(currentlyRequested || flexControlCancelled)( + participant -> IssueNoControl(activeTick) + ) + } + .foldLeft(correspondences) { + case (updatedStore, (participant, flexCtrl)) => + updatedStore.updateFlexControl(participant, flexCtrl, activeTick) + } + copy(correspondences = updatedStore) + } + + /** Completes the current state by collecting and returning the control + * messages for the current tick if possible, and otherwise a + * [[CriticalFailureException]] is thrown + * + * @return + * A collection of agent-and-message pairs and an updated + * [[AwaitingCompletions]] core + * @throws CriticalFailureException + * on critical error + */ + def complete() + : (Iterable[(Actor, IssueFlexControl)], AwaitingCompletions) = { + + val modelUuidToMsg = correspondences.store.flatMap { + case (modelUuid, correspondence) => + correspondence.issuedCtrlMsg.flatMap { + case WithTime(issueCtrl, tick) if tick == activeTick => + Some(modelUuid -> issueCtrl) + case _ => None + } + } + + val actorToMsg = modelUuidToMsg.map { case (modelUuid, issueCtrl) => + modelToActor + .getOrElse( + modelUuid, + throw new CriticalFailureException( + s"Could not find actor for model uuid $modelUuid" + ), + ) -> issueCtrl + } + + ( + actorToMsg, + AwaitingCompletions( + modelToActor, + activationQueue = activationQueue, + correspondences = correspondences, + awaitedCompletions = modelUuidToMsg.map { case (participant, _) => + participant + }.toSet, + activeTick = activeTick, + ), + ) + } + + } + + /** Data structure holding relevant data and providing methods that handle + * interactions with an [[EmAgent]] that is waiting to receive flex + * completions from all active connected agents (those that received flex + * control in this tick) + * + * @param modelToActor + * Map of model uuid to corresponding model actor + * @param activationQueue + * Queue of flex request activations per tick + * @param flexWithNext + * to be asked for flex options with the following active tick, whatever + * that tick is going to be (not with the currently active tick though!) + * @param correspondences + * The data structure storing received and sent flex messages with the + * corresponding tick + * @param awaitedCompletions + * The set of model uuids, from which flex completions are still expected + * @param activeTick + * The currently active tick + */ + final case class AwaitingCompletions( + private val modelToActor: Map[UUID, Actor], + private val activationQueue: PriorityMultiBiSet[Long, UUID], + private val flexWithNext: Set[UUID] = Set.empty, + private val correspondences: FlexCorrespondenceStore, + private val awaitedCompletions: Set[UUID], + activeTick: Long, + ) { + + /** Tries to handle the completion of some connected agent for the currently + * active tick. If completion is not valid, a [[CriticalFailureException]] + * is thrown. + * + * @param completion + * The completion message that has been received + * @return + * The updated [[AwaitingCompletions]] core + * @throws CriticalFailureException + * on critical error + */ + def handleCompletion( + completion: FlexCtrlCompletion + ): AwaitingCompletions = { + if (!awaitedCompletions.contains(completion.modelUuid)) + throw new CriticalFailureException( + s"Participant ${completion.modelUuid} is not part of the expected completing participants" + ) + + // mutable queue + completion.requestAtTick + .foreach { activationQueue.set(_, completion.modelUuid) } + + val updatedCorrespondence = + correspondences.updateResult( + completion.modelUuid, + completion.result, + activeTick, + ) + + val updatedFlexWithNext = + if (completion.requestAtNextActivation) + flexWithNext.incl(completion.modelUuid) + else flexWithNext + + copy( + correspondences = updatedCorrespondence, + flexWithNext = updatedFlexWithNext, + awaitedCompletions = awaitedCompletions.excl(completion.modelUuid), + ) + } + + /** Checks whether the current activation of the [[EmAgent]] can be + * completed, which is usually the case when all activated connected agents + * have completed and there are no new flex requests that can be sent out + * for the current tick. + * + * @return + * If the current activation of the EM agent can be completed, the + * [[Inactive]] data core is returned that should be used in the + * following inactive state. + */ + def maybeComplete(): Option[Inactive] = + Option.when( + awaitedCompletions.isEmpty && + !activationQueue.headKeyOption.contains(activeTick) + ) { + Inactive( + modelToActor, + activationQueue, + flexWithNext, + correspondences, + Some(activeTick), + ) + } + + } + +} diff --git a/src/main/scala/edu/ie3/simona/agent/em/FlexCorrespondenceStore.scala b/src/main/scala/edu/ie3/simona/agent/em/FlexCorrespondenceStore.scala new file mode 100644 index 0000000000..5d69a5190a --- /dev/null +++ b/src/main/scala/edu/ie3/simona/agent/em/FlexCorrespondenceStore.scala @@ -0,0 +1,143 @@ +/* + * © 2022. TU Dortmund University, + * Institute of Energy Systems, Energy Efficiency and Energy Economics, + * Research group Distribution grid planning and operation + */ + +package edu.ie3.simona.agent.em + +import edu.ie3.simona.agent.em.FlexCorrespondenceStore.{ + FlexCorrespondence, + WithTime, +} +import edu.ie3.simona.agent.participant.data.Data.PrimaryData.ApparentPower +import edu.ie3.simona.ontology.messages.flex.FlexibilityMessage.{ + IssueFlexControl, + ProvideFlexOptions, +} + +import java.time.ZonedDateTime +import java.util.UUID + +/** Data structure that supports storing flex correspondences, i.e. flex + * messages that have been sent by various flexibility providers and received + * by an [[EmAgent]]. This correspondence store only stores the last received + * message of each type per flex provider. + * + * @param store + * Map that stores a flex correspondence per flex provider model UUID + * @param startDate + * The start date of the simulation used for calculations involving ticks + */ +final case class FlexCorrespondenceStore( + store: Map[UUID, FlexCorrespondence] = Map.empty +)(implicit val startDate: ZonedDateTime) { + + /** Updates the latest flex options for the flex provider, overwriting the + * former flex options, if applicable + * + * @param flexOptions + * The new flex options + * @param tick + * The tick that the flex options were received at + * @return + * The updated flex options store + */ + def updateFlexOptions( + flexOptions: ProvideFlexOptions, + tick: Long, + ): FlexCorrespondenceStore = + updateCorrespondence( + flexOptions.modelUuid, + _.copy(receivedFlexOptions = Some(WithTime(flexOptions, tick))), + ) + + /** Updates the latest flex control for the flex provider, overwriting the + * former flex control, if applicable + * + * @param modelUuid + * The UUID of the flex provider model + * @param flexControl + * The new flex control message sent + * @param tick + * The tick that the flex control message was sent at + * @return + * The updated flex options store + */ + def updateFlexControl( + modelUuid: UUID, + flexControl: IssueFlexControl, + tick: Long, + ): FlexCorrespondenceStore = + updateCorrespondence( + modelUuid, + _.copy(issuedCtrlMsg = Some(WithTime(flexControl, tick))), + ) + + /** Updates the latest result for the flex provider, overwriting the former + * result, if applicable + * + * @param modelUuid + * The UUID of the flex provider model + * @param result + * The new result + * @param tick + * The tick that the result was received at + * @return + * The updated flex options store + */ + def updateResult( + modelUuid: UUID, + result: ApparentPower, + tick: Long, + ): FlexCorrespondenceStore = + updateCorrespondence( + modelUuid, + _.copy(receivedResult = Some(WithTime(result, tick))), + ) + + private def updateCorrespondence( + modelUuid: UUID, + update: FlexCorrespondence => FlexCorrespondence, + ): FlexCorrespondenceStore = { + val correspondence = store.getOrElse( + modelUuid, + FlexCorrespondence(), + ) + copy(store = store.updated(modelUuid, update(correspondence))) + } + +} + +object FlexCorrespondenceStore { + + /** Class that holds flex messages for one flex provider. Only the latest + * messages of a type are stored with the tick that they were received. + * + * @param receivedFlexOptions + * The latest flex options that have been received by the EmAgent + * @param issuedCtrlMsg + * The latest flex control message that has been sent to the flex provider + * @param receivedResult + * The latest result that has been received by the EmAgent + */ + final case class FlexCorrespondence( + receivedFlexOptions: Option[WithTime[ProvideFlexOptions]] = None, + issuedCtrlMsg: Option[WithTime[IssueFlexControl]] = None, + receivedResult: Option[WithTime[ApparentPower]] = None, + ) + + /** Wrapper that allows storing a tick with an object + * + * @param obj + * The object + * @param tick + * The tick + * @tparam T + * The type of the object + */ + final case class WithTime[T](private val obj: T, tick: Long) { + def get: T = obj + } + +} 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 8ea4b12f5d..195f5ee273 100644 --- a/src/main/scala/edu/ie3/simona/agent/grid/GridAgent.scala +++ b/src/main/scala/edu/ie3/simona/agent/grid/GridAgent.scala @@ -119,6 +119,7 @@ class GridAgent( TimeUtil.withDefaults .toZonedDateTime(simonaConfig.simona.time.endDateTime), simonaConfig.simona.runtime.participant, + simonaConfig.simona.runtime.rootEm, simonaConfig.simona.output.participant, resolution, listener, diff --git a/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala b/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala index 895afb7cc1..76ac7796bf 100644 --- a/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala +++ b/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala @@ -6,14 +6,22 @@ package edu.ie3.simona.agent.grid -import org.apache.pekko.actor.typed.scaladsl.adapter.ClassicActorRefOps -import org.apache.pekko.actor.{ActorContext, ActorRef} -import org.apache.pekko.event.LoggingAdapter import com.typesafe.scalalogging.LazyLogging -import edu.ie3.datamodel.models.input.container.{SubGridContainer, ThermalGrid} +import edu.ie3.datamodel.models.ControlStrategy +import edu.ie3.datamodel.models.input.container.{ + SubGridContainer, + SystemParticipants, + ThermalGrid, +} import edu.ie3.datamodel.models.input.system._ +import edu.ie3.datamodel.models.input.system.characteristic.CosPhiFixed +import edu.ie3.datamodel.models.input.{AssetInput, NodeInput} +import edu.ie3.datamodel.models.voltagelevels.GermanVoltageLevelUtils +import edu.ie3.simona.actor.SimonaActorNaming import edu.ie3.simona.actor.SimonaActorNaming._ import edu.ie3.simona.agent.EnvironmentRefs +import edu.ie3.simona.agent.em.EmAgent +import edu.ie3.simona.agent.em.EmAgent.EmMessage import edu.ie3.simona.agent.participant.data.secondary.SecondaryDataService.{ ActorEvMovementsService, ActorWeatherService, @@ -24,15 +32,29 @@ import edu.ie3.simona.agent.participant.hp.HpAgent import edu.ie3.simona.agent.participant.load.LoadAgent import edu.ie3.simona.agent.participant.pv.PvAgent import edu.ie3.simona.agent.participant.statedata.ParticipantStateData.ParticipantInitializeStateData +import edu.ie3.simona.agent.participant.storage.StorageAgent import edu.ie3.simona.agent.participant.wec.WecAgent import edu.ie3.simona.config.SimonaConfig import edu.ie3.simona.config.SimonaConfig._ +import edu.ie3.simona.event.ResultEvent import edu.ie3.simona.event.notifier.NotifierConfig import edu.ie3.simona.exceptions.agent.GridAgentInitializationException +import edu.ie3.simona.ontology.messages.flex.FlexibilityMessage.FlexResponse +import edu.ie3.simona.ontology.messages.SchedulerMessage import edu.ie3.simona.ontology.messages.SchedulerMessage.ScheduleActivation import edu.ie3.simona.util.ConfigUtil import edu.ie3.simona.util.ConfigUtil._ import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK +import edu.ie3.util.quantities.QuantityUtils.RichQuantityDouble +import org.apache.pekko.actor.typed.ActorRef +import org.apache.pekko.actor.typed.scaladsl.adapter.{ + ClassicActorContextOps, + ClassicActorRefOps, + TypedActorRefOps, +} +import org.apache.pekko.actor.{ActorContext, ActorRef => ClassicActorRef} +import org.apache.pekko.event.LoggingAdapter +import org.locationtech.jts.geom.{Coordinate, GeometryFactory} import java.time.ZonedDateTime import java.util.UUID @@ -67,15 +89,16 @@ class GridAgentController( simulationStartDate: ZonedDateTime, simulationEndDate: ZonedDateTime, participantsConfig: SimonaConfig.Simona.Runtime.Participant, + rootEmConfig: Option[SimonaConfig.Simona.Runtime.RootEm], outputConfig: SimonaConfig.Simona.Output.Participant, resolution: Long, - listener: Iterable[ActorRef], + listener: Iterable[ClassicActorRef], log: LoggingAdapter, ) extends LazyLogging { def buildSystemParticipants( subGridContainer: SubGridContainer, thermalIslandGridsByBusId: Map[UUID, ThermalGrid], - ): Map[UUID, Set[ActorRef]] = { + ): Map[UUID, Set[ClassicActorRef]] = { val systemParticipants = filterSysParts(subGridContainer, environmentRefs) @@ -84,30 +107,38 @@ class GridAgentController( buildParticipantToActorRef( participantsConfig, outputConfig, + subGridContainer.getSystemParticipants, systemParticipants, thermalIslandGridsByBusId, environmentRefs, + rootEmConfig, + subGridContainer.getSubnet, ) } /** Takes the provided [[SubGridContainer]] and removes all * [[SystemParticipantInput]] of which no agent implementations are available - * at the moment. This method needs to be adapted whenever a new agent - * implementation is ready. Hopefully, it can be removed soon. + * at the moment or which are connected to some EM system. This method needs + * to be adapted whenever a new agent implementation is ready. * - * To disabled a filter fo a specific asset adapt the code below. + * To disable a filter for a specific system participant, adapt the code + * below. * * @param subGridContainer * the original subGrid container * @return - * a filtered subGrid container w/o assets no agent implementations exist - * atm + * a filtered subGrid container w/o assets for which no agent + * implementations exist atm */ private def filterSysParts( subGridContainer: SubGridContainer, environmentRefs: EnvironmentRefs, ) = { + val emParticipants = + subGridContainer.getSystemParticipants.getEmSystems.asScala + .flatMap(_.getConnectedAssets) + val (notProcessedElements, availableSysParts) = subGridContainer.getSystemParticipants .allEntitiesAsList() @@ -118,8 +149,7 @@ class GridAgentController( curSysPart, ) => curSysPart match { - case entity @ (_: BmInput | _: ChpInput | _: EvInput | - _: StorageInput) => + case entity @ (_: BmInput | _: ChpInput | _: EvInput) => ( notProcessedElements + entity.getClass.getSimpleName, availableSystemParticipants, @@ -131,6 +161,12 @@ class GridAgentController( s"Evcs ${evcsInput.getId} has been removed because no ev movements service is present." ) (notProcessedElements, availableSystemParticipants) + case entity if emParticipants.contains(entity.getUuid) => + log.debug( + s"System participant {} is part of an energy-managed system and thus not directly connected to the grid.", + entity, + ) + (notProcessedElements, availableSystemParticipants) case entity => (notProcessedElements, availableSystemParticipants :+ entity) } @@ -155,7 +191,7 @@ class GridAgentController( * Configuration information for participant models * @param outputConfig * Configuration information for output behaviour - * @param participants + * @param filteredParticipants * Set of system participants to create agents for * @param thermalIslandGridsByBusId * Collection of thermal island grids, mapped by their thermal bus uuid @@ -167,18 +203,102 @@ class GridAgentController( private def buildParticipantToActorRef( participantsConfig: SimonaConfig.Simona.Runtime.Participant, outputConfig: SimonaConfig.Simona.Output.Participant, - participants: Vector[SystemParticipantInput], + allParticipants: SystemParticipants, + filteredParticipants: Vector[SystemParticipantInput], thermalIslandGridsByBusId: Map[UUID, ThermalGrid], environmentRefs: EnvironmentRefs, - ): Map[UUID, Set[ActorRef]] = { + rootEmConfig: Option[SimonaConfig.Simona.Runtime.RootEm], + subGrid: Int, + ): Map[UUID, Set[ClassicActorRef]] = { /* Prepare the config util for the participant models, which (possibly) utilizes as map to speed up the initialization * phase */ val participantConfigUtil = ConfigUtil.ParticipantConfigUtil(participantsConfig) val outputConfigUtil = ConfigUtil.OutputConfigUtil(outputConfig) - participants - .map(participant => { + val emParticipantsUuids = + allParticipants.getEmSystems.asScala + .flatMap(_.getConnectedAssets) + + val emParticipantMap = allParticipants + .allEntitiesAsList() + .asScala + .filter(sp => emParticipantsUuids.contains(sp.getUuid)) + .map(sp => sp.getUuid -> sp) + .toMap + + val emUnits = allParticipants.getEmSystems.asScala + + val participantToEm = emUnits + .flatMap(em => em.getConnectedAssets.toSeq.map(_ -> em.getUuid)) + .toMap + + val (uncontrolledEms, controlledEms) = + emUnits.map(em => em -> participantToEm.get(em.getUuid)).partitionMap { + case (em, None) => Left(em) + case (em, Some(parentEm)) => Right(em -> parentEm) + } + + val uncontrolledEmActors = uncontrolledEms.map { em => + buildEm( + em, + participantConfigUtil.getOrDefault[EmRuntimeConfig]( + em.getUuid + ), + outputConfigUtil.getOrDefault(NotifierIdentifier.Em), + maybeParentEm = None, + rootEmConfig = rootEmConfig, + ) + } + + val (emInputs, otherInputs) = filteredParticipants.partition { + case _: EmInput => true + case _ => false + } + + if (rootEmConfig.isDefined && emInputs.nonEmpty) { + val mockRootEmInput = new EmInput( + UUID.fromString(s"11111111-0000-0000-0000-${"%012d".format(subGrid)}"), + "Root EmAgent", + new NodeInput( + UUID.randomUUID(), + "Mock node for root EmAgent", + 1d.asPu, + false, + new GeometryFactory().createPoint(new Coordinate()), + GermanVoltageLevelUtils.LV, + 0, + ), + new CosPhiFixed("cosPhiFixed:{(0.00,0.90)}"), + emInputs.map(_.getUuid).toArray, + ControlStrategy.DefaultControlStrategies.NO_CONTROL_STRATEGY, + ) + + val completeEmParticipantMap = + emParticipantMap ++ emInputs.map(sp => sp.getUuid -> sp) + + val actorRef = buildEm( + mockRootEmInput, + EmRuntimeConfig( + calculateMissingReactivePowerWithModel = false, + 1d, + List.empty, + pvFlex = false, + aggregateFlex = "SIMPLE_SUM", + ), + outputConfigUtil.getOrDefault(NotifierIdentifier.Em), + None, + rootEmConfig, + ) + + // introduce to environment + introduceAgentToEnvironment(actorRef.toClassic) + } + + rootEmConfig + .map(_ => otherInputs) + .getOrElse(filteredParticipants) + .map { participant => val node = participant.getNode // build val actorRef = @@ -189,15 +309,43 @@ class GridAgentController( participant, thermalIslandGridsByBusId, environmentRefs, + emParticipantMap, ) introduceAgentToEnvironment(actorRef) // return uuid to actorRef node.getUuid -> actorRef - }) - .toSet[(UUID, ActorRef)] + } + .toSet[(UUID, ClassicActorRef)] .groupMap(entry => entry._1)(entry => entry._2) } + private def buildControlledEms() = { + // TODO recursive + } + + // TODO not needed anymore if psdm does this + private def matchWithEm[T <: AssetInput]( + existingEmActors: Map[UUID, ActorRef[EmMessage]], + remainingEms: Iterable[(T, UUID)], + ): Iterable[(T, ActorRef[EmMessage])] = { + remainingEms + .map { case (em, parentUuid) => + existingEmActors + .get(parentUuid) + .map(em -> _) + .toRight((em.getUuid, parentUuid)) + } + .partitionMap(identity) match { + // only return em actors if there are no missing parent ems + case (Nil, matchedAssets) => matchedAssets + case (emsAndParents, _) => + val (ems, parents) = emsAndParents.unzip + throw new GridAgentInitializationException( + s"The parent energy management unit(s) $ems could not be created because the em parent(s) ${parents.toSet} do not exist." + ) + } + } + private def buildParticipantActor( requestVoltageDeviationThreshold: Double, participantConfigUtil: ConfigUtil.ParticipantConfigUtil, @@ -205,7 +353,9 @@ class GridAgentController( participantInputModel: SystemParticipantInput, thermalIslandGridsByBusId: Map[UUID, ThermalGrid], environmentRefs: EnvironmentRefs, - ): ActorRef = participantInputModel match { + participantEmMap: Map[UUID, SystemParticipantInput], + maybeParentEm: Option[ActorRef[FlexResponse]] = None, + ): ClassicActorRef = participantInputModel match { case input: FixedFeedInInput => buildFixedFeedIn( input, @@ -218,6 +368,7 @@ class GridAgentController( resolution, requestVoltageDeviationThreshold, outputConfigUtil.getOrDefault(NotifierIdentifier.FixedFeedIn), + maybeParentEm, ) case input: LoadInput => buildLoad( @@ -231,6 +382,7 @@ class GridAgentController( resolution, requestVoltageDeviationThreshold, outputConfigUtil.getOrDefault(NotifierIdentifier.Load), + maybeParentEm, ) case input: PvInput => buildPv( @@ -245,6 +397,7 @@ class GridAgentController( resolution, requestVoltageDeviationThreshold, outputConfigUtil.getOrDefault(NotifierIdentifier.PvPlant), + maybeParentEm, ) case input: WecInput => buildWec( @@ -259,6 +412,7 @@ class GridAgentController( resolution, requestVoltageDeviationThreshold, outputConfigUtil.getOrDefault(NotifierIdentifier.Wec), + maybeParentEm, ) case input: EvcsInput => buildEvcs( @@ -277,6 +431,7 @@ class GridAgentController( resolution, requestVoltageDeviationThreshold, outputConfigUtil.getOrDefault(NotifierIdentifier.Evcs), + maybeParentEm, ) case hpInput: HpInput => thermalIslandGridsByBusId.get(hpInput.getThermalBus.getUuid) match { @@ -284,17 +439,34 @@ class GridAgentController( buildHp( hpInput, thermalGrid, - participantConfigUtil.getOrDefault(hpInput.getUuid), + participantConfigUtil.getOrDefault[HpRuntimeConfig]( + hpInput.getUuid + ), environmentRefs.primaryServiceProxy, environmentRefs.weather, requestVoltageDeviationThreshold, outputConfigUtil.getOrDefault(NotifierIdentifier.Hp), + maybeParentEm, ) case None => throw new GridAgentInitializationException( s"Unable to find thermal island grid for heat pump '${hpInput.getUuid}' with thermal bus '${hpInput.getThermalBus.getUuid}'." ) } + case input: StorageInput => + buildStorage( + input, + participantConfigUtil.getOrDefault[StorageRuntimeConfig]( + input.getUuid + ), + environmentRefs.primaryServiceProxy, + simulationStartDate, + simulationEndDate, + resolution, + requestVoltageDeviationThreshold, + outputConfigUtil.getOrDefault(NotifierIdentifier.Storage), + maybeParentEm, + ) case input: SystemParticipantInput => throw new NotImplementedError( s"Building ${input.getClass.getSimpleName} is not implemented, yet." @@ -324,19 +496,22 @@ class GridAgentController( * Maximum deviation in p.u. of request voltages to be considered equal * @param outputConfig * Configuration of the output behavior + * @param maybeParentEm + * The parent EmAgent, if applicable * @return - * The [[FixedFeedInAgent]] 's [[ActorRef]] + * The [[FixedFeedInAgent]] 's [[ClassicActorRef]] */ private def buildFixedFeedIn( fixedFeedInInput: FixedFeedInInput, modelConfiguration: FixedFeedInRuntimeConfig, - primaryServiceProxy: ActorRef, + primaryServiceProxy: ClassicActorRef, simulationStartDate: ZonedDateTime, simulationEndDate: ZonedDateTime, resolution: Long, requestVoltageDeviationThreshold: Double, outputConfig: NotifierConfig, - ): ActorRef = + maybeParentEm: Option[ActorRef[FlexResponse]] = None, + ): ClassicActorRef = gridAgentContext.simonaActorOf( FixedFeedInAgent.props( environmentRefs.scheduler, @@ -350,6 +525,7 @@ class GridAgentController( resolution, requestVoltageDeviationThreshold, outputConfig, + maybeParentEm, ), listener, ), @@ -375,19 +551,22 @@ class GridAgentController( * Maximum deviation in p.u. of request voltages to be considered equal * @param outputConfig * Configuration of the output behavior + * @param maybeParentEm + * The parent EmAgent, if applicable * @return - * The [[LoadAgent]] 's [[ActorRef]] + * The [[LoadAgent]] 's [[ClassicActorRef]] */ private def buildLoad( loadInput: LoadInput, modelConfiguration: LoadRuntimeConfig, - primaryServiceProxy: ActorRef, + primaryServiceProxy: ClassicActorRef, simulationStartDate: ZonedDateTime, simulationEndDate: ZonedDateTime, resolution: Long, requestVoltageDeviationThreshold: Double, outputConfig: NotifierConfig, - ): ActorRef = + maybeParentEm: Option[ActorRef[FlexResponse]] = None, + ): ClassicActorRef = gridAgentContext.simonaActorOf( LoadAgent.props( environmentRefs.scheduler, @@ -401,6 +580,7 @@ class GridAgentController( resolution, requestVoltageDeviationThreshold, outputConfig, + maybeParentEm, ), listener, ), @@ -428,20 +608,23 @@ class GridAgentController( * Maximum deviation in p.u. of request voltages to be considered equal * @param outputConfig * Configuration of the output behavior + * @param maybeParentEm + * The parent EmAgent, if applicable * @return - * The [[PvAgent]] 's [[ActorRef]] + * The [[PvAgent]] 's [[ClassicActorRef]] */ private def buildPv( pvInput: PvInput, modelConfiguration: PvRuntimeConfig, - primaryServiceProxy: ActorRef, - weatherService: ActorRef, + primaryServiceProxy: ClassicActorRef, + weatherService: ClassicActorRef, simulationStartDate: ZonedDateTime, simulationEndDate: ZonedDateTime, resolution: Long, requestVoltageDeviationThreshold: Double, outputConfig: NotifierConfig, - ): ActorRef = + maybeParentEm: Option[ActorRef[FlexResponse]] = None, + ): ClassicActorRef = gridAgentContext.simonaActorOf( PvAgent.props( environmentRefs.scheduler, @@ -455,6 +638,7 @@ class GridAgentController( resolution, requestVoltageDeviationThreshold, outputConfig, + maybeParentEm, ), listener, ), @@ -482,20 +666,23 @@ class GridAgentController( * Maximum deviation in p.u. of request voltages to be considered equal * @param outputConfig * Configuration of the output behavior + * @param maybeParentEm + * The parent EmAgent, if applicable * @return - * The [[EvcsAgent]] 's [[ActorRef]] + * The [[EvcsAgent]] 's [[ClassicActorRef]] */ private def buildEvcs( evcsInput: EvcsInput, modelConfiguration: EvcsRuntimeConfig, - primaryServiceProxy: ActorRef, - evMovementsService: ActorRef, + primaryServiceProxy: ClassicActorRef, + evMovementsService: ClassicActorRef, simulationStartDate: ZonedDateTime, simulationEndDate: ZonedDateTime, resolution: Long, requestVoltageDeviationThreshold: Double, outputConfig: NotifierConfig, - ): ActorRef = + maybeParentEm: Option[ActorRef[FlexResponse]] = None, + ): ClassicActorRef = gridAgentContext.simonaActorOf( EvcsAgent.props( environmentRefs.scheduler, @@ -513,12 +700,14 @@ class GridAgentController( resolution, requestVoltageDeviationThreshold, outputConfig, + maybeParentEm, ), listener, ) ) /** Builds an [[HpAgent]] from given input + * * @param hpInput * Input model * @param thermalGrid @@ -533,6 +722,8 @@ class GridAgentController( * Permissible voltage magnitude deviation to consider being equal * @param outputConfig * Configuration for output notification + * @param maybeParentEm + * The parent EmAgent, if applicable * @return * A tuple of actor reference and [[ParticipantInitializeStateData]] */ @@ -540,11 +731,12 @@ class GridAgentController( hpInput: HpInput, thermalGrid: ThermalGrid, modelConfiguration: HpRuntimeConfig, - primaryServiceProxy: ActorRef, - weatherService: ActorRef, + primaryServiceProxy: ClassicActorRef, + weatherService: ClassicActorRef, requestVoltageDeviationThreshold: Double, outputConfig: NotifierConfig, - ): ActorRef = + maybeParentEm: Option[ActorRef[FlexResponse]] = None, + ): ClassicActorRef = gridAgentContext.simonaActorOf( HpAgent.props( environmentRefs.scheduler, @@ -559,19 +751,20 @@ class GridAgentController( resolution, requestVoltageDeviationThreshold, outputConfig, + maybeParentEm, ), listener, ), hpInput.getId, ) - /** Creates a pv agent and determines the needed additional information for + /** Creates a wec agent and determines the needed additional information for * later initialization of the agent. * * @param wecInput * WEC input model to derive information from * @param modelConfiguration - * User-provided configuration for this specific load model + * User-provided configuration for this specific wec model * @param primaryServiceProxy * Reference to the primary data service proxy * @param weatherService @@ -586,20 +779,23 @@ class GridAgentController( * Maximum deviation in p.u. of request voltages to be considered equal * @param outputConfig * Configuration of the output behavior + * @param maybeParentEm + * The parent EmAgent, if applicable * @return - * The [[WecAgent]] 's [[ActorRef]] + * The [[WecAgent]] 's [[ClassicActorRef]] */ private def buildWec( wecInput: WecInput, modelConfiguration: WecRuntimeConfig, - primaryServiceProxy: ActorRef, - weatherService: ActorRef, + primaryServiceProxy: ClassicActorRef, + weatherService: ClassicActorRef, simulationStartDate: ZonedDateTime, simulationEndDate: ZonedDateTime, resolution: Long, requestVoltageDeviationThreshold: Double, outputConfig: NotifierConfig, - ): ActorRef = + maybeParentEm: Option[ActorRef[FlexResponse]] = None, + ): ClassicActorRef = gridAgentContext.simonaActorOf( WecAgent.props( environmentRefs.scheduler, @@ -613,19 +809,111 @@ class GridAgentController( resolution, requestVoltageDeviationThreshold, outputConfig, + maybeParentEm, ), listener, ), wecInput.getId, ) + /** Creates a storage agent and determines the needed additional information + * for later initialization of the agent. + * + * @param storageInput + * Storage input model to derive information from + * @param modelConfiguration + * User-provided configuration for this specific storage model + * @param primaryServiceProxy + * Reference to the primary data service proxy + * @param simulationStartDate + * First wall clock time in simulation + * @param simulationEndDate + * Last wall clock time in simulation + * @param resolution + * Frequency of power flow calculations + * @param requestVoltageDeviationThreshold + * Maximum deviation in p.u. of request voltages to be considered equal + * @param outputConfig + * Configuration of the output behavior + * @param maybeParentEm + * The parent EmAgent, if applicable + * @return + * The [[StorageAgent]] 's [[ClassicActorRef]] + */ + private def buildStorage( + storageInput: StorageInput, + modelConfiguration: SimonaConfig.StorageRuntimeConfig, + primaryServiceProxy: ClassicActorRef, + simulationStartDate: ZonedDateTime, + simulationEndDate: ZonedDateTime, + resolution: Long, + requestVoltageDeviationThreshold: Double, + outputConfig: NotifierConfig, + maybeParentEm: Option[ActorRef[FlexResponse]] = None, + ): ClassicActorRef = + gridAgentContext.simonaActorOf( + StorageAgent.props( + environmentRefs.scheduler, + ParticipantInitializeStateData( + storageInput, + modelConfiguration, + primaryServiceProxy, + None, + simulationStartDate, + simulationEndDate, + resolution, + requestVoltageDeviationThreshold, + outputConfig, + maybeParentEm, + ), + listener, + ), + storageInput.getId, + ) + + /** Builds an [[EmAgent]] from given input + * + * @param emInput + * Input model + * @param modelConfiguration + * Runtime configuration for the agent + * @param outputConfig + * Configuration for output notification + * @return + * A tuple of actor reference and [[ParticipantInitializeStateData]] + */ + private def buildEm( + emInput: EmInput, + modelConfiguration: EmRuntimeConfig, + outputConfig: NotifierConfig, + maybeParentEm: Option[ActorRef[FlexResponse]] = None, + rootEmConfig: Option[SimonaConfig.Simona.Runtime.RootEm] = None, + ): ActorRef[EmMessage] = + gridAgentContext.spawn( + EmAgent( + emInput, + modelConfiguration, + outputConfig, + rootEmConfig + .map(_ => "PROPORTIONAL") + .getOrElse("PRIORITIZED"), + simulationStartDate, + maybeParentEm.toRight( + environmentRefs.scheduler.toTyped[SchedulerMessage] + ), + rootEmConfig, + listener.map(_.toTyped[ResultEvent]), + ), + SimonaActorNaming.actorName(classOf[EmAgent.type], emInput.getId), + ) + /** Introduces the given agent to scheduler * * @param actorRef * Reference to the actor to add to the environment */ private def introduceAgentToEnvironment( - actorRef: ActorRef + actorRef: ClassicActorRef ): Unit = { gridAgentContext.watch(actorRef) environmentRefs.scheduler ! ScheduleActivation( diff --git a/src/main/scala/edu/ie3/simona/config/SimonaConfig.scala b/src/main/scala/edu/ie3/simona/config/SimonaConfig.scala index e437569147..7eed3958f0 100644 --- a/src/main/scala/edu/ie3/simona/config/SimonaConfig.scala +++ b/src/main/scala/edu/ie3/simona/config/SimonaConfig.scala @@ -79,6 +79,72 @@ object SimonaConfig { val isHierarchic: scala.Boolean, ) + final case class EmRuntimeConfig( + override val calculateMissingReactivePowerWithModel: scala.Boolean, + override val scaling: scala.Double, + override val uuids: scala.List[java.lang.String], + aggregateFlex: java.lang.String, + pvFlex: scala.Boolean, + ) extends BaseRuntimeConfig( + calculateMissingReactivePowerWithModel, + scaling, + uuids, + ) + object EmRuntimeConfig { + def apply( + c: com.typesafe.config.Config, + parentPath: java.lang.String, + $tsCfgValidator: $TsCfgValidator, + ): SimonaConfig.EmRuntimeConfig = { + SimonaConfig.EmRuntimeConfig( + aggregateFlex = + if (c.hasPathOrNull("aggregateFlex")) c.getString("aggregateFlex") + else "SELF_OPT_EXCL_PV", + pvFlex = c.hasPathOrNull("pvFlex") && c.getBoolean("pvFlex"), + calculateMissingReactivePowerWithModel = $_reqBln( + parentPath, + c, + "calculateMissingReactivePowerWithModel", + $tsCfgValidator, + ), + scaling = $_reqDbl(parentPath, c, "scaling", $tsCfgValidator), + uuids = $_L$_str(c.getList("uuids"), parentPath, $tsCfgValidator), + ) + } + private def $_reqBln( + parentPath: java.lang.String, + c: com.typesafe.config.Config, + path: java.lang.String, + $tsCfgValidator: $TsCfgValidator, + ): scala.Boolean = { + if (c == null) false + else + try c.getBoolean(path) + catch { + case e: com.typesafe.config.ConfigException => + $tsCfgValidator.addBadPath(parentPath + path, e) + false + } + } + + private def $_reqDbl( + parentPath: java.lang.String, + c: com.typesafe.config.Config, + path: java.lang.String, + $tsCfgValidator: $TsCfgValidator, + ): scala.Double = { + if (c == null) 0 + else + try c.getDouble(path) + catch { + case e: com.typesafe.config.ConfigException => + $tsCfgValidator.addBadPath(parentPath + path, e) + 0 + } + } + + } + final case class EvcsRuntimeConfig( override val calculateMissingReactivePowerWithModel: scala.Boolean, override val scaling: scala.Double, @@ -2125,6 +2191,7 @@ object SimonaConfig { final case class Runtime( listener: SimonaConfig.Simona.Runtime.Listener, participant: SimonaConfig.Simona.Runtime.Participant, + rootEm: scala.Option[SimonaConfig.Simona.Runtime.RootEm], selected_subgrids: scala.Option[scala.List[scala.Int]], selected_volt_lvls: scala.Option[scala.List[SimonaConfig.VoltLvlConfig]], ) @@ -2165,6 +2232,7 @@ object SimonaConfig { } final case class Participant( + em: SimonaConfig.Simona.Runtime.Participant.Em, evcs: SimonaConfig.Simona.Runtime.Participant.Evcs, fixedFeedIn: SimonaConfig.Simona.Runtime.Participant.FixedFeedIn, hp: SimonaConfig.Simona.Runtime.Participant.Hp, @@ -2174,6 +2242,51 @@ object SimonaConfig { wec: SimonaConfig.Simona.Runtime.Participant.Wec, ) object Participant { + final case class Em( + defaultConfig: SimonaConfig.EmRuntimeConfig, + individualConfigs: scala.List[SimonaConfig.EmRuntimeConfig], + ) + object Em { + def apply( + c: com.typesafe.config.Config, + parentPath: java.lang.String, + $tsCfgValidator: $TsCfgValidator, + ): SimonaConfig.Simona.Runtime.Participant.Em = { + SimonaConfig.Simona.Runtime.Participant.Em( + defaultConfig = SimonaConfig.EmRuntimeConfig( + if (c.hasPathOrNull("defaultConfig")) + c.getConfig("defaultConfig") + else + com.typesafe.config.ConfigFactory + .parseString("defaultConfig{}"), + parentPath + "defaultConfig.", + $tsCfgValidator, + ), + individualConfigs = $_LSimonaConfig_EmRuntimeConfig( + c.getList("individualConfigs"), + parentPath, + $tsCfgValidator, + ), + ) + } + private def $_LSimonaConfig_EmRuntimeConfig( + cl: com.typesafe.config.ConfigList, + parentPath: java.lang.String, + $tsCfgValidator: $TsCfgValidator, + ): scala.List[SimonaConfig.EmRuntimeConfig] = { + import scala.jdk.CollectionConverters._ + cl.asScala + .map(cv => + SimonaConfig.EmRuntimeConfig( + cv.asInstanceOf[com.typesafe.config.ConfigObject].toConfig, + parentPath, + $tsCfgValidator, + ) + ) + .toList + } + } + final case class Evcs( defaultConfig: SimonaConfig.EvcsRuntimeConfig, individualConfigs: scala.List[SimonaConfig.EvcsRuntimeConfig], @@ -2450,6 +2563,12 @@ object SimonaConfig { $tsCfgValidator: $TsCfgValidator, ): SimonaConfig.Simona.Runtime.Participant = { SimonaConfig.Simona.Runtime.Participant( + em = SimonaConfig.Simona.Runtime.Participant.Em( + if (c.hasPathOrNull("em")) c.getConfig("em") + else com.typesafe.config.ConfigFactory.parseString("em{}"), + parentPath + "em.", + $tsCfgValidator, + ), evcs = SimonaConfig.Simona.Runtime.Participant.Evcs( if (c.hasPathOrNull("evcs")) c.getConfig("evcs") else com.typesafe.config.ConfigFactory.parseString("evcs{}"), @@ -2495,6 +2614,60 @@ object SimonaConfig { } } + final case class RootEm( + filePath: java.lang.String, + nodeId: java.lang.String, + threshold: scala.Double, + timeSeriesType: java.lang.String, + ) + object RootEm { + def apply( + c: com.typesafe.config.Config, + parentPath: java.lang.String, + $tsCfgValidator: $TsCfgValidator, + ): SimonaConfig.Simona.Runtime.RootEm = { + SimonaConfig.Simona.Runtime.RootEm( + filePath = $_reqStr(parentPath, c, "filePath", $tsCfgValidator), + nodeId = $_reqStr(parentPath, c, "nodeId", $tsCfgValidator), + threshold = $_reqDbl(parentPath, c, "threshold", $tsCfgValidator), + timeSeriesType = + $_reqStr(parentPath, c, "timeSeriesType", $tsCfgValidator), + ) + } + private def $_reqDbl( + parentPath: java.lang.String, + c: com.typesafe.config.Config, + path: java.lang.String, + $tsCfgValidator: $TsCfgValidator, + ): scala.Double = { + if (c == null) 0 + else + try c.getDouble(path) + catch { + case e: com.typesafe.config.ConfigException => + $tsCfgValidator.addBadPath(parentPath + path, e) + 0 + } + } + + private def $_reqStr( + parentPath: java.lang.String, + c: com.typesafe.config.Config, + path: java.lang.String, + $tsCfgValidator: $TsCfgValidator, + ): java.lang.String = { + if (c == null) null + else + try c.getString(path) + catch { + case e: com.typesafe.config.ConfigException => + $tsCfgValidator.addBadPath(parentPath + path, e) + null + } + } + + } + def apply( c: com.typesafe.config.Config, parentPath: java.lang.String, @@ -2513,6 +2686,16 @@ object SimonaConfig { parentPath + "participant.", $tsCfgValidator, ), + rootEm = + if (c.hasPathOrNull("rootEm")) + scala.Some( + SimonaConfig.Simona.Runtime.RootEm( + c.getConfig("rootEm"), + parentPath + "rootEm.", + $tsCfgValidator, + ) + ) + else None, selected_subgrids = if (c.hasPathOrNull("selected_subgrids")) scala.Some( diff --git a/src/main/scala/edu/ie3/simona/io/grid/CsvGridSource.scala b/src/main/scala/edu/ie3/simona/io/grid/CsvGridSource.scala index 7726640c1f..7bdc49e066 100644 --- a/src/main/scala/edu/ie3/simona/io/grid/CsvGridSource.scala +++ b/src/main/scala/edu/ie3/simona/io/grid/CsvGridSource.scala @@ -7,7 +7,7 @@ package edu.ie3.simona.io.grid import edu.ie3.datamodel.io.naming.FileNamingStrategy -import edu.ie3.datamodel.io.source._ +import edu.ie3.datamodel.io.source.{ThermalSource, TypeSource} import edu.ie3.datamodel.io.source.csv._ import edu.ie3.datamodel.models.input.container._ import edu.ie3.datamodel.models.input.thermal.{ @@ -25,22 +25,24 @@ object CsvGridSource { baseFolder: Path, fileNamingStrategy: FileNamingStrategy, ): Map[ThermalBusInput, ThermalGrid] = { - val csvDataSource = + val dataSource = new CsvDataSource(csvSep, baseFolder, fileNamingStrategy) - val csvTypeSource: TypeSource = - new TypeSource(csvDataSource) - val csvThermalSource: ThermalSource = - new ThermalSource(csvTypeSource, csvDataSource) - val operators = csvTypeSource.getOperators - val busses = csvThermalSource.getThermalBuses() - val houses = csvThermalSource + val typeSource = new TypeSource(dataSource) + val thermalSource = new ThermalSource( + typeSource, + dataSource, + ) + + val operators = typeSource.getOperators + val busses = thermalSource.getThermalBuses() + val houses = thermalSource .getThermalHouses(operators, busses) .asScala .groupBy(thermalHouse => thermalHouse.getThermalBus) .map { case (bus, houses) => bus -> houses.toSet } - val storages = csvThermalSource + val storages = thermalSource .getThermalStorages(operators, busses) .asScala .groupBy(thermalStorage => thermalStorage.getThermalBus) diff --git a/src/main/scala/edu/ie3/simona/model/em/EmAggregateFlex.scala b/src/main/scala/edu/ie3/simona/model/em/EmAggregateFlex.scala new file mode 100644 index 0000000000..16d7f8a05a --- /dev/null +++ b/src/main/scala/edu/ie3/simona/model/em/EmAggregateFlex.scala @@ -0,0 +1,29 @@ +/* + * © 2022. TU Dortmund University, + * Institute of Energy Systems, Energy Efficiency and Energy Economics, + * Research group Distribution grid planning and operation + */ + +package edu.ie3.simona.model.em + +import edu.ie3.datamodel.models.input.AssetInput +import edu.ie3.simona.ontology.messages.flex.MinMaxFlexibilityMessage.ProvideMinMaxFlexOptions + +/** Offers method for aggregating flex options from connected agents which will + * then be provided to a superior EmAgent or sent out as a flex result + */ +trait EmAggregateFlex { + + /** Aggregates flex options of connected devices to one flex options object + * that describes the flexibility of this EmAgent + * @param flexOptions + * the flex options of all connected agents + * @return + * aggregated reference, minimum and maximum power + */ + def aggregateFlexOptions( + flexOptions: Iterable[ + (_ <: AssetInput, ProvideMinMaxFlexOptions) + ] + ): (squants.Power, squants.Power, squants.Power) +} diff --git a/src/main/scala/edu/ie3/simona/model/em/EmAggregateSelfOpt.scala b/src/main/scala/edu/ie3/simona/model/em/EmAggregateSelfOpt.scala new file mode 100644 index 0000000000..23aea26472 --- /dev/null +++ b/src/main/scala/edu/ie3/simona/model/em/EmAggregateSelfOpt.scala @@ -0,0 +1,60 @@ +/* + * © 2022. TU Dortmund University, + * Institute of Energy Systems, Energy Efficiency and Energy Economics, + * Research group Distribution grid planning and operation + */ + +package edu.ie3.simona.model.em + +import edu.ie3.datamodel.models.input.AssetInput +import edu.ie3.datamodel.models.input.system.PvInput +import edu.ie3.simona.ontology.messages.flex.MinMaxFlexibilityMessage.ProvideMinMaxFlexOptions +import squants.energy.Kilowatts + +/** Aggregates flex reference power with the target of reaching 0kW, while + * optionally excluding positive PV potential from the calculation + * + * @param pvFlex + * Whether to include positive PV flexibility in reference sum calculation + */ +final case class EmAggregateSelfOpt(pvFlex: Boolean) extends EmAggregateFlex { + + override def aggregateFlexOptions( + flexOptions: Iterable[ + (_ <: AssetInput, ProvideMinMaxFlexOptions) + ] + ): (squants.Power, squants.Power, squants.Power) = { + val (minSum, maxSum) = + flexOptions.foldLeft((Kilowatts(0d), Kilowatts(0d))) { + case ( + (sumMin, sumMax), + (_, ProvideMinMaxFlexOptions(_, _, addMin, addMax)), + ) => + ( + sumMin + addMin, + sumMax + addMax, + ) + } + + val maxRefSum = + if (pvFlex) + maxSum + else + flexOptions.foldLeft(Kilowatts(0d)) { + case ( + maxSumExclPv, + (inputModel, ProvideMinMaxFlexOptions(_, _, addMin, addMax)), + ) => + inputModel match { + case _: PvInput => + maxSumExclPv + addMin + case _ => maxSumExclPv + addMax + } + } + + // take the closest power possible to zero + val refAgg = minSum.max(maxRefSum.min(Kilowatts(0d))) + + (refAgg, minSum, maxSum) + } +} diff --git a/src/main/scala/edu/ie3/simona/model/em/EmAggregateSimpleSum.scala b/src/main/scala/edu/ie3/simona/model/em/EmAggregateSimpleSum.scala new file mode 100644 index 0000000000..5a0cc02c59 --- /dev/null +++ b/src/main/scala/edu/ie3/simona/model/em/EmAggregateSimpleSum.scala @@ -0,0 +1,37 @@ +/* + * © 2022. TU Dortmund University, + * Institute of Energy Systems, Energy Efficiency and Energy Economics, + * Research group Distribution grid planning and operation + */ + +package edu.ie3.simona.model.em + +import edu.ie3.datamodel.models.input.AssetInput +import edu.ie3.simona.ontology.messages.flex.MinMaxFlexibilityMessage.ProvideMinMaxFlexOptions +import squants.energy.Kilowatts + +/** Aggregates reference, minimum and maximum power by just simply summing up + * each value + */ +object EmAggregateSimpleSum extends EmAggregateFlex { + + override def aggregateFlexOptions( + flexOptions: Iterable[ + (_ <: AssetInput, ProvideMinMaxFlexOptions) + ] + ): (squants.Power, squants.Power, squants.Power) = { + flexOptions.foldLeft( + (Kilowatts(0d), Kilowatts(0d), Kilowatts(0d)) + ) { + case ( + (sumRef, sumMin, sumMax), + (_, ProvideMinMaxFlexOptions(_, addRef, addMin, addMax)), + ) => + ( + sumRef + addRef, + sumMin + addMin, + sumMax + addMax, + ) + } + } +} diff --git a/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala b/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala new file mode 100644 index 0000000000..ec6acc26a3 --- /dev/null +++ b/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala @@ -0,0 +1,133 @@ +/* + * © 2023. TU Dortmund University, + * Institute of Energy Systems, Energy Efficiency and Energy Economics, + * Research group Distribution grid planning and operation + */ + +package edu.ie3.simona.model.em + +import edu.ie3.datamodel.models.input.AssetInput +import edu.ie3.simona.config.SimonaConfig.EmRuntimeConfig +import edu.ie3.simona.exceptions.CriticalFailureException +import edu.ie3.simona.ontology.messages.flex.FlexibilityMessage.ProvideFlexOptions +import edu.ie3.simona.ontology.messages.flex.MinMaxFlexibilityMessage.ProvideMinMaxFlexOptions +import squants.Power + +import java.util.UUID + +/** Translating input data to a format that can be used by aggregation + * strategies, em strats etc. Furthermore, sanity checks on calculated data is + * performed. + */ +final case class EmModelShell( + uuid: UUID, + id: String, + modelStrategy: EmModelStrat, + aggregateFlex: EmAggregateFlex, + modelToParticipantInput: Map[UUID, AssetInput] = Map.empty, +) { + + def addParticipant(modelUuid: UUID, inputModel: AssetInput): EmModelShell = + copy( + modelToParticipantInput = + modelToParticipantInput.updated(modelUuid, inputModel) + ) + + def aggregateFlexOptions( + allFlexOptions: Iterable[ + (UUID, ProvideFlexOptions) + ] + ): (Power, Power, Power) = { + val updatedAllFlexOptions = allFlexOptions.map { + case (modelUuid, flexOptions) => + val assetInput = modelToParticipantInput.getOrElse( + modelUuid, + throw new CriticalFailureException( + s"Asset input for model with UUID $modelUuid was not found." + ), + ) + + val minMaxFlexOptions = flexOptions match { + case flex: ProvideMinMaxFlexOptions => flex + case unsupported => + throw new CriticalFailureException( + s"Received unsupported flex options $unsupported." + ) + } + + val updatedFlexOptions = + modelStrategy.adaptFlexOptions(assetInput, minMaxFlexOptions) + + assetInput -> updatedFlexOptions + } + + aggregateFlex.aggregateFlexOptions(updatedAllFlexOptions) + } + + def determineFlexControl( + allFlexOptions: Iterable[(UUID, ProvideFlexOptions)], + target: Power, + ): Iterable[(UUID, Power)] = { + + val minMaxFlexOptions = allFlexOptions.toMap.view.mapValues { + case flex: ProvideMinMaxFlexOptions => flex + case unsupported => + throw new CriticalFailureException( + s"Received unsupported flex options $unsupported." + ) + }.toMap + + val uuidToFlexOptions = minMaxFlexOptions.map { + case (modelUuid, flexOptions) => + val assetInput = modelToParticipantInput.getOrElse( + modelUuid, + throw new CriticalFailureException( + s"Asset input for model with UUID $modelUuid was not found." + ), + ) + assetInput -> flexOptions + } + + val setPoints = + modelStrategy.determineFlexControl(uuidToFlexOptions, target) + + setPoints.map { case (model, power) => + val flexOptions = + minMaxFlexOptions.getOrElse(model, throw new RuntimeException()) + if (!flexOptions.fits(power)) + throw new CriticalFailureException( + s"Calculated set point $power does not fit flex option" + ) + + model -> power + } + + // TODO sanity checks after strat calculation + // checkSetPower(flexOptions, power) + + } + +} + +object EmModelShell { + def apply( + uuid: UUID, + id: String, + modelStrat: String, + modelConfig: EmRuntimeConfig, + ): EmModelShell = { + + val modelStrategy = modelStrat match { + case "PROPORTIONAL" => ProportionalFlexStrat + case "PRIORITIZED" => PrioritizedFlexStrat(modelConfig.pvFlex) + } + + val aggregateFlex = modelConfig.aggregateFlex match { + case "SELF_OPT_EXCL_PV" => EmAggregateSelfOpt(false) + case "SELF_OPT" => EmAggregateSelfOpt(true) + case "SIMPLE_SUM" => EmAggregateSimpleSum + } + + EmModelShell(uuid, id, modelStrategy, aggregateFlex) + } +} diff --git a/src/main/scala/edu/ie3/simona/model/em/EmModelStrat.scala b/src/main/scala/edu/ie3/simona/model/em/EmModelStrat.scala new file mode 100644 index 0000000000..4a4919f0c4 --- /dev/null +++ b/src/main/scala/edu/ie3/simona/model/em/EmModelStrat.scala @@ -0,0 +1,60 @@ +/* + * © 2022. TU Dortmund University, + * Institute of Energy Systems, Energy Efficiency and Energy Economics, + * Research group Distribution grid planning and operation + */ + +package edu.ie3.simona.model.em + +import edu.ie3.datamodel.models.input.AssetInput +import edu.ie3.simona.ontology.messages.flex.MinMaxFlexibilityMessage.ProvideMinMaxFlexOptions +import squants.Power +import squants.energy.Kilowatts + +import java.util.UUID + +/** Trait that can be enhanced by multiple strategies to disaggregate + * flexibility control, i.e. given a target power, determining flex control for + * connected agents + */ +trait EmModelStrat { + + /** Determine the target power (set points) of connected agents that provided + * flex options before. Connected agents that have no result assigned in + * return data are + * + * @param flexOptions + * The flex options per connected agent + * @param target + * The target power to aim for when utilizing flexibility + * @return + * Power set points for connected agents, if applicable + */ + def determineFlexControl( + flexOptions: Iterable[ + (_ <: AssetInput, ProvideMinMaxFlexOptions) + ], + target: Power, + ): Iterable[(UUID, Power)] + + /** Depending on the model strategy used, not all flex options provided by + * connected agents might be usable by the parent + * [[edu.ie3.simona.agent.em.EmAgent]]. This method adapts the given flex + * options based on the given [[AssetInput]]. + * + * @param assetInput + * The [[AssetInput]] of the connected agent providing the flex options + * @param flexOptions + * The flex options + * @return + * adapted flex options + */ + def adaptFlexOptions( + assetInput: AssetInput, + flexOptions: ProvideMinMaxFlexOptions, + ): ProvideMinMaxFlexOptions +} + +object EmModelStrat { + val tolerance: Power = Kilowatts(1e-6d) +} diff --git a/src/main/scala/edu/ie3/simona/model/em/FlexTimeSeries.scala b/src/main/scala/edu/ie3/simona/model/em/FlexTimeSeries.scala new file mode 100644 index 0000000000..0ecfa80eaf --- /dev/null +++ b/src/main/scala/edu/ie3/simona/model/em/FlexTimeSeries.scala @@ -0,0 +1,107 @@ +/* + * © 2023. TU Dortmund University, + * Institute of Energy Systems, Energy Efficiency and Energy Economics, + * Research group Distribution grid planning and operation + */ + +package edu.ie3.simona.model.em + +import edu.ie3.datamodel.models.timeseries.individual.IndividualTimeSeries +import edu.ie3.datamodel.models.value.PValue +import edu.ie3.simona.config.SimonaConfig +import edu.ie3.simona.util.TickUtil.TickLong +import edu.ie3.util.scala.io.FlexSignalFromExcel +import squants.Power +import squants.energy.Kilowatts + +import java.time.ZonedDateTime +import java.time.temporal.ChronoUnit +import scala.util.{Failure, Success} +import edu.ie3.util.quantities.PowerSystemUnits + +import scala.jdk.OptionConverters.RichOptional + +final case class FlexTimeSeries( + timeSeries: IndividualTimeSeries[PValue], + startDateTime: ZonedDateTime, + resolutionHours: Int, + minValue: Power, + maxValue: Power, + threshold: Double, +) { + def get(tick: Long): Power = { + // round current time to flexTimeSeries.resolutionHours hrs + val currentDateTime = tick.toDateTime(startDateTime) + val currentHour = currentDateTime.getHour + val roundedHour = + currentHour - currentHour % resolutionHours + val roundedDateTime = currentDateTime + .withHour(roundedHour) + .withMinute(0) + .withSecond(0) + .withNano(0) + + timeSeries + .getTimeBasedValue(roundedDateTime) + .toScala + .getOrElse( + throw new RuntimeException( + s"Could not retrieve value for $roundedDateTime" + ) + ) + .getValue + .getP + .toScala + .map(p => Kilowatts(p.to(PowerSystemUnits.KILOWATT).getValue.doubleValue)) + .getOrElse( + throw new RuntimeException( + s"No value set for $roundedDateTime" + ) + ) + } +} + +object FlexTimeSeries { + + def apply( + config: SimonaConfig.Simona.Runtime.RootEm + )(implicit startDateTime: ZonedDateTime): FlexTimeSeries = { + val timeSeriesType = + FlexSignalFromExcel.TimeSeriesType(config.timeSeriesType) + val timeSeries = FlexSignalFromExcel + .flexSignals(config.filePath, config.nodeId, timeSeriesType) match { + case Success(timeSeries) => timeSeries + case Failure(exception) => throw exception + } + + val resolutionHours = + if (timeSeries.getEntries.size() < 2) + throw new RuntimeException( + s"Less than two entries for flex time series ${config.nodeId}" + ) + else { + val valueIt = timeSeries.getEntries.iterator() + val entry1 = valueIt.next().getTime + val entry2 = valueIt.next().getTime + + ChronoUnit.HOURS.between(entry1, entry2).intValue + } + + // in case of resLoad we use totalResload (considering Simona participants) for min max setting + val (minValue, maxValue) = + FlexSignalFromExcel.getCorrespondingMinMaxValues( + timeSeriesType, + timeSeries, + config, + ) + + FlexTimeSeries( + timeSeries, + startDateTime, + resolutionHours, + minValue, + maxValue, + config.threshold, + ) + } +} diff --git a/src/main/scala/edu/ie3/simona/model/em/PrioritizedFlexStrat.scala b/src/main/scala/edu/ie3/simona/model/em/PrioritizedFlexStrat.scala new file mode 100644 index 0000000000..f8c93ab7e0 --- /dev/null +++ b/src/main/scala/edu/ie3/simona/model/em/PrioritizedFlexStrat.scala @@ -0,0 +1,204 @@ +/* + * © 2022. TU Dortmund University, + * Institute of Energy Systems, Energy Efficiency and Energy Economics, + * Research group Distribution grid planning and operation + */ + +package edu.ie3.simona.model.em + +import edu.ie3.datamodel.models.input.AssetInput +import edu.ie3.datamodel.models.input.system.{ + EvcsInput, + HpInput, + PvInput, + StorageInput, +} +import EmModelStrat.tolerance +import edu.ie3.simona.ontology.messages.flex.MinMaxFlexibilityMessage.ProvideMinMaxFlexOptions +import squants.Power +import squants.energy.Kilowatts + +import java.util.UUID + +/** Determines flex control for connected agents by adhering to a priority + * hierarchy, with some devices not controlled at all. + * + * @param pvFlex + * Whether PV feed-in can be curtailed or not + */ +final case class PrioritizedFlexStrat(pvFlex: Boolean) extends EmModelStrat { + + /** Only heat pumps, battery storages, charging stations and PVs (if enabled) + * are controlled by this strategy + */ + private val controllableAssets: Seq[Class[_ <: AssetInput]] = + Seq(classOf[HpInput], classOf[StorageInput], classOf[EvcsInput]) ++ Option + .when(pvFlex)(Seq(classOf[PvInput])) + .getOrElse(Seq.empty) + + /** Determine the power of controllable devices by using flexibility according + * to a prioritized list of device types. This means that e.g. flexibility of + * storages is used before flexibility of heat pumps is used. Priority lists + * can differ depending on whether positive or negative flexibility needs to + * be used. + * + * @param flexOptions + * The flex options per connected system participant + * @param target + * The target power to aim for when utilizing flexibility + * @return + * Power set points for devices, if applicable + */ + override def determineFlexControl( + flexOptions: Iterable[ + (_ <: AssetInput, ProvideMinMaxFlexOptions) + ], + target: Power, + ): Seq[(UUID, Power)] = { + + val totalRefPower = + flexOptions + .map { case (_, ProvideMinMaxFlexOptions(_, refPower, _, _)) => + refPower + } + .reduceOption { (power1, power2) => + power1 + power2 + } + .getOrElse(throw new RuntimeException("No flexibilities provided")) + + val targetDelta = totalRefPower - target + + val evcsOpt = flexOptions.collectFirst { case flex @ (_: EvcsInput, _) => + flex + } + val storageOpt = flexOptions.collectFirst { + case flex @ (_: StorageInput, _) => flex + } + val heatPumpOpt = flexOptions.collectFirst { case flex @ (_: HpInput, _) => + flex + } + val pvOpt = flexOptions + .collectFirst { case flex @ (_: PvInput, _) => + flex + } + .filter(_ => pvFlex) // only if enabled + + if (Kilowatts(0d).~=(targetDelta)(tolerance)) { + Seq.empty + } else if (targetDelta < Kilowatts(0d)) { + // suggested power too low, try to store difference/increase load + + val orderedParticipants = + Seq(evcsOpt, storageOpt, heatPumpOpt, pvOpt).flatten + + orderedParticipants.foldLeft( + (Seq.empty[(UUID, Power)], Option(targetDelta)) + ) { + case ( + (issueCtrlMsgs, Some(remainingExcessPower)), + (inputModel, flexOption: ProvideMinMaxFlexOptions), + ) => + // potential for decreasing feed-in/increasing load (negative) + val flexPotential = + flexOption.ref - flexOption.max + + if (Kilowatts(0d).~=(remainingExcessPower)(tolerance)) { + // we're already there (besides rounding error) + (issueCtrlMsgs, None) + } else if (Kilowatts(0d).~=(flexPotential)(tolerance)) { + // device does not offer usable flex potential here + (issueCtrlMsgs, Some(remainingExcessPower)) + } else if (remainingExcessPower < flexPotential) { + // we cannot cover the excess feed-in with just this flexibility, + // thus use all of the available flexibility and continue + ( + issueCtrlMsgs :+ (inputModel.getUuid, flexOption.max), + Some(remainingExcessPower - flexPotential), + ) + } else { + + // this flexibility covers more than we need to reach zero excess, + // thus we only use as much as we need + val powerCtrl = flexOption.max.min( + flexOption.ref - remainingExcessPower + ) + + ( + issueCtrlMsgs :+ (inputModel.getUuid, powerCtrl), + None, + ) + } + case ((issueCtrlMsgs, None), (_, _)) => + // if no excess feed-in remains, do nothing + (issueCtrlMsgs, None) + } match { + case (issueCtrlMsgs, _) => issueCtrlMsgs + } + + } else { + // excess load, try to cover it with stored energy/by reducing load + + val orderedParticipants = Seq(storageOpt, evcsOpt, heatPumpOpt).flatten + + orderedParticipants.foldLeft( + (Seq.empty[(UUID, Power)], Option(targetDelta)) + ) { + case ( + (issueCtrlMsgs, Some(remainingExcessPower)), + (inputModel, flexOption: ProvideMinMaxFlexOptions), + ) => + // potential for decreasing load/increasing feed-in + val flexPotential = + flexOption.ref - flexOption.min + + if (Kilowatts(0d).~=(remainingExcessPower)(tolerance)) { + // we're already there (besides rounding error) + (issueCtrlMsgs, None) + } else if (Kilowatts(0d).~=(flexPotential)(tolerance)) { + // device does not offer usable flex potential here + (issueCtrlMsgs, Some(remainingExcessPower)) + } else if (remainingExcessPower > flexPotential) { + // we cannot cover the excess load with just this flexibility, + // thus use all of the available flexibility and continue + ( + issueCtrlMsgs :+ (inputModel.getUuid, flexOption.min), + Some(remainingExcessPower - flexPotential), + ) + } else { + + // this flexibility covers more than we need to reach zero excess, + // thus we only use as much as we need + val powerCtrl = flexOption.min.max( + flexOption.ref - remainingExcessPower + ) + + ( + issueCtrlMsgs :+ (inputModel.getUuid, powerCtrl), + None, + ) + } + case ((issueCtrlMsgs, None), (_, _)) => + // if no excess load remains, do nothing + (issueCtrlMsgs, None) + } match { + case (issueCtrlMsgs, _) => issueCtrlMsgs + } + } + + } + + override def adaptFlexOptions( + assetInput: AssetInput, + flexOptions: ProvideMinMaxFlexOptions, + ): ProvideMinMaxFlexOptions = { + if (controllableAssets.contains(assetInput.getClass)) + flexOptions + else { + // device is not controllable by this EmAgent + flexOptions.copy( + min = flexOptions.ref, + max = flexOptions.ref, + ) + } + } +} diff --git a/src/main/scala/edu/ie3/simona/model/em/ProportionalFlexStrat.scala b/src/main/scala/edu/ie3/simona/model/em/ProportionalFlexStrat.scala new file mode 100644 index 0000000000..f0034ad7c8 --- /dev/null +++ b/src/main/scala/edu/ie3/simona/model/em/ProportionalFlexStrat.scala @@ -0,0 +1,138 @@ +/* + * © 2022. TU Dortmund University, + * Institute of Energy Systems, Energy Efficiency and Energy Economics, + * Research group Distribution grid planning and operation + */ + +package edu.ie3.simona.model.em + +import edu.ie3.datamodel.models.input.AssetInput +import EmModelStrat.tolerance +import edu.ie3.simona.ontology.messages.flex.MinMaxFlexibilityMessage.ProvideMinMaxFlexOptions +import squants.Power +import squants.energy.Kilowatts + +import java.util.UUID + +/** Proportionally distributes flex control among connected agents, i.e. all + * agents contribute the same share of their offered flex options + */ +object ProportionalFlexStrat extends EmModelStrat { + + /** Determine the power of controllable devices by proportionally distributing + * flexibility usage to connected devices. This means that all devices are + * set to use the same share of their respective flexibility to reach target + * power. + * + * @param modelFlexOptions + * The flex options per connected agent + * @param target + * The target power to aim for when utilizing flexibility + * @return + * Power set points for devices, if applicable + */ + override def determineFlexControl( + modelFlexOptions: Iterable[ + (_ <: AssetInput, ProvideMinMaxFlexOptions) + ], + target: Power, + ): Iterable[(UUID, Power)] = { + + // Input models are not needed here + val flexOptions = modelFlexOptions + .map { case (_, flexOptions) => + flexOptions + } + + // sum up reference, minimum and maximum power of all connected devices + val (totalRef, totalMin, totalMax) = flexOptions + .foldLeft( + (Kilowatts(0d), Kilowatts(0d), Kilowatts(0d)) + ) { + case ( + (sumRef, sumMin, sumMax), + ProvideMinMaxFlexOptions(_, addRef, addMin, addMax), + ) => + ( + sumRef + addRef, + sumMin + addMin, + sumMax + addMax, + ) + } + + if (target.~=(totalRef)(tolerance)) { + Seq.empty + } else if (target < totalRef) { + val reducedOptions = flexOptions.map { + case ProvideMinMaxFlexOptions(uuid, refPower, minPower, _) => + (uuid, refPower, minPower) + } + + distributeFlexibility(target, totalRef, totalMin, reducedOptions) + } else { + val reducedOptions = flexOptions.map { + case ProvideMinMaxFlexOptions(uuid, refPower, _, maxPower) => + (uuid, refPower, maxPower) + } + + distributeFlexibility(target, totalRef, totalMax, reducedOptions) + } + } + + /** Proportionally distributes flexibility to given devices + * + * @param target + * The target power to aim for when utilizing flexibility + * @param totalRef + * The total reference power of all connected devices + * @param totalLimit + * The total limit of power (either positive or negative) of all connected + * devices + * @param options + * The flexibility options (model UUID, reference power and limit power) + * @return + * Power set points for devices, if applicable + */ + private def distributeFlexibility( + target: Power, + totalRef: Power, + totalLimit: Power, + options: Iterable[(UUID, Power, Power)], + ): Iterable[(UUID, Power)] = { + // filter out options with ref == limit because they're useless here + val filteredOptions = options.filterNot { case (_, refPower, limitPower) => + refPower.~=(limitPower)(tolerance) + } + + if ( + (target < totalRef && target <= totalLimit) || + (target > totalRef && target >= totalLimit) + ) { + // target is beyond limit, thus use limit powers for all applicable devices + filteredOptions.map { case (uuid, _, limitPower) => + uuid -> limitPower + } + } else { + // calculate share of flexibility that each device should carry + val normalizedLimit = totalLimit - totalRef + val normalizedTarget = target - totalRef + + val flexShare = normalizedTarget / normalizedLimit + + filteredOptions.map { case (uuid, refPower, limitPower) => + val diffLimitRef = limitPower - refPower + + // add the required share of flexibility to the reference power + val setPower = refPower + (diffLimitRef * flexShare) + + uuid -> setPower + } + } + } + + override def adaptFlexOptions( + assetInput: AssetInput, + flexOptions: ProvideMinMaxFlexOptions, + ): ProvideMinMaxFlexOptions = + flexOptions +} diff --git a/src/main/scala/edu/ie3/simona/ontology/messages/flex/FlexibilityMessage.scala b/src/main/scala/edu/ie3/simona/ontology/messages/flex/FlexibilityMessage.scala index f6994db75c..a4574054ad 100644 --- a/src/main/scala/edu/ie3/simona/ontology/messages/flex/FlexibilityMessage.scala +++ b/src/main/scala/edu/ie3/simona/ontology/messages/flex/FlexibilityMessage.scala @@ -7,6 +7,7 @@ package edu.ie3.simona.ontology.messages.flex import edu.ie3.datamodel.models.input.AssetInput +import edu.ie3.simona.agent.em.EmAgent.EmMessage import edu.ie3.simona.agent.participant.data.Data.PrimaryData.ApparentPower import edu.ie3.simona.scheduler.ScheduleLock.ScheduleKey import org.apache.pekko.actor.typed.ActorRef @@ -32,7 +33,7 @@ object FlexibilityMessage { /** Trait that is extended by all messages that are supposed to be received by * [[edu.ie3.simona.agent.em.EmAgent]]s. */ - sealed trait FlexResponse { + sealed trait FlexResponse extends EmMessage { val modelUuid: UUID } diff --git a/src/main/scala/edu/ie3/simona/sim/setup/SetupHelper.scala b/src/main/scala/edu/ie3/simona/sim/setup/SetupHelper.scala index 0df5aa30e1..3d49ca10eb 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SetupHelper.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SetupHelper.scala @@ -259,9 +259,11 @@ object SetupHelper { ).simulationResultEntitiesToConsider ++ (OutputConfigUtil( outputConfig.participant - ).simulationResultIdentifiersToConsider ++ OutputConfigUtil( + ).simulationResultIdentifiersToConsider(thermal = + false + ) ++ OutputConfigUtil( outputConfig.thermal - ).simulationResultIdentifiersToConsider) + ).simulationResultIdentifiersToConsider(thermal = true)) .map(notifierId => EntityMapperUtil.getResultEntityClass(notifierId)) ++ (if (outputConfig.flex) Seq(classOf[FlexOptionsResult]) else Seq.empty) } diff --git a/src/main/scala/edu/ie3/simona/util/ConfigUtil.scala b/src/main/scala/edu/ie3/simona/util/ConfigUtil.scala index 10acb7ddaf..fa02c1137f 100644 --- a/src/main/scala/edu/ie3/simona/util/ConfigUtil.scala +++ b/src/main/scala/edu/ie3/simona/util/ConfigUtil.scala @@ -89,7 +89,8 @@ object ConfigUtil { subConfig.pv.individualConfigs, subConfig.evcs.individualConfigs, subConfig.wec.individualConfigs, - ).reduceOption(_ ++ _).getOrElse(Seq.empty) + subConfig.em.individualConfigs, + ).flatten ), Seq( subConfig.load.defaultConfig, @@ -98,6 +99,7 @@ object ConfigUtil { subConfig.evcs.defaultConfig, subConfig.wec.defaultConfig, subConfig.hp.defaultConfig, + subConfig.em.defaultConfig, ).map { conf => conf.getClass -> conf }.toMap, ) } @@ -141,10 +143,16 @@ object ConfigUtil { * @return * A set of applicable notifiers */ - def simulationResultIdentifiersToConsider: Set[NotifierIdentifier.Value] = + def simulationResultIdentifiersToConsider( + thermal: Boolean + ): Set[NotifierIdentifier.Value] = { if (defaultConfig.simulationResultInfo) { + val notifiers = + if (thermal) NotifierIdentifier.getThermalIdentifiers + else + NotifierIdentifier.values -- NotifierIdentifier.getThermalIdentifiers /* Generally inform about all simulation results, but not on those, that are explicitly marked */ - NotifierIdentifier.values -- configs.flatMap { + notifiers -- configs.flatMap { case ( notifierId, NotifierConfig(resultInfo, _, _), @@ -163,9 +171,12 @@ object ConfigUtil { case _ => None }.toSet } + } - def simulationResultEntitiesToConsider: Set[Class[_ <: ResultEntity]] = - simulationResultIdentifiersToConsider.map(notifierId => + def simulationResultEntitiesToConsider( + thermal: Boolean + ): Set[Class[_ <: ResultEntity]] = + simulationResultIdentifiersToConsider(thermal).map(notifierId => EntityMapperUtil.getResultEntityClass(notifierId) ) } @@ -272,6 +283,7 @@ object ConfigUtil { object NotifierIdentifier extends ParsableEnumeration { val BioMassPlant: Value = Value("bm") val ChpPlant: Value = Value("chp") + val Em: Value = Value("em") val Ev: Value = Value("ev") val Evcs: Value = Value("evcs") val FixedFeedIn: Value = Value("fixedfeedin") @@ -281,6 +293,11 @@ object ConfigUtil { val Wec: Value = Value("wec") val Hp: Value = Value("hp") val House: Value = Value("house") + + /** All thermal identifiers */ + def getThermalIdentifiers: Set[Value] = Set( + NotifierIdentifier.House + ) } object CsvConfigUtil { diff --git a/src/main/scala/edu/ie3/simona/util/EntityMapperUtil.scala b/src/main/scala/edu/ie3/simona/util/EntityMapperUtil.scala index e52316c88a..ed9192dcb1 100644 --- a/src/main/scala/edu/ie3/simona/util/EntityMapperUtil.scala +++ b/src/main/scala/edu/ie3/simona/util/EntityMapperUtil.scala @@ -12,7 +12,7 @@ import edu.ie3.datamodel.models.result.thermal.ThermalHouseResult import edu.ie3.simona.util.ConfigUtil.NotifierIdentifier import edu.ie3.simona.util.ConfigUtil.NotifierIdentifier._ -case object EntityMapperUtil { +object EntityMapperUtil { private val entityMapping : Map[NotifierIdentifier.Value, Class[_ <: ResultEntity]] = Map( PvPlant -> classOf[PvResult], @@ -24,6 +24,7 @@ case object EntityMapperUtil { Evcs -> classOf[EvcsResult], ChpPlant -> classOf[ChpResult], Storage -> classOf[StorageResult], + Em -> classOf[EmResult], Hp -> classOf[HpResult], House -> classOf[ThermalHouseResult], ) diff --git a/src/main/scala/edu/ie3/util/scala/io/FlexSignalFromExcel.scala b/src/main/scala/edu/ie3/util/scala/io/FlexSignalFromExcel.scala new file mode 100644 index 0000000000..6852897f35 --- /dev/null +++ b/src/main/scala/edu/ie3/util/scala/io/FlexSignalFromExcel.scala @@ -0,0 +1,176 @@ +/* + * © 2022. TU Dortmund University, + * Institute of Energy Systems, Energy Efficiency and Energy Economics, + * Research group Distribution grid planning and operation + */ + +package edu.ie3.util.scala.io + +import edu.ie3.datamodel.models.timeseries.individual.{ + IndividualTimeSeries, + TimeBasedValue, +} +import edu.ie3.datamodel.models.value.PValue +import edu.ie3.simona.config.SimonaConfig.Simona.Runtime.RootEm +import edu.ie3.util.quantities.PowerSystemUnits +import org.apache.poi.ss.usermodel.CellType +import org.apache.poi.xssf.usermodel.XSSFWorkbook +import squants.energy.Kilowatts +import tech.units.indriya.quantity.Quantities + +import java.io.{File, FileInputStream} +import java.time.ZoneId +import java.util.UUID +import javax.measure +import javax.measure.quantity.Power +import scala.jdk.CollectionConverters.{ + IterableHasAsScala, + IteratorHasAsScala, + SetHasAsJava, +} +import scala.jdk.OptionConverters.RichOptional +import scala.util.{Failure, Success, Try, Using} + +object FlexSignalFromExcel { + + /** Gets flexibility signals from an Excel file with defined format. + * @param filePath + * Path to the excel file + * @param nodeId + * The node id, i.e. name of the sheet + * @param timeSeriesType + * The time series type + * @param unit + * Physical unit of input data + * @param zoneId + * Time zone of the incorporated date times + * @return + * A trial to the resulting time series + */ + def flexSignals( + filePath: String, + nodeId: String, + timeSeriesType: TimeSeriesType.Value = TimeSeriesType.ResidualLoad, + // ATTENTION: the unit configured here is not the output unit as `PValues` are transformed to kW by default + unit: measure.Unit[Power] = PowerSystemUnits.MEGAWATT, + zoneId: ZoneId = ZoneId.of("UTC"), + ): Try[IndividualTimeSeries[PValue]] = { + Using { + val file = new File(filePath) + val fileStream = new FileInputStream(file) + new XSSFWorkbook(fileStream) + } { workbook => + val sheet = workbook.getSheet(nodeId) + + val rows = sheet.rowIterator().asScala + /* Assess the table header */ + val header = + rows.next().cellIterator().asScala.toList.map(_.getStringCellValue) + val dateTimeIndex = header.indexOf("time") + val colToTimeseriesType = header.zipWithIndex + .filterNot(_._1 == "time") + .map { case (token, idx) => + idx -> TimeSeriesType(token) + } + .toMap + + /* Go through the file and parse the contents */ + val values = rows.toList + .filterNot(_.getCell(0).getCellType == CellType.BLANK) + .flatMap { row => + val dateTime = row + .getCell(dateTimeIndex) + .getLocalDateTimeCellValue + .atZone(zoneId) + + val tsTypeToValue = colToTimeseriesType.map { case (col, tsType) => + // negate the flex signal to get residual power + val raw = -row.getCell(col).getNumericCellValue + val value = new PValue( + Quantities.getQuantity(raw, unit) + ) + val timeBasedValue = + new TimeBasedValue[PValue](dateTime, value) + tsType -> timeBasedValue + } + tsTypeToValue + } + .groupBy(_._1) + .map { case (tsType, collection) => + tsType -> collection.map(_._2).toSet + } + + /* Finally process the single value lists into individual time series */ + val tsTypeToTs = values.map { case (tsType, valueSet) => + tsType -> new IndividualTimeSeries[PValue]( + UUID.randomUUID(), + valueSet.asJava, + ) + } + + tsTypeToTs(timeSeriesType) + } + } + + def getCorrespondingMinMaxValues( + timeSeriesType: TimeSeriesType.Value, + timeSeries: IndividualTimeSeries[PValue], + config: RootEm, + ): (squants.Power, squants.Power) = { + + // todo this is very use case dependant and has to be reworked + /* instead of using the residual load we take the total res load to determine min and max + values for threshold calculation as this also includes self oriented reference behavior of Simona*/ + val minMaxTs = if (timeSeriesType == TimeSeriesType.ResidualLoad) { + FlexSignalFromExcel + .flexSignals( + config.filePath, + config.nodeId, + TimeSeriesType.TotalResLoad, + ) match { + case Success(timeSeries) => timeSeries + case Failure(exception) => throw exception + } + } else timeSeries + val allValues = + minMaxTs.getEntries.asScala.flatMap(_.getValue.getP.toScala) + val maybeMinValue = allValues.minByOption( + _.to(PowerSystemUnits.KILOWATT).getValue.doubleValue + ) + val maybeMaxValue = allValues.maxByOption( + _.to(PowerSystemUnits.KILOWATT).getValue.doubleValue + ) + + val (minValue, maxValue) = maybeMinValue + .zip(maybeMaxValue) + .getOrElse( + throw new RuntimeException(s"Time series for $config is empty") + ) + ( + Kilowatts(minValue.getValue.doubleValue()), + Kilowatts(maxValue.getValue.doubleValue()), + ) + } + + object TimeSeriesType extends Enumeration { + val Generation, Load, OtherGeneration, OtherLoad, Import, ImportIntern, + ResidualLoad, TotalResLoad, SimonaGeneration, SimonaLoad = Value + + def apply(token: String): TimeSeriesType.Value = { + token match { + case "gen" => Generation + case "loads" => Load + case "otherGen" => OtherGeneration + case "otherLoads" => OtherLoad + case "importExport" => Import + case "importExport_intern" => ImportIntern + case "resLoad" => ResidualLoad + case "totalResLoad" => TotalResLoad + case "SIMONA_gen" => SimonaGeneration + case "SIMONA_load" => SimonaLoad + case _ => + throw new IllegalArgumentException(s"Don't know the token '$token'.") + } + } + } +} diff --git a/src/test/resources/edu/ie3/util/scala/io/flexSignal.xlsx b/src/test/resources/edu/ie3/util/scala/io/flexSignal.xlsx new file mode 100644 index 0000000000000000000000000000000000000000..1bb027a4433b5f934c66cb785db74e5f847d21ba GIT binary patch literal 47344 zcmeFa30%xw`}m(FAqhn&k}Zis(WbIS+?BF~HYB0FNogHQA`z8B8;T_DrM*!}5>piI zo7A+8Nk*w}>FtZJ@Nh?CgZmQ7=0<6)aVS!<`6sg;4LmEIv+a|42o*hLfLWsj#$Uh|M` zGI;)<|Na+S;I)s6S@womS83jTg3q{y9G-6EB8id&SaQrAmgP)1L*bKSpYfon_n@8b|laP!*)}`wGnvb+ulw@8zV^ z4IdkG%~QKB9ZNTfJ-zu3sZk(evB#@y+ZYSOCg-`ldN~31>g!(%^gk?3Yq}dh)z|X8 zWZBt^+!C=RKJqUbDC?EkZmvtKTxy)YZ*_X<^Lf0=1Ml7miVct-U6U$a^x|$_#I>x8 z5j*4QX<5`7!XGLJ=EYr~-m~(L*L6l86t$10%O$OueIwyOYWW|Bm+2%*Zk)nC$hPAY z*ES`-bL%}0^xYV`M3nW|x_(5$MY3yEJ&(*Z=IiaT)JtK8j6*B$ueiLzM6xc0NSr6% zq}+jU2KQT!B#Qia#*>+b~EN<0vqEHQ(u&q$~3=9ixydZISd#H1*@C}UXl^fku- zXHTB>uUbBLZjO6UnZi6+VxK3?CFdt7Nk8Ij+|_h0(^CP=P9}2-+ktWOHu3O=6+Ivr z{{YjH6U2e%AAWi6=W&9+*a?j@hkc+e-h7#!+O<#w;IbO=k;$Y+>^KJVK4D`f@D`gpzUj^ z895PeJM*1y2tKQsaFRTHa59e%e4{^=jcqa8^hpRE8OOIpeBgZf zAZNvM|69AvV?!UFp)P$AnywTgV!5QENF-3|!Oj-Z@|NL;qCV#YqC=T&lgeKoY@f;Q zq*>)$#?aY2T3dP7z`Mb_z;oLA;On;qk~b_ouO}qB@SWhs>yy{KrO)EtzvxB;zaY9j zCoi=4#p>>=PgUz@<~Z_GUnP?yOGsUomW$)eKs5ZAdk@U{*rhgwjm-jVp}4^lAgsO* zsL&(oX8s%II0Yv#FI^0KGj*@?zOa>-7K8^r7c)u~5Xp%iw2m ziUAo!o78#T^{Ur6IN6fjJ2cu>@7UY8D)PGQT&~8LINrF`i?4Ps4Oi08c;=;{qxqIl zJ?HZk3wxJ2`T84PTjpCnxRxF%Qp!I^S@dz1Y*qK5-iHfi@fw%1BSi;847_%nyqtMSG-0LZ+rZS7 zsR_rA;aC=-c?toZ}-@;bqgc9f!*qKiD;k3eoiVSU*>d&gzXgpXAO6g z>xC*_(~`FO&CMKRxbAz03-)?xve(plb zUAN1(Oav3mW30A+8JTGv;yb_4bl57%Pk&HTh16FtXA4LD<`B`|bt@$jzDRlSo%+1d zu9fB{)TwoWQ|d*Lfid@Roh@~R0b|w0w)zvN_bJ)Ei72a~uP(pnoz-~nOd)?yb|5vG zJt?*3tIl?TdH?NW`tiq!+qE;h zUN3w^J-5$_bN7jv;rx4^es~$5mfv{h6#rBC@;#c@AH_Kinq3X9|9tG@iktCmE4U1U z#TM}GsR#`uN^1gtXVoK@*2dz-Oq4hX)&t48THZFAT5v**D4G1umhaT1eGmqC1&b(n&u9c^irykmWFmzv~q)1?bCdVJA zJ`29R=V4qytlfNjTd8T}C%+SVm3fpPf+LTPl(QVibJ3@#J%&ZOWe%NLYiP+AV#WxU z+oNi5?fnc*uQ$zmMXdtUiF}8f-nN`k3%6j~ab#l!|3R^hQE~F~LsKTkJkpq6FPj@NLs@z0=ADg@5)>~g2xp2vH z%ZC;H2fbu&Di&WZZZ2MB(vvUmY;gSW^9`DpZX8yR@lbe@_KNe5xJff^i|_2-+0O21 zY1ZSq_uz|`%g2Te%$vXO!MprJ>s34^FFU{LsKVafolmW22vBWSsrMGk%$Yq`uzsc2 zt|C*Xozj)(&$yI*z3?o$LT^r1f2_+4CARgd?Atu(b2G$l@2U%Vbo^;dYfrI@!1uyVA`h7#z{$fA^ZBP*Xlo<<+aA} zblA%JBR6J$TQ6)sea`Zb0JHWfgSr97ju}44zLU7uHKjm*s@lg^;$r8iBQg4)RMy3B zJnkm%9H>3DO)_^8b8a}>^VU^I6nJ%sRYC^&o_gnFpU9fXU0$ChmU><%mazA$-u;|2 zMqefFGcUMB(w0WdpSp5k%kbXDc->o(g9}Y6MxWB>c9?ufKH0tK=A_ML(@MAQuPM7) zJZPKA@xI{P_VcFH!W$bGzX?7^K0oPHUs`6I%ck|aJg#;LYx$18ui4H|4|mR+t{3)Y zof64ZQ*c3BrKCBf<~&tZSkj8G{(Q&PV*k%}fnw>e(gx(u+nb%WGx3||J=_xawP^8$ zU}H5WiPzgcoc#3BOmJn7R zOhCSUNXI|%Th5b{&X%jybOQwz@5x$uZ&Y}OgXLsy;^ln73D=z!xwqt6!8b`@o5G95 z{@Ih32B!~NS)MaBM5ci8?GM-j&fIw~&Hp9GT;=$;aqe{-{T8}A z)a5i|(*}|ht#NLAjoWs|lcxm3uUa0ywB!7#w(yAhk&eEvGkI@>eBrtibJgth(Mq~R z&bsq`^Gb_uc$6L|6ziL(yRZ24!r?VIM zvx4k>LiYTf+kRzL19r^*VXJsK^&z!m6_nN2zh3p*dZ19yj5wifymjk7dhBFl%RDlL zZ6RtKxF!?M8W>m+#F2kC!W0qUQ+cam!5`U=$YC^n(KRE10%U7v}w2M=<5$* zZW}FjIxMb8B%R)}UOFhI^6p?_)fCe78eig`F~sZW1CXPWhAV_>gb{}eNz4&a=1?1Pq_>bbSjg-pjy5VXJGeWzncbeuky2)V-Y{u2 z@dLA)SlL%f8fhajNwm>XPtwSU;%Hm#XlI`aGbfnYlf-Ce-YcarDS~tolU`^>9Bfcz z4u>g@c35K=Wx)CGKh?J>#o!RF$S{`5QWN$Yr9o)inN;Pci4I7dl9_6tkrY4yv1_h;( z(wQel2Pj`MZJja&`3C8(w55D+z^5SJOA4prRCAIx!Awf{J2lU;fyS~z z&$8aeGI7tck;by<>%9YtgSCp!xD{PbI?bfjgMZDA6Bk$My$K6`UF_LhnddtCf!;1S zk=P*d;hto31-5~=qMd*p|^7L8Q%q&nK|2)=d2 z4CpAO#+PZYy|9oklc-$=By=XnDC+fhS=3+5>@04`9&o6q)gp&%7 zNb|%*z~5QX1hrz%V6}uD(M~B?_oma|8t)F%3|J+MP{jmC zw-P(Vf=9P1);=NL>(r#iWq)=aa$^UTl69jeU+JZBb|ewYXrCaQg*`~zs1exsbKN}f=CbWeW8i*4mp~ zU5vFB4onNf`c{**5l32UvQbS_CawUFMvB=%$=cw?Yvb1lQ zop8`zcb6tZ-Tur4t=oOI%%}!ULB6<3uY^*Xi@mN--=O218s8@wPJ&hN$(BLNlfEkR zLX{5NCj|;K>lV+{J~-9Ss(Md{?HwIQ!3zrF2@$jqG9}l#*RfN1qcL&BU20+aitJ+q|L#{bYmH+aD}X3vE+q zbx=w)a_@jGO8M8=sC?A3?QU&v#YIgDbU^{sFb&Pl}?{}XlYSTg=MNAQ~P1-=3G0dXkg(QdzFt^3+m=v zd$dJ%DQ_2S@wW89K!sPDpUpLLDR0jz9Yrx9xOcOwBl=iPDQ`a<8b#uP-oa)}11FU% zJ)r1rB3DQB`a=7GLiSd@gTsDSBgsIO$z{BgRwFIeo;Wz`9-{M_%PQ(8;Lt8}ZH`%A z4~KSHI24J9Lkp62fiW2phZ~sQSo})}0kvpzw}EZtw)N|519hQ|JuE;)K_<&xB0uxkAqh z2`8~=QNceD2L``yLj*mON5A)Ttm5eD=dq)%umfQU20wVhMzDCNU^~^tuGyv9@esFT zOd5CFBW`Pkh5TrGPSO<0jS|8cSxZORijOzsdr4z(!wQGBl3_LlyNM3{`w2AO?|2KSv*K z>oK*vcw8K>*A~d_LLCpM5^iQNm9S_4lh4t74e)HMgb#?E+Prq#hupJ~=e3Qus2S>TY`y(IgqY;E?F6d`B;(@A(KKB$feqIPa2Nl3M-(Vdqrh1ms zuPP*qz-*zfYi8W+;*Uf|lS54rhZLOfAWTj1S{N_r_rEH^{3O+rrNRoV-8j0WhAebftlpO_qdpH<=g56CJNbG1nj;AbfG)J;4$U%Dq ztKdS$*cdGZG8%jU`crPO4qXJGKYw3>cEn{OcEr~Tte*+$V2Zk>kc=;Nf29QTAg(Uh z)L@GLvIO!%as!3K1`EP2x=R`w8i|I(!*x*#ipY{?d+wAY3>I7;eGn`kkw$O^QJY=h z$_n==CN}6Wrd4y!J_K-IXvWEis`wvh=ZFor+;697F>!;hnWZM=T{||gSdOL z#um5Vmtf4Y6o)a3r{bTNV9ZW&>_paD7@rfCKpJlyLQ|RF>Mx9qG&8WW!4Ztpkv=+R zsv4pKZlL!=W5&dFg1zcLX@gsFyhrF`)F{x$TVa+!lRmy;KrMsOEn&K3EPNmMh`RuU z66zvk8->~lrU;g^j-+OSJtc01pSFaHPc-Et2U!1=H9gx7fz0YC7*FAG?1aVYy!L@z zf?Lr%VXuHb2J&c-T2IzNEMo`{)`cT$6722wC1?j+I%0jKpI??>#<;F3|Ep z#|XyE1{SDMIyNFqQ@F9F@S;{n_bS*Z;r0TW8hOD&)Pp%=EAEPtHm;s5uqP`ZiWMPF zMBcD)O^~=kyoclt=pD@dU{?os$ZK>bD3l2n@5$Pas1#4(bnMi? z0%eC_IugPg?K6uw#DQR#0d$*P{1BtSdH?$ooLwyGaC?jP^Q#if0DOaWP{&jx;(uR) zHoz4}A`cnDFH0~6T@@}OTQ2y9{TE8mv%n@L6!=Q`OkP~&1x+8RL9Ikn)WaWR{AY`1QS{-yB z9IgG%7%)wto?{8=O@TpGo7Z1=Q{kd`m;L87_|_|5yPpTl1o_%#mn6jJX9}eT>mm`y zl+(eJaP5Oy0#)#>hiOli77|L(#Xl=S7qcY6G=s|gRSCM7-?8%uj7?-Lzb`=-V@e{2 zD2TehE`2{3m|W0d{jvnpObB-kE6vn^MB)@Y6;*X487v*Fk=il`aFH8nikMa53^Bt^ z)$6omur338M@DRpzH*?3gzEw92quj5j~r%W;WD-ff!&W1zz#Babld*&W?L5ws zFcui-7WL;Xo~zj(+}3|BCvetWvRp=9y=v3#AR9Z+oWS=lD1kK%=E8&ng=qbgp7dgS z0qW^D^0c0%ZgTe<>}p!%XWML=5{y272i8Z^LW^QD9xW#CH*FA|9a!~7eYT-+QrCPp zRnvxNZt>og{q8%3lPq{&m*<-}buXk{*ve32U*RyE;w&m6X zX}Oe}(b86y{Cm^`fFMl)*_4|hYso+mDtG}RI9g6b+t}>c4Z0UD`puZ=21ImGzKOsS z3u|0(1tR!jwen?s`<)g6ez<7aG0{uMboLz6`7I*2O?9RaID-6^4z-4&S6Q+|FP35H zd}mDOeNb?}J$I$QbN-%mCoA{Wl)!$+`Mr%#>8it-?qBn5g`MGICu*YdHmX!)H$#*lTs!HMomX0xveqTn9f}Mb(f$*27PC zY(QnvCQ4wH(cSrOhE*m4@&wi>pO)9~o>u@K%Y8ssb+{#37GU$0pzXN1 zHJvB8VWt#NUO+QlhW47$zr3Kd(1Mpdp!gNoDrRz$#UV1Vp#ahIl)r+cZF3_g$2=GL%2kM~ma?x_cFE+5}TTOboi6Bbn3T&_}KA!j){X)=}*dCtj?D)%uze>5)9~<5B{4+VUpF@M*9+nXr2>7 zpn0OkE)5JIY(cCIl$&)8bA`;5Q;0p?RLPghWGbBtN|^~UNK-%0C^^a>0@ToecsfNU z4RJ%oMM@#T8u46W7de>H4m{zedcGl0v9LT4`mRK#W`G`04cvV}m=eIMu6hG%9kUt> zCOCjkM7lghEq?Br{HU4%hlL%xsL-4N3Sx)BORqYVxK zi8fSc7;RK6+RzSBH=={X#G&3nM}|v71``-Ru{g}v$oOGUKxu8NWEnX`yNSe2pd9p$ zHKN@_;?M~I#yTbqPJs5`j}?&y9K|W7M>O<7h5$0&cgn%=K&`_VMePatnD!cR$Lzp> zNnI2fH!z?A)f~ql35#TJ**p%SS%-Nrv}`!Rp_B1tHp3Q9(Vu()Ex|zyB1ogI0HNeI zv=#M_VxdeG1qP65!>}#69-lQbAPc}ikSRlYM#fGt$Hy*!gdB8C4(?_s8_rfj3}H4a ztBezex*3jxB@IRm4y<>LG3#5Bv&bnhowp}|PmmADT{i;> zNHW%-^uIyALqb8HQia&2L?$zqV2b7FsxlcVmQPsA0NNbpa>x^j0?jTHz00}2_&Oo28{-1ND6A_F>6d9dCX|g_s|{abgh(TI*~t&IH6PG zXJs(en=+HObj1~eya{H1+;AW%$=aq?%U{|`!AR8%{fnS3o$ zJ~m3O)k;gFMXCtMiWJli*c;hk&ZQ1#B&H43R3P&Z2&V$!h!t4Rr`!}@;3@|)3y2MD zMM#M!P!XrMCcYz+Q;;kiUbmr)87CQY8PEaAIH;1pl#IFT&tv(8WYlF)0mNnHGC$FIO!Alla0XF- zs`Hp+G-lw?;0&^{CXchmM3TphW)KEbueoqz07w@|o(-MVe1ftSnf*SKmb3?;nRe4d zmq0p_xHZVOF@&YqfVP&&aI>7iO5*5)aC&WBpq)AVp&xpwEBA3P7|>E&qFtmVD#uL~ z#6lL?7Fkt5EU*;OjSxT|hQs2vCQl@vmBGiW0TTl*2kshC;V>4U=BUBOmJl#*B+Bq1 zBAWk1GLjhJ0Ffxe)&7NKG|Hd?Fv{q-ZDC>M`g7m>4i?PoVRdWHc83vQx$l z%`Z?DcrYi>hY-{r5QZ$Pn;v=x(yibstuf6iju3>H?A;|*YjQDMrqZ? zY-LSIHC8Mj55l(uR1RWMY}(jHNmnUn9FZxPK_Blu-iXU6F}lm@Ir=cxAM~ZL9Hj-{ zQ^20l0i%k+*@x;4c75N^>6YjOeR-CVLQvP6qaNiCBRM|^6q zjZco!h@|LQ&8H~ocyPuNbXgM$NXh$4A zLrF(wwa=t#ymr$=|3JDmmKEX&HMm2|Wz1O9z7j?dGVPHE;p9W*z~REjV1%;zff1Tf z*KJ@A_7>P)p>(a39Hj%>3RIz8!}M=vO-5uZXDlV*9roI>)WpT33mbeAMgT4fi2&9M zA5u)vm|2?fG14U*c0eH zCK=5df6phHfrI{*l%0;+#STKP;8`1b#wts}&R81*Mz{hAtTWc%4;MdUrL>zh2e8gqe-sVE0#S9G9l;qZ zu<}mI*ct2hmi&dzZ6Md97Q)V0p%uo7hS9g@2N&tuXpqZmi)1LPup{SRnqy4ouDFBn z1+z-oBpsZ<_1cnyz_xRerULR0Ng=R$SDl7;}p47Iq+}hZhpa zlo_>xX4v0gAxU?7^O+)J51wJoVB}p)0@nYy%H>(cA&0!R*XBSHuom(UsySMuRK* zzjkzM{UF%IU(m&VKt2}Pc2L3|H`8tC5=)|ZoMxXLz0n5_o@E{mL`$T}JtB~a=Sr$) zC~`-SYI{AM9h)Rh(rGE$!>JYed&OM_`1IeBEZgV&Bx;d{S7 z&ldvd%O! zmg47})ZRaL2hI9d4+MWaZ2g7ysL!#LC>}p8#LiN&5Mu4-Sue}~p_5sIF!%zR?7itB zel%BN@8$wd7LiRJEQ9k3^evH*2j1?mgG;oGY{)=H`M-Q_NTEe80q3(XOYu_?{7tbA z{el>To`lwa3@5RU`$9*@j_3yEu#-#}1K|A^dFfToh~D%UPEcW_{Wl#_GK1sN_HeCF zd%SGxN{$Mml*PB}Rz-DH^2EgnJA$2~(vx3s$bB3r@+OeK(NxGX#4;HoaJ8EX738nq zmx!ioDFONQ``^jj2#iPCwRXgm$$;&@Wm#y=>|az8%`C~s+5^B_ij1+P6q$UCEaRQC z+6GL`iK_p*uWz0<%e2H5@!Y)Ju|+n-d``Ueme$Qgp%I@`YrtRTK%gfaQm zV46={HChAy_bE*EZ>9lE1;df-M#Yzq+}@Tes3X`9fvc(|I&tO zjw|^~qk(%Pz;6y<_XAdDXjd>^NjXE1(`NgF@vC|VS)??F#-9U`@7w_5|ZSNfAJ7Y^S+sf=W5`Nf=m^7{V=l>QTVT zfdpT0k6@ zdkWJ`m)-(L;@A*Cl+Thw$54{?4jb1*mH!RH7&kH)gR?Q!VU+*G{78O;101bNfR^<0 zA&ALQz)>5V*6-p7QP^Bx_wz~^y@Ya^QkLUb^LpYcBzg`w;n>i4oF>r=L3vs%WQ3|`HUss4*f z7*|W>D6ADVh&9tEu7c4)FOw<6ql3)QD@3Hq_dyz~qD^{EP?zY*--hV+qK(z)VLgzaItI<#-1Ew+!QV=0|fc ze5z0-7<_iFY77vRqaa4rP$W3D2f$jmnmhce2xo|*HF?ZsEZYvV{He&Amf$hOjd(|J zw>Q8F4=0vuk*@Qr`)O~jM>8$6y z0*=f+$p_2S__gQXD{v=&ymQ`)bdhQL((inPl+R-Bg=^*a@awmpn{zhq(PSgNFA2_p z8f!0eIXu7c&}g?s{WfO~i-l({zk0Dl_-c5n>rUTQD|*zul#1R3?B#7NyX2H{GLc#~ zHN{o+M50OS+&hn_xzBHSFR($?1;rhR314 zPr5G4NT17Gr^wyS<{i`$H|l@(vE%`(JmV=G&-+ch9$OU7=RTd!pC1?WeyQJsEidkc zD4rbhSf*q>eb33$Zt)cHW19*uX(-c!C1TvU&|si|Sw@k!z>r%?RUfO$r1xOajZFz+s#EXq>p zZ%kOXeLwG0?=GD)jZ-s%=(b@U`PG-i-)Wsx5nRd4pBW!pen+!to>bMW8$3j>*C*%Y zHaO8Es)(V+85`LyAEqRovv3G~@QR%+%gK22oO~f6kDwEM%Uy_FS-Fn>W)5#u7Mp}N z`y}-9%p0WMo+EeqtKqT5XO5K2U)EDLNyz$D8ELk%*7VoR0`>$kzSpy-&)1vlm&kT) z=hL>Y)Z^1%&-NNpt_>2+uM;m0cPSGp)>K_?CN#$_hdd1rrCKI?~NZ%xKbR9*x25rFD!7 zGQ&)WBb6j3gE=%(OJr8oG8+VmgH0WkqaT8q{Q%)XOpC5%2G`Pdj<|)AN zsL6E186F61LvSe{WOyhb3=cbPX@JTY22>02;cf)E(TgBAv@9aH3!63+Hre;*#1ELX zspSk5^!Iou$bSYhF_j5eIPs9le3iWTI;Um zcu`4;?g9A~A$nUMRfY-owCov50|1bYVu%z0V3EJ9)EKVgJ>@zzIOt>8l&-%gsj&HE z&_@9NFzgu^*s35OjNn6>YZY|*M=efn)$6afIBB2R+0c+Z>QEn+=h`rkUFZpL9FQME z0sIg_z9qA;4Ju2#}-&0g})gvK-Jl$PhvvxrLBN-h&}+6$=0=J8CBbm5)@l=15f3yg!)wiB{Mm zNYxw;CIvV$d0hG%cIpcAC8>+ws0=I3)k{#1=BNzgyP{X6lrbHkIM^L6_a1bIO$ zK9d&svJyfuS~uZvi-#SvG4RFg3hD-o8}VJGtPM`N0Kf|f+tAe6fKP0=fuTAg#1PJg&Bn^=ma1ylKLC5A6{nLbV3|w&x)p6JlA~zst1aLNRAEBbf{r=3bLXGK~je`P^hI!9xHbf;ByOs$dPN&! z`$a8Z0OCNg8*N0r^MzcDs*)g0a1Vto>d~vsyUmeH%6#R3vq7QnNE-wU3|(W*T5z?N zliCVy>)Cz;a1L-eCHbXCM`N%!Z_3lXVlCYkB0fgXEKQUv3p_b>qD1cP2X`mV7oH`k z)N6yATHx`Nr+SAx~rX1+K`b^+fvt%H+f-kto;{T)T)ha&JdT z9iO8_UP6usxFcG%dAAL?M#WpRxg_o7E-JVTU%Q$8B6?%%>_rtJjH1!7nH}S9_?GfE zp?C~C-GdMPtjhP?(cM{WRL}zM-f#PC2&=Pkx&?6r_Ipr728t@dFEr!mkv$OOf*~|Y`TN&D9M7h(CF zA|h6RR~4P45E#b4M&Rffy?ZK{`N=nWc%&01s6vGsu_i-vd>~=^F7Ta0g^LVDJ?19D>?tjfb}^GrwnV^rb(- zpbYm)akNA#xFCyxG~m6(aexdI83DuRXC*kBegmfXGpqud0jFKStEtKByX<>=&!g+m zPk|l=f{Yt%15%Sx_1NE%I1$t<}U_On0L63wVKk3{x_46C2_WszVDSa|y9HERht=wZL6v+zI2D7`=lmat91tgtNg= zj~;_-v>6UZIzyxeyd4dJ8-AYqzcUuRTa98)#?dv1g$VfrkY^xpLrf4{er34?UVubl zD)6c#)+0q?v?@YxW4!aMai+u?imhJ6hVhj`z16MFy}1qV_f z{zPwULeKuJ1mpT|05Vwc2!NTwK^8wRA*>2qT}XJr;Tk_L!Te4I@qpq-#-TSBDJmvM z6e7q2dwAhEFI_JQ@B1EOLL`YTa|~KSQ>5XP3z%h5RL1$j{X7MbsKL?*h7D>o=xbCB z=xc1WrTl19c&V0gH;P6d<#z(t`kh#|{GPZmkxy2=XxCTg2jNP9_0V?pWHckLR zI1VrYz|8c#5Xnyf?88D>Ou*iNH>)vB$2j-|yfHnty8Nv4?*e47)-VL00b>@2hQOG` z*c87iftc;whEp#fq5-|QKE4Ff_~0m#%7CQ-%VvllvlPseIK%_qnMW}tz*LN2Zrm*2 z(K%Q;!CsF7mJ)On)+6*Wx;9Th$-uPy{u2KL#0`Y7_h+mJbe1OwDpucc&y4GavpTSr zkQ&uy+mi(e4XqX`*e)fg*#$*E#8a*-NTXNg?SHI<46KhFD0>1<2DA+fk1-qvi?U%3 z;TbSrJmyw~-5{){-?xPh-X$2oy0-rNw$MYk*6>FB@7uyq<#9w5ME6tZMEXfv$~=;0 zZS7uUEKwlA(g%F-?!aw%HGJ?OY2YITck6NS`ZYw%j##H+fE5P#8%XEqHF!*4xUWEY zBhYy`K?CneJzA=W189OQFo+e=3;3wvVLYLxh2aJT!W;@XArROB)(WHxbSF0cF*GWW zep?B!_e=oga}a|B3GzQ|mXN+EaDq{G z3Y@$khOnuGBp(E{F%}BHg0g^R2Zz+Kz&!|R1*LV27Xl-YrHpJa0{VbiaBLYFizX;U z(-EM=;9HEsML-Gx{DxjC*eRbWEQN%pGleV`N;w?|mH>>?1WXaI`D8DX27k>vxVewGF&IBW) z{7@PPI)SOO6inS(;q53IMNtf_82Cs*hQKg3kQ;|v6(%*7(K7Vh6fT0zHaN&ZZH9q+ zQ2W4i0qHf+y)2)=jDRTz!97OUiw%NTK{E#myn)MuE{^VCL1185f&Spj^-o(w4JJsj zU-6W#%AZs32!1<#2y z{8H*r&Ja$+@X1NTe-BioKL7 z$z?Uch00kuTboFd)YSPg#RFA9FCeyP0Xzy{iWYT`Ta>wW;=m`|qy z9!DMHVGV?D9N0vY13%9_d!O+gg9AK){UBZ?K0~C$0bHhZ-wb+_i8?{ggjWwD)RDi; zk-yxf;2)W9yL$-&uJlp2WPnq7@ybNtE);W%v?sP&!HQl~P>%$%LMa@K?NsWO=sen;G`ae=F8gn-swmK+ zSFHe09f(x?r{?gNzJKLM?1Z1vI>vrBKJGW{zYxs&**Lt;^shrT@Sn!R8*r?f-s69T zj$XCFf4YA)_?7YR9{e|t0-`i(BMs1d2YV*dJ3G#mp0 z?{57Tje_hEVU%%$o0oopPk~U#VF0KL480*)E&~w_kHCeve6kGqG4||0pGhj#WQ2qf zI_qp40g^l5olQW>VBs;)2ccZFG1$tZ0^yzZ0PNC!0OUqRA6||3_3I*X(fu^i7`;Bq z0#OiHEDop&nqF}N2d?ukR2h>7udaCq{zM$MvLhD2rNNaQHx)jzF>%OC%6Msi&rahAa&Q3wRFj{WeFEju+pVDez=d}@PC)UQ6o}JIeobM41_R?~x&1FwQMAnmx+%H{ z0pZ5(ddS|v!6$&dgG&TVjsR2-XlFGX{F1R$@h|X5pdD=g>qsZx59CKM4SLI2jTOYu z>wnwC$oz#txs?^N<0T{E3CTDY%UdAJqzZuBaFj?mBk!-FSpH8SOITM``%kGSd;cr+ z6^&??nus;wFc6|=I<}Rv#X&AGV~%63pfd(V%;4Y@m@xqwMx5mUghlhCDfv zr~?kaGL!n?=@<5wLZNN#^};RyM5t(U6jsQ)O8_Tv1c$6A&95oRmgF=Rf5Q_D2(TqblhENDc4F=(`!XX@K z2Wg(QL_K#}=oTktByXlS;6;J|r>fwNJip!m0$+vz!d_Au)1Fj0g4xERBE7kyfG7*mhE?95`FrZlA4sd7F znzc9~tV$d=f-XrAmN9M|KOA|ov`2X>P#Rpr{%xe-zX=9umxM1SHX73;apHs>&Abz! ze!&1AdBZZm1S6K{9qk9uT`C=J{LwNp(^CprU<+uX@dt7!A8s50_9}ueh9>6ULL~>v$NC5q!t&j(8v{km(5Fl|;6-$E0kh#Pr47~wk z07q(sGY(IiptJw?>BT+FHEA}ySyAg|KHll|h;FZ{C4Xnr)Qvrc3hY3xm$HT#rGbx~w)V$LoHg>_ zKJ?$>J_ZN}A)!JK5Wz4$h6Zgdj)CZn`h-`Sxv?s(%aF=(5DR!UdSdLu7&hboKEXn9 z-Ap}jK^X;gd=JP7tvUPv!-=%&Ji(|L-&Mk(10)S3LU21k$jrp_kpBTA10q!Z1oA*{ zTmJwQ`I{gV)5YLMH6-oee2i8dgEAQekO9Vf6@zAcUpX6n^54P8_@BWgAR+=&9im?* zVtc@8Pt22q4jIjPOcMtjrU@`36S7CJ%KsIzpkuN}Y!!aQfRQ^Y(Apcfq2cZ};5kNY(Z4W?!479a4 z`Jy+<^5>OEI>#9uKPO$F=^XMf{#&6PLzfF`M>7!MRzLTRY`+P}8wiI6#c{x`1y0^* z?dbeHR0QTh+_rER@DqXvictX``1jkjst0rVKr8{}VDw5*@-R3C82zt`2!lkQG7NV@ z2&`=!a0A^@(ZP!`>UQ`T96zfgxCUat4X}Ar=C3gNJ@WQu zuLxd{C$Pk&=H3F)CmN9pS9ta7I-fdV@3={5MZ1~!=RT2U)zAY|R=i`})kwd!M9_Es zj&-~IENRm}`f^Knly}Xk_0HUL=XU4L0`V3_QnF+0wU?WO)=IryY&m7bFW+EtLymip zw;J~m?`hi?mQ;w2EZ){A8>L4p9ZbBdz_l%8?hDDp+%LzXtz2dqUHTL~yE?-z`uNMv ztD~(CK6D&#T$euW$)L;T;S)vd;>Ry??dz&vtk5ngC$B}C_N2dJai;EhhI8r@J(uPC zymB^MiM%+RR`5A$8IO`pKE ztWZ4@*QhrsGINq`S9>|#O)Xe)xMLt>Podzb*N4c%$1LU6+<#yh?KC_{qmepC2YcN# zJxR)83fK8ok2V(SWe$&cmWI2{r$k-s>hShc3r_E+Mko^P+KXzMnqOVvR`zhU;S#i$ zIyQ^bs?3wSzh&jsxyya0x^GL`c|lKUL1)Eeb-pMU6TX_JSDVD@8V7U=zKmQeTxIdz zVYB=Cx~2LFImyM7G(T%!b#tSiZVd`3RH;(Aa!Z?Fqkf7#SuuE0#yr1wrdQTWm>uxh zE15gPH)!zYl2h$RtC#FpN21=DYv`;JYU6fs$*usgMP5@x?^r%sGH<8$0XF5;ff z|3_+_{reiB)rOwS9I1DH^t^g znWzSof4r|2SyGdG@3Fq*o3gat`#H|D{f{XW-prggDWd&^<|&=+)=|$XQ#spPQqKe~ zx49mpVJ)auKFz*wf$%lM%U%n+R31ccFNrL1JxEr5Qrgm7G*x%Sf&T58(t(ybmu9a> z8S+o4xpw=4b+}-B%{sK-)}DFw{4{fbW@cx0ok& z;5EON0?&??Ozi{dqE@AQUh#jp$mWu|de@BVRGOqGqfENP%D3Qo#QoF*%kF#nWq+2o3P0^0 z^_fns-j_jjQ@!AK;oK!J`I~d!oPIg?u-n|;#$~(eFDNzN4ZJ>I#ce*<6Q@Pb4R|%9 z*ou#+U)D6`)-BO;S{Sx*N&w z7k0$&X@%CVicQ&jr7EYd;97BUE;+8)I3?KNg}?3I^>iOm-!vhqkedEEu}4-~I`f}B zwTWFNy}?&6;;MP@rXBIVikEL~bhv4FTW&|j1tD2iPQN?fTCNiL&fK>UvfsNaJZE!D zsPP=0>nVGbtGYjTYnE^0`5;vB(vp38eaPmh842C7i#WC?xp)ZkZ1p-*c5`lHOV8Ye z3XL-<(Z`3Yv-Wv?5ULvb*k301kl@6#-QBe}&a&>_^knHZVKo9g$uqR(9t$E0&wlD* z{UKsa;LaB5$MMq*gx!^!M6|t|-wBK)bYJfBiCiR*O3ii&mpxHCRN_cikKAf_`;60> z<1Zeosz}{;O5Rx$vsQ7}`Us8IFSL^7l+-4#wq@#pVXygBtCOVX%qs5AUsvpt+a~!! zT8=+aMJ~trRp%}fm)awSSBiOPcHJ&j`7aK0ypD@1xgITE@IlMO+MMrg2nWNBu-0|< zs;Xmzsk`;;rriHD&G?S}rii)q{!*0C##(XLj)z(^(p)uGwDUQiKd@FZ+d8pmcFy?} zLErrus*RHb)@^bxIBBP@cu90t>f5Nc-u{Zaj_K)7QZ}5jxYHs!@OX>1Ub&y>Em3>3 zXQoY#o@^~u(TAsfI3IsdR5GGtyHN9m#ja`=w_jOu2N4vu4sriiMwte~?eYgQ(Xa8HQ9n(>Wihz`C{ z1B9%FY-`x`FA%KEOpclui=VSHFgajpW==p5Sj!%-WLxtPtkJ*u@AT~t=W{m9xk7Wz zbop5AOFSON_0r|jn%hR_9%>3j>tBAd`IBsdPo}+r-9G!{XWRnK%FZxOhJBl)V@9tLtso%@a61>I=_6JY z*Eh2?fV-JxkEc#r^N#Y}h-yMR`E(Sf?g$KQm-9?`P> zr|?+5X_w>@%WOHz(AL^M*|3jLBoRv++D!^D-o;$S%ouT!liyr==KajupN-g^B@>TE z540>)tM8j zB~^1pL_B`p9ryl5gwC<+jU4I@>4FW(_XI=s-K3uFA-NycU2kI`bR;er?!uh5*0M`Fwp$faqcU^D(BDo=By>s%y=StAhEbG{Gjlji3C zMD^HA4)8{zdN+gJj^pysy*RC@uG=^64P0=nt<8qaEr zuv*eV|AogrPH!|w&koyBva3Vm-Rbkj%RZF#wI34NvQ7Jq%dEo&Tnba^pB@UQyrpYh z-?KySlx^`X&fFM_@{dfh^)4*~pXk(*O)*CpCT%Y!eN8GcGCH;Q(PLw-H+D0_$(aIN zva7=M-DQ%_>OOwrO}-o8d*0|(y}M`8{NyY9bQibIedK92Wy|VF+2ht{*Sua=9xPC| zWYR9K(7v>@SmmSBv7 zPS=-a#Ry7$?MHg$u~8w)oYnHH#EI8OC(PVeJ@#)11|hy}>Q9$K4II))-3pn!mM!2nXE34|h{2%$)oA~k~aB2^$v+x&QB;*^_f-@_l=)*?XV8*0*NPL32RbDjYf)dqOnerS;`G!x(u}jaZmfj&Jx_CG4)bd3)wwyZoDE(1|4Nqb4M0@ zz?EBcZ7(l~E-rWKHhqFnhQqGld+cUNmN2eLxz+^_NMBO43D7FidllX3F>1zF;KS;U zKk(2jB|_O2);;dM=vV9;G*Si2kRwQYtdeI!lJHnV4kb3A{$x6Kq5&DRpo&pV7c z^BmFPSZLJ=gGv`nzkiSk7i@s6I8Ihk&gxOUOQ*BXdsAA!i`t}KxW7Z;z)j2ep5Et@l< z=ZM4_nwCW?Yg4zx#7wDEG^2K_ie3A5=8}2%^q&cv*q?I~SAXtA%En05Clba572$vu zE7gxJ-)E*UHxp-#-6&;YZ#$n9(HYRwWG(c%-dqAxde{Av1l6^|IlW6`aRV)5KeXu> zFNd6uzaFZ2ra0s(br89h!_4-)vD#%;CAOj|txT>P-e8Wi`3cFl;<$h1mszxFYt0(_ zY)nrZM7=I+?NPdR%MPA(Hn8E-hsP6XaRjQQIJLG(z02nBt>t4$j9@8-UABjZB#rNV|XTAf>7^tYSDMq{##@F{QI2#9ay3I7&?o2veIvpNxDOb#yzs9+uq` zl04A@<{ML)42NWtM28frYs80Q>fqLsSZ5<5YH{0U&$m5Vm}(K%NCB{dx)wVHq)>V6 zDhK&h%}rc{cREBw8#Mh<@Fs^vvzyz_5U*Ew?6vHA;n}BmqIc`=TCee~_v$_Mz8%Mf zg0xDk>oucPGGiD}ycS;Sa=CSPc|Nlxfk#NWz>_ zc~ZW>=pi*bj z+}#|Zu`T>A?gng=bcprNkP7@fB8q*eEi1eFlj(xp$47g44=r_AwlQ8>AYvxJH_!C1 zzGNq=lu291E7xDxs2iN8s%?S#%pO8FCY=Z-3m zg+*>N`b18bPWIdV5fF5LT2a}=s97!Tk>~10>34s?XaS#G<*6v@^3~ALXOV}f>Z^@My4DmH>8q>aY61~=|T}{ zXSk|#`w4rQOfr>t-^;%E>fC{{sPu_j>`xV4GVpoFE?MmQjxB&T->60<;TIga*0hGV zMx(04GrZ!P%Uyie8}n560tr=mjM>P&n}Hizr1j&?bPEr**k|Sn%JsUiackbO%3F=? zE)Ah0t(Y0Z1@fLSPJ$v{W(j`jNM1||1f0|7w`F!wzO{$<^EVM=_+m)+tp$QKl11>1 zJii~r4jJsB=!X}iGVnJvSS!tBIvEpR-At~(PPvOS(#obu?7gBHsGC#L5G=ixWa$=x zfvj=oJ@8DCSFmLAUxJi!dOdJnD}DR|lrOG3`hxMxmo7YZfiL~g)B$nH`$+xj#nvfU zmUMZli>XR{Q+1c6wtWCop$K-U&uZaJ<)FW7d)=u1%zVF{dx6S88m*%L{NQM6_AUAZ zOi=hbgCe9iU677@(5*w(-9LX=Eg}_3Zi8HPb)v=TQn8#5@LLvEv$N2vti(o8FYnW8*(YQAQrRELGc1|$5TA{ z9;v|?-=I1D+FML6e}|@vwpR0VyV=!8JPgAXIPv|r!0jd?*}aB|YY%G;Y0lTm4(w25 zwW6x(E$^>vP?Q8EtzUK9VyO4fz-R<_3}~<4MmUX|oQno}3DA3ij`jZYTh?pM#jDxq zpG;$%(8)Z`_CIcN88_$kKCIP5fGxCUe_m_ryw;jK;YV#}x13-iw-upc=^8JVIZ-Rk zVRCv1kCMLP(FjDUTtJM&421B9slJI50_l8^@E;s>0{gfBrm4S!`oOM+FkC@$!pW*h z$=I>fPIs!PMR$F=>d?x`GL(>TlZ{p~*VY=H1~1>TAj#%l_PBIo{z~GomPohuWiE7t z2UHPHdj4xa9XR}PO)B>@KTe_Ls@Z2(KOuAYzR%FScqT6j3(Za)SiVRe&#Q8-%_4&_ zKUOGGLf@^7?X!!K18UDsX8mrSoC6DVi?DAqNd@~$lp?@wLt%vu)(WK_8Q0@7Z1if6 zt$Z&U$wgRg>RyqYH%9iS?P{~{IN525oMAjC>zn_x3Jw&Xe^Tp~h3+X4Khj&XxbMWna`FeU0??1lM6!Od9%+`$uI$w=jQxsC zD%jKp8Y_CKoZ-BLj_^6IU`x2VC+L0*Coexig1&p-fS{k6==R$SJW{UT%52IS^o)AX zEeB|u2wts{?C-l`SlWS5zOci>>>xYH3H68kGHfQ3*^soK09^SkT;SmHgAQFqO@z$* z0?#}aAiUqNFdlDr`$%5t!NPqLh>G#|21CLXYytUI41IIfi+!a<A)oa59RzM%W;AD zD9c0!2=q6(_$2(e5O)+_mURUFmn?S@eO%)>icZ9w>cXk>aXeW*$`bd-NtROyj)x*g z3EYd05}X#8oOJ2 + ref shouldBe loadAgent.toTyped + maybeNewTick shouldBe None + None + case ScheduleActivation(ref, tick, unlockKey) => + // em agent schedules itself + tick shouldBe 0 + unlockKey shouldBe None + Some(ref) + case unexpected => + fail(s"Received unexpected message $unexpected") + } + .headOption + .value + + // pv + pvAgent ! Activation(INIT_SIM_TICK) + + primaryServiceProxy.expectMessage( + PrimaryServiceRegistrationMessage(pvInput.getUuid) + ) + pvAgent ! RegistrationFailedMessage(primaryServiceProxy.ref.toClassic) + + // deal with weather service registration + weatherService.expectMessage( + RegisterForWeatherMessage( + pvInput.getNode.getGeoPosition.getY, + pvInput.getNode.getGeoPosition.getX, + ) + ) + + pvAgent ! RegistrationSuccessfulMessage( + weatherService.ref.toClassic, + Some(0L), + ) + + scheduler.expectMessage(Completion(pvAgent)) + + // storage + storageAgent ! Activation(INIT_SIM_TICK) + + primaryServiceProxy.expectMessage( + PrimaryServiceRegistrationMessage(householdStorageInput.getUuid) + ) + storageAgent ! RegistrationFailedMessage( + primaryServiceProxy.ref.toClassic + ) + + scheduler.expectMessage(Completion(storageAgent)) + + /* TICK 0 + LOAD: 0.000269 MW + PV: -0.005685 MW + STORAGE: SOC 0 % + -> charge with 5 kW + -> remaining -0.0004161 MW + */ + + emAgentActivation ! Activation(0) + + pvAgent ! ProvideWeatherMessage( + 0, + weatherService.ref.toClassic, + WeatherData( + WattsPerSquareMeter(400d), + WattsPerSquareMeter(200d), + Celsius(0d), + MetersPerSecond(0d), + ), + Some(7200), + ) + + resultListener.expectMessageType[ParticipantResultEvent] match { + case ParticipantResultEvent(emResult: EmResult) => + emResult.getInputModel shouldBe emInput.getUuid + emResult.getTime shouldBe 0L.toDateTime + emResult.getP should equalWithTolerance( + (-0.000416087825).asMegaWatt + ) + emResult.getQ should equalWithTolerance(0.0000882855367.asMegaVar) + } + + scheduler.expectMessage(Completion(emAgentActivation, Some(7200))) + + /* TICK 7200 + LOAD: 0.000269 MW (unchanged) + PV: -0.003797 MW + STORAGE: SOC 63.3 % + -> charge with 3.5282 kW + -> remaining 0 MW + */ + + emAgentActivation ! Activation(7200) + + pvAgent ! ProvideWeatherMessage( + 7200, + weatherService.ref.toClassic, + WeatherData( + WattsPerSquareMeter(300d), + WattsPerSquareMeter(500d), + Celsius(0d), + MetersPerSecond(0d), + ), + Some(14400), + ) + + resultListener.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.0000882855367.asMegaVar) + } + + scheduler.expectMessage(Completion(emAgentActivation, Some(13107))) + + /* TICK 13107 + LOAD: 0.000269 MW (unchanged) + PV: -0.003797 MW (unchanged) + STORAGE: SOC 100 % + -> charge with 0 kW + -> remaining -0.003528 MW + */ + + emAgentActivation ! Activation(13107) + + resultListener.expectMessageType[ParticipantResultEvent] match { + case ParticipantResultEvent(emResult: EmResult) => + emResult.getInputModel shouldBe emInput.getUuid + emResult.getTime shouldBe 13107L.toDateTime + emResult.getP should equalWithTolerance( + (-0.0035281545552).asMegaWatt + ) + emResult.getQ should equalWithTolerance(0.0000882855367.asMegaVar) + } + + scheduler.expectMessage(Completion(emAgentActivation, Some(14400))) + + /* TICK 14400 + LOAD: 0.000269 MW (unchanged) + PV: -0.000066 MW + STORAGE: SOC 100 % + -> charge with -0.202956 kW + -> remaining 0 MW + */ + + // send weather data before activation, which can happen + // it got cloudy now... + pvAgent ! ProvideWeatherMessage( + 14400, + weatherService.ref.toClassic, + WeatherData( + WattsPerSquareMeter(5d), + WattsPerSquareMeter(5d), + Celsius(0d), + MetersPerSecond(0d), + ), + Some(21600), + ) + + emAgentActivation ! Activation(14400) + + resultListener.expectMessageType[ParticipantResultEvent] match { + case ParticipantResultEvent(emResult: EmResult) => + emResult.getInputModel shouldBe emInput.getUuid + emResult.getTime shouldBe 14400L.toDateTime + emResult.getP should equalWithTolerance(0.asMegaWatt) + emResult.getQ should equalWithTolerance(0.000088285537.asMegaVar) + } + + scheduler.expectMessage(Completion(emAgentActivation, Some(21600))) + + } + } + + "having load, pv and heat pump agents connected" should { + "be initialized correctly and run through some activations" in { + val resultListener = TestProbe[ResultEvent]("ResultListener") + val primaryServiceProxy = + TestProbe[ServiceMessage]("PrimaryServiceProxy") + val weatherService = TestProbe[ServiceMessage]("WeatherService") + val scheduler = TestProbe[SchedulerMessage]("Scheduler") + + val emAgent = spawn( + EmAgent( + emInput, + modelConfig, + outputConfigOn, + "PRIORITIZED", + simulationStartDate, + parent = Left(scheduler.ref), + maybeRootEmConfig = None, + listener = Iterable(resultListener.ref), + ), + "EmAgent1", + ) + + val loadAgent = TestActorRef( + new FixedLoadAgent( + scheduler = scheduler.ref.toClassic, + initStateData = ParticipantInitializeStateData( + loadInput, + LoadRuntimeConfig( + calculateMissingReactivePowerWithModel = true, + scaling = 1d, + modelBehaviour = "fix", + reference = "power", + uuids = List.empty, + ), + primaryServiceProxy.ref.toClassic, + None, + simulationStartDate, + simulationEndDate, + resolution, + simonaConfig.simona.runtime.participant.requestVoltageDeviationThreshold, + outputConfigOff, + Some(emAgent), + ), + listener = Iterable(resultListener.ref.toClassic), + ), + "LoadAgent1", + ) + val pvAgent = TestActorRef( + new PvAgent( + scheduler = scheduler.ref.toClassic, + initStateData = ParticipantInitializeStateData( + pvInput, + PvRuntimeConfig( + calculateMissingReactivePowerWithModel = true, + scaling = 2d, + uuids = List.empty, + ), + primaryServiceProxy.ref.toClassic, + Iterable(ActorWeatherService(weatherService.ref.toClassic)), + simulationStartDate, + simulationEndDate, + resolution, + simonaConfig.simona.runtime.participant.requestVoltageDeviationThreshold, + outputConfigOff, + Some(emAgent), + ), + listener = Iterable(resultListener.ref.toClassic), + ), + "PvAgent1", + ) + val heatPumpAgent = TestActorRef( + new HpAgent( + scheduler = scheduler.ref.toClassic, + initStateData = ParticipantInitializeStateData( + adaptedHpInputModel, + adaptedThermalGrid, + HpRuntimeConfig( + calculateMissingReactivePowerWithModel = true, + 1.0, + List.empty[String], + ), + primaryServiceProxy.ref.toClassic, + Iterable(ActorWeatherService(weatherService.ref.toClassic)), + simulationStartDate, + simulationEndDate, + resolution, + simonaConfig.simona.runtime.participant.requestVoltageDeviationThreshold, + outputConfigOff, + Some(emAgent), + ), + listener = Iterable(resultListener.ref.toClassic), + ), + "HeatPumpAgent1", + ) + + scheduler.expectNoMessage() + + /* INIT */ + + // load + loadAgent ! Activation(INIT_SIM_TICK) + + primaryServiceProxy.expectMessage( + PrimaryServiceRegistrationMessage(loadInput.getUuid) + ) + loadAgent ! RegistrationFailedMessage(primaryServiceProxy.ref.toClassic) + + // the order of the two messages is not given + val emAgentActivation = scheduler + .receiveMessages(2) + .flatMap { + case Completion(ref, maybeNewTick) => + ref shouldBe loadAgent.toTyped + maybeNewTick shouldBe None + None + case ScheduleActivation(ref, tick, unlockKey) => + // em agent schedules itself + tick shouldBe 0 + unlockKey shouldBe None + Some(ref) + case unexpected => + fail(s"Received unexpected message $unexpected") + } + .headOption + .value + + // pv + pvAgent ! Activation(INIT_SIM_TICK) + + primaryServiceProxy.expectMessage( + PrimaryServiceRegistrationMessage(pvInput.getUuid) + ) + pvAgent ! RegistrationFailedMessage(primaryServiceProxy.ref.toClassic) + + // deal with weather service registration + weatherService.expectMessage( + RegisterForWeatherMessage( + pvInput.getNode.getGeoPosition.getY, + pvInput.getNode.getGeoPosition.getX, + ) + ) + + pvAgent ! RegistrationSuccessfulMessage( + weatherService.ref.toClassic, + Some(0L), + ) + + scheduler.expectMessage(Completion(pvAgent)) + + // heat pump + heatPumpAgent ! Activation(INIT_SIM_TICK) + + primaryServiceProxy.expectMessage( + PrimaryServiceRegistrationMessage(adaptedHpInputModel.getUuid) + ) + heatPumpAgent ! RegistrationFailedMessage( + primaryServiceProxy.ref.toClassic + ) + + weatherService.expectMessage( + RegisterForWeatherMessage( + hpInputModel.getNode.getGeoPosition.getY, + hpInputModel.getNode.getGeoPosition.getX, + ) + ) + + heatPumpAgent ! RegistrationSuccessfulMessage( + weatherService.ref.toClassic, + Some(0L), + ) + + scheduler.expectMessage(Completion(heatPumpAgent)) + + val weatherDependentAgents = Seq(pvAgent, heatPumpAgent) + + /* TICK 0 + LOAD: 0.000269 MW + PV: -0.005685 MW + Heat pump: off, can be turned on or stay off + -> set point ~3.5 kW (bigger than 50 % rated apparent power): turned on + -> remaining -0.000566 MW + */ + + emAgentActivation ! Activation(0) + + weatherDependentAgents.foreach { + _ ! ProvideWeatherMessage( + 0, + weatherService.ref.toClassic, + WeatherData( + WattsPerSquareMeter(400d), + WattsPerSquareMeter(200d), + Celsius(0d), + MetersPerSecond(0d), + ), + Some(7200), + ) + } + + resultListener.expectMessageType[ParticipantResultEvent] match { + case ParticipantResultEvent(emResult: EmResult) => + emResult.getInputModel shouldBe emInput.getUuid + emResult.getTime shouldBe 0.toDateTime + emResult.getP should equalWithTolerance( + (-0.000566087824).asMegaWatt + ) + emResult.getQ should equalWithTolerance(0.001073120041.asMegaVar) + } + + scheduler.expectMessage(Completion(emAgentActivation, Some(7200))) + + /* TICK 7200 + LOAD: 0.000269 MW (unchanged) + PV: -0.003797 MW + Heat pump: running (turned on from last request), can be turned on and stay off + -> set point ~3.5 kW (bigger than 50 % rated apparent power): stays turned on with unchanged state + -> remaining 0 MW + */ + + emAgentActivation ! Activation(7200) + + weatherDependentAgents.foreach { + _ ! ProvideWeatherMessage( + 7200, + weatherService.ref.toClassic, + WeatherData( + WattsPerSquareMeter(300d), + WattsPerSquareMeter(500d), + Celsius(0d), + MetersPerSecond(0d), + ), + Some(14400), + ) + } + + resultListener.expectMessageType[ParticipantResultEvent] match { + case ParticipantResultEvent(emResult: EmResult) => + emResult.getInputModel shouldBe emInput.getUuid + emResult.getTime shouldBe 7200.toDateTime + emResult.getP should equalWithTolerance(0.00132184544484.asMegaWatt) + emResult.getQ should equalWithTolerance(0.001073120041.asMegaVar) + } + + scheduler.expectMessage(Completion(emAgentActivation, Some(14400))) + + /* TICK 14400 + LOAD: 0.000269 MW (unchanged) + PV: -0.000066 MW + Heat pump: Is not running, can still run or be turned off + -> flex signal is 0 MW: Heat pump is turned off + */ + + emAgentActivation ! Activation(14400) + + // it got cloudy now... + weatherDependentAgents.foreach { + _ ! ProvideWeatherMessage( + 14400, + weatherService.ref.toClassic, + WeatherData( + WattsPerSquareMeter(5d), + WattsPerSquareMeter(5d), + Celsius(0d), + MetersPerSecond(0d), + ), + Some(21600), + ) + } + + resultListener.expectMessageType[ParticipantResultEvent] match { + case ParticipantResultEvent(emResult: EmResult) => + emResult.getInputModel shouldBe emInput.getUuid + emResult.getTime shouldBe 14400L.toDateTime + emResult.getP should equalWithTolerance(0.000202956264.asMegaWatt) + emResult.getQ should equalWithTolerance(0.000088285537.asMegaVar) + } + + scheduler.expectMessage(Completion(emAgentActivation, Some(21600))) + + /* TICK 21600 + LOAD: 0.000269 MW (unchanged) + PV: -0.000032 MW + Heat pump: Is not running, can run or be turned off + -> flex signal is 0 MW: Heat pump is turned off + */ + + emAgentActivation ! Activation(21600) + + weatherDependentAgents.foreach { + _ ! ProvideWeatherMessage( + 21600, + weatherService.ref.toClassic, + WeatherData( + WattsPerSquareMeter(5d), + WattsPerSquareMeter(5d), + Celsius(0d), + MetersPerSecond(0d), + ), + Some(28800), + ) + } + + resultListener.expectMessageType[ParticipantResultEvent] match { + case ParticipantResultEvent(emResult: EmResult) => + emResult.getInputModel shouldBe emInput.getUuid + emResult.getTime shouldBe 21600.toDateTime + emResult.getP should equalWithTolerance(0.0002367679996.asMegaWatt) + emResult.getQ should equalWithTolerance(0.000088285537.asMegaVar) + } + + scheduler.expectMessage(Completion(emAgentActivation, Some(28665))) + + /* TICK 28666 + LOAD: 0.000269 MW (unchanged) + PV: -0.000032 MW (unchanged) + Heat pump: Is turned off and has to be turned on + -> flex signal is no control -> 0.00485 MW + */ + + emAgentActivation ! Activation(28665) + + resultListener.expectMessageType[ParticipantResultEvent] match { + case ParticipantResultEvent(emResult: EmResult) => + emResult.getInputModel shouldBe emInput.getUuid + emResult.getTime shouldBe 28665.toDateTime + emResult.getP should equalWithTolerance(0.0050867679996.asMegaWatt) + emResult.getQ should equalWithTolerance(0.001073120040.asMegaVar) + } + + scheduler.expectMessage(Completion(emAgentActivation, Some(28800))) + } + } + + // TODO test stacked EmAgents + } +} diff --git a/src/test/scala/edu/ie3/simona/agent/em/EmAgentSpec.scala b/src/test/scala/edu/ie3/simona/agent/em/EmAgentSpec.scala new file mode 100644 index 0000000000..2c350a29b8 --- /dev/null +++ b/src/test/scala/edu/ie3/simona/agent/em/EmAgentSpec.scala @@ -0,0 +1,696 @@ +/* + * © 2022. TU Dortmund University, + * Institute of Energy Systems, Energy Efficiency and Energy Economics, + * Research group Distribution grid planning and operation + */ + +package edu.ie3.simona.agent.em + +import edu.ie3.datamodel.models.result.system.EmResult +import edu.ie3.simona.agent.participant.data.Data.PrimaryData.ApparentPower +import edu.ie3.simona.config.SimonaConfig.EmRuntimeConfig +import edu.ie3.simona.event.ResultEvent +import edu.ie3.simona.event.ResultEvent.{ + FlexOptionsResultEvent, + ParticipantResultEvent, +} +import edu.ie3.simona.event.notifier.NotifierConfig +import edu.ie3.simona.ontology.messages.flex.FlexibilityMessage._ +import edu.ie3.simona.ontology.messages.flex.MinMaxFlexibilityMessage.ProvideMinMaxFlexOptions +import edu.ie3.simona.ontology.messages.SchedulerMessage.{ + Completion, + ScheduleActivation, +} +import edu.ie3.simona.ontology.messages.{Activation, SchedulerMessage} +import edu.ie3.simona.test.common.input.EmInputTestData +import edu.ie3.simona.test.matchers.SquantsMatchers +import edu.ie3.simona.util.TickUtil.TickLong +import edu.ie3.util.TimeUtil +import edu.ie3.util.quantities.QuantityMatchers.equalWithTolerance +import edu.ie3.util.quantities.QuantityUtils.RichQuantityDouble +import edu.ie3.util.scala.quantities.{Kilovars, ReactivePower} +import org.apache.pekko.actor.testkit.typed.scaladsl.{ + ScalaTestWithActorTestKit, + TestProbe, +} +import org.scalatest.matchers.should +import org.scalatest.wordspec.AnyWordSpecLike +import org.scalatestplus.mockito.MockitoSugar +import squants.Power +import squants.energy.Kilowatts + +import java.time.ZonedDateTime + +class EmAgentSpec + extends ScalaTestWithActorTestKit + with AnyWordSpecLike + with should.Matchers + with EmInputTestData + with MockitoSugar + with SquantsMatchers { + + protected implicit val simulationStartDate: ZonedDateTime = + TimeUtil.withDefaults.toZonedDateTime("2020-01-01 00:00:00") + + private val outputConfig = NotifierConfig( + simulationResultInfo = true, + powerRequestReply = false, + flexResult = true, // also test FlexOptionsResult if EM-controlled + ) + + override protected val modelConfig: EmRuntimeConfig = EmRuntimeConfig( + calculateMissingReactivePowerWithModel = false, + scaling = 1d, + uuids = List("default"), + aggregateFlex = "SELF_OPT_EXCL_PV", + pvFlex = false, + ) + + private implicit val activePowerTolerance: Power = Kilowatts(1e-10) + private implicit val reactivePowerTolerance: ReactivePower = Kilovars(1e-10) + + "A self-optimizing EM agent" should { + "be initialized correctly and run through some activations" in { + val resultListener = TestProbe[ResultEvent]("ResultListener") + val scheduler = TestProbe[SchedulerMessage]("Scheduler") + + val emAgent = spawn( + EmAgent( + emInput, + modelConfig, + outputConfig, + "PRIORITIZED", + simulationStartDate, + parent = Left(scheduler.ref), + maybeRootEmConfig = None, + listener = Iterable(resultListener.ref), + ) + ) + + val pvAgent = TestProbe[FlexRequest]("PvAgent") + emAgent ! RegisterParticipant(pvInput.getUuid, pvAgent.ref, pvInput) + emAgent ! ScheduleFlexRequest(pvInput.getUuid, 0) + + val sa1 = scheduler.expectMessageType[ScheduleActivation] + sa1.tick shouldBe 0 + sa1.unlockKey shouldBe None + val emAgentActivation = sa1.actor + + val evcsAgent = TestProbe[FlexRequest]("EvcsAgent") + emAgent ! RegisterParticipant(evcsInput.getUuid, evcsAgent.ref, evcsInput) + emAgent ! ScheduleFlexRequest(evcsInput.getUuid, 0) + + // no additional scheduling message, since tick 0 has already been scheduled + scheduler.expectNoMessage() + + /* TICK 0 */ + emAgentActivation ! Activation(0) + + // expect flex requests + pvAgent.expectMessage(RequestFlexOptions(0)) + evcsAgent.expectMessage(RequestFlexOptions(0)) + + // send flex options + emAgent ! ProvideMinMaxFlexOptions( + pvInput.getUuid, + Kilowatts(-5d), + Kilowatts(-5d), + Kilowatts(0d), + ) + + pvAgent.expectNoMessage() + evcsAgent.expectNoMessage() + + emAgent ! ProvideMinMaxFlexOptions( + evcsInput.getUuid, + Kilowatts(2d), + Kilowatts(-11d), + Kilowatts(11d), + ) + + // receive flex control messages + pvAgent.expectMessage(IssueNoControl(0)) + emAgent ! FlexCtrlCompletion( + modelUuid = pvInput.getUuid, + result = ApparentPower(Kilowatts(-5d), Kilovars(-0.5d)), + requestAtTick = Some(600), + ) + + scheduler.expectNoMessage() + + evcsAgent.expectMessageType[IssuePowerControl] match { + case IssuePowerControl(0, setPower) => + setPower should approximate(Kilowatts(5.0)) + } + emAgent ! FlexCtrlCompletion( + modelUuid = evcsInput.getUuid, + result = ApparentPower(Kilowatts(5d), Kilovars(0.1d)), + requestAtTick = Some(300), + ) + + // expect correct results + resultListener.expectMessageType[ParticipantResultEvent] match { + case ParticipantResultEvent(emResult: EmResult) => + emResult.getInputModel shouldBe emInput.getUuid + emResult.getTime shouldBe simulationStartDate + emResult.getP should equalWithTolerance(0d.asMegaWatt) + emResult.getQ should equalWithTolerance((-.0004d).asMegaVar) + } + + // expect completion from EmAgent + scheduler.expectMessage( + Completion(emAgentActivation, Some(300)) + ) + + /* TICK 300 */ + emAgentActivation ! Activation(300) + + // expect activations and flex requests. + // only participant 2 has been scheduled for this tick, + // thus 1 does not get activated + pvAgent.expectNoMessage() + + evcsAgent.expectMessage(RequestFlexOptions(300)) + + // send flex options again, ev is fully charged + emAgent ! ProvideMinMaxFlexOptions( + evcsInput.getUuid, + Kilowatts(0d), + Kilowatts(-11d), + Kilowatts(0d), + ) + + // receive flex control messages + evcsAgent.expectMessage(IssueNoControl(300)) + + pvAgent.expectNoMessage() + + emAgent ! + FlexCtrlCompletion( + evcsInput.getUuid, + ApparentPower(Kilowatts(0d), Kilovars(0d)), + ) + + // expect correct results + resultListener.expectMessageType[ParticipantResultEvent] match { + case ParticipantResultEvent(emResult: EmResult) => + emResult.getInputModel shouldBe emInput.getUuid + emResult.getTime shouldBe 300.toDateTime(simulationStartDate) + emResult.getP should equalWithTolerance((-0.005d).asMegaWatt) + emResult.getQ should equalWithTolerance((-0.0005d).asMegaVar) + } + + // expect completion from EmAgent + scheduler.expectMessage(Completion(emAgentActivation, Some(600))) + + } + + "revoke triggers correctly" in { + val resultListener = TestProbe[ResultEvent]("ResultListener") + val scheduler = TestProbe[SchedulerMessage]("Scheduler") + + val emAgent = spawn( + EmAgent( + emInput, + modelConfig, + outputConfig, + "PRIORITIZED", + simulationStartDate, + parent = Left(scheduler.ref), + maybeRootEmConfig = None, + listener = Iterable(resultListener.ref), + ) + ) + + val pvAgent = TestProbe[FlexRequest]("PvAgent") + emAgent ! RegisterParticipant(pvInput.getUuid, pvAgent.ref, pvInput) + emAgent ! ScheduleFlexRequest(pvInput.getUuid, 0) + + val sa1 = scheduler.expectMessageType[ScheduleActivation] + sa1.tick shouldBe 0 + sa1.unlockKey shouldBe None + val emAgentActivation = sa1.actor + + val evcsAgent = TestProbe[FlexRequest]("EvcsAgent") + emAgent ! RegisterParticipant(evcsInput.getUuid, evcsAgent.ref, evcsInput) + emAgent ! ScheduleFlexRequest(evcsInput.getUuid, 0) + + // no additional scheduling message, since tick 0 has already been scheduled + scheduler.expectNoMessage() + + /* TICK 0 */ + emAgentActivation ! Activation(0) + + // expect flex requests + pvAgent.expectMessage(RequestFlexOptions(0)) + evcsAgent.expectMessage(RequestFlexOptions(0)) + + // send flex options + emAgent ! ProvideMinMaxFlexOptions( + pvInput.getUuid, + Kilowatts(-5d), + Kilowatts(-5d), + Kilowatts(0d), + ) + + pvAgent.expectNoMessage() + evcsAgent.expectNoMessage() + + emAgent ! ProvideMinMaxFlexOptions( + evcsInput.getUuid, + Kilowatts(2d), + Kilowatts(-11d), + Kilowatts(11d), + ) + + // receive flex control messages + pvAgent.expectMessage(IssueNoControl(0)) + evcsAgent.expectMessageType[IssuePowerControl] match { + case IssuePowerControl(0, setPower) => + setPower should approximate(Kilowatts(5.0)) + } + + // send completions + emAgent ! FlexCtrlCompletion( + modelUuid = pvInput.getUuid, + result = ApparentPower(Kilowatts(-5d), Kilovars(-0.5d)), + requestAtTick = Some(300), + ) + + scheduler.expectNoMessage() + + emAgent ! FlexCtrlCompletion( + modelUuid = evcsInput.getUuid, + result = ApparentPower(Kilowatts(5d), Kilovars(0.1d)), + requestAtTick = Some(600), + ) + + // expect correct results + resultListener.expectMessageType[ParticipantResultEvent] match { + case ParticipantResultEvent(emResult: EmResult) => + emResult.getInputModel shouldBe emInput.getUuid + emResult.getTime shouldBe simulationStartDate + emResult.getP should equalWithTolerance(0d.asMegaWatt) + emResult.getQ should equalWithTolerance((-.0004d).asMegaVar) + } + + // expect completion from EmAgent + scheduler.expectMessage(Completion(emAgentActivation, Some(300))) + + /* TICK 300 */ + emAgentActivation ! Activation(300) + + // expect activations and flex requests. + // only pv agent has been scheduled for this tick, + // thus evcs does not get activated + evcsAgent.expectNoMessage() + + pvAgent.expectMessage(RequestFlexOptions(300)) + + // send flex options again, now there's a cloud and thus less feed-in + emAgent ! ProvideMinMaxFlexOptions( + pvInput.getUuid, + Kilowatts(-3d), + Kilowatts(-3d), + Kilowatts(0d), + ) + + // receive flex control messages + pvAgent.expectMessage(IssueNoControl(300)) + + emAgent ! FlexCtrlCompletion( + pvInput.getUuid, + ApparentPower(Kilowatts(-3d), Kilovars(-0.06d)), + ) + + // evcs is now sent control too + evcsAgent.expectMessageType[IssuePowerControl] match { + case IssuePowerControl(300, setPower) => + setPower should approximate(Kilowatts(3.0)) + } + + scheduler.expectNoMessage() + + emAgent ! FlexCtrlCompletion( + evcsInput.getUuid, + ApparentPower(Kilowatts(3d), Kilovars(0.06d)), + requestAtTick = Some(800), // should overwrite tick 600 + ) + + // expect correct results + resultListener.expectMessageType[ParticipantResultEvent] match { + case ParticipantResultEvent(emResult: EmResult) => + emResult.getInputModel shouldBe emInput.getUuid + emResult.getTime shouldBe 300.toDateTime(simulationStartDate) + emResult.getP should equalWithTolerance(0d.asMegaWatt) + emResult.getQ should equalWithTolerance(0d.asMegaVar) + } + + // expect completion from EmAgent with new tick (800) instead of revoked tick (600) + scheduler.expectMessage( + Completion(emAgentActivation, Some(800)) + ) + } + + "handle ChangingFlexOptions indicator correctly" in { + val resultListener = TestProbe[ResultEvent]("ResultListener") + val scheduler = TestProbe[SchedulerMessage]("Scheduler") + + val emAgent = spawn( + EmAgent( + emInput, + modelConfig, + outputConfig, + "PRIORITIZED", + simulationStartDate, + parent = Left(scheduler.ref), + maybeRootEmConfig = None, + listener = Iterable(resultListener.ref), + ) + ) + + val pvAgent = TestProbe[FlexRequest]("PvAgent") + emAgent ! RegisterParticipant(pvInput.getUuid, pvAgent.ref, pvInput) + emAgent ! ScheduleFlexRequest(pvInput.getUuid, 0) + + val sa1 = scheduler.expectMessageType[ScheduleActivation] + sa1.tick shouldBe 0 + sa1.unlockKey shouldBe None + val emAgentActivation = sa1.actor + + val evcsAgent = TestProbe[FlexRequest]("EvcsAgent") + emAgent ! RegisterParticipant(evcsInput.getUuid, evcsAgent.ref, evcsInput) + emAgent ! ScheduleFlexRequest(evcsInput.getUuid, 0) + + // no additional scheduling message, since tick 0 has already been scheduled + scheduler.expectNoMessage() + + /* TICK 0 */ + emAgentActivation ! Activation(0) + + // expect flex requests + pvAgent.expectMessage(RequestFlexOptions(0)) + evcsAgent.expectMessage(RequestFlexOptions(0)) + + // send flex options + emAgent ! ProvideMinMaxFlexOptions( + pvInput.getUuid, + Kilowatts(-5d), + Kilowatts(-5d), + Kilowatts(0d), + ) + + pvAgent.expectNoMessage() + evcsAgent.expectNoMessage() + + emAgent ! ProvideMinMaxFlexOptions( + evcsInput.getUuid, + Kilowatts(2d), + Kilowatts(-11d), + Kilowatts(11d), + ) + + // receive flex control messages + pvAgent.expectMessage(IssueNoControl(0)) + + evcsAgent.expectMessageType[IssuePowerControl] match { + case IssuePowerControl(0, setPower) => + setPower should approximate(Kilowatts(5.0)) + } + + // send completions + emAgent ! FlexCtrlCompletion( + pvInput.getUuid, + ApparentPower(Kilowatts(-5d), Kilovars(-0.5d)), + requestAtTick = Some(300), + ) + + scheduler.expectNoMessage() + + emAgent ! FlexCtrlCompletion( + evcsInput.getUuid, + ApparentPower(Kilowatts(5d), Kilovars(0.1d)), + requestAtNextActivation = true, // sending ChangingFlexOptions indicator + requestAtTick = Some(600), + ) + + resultListener.expectMessageType[ParticipantResultEvent] match { + case ParticipantResultEvent(emResult: EmResult) => + emResult.getInputModel shouldBe emInput.getUuid + emResult.getTime shouldBe simulationStartDate + emResult.getP should equalWithTolerance(0d.asMegaWatt) + emResult.getQ should equalWithTolerance((-.0004d).asMegaVar) + } + + // expect completion from EmAgent + scheduler.expectMessage(Completion(emAgentActivation, Some(300))) + + /* TICK 300 */ + emAgentActivation ! Activation(300) + + // FLEX OPTIONS + + // expect activations and flex requests. + // pv is scheduled regularly and evcs at any next tick + // thus, we expect activations for both + pvAgent.expectMessage(RequestFlexOptions(300)) + + // send flex options again, now there's a cloud and thus less feed-in + emAgent ! ProvideMinMaxFlexOptions( + pvInput.getUuid, + Kilowatts(-3d), + Kilowatts(-3d), + Kilowatts(0d), + ) + + // expecting flex options request, since we asked for it last time + evcsAgent.expectMessage(RequestFlexOptions(300)) + + emAgent ! ProvideMinMaxFlexOptions( + evcsInput.getUuid, + Kilowatts(2d), + Kilowatts(-11d), + Kilowatts(11d), + ) + + // FLEX CONTROL + pvAgent.expectMessage(IssueNoControl(300)) + + emAgent ! FlexCtrlCompletion( + pvInput.getUuid, + ApparentPower(Kilowatts(-3d), Kilovars(-0.06d)), + ) + + evcsAgent.expectMessageType[IssuePowerControl] match { + case IssuePowerControl(300, setPower) => + setPower should approximate(Kilowatts(3.0)) + } + + scheduler.expectNoMessage() + + emAgent ! FlexCtrlCompletion( + evcsInput.getUuid, // revoking tick 600 + ApparentPower(Kilowatts(3d), Kilovars(0.06d)), + ) + + // expect correct results + resultListener.expectMessageType[ParticipantResultEvent] match { + case ParticipantResultEvent(emResult: EmResult) => + emResult.getInputModel shouldBe emInput.getUuid + emResult.getTime shouldBe 300.toDateTime(simulationStartDate) + emResult.getP should equalWithTolerance(0d.asMegaWatt) + emResult.getQ should equalWithTolerance(0d.asMegaVar) + } + + // no more activation, since evcs activation got revoked + scheduler.expectMessage(Completion(emAgentActivation)) + } + + } + + "An EM-controlled EM agent" should { + "be initialized correctly and run through some activations" in { + val resultListener = TestProbe[ResultEvent]("ResultListener") + + val parentEmAgent = TestProbe[FlexResponse]("ParentEmAgent") + + val emAgent = spawn( + EmAgent( + emInput, + modelConfig, + outputConfig, + "PRIORITIZED", + simulationStartDate, + parent = Right(parentEmAgent.ref), + maybeRootEmConfig = None, + listener = Iterable(resultListener.ref), + ) + ) + + val pvAgent = TestProbe[FlexRequest]("PvAgent") + emAgent ! RegisterParticipant(pvInput.getUuid, pvAgent.ref, pvInput) + emAgent ! ScheduleFlexRequest(pvInput.getUuid, 0) + + val emAgentFlex = + parentEmAgent.expectMessageType[RegisterParticipant] match { + case RegisterParticipant(modelUuid, participant, inputModel) => + modelUuid shouldBe emInput.getUuid + inputModel shouldBe emInput + participant + } + parentEmAgent.expectMessage(ScheduleFlexRequest(emInput.getUuid, 0)) + + val evcsAgent = TestProbe[FlexRequest]("EvcsAgent") + emAgent ! RegisterParticipant(evcsInput.getUuid, evcsAgent.ref, evcsInput) + emAgent ! ScheduleFlexRequest(evcsInput.getUuid, 0) + + // no additional scheduling message, since tick 0 has already been scheduled + parentEmAgent.expectNoMessage() + + /* TICK 0 */ + emAgentFlex ! RequestFlexOptions(0) + + // expect activations and flex requests + pvAgent.expectMessage(RequestFlexOptions(0)) + evcsAgent.expectMessage(RequestFlexOptions(0)) + + // send flex options + emAgent ! ProvideMinMaxFlexOptions( + pvInput.getUuid, + Kilowatts(-5d), + Kilowatts(-5d), + Kilowatts(0d), + ) + + pvAgent.expectNoMessage() + evcsAgent.expectNoMessage() + + emAgent ! ProvideMinMaxFlexOptions( + evcsInput.getUuid, + Kilowatts(2d), + Kilowatts(-11d), + Kilowatts(11d), + ) + + resultListener.expectMessageType[FlexOptionsResultEvent] match { + case FlexOptionsResultEvent(flexResult) => + flexResult.getInputModel shouldBe emInput.getUuid + flexResult.getTime shouldBe 0.toDateTime(simulationStartDate) + flexResult.getpRef() should equalWithTolerance(0d.asMegaWatt) + flexResult.getpMin() should equalWithTolerance((-0.016d).asMegaWatt) + flexResult.getpMax() should equalWithTolerance(0.006d.asMegaWatt) + } + + parentEmAgent.expectMessageType[ProvideFlexOptions] match { + case ProvideMinMaxFlexOptions( + modelUuid, + referencePower, + minPower, + maxPower, + ) => + modelUuid shouldBe emInput.getUuid + referencePower shouldBe Kilowatts(0d) + minPower shouldBe Kilowatts(-16d) + maxPower shouldBe Kilowatts(6d) // hint: PV is not flexible + } + + // issue power control and expect EmAgent to distribute it + // we want max power = 6 kW + emAgentFlex ! IssuePowerControl(0, Kilowatts(6d)) + + // expect issue power control + pvAgent.expectMessage(IssueNoControl(0)) + + emAgent ! FlexCtrlCompletion( + pvInput.getUuid, + ApparentPower(Kilowatts(-5), Kilovars(-0.5)), + requestAtTick = Some(600), + ) + + evcsAgent.expectMessageType[IssuePowerControl] match { + case IssuePowerControl(0, setPower) => + setPower should approximate(Kilowatts(11.0)) + } + + parentEmAgent.expectNoMessage() + + emAgent ! FlexCtrlCompletion( + evcsInput.getUuid, + ApparentPower(Kilowatts(11), Kilovars(1.1)), + requestAtTick = Some(300), + ) + + // expect correct results + resultListener.expectMessageType[ParticipantResultEvent] match { + case ParticipantResultEvent(emResult: EmResult) => + emResult.getInputModel shouldBe emInput.getUuid + emResult.getTime shouldBe 0.toDateTime(simulationStartDate) + emResult.getP should equalWithTolerance(0.006d.asMegaWatt) + emResult.getQ should equalWithTolerance(0.0006d.asMegaVar) + } + + parentEmAgent.expectMessageType[FlexCtrlCompletion] match { + case FlexCtrlCompletion( + modelUuid, + result, + requestAtNextActivation, + requestAtTick, + ) => + modelUuid shouldBe emInput.getUuid + result.p should approximate(Kilowatts(6)) + result.q should approximate(Kilovars(0.6)) + requestAtNextActivation shouldBe false + requestAtTick shouldBe Some(300) + } + + /* TICK 150 */ + // The mock parent EM now acts as if the situation changed before tick 300, + // so that the flex control changes before new flex option calculations are due + + // no control means reference power of the latest flex options = 0 kW + emAgentFlex ! IssueNoControl(150) + + // We already sent NoControl at last tick, so we're still at -5 kW + pvAgent.expectNoMessage() + + // We need 5 kW to compensate PV feed-in + evcsAgent.expectMessageType[IssuePowerControl] match { + case IssuePowerControl(150, setPower) => + setPower should approximate(Kilowatts(5.0)) + } + + parentEmAgent.expectNoMessage() + + emAgent ! FlexCtrlCompletion( + evcsInput.getUuid, + ApparentPower(Kilowatts(5.0), Kilovars(0.5)), + requestAtTick = Some(700), + ) + + // expect correct results + resultListener.expectMessageType[ParticipantResultEvent] match { + case ParticipantResultEvent(emResult: EmResult) => + emResult.getInputModel shouldBe emInput.getUuid + emResult.getTime shouldBe 150.toDateTime(simulationStartDate) + emResult.getP should equalWithTolerance(0d.asMegaWatt) + emResult.getQ should equalWithTolerance(0d.asMegaVar) + } + + parentEmAgent.expectMessageType[FlexCtrlCompletion] match { + case FlexCtrlCompletion( + modelUuid, + result, + requestAtNextActivation, + requestAtTick, + ) => + modelUuid shouldBe emInput.getUuid + result.p should approximate(Kilowatts(0)) + result.q should approximate(Kilovars(0)) + requestAtNextActivation shouldBe false + requestAtTick shouldBe Some(600) + } + + } + } + +} diff --git a/src/test/scala/edu/ie3/simona/event/NotifierSpec.scala b/src/test/scala/edu/ie3/simona/event/NotifierSpec.scala index 654e6ff8d4..83606317b7 100644 --- a/src/test/scala/edu/ie3/simona/event/NotifierSpec.scala +++ b/src/test/scala/edu/ie3/simona/event/NotifierSpec.scala @@ -89,6 +89,8 @@ class NotifierSpec classOf[StorageResult], Ev -> classOf[EvResult], + Em -> + classOf[EmResult], ) // TODO: Grid results are not covered, yet. diff --git a/src/test/scala/edu/ie3/simona/model/em/EmAggregateSelfOptSpec.scala b/src/test/scala/edu/ie3/simona/model/em/EmAggregateSelfOptSpec.scala new file mode 100644 index 0000000000..8ba0d1804e --- /dev/null +++ b/src/test/scala/edu/ie3/simona/model/em/EmAggregateSelfOptSpec.scala @@ -0,0 +1,171 @@ +/* + * © 2022. TU Dortmund University, + * Institute of Energy Systems, Energy Efficiency and Energy Economics, + * Research group Distribution grid planning and operation + */ + +package edu.ie3.simona.model.em + +import edu.ie3.datamodel.models.input.system.{PvInput, SystemParticipantInput} +import edu.ie3.simona.ontology.messages.flex.MinMaxFlexibilityMessage.ProvideMinMaxFlexOptions +import edu.ie3.simona.test.common.UnitSpec +import org.scalatestplus.mockito.MockitoSugar +import squants.energy.Kilowatts + +import java.util.UUID + +class EmAggregateSelfOptSpec extends UnitSpec with MockitoSugar { + + "The self-optimizing aggregating strategy with PV flex" should { + val strat = EmAggregateSelfOpt(pvFlex = true) + + "pick 0kW if possible" in { + val flexOptions1 = ProvideMinMaxFlexOptions( + modelUuid = UUID.randomUUID(), + ref = Kilowatts(2.0), + min = Kilowatts(-1.0), + max = Kilowatts(4.0), + ) + + val flexOptions2 = ProvideMinMaxFlexOptions( + modelUuid = UUID.randomUUID(), + ref = Kilowatts(-6.0), + min = Kilowatts(-6.0), + max = Kilowatts(0.0), + ) + + val actualResult = strat.aggregateFlexOptions( + Iterable( + (mock[SystemParticipantInput], flexOptions1), + (mock[SystemParticipantInput], flexOptions2), + ) + ) + + actualResult shouldBe ( + Kilowatts(0.0), + Kilowatts(-7.0), + Kilowatts(4.0) + ) + } + + "pick minSum if minSum > 0kW" in { + val flexOptions1 = ProvideMinMaxFlexOptions( + modelUuid = UUID.randomUUID(), + ref = Kilowatts(6.0), + min = Kilowatts(4.0), + max = Kilowatts(12.0), + ) + + val flexOptions2 = ProvideMinMaxFlexOptions( + modelUuid = UUID.randomUUID(), + ref = Kilowatts(2.0), + min = Kilowatts(-2.0), + max = Kilowatts(2.0), + ) + + val actualResult = strat.aggregateFlexOptions( + Iterable( + (mock[SystemParticipantInput], flexOptions1), + (mock[SystemParticipantInput], flexOptions2), + ) + ) + + actualResult shouldBe ( + Kilowatts(2.0), + Kilowatts(2.0), + Kilowatts(14.0) + ) + } + + "pick maxSum if maxSum < 0kW" in { + val flexOptions1 = ProvideMinMaxFlexOptions( + modelUuid = UUID.randomUUID(), + ref = Kilowatts(-1.0), + min = Kilowatts(-10.0), + max = Kilowatts(-1.0), + ) + + val flexOptions2 = ProvideMinMaxFlexOptions( + modelUuid = UUID.randomUUID(), + ref = Kilowatts(-6.0), + min = Kilowatts(-6.0), + max = Kilowatts(0.0), + ) + + val actualResult = strat.aggregateFlexOptions( + Iterable( + (mock[SystemParticipantInput], flexOptions1), + (mock[SystemParticipantInput], flexOptions2), + ) + ) + + actualResult shouldBe ( + Kilowatts(-1.0), + Kilowatts(-16.0), + Kilowatts(-1.0) + ) + } + } + + "The self-optimizing aggregating strategy without PV flex" should { + val strat = EmAggregateSelfOpt(pvFlex = false) + + "exclude PV max power when normally picking 0kW as target" in { + val flexOptions1 = ProvideMinMaxFlexOptions( + modelUuid = UUID.randomUUID(), + ref = Kilowatts(2.0), + min = Kilowatts(-1.0), + max = Kilowatts(4.0), + ) + + val flexOptions2 = ProvideMinMaxFlexOptions( + modelUuid = UUID.randomUUID(), + ref = Kilowatts(-6.0), + min = Kilowatts(-6.0), + max = Kilowatts(0.0), + ) + + val actualResult = strat.aggregateFlexOptions( + Iterable( + (mock[SystemParticipantInput], flexOptions1), + (mock[PvInput], flexOptions2), + ) + ) + + actualResult shouldBe ( + Kilowatts(-2.0), + Kilowatts(-7.0), + Kilowatts(4.0) + ) + } + + "exclude PV max power when normally picking maxSum as target" in { + val flexOptions1 = ProvideMinMaxFlexOptions( + modelUuid = UUID.randomUUID(), + ref = Kilowatts(-1.0), + min = Kilowatts(-10.0), + max = Kilowatts(-1.0), + ) + + val flexOptions2 = ProvideMinMaxFlexOptions( + modelUuid = UUID.randomUUID(), + ref = Kilowatts(-6.0), + min = Kilowatts(-6.0), + max = Kilowatts(0.0), + ) + + val actualResult = strat.aggregateFlexOptions( + Iterable( + (mock[SystemParticipantInput], flexOptions1), + (mock[PvInput], flexOptions2), + ) + ) + + actualResult shouldBe ( + Kilowatts(-7.0), + Kilowatts(-16.0), + Kilowatts(-1.0) + ) + } + } +} diff --git a/src/test/scala/edu/ie3/simona/model/em/EmAggregateSimpleSumSpec.scala b/src/test/scala/edu/ie3/simona/model/em/EmAggregateSimpleSumSpec.scala new file mode 100644 index 0000000000..e91f05d519 --- /dev/null +++ b/src/test/scala/edu/ie3/simona/model/em/EmAggregateSimpleSumSpec.scala @@ -0,0 +1,51 @@ +/* + * © 2024. TU Dortmund University, + * Institute of Energy Systems, Energy Efficiency and Energy Economics, + * Research group Distribution grid planning and operation + */ + +package edu.ie3.simona.model.em + +import edu.ie3.datamodel.models.input.system.SystemParticipantInput +import edu.ie3.simona.ontology.messages.flex.MinMaxFlexibilityMessage.ProvideMinMaxFlexOptions +import edu.ie3.simona.test.common.UnitSpec +import org.scalatestplus.mockito.MockitoSugar +import squants.energy.Kilowatts + +import java.util.UUID + +class EmAggregateSimpleSumSpec extends UnitSpec with MockitoSugar { + + "The simple sum aggregating strategy" should { + + "calculate ref, min and max power correctly" in { + val flexOptions1 = ProvideMinMaxFlexOptions( + modelUuid = UUID.randomUUID(), + ref = Kilowatts(6.0), + min = Kilowatts(4.0), + max = Kilowatts(12.0), + ) + + val flexOptions2 = ProvideMinMaxFlexOptions( + modelUuid = UUID.randomUUID(), + ref = Kilowatts(2.0), + min = Kilowatts(-2.0), + max = Kilowatts(2.0), + ) + + val actualResult = EmAggregateSimpleSum.aggregateFlexOptions( + Iterable( + (mock[SystemParticipantInput], flexOptions1), + (mock[SystemParticipantInput], flexOptions2), + ) + ) + + actualResult shouldBe ( + Kilowatts(8.0), + Kilowatts(2.0), + Kilowatts(14.0) + ) + } + + } +} diff --git a/src/test/scala/edu/ie3/simona/model/em/PrioritizedFlexStratSpec.scala b/src/test/scala/edu/ie3/simona/model/em/PrioritizedFlexStratSpec.scala new file mode 100644 index 0000000000..dc3bcd7ff5 --- /dev/null +++ b/src/test/scala/edu/ie3/simona/model/em/PrioritizedFlexStratSpec.scala @@ -0,0 +1,339 @@ +/* + * © 2022. TU Dortmund University, + * Institute of Energy Systems, Energy Efficiency and Energy Economics, + * Research group Distribution grid planning and operation + */ + +package edu.ie3.simona.model.em + +import edu.ie3.datamodel.models.input.system.{ + EvcsInput, + LoadInput, + PvInput, + StorageInput, +} +import edu.ie3.simona.ontology.messages.flex.MinMaxFlexibilityMessage.ProvideMinMaxFlexOptions +import edu.ie3.simona.test.common.UnitSpec +import edu.ie3.simona.test.helper.TableDrivenHelper +import org.mockito.Mockito.when +import org.scalatest.prop.TableDrivenPropertyChecks +import org.scalatestplus.mockito.MockitoSugar +import squants.energy.Kilowatts + +import java.util.UUID + +class PrioritizedFlexStratSpec + extends UnitSpec + with TableDrivenPropertyChecks + with TableDrivenHelper + with MockitoSugar { + + "The prioritized flex model" should { + + val load = UUID.fromString("0-0-0-0-1") + val loadInputModel = mock[LoadInput] + when(loadInputModel.getUuid).thenReturn(load) + + val pv = UUID.fromString("0-0-0-0-2") + val pvInputModel = mock[PvInput] + when(pvInputModel.getUuid).thenReturn(pv) + + val ev = UUID.fromString("0-0-0-0-3") + val evcsInputModel = mock[EvcsInput] + when(evcsInputModel.getUuid).thenReturn(ev) + + val st = UUID.fromString("0-0-0-0-4") + val storageInputModel = mock[StorageInput] + when(storageInputModel.getUuid).thenReturn(st) + + "determine flex control dependent on flex options" in { + val strat = PrioritizedFlexStrat(pvFlex = false) + + val cases = Table( + ( + "target", + "loadPower", + "pvPower", + "evcsSuggested", + "evcsMin", + "evcsMax", + "storageMin", + "storageMax", + "expectedResult", + ), + + /* excess feed-in */ + // excess is fully covered by parts of ev flexibility + (0d, 0d, -5d, 2d, -11d, 11d, -2d, 2d, L((ev, 5d))), + // excess is fully covered by maximum ev flexibility + (0d, 0d, -11d, 2d, -11d, 11d, -2d, 2d, L((ev, 11d))), + // excess is fully covered by max ev and parts of storage flex + (0d, 1d, -13d, 2d, -11d, 11d, -2d, 2d, L((ev, 11d), (st, 1d))), + // excess is fully covered by max ev and max storage flex + (0d, 0d, -14d, 2d, -11d, 11d, -2d, 2d, L((ev, 11d), (st, 2d))), + // excess is fully covered by max storage flex + (0d, 0d, -4d, 2d, 0d, 2d, -2d, 2d, L((st, 2d))), + // excess is partly covered by max ev and max storage flex, -2kW remains + (0d, 0d, -15d, 2d, -11d, 11d, -2d, 2d, L((ev, 11d), (st, 2d))), + // excess is partly covered by max storage flex, -7kW remains + (0d, 0d, -14d, 2d, 0d, 2d, -5d, 5d, L((st, 5d))), + // excess can't be covered because there is no flexibility + (0d, 0d, -5d, 2d, 2d, 2d, 0d, 0d, Seq.empty), + + /* excess load */ + // excess is fully covered by parts of storage flex + (0d, 5d, -1d, 0d, -11d, 11d, -5d, 5d, L((st, -4d))), + // excess is fully covered by min storage flex + (0d, 6d, -1d, 0d, -11d, 11d, -5d, 5d, L((st, -5d))), + // excess is fully covered by min storage and parts of ev flex, charging power reduced + (0d, 5d, -1d, 3d, -11d, 11d, -5d, 5d, L((st, -5d), (ev, 1d))), + // excess is fully covered by min storage and parts of ev flex, vehicle-to-home + (0d, 5d, -1d, 5d, -11d, 11d, -2d, 2d, L((st, -2d), (ev, -2d))), + // excess is fully covered by min storage and min ev flex + (0d, 14d, -1d, 5d, -11d, 11d, -2d, 2d, L((st, -2d), (ev, -11d))), + // excess is fully covered by min ev flex + (0d, 12d, -1d, 2d, -11d, 11d, 0d, 0d, L((ev, -11d))), + // excess is partly covered by min ev and min storage flex, 1kW remains + (0d, 15d, -1d, 2d, -11d, 11d, -2d, 2d, L((st, -2d), (ev, -11d))), + // excess is partly covered by min ev flex, 2kW remains + (0d, 14d, -1d, 4d, -11d, 11d, 0d, 0d, L((ev, -11d))), + // excess can't be covered because there is no flexibility + (0d, 5d, 0d, 2d, 2d, 2d, 0d, 0d, Seq.empty), + + /* target unequal to zero */ + // excess feed-in is fully covered by parts of ev flexibility + (2d, 0d, -5d, 2d, -11d, 11d, -2d, 2d, L((ev, 7d))), + // no excess + (-3d, 0d, -5d, 2d, -11d, 11d, -2d, 2d, L.empty), + // excess is fully covered by min storage and parts of ev flex, vehicle-to-home + (-3d, 5d, -1d, 5d, -11d, 11d, -2d, 2d, L((st, -2d), (ev, -5d))), + ) + + forAll(cases) { + ( + target, + loadPower, + pvPower, + evcsSuggested, + evcsMin, + evcsMax, + storageMin, + storageMax, + expectedResult, + ) => + val flexOptions = Seq( + ( + loadInputModel, + ProvideMinMaxFlexOptions( + load, + Kilowatts(loadPower), + Kilowatts(loadPower), + Kilowatts(loadPower), + ), + ), + ( + pvInputModel, + ProvideMinMaxFlexOptions( + pv, + Kilowatts(pvPower), + Kilowatts(pvPower), + Kilowatts(0d), + ), + ), + ( + evcsInputModel, + ProvideMinMaxFlexOptions( + ev, + Kilowatts(evcsSuggested), + Kilowatts(evcsMin), + Kilowatts(evcsMax), + ), + ), + ( + storageInputModel, + ProvideMinMaxFlexOptions( + st, + Kilowatts(0d), + Kilowatts(storageMin), + Kilowatts(storageMax), + ), + ), + ) + + val actualResults = + strat.determineFlexControl( + flexOptions, + Kilowatts(target), + ) + + actualResults should have size expectedResult.size withClue + s"actual results are $actualResults while $expectedResult was expected." + + val expectedResultMap = expectedResult.toMap + actualResults.foreach { case (uuid, power) => + val expectedRes: Double = expectedResultMap.getOrElse( + uuid, + fail( + s"Actual control message $power for model $uuid is not part of the expected" + ), + ) + + power.toKilowatts should ===(expectedRes +- 1e-6d) + } + } + } + + "determine flex control dependent on flex options with PV flex enabled" in { + val strat = PrioritizedFlexStrat(pvFlex = true) + + val cases = Table( + ( + "loadPower", + "pvPower", + "evcsSuggested", + "evcsMin", + "evcsMax", + "storageMin", + "storageMax", + "expectedResult", + ), + + /* excess feed-in */ + // excess is fully covered by parts of ev flexibility + (0d, -5d, 2d, -11d, 11d, -2d, 2d, L((ev, 5d))), + // excess is fully covered by maximum ev flexibility + (0d, -11d, 2d, -11d, 11d, -2d, 2d, L((ev, 11d))), + // excess is fully covered by max ev and parts of storage flex + (1d, -13d, 2d, -11d, 11d, -2d, 2d, L((ev, 11d), (st, 1d))), + // excess is fully covered by max ev and max storage flex, use pv 1kW flex + (0d, -14d, 2d, -11d, 11d, -2d, 2d, L((ev, 11d), (st, 2d), (pv, -13d))), + // excess is fully covered by max storage flex + (0d, -4d, 2d, 0d, 2d, -2d, 2d, L((st, 2d))), + // excess is partly covered by max ev and max storage flex, use pv 2kW flex + (0d, -15d, 2d, -11d, 11d, -2d, 2d, L((ev, 11d), (st, 2d), (pv, -13d))), + // excess is partly covered by max storage flex, use 7kW pv flex + (0d, -14d, 2d, 0d, 2d, -5d, 5d, L((st, 5d), (pv, -7d))), + // excess can't be covered without using pv flex + (0d, -5d, 2d, 2d, 2d, 0d, 0d, L((pv, -2d))), + + /* excess load, works same as above */ + // excess is fully covered by parts of storage flex + (5d, -1d, 0d, -11d, 11d, -5d, 5d, L((st, -4d))), + // excess is fully covered by min storage flex + (6d, -1d, 0d, -11d, 11d, -5d, 5d, L((st, -5d))), + ) + + forAll(cases) { + ( + loadPower, + pvPower, + evcsSuggested, + evcsMin, + evcsMax, + storageMin, + storageMax, + expectedResult, + ) => + val flexOptions = Seq( + ( + loadInputModel, + ProvideMinMaxFlexOptions( + load, + Kilowatts(loadPower), + Kilowatts(loadPower), + Kilowatts(loadPower), + ), + ), + ( + pvInputModel, + ProvideMinMaxFlexOptions( + pv, + Kilowatts(pvPower), + Kilowatts(pvPower), + Kilowatts(0d), + ), + ), + ( + evcsInputModel, + ProvideMinMaxFlexOptions( + ev, + Kilowatts(evcsSuggested), + Kilowatts(evcsMin), + Kilowatts(evcsMax), + ), + ), + ( + storageInputModel, + ProvideMinMaxFlexOptions( + st, + Kilowatts(0d), + Kilowatts(storageMin), + Kilowatts(storageMax), + ), + ), + ) + + val actualResults = + strat.determineFlexControl( + flexOptions, + Kilowatts(0d), + ) + + actualResults should have size expectedResult.size withClue + s"actual results are $actualResults while $expectedResult was expected." + + val expectedResultMap = expectedResult.toMap + actualResults.foreach { case (uuid, power) => + val expectedRes: Double = expectedResultMap.getOrElse( + uuid, + fail( + s"Actual control message $power for model $uuid is not part of the expected" + ), + ) + + power.toKilowatts should ===(expectedRes +- 1e-6d) + } + } + } + + "adapt flex options correctly" in { + + // flex options should be changed if corresponding + // agent is not controlled by this strategy + val cases = Table( + ("pvFlex", "inputModel", "expectedAdaptation"), + (false, loadInputModel, true), + (false, pvInputModel, true), + (false, evcsInputModel, false), + (false, storageInputModel, false), + (true, loadInputModel, true), + (true, pvInputModel, false), + (true, evcsInputModel, false), + (true, storageInputModel, false), + ) + + forAll(cases) { case (pvFlex, inputModel, expectedAdaptation) => + val flexOptionsIn = ProvideMinMaxFlexOptions( + inputModel.getUuid, + Kilowatts(1), + Kilowatts(-1), + Kilowatts(2), + ) + + val flexOptionsOut = PrioritizedFlexStrat(pvFlex) + .adaptFlexOptions(inputModel, flexOptionsIn) + + if (expectedAdaptation) { + flexOptionsOut shouldBe ProvideMinMaxFlexOptions + .noFlexOption( + inputModel.getUuid, + Kilowatts(1), + ) + } else { + flexOptionsOut shouldBe flexOptionsIn + } + } + } + + } +} diff --git a/src/test/scala/edu/ie3/simona/model/em/ProportionalFlexStratSpec.scala b/src/test/scala/edu/ie3/simona/model/em/ProportionalFlexStratSpec.scala new file mode 100644 index 0000000000..cec7c61d2a --- /dev/null +++ b/src/test/scala/edu/ie3/simona/model/em/ProportionalFlexStratSpec.scala @@ -0,0 +1,157 @@ +/* + * © 2022. TU Dortmund University, + * Institute of Energy Systems, Energy Efficiency and Energy Economics, + * Research group Distribution grid planning and operation + */ + +package edu.ie3.simona.model.em + +import edu.ie3.datamodel.models.input.AssetInput +import edu.ie3.simona.ontology.messages.flex.MinMaxFlexibilityMessage.ProvideMinMaxFlexOptions +import edu.ie3.simona.test.common.UnitSpec +import edu.ie3.simona.test.helper.TableDrivenHelper +import org.scalatest.prop.TableDrivenPropertyChecks +import org.scalatestplus.mockito.MockitoSugar +import squants.Power +import squants.energy.{Kilowatts, Watts} + +import java.util.UUID + +class ProportionalFlexStratSpec + extends UnitSpec + with TableDrivenPropertyChecks + with TableDrivenHelper + with MockitoSugar { + + private implicit val powerTolerance: Power = Watts(0.1) + + "The proportional flex model" should { + + "determine flex control dependent on flex options" in { + + val assetInput = mock[AssetInput] // is not used + + val cases = Table( + ( + "target", + "ref1", + "min1", + "max1", + "ref2", + "min2", + "max2", + "expected1", + "expected2", + ), + + /* target equal to reference sum */ + (4d, 2d, 0d, 3d, 2d, 0d, 4d, N, N), + (2d, 1d, 0d, 1d, 1d, 0d, 4d, N, N), + (2d, 1d, 0d, 1d, 1d, 0d, 1d, N, N), + (-5d, -2d, -4d, 0d, -3d, -5d, 1d, N, N), + (-2d, -1d, -1d, 1d, -1d, -2d, 0d, N, N), + (-2d, -1d, -1d, 0d, -1d, -1d, 0d, N, N), + + /* target higher than reference sum */ + // target lower than max sum + (5d, 1d, 0d, 2d, 0d, 0d, 4d, S(1.8d), S(3.2d)), + (3d, 1d, 0d, 2d, 0d, 0d, 4d, S(1.4d), S(1.6d)), + (2d, -1d, -1d, 2d, -1d, -1d, 4d, S(0.5d), S(1.5d)), + (4d, 2d, 0d, 2d, 0d, 0d, 4d, N, S(2d)), + (4d, 0d, 0d, 4d, 2d, 0d, 2d, S(2d), N), + // target higher than max sum + (7d, 1d, 0d, 2d, 0d, 0d, 4d, S(2d), S(4d)), + (7d, 2d, 0d, 2d, 0d, 0d, 4d, N, S(4d)), + (7d, 0d, 0d, 4d, 2d, 0d, 2d, S(4d), N), + + /* target lower than reference sum */ + // target higher than min sum + (-1d, 1d, -1d, 2d, 0d, -2d, 0d, S(0d), S(-1d)), + (-2d, 1d, -1d, 2d, 0d, -2d, 0d, S(-0.5d), S(-1.5d)), + (-4d, -1d, -2d, 2d, 1d, -3d, 1d, S(-1.8d), S(-2.2d)), + (-4d, -2d, -2d, 0d, 0d, -4d, 0d, N, S(-2d)), + (-4d, 0d, -4d, 0d, -2d, -2d, 0d, S(-2d), N), + // target lower than min sum + (-7d, -1d, -2d, 0d, 0d, -4d, 0d, S(-2d), S(-4d)), + (-7d, -2d, -2d, 0d, 0d, -4d, 0d, N, S(-4d)), + (-7d, 0d, -4d, 0d, -2d, -2d, 0d, S(-4d), N), + ) + + forAll(cases) { + ( + target, + ref1, + min1, + max1, + ref2, + min2, + max2, + expected1, + expected2, + ) => + val flexOptions1 = ProvideMinMaxFlexOptions( + modelUuid = UUID.randomUUID(), + ref = Kilowatts(ref1), + min = Kilowatts(min1), + max = Kilowatts(max1), + ) + + val flexOptions2 = ProvideMinMaxFlexOptions( + modelUuid = UUID.randomUUID(), + ref = Kilowatts(ref2), + min = Kilowatts(min2), + max = Kilowatts(max2), + ) + + val actualResults = ProportionalFlexStrat + .determineFlexControl( + Seq( + (assetInput, flexOptions1), + (assetInput, flexOptions2), + ), + Kilowatts(target), + ) + .toMap + + actualResults should have size Seq(expected1, expected2).flatten.size + + expected1.foreach { exp1 => + val res1 = actualResults.getOrElse( + flexOptions1.modelUuid, + fail( + "Results should include a set point for device 1, but doesn't" + ), + ) + (res1 ~= Kilowatts(exp1)) shouldBe true + } + + expected2.foreach { exp2 => + val res2 = actualResults.getOrElse( + flexOptions2.modelUuid, + fail( + "Results should include a set point for device 2, but doesn't" + ), + ) + (res2 ~= Kilowatts(exp2)) shouldBe true + } + + } + } + + "adapt flex options correctly" in { + val assetInput = mock[AssetInput] // is not used + + val flexOptionsIn = ProvideMinMaxFlexOptions( + UUID.randomUUID(), + Kilowatts(1), + Kilowatts(-1), + Kilowatts(2), + ) + + val flexOptionsOut = + ProportionalFlexStrat.adaptFlexOptions(assetInput, flexOptionsIn) + + flexOptionsOut shouldBe flexOptionsIn + } + } +} 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 f8d42fc3ef..26cb038257 100644 --- a/src/test/scala/edu/ie3/simona/test/common/ConfigTestData.scala +++ b/src/test/scala/edu/ie3/simona/test/common/ConfigTestData.scala @@ -121,6 +121,15 @@ trait ConfigTestData { | individualConfigs = [] |} | + |simona.runtime.participant.em = { + | defaultConfig = { + | calculateMissingReactivePowerWithModel = false + | uuids = ["default"] + | scaling = 1.0 + | } + | individualConfigs = [] + |} + | |simona.powerflow.maxSweepPowerDeviation = 1E-5 // the maximum allowed deviation in power between two sweeps, before overall convergence is assumed |simona.powerflow.skipOnFailure = true |simona.powerflow.newtonraphson.epsilon = [1E-12] diff --git a/src/test/scala/edu/ie3/simona/test/common/DefaultTestData.scala b/src/test/scala/edu/ie3/simona/test/common/DefaultTestData.scala index f989f9082b..4073df6cbd 100644 --- a/src/test/scala/edu/ie3/simona/test/common/DefaultTestData.scala +++ b/src/test/scala/edu/ie3/simona/test/common/DefaultTestData.scala @@ -203,6 +203,15 @@ trait DefaultTestData { | ] |} | + |simona.runtime.participant.em = { + | defaultConfig = { + | calculateMissingReactivePowerWithModel = false + | uuids = ["default"] + | scaling = 1.0 + | } + | individualConfigs = [] + |} + | |simona.powerflow.maxSweepPowerDeviation = 1E-5 // the maximum allowed deviation in power between two sweeps, before overall convergence is assumed |simona.powerflow.skipOnFailure = true |simona.powerflow.newtonraphson.epsilon = [1E-12] diff --git a/src/test/scala/edu/ie3/simona/test/common/input/EmInputTestData.scala b/src/test/scala/edu/ie3/simona/test/common/input/EmInputTestData.scala new file mode 100644 index 0000000000..e323085ece --- /dev/null +++ b/src/test/scala/edu/ie3/simona/test/common/input/EmInputTestData.scala @@ -0,0 +1,163 @@ +/* + * © 2022. TU Dortmund University, + * Institute of Energy Systems, Energy Efficiency and Energy Economics, + * Research group Distribution grid planning and operation + */ + +package edu.ie3.simona.test.common.input + +import edu.ie3.datamodel.models.input.OperatorInput +import edu.ie3.datamodel.models.input.container.ThermalGrid +import edu.ie3.datamodel.models.input.system.`type`.{ + HpTypeInput, + StorageTypeInput, +} +import edu.ie3.datamodel.models.input.system.`type`.chargingpoint.ChargingPointTypeUtils +import edu.ie3.datamodel.models.input.system.`type`.evcslocation.EvcsLocationType +import edu.ie3.datamodel.models.input.system.characteristic.{ + CosPhiFixed, + ReactivePowerCharacteristic, +} +import edu.ie3.datamodel.models.input.system.{ + EmInput, + EvcsInput, + HpInput, + StorageInput, +} +import edu.ie3.datamodel.models.input.thermal.{ + ThermalHouseInput, + ThermalStorageInput, +} +import edu.ie3.datamodel.models.{ControlStrategy, OperationTime, StandardUnits} +import edu.ie3.simona.config.SimonaConfig +import edu.ie3.simona.event.notifier.NotifierConfig +import edu.ie3.simona.model.participant.load.{LoadModelBehaviour, LoadReference} +import edu.ie3.simona.test.common.model.participant.HpTestData +import edu.ie3.simona.util.ConfigUtil +import edu.ie3.util.quantities.PowerSystemUnits._ +import squants.energy.Kilowatts +import tech.units.indriya.quantity.Quantities +import tech.units.indriya.unit.Units._ + +import java.util.UUID +import scala.jdk.CollectionConverters.SeqHasAsJava + +trait EmInputTestData + extends NodeInputTestData + with PvInputTestData + with LoadInputTestData + with HpTestData { + + protected val evcsInput = new EvcsInput( + UUID.randomUUID(), + "Dummy_EvcsModel", + new OperatorInput(UUID.randomUUID(), "NO_OPERATOR"), + OperationTime.notLimited(), + nodeInputNoSlackNs04KvA, + CosPhiFixed.CONSTANT_CHARACTERISTIC, + ChargingPointTypeUtils.ChargingStationType2, + 2, + 0.95, + EvcsLocationType.HOME, + true, + ) + + protected val householdStorageTypeInput = new StorageTypeInput( + UUID.randomUUID(), + "Dummy_Household_StorageTypeInput", + Quantities.getQuantity(100d, EURO), + Quantities.getQuantity(101d, EURO_PER_MEGAWATTHOUR), + Quantities.getQuantity(15d, KILOWATTHOUR), + Quantities.getQuantity(5d, KILOVOLTAMPERE), + 0.997, + Quantities.getQuantity(5d, KILOWATT), + Quantities.getQuantity(0.03, PU_PER_HOUR), + Quantities.getQuantity(0.95, PU), + Quantities.getQuantity(20d, PERCENT), + Quantities.getQuantity(50000d, HOUR), + 100000, + ) + + protected val householdStorageInput = new StorageInput( + UUID.randomUUID(), + "Dummy_Household_StorageInput", + new OperatorInput(UUID.randomUUID(), "NO_OPERATOR"), + OperationTime.notLimited(), + nodeInputNoSlackNs04KvA, + CosPhiFixed.CONSTANT_CHARACTERISTIC, + householdStorageTypeInput, + ) + + protected val emInput = new EmInput( + UUID.randomUUID(), + "Dummy_EmModel", + new OperatorInput(UUID.randomUUID(), "NO_OPERATOR"), + OperationTime.notLimited(), + nodeInputNoSlackNs04KvA, + CosPhiFixed.CONSTANT_CHARACTERISTIC, + Array.empty, + ControlStrategy.DefaultControlStrategies.NO_CONTROL_STRATEGY, // FIXME adapt once available + ) + + protected val simonaConfig: SimonaConfig = + createSimonaConfig( + LoadModelBehaviour.FIX, + LoadReference.ActivePower( + Kilowatts(0.0) + ), + ) + + private val configUtil = ConfigUtil.ParticipantConfigUtil( + simonaConfig.simona.runtime.participant + ) + + protected val defaultOutputConfig: NotifierConfig = + NotifierConfig( + simonaConfig.simona.output.participant.defaultConfig.simulationResult, + simonaConfig.simona.output.participant.defaultConfig.powerRequestReply, + simonaConfig.simona.output.participant.defaultConfig.flexResult, + ) + + protected val modelConfig: SimonaConfig.EmRuntimeConfig = + configUtil.getOrDefault[SimonaConfig.EmRuntimeConfig]( + emInput.getUuid + ) + + protected val adaptedTypeInput = new HpTypeInput( + UUID.fromString("9802bf35-2a4e-4ff5-be9b-cd9e6a78dcd6"), + "hp type", + Quantities.getQuantity(0.0, StandardUnits.CAPEX), + Quantities.getQuantity(0.0, StandardUnits.ENERGY_PRICE), + Quantities.getQuantity(5.0, StandardUnits.ACTIVE_POWER_IN), + 0.97, + Quantities.getQuantity(7.5, StandardUnits.ACTIVE_POWER_IN), + ) + + protected val adaptedHpInputModel = new HpInput( + UUID.fromString("7832dea4-8703-4b37-8752-e67b86e957df"), + "test hp", + OperatorInput.NO_OPERATOR_ASSIGNED, + OperationTime.notLimited(), + nodeInput, + thermalBusInput, + ReactivePowerCharacteristic.parse("cosPhiFixed:{(0.00,0.98)}"), + adaptedTypeInput, + ) + + /* Set inner temperature of house a bit lower */ + val adaptedThermalHouse = new ThermalHouseInput( + UUID.fromString("91940626-bdd0-41cf-96dd-47c94c86b20e"), + "thermal house", + thermalBusInput, + Quantities.getQuantity(0.325, StandardUnits.THERMAL_TRANSMISSION), + Quantities.getQuantity(75, StandardUnits.HEAT_CAPACITY), + Quantities.getQuantity(20.3, StandardUnits.TEMPERATURE), + Quantities.getQuantity(22.0, StandardUnits.TEMPERATURE), + Quantities.getQuantity(20.0, StandardUnits.TEMPERATURE), + ) + val adaptedThermalGrid = new ThermalGrid( + thermalBusInput, + Seq(adaptedThermalHouse).asJava, + Seq.empty[ThermalStorageInput].asJava, + ) +} diff --git a/src/test/scala/edu/ie3/simona/util/ConfigUtilSpec.scala b/src/test/scala/edu/ie3/simona/util/ConfigUtilSpec.scala index c04877efeb..92d295097a 100644 --- a/src/test/scala/edu/ie3/simona/util/ConfigUtilSpec.scala +++ b/src/test/scala/edu/ie3/simona/util/ConfigUtilSpec.scala @@ -58,7 +58,7 @@ class ConfigUtilSpec inside(actual) { case ParticipantConfigUtil(configs, defaultConfigs) => configs shouldBe Map.empty[UUID, SimonaConfig.LoadRuntimeConfig] - defaultConfigs.size shouldBe 6 + defaultConfigs.size shouldBe 8 inside(defaultConfigs.get(classOf[LoadRuntimeConfig])) { case Some( @@ -118,7 +118,7 @@ class ConfigUtilSpec UUID.fromString("49f250fa-41ff-4434-a083-79c98d260a76") ) - defaultConfigs.size shouldBe 6 + defaultConfigs.size shouldBe 8 inside(defaultConfigs.get(classOf[LoadRuntimeConfig])) { case Some( LoadRuntimeConfig( @@ -331,7 +331,7 @@ class ConfigUtilSpec UUID.fromString("49f250fa-41ff-4434-a083-79c98d260a76") ) - defaultConfigs.size shouldBe 6 + defaultConfigs.size shouldBe 8 inside(defaultConfigs.get(classOf[FixedFeedInRuntimeConfig])) { case Some( FixedFeedInRuntimeConfig( @@ -723,9 +723,11 @@ class ConfigUtilSpec val configUtil = OutputConfigUtil(inputConfig) val expectedResult: Set[Value] = NotifierIdentifier.values -- Vector( NotifierIdentifier.PvPlant - ) + ) -- NotifierIdentifier.getThermalIdentifiers.toVector - configUtil.simulationResultIdentifiersToConsider shouldBe expectedResult + configUtil.simulationResultIdentifiersToConsider( + false + ) shouldBe expectedResult } "return the correct notifier identifiers when the default is to NOT inform about new simulation results" in { @@ -761,7 +763,9 @@ class ConfigUtilSpec val expectedResult: Set[Value] = Set(NotifierIdentifier.Load, NotifierIdentifier.ChpPlant) - configUtil.simulationResultIdentifiersToConsider shouldBe expectedResult + configUtil.simulationResultIdentifiersToConsider( + false + ) shouldBe expectedResult } "return the correct result entity classes to be considered " in { @@ -797,7 +801,9 @@ class ConfigUtilSpec val expectedResult: Set[Class[_ <: ResultEntity]] = Set[Class[_ <: ResultEntity]](classOf[LoadResult], classOf[ChpResult]) - configUtil.simulationResultEntitiesToConsider shouldBe expectedResult + configUtil.simulationResultEntitiesToConsider( + false + ) shouldBe expectedResult } } diff --git a/src/test/scala/edu/ie3/util/scala/io/FlexSignalFromExcelSpec.scala b/src/test/scala/edu/ie3/util/scala/io/FlexSignalFromExcelSpec.scala new file mode 100644 index 0000000000..15c881cbdb --- /dev/null +++ b/src/test/scala/edu/ie3/util/scala/io/FlexSignalFromExcelSpec.scala @@ -0,0 +1,48 @@ +/* + * © 2022. TU Dortmund University, + * Institute of Energy Systems, Energy Efficiency and Energy Economics, + * Research group Distribution grid planning and operation + */ + +package edu.ie3.util.scala.io + +import edu.ie3.simona.test.common.UnitSpec +import edu.ie3.util.scala.io.FlexSignalFromExcel.TimeSeriesType +import org.scalatest.prop.TableDrivenPropertyChecks + +import scala.util.{Failure, Success} + +class FlexSignalFromExcelSpec extends UnitSpec with TableDrivenPropertyChecks { + "Reading flexibility signal from file" should { + val file = getClass.getResource("flexSignal.xlsx").getFile + + val nodeIds = Seq("node0", "node1") + val combinations = nodeIds.flatMap { nodeId => + TimeSeriesType.values.map { seriesType => (nodeId, seriesType) } + } + + val cases = Table( + ("nodeId", "timeSeriesType"), + combinations: _* + ) + + "provide all expected time series with correct amount of entries" in { + + forAll(cases) { case (nodeId, timeSeriesType) => + val resultTrial = + FlexSignalFromExcel.flexSignals(file, nodeId, timeSeriesType) + + resultTrial match { + case Success(timeSeries) => + timeSeries.getEntries.size() shouldBe 8 + case Failure(exception) => + fail( + "Reading flex signal from excel file failed with the following exception.", + exception, + ) + } + } + } + + } +} From 772c7833fd3b584f17885fc47a68d1148140edb7 Mon Sep 17 00:00:00 2001 From: Sebastian Peter Date: Fri, 23 Feb 2024 08:34:05 +0100 Subject: [PATCH 02/32] Removing storage stuff in GridAgentController for now --- .../agent/grid/GridAgentController.scala | 70 ------------------- 1 file changed, 70 deletions(-) diff --git a/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala b/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala index 76ac7796bf..e0d96af8b6 100644 --- a/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala +++ b/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala @@ -32,7 +32,6 @@ import edu.ie3.simona.agent.participant.hp.HpAgent import edu.ie3.simona.agent.participant.load.LoadAgent import edu.ie3.simona.agent.participant.pv.PvAgent import edu.ie3.simona.agent.participant.statedata.ParticipantStateData.ParticipantInitializeStateData -import edu.ie3.simona.agent.participant.storage.StorageAgent import edu.ie3.simona.agent.participant.wec.WecAgent import edu.ie3.simona.config.SimonaConfig import edu.ie3.simona.config.SimonaConfig._ @@ -453,20 +452,6 @@ class GridAgentController( s"Unable to find thermal island grid for heat pump '${hpInput.getUuid}' with thermal bus '${hpInput.getThermalBus.getUuid}'." ) } - case input: StorageInput => - buildStorage( - input, - participantConfigUtil.getOrDefault[StorageRuntimeConfig]( - input.getUuid - ), - environmentRefs.primaryServiceProxy, - simulationStartDate, - simulationEndDate, - resolution, - requestVoltageDeviationThreshold, - outputConfigUtil.getOrDefault(NotifierIdentifier.Storage), - maybeParentEm, - ) case input: SystemParticipantInput => throw new NotImplementedError( s"Building ${input.getClass.getSimpleName} is not implemented, yet." @@ -816,61 +801,6 @@ class GridAgentController( wecInput.getId, ) - /** Creates a storage agent and determines the needed additional information - * for later initialization of the agent. - * - * @param storageInput - * Storage input model to derive information from - * @param modelConfiguration - * User-provided configuration for this specific storage model - * @param primaryServiceProxy - * Reference to the primary data service proxy - * @param simulationStartDate - * First wall clock time in simulation - * @param simulationEndDate - * Last wall clock time in simulation - * @param resolution - * Frequency of power flow calculations - * @param requestVoltageDeviationThreshold - * Maximum deviation in p.u. of request voltages to be considered equal - * @param outputConfig - * Configuration of the output behavior - * @param maybeParentEm - * The parent EmAgent, if applicable - * @return - * The [[StorageAgent]] 's [[ClassicActorRef]] - */ - private def buildStorage( - storageInput: StorageInput, - modelConfiguration: SimonaConfig.StorageRuntimeConfig, - primaryServiceProxy: ClassicActorRef, - simulationStartDate: ZonedDateTime, - simulationEndDate: ZonedDateTime, - resolution: Long, - requestVoltageDeviationThreshold: Double, - outputConfig: NotifierConfig, - maybeParentEm: Option[ActorRef[FlexResponse]] = None, - ): ClassicActorRef = - gridAgentContext.simonaActorOf( - StorageAgent.props( - environmentRefs.scheduler, - ParticipantInitializeStateData( - storageInput, - modelConfiguration, - primaryServiceProxy, - None, - simulationStartDate, - simulationEndDate, - resolution, - requestVoltageDeviationThreshold, - outputConfig, - maybeParentEm, - ), - listener, - ), - storageInput.getId, - ) - /** Builds an [[EmAgent]] from given input * * @param emInput From ac9edb8b16d00cbec8504b5792dd0700355dd592 Mon Sep 17 00:00:00 2001 From: Sebastian Peter Date: Fri, 23 Feb 2024 08:41:16 +0100 Subject: [PATCH 03/32] Removing test involving storages for now --- .../edu/ie3/simona/agent/em/EmAgentIT.scala | 287 ------------------ 1 file changed, 287 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 d9164cf875..79af306e45 100644 --- a/src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala +++ b/src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala @@ -12,7 +12,6 @@ import edu.ie3.simona.agent.participant.hp.HpAgent import edu.ie3.simona.agent.participant.load.LoadAgent.FixedLoadAgent import edu.ie3.simona.agent.participant.pv.PvAgent import edu.ie3.simona.agent.participant.statedata.ParticipantStateData.ParticipantInitializeStateData -import edu.ie3.simona.agent.participant.storage.StorageAgent import edu.ie3.simona.config.SimonaConfig._ import edu.ie3.simona.event.ResultEvent import edu.ie3.simona.event.ResultEvent.ParticipantResultEvent @@ -97,292 +96,6 @@ class EmAgentIT private implicit val classicSystem: ActorSystem = system.toClassic "An em agent" when { - "having load, pv and storage agents connected" should { - "be initialized correctly and run through some activations" in { - val resultListener = TestProbe[ResultEvent]("ResultListener") - val primaryServiceProxy = - TestProbe[ServiceMessage]("PrimaryServiceProxy") - val weatherService = TestProbe[ServiceMessage]("WeatherService") - val scheduler = TestProbe[SchedulerMessage]("Scheduler") - - val emAgent = spawn( - EmAgent( - emInput, - modelConfig, - outputConfigOn, - "PRIORITIZED", - simulationStartDate, - parent = Left(scheduler.ref), - maybeRootEmConfig = None, - listener = Iterable(resultListener.ref), - ), - "EmAgent", - ) - - val loadAgent = TestActorRef( - new FixedLoadAgent( - scheduler = scheduler.ref.toClassic, - initStateData = ParticipantInitializeStateData( - loadInput, - LoadRuntimeConfig( - calculateMissingReactivePowerWithModel = true, - scaling = 1d, - modelBehaviour = "fix", - reference = "power", - uuids = List.empty, - ), - primaryServiceProxy.ref.toClassic, - None, - simulationStartDate, - simulationEndDate, - resolution, - simonaConfig.simona.runtime.participant.requestVoltageDeviationThreshold, - outputConfigOff, - Some(emAgent), - ), - listener = Iterable(resultListener.ref.toClassic), - ), - "LoadAgent", - ) - val pvAgent = TestActorRef( - new PvAgent( - scheduler = scheduler.ref.toClassic, - initStateData = ParticipantInitializeStateData( - pvInput, - PvRuntimeConfig( - calculateMissingReactivePowerWithModel = true, - scaling = 2d, - uuids = List.empty, - ), - primaryServiceProxy.ref.toClassic, - Iterable(ActorWeatherService(weatherService.ref.toClassic)), - simulationStartDate, - simulationEndDate, - resolution, - simonaConfig.simona.runtime.participant.requestVoltageDeviationThreshold, - outputConfigOff, - Some(emAgent), - ), - listener = Iterable(resultListener.ref.toClassic), - ), - "PvAgent", - ) - val storageAgent = TestActorRef( - new StorageAgent( - scheduler = scheduler.ref.toClassic, - initStateData = ParticipantInitializeStateData( - householdStorageInput, - StorageRuntimeConfig( - calculateMissingReactivePowerWithModel = true, - scaling = 1d, - uuids = List.empty, - initialSoc = 0d, - targetSoc = None, - ), - primaryServiceProxy.ref.toClassic, - None, - simulationStartDate, - simulationEndDate, - resolution, - simonaConfig.simona.runtime.participant.requestVoltageDeviationThreshold, - outputConfigOff, - Some(emAgent), - ), - listener = Iterable(resultListener.ref.toClassic), - ), - "StorageAgent", - ) - - scheduler.expectNoMessage() - - /* INIT */ - - // load - loadAgent ! Activation(INIT_SIM_TICK) - - primaryServiceProxy.expectMessage( - PrimaryServiceRegistrationMessage(loadInput.getUuid) - ) - loadAgent ! RegistrationFailedMessage(primaryServiceProxy.ref.toClassic) - - // the order of the two messages is not given - val emAgentActivation = scheduler - .receiveMessages(2) - .flatMap { - case Completion(ref, maybeNewTick) => - ref shouldBe loadAgent.toTyped - maybeNewTick shouldBe None - None - case ScheduleActivation(ref, tick, unlockKey) => - // em agent schedules itself - tick shouldBe 0 - unlockKey shouldBe None - Some(ref) - case unexpected => - fail(s"Received unexpected message $unexpected") - } - .headOption - .value - - // pv - pvAgent ! Activation(INIT_SIM_TICK) - - primaryServiceProxy.expectMessage( - PrimaryServiceRegistrationMessage(pvInput.getUuid) - ) - pvAgent ! RegistrationFailedMessage(primaryServiceProxy.ref.toClassic) - - // deal with weather service registration - weatherService.expectMessage( - RegisterForWeatherMessage( - pvInput.getNode.getGeoPosition.getY, - pvInput.getNode.getGeoPosition.getX, - ) - ) - - pvAgent ! RegistrationSuccessfulMessage( - weatherService.ref.toClassic, - Some(0L), - ) - - scheduler.expectMessage(Completion(pvAgent)) - - // storage - storageAgent ! Activation(INIT_SIM_TICK) - - primaryServiceProxy.expectMessage( - PrimaryServiceRegistrationMessage(householdStorageInput.getUuid) - ) - storageAgent ! RegistrationFailedMessage( - primaryServiceProxy.ref.toClassic - ) - - scheduler.expectMessage(Completion(storageAgent)) - - /* TICK 0 - LOAD: 0.000269 MW - PV: -0.005685 MW - STORAGE: SOC 0 % - -> charge with 5 kW - -> remaining -0.0004161 MW - */ - - emAgentActivation ! Activation(0) - - pvAgent ! ProvideWeatherMessage( - 0, - weatherService.ref.toClassic, - WeatherData( - WattsPerSquareMeter(400d), - WattsPerSquareMeter(200d), - Celsius(0d), - MetersPerSecond(0d), - ), - Some(7200), - ) - - resultListener.expectMessageType[ParticipantResultEvent] match { - case ParticipantResultEvent(emResult: EmResult) => - emResult.getInputModel shouldBe emInput.getUuid - emResult.getTime shouldBe 0L.toDateTime - emResult.getP should equalWithTolerance( - (-0.000416087825).asMegaWatt - ) - emResult.getQ should equalWithTolerance(0.0000882855367.asMegaVar) - } - - scheduler.expectMessage(Completion(emAgentActivation, Some(7200))) - - /* TICK 7200 - LOAD: 0.000269 MW (unchanged) - PV: -0.003797 MW - STORAGE: SOC 63.3 % - -> charge with 3.5282 kW - -> remaining 0 MW - */ - - emAgentActivation ! Activation(7200) - - pvAgent ! ProvideWeatherMessage( - 7200, - weatherService.ref.toClassic, - WeatherData( - WattsPerSquareMeter(300d), - WattsPerSquareMeter(500d), - Celsius(0d), - MetersPerSecond(0d), - ), - Some(14400), - ) - - resultListener.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.0000882855367.asMegaVar) - } - - scheduler.expectMessage(Completion(emAgentActivation, Some(13107))) - - /* TICK 13107 - LOAD: 0.000269 MW (unchanged) - PV: -0.003797 MW (unchanged) - STORAGE: SOC 100 % - -> charge with 0 kW - -> remaining -0.003528 MW - */ - - emAgentActivation ! Activation(13107) - - resultListener.expectMessageType[ParticipantResultEvent] match { - case ParticipantResultEvent(emResult: EmResult) => - emResult.getInputModel shouldBe emInput.getUuid - emResult.getTime shouldBe 13107L.toDateTime - emResult.getP should equalWithTolerance( - (-0.0035281545552).asMegaWatt - ) - emResult.getQ should equalWithTolerance(0.0000882855367.asMegaVar) - } - - scheduler.expectMessage(Completion(emAgentActivation, Some(14400))) - - /* TICK 14400 - LOAD: 0.000269 MW (unchanged) - PV: -0.000066 MW - STORAGE: SOC 100 % - -> charge with -0.202956 kW - -> remaining 0 MW - */ - - // send weather data before activation, which can happen - // it got cloudy now... - pvAgent ! ProvideWeatherMessage( - 14400, - weatherService.ref.toClassic, - WeatherData( - WattsPerSquareMeter(5d), - WattsPerSquareMeter(5d), - Celsius(0d), - MetersPerSecond(0d), - ), - Some(21600), - ) - - emAgentActivation ! Activation(14400) - - resultListener.expectMessageType[ParticipantResultEvent] match { - case ParticipantResultEvent(emResult: EmResult) => - emResult.getInputModel shouldBe emInput.getUuid - emResult.getTime shouldBe 14400L.toDateTime - emResult.getP should equalWithTolerance(0.asMegaWatt) - emResult.getQ should equalWithTolerance(0.000088285537.asMegaVar) - } - - scheduler.expectMessage(Completion(emAgentActivation, Some(21600))) - - } - } "having load, pv and heat pump agents connected" should { "be initialized correctly and run through some activations" in { From 90e36592c21795bfef9b437427b2e8118083bacc Mon Sep 17 00:00:00 2001 From: Sebastian Peter Date: Fri, 23 Feb 2024 14:54:47 +0100 Subject: [PATCH 04/32] Accounting for storage, which is not included yet --- src/main/scala/edu/ie3/simona/util/ConfigUtil.scala | 1 - src/test/scala/edu/ie3/simona/util/ConfigUtilSpec.scala | 6 +++--- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/src/main/scala/edu/ie3/simona/util/ConfigUtil.scala b/src/main/scala/edu/ie3/simona/util/ConfigUtil.scala index fa02c1137f..b041129c90 100644 --- a/src/main/scala/edu/ie3/simona/util/ConfigUtil.scala +++ b/src/main/scala/edu/ie3/simona/util/ConfigUtil.scala @@ -20,7 +20,6 @@ import edu.ie3.datamodel.models.result.connector.{ } import edu.ie3.datamodel.models.result.{NodeResult, ResultEntity} import edu.ie3.simona.config.SimonaConfig -import edu.ie3.simona.config.SimonaConfig.Simona.Input.Weather.Datasource.SqlParams import edu.ie3.simona.config.SimonaConfig._ import edu.ie3.simona.event.notifier.{Notifier, NotifierConfig} import edu.ie3.simona.exceptions.InvalidConfigParameterException diff --git a/src/test/scala/edu/ie3/simona/util/ConfigUtilSpec.scala b/src/test/scala/edu/ie3/simona/util/ConfigUtilSpec.scala index 92d295097a..1356fef9cb 100644 --- a/src/test/scala/edu/ie3/simona/util/ConfigUtilSpec.scala +++ b/src/test/scala/edu/ie3/simona/util/ConfigUtilSpec.scala @@ -58,7 +58,7 @@ class ConfigUtilSpec inside(actual) { case ParticipantConfigUtil(configs, defaultConfigs) => configs shouldBe Map.empty[UUID, SimonaConfig.LoadRuntimeConfig] - defaultConfigs.size shouldBe 8 + defaultConfigs.size shouldBe 7 inside(defaultConfigs.get(classOf[LoadRuntimeConfig])) { case Some( @@ -118,7 +118,7 @@ class ConfigUtilSpec UUID.fromString("49f250fa-41ff-4434-a083-79c98d260a76") ) - defaultConfigs.size shouldBe 8 + defaultConfigs.size shouldBe 7 inside(defaultConfigs.get(classOf[LoadRuntimeConfig])) { case Some( LoadRuntimeConfig( @@ -331,7 +331,7 @@ class ConfigUtilSpec UUID.fromString("49f250fa-41ff-4434-a083-79c98d260a76") ) - defaultConfigs.size shouldBe 8 + defaultConfigs.size shouldBe 7 inside(defaultConfigs.get(classOf[FixedFeedInRuntimeConfig])) { case Some( FixedFeedInRuntimeConfig( From 72cd1b3da57dc99fff4f09ab4bde2118ae3036bc Mon Sep 17 00:00:00 2001 From: Sebastian Peter Date: Thu, 7 Mar 2024 13:42:52 +0100 Subject: [PATCH 05/32] Adapting to changes of new PSDM, fixing compilation errors --- .../edu/ie3/simona/agent/em/EmAgent.scala | 2 +- .../agent/grid/GridAgentController.scala | 267 +++++++----------- .../ie3/simona/model/em/EmModelShell.scala | 6 + .../edu/ie3/simona/agent/em/EmAgentIT.scala | 2 +- .../test/common/input/EmInputTestData.scala | 38 ++- 5 files changed, 126 insertions(+), 189 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 f9b352f84c..6ae72af0cb 100644 --- a/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala +++ b/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala @@ -6,7 +6,7 @@ package edu.ie3.simona.agent.em -import edu.ie3.datamodel.models.input.system.EmInput +import edu.ie3.datamodel.models.input.EmInput import edu.ie3.datamodel.models.result.system.{EmResult, FlexOptionsResult} import edu.ie3.simona.agent.participant.data.Data.PrimaryData.ApparentPower import edu.ie3.simona.agent.participant.statedata.BaseStateData.FlexControlledData diff --git a/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala b/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala index d59081c6c0..02342b3f79 100644 --- a/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala +++ b/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala @@ -7,16 +7,9 @@ package edu.ie3.simona.agent.grid import com.typesafe.scalalogging.LazyLogging -import edu.ie3.datamodel.models.ControlStrategy -import edu.ie3.datamodel.models.input.container.{ - SubGridContainer, - SystemParticipants, - ThermalGrid, -} +import edu.ie3.datamodel.models.input.EmInput +import edu.ie3.datamodel.models.input.container.{SubGridContainer, ThermalGrid} import edu.ie3.datamodel.models.input.system._ -import edu.ie3.datamodel.models.input.system.characteristic.CosPhiFixed -import edu.ie3.datamodel.models.input.{AssetInput, NodeInput} -import edu.ie3.datamodel.models.voltagelevels.GermanVoltageLevelUtils import edu.ie3.simona.actor.SimonaActorNaming._ import edu.ie3.simona.agent.EnvironmentRefs import edu.ie3.simona.agent.em.EmAgent @@ -43,17 +36,16 @@ import edu.ie3.simona.ontology.messages.flex.FlexibilityMessage.FlexResponse import edu.ie3.simona.util.ConfigUtil import edu.ie3.simona.util.ConfigUtil._ import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK -import edu.ie3.util.quantities.QuantityUtils.RichQuantityDouble import org.apache.pekko.actor.typed.ActorRef import org.apache.pekko.actor.typed.scaladsl.ActorContext import org.apache.pekko.actor.typed.scaladsl.adapter._ import org.apache.pekko.actor.{ActorRef => ClassicRef} -import org.locationtech.jts.geom.{Coordinate, GeometryFactory} import org.slf4j.Logger import java.time.ZonedDateTime import java.util.UUID import scala.jdk.CollectionConverters._ +import scala.jdk.OptionConverters.RichOptional /** Holds all methods that should be available to a [[GridAgent]] * @@ -102,12 +94,10 @@ class GridAgentController( buildParticipantToActorRef( participantsConfig, outputConfig, - subGridContainer.getSystemParticipants, systemParticipants, thermalIslandGridsByBusId, environmentRefs, rootEmConfig, - subGridContainer.getSubnet, ) } @@ -130,10 +120,6 @@ class GridAgentController( environmentRefs: EnvironmentRefs, ) = { - val emParticipants = - subGridContainer.getSystemParticipants.getEmSystems.asScala - .flatMap(_.getConnectedAssets) - val (notProcessedElements, availableSysParts) = subGridContainer.getSystemParticipants .allEntitiesAsList() @@ -156,7 +142,7 @@ class GridAgentController( s"Evcs ${evcsInput.getId} has been removed because no ev movements service is present." ) (notProcessedElements, availableSystemParticipants) - case entity if emParticipants.contains(entity.getUuid) => + case entity if entity.getControllingEm.isPresent => log.debug( s"System participant {} is part of an energy-managed system and thus not directly connected to the grid.", entity, @@ -186,7 +172,7 @@ class GridAgentController( * Configuration information for participant models * @param outputConfig * Configuration information for output behaviour - * @param filteredParticipants + * @param participants * Set of system participants to create agents for * @param thermalIslandGridsByBusId * Collection of thermal island grids, mapped by their thermal bus uuid @@ -198,12 +184,10 @@ class GridAgentController( private def buildParticipantToActorRef( participantsConfig: SimonaConfig.Simona.Runtime.Participant, outputConfig: SimonaConfig.Simona.Output.Participant, - allParticipants: SystemParticipants, - filteredParticipants: Vector[SystemParticipantInput], + participants: Vector[SystemParticipantInput], thermalIslandGridsByBusId: Map[UUID, ThermalGrid], environmentRefs: EnvironmentRefs, rootEmConfig: Option[SimonaConfig.Simona.Runtime.RootEm], - subGrid: Int, ): Map[UUID, Set[ActorRef[ParticipantMessage]]] = { /* Prepare the config util for the participant models, which (possibly) utilizes as map to speed up the initialization * phase */ @@ -211,101 +195,80 @@ class GridAgentController( ConfigUtil.ParticipantConfigUtil(participantsConfig) val outputConfigUtil = ConfigUtil.OutputConfigUtil(outputConfig) - val emParticipantsUuids = - allParticipants.getEmSystems.asScala - .flatMap(_.getConnectedAssets) + def buildEmsRecursively( + emInputs: Map[UUID, EmInput], + existingEms: Map[UUID, ActorRef[EmMessage]], + ): Map[UUID, ActorRef[EmMessage]] = { + val (controlledEmInputs, uncontrolledEms) = emInputs + .partitionMap { case (uuid, emInput) => + if (emInput.getControllingEm.isPresent) + Left(uuid -> emInput) + else { + val actor = buildEm( + emInput, + participantConfigUtil.getOrDefault[EmRuntimeConfig](uuid), + outputConfigUtil.getOrDefault(NotifierIdentifier.Em), + maybeControllingEm = None, + rootEmConfig = rootEmConfig, + ) + Right(uuid -> actor) + } + } - val emParticipantMap = allParticipants - .allEntitiesAsList() - .asScala - .filter(sp => emParticipantsUuids.contains(sp.getUuid)) - .map(sp => sp.getUuid -> sp) - .toMap + val existingAndUncontrolledEms = existingEms ++ uncontrolledEms.toMap - val emUnits = allParticipants.getEmSystems.asScala + if (controlledEmInputs.nonEmpty) { + // EMs that are controlling EMs at this level + val controllingEms = controlledEmInputs.toMap.flatMap { + case (uuid, emInput) => + emInput.getControllingEm.toScala.map(uuid -> _) + } - val participantToEm = emUnits - .flatMap(em => em.getConnectedAssets.toSeq.map(_ -> em.getUuid)) - .toMap + val recursiveEms = buildEmsRecursively( + controllingEms, + existingAndUncontrolledEms, + ) - val (uncontrolledEms, controlledEms) = - emUnits.map(em => em -> participantToEm.get(em.getUuid)).partitionMap { - case (em, None) => Left(em) - case (em, Some(parentEm)) => Right(em -> parentEm) - } + val controlledEms = controlledEmInputs.map { case (uuid, emInput) => + val controllingEm = emInput.getControllingEm.toScala + .map(_.getUuid) + .flatMap(uuid => recursiveEms.get(uuid)) + + uuid -> buildEm( + emInput, + participantConfigUtil.getOrDefault[EmRuntimeConfig](uuid), + outputConfigUtil.getOrDefault(NotifierIdentifier.Em), + maybeControllingEm = controllingEm, + rootEmConfig = None, + ) + }.toMap - val uncontrolledEmActors = uncontrolledEms.map { em => - buildEm( - em, - participantConfigUtil.getOrDefault[EmRuntimeConfig]( - em.getUuid - ), - outputConfigUtil.getOrDefault(NotifierIdentifier.Em), - maybeParentEm = None, - rootEmConfig = rootEmConfig, - ) + recursiveEms ++ controlledEms + } else { + existingAndUncontrolledEms + } } - val (emInputs, otherInputs) = filteredParticipants.partition { - case _: EmInput => true - case _ => false - } + // all ems that control at least one participant directly + val firstLevelEms = participants.flatMap { + _.getControllingEm.toScala.map(em => em.getUuid -> em) + }.toMap - if (rootEmConfig.isDefined && emInputs.nonEmpty) { - val mockRootEmInput = new EmInput( - UUID.fromString(s"11111111-0000-0000-0000-${"%012d".format(subGrid)}"), - "Root EmAgent", - new NodeInput( - UUID.randomUUID(), - "Mock node for root EmAgent", - 1d.asPu, - false, - new GeometryFactory().createPoint(new Coordinate()), - GermanVoltageLevelUtils.LV, - 0, - ), - new CosPhiFixed("cosPhiFixed:{(0.00,0.90)}"), - emInputs.map(_.getUuid).toArray, - ControlStrategy.DefaultControlStrategies.NO_CONTROL_STRATEGY, - ) + val allEms = buildEmsRecursively(firstLevelEms, Map.empty) - val completeEmParticipantMap = - emParticipantMap ++ emInputs.map(sp => sp.getUuid -> sp) - - val actorRef = buildEm( - mockRootEmInput, - EmRuntimeConfig( - calculateMissingReactivePowerWithModel = false, - 1d, - List.empty, - pvFlex = false, - aggregateFlex = "SIMPLE_SUM", - ), - outputConfigUtil.getOrDefault(NotifierIdentifier.Em), - None, - rootEmConfig, - ) - - // introduce to environment - introduceAgentToEnvironment(actorRef.toClassic) - } - - rootEmConfig - .map(_ => otherInputs) - .getOrElse(filteredParticipants) + participants .map { participant => val node = participant.getNode - // build - val actorRef = - buildParticipantActor( - participantsConfig.requestVoltageDeviationThreshold, - participantConfigUtil, - outputConfigUtil, - participant, - thermalIslandGridsByBusId, - environmentRefs, - emParticipantMap, - ) + + val actorRef = buildParticipantActor( + participantsConfig.requestVoltageDeviationThreshold, + participantConfigUtil, + outputConfigUtil, + participant, + thermalIslandGridsByBusId, + environmentRefs, + allEms.get(participant.getUuid), + ) introduceAgentToEnvironment(actorRef) // return uuid to actorRef node.getUuid -> actorRef @@ -314,33 +277,6 @@ class GridAgentController( .groupMap(entry => entry._1)(entry => entry._2) } - private def buildControlledEms() = { - // TODO recursive - } - - // TODO not needed anymore if psdm does this - private def matchWithEm[T <: AssetInput]( - existingEmActors: Map[UUID, ActorRef[EmMessage]], - remainingEms: Iterable[(T, UUID)], - ): Iterable[(T, ActorRef[EmMessage])] = { - remainingEms - .map { case (em, parentUuid) => - existingEmActors - .get(parentUuid) - .map(em -> _) - .toRight((em.getUuid, parentUuid)) - } - .partitionMap(identity) match { - // only return em actors if there are no missing parent ems - case (Nil, matchedAssets) => matchedAssets - case (emsAndParents, _) => - val (ems, parents) = emsAndParents.unzip - throw new GridAgentInitializationException( - s"The parent energy management unit(s) $ems could not be created because the em parent(s) ${parents.toSet} do not exist." - ) - } - } - private def buildParticipantActor( requestVoltageDeviationThreshold: Double, participantConfigUtil: ConfigUtil.ParticipantConfigUtil, @@ -348,8 +284,7 @@ class GridAgentController( participantInputModel: SystemParticipantInput, thermalIslandGridsByBusId: Map[UUID, ThermalGrid], environmentRefs: EnvironmentRefs, - participantEmMap: Map[UUID, SystemParticipantInput], - maybeParentEm: Option[ActorRef[FlexResponse]] = None, + maybeControllingEm: Option[ActorRef[FlexResponse]] = None, ): ActorRef[ParticipantMessage] = participantInputModel match { case input: FixedFeedInInput => buildFixedFeedIn( @@ -363,7 +298,7 @@ class GridAgentController( resolution, requestVoltageDeviationThreshold, outputConfigUtil.getOrDefault(NotifierIdentifier.FixedFeedIn), - maybeParentEm, + maybeControllingEm, ) case input: LoadInput => buildLoad( @@ -377,7 +312,7 @@ class GridAgentController( resolution, requestVoltageDeviationThreshold, outputConfigUtil.getOrDefault(NotifierIdentifier.Load), - maybeParentEm, + maybeControllingEm, ) case input: PvInput => buildPv( @@ -392,7 +327,7 @@ class GridAgentController( resolution, requestVoltageDeviationThreshold, outputConfigUtil.getOrDefault(NotifierIdentifier.PvPlant), - maybeParentEm, + maybeControllingEm, ) case input: WecInput => buildWec( @@ -407,7 +342,7 @@ class GridAgentController( resolution, requestVoltageDeviationThreshold, outputConfigUtil.getOrDefault(NotifierIdentifier.Wec), - maybeParentEm, + maybeControllingEm, ) case input: EvcsInput => buildEvcs( @@ -426,7 +361,7 @@ class GridAgentController( resolution, requestVoltageDeviationThreshold, outputConfigUtil.getOrDefault(NotifierIdentifier.Evcs), - maybeParentEm, + maybeControllingEm, ) case hpInput: HpInput => thermalIslandGridsByBusId.get(hpInput.getThermalBus.getUuid) match { @@ -441,7 +376,7 @@ class GridAgentController( environmentRefs.weather, requestVoltageDeviationThreshold, outputConfigUtil.getOrDefault(NotifierIdentifier.Hp), - maybeParentEm, + maybeControllingEm, ) case None => throw new GridAgentInitializationException( @@ -477,10 +412,10 @@ class GridAgentController( * Maximum deviation in p.u. of request voltages to be considered equal * @param outputConfig * Configuration of the output behavior - * @param maybeParentEm + * @param maybeControllingEm * The parent EmAgent, if applicable * @return - * The [[FixedFeedInAgent]] 's [[ClassicActorRef]] + * The [[FixedFeedInAgent]] 's [[ActorRef]] */ private def buildFixedFeedIn( fixedFeedInInput: FixedFeedInInput, @@ -491,7 +426,7 @@ class GridAgentController( resolution: Long, requestVoltageDeviationThreshold: Double, outputConfig: NotifierConfig, - maybeParentEm: Option[ActorRef[FlexResponse]] = None, + maybeControllingEm: Option[ActorRef[FlexResponse]] = None, ): ActorRef[ParticipantMessage] = gridAgentContext.toClassic .simonaActorOf( @@ -507,7 +442,7 @@ class GridAgentController( resolution, requestVoltageDeviationThreshold, outputConfig, - maybeParentEm, + maybeControllingEm, ), listener.map(_.toClassic), ), @@ -534,10 +469,10 @@ class GridAgentController( * Maximum deviation in p.u. of request voltages to be considered equal * @param outputConfig * Configuration of the output behavior - * @param maybeParentEm + * @param maybeParmaybeControllingEm * The parent EmAgent, if applicable * @return - * The [[LoadAgent]] 's [[ClassicActorRef]] + * The [[LoadAgent]] 's [[ActorRef]] */ private def buildLoad( loadInput: LoadInput, @@ -548,7 +483,7 @@ class GridAgentController( resolution: Long, requestVoltageDeviationThreshold: Double, outputConfig: NotifierConfig, - maybeParentEm: Option[ActorRef[FlexResponse]] = None, + maybeParmaybeControllingEm: Option[ActorRef[FlexResponse]] = None, ): ActorRef[ParticipantMessage] = gridAgentContext.toClassic .simonaActorOf( @@ -564,7 +499,7 @@ class GridAgentController( resolution, requestVoltageDeviationThreshold, outputConfig, - maybeParentEm, + maybeParmaybeControllingEm, ), listener.map(_.toClassic), ), @@ -593,10 +528,10 @@ class GridAgentController( * Maximum deviation in p.u. of request voltages to be considered equal * @param outputConfig * Configuration of the output behavior - * @param maybeParentEm + * @param maybeControllingEm * The parent EmAgent, if applicable * @return - * The [[PvAgent]] 's [[ClassicActorRef]] + * The [[PvAgent]] 's [[ActorRef]] */ private def buildPv( pvInput: PvInput, @@ -608,7 +543,7 @@ class GridAgentController( resolution: Long, requestVoltageDeviationThreshold: Double, outputConfig: NotifierConfig, - maybeParentEm: Option[ActorRef[FlexResponse]] = None, + maybeControllingEm: Option[ActorRef[FlexResponse]] = None, ): ActorRef[ParticipantMessage] = gridAgentContext.toClassic .simonaActorOf( @@ -624,7 +559,7 @@ class GridAgentController( resolution, requestVoltageDeviationThreshold, outputConfig, - maybeParentEm, + maybeControllingEm, ), listener.map(_.toClassic), ), @@ -653,10 +588,10 @@ class GridAgentController( * Maximum deviation in p.u. of request voltages to be considered equal * @param outputConfig * Configuration of the output behavior - * @param maybeParentEm + * @param maybeControllingEm * The parent EmAgent, if applicable * @return - * The [[EvcsAgent]] 's [[ClassicActorRef]] + * The [[EvcsAgent]] 's [[ActorRef]] */ private def buildEvcs( evcsInput: EvcsInput, @@ -668,7 +603,7 @@ class GridAgentController( resolution: Long, requestVoltageDeviationThreshold: Double, outputConfig: NotifierConfig, - maybeParentEm: Option[ActorRef[FlexResponse]] = None, + maybeControllingEm: Option[ActorRef[FlexResponse]] = None, ): ActorRef[ParticipantMessage] = gridAgentContext.toClassic .simonaActorOf( @@ -688,7 +623,7 @@ class GridAgentController( resolution, requestVoltageDeviationThreshold, outputConfig, - maybeParentEm, + maybeControllingEm, ), listener.map(_.toClassic), ) @@ -711,7 +646,7 @@ class GridAgentController( * Permissible voltage magnitude deviation to consider being equal * @param outputConfig * Configuration for output notification - * @param maybeParentEm + * @param maybeControllingEm * The parent EmAgent, if applicable * @return * A tuple of actor reference and [[ParticipantInitializeStateData]] @@ -724,7 +659,7 @@ class GridAgentController( weatherService: ClassicRef, requestVoltageDeviationThreshold: Double, outputConfig: NotifierConfig, - maybeParentEm: Option[ActorRef[FlexResponse]] = None, + maybeControllingEm: Option[ActorRef[FlexResponse]] = None, ): ActorRef[ParticipantMessage] = gridAgentContext.toClassic .simonaActorOf( @@ -741,7 +676,7 @@ class GridAgentController( resolution, requestVoltageDeviationThreshold, outputConfig, - maybeParentEm, + maybeControllingEm, ), listener.map(_.toClassic), ), @@ -770,10 +705,10 @@ class GridAgentController( * Maximum deviation in p.u. of request voltages to be considered equal * @param outputConfig * Configuration of the output behavior - * @param maybeParentEm + * @param maybeControllingEm * The parent EmAgent, if applicable * @return - * The [[WecAgent]] 's [[ClassicActorRef]] + * The [[WecAgent]] 's [[ActorRef]] */ private def buildWec( wecInput: WecInput, @@ -785,7 +720,7 @@ class GridAgentController( resolution: Long, requestVoltageDeviationThreshold: Double, outputConfig: NotifierConfig, - maybeParentEm: Option[ActorRef[FlexResponse]] = None, + maybeControllingEm: Option[ActorRef[FlexResponse]] = None, ): ActorRef[ParticipantMessage] = gridAgentContext.toClassic .simonaActorOf( @@ -801,7 +736,7 @@ class GridAgentController( resolution, requestVoltageDeviationThreshold, outputConfig, - maybeParentEm, + maybeControllingEm, ), listener.map(_.toClassic), ), @@ -824,7 +759,7 @@ class GridAgentController( emInput: EmInput, modelConfiguration: EmRuntimeConfig, outputConfig: NotifierConfig, - maybeParentEm: Option[ActorRef[FlexResponse]] = None, + maybeControllingEm: Option[ActorRef[FlexResponse]] = None, rootEmConfig: Option[SimonaConfig.Simona.Runtime.RootEm] = None, ): ActorRef[EmMessage] = gridAgentContext.spawn( @@ -836,7 +771,7 @@ class GridAgentController( .map(_ => "PROPORTIONAL") .getOrElse("PRIORITIZED"), simulationStartDate, - maybeParentEm.toRight( + maybeControllingEm.toRight( environmentRefs.scheduler ), rootEmConfig, diff --git a/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala b/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala index ec6acc26a3..0ec7cd6c93 100644 --- a/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala +++ b/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala @@ -120,12 +120,18 @@ object EmModelShell { val modelStrategy = modelStrat match { case "PROPORTIONAL" => ProportionalFlexStrat case "PRIORITIZED" => PrioritizedFlexStrat(modelConfig.pvFlex) + case unknown => + throw new CriticalFailureException(s"Unknown model strategy $unknown") } val aggregateFlex = modelConfig.aggregateFlex match { case "SELF_OPT_EXCL_PV" => EmAggregateSelfOpt(false) case "SELF_OPT" => EmAggregateSelfOpt(true) case "SIMPLE_SUM" => EmAggregateSimpleSum + case unknown => + throw new CriticalFailureException( + s"Unknown aggregate flex strategy $unknown" + ) } EmModelShell(uuid, id, modelStrategy, aggregateFlex) 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 79af306e45..e3c443a786 100644 --- a/src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala +++ b/src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala @@ -148,7 +148,7 @@ class EmAgentIT new PvAgent( scheduler = scheduler.ref.toClassic, initStateData = ParticipantInitializeStateData( - pvInput, + pvInput, // TODO pv em PvRuntimeConfig( calculateMissingReactivePowerWithModel = true, scaling = 2d, diff --git a/src/test/scala/edu/ie3/simona/test/common/input/EmInputTestData.scala b/src/test/scala/edu/ie3/simona/test/common/input/EmInputTestData.scala index e323085ece..4c349c603e 100644 --- a/src/test/scala/edu/ie3/simona/test/common/input/EmInputTestData.scala +++ b/src/test/scala/edu/ie3/simona/test/common/input/EmInputTestData.scala @@ -6,29 +6,24 @@ package edu.ie3.simona.test.common.input -import edu.ie3.datamodel.models.input.OperatorInput +import edu.ie3.datamodel.models.input.{EmInput, OperatorInput} import edu.ie3.datamodel.models.input.container.ThermalGrid +import edu.ie3.datamodel.models.input.system.`type`.chargingpoint.ChargingPointTypeUtils +import edu.ie3.datamodel.models.input.system.`type`.evcslocation.EvcsLocationType import edu.ie3.datamodel.models.input.system.`type`.{ HpTypeInput, StorageTypeInput, } -import edu.ie3.datamodel.models.input.system.`type`.chargingpoint.ChargingPointTypeUtils -import edu.ie3.datamodel.models.input.system.`type`.evcslocation.EvcsLocationType import edu.ie3.datamodel.models.input.system.characteristic.{ CosPhiFixed, ReactivePowerCharacteristic, } -import edu.ie3.datamodel.models.input.system.{ - EmInput, - EvcsInput, - HpInput, - StorageInput, -} +import edu.ie3.datamodel.models.input.system.{EvcsInput, HpInput, StorageInput} import edu.ie3.datamodel.models.input.thermal.{ ThermalHouseInput, ThermalStorageInput, } -import edu.ie3.datamodel.models.{ControlStrategy, OperationTime, StandardUnits} +import edu.ie3.datamodel.models.{OperationTime, StandardUnits} import edu.ie3.simona.config.SimonaConfig import edu.ie3.simona.event.notifier.NotifierConfig import edu.ie3.simona.model.participant.load.{LoadModelBehaviour, LoadReference} @@ -48,6 +43,15 @@ trait EmInputTestData with LoadInputTestData with HpTestData { + protected val emInput = new EmInput( + UUID.randomUUID(), + "Dummy_EmModel", + new OperatorInput(UUID.randomUUID(), "NO_OPERATOR"), + OperationTime.notLimited(), + "PRIORITIZED", + null, + ) + protected val evcsInput = new EvcsInput( UUID.randomUUID(), "Dummy_EvcsModel", @@ -55,6 +59,7 @@ trait EmInputTestData OperationTime.notLimited(), nodeInputNoSlackNs04KvA, CosPhiFixed.CONSTANT_CHARACTERISTIC, + emInput, ChargingPointTypeUtils.ChargingStationType2, 2, 0.95, @@ -85,20 +90,10 @@ trait EmInputTestData OperationTime.notLimited(), nodeInputNoSlackNs04KvA, CosPhiFixed.CONSTANT_CHARACTERISTIC, + emInput, householdStorageTypeInput, ) - protected val emInput = new EmInput( - UUID.randomUUID(), - "Dummy_EmModel", - new OperatorInput(UUID.randomUUID(), "NO_OPERATOR"), - OperationTime.notLimited(), - nodeInputNoSlackNs04KvA, - CosPhiFixed.CONSTANT_CHARACTERISTIC, - Array.empty, - ControlStrategy.DefaultControlStrategies.NO_CONTROL_STRATEGY, // FIXME adapt once available - ) - protected val simonaConfig: SimonaConfig = createSimonaConfig( LoadModelBehaviour.FIX, @@ -141,6 +136,7 @@ trait EmInputTestData nodeInput, thermalBusInput, ReactivePowerCharacteristic.parse("cosPhiFixed:{(0.00,0.98)}"), + emInput, adaptedTypeInput, ) From a6b83c0caa99d887ed66c01d793d6fe2e7acb6cc Mon Sep 17 00:00:00 2001 From: Sebastian Peter Date: Mon, 11 Mar 2024 09:45:30 +0100 Subject: [PATCH 06/32] Adapting to changed time format --- src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala | 4 ++-- src/test/scala/edu/ie3/simona/agent/em/EmAgentSpec.scala | 2 +- 2 files changed, 3 insertions(+), 3 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 e3c443a786..900d9f7b8c 100644 --- a/src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala +++ b/src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala @@ -64,9 +64,9 @@ class EmAgentIT // start a bit later so the sun is up protected implicit val simulationStartDate: ZonedDateTime = - TimeUtil.withDefaults.toZonedDateTime("2020-01-01 10:00:00") + TimeUtil.withDefaults.toZonedDateTime("2020-01-01T10:00:00Z") protected val simulationEndDate: ZonedDateTime = - TimeUtil.withDefaults.toZonedDateTime("2020-01-02 02:00:00") + TimeUtil.withDefaults.toZonedDateTime("2020-01-02T02:00:00Z") private val resolution = simonaConfig.simona.powerflow.resolution.getSeconds 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 2c350a29b8..519d6af57f 100644 --- a/src/test/scala/edu/ie3/simona/agent/em/EmAgentSpec.scala +++ b/src/test/scala/edu/ie3/simona/agent/em/EmAgentSpec.scala @@ -50,7 +50,7 @@ class EmAgentSpec with SquantsMatchers { protected implicit val simulationStartDate: ZonedDateTime = - TimeUtil.withDefaults.toZonedDateTime("2020-01-01 00:00:00") + TimeUtil.withDefaults.toZonedDateTime("2020-01-01T00:00:00Z") private val outputConfig = NotifierConfig( simulationResultInfo = true, From ee54a17611e7535e028218e0b6166ecec5a0124e Mon Sep 17 00:00:00 2001 From: Sebastian Peter Date: Mon, 11 Mar 2024 09:45:51 +0100 Subject: [PATCH 07/32] Excluding storages for now --- .../scala/edu/ie3/simona/agent/grid/GridAgentController.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala b/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala index 02342b3f79..adc673adee 100644 --- a/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala +++ b/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala @@ -130,7 +130,8 @@ class GridAgentController( curSysPart, ) => curSysPart match { - case entity @ (_: BmInput | _: ChpInput | _: EvInput) => + case entity @ (_: BmInput | _: ChpInput | _: EvInput | + _: StorageInput) => ( notProcessedElements + entity.getClass.getSimpleName, availableSystemParticipants, From 0fedd64e75fd6be65206b1125a0b249771f31346 Mon Sep 17 00:00:00 2001 From: Sebastian Peter Date: Mon, 11 Mar 2024 09:46:01 +0100 Subject: [PATCH 08/32] Fixing typo --- .../edu/ie3/simona/agent/grid/GridAgentController.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala b/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala index adc673adee..7d550cf966 100644 --- a/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala +++ b/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala @@ -470,7 +470,7 @@ class GridAgentController( * Maximum deviation in p.u. of request voltages to be considered equal * @param outputConfig * Configuration of the output behavior - * @param maybeParmaybeControllingEm + * @param maybeControllingEm * The parent EmAgent, if applicable * @return * The [[LoadAgent]] 's [[ActorRef]] @@ -484,7 +484,7 @@ class GridAgentController( resolution: Long, requestVoltageDeviationThreshold: Double, outputConfig: NotifierConfig, - maybeParmaybeControllingEm: Option[ActorRef[FlexResponse]] = None, + maybeControllingEm: Option[ActorRef[FlexResponse]] = None, ): ActorRef[ParticipantMessage] = gridAgentContext.toClassic .simonaActorOf( @@ -500,7 +500,7 @@ class GridAgentController( resolution, requestVoltageDeviationThreshold, outputConfig, - maybeParmaybeControllingEm, + maybeControllingEm, ), listener.map(_.toClassic), ), From bd13b9d97ead574b4dd452a9fc7bbd658031537e Mon Sep 17 00:00:00 2001 From: Sebastian Peter Date: Mon, 11 Mar 2024 11:13:34 +0100 Subject: [PATCH 09/32] Extracting em building method --- .../agent/grid/GridAgentController.scala | 124 ++++++++++-------- 1 file changed, 68 insertions(+), 56 deletions(-) diff --git a/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala b/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala index 7d550cf966..9e2ce7e71b 100644 --- a/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala +++ b/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala @@ -196,66 +196,18 @@ class GridAgentController( ConfigUtil.ParticipantConfigUtil(participantsConfig) val outputConfigUtil = ConfigUtil.OutputConfigUtil(outputConfig) - def buildEmsRecursively( - emInputs: Map[UUID, EmInput], - existingEms: Map[UUID, ActorRef[EmMessage]], - ): Map[UUID, ActorRef[EmMessage]] = { - val (controlledEmInputs, uncontrolledEms) = emInputs - .partitionMap { case (uuid, emInput) => - if (emInput.getControllingEm.isPresent) - Left(uuid -> emInput) - else { - val actor = buildEm( - emInput, - participantConfigUtil.getOrDefault[EmRuntimeConfig](uuid), - outputConfigUtil.getOrDefault(NotifierIdentifier.Em), - maybeControllingEm = None, - rootEmConfig = rootEmConfig, - ) - Right(uuid -> actor) - } - } - - val existingAndUncontrolledEms = existingEms ++ uncontrolledEms.toMap - - if (controlledEmInputs.nonEmpty) { - // EMs that are controlling EMs at this level - val controllingEms = controlledEmInputs.toMap.flatMap { - case (uuid, emInput) => - emInput.getControllingEm.toScala.map(uuid -> _) - } - - val recursiveEms = buildEmsRecursively( - controllingEms, - existingAndUncontrolledEms, - ) - - val controlledEms = controlledEmInputs.map { case (uuid, emInput) => - val controllingEm = emInput.getControllingEm.toScala - .map(_.getUuid) - .flatMap(uuid => recursiveEms.get(uuid)) - - uuid -> buildEm( - emInput, - participantConfigUtil.getOrDefault[EmRuntimeConfig](uuid), - outputConfigUtil.getOrDefault(NotifierIdentifier.Em), - maybeControllingEm = controllingEm, - rootEmConfig = None, - ) - }.toMap - - recursiveEms ++ controlledEms - } else { - existingAndUncontrolledEms - } - } - - // all ems that control at least one participant directly + // ems that control at least one participant directly val firstLevelEms = participants.flatMap { _.getControllingEm.toScala.map(em => em.getUuid -> em) }.toMap - val allEms = buildEmsRecursively(firstLevelEms, Map.empty) + val allEms = buildEmsRecursively( + participantConfigUtil, + outputConfigUtil, + rootEmConfig, + firstLevelEms, + Map.empty, + ) participants .map { participant => @@ -278,6 +230,66 @@ class GridAgentController( .groupMap(entry => entry._1)(entry => entry._2) } + private def buildEmsRecursively( + participantConfigUtil: ConfigUtil.ParticipantConfigUtil, + outputConfigUtil: OutputConfigUtil, + rootEmConfig: Option[SimonaConfig.Simona.Runtime.RootEm], + emInputs: Map[UUID, EmInput], + existingEms: Map[UUID, ActorRef[EmMessage]], + ): Map[UUID, ActorRef[EmMessage]] = { + val (controlledEmInputs, uncontrolledEms) = emInputs + .partitionMap { case (uuid, emInput) => + if (emInput.getControllingEm.isPresent) + Left(uuid -> emInput) + else { + val actor = buildEm( + emInput, + participantConfigUtil.getOrDefault[EmRuntimeConfig](uuid), + outputConfigUtil.getOrDefault(NotifierIdentifier.Em), + maybeControllingEm = None, + rootEmConfig = rootEmConfig, + ) + Right(uuid -> actor) + } + } + + val existingAndUncontrolledEms = existingEms ++ uncontrolledEms.toMap + + if (controlledEmInputs.nonEmpty) { + // EMs that are controlling EMs at this level + val controllingEms = controlledEmInputs.toMap.flatMap { + case (uuid, emInput) => + emInput.getControllingEm.toScala.map(uuid -> _) + } + + val recursiveEms = buildEmsRecursively( + participantConfigUtil, + outputConfigUtil, + rootEmConfig, + controllingEms, + existingAndUncontrolledEms, + ) + + val controlledEms = controlledEmInputs.map { case (uuid, emInput) => + val controllingEm = emInput.getControllingEm.toScala + .map(_.getUuid) + .flatMap(uuid => recursiveEms.get(uuid)) + + uuid -> buildEm( + emInput, + participantConfigUtil.getOrDefault[EmRuntimeConfig](uuid), + outputConfigUtil.getOrDefault(NotifierIdentifier.Em), + maybeControllingEm = controllingEm, + rootEmConfig = None, + ) + }.toMap + + recursiveEms ++ controlledEms + } else { + existingAndUncontrolledEms + } + } + private def buildParticipantActor( requestVoltageDeviationThreshold: Double, participantConfigUtil: ConfigUtil.ParticipantConfigUtil, From b2a2c6e6b0e752de0996a8b218131cb47c7d0883 Mon Sep 17 00:00:00 2001 From: Sebastian Peter Date: Mon, 11 Mar 2024 11:37:59 +0100 Subject: [PATCH 10/32] Removing root-em/flex signal code for now --- .../resources/config/config-template.conf | 8 - .../edu/ie3/simona/agent/em/EmAgent.scala | 25 +-- .../edu/ie3/simona/agent/grid/GridAgent.scala | 1 - .../agent/grid/GridAgentController.scala | 16 +- .../edu/ie3/simona/config/SimonaConfig.scala | 65 ------- .../ie3/simona/model/em/FlexTimeSeries.scala | 107 ----------- .../util/scala/io/FlexSignalFromExcel.scala | 176 ------------------ .../edu/ie3/simona/agent/em/EmAgentIT.scala | 1 - .../edu/ie3/simona/agent/em/EmAgentSpec.scala | 4 - .../scala/io/FlexSignalFromExcelSpec.scala | 48 ----- 10 files changed, 7 insertions(+), 444 deletions(-) delete mode 100644 src/main/scala/edu/ie3/simona/model/em/FlexTimeSeries.scala delete mode 100644 src/main/scala/edu/ie3/util/scala/io/FlexSignalFromExcel.scala delete mode 100644 src/test/scala/edu/ie3/util/scala/io/FlexSignalFromExcelSpec.scala diff --git a/src/main/resources/config/config-template.conf b/src/main/resources/config/config-template.conf index d4588a3521..f9ace9eca9 100644 --- a/src/main/resources/config/config-template.conf +++ b/src/main/resources/config/config-template.conf @@ -326,14 +326,6 @@ simona.runtime.participant = { } } -#@optional -simona.runtime.rootEm = { - filePath: String - nodeId: String - timeSeriesType: String - threshold: Double -} - ################################################################## # Power Flow Configuration ################################################################## 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 6ae72af0cb..5c52a02561 100644 --- a/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala +++ b/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala @@ -10,7 +10,6 @@ import edu.ie3.datamodel.models.input.EmInput import edu.ie3.datamodel.models.result.system.{EmResult, FlexOptionsResult} import edu.ie3.simona.agent.participant.data.Data.PrimaryData.ApparentPower import edu.ie3.simona.agent.participant.statedata.BaseStateData.FlexControlledData -import edu.ie3.simona.config.SimonaConfig import edu.ie3.simona.config.SimonaConfig.EmRuntimeConfig import edu.ie3.simona.event.ResultEvent import edu.ie3.simona.event.ResultEvent.{ @@ -19,7 +18,7 @@ import edu.ie3.simona.event.ResultEvent.{ } import edu.ie3.simona.event.notifier.NotifierConfig import edu.ie3.simona.exceptions.CriticalFailureException -import edu.ie3.simona.model.em.{EmModelShell, EmTools, FlexTimeSeries} +import edu.ie3.simona.model.em.{EmModelShell, EmTools} import edu.ie3.simona.ontology.messages.SchedulerMessage.{ Completion, ScheduleActivation, @@ -91,8 +90,6 @@ object EmAgent { * Either a [[Right]] with a reference to the parent [[EmAgent]] if this * agent is em-controlled, or a [[Left]] with a reference to the scheduler * that is activating this agent - * @param maybeRootEmConfig - * Config for the root EM agent, if applicable * @param listener * A collection of result event listeners */ @@ -103,7 +100,6 @@ object EmAgent { modelStrategy: String, simulationStartDate: ZonedDateTime, parent: Either[ActorRef[SchedulerMessage], ActorRef[FlexResponse]], - maybeRootEmConfig: Option[SimonaConfig.Simona.Runtime.RootEm] = None, listener: Iterable[ActorRef[ResultEvent]], ): Behavior[EmMessage] = Behaviors.setup { ctx => val constantData = EmData( @@ -130,9 +126,6 @@ object EmAgent { SchedulerData(scheduler, activationAdapter) } }, - maybeRootEmConfig.map( - FlexTimeSeries(_)(simulationStartDate) - ), listener, ) @@ -278,16 +271,9 @@ object EmAgent { awaitingFlexCtrl(updatedEmData, modelShell, updatedCore) case Left(_) => - // if we're not EM-controlled ourselves, we're determining the set points - // either via flex time series or as 0 kW - val setPower = emData.flexTimeSeries match { - case Some(_) => - throw new NotImplementedError( - "Flex time series are currently not implemented" - ) - - case None => Kilowatts(0) - } + // We're not em-controlled ourselves, + // always desire to come as close as possible to 0 kW + val setPower = Kilowatts(0) val flexControl = modelShell.determineFlexControl(allFlexOptions, setPower) @@ -455,8 +441,6 @@ object EmAgent { * @param parentData * Either a [[Right]] with [[FlexControlledData]] if this agent is * em-controlled, or a [[Left]] with [[SchedulerData]] - * @param flexTimeSeries - * Flex time series if this is the root EM * @param listener * A collection of result event listeners */ @@ -464,7 +448,6 @@ object EmAgent { outputConfig: NotifierConfig, simulationStartDate: ZonedDateTime, parentData: Either[SchedulerData, FlexControlledData], - flexTimeSeries: Option[FlexTimeSeries], listener: Iterable[ActorRef[ResultEvent]], ) 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 ca8b804ff4..8cd2a6b1f6 100644 --- a/src/main/scala/edu/ie3/simona/agent/grid/GridAgent.scala +++ b/src/main/scala/edu/ie3/simona/agent/grid/GridAgent.scala @@ -136,7 +136,6 @@ object GridAgent extends DBFSAlgorithm { TimeUtil.withDefaults .toZonedDateTime(constantData.simonaConfig.simona.time.endDateTime), constantData.simonaConfig.simona.runtime.participant, - constantData.simonaConfig.simona.runtime.rootEm, constantData.simonaConfig.simona.output.participant, constantData.resolution, constantData.listener, diff --git a/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala b/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala index 9e2ce7e71b..4d774c4ada 100644 --- a/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala +++ b/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala @@ -76,7 +76,6 @@ class GridAgentController( simulationStartDate: ZonedDateTime, simulationEndDate: ZonedDateTime, participantsConfig: SimonaConfig.Simona.Runtime.Participant, - rootEmConfig: Option[SimonaConfig.Simona.Runtime.RootEm], outputConfig: SimonaConfig.Simona.Output.Participant, resolution: Long, listener: Iterable[ActorRef[ResultEvent]], @@ -97,7 +96,6 @@ class GridAgentController( systemParticipants, thermalIslandGridsByBusId, environmentRefs, - rootEmConfig, ) } @@ -188,7 +186,6 @@ class GridAgentController( participants: Vector[SystemParticipantInput], thermalIslandGridsByBusId: Map[UUID, ThermalGrid], environmentRefs: EnvironmentRefs, - rootEmConfig: Option[SimonaConfig.Simona.Runtime.RootEm], ): Map[UUID, Set[ActorRef[ParticipantMessage]]] = { /* Prepare the config util for the participant models, which (possibly) utilizes as map to speed up the initialization * phase */ @@ -204,7 +201,6 @@ class GridAgentController( val allEms = buildEmsRecursively( participantConfigUtil, outputConfigUtil, - rootEmConfig, firstLevelEms, Map.empty, ) @@ -233,7 +229,6 @@ class GridAgentController( private def buildEmsRecursively( participantConfigUtil: ConfigUtil.ParticipantConfigUtil, outputConfigUtil: OutputConfigUtil, - rootEmConfig: Option[SimonaConfig.Simona.Runtime.RootEm], emInputs: Map[UUID, EmInput], existingEms: Map[UUID, ActorRef[EmMessage]], ): Map[UUID, ActorRef[EmMessage]] = { @@ -247,7 +242,6 @@ class GridAgentController( participantConfigUtil.getOrDefault[EmRuntimeConfig](uuid), outputConfigUtil.getOrDefault(NotifierIdentifier.Em), maybeControllingEm = None, - rootEmConfig = rootEmConfig, ) Right(uuid -> actor) } @@ -265,7 +259,6 @@ class GridAgentController( val recursiveEms = buildEmsRecursively( participantConfigUtil, outputConfigUtil, - rootEmConfig, controllingEms, existingAndUncontrolledEms, ) @@ -280,7 +273,6 @@ class GridAgentController( participantConfigUtil.getOrDefault[EmRuntimeConfig](uuid), outputConfigUtil.getOrDefault(NotifierIdentifier.Em), maybeControllingEm = controllingEm, - rootEmConfig = None, ) }.toMap @@ -773,21 +765,19 @@ class GridAgentController( modelConfiguration: EmRuntimeConfig, outputConfig: NotifierConfig, maybeControllingEm: Option[ActorRef[FlexResponse]] = None, - rootEmConfig: Option[SimonaConfig.Simona.Runtime.RootEm] = None, ): ActorRef[EmMessage] = gridAgentContext.spawn( EmAgent( emInput, modelConfiguration, outputConfig, - rootEmConfig - .map(_ => "PROPORTIONAL") - .getOrElse("PRIORITIZED"), + maybeControllingEm + .map(_ => "PRIORITIZED") + .getOrElse("PROPORTIONAL"), simulationStartDate, maybeControllingEm.toRight( environmentRefs.scheduler ), - rootEmConfig, listener, ), actorName(classOf[EmAgent.type], emInput.getId), diff --git a/src/main/scala/edu/ie3/simona/config/SimonaConfig.scala b/src/main/scala/edu/ie3/simona/config/SimonaConfig.scala index 5f56e75ba2..9e3bc48802 100644 --- a/src/main/scala/edu/ie3/simona/config/SimonaConfig.scala +++ b/src/main/scala/edu/ie3/simona/config/SimonaConfig.scala @@ -2266,7 +2266,6 @@ object SimonaConfig { final case class Runtime( listener: SimonaConfig.Simona.Runtime.Listener, participant: SimonaConfig.Simona.Runtime.Participant, - rootEm: scala.Option[SimonaConfig.Simona.Runtime.RootEm], selected_subgrids: scala.Option[scala.List[scala.Int]], selected_volt_lvls: scala.Option[scala.List[SimonaConfig.VoltLvlConfig]], ) @@ -2689,60 +2688,6 @@ object SimonaConfig { } } - final case class RootEm( - filePath: java.lang.String, - nodeId: java.lang.String, - threshold: scala.Double, - timeSeriesType: java.lang.String, - ) - object RootEm { - def apply( - c: com.typesafe.config.Config, - parentPath: java.lang.String, - $tsCfgValidator: $TsCfgValidator, - ): SimonaConfig.Simona.Runtime.RootEm = { - SimonaConfig.Simona.Runtime.RootEm( - filePath = $_reqStr(parentPath, c, "filePath", $tsCfgValidator), - nodeId = $_reqStr(parentPath, c, "nodeId", $tsCfgValidator), - threshold = $_reqDbl(parentPath, c, "threshold", $tsCfgValidator), - timeSeriesType = - $_reqStr(parentPath, c, "timeSeriesType", $tsCfgValidator), - ) - } - private def $_reqDbl( - parentPath: java.lang.String, - c: com.typesafe.config.Config, - path: java.lang.String, - $tsCfgValidator: $TsCfgValidator, - ): scala.Double = { - if (c == null) 0 - else - try c.getDouble(path) - catch { - case e: com.typesafe.config.ConfigException => - $tsCfgValidator.addBadPath(parentPath + path, e) - 0 - } - } - - private def $_reqStr( - parentPath: java.lang.String, - c: com.typesafe.config.Config, - path: java.lang.String, - $tsCfgValidator: $TsCfgValidator, - ): java.lang.String = { - if (c == null) null - else - try c.getString(path) - catch { - case e: com.typesafe.config.ConfigException => - $tsCfgValidator.addBadPath(parentPath + path, e) - null - } - } - - } - def apply( c: com.typesafe.config.Config, parentPath: java.lang.String, @@ -2761,16 +2706,6 @@ object SimonaConfig { parentPath + "participant.", $tsCfgValidator, ), - rootEm = - if (c.hasPathOrNull("rootEm")) - scala.Some( - SimonaConfig.Simona.Runtime.RootEm( - c.getConfig("rootEm"), - parentPath + "rootEm.", - $tsCfgValidator, - ) - ) - else None, selected_subgrids = if (c.hasPathOrNull("selected_subgrids")) scala.Some( diff --git a/src/main/scala/edu/ie3/simona/model/em/FlexTimeSeries.scala b/src/main/scala/edu/ie3/simona/model/em/FlexTimeSeries.scala deleted file mode 100644 index 0ecfa80eaf..0000000000 --- a/src/main/scala/edu/ie3/simona/model/em/FlexTimeSeries.scala +++ /dev/null @@ -1,107 +0,0 @@ -/* - * © 2023. TU Dortmund University, - * Institute of Energy Systems, Energy Efficiency and Energy Economics, - * Research group Distribution grid planning and operation - */ - -package edu.ie3.simona.model.em - -import edu.ie3.datamodel.models.timeseries.individual.IndividualTimeSeries -import edu.ie3.datamodel.models.value.PValue -import edu.ie3.simona.config.SimonaConfig -import edu.ie3.simona.util.TickUtil.TickLong -import edu.ie3.util.scala.io.FlexSignalFromExcel -import squants.Power -import squants.energy.Kilowatts - -import java.time.ZonedDateTime -import java.time.temporal.ChronoUnit -import scala.util.{Failure, Success} -import edu.ie3.util.quantities.PowerSystemUnits - -import scala.jdk.OptionConverters.RichOptional - -final case class FlexTimeSeries( - timeSeries: IndividualTimeSeries[PValue], - startDateTime: ZonedDateTime, - resolutionHours: Int, - minValue: Power, - maxValue: Power, - threshold: Double, -) { - def get(tick: Long): Power = { - // round current time to flexTimeSeries.resolutionHours hrs - val currentDateTime = tick.toDateTime(startDateTime) - val currentHour = currentDateTime.getHour - val roundedHour = - currentHour - currentHour % resolutionHours - val roundedDateTime = currentDateTime - .withHour(roundedHour) - .withMinute(0) - .withSecond(0) - .withNano(0) - - timeSeries - .getTimeBasedValue(roundedDateTime) - .toScala - .getOrElse( - throw new RuntimeException( - s"Could not retrieve value for $roundedDateTime" - ) - ) - .getValue - .getP - .toScala - .map(p => Kilowatts(p.to(PowerSystemUnits.KILOWATT).getValue.doubleValue)) - .getOrElse( - throw new RuntimeException( - s"No value set for $roundedDateTime" - ) - ) - } -} - -object FlexTimeSeries { - - def apply( - config: SimonaConfig.Simona.Runtime.RootEm - )(implicit startDateTime: ZonedDateTime): FlexTimeSeries = { - val timeSeriesType = - FlexSignalFromExcel.TimeSeriesType(config.timeSeriesType) - val timeSeries = FlexSignalFromExcel - .flexSignals(config.filePath, config.nodeId, timeSeriesType) match { - case Success(timeSeries) => timeSeries - case Failure(exception) => throw exception - } - - val resolutionHours = - if (timeSeries.getEntries.size() < 2) - throw new RuntimeException( - s"Less than two entries for flex time series ${config.nodeId}" - ) - else { - val valueIt = timeSeries.getEntries.iterator() - val entry1 = valueIt.next().getTime - val entry2 = valueIt.next().getTime - - ChronoUnit.HOURS.between(entry1, entry2).intValue - } - - // in case of resLoad we use totalResload (considering Simona participants) for min max setting - val (minValue, maxValue) = - FlexSignalFromExcel.getCorrespondingMinMaxValues( - timeSeriesType, - timeSeries, - config, - ) - - FlexTimeSeries( - timeSeries, - startDateTime, - resolutionHours, - minValue, - maxValue, - config.threshold, - ) - } -} diff --git a/src/main/scala/edu/ie3/util/scala/io/FlexSignalFromExcel.scala b/src/main/scala/edu/ie3/util/scala/io/FlexSignalFromExcel.scala deleted file mode 100644 index 6852897f35..0000000000 --- a/src/main/scala/edu/ie3/util/scala/io/FlexSignalFromExcel.scala +++ /dev/null @@ -1,176 +0,0 @@ -/* - * © 2022. TU Dortmund University, - * Institute of Energy Systems, Energy Efficiency and Energy Economics, - * Research group Distribution grid planning and operation - */ - -package edu.ie3.util.scala.io - -import edu.ie3.datamodel.models.timeseries.individual.{ - IndividualTimeSeries, - TimeBasedValue, -} -import edu.ie3.datamodel.models.value.PValue -import edu.ie3.simona.config.SimonaConfig.Simona.Runtime.RootEm -import edu.ie3.util.quantities.PowerSystemUnits -import org.apache.poi.ss.usermodel.CellType -import org.apache.poi.xssf.usermodel.XSSFWorkbook -import squants.energy.Kilowatts -import tech.units.indriya.quantity.Quantities - -import java.io.{File, FileInputStream} -import java.time.ZoneId -import java.util.UUID -import javax.measure -import javax.measure.quantity.Power -import scala.jdk.CollectionConverters.{ - IterableHasAsScala, - IteratorHasAsScala, - SetHasAsJava, -} -import scala.jdk.OptionConverters.RichOptional -import scala.util.{Failure, Success, Try, Using} - -object FlexSignalFromExcel { - - /** Gets flexibility signals from an Excel file with defined format. - * @param filePath - * Path to the excel file - * @param nodeId - * The node id, i.e. name of the sheet - * @param timeSeriesType - * The time series type - * @param unit - * Physical unit of input data - * @param zoneId - * Time zone of the incorporated date times - * @return - * A trial to the resulting time series - */ - def flexSignals( - filePath: String, - nodeId: String, - timeSeriesType: TimeSeriesType.Value = TimeSeriesType.ResidualLoad, - // ATTENTION: the unit configured here is not the output unit as `PValues` are transformed to kW by default - unit: measure.Unit[Power] = PowerSystemUnits.MEGAWATT, - zoneId: ZoneId = ZoneId.of("UTC"), - ): Try[IndividualTimeSeries[PValue]] = { - Using { - val file = new File(filePath) - val fileStream = new FileInputStream(file) - new XSSFWorkbook(fileStream) - } { workbook => - val sheet = workbook.getSheet(nodeId) - - val rows = sheet.rowIterator().asScala - /* Assess the table header */ - val header = - rows.next().cellIterator().asScala.toList.map(_.getStringCellValue) - val dateTimeIndex = header.indexOf("time") - val colToTimeseriesType = header.zipWithIndex - .filterNot(_._1 == "time") - .map { case (token, idx) => - idx -> TimeSeriesType(token) - } - .toMap - - /* Go through the file and parse the contents */ - val values = rows.toList - .filterNot(_.getCell(0).getCellType == CellType.BLANK) - .flatMap { row => - val dateTime = row - .getCell(dateTimeIndex) - .getLocalDateTimeCellValue - .atZone(zoneId) - - val tsTypeToValue = colToTimeseriesType.map { case (col, tsType) => - // negate the flex signal to get residual power - val raw = -row.getCell(col).getNumericCellValue - val value = new PValue( - Quantities.getQuantity(raw, unit) - ) - val timeBasedValue = - new TimeBasedValue[PValue](dateTime, value) - tsType -> timeBasedValue - } - tsTypeToValue - } - .groupBy(_._1) - .map { case (tsType, collection) => - tsType -> collection.map(_._2).toSet - } - - /* Finally process the single value lists into individual time series */ - val tsTypeToTs = values.map { case (tsType, valueSet) => - tsType -> new IndividualTimeSeries[PValue]( - UUID.randomUUID(), - valueSet.asJava, - ) - } - - tsTypeToTs(timeSeriesType) - } - } - - def getCorrespondingMinMaxValues( - timeSeriesType: TimeSeriesType.Value, - timeSeries: IndividualTimeSeries[PValue], - config: RootEm, - ): (squants.Power, squants.Power) = { - - // todo this is very use case dependant and has to be reworked - /* instead of using the residual load we take the total res load to determine min and max - values for threshold calculation as this also includes self oriented reference behavior of Simona*/ - val minMaxTs = if (timeSeriesType == TimeSeriesType.ResidualLoad) { - FlexSignalFromExcel - .flexSignals( - config.filePath, - config.nodeId, - TimeSeriesType.TotalResLoad, - ) match { - case Success(timeSeries) => timeSeries - case Failure(exception) => throw exception - } - } else timeSeries - val allValues = - minMaxTs.getEntries.asScala.flatMap(_.getValue.getP.toScala) - val maybeMinValue = allValues.minByOption( - _.to(PowerSystemUnits.KILOWATT).getValue.doubleValue - ) - val maybeMaxValue = allValues.maxByOption( - _.to(PowerSystemUnits.KILOWATT).getValue.doubleValue - ) - - val (minValue, maxValue) = maybeMinValue - .zip(maybeMaxValue) - .getOrElse( - throw new RuntimeException(s"Time series for $config is empty") - ) - ( - Kilowatts(minValue.getValue.doubleValue()), - Kilowatts(maxValue.getValue.doubleValue()), - ) - } - - object TimeSeriesType extends Enumeration { - val Generation, Load, OtherGeneration, OtherLoad, Import, ImportIntern, - ResidualLoad, TotalResLoad, SimonaGeneration, SimonaLoad = Value - - def apply(token: String): TimeSeriesType.Value = { - token match { - case "gen" => Generation - case "loads" => Load - case "otherGen" => OtherGeneration - case "otherLoads" => OtherLoad - case "importExport" => Import - case "importExport_intern" => ImportIntern - case "resLoad" => ResidualLoad - case "totalResLoad" => TotalResLoad - case "SIMONA_gen" => SimonaGeneration - case "SIMONA_load" => SimonaLoad - case _ => - throw new IllegalArgumentException(s"Don't know the token '$token'.") - } - } - } -} 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 900d9f7b8c..8b59df7439 100644 --- a/src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala +++ b/src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala @@ -113,7 +113,6 @@ class EmAgentIT "PRIORITIZED", simulationStartDate, parent = Left(scheduler.ref), - maybeRootEmConfig = None, listener = Iterable(resultListener.ref), ), "EmAgent1", 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 519d6af57f..7625e2c261 100644 --- a/src/test/scala/edu/ie3/simona/agent/em/EmAgentSpec.scala +++ b/src/test/scala/edu/ie3/simona/agent/em/EmAgentSpec.scala @@ -82,7 +82,6 @@ class EmAgentSpec "PRIORITIZED", simulationStartDate, parent = Left(scheduler.ref), - maybeRootEmConfig = None, listener = Iterable(resultListener.ref), ) ) @@ -217,7 +216,6 @@ class EmAgentSpec "PRIORITIZED", simulationStartDate, parent = Left(scheduler.ref), - maybeRootEmConfig = None, listener = Iterable(resultListener.ref), ) ) @@ -364,7 +362,6 @@ class EmAgentSpec "PRIORITIZED", simulationStartDate, parent = Left(scheduler.ref), - maybeRootEmConfig = None, listener = Iterable(resultListener.ref), ) ) @@ -522,7 +519,6 @@ class EmAgentSpec "PRIORITIZED", simulationStartDate, parent = Right(parentEmAgent.ref), - maybeRootEmConfig = None, listener = Iterable(resultListener.ref), ) ) diff --git a/src/test/scala/edu/ie3/util/scala/io/FlexSignalFromExcelSpec.scala b/src/test/scala/edu/ie3/util/scala/io/FlexSignalFromExcelSpec.scala deleted file mode 100644 index 15c881cbdb..0000000000 --- a/src/test/scala/edu/ie3/util/scala/io/FlexSignalFromExcelSpec.scala +++ /dev/null @@ -1,48 +0,0 @@ -/* - * © 2022. TU Dortmund University, - * Institute of Energy Systems, Energy Efficiency and Energy Economics, - * Research group Distribution grid planning and operation - */ - -package edu.ie3.util.scala.io - -import edu.ie3.simona.test.common.UnitSpec -import edu.ie3.util.scala.io.FlexSignalFromExcel.TimeSeriesType -import org.scalatest.prop.TableDrivenPropertyChecks - -import scala.util.{Failure, Success} - -class FlexSignalFromExcelSpec extends UnitSpec with TableDrivenPropertyChecks { - "Reading flexibility signal from file" should { - val file = getClass.getResource("flexSignal.xlsx").getFile - - val nodeIds = Seq("node0", "node1") - val combinations = nodeIds.flatMap { nodeId => - TimeSeriesType.values.map { seriesType => (nodeId, seriesType) } - } - - val cases = Table( - ("nodeId", "timeSeriesType"), - combinations: _* - ) - - "provide all expected time series with correct amount of entries" in { - - forAll(cases) { case (nodeId, timeSeriesType) => - val resultTrial = - FlexSignalFromExcel.flexSignals(file, nodeId, timeSeriesType) - - resultTrial match { - case Success(timeSeries) => - timeSeries.getEntries.size() shouldBe 8 - case Failure(exception) => - fail( - "Reading flex signal from excel file failed with the following exception.", - exception, - ) - } - } - } - - } -} From 6d0b27e3fee70003bbce5b5a791758fd2d24e597 Mon Sep 17 00:00:00 2001 From: Sebastian Peter Date: Mon, 11 Mar 2024 12:24:33 +0100 Subject: [PATCH 11/32] Documentation and cleaner code --- .../agent/grid/GridAgentController.scala | 56 +++++++++++++------ 1 file changed, 40 insertions(+), 16 deletions(-) diff --git a/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala b/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala index 4d774c4ada..c1eecba92d 100644 --- a/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala +++ b/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala @@ -202,7 +202,6 @@ class GridAgentController( participantConfigUtil, outputConfigUtil, firstLevelEms, - Map.empty, ) participants @@ -226,12 +225,31 @@ class GridAgentController( .groupMap(entry => entry._1)(entry => entry._2) } + /** Recursively builds the [[EmAgent]] structure. Recursion starts with + * first-level EMs (controlling at least one system participant), and works + * its way up to EMs at root level, which are not EM-controlled themselves. + * The first level can also be root level. + * + * @param participantConfigUtil + * Configuration util for participant models + * @param outputConfigUtil + * Configuration util for output behaviour + * @param emInputs + * EMs of the current level, which can be controlled by further EMs at + * higher levels + * @param lastLevelEms + * EMs that have been built by the previous recursion level + * @return + * Map from model UUID to EmAgent ActorRef + */ private def buildEmsRecursively( participantConfigUtil: ConfigUtil.ParticipantConfigUtil, outputConfigUtil: OutputConfigUtil, emInputs: Map[UUID, EmInput], - existingEms: Map[UUID, ActorRef[EmMessage]], + lastLevelEms: Map[UUID, ActorRef[EmMessage]] = Map.empty, ): Map[UUID, ActorRef[EmMessage]] = { + // For the current level, split controlled and uncontrolled EMs. + // Uncontrolled EMs can be built right away. val (controlledEmInputs, uncontrolledEms) = emInputs .partitionMap { case (uuid, emInput) => if (emInput.getControllingEm.isPresent) @@ -247,25 +265,29 @@ class GridAgentController( } } - val existingAndUncontrolledEms = existingEms ++ uncontrolledEms.toMap + val lastLevelAndUncontrolledEms = lastLevelEms ++ uncontrolledEms.toMap if (controlledEmInputs.nonEmpty) { - // EMs that are controlling EMs at this level + // For controlled EMs at the current level, more EMs + // might need to be built at the next recursion level. val controllingEms = controlledEmInputs.toMap.flatMap { case (uuid, emInput) => emInput.getControllingEm.toScala.map(uuid -> _) } + // Return value includes last level and uncontrolled EMs of this level val recursiveEms = buildEmsRecursively( participantConfigUtil, outputConfigUtil, controllingEms, - existingAndUncontrolledEms, + lastLevelAndUncontrolledEms, ) val controlledEms = controlledEmInputs.map { case (uuid, emInput) => val controllingEm = emInput.getControllingEm.toScala .map(_.getUuid) + // We do not have to throw errors here because PSDM + // already takes care of valid input data .flatMap(uuid => recursiveEms.get(uuid)) uuid -> buildEm( @@ -278,7 +300,7 @@ class GridAgentController( recursiveEms ++ controlledEms } else { - existingAndUncontrolledEms + lastLevelAndUncontrolledEms } } @@ -289,7 +311,7 @@ class GridAgentController( participantInputModel: SystemParticipantInput, thermalIslandGridsByBusId: Map[UUID, ThermalGrid], environmentRefs: EnvironmentRefs, - maybeControllingEm: Option[ActorRef[FlexResponse]] = None, + maybeControllingEm: Option[ActorRef[FlexResponse]], ): ActorRef[ParticipantMessage] = participantInputModel match { case input: FixedFeedInInput => buildFixedFeedIn( @@ -431,7 +453,7 @@ class GridAgentController( resolution: Long, requestVoltageDeviationThreshold: Double, outputConfig: NotifierConfig, - maybeControllingEm: Option[ActorRef[FlexResponse]] = None, + maybeControllingEm: Option[ActorRef[FlexResponse]], ): ActorRef[ParticipantMessage] = gridAgentContext.toClassic .simonaActorOf( @@ -488,7 +510,7 @@ class GridAgentController( resolution: Long, requestVoltageDeviationThreshold: Double, outputConfig: NotifierConfig, - maybeControllingEm: Option[ActorRef[FlexResponse]] = None, + maybeControllingEm: Option[ActorRef[FlexResponse]], ): ActorRef[ParticipantMessage] = gridAgentContext.toClassic .simonaActorOf( @@ -548,7 +570,7 @@ class GridAgentController( resolution: Long, requestVoltageDeviationThreshold: Double, outputConfig: NotifierConfig, - maybeControllingEm: Option[ActorRef[FlexResponse]] = None, + maybeControllingEm: Option[ActorRef[FlexResponse]], ): ActorRef[ParticipantMessage] = gridAgentContext.toClassic .simonaActorOf( @@ -608,7 +630,7 @@ class GridAgentController( resolution: Long, requestVoltageDeviationThreshold: Double, outputConfig: NotifierConfig, - maybeControllingEm: Option[ActorRef[FlexResponse]] = None, + maybeControllingEm: Option[ActorRef[FlexResponse]], ): ActorRef[ParticipantMessage] = gridAgentContext.toClassic .simonaActorOf( @@ -664,7 +686,7 @@ class GridAgentController( weatherService: ClassicRef, requestVoltageDeviationThreshold: Double, outputConfig: NotifierConfig, - maybeControllingEm: Option[ActorRef[FlexResponse]] = None, + maybeControllingEm: Option[ActorRef[FlexResponse]], ): ActorRef[ParticipantMessage] = gridAgentContext.toClassic .simonaActorOf( @@ -725,7 +747,7 @@ class GridAgentController( resolution: Long, requestVoltageDeviationThreshold: Double, outputConfig: NotifierConfig, - maybeControllingEm: Option[ActorRef[FlexResponse]] = None, + maybeControllingEm: Option[ActorRef[FlexResponse]], ): ActorRef[ParticipantMessage] = gridAgentContext.toClassic .simonaActorOf( @@ -752,19 +774,21 @@ class GridAgentController( /** Builds an [[EmAgent]] from given input * * @param emInput - * Input model + * The input model * @param modelConfiguration * Runtime configuration for the agent * @param outputConfig * Configuration for output notification + * @param maybeControllingEm + * The parent EmAgent, if applicable * @return - * A tuple of actor reference and [[ParticipantInitializeStateData]] + * The [[EmAgent]] 's [[ActorRef]] */ private def buildEm( emInput: EmInput, modelConfiguration: EmRuntimeConfig, outputConfig: NotifierConfig, - maybeControllingEm: Option[ActorRef[FlexResponse]] = None, + maybeControllingEm: Option[ActorRef[FlexResponse]], ): ActorRef[EmMessage] = gridAgentContext.spawn( EmAgent( From 01d883352afa099151b18eea7d673603b2afb65d Mon Sep 17 00:00:00 2001 From: Sebastian Peter Date: Mon, 11 Mar 2024 12:25:56 +0100 Subject: [PATCH 12/32] System participants are now directly connected to the grid --- .../ie3/simona/agent/grid/GridAgentController.scala | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala b/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala index c1eecba92d..8947529056 100644 --- a/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala +++ b/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala @@ -101,8 +101,8 @@ class GridAgentController( /** Takes the provided [[SubGridContainer]] and removes all * [[SystemParticipantInput]] of which no agent implementations are available - * at the moment or which are connected to some EM system. This method needs - * to be adapted whenever a new agent implementation is ready. + * at the moment. This method needs to be adapted whenever a new agent + * implementation is ready. * * To disable a filter for a specific system participant, adapt the code * below. @@ -141,12 +141,6 @@ class GridAgentController( s"Evcs ${evcsInput.getId} has been removed because no ev movements service is present." ) (notProcessedElements, availableSystemParticipants) - case entity if entity.getControllingEm.isPresent => - log.debug( - s"System participant {} is part of an energy-managed system and thus not directly connected to the grid.", - entity, - ) - (notProcessedElements, availableSystemParticipants) case entity => (notProcessedElements, availableSystemParticipants :+ entity) } From e3a0a6f9000cd358f932097e157b5173842ca5fa Mon Sep 17 00:00:00 2001 From: Sebastian Peter Date: Mon, 11 Mar 2024 12:37:10 +0100 Subject: [PATCH 13/32] Removing unnecessary todo --- src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 8b59df7439..7cfa3dc8af 100644 --- a/src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala +++ b/src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala @@ -147,7 +147,7 @@ class EmAgentIT new PvAgent( scheduler = scheduler.ref.toClassic, initStateData = ParticipantInitializeStateData( - pvInput, // TODO pv em + pvInput, PvRuntimeConfig( calculateMissingReactivePowerWithModel = true, scaling = 2d, From cc72ad87ee753594fc90fe64fcda0272cff24f2e Mon Sep 17 00:00:00 2001 From: Sebastian Peter Date: Mon, 11 Mar 2024 12:39:04 +0100 Subject: [PATCH 14/32] Follow the message naming convention --- .../edu/ie3/simona/agent/em/EmAgent.scala | 24 +++++++++---------- .../agent/grid/GridAgentController.scala | 7 +++--- .../messages/flex/FlexibilityMessage.scala | 4 ++-- 3 files changed, 17 insertions(+), 18 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 5c52a02561..f2deabb007 100644 --- a/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala +++ b/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala @@ -46,12 +46,12 @@ object EmAgent { /** Extended by all messages that an [[EmAgent]] can receive */ - trait EmMessage + trait Request - /** Extended by all requests that an [[EmAgent]] can receive, i.e. - * activations, flex requests and control messages + /** Extended by all requests that activate an [[EmAgent]], i.e. activations, + * flex requests and control messages */ - private sealed trait EmRequest extends EmMessage { + private sealed trait ActivationRequest extends Request { val tick: Long } @@ -62,7 +62,7 @@ object EmAgent { * The tick to activate */ private final case class EmActivation(override val tick: Long) - extends EmRequest + extends ActivationRequest /** Wrapper for [[FlexRequest]] messages for usage by an adapter (if this * [[EmAgent]] is EM-controlled itself) @@ -70,7 +70,7 @@ object EmAgent { * @param msg * The wrapped flex request */ - private final case class Flex(msg: FlexRequest) extends EmRequest { + private final case class Flex(msg: FlexRequest) extends ActivationRequest { override val tick: Long = msg.tick } @@ -101,7 +101,7 @@ object EmAgent { simulationStartDate: ZonedDateTime, parent: Either[ActorRef[SchedulerMessage], ActorRef[FlexResponse]], listener: Iterable[ActorRef[ResultEvent]], - ): Behavior[EmMessage] = Behaviors.setup { ctx => + ): Behavior[Request] = Behaviors.setup[Request] { ctx => val constantData = EmData( outputConfig, simulationStartDate, @@ -150,7 +150,7 @@ object EmAgent { emData: EmData, modelShell: EmModelShell, core: EmDataCore.Inactive, - ): Behavior[EmMessage] = Behaviors.receivePartial { + ): Behavior[Request] = Behaviors.receivePartial { case (_, RegisterParticipant(model, actor, spi)) => val updatedModelShell = modelShell.addParticipant(model, spi) @@ -188,7 +188,7 @@ object EmAgent { } inactive(emData, modelShell, newCore) - case (ctx, msg: EmRequest) => + case (ctx, msg: ActivationRequest) => val flexOptionsCore = core.activate(msg.tick) msg match { @@ -221,7 +221,7 @@ object EmAgent { emData: EmData, modelShell: EmModelShell, flexOptionsCore: EmDataCore.AwaitingFlexOptions, - ): Behavior[EmMessage] = Behaviors.receiveMessagePartial { + ): Behavior[Request] = Behaviors.receiveMessagePartial { case flexOptions: ProvideFlexOptions => val updatedCore = flexOptionsCore.handleFlexOptions(flexOptions) @@ -313,7 +313,7 @@ object EmAgent { emData: EmData, modelShell: EmModelShell, flexOptionsCore: EmDataCore.AwaitingFlexOptions, - ): Behavior[EmMessage] = Behaviors.receiveMessagePartial { + ): Behavior[Request] = Behaviors.receiveMessagePartial { case Flex(flexCtrl: IssueFlexControl) => val flexData = emData.parentData.getOrElse( throw new CriticalFailureException(s"EmAgent is not EM-controlled.") @@ -360,7 +360,7 @@ object EmAgent { emData: EmData, modelShell: EmModelShell, core: EmDataCore.AwaitingCompletions, - ): Behavior[EmMessage] = Behaviors.receiveMessagePartial { + ): Behavior[Request] = Behaviors.receiveMessagePartial { // Completions and results case completion: FlexCtrlCompletion => val updatedCore = core.handleCompletion(completion) diff --git a/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala b/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala index 8947529056..f22667ebca 100644 --- a/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala +++ b/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala @@ -13,7 +13,6 @@ import edu.ie3.datamodel.models.input.system._ import edu.ie3.simona.actor.SimonaActorNaming._ import edu.ie3.simona.agent.EnvironmentRefs import edu.ie3.simona.agent.em.EmAgent -import edu.ie3.simona.agent.em.EmAgent.EmMessage import edu.ie3.simona.agent.participant.ParticipantAgent.ParticipantMessage import edu.ie3.simona.agent.participant.data.secondary.SecondaryDataService.{ ActorEvMovementsService, @@ -240,8 +239,8 @@ class GridAgentController( participantConfigUtil: ConfigUtil.ParticipantConfigUtil, outputConfigUtil: OutputConfigUtil, emInputs: Map[UUID, EmInput], - lastLevelEms: Map[UUID, ActorRef[EmMessage]] = Map.empty, - ): Map[UUID, ActorRef[EmMessage]] = { + lastLevelEms: Map[UUID, ActorRef[FlexResponse]] = Map.empty, + ): Map[UUID, ActorRef[FlexResponse]] = { // For the current level, split controlled and uncontrolled EMs. // Uncontrolled EMs can be built right away. val (controlledEmInputs, uncontrolledEms) = emInputs @@ -783,7 +782,7 @@ class GridAgentController( modelConfiguration: EmRuntimeConfig, outputConfig: NotifierConfig, maybeControllingEm: Option[ActorRef[FlexResponse]], - ): ActorRef[EmMessage] = + ): ActorRef[FlexResponse] = gridAgentContext.spawn( EmAgent( emInput, diff --git a/src/main/scala/edu/ie3/simona/ontology/messages/flex/FlexibilityMessage.scala b/src/main/scala/edu/ie3/simona/ontology/messages/flex/FlexibilityMessage.scala index a4574054ad..812eafbfff 100644 --- a/src/main/scala/edu/ie3/simona/ontology/messages/flex/FlexibilityMessage.scala +++ b/src/main/scala/edu/ie3/simona/ontology/messages/flex/FlexibilityMessage.scala @@ -7,7 +7,7 @@ package edu.ie3.simona.ontology.messages.flex import edu.ie3.datamodel.models.input.AssetInput -import edu.ie3.simona.agent.em.EmAgent.EmMessage +import edu.ie3.simona.agent.em.EmAgent import edu.ie3.simona.agent.participant.data.Data.PrimaryData.ApparentPower import edu.ie3.simona.scheduler.ScheduleLock.ScheduleKey import org.apache.pekko.actor.typed.ActorRef @@ -33,7 +33,7 @@ object FlexibilityMessage { /** Trait that is extended by all messages that are supposed to be received by * [[edu.ie3.simona.agent.em.EmAgent]]s. */ - sealed trait FlexResponse extends EmMessage { + sealed trait FlexResponse extends EmAgent.Request { val modelUuid: UUID } From 29440f4b23b362795770e47682c1adbe4f95708e Mon Sep 17 00:00:00 2001 From: Sebastian Peter Date: Mon, 11 Mar 2024 13:37:40 +0100 Subject: [PATCH 15/32] Implementing phase switch for MobSim+EM combinations --- .../scala/edu/ie3/simona/sim/SimonaSim.scala | 52 ++++---- .../simona/sim/setup/ExtSimSetupData.scala | 11 +- .../ie3/simona/sim/setup/SimonaSetup.scala | 16 ++- .../sim/setup/SimonaStandaloneSetup.scala | 120 ++++++++++-------- .../edu/ie3/simona/sim/SimonaSimSpec.scala | 15 ++- .../simona/sim/setup/SimonaSetupSpec.scala | 7 +- 6 files changed, 128 insertions(+), 93 deletions(-) diff --git a/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala b/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala index f62dc5e45a..9126926147 100644 --- a/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala +++ b/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala @@ -12,6 +12,7 @@ import edu.ie3.simona.event.RuntimeEvent import edu.ie3.simona.event.listener.{DelayedStopHelper, RuntimeEventListener} import edu.ie3.simona.main.RunSimona.SimonaEnded import edu.ie3.simona.scheduler.TimeAdvancer +import edu.ie3.simona.scheduler.core.PhaseSwitchCore import edu.ie3.simona.sim.setup.{ExtSimSetupData, SimonaSetup} import edu.ie3.util.scala.Scope import org.apache.pekko.actor.typed.scaladsl.adapter._ @@ -76,22 +77,30 @@ object SimonaSim { val timeAdvancer = simonaSetup.timeAdvancer(ctx, ctx.self, runtimeEventListener) - val scheduler = simonaSetup.scheduler(ctx, timeAdvancer) + val rootPhaseSwitch = + simonaSetup.scheduler(ctx, timeAdvancer, PhaseSwitchCore) + + // External simulations have to be scheduled for initialization first, + // so that the phase switch permanently activates them first + val extSimulationData: ExtSimSetupData = + simonaSetup.extSimulations(ctx, rootPhaseSwitch) + + // scheduler for all actors besides external simulation, + // which come second in line with phase switch + val simScheduler = + simonaSetup.scheduler(ctx, rootPhaseSwitch) /* start services */ // primary service proxy val primaryServiceProxy = - simonaSetup.primaryServiceProxy(ctx, scheduler) + simonaSetup.primaryServiceProxy(ctx, simScheduler) // weather service val weatherService = - simonaSetup.weatherService(ctx, scheduler) - - val extSimulationData: ExtSimSetupData = - simonaSetup.extSimulations(ctx, scheduler) + simonaSetup.weatherService(ctx, simScheduler) val environmentRefs = EnvironmentRefs( - scheduler, + simScheduler, runtimeEventListener.toClassic, primaryServiceProxy, weatherService, @@ -105,31 +114,26 @@ object SimonaSim { resultEventListeners, ) + val otherActors = Iterable[ActorRef[_]]( + timeAdvancer, + rootPhaseSwitch, + simScheduler, + primaryServiceProxy.toTyped, + weatherService.toTyped, + ) ++ + gridAgents ++ + extSimulationData.extDataServices.values.map(_.toTyped) ++ + extSimulationData.extScheduler.toSeq + /* watch all actors */ resultEventListeners.foreach(ctx.watch) ctx.watch(runtimeEventListener) - ctx.watch(timeAdvancer) - ctx.watch(scheduler) - ctx.watch(primaryServiceProxy.toTyped) - ctx.watch(weatherService.toTyped) extSimulationData.extSimAdapters.map(_.toTyped).foreach(ctx.watch) - extSimulationData.extDataServices.values - .map(_.toTyped) - .foreach(ctx.watch) - gridAgents.foreach(ref => ctx.watch(ref)) + otherActors.foreach(ctx.watch) // Start simulation timeAdvancer ! TimeAdvancer.Start() - val otherActors = Iterable( - timeAdvancer, - scheduler, - primaryServiceProxy.toTyped, - weatherService.toTyped, - ) ++ - gridAgents ++ - extSimulationData.extDataServices.values.map(_.toTyped) - idle( ActorData( starter, 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 d5b34606aa..40443cad5f 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/ExtSimSetupData.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/ExtSimSetupData.scala @@ -6,14 +6,17 @@ package edu.ie3.simona.sim.setup -import org.apache.pekko.actor.ActorRef +import edu.ie3.simona.ontology.messages.SchedulerMessage +import org.apache.pekko.actor.{ActorRef => ClassicRef} import edu.ie3.simona.service.ev.ExtEvDataService +import org.apache.pekko.actor.typed.ActorRef final case class ExtSimSetupData( - extSimAdapters: Iterable[ActorRef], - extDataServices: Map[Class[_], ActorRef], + extSimAdapters: Iterable[ClassicRef], + extDataServices: Map[Class[_], ClassicRef], + extScheduler: Option[ActorRef[SchedulerMessage]], ) { - def evDataService: Option[ActorRef] = + def evDataService: Option[ClassicRef] = extDataServices.get(classOf[ExtEvDataService]) } 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 16809499b0..f96a144a36 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaSetup.scala @@ -14,6 +14,8 @@ import edu.ie3.simona.event.listener.{ResultEventListener, RuntimeEventListener} import edu.ie3.simona.event.{ResultEvent, RuntimeEvent} import edu.ie3.simona.ontology.messages.SchedulerMessage 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.sim.SimonaSim import org.apache.pekko.actor.typed.ActorRef import org.apache.pekko.actor.typed.scaladsl.ActorContext @@ -93,14 +95,14 @@ trait SimonaSetup { * * @param context * Actor context to use - * @param scheduler + * @param rootScheduler * Actor reference to it's according scheduler to use * @return * External simulations and their init data */ def extSimulations( context: ActorContext[_], - scheduler: ActorRef[SchedulerMessage], + rootScheduler: ActorRef[SchedulerMessage], ): ExtSimSetupData /** Creates the time advancer @@ -124,14 +126,18 @@ trait SimonaSetup { * * @param context * Actor context to use - * @param timeAdvancer - * The time advancer, sitting at the root of the scheduler hierarchy + * @param parent + * 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 * @return * An actor reference to the scheduler */ def scheduler( context: ActorContext[_], - timeAdvancer: ActorRef[TimeAdvancer.Request], + parent: ActorRef[SchedulerMessage], + coreFactory: CoreFactory = RegularSchedulerCore, ): ActorRef[SchedulerMessage] /** Creates all the needed grid agents 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 322d02bc93..6788a652f2 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala @@ -26,6 +26,8 @@ import edu.ie3.simona.exceptions.agent.GridAgentInitializationException import edu.ie3.simona.io.grid.GridProvider import edu.ie3.simona.ontology.messages.SchedulerMessage import edu.ie3.simona.ontology.messages.SchedulerMessage.ScheduleActivation +import edu.ie3.simona.scheduler.core.Core.CoreFactory +import edu.ie3.simona.scheduler.core.RegularSchedulerCore import edu.ie3.simona.scheduler.{ScheduleLock, Scheduler, TimeAdvancer} import edu.ie3.simona.service.SimonaService import edu.ie3.simona.service.ev.ExtEvDataService @@ -48,6 +50,7 @@ import org.apache.pekko.actor.typed.scaladsl.adapter.{ } import org.apache.pekko.actor.{ActorRef => ClassicRef} +import java.util.UUID import java.util.concurrent.LinkedBlockingQueue import scala.jdk.CollectionConverters._ @@ -187,65 +190,75 @@ class SimonaStandaloneSetup( override def extSimulations( context: ActorContext[_], - scheduler: ActorRef[SchedulerMessage], + rootScheduler: ActorRef[SchedulerMessage], ): ExtSimSetupData = { val jars = ExtSimLoader.scanInputFolder() - val extLinks = jars.flatMap(ExtSimLoader.loadExtLink) + val extLinks = jars.flatMap(ExtSimLoader.loadExtLink).toSeq - val (extSimAdapters, extDataServices) = - extLinks.zipWithIndex.map { case (extLink, index) => - // external simulation always needs at least an ExtSimAdapter - val extSimAdapter = context.toClassic.simonaActorOf( - ExtSimAdapter.props(scheduler.toClassic), - s"$index", - ) - val extSimAdapterData = new ExtSimAdapterData(extSimAdapter, args) + if (extLinks.nonEmpty) { + val extScheduler = scheduler(context, parent = rootScheduler) - // send init data right away, init activation is scheduled - extSimAdapter ! ExtSimAdapter.Create( - extSimAdapterData, - ScheduleLock.singleKey(context, scheduler, INIT_SIM_TICK), - ) + val (extSimAdapters, extDataServices) = + extLinks.zipWithIndex.map { case (extLink, index) => + // external simulation always needs at least an ExtSimAdapter + val extSimAdapter = context.toClassic.simonaActorOf( + ExtSimAdapter.props(extScheduler.toClassic), + s"$index", + ) + val extSimAdapterData = new ExtSimAdapterData(extSimAdapter, args) - // setup data services that belong to this external simulation - val (extData, extDataInit): ( - Iterable[ExtData], - Iterable[(Class[_ <: SimonaService[_]], ClassicRef)], - ) = - extLink.getExtDataSimulations.asScala.zipWithIndex.map { - case (_: ExtEvSimulation, dIndex) => - val extEvDataService = context.toClassic.simonaActorOf( - ExtEvDataService.props(scheduler.toClassic), - s"$index-$dIndex", - ) - val extEvData = new ExtEvData(extEvDataService, extSimAdapter) - - extEvDataService ! SimonaService.Create( - InitExtEvData(extEvData), - ScheduleLock.singleKey( - context, - scheduler, - INIT_SIM_TICK, - ), - ) - - (extEvData, (classOf[ExtEvDataService], extEvDataService)) - }.unzip - - extLink.getExtSimulation.setup( - extSimAdapterData, - extData.toList.asJava, - ) + // send init data right away, init activation is scheduled + extSimAdapter ! ExtSimAdapter.Create( + extSimAdapterData, + ScheduleLock.singleKey(context, extScheduler, INIT_SIM_TICK), + ) + + // setup data services that belong to this external simulation + val (extData, extDataInit): ( + Iterable[ExtData], + Iterable[(Class[_ <: SimonaService[_]], ClassicRef)], + ) = + extLink.getExtDataSimulations.asScala.zipWithIndex.map { + case (_: ExtEvSimulation, dIndex) => + val extEvDataService = context.toClassic.simonaActorOf( + ExtEvDataService.props(extScheduler.toClassic), + s"$index-$dIndex", + ) + val extEvData = new ExtEvData(extEvDataService, extSimAdapter) + + extEvDataService ! SimonaService.Create( + InitExtEvData(extEvData), + ScheduleLock.singleKey( + context, + extScheduler, + INIT_SIM_TICK, + ), + ) + + (extEvData, (classOf[ExtEvDataService], extEvDataService)) + }.unzip + + extLink.getExtSimulation.setup( + extSimAdapterData, + extData.toList.asJava, + ) - // starting external simulation - new Thread(extLink.getExtSimulation, s"External simulation $index") - .start() + // starting external simulation + new Thread(extLink.getExtSimulation, s"External simulation $index") + .start() - (extSimAdapter, extDataInit) - }.unzip + (extSimAdapter, extDataInit) + }.unzip - ExtSimSetupData(extSimAdapters, extDataServices.flatten.toMap) + ExtSimSetupData( + extSimAdapters, + extDataServices.flatten.toMap, + Some(extScheduler), + ) + } else { + ExtSimSetupData(Iterable.empty, Map.empty, None) + } } override def timeAdvancer( @@ -273,12 +286,13 @@ class SimonaStandaloneSetup( override def scheduler( context: ActorContext[_], - timeAdvancer: ActorRef[TimeAdvancer.Request], + parent: ActorRef[SchedulerMessage], + coreFactory: CoreFactory = RegularSchedulerCore, ): ActorRef[SchedulerMessage] = context .spawn( - Scheduler(timeAdvancer), - Scheduler.getClass.getSimpleName, + Scheduler(parent, coreFactory), + s"${Scheduler.getClass.getSimpleName}_${coreFactory}_${UUID.randomUUID()}", ) override def runtimeEventListener( diff --git a/src/test/scala/edu/ie3/simona/sim/SimonaSimSpec.scala b/src/test/scala/edu/ie3/simona/sim/SimonaSimSpec.scala index 07522bfe44..bf0bd2cb05 100644 --- a/src/test/scala/edu/ie3/simona/sim/SimonaSimSpec.scala +++ b/src/test/scala/edu/ie3/simona/sim/SimonaSimSpec.scala @@ -19,6 +19,8 @@ import edu.ie3.simona.main.RunSimona.SimonaEnded import edu.ie3.simona.ontology.messages.SchedulerMessage import edu.ie3.simona.ontology.messages.SchedulerMessage.Completion 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.sim.SimonaSim.SimulationEnded import edu.ie3.simona.sim.SimonaSimSpec._ import edu.ie3.simona.sim.setup.{ExtSimSetupData, SimonaSetup} @@ -58,7 +60,7 @@ class SimonaSimSpec extends ScalaTestWithActorTestKit with UnitSpec { ) { override def extSimulations( context: ActorContext[_], - scheduler: ActorRef[SchedulerMessage], + rootScheduler: ActorRef[SchedulerMessage], ): ExtSimSetupData = { // We cannot return a TestProbe ref here, // needs to be a proper actor created by context @@ -121,7 +123,8 @@ class SimonaSimSpec extends ScalaTestWithActorTestKit with UnitSpec { override def scheduler( context: ActorContext[_], - timeAdvancer: ActorRef[TimeAdvancer.Request], + timeAdvancer: ActorRef[SchedulerMessage], + coreFactory: CoreFactory, ): ActorRef[SchedulerMessage] = { val throwingActor = context .spawn[SchedulerMessage]( @@ -186,7 +189,8 @@ class SimonaSimSpec extends ScalaTestWithActorTestKit with UnitSpec { override def scheduler( context: ActorContext[_], - timeAdvancer: ActorRef[TimeAdvancer.Request], + timeAdvancer: ActorRef[SchedulerMessage], + coreFactory: CoreFactory, ): ActorRef[SchedulerMessage] = { val stoppingActor = context.spawn[SchedulerMessage]( @@ -434,7 +438,8 @@ object SimonaSimSpec { override def scheduler( context: ActorContext[_], - timeAdvancer: ActorRef[TimeAdvancer.Request], + timeAdvancer: ActorRef[SchedulerMessage], + coreFactory: CoreFactory = RegularSchedulerCore, ): ActorRef[SchedulerMessage] = context.spawn(empty, uniqueName("scheduler")) @@ -446,7 +451,7 @@ object SimonaSimSpec { override def extSimulations( context: ActorContext[_], - scheduler: ActorRef[SchedulerMessage], + rootScheduler: ActorRef[SchedulerMessage], ): ExtSimSetupData = ExtSimSetupData(Iterable.empty, Map.empty) } 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 de7cc0e38d..d8cc467dd9 100644 --- a/src/test/scala/edu/ie3/simona/sim/setup/SimonaSetupSpec.scala +++ b/src/test/scala/edu/ie3/simona/sim/setup/SimonaSetupSpec.scala @@ -17,6 +17,8 @@ import edu.ie3.simona.event.listener.{ResultEventListener, RuntimeEventListener} import edu.ie3.simona.event.{ResultEvent, RuntimeEvent} import edu.ie3.simona.ontology.messages.SchedulerMessage 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.sim.SimonaSim import edu.ie3.simona.test.common.UnitSpec import edu.ie3.simona.test.common.model.grid.SubGridGateMokka @@ -54,7 +56,7 @@ class SimonaSetupSpec extends UnitSpec with SimonaSetup with SubGridGateMokka { override def extSimulations( context: ActorContext[_], - scheduler: ActorRef[SchedulerMessage], + rootScheduler: ActorRef[SchedulerMessage], ): ExtSimSetupData = throw new NotImplementedException( "This is a dummy setup" ) @@ -69,7 +71,8 @@ class SimonaSetupSpec extends UnitSpec with SimonaSetup with SubGridGateMokka { override def scheduler( context: ActorContext[_], - timeAdvancer: ActorRef[TimeAdvancer.Request], + timeAdvancer: ActorRef[SchedulerMessage], + coreFactory: CoreFactory = RegularSchedulerCore, ): ActorRef[SchedulerMessage] = throw new NotImplementedException( "This is a dummy setup" ) From 2a2b2065846815fbb32e4b4ca80ff68ce0f83045 Mon Sep 17 00:00:00 2001 From: Sebastian Peter Date: Mon, 11 Mar 2024 13:44:01 +0100 Subject: [PATCH 16/32] Fixing errors --- src/test/scala/edu/ie3/simona/sim/SimonaSimSpec.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/test/scala/edu/ie3/simona/sim/SimonaSimSpec.scala b/src/test/scala/edu/ie3/simona/sim/SimonaSimSpec.scala index bf0bd2cb05..c6ffda8fab 100644 --- a/src/test/scala/edu/ie3/simona/sim/SimonaSimSpec.scala +++ b/src/test/scala/edu/ie3/simona/sim/SimonaSimSpec.scala @@ -68,7 +68,7 @@ class SimonaSimSpec extends ScalaTestWithActorTestKit with UnitSpec { forwardMessage(Some(extSimAdapter.ref)), uniqueName("extSimAdapterForwarder"), ) - ExtSimSetupData(Iterable(extSim.toClassic), Map.empty) + ExtSimSetupData(Iterable(extSim.toClassic), Map.empty, None) } } ), @@ -453,6 +453,6 @@ object SimonaSimSpec { context: ActorContext[_], rootScheduler: ActorRef[SchedulerMessage], ): ExtSimSetupData = - ExtSimSetupData(Iterable.empty, Map.empty) + ExtSimSetupData(Iterable.empty, Map.empty, None) } } From a5fd8d1321a454ddda06f22632b10771c0b9f56d Mon Sep 17 00:00:00 2001 From: Sebastian Peter Date: Mon, 11 Mar 2024 14:24:49 +0100 Subject: [PATCH 17/32] Removing todo --- src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala | 1 - 1 file changed, 1 deletion(-) 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 7cfa3dc8af..04632486ee 100644 --- a/src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala +++ b/src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala @@ -429,6 +429,5 @@ class EmAgentIT } } - // TODO test stacked EmAgents } } From 5861489d8e3830098bf289467c13ad4db7cd5caf Mon Sep 17 00:00:00 2001 From: Sebastian Peter Date: Mon, 11 Mar 2024 14:35:59 +0100 Subject: [PATCH 18/32] Solving todo, cleanup --- .../scala/edu/ie3/simona/agent/em/EmAgent.scala | 9 ++++----- .../ie3/simona/agent/grid/GridAgentController.scala | 3 +-- .../edu/ie3/simona/model/em/EmModelShell.scala | 11 +++-------- .../messages/flex/MinMaxFlexibilityMessage.scala | 13 +------------ 4 files changed, 9 insertions(+), 27 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 f2deabb007..e811df8e59 100644 --- a/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala +++ b/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala @@ -326,11 +326,10 @@ object EmAgent { ) ) - val setPointActivePower = EmTools - .determineFlexPower( - ownFlexOptions, - flexCtrl, - ) + val setPointActivePower = EmTools.determineFlexPower( + ownFlexOptions, + flexCtrl, + ) // flex options calculated by connected agents val receivedFlexOptions = flexOptionsCore.getFlexOptions diff --git a/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala b/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala index f22667ebca..e707680218 100644 --- a/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala +++ b/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala @@ -6,7 +6,6 @@ package edu.ie3.simona.agent.grid -import com.typesafe.scalalogging.LazyLogging import edu.ie3.datamodel.models.input.EmInput import edu.ie3.datamodel.models.input.container.{SubGridContainer, ThermalGrid} import edu.ie3.datamodel.models.input.system._ @@ -79,7 +78,7 @@ class GridAgentController( resolution: Long, listener: Iterable[ActorRef[ResultEvent]], log: Logger, -) extends LazyLogging { +) { def buildSystemParticipants( subGridContainer: SubGridContainer, thermalIslandGridsByBusId: Map[UUID, ThermalGrid], diff --git a/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala b/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala index 0ec7cd6c93..9c70d99126 100644 --- a/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala +++ b/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala @@ -94,17 +94,12 @@ final case class EmModelShell( setPoints.map { case (model, power) => val flexOptions = minMaxFlexOptions.getOrElse(model, throw new RuntimeException()) - if (!flexOptions.fits(power)) - throw new CriticalFailureException( - s"Calculated set point $power does not fit flex option" - ) + + // sanity checks after strat calculation + EmTools.checkSetPower(flexOptions, power) model -> power } - - // TODO sanity checks after strat calculation - // checkSetPower(flexOptions, power) - } } diff --git a/src/main/scala/edu/ie3/simona/ontology/messages/flex/MinMaxFlexibilityMessage.scala b/src/main/scala/edu/ie3/simona/ontology/messages/flex/MinMaxFlexibilityMessage.scala index c814b14ef0..442d9ed785 100644 --- a/src/main/scala/edu/ie3/simona/ontology/messages/flex/MinMaxFlexibilityMessage.scala +++ b/src/main/scala/edu/ie3/simona/ontology/messages/flex/MinMaxFlexibilityMessage.scala @@ -39,18 +39,7 @@ object MinMaxFlexibilityMessage { ref: Power, min: Power, max: Power, - ) extends ProvideFlexOptions { - - /** Checks whether given power fits within the min-max interval and thus - * would be a feasible solution - * @param power - * The active power to check against the flex options - * @return - * Whether the given power is within the min-max interval or not - */ - def fits(power: Power): Boolean = - min <= power && power <= max - } + ) extends ProvideFlexOptions object ProvideMinMaxFlexOptions { From 6fd10ff51a620a98c7fa67b197f754a91b827726 Mon Sep 17 00:00:00 2001 From: Sebastian Peter Date: Mon, 11 Mar 2024 14:46:05 +0100 Subject: [PATCH 19/32] More cleanup --- .gitattributes | 1 + .gitignore | 2 +- CHANGELOG.md | 1 - build.gradle | 1 - 4 files changed, 2 insertions(+), 3 deletions(-) diff --git a/.gitattributes b/.gitattributes index 022b84144a..00a51aff5e 100644 --- a/.gitattributes +++ b/.gitattributes @@ -3,3 +3,4 @@ # # These are explicitly windows files and should use crlf *.bat text eol=crlf + diff --git a/.gitignore b/.gitignore index 83d4c33fc9..7cf28aed43 100644 --- a/.gitignore +++ b/.gitignore @@ -233,5 +233,5 @@ deploy.properties # Ignore Gradle build output directory build -# Remove unwanted input directories +# Ignore unwanted input directories input/ext_sim diff --git a/CHANGELOG.md b/CHANGELOG.md index 29b472e0ec..bb87db369e 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -23,7 +23,6 @@ and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0 - Config possibility for transformer control groups [#90](https://github.com/ie3-institute/simona/issues/90) - Implemented scaling of all relevant input parameters [#764](https://github.com/ie3-institute/simona/issues/764) - Consider scaling factor with flex options [#734](https://github.com/ie3-institute/simona/issues/734) -- Output of accompanying thermal result models - Implementation of Energy Management Agents [#204](https://github.com/ie3-institute/simona/issues/204) ### Changed diff --git a/build.gradle b/build.gradle index 7ecfb76452..6657dc411b 100644 --- a/build.gradle +++ b/build.gradle @@ -153,7 +153,6 @@ dependencies { implementation 'org.scalanlp:breeze_2.13:2.1.0' // scientific calculations (http://www.scalanlp.org/) implementation 'de.lmu.ifi.dbs.elki:elki:0.7.5' // Statistics (for random load model) implementation 'org.jgrapht:jgrapht-core:1.5.2' - implementation 'org.apache.poi:poi-ooxml:5.2.3' // Reading xlsx files } tasks.withType(JavaCompile) { From b624b05f6c0319c684d08b6ed7e37795b48c333f Mon Sep 17 00:00:00 2001 From: Sebastian Peter Date: Mon, 11 Mar 2024 14:51:18 +0100 Subject: [PATCH 20/32] Adding disclaimer to config-template --- src/main/resources/config/config-template.conf | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/main/resources/config/config-template.conf b/src/main/resources/config/config-template.conf index f9ace9eca9..b46d088f9f 100644 --- a/src/main/resources/config/config-template.conf +++ b/src/main/resources/config/config-template.conf @@ -56,6 +56,10 @@ EvcsRuntimeConfig { #@define extends BaseRuntimeConfig EmRuntimeConfig { + # # # # # + # ATTENTION: calculateMissingReactivePowerWithModel and scaling is ignored here. + # Cleaner solution is possible with different config framework. + # # # # # baseRuntimeConfig: BaseRuntimeConfig # this entry is ignored by the config generator, # but cannot removed bc otherwise EmRuntimeConfig is handled as String pvFlex: Boolean | false From 3a1c81dec0162bb189471ae0085c1041fdc6be85 Mon Sep 17 00:00:00 2001 From: Sebastian Peter Date: Mon, 11 Mar 2024 15:04:27 +0100 Subject: [PATCH 21/32] Replacing RuntimeExceptions --- .../scala/edu/ie3/simona/model/em/EmModelShell.scala | 7 ++++++- .../edu/ie3/simona/model/em/PrioritizedFlexStrat.scala | 9 +++++++-- 2 files changed, 13 insertions(+), 3 deletions(-) diff --git a/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala b/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala index 9c70d99126..c55165128b 100644 --- a/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala +++ b/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala @@ -93,7 +93,12 @@ final case class EmModelShell( setPoints.map { case (model, power) => val flexOptions = - minMaxFlexOptions.getOrElse(model, throw new RuntimeException()) + minMaxFlexOptions.getOrElse( + model, + throw new CriticalFailureException( + s"Set point for model $model has been calculated by ${modelStrategy.getClass.getSimpleName}, which is not connected to this EM." + ), + ) // sanity checks after strat calculation EmTools.checkSetPower(flexOptions, power) diff --git a/src/main/scala/edu/ie3/simona/model/em/PrioritizedFlexStrat.scala b/src/main/scala/edu/ie3/simona/model/em/PrioritizedFlexStrat.scala index f8c93ab7e0..27844f01e5 100644 --- a/src/main/scala/edu/ie3/simona/model/em/PrioritizedFlexStrat.scala +++ b/src/main/scala/edu/ie3/simona/model/em/PrioritizedFlexStrat.scala @@ -13,7 +13,8 @@ import edu.ie3.datamodel.models.input.system.{ PvInput, StorageInput, } -import EmModelStrat.tolerance +import edu.ie3.simona.exceptions.CriticalFailureException +import edu.ie3.simona.model.em.EmModelStrat.tolerance import edu.ie3.simona.ontology.messages.flex.MinMaxFlexibilityMessage.ProvideMinMaxFlexOptions import squants.Power import squants.energy.Kilowatts @@ -64,7 +65,11 @@ final case class PrioritizedFlexStrat(pvFlex: Boolean) extends EmModelStrat { .reduceOption { (power1, power2) => power1 + power2 } - .getOrElse(throw new RuntimeException("No flexibilities provided")) + .getOrElse( + throw new CriticalFailureException( + "No flexibilities have been provided" + ) + ) val targetDelta = totalRefPower - target From 45faf3facb006d09f5899d0bbd8060ee1342765d Mon Sep 17 00:00:00 2001 From: Sebastian Peter Date: Mon, 11 Mar 2024 15:05:18 +0100 Subject: [PATCH 22/32] Importing squants.Power --- src/main/scala/edu/ie3/simona/model/em/EmAggregateFlex.scala | 3 ++- .../scala/edu/ie3/simona/model/em/EmAggregateSelfOpt.scala | 3 ++- .../scala/edu/ie3/simona/model/em/EmAggregateSimpleSum.scala | 3 ++- 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/src/main/scala/edu/ie3/simona/model/em/EmAggregateFlex.scala b/src/main/scala/edu/ie3/simona/model/em/EmAggregateFlex.scala index 16d7f8a05a..216ceb44e0 100644 --- a/src/main/scala/edu/ie3/simona/model/em/EmAggregateFlex.scala +++ b/src/main/scala/edu/ie3/simona/model/em/EmAggregateFlex.scala @@ -8,6 +8,7 @@ package edu.ie3.simona.model.em import edu.ie3.datamodel.models.input.AssetInput import edu.ie3.simona.ontology.messages.flex.MinMaxFlexibilityMessage.ProvideMinMaxFlexOptions +import squants.Power /** Offers method for aggregating flex options from connected agents which will * then be provided to a superior EmAgent or sent out as a flex result @@ -25,5 +26,5 @@ trait EmAggregateFlex { flexOptions: Iterable[ (_ <: AssetInput, ProvideMinMaxFlexOptions) ] - ): (squants.Power, squants.Power, squants.Power) + ): (Power, Power, Power) } diff --git a/src/main/scala/edu/ie3/simona/model/em/EmAggregateSelfOpt.scala b/src/main/scala/edu/ie3/simona/model/em/EmAggregateSelfOpt.scala index 23aea26472..a5e8421e3c 100644 --- a/src/main/scala/edu/ie3/simona/model/em/EmAggregateSelfOpt.scala +++ b/src/main/scala/edu/ie3/simona/model/em/EmAggregateSelfOpt.scala @@ -9,6 +9,7 @@ package edu.ie3.simona.model.em import edu.ie3.datamodel.models.input.AssetInput import edu.ie3.datamodel.models.input.system.PvInput import edu.ie3.simona.ontology.messages.flex.MinMaxFlexibilityMessage.ProvideMinMaxFlexOptions +import squants.Power import squants.energy.Kilowatts /** Aggregates flex reference power with the target of reaching 0kW, while @@ -23,7 +24,7 @@ final case class EmAggregateSelfOpt(pvFlex: Boolean) extends EmAggregateFlex { flexOptions: Iterable[ (_ <: AssetInput, ProvideMinMaxFlexOptions) ] - ): (squants.Power, squants.Power, squants.Power) = { + ): (Power, Power, Power) = { val (minSum, maxSum) = flexOptions.foldLeft((Kilowatts(0d), Kilowatts(0d))) { case ( diff --git a/src/main/scala/edu/ie3/simona/model/em/EmAggregateSimpleSum.scala b/src/main/scala/edu/ie3/simona/model/em/EmAggregateSimpleSum.scala index 5a0cc02c59..28e343fb6d 100644 --- a/src/main/scala/edu/ie3/simona/model/em/EmAggregateSimpleSum.scala +++ b/src/main/scala/edu/ie3/simona/model/em/EmAggregateSimpleSum.scala @@ -8,6 +8,7 @@ package edu.ie3.simona.model.em import edu.ie3.datamodel.models.input.AssetInput import edu.ie3.simona.ontology.messages.flex.MinMaxFlexibilityMessage.ProvideMinMaxFlexOptions +import squants.Power import squants.energy.Kilowatts /** Aggregates reference, minimum and maximum power by just simply summing up @@ -19,7 +20,7 @@ object EmAggregateSimpleSum extends EmAggregateFlex { flexOptions: Iterable[ (_ <: AssetInput, ProvideMinMaxFlexOptions) ] - ): (squants.Power, squants.Power, squants.Power) = { + ): (Power, Power, Power) = { flexOptions.foldLeft( (Kilowatts(0d), Kilowatts(0d), Kilowatts(0d)) ) { From f129a5767538be2665ab2e9a6e68bd112ff7ed28 Mon Sep 17 00:00:00 2001 From: Sebastian Peter Date: Mon, 11 Mar 2024 15:07:20 +0100 Subject: [PATCH 23/32] Removing unused resources --- .../edu/ie3/util/scala/io/flexSignal.xlsx | Bin 47344 -> 0 bytes 1 file changed, 0 insertions(+), 0 deletions(-) delete mode 100644 src/test/resources/edu/ie3/util/scala/io/flexSignal.xlsx diff --git a/src/test/resources/edu/ie3/util/scala/io/flexSignal.xlsx b/src/test/resources/edu/ie3/util/scala/io/flexSignal.xlsx deleted file mode 100644 index 1bb027a4433b5f934c66cb785db74e5f847d21ba..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 47344 zcmeFa30%xw`}m(FAqhn&k}Zis(WbIS+?BF~HYB0FNogHQA`z8B8;T_DrM*!}5>piI zo7A+8Nk*w}>FtZJ@Nh?CgZmQ7=0<6)aVS!<`6sg;4LmEIv+a|42o*hLfLWsj#$Uh|M` zGI;)<|Na+S;I)s6S@womS83jTg3q{y9G-6EB8id&SaQrAmgP)1L*bKSpYfon_n@8b|laP!*)}`wGnvb+ulw@8zV^ z4IdkG%~QKB9ZNTfJ-zu3sZk(evB#@y+ZYSOCg-`ldN~31>g!(%^gk?3Yq}dh)z|X8 zWZBt^+!C=RKJqUbDC?EkZmvtKTxy)YZ*_X<^Lf0=1Ml7miVct-U6U$a^x|$_#I>x8 z5j*4QX<5`7!XGLJ=EYr~-m~(L*L6l86t$10%O$OueIwyOYWW|Bm+2%*Zk)nC$hPAY z*ES`-bL%}0^xYV`M3nW|x_(5$MY3yEJ&(*Z=IiaT)JtK8j6*B$ueiLzM6xc0NSr6% zq}+jU2KQT!B#Qia#*>+b~EN<0vqEHQ(u&q$~3=9ixydZISd#H1*@C}UXl^fku- zXHTB>uUbBLZjO6UnZi6+VxK3?CFdt7Nk8Ij+|_h0(^CP=P9}2-+ktWOHu3O=6+Ivr z{{YjH6U2e%AAWi6=W&9+*a?j@hkc+e-h7#!+O<#w;IbO=k;$Y+>^KJVK4D`f@D`gpzUj^ z895PeJM*1y2tKQsaFRTHa59e%e4{^=jcqa8^hpRE8OOIpeBgZf zAZNvM|69AvV?!UFp)P$AnywTgV!5QENF-3|!Oj-Z@|NL;qCV#YqC=T&lgeKoY@f;Q zq*>)$#?aY2T3dP7z`Mb_z;oLA;On;qk~b_ouO}qB@SWhs>yy{KrO)EtzvxB;zaY9j zCoi=4#p>>=PgUz@<~Z_GUnP?yOGsUomW$)eKs5ZAdk@U{*rhgwjm-jVp}4^lAgsO* zsL&(oX8s%II0Yv#FI^0KGj*@?zOa>-7K8^r7c)u~5Xp%iw2m ziUAo!o78#T^{Ur6IN6fjJ2cu>@7UY8D)PGQT&~8LINrF`i?4Ps4Oi08c;=;{qxqIl zJ?HZk3wxJ2`T84PTjpCnxRxF%Qp!I^S@dz1Y*qK5-iHfi@fw%1BSi;847_%nyqtMSG-0LZ+rZS7 zsR_rA;aC=-c?toZ}-@;bqgc9f!*qKiD;k3eoiVSU*>d&gzXgpXAO6g z>xC*_(~`FO&CMKRxbAz03-)?xve(plb zUAN1(Oav3mW30A+8JTGv;yb_4bl57%Pk&HTh16FtXA4LD<`B`|bt@$jzDRlSo%+1d zu9fB{)TwoWQ|d*Lfid@Roh@~R0b|w0w)zvN_bJ)Ei72a~uP(pnoz-~nOd)?yb|5vG zJt?*3tIl?TdH?NW`tiq!+qE;h zUN3w^J-5$_bN7jv;rx4^es~$5mfv{h6#rBC@;#c@AH_Kinq3X9|9tG@iktCmE4U1U z#TM}GsR#`uN^1gtXVoK@*2dz-Oq4hX)&t48THZFAT5v**D4G1umhaT1eGmqC1&b(n&u9c^irykmWFmzv~q)1?bCdVJA zJ`29R=V4qytlfNjTd8T}C%+SVm3fpPf+LTPl(QVibJ3@#J%&ZOWe%NLYiP+AV#WxU z+oNi5?fnc*uQ$zmMXdtUiF}8f-nN`k3%6j~ab#l!|3R^hQE~F~LsKTkJkpq6FPj@NLs@z0=ADg@5)>~g2xp2vH z%ZC;H2fbu&Di&WZZZ2MB(vvUmY;gSW^9`DpZX8yR@lbe@_KNe5xJff^i|_2-+0O21 zY1ZSq_uz|`%g2Te%$vXO!MprJ>s34^FFU{LsKVafolmW22vBWSsrMGk%$Yq`uzsc2 zt|C*Xozj)(&$yI*z3?o$LT^r1f2_+4CARgd?Atu(b2G$l@2U%Vbo^;dYfrI@!1uyVA`h7#z{$fA^ZBP*Xlo<<+aA} zblA%JBR6J$TQ6)sea`Zb0JHWfgSr97ju}44zLU7uHKjm*s@lg^;$r8iBQg4)RMy3B zJnkm%9H>3DO)_^8b8a}>^VU^I6nJ%sRYC^&o_gnFpU9fXU0$ChmU><%mazA$-u;|2 zMqefFGcUMB(w0WdpSp5k%kbXDc->o(g9}Y6MxWB>c9?ufKH0tK=A_ML(@MAQuPM7) zJZPKA@xI{P_VcFH!W$bGzX?7^K0oPHUs`6I%ck|aJg#;LYx$18ui4H|4|mR+t{3)Y zof64ZQ*c3BrKCBf<~&tZSkj8G{(Q&PV*k%}fnw>e(gx(u+nb%WGx3||J=_xawP^8$ zU}H5WiPzgcoc#3BOmJn7R zOhCSUNXI|%Th5b{&X%jybOQwz@5x$uZ&Y}OgXLsy;^ln73D=z!xwqt6!8b`@o5G95 z{@Ih32B!~NS)MaBM5ci8?GM-j&fIw~&Hp9GT;=$;aqe{-{T8}A z)a5i|(*}|ht#NLAjoWs|lcxm3uUa0ywB!7#w(yAhk&eEvGkI@>eBrtibJgth(Mq~R z&bsq`^Gb_uc$6L|6ziL(yRZ24!r?VIM zvx4k>LiYTf+kRzL19r^*VXJsK^&z!m6_nN2zh3p*dZ19yj5wifymjk7dhBFl%RDlL zZ6RtKxF!?M8W>m+#F2kC!W0qUQ+cam!5`U=$YC^n(KRE10%U7v}w2M=<5$* zZW}FjIxMb8B%R)}UOFhI^6p?_)fCe78eig`F~sZW1CXPWhAV_>gb{}eNz4&a=1?1Pq_>bbSjg-pjy5VXJGeWzncbeuky2)V-Y{u2 z@dLA)SlL%f8fhajNwm>XPtwSU;%Hm#XlI`aGbfnYlf-Ce-YcarDS~tolU`^>9Bfcz z4u>g@c35K=Wx)CGKh?J>#o!RF$S{`5QWN$Yr9o)inN;Pci4I7dl9_6tkrY4yv1_h;( z(wQel2Pj`MZJja&`3C8(w55D+z^5SJOA4prRCAIx!Awf{J2lU;fyS~z z&$8aeGI7tck;by<>%9YtgSCp!xD{PbI?bfjgMZDA6Bk$My$K6`UF_LhnddtCf!;1S zk=P*d;hto31-5~=qMd*p|^7L8Q%q&nK|2)=d2 z4CpAO#+PZYy|9oklc-$=By=XnDC+fhS=3+5>@04`9&o6q)gp&%7 zNb|%*z~5QX1hrz%V6}uD(M~B?_oma|8t)F%3|J+MP{jmC zw-P(Vf=9P1);=NL>(r#iWq)=aa$^UTl69jeU+JZBb|ewYXrCaQg*`~zs1exsbKN}f=CbWeW8i*4mp~ zU5vFB4onNf`c{**5l32UvQbS_CawUFMvB=%$=cw?Yvb1lQ zop8`zcb6tZ-Tur4t=oOI%%}!ULB6<3uY^*Xi@mN--=O218s8@wPJ&hN$(BLNlfEkR zLX{5NCj|;K>lV+{J~-9Ss(Md{?HwIQ!3zrF2@$jqG9}l#*RfN1qcL&BU20+aitJ+q|L#{bYmH+aD}X3vE+q zbx=w)a_@jGO8M8=sC?A3?QU&v#YIgDbU^{sFb&Pl}?{}XlYSTg=MNAQ~P1-=3G0dXkg(QdzFt^3+m=v zd$dJ%DQ_2S@wW89K!sPDpUpLLDR0jz9Yrx9xOcOwBl=iPDQ`a<8b#uP-oa)}11FU% zJ)r1rB3DQB`a=7GLiSd@gTsDSBgsIO$z{BgRwFIeo;Wz`9-{M_%PQ(8;Lt8}ZH`%A z4~KSHI24J9Lkp62fiW2phZ~sQSo})}0kvpzw}EZtw)N|519hQ|JuE;)K_<&xB0uxkAqh z2`8~=QNceD2L``yLj*mON5A)Ttm5eD=dq)%umfQU20wVhMzDCNU^~^tuGyv9@esFT zOd5CFBW`Pkh5TrGPSO<0jS|8cSxZORijOzsdr4z(!wQGBl3_LlyNM3{`w2AO?|2KSv*K z>oK*vcw8K>*A~d_LLCpM5^iQNm9S_4lh4t74e)HMgb#?E+Prq#hupJ~=e3Qus2S>TY`y(IgqY;E?F6d`B;(@A(KKB$feqIPa2Nl3M-(Vdqrh1ms zuPP*qz-*zfYi8W+;*Uf|lS54rhZLOfAWTj1S{N_r_rEH^{3O+rrNRoV-8j0WhAebftlpO_qdpH<=g56CJNbG1nj;AbfG)J;4$U%Dq ztKdS$*cdGZG8%jU`crPO4qXJGKYw3>cEn{OcEr~Tte*+$V2Zk>kc=;Nf29QTAg(Uh z)L@GLvIO!%as!3K1`EP2x=R`w8i|I(!*x*#ipY{?d+wAY3>I7;eGn`kkw$O^QJY=h z$_n==CN}6Wrd4y!J_K-IXvWEis`wvh=ZFor+;697F>!;hnWZM=T{||gSdOL z#um5Vmtf4Y6o)a3r{bTNV9ZW&>_paD7@rfCKpJlyLQ|RF>Mx9qG&8WW!4Ztpkv=+R zsv4pKZlL!=W5&dFg1zcLX@gsFyhrF`)F{x$TVa+!lRmy;KrMsOEn&K3EPNmMh`RuU z66zvk8->~lrU;g^j-+OSJtc01pSFaHPc-Et2U!1=H9gx7fz0YC7*FAG?1aVYy!L@z zf?Lr%VXuHb2J&c-T2IzNEMo`{)`cT$6722wC1?j+I%0jKpI??>#<;F3|Ep z#|XyE1{SDMIyNFqQ@F9F@S;{n_bS*Z;r0TW8hOD&)Pp%=EAEPtHm;s5uqP`ZiWMPF zMBcD)O^~=kyoclt=pD@dU{?os$ZK>bD3l2n@5$Pas1#4(bnMi? z0%eC_IugPg?K6uw#DQR#0d$*P{1BtSdH?$ooLwyGaC?jP^Q#if0DOaWP{&jx;(uR) zHoz4}A`cnDFH0~6T@@}OTQ2y9{TE8mv%n@L6!=Q`OkP~&1x+8RL9Ikn)WaWR{AY`1QS{-yB z9IgG%7%)wto?{8=O@TpGo7Z1=Q{kd`m;L87_|_|5yPpTl1o_%#mn6jJX9}eT>mm`y zl+(eJaP5Oy0#)#>hiOli77|L(#Xl=S7qcY6G=s|gRSCM7-?8%uj7?-Lzb`=-V@e{2 zD2TehE`2{3m|W0d{jvnpObB-kE6vn^MB)@Y6;*X487v*Fk=il`aFH8nikMa53^Bt^ z)$6omur338M@DRpzH*?3gzEw92quj5j~r%W;WD-ff!&W1zz#Babld*&W?L5ws zFcui-7WL;Xo~zj(+}3|BCvetWvRp=9y=v3#AR9Z+oWS=lD1kK%=E8&ng=qbgp7dgS z0qW^D^0c0%ZgTe<>}p!%XWML=5{y272i8Z^LW^QD9xW#CH*FA|9a!~7eYT-+QrCPp zRnvxNZt>og{q8%3lPq{&m*<-}buXk{*ve32U*RyE;w&m6X zX}Oe}(b86y{Cm^`fFMl)*_4|hYso+mDtG}RI9g6b+t}>c4Z0UD`puZ=21ImGzKOsS z3u|0(1tR!jwen?s`<)g6ez<7aG0{uMboLz6`7I*2O?9RaID-6^4z-4&S6Q+|FP35H zd}mDOeNb?}J$I$QbN-%mCoA{Wl)!$+`Mr%#>8it-?qBn5g`MGICu*YdHmX!)H$#*lTs!HMomX0xveqTn9f}Mb(f$*27PC zY(QnvCQ4wH(cSrOhE*m4@&wi>pO)9~o>u@K%Y8ssb+{#37GU$0pzXN1 zHJvB8VWt#NUO+QlhW47$zr3Kd(1Mpdp!gNoDrRz$#UV1Vp#ahIl)r+cZF3_g$2=GL%2kM~ma?x_cFE+5}TTOboi6Bbn3T&_}KA!j){X)=}*dCtj?D)%uze>5)9~<5B{4+VUpF@M*9+nXr2>7 zpn0OkE)5JIY(cCIl$&)8bA`;5Q;0p?RLPghWGbBtN|^~UNK-%0C^^a>0@ToecsfNU z4RJ%oMM@#T8u46W7de>H4m{zedcGl0v9LT4`mRK#W`G`04cvV}m=eIMu6hG%9kUt> zCOCjkM7lghEq?Br{HU4%hlL%xsL-4N3Sx)BORqYVxK zi8fSc7;RK6+RzSBH=={X#G&3nM}|v71``-Ru{g}v$oOGUKxu8NWEnX`yNSe2pd9p$ zHKN@_;?M~I#yTbqPJs5`j}?&y9K|W7M>O<7h5$0&cgn%=K&`_VMePatnD!cR$Lzp> zNnI2fH!z?A)f~ql35#TJ**p%SS%-Nrv}`!Rp_B1tHp3Q9(Vu()Ex|zyB1ogI0HNeI zv=#M_VxdeG1qP65!>}#69-lQbAPc}ikSRlYM#fGt$Hy*!gdB8C4(?_s8_rfj3}H4a ztBezex*3jxB@IRm4y<>LG3#5Bv&bnhowp}|PmmADT{i;> zNHW%-^uIyALqb8HQia&2L?$zqV2b7FsxlcVmQPsA0NNbpa>x^j0?jTHz00}2_&Oo28{-1ND6A_F>6d9dCX|g_s|{abgh(TI*~t&IH6PG zXJs(en=+HObj1~eya{H1+;AW%$=aq?%U{|`!AR8%{fnS3o$ zJ~m3O)k;gFMXCtMiWJli*c;hk&ZQ1#B&H43R3P&Z2&V$!h!t4Rr`!}@;3@|)3y2MD zMM#M!P!XrMCcYz+Q;;kiUbmr)87CQY8PEaAIH;1pl#IFT&tv(8WYlF)0mNnHGC$FIO!Alla0XF- zs`Hp+G-lw?;0&^{CXchmM3TphW)KEbueoqz07w@|o(-MVe1ftSnf*SKmb3?;nRe4d zmq0p_xHZVOF@&YqfVP&&aI>7iO5*5)aC&WBpq)AVp&xpwEBA3P7|>E&qFtmVD#uL~ z#6lL?7Fkt5EU*;OjSxT|hQs2vCQl@vmBGiW0TTl*2kshC;V>4U=BUBOmJl#*B+Bq1 zBAWk1GLjhJ0Ffxe)&7NKG|Hd?Fv{q-ZDC>M`g7m>4i?PoVRdWHc83vQx$l z%`Z?DcrYi>hY-{r5QZ$Pn;v=x(yibstuf6iju3>H?A;|*YjQDMrqZ? zY-LSIHC8Mj55l(uR1RWMY}(jHNmnUn9FZxPK_Blu-iXU6F}lm@Ir=cxAM~ZL9Hj-{ zQ^20l0i%k+*@x;4c75N^>6YjOeR-CVLQvP6qaNiCBRM|^6q zjZco!h@|LQ&8H~ocyPuNbXgM$NXh$4A zLrF(wwa=t#ymr$=|3JDmmKEX&HMm2|Wz1O9z7j?dGVPHE;p9W*z~REjV1%;zff1Tf z*KJ@A_7>P)p>(a39Hj%>3RIz8!}M=vO-5uZXDlV*9roI>)WpT33mbeAMgT4fi2&9M zA5u)vm|2?fG14U*c0eH zCK=5df6phHfrI{*l%0;+#STKP;8`1b#wts}&R81*Mz{hAtTWc%4;MdUrL>zh2e8gqe-sVE0#S9G9l;qZ zu<}mI*ct2hmi&dzZ6Md97Q)V0p%uo7hS9g@2N&tuXpqZmi)1LPup{SRnqy4ouDFBn z1+z-oBpsZ<_1cnyz_xRerULR0Ng=R$SDl7;}p47Iq+}hZhpa zlo_>xX4v0gAxU?7^O+)J51wJoVB}p)0@nYy%H>(cA&0!R*XBSHuom(UsySMuRK* zzjkzM{UF%IU(m&VKt2}Pc2L3|H`8tC5=)|ZoMxXLz0n5_o@E{mL`$T}JtB~a=Sr$) zC~`-SYI{AM9h)Rh(rGE$!>JYed&OM_`1IeBEZgV&Bx;d{S7 z&ldvd%O! zmg47})ZRaL2hI9d4+MWaZ2g7ysL!#LC>}p8#LiN&5Mu4-Sue}~p_5sIF!%zR?7itB zel%BN@8$wd7LiRJEQ9k3^evH*2j1?mgG;oGY{)=H`M-Q_NTEe80q3(XOYu_?{7tbA z{el>To`lwa3@5RU`$9*@j_3yEu#-#}1K|A^dFfToh~D%UPEcW_{Wl#_GK1sN_HeCF zd%SGxN{$Mml*PB}Rz-DH^2EgnJA$2~(vx3s$bB3r@+OeK(NxGX#4;HoaJ8EX738nq zmx!ioDFONQ``^jj2#iPCwRXgm$$;&@Wm#y=>|az8%`C~s+5^B_ij1+P6q$UCEaRQC z+6GL`iK_p*uWz0<%e2H5@!Y)Ju|+n-d``Ueme$Qgp%I@`YrtRTK%gfaQm zV46={HChAy_bE*EZ>9lE1;df-M#Yzq+}@Tes3X`9fvc(|I&tO zjw|^~qk(%Pz;6y<_XAdDXjd>^NjXE1(`NgF@vC|VS)??F#-9U`@7w_5|ZSNfAJ7Y^S+sf=W5`Nf=m^7{V=l>QTVT zfdpT0k6@ zdkWJ`m)-(L;@A*Cl+Thw$54{?4jb1*mH!RH7&kH)gR?Q!VU+*G{78O;101bNfR^<0 zA&ALQz)>5V*6-p7QP^Bx_wz~^y@Ya^QkLUb^LpYcBzg`w;n>i4oF>r=L3vs%WQ3|`HUss4*f z7*|W>D6ADVh&9tEu7c4)FOw<6ql3)QD@3Hq_dyz~qD^{EP?zY*--hV+qK(z)VLgzaItI<#-1Ew+!QV=0|fc ze5z0-7<_iFY77vRqaa4rP$W3D2f$jmnmhce2xo|*HF?ZsEZYvV{He&Amf$hOjd(|J zw>Q8F4=0vuk*@Qr`)O~jM>8$6y z0*=f+$p_2S__gQXD{v=&ymQ`)bdhQL((inPl+R-Bg=^*a@awmpn{zhq(PSgNFA2_p z8f!0eIXu7c&}g?s{WfO~i-l({zk0Dl_-c5n>rUTQD|*zul#1R3?B#7NyX2H{GLc#~ zHN{o+M50OS+&hn_xzBHSFR($?1;rhR314 zPr5G4NT17Gr^wyS<{i`$H|l@(vE%`(JmV=G&-+ch9$OU7=RTd!pC1?WeyQJsEidkc zD4rbhSf*q>eb33$Zt)cHW19*uX(-c!C1TvU&|si|Sw@k!z>r%?RUfO$r1xOajZFz+s#EXq>p zZ%kOXeLwG0?=GD)jZ-s%=(b@U`PG-i-)Wsx5nRd4pBW!pen+!to>bMW8$3j>*C*%Y zHaO8Es)(V+85`LyAEqRovv3G~@QR%+%gK22oO~f6kDwEM%Uy_FS-Fn>W)5#u7Mp}N z`y}-9%p0WMo+EeqtKqT5XO5K2U)EDLNyz$D8ELk%*7VoR0`>$kzSpy-&)1vlm&kT) z=hL>Y)Z^1%&-NNpt_>2+uM;m0cPSGp)>K_?CN#$_hdd1rrCKI?~NZ%xKbR9*x25rFD!7 zGQ&)WBb6j3gE=%(OJr8oG8+VmgH0WkqaT8q{Q%)XOpC5%2G`Pdj<|)AN zsL6E186F61LvSe{WOyhb3=cbPX@JTY22>02;cf)E(TgBAv@9aH3!63+Hre;*#1ELX zspSk5^!Iou$bSYhF_j5eIPs9le3iWTI;Um zcu`4;?g9A~A$nUMRfY-owCov50|1bYVu%z0V3EJ9)EKVgJ>@zzIOt>8l&-%gsj&HE z&_@9NFzgu^*s35OjNn6>YZY|*M=efn)$6afIBB2R+0c+Z>QEn+=h`rkUFZpL9FQME z0sIg_z9qA;4Ju2#}-&0g})gvK-Jl$PhvvxrLBN-h&}+6$=0=J8CBbm5)@l=15f3yg!)wiB{Mm zNYxw;CIvV$d0hG%cIpcAC8>+ws0=I3)k{#1=BNzgyP{X6lrbHkIM^L6_a1bIO$ zK9d&svJyfuS~uZvi-#SvG4RFg3hD-o8}VJGtPM`N0Kf|f+tAe6fKP0=fuTAg#1PJg&Bn^=ma1ylKLC5A6{nLbV3|w&x)p6JlA~zst1aLNRAEBbf{r=3bLXGK~je`P^hI!9xHbf;ByOs$dPN&! z`$a8Z0OCNg8*N0r^MzcDs*)g0a1Vto>d~vsyUmeH%6#R3vq7QnNE-wU3|(W*T5z?N zliCVy>)Cz;a1L-eCHbXCM`N%!Z_3lXVlCYkB0fgXEKQUv3p_b>qD1cP2X`mV7oH`k z)N6yATHx`Nr+SAx~rX1+K`b^+fvt%H+f-kto;{T)T)ha&JdT z9iO8_UP6usxFcG%dAAL?M#WpRxg_o7E-JVTU%Q$8B6?%%>_rtJjH1!7nH}S9_?GfE zp?C~C-GdMPtjhP?(cM{WRL}zM-f#PC2&=Pkx&?6r_Ipr728t@dFEr!mkv$OOf*~|Y`TN&D9M7h(CF zA|h6RR~4P45E#b4M&Rffy?ZK{`N=nWc%&01s6vGsu_i-vd>~=^F7Ta0g^LVDJ?19D>?tjfb}^GrwnV^rb(- zpbYm)akNA#xFCyxG~m6(aexdI83DuRXC*kBegmfXGpqud0jFKStEtKByX<>=&!g+m zPk|l=f{Yt%15%Sx_1NE%I1$t<}U_On0L63wVKk3{x_46C2_WszVDSa|y9HERht=wZL6v+zI2D7`=lmat91tgtNg= zj~;_-v>6UZIzyxeyd4dJ8-AYqzcUuRTa98)#?dv1g$VfrkY^xpLrf4{er34?UVubl zD)6c#)+0q?v?@YxW4!aMai+u?imhJ6hVhj`z16MFy}1qV_f z{zPwULeKuJ1mpT|05Vwc2!NTwK^8wRA*>2qT}XJr;Tk_L!Te4I@qpq-#-TSBDJmvM z6e7q2dwAhEFI_JQ@B1EOLL`YTa|~KSQ>5XP3z%h5RL1$j{X7MbsKL?*h7D>o=xbCB z=xc1WrTl19c&V0gH;P6d<#z(t`kh#|{GPZmkxy2=XxCTg2jNP9_0V?pWHckLR zI1VrYz|8c#5Xnyf?88D>Ou*iNH>)vB$2j-|yfHnty8Nv4?*e47)-VL00b>@2hQOG` z*c87iftc;whEp#fq5-|QKE4Ff_~0m#%7CQ-%VvllvlPseIK%_qnMW}tz*LN2Zrm*2 z(K%Q;!CsF7mJ)On)+6*Wx;9Th$-uPy{u2KL#0`Y7_h+mJbe1OwDpucc&y4GavpTSr zkQ&uy+mi(e4XqX`*e)fg*#$*E#8a*-NTXNg?SHI<46KhFD0>1<2DA+fk1-qvi?U%3 z;TbSrJmyw~-5{){-?xPh-X$2oy0-rNw$MYk*6>FB@7uyq<#9w5ME6tZMEXfv$~=;0 zZS7uUEKwlA(g%F-?!aw%HGJ?OY2YITck6NS`ZYw%j##H+fE5P#8%XEqHF!*4xUWEY zBhYy`K?CneJzA=W189OQFo+e=3;3wvVLYLxh2aJT!W;@XArROB)(WHxbSF0cF*GWW zep?B!_e=oga}a|B3GzQ|mXN+EaDq{G z3Y@$khOnuGBp(E{F%}BHg0g^R2Zz+Kz&!|R1*LV27Xl-YrHpJa0{VbiaBLYFizX;U z(-EM=;9HEsML-Gx{DxjC*eRbWEQN%pGleV`N;w?|mH>>?1WXaI`D8DX27k>vxVewGF&IBW) z{7@PPI)SOO6inS(;q53IMNtf_82Cs*hQKg3kQ;|v6(%*7(K7Vh6fT0zHaN&ZZH9q+ zQ2W4i0qHf+y)2)=jDRTz!97OUiw%NTK{E#myn)MuE{^VCL1185f&Spj^-o(w4JJsj zU-6W#%AZs32!1<#2y z{8H*r&Ja$+@X1NTe-BioKL7 z$z?Uch00kuTboFd)YSPg#RFA9FCeyP0Xzy{iWYT`Ta>wW;=m`|qy z9!DMHVGV?D9N0vY13%9_d!O+gg9AK){UBZ?K0~C$0bHhZ-wb+_i8?{ggjWwD)RDi; zk-yxf;2)W9yL$-&uJlp2WPnq7@ybNtE);W%v?sP&!HQl~P>%$%LMa@K?NsWO=sen;G`ae=F8gn-swmK+ zSFHe09f(x?r{?gNzJKLM?1Z1vI>vrBKJGW{zYxs&**Lt;^shrT@Sn!R8*r?f-s69T zj$XCFf4YA)_?7YR9{e|t0-`i(BMs1d2YV*dJ3G#mp0 z?{57Tje_hEVU%%$o0oopPk~U#VF0KL480*)E&~w_kHCeve6kGqG4||0pGhj#WQ2qf zI_qp40g^l5olQW>VBs;)2ccZFG1$tZ0^yzZ0PNC!0OUqRA6||3_3I*X(fu^i7`;Bq z0#OiHEDop&nqF}N2d?ukR2h>7udaCq{zM$MvLhD2rNNaQHx)jzF>%OC%6Msi&rahAa&Q3wRFj{WeFEju+pVDez=d}@PC)UQ6o}JIeobM41_R?~x&1FwQMAnmx+%H{ z0pZ5(ddS|v!6$&dgG&TVjsR2-XlFGX{F1R$@h|X5pdD=g>qsZx59CKM4SLI2jTOYu z>wnwC$oz#txs?^N<0T{E3CTDY%UdAJqzZuBaFj?mBk!-FSpH8SOITM``%kGSd;cr+ z6^&??nus;wFc6|=I<}Rv#X&AGV~%63pfd(V%;4Y@m@xqwMx5mUghlhCDfv zr~?kaGL!n?=@<5wLZNN#^};RyM5t(U6jsQ)O8_Tv1c$6A&95oRmgF=Rf5Q_D2(TqblhENDc4F=(`!XX@K z2Wg(QL_K#}=oTktByXlS;6;J|r>fwNJip!m0$+vz!d_Au)1Fj0g4xERBE7kyfG7*mhE?95`FrZlA4sd7F znzc9~tV$d=f-XrAmN9M|KOA|ov`2X>P#Rpr{%xe-zX=9umxM1SHX73;apHs>&Abz! ze!&1AdBZZm1S6K{9qk9uT`C=J{LwNp(^CprU<+uX@dt7!A8s50_9}ueh9>6ULL~>v$NC5q!t&j(8v{km(5Fl|;6-$E0kh#Pr47~wk z07q(sGY(IiptJw?>BT+FHEA}ySyAg|KHll|h;FZ{C4Xnr)Qvrc3hY3xm$HT#rGbx~w)V$LoHg>_ zKJ?$>J_ZN}A)!JK5Wz4$h6Zgdj)CZn`h-`Sxv?s(%aF=(5DR!UdSdLu7&hboKEXn9 z-Ap}jK^X;gd=JP7tvUPv!-=%&Ji(|L-&Mk(10)S3LU21k$jrp_kpBTA10q!Z1oA*{ zTmJwQ`I{gV)5YLMH6-oee2i8dgEAQekO9Vf6@zAcUpX6n^54P8_@BWgAR+=&9im?* zVtc@8Pt22q4jIjPOcMtjrU@`36S7CJ%KsIzpkuN}Y!!aQfRQ^Y(Apcfq2cZ};5kNY(Z4W?!479a4 z`Jy+<^5>OEI>#9uKPO$F=^XMf{#&6PLzfF`M>7!MRzLTRY`+P}8wiI6#c{x`1y0^* z?dbeHR0QTh+_rER@DqXvictX``1jkjst0rVKr8{}VDw5*@-R3C82zt`2!lkQG7NV@ z2&`=!a0A^@(ZP!`>UQ`T96zfgxCUat4X}Ar=C3gNJ@WQu zuLxd{C$Pk&=H3F)CmN9pS9ta7I-fdV@3={5MZ1~!=RT2U)zAY|R=i`})kwd!M9_Es zj&-~IENRm}`f^Knly}Xk_0HUL=XU4L0`V3_QnF+0wU?WO)=IryY&m7bFW+EtLymip zw;J~m?`hi?mQ;w2EZ){A8>L4p9ZbBdz_l%8?hDDp+%LzXtz2dqUHTL~yE?-z`uNMv ztD~(CK6D&#T$euW$)L;T;S)vd;>Ry??dz&vtk5ngC$B}C_N2dJai;EhhI8r@J(uPC zymB^MiM%+RR`5A$8IO`pKE ztWZ4@*QhrsGINq`S9>|#O)Xe)xMLt>Podzb*N4c%$1LU6+<#yh?KC_{qmepC2YcN# zJxR)83fK8ok2V(SWe$&cmWI2{r$k-s>hShc3r_E+Mko^P+KXzMnqOVvR`zhU;S#i$ zIyQ^bs?3wSzh&jsxyya0x^GL`c|lKUL1)Eeb-pMU6TX_JSDVD@8V7U=zKmQeTxIdz zVYB=Cx~2LFImyM7G(T%!b#tSiZVd`3RH;(Aa!Z?Fqkf7#SuuE0#yr1wrdQTWm>uxh zE15gPH)!zYl2h$RtC#FpN21=DYv`;JYU6fs$*usgMP5@x?^r%sGH<8$0XF5;ff z|3_+_{reiB)rOwS9I1DH^t^g znWzSof4r|2SyGdG@3Fq*o3gat`#H|D{f{XW-prggDWd&^<|&=+)=|$XQ#spPQqKe~ zx49mpVJ)auKFz*wf$%lM%U%n+R31ccFNrL1JxEr5Qrgm7G*x%Sf&T58(t(ybmu9a> z8S+o4xpw=4b+}-B%{sK-)}DFw{4{fbW@cx0ok& z;5EON0?&??Ozi{dqE@AQUh#jp$mWu|de@BVRGOqGqfENP%D3Qo#QoF*%kF#nWq+2o3P0^0 z^_fns-j_jjQ@!AK;oK!J`I~d!oPIg?u-n|;#$~(eFDNzN4ZJ>I#ce*<6Q@Pb4R|%9 z*ou#+U)D6`)-BO;S{Sx*N&w z7k0$&X@%CVicQ&jr7EYd;97BUE;+8)I3?KNg}?3I^>iOm-!vhqkedEEu}4-~I`f}B zwTWFNy}?&6;;MP@rXBIVikEL~bhv4FTW&|j1tD2iPQN?fTCNiL&fK>UvfsNaJZE!D zsPP=0>nVGbtGYjTYnE^0`5;vB(vp38eaPmh842C7i#WC?xp)ZkZ1p-*c5`lHOV8Ye z3XL-<(Z`3Yv-Wv?5ULvb*k301kl@6#-QBe}&a&>_^knHZVKo9g$uqR(9t$E0&wlD* z{UKsa;LaB5$MMq*gx!^!M6|t|-wBK)bYJfBiCiR*O3ii&mpxHCRN_cikKAf_`;60> z<1Zeosz}{;O5Rx$vsQ7}`Us8IFSL^7l+-4#wq@#pVXygBtCOVX%qs5AUsvpt+a~!! zT8=+aMJ~trRp%}fm)awSSBiOPcHJ&j`7aK0ypD@1xgITE@IlMO+MMrg2nWNBu-0|< zs;Xmzsk`;;rriHD&G?S}rii)q{!*0C##(XLj)z(^(p)uGwDUQiKd@FZ+d8pmcFy?} zLErrus*RHb)@^bxIBBP@cu90t>f5Nc-u{Zaj_K)7QZ}5jxYHs!@OX>1Ub&y>Em3>3 zXQoY#o@^~u(TAsfI3IsdR5GGtyHN9m#ja`=w_jOu2N4vu4sriiMwte~?eYgQ(Xa8HQ9n(>Wihz`C{ z1B9%FY-`x`FA%KEOpclui=VSHFgajpW==p5Sj!%-WLxtPtkJ*u@AT~t=W{m9xk7Wz zbop5AOFSON_0r|jn%hR_9%>3j>tBAd`IBsdPo}+r-9G!{XWRnK%FZxOhJBl)V@9tLtso%@a61>I=_6JY z*Eh2?fV-JxkEc#r^N#Y}h-yMR`E(Sf?g$KQm-9?`P> zr|?+5X_w>@%WOHz(AL^M*|3jLBoRv++D!^D-o;$S%ouT!liyr==KajupN-g^B@>TE z540>)tM8j zB~^1pL_B`p9ryl5gwC<+jU4I@>4FW(_XI=s-K3uFA-NycU2kI`bR;er?!uh5*0M`Fwp$faqcU^D(BDo=By>s%y=StAhEbG{Gjlji3C zMD^HA4)8{zdN+gJj^pysy*RC@uG=^64P0=nt<8qaEr zuv*eV|AogrPH!|w&koyBva3Vm-Rbkj%RZF#wI34NvQ7Jq%dEo&Tnba^pB@UQyrpYh z-?KySlx^`X&fFM_@{dfh^)4*~pXk(*O)*CpCT%Y!eN8GcGCH;Q(PLw-H+D0_$(aIN zva7=M-DQ%_>OOwrO}-o8d*0|(y}M`8{NyY9bQibIedK92Wy|VF+2ht{*Sua=9xPC| zWYR9K(7v>@SmmSBv7 zPS=-a#Ry7$?MHg$u~8w)oYnHH#EI8OC(PVeJ@#)11|hy}>Q9$K4II))-3pn!mM!2nXE34|h{2%$)oA~k~aB2^$v+x&QB;*^_f-@_l=)*?XV8*0*NPL32RbDjYf)dqOnerS;`G!x(u}jaZmfj&Jx_CG4)bd3)wwyZoDE(1|4Nqb4M0@ zz?EBcZ7(l~E-rWKHhqFnhQqGld+cUNmN2eLxz+^_NMBO43D7FidllX3F>1zF;KS;U zKk(2jB|_O2);;dM=vV9;G*Si2kRwQYtdeI!lJHnV4kb3A{$x6Kq5&DRpo&pV7c z^BmFPSZLJ=gGv`nzkiSk7i@s6I8Ihk&gxOUOQ*BXdsAA!i`t}KxW7Z;z)j2ep5Et@l< z=ZM4_nwCW?Yg4zx#7wDEG^2K_ie3A5=8}2%^q&cv*q?I~SAXtA%En05Clba572$vu zE7gxJ-)E*UHxp-#-6&;YZ#$n9(HYRwWG(c%-dqAxde{Av1l6^|IlW6`aRV)5KeXu> zFNd6uzaFZ2ra0s(br89h!_4-)vD#%;CAOj|txT>P-e8Wi`3cFl;<$h1mszxFYt0(_ zY)nrZM7=I+?NPdR%MPA(Hn8E-hsP6XaRjQQIJLG(z02nBt>t4$j9@8-UABjZB#rNV|XTAf>7^tYSDMq{##@F{QI2#9ay3I7&?o2veIvpNxDOb#yzs9+uq` zl04A@<{ML)42NWtM28frYs80Q>fqLsSZ5<5YH{0U&$m5Vm}(K%NCB{dx)wVHq)>V6 zDhK&h%}rc{cREBw8#Mh<@Fs^vvzyz_5U*Ew?6vHA;n}BmqIc`=TCee~_v$_Mz8%Mf zg0xDk>oucPGGiD}ycS;Sa=CSPc|Nlxfk#NWz>_ zc~ZW>=pi*bj z+}#|Zu`T>A?gng=bcprNkP7@fB8q*eEi1eFlj(xp$47g44=r_AwlQ8>AYvxJH_!C1 zzGNq=lu291E7xDxs2iN8s%?S#%pO8FCY=Z-3m zg+*>N`b18bPWIdV5fF5LT2a}=s97!Tk>~10>34s?XaS#G<*6v@^3~ALXOV}f>Z^@My4DmH>8q>aY61~=|T}{ zXSk|#`w4rQOfr>t-^;%E>fC{{sPu_j>`xV4GVpoFE?MmQjxB&T->60<;TIga*0hGV zMx(04GrZ!P%Uyie8}n560tr=mjM>P&n}Hizr1j&?bPEr**k|Sn%JsUiackbO%3F=? zE)Ah0t(Y0Z1@fLSPJ$v{W(j`jNM1||1f0|7w`F!wzO{$<^EVM=_+m)+tp$QKl11>1 zJii~r4jJsB=!X}iGVnJvSS!tBIvEpR-At~(PPvOS(#obu?7gBHsGC#L5G=ixWa$=x zfvj=oJ@8DCSFmLAUxJi!dOdJnD}DR|lrOG3`hxMxmo7YZfiL~g)B$nH`$+xj#nvfU zmUMZli>XR{Q+1c6wtWCop$K-U&uZaJ<)FW7d)=u1%zVF{dx6S88m*%L{NQM6_AUAZ zOi=hbgCe9iU677@(5*w(-9LX=Eg}_3Zi8HPb)v=TQn8#5@LLvEv$N2vti(o8FYnW8*(YQAQrRELGc1|$5TA{ z9;v|?-=I1D+FML6e}|@vwpR0VyV=!8JPgAXIPv|r!0jd?*}aB|YY%G;Y0lTm4(w25 zwW6x(E$^>vP?Q8EtzUK9VyO4fz-R<_3}~<4MmUX|oQno}3DA3ij`jZYTh?pM#jDxq zpG;$%(8)Z`_CIcN88_$kKCIP5fGxCUe_m_ryw;jK;YV#}x13-iw-upc=^8JVIZ-Rk zVRCv1kCMLP(FjDUTtJM&421B9slJI50_l8^@E;s>0{gfBrm4S!`oOM+FkC@$!pW*h z$=I>fPIs!PMR$F=>d?x`GL(>TlZ{p~*VY=H1~1>TAj#%l_PBIo{z~GomPohuWiE7t z2UHPHdj4xa9XR}PO)B>@KTe_Ls@Z2(KOuAYzR%FScqT6j3(Za)SiVRe&#Q8-%_4&_ zKUOGGLf@^7?X!!K18UDsX8mrSoC6DVi?DAqNd@~$lp?@wLt%vu)(WK_8Q0@7Z1if6 zt$Z&U$wgRg>RyqYH%9iS?P{~{IN525oMAjC>zn_x3Jw&Xe^Tp~h3+X4Khj&XxbMWna`FeU0??1lM6!Od9%+`$uI$w=jQxsC zD%jKp8Y_CKoZ-BLj_^6IU`x2VC+L0*Coexig1&p-fS{k6==R$SJW{UT%52IS^o)AX zEeB|u2wts{?C-l`SlWS5zOci>>>xYH3H68kGHfQ3*^soK09^SkT;SmHgAQFqO@z$* z0?#}aAiUqNFdlDr`$%5t!NPqLh>G#|21CLXYytUI41IIfi+!a<A)oa59RzM%W;AD zD9c0!2=q6(_$2(e5O)+_mURUFmn?S@eO%)>icZ9w>cXk>aXeW*$`bd-NtROyj)x*g z3EYd05}X#8oOJ2 Date: Tue, 19 Mar 2024 09:30:43 +0100 Subject: [PATCH 24/32] Update src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala Better ScalaDoc Co-authored-by: Daniel Feismann <98817556+danielfeismann@users.noreply.github.com> --- src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala b/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala index c55165128b..a4373ebb07 100644 --- a/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala +++ b/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala @@ -16,7 +16,7 @@ import squants.Power import java.util.UUID /** Translating input data to a format that can be used by aggregation - * strategies, em strats etc. Furthermore, sanity checks on calculated data is + * strategies, em strategies etc.. Furthermore, sanity checks on calculated data is * performed. */ final case class EmModelShell( From 7b70c0434534a74bde01ec90fc9dd75e6bba1364 Mon Sep 17 00:00:00 2001 From: Sebastian Peter Date: Tue, 19 Mar 2024 10:42:29 +0100 Subject: [PATCH 25/32] Taking care of some reviewer comments Using DefaultQuantities everywhere where possible --- input/samples/vn_simona/vn_simona.conf | 3 + .../edu/ie3/simona/agent/em/EmAgent.scala | 9 ++- .../ie3/simona/agent/grid/DBFSAlgorithm.scala | 7 +-- .../simona/agent/grid/PowerFlowSupport.scala | 9 ++- .../ParticipantAgentFundamentals.scala | 9 +-- .../simona/agent/participant/data/Data.scala | 10 ++-- .../participant/hp/HpAgentFundamentals.scala | 12 ++-- .../simona/model/em/EmAggregateSelfOpt.scala | 8 +-- .../model/em/EmAggregateSimpleSum.scala | 17 +----- .../ie3/simona/model/em/EmModelShell.scala | 8 +-- .../model/em/PrioritizedFlexStrat.scala | 14 ++--- .../model/em/ProportionalFlexStrat.scala | 16 +---- .../ApparentPowerAndHeatParticipant.scala | 4 +- .../simona/model/participant/BMModel.scala | 5 +- .../simona/model/participant/ChpModel.scala | 8 +-- .../simona/model/participant/HpModel.scala | 5 +- .../model/participant/SystemParticipant.scala | 3 +- .../simona/model/participant/WecModel.scala | 3 +- .../model/participant/control/QControl.scala | 3 +- .../model/participant/evcs/EvcsModel.scala | 60 +++++++++---------- .../thermal/CylindricalThermalStorage.scala | 12 ++-- .../model/thermal/RandomStorageState.scala | 4 +- .../simona/model/thermal/ThermalGrid.scala | 21 +++---- .../simona/model/thermal/ThermalHouse.scala | 17 +++--- .../simona/model/thermal/ThermalStorage.scala | 2 - .../flex/MinMaxFlexibilityMessage.scala | 18 ++++++ .../scala/quantities/DefaultQuantities.scala | 11 +++- 27 files changed, 150 insertions(+), 148 deletions(-) diff --git a/input/samples/vn_simona/vn_simona.conf b/input/samples/vn_simona/vn_simona.conf index e270dc8b94..38626c593d 100644 --- a/input/samples/vn_simona/vn_simona.conf +++ b/input/samples/vn_simona/vn_simona.conf @@ -158,6 +158,9 @@ simona.runtime.participant.hp = { individualConfigs = [] } +# # # # # +# ATTENTION: calculateMissingReactivePowerWithModel and scaling is ignored here. +# # # # # simona.runtime.participant.em = { defaultConfig = { calculateMissingReactivePowerWithModel = false 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 e811df8e59..06a7bf598a 100644 --- a/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala +++ b/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala @@ -28,10 +28,9 @@ import edu.ie3.simona.ontology.messages.flex.MinMaxFlexibilityMessage.ProvideMin import edu.ie3.simona.ontology.messages.{Activation, SchedulerMessage} import edu.ie3.simona.util.TickUtil.TickLong import edu.ie3.util.quantities.QuantityUtils.RichQuantityDouble -import edu.ie3.util.scala.quantities.Megavars +import edu.ie3.util.scala.quantities.DefaultQuantities._ import org.apache.pekko.actor.typed.scaladsl.Behaviors import org.apache.pekko.actor.typed.{ActorRef, Behavior} -import squants.energy.{Kilowatts, Megawatts} import java.time.ZonedDateTime @@ -273,7 +272,7 @@ object EmAgent { case Left(_) => // We're not em-controlled ourselves, // always desire to come as close as possible to 0 kW - val setPower = Kilowatts(0) + val setPower = zeroKW val flexControl = modelShell.determineFlexControl(allFlexOptions, setPower) @@ -399,8 +398,8 @@ object EmAgent { } .getOrElse( ApparentPower( - Megawatts(0d), - Megavars(0d), + zeroMW, + zeroMVAr, ) ) diff --git a/src/main/scala/edu/ie3/simona/agent/grid/DBFSAlgorithm.scala b/src/main/scala/edu/ie3/simona/agent/grid/DBFSAlgorithm.scala index f47f3ef8b8..039331b2a1 100644 --- a/src/main/scala/edu/ie3/simona/agent/grid/DBFSAlgorithm.scala +++ b/src/main/scala/edu/ie3/simona/agent/grid/DBFSAlgorithm.scala @@ -39,7 +39,7 @@ import edu.ie3.simona.ontology.messages.Activation import edu.ie3.simona.ontology.messages.PowerMessage._ import edu.ie3.simona.ontology.messages.SchedulerMessage.Completion import edu.ie3.simona.util.TickUtil.TickLong -import edu.ie3.util.scala.quantities.Megavars +import edu.ie3.util.scala.quantities.DefaultQuantities._ import edu.ie3.util.scala.quantities.SquantsUtils.RichElectricPotential import org.apache.pekko.actor.typed.scaladsl.AskPattern._ import org.apache.pekko.actor.typed.scaladsl.adapter.TypedActorRefOps @@ -53,7 +53,6 @@ import org.apache.pekko.pattern.ask import org.apache.pekko.util.{Timeout => PekkoTimeout} import org.slf4j.Logger import squants.Each -import squants.energy.Megawatts import java.time.{Duration, ZonedDateTime} import java.util.UUID @@ -370,8 +369,8 @@ trait DBFSAlgorithm extends PowerFlowSupport with GridResultsSupport { case _ => /* TODO: As long as there are no multiple slack nodes, provide "real" power only for the slack node */ ( - Megawatts(0d), - Megavars(0d), + zeroMW, + zeroMVAr, ) } .getOrElse { diff --git a/src/main/scala/edu/ie3/simona/agent/grid/PowerFlowSupport.scala b/src/main/scala/edu/ie3/simona/agent/grid/PowerFlowSupport.scala index 5d34d06bd4..c44d01582f 100644 --- a/src/main/scala/edu/ie3/simona/agent/grid/PowerFlowSupport.scala +++ b/src/main/scala/edu/ie3/simona/agent/grid/PowerFlowSupport.scala @@ -14,14 +14,13 @@ import edu.ie3.powerflow.model.PowerFlowResult.SuccessFullPowerFlowResult.ValidN import edu.ie3.powerflow.model.StartData.WithForcedStartVoltages import edu.ie3.powerflow.model.enums.NodeType import edu.ie3.simona.agent.grid.ReceivedValues.ReceivedSlackVoltageValues +import edu.ie3.simona.agent.grid.VoltageMessage.ProvideSlackVoltageMessage.ExchangeVoltage import edu.ie3.simona.exceptions.agent.DBFSAlgorithmException import edu.ie3.simona.model.grid._ import edu.ie3.simona.ontology.messages.PowerMessage.ProvidePowerMessage -import VoltageMessage.ProvideSlackVoltageMessage.ExchangeVoltage -import edu.ie3.util.scala.quantities.Kilovars +import edu.ie3.util.scala.quantities.DefaultQuantities._ import org.slf4j.Logger import squants.electro.ElectricPotential -import squants.energy.Kilowatts import java.util.UUID import scala.collection.mutable @@ -104,8 +103,8 @@ trait PowerFlowSupport { } .foldLeft( ( - Kilowatts(0d), - Kilovars(0d), + zeroKW, + zeroKVAr, ) ) { case ((pSum, qSum), powerMessage) => ( diff --git a/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgentFundamentals.scala b/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgentFundamentals.scala index 894c0e1c2e..2ec46f6450 100644 --- a/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgentFundamentals.scala +++ b/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgentFundamentals.scala @@ -85,6 +85,7 @@ import edu.ie3.simona.ontology.messages.services.ServiceMessage.{ import edu.ie3.simona.util.TickUtil._ import edu.ie3.util.quantities.PowerSystemUnits._ import edu.ie3.util.quantities.QuantityUtils.RichQuantityDouble +import edu.ie3.util.scala.quantities.DefaultQuantities._ import edu.ie3.util.scala.quantities.{Megavars, QuantityUtil, ReactivePower} import org.apache.pekko.actor.typed.scaladsl.adapter.ClassicActorRefOps import org.apache.pekko.actor.typed.{ActorRef => TypedActorRef} @@ -1001,7 +1002,7 @@ protected trait ParticipantAgentFundamentals[ )(p) } else { _: Power => /* Use trivial reactive power */ - Megavars(0d) + zeroMVAr } /** Try to get and process the received data @@ -2051,7 +2052,7 @@ object ParticipantAgentFundamentals { "Unable to determine average active power. Apply 0 instead. Cause:\n\t{}", exception, ) - Megawatts(0d) + zeroMW } val q = QuantityUtil.average[Power, Energy]( @@ -2074,7 +2075,7 @@ object ParticipantAgentFundamentals { "Unable to determine average reactive power. Apply 0 instead. Cause:\n\t{}", exception, ) - Megavars(0d) + zeroMVAr } ApparentPower(p, q) @@ -2132,7 +2133,7 @@ object ParticipantAgentFundamentals { "Unable to determine average thermal power. Apply 0 instead. Cause:\n\t{}", exception, ) - Megawatts(0d) + zeroMW } ApparentPowerAndHeat(apparentPower.p, apparentPower.q, qDot) diff --git a/src/main/scala/edu/ie3/simona/agent/participant/data/Data.scala b/src/main/scala/edu/ie3/simona/agent/participant/data/Data.scala index 35748f755f..cb56e1af92 100644 --- a/src/main/scala/edu/ie3/simona/agent/participant/data/Data.scala +++ b/src/main/scala/edu/ie3/simona/agent/participant/data/Data.scala @@ -13,6 +13,7 @@ import edu.ie3.util.quantities.interfaces.EnergyPrice import edu.ie3.util.scala.quantities.{Kilovars, Megavars, ReactivePower} import squants.energy.{Power, Kilowatts, Megawatts} import tech.units.indriya.ComparableQuantity +import edu.ie3.util.scala.quantities.DefaultQuantities._ import java.time.ZonedDateTime import scala.jdk.OptionConverters.RichOptional @@ -60,10 +61,7 @@ object Data { val qDot: Power } - val ZERO_POWER: ApparentPower = ApparentPower( - Megawatts(0d), - Megavars(0d), - ) + val ZERO_POWER: ApparentPower = ApparentPower(zeroMW, zeroMVAr) /** Active power as participant simulation result * @@ -76,7 +74,7 @@ object Data { override def toApparentPower: ApparentPower = ApparentPower( p, - Megavars(0d), + zeroMVAr, ) override def add(q: ReactivePower): ApparentPower = @@ -116,7 +114,7 @@ object Data { override def toApparentPower: ApparentPower = ApparentPower( p, - Megavars(0d), + zeroMVAr, ) override def add(q: ReactivePower): ApparentPowerAndHeat = diff --git a/src/main/scala/edu/ie3/simona/agent/participant/hp/HpAgentFundamentals.scala b/src/main/scala/edu/ie3/simona/agent/participant/hp/HpAgentFundamentals.scala index 0fb9103312..a428cd3cab 100644 --- a/src/main/scala/edu/ie3/simona/agent/participant/hp/HpAgentFundamentals.scala +++ b/src/main/scala/edu/ie3/simona/agent/participant/hp/HpAgentFundamentals.scala @@ -50,11 +50,11 @@ import edu.ie3.simona.ontology.messages.flex.FlexibilityMessage.{ import edu.ie3.simona.ontology.messages.services.WeatherMessage.WeatherData import edu.ie3.util.quantities.PowerSystemUnits.PU import edu.ie3.util.quantities.QuantityUtils.RichQuantityDouble +import edu.ie3.util.scala.quantities.DefaultQuantities._ import edu.ie3.util.scala.quantities.{Megavars, ReactivePower} import org.apache.pekko.actor.typed.scaladsl.adapter.ClassicActorRefOps import org.apache.pekko.actor.typed.{ActorRef => TypedActorRef} import org.apache.pekko.actor.{ActorRef, FSM} -import squants.energy.Megawatts import squants.{Dimensionless, Each, Power} import java.time.ZonedDateTime @@ -76,9 +76,9 @@ trait HpAgentFundamentals override protected val pdClassTag: ClassTag[ApparentPowerAndHeat] = classTag[ApparentPowerAndHeat] override val alternativeResult: ApparentPowerAndHeat = ApparentPowerAndHeat( - Megawatts(0d), - Megavars(0d), - Megawatts(0d), + zeroMW, + zeroMVAr, + zeroMW, ) /** Partial function, that is able to transfer @@ -116,8 +116,8 @@ trait HpAgentFundamentals isRunning = false, -1, None, - Megawatts(0d), - Megawatts(0d), + zeroMW, + zeroMW, ThermalGrid.startingState(thermalGrid), None, ) diff --git a/src/main/scala/edu/ie3/simona/model/em/EmAggregateSelfOpt.scala b/src/main/scala/edu/ie3/simona/model/em/EmAggregateSelfOpt.scala index a5e8421e3c..c3dbd27ebe 100644 --- a/src/main/scala/edu/ie3/simona/model/em/EmAggregateSelfOpt.scala +++ b/src/main/scala/edu/ie3/simona/model/em/EmAggregateSelfOpt.scala @@ -9,8 +9,8 @@ package edu.ie3.simona.model.em import edu.ie3.datamodel.models.input.AssetInput import edu.ie3.datamodel.models.input.system.PvInput import edu.ie3.simona.ontology.messages.flex.MinMaxFlexibilityMessage.ProvideMinMaxFlexOptions +import edu.ie3.util.scala.quantities.DefaultQuantities._ import squants.Power -import squants.energy.Kilowatts /** Aggregates flex reference power with the target of reaching 0kW, while * optionally excluding positive PV potential from the calculation @@ -26,7 +26,7 @@ final case class EmAggregateSelfOpt(pvFlex: Boolean) extends EmAggregateFlex { ] ): (Power, Power, Power) = { val (minSum, maxSum) = - flexOptions.foldLeft((Kilowatts(0d), Kilowatts(0d))) { + flexOptions.foldLeft((zeroKW, zeroKW)) { case ( (sumMin, sumMax), (_, ProvideMinMaxFlexOptions(_, _, addMin, addMax)), @@ -41,7 +41,7 @@ final case class EmAggregateSelfOpt(pvFlex: Boolean) extends EmAggregateFlex { if (pvFlex) maxSum else - flexOptions.foldLeft(Kilowatts(0d)) { + flexOptions.foldLeft(zeroKW) { case ( maxSumExclPv, (inputModel, ProvideMinMaxFlexOptions(_, _, addMin, addMax)), @@ -54,7 +54,7 @@ final case class EmAggregateSelfOpt(pvFlex: Boolean) extends EmAggregateFlex { } // take the closest power possible to zero - val refAgg = minSum.max(maxRefSum.min(Kilowatts(0d))) + val refAgg = minSum.max(maxRefSum.min(zeroKW)) (refAgg, minSum, maxSum) } diff --git a/src/main/scala/edu/ie3/simona/model/em/EmAggregateSimpleSum.scala b/src/main/scala/edu/ie3/simona/model/em/EmAggregateSimpleSum.scala index 28e343fb6d..f869616801 100644 --- a/src/main/scala/edu/ie3/simona/model/em/EmAggregateSimpleSum.scala +++ b/src/main/scala/edu/ie3/simona/model/em/EmAggregateSimpleSum.scala @@ -9,7 +9,6 @@ package edu.ie3.simona.model.em import edu.ie3.datamodel.models.input.AssetInput import edu.ie3.simona.ontology.messages.flex.MinMaxFlexibilityMessage.ProvideMinMaxFlexOptions import squants.Power -import squants.energy.Kilowatts /** Aggregates reference, minimum and maximum power by just simply summing up * each value @@ -21,18 +20,8 @@ object EmAggregateSimpleSum extends EmAggregateFlex { (_ <: AssetInput, ProvideMinMaxFlexOptions) ] ): (Power, Power, Power) = { - flexOptions.foldLeft( - (Kilowatts(0d), Kilowatts(0d), Kilowatts(0d)) - ) { - case ( - (sumRef, sumMin, sumMax), - (_, ProvideMinMaxFlexOptions(_, addRef, addMin, addMax)), - ) => - ( - sumRef + addRef, - sumMin + addMin, - sumMax + addMax, - ) - } + flexOptions.map { case (_, flex: ProvideMinMaxFlexOptions) => + flex + }.flexSum } } diff --git a/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala b/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala index a4373ebb07..79efa1d913 100644 --- a/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala +++ b/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala @@ -16,8 +16,8 @@ import squants.Power import java.util.UUID /** Translating input data to a format that can be used by aggregation - * strategies, em strategies etc.. Furthermore, sanity checks on calculated data is - * performed. + * strategies, em strategies etc.. Furthermore, sanity checks on calculated + * data is performed. */ final case class EmModelShell( uuid: UUID, @@ -113,11 +113,11 @@ object EmModelShell { def apply( uuid: UUID, id: String, - modelStrat: String, + modelStrategyName: String, modelConfig: EmRuntimeConfig, ): EmModelShell = { - val modelStrategy = modelStrat match { + val modelStrategy = modelStrategyName match { case "PROPORTIONAL" => ProportionalFlexStrat case "PRIORITIZED" => PrioritizedFlexStrat(modelConfig.pvFlex) case unknown => diff --git a/src/main/scala/edu/ie3/simona/model/em/PrioritizedFlexStrat.scala b/src/main/scala/edu/ie3/simona/model/em/PrioritizedFlexStrat.scala index 27844f01e5..3fc7ced588 100644 --- a/src/main/scala/edu/ie3/simona/model/em/PrioritizedFlexStrat.scala +++ b/src/main/scala/edu/ie3/simona/model/em/PrioritizedFlexStrat.scala @@ -16,8 +16,8 @@ import edu.ie3.datamodel.models.input.system.{ import edu.ie3.simona.exceptions.CriticalFailureException import edu.ie3.simona.model.em.EmModelStrat.tolerance import edu.ie3.simona.ontology.messages.flex.MinMaxFlexibilityMessage.ProvideMinMaxFlexOptions +import edu.ie3.util.scala.quantities.DefaultQuantities._ import squants.Power -import squants.energy.Kilowatts import java.util.UUID @@ -88,9 +88,9 @@ final case class PrioritizedFlexStrat(pvFlex: Boolean) extends EmModelStrat { } .filter(_ => pvFlex) // only if enabled - if (Kilowatts(0d).~=(targetDelta)(tolerance)) { + if (zeroKW.~=(targetDelta)(tolerance)) { Seq.empty - } else if (targetDelta < Kilowatts(0d)) { + } else if (targetDelta < zeroKW) { // suggested power too low, try to store difference/increase load val orderedParticipants = @@ -107,10 +107,10 @@ final case class PrioritizedFlexStrat(pvFlex: Boolean) extends EmModelStrat { val flexPotential = flexOption.ref - flexOption.max - if (Kilowatts(0d).~=(remainingExcessPower)(tolerance)) { + if (zeroKW.~=(remainingExcessPower)(tolerance)) { // we're already there (besides rounding error) (issueCtrlMsgs, None) - } else if (Kilowatts(0d).~=(flexPotential)(tolerance)) { + } else if (zeroKW.~=(flexPotential)(tolerance)) { // device does not offer usable flex potential here (issueCtrlMsgs, Some(remainingExcessPower)) } else if (remainingExcessPower < flexPotential) { @@ -156,10 +156,10 @@ final case class PrioritizedFlexStrat(pvFlex: Boolean) extends EmModelStrat { val flexPotential = flexOption.ref - flexOption.min - if (Kilowatts(0d).~=(remainingExcessPower)(tolerance)) { + if (zeroKW.~=(remainingExcessPower)(tolerance)) { // we're already there (besides rounding error) (issueCtrlMsgs, None) - } else if (Kilowatts(0d).~=(flexPotential)(tolerance)) { + } else if (zeroKW.~=(flexPotential)(tolerance)) { // device does not offer usable flex potential here (issueCtrlMsgs, Some(remainingExcessPower)) } else if (remainingExcessPower > flexPotential) { diff --git a/src/main/scala/edu/ie3/simona/model/em/ProportionalFlexStrat.scala b/src/main/scala/edu/ie3/simona/model/em/ProportionalFlexStrat.scala index f0034ad7c8..4be1699cef 100644 --- a/src/main/scala/edu/ie3/simona/model/em/ProportionalFlexStrat.scala +++ b/src/main/scala/edu/ie3/simona/model/em/ProportionalFlexStrat.scala @@ -10,7 +10,6 @@ import edu.ie3.datamodel.models.input.AssetInput import EmModelStrat.tolerance import edu.ie3.simona.ontology.messages.flex.MinMaxFlexibilityMessage.ProvideMinMaxFlexOptions import squants.Power -import squants.energy.Kilowatts import java.util.UUID @@ -45,20 +44,7 @@ object ProportionalFlexStrat extends EmModelStrat { } // sum up reference, minimum and maximum power of all connected devices - val (totalRef, totalMin, totalMax) = flexOptions - .foldLeft( - (Kilowatts(0d), Kilowatts(0d), Kilowatts(0d)) - ) { - case ( - (sumRef, sumMin, sumMax), - ProvideMinMaxFlexOptions(_, addRef, addMin, addMax), - ) => - ( - sumRef + addRef, - sumMin + addMin, - sumMax + addMax, - ) - } + val (totalRef, totalMin, totalMax) = flexOptions.flexSum if (target.~=(totalRef)(tolerance)) { Seq.empty diff --git a/src/main/scala/edu/ie3/simona/model/participant/ApparentPowerAndHeatParticipant.scala b/src/main/scala/edu/ie3/simona/model/participant/ApparentPowerAndHeatParticipant.scala index a899dc3805..eb51174cb5 100644 --- a/src/main/scala/edu/ie3/simona/model/participant/ApparentPowerAndHeatParticipant.scala +++ b/src/main/scala/edu/ie3/simona/model/participant/ApparentPowerAndHeatParticipant.scala @@ -7,7 +7,7 @@ package edu.ie3.simona.model.participant import edu.ie3.simona.agent.participant.data.Data.PrimaryData.ApparentPowerAndHeat -import squants.energy.Megawatts +import edu.ie3.util.scala.quantities.DefaultQuantities._ import squants.{Dimensionless, Power} trait ApparentPowerAndHeatParticipant[ @@ -27,7 +27,7 @@ trait ApparentPowerAndHeatParticipant[ if (isInOperation(tick)) calculateHeat(tick, modelState, data) else - Megawatts(0d) + zeroMW ApparentPowerAndHeat(apparentPower.p, apparentPower.q, heat) } diff --git a/src/main/scala/edu/ie3/simona/model/participant/BMModel.scala b/src/main/scala/edu/ie3/simona/model/participant/BMModel.scala index 9309bcc381..418af563d5 100644 --- a/src/main/scala/edu/ie3/simona/model/participant/BMModel.scala +++ b/src/main/scala/edu/ie3/simona/model/participant/BMModel.scala @@ -13,8 +13,9 @@ import edu.ie3.simona.model.participant.control.QControl import edu.ie3.simona.ontology.messages.flex.FlexibilityMessage.ProvideFlexOptions import edu.ie3.simona.ontology.messages.flex.MinMaxFlexibilityMessage.ProvideMinMaxFlexOptions import edu.ie3.util.scala.OperationInterval +import edu.ie3.util.scala.quantities.DefaultQuantities._ import edu.ie3.util.scala.quantities.EnergyPrice -import squants.energy.{Kilowatts, Megawatts} +import squants.energy.Megawatts import squants.{Dimensionless, Money, Power, Temperature} import java.time.ZonedDateTime @@ -230,7 +231,7 @@ final case class BMModel( ): ProvideFlexOptions = { val power = calculateActivePower(lastState, data) - ProvideMinMaxFlexOptions(uuid, power, power, Kilowatts(0d)) + ProvideMinMaxFlexOptions(uuid, power, power, zeroKW) } override def handleControlledPowerChange( diff --git a/src/main/scala/edu/ie3/simona/model/participant/ChpModel.scala b/src/main/scala/edu/ie3/simona/model/participant/ChpModel.scala index 31bc84c8c5..5af1b68236 100644 --- a/src/main/scala/edu/ie3/simona/model/participant/ChpModel.scala +++ b/src/main/scala/edu/ie3/simona/model/participant/ChpModel.scala @@ -18,12 +18,12 @@ import edu.ie3.simona.ontology.messages.flex.MinMaxFlexibilityMessage.ProvideMin import edu.ie3.util.quantities.PowerSystemUnits import edu.ie3.util.scala.OperationInterval import edu.ie3.util.scala.quantities.DefaultQuantities +import edu.ie3.util.scala.quantities.DefaultQuantities._ +import squants.energy.Kilowatts import squants.{Energy, Power, Seconds, Time} -import squants.energy.{KilowattHours, Kilowatts} - -import java.util.UUID import java.time.ZonedDateTime +import java.util.UUID /** Model of a combined heat and power plant (CHP) with a [[ThermalStorage]] * medium and its current [[ChpState]]. @@ -199,7 +199,7 @@ final case class ChpModel( chpData: ChpRelevantData ): ChpState = { val differenceEnergy = chpEnergy(chpData) - chpData.heatDemand - if (differenceEnergy < KilowattHours(0d)) { + if (differenceEnergy < zeroKWH) { // Returned lack is always zero, because demand is covered. storage.tryToTakeAndReturnLack(differenceEnergy * -1) calculateStateRunningSurplus(chpData) diff --git a/src/main/scala/edu/ie3/simona/model/participant/HpModel.scala b/src/main/scala/edu/ie3/simona/model/participant/HpModel.scala index de90c4c839..7af9279611 100644 --- a/src/main/scala/edu/ie3/simona/model/participant/HpModel.scala +++ b/src/main/scala/edu/ie3/simona/model/participant/HpModel.scala @@ -18,6 +18,7 @@ import edu.ie3.simona.ontology.messages.flex.MinMaxFlexibilityMessage.ProvideMin import edu.ie3.util.quantities.PowerSystemUnits import edu.ie3.util.scala.OperationInterval import edu.ie3.util.scala.quantities.DefaultQuantities +import edu.ie3.util.scala.quantities.DefaultQuantities._ import squants.energy.Kilowatts import squants.{Power, Temperature} @@ -217,14 +218,14 @@ final case class HpModel( val lowerBoundary = if (canBeOutOfOperation) - Kilowatts(0d) + zeroKW else updatedState.activePower val upperBoundary = if (canOperate) sRated * cosPhiRated else - Kilowatts(0d) + zeroKW ProvideMinMaxFlexOptions( uuid, diff --git a/src/main/scala/edu/ie3/simona/model/participant/SystemParticipant.scala b/src/main/scala/edu/ie3/simona/model/participant/SystemParticipant.scala index ed502038d8..44c70539d7 100644 --- a/src/main/scala/edu/ie3/simona/model/participant/SystemParticipant.scala +++ b/src/main/scala/edu/ie3/simona/model/participant/SystemParticipant.scala @@ -14,6 +14,7 @@ import edu.ie3.simona.model.SystemComponent import edu.ie3.simona.model.participant.control.QControl import edu.ie3.simona.ontology.messages.flex.FlexibilityMessage.ProvideFlexOptions import edu.ie3.util.scala.OperationInterval +import edu.ie3.util.scala.quantities.DefaultQuantities._ import edu.ie3.util.scala.quantities.{ DefaultQuantities, Megavars, @@ -238,7 +239,7 @@ abstract class SystemParticipant[ logger.warn( s"Active power of model exceeds sRated. Set reactive power to 0!" ) - Megavars(0d) + zeroMVAr } else { Megavars( Math.sqrt(powerSquaredDifference) diff --git a/src/main/scala/edu/ie3/simona/model/participant/WecModel.scala b/src/main/scala/edu/ie3/simona/model/participant/WecModel.scala index 9b02f1dd05..f2f34ac771 100644 --- a/src/main/scala/edu/ie3/simona/model/participant/WecModel.scala +++ b/src/main/scala/edu/ie3/simona/model/participant/WecModel.scala @@ -22,6 +22,7 @@ import edu.ie3.simona.ontology.messages.flex.FlexibilityMessage.ProvideFlexOptio import edu.ie3.simona.ontology.messages.flex.MinMaxFlexibilityMessage.ProvideMinMaxFlexOptions import edu.ie3.util.quantities.PowerSystemUnits._ import edu.ie3.util.scala.OperationInterval +import edu.ie3.util.scala.quantities.DefaultQuantities._ import squants._ import squants.energy.{Kilowatts, Watts} import squants.mass.{Kilograms, KilogramsPerCubicMeter} @@ -195,7 +196,7 @@ final case class WecModel( ): ProvideFlexOptions = { val power = calculateActivePower(ConstantState, data) - ProvideMinMaxFlexOptions(uuid, power, power, Kilowatts(0d)) + ProvideMinMaxFlexOptions(uuid, power, power, zeroKW) } override def handleControlledPowerChange( diff --git a/src/main/scala/edu/ie3/simona/model/participant/control/QControl.scala b/src/main/scala/edu/ie3/simona/model/participant/control/QControl.scala index 37355d55b4..25504c2fff 100644 --- a/src/main/scala/edu/ie3/simona/model/participant/control/QControl.scala +++ b/src/main/scala/edu/ie3/simona/model/participant/control/QControl.scala @@ -12,6 +12,7 @@ import edu.ie3.simona.exceptions.QControlException import edu.ie3.simona.model.system.Characteristic import edu.ie3.simona.model.system.Characteristic.XYPair import edu.ie3.util.quantities.PowerSystemUnits.PU +import edu.ie3.util.scala.quantities.DefaultQuantities._ import edu.ie3.util.scala.quantities.{Megavars, ReactivePower} import squants.{Dimensionless, Each, Power} import tech.units.indriya.AbstractUnit @@ -25,7 +26,7 @@ sealed trait QControl { protected val _cosPhiMultiplication: (Double, Power) => ReactivePower = (cosPhi: Double, p: Power) => if ((cosPhi - 1).abs < 0.0000001) { - Megavars(0d) + zeroMVAr } else { /* q = p * tan( phi ) = p * tan( acos( cosphi )) */ Megavars((p * tan(acos(cosPhi))).toMegawatts) diff --git a/src/main/scala/edu/ie3/simona/model/participant/evcs/EvcsModel.scala b/src/main/scala/edu/ie3/simona/model/participant/evcs/EvcsModel.scala index ff797c10f6..b00ec5e5e3 100644 --- a/src/main/scala/edu/ie3/simona/model/participant/evcs/EvcsModel.scala +++ b/src/main/scala/edu/ie3/simona/model/participant/evcs/EvcsModel.scala @@ -19,6 +19,7 @@ import edu.ie3.simona.model.participant.evcs.uncontrolled.{ ConstantPowerCharging, MaximumPowerCharging, } +import edu.ie3.util.scala.quantities.DefaultQuantities._ import edu.ie3.simona.model.participant.{ CalcRelevantData, FlexChangeIndicator, @@ -221,17 +222,16 @@ final case class EvcsModel( tickStop > lastSchedulingTick && tickStart < currentTick } .sortBy(_.tickStart) - .foldLeft(KilowattHours(0d)) { - case (accumulatedEnergy, scheduleEntry) => - /* Only the timeframe from the start of last scheduling update and current tick must be considered */ - val trimmedEntry = trimScheduleEntry( - scheduleEntry, - lastSchedulingTick, - currentTick, - ) + .foldLeft(zeroKWH) { case (accumulatedEnergy, scheduleEntry) => + /* Only the timeframe from the start of last scheduling update and current tick must be considered */ + val trimmedEntry = trimScheduleEntry( + scheduleEntry, + lastSchedulingTick, + currentTick, + ) - /* Determine the energy charged within this slice of the schedule and accumulate it */ - accumulatedEnergy + calcChargedEnergy(trimmedEntry) + /* Determine the energy charged within this slice of the schedule and accumulate it */ + accumulatedEnergy + calcChargedEnergy(trimmedEntry) } /* Update EV with the charged energy during the charging interval */ ev.copy( @@ -312,7 +312,7 @@ final case class EvcsModel( // start charging right away at lastTick, create mock // schedule entries that end before lastTick val startingSchedules = lastEvMap.keys.map { - _ -> ScheduleEntry(lastTick, lastTick, Kilowatts(0d)) + _ -> ScheduleEntry(lastTick, lastTick, zeroKW) } val (currentEvs, currentSchedules, evResults, evcsResults) = @@ -348,7 +348,7 @@ final case class EvcsModel( createEvResult( ev, tick, - Kilowatts(0d), + zeroKW, voltageMagnitude, ) } @@ -381,7 +381,7 @@ final case class EvcsModel( val currentActiveEntries = stillActive ++ newActiveEntries // create the EVCS result with all currently active entries - val evcsP = currentActiveEntries.foldLeft(Kilowatts(0d)) { + val evcsP = currentActiveEntries.foldLeft(zeroKW) { case (powerSum, _ -> entry) => powerSum + entry.chargingPower } @@ -418,7 +418,7 @@ final case class EvcsModel( createEvResult( _, currentTick, - Kilowatts(0d), + zeroKW, voltageMagnitude, ) } @@ -533,7 +533,7 @@ final case class EvcsModel( val (maxCharging, preferredPower, forcedCharging, maxDischarging) = currentEvs.foldLeft( - (Kilowatts(0d), Kilowatts(0d), Kilowatts(0d), Kilowatts(0d)) + (zeroKW, zeroKW, zeroKW, zeroKW) ) { case ( (chargingSum, preferredSum, forcedSum, dischargingSum), @@ -553,23 +553,23 @@ final case class EvcsModel( if (!isFull(ev)) maxPower else - Kilowatts(0d) + zeroKW val forced = if (isEmpty(ev) && !isInLowerMargin(ev)) preferred.getOrElse(maxPower) else - Kilowatts(0d) + zeroKW val maxDischarging = if (!isEmpty(ev) && vehicle2grid) maxPower * -1 else - Kilowatts(0d) + zeroKW ( chargingSum + maxCharging, - preferredSum + preferred.getOrElse(Kilowatts(0d)), + preferredSum + preferred.getOrElse(zeroKW), forcedSum + forced, dischargingSum + maxDischarging, ) @@ -577,7 +577,7 @@ final case class EvcsModel( // if we need to charge at least one EV, we cannot discharge any other val (adaptedMaxDischarging, adaptedPreferred) = - if (forcedCharging > Kilowatts(0d)) + if (forcedCharging > zeroKW) (forcedCharging, preferredPower.max(forcedCharging)) else (maxDischarging, preferredPower) @@ -599,7 +599,7 @@ final case class EvcsModel( ): (EvcsState, FlexChangeIndicator) = { val currentEvs = determineCurrentEvs(data, lastState) - if (setPower == Kilowatts(0d)) + if (setPower == zeroKW) return ( EvcsState( evs = currentEvs, @@ -611,14 +611,14 @@ final case class EvcsModel( // applicable evs can be charged/discharged, other evs cannot val applicableEvs = currentEvs.filter { ev => - if (setPower > Kilowatts(0d)) + if (setPower > zeroKW) !isFull(ev) else !isEmpty(ev) } val (forcedChargingEvs, regularChargingEvs) = - if (setPower > Kilowatts(0d)) + if (setPower > zeroKW) // lower margin is excluded since charging is not required here anymore applicableEvs.partition { ev => isEmpty(ev) && !isInLowerMargin(ev) @@ -702,15 +702,15 @@ final case class EvcsModel( if (evs.isEmpty) return (Seq.empty, setPower) - if (setPower.~=(Kilowatts(0d))(Kilowatts(1e-6))) { + if (setPower.~=(zeroKW)(Kilowatts(1e-6))) { // No power left. Rest is not charging - return (Seq.empty, Kilowatts(0d)) + return (Seq.empty, zeroKW) } val proposedPower = setPower.divide(evs.size) val (exceedingPowerEvs, fittingPowerEvs) = evs.partition { ev => - if (setPower > Kilowatts(0d)) + if (setPower > zeroKW) proposedPower > getMaxAvailableChargingPower(ev) else proposedPower < (getMaxAvailableChargingPower(ev) * -1) @@ -735,7 +735,7 @@ final case class EvcsModel( ) } - (results, Kilowatts(0d)) + (results, zeroKW) } else { // not all evs can be charged with proposed power @@ -743,7 +743,7 @@ final case class EvcsModel( val maxCharged = exceedingPowerEvs.map { ev => val maxPower = getMaxAvailableChargingPower(ev) val power = - if (setPower > Kilowatts(0d)) + if (setPower > zeroKW) maxPower else maxPower * (-1) @@ -766,7 +766,7 @@ final case class EvcsModel( } // sum up allocated power - val chargingPowerSum = maxCharged.foldLeft(Kilowatts(0d)) { + val chargingPowerSum = maxCharged.foldLeft(zeroKW) { case (powerSum, (_, chargingPower, _)) => powerSum + chargingPower } @@ -804,7 +804,7 @@ final case class EvcsModel( power: Power, ): Long = { val timeUntilFullOrEmpty = - if (power > Kilowatts(0d)) { + if (power > zeroKW) { // if we're below lowest SOC, flex options will change at that point val targetEnergy = diff --git a/src/main/scala/edu/ie3/simona/model/thermal/CylindricalThermalStorage.scala b/src/main/scala/edu/ie3/simona/model/thermal/CylindricalThermalStorage.scala index b6828f7dc5..b6e5d9916d 100644 --- a/src/main/scala/edu/ie3/simona/model/thermal/CylindricalThermalStorage.scala +++ b/src/main/scala/edu/ie3/simona/model/thermal/CylindricalThermalStorage.scala @@ -18,13 +18,13 @@ import edu.ie3.simona.model.thermal.ThermalStorage.ThermalStorageThreshold.{ StorageFull, } import edu.ie3.util.quantities.PowerSystemUnits +import edu.ie3.util.scala.quantities.DefaultQuantities._ import edu.ie3.util.scala.quantities.SquantsUtils.RichEnergy import edu.ie3.util.scala.quantities.{ DefaultQuantities, KilowattHoursPerKelvinCubicMeters, SpecificHeatCapacity, } -import squants.energy.{Kilowatts, Megawatts} import squants.space.{CubicMeters, Volume} import squants.thermal.Celsius import squants.time.{Hours, Seconds} @@ -108,14 +108,14 @@ final case class CylindricalThermalStorage( /* Determine, when a threshold is reached */ val nextThreshold = - if (qDot > Megawatts(0d)) { + if (qDot > zeroMW) { val duration = (maxEnergyThreshold - updatedEnergy) / qDot val durationInTicks = Math.round(duration.toSeconds) if (durationInTicks <= 0L) None else Some(StorageFull(tick + durationInTicks)) - } else if (qDot < Megawatts(0d)) { + } else if (qDot < zeroMW) { val duration = (updatedEnergy - minEnergyThreshold) / qDot * (-1) val durationInTicks = Math.round(duration.toSeconds) if (durationInTicks <= 0L) @@ -132,7 +132,7 @@ final case class CylindricalThermalStorage( override def startingState: ThermalStorageState = ThermalStorageState( -1L, getMinEnergyThreshold, - Kilowatts(0d), + zeroKW, ) @deprecated("Use thermal storage state instead") @@ -143,7 +143,7 @@ final case class CylindricalThermalStorage( override def tryToStoreAndReturnRemainder( addedEnergy: Energy ): Option[Energy] = { - if (addedEnergy > zeroEnergy) { + if (addedEnergy > zeroKWH) { _storedEnergy = _storedEnergy + addedEnergy if (_storedEnergy > maxEnergyThreshold) { val surplus = _storedEnergy - maxEnergyThreshold @@ -158,7 +158,7 @@ final case class CylindricalThermalStorage( override def tryToTakeAndReturnLack( takenEnergy: Energy ): Option[Energy] = { - if (takenEnergy > zeroEnergy) { + if (takenEnergy > zeroKWH) { _storedEnergy = _storedEnergy - takenEnergy if (_storedEnergy < minEnergyThreshold) { val lack = minEnergyThreshold - _storedEnergy diff --git a/src/main/scala/edu/ie3/simona/model/thermal/RandomStorageState.scala b/src/main/scala/edu/ie3/simona/model/thermal/RandomStorageState.scala index c447a5bf73..6d497d5a5b 100644 --- a/src/main/scala/edu/ie3/simona/model/thermal/RandomStorageState.scala +++ b/src/main/scala/edu/ie3/simona/model/thermal/RandomStorageState.scala @@ -7,8 +7,8 @@ package edu.ie3.simona.model.thermal import edu.ie3.simona.model.thermal.ThermalStorage.ThermalStorageState +import edu.ie3.util.scala.quantities.DefaultQuantities._ import squants.Energy -import squants.energy.Kilowatts import scala.util.Random @@ -24,7 +24,7 @@ trait RandomStorageState { ThermalStorageState( -1L, storedEnergy, - Kilowatts(0d), + zeroKW, ) } } diff --git a/src/main/scala/edu/ie3/simona/model/thermal/ThermalGrid.scala b/src/main/scala/edu/ie3/simona/model/thermal/ThermalGrid.scala index 0586a20ee7..74c24dd65d 100644 --- a/src/main/scala/edu/ie3/simona/model/thermal/ThermalGrid.scala +++ b/src/main/scala/edu/ie3/simona/model/thermal/ThermalGrid.scala @@ -22,7 +22,8 @@ import edu.ie3.simona.model.thermal.ThermalHouse.ThermalHouseState import edu.ie3.simona.model.thermal.ThermalStorage.ThermalStorageState import edu.ie3.simona.util.TickUtil.TickLong import edu.ie3.util.quantities.QuantityUtils.RichQuantityDouble -import squants.energy.{Kilowatts, MegawattHours, Megawatts} +import edu.ie3.util.scala.quantities.DefaultQuantities._ +import squants.energy.Kilowatts import squants.{Energy, Power, Temperature} import java.time.ZonedDateTime @@ -82,7 +83,7 @@ final case class ThermalGrid( ) } .getOrElse( - (MegawattHours(0d), MegawattHours(0d)) + (zeroMWH, zeroMWH) ) } @@ -116,7 +117,7 @@ final case class ThermalGrid( state: ThermalGridState, ambientTemperature: Temperature, qDot: Power, - ): (ThermalGridState, Option[ThermalThreshold]) = if (qDot > Kilowatts(0d)) + ): (ThermalGridState, Option[ThermalThreshold]) = if (qDot > zeroKW) handleInfeed(tick, ambientTemperature, state, qDot) else handleConsumption(tick, ambientTemperature, state, qDot) @@ -150,7 +151,7 @@ final case class ThermalGrid( thermalStorage .updateState( tick, - Kilowatts(0d), + zeroKW, storageState, ) ._1 @@ -177,7 +178,7 @@ final case class ThermalGrid( tick, lastHouseState, ambientTemperature, - Kilowatts(0d), + zeroKW, ) storage.zip(updatedStorageState) match { case Some((thermalStorage, storageState)) => @@ -269,7 +270,7 @@ final case class ThermalGrid( tick, houseState, ambientTemperature, - Megawatts(0d), + zeroMW, ) } @@ -345,7 +346,7 @@ final case class ThermalGrid( (thermalStorage, (storageState, _)), ) ) - if qDot.~=(Kilowatts(0d))(Kilowatts(10e-3)) && + if qDot.~=(zeroKW)(Kilowatts(10e-3)) && thermalHouse.isInnerTemperatureTooLow( houseState.innerTemperature ) && !thermalStorage.isEmpty(storageState.storedEnergy) => @@ -480,7 +481,7 @@ object ThermalGrid { possible + rhs.possible, ) - def hasRequiredDemand: Boolean = required > MegawattHours(0d) + def hasRequiredDemand: Boolean = required > zeroMWH def hasAdditionalDemand: Boolean = possible > required } @@ -507,8 +508,8 @@ object ThermalGrid { } def noDemand: ThermalEnergyDemand = ThermalEnergyDemand( - MegawattHours(0d), - MegawattHours(0d), + zeroMWH, + zeroMWH, ) } } diff --git a/src/main/scala/edu/ie3/simona/model/thermal/ThermalHouse.scala b/src/main/scala/edu/ie3/simona/model/thermal/ThermalHouse.scala index b702102080..c08a6a3114 100644 --- a/src/main/scala/edu/ie3/simona/model/thermal/ThermalHouse.scala +++ b/src/main/scala/edu/ie3/simona/model/thermal/ThermalHouse.scala @@ -22,8 +22,9 @@ import edu.ie3.simona.model.thermal.ThermalHouse.{ temperatureTolerance, } import edu.ie3.util.quantities.PowerSystemUnits +import edu.ie3.util.scala.quantities.DefaultQuantities._ import edu.ie3.util.scala.quantities.{ThermalConductance, WattsPerKelvin} -import squants.energy.{KilowattHours, Kilowatts, MegawattHours, Megawatts} +import squants.energy.KilowattHours import squants.thermal.{Kelvin, ThermalCapacity} import squants.time.{Hours, Seconds} import squants.{Energy, Power, Temperature, Time} @@ -108,7 +109,7 @@ final case class ThermalHouse( val temperatureToTriggerRequiredEnergy = if ( currentInnerTemp <= state.innerTemperature && - state.qDot <= Kilowatts(0d) + state.qDot <= zeroKW ) { // temperature has been decreasing and heat source has been turned off // => we have reached target temp before and are now targeting lower temp @@ -122,7 +123,7 @@ final case class ThermalHouse( ) ) energy(targetTemperature, currentInnerTemp) else - MegawattHours(0d) + zeroMWH val possibleEnergy = if (!isInnerTemperatureTooHigh(currentInnerTemp)) { @@ -130,7 +131,7 @@ final case class ThermalHouse( // there is an amount of optional energy that could be stored energy(upperBoundaryTemperature, currentInnerTemp) } else - MegawattHours(0d) + zeroMWH ThermalEnergyDemand(requiredEnergy, possibleEnergy) } @@ -360,7 +361,7 @@ final case class ThermalHouse( ) / artificialDuration val resultingQDot = qDotExternal - loss if ( - resultingQDot < Megawatts(0d) && !isInnerTemperatureTooLow( + resultingQDot < zeroMW && !isInnerTemperatureTooLow( innerTemperature ) ) { @@ -372,7 +373,7 @@ final case class ThermalHouse( resultingQDot, ).map(HouseTemperatureLowerBoundaryReached) } else if ( - resultingQDot > Megawatts(0d) && !isInnerTemperatureTooHigh( + resultingQDot > zeroMW && !isInnerTemperatureTooHigh( innerTemperature ) ) { @@ -396,7 +397,7 @@ final case class ThermalHouse( qDot: Power, ): Option[Long] = { val flexibleEnergy = energy(higherTemperature, lowerTemperature) - if (flexibleEnergy < MegawattHours(0d)) + if (flexibleEnergy < zeroMWH) None else { val duration = Math.round( @@ -459,7 +460,7 @@ object ThermalHouse { ThermalHouseState( -1L, house.targetTemperature, - Megawatts(0d), + zeroMW, ) object ThermalHouseThreshold { diff --git a/src/main/scala/edu/ie3/simona/model/thermal/ThermalStorage.scala b/src/main/scala/edu/ie3/simona/model/thermal/ThermalStorage.scala index 03460fee5e..5886a4e7f8 100644 --- a/src/main/scala/edu/ie3/simona/model/thermal/ThermalStorage.scala +++ b/src/main/scala/edu/ie3/simona/model/thermal/ThermalStorage.scala @@ -11,7 +11,6 @@ import edu.ie3.datamodel.models.input.OperatorInput import edu.ie3.datamodel.models.input.thermal.ThermalBusInput import edu.ie3.simona.model.thermal.ThermalStorage.ThermalStorageState import squants.{Energy, Power, Seconds} -import squants.energy.KilowattHours import java.util.UUID @@ -44,7 +43,6 @@ abstract class ThermalStorage( maxEnergyThreshold: Energy, chargingPower: Power, ) { - protected val zeroEnergy: Energy = KilowattHours(0d) /** In order to avoid faulty flexibility options, we want to avoid offering * charging/discharging that could last less than one second. diff --git a/src/main/scala/edu/ie3/simona/ontology/messages/flex/MinMaxFlexibilityMessage.scala b/src/main/scala/edu/ie3/simona/ontology/messages/flex/MinMaxFlexibilityMessage.scala index 442d9ed785..af72bf52c6 100644 --- a/src/main/scala/edu/ie3/simona/ontology/messages/flex/MinMaxFlexibilityMessage.scala +++ b/src/main/scala/edu/ie3/simona/ontology/messages/flex/MinMaxFlexibilityMessage.scala @@ -8,6 +8,7 @@ package edu.ie3.simona.ontology.messages.flex import edu.ie3.simona.exceptions.CriticalFailureException import edu.ie3.simona.ontology.messages.flex.FlexibilityMessage.ProvideFlexOptions +import edu.ie3.util.scala.quantities.DefaultQuantities._ import squants.Power import java.util.UUID @@ -43,6 +44,23 @@ object MinMaxFlexibilityMessage { object ProvideMinMaxFlexOptions { + implicit class RichIterable( + private val flexOptions: Iterable[ProvideMinMaxFlexOptions] + ) extends AnyVal { + def flexSum: (Power, Power, Power) = + flexOptions.foldLeft((zeroKW, zeroKW, zeroKW)) { + case ( + (sumRef, sumMin, sumMax), + ProvideMinMaxFlexOptions(_, addRef, addMin, addMax), + ) => + ( + sumRef + addRef, + sumMin + addMin, + sumMax + addMax, + ) + } + } + /** Creates a [[ProvideMinMaxFlexOptions]] message with sanity checks * regarding the power values * diff --git a/src/main/scala/edu/ie3/util/scala/quantities/DefaultQuantities.scala b/src/main/scala/edu/ie3/util/scala/quantities/DefaultQuantities.scala index 8ac476e888..ac401db84c 100644 --- a/src/main/scala/edu/ie3/util/scala/quantities/DefaultQuantities.scala +++ b/src/main/scala/edu/ie3/util/scala/quantities/DefaultQuantities.scala @@ -6,15 +6,20 @@ package edu.ie3.util.scala.quantities -import squants.{Each, Energy, Power, Dimensionless} -import squants.energy.{KilowattHours, Kilowatts, Megawatts} +import squants.energy.{KilowattHours, Kilowatts, MegawattHours, Megawatts} +import squants.{Dimensionless, Each, Energy, Power} object DefaultQuantities { - val zeroKWH: Energy = KilowattHours(0d) val zeroKW: Power = Kilowatts(0d) val zeroMW: Power = Megawatts(0d) + + val zeroKVAr: ReactivePower = Kilovars(0d) val zeroMVAr: ReactivePower = Megavars(0d) + + val zeroKWH: Energy = KilowattHours(0d) + val zeroMWH: Energy = MegawattHours(0d) + val zeroPU: Dimensionless = Each(0d) } From 5d4e673f5cb7f4b4fb44c92793df02c03e7cf53f Mon Sep 17 00:00:00 2001 From: Sebastian Peter Date: Tue, 19 Mar 2024 12:10:43 +0100 Subject: [PATCH 26/32] Curtailing regeneratives in general instead of just PV --- .../resources/config/config-template.conf | 2 +- .../edu/ie3/simona/config/SimonaConfig.scala | 7 ++++-- .../ie3/simona/model/em/EmModelShell.scala | 3 ++- .../model/em/PrioritizedFlexStrat.scala | 23 ++++++++++++------- .../model/em/PrioritizedFlexStratSpec.scala | 6 ++--- 5 files changed, 26 insertions(+), 15 deletions(-) diff --git a/src/main/resources/config/config-template.conf b/src/main/resources/config/config-template.conf index b46d088f9f..d9006556ed 100644 --- a/src/main/resources/config/config-template.conf +++ b/src/main/resources/config/config-template.conf @@ -62,7 +62,7 @@ EmRuntimeConfig { # # # # # baseRuntimeConfig: BaseRuntimeConfig # this entry is ignored by the config generator, # but cannot removed bc otherwise EmRuntimeConfig is handled as String - pvFlex: Boolean | false + curtailRegenerative: Boolean | false aggregateFlex: String | "SELF_OPT_EXCL_PV" } diff --git a/src/main/scala/edu/ie3/simona/config/SimonaConfig.scala b/src/main/scala/edu/ie3/simona/config/SimonaConfig.scala index 9e3bc48802..ad870e54f8 100644 --- a/src/main/scala/edu/ie3/simona/config/SimonaConfig.scala +++ b/src/main/scala/edu/ie3/simona/config/SimonaConfig.scala @@ -84,7 +84,7 @@ object SimonaConfig { override val scaling: scala.Double, override val uuids: scala.List[java.lang.String], aggregateFlex: java.lang.String, - pvFlex: scala.Boolean, + curtailRegenerative: scala.Boolean, ) extends BaseRuntimeConfig( calculateMissingReactivePowerWithModel, scaling, @@ -100,7 +100,10 @@ object SimonaConfig { aggregateFlex = if (c.hasPathOrNull("aggregateFlex")) c.getString("aggregateFlex") else "SELF_OPT_EXCL_PV", - pvFlex = c.hasPathOrNull("pvFlex") && c.getBoolean("pvFlex"), + curtailRegenerative = + c.hasPathOrNull("curtailRegenerative") && c.getBoolean( + "curtailRegenerative" + ), calculateMissingReactivePowerWithModel = $_reqBln( parentPath, c, diff --git a/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala b/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala index 79efa1d913..54baa95367 100644 --- a/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala +++ b/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala @@ -119,7 +119,8 @@ object EmModelShell { val modelStrategy = modelStrategyName match { case "PROPORTIONAL" => ProportionalFlexStrat - case "PRIORITIZED" => PrioritizedFlexStrat(modelConfig.pvFlex) + case "PRIORITIZED" => + PrioritizedFlexStrat(modelConfig.curtailRegenerative) case unknown => throw new CriticalFailureException(s"Unknown model strategy $unknown") } diff --git a/src/main/scala/edu/ie3/simona/model/em/PrioritizedFlexStrat.scala b/src/main/scala/edu/ie3/simona/model/em/PrioritizedFlexStrat.scala index 3fc7ced588..98162db86c 100644 --- a/src/main/scala/edu/ie3/simona/model/em/PrioritizedFlexStrat.scala +++ b/src/main/scala/edu/ie3/simona/model/em/PrioritizedFlexStrat.scala @@ -12,6 +12,7 @@ import edu.ie3.datamodel.models.input.system.{ HpInput, PvInput, StorageInput, + WecInput, } import edu.ie3.simona.exceptions.CriticalFailureException import edu.ie3.simona.model.em.EmModelStrat.tolerance @@ -24,17 +25,18 @@ import java.util.UUID /** Determines flex control for connected agents by adhering to a priority * hierarchy, with some devices not controlled at all. * - * @param pvFlex - * Whether PV feed-in can be curtailed or not + * @param curtailRegenerative + * Whether PV and WEC feed-in can be curtailed or not */ -final case class PrioritizedFlexStrat(pvFlex: Boolean) extends EmModelStrat { +final case class PrioritizedFlexStrat(curtailRegenerative: Boolean) + extends EmModelStrat { - /** Only heat pumps, battery storages, charging stations and PVs (if enabled) - * are controlled by this strategy + /** Only heat pumps, battery storages, charging stations and PVs/WECs (if + * enabled) are controlled by this strategy */ private val controllableAssets: Seq[Class[_ <: AssetInput]] = Seq(classOf[HpInput], classOf[StorageInput], classOf[EvcsInput]) ++ Option - .when(pvFlex)(Seq(classOf[PvInput])) + .when(curtailRegenerative)(Seq(classOf[PvInput], classOf[WecInput])) .getOrElse(Seq.empty) /** Determine the power of controllable devices by using flexibility according @@ -86,7 +88,12 @@ final case class PrioritizedFlexStrat(pvFlex: Boolean) extends EmModelStrat { .collectFirst { case flex @ (_: PvInput, _) => flex } - .filter(_ => pvFlex) // only if enabled + .filter(_ => curtailRegenerative) // only if enabled + val wecOpt = flexOptions + .collectFirst { case flex @ (_: WecInput, _) => + flex + } + .filter(_ => curtailRegenerative) // only if enabled if (zeroKW.~=(targetDelta)(tolerance)) { Seq.empty @@ -94,7 +101,7 @@ final case class PrioritizedFlexStrat(pvFlex: Boolean) extends EmModelStrat { // suggested power too low, try to store difference/increase load val orderedParticipants = - Seq(evcsOpt, storageOpt, heatPumpOpt, pvOpt).flatten + Seq(evcsOpt, storageOpt, heatPumpOpt, pvOpt, wecOpt).flatten orderedParticipants.foldLeft( (Seq.empty[(UUID, Power)], Option(targetDelta)) diff --git a/src/test/scala/edu/ie3/simona/model/em/PrioritizedFlexStratSpec.scala b/src/test/scala/edu/ie3/simona/model/em/PrioritizedFlexStratSpec.scala index dc3bcd7ff5..f6928fc75e 100644 --- a/src/test/scala/edu/ie3/simona/model/em/PrioritizedFlexStratSpec.scala +++ b/src/test/scala/edu/ie3/simona/model/em/PrioritizedFlexStratSpec.scala @@ -47,7 +47,7 @@ class PrioritizedFlexStratSpec when(storageInputModel.getUuid).thenReturn(st) "determine flex control dependent on flex options" in { - val strat = PrioritizedFlexStrat(pvFlex = false) + val strat = PrioritizedFlexStrat(curtailRegenerative = false) val cases = Table( ( @@ -183,8 +183,8 @@ class PrioritizedFlexStratSpec } } - "determine flex control dependent on flex options with PV flex enabled" in { - val strat = PrioritizedFlexStrat(pvFlex = true) + "determine flex control dependent on flex options with curtailment enabled" in { + val strat = PrioritizedFlexStrat(curtailRegenerative = true) val cases = Table( ( From e18b4603f523a2dbf7e4c8d165234c4025ce99d5 Mon Sep 17 00:00:00 2001 From: Sebastian Peter Date: Tue, 19 Mar 2024 12:19:09 +0100 Subject: [PATCH 27/32] Also adapting EmAggregateSelfOpt --- .../resources/config/config-template.conf | 2 +- .../edu/ie3/simona/config/SimonaConfig.scala | 2 +- .../simona/model/em/EmAggregateSelfOpt.scala | 22 +++++----- .../ie3/simona/model/em/EmModelShell.scala | 6 +-- .../edu/ie3/simona/agent/em/EmAgentIT.scala | 2 +- .../edu/ie3/simona/agent/em/EmAgentSpec.scala | 4 +- .../model/em/EmAggregateSelfOptSpec.scala | 4 +- .../model/em/PrioritizedFlexStratSpec.scala | 43 ++++++++++--------- 8 files changed, 44 insertions(+), 41 deletions(-) diff --git a/src/main/resources/config/config-template.conf b/src/main/resources/config/config-template.conf index d9006556ed..cb446ae680 100644 --- a/src/main/resources/config/config-template.conf +++ b/src/main/resources/config/config-template.conf @@ -63,7 +63,7 @@ EmRuntimeConfig { baseRuntimeConfig: BaseRuntimeConfig # this entry is ignored by the config generator, # but cannot removed bc otherwise EmRuntimeConfig is handled as String curtailRegenerative: Boolean | false - aggregateFlex: String | "SELF_OPT_EXCL_PV" + aggregateFlex: String | "SELF_OPT_EXCL_REG" } #@define extends BaseRuntimeConfig diff --git a/src/main/scala/edu/ie3/simona/config/SimonaConfig.scala b/src/main/scala/edu/ie3/simona/config/SimonaConfig.scala index ad870e54f8..c64c62ca3f 100644 --- a/src/main/scala/edu/ie3/simona/config/SimonaConfig.scala +++ b/src/main/scala/edu/ie3/simona/config/SimonaConfig.scala @@ -99,7 +99,7 @@ object SimonaConfig { SimonaConfig.EmRuntimeConfig( aggregateFlex = if (c.hasPathOrNull("aggregateFlex")) c.getString("aggregateFlex") - else "SELF_OPT_EXCL_PV", + else "SELF_OPT_EXCL_REG", curtailRegenerative = c.hasPathOrNull("curtailRegenerative") && c.getBoolean( "curtailRegenerative" diff --git a/src/main/scala/edu/ie3/simona/model/em/EmAggregateSelfOpt.scala b/src/main/scala/edu/ie3/simona/model/em/EmAggregateSelfOpt.scala index c3dbd27ebe..4451d4bb00 100644 --- a/src/main/scala/edu/ie3/simona/model/em/EmAggregateSelfOpt.scala +++ b/src/main/scala/edu/ie3/simona/model/em/EmAggregateSelfOpt.scala @@ -7,18 +7,20 @@ package edu.ie3.simona.model.em import edu.ie3.datamodel.models.input.AssetInput -import edu.ie3.datamodel.models.input.system.PvInput +import edu.ie3.datamodel.models.input.system.{PvInput, WecInput} import edu.ie3.simona.ontology.messages.flex.MinMaxFlexibilityMessage.ProvideMinMaxFlexOptions import edu.ie3.util.scala.quantities.DefaultQuantities._ import squants.Power /** Aggregates flex reference power with the target of reaching 0kW, while - * optionally excluding positive PV potential from the calculation + * optionally excluding positive flex potential of PV/WEC from the calculation * - * @param pvFlex - * Whether to include positive PV flexibility in reference sum calculation + * @param curtailRegenerative + * Whether to include positive flexibility of PV/WEC in reference sum + * calculation */ -final case class EmAggregateSelfOpt(pvFlex: Boolean) extends EmAggregateFlex { +final case class EmAggregateSelfOpt(curtailRegenerative: Boolean) + extends EmAggregateFlex { override def aggregateFlexOptions( flexOptions: Iterable[ @@ -38,18 +40,18 @@ final case class EmAggregateSelfOpt(pvFlex: Boolean) extends EmAggregateFlex { } val maxRefSum = - if (pvFlex) + if (curtailRegenerative) maxSum else flexOptions.foldLeft(zeroKW) { case ( - maxSumExclPv, + maxSumExclReg, (inputModel, ProvideMinMaxFlexOptions(_, _, addMin, addMax)), ) => inputModel match { - case _: PvInput => - maxSumExclPv + addMin - case _ => maxSumExclPv + addMax + case _: PvInput | _: WecInput => + maxSumExclReg + addMin + case _ => maxSumExclReg + addMax } } diff --git a/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala b/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala index 54baa95367..6a15d08624 100644 --- a/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala +++ b/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala @@ -126,9 +126,9 @@ object EmModelShell { } val aggregateFlex = modelConfig.aggregateFlex match { - case "SELF_OPT_EXCL_PV" => EmAggregateSelfOpt(false) - case "SELF_OPT" => EmAggregateSelfOpt(true) - case "SIMPLE_SUM" => EmAggregateSimpleSum + case "SELF_OPT_EXCL_REG" => EmAggregateSelfOpt(false) + case "SELF_OPT" => EmAggregateSelfOpt(true) + case "SIMPLE_SUM" => EmAggregateSimpleSum case unknown => throw new CriticalFailureException( s"Unknown aggregate flex strategy $unknown" 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 04632486ee..a7243d5120 100644 --- a/src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala +++ b/src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala @@ -88,7 +88,7 @@ class EmAgentIT scaling = 1d, uuids = List("default"), aggregateFlex = "SELF_OPT", - pvFlex = false, + curtailRegenerative = false, ) private implicit val quantityTolerance: Double = 1e-10d 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 7625e2c261..fb06c37e8d 100644 --- a/src/test/scala/edu/ie3/simona/agent/em/EmAgentSpec.scala +++ b/src/test/scala/edu/ie3/simona/agent/em/EmAgentSpec.scala @@ -62,8 +62,8 @@ class EmAgentSpec calculateMissingReactivePowerWithModel = false, scaling = 1d, uuids = List("default"), - aggregateFlex = "SELF_OPT_EXCL_PV", - pvFlex = false, + aggregateFlex = "SELF_OPT_EXCL_REG", + curtailRegenerative = false, ) private implicit val activePowerTolerance: Power = Kilowatts(1e-10) diff --git a/src/test/scala/edu/ie3/simona/model/em/EmAggregateSelfOptSpec.scala b/src/test/scala/edu/ie3/simona/model/em/EmAggregateSelfOptSpec.scala index 8ba0d1804e..9ae0ee63c1 100644 --- a/src/test/scala/edu/ie3/simona/model/em/EmAggregateSelfOptSpec.scala +++ b/src/test/scala/edu/ie3/simona/model/em/EmAggregateSelfOptSpec.scala @@ -17,7 +17,7 @@ import java.util.UUID class EmAggregateSelfOptSpec extends UnitSpec with MockitoSugar { "The self-optimizing aggregating strategy with PV flex" should { - val strat = EmAggregateSelfOpt(pvFlex = true) + val strat = EmAggregateSelfOpt(curtailRegenerative = true) "pick 0kW if possible" in { val flexOptions1 = ProvideMinMaxFlexOptions( @@ -108,7 +108,7 @@ class EmAggregateSelfOptSpec extends UnitSpec with MockitoSugar { } "The self-optimizing aggregating strategy without PV flex" should { - val strat = EmAggregateSelfOpt(pvFlex = false) + val strat = EmAggregateSelfOpt(curtailRegenerative = false) "exclude PV max power when normally picking 0kW as target" in { val flexOptions1 = ProvideMinMaxFlexOptions( diff --git a/src/test/scala/edu/ie3/simona/model/em/PrioritizedFlexStratSpec.scala b/src/test/scala/edu/ie3/simona/model/em/PrioritizedFlexStratSpec.scala index f6928fc75e..cbaee93277 100644 --- a/src/test/scala/edu/ie3/simona/model/em/PrioritizedFlexStratSpec.scala +++ b/src/test/scala/edu/ie3/simona/model/em/PrioritizedFlexStratSpec.scala @@ -301,7 +301,7 @@ class PrioritizedFlexStratSpec // flex options should be changed if corresponding // agent is not controlled by this strategy val cases = Table( - ("pvFlex", "inputModel", "expectedAdaptation"), + ("curtailRegenerative", "inputModel", "expectedAdaptation"), (false, loadInputModel, true), (false, pvInputModel, true), (false, evcsInputModel, false), @@ -312,26 +312,27 @@ class PrioritizedFlexStratSpec (true, storageInputModel, false), ) - forAll(cases) { case (pvFlex, inputModel, expectedAdaptation) => - val flexOptionsIn = ProvideMinMaxFlexOptions( - inputModel.getUuid, - Kilowatts(1), - Kilowatts(-1), - Kilowatts(2), - ) - - val flexOptionsOut = PrioritizedFlexStrat(pvFlex) - .adaptFlexOptions(inputModel, flexOptionsIn) - - if (expectedAdaptation) { - flexOptionsOut shouldBe ProvideMinMaxFlexOptions - .noFlexOption( - inputModel.getUuid, - Kilowatts(1), - ) - } else { - flexOptionsOut shouldBe flexOptionsIn - } + forAll(cases) { + case (curtailRegenerative, inputModel, expectedAdaptation) => + val flexOptionsIn = ProvideMinMaxFlexOptions( + inputModel.getUuid, + Kilowatts(1), + Kilowatts(-1), + Kilowatts(2), + ) + + val flexOptionsOut = PrioritizedFlexStrat(curtailRegenerative) + .adaptFlexOptions(inputModel, flexOptionsIn) + + if (expectedAdaptation) { + flexOptionsOut shouldBe ProvideMinMaxFlexOptions + .noFlexOption( + inputModel.getUuid, + Kilowatts(1), + ) + } else { + flexOptionsOut shouldBe flexOptionsIn + } } } From 1d02a56e28de9260d79e5e21082d147769448795 Mon Sep 17 00:00:00 2001 From: Sebastian Peter Date: Tue, 19 Mar 2024 13:01:33 +0100 Subject: [PATCH 28/32] More addressing of reviewer's comments --- .../agent/grid/GridAgentController.scala | 25 ++++++++++++------- .../model/em/ProportionalFlexStrat.scala | 6 ++--- 2 files changed, 19 insertions(+), 12 deletions(-) diff --git a/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala b/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala index e707680218..7ba4abef77 100644 --- a/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala +++ b/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala @@ -28,6 +28,7 @@ import edu.ie3.simona.config.SimonaConfig import edu.ie3.simona.config.SimonaConfig._ import edu.ie3.simona.event.ResultEvent import edu.ie3.simona.event.notifier.NotifierConfig +import edu.ie3.simona.exceptions.CriticalFailureException import edu.ie3.simona.exceptions.agent.GridAgentInitializationException import edu.ie3.simona.ontology.messages.SchedulerMessage.ScheduleActivation import edu.ie3.simona.ontology.messages.flex.FlexibilityMessage.FlexResponse @@ -229,7 +230,7 @@ class GridAgentController( * @param emInputs * EMs of the current level, which can be controlled by further EMs at * higher levels - * @param lastLevelEms + * @param previousLevelEms * EMs that have been built by the previous recursion level * @return * Map from model UUID to EmAgent ActorRef @@ -238,7 +239,7 @@ class GridAgentController( participantConfigUtil: ConfigUtil.ParticipantConfigUtil, outputConfigUtil: OutputConfigUtil, emInputs: Map[UUID, EmInput], - lastLevelEms: Map[UUID, ActorRef[FlexResponse]] = Map.empty, + previousLevelEms: Map[UUID, ActorRef[FlexResponse]] = Map.empty, ): Map[UUID, ActorRef[FlexResponse]] = { // For the current level, split controlled and uncontrolled EMs. // Uncontrolled EMs can be built right away. @@ -257,7 +258,8 @@ class GridAgentController( } } - val lastLevelAndUncontrolledEms = lastLevelEms ++ uncontrolledEms.toMap + val previousLevelAndUncontrolledEms = + previousLevelEms ++ uncontrolledEms.toMap if (controlledEmInputs.nonEmpty) { // For controlled EMs at the current level, more EMs @@ -267,20 +269,25 @@ class GridAgentController( emInput.getControllingEm.toScala.map(uuid -> _) } - // Return value includes last level and uncontrolled EMs of this level + // Return value includes previous level and uncontrolled EMs of this level val recursiveEms = buildEmsRecursively( participantConfigUtil, outputConfigUtil, controllingEms, - lastLevelAndUncontrolledEms, + previousLevelAndUncontrolledEms, ) val controlledEms = controlledEmInputs.map { case (uuid, emInput) => val controllingEm = emInput.getControllingEm.toScala .map(_.getUuid) - // We do not have to throw errors here because PSDM - // already takes care of valid input data - .flatMap(uuid => recursiveEms.get(uuid)) + .map(uuid => + recursiveEms.getOrElse( + uuid, + throw new CriticalFailureException( + s"Actor for EM $uuid not found." + ), + ) + ) uuid -> buildEm( emInput, @@ -292,7 +299,7 @@ class GridAgentController( recursiveEms ++ controlledEms } else { - lastLevelAndUncontrolledEms + previousLevelAndUncontrolledEms } } diff --git a/src/main/scala/edu/ie3/simona/model/em/ProportionalFlexStrat.scala b/src/main/scala/edu/ie3/simona/model/em/ProportionalFlexStrat.scala index 4be1699cef..97647ebaa3 100644 --- a/src/main/scala/edu/ie3/simona/model/em/ProportionalFlexStrat.scala +++ b/src/main/scala/edu/ie3/simona/model/em/ProportionalFlexStrat.scala @@ -100,10 +100,10 @@ object ProportionalFlexStrat extends EmModelStrat { } } else { // calculate share of flexibility that each device should carry - val normalizedLimit = totalLimit - totalRef - val normalizedTarget = target - totalRef + val deltaToLimit = totalLimit - totalRef + val deltaToTarget = target - totalRef - val flexShare = normalizedTarget / normalizedLimit + val flexShare = deltaToTarget / deltaToLimit filteredOptions.map { case (uuid, refPower, limitPower) => val diffLimitRef = limitPower - refPower From 380a362254112fa1723e34d31227c937e4f92dd6 Mon Sep 17 00:00:00 2001 From: Sebastian Peter Date: Tue, 19 Mar 2024 13:02:09 +0100 Subject: [PATCH 29/32] Update src/test/scala/edu/ie3/simona/model/em/PrioritizedFlexStratSpec.scala Fixing test case Co-authored-by: Daniel Feismann <98817556+danielfeismann@users.noreply.github.com> --- .../edu/ie3/simona/model/em/PrioritizedFlexStratSpec.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/test/scala/edu/ie3/simona/model/em/PrioritizedFlexStratSpec.scala b/src/test/scala/edu/ie3/simona/model/em/PrioritizedFlexStratSpec.scala index cbaee93277..b5ebb9c50b 100644 --- a/src/test/scala/edu/ie3/simona/model/em/PrioritizedFlexStratSpec.scala +++ b/src/test/scala/edu/ie3/simona/model/em/PrioritizedFlexStratSpec.scala @@ -70,7 +70,7 @@ class PrioritizedFlexStratSpec // excess is fully covered by max ev and parts of storage flex (0d, 1d, -13d, 2d, -11d, 11d, -2d, 2d, L((ev, 11d), (st, 1d))), // excess is fully covered by max ev and max storage flex - (0d, 0d, -14d, 2d, -11d, 11d, -2d, 2d, L((ev, 11d), (st, 2d))), + (0d, 1d, -14d, 2d, -11d, 11d, -2d, 2d, L((ev, 11d), (st, 2d))), // excess is fully covered by max storage flex (0d, 0d, -4d, 2d, 0d, 2d, -2d, 2d, L((st, 2d))), // excess is partly covered by max ev and max storage flex, -2kW remains From f6a30ef4f545dc15bc36ed6c3bcbfac8dacc3b14 Mon Sep 17 00:00:00 2001 From: Sebastian Peter Date: Tue, 19 Mar 2024 14:16:50 +0100 Subject: [PATCH 30/32] Exemplary explanation of test cases in ProportionalFlexStratSpec --- .../model/em/ProportionalFlexStratSpec.scala | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/src/test/scala/edu/ie3/simona/model/em/ProportionalFlexStratSpec.scala b/src/test/scala/edu/ie3/simona/model/em/ProportionalFlexStratSpec.scala index cec7c61d2a..4a4834fff9 100644 --- a/src/test/scala/edu/ie3/simona/model/em/ProportionalFlexStratSpec.scala +++ b/src/test/scala/edu/ie3/simona/model/em/ProportionalFlexStratSpec.scala @@ -31,6 +31,22 @@ class ProportionalFlexStratSpec val assetInput = mock[AssetInput] // is not used + /* As an example, this is how the test cases should be interpreted, + * using the second test case of target being higher than reference sum: + * + * We have a target of 3 kW, but a reference power of 1 kW + 0 kW = 1 kW. + * Thus, we want to increase load by 2 kW. + * + * In total, we have a maximum power of 2 kW + 4 kW = 6 kW. This is 5 kW + * more than our reference power of 1 kW. Since we need only 2 kW of the + * 5 kW potential, we use 40% of the available positive flexibility. + * + * Specifically, we use 40% of the flex potential 2 kW - 1 kW = 1 kW of + * the first unit and 40% of the flex potential 4 kW - 0 kW = 4 kW of the + * second unit. Thus we arrive at 1 kW + 40% * 1 kW = 1.4 kW for the + * first unit and 0 kW + 40% * 4 kW = 1.6 kW for the second unit. + */ + val cases = Table( ( "target", From 8b45e2bca0429095b913e5656d187c6b998c8c6b Mon Sep 17 00:00:00 2001 From: Sebastian Peter Date: Tue, 19 Mar 2024 14:40:32 +0100 Subject: [PATCH 31/32] Correcting commentary --- src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 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 a7243d5120..3654585e80 100644 --- a/src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala +++ b/src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala @@ -310,7 +310,7 @@ class EmAgentIT /* TICK 7200 LOAD: 0.000269 MW (unchanged) PV: -0.003797 MW - Heat pump: running (turned on from last request), can be turned on and stay off + Heat pump: running (turned on from last request), can also be turned off -> set point ~3.5 kW (bigger than 50 % rated apparent power): stays turned on with unchanged state -> remaining 0 MW */ @@ -344,7 +344,7 @@ class EmAgentIT /* TICK 14400 LOAD: 0.000269 MW (unchanged) PV: -0.000066 MW - Heat pump: Is not running, can still run or be turned off + Heat pump: Is still running, can still be turned off -> flex signal is 0 MW: Heat pump is turned off */ @@ -411,7 +411,7 @@ class EmAgentIT /* TICK 28666 LOAD: 0.000269 MW (unchanged) PV: -0.000032 MW (unchanged) - Heat pump: Is turned off and has to be turned on + Heat pump: Is turned on again and cannot be turned off -> flex signal is no control -> 0.00485 MW */ From c80df93361df891364cef6b1aff07464c0d74527 Mon Sep 17 00:00:00 2001 From: Sebastian Peter Date: Tue, 19 Mar 2024 16:07:49 +0100 Subject: [PATCH 32/32] Improving commentary --- src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 3654585e80..cc0ce1f43b 100644 --- a/src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala +++ b/src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala @@ -378,7 +378,7 @@ class EmAgentIT /* TICK 21600 LOAD: 0.000269 MW (unchanged) PV: -0.000032 MW - Heat pump: Is not running, can run or be turned off + Heat pump: Is not running, can run or stay off -> flex signal is 0 MW: Heat pump is turned off */