diff --git a/.gitignore b/.gitignore index 7b25571331..7cf28aed43 100644 --- a/.gitignore +++ b/.gitignore @@ -232,3 +232,6 @@ deploy.properties # Ignore Gradle build output directory build + +# Ignore unwanted input directories +input/ext_sim diff --git a/CHANGELOG.md b/CHANGELOG.md index c83520da4e..33813e38c3 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -23,6 +23,7 @@ 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) +- 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/input/samples/vn_simona/vn_simona.conf b/input/samples/vn_simona/vn_simona.conf index 7783f80bf4..38626c593d 100644 --- a/input/samples/vn_simona/vn_simona.conf +++ b/input/samples/vn_simona/vn_simona.conf @@ -158,6 +158,18 @@ simona.runtime.participant.hp = { individualConfigs = [] } +# # # # # +# ATTENTION: calculateMissingReactivePowerWithModel and scaling is ignored here. +# # # # # +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 9da0b136f0..cb446ae680 100644 --- a/src/main/resources/config/config-template.conf +++ b/src/main/resources/config/config-template.conf @@ -54,6 +54,18 @@ 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 { + # # # # # + # 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 + curtailRegenerative: Boolean | false + aggregateFlex: String | "SELF_OPT_EXCL_REG" +} + #@define extends BaseRuntimeConfig HpRuntimeConfig { baseRuntimeConfig: BaseRuntimeConfig # this entry is ignored by the config generator, @@ -312,6 +324,10 @@ 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] + } } ################################################################## 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..06a7bf598a --- /dev/null +++ b/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala @@ -0,0 +1,465 @@ +/* + * © 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.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.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} +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.DefaultQuantities._ +import org.apache.pekko.actor.typed.scaladsl.Behaviors +import org.apache.pekko.actor.typed.{ActorRef, Behavior} + +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 Request + + /** Extended by all requests that activate an [[EmAgent]], i.e. activations, + * flex requests and control messages + */ + private sealed trait ActivationRequest extends Request { + 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 ActivationRequest + + /** 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 ActivationRequest { + 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 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]], + listener: Iterable[ActorRef[ResultEvent]], + ): Behavior[Request] = Behaviors.setup[Request] { 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) + } + }, + 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[Request] = 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: ActivationRequest) => + 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[Request] = 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(_) => + // We're not em-controlled ourselves, + // always desire to come as close as possible to 0 kW + val setPower = zeroKW + + 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[Request] = 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[Request] = 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( + zeroMW, + zeroMVAr, + ) + ) + + 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 listener + * A collection of result event listeners + */ + private final case class EmData( + outputConfig: NotifierConfig, + simulationStartDate: ZonedDateTime, + parentData: Either[SchedulerData, FlexControlledData], + 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/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/GridAgentController.scala b/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala index a8904c9ef9..7ba4abef77 100644 --- a/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala +++ b/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala @@ -6,11 +6,12 @@ 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._ import edu.ie3.simona.actor.SimonaActorNaming._ import edu.ie3.simona.agent.EnvironmentRefs +import edu.ie3.simona.agent.em.EmAgent import edu.ie3.simona.agent.participant.ParticipantAgent.ParticipantMessage import edu.ie3.simona.agent.participant.data.secondary.SecondaryDataService.{ ActorEvMovementsService, @@ -27,24 +28,23 @@ 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 import edu.ie3.simona.util.ConfigUtil import edu.ie3.simona.util.ConfigUtil._ import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK import org.apache.pekko.actor.typed.ActorRef import org.apache.pekko.actor.typed.scaladsl.ActorContext -import org.apache.pekko.actor.typed.scaladsl.adapter.{ - ClassicActorRefOps, - TypedActorContextOps, - TypedActorRefOps, -} +import org.apache.pekko.actor.typed.scaladsl.adapter._ import org.apache.pekko.actor.{ActorRef => ClassicRef} 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]] * @@ -79,7 +79,7 @@ class GridAgentController( resolution: Long, listener: Iterable[ActorRef[ResultEvent]], log: Logger, -) extends LazyLogging { +) { def buildSystemParticipants( subGridContainer: SubGridContainer, thermalIslandGridsByBusId: Map[UUID, ThermalGrid], @@ -101,15 +101,16 @@ class GridAgentController( /** 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. + * 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, @@ -185,27 +186,123 @@ class GridAgentController( ConfigUtil.ParticipantConfigUtil(participantsConfig) val outputConfigUtil = ConfigUtil.OutputConfigUtil(outputConfig) + // ems that control at least one participant directly + val firstLevelEms = participants.flatMap { + _.getControllingEm.toScala.map(em => em.getUuid -> em) + }.toMap + + val allEms = buildEmsRecursively( + participantConfigUtil, + outputConfigUtil, + firstLevelEms, + ) + participants - .map(participant => { + .map { participant => val node = participant.getNode - // build - val actorRef = - buildParticipantActor( - participantsConfig.requestVoltageDeviationThreshold, - participantConfigUtil, - outputConfigUtil, - participant, - thermalIslandGridsByBusId, - environmentRefs, - ) + + val actorRef = buildParticipantActor( + participantsConfig.requestVoltageDeviationThreshold, + participantConfigUtil, + outputConfigUtil, + participant, + thermalIslandGridsByBusId, + environmentRefs, + allEms.get(participant.getUuid), + ) introduceAgentToEnvironment(actorRef) // return uuid to actorRef node.getUuid -> actorRef - }) + } .toSet[(UUID, ActorRef[ParticipantMessage])] .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 previousLevelEms + * 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], + 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. + 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, + ) + Right(uuid -> actor) + } + } + + val previousLevelAndUncontrolledEms = + previousLevelEms ++ uncontrolledEms.toMap + + if (controlledEmInputs.nonEmpty) { + // 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 previous level and uncontrolled EMs of this level + val recursiveEms = buildEmsRecursively( + participantConfigUtil, + outputConfigUtil, + controllingEms, + previousLevelAndUncontrolledEms, + ) + + val controlledEms = controlledEmInputs.map { case (uuid, emInput) => + val controllingEm = emInput.getControllingEm.toScala + .map(_.getUuid) + .map(uuid => + recursiveEms.getOrElse( + uuid, + throw new CriticalFailureException( + s"Actor for EM $uuid not found." + ), + ) + ) + + uuid -> buildEm( + emInput, + participantConfigUtil.getOrDefault[EmRuntimeConfig](uuid), + outputConfigUtil.getOrDefault(NotifierIdentifier.Em), + maybeControllingEm = controllingEm, + ) + }.toMap + + recursiveEms ++ controlledEms + } else { + previousLevelAndUncontrolledEms + } + } + private def buildParticipantActor( requestVoltageDeviationThreshold: Double, participantConfigUtil: ConfigUtil.ParticipantConfigUtil, @@ -213,6 +310,7 @@ class GridAgentController( participantInputModel: SystemParticipantInput, thermalIslandGridsByBusId: Map[UUID, ThermalGrid], environmentRefs: EnvironmentRefs, + maybeControllingEm: Option[ActorRef[FlexResponse]], ): ActorRef[ParticipantMessage] = participantInputModel match { case input: FixedFeedInInput => buildFixedFeedIn( @@ -226,6 +324,7 @@ class GridAgentController( resolution, requestVoltageDeviationThreshold, outputConfigUtil.getOrDefault(NotifierIdentifier.FixedFeedIn), + maybeControllingEm, ) case input: LoadInput => buildLoad( @@ -239,6 +338,7 @@ class GridAgentController( resolution, requestVoltageDeviationThreshold, outputConfigUtil.getOrDefault(NotifierIdentifier.Load), + maybeControllingEm, ) case input: PvInput => buildPv( @@ -253,6 +353,7 @@ class GridAgentController( resolution, requestVoltageDeviationThreshold, outputConfigUtil.getOrDefault(NotifierIdentifier.PvPlant), + maybeControllingEm, ) case input: WecInput => buildWec( @@ -267,6 +368,7 @@ class GridAgentController( resolution, requestVoltageDeviationThreshold, outputConfigUtil.getOrDefault(NotifierIdentifier.Wec), + maybeControllingEm, ) case input: EvcsInput => buildEvcs( @@ -285,6 +387,7 @@ class GridAgentController( resolution, requestVoltageDeviationThreshold, outputConfigUtil.getOrDefault(NotifierIdentifier.Evcs), + maybeControllingEm, ) case hpInput: HpInput => thermalIslandGridsByBusId.get(hpInput.getThermalBus.getUuid) match { @@ -292,11 +395,14 @@ class GridAgentController( buildHp( hpInput, thermalGrid, - participantConfigUtil.getOrDefault(hpInput.getUuid), + participantConfigUtil.getOrDefault[HpRuntimeConfig]( + hpInput.getUuid + ), environmentRefs.primaryServiceProxy, environmentRefs.weather, requestVoltageDeviationThreshold, outputConfigUtil.getOrDefault(NotifierIdentifier.Hp), + maybeControllingEm, ) case None => throw new GridAgentInitializationException( @@ -332,6 +438,8 @@ class GridAgentController( * Maximum deviation in p.u. of request voltages to be considered equal * @param outputConfig * Configuration of the output behavior + * @param maybeControllingEm + * The parent EmAgent, if applicable * @return * The [[FixedFeedInAgent]] 's [[ActorRef]] */ @@ -344,6 +452,7 @@ class GridAgentController( resolution: Long, requestVoltageDeviationThreshold: Double, outputConfig: NotifierConfig, + maybeControllingEm: Option[ActorRef[FlexResponse]], ): ActorRef[ParticipantMessage] = gridAgentContext.toClassic .simonaActorOf( @@ -359,6 +468,7 @@ class GridAgentController( resolution, requestVoltageDeviationThreshold, outputConfig, + maybeControllingEm, ), listener.map(_.toClassic), ), @@ -385,6 +495,8 @@ class GridAgentController( * Maximum deviation in p.u. of request voltages to be considered equal * @param outputConfig * Configuration of the output behavior + * @param maybeControllingEm + * The parent EmAgent, if applicable * @return * The [[LoadAgent]] 's [[ActorRef]] */ @@ -397,6 +509,7 @@ class GridAgentController( resolution: Long, requestVoltageDeviationThreshold: Double, outputConfig: NotifierConfig, + maybeControllingEm: Option[ActorRef[FlexResponse]], ): ActorRef[ParticipantMessage] = gridAgentContext.toClassic .simonaActorOf( @@ -412,6 +525,7 @@ class GridAgentController( resolution, requestVoltageDeviationThreshold, outputConfig, + maybeControllingEm, ), listener.map(_.toClassic), ), @@ -440,6 +554,8 @@ class GridAgentController( * Maximum deviation in p.u. of request voltages to be considered equal * @param outputConfig * Configuration of the output behavior + * @param maybeControllingEm + * The parent EmAgent, if applicable * @return * The [[PvAgent]] 's [[ActorRef]] */ @@ -453,6 +569,7 @@ class GridAgentController( resolution: Long, requestVoltageDeviationThreshold: Double, outputConfig: NotifierConfig, + maybeControllingEm: Option[ActorRef[FlexResponse]], ): ActorRef[ParticipantMessage] = gridAgentContext.toClassic .simonaActorOf( @@ -468,6 +585,7 @@ class GridAgentController( resolution, requestVoltageDeviationThreshold, outputConfig, + maybeControllingEm, ), listener.map(_.toClassic), ), @@ -496,6 +614,8 @@ class GridAgentController( * Maximum deviation in p.u. of request voltages to be considered equal * @param outputConfig * Configuration of the output behavior + * @param maybeControllingEm + * The parent EmAgent, if applicable * @return * The [[EvcsAgent]] 's [[ActorRef]] */ @@ -509,6 +629,7 @@ class GridAgentController( resolution: Long, requestVoltageDeviationThreshold: Double, outputConfig: NotifierConfig, + maybeControllingEm: Option[ActorRef[FlexResponse]], ): ActorRef[ParticipantMessage] = gridAgentContext.toClassic .simonaActorOf( @@ -528,6 +649,7 @@ class GridAgentController( resolution, requestVoltageDeviationThreshold, outputConfig, + maybeControllingEm, ), listener.map(_.toClassic), ) @@ -535,6 +657,7 @@ class GridAgentController( .toTyped /** Builds an [[HpAgent]] from given input + * * @param hpInput * Input model * @param thermalGrid @@ -549,6 +672,8 @@ class GridAgentController( * Permissible voltage magnitude deviation to consider being equal * @param outputConfig * Configuration for output notification + * @param maybeControllingEm + * The parent EmAgent, if applicable * @return * A tuple of actor reference and [[ParticipantInitializeStateData]] */ @@ -560,6 +685,7 @@ class GridAgentController( weatherService: ClassicRef, requestVoltageDeviationThreshold: Double, outputConfig: NotifierConfig, + maybeControllingEm: Option[ActorRef[FlexResponse]], ): ActorRef[ParticipantMessage] = gridAgentContext.toClassic .simonaActorOf( @@ -576,6 +702,7 @@ class GridAgentController( resolution, requestVoltageDeviationThreshold, outputConfig, + maybeControllingEm, ), listener.map(_.toClassic), ), @@ -583,13 +710,13 @@ class GridAgentController( ) .toTyped - /** 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 @@ -604,6 +731,8 @@ class GridAgentController( * Maximum deviation in p.u. of request voltages to be considered equal * @param outputConfig * Configuration of the output behavior + * @param maybeControllingEm + * The parent EmAgent, if applicable * @return * The [[WecAgent]] 's [[ActorRef]] */ @@ -617,6 +746,7 @@ class GridAgentController( resolution: Long, requestVoltageDeviationThreshold: Double, outputConfig: NotifierConfig, + maybeControllingEm: Option[ActorRef[FlexResponse]], ): ActorRef[ParticipantMessage] = gridAgentContext.toClassic .simonaActorOf( @@ -632,6 +762,7 @@ class GridAgentController( resolution, requestVoltageDeviationThreshold, outputConfig, + maybeControllingEm, ), listener.map(_.toClassic), ), @@ -639,6 +770,42 @@ class GridAgentController( ) .toTyped + /** Builds an [[EmAgent]] from given input + * + * @param emInput + * The input model + * @param modelConfiguration + * Runtime configuration for the agent + * @param outputConfig + * Configuration for output notification + * @param maybeControllingEm + * The parent EmAgent, if applicable + * @return + * The [[EmAgent]] 's [[ActorRef]] + */ + private def buildEm( + emInput: EmInput, + modelConfiguration: EmRuntimeConfig, + outputConfig: NotifierConfig, + maybeControllingEm: Option[ActorRef[FlexResponse]], + ): ActorRef[FlexResponse] = + gridAgentContext.spawn( + EmAgent( + emInput, + modelConfiguration, + outputConfig, + maybeControllingEm + .map(_ => "PRIORITIZED") + .getOrElse("PROPORTIONAL"), + simulationStartDate, + maybeControllingEm.toRight( + environmentRefs.scheduler + ), + listener, + ), + actorName(classOf[EmAgent.type], emInput.getId), + ) + /** Introduces the given agent to scheduler * * @param actorRef 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/config/SimonaConfig.scala b/src/main/scala/edu/ie3/simona/config/SimonaConfig.scala index 8e264f6f18..c64c62ca3f 100644 --- a/src/main/scala/edu/ie3/simona/config/SimonaConfig.scala +++ b/src/main/scala/edu/ie3/simona/config/SimonaConfig.scala @@ -79,6 +79,75 @@ 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, + curtailRegenerative: 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_REG", + curtailRegenerative = + c.hasPathOrNull("curtailRegenerative") && c.getBoolean( + "curtailRegenerative" + ), + 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, @@ -2240,6 +2309,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, @@ -2249,6 +2319,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], @@ -2525,6 +2640,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{}"), 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..216ceb44e0 --- /dev/null +++ b/src/main/scala/edu/ie3/simona/model/em/EmAggregateFlex.scala @@ -0,0 +1,30 @@ +/* + * © 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 + +/** 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) + ] + ): (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 new file mode 100644 index 0000000000..4451d4bb00 --- /dev/null +++ b/src/main/scala/edu/ie3/simona/model/em/EmAggregateSelfOpt.scala @@ -0,0 +1,63 @@ +/* + * © 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, 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 flex potential of PV/WEC from the calculation + * + * @param curtailRegenerative + * Whether to include positive flexibility of PV/WEC in reference sum + * calculation + */ +final case class EmAggregateSelfOpt(curtailRegenerative: Boolean) + extends EmAggregateFlex { + + override def aggregateFlexOptions( + flexOptions: Iterable[ + (_ <: AssetInput, ProvideMinMaxFlexOptions) + ] + ): (Power, Power, Power) = { + val (minSum, maxSum) = + flexOptions.foldLeft((zeroKW, zeroKW)) { + case ( + (sumMin, sumMax), + (_, ProvideMinMaxFlexOptions(_, _, addMin, addMax)), + ) => + ( + sumMin + addMin, + sumMax + addMax, + ) + } + + val maxRefSum = + if (curtailRegenerative) + maxSum + else + flexOptions.foldLeft(zeroKW) { + case ( + maxSumExclReg, + (inputModel, ProvideMinMaxFlexOptions(_, _, addMin, addMax)), + ) => + inputModel match { + case _: PvInput | _: WecInput => + maxSumExclReg + addMin + case _ => maxSumExclReg + addMax + } + } + + // take the closest power possible to zero + 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 new file mode 100644 index 0000000000..f869616801 --- /dev/null +++ b/src/main/scala/edu/ie3/simona/model/em/EmAggregateSimpleSum.scala @@ -0,0 +1,27 @@ +/* + * © 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 + +/** Aggregates reference, minimum and maximum power by just simply summing up + * each value + */ +object EmAggregateSimpleSum extends EmAggregateFlex { + + override def aggregateFlexOptions( + flexOptions: Iterable[ + (_ <: AssetInput, ProvideMinMaxFlexOptions) + ] + ): (Power, Power, Power) = { + 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 new file mode 100644 index 0000000000..6a15d08624 --- /dev/null +++ b/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala @@ -0,0 +1,140 @@ +/* + * © 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 strategies 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 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) + + model -> power + } + } + +} + +object EmModelShell { + def apply( + uuid: UUID, + id: String, + modelStrategyName: String, + modelConfig: EmRuntimeConfig, + ): EmModelShell = { + + val modelStrategy = modelStrategyName match { + case "PROPORTIONAL" => ProportionalFlexStrat + case "PRIORITIZED" => + PrioritizedFlexStrat(modelConfig.curtailRegenerative) + case unknown => + throw new CriticalFailureException(s"Unknown model strategy $unknown") + } + + val aggregateFlex = modelConfig.aggregateFlex match { + 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" + ) + } + + 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/PrioritizedFlexStrat.scala b/src/main/scala/edu/ie3/simona/model/em/PrioritizedFlexStrat.scala new file mode 100644 index 0000000000..98162db86c --- /dev/null +++ b/src/main/scala/edu/ie3/simona/model/em/PrioritizedFlexStrat.scala @@ -0,0 +1,216 @@ +/* + * © 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, + WecInput, +} +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 java.util.UUID + +/** Determines flex control for connected agents by adhering to a priority + * hierarchy, with some devices not controlled at all. + * + * @param curtailRegenerative + * Whether PV and WEC feed-in can be curtailed or not + */ +final case class PrioritizedFlexStrat(curtailRegenerative: Boolean) + extends EmModelStrat { + + /** 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(curtailRegenerative)(Seq(classOf[PvInput], classOf[WecInput])) + .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 CriticalFailureException( + "No flexibilities have been 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(_ => curtailRegenerative) // only if enabled + val wecOpt = flexOptions + .collectFirst { case flex @ (_: WecInput, _) => + flex + } + .filter(_ => curtailRegenerative) // only if enabled + + if (zeroKW.~=(targetDelta)(tolerance)) { + Seq.empty + } else if (targetDelta < zeroKW) { + // suggested power too low, try to store difference/increase load + + val orderedParticipants = + Seq(evcsOpt, storageOpt, heatPumpOpt, pvOpt, wecOpt).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 (zeroKW.~=(remainingExcessPower)(tolerance)) { + // we're already there (besides rounding error) + (issueCtrlMsgs, None) + } else if (zeroKW.~=(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 (zeroKW.~=(remainingExcessPower)(tolerance)) { + // we're already there (besides rounding error) + (issueCtrlMsgs, None) + } else if (zeroKW.~=(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..97647ebaa3 --- /dev/null +++ b/src/main/scala/edu/ie3/simona/model/em/ProportionalFlexStrat.scala @@ -0,0 +1,124 @@ +/* + * © 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 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.flexSum + + 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 deltaToLimit = totalLimit - totalRef + val deltaToTarget = target - totalRef + + val flexShare = deltaToTarget / deltaToLimit + + 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/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/FlexibilityMessage.scala b/src/main/scala/edu/ie3/simona/ontology/messages/flex/FlexibilityMessage.scala index f6994db75c..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,6 +7,7 @@ package edu.ie3.simona.ontology.messages.flex import edu.ie3.datamodel.models.input.AssetInput +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 @@ -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 EmAgent.Request { val modelUuid: UUID } 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..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 @@ -39,21 +40,27 @@ 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 { + 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/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/SetupHelper.scala b/src/main/scala/edu/ie3/simona/sim/setup/SetupHelper.scala index 55d907ae78..355cd7156e 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SetupHelper.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SetupHelper.scala @@ -260,9 +260,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/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/main/scala/edu/ie3/simona/util/ConfigUtil.scala b/src/main/scala/edu/ie3/simona/util/ConfigUtil.scala index 10acb7ddaf..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 @@ -89,7 +88,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 +98,7 @@ object ConfigUtil { subConfig.evcs.defaultConfig, subConfig.wec.defaultConfig, subConfig.hp.defaultConfig, + subConfig.em.defaultConfig, ).map { conf => conf.getClass -> conf }.toMap, ) } @@ -141,10 +142,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 +170,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 +282,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 +292,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/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) } diff --git a/src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala b/src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala new file mode 100644 index 0000000000..cc0ce1f43b --- /dev/null +++ b/src/test/scala/edu/ie3/simona/agent/em/EmAgentIT.scala @@ -0,0 +1,433 @@ +/* + * © 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.secondary.SecondaryDataService.ActorWeatherService +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.config.SimonaConfig._ +import edu.ie3.simona.event.ResultEvent +import edu.ie3.simona.event.ResultEvent.ParticipantResultEvent +import edu.ie3.simona.event.notifier.NotifierConfig +import edu.ie3.simona.ontology.messages.SchedulerMessage.{ + Completion, + ScheduleActivation, +} +import edu.ie3.simona.ontology.messages.services.ServiceMessage +import edu.ie3.simona.ontology.messages.services.ServiceMessage.PrimaryServiceRegistrationMessage +import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.{ + RegistrationFailedMessage, + RegistrationSuccessfulMessage, +} +import edu.ie3.simona.ontology.messages.services.WeatherMessage.{ + ProvideWeatherMessage, + RegisterForWeatherMessage, + WeatherData, +} +import edu.ie3.simona.ontology.messages.{Activation, SchedulerMessage} +import edu.ie3.simona.test.common.input.EmInputTestData +import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK +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.WattsPerSquareMeter +import org.apache.pekko.actor.ActorSystem +import org.apache.pekko.actor.testkit.typed.scaladsl.{ + ScalaTestWithActorTestKit, + TestProbe, +} +import org.apache.pekko.actor.typed.scaladsl.adapter._ +import org.apache.pekko.testkit.TestActorRef +import org.scalatest.OptionValues._ +import org.scalatest.matchers.should +import org.scalatest.wordspec.AnyWordSpecLike +import org.scalatestplus.mockito.MockitoSugar +import squants.motion.MetersPerSecond +import squants.thermal.Celsius + +import java.time.ZonedDateTime + +class EmAgentIT + extends ScalaTestWithActorTestKit + with AnyWordSpecLike + with should.Matchers + with EmInputTestData + with MockitoSugar { + + // start a bit later so the sun is up + protected implicit val simulationStartDate: ZonedDateTime = + TimeUtil.withDefaults.toZonedDateTime("2020-01-01T10:00:00Z") + protected val simulationEndDate: ZonedDateTime = + TimeUtil.withDefaults.toZonedDateTime("2020-01-02T02:00:00Z") + + private val resolution = + simonaConfig.simona.powerflow.resolution.getSeconds + + private val outputConfigOn = NotifierConfig( + simulationResultInfo = true, + powerRequestReply = false, + flexResult = false, + ) + + private val outputConfigOff = NotifierConfig( + simulationResultInfo = false, + powerRequestReply = false, + flexResult = false, + ) + + override protected val modelConfig: EmRuntimeConfig = EmRuntimeConfig( + calculateMissingReactivePowerWithModel = false, + scaling = 1d, + uuids = List("default"), + aggregateFlex = "SELF_OPT", + curtailRegenerative = false, + ) + + private implicit val quantityTolerance: Double = 1e-10d + + private implicit val classicSystem: ActorSystem = system.toClassic + + "An em agent" when { + + "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), + 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 also be turned 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 still running, can still 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 stay 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 on again and cannot be turned off + -> 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))) + } + } + + } +} 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..fb06c37e8d --- /dev/null +++ b/src/test/scala/edu/ie3/simona/agent/em/EmAgentSpec.scala @@ -0,0 +1,692 @@ +/* + * © 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-01T00:00:00Z") + + 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_REG", + curtailRegenerative = 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), + 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), + 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), + 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), + 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 23c9062a79..a5b11ec086 100644 --- a/src/test/scala/edu/ie3/simona/event/NotifierSpec.scala +++ b/src/test/scala/edu/ie3/simona/event/NotifierSpec.scala @@ -96,6 +96,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..9ae0ee63c1 --- /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(curtailRegenerative = 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(curtailRegenerative = 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..b5ebb9c50b --- /dev/null +++ b/src/test/scala/edu/ie3/simona/model/em/PrioritizedFlexStratSpec.scala @@ -0,0 +1,340 @@ +/* + * © 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(curtailRegenerative = 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, 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 + (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 curtailment enabled" in { + val strat = PrioritizedFlexStrat(curtailRegenerative = 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( + ("curtailRegenerative", "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 (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 + } + } + } + + } +} 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..4a4834fff9 --- /dev/null +++ b/src/test/scala/edu/ie3/simona/model/em/ProportionalFlexStratSpec.scala @@ -0,0 +1,173 @@ +/* + * © 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 + + /* 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", + "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/sim/SimonaSimSpec.scala b/src/test/scala/edu/ie3/simona/sim/SimonaSimSpec.scala index 07522bfe44..c6ffda8fab 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 @@ -66,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) } } ), @@ -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,8 +451,8 @@ object SimonaSimSpec { override def extSimulations( context: ActorContext[_], - scheduler: ActorRef[SchedulerMessage], + rootScheduler: ActorRef[SchedulerMessage], ): ExtSimSetupData = - ExtSimSetupData(Iterable.empty, Map.empty) + ExtSimSetupData(Iterable.empty, Map.empty, None) } } 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" ) 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 bb3d526e05..c1d93d7476 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..4c349c603e --- /dev/null +++ b/src/test/scala/edu/ie3/simona/test/common/input/EmInputTestData.scala @@ -0,0 +1,159 @@ +/* + * © 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.{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.characteristic.{ + CosPhiFixed, + ReactivePowerCharacteristic, +} +import edu.ie3.datamodel.models.input.system.{EvcsInput, HpInput, StorageInput} +import edu.ie3.datamodel.models.input.thermal.{ + ThermalHouseInput, + ThermalStorageInput, +} +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} +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 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", + new OperatorInput(UUID.randomUUID(), "NO_OPERATOR"), + OperationTime.notLimited(), + nodeInputNoSlackNs04KvA, + CosPhiFixed.CONSTANT_CHARACTERISTIC, + emInput, + 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, + emInput, + householdStorageTypeInput, + ) + + 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)}"), + emInput, + 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..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 6 + 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 6 + 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 6 + defaultConfigs.size shouldBe 7 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 } }