From a68277650b1f66e6aeea3653a5bb59e4943eb7e2 Mon Sep 17 00:00:00 2001 From: Johannes Bao Date: Tue, 16 Jan 2024 14:58:12 +0100 Subject: [PATCH 01/41] Temporary --- .../statedata/ParticipantStateData.scala | 10 +- .../edu/ie3/simona/event/ResultEvent.scala | 33 +- .../ontology/messages/SchedulerMessage.scala | 2 + .../messages/services/DataMessage.scala | 3 + .../messages/services/EvMessage.scala | 2 +- .../messages/services/ResultMessage.scala | 11 + .../ie3/simona/service/ExtDataSupport.scala | 3 +- .../simona/service/ev/ExtEvDataService.scala | 13 +- .../primary/ExtPrimaryDataService.scala | 93 +++ .../primary/ExtPrimaryServiceProxy.scala | 99 +++ .../primary/ExtPrimaryServiceWorker.scala | 218 +++++++ .../primary/IntPrimaryServiceProxy.scala | 566 ++++++++++++++++ .../service/primary/PrimaryServiceProxy.scala | 614 ++---------------- .../results/ExtResultDataService.scala | 135 ++++ .../scala/edu/ie3/simona/sim/SimonaSim.scala | 45 +- .../simona/sim/setup/ExtSimSetupData.scala | 4 + .../ie3/simona/sim/setup/SimonaSetup.scala | 3 +- .../sim/setup/SimonaStandaloneSetup.scala | 54 +- 18 files changed, 1297 insertions(+), 611 deletions(-) create mode 100644 src/main/scala/edu/ie3/simona/ontology/messages/services/DataMessage.scala create mode 100644 src/main/scala/edu/ie3/simona/ontology/messages/services/ResultMessage.scala create mode 100644 src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala create mode 100644 src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryServiceProxy.scala create mode 100644 src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryServiceWorker.scala create mode 100644 src/main/scala/edu/ie3/simona/service/primary/IntPrimaryServiceProxy.scala create mode 100644 src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala diff --git a/src/main/scala/edu/ie3/simona/agent/participant/statedata/ParticipantStateData.scala b/src/main/scala/edu/ie3/simona/agent/participant/statedata/ParticipantStateData.scala index d52896da0c..163b0a8151 100644 --- a/src/main/scala/edu/ie3/simona/agent/participant/statedata/ParticipantStateData.scala +++ b/src/main/scala/edu/ie3/simona/agent/participant/statedata/ParticipantStateData.scala @@ -110,7 +110,7 @@ object ParticipantStateData { ]( inputModel: InputModelContainer[I], modelConfig: C, - primaryServiceProxy: ActorRef, + primaryServiceProxies: Option[Vector[ActorRef]], secondaryDataServices: Option[ Vector[SecondaryDataService[_ <: SecondaryData]] ], @@ -129,7 +129,7 @@ object ParticipantStateData { ]( inputModel: I, modelConfig: C, - primaryServiceProxy: ActorRef, + primaryServiceProxies: Option[Vector[ActorRef]], secondaryDataServices: Option[ Vector[SecondaryDataService[_ <: SecondaryData]] ], @@ -142,7 +142,7 @@ object ParticipantStateData { new ParticipantInitializeStateData[I, C, PD]( SimpleInputContainer(inputModel), modelConfig, - primaryServiceProxy, + primaryServiceProxies, secondaryDataServices, simulationStartDate, simulationEndDate, @@ -159,7 +159,7 @@ object ParticipantStateData { inputModel: I, thermalGrid: ThermalGrid, modelConfig: C, - primaryServiceProxy: ActorRef, + primaryServiceProxies: Option[Vector[ActorRef]], secondaryDataServices: Option[ Vector[SecondaryDataService[_ <: SecondaryData]] ], @@ -172,7 +172,7 @@ object ParticipantStateData { new ParticipantInitializeStateData[I, C, PD]( WithHeatInputContainer(inputModel, thermalGrid), modelConfig, - primaryServiceProxy, + primaryServiceProxies, secondaryDataServices, simulationStartDate, simulationEndDate, diff --git a/src/main/scala/edu/ie3/simona/event/ResultEvent.scala b/src/main/scala/edu/ie3/simona/event/ResultEvent.scala index 7244d941f5..e0198883d2 100644 --- a/src/main/scala/edu/ie3/simona/event/ResultEvent.scala +++ b/src/main/scala/edu/ie3/simona/event/ResultEvent.scala @@ -6,18 +6,17 @@ package edu.ie3.simona.event -import edu.ie3.datamodel.models.result.NodeResult -import edu.ie3.datamodel.models.result.connector.{ - LineResult, - SwitchResult, - Transformer2WResult -} +import edu.ie3.datamodel.models.result.{NodeResult, ResultEntity} +import edu.ie3.datamodel.models.result.connector.{LineResult, SwitchResult, Transformer2WResult} import edu.ie3.datamodel.models.result.system.SystemParticipantResult import edu.ie3.datamodel.models.result.thermal.ThermalUnitResult import edu.ie3.simona.agent.grid.GridResultsSupport.PartialTransformer3wResult import edu.ie3.simona.event.listener.ResultEventListener.ResultMessage -sealed trait ResultEvent extends ResultMessage + +sealed trait ResultEvent extends ResultMessage { + def getResults(): Iterable[ResultEntity] +} /** Calculation result events */ @@ -31,7 +30,9 @@ object ResultEvent { */ final case class ParticipantResultEvent( systemParticipantResult: SystemParticipantResult - ) extends ResultEvent + ) extends ResultEvent { + override def getResults(): Iterable[ResultEntity] = Iterable(systemParticipantResult) + } /** Event, that is triggered every time a thermal model has a new result * @param thermalResult @@ -39,7 +40,9 @@ object ResultEvent { */ final case class ThermalResultEvent( thermalResult: ThermalUnitResult - ) extends ResultEvent + ) extends ResultEvent { + override def getResults(): Iterable[ResultEntity] = Iterable(thermalResult) + } /** Event that holds all grid calculation results of a power flow calculation. * The usage of a type is necessary here, to avoid passing in other instances @@ -63,6 +66,16 @@ object ResultEvent { lineResults: Iterable[LineResult], transformer2wResults: Iterable[Transformer2WResult], transformer3wResults: Iterable[PartialTransformer3wResult] - ) extends ResultEvent + ) extends ResultEvent { + override def getResults(): Iterable[ResultEntity] = { + var results: Iterable[ResultEntity] = Iterable.empty[ResultEntity] + results = results ++ nodeResults + results = results ++ switchResults + results = results ++ lineResults + results = results ++ transformer2wResults + results = results ++ transformer3wResults + results + } + } } diff --git a/src/main/scala/edu/ie3/simona/ontology/messages/SchedulerMessage.scala b/src/main/scala/edu/ie3/simona/ontology/messages/SchedulerMessage.scala index 866cae63a8..663e457327 100644 --- a/src/main/scala/edu/ie3/simona/ontology/messages/SchedulerMessage.scala +++ b/src/main/scala/edu/ie3/simona/ontology/messages/SchedulerMessage.scala @@ -6,6 +6,7 @@ package edu.ie3.simona.ontology.messages +import edu.ie3.datamodel.models.result.ResultEntity import org.apache.pekko.actor.typed.ActorRef import edu.ie3.simona.scheduler.ScheduleLock.ScheduleKey import edu.ie3.simona.scheduler.{Scheduler, TimeAdvancer} @@ -24,4 +25,5 @@ object SchedulerMessage { unlockKey: Option[ScheduleKey] = None ) extends SchedulerMessage + final case class ResultForExtCompleteMessage(results: Iterable[ResultEntity]) extends SchedulerMessage } diff --git a/src/main/scala/edu/ie3/simona/ontology/messages/services/DataMessage.scala b/src/main/scala/edu/ie3/simona/ontology/messages/services/DataMessage.scala new file mode 100644 index 0000000000..92a08aa584 --- /dev/null +++ b/src/main/scala/edu/ie3/simona/ontology/messages/services/DataMessage.scala @@ -0,0 +1,3 @@ +package edu.ie3.simona.ontology.messages.services + +trait DataMessage {} diff --git a/src/main/scala/edu/ie3/simona/ontology/messages/services/EvMessage.scala b/src/main/scala/edu/ie3/simona/ontology/messages/services/EvMessage.scala index 760a47296c..4b1dd3e727 100644 --- a/src/main/scala/edu/ie3/simona/ontology/messages/services/EvMessage.scala +++ b/src/main/scala/edu/ie3/simona/ontology/messages/services/EvMessage.scala @@ -16,7 +16,7 @@ import edu.ie3.simona.scheduler.ScheduleLock.ScheduleKey import java.util.UUID -sealed trait EvMessage +sealed trait EvMessage extends DataMessage object EvMessage { diff --git a/src/main/scala/edu/ie3/simona/ontology/messages/services/ResultMessage.scala b/src/main/scala/edu/ie3/simona/ontology/messages/services/ResultMessage.scala new file mode 100644 index 0000000000..d5dd04b7a2 --- /dev/null +++ b/src/main/scala/edu/ie3/simona/ontology/messages/services/ResultMessage.scala @@ -0,0 +1,11 @@ +package edu.ie3.simona.ontology.messages.services + +import edu.ie3.datamodel.models.result.ResultEntity + +sealed trait ResultMessage extends DataMessage + +object ResultMessage { + final case class ResultRequest(tick: Long) + + final case class ResultResponseMessage(results: Iterable[ResultEntity]) extends ResultMessage +} \ No newline at end of file diff --git a/src/main/scala/edu/ie3/simona/service/ExtDataSupport.scala b/src/main/scala/edu/ie3/simona/service/ExtDataSupport.scala index 31800da210..e8297bfbbe 100644 --- a/src/main/scala/edu/ie3/simona/service/ExtDataSupport.scala +++ b/src/main/scala/edu/ie3/simona/service/ExtDataSupport.scala @@ -7,6 +7,7 @@ package edu.ie3.simona.service import edu.ie3.simona.api.data.ontology.DataMessageFromExt +import edu.ie3.simona.ontology.messages.services.{DataMessage, EvMessage} import edu.ie3.simona.ontology.messages.services.EvMessage.EvResponseMessage import edu.ie3.simona.service.ServiceStateData.ServiceBaseStateData @@ -49,6 +50,6 @@ trait ExtDataSupport[ * the updated state data */ protected def handleDataResponseMessage( - extResponseMsg: EvResponseMessage + extResponseMsg: DataMessage //extResponseMsg )(implicit serviceStateData: S): S } diff --git a/src/main/scala/edu/ie3/simona/service/ev/ExtEvDataService.scala b/src/main/scala/edu/ie3/simona/service/ev/ExtEvDataService.scala index 4bd2975747..8bfdd339e4 100644 --- a/src/main/scala/edu/ie3/simona/service/ev/ExtEvDataService.scala +++ b/src/main/scala/edu/ie3/simona/service/ev/ExtEvDataService.scala @@ -14,18 +14,13 @@ import edu.ie3.simona.api.data.ev.ontology._ import edu.ie3.simona.api.data.ontology.DataMessageFromExt import edu.ie3.simona.exceptions.WeatherServiceException.InvalidRegistrationRequestException import edu.ie3.simona.exceptions.{InitializationException, ServiceException} +import edu.ie3.simona.ontology.messages.services.{DataMessage, EvMessage} import edu.ie3.simona.ontology.messages.services.EvMessage._ import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.RegistrationSuccessfulMessage import edu.ie3.simona.ontology.messages.services.ServiceMessage.ServiceRegistrationMessage import edu.ie3.simona.scheduler.ScheduleLock -import edu.ie3.simona.service.ServiceStateData.{ - InitializeServiceStateData, - ServiceBaseStateData -} -import edu.ie3.simona.service.ev.ExtEvDataService.{ - ExtEvStateData, - InitExtEvData -} +import edu.ie3.simona.service.ServiceStateData.{InitializeServiceStateData, ServiceBaseStateData} +import edu.ie3.simona.service.ev.ExtEvDataService.{ExtEvStateData, InitExtEvData} import edu.ie3.simona.service.{ExtDataSupport, ServiceStateData, SimonaService} import java.util.UUID @@ -308,7 +303,7 @@ class ExtEvDataService(override val scheduler: ActorRef) } override protected def handleDataResponseMessage( - extResponseMsg: EvResponseMessage + extResponseMsg: DataMessage )(implicit serviceStateData: ExtEvStateData): ExtEvStateData = { extResponseMsg match { case DepartingEvsResponse(evcs, evModels) => diff --git a/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala b/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala new file mode 100644 index 0000000000..6d7d8af2cc --- /dev/null +++ b/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala @@ -0,0 +1,93 @@ +package edu.ie3.simona.service.primary + +import edu.ie3.simona.api.data.ontology.DataMessageFromExt +import edu.ie3.simona.ontology.messages.services.{DataMessage, ServiceMessage} +import edu.ie3.simona.service.{ExtDataSupport, ServiceStateData, SimonaService} +import edu.ie3.simona.service.primary.ExtPrimaryServiceWorker.ExtPrimaryDataStateData +import org.apache.pekko.actor.{ActorContext, ActorRef, Props} + +import scala.util.Try + +object ExtPrimaryDataService { + + def props(scheduler: ActorRef): Props = + Props( + new ExtPrimaryDataService(scheduler: ActorRef) + ) + + +} +final case class ExtPrimaryDataService( + override val scheduler: ActorRef + ) + extends SimonaService[ExtPrimaryDataStateData](scheduler) + with ExtDataSupport[ExtPrimaryDataStateData] { + + /** Initialize the concrete service implementation using the provided + * initialization data. This method should perform all heavyweight tasks + * before the actor becomes ready. The return values are a) the state data of + * the initialized service and b) optional triggers that should be send to + * the [[edu.ie3.simona.scheduler.Scheduler]] together with the completion + * message that is send in response to the trigger that is send to start the + * initialization process + * + * @param initServiceData + * the data that should be used for initialization + * @return + * the state data of this service and optional tick that should be included + * in the completion message + */ + override def init(initServiceData: ServiceStateData.InitializeServiceStateData): Try[(ExtPrimaryDataStateData, Option[Long])] = ??? + + /** Handle a request to register for information from this service + * + * @param registrationMessage + * registration message to handle + * @param serviceStateData + * current state data of the actor + * @return + * the service stata data that should be used in the next state (normally + * with updated values) + */ + override protected def handleRegistrationRequest( + registrationMessage: ServiceMessage.ServiceRegistrationMessage + )(implicit serviceStateData: ExtPrimaryDataStateData): Try[ExtPrimaryDataStateData] = ??? + + /** Send out the information to all registered recipients + * + * @param tick + * current tick data should be announced for + * @param serviceStateData + * the current state data of this service + * @return + * the service stata data that should be used in the next state (normally + * with updated values) together with the completion message that is send + * in response to the trigger that was sent to start this announcement + */ + override protected def announceInformation(tick: Long)(implicit serviceStateData: ExtPrimaryDataStateData, ctx: ActorContext): (ExtPrimaryDataStateData, Option[Long]) = ??? + + /** Handle a message from outside the simulation + * + * @param extMsg + * the external incoming message + * @param serviceStateData + * the current state data of this service + * @return + * the updated state data + */ + override protected def handleDataMessage(extMsg: DataMessageFromExt)(implicit serviceStateData: ExtPrimaryDataStateData): ExtPrimaryDataStateData = ??? + + /** Handle a message from inside SIMONA sent to external + * + * @param extResponseMsg + * the external incoming message + * @param serviceStateData + * the current state data of this service + * @return + * the updated state data + */ + override protected def handleDataResponseMessage(extResponseMsg: DataMessage)(implicit serviceStateData: ExtPrimaryDataStateData): ExtPrimaryDataStateData = ??? + + +} + diff --git a/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryServiceProxy.scala b/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryServiceProxy.scala new file mode 100644 index 0000000000..ffc76386e0 --- /dev/null +++ b/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryServiceProxy.scala @@ -0,0 +1,99 @@ +package edu.ie3.simona.service.primary + +import edu.ie3.datamodel.io.naming.timeseries.IndividualTimeSeriesMetaInformation +import edu.ie3.simona.api.ExtSimAdapter +import edu.ie3.simona.api.data.primarydata.ExtPrimaryData +import edu.ie3.simona.config.SimonaConfig.Simona.Input +import edu.ie3.simona.ontology.messages.services.ServiceMessage.{PrimaryServiceRegistrationMessage, WorkerRegistrationMessage} +import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.RegistrationFailedMessage +import edu.ie3.simona.service.primary.ExtPrimaryServiceProxy.ExtPrimaryServiceStateData +import edu.ie3.simona.service.primary.PrimaryServiceProxy.{InitPrimaryServiceProxyStateData, PrimaryServiceStateData} +import org.apache.pekko.actor.{Actor, ActorRef} + +import java.time.ZonedDateTime +import java.util.UUID +import scala.util.{Failure, Success, Try} + +case class ExtPrimaryServiceProxy( + scheduler: ActorRef, + initStateData: InitPrimaryServiceProxyStateData, + private implicit val startDateTime: ZonedDateTime + ) extends PrimaryServiceProxy(scheduler, initStateData, startDateTime) { + + /** Message handling, if the actor has been initialized already. This method + * basically handles registration requests, checks, if pre-calculated, + * primary data is available and forwards the request to worker actors. If + * needed, new workers are spun off. + * + * @param stateData + * Representing the current state of the agent + * @return + * Message handling routine + */ + private def onMessage(stateData: ExtPrimaryServiceStateData): Receive = { + case PrimaryServiceRegistrationMessage(modelUuid) => + // Ist für modelUuid eine externe primaere Datenquellen vorgesehen? + stateData.extPrimaryDataReferenceMap.get(modelUuid) match { + case Some(ExtPrimaryData) => // Ja! + // Registriere Agent beim entsprechenden Service + handleCoveredModel( + modelUuid, + stateData, + sender() + ) + case None => + log.debug( + s"There is no external data apparent for the model with uuid '{}'.", + modelUuid + ) + sender() ! RegistrationFailedMessage + } + case x => + log.error( + s"Received message '$x', but I'm only able to handle registration requests." + ) + unhandled(x) + } + + override def prepareStateData( + primaryConfig: Input.Primary, + simulationStart: ZonedDateTime + ): Try[PrimaryServiceStateData] = { + val participantToExtSimMapping: Map[UUID, ExtSimAdapter] = null + + + + + ExtPrimaryServiceStateData( + simulationStart, + extPrimaryServices, + extPrimaryDataReferenceMap + ) + } + + protected def handleCoveredModel( + modelUuid: UUID, + stateData: ExtPrimaryServiceStateData, + requestingActor: ActorRef + ): Unit = { + // Zu welchem DataService gehoert der Agent? + val participantToExtReferenceMap = stateData.extPrimaryDataReferenceMap + val worker = requestingActor + worker ! WorkerRegistrationMessage(requestingActor) + } +} + +object ExtPrimaryServiceProxy { + + final case class ExtPrimaryServiceStateData( + simulationStart: ZonedDateTime, + extPrimaryServices: Vector[ExtPrimaryServiceWorker], + extPrimaryDataReferenceMap: Map[UUID, ExtPrimaryData] + ) extends PrimaryServiceStateData( + simulationStart, null + ) + + + + +} diff --git a/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryServiceWorker.scala b/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryServiceWorker.scala new file mode 100644 index 0000000000..ea13f1be2e --- /dev/null +++ b/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryServiceWorker.scala @@ -0,0 +1,218 @@ +package edu.ie3.simona.service.primary + +import edu.ie3.datamodel.models.value.Value +import edu.ie3.simona.agent.participant.data.Data.PrimaryData +import edu.ie3.simona.agent.participant.data.Data.PrimaryData.RichValue +import edu.ie3.simona.api.data.ontology.DataMessageFromExt +import edu.ie3.simona.api.data.primarydata.ExtPrimaryData +import edu.ie3.simona.api.data.primarydata.ontology.{PrimaryDataMessageFromExt, ProvidePrimaryData} +import edu.ie3.simona.exceptions.WeatherServiceException.InvalidRegistrationRequestException +import edu.ie3.simona.exceptions.{InitializationException, ServiceException} +import edu.ie3.simona.ontology.messages.services.ServiceMessage.PrimaryServiceRegistrationMessage +import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.RegistrationSuccessfulMessage +import edu.ie3.simona.ontology.messages.services.{EvMessage, ServiceMessage} +import edu.ie3.simona.scheduler.ScheduleLock +import edu.ie3.simona.service.ServiceStateData.{InitializeServiceStateData, ServiceBaseStateData} +import edu.ie3.simona.service.primary.ExtPrimaryServiceWorker.{ExtPrimaryDataStateData, InitExtPrimaryData} +import edu.ie3.simona.service.primary.PrimaryServiceWorker.{PrimaryServiceInitializedStateData, ProvidePrimaryDataMessage} +import edu.ie3.simona.service.{ExtDataSupport, ServiceStateData, SimonaService} +import org.apache.pekko.actor.typed.scaladsl.adapter.ClassicActorRefOps +import org.apache.pekko.actor.{ActorContext, ActorRef, Props} + +import java.util.UUID +import scala.jdk.CollectionConverters.MapHasAsScala +import scala.util.{Failure, Success, Try} + +object ExtPrimaryServiceWorker { + def props(scheduler: ActorRef): Props = Props(new ExtPrimaryServiceWorker(scheduler: ActorRef)) + + final case class ExtPrimaryDataStateData( + extPrimaryData: ExtPrimaryData, + uuidToActorRef: Map[UUID, ActorRef] = Map.empty[UUID, ActorRef], // subscribers + extPrimaryDataMessage: Option[PrimaryDataMessageFromExt] = None + ) extends ServiceBaseStateData + + final case class InitExtPrimaryData( + extPrimaryData: ExtPrimaryData + ) extends InitializeServiceStateData +} + + +final case class ExtPrimaryServiceWorker( + override protected val scheduler: ActorRef + ) extends SimonaService[ExtPrimaryDataStateData](scheduler) + with ExtDataSupport[ExtPrimaryDataStateData] { + + + override def init( + initServiceData: ServiceStateData.InitializeServiceStateData + ): Try[(ExtPrimaryDataStateData, Option[Long])] = initServiceData match { + case InitExtPrimaryData(extPrimaryData) => + val evInitializedStateData = ExtPrimaryDataStateData( + extPrimaryData + ) + + Success( + evInitializedStateData, + None + ) + + case invalidData => + Failure( + new InitializationException( + s"Provided init data '${invalidData.getClass.getSimpleName}' for ExtPrimaryService are invalid!" + ) + ) + } + + override protected def handleRegistrationRequest( + registrationMessage: ServiceMessage.ServiceRegistrationMessage + )(implicit serviceStateData: ExtPrimaryDataStateData): + Try[ExtPrimaryDataStateData] = registrationMessage match { + case PrimaryServiceRegistrationMessage(participant) => + Success(handleRegistrationRequest(sender(), participant)) + case invalidMessage => + Failure( + InvalidRegistrationRequestException( + "Cannot register an agent for ev movement service with registration " + + s"request message '${invalidMessage.getClass.getSimpleName}'!" + ) + ) + } + + + private def handleRegistrationRequest( + agentToBeRegistered: ActorRef, + agentUUID: UUID + )(implicit serviceStateData: ExtPrimaryDataStateData): + ExtPrimaryDataStateData = { + serviceStateData.uuidToActorRef.get(agentUUID) match { + case None => + // Actor is not registered yet + agentToBeRegistered ! RegistrationSuccessfulMessage(None) + serviceStateData.copy( + uuidToActorRef = + serviceStateData.uuidToActorRef + (agentUUID -> agentToBeRegistered) + ) + case Some(_) => + // actor is already registered, do nothing + log.warning( + "Sending actor {} is already registered", + agentToBeRegistered + ) + serviceStateData + } + } + + /** Send out the information to all registered recipients + * + * @param tick + * current tick data should be announced for + * @param serviceStateData + * the current state data of this service + * @return + * the service stata data that should be used in the next state (normally + * with updated values) together with the completion message that is send + * in response to the trigger that was sent to start this announcement + */ + override protected def announceInformation( + tick: Long + )( + implicit serviceStateData: ExtPrimaryDataStateData, + ctx: ActorContext + ): (ExtPrimaryDataStateData, Option[Long]) = { + serviceStateData.extPrimaryDataMessage.getOrElse( + throw ServiceException( + "ExtPrimaryDataService was triggered without ExtPrimaryDataMessage available" + ) + ) match { + case providedPrimaryData: ProvidePrimaryData => + processDataAndAnnounce(tick, providedPrimaryData.primaryData)(serviceStateData, ctx) + } + } + + private def processDataAndAnnounce( + tick: Long, + primaryData: java.util.Map[UUID, Value] + )( + implicit + serviceStateData: ExtPrimaryDataStateData, + ctx: ActorContext + ): ( + ExtPrimaryDataStateData, + Option[Long] + ) = { + val actorToPrimaryData = primaryData.asScala.flatMap { + case (agent, primaryDataPerAgent) => + serviceStateData.uuidToActorRef + .get(agent) + .map((_, primaryDataPerAgent)) + .orElse { + log.warning( + "A corresponding actor ref for UUID {} could not be found", + agent + ) + None + } + } + + // Verteile Primary Data + if (actorToPrimaryData.nonEmpty) { + val keys = + ScheduleLock.multiKey(ctx, scheduler.toTyped, tick, actorToPrimaryData.size) + + actorToPrimaryData.zip(keys).foreach { + case ((actor, primaryDataPerAgent), key) => { + primaryDataPerAgent.toPrimaryData match { + case Success(primaryData) => + actor ! ProvidePrimaryDataMessage( + tick, + primaryData, + null, // nextDataTick + unlockKey = Some(key) + ) + case Failure(exception) => + /* Processing of data failed */ + log.warning( + "Unable to convert received value to primary data. Skipped that data." + + "\nException: {}", + exception + ) + } + + } + } + + } + + + ( // Message leeren + serviceStateData.copy( + extPrimaryDataMessage = None + ), + None + ) + + } + + override protected def handleDataMessage( + extMsg: DataMessageFromExt + )(implicit serviceStateData: ExtPrimaryDataStateData): + ExtPrimaryDataStateData = { + extMsg match { + case extPrimaryDataMessage: PrimaryDataMessageFromExt => + serviceStateData.copy( + extPrimaryDataMessage = Some(extPrimaryDataMessage) + ) + } + } + + override protected def handleDataResponseMessage( + extResponseMsg: EvMessage + )( + implicit serviceStateData: ExtPrimaryDataStateData): + ExtPrimaryDataStateData = { + // not implemented + null + } +} diff --git a/src/main/scala/edu/ie3/simona/service/primary/IntPrimaryServiceProxy.scala b/src/main/scala/edu/ie3/simona/service/primary/IntPrimaryServiceProxy.scala new file mode 100644 index 0000000000..1ee578a7a7 --- /dev/null +++ b/src/main/scala/edu/ie3/simona/service/primary/IntPrimaryServiceProxy.scala @@ -0,0 +1,566 @@ +/* + * © 2021. TU Dortmund University, + * Institute of Energy Systems, Energy Efficiency and Energy Economics, + * Research group Distribution grid planning and operation + */ + +package edu.ie3.simona.service.primary + +import org.apache.pekko.actor.typed.scaladsl.adapter.ClassicActorRefOps +import org.apache.pekko.actor.{Actor, ActorRef, PoisonPill, Props} +import edu.ie3.datamodel.io.connectors.SqlConnector +import edu.ie3.datamodel.io.csv.CsvIndividualTimeSeriesMetaInformation +import edu.ie3.datamodel.io.naming.timeseries.IndividualTimeSeriesMetaInformation +import edu.ie3.datamodel.io.naming.{DatabaseNamingStrategy, EntityPersistenceNamingStrategy, FileNamingStrategy} +import edu.ie3.datamodel.io.source.csv.{CsvTimeSeriesMappingSource, CsvTimeSeriesMetaInformationSource} +import edu.ie3.datamodel.io.source.sql.{SqlTimeSeriesMappingSource, SqlTimeSeriesMetaInformationSource} +import edu.ie3.datamodel.io.source.{TimeSeriesMappingSource, TimeSeriesMetaInformationSource} +import edu.ie3.datamodel.models.value.Value +import edu.ie3.simona.config.SimonaConfig.PrimaryDataCsvParams +import edu.ie3.simona.config.SimonaConfig.Simona.Input.Primary.SqlParams +import edu.ie3.simona.config.SimonaConfig.Simona.Input.{Primary => PrimaryConfig} +import edu.ie3.simona.exceptions.{InitializationException, InvalidConfigParameterException} +import edu.ie3.simona.logging.SimonaActorLogging +import edu.ie3.simona.ontology.messages.Activation +import edu.ie3.simona.ontology.messages.SchedulerMessage.Completion +import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.RegistrationFailedMessage +import edu.ie3.simona.ontology.messages.services.ServiceMessage.{PrimaryServiceRegistrationMessage, WorkerRegistrationMessage} +import edu.ie3.simona.scheduler.ScheduleLock +import edu.ie3.simona.service.{ServiceStateData, SimonaService} +import edu.ie3.simona.service.ServiceStateData.InitializeServiceStateData +import edu.ie3.simona.service.primary.IntPrimaryServiceProxy.{IntPrimaryServiceStateData, SourceRef} +import edu.ie3.simona.service.primary.PrimaryServiceProxy.{InitPrimaryServiceProxyStateData, PrimaryServiceStateData} +import edu.ie3.simona.service.primary.PrimaryServiceWorker.{CsvInitPrimaryServiceStateData, InitPrimaryServiceStateData, SqlInitPrimaryServiceStateData} +import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK + +import java.nio.file.Paths +import java.text.SimpleDateFormat +import java.time.ZonedDateTime +import java.util.UUID +import scala.Option.when +import scala.jdk.CollectionConverters._ +import scala.jdk.OptionConverters.RichOptional +import scala.util.{Failure, Success, Try} + +/** This actor has information on which models can be replaced by precalculated + * (primary) data and how to obtain those time series. It offers possibility to + * register for a certain model. If data is available, a child actor is spun + * of, that will do the actual provision and the requesting agent is informed + * accordingly. + * + * @param scheduler + * Reference to the scheduler of the simulation + * @param startDateTime + * Wall clock time of the first instant in simulation + */ +case class IntPrimaryServiceProxy( + scheduler: ActorRef, + initStateData: InitPrimaryServiceProxyStateData, + private implicit val startDateTime: ZonedDateTime +) extends PrimaryServiceProxy(scheduler, initStateData, startDateTime) { + + + override def onMessage(stateData: PrimaryServiceStateData): Receive = { + stateData match { + case IntPrimaryServiceStateData( + modelToTimeSeries, + timeSeriesToSourceRef, + simulationStart, + primaryConfig, + mappingSource + ) => onMessage(IntPrimaryServiceStateData( + modelToTimeSeries, + timeSeriesToSourceRef, + simulationStart, + primaryConfig, + mappingSource + )) + } + } + + /** Message handling, if the actor has been initialized already. This method + * basically handles registration requests, checks, if pre-calculated, + * primary data is available and forwards the request to worker actors. If + * needed, new workers are spun off. + * + * @param stateData + * Representing the current state of the agent + * @return + * Message handling routine + */ + private def onMessage(stateData: IntPrimaryServiceStateData): Receive = { + case PrimaryServiceRegistrationMessage(modelUuid) => + /* Try to register for this model */ + stateData.modelToTimeSeries.get(modelUuid) match { + case Some(timeSeriesUuid) => + /* There is a time series apparent for this model, try to get a worker for it */ + handleCoveredModel( + modelUuid, + timeSeriesUuid, + stateData, + sender() + ) + case None => + log.debug( + s"There is no time series apparent for the model with uuid '{}'.", + modelUuid + ) + sender() ! RegistrationFailedMessage + } + case x => + log.error( + s"Received message '$x', but I'm only able to handle registration requests." + ) + unhandled(x) + } + + /** Prepare the needed state data by building a + * [[edu.ie3.datamodel.io.source.TimeSeriesMappingSource]], obtain it's + * information and compile them to state data + * + * @param primaryConfig + * Configuration for the primary source + * @param simulationStart + * Wall clock time of first instant in simulation + * @return + * State data, containing the known model and time series identifiers + */ + override def prepareStateData( + primaryConfig: PrimaryConfig, + simulationStart: ZonedDateTime + ): Try[PrimaryServiceStateData] = { + createSources(primaryConfig).map { + case (mappingSource, metaInformationSource) => + val modelToTimeSeries = mappingSource.getMapping.asScala.toMap + val timeSeriesToSourceRef = modelToTimeSeries.values + .to(LazyList) + .distinct + .flatMap { timeSeriesUuid => + metaInformationSource + .getTimeSeriesMetaInformation(timeSeriesUuid) + .toScala match { + case Some(metaInformation) => + /* Only register those entries, that meet the supported column schemes */ + when( + PrimaryServiceWorker.supportedColumnSchemes + .contains(metaInformation.getColumnScheme) + ) { + timeSeriesUuid -> SourceRef(metaInformation, None) + } + case None => + log.warning( + "Unable to acquire meta information for time series '{}'. Leave that out.", + timeSeriesUuid + ) + None + } + } + .toMap + IntPrimaryServiceStateData( + modelToTimeSeries, + timeSeriesToSourceRef, + simulationStart, + primaryConfig, + mappingSource + ) + } + } + + private def createSources( + primaryConfig: PrimaryConfig + ): Try[(TimeSeriesMappingSource, TimeSeriesMetaInformationSource)] = { + Seq( + primaryConfig.sqlParams, + primaryConfig.influxDb1xParams, + primaryConfig.csvParams, + primaryConfig.couchbaseParams + ).filter(_.isDefined).flatten.headOption match { + case Some(PrimaryDataCsvParams(csvSep, directoryPath, _, _)) => + val fileNamingStrategy = new FileNamingStrategy() + Success( + new CsvTimeSeriesMappingSource( + csvSep, + Paths.get(directoryPath), + fileNamingStrategy + ), + new CsvTimeSeriesMetaInformationSource( + csvSep, + Paths.get(directoryPath), + fileNamingStrategy + ) + ) + case Some(sqlParams: SqlParams) => + val sqlConnector = new SqlConnector( + sqlParams.jdbcUrl, + sqlParams.userName, + sqlParams.password + ) + Success( + new SqlTimeSeriesMappingSource( + sqlConnector, + sqlParams.schemaName, + new EntityPersistenceNamingStrategy() + ), + new SqlTimeSeriesMetaInformationSource( + sqlConnector, + sqlParams.schemaName, + new DatabaseNamingStrategy() + ) + ) + case Some(x) => + Failure( + new IllegalArgumentException( + s"Unsupported config for mapping source: '$x'" + ) + ) + case None => + Failure( + new IllegalArgumentException( + "You have to provide exactly one config for the mapping source." + ) + ) + } + } + + + /** Handle the registration request for a covered model. First, try to get a + * already existing worker for this time series, otherwise spin-off a new + * one, remember it and forward the request + * + * @param modelUuid + * Unique identifier of the model + * @param timeSeriesUuid + * Unique identifier of the equivalent time series + * @param stateData + * Current state data of the actor + */ + override protected def handleCoveredModel( + modelUuid: UUID, + timeSeriesUuid: UUID, + stateData: PrimaryServiceStateData, + requestingActor: ActorRef + ): Unit = { + stateData match { + case IntPrimaryServiceStateData(modelToTimeSeries, timeSeriesToSourceRef, simulationStart, primaryConfig, mappingSource) => { + timeSeriesToSourceRef.get(timeSeriesUuid) match { + case Some(SourceRef(_, Some(worker))) => + /* There is yet a worker apparent. Register the requesting actor. The worker will reply to the original + * requesting actor. */ + worker ! WorkerRegistrationMessage(requestingActor) + case Some(SourceRef(metaInformation, None)) => + /* There is NO worker apparent, yet. Spin one off. */ + initializeWorker( + metaInformation, + simulationStart, + primaryConfig + ) match { + case Success(workerRef) => + /* Forward the registration request. The worker will reply about successful registration or not. */ + workerRef ! WorkerRegistrationMessage(requestingActor) + + /* Register the new worker within the state data and change the context */ + context become onMessage( + updateStateData( + IntPrimaryServiceStateData(modelToTimeSeries, timeSeriesToSourceRef, simulationStart, primaryConfig, mappingSource), + timeSeriesUuid, + workerRef) + ) + case Failure(exception) => + log.warning( + s"A failure occurred during spin-off of a primary source for time series '$timeSeriesUuid'. " + + s"Will inform the requesting actor, that registration is not possible.", + exception + ) + requestingActor ! RegistrationFailedMessage + } + + case None => + log.warning( + s"There is no source information for time series '$timeSeriesUuid' (requested for model " + + s"'$modelUuid'), although the mapping contains information about it." + ) + requestingActor ! RegistrationFailedMessage + } + } + case invalidData => Failure( + new InitializationException( + s"Provided init data '${invalidData.getClass.getSimpleName}' for weather service are invalid!" + ) + ) + + + } + } + + /** Instantiate a new [[PrimaryServiceWorker]] and send initialization + * information + * + * @param metaInformation + * Meta information (including column scheme) of the time series + * @param simulationStart + * The time of the simulation start + * @param primaryConfig + * Configuration for the primary config + * @return + * The [[ActorRef]] to the worker + */ + protected def initializeWorker( + metaInformation: IndividualTimeSeriesMetaInformation, + simulationStart: ZonedDateTime, + primaryConfig: PrimaryConfig + ): Try[ActorRef] = { + val workerRef = classToWorkerRef( + metaInformation.getColumnScheme.getValueClass, + metaInformation.getUuid.toString + ) + toInitData( + metaInformation, + simulationStart, + primaryConfig + ) match { + case Success(initData) => + workerRef ! SimonaService.Create( + initData, + ScheduleLock.singleKey(context, scheduler.toTyped, INIT_SIM_TICK) + ) + Success(workerRef) + case Failure(cause) => + workerRef ! PoisonPill + Failure( + new InitializationException( + "Unable to build init data for worker. Kill the uninitialized worker. Goodbye my friend!", + cause + ) + ) + } + } + + /** Register the worker within the state data. + * + * @param stateData + * Current state information + * @param timeSeriesUuid + * Unique identifier of the time series, the worker takes care of + * @param workerRef + * [[ActorRef]] to the new worker actor + * @return + * The updated state data, that holds reference to the worker + */ + private def updateStateData( + stateData: IntPrimaryServiceStateData, + timeSeriesUuid: UUID, + workerRef: ActorRef + ): IntPrimaryServiceStateData = { + val timeSeriesToSourceRef = stateData.timeSeriesToSourceRef + val sourceRef = timeSeriesToSourceRef.getOrElse( + timeSeriesUuid, + throw new IllegalArgumentException( + s"Cannot update entry for time series '$timeSeriesUuid', as it hasn't been part of it before." + ) + ) + val updatedTimeSeriesToSourceRef = timeSeriesToSourceRef.updated( + timeSeriesUuid, + sourceRef.copy(worker = Some(workerRef)) + ) + stateData.copy(timeSeriesToSourceRef = updatedTimeSeriesToSourceRef) + } + + /** Build a primary source worker and type it to the foreseen value class to + * come + * + * @param valueClass + * Class of the values to provide later on + * @param timeSeriesUuid + * uuid of the time series the actor processes + * @tparam V + * Type of the class to provide + * @return + * The [[ActorRef]] to the spun off actor + */ + protected def classToWorkerRef[V <: Value]( + valueClass: Class[V], + timeSeriesUuid: String + ): ActorRef = { + import edu.ie3.simona.actor.SimonaActorNaming._ + context.system.simonaActorOf( + PrimaryServiceWorker.props(scheduler, valueClass), + timeSeriesUuid + ) + } + + /** Building proper init data for the worker + * + * @param metaInformation + * Meta information (including column scheme) of the time series + * @param simulationStart + * The time of the simulation start + * @param primaryConfig + * Configuration for the primary config + * @return + */ + private def toInitData( + metaInformation: IndividualTimeSeriesMetaInformation, + simulationStart: ZonedDateTime, + primaryConfig: PrimaryConfig + ): Try[InitPrimaryServiceStateData] = + primaryConfig match { + case PrimaryConfig( + None, + Some(PrimaryDataCsvParams(csvSep, directoryPath, _, timePattern)), + None, + None + ) => + /* The actual data sources are from csv. Meta information have to match */ + metaInformation match { + case csvMetaData: CsvIndividualTimeSeriesMetaInformation => + Success( + CsvInitPrimaryServiceStateData( + csvMetaData.getUuid, + simulationStart, + csvSep, + Paths.get(directoryPath), + csvMetaData.getFullFilePath, + new FileNamingStrategy(), + timePattern + ) + ) + case invalidMetaData => + Failure( + new InitializationException( + s"Expected '${classOf[CsvIndividualTimeSeriesMetaInformation]}', but got '$invalidMetaData'." + ) + ) + } + + case PrimaryConfig( + None, + None, + None, + Some(sqlParams: SqlParams) + ) => + Success( + SqlInitPrimaryServiceStateData( + metaInformation.getUuid, + simulationStart, + sqlParams, + new DatabaseNamingStrategy() + ) + ) + + case unsupported => + Failure( + new InitializationException( + s"Cannot build initialization data for a worker due to unsupported source config '$unsupported'." + ) + ) + } +} + +object IntPrimaryServiceProxy { + + def props( + scheduler: ActorRef, + initStateData: InitPrimaryServiceProxyStateData, + startDateTime: ZonedDateTime + ): Props = Props( + new IntPrimaryServiceProxy(scheduler, initStateData, startDateTime) + ) + + /** Holding the state of an initialized proxy. + * + * @param modelToTimeSeries + * Mapping from models' to time series unique identifiers + * @param timeSeriesToSourceRef + * Mapping from time series identifier to [[SourceRef]] + * @param simulationStart + * Wall clock time of the first instant in simulation + * @param primaryConfig + * The configuration for the sources + * @param mappingSource + * The mapping source + */ + final case class IntPrimaryServiceStateData( + modelToTimeSeries: Map[UUID, UUID], + timeSeriesToSourceRef: Map[UUID, SourceRef], + simulationStart: ZonedDateTime, + primaryConfig: PrimaryConfig, + mappingSource: TimeSeriesMappingSource + ) extends PrimaryServiceStateData( + simulationStart, primaryConfig + ) + + /** Giving reference to the target time series and source worker. + * + * @param metaInformation + * Meta information (including column scheme) of the time series + * @param worker + * Optional reference to an already existing worker providing information + * on that time series + */ + final case class SourceRef( + metaInformation: IndividualTimeSeriesMetaInformation, + worker: Option[ActorRef] + ) + + /** Check if the config holds correct information to instantiate a mapping + * source + * + * @param primaryConfig + * Config entries for primary source + */ + def checkConfig(primaryConfig: PrimaryConfig): Unit = { + + def checkTimePattern(dtfPattern: String): Unit = + Try { + new SimpleDateFormat(dtfPattern) + } match { + case Failure(exception) => + throw new InvalidConfigParameterException( + s"Invalid timePattern '$dtfPattern' for a time series source. Please provide a valid pattern!" + + s"\nException: $exception" + ) + case Success(_) => + // this is fine + } + + val supportedSources = + Set("csv", "sql") + + val sourceConfigs = Seq( + primaryConfig.couchbaseParams, + primaryConfig.csvParams, + primaryConfig.influxDb1xParams, + primaryConfig.sqlParams + ).filter(_.isDefined).flatten + if (sourceConfigs.size > 1) + throw new InvalidConfigParameterException( + s"${sourceConfigs.size} time series source types defined. " + + s"Please define only one type!\nAvailable types:\n\t${supportedSources.mkString("\n\t")}" + ) + else if (sourceConfigs.isEmpty) + throw new InvalidConfigParameterException( + s"No time series source type defined. Please define exactly one type!" + + s"\nAvailable types:\n\t${supportedSources.mkString("\n\t")}" + ) + else { + sourceConfigs.headOption match { + case Some(csvParams: PrimaryDataCsvParams) => + // note: if inheritance is supported by tscfg, + // the following method should be called for all different supported sources! + checkTimePattern(csvParams.timePattern) + case Some(sqlParams: SqlParams) => + checkTimePattern(sqlParams.timePattern) + case Some(x) => + throw new InvalidConfigParameterException( + s"Invalid configuration '$x' for a time series source.\nAvailable types:\n\t${supportedSources + .mkString("\n\t")}" + ) + case None => + throw new InvalidConfigParameterException( + s"No configuration for a time series mapping source provided.\nPlease provide one of the available sources:\n\t${supportedSources + .mkString("\n\t")}" + ) + } + } + } +} diff --git a/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala b/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala index 95103e347e..17efab5b98 100644 --- a/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala +++ b/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala @@ -1,105 +1,46 @@ -/* - * © 2021. TU Dortmund University, - * Institute of Energy Systems, Energy Efficiency and Energy Economics, - * Research group Distribution grid planning and operation - */ - package edu.ie3.simona.service.primary -import org.apache.pekko.actor.typed.scaladsl.adapter.ClassicActorRefOps -import org.apache.pekko.actor.{Actor, ActorRef, PoisonPill, Props} -import edu.ie3.datamodel.io.connectors.SqlConnector -import edu.ie3.datamodel.io.csv.CsvIndividualTimeSeriesMetaInformation import edu.ie3.datamodel.io.naming.timeseries.IndividualTimeSeriesMetaInformation -import edu.ie3.datamodel.io.naming.{ - DatabaseNamingStrategy, - EntityPersistenceNamingStrategy, - FileNamingStrategy -} -import edu.ie3.datamodel.io.source.csv.{ - CsvTimeSeriesMappingSource, - CsvTimeSeriesMetaInformationSource -} -import edu.ie3.datamodel.io.source.sql.{ - SqlTimeSeriesMappingSource, - SqlTimeSeriesMetaInformationSource -} -import edu.ie3.datamodel.io.source.{ - TimeSeriesMappingSource, - TimeSeriesMetaInformationSource -} -import edu.ie3.datamodel.models.value.Value -import edu.ie3.simona.config.SimonaConfig.PrimaryDataCsvParams -import edu.ie3.simona.config.SimonaConfig.Simona.Input.Primary.SqlParams -import edu.ie3.simona.config.SimonaConfig.Simona.Input.{ - Primary => PrimaryConfig -} -import edu.ie3.simona.exceptions.{ - InitializationException, - InvalidConfigParameterException -} +import edu.ie3.datamodel.io.source.TimeSeriesMappingSource +import edu.ie3.simona.config.SimonaConfig import edu.ie3.simona.logging.SimonaActorLogging import edu.ie3.simona.ontology.messages.Activation import edu.ie3.simona.ontology.messages.SchedulerMessage.Completion import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.RegistrationFailedMessage -import edu.ie3.simona.ontology.messages.services.ServiceMessage.{ - PrimaryServiceRegistrationMessage, - WorkerRegistrationMessage -} -import edu.ie3.simona.scheduler.ScheduleLock -import edu.ie3.simona.service.{ServiceStateData, SimonaService} +import edu.ie3.simona.ontology.messages.services.ServiceMessage.{PrimaryServiceRegistrationMessage, WorkerRegistrationMessage} +import edu.ie3.simona.service.ServiceStateData import edu.ie3.simona.service.ServiceStateData.InitializeServiceStateData -import edu.ie3.simona.service.primary.PrimaryServiceProxy.{ - InitPrimaryServiceProxyStateData, - PrimaryServiceStateData, - SourceRef -} -import edu.ie3.simona.service.primary.PrimaryServiceWorker.{ - CsvInitPrimaryServiceStateData, - InitPrimaryServiceStateData, - SqlInitPrimaryServiceStateData -} +import edu.ie3.simona.service.primary.IntPrimaryServiceProxy.SourceRef +import edu.ie3.simona.service.primary.PrimaryServiceProxy.{InitPrimaryServiceProxyStateData, PrimaryServiceStateData} import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK +import edu.ie3.simona.config.SimonaConfig.Simona.Input.{Primary => PrimaryConfig} +import org.apache.pekko.actor.typed.scaladsl.adapter.ClassicActorRefOps +import org.apache.pekko.actor.{Actor, ActorRef, PoisonPill} -import java.nio.file.Paths -import java.text.SimpleDateFormat import java.time.ZonedDateTime import java.util.UUID -import scala.Option.when -import scala.jdk.CollectionConverters._ -import scala.jdk.OptionConverters.RichOptional import scala.util.{Failure, Success, Try} -/** This actor has information on which models can be replaced by precalculated - * (primary) data and how to obtain those time series. It offers possibility to - * register for a certain model. If data is available, a child actor is spun - * of, that will do the actual provision and the requesting agent is informed - * accordingly. - * - * @param scheduler - * Reference to the scheduler of the simulation - * @param startDateTime - * Wall clock time of the first instant in simulation - */ -case class PrimaryServiceProxy( - scheduler: ActorRef, - initStateData: InitPrimaryServiceProxyStateData, - private implicit val startDateTime: ZonedDateTime -) extends Actor - with SimonaActorLogging { +abstract class PrimaryServiceProxy( + scheduler: ActorRef, + initStateData: InitPrimaryServiceProxyStateData, + private implicit val startDateTime: ZonedDateTime + ) extends Actor + with SimonaActorLogging { + /** Start receiving without knowing specifics about myself - * - * @return - * How receiving should be handled - */ + * + * @return + * How receiving should be handled + */ override def receive: Receive = uninitialized /** Handle all messages, when the actor isn't initialized, yet. - * - * @return - * How receiving should be handled with gained insight of myself - */ + * + * @return + * How receiving should be handled with gained insight of myself + */ private def uninitialized: Receive = { case Activation(INIT_SIM_TICK) => /* The proxy is asked to initialize itself. If that happened successfully, change the logic of receiving @@ -125,486 +66,53 @@ case class PrimaryServiceProxy( unhandled(x) } - /** Prepare the needed state data by building a - * [[edu.ie3.datamodel.io.source.TimeSeriesMappingSource]], obtain it's - * information and compile them to state data - * - * @param primaryConfig - * Configuration for the primary source - * @param simulationStart - * Wall clock time of first instant in simulation - * @return - * State data, containing the known model and time series identifiers - */ - private def prepareStateData( - primaryConfig: PrimaryConfig, - simulationStart: ZonedDateTime - ): Try[PrimaryServiceStateData] = { - createSources(primaryConfig).map { - case (mappingSource, metaInformationSource) => - val modelToTimeSeries = mappingSource.getMapping.asScala.toMap - val timeSeriesToSourceRef = modelToTimeSeries.values - .to(LazyList) - .distinct - .flatMap { timeSeriesUuid => - metaInformationSource - .getTimeSeriesMetaInformation(timeSeriesUuid) - .toScala match { - case Some(metaInformation) => - /* Only register those entries, that meet the supported column schemes */ - when( - PrimaryServiceWorker.supportedColumnSchemes - .contains(metaInformation.getColumnScheme) - ) { - timeSeriesUuid -> SourceRef(metaInformation, None) - } - case None => - log.warning( - "Unable to acquire meta information for time series '{}'. Leave that out.", - timeSeriesUuid - ) - None - } - } - .toMap - PrimaryServiceStateData( - modelToTimeSeries, - timeSeriesToSourceRef, - simulationStart, - primaryConfig, - mappingSource - ) - } - } + def onMessage(stateData: PrimaryServiceStateData): Receive - private def createSources( - primaryConfig: PrimaryConfig - ): Try[(TimeSeriesMappingSource, TimeSeriesMetaInformationSource)] = { - Seq( - primaryConfig.sqlParams, - primaryConfig.influxDb1xParams, - primaryConfig.csvParams, - primaryConfig.couchbaseParams - ).filter(_.isDefined).flatten.headOption match { - case Some(PrimaryDataCsvParams(csvSep, directoryPath, _, _)) => - val fileNamingStrategy = new FileNamingStrategy() - Success( - new CsvTimeSeriesMappingSource( - csvSep, - Paths.get(directoryPath), - fileNamingStrategy - ), - new CsvTimeSeriesMetaInformationSource( - csvSep, - Paths.get(directoryPath), - fileNamingStrategy - ) - ) - case Some(sqlParams: SqlParams) => - val sqlConnector = new SqlConnector( - sqlParams.jdbcUrl, - sqlParams.userName, - sqlParams.password - ) - Success( - new SqlTimeSeriesMappingSource( - sqlConnector, - sqlParams.schemaName, - new EntityPersistenceNamingStrategy() - ), - new SqlTimeSeriesMetaInformationSource( - sqlConnector, - sqlParams.schemaName, - new DatabaseNamingStrategy() - ) - ) - case Some(x) => - Failure( - new IllegalArgumentException( - s"Unsupported config for mapping source: '$x'" - ) - ) - case None => - Failure( - new IllegalArgumentException( - "You have to provide exactly one config for the mapping source." - ) - ) - } - } + def askForExternalSimulation( - /** Message handling, if the actor has been initialized already. This method - * basically handles registration requests, checks, if pre-calculated, - * primary data is available and forwards the request to worker actors. If - * needed, new workers are spun off. - * - * @param stateData - * Representing the current state of the agent - * @return - * Message handling routine - */ - private def onMessage(stateData: PrimaryServiceStateData): Receive = { - case PrimaryServiceRegistrationMessage(modelUuid) => - /* Try to register for this model */ - stateData.modelToTimeSeries.get(modelUuid) match { - case Some(timeSeriesUuid) => - /* There is a time series apparent for this model, try to get a worker for it */ - handleCoveredModel( - modelUuid, - timeSeriesUuid, - stateData, - sender() - ) - case None => - log.debug( - s"There is no time series apparent for the model with uuid '{}'.", - modelUuid - ) - sender() ! RegistrationFailedMessage - } - case x => - log.error( - s"Received message '$x', but I'm only able to handle registration requests." - ) - unhandled(x) - } + ) - /** Handle the registration request for a covered model. First, try to get a - * already existing worker for this time series, otherwise spin-off a new - * one, remember it and forward the request - * - * @param modelUuid - * Unique identifier of the model - * @param timeSeriesUuid - * Unique identifier of the equivalent time series - * @param stateData - * Current state data of the actor - */ - protected def handleCoveredModel( - modelUuid: UUID, - timeSeriesUuid: UUID, - stateData: PrimaryServiceStateData, - requestingActor: ActorRef - ): Unit = { - val timeSeriesToSourceRef = stateData.timeSeriesToSourceRef - timeSeriesToSourceRef.get(timeSeriesUuid) match { - case Some(SourceRef(_, Some(worker))) => - /* There is yet a worker apparent. Register the requesting actor. The worker will reply to the original - * requesting actor. */ - worker ! WorkerRegistrationMessage(requestingActor) - case Some(SourceRef(metaInformation, None)) => - /* There is NO worker apparent, yet. Spin one off. */ - initializeWorker( - metaInformation, - stateData.simulationStart, - stateData.primaryConfig - ) match { - case Success(workerRef) => - /* Forward the registration request. The worker will reply about successful registration or not. */ - workerRef ! WorkerRegistrationMessage(requestingActor) + def prepareStateData( + primaryConfig: PrimaryConfig, + time: ZonedDateTime): Try[PrimaryServiceStateData] - /* Register the new worker within the state data and change the context */ - context become onMessage( - updateStateData(stateData, timeSeriesUuid, workerRef) - ) - case Failure(exception) => - log.warning( - s"A failure occurred during spin-off of a primary source for time series '$timeSeriesUuid'. " + - s"Will inform the requesting actor, that registration is not possible.", - exception - ) - requestingActor ! RegistrationFailedMessage - } - case None => - log.warning( - s"There is no source information for time series '$timeSeriesUuid' (requested for model " + - s"'$modelUuid'), although the mapping contains information about it." - ) - requestingActor ! RegistrationFailedMessage - } - } - - /** Instantiate a new [[PrimaryServiceWorker]] and send initialization - * information - * - * @param metaInformation - * Meta information (including column scheme) of the time series - * @param simulationStart - * The time of the simulation start - * @param primaryConfig - * Configuration for the primary config - * @return - * The [[ActorRef]] to the worker - */ - protected def initializeWorker( - metaInformation: IndividualTimeSeriesMetaInformation, - simulationStart: ZonedDateTime, - primaryConfig: PrimaryConfig - ): Try[ActorRef] = { - val workerRef = classToWorkerRef( - metaInformation.getColumnScheme.getValueClass, - metaInformation.getUuid.toString - ) - toInitData( - metaInformation, - simulationStart, - primaryConfig - ) match { - case Success(initData) => - workerRef ! SimonaService.Create( - initData, - ScheduleLock.singleKey(context, scheduler.toTyped, INIT_SIM_TICK) - ) - Success(workerRef) - case Failure(cause) => - workerRef ! PoisonPill - Failure( - new InitializationException( - "Unable to build init data for worker. Kill the uninitialized worker. Goodbye my friend!", - cause - ) - ) - } - } - - /** Build a primary source worker and type it to the foreseen value class to - * come - * - * @param valueClass - * Class of the values to provide later on - * @param timeSeriesUuid - * uuid of the time series the actor processes - * @tparam V - * Type of the class to provide - * @return - * The [[ActorRef]] to the spun off actor - */ - protected def classToWorkerRef[V <: Value]( - valueClass: Class[V], - timeSeriesUuid: String - ): ActorRef = { - import edu.ie3.simona.actor.SimonaActorNaming._ - context.system.simonaActorOf( - PrimaryServiceWorker.props(scheduler, valueClass), - timeSeriesUuid - ) - } - - /** Building proper init data for the worker - * - * @param metaInformation - * Meta information (including column scheme) of the time series - * @param simulationStart - * The time of the simulation start - * @param primaryConfig - * Configuration for the primary config - * @return - */ - private def toInitData( - metaInformation: IndividualTimeSeriesMetaInformation, - simulationStart: ZonedDateTime, - primaryConfig: PrimaryConfig - ): Try[InitPrimaryServiceStateData] = - primaryConfig match { - case PrimaryConfig( - None, - Some(PrimaryDataCsvParams(csvSep, directoryPath, _, timePattern)), - None, - None - ) => - /* The actual data sources are from csv. Meta information have to match */ - metaInformation match { - case csvMetaData: CsvIndividualTimeSeriesMetaInformation => - Success( - CsvInitPrimaryServiceStateData( - csvMetaData.getUuid, - simulationStart, - csvSep, - Paths.get(directoryPath), - csvMetaData.getFullFilePath, - new FileNamingStrategy(), - timePattern - ) - ) - case invalidMetaData => - Failure( - new InitializationException( - s"Expected '${classOf[CsvIndividualTimeSeriesMetaInformation]}', but got '$invalidMetaData'." - ) - ) - } - - case PrimaryConfig( - None, - None, - None, - Some(sqlParams: SqlParams) - ) => - Success( - SqlInitPrimaryServiceStateData( - metaInformation.getUuid, - simulationStart, - sqlParams, - new DatabaseNamingStrategy() - ) - ) - - case unsupported => - Failure( - new InitializationException( - s"Cannot build initialization data for a worker due to unsupported source config '$unsupported'." - ) - ) - } - - /** Register the worker within the state data. - * - * @param stateData - * Current state information - * @param timeSeriesUuid - * Unique identifier of the time series, the worker takes care of - * @param workerRef - * [[ActorRef]] to the new worker actor - * @return - * The updated state data, that holds reference to the worker - */ - private def updateStateData( - stateData: PrimaryServiceStateData, - timeSeriesUuid: UUID, - workerRef: ActorRef - ): PrimaryServiceStateData = { - val timeSeriesToSourceRef = stateData.timeSeriesToSourceRef - val sourceRef = timeSeriesToSourceRef.getOrElse( - timeSeriesUuid, - throw new IllegalArgumentException( - s"Cannot update entry for time series '$timeSeriesUuid', as it hasn't been part of it before." - ) - ) - val updatedTimeSeriesToSourceRef = timeSeriesToSourceRef.updated( - timeSeriesUuid, - sourceRef.copy(worker = Some(workerRef)) - ) - stateData.copy(timeSeriesToSourceRef = updatedTimeSeriesToSourceRef) - } + /** Handle the registration request for a covered model. First, try to get a + * already existing worker for this time series, otherwise spin-off a new + * one, remember it and forward the request + * + * @param modelUuid + * Unique identifier of the model + * @param timeSeriesUuid + * Unique identifier of the equivalent time series + * @param stateData + * Current state data of the actor + */ + protected def handleCoveredModel( + modelUuid: UUID, + timeSeriesUuid: UUID, + stateData: PrimaryServiceStateData, + requestingActor: ActorRef + ): Unit } object PrimaryServiceProxy { - - def props( - scheduler: ActorRef, - initStateData: InitPrimaryServiceProxyStateData, - startDateTime: ZonedDateTime - ): Props = Props( - new PrimaryServiceProxy(scheduler, initStateData, startDateTime) - ) - /** State data with needed information to initialize this primary service - * provider proxy - * - * @param primaryConfig - * Configuration for the primary source - * @param simulationStart - * Wall clock time of the first instant in simulation - */ + * provider proxy + * + * @param primaryConfig + * Configuration for the primary source + * @param simulationStart + * Wall clock time of the first instant in simulation + */ final case class InitPrimaryServiceProxyStateData( - primaryConfig: PrimaryConfig, - simulationStart: ZonedDateTime - ) extends InitializeServiceStateData - - /** Holding the state of an initialized proxy. - * - * @param modelToTimeSeries - * Mapping from models' to time series unique identifiers - * @param timeSeriesToSourceRef - * Mapping from time series identifier to [[SourceRef]] - * @param simulationStart - * Wall clock time of the first instant in simulation - * @param primaryConfig - * The configuration for the sources - * @param mappingSource - * The mapping source - */ - final case class PrimaryServiceStateData( - modelToTimeSeries: Map[UUID, UUID], - timeSeriesToSourceRef: Map[UUID, SourceRef], - simulationStart: ZonedDateTime, - primaryConfig: PrimaryConfig, - mappingSource: TimeSeriesMappingSource - ) extends ServiceStateData - - /** Giving reference to the target time series and source worker. - * - * @param metaInformation - * Meta information (including column scheme) of the time series - * @param worker - * Optional reference to an already existing worker providing information - * on that time series - */ - final case class SourceRef( - metaInformation: IndividualTimeSeriesMetaInformation, - worker: Option[ActorRef] - ) + primaryConfig: PrimaryConfig, + simulationStart: ZonedDateTime + ) extends InitializeServiceStateData - /** Check if the config holds correct information to instantiate a mapping - * source - * - * @param primaryConfig - * Config entries for primary source - */ - def checkConfig(primaryConfig: PrimaryConfig): Unit = { + abstract class PrimaryServiceStateData( + simulationStart: ZonedDateTime, + primaryConfig: PrimaryConfig + ) extends ServiceStateData - def checkTimePattern(dtfPattern: String): Unit = - Try { - new SimpleDateFormat(dtfPattern) - } match { - case Failure(exception) => - throw new InvalidConfigParameterException( - s"Invalid timePattern '$dtfPattern' for a time series source. Please provide a valid pattern!" + - s"\nException: $exception" - ) - case Success(_) => - // this is fine - } - - val supportedSources = - Set("csv", "sql") - - val sourceConfigs = Seq( - primaryConfig.couchbaseParams, - primaryConfig.csvParams, - primaryConfig.influxDb1xParams, - primaryConfig.sqlParams - ).filter(_.isDefined).flatten - if (sourceConfigs.size > 1) - throw new InvalidConfigParameterException( - s"${sourceConfigs.size} time series source types defined. " + - s"Please define only one type!\nAvailable types:\n\t${supportedSources.mkString("\n\t")}" - ) - else if (sourceConfigs.isEmpty) - throw new InvalidConfigParameterException( - s"No time series source type defined. Please define exactly one type!" + - s"\nAvailable types:\n\t${supportedSources.mkString("\n\t")}" - ) - else { - sourceConfigs.headOption match { - case Some(csvParams: PrimaryDataCsvParams) => - // note: if inheritance is supported by tscfg, - // the following method should be called for all different supported sources! - checkTimePattern(csvParams.timePattern) - case Some(sqlParams: SqlParams) => - checkTimePattern(sqlParams.timePattern) - case Some(x) => - throw new InvalidConfigParameterException( - s"Invalid configuration '$x' for a time series source.\nAvailable types:\n\t${supportedSources - .mkString("\n\t")}" - ) - case None => - throw new InvalidConfigParameterException( - s"No configuration for a time series mapping source provided.\nPlease provide one of the available sources:\n\t${supportedSources - .mkString("\n\t")}" - ) - } - } - } } diff --git a/src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala b/src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala new file mode 100644 index 0000000000..132d4f59ae --- /dev/null +++ b/src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala @@ -0,0 +1,135 @@ +package edu.ie3.simona.service.results + +import edu.ie3.simona.api.data.ontology.DataMessageFromExt +import edu.ie3.simona.api.data.results.ExtResultsData +import edu.ie3.simona.api.data.results.ontology.{ProvideResultEntities, RequestResultEntities, ResultDataMessageFromExt} +import edu.ie3.simona.exceptions.{InitializationException, ServiceException} +import edu.ie3.simona.ontology.messages.services.ResultMessage.ResultResponseMessage +import edu.ie3.simona.ontology.messages.services.ServiceMessage.ServiceRegistrationMessage +import edu.ie3.simona.ontology.messages.services.DataMessage +import edu.ie3.simona.service.ServiceStateData.{InitializeServiceStateData, ServiceBaseStateData} +import edu.ie3.simona.service.{ExtDataSupport, SimonaService} +import edu.ie3.simona.service.results.ExtResultDataService.{ExtResultsStateData, InitExtResultsData} +import org.apache.pekko.actor.{ActorContext, ActorRef, Props} + +import java.util.UUID +import scala.jdk.CollectionConverters.SeqHasAsJava +import scala.util.{Failure, Success, Try} + +object ExtResultDataService { + def props(scheduler: ActorRef): Props = + Props( + new ExtResultDataService(scheduler: ActorRef) + ) + + final case class ExtResultsStateData( + extResultsData: ExtResultsData, + uuidToActorRef: Map[UUID, ActorRef] = Map.empty[UUID, ActorRef], + extResultsMessage: Option[ResultDataMessageFromExt] = None + ) extends ServiceBaseStateData + + final case class InitExtResultsData( + extResultsData: ExtResultsData + ) extends InitializeServiceStateData +} + + +class ExtResultDataService(override val scheduler: ActorRef) + extends SimonaService[ExtResultsStateData](scheduler) + with ExtDataSupport[ExtResultsStateData] { + + override def init( + initServiceData: InitializeServiceStateData + ): Try[(ExtResultsStateData, Option[Long])] = { + initServiceData match { + case InitExtResultsData(extResultsData) => + val resultInitializedStateData = ExtResultsStateData(extResultsData) + Success(resultInitializedStateData, None) + + case invalidData => + Failure( + new InitializationException( + s"Provided init data '${invalidData.getClass.getSimpleName}' for EV service are invalid!" + ) + ) + } + } + + override protected def handleRegistrationRequest( + registrationMessage: ServiceRegistrationMessage + )(implicit serviceStateData: ExtResultsStateData): + Try[ExtResultsStateData] = + Failure( + ServiceException( + "For this service is no registration possible!" + ) + ) + + override protected def announceInformation( + tick: Long + )( + implicit serviceStateData: ExtResultsStateData, + ctx: ActorContext): (ExtResultsStateData, Option[Long]) = { + + serviceStateData.extResultsMessage.getOrElse( + throw ServiceException( + "ExtEvDataService was triggered without ExtEvMessage available" + ) + ) match { + case _: RequestResultEntities => + requestResults(tick) + } + (null, None) // No Annoucement + } + + /** Handle a message from outside the simulation + * + * @param extMsg + * the external incoming message + * @param serviceStateData + * the current state data of this service + * @return + * the updated state data + */ + override protected def handleDataMessage( + extMsg: DataMessageFromExt + )( + implicit serviceStateData: ExtResultsStateData + ): ExtResultsStateData = { + serviceStateData + } + + /** Handle a message from inside SIMONA sent to external + * + * @param extResponseMsg + * the external incoming message + * @param serviceStateData + * the current state data of this service + * @return + * the updated state data + */ + override protected def handleDataResponseMessage( + extResponseMsg: DataMessage + )( + implicit serviceStateData: ExtResultsStateData): ExtResultsStateData = { + extResponseMsg match { + case ResultResponseMessage(results) => + serviceStateData.extResultsData.queueExtResponseMsg( + new ProvideResultEntities(results.toList.asJava)) + serviceStateData.copy() + } + } + + private def requestResults( + tick: Long + )(implicit serviceStateData: ExtResultsStateData): (ExtResultsStateData, Option[Long]) = { + + (serviceStateData.copy(), None) + } + + + + + + +} diff --git a/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala b/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala index 7a619552eb..6ccd385a6f 100644 --- a/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala +++ b/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala @@ -8,31 +8,18 @@ package edu.ie3.simona.sim import org.apache.pekko.actor.typed.scaladsl.adapter.TypedActorRefOps import org.apache.pekko.actor.SupervisorStrategy.Stop -import org.apache.pekko.actor.{ - Actor, - ActorRef, - AllForOneStrategy, - Props, - Stash, - SupervisorStrategy, - Terminated -} +import org.apache.pekko.actor.{Actor, ActorRef, AllForOneStrategy, Props, Stash, SupervisorStrategy, Terminated} import com.typesafe.scalalogging.LazyLogging import edu.ie3.simona.agent.EnvironmentRefs -import edu.ie3.simona.event.RuntimeEvent +import edu.ie3.simona.event.ResultEvent.PowerFlowResultEvent +import edu.ie3.simona.event.{ResultEvent, RuntimeEvent} +import edu.ie3.simona.ontology.messages.SchedulerMessage.ResultForExtCompleteMessage import edu.ie3.simona.ontology.messages.StopMessage +import edu.ie3.simona.ontology.messages.services.ResultMessage.ResultResponseMessage import edu.ie3.simona.scheduler.TimeAdvancer import edu.ie3.simona.scheduler.TimeAdvancer.StartSimMessage -import edu.ie3.simona.sim.SimMessage.{ - InitSim, - SimulationFailure, - SimulationSuccessful, - StartSimulation -} -import edu.ie3.simona.sim.SimonaSim.{ - EmergencyShutdownInitiated, - SimonaSimStateData -} +import edu.ie3.simona.sim.SimMessage.{InitSim, SimulationFailure, SimulationSuccessful, StartSimulation} +import edu.ie3.simona.sim.SimonaSim.{EmergencyShutdownInitiated, PowerFlowResultMessage, ResultMessage, SimonaSimStateData} import edu.ie3.simona.sim.setup.{ExtSimSetupData, SimonaSetup} import scala.concurrent.duration.DurationInt @@ -85,16 +72,18 @@ class SimonaSim(simonaSetup: SimonaSetup) val scheduler: ActorRef = simonaSetup.scheduler(context, timeAdvancer) /* start services */ + // Load external simulations + val extSimulationData: ExtSimSetupData = + simonaSetup.extSimulations(context, scheduler) + // primary service proxy val primaryServiceProxy: ActorRef = - simonaSetup.primaryServiceProxy(context, scheduler) + simonaSetup.primaryServiceProxy(context, scheduler, extSimulationData) // weather service val weatherService: ActorRef = simonaSetup.weatherService(context, scheduler) - val extSimulationData: ExtSimSetupData = - simonaSetup.extSimulations(context, scheduler) /* start grid agents */ val gridAgents: Iterable[ActorRef] = simonaSetup.gridAgents( @@ -154,6 +143,14 @@ class SimonaSim(simonaSetup: SimonaSetup) systemParticipantsListener ) + /* ExtSimulation */ + case ResultMessage(res) => + data.initSimSender ! ( + res.getResults() + ) + + + case EmergencyShutdownInitiated => logger.debug( "Simulation guardian is aware, that emergency shutdown has been initiated. Inform the init sender." @@ -269,6 +266,8 @@ object SimonaSim { */ case object EmergencyShutdownInitiated + final case class ResultMessage(results: ResultEvent) + private[SimonaSim] final case class SimonaSimStateData( initSimSender: ActorRef = ActorRef.noSender ) 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 4706a50f09..c325a828dd 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/ExtSimSetupData.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/ExtSimSetupData.scala @@ -8,6 +8,7 @@ package edu.ie3.simona.sim.setup import org.apache.pekko.actor.ActorRef import edu.ie3.simona.service.ev.ExtEvDataService +import edu.ie3.simona.service.primary.{ExtPrimaryDataService, ExtPrimaryServiceWorker} final case class ExtSimSetupData( extSimAdapters: Iterable[ActorRef], @@ -16,4 +17,7 @@ final case class ExtSimSetupData( def evDataService: Option[ActorRef] = extDataServices.get(classOf[ExtEvDataService]) + + def extPrimaryDataService: Option[ActorRef] = extDataServices.get(classOf[ExtPrimaryDataService]) + } 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 caad2d1b7b..618a9521f1 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaSetup.scala @@ -70,7 +70,8 @@ trait SimonaSetup { */ def primaryServiceProxy( context: ActorContext, - scheduler: ActorRef + scheduler: ActorRef, + extSimulationData: ExtSimSetupData ): ActorRef /** Creates a weather service 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 246fda0101..b2b3398eeb 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala @@ -6,11 +6,7 @@ package edu.ie3.simona.sim.setup -import org.apache.pekko.actor.typed.scaladsl.adapter.{ - ClassicActorContextOps, - ClassicActorRefOps, - TypedActorRefOps -} +import org.apache.pekko.actor.typed.scaladsl.adapter.{ClassicActorContextOps, ClassicActorRefOps, TypedActorRefOps} import org.apache.pekko.actor.{ActorContext, ActorRef, ActorSystem} import com.typesafe.config.Config import com.typesafe.scalalogging.LazyLogging @@ -23,6 +19,8 @@ import edu.ie3.simona.agent.grid.GridAgent import edu.ie3.simona.api.ExtSimAdapter import edu.ie3.simona.api.data.ExtData import edu.ie3.simona.api.data.ev.{ExtEvData, ExtEvSimulation} +import edu.ie3.simona.api.data.primarydata.{ExtPrimaryData, ExtPrimaryDataSimulation} +import edu.ie3.simona.api.data.results.{ExtResultDataSimulation, ExtResultsData} import edu.ie3.simona.api.simulation.ExtSimAdapterData import edu.ie3.simona.config.{ArgsParser, RefSystemParser, SimonaConfig} import edu.ie3.simona.event.RuntimeEvent @@ -34,8 +32,11 @@ import edu.ie3.simona.scheduler.{ScheduleLock, Scheduler, TimeAdvancer} import edu.ie3.simona.service.SimonaService import edu.ie3.simona.service.ev.ExtEvDataService import edu.ie3.simona.service.ev.ExtEvDataService.InitExtEvData -import edu.ie3.simona.service.primary.PrimaryServiceProxy +import edu.ie3.simona.service.primary.ExtPrimaryServiceWorker.InitExtPrimaryData +import edu.ie3.simona.service.primary.{ExtPrimaryServiceWorker, IntPrimaryServiceProxy, PrimaryServiceProxy} import edu.ie3.simona.service.primary.PrimaryServiceProxy.InitPrimaryServiceProxyStateData +import edu.ie3.simona.service.results.ExtResultDataService +import edu.ie3.simona.service.results.ExtResultDataService.InitExtResultsData import edu.ie3.simona.service.weather.WeatherService import edu.ie3.simona.service.weather.WeatherService.InitWeatherServiceStateData import edu.ie3.simona.util.ResultFileHierarchy @@ -137,13 +138,14 @@ class SimonaStandaloneSetup( override def primaryServiceProxy( context: ActorContext, - scheduler: ActorRef + scheduler: ActorRef, + extSimSetupData: ExtSimSetupData ): ActorRef = { val simulationStart = TimeUtil.withDefaults.toZonedDateTime( simonaConfig.simona.time.startDateTime ) val primaryServiceProxy = context.simonaActorOf( - PrimaryServiceProxy.props( + IntPrimaryServiceProxy.props( scheduler, InitPrimaryServiceProxyStateData( simonaConfig.simona.input.primary, @@ -226,6 +228,42 @@ class SimonaStandaloneSetup( ) (extEvData, (classOf[ExtEvDataService], extEvDataService)) + + case (_: ExtPrimaryDataSimulation, dIndex) => + val extPrimaryDataService = context.simonaActorOf( + ExtPrimaryServiceWorker.props(scheduler), + s"$index-$dIndex" + ) + val extPrimaryData = new ExtPrimaryData(extPrimaryDataService, extSimAdapter) + + extPrimaryDataService ! SimonaService.Create( + InitExtPrimaryData(extPrimaryData), + ScheduleLock.singleKey( + context, + scheduler.toTyped, + INIT_SIM_TICK + ) + ) + + (null, (classOf[ExtResultDataService], extPrimaryDataService)) + + case (_: ExtResultDataSimulation, dIndex) => + val extResultDataService = context.simonaActorOf( + ExtResultDataService.props(scheduler), + s"$index-$dIndex" + ) + val extResultsData = new ExtResultsData(extResultDataService, extSimAdapter) + + extResultDataService ! SimonaService.Create( + InitExtResultsData(extResultsData), + ScheduleLock.singleKey( + context, + scheduler.toTyped, + INIT_SIM_TICK + ) + ) + + (extResultsData, (classOf[ExtResultDataService], extResultDataService)) }.unzip extLink.getExtSimulation.setup( From 5765cdc5a1501be92f1cfde7d0ff47416612477a Mon Sep 17 00:00:00 2001 From: Johannes Bao Date: Wed, 17 Jan 2024 14:52:01 +0100 Subject: [PATCH 02/41] Implement ExtPrimaryDataService --- .../statedata/ParticipantStateData.scala | 10 +- .../edu/ie3/simona/event/ResultEvent.scala | 2 +- .../messages/services/ServiceMessage.scala | 19 + .../primary/ExtPrimaryDataService.scala | 220 +++++-- .../primary/ExtPrimaryServiceProxy.scala | 99 --- .../primary/ExtPrimaryServiceWorker.scala | 218 ------- .../ExtTimeSeriesSubscribersSource.scala | 9 + .../primary/IntPrimaryServiceProxy.scala | 566 ------------------ .../service/primary/PrimaryServiceProxy.scala | 558 ++++++++++++++++- .../scala/edu/ie3/simona/sim/SimonaSim.scala | 8 +- .../simona/sim/setup/ExtSimSetupData.scala | 2 +- .../sim/setup/SimonaStandaloneSetup.scala | 11 +- .../primary/PrimaryServiceProxySpec.scala | 11 +- .../primary/PrimaryServiceProxySqlIT.scala | 3 +- .../ie3/simona/sim/SimonaSimFailSpec.scala | 3 +- .../simona/sim/setup/SimonaSetupSpec.scala | 3 +- 16 files changed, 757 insertions(+), 985 deletions(-) delete mode 100644 src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryServiceProxy.scala delete mode 100644 src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryServiceWorker.scala create mode 100644 src/main/scala/edu/ie3/simona/service/primary/ExtTimeSeriesSubscribersSource.scala delete mode 100644 src/main/scala/edu/ie3/simona/service/primary/IntPrimaryServiceProxy.scala diff --git a/src/main/scala/edu/ie3/simona/agent/participant/statedata/ParticipantStateData.scala b/src/main/scala/edu/ie3/simona/agent/participant/statedata/ParticipantStateData.scala index 163b0a8151..d52896da0c 100644 --- a/src/main/scala/edu/ie3/simona/agent/participant/statedata/ParticipantStateData.scala +++ b/src/main/scala/edu/ie3/simona/agent/participant/statedata/ParticipantStateData.scala @@ -110,7 +110,7 @@ object ParticipantStateData { ]( inputModel: InputModelContainer[I], modelConfig: C, - primaryServiceProxies: Option[Vector[ActorRef]], + primaryServiceProxy: ActorRef, secondaryDataServices: Option[ Vector[SecondaryDataService[_ <: SecondaryData]] ], @@ -129,7 +129,7 @@ object ParticipantStateData { ]( inputModel: I, modelConfig: C, - primaryServiceProxies: Option[Vector[ActorRef]], + primaryServiceProxy: ActorRef, secondaryDataServices: Option[ Vector[SecondaryDataService[_ <: SecondaryData]] ], @@ -142,7 +142,7 @@ object ParticipantStateData { new ParticipantInitializeStateData[I, C, PD]( SimpleInputContainer(inputModel), modelConfig, - primaryServiceProxies, + primaryServiceProxy, secondaryDataServices, simulationStartDate, simulationEndDate, @@ -159,7 +159,7 @@ object ParticipantStateData { inputModel: I, thermalGrid: ThermalGrid, modelConfig: C, - primaryServiceProxies: Option[Vector[ActorRef]], + primaryServiceProxy: ActorRef, secondaryDataServices: Option[ Vector[SecondaryDataService[_ <: SecondaryData]] ], @@ -172,7 +172,7 @@ object ParticipantStateData { new ParticipantInitializeStateData[I, C, PD]( WithHeatInputContainer(inputModel, thermalGrid), modelConfig, - primaryServiceProxies, + primaryServiceProxy, secondaryDataServices, simulationStartDate, simulationEndDate, diff --git a/src/main/scala/edu/ie3/simona/event/ResultEvent.scala b/src/main/scala/edu/ie3/simona/event/ResultEvent.scala index e0198883d2..7e036a7d68 100644 --- a/src/main/scala/edu/ie3/simona/event/ResultEvent.scala +++ b/src/main/scala/edu/ie3/simona/event/ResultEvent.scala @@ -73,7 +73,7 @@ object ResultEvent { results = results ++ switchResults results = results ++ lineResults results = results ++ transformer2wResults - results = results ++ transformer3wResults + // results = results ++ transformer3wResults results } } diff --git a/src/main/scala/edu/ie3/simona/ontology/messages/services/ServiceMessage.scala b/src/main/scala/edu/ie3/simona/ontology/messages/services/ServiceMessage.scala index 6d31cd8a8a..af832bfc9f 100644 --- a/src/main/scala/edu/ie3/simona/ontology/messages/services/ServiceMessage.scala +++ b/src/main/scala/edu/ie3/simona/ontology/messages/services/ServiceMessage.scala @@ -10,6 +10,7 @@ import org.apache.pekko.actor.ActorRef import java.util.UUID import edu.ie3.simona.agent.participant.data.Data +import edu.ie3.simona.ontology.messages.services import edu.ie3.simona.scheduler.ScheduleLock.ScheduleKey /** Collections of all messages, that are send to and from the different @@ -41,6 +42,11 @@ case object ServiceMessage { final case class WorkerRegistrationMessage(requestingActor: ActorRef) extends ServiceRegistrationMessage + final case class ExtPrimaryDataServiceRegistrationMessage( + modelUuid: UUID, + requestingActor: ActorRef + ) extends ServiceRegistrationMessage + sealed trait RegistrationResponseMessage extends ServiceMessage case object RegistrationResponseMessage { @@ -71,4 +77,17 @@ case object ServiceMessage { val nextDataTick: Option[Long] val unlockKey: Option[ScheduleKey] } + + // -=- Subscriber of extPrimaryData -=- + trait ExtSubscriberMessage extends ServiceMessage + + final case class RequestExtSubscriberMessage() extends ExtSubscriberMessage + + final case class ProvideExtSubscriberMessage() extends ExtSubscriberMessage + + + + + + } diff --git a/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala b/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala index 6d7d8af2cc..6152d5c409 100644 --- a/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala +++ b/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala @@ -1,12 +1,26 @@ package edu.ie3.simona.service.primary +import edu.ie3.datamodel.models.value.Value +import edu.ie3.simona.agent.participant.data.Data.PrimaryData.RichValue import edu.ie3.simona.api.data.ontology.DataMessageFromExt +import edu.ie3.simona.api.data.primarydata.ExtPrimaryData +import edu.ie3.simona.api.data.primarydata.ontology.{PrimaryDataMessageFromExt, ProvidePrimaryData} +import edu.ie3.simona.exceptions.WeatherServiceException.InvalidRegistrationRequestException +import edu.ie3.simona.exceptions.{InitializationException, ServiceException} +import edu.ie3.simona.ontology.messages.services.ServiceMessage.{ExtPrimaryDataServiceRegistrationMessage, PrimaryServiceRegistrationMessage, WorkerRegistrationMessage} +import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.RegistrationSuccessfulMessage import edu.ie3.simona.ontology.messages.services.{DataMessage, ServiceMessage} +import edu.ie3.simona.scheduler.ScheduleLock +import edu.ie3.simona.service.ServiceStateData.{InitializeServiceStateData, ServiceBaseStateData} +import edu.ie3.simona.service.primary.ExtPrimaryDataService.{ExtPrimaryDataStateData, InitExtPrimaryData} +import edu.ie3.simona.service.primary.PrimaryServiceWorker.ProvidePrimaryDataMessage import edu.ie3.simona.service.{ExtDataSupport, ServiceStateData, SimonaService} -import edu.ie3.simona.service.primary.ExtPrimaryServiceWorker.ExtPrimaryDataStateData +import org.apache.pekko.actor.typed.scaladsl.adapter.ClassicActorRefOps import org.apache.pekko.actor.{ActorContext, ActorRef, Props} -import scala.util.Try +import java.util.UUID +import scala.jdk.CollectionConverters.MapHasAsScala +import scala.util.{Failure, Success, Try} object ExtPrimaryDataService { @@ -15,6 +29,15 @@ object ExtPrimaryDataService { new ExtPrimaryDataService(scheduler: ActorRef) ) + final case class ExtPrimaryDataStateData( + extPrimaryData: ExtPrimaryData, + uuidToActorRef: Map[UUID, ActorRef] = Map.empty[UUID, ActorRef], // subscribers in SIMONA + extPrimaryDataMessage: Option[PrimaryDataMessageFromExt] = None + ) extends ServiceBaseStateData + + final case class InitExtPrimaryData( + extPrimaryData: ExtPrimaryData + ) extends InitializeServiceStateData } final case class ExtPrimaryDataService( @@ -23,35 +46,65 @@ final case class ExtPrimaryDataService( extends SimonaService[ExtPrimaryDataStateData](scheduler) with ExtDataSupport[ExtPrimaryDataStateData] { - /** Initialize the concrete service implementation using the provided - * initialization data. This method should perform all heavyweight tasks - * before the actor becomes ready. The return values are a) the state data of - * the initialized service and b) optional triggers that should be send to - * the [[edu.ie3.simona.scheduler.Scheduler]] together with the completion - * message that is send in response to the trigger that is send to start the - * initialization process - * - * @param initServiceData - * the data that should be used for initialization - * @return - * the state data of this service and optional tick that should be included - * in the completion message - */ - override def init(initServiceData: ServiceStateData.InitializeServiceStateData): Try[(ExtPrimaryDataStateData, Option[Long])] = ??? - /** Handle a request to register for information from this service - * - * @param registrationMessage - * registration message to handle - * @param serviceStateData - * current state data of the actor - * @return - * the service stata data that should be used in the next state (normally - * with updated values) - */ + override def init( + initServiceData: ServiceStateData.InitializeServiceStateData + ): Try[(ExtPrimaryDataStateData, Option[Long])] = initServiceData match { + case InitExtPrimaryData(extPrimaryData) => + val primaryDataInitializedStateData = ExtPrimaryDataStateData( + extPrimaryData + ) + + Success( + primaryDataInitializedStateData, + None + ) + + case invalidData => + Failure( + new InitializationException( + s"Provided init data '${invalidData.getClass.getSimpleName}' for ExtPrimaryService are invalid!" + ) + ) + } + override protected def handleRegistrationRequest( registrationMessage: ServiceMessage.ServiceRegistrationMessage - )(implicit serviceStateData: ExtPrimaryDataStateData): Try[ExtPrimaryDataStateData] = ??? + )(implicit serviceStateData: ExtPrimaryDataStateData): + Try[ExtPrimaryDataStateData] = registrationMessage match { + case ExtPrimaryDataServiceRegistrationMessage(modelUuid, requestingActor) => + Success(handleRegistrationRequest(requestingActor, modelUuid)) + case invalidMessage => + Failure( + InvalidRegistrationRequestException( + s"A primary service provider is not able to handle registration request '$invalidMessage'." + ) + ) + } + + + private def handleRegistrationRequest( + agentToBeRegistered: ActorRef, + agentUUID: UUID + )(implicit serviceStateData: ExtPrimaryDataStateData): + ExtPrimaryDataStateData = { + serviceStateData.uuidToActorRef.get(agentUUID) match { + case None => + // Actor is not registered yet + agentToBeRegistered ! RegistrationSuccessfulMessage(None) + serviceStateData.copy( + uuidToActorRef = + serviceStateData.uuidToActorRef + (agentUUID -> agentToBeRegistered) + ) + case Some(_) => + // actor is already registered, do nothing + log.warning( + "Sending actor {} is already registered", + agentToBeRegistered + ) + serviceStateData + } + } /** Send out the information to all registered recipients * @@ -64,30 +117,99 @@ final case class ExtPrimaryDataService( * with updated values) together with the completion message that is send * in response to the trigger that was sent to start this announcement */ - override protected def announceInformation(tick: Long)(implicit serviceStateData: ExtPrimaryDataStateData, ctx: ActorContext): (ExtPrimaryDataStateData, Option[Long]) = ??? + override protected def announceInformation( + tick: Long + )( + implicit serviceStateData: ExtPrimaryDataStateData, + ctx: ActorContext + ): (ExtPrimaryDataStateData, Option[Long]) = { + serviceStateData.extPrimaryDataMessage.getOrElse( + throw ServiceException( + "ExtPrimaryDataService was triggered without ExtPrimaryDataMessage available" + ) + ) match { + case providedPrimaryData: ProvidePrimaryData => + processDataAndAnnounce(tick, providedPrimaryData.primaryData)(serviceStateData, ctx) + } + } - /** Handle a message from outside the simulation - * - * @param extMsg - * the external incoming message - * @param serviceStateData - * the current state data of this service - * @return - * the updated state data - */ - override protected def handleDataMessage(extMsg: DataMessageFromExt)(implicit serviceStateData: ExtPrimaryDataStateData): ExtPrimaryDataStateData = ??? + private def processDataAndAnnounce( + tick: Long, + primaryData: java.util.Map[UUID, Value] + )( + implicit + serviceStateData: ExtPrimaryDataStateData, + ctx: ActorContext + ): ( + ExtPrimaryDataStateData, + Option[Long] + ) = { + val actorToPrimaryData = primaryData.asScala.flatMap { + case (agent, primaryDataPerAgent) => + serviceStateData.uuidToActorRef + .get(agent) + .map((_, primaryDataPerAgent)) + .orElse { + log.warning( + "A corresponding actor ref for UUID {} could not be found", + agent + ) + None + } + } - /** Handle a message from inside SIMONA sent to external - * - * @param extResponseMsg - * the external incoming message - * @param serviceStateData - * the current state data of this service - * @return - * the updated state data - */ - override protected def handleDataResponseMessage(extResponseMsg: DataMessage)(implicit serviceStateData: ExtPrimaryDataStateData): ExtPrimaryDataStateData = ??? + // Verteile Primary Data + if (actorToPrimaryData.nonEmpty) { + val keys = + ScheduleLock.multiKey(ctx, scheduler.toTyped, tick, actorToPrimaryData.size) + + actorToPrimaryData.zip(keys).foreach { + case ((actor, primaryDataPerAgent), key) => + primaryDataPerAgent.toPrimaryData match { + case Success(primaryData) => + actor ! ProvidePrimaryDataMessage( + tick, + primaryData, + None, + unlockKey = Some(key) + ) + case Failure(exception) => + /* Processing of data failed */ + log.warning( + "Unable to convert received value to primary data. Skipped that data." + + "\nException: {}", + exception + ) + } + } + } + + ( // Message leeren + serviceStateData.copy(extPrimaryDataMessage = None), + None + ) + + } + override protected def handleDataMessage( + extMsg: DataMessageFromExt + )(implicit serviceStateData: ExtPrimaryDataStateData): + ExtPrimaryDataStateData = { + extMsg match { + case extPrimaryDataMessage: PrimaryDataMessageFromExt => + serviceStateData.copy( + extPrimaryDataMessage = Some(extPrimaryDataMessage) + ) + } + } + override protected def handleDataResponseMessage( + extResponseMsg: DataMessage + )( + implicit serviceStateData: ExtPrimaryDataStateData): + ExtPrimaryDataStateData = { + // not implemented + null + } } diff --git a/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryServiceProxy.scala b/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryServiceProxy.scala deleted file mode 100644 index ffc76386e0..0000000000 --- a/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryServiceProxy.scala +++ /dev/null @@ -1,99 +0,0 @@ -package edu.ie3.simona.service.primary - -import edu.ie3.datamodel.io.naming.timeseries.IndividualTimeSeriesMetaInformation -import edu.ie3.simona.api.ExtSimAdapter -import edu.ie3.simona.api.data.primarydata.ExtPrimaryData -import edu.ie3.simona.config.SimonaConfig.Simona.Input -import edu.ie3.simona.ontology.messages.services.ServiceMessage.{PrimaryServiceRegistrationMessage, WorkerRegistrationMessage} -import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.RegistrationFailedMessage -import edu.ie3.simona.service.primary.ExtPrimaryServiceProxy.ExtPrimaryServiceStateData -import edu.ie3.simona.service.primary.PrimaryServiceProxy.{InitPrimaryServiceProxyStateData, PrimaryServiceStateData} -import org.apache.pekko.actor.{Actor, ActorRef} - -import java.time.ZonedDateTime -import java.util.UUID -import scala.util.{Failure, Success, Try} - -case class ExtPrimaryServiceProxy( - scheduler: ActorRef, - initStateData: InitPrimaryServiceProxyStateData, - private implicit val startDateTime: ZonedDateTime - ) extends PrimaryServiceProxy(scheduler, initStateData, startDateTime) { - - /** Message handling, if the actor has been initialized already. This method - * basically handles registration requests, checks, if pre-calculated, - * primary data is available and forwards the request to worker actors. If - * needed, new workers are spun off. - * - * @param stateData - * Representing the current state of the agent - * @return - * Message handling routine - */ - private def onMessage(stateData: ExtPrimaryServiceStateData): Receive = { - case PrimaryServiceRegistrationMessage(modelUuid) => - // Ist für modelUuid eine externe primaere Datenquellen vorgesehen? - stateData.extPrimaryDataReferenceMap.get(modelUuid) match { - case Some(ExtPrimaryData) => // Ja! - // Registriere Agent beim entsprechenden Service - handleCoveredModel( - modelUuid, - stateData, - sender() - ) - case None => - log.debug( - s"There is no external data apparent for the model with uuid '{}'.", - modelUuid - ) - sender() ! RegistrationFailedMessage - } - case x => - log.error( - s"Received message '$x', but I'm only able to handle registration requests." - ) - unhandled(x) - } - - override def prepareStateData( - primaryConfig: Input.Primary, - simulationStart: ZonedDateTime - ): Try[PrimaryServiceStateData] = { - val participantToExtSimMapping: Map[UUID, ExtSimAdapter] = null - - - - - ExtPrimaryServiceStateData( - simulationStart, - extPrimaryServices, - extPrimaryDataReferenceMap - ) - } - - protected def handleCoveredModel( - modelUuid: UUID, - stateData: ExtPrimaryServiceStateData, - requestingActor: ActorRef - ): Unit = { - // Zu welchem DataService gehoert der Agent? - val participantToExtReferenceMap = stateData.extPrimaryDataReferenceMap - val worker = requestingActor - worker ! WorkerRegistrationMessage(requestingActor) - } -} - -object ExtPrimaryServiceProxy { - - final case class ExtPrimaryServiceStateData( - simulationStart: ZonedDateTime, - extPrimaryServices: Vector[ExtPrimaryServiceWorker], - extPrimaryDataReferenceMap: Map[UUID, ExtPrimaryData] - ) extends PrimaryServiceStateData( - simulationStart, null - ) - - - - -} diff --git a/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryServiceWorker.scala b/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryServiceWorker.scala deleted file mode 100644 index ea13f1be2e..0000000000 --- a/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryServiceWorker.scala +++ /dev/null @@ -1,218 +0,0 @@ -package edu.ie3.simona.service.primary - -import edu.ie3.datamodel.models.value.Value -import edu.ie3.simona.agent.participant.data.Data.PrimaryData -import edu.ie3.simona.agent.participant.data.Data.PrimaryData.RichValue -import edu.ie3.simona.api.data.ontology.DataMessageFromExt -import edu.ie3.simona.api.data.primarydata.ExtPrimaryData -import edu.ie3.simona.api.data.primarydata.ontology.{PrimaryDataMessageFromExt, ProvidePrimaryData} -import edu.ie3.simona.exceptions.WeatherServiceException.InvalidRegistrationRequestException -import edu.ie3.simona.exceptions.{InitializationException, ServiceException} -import edu.ie3.simona.ontology.messages.services.ServiceMessage.PrimaryServiceRegistrationMessage -import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.RegistrationSuccessfulMessage -import edu.ie3.simona.ontology.messages.services.{EvMessage, ServiceMessage} -import edu.ie3.simona.scheduler.ScheduleLock -import edu.ie3.simona.service.ServiceStateData.{InitializeServiceStateData, ServiceBaseStateData} -import edu.ie3.simona.service.primary.ExtPrimaryServiceWorker.{ExtPrimaryDataStateData, InitExtPrimaryData} -import edu.ie3.simona.service.primary.PrimaryServiceWorker.{PrimaryServiceInitializedStateData, ProvidePrimaryDataMessage} -import edu.ie3.simona.service.{ExtDataSupport, ServiceStateData, SimonaService} -import org.apache.pekko.actor.typed.scaladsl.adapter.ClassicActorRefOps -import org.apache.pekko.actor.{ActorContext, ActorRef, Props} - -import java.util.UUID -import scala.jdk.CollectionConverters.MapHasAsScala -import scala.util.{Failure, Success, Try} - -object ExtPrimaryServiceWorker { - def props(scheduler: ActorRef): Props = Props(new ExtPrimaryServiceWorker(scheduler: ActorRef)) - - final case class ExtPrimaryDataStateData( - extPrimaryData: ExtPrimaryData, - uuidToActorRef: Map[UUID, ActorRef] = Map.empty[UUID, ActorRef], // subscribers - extPrimaryDataMessage: Option[PrimaryDataMessageFromExt] = None - ) extends ServiceBaseStateData - - final case class InitExtPrimaryData( - extPrimaryData: ExtPrimaryData - ) extends InitializeServiceStateData -} - - -final case class ExtPrimaryServiceWorker( - override protected val scheduler: ActorRef - ) extends SimonaService[ExtPrimaryDataStateData](scheduler) - with ExtDataSupport[ExtPrimaryDataStateData] { - - - override def init( - initServiceData: ServiceStateData.InitializeServiceStateData - ): Try[(ExtPrimaryDataStateData, Option[Long])] = initServiceData match { - case InitExtPrimaryData(extPrimaryData) => - val evInitializedStateData = ExtPrimaryDataStateData( - extPrimaryData - ) - - Success( - evInitializedStateData, - None - ) - - case invalidData => - Failure( - new InitializationException( - s"Provided init data '${invalidData.getClass.getSimpleName}' for ExtPrimaryService are invalid!" - ) - ) - } - - override protected def handleRegistrationRequest( - registrationMessage: ServiceMessage.ServiceRegistrationMessage - )(implicit serviceStateData: ExtPrimaryDataStateData): - Try[ExtPrimaryDataStateData] = registrationMessage match { - case PrimaryServiceRegistrationMessage(participant) => - Success(handleRegistrationRequest(sender(), participant)) - case invalidMessage => - Failure( - InvalidRegistrationRequestException( - "Cannot register an agent for ev movement service with registration " + - s"request message '${invalidMessage.getClass.getSimpleName}'!" - ) - ) - } - - - private def handleRegistrationRequest( - agentToBeRegistered: ActorRef, - agentUUID: UUID - )(implicit serviceStateData: ExtPrimaryDataStateData): - ExtPrimaryDataStateData = { - serviceStateData.uuidToActorRef.get(agentUUID) match { - case None => - // Actor is not registered yet - agentToBeRegistered ! RegistrationSuccessfulMessage(None) - serviceStateData.copy( - uuidToActorRef = - serviceStateData.uuidToActorRef + (agentUUID -> agentToBeRegistered) - ) - case Some(_) => - // actor is already registered, do nothing - log.warning( - "Sending actor {} is already registered", - agentToBeRegistered - ) - serviceStateData - } - } - - /** Send out the information to all registered recipients - * - * @param tick - * current tick data should be announced for - * @param serviceStateData - * the current state data of this service - * @return - * the service stata data that should be used in the next state (normally - * with updated values) together with the completion message that is send - * in response to the trigger that was sent to start this announcement - */ - override protected def announceInformation( - tick: Long - )( - implicit serviceStateData: ExtPrimaryDataStateData, - ctx: ActorContext - ): (ExtPrimaryDataStateData, Option[Long]) = { - serviceStateData.extPrimaryDataMessage.getOrElse( - throw ServiceException( - "ExtPrimaryDataService was triggered without ExtPrimaryDataMessage available" - ) - ) match { - case providedPrimaryData: ProvidePrimaryData => - processDataAndAnnounce(tick, providedPrimaryData.primaryData)(serviceStateData, ctx) - } - } - - private def processDataAndAnnounce( - tick: Long, - primaryData: java.util.Map[UUID, Value] - )( - implicit - serviceStateData: ExtPrimaryDataStateData, - ctx: ActorContext - ): ( - ExtPrimaryDataStateData, - Option[Long] - ) = { - val actorToPrimaryData = primaryData.asScala.flatMap { - case (agent, primaryDataPerAgent) => - serviceStateData.uuidToActorRef - .get(agent) - .map((_, primaryDataPerAgent)) - .orElse { - log.warning( - "A corresponding actor ref for UUID {} could not be found", - agent - ) - None - } - } - - // Verteile Primary Data - if (actorToPrimaryData.nonEmpty) { - val keys = - ScheduleLock.multiKey(ctx, scheduler.toTyped, tick, actorToPrimaryData.size) - - actorToPrimaryData.zip(keys).foreach { - case ((actor, primaryDataPerAgent), key) => { - primaryDataPerAgent.toPrimaryData match { - case Success(primaryData) => - actor ! ProvidePrimaryDataMessage( - tick, - primaryData, - null, // nextDataTick - unlockKey = Some(key) - ) - case Failure(exception) => - /* Processing of data failed */ - log.warning( - "Unable to convert received value to primary data. Skipped that data." + - "\nException: {}", - exception - ) - } - - } - } - - } - - - ( // Message leeren - serviceStateData.copy( - extPrimaryDataMessage = None - ), - None - ) - - } - - override protected def handleDataMessage( - extMsg: DataMessageFromExt - )(implicit serviceStateData: ExtPrimaryDataStateData): - ExtPrimaryDataStateData = { - extMsg match { - case extPrimaryDataMessage: PrimaryDataMessageFromExt => - serviceStateData.copy( - extPrimaryDataMessage = Some(extPrimaryDataMessage) - ) - } - } - - override protected def handleDataResponseMessage( - extResponseMsg: EvMessage - )( - implicit serviceStateData: ExtPrimaryDataStateData): - ExtPrimaryDataStateData = { - // not implemented - null - } -} diff --git a/src/main/scala/edu/ie3/simona/service/primary/ExtTimeSeriesSubscribersSource.scala b/src/main/scala/edu/ie3/simona/service/primary/ExtTimeSeriesSubscribersSource.scala new file mode 100644 index 0000000000..d125b95cc4 --- /dev/null +++ b/src/main/scala/edu/ie3/simona/service/primary/ExtTimeSeriesSubscribersSource.scala @@ -0,0 +1,9 @@ +package edu.ie3.simona.service.primary + +import java.util.UUID + +object ExtTimeSeriesSubscribersSource { + private var subscribers: Iterable[UUID] = Iterable.empty[UUID] + + def getSubscribers(): Iterable[UUID] = subscribers +} diff --git a/src/main/scala/edu/ie3/simona/service/primary/IntPrimaryServiceProxy.scala b/src/main/scala/edu/ie3/simona/service/primary/IntPrimaryServiceProxy.scala deleted file mode 100644 index 1ee578a7a7..0000000000 --- a/src/main/scala/edu/ie3/simona/service/primary/IntPrimaryServiceProxy.scala +++ /dev/null @@ -1,566 +0,0 @@ -/* - * © 2021. TU Dortmund University, - * Institute of Energy Systems, Energy Efficiency and Energy Economics, - * Research group Distribution grid planning and operation - */ - -package edu.ie3.simona.service.primary - -import org.apache.pekko.actor.typed.scaladsl.adapter.ClassicActorRefOps -import org.apache.pekko.actor.{Actor, ActorRef, PoisonPill, Props} -import edu.ie3.datamodel.io.connectors.SqlConnector -import edu.ie3.datamodel.io.csv.CsvIndividualTimeSeriesMetaInformation -import edu.ie3.datamodel.io.naming.timeseries.IndividualTimeSeriesMetaInformation -import edu.ie3.datamodel.io.naming.{DatabaseNamingStrategy, EntityPersistenceNamingStrategy, FileNamingStrategy} -import edu.ie3.datamodel.io.source.csv.{CsvTimeSeriesMappingSource, CsvTimeSeriesMetaInformationSource} -import edu.ie3.datamodel.io.source.sql.{SqlTimeSeriesMappingSource, SqlTimeSeriesMetaInformationSource} -import edu.ie3.datamodel.io.source.{TimeSeriesMappingSource, TimeSeriesMetaInformationSource} -import edu.ie3.datamodel.models.value.Value -import edu.ie3.simona.config.SimonaConfig.PrimaryDataCsvParams -import edu.ie3.simona.config.SimonaConfig.Simona.Input.Primary.SqlParams -import edu.ie3.simona.config.SimonaConfig.Simona.Input.{Primary => PrimaryConfig} -import edu.ie3.simona.exceptions.{InitializationException, InvalidConfigParameterException} -import edu.ie3.simona.logging.SimonaActorLogging -import edu.ie3.simona.ontology.messages.Activation -import edu.ie3.simona.ontology.messages.SchedulerMessage.Completion -import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.RegistrationFailedMessage -import edu.ie3.simona.ontology.messages.services.ServiceMessage.{PrimaryServiceRegistrationMessage, WorkerRegistrationMessage} -import edu.ie3.simona.scheduler.ScheduleLock -import edu.ie3.simona.service.{ServiceStateData, SimonaService} -import edu.ie3.simona.service.ServiceStateData.InitializeServiceStateData -import edu.ie3.simona.service.primary.IntPrimaryServiceProxy.{IntPrimaryServiceStateData, SourceRef} -import edu.ie3.simona.service.primary.PrimaryServiceProxy.{InitPrimaryServiceProxyStateData, PrimaryServiceStateData} -import edu.ie3.simona.service.primary.PrimaryServiceWorker.{CsvInitPrimaryServiceStateData, InitPrimaryServiceStateData, SqlInitPrimaryServiceStateData} -import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK - -import java.nio.file.Paths -import java.text.SimpleDateFormat -import java.time.ZonedDateTime -import java.util.UUID -import scala.Option.when -import scala.jdk.CollectionConverters._ -import scala.jdk.OptionConverters.RichOptional -import scala.util.{Failure, Success, Try} - -/** This actor has information on which models can be replaced by precalculated - * (primary) data and how to obtain those time series. It offers possibility to - * register for a certain model. If data is available, a child actor is spun - * of, that will do the actual provision and the requesting agent is informed - * accordingly. - * - * @param scheduler - * Reference to the scheduler of the simulation - * @param startDateTime - * Wall clock time of the first instant in simulation - */ -case class IntPrimaryServiceProxy( - scheduler: ActorRef, - initStateData: InitPrimaryServiceProxyStateData, - private implicit val startDateTime: ZonedDateTime -) extends PrimaryServiceProxy(scheduler, initStateData, startDateTime) { - - - override def onMessage(stateData: PrimaryServiceStateData): Receive = { - stateData match { - case IntPrimaryServiceStateData( - modelToTimeSeries, - timeSeriesToSourceRef, - simulationStart, - primaryConfig, - mappingSource - ) => onMessage(IntPrimaryServiceStateData( - modelToTimeSeries, - timeSeriesToSourceRef, - simulationStart, - primaryConfig, - mappingSource - )) - } - } - - /** Message handling, if the actor has been initialized already. This method - * basically handles registration requests, checks, if pre-calculated, - * primary data is available and forwards the request to worker actors. If - * needed, new workers are spun off. - * - * @param stateData - * Representing the current state of the agent - * @return - * Message handling routine - */ - private def onMessage(stateData: IntPrimaryServiceStateData): Receive = { - case PrimaryServiceRegistrationMessage(modelUuid) => - /* Try to register for this model */ - stateData.modelToTimeSeries.get(modelUuid) match { - case Some(timeSeriesUuid) => - /* There is a time series apparent for this model, try to get a worker for it */ - handleCoveredModel( - modelUuid, - timeSeriesUuid, - stateData, - sender() - ) - case None => - log.debug( - s"There is no time series apparent for the model with uuid '{}'.", - modelUuid - ) - sender() ! RegistrationFailedMessage - } - case x => - log.error( - s"Received message '$x', but I'm only able to handle registration requests." - ) - unhandled(x) - } - - /** Prepare the needed state data by building a - * [[edu.ie3.datamodel.io.source.TimeSeriesMappingSource]], obtain it's - * information and compile them to state data - * - * @param primaryConfig - * Configuration for the primary source - * @param simulationStart - * Wall clock time of first instant in simulation - * @return - * State data, containing the known model and time series identifiers - */ - override def prepareStateData( - primaryConfig: PrimaryConfig, - simulationStart: ZonedDateTime - ): Try[PrimaryServiceStateData] = { - createSources(primaryConfig).map { - case (mappingSource, metaInformationSource) => - val modelToTimeSeries = mappingSource.getMapping.asScala.toMap - val timeSeriesToSourceRef = modelToTimeSeries.values - .to(LazyList) - .distinct - .flatMap { timeSeriesUuid => - metaInformationSource - .getTimeSeriesMetaInformation(timeSeriesUuid) - .toScala match { - case Some(metaInformation) => - /* Only register those entries, that meet the supported column schemes */ - when( - PrimaryServiceWorker.supportedColumnSchemes - .contains(metaInformation.getColumnScheme) - ) { - timeSeriesUuid -> SourceRef(metaInformation, None) - } - case None => - log.warning( - "Unable to acquire meta information for time series '{}'. Leave that out.", - timeSeriesUuid - ) - None - } - } - .toMap - IntPrimaryServiceStateData( - modelToTimeSeries, - timeSeriesToSourceRef, - simulationStart, - primaryConfig, - mappingSource - ) - } - } - - private def createSources( - primaryConfig: PrimaryConfig - ): Try[(TimeSeriesMappingSource, TimeSeriesMetaInformationSource)] = { - Seq( - primaryConfig.sqlParams, - primaryConfig.influxDb1xParams, - primaryConfig.csvParams, - primaryConfig.couchbaseParams - ).filter(_.isDefined).flatten.headOption match { - case Some(PrimaryDataCsvParams(csvSep, directoryPath, _, _)) => - val fileNamingStrategy = new FileNamingStrategy() - Success( - new CsvTimeSeriesMappingSource( - csvSep, - Paths.get(directoryPath), - fileNamingStrategy - ), - new CsvTimeSeriesMetaInformationSource( - csvSep, - Paths.get(directoryPath), - fileNamingStrategy - ) - ) - case Some(sqlParams: SqlParams) => - val sqlConnector = new SqlConnector( - sqlParams.jdbcUrl, - sqlParams.userName, - sqlParams.password - ) - Success( - new SqlTimeSeriesMappingSource( - sqlConnector, - sqlParams.schemaName, - new EntityPersistenceNamingStrategy() - ), - new SqlTimeSeriesMetaInformationSource( - sqlConnector, - sqlParams.schemaName, - new DatabaseNamingStrategy() - ) - ) - case Some(x) => - Failure( - new IllegalArgumentException( - s"Unsupported config for mapping source: '$x'" - ) - ) - case None => - Failure( - new IllegalArgumentException( - "You have to provide exactly one config for the mapping source." - ) - ) - } - } - - - /** Handle the registration request for a covered model. First, try to get a - * already existing worker for this time series, otherwise spin-off a new - * one, remember it and forward the request - * - * @param modelUuid - * Unique identifier of the model - * @param timeSeriesUuid - * Unique identifier of the equivalent time series - * @param stateData - * Current state data of the actor - */ - override protected def handleCoveredModel( - modelUuid: UUID, - timeSeriesUuid: UUID, - stateData: PrimaryServiceStateData, - requestingActor: ActorRef - ): Unit = { - stateData match { - case IntPrimaryServiceStateData(modelToTimeSeries, timeSeriesToSourceRef, simulationStart, primaryConfig, mappingSource) => { - timeSeriesToSourceRef.get(timeSeriesUuid) match { - case Some(SourceRef(_, Some(worker))) => - /* There is yet a worker apparent. Register the requesting actor. The worker will reply to the original - * requesting actor. */ - worker ! WorkerRegistrationMessage(requestingActor) - case Some(SourceRef(metaInformation, None)) => - /* There is NO worker apparent, yet. Spin one off. */ - initializeWorker( - metaInformation, - simulationStart, - primaryConfig - ) match { - case Success(workerRef) => - /* Forward the registration request. The worker will reply about successful registration or not. */ - workerRef ! WorkerRegistrationMessage(requestingActor) - - /* Register the new worker within the state data and change the context */ - context become onMessage( - updateStateData( - IntPrimaryServiceStateData(modelToTimeSeries, timeSeriesToSourceRef, simulationStart, primaryConfig, mappingSource), - timeSeriesUuid, - workerRef) - ) - case Failure(exception) => - log.warning( - s"A failure occurred during spin-off of a primary source for time series '$timeSeriesUuid'. " + - s"Will inform the requesting actor, that registration is not possible.", - exception - ) - requestingActor ! RegistrationFailedMessage - } - - case None => - log.warning( - s"There is no source information for time series '$timeSeriesUuid' (requested for model " + - s"'$modelUuid'), although the mapping contains information about it." - ) - requestingActor ! RegistrationFailedMessage - } - } - case invalidData => Failure( - new InitializationException( - s"Provided init data '${invalidData.getClass.getSimpleName}' for weather service are invalid!" - ) - ) - - - } - } - - /** Instantiate a new [[PrimaryServiceWorker]] and send initialization - * information - * - * @param metaInformation - * Meta information (including column scheme) of the time series - * @param simulationStart - * The time of the simulation start - * @param primaryConfig - * Configuration for the primary config - * @return - * The [[ActorRef]] to the worker - */ - protected def initializeWorker( - metaInformation: IndividualTimeSeriesMetaInformation, - simulationStart: ZonedDateTime, - primaryConfig: PrimaryConfig - ): Try[ActorRef] = { - val workerRef = classToWorkerRef( - metaInformation.getColumnScheme.getValueClass, - metaInformation.getUuid.toString - ) - toInitData( - metaInformation, - simulationStart, - primaryConfig - ) match { - case Success(initData) => - workerRef ! SimonaService.Create( - initData, - ScheduleLock.singleKey(context, scheduler.toTyped, INIT_SIM_TICK) - ) - Success(workerRef) - case Failure(cause) => - workerRef ! PoisonPill - Failure( - new InitializationException( - "Unable to build init data for worker. Kill the uninitialized worker. Goodbye my friend!", - cause - ) - ) - } - } - - /** Register the worker within the state data. - * - * @param stateData - * Current state information - * @param timeSeriesUuid - * Unique identifier of the time series, the worker takes care of - * @param workerRef - * [[ActorRef]] to the new worker actor - * @return - * The updated state data, that holds reference to the worker - */ - private def updateStateData( - stateData: IntPrimaryServiceStateData, - timeSeriesUuid: UUID, - workerRef: ActorRef - ): IntPrimaryServiceStateData = { - val timeSeriesToSourceRef = stateData.timeSeriesToSourceRef - val sourceRef = timeSeriesToSourceRef.getOrElse( - timeSeriesUuid, - throw new IllegalArgumentException( - s"Cannot update entry for time series '$timeSeriesUuid', as it hasn't been part of it before." - ) - ) - val updatedTimeSeriesToSourceRef = timeSeriesToSourceRef.updated( - timeSeriesUuid, - sourceRef.copy(worker = Some(workerRef)) - ) - stateData.copy(timeSeriesToSourceRef = updatedTimeSeriesToSourceRef) - } - - /** Build a primary source worker and type it to the foreseen value class to - * come - * - * @param valueClass - * Class of the values to provide later on - * @param timeSeriesUuid - * uuid of the time series the actor processes - * @tparam V - * Type of the class to provide - * @return - * The [[ActorRef]] to the spun off actor - */ - protected def classToWorkerRef[V <: Value]( - valueClass: Class[V], - timeSeriesUuid: String - ): ActorRef = { - import edu.ie3.simona.actor.SimonaActorNaming._ - context.system.simonaActorOf( - PrimaryServiceWorker.props(scheduler, valueClass), - timeSeriesUuid - ) - } - - /** Building proper init data for the worker - * - * @param metaInformation - * Meta information (including column scheme) of the time series - * @param simulationStart - * The time of the simulation start - * @param primaryConfig - * Configuration for the primary config - * @return - */ - private def toInitData( - metaInformation: IndividualTimeSeriesMetaInformation, - simulationStart: ZonedDateTime, - primaryConfig: PrimaryConfig - ): Try[InitPrimaryServiceStateData] = - primaryConfig match { - case PrimaryConfig( - None, - Some(PrimaryDataCsvParams(csvSep, directoryPath, _, timePattern)), - None, - None - ) => - /* The actual data sources are from csv. Meta information have to match */ - metaInformation match { - case csvMetaData: CsvIndividualTimeSeriesMetaInformation => - Success( - CsvInitPrimaryServiceStateData( - csvMetaData.getUuid, - simulationStart, - csvSep, - Paths.get(directoryPath), - csvMetaData.getFullFilePath, - new FileNamingStrategy(), - timePattern - ) - ) - case invalidMetaData => - Failure( - new InitializationException( - s"Expected '${classOf[CsvIndividualTimeSeriesMetaInformation]}', but got '$invalidMetaData'." - ) - ) - } - - case PrimaryConfig( - None, - None, - None, - Some(sqlParams: SqlParams) - ) => - Success( - SqlInitPrimaryServiceStateData( - metaInformation.getUuid, - simulationStart, - sqlParams, - new DatabaseNamingStrategy() - ) - ) - - case unsupported => - Failure( - new InitializationException( - s"Cannot build initialization data for a worker due to unsupported source config '$unsupported'." - ) - ) - } -} - -object IntPrimaryServiceProxy { - - def props( - scheduler: ActorRef, - initStateData: InitPrimaryServiceProxyStateData, - startDateTime: ZonedDateTime - ): Props = Props( - new IntPrimaryServiceProxy(scheduler, initStateData, startDateTime) - ) - - /** Holding the state of an initialized proxy. - * - * @param modelToTimeSeries - * Mapping from models' to time series unique identifiers - * @param timeSeriesToSourceRef - * Mapping from time series identifier to [[SourceRef]] - * @param simulationStart - * Wall clock time of the first instant in simulation - * @param primaryConfig - * The configuration for the sources - * @param mappingSource - * The mapping source - */ - final case class IntPrimaryServiceStateData( - modelToTimeSeries: Map[UUID, UUID], - timeSeriesToSourceRef: Map[UUID, SourceRef], - simulationStart: ZonedDateTime, - primaryConfig: PrimaryConfig, - mappingSource: TimeSeriesMappingSource - ) extends PrimaryServiceStateData( - simulationStart, primaryConfig - ) - - /** Giving reference to the target time series and source worker. - * - * @param metaInformation - * Meta information (including column scheme) of the time series - * @param worker - * Optional reference to an already existing worker providing information - * on that time series - */ - final case class SourceRef( - metaInformation: IndividualTimeSeriesMetaInformation, - worker: Option[ActorRef] - ) - - /** Check if the config holds correct information to instantiate a mapping - * source - * - * @param primaryConfig - * Config entries for primary source - */ - def checkConfig(primaryConfig: PrimaryConfig): Unit = { - - def checkTimePattern(dtfPattern: String): Unit = - Try { - new SimpleDateFormat(dtfPattern) - } match { - case Failure(exception) => - throw new InvalidConfigParameterException( - s"Invalid timePattern '$dtfPattern' for a time series source. Please provide a valid pattern!" + - s"\nException: $exception" - ) - case Success(_) => - // this is fine - } - - val supportedSources = - Set("csv", "sql") - - val sourceConfigs = Seq( - primaryConfig.couchbaseParams, - primaryConfig.csvParams, - primaryConfig.influxDb1xParams, - primaryConfig.sqlParams - ).filter(_.isDefined).flatten - if (sourceConfigs.size > 1) - throw new InvalidConfigParameterException( - s"${sourceConfigs.size} time series source types defined. " + - s"Please define only one type!\nAvailable types:\n\t${supportedSources.mkString("\n\t")}" - ) - else if (sourceConfigs.isEmpty) - throw new InvalidConfigParameterException( - s"No time series source type defined. Please define exactly one type!" + - s"\nAvailable types:\n\t${supportedSources.mkString("\n\t")}" - ) - else { - sourceConfigs.headOption match { - case Some(csvParams: PrimaryDataCsvParams) => - // note: if inheritance is supported by tscfg, - // the following method should be called for all different supported sources! - checkTimePattern(csvParams.timePattern) - case Some(sqlParams: SqlParams) => - checkTimePattern(sqlParams.timePattern) - case Some(x) => - throw new InvalidConfigParameterException( - s"Invalid configuration '$x' for a time series source.\nAvailable types:\n\t${supportedSources - .mkString("\n\t")}" - ) - case None => - throw new InvalidConfigParameterException( - s"No configuration for a time series mapping source provided.\nPlease provide one of the available sources:\n\t${supportedSources - .mkString("\n\t")}" - ) - } - } - } -} diff --git a/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala b/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala index 17efab5b98..6a6de0d3ec 100644 --- a/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala +++ b/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala @@ -1,34 +1,65 @@ +/* + * © 2021. TU Dortmund University, + * Institute of Energy Systems, Energy Efficiency and Energy Economics, + * Research group Distribution grid planning and operation + */ + package edu.ie3.simona.service.primary +import org.apache.pekko.actor.typed.scaladsl.adapter.ClassicActorRefOps +import org.apache.pekko.actor.{Actor, ActorRef, PoisonPill, Props} +import edu.ie3.datamodel.io.connectors.SqlConnector +import edu.ie3.datamodel.io.csv.CsvIndividualTimeSeriesMetaInformation import edu.ie3.datamodel.io.naming.timeseries.IndividualTimeSeriesMetaInformation -import edu.ie3.datamodel.io.source.TimeSeriesMappingSource -import edu.ie3.simona.config.SimonaConfig +import edu.ie3.datamodel.io.naming.{DatabaseNamingStrategy, EntityPersistenceNamingStrategy, FileNamingStrategy} +import edu.ie3.datamodel.io.source.csv.{CsvTimeSeriesMappingSource, CsvTimeSeriesMetaInformationSource} +import edu.ie3.datamodel.io.source.sql.{SqlTimeSeriesMappingSource, SqlTimeSeriesMetaInformationSource} +import edu.ie3.datamodel.io.source.{TimeSeriesMappingSource, TimeSeriesMetaInformationSource} +import edu.ie3.datamodel.models.value.Value +import edu.ie3.simona.config.SimonaConfig.PrimaryDataCsvParams +import edu.ie3.simona.config.SimonaConfig.Simona.Input.Primary.SqlParams +import edu.ie3.simona.config.SimonaConfig.Simona.Input.{Primary => PrimaryConfig} +import edu.ie3.simona.exceptions.{InitializationException, InvalidConfigParameterException} import edu.ie3.simona.logging.SimonaActorLogging import edu.ie3.simona.ontology.messages.Activation import edu.ie3.simona.ontology.messages.SchedulerMessage.Completion import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.RegistrationFailedMessage -import edu.ie3.simona.ontology.messages.services.ServiceMessage.{PrimaryServiceRegistrationMessage, WorkerRegistrationMessage} -import edu.ie3.simona.service.ServiceStateData +import edu.ie3.simona.ontology.messages.services.ServiceMessage.{ExtPrimaryDataServiceRegistrationMessage, PrimaryServiceRegistrationMessage, WorkerRegistrationMessage} +import edu.ie3.simona.scheduler.ScheduleLock +import edu.ie3.simona.service.{ServiceStateData, SimonaService} import edu.ie3.simona.service.ServiceStateData.InitializeServiceStateData -import edu.ie3.simona.service.primary.IntPrimaryServiceProxy.SourceRef -import edu.ie3.simona.service.primary.PrimaryServiceProxy.{InitPrimaryServiceProxyStateData, PrimaryServiceStateData} +import edu.ie3.simona.service.primary.ExtTimeSeriesSubscribersSource.getSubscribers +import edu.ie3.simona.service.primary.PrimaryServiceProxy.{InitPrimaryServiceProxyStateData, PrimaryServiceStateData, SourceRef} +import edu.ie3.simona.service.primary.PrimaryServiceWorker.{CsvInitPrimaryServiceStateData, InitPrimaryServiceStateData, SqlInitPrimaryServiceStateData} import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK -import edu.ie3.simona.config.SimonaConfig.Simona.Input.{Primary => PrimaryConfig} -import org.apache.pekko.actor.typed.scaladsl.adapter.ClassicActorRefOps -import org.apache.pekko.actor.{Actor, ActorRef, PoisonPill} +import java.nio.file.Paths +import java.text.SimpleDateFormat import java.time.ZonedDateTime import java.util.UUID +import scala.Option.when +import scala.jdk.CollectionConverters._ +import scala.jdk.OptionConverters.RichOptional import scala.util.{Failure, Success, Try} -abstract class PrimaryServiceProxy( - scheduler: ActorRef, - initStateData: InitPrimaryServiceProxyStateData, - private implicit val startDateTime: ZonedDateTime - ) extends Actor +/** This actor has information on which models can be replaced by precalculated + * (primary) data and how to obtain those time series. It offers possibility to + * register for a certain model. If data is available, a child actor is spun + * of, that will do the actual provision and the requesting agent is informed + * accordingly. + * + * @param scheduler + * Reference to the scheduler of the simulation + * @param startDateTime + * Wall clock time of the first instant in simulation + */ +case class PrimaryServiceProxy( + scheduler: ActorRef, + initStateData: InitPrimaryServiceProxyStateData, + private implicit val startDateTime: ZonedDateTime + ) extends Actor with SimonaActorLogging { - /** Start receiving without knowing specifics about myself * * @return @@ -47,7 +78,8 @@ abstract class PrimaryServiceProxy( * messages */ prepareStateData( initStateData.primaryConfig, - initStateData.simulationStart + initStateData.simulationStart, + initStateData.extSimulation ) match { case Success(stateData) => scheduler ! Completion(self.toTyped) @@ -66,16 +98,172 @@ abstract class PrimaryServiceProxy( unhandled(x) } - def onMessage(stateData: PrimaryServiceStateData): Receive - - def askForExternalSimulation( - - ) + /** Prepare the needed state data by building a + * [[edu.ie3.datamodel.io.source.TimeSeriesMappingSource]], obtain it's + * information and compile them to state data + * + * @param primaryConfig + * Configuration for the primary source + * @param simulationStart + * Wall clock time of first instant in simulation + * @return + * State data, containing the known model and time series identifiers + */ + private def prepareStateData( + primaryConfig: PrimaryConfig, + simulationStart: ZonedDateTime, + extSimulation: Option[ActorRef] + ): Try[PrimaryServiceStateData] = { + var serviceStateData = PrimaryServiceStateData(Map.empty, Map.empty, simulationStart, primaryConfig, null) + createSources(primaryConfig).map { + case (mappingSource, metaInformationSource) => + val modelToTimeSeries = mappingSource.getMapping.asScala.toMap + val timeSeriesToSourceRef = modelToTimeSeries.values + .to(LazyList) + .distinct + .flatMap { timeSeriesUuid => + metaInformationSource + .getTimeSeriesMetaInformation(timeSeriesUuid) + .toScala match { + case Some(metaInformation) => + /* Only register those entries, that meet the supported column schemes */ + when( + PrimaryServiceWorker.supportedColumnSchemes + .contains(metaInformation.getColumnScheme) + ) { + timeSeriesUuid -> SourceRef(metaInformation, None) + } + case None => + log.warning( + "Unable to acquire meta information for time series '{}'. Leave that out.", + timeSeriesUuid + ) + None + } + } + .toMap + serviceStateData = PrimaryServiceStateData( + modelToTimeSeries, + timeSeriesToSourceRef, + simulationStart, + primaryConfig, + mappingSource + ) + } + if (extSimulation.isDefined) { + // Ask ExtPrimaryDataService which UUIDs should be substituted + Success(serviceStateData.copy( + extSubscribers = getSubscribers, + extPrimaryDataService = extSimulation + )) + } else { + Success(serviceStateData) + } + } - def prepareStateData( - primaryConfig: PrimaryConfig, - time: ZonedDateTime): Try[PrimaryServiceStateData] + private def createSources( + primaryConfig: PrimaryConfig + ): Try[(TimeSeriesMappingSource, TimeSeriesMetaInformationSource)] = { + Seq( + primaryConfig.sqlParams, + primaryConfig.influxDb1xParams, + primaryConfig.csvParams, + primaryConfig.couchbaseParams + ).filter(_.isDefined).flatten.headOption match { + case Some(PrimaryDataCsvParams(csvSep, directoryPath, _, _)) => + val fileNamingStrategy = new FileNamingStrategy() + Success( + new CsvTimeSeriesMappingSource( + csvSep, + Paths.get(directoryPath), + fileNamingStrategy + ), + new CsvTimeSeriesMetaInformationSource( + csvSep, + Paths.get(directoryPath), + fileNamingStrategy + ) + ) + case Some(sqlParams: SqlParams) => + val sqlConnector = new SqlConnector( + sqlParams.jdbcUrl, + sqlParams.userName, + sqlParams.password + ) + Success( + new SqlTimeSeriesMappingSource( + sqlConnector, + sqlParams.schemaName, + new EntityPersistenceNamingStrategy() + ), + new SqlTimeSeriesMetaInformationSource( + sqlConnector, + sqlParams.schemaName, + new DatabaseNamingStrategy() + ) + ) + case Some(x) => + Failure( + new IllegalArgumentException( + s"Unsupported config for mapping source: '$x'" + ) + ) + case None => + Failure( + new IllegalArgumentException( + "You have to provide exactly one config for the mapping source." + ) + ) + } + } + /** Message handling, if the actor has been initialized already. This method + * basically handles registration requests, checks, if pre-calculated, + * primary data is available and forwards the request to worker actors. If + * needed, new workers are spun off. + * + * @param stateData + * Representing the current state of the agent + * @return + * Message handling routine + */ + private def onMessage(stateData: PrimaryServiceStateData): Receive = { + case PrimaryServiceRegistrationMessage(modelUuid) => + /* Try to register for this model */ + stateData.modelToTimeSeries.get(modelUuid) match { + case Some(timeSeriesUuid) => + /* There is a time series apparent for this model, try to get a worker for it */ + handleCoveredModel( + modelUuid, + timeSeriesUuid, + stateData, + sender() + ) + case None => + if (stateData.extSubscribers.nonEmpty) { + if (stateData.extSubscribers.exists(_ == modelUuid)) { + handleExternalModel(modelUuid, stateData, sender()) + } else { + log.debug( + s"There is no time series apparent for the model with uuid '{}'.", + modelUuid + ) + sender() ! RegistrationFailedMessage + } + } else { + log.debug( + s"There is no time series apparent for the model with uuid '{}'.", + modelUuid + ) + sender() ! RegistrationFailedMessage + } + } + case x => + log.error( + s"Received message '$x', but I'm only able to handle registration requests." + ) + unhandled(x) + } /** Handle the registration request for a covered model. First, try to get a * already existing worker for this time series, otherwise spin-off a new @@ -93,10 +281,229 @@ abstract class PrimaryServiceProxy( timeSeriesUuid: UUID, stateData: PrimaryServiceStateData, requestingActor: ActorRef - ): Unit + ): Unit = { + val timeSeriesToSourceRef = stateData.timeSeriesToSourceRef + timeSeriesToSourceRef.get(timeSeriesUuid) match { + case Some(SourceRef(_, Some(worker))) => + /* There is yet a worker apparent. Register the requesting actor. The worker will reply to the original + * requesting actor. */ + worker ! WorkerRegistrationMessage(requestingActor) + case Some(SourceRef(metaInformation, None)) => + /* There is NO worker apparent, yet. Spin one off. */ + initializeWorker( + metaInformation, + stateData.simulationStart, + stateData.primaryConfig + ) match { + case Success(workerRef) => + /* Forward the registration request. The worker will reply about successful registration or not. */ + workerRef ! WorkerRegistrationMessage(requestingActor) + + /* Register the new worker within the state data and change the context */ + context become onMessage( + updateStateData(stateData, timeSeriesUuid, workerRef) + ) + case Failure(exception) => + log.warning( + s"A failure occurred during spin-off of a primary source for time series '$timeSeriesUuid'. " + + s"Will inform the requesting actor, that registration is not possible.", + exception + ) + requestingActor ! RegistrationFailedMessage + } + + case None => + log.warning( + s"There is no source information for time series '$timeSeriesUuid' (requested for model " + + s"'$modelUuid'), although the mapping contains information about it." + ) + requestingActor ! RegistrationFailedMessage + } + } + + protected def handleExternalModel( + modelUuid: UUID, + stateData: PrimaryServiceStateData, + requestingActor: ActorRef): Unit = { + stateData.extPrimaryDataService match { + case Some(reqActor) => reqActor ! ExtPrimaryDataServiceRegistrationMessage(modelUuid, reqActor) + } + } + + /** Instantiate a new [[PrimaryServiceWorker]] and send initialization + * information + * + * @param metaInformation + * Meta information (including column scheme) of the time series + * @param simulationStart + * The time of the simulation start + * @param primaryConfig + * Configuration for the primary config + * @return + * The [[ActorRef]] to the worker + */ + protected def initializeWorker( + metaInformation: IndividualTimeSeriesMetaInformation, + simulationStart: ZonedDateTime, + primaryConfig: PrimaryConfig + ): Try[ActorRef] = { + val workerRef = classToWorkerRef( + metaInformation.getColumnScheme.getValueClass, + metaInformation.getUuid.toString + ) + toInitData( + metaInformation, + simulationStart, + primaryConfig + ) match { + case Success(initData) => + workerRef ! SimonaService.Create( + initData, + ScheduleLock.singleKey(context, scheduler.toTyped, INIT_SIM_TICK) + ) + Success(workerRef) + case Failure(cause) => + workerRef ! PoisonPill + Failure( + new InitializationException( + "Unable to build init data for worker. Kill the uninitialized worker. Goodbye my friend!", + cause + ) + ) + } + } + + /** Build a primary source worker and type it to the foreseen value class to + * come + * + * @param valueClass + * Class of the values to provide later on + * @param timeSeriesUuid + * uuid of the time series the actor processes + * @tparam V + * Type of the class to provide + * @return + * The [[ActorRef]] to the spun off actor + */ + protected def classToWorkerRef[V <: Value]( + valueClass: Class[V], + timeSeriesUuid: String + ): ActorRef = { + import edu.ie3.simona.actor.SimonaActorNaming._ + context.system.simonaActorOf( + PrimaryServiceWorker.props(scheduler, valueClass), + timeSeriesUuid + ) + } + + /** Building proper init data for the worker + * + * @param metaInformation + * Meta information (including column scheme) of the time series + * @param simulationStart + * The time of the simulation start + * @param primaryConfig + * Configuration for the primary config + * @return + */ + private def toInitData( + metaInformation: IndividualTimeSeriesMetaInformation, + simulationStart: ZonedDateTime, + primaryConfig: PrimaryConfig + ): Try[InitPrimaryServiceStateData] = + primaryConfig match { + case PrimaryConfig( + None, + Some(PrimaryDataCsvParams(csvSep, directoryPath, _, timePattern)), + None, + None + ) => + /* The actual data sources are from csv. Meta information have to match */ + metaInformation match { + case csvMetaData: CsvIndividualTimeSeriesMetaInformation => + Success( + CsvInitPrimaryServiceStateData( + csvMetaData.getUuid, + simulationStart, + csvSep, + Paths.get(directoryPath), + csvMetaData.getFullFilePath, + new FileNamingStrategy(), + timePattern + ) + ) + case invalidMetaData => + Failure( + new InitializationException( + s"Expected '${classOf[CsvIndividualTimeSeriesMetaInformation]}', but got '$invalidMetaData'." + ) + ) + } + + case PrimaryConfig( + None, + None, + None, + Some(sqlParams: SqlParams) + ) => + Success( + SqlInitPrimaryServiceStateData( + metaInformation.getUuid, + simulationStart, + sqlParams, + new DatabaseNamingStrategy() + ) + ) + + case unsupported => + Failure( + new InitializationException( + s"Cannot build initialization data for a worker due to unsupported source config '$unsupported'." + ) + ) + } + + /** Register the worker within the state data. + * + * @param stateData + * Current state information + * @param timeSeriesUuid + * Unique identifier of the time series, the worker takes care of + * @param workerRef + * [[ActorRef]] to the new worker actor + * @return + * The updated state data, that holds reference to the worker + */ + private def updateStateData( + stateData: PrimaryServiceStateData, + timeSeriesUuid: UUID, + workerRef: ActorRef + ): PrimaryServiceStateData = { + val timeSeriesToSourceRef = stateData.timeSeriesToSourceRef + val sourceRef = timeSeriesToSourceRef.getOrElse( + timeSeriesUuid, + throw new IllegalArgumentException( + s"Cannot update entry for time series '$timeSeriesUuid', as it hasn't been part of it before." + ) + ) + val updatedTimeSeriesToSourceRef = timeSeriesToSourceRef.updated( + timeSeriesUuid, + sourceRef.copy(worker = Some(workerRef)) + ) + stateData.copy(timeSeriesToSourceRef = updatedTimeSeriesToSourceRef) + } } object PrimaryServiceProxy { + + def props( + scheduler: ActorRef, + initStateData: InitPrimaryServiceProxyStateData, + startDateTime: ZonedDateTime + ): Props = Props( + new PrimaryServiceProxy(scheduler, initStateData, startDateTime) + ) + /** State data with needed information to initialize this primary service * provider proxy * @@ -107,12 +514,105 @@ object PrimaryServiceProxy { */ final case class InitPrimaryServiceProxyStateData( primaryConfig: PrimaryConfig, - simulationStart: ZonedDateTime + simulationStart: ZonedDateTime, + extSimulation: Option[ActorRef] ) extends InitializeServiceStateData - abstract class PrimaryServiceStateData( + /** Holding the state of an initialized proxy. + * + * @param modelToTimeSeries + * Mapping from models' to time series unique identifiers + * @param timeSeriesToSourceRef + * Mapping from time series identifier to [[SourceRef]] + * @param simulationStart + * Wall clock time of the first instant in simulation + * @param primaryConfig + * The configuration for the sources + * @param mappingSource + * The mapping source + */ + final case class PrimaryServiceStateData( + modelToTimeSeries: Map[UUID, UUID], + timeSeriesToSourceRef: Map[UUID, SourceRef], simulationStart: ZonedDateTime, - primaryConfig: PrimaryConfig + primaryConfig: PrimaryConfig, + mappingSource: TimeSeriesMappingSource, + extSubscribers: Iterable[UUID] = Iterable.empty[UUID], + extPrimaryDataService: Option[ActorRef] = None ) extends ServiceStateData -} + /** Giving reference to the target time series and source worker. + * + * @param metaInformation + * Meta information (including column scheme) of the time series + * @param worker + * Optional reference to an already existing worker providing information + * on that time series + */ + final case class SourceRef( + metaInformation: IndividualTimeSeriesMetaInformation, + worker: Option[ActorRef] + ) + + /** Check if the config holds correct information to instantiate a mapping + * source + * + * @param primaryConfig + * Config entries for primary source + */ + def checkConfig(primaryConfig: PrimaryConfig): Unit = { + + def checkTimePattern(dtfPattern: String): Unit = + Try { + new SimpleDateFormat(dtfPattern) + } match { + case Failure(exception) => + throw new InvalidConfigParameterException( + s"Invalid timePattern '$dtfPattern' for a time series source. Please provide a valid pattern!" + + s"\nException: $exception" + ) + case Success(_) => + // this is fine + } + + val supportedSources = + Set("csv", "sql") + + val sourceConfigs = Seq( + primaryConfig.couchbaseParams, + primaryConfig.csvParams, + primaryConfig.influxDb1xParams, + primaryConfig.sqlParams + ).filter(_.isDefined).flatten + if (sourceConfigs.size > 1) + throw new InvalidConfigParameterException( + s"${sourceConfigs.size} time series source types defined. " + + s"Please define only one type!\nAvailable types:\n\t${supportedSources.mkString("\n\t")}" + ) + else if (sourceConfigs.isEmpty) + throw new InvalidConfigParameterException( + s"No time series source type defined. Please define exactly one type!" + + s"\nAvailable types:\n\t${supportedSources.mkString("\n\t")}" + ) + else { + sourceConfigs.headOption match { + case Some(csvParams: PrimaryDataCsvParams) => + // note: if inheritance is supported by tscfg, + // the following method should be called for all different supported sources! + checkTimePattern(csvParams.timePattern) + case Some(sqlParams: SqlParams) => + checkTimePattern(sqlParams.timePattern) + case Some(x) => + throw new InvalidConfigParameterException( + s"Invalid configuration '$x' for a time series source.\nAvailable types:\n\t${supportedSources + .mkString("\n\t")}" + ) + case None => + throw new InvalidConfigParameterException( + s"No configuration for a time series mapping source provided.\nPlease provide one of the available sources:\n\t${supportedSources + .mkString("\n\t")}" + ) + } + } + } +} \ No newline at end of file diff --git a/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala b/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala index 6ccd385a6f..8f4bd40bcc 100644 --- a/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala +++ b/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala @@ -19,7 +19,7 @@ import edu.ie3.simona.ontology.messages.services.ResultMessage.ResultResponseMes import edu.ie3.simona.scheduler.TimeAdvancer import edu.ie3.simona.scheduler.TimeAdvancer.StartSimMessage import edu.ie3.simona.sim.SimMessage.{InitSim, SimulationFailure, SimulationSuccessful, StartSimulation} -import edu.ie3.simona.sim.SimonaSim.{EmergencyShutdownInitiated, PowerFlowResultMessage, ResultMessage, SimonaSimStateData} +import edu.ie3.simona.sim.SimonaSim.{EmergencyShutdownInitiated, ResultMessage, SimonaSimStateData} import edu.ie3.simona.sim.setup.{ExtSimSetupData, SimonaSetup} import scala.concurrent.duration.DurationInt @@ -145,11 +145,7 @@ class SimonaSim(simonaSetup: SimonaSetup) /* ExtSimulation */ case ResultMessage(res) => - data.initSimSender ! ( - res.getResults() - ) - - + data.initSimSender ! res.getResults() case EmergencyShutdownInitiated => logger.debug( 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 c325a828dd..b74ade053e 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/ExtSimSetupData.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/ExtSimSetupData.scala @@ -8,7 +8,7 @@ package edu.ie3.simona.sim.setup import org.apache.pekko.actor.ActorRef import edu.ie3.simona.service.ev.ExtEvDataService -import edu.ie3.simona.service.primary.{ExtPrimaryDataService, ExtPrimaryServiceWorker} +import edu.ie3.simona.service.primary.ExtPrimaryDataService final case class ExtSimSetupData( extSimAdapters: Iterable[ActorRef], 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 b2b3398eeb..6295ffe6de 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala @@ -32,8 +32,8 @@ import edu.ie3.simona.scheduler.{ScheduleLock, Scheduler, TimeAdvancer} import edu.ie3.simona.service.SimonaService import edu.ie3.simona.service.ev.ExtEvDataService import edu.ie3.simona.service.ev.ExtEvDataService.InitExtEvData -import edu.ie3.simona.service.primary.ExtPrimaryServiceWorker.InitExtPrimaryData -import edu.ie3.simona.service.primary.{ExtPrimaryServiceWorker, IntPrimaryServiceProxy, PrimaryServiceProxy} +import edu.ie3.simona.service.primary.ExtPrimaryDataService.InitExtPrimaryData +import edu.ie3.simona.service.primary.{ExtPrimaryDataService, PrimaryServiceProxy} import edu.ie3.simona.service.primary.PrimaryServiceProxy.InitPrimaryServiceProxyStateData import edu.ie3.simona.service.results.ExtResultDataService import edu.ie3.simona.service.results.ExtResultDataService.InitExtResultsData @@ -145,11 +145,12 @@ class SimonaStandaloneSetup( simonaConfig.simona.time.startDateTime ) val primaryServiceProxy = context.simonaActorOf( - IntPrimaryServiceProxy.props( + PrimaryServiceProxy.props( scheduler, InitPrimaryServiceProxyStateData( simonaConfig.simona.input.primary, - simulationStart + simulationStart, + extSimSetupData.extPrimaryDataService ), simulationStart ) @@ -231,7 +232,7 @@ class SimonaStandaloneSetup( case (_: ExtPrimaryDataSimulation, dIndex) => val extPrimaryDataService = context.simonaActorOf( - ExtPrimaryServiceWorker.props(scheduler), + ExtPrimaryDataService.props(scheduler), s"$index-$dIndex" ) val extPrimaryData = new ExtPrimaryData(extPrimaryDataService, extSimAdapter) diff --git a/src/test/scala/edu/ie3/simona/service/primary/PrimaryServiceProxySpec.scala b/src/test/scala/edu/ie3/simona/service/primary/PrimaryServiceProxySpec.scala index 25f32e4c46..2a3d10edea 100644 --- a/src/test/scala/edu/ie3/simona/service/primary/PrimaryServiceProxySpec.scala +++ b/src/test/scala/edu/ie3/simona/service/primary/PrimaryServiceProxySpec.scala @@ -238,7 +238,8 @@ class PrimaryServiceProxySpec val initStateData: InitPrimaryServiceProxyStateData = InitPrimaryServiceProxyStateData( validPrimaryConfig, - simulationStart + simulationStart, + None ) val proxyRef: TestActorRef[PrimaryServiceProxy] = TestActorRef( new PrimaryServiceProxy(scheduler.ref, initStateData, simulationStart) @@ -300,7 +301,9 @@ class PrimaryServiceProxySpec timeSeriesToSourceRef, simulationStart, primaryConfig, - mappingSource + mappingSource, + extSubscribers, + extPrimaryDataService ) ) => modelToTimeSeries shouldBe Map( @@ -561,7 +564,9 @@ class PrimaryServiceProxySpec timeSeriesToSourceRef, simulationStart, primaryConfig, - mappingSource + mappingSource, + extSubscribers, + extPrimaryDataService ) => modelToTimeSeries shouldBe proxyStateData.modelToTimeSeries timeSeriesToSourceRef shouldBe Map( diff --git a/src/test/scala/edu/ie3/simona/service/primary/PrimaryServiceProxySqlIT.scala b/src/test/scala/edu/ie3/simona/service/primary/PrimaryServiceProxySqlIT.scala index 99418e035f..17391bfe84 100644 --- a/src/test/scala/edu/ie3/simona/service/primary/PrimaryServiceProxySqlIT.scala +++ b/src/test/scala/edu/ie3/simona/service/primary/PrimaryServiceProxySqlIT.scala @@ -98,7 +98,8 @@ class PrimaryServiceProxySqlIT None, sqlParams = Some(sqlParams) ), - simulationStart + simulationStart, + None ) TestActorRef( diff --git a/src/test/scala/edu/ie3/simona/sim/SimonaSimFailSpec.scala b/src/test/scala/edu/ie3/simona/sim/SimonaSimFailSpec.scala index 27c6c80fae..2ef5fff545 100644 --- a/src/test/scala/edu/ie3/simona/sim/SimonaSimFailSpec.scala +++ b/src/test/scala/edu/ie3/simona/sim/SimonaSimFailSpec.scala @@ -112,7 +112,8 @@ object SimonaSimFailSpec { override def primaryServiceProxy( context: ActorContext, - scheduler: ActorRef + scheduler: ActorRef, + extSimSetupData: ExtSimSetupData ): ActorRef = TestProbe("primaryService")(actorSystem).ref 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 26b6246e42..d43550718a 100644 --- a/src/test/scala/edu/ie3/simona/sim/setup/SimonaSetupSpec.scala +++ b/src/test/scala/edu/ie3/simona/sim/setup/SimonaSetupSpec.scala @@ -38,7 +38,8 @@ class SimonaSetupSpec extends UnitSpec with SimonaSetup with SubGridGateMokka { override def primaryServiceProxy( context: ActorContext, - scheduler: ActorRef + scheduler: ActorRef, + extSimSetupData: ExtSimSetupData ): ActorRef = throw new NotImplementedException("This is a dummy setup") From 778b4e631e07723958196a29e52038176ed84356 Mon Sep 17 00:00:00 2001 From: Johannes Bao Date: Mon, 19 Feb 2024 15:59:23 +0100 Subject: [PATCH 03/41] in development --- build.gradle | 4 +- docs/uml/protocol/ExtDataService.puml | 102 +++++++++++++ .../edu/ie3/simona/event/ResultEvent.scala | 29 +--- .../event/listener/ResultEventListener.scala | 39 +++-- .../messages/services/ResultMessage.scala | 2 +- .../ie3/simona/service/SimonaService.scala | 14 +- .../primary/ExtPrimaryDataService.scala | 9 +- .../results/ExtResultDataService.scala | 47 +++--- .../scala/edu/ie3/simona/sim/SimonaSim.scala | 11 +- .../simona/sim/setup/ExtSimSetupData.scala | 2 + .../ie3/simona/sim/setup/SimonaSetup.scala | 3 +- .../sim/setup/SimonaStandaloneSetup.scala | 13 +- .../primary/ExtPrimaryDataServiceSpec.scala | 139 ++++++++++++++++++ .../service/PrimaryDataFactoryDefault.scala | 10 ++ .../common/service/ResultFactoryDefault.scala | 22 +++ 15 files changed, 370 insertions(+), 76 deletions(-) create mode 100644 docs/uml/protocol/ExtDataService.puml create mode 100644 src/test/scala/edu/ie3/simona/service/primary/ExtPrimaryDataServiceSpec.scala create mode 100644 src/test/scala/edu/ie3/simona/test/common/service/PrimaryDataFactoryDefault.scala create mode 100644 src/test/scala/edu/ie3/simona/test/common/service/ResultFactoryDefault.scala diff --git a/build.gradle b/build.gradle index 016036315b..39e47a17a1 100644 --- a/build.gradle +++ b/build.gradle @@ -7,7 +7,7 @@ plugins { id 'signing' id 'maven-publish' // publish to a maven repo (local or mvn central, has to be defined) id 'pmd' // code check, working on source code - id 'com.diffplug.spotless' version '6.23.3'// code format + id 'com.diffplug.spotless' version '6.24.0'// code format id "com.github.ben-manes.versions" version '0.50.0' id "de.undercouch.download" version "5.5.0" // downloads plugin id "kr.motd.sphinx" version "2.10.1" // documentation generation @@ -107,7 +107,7 @@ dependencies { /* testing */ testImplementation 'org.spockframework:spock-core:2.3-groovy-4.0' testImplementation 'org.scalatestplus:mockito-3-4_2.13:3.2.10.0' - testImplementation 'org.mockito:mockito-core:5.8.0' // mocking framework + testImplementation 'org.mockito:mockito-core:5.9.0' // mocking framework testImplementation "org.scalatest:scalatest_${scalaVersion}:3.2.17" testRuntimeOnly 'com.vladsch.flexmark:flexmark-all:0.64.8' //scalatest html output testImplementation group: 'org.pegdown', name: 'pegdown', version: '1.6.0' diff --git a/docs/uml/protocol/ExtDataService.puml b/docs/uml/protocol/ExtDataService.puml new file mode 100644 index 0000000000..283eed4175 --- /dev/null +++ b/docs/uml/protocol/ExtDataService.puml @@ -0,0 +1,102 @@ +@startuml + +!theme plain + +==Init== + + + + +SimScheduler -> ExtSimAdapter: ! ActivityStartTrigger(-1L) +activate ExtSimAdapter + +ExtSimAdapter -> ExtSimulation: queue(ActivityStartTrigger(-1L)) +deactivate ExtSimAdapter +activate ExtSimulation +... Initialize external mobility simulation ... + +ExtSimulation -> ExtSimAdapter: ! CompletionMessage(newTriggers) +deactivate ExtSimulation +activate ExtSimAdapter + +ExtSimAdapter -> SimScheduler: ! CompletionMessage(newTriggers) +deactivate ExtSimAdapter + +==Sim== +SimScheduler -> ExtSimAdapter: ! ActivityStartTrigger(tick) +activate ExtSimAdapter + +ExtSimAdapter -> ExtSimulation: queue(ActivityStartTrigger(tick)) +deactivate ExtSimAdapter + +activate ExtSimulation +ExtSimulation -> ExtEvDataService: ! RequestEvcsFreeLots +ExtSimulation -> ExtSimAdapter: ! ScheduleDataServiceMessage(\n\tdataServiceRef\n) + +activate ExtSimAdapter +ExtSimAdapter -> SimScheduler: ! ScheduleTriggerMessage(\n\t_, dataServiceRef) +deactivate ExtSimAdapter + +activate SimScheduler +SimScheduler -> ExtEvDataService: ! ActivityStartTrigger(tick) +deactivate SimScheduler + +activate ExtEvDataService +ExtEvDataService -> EvcsAgent1: ! EvFreeLotsRequest(tick) +activate EvcsAgent1 +ExtEvDataService -> EvcsAgent2: ! EvFreeLotsRequest(tick) +activate EvcsAgent2 + +ExtEvDataService -> SimScheduler: ! CompletionMessage(None) + +EvcsAgent2 -> ExtEvDataService: ! FreeLotsResponse(_, _) +deactivate EvcsAgent2 +EvcsAgent1 -> ExtEvDataService: ! FreeLotsResponse(_, _) +deactivate EvcsAgent1 +ExtEvDataService -> ExtSimulation: queue(ProvideEvcsFreeLots(_)) +deactivate ExtEvDataService + +... Running external mobility simulation,\n determining EV positions ... +ExtSimulation -> ExtEvDataService: ! EvMovementsMessage(_) +ExtSimulation -> ExtSimAdapter: ! ScheduleDataServiceMessage(\n\tdataServiceRef\n) + +activate ExtSimAdapter +ExtSimAdapter -> SimScheduler: ! ScheduleTriggerMessage(\n\t_, dataServiceRef) +deactivate ExtSimAdapter + +activate SimScheduler +SimScheduler -> ExtEvDataService: ! ActivityStartTrigger(tick) +deactivate SimScheduler + +activate ExtEvDataService +ExtEvDataService -> EvcsAgent1: ! ProvideEvDataMessage(\n\ttick, _) +ExtEvDataService -> EvcsAgent2: ! ProvideEvDataMessage(\n\ttick, _) +ExtEvDataService -> SimScheduler: ! CompletionMessage(evcsTriggers) +deactivate ExtEvDataService + +activate SimScheduler +SimScheduler -> EvcsAgent1: ! ActivityStartTrigger(tick) +activate EvcsAgent1 +SimScheduler -> EvcsAgent2: ! ActivityStartTrigger(tick) +deactivate SimScheduler + +activate EvcsAgent2 +EvcsAgent1 -> SimScheduler: ! CompletionMessage(None) +deactivate EvcsAgent1 + +EvcsAgent2 -> ExtEvDataService: ! DepartedEvsResponse(_, _) +activate ExtEvDataService +EvcsAgent2 -> SimScheduler: ! CompletionMessage(None) +deactivate EvcsAgent2 + +ExtEvDataService -> ExtSimulation: queue(AllDepartedEvsResponse(_)) +deactivate ExtEvDataService + +ExtSimulation -> ExtSimAdapter: ! CompletionMessage(newTriggers) +deactivate ExtSimulation + +activate ExtSimAdapter +ExtSimAdapter -> SimScheduler: ! CompletionMessage(newTriggers) +deactivate ExtSimAdapter + +@enduml \ No newline at end of file diff --git a/src/main/scala/edu/ie3/simona/event/ResultEvent.scala b/src/main/scala/edu/ie3/simona/event/ResultEvent.scala index 7e036a7d68..83157adb70 100644 --- a/src/main/scala/edu/ie3/simona/event/ResultEvent.scala +++ b/src/main/scala/edu/ie3/simona/event/ResultEvent.scala @@ -14,14 +14,16 @@ import edu.ie3.simona.agent.grid.GridResultsSupport.PartialTransformer3wResult import edu.ie3.simona.event.listener.ResultEventListener.ResultMessage -sealed trait ResultEvent extends ResultMessage { - def getResults(): Iterable[ResultEntity] -} +sealed trait ResultEvent extends ResultMessage /** Calculation result events */ object ResultEvent { + final case class ExternalResultEvent( + resultEntity: ResultEntity + ) extends ResultEvent + /** Event that holds a calculation result of a * [[edu.ie3.simona.model.participant.SystemParticipant]] * @@ -30,9 +32,7 @@ object ResultEvent { */ final case class ParticipantResultEvent( systemParticipantResult: SystemParticipantResult - ) extends ResultEvent { - override def getResults(): Iterable[ResultEntity] = Iterable(systemParticipantResult) - } + ) extends ResultEvent /** Event, that is triggered every time a thermal model has a new result * @param thermalResult @@ -40,9 +40,7 @@ object ResultEvent { */ final case class ThermalResultEvent( thermalResult: ThermalUnitResult - ) extends ResultEvent { - override def getResults(): Iterable[ResultEntity] = Iterable(thermalResult) - } + ) extends ResultEvent /** Event that holds all grid calculation results of a power flow calculation. * The usage of a type is necessary here, to avoid passing in other instances @@ -66,16 +64,5 @@ object ResultEvent { lineResults: Iterable[LineResult], transformer2wResults: Iterable[Transformer2WResult], transformer3wResults: Iterable[PartialTransformer3wResult] - ) extends ResultEvent { - override def getResults(): Iterable[ResultEntity] = { - var results: Iterable[ResultEntity] = Iterable.empty[ResultEntity] - results = results ++ nodeResults - results = results ++ switchResults - results = results ++ lineResults - results = results ++ transformer2wResults - // results = results ++ transformer3wResults - results - } - } - + ) extends ResultEvent } diff --git a/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala b/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala index 38539b2baa..30ce73b2f8 100644 --- a/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala +++ b/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala @@ -12,18 +12,14 @@ import edu.ie3.datamodel.io.processor.result.ResultEntityProcessor import edu.ie3.datamodel.models.result.{NodeResult, ResultEntity} import edu.ie3.simona.agent.grid.GridResultsSupport.PartialTransformer3wResult import edu.ie3.simona.event.Event -import edu.ie3.simona.event.ResultEvent.{ - ParticipantResultEvent, - PowerFlowResultEvent, - ThermalResultEvent -} -import edu.ie3.simona.exceptions.{ - FileHierarchyException, - ProcessResultEventException -} +import edu.ie3.simona.event.ResultEvent.{ExternalResultEvent, ParticipantResultEvent, PowerFlowResultEvent, ThermalResultEvent} +import edu.ie3.simona.exceptions.{FileHierarchyException, ProcessResultEventException} import edu.ie3.simona.io.result._ import edu.ie3.simona.ontology.messages.StopMessage +import edu.ie3.simona.ontology.messages.services.ResultMessage.ResultResponseMessage +import edu.ie3.simona.sim.SimonaSim.ResultMessage import edu.ie3.simona.util.ResultFileHierarchy +import org.apache.pekko.actor.ActorRef import org.slf4j.Logger import scala.concurrent.ExecutionContext.Implicits.global @@ -43,15 +39,19 @@ object ResultEventListener extends Transformer3wResultSupport { private final case object StopTimeout extends ResultMessage + /** [[ResultEventListener]] base data containing all information the listener * needs * * @param classToSink * a map containing the sink for each class that should be processed by the * listener + * @param extResultDataService + * actor for the external data service */ private final case class BaseData( classToSink: Map[Class[_], ResultEntitySink], + extResultDataService: Option[ActorRef], threeWindingResults: Map[ Transformer3wKey, AggregatedTransformer3wResult @@ -157,6 +157,9 @@ object ResultEventListener extends Transformer3wResultSupport { log: Logger ): BaseData = { handOverToSink(resultEntity, baseData.classToSink, log) + if (baseData.extResultDataService.isDefined) { + handOverToExternalService(resultEntity, baseData.extResultDataService, log) + } baseData } @@ -231,8 +234,18 @@ object ResultEventListener extends Transformer3wResultSupport { log.error("Error while writing result event: ", exception) } + private def handOverToExternalService( + resultEntity: ResultEntity, + extResultDataService: Option[ActorRef], + log: Logger + ): Unit = Try { + val extResultDataServiceRef = extResultDataService.getOrElse(throw new Exception("No external data service registered!")) + extResultDataServiceRef ! ResultResponseMessage(resultEntity) + } + def apply( - resultFileHierarchy: ResultFileHierarchy + resultFileHierarchy: ResultFileHierarchy, + extResultDataService: Option[ActorRef] = Option.empty[ActorRef] ): Behavior[ResultMessage] = Behaviors.setup[ResultMessage] { ctx => ctx.log.debug("Starting initialization!") resultFileHierarchy.resultSinkType match { @@ -256,15 +269,15 @@ object ResultEventListener extends Transformer3wResultSupport { case Success(result) => SinkResponse(result.toMap) } - init() + init(extResultDataService) } - private def init(): Behavior[ResultMessage] = Behaviors.withStash(200) { + private def init(extResultDataService: Option[ActorRef]): Behavior[ResultMessage] = Behaviors.withStash(200) { buffer => Behaviors.receive[ResultMessage] { case (ctx, SinkResponse(response)) => ctx.log.debug("Initialization complete!") - buffer.unstashAll(idle(BaseData(response))) + buffer.unstashAll(idle(BaseData(response, extResultDataService))) case (ctx, Failed(ex)) => ctx.log.error("Unable to setup ResultEventListener.", ex) diff --git a/src/main/scala/edu/ie3/simona/ontology/messages/services/ResultMessage.scala b/src/main/scala/edu/ie3/simona/ontology/messages/services/ResultMessage.scala index d5dd04b7a2..6e17f11554 100644 --- a/src/main/scala/edu/ie3/simona/ontology/messages/services/ResultMessage.scala +++ b/src/main/scala/edu/ie3/simona/ontology/messages/services/ResultMessage.scala @@ -7,5 +7,5 @@ sealed trait ResultMessage extends DataMessage object ResultMessage { final case class ResultRequest(tick: Long) - final case class ResultResponseMessage(results: Iterable[ResultEntity]) extends ResultMessage + final case class ResultResponseMessage(result: ResultEntity) extends ResultMessage } \ No newline at end of file diff --git a/src/main/scala/edu/ie3/simona/service/SimonaService.scala b/src/main/scala/edu/ie3/simona/service/SimonaService.scala index 89d60e57c4..5742867512 100644 --- a/src/main/scala/edu/ie3/simona/service/SimonaService.scala +++ b/src/main/scala/edu/ie3/simona/service/SimonaService.scala @@ -6,21 +6,17 @@ package edu.ie3.simona.service +import edu.ie3.simona.event +import edu.ie3.simona.event.Event import org.apache.pekko.actor.typed.scaladsl.adapter.ClassicActorRefOps import org.apache.pekko.actor.{Actor, ActorContext, ActorRef, Stash} import edu.ie3.simona.logging.SimonaActorLogging import edu.ie3.simona.ontology.messages.Activation -import edu.ie3.simona.ontology.messages.SchedulerMessage.{ - Completion, - ScheduleActivation -} +import edu.ie3.simona.ontology.messages.SchedulerMessage.{Completion, ScheduleActivation} import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.ScheduleServiceActivation import edu.ie3.simona.ontology.messages.services.ServiceMessage.ServiceRegistrationMessage import edu.ie3.simona.scheduler.ScheduleLock.ScheduleKey -import edu.ie3.simona.service.ServiceStateData.{ - InitializeServiceStateData, - ServiceBaseStateData -} +import edu.ie3.simona.service.ServiceStateData.{InitializeServiceStateData, ServiceBaseStateData} import edu.ie3.simona.service.SimonaService.Create import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK @@ -132,6 +128,7 @@ abstract class SimonaService[ private def idleInternal(implicit stateData: S): Receive = { // agent registration process + case registrationMsg: ServiceRegistrationMessage => /* Someone asks to register for information from the service */ handleRegistrationRequest(registrationMsg) match { @@ -168,6 +165,7 @@ abstract class SimonaService[ unhandled(x) } + /** Internal api method that allows handling incoming messages from external * simulations * diff --git a/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala b/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala index 6152d5c409..1039bedd8d 100644 --- a/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala +++ b/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala @@ -35,7 +35,7 @@ object ExtPrimaryDataService { extPrimaryDataMessage: Option[PrimaryDataMessageFromExt] = None ) extends ServiceBaseStateData - final case class InitExtPrimaryData( + case class InitExtPrimaryData( extPrimaryData: ExtPrimaryData ) extends InitializeServiceStateData @@ -71,8 +71,11 @@ final case class ExtPrimaryDataService( override protected def handleRegistrationRequest( registrationMessage: ServiceMessage.ServiceRegistrationMessage )(implicit serviceStateData: ExtPrimaryDataStateData): - Try[ExtPrimaryDataStateData] = registrationMessage match { + Try[ExtPrimaryDataStateData] = { + println("Habe erhalten 0") + registrationMessage match { case ExtPrimaryDataServiceRegistrationMessage(modelUuid, requestingActor) => + println("Habe erhalten") Success(handleRegistrationRequest(requestingActor, modelUuid)) case invalidMessage => Failure( @@ -80,9 +83,11 @@ final case class ExtPrimaryDataService( s"A primary service provider is not able to handle registration request '$invalidMessage'." ) ) + } } + private def handleRegistrationRequest( agentToBeRegistered: ActorRef, agentUUID: UUID diff --git a/src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala b/src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala index 132d4f59ae..37f2acc415 100644 --- a/src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala +++ b/src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala @@ -1,5 +1,6 @@ package edu.ie3.simona.service.results +import edu.ie3.datamodel.models.result.ResultEntity import edu.ie3.simona.api.data.ontology.DataMessageFromExt import edu.ie3.simona.api.data.results.ExtResultsData import edu.ie3.simona.api.data.results.ontology.{ProvideResultEntities, RequestResultEntities, ResultDataMessageFromExt} @@ -10,6 +11,7 @@ import edu.ie3.simona.ontology.messages.services.DataMessage import edu.ie3.simona.service.ServiceStateData.{InitializeServiceStateData, ServiceBaseStateData} import edu.ie3.simona.service.{ExtDataSupport, SimonaService} import edu.ie3.simona.service.results.ExtResultDataService.{ExtResultsStateData, InitExtResultsData} + import org.apache.pekko.actor.{ActorContext, ActorRef, Props} import java.util.UUID @@ -25,7 +27,8 @@ object ExtResultDataService { final case class ExtResultsStateData( extResultsData: ExtResultsData, uuidToActorRef: Map[UUID, ActorRef] = Map.empty[UUID, ActorRef], - extResultsMessage: Option[ResultDataMessageFromExt] = None + extResultsMessage: Option[ResultDataMessageFromExt] = None, + resultStorage: Map[UUID, ResultEntity] = Map.empty ) extends ServiceBaseStateData final case class InitExtResultsData( @@ -70,10 +73,9 @@ class ExtResultDataService(override val scheduler: ActorRef) )( implicit serviceStateData: ExtResultsStateData, ctx: ActorContext): (ExtResultsStateData, Option[Long]) = { - serviceStateData.extResultsMessage.getOrElse( throw ServiceException( - "ExtEvDataService was triggered without ExtEvMessage available" + "ExtResultDataService was triggered without ResultDataMessageFromExt available" ) ) match { case _: RequestResultEntities => @@ -108,28 +110,39 @@ class ExtResultDataService(override val scheduler: ActorRef) * @return * the updated state data */ - override protected def handleDataResponseMessage( - extResponseMsg: DataMessage - )( + override protected def handleDataResponseMessage(extResponseMsg: DataMessage)( implicit serviceStateData: ExtResultsStateData): ExtResultsStateData = { extResponseMsg match { - case ResultResponseMessage(results) => - serviceStateData.extResultsData.queueExtResponseMsg( - new ProvideResultEntities(results.toList.asJava)) - serviceStateData.copy() + case ResultResponseMessage(result) => + if (serviceStateData.uuidToActorRef.contains(result.getUuid)) { + // A valid result was sent + val updatedResultStorage = serviceStateData.resultStorage + (result.getUuid -> result) + if (updatedResultStorage.size == serviceStateData.uuidToActorRef.size) { + // all responses received, forward them to external simulation in a bundle + serviceStateData.extResultsData.queueExtResponseMsg( + new ProvideResultEntities(updatedResultStorage.values.toList.asJava) + ) + serviceStateData.copy( + resultStorage = Map.empty + ) + + + } else { + // responses are still incomplete + serviceStateData.copy( + resultStorage = updatedResultStorage + ) + } + } + else { + serviceStateData + } } } private def requestResults( tick: Long )(implicit serviceStateData: ExtResultsStateData): (ExtResultsStateData, Option[Long]) = { - (serviceStateData.copy(), None) } - - - - - - } diff --git a/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala b/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala index 8f4bd40bcc..79fab9885f 100644 --- a/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala +++ b/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala @@ -56,10 +56,6 @@ class SimonaSim(simonaSetup: SimonaSetup) } /* start listener */ - // output listener - val systemParticipantsListener: Seq[ActorRef] = - simonaSetup.systemParticipantsListener(context) - // runtime event listener val runtimeEventListener : org.apache.pekko.actor.typed.ActorRef[RuntimeEvent] = @@ -84,6 +80,9 @@ class SimonaSim(simonaSetup: SimonaSetup) val weatherService: ActorRef = simonaSetup.weatherService(context, scheduler) + // output listener + val systemParticipantsListener: Seq[ActorRef] = + simonaSetup.systemParticipantsListener(context, extSimulationData) /* start grid agents */ val gridAgents: Iterable[ActorRef] = simonaSetup.gridAgents( @@ -144,8 +143,8 @@ class SimonaSim(simonaSetup: SimonaSetup) ) /* ExtSimulation */ - case ResultMessage(res) => - data.initSimSender ! res.getResults() + //case ResultMessage(res) => + // data.initSimSender ! res.getResults() case EmergencyShutdownInitiated => logger.debug( 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 b74ade053e..b592f261e4 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/ExtSimSetupData.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/ExtSimSetupData.scala @@ -9,6 +9,7 @@ package edu.ie3.simona.sim.setup import org.apache.pekko.actor.ActorRef import edu.ie3.simona.service.ev.ExtEvDataService import edu.ie3.simona.service.primary.ExtPrimaryDataService +import edu.ie3.simona.service.results.ExtResultDataService final case class ExtSimSetupData( extSimAdapters: Iterable[ActorRef], @@ -20,4 +21,5 @@ final case class ExtSimSetupData( def extPrimaryDataService: Option[ActorRef] = extDataServices.get(classOf[ExtPrimaryDataService]) + def extResultDataService: Option[ActorRef] = extDataServices.get(classOf[ExtResultDataService]) } 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 618a9521f1..fd1e21d262 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaSetup.scala @@ -54,7 +54,8 @@ trait SimonaSetup { * A sequence of actor references to runtime event listeners */ def systemParticipantsListener( - context: ActorContext + context: ActorContext, + extSimulationData: ExtSimSetupData ): Seq[ActorRef] /** Creates a primary service proxy. The proxy is the first instance to ask 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 6295ffe6de..2ad94f46ec 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala @@ -230,12 +230,12 @@ class SimonaStandaloneSetup( (extEvData, (classOf[ExtEvDataService], extEvDataService)) - case (_: ExtPrimaryDataSimulation, dIndex) => + case (extPrimaryDataSimulation: ExtPrimaryDataSimulation, dIndex) => val extPrimaryDataService = context.simonaActorOf( ExtPrimaryDataService.props(scheduler), s"$index-$dIndex" ) - val extPrimaryData = new ExtPrimaryData(extPrimaryDataService, extSimAdapter) + val extPrimaryData = new ExtPrimaryData(extPrimaryDataService, extSimAdapter, extPrimaryDataSimulation.getFactory) extPrimaryDataService ! SimonaService.Create( InitExtPrimaryData(extPrimaryData), @@ -253,7 +253,7 @@ class SimonaStandaloneSetup( ExtResultDataService.props(scheduler), s"$index-$dIndex" ) - val extResultsData = new ExtResultsData(extResultDataService, extSimAdapter) + val extResultsData = new ExtResultsData(extResultDataService, extSimAdapter, null) extResultDataService ! SimonaService.Create( InitExtResultsData(extResultsData), @@ -332,8 +332,10 @@ class SimonaStandaloneSetup( ) override def systemParticipantsListener( - context: ActorContext + context: ActorContext, + extSimulationData: ExtSimSetupData ): Seq[ActorRef] = { + val extResultDataService: Option[ActorRef] = extSimulationData.extResultDataService // append ResultEventListener as well to write raw output files ArgsParser .parseListenerConfigOption(simonaConfig.simona.event.listener) @@ -347,7 +349,8 @@ class SimonaStandaloneSetup( .toSeq :+ context .spawn( ResultEventListener( - resultFileHierarchy + resultFileHierarchy, + extResultDataService ), ResultEventListener.getClass.getSimpleName ) diff --git a/src/test/scala/edu/ie3/simona/service/primary/ExtPrimaryDataServiceSpec.scala b/src/test/scala/edu/ie3/simona/service/primary/ExtPrimaryDataServiceSpec.scala new file mode 100644 index 0000000000..66458ef384 --- /dev/null +++ b/src/test/scala/edu/ie3/simona/service/primary/ExtPrimaryDataServiceSpec.scala @@ -0,0 +1,139 @@ +/* + * © 2021. TU Dortmund University, + * Institute of Energy Systems, Energy Efficiency and Energy Economics, + * Research group Distribution grid planning and operation + */ + +package edu.ie3.simona.service.primary + +import com.typesafe.config.ConfigFactory +import edu.ie3.simona.test.common.service.{PrimaryDataFactoryDefault, ResultFactoryDefault} +import edu.ie3.datamodel.io.factory.timeseries.TimeBasedSimpleValueFactory +import edu.ie3.datamodel.io.naming.FileNamingStrategy +import edu.ie3.datamodel.io.source.csv.CsvTimeSeriesSource +import edu.ie3.datamodel.models.StandardUnits +import edu.ie3.datamodel.models.value.{HeatDemandValue, PValue} +import edu.ie3.simona.agent.participant.data.Data.PrimaryData.ActivePower +import edu.ie3.simona.api.data.ev.ExtEvData +import edu.ie3.simona.api.data.ev.model.EvModel +import edu.ie3.simona.api.data.ev.ontology._ +import edu.ie3.simona.api.data.ontology.ScheduleDataServiceMessage +import edu.ie3.simona.api.data.primarydata.ExtPrimaryData +import edu.ie3.simona.exceptions.ServiceException +import edu.ie3.simona.ontology.messages.Activation +import edu.ie3.simona.ontology.messages.SchedulerMessage.{Completion, ScheduleActivation} +import edu.ie3.simona.ontology.messages.services.EvMessage._ +import edu.ie3.simona.ontology.messages.services.ServiceMessage.{ExtPrimaryDataServiceRegistrationMessage, WorkerRegistrationMessage} +import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.RegistrationSuccessfulMessage +import edu.ie3.simona.ontology.messages.services.WeatherMessage.RegisterForWeatherMessage +import edu.ie3.simona.scheduler.ScheduleLock +import edu.ie3.simona.service.SimonaService +import edu.ie3.simona.service.ev.ExtEvDataService.InitExtEvData +import edu.ie3.simona.service.primary.ExtPrimaryDataService.InitExtPrimaryData +import edu.ie3.simona.service.primary.PrimaryServiceWorker.{CsvInitPrimaryServiceStateData, PrimaryServiceInitializedStateData, ProvidePrimaryDataMessage} +import edu.ie3.simona.service.primary.PrimaryServiceWorkerSpec.WrongInitPrimaryServiceStateData +import edu.ie3.simona.test.common.{EvTestData, TestKitWithShutdown, TestSpawnerClassic} +import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK +import edu.ie3.util.TimeUtil +import edu.ie3.util.quantities.PowerSystemUnits +import edu.ie3.util.scala.collection.immutable.SortedDistinctSeq +import org.apache.pekko.actor.typed.scaladsl.adapter.ClassicActorRefOps +import org.apache.pekko.actor.{ActorRef, ActorSystem} +import org.apache.pekko.testkit.{TestActorRef, TestProbe} +import org.scalatest.wordspec.AnyWordSpecLike +import squants.energy.Kilowatts +import tech.units.indriya.quantity.Quantities + +import java.nio.file.Paths +import java.time.ZonedDateTime +import java.util.UUID +import scala.concurrent.duration.DurationInt +import scala.jdk.CollectionConverters._ +import scala.util.{Failure, Success} + +class ExtPrimaryDataServiceSpec + extends TestKitWithShutdown( + ActorSystem( + "ExtPrimaryDataServiceSpec", + ConfigFactory + .parseString(""" + |pekko.loggers = ["org.apache.pekko.testkit.TestEventListener"] + |pekko.loglevel = "INFO" + |""".stripMargin) + ) + ) + with TestSpawnerClassic { + + private val scheduler = TestProbe("scheduler") + private val extSimAdapter = TestProbe("extSimAdapter") + + private val primaryDataFactory = new PrimaryDataFactoryDefault + + private val extPrimaryData = (dataService: ActorRef) => + new ExtPrimaryData( + dataService, + extSimAdapter.ref, + primaryDataFactory + ) + + private val participant1UUID = + UUID.fromString("06a14909-366e-4e94-a593-1016e1455b30") + + + "An uninitialized external primary data service" must { + + + "send correct completion message after initialisation" in { + val primaryDataService = TestActorRef( + new ExtPrimaryDataService( + scheduler.ref + ) + ) + + val key = + ScheduleLock.singleKey(TSpawner, scheduler.ref.toTyped, INIT_SIM_TICK) + scheduler.expectMsgType[ScheduleActivation] // lock activation scheduled + + scheduler.send( + primaryDataService, + SimonaService.Create(InitExtPrimaryData(extPrimaryData(primaryDataService)), key) + ) + scheduler.expectMsg( + ScheduleActivation(primaryDataService.toTyped, INIT_SIM_TICK, Some(key)) + ) + + scheduler.send(primaryDataService, Activation(INIT_SIM_TICK)) + scheduler.expectMsg(Completion(primaryDataService.toTyped)) + } + } + + "An external primary service actor" should { + val serviceRef = + TestActorRef( + new ExtPrimaryDataService( + scheduler.ref + ) + ) + val validInitData = InitExtPrimaryData(extPrimaryData(serviceRef)) + + "refuse registration for wrong registration request" in { + serviceRef ! RegisterForWeatherMessage(51.4843281, 7.4116482) + expectNoMessage() + } + + val systemParticipant: TestProbe = TestProbe("dummySystemParticipant") + "correctly register a forwarded request" in { + serviceRef ! ExtPrimaryDataServiceRegistrationMessage(UUID.randomUUID(), systemParticipant.ref) + println("Try to register") + + /* Wait for request approval */ + systemParticipant.expectMsg(RegistrationSuccessfulMessage(Some(0L))) + + /* We cannot directly check, if the requesting actor is among the subscribers, therefore we ask the actor to + * provide data to all subscribed actors and check, if the subscribed probe gets one */ + scheduler.send(serviceRef, Activation(0)) + scheduler.expectMsgType[Completion] + systemParticipant.expectMsgAllClassOf(classOf[ProvidePrimaryDataMessage]) + } + } +} \ No newline at end of file diff --git a/src/test/scala/edu/ie3/simona/test/common/service/PrimaryDataFactoryDefault.scala b/src/test/scala/edu/ie3/simona/test/common/service/PrimaryDataFactoryDefault.scala new file mode 100644 index 0000000000..7f7ce9c1fd --- /dev/null +++ b/src/test/scala/edu/ie3/simona/test/common/service/PrimaryDataFactoryDefault.scala @@ -0,0 +1,10 @@ +package edu.ie3.simona.test.common.service + +import edu.ie3.datamodel.models.value.Value +import edu.ie3.simona.api.data.primarydata.PrimaryDataFactory + +class PrimaryDataFactoryDefault extends PrimaryDataFactory { + + /** Should convert an object to an primary data value with a check if the object is primary data */ + override def convertObjectToValue(entity: Any): Value = ??? +} diff --git a/src/test/scala/edu/ie3/simona/test/common/service/ResultFactoryDefault.scala b/src/test/scala/edu/ie3/simona/test/common/service/ResultFactoryDefault.scala new file mode 100644 index 0000000000..897bc03b4d --- /dev/null +++ b/src/test/scala/edu/ie3/simona/test/common/service/ResultFactoryDefault.scala @@ -0,0 +1,22 @@ +package edu.ie3.simona.test.common.service + +import edu.ie3.datamodel.models.result.{NodeResult, ResultEntity} +import edu.ie3.datamodel.models.result.connector.ConnectorResult +import edu.ie3.datamodel.models.result.system.{ElectricalEnergyStorageResult, SystemParticipantResult, SystemParticipantWithHeatResult} +import edu.ie3.datamodel.models.result.thermal.ThermalUnitResult +import edu.ie3.simona.api.data.results.ResultDataFactory +class ResultFactoryDefault extends ResultDataFactory { + @throws[Exception] + override def convertResultToString(entity: ResultEntity): AnyRef = { + var resultObject: String = null + if (entity.isInstanceOf[SystemParticipantWithHeatResult]) resultObject = "{\"p\":\"" + systemParticipantWithHeatResult.getP + ",\"q\":\"" + systemParticipantWithHeatResult.getQ + ",\"qDot\":\"" + systemParticipantWithHeatResult.getqDot + "\"}" + else if (entity.isInstanceOf[ElectricalEnergyStorageResult]) resultObject = "{\"p\":\"" + electricalEnergyStorageResult.getP + ",\"q\":\"" + electricalEnergyStorageResult.getQ + ",\"soc\":\"" + electricalEnergyStorageResult.getSoc + "\"}" + else if (entity.isInstanceOf[ConnectorResult]) resultObject = "{\"iAMag\":\"" + connectorResult.getiAMag + ",\"iAAng\":\"" + connectorResult.getiAAng + ",\"iBMag\":\"" + connectorResult.getiBMag + ",\"iBAng\":\"" + connectorResult.getiBAng + "\"}" + else if (entity.isInstanceOf[NodeResult]) resultObject = "{\"vMag\":\"" + nodeResult.getvMag + ",\"vAng\":\"" + nodeResult.getvAng + "\"}" + else if (entity.isInstanceOf[ThermalUnitResult]) resultObject = "{\"qDot\":\"" + thermalUnitResult.getqDot + "\"}" + else if (entity.isInstanceOf[SystemParticipantResult]) resultObject = "{\"p\":\"" + systemParticipantResult.getP + ",\"q\":\"" + systemParticipantResult.getQ + "\"}" + else resultObject = "{}" + resultObject + } +} + From ffaa52064ef385f26869205d66edc15d7d4fe1ec Mon Sep 17 00:00:00 2001 From: jo-bao Date: Wed, 21 Feb 2024 09:14:06 +0100 Subject: [PATCH 04/41] tests for PrimaryServiceProxy --- .../service/primary/PrimaryServiceProxy.scala | 39 +++++++++++-------- .../primary/PrimaryServiceProxySpec.scala | 38 ++++++++++++++++-- 2 files changed, 57 insertions(+), 20 deletions(-) diff --git a/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala b/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala index 6a6de0d3ec..24af493048 100644 --- a/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala +++ b/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala @@ -114,7 +114,8 @@ case class PrimaryServiceProxy( simulationStart: ZonedDateTime, extSimulation: Option[ActorRef] ): Try[PrimaryServiceStateData] = { - var serviceStateData = PrimaryServiceStateData(Map.empty, Map.empty, simulationStart, primaryConfig, null) + + createSources(primaryConfig).map { case (mappingSource, metaInformationSource) => val modelToTimeSeries = mappingSource.getMapping.asScala.toMap @@ -142,22 +143,26 @@ case class PrimaryServiceProxy( } } .toMap - serviceStateData = PrimaryServiceStateData( - modelToTimeSeries, - timeSeriesToSourceRef, - simulationStart, - primaryConfig, - mappingSource - ) - } - if (extSimulation.isDefined) { - // Ask ExtPrimaryDataService which UUIDs should be substituted - Success(serviceStateData.copy( - extSubscribers = getSubscribers, - extPrimaryDataService = extSimulation - )) - } else { - Success(serviceStateData) + if (extSimulation.isDefined) { + // Ask ExtPrimaryDataService which UUIDs should be substituted + PrimaryServiceStateData( + modelToTimeSeries, + timeSeriesToSourceRef, + simulationStart, + primaryConfig, + mappingSource, + getSubscribers(), + extSimulation + ) + } else { + PrimaryServiceStateData( + modelToTimeSeries, + timeSeriesToSourceRef, + simulationStart, + primaryConfig, + mappingSource + ) + } } } diff --git a/src/test/scala/edu/ie3/simona/service/primary/PrimaryServiceProxySpec.scala b/src/test/scala/edu/ie3/simona/service/primary/PrimaryServiceProxySpec.scala index 2a3d10edea..a54b3b85fe 100644 --- a/src/test/scala/edu/ie3/simona/service/primary/PrimaryServiceProxySpec.scala +++ b/src/test/scala/edu/ie3/simona/service/primary/PrimaryServiceProxySpec.scala @@ -129,6 +129,12 @@ class PrimaryServiceProxySpec private val scheduler: TestProbe = TestProbe("scheduler") + val validExtPrimaryDataService = TestActorRef( + new ExtPrimaryDataService( + scheduler.ref + ) + ) + "Testing a primary service config" should { "lead to complaining about too much source definitions" in { val maliciousConfig = PrimaryConfig( @@ -260,7 +266,8 @@ class PrimaryServiceProxySpec proxy invokePrivate prepareStateData( maliciousConfig, - simulationStart + simulationStart, + Option.empty ) match { case Success(_) => fail("Building state data with missing config should fail") @@ -280,7 +287,8 @@ class PrimaryServiceProxySpec proxy invokePrivate prepareStateData( maliciousConfig, - simulationStart + simulationStart, + Option.empty ) match { case Success(_) => fail("Building state data with missing config should fail") @@ -293,7 +301,8 @@ class PrimaryServiceProxySpec "result in correct data" in { proxy invokePrivate prepareStateData( validPrimaryConfig, - simulationStart + simulationStart, + Option.empty ) match { case Success( PrimaryServiceStateData( @@ -341,6 +350,29 @@ class PrimaryServiceProxySpec ) } } + + "build proxy correctly when there is an external simulation" in { + proxy invokePrivate prepareStateData( + validPrimaryConfig, + simulationStart, + Some(validExtPrimaryDataService) + ) match { + case Success( + PrimaryServiceStateData( + modelToTimeSeries, + timeSeriesToSourceRef, + simulationStart, + primaryConfig, + mappingSource, + extSubscribers, + extPrimaryDataService + ) + ) => extPrimaryDataService should contain (validExtPrimaryDataService) + extSubscribers shouldBe Iterable.empty + } + } + + } "Sending initialization information to an uninitialized actor" should { From 16c485c80635595b611f7591e2dbaa15afbeb7d7 Mon Sep 17 00:00:00 2001 From: jo-bao Date: Wed, 21 Feb 2024 10:35:23 +0100 Subject: [PATCH 05/41] spotless --- .../edu/ie3/simona/event/ResultEvent.scala | 11 +- .../event/listener/ResultEventListener.scala | 64 ++- .../ontology/messages/SchedulerMessage.scala | 3 +- .../messages/services/DataMessage.scala | 6 + .../messages/services/ResultMessage.scala | 11 +- .../messages/services/ServiceMessage.scala | 11 +- .../ie3/simona/service/ExtDataSupport.scala | 2 +- .../ie3/simona/service/SimonaService.scala | 11 +- .../simona/service/ev/ExtEvDataService.scala | 10 +- .../primary/ExtPrimaryDataService.scala | 164 ++++--- .../ExtTimeSeriesSubscribersSource.scala | 8 +- .../service/primary/PrimaryServiceProxy.scala | 431 ++++++++++-------- .../results/ExtResultDataService.scala | 133 +++--- .../scala/edu/ie3/simona/sim/SimonaSim.scala | 25 +- .../simona/sim/setup/ExtSimSetupData.scala | 6 +- .../sim/setup/SimonaStandaloneSetup.scala | 33 +- .../primary/ExtPrimaryDataServiceSpec.scala | 39 +- .../primary/PrimaryServiceProxySpec.scala | 24 +- .../service/PrimaryDataFactoryDefault.scala | 12 +- .../common/service/ResultFactoryDefault.scala | 22 - 20 files changed, 596 insertions(+), 430 deletions(-) delete mode 100644 src/test/scala/edu/ie3/simona/test/common/service/ResultFactoryDefault.scala diff --git a/src/main/scala/edu/ie3/simona/event/ResultEvent.scala b/src/main/scala/edu/ie3/simona/event/ResultEvent.scala index 83157adb70..3605bfe65c 100644 --- a/src/main/scala/edu/ie3/simona/event/ResultEvent.scala +++ b/src/main/scala/edu/ie3/simona/event/ResultEvent.scala @@ -7,13 +7,16 @@ package edu.ie3.simona.event import edu.ie3.datamodel.models.result.{NodeResult, ResultEntity} -import edu.ie3.datamodel.models.result.connector.{LineResult, SwitchResult, Transformer2WResult} +import edu.ie3.datamodel.models.result.connector.{ + LineResult, + SwitchResult, + Transformer2WResult +} import edu.ie3.datamodel.models.result.system.SystemParticipantResult import edu.ie3.datamodel.models.result.thermal.ThermalUnitResult import edu.ie3.simona.agent.grid.GridResultsSupport.PartialTransformer3wResult import edu.ie3.simona.event.listener.ResultEventListener.ResultMessage - sealed trait ResultEvent extends ResultMessage /** Calculation result events @@ -21,8 +24,8 @@ sealed trait ResultEvent extends ResultMessage object ResultEvent { final case class ExternalResultEvent( - resultEntity: ResultEntity - ) extends ResultEvent + resultEntity: ResultEntity + ) extends ResultEvent /** Event that holds a calculation result of a * [[edu.ie3.simona.model.participant.SystemParticipant]] diff --git a/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala b/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala index 30ce73b2f8..f6f23215ae 100644 --- a/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala +++ b/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala @@ -12,8 +12,16 @@ import edu.ie3.datamodel.io.processor.result.ResultEntityProcessor import edu.ie3.datamodel.models.result.{NodeResult, ResultEntity} import edu.ie3.simona.agent.grid.GridResultsSupport.PartialTransformer3wResult import edu.ie3.simona.event.Event -import edu.ie3.simona.event.ResultEvent.{ExternalResultEvent, ParticipantResultEvent, PowerFlowResultEvent, ThermalResultEvent} -import edu.ie3.simona.exceptions.{FileHierarchyException, ProcessResultEventException} +import edu.ie3.simona.event.ResultEvent.{ + ExternalResultEvent, + ParticipantResultEvent, + PowerFlowResultEvent, + ThermalResultEvent +} +import edu.ie3.simona.exceptions.{ + FileHierarchyException, + ProcessResultEventException +} import edu.ie3.simona.io.result._ import edu.ie3.simona.ontology.messages.StopMessage import edu.ie3.simona.ontology.messages.services.ResultMessage.ResultResponseMessage @@ -39,7 +47,6 @@ object ResultEventListener extends Transformer3wResultSupport { private final case object StopTimeout extends ResultMessage - /** [[ResultEventListener]] base data containing all information the listener * needs * @@ -158,7 +165,11 @@ object ResultEventListener extends Transformer3wResultSupport { ): BaseData = { handOverToSink(resultEntity, baseData.classToSink, log) if (baseData.extResultDataService.isDefined) { - handOverToExternalService(resultEntity, baseData.extResultDataService, log) + handOverToExternalService( + resultEntity, + baseData.extResultDataService, + log + ) } baseData } @@ -235,11 +246,13 @@ object ResultEventListener extends Transformer3wResultSupport { } private def handOverToExternalService( - resultEntity: ResultEntity, - extResultDataService: Option[ActorRef], - log: Logger - ): Unit = Try { - val extResultDataServiceRef = extResultDataService.getOrElse(throw new Exception("No external data service registered!")) + resultEntity: ResultEntity, + extResultDataService: Option[ActorRef], + log: Logger + ): Unit = Try { + val extResultDataServiceRef = extResultDataService.getOrElse( + throw new Exception("No external data service registered!") + ) extResultDataServiceRef ! ResultResponseMessage(resultEntity) } @@ -272,22 +285,23 @@ object ResultEventListener extends Transformer3wResultSupport { init(extResultDataService) } - private def init(extResultDataService: Option[ActorRef]): Behavior[ResultMessage] = Behaviors.withStash(200) { - buffer => - Behaviors.receive[ResultMessage] { - case (ctx, SinkResponse(response)) => - ctx.log.debug("Initialization complete!") - buffer.unstashAll(idle(BaseData(response, extResultDataService))) - - case (ctx, Failed(ex)) => - ctx.log.error("Unable to setup ResultEventListener.", ex) - Behaviors.stopped - - case (_, msg) => - // stash all messages - buffer.stash(msg) - Behaviors.same - } + private def init( + extResultDataService: Option[ActorRef] + ): Behavior[ResultMessage] = Behaviors.withStash(200) { buffer => + Behaviors.receive[ResultMessage] { + case (ctx, SinkResponse(response)) => + ctx.log.debug("Initialization complete!") + buffer.unstashAll(idle(BaseData(response, extResultDataService))) + + case (ctx, Failed(ex)) => + ctx.log.error("Unable to setup ResultEventListener.", ex) + Behaviors.stopped + + case (_, msg) => + // stash all messages + buffer.stash(msg) + Behaviors.same + } } private def idle(baseData: BaseData): Behavior[ResultMessage] = Behaviors diff --git a/src/main/scala/edu/ie3/simona/ontology/messages/SchedulerMessage.scala b/src/main/scala/edu/ie3/simona/ontology/messages/SchedulerMessage.scala index 663e457327..89a62d3446 100644 --- a/src/main/scala/edu/ie3/simona/ontology/messages/SchedulerMessage.scala +++ b/src/main/scala/edu/ie3/simona/ontology/messages/SchedulerMessage.scala @@ -25,5 +25,6 @@ object SchedulerMessage { unlockKey: Option[ScheduleKey] = None ) extends SchedulerMessage - final case class ResultForExtCompleteMessage(results: Iterable[ResultEntity]) extends SchedulerMessage + final case class ResultForExtCompleteMessage(results: Iterable[ResultEntity]) + extends SchedulerMessage } diff --git a/src/main/scala/edu/ie3/simona/ontology/messages/services/DataMessage.scala b/src/main/scala/edu/ie3/simona/ontology/messages/services/DataMessage.scala index 92a08aa584..ff93925fe9 100644 --- a/src/main/scala/edu/ie3/simona/ontology/messages/services/DataMessage.scala +++ b/src/main/scala/edu/ie3/simona/ontology/messages/services/DataMessage.scala @@ -1,3 +1,9 @@ +/* + * © 2024. TU Dortmund University, + * Institute of Energy Systems, Energy Efficiency and Energy Economics, + * Research group Distribution grid planning and operation + */ + package edu.ie3.simona.ontology.messages.services trait DataMessage {} diff --git a/src/main/scala/edu/ie3/simona/ontology/messages/services/ResultMessage.scala b/src/main/scala/edu/ie3/simona/ontology/messages/services/ResultMessage.scala index 6e17f11554..b3c79a0d44 100644 --- a/src/main/scala/edu/ie3/simona/ontology/messages/services/ResultMessage.scala +++ b/src/main/scala/edu/ie3/simona/ontology/messages/services/ResultMessage.scala @@ -1,3 +1,9 @@ +/* + * © 2024. TU Dortmund University, + * Institute of Energy Systems, Energy Efficiency and Energy Economics, + * Research group Distribution grid planning and operation + */ + package edu.ie3.simona.ontology.messages.services import edu.ie3.datamodel.models.result.ResultEntity @@ -7,5 +13,6 @@ sealed trait ResultMessage extends DataMessage object ResultMessage { final case class ResultRequest(tick: Long) - final case class ResultResponseMessage(result: ResultEntity) extends ResultMessage -} \ No newline at end of file + final case class ResultResponseMessage(result: ResultEntity) + extends ResultMessage +} diff --git a/src/main/scala/edu/ie3/simona/ontology/messages/services/ServiceMessage.scala b/src/main/scala/edu/ie3/simona/ontology/messages/services/ServiceMessage.scala index af832bfc9f..ac92310da0 100644 --- a/src/main/scala/edu/ie3/simona/ontology/messages/services/ServiceMessage.scala +++ b/src/main/scala/edu/ie3/simona/ontology/messages/services/ServiceMessage.scala @@ -43,9 +43,9 @@ case object ServiceMessage { extends ServiceRegistrationMessage final case class ExtPrimaryDataServiceRegistrationMessage( - modelUuid: UUID, - requestingActor: ActorRef - ) extends ServiceRegistrationMessage + modelUuid: UUID, + requestingActor: ActorRef + ) extends ServiceRegistrationMessage sealed trait RegistrationResponseMessage extends ServiceMessage @@ -85,9 +85,4 @@ case object ServiceMessage { final case class ProvideExtSubscriberMessage() extends ExtSubscriberMessage - - - - - } diff --git a/src/main/scala/edu/ie3/simona/service/ExtDataSupport.scala b/src/main/scala/edu/ie3/simona/service/ExtDataSupport.scala index e8297bfbbe..5449f2ebd9 100644 --- a/src/main/scala/edu/ie3/simona/service/ExtDataSupport.scala +++ b/src/main/scala/edu/ie3/simona/service/ExtDataSupport.scala @@ -50,6 +50,6 @@ trait ExtDataSupport[ * the updated state data */ protected def handleDataResponseMessage( - extResponseMsg: DataMessage //extResponseMsg + extResponseMsg: DataMessage // extResponseMsg )(implicit serviceStateData: S): S } diff --git a/src/main/scala/edu/ie3/simona/service/SimonaService.scala b/src/main/scala/edu/ie3/simona/service/SimonaService.scala index 5742867512..b627684152 100644 --- a/src/main/scala/edu/ie3/simona/service/SimonaService.scala +++ b/src/main/scala/edu/ie3/simona/service/SimonaService.scala @@ -12,11 +12,17 @@ import org.apache.pekko.actor.typed.scaladsl.adapter.ClassicActorRefOps import org.apache.pekko.actor.{Actor, ActorContext, ActorRef, Stash} import edu.ie3.simona.logging.SimonaActorLogging import edu.ie3.simona.ontology.messages.Activation -import edu.ie3.simona.ontology.messages.SchedulerMessage.{Completion, ScheduleActivation} +import edu.ie3.simona.ontology.messages.SchedulerMessage.{ + Completion, + ScheduleActivation +} import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.ScheduleServiceActivation import edu.ie3.simona.ontology.messages.services.ServiceMessage.ServiceRegistrationMessage import edu.ie3.simona.scheduler.ScheduleLock.ScheduleKey -import edu.ie3.simona.service.ServiceStateData.{InitializeServiceStateData, ServiceBaseStateData} +import edu.ie3.simona.service.ServiceStateData.{ + InitializeServiceStateData, + ServiceBaseStateData +} import edu.ie3.simona.service.SimonaService.Create import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK @@ -165,7 +171,6 @@ abstract class SimonaService[ unhandled(x) } - /** Internal api method that allows handling incoming messages from external * simulations * diff --git a/src/main/scala/edu/ie3/simona/service/ev/ExtEvDataService.scala b/src/main/scala/edu/ie3/simona/service/ev/ExtEvDataService.scala index 8bfdd339e4..13e84edd25 100644 --- a/src/main/scala/edu/ie3/simona/service/ev/ExtEvDataService.scala +++ b/src/main/scala/edu/ie3/simona/service/ev/ExtEvDataService.scala @@ -19,8 +19,14 @@ import edu.ie3.simona.ontology.messages.services.EvMessage._ import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.RegistrationSuccessfulMessage import edu.ie3.simona.ontology.messages.services.ServiceMessage.ServiceRegistrationMessage import edu.ie3.simona.scheduler.ScheduleLock -import edu.ie3.simona.service.ServiceStateData.{InitializeServiceStateData, ServiceBaseStateData} -import edu.ie3.simona.service.ev.ExtEvDataService.{ExtEvStateData, InitExtEvData} +import edu.ie3.simona.service.ServiceStateData.{ + InitializeServiceStateData, + ServiceBaseStateData +} +import edu.ie3.simona.service.ev.ExtEvDataService.{ + ExtEvStateData, + InitExtEvData +} import edu.ie3.simona.service.{ExtDataSupport, ServiceStateData, SimonaService} import java.util.UUID diff --git a/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala b/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala index 1039bedd8d..81828f795a 100644 --- a/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala +++ b/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala @@ -1,18 +1,37 @@ +/* + * © 2024. TU Dortmund University, + * Institute of Energy Systems, Energy Efficiency and Energy Economics, + * Research group Distribution grid planning and operation + */ + package edu.ie3.simona.service.primary import edu.ie3.datamodel.models.value.Value import edu.ie3.simona.agent.participant.data.Data.PrimaryData.RichValue import edu.ie3.simona.api.data.ontology.DataMessageFromExt import edu.ie3.simona.api.data.primarydata.ExtPrimaryData -import edu.ie3.simona.api.data.primarydata.ontology.{PrimaryDataMessageFromExt, ProvidePrimaryData} +import edu.ie3.simona.api.data.primarydata.ontology.{ + PrimaryDataMessageFromExt, + ProvidePrimaryData +} import edu.ie3.simona.exceptions.WeatherServiceException.InvalidRegistrationRequestException import edu.ie3.simona.exceptions.{InitializationException, ServiceException} -import edu.ie3.simona.ontology.messages.services.ServiceMessage.{ExtPrimaryDataServiceRegistrationMessage, PrimaryServiceRegistrationMessage, WorkerRegistrationMessage} +import edu.ie3.simona.ontology.messages.services.ServiceMessage.{ + ExtPrimaryDataServiceRegistrationMessage, + PrimaryServiceRegistrationMessage, + WorkerRegistrationMessage +} import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.RegistrationSuccessfulMessage import edu.ie3.simona.ontology.messages.services.{DataMessage, ServiceMessage} import edu.ie3.simona.scheduler.ScheduleLock -import edu.ie3.simona.service.ServiceStateData.{InitializeServiceStateData, ServiceBaseStateData} -import edu.ie3.simona.service.primary.ExtPrimaryDataService.{ExtPrimaryDataStateData, InitExtPrimaryData} +import edu.ie3.simona.service.ServiceStateData.{ + InitializeServiceStateData, + ServiceBaseStateData +} +import edu.ie3.simona.service.primary.ExtPrimaryDataService.{ + ExtPrimaryDataStateData, + InitExtPrimaryData +} import edu.ie3.simona.service.primary.PrimaryServiceWorker.ProvidePrimaryDataMessage import edu.ie3.simona.service.{ExtDataSupport, ServiceStateData, SimonaService} import org.apache.pekko.actor.typed.scaladsl.adapter.ClassicActorRefOps @@ -30,26 +49,25 @@ object ExtPrimaryDataService { ) final case class ExtPrimaryDataStateData( - extPrimaryData: ExtPrimaryData, - uuidToActorRef: Map[UUID, ActorRef] = Map.empty[UUID, ActorRef], // subscribers in SIMONA - extPrimaryDataMessage: Option[PrimaryDataMessageFromExt] = None - ) extends ServiceBaseStateData + extPrimaryData: ExtPrimaryData, + uuidToActorRef: Map[UUID, ActorRef] = + Map.empty[UUID, ActorRef], // subscribers in SIMONA + extPrimaryDataMessage: Option[PrimaryDataMessageFromExt] = None + ) extends ServiceBaseStateData case class InitExtPrimaryData( - extPrimaryData: ExtPrimaryData - ) extends InitializeServiceStateData + extPrimaryData: ExtPrimaryData + ) extends InitializeServiceStateData } final case class ExtPrimaryDataService( - override val scheduler: ActorRef - ) - extends SimonaService[ExtPrimaryDataStateData](scheduler) - with ExtDataSupport[ExtPrimaryDataStateData] { - + override val scheduler: ActorRef +) extends SimonaService[ExtPrimaryDataStateData](scheduler) + with ExtDataSupport[ExtPrimaryDataStateData] { override def init( - initServiceData: ServiceStateData.InitializeServiceStateData - ): Try[(ExtPrimaryDataStateData, Option[Long])] = initServiceData match { + initServiceData: ServiceStateData.InitializeServiceStateData + ): Try[(ExtPrimaryDataStateData, Option[Long])] = initServiceData match { case InitExtPrimaryData(extPrimaryData) => val primaryDataInitializedStateData = ExtPrimaryDataStateData( extPrimaryData @@ -69,30 +87,33 @@ final case class ExtPrimaryDataService( } override protected def handleRegistrationRequest( - registrationMessage: ServiceMessage.ServiceRegistrationMessage - )(implicit serviceStateData: ExtPrimaryDataStateData): - Try[ExtPrimaryDataStateData] = { + registrationMessage: ServiceMessage.ServiceRegistrationMessage + )(implicit + serviceStateData: ExtPrimaryDataStateData + ): Try[ExtPrimaryDataStateData] = { println("Habe erhalten 0") registrationMessage match { - case ExtPrimaryDataServiceRegistrationMessage(modelUuid, requestingActor) => - println("Habe erhalten") - Success(handleRegistrationRequest(requestingActor, modelUuid)) - case invalidMessage => - Failure( - InvalidRegistrationRequestException( - s"A primary service provider is not able to handle registration request '$invalidMessage'." + case ExtPrimaryDataServiceRegistrationMessage( + modelUuid, + requestingActor + ) => + println("Habe erhalten") + Success(handleRegistrationRequest(requestingActor, modelUuid)) + case invalidMessage => + Failure( + InvalidRegistrationRequestException( + s"A primary service provider is not able to handle registration request '$invalidMessage'." + ) ) - ) } } - - private def handleRegistrationRequest( - agentToBeRegistered: ActorRef, - agentUUID: UUID - )(implicit serviceStateData: ExtPrimaryDataStateData): - ExtPrimaryDataStateData = { + agentToBeRegistered: ActorRef, + agentUUID: UUID + )(implicit + serviceStateData: ExtPrimaryDataStateData + ): ExtPrimaryDataStateData = { serviceStateData.uuidToActorRef.get(agentUUID) match { case None => // Actor is not registered yet @@ -112,43 +133,45 @@ final case class ExtPrimaryDataService( } /** Send out the information to all registered recipients - * - * @param tick - * current tick data should be announced for - * @param serviceStateData - * the current state data of this service - * @return - * the service stata data that should be used in the next state (normally - * with updated values) together with the completion message that is send - * in response to the trigger that was sent to start this announcement - */ + * + * @param tick + * current tick data should be announced for + * @param serviceStateData + * the current state data of this service + * @return + * the service stata data that should be used in the next state (normally + * with updated values) together with the completion message that is send + * in response to the trigger that was sent to start this announcement + */ override protected def announceInformation( - tick: Long - )( - implicit serviceStateData: ExtPrimaryDataStateData, - ctx: ActorContext - ): (ExtPrimaryDataStateData, Option[Long]) = { + tick: Long + )(implicit + serviceStateData: ExtPrimaryDataStateData, + ctx: ActorContext + ): (ExtPrimaryDataStateData, Option[Long]) = { serviceStateData.extPrimaryDataMessage.getOrElse( throw ServiceException( "ExtPrimaryDataService was triggered without ExtPrimaryDataMessage available" ) ) match { case providedPrimaryData: ProvidePrimaryData => - processDataAndAnnounce(tick, providedPrimaryData.primaryData)(serviceStateData, ctx) + processDataAndAnnounce(tick, providedPrimaryData.primaryData)( + serviceStateData, + ctx + ) } } private def processDataAndAnnounce( - tick: Long, - primaryData: java.util.Map[UUID, Value] - )( - implicit - serviceStateData: ExtPrimaryDataStateData, - ctx: ActorContext - ): ( - ExtPrimaryDataStateData, + tick: Long, + primaryData: java.util.Map[UUID, Value] + )(implicit + serviceStateData: ExtPrimaryDataStateData, + ctx: ActorContext + ): ( + ExtPrimaryDataStateData, Option[Long] - ) = { + ) = { val actorToPrimaryData = primaryData.asScala.flatMap { case (agent, primaryDataPerAgent) => serviceStateData.uuidToActorRef @@ -166,7 +189,12 @@ final case class ExtPrimaryDataService( // Verteile Primary Data if (actorToPrimaryData.nonEmpty) { val keys = - ScheduleLock.multiKey(ctx, scheduler.toTyped, tick, actorToPrimaryData.size) + ScheduleLock.multiKey( + ctx, + scheduler.toTyped, + tick, + actorToPrimaryData.size + ) actorToPrimaryData.zip(keys).foreach { case ((actor, primaryDataPerAgent), key) => @@ -197,9 +225,10 @@ final case class ExtPrimaryDataService( } override protected def handleDataMessage( - extMsg: DataMessageFromExt - )(implicit serviceStateData: ExtPrimaryDataStateData): - ExtPrimaryDataStateData = { + extMsg: DataMessageFromExt + )(implicit + serviceStateData: ExtPrimaryDataStateData + ): ExtPrimaryDataStateData = { extMsg match { case extPrimaryDataMessage: PrimaryDataMessageFromExt => serviceStateData.copy( @@ -209,12 +238,11 @@ final case class ExtPrimaryDataService( } override protected def handleDataResponseMessage( - extResponseMsg: DataMessage - )( - implicit serviceStateData: ExtPrimaryDataStateData): - ExtPrimaryDataStateData = { + extResponseMsg: DataMessage + )(implicit + serviceStateData: ExtPrimaryDataStateData + ): ExtPrimaryDataStateData = { // not implemented null } } - diff --git a/src/main/scala/edu/ie3/simona/service/primary/ExtTimeSeriesSubscribersSource.scala b/src/main/scala/edu/ie3/simona/service/primary/ExtTimeSeriesSubscribersSource.scala index d125b95cc4..a4689aa6a5 100644 --- a/src/main/scala/edu/ie3/simona/service/primary/ExtTimeSeriesSubscribersSource.scala +++ b/src/main/scala/edu/ie3/simona/service/primary/ExtTimeSeriesSubscribersSource.scala @@ -1,3 +1,9 @@ +/* + * © 2024. TU Dortmund University, + * Institute of Energy Systems, Energy Efficiency and Energy Economics, + * Research group Distribution grid planning and operation + */ + package edu.ie3.simona.service.primary import java.util.UUID @@ -5,5 +11,5 @@ import java.util.UUID object ExtTimeSeriesSubscribersSource { private var subscribers: Iterable[UUID] = Iterable.empty[UUID] - def getSubscribers(): Iterable[UUID] = subscribers + def getSubscribers: Iterable[UUID] = subscribers } diff --git a/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala b/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala index 24af493048..177f70dd62 100644 --- a/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala +++ b/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala @@ -11,26 +11,56 @@ import org.apache.pekko.actor.{Actor, ActorRef, PoisonPill, Props} import edu.ie3.datamodel.io.connectors.SqlConnector import edu.ie3.datamodel.io.csv.CsvIndividualTimeSeriesMetaInformation import edu.ie3.datamodel.io.naming.timeseries.IndividualTimeSeriesMetaInformation -import edu.ie3.datamodel.io.naming.{DatabaseNamingStrategy, EntityPersistenceNamingStrategy, FileNamingStrategy} -import edu.ie3.datamodel.io.source.csv.{CsvTimeSeriesMappingSource, CsvTimeSeriesMetaInformationSource} -import edu.ie3.datamodel.io.source.sql.{SqlTimeSeriesMappingSource, SqlTimeSeriesMetaInformationSource} -import edu.ie3.datamodel.io.source.{TimeSeriesMappingSource, TimeSeriesMetaInformationSource} +import edu.ie3.datamodel.io.naming.{ + DatabaseNamingStrategy, + EntityPersistenceNamingStrategy, + FileNamingStrategy +} +import edu.ie3.datamodel.io.source.csv.{ + CsvTimeSeriesMappingSource, + CsvTimeSeriesMetaInformationSource +} +import edu.ie3.datamodel.io.source.sql.{ + SqlTimeSeriesMappingSource, + SqlTimeSeriesMetaInformationSource +} +import edu.ie3.datamodel.io.source.{ + TimeSeriesMappingSource, + TimeSeriesMetaInformationSource +} import edu.ie3.datamodel.models.value.Value import edu.ie3.simona.config.SimonaConfig.PrimaryDataCsvParams import edu.ie3.simona.config.SimonaConfig.Simona.Input.Primary.SqlParams -import edu.ie3.simona.config.SimonaConfig.Simona.Input.{Primary => PrimaryConfig} -import edu.ie3.simona.exceptions.{InitializationException, InvalidConfigParameterException} +import edu.ie3.simona.config.SimonaConfig.Simona.Input.{ + Primary => PrimaryConfig +} +import edu.ie3.simona.exceptions.{ + InitializationException, + InvalidConfigParameterException +} import edu.ie3.simona.logging.SimonaActorLogging import edu.ie3.simona.ontology.messages.Activation import edu.ie3.simona.ontology.messages.SchedulerMessage.Completion import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.RegistrationFailedMessage -import edu.ie3.simona.ontology.messages.services.ServiceMessage.{ExtPrimaryDataServiceRegistrationMessage, PrimaryServiceRegistrationMessage, WorkerRegistrationMessage} +import edu.ie3.simona.ontology.messages.services.ServiceMessage.{ + ExtPrimaryDataServiceRegistrationMessage, + PrimaryServiceRegistrationMessage, + WorkerRegistrationMessage +} import edu.ie3.simona.scheduler.ScheduleLock import edu.ie3.simona.service.{ServiceStateData, SimonaService} import edu.ie3.simona.service.ServiceStateData.InitializeServiceStateData import edu.ie3.simona.service.primary.ExtTimeSeriesSubscribersSource.getSubscribers -import edu.ie3.simona.service.primary.PrimaryServiceProxy.{InitPrimaryServiceProxyStateData, PrimaryServiceStateData, SourceRef} -import edu.ie3.simona.service.primary.PrimaryServiceWorker.{CsvInitPrimaryServiceStateData, InitPrimaryServiceStateData, SqlInitPrimaryServiceStateData} +import edu.ie3.simona.service.primary.PrimaryServiceProxy.{ + InitPrimaryServiceProxyStateData, + PrimaryServiceStateData, + SourceRef +} +import edu.ie3.simona.service.primary.PrimaryServiceWorker.{ + CsvInitPrimaryServiceStateData, + InitPrimaryServiceStateData, + SqlInitPrimaryServiceStateData +} import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK import java.nio.file.Paths @@ -43,35 +73,35 @@ import scala.jdk.OptionConverters.RichOptional import scala.util.{Failure, Success, Try} /** This actor has information on which models can be replaced by precalculated - * (primary) data and how to obtain those time series. It offers possibility to - * register for a certain model. If data is available, a child actor is spun - * of, that will do the actual provision and the requesting agent is informed - * accordingly. - * - * @param scheduler - * Reference to the scheduler of the simulation - * @param startDateTime - * Wall clock time of the first instant in simulation - */ + * (primary) data and how to obtain those time series. It offers possibility to + * register for a certain model. If data is available, a child actor is spun + * of, that will do the actual provision and the requesting agent is informed + * accordingly. + * + * @param scheduler + * Reference to the scheduler of the simulation + * @param startDateTime + * Wall clock time of the first instant in simulation + */ case class PrimaryServiceProxy( - scheduler: ActorRef, - initStateData: InitPrimaryServiceProxyStateData, - private implicit val startDateTime: ZonedDateTime - ) extends Actor - with SimonaActorLogging { + scheduler: ActorRef, + initStateData: InitPrimaryServiceProxyStateData, + private implicit val startDateTime: ZonedDateTime +) extends Actor + with SimonaActorLogging { /** Start receiving without knowing specifics about myself - * - * @return - * How receiving should be handled - */ + * + * @return + * How receiving should be handled + */ override def receive: Receive = uninitialized /** Handle all messages, when the actor isn't initialized, yet. - * - * @return - * How receiving should be handled with gained insight of myself - */ + * + * @return + * How receiving should be handled with gained insight of myself + */ private def uninitialized: Receive = { case Activation(INIT_SIM_TICK) => /* The proxy is asked to initialize itself. If that happened successfully, change the logic of receiving @@ -99,22 +129,21 @@ case class PrimaryServiceProxy( } /** Prepare the needed state data by building a - * [[edu.ie3.datamodel.io.source.TimeSeriesMappingSource]], obtain it's - * information and compile them to state data - * - * @param primaryConfig - * Configuration for the primary source - * @param simulationStart - * Wall clock time of first instant in simulation - * @return - * State data, containing the known model and time series identifiers - */ + * [[edu.ie3.datamodel.io.source.TimeSeriesMappingSource]], obtain it's + * information and compile them to state data + * + * @param primaryConfig + * Configuration for the primary source + * @param simulationStart + * Wall clock time of first instant in simulation + * @return + * State data, containing the known model and time series identifiers + */ private def prepareStateData( - primaryConfig: PrimaryConfig, - simulationStart: ZonedDateTime, - extSimulation: Option[ActorRef] - ): Try[PrimaryServiceStateData] = { - + primaryConfig: PrimaryConfig, + simulationStart: ZonedDateTime, + extSimulation: Option[ActorRef] + ): Try[PrimaryServiceStateData] = { createSources(primaryConfig).map { case (mappingSource, metaInformationSource) => @@ -151,10 +180,10 @@ case class PrimaryServiceProxy( simulationStart, primaryConfig, mappingSource, - getSubscribers(), + getSubscribers, extSimulation ) - } else { + } else { PrimaryServiceStateData( modelToTimeSeries, timeSeriesToSourceRef, @@ -162,13 +191,13 @@ case class PrimaryServiceProxy( primaryConfig, mappingSource ) - } + } } } private def createSources( - primaryConfig: PrimaryConfig - ): Try[(TimeSeriesMappingSource, TimeSeriesMetaInformationSource)] = { + primaryConfig: PrimaryConfig + ): Try[(TimeSeriesMappingSource, TimeSeriesMetaInformationSource)] = { Seq( primaryConfig.sqlParams, primaryConfig.influxDb1xParams, @@ -223,15 +252,15 @@ case class PrimaryServiceProxy( } /** Message handling, if the actor has been initialized already. This method - * basically handles registration requests, checks, if pre-calculated, - * primary data is available and forwards the request to worker actors. If - * needed, new workers are spun off. - * - * @param stateData - * Representing the current state of the agent - * @return - * Message handling routine - */ + * basically handles registration requests, checks, if pre-calculated, + * primary data is available and forwards the request to worker actors. If + * needed, new workers are spun off. + * + * @param stateData + * Representing the current state of the agent + * @return + * Message handling routine + */ private def onMessage(stateData: PrimaryServiceStateData): Receive = { case PrimaryServiceRegistrationMessage(modelUuid) => /* Try to register for this model */ @@ -260,7 +289,7 @@ case class PrimaryServiceProxy( s"There is no time series apparent for the model with uuid '{}'.", modelUuid ) - sender() ! RegistrationFailedMessage + sender() ! RegistrationFailedMessage } } case x => @@ -271,22 +300,22 @@ case class PrimaryServiceProxy( } /** Handle the registration request for a covered model. First, try to get a - * already existing worker for this time series, otherwise spin-off a new - * one, remember it and forward the request - * - * @param modelUuid - * Unique identifier of the model - * @param timeSeriesUuid - * Unique identifier of the equivalent time series - * @param stateData - * Current state data of the actor - */ + * already existing worker for this time series, otherwise spin-off a new + * one, remember it and forward the request + * + * @param modelUuid + * Unique identifier of the model + * @param timeSeriesUuid + * Unique identifier of the equivalent time series + * @param stateData + * Current state data of the actor + */ protected def handleCoveredModel( - modelUuid: UUID, - timeSeriesUuid: UUID, - stateData: PrimaryServiceStateData, - requestingActor: ActorRef - ): Unit = { + modelUuid: UUID, + timeSeriesUuid: UUID, + stateData: PrimaryServiceStateData, + requestingActor: ActorRef + ): Unit = { val timeSeriesToSourceRef = stateData.timeSeriesToSourceRef timeSeriesToSourceRef.get(timeSeriesUuid) match { case Some(SourceRef(_, Some(worker))) => @@ -327,31 +356,33 @@ case class PrimaryServiceProxy( } protected def handleExternalModel( - modelUuid: UUID, - stateData: PrimaryServiceStateData, - requestingActor: ActorRef): Unit = { + modelUuid: UUID, + stateData: PrimaryServiceStateData, + requestingActor: ActorRef + ): Unit = { stateData.extPrimaryDataService match { - case Some(reqActor) => reqActor ! ExtPrimaryDataServiceRegistrationMessage(modelUuid, reqActor) + case Some(reqActor) => + reqActor ! ExtPrimaryDataServiceRegistrationMessage(modelUuid, reqActor) } } /** Instantiate a new [[PrimaryServiceWorker]] and send initialization - * information - * - * @param metaInformation - * Meta information (including column scheme) of the time series - * @param simulationStart - * The time of the simulation start - * @param primaryConfig - * Configuration for the primary config - * @return - * The [[ActorRef]] to the worker - */ + * information + * + * @param metaInformation + * Meta information (including column scheme) of the time series + * @param simulationStart + * The time of the simulation start + * @param primaryConfig + * Configuration for the primary config + * @return + * The [[ActorRef]] to the worker + */ protected def initializeWorker( - metaInformation: IndividualTimeSeriesMetaInformation, - simulationStart: ZonedDateTime, - primaryConfig: PrimaryConfig - ): Try[ActorRef] = { + metaInformation: IndividualTimeSeriesMetaInformation, + simulationStart: ZonedDateTime, + primaryConfig: PrimaryConfig + ): Try[ActorRef] = { val workerRef = classToWorkerRef( metaInformation.getColumnScheme.getValueClass, metaInformation.getUuid.toString @@ -379,21 +410,21 @@ case class PrimaryServiceProxy( } /** Build a primary source worker and type it to the foreseen value class to - * come - * - * @param valueClass - * Class of the values to provide later on - * @param timeSeriesUuid - * uuid of the time series the actor processes - * @tparam V - * Type of the class to provide - * @return - * The [[ActorRef]] to the spun off actor - */ + * come + * + * @param valueClass + * Class of the values to provide later on + * @param timeSeriesUuid + * uuid of the time series the actor processes + * @tparam V + * Type of the class to provide + * @return + * The [[ActorRef]] to the spun off actor + */ protected def classToWorkerRef[V <: Value]( - valueClass: Class[V], - timeSeriesUuid: String - ): ActorRef = { + valueClass: Class[V], + timeSeriesUuid: String + ): ActorRef = { import edu.ie3.simona.actor.SimonaActorNaming._ context.system.simonaActorOf( PrimaryServiceWorker.props(scheduler, valueClass), @@ -402,27 +433,27 @@ case class PrimaryServiceProxy( } /** Building proper init data for the worker - * - * @param metaInformation - * Meta information (including column scheme) of the time series - * @param simulationStart - * The time of the simulation start - * @param primaryConfig - * Configuration for the primary config - * @return - */ + * + * @param metaInformation + * Meta information (including column scheme) of the time series + * @param simulationStart + * The time of the simulation start + * @param primaryConfig + * Configuration for the primary config + * @return + */ private def toInitData( - metaInformation: IndividualTimeSeriesMetaInformation, - simulationStart: ZonedDateTime, - primaryConfig: PrimaryConfig - ): Try[InitPrimaryServiceStateData] = + metaInformation: IndividualTimeSeriesMetaInformation, + simulationStart: ZonedDateTime, + primaryConfig: PrimaryConfig + ): Try[InitPrimaryServiceStateData] = primaryConfig match { case PrimaryConfig( - None, - Some(PrimaryDataCsvParams(csvSep, directoryPath, _, timePattern)), - None, - None - ) => + None, + Some(PrimaryDataCsvParams(csvSep, directoryPath, _, timePattern)), + None, + None + ) => /* The actual data sources are from csv. Meta information have to match */ metaInformation match { case csvMetaData: CsvIndividualTimeSeriesMetaInformation => @@ -446,11 +477,11 @@ case class PrimaryServiceProxy( } case PrimaryConfig( - None, - None, - None, - Some(sqlParams: SqlParams) - ) => + None, + None, + None, + Some(sqlParams: SqlParams) + ) => Success( SqlInitPrimaryServiceStateData( metaInformation.getUuid, @@ -469,21 +500,21 @@ case class PrimaryServiceProxy( } /** Register the worker within the state data. - * - * @param stateData - * Current state information - * @param timeSeriesUuid - * Unique identifier of the time series, the worker takes care of - * @param workerRef - * [[ActorRef]] to the new worker actor - * @return - * The updated state data, that holds reference to the worker - */ + * + * @param stateData + * Current state information + * @param timeSeriesUuid + * Unique identifier of the time series, the worker takes care of + * @param workerRef + * [[ActorRef]] to the new worker actor + * @return + * The updated state data, that holds reference to the worker + */ private def updateStateData( - stateData: PrimaryServiceStateData, - timeSeriesUuid: UUID, - workerRef: ActorRef - ): PrimaryServiceStateData = { + stateData: PrimaryServiceStateData, + timeSeriesUuid: UUID, + workerRef: ActorRef + ): PrimaryServiceStateData = { val timeSeriesToSourceRef = stateData.timeSeriesToSourceRef val sourceRef = timeSeriesToSourceRef.getOrElse( timeSeriesUuid, @@ -502,69 +533,69 @@ case class PrimaryServiceProxy( object PrimaryServiceProxy { def props( - scheduler: ActorRef, - initStateData: InitPrimaryServiceProxyStateData, - startDateTime: ZonedDateTime - ): Props = Props( + scheduler: ActorRef, + initStateData: InitPrimaryServiceProxyStateData, + startDateTime: ZonedDateTime + ): Props = Props( new PrimaryServiceProxy(scheduler, initStateData, startDateTime) ) /** State data with needed information to initialize this primary service - * provider proxy - * - * @param primaryConfig - * Configuration for the primary source - * @param simulationStart - * Wall clock time of the first instant in simulation - */ + * provider proxy + * + * @param primaryConfig + * Configuration for the primary source + * @param simulationStart + * Wall clock time of the first instant in simulation + */ final case class InitPrimaryServiceProxyStateData( - primaryConfig: PrimaryConfig, - simulationStart: ZonedDateTime, - extSimulation: Option[ActorRef] - ) extends InitializeServiceStateData + primaryConfig: PrimaryConfig, + simulationStart: ZonedDateTime, + extSimulation: Option[ActorRef] + ) extends InitializeServiceStateData /** Holding the state of an initialized proxy. - * - * @param modelToTimeSeries - * Mapping from models' to time series unique identifiers - * @param timeSeriesToSourceRef - * Mapping from time series identifier to [[SourceRef]] - * @param simulationStart - * Wall clock time of the first instant in simulation - * @param primaryConfig - * The configuration for the sources - * @param mappingSource - * The mapping source - */ + * + * @param modelToTimeSeries + * Mapping from models' to time series unique identifiers + * @param timeSeriesToSourceRef + * Mapping from time series identifier to [[SourceRef]] + * @param simulationStart + * Wall clock time of the first instant in simulation + * @param primaryConfig + * The configuration for the sources + * @param mappingSource + * The mapping source + */ final case class PrimaryServiceStateData( - modelToTimeSeries: Map[UUID, UUID], - timeSeriesToSourceRef: Map[UUID, SourceRef], - simulationStart: ZonedDateTime, - primaryConfig: PrimaryConfig, - mappingSource: TimeSeriesMappingSource, - extSubscribers: Iterable[UUID] = Iterable.empty[UUID], - extPrimaryDataService: Option[ActorRef] = None - ) extends ServiceStateData + modelToTimeSeries: Map[UUID, UUID], + timeSeriesToSourceRef: Map[UUID, SourceRef], + simulationStart: ZonedDateTime, + primaryConfig: PrimaryConfig, + mappingSource: TimeSeriesMappingSource, + extSubscribers: Iterable[UUID] = Iterable.empty[UUID], + extPrimaryDataService: Option[ActorRef] = None + ) extends ServiceStateData /** Giving reference to the target time series and source worker. - * - * @param metaInformation - * Meta information (including column scheme) of the time series - * @param worker - * Optional reference to an already existing worker providing information - * on that time series - */ + * + * @param metaInformation + * Meta information (including column scheme) of the time series + * @param worker + * Optional reference to an already existing worker providing information + * on that time series + */ final case class SourceRef( - metaInformation: IndividualTimeSeriesMetaInformation, - worker: Option[ActorRef] - ) + metaInformation: IndividualTimeSeriesMetaInformation, + worker: Option[ActorRef] + ) /** Check if the config holds correct information to instantiate a mapping - * source - * - * @param primaryConfig - * Config entries for primary source - */ + * source + * + * @param primaryConfig + * Config entries for primary source + */ def checkConfig(primaryConfig: PrimaryConfig): Unit = { def checkTimePattern(dtfPattern: String): Unit = @@ -610,14 +641,14 @@ object PrimaryServiceProxy { case Some(x) => throw new InvalidConfigParameterException( s"Invalid configuration '$x' for a time series source.\nAvailable types:\n\t${supportedSources - .mkString("\n\t")}" + .mkString("\n\t")}" ) case None => throw new InvalidConfigParameterException( s"No configuration for a time series mapping source provided.\nPlease provide one of the available sources:\n\t${supportedSources - .mkString("\n\t")}" + .mkString("\n\t")}" ) } } } -} \ No newline at end of file +} diff --git a/src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala b/src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala index 37f2acc415..1bda76a02f 100644 --- a/src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala +++ b/src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala @@ -1,16 +1,32 @@ +/* + * © 2024. TU Dortmund University, + * Institute of Energy Systems, Energy Efficiency and Energy Economics, + * Research group Distribution grid planning and operation + */ + package edu.ie3.simona.service.results import edu.ie3.datamodel.models.result.ResultEntity import edu.ie3.simona.api.data.ontology.DataMessageFromExt import edu.ie3.simona.api.data.results.ExtResultsData -import edu.ie3.simona.api.data.results.ontology.{ProvideResultEntities, RequestResultEntities, ResultDataMessageFromExt} +import edu.ie3.simona.api.data.results.ontology.{ + ProvideResultEntities, + RequestResultEntities, + ResultDataMessageFromExt +} import edu.ie3.simona.exceptions.{InitializationException, ServiceException} import edu.ie3.simona.ontology.messages.services.ResultMessage.ResultResponseMessage import edu.ie3.simona.ontology.messages.services.ServiceMessage.ServiceRegistrationMessage import edu.ie3.simona.ontology.messages.services.DataMessage -import edu.ie3.simona.service.ServiceStateData.{InitializeServiceStateData, ServiceBaseStateData} +import edu.ie3.simona.service.ServiceStateData.{ + InitializeServiceStateData, + ServiceBaseStateData +} import edu.ie3.simona.service.{ExtDataSupport, SimonaService} -import edu.ie3.simona.service.results.ExtResultDataService.{ExtResultsStateData, InitExtResultsData} +import edu.ie3.simona.service.results.ExtResultDataService.{ + ExtResultsStateData, + InitExtResultsData +} import org.apache.pekko.actor.{ActorContext, ActorRef, Props} @@ -25,25 +41,24 @@ object ExtResultDataService { ) final case class ExtResultsStateData( - extResultsData: ExtResultsData, - uuidToActorRef: Map[UUID, ActorRef] = Map.empty[UUID, ActorRef], - extResultsMessage: Option[ResultDataMessageFromExt] = None, - resultStorage: Map[UUID, ResultEntity] = Map.empty - ) extends ServiceBaseStateData + extResultsData: ExtResultsData, + uuidToActorRef: Map[UUID, ActorRef] = Map.empty[UUID, ActorRef], + extResultsMessage: Option[ResultDataMessageFromExt] = None, + resultStorage: Map[UUID, ResultEntity] = Map.empty + ) extends ServiceBaseStateData final case class InitExtResultsData( - extResultsData: ExtResultsData - ) extends InitializeServiceStateData + extResultsData: ExtResultsData + ) extends InitializeServiceStateData } - class ExtResultDataService(override val scheduler: ActorRef) - extends SimonaService[ExtResultsStateData](scheduler) + extends SimonaService[ExtResultsStateData](scheduler) with ExtDataSupport[ExtResultsStateData] { override def init( - initServiceData: InitializeServiceStateData - ): Try[(ExtResultsStateData, Option[Long])] = { + initServiceData: InitializeServiceStateData + ): Try[(ExtResultsStateData, Option[Long])] = { initServiceData match { case InitExtResultsData(extResultsData) => val resultInitializedStateData = ExtResultsStateData(extResultsData) @@ -59,20 +74,20 @@ class ExtResultDataService(override val scheduler: ActorRef) } override protected def handleRegistrationRequest( - registrationMessage: ServiceRegistrationMessage - )(implicit serviceStateData: ExtResultsStateData): - Try[ExtResultsStateData] = - Failure( - ServiceException( - "For this service is no registration possible!" - ) - ) + registrationMessage: ServiceRegistrationMessage + )(implicit serviceStateData: ExtResultsStateData): Try[ExtResultsStateData] = + Failure( + ServiceException( + "For this service is no registration possible!" + ) + ) override protected def announceInformation( - tick: Long - )( - implicit serviceStateData: ExtResultsStateData, - ctx: ActorContext): (ExtResultsStateData, Option[Long]) = { + tick: Long + )(implicit + serviceStateData: ExtResultsStateData, + ctx: ActorContext + ): (ExtResultsStateData, Option[Long]) = { serviceStateData.extResultsMessage.getOrElse( throw ServiceException( "ExtResultDataService was triggered without ResultDataMessageFromExt available" @@ -85,64 +100,70 @@ class ExtResultDataService(override val scheduler: ActorRef) } /** Handle a message from outside the simulation - * - * @param extMsg - * the external incoming message - * @param serviceStateData - * the current state data of this service - * @return - * the updated state data - */ + * + * @param extMsg + * the external incoming message + * @param serviceStateData + * the current state data of this service + * @return + * the updated state data + */ override protected def handleDataMessage( - extMsg: DataMessageFromExt - )( - implicit serviceStateData: ExtResultsStateData + extMsg: DataMessageFromExt + )(implicit + serviceStateData: ExtResultsStateData ): ExtResultsStateData = { serviceStateData } /** Handle a message from inside SIMONA sent to external - * - * @param extResponseMsg - * the external incoming message - * @param serviceStateData - * the current state data of this service - * @return - * the updated state data - */ - override protected def handleDataResponseMessage(extResponseMsg: DataMessage)( - implicit serviceStateData: ExtResultsStateData): ExtResultsStateData = { + * + * @param extResponseMsg + * the external incoming message + * @param serviceStateData + * the current state data of this service + * @return + * the updated state data + */ + override protected def handleDataResponseMessage( + extResponseMsg: DataMessage + )(implicit serviceStateData: ExtResultsStateData): ExtResultsStateData = { extResponseMsg match { case ResultResponseMessage(result) => if (serviceStateData.uuidToActorRef.contains(result.getUuid)) { // A valid result was sent - val updatedResultStorage = serviceStateData.resultStorage + (result.getUuid -> result) - if (updatedResultStorage.size == serviceStateData.uuidToActorRef.size) { + val updatedResultStorage = + serviceStateData.resultStorage + (result.getUuid -> result) + if ( + updatedResultStorage.size == serviceStateData.uuidToActorRef.size + ) { // all responses received, forward them to external simulation in a bundle serviceStateData.extResultsData.queueExtResponseMsg( - new ProvideResultEntities(updatedResultStorage.values.toList.asJava) + new ProvideResultEntities( + updatedResultStorage.values.toList.asJava + ) ) serviceStateData.copy( resultStorage = Map.empty ) - } else { // responses are still incomplete serviceStateData.copy( resultStorage = updatedResultStorage ) } + } else { + serviceStateData } - else { - serviceStateData - } } } private def requestResults( - tick: Long - )(implicit serviceStateData: ExtResultsStateData): (ExtResultsStateData, Option[Long]) = { + tick: Long + )(implicit + serviceStateData: ExtResultsStateData + ): (ExtResultsStateData, Option[Long]) = { (serviceStateData.copy(), None) } } diff --git a/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala b/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala index 79fab9885f..06beffdb4b 100644 --- a/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala +++ b/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala @@ -8,7 +8,15 @@ package edu.ie3.simona.sim import org.apache.pekko.actor.typed.scaladsl.adapter.TypedActorRefOps import org.apache.pekko.actor.SupervisorStrategy.Stop -import org.apache.pekko.actor.{Actor, ActorRef, AllForOneStrategy, Props, Stash, SupervisorStrategy, Terminated} +import org.apache.pekko.actor.{ + Actor, + ActorRef, + AllForOneStrategy, + Props, + Stash, + SupervisorStrategy, + Terminated +} import com.typesafe.scalalogging.LazyLogging import edu.ie3.simona.agent.EnvironmentRefs import edu.ie3.simona.event.ResultEvent.PowerFlowResultEvent @@ -18,8 +26,17 @@ import edu.ie3.simona.ontology.messages.StopMessage import edu.ie3.simona.ontology.messages.services.ResultMessage.ResultResponseMessage import edu.ie3.simona.scheduler.TimeAdvancer import edu.ie3.simona.scheduler.TimeAdvancer.StartSimMessage -import edu.ie3.simona.sim.SimMessage.{InitSim, SimulationFailure, SimulationSuccessful, StartSimulation} -import edu.ie3.simona.sim.SimonaSim.{EmergencyShutdownInitiated, ResultMessage, SimonaSimStateData} +import edu.ie3.simona.sim.SimMessage.{ + InitSim, + SimulationFailure, + SimulationSuccessful, + StartSimulation +} +import edu.ie3.simona.sim.SimonaSim.{ + EmergencyShutdownInitiated, + ResultMessage, + SimonaSimStateData +} import edu.ie3.simona.sim.setup.{ExtSimSetupData, SimonaSetup} import scala.concurrent.duration.DurationInt @@ -143,7 +160,7 @@ class SimonaSim(simonaSetup: SimonaSetup) ) /* ExtSimulation */ - //case ResultMessage(res) => + // case ResultMessage(res) => // data.initSimSender ! res.getResults() case EmergencyShutdownInitiated => 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 b592f261e4..6710082936 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/ExtSimSetupData.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/ExtSimSetupData.scala @@ -19,7 +19,9 @@ final case class ExtSimSetupData( def evDataService: Option[ActorRef] = extDataServices.get(classOf[ExtEvDataService]) - def extPrimaryDataService: Option[ActorRef] = extDataServices.get(classOf[ExtPrimaryDataService]) + def extPrimaryDataService: Option[ActorRef] = + extDataServices.get(classOf[ExtPrimaryDataService]) - def extResultDataService: Option[ActorRef] = extDataServices.get(classOf[ExtResultDataService]) + def extResultDataService: Option[ActorRef] = + extDataServices.get(classOf[ExtResultDataService]) } 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 2ad94f46ec..fd5b7dc7f9 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala @@ -6,7 +6,11 @@ package edu.ie3.simona.sim.setup -import org.apache.pekko.actor.typed.scaladsl.adapter.{ClassicActorContextOps, ClassicActorRefOps, TypedActorRefOps} +import org.apache.pekko.actor.typed.scaladsl.adapter.{ + ClassicActorContextOps, + ClassicActorRefOps, + TypedActorRefOps +} import org.apache.pekko.actor.{ActorContext, ActorRef, ActorSystem} import com.typesafe.config.Config import com.typesafe.scalalogging.LazyLogging @@ -19,7 +23,10 @@ import edu.ie3.simona.agent.grid.GridAgent import edu.ie3.simona.api.ExtSimAdapter import edu.ie3.simona.api.data.ExtData import edu.ie3.simona.api.data.ev.{ExtEvData, ExtEvSimulation} -import edu.ie3.simona.api.data.primarydata.{ExtPrimaryData, ExtPrimaryDataSimulation} +import edu.ie3.simona.api.data.primarydata.{ + ExtPrimaryData, + ExtPrimaryDataSimulation +} import edu.ie3.simona.api.data.results.{ExtResultDataSimulation, ExtResultsData} import edu.ie3.simona.api.simulation.ExtSimAdapterData import edu.ie3.simona.config.{ArgsParser, RefSystemParser, SimonaConfig} @@ -33,7 +40,10 @@ import edu.ie3.simona.service.SimonaService import edu.ie3.simona.service.ev.ExtEvDataService import edu.ie3.simona.service.ev.ExtEvDataService.InitExtEvData import edu.ie3.simona.service.primary.ExtPrimaryDataService.InitExtPrimaryData -import edu.ie3.simona.service.primary.{ExtPrimaryDataService, PrimaryServiceProxy} +import edu.ie3.simona.service.primary.{ + ExtPrimaryDataService, + PrimaryServiceProxy +} import edu.ie3.simona.service.primary.PrimaryServiceProxy.InitPrimaryServiceProxyStateData import edu.ie3.simona.service.results.ExtResultDataService import edu.ie3.simona.service.results.ExtResultDataService.InitExtResultsData @@ -235,7 +245,11 @@ class SimonaStandaloneSetup( ExtPrimaryDataService.props(scheduler), s"$index-$dIndex" ) - val extPrimaryData = new ExtPrimaryData(extPrimaryDataService, extSimAdapter, extPrimaryDataSimulation.getFactory) + val extPrimaryData = new ExtPrimaryData( + extPrimaryDataService, + extSimAdapter, + extPrimaryDataSimulation.getFactory + ) extPrimaryDataService ! SimonaService.Create( InitExtPrimaryData(extPrimaryData), @@ -253,7 +267,8 @@ class SimonaStandaloneSetup( ExtResultDataService.props(scheduler), s"$index-$dIndex" ) - val extResultsData = new ExtResultsData(extResultDataService, extSimAdapter, null) + val extResultsData = + new ExtResultsData(extResultDataService, extSimAdapter, null) extResultDataService ! SimonaService.Create( InitExtResultsData(extResultsData), @@ -264,7 +279,10 @@ class SimonaStandaloneSetup( ) ) - (extResultsData, (classOf[ExtResultDataService], extResultDataService)) + ( + extResultsData, + (classOf[ExtResultDataService], extResultDataService) + ) }.unzip extLink.getExtSimulation.setup( @@ -335,7 +353,8 @@ class SimonaStandaloneSetup( context: ActorContext, extSimulationData: ExtSimSetupData ): Seq[ActorRef] = { - val extResultDataService: Option[ActorRef] = extSimulationData.extResultDataService + val extResultDataService: Option[ActorRef] = + extSimulationData.extResultDataService // append ResultEventListener as well to write raw output files ArgsParser .parseListenerConfigOption(simonaConfig.simona.event.listener) diff --git a/src/test/scala/edu/ie3/simona/service/primary/ExtPrimaryDataServiceSpec.scala b/src/test/scala/edu/ie3/simona/service/primary/ExtPrimaryDataServiceSpec.scala index 66458ef384..711e5f000e 100644 --- a/src/test/scala/edu/ie3/simona/service/primary/ExtPrimaryDataServiceSpec.scala +++ b/src/test/scala/edu/ie3/simona/service/primary/ExtPrimaryDataServiceSpec.scala @@ -7,7 +7,8 @@ package edu.ie3.simona.service.primary import com.typesafe.config.ConfigFactory -import edu.ie3.simona.test.common.service.{PrimaryDataFactoryDefault, ResultFactoryDefault} +import edu.ie3.simona.test.common.service. + PrimaryDataFactoryDefault import edu.ie3.datamodel.io.factory.timeseries.TimeBasedSimpleValueFactory import edu.ie3.datamodel.io.naming.FileNamingStrategy import edu.ie3.datamodel.io.source.csv.CsvTimeSeriesSource @@ -21,18 +22,32 @@ import edu.ie3.simona.api.data.ontology.ScheduleDataServiceMessage import edu.ie3.simona.api.data.primarydata.ExtPrimaryData import edu.ie3.simona.exceptions.ServiceException import edu.ie3.simona.ontology.messages.Activation -import edu.ie3.simona.ontology.messages.SchedulerMessage.{Completion, ScheduleActivation} +import edu.ie3.simona.ontology.messages.SchedulerMessage.{ + Completion, + ScheduleActivation +} import edu.ie3.simona.ontology.messages.services.EvMessage._ -import edu.ie3.simona.ontology.messages.services.ServiceMessage.{ExtPrimaryDataServiceRegistrationMessage, WorkerRegistrationMessage} +import edu.ie3.simona.ontology.messages.services.ServiceMessage.{ + ExtPrimaryDataServiceRegistrationMessage, + WorkerRegistrationMessage +} import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.RegistrationSuccessfulMessage import edu.ie3.simona.ontology.messages.services.WeatherMessage.RegisterForWeatherMessage import edu.ie3.simona.scheduler.ScheduleLock import edu.ie3.simona.service.SimonaService import edu.ie3.simona.service.ev.ExtEvDataService.InitExtEvData import edu.ie3.simona.service.primary.ExtPrimaryDataService.InitExtPrimaryData -import edu.ie3.simona.service.primary.PrimaryServiceWorker.{CsvInitPrimaryServiceStateData, PrimaryServiceInitializedStateData, ProvidePrimaryDataMessage} +import edu.ie3.simona.service.primary.PrimaryServiceWorker.{ + CsvInitPrimaryServiceStateData, + PrimaryServiceInitializedStateData, + ProvidePrimaryDataMessage +} import edu.ie3.simona.service.primary.PrimaryServiceWorkerSpec.WrongInitPrimaryServiceStateData -import edu.ie3.simona.test.common.{EvTestData, TestKitWithShutdown, TestSpawnerClassic} +import edu.ie3.simona.test.common.{ + EvTestData, + TestKitWithShutdown, + TestSpawnerClassic +} import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK import edu.ie3.util.TimeUtil import edu.ie3.util.quantities.PowerSystemUnits @@ -79,10 +94,8 @@ class ExtPrimaryDataServiceSpec private val participant1UUID = UUID.fromString("06a14909-366e-4e94-a593-1016e1455b30") - "An uninitialized external primary data service" must { - "send correct completion message after initialisation" in { val primaryDataService = TestActorRef( new ExtPrimaryDataService( @@ -96,7 +109,10 @@ class ExtPrimaryDataServiceSpec scheduler.send( primaryDataService, - SimonaService.Create(InitExtPrimaryData(extPrimaryData(primaryDataService)), key) + SimonaService.Create( + InitExtPrimaryData(extPrimaryData(primaryDataService)), + key + ) ) scheduler.expectMsg( ScheduleActivation(primaryDataService.toTyped, INIT_SIM_TICK, Some(key)) @@ -123,7 +139,10 @@ class ExtPrimaryDataServiceSpec val systemParticipant: TestProbe = TestProbe("dummySystemParticipant") "correctly register a forwarded request" in { - serviceRef ! ExtPrimaryDataServiceRegistrationMessage(UUID.randomUUID(), systemParticipant.ref) + serviceRef ! ExtPrimaryDataServiceRegistrationMessage( + UUID.randomUUID(), + systemParticipant.ref + ) println("Try to register") /* Wait for request approval */ @@ -136,4 +155,4 @@ class ExtPrimaryDataServiceSpec systemParticipant.expectMsgAllClassOf(classOf[ProvidePrimaryDataMessage]) } } -} \ No newline at end of file +} diff --git a/src/test/scala/edu/ie3/simona/service/primary/PrimaryServiceProxySpec.scala b/src/test/scala/edu/ie3/simona/service/primary/PrimaryServiceProxySpec.scala index a54b3b85fe..c51b6d0cde 100644 --- a/src/test/scala/edu/ie3/simona/service/primary/PrimaryServiceProxySpec.scala +++ b/src/test/scala/edu/ie3/simona/service/primary/PrimaryServiceProxySpec.scala @@ -129,7 +129,7 @@ class PrimaryServiceProxySpec private val scheduler: TestProbe = TestProbe("scheduler") - val validExtPrimaryDataService = TestActorRef( + private val validExtPrimaryDataService = TestActorRef( new ExtPrimaryDataService( scheduler.ref ) @@ -358,21 +358,21 @@ class PrimaryServiceProxySpec Some(validExtPrimaryDataService) ) match { case Success( - PrimaryServiceStateData( - modelToTimeSeries, - timeSeriesToSourceRef, - simulationStart, - primaryConfig, - mappingSource, - extSubscribers, - extPrimaryDataService - ) - ) => extPrimaryDataService should contain (validExtPrimaryDataService) + PrimaryServiceStateData( + modelToTimeSeries, + timeSeriesToSourceRef, + simulationStart, + primaryConfig, + mappingSource, + extSubscribers, + extPrimaryDataService + ) + ) => + extPrimaryDataService should contain(validExtPrimaryDataService) extSubscribers shouldBe Iterable.empty } } - } "Sending initialization information to an uninitialized actor" should { diff --git a/src/test/scala/edu/ie3/simona/test/common/service/PrimaryDataFactoryDefault.scala b/src/test/scala/edu/ie3/simona/test/common/service/PrimaryDataFactoryDefault.scala index 7f7ce9c1fd..e3d0bc0714 100644 --- a/src/test/scala/edu/ie3/simona/test/common/service/PrimaryDataFactoryDefault.scala +++ b/src/test/scala/edu/ie3/simona/test/common/service/PrimaryDataFactoryDefault.scala @@ -1,3 +1,9 @@ +/* + * © 2024. 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.service import edu.ie3.datamodel.models.value.Value @@ -5,6 +11,8 @@ import edu.ie3.simona.api.data.primarydata.PrimaryDataFactory class PrimaryDataFactoryDefault extends PrimaryDataFactory { - /** Should convert an object to an primary data value with a check if the object is primary data */ - override def convertObjectToValue(entity: Any): Value = ??? + /** Should convert an object to an primary data value with a check if the + * object is primary data + */ + override def convertObjectToValue(entity: Any): Value = null } diff --git a/src/test/scala/edu/ie3/simona/test/common/service/ResultFactoryDefault.scala b/src/test/scala/edu/ie3/simona/test/common/service/ResultFactoryDefault.scala deleted file mode 100644 index 897bc03b4d..0000000000 --- a/src/test/scala/edu/ie3/simona/test/common/service/ResultFactoryDefault.scala +++ /dev/null @@ -1,22 +0,0 @@ -package edu.ie3.simona.test.common.service - -import edu.ie3.datamodel.models.result.{NodeResult, ResultEntity} -import edu.ie3.datamodel.models.result.connector.ConnectorResult -import edu.ie3.datamodel.models.result.system.{ElectricalEnergyStorageResult, SystemParticipantResult, SystemParticipantWithHeatResult} -import edu.ie3.datamodel.models.result.thermal.ThermalUnitResult -import edu.ie3.simona.api.data.results.ResultDataFactory -class ResultFactoryDefault extends ResultDataFactory { - @throws[Exception] - override def convertResultToString(entity: ResultEntity): AnyRef = { - var resultObject: String = null - if (entity.isInstanceOf[SystemParticipantWithHeatResult]) resultObject = "{\"p\":\"" + systemParticipantWithHeatResult.getP + ",\"q\":\"" + systemParticipantWithHeatResult.getQ + ",\"qDot\":\"" + systemParticipantWithHeatResult.getqDot + "\"}" - else if (entity.isInstanceOf[ElectricalEnergyStorageResult]) resultObject = "{\"p\":\"" + electricalEnergyStorageResult.getP + ",\"q\":\"" + electricalEnergyStorageResult.getQ + ",\"soc\":\"" + electricalEnergyStorageResult.getSoc + "\"}" - else if (entity.isInstanceOf[ConnectorResult]) resultObject = "{\"iAMag\":\"" + connectorResult.getiAMag + ",\"iAAng\":\"" + connectorResult.getiAAng + ",\"iBMag\":\"" + connectorResult.getiBMag + ",\"iBAng\":\"" + connectorResult.getiBAng + "\"}" - else if (entity.isInstanceOf[NodeResult]) resultObject = "{\"vMag\":\"" + nodeResult.getvMag + ",\"vAng\":\"" + nodeResult.getvAng + "\"}" - else if (entity.isInstanceOf[ThermalUnitResult]) resultObject = "{\"qDot\":\"" + thermalUnitResult.getqDot + "\"}" - else if (entity.isInstanceOf[SystemParticipantResult]) resultObject = "{\"p\":\"" + systemParticipantResult.getP + ",\"q\":\"" + systemParticipantResult.getQ + "\"}" - else resultObject = "{}" - resultObject - } -} - From 4c22d875aeec704d4b52c19afd1bb63866deeb33 Mon Sep 17 00:00:00 2001 From: jo-bao Date: Wed, 21 Feb 2024 11:48:53 +0100 Subject: [PATCH 06/41] spotless scala --- .../event/listener/ResultEventListener.scala | 6 +-- .../messages/services/ServiceMessage.scala | 3 +- .../primary/ExtPrimaryDataService.scala | 41 ++++++++++--------- .../service/primary/PrimaryServiceProxy.scala | 18 ++++---- .../results/ExtResultDataService.scala | 10 ++--- .../scala/edu/ie3/simona/sim/SimonaSim.scala | 2 +- .../ie3/simona/sim/setup/SimonaSetup.scala | 4 +- .../sim/setup/SimonaStandaloneSetup.scala | 28 ++++++------- .../primary/ExtPrimaryDataServiceSpec.scala | 25 +++++------ .../primary/PrimaryServiceProxySpec.scala | 16 ++++---- .../primary/PrimaryServiceProxySqlIT.scala | 2 +- .../ie3/simona/sim/SimonaSimFailSpec.scala | 5 ++- .../simona/sim/setup/SimonaSetupSpec.scala | 5 ++- 13 files changed, 84 insertions(+), 81 deletions(-) diff --git a/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala b/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala index bcdc705ae4..9469af8bb9 100644 --- a/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala +++ b/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala @@ -169,7 +169,7 @@ object ResultEventListener extends Transformer3wResultSupport { handOverToExternalService( resultEntity, baseData.extResultDataService, - log + log, ) } baseData @@ -249,7 +249,7 @@ object ResultEventListener extends Transformer3wResultSupport { private def handOverToExternalService( resultEntity: ResultEntity, extResultDataService: Option[ActorRef], - log: Logger + log: Logger, ): Unit = Try { val extResultDataServiceRef = extResultDataService.getOrElse( throw new Exception("No external data service registered!") @@ -259,7 +259,7 @@ object ResultEventListener extends Transformer3wResultSupport { def apply( resultFileHierarchy: ResultFileHierarchy, - extResultDataService: Option[ActorRef] = Option.empty[ActorRef] + extResultDataService: Option[ActorRef] = Option.empty[ActorRef], ): Behavior[ResultMessage] = Behaviors.setup[ResultMessage] { ctx => ctx.log.debug("Starting initialization!") resultFileHierarchy.resultSinkType match { diff --git a/src/main/scala/edu/ie3/simona/ontology/messages/services/ServiceMessage.scala b/src/main/scala/edu/ie3/simona/ontology/messages/services/ServiceMessage.scala index 27045677f5..769a29daae 100644 --- a/src/main/scala/edu/ie3/simona/ontology/messages/services/ServiceMessage.scala +++ b/src/main/scala/edu/ie3/simona/ontology/messages/services/ServiceMessage.scala @@ -44,10 +44,9 @@ case object ServiceMessage { final case class ExtPrimaryDataServiceRegistrationMessage( modelUuid: UUID, - requestingActor: ActorRef + requestingActor: ActorRef, ) extends ServiceRegistrationMessage - sealed trait RegistrationResponseMessage extends ServiceMessage { val serviceRef: ActorRef } diff --git a/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala b/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala index 81828f795a..0fce0932b8 100644 --- a/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala +++ b/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala @@ -12,25 +12,25 @@ import edu.ie3.simona.api.data.ontology.DataMessageFromExt import edu.ie3.simona.api.data.primarydata.ExtPrimaryData import edu.ie3.simona.api.data.primarydata.ontology.{ PrimaryDataMessageFromExt, - ProvidePrimaryData + ProvidePrimaryData, } import edu.ie3.simona.exceptions.WeatherServiceException.InvalidRegistrationRequestException import edu.ie3.simona.exceptions.{InitializationException, ServiceException} import edu.ie3.simona.ontology.messages.services.ServiceMessage.{ ExtPrimaryDataServiceRegistrationMessage, PrimaryServiceRegistrationMessage, - WorkerRegistrationMessage + WorkerRegistrationMessage, } import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.RegistrationSuccessfulMessage import edu.ie3.simona.ontology.messages.services.{DataMessage, ServiceMessage} import edu.ie3.simona.scheduler.ScheduleLock import edu.ie3.simona.service.ServiceStateData.{ InitializeServiceStateData, - ServiceBaseStateData + ServiceBaseStateData, } import edu.ie3.simona.service.primary.ExtPrimaryDataService.{ ExtPrimaryDataStateData, - InitExtPrimaryData + InitExtPrimaryData, } import edu.ie3.simona.service.primary.PrimaryServiceWorker.ProvidePrimaryDataMessage import edu.ie3.simona.service.{ExtDataSupport, ServiceStateData, SimonaService} @@ -52,7 +52,7 @@ object ExtPrimaryDataService { extPrimaryData: ExtPrimaryData, uuidToActorRef: Map[UUID, ActorRef] = Map.empty[UUID, ActorRef], // subscribers in SIMONA - extPrimaryDataMessage: Option[PrimaryDataMessageFromExt] = None + extPrimaryDataMessage: Option[PrimaryDataMessageFromExt] = None, ) extends ServiceBaseStateData case class InitExtPrimaryData( @@ -75,7 +75,7 @@ final case class ExtPrimaryDataService( Success( primaryDataInitializedStateData, - None + None, ) case invalidData => @@ -95,7 +95,7 @@ final case class ExtPrimaryDataService( registrationMessage match { case ExtPrimaryDataServiceRegistrationMessage( modelUuid, - requestingActor + requestingActor, ) => println("Habe erhalten") Success(handleRegistrationRequest(requestingActor, modelUuid)) @@ -110,14 +110,14 @@ final case class ExtPrimaryDataService( private def handleRegistrationRequest( agentToBeRegistered: ActorRef, - agentUUID: UUID + agentUUID: UUID, )(implicit serviceStateData: ExtPrimaryDataStateData ): ExtPrimaryDataStateData = { serviceStateData.uuidToActorRef.get(agentUUID) match { case None => // Actor is not registered yet - agentToBeRegistered ! RegistrationSuccessfulMessage(None) + agentToBeRegistered ! RegistrationSuccessfulMessage(self, None) serviceStateData.copy( uuidToActorRef = serviceStateData.uuidToActorRef + (agentUUID -> agentToBeRegistered) @@ -126,7 +126,7 @@ final case class ExtPrimaryDataService( // actor is already registered, do nothing log.warning( "Sending actor {} is already registered", - agentToBeRegistered + agentToBeRegistered, ) serviceStateData } @@ -147,7 +147,7 @@ final case class ExtPrimaryDataService( tick: Long )(implicit serviceStateData: ExtPrimaryDataStateData, - ctx: ActorContext + ctx: ActorContext, ): (ExtPrimaryDataStateData, Option[Long]) = { serviceStateData.extPrimaryDataMessage.getOrElse( throw ServiceException( @@ -157,20 +157,20 @@ final case class ExtPrimaryDataService( case providedPrimaryData: ProvidePrimaryData => processDataAndAnnounce(tick, providedPrimaryData.primaryData)( serviceStateData, - ctx + ctx, ) } } private def processDataAndAnnounce( tick: Long, - primaryData: java.util.Map[UUID, Value] + primaryData: java.util.Map[UUID, Value], )(implicit serviceStateData: ExtPrimaryDataStateData, - ctx: ActorContext + ctx: ActorContext, ): ( ExtPrimaryDataStateData, - Option[Long] + Option[Long], ) = { val actorToPrimaryData = primaryData.asScala.flatMap { case (agent, primaryDataPerAgent) => @@ -180,7 +180,7 @@ final case class ExtPrimaryDataService( .orElse { log.warning( "A corresponding actor ref for UUID {} could not be found", - agent + agent, ) None } @@ -193,7 +193,7 @@ final case class ExtPrimaryDataService( ctx, scheduler.toTyped, tick, - actorToPrimaryData.size + actorToPrimaryData.size, ) actorToPrimaryData.zip(keys).foreach { @@ -202,16 +202,17 @@ final case class ExtPrimaryDataService( case Success(primaryData) => actor ! ProvidePrimaryDataMessage( tick, + self, primaryData, None, - unlockKey = Some(key) + unlockKey = Some(key), ) case Failure(exception) => /* Processing of data failed */ log.warning( "Unable to convert received value to primary data. Skipped that data." + "\nException: {}", - exception + exception, ) } } @@ -219,7 +220,7 @@ final case class ExtPrimaryDataService( ( // Message leeren serviceStateData.copy(extPrimaryDataMessage = None), - None + None, ) } diff --git a/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala b/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala index 3a49743c36..b964cf228e 100644 --- a/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala +++ b/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala @@ -109,7 +109,7 @@ case class PrimaryServiceProxy( prepareStateData( initStateData.primaryConfig, initStateData.simulationStart, - initStateData.extSimulation + initStateData.extSimulation, ) match { case Success(stateData) => scheduler ! Completion(self.toTyped) @@ -142,7 +142,7 @@ case class PrimaryServiceProxy( private def prepareStateData( primaryConfig: PrimaryConfig, simulationStart: ZonedDateTime, - extSimulation: Option[ActorRef] + extSimulation: Option[ActorRef], ): Try[PrimaryServiceStateData] = { createSources(primaryConfig).map { @@ -181,7 +181,7 @@ case class PrimaryServiceProxy( primaryConfig, mappingSource, getSubscribers, - extSimulation + extSimulation, ) } else { PrimaryServiceStateData( @@ -189,7 +189,7 @@ case class PrimaryServiceProxy( timeSeriesToSourceRef, simulationStart, primaryConfig, - mappingSource + mappingSource, ) } } @@ -280,14 +280,14 @@ case class PrimaryServiceProxy( } else { log.debug( s"There is no time series apparent for the model with uuid '{}'.", - modelUuid + modelUuid, ) sender() ! RegistrationFailedMessage } } else { log.debug( s"There is no time series apparent for the model with uuid '{}'.", - modelUuid + modelUuid, ) sender() ! RegistrationFailedMessage } @@ -358,7 +358,7 @@ case class PrimaryServiceProxy( protected def handleExternalModel( modelUuid: UUID, stateData: PrimaryServiceStateData, - requestingActor: ActorRef + requestingActor: ActorRef, ): Unit = { stateData.extPrimaryDataService match { case Some(reqActor) => @@ -551,7 +551,7 @@ object PrimaryServiceProxy { final case class InitPrimaryServiceProxyStateData( primaryConfig: PrimaryConfig, simulationStart: ZonedDateTime, - extSimulation: Option[ActorRef] + extSimulation: Option[ActorRef], ) extends InitializeServiceStateData /** Holding the state of an initialized proxy. @@ -574,7 +574,7 @@ object PrimaryServiceProxy { primaryConfig: PrimaryConfig, mappingSource: TimeSeriesMappingSource, extSubscribers: Iterable[UUID] = Iterable.empty[UUID], - extPrimaryDataService: Option[ActorRef] = None + extPrimaryDataService: Option[ActorRef] = None, ) extends ServiceStateData /** Giving reference to the target time series and source worker. diff --git a/src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala b/src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala index 1bda76a02f..790cc1bb30 100644 --- a/src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala +++ b/src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala @@ -12,7 +12,7 @@ import edu.ie3.simona.api.data.results.ExtResultsData import edu.ie3.simona.api.data.results.ontology.{ ProvideResultEntities, RequestResultEntities, - ResultDataMessageFromExt + ResultDataMessageFromExt, } import edu.ie3.simona.exceptions.{InitializationException, ServiceException} import edu.ie3.simona.ontology.messages.services.ResultMessage.ResultResponseMessage @@ -20,12 +20,12 @@ import edu.ie3.simona.ontology.messages.services.ServiceMessage.ServiceRegistrat import edu.ie3.simona.ontology.messages.services.DataMessage import edu.ie3.simona.service.ServiceStateData.{ InitializeServiceStateData, - ServiceBaseStateData + ServiceBaseStateData, } import edu.ie3.simona.service.{ExtDataSupport, SimonaService} import edu.ie3.simona.service.results.ExtResultDataService.{ ExtResultsStateData, - InitExtResultsData + InitExtResultsData, } import org.apache.pekko.actor.{ActorContext, ActorRef, Props} @@ -44,7 +44,7 @@ object ExtResultDataService { extResultsData: ExtResultsData, uuidToActorRef: Map[UUID, ActorRef] = Map.empty[UUID, ActorRef], extResultsMessage: Option[ResultDataMessageFromExt] = None, - resultStorage: Map[UUID, ResultEntity] = Map.empty + resultStorage: Map[UUID, ResultEntity] = Map.empty, ) extends ServiceBaseStateData final case class InitExtResultsData( @@ -86,7 +86,7 @@ class ExtResultDataService(override val scheduler: ActorRef) tick: Long )(implicit serviceStateData: ExtResultsStateData, - ctx: ActorContext + ctx: ActorContext, ): (ExtResultsStateData, Option[Long]) = { serviceStateData.extResultsMessage.getOrElse( throw ServiceException( diff --git a/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala b/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala index 92ce13ea44..c2c167fe76 100644 --- a/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala +++ b/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala @@ -35,7 +35,7 @@ import edu.ie3.simona.sim.SimMessage.{ import edu.ie3.simona.sim.SimonaSim.{ EmergencyShutdownInitiated, ResultMessage, - SimonaSimStateData + SimonaSimStateData, } import edu.ie3.simona.sim.setup.{ExtSimSetupData, SimonaSetup} 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 6245c20a00..67154f27b0 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaSetup.scala @@ -55,7 +55,7 @@ trait SimonaSetup { */ def systemParticipantsListener( context: ActorContext, - extSimulationData: ExtSimSetupData + extSimulationData: ExtSimSetupData, ): Seq[ActorRef] /** Creates a primary service proxy. The proxy is the first instance to ask @@ -72,7 +72,7 @@ trait SimonaSetup { def primaryServiceProxy( context: ActorContext, scheduler: ActorRef, - extSimulationData: ExtSimSetupData + extSimulationData: ExtSimSetupData, ): ActorRef /** Creates a weather service 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 002e1e319f..dd1e2adc89 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala @@ -25,7 +25,7 @@ import edu.ie3.simona.api.data.ExtData import edu.ie3.simona.api.data.ev.{ExtEvData, ExtEvSimulation} import edu.ie3.simona.api.data.primarydata.{ ExtPrimaryData, - ExtPrimaryDataSimulation + ExtPrimaryDataSimulation, } import edu.ie3.simona.api.data.results.{ExtResultDataSimulation, ExtResultsData} import edu.ie3.simona.api.simulation.ExtSimAdapterData @@ -42,7 +42,7 @@ import edu.ie3.simona.service.ev.ExtEvDataService.InitExtEvData import edu.ie3.simona.service.primary.ExtPrimaryDataService.InitExtPrimaryData import edu.ie3.simona.service.primary.{ ExtPrimaryDataService, - PrimaryServiceProxy + PrimaryServiceProxy, } import edu.ie3.simona.service.primary.PrimaryServiceProxy.InitPrimaryServiceProxyStateData import edu.ie3.simona.service.results.ExtResultDataService @@ -149,7 +149,7 @@ class SimonaStandaloneSetup( override def primaryServiceProxy( context: ActorContext, scheduler: ActorRef, - extSimSetupData: ExtSimSetupData + extSimSetupData: ExtSimSetupData, ): ActorRef = { val simulationStart = TimeUtil.withDefaults.toZonedDateTime( simonaConfig.simona.time.startDateTime @@ -160,7 +160,7 @@ class SimonaStandaloneSetup( InitPrimaryServiceProxyStateData( simonaConfig.simona.input.primary, simulationStart, - extSimSetupData.extPrimaryDataService + extSimSetupData.extPrimaryDataService, ), simulationStart, ) @@ -243,12 +243,12 @@ class SimonaStandaloneSetup( case (extPrimaryDataSimulation: ExtPrimaryDataSimulation, dIndex) => val extPrimaryDataService = context.simonaActorOf( ExtPrimaryDataService.props(scheduler), - s"$index-$dIndex" + s"$index-$dIndex", ) val extPrimaryData = new ExtPrimaryData( extPrimaryDataService, extSimAdapter, - extPrimaryDataSimulation.getFactory + extPrimaryDataSimulation.getFactory, ) extPrimaryDataService ! SimonaService.Create( @@ -256,8 +256,8 @@ class SimonaStandaloneSetup( ScheduleLock.singleKey( context, scheduler.toTyped, - INIT_SIM_TICK - ) + INIT_SIM_TICK, + ), ) (null, (classOf[ExtResultDataService], extPrimaryDataService)) @@ -265,7 +265,7 @@ class SimonaStandaloneSetup( case (_: ExtResultDataSimulation, dIndex) => val extResultDataService = context.simonaActorOf( ExtResultDataService.props(scheduler), - s"$index-$dIndex" + s"$index-$dIndex", ) val extResultsData = new ExtResultsData(extResultDataService, extSimAdapter, null) @@ -275,13 +275,13 @@ class SimonaStandaloneSetup( ScheduleLock.singleKey( context, scheduler.toTyped, - INIT_SIM_TICK - ) + INIT_SIM_TICK, + ), ) ( extResultsData, - (classOf[ExtResultDataService], extResultDataService) + (classOf[ExtResultDataService], extResultDataService), ) }.unzip @@ -351,7 +351,7 @@ class SimonaStandaloneSetup( override def systemParticipantsListener( context: ActorContext, - extSimulationData: ExtSimSetupData + extSimulationData: ExtSimSetupData, ): Seq[ActorRef] = { val extResultDataService: Option[ActorRef] = extSimulationData.extResultDataService @@ -369,7 +369,7 @@ class SimonaStandaloneSetup( .spawn( ResultEventListener( resultFileHierarchy, - extResultDataService + extResultDataService, ), ResultEventListener.getClass.getSimpleName, ) diff --git a/src/test/scala/edu/ie3/simona/service/primary/ExtPrimaryDataServiceSpec.scala b/src/test/scala/edu/ie3/simona/service/primary/ExtPrimaryDataServiceSpec.scala index 711e5f000e..1de4096bbb 100644 --- a/src/test/scala/edu/ie3/simona/service/primary/ExtPrimaryDataServiceSpec.scala +++ b/src/test/scala/edu/ie3/simona/service/primary/ExtPrimaryDataServiceSpec.scala @@ -7,8 +7,7 @@ package edu.ie3.simona.service.primary import com.typesafe.config.ConfigFactory -import edu.ie3.simona.test.common.service. - PrimaryDataFactoryDefault +import edu.ie3.simona.test.common.service.PrimaryDataFactoryDefault import edu.ie3.datamodel.io.factory.timeseries.TimeBasedSimpleValueFactory import edu.ie3.datamodel.io.naming.FileNamingStrategy import edu.ie3.datamodel.io.source.csv.CsvTimeSeriesSource @@ -24,12 +23,12 @@ import edu.ie3.simona.exceptions.ServiceException import edu.ie3.simona.ontology.messages.Activation import edu.ie3.simona.ontology.messages.SchedulerMessage.{ Completion, - ScheduleActivation + ScheduleActivation, } import edu.ie3.simona.ontology.messages.services.EvMessage._ import edu.ie3.simona.ontology.messages.services.ServiceMessage.{ ExtPrimaryDataServiceRegistrationMessage, - WorkerRegistrationMessage + WorkerRegistrationMessage, } import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.RegistrationSuccessfulMessage import edu.ie3.simona.ontology.messages.services.WeatherMessage.RegisterForWeatherMessage @@ -40,13 +39,13 @@ import edu.ie3.simona.service.primary.ExtPrimaryDataService.InitExtPrimaryData import edu.ie3.simona.service.primary.PrimaryServiceWorker.{ CsvInitPrimaryServiceStateData, PrimaryServiceInitializedStateData, - ProvidePrimaryDataMessage + ProvidePrimaryDataMessage, } import edu.ie3.simona.service.primary.PrimaryServiceWorkerSpec.WrongInitPrimaryServiceStateData import edu.ie3.simona.test.common.{ EvTestData, TestKitWithShutdown, - TestSpawnerClassic + TestSpawnerClassic, } import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK import edu.ie3.util.TimeUtil @@ -74,7 +73,7 @@ class ExtPrimaryDataServiceSpec .parseString(""" |pekko.loggers = ["org.apache.pekko.testkit.TestEventListener"] |pekko.loglevel = "INFO" - |""".stripMargin) + |""".stripMargin), ) ) with TestSpawnerClassic { @@ -88,7 +87,7 @@ class ExtPrimaryDataServiceSpec new ExtPrimaryData( dataService, extSimAdapter.ref, - primaryDataFactory + primaryDataFactory, ) private val participant1UUID = @@ -111,8 +110,8 @@ class ExtPrimaryDataServiceSpec primaryDataService, SimonaService.Create( InitExtPrimaryData(extPrimaryData(primaryDataService)), - key - ) + key, + ), ) scheduler.expectMsg( ScheduleActivation(primaryDataService.toTyped, INIT_SIM_TICK, Some(key)) @@ -141,12 +140,14 @@ class ExtPrimaryDataServiceSpec "correctly register a forwarded request" in { serviceRef ! ExtPrimaryDataServiceRegistrationMessage( UUID.randomUUID(), - systemParticipant.ref + systemParticipant.ref, ) println("Try to register") /* Wait for request approval */ - systemParticipant.expectMsg(RegistrationSuccessfulMessage(Some(0L))) + systemParticipant.expectMsg( + RegistrationSuccessfulMessage(systemParticipant.ref, Some(0L)) + ) /* We cannot directly check, if the requesting actor is among the subscribers, therefore we ask the actor to * provide data to all subscribed actors and check, if the subscribed probe gets one */ diff --git a/src/test/scala/edu/ie3/simona/service/primary/PrimaryServiceProxySpec.scala b/src/test/scala/edu/ie3/simona/service/primary/PrimaryServiceProxySpec.scala index 3aa0fa5b89..ee5ac4ac67 100644 --- a/src/test/scala/edu/ie3/simona/service/primary/PrimaryServiceProxySpec.scala +++ b/src/test/scala/edu/ie3/simona/service/primary/PrimaryServiceProxySpec.scala @@ -245,7 +245,7 @@ class PrimaryServiceProxySpec InitPrimaryServiceProxyStateData( validPrimaryConfig, simulationStart, - None + None, ) val proxyRef: TestActorRef[PrimaryServiceProxy] = TestActorRef( new PrimaryServiceProxy(scheduler.ref, initStateData, simulationStart) @@ -267,7 +267,7 @@ class PrimaryServiceProxySpec proxy invokePrivate prepareStateData( maliciousConfig, simulationStart, - Option.empty + Option.empty, ) match { case Success(_) => fail("Building state data with missing config should fail") @@ -288,7 +288,7 @@ class PrimaryServiceProxySpec proxy invokePrivate prepareStateData( maliciousConfig, simulationStart, - Option.empty + Option.empty, ) match { case Success(_) => fail("Building state data with missing config should fail") @@ -302,7 +302,7 @@ class PrimaryServiceProxySpec proxy invokePrivate prepareStateData( validPrimaryConfig, simulationStart, - Option.empty + Option.empty, ) match { case Success( PrimaryServiceStateData( @@ -312,7 +312,7 @@ class PrimaryServiceProxySpec primaryConfig, mappingSource, extSubscribers, - extPrimaryDataService + extPrimaryDataService, ) ) => modelToTimeSeries shouldBe Map( @@ -355,7 +355,7 @@ class PrimaryServiceProxySpec proxy invokePrivate prepareStateData( validPrimaryConfig, simulationStart, - Some(validExtPrimaryDataService) + Some(validExtPrimaryDataService), ) match { case Success( PrimaryServiceStateData( @@ -365,7 +365,7 @@ class PrimaryServiceProxySpec primaryConfig, mappingSource, extSubscribers, - extPrimaryDataService + extPrimaryDataService, ) ) => extPrimaryDataService should contain(validExtPrimaryDataService) @@ -598,7 +598,7 @@ class PrimaryServiceProxySpec primaryConfig, mappingSource, extSubscribers, - extPrimaryDataService + extPrimaryDataService, ) => modelToTimeSeries shouldBe proxyStateData.modelToTimeSeries timeSeriesToSourceRef shouldBe Map( diff --git a/src/test/scala/edu/ie3/simona/service/primary/PrimaryServiceProxySqlIT.scala b/src/test/scala/edu/ie3/simona/service/primary/PrimaryServiceProxySqlIT.scala index 1d6d2ab4ec..15679bbd0a 100644 --- a/src/test/scala/edu/ie3/simona/service/primary/PrimaryServiceProxySqlIT.scala +++ b/src/test/scala/edu/ie3/simona/service/primary/PrimaryServiceProxySqlIT.scala @@ -100,7 +100,7 @@ class PrimaryServiceProxySqlIT sqlParams = Some(sqlParams), ), simulationStart, - None + None, ) TestActorRef( diff --git a/src/test/scala/edu/ie3/simona/sim/SimonaSimFailSpec.scala b/src/test/scala/edu/ie3/simona/sim/SimonaSimFailSpec.scala index 56d386167b..de3c96951f 100644 --- a/src/test/scala/edu/ie3/simona/sim/SimonaSimFailSpec.scala +++ b/src/test/scala/edu/ie3/simona/sim/SimonaSimFailSpec.scala @@ -107,13 +107,14 @@ object SimonaSimFailSpec { .ref override def systemParticipantsListener( - context: ActorContext + context: ActorContext, + extSimSetupData: ExtSimSetupData, ): Seq[ActorRef] = Seq.empty[ActorRef] override def primaryServiceProxy( context: ActorContext, scheduler: ActorRef, - extSimSetupData: ExtSimSetupData + extSimSetupData: ExtSimSetupData, ): ActorRef = TestProbe("primaryService")(actorSystem).ref 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 8fcc71f7ec..a77484749e 100644 --- a/src/test/scala/edu/ie3/simona/sim/setup/SimonaSetupSpec.scala +++ b/src/test/scala/edu/ie3/simona/sim/setup/SimonaSetupSpec.scala @@ -33,13 +33,14 @@ class SimonaSetupSpec extends UnitSpec with SimonaSetup with SubGridGateMokka { throw new NotImplementedException("This is a dummy setup") override def systemParticipantsListener( - context: ActorContext + context: ActorContext, + extSimSetupData: ExtSimSetupData, ): Seq[ActorRef] = throw new NotImplementedException("This is a dummy setup") override def primaryServiceProxy( context: ActorContext, scheduler: ActorRef, - extSimSetupData: ExtSimSetupData + extSimSetupData: ExtSimSetupData, ): ActorRef = throw new NotImplementedException("This is a dummy setup") From fff290a2c7df376f42cd0e85c6afc12145b2784f Mon Sep 17 00:00:00 2001 From: jo-bao Date: Wed, 21 Feb 2024 16:37:22 +0100 Subject: [PATCH 07/41] spotless --- .../ie3/simona/event/listener/ResultEventListener.scala | 4 ---- src/main/scala/edu/ie3/simona/service/ExtDataSupport.scala | 2 +- .../scala/edu/ie3/simona/service/ev/ExtEvDataService.scala | 1 + .../ie3/simona/service/results/ExtResultDataService.scala | 7 ++++--- .../simona/event/listener/ResultEventListenerSpec.scala | 1 - 5 files changed, 6 insertions(+), 9 deletions(-) diff --git a/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala b/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala index 9469af8bb9..dcfd3d7a89 100644 --- a/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala +++ b/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala @@ -13,7 +13,6 @@ import edu.ie3.datamodel.models.result.{NodeResult, ResultEntity} import edu.ie3.simona.agent.grid.GridResultsSupport.PartialTransformer3wResult import edu.ie3.simona.event.Event import edu.ie3.simona.event.ResultEvent.{ - ExternalResultEvent, FlexOptionsResultEvent, ParticipantResultEvent, PowerFlowResultEvent, @@ -26,7 +25,6 @@ import edu.ie3.simona.exceptions.{ import edu.ie3.simona.io.result._ import edu.ie3.simona.ontology.messages.StopMessage import edu.ie3.simona.ontology.messages.services.ResultMessage.ResultResponseMessage -import edu.ie3.simona.sim.SimonaSim.ResultMessage import edu.ie3.simona.util.ResultFileHierarchy import org.apache.pekko.actor.ActorRef import org.slf4j.Logger @@ -169,7 +167,6 @@ object ResultEventListener extends Transformer3wResultSupport { handOverToExternalService( resultEntity, baseData.extResultDataService, - log, ) } baseData @@ -249,7 +246,6 @@ object ResultEventListener extends Transformer3wResultSupport { private def handOverToExternalService( resultEntity: ResultEntity, extResultDataService: Option[ActorRef], - log: Logger, ): Unit = Try { val extResultDataServiceRef = extResultDataService.getOrElse( throw new Exception("No external data service registered!") diff --git a/src/main/scala/edu/ie3/simona/service/ExtDataSupport.scala b/src/main/scala/edu/ie3/simona/service/ExtDataSupport.scala index 5449f2ebd9..03e27b2e41 100644 --- a/src/main/scala/edu/ie3/simona/service/ExtDataSupport.scala +++ b/src/main/scala/edu/ie3/simona/service/ExtDataSupport.scala @@ -50,6 +50,6 @@ trait ExtDataSupport[ * the updated state data */ protected def handleDataResponseMessage( - extResponseMsg: DataMessage // extResponseMsg + extResponseMsg: DataMessage )(implicit serviceStateData: S): S } diff --git a/src/main/scala/edu/ie3/simona/service/ev/ExtEvDataService.scala b/src/main/scala/edu/ie3/simona/service/ev/ExtEvDataService.scala index c10bfa72f4..2e3a8247c9 100644 --- a/src/main/scala/edu/ie3/simona/service/ev/ExtEvDataService.scala +++ b/src/main/scala/edu/ie3/simona/service/ev/ExtEvDataService.scala @@ -28,6 +28,7 @@ import edu.ie3.simona.service.ev.ExtEvDataService.{ InitExtEvData, } import edu.ie3.simona.service.{ExtDataSupport, ServiceStateData, SimonaService} +import edu.ie3.simona.ontology.messages.services.DataMessage import edu.ie3.simona.util.ReceiveDataMap import java.util.UUID diff --git a/src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala b/src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala index 790cc1bb30..116d0eb136 100644 --- a/src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala +++ b/src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala @@ -96,7 +96,7 @@ class ExtResultDataService(override val scheduler: ActorRef) case _: RequestResultEntities => requestResults(tick) } - (null, None) // No Annoucement + (null, None) } /** Handle a message from outside the simulation @@ -112,8 +112,9 @@ class ExtResultDataService(override val scheduler: ActorRef) extMsg: DataMessageFromExt )(implicit serviceStateData: ExtResultsStateData - ): ExtResultsStateData = { - serviceStateData + ): ExtResultsStateData = extMsg match { + case extResultsMessageFromExt: ResultDataMessageFromExt => + serviceStateData.copy(extResultsMessage = Some(extResultsMessageFromExt)) } /** Handle a message from inside SIMONA sent to external diff --git a/src/test/scala/edu/ie3/simona/event/listener/ResultEventListenerSpec.scala b/src/test/scala/edu/ie3/simona/event/listener/ResultEventListenerSpec.scala index 067162f30b..bc158c2e1c 100644 --- a/src/test/scala/edu/ie3/simona/event/listener/ResultEventListenerSpec.scala +++ b/src/test/scala/edu/ie3/simona/event/listener/ResultEventListenerSpec.scala @@ -32,7 +32,6 @@ import org.apache.pekko.actor.testkit.typed.scaladsl.{ ScalaTestWithActorTestKit, } import org.apache.pekko.testkit.TestKit.awaitCond -import org.apache.pekko.testkit.TestProbe import java.io.{File, FileInputStream} import java.util.UUID From bdb9b0b3dba9e208f24feb05527c995587731c42 Mon Sep 17 00:00:00 2001 From: jo-bao Date: Fri, 23 Feb 2024 10:32:56 +0100 Subject: [PATCH 08/41] fixed registration failed message --- .../ie3/simona/service/primary/PrimaryServiceProxy.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala b/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala index b964cf228e..35324c28c4 100644 --- a/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala +++ b/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala @@ -144,7 +144,6 @@ case class PrimaryServiceProxy( simulationStart: ZonedDateTime, extSimulation: Option[ActorRef], ): Try[PrimaryServiceStateData] = { - createSources(primaryConfig).map { case (mappingSource, metaInformationSource) => val modelToTimeSeries = mappingSource.getMapping.asScala.toMap @@ -275,21 +274,22 @@ case class PrimaryServiceProxy( ) case None => if (stateData.extSubscribers.nonEmpty) { + log.debug(s"Try to find external primary data for the model with uuid '{}'.", modelUuid) if (stateData.extSubscribers.exists(_ == modelUuid)) { handleExternalModel(modelUuid, stateData, sender()) } else { log.debug( - s"There is no time series apparent for the model with uuid '{}'.", + s"There is no external data apparent for the model with uuid '{}'.", modelUuid, ) - sender() ! RegistrationFailedMessage + sender() ! RegistrationFailedMessage(self) } } else { log.debug( s"There is no time series apparent for the model with uuid '{}'.", modelUuid, ) - sender() ! RegistrationFailedMessage + sender() ! RegistrationFailedMessage(self) } } case x => From 94aba27c232333e0520456e5d47f1fcb624de6ed Mon Sep 17 00:00:00 2001 From: jo-bao Date: Fri, 23 Feb 2024 10:33:47 +0100 Subject: [PATCH 09/41] spotless --- .../edu/ie3/simona/service/primary/PrimaryServiceProxy.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala b/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala index 35324c28c4..a886ed1ffc 100644 --- a/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala +++ b/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala @@ -274,7 +274,10 @@ case class PrimaryServiceProxy( ) case None => if (stateData.extSubscribers.nonEmpty) { - log.debug(s"Try to find external primary data for the model with uuid '{}'.", modelUuid) + log.debug( + s"Try to find external primary data for the model with uuid '{}'.", + modelUuid, + ) if (stateData.extSubscribers.exists(_ == modelUuid)) { handleExternalModel(modelUuid, stateData, sender()) } else { From f04c4e6fc8462f0c5e40ef4fffd391fa44916e11 Mon Sep 17 00:00:00 2001 From: jo-bao Date: Fri, 23 Feb 2024 15:13:58 +0100 Subject: [PATCH 10/41] code quality --- .../messages/services/DataMessage.scala | 2 +- .../messages/services/ResultMessage.scala | 2 -- .../messages/services/ServiceMessage.scala | 8 -------- .../ExtTimeSeriesSubscribersSource.scala | 2 +- .../service/primary/PrimaryServiceProxy.scala | 5 ++++- .../service/results/ExtResultDataService.scala | 1 + .../primary/ExtPrimaryDataServiceSpec.scala | 4 ---- .../primary/PrimaryServiceProxySpec.scala | 18 +++++++++--------- 8 files changed, 16 insertions(+), 26 deletions(-) diff --git a/src/main/scala/edu/ie3/simona/ontology/messages/services/DataMessage.scala b/src/main/scala/edu/ie3/simona/ontology/messages/services/DataMessage.scala index ff93925fe9..f83bf2cb09 100644 --- a/src/main/scala/edu/ie3/simona/ontology/messages/services/DataMessage.scala +++ b/src/main/scala/edu/ie3/simona/ontology/messages/services/DataMessage.scala @@ -6,4 +6,4 @@ package edu.ie3.simona.ontology.messages.services -trait DataMessage {} +trait DataMessage diff --git a/src/main/scala/edu/ie3/simona/ontology/messages/services/ResultMessage.scala b/src/main/scala/edu/ie3/simona/ontology/messages/services/ResultMessage.scala index b3c79a0d44..9443723e77 100644 --- a/src/main/scala/edu/ie3/simona/ontology/messages/services/ResultMessage.scala +++ b/src/main/scala/edu/ie3/simona/ontology/messages/services/ResultMessage.scala @@ -11,8 +11,6 @@ import edu.ie3.datamodel.models.result.ResultEntity sealed trait ResultMessage extends DataMessage object ResultMessage { - final case class ResultRequest(tick: Long) - final case class ResultResponseMessage(result: ResultEntity) extends ResultMessage } diff --git a/src/main/scala/edu/ie3/simona/ontology/messages/services/ServiceMessage.scala b/src/main/scala/edu/ie3/simona/ontology/messages/services/ServiceMessage.scala index 769a29daae..4d5cbe842b 100644 --- a/src/main/scala/edu/ie3/simona/ontology/messages/services/ServiceMessage.scala +++ b/src/main/scala/edu/ie3/simona/ontology/messages/services/ServiceMessage.scala @@ -84,12 +84,4 @@ case object ServiceMessage { val nextDataTick: Option[Long] val unlockKey: Option[ScheduleKey] } - - // -=- Subscriber of extPrimaryData -=- - trait ExtSubscriberMessage extends ServiceMessage - - final case class RequestExtSubscriberMessage() extends ExtSubscriberMessage - - final case class ProvideExtSubscriberMessage() extends ExtSubscriberMessage - } diff --git a/src/main/scala/edu/ie3/simona/service/primary/ExtTimeSeriesSubscribersSource.scala b/src/main/scala/edu/ie3/simona/service/primary/ExtTimeSeriesSubscribersSource.scala index a4689aa6a5..37cceabe41 100644 --- a/src/main/scala/edu/ie3/simona/service/primary/ExtTimeSeriesSubscribersSource.scala +++ b/src/main/scala/edu/ie3/simona/service/primary/ExtTimeSeriesSubscribersSource.scala @@ -9,7 +9,7 @@ package edu.ie3.simona.service.primary import java.util.UUID object ExtTimeSeriesSubscribersSource { - private var subscribers: Iterable[UUID] = Iterable.empty[UUID] + private val subscribers: Iterable[UUID] = Iterable.empty[UUID] def getSubscribers: Iterable[UUID] = subscribers } diff --git a/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala b/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala index a886ed1ffc..66c2b2484f 100644 --- a/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala +++ b/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala @@ -365,7 +365,10 @@ case class PrimaryServiceProxy( ): Unit = { stateData.extPrimaryDataService match { case Some(reqActor) => - reqActor ! ExtPrimaryDataServiceRegistrationMessage(modelUuid, reqActor) + reqActor ! ExtPrimaryDataServiceRegistrationMessage( + modelUuid, + requestingActor, + ) } } diff --git a/src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala b/src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala index 116d0eb136..60c704c0bb 100644 --- a/src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala +++ b/src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala @@ -165,6 +165,7 @@ class ExtResultDataService(override val scheduler: ActorRef) )(implicit serviceStateData: ExtResultsStateData ): (ExtResultsStateData, Option[Long]) = { + /* has to be filled -> For ReCoDE it isn't necessary that an external simulation requests results */ (serviceStateData.copy(), None) } } diff --git a/src/test/scala/edu/ie3/simona/service/primary/ExtPrimaryDataServiceSpec.scala b/src/test/scala/edu/ie3/simona/service/primary/ExtPrimaryDataServiceSpec.scala index 1de4096bbb..0b19da082c 100644 --- a/src/test/scala/edu/ie3/simona/service/primary/ExtPrimaryDataServiceSpec.scala +++ b/src/test/scala/edu/ie3/simona/service/primary/ExtPrimaryDataServiceSpec.scala @@ -90,9 +90,6 @@ class ExtPrimaryDataServiceSpec primaryDataFactory, ) - private val participant1UUID = - UUID.fromString("06a14909-366e-4e94-a593-1016e1455b30") - "An uninitialized external primary data service" must { "send correct completion message after initialisation" in { @@ -129,7 +126,6 @@ class ExtPrimaryDataServiceSpec scheduler.ref ) ) - val validInitData = InitExtPrimaryData(extPrimaryData(serviceRef)) "refuse registration for wrong registration request" in { serviceRef ! RegisterForWeatherMessage(51.4843281, 7.4116482) diff --git a/src/test/scala/edu/ie3/simona/service/primary/PrimaryServiceProxySpec.scala b/src/test/scala/edu/ie3/simona/service/primary/PrimaryServiceProxySpec.scala index ee5ac4ac67..b3fd066f56 100644 --- a/src/test/scala/edu/ie3/simona/service/primary/PrimaryServiceProxySpec.scala +++ b/src/test/scala/edu/ie3/simona/service/primary/PrimaryServiceProxySpec.scala @@ -311,8 +311,8 @@ class PrimaryServiceProxySpec simulationStart, primaryConfig, mappingSource, - extSubscribers, - extPrimaryDataService, + _, + _, ) ) => modelToTimeSeries shouldBe Map( @@ -359,11 +359,11 @@ class PrimaryServiceProxySpec ) match { case Success( PrimaryServiceStateData( - modelToTimeSeries, - timeSeriesToSourceRef, - simulationStart, - primaryConfig, - mappingSource, + _, + _, + _, + _, + _, extSubscribers, extPrimaryDataService, ) @@ -597,8 +597,8 @@ class PrimaryServiceProxySpec simulationStart, primaryConfig, mappingSource, - extSubscribers, - extPrimaryDataService, + _, + _, ) => modelToTimeSeries shouldBe proxyStateData.modelToTimeSeries timeSeriesToSourceRef shouldBe Map( From d2b8b67e94839b67e94a7cc514ef560ef731faad Mon Sep 17 00:00:00 2001 From: jo-bao Date: Mon, 26 Feb 2024 08:44:54 +0100 Subject: [PATCH 11/41] pekko typed --- .../event/listener/ResultEventListener.scala | 1 - .../scala/edu/ie3/simona/sim/SimonaSim.scala | 15 +++++----- .../ie3/simona/sim/setup/SimonaSetup.scala | 6 ++-- .../sim/setup/SimonaStandaloneSetup.scala | 29 ++++++++++++------- .../edu/ie3/simona/sim/SimonaSimSpec.scala | 7 +++-- .../simona/sim/setup/SimonaSetupSpec.scala | 8 ++--- 6 files changed, 37 insertions(+), 29 deletions(-) diff --git a/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala b/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala index 0052071f60..551f5aaeee 100644 --- a/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala +++ b/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala @@ -22,7 +22,6 @@ import edu.ie3.simona.exceptions.{ ProcessResultEventException, } import edu.ie3.simona.io.result._ -import edu.ie3.simona.ontology.messages.StopMessage import edu.ie3.simona.ontology.messages.services.ResultMessage.ResultResponseMessage import edu.ie3.simona.util.ResultFileHierarchy import org.apache.pekko.actor.ActorRef diff --git a/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala b/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala index 47bc4bf860..a026cd9a9e 100644 --- a/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala +++ b/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala @@ -69,9 +69,6 @@ object SimonaSim { ): Behavior[Request] = Behaviors .receivePartial[Request] { case (ctx, Start(_)) => - val resultEventListeners = - simonaSetup.resultEventListener(ctx) - val runtimeEventListener = simonaSetup.runtimeEventListener(ctx) val timeAdvancer = @@ -79,17 +76,18 @@ object SimonaSim { val scheduler = simonaSetup.scheduler(ctx, timeAdvancer) /* start services */ + // Load external simulations + val extSimulationData: ExtSimSetupData = + simonaSetup.extSimulations(ctx, scheduler) + // primary service proxy val primaryServiceProxy = - simonaSetup.primaryServiceProxy(ctx, scheduler) + simonaSetup.primaryServiceProxy(ctx, scheduler, extSimulationData) // weather service val weatherService = simonaSetup.weatherService(ctx, scheduler) - val extSimulationData: ExtSimSetupData = - simonaSetup.extSimulations(ctx, scheduler) - val environmentRefs = EnvironmentRefs( scheduler, runtimeEventListener.toClassic, @@ -98,6 +96,9 @@ object SimonaSim { extSimulationData.evDataService, ) + val resultEventListeners = + simonaSetup.resultEventListener(ctx, extSimulationData) + /* start grid agents */ val gridAgents = simonaSetup.gridAgents( ctx, 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 b556b8b2a3..6626ffbffb 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaSetup.scala @@ -70,9 +70,9 @@ trait SimonaSetup { * An actor reference to the service */ def primaryServiceProxy( - context: ActorContext[_], - scheduler: ActorRef[SchedulerMessage], - extSimulationData: ExtSimSetupData, + context: ActorContext[_], + scheduler: ActorRef[SchedulerMessage], + extSimulationData: ExtSimSetupData, ): ClassicRef /** Creates a weather service 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 9e95d358cf..74ccd33975 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala @@ -17,7 +17,10 @@ import edu.ie3.simona.agent.grid.GridAgent import edu.ie3.simona.api.ExtSimAdapter import edu.ie3.simona.api.data.ExtData import edu.ie3.simona.api.data.ev.{ExtEvData, ExtEvSimulation} -import edu.ie3.simona.api.data.primarydata.{ExtPrimaryData, ExtPrimaryDataSimulation} +import edu.ie3.simona.api.data.primarydata.{ + ExtPrimaryData, + ExtPrimaryDataSimulation, +} import edu.ie3.simona.api.data.results.{ExtResultDataSimulation, ExtResultsData} import edu.ie3.simona.api.simulation.ExtSimAdapterData import edu.ie3.simona.config.{ArgsParser, RefSystemParser, SimonaConfig} @@ -32,7 +35,10 @@ import edu.ie3.simona.service.SimonaService import edu.ie3.simona.service.ev.ExtEvDataService import edu.ie3.simona.service.ev.ExtEvDataService.InitExtEvData import edu.ie3.simona.service.primary.ExtPrimaryDataService.InitExtPrimaryData -import edu.ie3.simona.service.primary.{ExtPrimaryDataService, PrimaryServiceProxy} +import edu.ie3.simona.service.primary.{ + ExtPrimaryDataService, + PrimaryServiceProxy, +} import edu.ie3.simona.service.primary.PrimaryServiceProxy.InitPrimaryServiceProxyStateData import edu.ie3.simona.service.results.ExtResultDataService import edu.ie3.simona.service.results.ExtResultDataService.InitExtResultsData @@ -46,7 +52,10 @@ import edu.ie3.util.TimeUtil import org.apache.pekko.actor.typed.ActorRef import org.apache.pekko.actor.typed.scaladsl.ActorContext import org.apache.pekko.actor.typed.scaladsl.adapter._ -import org.apache.pekko.actor.{ActorRef, ActorContext => ClassicContext, ActorRef => ClassicRef} +import org.apache.pekko.actor.{ + ActorContext => ClassicContext, + ActorRef => ClassicRef, +} import java.util.concurrent.LinkedBlockingQueue import scala.jdk.CollectionConverters._ @@ -235,8 +244,8 @@ class SimonaStandaloneSetup( (extEvData, (classOf[ExtEvDataService], extEvDataService)) case (extPrimaryDataSimulation: ExtPrimaryDataSimulation, dIndex) => - val extPrimaryDataService = context.simonaActorOf( - ExtPrimaryDataService.props(scheduler), + val extPrimaryDataService = context.toClassic.simonaActorOf( + ExtPrimaryDataService.props(scheduler.toClassic), s"$index-$dIndex", ) val extPrimaryData = new ExtPrimaryData( @@ -249,7 +258,7 @@ class SimonaStandaloneSetup( InitExtPrimaryData(extPrimaryData), ScheduleLock.singleKey( context, - scheduler.toTyped, + scheduler, INIT_SIM_TICK, ), ) @@ -257,8 +266,8 @@ class SimonaStandaloneSetup( (null, (classOf[ExtResultDataService], extPrimaryDataService)) case (_: ExtResultDataSimulation, dIndex) => - val extResultDataService = context.simonaActorOf( - ExtResultDataService.props(scheduler), + val extResultDataService = context.toClassic.simonaActorOf( + ExtResultDataService.props(scheduler.toClassic), s"$index-$dIndex", ) val extResultsData = @@ -268,7 +277,7 @@ class SimonaStandaloneSetup( InitExtResultsData(extResultsData), ScheduleLock.singleKey( context, - scheduler.toTyped, + scheduler, INIT_SIM_TICK, ), ) @@ -344,7 +353,7 @@ class SimonaStandaloneSetup( context: ActorContext[_], extSimulationData: ExtSimSetupData, ): Seq[ActorRef[ResultEventListener.Request]] = { - val extResultDataService: Option[ActorRef] = + val extResultDataService: Option[ClassicRef] = extSimulationData.extResultDataService // append ResultEventListener as well to write raw output files ArgsParser diff --git a/src/test/scala/edu/ie3/simona/sim/SimonaSimSpec.scala b/src/test/scala/edu/ie3/simona/sim/SimonaSimSpec.scala index fc2308c9f0..e6dd11b28b 100644 --- a/src/test/scala/edu/ie3/simona/sim/SimonaSimSpec.scala +++ b/src/test/scala/edu/ie3/simona/sim/SimonaSimSpec.scala @@ -298,7 +298,8 @@ class SimonaSimSpec extends ScalaTestWithActorTestKit with UnitSpec { new MockSetup() { override def resultEventListener( - context: ActorContext[_] + context: ActorContext[_], + extSimSetupData: ExtSimSetupData, ): Seq[ActorRef[ResultEventListener.Request]] = throwTestException() } @@ -402,7 +403,8 @@ object SimonaSimSpec { ) override def resultEventListener( - context: ActorContext[_] + context: ActorContext[_], + extSimSetupData: ExtSimSetupData, ): Seq[ActorRef[ResultEventListener.Request]] = Seq( context.spawn( stoppableForwardMessage(resultEventProbe), @@ -413,6 +415,7 @@ object SimonaSimSpec { override def primaryServiceProxy( context: ActorContext[_], scheduler: ActorRef[SchedulerMessage], + extSimSetupData: ExtSimSetupData, ): ClassicRef = context.spawn(empty, uniqueName("primaryService")).toClassic 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 42e70f093c..552ce6ac46 100644 --- a/src/test/scala/edu/ie3/simona/sim/setup/SimonaSetupSpec.scala +++ b/src/test/scala/edu/ie3/simona/sim/setup/SimonaSetupSpec.scala @@ -36,15 +36,11 @@ class SimonaSetupSpec extends UnitSpec with SimonaSetup with SubGridGateMokka { ) override def resultEventListener( - context: scaladsl.ActorContext[_] + context: scaladsl.ActorContext[_], + extSimulationData: ExtSimSetupData, ): Seq[typed.ActorRef[ResultEventListener.Request]] = throw new NotImplementedException("This is a dummy setup") - override def systemParticipantsListener( - context: ActorContext, - extSimSetupData: ExtSimSetupData, - ): Seq[ActorRef] = throw new NotImplementedException("This is a dummy setup") - override def primaryServiceProxy( context: scaladsl.ActorContext[_], scheduler: typed.ActorRef[SchedulerMessage], From 6cdbf836337b0ce344912a68eec4abe5bc03f9d3 Mon Sep 17 00:00:00 2001 From: jo-bao Date: Wed, 13 Mar 2024 11:57:27 +0100 Subject: [PATCH 12/41] working --- .../agent/participant/ParticipantAgent.scala | 7 + .../ParticipantAgentFundamentals.scala | 95 +++++++- .../edu/ie3/simona/event/ResultEvent.scala | 6 +- .../event/listener/ResultEventListener.scala | 13 +- .../messages/services/ResultMessage.scala | 9 +- .../edu/ie3/simona/scheduler/Scheduler.scala | 4 +- .../ie3/simona/service/ExtDataSupport.scala | 12 + .../ie3/simona/service/SimonaService.scala | 17 +- .../primary/ExtPrimaryDataService.scala | 12 +- .../ExtTimeSeriesSubscribersSource.scala | 5 +- .../results/ExtResultDataService.scala | 205 ++++++++++++++---- .../sim/setup/SimonaStandaloneSetup.scala | 184 +++++++--------- .../edu/ie3/simona/util/ReceiveDataMap.scala | 17 +- 13 files changed, 398 insertions(+), 188 deletions(-) diff --git a/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgent.scala b/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgent.scala index a01c5677a1..a370c0054a 100644 --- a/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgent.scala +++ b/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgent.scala @@ -44,6 +44,7 @@ import edu.ie3.simona.model.participant.{ SystemParticipant, } import edu.ie3.simona.ontology.messages.Activation +import edu.ie3.simona.ontology.messages.services.ResultMessage.ResultRequestMessage import edu.ie3.simona.ontology.messages.PowerMessage.RequestAssetPowerMessage import edu.ie3.simona.ontology.messages.SchedulerMessage.ScheduleActivation import edu.ie3.simona.ontology.messages.flex.FlexibilityMessage.{ @@ -801,6 +802,12 @@ abstract class ParticipantAgent[ setPower: squants.Power, ): (MS, PD, FlexChangeIndicator) + def answerResultRequestAndStayWithUpdatedStateData( + baseStateData: BaseStateData[PD], + requestTick: Long, + alternativeResult: PD, + ): FSM.State[AgentState, ParticipantStateData[PD]] + /** Determining the reply to an * [[edu.ie3.simona.ontology.messages.PowerMessage.RequestAssetPowerMessage]], * send this answer and stay in the current state. If no reply can be 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..f10e39f54c 100644 --- a/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgentFundamentals.scala +++ b/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgentFundamentals.scala @@ -141,8 +141,8 @@ protected trait ParticipantAgentFundamentals[ /* Confirm final initialization */ releaseTick() - senderToMaybeTick._2.foreach { tick => - scheduler ! Completion(self.toTyped, Some(tick)) + senderToMaybeTick match { case (_, maybeTick) => + scheduler ! Completion(self.toTyped, maybeTick) } goto(Idle) using stateData } @@ -538,8 +538,13 @@ protected trait ParticipantAgentFundamentals[ msg.unlockKey, ) } - case _ => - false + + case _: FromOutsideBaseStateData[_, _] => + scheduler ! ScheduleActivation( + self.toTyped, + msg.tick, + msg.unlockKey, + ) } } @@ -1191,6 +1196,77 @@ protected trait ParticipantAgentFundamentals[ } } + override def answerResultRequestAndStayWithUpdatedStateData( + baseStateData: BaseStateData[PD], + requestTick: Long, + alternativeResult: PD): FSM.State[AgentState, ParticipantStateData[PD]] = { + /* Check, if there is any calculation foreseen for this tick. If so, wait with the response. */ + val activationExpected = + baseStateData.additionalActivationTicks.headOption.exists(_ < requestTick) + val dataExpected = + baseStateData.foreseenDataTicks.values.flatten.exists(_ < requestTick) + if (activationExpected || dataExpected) { + log.debug( + s"Received power request from '{}' for tick '{}', but I'm still waiting for new results before " + + s"this tick. Waiting with the response.", + sender(), + requestTick, + ) + stash() + stay() using baseStateData + } else { + /* Update the voltage value store */ + val lastNodalVoltage = + baseStateData.voltageValueStore.last(requestTick) + + /* Determine the most recent request */ + val mostRecentRequest = + baseStateData.requestValueStore.last(requestTick) + + /* === Check if this request has already been answered with same tick and nodal voltage === */ + determineFastReplyForResults( + baseStateData, + mostRecentRequest, + requestTick, + ) + } + + + + + + + + + } + + + def determineFastReplyForResults( + baseStateData: BaseStateData[PD], + mostRecentRequest: Option[(Long, PD)], + requestTick: Long, + ): FSM.State[AgentState, ParticipantStateData[PD]] = { + mostRecentRequest match { + case Some((mostRecentRequestTick, latestProvidedValues)) + if mostRecentRequestTick == requestTick => + /* A request for this tick has already been answered. Check, if it has been the same request. + * if it has been the same request we wanna answer with the same values afterwards, this data MUST always + * be available when we already provided data for this tick */ + baseStateData match { + case externalBaseStateData: FromOutsideBaseStateData[M, PD] => + /* When data is provided from outside it is NOT altered depending on the node voltage. Send an + * AssetPowerUnchangedMessage */ + stay() using externalBaseStateData.copy() replying AssetPowerUnchangedMessage( + latestProvidedValues.p, + latestProvidedValues.q, + ) + + } + } + } + + + /** Determining the reply to an * [[edu.ie3.simona.ontology.messages.PowerMessage.RequestAssetPowerMessage]], * send this answer and stay in the current state. If no reply can be @@ -1788,15 +1864,18 @@ protected trait ParticipantAgentFundamentals[ baseStateData: BaseStateData[PD], tick: Long, result: AccompaniedSimulationResult[PD], - )(implicit outputConfig: NotifierConfig): Unit = + )(implicit outputConfig: NotifierConfig): Unit = { if (outputConfig.simulationResultInfo) { + var (nextTick, _) = popNextActivationTrigger(baseStateData) + notifyListener( - buildResultEvent(baseStateData, tick, result.primaryData) + buildResultEvent(baseStateData, tick, result.primaryData, nextTick) ) result.accompanyingResults .flatMap(result => buildResultEvent(result)) .foreach(notifyListener(_)) } + } /** Update the result value store, inform all registered listeners and go to * Idle using the updated base state data @@ -1919,11 +1998,13 @@ protected trait ParticipantAgentFundamentals[ baseStateData: BaseStateData[PD], tick: Long, result: PD, + nextTick: Option[Long] = None ): ParticipantResultEvent = { val uuid = baseStateData.modelUuid val dateTime = tick.toDateTime(baseStateData.startDate) ParticipantResultEvent( - buildResult(uuid, dateTime, result) + buildResult(uuid, dateTime, result), + nextTick ) } diff --git a/src/main/scala/edu/ie3/simona/event/ResultEvent.scala b/src/main/scala/edu/ie3/simona/event/ResultEvent.scala index fa6e07adce..5d3cc78694 100644 --- a/src/main/scala/edu/ie3/simona/event/ResultEvent.scala +++ b/src/main/scala/edu/ie3/simona/event/ResultEvent.scala @@ -27,7 +27,8 @@ sealed trait ResultEvent extends Event with ResultEventListener.Request object ResultEvent { final case class ExternalResultEvent( - resultEntity: ResultEntity + resultEntity: ResultEntity, + nextTick: Long ) extends ResultEvent /** Event that holds a calculation result of a @@ -37,7 +38,8 @@ object ResultEvent { * the calculation result */ final case class ParticipantResultEvent( - systemParticipantResult: SystemParticipantResult + systemParticipantResult: SystemParticipantResult, + nextTick: Option[Long] = None ) extends ResultEvent /** Event, that is triggered every time a thermal model has a new result diff --git a/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala b/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala index 551f5aaeee..57315f6bd9 100644 --- a/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala +++ b/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala @@ -157,12 +157,15 @@ object ResultEventListener extends Transformer3wResultSupport { resultEntity: ResultEntity, baseData: BaseData, log: Logger, + nextTick: Option[Long] = None ): BaseData = { handOverToSink(resultEntity, baseData.classToSink, log) if (baseData.extResultDataService.isDefined) { handOverToExternalService( resultEntity, baseData.extResultDataService, + log, + nextTick ) } baseData @@ -242,11 +245,15 @@ object ResultEventListener extends Transformer3wResultSupport { private def handOverToExternalService( resultEntity: ResultEntity, extResultDataService: Option[ActorRef], + log: Logger, + nextTick: Option[Long] = None ): Unit = Try { val extResultDataServiceRef = extResultDataService.getOrElse( throw new Exception("No external data service registered!") ) - extResultDataServiceRef ! ResultResponseMessage(resultEntity) + //log.info(s"Sending a ResultResponseMessage to $extResultDataServiceRef $resultEntity") + extResultDataServiceRef ! ResultResponseMessage(resultEntity, nextTick) + //log.info(s"Sended a ResultResponseMessage to $extResultDataServiceRef $resultEntity") } def apply( @@ -299,8 +306,8 @@ object ResultEventListener extends Transformer3wResultSupport { private def idle(baseData: BaseData): Behavior[Request] = Behaviors .receivePartial[Request] { - case (ctx, ParticipantResultEvent(participantResult)) => - val updatedBaseData = handleResult(participantResult, baseData, ctx.log) + case (ctx, ParticipantResultEvent(participantResult, nextTick)) => + val updatedBaseData = handleResult(participantResult, baseData, ctx.log, nextTick) idle(updatedBaseData) case (ctx, ThermalResultEvent(thermalResult)) => diff --git a/src/main/scala/edu/ie3/simona/ontology/messages/services/ResultMessage.scala b/src/main/scala/edu/ie3/simona/ontology/messages/services/ResultMessage.scala index 9443723e77..1fc1eaae0b 100644 --- a/src/main/scala/edu/ie3/simona/ontology/messages/services/ResultMessage.scala +++ b/src/main/scala/edu/ie3/simona/ontology/messages/services/ResultMessage.scala @@ -7,10 +7,17 @@ package edu.ie3.simona.ontology.messages.services import edu.ie3.datamodel.models.result.ResultEntity +import edu.ie3.simona.ontology.messages.PowerMessage sealed trait ResultMessage extends DataMessage object ResultMessage { - final case class ResultResponseMessage(result: ResultEntity) + final case class ResultResponseMessage( + result: ResultEntity, + nextTick: Option[Long] + ) extends ResultMessage + final case class ResultRequestMessage( + currentTick: Long + ) extends ResultMessage } diff --git a/src/main/scala/edu/ie3/simona/scheduler/Scheduler.scala b/src/main/scala/edu/ie3/simona/scheduler/Scheduler.scala index 7339fbbe61..6f8b9d9138 100644 --- a/src/main/scala/edu/ie3/simona/scheduler/Scheduler.scala +++ b/src/main/scala/edu/ie3/simona/scheduler/Scheduler.scala @@ -123,6 +123,8 @@ object Scheduler { _ ! Activation(newCore.activeTick) } + //println("ScheduleActivation from " + actor + ", nextTick = " + newTick + ", newCore = " + newCore) + active(data, newCore) case (_, Completion(actor, maybeNewTick)) => @@ -137,7 +139,7 @@ object Scheduler { toActivate.foreach { _ ! Activation(updatedCore.activeTick) } - + //println("Completion from " + actor + ", nextTick = " + maybeNewTick + ", newCore = " + updatedCore) updatedCore } .map { newCore => diff --git a/src/main/scala/edu/ie3/simona/service/ExtDataSupport.scala b/src/main/scala/edu/ie3/simona/service/ExtDataSupport.scala index 03e27b2e41..0e01de03bc 100644 --- a/src/main/scala/edu/ie3/simona/service/ExtDataSupport.scala +++ b/src/main/scala/edu/ie3/simona/service/ExtDataSupport.scala @@ -10,6 +10,8 @@ import edu.ie3.simona.api.data.ontology.DataMessageFromExt import edu.ie3.simona.ontology.messages.services.{DataMessage, EvMessage} import edu.ie3.simona.ontology.messages.services.EvMessage.EvResponseMessage import edu.ie3.simona.service.ServiceStateData.ServiceBaseStateData +import edu.ie3.simona.ontology.messages.services.ResultMessage +import edu.ie3.simona.api.data.results.ontology.ResultDataMessageFromExt trait ExtDataSupport[ S <: ServiceBaseStateData @@ -18,9 +20,19 @@ trait ExtDataSupport[ override def idleExternal(implicit stateData: S): Receive = { case extMsg: DataMessageFromExt => + //log.info("Got a DataMessageFromExt with content " + extMsg) val updatedStateData = handleDataMessage(extMsg)(stateData) context become idle(updatedStateData) + case extResponseMsgToExt: ResultMessage => + //log.info("Got a DataResponse with content " + extResponseMsgToExt) + val updatedStateData = handleDataResponseMessage(extResponseMsgToExt)(stateData) + context become idle(updatedStateData) + + case extResultDataMsg: ResultDataMessageFromExt => + val updatedStateData = handleDataMessage(extResultDataMsg)(stateData) + context become idle(updatedStateData) + case extResponseMsg: EvResponseMessage => val updatedStateData = handleDataResponseMessage(extResponseMsg)(stateData) diff --git a/src/main/scala/edu/ie3/simona/service/SimonaService.scala b/src/main/scala/edu/ie3/simona/service/SimonaService.scala index 6cd6042141..aa219da531 100644 --- a/src/main/scala/edu/ie3/simona/service/SimonaService.scala +++ b/src/main/scala/edu/ie3/simona/service/SimonaService.scala @@ -12,17 +12,12 @@ import org.apache.pekko.actor.typed.scaladsl.adapter.ClassicActorRefOps import org.apache.pekko.actor.{Actor, ActorContext, ActorRef, Stash} import edu.ie3.simona.logging.SimonaActorLogging import edu.ie3.simona.ontology.messages.Activation -import edu.ie3.simona.ontology.messages.SchedulerMessage.{ - Completion, - ScheduleActivation, -} +import edu.ie3.simona.ontology.messages.SchedulerMessage.{Completion, ScheduleActivation} +import edu.ie3.simona.ontology.messages.services.ResultMessage.ResultResponseMessage import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.ScheduleServiceActivation import edu.ie3.simona.ontology.messages.services.ServiceMessage.ServiceRegistrationMessage import edu.ie3.simona.scheduler.ScheduleLock.ScheduleKey -import edu.ie3.simona.service.ServiceStateData.{ - InitializeServiceStateData, - ServiceBaseStateData, -} +import edu.ie3.simona.service.ServiceStateData.{InitializeServiceStateData, ServiceBaseStateData} import edu.ie3.simona.service.SimonaService.Create import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK @@ -151,6 +146,7 @@ abstract class SimonaService[ } case ScheduleServiceActivation(tick, unlockKey) => + //log.info("Send a ScheduleServiceActivation") scheduler ! ScheduleActivation( self.toTyped, tick, @@ -159,9 +155,12 @@ abstract class SimonaService[ // activity start trigger for this service case Activation(tick) => + //log.info("Got Activation") /* The scheduler sends out an activity start trigger. Announce new data to all registered recipients. */ - val (updatedStateData, maybeNewTriggers) = + val (updatedStateData, maybeNewTriggers) = { announceInformation(tick)(stateData, context) + } + //log.info("Send Completion") scheduler ! Completion(self.toTyped, maybeNewTriggers) context become idle(updatedStateData) diff --git a/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala b/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala index 0fce0932b8..057aa42fa3 100644 --- a/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala +++ b/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala @@ -90,14 +90,11 @@ final case class ExtPrimaryDataService( registrationMessage: ServiceMessage.ServiceRegistrationMessage )(implicit serviceStateData: ExtPrimaryDataStateData - ): Try[ExtPrimaryDataStateData] = { - println("Habe erhalten 0") - registrationMessage match { + ): Try[ExtPrimaryDataStateData] = registrationMessage match { case ExtPrimaryDataServiceRegistrationMessage( modelUuid, requestingActor, ) => - println("Habe erhalten") Success(handleRegistrationRequest(requestingActor, modelUuid)) case invalidMessage => Failure( @@ -106,7 +103,6 @@ final case class ExtPrimaryDataService( ) ) } - } private def handleRegistrationRequest( agentToBeRegistered: ActorRef, @@ -186,7 +182,7 @@ final case class ExtPrimaryDataService( } } - // Verteile Primary Data + // Distribute Primary Data if (actorToPrimaryData.nonEmpty) { val keys = ScheduleLock.multiKey( @@ -217,12 +213,10 @@ final case class ExtPrimaryDataService( } } } - - ( // Message leeren + ( serviceStateData.copy(extPrimaryDataMessage = None), None, ) - } override protected def handleDataMessage( diff --git a/src/main/scala/edu/ie3/simona/service/primary/ExtTimeSeriesSubscribersSource.scala b/src/main/scala/edu/ie3/simona/service/primary/ExtTimeSeriesSubscribersSource.scala index 37cceabe41..86bfece987 100644 --- a/src/main/scala/edu/ie3/simona/service/primary/ExtTimeSeriesSubscribersSource.scala +++ b/src/main/scala/edu/ie3/simona/service/primary/ExtTimeSeriesSubscribersSource.scala @@ -9,7 +9,10 @@ package edu.ie3.simona.service.primary import java.util.UUID object ExtTimeSeriesSubscribersSource { - private val subscribers: Iterable[UUID] = Iterable.empty[UUID] + private val subscribers: Iterable[UUID] = Iterable( + UUID.fromString("fd1a8de9-722a-4304-8799-e1e976d9979c"), + UUID.fromString("ff0b995a-86ff-4f4d-987e-e475a64f2180") + ) def getSubscribers: Iterable[UUID] = subscribers } diff --git a/src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala b/src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala index 60c704c0bb..2be13adcd4 100644 --- a/src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala +++ b/src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala @@ -6,30 +6,26 @@ package edu.ie3.simona.service.results +import org.apache.pekko.actor.typed.scaladsl.adapter.ClassicActorRefOps import edu.ie3.datamodel.models.result.ResultEntity import edu.ie3.simona.api.data.ontology.DataMessageFromExt -import edu.ie3.simona.api.data.results.ExtResultsData -import edu.ie3.simona.api.data.results.ontology.{ - ProvideResultEntities, - RequestResultEntities, - ResultDataMessageFromExt, -} +import edu.ie3.simona.api.data.results.ExtResultData +import edu.ie3.simona.api.data.results.ontology.{ProvideResultEntities, RequestResultEntities, ResultDataMessageFromExt} import edu.ie3.simona.exceptions.{InitializationException, ServiceException} -import edu.ie3.simona.ontology.messages.services.ResultMessage.ResultResponseMessage -import edu.ie3.simona.ontology.messages.services.ServiceMessage.ServiceRegistrationMessage +import edu.ie3.simona.ontology.messages.SchedulerMessage.ScheduleActivation import edu.ie3.simona.ontology.messages.services.DataMessage -import edu.ie3.simona.service.ServiceStateData.{ - InitializeServiceStateData, - ServiceBaseStateData, -} +import edu.ie3.simona.ontology.messages.services.ResultMessage.{ResultResponseMessage, ResultRequestMessage} +import edu.ie3.simona.ontology.messages.services.ServiceMessage.ServiceRegistrationMessage +import edu.ie3.simona.scheduler.ScheduleLock +import edu.ie3.simona.scheduler.ScheduleLock.ScheduleKey +import edu.ie3.simona.service.ServiceStateData.{InitializeServiceStateData, ServiceBaseStateData} +import edu.ie3.simona.service.results.ExtResultDataService.{ExtResultsStateData, InitExtResultData} import edu.ie3.simona.service.{ExtDataSupport, SimonaService} -import edu.ie3.simona.service.results.ExtResultDataService.{ - ExtResultsStateData, - InitExtResultsData, -} - +import edu.ie3.simona.util.ReceiveDataMap +import edu.ie3.util.scala.collection.immutable.SortedDistinctSeq import org.apache.pekko.actor.{ActorContext, ActorRef, Props} +import scala.collection.immutable.SortedSet import java.util.UUID import scala.jdk.CollectionConverters.SeqHasAsJava import scala.util.{Failure, Success, Try} @@ -41,14 +37,19 @@ object ExtResultDataService { ) final case class ExtResultsStateData( - extResultsData: ExtResultsData, - uuidToActorRef: Map[UUID, ActorRef] = Map.empty[UUID, ActorRef], - extResultsMessage: Option[ResultDataMessageFromExt] = None, - resultStorage: Map[UUID, ResultEntity] = Map.empty, + extResultsData: ExtResultData, + subscribers: List[UUID] = List.empty, + extResultsMessage: Option[ResultDataMessageFromExt] = None, + resultStorage: Map[UUID, (Option[ResultEntity], Option[Long])] = Map.empty, // UUID -> Result, nextTick + maybeNextActivationTick: Option[Long] = None, + recentResults: ReceiveDataMap[UUID, ResultEntity] = ReceiveDataMap.empty, + receivedResults: Int = 0, + resultSink: List[ResultEntity] = List.empty, + unlockKey: Option[ScheduleKey] = None, ) extends ServiceBaseStateData - final case class InitExtResultsData( - extResultsData: ExtResultsData + final case class InitExtResultData( + extResultsData: ExtResultData ) extends InitializeServiceStateData } @@ -60,8 +61,19 @@ class ExtResultDataService(override val scheduler: ActorRef) initServiceData: InitializeServiceStateData ): Try[(ExtResultsStateData, Option[Long])] = { initServiceData match { - case InitExtResultsData(extResultsData) => - val resultInitializedStateData = ExtResultsStateData(extResultsData) + case InitExtResultData(extResultsData) => + val initSubscribers = List( + UUID.fromString("de8cfef5-7620-4b9e-9a10-1faebb5a80c0"), + UUID.fromString("2560c371-f420-4c2a-b4e6-e04c11b64c03")) + val resultInitializedStateData = ExtResultsStateData( + extResultsData = extResultsData, + subscribers = initSubscribers, + resultStorage = Map( + UUID.fromString("de8cfef5-7620-4b9e-9a10-1faebb5a80c0") -> (None, Some(0)), + UUID.fromString("2560c371-f420-4c2a-b4e6-e04c11b64c03") -> (None, Some(0)), + ), + recentResults = ReceiveDataMap(initSubscribers.toSet) + ) Success(resultInitializedStateData, None) case invalidData => @@ -96,7 +108,6 @@ class ExtResultDataService(override val scheduler: ActorRef) case _: RequestResultEntities => requestResults(tick) } - (null, None) } /** Handle a message from outside the simulation @@ -114,7 +125,10 @@ class ExtResultDataService(override val scheduler: ActorRef) serviceStateData: ExtResultsStateData ): ExtResultsStateData = extMsg match { case extResultsMessageFromExt: ResultDataMessageFromExt => - serviceStateData.copy(extResultsMessage = Some(extResultsMessageFromExt)) + //log.info("Received ResultDataMessageFromExt with content: " + extResultsMessageFromExt) + serviceStateData.copy( + extResultsMessage = Some(extResultsMessageFromExt) + ) } /** Handle a message from inside SIMONA sent to external @@ -130,34 +144,95 @@ class ExtResultDataService(override val scheduler: ActorRef) extResponseMsg: DataMessage )(implicit serviceStateData: ExtResultsStateData): ExtResultsStateData = { extResponseMsg match { - case ResultResponseMessage(result) => - if (serviceStateData.uuidToActorRef.contains(result.getUuid)) { + case ResultResponseMessage(result, nextTick) => + if (serviceStateData.subscribers.contains(result.getInputModel)) { + log.info("[handleDataResponseMessage] Received ResultsResponseMessage with content " + extResponseMsg) + log.info("[handleDataResponseMessage] RecentResults " + serviceStateData.recentResults) + var updatedReceivedResults = serviceStateData.recentResults.addData(result.getInputModel, result) + log.info("[handleDataResponseMessage] AddData to RecentResults -> updatedReceivedResults = " + updatedReceivedResults) + var updatedResultStorage = + serviceStateData.resultStorage + (result.getInputModel -> (Some(result), nextTick)) + if (updatedReceivedResults.nonComplete) { + // all responses received, forward them to external simulation in a bundle + serviceStateData.copy( + recentResults = updatedReceivedResults, + resultStorage = updatedResultStorage + ) + } else { + log.info("Got all ResultResponseMessage -> Now forward to external simulation in a bundle ") + var resultList = List.empty[ResultEntity] + + updatedResultStorage.values.foreach( + result => resultList = resultList :+ result._1.getOrElse( + throw new RuntimeException("There is no result!") + ) + ) + + // all responses received, forward them to external simulation in a bundle + serviceStateData.extResultsData.queueExtResponseMsg( + new ProvideResultEntities(resultList.asJava) + ) + + //log.info("updatedResultStorage = " + updatedResultStorage) + + serviceStateData.copy( + resultStorage = updatedResultStorage, + recentResults = ReceiveDataMap(serviceStateData.subscribers.toSet) + ) + } + } else { + //log.info("No Subscriber!") + serviceStateData + } + } + + /* + + if (serviceStateData.recentResults.isDefined) { + val updatedReceivedResults = serviceStateData.recentResults.get.addDataWithoutCheck(result.getInputModel, result) // A valid result was sent val updatedResultStorage = - serviceStateData.resultStorage + (result.getUuid -> result) + serviceStateData.resultStorage + (result.getInputModel -> (Some(result), nextTick)) if ( - updatedResultStorage.size == serviceStateData.uuidToActorRef.size + updatedReceivedResults.nonComplete ) { // all responses received, forward them to external simulation in a bundle - serviceStateData.extResultsData.queueExtResponseMsg( - new ProvideResultEntities( - updatedResultStorage.values.toList.asJava + serviceStateData.copy( + recentResults = updatedReceivedResults, + resultStorage = updatedResultStorage + ) + } else { + log.info("Got all ResultResponseMessage -> Now forward to external simulation in a bundle ") + var resultList = List.empty[ResultEntity] + + updatedResultStorage.values.foreach( + result => resultList = resultList :+ result._1.getOrElse( + throw new RuntimeException("There is no result!") ) ) - serviceStateData.copy( - resultStorage = Map.empty + + // all responses received, forward them to external simulation in a bundle + serviceStateData.extResultsData.queueExtResponseMsg( + new ProvideResultEntities(resultList.asJava) ) - } else { - // responses are still incomplete + //log.info("updatedResultStorage = " + updatedResultStorage) + serviceStateData.copy( - resultStorage = updatedResultStorage + resultStorage = updatedResultStorage, + recentResults = None ) + } else { + //log.info("No Subscriber!") + serviceStateData } - } else { - serviceStateData + } else { // es wurde noch nicht berechnet, für welche Assets Results + } - } + case ResultRequestMessage(tick) => + + */ + } private def requestResults( @@ -165,7 +240,49 @@ class ExtResultDataService(override val scheduler: ActorRef) )(implicit serviceStateData: ExtResultsStateData ): (ExtResultsStateData, Option[Long]) = { - /* has to be filled -> For ReCoDE it isn't necessary that an external simulation requests results */ - (serviceStateData.copy(), None) + log.info(s"[requestResults] for tick $tick and resultStorage ${serviceStateData.resultStorage}") + var receiveDataMap = ReceiveDataMap[UUID, ResultEntity](serviceStateData.subscribers.toSet) + log.info(s"[requestResults] tick $tick -> created a receivedatamap " + receiveDataMap) + serviceStateData.resultStorage.foreach({ + case (uuid, (res, t)) => + log.info(s"[requestResults] tick = $tick, uuid = $uuid, and time = ${t.getOrElse("Option")}, result = ${res.getOrElse("Option")}") + if (t.getOrElse(-1) != tick) { //wenn nicht in diesem Tick gefragt, nehme Wert aus ResultDataStorage + receiveDataMap = receiveDataMap.addData( + uuid, + res.getOrElse( + throw new Exception("noResult") + ) + ) + log.info(s"[requestResults] tick $tick -> added to receivedatamap " + receiveDataMap) + } + }) + + /* + var requestedAssets = Set.empty[UUID] + serviceStateData.resultStorage.foreach({ + case (uuid, (_, t)) => + log.info(s"tick = $tick, uuid = $uuid, and time = $t") + if (t.getOrElse(throw new Exception("noTick")) == tick) { + requestedAssets += uuid + } + }) + + */ + + + log.info(s"[requestResults] tick $tick -> requestResults for " + receiveDataMap) + + if (receiveDataMap.isComplete) { + log.info(s"[requestResults] tick $tick -> ReceiveDataMap is complete -> send it right away") + serviceStateData.extResultsData.queueExtResponseMsg(new ProvideResultEntities()) + (serviceStateData.copy( + extResultsMessage = None, + recentResults = ReceiveDataMap(serviceStateData.subscribers.toSet)), None) + } else { + ( + serviceStateData.copy( + extResultsMessage = None, + recentResults = receiveDataMap + ), None)} } } 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 71900875dd..bdd20a5ab2 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala @@ -15,14 +15,12 @@ import edu.ie3.simona.actor.SimonaActorNaming.RichActorRefFactory import edu.ie3.simona.agent.EnvironmentRefs import edu.ie3.simona.agent.grid.GridAgentMessage.CreateGridAgent import edu.ie3.simona.agent.grid.{GridAgent, GridAgentMessage} -import edu.ie3.simona.api.ExtSimAdapter +import edu.ie3.simona.api.{ExtLinkInterface, ExtSimAdapter} import edu.ie3.simona.api.data.ExtData +import edu.ie3.simona.api.data.ExtDataSimulation import edu.ie3.simona.api.data.ev.{ExtEvData, ExtEvSimulation} -import edu.ie3.simona.api.data.primarydata.{ - ExtPrimaryData, - ExtPrimaryDataSimulation, -} -import edu.ie3.simona.api.data.results.{ExtResultDataSimulation, ExtResultsData} +import edu.ie3.simona.api.data.primarydata.{ExtPrimaryData, ExtPrimaryDataSimulation} +import edu.ie3.simona.api.data.results.{ExtResultData, ExtResultDataSimulation} import edu.ie3.simona.api.simulation.ExtSimAdapterData import edu.ie3.simona.config.{ArgsParser, RefSystemParser, SimonaConfig} import edu.ie3.simona.event.listener.{ResultEventListener, RuntimeEventListener} @@ -36,13 +34,10 @@ import edu.ie3.simona.service.SimonaService import edu.ie3.simona.service.ev.ExtEvDataService import edu.ie3.simona.service.ev.ExtEvDataService.InitExtEvData import edu.ie3.simona.service.primary.ExtPrimaryDataService.InitExtPrimaryData -import edu.ie3.simona.service.primary.{ - ExtPrimaryDataService, - PrimaryServiceProxy, -} +import edu.ie3.simona.service.primary.{ExtPrimaryDataService, PrimaryServiceProxy} import edu.ie3.simona.service.primary.PrimaryServiceProxy.InitPrimaryServiceProxyStateData import edu.ie3.simona.service.results.ExtResultDataService -import edu.ie3.simona.service.results.ExtResultDataService.InitExtResultsData +import edu.ie3.simona.service.results.ExtResultDataService.InitExtResultData import edu.ie3.simona.service.weather.WeatherService import edu.ie3.simona.service.weather.WeatherService.InitWeatherServiceStateData import edu.ie3.simona.sim.SimonaSim @@ -52,13 +47,11 @@ import edu.ie3.simona.util.TickUtil.RichZonedDateTime import edu.ie3.util.TimeUtil 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.{ClassicActorRefOps, TypedActorContextOps, TypedActorRefOps} import org.apache.pekko.actor.{ActorRef => ClassicRef} +import edu.ie3.simpleextsim.SimpleExtSimulation + import java.util.concurrent.LinkedBlockingQueue import scala.jdk.CollectionConverters._ @@ -202,107 +195,78 @@ class SimonaStandaloneSetup( context: ActorContext[_], scheduler: ActorRef[SchedulerMessage], ): ExtSimSetupData = { - val jars = ExtSimLoader.scanInputFolder() + val simpleExtSim = new SimpleExtSimulation() - val extLinks = jars.flatMap(ExtSimLoader.loadExtLink) + val extSimAdapter = context.toClassic.simonaActorOf( + ExtSimAdapter.props(scheduler.toClassic), + s"0", + ) + val extSimAdapterData = new ExtSimAdapterData(extSimAdapter, args) - 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) + // send init data right away, init activation is scheduled + extSimAdapter ! ExtSimAdapter.Create( + extSimAdapterData, + ScheduleLock.singleKey(context, scheduler, INIT_SIM_TICK), + ) - // send init data right away, init activation is scheduled - extSimAdapter ! ExtSimAdapter.Create( - extSimAdapterData, - ScheduleLock.singleKey(context, scheduler, INIT_SIM_TICK), - ) + val extPrimaryDataService = context.toClassic.simonaActorOf( + ExtPrimaryDataService.props(scheduler.toClassic), + s"0-0", + ) + val extPrimaryData = new ExtPrimaryData( + extPrimaryDataService, + extSimAdapter, + simpleExtSim.getPrimaryDataFactory, + ) - // 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)) - - case (extPrimaryDataSimulation: ExtPrimaryDataSimulation, dIndex) => - val extPrimaryDataService = context.toClassic.simonaActorOf( - ExtPrimaryDataService.props(scheduler.toClassic), - s"$index-$dIndex", - ) - val extPrimaryData = new ExtPrimaryData( - extPrimaryDataService, - extSimAdapter, - extPrimaryDataSimulation.getFactory, - ) - - extPrimaryDataService ! SimonaService.Create( - InitExtPrimaryData(extPrimaryData), - ScheduleLock.singleKey( - context, - scheduler, - INIT_SIM_TICK, - ), - ) - - (null, (classOf[ExtResultDataService], extPrimaryDataService)) - - case (_: ExtResultDataSimulation, dIndex) => - val extResultDataService = context.toClassic.simonaActorOf( - ExtResultDataService.props(scheduler.toClassic), - s"$index-$dIndex", - ) - val extResultsData = - new ExtResultsData(extResultDataService, extSimAdapter, null) - - extResultDataService ! SimonaService.Create( - InitExtResultsData(extResultsData), - ScheduleLock.singleKey( - context, - scheduler, - INIT_SIM_TICK, - ), - ) - - ( - extResultsData, - (classOf[ExtResultDataService], extResultDataService), - ) - }.unzip - - extLink.getExtSimulation.setup( - extSimAdapterData, - extData.toList.asJava, - ) + extPrimaryDataService ! SimonaService.Create( + InitExtPrimaryData(extPrimaryData), + ScheduleLock.singleKey( + context, + scheduler, + INIT_SIM_TICK, + ), + ) + + val extResultDataService = context.toClassic.simonaActorOf( + ExtResultDataService.props(scheduler.toClassic), + s"0-1", + ) + val extResultData = + new ExtResultData( + extResultDataService, + extSimAdapter, + simpleExtSim.getResultDataFactory, + ) - // starting external simulation - new Thread(extLink.getExtSimulation, s"External simulation $index") - .start() + extResultDataService ! SimonaService.Create( + InitExtResultData(extResultData), + ScheduleLock.singleKey( + context, + scheduler, + INIT_SIM_TICK, + ), + ) + + var simpleExtSimDatas: List[ExtData] = List(extResultData, extPrimaryData) + + simpleExtSim.setup( + extSimAdapterData, + simpleExtSimDatas.asJava, + ) + // starting external simulation + context.log.info("Starting External simulation") + new Thread(simpleExtSim, s"External simulation 0") + .start() + + val extDataServicesMap: Map[Class[_], ClassicRef] = Map( + classOf[ExtPrimaryDataService] -> extPrimaryDataService, + classOf[ExtResultDataService] -> extResultDataService + ) - (extSimAdapter, extDataInit) - }.unzip + val extSimAdapters = Iterable(extSimAdapter) - ExtSimSetupData(extSimAdapters, extDataServices.flatten.toMap) + ExtSimSetupData(extSimAdapters, extDataServicesMap) } override def timeAdvancer( diff --git a/src/main/scala/edu/ie3/simona/util/ReceiveDataMap.scala b/src/main/scala/edu/ie3/simona/util/ReceiveDataMap.scala index 1f7fca229e..11b87b6349 100644 --- a/src/main/scala/edu/ie3/simona/util/ReceiveDataMap.scala +++ b/src/main/scala/edu/ie3/simona/util/ReceiveDataMap.scala @@ -41,7 +41,22 @@ final case class ReceiveDataMap[K, V]( ) } -} + def addDataWithoutCheck( + key: K, + value: V, + ): ReceiveDataMap[K, V] = { + + if (expectedKeys.contains(key)) { + copy( + expectedKeys = expectedKeys.excl(key), + receivedData.updated(key, value), + ) + } else { + copy() + } + } + + } object ReceiveDataMap { From a39da228bb32aa96ec3f440eaaf8e23ad21439e0 Mon Sep 17 00:00:00 2001 From: jo-bao Date: Thu, 14 Mar 2024 08:19:54 +0100 Subject: [PATCH 13/41] working --- .../primary/ExtPrimaryDataService.scala | 5 +- .../results/ExtResultDataService.scala | 135 ++++++++---------- 2 files changed, 58 insertions(+), 82 deletions(-) diff --git a/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala b/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala index 057aa42fa3..520c6c918b 100644 --- a/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala +++ b/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala @@ -236,8 +236,5 @@ final case class ExtPrimaryDataService( extResponseMsg: DataMessage )(implicit serviceStateData: ExtPrimaryDataStateData - ): ExtPrimaryDataStateData = { - // not implemented - null - } + ): ExtPrimaryDataStateData = serviceStateData } diff --git a/src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala b/src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala index 2be13adcd4..1f7013c4e8 100644 --- a/src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala +++ b/src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala @@ -8,13 +8,14 @@ package edu.ie3.simona.service.results import org.apache.pekko.actor.typed.scaladsl.adapter.ClassicActorRefOps import edu.ie3.datamodel.models.result.ResultEntity +import edu.ie3.simona.agent.grid.GridAgentMessage import edu.ie3.simona.api.data.ontology.DataMessageFromExt import edu.ie3.simona.api.data.results.ExtResultData import edu.ie3.simona.api.data.results.ontology.{ProvideResultEntities, RequestResultEntities, ResultDataMessageFromExt} import edu.ie3.simona.exceptions.{InitializationException, ServiceException} import edu.ie3.simona.ontology.messages.SchedulerMessage.ScheduleActivation import edu.ie3.simona.ontology.messages.services.DataMessage -import edu.ie3.simona.ontology.messages.services.ResultMessage.{ResultResponseMessage, ResultRequestMessage} +import edu.ie3.simona.ontology.messages.services.ResultMessage.{ResultRequestMessage, ResultResponseMessage} import edu.ie3.simona.ontology.messages.services.ServiceMessage.ServiceRegistrationMessage import edu.ie3.simona.scheduler.ScheduleLock import edu.ie3.simona.scheduler.ScheduleLock.ScheduleKey @@ -23,9 +24,10 @@ import edu.ie3.simona.service.results.ExtResultDataService.{ExtResultsStateData, import edu.ie3.simona.service.{ExtDataSupport, SimonaService} import edu.ie3.simona.util.ReceiveDataMap import edu.ie3.util.scala.collection.immutable.SortedDistinctSeq +import org.apache.pekko.actor.typed.scaladsl.StashBuffer import org.apache.pekko.actor.{ActorContext, ActorRef, Props} -import scala.collection.immutable.SortedSet +import scala.collection.immutable.{Map, SortedSet} import java.util.UUID import scala.jdk.CollectionConverters.SeqHasAsJava import scala.util.{Failure, Success, Try} @@ -46,6 +48,8 @@ object ExtResultDataService { receivedResults: Int = 0, resultSink: List[ResultEntity] = List.empty, unlockKey: Option[ScheduleKey] = None, + sendedMessage: Boolean = true, + buffer: StashBuffer[ResultResponseMessage], ) extends ServiceBaseStateData final case class InitExtResultData( @@ -125,7 +129,6 @@ class ExtResultDataService(override val scheduler: ActorRef) serviceStateData: ExtResultsStateData ): ExtResultsStateData = extMsg match { case extResultsMessageFromExt: ResultDataMessageFromExt => - //log.info("Received ResultDataMessageFromExt with content: " + extResultsMessageFromExt) serviceStateData.copy( extResultsMessage = Some(extResultsMessageFromExt) ) @@ -148,9 +151,9 @@ class ExtResultDataService(override val scheduler: ActorRef) if (serviceStateData.subscribers.contains(result.getInputModel)) { log.info("[handleDataResponseMessage] Received ResultsResponseMessage with content " + extResponseMsg) log.info("[handleDataResponseMessage] RecentResults " + serviceStateData.recentResults) - var updatedReceivedResults = serviceStateData.recentResults.addData(result.getInputModel, result) + val updatedReceivedResults = serviceStateData.recentResults.addData(result.getInputModel, result) log.info("[handleDataResponseMessage] AddData to RecentResults -> updatedReceivedResults = " + updatedReceivedResults) - var updatedResultStorage = + val updatedResultStorage = serviceStateData.resultStorage + (result.getInputModel -> (Some(result), nextTick)) if (updatedReceivedResults.nonComplete) { // all responses received, forward them to external simulation in a bundle @@ -159,80 +162,37 @@ class ExtResultDataService(override val scheduler: ActorRef) resultStorage = updatedResultStorage ) } else { - log.info("Got all ResultResponseMessage -> Now forward to external simulation in a bundle ") + var resultList = List.empty[ResultEntity] - updatedResultStorage.values.foreach( - result => resultList = resultList :+ result._1.getOrElse( - throw new RuntimeException("There is no result!") - ) + + updatedReceivedResults.receivedData.values.foreach( + result => resultList = resultList :+ result ) // all responses received, forward them to external simulation in a bundle serviceStateData.extResultsData.queueExtResponseMsg( new ProvideResultEntities(resultList.asJava) ) - - //log.info("updatedResultStorage = " + updatedResultStorage) - + log.info("[handleDataResponseMessage] Got all ResultResponseMessage -> Now forward to external simulation in a bundle: " + resultList) serviceStateData.copy( resultStorage = updatedResultStorage, recentResults = ReceiveDataMap(serviceStateData.subscribers.toSet) ) + /* + + sendResultData(updatedResultStorage) + self ! ResultRequestMessage(null) + serviceStateData.copy( + recentResults = updatedReceivedResults, + resultStorage = updatedResultStorage + ) + */ } } else { - //log.info("No Subscriber!") serviceStateData } } - - /* - - if (serviceStateData.recentResults.isDefined) { - val updatedReceivedResults = serviceStateData.recentResults.get.addDataWithoutCheck(result.getInputModel, result) - // A valid result was sent - val updatedResultStorage = - serviceStateData.resultStorage + (result.getInputModel -> (Some(result), nextTick)) - if ( - updatedReceivedResults.nonComplete - ) { - // all responses received, forward them to external simulation in a bundle - serviceStateData.copy( - recentResults = updatedReceivedResults, - resultStorage = updatedResultStorage - ) - } else { - log.info("Got all ResultResponseMessage -> Now forward to external simulation in a bundle ") - var resultList = List.empty[ResultEntity] - - updatedResultStorage.values.foreach( - result => resultList = resultList :+ result._1.getOrElse( - throw new RuntimeException("There is no result!") - ) - ) - - // all responses received, forward them to external simulation in a bundle - serviceStateData.extResultsData.queueExtResponseMsg( - new ProvideResultEntities(resultList.asJava) - ) - - //log.info("updatedResultStorage = " + updatedResultStorage) - - serviceStateData.copy( - resultStorage = updatedResultStorage, - recentResults = None - ) - } else { - //log.info("No Subscriber!") - serviceStateData - } - } else { // es wurde noch nicht berechnet, für welche Assets Results - - } - case ResultRequestMessage(tick) => - - */ - } private def requestResults( @@ -245,7 +205,7 @@ class ExtResultDataService(override val scheduler: ActorRef) log.info(s"[requestResults] tick $tick -> created a receivedatamap " + receiveDataMap) serviceStateData.resultStorage.foreach({ case (uuid, (res, t)) => - log.info(s"[requestResults] tick = $tick, uuid = $uuid, and time = ${t.getOrElse("Option")}, result = ${res.getOrElse("Option")}") + log.info(s"[requestResults] tick = $tick, uuid = $uuid, and time = ${t.getOrElse("Option")}, result = ${res.getOrElse("Option")}") if (t.getOrElse(-1) != tick) { //wenn nicht in diesem Tick gefragt, nehme Wert aus ResultDataStorage receiveDataMap = receiveDataMap.addData( uuid, @@ -257,24 +217,22 @@ class ExtResultDataService(override val scheduler: ActorRef) } }) - /* - var requestedAssets = Set.empty[UUID] - serviceStateData.resultStorage.foreach({ - case (uuid, (_, t)) => - log.info(s"tick = $tick, uuid = $uuid, and time = $t") - if (t.getOrElse(throw new Exception("noTick")) == tick) { - requestedAssets += uuid - } - }) - - */ - - log.info(s"[requestResults] tick $tick -> requestResults for " + receiveDataMap) if (receiveDataMap.isComplete) { - log.info(s"[requestResults] tick $tick -> ReceiveDataMap is complete -> send it right away") - serviceStateData.extResultsData.queueExtResponseMsg(new ProvideResultEntities()) + var resultList = List.empty[ResultEntity] + + serviceStateData.resultStorage.values.foreach( + result => resultList = resultList :+ result._1.getOrElse( + throw new RuntimeException("There is no result!") + ) + ) + + log.info(s"[requestResults] tick $tick -> ReceiveDataMap is complete -> send it right away: " + resultList) + // all responses received, forward them to external simulation in a bundle + serviceStateData.extResultsData.queueExtResponseMsg( + new ProvideResultEntities(resultList.asJava) + ) (serviceStateData.copy( extResultsMessage = None, recentResults = ReceiveDataMap(serviceStateData.subscribers.toSet)), None) @@ -285,4 +243,25 @@ class ExtResultDataService(override val scheduler: ActorRef) recentResults = receiveDataMap ), None)} } + + private def sendResultData( + resultStorage: Map[UUID, (Option[ResultEntity], Option[Long])] + )(implicit + serviceStateData: ExtResultsStateData + ): Unit = { + var resultList = List.empty[ResultEntity] + + resultStorage.values.foreach( + result => resultList = resultList :+ result._1.getOrElse( + throw new RuntimeException("There is no result!") + ) + ) + + log.info("sendResultData " + resultList) + // all responses received, forward them to external simulation in a bundle + serviceStateData.extResultsData.queueExtResponseMsg( + new ProvideResultEntities(resultList.asJava) + ) + } } + From 10f8536f4c0e9c907bce74f09aa77be8d997a249 Mon Sep 17 00:00:00 2001 From: Johannes Bao Date: Fri, 15 Mar 2024 09:18:25 +0100 Subject: [PATCH 14/41] primary and result init --- .../edu/ie3/simona/event/ResultEvent.scala | 5 - .../event/listener/DelayedStopHelper.scala | 5 +- .../messages/services/ServiceMessage.scala | 11 ++ .../ie3/simona/service/SimonaService.scala | 11 +- .../primary/ExtPrimaryDataService.scala | 8 +- .../service/primary/PrimaryServiceProxy.scala | 10 +- .../results/ExtResultDataService.scala | 170 +++++++++--------- .../scala/edu/ie3/simona/sim/SimonaSim.scala | 20 ++- .../simona/sim/setup/ExtSimSetupData.scala | 15 ++ .../sim/setup/SimonaStandaloneSetup.scala | 12 +- 10 files changed, 170 insertions(+), 97 deletions(-) diff --git a/src/main/scala/edu/ie3/simona/event/ResultEvent.scala b/src/main/scala/edu/ie3/simona/event/ResultEvent.scala index 5d3cc78694..4a8720304a 100644 --- a/src/main/scala/edu/ie3/simona/event/ResultEvent.scala +++ b/src/main/scala/edu/ie3/simona/event/ResultEvent.scala @@ -26,11 +26,6 @@ sealed trait ResultEvent extends Event with ResultEventListener.Request */ object ResultEvent { - final case class ExternalResultEvent( - resultEntity: ResultEntity, - nextTick: Long - ) extends ResultEvent - /** Event that holds a calculation result of a * [[edu.ie3.simona.model.participant.SystemParticipant]] * diff --git a/src/main/scala/edu/ie3/simona/event/listener/DelayedStopHelper.scala b/src/main/scala/edu/ie3/simona/event/listener/DelayedStopHelper.scala index c6d8cea4d5..02f5c5d516 100644 --- a/src/main/scala/edu/ie3/simona/event/listener/DelayedStopHelper.scala +++ b/src/main/scala/edu/ie3/simona/event/listener/DelayedStopHelper.scala @@ -6,6 +6,7 @@ package edu.ie3.simona.event.listener +import edu.ie3.simona.service.results.ExtResultDataService import org.apache.pekko.actor.typed.Behavior import org.apache.pekko.actor.typed.scaladsl.{ActorContext, Behaviors} @@ -37,8 +38,8 @@ object DelayedStopHelper { : PartialFunction[(ActorContext[T], StoppingMsg), Behavior[T]] = { case (ctx, FlushAndStop) => - ctx.log.debug( - s"Received FlushAndStop message, shutting down once no message has been received for 5 seconds." + ctx.log.info( + s"$this Received FlushAndStop message, shutting down once no message has been received for 5 seconds." ) ctx.setReceiveTimeout(5.seconds, StopTimeout) Behaviors.same diff --git a/src/main/scala/edu/ie3/simona/ontology/messages/services/ServiceMessage.scala b/src/main/scala/edu/ie3/simona/ontology/messages/services/ServiceMessage.scala index 4d5cbe842b..015828b485 100644 --- a/src/main/scala/edu/ie3/simona/ontology/messages/services/ServiceMessage.scala +++ b/src/main/scala/edu/ie3/simona/ontology/messages/services/ServiceMessage.scala @@ -12,6 +12,7 @@ import java.util.UUID import edu.ie3.simona.agent.participant.data.Data import edu.ie3.simona.ontology.messages.services import edu.ie3.simona.scheduler.ScheduleLock.ScheduleKey +import org.apache.pekko.util.Timeout /** Collections of all messages, that are send to and from the different * services @@ -20,6 +21,9 @@ sealed trait ServiceMessage case object ServiceMessage { + final case class RequestExtPrimaryDataAssets() extends ServiceMessage { + } + /** Message used to register for a service */ trait ServiceRegistrationMessage extends ServiceMessage @@ -47,6 +51,8 @@ case object ServiceMessage { requestingActor: ActorRef, ) extends ServiceRegistrationMessage + final case class ExtPrimaryDataServiceInitRequestMessage() extends ServiceRegistrationMessage + sealed trait RegistrationResponseMessage extends ServiceMessage { val serviceRef: ActorRef } @@ -70,6 +76,11 @@ case object ServiceMessage { tick: Long, unlockKey: ScheduleKey, ) + + final case class ExtPrimaryDataServiceInitResponseMessage( + override val serviceRef: ActorRef, + subscribers: List[UUID] + ) extends RegistrationResponseMessage } /** Actual provision of data diff --git a/src/main/scala/edu/ie3/simona/service/SimonaService.scala b/src/main/scala/edu/ie3/simona/service/SimonaService.scala index aa219da531..a090c3e3a9 100644 --- a/src/main/scala/edu/ie3/simona/service/SimonaService.scala +++ b/src/main/scala/edu/ie3/simona/service/SimonaService.scala @@ -6,8 +6,10 @@ package edu.ie3.simona.service +import edu.ie3.simona.api.data.results.ontology.ResultDataMessageFromExt import edu.ie3.simona.event import edu.ie3.simona.event.Event +import edu.ie3.simona.event.listener.DelayedStopHelper import org.apache.pekko.actor.typed.scaladsl.adapter.ClassicActorRefOps import org.apache.pekko.actor.{Actor, ActorContext, ActorRef, Stash} import edu.ie3.simona.logging.SimonaActorLogging @@ -20,6 +22,7 @@ import edu.ie3.simona.scheduler.ScheduleLock.ScheduleKey import edu.ie3.simona.service.ServiceStateData.{InitializeServiceStateData, ServiceBaseStateData} import edu.ie3.simona.service.SimonaService.Create import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK +import edu.ie3.simona.ontology.messages.services.ServiceMessage.RequestExtPrimaryDataAssets import scala.util.{Failure, Success, Try} @@ -106,7 +109,7 @@ abstract class SimonaService[ } // not ready yet to handle registrations, stash request away - case _: ServiceRegistrationMessage | _: Activation => + case _: ServiceRegistrationMessage | _: Activation | _: ResultDataMessageFromExt | _: RequestExtPrimaryDataAssets => stash() // unhandled message @@ -164,6 +167,12 @@ abstract class SimonaService[ scheduler ! Completion(self.toTyped, maybeNewTriggers) context become idle(updatedStateData) + /* + case msg: DelayedStopHelper.StoppingMsg => + DelayedStopHelper.handleMsg((context[_], msg)) + + */ + // unhandled message case x => log.error("Unhandled message received:{}", x) diff --git a/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala b/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala index 520c6c918b..dfcfbe7d18 100644 --- a/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala +++ b/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala @@ -20,8 +20,12 @@ import edu.ie3.simona.ontology.messages.services.ServiceMessage.{ ExtPrimaryDataServiceRegistrationMessage, PrimaryServiceRegistrationMessage, WorkerRegistrationMessage, + ExtPrimaryDataServiceInitRequestMessage +} +import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.{ + RegistrationSuccessfulMessage, + ExtPrimaryDataServiceInitResponseMessage } -import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.RegistrationSuccessfulMessage import edu.ie3.simona.ontology.messages.services.{DataMessage, ServiceMessage} import edu.ie3.simona.scheduler.ScheduleLock import edu.ie3.simona.service.ServiceStateData.{ @@ -50,6 +54,7 @@ object ExtPrimaryDataService { final case class ExtPrimaryDataStateData( extPrimaryData: ExtPrimaryData, + subscribers: List[UUID] = List.empty, uuidToActorRef: Map[UUID, ActorRef] = Map.empty[UUID, ActorRef], // subscribers in SIMONA extPrimaryDataMessage: Option[PrimaryDataMessageFromExt] = None, @@ -72,7 +77,6 @@ final case class ExtPrimaryDataService( val primaryDataInitializedStateData = ExtPrimaryDataStateData( extPrimaryData ) - Success( primaryDataInitializedStateData, None, diff --git a/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala b/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala index 66c2b2484f..7670a56e4a 100644 --- a/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala +++ b/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala @@ -6,8 +6,11 @@ package edu.ie3.simona.service.primary +import scala.concurrent.duration._ import org.apache.pekko.actor.typed.scaladsl.adapter.ClassicActorRefOps import org.apache.pekko.actor.{Actor, ActorRef, PoisonPill, Props} +import org.apache.pekko.pattern.ask +import org.apache.pekko.util.Timeout import edu.ie3.datamodel.io.connectors.SqlConnector import edu.ie3.datamodel.io.csv.CsvIndividualTimeSeriesMetaInformation import edu.ie3.datamodel.io.naming.timeseries.IndividualTimeSeriesMetaInformation @@ -29,6 +32,7 @@ import edu.ie3.datamodel.io.source.{ TimeSeriesMetaInformationSource, } import edu.ie3.datamodel.models.value.Value +import edu.ie3.simona.api.data.primarydata.ExtPrimaryData import edu.ie3.simona.config.SimonaConfig.PrimaryDataCsvParams import edu.ie3.simona.config.SimonaConfig.Simona.Input.Primary.SqlParams import edu.ie3.simona.config.SimonaConfig.Simona.Input.{ @@ -46,6 +50,7 @@ import edu.ie3.simona.ontology.messages.services.ServiceMessage.{ ExtPrimaryDataServiceRegistrationMessage, PrimaryServiceRegistrationMessage, WorkerRegistrationMessage, + RequestExtPrimaryDataAssets } import edu.ie3.simona.scheduler.ScheduleLock import edu.ie3.simona.service.{ServiceStateData, SimonaService} @@ -110,6 +115,7 @@ case class PrimaryServiceProxy( initStateData.primaryConfig, initStateData.simulationStart, initStateData.extSimulation, + initStateData.extPrimaryData ) match { case Success(stateData) => scheduler ! Completion(self.toTyped) @@ -143,6 +149,7 @@ case class PrimaryServiceProxy( primaryConfig: PrimaryConfig, simulationStart: ZonedDateTime, extSimulation: Option[ActorRef], + extPrimaryData: Option[ExtPrimaryData] ): Try[PrimaryServiceStateData] = { createSources(primaryConfig).map { case (mappingSource, metaInformationSource) => @@ -179,7 +186,7 @@ case class PrimaryServiceProxy( simulationStart, primaryConfig, mappingSource, - getSubscribers, + extPrimaryData.getOrElse(throw new Exception("External Primary Data Simulation is requested without ExtPrimaryData")).getPrimaryDataAssets.asScala, extSimulation, ) } else { @@ -558,6 +565,7 @@ object PrimaryServiceProxy { primaryConfig: PrimaryConfig, simulationStart: ZonedDateTime, extSimulation: Option[ActorRef], + extPrimaryData: Option[ExtPrimaryData] ) extends InitializeServiceStateData /** Holding the state of an initialized proxy. diff --git a/src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala b/src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala index 1f7013c4e8..03522ade62 100644 --- a/src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala +++ b/src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala @@ -11,7 +11,7 @@ import edu.ie3.datamodel.models.result.ResultEntity import edu.ie3.simona.agent.grid.GridAgentMessage import edu.ie3.simona.api.data.ontology.DataMessageFromExt import edu.ie3.simona.api.data.results.ExtResultData -import edu.ie3.simona.api.data.results.ontology.{ProvideResultEntities, RequestResultEntities, ResultDataMessageFromExt} +import edu.ie3.simona.api.data.results.ontology.{ProvideExpectedAssets, ProvideResultEntities, RequestResultEntities, ResultDataMessageFromExt} import edu.ie3.simona.exceptions.{InitializationException, ServiceException} import edu.ie3.simona.ontology.messages.SchedulerMessage.ScheduleActivation import edu.ie3.simona.ontology.messages.services.DataMessage @@ -24,15 +24,18 @@ import edu.ie3.simona.service.results.ExtResultDataService.{ExtResultsStateData, import edu.ie3.simona.service.{ExtDataSupport, SimonaService} import edu.ie3.simona.util.ReceiveDataMap import edu.ie3.util.scala.collection.immutable.SortedDistinctSeq +import org.apache.pekko.actor.typed.Behavior import org.apache.pekko.actor.typed.scaladsl.StashBuffer import org.apache.pekko.actor.{ActorContext, ActorRef, Props} +import scala.jdk.CollectionConverters._ import scala.collection.immutable.{Map, SortedSet} import java.util.UUID import scala.jdk.CollectionConverters.SeqHasAsJava import scala.util.{Failure, Success, Try} object ExtResultDataService { + def props(scheduler: ActorRef): Props = Props( new ExtResultDataService(scheduler: ActorRef) @@ -44,12 +47,7 @@ object ExtResultDataService { extResultsMessage: Option[ResultDataMessageFromExt] = None, resultStorage: Map[UUID, (Option[ResultEntity], Option[Long])] = Map.empty, // UUID -> Result, nextTick maybeNextActivationTick: Option[Long] = None, - recentResults: ReceiveDataMap[UUID, ResultEntity] = ReceiveDataMap.empty, - receivedResults: Int = 0, - resultSink: List[ResultEntity] = List.empty, - unlockKey: Option[ScheduleKey] = None, - sendedMessage: Boolean = true, - buffer: StashBuffer[ResultResponseMessage], + recentResults: Option[ReceiveDataMap[UUID, ResultEntity]] = None, ) extends ServiceBaseStateData final case class InitExtResultData( @@ -65,10 +63,18 @@ class ExtResultDataService(override val scheduler: ActorRef) initServiceData: InitializeServiceStateData ): Try[(ExtResultsStateData, Option[Long])] = { initServiceData match { - case InitExtResultData(extResultsData) => + case InitExtResultData(extResultData) => + val initSubscribers = extResultData.getResultDataAssets.asScala.toList + + var initResultStorage = Map.empty[UUID, (Option[ResultEntity], Option[Long])] + initSubscribers.foreach( + uuid => initResultStorage = initResultStorage + (uuid -> (None, Some(0L))) + ) + /* val initSubscribers = List( UUID.fromString("de8cfef5-7620-4b9e-9a10-1faebb5a80c0"), UUID.fromString("2560c371-f420-4c2a-b4e6-e04c11b64c03")) + val resultInitializedStateData = ExtResultsStateData( extResultsData = extResultsData, subscribers = initSubscribers, @@ -76,7 +82,14 @@ class ExtResultDataService(override val scheduler: ActorRef) UUID.fromString("de8cfef5-7620-4b9e-9a10-1faebb5a80c0") -> (None, Some(0)), UUID.fromString("2560c371-f420-4c2a-b4e6-e04c11b64c03") -> (None, Some(0)), ), - recentResults = ReceiveDataMap(initSubscribers.toSet) + recentResults = Option.empty[ReceiveDataMap[UUID, ResultEntity]] + ) + stash() + */ + val resultInitializedStateData = ExtResultsStateData( + extResultsData = extResultData, + subscribers = initSubscribers, + resultStorage = initResultStorage ) Success(resultInitializedStateData, None) @@ -127,8 +140,19 @@ class ExtResultDataService(override val scheduler: ActorRef) extMsg: DataMessageFromExt )(implicit serviceStateData: ExtResultsStateData - ): ExtResultsStateData = extMsg match { - case extResultsMessageFromExt: ResultDataMessageFromExt => + ): ExtResultsStateData = + extMsg match { + /* + case extExpectedAssets: ProvideExpectedAssets => + var initResultStorage = Map.empty[UUID, (Option[ResultEntity], Option[Long])] + extExpectedAssets.expectedAssets.forEach( + uuid => initResultStorage = initResultStorage + (uuid -> (None, Some(0L))) + ) + serviceStateData.copy( + subscribers = extExpectedAssets.expectedAssets.asScala.toList + ) + */ + case extResultsMessageFromExt: RequestResultEntities => serviceStateData.copy( extResultsMessage = Some(extResultsMessageFromExt) ) @@ -148,50 +172,48 @@ class ExtResultDataService(override val scheduler: ActorRef) )(implicit serviceStateData: ExtResultsStateData): ExtResultsStateData = { extResponseMsg match { case ResultResponseMessage(result, nextTick) => - if (serviceStateData.subscribers.contains(result.getInputModel)) { - log.info("[handleDataResponseMessage] Received ResultsResponseMessage with content " + extResponseMsg) - log.info("[handleDataResponseMessage] RecentResults " + serviceStateData.recentResults) - val updatedReceivedResults = serviceStateData.recentResults.addData(result.getInputModel, result) - log.info("[handleDataResponseMessage] AddData to RecentResults -> updatedReceivedResults = " + updatedReceivedResults) - val updatedResultStorage = - serviceStateData.resultStorage + (result.getInputModel -> (Some(result), nextTick)) - if (updatedReceivedResults.nonComplete) { - // all responses received, forward them to external simulation in a bundle - serviceStateData.copy( - recentResults = updatedReceivedResults, - resultStorage = updatedResultStorage - ) - } else { - - var resultList = List.empty[ResultEntity] - - - updatedReceivedResults.receivedData.values.foreach( - result => resultList = resultList :+ result - ) - - // all responses received, forward them to external simulation in a bundle - serviceStateData.extResultsData.queueExtResponseMsg( - new ProvideResultEntities(resultList.asJava) - ) - log.info("[handleDataResponseMessage] Got all ResultResponseMessage -> Now forward to external simulation in a bundle: " + resultList) - serviceStateData.copy( - resultStorage = updatedResultStorage, - recentResults = ReceiveDataMap(serviceStateData.subscribers.toSet) - ) - /* - - sendResultData(updatedResultStorage) - self ! ResultRequestMessage(null) + if (serviceStateData.recentResults.isDefined) { + // process dataResponses + if (serviceStateData.subscribers.contains(result.getInputModel)) { + //log.info("[handleDataResponseMessage] Received ResultsResponseMessage with content " + extResponseMsg) + //log.info("[handleDataResponseMessage] RecentResults " + serviceStateData.recentResults) + val updatedReceivedResults = serviceStateData.recentResults.getOrElse(throw new Exception("noMap")).addData(result.getInputModel, result) + //log.info("[handleDataResponseMessage] AddData to RecentResults -> updatedReceivedResults = " + updatedReceivedResults) + val updatedResultStorage = + serviceStateData.resultStorage + (result.getInputModel -> (Some(result), nextTick)) + if (updatedReceivedResults.nonComplete) { + // all responses received, forward them to external simulation in a bundle serviceStateData.copy( - recentResults = updatedReceivedResults, + recentResults = Some(updatedReceivedResults), resultStorage = updatedResultStorage ) - */ + } else { + var resultList = List.empty[ResultEntity] + updatedReceivedResults.receivedData.values.foreach( + result => resultList = resultList :+ result + ) + // all responses received, forward them to external simulation in a bundle + serviceStateData.extResultsData.queueExtResponseMsg( + new ProvideResultEntities(resultList.asJava) + ) + //log.info("[handleDataResponseMessage] Got all ResultResponseMessage -> Now forward to external simulation in a bundle: " + resultList) + serviceStateData.copy( + resultStorage = updatedResultStorage, + recentResults = None + ) + } + } else { + serviceStateData } } else { - serviceStateData + // the results arrived too early -> stash them away + stash() + serviceStateData } + case ResultRequestMessage(tick) => + // now the expected results are calculated -> + unstashAll() + serviceStateData } } @@ -200,12 +222,12 @@ class ExtResultDataService(override val scheduler: ActorRef) )(implicit serviceStateData: ExtResultsStateData ): (ExtResultsStateData, Option[Long]) = { - log.info(s"[requestResults] for tick $tick and resultStorage ${serviceStateData.resultStorage}") + //log.info(s"[requestResults] for tick $tick and resultStorage ${serviceStateData.resultStorage}") var receiveDataMap = ReceiveDataMap[UUID, ResultEntity](serviceStateData.subscribers.toSet) - log.info(s"[requestResults] tick $tick -> created a receivedatamap " + receiveDataMap) + //log.info(s"[requestResults] tick $tick -> created a receivedatamap " + receiveDataMap) serviceStateData.resultStorage.foreach({ case (uuid, (res, t)) => - log.info(s"[requestResults] tick = $tick, uuid = $uuid, and time = ${t.getOrElse("Option")}, result = ${res.getOrElse("Option")}") + //log.info(s"[requestResults] tick = $tick, uuid = $uuid, and time = ${t.getOrElse("Option")}, result = ${res.getOrElse("Option")}") if (t.getOrElse(-1) != tick) { //wenn nicht in diesem Tick gefragt, nehme Wert aus ResultDataStorage receiveDataMap = receiveDataMap.addData( uuid, @@ -213,55 +235,37 @@ class ExtResultDataService(override val scheduler: ActorRef) throw new Exception("noResult") ) ) - log.info(s"[requestResults] tick $tick -> added to receivedatamap " + receiveDataMap) + //log.info(s"[requestResults] tick $tick -> added to receivedatamap " + receiveDataMap) } }) - log.info(s"[requestResults] tick $tick -> requestResults for " + receiveDataMap) + //log.info(s"[requestResults] tick $tick -> requestResults for " + receiveDataMap) if (receiveDataMap.isComplete) { var resultList = List.empty[ResultEntity] - serviceStateData.resultStorage.values.foreach( result => resultList = resultList :+ result._1.getOrElse( throw new RuntimeException("There is no result!") ) ) - - log.info(s"[requestResults] tick $tick -> ReceiveDataMap is complete -> send it right away: " + resultList) + //log.info(s"[requestResults] tick $tick -> ReceiveDataMap is complete -> send it right away: " + resultList) // all responses received, forward them to external simulation in a bundle serviceStateData.extResultsData.queueExtResponseMsg( new ProvideResultEntities(resultList.asJava) ) (serviceStateData.copy( extResultsMessage = None, - recentResults = ReceiveDataMap(serviceStateData.subscribers.toSet)), None) - } else { - ( - serviceStateData.copy( - extResultsMessage = None, - recentResults = receiveDataMap - ), None)} - } - - private def sendResultData( - resultStorage: Map[UUID, (Option[ResultEntity], Option[Long])] - )(implicit - serviceStateData: ExtResultsStateData - ): Unit = { - var resultList = List.empty[ResultEntity] - - resultStorage.values.foreach( - result => resultList = resultList :+ result._1.getOrElse( - throw new RuntimeException("There is no result!") - ) - ) + recentResults = None), None) - log.info("sendResultData " + resultList) - // all responses received, forward them to external simulation in a bundle - serviceStateData.extResultsData.queueExtResponseMsg( - new ProvideResultEntities(resultList.asJava) - ) + } else { + // We wait for result requests! + self ! ResultRequestMessage(tick) + ( + serviceStateData.copy( + extResultsMessage = None, + recentResults = Some(receiveDataMap) + ), None) + } } } diff --git a/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala b/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala index 2c800b833c..35c4fc2029 100644 --- a/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala +++ b/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala @@ -12,12 +12,15 @@ 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.service.results.ExtResultDataService import edu.ie3.simona.sim.setup.{ExtSimSetupData, SimonaSetup} import edu.ie3.util.scala.Scope import org.apache.pekko.actor.typed.scaladsl.adapter._ import org.apache.pekko.actor.typed.scaladsl.{ActorContext, Behaviors} import org.apache.pekko.actor.typed.{ActorRef, Behavior, PostStop, Terminated} import org.apache.pekko.actor.{ActorRef => ClassicRef} +import org.apache.pekko.actor.typed.scaladsl.adapter.ClassicActorRefOps +import org.apache.pekko.actor.typed.{ActorRef => TypedActorRef} /** Main entrance point to a simona simulation as the guardian actor. This actor * starts the initialization of all actors and waits for the simulation to end. @@ -131,6 +134,16 @@ object SimonaSim { gridAgents ++ extSimulationData.extDataServices.values.map(_.toTyped) + /* + val delayedActors = resultEventListeners.appended(runtimeEventListener).appended(extSimulationData.extResultDataService.getOrElse( + throw new Exception("Problem!") + ).toTyped[DelayedStopHelper.StoppingMsg]) + + + ctx.log.info(s"delayedActors: $delayedActors") + */ + + idle( ActorData( starter, @@ -204,10 +217,15 @@ object SimonaSim { ref ! ExtSimAdapter.Stop(simulationSuccessful) } + ctx.log.info(s"delayedActors: ${actorData.delayedStoppingActors}") + // if the simulation is successful, we're waiting for the delayed // stopping listeners to terminate and thus do not unwatch them here actorData.delayedStoppingActors.foreach( - _ ! DelayedStopHelper.FlushAndStop + actor => { + ctx.log.info(s"send FlushAndStop to $actor") + actor ! DelayedStopHelper.FlushAndStop + } ) maybeStop( 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 8a42c828e5..07cef84d02 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/ExtSimSetupData.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/ExtSimSetupData.scala @@ -10,10 +10,15 @@ import org.apache.pekko.actor.ActorRef import edu.ie3.simona.service.ev.ExtEvDataService import edu.ie3.simona.service.primary.ExtPrimaryDataService import edu.ie3.simona.service.results.ExtResultDataService +import edu.ie3.simona.api.data.ExtData +import edu.ie3.simona.api.data.primarydata.ExtPrimaryData +import edu.ie3.simona.api.data.results.ExtResultData +import edu.ie3.simona.api.data.ev.ExtEvData final case class ExtSimSetupData( extSimAdapters: Iterable[ActorRef], extDataServices: Map[Class[_], ActorRef], + extDatas: Set[ExtData], ) { def evDataService: Option[ActorRef] = @@ -24,4 +29,14 @@ final case class ExtSimSetupData( def extResultDataService: Option[ActorRef] = extDataServices.get(classOf[ExtResultDataService]) + + def extEvData: Option[ExtEvData] = { + extDatas.collectFirst { case extData: ExtEvData => extData } + } + def extPrimaryData: Option[ExtPrimaryData] = { + extDatas.collectFirst { case extData: ExtPrimaryData => extData } + } + def extResultData: Option[ExtResultData] = { + extDatas.collectFirst { case extData: ExtResultData => extData } + } } 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 bdd20a5ab2..5bbf087803 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala @@ -159,6 +159,7 @@ class SimonaStandaloneSetup( simonaConfig.simona.input.primary, simulationStart, extSimSetupData.extPrimaryDataService, + extSimSetupData.extPrimaryData ), simulationStart, ) @@ -217,6 +218,7 @@ class SimonaStandaloneSetup( extPrimaryDataService, extSimAdapter, simpleExtSim.getPrimaryDataFactory, + simpleExtSim.getPrimaryDataAssets ) extPrimaryDataService ! SimonaService.Create( @@ -237,6 +239,7 @@ class SimonaStandaloneSetup( extResultDataService, extSimAdapter, simpleExtSim.getResultDataFactory, + simpleExtSim.getResultDataAssets ) extResultDataService ! SimonaService.Create( @@ -248,7 +251,7 @@ class SimonaStandaloneSetup( ), ) - var simpleExtSimDatas: List[ExtData] = List(extResultData, extPrimaryData) + val simpleExtSimDatas: List[ExtData] = List(extResultData, extPrimaryData) simpleExtSim.setup( extSimAdapterData, @@ -266,7 +269,12 @@ class SimonaStandaloneSetup( val extSimAdapters = Iterable(extSimAdapter) - ExtSimSetupData(extSimAdapters, extDataServicesMap) + val extDatas = Set( + extPrimaryData, + extResultData + ) + + ExtSimSetupData(extSimAdapters, extDataServicesMap, extDatas) } override def timeAdvancer( From 9d24e727c673d4690b826a149ce96f7d7a2d1809 Mon Sep 17 00:00:00 2001 From: Johannes Bao Date: Fri, 15 Mar 2024 18:37:11 +0100 Subject: [PATCH 15/41] delayed stopping --- build.gradle | 8 + ...SimulationForPrimaryDataAndResultData.puml | 102 ++++++ .../agent/participant/ParticipantAgent.scala | 2 +- .../edu/ie3/simona/api/ExtSimAdapter.scala | 18 +- .../event/listener/DelayedStopHelper.scala | 5 +- .../event/listener/ResultEventListener.scala | 3 +- .../ie3/simona/io/grid/CsvGridSource.scala | 6 +- .../edu/ie3/simona/io/grid/GridProvider.scala | 5 +- .../io/result/ResultEntityCsvSink.scala | 2 +- .../io/result/ResultEntityKafkaSink.scala | 6 +- .../io/runtime/RuntimeEventKafkaSink.scala | 6 +- .../messages/services/ResultMessage.scala | 10 +- .../ie3/simona/service/SimonaService.scala | 1 - .../ExtTimeSeriesSubscribersSource.scala | 18 - .../service/primary/PrimaryServiceProxy.scala | 59 +-- .../results/ExtResultDataProvider.scala | 336 ++++++++++++++++++ .../results/ExtResultDataService.scala | 271 -------------- .../scala/edu/ie3/simona/sim/SimonaSim.scala | 15 +- .../simona/sim/setup/ExtSimSetupData.scala | 12 +- .../ie3/simona/sim/setup/SetupHelper.scala | 2 +- .../sim/setup/SimonaStandaloneSetup.scala | 69 ++-- .../util/scala/io/GraphicDataCleaner.scala | 2 +- .../util/scala/io/ScalaReflectionSerde.scala | 5 +- .../scala/quantities/DefaultQuantities.scala | 2 +- .../util/scala/quantities/QuantityUtil.scala | 1 - .../util/scala/quantities/ReactivePower.scala | 2 +- .../service/PrimaryDataFactoryDefault.scala | 2 +- 27 files changed, 533 insertions(+), 437 deletions(-) create mode 100644 docs/uml/protocol/ExtSimulationForPrimaryDataAndResultData.puml delete mode 100644 src/main/scala/edu/ie3/simona/service/primary/ExtTimeSeriesSubscribersSource.scala create mode 100644 src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala delete mode 100644 src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala diff --git a/build.gradle b/build.gradle index b367e10368..4b9c876b3c 100644 --- a/build.gradle +++ b/build.gradle @@ -100,6 +100,14 @@ dependencies { exclude group: 'edu.ie3' } + implementation('com.github.ie3-institute:simosaik') { + exclude group: 'org.apache.logging.log4j' + exclude group: 'org.slf4j' + /* Exclude our own nested dependencies */ + exclude group: 'edu.ie3' + } + + /* logging */ implementation "com.typesafe.scala-logging:scala-logging_${scalaVersion}:3.9.5" // pekko scala logging implementation "ch.qos.logback:logback-classic:1.5.3" diff --git a/docs/uml/protocol/ExtSimulationForPrimaryDataAndResultData.puml b/docs/uml/protocol/ExtSimulationForPrimaryDataAndResultData.puml new file mode 100644 index 0000000000..283eed4175 --- /dev/null +++ b/docs/uml/protocol/ExtSimulationForPrimaryDataAndResultData.puml @@ -0,0 +1,102 @@ +@startuml + +!theme plain + +==Init== + + + + +SimScheduler -> ExtSimAdapter: ! ActivityStartTrigger(-1L) +activate ExtSimAdapter + +ExtSimAdapter -> ExtSimulation: queue(ActivityStartTrigger(-1L)) +deactivate ExtSimAdapter +activate ExtSimulation +... Initialize external mobility simulation ... + +ExtSimulation -> ExtSimAdapter: ! CompletionMessage(newTriggers) +deactivate ExtSimulation +activate ExtSimAdapter + +ExtSimAdapter -> SimScheduler: ! CompletionMessage(newTriggers) +deactivate ExtSimAdapter + +==Sim== +SimScheduler -> ExtSimAdapter: ! ActivityStartTrigger(tick) +activate ExtSimAdapter + +ExtSimAdapter -> ExtSimulation: queue(ActivityStartTrigger(tick)) +deactivate ExtSimAdapter + +activate ExtSimulation +ExtSimulation -> ExtEvDataService: ! RequestEvcsFreeLots +ExtSimulation -> ExtSimAdapter: ! ScheduleDataServiceMessage(\n\tdataServiceRef\n) + +activate ExtSimAdapter +ExtSimAdapter -> SimScheduler: ! ScheduleTriggerMessage(\n\t_, dataServiceRef) +deactivate ExtSimAdapter + +activate SimScheduler +SimScheduler -> ExtEvDataService: ! ActivityStartTrigger(tick) +deactivate SimScheduler + +activate ExtEvDataService +ExtEvDataService -> EvcsAgent1: ! EvFreeLotsRequest(tick) +activate EvcsAgent1 +ExtEvDataService -> EvcsAgent2: ! EvFreeLotsRequest(tick) +activate EvcsAgent2 + +ExtEvDataService -> SimScheduler: ! CompletionMessage(None) + +EvcsAgent2 -> ExtEvDataService: ! FreeLotsResponse(_, _) +deactivate EvcsAgent2 +EvcsAgent1 -> ExtEvDataService: ! FreeLotsResponse(_, _) +deactivate EvcsAgent1 +ExtEvDataService -> ExtSimulation: queue(ProvideEvcsFreeLots(_)) +deactivate ExtEvDataService + +... Running external mobility simulation,\n determining EV positions ... +ExtSimulation -> ExtEvDataService: ! EvMovementsMessage(_) +ExtSimulation -> ExtSimAdapter: ! ScheduleDataServiceMessage(\n\tdataServiceRef\n) + +activate ExtSimAdapter +ExtSimAdapter -> SimScheduler: ! ScheduleTriggerMessage(\n\t_, dataServiceRef) +deactivate ExtSimAdapter + +activate SimScheduler +SimScheduler -> ExtEvDataService: ! ActivityStartTrigger(tick) +deactivate SimScheduler + +activate ExtEvDataService +ExtEvDataService -> EvcsAgent1: ! ProvideEvDataMessage(\n\ttick, _) +ExtEvDataService -> EvcsAgent2: ! ProvideEvDataMessage(\n\ttick, _) +ExtEvDataService -> SimScheduler: ! CompletionMessage(evcsTriggers) +deactivate ExtEvDataService + +activate SimScheduler +SimScheduler -> EvcsAgent1: ! ActivityStartTrigger(tick) +activate EvcsAgent1 +SimScheduler -> EvcsAgent2: ! ActivityStartTrigger(tick) +deactivate SimScheduler + +activate EvcsAgent2 +EvcsAgent1 -> SimScheduler: ! CompletionMessage(None) +deactivate EvcsAgent1 + +EvcsAgent2 -> ExtEvDataService: ! DepartedEvsResponse(_, _) +activate ExtEvDataService +EvcsAgent2 -> SimScheduler: ! CompletionMessage(None) +deactivate EvcsAgent2 + +ExtEvDataService -> ExtSimulation: queue(AllDepartedEvsResponse(_)) +deactivate ExtEvDataService + +ExtSimulation -> ExtSimAdapter: ! CompletionMessage(newTriggers) +deactivate ExtSimulation + +activate ExtSimAdapter +ExtSimAdapter -> SimScheduler: ! CompletionMessage(newTriggers) +deactivate ExtSimAdapter + +@enduml \ No newline at end of file diff --git a/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgent.scala b/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgent.scala index a370c0054a..c94d0db8c6 100644 --- a/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgent.scala +++ b/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgent.scala @@ -44,7 +44,7 @@ import edu.ie3.simona.model.participant.{ SystemParticipant, } import edu.ie3.simona.ontology.messages.Activation -import edu.ie3.simona.ontology.messages.services.ResultMessage.ResultRequestMessage + import edu.ie3.simona.ontology.messages.PowerMessage.RequestAssetPowerMessage import edu.ie3.simona.ontology.messages.SchedulerMessage.ScheduleActivation import edu.ie3.simona.ontology.messages.flex.FlexibilityMessage.{ diff --git a/src/main/scala/edu/ie3/simona/api/ExtSimAdapter.scala b/src/main/scala/edu/ie3/simona/api/ExtSimAdapter.scala index b71adf4a30..64ff24eaf6 100644 --- a/src/main/scala/edu/ie3/simona/api/ExtSimAdapter.scala +++ b/src/main/scala/edu/ie3/simona/api/ExtSimAdapter.scala @@ -6,27 +6,19 @@ package edu.ie3.simona.api -import org.apache.pekko.actor.typed.scaladsl.adapter.ClassicActorRefOps -import org.apache.pekko.actor.{Actor, ActorRef, PoisonPill, Props} import edu.ie3.simona.api.ExtSimAdapter.{Create, ExtSimAdapterStateData, Stop} import edu.ie3.simona.api.data.ontology.ScheduleDataServiceMessage import edu.ie3.simona.api.simulation.ExtSimAdapterData -import edu.ie3.simona.api.simulation.ontology.{ - ActivationMessage, - TerminationCompleted, - TerminationMessage, - CompletionMessage => ExtCompletionMessage, -} +import edu.ie3.simona.api.simulation.ontology.{ActivationMessage, TerminationCompleted, TerminationMessage, CompletionMessage => ExtCompletionMessage} import edu.ie3.simona.logging.SimonaActorLogging -import edu.ie3.simona.ontology.messages.SchedulerMessage.{ - Completion, - ScheduleActivation, -} -import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.ScheduleServiceActivation import edu.ie3.simona.ontology.messages.Activation +import edu.ie3.simona.ontology.messages.SchedulerMessage.{Completion, ScheduleActivation} +import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.ScheduleServiceActivation import edu.ie3.simona.scheduler.ScheduleLock import edu.ie3.simona.scheduler.ScheduleLock.ScheduleKey import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK +import org.apache.pekko.actor.typed.scaladsl.adapter.ClassicActorRefOps +import org.apache.pekko.actor.{Actor, ActorRef, PoisonPill, Props} import scala.jdk.OptionConverters._ diff --git a/src/main/scala/edu/ie3/simona/event/listener/DelayedStopHelper.scala b/src/main/scala/edu/ie3/simona/event/listener/DelayedStopHelper.scala index 02f5c5d516..3782ea7213 100644 --- a/src/main/scala/edu/ie3/simona/event/listener/DelayedStopHelper.scala +++ b/src/main/scala/edu/ie3/simona/event/listener/DelayedStopHelper.scala @@ -6,7 +6,7 @@ package edu.ie3.simona.event.listener -import edu.ie3.simona.service.results.ExtResultDataService +import edu.ie3.simona.service.results.ExtResultDataProvider import org.apache.pekko.actor.typed.Behavior import org.apache.pekko.actor.typed.scaladsl.{ActorContext, Behaviors} @@ -24,6 +24,7 @@ object DelayedStopHelper { sealed trait StoppingMsg extends ResultEventListener.Request with RuntimeEventListener.Request + with ExtResultDataProvider.Request /** Message indicating that [[RuntimeEventListener]] should stop. Instead of * using [[org.apache.pekko.actor.typed.scaladsl.ActorContext.stop]], this @@ -38,7 +39,7 @@ object DelayedStopHelper { : PartialFunction[(ActorContext[T], StoppingMsg), Behavior[T]] = { case (ctx, FlushAndStop) => - ctx.log.info( + ctx.log.debug( s"$this Received FlushAndStop message, shutting down once no message has been received for 5 seconds." ) ctx.setReceiveTimeout(5.seconds, StopTimeout) diff --git a/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala b/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala index 57315f6bd9..aa5907646d 100644 --- a/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala +++ b/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala @@ -22,7 +22,8 @@ import edu.ie3.simona.exceptions.{ ProcessResultEventException, } import edu.ie3.simona.io.result._ -import edu.ie3.simona.ontology.messages.services.ResultMessage.ResultResponseMessage +import edu.ie3.simona.service.results.ExtResultDataProvider.ResultResponseMessage + import edu.ie3.simona.util.ResultFileHierarchy import org.apache.pekko.actor.ActorRef import org.slf4j.Logger diff --git a/src/main/scala/edu/ie3/simona/io/grid/CsvGridSource.scala b/src/main/scala/edu/ie3/simona/io/grid/CsvGridSource.scala index 7726640c1f..cf089a3f39 100644 --- a/src/main/scala/edu/ie3/simona/io/grid/CsvGridSource.scala +++ b/src/main/scala/edu/ie3/simona/io/grid/CsvGridSource.scala @@ -10,11 +10,7 @@ import edu.ie3.datamodel.io.naming.FileNamingStrategy import edu.ie3.datamodel.io.source._ import edu.ie3.datamodel.io.source.csv._ import edu.ie3.datamodel.models.input.container._ -import edu.ie3.datamodel.models.input.thermal.{ - ThermalBusInput, - ThermalHouseInput, - ThermalStorageInput, -} +import edu.ie3.datamodel.models.input.thermal.{ThermalBusInput, ThermalHouseInput, ThermalStorageInput} import java.nio.file.Path import scala.jdk.CollectionConverters._ diff --git a/src/main/scala/edu/ie3/simona/io/grid/GridProvider.scala b/src/main/scala/edu/ie3/simona/io/grid/GridProvider.scala index 24277fcd7c..f7fd1ea4e3 100644 --- a/src/main/scala/edu/ie3/simona/io/grid/GridProvider.scala +++ b/src/main/scala/edu/ie3/simona/io/grid/GridProvider.scala @@ -9,10 +9,7 @@ package edu.ie3.simona.io.grid import com.typesafe.scalalogging.LazyLogging import edu.ie3.datamodel.io.naming.FileNamingStrategy import edu.ie3.datamodel.io.source.csv.CsvJointGridContainerSource -import edu.ie3.datamodel.models.input.container.{ - JointGridContainer, - ThermalGrid, -} +import edu.ie3.datamodel.models.input.container.{JointGridContainer, ThermalGrid} import edu.ie3.datamodel.models.input.thermal.ThermalBusInput import edu.ie3.datamodel.utils.validation.ValidationUtils import edu.ie3.simona.config.SimonaConfig diff --git a/src/main/scala/edu/ie3/simona/io/result/ResultEntityCsvSink.scala b/src/main/scala/edu/ie3/simona/io/result/ResultEntityCsvSink.scala index 88e6d5a080..66d7d1e0e6 100644 --- a/src/main/scala/edu/ie3/simona/io/result/ResultEntityCsvSink.scala +++ b/src/main/scala/edu/ie3/simona/io/result/ResultEntityCsvSink.scala @@ -6,7 +6,6 @@ package edu.ie3.simona.io.result -import org.apache.pekko.stream.IOResult import com.typesafe.scalalogging.LazyLogging import edu.ie3.datamodel.exceptions.EntityProcessorException import edu.ie3.datamodel.io.processor.result.ResultEntityProcessor @@ -14,6 +13,7 @@ import edu.ie3.datamodel.models.result.ResultEntity import edu.ie3.simona.exceptions.ProcessResultEventException import edu.ie3.util.StringUtils import edu.ie3.util.io.FileIOUtils +import org.apache.pekko.stream.IOResult import java.io.{BufferedWriter, File, FileWriter, Writer} import java.lang diff --git a/src/main/scala/edu/ie3/simona/io/result/ResultEntityKafkaSink.scala b/src/main/scala/edu/ie3/simona/io/result/ResultEntityKafkaSink.scala index 3effd9dd2d..89c56fa6a8 100644 --- a/src/main/scala/edu/ie3/simona/io/result/ResultEntityKafkaSink.scala +++ b/src/main/scala/edu/ie3/simona/io/result/ResultEntityKafkaSink.scala @@ -13,11 +13,7 @@ import edu.ie3.simona.io.result.plain.PlainWriter.NodeResultWriter import edu.ie3.simona.io.result.plain.{PlainResult, PlainWriter} import edu.ie3.util.scala.io.ScalaReflectionSerde.reflectionSerializer4S import io.confluent.kafka.serializers.AbstractKafkaSchemaSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG -import org.apache.kafka.clients.producer.{ - KafkaProducer, - ProducerConfig, - ProducerRecord, -} +import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} import org.apache.kafka.common.serialization.{Serdes, Serializer} import java.util.{Properties, UUID} diff --git a/src/main/scala/edu/ie3/simona/io/runtime/RuntimeEventKafkaSink.scala b/src/main/scala/edu/ie3/simona/io/runtime/RuntimeEventKafkaSink.scala index 4698d25215..2ca358109a 100644 --- a/src/main/scala/edu/ie3/simona/io/runtime/RuntimeEventKafkaSink.scala +++ b/src/main/scala/edu/ie3/simona/io/runtime/RuntimeEventKafkaSink.scala @@ -14,11 +14,7 @@ import edu.ie3.simona.io.runtime.RuntimeEventKafkaSink.SimonaEndMessage import edu.ie3.simona.io.runtime.RuntimeEventSink.RuntimeStats import edu.ie3.util.scala.io.ScalaReflectionSerde.reflectionSerializer4S import io.confluent.kafka.serializers.AbstractKafkaSchemaSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG -import org.apache.kafka.clients.producer.{ - KafkaProducer, - ProducerConfig, - ProducerRecord, -} +import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} import org.apache.kafka.common.serialization.{Serdes, Serializer} import org.slf4j.Logger diff --git a/src/main/scala/edu/ie3/simona/ontology/messages/services/ResultMessage.scala b/src/main/scala/edu/ie3/simona/ontology/messages/services/ResultMessage.scala index 1fc1eaae0b..a2c56bf46c 100644 --- a/src/main/scala/edu/ie3/simona/ontology/messages/services/ResultMessage.scala +++ b/src/main/scala/edu/ie3/simona/ontology/messages/services/ResultMessage.scala @@ -12,12 +12,6 @@ import edu.ie3.simona.ontology.messages.PowerMessage sealed trait ResultMessage extends DataMessage object ResultMessage { - final case class ResultResponseMessage( - result: ResultEntity, - nextTick: Option[Long] - ) - extends ResultMessage - final case class ResultRequestMessage( - currentTick: Long - ) extends ResultMessage + final case class ResultResponseMessageIdea() extends ResultMessage } + diff --git a/src/main/scala/edu/ie3/simona/service/SimonaService.scala b/src/main/scala/edu/ie3/simona/service/SimonaService.scala index a090c3e3a9..c6f2dc7b62 100644 --- a/src/main/scala/edu/ie3/simona/service/SimonaService.scala +++ b/src/main/scala/edu/ie3/simona/service/SimonaService.scala @@ -15,7 +15,6 @@ import org.apache.pekko.actor.{Actor, ActorContext, ActorRef, Stash} import edu.ie3.simona.logging.SimonaActorLogging import edu.ie3.simona.ontology.messages.Activation import edu.ie3.simona.ontology.messages.SchedulerMessage.{Completion, ScheduleActivation} -import edu.ie3.simona.ontology.messages.services.ResultMessage.ResultResponseMessage import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.ScheduleServiceActivation import edu.ie3.simona.ontology.messages.services.ServiceMessage.ServiceRegistrationMessage import edu.ie3.simona.scheduler.ScheduleLock.ScheduleKey diff --git a/src/main/scala/edu/ie3/simona/service/primary/ExtTimeSeriesSubscribersSource.scala b/src/main/scala/edu/ie3/simona/service/primary/ExtTimeSeriesSubscribersSource.scala deleted file mode 100644 index 86bfece987..0000000000 --- a/src/main/scala/edu/ie3/simona/service/primary/ExtTimeSeriesSubscribersSource.scala +++ /dev/null @@ -1,18 +0,0 @@ -/* - * © 2024. TU Dortmund University, - * Institute of Energy Systems, Energy Efficiency and Energy Economics, - * Research group Distribution grid planning and operation - */ - -package edu.ie3.simona.service.primary - -import java.util.UUID - -object ExtTimeSeriesSubscribersSource { - private val subscribers: Iterable[UUID] = Iterable( - UUID.fromString("fd1a8de9-722a-4304-8799-e1e976d9979c"), - UUID.fromString("ff0b995a-86ff-4f4d-987e-e475a64f2180") - ) - - def getSubscribers: Iterable[UUID] = subscribers -} diff --git a/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala b/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala index 7670a56e4a..cf8818f93c 100644 --- a/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala +++ b/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala @@ -6,67 +6,32 @@ package edu.ie3.simona.service.primary -import scala.concurrent.duration._ -import org.apache.pekko.actor.typed.scaladsl.adapter.ClassicActorRefOps -import org.apache.pekko.actor.{Actor, ActorRef, PoisonPill, Props} -import org.apache.pekko.pattern.ask -import org.apache.pekko.util.Timeout import edu.ie3.datamodel.io.connectors.SqlConnector import edu.ie3.datamodel.io.csv.CsvIndividualTimeSeriesMetaInformation import edu.ie3.datamodel.io.naming.timeseries.IndividualTimeSeriesMetaInformation -import edu.ie3.datamodel.io.naming.{ - DatabaseNamingStrategy, - EntityPersistenceNamingStrategy, - FileNamingStrategy, -} -import edu.ie3.datamodel.io.source.csv.{ - CsvTimeSeriesMappingSource, - CsvTimeSeriesMetaInformationSource, -} -import edu.ie3.datamodel.io.source.sql.{ - SqlTimeSeriesMappingSource, - SqlTimeSeriesMetaInformationSource, -} -import edu.ie3.datamodel.io.source.{ - TimeSeriesMappingSource, - TimeSeriesMetaInformationSource, -} +import edu.ie3.datamodel.io.naming.{DatabaseNamingStrategy, EntityPersistenceNamingStrategy, FileNamingStrategy} +import edu.ie3.datamodel.io.source.csv.{CsvTimeSeriesMappingSource, CsvTimeSeriesMetaInformationSource} +import edu.ie3.datamodel.io.source.sql.{SqlTimeSeriesMappingSource, SqlTimeSeriesMetaInformationSource} +import edu.ie3.datamodel.io.source.{TimeSeriesMappingSource, TimeSeriesMetaInformationSource} import edu.ie3.datamodel.models.value.Value import edu.ie3.simona.api.data.primarydata.ExtPrimaryData import edu.ie3.simona.config.SimonaConfig.PrimaryDataCsvParams import edu.ie3.simona.config.SimonaConfig.Simona.Input.Primary.SqlParams -import edu.ie3.simona.config.SimonaConfig.Simona.Input.{ - Primary => PrimaryConfig -} -import edu.ie3.simona.exceptions.{ - InitializationException, - InvalidConfigParameterException, -} +import edu.ie3.simona.config.SimonaConfig.Simona.Input.{Primary => PrimaryConfig} +import edu.ie3.simona.exceptions.{InitializationException, InvalidConfigParameterException} import edu.ie3.simona.logging.SimonaActorLogging import edu.ie3.simona.ontology.messages.Activation import edu.ie3.simona.ontology.messages.SchedulerMessage.Completion import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.RegistrationFailedMessage -import edu.ie3.simona.ontology.messages.services.ServiceMessage.{ - ExtPrimaryDataServiceRegistrationMessage, - PrimaryServiceRegistrationMessage, - WorkerRegistrationMessage, - RequestExtPrimaryDataAssets -} +import edu.ie3.simona.ontology.messages.services.ServiceMessage.{ExtPrimaryDataServiceRegistrationMessage, PrimaryServiceRegistrationMessage, WorkerRegistrationMessage} import edu.ie3.simona.scheduler.ScheduleLock -import edu.ie3.simona.service.{ServiceStateData, SimonaService} import edu.ie3.simona.service.ServiceStateData.InitializeServiceStateData -import edu.ie3.simona.service.primary.ExtTimeSeriesSubscribersSource.getSubscribers -import edu.ie3.simona.service.primary.PrimaryServiceProxy.{ - InitPrimaryServiceProxyStateData, - PrimaryServiceStateData, - SourceRef, -} -import edu.ie3.simona.service.primary.PrimaryServiceWorker.{ - CsvInitPrimaryServiceStateData, - InitPrimaryServiceStateData, - SqlInitPrimaryServiceStateData, -} +import edu.ie3.simona.service.primary.PrimaryServiceProxy.{InitPrimaryServiceProxyStateData, PrimaryServiceStateData, SourceRef} +import edu.ie3.simona.service.primary.PrimaryServiceWorker.{CsvInitPrimaryServiceStateData, InitPrimaryServiceStateData, SqlInitPrimaryServiceStateData} +import edu.ie3.simona.service.{ServiceStateData, SimonaService} import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK +import org.apache.pekko.actor.typed.scaladsl.adapter.ClassicActorRefOps +import org.apache.pekko.actor.{Actor, ActorRef, PoisonPill, Props} import java.nio.file.Paths import java.text.SimpleDateFormat diff --git a/src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala b/src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala new file mode 100644 index 0000000000..a029985be4 --- /dev/null +++ b/src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala @@ -0,0 +1,336 @@ +package edu.ie3.simona.service.results + +import edu.ie3.datamodel.models.result.ResultEntity +import edu.ie3.simona.api.data.results.ExtResultData +import edu.ie3.simona.api.data.results.ontology.{ProvideResultEntities, RequestResultEntities, ResultDataMessageFromExt} +import edu.ie3.simona.event.listener.DelayedStopHelper +import edu.ie3.simona.exceptions.{InitializationException, ServiceException} +import edu.ie3.simona.ontology.messages.{Activation, SchedulerMessage} +import edu.ie3.simona.ontology.messages.SchedulerMessage.{Completion, ScheduleActivation} +import edu.ie3.simona.scheduler.ScheduleLock.ScheduleKey +import edu.ie3.simona.service.ServiceStateData.{InitializeServiceStateData, ServiceBaseStateData} +import edu.ie3.simona.util.ReceiveDataMap +import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK +import org.apache.pekko.actor.typed.{Behavior, PostStop} +import org.apache.pekko.actor.typed.{ActorRef, Behavior} +import org.apache.pekko.actor.typed.scaladsl.{Behaviors, StashBuffer} +import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.ScheduleServiceActivation +import org.apache.pekko.actor.ActorContext + +import java.util.UUID +import scala.jdk.CollectionConverters._ + +object ExtResultDataProvider { + + trait Request + + final case class WrappedActivation(activation: Activation) extends Request + + /** ExtSimulation -> ExtResultDataProvider */ + final case class WrappedResultDataMessageFromExt(extResultDataMessageFromExt: ResultDataMessageFromExt) extends Request + final case class WrappedScheduleServiceActivationAdapter(scheduleServiceActivationMsg: ScheduleServiceActivation) extends Request + + final case class RequestDataMessageAdapter(sender: ActorRef[ActorRef[ResultDataMessageFromExt]]) extends Request + + final case class RequestScheduleActivationAdapter(sender: ActorRef[ActorRef[ScheduleServiceActivation]]) extends Request + + + /** ResultEventListener -> ExtResultDataProvider */ + final case class ResultResponseMessage( + result: ResultEntity, + nextTick: Option[Long] + ) + extends Request + + /** ExtResultDataProvider -> ExtResultDataProvider */ + final case class ResultRequestMessage( + currentTick: Long + ) extends Request + + final case class Create( + initializeStateData: InitExtResultData, + unlockKey: ScheduleKey, + ) extends Request + + // -=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=- + + def apply( + scheduler: ActorRef[SchedulerMessage] + ): Behavior[Request] = Behaviors.withStash(500) { buffer => + Behaviors.setup[Request] { ctx => + //ctx.log.info("Starting initialization!") + val activationAdapter: ActorRef[Activation] = ctx.messageAdapter[Activation](msg => WrappedActivation(msg)) + val resultDataMessageFromExtAdapter: ActorRef[ResultDataMessageFromExt] = ctx.messageAdapter[ResultDataMessageFromExt](msg => WrappedResultDataMessageFromExt(msg)) + val scheduleServiceActivationAdapter: ActorRef[ScheduleServiceActivation] = ctx.messageAdapter[ScheduleServiceActivation](msg => WrappedScheduleServiceActivationAdapter(msg)) + + uninitialized(scheduler, activationAdapter, resultDataMessageFromExtAdapter, scheduleServiceActivationAdapter, buffer) + } + } + + + private def uninitialized( + implicit scheduler: ActorRef[SchedulerMessage], + activationAdapter: ActorRef[Activation], + resultDataMessageFromExtAdapter: ActorRef[ResultDataMessageFromExt], + scheduleServiceActivationAdapter: ActorRef[ScheduleServiceActivation], + buffer: StashBuffer[Request], + ): Behavior[Request] = Behaviors.receiveMessagePartial { + case RequestDataMessageAdapter(sender) => + sender ! resultDataMessageFromExtAdapter + Behaviors.same + + case RequestScheduleActivationAdapter(sender) => + sender ! scheduleServiceActivationAdapter + Behaviors.same + + case Create( + initializeStateData: InitExtResultData, + unlockKey: ScheduleKey, + ) => + println("Send Completion") + scheduler ! ScheduleActivation(activationAdapter, + INIT_SIM_TICK, + Some(unlockKey)) + + initializing(initializeStateData) + } + + private def initializing( + initServiceData: InitExtResultData + )( + implicit scheduler: ActorRef[SchedulerMessage], + activationAdapter: ActorRef[Activation], + resultDataMessageFromExtAdapter: ActorRef[ResultDataMessageFromExt], + buffer: StashBuffer[Request]): Behavior[Request] = { + Behaviors.receivePartial { + case (_, WrappedActivation(Activation(INIT_SIM_TICK))) => + val initSubscribers = initServiceData.extResultData.getResultDataAssets.asScala.toList + + var initResultStorage = Map.empty[UUID, (Option[ResultEntity], Option[Long])] + initSubscribers.foreach( + uuid => initResultStorage = initResultStorage + (uuid -> (None, Some(0L))) + ) + val resultInitializedStateData = ExtResultStateData( + extResultData = initServiceData.extResultData, + subscribers = initSubscribers, + resultStorage = initResultStorage + ) + scheduler ! Completion( + activationAdapter, + None + ) + idle(resultInitializedStateData) + } + + } + + private def idle(serviceStateData: ExtResultStateData)( + implicit scheduler: ActorRef[SchedulerMessage], + activationAdapter: ActorRef[Activation], + resultDataMessageFromExtAdapter: ActorRef[ResultDataMessageFromExt], + buffer: StashBuffer[Request], + ): Behavior[Request] = Behaviors + .receivePartial[Request] { + case (ctx, WrappedActivation(activation: Activation)) => + //info("Received Activation") + var updatedStateData = serviceStateData + serviceStateData.extResultsMessage.getOrElse( + throw ServiceException( + "ExtResultDataService was triggered without ResultDataMessageFromExt available" + ) + ) match { + case msg: RequestResultEntities => + //ctx.log.info(s"[requestResults] for tick ${msg.tick} and resultStorage ${serviceStateData.resultStorage}") + var receiveDataMap = ReceiveDataMap[UUID, ResultEntity](serviceStateData.subscribers.toSet) + //ctx.log.info(s"[requestResults] tick ${msg.tick} -> created a receivedatamap " + receiveDataMap) + serviceStateData.resultStorage.foreach({ + case (uuid, (res, t)) => + //ctx.log.info(s"[requestResults] tick = ${msg.tick}, uuid = $uuid, and time = ${t.getOrElse("Option")}, result = ${res.getOrElse("Option")}") + if (t.getOrElse(-1L) != msg.tick) { //wenn nicht in diesem Tick gefragt, nehme Wert aus ResultDataStorage + receiveDataMap = receiveDataMap.addData( + uuid, + res.getOrElse( + throw new Exception("noResult") + ) + ) + //ctx.log.info(s"[requestResults] tick ${msg.tick} -> added to receivedatamap " + receiveDataMap) + } + }) + + //ctx.log.info(s"[requestResults] tick ${msg.tick} -> requestResults for " + receiveDataMap) + + if (receiveDataMap.isComplete) { + var resultList = List.empty[ResultEntity] + serviceStateData.resultStorage.values.foreach( + result => resultList = resultList :+ result._1.getOrElse( + throw new RuntimeException("There is no result!") + ) + ) + //log.info(s"[requestResults] tick $tick -> ReceiveDataMap is complete -> send it right away: " + resultList) + // all responses received, forward them to external simulation in a bundle + serviceStateData.extResultData.queueExtResponseMsg( + new ProvideResultEntities(resultList.asJava) + ) + updatedStateData = serviceStateData.copy( + extResultsMessage = None, + recentResults = None) + + } else { + ctx.self ! ResultRequestMessage(msg.tick) + updatedStateData = serviceStateData.copy( + extResultsMessage = None, + recentResults = Some(receiveDataMap) + ) + } + } + + scheduler ! Completion(activationAdapter, None) + idle(updatedStateData) + + case (_, scheduleServiceActivationMsg: WrappedScheduleServiceActivationAdapter) => + scheduler ! ScheduleActivation( + activationAdapter, + scheduleServiceActivationMsg.scheduleServiceActivationMsg.tick, + Some(scheduleServiceActivationMsg.scheduleServiceActivationMsg.unlockKey), + ) + Behaviors.same + + case (_, extRequestResultEntitiesMsg: WrappedResultDataMessageFromExt) => + idle( + serviceStateData.copy( + extResultsMessage = Some(extRequestResultEntitiesMsg.extResultDataMessageFromExt) + )) + + case (_, extResultResponseMsg: ResultResponseMessage) => + if (serviceStateData.recentResults.isDefined) { + // process dataResponses + if (serviceStateData.subscribers.contains(extResultResponseMsg.result.getInputModel)) { + //ctx.log.info("[handleDataResponseMessage] Received ResultsResponseMessage with content " + extResultResponseMsg) + //ctx.log.info("[handleDataResponseMessage] RecentResults " + serviceStateData.recentResults) + val updatedReceivedResults = serviceStateData.recentResults.getOrElse(throw new Exception("noMap")).addData(extResultResponseMsg.result.getInputModel, extResultResponseMsg.result) + //ctx.log.info("[handleDataResponseMessage] AddData to RecentResults -> updatedReceivedResults = " + updatedReceivedResults) + val updatedResultStorage = + serviceStateData.resultStorage + (extResultResponseMsg.result.getInputModel -> (Some(extResultResponseMsg.result), extResultResponseMsg.nextTick)) + if (updatedReceivedResults.nonComplete) { + // all responses received, forward them to external simulation in a bundle + idle(serviceStateData.copy( + recentResults = Some(updatedReceivedResults), + resultStorage = updatedResultStorage + )) + } else { + var resultList = List.empty[ResultEntity] + updatedReceivedResults.receivedData.values.foreach( + result => resultList = resultList :+ result + ) + // all responses received, forward them to external simulation in a bundle + serviceStateData.extResultData.queueExtResponseMsg( + new ProvideResultEntities(resultList.asJava) + ) + //log.info("[handleDataResponseMessage] Got all ResultResponseMessage -> Now forward to external simulation in a bundle: " + resultList) + idle(serviceStateData.copy( + resultStorage = updatedResultStorage, + recentResults = None + )) + } + } else { + idle(serviceStateData) + } + } else { + // the results arrived too early -> stash them away + buffer.stash(extResultResponseMsg) + idle(serviceStateData) + } + case (_, msg: ResultRequestMessage) => + buffer.unstashAll(idle(serviceStateData)) + + case (ctx, msg: DelayedStopHelper.StoppingMsg) => + DelayedStopHelper.handleMsg((ctx, msg)) + } + + // -=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=- + /* + protected def announceInformation( + tick: Long + )(implicit + serviceStateData: ExtResultStateData, + activationAdapter: ActorRef[Activation], + resultDataMessageFromExtAdapter: ActorRef[ResultDataMessageFromExt], + buffer: StashBuffer[Request] + ): (ExtResultStateData, Option[Long]) = { + serviceStateData.extResultsMessage.getOrElse( + throw ServiceException( + "ExtResultDataService was triggered without ResultDataMessageFromExt available" + ) + ) match { + case _: RequestResultEntities => + requestResults(tick) + } + } + + private def requestResults( + tick: Long + )(implicit + serviceStateData: ExtResultStateData, + activationAdapter: ActorRef[Activation], + buffer: StashBuffer[Request] + ): (ExtResultStateData, Option[Long]) = { + //log.info(s"[requestResults] for tick $tick and resultStorage ${serviceStateData.resultStorage}") + var receiveDataMap = ReceiveDataMap[UUID, ResultEntity](serviceStateData.subscribers.toSet) + //log.info(s"[requestResults] tick $tick -> created a receivedatamap " + receiveDataMap) + serviceStateData.resultStorage.foreach({ + case (uuid, (res, t)) => + //log.info(s"[requestResults] tick = $tick, uuid = $uuid, and time = ${t.getOrElse("Option")}, result = ${res.getOrElse("Option")}") + if (t.getOrElse(-1) != tick) { //wenn nicht in diesem Tick gefragt, nehme Wert aus ResultDataStorage + receiveDataMap = receiveDataMap.addData( + uuid, + res.getOrElse( + throw new Exception("noResult") + ) + ) + //log.info(s"[requestResults] tick $tick -> added to receivedatamap " + receiveDataMap) + } + }) + + //log.info(s"[requestResults] tick $tick -> requestResults for " + receiveDataMap) + + if (receiveDataMap.isComplete) { + var resultList = List.empty[ResultEntity] + serviceStateData.resultStorage.values.foreach( + result => resultList = resultList :+ result._1.getOrElse( + throw new RuntimeException("There is no result!") + ) + ) + //log.info(s"[requestResults] tick $tick -> ReceiveDataMap is complete -> send it right away: " + resultList) + // all responses received, forward them to external simulation in a bundle + serviceStateData.extResultData.queueExtResponseMsg( + new ProvideResultEntities(resultList.asJava) + ) + (serviceStateData.copy( + extResultsMessage = None, + recentResults = None), None) + + } else { + ( + serviceStateData.copy( + extResultsMessage = None, + recentResults = Some(receiveDataMap) + ), None) + } + } + + */ + + + + + // -=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=- + final case class ExtResultStateData( + extResultData: ExtResultData, + subscribers: List[UUID] = List.empty, + extResultsMessage: Option[ResultDataMessageFromExt] = None, + resultStorage: Map[UUID, (Option[ResultEntity], Option[Long])] = Map.empty, // UUID -> Result, nextTick + maybeNextActivationTick: Option[Long] = None, + recentResults: Option[ReceiveDataMap[UUID, ResultEntity]] = None, + ) + final case class InitExtResultData(extResultData: ExtResultData) +} \ No newline at end of file diff --git a/src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala b/src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala deleted file mode 100644 index 03522ade62..0000000000 --- a/src/main/scala/edu/ie3/simona/service/results/ExtResultDataService.scala +++ /dev/null @@ -1,271 +0,0 @@ -/* - * © 2024. TU Dortmund University, - * Institute of Energy Systems, Energy Efficiency and Energy Economics, - * Research group Distribution grid planning and operation - */ - -package edu.ie3.simona.service.results - -import org.apache.pekko.actor.typed.scaladsl.adapter.ClassicActorRefOps -import edu.ie3.datamodel.models.result.ResultEntity -import edu.ie3.simona.agent.grid.GridAgentMessage -import edu.ie3.simona.api.data.ontology.DataMessageFromExt -import edu.ie3.simona.api.data.results.ExtResultData -import edu.ie3.simona.api.data.results.ontology.{ProvideExpectedAssets, ProvideResultEntities, RequestResultEntities, ResultDataMessageFromExt} -import edu.ie3.simona.exceptions.{InitializationException, ServiceException} -import edu.ie3.simona.ontology.messages.SchedulerMessage.ScheduleActivation -import edu.ie3.simona.ontology.messages.services.DataMessage -import edu.ie3.simona.ontology.messages.services.ResultMessage.{ResultRequestMessage, ResultResponseMessage} -import edu.ie3.simona.ontology.messages.services.ServiceMessage.ServiceRegistrationMessage -import edu.ie3.simona.scheduler.ScheduleLock -import edu.ie3.simona.scheduler.ScheduleLock.ScheduleKey -import edu.ie3.simona.service.ServiceStateData.{InitializeServiceStateData, ServiceBaseStateData} -import edu.ie3.simona.service.results.ExtResultDataService.{ExtResultsStateData, InitExtResultData} -import edu.ie3.simona.service.{ExtDataSupport, SimonaService} -import edu.ie3.simona.util.ReceiveDataMap -import edu.ie3.util.scala.collection.immutable.SortedDistinctSeq -import org.apache.pekko.actor.typed.Behavior -import org.apache.pekko.actor.typed.scaladsl.StashBuffer -import org.apache.pekko.actor.{ActorContext, ActorRef, Props} - -import scala.jdk.CollectionConverters._ -import scala.collection.immutable.{Map, SortedSet} -import java.util.UUID -import scala.jdk.CollectionConverters.SeqHasAsJava -import scala.util.{Failure, Success, Try} - -object ExtResultDataService { - - def props(scheduler: ActorRef): Props = - Props( - new ExtResultDataService(scheduler: ActorRef) - ) - - final case class ExtResultsStateData( - extResultsData: ExtResultData, - subscribers: List[UUID] = List.empty, - extResultsMessage: Option[ResultDataMessageFromExt] = None, - resultStorage: Map[UUID, (Option[ResultEntity], Option[Long])] = Map.empty, // UUID -> Result, nextTick - maybeNextActivationTick: Option[Long] = None, - recentResults: Option[ReceiveDataMap[UUID, ResultEntity]] = None, - ) extends ServiceBaseStateData - - final case class InitExtResultData( - extResultsData: ExtResultData - ) extends InitializeServiceStateData -} - -class ExtResultDataService(override val scheduler: ActorRef) - extends SimonaService[ExtResultsStateData](scheduler) - with ExtDataSupport[ExtResultsStateData] { - - override def init( - initServiceData: InitializeServiceStateData - ): Try[(ExtResultsStateData, Option[Long])] = { - initServiceData match { - case InitExtResultData(extResultData) => - val initSubscribers = extResultData.getResultDataAssets.asScala.toList - - var initResultStorage = Map.empty[UUID, (Option[ResultEntity], Option[Long])] - initSubscribers.foreach( - uuid => initResultStorage = initResultStorage + (uuid -> (None, Some(0L))) - ) - /* - val initSubscribers = List( - UUID.fromString("de8cfef5-7620-4b9e-9a10-1faebb5a80c0"), - UUID.fromString("2560c371-f420-4c2a-b4e6-e04c11b64c03")) - - val resultInitializedStateData = ExtResultsStateData( - extResultsData = extResultsData, - subscribers = initSubscribers, - resultStorage = Map( - UUID.fromString("de8cfef5-7620-4b9e-9a10-1faebb5a80c0") -> (None, Some(0)), - UUID.fromString("2560c371-f420-4c2a-b4e6-e04c11b64c03") -> (None, Some(0)), - ), - recentResults = Option.empty[ReceiveDataMap[UUID, ResultEntity]] - ) - stash() - */ - val resultInitializedStateData = ExtResultsStateData( - extResultsData = extResultData, - subscribers = initSubscribers, - resultStorage = initResultStorage - ) - Success(resultInitializedStateData, None) - - case invalidData => - Failure( - new InitializationException( - s"Provided init data '${invalidData.getClass.getSimpleName}' for EV service are invalid!" - ) - ) - } - } - - override protected def handleRegistrationRequest( - registrationMessage: ServiceRegistrationMessage - )(implicit serviceStateData: ExtResultsStateData): Try[ExtResultsStateData] = - Failure( - ServiceException( - "For this service is no registration possible!" - ) - ) - - override protected def announceInformation( - tick: Long - )(implicit - serviceStateData: ExtResultsStateData, - ctx: ActorContext, - ): (ExtResultsStateData, Option[Long]) = { - serviceStateData.extResultsMessage.getOrElse( - throw ServiceException( - "ExtResultDataService was triggered without ResultDataMessageFromExt available" - ) - ) match { - case _: RequestResultEntities => - requestResults(tick) - } - } - - /** Handle a message from outside the simulation - * - * @param extMsg - * the external incoming message - * @param serviceStateData - * the current state data of this service - * @return - * the updated state data - */ - override protected def handleDataMessage( - extMsg: DataMessageFromExt - )(implicit - serviceStateData: ExtResultsStateData - ): ExtResultsStateData = - extMsg match { - /* - case extExpectedAssets: ProvideExpectedAssets => - var initResultStorage = Map.empty[UUID, (Option[ResultEntity], Option[Long])] - extExpectedAssets.expectedAssets.forEach( - uuid => initResultStorage = initResultStorage + (uuid -> (None, Some(0L))) - ) - serviceStateData.copy( - subscribers = extExpectedAssets.expectedAssets.asScala.toList - ) - */ - case extResultsMessageFromExt: RequestResultEntities => - serviceStateData.copy( - extResultsMessage = Some(extResultsMessageFromExt) - ) - } - - /** Handle a message from inside SIMONA sent to external - * - * @param extResponseMsg - * the external incoming message - * @param serviceStateData - * the current state data of this service - * @return - * the updated state data - */ - override protected def handleDataResponseMessage( - extResponseMsg: DataMessage - )(implicit serviceStateData: ExtResultsStateData): ExtResultsStateData = { - extResponseMsg match { - case ResultResponseMessage(result, nextTick) => - if (serviceStateData.recentResults.isDefined) { - // process dataResponses - if (serviceStateData.subscribers.contains(result.getInputModel)) { - //log.info("[handleDataResponseMessage] Received ResultsResponseMessage with content " + extResponseMsg) - //log.info("[handleDataResponseMessage] RecentResults " + serviceStateData.recentResults) - val updatedReceivedResults = serviceStateData.recentResults.getOrElse(throw new Exception("noMap")).addData(result.getInputModel, result) - //log.info("[handleDataResponseMessage] AddData to RecentResults -> updatedReceivedResults = " + updatedReceivedResults) - val updatedResultStorage = - serviceStateData.resultStorage + (result.getInputModel -> (Some(result), nextTick)) - if (updatedReceivedResults.nonComplete) { - // all responses received, forward them to external simulation in a bundle - serviceStateData.copy( - recentResults = Some(updatedReceivedResults), - resultStorage = updatedResultStorage - ) - } else { - var resultList = List.empty[ResultEntity] - updatedReceivedResults.receivedData.values.foreach( - result => resultList = resultList :+ result - ) - // all responses received, forward them to external simulation in a bundle - serviceStateData.extResultsData.queueExtResponseMsg( - new ProvideResultEntities(resultList.asJava) - ) - //log.info("[handleDataResponseMessage] Got all ResultResponseMessage -> Now forward to external simulation in a bundle: " + resultList) - serviceStateData.copy( - resultStorage = updatedResultStorage, - recentResults = None - ) - } - } else { - serviceStateData - } - } else { - // the results arrived too early -> stash them away - stash() - serviceStateData - } - case ResultRequestMessage(tick) => - // now the expected results are calculated -> - unstashAll() - serviceStateData - } - } - - private def requestResults( - tick: Long - )(implicit - serviceStateData: ExtResultsStateData - ): (ExtResultsStateData, Option[Long]) = { - //log.info(s"[requestResults] for tick $tick and resultStorage ${serviceStateData.resultStorage}") - var receiveDataMap = ReceiveDataMap[UUID, ResultEntity](serviceStateData.subscribers.toSet) - //log.info(s"[requestResults] tick $tick -> created a receivedatamap " + receiveDataMap) - serviceStateData.resultStorage.foreach({ - case (uuid, (res, t)) => - //log.info(s"[requestResults] tick = $tick, uuid = $uuid, and time = ${t.getOrElse("Option")}, result = ${res.getOrElse("Option")}") - if (t.getOrElse(-1) != tick) { //wenn nicht in diesem Tick gefragt, nehme Wert aus ResultDataStorage - receiveDataMap = receiveDataMap.addData( - uuid, - res.getOrElse( - throw new Exception("noResult") - ) - ) - //log.info(s"[requestResults] tick $tick -> added to receivedatamap " + receiveDataMap) - } - }) - - //log.info(s"[requestResults] tick $tick -> requestResults for " + receiveDataMap) - - if (receiveDataMap.isComplete) { - var resultList = List.empty[ResultEntity] - serviceStateData.resultStorage.values.foreach( - result => resultList = resultList :+ result._1.getOrElse( - throw new RuntimeException("There is no result!") - ) - ) - //log.info(s"[requestResults] tick $tick -> ReceiveDataMap is complete -> send it right away: " + resultList) - // all responses received, forward them to external simulation in a bundle - serviceStateData.extResultsData.queueExtResponseMsg( - new ProvideResultEntities(resultList.asJava) - ) - (serviceStateData.copy( - extResultsMessage = None, - recentResults = None), None) - - } else { - // We wait for result requests! - self ! ResultRequestMessage(tick) - ( - serviceStateData.copy( - extResultsMessage = None, - recentResults = Some(receiveDataMap) - ), None) - } - } -} - diff --git a/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala b/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala index 35c4fc2029..c01742c38e 100644 --- a/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala +++ b/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala @@ -12,7 +12,6 @@ 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.service.results.ExtResultDataService import edu.ie3.simona.sim.setup.{ExtSimSetupData, SimonaSetup} import edu.ie3.util.scala.Scope import org.apache.pekko.actor.typed.scaladsl.adapter._ @@ -131,25 +130,18 @@ object SimonaSim { primaryServiceProxy.toTyped, weatherService.toTyped, ) ++ - gridAgents ++ - extSimulationData.extDataServices.values.map(_.toTyped) + gridAgents - /* val delayedActors = resultEventListeners.appended(runtimeEventListener).appended(extSimulationData.extResultDataService.getOrElse( throw new Exception("Problem!") ).toTyped[DelayedStopHelper.StoppingMsg]) - - ctx.log.info(s"delayedActors: $delayedActors") - */ - - idle( ActorData( starter, extSimulationData.extSimAdapters, runtimeEventListener, - resultEventListeners.appended(runtimeEventListener), + delayedActors, otherActors, ) ) @@ -217,13 +209,10 @@ object SimonaSim { ref ! ExtSimAdapter.Stop(simulationSuccessful) } - ctx.log.info(s"delayedActors: ${actorData.delayedStoppingActors}") - // if the simulation is successful, we're waiting for the delayed // stopping listeners to terminate and thus do not unwatch them here actorData.delayedStoppingActors.foreach( actor => { - ctx.log.info(s"send FlushAndStop to $actor") actor ! DelayedStopHelper.FlushAndStop } ) 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 07cef84d02..6e2299ea12 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,14 @@ package edu.ie3.simona.sim.setup -import org.apache.pekko.actor.ActorRef -import edu.ie3.simona.service.ev.ExtEvDataService -import edu.ie3.simona.service.primary.ExtPrimaryDataService -import edu.ie3.simona.service.results.ExtResultDataService import edu.ie3.simona.api.data.ExtData +import edu.ie3.simona.api.data.ev.ExtEvData import edu.ie3.simona.api.data.primarydata.ExtPrimaryData import edu.ie3.simona.api.data.results.ExtResultData -import edu.ie3.simona.api.data.ev.ExtEvData +import edu.ie3.simona.service.ev.ExtEvDataService +import edu.ie3.simona.service.primary.ExtPrimaryDataService +import edu.ie3.simona.service.results.ExtResultDataProvider +import org.apache.pekko.actor.ActorRef final case class ExtSimSetupData( extSimAdapters: Iterable[ActorRef], @@ -28,7 +28,7 @@ final case class ExtSimSetupData( extDataServices.get(classOf[ExtPrimaryDataService]) def extResultDataService: Option[ActorRef] = - extDataServices.get(classOf[ExtResultDataService]) + extDataServices.get(ExtResultDataProvider.getClass) def extEvData: Option[ExtEvData] = { extDatas.collectFirst { case extData: ExtEvData => extData } 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..cb5d40037d 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SetupHelper.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SetupHelper.scala @@ -24,8 +24,8 @@ import edu.ie3.simona.model.grid.RefSystem import edu.ie3.simona.util.ConfigUtil.{GridOutputConfigUtil, OutputConfigUtil} import edu.ie3.simona.util.ResultFileHierarchy.ResultEntityPathConfig import edu.ie3.simona.util.{EntityMapperUtil, ResultFileHierarchy} -import org.apache.pekko.actor.typed.ActorRef import edu.ie3.util.quantities.PowerSystemUnits +import org.apache.pekko.actor.typed.ActorRef import squants.electro.Kilovolts /** Methods to support the setup of a simona simulation 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 5bbf087803..a1dec14709 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala @@ -15,12 +15,11 @@ import edu.ie3.simona.actor.SimonaActorNaming.RichActorRefFactory import edu.ie3.simona.agent.EnvironmentRefs import edu.ie3.simona.agent.grid.GridAgentMessage.CreateGridAgent import edu.ie3.simona.agent.grid.{GridAgent, GridAgentMessage} -import edu.ie3.simona.api.{ExtLinkInterface, ExtSimAdapter} +import edu.ie3.simona.api.ExtSimAdapter import edu.ie3.simona.api.data.ExtData -import edu.ie3.simona.api.data.ExtDataSimulation -import edu.ie3.simona.api.data.ev.{ExtEvData, ExtEvSimulation} -import edu.ie3.simona.api.data.primarydata.{ExtPrimaryData, ExtPrimaryDataSimulation} -import edu.ie3.simona.api.data.results.{ExtResultData, ExtResultDataSimulation} +import edu.ie3.simona.api.data.primarydata.ExtPrimaryData +import edu.ie3.simona.api.data.results.ExtResultData +import edu.ie3.simona.api.data.results.ontology.ResultDataMessageFromExt import edu.ie3.simona.api.simulation.ExtSimAdapterData import edu.ie3.simona.config.{ArgsParser, RefSystemParser, SimonaConfig} import edu.ie3.simona.event.listener.{ResultEventListener, RuntimeEventListener} @@ -29,15 +28,14 @@ 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.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.ScheduleServiceActivation import edu.ie3.simona.scheduler.{ScheduleLock, Scheduler, TimeAdvancer} import edu.ie3.simona.service.SimonaService -import edu.ie3.simona.service.ev.ExtEvDataService -import edu.ie3.simona.service.ev.ExtEvDataService.InitExtEvData import edu.ie3.simona.service.primary.ExtPrimaryDataService.InitExtPrimaryData -import edu.ie3.simona.service.primary.{ExtPrimaryDataService, PrimaryServiceProxy} import edu.ie3.simona.service.primary.PrimaryServiceProxy.InitPrimaryServiceProxyStateData -import edu.ie3.simona.service.results.ExtResultDataService -import edu.ie3.simona.service.results.ExtResultDataService.InitExtResultData +import edu.ie3.simona.service.primary.{ExtPrimaryDataService, PrimaryServiceProxy} +import edu.ie3.simona.service.results.ExtResultDataProvider +import edu.ie3.simona.service.results.ExtResultDataProvider.{InitExtResultData, RequestDataMessageAdapter, RequestScheduleActivationAdapter, WrappedResultDataMessageFromExt} import edu.ie3.simona.service.weather.WeatherService import edu.ie3.simona.service.weather.WeatherService.InitWeatherServiceStateData import edu.ie3.simona.sim.SimonaSim @@ -45,15 +43,19 @@ import edu.ie3.simona.util.ResultFileHierarchy import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK import edu.ie3.simona.util.TickUtil.RichZonedDateTime import edu.ie3.util.TimeUtil -import org.apache.pekko.actor.typed.ActorRef +import edu.ie3.simpleextsim.SimpleExtSimulation +import org.apache.pekko.actor.typed.{ActorRef, Scheduler} import org.apache.pekko.actor.typed.scaladsl.ActorContext import org.apache.pekko.actor.typed.scaladsl.adapter.{ClassicActorRefOps, TypedActorContextOps, TypedActorRefOps} import org.apache.pekko.actor.{ActorRef => ClassicRef} - -import edu.ie3.simpleextsim.SimpleExtSimulation +import org.apache.pekko.actor.typed.scaladsl.AskPattern._ +import org.apache.pekko.util.{Timeout => PekkoTimeout} import java.util.concurrent.LinkedBlockingQueue +import scala.concurrent.{Await, ExecutionContext} import scala.jdk.CollectionConverters._ +import scala.concurrent.duration.DurationInt +import scala.jdk.DurationConverters._ /** Sample implementation to run a standalone simulation of simona configured * with the provided [[SimonaConfig]] and [[ResultFileHierarchy]] @@ -194,12 +196,12 @@ class SimonaStandaloneSetup( override def extSimulations( context: ActorContext[_], - scheduler: ActorRef[SchedulerMessage], + theScheduler: ActorRef[SchedulerMessage], ): ExtSimSetupData = { val simpleExtSim = new SimpleExtSimulation() val extSimAdapter = context.toClassic.simonaActorOf( - ExtSimAdapter.props(scheduler.toClassic), + ExtSimAdapter.props(theScheduler.toClassic), s"0", ) val extSimAdapterData = new ExtSimAdapterData(extSimAdapter, args) @@ -207,11 +209,11 @@ class SimonaStandaloneSetup( // send init data right away, init activation is scheduled extSimAdapter ! ExtSimAdapter.Create( extSimAdapterData, - ScheduleLock.singleKey(context, scheduler, INIT_SIM_TICK), + ScheduleLock.singleKey(context, theScheduler, INIT_SIM_TICK), ) val extPrimaryDataService = context.toClassic.simonaActorOf( - ExtPrimaryDataService.props(scheduler.toClassic), + ExtPrimaryDataService.props(theScheduler.toClassic), s"0-0", ) val extPrimaryData = new ExtPrimaryData( @@ -225,28 +227,43 @@ class SimonaStandaloneSetup( InitExtPrimaryData(extPrimaryData), ScheduleLock.singleKey( context, - scheduler, + theScheduler, INIT_SIM_TICK, ), ) - val extResultDataService = context.toClassic.simonaActorOf( - ExtResultDataService.props(scheduler.toClassic), - s"0-1", - ) + + val extResultDataProvider = { + context.spawn( + ExtResultDataProvider(theScheduler), + s"ExtResultDataProvider", + ) + } + + implicit val timeout: PekkoTimeout = PekkoTimeout.create(5.seconds.toJava) + implicit val scheduler: Scheduler = context.system.scheduler + + val adapterRef = Await.result( + extResultDataProvider.ask[ActorRef[ResultDataMessageFromExt]] (ref => RequestDataMessageAdapter(ref)), timeout.duration) + val adapterScheduleRef = Await.result( + extResultDataProvider.ask[ActorRef[ScheduleServiceActivation]] (ref => RequestScheduleActivationAdapter(ref)), timeout.duration) + val extResultData = new ExtResultData( - extResultDataService, + adapterRef.toClassic, + adapterScheduleRef.toClassic, extSimAdapter, simpleExtSim.getResultDataFactory, simpleExtSim.getResultDataAssets ) - extResultDataService ! SimonaService.Create( + context.log.info(s"extResultData $extResultData") + + extResultDataProvider ! ExtResultDataProvider.Create( InitExtResultData(extResultData), ScheduleLock.singleKey( context, - scheduler, + theScheduler, INIT_SIM_TICK, ), ) @@ -264,7 +281,7 @@ class SimonaStandaloneSetup( val extDataServicesMap: Map[Class[_], ClassicRef] = Map( classOf[ExtPrimaryDataService] -> extPrimaryDataService, - classOf[ExtResultDataService] -> extResultDataService + ExtResultDataProvider.getClass -> extResultDataProvider.toClassic ) val extSimAdapters = Iterable(extSimAdapter) diff --git a/src/main/scala/edu/ie3/util/scala/io/GraphicDataCleaner.scala b/src/main/scala/edu/ie3/util/scala/io/GraphicDataCleaner.scala index 0e1515df69..6d44b34211 100644 --- a/src/main/scala/edu/ie3/util/scala/io/GraphicDataCleaner.scala +++ b/src/main/scala/edu/ie3/util/scala/io/GraphicDataCleaner.scala @@ -8,8 +8,8 @@ package edu.ie3.util.scala.io import edu.ie3.datamodel.io.naming.FileNamingStrategy import edu.ie3.datamodel.io.sink.CsvFileSink -import edu.ie3.datamodel.io.source.{GraphicSource, RawGridSource, TypeSource} import edu.ie3.datamodel.io.source.csv.CsvDataSource +import edu.ie3.datamodel.io.source.{GraphicSource, RawGridSource, TypeSource} import java.nio.file.Paths diff --git a/src/main/scala/edu/ie3/util/scala/io/ScalaReflectionSerde.scala b/src/main/scala/edu/ie3/util/scala/io/ScalaReflectionSerde.scala index f7d6707254..b1bd6043ef 100644 --- a/src/main/scala/edu/ie3/util/scala/io/ScalaReflectionSerde.scala +++ b/src/main/scala/edu/ie3/util/scala/io/ScalaReflectionSerde.scala @@ -7,10 +7,7 @@ package edu.ie3.util.scala.io import com.sksamuel.avro4s.RecordFormat -import io.confluent.kafka.streams.serdes.avro.{ - GenericAvroDeserializer, - GenericAvroSerializer, -} +import io.confluent.kafka.streams.serdes.avro.{GenericAvroDeserializer, GenericAvroSerializer} import org.apache.kafka.common.serialization.{Deserializer, Serializer} /** As seen at 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..68c66a2c93 100644 --- a/src/main/scala/edu/ie3/util/scala/quantities/DefaultQuantities.scala +++ b/src/main/scala/edu/ie3/util/scala/quantities/DefaultQuantities.scala @@ -6,8 +6,8 @@ package edu.ie3.util.scala.quantities -import squants.{Each, Energy, Power, Dimensionless} import squants.energy.{KilowattHours, Kilowatts, Megawatts} +import squants.{Dimensionless, Each, Energy, Power} object DefaultQuantities { diff --git a/src/main/scala/edu/ie3/util/scala/quantities/QuantityUtil.scala b/src/main/scala/edu/ie3/util/scala/quantities/QuantityUtil.scala index 4ca6d188fb..67ceec5bd7 100644 --- a/src/main/scala/edu/ie3/util/scala/quantities/QuantityUtil.scala +++ b/src/main/scala/edu/ie3/util/scala/quantities/QuantityUtil.scala @@ -7,7 +7,6 @@ package edu.ie3.util.scala.quantities import edu.ie3.simona.exceptions.QuantityException -import edu.ie3.util.quantities.{QuantityUtil => PSQuantityUtil} import squants.time.{Hours, TimeDerivative, TimeIntegral} import squants.{Quantity, Seconds, UnitOfMeasure} import tech.units.indriya.ComparableQuantity diff --git a/src/main/scala/edu/ie3/util/scala/quantities/ReactivePower.scala b/src/main/scala/edu/ie3/util/scala/quantities/ReactivePower.scala index aaddb36edf..ebaf0f702a 100644 --- a/src/main/scala/edu/ie3/util/scala/quantities/ReactivePower.scala +++ b/src/main/scala/edu/ie3/util/scala/quantities/ReactivePower.scala @@ -6,9 +6,9 @@ package edu.ie3.util.scala.quantities +import squants._ import squants.energy._ import squants.time.{Hours, Time, TimeIntegral} -import squants._ import scala.util.Try diff --git a/src/test/scala/edu/ie3/simona/test/common/service/PrimaryDataFactoryDefault.scala b/src/test/scala/edu/ie3/simona/test/common/service/PrimaryDataFactoryDefault.scala index e3d0bc0714..ee50f5356d 100644 --- a/src/test/scala/edu/ie3/simona/test/common/service/PrimaryDataFactoryDefault.scala +++ b/src/test/scala/edu/ie3/simona/test/common/service/PrimaryDataFactoryDefault.scala @@ -14,5 +14,5 @@ class PrimaryDataFactoryDefault extends PrimaryDataFactory { /** Should convert an object to an primary data value with a check if the * object is primary data */ - override def convertObjectToValue(entity: Any): Value = null + override def convert(entity: Any): Value = null } From ab38e48c12914f3879ad55791311fc6bf703820c Mon Sep 17 00:00:00 2001 From: Johannes Bao Date: Fri, 15 Mar 2024 18:50:48 +0100 Subject: [PATCH 16/41] imports --- build.gradle | 1 - .../ie3/simona/sim/setup/SimonaStandaloneSetup.scala | 12 ++++++------ 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/build.gradle b/build.gradle index 4b9c876b3c..f22406b854 100644 --- a/build.gradle +++ b/build.gradle @@ -103,7 +103,6 @@ dependencies { implementation('com.github.ie3-institute:simosaik') { exclude group: 'org.apache.logging.log4j' exclude group: 'org.slf4j' - /* Exclude our own nested dependencies */ exclude group: 'edu.ie3' } 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 a1dec14709..73ef760563 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala @@ -35,26 +35,26 @@ import edu.ie3.simona.service.primary.ExtPrimaryDataService.InitExtPrimaryData import edu.ie3.simona.service.primary.PrimaryServiceProxy.InitPrimaryServiceProxyStateData import edu.ie3.simona.service.primary.{ExtPrimaryDataService, PrimaryServiceProxy} import edu.ie3.simona.service.results.ExtResultDataProvider -import edu.ie3.simona.service.results.ExtResultDataProvider.{InitExtResultData, RequestDataMessageAdapter, RequestScheduleActivationAdapter, WrappedResultDataMessageFromExt} +import edu.ie3.simona.service.results.ExtResultDataProvider.{InitExtResultData, RequestDataMessageAdapter, RequestScheduleActivationAdapter} import edu.ie3.simona.service.weather.WeatherService import edu.ie3.simona.service.weather.WeatherService.InitWeatherServiceStateData import edu.ie3.simona.sim.SimonaSim import edu.ie3.simona.util.ResultFileHierarchy import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK import edu.ie3.simona.util.TickUtil.RichZonedDateTime +import edu.ie3.simosaik.simpleextsim.SimpleExtSimulation import edu.ie3.util.TimeUtil -import edu.ie3.simpleextsim.SimpleExtSimulation -import org.apache.pekko.actor.typed.{ActorRef, Scheduler} import org.apache.pekko.actor.typed.scaladsl.ActorContext +import org.apache.pekko.actor.typed.scaladsl.AskPattern._ import org.apache.pekko.actor.typed.scaladsl.adapter.{ClassicActorRefOps, TypedActorContextOps, TypedActorRefOps} +import org.apache.pekko.actor.typed.{ActorRef, Scheduler} import org.apache.pekko.actor.{ActorRef => ClassicRef} -import org.apache.pekko.actor.typed.scaladsl.AskPattern._ import org.apache.pekko.util.{Timeout => PekkoTimeout} import java.util.concurrent.LinkedBlockingQueue -import scala.concurrent.{Await, ExecutionContext} -import scala.jdk.CollectionConverters._ +import scala.concurrent.Await import scala.concurrent.duration.DurationInt +import scala.jdk.CollectionConverters._ import scala.jdk.DurationConverters._ /** Sample implementation to run a standalone simulation of simona configured From fdbcef88170184da4ae3cc320bd162f1dd02e69c Mon Sep 17 00:00:00 2001 From: Johannes Bao Date: Fri, 15 Mar 2024 23:50:21 +0100 Subject: [PATCH 17/41] simplify --- .../agent/participant/ParticipantAgent.scala | 7 -- .../ParticipantAgentFundamentals.scala | 74 +------------------ .../edu/ie3/simona/event/ResultEvent.scala | 3 +- .../event/listener/DelayedStopHelper.scala | 2 +- .../event/listener/ResultEventListener.scala | 4 - .../edu/ie3/simona/io/grid/GridProvider.scala | 2 - .../ontology/messages/SchedulerMessage.scala | 3 - .../messages/services/ResultMessage.scala | 17 ----- .../messages/services/ServiceMessage.scala | 12 +-- .../edu/ie3/simona/scheduler/Scheduler.scala | 4 +- .../ie3/simona/service/ExtDataSupport.scala | 15 +--- .../ie3/simona/service/SimonaService.scala | 24 +----- .../primary/ExtPrimaryDataService.scala | 12 +-- .../results/ExtResultDataProvider.scala | 10 +-- .../scala/edu/ie3/simona/sim/SimonaSim.scala | 4 +- .../ie3/simona/sim/setup/SetupHelper.scala | 2 +- .../edu/ie3/simona/util/ReceiveDataMap.scala | 17 +---- 17 files changed, 20 insertions(+), 192 deletions(-) delete mode 100644 src/main/scala/edu/ie3/simona/ontology/messages/services/ResultMessage.scala diff --git a/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgent.scala b/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgent.scala index c94d0db8c6..a01c5677a1 100644 --- a/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgent.scala +++ b/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgent.scala @@ -44,7 +44,6 @@ import edu.ie3.simona.model.participant.{ SystemParticipant, } import edu.ie3.simona.ontology.messages.Activation - import edu.ie3.simona.ontology.messages.PowerMessage.RequestAssetPowerMessage import edu.ie3.simona.ontology.messages.SchedulerMessage.ScheduleActivation import edu.ie3.simona.ontology.messages.flex.FlexibilityMessage.{ @@ -802,12 +801,6 @@ abstract class ParticipantAgent[ setPower: squants.Power, ): (MS, PD, FlexChangeIndicator) - def answerResultRequestAndStayWithUpdatedStateData( - baseStateData: BaseStateData[PD], - requestTick: Long, - alternativeResult: PD, - ): FSM.State[AgentState, ParticipantStateData[PD]] - /** Determining the reply to an * [[edu.ie3.simona.ontology.messages.PowerMessage.RequestAssetPowerMessage]], * send this answer and stay in the current state. If no reply can be 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 f10e39f54c..deb48f8ba4 100644 --- a/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgentFundamentals.scala +++ b/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgentFundamentals.scala @@ -1196,77 +1196,6 @@ protected trait ParticipantAgentFundamentals[ } } - override def answerResultRequestAndStayWithUpdatedStateData( - baseStateData: BaseStateData[PD], - requestTick: Long, - alternativeResult: PD): FSM.State[AgentState, ParticipantStateData[PD]] = { - /* Check, if there is any calculation foreseen for this tick. If so, wait with the response. */ - val activationExpected = - baseStateData.additionalActivationTicks.headOption.exists(_ < requestTick) - val dataExpected = - baseStateData.foreseenDataTicks.values.flatten.exists(_ < requestTick) - if (activationExpected || dataExpected) { - log.debug( - s"Received power request from '{}' for tick '{}', but I'm still waiting for new results before " + - s"this tick. Waiting with the response.", - sender(), - requestTick, - ) - stash() - stay() using baseStateData - } else { - /* Update the voltage value store */ - val lastNodalVoltage = - baseStateData.voltageValueStore.last(requestTick) - - /* Determine the most recent request */ - val mostRecentRequest = - baseStateData.requestValueStore.last(requestTick) - - /* === Check if this request has already been answered with same tick and nodal voltage === */ - determineFastReplyForResults( - baseStateData, - mostRecentRequest, - requestTick, - ) - } - - - - - - - - - } - - - def determineFastReplyForResults( - baseStateData: BaseStateData[PD], - mostRecentRequest: Option[(Long, PD)], - requestTick: Long, - ): FSM.State[AgentState, ParticipantStateData[PD]] = { - mostRecentRequest match { - case Some((mostRecentRequestTick, latestProvidedValues)) - if mostRecentRequestTick == requestTick => - /* A request for this tick has already been answered. Check, if it has been the same request. - * if it has been the same request we wanna answer with the same values afterwards, this data MUST always - * be available when we already provided data for this tick */ - baseStateData match { - case externalBaseStateData: FromOutsideBaseStateData[M, PD] => - /* When data is provided from outside it is NOT altered depending on the node voltage. Send an - * AssetPowerUnchangedMessage */ - stay() using externalBaseStateData.copy() replying AssetPowerUnchangedMessage( - latestProvidedValues.p, - latestProvidedValues.q, - ) - - } - } - } - - - /** Determining the reply to an * [[edu.ie3.simona.ontology.messages.PowerMessage.RequestAssetPowerMessage]], * send this answer and stay in the current state. If no reply can be @@ -1866,8 +1795,7 @@ protected trait ParticipantAgentFundamentals[ result: AccompaniedSimulationResult[PD], )(implicit outputConfig: NotifierConfig): Unit = { if (outputConfig.simulationResultInfo) { - var (nextTick, _) = popNextActivationTrigger(baseStateData) - + val (nextTick, _) = popNextActivationTrigger(baseStateData) notifyListener( buildResultEvent(baseStateData, tick, result.primaryData, nextTick) ) diff --git a/src/main/scala/edu/ie3/simona/event/ResultEvent.scala b/src/main/scala/edu/ie3/simona/event/ResultEvent.scala index 4a8720304a..13a23eb499 100644 --- a/src/main/scala/edu/ie3/simona/event/ResultEvent.scala +++ b/src/main/scala/edu/ie3/simona/event/ResultEvent.scala @@ -6,7 +6,7 @@ package edu.ie3.simona.event -import edu.ie3.datamodel.models.result.{NodeResult, ResultEntity} +import edu.ie3.datamodel.models.result.NodeResult import edu.ie3.datamodel.models.result.connector.{ LineResult, SwitchResult, @@ -78,4 +78,5 @@ object ResultEvent { final case class FlexOptionsResultEvent( flexOptionsResult: FlexOptionsResult ) extends ResultEvent + } diff --git a/src/main/scala/edu/ie3/simona/event/listener/DelayedStopHelper.scala b/src/main/scala/edu/ie3/simona/event/listener/DelayedStopHelper.scala index 3782ea7213..3c3a84f3ca 100644 --- a/src/main/scala/edu/ie3/simona/event/listener/DelayedStopHelper.scala +++ b/src/main/scala/edu/ie3/simona/event/listener/DelayedStopHelper.scala @@ -40,7 +40,7 @@ object DelayedStopHelper { case (ctx, FlushAndStop) => ctx.log.debug( - s"$this Received FlushAndStop message, shutting down once no message has been received for 5 seconds." + s"Received FlushAndStop message, shutting down once no message has been received for 5 seconds." ) ctx.setReceiveTimeout(5.seconds, StopTimeout) Behaviors.same diff --git a/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala b/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala index aa5907646d..1a3fe0ea9e 100644 --- a/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala +++ b/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala @@ -165,7 +165,6 @@ object ResultEventListener extends Transformer3wResultSupport { handOverToExternalService( resultEntity, baseData.extResultDataService, - log, nextTick ) } @@ -246,15 +245,12 @@ object ResultEventListener extends Transformer3wResultSupport { private def handOverToExternalService( resultEntity: ResultEntity, extResultDataService: Option[ActorRef], - log: Logger, nextTick: Option[Long] = None ): Unit = Try { val extResultDataServiceRef = extResultDataService.getOrElse( throw new Exception("No external data service registered!") ) - //log.info(s"Sending a ResultResponseMessage to $extResultDataServiceRef $resultEntity") extResultDataServiceRef ! ResultResponseMessage(resultEntity, nextTick) - //log.info(s"Sended a ResultResponseMessage to $extResultDataServiceRef $resultEntity") } def apply( diff --git a/src/main/scala/edu/ie3/simona/io/grid/GridProvider.scala b/src/main/scala/edu/ie3/simona/io/grid/GridProvider.scala index b99a3f6cad..3dda241c4d 100644 --- a/src/main/scala/edu/ie3/simona/io/grid/GridProvider.scala +++ b/src/main/scala/edu/ie3/simona/io/grid/GridProvider.scala @@ -16,8 +16,6 @@ import edu.ie3.datamodel.models.input.container.{ JointGridContainer, ThermalGrid, } -import edu.ie3.datamodel.io.source.csv.CsvJointGridContainerSource -import edu.ie3.datamodel.models.input.container.{JointGridContainer, ThermalGrid} import edu.ie3.datamodel.models.input.thermal.ThermalBusInput import edu.ie3.datamodel.utils.validation.ValidationUtils import edu.ie3.simona.config.SimonaConfig diff --git a/src/main/scala/edu/ie3/simona/ontology/messages/SchedulerMessage.scala b/src/main/scala/edu/ie3/simona/ontology/messages/SchedulerMessage.scala index 8d2a70d035..443971ef18 100644 --- a/src/main/scala/edu/ie3/simona/ontology/messages/SchedulerMessage.scala +++ b/src/main/scala/edu/ie3/simona/ontology/messages/SchedulerMessage.scala @@ -6,7 +6,6 @@ package edu.ie3.simona.ontology.messages -import edu.ie3.datamodel.models.result.ResultEntity import org.apache.pekko.actor.typed.ActorRef import edu.ie3.simona.scheduler.ScheduleLock.ScheduleKey import edu.ie3.simona.scheduler.{Scheduler, TimeAdvancer} @@ -25,6 +24,4 @@ object SchedulerMessage { unlockKey: Option[ScheduleKey] = None, ) extends SchedulerMessage - final case class ResultForExtCompleteMessage(results: Iterable[ResultEntity]) - extends SchedulerMessage } diff --git a/src/main/scala/edu/ie3/simona/ontology/messages/services/ResultMessage.scala b/src/main/scala/edu/ie3/simona/ontology/messages/services/ResultMessage.scala deleted file mode 100644 index a2c56bf46c..0000000000 --- a/src/main/scala/edu/ie3/simona/ontology/messages/services/ResultMessage.scala +++ /dev/null @@ -1,17 +0,0 @@ -/* - * © 2024. TU Dortmund University, - * Institute of Energy Systems, Energy Efficiency and Energy Economics, - * Research group Distribution grid planning and operation - */ - -package edu.ie3.simona.ontology.messages.services - -import edu.ie3.datamodel.models.result.ResultEntity -import edu.ie3.simona.ontology.messages.PowerMessage - -sealed trait ResultMessage extends DataMessage - -object ResultMessage { - final case class ResultResponseMessageIdea() extends ResultMessage -} - diff --git a/src/main/scala/edu/ie3/simona/ontology/messages/services/ServiceMessage.scala b/src/main/scala/edu/ie3/simona/ontology/messages/services/ServiceMessage.scala index 015828b485..53ede35f10 100644 --- a/src/main/scala/edu/ie3/simona/ontology/messages/services/ServiceMessage.scala +++ b/src/main/scala/edu/ie3/simona/ontology/messages/services/ServiceMessage.scala @@ -10,9 +10,7 @@ import org.apache.pekko.actor.ActorRef import java.util.UUID import edu.ie3.simona.agent.participant.data.Data -import edu.ie3.simona.ontology.messages.services import edu.ie3.simona.scheduler.ScheduleLock.ScheduleKey -import org.apache.pekko.util.Timeout /** Collections of all messages, that are send to and from the different * services @@ -21,8 +19,7 @@ sealed trait ServiceMessage case object ServiceMessage { - final case class RequestExtPrimaryDataAssets() extends ServiceMessage { - } + final case class RequestExtPrimaryDataAssets() extends ServiceMessage {} /** Message used to register for a service */ @@ -51,8 +48,6 @@ case object ServiceMessage { requestingActor: ActorRef, ) extends ServiceRegistrationMessage - final case class ExtPrimaryDataServiceInitRequestMessage() extends ServiceRegistrationMessage - sealed trait RegistrationResponseMessage extends ServiceMessage { val serviceRef: ActorRef } @@ -76,11 +71,6 @@ case object ServiceMessage { tick: Long, unlockKey: ScheduleKey, ) - - final case class ExtPrimaryDataServiceInitResponseMessage( - override val serviceRef: ActorRef, - subscribers: List[UUID] - ) extends RegistrationResponseMessage } /** Actual provision of data diff --git a/src/main/scala/edu/ie3/simona/scheduler/Scheduler.scala b/src/main/scala/edu/ie3/simona/scheduler/Scheduler.scala index 6f8b9d9138..7339fbbe61 100644 --- a/src/main/scala/edu/ie3/simona/scheduler/Scheduler.scala +++ b/src/main/scala/edu/ie3/simona/scheduler/Scheduler.scala @@ -123,8 +123,6 @@ object Scheduler { _ ! Activation(newCore.activeTick) } - //println("ScheduleActivation from " + actor + ", nextTick = " + newTick + ", newCore = " + newCore) - active(data, newCore) case (_, Completion(actor, maybeNewTick)) => @@ -139,7 +137,7 @@ object Scheduler { toActivate.foreach { _ ! Activation(updatedCore.activeTick) } - //println("Completion from " + actor + ", nextTick = " + maybeNewTick + ", newCore = " + updatedCore) + updatedCore } .map { newCore => diff --git a/src/main/scala/edu/ie3/simona/service/ExtDataSupport.scala b/src/main/scala/edu/ie3/simona/service/ExtDataSupport.scala index 0e01de03bc..21176bc8fe 100644 --- a/src/main/scala/edu/ie3/simona/service/ExtDataSupport.scala +++ b/src/main/scala/edu/ie3/simona/service/ExtDataSupport.scala @@ -7,11 +7,10 @@ package edu.ie3.simona.service import edu.ie3.simona.api.data.ontology.DataMessageFromExt -import edu.ie3.simona.ontology.messages.services.{DataMessage, EvMessage} +import edu.ie3.simona.api.data.results.ontology.ResultDataMessageFromExt +import edu.ie3.simona.ontology.messages.services.DataMessage import edu.ie3.simona.ontology.messages.services.EvMessage.EvResponseMessage import edu.ie3.simona.service.ServiceStateData.ServiceBaseStateData -import edu.ie3.simona.ontology.messages.services.ResultMessage -import edu.ie3.simona.api.data.results.ontology.ResultDataMessageFromExt trait ExtDataSupport[ S <: ServiceBaseStateData @@ -20,19 +19,9 @@ trait ExtDataSupport[ override def idleExternal(implicit stateData: S): Receive = { case extMsg: DataMessageFromExt => - //log.info("Got a DataMessageFromExt with content " + extMsg) val updatedStateData = handleDataMessage(extMsg)(stateData) context become idle(updatedStateData) - case extResponseMsgToExt: ResultMessage => - //log.info("Got a DataResponse with content " + extResponseMsgToExt) - val updatedStateData = handleDataResponseMessage(extResponseMsgToExt)(stateData) - context become idle(updatedStateData) - - case extResultDataMsg: ResultDataMessageFromExt => - val updatedStateData = handleDataMessage(extResultDataMsg)(stateData) - context become idle(updatedStateData) - case extResponseMsg: EvResponseMessage => val updatedStateData = handleDataResponseMessage(extResponseMsg)(stateData) diff --git a/src/main/scala/edu/ie3/simona/service/SimonaService.scala b/src/main/scala/edu/ie3/simona/service/SimonaService.scala index c6f2dc7b62..caa9f12825 100644 --- a/src/main/scala/edu/ie3/simona/service/SimonaService.scala +++ b/src/main/scala/edu/ie3/simona/service/SimonaService.scala @@ -6,12 +6,6 @@ package edu.ie3.simona.service -import edu.ie3.simona.api.data.results.ontology.ResultDataMessageFromExt -import edu.ie3.simona.event -import edu.ie3.simona.event.Event -import edu.ie3.simona.event.listener.DelayedStopHelper -import org.apache.pekko.actor.typed.scaladsl.adapter.ClassicActorRefOps -import org.apache.pekko.actor.{Actor, ActorContext, ActorRef, Stash} import edu.ie3.simona.logging.SimonaActorLogging import edu.ie3.simona.ontology.messages.Activation import edu.ie3.simona.ontology.messages.SchedulerMessage.{Completion, ScheduleActivation} @@ -21,7 +15,8 @@ import edu.ie3.simona.scheduler.ScheduleLock.ScheduleKey import edu.ie3.simona.service.ServiceStateData.{InitializeServiceStateData, ServiceBaseStateData} import edu.ie3.simona.service.SimonaService.Create import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK -import edu.ie3.simona.ontology.messages.services.ServiceMessage.RequestExtPrimaryDataAssets +import org.apache.pekko.actor.typed.scaladsl.adapter.ClassicActorRefOps +import org.apache.pekko.actor.{Actor, ActorContext, ActorRef, Stash} import scala.util.{Failure, Success, Try} @@ -108,7 +103,7 @@ abstract class SimonaService[ } // not ready yet to handle registrations, stash request away - case _: ServiceRegistrationMessage | _: Activation | _: ResultDataMessageFromExt | _: RequestExtPrimaryDataAssets => + case _: ServiceRegistrationMessage | _: Activation => stash() // unhandled message @@ -131,7 +126,6 @@ abstract class SimonaService[ private def idleInternal(implicit stateData: S): Receive = { // agent registration process - case registrationMsg: ServiceRegistrationMessage => /* Someone asks to register for information from the service */ handleRegistrationRequest(registrationMsg) match { @@ -148,7 +142,6 @@ abstract class SimonaService[ } case ScheduleServiceActivation(tick, unlockKey) => - //log.info("Send a ScheduleServiceActivation") scheduler ! ScheduleActivation( self.toTyped, tick, @@ -157,21 +150,12 @@ abstract class SimonaService[ // activity start trigger for this service case Activation(tick) => - //log.info("Got Activation") /* The scheduler sends out an activity start trigger. Announce new data to all registered recipients. */ - val (updatedStateData, maybeNewTriggers) = { + val (updatedStateData, maybeNewTriggers) = announceInformation(tick)(stateData, context) - } - //log.info("Send Completion") scheduler ! Completion(self.toTyped, maybeNewTriggers) context become idle(updatedStateData) - /* - case msg: DelayedStopHelper.StoppingMsg => - DelayedStopHelper.handleMsg((context[_], msg)) - - */ - // unhandled message case x => log.error("Unhandled message received:{}", x) diff --git a/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala b/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala index dfcfbe7d18..dcaa46ec65 100644 --- a/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala +++ b/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala @@ -16,16 +16,8 @@ import edu.ie3.simona.api.data.primarydata.ontology.{ } import edu.ie3.simona.exceptions.WeatherServiceException.InvalidRegistrationRequestException import edu.ie3.simona.exceptions.{InitializationException, ServiceException} -import edu.ie3.simona.ontology.messages.services.ServiceMessage.{ - ExtPrimaryDataServiceRegistrationMessage, - PrimaryServiceRegistrationMessage, - WorkerRegistrationMessage, - ExtPrimaryDataServiceInitRequestMessage -} -import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.{ - RegistrationSuccessfulMessage, - ExtPrimaryDataServiceInitResponseMessage -} +import edu.ie3.simona.ontology.messages.services.ServiceMessage.ExtPrimaryDataServiceRegistrationMessage +import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.RegistrationSuccessfulMessage import edu.ie3.simona.ontology.messages.services.{DataMessage, ServiceMessage} import edu.ie3.simona.scheduler.ScheduleLock import edu.ie3.simona.service.ServiceStateData.{ diff --git a/src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala b/src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala index a029985be4..ae192b12d2 100644 --- a/src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala +++ b/src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala @@ -4,18 +4,15 @@ import edu.ie3.datamodel.models.result.ResultEntity import edu.ie3.simona.api.data.results.ExtResultData import edu.ie3.simona.api.data.results.ontology.{ProvideResultEntities, RequestResultEntities, ResultDataMessageFromExt} import edu.ie3.simona.event.listener.DelayedStopHelper -import edu.ie3.simona.exceptions.{InitializationException, ServiceException} -import edu.ie3.simona.ontology.messages.{Activation, SchedulerMessage} +import edu.ie3.simona.exceptions.ServiceException import edu.ie3.simona.ontology.messages.SchedulerMessage.{Completion, ScheduleActivation} +import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.ScheduleServiceActivation +import edu.ie3.simona.ontology.messages.{Activation, SchedulerMessage} import edu.ie3.simona.scheduler.ScheduleLock.ScheduleKey -import edu.ie3.simona.service.ServiceStateData.{InitializeServiceStateData, ServiceBaseStateData} import edu.ie3.simona.util.ReceiveDataMap import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK -import org.apache.pekko.actor.typed.{Behavior, PostStop} import org.apache.pekko.actor.typed.{ActorRef, Behavior} import org.apache.pekko.actor.typed.scaladsl.{Behaviors, StashBuffer} -import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.ScheduleServiceActivation -import org.apache.pekko.actor.ActorContext import java.util.UUID import scala.jdk.CollectionConverters._ @@ -87,7 +84,6 @@ object ExtResultDataProvider { initializeStateData: InitExtResultData, unlockKey: ScheduleKey, ) => - println("Send Completion") scheduler ! ScheduleActivation(activationAdapter, INIT_SIM_TICK, Some(unlockKey)) diff --git a/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala b/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala index c01742c38e..6f0b75448b 100644 --- a/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala +++ b/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala @@ -212,9 +212,7 @@ object SimonaSim { // if the simulation is successful, we're waiting for the delayed // stopping listeners to terminate and thus do not unwatch them here actorData.delayedStoppingActors.foreach( - actor => { - actor ! DelayedStopHelper.FlushAndStop - } + _ ! DelayedStopHelper.FlushAndStop ) maybeStop( 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 cb5d40037d..55d907ae78 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SetupHelper.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SetupHelper.scala @@ -24,8 +24,8 @@ import edu.ie3.simona.model.grid.RefSystem import edu.ie3.simona.util.ConfigUtil.{GridOutputConfigUtil, OutputConfigUtil} import edu.ie3.simona.util.ResultFileHierarchy.ResultEntityPathConfig import edu.ie3.simona.util.{EntityMapperUtil, ResultFileHierarchy} -import edu.ie3.util.quantities.PowerSystemUnits import org.apache.pekko.actor.typed.ActorRef +import edu.ie3.util.quantities.PowerSystemUnits import squants.electro.Kilovolts /** Methods to support the setup of a simona simulation diff --git a/src/main/scala/edu/ie3/simona/util/ReceiveDataMap.scala b/src/main/scala/edu/ie3/simona/util/ReceiveDataMap.scala index 11b87b6349..1f7fca229e 100644 --- a/src/main/scala/edu/ie3/simona/util/ReceiveDataMap.scala +++ b/src/main/scala/edu/ie3/simona/util/ReceiveDataMap.scala @@ -41,22 +41,7 @@ final case class ReceiveDataMap[K, V]( ) } - def addDataWithoutCheck( - key: K, - value: V, - ): ReceiveDataMap[K, V] = { - - if (expectedKeys.contains(key)) { - copy( - expectedKeys = expectedKeys.excl(key), - receivedData.updated(key, value), - ) - } else { - copy() - } - } - - } +} object ReceiveDataMap { From d8752262afbaf83cf7f6cca1a37ce35e5c83ab7b Mon Sep 17 00:00:00 2001 From: smjobaoo Date: Tue, 19 Mar 2024 10:33:14 +0100 Subject: [PATCH 18/41] uml --- docs/uml/main/ExtSimulationAndAPIClasses.puml | 133 ++++++++++++++++++ 1 file changed, 133 insertions(+) create mode 100644 docs/uml/main/ExtSimulationAndAPIClasses.puml diff --git a/docs/uml/main/ExtSimulationAndAPIClasses.puml b/docs/uml/main/ExtSimulationAndAPIClasses.puml new file mode 100644 index 0000000000..98942a8d32 --- /dev/null +++ b/docs/uml/main/ExtSimulationAndAPIClasses.puml @@ -0,0 +1,133 @@ +@startuml +'https://plantuml.com/class-diagram +skinparam linetype ortho + +package simona-api { + + interface DataMessageFromExt + interface PrimaryDataMessageFromExt + interface EvDataMessageFromExt + + PrimaryDataMessageFromExt --> DataMessageFromExt + EvDataMessageFromExt --> DataMessageFromExt + ResultDataMessageFromExt --> DataMessageFromExt + + class ProvidePrimaryData { + + long tick + + Map primaryData + } + + ProvidePrimaryData --> PrimaryDataMessageFromExt + + class ProvideArrivingEvs { + + Map> arrivals) + } + + class RequestCurrentPrices + class RequestDepartingEvs + class RequestEvcsFreeLots + + ProvideArrivingEvs --> EvDataMessageFromExt + RequestCurrentPrices --> EvDataMessageFromExt + RequestDepartingEvs --> EvDataMessageFromExt + RequestEvcsFreeLots --> EvDataMessageFromExt + + class RequestResultEntities { + + Long tick + } + + RequestResultEntities --> ResultDataMessageFromExt + + interface DataResponseMessageToExt + interface ResultDataResponseMessageToExt + interface EvDataResponseMessageToExt + + ResultDataResponseMessageToExt --> DataResponseMessageToExt + EvDataResponseMessageToExt --> DataResponseMessageToExt + + class ProvideResultEntities { + + List results + } + + ProvideResultEntities --> ResultDataResponseMessageToExt + + class ProvideCurrentPrices + + class ProvideDepartingEvs + + class ProvideEvcsFreeLots + ProvideCurrentPrices --> EvDataResponseMessageToExt + ProvideDepartingEvs --> EvDataResponseMessageToExt + ProvideEvcsFreeLots --> EvDataResponseMessageToExt + + interface ExtData + + class ExtEvData + class ExtPrimaryData { + PrimaryDataFactory primaryDataFactory + } + class ExtResultData + + ExtEvData --> ExtData + ExtPrimaryData --> ExtData + ExtResultData --> ExtData + + + + interface PrimaryDataFactory { + + Value convert(Object entity) + } + PrimaryDataFactory -- ExtPrimaryData + + interface ResultDataFactory { + + Object convert(ResultEntity entity) + } + ResultDataFactory -- ExtResultData + + interface ExtDataSimulation + interface ExtResultDataSimulation + interface ExtPrimaryDataSimulation + interface ExtEvDataSimulation + + ExtPrimaryDataSimulation --> ExtDataSimulation + ExtResultDataSimulation --> ExtDataSimulation + ExtEvDataSimulation --> ExtDataSimulation + + abstract class ExtSimulation { + - ExtSimAdapterData data + + # Optional initialize() + # Optional doActivity(long tick) + } +} + +package simona { + class SimonaSim + + class SimScheduler + + class SimonaStandaloneSetup + + class ExtSimLoader + + class ExtSimAdapter + + class ExtResultDataProvider + ExtResultDataProvider -- ExtResultData + + class ExtPrimaryDataService + ExtPrimaryDataService -- ExtPrimaryData + + SimScheduler -- SimonaSim + SimonaSim *- SimonaStandaloneSetup + SimonaStandaloneSetup *- ExtSimLoader + + ExtSimAdapter -- SimScheduler + ExtPrimaryDataService -- SimScheduler + + SimonaService <|- ExtPrimaryDataService + + ExtSimAdapterData <--- ExtSimAdapter +} + +@enduml \ No newline at end of file From dc2ee3a65b1516a43f4cb8bcf88a387ffa3d32cf Mon Sep 17 00:00:00 2001 From: Johannes Bao Date: Tue, 19 Mar 2024 10:36:49 +0100 Subject: [PATCH 19/41] extSimulation --- .../ExtPrimaryResultSimulationClasses.puml | 223 ++++++++++++++++++ .../scala/edu/ie3/simona/sim/SimonaSim.scala | 2 + .../sim/setup/SimonaStandaloneSetup.scala | 163 ++++++++++++- 3 files changed, 385 insertions(+), 3 deletions(-) create mode 100644 docs/uml/main/ExtPrimaryResultSimulationClasses.puml diff --git a/docs/uml/main/ExtPrimaryResultSimulationClasses.puml b/docs/uml/main/ExtPrimaryResultSimulationClasses.puml new file mode 100644 index 0000000000..e35c8e130a --- /dev/null +++ b/docs/uml/main/ExtPrimaryResultSimulationClasses.puml @@ -0,0 +1,223 @@ +@startuml +'https://plantuml.com/class-diagram +skinparam linetype ortho + +package ev-simulation { + class ExtLink + class ExternalSampleSim + class EvModelImpl + + ' adding hidden elements for layout purposes + class placeholderA { + w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w + } + hide placeholderA + + class placeholderB { + w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w + } + hide placeholderB + + ExtLink -[hidden]> placeholderA + placeholderA -[hidden]> ExternalSampleSim + ExternalSampleSim -[hidden]> placeholderB + placeholderB -[hidden]> EvModelImpl +} + +package simona-api { + ' MIDDLE PART + class ExtEvData { + ~ LinkedBlockingQueue receiveTriggerQueue + - ActorRef dataService + - ActorRef extSimAdapter + + List requestAvailablePublicEvCs() + + List sendEvPositions(EvMovementsMessage evMovementsMessage) + - void sendData(ExtEvMessage msg) + + void queueExtMsg(ExtEvResponseMessage extEvResponse) + } + + class ExtSimAdapterData { + ~ LinkedBlockingQueue receiveTriggerQueue + - ActorRef extSimAdapter + + void queueExtMsg(ExtTrigger trigger) + + void send(ExtTriggerResponse msg) + } + + interface ExtData + + abstract class ExtSimulation { + + void setup(ExtSimAdapterData data, List adapters) + + void run() + # {abstract} List doActivity(long tick) + } + + interface ExtEvSimulation { + + void setExtEvData(ExtEvData evData) + } + + interface ExtDataSimulation + + ExternalSampleSim --|> ExtSimulation + ExternalSampleSim --|> ExtEvSimulation + + ExtSimulation -[hidden]> ExtDataSimulation + ExtEvSimulation -|> ExtDataSimulation + ExtEvSimulation --> ExtEvData + + ExtSimulation --> ExtData + ExtSimulation --> ExtSimAdapterData + + ExtSimAdapterData -[hidden]> ExtEvData + ExtData <|-- ExtEvData + + ' placeholder + class plchldr1 { + w-w-w-w-w + } + hide plchldr1 + class placeholder2 { + w-w-w-w-w-w-w-w-w-w-w-w-w-w-w + } + hide placeholder2 + class placeholder3 { + w-w-w-w-w-w-w-w-w-w-w + } + hide placeholder3 + class placeholder4 { + w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w + } + hide placeholder4 + + ExtData -[hidden]> plchldr1 + plchldr1 -[hidden]> placeholder4 + placeholder4 -[hidden]> EvMovement + ExtSimulation -[hidden]-> placeholder2 + placeholder2 -[hidden]-> placeholder3 + placeholder3 -[hidden]-> ExtSimAdapterData + placeholder2 -[hidden]> ExtData + + + ' RIGHT PART + abstract class EvMovement + class Arrival + class Departure + Arrival --|> EvMovement + Departure --|> EvMovement + Arrival -[hidden]> Departure + + interface EvModel { + + UUID getUuid() + + String getId() + + ComparableQuantity getSRated() + + ComparableQuantity getEStorage() + + ComparableQuantity getStoredEnergy() + + EvModel copyWith(ComparableQuantity newStoredEnergy) + } + + ExtDataSimulation -[hidden]> Arrival + EvMovement *-> EvModel + EvModelImpl --|> EvModel + + + interface ExtEvMessage + class EvMovementsMessage { + - Map> movements + } + class RequestEvcsFreeLots + RequestEvcsFreeLots --|> ExtEvMessage + EvMovementsMessage --|> ExtEvMessage + RequestEvcsFreeLots -[hidden]> EvMovementsMessage + + interface ExtEvResponseMessage + class AllDepartedEvsRepsonse { + - Map> movements + } + class ProvideEvcsFreeLots + ProvideEvcsFreeLots --|> ExtEvResponseMessage + AllDepartedEvsRepsonse --|> ExtEvResponseMessage + ExtEvData -> ExtEvMessage + ExtEvData -> ExtEvResponseMessage + EvMovement -[hidden]-> RequestEvcsFreeLots + ExtEvMessage -[hidden]> ExtEvResponseMessage + EvMovementsMessage -[hidden]> ProvideEvcsFreeLots + ProvideEvcsFreeLots -[hidden]> AllDepartedEvsRepsonse + + class ScheduleDataServiceMessage { + - ExtEvDataService dataService + } + ExtEvResponseMessage -[hidden]> ScheduleDataServiceMessage + ExtEvData -> ScheduleDataServiceMessage + + + ' LEFT PART + interface ExtLinkInterface { + + ExtSimulation getExtSimulation() + + List getExtDataSimulations() + } + ExtLinkInterface -[hidden]> ExtSimulation + ExtLink --|> ExtLinkInterface + + interface ExtTrigger + class ActivityStartTrigger { + - Long tick + } + ActivityStartTrigger --|> ExtTrigger + + interface ExtTriggerResponse + class CompletionMessage { + - List newTriggers + } + CompletionMessage --|> ExtTriggerResponse + + ExtTrigger -[hidden]> ExtTriggerResponse + ExtTrigger <- ExtSimAdapterData + ExtTriggerResponse <- ExtSimAdapterData +} + +package simona { + abstract class ParticipantAgent + + class EvcsModel + + class EvcsAgent + + interface SecondaryData + class EvMovementData { + movements: Set[EvMovement] + } + + class SimonaSim + + class SimScheduler + + class SimonaStandaloneSetup + + class ExtSimLoader + + class ExtSimAdapter + + class ExtEvDataService + + SimScheduler -- SimonaSim + SimonaSim *- SimonaStandaloneSetup + SimonaStandaloneSetup *- ExtSimLoader + + ExtSimAdapter -- SimScheduler + ExtEvDataService -- SimScheduler + + SecondaryData <|-- EvMovementData + + SimonaService <|- ExtEvDataService + ExtEvData <--- ExtEvDataService + + ExtSimAdapterData <--- ExtSimAdapter + + ' 1 and n are swapped here due to bug with linetype ortho and labels + ExtEvDataService "n" <-> "1" EvcsAgent + EvcsAgent --|> ParticipantAgent + EvcsAgent *- EvcsModel + + ExtSimAdapter -[hidden]> SimonaService +} + +@enduml \ No newline at end of file diff --git a/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala b/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala index 6f0b75448b..8194aefaf5 100644 --- a/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala +++ b/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala @@ -132,6 +132,8 @@ object SimonaSim { ) ++ gridAgents + ctx.log.info(s"extSimulationData = $extSimulationData") + val delayedActors = resultEventListeners.appended(runtimeEventListener).appended(extSimulationData.extResultDataService.getOrElse( throw new Exception("Problem!") ).toTyped[DelayedStopHelper.StoppingMsg]) 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 73ef760563..eba8ca48e4 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala @@ -17,8 +17,9 @@ import edu.ie3.simona.agent.grid.GridAgentMessage.CreateGridAgent import edu.ie3.simona.agent.grid.{GridAgent, GridAgentMessage} import edu.ie3.simona.api.ExtSimAdapter import edu.ie3.simona.api.data.ExtData -import edu.ie3.simona.api.data.primarydata.ExtPrimaryData -import edu.ie3.simona.api.data.results.ExtResultData +import edu.ie3.simona.api.data.ev.{ExtEvData, ExtEvSimulation} +import edu.ie3.simona.api.data.primarydata.{ExtPrimaryData, ExtPrimaryDataSimulation} +import edu.ie3.simona.api.data.results.{ExtResultData, ExtResultDataSimulation} import edu.ie3.simona.api.data.results.ontology.ResultDataMessageFromExt import edu.ie3.simona.api.simulation.ExtSimAdapterData import edu.ie3.simona.config.{ArgsParser, RefSystemParser, SimonaConfig} @@ -31,6 +32,8 @@ import edu.ie3.simona.ontology.messages.SchedulerMessage.ScheduleActivation import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.ScheduleServiceActivation import edu.ie3.simona.scheduler.{ScheduleLock, Scheduler, TimeAdvancer} import edu.ie3.simona.service.SimonaService +import edu.ie3.simona.service.ev.ExtEvDataService +import edu.ie3.simona.service.ev.ExtEvDataService.InitExtEvData import edu.ie3.simona.service.primary.ExtPrimaryDataService.InitExtPrimaryData import edu.ie3.simona.service.primary.PrimaryServiceProxy.InitPrimaryServiceProxyStateData import edu.ie3.simona.service.primary.{ExtPrimaryDataService, PrimaryServiceProxy} @@ -42,7 +45,7 @@ import edu.ie3.simona.sim.SimonaSim import edu.ie3.simona.util.ResultFileHierarchy import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK import edu.ie3.simona.util.TickUtil.RichZonedDateTime -import edu.ie3.simosaik.simpleextsim.SimpleExtSimulation +//import edu.ie3.simosaik.simpleextsim.SimpleExtSimulation import edu.ie3.util.TimeUtil import org.apache.pekko.actor.typed.scaladsl.ActorContext import org.apache.pekko.actor.typed.scaladsl.AskPattern._ @@ -194,7 +197,159 @@ class SimonaStandaloneSetup( weatherService } + override def extSimulations( + context: ActorContext[_], + theScheduler: ActorRef[SchedulerMessage], + ): ExtSimSetupData = { + val jars = ExtSimLoader.scanInputFolder() + + val extLinks = jars.flatMap(ExtSimLoader.loadExtLink) + + context.log.info(s"extLinks = $extLinks") + + val (extSimAdapters, extDatasAndServices) = + extLinks.zipWithIndex.map { case (extLink, index) => + // external simulation always needs at least an ExtSimAdapter + val extSimAdapter = context.toClassic.simonaActorOf( + ExtSimAdapter.props(theScheduler.toClassic), + s"$index", + ) + val extSimAdapterData = new ExtSimAdapterData(extSimAdapter, args) + + // send init data right away, init activation is scheduled + extSimAdapter ! ExtSimAdapter.Create( + extSimAdapterData, + ScheduleLock.singleKey(context, theScheduler, INIT_SIM_TICK), + ) + + // setup data services that belong to this external simulation + val (extData, extDataServiceToRef): ( + Iterable[ExtData], + Iterable[(Class[_], ClassicRef)], + ) = + extLink.getExtDataSimulations.asScala.zipWithIndex.map { + case (_: ExtEvSimulation, dIndex) => + val extEvDataService = context.toClassic.simonaActorOf( + ExtEvDataService.props(theScheduler.toClassic), + s"$index-$dIndex", + ) + val extEvData = new ExtEvData(extEvDataService, extSimAdapter) + + extEvDataService ! SimonaService.Create( + InitExtEvData(extEvData), + ScheduleLock.singleKey( + context, + theScheduler, + INIT_SIM_TICK, + ), + ) + + (extEvData, (classOf[ExtEvDataService], extEvDataService)) + + case (extPrimaryDataSimulation: ExtPrimaryDataSimulation, dIndex) => + val extPrimaryDataService = context.toClassic.simonaActorOf( + ExtPrimaryDataService.props(theScheduler.toClassic), + s"$index-$dIndex", + ) + val extPrimaryData = new ExtPrimaryData( + extPrimaryDataService, + extSimAdapter, + extPrimaryDataSimulation.getPrimaryDataFactory, + extPrimaryDataSimulation.getPrimaryDataAssets + ) + + extPrimaryDataService ! SimonaService.Create( + InitExtPrimaryData(extPrimaryData), + ScheduleLock.singleKey( + context, + theScheduler, + INIT_SIM_TICK, + ), + ) + + (extPrimaryData, (classOf[ExtPrimaryDataService], extPrimaryDataService)) + + case (extResultDataSimulation: ExtResultDataSimulation, dIndex) => + + val extResultDataProvider = { + context.spawn( + ExtResultDataProvider(theScheduler), + s"$index-$dIndex", + ) + } + + implicit val timeout: PekkoTimeout = PekkoTimeout.create(5.seconds.toJava) + implicit val scheduler: Scheduler = context.system.scheduler + + val adapterRef = Await.result( + extResultDataProvider.ask[ActorRef[ResultDataMessageFromExt]] (ref => RequestDataMessageAdapter(ref)), timeout.duration) + val adapterScheduleRef = Await.result( + extResultDataProvider.ask[ActorRef[ScheduleServiceActivation]] (ref => RequestScheduleActivationAdapter(ref)), timeout.duration) + + val extResultData = + new ExtResultData( + adapterRef.toClassic, + adapterScheduleRef.toClassic, + extSimAdapter, + extResultDataSimulation.getResultDataFactory, + extResultDataSimulation.getResultDataAssets + ) + + context.log.info(s"extResultData $extResultData") + + extResultDataProvider ! ExtResultDataProvider.Create( + InitExtResultData(extResultData), + ScheduleLock.singleKey( + context, + theScheduler, + INIT_SIM_TICK, + ), + ) + /* + val extResultDataService = context.toClassic.simonaActorOf( + ExtResultDataProvider.props(scheduler.toClassic), + s"$index-$dIndex", + ) + val extResultsData = + new ExtResultsData(extResultDataService, extSimAdapter, null) + + extResultDataService ! SimonaService.Create( + InitExtResultsData(extResultsData), + ScheduleLock.singleKey( + context, + scheduler, + INIT_SIM_TICK, + ), + ) + */ + + ( + extResultData, + (ExtResultDataProvider.getClass, extResultDataProvider.toClassic), + ) + }.unzip + + extLink.getExtSimulation.setup( + extSimAdapterData, + extData.toList.asJava, + ) + + // starting external simulation + new Thread(extLink.getExtSimulation, s"External simulation $index").start() + + (extSimAdapter, (extDataServiceToRef, extData)) + }.unzip + + val extDataServices = extDatasAndServices.map(_._1) + val extDatas = extDatasAndServices.flatMap(_._2).toSet + context.log.info(s"extDatasAndServices = $extDatasAndServices") + ExtSimSetupData(extSimAdapters, extDataServices.flatten.toMap, extDatas) + } + + /* + + def extTestSimulations( context: ActorContext[_], theScheduler: ActorRef[SchedulerMessage], ): ExtSimSetupData = { @@ -294,6 +449,8 @@ class SimonaStandaloneSetup( ExtSimSetupData(extSimAdapters, extDataServicesMap, extDatas) } + */ + override def timeAdvancer( context: ActorContext[_], simulation: ActorRef[SimonaSim.SimulationEnded.type], From c4c753e654585d9f74e03840a9efbe663297e739 Mon Sep 17 00:00:00 2001 From: Johannes Bao Date: Tue, 19 Mar 2024 11:09:13 +0100 Subject: [PATCH 20/41] testgrids --- .../fullGrid/line_graphic_input.csv | 16 ++ .../vn_simona_ns/fullGrid/line_input.csv | 16 ++ .../vn_simona_ns/fullGrid/line_type_input.csv | 16 ++ .../vn_simona_ns/fullGrid/load_input.csv | 16 ++ .../fullGrid/node_graphic_input.csv | 17 ++ .../vn_simona_ns/fullGrid/node_input.csv | 18 ++ .../vn_simona_ns/fullGrid/pv_input.csv | 3 + .../fullGrid/transformer_2_w_input.csv | 2 + .../fullGrid/transformer_2_w_type_input.csv | 2 + input/samples/vn_simona_ns/vn_simona_ns.conf | 214 ++++++++++++++++++ 10 files changed, 320 insertions(+) create mode 100644 input/samples/vn_simona_ns/fullGrid/line_graphic_input.csv create mode 100644 input/samples/vn_simona_ns/fullGrid/line_input.csv create mode 100644 input/samples/vn_simona_ns/fullGrid/line_type_input.csv create mode 100644 input/samples/vn_simona_ns/fullGrid/load_input.csv create mode 100644 input/samples/vn_simona_ns/fullGrid/node_graphic_input.csv create mode 100644 input/samples/vn_simona_ns/fullGrid/node_input.csv create mode 100644 input/samples/vn_simona_ns/fullGrid/pv_input.csv create mode 100644 input/samples/vn_simona_ns/fullGrid/transformer_2_w_input.csv create mode 100644 input/samples/vn_simona_ns/fullGrid/transformer_2_w_type_input.csv create mode 100644 input/samples/vn_simona_ns/vn_simona_ns.conf diff --git a/input/samples/vn_simona_ns/fullGrid/line_graphic_input.csv b/input/samples/vn_simona_ns/fullGrid/line_graphic_input.csv new file mode 100644 index 0000000000..495bcf8dd6 --- /dev/null +++ b/input/samples/vn_simona_ns/fullGrid/line_graphic_input.csv @@ -0,0 +1,16 @@ +uuid,graphic_layer,line,path +8ae601e8-3dcf-497e-a258-d8ac976e95e8,Standard,106f03b3-b84e-4d36-ab09-13b3dc3403e7,"{""type"":""LineString"",""coordinates"":[[0.20338983,0.1557377],[0.25423729,0.1557377],[0.22881291,0.1557377]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}" +bd830d87-73b0-44c8-9250-26afd33ac4f5,Standard,10c6a103-3132-4324-8f76-8c1858d51fcb,"{""type"":""LineString"",""coordinates"":[[0.20338983,0.24590164],[0.17514124,0.24590164],[0.14689266,0.24590164]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}" +26138708-ff02-48c2-a465-a8b5e5d165a8,Standard,25a1e96e-b425-48bf-a3ae-dc54ea9da47f,"{""type"":""LineString"",""coordinates"":[[0.03954802,0.27868852],[0.03954802,0.17213115],[0.03954802,0.06557377],[0.09039548,0.06557377]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}" +d923185a-03a0-484b-a66d-08f4757aadf7,Standard,2d552f15-6871-4e09-84b1-003da098d909,"{""type"":""LineString"",""coordinates"":[[0.09039548,0.1557377],[0.11864407,0.1557377],[0.14689266,0.1557377]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}" +eb5e4437-0f18-43c1-9e51-51bcfc038bea,Standard,37c8ebe1-5d29-4806-a84a-fc00e4279fa6,"{""type"":""LineString"",""coordinates"":[[0.22881291,0.06557377],[0.25423729,0.06557377],[0.20338983,0.06557377]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}" +75e06780-090a-411f-a47c-47c5a90a2e09,Standard,4bdb6569-72da-4d60-97b1-b5a639032d5d,"{""type"":""LineString"",""coordinates"":[[0.25423729,0.24590164],[0.28248588,0.24590164],[0.31073446,0.24590164]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}" +723fb8fc-90ff-44d5-9c21-88a9f194e83a,Standard,5d1176ac-e64e-4811-8d0c-c87d9060ba2a,"{""type"":""LineString"",""coordinates"":[[0.09039548,0.1557377],[0.05084746,0.27868852],[0.05084746,0.21721405],[0.05084746,0.1557377]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}" +90ad029a-358f-4f05-8a01-62820461dbb2,Standard,814dec01-f6f2-4817-8f83-2e758011b033,"{""type"":""LineString"",""coordinates"":[[0.09039548,0.24590164],[0.06214689,0.27868852],[0.06214689,0.24590164],[0.06214689,0.26229508]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}" +79edb34f-6eea-4190-a0d4-2664ace2deb8,Standard,8bb29769-404b-4bf6-837e-f9ca1b389bf9,"{""type"":""LineString"",""coordinates"":[[0.17514124,0.06557377],[0.14689266,0.06557377],[0.20338983,0.06557377]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}" +90344d84-9485-4986-93e1-e94d73ee0aa3,Standard,91179f48-2d6c-4ae4-864c-d667f57a6e66,"{""type"":""LineString"",""coordinates"":[[0.09039548,0.24590164],[0.11864407,0.24590164],[0.14689266,0.24590164]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}" +31dab450-fe89-4ede-99e2-cba361671910,Standard,b714db4a-db55-4957-b503-2300559bb81f,"{""type"":""LineString"",""coordinates"":[[0.20338983,0.1557377],[0.14689266,0.1557377],[0.17514124,0.1557377]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}" +79a153b2-2b64-4b59-bbe5-946a3c0073bd,Standard,c3206971-b22f-4090-826b-e08d4bc8ffad,"{""type"":""LineString"",""coordinates"":[[0.20338983,0.24590164],[0.25423729,0.24590164],[0.22881421,0.24590164]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}" +6dd7072b-da09-4d47-a7ad-a57389a918f9,Standard,cba98774-b8c6-45f4-84cc-2fde121a77fa,"{""type"":""LineString"",""coordinates"":[[0.14689266,0.06557377],[0.11864407,0.06557377],[0.09039548,0.06557377]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}" +74887fe3-c3a5-4919-8b77-743e3f2c5bd4,Standard,d48db7e8-5fde-456f-8b17-02d18a5a2e32,"{""type"":""LineString"",""coordinates"":[[0.31073446,0.1557377],[0.25423729,0.1557377],[0.28248588,0.1557377]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}" +a8cd181f-6834-437f-aa9e-6b165504b870,Standard,f0d4491e-eb64-49a0-8fc3-44fa5bd9dcee,"{""type"":""LineString"",""coordinates"":[[0.31073446,0.06557377],[0.25423729,0.06557377],[0.28248588,0.06557377]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}" \ No newline at end of file diff --git a/input/samples/vn_simona_ns/fullGrid/line_input.csv b/input/samples/vn_simona_ns/fullGrid/line_input.csv new file mode 100644 index 0000000000..0eea1cc0f7 --- /dev/null +++ b/input/samples/vn_simona_ns/fullGrid/line_input.csv @@ -0,0 +1,16 @@ +uuid,geo_position,id,length,node_a,node_b,olm_characteristic,operates_from,operates_until,operator,parallel_devices,type +5d1176ac-e64e-4811-8d0c-c87d9060ba2a,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_NET116_ONS-NS_NET116_S2_1,0.029999999329447746,31a2b9bf-e785-4475-aa44-1c34646e8c79,39112046-8953-4e73-a5d9-6a8183a77436,"olm:{(0.0,1.0)}",,,,1,4ee89e0b-1e97-4dde-ba57-3b0b862957c5 +106f03b3-b84e-4d36-ab09-13b3dc3403e7,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_NET116_S2_3-NS_NET116_S2_4,0.029999999329447746,df9c3d91-41a0-4e49-bb1f-1cd2a8fd9a2e,e05c68b1-11cd-43fd-a4b2-31e4db380c78,"olm:{(0.0,1.0)}",,,,1,17e87208-d092-4514-8b23-ab91598798ce +c3206971-b22f-4090-826b-e08d4bc8ffad,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_NET116_S1_3-NS_NET116_S1_4,0.029999999329447746,e4c8b492-965c-4b5a-8d14-e8ff2c027ce9,b3a7431d-89b0-41cb-87a9-5853890796cd,"olm:{(0.0,1.0)}",,,,1,b63674e1-6827-4e8b-b3e6-34ee6d4823a9 +814dec01-f6f2-4817-8f83-2e758011b033,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_NET116_ONS-NS_NET116_S1_1,0.029999999329447746,31a2b9bf-e785-4475-aa44-1c34646e8c79,8a50fb37-81cf-47c9-8850-a12b4391e2e7,"olm:{(0.0,1.0)}",,,,1,a7071b97-2be4-40bf-a0b1-b8faf6bff6d6 +cba98774-b8c6-45f4-84cc-2fde121a77fa,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_NET116_S3_1-NS_NET116_S3_2,0.029999999329447746,4a6f6058-e654-464d-9367-2dca7185c6d7,550ebca7-1455-44eb-9431-ffbf08e58bd4,"olm:{(0.0,1.0)}",,,,1,6986321a-0a85-411e-8b2d-9d929a7f298b +2d552f15-6871-4e09-84b1-003da098d909,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_NET116_S2_1-NS_NET116_S2_2,0.029999999329447746,39112046-8953-4e73-a5d9-6a8183a77436,d53ff076-dadd-44f8-85d4-68f48991f7d0,"olm:{(0.0,1.0)}",,,,1,e0de3267-5ef6-4fcb-83c9-771d53e7fde8 +8bb29769-404b-4bf6-837e-f9ca1b389bf9,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_NET116_S3_2-NS_NET116_S3_3,0.029999999329447746,550ebca7-1455-44eb-9431-ffbf08e58bd4,d69efff2-ba8b-4aa6-a4a1-27267964147a,"olm:{(0.0,1.0)}",,,,1,9db7e41e-7360-4529-bbdb-dbd220117f61 +37c8ebe1-5d29-4806-a84a-fc00e4279fa6,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_NET116_S3_3-NS_NET116_S3_4,0.029999999329447746,d69efff2-ba8b-4aa6-a4a1-27267964147a,36dccefc-f04c-493f-bb88-11343583bf9f,"olm:{(0.0,1.0)}",,,,1,ea01036f-5b25-47dd-98e3-0d944fd2e670 +f0d4491e-eb64-49a0-8fc3-44fa5bd9dcee,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_NET116_S3_4-NS_NET116_S3_5,0.029999999329447746,36dccefc-f04c-493f-bb88-11343583bf9f,b9a28095-68f7-44c1-9ccc-6efc5ea84c59,"olm:{(0.0,1.0)}",,,,1,e2ee2ab1-8e22-41be-9a1f-cfdee99d611e +4bdb6569-72da-4d60-97b1-b5a639032d5d,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_NET116_S1_4-NS_NET116_S1_5,0.029999999329447746,b3a7431d-89b0-41cb-87a9-5853890796cd,92301422-94ae-48ab-89c7-a69eea9450b2,"olm:{(0.0,1.0)}",,,,1,3362b510-7d57-44b1-a6cf-8081aac84a5b +91179f48-2d6c-4ae4-864c-d667f57a6e66,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_NET116_S1_1-NS_NET116_S1_2,0.029999999329447746,8a50fb37-81cf-47c9-8850-a12b4391e2e7,3e21f3a1-2c9c-4138-bcc9-466b004609ed,"olm:{(0.0,1.0)}",,,,1,9312de84-9197-462f-9d69-7d2fea8f7f68 +10c6a103-3132-4324-8f76-8c1858d51fcb,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_NET116_S1_2-NS_NET116_S1_3,0.029999999329447746,3e21f3a1-2c9c-4138-bcc9-466b004609ed,e4c8b492-965c-4b5a-8d14-e8ff2c027ce9,"olm:{(0.0,1.0)}",,,,1,1173fa67-2573-4476-aa5e-c3fe756a7f07 +b714db4a-db55-4957-b503-2300559bb81f,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_NET116_S2_2-NS_NET116_S2_3,0.029999999329447746,d53ff076-dadd-44f8-85d4-68f48991f7d0,df9c3d91-41a0-4e49-bb1f-1cd2a8fd9a2e,"olm:{(0.0,1.0)}",,,,1,567497fb-f77c-4e25-a030-9d40b377711c +d48db7e8-5fde-456f-8b17-02d18a5a2e32,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_NET116_S2_4-NS_NET116_S2_5,0.029999999329447746,e05c68b1-11cd-43fd-a4b2-31e4db380c78,32bd37df-255b-4eb2-9d16-5b711132eee6,"olm:{(0.0,1.0)}",,,,1,b42b9195-4953-403f-8e52-55cc8e653c94 +25a1e96e-b425-48bf-a3ae-dc54ea9da47f,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_NET116_ONS-NS_NET116_S3_1,0.029999999329447746,31a2b9bf-e785-4475-aa44-1c34646e8c79,4a6f6058-e654-464d-9367-2dca7185c6d7,"olm:{(0.0,1.0)}",,,,1,cde5f3a8-4a99-49b7-a7e7-b73e1c6d8003 diff --git a/input/samples/vn_simona_ns/fullGrid/line_type_input.csv b/input/samples/vn_simona_ns/fullGrid/line_type_input.csv new file mode 100644 index 0000000000..fca609822e --- /dev/null +++ b/input/samples/vn_simona_ns/fullGrid/line_type_input.csv @@ -0,0 +1,16 @@ +uuid,b,g,i_max,id,r,v_rated,x +a7071b97-2be4-40bf-a0b1-b8faf6bff6d6,191.636993408203,0.0,265.0,NS_3,0.253899991512299,0.4,0.0691149979829788 +4ee89e0b-1e97-4dde-ba57-3b0b862957c5,185.35400390625,0.0,235.0,NS_2,0.319999992847443,0.4,0.0691149979829788 +cde5f3a8-4a99-49b7-a7e7-b73e1c6d8003,185.35400390625,0.0,235.0,NS_2,0.319999992847443,0.4,0.0691149979829788 +9312de84-9197-462f-9d69-7d2fea8f7f68,163.363006591797,0.0,195.0,NS_1,0.442999988794327,0.4,0.0722566023468971 +1173fa67-2573-4476-aa5e-c3fe756a7f07,191.636993408203,0.0,265.0,NS_3,0.253899991512299,0.4,0.0691149979829788 +b63674e1-6827-4e8b-b3e6-34ee6d4823a9,191.636993408203,0.0,265.0,NS_3,0.253899991512299,0.4,0.0691149979829788 +3362b510-7d57-44b1-a6cf-8081aac84a5b,191.636993408203,0.0,265.0,NS_3,0.253899991512299,0.4,0.0691149979829788 +e0de3267-5ef6-4fcb-83c9-771d53e7fde8,185.35400390625,0.0,235.0,NS_2,0.319999992847443,0.4,0.0691149979829788 +567497fb-f77c-4e25-a030-9d40b377711c,191.636993408203,0.0,265.0,NS_3,0.253899991512299,0.4,0.0691149979829788 +17e87208-d092-4514-8b23-ab91598798ce,163.363006591797,0.0,195.0,NS_1,0.442999988794327,0.4,0.0722566023468971 +b42b9195-4953-403f-8e52-55cc8e653c94,163.363006591797,0.0,195.0,NS_1,0.442999988794327,0.4,0.0722566023468971 +6986321a-0a85-411e-8b2d-9d929a7f298b,163.363006591797,0.0,195.0,NS_1,0.442999988794327,0.4,0.0722566023468971 +9db7e41e-7360-4529-bbdb-dbd220117f61,163.363006591797,0.0,195.0,NS_1,0.442999988794327,0.4,0.0722566023468971 +ea01036f-5b25-47dd-98e3-0d944fd2e670,185.35400390625,0.0,235.0,NS_2,0.319999992847443,0.4,0.0691149979829788 +e2ee2ab1-8e22-41be-9a1f-cfdee99d611e,163.363006591797,0.0,195.0,NS_1,0.442999988794327,0.4,0.0722566023468971 \ No newline at end of file diff --git a/input/samples/vn_simona_ns/fullGrid/load_input.csv b/input/samples/vn_simona_ns/fullGrid/load_input.csv new file mode 100644 index 0000000000..0a0c717150 --- /dev/null +++ b/input/samples/vn_simona_ns/fullGrid/load_input.csv @@ -0,0 +1,16 @@ +uuid,cos_phi_rated,dsm,e_cons_annual,id,load_profile,node,operates_from,operates_until,operator,q_characteristics,s_rated +c3434742-e4f0-49e5-baa7-c1e3045c732c,0.9700000286102295,false,4000.0,NS_NET116_L_S1_5(1),h0,92301422-94ae-48ab-89c7-a69eea9450b2,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605 +40ddb9cf-46f1-4ee6-8feb-432affbc95bf,0.9700000286102295,false,4000.0,NS_NET116_L_S2_4(1),h0,e05c68b1-11cd-43fd-a4b2-31e4db380c78,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605 +a1af0e38-339c-491c-94d8-446cf662d89b,0.9700000286102295,false,4000.0,NS_NET116_L_S3_4(1),h0,36dccefc-f04c-493f-bb88-11343583bf9f,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605 +4642d648-b0dd-4597-a3bd-2cc1fce74f27,0.9700000286102295,false,4000.0,NS_NET116_L_S3_1(1),h0,4a6f6058-e654-464d-9367-2dca7185c6d7,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605 +ff0b995a-86ff-4f4d-987e-e475a64f2180,0.9700000286102295,false,4000.0,NS_NET116_L_S1_2(1),h0,3e21f3a1-2c9c-4138-bcc9-466b004609ed,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605 +2efedcfd-4972-4d13-9d62-180f1b9a76ad,0.9700000286102295,false,4000.0,NS_NET116_L_S2_5(1),h0,32bd37df-255b-4eb2-9d16-5b711132eee6,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605 +fd1a8de9-722a-4304-8799-e1e976d9979c,0.9700000286102295,false,4000.0,NS_NET116_L_S1_1(1),h0,8a50fb37-81cf-47c9-8850-a12b4391e2e7,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605 +f4536a0c-33d2-4c57-918c-0ebdf768265a,0.9700000286102295,false,4000.0,NS_NET116_L_S3_3(1),h0,d69efff2-ba8b-4aa6-a4a1-27267964147a,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605 +7b040b43-011a-4402-a176-9b9b41ce0c37,0.9700000286102295,false,4000.0,NS_NET116_L_S2_3(1),h0,df9c3d91-41a0-4e49-bb1f-1cd2a8fd9a2e,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605 +11e776fa-ecc1-4332-821f-d7e56758b988,0.9700000286102295,false,4000.0,NS_NET116_L_S2_2(1),h0,d53ff076-dadd-44f8-85d4-68f48991f7d0,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605 +98c1a2ab-bd09-4c77-a389-d088aed894b1,0.9700000286102295,false,4000.0,NS_NET116_L_S3_2(1),h0,550ebca7-1455-44eb-9431-ffbf08e58bd4,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605 +fd2e19b6-d5e3-4776-9456-8787a2160d9d,0.9700000286102295,false,4000.0,NS_NET116_L_S1_4(1),h0,b3a7431d-89b0-41cb-87a9-5853890796cd,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605 +a12570e4-afeb-4af8-a86c-9e804eea4886,0.9700000286102295,false,4000.0,NS_NET116_L_S3_5(1),h0,b9a28095-68f7-44c1-9ccc-6efc5ea84c59,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605 +70fcdefd-49c1-455f-9ceb-013dcbf26887,0.9700000286102295,false,4000.0,NS_NET116_L_S1_3(1),h0,e4c8b492-965c-4b5a-8d14-e8ff2c027ce9,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605 +2a9149a4-4778-41d8-8865-090499833016,0.9700000286102295,false,4000.0,NS_NET116_L_S2_1(1),h0,39112046-8953-4e73-a5d9-6a8183a77436,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605 diff --git a/input/samples/vn_simona_ns/fullGrid/node_graphic_input.csv b/input/samples/vn_simona_ns/fullGrid/node_graphic_input.csv new file mode 100644 index 0000000000..1a0280a78d --- /dev/null +++ b/input/samples/vn_simona_ns/fullGrid/node_graphic_input.csv @@ -0,0 +1,17 @@ +uuid,graphic_layer,node,path,point +08d4e2ec-8532-4327-9d9d-aa51393d53e8,Standard,31a2b9bf-e785-4475-aa44-1c34646e8c79,"{""type"":""LineString"",""coordinates"":[[0.03389831,0.27868852],[0.06779661,0.27868852]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}","{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}" +cc657d07-3495-45b8-a90b-76769e14f2d8,Standard,8a50fb37-81cf-47c9-8850-a12b4391e2e7,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}","{""type"":""Point"",""coordinates"":[0.09039548,0.24590164],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}" +0434dc12-6149-4832-a288-6620ee0389f2,Standard,3e21f3a1-2c9c-4138-bcc9-466b004609ed,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}","{""type"":""Point"",""coordinates"":[0.14689266,0.24590164],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}" +f7e1e922-52d3-469f-98c6-5353a5d74641,Standard,e4c8b492-965c-4b5a-8d14-e8ff2c027ce9,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}","{""type"":""Point"",""coordinates"":[0.20338983,0.24590164],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}" +a485471a-ad4f-4ca8-b960-ebf1a5504fd4,Standard,b3a7431d-89b0-41cb-87a9-5853890796cd,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}","{""type"":""Point"",""coordinates"":[0.25423729,0.24590164],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}" +3080afb0-0cdc-4641-ae10-71c05a48fa0f,Standard,92301422-94ae-48ab-89c7-a69eea9450b2,"{""type"":""LineString"",""coordinates"":[[12.8273,52.2895],[12.8273,52.2895]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}","{""type"":""Point"",""coordinates"":[0.31073446,0.24590164],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}" +f17d8fce-ffcf-4214-af18-af2bb64985b0,Standard,39112046-8953-4e73-a5d9-6a8183a77436,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}","{""type"":""Point"",""coordinates"":[0.09039548,0.1557377],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}" +c67ff8a5-d296-47db-970a-4f6b2ab7c77a,Standard,d53ff076-dadd-44f8-85d4-68f48991f7d0,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}","{""type"":""Point"",""coordinates"":[0.14689266,0.1557377],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}" +7bf5033d-efa1-4b45-b885-4c22ede1336d,Standard,df9c3d91-41a0-4e49-bb1f-1cd2a8fd9a2e,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}","{""type"":""Point"",""coordinates"":[0.20338983,0.1557377],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}" +5cb93639-bc3a-4e87-bd02-1d8e16680219,Standard,e05c68b1-11cd-43fd-a4b2-31e4db380c78,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}","{""type"":""Point"",""coordinates"":[0.25423729,0.1557377],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}" +3767a331-5208-4bc1-a8cf-cb4b697fdf0f,Standard,32bd37df-255b-4eb2-9d16-5b711132eee6,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}","{""type"":""Point"",""coordinates"":[0.31073446,0.1557377],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}" +94f85ca1-6313-4083-9f67-4f352b5fee15,Standard,4a6f6058-e654-464d-9367-2dca7185c6d7,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}","{""type"":""Point"",""coordinates"":[0.09039548,0.06557377],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}" +85393fb1-b60f-48ab-b0d5-56002cc2ebe2,Standard,550ebca7-1455-44eb-9431-ffbf08e58bd4,"{""type"":""LineString"",""coordinates"":[[12.8273,52.2895],[12.8273,52.2895]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}","{""type"":""Point"",""coordinates"":[0.14689266,0.06557377],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}" +ffd94a18-e4f0-4088-893a-3d9dae11ddf5,Standard,d69efff2-ba8b-4aa6-a4a1-27267964147a,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}","{""type"":""Point"",""coordinates"":[0.20338983,0.06557377],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}" +416dd5b1-8b68-435b-88da-8ff638754355,Standard,36dccefc-f04c-493f-bb88-11343583bf9f,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}","{""type"":""Point"",""coordinates"":[0.25423729,0.06557377],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}" +2ffc1daf-fd76-4180-8274-6c57ae461e14,Standard,b9a28095-68f7-44c1-9ccc-6efc5ea84c59,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}","{""type"":""Point"",""coordinates"":[0.31073446,0.06557377],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}" \ No newline at end of file diff --git a/input/samples/vn_simona_ns/fullGrid/node_input.csv b/input/samples/vn_simona_ns/fullGrid/node_input.csv new file mode 100644 index 0000000000..72bb318779 --- /dev/null +++ b/input/samples/vn_simona_ns/fullGrid/node_input.csv @@ -0,0 +1,18 @@ +uuid,geo_position,id,operates_from,operates_until,operator,slack,subnet,v_rated,v_target,volt_lvl +39112046-8953-4e73-a5d9-6a8183a77436,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",NS_NET116_S2_1,,,,false,116,0.4,1.0,Niederspannung +b3a7431d-89b0-41cb-87a9-5853890796cd,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",NS_NET116_S1_4,,,,false,116,0.4,1.0,Niederspannung +e05c68b1-11cd-43fd-a4b2-31e4db380c78,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",NS_NET116_S2_4,,,,false,116,0.4,1.0,Niederspannung +31a2b9bf-e785-4475-aa44-1c34646e8c79,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",NS_NET116_ONS,,,,false,116,0.4,1.0,Niederspannung +e4c8b492-965c-4b5a-8d14-e8ff2c027ce9,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",NS_NET116_S1_3,,,,false,116,0.4,1.0,Niederspannung +b9a28095-68f7-44c1-9ccc-6efc5ea84c59,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",NS_NET116_S3_5,,,,false,116,0.4,1.0,Niederspannung +92301422-94ae-48ab-89c7-a69eea9450b2,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}",NS_NET116_S1_5,,,,false,116,0.4,1.0,Niederspannung +4a6f6058-e654-464d-9367-2dca7185c6d7,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",NS_NET116_S3_1,,,,false,116,0.4,1.0,Niederspannung +36dccefc-f04c-493f-bb88-11343583bf9f,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",NS_NET116_S3_4,,,,false,116,0.4,1.0,Niederspannung +8a50fb37-81cf-47c9-8850-a12b4391e2e7,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",NS_NET116_S1_1,,,,false,116,0.4,1.0,Niederspannung +d53ff076-dadd-44f8-85d4-68f48991f7d0,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",NS_NET116_S2_2,,,,false,116,0.4,1.0,Niederspannung +3e21f3a1-2c9c-4138-bcc9-466b004609ed,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",NS_NET116_S1_2,,,,false,116,0.4,1.0,Niederspannung +550ebca7-1455-44eb-9431-ffbf08e58bd4,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}",NS_NET116_S3_2,,,,false,116,0.4,1.0,Niederspannung +32bd37df-255b-4eb2-9d16-5b711132eee6,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",NS_NET116_S2_5,,,,false,116,0.4,1.0,Niederspannung +df9c3d91-41a0-4e49-bb1f-1cd2a8fd9a2e,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",NS_NET116_S2_3,,,,false,116,0.4,1.0,Niederspannung +d69efff2-ba8b-4aa6-a4a1-27267964147a,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",NS_NET116_S3_3,,,,false,116,0.4,1.0,Niederspannung +b22c9299-5fb4-46a7-b566-fc17e0d51d60,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",MS1_06,,,,true,11,20.0,1.0,Mittelspannung \ No newline at end of file diff --git a/input/samples/vn_simona_ns/fullGrid/pv_input.csv b/input/samples/vn_simona_ns/fullGrid/pv_input.csv new file mode 100644 index 0000000000..64bcc385e9 --- /dev/null +++ b/input/samples/vn_simona_ns/fullGrid/pv_input.csv @@ -0,0 +1,3 @@ +uuid,albedo,azimuth,cos_phi_rated,elevation_angle,eta_conv,id,k_g,k_t,market_reaction,node,operates_from,operates_until,operator,q_characteristics,s_rated +de8cfef5-7620-4b9e-9a10-1faebb5a80c0,0.20000000298023224,4.093344211578369,0.8999999761581421,37.69556427001953,97.0,NS_NET116_S1_5_PV,0.8999999761581421,1.0,false,92301422-94ae-48ab-89c7-a69eea9450b2,,,,"cosPhiFixed:{(0.0,0.9)}",30.0 +2560c371-f420-4c2a-b4e6-e04c11b64c03,0.20000000298023224,0.7802008390426636,0.8999999761581421,40.086585998535156,98.0,NS_NET116_S3_2_PV,0.8999999761581421,1.0,false,550ebca7-1455-44eb-9431-ffbf08e58bd4,,,,"cosPhiFixed:{(0.0,0.9)}",30.0 diff --git a/input/samples/vn_simona_ns/fullGrid/transformer_2_w_input.csv b/input/samples/vn_simona_ns/fullGrid/transformer_2_w_input.csv new file mode 100644 index 0000000000..592200c023 --- /dev/null +++ b/input/samples/vn_simona_ns/fullGrid/transformer_2_w_input.csv @@ -0,0 +1,2 @@ +uuid,auto_tap,id,node_a,node_b,operates_from,operates_until,operator,parallel_devices,tap_pos,type +2cbbd48e-e6d8-40b4-80cc-010157dfbf3e,false,TRA_MS1_06-NS_NET116,b22c9299-5fb4-46a7-b566-fc17e0d51d60,31a2b9bf-e785-4475-aa44-1c34646e8c79,,,,1,10,4984f493-d6e5-4201-a040-c10722b30362 diff --git a/input/samples/vn_simona_ns/fullGrid/transformer_2_w_type_input.csv b/input/samples/vn_simona_ns/fullGrid/transformer_2_w_type_input.csv new file mode 100644 index 0000000000..1140eb9b9b --- /dev/null +++ b/input/samples/vn_simona_ns/fullGrid/transformer_2_w_type_input.csv @@ -0,0 +1,2 @@ +uuid,b_m,d_phi,d_v,g_m,id,r_sc,s_rated,tap_max,tap_min,tap_neutr,tap_side,v_rated_a,v_rated_b,x_sc +4984f493-d6e5-4201-a040-c10722b30362,0.0,0.0,0.5,0.0,MS-NS_1,10.078,630.0,10,-10,0,false,20.0,0.4,23.312 diff --git a/input/samples/vn_simona_ns/vn_simona_ns.conf b/input/samples/vn_simona_ns/vn_simona_ns.conf new file mode 100644 index 0000000000..224e48981f --- /dev/null +++ b/input/samples/vn_simona_ns/vn_simona_ns.conf @@ -0,0 +1,214 @@ +include "../common/pekko.conf" + +######### +# ATTENTION: Do not change this config file directly but use it as a base for your personal delta config for the +# vn_simona scenario! Delta configs can be created by including the config you want to change +# parameters from via include (e.g. include "input/samples/vn_simona/vn_simona.conf") at the +# beginning of your config file and then just override the parameters you want to change! +######### + +################################################################## +# Simulation Parameters +################################################################## +simona.simulationName = "vn_simona_ns" + +################################################################## +# Time Parameters +################################################################## +simona.time.startDateTime = "2020-09-01T00:00:00Z" +simona.time.endDateTime = "2020-09-02T00:00:00Z" +simona.time.schedulerReadyCheckWindow = 900 + +################################################################## +# Input Parameters +################################################################## +simona.input.primary.csvParams = { + directoryPath: "input/samples/vn_simona_ns/fullGrid" + csvSep: "," + isHierarchic: false +} +simona.input.grid.datasource.id = "csv" +simona.input.grid.datasource.csvParams = { + directoryPath: "input/samples/vn_simona_ns/fullGrid" + csvSep: "," + isHierarchic: false +} + +simona.input.weather.datasource = { + scheme = "icon" + sampleParams.use = true + coordinateSource.sampleParams.use = true + maxCoordinateDistance = 50000 +} + +################################################################## +# Output Parameters +################################################################## +simona.output.base.dir = "output/vn_simona_ns" +simona.output.base.addTimestampToOutputDir = true + +simona.output.sink.csv { + fileFormat = ".csv" + filePrefix = "" + fileSuffix = "" +} + +simona.output.grid = { + notifier = "grid" + nodes = false + lines = false + switches = false + transformers2w = false + transformers3w = false +} +simona.output.participant.defaultConfig = { + notifier = "default" + powerRequestReply = false + simulationResult = true +} +simona.output.participant.individualConfigs = [ + { + notifier = "pv" + powerRequestReply = false + simulationResult = true + }, + { + notifier = "wec" + powerRequestReply = false + simulationResult = false + }, + { + notifier = "evcs" + powerRequestReply = false + simulationResult = false + }, + { + notifier = "bm" + powerRequestReply = false + simulationResult = false + }, + { + notifier = "chp" + powerRequestReply = false + simulationResult = false + }, + { + notifier = "ev" + powerRequestReply = false + simulationResult = false + }, + { + notifier = "hp" + powerRequestReply = false + simulationResult = false + }, + { + notifier = "storage" + powerRequestReply = false + simulationResult = false + }, + { + notifier = "fixedFeedIn" + powerRequestReply = false + simulationResult = false + } +] +simona.output.thermal = { + defaultConfig = { + notifier = "default", + simulationResult = false + } + individualConfigs = [ + { + notifier = "house", + simulationResult = false + } + ] +} + +################################################################## +# Runtime Configuration // todo refactor as this naming is misleading and partly unneeded +################################################################## +simona.runtime.selected_subgrids = [] +simona.runtime.selected_volt_lvls = [] + +simona.runtime.participant.load = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + modelBehaviour = "fix" + reference = "power" + } + individualConfigs = [] +} + +simona.runtime.participant.fixedFeedIn = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +simona.runtime.participant.pv = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +simona.runtime.participant.wec = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +simona.runtime.participant.evcs = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +simona.runtime.participant.hp = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +################################################################## +# Event Configuration +################################################################## +simona.event.listener = [] + +################################################################## +# Grid Configuration +################################################################## + +simona.gridConfig.refSystems = [ + {sNom = "100 kVA", vNom = "0.4 kV", voltLvls = [{id = "NS", vNom = "0.4 kV"}]}, + {sNom = "60 MVA", vNom = "20 kV", voltLvls = [{id = "MS", vNom = "20 kV"}]}, + {sNom = "600 MVA", vNom = "110 kV", voltLvls = [{id = "HS", vNom = "110 kV"}]}, + {sNom = "1000 MVA", vNom = "380 kV", voltLvls = [{id = "HoeS", vNom = "380 kV"}]} +] + +################################################################## +# Power Flow Configuration +################################################################## +simona.powerflow.maxSweepPowerDeviation = 1E-5 // the maximum allowed deviation in power between two sweeps, before overall convergence is assumed +simona.powerflow.newtonraphson.epsilon = [1E-12] +simona.powerflow.newtonraphson.iterations = 50 +simona.powerflow.resolution = "3600s" +simona.powerflow.stopOnFailure = true From e8bdad6db107040e0f895786301f298faa30429f Mon Sep 17 00:00:00 2001 From: Johannes Bao Date: Tue, 19 Mar 2024 23:16:02 +0100 Subject: [PATCH 21/41] fine tuning --- .../sim/setup/SimonaStandaloneSetup.scala | 49 ++++++------------- 1 file changed, 15 insertions(+), 34 deletions(-) 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 eba8ca48e4..ccdf35cfe5 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala @@ -45,7 +45,7 @@ import edu.ie3.simona.sim.SimonaSim import edu.ie3.simona.util.ResultFileHierarchy import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK import edu.ie3.simona.util.TickUtil.RichZonedDateTime -//import edu.ie3.simosaik.simpleextsim.SimpleExtSimulation +import edu.ie3.simosaik.simpleextsim.SimpleExtSimulation import edu.ie3.util.TimeUtil import org.apache.pekko.actor.typed.scaladsl.ActorContext import org.apache.pekko.actor.typed.scaladsl.AskPattern._ @@ -196,8 +196,7 @@ class SimonaStandaloneSetup( weatherService } - - +/* override def extSimulations( context: ActorContext[_], theScheduler: ActorRef[SchedulerMessage], @@ -206,8 +205,6 @@ class SimonaStandaloneSetup( val extLinks = jars.flatMap(ExtSimLoader.loadExtLink) - context.log.info(s"extLinks = $extLinks") - val (extSimAdapters, extDatasAndServices) = extLinks.zipWithIndex.map { case (extLink, index) => // external simulation always needs at least an ExtSimAdapter @@ -259,6 +256,8 @@ class SimonaStandaloneSetup( extPrimaryDataSimulation.getPrimaryDataAssets ) + extPrimaryDataSimulation.setExtPrimaryData(extPrimaryData) + extPrimaryDataService ! SimonaService.Create( InitExtPrimaryData(extPrimaryData), ScheduleLock.singleKey( @@ -296,7 +295,7 @@ class SimonaStandaloneSetup( extResultDataSimulation.getResultDataAssets ) - context.log.info(s"extResultData $extResultData") + extResultDataSimulation.setExtResultData(extResultData) extResultDataProvider ! ExtResultDataProvider.Create( InitExtResultData(extResultData), @@ -306,23 +305,6 @@ class SimonaStandaloneSetup( INIT_SIM_TICK, ), ) - /* - val extResultDataService = context.toClassic.simonaActorOf( - ExtResultDataProvider.props(scheduler.toClassic), - s"$index-$dIndex", - ) - val extResultsData = - new ExtResultsData(extResultDataService, extSimAdapter, null) - - extResultDataService ! SimonaService.Create( - InitExtResultsData(extResultsData), - ScheduleLock.singleKey( - context, - scheduler, - INIT_SIM_TICK, - ), - ) - */ ( extResultData, @@ -343,13 +325,11 @@ class SimonaStandaloneSetup( val extDataServices = extDatasAndServices.map(_._1) val extDatas = extDatasAndServices.flatMap(_._2).toSet - context.log.info(s"extDatasAndServices = $extDatasAndServices") + ExtSimSetupData(extSimAdapters, extDataServices.flatten.toMap, extDatas) } - - /* - - def extTestSimulations( +*/ + override def extSimulations( context: ActorContext[_], theScheduler: ActorRef[SchedulerMessage], ): ExtSimSetupData = { @@ -374,10 +354,12 @@ class SimonaStandaloneSetup( val extPrimaryData = new ExtPrimaryData( extPrimaryDataService, extSimAdapter, - simpleExtSim.getPrimaryDataFactory, - simpleExtSim.getPrimaryDataAssets + simpleExtSim.getExtPrimaryDataSimulation.getPrimaryDataFactory, + simpleExtSim.getExtPrimaryDataSimulation.getPrimaryDataAssets ) + simpleExtSim.getExtPrimaryDataSimulation.setExtPrimaryData(extPrimaryData) + extPrimaryDataService ! SimonaService.Create( InitExtPrimaryData(extPrimaryData), ScheduleLock.singleKey( @@ -408,11 +390,11 @@ class SimonaStandaloneSetup( adapterRef.toClassic, adapterScheduleRef.toClassic, extSimAdapter, - simpleExtSim.getResultDataFactory, - simpleExtSim.getResultDataAssets + simpleExtSim.getExtResultDataSimulation.getResultDataFactory, + simpleExtSim.getExtResultDataSimulation.getResultDataAssets ) - context.log.info(s"extResultData $extResultData") + simpleExtSim.getExtResultDataSimulation.setExtResultData(extResultData) extResultDataProvider ! ExtResultDataProvider.Create( InitExtResultData(extResultData), @@ -449,7 +431,6 @@ class SimonaStandaloneSetup( ExtSimSetupData(extSimAdapters, extDataServicesMap, extDatas) } - */ override def timeAdvancer( context: ActorContext[_], From 1f847936f834a1b2e9865a7b336c6d1536ced0be Mon Sep 17 00:00:00 2001 From: Johannes Bao Date: Fri, 22 Mar 2024 15:24:01 +0100 Subject: [PATCH 22/41] temp --- input/samples/vn_simona_ns/vn_simona_ns.conf | 28 +- .../edu/ie3/simona/api/ExtSimAdapter.scala | 2 + .../event/listener/DelayedStopHelper.scala | 4 +- .../event/listener/ResultEventListener.scala | 25 +- .../ie3/simona/main/RunSimonaStandalone.scala | 2 - .../edu/ie3/simona/scheduler/Scheduler.scala | 4 +- .../results/ExtResultDataProvider.scala | 91 +---- .../scala/edu/ie3/simona/sim/SimonaSim.scala | 29 +- .../simona/sim/setup/ExtSimSetupData.scala | 18 +- .../ie3/simona/sim/setup/SimonaSetup.scala | 1 + .../sim/setup/SimonaStandaloneSetup.scala | 316 ++++++++++-------- 11 files changed, 249 insertions(+), 271 deletions(-) diff --git a/input/samples/vn_simona_ns/vn_simona_ns.conf b/input/samples/vn_simona_ns/vn_simona_ns.conf index 224e48981f..016d7ddb60 100644 --- a/input/samples/vn_simona_ns/vn_simona_ns.conf +++ b/input/samples/vn_simona_ns/vn_simona_ns.conf @@ -16,7 +16,7 @@ simona.simulationName = "vn_simona_ns" # Time Parameters ################################################################## simona.time.startDateTime = "2020-09-01T00:00:00Z" -simona.time.endDateTime = "2020-09-02T00:00:00Z" +simona.time.endDateTime = "2020-09-01T01:00:00Z" simona.time.schedulerReadyCheckWindow = 900 ################################################################## @@ -188,6 +188,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 ################################################################## @@ -212,3 +224,17 @@ simona.powerflow.newtonraphson.epsilon = [1E-12] simona.powerflow.newtonraphson.iterations = 50 simona.powerflow.resolution = "3600s" simona.powerflow.stopOnFailure = true + +simona.control.transformer = [ + { + transformers = ["31a2b9bf-e785-4475-aa44-1c34646e8c79"], + measurements = ["923f2d69-3093-4198-86e4-13d2d1c220f8"], + vMin = 0.98, + vMax = 1.02 + }, { + transformers = ["1132dbf4-e8a1-44ae-8415-f42d4497aa1d"], + measurements = ["7686b818-a0ba-465c-8e4e-f7d3c4e171fc"], + vMin = 0.98, + vMax = 1.02 + } +] diff --git a/src/main/scala/edu/ie3/simona/api/ExtSimAdapter.scala b/src/main/scala/edu/ie3/simona/api/ExtSimAdapter.scala index 64ff24eaf6..2bbb090880 100644 --- a/src/main/scala/edu/ie3/simona/api/ExtSimAdapter.scala +++ b/src/main/scala/edu/ie3/simona/api/ExtSimAdapter.scala @@ -71,6 +71,7 @@ final case class ExtSimAdapter(scheduler: ActorRef) "Tick {} has been activated in external simulation", tick, ) + log.info("stateData after Activation(" + tick + ") = " + stateData) context become receiveIdle( stateData.copy(currentTick = Some(tick)) @@ -91,6 +92,7 @@ final case class ExtSimAdapter(scheduler: ActorRef) context become receiveIdle(stateData.copy(currentTick = None)) case scheduleDataService: ScheduleDataServiceMessage => + log.info("stateData after ScheduleDataService = " + stateData) val tick = stateData.currentTick.getOrElse( throw new RuntimeException("No tick has been triggered") ) diff --git a/src/main/scala/edu/ie3/simona/event/listener/DelayedStopHelper.scala b/src/main/scala/edu/ie3/simona/event/listener/DelayedStopHelper.scala index 3c3a84f3ca..13225a4dcf 100644 --- a/src/main/scala/edu/ie3/simona/event/listener/DelayedStopHelper.scala +++ b/src/main/scala/edu/ie3/simona/event/listener/DelayedStopHelper.scala @@ -39,7 +39,7 @@ object DelayedStopHelper { : PartialFunction[(ActorContext[T], StoppingMsg), Behavior[T]] = { case (ctx, FlushAndStop) => - ctx.log.debug( + ctx.log.info( s"Received FlushAndStop message, shutting down once no message has been received for 5 seconds." ) ctx.setReceiveTimeout(5.seconds, StopTimeout) @@ -47,7 +47,7 @@ object DelayedStopHelper { case (ctx, StopTimeout) => // there have been no messages for 5 seconds, let's end this - ctx.log.debug(s"${getClass.getSimpleName} is now stopped.") + ctx.log.info(s"${getClass.getSimpleName} is now stopped.") Behaviors.stopped } diff --git a/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala b/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala index 1a3fe0ea9e..aeb415f121 100644 --- a/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala +++ b/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala @@ -7,25 +7,16 @@ package edu.ie3.simona.event.listener import org.apache.pekko.actor.typed.scaladsl.Behaviors -import org.apache.pekko.actor.typed.{Behavior, PostStop} +import org.apache.pekko.actor.typed.{ActorRef, Behavior, PostStop} import edu.ie3.datamodel.io.processor.result.ResultEntityProcessor import edu.ie3.datamodel.models.result.{NodeResult, ResultEntity} import edu.ie3.simona.agent.grid.GridResultsSupport.PartialTransformer3wResult -import edu.ie3.simona.event.ResultEvent.{ - FlexOptionsResultEvent, - ParticipantResultEvent, - PowerFlowResultEvent, - ThermalResultEvent, -} -import edu.ie3.simona.exceptions.{ - FileHierarchyException, - ProcessResultEventException, -} +import edu.ie3.simona.event.ResultEvent.{FlexOptionsResultEvent, ParticipantResultEvent, PowerFlowResultEvent, ThermalResultEvent} +import edu.ie3.simona.exceptions.{FileHierarchyException, ProcessResultEventException} import edu.ie3.simona.io.result._ +import edu.ie3.simona.service.results.ExtResultDataProvider import edu.ie3.simona.service.results.ExtResultDataProvider.ResultResponseMessage - import edu.ie3.simona.util.ResultFileHierarchy -import org.apache.pekko.actor.ActorRef import org.slf4j.Logger import scala.concurrent.ExecutionContext.Implicits.global @@ -54,7 +45,7 @@ object ResultEventListener extends Transformer3wResultSupport { */ private final case class BaseData( classToSink: Map[Class[_], ResultEntitySink], - extResultDataService: Option[ActorRef], + extResultDataService: Option[ActorRef[ExtResultDataProvider.Request]], threeWindingResults: Map[ Transformer3wKey, AggregatedTransformer3wResult, @@ -244,7 +235,7 @@ object ResultEventListener extends Transformer3wResultSupport { private def handOverToExternalService( resultEntity: ResultEntity, - extResultDataService: Option[ActorRef], + extResultDataService: Option[ActorRef[ExtResultDataProvider.Request]], nextTick: Option[Long] = None ): Unit = Try { val extResultDataServiceRef = extResultDataService.getOrElse( @@ -255,7 +246,7 @@ object ResultEventListener extends Transformer3wResultSupport { def apply( resultFileHierarchy: ResultFileHierarchy, - extResultDataService: Option[ActorRef] = Option.empty[ActorRef], + extResultDataService: Option[ActorRef[ExtResultDataProvider.Request]] = Option.empty[ActorRef[ExtResultDataProvider.Request]], ): Behavior[Request] = Behaviors.setup[Request] { ctx => ctx.log.debug("Starting initialization!") resultFileHierarchy.resultSinkType match { @@ -283,7 +274,7 @@ object ResultEventListener extends Transformer3wResultSupport { } private def init( - extResultDataService: Option[ActorRef] + extResultDataService: Option[ActorRef[ExtResultDataProvider.Request]] ): Behavior[Request] = Behaviors.withStash(200) { buffer => Behaviors.receive[Request] { case (ctx, SinkResponse(response)) => diff --git a/src/main/scala/edu/ie3/simona/main/RunSimonaStandalone.scala b/src/main/scala/edu/ie3/simona/main/RunSimonaStandalone.scala index d0a50802b8..b8862655fa 100644 --- a/src/main/scala/edu/ie3/simona/main/RunSimonaStandalone.scala +++ b/src/main/scala/edu/ie3/simona/main/RunSimonaStandalone.scala @@ -58,7 +58,5 @@ object RunSimonaStandalone extends RunSimona[SimonaStandaloneSetup] { successful } - } - } diff --git a/src/main/scala/edu/ie3/simona/scheduler/Scheduler.scala b/src/main/scala/edu/ie3/simona/scheduler/Scheduler.scala index 7339fbbe61..303968e597 100644 --- a/src/main/scala/edu/ie3/simona/scheduler/Scheduler.scala +++ b/src/main/scala/edu/ie3/simona/scheduler/Scheduler.scala @@ -122,6 +122,7 @@ object Scheduler { toActivate.foreach { _ ! Activation(newCore.activeTick) } + //println(this + " ScheduleActivation from " + actor + ", newTick = " + newTick + ", newCore: " + newCore) active(data, newCore) @@ -137,13 +138,14 @@ object Scheduler { toActivate.foreach { _ ! Activation(updatedCore.activeTick) } - + println(this + " Completion from " + actor + ", maybeNextTick = " + maybeNewTick + ", updatedCore: " + updatedCore) updatedCore } .map { newCore => newCore .maybeComplete() .map { case (maybeScheduleTick, inactiveCore) => + println(this + " Send Completion from " + data.activationAdapter + " to " + data.parent) data.parent ! Completion( data.activationAdapter, maybeScheduleTick, diff --git a/src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala b/src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala index ae192b12d2..77420a96f2 100644 --- a/src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala +++ b/src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala @@ -11,7 +11,7 @@ import edu.ie3.simona.ontology.messages.{Activation, SchedulerMessage} import edu.ie3.simona.scheduler.ScheduleLock.ScheduleKey import edu.ie3.simona.util.ReceiveDataMap import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK -import org.apache.pekko.actor.typed.{ActorRef, Behavior} +import org.apache.pekko.actor.typed.{ActorRef, Behavior, PostStop} import org.apache.pekko.actor.typed.scaladsl.{Behaviors, StashBuffer} import java.util.UUID @@ -128,8 +128,9 @@ object ExtResultDataProvider { ): Behavior[Request] = Behaviors .receivePartial[Request] { case (ctx, WrappedActivation(activation: Activation)) => - //info("Received Activation") + //ctx.log.info("Received Activation") var updatedStateData = serviceStateData + serviceStateData.extResultsMessage.getOrElse( throw ServiceException( "ExtResultDataService was triggered without ResultDataMessageFromExt available" @@ -162,7 +163,7 @@ object ExtResultDataProvider { throw new RuntimeException("There is no result!") ) ) - //log.info(s"[requestResults] tick $tick -> ReceiveDataMap is complete -> send it right away: " + resultList) + //ctx.log.info(s"[requestResults] tick ${msg.tick} -> ReceiveDataMap is complete -> send it right away: " + resultList) // all responses received, forward them to external simulation in a bundle serviceStateData.extResultData.queueExtResponseMsg( new ProvideResultEntities(resultList.asJava) @@ -172,6 +173,7 @@ object ExtResultDataProvider { recentResults = None) } else { + //ctx.log.info(s"[requestResults] receiveDataMap was built -> now sending ResultRequestMessage") ctx.self ! ResultRequestMessage(msg.tick) updatedStateData = serviceStateData.copy( extResultsMessage = None, @@ -191,13 +193,16 @@ object ExtResultDataProvider { ) Behaviors.same - case (_, extRequestResultEntitiesMsg: WrappedResultDataMessageFromExt) => + case (ctx, extRequestResultEntitiesMsg: WrappedResultDataMessageFromExt) => + //ctx.log.info("Received WrappedResultDataMessageFromExt") idle( serviceStateData.copy( extResultsMessage = Some(extRequestResultEntitiesMsg.extResultDataMessageFromExt) )) - case (_, extResultResponseMsg: ResultResponseMessage) => + case (ctx, extResultResponseMsg: ResultResponseMessage) => + //ctx.log.info("[handleDataResponseMessage] Received ResultsResponseMessage") + if (serviceStateData.recentResults.isDefined) { // process dataResponses if (serviceStateData.subscribers.contains(extResultResponseMsg.result.getInputModel)) { @@ -243,82 +248,6 @@ object ExtResultDataProvider { DelayedStopHelper.handleMsg((ctx, msg)) } - // -=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=- - /* - protected def announceInformation( - tick: Long - )(implicit - serviceStateData: ExtResultStateData, - activationAdapter: ActorRef[Activation], - resultDataMessageFromExtAdapter: ActorRef[ResultDataMessageFromExt], - buffer: StashBuffer[Request] - ): (ExtResultStateData, Option[Long]) = { - serviceStateData.extResultsMessage.getOrElse( - throw ServiceException( - "ExtResultDataService was triggered without ResultDataMessageFromExt available" - ) - ) match { - case _: RequestResultEntities => - requestResults(tick) - } - } - - private def requestResults( - tick: Long - )(implicit - serviceStateData: ExtResultStateData, - activationAdapter: ActorRef[Activation], - buffer: StashBuffer[Request] - ): (ExtResultStateData, Option[Long]) = { - //log.info(s"[requestResults] for tick $tick and resultStorage ${serviceStateData.resultStorage}") - var receiveDataMap = ReceiveDataMap[UUID, ResultEntity](serviceStateData.subscribers.toSet) - //log.info(s"[requestResults] tick $tick -> created a receivedatamap " + receiveDataMap) - serviceStateData.resultStorage.foreach({ - case (uuid, (res, t)) => - //log.info(s"[requestResults] tick = $tick, uuid = $uuid, and time = ${t.getOrElse("Option")}, result = ${res.getOrElse("Option")}") - if (t.getOrElse(-1) != tick) { //wenn nicht in diesem Tick gefragt, nehme Wert aus ResultDataStorage - receiveDataMap = receiveDataMap.addData( - uuid, - res.getOrElse( - throw new Exception("noResult") - ) - ) - //log.info(s"[requestResults] tick $tick -> added to receivedatamap " + receiveDataMap) - } - }) - - //log.info(s"[requestResults] tick $tick -> requestResults for " + receiveDataMap) - - if (receiveDataMap.isComplete) { - var resultList = List.empty[ResultEntity] - serviceStateData.resultStorage.values.foreach( - result => resultList = resultList :+ result._1.getOrElse( - throw new RuntimeException("There is no result!") - ) - ) - //log.info(s"[requestResults] tick $tick -> ReceiveDataMap is complete -> send it right away: " + resultList) - // all responses received, forward them to external simulation in a bundle - serviceStateData.extResultData.queueExtResponseMsg( - new ProvideResultEntities(resultList.asJava) - ) - (serviceStateData.copy( - extResultsMessage = None, - recentResults = None), None) - - } else { - ( - serviceStateData.copy( - extResultsMessage = None, - recentResults = Some(receiveDataMap) - ), None) - } - } - - */ - - - - // -=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=- final case class ExtResultStateData( extResultData: ExtResultData, diff --git a/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala b/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala index f41868fbae..65847b0b18 100644 --- a/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala +++ b/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala @@ -19,8 +19,6 @@ import org.apache.pekko.actor.typed.scaladsl.adapter._ import org.apache.pekko.actor.typed.scaladsl.{ActorContext, Behaviors} import org.apache.pekko.actor.typed.{ActorRef, Behavior, PostStop, Terminated} import org.apache.pekko.actor.{ActorRef => ClassicRef} -import org.apache.pekko.actor.typed.scaladsl.adapter.ClassicActorRefOps -import org.apache.pekko.actor.typed.{ActorRef => TypedActorRef} /** Main entrance point to a simona simulation as the guardian actor. This actor * starts the initialization of all actors and waits for the simulation to end. @@ -76,24 +74,21 @@ object SimonaSim { val timeAdvancer = simonaSetup.timeAdvancer(ctx, ctx.self, runtimeEventListener) + 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 */ - // Load external simulations + // External simulations have to be scheduled for initialization first, + // so that the phase switch permanently activates them first val extSimulationData: ExtSimSetupData = - simonaSetup.extSimulations(ctx, scheduler) + simonaSetup.extSimulations(ctx, rootPhaseSwitch, simScheduler) + /* start services */ // primary service proxy val primaryServiceProxy = simonaSetup.primaryServiceProxy(ctx, simScheduler, extSimulationData) @@ -128,21 +123,25 @@ object SimonaSim { weatherService.toTyped, ) ++ gridAgents ++ - //extSimulationData.extDataServices.values.map(_.toTyped) ++ + extSimulationData.extDataServices.values.map(_.toTyped) ++ extSimulationData.extScheduler.toSeq /* watch all actors */ resultEventListeners.foreach(ctx.watch) ctx.watch(runtimeEventListener) + ctx.watch(extSimulationData.extResultDataService.getOrElse(throw new Exception(""))) extSimulationData.extSimAdapters.map(_.toTyped).foreach(ctx.watch) otherActors.foreach(ctx.watch) // Start simulation timeAdvancer ! TimeAdvancer.Start() - val delayedActors = resultEventListeners.appended(runtimeEventListener).appended(extSimulationData.extResultDataService.getOrElse( - throw new Exception("Problem!") - ).toTyped[DelayedStopHelper.StoppingMsg]) + val delayedActors = resultEventListeners.appended(runtimeEventListener) + .appended( + extSimulationData.extResultDataService.getOrElse( + throw new Exception("Problem!") + ) + ) idle( ActorData( @@ -217,6 +216,8 @@ object SimonaSim { ref ! ExtSimAdapter.Stop(simulationSuccessful) } + //ctx.log.info(s"delayedStoppingActors = ${actorData.delayedStoppingActors}") + // if the simulation is successful, we're waiting for the delayed // stopping listeners to terminate and thus do not unwatch them here actorData.delayedStoppingActors.foreach( 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 81dccff52d..dfd55222a5 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/ExtSimSetupData.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/ExtSimSetupData.scala @@ -14,26 +14,26 @@ import edu.ie3.simona.api.data.primarydata.ExtPrimaryData import edu.ie3.simona.api.data.results.ExtResultData import edu.ie3.simona.service.ev.ExtEvDataService import org.apache.pekko.actor.typed.ActorRef -import org.apache.pekko.actor.typed.ActorRef import edu.ie3.simona.service.primary.ExtPrimaryDataService import edu.ie3.simona.service.results.ExtResultDataProvider -import org.apache.pekko.actor.ActorRef +import edu.ie3.simona.service.results.ExtResultDataProvider.Request final case class ExtSimSetupData( - extSimAdapters: Iterable[ClassicRef], - extDataServices: Map[Class[_], ClassicRef], - extDatas: Set[ExtData], - extScheduler: Option[ActorRef[SchedulerMessage]], + extSimAdapters: Iterable[ClassicRef], + extDataServices: Map[Class[_], ClassicRef], + extDataListener: Map[Class[_], ActorRef[ExtResultDataProvider.Request]], + extDatas: Set[ExtData], + extScheduler: Option[ActorRef[SchedulerMessage]], ) { def evDataService: Option[ClassicRef] = extDataServices.get(classOf[ExtEvDataService]) - def extPrimaryDataService: Option[ActorRef] = + def extPrimaryDataService: Option[ClassicRef] = extDataServices.get(classOf[ExtPrimaryDataService]) - def extResultDataService: Option[ActorRef] = - extDataServices.get(ExtResultDataProvider.getClass) + def extResultDataService: Option[ActorRef[ExtResultDataProvider.Request]] = + extDataListener.get(ExtResultDataProvider.getClass) def extEvData: Option[ExtEvData] = { extDatas.collectFirst { case extData: ExtEvData => extData } 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 d267742ad4..45319c119f 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaSetup.scala @@ -105,6 +105,7 @@ trait SimonaSetup { def extSimulations( context: ActorContext[_], rootScheduler: ActorRef[SchedulerMessage], + simScheduler: ActorRef[SchedulerMessage], ): ExtSimSetupData /** Creates the time advancer 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 0aee4347cd..122021c734 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala @@ -55,6 +55,7 @@ import org.apache.pekko.actor.typed.scaladsl.adapter.{ClassicActorRefOps, TypedA import org.apache.pekko.actor.typed.{ActorRef, Scheduler} import org.apache.pekko.actor.{ActorRef => ClassicRef} import org.apache.pekko.util.{Timeout => PekkoTimeout} +import edu.ie3.simona.service.results.ExtResultDataProvider.Request import java.util.UUID import java.util.concurrent.LinkedBlockingQueue @@ -199,7 +200,8 @@ class SimonaStandaloneSetup( weatherService } -/* + + /* override def extSimulations( context: ActorContext[_], rootScheduler: ActorRef[SchedulerMessage], @@ -211,163 +213,181 @@ class SimonaStandaloneSetup( if (extLinks.nonEmpty) { val extScheduler = scheduler(context, parent = rootScheduler) - val (extSimAdapters, extDatasAndServices) = - 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) + val (extSimAdapters, extDatasAndServices) = + 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) - // send init data right away, init activation is scheduled - extSimAdapter ! ExtSimAdapter.Create( - extSimAdapterData, - ScheduleLock.singleKey(context, extScheduler, INIT_SIM_TICK), - ) + // 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, extDataServiceToRef): ( - Iterable[ExtData], - Iterable[(Class[_], 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)) - - case (extPrimaryDataSimulation: ExtPrimaryDataSimulation, dIndex) => - val extPrimaryDataService = context.toClassic.simonaActorOf( - ExtPrimaryDataService.props(theScheduler.toClassic), - s"$index-$dIndex", - ) - val extPrimaryData = new ExtPrimaryData( - extPrimaryDataService, - extSimAdapter, - extPrimaryDataSimulation.getPrimaryDataFactory, - extPrimaryDataSimulation.getPrimaryDataAssets - ) - - extPrimaryDataSimulation.setExtPrimaryData(extPrimaryData) - - extPrimaryDataService ! SimonaService.Create( - InitExtPrimaryData(extPrimaryData), - ScheduleLock.singleKey( - context, - extScheduler, - INIT_SIM_TICK, - ), - ) - - (extPrimaryData, (classOf[ExtPrimaryDataService], extPrimaryDataService)) - - case (extResultDataSimulation: ExtResultDataSimulation, dIndex) => - - val extResultDataProvider = { - context.spawn( - ExtResultDataProvider(theScheduler), + // setup data services that belong to this external simulation + val (extData, extDataServiceToRef): ( + Iterable[ExtData], + Iterable[(Class[_], ClassicRef)], + ) = + extLink.getExtDataSimulations.asScala.zipWithIndex.map { + case (_: ExtEvSimulation, dIndex) => + val extEvDataService = context.toClassic.simonaActorOf( + ExtEvDataService.props(extScheduler.toClassic), s"$index-$dIndex", ) - } - - implicit val timeout: PekkoTimeout = PekkoTimeout.create(5.seconds.toJava) - implicit val scheduler: Scheduler = context.system.scheduler + val extEvData = new ExtEvData(extEvDataService, extSimAdapter) + + extEvDataService ! SimonaService.Create( + InitExtEvData(extEvData), + ScheduleLock.singleKey( + context, + extScheduler, + INIT_SIM_TICK, + ), + ) - val adapterRef = Await.result( - extResultDataProvider.ask[ActorRef[ResultDataMessageFromExt]] (ref => RequestDataMessageAdapter(ref)), timeout.duration) - val adapterScheduleRef = Await.result( - extResultDataProvider.ask[ActorRef[ScheduleServiceActivation]] (ref => RequestScheduleActivationAdapter(ref)), timeout.duration) + (extEvData, (classOf[ExtEvDataService], extEvDataService)) - val extResultData = - new ExtResultData( - adapterRef.toClassic, - adapterScheduleRef.toClassic, + case (extPrimaryDataSimulation: ExtPrimaryDataSimulation, dIndex) => + val extPrimaryDataService = context.toClassic.simonaActorOf( + ExtPrimaryDataService.props(extScheduler.toClassic), + s"$index-$dIndex", + ) + val extPrimaryData = new ExtPrimaryData( + extPrimaryDataService, extSimAdapter, - extResultDataSimulation.getResultDataFactory, - extResultDataSimulation.getResultDataAssets + extPrimaryDataSimulation.getPrimaryDataFactory, + extPrimaryDataSimulation.getPrimaryDataAssets ) - extResultDataSimulation.setExtResultData(extResultData) + extPrimaryDataSimulation.setExtPrimaryData(extPrimaryData) + + extPrimaryDataService ! SimonaService.Create( + InitExtPrimaryData(extPrimaryData), + ScheduleLock.singleKey( + context, + extScheduler, + INIT_SIM_TICK, + ), + ) - extResultDataProvider ! ExtResultDataProvider.Create( - InitExtResultData(extResultData), - ScheduleLock.singleKey( - context, - extScheduler, - INIT_SIM_TICK, - ), - ) + (extPrimaryData, (classOf[ExtPrimaryDataService], extPrimaryDataService)) + + case (extResultDataSimulation: ExtResultDataSimulation, dIndex) => + + val extResultDataProvider = { + context.spawn( + ExtResultDataProvider(extScheduler), + s"$index-$dIndex", + ) + } + + implicit val timeout: PekkoTimeout = PekkoTimeout.create(5.seconds.toJava) + implicit val scheduler: Scheduler = context.system.scheduler + + val adapterRef = Await.result( + extResultDataProvider.ask[ActorRef[ResultDataMessageFromExt]] (ref => RequestDataMessageAdapter(ref)), timeout.duration) + val adapterScheduleRef = Await.result( + extResultDataProvider.ask[ActorRef[ScheduleServiceActivation]] (ref => RequestScheduleActivationAdapter(ref)), timeout.duration) + + val extResultData = + new ExtResultData( + adapterRef.toClassic, + adapterScheduleRef.toClassic, + extSimAdapter, + extResultDataSimulation.getResultDataFactory, + extResultDataSimulation.getResultDataAssets + ) + + extResultDataSimulation.setExtResultData(extResultData) + + extResultDataProvider ! ExtResultDataProvider.Create( + InitExtResultData(extResultData), + ScheduleLock.singleKey( + context, + extScheduler, + INIT_SIM_TICK, + ), + ) - ( - extResultData, - (ExtResultDataProvider.getClass, extResultDataProvider.toClassic), - ) - }.unzip + ( + extResultData, + (ExtResultDataProvider.getClass, extResultDataProvider.toClassic), + ) + }.unzip - extLink.getExtSimulation.setup( - extSimAdapterData, - extData.toList.asJava, - ) + 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, (extDataServiceToRef, extData)) - }.unzip + (extSimAdapter, (extDataServiceToRef, extData)) + }.unzip - val extDataServices = extDatasAndServices.map(_._1) - val extDatas = extDatasAndServices.flatMap(_._2).toSet + val extDataServices = extDatasAndServices.map(_._1) + val extDatas = extDatasAndServices.flatMap(_._2).toSet - ExtSimSetupData( - extSimAdapters, - extDataServices.flatten.toMap, - extDatas, - Some(extScheduler)) + ExtSimSetupData( + extSimAdapters, + extDataServices.flatten.toMap, + extDatas, + Some(extScheduler)) } else { - ExtSimSetupData(Iterable.empty, Map.empty, None) + ExtSimSetupData(Iterable.empty, Map.empty, Set.empty, None) } } -*/ + + */ + override def extSimulations( - context: ActorContext[_], - theScheduler: ActorRef[SchedulerMessage], + context: ActorContext[_], + rootScheduler: ActorRef[SchedulerMessage], + simScheduler: ActorRef[SchedulerMessage], ): ExtSimSetupData = { + val extScheduler = scheduler(context, parent = rootScheduler) val simpleExtSim = new SimpleExtSimulation() - val extSimAdapter = context.toClassic.simonaActorOf( - ExtSimAdapter.props(theScheduler.toClassic), - s"0", + val extSimAdapterPhase1 = context.toClassic.simonaActorOf( + ExtSimAdapter.props(extScheduler.toClassic), + s"1", + ) + val extSimAdapterDataPhase1 = new ExtSimAdapterData(extSimAdapterPhase1, args) + + + val extSimAdapterPhase2 = context.toClassic.simonaActorOf( + ExtSimAdapter.props(simScheduler.toClassic), + s"2", + ) + val extSimAdapterDataPhase2 = new ExtSimAdapterData(extSimAdapterPhase2, args) + + + // send init data right away, init activation is scheduled + extSimAdapterPhase1 ! ExtSimAdapter.Create( + extSimAdapterDataPhase1, + ScheduleLock.singleKey(context, extScheduler, INIT_SIM_TICK), ) - val extSimAdapterData = new ExtSimAdapterData(extSimAdapter, args) // send init data right away, init activation is scheduled - extSimAdapter ! ExtSimAdapter.Create( - extSimAdapterData, - ScheduleLock.singleKey(context, theScheduler, INIT_SIM_TICK), + extSimAdapterPhase2 ! ExtSimAdapter.Create( + extSimAdapterDataPhase2, + ScheduleLock.singleKey(context, simScheduler, INIT_SIM_TICK), ) val extPrimaryDataService = context.toClassic.simonaActorOf( - ExtPrimaryDataService.props(theScheduler.toClassic), + ExtPrimaryDataService.props(extScheduler.toClassic), s"0-0", ) val extPrimaryData = new ExtPrimaryData( extPrimaryDataService, - extSimAdapter, + extSimAdapterPhase1, simpleExtSim.getExtPrimaryDataSimulation.getPrimaryDataFactory, simpleExtSim.getExtPrimaryDataSimulation.getPrimaryDataAssets ) @@ -378,32 +398,34 @@ class SimonaStandaloneSetup( InitExtPrimaryData(extPrimaryData), ScheduleLock.singleKey( context, - theScheduler, + extScheduler, INIT_SIM_TICK, ), ) + //Result Data val extResultDataProvider = { context.spawn( - ExtResultDataProvider(theScheduler), + ExtResultDataProvider(simScheduler), s"ExtResultDataProvider", ) } - implicit val timeout: PekkoTimeout = PekkoTimeout.create(5.seconds.toJava) - implicit val scheduler: Scheduler = context.system.scheduler + + val timeout: PekkoTimeout = PekkoTimeout.create(5.seconds.toJava) + val scheduler2: Scheduler = context.system.scheduler val adapterRef = Await.result( - extResultDataProvider.ask[ActorRef[ResultDataMessageFromExt]] (ref => RequestDataMessageAdapter(ref)), timeout.duration) + extResultDataProvider.ask[ActorRef[ResultDataMessageFromExt]] (ref => RequestDataMessageAdapter(ref))(timeout, scheduler2), timeout.duration) val adapterScheduleRef = Await.result( - extResultDataProvider.ask[ActorRef[ScheduleServiceActivation]] (ref => RequestScheduleActivationAdapter(ref)), timeout.duration) + extResultDataProvider.ask[ActorRef[ScheduleServiceActivation]] (ref => RequestScheduleActivationAdapter(ref))(timeout, scheduler2), timeout.duration) val extResultData = new ExtResultData( adapterRef.toClassic, adapterScheduleRef.toClassic, - extSimAdapter, + extSimAdapterPhase2, simpleExtSim.getExtResultDataSimulation.getResultDataFactory, simpleExtSim.getExtResultDataSimulation.getResultDataAssets ) @@ -414,35 +436,41 @@ class SimonaStandaloneSetup( InitExtResultData(extResultData), ScheduleLock.singleKey( context, - theScheduler, + simScheduler, INIT_SIM_TICK, ), ) - val simpleExtSimDatas: List[ExtData] = List(extResultData, extPrimaryData) + + val simpleExtSimDatas: List[ExtData] = List( + extResultData, + extPrimaryData + ) simpleExtSim.setup( - extSimAdapterData, + extSimAdapterDataPhase1, + extSimAdapterDataPhase2, simpleExtSimDatas.asJava, ) // starting external simulation context.log.info("Starting External simulation") - new Thread(simpleExtSim, s"External simulation 0") + new Thread(simpleExtSim, s"External simulation") .start() val extDataServicesMap: Map[Class[_], ClassicRef] = Map( classOf[ExtPrimaryDataService] -> extPrimaryDataService, - ExtResultDataProvider.getClass -> extResultDataProvider.toClassic + //ExtResultDataProvider.getClass -> extResultDataProvider.toClassic ) - val extSimAdapters = Iterable(extSimAdapter) - - val extDatas = Set( - extPrimaryData, - extResultData + val extDataListenerMap: Map[Class[_], ActorRef[ExtResultDataProvider.Request]] = Map( + ExtResultDataProvider.getClass -> extResultDataProvider ) - ExtSimSetupData(extSimAdapters, extDataServicesMap, extDatas) + val extSimAdapters = Iterable(extSimAdapterPhase1, extSimAdapterPhase2) + + val extDatas = simpleExtSimDatas.toSet + + ExtSimSetupData(extSimAdapters, extDataServicesMap, extDataListenerMap, extDatas, Some(extScheduler)) } @@ -497,7 +525,7 @@ class SimonaStandaloneSetup( context: ActorContext[_], extSimulationData: ExtSimSetupData, ): Seq[ActorRef[ResultEventListener.Request]] = { - val extResultDataService: Option[ClassicRef] = + val extResultDataService: Option[ActorRef[ExtResultDataProvider.Request]] = extSimulationData.extResultDataService // append ResultEventListener as well to write raw output files ArgsParser From 55257384151ef24c9a73a99a3b90ef8118d3fdb1 Mon Sep 17 00:00:00 2001 From: Johannes Bao Date: Wed, 27 Mar 2024 08:55:45 +0100 Subject: [PATCH 23/41] phase in extsim --- .../edu/ie3/simona/api/ExtSimAdapter.scala | 13 ++++--- .../edu/ie3/simona/scheduler/Scheduler.scala | 5 +-- .../primary/ExtPrimaryDataService.scala | 35 +++++++++++------- .../service/primary/PrimaryServiceProxy.scala | 4 +-- .../sim/setup/SimonaStandaloneSetup.scala | 36 ++++++++++++------- 5 files changed, 56 insertions(+), 37 deletions(-) diff --git a/src/main/scala/edu/ie3/simona/api/ExtSimAdapter.scala b/src/main/scala/edu/ie3/simona/api/ExtSimAdapter.scala index 2bbb090880..53e5c05550 100644 --- a/src/main/scala/edu/ie3/simona/api/ExtSimAdapter.scala +++ b/src/main/scala/edu/ie3/simona/api/ExtSimAdapter.scala @@ -35,12 +35,13 @@ object ExtSimAdapter { * @param extSimData * The [[ExtSimAdapterData]] of the corresponding external simulation */ - final case class Create(extSimData: ExtSimAdapterData, unlockKey: ScheduleKey) + final case class Create(extSimData: ExtSimAdapterData, phase: Int, unlockKey: ScheduleKey) final case class Stop(simulationSuccessful: Boolean) final case class ExtSimAdapterStateData( extSimData: ExtSimAdapterData, + phase: Int, currentTick: Option[Long] = None, ) } @@ -48,7 +49,7 @@ object ExtSimAdapter { final case class ExtSimAdapter(scheduler: ActorRef) extends Actor with SimonaActorLogging { - override def receive: Receive = { case Create(extSimAdapterData, unlockKey) => + override def receive: Receive = { case Create(extSimAdapterData, phase, unlockKey) => // triggering first time at init tick scheduler ! ScheduleActivation( self.toTyped, @@ -56,7 +57,7 @@ final case class ExtSimAdapter(scheduler: ActorRef) Some(unlockKey), ) context become receiveIdle( - ExtSimAdapterStateData(extSimAdapterData) + ExtSimAdapterStateData(extSimAdapterData, phase) ) } @@ -65,13 +66,12 @@ final case class ExtSimAdapter(scheduler: ActorRef) ): Receive = { case Activation(tick) => stateData.extSimData.queueExtMsg( - new ActivationMessage(tick) + new ActivationMessage(tick, stateData.phase) ) log.debug( "Tick {} has been activated in external simulation", tick, ) - log.info("stateData after Activation(" + tick + ") = " + stateData) context become receiveIdle( stateData.copy(currentTick = Some(tick)) @@ -92,7 +92,6 @@ final case class ExtSimAdapter(scheduler: ActorRef) context become receiveIdle(stateData.copy(currentTick = None)) case scheduleDataService: ScheduleDataServiceMessage => - log.info("stateData after ScheduleDataService = " + stateData) val tick = stateData.currentTick.getOrElse( throw new RuntimeException("No tick has been triggered") ) @@ -106,7 +105,7 @@ final case class ExtSimAdapter(scheduler: ActorRef) case Stop(simulationSuccessful) => // let external sim know that we have terminated stateData.extSimData.queueExtMsg( - new TerminationMessage(simulationSuccessful) + new TerminationMessage(simulationSuccessful, stateData.phase) ) case _: TerminationCompleted => diff --git a/src/main/scala/edu/ie3/simona/scheduler/Scheduler.scala b/src/main/scala/edu/ie3/simona/scheduler/Scheduler.scala index 303968e597..2fc075b779 100644 --- a/src/main/scala/edu/ie3/simona/scheduler/Scheduler.scala +++ b/src/main/scala/edu/ie3/simona/scheduler/Scheduler.scala @@ -116,6 +116,7 @@ object Scheduler { // since we're active and any scheduled activation can still influence our next activation, // we can directly unlock the lock with the key unlockKey.foreach { + //println(this + " now Unlock ScheduleActivation from " + actor + ", newTick = " + newTick + ", newCore: " + newCore) _.unlock() } @@ -138,14 +139,14 @@ object Scheduler { toActivate.foreach { _ ! Activation(updatedCore.activeTick) } - println(this + " Completion from " + actor + ", maybeNextTick = " + maybeNewTick + ", updatedCore: " + updatedCore) + //println(this + " Completion from " + actor + ", maybeNextTick = " + maybeNewTick + ", updatedCore: " + updatedCore) updatedCore } .map { newCore => newCore .maybeComplete() .map { case (maybeScheduleTick, inactiveCore) => - println(this + " Send Completion from " + data.activationAdapter + " to " + data.parent) + //println(this + " Send Completion from " + data.activationAdapter + " to " + data.parent) data.parent ! Completion( data.activationAdapter, maybeScheduleTick, diff --git a/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala b/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala index dcaa46ec65..9e7b804b64 100644 --- a/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala +++ b/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala @@ -180,24 +180,15 @@ final case class ExtPrimaryDataService( // Distribute Primary Data if (actorToPrimaryData.nonEmpty) { - val keys = - ScheduleLock.multiKey( - ctx, - scheduler.toTyped, - tick, - actorToPrimaryData.size, - ) - - actorToPrimaryData.zip(keys).foreach { - case ((actor, primaryDataPerAgent), key) => - primaryDataPerAgent.toPrimaryData match { + actorToPrimaryData.foreach { + case (actor, value) => + value.toPrimaryData match { case Success(primaryData) => actor ! ProvidePrimaryDataMessage( tick, self, primaryData, - None, - unlockKey = Some(key), + None ) case Failure(exception) => /* Processing of data failed */ @@ -209,6 +200,24 @@ final case class ExtPrimaryDataService( } } } + + /* + val keys = + ScheduleLock.multiKey( + ctx, + scheduler.toTyped, + tick, + actorToPrimaryData.size, + ) + + actorToPrimaryData.zip(keys).foreach { + case ((actor, primaryDataPerAgent), key) => + log.info(s"actor $actor, unlockKey $key") + primaryDataPerAgent.toPrimaryData v + } + + */ + ( serviceStateData.copy(extPrimaryDataMessage = None), None, diff --git a/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala b/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala index 5194102394..0002286e12 100644 --- a/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala +++ b/src/main/scala/edu/ie3/simona/service/primary/PrimaryServiceProxy.scala @@ -336,8 +336,8 @@ case class PrimaryServiceProxy( requestingActor: ActorRef, ): Unit = { stateData.extPrimaryDataService match { - case Some(reqActor) => - reqActor ! ExtPrimaryDataServiceRegistrationMessage( + case Some(primaryDataService) => + primaryDataService ! ExtPrimaryDataServiceRegistrationMessage( modelUuid, requestingActor, ) 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 122021c734..1ac5977e1e 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala @@ -359,28 +359,37 @@ class SimonaStandaloneSetup( ExtSimAdapter.props(extScheduler.toClassic), s"1", ) - val extSimAdapterDataPhase1 = new ExtSimAdapterData(extSimAdapterPhase1, args) - - val extSimAdapterPhase2 = context.toClassic.simonaActorOf( ExtSimAdapter.props(simScheduler.toClassic), s"2", ) - val extSimAdapterDataPhase2 = new ExtSimAdapterData(extSimAdapterPhase2, args) + + val extSimAdapters: Map[java.lang.Integer, ClassicRef] = Map( + 1.asInstanceOf[java.lang.Integer] -> extSimAdapterPhase1, + 2.asInstanceOf[java.lang.Integer] -> extSimAdapterPhase2 + ) + + val extSimAdapterData = new ExtSimAdapterData(extSimAdapters.asJava, args) + // send init data right away, init activation is scheduled extSimAdapterPhase1 ! ExtSimAdapter.Create( - extSimAdapterDataPhase1, + extSimAdapterData, + 1, ScheduleLock.singleKey(context, extScheduler, INIT_SIM_TICK), ) + /* // send init data right away, init activation is scheduled extSimAdapterPhase2 ! ExtSimAdapter.Create( - extSimAdapterDataPhase2, + extSimAdapterData, + 2, ScheduleLock.singleKey(context, simScheduler, INIT_SIM_TICK), ) + */ + val extPrimaryDataService = context.toClassic.simonaActorOf( ExtPrimaryDataService.props(extScheduler.toClassic), s"0-0", @@ -448,29 +457,30 @@ class SimonaStandaloneSetup( ) simpleExtSim.setup( - extSimAdapterDataPhase1, - extSimAdapterDataPhase2, + extSimAdapterData, simpleExtSimDatas.asJava, ) // starting external simulation - context.log.info("Starting External simulation") new Thread(simpleExtSim, s"External simulation") .start() val extDataServicesMap: Map[Class[_], ClassicRef] = Map( classOf[ExtPrimaryDataService] -> extPrimaryDataService, - //ExtResultDataProvider.getClass -> extResultDataProvider.toClassic ) val extDataListenerMap: Map[Class[_], ActorRef[ExtResultDataProvider.Request]] = Map( ExtResultDataProvider.getClass -> extResultDataProvider ) - val extSimAdapters = Iterable(extSimAdapterPhase1, extSimAdapterPhase2) + val extSimAdaptersIt = Iterable(extSimAdapterPhase1, extSimAdapterPhase2) val extDatas = simpleExtSimDatas.toSet - - ExtSimSetupData(extSimAdapters, extDataServicesMap, extDataListenerMap, extDatas, Some(extScheduler)) + extSimAdapterPhase2 ! ExtSimAdapter.Create( + extSimAdapterData, + 2, + ScheduleLock.singleKey(context, simScheduler, INIT_SIM_TICK), + ) + ExtSimSetupData(extSimAdaptersIt, extDataServicesMap, extDataListenerMap, extDatas, Some(extScheduler)) } From 16130a32d8b99ef22c54fff9c8410b5d8f986e9c Mon Sep 17 00:00:00 2001 From: jo-bao Date: Tue, 2 Apr 2024 15:15:54 +0200 Subject: [PATCH 24/41] uml --- .../ExtPrimaryResultSimulationClasses.puml | 275 +++++++----------- docs/uml/protocol/ExtSimulationSequence.puml | 28 ++ 2 files changed, 137 insertions(+), 166 deletions(-) create mode 100644 docs/uml/protocol/ExtSimulationSequence.puml diff --git a/docs/uml/main/ExtPrimaryResultSimulationClasses.puml b/docs/uml/main/ExtPrimaryResultSimulationClasses.puml index e35c8e130a..e13c3e6be2 100644 --- a/docs/uml/main/ExtPrimaryResultSimulationClasses.puml +++ b/docs/uml/main/ExtPrimaryResultSimulationClasses.puml @@ -2,222 +2,165 @@ 'https://plantuml.com/class-diagram skinparam linetype ortho -package ev-simulation { +package simple-ext-simulation { class ExtLink - class ExternalSampleSim - class EvModelImpl - - ' adding hidden elements for layout purposes - class placeholderA { - w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w + class SimpleExtSimulation { + - ExtPrimaryDataSimulation extPrimaryDataSimulation + - ExtResultDataSimulation extResultDataSimulation } - hide placeholderA + class SimplePrimaryDataFactory - class placeholderB { - w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w - } - hide placeholderB + class SimpleResultDataFactory - ExtLink -[hidden]> placeholderA - placeholderA -[hidden]> ExternalSampleSim - ExternalSampleSim -[hidden]> placeholderB - placeholderB -[hidden]> EvModelImpl + SimplePrimaryDataFactory --> SimpleExtSimulation + SimpleResultDataFactory --> SimpleExtSimulation } package simona-api { - ' MIDDLE PART - class ExtEvData { - ~ LinkedBlockingQueue receiveTriggerQueue + + interface ExtData + + class ExtPrimaryData { - ActorRef dataService - ActorRef extSimAdapter - + List requestAvailablePublicEvCs() - + List sendEvPositions(EvMovementsMessage evMovementsMessage) - - void sendData(ExtEvMessage msg) - + void queueExtMsg(ExtEvResponseMessage extEvResponse) + - PrimaryDataFactory factory + - List primaryDataAssets + --- + + void providePrimaryData(Long tick, Map primaryData) + + void sendExtMsg(PrimaryDataMessageFromExt msg) + } + ExtPrimaryData --|> ExtData + + class ExtResultData { + - ActorRef dataService + - ActorRef dataServiceActivation + - ActorRef extSimAdapter + - ResultDataFactory factory + - List resultDataAssets + --- + + List requestResults(long tick) + + Map requestResultObjects(long tick) + ~ Map convertResultsList(List results) + } + ExtResultData --|> ExtData + + ' MIDDLE PART + class ExtEvData + ExtEvData --|> ExtData + + interface ExtDataSimulation + + class ExtPrimaryDataSimulation { + - List primaryDataAssets + - PrimaryDataFactory primaryDataFactory + - ExtPrimaryData extPrimaryData + --- + + void setExtPrimaryData(ExtPrimaryData extPrimaryData) + } + ExtPrimaryDataSimulation --|> ExtDataSimulation + + class ExtResultDataSimulation { + - List resultDataAssets + - ResultDataFactory resultDataFactory + - ExtResultData extResultData + --- + + void setExtResultData(ExtResultData extResultData) + } + ExtResultDataSimulation --|> ExtDataSimulation + + interface PrimaryDataFactory { + Value convert(Object entity) } + PrimaryDataFactory --> ExtPrimaryDataSimulation + + interface ResultDataFactory { + Object convert(ResultEntity entity) + } + ResultDataFactory --> ExtResultDataSimulation + + SimplePrimaryDataFactory --|> PrimaryDataFactory + SimpleResultDataFactory --|> ResultDataFactory class ExtSimAdapterData { ~ LinkedBlockingQueue receiveTriggerQueue - - ActorRef extSimAdapter + - Map extSimAdapters + void queueExtMsg(ExtTrigger trigger) + void send(ExtTriggerResponse msg) } - interface ExtData abstract class ExtSimulation { + - ExtSimAdapterData data + void setup(ExtSimAdapterData data, List adapters) + void run() - # {abstract} List doActivity(long tick) + # Optional doActivity(long tick, int phase) + # {abstract} Optional initialize() + # {abstract} Optional doPreActivity(long tick) + # {abstract} Optional doPostActivity(long tick) } + SimpleExtSimulation --|> ExtSimulation + interface ExtEvSimulation { + void setExtEvData(ExtEvData evData) } - interface ExtDataSimulation - - ExternalSampleSim --|> ExtSimulation - ExternalSampleSim --|> ExtEvSimulation - - ExtSimulation -[hidden]> ExtDataSimulation - ExtEvSimulation -|> ExtDataSimulation - ExtEvSimulation --> ExtEvData - - ExtSimulation --> ExtData - ExtSimulation --> ExtSimAdapterData + ExtEvSimulation --|> ExtDataSimulation - ExtSimAdapterData -[hidden]> ExtEvData - ExtData <|-- ExtEvData + interface DataMessageFromExt + interface PrimaryDataMessageFromExt + interface ResultDataMessageFromExt + interface EvDataMessageFromExt + PrimaryDataMessageFromExt --|> DataMessageFromExt + ResultDataMessageFromExt --|> DataMessageFromExt + EvDataMessageFromExt --|> DataMessageFromExt - ' placeholder - class plchldr1 { - w-w-w-w-w - } - hide plchldr1 - class placeholder2 { - w-w-w-w-w-w-w-w-w-w-w-w-w-w-w - } - hide placeholder2 - class placeholder3 { - w-w-w-w-w-w-w-w-w-w-w - } - hide placeholder3 - class placeholder4 { - w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w-w + class ProvidePrimaryData { + - long tick + - Map primaryData } - hide placeholder4 - - ExtData -[hidden]> plchldr1 - plchldr1 -[hidden]> placeholder4 - placeholder4 -[hidden]> EvMovement - ExtSimulation -[hidden]-> placeholder2 - placeholder2 -[hidden]-> placeholder3 - placeholder3 -[hidden]-> ExtSimAdapterData - placeholder2 -[hidden]> ExtData - - - ' RIGHT PART - abstract class EvMovement - class Arrival - class Departure - Arrival --|> EvMovement - Departure --|> EvMovement - Arrival -[hidden]> Departure - - interface EvModel { - + UUID getUuid() - + String getId() - + ComparableQuantity getSRated() - + ComparableQuantity getEStorage() - + ComparableQuantity getStoredEnergy() - + EvModel copyWith(ComparableQuantity newStoredEnergy) + ProvidePrimaryData --|> PrimaryDataMessageFromExt + + class RequestResultEntities { + - long tick } + RequestResultEntities --|> ResultDataMessageFromExt - ExtDataSimulation -[hidden]> Arrival - EvMovement *-> EvModel - EvModelImpl --|> EvModel + interface DataResponseMessageToExt + interface ResultDataResponseMessageToExt + interface EvDataResponseMessageToExt - interface ExtEvMessage - class EvMovementsMessage { - - Map> movements - } - class RequestEvcsFreeLots - RequestEvcsFreeLots --|> ExtEvMessage - EvMovementsMessage --|> ExtEvMessage - RequestEvcsFreeLots -[hidden]> EvMovementsMessage - - interface ExtEvResponseMessage - class AllDepartedEvsRepsonse { - - Map> movements - } - class ProvideEvcsFreeLots - ProvideEvcsFreeLots --|> ExtEvResponseMessage - AllDepartedEvsRepsonse --|> ExtEvResponseMessage - ExtEvData -> ExtEvMessage - ExtEvData -> ExtEvResponseMessage - EvMovement -[hidden]-> RequestEvcsFreeLots - ExtEvMessage -[hidden]> ExtEvResponseMessage - EvMovementsMessage -[hidden]> ProvideEvcsFreeLots - ProvideEvcsFreeLots -[hidden]> AllDepartedEvsRepsonse - - class ScheduleDataServiceMessage { - - ExtEvDataService dataService + ResultDataResponseMessageToExt --|> DataResponseMessageToExt + EvDataResponseMessageToExt --|> DataResponseMessageToExt + class ProvideResultEntities { + + List results } - ExtEvResponseMessage -[hidden]> ScheduleDataServiceMessage - ExtEvData -> ScheduleDataServiceMessage + ProvideResultEntities --|> ResultDataResponseMessageToExt + ExtSimAdapterData --> ExtSimulation - ' LEFT PART interface ExtLinkInterface { - + ExtSimulation getExtSimulation() - + List getExtDataSimulations() + ExtSimulation getExtSimulation() + List getExtDataSimulation() } - ExtLinkInterface -[hidden]> ExtSimulation ExtLink --|> ExtLinkInterface - interface ExtTrigger - class ActivityStartTrigger { - - Long tick - } - ActivityStartTrigger --|> ExtTrigger - - interface ExtTriggerResponse - class CompletionMessage { - - List newTriggers - } - CompletionMessage --|> ExtTriggerResponse - - ExtTrigger -[hidden]> ExtTriggerResponse - ExtTrigger <- ExtSimAdapterData - ExtTriggerResponse <- ExtSimAdapterData + class ScheduleDataServiceMessage } package simona { - abstract class ParticipantAgent - - class EvcsModel + class ExtPrimaryDataService - class EvcsAgent - - interface SecondaryData - class EvMovementData { - movements: Set[EvMovement] - } + ExtPrimaryData --> ExtPrimaryDataService - class SimonaSim + class ExtResultDataProvider - class SimScheduler - - class SimonaStandaloneSetup - - class ExtSimLoader + ExtResultData --> ExtResultDataProvider class ExtSimAdapter - class ExtEvDataService - - SimScheduler -- SimonaSim - SimonaSim *- SimonaStandaloneSetup - SimonaStandaloneSetup *- ExtSimLoader - - ExtSimAdapter -- SimScheduler - ExtEvDataService -- SimScheduler - - SecondaryData <|-- EvMovementData - - SimonaService <|- ExtEvDataService - ExtEvData <--- ExtEvDataService - - ExtSimAdapterData <--- ExtSimAdapter - - ' 1 and n are swapped here due to bug with linetype ortho and labels - ExtEvDataService "n" <-> "1" EvcsAgent - EvcsAgent --|> ParticipantAgent - EvcsAgent *- EvcsModel - - ExtSimAdapter -[hidden]> SimonaService + ExtSimAdapterData --> ExtSimAdapter } @enduml \ No newline at end of file diff --git a/docs/uml/protocol/ExtSimulationSequence.puml b/docs/uml/protocol/ExtSimulationSequence.puml new file mode 100644 index 0000000000..c0df9bd9d0 --- /dev/null +++ b/docs/uml/protocol/ExtSimulationSequence.puml @@ -0,0 +1,28 @@ +@startuml + +== Phase 1 == +ExtScheduler -> ExtSimAdapter: ! Activation(tick) +ExtSimAdapter -> ExtSimAdapter: StateData(currentTick = Some(tick)) +ExtSimAdapter -> ExtSimAdapterData: queueExtMessage(ActivationMessage(tick)) + +ExtSimulation -> ExtPrimaryDataService: ProvidePrimaryData +ExtSimulation -> ExtSimAdapter: ! ScheduleDataServiceMessage(dataService) +ExtSimulation -> ExtSimAdapter: ! CompletionMessage(nextTriggers) +ExtSimAdapter -> ExtScheduler: ! Completion +ExtSimAdapter -> ExtPrimaryDataService: ! ScheduleServiceActivation(tick, key) +ExtPrimaryDataService -> ExtScheduler: ! ScheduleActivation(_, tick, _) +ExtScheduler -> ExtPrimaryDataService: ! Activation(tick) + +ExtPrimaryDataService -> ParticipantAgent1: ! ProvisionMessage +ExtPrimaryDataService -> ParticipantAgent2: ! ProvisionMessage +ExtPrimaryDataService -> ExtScheduler: ! Completion(_, maybeNextTriggers) +ExtScheduler -> SimScheduler: ! Completion + +== Phase 2 == + +SimScheduler -> ParticipantAgent1: ! Activation(tick) +SimScheduler -> ParticipantAgent2: ! Activation(tick) + +activate ParticipantAgent1 + +@enduml \ No newline at end of file From f45e5c3f9ca21eb629a023176ffa6c2742ca1cae Mon Sep 17 00:00:00 2001 From: smjobaoo Date: Tue, 9 Apr 2024 10:08:33 +0200 Subject: [PATCH 25/41] opsim coupling --- build.gradle | 1 - input/samples/vn_simona_ns/vn_simona_ns.conf | 6 +- .../simona/service/ev/ExtEvDataService.scala | 2 +- .../sim/setup/SimonaStandaloneSetup.scala | 153 ++++++++++++++++-- 4 files changed, 147 insertions(+), 15 deletions(-) diff --git a/build.gradle b/build.gradle index 50dce58f3d..f148f60a8e 100644 --- a/build.gradle +++ b/build.gradle @@ -106,7 +106,6 @@ dependencies { exclude group: 'edu.ie3' } - /* logging */ implementation "com.typesafe.scala-logging:scala-logging_${scalaVersion}:3.9.5" // pekko scala logging implementation "ch.qos.logback:logback-classic:1.5.3" diff --git a/input/samples/vn_simona_ns/vn_simona_ns.conf b/input/samples/vn_simona_ns/vn_simona_ns.conf index 016d7ddb60..9f85016d4c 100644 --- a/input/samples/vn_simona_ns/vn_simona_ns.conf +++ b/input/samples/vn_simona_ns/vn_simona_ns.conf @@ -23,13 +23,13 @@ simona.time.schedulerReadyCheckWindow = 900 # Input Parameters ################################################################## simona.input.primary.csvParams = { - directoryPath: "input/samples/vn_simona_ns/fullGrid" + directoryPath: "simona/input/samples/vn_simona_ns/fullGrid" csvSep: "," isHierarchic: false } simona.input.grid.datasource.id = "csv" simona.input.grid.datasource.csvParams = { - directoryPath: "input/samples/vn_simona_ns/fullGrid" + directoryPath: "simona/input/samples/vn_simona_ns/fullGrid" csvSep: "," isHierarchic: false } @@ -44,7 +44,7 @@ simona.input.weather.datasource = { ################################################################## # Output Parameters ################################################################## -simona.output.base.dir = "output/vn_simona_ns" +simona.output.base.dir = "simona/output/vn_simona_ns" simona.output.base.addTimestampToOutputDir = true simona.output.sink.csv { diff --git a/src/main/scala/edu/ie3/simona/service/ev/ExtEvDataService.scala b/src/main/scala/edu/ie3/simona/service/ev/ExtEvDataService.scala index 2e3a8247c9..27587bfafe 100644 --- a/src/main/scala/edu/ie3/simona/service/ev/ExtEvDataService.scala +++ b/src/main/scala/edu/ie3/simona/service/ev/ExtEvDataService.scala @@ -186,7 +186,7 @@ class ExtEvDataService(override val scheduler: ActorRef) case departingEvsRequest: RequestDepartingEvs => requestDepartingEvs(tick, departingEvsRequest.departures) case arrivingEvsProvision: ProvideArrivingEvs => - handleArrivingEvs(tick, arrivingEvsProvision.arrivals)( + handleArrivingEvs(tick, null)( serviceStateData, ctx, ) 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 1ac5977e1e..6a8723f39b 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala @@ -56,6 +56,7 @@ import org.apache.pekko.actor.typed.{ActorRef, Scheduler} import org.apache.pekko.actor.{ActorRef => ClassicRef} import org.apache.pekko.util.{Timeout => PekkoTimeout} import edu.ie3.simona.service.results.ExtResultDataProvider.Request +import edu.ie3.simopsim.OpsimSimulator import java.util.UUID import java.util.concurrent.LinkedBlockingQueue @@ -351,7 +352,17 @@ class SimonaStandaloneSetup( context: ActorContext[_], rootScheduler: ActorRef[SchedulerMessage], simScheduler: ActorRef[SchedulerMessage], + ): ExtSimSetupData = { + opsimAsExtSimulation( + context, rootScheduler, simScheduler + ) + } + def extMosaikSimulation( + context: ActorContext[_], + rootScheduler: ActorRef[SchedulerMessage], + simScheduler: ActorRef[SchedulerMessage], ): ExtSimSetupData = { + /* val extScheduler = scheduler(context, parent = rootScheduler) val simpleExtSim = new SimpleExtSimulation() @@ -380,16 +391,6 @@ class SimonaStandaloneSetup( ScheduleLock.singleKey(context, extScheduler, INIT_SIM_TICK), ) - /* - // send init data right away, init activation is scheduled - extSimAdapterPhase2 ! ExtSimAdapter.Create( - extSimAdapterData, - 2, - ScheduleLock.singleKey(context, simScheduler, INIT_SIM_TICK), - ) - - */ - val extPrimaryDataService = context.toClassic.simonaActorOf( ExtPrimaryDataService.props(extScheduler.toClassic), s"0-0", @@ -474,6 +475,138 @@ class SimonaStandaloneSetup( val extSimAdaptersIt = Iterable(extSimAdapterPhase1, extSimAdapterPhase2) + val extDatas = simpleExtSimDatas.toSet + extSimAdapterPhase2 ! ExtSimAdapter.Create( + extSimAdapterData, + 2, + ScheduleLock.singleKey(context, simScheduler, INIT_SIM_TICK), + ) + ExtSimSetupData(extSimAdaptersIt, extDataServicesMap, extDataListenerMap, extDatas, Some(extScheduler)) + + */ + null + } + + + def opsimAsExtSimulation( + context: ActorContext[_], + rootScheduler: ActorRef[SchedulerMessage], + simScheduler: ActorRef[SchedulerMessage], + ): ExtSimSetupData = { + val simulationStart = TimeUtil.withDefaults.toZonedDateTime( + simonaConfig.simona.time.startDateTime + ) + val extScheduler = scheduler(context, parent = rootScheduler) + val opsimSimulator = new OpsimSimulator( + "amqp://guest:guest@localhost:5672/myvhost" + ) + + val extSimAdapterPhase1 = context.toClassic.simonaActorOf( + ExtSimAdapter.props(extScheduler.toClassic), + s"1", + ) + val extSimAdapterPhase2 = context.toClassic.simonaActorOf( + ExtSimAdapter.props(simScheduler.toClassic), + s"2", + ) + + val extSimAdapters: Map[java.lang.Integer, ClassicRef] = Map( + 1.asInstanceOf[java.lang.Integer] -> extSimAdapterPhase1, + 2.asInstanceOf[java.lang.Integer] -> extSimAdapterPhase2 + ) + + val extSimAdapterData = new ExtSimAdapterData(extSimAdapters.asJava, args) + + // send init data right away, init activation is scheduled + extSimAdapterPhase1 ! ExtSimAdapter.Create( + extSimAdapterData, + 1, + ScheduleLock.singleKey(context, extScheduler, INIT_SIM_TICK), + ) + + val extPrimaryDataService = context.toClassic.simonaActorOf( + ExtPrimaryDataService.props(extScheduler.toClassic), + s"0-0", + ) + val extPrimaryData = new ExtPrimaryData( + extPrimaryDataService, + extSimAdapterPhase1, + opsimSimulator.getExtPrimaryDataSimulation.getPrimaryDataFactory, + opsimSimulator.getExtPrimaryDataSimulation.getPrimaryDataAssets + ) + + opsimSimulator.getExtPrimaryDataSimulation.setExtPrimaryData(extPrimaryData) + + extPrimaryDataService ! SimonaService.Create( + InitExtPrimaryData(extPrimaryData), + ScheduleLock.singleKey( + context, + extScheduler, + INIT_SIM_TICK, + ), + ) + //Result Data + + val extResultDataProvider = { + context.spawn( + ExtResultDataProvider(simScheduler), + s"ExtResultDataProvider", + ) + } + + + val timeout: PekkoTimeout = PekkoTimeout.create(5.seconds.toJava) + val scheduler2: Scheduler = context.system.scheduler + + val adapterRef = Await.result( + extResultDataProvider.ask[ActorRef[ResultDataMessageFromExt]] (ref => RequestDataMessageAdapter(ref))(timeout, scheduler2), timeout.duration) + val adapterScheduleRef = Await.result( + extResultDataProvider.ask[ActorRef[ScheduleServiceActivation]] (ref => RequestScheduleActivationAdapter(ref))(timeout, scheduler2), timeout.duration) + + val extResultData = + new ExtResultData( + adapterRef.toClassic, + adapterScheduleRef.toClassic, + extSimAdapterPhase2, + opsimSimulator.getExtResultDataSimulation.getResultDataFactory, + opsimSimulator.getExtResultDataSimulation.getResultDataAssets, + simulationStart + ) + + opsimSimulator.getExtResultDataSimulation.setExtResultData(extResultData) + + extResultDataProvider ! ExtResultDataProvider.Create( + InitExtResultData(extResultData), + ScheduleLock.singleKey( + context, + simScheduler, + INIT_SIM_TICK, + ), + ) + + val simpleExtSimDatas: List[ExtData] = List( + extResultData, + extPrimaryData + ) + + opsimSimulator.setup( + extSimAdapterData, + simpleExtSimDatas.asJava, + ) + // starting external simulation + new Thread(opsimSimulator, s"External simulation") + .start() + + val extDataServicesMap: Map[Class[_], ClassicRef] = Map( + classOf[ExtPrimaryDataService] -> extPrimaryDataService, + ) + + val extDataListenerMap: Map[Class[_], ActorRef[ExtResultDataProvider.Request]] = Map( + ExtResultDataProvider.getClass -> extResultDataProvider + ) + + val extSimAdaptersIt = Iterable(extSimAdapterPhase1, extSimAdapterPhase2) + val extDatas = simpleExtSimDatas.toSet extSimAdapterPhase2 ! ExtSimAdapter.Create( extSimAdapterData, From 063dc898b7deb1005bd487515b4d3746f223c009 Mon Sep 17 00:00:00 2001 From: Johannes Bao Date: Thu, 16 May 2024 22:28:20 +0200 Subject: [PATCH 26/41] temp --- .../event/listener/ResultEventListener.scala | 1 + .../ie3/simona/main/RunSimonaWithMosaik.scala | 62 ++ .../messages/flex/FlexibilityMessage.scala | 2 + .../simona/service/em/ExtEmDataService.scala | 198 ++++++ .../simona/sim/setup/SimonaMosaikSetup.scala | 627 ++++++++++++++++++ .../sim/setup/SimonaStandaloneSetup.scala | 49 +- 6 files changed, 932 insertions(+), 7 deletions(-) create mode 100644 src/main/scala/edu/ie3/simona/main/RunSimonaWithMosaik.scala create mode 100644 src/main/scala/edu/ie3/simona/service/em/ExtEmDataService.scala create mode 100644 src/main/scala/edu/ie3/simona/sim/setup/SimonaMosaikSetup.scala diff --git a/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala b/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala index aeb415f121..86e5d87c24 100644 --- a/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala +++ b/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala @@ -151,6 +151,7 @@ object ResultEventListener extends Transformer3wResultSupport { log: Logger, nextTick: Option[Long] = None ): BaseData = { + log.info("Got Result " + resultEntity) handOverToSink(resultEntity, baseData.classToSink, log) if (baseData.extResultDataService.isDefined) { handOverToExternalService( diff --git a/src/main/scala/edu/ie3/simona/main/RunSimonaWithMosaik.scala b/src/main/scala/edu/ie3/simona/main/RunSimonaWithMosaik.scala new file mode 100644 index 0000000000..5c5d13ce4d --- /dev/null +++ b/src/main/scala/edu/ie3/simona/main/RunSimonaWithMosaik.scala @@ -0,0 +1,62 @@ +/* + * © 2020. TU Dortmund University, + * Institute of Energy Systems, Energy Efficiency and Energy Economics, + * Research group Distribution grid planning and operation + */ + +package edu.ie3.simona.main + +import edu.ie3.simona.config.{ArgsParser, ConfigFailFast, SimonaConfig} +import edu.ie3.simona.main.RunSimona._ +import edu.ie3.simona.sim.SimonaSim +import edu.ie3.simona.sim.setup.SimonaMosaikSetup +import org.apache.pekko.actor.typed.scaladsl.AskPattern._ +import org.apache.pekko.actor.typed.{ActorSystem, Scheduler} +import org.apache.pekko.util.Timeout + +import scala.concurrent.Await +import scala.concurrent.duration.DurationInt + +/** Run a standalone simulation of simona + * + * @since 01.07.20 + */ +object RunSimonaWithMosaik extends RunSimona[SimonaMosaikSetup] { + + override implicit val timeout: Timeout = Timeout(12.hours) + + override def setup(args: Array[String]): SimonaMosaikSetup = { + // get the config and prepare it with the provided args + val (arguments, parsedConfig) = ArgsParser.prepareConfig(args) + + // config fail fast check + val simonaConfig = SimonaConfig(parsedConfig) + ConfigFailFast.check(parsedConfig, simonaConfig) + + SimonaMosaikSetup( + parsedConfig, + SimonaMosaikSetup.buildResultFileHierarchy(parsedConfig), + mainArgs = arguments.mainArgs, + ) + } + + override def run(simonaSetup: SimonaMosaikSetup): Boolean = { + val simonaSim = ActorSystem( + SimonaSim(simonaSetup), + name = "Simona", + config = simonaSetup.typeSafeConfig, + ) + + implicit val scheduler: Scheduler = simonaSim.scheduler + + // run the simulation + val terminated = simonaSim.ask[SimonaEnded](ref => SimonaSim.Start(ref)) + + Await.result(terminated, timeout.duration) match { + case SimonaEnded(successful) => + simonaSim.terminate() + + successful + } + } +} 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 812eafbfff..d339ba3eb0 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 @@ -85,6 +85,8 @@ object FlexibilityMessage { */ trait ProvideFlexOptions extends FlexResponse + trait ProvideEmSetPoints extends EmAgent.Request + /** Message that issues flexibility control to a flex options provider, i.e. a * feasible set point is delivered that the flex options provider should * adhere to diff --git a/src/main/scala/edu/ie3/simona/service/em/ExtEmDataService.scala b/src/main/scala/edu/ie3/simona/service/em/ExtEmDataService.scala new file mode 100644 index 0000000000..6a593c08ae --- /dev/null +++ b/src/main/scala/edu/ie3/simona/service/em/ExtEmDataService.scala @@ -0,0 +1,198 @@ +package edu.ie3.simona.service.em + +import edu.ie3.datamodel.models.value.PValue +import edu.ie3.simona.api.data.em.ExtEmData +import edu.ie3.simona.api.data.em.ontology.{EmDataMessageFromExt, ProvideEmData} +import edu.ie3.simona.api.data.ontology.DataMessageFromExt +import edu.ie3.simona.exceptions.WeatherServiceException.InvalidRegistrationRequestException +import edu.ie3.simona.exceptions.{InitializationException, ServiceException} +import edu.ie3.simona.ontology.messages.flex.FlexibilityMessage.IssuePowerControl +import edu.ie3.simona.ontology.messages.services.ServiceMessage.ExtPrimaryDataServiceRegistrationMessage +import edu.ie3.simona.ontology.messages.services.{DataMessage, ServiceMessage} +import edu.ie3.simona.service.ServiceStateData.{InitializeServiceStateData, ServiceBaseStateData} +import edu.ie3.simona.service.em.ExtEmDataService.{ExtEmDataStateData, InitExtEmData} +import edu.ie3.simona.service.primary.ExtPrimaryDataService.{ExtPrimaryDataStateData, InitExtPrimaryData} +import edu.ie3.simona.service.{ExtDataSupport, SimonaService} +import org.apache.pekko.actor.{ActorContext, ActorRef, Props} +import squants.Power +import squants.energy.Kilowatts + +import java.util.UUID +import scala.jdk.CollectionConverters.MapHasAsScala +import scala.util.{Failure, Success, Try} + +object ExtEmDataService { + + def props(scheduler: ActorRef): Props = + Props( + new ExtEmDataService(scheduler: ActorRef) + ) + + final case class ExtEmDataStateData( + extEmData: ExtEmData, + subscribers: List[UUID] = List.empty, + uuidToActorRef: Map[UUID, ActorRef] = Map.empty[UUID, ActorRef], // subscribers in SIMONA + extEmDataMessage: Option[EmDataMessageFromExt] = None, + ) extends ServiceBaseStateData + + case class InitExtEmData( + extEmData: ExtEmData + ) extends InitializeServiceStateData +} + + + +final case class ExtEmDataService( + override val scheduler: ActorRef + ) extends SimonaService[ExtEmDataStateData](scheduler) + with ExtDataSupport[ExtEmDataStateData] { + + /** Initialize the concrete service implementation using the provided + * initialization data. This method should perform all heavyweight tasks + * before the actor becomes ready. The return values are a) the state data of + * the initialized service and b) optional triggers that should be send to + * the [[edu.ie3.simona.scheduler.Scheduler]] together with the completion + * message that is send in response to the trigger that is send to start the + * initialization process + * + * @param initServiceData + * the data that should be used for initialization + * @return + * the state data of this service and optional tick that should be included + * in the completion message + */ + override def init(initServiceData: InitializeServiceStateData): Try[(ExtEmDataStateData, Option[Long])] = initServiceData match { + case InitExtEmData(extEmData) => + val emDataInitializedStateData = ExtEmDataStateData( + extEmData + ) + Success( + emDataInitializedStateData, + None, + ) + + case invalidData => + Failure( + new InitializationException( + s"Provided init data '${invalidData.getClass.getSimpleName}' for ExtEmDataService are invalid!" + ) + ) + } + + /** Handle a request to register for information from this service + * + * @param registrationMessage + * registration message to handle + * @param serviceStateData + * current state data of the actor + * @return + * the service stata data that should be used in the next state (normally + * with updated values) + */ + override protected def handleRegistrationRequest( + registrationMessage: ServiceMessage.ServiceRegistrationMessage + )( + implicit serviceStateData: ExtEmDataStateData): Try[ExtEmDataStateData] = registrationMessage match { + case ExtPrimaryDataServiceRegistrationMessage( + modelUuid, + requestingActor, + ) => + null + case invalidMessage => + Failure( + InvalidRegistrationRequestException( + s"A primary service provider is not able to handle registration request '$invalidMessage'." + ) + ) + } + /** Send out the information to all registered recipients + * + * @param tick + * current tick data should be announced for + * @param serviceStateData + * the current state data of this service + * @return + * the service stata data that should be used in the next state (normally + * with updated values) together with the completion message that is send + * in response to the trigger that was sent to start this announcement + */ + override protected def announceInformation(tick: Long)(implicit serviceStateData: ExtEmDataStateData, ctx: ActorContext): (ExtEmDataStateData, Option[Long]) = { + serviceStateData.extEmDataMessage.getOrElse( + throw ServiceException( + "ExtPrimaryDataService was triggered without ExtPrimaryDataMessage available" + ) + ) match { + case providedEmData: ProvideEmData => + announceEmData(tick, providedEmData.emData)( + serviceStateData, + ctx, + ) + } + } + + private def announceEmData( + tick: Long, + emData: java.util.Map[UUID, PValue], + )(implicit serviceStateData: ExtEmDataStateData, ctx: ActorContext): ( + ExtEmDataStateData, + Option[Long] + ) = { + val actorToEmData = emData.asScala.flatMap { + case (agent, emDataPerAgent) => + serviceStateData.uuidToActorRef + .get(agent) + .map((_, convertToSetPoint(emDataPerAgent))) + .orElse { + log.warning( + "A corresponding actor ref for UUID {} could not be found", + agent, + ) + None + } + } + + if (actorToEmData.nonEmpty) { + actorToEmData.foreach { + case (actor, setPoint) => actor ! IssuePowerControl( + tick, + setPoint + ) + } + } + (serviceStateData.copy(extEmDataMessage = None), None) + } + + private def convertToSetPoint( + value: PValue + ): Power = { + Kilowatts(value.getP.get.getValue.doubleValue()) + } + + /** Handle a message from outside the simulation + * + * @param extMsg + * the external incoming message + * @param serviceStateData + * the current state data of this service + * @return + * the updated state data + */ + override protected def handleDataMessage(extMsg: DataMessageFromExt)(implicit serviceStateData: ExtEmDataStateData): ExtEmDataStateData = { + extMsg match { + case extEmDataMessage: EmDataMessageFromExt => serviceStateData.copy( + extEmDataMessage = Some(extEmDataMessage) + ) + } + } + + /** Handle a message from inside SIMONA sent to external + * + * @param extResponseMsg + * the external incoming message + * @param serviceStateData + * the current state data of this service + * @return + * the updated state data + */ + override protected def handleDataResponseMessage(extResponseMsg: DataMessage)(implicit serviceStateData: ExtEmDataStateData): ExtEmDataStateData = serviceStateData +} diff --git a/src/main/scala/edu/ie3/simona/sim/setup/SimonaMosaikSetup.scala b/src/main/scala/edu/ie3/simona/sim/setup/SimonaMosaikSetup.scala new file mode 100644 index 0000000000..07fdd2cf09 --- /dev/null +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaMosaikSetup.scala @@ -0,0 +1,627 @@ +/* + * © 2020. TU Dortmund University, + * Institute of Energy Systems, Energy Efficiency and Energy Economics, + * Research group Distribution grid planning and operation + */ + +package edu.ie3.simona.sim.setup + +import com.typesafe.config.Config +import com.typesafe.scalalogging.LazyLogging +import edu.ie3.datamodel.graph.SubGridTopologyGraph +import edu.ie3.datamodel.models.input.container.{GridContainer, ThermalGrid} +import edu.ie3.datamodel.models.input.thermal.ThermalBusInput +import edu.ie3.simona.actor.SimonaActorNaming.RichActorRefFactory +import edu.ie3.simona.agent.EnvironmentRefs +import edu.ie3.simona.agent.grid.GridAgentMessage.CreateGridAgent +import edu.ie3.simona.agent.grid.{GridAgent, GridAgentMessage} +import edu.ie3.simona.api.ExtSimAdapter +import edu.ie3.simona.api.data.ExtData +import edu.ie3.simona.api.data.em.ExtEmData +import edu.ie3.simona.api.data.primarydata.ExtPrimaryData +import edu.ie3.simona.api.data.results.ExtResultData +import edu.ie3.simona.api.data.results.ontology.ResultDataMessageFromExt +import edu.ie3.simona.api.simulation.ExtSimAdapterData +import edu.ie3.simona.config.{ArgsParser, RefSystemParser, SimonaConfig} +import edu.ie3.simona.event.listener.{ResultEventListener, RuntimeEventListener} +import edu.ie3.simona.event.{ResultEvent, RuntimeEvent} +import edu.ie3.simona.exceptions.agent.GridAgentInitializationException +import edu.ie3.simona.io.grid.GridProvider +import edu.ie3.simona.ontology.messages.SchedulerMessage +import edu.ie3.simona.ontology.messages.SchedulerMessage.ScheduleActivation +import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.ScheduleServiceActivation +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.em.ExtEmDataService +import edu.ie3.simona.service.em.ExtEmDataService.InitExtEmData +import edu.ie3.simona.service.primary.ExtPrimaryDataService.InitExtPrimaryData +import edu.ie3.simona.service.primary.PrimaryServiceProxy.InitPrimaryServiceProxyStateData +import edu.ie3.simona.service.primary.{ExtPrimaryDataService, PrimaryServiceProxy} +import edu.ie3.simona.service.results.ExtResultDataProvider +import edu.ie3.simona.service.results.ExtResultDataProvider.{InitExtResultData, RequestDataMessageAdapter, RequestScheduleActivationAdapter} +import edu.ie3.simona.service.weather.WeatherService +import edu.ie3.simona.service.weather.WeatherService.InitWeatherServiceStateData +import edu.ie3.simona.sim.SimonaSim +import edu.ie3.simona.util.ResultFileHierarchy +import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK +import edu.ie3.simona.util.TickUtil.RichZonedDateTime +import edu.ie3.simopsim.OpsimSimulator +import edu.ie3.simosaik.MosaikSimulation +import edu.ie3.util.TimeUtil +import org.apache.pekko.actor.typed.scaladsl.ActorContext +import org.apache.pekko.actor.typed.scaladsl.AskPattern._ +import org.apache.pekko.actor.typed.scaladsl.adapter.{ClassicActorRefOps, TypedActorContextOps, TypedActorRefOps} +import org.apache.pekko.actor.typed.{ActorRef, Scheduler} +import org.apache.pekko.actor.{ActorRef => ClassicRef} +import org.apache.pekko.util.{Timeout => PekkoTimeout} + +import java.util.UUID +import java.util.concurrent.LinkedBlockingQueue +import scala.concurrent.Await +import scala.concurrent.duration.DurationInt +import scala.jdk.CollectionConverters._ +import scala.jdk.DurationConverters._ + +/** Sample implementation to run a standalone simulation of simona configured + * with the provided [[SimonaConfig]] and [[ResultFileHierarchy]] + * + * @version 0.1 + * @since 01.07.20 + */ +class SimonaMosaikSetup( + val typeSafeConfig: Config, + simonaConfig: SimonaConfig, + resultFileHierarchy: ResultFileHierarchy, + runtimeEventQueue: Option[LinkedBlockingQueue[RuntimeEvent]] = None, + override val args: Array[String], +) extends SimonaSetup { + + override def gridAgents( + context: ActorContext[_], + environmentRefs: EnvironmentRefs, + resultEventListeners: Seq[ActorRef[ResultEvent]], + ): Iterable[ActorRef[GridAgentMessage]] = { + + /* get the grid */ + val subGridTopologyGraph = GridProvider + .gridFromConfig( + simonaConfig.simona.simulationName, + simonaConfig.simona.input.grid.datasource, + ) + .getSubGridTopologyGraph + val thermalGridsByThermalBus = GridProvider.getThermalGridsFromConfig( + simonaConfig.simona.input.grid.datasource + ) + + /* extract and prepare refSystem information from config */ + val configRefSystems = + RefSystemParser.parse(simonaConfig.simona.gridConfig.refSystems) + + /* Create all agents and map the sub grid id to their actor references */ + val subGridToActorRefMap = buildSubGridToActorRefMap( + subGridTopologyGraph, + context, + environmentRefs, + resultEventListeners, + ) + + val keys = ScheduleLock.multiKey( + context, + environmentRefs.scheduler, + INIT_SIM_TICK, + subGridTopologyGraph.vertexSet().size, + ) + + /* build the initialization data */ + subGridTopologyGraph + .vertexSet() + .asScala + .zip(keys) + .map { case (subGridContainer, key) => + /* Get all connections to superior and inferior sub grids */ + val subGridGates = + Set.from( + subGridTopologyGraph + .edgesOf(subGridContainer) + .asScala + .map(modifySubGridGateForThreeWindingSupport) + ) + val currentSubGrid = subGridContainer.getSubnet + val currentActorRef = subGridToActorRefMap.getOrElse( + currentSubGrid, + throw new GridAgentInitializationException( + "Was asked to setup agent for sub grid " + currentSubGrid + ", but did not found it's actor reference." + ), + ) + val thermalGrids = + getThermalGrids(subGridContainer, thermalGridsByThermalBus) + + /* build the grid agent data and check for its validity */ + val gridAgentInitData = SimonaStandaloneSetup.buildGridAgentInitData( + subGridContainer, + subGridToActorRefMap, + subGridGates, + configRefSystems, + thermalGrids, + ) + + currentActorRef ! CreateGridAgent(gridAgentInitData, key) + + currentActorRef + } + } + + override def primaryServiceProxy( + context: ActorContext[_], + scheduler: ActorRef[SchedulerMessage], + extSimSetupData: ExtSimSetupData, + ): ClassicRef = { + val simulationStart = TimeUtil.withDefaults.toZonedDateTime( + simonaConfig.simona.time.startDateTime + ) + val primaryServiceProxy = context.toClassic.simonaActorOf( + PrimaryServiceProxy.props( + scheduler.toClassic, + InitPrimaryServiceProxyStateData( + simonaConfig.simona.input.primary, + simulationStart, + extSimSetupData.extPrimaryDataService, + extSimSetupData.extPrimaryData + ), + simulationStart, + ) + ) + + scheduler ! ScheduleActivation(primaryServiceProxy.toTyped, INIT_SIM_TICK) + primaryServiceProxy + } + + override def weatherService( + context: ActorContext[_], + scheduler: ActorRef[SchedulerMessage], + ): ClassicRef = { + val weatherService = context.toClassic.simonaActorOf( + WeatherService.props( + scheduler.toClassic, + TimeUtil.withDefaults + .toZonedDateTime(simonaConfig.simona.time.startDateTime), + TimeUtil.withDefaults + .toZonedDateTime(simonaConfig.simona.time.endDateTime), + ) + ) + weatherService ! SimonaService.Create( + InitWeatherServiceStateData( + simonaConfig.simona.input.weather.datasource + ), + ScheduleLock.singleKey(context, scheduler, INIT_SIM_TICK), + ) + + weatherService + } + + /* + override def extSimulations( + context: ActorContext[_], + rootScheduler: ActorRef[SchedulerMessage], + ): ExtSimSetupData = { + val jars = ExtSimLoader.scanInputFolder() + + val extLinks = jars.flatMap(ExtSimLoader.loadExtLink).toSeq + + if (extLinks.nonEmpty) { + val extScheduler = scheduler(context, parent = rootScheduler) + + val (extSimAdapters, extDatasAndServices) = + 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) + + // 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, extDataServiceToRef): ( + Iterable[ExtData], + Iterable[(Class[_], 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)) + + case (extPrimaryDataSimulation: ExtPrimaryDataSimulation, dIndex) => + val extPrimaryDataService = context.toClassic.simonaActorOf( + ExtPrimaryDataService.props(extScheduler.toClassic), + s"$index-$dIndex", + ) + val extPrimaryData = new ExtPrimaryData( + extPrimaryDataService, + extSimAdapter, + extPrimaryDataSimulation.getPrimaryDataFactory, + extPrimaryDataSimulation.getPrimaryDataAssets + ) + + extPrimaryDataSimulation.setExtPrimaryData(extPrimaryData) + + extPrimaryDataService ! SimonaService.Create( + InitExtPrimaryData(extPrimaryData), + ScheduleLock.singleKey( + context, + extScheduler, + INIT_SIM_TICK, + ), + ) + + (extPrimaryData, (classOf[ExtPrimaryDataService], extPrimaryDataService)) + + case (extResultDataSimulation: ExtResultDataSimulation, dIndex) => + + val extResultDataProvider = { + context.spawn( + ExtResultDataProvider(extScheduler), + s"$index-$dIndex", + ) + } + + implicit val timeout: PekkoTimeout = PekkoTimeout.create(5.seconds.toJava) + implicit val scheduler: Scheduler = context.system.scheduler + + val adapterRef = Await.result( + extResultDataProvider.ask[ActorRef[ResultDataMessageFromExt]] (ref => RequestDataMessageAdapter(ref)), timeout.duration) + val adapterScheduleRef = Await.result( + extResultDataProvider.ask[ActorRef[ScheduleServiceActivation]] (ref => RequestScheduleActivationAdapter(ref)), timeout.duration) + + val extResultData = + new ExtResultData( + adapterRef.toClassic, + adapterScheduleRef.toClassic, + extSimAdapter, + extResultDataSimulation.getResultDataFactory, + extResultDataSimulation.getResultDataAssets + ) + + extResultDataSimulation.setExtResultData(extResultData) + + extResultDataProvider ! ExtResultDataProvider.Create( + InitExtResultData(extResultData), + ScheduleLock.singleKey( + context, + extScheduler, + INIT_SIM_TICK, + ), + ) + + ( + extResultData, + (ExtResultDataProvider.getClass, extResultDataProvider.toClassic), + ) + }.unzip + + extLink.getExtSimulation.setup( + extSimAdapterData, + extData.toList.asJava, + ) + + // starting external simulation + new Thread(extLink.getExtSimulation, s"External simulation $index") + .start() + + (extSimAdapter, (extDataServiceToRef, extData)) + }.unzip + + val extDataServices = extDatasAndServices.map(_._1) + val extDatas = extDatasAndServices.flatMap(_._2).toSet + + ExtSimSetupData( + extSimAdapters, + extDataServices.flatten.toMap, + extDatas, + Some(extScheduler)) + } else { + ExtSimSetupData(Iterable.empty, Map.empty, Set.empty, None) + } + } + + */ + + override def extSimulations( + context: ActorContext[_], + rootScheduler: ActorRef[SchedulerMessage], + simScheduler: ActorRef[SchedulerMessage] + ): ExtSimSetupData = { + mosaikAsExtSimulation( + context, rootScheduler, simScheduler + ) + } + + def mosaikAsExtSimulation( + context: ActorContext[_], + rootScheduler: ActorRef[SchedulerMessage], + simScheduler: ActorRef[SchedulerMessage] + ): ExtSimSetupData = { + val simulationStart = TimeUtil.withDefaults.toZonedDateTime( + simonaConfig.simona.time.startDateTime + ) + val extScheduler = scheduler(context, parent = rootScheduler) + val mosaikExtSim = new MosaikSimulation("127.0.0.1:5555") + + val extSimAdapterPhase1 = context.toClassic.simonaActorOf( + ExtSimAdapter.props(extScheduler.toClassic), + s"1", + ) + val extSimAdapterPhase2 = context.toClassic.simonaActorOf( + ExtSimAdapter.props(simScheduler.toClassic), + s"2", + ) + + val extSimAdapters: Map[java.lang.Integer, ClassicRef] = Map( + 1.asInstanceOf[java.lang.Integer] -> extSimAdapterPhase1, + 2.asInstanceOf[java.lang.Integer] -> extSimAdapterPhase2 + ) + + val extSimAdapterData = new ExtSimAdapterData(extSimAdapters.asJava, args) + + + + // send init data right away, init activation is scheduled + extSimAdapterPhase1 ! ExtSimAdapter.Create( + extSimAdapterData, + 1, + ScheduleLock.singleKey(context, extScheduler, INIT_SIM_TICK), + ) + + val extPrimaryDataService = context.toClassic.simonaActorOf( + ExtPrimaryDataService.props(extScheduler.toClassic), + s"0-0", + ) + val extPrimaryData = new ExtPrimaryData( + extPrimaryDataService, + extSimAdapterPhase1, + mosaikExtSim.getExtPrimaryDataSimulation.getPrimaryDataFactory, + mosaikExtSim.getExtPrimaryDataSimulation.getPrimaryDataAssets + ) + + mosaikExtSim.getExtPrimaryDataSimulation.setExtPrimaryData(extPrimaryData) + + extPrimaryDataService ! SimonaService.Create( + InitExtPrimaryData(extPrimaryData), + ScheduleLock.singleKey( + context, + extScheduler, + INIT_SIM_TICK, + ), + ) + + //Result Data + + val extResultDataProvider = { + context.spawn( + ExtResultDataProvider(simScheduler), + s"ExtResultDataProvider", + ) + } + + + val timeout: PekkoTimeout = PekkoTimeout.create(5.seconds.toJava) + val scheduler2: Scheduler = context.system.scheduler + + val adapterRef = Await.result( + extResultDataProvider.ask[ActorRef[ResultDataMessageFromExt]] (ref => RequestDataMessageAdapter(ref))(timeout, scheduler2), timeout.duration) + val adapterScheduleRef = Await.result( + extResultDataProvider.ask[ActorRef[ScheduleServiceActivation]] (ref => RequestScheduleActivationAdapter(ref))(timeout, scheduler2), timeout.duration) + + val extResultData = + new ExtResultData( + adapterRef.toClassic, + adapterScheduleRef.toClassic, + extSimAdapterPhase2, + mosaikExtSim.getExtResultDataSimulation.getResultDataFactory, + mosaikExtSim.getExtResultDataSimulation.getResultDataAssets, + simulationStart + ) + + mosaikExtSim.getExtResultDataSimulation.setExtResultData(extResultData) + + extResultDataProvider ! ExtResultDataProvider.Create( + InitExtResultData(extResultData), + ScheduleLock.singleKey( + context, + simScheduler, + INIT_SIM_TICK, + ), + ) + + + val simpleExtSimDatas: List[ExtData] = List( + extResultData, + extPrimaryData + ) + + mosaikExtSim.setup( + extSimAdapterData, + simpleExtSimDatas.asJava, + ) + // starting external simulation + new Thread(mosaikExtSim, s"External simulation") + .start() + + + + val extDataServicesMap: Map[Class[_], ClassicRef] = Map( + classOf[ExtPrimaryDataService] -> extPrimaryDataService, + ) + + val extDataListenerMap: Map[Class[_], ActorRef[ExtResultDataProvider.Request]] = Map( + ExtResultDataProvider.getClass -> extResultDataProvider + ) + + val extSimAdaptersIt = Iterable(extSimAdapterPhase1, extSimAdapterPhase2) + + val extDatas = simpleExtSimDatas.toSet + extSimAdapterPhase2 ! ExtSimAdapter.Create( + extSimAdapterData, + 2, + ScheduleLock.singleKey(context, simScheduler, INIT_SIM_TICK), + ) + ExtSimSetupData(extSimAdaptersIt, extDataServicesMap, extDataListenerMap, extDatas, Some(extScheduler)) + } + + override def timeAdvancer( + context: ActorContext[_], + simulation: ActorRef[SimonaSim.SimulationEnded.type], + runtimeEventListener: ActorRef[RuntimeEvent], + ): ActorRef[TimeAdvancer.Request] = { + val startDateTime = TimeUtil.withDefaults.toZonedDateTime( + simonaConfig.simona.time.startDateTime + ) + val endDateTime = TimeUtil.withDefaults.toZonedDateTime( + simonaConfig.simona.time.endDateTime + ) + + context.spawn( + TimeAdvancer( + simulation, + Some(runtimeEventListener), + simonaConfig.simona.time.schedulerReadyCheckWindow, + endDateTime.toTick(startDateTime), + ), + TimeAdvancer.getClass.getSimpleName, + ) + } + + override def scheduler( + context: ActorContext[_], + parent: ActorRef[SchedulerMessage], + coreFactory: CoreFactory = RegularSchedulerCore, + ): ActorRef[SchedulerMessage] = + context + .spawn( + Scheduler(parent, coreFactory), + s"${Scheduler.getClass.getSimpleName}_${coreFactory}_${UUID.randomUUID()}", + ) + + override def runtimeEventListener( + context: ActorContext[_] + ): ActorRef[RuntimeEventListener.Request] = + context + .spawn( + RuntimeEventListener( + simonaConfig.simona.runtime.listener, + runtimeEventQueue, + startDateTimeString = simonaConfig.simona.time.startDateTime, + ), + RuntimeEventListener.getClass.getSimpleName, + ) + + override def resultEventListener( + context: ActorContext[_], + extSimulationData: ExtSimSetupData, + ): Seq[ActorRef[ResultEventListener.Request]] = { + val extResultDataService: Option[ActorRef[ExtResultDataProvider.Request]] = + extSimulationData.extResultDataService + // append ResultEventListener as well to write raw output files + ArgsParser + .parseListenerConfigOption(simonaConfig.simona.event.listener) + .zipWithIndex + .map { case ((listenerCompanion, events), index) => + context.toClassic + .simonaActorOf( + listenerCompanion.props(events), + index.toString, + ) + .toTyped + } + .toSeq :+ context + .spawn( + ResultEventListener( + resultFileHierarchy, + extResultDataService, + ), + ResultEventListener.getClass.getSimpleName, + ) + } + + def buildSubGridToActorRefMap( + subGridTopologyGraph: SubGridTopologyGraph, + context: ActorContext[_], + environmentRefs: EnvironmentRefs, + resultEventListeners: Seq[ActorRef[ResultEvent]], + ): Map[Int, ActorRef[GridAgentMessage]] = { + subGridTopologyGraph + .vertexSet() + .asScala + .map(subGridContainer => { + val gridAgentRef = + context.spawn( + GridAgent( + environmentRefs, + simonaConfig, + resultEventListeners, + ), + subGridContainer.getSubnet.toString, + ) + subGridContainer.getSubnet -> gridAgentRef + }) + .toMap + } + + /** Get all thermal grids, that apply for the given grid container + * @param grid + * The grid container to assess + * @param thermalGridByBus + * Mapping from thermal bus to thermal grid + * @return + * A sequence of applicable thermal grids + */ + private def getThermalGrids( + grid: GridContainer, + thermalGridByBus: Map[ThermalBusInput, ThermalGrid], + ): Seq[ThermalGrid] = { + grid.getSystemParticipants.getHeatPumps.asScala + .flatten(hpInput => thermalGridByBus.get(hpInput.getThermalBus)) + .toSeq + } +} + +/** Companion object to provide [[SetupHelper]] methods for + * [[SimonaStandaloneSetup]] + */ +object SimonaMosaikSetup extends LazyLogging with SetupHelper { + + def apply( + typeSafeConfig: Config, + resultFileHierarchy: ResultFileHierarchy, + runtimeEventQueue: Option[LinkedBlockingQueue[RuntimeEvent]] = None, + mainArgs: Array[String] = Array.empty[String], + ): SimonaMosaikSetup = + new SimonaMosaikSetup( + typeSafeConfig, + SimonaConfig(typeSafeConfig), + resultFileHierarchy, + runtimeEventQueue, + mainArgs, + ) +} 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 6a8723f39b..3c7f71ea2e 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala @@ -17,6 +17,7 @@ import edu.ie3.simona.agent.grid.GridAgentMessage.CreateGridAgent import edu.ie3.simona.agent.grid.{GridAgent, GridAgentMessage} import edu.ie3.simona.api.ExtSimAdapter import edu.ie3.simona.api.data.ExtData +import edu.ie3.simona.api.data.em.ExtEmData import edu.ie3.simona.api.data.ev.{ExtEvData, ExtEvSimulation} import edu.ie3.simona.api.data.primarydata.{ExtPrimaryData, ExtPrimaryDataSimulation} import edu.ie3.simona.api.data.results.{ExtResultData, ExtResultDataSimulation} @@ -34,6 +35,8 @@ import edu.ie3.simona.scheduler.core.RegularSchedulerCore import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.ScheduleServiceActivation import edu.ie3.simona.scheduler.{ScheduleLock, Scheduler, TimeAdvancer} import edu.ie3.simona.service.SimonaService +import edu.ie3.simona.service.em.ExtEmDataService +import edu.ie3.simona.service.em.ExtEmDataService.InitExtEmData import edu.ie3.simona.service.ev.ExtEvDataService import edu.ie3.simona.service.ev.ExtEvDataService.InitExtEvData import edu.ie3.simona.service.primary.ExtPrimaryDataService.InitExtPrimaryData @@ -47,7 +50,6 @@ import edu.ie3.simona.sim.SimonaSim import edu.ie3.simona.util.ResultFileHierarchy import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK import edu.ie3.simona.util.TickUtil.RichZonedDateTime -import edu.ie3.simosaik.simpleextsim.SimpleExtSimulation import edu.ie3.util.TimeUtil import org.apache.pekko.actor.typed.scaladsl.ActorContext import org.apache.pekko.actor.typed.scaladsl.AskPattern._ @@ -57,6 +59,8 @@ import org.apache.pekko.actor.{ActorRef => ClassicRef} import org.apache.pekko.util.{Timeout => PekkoTimeout} import edu.ie3.simona.service.results.ExtResultDataProvider.Request import edu.ie3.simopsim.OpsimSimulator +import edu.ie3.simosaik.MosaikSimulation +import edu.ie3.simpleextsim.SimpleExtSimulation import java.util.UUID import java.util.concurrent.LinkedBlockingQueue @@ -356,13 +360,18 @@ class SimonaStandaloneSetup( opsimAsExtSimulation( context, rootScheduler, simScheduler ) + //simpleExtSimulation( + // context, rootScheduler, simScheduler + //) } - def extMosaikSimulation( + def simpleExtSimulation( context: ActorContext[_], rootScheduler: ActorRef[SchedulerMessage], simScheduler: ActorRef[SchedulerMessage], ): ExtSimSetupData = { - /* + val simulationStart = TimeUtil.withDefaults.toZonedDateTime( + simonaConfig.simona.time.startDateTime + ) val extScheduler = scheduler(context, parent = rootScheduler) val simpleExtSim = new SimpleExtSimulation() @@ -437,7 +446,8 @@ class SimonaStandaloneSetup( adapterScheduleRef.toClassic, extSimAdapterPhase2, simpleExtSim.getExtResultDataSimulation.getResultDataFactory, - simpleExtSim.getExtResultDataSimulation.getResultDataAssets + simpleExtSim.getExtResultDataSimulation.getResultDataAssets, + simulationStart ) simpleExtSim.getExtResultDataSimulation.setExtResultData(extResultData) @@ -482,9 +492,6 @@ class SimonaStandaloneSetup( ScheduleLock.singleKey(context, simScheduler, INIT_SIM_TICK), ) ExtSimSetupData(extSimAdaptersIt, extDataServicesMap, extDataListenerMap, extDatas, Some(extScheduler)) - - */ - null } @@ -545,6 +552,34 @@ class SimonaStandaloneSetup( INIT_SIM_TICK, ), ) + + /* + // --- External Em Data + val extEmDataService = context.toClassic.simonaActorOf( + ExtEmDataService.props(extScheduler.toClassic), + s"0-0", + ) + val extEmData = new ExtEmData( + extEmDataService, + extSimAdapterPhase1, + opsimSimulator.getExtEmDataSimulation.getEmDataFactory, + opsimSimulator.getExtEmDataSimulation.getControlledEms + ) + + opsimSimulator.getExtEmDataSimulation.setExtEmData(extEmData) + + extEmDataService ! SimonaService.Create( + InitExtEmData(extEmData), + ScheduleLock.singleKey( + context, + extScheduler, + INIT_SIM_TICK, + ), + ) + + */ + + //Result Data val extResultDataProvider = { From 58ed7a0e5ea17e558b0c3cc1facf0c11d6f2ab36 Mon Sep 17 00:00:00 2001 From: Johannes Bao Date: Mon, 20 May 2024 22:20:57 +0200 Subject: [PATCH 27/41] simple em ext simulation --- .../fullGrid/em_input.csv | 4 + .../fullGrid/line_input.csv | 110 +++ .../fullGrid/line_type_input.csv | 2 + .../fullGrid/load_input.csv | 4 + .../fullGrid/node_input.csv | 112 +++ .../fullGrid/pv_input.csv | 4 + .../fullGrid/transformer_2_w_input.csv | 2 + .../fullGrid/transformer_2_w_type_input.csv | 2 + .../simopsimtestgrid.conf | 240 ++++++ .../fullGrid/em_input.csv | 3 + .../fullGrid/fixed_feed_in_input.csv | 2 + .../fullGrid/line_input.csv | 6 + .../fullGrid/line_type_input.csv | 2 + .../fullGrid/load_input.csv | 5 + .../fullGrid/node_input.csv | 6 + .../fullGrid/pv_input.csv | 3 + .../fullGrid/storage_input.csv | 3 + .../fullGrid/storage_type_input.csv | 2 + .../fullGrid/transformer_2_w_input.csv | 2 + .../fullGrid/transformer_2_w_type_input.csv | 2 + .../vn_simona_small_withem.conf | 210 +++++ .../fullGrid/em_input.csv | 5 + .../fullGrid/fixed_feed_in_input.csv | 2 + .../fullGrid/line_input.csv | 6 + .../fullGrid/line_type_input.csv | 2 + .../fullGrid/load_input.csv | 5 + .../fullGrid/node_input.csv | 6 + .../fullGrid/pv_input.csv | 3 + .../fullGrid/storage_input.csv | 3 + .../fullGrid/storage_type_input.csv | 2 + .../fullGrid/transformer_2_w_input.csv | 2 + .../fullGrid/transformer_2_w_type_input.csv | 2 + .../vn_simona_small_withem.conf | 210 +++++ .../ie3/simona/agent/EnvironmentRefs.scala | 1 + .../edu/ie3/simona/agent/em/EmAgent.scala | 144 +++- .../agent/grid/GridAgentController.scala | 44 +- .../agent/participant/ParticipantAgent.scala | 2 +- .../ParticipantAgentFundamentals.scala | 4 + .../event/listener/ResultEventListener.scala | 2 +- .../ie3/simona/main/RunSimonaWithOpsim.scala | 62 ++ .../RunSimonaWithSimpleExtSimulation.scala | 62 ++ .../messages/flex/FlexibilityMessage.scala | 2 +- .../messages/services/ServiceMessage.scala | 28 +- .../simona/service/em/ExtEmDataService.scala | 73 +- .../results/ExtResultDataProvider.scala | 2 +- .../scala/edu/ie3/simona/sim/SimonaSim.scala | 1 + .../simona/sim/setup/ExtSimSetupData.scala | 10 + .../simona/sim/setup/SimonaMosaikSetup.scala | 2 +- .../simona/sim/setup/SimonaOpsimSetup.scala | 755 +++++++++++++++++ .../SimonaSimpleExtSimulationSetup.scala | 756 ++++++++++++++++++ 50 files changed, 2859 insertions(+), 65 deletions(-) create mode 100755 input/samples/simopsimtestgrid_reduced/fullGrid/em_input.csv create mode 100755 input/samples/simopsimtestgrid_reduced/fullGrid/line_input.csv create mode 100755 input/samples/simopsimtestgrid_reduced/fullGrid/line_type_input.csv create mode 100755 input/samples/simopsimtestgrid_reduced/fullGrid/load_input.csv create mode 100755 input/samples/simopsimtestgrid_reduced/fullGrid/node_input.csv create mode 100755 input/samples/simopsimtestgrid_reduced/fullGrid/pv_input.csv create mode 100755 input/samples/simopsimtestgrid_reduced/fullGrid/transformer_2_w_input.csv create mode 100755 input/samples/simopsimtestgrid_reduced/fullGrid/transformer_2_w_type_input.csv create mode 100755 input/samples/simopsimtestgrid_reduced/simopsimtestgrid.conf create mode 100755 input/samples/vn_simona_small_with_em/fullGrid/em_input.csv create mode 100755 input/samples/vn_simona_small_with_em/fullGrid/fixed_feed_in_input.csv create mode 100755 input/samples/vn_simona_small_with_em/fullGrid/line_input.csv create mode 100755 input/samples/vn_simona_small_with_em/fullGrid/line_type_input.csv create mode 100755 input/samples/vn_simona_small_with_em/fullGrid/load_input.csv create mode 100755 input/samples/vn_simona_small_with_em/fullGrid/node_input.csv create mode 100644 input/samples/vn_simona_small_with_em/fullGrid/pv_input.csv create mode 100644 input/samples/vn_simona_small_with_em/fullGrid/storage_input.csv create mode 100644 input/samples/vn_simona_small_with_em/fullGrid/storage_type_input.csv create mode 100755 input/samples/vn_simona_small_with_em/fullGrid/transformer_2_w_input.csv create mode 100755 input/samples/vn_simona_small_with_em/fullGrid/transformer_2_w_type_input.csv create mode 100755 input/samples/vn_simona_small_with_em/vn_simona_small_withem.conf create mode 100755 input/samples/vn_simona_small_with_em_original/fullGrid/em_input.csv create mode 100755 input/samples/vn_simona_small_with_em_original/fullGrid/fixed_feed_in_input.csv create mode 100755 input/samples/vn_simona_small_with_em_original/fullGrid/line_input.csv create mode 100755 input/samples/vn_simona_small_with_em_original/fullGrid/line_type_input.csv create mode 100755 input/samples/vn_simona_small_with_em_original/fullGrid/load_input.csv create mode 100755 input/samples/vn_simona_small_with_em_original/fullGrid/node_input.csv create mode 100644 input/samples/vn_simona_small_with_em_original/fullGrid/pv_input.csv create mode 100644 input/samples/vn_simona_small_with_em_original/fullGrid/storage_input.csv create mode 100644 input/samples/vn_simona_small_with_em_original/fullGrid/storage_type_input.csv create mode 100755 input/samples/vn_simona_small_with_em_original/fullGrid/transformer_2_w_input.csv create mode 100755 input/samples/vn_simona_small_with_em_original/fullGrid/transformer_2_w_type_input.csv create mode 100755 input/samples/vn_simona_small_with_em_original/vn_simona_small_withem.conf create mode 100644 src/main/scala/edu/ie3/simona/main/RunSimonaWithOpsim.scala create mode 100644 src/main/scala/edu/ie3/simona/main/RunSimonaWithSimpleExtSimulation.scala create mode 100644 src/main/scala/edu/ie3/simona/sim/setup/SimonaOpsimSetup.scala create mode 100644 src/main/scala/edu/ie3/simona/sim/setup/SimonaSimpleExtSimulationSetup.scala diff --git a/input/samples/simopsimtestgrid_reduced/fullGrid/em_input.csv b/input/samples/simopsimtestgrid_reduced/fullGrid/em_input.csv new file mode 100755 index 0000000000..b6a794fe65 --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced/fullGrid/em_input.csv @@ -0,0 +1,4 @@ +uuid;control_strategy;parent_em;id;operates_from;operates_until;operator +c3a7e9f5-b492-4c85-af2d-1e93f6a25443;self_optimization;;EM_HH_Bus_25;;; +f9dc7ce6-658c-4101-a12f-d58bb889286b;self_optimization;;EM_HH_Bus_81;;; +957938b7-0476-4fab-a1b3-6ce8615857b3;self_optimization;;EM_HH_Bus_110;;; diff --git a/input/samples/simopsimtestgrid_reduced/fullGrid/line_input.csv b/input/samples/simopsimtestgrid_reduced/fullGrid/line_input.csv new file mode 100755 index 0000000000..f4ff92fbba --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced/fullGrid/line_input.csv @@ -0,0 +1,110 @@ +uuid;geo_position;id;length;node_a;node_b;olm_characteristic;operates_from;operates_until;operator;parallel_devices;type +bdbbb247-57b7-473b-9411-53fcf35032db;"{""type"":""LineString"",""coordinates"":[[11.8213,53.426],[11.8213,53.4257]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 27;0.0161277;093160c4-6482-4c58-b952-217c615e3ada;9cdb3115-cc00-4d61-bc33-442e8f30fb63;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +c48eeb2c-6858-4f96-9a90-20641cc0903d;"{""type"":""LineString"",""coordinates"":[[11.8214,53.4229],[11.8213,53.4225]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 35;0.0150703;002a4495-96e4-49c9-abbe-8fccb3e9c83e;b909fb45-b6ee-427f-afd7-e8a0ec7274c6;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +97e498ae-a4f7-4485-a703-b8c00d0a5e8a;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4221],[11.8201,53.4225]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 8;0.0228449;b565ae3b-68f9-4bca-816d-9b0fc1c6b13f;a7725293-05fc-447f-bc12-38b689b0a956;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +f907f152-7bfe-4a17-a63f-adda500f6f0e;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4241],[11.8184,53.4293]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 107;0.04;4749ab2b-4d96-4100-8081-73e77c797d6b;ea4a6507-e504-4542-be5f-1019719b2257;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +6bf14400-8f6c-41ef-b6a4-9e861a3ad08f;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4241],[11.822,53.4294]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 3;0.030304;4749ab2b-4d96-4100-8081-73e77c797d6b;c5b6bfaf-1621-40a7-9c53-02cfb59c04d9;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +18b92b71-320d-4294-a47e-6715f1594755;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4243],[11.8196,53.4247]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 102;0.0251089;f4da61e4-7600-4cd1-95b6-c70b56c049fc;e7908208-77b4-4059-806e-4857262992fc;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +4128307f-3e00-4da9-b629-b696b72165a4;"{""type"":""LineString"",""coordinates"":[[11.815,53.4289],[11.815,53.4293]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 28;0.0377768;8e809cf6-7e05-437c-89a6-f6ca135a403b;35748e60-3be8-4930-8a61-209fd5df1bec;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +92876b53-6bbc-48ff-ba5f-5f5c08313e4d;"{""type"":""LineString"",""coordinates"":[[11.815,53.4277],[11.815,53.4272]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 111;0.031;3e6be3ac-2b51-4080-b815-391313612fc7;78815cf6-70db-432c-96e6-87fe8cf67eee;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +4b557d0c-9992-48b1-b45e-26cccc01db03;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4235],[11.8191,53.4231]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 90;0.00167204;f2724954-34d3-4ddf-b6b0-7a1531639990;2efac9b1-fb0d-4e08-bfac-501798826deb;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +0652d3e0-8c7c-4be2-9a41-00534aa9774b;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4212],[11.8191,53.4209]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 94;0.00182131;02e63e81-2e62-4ef6-8fdb-0b0905a437b6;fd4bebb8-40ca-4eed-92c0-cdd10b86ac20;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +782d8c4c-accb-424b-91ae-9af1b5d4b1af;"{""type"":""LineString"",""coordinates"":[[11.8201,53.4225],[11.8201,53.4229]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 79;0.0338129;a7725293-05fc-447f-bc12-38b689b0a956;83da8d60-405a-45f7-9bb9-9d35607b7927;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +b7cac7a5-f528-45bd-8ced-a16234979e13;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4239],[11.8191,53.4235]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 89;0.0111411;b5c1e826-63fd-4b0c-bec0-0c758389ef58;f2724954-34d3-4ddf-b6b0-7a1531639990;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +7ac1a715-e141-429a-8cfe-ff2badd41074;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4257],[11.8174,53.4262]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 65;0.019;8726dc29-621e-4455-a541-cd88d7da457f;814f784b-687f-4dd5-8a91-c7772c916d46;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +77ae4f1e-9ecb-4dda-a5f2-ce7ae3f9bbec;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4264],[11.8213,53.426]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 23;0.00488535;18b4157b-0e47-4c5a-adb8-ccae47372336;093160c4-6482-4c58-b952-217c615e3ada;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +6aaa54d9-d7b7-4adc-a85a-335cdf1393d6;"{""type"":""LineString"",""coordinates"":[[11.8162,53.4284],[11.8169,53.4289]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 40;0.036;6b2881ce-3965-4f5e-98a3-74eb47b0a7ca;a882e666-82d1-4ba6-87df-fc702fe06187;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +c85c107e-82c0-4002-acfa-d7000512a2ad;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4274],[11.8213,53.4271]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 20;0.0182765;2b3d7fb8-0583-4d47-97b1-3b5f232fd462;7b81b518-00e0-4ff1-b4cf-876903958d7a;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +2ef5f888-4341-4eee-b505-ae07a9a60c8d;"{""type"":""LineString"",""coordinates"":[[11.8201,53.4236],[11.8201,53.4239]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 55;0.0259846;f713593a-3fd3-410a-ac08-74202d4f5798;80962bd3-a10f-4ed2-ba6a-3e802189939c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +139ea4f9-9b7f-4825-8919-537a94ff4794;"{""type"":""LineString"",""coordinates"":[[11.8175,53.4277],[11.8174,53.4281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 59;0.00160952;7f692039-eef6-45f6-9e30-b5983f6750a5;3f6c26dd-842b-4dee-b71f-4aa32e2654ff;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +68853808-8b09-4ff2-9c92-88ed1d78c639;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4236],[11.8213,53.4232]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 33;0.0217875;3464496c-7dd4-41e9-ae0a-99ade0b51572;a966644c-37d5-4400-9fa0-725fd88586a8;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +91bec60c-2951-420c-a35e-2633119ee450;"{""type"":""LineString"",""coordinates"":[[11.8162,53.4277],[11.8162,53.4281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 30;0.00820054;2d33314e-31db-4ad4-a898-2be5d56a896d;555cd075-0fe4-4a65-b027-f45cffa960d9;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +35ba0827-e27e-496e-b735-c778f3b03019;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4241],[11.8213,53.4293]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 18;0.0139169;4749ab2b-4d96-4100-8081-73e77c797d6b;844c0b9c-058a-4228-a8c4-bf2defff6958;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +60efd159-72d6-4546-b245-8b8fc120a9f8;"{""type"":""LineString"",""coordinates"":[[11.817,53.4293],[11.8166,53.4241]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 26;0.098;85751c9b-3e5e-468a-9dc7-43775b0d4a6f;4749ab2b-4d96-4100-8081-73e77c797d6b;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +212b62aa-73c2-47af-95ec-00cad8d4a4f4;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4222],[11.8214,53.4218]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 38;0.00992653;589bf32a-b361-4536-ae96-6d56d184eedb;3b86661a-187d-4aa6-bf37-2014789afc08;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +7e61def1-8414-40b3-8775-fad9124f4369;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4282],[11.8213,53.4277]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 16;0.0164234;09285b78-9c18-4af7-9c7a-942cc868016f;3a557b4e-06b8-4f29-929f-81d95c42c897;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +f9b4c7dc-c199-4691-8c6d-1faf438cf336;"{""type"":""LineString"",""coordinates"":[[11.8221,53.429],[11.8221,53.4286]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 5;0.0143698;9644f198-e801-4545-87ee-a24e2a8039bd;6ee7ea93-ea9c-40cb-b79a-1c5f287c97a5;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +eebc456e-d11b-4b19-9100-cc1d8e91f926;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4245],[11.8166,53.4249]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 71;0.00648856;7efabb8d-ba17-4487-96d9-5744b1fedf8a;7c35a794-f569-4a9c-acb0-d03647610086;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +8f4c4009-e1ed-4985-8095-729aa5dc8cdd;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4192],[11.8198,53.4195]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 85;0.00737032;594d101c-3a05-45e3-a061-9189f3e848b7;af2b07ce-1a96-4b50-9e21-badf29eed519;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +20db47c8-d154-4ab2-ad65-caa4a37466d2;"{""type"":""LineString"",""coordinates"":[[11.8175,53.4257],[11.8174,53.4262]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 68;0.037;5f153bf6-4f25-41f1-8545-18fe6323bc49;814f784b-687f-4dd5-8a91-c7772c916d46;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +7d838003-cd6e-42fd-b75a-eaa354c3993b;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4216],[11.8191,53.4212]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 93;0.0043578;8f53645e-1f28-4eb1-807f-eb2a473f9d4c;02e63e81-2e62-4ef6-8fdb-0b0905a437b6;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +a92013e1-78b7-4447-a505-f25fffbf56f2;"{""type"":""LineString"",""coordinates"":[[11.8174,53.4262],[11.8175,53.4265]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 48;0.0118531;814f784b-687f-4dd5-8a91-c7772c916d46;94fe96b1-f36a-4edd-a107-4ff0376f1066;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +690f8f61-5cc7-448e-8971-a516f784bf11;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4293],[11.8166,53.4241]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 106;0.0131451;20ad9754-e966-4ad1-9541-f968c207f3df;4749ab2b-4d96-4100-8081-73e77c797d6b;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +6a00d582-32b1-4581-bdd1-a638ca279597;"{""type"":""LineString"",""coordinates"":[[11.815,53.4272],[11.815,53.4269]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 112;0.028;78815cf6-70db-432c-96e6-87fe8cf67eee;ab8c8f6c-e7a2-4b81-a0d7-5f13789267a2;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +1d5877c1-31cd-4ee8-b0dd-667fa4fffb8a;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4277],[11.8213,53.4274]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 17;0.00286133;3a557b4e-06b8-4f29-929f-81d95c42c897;2b3d7fb8-0583-4d47-97b1-3b5f232fd462;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +a15cd6a4-4b22-434f-be9b-ad2abe6e538c;"{""type"":""LineString"",""coordinates"":[[11.815,53.4289],[11.815,53.4285]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 108;0.033;8e809cf6-7e05-437c-89a6-f6ca135a403b;88cf719a-92df-4dfd-9a83-f84330e28fe0;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +2bd81511-4f19-4374-8ac5-96c2b19eda64;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4223],[11.8191,53.422]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 52;0.00980013;c7e48384-5699-4a38-a887-7e15a9145202;c6dac3ab-f44f-4b87-800c-0f4da64673f1;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +beee40ad-aca3-490f-87d5-a6dfc4bb76e3;"{""type"":""LineString"",""coordinates"":[[11.815,53.4293],[11.8166,53.4241]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 39;0.09;35748e60-3be8-4930-8a61-209fd5df1bec;4749ab2b-4d96-4100-8081-73e77c797d6b;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +96b06f87-7905-4d12-99d8-1ed330050659;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4203],[11.8197,53.4207]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 12;0.00473835;5596da2f-ca32-4ad3-81da-1ffa17cd3d7b;839ff0f4-93db-42ec-a928-bbc448b6cf5c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +e66890ef-ec02-407f-a5bb-ce32128e7490;"{""type"":""LineString"",""coordinates"":[[11.8162,53.4281],[11.8162,53.4284]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 41;0.00828244;555cd075-0fe4-4a65-b027-f45cffa960d9;6b2881ce-3965-4f5e-98a3-74eb47b0a7ca;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +015b0337-98bd-40d4-97d3-13a0d1da88ee;"{""type"":""LineString"",""coordinates"":[[11.8213,53.425],[11.8214,53.4246]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 84;0.0489789;58b551b6-83bd-4f1c-8d9c-8c9a7f638c0b;a9288e77-2919-4db6-89eb-9737bd07f111;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +f9e6b2f1-f8fb-4763-a1d0-36d06170fea0;"{""type"":""LineString"",""coordinates"":[[11.8213,53.429],[11.8213,53.4286]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 98;0.0144283;16091b6d-f1ea-4a07-9ad4-30d595aba68d;1403edf9-e47c-4705-8563-83bcd639482e;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +f270d6b5-7c94-4397-8ab0-7c39c888d726;"{""type"":""LineString"",""coordinates"":[[11.8175,53.4265],[11.8175,53.427]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 58;0.0036937;94fe96b1-f36a-4edd-a107-4ff0376f1066;80b8d1f8-7e83-421d-a95a-c193fc35f4f7;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +af8c65aa-cfd6-434a-8512-2d80106a2f2c;"{""type"":""LineString"",""coordinates"":[[11.8195,53.426],[11.8195,53.4264]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 31;0.00626899;9f1baf4e-12e1-41d1-8efc-81cfc78f1957;b8e95bf0-3ba8-4d53-a0bf-a3720fb785fb;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +fe28e831-6405-4dfe-987b-d688367694f1;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4247],[11.8195,53.425]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 104;0.00221503;e7908208-77b4-4059-806e-4857262992fc;29516ae3-6676-4797-99c1-1f0a32b989d8;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +eac56d69-5500-4261-9690-adb16c867485;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4231],[11.8191,53.4227]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 92;0.0149219;2efac9b1-fb0d-4e08-bfac-501798826deb;15a86f7d-fb73-49a4-af6a-25b14122378d;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +183f3976-48ca-42b2-9af9-7998436fac5b;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4289],[11.8196,53.4293]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 105;0.00721947;47246a84-ad0e-4d04-9d98-1c9cd5d363c1;20ad9754-e966-4ad1-9541-f968c207f3df;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +7feef458-03f3-4d23-b3a2-e6f1035398c4;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4243],[11.8213,53.4239]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 29;0.0204277;3fcb94e3-7781-4d83-9030-d9853822e78e;d0bfabdb-0e83-423b-a20a-ab9197c4284e;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +592cd979-16c9-43d8-a311-8ac938aa5d03;"{""type"":""LineString"",""coordinates"":[[11.8174,53.4262],[11.8182,53.4257]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 47;0.0181601;814f784b-687f-4dd5-8a91-c7772c916d46;c86d6361-4159-4787-b5f4-e41dcaa95195;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +684146e5-3b58-43b4-9589-45325ab1c0bc;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4257],[11.8213,53.4253]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 24;0.0102233;9cdb3115-cc00-4d61-bc33-442e8f30fb63;a432b8ce-0462-478b-83e7-3107cd2e909c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +d9f9ee76-b016-4588-ac6d-46681894ada7;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4188],[11.8198,53.4192]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 74;0.00421724;952c6b04-0d02-4ea2-a6cc-bb44fbbe4e52;594d101c-3a05-45e3-a061-9189f3e848b7;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +1f7e9cae-460e-47db-8cb8-da5d9f695fd8;"{""type"":""LineString"",""coordinates"":[[11.8197,53.4207],[11.8198,53.4211]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 13;0.00987733;839ff0f4-93db-42ec-a928-bbc448b6cf5c;27b84da5-478e-4a05-8fe7-a9f800db5eff;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +f3a592d0-0fd7-42ea-b928-f39473b419aa;"{""type"":""LineString"",""coordinates"":[[11.8214,53.4246],[11.8213,53.4243]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 10;0.0344866;a9288e77-2919-4db6-89eb-9737bd07f111;3fcb94e3-7781-4d83-9030-d9853822e78e;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +f505bff9-0803-415f-a765-9da981ff6024;"{""type"":""LineString"",""coordinates"":[[11.8195,53.4264],[11.8195,53.4268]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 43;0.00839819;b8e95bf0-3ba8-4d53-a0bf-a3720fb785fb;34031e92-3444-47d5-94ae-cceeb5d96bb2;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +c6501262-2e05-462b-8872-445d2aa1cab8;"{""type"":""LineString"",""coordinates"":[[11.8195,53.4272],[11.8195,53.4275]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 99;0.0157523;120eaa58-a500-4ae2-a86a-56a40b931ec1;9d136a6b-5fdc-44ed-a5ed-599a55281024;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +caac4b4b-4871-4e20-994c-6517931546cb;"{""type"":""LineString"",""coordinates"":[[11.8201,53.4229],[11.8201,53.4232]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 91;0.0220129;83da8d60-405a-45f7-9bb9-9d35607b7927;24b63115-12eb-4e77-b9ef-ca474fed960f;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +e9721561-53c0-45cc-a8ed-28861ef9dc66;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4216],[11.8198,53.4221]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 7;0.0333523;883edf38-9a18-4f61-981a-691aaf436cc7;b565ae3b-68f9-4bca-816d-9b0fc1c6b13f;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +ab143df6-d050-47b6-911a-93e462d928ac;"{""type"":""LineString"",""coordinates"":[[11.8173,53.4234],[11.8166,53.4238]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 73;0.0287121;e68a088d-cf1a-40b7-9b1a-e0933352f4e6;4129e079-6712-4275-911c-36729d698c42;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +ab7ab785-36ba-4da1-b176-a7c636cb1372;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4267],[11.8213,53.4264]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 22;0.0102382;dc54bd8a-b7d8-4e99-adb0-d6ee5084241c;18b4157b-0e47-4c5a-adb8-ccae47372336;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +1ae90a03-52a3-40ef-8e06-4ba01888aa5c;"{""type"":""LineString"",""coordinates"":[[11.816,53.4222],[11.816,53.4227]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 44;0.00743191;62d603c3-f306-40b3-a665-ba9892d226f0;3faac527-0ff3-44a7-9e4f-24a41940da90;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +20be9235-f4db-4753-9fa1-223c8519fcd3;"{""type"":""LineString"",""coordinates"":[[11.8175,53.4253],[11.8175,53.4257]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 66;0.00791441;73e7a7e8-2154-46ea-9727-a4916af3570c;5f153bf6-4f25-41f1-8545-18fe6323bc49;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +558e1545-a944-419a-9153-83caa09e1a3c;"{""type"":""LineString"",""coordinates"":[[11.8162,53.4273],[11.8162,53.4277]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 42;0.00357726;0d6ef8f6-0ba1-4fa7-8e63-e55cee12b165;2d33314e-31db-4ad4-a898-2be5d56a896d;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +38d60cf7-6099-4bc0-a616-0f0b66c70c9a;"{""type"":""LineString"",""coordinates"":[[11.8201,53.4232],[11.8201,53.4236]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 67;0.0179092;24b63115-12eb-4e77-b9ef-ca474fed960f;f713593a-3fd3-410a-ac08-74202d4f5798;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +e74dbbe9-948f-4056-8134-fd1d9d39e773;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4253],[11.8196,53.4257]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 86;0.0190003;03b2aa45-84f6-48c0-9dab-427e046a5672;79e19265-08e8-407f-ae95-2f78e344d3a4;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +24c0dc1c-2e99-46ed-a52b-5a7aae7c9afb;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4249],[11.8166,53.4253]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 72;0.00189914;7c35a794-f569-4a9c-acb0-d03647610086;50cac08e-bf24-4526-9466-53ca5edccd15;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +41a72cb2-f037-4196-a248-2b18a578db50;"{""type"":""LineString"",""coordinates"":[[11.816,53.421],[11.816,53.4214]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 78;0.00156586;98c14f60-e196-4f12-903b-8485f1eacb16;ca0b9e34-ddc8-405e-86a4-ab939626c4a2;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +e2919117-f70a-4b32-a5ab-f077fe0a84ef;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4253],[11.8166,53.4257]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 70;0.00263875;50cac08e-bf24-4526-9466-53ca5edccd15;8726dc29-621e-4455-a541-cd88d7da457f;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +7d3ee5e1-1749-4e43-afeb-4ccf5f431312;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4282],[11.8196,53.4286]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 103;0.011354;a3ed5af5-1fcb-4fce-af0f-708d3d604124;31e6e197-719d-4aaf-8ca5-ab9e7549390e;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +cbba4a6b-27d1-4724-a193-134af220a57d;"{""type"":""LineString"",""coordinates"":[[11.8182,53.4257],[11.8182,53.4254]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 49;0.018118;c86d6361-4159-4787-b5f4-e41dcaa95195;c6c20ab1-16fb-4f82-8017-273022da8bb0;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +b84fad90-182c-4ac1-b77d-394f39024ffc;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4286],[11.8196,53.4289]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 19;0.0115943;31e6e197-719d-4aaf-8ca5-ab9e7549390e;47246a84-ad0e-4d04-9d98-1c9cd5d363c1;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +6c87a571-9d8b-4df9-bd86-cde7954bee28;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4271],[11.8213,53.4267]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 21;0.00282037;7b81b518-00e0-4ff1-b4cf-876903958d7a;dc54bd8a-b7d8-4e99-adb0-d6ee5084241c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +536ef1a2-b988-4474-a5d4-1254598c4716;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4227],[11.8191,53.4223]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 82;0.00544392;15a86f7d-fb73-49a4-af6a-25b14122378d;c7e48384-5699-4a38-a887-7e15a9145202;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +c0c99b3a-1c61-47b8-931d-571d9494d98f;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4241],[11.8166,53.4245]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 51;0.0182324;5682ac05-7336-4ebc-a5d1-3b69c79fb3b1;7efabb8d-ba17-4487-96d9-5744b1fedf8a;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +d3366635-447e-4d45-b839-7a7561f869a3;"{""type"":""LineString"",""coordinates"":[[11.822,53.4294],[11.8221,53.429]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 4;0.0141339;c5b6bfaf-1621-40a7-9c53-02cfb59c04d9;9644f198-e801-4545-87ee-a24e2a8039bd;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +932dd0d7-2051-4a3a-a0e3-489210d3b763;"{""type"":""LineString"",""coordinates"":[[11.8169,53.4289],[11.817,53.4293]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 57;0.0219657;a882e666-82d1-4ba6-87df-fc702fe06187;85751c9b-3e5e-468a-9dc7-43775b0d4a6f;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +e718d253-34ea-4c00-a653-80ac0af2934e;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4293],[11.8213,53.429]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 11;0.00557129;844c0b9c-058a-4228-a8c4-bf2defff6958;16091b6d-f1ea-4a07-9ad4-30d595aba68d;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +3ac954bf-5c1e-428a-9da7-37331f53d8fe;"{""type"":""LineString"",""coordinates"":[[11.8175,53.4248],[11.8175,53.4253]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 69;0.0017759;0d6c3f2b-5296-4ec1-995c-b150e72f035f;73e7a7e8-2154-46ea-9727-a4916af3570c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +7e9a0440-2a62-4daf-a4ab-e9f14f6cfd77;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4238],[11.8166,53.4234]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 53;0.0337731;4129e079-6712-4275-911c-36729d698c42;57b40047-4f9d-46bb-bf19-c0a86bbd4f5b;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +10a2050e-173b-43a7-91a0-157cee5c900d;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4257],[11.8195,53.426]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 87;0.00654022;79e19265-08e8-407f-ae95-2f78e344d3a4;9f1baf4e-12e1-41d1-8efc-81cfc78f1957;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +0dfd30a5-394e-49ff-95e9-50e73bf2604c;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4232],[11.8214,53.4229]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 34;0.0199334;a966644c-37d5-4400-9fa0-725fd88586a8;002a4495-96e4-49c9-abbe-8fccb3e9c83e;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +847934a0-e2b7-4caf-bb40-2e6af5bcb0ab;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4238],[11.8166,53.4241]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 50;0.0101795;4129e079-6712-4275-911c-36729d698c42;5682ac05-7336-4ebc-a5d1-3b69c79fb3b1;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +2daf9021-eb24-4d0c-bced-00e713a65b3a;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4225],[11.8213,53.4222]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 36;0.010188;b909fb45-b6ee-427f-afd7-e8a0ec7274c6;589bf32a-b361-4536-ae96-6d56d184eedb;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +2f1b02a3-5a58-4723-ab8f-827118a4c611;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4239],[11.8213,53.4236]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 32;0.0188563;d0bfabdb-0e83-423b-a20a-ab9197c4284e;3464496c-7dd4-41e9-ae0a-99ade0b51572;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +a5e9b6b0-2afc-49b0-8aad-705410b823c2;"{""type"":""LineString"",""coordinates"":[[11.815,53.4285],[11.815,53.4281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 109;0.027;88cf719a-92df-4dfd-9a83-f84330e28fe0;d38d936a-9c05-4bdc-8331-418fef27f492;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +ae82a1c1-9790-44d1-8dab-52d75c7b79ce;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4286],[11.8213,53.4282]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 6;0.0050064;1403edf9-e47c-4705-8563-83bcd639482e;09285b78-9c18-4af7-9c7a-942cc868016f;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +e642eafb-2b6e-4b78-80b8-1ab15db8cfeb;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4278],[11.8196,53.4282]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 101;0.0153312;7dc43c81-9a61-45a0-9745-800a28bf4a9d;a3ed5af5-1fcb-4fce-af0f-708d3d604124;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +41a4c5e5-7b54-46a8-b6b6-d7b5861a251c;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4195],[11.8198,53.4199]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 96;0.0010841;af2b07ce-1a96-4b50-9e21-badf29eed519;215eaa45-82c3-49c7-a60f-4fa13215de05;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +44821cfc-7670-4f28-8941-70e9345cb069;"{""type"":""LineString"",""coordinates"":[[11.8201,53.4225],[11.8203,53.4221]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 15;0.027677;a7725293-05fc-447f-bc12-38b689b0a956;eae8a04c-44f2-4da3-95f6-cae48f85737c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +d665e8c6-40fb-4608-97fb-99a4cf52560e;"{""type"":""LineString"",""coordinates"":[[11.815,53.4281],[11.815,53.4277]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 110;0.041;d38d936a-9c05-4bdc-8331-418fef27f492;3e6be3ac-2b51-4080-b815-391313612fc7;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +2f37bd36-f356-4d29-81c7-dd9c2fba5e7d;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4234],[11.8166,53.423]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 45;0.00223482;57b40047-4f9d-46bb-bf19-c0a86bbd4f5b;5b73ded9-3ca4-4f18-a2ab-c27b9a3dcf9d;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +bcccf0b5-e0e3-4cc7-82bb-262ebc19415e;"{""type"":""LineString"",""coordinates"":[[11.8169,53.4289],[11.8168,53.4285]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 56;0.039;a882e666-82d1-4ba6-87df-fc702fe06187;c41ebab8-16a0-4a3a-b4af-26073932d462;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +3685f389-6822-4522-a633-74265b67eaee;"{""type"":""LineString"",""coordinates"":[[11.816,53.4227],[11.816,53.423]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 76;0.00185168;3faac527-0ff3-44a7-9e4f-24a41940da90;2575f527-1f4e-45e2-bed2-4a5427f122e0;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +48347702-9e58-49f3-a7de-72024d4b296c;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4199],[11.8198,53.4203]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 2;0.00995493;215eaa45-82c3-49c7-a60f-4fa13215de05;5596da2f-ca32-4ad3-81da-1ffa17cd3d7b;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +ec2a73ea-50ba-4187-b8b0-a5046cf6b632;"{""type"":""LineString"",""coordinates"":[[11.8195,53.425],[11.8196,53.4253]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 83;0.00510961;29516ae3-6676-4797-99c1-1f0a32b989d8;03b2aa45-84f6-48c0-9dab-427e046a5672;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +d1940183-f7bb-42df-b8ff-63ac7aff0b1d;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4211],[11.8198,53.4216]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 9;0.00439896;27b84da5-478e-4a05-8fe7-a9f800db5eff;883edf38-9a18-4f61-981a-691aaf436cc7;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +dd618e3f-9ef9-4e4b-b632-737d79c4d8c3;"{""type"":""LineString"",""coordinates"":[[11.8175,53.427],[11.8174,53.4273]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 62;0.00124645;80b8d1f8-7e83-421d-a95a-c193fc35f4f7;8ace5c2b-584a-4015-990f-6f1e14de4ddb;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +ed80666a-cf86-404a-b3a7-ad49be1cd40c;"{""type"":""LineString"",""coordinates"":[[11.816,53.423],[11.816,53.4234]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 77;0.00232237;2575f527-1f4e-45e2-bed2-4a5427f122e0;5d3bcf55-0520-43ff-8d63-3d0eb421e442;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +444615ed-26f2-45a7-8b5d-213c72e83a4f;"{""type"":""LineString"",""coordinates"":[[11.8201,53.4239],[11.8196,53.4243]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 95;0.0281251;80962bd3-a10f-4ed2-ba6a-3e802189939c;f4da61e4-7600-4cd1-95b6-c70b56c049fc;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +05ebbc17-61bc-4810-ae19-9ee04d7ce8d0;"{""type"":""LineString"",""coordinates"":[[11.8175,53.4285],[11.8169,53.4289]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 54;0.0173762;30b2ce2d-41dd-4d1d-866d-47abd24bfc3e;a882e666-82d1-4ba6-87df-fc702fe06187;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +caf90323-aad2-496e-a0e3-a4ba7c9a481c;"{""type"":""LineString"",""coordinates"":[[11.8174,53.4273],[11.8175,53.4277]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 64;0.00314803;8ace5c2b-584a-4015-990f-6f1e14de4ddb;7f692039-eef6-45f6-9e30-b5983f6750a5;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +60b400db-b048-4bed-be29-8abc18780d10;"{""type"":""LineString"",""coordinates"":[[11.816,53.4206],[11.816,53.421]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 14;0.045296;d6dbb0ae-13c9-438e-93b3-b6c63a0708df;98c14f60-e196-4f12-903b-8485f1eacb16;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +cf13634d-abd2-465d-8839-c95a54af7a80;"{""type"":""LineString"",""coordinates"":[[11.816,53.4214],[11.816,53.4218]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 80;0.0101988;ca0b9e34-ddc8-405e-86a4-ab939626c4a2;43040a39-8b6c-401f-9dfd-82b42aa6dec6;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +30044e09-a0f8-417d-a949-afcfa940f671;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4253],[11.8213,53.425]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 25;0.0124821;a432b8ce-0462-478b-83e7-3107cd2e909c;58b551b6-83bd-4f1c-8d9c-8c9a7f638c0b;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +fe71fe2b-8dd0-4942-b0b5-d241e095b912;"{""type"":""LineString"",""coordinates"":[[11.8195,53.4268],[11.8195,53.4272]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 97;0.0032723;34031e92-3444-47d5-94ae-cceeb5d96bb2;120eaa58-a500-4ae2-a86a-56a40b931ec1;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +66e44163-7091-4f1b-991c-64108e2238f6;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4243],[11.8191,53.4239]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 88;0.0185855;f4da61e4-7600-4cd1-95b6-c70b56c049fc;b5c1e826-63fd-4b0c-bec0-0c758389ef58;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +82bac681-169f-48d8-9842-fd69d3adbfe0;"{""type"":""LineString"",""coordinates"":[[11.816,53.4234],[11.8166,53.4238]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 75;0.0109137;5d3bcf55-0520-43ff-8d63-3d0eb421e442;4129e079-6712-4275-911c-36729d698c42;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +74a5dc42-e689-490f-a611-ae7c3767f01b;"{""type"":""LineString"",""coordinates"":[[11.816,53.4218],[11.816,53.4222]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 81;0.0100396;43040a39-8b6c-401f-9dfd-82b42aa6dec6;62d603c3-f306-40b3-a665-ba9892d226f0;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +0eedb3d0-cedc-4798-b42e-d8e8ef646b82;"{""type"":""LineString"",""coordinates"":[[11.8195,53.4275],[11.8196,53.4278]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 100;0.0129061;9d136a6b-5fdc-44ed-a5ed-599a55281024;7dc43c81-9a61-45a0-9745-800a28bf4a9d;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +b44446fd-2125-4c7b-850e-b0f3d6c8b110;"{""type"":""LineString"",""coordinates"":[[11.8174,53.4281],[11.8175,53.4285]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 61;0.014766;3f6c26dd-842b-4dee-b71f-4aa32e2654ff;30b2ce2d-41dd-4d1d-866d-47abd24bfc3e;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +31c914bc-dd9e-4825-9b4b-b5fc1b971e0d;"{""type"":""LineString"",""coordinates"":[[11.8166,53.423],[11.8166,53.4227]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 46;0.00683612;5b73ded9-3ca4-4f18-a2ab-c27b9a3dcf9d;ad30322c-0c99-4669-8e4b-25265087a66d;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +e7ac7023-f4e3-460d-8d02-4c9c444d2e18;"{""type"":""LineString"",""coordinates"":[[11.8191,53.422],[11.8191,53.4216]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 63;0.0149947;c6dac3ab-f44f-4b87-800c-0f4da64673f1;8f53645e-1f28-4eb1-807f-eb2a473f9d4c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e diff --git a/input/samples/simopsimtestgrid_reduced/fullGrid/line_type_input.csv b/input/samples/simopsimtestgrid_reduced/fullGrid/line_type_input.csv new file mode 100755 index 0000000000..d7786c38cd --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced/fullGrid/line_type_input.csv @@ -0,0 +1,2 @@ +uuid;b;g;i_max;id;r;v_rated;x +9a8e9b63-af21-4c1b-8db7-fc2924f9610e;273.31899999999996;0.0;357.0;NAYY 4x240SE 0.6/1kV;0.1267;0.4;0.0797965 diff --git a/input/samples/simopsimtestgrid_reduced/fullGrid/load_input.csv b/input/samples/simopsimtestgrid_reduced/fullGrid/load_input.csv new file mode 100755 index 0000000000..6ad16189a5 --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced/fullGrid/load_input.csv @@ -0,0 +1,4 @@ +uuid;cos_phi_rated;dsm;e_cons_annual;id;load_profile;node;operates_from;operates_until;operator;q_characteristics;s_rated;em +c3434742-e4f0-49e5-baa7-c1e3045c732c;0.93;false;0.0;LV5.201 Load 64;h0;dc54bd8a-b7d8-4e99-adb0-d6ee5084241c;;;;"cosPhiFixed:{(0.0,0.93)}";3.2;f9dc7ce6-658c-4101-a12f-d58bb889286b +fd2e19b6-d5e3-4776-9456-8787a2160d9d;0.93;false;0.0;LV5.201 Load 74;h0;43040a39-8b6c-401f-9dfd-82b42aa6dec6;;;;"cosPhiFixed:{(0.0,0.93)}";1.1;c3a7e9f5-b492-4c85-af2d-1e93f6a25443 +98c1a2ab-bd09-4c77-a389-d088aed894b1;0.93;false;0.0;LV5.201 Load 102;h0;3e6be3ac-2b51-4080-b815-391313612fc7;;;;"cosPhiFixed:{(0.0,0.93)}";5.3;957938b7-0476-4fab-a1b3-6ce8615857b3 diff --git a/input/samples/simopsimtestgrid_reduced/fullGrid/node_input.csv b/input/samples/simopsimtestgrid_reduced/fullGrid/node_input.csv new file mode 100755 index 0000000000..ab9f9335ca --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced/fullGrid/node_input.csv @@ -0,0 +1,112 @@ +uuid;geo_position;id;operates_from;operates_until;operator;slack;subnet;v_rated;v_target;volt_lvl +16091b6d-f1ea-4a07-9ad4-30d595aba68d;"{""type"":""Point"",""coordinates"":[11.8213,53.429],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 26;;;;false;2;0.4;1.0;Niederspannung +27b84da5-478e-4a05-8fe7-a9f800db5eff;"{""type"":""Point"",""coordinates"":[11.8198,53.4211],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 7;;;;false;2;0.4;1.0;Niederspannung +7f692039-eef6-45f6-9e30-b5983f6750a5;"{""type"":""Point"",""coordinates"":[11.8175,53.4277],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 54;;;;false;2;0.4;1.0;Niederspannung +20ad9754-e966-4ad1-9541-f968c207f3df;"{""type"":""Point"",""coordinates"":[11.8196,53.4293],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 84;;;;false;2;0.4;1.0;Niederspannung +98c14f60-e196-4f12-903b-8485f1eacb16;"{""type"":""Point"",""coordinates"":[11.816,53.421],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 29;;;;false;2;0.4;1.0;Niederspannung +09285b78-9c18-4af7-9c7a-942cc868016f;"{""type"":""Point"",""coordinates"":[11.8213,53.4282],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 58;;;;false;2;0.4;1.0;Niederspannung +0d6ef8f6-0ba1-4fa7-8e63-e55cee12b165;"{""type"":""Point"",""coordinates"":[11.8162,53.4273],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 103;;;;false;2;0.4;1.0;Niederspannung +43040a39-8b6c-401f-9dfd-82b42aa6dec6;"{""type"":""Point"",""coordinates"":[11.816,53.4218],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 25;;;;false;2;0.4;1.0;Niederspannung +dc54bd8a-b7d8-4e99-adb0-d6ee5084241c;"{""type"":""Point"",""coordinates"":[11.8213,53.4267],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 81;;;;false;2;0.4;1.0;Niederspannung +2575f527-1f4e-45e2-bed2-4a5427f122e0;"{""type"":""Point"",""coordinates"":[11.816,53.423],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 3;;;;false;2;0.4;1.0;Niederspannung +30b2ce2d-41dd-4d1d-866d-47abd24bfc3e;"{""type"":""Point"",""coordinates"":[11.8175,53.4285],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 34;;;;false;2;0.4;1.0;Niederspannung +589bf32a-b361-4536-ae96-6d56d184eedb;"{""type"":""Point"",""coordinates"":[11.8213,53.4222],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 101;;;;false;2;0.4;1.0;Niederspannung +555cd075-0fe4-4a65-b027-f45cffa960d9;"{""type"":""Point"",""coordinates"":[11.8162,53.4281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 31;;;;false;2;0.4;1.0;Niederspannung +62d603c3-f306-40b3-a665-ba9892d226f0;"{""type"":""Point"",""coordinates"":[11.816,53.4222],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 55;;;;false;2;0.4;1.0;Niederspannung +215eaa45-82c3-49c7-a60f-4fa13215de05;"{""type"":""Point"",""coordinates"":[11.8198,53.4199],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 24;;;;false;2;0.4;1.0;Niederspannung +f4da61e4-7600-4cd1-95b6-c70b56c049fc;"{""type"":""Point"",""coordinates"":[11.8196,53.4243],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 76;;;;false;2;0.4;1.0;Niederspannung +73e7a7e8-2154-46ea-9727-a4916af3570c;"{""type"":""Point"",""coordinates"":[11.8175,53.4253],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 18;;;;false;2;0.4;1.0;Niederspannung +85751c9b-3e5e-468a-9dc7-43775b0d4a6f;"{""type"":""Point"",""coordinates"":[11.817,53.4293],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 51;;;;false;2;0.4;1.0;Niederspannung +093160c4-6482-4c58-b952-217c615e3ada;"{""type"":""Point"",""coordinates"":[11.8213,53.426],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 68;;;;false;2;0.4;1.0;Niederspannung +79e19265-08e8-407f-ae95-2f78e344d3a4;"{""type"":""Point"",""coordinates"":[11.8196,53.4257],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 28;;;;false;2;0.4;1.0;Niederspannung +a966644c-37d5-4400-9fa0-725fd88586a8;"{""type"":""Point"",""coordinates"":[11.8213,53.4232],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 59;;;;false;2;0.4;1.0;Niederspannung +952c6b04-0d02-4ea2-a6cc-bb44fbbe4e52;"{""type"":""Point"",""coordinates"":[11.8198,53.4188],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 94;;;;false;2;0.4;1.0;Niederspannung +c6c20ab1-16fb-4f82-8017-273022da8bb0;"{""type"":""Point"",""coordinates"":[11.8182,53.4254],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 62;;;;false;2;0.4;1.0;Niederspannung +e68a088d-cf1a-40b7-9b1a-e0933352f4e6;"{""type"":""Point"",""coordinates"":[11.8173,53.4234],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 67;;;;false;2;0.4;1.0;Niederspannung +8f53645e-1f28-4eb1-807f-eb2a473f9d4c;"{""type"":""Point"",""coordinates"":[11.8191,53.4216],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 86;;;;false;2;0.4;1.0;Niederspannung +0d6c3f2b-5296-4ec1-995c-b150e72f035f;"{""type"":""Point"",""coordinates"":[11.8175,53.4248],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 80;;;;false;2;0.4;1.0;Niederspannung +ea4a6507-e504-4542-be5f-1019719b2257;"{""type"":""Point"",""coordinates"":[11.8184,53.4293],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 111;;;;false;2;0.4;1.0;Niederspannung +9644f198-e801-4545-87ee-a24e2a8039bd;"{""type"":""Point"",""coordinates"":[11.8221,53.429],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 39;;;;false;2;0.4;1.0;Niederspannung +a9288e77-2919-4db6-89eb-9737bd07f111;"{""type"":""Point"",""coordinates"":[11.8214,53.4246],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 63;;;;false;2;0.4;1.0;Niederspannung +d6dbb0ae-13c9-438e-93b3-b6c63a0708df;"{""type"":""Point"",""coordinates"":[11.816,53.4206],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 89;;;;false;2;0.4;1.0;Niederspannung +5596da2f-ca32-4ad3-81da-1ffa17cd3d7b;"{""type"":""Point"",""coordinates"":[11.8198,53.4203],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 102;;;;false;2;0.4;1.0;Niederspannung +ad30322c-0c99-4669-8e4b-25265087a66d;"{""type"":""Point"",""coordinates"":[11.8166,53.4227],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 65;;;;false;2;0.4;1.0;Niederspannung +120eaa58-a500-4ae2-a86a-56a40b931ec1;"{""type"":""Point"",""coordinates"":[11.8195,53.4272],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 57;;;;false;2;0.4;1.0;Niederspannung +29516ae3-6676-4797-99c1-1f0a32b989d8;"{""type"":""Point"",""coordinates"":[11.8195,53.425],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 52;;;;false;2;0.4;1.0;Niederspannung +24b63115-12eb-4e77-b9ef-ca474fed960f;"{""type"":""Point"",""coordinates"":[11.8201,53.4232],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 46;;;;false;2;0.4;1.0;Niederspannung +a7725293-05fc-447f-bc12-38b689b0a956;"{""type"":""Point"",""coordinates"":[11.8201,53.4225],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 72;;;;false;2;0.4;1.0;Niederspannung +3b86661a-187d-4aa6-bf37-2014789afc08;"{""type"":""Point"",""coordinates"":[11.8214,53.4218],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 20;;;;false;2;0.4;1.0;Niederspannung +a3ed5af5-1fcb-4fce-af0f-708d3d604124;"{""type"":""Point"",""coordinates"":[11.8196,53.4282],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 33;;;;false;2;0.4;1.0;Niederspannung +ec8f2c82-a1b2-487c-b573-250859e3b414;"{""type"":""Point"",""coordinates"":[11.8166,53.4241],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";MV2.101 Bus 5;;;;true;1;20.0;1.025;Mittelspannung +5f153bf6-4f25-41f1-8545-18fe6323bc49;"{""type"":""Point"",""coordinates"":[11.8175,53.4257],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 17;;;;false;2;0.4;1.0;Niederspannung +b565ae3b-68f9-4bca-816d-9b0fc1c6b13f;"{""type"":""Point"",""coordinates"":[11.8198,53.4221],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 70;;;;false;2;0.4;1.0;Niederspannung +8e809cf6-7e05-437c-89a6-f6ca135a403b;"{""type"":""Point"",""coordinates"":[11.815,53.4289],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 47;;;;false;2;0.4;1.0;Niederspannung +ca0b9e34-ddc8-405e-86a4-ab939626c4a2;"{""type"":""Point"",""coordinates"":[11.816,53.4214],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 99;;;;false;2;0.4;1.0;Niederspannung +3e6be3ac-2b51-4080-b815-391313612fc7;"{""type"":""Point"",""coordinates"":[11.815,53.4277],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 110;;;;false;2;0.4;1.0;Niederspannung +4749ab2b-4d96-4100-8081-73e77c797d6b;"{""type"":""Point"",""coordinates"":[11.8166,53.4241],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 73;;;;false;2;0.4;1.0;Niederspannung +3f6c26dd-842b-4dee-b71f-4aa32e2654ff;"{""type"":""Point"",""coordinates"":[11.8174,53.4281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 97;;;;false;2;0.4;1.0;Niederspannung +c6dac3ab-f44f-4b87-800c-0f4da64673f1;"{""type"":""Point"",""coordinates"":[11.8191,53.422],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 10;;;;false;2;0.4;1.0;Niederspannung +eae8a04c-44f2-4da3-95f6-cae48f85737c;"{""type"":""Point"",""coordinates"":[11.8203,53.4221],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 45;;;;false;2;0.4;1.0;Niederspannung +94fe96b1-f36a-4edd-a107-4ff0376f1066;"{""type"":""Point"",""coordinates"":[11.8175,53.4265],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 61;;;;false;2;0.4;1.0;Niederspannung +31e6e197-719d-4aaf-8ca5-ab9e7549390e;"{""type"":""Point"",""coordinates"":[11.8196,53.4286],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 32;;;;false;2;0.4;1.0;Niederspannung +002a4495-96e4-49c9-abbe-8fccb3e9c83e;"{""type"":""Point"",""coordinates"":[11.8214,53.4229],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 90;;;;false;2;0.4;1.0;Niederspannung +c7e48384-5699-4a38-a887-7e15a9145202;"{""type"":""Point"",""coordinates"":[11.8191,53.4223],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 107;;;;false;2;0.4;1.0;Niederspannung +8ace5c2b-584a-4015-990f-6f1e14de4ddb;"{""type"":""Point"",""coordinates"":[11.8174,53.4273],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 27;;;;false;2;0.4;1.0;Niederspannung +c41ebab8-16a0-4a3a-b4af-26073932d462;"{""type"":""Point"",""coordinates"":[11.8168,53.4285],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 35;;;;false;2;0.4;1.0;Niederspannung +ab8c8f6c-e7a2-4b81-a0d7-5f13789267a2;"{""type"":""Point"",""coordinates"":[11.815,53.4269],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 113;;;;false;2;0.4;1.0;Niederspannung +9f1baf4e-12e1-41d1-8efc-81cfc78f1957;"{""type"":""Point"",""coordinates"":[11.8195,53.426],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 71;;;;false;2;0.4;1.0;Niederspannung +02e63e81-2e62-4ef6-8fdb-0b0905a437b6;"{""type"":""Point"",""coordinates"":[11.8191,53.4212],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 12;;;;false;2;0.4;1.0;Niederspannung +2d33314e-31db-4ad4-a898-2be5d56a896d;"{""type"":""Point"",""coordinates"":[11.8162,53.4277],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 48;;;;false;2;0.4;1.0;Niederspannung +b8e95bf0-3ba8-4d53-a0bf-a3720fb785fb;"{""type"":""Point"",""coordinates"":[11.8195,53.4264],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 49;;;;false;2;0.4;1.0;Niederspannung +d0bfabdb-0e83-423b-a20a-ab9197c4284e;"{""type"":""Point"",""coordinates"":[11.8213,53.4239],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 82;;;;false;2;0.4;1.0;Niederspannung +6b2881ce-3965-4f5e-98a3-74eb47b0a7ca;"{""type"":""Point"",""coordinates"":[11.8162,53.4284],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 53;;;;false;2;0.4;1.0;Niederspannung +af2b07ce-1a96-4b50-9e21-badf29eed519;"{""type"":""Point"",""coordinates"":[11.8198,53.4195],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 100;;;;false;2;0.4;1.0;Niederspannung +844c0b9c-058a-4228-a8c4-bf2defff6958;"{""type"":""Point"",""coordinates"":[11.8213,53.4293],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 1;;;;false;2;0.4;1.0;Niederspannung +a432b8ce-0462-478b-83e7-3107cd2e909c;"{""type"":""Point"",""coordinates"":[11.8213,53.4253],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 56;;;;false;2;0.4;1.0;Niederspannung +4129e079-6712-4275-911c-36729d698c42;"{""type"":""Point"",""coordinates"":[11.8166,53.4238],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 77;;;;false;2;0.4;1.0;Niederspannung +883edf38-9a18-4f61-981a-691aaf436cc7;"{""type"":""Point"",""coordinates"":[11.8198,53.4216],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 4;;;;false;2;0.4;1.0;Niederspannung +f2724954-34d3-4ddf-b6b0-7a1531639990;"{""type"":""Point"",""coordinates"":[11.8191,53.4235],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 6;;;;false;2;0.4;1.0;Niederspannung +5d3bcf55-0520-43ff-8d63-3d0eb421e442;"{""type"":""Point"",""coordinates"":[11.816,53.4234],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 91;;;;false;2;0.4;1.0;Niederspannung +3a557b4e-06b8-4f29-929f-81d95c42c897;"{""type"":""Point"",""coordinates"":[11.8213,53.4277],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 42;;;;false;2;0.4;1.0;Niederspannung +7dc43c81-9a61-45a0-9745-800a28bf4a9d;"{""type"":""Point"",""coordinates"":[11.8196,53.4278],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 50;;;;false;2;0.4;1.0;Niederspannung +fd4bebb8-40ca-4eed-92c0-cdd10b86ac20;"{""type"":""Point"",""coordinates"":[11.8191,53.4209],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 60;;;;false;2;0.4;1.0;Niederspannung +3464496c-7dd4-41e9-ae0a-99ade0b51572;"{""type"":""Point"",""coordinates"":[11.8213,53.4236],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 40;;;;false;2;0.4;1.0;Niederspannung +d38d936a-9c05-4bdc-8331-418fef27f492;"{""type"":""Point"",""coordinates"":[11.815,53.4281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 109;;;;false;2;0.4;1.0;Niederspannung +58b551b6-83bd-4f1c-8d9c-8c9a7f638c0b;"{""type"":""Point"",""coordinates"":[11.8213,53.425],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 38;;;;false;2;0.4;1.0;Niederspannung +b5c1e826-63fd-4b0c-bec0-0c758389ef58;"{""type"":""Point"",""coordinates"":[11.8191,53.4239],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 104;;;;false;2;0.4;1.0;Niederspannung +7c35a794-f569-4a9c-acb0-d03647610086;"{""type"":""Point"",""coordinates"":[11.8166,53.4249],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 98;;;;false;2;0.4;1.0;Niederspannung +78815cf6-70db-432c-96e6-87fe8cf67eee;"{""type"":""Point"",""coordinates"":[11.815,53.4272],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 112;;;;false;2;0.4;1.0;Niederspannung +c5b6bfaf-1621-40a7-9c53-02cfb59c04d9;"{""type"":""Point"",""coordinates"":[11.822,53.4294],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 14;;;;false;2;0.4;1.0;Niederspannung +2b3d7fb8-0583-4d47-97b1-3b5f232fd462;"{""type"":""Point"",""coordinates"":[11.8213,53.4274],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 87;;;;false;2;0.4;1.0;Niederspannung +3fcb94e3-7781-4d83-9030-d9853822e78e;"{""type"":""Point"",""coordinates"":[11.8213,53.4243],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 85;;;;false;2;0.4;1.0;Niederspannung +f713593a-3fd3-410a-ac08-74202d4f5798;"{""type"":""Point"",""coordinates"":[11.8201,53.4236],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 79;;;;false;2;0.4;1.0;Niederspannung +47246a84-ad0e-4d04-9d98-1c9cd5d363c1;"{""type"":""Point"",""coordinates"":[11.8196,53.4289],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 16;;;;false;2;0.4;1.0;Niederspannung +9cdb3115-cc00-4d61-bc33-442e8f30fb63;"{""type"":""Point"",""coordinates"":[11.8213,53.4257],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 83;;;;false;2;0.4;1.0;Niederspannung +839ff0f4-93db-42ec-a928-bbc448b6cf5c;"{""type"":""Point"",""coordinates"":[11.8197,53.4207],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 69;;;;false;2;0.4;1.0;Niederspannung +e7908208-77b4-4059-806e-4857262992fc;"{""type"":""Point"",""coordinates"":[11.8196,53.4247],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 105;;;;false;2;0.4;1.0;Niederspannung +83da8d60-405a-45f7-9bb9-9d35607b7927;"{""type"":""Point"",""coordinates"":[11.8201,53.4229],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 2;;;;false;2;0.4;1.0;Niederspannung +9d136a6b-5fdc-44ed-a5ed-599a55281024;"{""type"":""Point"",""coordinates"":[11.8195,53.4275],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 9;;;;false;2;0.4;1.0;Niederspannung +3faac527-0ff3-44a7-9e4f-24a41940da90;"{""type"":""Point"",""coordinates"":[11.816,53.4227],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 15;;;;false;2;0.4;1.0;Niederspannung +5682ac05-7336-4ebc-a5d1-3b69c79fb3b1;"{""type"":""Point"",""coordinates"":[11.8166,53.4241],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 44;;;;false;2;0.4;1.0;Niederspannung +5b73ded9-3ca4-4f18-a2ab-c27b9a3dcf9d;"{""type"":""Point"",""coordinates"":[11.8166,53.423],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 36;;;;false;2;0.4;1.0;Niederspannung +594d101c-3a05-45e3-a061-9189f3e848b7;"{""type"":""Point"",""coordinates"":[11.8198,53.4192],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 88;;;;false;2;0.4;1.0;Niederspannung +7b81b518-00e0-4ff1-b4cf-876903958d7a;"{""type"":""Point"",""coordinates"":[11.8213,53.4271],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 8;;;;false;2;0.4;1.0;Niederspannung +a882e666-82d1-4ba6-87df-fc702fe06187;"{""type"":""Point"",""coordinates"":[11.8169,53.4289],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 74;;;;false;2;0.4;1.0;Niederspannung +1403edf9-e47c-4705-8563-83bcd639482e;"{""type"":""Point"",""coordinates"":[11.8213,53.4286],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 75;;;;false;2;0.4;1.0;Niederspannung +03b2aa45-84f6-48c0-9dab-427e046a5672;"{""type"":""Point"",""coordinates"":[11.8196,53.4253],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 22;;;;false;2;0.4;1.0;Niederspannung +6ee7ea93-ea9c-40cb-b79a-1c5f287c97a5;"{""type"":""Point"",""coordinates"":[11.8221,53.4286],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 66;;;;false;2;0.4;1.0;Niederspannung +88cf719a-92df-4dfd-9a83-f84330e28fe0;"{""type"":""Point"",""coordinates"":[11.815,53.4285],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 108;;;;false;2;0.4;1.0;Niederspannung +80962bd3-a10f-4ed2-ba6a-3e802189939c;"{""type"":""Point"",""coordinates"":[11.8201,53.4239],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 21;;;;false;2;0.4;1.0;Niederspannung +80b8d1f8-7e83-421d-a95a-c193fc35f4f7;"{""type"":""Point"",""coordinates"":[11.8175,53.427],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 95;;;;false;2;0.4;1.0;Niederspannung +b909fb45-b6ee-427f-afd7-e8a0ec7274c6;"{""type"":""Point"",""coordinates"":[11.8213,53.4225],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 92;;;;false;2;0.4;1.0;Niederspannung +57b40047-4f9d-46bb-bf19-c0a86bbd4f5b;"{""type"":""Point"",""coordinates"":[11.8166,53.4234],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 37;;;;false;2;0.4;1.0;Niederspannung +18b4157b-0e47-4c5a-adb8-ccae47372336;"{""type"":""Point"",""coordinates"":[11.8213,53.4264],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 5;;;;false;2;0.4;1.0;Niederspannung +8726dc29-621e-4455-a541-cd88d7da457f;"{""type"":""Point"",""coordinates"":[11.8166,53.4257],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 96;;;;false;2;0.4;1.0;Niederspannung +814f784b-687f-4dd5-8a91-c7772c916d46;"{""type"":""Point"",""coordinates"":[11.8174,53.4262],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 19;;;;false;2;0.4;1.0;Niederspannung +7efabb8d-ba17-4487-96d9-5744b1fedf8a;"{""type"":""Point"",""coordinates"":[11.8166,53.4245],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 23;;;;false;2;0.4;1.0;Niederspannung +15a86f7d-fb73-49a4-af6a-25b14122378d;"{""type"":""Point"",""coordinates"":[11.8191,53.4227],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 11;;;;false;2;0.4;1.0;Niederspannung +34031e92-3444-47d5-94ae-cceeb5d96bb2;"{""type"":""Point"",""coordinates"":[11.8195,53.4268],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 106;;;;false;2;0.4;1.0;Niederspannung +35748e60-3be8-4930-8a61-209fd5df1bec;"{""type"":""Point"",""coordinates"":[11.815,53.4293],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 30;;;;false;2;0.4;1.0;Niederspannung +50cac08e-bf24-4526-9466-53ca5edccd15;"{""type"":""Point"",""coordinates"":[11.8166,53.4253],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 64;;;;false;2;0.4;1.0;Niederspannung +2efac9b1-fb0d-4e08-bfac-501798826deb;"{""type"":""Point"",""coordinates"":[11.8191,53.4231],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 93;;;;false;2;0.4;1.0;Niederspannung +c86d6361-4159-4787-b5f4-e41dcaa95195;"{""type"":""Point"",""coordinates"":[11.8182,53.4257],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 13;;;;false;2;0.4;1.0;Niederspannung diff --git a/input/samples/simopsimtestgrid_reduced/fullGrid/pv_input.csv b/input/samples/simopsimtestgrid_reduced/fullGrid/pv_input.csv new file mode 100755 index 0000000000..b883b51783 --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced/fullGrid/pv_input.csv @@ -0,0 +1,4 @@ +uuid;albedo;azimuth;cos_phi_rated;elevation_angle;eta_conv;id;k_g;k_t;market_reaction;node;operates_from;operates_until;operator;q_characteristics;s_rated;em +a1eb7fc1-3bee-4b65-a387-ef3046644bf0;0.20000000298023224;4.093344211578369;0.8999999761581421;37.69556427001953;97.0;LV5.201 PV 10;0.8999999761581421;1.0;false;dc54bd8a-b7d8-4e99-adb0-d6ee5084241c;;;;"cosPhiFixed:{(0.0,0.9)}";4.2;f9dc7ce6-658c-4101-a12f-d58bb889286b +de8cfef5-7620-4b9e-9a10-1faebb5a80c0;0.20000000298023224;4.093344211578369;0.8999999761581421;37.69556427001953;97.0;LV5.201 PV 11;0.8999999761581421;1.0;false;3e6be3ac-2b51-4080-b815-391313612fc7;;;;"cosPhiFixed:{(0.0,0.9)}";10.9;957938b7-0476-4fab-a1b3-6ce8615857b3 +2560c371-f420-4c2a-b4e6-e04c11b64c03;0.20000000298023224;0.7802008390426636;0.8999999761581421;40.086585998535156;98.0;LV5.201 PV 15;0.8999999761581421;1.0;false;43040a39-8b6c-401f-9dfd-82b42aa6dec6;;;;"cosPhiFixed:{(0.0,0.9)}";2.9;c3a7e9f5-b492-4c85-af2d-1e93f6a25443 diff --git a/input/samples/simopsimtestgrid_reduced/fullGrid/transformer_2_w_input.csv b/input/samples/simopsimtestgrid_reduced/fullGrid/transformer_2_w_input.csv new file mode 100755 index 0000000000..d1f9f5d13c --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced/fullGrid/transformer_2_w_input.csv @@ -0,0 +1,2 @@ +uuid;auto_tap;id;node_a;node_b;operates_from;operates_until;operator;parallel_devices;tap_pos;type +adaba416-9b52-45df-9d91-f67d0dd28ecb;false;MV2.101-LV5.201-Trafo 1;ec8f2c82-a1b2-487c-b573-250859e3b414;4749ab2b-4d96-4100-8081-73e77c797d6b;;;;1;0;417407d2-1e74-4f37-9b64-f701f53f8842 diff --git a/input/samples/simopsimtestgrid_reduced/fullGrid/transformer_2_w_type_input.csv b/input/samples/simopsimtestgrid_reduced/fullGrid/transformer_2_w_type_input.csv new file mode 100755 index 0000000000..b13848b159 --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced/fullGrid/transformer_2_w_type_input.csv @@ -0,0 +1,2 @@ +uuid;b_m;d_phi;d_v;g_m;id;r_sc;s_rated;tap_max;tap_min;tap_neutr;tap_side;v_rated_a;v_rated_b;x_sc +417407d2-1e74-4f37-9b64-f701f53f8842;-36.47380569074435;0.0;2.5;4124.999999999999;0.63 MVA 20/0.4 kV Dyn5 ASEA;6.953892668178382;630.0;2;-2;0;false;20.0;0.4;37.45518044666632 diff --git a/input/samples/simopsimtestgrid_reduced/simopsimtestgrid.conf b/input/samples/simopsimtestgrid_reduced/simopsimtestgrid.conf new file mode 100755 index 0000000000..d685a161e6 --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced/simopsimtestgrid.conf @@ -0,0 +1,240 @@ +include "../common/pekko.conf" + +######### +# ATTENTION: Do not change this config file directly but use it as a base for your personal delta config for the +# vn_simona scenario! Delta configs can be created by including the config you want to change +# parameters from via include (e.g. include "input/samples/vn_simona/vn_simona.conf") at the +# beginning of your config file and then just override the parameters you want to change! +######### + +################################################################## +# Simulation Parameters +################################################################## +simona.simulationName = "simopsimtest" + +################################################################## +# Time Parameters +################################################################## +simona.time.startDateTime = "2024-02-27T00:00:00Z" +simona.time.endDateTime = "2024-02-27T23:45:00Z" +simona.time.schedulerReadyCheckWindow = 900 + +################################################################## +# Input Parameters +################################################################## +simona.input.primary.csvParams = { + directoryPath: "simona/input/samples/simopsimtestgrid_reduced/fullGrid" + csvSep: ";" + isHierarchic: false +} +simona.input.grid.datasource.id = "csv" +simona.input.grid.datasource.csvParams = { + directoryPath: "simona/input/samples/simopsimtestgrid_reduced/fullGrid" + csvSep: ";" + isHierarchic: false +} + +simona.input.weather.datasource = { + scheme = "icon" + sampleParams.use = true + coordinateSource.sampleParams.use = true + maxCoordinateDistance = 50000 +} + +################################################################## +# Output Parameters +################################################################## +simona.output.base.dir = "simona/output/simopsimtestgrid_reduced" +simona.output.base.addTimestampToOutputDir = true + +simona.output.sink.csv { + fileFormat = ".csv" + filePrefix = "" + fileSuffix = "" +} + +simona.output.grid = { + notifier = "grid" + nodes = false + lines = false + switches = false + transformers2w = false + transformers3w = false +} +simona.output.participant.defaultConfig = { + notifier = "default" + powerRequestReply = false + simulationResult = true +} +simona.output.participant.individualConfigs = [ + { + notifier = "pv" + powerRequestReply = false + simulationResult = true + }, + { + notifier = "wec" + powerRequestReply = false + simulationResult = false + }, + { + notifier = "evcs" + powerRequestReply = false + simulationResult = false + }, + { + notifier = "bm" + powerRequestReply = false + simulationResult = false + }, + { + notifier = "chp" + powerRequestReply = false + simulationResult = false + }, + { + notifier = "ev" + powerRequestReply = false + simulationResult = false + }, + { + notifier = "hp" + powerRequestReply = false + simulationResult = false + }, + { + notifier = "storage" + powerRequestReply = false + simulationResult = false + }, + { + notifier = "fixedFeedIn" + powerRequestReply = false + simulationResult = false + } +] +simona.output.thermal = { + defaultConfig = { + notifier = "default", + simulationResult = false + } + individualConfigs = [ + { + notifier = "house", + simulationResult = false + } + ] +} + +################################################################## +# Runtime Configuration // todo refactor as this naming is misleading and partly unneeded +################################################################## +simona.runtime.selected_subgrids = [] +simona.runtime.selected_volt_lvls = [] + +simona.runtime.participant.load = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + modelBehaviour = "fix" + reference = "power" + } + individualConfigs = [] +} + +simona.runtime.participant.fixedFeedIn = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +simona.runtime.participant.pv = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +simona.runtime.participant.wec = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +simona.runtime.participant.evcs = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +simona.runtime.participant.hp = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +# # # # # +# ATTENTION: calculateMissingReactivePowerWithModel and scaling is ignored here. +# # # # # +simona.runtime.participant.em = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +################################################################## +# Event Configuration +################################################################## +simona.event.listener = [] + +################################################################## +# Grid Configuration +################################################################## + +simona.gridConfig.refSystems = [ + {sNom = "100 kVA", vNom = "0.4 kV", voltLvls = [{id = "NS", vNom = "0.4 kV"}]}, + {sNom = "60 MVA", vNom = "20 kV", voltLvls = [{id = "MS", vNom = "20 kV"}]}, + {sNom = "600 MVA", vNom = "110 kV", voltLvls = [{id = "HS", vNom = "110 kV"}]}, + {sNom = "1000 MVA", vNom = "380 kV", voltLvls = [{id = "HoeS", vNom = "380 kV"}]} +] + +################################################################## +# Power Flow Configuration +################################################################## +simona.powerflow.maxSweepPowerDeviation = 1E-5 // the maximum allowed deviation in power between two sweeps, before overall convergence is assumed +simona.powerflow.newtonraphson.epsilon = [1E-12] +simona.powerflow.newtonraphson.iterations = 50 +simona.powerflow.resolution = "86400s" +simona.powerflow.stopOnFailure = true + +simona.control.transformer = [ + { + transformers = ["31a2b9bf-e785-4475-aa44-1c34646e8c79"], + measurements = ["923f2d69-3093-4198-86e4-13d2d1c220f8"], + vMin = 0.98, + vMax = 1.02 + }, { + transformers = ["1132dbf4-e8a1-44ae-8415-f42d4497aa1d"], + measurements = ["7686b818-a0ba-465c-8e4e-f7d3c4e171fc"], + vMin = 0.98, + vMax = 1.02 + } +] diff --git a/input/samples/vn_simona_small_with_em/fullGrid/em_input.csv b/input/samples/vn_simona_small_with_em/fullGrid/em_input.csv new file mode 100755 index 0000000000..22729f2b61 --- /dev/null +++ b/input/samples/vn_simona_small_with_em/fullGrid/em_input.csv @@ -0,0 +1,3 @@ +uuid,control_strategy,parent_em,id,operates_from,operates_until,operator +fd1a8de9-722a-4304-8799-e1e976d9979c,self_optimization,,EM_NS_Node_3,,, +ff0b995a-86ff-4f4d-987e-e475a64f2180,self_optimization,,EM_NS_Node_4,,, \ No newline at end of file diff --git a/input/samples/vn_simona_small_with_em/fullGrid/fixed_feed_in_input.csv b/input/samples/vn_simona_small_with_em/fullGrid/fixed_feed_in_input.csv new file mode 100755 index 0000000000..90a18373ca --- /dev/null +++ b/input/samples/vn_simona_small_with_em/fullGrid/fixed_feed_in_input.csv @@ -0,0 +1,2 @@ +uuid,cos_phi_rated,id,node,operates_from,operates_until,operator,q_characteristics,s_rated +9abe950d-362e-4efe-b686-500f84d8f368,0.9,feed_in_hs_s4,401f37f8-6f2c-4564-bc78-6736cb9cbf8d,,,,"cosPhiFixed:{(0.0,0.95)}",200.0 diff --git a/input/samples/vn_simona_small_with_em/fullGrid/line_input.csv b/input/samples/vn_simona_small_with_em/fullGrid/line_input.csv new file mode 100755 index 0000000000..0f7b0204e7 --- /dev/null +++ b/input/samples/vn_simona_small_with_em/fullGrid/line_input.csv @@ -0,0 +1,6 @@ +uuid,geo_position,id,length,node_a,node_b,olm_characteristic,operates_from,operates_until,operator,parallel_devices,type +fd04cc09-6ff8-48ea-a5eb-453e9d59d6d7,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_Node_1-NS_Node_3,0.04,00d03670-7833-47ee-ad52-04d18d1c64fd,33f29587-f63e-45b7-960b-037bda37a3cb,"olm:{(0.0,1.0)}",,,,1,9a8e9b63-af21-4c1b-8db7-fc2924f9610e +b5a66c60-7189-4c86-a32a-4d1aa6568475,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_Node_4-NS_Node_1,0.030304,401f37f8-6f2c-4564-bc78-6736cb9cbf8d,00d03670-7833-47ee-ad52-04d18d1c64fd,"olm:{(0.0,1.0)}",,,,1,9a8e9b63-af21-4c1b-8db7-fc2924f9610e +42828cac-b67e-4d5c-b4fa-787c57f16fde,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_Node_3-NS_Node_2,0.0251089,33f29587-f63e-45b7-960b-037bda37a3cb,dfae9806-9b44-4995-ba27-d66d8e4a43e0,"olm:{(0.0,1.0)}",,,,1,9a8e9b63-af21-4c1b-8db7-fc2924f9610e +997840bf-8c94-444f-83f1-e9c991706d7c,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_Node_1-NS_Node_2,0.0377768,00d03670-7833-47ee-ad52-04d18d1c64fd,dfae9806-9b44-4995-ba27-d66d8e4a43e0,"olm:{(0.0,1.0)}",,,,1,9a8e9b63-af21-4c1b-8db7-fc2924f9610e +81f4c585-6170-4a9e-981f-2185a0d7f2ec,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_Node_3-NS_Node_4,0.031,33f29587-f63e-45b7-960b-037bda37a3cb,401f37f8-6f2c-4564-bc78-6736cb9cbf8d,"olm:{(0.0,1.0)}",,,,1,9a8e9b63-af21-4c1b-8db7-fc2924f9610e diff --git a/input/samples/vn_simona_small_with_em/fullGrid/line_type_input.csv b/input/samples/vn_simona_small_with_em/fullGrid/line_type_input.csv new file mode 100755 index 0000000000..91202b2778 --- /dev/null +++ b/input/samples/vn_simona_small_with_em/fullGrid/line_type_input.csv @@ -0,0 +1,2 @@ +uuid,b,g,i_max,id,r,v_rated,x +9a8e9b63-af21-4c1b-8db7-fc2924f9610e,273.31899999999996,0.0,357.0,NAYY 4x240SE 0.6/1kV,0.1267,0.4,0.0797965 diff --git a/input/samples/vn_simona_small_with_em/fullGrid/load_input.csv b/input/samples/vn_simona_small_with_em/fullGrid/load_input.csv new file mode 100755 index 0000000000..2324353168 --- /dev/null +++ b/input/samples/vn_simona_small_with_em/fullGrid/load_input.csv @@ -0,0 +1,5 @@ +uuid,cos_phi_rated,dsm,e_cons_annual,id,load_profile,node,operates_from,operates_until,operator,q_characteristics,s_rated,em +4dca3b1d-5d24-444a-b4df-f4fa23b9ef1b,0.949999988079071,false,4000.0,LOAD_NS_Node_1,h0,00d03670-7833-47ee-ad52-04d18d1c64fd,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605, +9c5991bc-24df-496b-b4ce-5ec27657454c,0.949999988079071,false,4000.0,LOAD_NS_Node_2,h0,dfae9806-9b44-4995-ba27-d66d8e4a43e0,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605, +58b9f934-f7c4-4335-9894-3c80d9e6b852,0.949999988079071,false,4000.0,LOAD_NS_Node_3,h0,33f29587-f63e-45b7-960b-037bda37a3cb,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605,fd1a8de9-722a-4304-8799-e1e976d9979c +283a1252-a774-4b04-bfcf-fe8879065982,0.949999988079071,false,4000.0,LOAD_NS_Node_4,h0,401f37f8-6f2c-4564-bc78-6736cb9cbf8d,,,,"cosPhiFixed:{(0.0,1.0)}",2.3157899379730225,ff0b995a-86ff-4f4d-987e-e475a64f2180 diff --git a/input/samples/vn_simona_small_with_em/fullGrid/node_input.csv b/input/samples/vn_simona_small_with_em/fullGrid/node_input.csv new file mode 100755 index 0000000000..21772153f5 --- /dev/null +++ b/input/samples/vn_simona_small_with_em/fullGrid/node_input.csv @@ -0,0 +1,6 @@ +uuid,geo_position,id,operates_from,operates_until,operator,slack,subnet,v_rated,v_target,volt_lvl +00d03670-7833-47ee-ad52-04d18d1c64fd,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",NS_Node_1,,,,false,2,0.4,1.0,Niederspannung +dfae9806-9b44-4995-ba27-d66d8e4a43e0,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}",NS_Node_2,,,,false,2,0.4,1.0,Niederspannung +34cd8ee0-e607-4c47-89a7-121c3e32768a,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",MS_Node_0,,,,true,1,20.0,1.025,Mittelspannung +401f37f8-6f2c-4564-bc78-6736cb9cbf8d,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}",NS_Node_4,,,,false,2,0.4,1.0,Niederspannung +33f29587-f63e-45b7-960b-037bda37a3cb,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",NS_Node_3,,,,false,2,0.4,1.0,Niederspannung diff --git a/input/samples/vn_simona_small_with_em/fullGrid/pv_input.csv b/input/samples/vn_simona_small_with_em/fullGrid/pv_input.csv new file mode 100644 index 0000000000..e418358125 --- /dev/null +++ b/input/samples/vn_simona_small_with_em/fullGrid/pv_input.csv @@ -0,0 +1,3 @@ +uuid,albedo,azimuth,cos_phi_rated,elevation_angle,eta_conv,id,k_g,k_t,market_reaction,node,operates_from,operates_until,operator,q_characteristics,s_rated,em +a1eb7fc1-3bee-4b65-a387-ef3046644bf0,0.20000000298023224,-8.999500274658203,0.8999999761581421,37.14517593383789,98.0,PV_NS_Node_4,0.8999999761581421,1.0,false,401f37f8-6f2c-4564-bc78-6736cb9cbf8d,,,,"cosPhiFixed:{(0.0,0.9)}",10.0,ff0b995a-86ff-4f4d-987e-e475a64f2180 +9d7cd8e2-d859-4f4f-9c01-abba06ef2e2c,0.20000000298023224,-14.803051948547363,0.8999999761581421,42.391395568847656,96.0,PV_NS_Node_3,0.8999999761581421,1.0,false,33f29587-f63e-45b7-960b-037bda37a3cb,,,,"cosPhiFixed:{(0.0,0.9)}",10.0,fd1a8de9-722a-4304-8799-e1e976d9979c diff --git a/input/samples/vn_simona_small_with_em/fullGrid/storage_input.csv b/input/samples/vn_simona_small_with_em/fullGrid/storage_input.csv new file mode 100644 index 0000000000..35eb72ffac --- /dev/null +++ b/input/samples/vn_simona_small_with_em/fullGrid/storage_input.csv @@ -0,0 +1,3 @@ +uuid,id,node,operates_from,operates_until,operator,q_characteristics,type,em +a2a92cfd-3492-465f-9587-e789f4620af8,Speicher_3,33f29587-f63e-45b7-960b-037bda37a3cb,,,,"cosPhiFixed:{(0.0,0.98)}",95d4c980-d9e1-4813-9f2a-b0942488a570,fd1a8de9-722a-4304-8799-e1e976d9979c +c96f0a90-980a-4813-b175-ecc04aded773,Speicher_4,401f37f8-6f2c-4564-bc78-6736cb9cbf8d,,,,"cosPhiFixed:{(0.0,0.98)}",95d4c980-d9e1-4813-9f2a-b0942488a570,ff0b995a-86ff-4f4d-987e-e475a64f2180 diff --git a/input/samples/vn_simona_small_with_em/fullGrid/storage_type_input.csv b/input/samples/vn_simona_small_with_em/fullGrid/storage_type_input.csv new file mode 100644 index 0000000000..31cce697f1 --- /dev/null +++ b/input/samples/vn_simona_small_with_em/fullGrid/storage_type_input.csv @@ -0,0 +1,2 @@ +uuid,active_power_gradient,capex,cos_phi_rated,dod,e_storage,eta,id,life_cycle,life_time,opex,p_max,s_rated +95d4c980-d9e1-4813-9f2a-b0942488a570,1.0,0.0,0.96,8.0,16.0,93.0,Typ_1,5000,5000.0,0.65,4.0,4.166666666666667 diff --git a/input/samples/vn_simona_small_with_em/fullGrid/transformer_2_w_input.csv b/input/samples/vn_simona_small_with_em/fullGrid/transformer_2_w_input.csv new file mode 100755 index 0000000000..9bb70380b6 --- /dev/null +++ b/input/samples/vn_simona_small_with_em/fullGrid/transformer_2_w_input.csv @@ -0,0 +1,2 @@ +uuid,auto_tap,id,node_a,node_b,operates_from,operates_until,operator,parallel_devices,tap_pos,type +d2a3736d-d10b-4f38-88cf-871df43e3665,true,HöS-Trafo_S1,34cd8ee0-e607-4c47-89a7-121c3e32768a,00d03670-7833-47ee-ad52-04d18d1c64fd,,,,1,0,97735722-05cc-4ca8-8a8d-c08ac3ded19a diff --git a/input/samples/vn_simona_small_with_em/fullGrid/transformer_2_w_type_input.csv b/input/samples/vn_simona_small_with_em/fullGrid/transformer_2_w_type_input.csv new file mode 100755 index 0000000000..0d2400e17e --- /dev/null +++ b/input/samples/vn_simona_small_with_em/fullGrid/transformer_2_w_type_input.csv @@ -0,0 +1,2 @@ +uuid,b_m,d_phi,d_v,g_m,id,r_sc,s_rated,tap_max,tap_min,tap_neutr,tap_side,v_rated_a,v_rated_b,x_sc +97735722-05cc-4ca8-8a8d-c08ac3ded19a,-36.47380569074435,0.0,2.5,4124.999999999999,0.63 MVA 20/0.4 kV Dyn5 ASEA,6.953892668178382,630.0,2,-2,0,false,20.0,0.4,37.45518044666632 diff --git a/input/samples/vn_simona_small_with_em/vn_simona_small_withem.conf b/input/samples/vn_simona_small_with_em/vn_simona_small_withem.conf new file mode 100755 index 0000000000..6185ad6e4d --- /dev/null +++ b/input/samples/vn_simona_small_with_em/vn_simona_small_withem.conf @@ -0,0 +1,210 @@ +include "../common/pekko.conf" + +######### +# ATTENTION: Do not change this config file directly but use it as a base for your personal delta config for the +# vn_simona scenario! Delta configs can be created by including the config you want to change +# parameters from via include (e.g. include "input/samples/vn_simona/vn_simona.conf") at the +# beginning of your config file and then just override the parameters you want to change! +######### + +################################################################## +# Simulation Parameters +################################################################## +simona.simulationName = "simple_vn_simona_withem" + +################################################################## +# Time Parameters +################################################################## +simona.time.startDateTime = "2023-06-01T00:00:00Z" +simona.time.endDateTime = "2023-06-01T23:45:00Z" +simona.time.schedulerReadyCheckWindow = 900 + +################################################################## +# Input Parameters +################################################################## +simona.input.primary.csvParams = { + directoryPath: "simona/input/samples/vn_simona_small_with_em/fullGrid" + csvSep: "," + isHierarchic: false +} +simona.input.grid.datasource.id = "csv" +simona.input.grid.datasource.csvParams = { + directoryPath: "simona/input/samples/vn_simona_small_with_em/fullGrid" + csvSep: "," + isHierarchic: false +} + +simona.input.weather.datasource = { + scheme = "icon" + sampleParams.use = true + coordinateSource.sampleParams.use = true + maxCoordinateDistance = 50000 +} + +################################################################## +# Output Parameters +################################################################## +simona.output.base.dir = "simona/output/vn_simona_small_with_em" +simona.output.base.addTimestampToOutputDir = true + +simona.output.sink.csv { + fileFormat = ".csv" + filePrefix = "" + fileSuffix = "" +} + +simona.output.grid = { + notifier = "grid" + nodes = false + lines = false + switches = false + transformers2w = false + transformers3w = false +} +simona.output.participant.defaultConfig = { + notifier = "default" + powerRequestReply = false + simulationResult = true +} +simona.output.participant.individualConfigs = [ + { + notifier = "pv" + powerRequestReply = false + simulationResult = true + }, + { + notifier = "wec" + powerRequestReply = false + simulationResult = true + }, + { + notifier = "evcs" + powerRequestReply = false + simulationResult = true + } +] +simona.output.thermal = { + defaultConfig = { + notifier = "default", + simulationResult = false + } + individualConfigs = [ + { + notifier = "house", + simulationResult = true + } + ] +} + +################################################################## +# Runtime Configuration // todo refactor as this naming is misleading and partly unneeded +################################################################## +simona.runtime.selected_subgrids = [] +simona.runtime.selected_volt_lvls = [] + +simona.runtime.participant.load = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + modelBehaviour = "profile" + reference = "power" + } + individualConfigs = [] +} + +simona.runtime.participant.fixedFeedIn = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +simona.runtime.participant.pv = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +simona.runtime.participant.wec = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +simona.runtime.participant.evcs = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +simona.runtime.participant.hp = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +# # # # # +# ATTENTION: calculateMissingReactivePowerWithModel and scaling is ignored here. +# # # # # +simona.runtime.participant.em = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +################################################################## +# Event Configuration +################################################################## +simona.event.listener = [] + +################################################################## +# Grid Configuration +################################################################## + +simona.gridConfig.refSystems = [ + {sNom = "100 kVA", vNom = "0.4 kV", voltLvls = [{id = "LV", vNom = "0.4 kV"}]}, + {sNom = "60 MVA", vNom = "20 kV", voltLvls = [{id = "MV", vNom = "20 kV"}]}, + {sNom = "600 MVA", vNom = "110 kV", voltLvls = [{id = "HV", vNom = "110 kV"}]}, + {sNom = "1000 MVA", vNom = "380 kV", voltLvls = [{id = "EHV", vNom = "380 kV"}]} +] + +################################################################## +# Power Flow Configuration +################################################################## +simona.powerflow.maxSweepPowerDeviation = 1E-5 // the maximum allowed deviation in power between two sweeps, before overall convergence is assumed +simona.powerflow.newtonraphson.epsilon = [1E-12] +simona.powerflow.newtonraphson.iterations = 50 +simona.powerflow.resolution = "86400s" +simona.powerflow.stopOnFailure = true + +simona.control.transformer = [ + { + transformers = ["31a2b9bf-e785-4475-aa44-1c34646e8c79"], + measurements = ["923f2d69-3093-4198-86e4-13d2d1c220f8"], + vMin = 0.98, + vMax = 1.02 + }, { + transformers = ["1132dbf4-e8a1-44ae-8415-f42d4497aa1d"], + measurements = ["7686b818-a0ba-465c-8e4e-f7d3c4e171fc"], + vMin = 0.98, + vMax = 1.02 + } +] diff --git a/input/samples/vn_simona_small_with_em_original/fullGrid/em_input.csv b/input/samples/vn_simona_small_with_em_original/fullGrid/em_input.csv new file mode 100755 index 0000000000..8f4ac1f81a --- /dev/null +++ b/input/samples/vn_simona_small_with_em_original/fullGrid/em_input.csv @@ -0,0 +1,5 @@ +uuid,control_strategy,parent_em,id,operates_from,operates_until,operator +fd1a8de9-722a-4304-8799-e1e976d9979c,self_optimization,c40d5386-d2ab-49f8-a1b4-02991b68f502,EM_NS_Node_3,,, +ff0b995a-86ff-4f4d-987e-e475a64f2180,self_optimization,5f4c7c32-0e14-4f88-b727-467f270624e2,EM_NS_Node_4,,, +c40d5386-d2ab-49f8-a1b4-02991b68f502,self_optimization,,EM_Ext_Interface_3,,, +5f4c7c32-0e14-4f88-b727-467f270624e2,self_optimization,,EM_Ext_Interface_4,,, \ No newline at end of file diff --git a/input/samples/vn_simona_small_with_em_original/fullGrid/fixed_feed_in_input.csv b/input/samples/vn_simona_small_with_em_original/fullGrid/fixed_feed_in_input.csv new file mode 100755 index 0000000000..90a18373ca --- /dev/null +++ b/input/samples/vn_simona_small_with_em_original/fullGrid/fixed_feed_in_input.csv @@ -0,0 +1,2 @@ +uuid,cos_phi_rated,id,node,operates_from,operates_until,operator,q_characteristics,s_rated +9abe950d-362e-4efe-b686-500f84d8f368,0.9,feed_in_hs_s4,401f37f8-6f2c-4564-bc78-6736cb9cbf8d,,,,"cosPhiFixed:{(0.0,0.95)}",200.0 diff --git a/input/samples/vn_simona_small_with_em_original/fullGrid/line_input.csv b/input/samples/vn_simona_small_with_em_original/fullGrid/line_input.csv new file mode 100755 index 0000000000..0f7b0204e7 --- /dev/null +++ b/input/samples/vn_simona_small_with_em_original/fullGrid/line_input.csv @@ -0,0 +1,6 @@ +uuid,geo_position,id,length,node_a,node_b,olm_characteristic,operates_from,operates_until,operator,parallel_devices,type +fd04cc09-6ff8-48ea-a5eb-453e9d59d6d7,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_Node_1-NS_Node_3,0.04,00d03670-7833-47ee-ad52-04d18d1c64fd,33f29587-f63e-45b7-960b-037bda37a3cb,"olm:{(0.0,1.0)}",,,,1,9a8e9b63-af21-4c1b-8db7-fc2924f9610e +b5a66c60-7189-4c86-a32a-4d1aa6568475,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_Node_4-NS_Node_1,0.030304,401f37f8-6f2c-4564-bc78-6736cb9cbf8d,00d03670-7833-47ee-ad52-04d18d1c64fd,"olm:{(0.0,1.0)}",,,,1,9a8e9b63-af21-4c1b-8db7-fc2924f9610e +42828cac-b67e-4d5c-b4fa-787c57f16fde,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_Node_3-NS_Node_2,0.0251089,33f29587-f63e-45b7-960b-037bda37a3cb,dfae9806-9b44-4995-ba27-d66d8e4a43e0,"olm:{(0.0,1.0)}",,,,1,9a8e9b63-af21-4c1b-8db7-fc2924f9610e +997840bf-8c94-444f-83f1-e9c991706d7c,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_Node_1-NS_Node_2,0.0377768,00d03670-7833-47ee-ad52-04d18d1c64fd,dfae9806-9b44-4995-ba27-d66d8e4a43e0,"olm:{(0.0,1.0)}",,,,1,9a8e9b63-af21-4c1b-8db7-fc2924f9610e +81f4c585-6170-4a9e-981f-2185a0d7f2ec,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_Node_3-NS_Node_4,0.031,33f29587-f63e-45b7-960b-037bda37a3cb,401f37f8-6f2c-4564-bc78-6736cb9cbf8d,"olm:{(0.0,1.0)}",,,,1,9a8e9b63-af21-4c1b-8db7-fc2924f9610e diff --git a/input/samples/vn_simona_small_with_em_original/fullGrid/line_type_input.csv b/input/samples/vn_simona_small_with_em_original/fullGrid/line_type_input.csv new file mode 100755 index 0000000000..91202b2778 --- /dev/null +++ b/input/samples/vn_simona_small_with_em_original/fullGrid/line_type_input.csv @@ -0,0 +1,2 @@ +uuid,b,g,i_max,id,r,v_rated,x +9a8e9b63-af21-4c1b-8db7-fc2924f9610e,273.31899999999996,0.0,357.0,NAYY 4x240SE 0.6/1kV,0.1267,0.4,0.0797965 diff --git a/input/samples/vn_simona_small_with_em_original/fullGrid/load_input.csv b/input/samples/vn_simona_small_with_em_original/fullGrid/load_input.csv new file mode 100755 index 0000000000..4d0adff998 --- /dev/null +++ b/input/samples/vn_simona_small_with_em_original/fullGrid/load_input.csv @@ -0,0 +1,5 @@ +uuid,cos_phi_rated,dsm,e_cons_annual,id,load_profile,node,operates_from,operates_until,operator,q_characteristics,s_rated +4dca3b1d-5d24-444a-b4df-f4fa23b9ef1b,0.949999988079071,false,4000.0,LOAD_NS_Node_1,h0,00d03670-7833-47ee-ad52-04d18d1c64fd,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605 +9c5991bc-24df-496b-b4ce-5ec27657454c,0.949999988079071,false,4000.0,LOAD_NS_Node_2,h0,dfae9806-9b44-4995-ba27-d66d8e4a43e0,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605 +58b9f934-f7c4-4335-9894-3c80d9e6b852,0.949999988079071,false,4000.0,LOAD_NS_Node_3,h0,33f29587-f63e-45b7-960b-037bda37a3cb,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605 +283a1252-a774-4b04-bfcf-fe8879065982,0.949999988079071,false,4000.0,LOAD_NS_Node_4,h0,401f37f8-6f2c-4564-bc78-6736cb9cbf8d,,,,"cosPhiFixed:{(0.0,1.0)}",2.3157899379730225 diff --git a/input/samples/vn_simona_small_with_em_original/fullGrid/node_input.csv b/input/samples/vn_simona_small_with_em_original/fullGrid/node_input.csv new file mode 100755 index 0000000000..21772153f5 --- /dev/null +++ b/input/samples/vn_simona_small_with_em_original/fullGrid/node_input.csv @@ -0,0 +1,6 @@ +uuid,geo_position,id,operates_from,operates_until,operator,slack,subnet,v_rated,v_target,volt_lvl +00d03670-7833-47ee-ad52-04d18d1c64fd,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",NS_Node_1,,,,false,2,0.4,1.0,Niederspannung +dfae9806-9b44-4995-ba27-d66d8e4a43e0,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}",NS_Node_2,,,,false,2,0.4,1.0,Niederspannung +34cd8ee0-e607-4c47-89a7-121c3e32768a,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",MS_Node_0,,,,true,1,20.0,1.025,Mittelspannung +401f37f8-6f2c-4564-bc78-6736cb9cbf8d,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}",NS_Node_4,,,,false,2,0.4,1.0,Niederspannung +33f29587-f63e-45b7-960b-037bda37a3cb,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",NS_Node_3,,,,false,2,0.4,1.0,Niederspannung diff --git a/input/samples/vn_simona_small_with_em_original/fullGrid/pv_input.csv b/input/samples/vn_simona_small_with_em_original/fullGrid/pv_input.csv new file mode 100644 index 0000000000..e418358125 --- /dev/null +++ b/input/samples/vn_simona_small_with_em_original/fullGrid/pv_input.csv @@ -0,0 +1,3 @@ +uuid,albedo,azimuth,cos_phi_rated,elevation_angle,eta_conv,id,k_g,k_t,market_reaction,node,operates_from,operates_until,operator,q_characteristics,s_rated,em +a1eb7fc1-3bee-4b65-a387-ef3046644bf0,0.20000000298023224,-8.999500274658203,0.8999999761581421,37.14517593383789,98.0,PV_NS_Node_4,0.8999999761581421,1.0,false,401f37f8-6f2c-4564-bc78-6736cb9cbf8d,,,,"cosPhiFixed:{(0.0,0.9)}",10.0,ff0b995a-86ff-4f4d-987e-e475a64f2180 +9d7cd8e2-d859-4f4f-9c01-abba06ef2e2c,0.20000000298023224,-14.803051948547363,0.8999999761581421,42.391395568847656,96.0,PV_NS_Node_3,0.8999999761581421,1.0,false,33f29587-f63e-45b7-960b-037bda37a3cb,,,,"cosPhiFixed:{(0.0,0.9)}",10.0,fd1a8de9-722a-4304-8799-e1e976d9979c diff --git a/input/samples/vn_simona_small_with_em_original/fullGrid/storage_input.csv b/input/samples/vn_simona_small_with_em_original/fullGrid/storage_input.csv new file mode 100644 index 0000000000..35eb72ffac --- /dev/null +++ b/input/samples/vn_simona_small_with_em_original/fullGrid/storage_input.csv @@ -0,0 +1,3 @@ +uuid,id,node,operates_from,operates_until,operator,q_characteristics,type,em +a2a92cfd-3492-465f-9587-e789f4620af8,Speicher_3,33f29587-f63e-45b7-960b-037bda37a3cb,,,,"cosPhiFixed:{(0.0,0.98)}",95d4c980-d9e1-4813-9f2a-b0942488a570,fd1a8de9-722a-4304-8799-e1e976d9979c +c96f0a90-980a-4813-b175-ecc04aded773,Speicher_4,401f37f8-6f2c-4564-bc78-6736cb9cbf8d,,,,"cosPhiFixed:{(0.0,0.98)}",95d4c980-d9e1-4813-9f2a-b0942488a570,ff0b995a-86ff-4f4d-987e-e475a64f2180 diff --git a/input/samples/vn_simona_small_with_em_original/fullGrid/storage_type_input.csv b/input/samples/vn_simona_small_with_em_original/fullGrid/storage_type_input.csv new file mode 100644 index 0000000000..31cce697f1 --- /dev/null +++ b/input/samples/vn_simona_small_with_em_original/fullGrid/storage_type_input.csv @@ -0,0 +1,2 @@ +uuid,active_power_gradient,capex,cos_phi_rated,dod,e_storage,eta,id,life_cycle,life_time,opex,p_max,s_rated +95d4c980-d9e1-4813-9f2a-b0942488a570,1.0,0.0,0.96,8.0,16.0,93.0,Typ_1,5000,5000.0,0.65,4.0,4.166666666666667 diff --git a/input/samples/vn_simona_small_with_em_original/fullGrid/transformer_2_w_input.csv b/input/samples/vn_simona_small_with_em_original/fullGrid/transformer_2_w_input.csv new file mode 100755 index 0000000000..9bb70380b6 --- /dev/null +++ b/input/samples/vn_simona_small_with_em_original/fullGrid/transformer_2_w_input.csv @@ -0,0 +1,2 @@ +uuid,auto_tap,id,node_a,node_b,operates_from,operates_until,operator,parallel_devices,tap_pos,type +d2a3736d-d10b-4f38-88cf-871df43e3665,true,HöS-Trafo_S1,34cd8ee0-e607-4c47-89a7-121c3e32768a,00d03670-7833-47ee-ad52-04d18d1c64fd,,,,1,0,97735722-05cc-4ca8-8a8d-c08ac3ded19a diff --git a/input/samples/vn_simona_small_with_em_original/fullGrid/transformer_2_w_type_input.csv b/input/samples/vn_simona_small_with_em_original/fullGrid/transformer_2_w_type_input.csv new file mode 100755 index 0000000000..0d2400e17e --- /dev/null +++ b/input/samples/vn_simona_small_with_em_original/fullGrid/transformer_2_w_type_input.csv @@ -0,0 +1,2 @@ +uuid,b_m,d_phi,d_v,g_m,id,r_sc,s_rated,tap_max,tap_min,tap_neutr,tap_side,v_rated_a,v_rated_b,x_sc +97735722-05cc-4ca8-8a8d-c08ac3ded19a,-36.47380569074435,0.0,2.5,4124.999999999999,0.63 MVA 20/0.4 kV Dyn5 ASEA,6.953892668178382,630.0,2,-2,0,false,20.0,0.4,37.45518044666632 diff --git a/input/samples/vn_simona_small_with_em_original/vn_simona_small_withem.conf b/input/samples/vn_simona_small_with_em_original/vn_simona_small_withem.conf new file mode 100755 index 0000000000..65aa0efdf6 --- /dev/null +++ b/input/samples/vn_simona_small_with_em_original/vn_simona_small_withem.conf @@ -0,0 +1,210 @@ +include "../common/pekko.conf" + +######### +# ATTENTION: Do not change this config file directly but use it as a base for your personal delta config for the +# vn_simona scenario! Delta configs can be created by including the config you want to change +# parameters from via include (e.g. include "input/samples/vn_simona/vn_simona.conf") at the +# beginning of your config file and then just override the parameters you want to change! +######### + +################################################################## +# Simulation Parameters +################################################################## +simona.simulationName = "simple_vn_simona_withem" + +################################################################## +# Time Parameters +################################################################## +simona.time.startDateTime = "2024-01-01T12:00:00Z" +simona.time.endDateTime = "2024-01-01T13:00:00Z" +simona.time.schedulerReadyCheckWindow = 900 + +################################################################## +# Input Parameters +################################################################## +simona.input.primary.csvParams = { + directoryPath: "simona/input/samples/vn_simona_small_with_em/fullGrid" + csvSep: "," + isHierarchic: false +} +simona.input.grid.datasource.id = "csv" +simona.input.grid.datasource.csvParams = { + directoryPath: "simona/input/samples/vn_simona_small_with_em/fullGrid" + csvSep: "," + isHierarchic: false +} + +simona.input.weather.datasource = { + scheme = "icon" + sampleParams.use = true + coordinateSource.sampleParams.use = true + maxCoordinateDistance = 50000 +} + +################################################################## +# Output Parameters +################################################################## +simona.output.base.dir = "simona/output/vn_simona_small_with_em" +simona.output.base.addTimestampToOutputDir = true + +simona.output.sink.csv { + fileFormat = ".csv" + filePrefix = "" + fileSuffix = "" +} + +simona.output.grid = { + notifier = "grid" + nodes = false + lines = false + switches = false + transformers2w = false + transformers3w = false +} +simona.output.participant.defaultConfig = { + notifier = "default" + powerRequestReply = false + simulationResult = true +} +simona.output.participant.individualConfigs = [ + { + notifier = "pv" + powerRequestReply = false + simulationResult = true + }, + { + notifier = "wec" + powerRequestReply = false + simulationResult = true + }, + { + notifier = "evcs" + powerRequestReply = false + simulationResult = true + } +] +simona.output.thermal = { + defaultConfig = { + notifier = "default", + simulationResult = false + } + individualConfigs = [ + { + notifier = "house", + simulationResult = true + } + ] +} + +################################################################## +# Runtime Configuration // todo refactor as this naming is misleading and partly unneeded +################################################################## +simona.runtime.selected_subgrids = [] +simona.runtime.selected_volt_lvls = [] + +simona.runtime.participant.load = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + modelBehaviour = "fix" + reference = "power" + } + individualConfigs = [] +} + +simona.runtime.participant.fixedFeedIn = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +simona.runtime.participant.pv = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +simona.runtime.participant.wec = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +simona.runtime.participant.evcs = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +simona.runtime.participant.hp = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +# # # # # +# ATTENTION: calculateMissingReactivePowerWithModel and scaling is ignored here. +# # # # # +simona.runtime.participant.em = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +################################################################## +# Event Configuration +################################################################## +simona.event.listener = [] + +################################################################## +# Grid Configuration +################################################################## + +simona.gridConfig.refSystems = [ + {sNom = "100 kVA", vNom = "0.4 kV", voltLvls = [{id = "LV", vNom = "0.4 kV"}]}, + {sNom = "60 MVA", vNom = "20 kV", voltLvls = [{id = "MV", vNom = "20 kV"}]}, + {sNom = "600 MVA", vNom = "110 kV", voltLvls = [{id = "HV", vNom = "110 kV"}]}, + {sNom = "1000 MVA", vNom = "380 kV", voltLvls = [{id = "EHV", vNom = "380 kV"}]} +] + +################################################################## +# Power Flow Configuration +################################################################## +simona.powerflow.maxSweepPowerDeviation = 1E-5 // the maximum allowed deviation in power between two sweeps, before overall convergence is assumed +simona.powerflow.newtonraphson.epsilon = [1E-12] +simona.powerflow.newtonraphson.iterations = 50 +simona.powerflow.resolution = "3600s" +simona.powerflow.stopOnFailure = true + +simona.control.transformer = [ + { + transformers = ["31a2b9bf-e785-4475-aa44-1c34646e8c79"], + measurements = ["923f2d69-3093-4198-86e4-13d2d1c220f8"], + vMin = 0.98, + vMax = 1.02 + }, { + transformers = ["1132dbf4-e8a1-44ae-8415-f42d4497aa1d"], + measurements = ["7686b818-a0ba-465c-8e4e-f7d3c4e171fc"], + vMin = 0.98, + vMax = 1.02 + } +] diff --git a/src/main/scala/edu/ie3/simona/agent/EnvironmentRefs.scala b/src/main/scala/edu/ie3/simona/agent/EnvironmentRefs.scala index 4b5c6c920d..3de35b171c 100644 --- a/src/main/scala/edu/ie3/simona/agent/EnvironmentRefs.scala +++ b/src/main/scala/edu/ie3/simona/agent/EnvironmentRefs.scala @@ -31,4 +31,5 @@ final case class EnvironmentRefs( primaryServiceProxy: ClassicRef, weather: ClassicRef, evDataService: Option[ClassicRef], + emDataService: Option[ClassicRef] ) diff --git a/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala b/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala index 06a7bf598a..2558fc0b02 100644 --- a/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala +++ b/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala @@ -12,25 +12,24 @@ 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.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.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.services.ServiceMessage.ExtEmDataServiceRegistrationMessage +import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.WrappedRegistrationSuccessfulMessage +import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.RegistrationSuccessfulMessage 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.scaladsl.{Behaviors, StashBuffer} import org.apache.pekko.actor.typed.{ActorRef, Behavior} +import org.apache.pekko.actor.{ActorRef => ClassicRef} +import org.apache.pekko.actor.typed.scaladsl.adapter._ import java.time.ZonedDateTime @@ -73,6 +72,7 @@ object EmAgent { override val tick: Long = msg.tick } + /** Creates the initial [[Behavior]] for an [[EmAgent]] in an inactive state * * @param inputModel @@ -100,7 +100,17 @@ object EmAgent { simulationStartDate: ZonedDateTime, parent: Either[ActorRef[SchedulerMessage], ActorRef[FlexResponse]], listener: Iterable[ActorRef[ResultEvent]], + extEmDataService: Option[ClassicRef] ): Behavior[Request] = Behaviors.setup[Request] { ctx => + val flexAdapterEmDataService = ctx.messageAdapter[FlexRequest](Flex) + if (extEmDataService.isDefined) { + extEmDataService.getOrElse(throw new RuntimeException("No Service")) ! ExtEmDataServiceRegistrationMessage( + inputModel.getUuid, + ctx.self, + flexAdapterEmDataService + ) + } + val constantData = EmData( outputConfig, simulationStartDate, @@ -117,17 +127,18 @@ object EmAgent { FlexControlledData(parentEm, flexAdapter) } .left - .map { scheduler => - { - val activationAdapter = ctx.messageAdapter[Activation] { msg => - EmActivation(msg.tick) - } - SchedulerData(scheduler, activationAdapter) + .map { scheduler => { + val activationAdapter = ctx.messageAdapter[Activation] { msg => + EmActivation(msg.tick) } + SchedulerData(scheduler, activationAdapter) + } }, listener, + ExternalEmDataServiceData(extEmDataService, None) ) + val modelShell = EmModelShell( inputModel.getUuid, inputModel.getId, @@ -135,10 +146,12 @@ object EmAgent { modelConfig, ) + ctx.log.info(s"EMAgent ${modelShell.uuid} goes to inactive!") + inactive( constantData, modelShell, - EmDataCore.create(simulationStartDate), + EmDataCore.create(simulationStartDate) ) } @@ -146,22 +159,36 @@ object EmAgent { * request to be activated. */ private def inactive( - emData: EmData, - modelShell: EmModelShell, - core: EmDataCore.Inactive, + emData: EmData, + modelShell: EmModelShell, + core: EmDataCore.Inactive, ): Behavior[Request] = Behaviors.receivePartial { - case (_, RegisterParticipant(model, actor, spi)) => + case (ctx, RegisterParticipant(model, actor, spi)) => + ctx.log.info(s"EM Agent ${modelShell.uuid} RegisterParticipant model $model") val updatedModelShell = modelShell.addParticipant(model, spi) val updatedCore = core.addParticipant(actor, model) inactive(emData, updatedModelShell, updatedCore) - case (_, ScheduleFlexRequest(participant, newTick, scheduleKey)) => + case (ctx, WrappedRegistrationSuccessfulMessage(RegistrationSuccessfulMessage(serviceRef, nextDataTick))) => + ctx.log.info(s"EM Agent ${modelShell.uuid} will use external set points!") + /* + val flexAdapter = ctx.messageAdapter[FlexRequest](Flex) + val updatedEmData = emData.copy( + parentData = Right(FlexControlledData(emData.extEmDataService.getOrElse(throw new RuntimeException("")).toTyped, flexAdapter)) + ) + */ + inactive(emData, modelShell, core) + + + case (ctx, ScheduleFlexRequest(participant, newTick, scheduleKey)) => + ctx.log.info(s"EM Agent ${modelShell.uuid} got ScheduleFlexRequest!") val (maybeSchedule, newCore) = core .handleSchedule(participant, newTick) maybeSchedule match { case Some(scheduleTick) => + ctx.log.info(s"EM Agent ${modelShell.uuid} -> parentData = ${emData.parentData}") // also potentially schedule with parent if the new earliest tick is // different from the old earliest tick (including if nothing had // been scheduled before) @@ -188,14 +215,17 @@ object EmAgent { inactive(emData, modelShell, newCore) case (ctx, msg: ActivationRequest) => + ctx.log.info(s"EM Agent ${modelShell.uuid} got ActivationRequest = $msg") val flexOptionsCore = core.activate(msg.tick) msg match { case Flex(_: RequestFlexOptions) | EmActivation(_) => + ctx.log.info(s"Activation for tick ${msg.tick}") val (toActivate, newCore) = flexOptionsCore.takeNewFlexRequests() toActivate.foreach { _ ! RequestFlexOptions(msg.tick) } + ctx.log.info(s"toActivate $toActivate") awaitingFlexOptions(emData, modelShell, newCore) @@ -209,8 +239,16 @@ object EmAgent { ctx.self ! msg awaitingFlexCtrl(emData, modelShell, flexOptionsCore) + case Flex(dataMsg: ProvideExtEmSetPoint) => + // got set point before activation -> put msg in queue and wait + ctx.log.info(s"Agent ${ctx.self} got external set point = $dataMsg") + val updatedEmData = emData.copy( + extEmDataServiceData = emData.extEmDataServiceData.copy( + dataProvisionMessage = Some(dataMsg) + ) + ) + inactive(updatedEmData, modelShell, core) } - } /** Behavior of an [[EmAgent]] waiting for flex options to be received in @@ -219,15 +257,17 @@ object EmAgent { private def awaitingFlexOptions( emData: EmData, modelShell: EmModelShell, - flexOptionsCore: EmDataCore.AwaitingFlexOptions, - ): Behavior[Request] = Behaviors.receiveMessagePartial { - case flexOptions: ProvideFlexOptions => + flexOptionsCore: EmDataCore.AwaitingFlexOptions + ): Behavior[Request] = Behaviors.receivePartial { + case (ctx, flexOptions: ProvideFlexOptions) => val updatedCore = flexOptionsCore.handleFlexOptions(flexOptions) if (updatedCore.isComplete) { val allFlexOptions = updatedCore.getFlexOptions + ctx.log.info(s"EM Agent ${ctx.self} allFlexOptions = $allFlexOptions") + emData.parentData match { case Right(flexStateData) => // aggregate flex options and provide to parent @@ -271,11 +311,31 @@ object EmAgent { case Left(_) => // We're not em-controlled ourselves, - // always desire to come as close as possible to 0 kW - val setPower = zeroKW - + // always desire to come as close as possible to 0 kW -> maybe overwrite it if we get a set point + var setPower = zeroKW + var updatedEmData = emData + if (emData.extEmDataServiceData.extEmDataService.isDefined) { // We get external set points + if (emData.extEmDataServiceData.dataProvisionMessage.isEmpty) { // Still waiting for set points... + awaitingFlexOptions( + emData, + modelShell, + updatedCore, + ) + } else { // We got set points + setPower = emData.extEmDataServiceData.dataProvisionMessage.map(setPoint => setPoint.setPower).getOrElse(throw new RuntimeException("Got a wrong set point!")) + ctx.log.info(s"[UNCONTROLLED] EM Agent ${ctx.self}: Got a external Set Power = $setPower") + updatedEmData = emData.copy( + extEmDataServiceData = emData.extEmDataServiceData.copy( + dataProvisionMessage = None + ) + ) + } + } + ctx.log.info(s"[UNCONTROLLED] EM Agent ${ctx.self}: Starting determination of flex control with set power = $setPower") val flexControl = modelShell.determineFlexControl(allFlexOptions, setPower) + ctx.log.info(s"[UNCONTROLLED] EM Agent ${ctx.self}: Got flexControl = $flexControl") + val (allFlexMsgs, newCore) = updatedCore .handleFlexCtrl(flexControl) @@ -283,12 +343,12 @@ object EmAgent { .complete() allFlexMsgs.foreach { case (actor, msg) => + ctx.log.info(s"[UNCONTROLLED] EM Agent ${ctx.self}: For actor = $actor send msg = $msg") actor ! msg } - awaitingCompletions(emData, modelShell, newCore) + awaitingCompletions(updatedEmData, modelShell, newCore) } - } else { // more flex options expected awaitingFlexOptions( @@ -297,7 +357,15 @@ object EmAgent { updatedCore, ) } - + case (ctx, Flex(dataMsg: ProvideExtEmSetPoint)) => + // got set point before activation -> put msg in queue and wait + ctx.log.info(s"Agent ${ctx.self} got external set point = $dataMsg") + val updatedEmData = emData.copy( + extEmDataServiceData = emData.extEmDataServiceData.copy( + dataProvisionMessage = Some(dataMsg) + ) + ) + awaitingFlexOptions(updatedEmData, modelShell, flexOptionsCore) /* 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 @@ -312,8 +380,11 @@ object EmAgent { emData: EmData, modelShell: EmModelShell, flexOptionsCore: EmDataCore.AwaitingFlexOptions, - ): Behavior[Request] = Behaviors.receiveMessagePartial { - case Flex(flexCtrl: IssueFlexControl) => + ): Behavior[Request] = Behaviors.receivePartial { + case (ctx, Flex(flexCtrl: IssueFlexControl)) => + ctx.log.info(s"emData = $emData") + ctx.log.info(s"modelShell = $modelShell") + ctx.log.info(s"agent ${ctx.self}: flexCtrl = $flexCtrl") val flexData = emData.parentData.getOrElse( throw new CriticalFailureException(s"EmAgent is not EM-controlled.") ) @@ -345,6 +416,7 @@ object EmAgent { .complete() allFlexMsgs.foreach { case (actor, msg) => + ctx.log.info(s"Agent ${ctx.self}: For actor = $actor send msg = $msg") actor ! msg } @@ -447,6 +519,7 @@ object EmAgent { simulationStartDate: ZonedDateTime, parentData: Either[SchedulerData, FlexControlledData], listener: Iterable[ActorRef[ResultEvent]], + extEmDataServiceData: ExternalEmDataServiceData ) /** The existence of this data object indicates that the corresponding agent @@ -462,4 +535,9 @@ object EmAgent { scheduler: ActorRef[SchedulerMessage], activationAdapter: ActorRef[Activation], ) + + final case class ExternalEmDataServiceData( + extEmDataService: Option[ClassicRef], + dataProvisionMessage: Option[ProvideExtEmSetPoint] + ) } 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 7ba4abef77..7936d075c8 100644 --- a/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala +++ b/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala @@ -191,16 +191,25 @@ class GridAgentController( _.getControllingEm.toScala.map(em => em.getUuid -> em) }.toMap + //log.info(s"firstLevelEms = $firstLevelEms") + val allEms = buildEmsRecursively( participantConfigUtil, outputConfigUtil, firstLevelEms, + extEmDataService = environmentRefs.emDataService ) + log.info(s"Built allEms = $allEms") + log.info(s"Particpants = $participants") + participants .map { participant => + log.info(s"Built Participant = $participant") val node = participant.getNode + + val actorRef = buildParticipantActor( participantsConfig.requestVoltageDeviationThreshold, participantConfigUtil, @@ -208,7 +217,14 @@ class GridAgentController( participant, thermalIslandGridsByBusId, environmentRefs, - allEms.get(participant.getUuid), + participant.getControllingEm.toScala.map(_.getUuid).map( + uuid => allEms.getOrElse( + uuid, + throw new CriticalFailureException( + s"Actor for EM $uuid not found." + ), + ), + ) ) introduceAgentToEnvironment(actorRef) // return uuid to actorRef @@ -240,6 +256,7 @@ class GridAgentController( outputConfigUtil: OutputConfigUtil, emInputs: Map[UUID, EmInput], previousLevelEms: Map[UUID, ActorRef[FlexResponse]] = Map.empty, + extEmDataService: Option[ClassicRef] = None ): Map[UUID, ActorRef[FlexResponse]] = { // For the current level, split controlled and uncontrolled EMs. // Uncontrolled EMs can be built right away. @@ -261,6 +278,9 @@ class GridAgentController( val previousLevelAndUncontrolledEms = previousLevelEms ++ uncontrolledEms.toMap + log.info(s"controlledEmInputs = $controlledEmInputs") + log.info(s"previousLevelAndUncontrolledEms = $previousLevelAndUncontrolledEms") + if (controlledEmInputs.nonEmpty) { // For controlled EMs at the current level, more EMs // might need to be built at the next recursion level. @@ -268,16 +288,20 @@ class GridAgentController( case (uuid, emInput) => emInput.getControllingEm.toScala.map(uuid -> _) } - + log.info(s"now build controlled Ems -> These are possible controllers = $controllingEms") // Return value includes previous level and uncontrolled EMs of this level val recursiveEms = buildEmsRecursively( participantConfigUtil, outputConfigUtil, controllingEms, previousLevelAndUncontrolledEms, + extEmDataService = extEmDataService ) + log.info(s"-> after recursion recursiveEms = $recursiveEms") val controlledEms = controlledEmInputs.map { case (uuid, emInput) => + log.info(s"-> uuid = $uuid, emInput = $emInput") + /* val controllingEm = emInput.getControllingEm.toScala .map(_.getUuid) .map(uuid => @@ -289,6 +313,15 @@ class GridAgentController( ) ) + */ + val controllingEm = Some(recursiveEms.getOrElse( + uuid, + throw new CriticalFailureException( + s"Actor for EM $uuid not found." + ), + )) + log.info(s"-> contorllingEm = $controllingEm") + uuid -> buildEm( emInput, participantConfigUtil.getOrDefault[EmRuntimeConfig](uuid), @@ -296,7 +329,7 @@ class GridAgentController( maybeControllingEm = controllingEm, ) }.toMap - + log.info(s"-> controlledEms = $controlledEms, recursiveEms = $recursiveEms") recursiveEms ++ controlledEms } else { previousLevelAndUncontrolledEms @@ -788,7 +821,8 @@ class GridAgentController( modelConfiguration: EmRuntimeConfig, outputConfig: NotifierConfig, maybeControllingEm: Option[ActorRef[FlexResponse]], - ): ActorRef[FlexResponse] = + ): ActorRef[FlexResponse] = { + log.info("Spawn Em = " + emInput + ", maybeControlling Em = " + maybeControllingEm) gridAgentContext.spawn( EmAgent( emInput, @@ -802,9 +836,11 @@ class GridAgentController( environmentRefs.scheduler ), listener, + environmentRefs.emDataService ), actorName(classOf[EmAgent.type], emInput.getId), ) + } /** Introduces the given agent to scheduler * diff --git a/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgent.scala b/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgent.scala index a01c5677a1..e42c3f2de2 100644 --- a/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgent.scala +++ b/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgent.scala @@ -301,7 +301,7 @@ abstract class ParticipantAgent[ maybeEmAgent, ), ) => - log.debug("Will perform model calculations") + log.info("Will perform model calculations") initializeParticipantForModelCalculation( inputModel, modelConfig, 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 9b3b8580eb..71f092d3c8 100644 --- a/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgentFundamentals.scala +++ b/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgentFundamentals.scala @@ -275,8 +275,12 @@ protected trait ParticipantAgentFundamentals[ val awaitRegistrationResponsesFrom = registerForServices(inputModel.electricalInputModel, services) + log.info(s"Participant Agent maybeEmAgent $maybeEmAgent") // register with EM if applicable maybeEmAgent.foreach { emAgent => + log.info(s"Agent ${ + inputModel.electricalInputModel.getUuid + } register for Em!") emAgent ! RegisterParticipant( inputModel.electricalInputModel.getUuid, self.toTyped[FlexRequest], diff --git a/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala b/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala index 86e5d87c24..a1d31f0d25 100644 --- a/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala +++ b/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala @@ -151,7 +151,7 @@ object ResultEventListener extends Transformer3wResultSupport { log: Logger, nextTick: Option[Long] = None ): BaseData = { - log.info("Got Result " + resultEntity) + //log.info("Got Result " + resultEntity) handOverToSink(resultEntity, baseData.classToSink, log) if (baseData.extResultDataService.isDefined) { handOverToExternalService( diff --git a/src/main/scala/edu/ie3/simona/main/RunSimonaWithOpsim.scala b/src/main/scala/edu/ie3/simona/main/RunSimonaWithOpsim.scala new file mode 100644 index 0000000000..9d3c54dfa8 --- /dev/null +++ b/src/main/scala/edu/ie3/simona/main/RunSimonaWithOpsim.scala @@ -0,0 +1,62 @@ +/* + * © 2020. TU Dortmund University, + * Institute of Energy Systems, Energy Efficiency and Energy Economics, + * Research group Distribution grid planning and operation + */ + +package edu.ie3.simona.main + +import edu.ie3.simona.config.{ArgsParser, ConfigFailFast, SimonaConfig} +import edu.ie3.simona.main.RunSimona._ +import edu.ie3.simona.sim.SimonaSim +import edu.ie3.simona.sim.setup.{SimonaOpsimSetup, SimonaSimpleExtSimulationSetup, SimonaStandaloneSetup} +import org.apache.pekko.actor.typed.scaladsl.AskPattern._ +import org.apache.pekko.actor.typed.{ActorSystem, Scheduler} +import org.apache.pekko.util.Timeout + +import scala.concurrent.Await +import scala.concurrent.duration.DurationInt + +/** Run a standalone simulation of simona + * + * @since 01.07.20 + */ +object RunSimonaWithOpsim extends RunSimona[SimonaOpsimSetup] { + + override implicit val timeout: Timeout = Timeout(12.hours) + + override def setup(args: Array[String]): SimonaOpsimSetup = { + // get the config and prepare it with the provided args + val (arguments, parsedConfig) = ArgsParser.prepareConfig(args) + + // config fail fast check + val simonaConfig = SimonaConfig(parsedConfig) + ConfigFailFast.check(parsedConfig, simonaConfig) + + SimonaOpsimSetup( + parsedConfig, + SimonaOpsimSetup.buildResultFileHierarchy(parsedConfig), + mainArgs = arguments.mainArgs, + ) + } + + override def run(simonaSetup: SimonaOpsimSetup): Boolean = { + val simonaSim = ActorSystem( + SimonaSim(simonaSetup), + name = "Simona", + config = simonaSetup.typeSafeConfig, + ) + + implicit val scheduler: Scheduler = simonaSim.scheduler + + // run the simulation + val terminated = simonaSim.ask[SimonaEnded](ref => SimonaSim.Start(ref)) + + Await.result(terminated, timeout.duration) match { + case SimonaEnded(successful) => + simonaSim.terminate() + + successful + } + } +} diff --git a/src/main/scala/edu/ie3/simona/main/RunSimonaWithSimpleExtSimulation.scala b/src/main/scala/edu/ie3/simona/main/RunSimonaWithSimpleExtSimulation.scala new file mode 100644 index 0000000000..08052cd169 --- /dev/null +++ b/src/main/scala/edu/ie3/simona/main/RunSimonaWithSimpleExtSimulation.scala @@ -0,0 +1,62 @@ +/* + * © 2020. TU Dortmund University, + * Institute of Energy Systems, Energy Efficiency and Energy Economics, + * Research group Distribution grid planning and operation + */ + +package edu.ie3.simona.main + +import edu.ie3.simona.config.{ArgsParser, ConfigFailFast, SimonaConfig} +import edu.ie3.simona.main.RunSimona._ +import edu.ie3.simona.sim.SimonaSim +import edu.ie3.simona.sim.setup.{SimonaSimpleExtSimulationSetup, SimonaStandaloneSetup} +import org.apache.pekko.actor.typed.scaladsl.AskPattern._ +import org.apache.pekko.actor.typed.{ActorSystem, Scheduler} +import org.apache.pekko.util.Timeout + +import scala.concurrent.Await +import scala.concurrent.duration.DurationInt + +/** Run a standalone simulation of simona + * + * @since 01.07.20 + */ +object RunSimonaWithSimpleExtSimulation extends RunSimona[SimonaSimpleExtSimulationSetup] { + + override implicit val timeout: Timeout = Timeout(12.hours) + + override def setup(args: Array[String]): SimonaSimpleExtSimulationSetup = { + // get the config and prepare it with the provided args + val (arguments, parsedConfig) = ArgsParser.prepareConfig(args) + + // config fail fast check + val simonaConfig = SimonaConfig(parsedConfig) + ConfigFailFast.check(parsedConfig, simonaConfig) + + SimonaSimpleExtSimulationSetup( + parsedConfig, + SimonaSimpleExtSimulationSetup.buildResultFileHierarchy(parsedConfig), + mainArgs = arguments.mainArgs, + ) + } + + override def run(simonaSetup: SimonaSimpleExtSimulationSetup): Boolean = { + val simonaSim = ActorSystem( + SimonaSim(simonaSetup), + name = "Simona", + config = simonaSetup.typeSafeConfig, + ) + + implicit val scheduler: Scheduler = simonaSim.scheduler + + // run the simulation + val terminated = simonaSim.ask[SimonaEnded](ref => SimonaSim.Start(ref)) + + Await.result(terminated, timeout.duration) match { + case SimonaEnded(successful) => + simonaSim.terminate() + + successful + } + } +} 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 d339ba3eb0..c225f5bcb4 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 @@ -85,7 +85,7 @@ object FlexibilityMessage { */ trait ProvideFlexOptions extends FlexResponse - trait ProvideEmSetPoints extends EmAgent.Request + final case class ProvideExtEmSetPoint(override val tick: Long, setPower: Power) extends FlexRequest /** Message that issues flexibility control to a flex options provider, i.e. a * feasible set point is delivered that the flex options provider should diff --git a/src/main/scala/edu/ie3/simona/ontology/messages/services/ServiceMessage.scala b/src/main/scala/edu/ie3/simona/ontology/messages/services/ServiceMessage.scala index 53ede35f10..9a1f2e54da 100644 --- a/src/main/scala/edu/ie3/simona/ontology/messages/services/ServiceMessage.scala +++ b/src/main/scala/edu/ie3/simona/ontology/messages/services/ServiceMessage.scala @@ -6,10 +6,14 @@ package edu.ie3.simona.ontology.messages.services -import org.apache.pekko.actor.ActorRef +import edu.ie3.simona.agent.em.EmAgent + +import org.apache.pekko.actor.{ActorRef => ClassicRef} +import org.apache.pekko.actor.typed.ActorRef import java.util.UUID import edu.ie3.simona.agent.participant.data.Data +import edu.ie3.simona.ontology.messages.flex.FlexibilityMessage.FlexRequest import edu.ie3.simona.scheduler.ScheduleLock.ScheduleKey /** Collections of all messages, that are send to and from the different @@ -40,16 +44,22 @@ case object ServiceMessage { * @param requestingActor * Reference to the requesting actor */ - final case class WorkerRegistrationMessage(requestingActor: ActorRef) + final case class WorkerRegistrationMessage(requestingActor: ClassicRef) extends ServiceRegistrationMessage final case class ExtPrimaryDataServiceRegistrationMessage( modelUuid: UUID, - requestingActor: ActorRef, + requestingActor: ClassicRef, ) extends ServiceRegistrationMessage + final case class ExtEmDataServiceRegistrationMessage( + modelUuid: UUID, + requestingActor: ActorRef[EmAgent.Request], + flexAdapter: ActorRef[FlexRequest] + ) extends ServiceRegistrationMessage + sealed trait RegistrationResponseMessage extends ServiceMessage { - val serviceRef: ActorRef + val serviceRef: ClassicRef } object RegistrationResponseMessage { @@ -57,14 +67,18 @@ case object ServiceMessage { /** Message, that is used to confirm a successful registration */ final case class RegistrationSuccessfulMessage( - override val serviceRef: ActorRef, + override val serviceRef: ClassicRef, nextDataTick: Option[Long], ) extends RegistrationResponseMessage + final case class WrappedRegistrationSuccessfulMessage( + registrationSuccessfulMessage: RegistrationSuccessfulMessage + ) extends EmAgent.Request + /** Message, that is used to announce a failed registration */ final case class RegistrationFailedMessage( - override val serviceRef: ActorRef + override val serviceRef: ClassicRef ) extends RegistrationResponseMessage final case class ScheduleServiceActivation( @@ -80,7 +94,7 @@ case object ServiceMessage { */ trait ProvisionMessage[D <: Data] extends ServiceMessage { val tick: Long - val serviceRef: ActorRef + val serviceRef: ClassicRef val data: D val nextDataTick: Option[Long] val unlockKey: Option[ScheduleKey] diff --git a/src/main/scala/edu/ie3/simona/service/em/ExtEmDataService.scala b/src/main/scala/edu/ie3/simona/service/em/ExtEmDataService.scala index 6a593c08ae..e00ce57875 100644 --- a/src/main/scala/edu/ie3/simona/service/em/ExtEmDataService.scala +++ b/src/main/scala/edu/ie3/simona/service/em/ExtEmDataService.scala @@ -6,16 +6,18 @@ import edu.ie3.simona.api.data.em.ontology.{EmDataMessageFromExt, ProvideEmData} import edu.ie3.simona.api.data.ontology.DataMessageFromExt import edu.ie3.simona.exceptions.WeatherServiceException.InvalidRegistrationRequestException import edu.ie3.simona.exceptions.{InitializationException, ServiceException} -import edu.ie3.simona.ontology.messages.flex.FlexibilityMessage.IssuePowerControl -import edu.ie3.simona.ontology.messages.services.ServiceMessage.ExtPrimaryDataServiceRegistrationMessage +import edu.ie3.simona.ontology.messages.flex.FlexibilityMessage.{FlexRequest, IssuePowerControl, ProvideExtEmSetPoint, RequestFlexOptions} +import edu.ie3.simona.ontology.messages.services.ServiceMessage.ExtEmDataServiceRegistrationMessage import edu.ie3.simona.ontology.messages.services.{DataMessage, ServiceMessage} import edu.ie3.simona.service.ServiceStateData.{InitializeServiceStateData, ServiceBaseStateData} -import edu.ie3.simona.service.em.ExtEmDataService.{ExtEmDataStateData, InitExtEmData} -import edu.ie3.simona.service.primary.ExtPrimaryDataService.{ExtPrimaryDataStateData, InitExtPrimaryData} +import edu.ie3.simona.service.em.ExtEmDataService.{ExtEmDataStateData, InitExtEmData, WrappedIssuePowerControl} import edu.ie3.simona.service.{ExtDataSupport, SimonaService} -import org.apache.pekko.actor.{ActorContext, ActorRef, Props} +import org.apache.pekko.actor.typed.ActorRef +import org.apache.pekko.actor.{ActorContext, Props, ActorRef => ClassicRef} import squants.Power import squants.energy.Kilowatts +import edu.ie3.simona.agent.em.EmAgent +import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.{RegistrationSuccessfulMessage, WrappedRegistrationSuccessfulMessage} import java.util.UUID import scala.jdk.CollectionConverters.MapHasAsScala @@ -23,27 +25,32 @@ import scala.util.{Failure, Success, Try} object ExtEmDataService { - def props(scheduler: ActorRef): Props = + def props(scheduler: ClassicRef): Props = Props( - new ExtEmDataService(scheduler: ActorRef) + new ExtEmDataService(scheduler: ClassicRef) ) final case class ExtEmDataStateData( extEmData: ExtEmData, subscribers: List[UUID] = List.empty, - uuidToActorRef: Map[UUID, ActorRef] = Map.empty[UUID, ActorRef], // subscribers in SIMONA + uuidToActorRef: Map[UUID, ActorRef[EmAgent.Request]] = Map.empty[UUID, ActorRef[EmAgent.Request]], // subscribers in SIMONA + uuidToAdapterRef: Map[UUID, ActorRef[FlexRequest]] = Map.empty[UUID, ActorRef[FlexRequest]], // subscribers in SIMONA extEmDataMessage: Option[EmDataMessageFromExt] = None, ) extends ServiceBaseStateData case class InitExtEmData( extEmData: ExtEmData ) extends InitializeServiceStateData + + final case class WrappedIssuePowerControl( + issuePowerControl: IssuePowerControl + ) extends EmAgent.Request } final case class ExtEmDataService( - override val scheduler: ActorRef + override val scheduler: ClassicRef ) extends SimonaService[ExtEmDataStateData](scheduler) with ExtDataSupport[ExtEmDataStateData] { @@ -64,7 +71,16 @@ final case class ExtEmDataService( override def init(initServiceData: InitializeServiceStateData): Try[(ExtEmDataStateData, Option[Long])] = initServiceData match { case InitExtEmData(extEmData) => val emDataInitializedStateData = ExtEmDataStateData( - extEmData + extEmData, + subscribers = List( + UUID.fromString("c3a7e9f5-b492-4c85-af2d-1e93f6a25443"), + UUID.fromString("f9dc7ce6-658c-4101-a12f-d58bb889286b"), + UUID.fromString("957938b7-0476-4fab-a1b3-6ce8615857b3") + ) + //subscribers = List( + // UUID.fromString("fd1a8de9-722a-4304-8799-e1e976d9979c"), + // UUID.fromString("ff0b995a-86ff-4f4d-987e-e475a64f2180") + //) ) Success( emDataInitializedStateData, @@ -93,11 +109,12 @@ final case class ExtEmDataService( registrationMessage: ServiceMessage.ServiceRegistrationMessage )( implicit serviceStateData: ExtEmDataStateData): Try[ExtEmDataStateData] = registrationMessage match { - case ExtPrimaryDataServiceRegistrationMessage( + case ExtEmDataServiceRegistrationMessage( modelUuid, requestingActor, + flexAdapter ) => - null + Success(handleEmRegistrationRequest(modelUuid, requestingActor, flexAdapter)) case invalidMessage => Failure( InvalidRegistrationRequestException( @@ -105,6 +122,25 @@ final case class ExtEmDataService( ) ) } + + private def handleEmRegistrationRequest( + modelUuid: UUID, + modelActorRef: ActorRef[EmAgent.Request], + flexAdapterRef: ActorRef[FlexRequest] + )( + implicit serviceStateData: ExtEmDataStateData): ExtEmDataStateData = { + if (serviceStateData.subscribers.contains(modelUuid)) { + modelActorRef ! WrappedRegistrationSuccessfulMessage(RegistrationSuccessfulMessage(self, None)) + serviceStateData.copy( + uuidToActorRef = serviceStateData.uuidToActorRef + (modelUuid -> modelActorRef), + uuidToAdapterRef = serviceStateData.uuidToAdapterRef + (modelUuid -> flexAdapterRef) + ) + } else { + serviceStateData + } + } + + /** Send out the information to all registered recipients * * @param tick @@ -139,7 +175,7 @@ final case class ExtEmDataService( ) = { val actorToEmData = emData.asScala.flatMap { case (agent, emDataPerAgent) => - serviceStateData.uuidToActorRef + serviceStateData.uuidToAdapterRef .get(agent) .map((_, convertToSetPoint(emDataPerAgent))) .orElse { @@ -150,10 +186,13 @@ final case class ExtEmDataService( None } } + log.info(s"Received ActorToEmData = $actorToEmData") if (actorToEmData.nonEmpty) { + log.info("ProvideEmData") + actorToEmData.foreach { - case (actor, setPoint) => actor ! IssuePowerControl( + case (actor, setPoint) => actor ! ProvideExtEmSetPoint( tick, setPoint ) @@ -179,8 +218,10 @@ final case class ExtEmDataService( */ override protected def handleDataMessage(extMsg: DataMessageFromExt)(implicit serviceStateData: ExtEmDataStateData): ExtEmDataStateData = { extMsg match { - case extEmDataMessage: EmDataMessageFromExt => serviceStateData.copy( - extEmDataMessage = Some(extEmDataMessage) + case extEmDataMessage: EmDataMessageFromExt => + log.info("Received EmDataMessageFromExt") + serviceStateData.copy( + extEmDataMessage = Some(extEmDataMessage) ) } } diff --git a/src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala b/src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala index 77420a96f2..0907901873 100644 --- a/src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala +++ b/src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala @@ -53,7 +53,7 @@ object ExtResultDataProvider { def apply( scheduler: ActorRef[SchedulerMessage] - ): Behavior[Request] = Behaviors.withStash(500) { buffer => + ): Behavior[Request] = Behaviors.withStash(5000) { buffer => Behaviors.setup[Request] { ctx => //ctx.log.info("Starting initialization!") val activationAdapter: ActorRef[Activation] = ctx.messageAdapter[Activation](msg => WrappedActivation(msg)) diff --git a/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala b/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala index 65847b0b18..f13c63999c 100644 --- a/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala +++ b/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala @@ -103,6 +103,7 @@ object SimonaSim { primaryServiceProxy, weatherService, extSimulationData.evDataService, + extSimulationData.extEmDataService ) val resultEventListeners = 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 dfd55222a5..0d68368bfe 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/ExtSimSetupData.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/ExtSimSetupData.scala @@ -12,6 +12,8 @@ import edu.ie3.simona.api.data.ExtData import edu.ie3.simona.api.data.ev.ExtEvData import edu.ie3.simona.api.data.primarydata.ExtPrimaryData import edu.ie3.simona.api.data.results.ExtResultData +import edu.ie3.simona.api.data.em.ExtEmData +import edu.ie3.simona.service.em.ExtEmDataService import edu.ie3.simona.service.ev.ExtEvDataService import org.apache.pekko.actor.typed.ActorRef import edu.ie3.simona.service.primary.ExtPrimaryDataService @@ -32,6 +34,9 @@ final case class ExtSimSetupData( def extPrimaryDataService: Option[ClassicRef] = extDataServices.get(classOf[ExtPrimaryDataService]) + def extEmDataService: Option[ClassicRef] = + extDataServices.get(classOf[ExtEmDataService]) + def extResultDataService: Option[ActorRef[ExtResultDataProvider.Request]] = extDataListener.get(ExtResultDataProvider.getClass) @@ -41,6 +46,11 @@ final case class ExtSimSetupData( def extPrimaryData: Option[ExtPrimaryData] = { extDatas.collectFirst { case extData: ExtPrimaryData => extData } } + + def extEmData: Option[ExtEmData] = { + extDatas.collectFirst { case extData: ExtEmData => extData } + } + def extResultData: Option[ExtResultData] = { extDatas.collectFirst { case extData: ExtResultData => extData } } diff --git a/src/main/scala/edu/ie3/simona/sim/setup/SimonaMosaikSetup.scala b/src/main/scala/edu/ie3/simona/sim/setup/SimonaMosaikSetup.scala index 07fdd2cf09..446f0d7914 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaMosaikSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaMosaikSetup.scala @@ -366,7 +366,7 @@ class SimonaMosaikSetup( simonaConfig.simona.time.startDateTime ) val extScheduler = scheduler(context, parent = rootScheduler) - val mosaikExtSim = new MosaikSimulation("127.0.0.1:5555") + val mosaikExtSim = new MosaikSimulation("127.0.0.1:37699") val extSimAdapterPhase1 = context.toClassic.simonaActorOf( ExtSimAdapter.props(extScheduler.toClassic), diff --git a/src/main/scala/edu/ie3/simona/sim/setup/SimonaOpsimSetup.scala b/src/main/scala/edu/ie3/simona/sim/setup/SimonaOpsimSetup.scala new file mode 100644 index 0000000000..eef0e2f72c --- /dev/null +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaOpsimSetup.scala @@ -0,0 +1,755 @@ +/* + * © 2020. TU Dortmund University, + * Institute of Energy Systems, Energy Efficiency and Energy Economics, + * Research group Distribution grid planning and operation + */ + +package edu.ie3.simona.sim.setup + +import com.typesafe.config.Config +import com.typesafe.scalalogging.LazyLogging +import edu.ie3.datamodel.graph.SubGridTopologyGraph +import edu.ie3.datamodel.models.input.container.{GridContainer, ThermalGrid} +import edu.ie3.datamodel.models.input.thermal.ThermalBusInput +import edu.ie3.simona.actor.SimonaActorNaming.RichActorRefFactory +import edu.ie3.simona.agent.EnvironmentRefs +import edu.ie3.simona.agent.grid.GridAgentMessage.CreateGridAgent +import edu.ie3.simona.agent.grid.{GridAgent, GridAgentMessage} +import edu.ie3.simona.api.ExtSimAdapter +import edu.ie3.simona.api.data.ExtData +import edu.ie3.simona.api.data.em.ExtEmData +import edu.ie3.simona.api.data.primarydata.ExtPrimaryData +import edu.ie3.simona.api.data.results.ExtResultData +import edu.ie3.simona.api.data.results.ontology.ResultDataMessageFromExt +import edu.ie3.simona.api.simulation.ExtSimAdapterData +import edu.ie3.simona.config.{ArgsParser, RefSystemParser, SimonaConfig} +import edu.ie3.simona.event.listener.{ResultEventListener, RuntimeEventListener} +import edu.ie3.simona.event.{ResultEvent, RuntimeEvent} +import edu.ie3.simona.exceptions.agent.GridAgentInitializationException +import edu.ie3.simona.io.grid.GridProvider +import edu.ie3.simona.ontology.messages.SchedulerMessage +import edu.ie3.simona.ontology.messages.SchedulerMessage.ScheduleActivation +import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.ScheduleServiceActivation +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.em.ExtEmDataService +import edu.ie3.simona.service.em.ExtEmDataService.InitExtEmData +import edu.ie3.simona.service.primary.ExtPrimaryDataService.InitExtPrimaryData +import edu.ie3.simona.service.primary.PrimaryServiceProxy.InitPrimaryServiceProxyStateData +import edu.ie3.simona.service.primary.{ExtPrimaryDataService, PrimaryServiceProxy} +import edu.ie3.simona.service.results.ExtResultDataProvider +import edu.ie3.simona.service.results.ExtResultDataProvider.{InitExtResultData, RequestDataMessageAdapter, RequestScheduleActivationAdapter} +import edu.ie3.simona.service.weather.WeatherService +import edu.ie3.simona.service.weather.WeatherService.InitWeatherServiceStateData +import edu.ie3.simona.sim.SimonaSim +import edu.ie3.simona.util.ResultFileHierarchy +import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK +import edu.ie3.simona.util.TickUtil.RichZonedDateTime +import edu.ie3.simopsim.OpsimEmSimulator +import edu.ie3.simpleextsim.{SimpleExtSimulation, SimpleExtSimulationWithEm} +import edu.ie3.util.TimeUtil +import org.apache.pekko.actor.typed.scaladsl.ActorContext +import org.apache.pekko.actor.typed.scaladsl.AskPattern._ +import org.apache.pekko.actor.typed.scaladsl.adapter.{ClassicActorRefOps, TypedActorContextOps, TypedActorRefOps} +import org.apache.pekko.actor.typed.{ActorRef, Scheduler} +import org.apache.pekko.actor.{ActorRef => ClassicRef} +import org.apache.pekko.util.{Timeout => PekkoTimeout} + +import java.util.UUID +import java.util.concurrent.LinkedBlockingQueue +import scala.concurrent.Await +import scala.concurrent.duration.DurationInt +import scala.jdk.CollectionConverters._ +import scala.jdk.DurationConverters._ + +/** Sample implementation to run a standalone simulation of simona configured + * with the provided [[SimonaConfig]] and [[ResultFileHierarchy]] + * + * @version 0.1 + * @since 01.07.20 + */ +class SimonaOpsimSetup( + val typeSafeConfig: Config, + simonaConfig: SimonaConfig, + resultFileHierarchy: ResultFileHierarchy, + runtimeEventQueue: Option[LinkedBlockingQueue[RuntimeEvent]] = None, + override val args: Array[String], +) extends SimonaSetup { + + override def gridAgents( + context: ActorContext[_], + environmentRefs: EnvironmentRefs, + resultEventListeners: Seq[ActorRef[ResultEvent]], + ): Iterable[ActorRef[GridAgentMessage]] = { + + /* get the grid */ + val subGridTopologyGraph = GridProvider + .gridFromConfig( + simonaConfig.simona.simulationName, + simonaConfig.simona.input.grid.datasource, + ) + .getSubGridTopologyGraph + val thermalGridsByThermalBus = GridProvider.getThermalGridsFromConfig( + simonaConfig.simona.input.grid.datasource + ) + + /* extract and prepare refSystem information from config */ + val configRefSystems = + RefSystemParser.parse(simonaConfig.simona.gridConfig.refSystems) + + /* Create all agents and map the sub grid id to their actor references */ + val subGridToActorRefMap = buildSubGridToActorRefMap( + subGridTopologyGraph, + context, + environmentRefs, + resultEventListeners, + ) + + val keys = ScheduleLock.multiKey( + context, + environmentRefs.scheduler, + INIT_SIM_TICK, + subGridTopologyGraph.vertexSet().size, + ) + + /* build the initialization data */ + subGridTopologyGraph + .vertexSet() + .asScala + .zip(keys) + .map { case (subGridContainer, key) => + /* Get all connections to superior and inferior sub grids */ + val subGridGates = + Set.from( + subGridTopologyGraph + .edgesOf(subGridContainer) + .asScala + .map(modifySubGridGateForThreeWindingSupport) + ) + val currentSubGrid = subGridContainer.getSubnet + val currentActorRef = subGridToActorRefMap.getOrElse( + currentSubGrid, + throw new GridAgentInitializationException( + "Was asked to setup agent for sub grid " + currentSubGrid + ", but did not found it's actor reference." + ), + ) + val thermalGrids = + getThermalGrids(subGridContainer, thermalGridsByThermalBus) + + /* build the grid agent data and check for its validity */ + val gridAgentInitData = SimonaStandaloneSetup.buildGridAgentInitData( + subGridContainer, + subGridToActorRefMap, + subGridGates, + configRefSystems, + thermalGrids, + ) + + currentActorRef ! CreateGridAgent(gridAgentInitData, key) + + currentActorRef + } + } + + override def primaryServiceProxy( + context: ActorContext[_], + scheduler: ActorRef[SchedulerMessage], + extSimSetupData: ExtSimSetupData, + ): ClassicRef = { + val simulationStart = TimeUtil.withDefaults.toZonedDateTime( + simonaConfig.simona.time.startDateTime + ) + val primaryServiceProxy = context.toClassic.simonaActorOf( + PrimaryServiceProxy.props( + scheduler.toClassic, + InitPrimaryServiceProxyStateData( + simonaConfig.simona.input.primary, + simulationStart, + extSimSetupData.extPrimaryDataService, + extSimSetupData.extPrimaryData + ), + simulationStart, + ) + ) + + scheduler ! ScheduleActivation(primaryServiceProxy.toTyped, INIT_SIM_TICK) + primaryServiceProxy + } + + override def weatherService( + context: ActorContext[_], + scheduler: ActorRef[SchedulerMessage], + ): ClassicRef = { + val weatherService = context.toClassic.simonaActorOf( + WeatherService.props( + scheduler.toClassic, + TimeUtil.withDefaults + .toZonedDateTime(simonaConfig.simona.time.startDateTime), + TimeUtil.withDefaults + .toZonedDateTime(simonaConfig.simona.time.endDateTime), + ) + ) + weatherService ! SimonaService.Create( + InitWeatherServiceStateData( + simonaConfig.simona.input.weather.datasource + ), + ScheduleLock.singleKey(context, scheduler, INIT_SIM_TICK), + ) + + weatherService + } + + /* + override def extSimulations( + context: ActorContext[_], + rootScheduler: ActorRef[SchedulerMessage], + ): ExtSimSetupData = { + val jars = ExtSimLoader.scanInputFolder() + + val extLinks = jars.flatMap(ExtSimLoader.loadExtLink).toSeq + + if (extLinks.nonEmpty) { + val extScheduler = scheduler(context, parent = rootScheduler) + + val (extSimAdapters, extDatasAndServices) = + 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) + + // 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, extDataServiceToRef): ( + Iterable[ExtData], + Iterable[(Class[_], 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)) + + case (extPrimaryDataSimulation: ExtPrimaryDataSimulation, dIndex) => + val extPrimaryDataService = context.toClassic.simonaActorOf( + ExtPrimaryDataService.props(extScheduler.toClassic), + s"$index-$dIndex", + ) + val extPrimaryData = new ExtPrimaryData( + extPrimaryDataService, + extSimAdapter, + extPrimaryDataSimulation.getPrimaryDataFactory, + extPrimaryDataSimulation.getPrimaryDataAssets + ) + + extPrimaryDataSimulation.setExtPrimaryData(extPrimaryData) + + extPrimaryDataService ! SimonaService.Create( + InitExtPrimaryData(extPrimaryData), + ScheduleLock.singleKey( + context, + extScheduler, + INIT_SIM_TICK, + ), + ) + + (extPrimaryData, (classOf[ExtPrimaryDataService], extPrimaryDataService)) + + case (extResultDataSimulation: ExtResultDataSimulation, dIndex) => + + val extResultDataProvider = { + context.spawn( + ExtResultDataProvider(extScheduler), + s"$index-$dIndex", + ) + } + + implicit val timeout: PekkoTimeout = PekkoTimeout.create(5.seconds.toJava) + implicit val scheduler: Scheduler = context.system.scheduler + + val adapterRef = Await.result( + extResultDataProvider.ask[ActorRef[ResultDataMessageFromExt]] (ref => RequestDataMessageAdapter(ref)), timeout.duration) + val adapterScheduleRef = Await.result( + extResultDataProvider.ask[ActorRef[ScheduleServiceActivation]] (ref => RequestScheduleActivationAdapter(ref)), timeout.duration) + + val extResultData = + new ExtResultData( + adapterRef.toClassic, + adapterScheduleRef.toClassic, + extSimAdapter, + extResultDataSimulation.getResultDataFactory, + extResultDataSimulation.getResultDataAssets + ) + + extResultDataSimulation.setExtResultData(extResultData) + + extResultDataProvider ! ExtResultDataProvider.Create( + InitExtResultData(extResultData), + ScheduleLock.singleKey( + context, + extScheduler, + INIT_SIM_TICK, + ), + ) + + ( + extResultData, + (ExtResultDataProvider.getClass, extResultDataProvider.toClassic), + ) + }.unzip + + extLink.getExtSimulation.setup( + extSimAdapterData, + extData.toList.asJava, + ) + + // starting external simulation + new Thread(extLink.getExtSimulation, s"External simulation $index") + .start() + + (extSimAdapter, (extDataServiceToRef, extData)) + }.unzip + + val extDataServices = extDatasAndServices.map(_._1) + val extDatas = extDatasAndServices.flatMap(_._2).toSet + + ExtSimSetupData( + extSimAdapters, + extDataServices.flatten.toMap, + extDatas, + Some(extScheduler)) + } else { + ExtSimSetupData(Iterable.empty, Map.empty, Set.empty, None) + } + } + + */ + + override def extSimulations( + context: ActorContext[_], + rootScheduler: ActorRef[SchedulerMessage], + simScheduler: ActorRef[SchedulerMessage], + ): ExtSimSetupData = { + opsimSimulationWithEm( + context, rootScheduler, simScheduler + ) + } + + def opsimSimulationWithEm( + context: ActorContext[_], + rootScheduler: ActorRef[SchedulerMessage], + simScheduler: ActorRef[SchedulerMessage], + ): ExtSimSetupData = { + val simulationStart = TimeUtil.withDefaults.toZonedDateTime( + simonaConfig.simona.time.startDateTime + ) + val extScheduler = scheduler(context, parent = rootScheduler) + val opsimSim = new OpsimEmSimulator("amqp://guest:guest@localhost:5672/myvhost") + + val extSimAdapterPhase1 = context.toClassic.simonaActorOf( + ExtSimAdapter.props(extScheduler.toClassic), + s"1", + ) + val extSimAdapterPhase2 = context.toClassic.simonaActorOf( + ExtSimAdapter.props(simScheduler.toClassic), + s"2", + ) + + val extSimAdapters: Map[java.lang.Integer, ClassicRef] = Map( + 1.asInstanceOf[java.lang.Integer] -> extSimAdapterPhase1, + 2.asInstanceOf[java.lang.Integer] -> extSimAdapterPhase2 + ) + + val extSimAdapterData = new ExtSimAdapterData(extSimAdapters.asJava, args) + + + + // send init data right away, init activation is scheduled + extSimAdapterPhase1 ! ExtSimAdapter.Create( + extSimAdapterData, + 1, + ScheduleLock.singleKey(context, extScheduler, INIT_SIM_TICK), + ) + + val extEmDataService = context.toClassic.simonaActorOf( + ExtEmDataService.props(extScheduler.toClassic), + s"0-0", + ) + val extEmData = new ExtEmData( + extEmDataService, + extSimAdapterPhase1, + opsimSim.getExtEmDataSimulation.getEmDataFactory, + opsimSim.getExtEmDataSimulation.getControlledEms + ) + + opsimSim.getExtEmDataSimulation.setExtEmData(extEmData) + + extEmDataService ! SimonaService.Create( + InitExtEmData(extEmData), + ScheduleLock.singleKey( + context, + extScheduler, + INIT_SIM_TICK, + ), + ) + + //Result Data + + val extResultDataProvider = { + context.spawn( + ExtResultDataProvider(simScheduler), + s"ExtResultDataProvider", + ) + } + + + val timeout: PekkoTimeout = PekkoTimeout.create(5.seconds.toJava) + val scheduler2: Scheduler = context.system.scheduler + + val adapterRef = Await.result( + extResultDataProvider.ask[ActorRef[ResultDataMessageFromExt]] (ref => RequestDataMessageAdapter(ref))(timeout, scheduler2), timeout.duration) + val adapterScheduleRef = Await.result( + extResultDataProvider.ask[ActorRef[ScheduleServiceActivation]] (ref => RequestScheduleActivationAdapter(ref))(timeout, scheduler2), timeout.duration) + + val extResultData = + new ExtResultData( + adapterRef.toClassic, + adapterScheduleRef.toClassic, + extSimAdapterPhase2, + opsimSim.getExtResultDataSimulation.getResultDataFactory, + opsimSim.getExtResultDataSimulation.getResultDataAssets, + simulationStart + ) + + opsimSim.getExtResultDataSimulation.setExtResultData(extResultData) + + extResultDataProvider ! ExtResultDataProvider.Create( + InitExtResultData(extResultData), + ScheduleLock.singleKey( + context, + simScheduler, + INIT_SIM_TICK, + ), + ) + + + val simpleExtSimDatas: List[ExtData] = List( + extResultData, + extEmData + ) + + opsimSim.setup( + extSimAdapterData, + simpleExtSimDatas.asJava, + ) + // starting external simulation + new Thread(opsimSim, s"External simulation") + .start() + + val extDataServicesMap: Map[Class[_], ClassicRef] = Map( + classOf[ExtEmDataService] -> extEmDataService, + ) + + val extDataListenerMap: Map[Class[_], ActorRef[ExtResultDataProvider.Request]] = Map( + ExtResultDataProvider.getClass -> extResultDataProvider + ) + + val extSimAdaptersIt = Iterable(extSimAdapterPhase1, extSimAdapterPhase2) + + val extDatas = simpleExtSimDatas.toSet + extSimAdapterPhase2 ! ExtSimAdapter.Create( + extSimAdapterData, + 2, + ScheduleLock.singleKey(context, simScheduler, INIT_SIM_TICK), + ) + ExtSimSetupData(extSimAdaptersIt, extDataServicesMap, extDataListenerMap, extDatas, Some(extScheduler)) + } + + + def simpleExtSimulation( + context: ActorContext[_], + rootScheduler: ActorRef[SchedulerMessage], + simScheduler: ActorRef[SchedulerMessage], + ): ExtSimSetupData = { + val simulationStart = TimeUtil.withDefaults.toZonedDateTime( + simonaConfig.simona.time.startDateTime + ) + val extScheduler = scheduler(context, parent = rootScheduler) + val simpleExtSim = new SimpleExtSimulation() + + val extSimAdapterPhase1 = context.toClassic.simonaActorOf( + ExtSimAdapter.props(extScheduler.toClassic), + s"1", + ) + val extSimAdapterPhase2 = context.toClassic.simonaActorOf( + ExtSimAdapter.props(simScheduler.toClassic), + s"2", + ) + + val extSimAdapters: Map[java.lang.Integer, ClassicRef] = Map( + 1.asInstanceOf[java.lang.Integer] -> extSimAdapterPhase1, + 2.asInstanceOf[java.lang.Integer] -> extSimAdapterPhase2 + ) + + val extSimAdapterData = new ExtSimAdapterData(extSimAdapters.asJava, args) + + + + // send init data right away, init activation is scheduled + extSimAdapterPhase1 ! ExtSimAdapter.Create( + extSimAdapterData, + 1, + ScheduleLock.singleKey(context, extScheduler, INIT_SIM_TICK), + ) + + val extPrimaryDataService = context.toClassic.simonaActorOf( + ExtPrimaryDataService.props(extScheduler.toClassic), + s"0-0", + ) + val extPrimaryData = new ExtPrimaryData( + extPrimaryDataService, + extSimAdapterPhase1, + simpleExtSim.getExtPrimaryDataSimulation.getPrimaryDataFactory, + simpleExtSim.getExtPrimaryDataSimulation.getPrimaryDataAssets + ) + + simpleExtSim.getExtPrimaryDataSimulation.setExtPrimaryData(extPrimaryData) + + extPrimaryDataService ! SimonaService.Create( + InitExtPrimaryData(extPrimaryData), + ScheduleLock.singleKey( + context, + extScheduler, + INIT_SIM_TICK, + ), + ) + + //Result Data + + val extResultDataProvider = { + context.spawn( + ExtResultDataProvider(simScheduler), + s"ExtResultDataProvider", + ) + } + + + val timeout: PekkoTimeout = PekkoTimeout.create(5.seconds.toJava) + val scheduler2: Scheduler = context.system.scheduler + + val adapterRef = Await.result( + extResultDataProvider.ask[ActorRef[ResultDataMessageFromExt]] (ref => RequestDataMessageAdapter(ref))(timeout, scheduler2), timeout.duration) + val adapterScheduleRef = Await.result( + extResultDataProvider.ask[ActorRef[ScheduleServiceActivation]] (ref => RequestScheduleActivationAdapter(ref))(timeout, scheduler2), timeout.duration) + + val extResultData = + new ExtResultData( + adapterRef.toClassic, + adapterScheduleRef.toClassic, + extSimAdapterPhase2, + simpleExtSim.getExtResultDataSimulation.getResultDataFactory, + simpleExtSim.getExtResultDataSimulation.getResultDataAssets, + simulationStart + ) + + simpleExtSim.getExtResultDataSimulation.setExtResultData(extResultData) + + extResultDataProvider ! ExtResultDataProvider.Create( + InitExtResultData(extResultData), + ScheduleLock.singleKey( + context, + simScheduler, + INIT_SIM_TICK, + ), + ) + + + val simpleExtSimDatas: List[ExtData] = List( + extResultData, + extPrimaryData + ) + + simpleExtSim.setup( + extSimAdapterData, + simpleExtSimDatas.asJava, + ) + // starting external simulation + new Thread(simpleExtSim, s"External simulation") + .start() + + val extDataServicesMap: Map[Class[_], ClassicRef] = Map( + classOf[ExtPrimaryDataService] -> extPrimaryDataService, + ) + + val extDataListenerMap: Map[Class[_], ActorRef[ExtResultDataProvider.Request]] = Map( + ExtResultDataProvider.getClass -> extResultDataProvider + ) + + val extSimAdaptersIt = Iterable(extSimAdapterPhase1, extSimAdapterPhase2) + + val extDatas = simpleExtSimDatas.toSet + extSimAdapterPhase2 ! ExtSimAdapter.Create( + extSimAdapterData, + 2, + ScheduleLock.singleKey(context, simScheduler, INIT_SIM_TICK), + ) + ExtSimSetupData(extSimAdaptersIt, extDataServicesMap, extDataListenerMap, extDatas, Some(extScheduler)) + } + + override def timeAdvancer( + context: ActorContext[_], + simulation: ActorRef[SimonaSim.SimulationEnded.type], + runtimeEventListener: ActorRef[RuntimeEvent], + ): ActorRef[TimeAdvancer.Request] = { + val startDateTime = TimeUtil.withDefaults.toZonedDateTime( + simonaConfig.simona.time.startDateTime + ) + val endDateTime = TimeUtil.withDefaults.toZonedDateTime( + simonaConfig.simona.time.endDateTime + ) + + context.spawn( + TimeAdvancer( + simulation, + Some(runtimeEventListener), + simonaConfig.simona.time.schedulerReadyCheckWindow, + endDateTime.toTick(startDateTime), + ), + TimeAdvancer.getClass.getSimpleName, + ) + } + + override def scheduler( + context: ActorContext[_], + parent: ActorRef[SchedulerMessage], + coreFactory: CoreFactory = RegularSchedulerCore, + ): ActorRef[SchedulerMessage] = + context + .spawn( + Scheduler(parent, coreFactory), + s"${Scheduler.getClass.getSimpleName}_${coreFactory}_${UUID.randomUUID()}", + ) + + override def runtimeEventListener( + context: ActorContext[_] + ): ActorRef[RuntimeEventListener.Request] = + context + .spawn( + RuntimeEventListener( + simonaConfig.simona.runtime.listener, + runtimeEventQueue, + startDateTimeString = simonaConfig.simona.time.startDateTime, + ), + RuntimeEventListener.getClass.getSimpleName, + ) + + override def resultEventListener( + context: ActorContext[_], + extSimulationData: ExtSimSetupData, + ): Seq[ActorRef[ResultEventListener.Request]] = { + val extResultDataService: Option[ActorRef[ExtResultDataProvider.Request]] = + extSimulationData.extResultDataService + // append ResultEventListener as well to write raw output files + ArgsParser + .parseListenerConfigOption(simonaConfig.simona.event.listener) + .zipWithIndex + .map { case ((listenerCompanion, events), index) => + context.toClassic + .simonaActorOf( + listenerCompanion.props(events), + index.toString, + ) + .toTyped + } + .toSeq :+ context + .spawn( + ResultEventListener( + resultFileHierarchy, + extResultDataService, + ), + ResultEventListener.getClass.getSimpleName, + ) + } + + def buildSubGridToActorRefMap( + subGridTopologyGraph: SubGridTopologyGraph, + context: ActorContext[_], + environmentRefs: EnvironmentRefs, + resultEventListeners: Seq[ActorRef[ResultEvent]], + ): Map[Int, ActorRef[GridAgentMessage]] = { + subGridTopologyGraph + .vertexSet() + .asScala + .map(subGridContainer => { + val gridAgentRef = + context.spawn( + GridAgent( + environmentRefs, + simonaConfig, + resultEventListeners, + ), + subGridContainer.getSubnet.toString, + ) + subGridContainer.getSubnet -> gridAgentRef + }) + .toMap + } + + /** Get all thermal grids, that apply for the given grid container + * @param grid + * The grid container to assess + * @param thermalGridByBus + * Mapping from thermal bus to thermal grid + * @return + * A sequence of applicable thermal grids + */ + private def getThermalGrids( + grid: GridContainer, + thermalGridByBus: Map[ThermalBusInput, ThermalGrid], + ): Seq[ThermalGrid] = { + grid.getSystemParticipants.getHeatPumps.asScala + .flatten(hpInput => thermalGridByBus.get(hpInput.getThermalBus)) + .toSeq + } +} + +/** Companion object to provide [[SetupHelper]] methods for + * [[SimonaStandaloneSetup]] + */ +object SimonaOpsimSetup extends LazyLogging with SetupHelper { + def apply( + typeSafeConfig: Config, + resultFileHierarchy: ResultFileHierarchy, + runtimeEventQueue: Option[LinkedBlockingQueue[RuntimeEvent]] = None, + mainArgs: Array[String] = Array.empty[String], + ): SimonaOpsimSetup = + new SimonaOpsimSetup( + typeSafeConfig, + SimonaConfig(typeSafeConfig), + resultFileHierarchy, + runtimeEventQueue, + mainArgs, + ) +} diff --git a/src/main/scala/edu/ie3/simona/sim/setup/SimonaSimpleExtSimulationSetup.scala b/src/main/scala/edu/ie3/simona/sim/setup/SimonaSimpleExtSimulationSetup.scala new file mode 100644 index 0000000000..e02c601d40 --- /dev/null +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaSimpleExtSimulationSetup.scala @@ -0,0 +1,756 @@ +/* + * © 2020. TU Dortmund University, + * Institute of Energy Systems, Energy Efficiency and Energy Economics, + * Research group Distribution grid planning and operation + */ + +package edu.ie3.simona.sim.setup + +import com.typesafe.config.Config +import com.typesafe.scalalogging.LazyLogging +import edu.ie3.datamodel.graph.SubGridTopologyGraph +import edu.ie3.datamodel.models.input.container.{GridContainer, ThermalGrid} +import edu.ie3.datamodel.models.input.thermal.ThermalBusInput +import edu.ie3.simona.actor.SimonaActorNaming.RichActorRefFactory +import edu.ie3.simona.agent.EnvironmentRefs +import edu.ie3.simona.agent.grid.GridAgentMessage.CreateGridAgent +import edu.ie3.simona.agent.grid.{GridAgent, GridAgentMessage} +import edu.ie3.simona.api.ExtSimAdapter +import edu.ie3.simona.api.data.ExtData +import edu.ie3.simona.api.data.em.ExtEmData +import edu.ie3.simona.api.data.primarydata.ExtPrimaryData +import edu.ie3.simona.api.data.results.ExtResultData +import edu.ie3.simona.api.data.results.ontology.ResultDataMessageFromExt +import edu.ie3.simona.api.simulation.ExtSimAdapterData +import edu.ie3.simona.config.{ArgsParser, RefSystemParser, SimonaConfig} +import edu.ie3.simona.event.listener.{ResultEventListener, RuntimeEventListener} +import edu.ie3.simona.event.{ResultEvent, RuntimeEvent} +import edu.ie3.simona.exceptions.agent.GridAgentInitializationException +import edu.ie3.simona.io.grid.GridProvider +import edu.ie3.simona.ontology.messages.SchedulerMessage +import edu.ie3.simona.ontology.messages.SchedulerMessage.ScheduleActivation +import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.ScheduleServiceActivation +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.em.ExtEmDataService +import edu.ie3.simona.service.em.ExtEmDataService.InitExtEmData +import edu.ie3.simona.service.primary.ExtPrimaryDataService.InitExtPrimaryData +import edu.ie3.simona.service.primary.PrimaryServiceProxy.InitPrimaryServiceProxyStateData +import edu.ie3.simona.service.primary.{ExtPrimaryDataService, PrimaryServiceProxy} +import edu.ie3.simona.service.results.ExtResultDataProvider +import edu.ie3.simona.service.results.ExtResultDataProvider.{InitExtResultData, RequestDataMessageAdapter, RequestScheduleActivationAdapter} +import edu.ie3.simona.service.weather.WeatherService +import edu.ie3.simona.service.weather.WeatherService.InitWeatherServiceStateData +import edu.ie3.simona.sim.SimonaSim +import edu.ie3.simona.util.ResultFileHierarchy +import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK +import edu.ie3.simona.util.TickUtil.RichZonedDateTime +import edu.ie3.simopsim.OpsimSimulator +import edu.ie3.simpleextsim.{SimpleExtSimulation, SimpleExtSimulationWithEm} +import edu.ie3.util.TimeUtil +import org.apache.pekko.actor.typed.scaladsl.ActorContext +import org.apache.pekko.actor.typed.scaladsl.AskPattern._ +import org.apache.pekko.actor.typed.scaladsl.adapter.{ClassicActorRefOps, TypedActorContextOps, TypedActorRefOps} +import org.apache.pekko.actor.typed.{ActorRef, Scheduler} +import org.apache.pekko.actor.{ActorRef => ClassicRef} +import org.apache.pekko.util.{Timeout => PekkoTimeout} + +import java.util.UUID +import java.util.concurrent.LinkedBlockingQueue +import scala.concurrent.Await +import scala.concurrent.duration.DurationInt +import scala.jdk.CollectionConverters._ +import scala.jdk.DurationConverters._ + +/** Sample implementation to run a standalone simulation of simona configured + * with the provided [[SimonaConfig]] and [[ResultFileHierarchy]] + * + * @version 0.1 + * @since 01.07.20 + */ +class SimonaSimpleExtSimulationSetup( + val typeSafeConfig: Config, + simonaConfig: SimonaConfig, + resultFileHierarchy: ResultFileHierarchy, + runtimeEventQueue: Option[LinkedBlockingQueue[RuntimeEvent]] = None, + override val args: Array[String], +) extends SimonaSetup { + + override def gridAgents( + context: ActorContext[_], + environmentRefs: EnvironmentRefs, + resultEventListeners: Seq[ActorRef[ResultEvent]], + ): Iterable[ActorRef[GridAgentMessage]] = { + + /* get the grid */ + val subGridTopologyGraph = GridProvider + .gridFromConfig( + simonaConfig.simona.simulationName, + simonaConfig.simona.input.grid.datasource, + ) + .getSubGridTopologyGraph + val thermalGridsByThermalBus = GridProvider.getThermalGridsFromConfig( + simonaConfig.simona.input.grid.datasource + ) + + /* extract and prepare refSystem information from config */ + val configRefSystems = + RefSystemParser.parse(simonaConfig.simona.gridConfig.refSystems) + + /* Create all agents and map the sub grid id to their actor references */ + val subGridToActorRefMap = buildSubGridToActorRefMap( + subGridTopologyGraph, + context, + environmentRefs, + resultEventListeners, + ) + + val keys = ScheduleLock.multiKey( + context, + environmentRefs.scheduler, + INIT_SIM_TICK, + subGridTopologyGraph.vertexSet().size, + ) + + /* build the initialization data */ + subGridTopologyGraph + .vertexSet() + .asScala + .zip(keys) + .map { case (subGridContainer, key) => + /* Get all connections to superior and inferior sub grids */ + val subGridGates = + Set.from( + subGridTopologyGraph + .edgesOf(subGridContainer) + .asScala + .map(modifySubGridGateForThreeWindingSupport) + ) + val currentSubGrid = subGridContainer.getSubnet + val currentActorRef = subGridToActorRefMap.getOrElse( + currentSubGrid, + throw new GridAgentInitializationException( + "Was asked to setup agent for sub grid " + currentSubGrid + ", but did not found it's actor reference." + ), + ) + val thermalGrids = + getThermalGrids(subGridContainer, thermalGridsByThermalBus) + + /* build the grid agent data and check for its validity */ + val gridAgentInitData = SimonaStandaloneSetup.buildGridAgentInitData( + subGridContainer, + subGridToActorRefMap, + subGridGates, + configRefSystems, + thermalGrids, + ) + + currentActorRef ! CreateGridAgent(gridAgentInitData, key) + + currentActorRef + } + } + + override def primaryServiceProxy( + context: ActorContext[_], + scheduler: ActorRef[SchedulerMessage], + extSimSetupData: ExtSimSetupData, + ): ClassicRef = { + val simulationStart = TimeUtil.withDefaults.toZonedDateTime( + simonaConfig.simona.time.startDateTime + ) + val primaryServiceProxy = context.toClassic.simonaActorOf( + PrimaryServiceProxy.props( + scheduler.toClassic, + InitPrimaryServiceProxyStateData( + simonaConfig.simona.input.primary, + simulationStart, + extSimSetupData.extPrimaryDataService, + extSimSetupData.extPrimaryData + ), + simulationStart, + ) + ) + + scheduler ! ScheduleActivation(primaryServiceProxy.toTyped, INIT_SIM_TICK) + primaryServiceProxy + } + + override def weatherService( + context: ActorContext[_], + scheduler: ActorRef[SchedulerMessage], + ): ClassicRef = { + val weatherService = context.toClassic.simonaActorOf( + WeatherService.props( + scheduler.toClassic, + TimeUtil.withDefaults + .toZonedDateTime(simonaConfig.simona.time.startDateTime), + TimeUtil.withDefaults + .toZonedDateTime(simonaConfig.simona.time.endDateTime), + ) + ) + weatherService ! SimonaService.Create( + InitWeatherServiceStateData( + simonaConfig.simona.input.weather.datasource + ), + ScheduleLock.singleKey(context, scheduler, INIT_SIM_TICK), + ) + + weatherService + } + + /* + override def extSimulations( + context: ActorContext[_], + rootScheduler: ActorRef[SchedulerMessage], + ): ExtSimSetupData = { + val jars = ExtSimLoader.scanInputFolder() + + val extLinks = jars.flatMap(ExtSimLoader.loadExtLink).toSeq + + if (extLinks.nonEmpty) { + val extScheduler = scheduler(context, parent = rootScheduler) + + val (extSimAdapters, extDatasAndServices) = + 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) + + // 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, extDataServiceToRef): ( + Iterable[ExtData], + Iterable[(Class[_], 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)) + + case (extPrimaryDataSimulation: ExtPrimaryDataSimulation, dIndex) => + val extPrimaryDataService = context.toClassic.simonaActorOf( + ExtPrimaryDataService.props(extScheduler.toClassic), + s"$index-$dIndex", + ) + val extPrimaryData = new ExtPrimaryData( + extPrimaryDataService, + extSimAdapter, + extPrimaryDataSimulation.getPrimaryDataFactory, + extPrimaryDataSimulation.getPrimaryDataAssets + ) + + extPrimaryDataSimulation.setExtPrimaryData(extPrimaryData) + + extPrimaryDataService ! SimonaService.Create( + InitExtPrimaryData(extPrimaryData), + ScheduleLock.singleKey( + context, + extScheduler, + INIT_SIM_TICK, + ), + ) + + (extPrimaryData, (classOf[ExtPrimaryDataService], extPrimaryDataService)) + + case (extResultDataSimulation: ExtResultDataSimulation, dIndex) => + + val extResultDataProvider = { + context.spawn( + ExtResultDataProvider(extScheduler), + s"$index-$dIndex", + ) + } + + implicit val timeout: PekkoTimeout = PekkoTimeout.create(5.seconds.toJava) + implicit val scheduler: Scheduler = context.system.scheduler + + val adapterRef = Await.result( + extResultDataProvider.ask[ActorRef[ResultDataMessageFromExt]] (ref => RequestDataMessageAdapter(ref)), timeout.duration) + val adapterScheduleRef = Await.result( + extResultDataProvider.ask[ActorRef[ScheduleServiceActivation]] (ref => RequestScheduleActivationAdapter(ref)), timeout.duration) + + val extResultData = + new ExtResultData( + adapterRef.toClassic, + adapterScheduleRef.toClassic, + extSimAdapter, + extResultDataSimulation.getResultDataFactory, + extResultDataSimulation.getResultDataAssets + ) + + extResultDataSimulation.setExtResultData(extResultData) + + extResultDataProvider ! ExtResultDataProvider.Create( + InitExtResultData(extResultData), + ScheduleLock.singleKey( + context, + extScheduler, + INIT_SIM_TICK, + ), + ) + + ( + extResultData, + (ExtResultDataProvider.getClass, extResultDataProvider.toClassic), + ) + }.unzip + + extLink.getExtSimulation.setup( + extSimAdapterData, + extData.toList.asJava, + ) + + // starting external simulation + new Thread(extLink.getExtSimulation, s"External simulation $index") + .start() + + (extSimAdapter, (extDataServiceToRef, extData)) + }.unzip + + val extDataServices = extDatasAndServices.map(_._1) + val extDatas = extDatasAndServices.flatMap(_._2).toSet + + ExtSimSetupData( + extSimAdapters, + extDataServices.flatten.toMap, + extDatas, + Some(extScheduler)) + } else { + ExtSimSetupData(Iterable.empty, Map.empty, Set.empty, None) + } + } + + */ + + override def extSimulations( + context: ActorContext[_], + rootScheduler: ActorRef[SchedulerMessage], + simScheduler: ActorRef[SchedulerMessage], + ): ExtSimSetupData = { + simpleExtSimulationWithEm( + context, rootScheduler, simScheduler + ) + } + + def simpleExtSimulationWithEm( + context: ActorContext[_], + rootScheduler: ActorRef[SchedulerMessage], + simScheduler: ActorRef[SchedulerMessage], + ): ExtSimSetupData = { + val simulationStart = TimeUtil.withDefaults.toZonedDateTime( + simonaConfig.simona.time.startDateTime + ) + val extScheduler = scheduler(context, parent = rootScheduler) + val simpleExtSim = new SimpleExtSimulationWithEm() + + val extSimAdapterPhase1 = context.toClassic.simonaActorOf( + ExtSimAdapter.props(extScheduler.toClassic), + s"1", + ) + val extSimAdapterPhase2 = context.toClassic.simonaActorOf( + ExtSimAdapter.props(simScheduler.toClassic), + s"2", + ) + + val extSimAdapters: Map[java.lang.Integer, ClassicRef] = Map( + 1.asInstanceOf[java.lang.Integer] -> extSimAdapterPhase1, + 2.asInstanceOf[java.lang.Integer] -> extSimAdapterPhase2 + ) + + val extSimAdapterData = new ExtSimAdapterData(extSimAdapters.asJava, args) + + + + // send init data right away, init activation is scheduled + extSimAdapterPhase1 ! ExtSimAdapter.Create( + extSimAdapterData, + 1, + ScheduleLock.singleKey(context, extScheduler, INIT_SIM_TICK), + ) + + val extEmDataService = context.toClassic.simonaActorOf( + ExtEmDataService.props(extScheduler.toClassic), + s"0-0", + ) + val extEmData = new ExtEmData( + extEmDataService, + extSimAdapterPhase1, + simpleExtSim.getExtEmDataSimulation.getEmDataFactory, + simpleExtSim.getExtEmDataSimulation.getControlledEms + ) + + simpleExtSim.getExtEmDataSimulation.setExtEmData(extEmData) + + extEmDataService ! SimonaService.Create( + InitExtEmData(extEmData), + ScheduleLock.singleKey( + context, + extScheduler, + INIT_SIM_TICK, + ), + ) + + //Result Data + + val extResultDataProvider = { + context.spawn( + ExtResultDataProvider(simScheduler), + s"ExtResultDataProvider", + ) + } + + + val timeout: PekkoTimeout = PekkoTimeout.create(5.seconds.toJava) + val scheduler2: Scheduler = context.system.scheduler + + val adapterRef = Await.result( + extResultDataProvider.ask[ActorRef[ResultDataMessageFromExt]] (ref => RequestDataMessageAdapter(ref))(timeout, scheduler2), timeout.duration) + val adapterScheduleRef = Await.result( + extResultDataProvider.ask[ActorRef[ScheduleServiceActivation]] (ref => RequestScheduleActivationAdapter(ref))(timeout, scheduler2), timeout.duration) + + val extResultData = + new ExtResultData( + adapterRef.toClassic, + adapterScheduleRef.toClassic, + extSimAdapterPhase2, + simpleExtSim.getExtResultDataSimulation.getResultDataFactory, + simpleExtSim.getExtResultDataSimulation.getResultDataAssets, + simulationStart + ) + + simpleExtSim.getExtResultDataSimulation.setExtResultData(extResultData) + + extResultDataProvider ! ExtResultDataProvider.Create( + InitExtResultData(extResultData), + ScheduleLock.singleKey( + context, + simScheduler, + INIT_SIM_TICK, + ), + ) + + + val simpleExtSimDatas: List[ExtData] = List( + extResultData, + extEmData + ) + + simpleExtSim.setup( + extSimAdapterData, + simpleExtSimDatas.asJava, + ) + // starting external simulation + new Thread(simpleExtSim, s"External simulation") + .start() + + val extDataServicesMap: Map[Class[_], ClassicRef] = Map( + classOf[ExtEmDataService] -> extEmDataService, + ) + + val extDataListenerMap: Map[Class[_], ActorRef[ExtResultDataProvider.Request]] = Map( + ExtResultDataProvider.getClass -> extResultDataProvider + ) + + val extSimAdaptersIt = Iterable(extSimAdapterPhase1, extSimAdapterPhase2) + + val extDatas = simpleExtSimDatas.toSet + extSimAdapterPhase2 ! ExtSimAdapter.Create( + extSimAdapterData, + 2, + ScheduleLock.singleKey(context, simScheduler, INIT_SIM_TICK), + ) + ExtSimSetupData(extSimAdaptersIt, extDataServicesMap, extDataListenerMap, extDatas, Some(extScheduler)) + } + + + def simpleExtSimulation( + context: ActorContext[_], + rootScheduler: ActorRef[SchedulerMessage], + simScheduler: ActorRef[SchedulerMessage], + ): ExtSimSetupData = { + val simulationStart = TimeUtil.withDefaults.toZonedDateTime( + simonaConfig.simona.time.startDateTime + ) + val extScheduler = scheduler(context, parent = rootScheduler) + val simpleExtSim = new SimpleExtSimulation() + + val extSimAdapterPhase1 = context.toClassic.simonaActorOf( + ExtSimAdapter.props(extScheduler.toClassic), + s"1", + ) + val extSimAdapterPhase2 = context.toClassic.simonaActorOf( + ExtSimAdapter.props(simScheduler.toClassic), + s"2", + ) + + val extSimAdapters: Map[java.lang.Integer, ClassicRef] = Map( + 1.asInstanceOf[java.lang.Integer] -> extSimAdapterPhase1, + 2.asInstanceOf[java.lang.Integer] -> extSimAdapterPhase2 + ) + + val extSimAdapterData = new ExtSimAdapterData(extSimAdapters.asJava, args) + + + + // send init data right away, init activation is scheduled + extSimAdapterPhase1 ! ExtSimAdapter.Create( + extSimAdapterData, + 1, + ScheduleLock.singleKey(context, extScheduler, INIT_SIM_TICK), + ) + + val extPrimaryDataService = context.toClassic.simonaActorOf( + ExtPrimaryDataService.props(extScheduler.toClassic), + s"0-0", + ) + val extPrimaryData = new ExtPrimaryData( + extPrimaryDataService, + extSimAdapterPhase1, + simpleExtSim.getExtPrimaryDataSimulation.getPrimaryDataFactory, + simpleExtSim.getExtPrimaryDataSimulation.getPrimaryDataAssets + ) + + simpleExtSim.getExtPrimaryDataSimulation.setExtPrimaryData(extPrimaryData) + + extPrimaryDataService ! SimonaService.Create( + InitExtPrimaryData(extPrimaryData), + ScheduleLock.singleKey( + context, + extScheduler, + INIT_SIM_TICK, + ), + ) + + //Result Data + + val extResultDataProvider = { + context.spawn( + ExtResultDataProvider(simScheduler), + s"ExtResultDataProvider", + ) + } + + + val timeout: PekkoTimeout = PekkoTimeout.create(5.seconds.toJava) + val scheduler2: Scheduler = context.system.scheduler + + val adapterRef = Await.result( + extResultDataProvider.ask[ActorRef[ResultDataMessageFromExt]] (ref => RequestDataMessageAdapter(ref))(timeout, scheduler2), timeout.duration) + val adapterScheduleRef = Await.result( + extResultDataProvider.ask[ActorRef[ScheduleServiceActivation]] (ref => RequestScheduleActivationAdapter(ref))(timeout, scheduler2), timeout.duration) + + val extResultData = + new ExtResultData( + adapterRef.toClassic, + adapterScheduleRef.toClassic, + extSimAdapterPhase2, + simpleExtSim.getExtResultDataSimulation.getResultDataFactory, + simpleExtSim.getExtResultDataSimulation.getResultDataAssets, + simulationStart + ) + + simpleExtSim.getExtResultDataSimulation.setExtResultData(extResultData) + + extResultDataProvider ! ExtResultDataProvider.Create( + InitExtResultData(extResultData), + ScheduleLock.singleKey( + context, + simScheduler, + INIT_SIM_TICK, + ), + ) + + + val simpleExtSimDatas: List[ExtData] = List( + extResultData, + extPrimaryData + ) + + simpleExtSim.setup( + extSimAdapterData, + simpleExtSimDatas.asJava, + ) + // starting external simulation + new Thread(simpleExtSim, s"External simulation") + .start() + + val extDataServicesMap: Map[Class[_], ClassicRef] = Map( + classOf[ExtPrimaryDataService] -> extPrimaryDataService, + ) + + val extDataListenerMap: Map[Class[_], ActorRef[ExtResultDataProvider.Request]] = Map( + ExtResultDataProvider.getClass -> extResultDataProvider + ) + + val extSimAdaptersIt = Iterable(extSimAdapterPhase1, extSimAdapterPhase2) + + val extDatas = simpleExtSimDatas.toSet + extSimAdapterPhase2 ! ExtSimAdapter.Create( + extSimAdapterData, + 2, + ScheduleLock.singleKey(context, simScheduler, INIT_SIM_TICK), + ) + ExtSimSetupData(extSimAdaptersIt, extDataServicesMap, extDataListenerMap, extDatas, Some(extScheduler)) + } + + override def timeAdvancer( + context: ActorContext[_], + simulation: ActorRef[SimonaSim.SimulationEnded.type], + runtimeEventListener: ActorRef[RuntimeEvent], + ): ActorRef[TimeAdvancer.Request] = { + val startDateTime = TimeUtil.withDefaults.toZonedDateTime( + simonaConfig.simona.time.startDateTime + ) + val endDateTime = TimeUtil.withDefaults.toZonedDateTime( + simonaConfig.simona.time.endDateTime + ) + + context.spawn( + TimeAdvancer( + simulation, + Some(runtimeEventListener), + simonaConfig.simona.time.schedulerReadyCheckWindow, + endDateTime.toTick(startDateTime), + ), + TimeAdvancer.getClass.getSimpleName, + ) + } + + override def scheduler( + context: ActorContext[_], + parent: ActorRef[SchedulerMessage], + coreFactory: CoreFactory = RegularSchedulerCore, + ): ActorRef[SchedulerMessage] = + context + .spawn( + Scheduler(parent, coreFactory), + s"${Scheduler.getClass.getSimpleName}_${coreFactory}_${UUID.randomUUID()}", + ) + + override def runtimeEventListener( + context: ActorContext[_] + ): ActorRef[RuntimeEventListener.Request] = + context + .spawn( + RuntimeEventListener( + simonaConfig.simona.runtime.listener, + runtimeEventQueue, + startDateTimeString = simonaConfig.simona.time.startDateTime, + ), + RuntimeEventListener.getClass.getSimpleName, + ) + + override def resultEventListener( + context: ActorContext[_], + extSimulationData: ExtSimSetupData, + ): Seq[ActorRef[ResultEventListener.Request]] = { + val extResultDataService: Option[ActorRef[ExtResultDataProvider.Request]] = + extSimulationData.extResultDataService + // append ResultEventListener as well to write raw output files + ArgsParser + .parseListenerConfigOption(simonaConfig.simona.event.listener) + .zipWithIndex + .map { case ((listenerCompanion, events), index) => + context.toClassic + .simonaActorOf( + listenerCompanion.props(events), + index.toString, + ) + .toTyped + } + .toSeq :+ context + .spawn( + ResultEventListener( + resultFileHierarchy, + extResultDataService, + ), + ResultEventListener.getClass.getSimpleName, + ) + } + + def buildSubGridToActorRefMap( + subGridTopologyGraph: SubGridTopologyGraph, + context: ActorContext[_], + environmentRefs: EnvironmentRefs, + resultEventListeners: Seq[ActorRef[ResultEvent]], + ): Map[Int, ActorRef[GridAgentMessage]] = { + subGridTopologyGraph + .vertexSet() + .asScala + .map(subGridContainer => { + val gridAgentRef = + context.spawn( + GridAgent( + environmentRefs, + simonaConfig, + resultEventListeners, + ), + subGridContainer.getSubnet.toString, + ) + subGridContainer.getSubnet -> gridAgentRef + }) + .toMap + } + + /** Get all thermal grids, that apply for the given grid container + * @param grid + * The grid container to assess + * @param thermalGridByBus + * Mapping from thermal bus to thermal grid + * @return + * A sequence of applicable thermal grids + */ + private def getThermalGrids( + grid: GridContainer, + thermalGridByBus: Map[ThermalBusInput, ThermalGrid], + ): Seq[ThermalGrid] = { + grid.getSystemParticipants.getHeatPumps.asScala + .flatten(hpInput => thermalGridByBus.get(hpInput.getThermalBus)) + .toSeq + } +} + +/** Companion object to provide [[SetupHelper]] methods for + * [[SimonaStandaloneSetup]] + */ +object SimonaSimpleExtSimulationSetup extends LazyLogging with SetupHelper { + + def apply( + typeSafeConfig: Config, + resultFileHierarchy: ResultFileHierarchy, + runtimeEventQueue: Option[LinkedBlockingQueue[RuntimeEvent]] = None, + mainArgs: Array[String] = Array.empty[String], + ): SimonaSimpleExtSimulationSetup = + new SimonaSimpleExtSimulationSetup( + typeSafeConfig, + SimonaConfig(typeSafeConfig), + resultFileHierarchy, + runtimeEventQueue, + mainArgs, + ) +} From 8da5deff9ae5003cd87cbb0ee0ebccb60f1dc9e3 Mon Sep 17 00:00:00 2001 From: Johannes Bao Date: Tue, 21 May 2024 16:26:45 +0200 Subject: [PATCH 28/41] temp --- .../fullGrid/em_input.csv | 2 +- .../simopsimtestgrid.conf | 6 ++-- .../edu/ie3/simona/agent/em/EmAgent.scala | 32 +++++++++---------- .../ie3/simona/model/em/EmModelShell.scala | 1 - .../simona/service/em/ExtEmDataService.scala | 5 +-- .../results/ExtResultDataProvider.scala | 3 ++ 6 files changed, 24 insertions(+), 25 deletions(-) diff --git a/input/samples/simopsimtestgrid_reduced/fullGrid/em_input.csv b/input/samples/simopsimtestgrid_reduced/fullGrid/em_input.csv index b6a794fe65..ff0cc8720c 100755 --- a/input/samples/simopsimtestgrid_reduced/fullGrid/em_input.csv +++ b/input/samples/simopsimtestgrid_reduced/fullGrid/em_input.csv @@ -1,4 +1,4 @@ uuid;control_strategy;parent_em;id;operates_from;operates_until;operator -c3a7e9f5-b492-4c85-af2d-1e93f6a25443;self_optimization;;EM_HH_Bus_25;;; +c3a7e9f5-b492-4c85-af2d-1e93f6a25443;kein_bock;;EM_HH_Bus_25;;; f9dc7ce6-658c-4101-a12f-d58bb889286b;self_optimization;;EM_HH_Bus_81;;; 957938b7-0476-4fab-a1b3-6ce8615857b3;self_optimization;;EM_HH_Bus_110;;; diff --git a/input/samples/simopsimtestgrid_reduced/simopsimtestgrid.conf b/input/samples/simopsimtestgrid_reduced/simopsimtestgrid.conf index d685a161e6..6144e50874 100755 --- a/input/samples/simopsimtestgrid_reduced/simopsimtestgrid.conf +++ b/input/samples/simopsimtestgrid_reduced/simopsimtestgrid.conf @@ -15,8 +15,8 @@ simona.simulationName = "simopsimtest" ################################################################## # Time Parameters ################################################################## -simona.time.startDateTime = "2024-02-27T00:00:00Z" -simona.time.endDateTime = "2024-02-27T23:45:00Z" +simona.time.startDateTime = "2023-07-27T00:00:00Z" +simona.time.endDateTime = "2023-07-28T00:15:00Z" simona.time.schedulerReadyCheckWindow = 900 ################################################################## @@ -137,7 +137,7 @@ simona.runtime.participant.load = { calculateMissingReactivePowerWithModel = false uuids = ["default"] scaling = 1.0 - modelBehaviour = "fix" + modelBehaviour = "profile" reference = "power" } individualConfigs = [] diff --git a/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala b/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala index 2558fc0b02..0e53918488 100644 --- a/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala +++ b/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala @@ -146,6 +146,7 @@ object EmAgent { modelConfig, ) + ctx.log.info(s"EMAgent ${modelShell.uuid} with $modelShell") ctx.log.info(s"EMAgent ${modelShell.uuid} goes to inactive!") inactive( @@ -165,7 +166,7 @@ object EmAgent { ): Behavior[Request] = Behaviors.receivePartial { case (ctx, RegisterParticipant(model, actor, spi)) => - ctx.log.info(s"EM Agent ${modelShell.uuid} RegisterParticipant model $model") + ctx.log.debug(s"EM Agent ${modelShell.uuid} RegisterParticipant model $model") val updatedModelShell = modelShell.addParticipant(model, spi) val updatedCore = core.addParticipant(actor, model) inactive(emData, updatedModelShell, updatedCore) @@ -182,13 +183,13 @@ object EmAgent { case (ctx, ScheduleFlexRequest(participant, newTick, scheduleKey)) => - ctx.log.info(s"EM Agent ${modelShell.uuid} got ScheduleFlexRequest!") + ctx.log.debug(s"EM Agent ${modelShell.uuid} got ScheduleFlexRequest!") val (maybeSchedule, newCore) = core .handleSchedule(participant, newTick) maybeSchedule match { case Some(scheduleTick) => - ctx.log.info(s"EM Agent ${modelShell.uuid} -> parentData = ${emData.parentData}") + ctx.log.debug(s"EM Agent ${modelShell.uuid} -> parentData = ${emData.parentData}") // also potentially schedule with parent if the new earliest tick is // different from the old earliest tick (including if nothing had // been scheduled before) @@ -215,17 +216,16 @@ object EmAgent { inactive(emData, modelShell, newCore) case (ctx, msg: ActivationRequest) => - ctx.log.info(s"EM Agent ${modelShell.uuid} got ActivationRequest = $msg") + ctx.log.debug(s"EM Agent ${modelShell.uuid} got ActivationRequest = $msg") val flexOptionsCore = core.activate(msg.tick) msg match { case Flex(_: RequestFlexOptions) | EmActivation(_) => - ctx.log.info(s"Activation for tick ${msg.tick}") + ctx.log.debug(s"Activation for tick ${msg.tick}") val (toActivate, newCore) = flexOptionsCore.takeNewFlexRequests() toActivate.foreach { _ ! RequestFlexOptions(msg.tick) } - ctx.log.info(s"toActivate $toActivate") awaitingFlexOptions(emData, modelShell, newCore) @@ -241,7 +241,7 @@ object EmAgent { awaitingFlexCtrl(emData, modelShell, flexOptionsCore) case Flex(dataMsg: ProvideExtEmSetPoint) => // got set point before activation -> put msg in queue and wait - ctx.log.info(s"Agent ${ctx.self} got external set point = $dataMsg") + ctx.log.debug(s"Agent ${ctx.self} got external set point = $dataMsg") val updatedEmData = emData.copy( extEmDataServiceData = emData.extEmDataServiceData.copy( dataProvisionMessage = Some(dataMsg) @@ -266,7 +266,7 @@ object EmAgent { val allFlexOptions = updatedCore.getFlexOptions - ctx.log.info(s"EM Agent ${ctx.self} allFlexOptions = $allFlexOptions") + ctx.log.debug(s"EM Agent ${ctx.self} allFlexOptions = $allFlexOptions") emData.parentData match { case Right(flexStateData) => @@ -331,10 +331,10 @@ object EmAgent { ) } } - ctx.log.info(s"[UNCONTROLLED] EM Agent ${ctx.self}: Starting determination of flex control with set power = $setPower") + ctx.log.debug(s"[UNCONTROLLED] EM Agent ${ctx.self}: Starting determination of flex control with set power = $setPower") val flexControl = modelShell.determineFlexControl(allFlexOptions, setPower) - ctx.log.info(s"[UNCONTROLLED] EM Agent ${ctx.self}: Got flexControl = $flexControl") + ctx.log.debug(s"[UNCONTROLLED] EM Agent ${ctx.self}: Got flexControl = $flexControl") val (allFlexMsgs, newCore) = updatedCore @@ -343,7 +343,7 @@ object EmAgent { .complete() allFlexMsgs.foreach { case (actor, msg) => - ctx.log.info(s"[UNCONTROLLED] EM Agent ${ctx.self}: For actor = $actor send msg = $msg") + ctx.log.debug(s"[UNCONTROLLED] EM Agent ${ctx.self}: For actor = $actor send msg = $msg") actor ! msg } @@ -359,7 +359,7 @@ object EmAgent { } case (ctx, Flex(dataMsg: ProvideExtEmSetPoint)) => // got set point before activation -> put msg in queue and wait - ctx.log.info(s"Agent ${ctx.self} got external set point = $dataMsg") + ctx.log.debug(s"Agent ${ctx.self} got external set point = $dataMsg") val updatedEmData = emData.copy( extEmDataServiceData = emData.extEmDataServiceData.copy( dataProvisionMessage = Some(dataMsg) @@ -382,9 +382,9 @@ object EmAgent { flexOptionsCore: EmDataCore.AwaitingFlexOptions, ): Behavior[Request] = Behaviors.receivePartial { case (ctx, Flex(flexCtrl: IssueFlexControl)) => - ctx.log.info(s"emData = $emData") - ctx.log.info(s"modelShell = $modelShell") - ctx.log.info(s"agent ${ctx.self}: flexCtrl = $flexCtrl") + ctx.log.debug(s"emData = $emData") + ctx.log.debug(s"modelShell = $modelShell") + ctx.log.debug(s"agent ${ctx.self}: flexCtrl = $flexCtrl") val flexData = emData.parentData.getOrElse( throw new CriticalFailureException(s"EmAgent is not EM-controlled.") ) @@ -416,7 +416,7 @@ object EmAgent { .complete() allFlexMsgs.foreach { case (actor, msg) => - ctx.log.info(s"Agent ${ctx.self}: For actor = $actor send msg = $msg") + ctx.log.debug(s"Agent ${ctx.self}: For actor = $actor send msg = $msg") actor ! msg } diff --git a/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala b/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala index 6a15d08624..a96cea57de 100644 --- a/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala +++ b/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala @@ -106,7 +106,6 @@ final case class EmModelShell( model -> power } } - } object EmModelShell { diff --git a/src/main/scala/edu/ie3/simona/service/em/ExtEmDataService.scala b/src/main/scala/edu/ie3/simona/service/em/ExtEmDataService.scala index e00ce57875..5de090fdb7 100644 --- a/src/main/scala/edu/ie3/simona/service/em/ExtEmDataService.scala +++ b/src/main/scala/edu/ie3/simona/service/em/ExtEmDataService.scala @@ -186,11 +186,9 @@ final case class ExtEmDataService( None } } - log.info(s"Received ActorToEmData = $actorToEmData") + log.debug(s"Received ActorToEmData = $actorToEmData") if (actorToEmData.nonEmpty) { - log.info("ProvideEmData") - actorToEmData.foreach { case (actor, setPoint) => actor ! ProvideExtEmSetPoint( tick, @@ -219,7 +217,6 @@ final case class ExtEmDataService( override protected def handleDataMessage(extMsg: DataMessageFromExt)(implicit serviceStateData: ExtEmDataStateData): ExtEmDataStateData = { extMsg match { case extEmDataMessage: EmDataMessageFromExt => - log.info("Received EmDataMessageFromExt") serviceStateData.copy( extEmDataMessage = Some(extEmDataMessage) ) diff --git a/src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala b/src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala index 0907901873..53bb68c8cf 100644 --- a/src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala +++ b/src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala @@ -140,6 +140,7 @@ object ExtResultDataProvider { //ctx.log.info(s"[requestResults] for tick ${msg.tick} and resultStorage ${serviceStateData.resultStorage}") var receiveDataMap = ReceiveDataMap[UUID, ResultEntity](serviceStateData.subscribers.toSet) //ctx.log.info(s"[requestResults] tick ${msg.tick} -> created a receivedatamap " + receiveDataMap) + /* serviceStateData.resultStorage.foreach({ case (uuid, (res, t)) => //ctx.log.info(s"[requestResults] tick = ${msg.tick}, uuid = $uuid, and time = ${t.getOrElse("Option")}, result = ${res.getOrElse("Option")}") @@ -154,6 +155,8 @@ object ExtResultDataProvider { } }) + */ + //ctx.log.info(s"[requestResults] tick ${msg.tick} -> requestResults for " + receiveDataMap) if (receiveDataMap.isComplete) { From a8e8d8b32c3eb3aac7e6989376022120451935a8 Mon Sep 17 00:00:00 2001 From: Johannes Bao Date: Tue, 21 May 2024 18:18:43 +0200 Subject: [PATCH 29/41] UncontrolledStrat --- .../edu/ie3/simona/agent/em/EmAgent.scala | 2 +- .../ie3/simona/model/em/EmModelShell.scala | 2 + .../simona/model/em/UncontrolledStrat.scala | 76 +++++++++++++++++++ 3 files changed, 79 insertions(+), 1 deletion(-) create mode 100644 src/main/scala/edu/ie3/simona/model/em/UncontrolledStrat.scala diff --git a/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala b/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala index 0e53918488..ace0a87412 100644 --- a/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala +++ b/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala @@ -142,7 +142,7 @@ object EmAgent { val modelShell = EmModelShell( inputModel.getUuid, inputModel.getId, - modelStrategy, + inputModel.getControlStrategy, modelConfig, ) diff --git a/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala b/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala index a96cea57de..8ca17b9ab2 100644 --- a/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala +++ b/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala @@ -120,6 +120,8 @@ object EmModelShell { case "PROPORTIONAL" => ProportionalFlexStrat case "PRIORITIZED" => PrioritizedFlexStrat(modelConfig.curtailRegenerative) + case "self_optimization" => PrioritizedFlexStrat(modelConfig.curtailRegenerative) + case "uncontrolled" => UncontrolledStrat(modelConfig.curtailRegenerative) case unknown => throw new CriticalFailureException(s"Unknown model strategy $unknown") } diff --git a/src/main/scala/edu/ie3/simona/model/em/UncontrolledStrat.scala b/src/main/scala/edu/ie3/simona/model/em/UncontrolledStrat.scala new file mode 100644 index 0000000000..ab32a136e4 --- /dev/null +++ b/src/main/scala/edu/ie3/simona/model/em/UncontrolledStrat.scala @@ -0,0 +1,76 @@ +/* + * © 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 UncontrolledStrat(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)] = Seq.empty + + 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, + ) + } + } +} From 37fd8e3df972911e14d809259e56465fe84ee1f3 Mon Sep 17 00:00:00 2001 From: Johannes Bao Date: Tue, 21 May 2024 21:00:51 +0200 Subject: [PATCH 30/41] temp --- .../fullGrid/em_input.csv | 2 +- .../simopsimtestgrid.conf | 4 +- .../fullGrid/em_input.csv | 4 + .../fullGrid/line_input.csv | 110 ++++++++ .../fullGrid/line_type_input.csv | 2 + .../fullGrid/load_input.csv | 4 + .../fullGrid/node_input.csv | 112 ++++++++ .../fullGrid/pv_input.csv | 4 + .../fullGrid/transformer_2_w_input.csv | 2 + .../fullGrid/transformer_2_w_type_input.csv | 2 + .../simopsimtestgrid.conf | 240 ++++++++++++++++++ .../edu/ie3/simona/agent/em/EmAgent.scala | 2 +- .../ie3/simona/model/em/EmModelShell.scala | 2 +- 13 files changed, 485 insertions(+), 5 deletions(-) create mode 100644 input/samples/simopsimtestgrid_reduced_reference/fullGrid/em_input.csv create mode 100644 input/samples/simopsimtestgrid_reduced_reference/fullGrid/line_input.csv create mode 100644 input/samples/simopsimtestgrid_reduced_reference/fullGrid/line_type_input.csv create mode 100644 input/samples/simopsimtestgrid_reduced_reference/fullGrid/load_input.csv create mode 100644 input/samples/simopsimtestgrid_reduced_reference/fullGrid/node_input.csv create mode 100644 input/samples/simopsimtestgrid_reduced_reference/fullGrid/pv_input.csv create mode 100644 input/samples/simopsimtestgrid_reduced_reference/fullGrid/transformer_2_w_input.csv create mode 100644 input/samples/simopsimtestgrid_reduced_reference/fullGrid/transformer_2_w_type_input.csv create mode 100644 input/samples/simopsimtestgrid_reduced_reference/simopsimtestgrid.conf diff --git a/input/samples/simopsimtestgrid_reduced/fullGrid/em_input.csv b/input/samples/simopsimtestgrid_reduced/fullGrid/em_input.csv index ff0cc8720c..b6a794fe65 100755 --- a/input/samples/simopsimtestgrid_reduced/fullGrid/em_input.csv +++ b/input/samples/simopsimtestgrid_reduced/fullGrid/em_input.csv @@ -1,4 +1,4 @@ uuid;control_strategy;parent_em;id;operates_from;operates_until;operator -c3a7e9f5-b492-4c85-af2d-1e93f6a25443;kein_bock;;EM_HH_Bus_25;;; +c3a7e9f5-b492-4c85-af2d-1e93f6a25443;self_optimization;;EM_HH_Bus_25;;; f9dc7ce6-658c-4101-a12f-d58bb889286b;self_optimization;;EM_HH_Bus_81;;; 957938b7-0476-4fab-a1b3-6ce8615857b3;self_optimization;;EM_HH_Bus_110;;; diff --git a/input/samples/simopsimtestgrid_reduced/simopsimtestgrid.conf b/input/samples/simopsimtestgrid_reduced/simopsimtestgrid.conf index 6144e50874..501f00a065 100755 --- a/input/samples/simopsimtestgrid_reduced/simopsimtestgrid.conf +++ b/input/samples/simopsimtestgrid_reduced/simopsimtestgrid.conf @@ -15,8 +15,8 @@ simona.simulationName = "simopsimtest" ################################################################## # Time Parameters ################################################################## -simona.time.startDateTime = "2023-07-27T00:00:00Z" -simona.time.endDateTime = "2023-07-28T00:15:00Z" +simona.time.startDateTime = "2024-02-27T00:00:00Z" +simona.time.endDateTime = "2024-02-28T00:15:00Z" simona.time.schedulerReadyCheckWindow = 900 ################################################################## diff --git a/input/samples/simopsimtestgrid_reduced_reference/fullGrid/em_input.csv b/input/samples/simopsimtestgrid_reduced_reference/fullGrid/em_input.csv new file mode 100644 index 0000000000..100867a21d --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced_reference/fullGrid/em_input.csv @@ -0,0 +1,4 @@ +uuid;control_strategy;parent_em;id;operates_from;operates_until;operator +c3a7e9f5-b492-4c85-af2d-1e93f6a25443;uncontrolled;;EM_HH_Bus_25;;; +f9dc7ce6-658c-4101-a12f-d58bb889286b;uncontrolled;;EM_HH_Bus_81;;; +957938b7-0476-4fab-a1b3-6ce8615857b3;uncontrolled;;EM_HH_Bus_110;;; diff --git a/input/samples/simopsimtestgrid_reduced_reference/fullGrid/line_input.csv b/input/samples/simopsimtestgrid_reduced_reference/fullGrid/line_input.csv new file mode 100644 index 0000000000..f4ff92fbba --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced_reference/fullGrid/line_input.csv @@ -0,0 +1,110 @@ +uuid;geo_position;id;length;node_a;node_b;olm_characteristic;operates_from;operates_until;operator;parallel_devices;type +bdbbb247-57b7-473b-9411-53fcf35032db;"{""type"":""LineString"",""coordinates"":[[11.8213,53.426],[11.8213,53.4257]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 27;0.0161277;093160c4-6482-4c58-b952-217c615e3ada;9cdb3115-cc00-4d61-bc33-442e8f30fb63;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +c48eeb2c-6858-4f96-9a90-20641cc0903d;"{""type"":""LineString"",""coordinates"":[[11.8214,53.4229],[11.8213,53.4225]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 35;0.0150703;002a4495-96e4-49c9-abbe-8fccb3e9c83e;b909fb45-b6ee-427f-afd7-e8a0ec7274c6;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +97e498ae-a4f7-4485-a703-b8c00d0a5e8a;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4221],[11.8201,53.4225]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 8;0.0228449;b565ae3b-68f9-4bca-816d-9b0fc1c6b13f;a7725293-05fc-447f-bc12-38b689b0a956;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +f907f152-7bfe-4a17-a63f-adda500f6f0e;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4241],[11.8184,53.4293]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 107;0.04;4749ab2b-4d96-4100-8081-73e77c797d6b;ea4a6507-e504-4542-be5f-1019719b2257;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +6bf14400-8f6c-41ef-b6a4-9e861a3ad08f;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4241],[11.822,53.4294]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 3;0.030304;4749ab2b-4d96-4100-8081-73e77c797d6b;c5b6bfaf-1621-40a7-9c53-02cfb59c04d9;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +18b92b71-320d-4294-a47e-6715f1594755;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4243],[11.8196,53.4247]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 102;0.0251089;f4da61e4-7600-4cd1-95b6-c70b56c049fc;e7908208-77b4-4059-806e-4857262992fc;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +4128307f-3e00-4da9-b629-b696b72165a4;"{""type"":""LineString"",""coordinates"":[[11.815,53.4289],[11.815,53.4293]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 28;0.0377768;8e809cf6-7e05-437c-89a6-f6ca135a403b;35748e60-3be8-4930-8a61-209fd5df1bec;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +92876b53-6bbc-48ff-ba5f-5f5c08313e4d;"{""type"":""LineString"",""coordinates"":[[11.815,53.4277],[11.815,53.4272]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 111;0.031;3e6be3ac-2b51-4080-b815-391313612fc7;78815cf6-70db-432c-96e6-87fe8cf67eee;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +4b557d0c-9992-48b1-b45e-26cccc01db03;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4235],[11.8191,53.4231]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 90;0.00167204;f2724954-34d3-4ddf-b6b0-7a1531639990;2efac9b1-fb0d-4e08-bfac-501798826deb;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +0652d3e0-8c7c-4be2-9a41-00534aa9774b;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4212],[11.8191,53.4209]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 94;0.00182131;02e63e81-2e62-4ef6-8fdb-0b0905a437b6;fd4bebb8-40ca-4eed-92c0-cdd10b86ac20;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +782d8c4c-accb-424b-91ae-9af1b5d4b1af;"{""type"":""LineString"",""coordinates"":[[11.8201,53.4225],[11.8201,53.4229]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 79;0.0338129;a7725293-05fc-447f-bc12-38b689b0a956;83da8d60-405a-45f7-9bb9-9d35607b7927;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +b7cac7a5-f528-45bd-8ced-a16234979e13;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4239],[11.8191,53.4235]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 89;0.0111411;b5c1e826-63fd-4b0c-bec0-0c758389ef58;f2724954-34d3-4ddf-b6b0-7a1531639990;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +7ac1a715-e141-429a-8cfe-ff2badd41074;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4257],[11.8174,53.4262]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 65;0.019;8726dc29-621e-4455-a541-cd88d7da457f;814f784b-687f-4dd5-8a91-c7772c916d46;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +77ae4f1e-9ecb-4dda-a5f2-ce7ae3f9bbec;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4264],[11.8213,53.426]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 23;0.00488535;18b4157b-0e47-4c5a-adb8-ccae47372336;093160c4-6482-4c58-b952-217c615e3ada;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +6aaa54d9-d7b7-4adc-a85a-335cdf1393d6;"{""type"":""LineString"",""coordinates"":[[11.8162,53.4284],[11.8169,53.4289]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 40;0.036;6b2881ce-3965-4f5e-98a3-74eb47b0a7ca;a882e666-82d1-4ba6-87df-fc702fe06187;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +c85c107e-82c0-4002-acfa-d7000512a2ad;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4274],[11.8213,53.4271]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 20;0.0182765;2b3d7fb8-0583-4d47-97b1-3b5f232fd462;7b81b518-00e0-4ff1-b4cf-876903958d7a;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +2ef5f888-4341-4eee-b505-ae07a9a60c8d;"{""type"":""LineString"",""coordinates"":[[11.8201,53.4236],[11.8201,53.4239]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 55;0.0259846;f713593a-3fd3-410a-ac08-74202d4f5798;80962bd3-a10f-4ed2-ba6a-3e802189939c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +139ea4f9-9b7f-4825-8919-537a94ff4794;"{""type"":""LineString"",""coordinates"":[[11.8175,53.4277],[11.8174,53.4281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 59;0.00160952;7f692039-eef6-45f6-9e30-b5983f6750a5;3f6c26dd-842b-4dee-b71f-4aa32e2654ff;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +68853808-8b09-4ff2-9c92-88ed1d78c639;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4236],[11.8213,53.4232]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 33;0.0217875;3464496c-7dd4-41e9-ae0a-99ade0b51572;a966644c-37d5-4400-9fa0-725fd88586a8;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +91bec60c-2951-420c-a35e-2633119ee450;"{""type"":""LineString"",""coordinates"":[[11.8162,53.4277],[11.8162,53.4281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 30;0.00820054;2d33314e-31db-4ad4-a898-2be5d56a896d;555cd075-0fe4-4a65-b027-f45cffa960d9;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +35ba0827-e27e-496e-b735-c778f3b03019;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4241],[11.8213,53.4293]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 18;0.0139169;4749ab2b-4d96-4100-8081-73e77c797d6b;844c0b9c-058a-4228-a8c4-bf2defff6958;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +60efd159-72d6-4546-b245-8b8fc120a9f8;"{""type"":""LineString"",""coordinates"":[[11.817,53.4293],[11.8166,53.4241]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 26;0.098;85751c9b-3e5e-468a-9dc7-43775b0d4a6f;4749ab2b-4d96-4100-8081-73e77c797d6b;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +212b62aa-73c2-47af-95ec-00cad8d4a4f4;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4222],[11.8214,53.4218]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 38;0.00992653;589bf32a-b361-4536-ae96-6d56d184eedb;3b86661a-187d-4aa6-bf37-2014789afc08;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +7e61def1-8414-40b3-8775-fad9124f4369;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4282],[11.8213,53.4277]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 16;0.0164234;09285b78-9c18-4af7-9c7a-942cc868016f;3a557b4e-06b8-4f29-929f-81d95c42c897;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +f9b4c7dc-c199-4691-8c6d-1faf438cf336;"{""type"":""LineString"",""coordinates"":[[11.8221,53.429],[11.8221,53.4286]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 5;0.0143698;9644f198-e801-4545-87ee-a24e2a8039bd;6ee7ea93-ea9c-40cb-b79a-1c5f287c97a5;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +eebc456e-d11b-4b19-9100-cc1d8e91f926;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4245],[11.8166,53.4249]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 71;0.00648856;7efabb8d-ba17-4487-96d9-5744b1fedf8a;7c35a794-f569-4a9c-acb0-d03647610086;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +8f4c4009-e1ed-4985-8095-729aa5dc8cdd;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4192],[11.8198,53.4195]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 85;0.00737032;594d101c-3a05-45e3-a061-9189f3e848b7;af2b07ce-1a96-4b50-9e21-badf29eed519;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +20db47c8-d154-4ab2-ad65-caa4a37466d2;"{""type"":""LineString"",""coordinates"":[[11.8175,53.4257],[11.8174,53.4262]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 68;0.037;5f153bf6-4f25-41f1-8545-18fe6323bc49;814f784b-687f-4dd5-8a91-c7772c916d46;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +7d838003-cd6e-42fd-b75a-eaa354c3993b;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4216],[11.8191,53.4212]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 93;0.0043578;8f53645e-1f28-4eb1-807f-eb2a473f9d4c;02e63e81-2e62-4ef6-8fdb-0b0905a437b6;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +a92013e1-78b7-4447-a505-f25fffbf56f2;"{""type"":""LineString"",""coordinates"":[[11.8174,53.4262],[11.8175,53.4265]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 48;0.0118531;814f784b-687f-4dd5-8a91-c7772c916d46;94fe96b1-f36a-4edd-a107-4ff0376f1066;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +690f8f61-5cc7-448e-8971-a516f784bf11;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4293],[11.8166,53.4241]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 106;0.0131451;20ad9754-e966-4ad1-9541-f968c207f3df;4749ab2b-4d96-4100-8081-73e77c797d6b;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +6a00d582-32b1-4581-bdd1-a638ca279597;"{""type"":""LineString"",""coordinates"":[[11.815,53.4272],[11.815,53.4269]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 112;0.028;78815cf6-70db-432c-96e6-87fe8cf67eee;ab8c8f6c-e7a2-4b81-a0d7-5f13789267a2;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +1d5877c1-31cd-4ee8-b0dd-667fa4fffb8a;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4277],[11.8213,53.4274]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 17;0.00286133;3a557b4e-06b8-4f29-929f-81d95c42c897;2b3d7fb8-0583-4d47-97b1-3b5f232fd462;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +a15cd6a4-4b22-434f-be9b-ad2abe6e538c;"{""type"":""LineString"",""coordinates"":[[11.815,53.4289],[11.815,53.4285]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 108;0.033;8e809cf6-7e05-437c-89a6-f6ca135a403b;88cf719a-92df-4dfd-9a83-f84330e28fe0;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +2bd81511-4f19-4374-8ac5-96c2b19eda64;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4223],[11.8191,53.422]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 52;0.00980013;c7e48384-5699-4a38-a887-7e15a9145202;c6dac3ab-f44f-4b87-800c-0f4da64673f1;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +beee40ad-aca3-490f-87d5-a6dfc4bb76e3;"{""type"":""LineString"",""coordinates"":[[11.815,53.4293],[11.8166,53.4241]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 39;0.09;35748e60-3be8-4930-8a61-209fd5df1bec;4749ab2b-4d96-4100-8081-73e77c797d6b;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +96b06f87-7905-4d12-99d8-1ed330050659;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4203],[11.8197,53.4207]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 12;0.00473835;5596da2f-ca32-4ad3-81da-1ffa17cd3d7b;839ff0f4-93db-42ec-a928-bbc448b6cf5c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +e66890ef-ec02-407f-a5bb-ce32128e7490;"{""type"":""LineString"",""coordinates"":[[11.8162,53.4281],[11.8162,53.4284]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 41;0.00828244;555cd075-0fe4-4a65-b027-f45cffa960d9;6b2881ce-3965-4f5e-98a3-74eb47b0a7ca;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +015b0337-98bd-40d4-97d3-13a0d1da88ee;"{""type"":""LineString"",""coordinates"":[[11.8213,53.425],[11.8214,53.4246]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 84;0.0489789;58b551b6-83bd-4f1c-8d9c-8c9a7f638c0b;a9288e77-2919-4db6-89eb-9737bd07f111;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +f9e6b2f1-f8fb-4763-a1d0-36d06170fea0;"{""type"":""LineString"",""coordinates"":[[11.8213,53.429],[11.8213,53.4286]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 98;0.0144283;16091b6d-f1ea-4a07-9ad4-30d595aba68d;1403edf9-e47c-4705-8563-83bcd639482e;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +f270d6b5-7c94-4397-8ab0-7c39c888d726;"{""type"":""LineString"",""coordinates"":[[11.8175,53.4265],[11.8175,53.427]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 58;0.0036937;94fe96b1-f36a-4edd-a107-4ff0376f1066;80b8d1f8-7e83-421d-a95a-c193fc35f4f7;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +af8c65aa-cfd6-434a-8512-2d80106a2f2c;"{""type"":""LineString"",""coordinates"":[[11.8195,53.426],[11.8195,53.4264]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 31;0.00626899;9f1baf4e-12e1-41d1-8efc-81cfc78f1957;b8e95bf0-3ba8-4d53-a0bf-a3720fb785fb;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +fe28e831-6405-4dfe-987b-d688367694f1;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4247],[11.8195,53.425]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 104;0.00221503;e7908208-77b4-4059-806e-4857262992fc;29516ae3-6676-4797-99c1-1f0a32b989d8;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +eac56d69-5500-4261-9690-adb16c867485;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4231],[11.8191,53.4227]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 92;0.0149219;2efac9b1-fb0d-4e08-bfac-501798826deb;15a86f7d-fb73-49a4-af6a-25b14122378d;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +183f3976-48ca-42b2-9af9-7998436fac5b;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4289],[11.8196,53.4293]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 105;0.00721947;47246a84-ad0e-4d04-9d98-1c9cd5d363c1;20ad9754-e966-4ad1-9541-f968c207f3df;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +7feef458-03f3-4d23-b3a2-e6f1035398c4;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4243],[11.8213,53.4239]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 29;0.0204277;3fcb94e3-7781-4d83-9030-d9853822e78e;d0bfabdb-0e83-423b-a20a-ab9197c4284e;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +592cd979-16c9-43d8-a311-8ac938aa5d03;"{""type"":""LineString"",""coordinates"":[[11.8174,53.4262],[11.8182,53.4257]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 47;0.0181601;814f784b-687f-4dd5-8a91-c7772c916d46;c86d6361-4159-4787-b5f4-e41dcaa95195;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +684146e5-3b58-43b4-9589-45325ab1c0bc;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4257],[11.8213,53.4253]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 24;0.0102233;9cdb3115-cc00-4d61-bc33-442e8f30fb63;a432b8ce-0462-478b-83e7-3107cd2e909c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +d9f9ee76-b016-4588-ac6d-46681894ada7;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4188],[11.8198,53.4192]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 74;0.00421724;952c6b04-0d02-4ea2-a6cc-bb44fbbe4e52;594d101c-3a05-45e3-a061-9189f3e848b7;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +1f7e9cae-460e-47db-8cb8-da5d9f695fd8;"{""type"":""LineString"",""coordinates"":[[11.8197,53.4207],[11.8198,53.4211]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 13;0.00987733;839ff0f4-93db-42ec-a928-bbc448b6cf5c;27b84da5-478e-4a05-8fe7-a9f800db5eff;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +f3a592d0-0fd7-42ea-b928-f39473b419aa;"{""type"":""LineString"",""coordinates"":[[11.8214,53.4246],[11.8213,53.4243]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 10;0.0344866;a9288e77-2919-4db6-89eb-9737bd07f111;3fcb94e3-7781-4d83-9030-d9853822e78e;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +f505bff9-0803-415f-a765-9da981ff6024;"{""type"":""LineString"",""coordinates"":[[11.8195,53.4264],[11.8195,53.4268]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 43;0.00839819;b8e95bf0-3ba8-4d53-a0bf-a3720fb785fb;34031e92-3444-47d5-94ae-cceeb5d96bb2;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +c6501262-2e05-462b-8872-445d2aa1cab8;"{""type"":""LineString"",""coordinates"":[[11.8195,53.4272],[11.8195,53.4275]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 99;0.0157523;120eaa58-a500-4ae2-a86a-56a40b931ec1;9d136a6b-5fdc-44ed-a5ed-599a55281024;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +caac4b4b-4871-4e20-994c-6517931546cb;"{""type"":""LineString"",""coordinates"":[[11.8201,53.4229],[11.8201,53.4232]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 91;0.0220129;83da8d60-405a-45f7-9bb9-9d35607b7927;24b63115-12eb-4e77-b9ef-ca474fed960f;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +e9721561-53c0-45cc-a8ed-28861ef9dc66;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4216],[11.8198,53.4221]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 7;0.0333523;883edf38-9a18-4f61-981a-691aaf436cc7;b565ae3b-68f9-4bca-816d-9b0fc1c6b13f;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +ab143df6-d050-47b6-911a-93e462d928ac;"{""type"":""LineString"",""coordinates"":[[11.8173,53.4234],[11.8166,53.4238]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 73;0.0287121;e68a088d-cf1a-40b7-9b1a-e0933352f4e6;4129e079-6712-4275-911c-36729d698c42;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +ab7ab785-36ba-4da1-b176-a7c636cb1372;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4267],[11.8213,53.4264]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 22;0.0102382;dc54bd8a-b7d8-4e99-adb0-d6ee5084241c;18b4157b-0e47-4c5a-adb8-ccae47372336;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +1ae90a03-52a3-40ef-8e06-4ba01888aa5c;"{""type"":""LineString"",""coordinates"":[[11.816,53.4222],[11.816,53.4227]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 44;0.00743191;62d603c3-f306-40b3-a665-ba9892d226f0;3faac527-0ff3-44a7-9e4f-24a41940da90;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +20be9235-f4db-4753-9fa1-223c8519fcd3;"{""type"":""LineString"",""coordinates"":[[11.8175,53.4253],[11.8175,53.4257]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 66;0.00791441;73e7a7e8-2154-46ea-9727-a4916af3570c;5f153bf6-4f25-41f1-8545-18fe6323bc49;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +558e1545-a944-419a-9153-83caa09e1a3c;"{""type"":""LineString"",""coordinates"":[[11.8162,53.4273],[11.8162,53.4277]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 42;0.00357726;0d6ef8f6-0ba1-4fa7-8e63-e55cee12b165;2d33314e-31db-4ad4-a898-2be5d56a896d;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +38d60cf7-6099-4bc0-a616-0f0b66c70c9a;"{""type"":""LineString"",""coordinates"":[[11.8201,53.4232],[11.8201,53.4236]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 67;0.0179092;24b63115-12eb-4e77-b9ef-ca474fed960f;f713593a-3fd3-410a-ac08-74202d4f5798;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +e74dbbe9-948f-4056-8134-fd1d9d39e773;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4253],[11.8196,53.4257]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 86;0.0190003;03b2aa45-84f6-48c0-9dab-427e046a5672;79e19265-08e8-407f-ae95-2f78e344d3a4;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +24c0dc1c-2e99-46ed-a52b-5a7aae7c9afb;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4249],[11.8166,53.4253]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 72;0.00189914;7c35a794-f569-4a9c-acb0-d03647610086;50cac08e-bf24-4526-9466-53ca5edccd15;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +41a72cb2-f037-4196-a248-2b18a578db50;"{""type"":""LineString"",""coordinates"":[[11.816,53.421],[11.816,53.4214]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 78;0.00156586;98c14f60-e196-4f12-903b-8485f1eacb16;ca0b9e34-ddc8-405e-86a4-ab939626c4a2;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +e2919117-f70a-4b32-a5ab-f077fe0a84ef;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4253],[11.8166,53.4257]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 70;0.00263875;50cac08e-bf24-4526-9466-53ca5edccd15;8726dc29-621e-4455-a541-cd88d7da457f;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +7d3ee5e1-1749-4e43-afeb-4ccf5f431312;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4282],[11.8196,53.4286]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 103;0.011354;a3ed5af5-1fcb-4fce-af0f-708d3d604124;31e6e197-719d-4aaf-8ca5-ab9e7549390e;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +cbba4a6b-27d1-4724-a193-134af220a57d;"{""type"":""LineString"",""coordinates"":[[11.8182,53.4257],[11.8182,53.4254]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 49;0.018118;c86d6361-4159-4787-b5f4-e41dcaa95195;c6c20ab1-16fb-4f82-8017-273022da8bb0;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +b84fad90-182c-4ac1-b77d-394f39024ffc;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4286],[11.8196,53.4289]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 19;0.0115943;31e6e197-719d-4aaf-8ca5-ab9e7549390e;47246a84-ad0e-4d04-9d98-1c9cd5d363c1;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +6c87a571-9d8b-4df9-bd86-cde7954bee28;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4271],[11.8213,53.4267]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 21;0.00282037;7b81b518-00e0-4ff1-b4cf-876903958d7a;dc54bd8a-b7d8-4e99-adb0-d6ee5084241c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +536ef1a2-b988-4474-a5d4-1254598c4716;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4227],[11.8191,53.4223]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 82;0.00544392;15a86f7d-fb73-49a4-af6a-25b14122378d;c7e48384-5699-4a38-a887-7e15a9145202;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +c0c99b3a-1c61-47b8-931d-571d9494d98f;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4241],[11.8166,53.4245]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 51;0.0182324;5682ac05-7336-4ebc-a5d1-3b69c79fb3b1;7efabb8d-ba17-4487-96d9-5744b1fedf8a;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +d3366635-447e-4d45-b839-7a7561f869a3;"{""type"":""LineString"",""coordinates"":[[11.822,53.4294],[11.8221,53.429]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 4;0.0141339;c5b6bfaf-1621-40a7-9c53-02cfb59c04d9;9644f198-e801-4545-87ee-a24e2a8039bd;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +932dd0d7-2051-4a3a-a0e3-489210d3b763;"{""type"":""LineString"",""coordinates"":[[11.8169,53.4289],[11.817,53.4293]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 57;0.0219657;a882e666-82d1-4ba6-87df-fc702fe06187;85751c9b-3e5e-468a-9dc7-43775b0d4a6f;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +e718d253-34ea-4c00-a653-80ac0af2934e;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4293],[11.8213,53.429]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 11;0.00557129;844c0b9c-058a-4228-a8c4-bf2defff6958;16091b6d-f1ea-4a07-9ad4-30d595aba68d;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +3ac954bf-5c1e-428a-9da7-37331f53d8fe;"{""type"":""LineString"",""coordinates"":[[11.8175,53.4248],[11.8175,53.4253]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 69;0.0017759;0d6c3f2b-5296-4ec1-995c-b150e72f035f;73e7a7e8-2154-46ea-9727-a4916af3570c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +7e9a0440-2a62-4daf-a4ab-e9f14f6cfd77;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4238],[11.8166,53.4234]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 53;0.0337731;4129e079-6712-4275-911c-36729d698c42;57b40047-4f9d-46bb-bf19-c0a86bbd4f5b;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +10a2050e-173b-43a7-91a0-157cee5c900d;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4257],[11.8195,53.426]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 87;0.00654022;79e19265-08e8-407f-ae95-2f78e344d3a4;9f1baf4e-12e1-41d1-8efc-81cfc78f1957;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +0dfd30a5-394e-49ff-95e9-50e73bf2604c;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4232],[11.8214,53.4229]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 34;0.0199334;a966644c-37d5-4400-9fa0-725fd88586a8;002a4495-96e4-49c9-abbe-8fccb3e9c83e;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +847934a0-e2b7-4caf-bb40-2e6af5bcb0ab;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4238],[11.8166,53.4241]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 50;0.0101795;4129e079-6712-4275-911c-36729d698c42;5682ac05-7336-4ebc-a5d1-3b69c79fb3b1;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +2daf9021-eb24-4d0c-bced-00e713a65b3a;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4225],[11.8213,53.4222]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 36;0.010188;b909fb45-b6ee-427f-afd7-e8a0ec7274c6;589bf32a-b361-4536-ae96-6d56d184eedb;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +2f1b02a3-5a58-4723-ab8f-827118a4c611;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4239],[11.8213,53.4236]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 32;0.0188563;d0bfabdb-0e83-423b-a20a-ab9197c4284e;3464496c-7dd4-41e9-ae0a-99ade0b51572;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +a5e9b6b0-2afc-49b0-8aad-705410b823c2;"{""type"":""LineString"",""coordinates"":[[11.815,53.4285],[11.815,53.4281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 109;0.027;88cf719a-92df-4dfd-9a83-f84330e28fe0;d38d936a-9c05-4bdc-8331-418fef27f492;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +ae82a1c1-9790-44d1-8dab-52d75c7b79ce;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4286],[11.8213,53.4282]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 6;0.0050064;1403edf9-e47c-4705-8563-83bcd639482e;09285b78-9c18-4af7-9c7a-942cc868016f;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +e642eafb-2b6e-4b78-80b8-1ab15db8cfeb;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4278],[11.8196,53.4282]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 101;0.0153312;7dc43c81-9a61-45a0-9745-800a28bf4a9d;a3ed5af5-1fcb-4fce-af0f-708d3d604124;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +41a4c5e5-7b54-46a8-b6b6-d7b5861a251c;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4195],[11.8198,53.4199]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 96;0.0010841;af2b07ce-1a96-4b50-9e21-badf29eed519;215eaa45-82c3-49c7-a60f-4fa13215de05;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +44821cfc-7670-4f28-8941-70e9345cb069;"{""type"":""LineString"",""coordinates"":[[11.8201,53.4225],[11.8203,53.4221]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 15;0.027677;a7725293-05fc-447f-bc12-38b689b0a956;eae8a04c-44f2-4da3-95f6-cae48f85737c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +d665e8c6-40fb-4608-97fb-99a4cf52560e;"{""type"":""LineString"",""coordinates"":[[11.815,53.4281],[11.815,53.4277]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 110;0.041;d38d936a-9c05-4bdc-8331-418fef27f492;3e6be3ac-2b51-4080-b815-391313612fc7;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +2f37bd36-f356-4d29-81c7-dd9c2fba5e7d;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4234],[11.8166,53.423]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 45;0.00223482;57b40047-4f9d-46bb-bf19-c0a86bbd4f5b;5b73ded9-3ca4-4f18-a2ab-c27b9a3dcf9d;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +bcccf0b5-e0e3-4cc7-82bb-262ebc19415e;"{""type"":""LineString"",""coordinates"":[[11.8169,53.4289],[11.8168,53.4285]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 56;0.039;a882e666-82d1-4ba6-87df-fc702fe06187;c41ebab8-16a0-4a3a-b4af-26073932d462;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +3685f389-6822-4522-a633-74265b67eaee;"{""type"":""LineString"",""coordinates"":[[11.816,53.4227],[11.816,53.423]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 76;0.00185168;3faac527-0ff3-44a7-9e4f-24a41940da90;2575f527-1f4e-45e2-bed2-4a5427f122e0;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +48347702-9e58-49f3-a7de-72024d4b296c;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4199],[11.8198,53.4203]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 2;0.00995493;215eaa45-82c3-49c7-a60f-4fa13215de05;5596da2f-ca32-4ad3-81da-1ffa17cd3d7b;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +ec2a73ea-50ba-4187-b8b0-a5046cf6b632;"{""type"":""LineString"",""coordinates"":[[11.8195,53.425],[11.8196,53.4253]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 83;0.00510961;29516ae3-6676-4797-99c1-1f0a32b989d8;03b2aa45-84f6-48c0-9dab-427e046a5672;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +d1940183-f7bb-42df-b8ff-63ac7aff0b1d;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4211],[11.8198,53.4216]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 9;0.00439896;27b84da5-478e-4a05-8fe7-a9f800db5eff;883edf38-9a18-4f61-981a-691aaf436cc7;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +dd618e3f-9ef9-4e4b-b632-737d79c4d8c3;"{""type"":""LineString"",""coordinates"":[[11.8175,53.427],[11.8174,53.4273]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 62;0.00124645;80b8d1f8-7e83-421d-a95a-c193fc35f4f7;8ace5c2b-584a-4015-990f-6f1e14de4ddb;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +ed80666a-cf86-404a-b3a7-ad49be1cd40c;"{""type"":""LineString"",""coordinates"":[[11.816,53.423],[11.816,53.4234]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 77;0.00232237;2575f527-1f4e-45e2-bed2-4a5427f122e0;5d3bcf55-0520-43ff-8d63-3d0eb421e442;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +444615ed-26f2-45a7-8b5d-213c72e83a4f;"{""type"":""LineString"",""coordinates"":[[11.8201,53.4239],[11.8196,53.4243]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 95;0.0281251;80962bd3-a10f-4ed2-ba6a-3e802189939c;f4da61e4-7600-4cd1-95b6-c70b56c049fc;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +05ebbc17-61bc-4810-ae19-9ee04d7ce8d0;"{""type"":""LineString"",""coordinates"":[[11.8175,53.4285],[11.8169,53.4289]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 54;0.0173762;30b2ce2d-41dd-4d1d-866d-47abd24bfc3e;a882e666-82d1-4ba6-87df-fc702fe06187;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +caf90323-aad2-496e-a0e3-a4ba7c9a481c;"{""type"":""LineString"",""coordinates"":[[11.8174,53.4273],[11.8175,53.4277]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 64;0.00314803;8ace5c2b-584a-4015-990f-6f1e14de4ddb;7f692039-eef6-45f6-9e30-b5983f6750a5;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +60b400db-b048-4bed-be29-8abc18780d10;"{""type"":""LineString"",""coordinates"":[[11.816,53.4206],[11.816,53.421]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 14;0.045296;d6dbb0ae-13c9-438e-93b3-b6c63a0708df;98c14f60-e196-4f12-903b-8485f1eacb16;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +cf13634d-abd2-465d-8839-c95a54af7a80;"{""type"":""LineString"",""coordinates"":[[11.816,53.4214],[11.816,53.4218]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 80;0.0101988;ca0b9e34-ddc8-405e-86a4-ab939626c4a2;43040a39-8b6c-401f-9dfd-82b42aa6dec6;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +30044e09-a0f8-417d-a949-afcfa940f671;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4253],[11.8213,53.425]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 25;0.0124821;a432b8ce-0462-478b-83e7-3107cd2e909c;58b551b6-83bd-4f1c-8d9c-8c9a7f638c0b;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +fe71fe2b-8dd0-4942-b0b5-d241e095b912;"{""type"":""LineString"",""coordinates"":[[11.8195,53.4268],[11.8195,53.4272]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 97;0.0032723;34031e92-3444-47d5-94ae-cceeb5d96bb2;120eaa58-a500-4ae2-a86a-56a40b931ec1;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +66e44163-7091-4f1b-991c-64108e2238f6;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4243],[11.8191,53.4239]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 88;0.0185855;f4da61e4-7600-4cd1-95b6-c70b56c049fc;b5c1e826-63fd-4b0c-bec0-0c758389ef58;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +82bac681-169f-48d8-9842-fd69d3adbfe0;"{""type"":""LineString"",""coordinates"":[[11.816,53.4234],[11.8166,53.4238]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 75;0.0109137;5d3bcf55-0520-43ff-8d63-3d0eb421e442;4129e079-6712-4275-911c-36729d698c42;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +74a5dc42-e689-490f-a611-ae7c3767f01b;"{""type"":""LineString"",""coordinates"":[[11.816,53.4218],[11.816,53.4222]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 81;0.0100396;43040a39-8b6c-401f-9dfd-82b42aa6dec6;62d603c3-f306-40b3-a665-ba9892d226f0;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +0eedb3d0-cedc-4798-b42e-d8e8ef646b82;"{""type"":""LineString"",""coordinates"":[[11.8195,53.4275],[11.8196,53.4278]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 100;0.0129061;9d136a6b-5fdc-44ed-a5ed-599a55281024;7dc43c81-9a61-45a0-9745-800a28bf4a9d;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +b44446fd-2125-4c7b-850e-b0f3d6c8b110;"{""type"":""LineString"",""coordinates"":[[11.8174,53.4281],[11.8175,53.4285]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 61;0.014766;3f6c26dd-842b-4dee-b71f-4aa32e2654ff;30b2ce2d-41dd-4d1d-866d-47abd24bfc3e;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +31c914bc-dd9e-4825-9b4b-b5fc1b971e0d;"{""type"":""LineString"",""coordinates"":[[11.8166,53.423],[11.8166,53.4227]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 46;0.00683612;5b73ded9-3ca4-4f18-a2ab-c27b9a3dcf9d;ad30322c-0c99-4669-8e4b-25265087a66d;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +e7ac7023-f4e3-460d-8d02-4c9c444d2e18;"{""type"":""LineString"",""coordinates"":[[11.8191,53.422],[11.8191,53.4216]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 63;0.0149947;c6dac3ab-f44f-4b87-800c-0f4da64673f1;8f53645e-1f28-4eb1-807f-eb2a473f9d4c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e diff --git a/input/samples/simopsimtestgrid_reduced_reference/fullGrid/line_type_input.csv b/input/samples/simopsimtestgrid_reduced_reference/fullGrid/line_type_input.csv new file mode 100644 index 0000000000..d7786c38cd --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced_reference/fullGrid/line_type_input.csv @@ -0,0 +1,2 @@ +uuid;b;g;i_max;id;r;v_rated;x +9a8e9b63-af21-4c1b-8db7-fc2924f9610e;273.31899999999996;0.0;357.0;NAYY 4x240SE 0.6/1kV;0.1267;0.4;0.0797965 diff --git a/input/samples/simopsimtestgrid_reduced_reference/fullGrid/load_input.csv b/input/samples/simopsimtestgrid_reduced_reference/fullGrid/load_input.csv new file mode 100644 index 0000000000..6ad16189a5 --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced_reference/fullGrid/load_input.csv @@ -0,0 +1,4 @@ +uuid;cos_phi_rated;dsm;e_cons_annual;id;load_profile;node;operates_from;operates_until;operator;q_characteristics;s_rated;em +c3434742-e4f0-49e5-baa7-c1e3045c732c;0.93;false;0.0;LV5.201 Load 64;h0;dc54bd8a-b7d8-4e99-adb0-d6ee5084241c;;;;"cosPhiFixed:{(0.0,0.93)}";3.2;f9dc7ce6-658c-4101-a12f-d58bb889286b +fd2e19b6-d5e3-4776-9456-8787a2160d9d;0.93;false;0.0;LV5.201 Load 74;h0;43040a39-8b6c-401f-9dfd-82b42aa6dec6;;;;"cosPhiFixed:{(0.0,0.93)}";1.1;c3a7e9f5-b492-4c85-af2d-1e93f6a25443 +98c1a2ab-bd09-4c77-a389-d088aed894b1;0.93;false;0.0;LV5.201 Load 102;h0;3e6be3ac-2b51-4080-b815-391313612fc7;;;;"cosPhiFixed:{(0.0,0.93)}";5.3;957938b7-0476-4fab-a1b3-6ce8615857b3 diff --git a/input/samples/simopsimtestgrid_reduced_reference/fullGrid/node_input.csv b/input/samples/simopsimtestgrid_reduced_reference/fullGrid/node_input.csv new file mode 100644 index 0000000000..ab9f9335ca --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced_reference/fullGrid/node_input.csv @@ -0,0 +1,112 @@ +uuid;geo_position;id;operates_from;operates_until;operator;slack;subnet;v_rated;v_target;volt_lvl +16091b6d-f1ea-4a07-9ad4-30d595aba68d;"{""type"":""Point"",""coordinates"":[11.8213,53.429],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 26;;;;false;2;0.4;1.0;Niederspannung +27b84da5-478e-4a05-8fe7-a9f800db5eff;"{""type"":""Point"",""coordinates"":[11.8198,53.4211],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 7;;;;false;2;0.4;1.0;Niederspannung +7f692039-eef6-45f6-9e30-b5983f6750a5;"{""type"":""Point"",""coordinates"":[11.8175,53.4277],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 54;;;;false;2;0.4;1.0;Niederspannung +20ad9754-e966-4ad1-9541-f968c207f3df;"{""type"":""Point"",""coordinates"":[11.8196,53.4293],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 84;;;;false;2;0.4;1.0;Niederspannung +98c14f60-e196-4f12-903b-8485f1eacb16;"{""type"":""Point"",""coordinates"":[11.816,53.421],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 29;;;;false;2;0.4;1.0;Niederspannung +09285b78-9c18-4af7-9c7a-942cc868016f;"{""type"":""Point"",""coordinates"":[11.8213,53.4282],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 58;;;;false;2;0.4;1.0;Niederspannung +0d6ef8f6-0ba1-4fa7-8e63-e55cee12b165;"{""type"":""Point"",""coordinates"":[11.8162,53.4273],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 103;;;;false;2;0.4;1.0;Niederspannung +43040a39-8b6c-401f-9dfd-82b42aa6dec6;"{""type"":""Point"",""coordinates"":[11.816,53.4218],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 25;;;;false;2;0.4;1.0;Niederspannung +dc54bd8a-b7d8-4e99-adb0-d6ee5084241c;"{""type"":""Point"",""coordinates"":[11.8213,53.4267],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 81;;;;false;2;0.4;1.0;Niederspannung +2575f527-1f4e-45e2-bed2-4a5427f122e0;"{""type"":""Point"",""coordinates"":[11.816,53.423],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 3;;;;false;2;0.4;1.0;Niederspannung +30b2ce2d-41dd-4d1d-866d-47abd24bfc3e;"{""type"":""Point"",""coordinates"":[11.8175,53.4285],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 34;;;;false;2;0.4;1.0;Niederspannung +589bf32a-b361-4536-ae96-6d56d184eedb;"{""type"":""Point"",""coordinates"":[11.8213,53.4222],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 101;;;;false;2;0.4;1.0;Niederspannung +555cd075-0fe4-4a65-b027-f45cffa960d9;"{""type"":""Point"",""coordinates"":[11.8162,53.4281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 31;;;;false;2;0.4;1.0;Niederspannung +62d603c3-f306-40b3-a665-ba9892d226f0;"{""type"":""Point"",""coordinates"":[11.816,53.4222],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 55;;;;false;2;0.4;1.0;Niederspannung +215eaa45-82c3-49c7-a60f-4fa13215de05;"{""type"":""Point"",""coordinates"":[11.8198,53.4199],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 24;;;;false;2;0.4;1.0;Niederspannung +f4da61e4-7600-4cd1-95b6-c70b56c049fc;"{""type"":""Point"",""coordinates"":[11.8196,53.4243],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 76;;;;false;2;0.4;1.0;Niederspannung +73e7a7e8-2154-46ea-9727-a4916af3570c;"{""type"":""Point"",""coordinates"":[11.8175,53.4253],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 18;;;;false;2;0.4;1.0;Niederspannung +85751c9b-3e5e-468a-9dc7-43775b0d4a6f;"{""type"":""Point"",""coordinates"":[11.817,53.4293],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 51;;;;false;2;0.4;1.0;Niederspannung +093160c4-6482-4c58-b952-217c615e3ada;"{""type"":""Point"",""coordinates"":[11.8213,53.426],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 68;;;;false;2;0.4;1.0;Niederspannung +79e19265-08e8-407f-ae95-2f78e344d3a4;"{""type"":""Point"",""coordinates"":[11.8196,53.4257],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 28;;;;false;2;0.4;1.0;Niederspannung +a966644c-37d5-4400-9fa0-725fd88586a8;"{""type"":""Point"",""coordinates"":[11.8213,53.4232],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 59;;;;false;2;0.4;1.0;Niederspannung +952c6b04-0d02-4ea2-a6cc-bb44fbbe4e52;"{""type"":""Point"",""coordinates"":[11.8198,53.4188],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 94;;;;false;2;0.4;1.0;Niederspannung +c6c20ab1-16fb-4f82-8017-273022da8bb0;"{""type"":""Point"",""coordinates"":[11.8182,53.4254],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 62;;;;false;2;0.4;1.0;Niederspannung +e68a088d-cf1a-40b7-9b1a-e0933352f4e6;"{""type"":""Point"",""coordinates"":[11.8173,53.4234],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 67;;;;false;2;0.4;1.0;Niederspannung +8f53645e-1f28-4eb1-807f-eb2a473f9d4c;"{""type"":""Point"",""coordinates"":[11.8191,53.4216],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 86;;;;false;2;0.4;1.0;Niederspannung +0d6c3f2b-5296-4ec1-995c-b150e72f035f;"{""type"":""Point"",""coordinates"":[11.8175,53.4248],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 80;;;;false;2;0.4;1.0;Niederspannung +ea4a6507-e504-4542-be5f-1019719b2257;"{""type"":""Point"",""coordinates"":[11.8184,53.4293],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 111;;;;false;2;0.4;1.0;Niederspannung +9644f198-e801-4545-87ee-a24e2a8039bd;"{""type"":""Point"",""coordinates"":[11.8221,53.429],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 39;;;;false;2;0.4;1.0;Niederspannung +a9288e77-2919-4db6-89eb-9737bd07f111;"{""type"":""Point"",""coordinates"":[11.8214,53.4246],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 63;;;;false;2;0.4;1.0;Niederspannung +d6dbb0ae-13c9-438e-93b3-b6c63a0708df;"{""type"":""Point"",""coordinates"":[11.816,53.4206],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 89;;;;false;2;0.4;1.0;Niederspannung +5596da2f-ca32-4ad3-81da-1ffa17cd3d7b;"{""type"":""Point"",""coordinates"":[11.8198,53.4203],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 102;;;;false;2;0.4;1.0;Niederspannung +ad30322c-0c99-4669-8e4b-25265087a66d;"{""type"":""Point"",""coordinates"":[11.8166,53.4227],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 65;;;;false;2;0.4;1.0;Niederspannung +120eaa58-a500-4ae2-a86a-56a40b931ec1;"{""type"":""Point"",""coordinates"":[11.8195,53.4272],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 57;;;;false;2;0.4;1.0;Niederspannung +29516ae3-6676-4797-99c1-1f0a32b989d8;"{""type"":""Point"",""coordinates"":[11.8195,53.425],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 52;;;;false;2;0.4;1.0;Niederspannung +24b63115-12eb-4e77-b9ef-ca474fed960f;"{""type"":""Point"",""coordinates"":[11.8201,53.4232],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 46;;;;false;2;0.4;1.0;Niederspannung +a7725293-05fc-447f-bc12-38b689b0a956;"{""type"":""Point"",""coordinates"":[11.8201,53.4225],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 72;;;;false;2;0.4;1.0;Niederspannung +3b86661a-187d-4aa6-bf37-2014789afc08;"{""type"":""Point"",""coordinates"":[11.8214,53.4218],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 20;;;;false;2;0.4;1.0;Niederspannung +a3ed5af5-1fcb-4fce-af0f-708d3d604124;"{""type"":""Point"",""coordinates"":[11.8196,53.4282],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 33;;;;false;2;0.4;1.0;Niederspannung +ec8f2c82-a1b2-487c-b573-250859e3b414;"{""type"":""Point"",""coordinates"":[11.8166,53.4241],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";MV2.101 Bus 5;;;;true;1;20.0;1.025;Mittelspannung +5f153bf6-4f25-41f1-8545-18fe6323bc49;"{""type"":""Point"",""coordinates"":[11.8175,53.4257],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 17;;;;false;2;0.4;1.0;Niederspannung +b565ae3b-68f9-4bca-816d-9b0fc1c6b13f;"{""type"":""Point"",""coordinates"":[11.8198,53.4221],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 70;;;;false;2;0.4;1.0;Niederspannung +8e809cf6-7e05-437c-89a6-f6ca135a403b;"{""type"":""Point"",""coordinates"":[11.815,53.4289],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 47;;;;false;2;0.4;1.0;Niederspannung +ca0b9e34-ddc8-405e-86a4-ab939626c4a2;"{""type"":""Point"",""coordinates"":[11.816,53.4214],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 99;;;;false;2;0.4;1.0;Niederspannung +3e6be3ac-2b51-4080-b815-391313612fc7;"{""type"":""Point"",""coordinates"":[11.815,53.4277],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 110;;;;false;2;0.4;1.0;Niederspannung +4749ab2b-4d96-4100-8081-73e77c797d6b;"{""type"":""Point"",""coordinates"":[11.8166,53.4241],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 73;;;;false;2;0.4;1.0;Niederspannung +3f6c26dd-842b-4dee-b71f-4aa32e2654ff;"{""type"":""Point"",""coordinates"":[11.8174,53.4281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 97;;;;false;2;0.4;1.0;Niederspannung +c6dac3ab-f44f-4b87-800c-0f4da64673f1;"{""type"":""Point"",""coordinates"":[11.8191,53.422],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 10;;;;false;2;0.4;1.0;Niederspannung +eae8a04c-44f2-4da3-95f6-cae48f85737c;"{""type"":""Point"",""coordinates"":[11.8203,53.4221],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 45;;;;false;2;0.4;1.0;Niederspannung +94fe96b1-f36a-4edd-a107-4ff0376f1066;"{""type"":""Point"",""coordinates"":[11.8175,53.4265],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 61;;;;false;2;0.4;1.0;Niederspannung +31e6e197-719d-4aaf-8ca5-ab9e7549390e;"{""type"":""Point"",""coordinates"":[11.8196,53.4286],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 32;;;;false;2;0.4;1.0;Niederspannung +002a4495-96e4-49c9-abbe-8fccb3e9c83e;"{""type"":""Point"",""coordinates"":[11.8214,53.4229],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 90;;;;false;2;0.4;1.0;Niederspannung +c7e48384-5699-4a38-a887-7e15a9145202;"{""type"":""Point"",""coordinates"":[11.8191,53.4223],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 107;;;;false;2;0.4;1.0;Niederspannung +8ace5c2b-584a-4015-990f-6f1e14de4ddb;"{""type"":""Point"",""coordinates"":[11.8174,53.4273],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 27;;;;false;2;0.4;1.0;Niederspannung +c41ebab8-16a0-4a3a-b4af-26073932d462;"{""type"":""Point"",""coordinates"":[11.8168,53.4285],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 35;;;;false;2;0.4;1.0;Niederspannung +ab8c8f6c-e7a2-4b81-a0d7-5f13789267a2;"{""type"":""Point"",""coordinates"":[11.815,53.4269],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 113;;;;false;2;0.4;1.0;Niederspannung +9f1baf4e-12e1-41d1-8efc-81cfc78f1957;"{""type"":""Point"",""coordinates"":[11.8195,53.426],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 71;;;;false;2;0.4;1.0;Niederspannung +02e63e81-2e62-4ef6-8fdb-0b0905a437b6;"{""type"":""Point"",""coordinates"":[11.8191,53.4212],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 12;;;;false;2;0.4;1.0;Niederspannung +2d33314e-31db-4ad4-a898-2be5d56a896d;"{""type"":""Point"",""coordinates"":[11.8162,53.4277],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 48;;;;false;2;0.4;1.0;Niederspannung +b8e95bf0-3ba8-4d53-a0bf-a3720fb785fb;"{""type"":""Point"",""coordinates"":[11.8195,53.4264],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 49;;;;false;2;0.4;1.0;Niederspannung +d0bfabdb-0e83-423b-a20a-ab9197c4284e;"{""type"":""Point"",""coordinates"":[11.8213,53.4239],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 82;;;;false;2;0.4;1.0;Niederspannung +6b2881ce-3965-4f5e-98a3-74eb47b0a7ca;"{""type"":""Point"",""coordinates"":[11.8162,53.4284],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 53;;;;false;2;0.4;1.0;Niederspannung +af2b07ce-1a96-4b50-9e21-badf29eed519;"{""type"":""Point"",""coordinates"":[11.8198,53.4195],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 100;;;;false;2;0.4;1.0;Niederspannung +844c0b9c-058a-4228-a8c4-bf2defff6958;"{""type"":""Point"",""coordinates"":[11.8213,53.4293],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 1;;;;false;2;0.4;1.0;Niederspannung +a432b8ce-0462-478b-83e7-3107cd2e909c;"{""type"":""Point"",""coordinates"":[11.8213,53.4253],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 56;;;;false;2;0.4;1.0;Niederspannung +4129e079-6712-4275-911c-36729d698c42;"{""type"":""Point"",""coordinates"":[11.8166,53.4238],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 77;;;;false;2;0.4;1.0;Niederspannung +883edf38-9a18-4f61-981a-691aaf436cc7;"{""type"":""Point"",""coordinates"":[11.8198,53.4216],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 4;;;;false;2;0.4;1.0;Niederspannung +f2724954-34d3-4ddf-b6b0-7a1531639990;"{""type"":""Point"",""coordinates"":[11.8191,53.4235],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 6;;;;false;2;0.4;1.0;Niederspannung +5d3bcf55-0520-43ff-8d63-3d0eb421e442;"{""type"":""Point"",""coordinates"":[11.816,53.4234],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 91;;;;false;2;0.4;1.0;Niederspannung +3a557b4e-06b8-4f29-929f-81d95c42c897;"{""type"":""Point"",""coordinates"":[11.8213,53.4277],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 42;;;;false;2;0.4;1.0;Niederspannung +7dc43c81-9a61-45a0-9745-800a28bf4a9d;"{""type"":""Point"",""coordinates"":[11.8196,53.4278],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 50;;;;false;2;0.4;1.0;Niederspannung +fd4bebb8-40ca-4eed-92c0-cdd10b86ac20;"{""type"":""Point"",""coordinates"":[11.8191,53.4209],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 60;;;;false;2;0.4;1.0;Niederspannung +3464496c-7dd4-41e9-ae0a-99ade0b51572;"{""type"":""Point"",""coordinates"":[11.8213,53.4236],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 40;;;;false;2;0.4;1.0;Niederspannung +d38d936a-9c05-4bdc-8331-418fef27f492;"{""type"":""Point"",""coordinates"":[11.815,53.4281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 109;;;;false;2;0.4;1.0;Niederspannung +58b551b6-83bd-4f1c-8d9c-8c9a7f638c0b;"{""type"":""Point"",""coordinates"":[11.8213,53.425],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 38;;;;false;2;0.4;1.0;Niederspannung +b5c1e826-63fd-4b0c-bec0-0c758389ef58;"{""type"":""Point"",""coordinates"":[11.8191,53.4239],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 104;;;;false;2;0.4;1.0;Niederspannung +7c35a794-f569-4a9c-acb0-d03647610086;"{""type"":""Point"",""coordinates"":[11.8166,53.4249],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 98;;;;false;2;0.4;1.0;Niederspannung +78815cf6-70db-432c-96e6-87fe8cf67eee;"{""type"":""Point"",""coordinates"":[11.815,53.4272],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 112;;;;false;2;0.4;1.0;Niederspannung +c5b6bfaf-1621-40a7-9c53-02cfb59c04d9;"{""type"":""Point"",""coordinates"":[11.822,53.4294],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 14;;;;false;2;0.4;1.0;Niederspannung +2b3d7fb8-0583-4d47-97b1-3b5f232fd462;"{""type"":""Point"",""coordinates"":[11.8213,53.4274],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 87;;;;false;2;0.4;1.0;Niederspannung +3fcb94e3-7781-4d83-9030-d9853822e78e;"{""type"":""Point"",""coordinates"":[11.8213,53.4243],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 85;;;;false;2;0.4;1.0;Niederspannung +f713593a-3fd3-410a-ac08-74202d4f5798;"{""type"":""Point"",""coordinates"":[11.8201,53.4236],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 79;;;;false;2;0.4;1.0;Niederspannung +47246a84-ad0e-4d04-9d98-1c9cd5d363c1;"{""type"":""Point"",""coordinates"":[11.8196,53.4289],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 16;;;;false;2;0.4;1.0;Niederspannung +9cdb3115-cc00-4d61-bc33-442e8f30fb63;"{""type"":""Point"",""coordinates"":[11.8213,53.4257],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 83;;;;false;2;0.4;1.0;Niederspannung +839ff0f4-93db-42ec-a928-bbc448b6cf5c;"{""type"":""Point"",""coordinates"":[11.8197,53.4207],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 69;;;;false;2;0.4;1.0;Niederspannung +e7908208-77b4-4059-806e-4857262992fc;"{""type"":""Point"",""coordinates"":[11.8196,53.4247],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 105;;;;false;2;0.4;1.0;Niederspannung +83da8d60-405a-45f7-9bb9-9d35607b7927;"{""type"":""Point"",""coordinates"":[11.8201,53.4229],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 2;;;;false;2;0.4;1.0;Niederspannung +9d136a6b-5fdc-44ed-a5ed-599a55281024;"{""type"":""Point"",""coordinates"":[11.8195,53.4275],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 9;;;;false;2;0.4;1.0;Niederspannung +3faac527-0ff3-44a7-9e4f-24a41940da90;"{""type"":""Point"",""coordinates"":[11.816,53.4227],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 15;;;;false;2;0.4;1.0;Niederspannung +5682ac05-7336-4ebc-a5d1-3b69c79fb3b1;"{""type"":""Point"",""coordinates"":[11.8166,53.4241],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 44;;;;false;2;0.4;1.0;Niederspannung +5b73ded9-3ca4-4f18-a2ab-c27b9a3dcf9d;"{""type"":""Point"",""coordinates"":[11.8166,53.423],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 36;;;;false;2;0.4;1.0;Niederspannung +594d101c-3a05-45e3-a061-9189f3e848b7;"{""type"":""Point"",""coordinates"":[11.8198,53.4192],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 88;;;;false;2;0.4;1.0;Niederspannung +7b81b518-00e0-4ff1-b4cf-876903958d7a;"{""type"":""Point"",""coordinates"":[11.8213,53.4271],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 8;;;;false;2;0.4;1.0;Niederspannung +a882e666-82d1-4ba6-87df-fc702fe06187;"{""type"":""Point"",""coordinates"":[11.8169,53.4289],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 74;;;;false;2;0.4;1.0;Niederspannung +1403edf9-e47c-4705-8563-83bcd639482e;"{""type"":""Point"",""coordinates"":[11.8213,53.4286],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 75;;;;false;2;0.4;1.0;Niederspannung +03b2aa45-84f6-48c0-9dab-427e046a5672;"{""type"":""Point"",""coordinates"":[11.8196,53.4253],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 22;;;;false;2;0.4;1.0;Niederspannung +6ee7ea93-ea9c-40cb-b79a-1c5f287c97a5;"{""type"":""Point"",""coordinates"":[11.8221,53.4286],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 66;;;;false;2;0.4;1.0;Niederspannung +88cf719a-92df-4dfd-9a83-f84330e28fe0;"{""type"":""Point"",""coordinates"":[11.815,53.4285],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 108;;;;false;2;0.4;1.0;Niederspannung +80962bd3-a10f-4ed2-ba6a-3e802189939c;"{""type"":""Point"",""coordinates"":[11.8201,53.4239],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 21;;;;false;2;0.4;1.0;Niederspannung +80b8d1f8-7e83-421d-a95a-c193fc35f4f7;"{""type"":""Point"",""coordinates"":[11.8175,53.427],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 95;;;;false;2;0.4;1.0;Niederspannung +b909fb45-b6ee-427f-afd7-e8a0ec7274c6;"{""type"":""Point"",""coordinates"":[11.8213,53.4225],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 92;;;;false;2;0.4;1.0;Niederspannung +57b40047-4f9d-46bb-bf19-c0a86bbd4f5b;"{""type"":""Point"",""coordinates"":[11.8166,53.4234],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 37;;;;false;2;0.4;1.0;Niederspannung +18b4157b-0e47-4c5a-adb8-ccae47372336;"{""type"":""Point"",""coordinates"":[11.8213,53.4264],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 5;;;;false;2;0.4;1.0;Niederspannung +8726dc29-621e-4455-a541-cd88d7da457f;"{""type"":""Point"",""coordinates"":[11.8166,53.4257],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 96;;;;false;2;0.4;1.0;Niederspannung +814f784b-687f-4dd5-8a91-c7772c916d46;"{""type"":""Point"",""coordinates"":[11.8174,53.4262],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 19;;;;false;2;0.4;1.0;Niederspannung +7efabb8d-ba17-4487-96d9-5744b1fedf8a;"{""type"":""Point"",""coordinates"":[11.8166,53.4245],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 23;;;;false;2;0.4;1.0;Niederspannung +15a86f7d-fb73-49a4-af6a-25b14122378d;"{""type"":""Point"",""coordinates"":[11.8191,53.4227],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 11;;;;false;2;0.4;1.0;Niederspannung +34031e92-3444-47d5-94ae-cceeb5d96bb2;"{""type"":""Point"",""coordinates"":[11.8195,53.4268],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 106;;;;false;2;0.4;1.0;Niederspannung +35748e60-3be8-4930-8a61-209fd5df1bec;"{""type"":""Point"",""coordinates"":[11.815,53.4293],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 30;;;;false;2;0.4;1.0;Niederspannung +50cac08e-bf24-4526-9466-53ca5edccd15;"{""type"":""Point"",""coordinates"":[11.8166,53.4253],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 64;;;;false;2;0.4;1.0;Niederspannung +2efac9b1-fb0d-4e08-bfac-501798826deb;"{""type"":""Point"",""coordinates"":[11.8191,53.4231],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 93;;;;false;2;0.4;1.0;Niederspannung +c86d6361-4159-4787-b5f4-e41dcaa95195;"{""type"":""Point"",""coordinates"":[11.8182,53.4257],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 13;;;;false;2;0.4;1.0;Niederspannung diff --git a/input/samples/simopsimtestgrid_reduced_reference/fullGrid/pv_input.csv b/input/samples/simopsimtestgrid_reduced_reference/fullGrid/pv_input.csv new file mode 100644 index 0000000000..b883b51783 --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced_reference/fullGrid/pv_input.csv @@ -0,0 +1,4 @@ +uuid;albedo;azimuth;cos_phi_rated;elevation_angle;eta_conv;id;k_g;k_t;market_reaction;node;operates_from;operates_until;operator;q_characteristics;s_rated;em +a1eb7fc1-3bee-4b65-a387-ef3046644bf0;0.20000000298023224;4.093344211578369;0.8999999761581421;37.69556427001953;97.0;LV5.201 PV 10;0.8999999761581421;1.0;false;dc54bd8a-b7d8-4e99-adb0-d6ee5084241c;;;;"cosPhiFixed:{(0.0,0.9)}";4.2;f9dc7ce6-658c-4101-a12f-d58bb889286b +de8cfef5-7620-4b9e-9a10-1faebb5a80c0;0.20000000298023224;4.093344211578369;0.8999999761581421;37.69556427001953;97.0;LV5.201 PV 11;0.8999999761581421;1.0;false;3e6be3ac-2b51-4080-b815-391313612fc7;;;;"cosPhiFixed:{(0.0,0.9)}";10.9;957938b7-0476-4fab-a1b3-6ce8615857b3 +2560c371-f420-4c2a-b4e6-e04c11b64c03;0.20000000298023224;0.7802008390426636;0.8999999761581421;40.086585998535156;98.0;LV5.201 PV 15;0.8999999761581421;1.0;false;43040a39-8b6c-401f-9dfd-82b42aa6dec6;;;;"cosPhiFixed:{(0.0,0.9)}";2.9;c3a7e9f5-b492-4c85-af2d-1e93f6a25443 diff --git a/input/samples/simopsimtestgrid_reduced_reference/fullGrid/transformer_2_w_input.csv b/input/samples/simopsimtestgrid_reduced_reference/fullGrid/transformer_2_w_input.csv new file mode 100644 index 0000000000..d1f9f5d13c --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced_reference/fullGrid/transformer_2_w_input.csv @@ -0,0 +1,2 @@ +uuid;auto_tap;id;node_a;node_b;operates_from;operates_until;operator;parallel_devices;tap_pos;type +adaba416-9b52-45df-9d91-f67d0dd28ecb;false;MV2.101-LV5.201-Trafo 1;ec8f2c82-a1b2-487c-b573-250859e3b414;4749ab2b-4d96-4100-8081-73e77c797d6b;;;;1;0;417407d2-1e74-4f37-9b64-f701f53f8842 diff --git a/input/samples/simopsimtestgrid_reduced_reference/fullGrid/transformer_2_w_type_input.csv b/input/samples/simopsimtestgrid_reduced_reference/fullGrid/transformer_2_w_type_input.csv new file mode 100644 index 0000000000..b13848b159 --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced_reference/fullGrid/transformer_2_w_type_input.csv @@ -0,0 +1,2 @@ +uuid;b_m;d_phi;d_v;g_m;id;r_sc;s_rated;tap_max;tap_min;tap_neutr;tap_side;v_rated_a;v_rated_b;x_sc +417407d2-1e74-4f37-9b64-f701f53f8842;-36.47380569074435;0.0;2.5;4124.999999999999;0.63 MVA 20/0.4 kV Dyn5 ASEA;6.953892668178382;630.0;2;-2;0;false;20.0;0.4;37.45518044666632 diff --git a/input/samples/simopsimtestgrid_reduced_reference/simopsimtestgrid.conf b/input/samples/simopsimtestgrid_reduced_reference/simopsimtestgrid.conf new file mode 100644 index 0000000000..44fbc2f8e4 --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced_reference/simopsimtestgrid.conf @@ -0,0 +1,240 @@ +include "../common/pekko.conf" + +######### +# ATTENTION: Do not change this config file directly but use it as a base for your personal delta config for the +# vn_simona scenario! Delta configs can be created by including the config you want to change +# parameters from via include (e.g. include "input/samples/vn_simona/vn_simona.conf") at the +# beginning of your config file and then just override the parameters you want to change! +######### + +################################################################## +# Simulation Parameters +################################################################## +simona.simulationName = "simopsimtest" + +################################################################## +# Time Parameters +################################################################## +simona.time.startDateTime = "2024-02-27T00:00:00Z" +simona.time.endDateTime = "2024-02-28T00:15:00Z" +simona.time.schedulerReadyCheckWindow = 900 + +################################################################## +# Input Parameters +################################################################## +simona.input.primary.csvParams = { + directoryPath: "simona/input/samples/simopsimtestgrid_reduced_reference/fullGrid" + csvSep: ";" + isHierarchic: false +} +simona.input.grid.datasource.id = "csv" +simona.input.grid.datasource.csvParams = { + directoryPath: "simona/input/samples/simopsimtestgrid_reduced_reference/fullGrid" + csvSep: ";" + isHierarchic: false +} + +simona.input.weather.datasource = { + scheme = "icon" + sampleParams.use = true + coordinateSource.sampleParams.use = true + maxCoordinateDistance = 50000 +} + +################################################################## +# Output Parameters +################################################################## +simona.output.base.dir = "simona/output/simopsimtestgrid_reduced_reference" +simona.output.base.addTimestampToOutputDir = true + +simona.output.sink.csv { + fileFormat = ".csv" + filePrefix = "" + fileSuffix = "" +} + +simona.output.grid = { + notifier = "grid" + nodes = false + lines = false + switches = false + transformers2w = false + transformers3w = false +} +simona.output.participant.defaultConfig = { + notifier = "default" + powerRequestReply = false + simulationResult = true +} +simona.output.participant.individualConfigs = [ + { + notifier = "pv" + powerRequestReply = false + simulationResult = true + }, + { + notifier = "wec" + powerRequestReply = false + simulationResult = false + }, + { + notifier = "evcs" + powerRequestReply = false + simulationResult = false + }, + { + notifier = "bm" + powerRequestReply = false + simulationResult = false + }, + { + notifier = "chp" + powerRequestReply = false + simulationResult = false + }, + { + notifier = "ev" + powerRequestReply = false + simulationResult = false + }, + { + notifier = "hp" + powerRequestReply = false + simulationResult = false + }, + { + notifier = "storage" + powerRequestReply = false + simulationResult = false + }, + { + notifier = "fixedFeedIn" + powerRequestReply = false + simulationResult = false + } +] +simona.output.thermal = { + defaultConfig = { + notifier = "default", + simulationResult = false + } + individualConfigs = [ + { + notifier = "house", + simulationResult = false + } + ] +} + +################################################################## +# Runtime Configuration // todo refactor as this naming is misleading and partly unneeded +################################################################## +simona.runtime.selected_subgrids = [] +simona.runtime.selected_volt_lvls = [] + +simona.runtime.participant.load = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + modelBehaviour = "profile" + reference = "power" + } + individualConfigs = [] +} + +simona.runtime.participant.fixedFeedIn = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +simona.runtime.participant.pv = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +simona.runtime.participant.wec = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +simona.runtime.participant.evcs = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +simona.runtime.participant.hp = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +# # # # # +# ATTENTION: calculateMissingReactivePowerWithModel and scaling is ignored here. +# # # # # +simona.runtime.participant.em = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +################################################################## +# Event Configuration +################################################################## +simona.event.listener = [] + +################################################################## +# Grid Configuration +################################################################## + +simona.gridConfig.refSystems = [ + {sNom = "100 kVA", vNom = "0.4 kV", voltLvls = [{id = "NS", vNom = "0.4 kV"}]}, + {sNom = "60 MVA", vNom = "20 kV", voltLvls = [{id = "MS", vNom = "20 kV"}]}, + {sNom = "600 MVA", vNom = "110 kV", voltLvls = [{id = "HS", vNom = "110 kV"}]}, + {sNom = "1000 MVA", vNom = "380 kV", voltLvls = [{id = "HoeS", vNom = "380 kV"}]} +] + +################################################################## +# Power Flow Configuration +################################################################## +simona.powerflow.maxSweepPowerDeviation = 1E-5 // the maximum allowed deviation in power between two sweeps, before overall convergence is assumed +simona.powerflow.newtonraphson.epsilon = [1E-12] +simona.powerflow.newtonraphson.iterations = 50 +simona.powerflow.resolution = "86400s" +simona.powerflow.stopOnFailure = true + +simona.control.transformer = [ + { + transformers = ["31a2b9bf-e785-4475-aa44-1c34646e8c79"], + measurements = ["923f2d69-3093-4198-86e4-13d2d1c220f8"], + vMin = 0.98, + vMax = 1.02 + }, { + transformers = ["1132dbf4-e8a1-44ae-8415-f42d4497aa1d"], + measurements = ["7686b818-a0ba-465c-8e4e-f7d3c4e171fc"], + vMin = 0.98, + vMax = 1.02 + } +] diff --git a/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala b/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala index ace0a87412..80cdefa3c1 100644 --- a/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala +++ b/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala @@ -323,7 +323,7 @@ object EmAgent { ) } else { // We got set points setPower = emData.extEmDataServiceData.dataProvisionMessage.map(setPoint => setPoint.setPower).getOrElse(throw new RuntimeException("Got a wrong set point!")) - ctx.log.info(s"[UNCONTROLLED] EM Agent ${ctx.self}: Got a external Set Power = $setPower") + ctx.log.debug(s"[UNCONTROLLED] EM Agent ${ctx.self}: Got a external Set Power = $setPower") updatedEmData = emData.copy( extEmDataServiceData = emData.extEmDataServiceData.copy( dataProvisionMessage = None diff --git a/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala b/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala index 8ca17b9ab2..659d16a4e3 100644 --- a/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala +++ b/src/main/scala/edu/ie3/simona/model/em/EmModelShell.scala @@ -120,7 +120,7 @@ object EmModelShell { case "PROPORTIONAL" => ProportionalFlexStrat case "PRIORITIZED" => PrioritizedFlexStrat(modelConfig.curtailRegenerative) - case "self_optimization" => PrioritizedFlexStrat(modelConfig.curtailRegenerative) + case "self_optimization" => ProportionalFlexStrat case "uncontrolled" => UncontrolledStrat(modelConfig.curtailRegenerative) case unknown => throw new CriticalFailureException(s"Unknown model strategy $unknown") From 575955de75e7f5e1dcd77600694bd3d6708db88e Mon Sep 17 00:00:00 2001 From: smjobaoo Date: Fri, 24 May 2024 08:54:43 +0200 Subject: [PATCH 31/41] ext address as argument --- .../fullGrid/load_input.csv | 6 +- .../fullGrid/pv_input.csv | 6 +- .../agent/grid/GridAgentController.scala | 22 +- .../agent/participant/ParticipantAgent.scala | 2 +- .../ParticipantAgentFundamentals.scala | 4 - .../edu/ie3/simona/config/ArgsParser.scala | 10 + .../ie3/simona/main/RunSimonaWithMosaik.scala | 4 + .../ie3/simona/main/RunSimonaWithOpsim.scala | 1 + .../primary/ExtPrimaryDataService.scala | 1 + .../results/ExtResultDataProvider.scala | 40 ++- .../simona/sim/setup/SimonaMosaikSetup.scala | 18 +- .../simona/sim/setup/SimonaOpsimSetup.scala | 160 ++-------- .../SimonaSimpleExtSimulationSetup.scala | 140 +-------- .../sim/setup/SimonaStandaloneSetup.scala | 294 +----------------- .../edu/ie3/simona/util/ReceiveDataMap.scala | 2 + 15 files changed, 107 insertions(+), 603 deletions(-) diff --git a/input/samples/simopsimtestgrid_reduced_reference/fullGrid/load_input.csv b/input/samples/simopsimtestgrid_reduced_reference/fullGrid/load_input.csv index 6ad16189a5..305aa1c8d2 100644 --- a/input/samples/simopsimtestgrid_reduced_reference/fullGrid/load_input.csv +++ b/input/samples/simopsimtestgrid_reduced_reference/fullGrid/load_input.csv @@ -1,4 +1,4 @@ uuid;cos_phi_rated;dsm;e_cons_annual;id;load_profile;node;operates_from;operates_until;operator;q_characteristics;s_rated;em -c3434742-e4f0-49e5-baa7-c1e3045c732c;0.93;false;0.0;LV5.201 Load 64;h0;dc54bd8a-b7d8-4e99-adb0-d6ee5084241c;;;;"cosPhiFixed:{(0.0,0.93)}";3.2;f9dc7ce6-658c-4101-a12f-d58bb889286b -fd2e19b6-d5e3-4776-9456-8787a2160d9d;0.93;false;0.0;LV5.201 Load 74;h0;43040a39-8b6c-401f-9dfd-82b42aa6dec6;;;;"cosPhiFixed:{(0.0,0.93)}";1.1;c3a7e9f5-b492-4c85-af2d-1e93f6a25443 -98c1a2ab-bd09-4c77-a389-d088aed894b1;0.93;false;0.0;LV5.201 Load 102;h0;3e6be3ac-2b51-4080-b815-391313612fc7;;;;"cosPhiFixed:{(0.0,0.93)}";5.3;957938b7-0476-4fab-a1b3-6ce8615857b3 +c3434742-e4f0-49e5-baa7-c1e3045c732c;0.93;false;0.0;LV5.201 Load 64;h0;dc54bd8a-b7d8-4e99-adb0-d6ee5084241c;;;;"cosPhiFixed:{(0.0,0.93)}";30.2;f9dc7ce6-658c-4101-a12f-d58bb889286b +fd2e19b6-d5e3-4776-9456-8787a2160d9d;0.93;false;0.0;LV5.201 Load 74;h0;43040a39-8b6c-401f-9dfd-82b42aa6dec6;;;;"cosPhiFixed:{(0.0,0.93)}";11.0;c3a7e9f5-b492-4c85-af2d-1e93f6a25443 +98c1a2ab-bd09-4c77-a389-d088aed894b1;0.93;false;0.0;LV5.201 Load 102;h0;3e6be3ac-2b51-4080-b815-391313612fc7;;;;"cosPhiFixed:{(0.0,0.93)}";50.3;957938b7-0476-4fab-a1b3-6ce8615857b3 diff --git a/input/samples/simopsimtestgrid_reduced_reference/fullGrid/pv_input.csv b/input/samples/simopsimtestgrid_reduced_reference/fullGrid/pv_input.csv index b883b51783..cb4eeaa67e 100644 --- a/input/samples/simopsimtestgrid_reduced_reference/fullGrid/pv_input.csv +++ b/input/samples/simopsimtestgrid_reduced_reference/fullGrid/pv_input.csv @@ -1,4 +1,4 @@ uuid;albedo;azimuth;cos_phi_rated;elevation_angle;eta_conv;id;k_g;k_t;market_reaction;node;operates_from;operates_until;operator;q_characteristics;s_rated;em -a1eb7fc1-3bee-4b65-a387-ef3046644bf0;0.20000000298023224;4.093344211578369;0.8999999761581421;37.69556427001953;97.0;LV5.201 PV 10;0.8999999761581421;1.0;false;dc54bd8a-b7d8-4e99-adb0-d6ee5084241c;;;;"cosPhiFixed:{(0.0,0.9)}";4.2;f9dc7ce6-658c-4101-a12f-d58bb889286b -de8cfef5-7620-4b9e-9a10-1faebb5a80c0;0.20000000298023224;4.093344211578369;0.8999999761581421;37.69556427001953;97.0;LV5.201 PV 11;0.8999999761581421;1.0;false;3e6be3ac-2b51-4080-b815-391313612fc7;;;;"cosPhiFixed:{(0.0,0.9)}";10.9;957938b7-0476-4fab-a1b3-6ce8615857b3 -2560c371-f420-4c2a-b4e6-e04c11b64c03;0.20000000298023224;0.7802008390426636;0.8999999761581421;40.086585998535156;98.0;LV5.201 PV 15;0.8999999761581421;1.0;false;43040a39-8b6c-401f-9dfd-82b42aa6dec6;;;;"cosPhiFixed:{(0.0,0.9)}";2.9;c3a7e9f5-b492-4c85-af2d-1e93f6a25443 +a1eb7fc1-3bee-4b65-a387-ef3046644bf0;0.20000000298023224;4.093344211578369;0.8999999761581421;37.69556427001953;97.0;LV5.201 PV 10;0.8999999761581421;1.0;false;dc54bd8a-b7d8-4e99-adb0-d6ee5084241c;;;;"cosPhiFixed:{(0.0,0.9)}";40.2;f9dc7ce6-658c-4101-a12f-d58bb889286b +de8cfef5-7620-4b9e-9a10-1faebb5a80c0;0.20000000298023224;4.093344211578369;0.8999999761581421;37.69556427001953;97.0;LV5.201 PV 11;0.8999999761581421;1.0;false;3e6be3ac-2b51-4080-b815-391313612fc7;;;;"cosPhiFixed:{(0.0,0.9)}";100.9;957938b7-0476-4fab-a1b3-6ce8615857b3 +2560c371-f420-4c2a-b4e6-e04c11b64c03;0.20000000298023224;0.7802008390426636;0.8999999761581421;40.086585998535156;98.0;LV5.201 PV 15;0.8999999761581421;1.0;false;43040a39-8b6c-401f-9dfd-82b42aa6dec6;;;;"cosPhiFixed:{(0.0,0.9)}";29.0;c3a7e9f5-b492-4c85-af2d-1e93f6a25443 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 7936d075c8..e32001d994 100644 --- a/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala +++ b/src/main/scala/edu/ie3/simona/agent/grid/GridAgentController.scala @@ -200,12 +200,12 @@ class GridAgentController( extEmDataService = environmentRefs.emDataService ) - log.info(s"Built allEms = $allEms") - log.info(s"Particpants = $participants") + //log.info(s"Built allEms = $allEms") + //log.info(s"Particpants = $participants") participants .map { participant => - log.info(s"Built Participant = $participant") + //log.info(s"Built Participant = $participant") val node = participant.getNode @@ -278,8 +278,8 @@ class GridAgentController( val previousLevelAndUncontrolledEms = previousLevelEms ++ uncontrolledEms.toMap - log.info(s"controlledEmInputs = $controlledEmInputs") - log.info(s"previousLevelAndUncontrolledEms = $previousLevelAndUncontrolledEms") + //log.info(s"controlledEmInputs = $controlledEmInputs") + //log.info(s"previousLevelAndUncontrolledEms = $previousLevelAndUncontrolledEms") if (controlledEmInputs.nonEmpty) { // For controlled EMs at the current level, more EMs @@ -288,7 +288,7 @@ class GridAgentController( case (uuid, emInput) => emInput.getControllingEm.toScala.map(uuid -> _) } - log.info(s"now build controlled Ems -> These are possible controllers = $controllingEms") + //log.info(s"now build controlled Ems -> These are possible controllers = $controllingEms") // Return value includes previous level and uncontrolled EMs of this level val recursiveEms = buildEmsRecursively( participantConfigUtil, @@ -297,10 +297,10 @@ class GridAgentController( previousLevelAndUncontrolledEms, extEmDataService = extEmDataService ) - log.info(s"-> after recursion recursiveEms = $recursiveEms") + //log.info(s"-> after recursion recursiveEms = $recursiveEms") val controlledEms = controlledEmInputs.map { case (uuid, emInput) => - log.info(s"-> uuid = $uuid, emInput = $emInput") + //log.info(s"-> uuid = $uuid, emInput = $emInput") /* val controllingEm = emInput.getControllingEm.toScala .map(_.getUuid) @@ -320,7 +320,7 @@ class GridAgentController( s"Actor for EM $uuid not found." ), )) - log.info(s"-> contorllingEm = $controllingEm") + //log.info(s"-> contorllingEm = $controllingEm") uuid -> buildEm( emInput, @@ -329,7 +329,7 @@ class GridAgentController( maybeControllingEm = controllingEm, ) }.toMap - log.info(s"-> controlledEms = $controlledEms, recursiveEms = $recursiveEms") + //log.info(s"-> controlledEms = $controlledEms, recursiveEms = $recursiveEms") recursiveEms ++ controlledEms } else { previousLevelAndUncontrolledEms @@ -822,7 +822,7 @@ class GridAgentController( outputConfig: NotifierConfig, maybeControllingEm: Option[ActorRef[FlexResponse]], ): ActorRef[FlexResponse] = { - log.info("Spawn Em = " + emInput + ", maybeControlling Em = " + maybeControllingEm) + //log.info("Spawn Em = " + emInput + ", maybeControlling Em = " + maybeControllingEm) gridAgentContext.spawn( EmAgent( emInput, diff --git a/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgent.scala b/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgent.scala index e42c3f2de2..a01c5677a1 100644 --- a/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgent.scala +++ b/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgent.scala @@ -301,7 +301,7 @@ abstract class ParticipantAgent[ maybeEmAgent, ), ) => - log.info("Will perform model calculations") + log.debug("Will perform model calculations") initializeParticipantForModelCalculation( inputModel, modelConfig, 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 71f092d3c8..9b3b8580eb 100644 --- a/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgentFundamentals.scala +++ b/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgentFundamentals.scala @@ -275,12 +275,8 @@ protected trait ParticipantAgentFundamentals[ val awaitRegistrationResponsesFrom = registerForServices(inputModel.electricalInputModel, services) - log.info(s"Participant Agent maybeEmAgent $maybeEmAgent") // register with EM if applicable maybeEmAgent.foreach { emAgent => - log.info(s"Agent ${ - inputModel.electricalInputModel.getUuid - } register for Em!") emAgent ! RegisterParticipant( inputModel.electricalInputModel.getUuid, self.toTyped[FlexRequest], diff --git a/src/main/scala/edu/ie3/simona/config/ArgsParser.scala b/src/main/scala/edu/ie3/simona/config/ArgsParser.scala index 183aa8149c..affb8b91ac 100644 --- a/src/main/scala/edu/ie3/simona/config/ArgsParser.scala +++ b/src/main/scala/edu/ie3/simona/config/ArgsParser.scala @@ -31,6 +31,7 @@ object ArgsParser extends LazyLogging { seedAddress: Option[String] = None, useLocalWorker: Option[Boolean] = None, tArgs: Map[String, String] = Map.empty, + extAddress: Option[String] = None ) { val useCluster: Boolean = clusterType.isDefined } @@ -102,6 +103,15 @@ object ArgsParser extends LazyLogging { "If cluster is specified then this defaults to false and must be explicitly set to true. " + "NOTE: For cluster, this will ONLY be checked if cluster-type=master" ) + opt[String]("ext-address") + .action((value, args) => args.copy(extAddress = Option(value))) + .validate(value => + if (value.trim.isEmpty) failure("ext-address cannot be empty") + else success + ) + .text( + "Comma separated list (no whitespaces!) of initial addresses used for the rest of the cluster to bootstrap" + ) checkConfig(args => if ( diff --git a/src/main/scala/edu/ie3/simona/main/RunSimonaWithMosaik.scala b/src/main/scala/edu/ie3/simona/main/RunSimonaWithMosaik.scala index 5c5d13ce4d..0c840831b0 100644 --- a/src/main/scala/edu/ie3/simona/main/RunSimonaWithMosaik.scala +++ b/src/main/scala/edu/ie3/simona/main/RunSimonaWithMosaik.scala @@ -29,6 +29,9 @@ object RunSimonaWithMosaik extends RunSimona[SimonaMosaikSetup] { // get the config and prepare it with the provided args val (arguments, parsedConfig) = ArgsParser.prepareConfig(args) + print("arguments:") + print(arguments) + // config fail fast check val simonaConfig = SimonaConfig(parsedConfig) ConfigFailFast.check(parsedConfig, simonaConfig) @@ -37,6 +40,7 @@ object RunSimonaWithMosaik extends RunSimona[SimonaMosaikSetup] { parsedConfig, SimonaMosaikSetup.buildResultFileHierarchy(parsedConfig), mainArgs = arguments.mainArgs, + mosaikIP = arguments.extAddress ) } diff --git a/src/main/scala/edu/ie3/simona/main/RunSimonaWithOpsim.scala b/src/main/scala/edu/ie3/simona/main/RunSimonaWithOpsim.scala index 9d3c54dfa8..849db51884 100644 --- a/src/main/scala/edu/ie3/simona/main/RunSimonaWithOpsim.scala +++ b/src/main/scala/edu/ie3/simona/main/RunSimonaWithOpsim.scala @@ -37,6 +37,7 @@ object RunSimonaWithOpsim extends RunSimona[SimonaOpsimSetup] { parsedConfig, SimonaOpsimSetup.buildResultFileHierarchy(parsedConfig), mainArgs = arguments.mainArgs, + opsimIP = arguments.extAddress ) } diff --git a/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala b/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala index 9e7b804b64..5a3eed9f82 100644 --- a/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala +++ b/src/main/scala/edu/ie3/simona/service/primary/ExtPrimaryDataService.scala @@ -164,6 +164,7 @@ final case class ExtPrimaryDataService( ExtPrimaryDataStateData, Option[Long], ) = { + log.debug(s"Got activation to distribute primaryData = $primaryData") val actorToPrimaryData = primaryData.asScala.flatMap { case (agent, primaryDataPerAgent) => serviceStateData.uuidToActorRef diff --git a/src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala b/src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala index 53bb68c8cf..c57319029d 100644 --- a/src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala +++ b/src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala @@ -100,15 +100,20 @@ object ExtResultDataProvider { buffer: StashBuffer[Request]): Behavior[Request] = { Behaviors.receivePartial { case (_, WrappedActivation(Activation(INIT_SIM_TICK))) => - val initSubscribers = initServiceData.extResultData.getResultDataAssets.asScala.toList + val initGridSubscribers = initServiceData.extResultData.getGridResultDataAssets.asScala.toList + val initParticipantSubscribers = initServiceData.extResultData.getParticpantResultDataAssets.asScala.toList var initResultStorage = Map.empty[UUID, (Option[ResultEntity], Option[Long])] - initSubscribers.foreach( + initParticipantSubscribers.foreach( uuid => initResultStorage = initResultStorage + (uuid -> (None, Some(0L))) ) + initGridSubscribers.foreach( + uuid => initResultStorage = initResultStorage + (uuid -> (None, Some(initServiceData.extResultData.getPowerFlowResolution))) + ) val resultInitializedStateData = ExtResultStateData( extResultData = initServiceData.extResultData, - subscribers = initSubscribers, + gridSubscribers = initGridSubscribers, + participantSubscribers = initParticipantSubscribers, resultStorage = initResultStorage ) scheduler ! Completion( @@ -138,7 +143,12 @@ object ExtResultDataProvider { ) match { case msg: RequestResultEntities => //ctx.log.info(s"[requestResults] for tick ${msg.tick} and resultStorage ${serviceStateData.resultStorage}") - var receiveDataMap = ReceiveDataMap[UUID, ResultEntity](serviceStateData.subscribers.toSet) + var receiveDataMap = ReceiveDataMap.empty[UUID, ResultEntity] + if (activation.tick == 0L) { + receiveDataMap = ReceiveDataMap[UUID, ResultEntity](serviceStateData.participantSubscribers.toSet) + } else { + receiveDataMap = ReceiveDataMap[UUID, ResultEntity]((serviceStateData.participantSubscribers ++ serviceStateData.gridSubscribers).toSet) + } //ctx.log.info(s"[requestResults] tick ${msg.tick} -> created a receivedatamap " + receiveDataMap) /* serviceStateData.resultStorage.foreach({ @@ -159,13 +169,15 @@ object ExtResultDataProvider { //ctx.log.info(s"[requestResults] tick ${msg.tick} -> requestResults for " + receiveDataMap) + var resultList = List.empty[ResultEntity] if (receiveDataMap.isComplete) { - var resultList = List.empty[ResultEntity] - serviceStateData.resultStorage.values.foreach( - result => resultList = resultList :+ result._1.getOrElse( - throw new RuntimeException("There is no result!") + if (receiveDataMap.getExpectedKeys.nonEmpty) { + serviceStateData.resultStorage.values.foreach( + result => resultList = resultList :+ result._1.getOrElse( + throw new RuntimeException("There is no result!") + ) ) - ) + } //ctx.log.info(s"[requestResults] tick ${msg.tick} -> ReceiveDataMap is complete -> send it right away: " + resultList) // all responses received, forward them to external simulation in a bundle serviceStateData.extResultData.queueExtResponseMsg( @@ -208,7 +220,8 @@ object ExtResultDataProvider { if (serviceStateData.recentResults.isDefined) { // process dataResponses - if (serviceStateData.subscribers.contains(extResultResponseMsg.result.getInputModel)) { + if (serviceStateData.recentResults.getOrElse(throw new Exception("no Receive Data Map!")).getExpectedKeys.contains(extResultResponseMsg.result.getInputModel)) { + //if (serviceStateData.participantSubscribers.contains(extResultResponseMsg.result.getInputModel) || serviceStateData.gridSubscribers.contains(extResultResponseMsg.result.getInputModel)) { //ctx.log.info("[handleDataResponseMessage] Received ResultsResponseMessage with content " + extResultResponseMsg) //ctx.log.info("[handleDataResponseMessage] RecentResults " + serviceStateData.recentResults) val updatedReceivedResults = serviceStateData.recentResults.getOrElse(throw new Exception("noMap")).addData(extResultResponseMsg.result.getInputModel, extResultResponseMsg.result) @@ -254,11 +267,14 @@ object ExtResultDataProvider { // -=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=- final case class ExtResultStateData( extResultData: ExtResultData, - subscribers: List[UUID] = List.empty, + gridSubscribers: List[UUID] = List.empty, + participantSubscribers: List[UUID] = List.empty, extResultsMessage: Option[ResultDataMessageFromExt] = None, resultStorage: Map[UUID, (Option[ResultEntity], Option[Long])] = Map.empty, // UUID -> Result, nextTick maybeNextActivationTick: Option[Long] = None, recentResults: Option[ReceiveDataMap[UUID, ResultEntity]] = None, ) - final case class InitExtResultData(extResultData: ExtResultData) + final case class InitExtResultData( + extResultData: ExtResultData + ) } \ No newline at end of file diff --git a/src/main/scala/edu/ie3/simona/sim/setup/SimonaMosaikSetup.scala b/src/main/scala/edu/ie3/simona/sim/setup/SimonaMosaikSetup.scala index 446f0d7914..ce2ee3776d 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaMosaikSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaMosaikSetup.scala @@ -57,6 +57,7 @@ import org.apache.pekko.actor.typed.{ActorRef, Scheduler} import org.apache.pekko.actor.{ActorRef => ClassicRef} import org.apache.pekko.util.{Timeout => PekkoTimeout} +import java.time.temporal.ChronoUnit import java.util.UUID import java.util.concurrent.LinkedBlockingQueue import scala.concurrent.Await @@ -76,6 +77,7 @@ class SimonaMosaikSetup( resultFileHierarchy: ResultFileHierarchy, runtimeEventQueue: Option[LinkedBlockingQueue[RuntimeEvent]] = None, override val args: Array[String], + mosaikIP: Option[String] = None ) extends SimonaSetup { override def gridAgents( @@ -362,11 +364,17 @@ class SimonaMosaikSetup( rootScheduler: ActorRef[SchedulerMessage], simScheduler: ActorRef[SchedulerMessage] ): ExtSimSetupData = { + val mosaikAddress = mosaikIP.getOrElse( + "127.0.0.1:5678" + ) val simulationStart = TimeUtil.withDefaults.toZonedDateTime( simonaConfig.simona.time.startDateTime ) + val powerFlowResolution = simonaConfig.simona.powerflow.resolution.get( + ChronoUnit.SECONDS + ) val extScheduler = scheduler(context, parent = rootScheduler) - val mosaikExtSim = new MosaikSimulation("127.0.0.1:37699") + val mosaikExtSim = new MosaikSimulation(mosaikAddress) val extSimAdapterPhase1 = context.toClassic.simonaActorOf( ExtSimAdapter.props(extScheduler.toClassic), @@ -439,8 +447,10 @@ class SimonaMosaikSetup( adapterScheduleRef.toClassic, extSimAdapterPhase2, mosaikExtSim.getExtResultDataSimulation.getResultDataFactory, - mosaikExtSim.getExtResultDataSimulation.getResultDataAssets, - simulationStart + mosaikExtSim.getExtResultDataSimulation.getGridResultDataAssets, + mosaikExtSim.getExtResultDataSimulation.getParticipantResultDataAssets, + simulationStart, + powerFlowResolution ) mosaikExtSim.getExtResultDataSimulation.setExtResultData(extResultData) @@ -616,6 +626,7 @@ object SimonaMosaikSetup extends LazyLogging with SetupHelper { resultFileHierarchy: ResultFileHierarchy, runtimeEventQueue: Option[LinkedBlockingQueue[RuntimeEvent]] = None, mainArgs: Array[String] = Array.empty[String], + mosaikIP: Option[String] = None ): SimonaMosaikSetup = new SimonaMosaikSetup( typeSafeConfig, @@ -623,5 +634,6 @@ object SimonaMosaikSetup extends LazyLogging with SetupHelper { resultFileHierarchy, runtimeEventQueue, mainArgs, + mosaikIP ) } diff --git a/src/main/scala/edu/ie3/simona/sim/setup/SimonaOpsimSetup.scala b/src/main/scala/edu/ie3/simona/sim/setup/SimonaOpsimSetup.scala index eef0e2f72c..265e406126 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaOpsimSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaOpsimSetup.scala @@ -57,6 +57,7 @@ import org.apache.pekko.actor.typed.{ActorRef, Scheduler} import org.apache.pekko.actor.{ActorRef => ClassicRef} import org.apache.pekko.util.{Timeout => PekkoTimeout} +import java.time.temporal.ChronoUnit import java.util.UUID import java.util.concurrent.LinkedBlockingQueue import scala.concurrent.Await @@ -76,6 +77,7 @@ class SimonaOpsimSetup( resultFileHierarchy: ResultFileHierarchy, runtimeEventQueue: Option[LinkedBlockingQueue[RuntimeEvent]] = None, override val args: Array[String], + opsimIP: Option[String] = None ) extends SimonaSetup { override def gridAgents( @@ -362,11 +364,15 @@ class SimonaOpsimSetup( rootScheduler: ActorRef[SchedulerMessage], simScheduler: ActorRef[SchedulerMessage], ): ExtSimSetupData = { + val opsimAddress = opsimIP.getOrElse(throw new RuntimeException("Cannot connect to Opsim, because there is no address!")) val simulationStart = TimeUtil.withDefaults.toZonedDateTime( simonaConfig.simona.time.startDateTime ) + val powerFlowResolution = simonaConfig.simona.powerflow.resolution.get( + ChronoUnit.SECONDS + ) val extScheduler = scheduler(context, parent = rootScheduler) - val opsimSim = new OpsimEmSimulator("amqp://guest:guest@localhost:5672/myvhost") + val opsimSim = new OpsimEmSimulator(opsimAddress) val extSimAdapterPhase1 = context.toClassic.simonaActorOf( ExtSimAdapter.props(extScheduler.toClassic), @@ -433,15 +439,16 @@ class SimonaOpsimSetup( val adapterScheduleRef = Await.result( extResultDataProvider.ask[ActorRef[ScheduleServiceActivation]] (ref => RequestScheduleActivationAdapter(ref))(timeout, scheduler2), timeout.duration) - val extResultData = - new ExtResultData( - adapterRef.toClassic, - adapterScheduleRef.toClassic, - extSimAdapterPhase2, - opsimSim.getExtResultDataSimulation.getResultDataFactory, - opsimSim.getExtResultDataSimulation.getResultDataAssets, - simulationStart - ) + val extResultData = new ExtResultData( + adapterRef.toClassic, + adapterScheduleRef.toClassic, + extSimAdapterPhase2, + opsimSim.getExtResultDataSimulation.getResultDataFactory, + opsimSim.getExtResultDataSimulation.getGridResultDataAssets, + opsimSim.getExtResultDataSimulation.getParticipantResultDataAssets, + simulationStart, + powerFlowResolution + ) opsimSim.getExtResultDataSimulation.setExtResultData(extResultData) @@ -487,137 +494,6 @@ class SimonaOpsimSetup( ExtSimSetupData(extSimAdaptersIt, extDataServicesMap, extDataListenerMap, extDatas, Some(extScheduler)) } - - def simpleExtSimulation( - context: ActorContext[_], - rootScheduler: ActorRef[SchedulerMessage], - simScheduler: ActorRef[SchedulerMessage], - ): ExtSimSetupData = { - val simulationStart = TimeUtil.withDefaults.toZonedDateTime( - simonaConfig.simona.time.startDateTime - ) - val extScheduler = scheduler(context, parent = rootScheduler) - val simpleExtSim = new SimpleExtSimulation() - - val extSimAdapterPhase1 = context.toClassic.simonaActorOf( - ExtSimAdapter.props(extScheduler.toClassic), - s"1", - ) - val extSimAdapterPhase2 = context.toClassic.simonaActorOf( - ExtSimAdapter.props(simScheduler.toClassic), - s"2", - ) - - val extSimAdapters: Map[java.lang.Integer, ClassicRef] = Map( - 1.asInstanceOf[java.lang.Integer] -> extSimAdapterPhase1, - 2.asInstanceOf[java.lang.Integer] -> extSimAdapterPhase2 - ) - - val extSimAdapterData = new ExtSimAdapterData(extSimAdapters.asJava, args) - - - - // send init data right away, init activation is scheduled - extSimAdapterPhase1 ! ExtSimAdapter.Create( - extSimAdapterData, - 1, - ScheduleLock.singleKey(context, extScheduler, INIT_SIM_TICK), - ) - - val extPrimaryDataService = context.toClassic.simonaActorOf( - ExtPrimaryDataService.props(extScheduler.toClassic), - s"0-0", - ) - val extPrimaryData = new ExtPrimaryData( - extPrimaryDataService, - extSimAdapterPhase1, - simpleExtSim.getExtPrimaryDataSimulation.getPrimaryDataFactory, - simpleExtSim.getExtPrimaryDataSimulation.getPrimaryDataAssets - ) - - simpleExtSim.getExtPrimaryDataSimulation.setExtPrimaryData(extPrimaryData) - - extPrimaryDataService ! SimonaService.Create( - InitExtPrimaryData(extPrimaryData), - ScheduleLock.singleKey( - context, - extScheduler, - INIT_SIM_TICK, - ), - ) - - //Result Data - - val extResultDataProvider = { - context.spawn( - ExtResultDataProvider(simScheduler), - s"ExtResultDataProvider", - ) - } - - - val timeout: PekkoTimeout = PekkoTimeout.create(5.seconds.toJava) - val scheduler2: Scheduler = context.system.scheduler - - val adapterRef = Await.result( - extResultDataProvider.ask[ActorRef[ResultDataMessageFromExt]] (ref => RequestDataMessageAdapter(ref))(timeout, scheduler2), timeout.duration) - val adapterScheduleRef = Await.result( - extResultDataProvider.ask[ActorRef[ScheduleServiceActivation]] (ref => RequestScheduleActivationAdapter(ref))(timeout, scheduler2), timeout.duration) - - val extResultData = - new ExtResultData( - adapterRef.toClassic, - adapterScheduleRef.toClassic, - extSimAdapterPhase2, - simpleExtSim.getExtResultDataSimulation.getResultDataFactory, - simpleExtSim.getExtResultDataSimulation.getResultDataAssets, - simulationStart - ) - - simpleExtSim.getExtResultDataSimulation.setExtResultData(extResultData) - - extResultDataProvider ! ExtResultDataProvider.Create( - InitExtResultData(extResultData), - ScheduleLock.singleKey( - context, - simScheduler, - INIT_SIM_TICK, - ), - ) - - - val simpleExtSimDatas: List[ExtData] = List( - extResultData, - extPrimaryData - ) - - simpleExtSim.setup( - extSimAdapterData, - simpleExtSimDatas.asJava, - ) - // starting external simulation - new Thread(simpleExtSim, s"External simulation") - .start() - - val extDataServicesMap: Map[Class[_], ClassicRef] = Map( - classOf[ExtPrimaryDataService] -> extPrimaryDataService, - ) - - val extDataListenerMap: Map[Class[_], ActorRef[ExtResultDataProvider.Request]] = Map( - ExtResultDataProvider.getClass -> extResultDataProvider - ) - - val extSimAdaptersIt = Iterable(extSimAdapterPhase1, extSimAdapterPhase2) - - val extDatas = simpleExtSimDatas.toSet - extSimAdapterPhase2 ! ExtSimAdapter.Create( - extSimAdapterData, - 2, - ScheduleLock.singleKey(context, simScheduler, INIT_SIM_TICK), - ) - ExtSimSetupData(extSimAdaptersIt, extDataServicesMap, extDataListenerMap, extDatas, Some(extScheduler)) - } - override def timeAdvancer( context: ActorContext[_], simulation: ActorRef[SimonaSim.SimulationEnded.type], @@ -744,6 +620,7 @@ object SimonaOpsimSetup extends LazyLogging with SetupHelper { resultFileHierarchy: ResultFileHierarchy, runtimeEventQueue: Option[LinkedBlockingQueue[RuntimeEvent]] = None, mainArgs: Array[String] = Array.empty[String], + opsimIP: Option[String] = None ): SimonaOpsimSetup = new SimonaOpsimSetup( typeSafeConfig, @@ -751,5 +628,6 @@ object SimonaOpsimSetup extends LazyLogging with SetupHelper { resultFileHierarchy, runtimeEventQueue, mainArgs, + opsimIP ) } diff --git a/src/main/scala/edu/ie3/simona/sim/setup/SimonaSimpleExtSimulationSetup.scala b/src/main/scala/edu/ie3/simona/sim/setup/SimonaSimpleExtSimulationSetup.scala index e02c601d40..7e2b28de71 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaSimpleExtSimulationSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaSimpleExtSimulationSetup.scala @@ -57,6 +57,7 @@ import org.apache.pekko.actor.typed.{ActorRef, Scheduler} import org.apache.pekko.actor.{ActorRef => ClassicRef} import org.apache.pekko.util.{Timeout => PekkoTimeout} +import java.time.temporal.ChronoUnit import java.util.UUID import java.util.concurrent.LinkedBlockingQueue import scala.concurrent.Await @@ -365,6 +366,9 @@ class SimonaSimpleExtSimulationSetup( val simulationStart = TimeUtil.withDefaults.toZonedDateTime( simonaConfig.simona.time.startDateTime ) + val powerFlowResolution = simonaConfig.simona.powerflow.resolution.get( + ChronoUnit.SECONDS + ) val extScheduler = scheduler(context, parent = rootScheduler) val simpleExtSim = new SimpleExtSimulationWithEm() @@ -439,8 +443,10 @@ class SimonaSimpleExtSimulationSetup( adapterScheduleRef.toClassic, extSimAdapterPhase2, simpleExtSim.getExtResultDataSimulation.getResultDataFactory, - simpleExtSim.getExtResultDataSimulation.getResultDataAssets, - simulationStart + simpleExtSim.getExtResultDataSimulation.getGridResultDataAssets, + simpleExtSim.getExtResultDataSimulation.getParticipantResultDataAssets, + simulationStart, + powerFlowResolution ) simpleExtSim.getExtResultDataSimulation.setExtResultData(extResultData) @@ -488,136 +494,6 @@ class SimonaSimpleExtSimulationSetup( } - def simpleExtSimulation( - context: ActorContext[_], - rootScheduler: ActorRef[SchedulerMessage], - simScheduler: ActorRef[SchedulerMessage], - ): ExtSimSetupData = { - val simulationStart = TimeUtil.withDefaults.toZonedDateTime( - simonaConfig.simona.time.startDateTime - ) - val extScheduler = scheduler(context, parent = rootScheduler) - val simpleExtSim = new SimpleExtSimulation() - - val extSimAdapterPhase1 = context.toClassic.simonaActorOf( - ExtSimAdapter.props(extScheduler.toClassic), - s"1", - ) - val extSimAdapterPhase2 = context.toClassic.simonaActorOf( - ExtSimAdapter.props(simScheduler.toClassic), - s"2", - ) - - val extSimAdapters: Map[java.lang.Integer, ClassicRef] = Map( - 1.asInstanceOf[java.lang.Integer] -> extSimAdapterPhase1, - 2.asInstanceOf[java.lang.Integer] -> extSimAdapterPhase2 - ) - - val extSimAdapterData = new ExtSimAdapterData(extSimAdapters.asJava, args) - - - - // send init data right away, init activation is scheduled - extSimAdapterPhase1 ! ExtSimAdapter.Create( - extSimAdapterData, - 1, - ScheduleLock.singleKey(context, extScheduler, INIT_SIM_TICK), - ) - - val extPrimaryDataService = context.toClassic.simonaActorOf( - ExtPrimaryDataService.props(extScheduler.toClassic), - s"0-0", - ) - val extPrimaryData = new ExtPrimaryData( - extPrimaryDataService, - extSimAdapterPhase1, - simpleExtSim.getExtPrimaryDataSimulation.getPrimaryDataFactory, - simpleExtSim.getExtPrimaryDataSimulation.getPrimaryDataAssets - ) - - simpleExtSim.getExtPrimaryDataSimulation.setExtPrimaryData(extPrimaryData) - - extPrimaryDataService ! SimonaService.Create( - InitExtPrimaryData(extPrimaryData), - ScheduleLock.singleKey( - context, - extScheduler, - INIT_SIM_TICK, - ), - ) - - //Result Data - - val extResultDataProvider = { - context.spawn( - ExtResultDataProvider(simScheduler), - s"ExtResultDataProvider", - ) - } - - - val timeout: PekkoTimeout = PekkoTimeout.create(5.seconds.toJava) - val scheduler2: Scheduler = context.system.scheduler - - val adapterRef = Await.result( - extResultDataProvider.ask[ActorRef[ResultDataMessageFromExt]] (ref => RequestDataMessageAdapter(ref))(timeout, scheduler2), timeout.duration) - val adapterScheduleRef = Await.result( - extResultDataProvider.ask[ActorRef[ScheduleServiceActivation]] (ref => RequestScheduleActivationAdapter(ref))(timeout, scheduler2), timeout.duration) - - val extResultData = - new ExtResultData( - adapterRef.toClassic, - adapterScheduleRef.toClassic, - extSimAdapterPhase2, - simpleExtSim.getExtResultDataSimulation.getResultDataFactory, - simpleExtSim.getExtResultDataSimulation.getResultDataAssets, - simulationStart - ) - - simpleExtSim.getExtResultDataSimulation.setExtResultData(extResultData) - - extResultDataProvider ! ExtResultDataProvider.Create( - InitExtResultData(extResultData), - ScheduleLock.singleKey( - context, - simScheduler, - INIT_SIM_TICK, - ), - ) - - - val simpleExtSimDatas: List[ExtData] = List( - extResultData, - extPrimaryData - ) - - simpleExtSim.setup( - extSimAdapterData, - simpleExtSimDatas.asJava, - ) - // starting external simulation - new Thread(simpleExtSim, s"External simulation") - .start() - - val extDataServicesMap: Map[Class[_], ClassicRef] = Map( - classOf[ExtPrimaryDataService] -> extPrimaryDataService, - ) - - val extDataListenerMap: Map[Class[_], ActorRef[ExtResultDataProvider.Request]] = Map( - ExtResultDataProvider.getClass -> extResultDataProvider - ) - - val extSimAdaptersIt = Iterable(extSimAdapterPhase1, extSimAdapterPhase2) - - val extDatas = simpleExtSimDatas.toSet - extSimAdapterPhase2 ! ExtSimAdapter.Create( - extSimAdapterData, - 2, - ScheduleLock.singleKey(context, simScheduler, INIT_SIM_TICK), - ) - ExtSimSetupData(extSimAdaptersIt, extDataServicesMap, extDataListenerMap, extDatas, Some(extScheduler)) - } - override def timeAdvancer( context: ActorContext[_], simulation: ActorRef[SimonaSim.SimulationEnded.type], 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 3c7f71ea2e..a9044e1b07 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala @@ -357,301 +357,9 @@ class SimonaStandaloneSetup( rootScheduler: ActorRef[SchedulerMessage], simScheduler: ActorRef[SchedulerMessage], ): ExtSimSetupData = { - opsimAsExtSimulation( - context, rootScheduler, simScheduler - ) - //simpleExtSimulation( - // context, rootScheduler, simScheduler - //) - } - def simpleExtSimulation( - context: ActorContext[_], - rootScheduler: ActorRef[SchedulerMessage], - simScheduler: ActorRef[SchedulerMessage], - ): ExtSimSetupData = { - val simulationStart = TimeUtil.withDefaults.toZonedDateTime( - simonaConfig.simona.time.startDateTime - ) - val extScheduler = scheduler(context, parent = rootScheduler) - val simpleExtSim = new SimpleExtSimulation() - - val extSimAdapterPhase1 = context.toClassic.simonaActorOf( - ExtSimAdapter.props(extScheduler.toClassic), - s"1", - ) - val extSimAdapterPhase2 = context.toClassic.simonaActorOf( - ExtSimAdapter.props(simScheduler.toClassic), - s"2", - ) - - val extSimAdapters: Map[java.lang.Integer, ClassicRef] = Map( - 1.asInstanceOf[java.lang.Integer] -> extSimAdapterPhase1, - 2.asInstanceOf[java.lang.Integer] -> extSimAdapterPhase2 - ) - - val extSimAdapterData = new ExtSimAdapterData(extSimAdapters.asJava, args) - - - - // send init data right away, init activation is scheduled - extSimAdapterPhase1 ! ExtSimAdapter.Create( - extSimAdapterData, - 1, - ScheduleLock.singleKey(context, extScheduler, INIT_SIM_TICK), - ) - - val extPrimaryDataService = context.toClassic.simonaActorOf( - ExtPrimaryDataService.props(extScheduler.toClassic), - s"0-0", - ) - val extPrimaryData = new ExtPrimaryData( - extPrimaryDataService, - extSimAdapterPhase1, - simpleExtSim.getExtPrimaryDataSimulation.getPrimaryDataFactory, - simpleExtSim.getExtPrimaryDataSimulation.getPrimaryDataAssets - ) - - simpleExtSim.getExtPrimaryDataSimulation.setExtPrimaryData(extPrimaryData) - - extPrimaryDataService ! SimonaService.Create( - InitExtPrimaryData(extPrimaryData), - ScheduleLock.singleKey( - context, - extScheduler, - INIT_SIM_TICK, - ), - ) - - //Result Data - - val extResultDataProvider = { - context.spawn( - ExtResultDataProvider(simScheduler), - s"ExtResultDataProvider", - ) - } - - - val timeout: PekkoTimeout = PekkoTimeout.create(5.seconds.toJava) - val scheduler2: Scheduler = context.system.scheduler - - val adapterRef = Await.result( - extResultDataProvider.ask[ActorRef[ResultDataMessageFromExt]] (ref => RequestDataMessageAdapter(ref))(timeout, scheduler2), timeout.duration) - val adapterScheduleRef = Await.result( - extResultDataProvider.ask[ActorRef[ScheduleServiceActivation]] (ref => RequestScheduleActivationAdapter(ref))(timeout, scheduler2), timeout.duration) - - val extResultData = - new ExtResultData( - adapterRef.toClassic, - adapterScheduleRef.toClassic, - extSimAdapterPhase2, - simpleExtSim.getExtResultDataSimulation.getResultDataFactory, - simpleExtSim.getExtResultDataSimulation.getResultDataAssets, - simulationStart - ) - - simpleExtSim.getExtResultDataSimulation.setExtResultData(extResultData) - - extResultDataProvider ! ExtResultDataProvider.Create( - InitExtResultData(extResultData), - ScheduleLock.singleKey( - context, - simScheduler, - INIT_SIM_TICK, - ), - ) - - - val simpleExtSimDatas: List[ExtData] = List( - extResultData, - extPrimaryData - ) - - simpleExtSim.setup( - extSimAdapterData, - simpleExtSimDatas.asJava, - ) - // starting external simulation - new Thread(simpleExtSim, s"External simulation") - .start() - - val extDataServicesMap: Map[Class[_], ClassicRef] = Map( - classOf[ExtPrimaryDataService] -> extPrimaryDataService, - ) - - val extDataListenerMap: Map[Class[_], ActorRef[ExtResultDataProvider.Request]] = Map( - ExtResultDataProvider.getClass -> extResultDataProvider - ) - - val extSimAdaptersIt = Iterable(extSimAdapterPhase1, extSimAdapterPhase2) - - val extDatas = simpleExtSimDatas.toSet - extSimAdapterPhase2 ! ExtSimAdapter.Create( - extSimAdapterData, - 2, - ScheduleLock.singleKey(context, simScheduler, INIT_SIM_TICK), - ) - ExtSimSetupData(extSimAdaptersIt, extDataServicesMap, extDataListenerMap, extDatas, Some(extScheduler)) + ??? } - - def opsimAsExtSimulation( - context: ActorContext[_], - rootScheduler: ActorRef[SchedulerMessage], - simScheduler: ActorRef[SchedulerMessage], - ): ExtSimSetupData = { - val simulationStart = TimeUtil.withDefaults.toZonedDateTime( - simonaConfig.simona.time.startDateTime - ) - val extScheduler = scheduler(context, parent = rootScheduler) - val opsimSimulator = new OpsimSimulator( - "amqp://guest:guest@localhost:5672/myvhost" - ) - - val extSimAdapterPhase1 = context.toClassic.simonaActorOf( - ExtSimAdapter.props(extScheduler.toClassic), - s"1", - ) - val extSimAdapterPhase2 = context.toClassic.simonaActorOf( - ExtSimAdapter.props(simScheduler.toClassic), - s"2", - ) - - val extSimAdapters: Map[java.lang.Integer, ClassicRef] = Map( - 1.asInstanceOf[java.lang.Integer] -> extSimAdapterPhase1, - 2.asInstanceOf[java.lang.Integer] -> extSimAdapterPhase2 - ) - - val extSimAdapterData = new ExtSimAdapterData(extSimAdapters.asJava, args) - - // send init data right away, init activation is scheduled - extSimAdapterPhase1 ! ExtSimAdapter.Create( - extSimAdapterData, - 1, - ScheduleLock.singleKey(context, extScheduler, INIT_SIM_TICK), - ) - - val extPrimaryDataService = context.toClassic.simonaActorOf( - ExtPrimaryDataService.props(extScheduler.toClassic), - s"0-0", - ) - val extPrimaryData = new ExtPrimaryData( - extPrimaryDataService, - extSimAdapterPhase1, - opsimSimulator.getExtPrimaryDataSimulation.getPrimaryDataFactory, - opsimSimulator.getExtPrimaryDataSimulation.getPrimaryDataAssets - ) - - opsimSimulator.getExtPrimaryDataSimulation.setExtPrimaryData(extPrimaryData) - - extPrimaryDataService ! SimonaService.Create( - InitExtPrimaryData(extPrimaryData), - ScheduleLock.singleKey( - context, - extScheduler, - INIT_SIM_TICK, - ), - ) - - /* - // --- External Em Data - val extEmDataService = context.toClassic.simonaActorOf( - ExtEmDataService.props(extScheduler.toClassic), - s"0-0", - ) - val extEmData = new ExtEmData( - extEmDataService, - extSimAdapterPhase1, - opsimSimulator.getExtEmDataSimulation.getEmDataFactory, - opsimSimulator.getExtEmDataSimulation.getControlledEms - ) - - opsimSimulator.getExtEmDataSimulation.setExtEmData(extEmData) - - extEmDataService ! SimonaService.Create( - InitExtEmData(extEmData), - ScheduleLock.singleKey( - context, - extScheduler, - INIT_SIM_TICK, - ), - ) - - */ - - - //Result Data - - val extResultDataProvider = { - context.spawn( - ExtResultDataProvider(simScheduler), - s"ExtResultDataProvider", - ) - } - - - val timeout: PekkoTimeout = PekkoTimeout.create(5.seconds.toJava) - val scheduler2: Scheduler = context.system.scheduler - - val adapterRef = Await.result( - extResultDataProvider.ask[ActorRef[ResultDataMessageFromExt]] (ref => RequestDataMessageAdapter(ref))(timeout, scheduler2), timeout.duration) - val adapterScheduleRef = Await.result( - extResultDataProvider.ask[ActorRef[ScheduleServiceActivation]] (ref => RequestScheduleActivationAdapter(ref))(timeout, scheduler2), timeout.duration) - - val extResultData = - new ExtResultData( - adapterRef.toClassic, - adapterScheduleRef.toClassic, - extSimAdapterPhase2, - opsimSimulator.getExtResultDataSimulation.getResultDataFactory, - opsimSimulator.getExtResultDataSimulation.getResultDataAssets, - simulationStart - ) - - opsimSimulator.getExtResultDataSimulation.setExtResultData(extResultData) - - extResultDataProvider ! ExtResultDataProvider.Create( - InitExtResultData(extResultData), - ScheduleLock.singleKey( - context, - simScheduler, - INIT_SIM_TICK, - ), - ) - - val simpleExtSimDatas: List[ExtData] = List( - extResultData, - extPrimaryData - ) - - opsimSimulator.setup( - extSimAdapterData, - simpleExtSimDatas.asJava, - ) - // starting external simulation - new Thread(opsimSimulator, s"External simulation") - .start() - - val extDataServicesMap: Map[Class[_], ClassicRef] = Map( - classOf[ExtPrimaryDataService] -> extPrimaryDataService, - ) - - val extDataListenerMap: Map[Class[_], ActorRef[ExtResultDataProvider.Request]] = Map( - ExtResultDataProvider.getClass -> extResultDataProvider - ) - - val extSimAdaptersIt = Iterable(extSimAdapterPhase1, extSimAdapterPhase2) - - val extDatas = simpleExtSimDatas.toSet - extSimAdapterPhase2 ! ExtSimAdapter.Create( - extSimAdapterData, - 2, - ScheduleLock.singleKey(context, simScheduler, INIT_SIM_TICK), - ) - ExtSimSetupData(extSimAdaptersIt, extDataServicesMap, extDataListenerMap, extDatas, Some(extScheduler)) - } - - override def timeAdvancer( context: ActorContext[_], simulation: ActorRef[SimonaSim.SimulationEnded.type], diff --git a/src/main/scala/edu/ie3/simona/util/ReceiveDataMap.scala b/src/main/scala/edu/ie3/simona/util/ReceiveDataMap.scala index 1f7fca229e..4550f9e1de 100644 --- a/src/main/scala/edu/ie3/simona/util/ReceiveDataMap.scala +++ b/src/main/scala/edu/ie3/simona/util/ReceiveDataMap.scala @@ -41,6 +41,8 @@ final case class ReceiveDataMap[K, V]( ) } + def getExpectedKeys: Set[K] = expectedKeys + } object ReceiveDataMap { From c75518a65acc4f7c5af0e7a002f7e079569ef083 Mon Sep 17 00:00:00 2001 From: Johannes Bao Date: Mon, 3 Jun 2024 15:39:09 +0200 Subject: [PATCH 32/41] mapping in runtime --- .../fullGrid/ext_entity_mapping.csv | 7 +++++ .../simopsimtestgrid.conf | 2 +- .../edu/ie3/simona/agent/em/EmAgent.scala | 3 +- .../edu/ie3/simona/config/ArgsParser.scala | 12 +++++++- .../ie3/simona/main/RunSimonaWithMosaik.scala | 6 ++-- .../ie3/simona/main/RunSimonaWithOpsim.scala | 3 +- .../simona/sim/setup/SimonaMosaikSetup.scala | 18 ++++++------ .../simona/sim/setup/SimonaOpsimSetup.scala | 17 ++++++----- .../SimonaSimpleExtSimulationSetup.scala | 10 +++---- .../sim/setup/SimonaStandaloneSetup.scala | 28 ++----------------- 10 files changed, 50 insertions(+), 56 deletions(-) create mode 100644 input/samples/simopsimtestgrid_reduced/fullGrid/ext_entity_mapping.csv diff --git a/input/samples/simopsimtestgrid_reduced/fullGrid/ext_entity_mapping.csv b/input/samples/simopsimtestgrid_reduced/fullGrid/ext_entity_mapping.csv new file mode 100644 index 0000000000..58ec9dcd0f --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced/fullGrid/ext_entity_mapping.csv @@ -0,0 +1,7 @@ +uuid,id,columnScheme,dataType +f9dc7ce6-658c-4101-a12f-d58bb889286b,EM_HH_Bus_81,p,result_participant +957938b7-0476-4fab-a1b3-6ce8615857b3,EM_HH_Bus_110,p,result_participant +c3a7e9f5-b492-4c85-af2d-1e93f6a25443,EM_HH_Bus_25,p,result_participant +f9dc7ce6-658c-4101-a12f-d58bb889286b,EM_HH_Bus_81/Schedule,p,input +957938b7-0476-4fab-a1b3-6ce8615857b3,EM_HH_Bus_110/Schedule,p,input +c3a7e9f5-b492-4c85-af2d-1e93f6a25443,EM_HH_Bus_25/Schedule,p,input \ No newline at end of file diff --git a/input/samples/simopsimtestgrid_reduced/simopsimtestgrid.conf b/input/samples/simopsimtestgrid_reduced/simopsimtestgrid.conf index 501f00a065..8390756790 100755 --- a/input/samples/simopsimtestgrid_reduced/simopsimtestgrid.conf +++ b/input/samples/simopsimtestgrid_reduced/simopsimtestgrid.conf @@ -222,7 +222,7 @@ simona.gridConfig.refSystems = [ simona.powerflow.maxSweepPowerDeviation = 1E-5 // the maximum allowed deviation in power between two sweeps, before overall convergence is assumed simona.powerflow.newtonraphson.epsilon = [1E-12] simona.powerflow.newtonraphson.iterations = 50 -simona.powerflow.resolution = "86400s" +simona.powerflow.resolution = "87300s" simona.powerflow.stopOnFailure = true simona.control.transformer = [ diff --git a/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala b/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala index 80cdefa3c1..01c3f04fbc 100644 --- a/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala +++ b/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala @@ -146,8 +146,7 @@ object EmAgent { modelConfig, ) - ctx.log.info(s"EMAgent ${modelShell.uuid} with $modelShell") - ctx.log.info(s"EMAgent ${modelShell.uuid} goes to inactive!") + ctx.log.debug(s"EMAgent ${modelShell.uuid} with $modelShell") inactive( constantData, diff --git a/src/main/scala/edu/ie3/simona/config/ArgsParser.scala b/src/main/scala/edu/ie3/simona/config/ArgsParser.scala index affb8b91ac..8629d3dd2e 100644 --- a/src/main/scala/edu/ie3/simona/config/ArgsParser.scala +++ b/src/main/scala/edu/ie3/simona/config/ArgsParser.scala @@ -31,7 +31,8 @@ object ArgsParser extends LazyLogging { seedAddress: Option[String] = None, useLocalWorker: Option[Boolean] = None, tArgs: Map[String, String] = Map.empty, - extAddress: Option[String] = None + extAddress: Option[String] = None, + mappingPath: Option[String] = None ) { val useCluster: Boolean = clusterType.isDefined } @@ -112,6 +113,15 @@ object ArgsParser extends LazyLogging { .text( "Comma separated list (no whitespaces!) of initial addresses used for the rest of the cluster to bootstrap" ) + opt[String]("mapping-path") + .action((value, args) => args.copy(mappingPath = Option(value))) + .validate(value => + if (value.trim.isEmpty) failure("ext-address cannot be empty") + else success + ) + .text( + "Comma separated list (no whitespaces!) of initial addresses used for the rest of the cluster to bootstrap" + ) checkConfig(args => if ( diff --git a/src/main/scala/edu/ie3/simona/main/RunSimonaWithMosaik.scala b/src/main/scala/edu/ie3/simona/main/RunSimonaWithMosaik.scala index 0c840831b0..6d950f5620 100644 --- a/src/main/scala/edu/ie3/simona/main/RunSimonaWithMosaik.scala +++ b/src/main/scala/edu/ie3/simona/main/RunSimonaWithMosaik.scala @@ -29,9 +29,6 @@ object RunSimonaWithMosaik extends RunSimona[SimonaMosaikSetup] { // get the config and prepare it with the provided args val (arguments, parsedConfig) = ArgsParser.prepareConfig(args) - print("arguments:") - print(arguments) - // config fail fast check val simonaConfig = SimonaConfig(parsedConfig) ConfigFailFast.check(parsedConfig, simonaConfig) @@ -40,7 +37,8 @@ object RunSimonaWithMosaik extends RunSimona[SimonaMosaikSetup] { parsedConfig, SimonaMosaikSetup.buildResultFileHierarchy(parsedConfig), mainArgs = arguments.mainArgs, - mosaikIP = arguments.extAddress + mosaikIP = arguments.extAddress, + mosaikMappingPath = arguments.mappingPath ) } diff --git a/src/main/scala/edu/ie3/simona/main/RunSimonaWithOpsim.scala b/src/main/scala/edu/ie3/simona/main/RunSimonaWithOpsim.scala index 849db51884..b0b1070673 100644 --- a/src/main/scala/edu/ie3/simona/main/RunSimonaWithOpsim.scala +++ b/src/main/scala/edu/ie3/simona/main/RunSimonaWithOpsim.scala @@ -37,7 +37,8 @@ object RunSimonaWithOpsim extends RunSimona[SimonaOpsimSetup] { parsedConfig, SimonaOpsimSetup.buildResultFileHierarchy(parsedConfig), mainArgs = arguments.mainArgs, - opsimIP = arguments.extAddress + opsimIP = arguments.extAddress, + opsimMapping = arguments.mappingPath ) } diff --git a/src/main/scala/edu/ie3/simona/sim/setup/SimonaMosaikSetup.scala b/src/main/scala/edu/ie3/simona/sim/setup/SimonaMosaikSetup.scala index ce2ee3776d..94598d277f 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaMosaikSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaMosaikSetup.scala @@ -17,7 +17,6 @@ import edu.ie3.simona.agent.grid.GridAgentMessage.CreateGridAgent import edu.ie3.simona.agent.grid.{GridAgent, GridAgentMessage} import edu.ie3.simona.api.ExtSimAdapter import edu.ie3.simona.api.data.ExtData -import edu.ie3.simona.api.data.em.ExtEmData import edu.ie3.simona.api.data.primarydata.ExtPrimaryData import edu.ie3.simona.api.data.results.ExtResultData import edu.ie3.simona.api.data.results.ontology.ResultDataMessageFromExt @@ -34,8 +33,6 @@ 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.em.ExtEmDataService -import edu.ie3.simona.service.em.ExtEmDataService.InitExtEmData import edu.ie3.simona.service.primary.ExtPrimaryDataService.InitExtPrimaryData import edu.ie3.simona.service.primary.PrimaryServiceProxy.InitPrimaryServiceProxyStateData import edu.ie3.simona.service.primary.{ExtPrimaryDataService, PrimaryServiceProxy} @@ -47,7 +44,6 @@ import edu.ie3.simona.sim.SimonaSim import edu.ie3.simona.util.ResultFileHierarchy import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK import edu.ie3.simona.util.TickUtil.RichZonedDateTime -import edu.ie3.simopsim.OpsimSimulator import edu.ie3.simosaik.MosaikSimulation import edu.ie3.util.TimeUtil import org.apache.pekko.actor.typed.scaladsl.ActorContext @@ -57,6 +53,7 @@ import org.apache.pekko.actor.typed.{ActorRef, Scheduler} import org.apache.pekko.actor.{ActorRef => ClassicRef} import org.apache.pekko.util.{Timeout => PekkoTimeout} +import java.nio.file.Path import java.time.temporal.ChronoUnit import java.util.UUID import java.util.concurrent.LinkedBlockingQueue @@ -77,7 +74,8 @@ class SimonaMosaikSetup( resultFileHierarchy: ResultFileHierarchy, runtimeEventQueue: Option[LinkedBlockingQueue[RuntimeEvent]] = None, override val args: Array[String], - mosaikIP: Option[String] = None + mosaikIP: Option[String] = None, + mosaikMappingPath: Option[String] = None ) extends SimonaSetup { override def gridAgents( @@ -367,6 +365,7 @@ class SimonaMosaikSetup( val mosaikAddress = mosaikIP.getOrElse( "127.0.0.1:5678" ) + val mosaikMapping = mosaikMappingPath.getOrElse(throw new RuntimeException("Cannot connect to Mosaik, because there is no mapping!")) val simulationStart = TimeUtil.withDefaults.toZonedDateTime( simonaConfig.simona.time.startDateTime ) @@ -374,7 +373,7 @@ class SimonaMosaikSetup( ChronoUnit.SECONDS ) val extScheduler = scheduler(context, parent = rootScheduler) - val mosaikExtSim = new MosaikSimulation(mosaikAddress) + val mosaikExtSim = new MosaikSimulation(mosaikAddress, Path.of(mosaikMapping)) val extSimAdapterPhase1 = context.toClassic.simonaActorOf( ExtSimAdapter.props(extScheduler.toClassic), @@ -446,7 +445,6 @@ class SimonaMosaikSetup( adapterRef.toClassic, adapterScheduleRef.toClassic, extSimAdapterPhase2, - mosaikExtSim.getExtResultDataSimulation.getResultDataFactory, mosaikExtSim.getExtResultDataSimulation.getGridResultDataAssets, mosaikExtSim.getExtResultDataSimulation.getParticipantResultDataAssets, simulationStart, @@ -626,7 +624,8 @@ object SimonaMosaikSetup extends LazyLogging with SetupHelper { resultFileHierarchy: ResultFileHierarchy, runtimeEventQueue: Option[LinkedBlockingQueue[RuntimeEvent]] = None, mainArgs: Array[String] = Array.empty[String], - mosaikIP: Option[String] = None + mosaikIP: Option[String] = None, + mosaikMappingPath: Option[String] = None ): SimonaMosaikSetup = new SimonaMosaikSetup( typeSafeConfig, @@ -634,6 +633,7 @@ object SimonaMosaikSetup extends LazyLogging with SetupHelper { resultFileHierarchy, runtimeEventQueue, mainArgs, - mosaikIP + mosaikIP, + mosaikMappingPath ) } diff --git a/src/main/scala/edu/ie3/simona/sim/setup/SimonaOpsimSetup.scala b/src/main/scala/edu/ie3/simona/sim/setup/SimonaOpsimSetup.scala index 265e406126..cf42635915 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaOpsimSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaOpsimSetup.scala @@ -47,8 +47,7 @@ import edu.ie3.simona.sim.SimonaSim import edu.ie3.simona.util.ResultFileHierarchy import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK import edu.ie3.simona.util.TickUtil.RichZonedDateTime -import edu.ie3.simopsim.OpsimEmSimulator -import edu.ie3.simpleextsim.{SimpleExtSimulation, SimpleExtSimulationWithEm} +import edu.ie3.simopsim.OpsimEmSimulation import edu.ie3.util.TimeUtil import org.apache.pekko.actor.typed.scaladsl.ActorContext import org.apache.pekko.actor.typed.scaladsl.AskPattern._ @@ -57,6 +56,7 @@ import org.apache.pekko.actor.typed.{ActorRef, Scheduler} import org.apache.pekko.actor.{ActorRef => ClassicRef} import org.apache.pekko.util.{Timeout => PekkoTimeout} +import java.nio.file.Path import java.time.temporal.ChronoUnit import java.util.UUID import java.util.concurrent.LinkedBlockingQueue @@ -77,7 +77,8 @@ class SimonaOpsimSetup( resultFileHierarchy: ResultFileHierarchy, runtimeEventQueue: Option[LinkedBlockingQueue[RuntimeEvent]] = None, override val args: Array[String], - opsimIP: Option[String] = None + opsimIP: Option[String] = None, + opsimMappingPath: Option[String] = None, ) extends SimonaSetup { override def gridAgents( @@ -365,6 +366,7 @@ class SimonaOpsimSetup( simScheduler: ActorRef[SchedulerMessage], ): ExtSimSetupData = { val opsimAddress = opsimIP.getOrElse(throw new RuntimeException("Cannot connect to Opsim, because there is no address!")) + val opsimMapping = opsimMappingPath.getOrElse(throw new RuntimeException("Cannot connect to Opsim, because there is no mapping!")) val simulationStart = TimeUtil.withDefaults.toZonedDateTime( simonaConfig.simona.time.startDateTime ) @@ -372,7 +374,7 @@ class SimonaOpsimSetup( ChronoUnit.SECONDS ) val extScheduler = scheduler(context, parent = rootScheduler) - val opsimSim = new OpsimEmSimulator(opsimAddress) + val opsimSim = new OpsimEmSimulation(opsimAddress, Path.of(opsimMapping)) val extSimAdapterPhase1 = context.toClassic.simonaActorOf( ExtSimAdapter.props(extScheduler.toClassic), @@ -443,7 +445,6 @@ class SimonaOpsimSetup( adapterRef.toClassic, adapterScheduleRef.toClassic, extSimAdapterPhase2, - opsimSim.getExtResultDataSimulation.getResultDataFactory, opsimSim.getExtResultDataSimulation.getGridResultDataAssets, opsimSim.getExtResultDataSimulation.getParticipantResultDataAssets, simulationStart, @@ -620,7 +621,8 @@ object SimonaOpsimSetup extends LazyLogging with SetupHelper { resultFileHierarchy: ResultFileHierarchy, runtimeEventQueue: Option[LinkedBlockingQueue[RuntimeEvent]] = None, mainArgs: Array[String] = Array.empty[String], - opsimIP: Option[String] = None + opsimIP: Option[String] = None, + opsimMapping: Option[String] = None ): SimonaOpsimSetup = new SimonaOpsimSetup( typeSafeConfig, @@ -628,6 +630,7 @@ object SimonaOpsimSetup extends LazyLogging with SetupHelper { resultFileHierarchy, runtimeEventQueue, mainArgs, - opsimIP + opsimIP, + opsimMapping ) } diff --git a/src/main/scala/edu/ie3/simona/sim/setup/SimonaSimpleExtSimulationSetup.scala b/src/main/scala/edu/ie3/simona/sim/setup/SimonaSimpleExtSimulationSetup.scala index 7e2b28de71..dcf4f280aa 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaSimpleExtSimulationSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaSimpleExtSimulationSetup.scala @@ -18,7 +18,6 @@ import edu.ie3.simona.agent.grid.{GridAgent, GridAgentMessage} import edu.ie3.simona.api.ExtSimAdapter import edu.ie3.simona.api.data.ExtData import edu.ie3.simona.api.data.em.ExtEmData -import edu.ie3.simona.api.data.primarydata.ExtPrimaryData import edu.ie3.simona.api.data.results.ExtResultData import edu.ie3.simona.api.data.results.ontology.ResultDataMessageFromExt import edu.ie3.simona.api.simulation.ExtSimAdapterData @@ -36,9 +35,8 @@ import edu.ie3.simona.scheduler.{ScheduleLock, Scheduler, TimeAdvancer} import edu.ie3.simona.service.SimonaService import edu.ie3.simona.service.em.ExtEmDataService import edu.ie3.simona.service.em.ExtEmDataService.InitExtEmData -import edu.ie3.simona.service.primary.ExtPrimaryDataService.InitExtPrimaryData +import edu.ie3.simona.service.primary.PrimaryServiceProxy import edu.ie3.simona.service.primary.PrimaryServiceProxy.InitPrimaryServiceProxyStateData -import edu.ie3.simona.service.primary.{ExtPrimaryDataService, PrimaryServiceProxy} import edu.ie3.simona.service.results.ExtResultDataProvider import edu.ie3.simona.service.results.ExtResultDataProvider.{InitExtResultData, RequestDataMessageAdapter, RequestScheduleActivationAdapter} import edu.ie3.simona.service.weather.WeatherService @@ -47,8 +45,6 @@ import edu.ie3.simona.sim.SimonaSim import edu.ie3.simona.util.ResultFileHierarchy import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK import edu.ie3.simona.util.TickUtil.RichZonedDateTime -import edu.ie3.simopsim.OpsimSimulator -import edu.ie3.simpleextsim.{SimpleExtSimulation, SimpleExtSimulationWithEm} import edu.ie3.util.TimeUtil import org.apache.pekko.actor.typed.scaladsl.ActorContext import org.apache.pekko.actor.typed.scaladsl.AskPattern._ @@ -363,6 +359,7 @@ class SimonaSimpleExtSimulationSetup( rootScheduler: ActorRef[SchedulerMessage], simScheduler: ActorRef[SchedulerMessage], ): ExtSimSetupData = { + /* val simulationStart = TimeUtil.withDefaults.toZonedDateTime( simonaConfig.simona.time.startDateTime ) @@ -491,6 +488,9 @@ class SimonaSimpleExtSimulationSetup( ScheduleLock.singleKey(context, simScheduler, INIT_SIM_TICK), ) ExtSimSetupData(extSimAdaptersIt, extDataServicesMap, extDataListenerMap, extDatas, Some(extScheduler)) + + */ + ??? } 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 a9044e1b07..c9f86272e1 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala @@ -15,14 +15,6 @@ import edu.ie3.simona.actor.SimonaActorNaming.RichActorRefFactory import edu.ie3.simona.agent.EnvironmentRefs import edu.ie3.simona.agent.grid.GridAgentMessage.CreateGridAgent import edu.ie3.simona.agent.grid.{GridAgent, GridAgentMessage} -import edu.ie3.simona.api.ExtSimAdapter -import edu.ie3.simona.api.data.ExtData -import edu.ie3.simona.api.data.em.ExtEmData -import edu.ie3.simona.api.data.ev.{ExtEvData, ExtEvSimulation} -import edu.ie3.simona.api.data.primarydata.{ExtPrimaryData, ExtPrimaryDataSimulation} -import edu.ie3.simona.api.data.results.{ExtResultData, ExtResultDataSimulation} -import edu.ie3.simona.api.data.results.ontology.ResultDataMessageFromExt -import edu.ie3.simona.api.simulation.ExtSimAdapterData import edu.ie3.simona.config.{ArgsParser, RefSystemParser, SimonaConfig} import edu.ie3.simona.event.listener.{ResultEventListener, RuntimeEventListener} import edu.ie3.simona.event.{ResultEvent, RuntimeEvent} @@ -32,18 +24,11 @@ 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.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.ScheduleServiceActivation import edu.ie3.simona.scheduler.{ScheduleLock, Scheduler, TimeAdvancer} import edu.ie3.simona.service.SimonaService -import edu.ie3.simona.service.em.ExtEmDataService -import edu.ie3.simona.service.em.ExtEmDataService.InitExtEmData -import edu.ie3.simona.service.ev.ExtEvDataService -import edu.ie3.simona.service.ev.ExtEvDataService.InitExtEvData -import edu.ie3.simona.service.primary.ExtPrimaryDataService.InitExtPrimaryData +import edu.ie3.simona.service.primary.PrimaryServiceProxy import edu.ie3.simona.service.primary.PrimaryServiceProxy.InitPrimaryServiceProxyStateData -import edu.ie3.simona.service.primary.{ExtPrimaryDataService, PrimaryServiceProxy} import edu.ie3.simona.service.results.ExtResultDataProvider -import edu.ie3.simona.service.results.ExtResultDataProvider.{InitExtResultData, RequestDataMessageAdapter, RequestScheduleActivationAdapter} import edu.ie3.simona.service.weather.WeatherService import edu.ie3.simona.service.weather.WeatherService.InitWeatherServiceStateData import edu.ie3.simona.sim.SimonaSim @@ -51,23 +36,14 @@ import edu.ie3.simona.util.ResultFileHierarchy import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK import edu.ie3.simona.util.TickUtil.RichZonedDateTime import edu.ie3.util.TimeUtil +import org.apache.pekko.actor.typed.ActorRef import org.apache.pekko.actor.typed.scaladsl.ActorContext -import org.apache.pekko.actor.typed.scaladsl.AskPattern._ import org.apache.pekko.actor.typed.scaladsl.adapter.{ClassicActorRefOps, TypedActorContextOps, TypedActorRefOps} -import org.apache.pekko.actor.typed.{ActorRef, Scheduler} import org.apache.pekko.actor.{ActorRef => ClassicRef} -import org.apache.pekko.util.{Timeout => PekkoTimeout} -import edu.ie3.simona.service.results.ExtResultDataProvider.Request -import edu.ie3.simopsim.OpsimSimulator -import edu.ie3.simosaik.MosaikSimulation -import edu.ie3.simpleextsim.SimpleExtSimulation import java.util.UUID import java.util.concurrent.LinkedBlockingQueue -import scala.concurrent.Await -import scala.concurrent.duration.DurationInt import scala.jdk.CollectionConverters._ -import scala.jdk.DurationConverters._ /** Sample implementation to run a standalone simulation of simona configured * with the provided [[SimonaConfig]] and [[ResultFileHierarchy]] From d3610707aae173bae2c6ade5a92bb8c82f4c126d Mon Sep 17 00:00:00 2001 From: Johannes Bao Date: Tue, 4 Jun 2024 11:22:52 +0200 Subject: [PATCH 33/41] temp --- .../scala/edu/ie3/simona/sim/setup/SimonaOpsimSetup.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/main/scala/edu/ie3/simona/sim/setup/SimonaOpsimSetup.scala b/src/main/scala/edu/ie3/simona/sim/setup/SimonaOpsimSetup.scala index cf42635915..576e735bfd 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaOpsimSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaOpsimSetup.scala @@ -18,7 +18,6 @@ import edu.ie3.simona.agent.grid.{GridAgent, GridAgentMessage} import edu.ie3.simona.api.ExtSimAdapter import edu.ie3.simona.api.data.ExtData import edu.ie3.simona.api.data.em.ExtEmData -import edu.ie3.simona.api.data.primarydata.ExtPrimaryData import edu.ie3.simona.api.data.results.ExtResultData import edu.ie3.simona.api.data.results.ontology.ResultDataMessageFromExt import edu.ie3.simona.api.simulation.ExtSimAdapterData @@ -36,9 +35,8 @@ import edu.ie3.simona.scheduler.{ScheduleLock, Scheduler, TimeAdvancer} import edu.ie3.simona.service.SimonaService import edu.ie3.simona.service.em.ExtEmDataService import edu.ie3.simona.service.em.ExtEmDataService.InitExtEmData -import edu.ie3.simona.service.primary.ExtPrimaryDataService.InitExtPrimaryData +import edu.ie3.simona.service.primary.PrimaryServiceProxy import edu.ie3.simona.service.primary.PrimaryServiceProxy.InitPrimaryServiceProxyStateData -import edu.ie3.simona.service.primary.{ExtPrimaryDataService, PrimaryServiceProxy} import edu.ie3.simona.service.results.ExtResultDataProvider import edu.ie3.simona.service.results.ExtResultDataProvider.{InitExtResultData, RequestDataMessageAdapter, RequestScheduleActivationAdapter} import edu.ie3.simona.service.weather.WeatherService From 126e1072fbc1929fb3e6ea0b698518faacf3c686 Mon Sep 17 00:00:00 2001 From: smjobaoo Date: Wed, 5 Jun 2024 13:11:25 +0200 Subject: [PATCH 34/41] removed dataSimulation --- .../simopsimtestgrid.conf | 2 +- .../results/ExtResultDataProvider.scala | 2 +- .../simona/sim/setup/SimonaMosaikSetup.scala | 50 ++++++++++--------- .../simona/sim/setup/SimonaOpsimSetup.scala | 23 ++++----- 4 files changed, 40 insertions(+), 37 deletions(-) diff --git a/input/samples/simopsimtestgrid_reduced/simopsimtestgrid.conf b/input/samples/simopsimtestgrid_reduced/simopsimtestgrid.conf index 8390756790..e3d997c666 100755 --- a/input/samples/simopsimtestgrid_reduced/simopsimtestgrid.conf +++ b/input/samples/simopsimtestgrid_reduced/simopsimtestgrid.conf @@ -16,7 +16,7 @@ simona.simulationName = "simopsimtest" # Time Parameters ################################################################## simona.time.startDateTime = "2024-02-27T00:00:00Z" -simona.time.endDateTime = "2024-02-28T00:15:00Z" +simona.time.endDateTime = "2024-02-27T23:45:00Z" simona.time.schedulerReadyCheckWindow = 900 ################################################################## diff --git a/src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala b/src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala index c57319029d..3ade0e1ee7 100644 --- a/src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala +++ b/src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala @@ -101,7 +101,7 @@ object ExtResultDataProvider { Behaviors.receivePartial { case (_, WrappedActivation(Activation(INIT_SIM_TICK))) => val initGridSubscribers = initServiceData.extResultData.getGridResultDataAssets.asScala.toList - val initParticipantSubscribers = initServiceData.extResultData.getParticpantResultDataAssets.asScala.toList + val initParticipantSubscribers = initServiceData.extResultData.getParticipantResultDataAssets.asScala.toList var initResultStorage = Map.empty[UUID, (Option[ResultEntity], Option[Long])] initParticipantSubscribers.foreach( diff --git a/src/main/scala/edu/ie3/simona/sim/setup/SimonaMosaikSetup.scala b/src/main/scala/edu/ie3/simona/sim/setup/SimonaMosaikSetup.scala index 9a3ae4d785..78c6d05195 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaMosaikSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaMosaikSetup.scala @@ -362,18 +362,20 @@ class SimonaMosaikSetup( rootScheduler: ActorRef[SchedulerMessage], simScheduler: ActorRef[SchedulerMessage] ): ExtSimSetupData = { - val mosaikAddress = mosaikIP.getOrElse( - "127.0.0.1:5678" - ) + + val mosaikAddress = mosaikIP.getOrElse("127.0.0.1:5678") val mosaikMapping = mosaikMappingPath.getOrElse(throw new RuntimeException("Cannot connect to Mosaik, because there is no mapping!")) + val mosaikExtSim = new MosaikSimulation(mosaikAddress, Path.of(mosaikMapping)) + val simulationStart = TimeUtil.withDefaults.toZonedDateTime( simonaConfig.simona.time.startDateTime ) + val powerFlowResolution = simonaConfig.simona.powerflow.resolution.get( ChronoUnit.SECONDS ) + val extScheduler = scheduler(context, parent = rootScheduler) - val mosaikExtSim = new MosaikSimulation(mosaikAddress, Path.of(mosaikMapping)) val extSimAdapterPhase1 = context.toClassic.simonaActorOf( ExtSimAdapter.props(extScheduler.toClassic), @@ -391,7 +393,9 @@ class SimonaMosaikSetup( val extSimAdapterData = new ExtSimAdapterData(extSimAdapters.asJava, args) + // -=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=- + // --- Primary Data --- // send init data right away, init activation is scheduled extSimAdapterPhase1 ! ExtSimAdapter.Create( @@ -404,15 +408,14 @@ class SimonaMosaikSetup( ExtPrimaryDataService.props(extScheduler.toClassic), s"0-0", ) - val extPrimaryData = new ExtPrimaryData( + + val extPrimaryData = mosaikExtSim.getExtPrimaryData + + extPrimaryData.setActorRefs( extPrimaryDataService, - extSimAdapterPhase1, - mosaikExtSim.getExtPrimaryDataSimulation.getPrimaryDataFactory, - mosaikExtSim.getExtPrimaryDataSimulation.getPrimaryDataAssets + extSimAdapterPhase1 ) - mosaikExtSim.getExtPrimaryDataSimulation.setExtPrimaryData(extPrimaryData) - extPrimaryDataService ! SimonaService.Create( InitExtPrimaryData(extPrimaryData), ScheduleLock.singleKey( @@ -422,7 +425,9 @@ class SimonaMosaikSetup( ), ) - //Result Data + // -=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=- + + //--- Result Data --- val extResultDataProvider = { context.spawn( @@ -431,7 +436,6 @@ class SimonaMosaikSetup( ) } - val timeout: PekkoTimeout = PekkoTimeout.create(5.seconds.toJava) val scheduler2: Scheduler = context.system.scheduler @@ -440,18 +444,18 @@ class SimonaMosaikSetup( val adapterScheduleRef = Await.result( extResultDataProvider.ask[ActorRef[ScheduleServiceActivation]] (ref => RequestScheduleActivationAdapter(ref))(timeout, scheduler2), timeout.duration) - val extResultData = - new ExtResultData( - adapterRef.toClassic, - adapterScheduleRef.toClassic, - extSimAdapterPhase2, - mosaikExtSim.getExtResultDataSimulation.getGridResultDataAssets, - mosaikExtSim.getExtResultDataSimulation.getParticipantResultDataAssets, - simulationStart, - powerFlowResolution - ) + val extResultData = mosaikExtSim.getExtResultData + + extResultData.setActorRefs( + adapterRef.toClassic, + adapterScheduleRef.toClassic, + extSimAdapterPhase2 + ) - mosaikExtSim.getExtResultDataSimulation.setExtResultData(extResultData) + extResultData.setSimulationData( + simulationStart, + powerFlowResolution + ) extResultDataProvider ! ExtResultDataProvider.Create( InitExtResultData(extResultData), diff --git a/src/main/scala/edu/ie3/simona/sim/setup/SimonaOpsimSetup.scala b/src/main/scala/edu/ie3/simona/sim/setup/SimonaOpsimSetup.scala index ca1d0090d4..5f3d5a52e9 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaOpsimSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaOpsimSetup.scala @@ -403,15 +403,13 @@ class SimonaOpsimSetup( ExtEmDataService.props(extScheduler.toClassic), s"0-0", ) - val extEmData = new ExtEmData( + val extEmData = opsimSim.getExtEmData + + extEmData.setActorRefs( extEmDataService, - extSimAdapterPhase1, - opsimSim.getExtEmDataSimulation.getEmDataFactory, - opsimSim.getExtEmDataSimulation.getControlledEms + extSimAdapterPhase1 ) - opsimSim.getExtEmDataSimulation.setExtEmData(extEmData) - extEmDataService ! SimonaService.Create( InitExtEmData(extEmData), ScheduleLock.singleKey( @@ -439,18 +437,19 @@ class SimonaOpsimSetup( val adapterScheduleRef = Await.result( extResultDataProvider.ask[ActorRef[ScheduleServiceActivation]] (ref => RequestScheduleActivationAdapter(ref))(timeout, scheduler2), timeout.duration) - val extResultData = new ExtResultData( + val extResultData = opsimSim.getExtResultData + + extResultData.setActorRefs( adapterRef.toClassic, adapterScheduleRef.toClassic, - extSimAdapterPhase2, - opsimSim.getExtResultDataSimulation.getGridResultDataAssets, - opsimSim.getExtResultDataSimulation.getParticipantResultDataAssets, + extSimAdapterPhase2 + ) + + extResultData.setSimulationData( simulationStart, powerFlowResolution ) - opsimSim.getExtResultDataSimulation.setExtResultData(extResultData) - extResultDataProvider ! ExtResultDataProvider.Create( InitExtResultData(extResultData), ScheduleLock.singleKey( From da390656065dceae9764f105bee28a01af76b2a1 Mon Sep 17 00:00:00 2001 From: Johannes Bao Date: Wed, 5 Jun 2024 13:14:47 +0200 Subject: [PATCH 35/41] gitignore --- .gitignore | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/.gitignore b/.gitignore index 7cf28aed43..5bebd61aa7 100644 --- a/.gitignore +++ b/.gitignore @@ -235,3 +235,9 @@ build # Ignore unwanted input directories input/ext_sim +/input/samples/simopsimtestgrid_reduced/ +/input/samples/simopsimtestgrid_reduced_reference/ +/input/samples/simopsimtestgrid_reduced_withoutem/ +/input/samples/vn_simona_ns/ +/input/samples/vn_simona_small_with_em/ +/input/samples/vn_simona_small_with_em_original/ From 1b8cee4d7b1fce465e9e5d96dab71e58c21fc124 Mon Sep 17 00:00:00 2001 From: Johannes Bao Date: Wed, 5 Jun 2024 13:15:41 +0200 Subject: [PATCH 36/41] removed testgrids --- .../fullGrid/em_input.csv | 4 - .../fullGrid/ext_entity_mapping.csv | 7 - .../fullGrid/line_input.csv | 110 -------- .../fullGrid/line_type_input.csv | 2 - .../fullGrid/load_input.csv | 4 - .../fullGrid/node_input.csv | 112 -------- .../fullGrid/pv_input.csv | 4 - .../fullGrid/transformer_2_w_input.csv | 2 - .../fullGrid/transformer_2_w_type_input.csv | 2 - .../simopsimtestgrid.conf | 240 ------------------ .../fullGrid/em_input.csv | 4 - .../fullGrid/line_input.csv | 110 -------- .../fullGrid/line_type_input.csv | 2 - .../fullGrid/load_input.csv | 4 - .../fullGrid/node_input.csv | 112 -------- .../fullGrid/pv_input.csv | 4 - .../fullGrid/transformer_2_w_input.csv | 2 - .../fullGrid/transformer_2_w_type_input.csv | 2 - .../simopsimtestgrid.conf | 240 ------------------ .../fullGrid/line_graphic_input.csv | 16 -- .../vn_simona_ns/fullGrid/line_input.csv | 16 -- .../vn_simona_ns/fullGrid/line_type_input.csv | 16 -- .../vn_simona_ns/fullGrid/load_input.csv | 16 -- .../fullGrid/node_graphic_input.csv | 17 -- .../vn_simona_ns/fullGrid/node_input.csv | 18 -- .../vn_simona_ns/fullGrid/pv_input.csv | 3 - .../fullGrid/transformer_2_w_input.csv | 2 - .../fullGrid/transformer_2_w_type_input.csv | 2 - input/samples/vn_simona_ns/vn_simona_ns.conf | 240 ------------------ .../fullGrid/em_input.csv | 3 - .../fullGrid/fixed_feed_in_input.csv | 2 - .../fullGrid/line_input.csv | 6 - .../fullGrid/line_type_input.csv | 2 - .../fullGrid/load_input.csv | 5 - .../fullGrid/node_input.csv | 6 - .../fullGrid/pv_input.csv | 3 - .../fullGrid/storage_input.csv | 3 - .../fullGrid/storage_type_input.csv | 2 - .../fullGrid/transformer_2_w_input.csv | 2 - .../fullGrid/transformer_2_w_type_input.csv | 2 - .../vn_simona_small_withem.conf | 210 --------------- .../fullGrid/em_input.csv | 5 - .../fullGrid/fixed_feed_in_input.csv | 2 - .../fullGrid/line_input.csv | 6 - .../fullGrid/line_type_input.csv | 2 - .../fullGrid/load_input.csv | 5 - .../fullGrid/node_input.csv | 6 - .../fullGrid/pv_input.csv | 3 - .../fullGrid/storage_input.csv | 3 - .../fullGrid/storage_type_input.csv | 2 - .../fullGrid/transformer_2_w_input.csv | 2 - .../fullGrid/transformer_2_w_type_input.csv | 2 - .../vn_simona_small_withem.conf | 210 --------------- 53 files changed, 1807 deletions(-) delete mode 100755 input/samples/simopsimtestgrid_reduced/fullGrid/em_input.csv delete mode 100644 input/samples/simopsimtestgrid_reduced/fullGrid/ext_entity_mapping.csv delete mode 100755 input/samples/simopsimtestgrid_reduced/fullGrid/line_input.csv delete mode 100755 input/samples/simopsimtestgrid_reduced/fullGrid/line_type_input.csv delete mode 100755 input/samples/simopsimtestgrid_reduced/fullGrid/load_input.csv delete mode 100755 input/samples/simopsimtestgrid_reduced/fullGrid/node_input.csv delete mode 100755 input/samples/simopsimtestgrid_reduced/fullGrid/pv_input.csv delete mode 100755 input/samples/simopsimtestgrid_reduced/fullGrid/transformer_2_w_input.csv delete mode 100755 input/samples/simopsimtestgrid_reduced/fullGrid/transformer_2_w_type_input.csv delete mode 100755 input/samples/simopsimtestgrid_reduced/simopsimtestgrid.conf delete mode 100644 input/samples/simopsimtestgrid_reduced_reference/fullGrid/em_input.csv delete mode 100644 input/samples/simopsimtestgrid_reduced_reference/fullGrid/line_input.csv delete mode 100644 input/samples/simopsimtestgrid_reduced_reference/fullGrid/line_type_input.csv delete mode 100644 input/samples/simopsimtestgrid_reduced_reference/fullGrid/load_input.csv delete mode 100644 input/samples/simopsimtestgrid_reduced_reference/fullGrid/node_input.csv delete mode 100644 input/samples/simopsimtestgrid_reduced_reference/fullGrid/pv_input.csv delete mode 100644 input/samples/simopsimtestgrid_reduced_reference/fullGrid/transformer_2_w_input.csv delete mode 100644 input/samples/simopsimtestgrid_reduced_reference/fullGrid/transformer_2_w_type_input.csv delete mode 100644 input/samples/simopsimtestgrid_reduced_reference/simopsimtestgrid.conf delete mode 100644 input/samples/vn_simona_ns/fullGrid/line_graphic_input.csv delete mode 100644 input/samples/vn_simona_ns/fullGrid/line_input.csv delete mode 100644 input/samples/vn_simona_ns/fullGrid/line_type_input.csv delete mode 100644 input/samples/vn_simona_ns/fullGrid/load_input.csv delete mode 100644 input/samples/vn_simona_ns/fullGrid/node_graphic_input.csv delete mode 100644 input/samples/vn_simona_ns/fullGrid/node_input.csv delete mode 100644 input/samples/vn_simona_ns/fullGrid/pv_input.csv delete mode 100644 input/samples/vn_simona_ns/fullGrid/transformer_2_w_input.csv delete mode 100644 input/samples/vn_simona_ns/fullGrid/transformer_2_w_type_input.csv delete mode 100644 input/samples/vn_simona_ns/vn_simona_ns.conf delete mode 100755 input/samples/vn_simona_small_with_em/fullGrid/em_input.csv delete mode 100755 input/samples/vn_simona_small_with_em/fullGrid/fixed_feed_in_input.csv delete mode 100755 input/samples/vn_simona_small_with_em/fullGrid/line_input.csv delete mode 100755 input/samples/vn_simona_small_with_em/fullGrid/line_type_input.csv delete mode 100755 input/samples/vn_simona_small_with_em/fullGrid/load_input.csv delete mode 100755 input/samples/vn_simona_small_with_em/fullGrid/node_input.csv delete mode 100644 input/samples/vn_simona_small_with_em/fullGrid/pv_input.csv delete mode 100644 input/samples/vn_simona_small_with_em/fullGrid/storage_input.csv delete mode 100644 input/samples/vn_simona_small_with_em/fullGrid/storage_type_input.csv delete mode 100755 input/samples/vn_simona_small_with_em/fullGrid/transformer_2_w_input.csv delete mode 100755 input/samples/vn_simona_small_with_em/fullGrid/transformer_2_w_type_input.csv delete mode 100755 input/samples/vn_simona_small_with_em/vn_simona_small_withem.conf delete mode 100755 input/samples/vn_simona_small_with_em_original/fullGrid/em_input.csv delete mode 100755 input/samples/vn_simona_small_with_em_original/fullGrid/fixed_feed_in_input.csv delete mode 100755 input/samples/vn_simona_small_with_em_original/fullGrid/line_input.csv delete mode 100755 input/samples/vn_simona_small_with_em_original/fullGrid/line_type_input.csv delete mode 100755 input/samples/vn_simona_small_with_em_original/fullGrid/load_input.csv delete mode 100755 input/samples/vn_simona_small_with_em_original/fullGrid/node_input.csv delete mode 100644 input/samples/vn_simona_small_with_em_original/fullGrid/pv_input.csv delete mode 100644 input/samples/vn_simona_small_with_em_original/fullGrid/storage_input.csv delete mode 100644 input/samples/vn_simona_small_with_em_original/fullGrid/storage_type_input.csv delete mode 100755 input/samples/vn_simona_small_with_em_original/fullGrid/transformer_2_w_input.csv delete mode 100755 input/samples/vn_simona_small_with_em_original/fullGrid/transformer_2_w_type_input.csv delete mode 100755 input/samples/vn_simona_small_with_em_original/vn_simona_small_withem.conf diff --git a/input/samples/simopsimtestgrid_reduced/fullGrid/em_input.csv b/input/samples/simopsimtestgrid_reduced/fullGrid/em_input.csv deleted file mode 100755 index b6a794fe65..0000000000 --- a/input/samples/simopsimtestgrid_reduced/fullGrid/em_input.csv +++ /dev/null @@ -1,4 +0,0 @@ -uuid;control_strategy;parent_em;id;operates_from;operates_until;operator -c3a7e9f5-b492-4c85-af2d-1e93f6a25443;self_optimization;;EM_HH_Bus_25;;; -f9dc7ce6-658c-4101-a12f-d58bb889286b;self_optimization;;EM_HH_Bus_81;;; -957938b7-0476-4fab-a1b3-6ce8615857b3;self_optimization;;EM_HH_Bus_110;;; diff --git a/input/samples/simopsimtestgrid_reduced/fullGrid/ext_entity_mapping.csv b/input/samples/simopsimtestgrid_reduced/fullGrid/ext_entity_mapping.csv deleted file mode 100644 index 58ec9dcd0f..0000000000 --- a/input/samples/simopsimtestgrid_reduced/fullGrid/ext_entity_mapping.csv +++ /dev/null @@ -1,7 +0,0 @@ -uuid,id,columnScheme,dataType -f9dc7ce6-658c-4101-a12f-d58bb889286b,EM_HH_Bus_81,p,result_participant -957938b7-0476-4fab-a1b3-6ce8615857b3,EM_HH_Bus_110,p,result_participant -c3a7e9f5-b492-4c85-af2d-1e93f6a25443,EM_HH_Bus_25,p,result_participant -f9dc7ce6-658c-4101-a12f-d58bb889286b,EM_HH_Bus_81/Schedule,p,input -957938b7-0476-4fab-a1b3-6ce8615857b3,EM_HH_Bus_110/Schedule,p,input -c3a7e9f5-b492-4c85-af2d-1e93f6a25443,EM_HH_Bus_25/Schedule,p,input \ No newline at end of file diff --git a/input/samples/simopsimtestgrid_reduced/fullGrid/line_input.csv b/input/samples/simopsimtestgrid_reduced/fullGrid/line_input.csv deleted file mode 100755 index f4ff92fbba..0000000000 --- a/input/samples/simopsimtestgrid_reduced/fullGrid/line_input.csv +++ /dev/null @@ -1,110 +0,0 @@ -uuid;geo_position;id;length;node_a;node_b;olm_characteristic;operates_from;operates_until;operator;parallel_devices;type -bdbbb247-57b7-473b-9411-53fcf35032db;"{""type"":""LineString"",""coordinates"":[[11.8213,53.426],[11.8213,53.4257]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 27;0.0161277;093160c4-6482-4c58-b952-217c615e3ada;9cdb3115-cc00-4d61-bc33-442e8f30fb63;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -c48eeb2c-6858-4f96-9a90-20641cc0903d;"{""type"":""LineString"",""coordinates"":[[11.8214,53.4229],[11.8213,53.4225]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 35;0.0150703;002a4495-96e4-49c9-abbe-8fccb3e9c83e;b909fb45-b6ee-427f-afd7-e8a0ec7274c6;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -97e498ae-a4f7-4485-a703-b8c00d0a5e8a;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4221],[11.8201,53.4225]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 8;0.0228449;b565ae3b-68f9-4bca-816d-9b0fc1c6b13f;a7725293-05fc-447f-bc12-38b689b0a956;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -f907f152-7bfe-4a17-a63f-adda500f6f0e;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4241],[11.8184,53.4293]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 107;0.04;4749ab2b-4d96-4100-8081-73e77c797d6b;ea4a6507-e504-4542-be5f-1019719b2257;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -6bf14400-8f6c-41ef-b6a4-9e861a3ad08f;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4241],[11.822,53.4294]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 3;0.030304;4749ab2b-4d96-4100-8081-73e77c797d6b;c5b6bfaf-1621-40a7-9c53-02cfb59c04d9;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -18b92b71-320d-4294-a47e-6715f1594755;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4243],[11.8196,53.4247]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 102;0.0251089;f4da61e4-7600-4cd1-95b6-c70b56c049fc;e7908208-77b4-4059-806e-4857262992fc;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -4128307f-3e00-4da9-b629-b696b72165a4;"{""type"":""LineString"",""coordinates"":[[11.815,53.4289],[11.815,53.4293]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 28;0.0377768;8e809cf6-7e05-437c-89a6-f6ca135a403b;35748e60-3be8-4930-8a61-209fd5df1bec;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -92876b53-6bbc-48ff-ba5f-5f5c08313e4d;"{""type"":""LineString"",""coordinates"":[[11.815,53.4277],[11.815,53.4272]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 111;0.031;3e6be3ac-2b51-4080-b815-391313612fc7;78815cf6-70db-432c-96e6-87fe8cf67eee;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -4b557d0c-9992-48b1-b45e-26cccc01db03;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4235],[11.8191,53.4231]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 90;0.00167204;f2724954-34d3-4ddf-b6b0-7a1531639990;2efac9b1-fb0d-4e08-bfac-501798826deb;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -0652d3e0-8c7c-4be2-9a41-00534aa9774b;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4212],[11.8191,53.4209]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 94;0.00182131;02e63e81-2e62-4ef6-8fdb-0b0905a437b6;fd4bebb8-40ca-4eed-92c0-cdd10b86ac20;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -782d8c4c-accb-424b-91ae-9af1b5d4b1af;"{""type"":""LineString"",""coordinates"":[[11.8201,53.4225],[11.8201,53.4229]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 79;0.0338129;a7725293-05fc-447f-bc12-38b689b0a956;83da8d60-405a-45f7-9bb9-9d35607b7927;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -b7cac7a5-f528-45bd-8ced-a16234979e13;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4239],[11.8191,53.4235]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 89;0.0111411;b5c1e826-63fd-4b0c-bec0-0c758389ef58;f2724954-34d3-4ddf-b6b0-7a1531639990;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -7ac1a715-e141-429a-8cfe-ff2badd41074;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4257],[11.8174,53.4262]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 65;0.019;8726dc29-621e-4455-a541-cd88d7da457f;814f784b-687f-4dd5-8a91-c7772c916d46;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -77ae4f1e-9ecb-4dda-a5f2-ce7ae3f9bbec;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4264],[11.8213,53.426]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 23;0.00488535;18b4157b-0e47-4c5a-adb8-ccae47372336;093160c4-6482-4c58-b952-217c615e3ada;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -6aaa54d9-d7b7-4adc-a85a-335cdf1393d6;"{""type"":""LineString"",""coordinates"":[[11.8162,53.4284],[11.8169,53.4289]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 40;0.036;6b2881ce-3965-4f5e-98a3-74eb47b0a7ca;a882e666-82d1-4ba6-87df-fc702fe06187;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -c85c107e-82c0-4002-acfa-d7000512a2ad;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4274],[11.8213,53.4271]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 20;0.0182765;2b3d7fb8-0583-4d47-97b1-3b5f232fd462;7b81b518-00e0-4ff1-b4cf-876903958d7a;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -2ef5f888-4341-4eee-b505-ae07a9a60c8d;"{""type"":""LineString"",""coordinates"":[[11.8201,53.4236],[11.8201,53.4239]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 55;0.0259846;f713593a-3fd3-410a-ac08-74202d4f5798;80962bd3-a10f-4ed2-ba6a-3e802189939c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -139ea4f9-9b7f-4825-8919-537a94ff4794;"{""type"":""LineString"",""coordinates"":[[11.8175,53.4277],[11.8174,53.4281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 59;0.00160952;7f692039-eef6-45f6-9e30-b5983f6750a5;3f6c26dd-842b-4dee-b71f-4aa32e2654ff;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -68853808-8b09-4ff2-9c92-88ed1d78c639;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4236],[11.8213,53.4232]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 33;0.0217875;3464496c-7dd4-41e9-ae0a-99ade0b51572;a966644c-37d5-4400-9fa0-725fd88586a8;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -91bec60c-2951-420c-a35e-2633119ee450;"{""type"":""LineString"",""coordinates"":[[11.8162,53.4277],[11.8162,53.4281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 30;0.00820054;2d33314e-31db-4ad4-a898-2be5d56a896d;555cd075-0fe4-4a65-b027-f45cffa960d9;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -35ba0827-e27e-496e-b735-c778f3b03019;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4241],[11.8213,53.4293]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 18;0.0139169;4749ab2b-4d96-4100-8081-73e77c797d6b;844c0b9c-058a-4228-a8c4-bf2defff6958;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -60efd159-72d6-4546-b245-8b8fc120a9f8;"{""type"":""LineString"",""coordinates"":[[11.817,53.4293],[11.8166,53.4241]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 26;0.098;85751c9b-3e5e-468a-9dc7-43775b0d4a6f;4749ab2b-4d96-4100-8081-73e77c797d6b;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -212b62aa-73c2-47af-95ec-00cad8d4a4f4;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4222],[11.8214,53.4218]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 38;0.00992653;589bf32a-b361-4536-ae96-6d56d184eedb;3b86661a-187d-4aa6-bf37-2014789afc08;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -7e61def1-8414-40b3-8775-fad9124f4369;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4282],[11.8213,53.4277]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 16;0.0164234;09285b78-9c18-4af7-9c7a-942cc868016f;3a557b4e-06b8-4f29-929f-81d95c42c897;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -f9b4c7dc-c199-4691-8c6d-1faf438cf336;"{""type"":""LineString"",""coordinates"":[[11.8221,53.429],[11.8221,53.4286]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 5;0.0143698;9644f198-e801-4545-87ee-a24e2a8039bd;6ee7ea93-ea9c-40cb-b79a-1c5f287c97a5;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -eebc456e-d11b-4b19-9100-cc1d8e91f926;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4245],[11.8166,53.4249]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 71;0.00648856;7efabb8d-ba17-4487-96d9-5744b1fedf8a;7c35a794-f569-4a9c-acb0-d03647610086;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -8f4c4009-e1ed-4985-8095-729aa5dc8cdd;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4192],[11.8198,53.4195]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 85;0.00737032;594d101c-3a05-45e3-a061-9189f3e848b7;af2b07ce-1a96-4b50-9e21-badf29eed519;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -20db47c8-d154-4ab2-ad65-caa4a37466d2;"{""type"":""LineString"",""coordinates"":[[11.8175,53.4257],[11.8174,53.4262]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 68;0.037;5f153bf6-4f25-41f1-8545-18fe6323bc49;814f784b-687f-4dd5-8a91-c7772c916d46;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -7d838003-cd6e-42fd-b75a-eaa354c3993b;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4216],[11.8191,53.4212]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 93;0.0043578;8f53645e-1f28-4eb1-807f-eb2a473f9d4c;02e63e81-2e62-4ef6-8fdb-0b0905a437b6;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -a92013e1-78b7-4447-a505-f25fffbf56f2;"{""type"":""LineString"",""coordinates"":[[11.8174,53.4262],[11.8175,53.4265]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 48;0.0118531;814f784b-687f-4dd5-8a91-c7772c916d46;94fe96b1-f36a-4edd-a107-4ff0376f1066;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -690f8f61-5cc7-448e-8971-a516f784bf11;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4293],[11.8166,53.4241]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 106;0.0131451;20ad9754-e966-4ad1-9541-f968c207f3df;4749ab2b-4d96-4100-8081-73e77c797d6b;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -6a00d582-32b1-4581-bdd1-a638ca279597;"{""type"":""LineString"",""coordinates"":[[11.815,53.4272],[11.815,53.4269]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 112;0.028;78815cf6-70db-432c-96e6-87fe8cf67eee;ab8c8f6c-e7a2-4b81-a0d7-5f13789267a2;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -1d5877c1-31cd-4ee8-b0dd-667fa4fffb8a;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4277],[11.8213,53.4274]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 17;0.00286133;3a557b4e-06b8-4f29-929f-81d95c42c897;2b3d7fb8-0583-4d47-97b1-3b5f232fd462;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -a15cd6a4-4b22-434f-be9b-ad2abe6e538c;"{""type"":""LineString"",""coordinates"":[[11.815,53.4289],[11.815,53.4285]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 108;0.033;8e809cf6-7e05-437c-89a6-f6ca135a403b;88cf719a-92df-4dfd-9a83-f84330e28fe0;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -2bd81511-4f19-4374-8ac5-96c2b19eda64;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4223],[11.8191,53.422]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 52;0.00980013;c7e48384-5699-4a38-a887-7e15a9145202;c6dac3ab-f44f-4b87-800c-0f4da64673f1;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -beee40ad-aca3-490f-87d5-a6dfc4bb76e3;"{""type"":""LineString"",""coordinates"":[[11.815,53.4293],[11.8166,53.4241]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 39;0.09;35748e60-3be8-4930-8a61-209fd5df1bec;4749ab2b-4d96-4100-8081-73e77c797d6b;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -96b06f87-7905-4d12-99d8-1ed330050659;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4203],[11.8197,53.4207]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 12;0.00473835;5596da2f-ca32-4ad3-81da-1ffa17cd3d7b;839ff0f4-93db-42ec-a928-bbc448b6cf5c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -e66890ef-ec02-407f-a5bb-ce32128e7490;"{""type"":""LineString"",""coordinates"":[[11.8162,53.4281],[11.8162,53.4284]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 41;0.00828244;555cd075-0fe4-4a65-b027-f45cffa960d9;6b2881ce-3965-4f5e-98a3-74eb47b0a7ca;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -015b0337-98bd-40d4-97d3-13a0d1da88ee;"{""type"":""LineString"",""coordinates"":[[11.8213,53.425],[11.8214,53.4246]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 84;0.0489789;58b551b6-83bd-4f1c-8d9c-8c9a7f638c0b;a9288e77-2919-4db6-89eb-9737bd07f111;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -f9e6b2f1-f8fb-4763-a1d0-36d06170fea0;"{""type"":""LineString"",""coordinates"":[[11.8213,53.429],[11.8213,53.4286]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 98;0.0144283;16091b6d-f1ea-4a07-9ad4-30d595aba68d;1403edf9-e47c-4705-8563-83bcd639482e;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -f270d6b5-7c94-4397-8ab0-7c39c888d726;"{""type"":""LineString"",""coordinates"":[[11.8175,53.4265],[11.8175,53.427]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 58;0.0036937;94fe96b1-f36a-4edd-a107-4ff0376f1066;80b8d1f8-7e83-421d-a95a-c193fc35f4f7;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -af8c65aa-cfd6-434a-8512-2d80106a2f2c;"{""type"":""LineString"",""coordinates"":[[11.8195,53.426],[11.8195,53.4264]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 31;0.00626899;9f1baf4e-12e1-41d1-8efc-81cfc78f1957;b8e95bf0-3ba8-4d53-a0bf-a3720fb785fb;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -fe28e831-6405-4dfe-987b-d688367694f1;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4247],[11.8195,53.425]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 104;0.00221503;e7908208-77b4-4059-806e-4857262992fc;29516ae3-6676-4797-99c1-1f0a32b989d8;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -eac56d69-5500-4261-9690-adb16c867485;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4231],[11.8191,53.4227]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 92;0.0149219;2efac9b1-fb0d-4e08-bfac-501798826deb;15a86f7d-fb73-49a4-af6a-25b14122378d;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -183f3976-48ca-42b2-9af9-7998436fac5b;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4289],[11.8196,53.4293]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 105;0.00721947;47246a84-ad0e-4d04-9d98-1c9cd5d363c1;20ad9754-e966-4ad1-9541-f968c207f3df;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -7feef458-03f3-4d23-b3a2-e6f1035398c4;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4243],[11.8213,53.4239]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 29;0.0204277;3fcb94e3-7781-4d83-9030-d9853822e78e;d0bfabdb-0e83-423b-a20a-ab9197c4284e;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -592cd979-16c9-43d8-a311-8ac938aa5d03;"{""type"":""LineString"",""coordinates"":[[11.8174,53.4262],[11.8182,53.4257]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 47;0.0181601;814f784b-687f-4dd5-8a91-c7772c916d46;c86d6361-4159-4787-b5f4-e41dcaa95195;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -684146e5-3b58-43b4-9589-45325ab1c0bc;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4257],[11.8213,53.4253]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 24;0.0102233;9cdb3115-cc00-4d61-bc33-442e8f30fb63;a432b8ce-0462-478b-83e7-3107cd2e909c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -d9f9ee76-b016-4588-ac6d-46681894ada7;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4188],[11.8198,53.4192]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 74;0.00421724;952c6b04-0d02-4ea2-a6cc-bb44fbbe4e52;594d101c-3a05-45e3-a061-9189f3e848b7;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -1f7e9cae-460e-47db-8cb8-da5d9f695fd8;"{""type"":""LineString"",""coordinates"":[[11.8197,53.4207],[11.8198,53.4211]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 13;0.00987733;839ff0f4-93db-42ec-a928-bbc448b6cf5c;27b84da5-478e-4a05-8fe7-a9f800db5eff;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -f3a592d0-0fd7-42ea-b928-f39473b419aa;"{""type"":""LineString"",""coordinates"":[[11.8214,53.4246],[11.8213,53.4243]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 10;0.0344866;a9288e77-2919-4db6-89eb-9737bd07f111;3fcb94e3-7781-4d83-9030-d9853822e78e;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -f505bff9-0803-415f-a765-9da981ff6024;"{""type"":""LineString"",""coordinates"":[[11.8195,53.4264],[11.8195,53.4268]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 43;0.00839819;b8e95bf0-3ba8-4d53-a0bf-a3720fb785fb;34031e92-3444-47d5-94ae-cceeb5d96bb2;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -c6501262-2e05-462b-8872-445d2aa1cab8;"{""type"":""LineString"",""coordinates"":[[11.8195,53.4272],[11.8195,53.4275]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 99;0.0157523;120eaa58-a500-4ae2-a86a-56a40b931ec1;9d136a6b-5fdc-44ed-a5ed-599a55281024;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -caac4b4b-4871-4e20-994c-6517931546cb;"{""type"":""LineString"",""coordinates"":[[11.8201,53.4229],[11.8201,53.4232]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 91;0.0220129;83da8d60-405a-45f7-9bb9-9d35607b7927;24b63115-12eb-4e77-b9ef-ca474fed960f;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -e9721561-53c0-45cc-a8ed-28861ef9dc66;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4216],[11.8198,53.4221]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 7;0.0333523;883edf38-9a18-4f61-981a-691aaf436cc7;b565ae3b-68f9-4bca-816d-9b0fc1c6b13f;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -ab143df6-d050-47b6-911a-93e462d928ac;"{""type"":""LineString"",""coordinates"":[[11.8173,53.4234],[11.8166,53.4238]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 73;0.0287121;e68a088d-cf1a-40b7-9b1a-e0933352f4e6;4129e079-6712-4275-911c-36729d698c42;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -ab7ab785-36ba-4da1-b176-a7c636cb1372;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4267],[11.8213,53.4264]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 22;0.0102382;dc54bd8a-b7d8-4e99-adb0-d6ee5084241c;18b4157b-0e47-4c5a-adb8-ccae47372336;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -1ae90a03-52a3-40ef-8e06-4ba01888aa5c;"{""type"":""LineString"",""coordinates"":[[11.816,53.4222],[11.816,53.4227]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 44;0.00743191;62d603c3-f306-40b3-a665-ba9892d226f0;3faac527-0ff3-44a7-9e4f-24a41940da90;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -20be9235-f4db-4753-9fa1-223c8519fcd3;"{""type"":""LineString"",""coordinates"":[[11.8175,53.4253],[11.8175,53.4257]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 66;0.00791441;73e7a7e8-2154-46ea-9727-a4916af3570c;5f153bf6-4f25-41f1-8545-18fe6323bc49;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -558e1545-a944-419a-9153-83caa09e1a3c;"{""type"":""LineString"",""coordinates"":[[11.8162,53.4273],[11.8162,53.4277]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 42;0.00357726;0d6ef8f6-0ba1-4fa7-8e63-e55cee12b165;2d33314e-31db-4ad4-a898-2be5d56a896d;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -38d60cf7-6099-4bc0-a616-0f0b66c70c9a;"{""type"":""LineString"",""coordinates"":[[11.8201,53.4232],[11.8201,53.4236]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 67;0.0179092;24b63115-12eb-4e77-b9ef-ca474fed960f;f713593a-3fd3-410a-ac08-74202d4f5798;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -e74dbbe9-948f-4056-8134-fd1d9d39e773;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4253],[11.8196,53.4257]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 86;0.0190003;03b2aa45-84f6-48c0-9dab-427e046a5672;79e19265-08e8-407f-ae95-2f78e344d3a4;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -24c0dc1c-2e99-46ed-a52b-5a7aae7c9afb;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4249],[11.8166,53.4253]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 72;0.00189914;7c35a794-f569-4a9c-acb0-d03647610086;50cac08e-bf24-4526-9466-53ca5edccd15;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -41a72cb2-f037-4196-a248-2b18a578db50;"{""type"":""LineString"",""coordinates"":[[11.816,53.421],[11.816,53.4214]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 78;0.00156586;98c14f60-e196-4f12-903b-8485f1eacb16;ca0b9e34-ddc8-405e-86a4-ab939626c4a2;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -e2919117-f70a-4b32-a5ab-f077fe0a84ef;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4253],[11.8166,53.4257]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 70;0.00263875;50cac08e-bf24-4526-9466-53ca5edccd15;8726dc29-621e-4455-a541-cd88d7da457f;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -7d3ee5e1-1749-4e43-afeb-4ccf5f431312;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4282],[11.8196,53.4286]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 103;0.011354;a3ed5af5-1fcb-4fce-af0f-708d3d604124;31e6e197-719d-4aaf-8ca5-ab9e7549390e;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -cbba4a6b-27d1-4724-a193-134af220a57d;"{""type"":""LineString"",""coordinates"":[[11.8182,53.4257],[11.8182,53.4254]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 49;0.018118;c86d6361-4159-4787-b5f4-e41dcaa95195;c6c20ab1-16fb-4f82-8017-273022da8bb0;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -b84fad90-182c-4ac1-b77d-394f39024ffc;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4286],[11.8196,53.4289]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 19;0.0115943;31e6e197-719d-4aaf-8ca5-ab9e7549390e;47246a84-ad0e-4d04-9d98-1c9cd5d363c1;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -6c87a571-9d8b-4df9-bd86-cde7954bee28;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4271],[11.8213,53.4267]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 21;0.00282037;7b81b518-00e0-4ff1-b4cf-876903958d7a;dc54bd8a-b7d8-4e99-adb0-d6ee5084241c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -536ef1a2-b988-4474-a5d4-1254598c4716;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4227],[11.8191,53.4223]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 82;0.00544392;15a86f7d-fb73-49a4-af6a-25b14122378d;c7e48384-5699-4a38-a887-7e15a9145202;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -c0c99b3a-1c61-47b8-931d-571d9494d98f;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4241],[11.8166,53.4245]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 51;0.0182324;5682ac05-7336-4ebc-a5d1-3b69c79fb3b1;7efabb8d-ba17-4487-96d9-5744b1fedf8a;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -d3366635-447e-4d45-b839-7a7561f869a3;"{""type"":""LineString"",""coordinates"":[[11.822,53.4294],[11.8221,53.429]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 4;0.0141339;c5b6bfaf-1621-40a7-9c53-02cfb59c04d9;9644f198-e801-4545-87ee-a24e2a8039bd;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -932dd0d7-2051-4a3a-a0e3-489210d3b763;"{""type"":""LineString"",""coordinates"":[[11.8169,53.4289],[11.817,53.4293]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 57;0.0219657;a882e666-82d1-4ba6-87df-fc702fe06187;85751c9b-3e5e-468a-9dc7-43775b0d4a6f;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -e718d253-34ea-4c00-a653-80ac0af2934e;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4293],[11.8213,53.429]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 11;0.00557129;844c0b9c-058a-4228-a8c4-bf2defff6958;16091b6d-f1ea-4a07-9ad4-30d595aba68d;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -3ac954bf-5c1e-428a-9da7-37331f53d8fe;"{""type"":""LineString"",""coordinates"":[[11.8175,53.4248],[11.8175,53.4253]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 69;0.0017759;0d6c3f2b-5296-4ec1-995c-b150e72f035f;73e7a7e8-2154-46ea-9727-a4916af3570c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -7e9a0440-2a62-4daf-a4ab-e9f14f6cfd77;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4238],[11.8166,53.4234]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 53;0.0337731;4129e079-6712-4275-911c-36729d698c42;57b40047-4f9d-46bb-bf19-c0a86bbd4f5b;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -10a2050e-173b-43a7-91a0-157cee5c900d;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4257],[11.8195,53.426]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 87;0.00654022;79e19265-08e8-407f-ae95-2f78e344d3a4;9f1baf4e-12e1-41d1-8efc-81cfc78f1957;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -0dfd30a5-394e-49ff-95e9-50e73bf2604c;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4232],[11.8214,53.4229]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 34;0.0199334;a966644c-37d5-4400-9fa0-725fd88586a8;002a4495-96e4-49c9-abbe-8fccb3e9c83e;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -847934a0-e2b7-4caf-bb40-2e6af5bcb0ab;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4238],[11.8166,53.4241]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 50;0.0101795;4129e079-6712-4275-911c-36729d698c42;5682ac05-7336-4ebc-a5d1-3b69c79fb3b1;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -2daf9021-eb24-4d0c-bced-00e713a65b3a;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4225],[11.8213,53.4222]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 36;0.010188;b909fb45-b6ee-427f-afd7-e8a0ec7274c6;589bf32a-b361-4536-ae96-6d56d184eedb;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -2f1b02a3-5a58-4723-ab8f-827118a4c611;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4239],[11.8213,53.4236]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 32;0.0188563;d0bfabdb-0e83-423b-a20a-ab9197c4284e;3464496c-7dd4-41e9-ae0a-99ade0b51572;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -a5e9b6b0-2afc-49b0-8aad-705410b823c2;"{""type"":""LineString"",""coordinates"":[[11.815,53.4285],[11.815,53.4281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 109;0.027;88cf719a-92df-4dfd-9a83-f84330e28fe0;d38d936a-9c05-4bdc-8331-418fef27f492;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -ae82a1c1-9790-44d1-8dab-52d75c7b79ce;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4286],[11.8213,53.4282]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 6;0.0050064;1403edf9-e47c-4705-8563-83bcd639482e;09285b78-9c18-4af7-9c7a-942cc868016f;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -e642eafb-2b6e-4b78-80b8-1ab15db8cfeb;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4278],[11.8196,53.4282]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 101;0.0153312;7dc43c81-9a61-45a0-9745-800a28bf4a9d;a3ed5af5-1fcb-4fce-af0f-708d3d604124;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -41a4c5e5-7b54-46a8-b6b6-d7b5861a251c;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4195],[11.8198,53.4199]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 96;0.0010841;af2b07ce-1a96-4b50-9e21-badf29eed519;215eaa45-82c3-49c7-a60f-4fa13215de05;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -44821cfc-7670-4f28-8941-70e9345cb069;"{""type"":""LineString"",""coordinates"":[[11.8201,53.4225],[11.8203,53.4221]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 15;0.027677;a7725293-05fc-447f-bc12-38b689b0a956;eae8a04c-44f2-4da3-95f6-cae48f85737c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -d665e8c6-40fb-4608-97fb-99a4cf52560e;"{""type"":""LineString"",""coordinates"":[[11.815,53.4281],[11.815,53.4277]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 110;0.041;d38d936a-9c05-4bdc-8331-418fef27f492;3e6be3ac-2b51-4080-b815-391313612fc7;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -2f37bd36-f356-4d29-81c7-dd9c2fba5e7d;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4234],[11.8166,53.423]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 45;0.00223482;57b40047-4f9d-46bb-bf19-c0a86bbd4f5b;5b73ded9-3ca4-4f18-a2ab-c27b9a3dcf9d;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -bcccf0b5-e0e3-4cc7-82bb-262ebc19415e;"{""type"":""LineString"",""coordinates"":[[11.8169,53.4289],[11.8168,53.4285]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 56;0.039;a882e666-82d1-4ba6-87df-fc702fe06187;c41ebab8-16a0-4a3a-b4af-26073932d462;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -3685f389-6822-4522-a633-74265b67eaee;"{""type"":""LineString"",""coordinates"":[[11.816,53.4227],[11.816,53.423]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 76;0.00185168;3faac527-0ff3-44a7-9e4f-24a41940da90;2575f527-1f4e-45e2-bed2-4a5427f122e0;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -48347702-9e58-49f3-a7de-72024d4b296c;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4199],[11.8198,53.4203]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 2;0.00995493;215eaa45-82c3-49c7-a60f-4fa13215de05;5596da2f-ca32-4ad3-81da-1ffa17cd3d7b;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -ec2a73ea-50ba-4187-b8b0-a5046cf6b632;"{""type"":""LineString"",""coordinates"":[[11.8195,53.425],[11.8196,53.4253]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 83;0.00510961;29516ae3-6676-4797-99c1-1f0a32b989d8;03b2aa45-84f6-48c0-9dab-427e046a5672;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -d1940183-f7bb-42df-b8ff-63ac7aff0b1d;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4211],[11.8198,53.4216]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 9;0.00439896;27b84da5-478e-4a05-8fe7-a9f800db5eff;883edf38-9a18-4f61-981a-691aaf436cc7;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -dd618e3f-9ef9-4e4b-b632-737d79c4d8c3;"{""type"":""LineString"",""coordinates"":[[11.8175,53.427],[11.8174,53.4273]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 62;0.00124645;80b8d1f8-7e83-421d-a95a-c193fc35f4f7;8ace5c2b-584a-4015-990f-6f1e14de4ddb;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -ed80666a-cf86-404a-b3a7-ad49be1cd40c;"{""type"":""LineString"",""coordinates"":[[11.816,53.423],[11.816,53.4234]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 77;0.00232237;2575f527-1f4e-45e2-bed2-4a5427f122e0;5d3bcf55-0520-43ff-8d63-3d0eb421e442;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -444615ed-26f2-45a7-8b5d-213c72e83a4f;"{""type"":""LineString"",""coordinates"":[[11.8201,53.4239],[11.8196,53.4243]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 95;0.0281251;80962bd3-a10f-4ed2-ba6a-3e802189939c;f4da61e4-7600-4cd1-95b6-c70b56c049fc;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -05ebbc17-61bc-4810-ae19-9ee04d7ce8d0;"{""type"":""LineString"",""coordinates"":[[11.8175,53.4285],[11.8169,53.4289]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 54;0.0173762;30b2ce2d-41dd-4d1d-866d-47abd24bfc3e;a882e666-82d1-4ba6-87df-fc702fe06187;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -caf90323-aad2-496e-a0e3-a4ba7c9a481c;"{""type"":""LineString"",""coordinates"":[[11.8174,53.4273],[11.8175,53.4277]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 64;0.00314803;8ace5c2b-584a-4015-990f-6f1e14de4ddb;7f692039-eef6-45f6-9e30-b5983f6750a5;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -60b400db-b048-4bed-be29-8abc18780d10;"{""type"":""LineString"",""coordinates"":[[11.816,53.4206],[11.816,53.421]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 14;0.045296;d6dbb0ae-13c9-438e-93b3-b6c63a0708df;98c14f60-e196-4f12-903b-8485f1eacb16;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -cf13634d-abd2-465d-8839-c95a54af7a80;"{""type"":""LineString"",""coordinates"":[[11.816,53.4214],[11.816,53.4218]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 80;0.0101988;ca0b9e34-ddc8-405e-86a4-ab939626c4a2;43040a39-8b6c-401f-9dfd-82b42aa6dec6;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -30044e09-a0f8-417d-a949-afcfa940f671;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4253],[11.8213,53.425]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 25;0.0124821;a432b8ce-0462-478b-83e7-3107cd2e909c;58b551b6-83bd-4f1c-8d9c-8c9a7f638c0b;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -fe71fe2b-8dd0-4942-b0b5-d241e095b912;"{""type"":""LineString"",""coordinates"":[[11.8195,53.4268],[11.8195,53.4272]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 97;0.0032723;34031e92-3444-47d5-94ae-cceeb5d96bb2;120eaa58-a500-4ae2-a86a-56a40b931ec1;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -66e44163-7091-4f1b-991c-64108e2238f6;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4243],[11.8191,53.4239]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 88;0.0185855;f4da61e4-7600-4cd1-95b6-c70b56c049fc;b5c1e826-63fd-4b0c-bec0-0c758389ef58;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -82bac681-169f-48d8-9842-fd69d3adbfe0;"{""type"":""LineString"",""coordinates"":[[11.816,53.4234],[11.8166,53.4238]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 75;0.0109137;5d3bcf55-0520-43ff-8d63-3d0eb421e442;4129e079-6712-4275-911c-36729d698c42;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -74a5dc42-e689-490f-a611-ae7c3767f01b;"{""type"":""LineString"",""coordinates"":[[11.816,53.4218],[11.816,53.4222]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 81;0.0100396;43040a39-8b6c-401f-9dfd-82b42aa6dec6;62d603c3-f306-40b3-a665-ba9892d226f0;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -0eedb3d0-cedc-4798-b42e-d8e8ef646b82;"{""type"":""LineString"",""coordinates"":[[11.8195,53.4275],[11.8196,53.4278]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 100;0.0129061;9d136a6b-5fdc-44ed-a5ed-599a55281024;7dc43c81-9a61-45a0-9745-800a28bf4a9d;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -b44446fd-2125-4c7b-850e-b0f3d6c8b110;"{""type"":""LineString"",""coordinates"":[[11.8174,53.4281],[11.8175,53.4285]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 61;0.014766;3f6c26dd-842b-4dee-b71f-4aa32e2654ff;30b2ce2d-41dd-4d1d-866d-47abd24bfc3e;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -31c914bc-dd9e-4825-9b4b-b5fc1b971e0d;"{""type"":""LineString"",""coordinates"":[[11.8166,53.423],[11.8166,53.4227]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 46;0.00683612;5b73ded9-3ca4-4f18-a2ab-c27b9a3dcf9d;ad30322c-0c99-4669-8e4b-25265087a66d;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -e7ac7023-f4e3-460d-8d02-4c9c444d2e18;"{""type"":""LineString"",""coordinates"":[[11.8191,53.422],[11.8191,53.4216]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 63;0.0149947;c6dac3ab-f44f-4b87-800c-0f4da64673f1;8f53645e-1f28-4eb1-807f-eb2a473f9d4c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e diff --git a/input/samples/simopsimtestgrid_reduced/fullGrid/line_type_input.csv b/input/samples/simopsimtestgrid_reduced/fullGrid/line_type_input.csv deleted file mode 100755 index d7786c38cd..0000000000 --- a/input/samples/simopsimtestgrid_reduced/fullGrid/line_type_input.csv +++ /dev/null @@ -1,2 +0,0 @@ -uuid;b;g;i_max;id;r;v_rated;x -9a8e9b63-af21-4c1b-8db7-fc2924f9610e;273.31899999999996;0.0;357.0;NAYY 4x240SE 0.6/1kV;0.1267;0.4;0.0797965 diff --git a/input/samples/simopsimtestgrid_reduced/fullGrid/load_input.csv b/input/samples/simopsimtestgrid_reduced/fullGrid/load_input.csv deleted file mode 100755 index 6ad16189a5..0000000000 --- a/input/samples/simopsimtestgrid_reduced/fullGrid/load_input.csv +++ /dev/null @@ -1,4 +0,0 @@ -uuid;cos_phi_rated;dsm;e_cons_annual;id;load_profile;node;operates_from;operates_until;operator;q_characteristics;s_rated;em -c3434742-e4f0-49e5-baa7-c1e3045c732c;0.93;false;0.0;LV5.201 Load 64;h0;dc54bd8a-b7d8-4e99-adb0-d6ee5084241c;;;;"cosPhiFixed:{(0.0,0.93)}";3.2;f9dc7ce6-658c-4101-a12f-d58bb889286b -fd2e19b6-d5e3-4776-9456-8787a2160d9d;0.93;false;0.0;LV5.201 Load 74;h0;43040a39-8b6c-401f-9dfd-82b42aa6dec6;;;;"cosPhiFixed:{(0.0,0.93)}";1.1;c3a7e9f5-b492-4c85-af2d-1e93f6a25443 -98c1a2ab-bd09-4c77-a389-d088aed894b1;0.93;false;0.0;LV5.201 Load 102;h0;3e6be3ac-2b51-4080-b815-391313612fc7;;;;"cosPhiFixed:{(0.0,0.93)}";5.3;957938b7-0476-4fab-a1b3-6ce8615857b3 diff --git a/input/samples/simopsimtestgrid_reduced/fullGrid/node_input.csv b/input/samples/simopsimtestgrid_reduced/fullGrid/node_input.csv deleted file mode 100755 index ab9f9335ca..0000000000 --- a/input/samples/simopsimtestgrid_reduced/fullGrid/node_input.csv +++ /dev/null @@ -1,112 +0,0 @@ -uuid;geo_position;id;operates_from;operates_until;operator;slack;subnet;v_rated;v_target;volt_lvl -16091b6d-f1ea-4a07-9ad4-30d595aba68d;"{""type"":""Point"",""coordinates"":[11.8213,53.429],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 26;;;;false;2;0.4;1.0;Niederspannung -27b84da5-478e-4a05-8fe7-a9f800db5eff;"{""type"":""Point"",""coordinates"":[11.8198,53.4211],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 7;;;;false;2;0.4;1.0;Niederspannung -7f692039-eef6-45f6-9e30-b5983f6750a5;"{""type"":""Point"",""coordinates"":[11.8175,53.4277],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 54;;;;false;2;0.4;1.0;Niederspannung -20ad9754-e966-4ad1-9541-f968c207f3df;"{""type"":""Point"",""coordinates"":[11.8196,53.4293],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 84;;;;false;2;0.4;1.0;Niederspannung -98c14f60-e196-4f12-903b-8485f1eacb16;"{""type"":""Point"",""coordinates"":[11.816,53.421],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 29;;;;false;2;0.4;1.0;Niederspannung -09285b78-9c18-4af7-9c7a-942cc868016f;"{""type"":""Point"",""coordinates"":[11.8213,53.4282],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 58;;;;false;2;0.4;1.0;Niederspannung -0d6ef8f6-0ba1-4fa7-8e63-e55cee12b165;"{""type"":""Point"",""coordinates"":[11.8162,53.4273],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 103;;;;false;2;0.4;1.0;Niederspannung -43040a39-8b6c-401f-9dfd-82b42aa6dec6;"{""type"":""Point"",""coordinates"":[11.816,53.4218],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 25;;;;false;2;0.4;1.0;Niederspannung -dc54bd8a-b7d8-4e99-adb0-d6ee5084241c;"{""type"":""Point"",""coordinates"":[11.8213,53.4267],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 81;;;;false;2;0.4;1.0;Niederspannung -2575f527-1f4e-45e2-bed2-4a5427f122e0;"{""type"":""Point"",""coordinates"":[11.816,53.423],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 3;;;;false;2;0.4;1.0;Niederspannung -30b2ce2d-41dd-4d1d-866d-47abd24bfc3e;"{""type"":""Point"",""coordinates"":[11.8175,53.4285],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 34;;;;false;2;0.4;1.0;Niederspannung -589bf32a-b361-4536-ae96-6d56d184eedb;"{""type"":""Point"",""coordinates"":[11.8213,53.4222],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 101;;;;false;2;0.4;1.0;Niederspannung -555cd075-0fe4-4a65-b027-f45cffa960d9;"{""type"":""Point"",""coordinates"":[11.8162,53.4281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 31;;;;false;2;0.4;1.0;Niederspannung -62d603c3-f306-40b3-a665-ba9892d226f0;"{""type"":""Point"",""coordinates"":[11.816,53.4222],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 55;;;;false;2;0.4;1.0;Niederspannung -215eaa45-82c3-49c7-a60f-4fa13215de05;"{""type"":""Point"",""coordinates"":[11.8198,53.4199],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 24;;;;false;2;0.4;1.0;Niederspannung -f4da61e4-7600-4cd1-95b6-c70b56c049fc;"{""type"":""Point"",""coordinates"":[11.8196,53.4243],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 76;;;;false;2;0.4;1.0;Niederspannung -73e7a7e8-2154-46ea-9727-a4916af3570c;"{""type"":""Point"",""coordinates"":[11.8175,53.4253],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 18;;;;false;2;0.4;1.0;Niederspannung -85751c9b-3e5e-468a-9dc7-43775b0d4a6f;"{""type"":""Point"",""coordinates"":[11.817,53.4293],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 51;;;;false;2;0.4;1.0;Niederspannung -093160c4-6482-4c58-b952-217c615e3ada;"{""type"":""Point"",""coordinates"":[11.8213,53.426],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 68;;;;false;2;0.4;1.0;Niederspannung -79e19265-08e8-407f-ae95-2f78e344d3a4;"{""type"":""Point"",""coordinates"":[11.8196,53.4257],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 28;;;;false;2;0.4;1.0;Niederspannung -a966644c-37d5-4400-9fa0-725fd88586a8;"{""type"":""Point"",""coordinates"":[11.8213,53.4232],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 59;;;;false;2;0.4;1.0;Niederspannung -952c6b04-0d02-4ea2-a6cc-bb44fbbe4e52;"{""type"":""Point"",""coordinates"":[11.8198,53.4188],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 94;;;;false;2;0.4;1.0;Niederspannung -c6c20ab1-16fb-4f82-8017-273022da8bb0;"{""type"":""Point"",""coordinates"":[11.8182,53.4254],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 62;;;;false;2;0.4;1.0;Niederspannung -e68a088d-cf1a-40b7-9b1a-e0933352f4e6;"{""type"":""Point"",""coordinates"":[11.8173,53.4234],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 67;;;;false;2;0.4;1.0;Niederspannung -8f53645e-1f28-4eb1-807f-eb2a473f9d4c;"{""type"":""Point"",""coordinates"":[11.8191,53.4216],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 86;;;;false;2;0.4;1.0;Niederspannung -0d6c3f2b-5296-4ec1-995c-b150e72f035f;"{""type"":""Point"",""coordinates"":[11.8175,53.4248],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 80;;;;false;2;0.4;1.0;Niederspannung -ea4a6507-e504-4542-be5f-1019719b2257;"{""type"":""Point"",""coordinates"":[11.8184,53.4293],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 111;;;;false;2;0.4;1.0;Niederspannung -9644f198-e801-4545-87ee-a24e2a8039bd;"{""type"":""Point"",""coordinates"":[11.8221,53.429],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 39;;;;false;2;0.4;1.0;Niederspannung -a9288e77-2919-4db6-89eb-9737bd07f111;"{""type"":""Point"",""coordinates"":[11.8214,53.4246],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 63;;;;false;2;0.4;1.0;Niederspannung -d6dbb0ae-13c9-438e-93b3-b6c63a0708df;"{""type"":""Point"",""coordinates"":[11.816,53.4206],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 89;;;;false;2;0.4;1.0;Niederspannung -5596da2f-ca32-4ad3-81da-1ffa17cd3d7b;"{""type"":""Point"",""coordinates"":[11.8198,53.4203],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 102;;;;false;2;0.4;1.0;Niederspannung -ad30322c-0c99-4669-8e4b-25265087a66d;"{""type"":""Point"",""coordinates"":[11.8166,53.4227],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 65;;;;false;2;0.4;1.0;Niederspannung -120eaa58-a500-4ae2-a86a-56a40b931ec1;"{""type"":""Point"",""coordinates"":[11.8195,53.4272],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 57;;;;false;2;0.4;1.0;Niederspannung -29516ae3-6676-4797-99c1-1f0a32b989d8;"{""type"":""Point"",""coordinates"":[11.8195,53.425],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 52;;;;false;2;0.4;1.0;Niederspannung -24b63115-12eb-4e77-b9ef-ca474fed960f;"{""type"":""Point"",""coordinates"":[11.8201,53.4232],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 46;;;;false;2;0.4;1.0;Niederspannung -a7725293-05fc-447f-bc12-38b689b0a956;"{""type"":""Point"",""coordinates"":[11.8201,53.4225],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 72;;;;false;2;0.4;1.0;Niederspannung -3b86661a-187d-4aa6-bf37-2014789afc08;"{""type"":""Point"",""coordinates"":[11.8214,53.4218],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 20;;;;false;2;0.4;1.0;Niederspannung -a3ed5af5-1fcb-4fce-af0f-708d3d604124;"{""type"":""Point"",""coordinates"":[11.8196,53.4282],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 33;;;;false;2;0.4;1.0;Niederspannung -ec8f2c82-a1b2-487c-b573-250859e3b414;"{""type"":""Point"",""coordinates"":[11.8166,53.4241],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";MV2.101 Bus 5;;;;true;1;20.0;1.025;Mittelspannung -5f153bf6-4f25-41f1-8545-18fe6323bc49;"{""type"":""Point"",""coordinates"":[11.8175,53.4257],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 17;;;;false;2;0.4;1.0;Niederspannung -b565ae3b-68f9-4bca-816d-9b0fc1c6b13f;"{""type"":""Point"",""coordinates"":[11.8198,53.4221],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 70;;;;false;2;0.4;1.0;Niederspannung -8e809cf6-7e05-437c-89a6-f6ca135a403b;"{""type"":""Point"",""coordinates"":[11.815,53.4289],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 47;;;;false;2;0.4;1.0;Niederspannung -ca0b9e34-ddc8-405e-86a4-ab939626c4a2;"{""type"":""Point"",""coordinates"":[11.816,53.4214],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 99;;;;false;2;0.4;1.0;Niederspannung -3e6be3ac-2b51-4080-b815-391313612fc7;"{""type"":""Point"",""coordinates"":[11.815,53.4277],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 110;;;;false;2;0.4;1.0;Niederspannung -4749ab2b-4d96-4100-8081-73e77c797d6b;"{""type"":""Point"",""coordinates"":[11.8166,53.4241],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 73;;;;false;2;0.4;1.0;Niederspannung -3f6c26dd-842b-4dee-b71f-4aa32e2654ff;"{""type"":""Point"",""coordinates"":[11.8174,53.4281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 97;;;;false;2;0.4;1.0;Niederspannung -c6dac3ab-f44f-4b87-800c-0f4da64673f1;"{""type"":""Point"",""coordinates"":[11.8191,53.422],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 10;;;;false;2;0.4;1.0;Niederspannung -eae8a04c-44f2-4da3-95f6-cae48f85737c;"{""type"":""Point"",""coordinates"":[11.8203,53.4221],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 45;;;;false;2;0.4;1.0;Niederspannung -94fe96b1-f36a-4edd-a107-4ff0376f1066;"{""type"":""Point"",""coordinates"":[11.8175,53.4265],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 61;;;;false;2;0.4;1.0;Niederspannung -31e6e197-719d-4aaf-8ca5-ab9e7549390e;"{""type"":""Point"",""coordinates"":[11.8196,53.4286],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 32;;;;false;2;0.4;1.0;Niederspannung -002a4495-96e4-49c9-abbe-8fccb3e9c83e;"{""type"":""Point"",""coordinates"":[11.8214,53.4229],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 90;;;;false;2;0.4;1.0;Niederspannung -c7e48384-5699-4a38-a887-7e15a9145202;"{""type"":""Point"",""coordinates"":[11.8191,53.4223],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 107;;;;false;2;0.4;1.0;Niederspannung -8ace5c2b-584a-4015-990f-6f1e14de4ddb;"{""type"":""Point"",""coordinates"":[11.8174,53.4273],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 27;;;;false;2;0.4;1.0;Niederspannung -c41ebab8-16a0-4a3a-b4af-26073932d462;"{""type"":""Point"",""coordinates"":[11.8168,53.4285],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 35;;;;false;2;0.4;1.0;Niederspannung -ab8c8f6c-e7a2-4b81-a0d7-5f13789267a2;"{""type"":""Point"",""coordinates"":[11.815,53.4269],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 113;;;;false;2;0.4;1.0;Niederspannung -9f1baf4e-12e1-41d1-8efc-81cfc78f1957;"{""type"":""Point"",""coordinates"":[11.8195,53.426],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 71;;;;false;2;0.4;1.0;Niederspannung -02e63e81-2e62-4ef6-8fdb-0b0905a437b6;"{""type"":""Point"",""coordinates"":[11.8191,53.4212],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 12;;;;false;2;0.4;1.0;Niederspannung -2d33314e-31db-4ad4-a898-2be5d56a896d;"{""type"":""Point"",""coordinates"":[11.8162,53.4277],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 48;;;;false;2;0.4;1.0;Niederspannung -b8e95bf0-3ba8-4d53-a0bf-a3720fb785fb;"{""type"":""Point"",""coordinates"":[11.8195,53.4264],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 49;;;;false;2;0.4;1.0;Niederspannung -d0bfabdb-0e83-423b-a20a-ab9197c4284e;"{""type"":""Point"",""coordinates"":[11.8213,53.4239],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 82;;;;false;2;0.4;1.0;Niederspannung -6b2881ce-3965-4f5e-98a3-74eb47b0a7ca;"{""type"":""Point"",""coordinates"":[11.8162,53.4284],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 53;;;;false;2;0.4;1.0;Niederspannung -af2b07ce-1a96-4b50-9e21-badf29eed519;"{""type"":""Point"",""coordinates"":[11.8198,53.4195],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 100;;;;false;2;0.4;1.0;Niederspannung -844c0b9c-058a-4228-a8c4-bf2defff6958;"{""type"":""Point"",""coordinates"":[11.8213,53.4293],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 1;;;;false;2;0.4;1.0;Niederspannung -a432b8ce-0462-478b-83e7-3107cd2e909c;"{""type"":""Point"",""coordinates"":[11.8213,53.4253],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 56;;;;false;2;0.4;1.0;Niederspannung -4129e079-6712-4275-911c-36729d698c42;"{""type"":""Point"",""coordinates"":[11.8166,53.4238],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 77;;;;false;2;0.4;1.0;Niederspannung -883edf38-9a18-4f61-981a-691aaf436cc7;"{""type"":""Point"",""coordinates"":[11.8198,53.4216],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 4;;;;false;2;0.4;1.0;Niederspannung -f2724954-34d3-4ddf-b6b0-7a1531639990;"{""type"":""Point"",""coordinates"":[11.8191,53.4235],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 6;;;;false;2;0.4;1.0;Niederspannung -5d3bcf55-0520-43ff-8d63-3d0eb421e442;"{""type"":""Point"",""coordinates"":[11.816,53.4234],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 91;;;;false;2;0.4;1.0;Niederspannung -3a557b4e-06b8-4f29-929f-81d95c42c897;"{""type"":""Point"",""coordinates"":[11.8213,53.4277],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 42;;;;false;2;0.4;1.0;Niederspannung -7dc43c81-9a61-45a0-9745-800a28bf4a9d;"{""type"":""Point"",""coordinates"":[11.8196,53.4278],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 50;;;;false;2;0.4;1.0;Niederspannung -fd4bebb8-40ca-4eed-92c0-cdd10b86ac20;"{""type"":""Point"",""coordinates"":[11.8191,53.4209],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 60;;;;false;2;0.4;1.0;Niederspannung -3464496c-7dd4-41e9-ae0a-99ade0b51572;"{""type"":""Point"",""coordinates"":[11.8213,53.4236],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 40;;;;false;2;0.4;1.0;Niederspannung -d38d936a-9c05-4bdc-8331-418fef27f492;"{""type"":""Point"",""coordinates"":[11.815,53.4281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 109;;;;false;2;0.4;1.0;Niederspannung -58b551b6-83bd-4f1c-8d9c-8c9a7f638c0b;"{""type"":""Point"",""coordinates"":[11.8213,53.425],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 38;;;;false;2;0.4;1.0;Niederspannung -b5c1e826-63fd-4b0c-bec0-0c758389ef58;"{""type"":""Point"",""coordinates"":[11.8191,53.4239],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 104;;;;false;2;0.4;1.0;Niederspannung -7c35a794-f569-4a9c-acb0-d03647610086;"{""type"":""Point"",""coordinates"":[11.8166,53.4249],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 98;;;;false;2;0.4;1.0;Niederspannung -78815cf6-70db-432c-96e6-87fe8cf67eee;"{""type"":""Point"",""coordinates"":[11.815,53.4272],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 112;;;;false;2;0.4;1.0;Niederspannung -c5b6bfaf-1621-40a7-9c53-02cfb59c04d9;"{""type"":""Point"",""coordinates"":[11.822,53.4294],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 14;;;;false;2;0.4;1.0;Niederspannung -2b3d7fb8-0583-4d47-97b1-3b5f232fd462;"{""type"":""Point"",""coordinates"":[11.8213,53.4274],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 87;;;;false;2;0.4;1.0;Niederspannung -3fcb94e3-7781-4d83-9030-d9853822e78e;"{""type"":""Point"",""coordinates"":[11.8213,53.4243],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 85;;;;false;2;0.4;1.0;Niederspannung -f713593a-3fd3-410a-ac08-74202d4f5798;"{""type"":""Point"",""coordinates"":[11.8201,53.4236],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 79;;;;false;2;0.4;1.0;Niederspannung -47246a84-ad0e-4d04-9d98-1c9cd5d363c1;"{""type"":""Point"",""coordinates"":[11.8196,53.4289],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 16;;;;false;2;0.4;1.0;Niederspannung -9cdb3115-cc00-4d61-bc33-442e8f30fb63;"{""type"":""Point"",""coordinates"":[11.8213,53.4257],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 83;;;;false;2;0.4;1.0;Niederspannung -839ff0f4-93db-42ec-a928-bbc448b6cf5c;"{""type"":""Point"",""coordinates"":[11.8197,53.4207],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 69;;;;false;2;0.4;1.0;Niederspannung -e7908208-77b4-4059-806e-4857262992fc;"{""type"":""Point"",""coordinates"":[11.8196,53.4247],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 105;;;;false;2;0.4;1.0;Niederspannung -83da8d60-405a-45f7-9bb9-9d35607b7927;"{""type"":""Point"",""coordinates"":[11.8201,53.4229],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 2;;;;false;2;0.4;1.0;Niederspannung -9d136a6b-5fdc-44ed-a5ed-599a55281024;"{""type"":""Point"",""coordinates"":[11.8195,53.4275],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 9;;;;false;2;0.4;1.0;Niederspannung -3faac527-0ff3-44a7-9e4f-24a41940da90;"{""type"":""Point"",""coordinates"":[11.816,53.4227],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 15;;;;false;2;0.4;1.0;Niederspannung -5682ac05-7336-4ebc-a5d1-3b69c79fb3b1;"{""type"":""Point"",""coordinates"":[11.8166,53.4241],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 44;;;;false;2;0.4;1.0;Niederspannung -5b73ded9-3ca4-4f18-a2ab-c27b9a3dcf9d;"{""type"":""Point"",""coordinates"":[11.8166,53.423],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 36;;;;false;2;0.4;1.0;Niederspannung -594d101c-3a05-45e3-a061-9189f3e848b7;"{""type"":""Point"",""coordinates"":[11.8198,53.4192],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 88;;;;false;2;0.4;1.0;Niederspannung -7b81b518-00e0-4ff1-b4cf-876903958d7a;"{""type"":""Point"",""coordinates"":[11.8213,53.4271],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 8;;;;false;2;0.4;1.0;Niederspannung -a882e666-82d1-4ba6-87df-fc702fe06187;"{""type"":""Point"",""coordinates"":[11.8169,53.4289],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 74;;;;false;2;0.4;1.0;Niederspannung -1403edf9-e47c-4705-8563-83bcd639482e;"{""type"":""Point"",""coordinates"":[11.8213,53.4286],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 75;;;;false;2;0.4;1.0;Niederspannung -03b2aa45-84f6-48c0-9dab-427e046a5672;"{""type"":""Point"",""coordinates"":[11.8196,53.4253],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 22;;;;false;2;0.4;1.0;Niederspannung -6ee7ea93-ea9c-40cb-b79a-1c5f287c97a5;"{""type"":""Point"",""coordinates"":[11.8221,53.4286],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 66;;;;false;2;0.4;1.0;Niederspannung -88cf719a-92df-4dfd-9a83-f84330e28fe0;"{""type"":""Point"",""coordinates"":[11.815,53.4285],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 108;;;;false;2;0.4;1.0;Niederspannung -80962bd3-a10f-4ed2-ba6a-3e802189939c;"{""type"":""Point"",""coordinates"":[11.8201,53.4239],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 21;;;;false;2;0.4;1.0;Niederspannung -80b8d1f8-7e83-421d-a95a-c193fc35f4f7;"{""type"":""Point"",""coordinates"":[11.8175,53.427],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 95;;;;false;2;0.4;1.0;Niederspannung -b909fb45-b6ee-427f-afd7-e8a0ec7274c6;"{""type"":""Point"",""coordinates"":[11.8213,53.4225],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 92;;;;false;2;0.4;1.0;Niederspannung -57b40047-4f9d-46bb-bf19-c0a86bbd4f5b;"{""type"":""Point"",""coordinates"":[11.8166,53.4234],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 37;;;;false;2;0.4;1.0;Niederspannung -18b4157b-0e47-4c5a-adb8-ccae47372336;"{""type"":""Point"",""coordinates"":[11.8213,53.4264],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 5;;;;false;2;0.4;1.0;Niederspannung -8726dc29-621e-4455-a541-cd88d7da457f;"{""type"":""Point"",""coordinates"":[11.8166,53.4257],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 96;;;;false;2;0.4;1.0;Niederspannung -814f784b-687f-4dd5-8a91-c7772c916d46;"{""type"":""Point"",""coordinates"":[11.8174,53.4262],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 19;;;;false;2;0.4;1.0;Niederspannung -7efabb8d-ba17-4487-96d9-5744b1fedf8a;"{""type"":""Point"",""coordinates"":[11.8166,53.4245],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 23;;;;false;2;0.4;1.0;Niederspannung -15a86f7d-fb73-49a4-af6a-25b14122378d;"{""type"":""Point"",""coordinates"":[11.8191,53.4227],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 11;;;;false;2;0.4;1.0;Niederspannung -34031e92-3444-47d5-94ae-cceeb5d96bb2;"{""type"":""Point"",""coordinates"":[11.8195,53.4268],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 106;;;;false;2;0.4;1.0;Niederspannung -35748e60-3be8-4930-8a61-209fd5df1bec;"{""type"":""Point"",""coordinates"":[11.815,53.4293],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 30;;;;false;2;0.4;1.0;Niederspannung -50cac08e-bf24-4526-9466-53ca5edccd15;"{""type"":""Point"",""coordinates"":[11.8166,53.4253],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 64;;;;false;2;0.4;1.0;Niederspannung -2efac9b1-fb0d-4e08-bfac-501798826deb;"{""type"":""Point"",""coordinates"":[11.8191,53.4231],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 93;;;;false;2;0.4;1.0;Niederspannung -c86d6361-4159-4787-b5f4-e41dcaa95195;"{""type"":""Point"",""coordinates"":[11.8182,53.4257],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 13;;;;false;2;0.4;1.0;Niederspannung diff --git a/input/samples/simopsimtestgrid_reduced/fullGrid/pv_input.csv b/input/samples/simopsimtestgrid_reduced/fullGrid/pv_input.csv deleted file mode 100755 index b883b51783..0000000000 --- a/input/samples/simopsimtestgrid_reduced/fullGrid/pv_input.csv +++ /dev/null @@ -1,4 +0,0 @@ -uuid;albedo;azimuth;cos_phi_rated;elevation_angle;eta_conv;id;k_g;k_t;market_reaction;node;operates_from;operates_until;operator;q_characteristics;s_rated;em -a1eb7fc1-3bee-4b65-a387-ef3046644bf0;0.20000000298023224;4.093344211578369;0.8999999761581421;37.69556427001953;97.0;LV5.201 PV 10;0.8999999761581421;1.0;false;dc54bd8a-b7d8-4e99-adb0-d6ee5084241c;;;;"cosPhiFixed:{(0.0,0.9)}";4.2;f9dc7ce6-658c-4101-a12f-d58bb889286b -de8cfef5-7620-4b9e-9a10-1faebb5a80c0;0.20000000298023224;4.093344211578369;0.8999999761581421;37.69556427001953;97.0;LV5.201 PV 11;0.8999999761581421;1.0;false;3e6be3ac-2b51-4080-b815-391313612fc7;;;;"cosPhiFixed:{(0.0,0.9)}";10.9;957938b7-0476-4fab-a1b3-6ce8615857b3 -2560c371-f420-4c2a-b4e6-e04c11b64c03;0.20000000298023224;0.7802008390426636;0.8999999761581421;40.086585998535156;98.0;LV5.201 PV 15;0.8999999761581421;1.0;false;43040a39-8b6c-401f-9dfd-82b42aa6dec6;;;;"cosPhiFixed:{(0.0,0.9)}";2.9;c3a7e9f5-b492-4c85-af2d-1e93f6a25443 diff --git a/input/samples/simopsimtestgrid_reduced/fullGrid/transformer_2_w_input.csv b/input/samples/simopsimtestgrid_reduced/fullGrid/transformer_2_w_input.csv deleted file mode 100755 index d1f9f5d13c..0000000000 --- a/input/samples/simopsimtestgrid_reduced/fullGrid/transformer_2_w_input.csv +++ /dev/null @@ -1,2 +0,0 @@ -uuid;auto_tap;id;node_a;node_b;operates_from;operates_until;operator;parallel_devices;tap_pos;type -adaba416-9b52-45df-9d91-f67d0dd28ecb;false;MV2.101-LV5.201-Trafo 1;ec8f2c82-a1b2-487c-b573-250859e3b414;4749ab2b-4d96-4100-8081-73e77c797d6b;;;;1;0;417407d2-1e74-4f37-9b64-f701f53f8842 diff --git a/input/samples/simopsimtestgrid_reduced/fullGrid/transformer_2_w_type_input.csv b/input/samples/simopsimtestgrid_reduced/fullGrid/transformer_2_w_type_input.csv deleted file mode 100755 index b13848b159..0000000000 --- a/input/samples/simopsimtestgrid_reduced/fullGrid/transformer_2_w_type_input.csv +++ /dev/null @@ -1,2 +0,0 @@ -uuid;b_m;d_phi;d_v;g_m;id;r_sc;s_rated;tap_max;tap_min;tap_neutr;tap_side;v_rated_a;v_rated_b;x_sc -417407d2-1e74-4f37-9b64-f701f53f8842;-36.47380569074435;0.0;2.5;4124.999999999999;0.63 MVA 20/0.4 kV Dyn5 ASEA;6.953892668178382;630.0;2;-2;0;false;20.0;0.4;37.45518044666632 diff --git a/input/samples/simopsimtestgrid_reduced/simopsimtestgrid.conf b/input/samples/simopsimtestgrid_reduced/simopsimtestgrid.conf deleted file mode 100755 index e3d997c666..0000000000 --- a/input/samples/simopsimtestgrid_reduced/simopsimtestgrid.conf +++ /dev/null @@ -1,240 +0,0 @@ -include "../common/pekko.conf" - -######### -# ATTENTION: Do not change this config file directly but use it as a base for your personal delta config for the -# vn_simona scenario! Delta configs can be created by including the config you want to change -# parameters from via include (e.g. include "input/samples/vn_simona/vn_simona.conf") at the -# beginning of your config file and then just override the parameters you want to change! -######### - -################################################################## -# Simulation Parameters -################################################################## -simona.simulationName = "simopsimtest" - -################################################################## -# Time Parameters -################################################################## -simona.time.startDateTime = "2024-02-27T00:00:00Z" -simona.time.endDateTime = "2024-02-27T23:45:00Z" -simona.time.schedulerReadyCheckWindow = 900 - -################################################################## -# Input Parameters -################################################################## -simona.input.primary.csvParams = { - directoryPath: "simona/input/samples/simopsimtestgrid_reduced/fullGrid" - csvSep: ";" - isHierarchic: false -} -simona.input.grid.datasource.id = "csv" -simona.input.grid.datasource.csvParams = { - directoryPath: "simona/input/samples/simopsimtestgrid_reduced/fullGrid" - csvSep: ";" - isHierarchic: false -} - -simona.input.weather.datasource = { - scheme = "icon" - sampleParams.use = true - coordinateSource.sampleParams.use = true - maxCoordinateDistance = 50000 -} - -################################################################## -# Output Parameters -################################################################## -simona.output.base.dir = "simona/output/simopsimtestgrid_reduced" -simona.output.base.addTimestampToOutputDir = true - -simona.output.sink.csv { - fileFormat = ".csv" - filePrefix = "" - fileSuffix = "" -} - -simona.output.grid = { - notifier = "grid" - nodes = false - lines = false - switches = false - transformers2w = false - transformers3w = false -} -simona.output.participant.defaultConfig = { - notifier = "default" - powerRequestReply = false - simulationResult = true -} -simona.output.participant.individualConfigs = [ - { - notifier = "pv" - powerRequestReply = false - simulationResult = true - }, - { - notifier = "wec" - powerRequestReply = false - simulationResult = false - }, - { - notifier = "evcs" - powerRequestReply = false - simulationResult = false - }, - { - notifier = "bm" - powerRequestReply = false - simulationResult = false - }, - { - notifier = "chp" - powerRequestReply = false - simulationResult = false - }, - { - notifier = "ev" - powerRequestReply = false - simulationResult = false - }, - { - notifier = "hp" - powerRequestReply = false - simulationResult = false - }, - { - notifier = "storage" - powerRequestReply = false - simulationResult = false - }, - { - notifier = "fixedFeedIn" - powerRequestReply = false - simulationResult = false - } -] -simona.output.thermal = { - defaultConfig = { - notifier = "default", - simulationResult = false - } - individualConfigs = [ - { - notifier = "house", - simulationResult = false - } - ] -} - -################################################################## -# Runtime Configuration // todo refactor as this naming is misleading and partly unneeded -################################################################## -simona.runtime.selected_subgrids = [] -simona.runtime.selected_volt_lvls = [] - -simona.runtime.participant.load = { - defaultConfig = { - calculateMissingReactivePowerWithModel = false - uuids = ["default"] - scaling = 1.0 - modelBehaviour = "profile" - reference = "power" - } - individualConfigs = [] -} - -simona.runtime.participant.fixedFeedIn = { - defaultConfig = { - calculateMissingReactivePowerWithModel = false - uuids = ["default"] - scaling = 1.0 - } - individualConfigs = [] -} - -simona.runtime.participant.pv = { - defaultConfig = { - calculateMissingReactivePowerWithModel = false - uuids = ["default"] - scaling = 1.0 - } - individualConfigs = [] -} - -simona.runtime.participant.wec = { - defaultConfig = { - calculateMissingReactivePowerWithModel = false - uuids = ["default"] - scaling = 1.0 - } - individualConfigs = [] -} - -simona.runtime.participant.evcs = { - defaultConfig = { - calculateMissingReactivePowerWithModel = false - uuids = ["default"] - scaling = 1.0 - } - individualConfigs = [] -} - -simona.runtime.participant.hp = { - defaultConfig = { - calculateMissingReactivePowerWithModel = false - uuids = ["default"] - scaling = 1.0 - } - individualConfigs = [] -} - -# # # # # -# ATTENTION: calculateMissingReactivePowerWithModel and scaling is ignored here. -# # # # # -simona.runtime.participant.em = { - defaultConfig = { - calculateMissingReactivePowerWithModel = false - uuids = ["default"] - scaling = 1.0 - } - individualConfigs = [] -} - -################################################################## -# Event Configuration -################################################################## -simona.event.listener = [] - -################################################################## -# Grid Configuration -################################################################## - -simona.gridConfig.refSystems = [ - {sNom = "100 kVA", vNom = "0.4 kV", voltLvls = [{id = "NS", vNom = "0.4 kV"}]}, - {sNom = "60 MVA", vNom = "20 kV", voltLvls = [{id = "MS", vNom = "20 kV"}]}, - {sNom = "600 MVA", vNom = "110 kV", voltLvls = [{id = "HS", vNom = "110 kV"}]}, - {sNom = "1000 MVA", vNom = "380 kV", voltLvls = [{id = "HoeS", vNom = "380 kV"}]} -] - -################################################################## -# Power Flow Configuration -################################################################## -simona.powerflow.maxSweepPowerDeviation = 1E-5 // the maximum allowed deviation in power between two sweeps, before overall convergence is assumed -simona.powerflow.newtonraphson.epsilon = [1E-12] -simona.powerflow.newtonraphson.iterations = 50 -simona.powerflow.resolution = "87300s" -simona.powerflow.stopOnFailure = true - -simona.control.transformer = [ - { - transformers = ["31a2b9bf-e785-4475-aa44-1c34646e8c79"], - measurements = ["923f2d69-3093-4198-86e4-13d2d1c220f8"], - vMin = 0.98, - vMax = 1.02 - }, { - transformers = ["1132dbf4-e8a1-44ae-8415-f42d4497aa1d"], - measurements = ["7686b818-a0ba-465c-8e4e-f7d3c4e171fc"], - vMin = 0.98, - vMax = 1.02 - } -] diff --git a/input/samples/simopsimtestgrid_reduced_reference/fullGrid/em_input.csv b/input/samples/simopsimtestgrid_reduced_reference/fullGrid/em_input.csv deleted file mode 100644 index 100867a21d..0000000000 --- a/input/samples/simopsimtestgrid_reduced_reference/fullGrid/em_input.csv +++ /dev/null @@ -1,4 +0,0 @@ -uuid;control_strategy;parent_em;id;operates_from;operates_until;operator -c3a7e9f5-b492-4c85-af2d-1e93f6a25443;uncontrolled;;EM_HH_Bus_25;;; -f9dc7ce6-658c-4101-a12f-d58bb889286b;uncontrolled;;EM_HH_Bus_81;;; -957938b7-0476-4fab-a1b3-6ce8615857b3;uncontrolled;;EM_HH_Bus_110;;; diff --git a/input/samples/simopsimtestgrid_reduced_reference/fullGrid/line_input.csv b/input/samples/simopsimtestgrid_reduced_reference/fullGrid/line_input.csv deleted file mode 100644 index f4ff92fbba..0000000000 --- a/input/samples/simopsimtestgrid_reduced_reference/fullGrid/line_input.csv +++ /dev/null @@ -1,110 +0,0 @@ -uuid;geo_position;id;length;node_a;node_b;olm_characteristic;operates_from;operates_until;operator;parallel_devices;type -bdbbb247-57b7-473b-9411-53fcf35032db;"{""type"":""LineString"",""coordinates"":[[11.8213,53.426],[11.8213,53.4257]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 27;0.0161277;093160c4-6482-4c58-b952-217c615e3ada;9cdb3115-cc00-4d61-bc33-442e8f30fb63;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -c48eeb2c-6858-4f96-9a90-20641cc0903d;"{""type"":""LineString"",""coordinates"":[[11.8214,53.4229],[11.8213,53.4225]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 35;0.0150703;002a4495-96e4-49c9-abbe-8fccb3e9c83e;b909fb45-b6ee-427f-afd7-e8a0ec7274c6;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -97e498ae-a4f7-4485-a703-b8c00d0a5e8a;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4221],[11.8201,53.4225]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 8;0.0228449;b565ae3b-68f9-4bca-816d-9b0fc1c6b13f;a7725293-05fc-447f-bc12-38b689b0a956;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -f907f152-7bfe-4a17-a63f-adda500f6f0e;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4241],[11.8184,53.4293]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 107;0.04;4749ab2b-4d96-4100-8081-73e77c797d6b;ea4a6507-e504-4542-be5f-1019719b2257;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -6bf14400-8f6c-41ef-b6a4-9e861a3ad08f;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4241],[11.822,53.4294]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 3;0.030304;4749ab2b-4d96-4100-8081-73e77c797d6b;c5b6bfaf-1621-40a7-9c53-02cfb59c04d9;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -18b92b71-320d-4294-a47e-6715f1594755;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4243],[11.8196,53.4247]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 102;0.0251089;f4da61e4-7600-4cd1-95b6-c70b56c049fc;e7908208-77b4-4059-806e-4857262992fc;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -4128307f-3e00-4da9-b629-b696b72165a4;"{""type"":""LineString"",""coordinates"":[[11.815,53.4289],[11.815,53.4293]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 28;0.0377768;8e809cf6-7e05-437c-89a6-f6ca135a403b;35748e60-3be8-4930-8a61-209fd5df1bec;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -92876b53-6bbc-48ff-ba5f-5f5c08313e4d;"{""type"":""LineString"",""coordinates"":[[11.815,53.4277],[11.815,53.4272]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 111;0.031;3e6be3ac-2b51-4080-b815-391313612fc7;78815cf6-70db-432c-96e6-87fe8cf67eee;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -4b557d0c-9992-48b1-b45e-26cccc01db03;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4235],[11.8191,53.4231]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 90;0.00167204;f2724954-34d3-4ddf-b6b0-7a1531639990;2efac9b1-fb0d-4e08-bfac-501798826deb;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -0652d3e0-8c7c-4be2-9a41-00534aa9774b;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4212],[11.8191,53.4209]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 94;0.00182131;02e63e81-2e62-4ef6-8fdb-0b0905a437b6;fd4bebb8-40ca-4eed-92c0-cdd10b86ac20;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -782d8c4c-accb-424b-91ae-9af1b5d4b1af;"{""type"":""LineString"",""coordinates"":[[11.8201,53.4225],[11.8201,53.4229]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 79;0.0338129;a7725293-05fc-447f-bc12-38b689b0a956;83da8d60-405a-45f7-9bb9-9d35607b7927;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -b7cac7a5-f528-45bd-8ced-a16234979e13;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4239],[11.8191,53.4235]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 89;0.0111411;b5c1e826-63fd-4b0c-bec0-0c758389ef58;f2724954-34d3-4ddf-b6b0-7a1531639990;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -7ac1a715-e141-429a-8cfe-ff2badd41074;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4257],[11.8174,53.4262]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 65;0.019;8726dc29-621e-4455-a541-cd88d7da457f;814f784b-687f-4dd5-8a91-c7772c916d46;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -77ae4f1e-9ecb-4dda-a5f2-ce7ae3f9bbec;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4264],[11.8213,53.426]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 23;0.00488535;18b4157b-0e47-4c5a-adb8-ccae47372336;093160c4-6482-4c58-b952-217c615e3ada;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -6aaa54d9-d7b7-4adc-a85a-335cdf1393d6;"{""type"":""LineString"",""coordinates"":[[11.8162,53.4284],[11.8169,53.4289]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 40;0.036;6b2881ce-3965-4f5e-98a3-74eb47b0a7ca;a882e666-82d1-4ba6-87df-fc702fe06187;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -c85c107e-82c0-4002-acfa-d7000512a2ad;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4274],[11.8213,53.4271]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 20;0.0182765;2b3d7fb8-0583-4d47-97b1-3b5f232fd462;7b81b518-00e0-4ff1-b4cf-876903958d7a;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -2ef5f888-4341-4eee-b505-ae07a9a60c8d;"{""type"":""LineString"",""coordinates"":[[11.8201,53.4236],[11.8201,53.4239]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 55;0.0259846;f713593a-3fd3-410a-ac08-74202d4f5798;80962bd3-a10f-4ed2-ba6a-3e802189939c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -139ea4f9-9b7f-4825-8919-537a94ff4794;"{""type"":""LineString"",""coordinates"":[[11.8175,53.4277],[11.8174,53.4281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 59;0.00160952;7f692039-eef6-45f6-9e30-b5983f6750a5;3f6c26dd-842b-4dee-b71f-4aa32e2654ff;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -68853808-8b09-4ff2-9c92-88ed1d78c639;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4236],[11.8213,53.4232]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 33;0.0217875;3464496c-7dd4-41e9-ae0a-99ade0b51572;a966644c-37d5-4400-9fa0-725fd88586a8;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -91bec60c-2951-420c-a35e-2633119ee450;"{""type"":""LineString"",""coordinates"":[[11.8162,53.4277],[11.8162,53.4281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 30;0.00820054;2d33314e-31db-4ad4-a898-2be5d56a896d;555cd075-0fe4-4a65-b027-f45cffa960d9;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -35ba0827-e27e-496e-b735-c778f3b03019;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4241],[11.8213,53.4293]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 18;0.0139169;4749ab2b-4d96-4100-8081-73e77c797d6b;844c0b9c-058a-4228-a8c4-bf2defff6958;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -60efd159-72d6-4546-b245-8b8fc120a9f8;"{""type"":""LineString"",""coordinates"":[[11.817,53.4293],[11.8166,53.4241]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 26;0.098;85751c9b-3e5e-468a-9dc7-43775b0d4a6f;4749ab2b-4d96-4100-8081-73e77c797d6b;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -212b62aa-73c2-47af-95ec-00cad8d4a4f4;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4222],[11.8214,53.4218]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 38;0.00992653;589bf32a-b361-4536-ae96-6d56d184eedb;3b86661a-187d-4aa6-bf37-2014789afc08;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -7e61def1-8414-40b3-8775-fad9124f4369;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4282],[11.8213,53.4277]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 16;0.0164234;09285b78-9c18-4af7-9c7a-942cc868016f;3a557b4e-06b8-4f29-929f-81d95c42c897;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -f9b4c7dc-c199-4691-8c6d-1faf438cf336;"{""type"":""LineString"",""coordinates"":[[11.8221,53.429],[11.8221,53.4286]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 5;0.0143698;9644f198-e801-4545-87ee-a24e2a8039bd;6ee7ea93-ea9c-40cb-b79a-1c5f287c97a5;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -eebc456e-d11b-4b19-9100-cc1d8e91f926;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4245],[11.8166,53.4249]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 71;0.00648856;7efabb8d-ba17-4487-96d9-5744b1fedf8a;7c35a794-f569-4a9c-acb0-d03647610086;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -8f4c4009-e1ed-4985-8095-729aa5dc8cdd;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4192],[11.8198,53.4195]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 85;0.00737032;594d101c-3a05-45e3-a061-9189f3e848b7;af2b07ce-1a96-4b50-9e21-badf29eed519;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -20db47c8-d154-4ab2-ad65-caa4a37466d2;"{""type"":""LineString"",""coordinates"":[[11.8175,53.4257],[11.8174,53.4262]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 68;0.037;5f153bf6-4f25-41f1-8545-18fe6323bc49;814f784b-687f-4dd5-8a91-c7772c916d46;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -7d838003-cd6e-42fd-b75a-eaa354c3993b;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4216],[11.8191,53.4212]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 93;0.0043578;8f53645e-1f28-4eb1-807f-eb2a473f9d4c;02e63e81-2e62-4ef6-8fdb-0b0905a437b6;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -a92013e1-78b7-4447-a505-f25fffbf56f2;"{""type"":""LineString"",""coordinates"":[[11.8174,53.4262],[11.8175,53.4265]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 48;0.0118531;814f784b-687f-4dd5-8a91-c7772c916d46;94fe96b1-f36a-4edd-a107-4ff0376f1066;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -690f8f61-5cc7-448e-8971-a516f784bf11;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4293],[11.8166,53.4241]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 106;0.0131451;20ad9754-e966-4ad1-9541-f968c207f3df;4749ab2b-4d96-4100-8081-73e77c797d6b;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -6a00d582-32b1-4581-bdd1-a638ca279597;"{""type"":""LineString"",""coordinates"":[[11.815,53.4272],[11.815,53.4269]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 112;0.028;78815cf6-70db-432c-96e6-87fe8cf67eee;ab8c8f6c-e7a2-4b81-a0d7-5f13789267a2;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -1d5877c1-31cd-4ee8-b0dd-667fa4fffb8a;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4277],[11.8213,53.4274]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 17;0.00286133;3a557b4e-06b8-4f29-929f-81d95c42c897;2b3d7fb8-0583-4d47-97b1-3b5f232fd462;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -a15cd6a4-4b22-434f-be9b-ad2abe6e538c;"{""type"":""LineString"",""coordinates"":[[11.815,53.4289],[11.815,53.4285]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 108;0.033;8e809cf6-7e05-437c-89a6-f6ca135a403b;88cf719a-92df-4dfd-9a83-f84330e28fe0;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -2bd81511-4f19-4374-8ac5-96c2b19eda64;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4223],[11.8191,53.422]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 52;0.00980013;c7e48384-5699-4a38-a887-7e15a9145202;c6dac3ab-f44f-4b87-800c-0f4da64673f1;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -beee40ad-aca3-490f-87d5-a6dfc4bb76e3;"{""type"":""LineString"",""coordinates"":[[11.815,53.4293],[11.8166,53.4241]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 39;0.09;35748e60-3be8-4930-8a61-209fd5df1bec;4749ab2b-4d96-4100-8081-73e77c797d6b;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -96b06f87-7905-4d12-99d8-1ed330050659;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4203],[11.8197,53.4207]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 12;0.00473835;5596da2f-ca32-4ad3-81da-1ffa17cd3d7b;839ff0f4-93db-42ec-a928-bbc448b6cf5c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -e66890ef-ec02-407f-a5bb-ce32128e7490;"{""type"":""LineString"",""coordinates"":[[11.8162,53.4281],[11.8162,53.4284]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 41;0.00828244;555cd075-0fe4-4a65-b027-f45cffa960d9;6b2881ce-3965-4f5e-98a3-74eb47b0a7ca;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -015b0337-98bd-40d4-97d3-13a0d1da88ee;"{""type"":""LineString"",""coordinates"":[[11.8213,53.425],[11.8214,53.4246]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 84;0.0489789;58b551b6-83bd-4f1c-8d9c-8c9a7f638c0b;a9288e77-2919-4db6-89eb-9737bd07f111;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -f9e6b2f1-f8fb-4763-a1d0-36d06170fea0;"{""type"":""LineString"",""coordinates"":[[11.8213,53.429],[11.8213,53.4286]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 98;0.0144283;16091b6d-f1ea-4a07-9ad4-30d595aba68d;1403edf9-e47c-4705-8563-83bcd639482e;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -f270d6b5-7c94-4397-8ab0-7c39c888d726;"{""type"":""LineString"",""coordinates"":[[11.8175,53.4265],[11.8175,53.427]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 58;0.0036937;94fe96b1-f36a-4edd-a107-4ff0376f1066;80b8d1f8-7e83-421d-a95a-c193fc35f4f7;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -af8c65aa-cfd6-434a-8512-2d80106a2f2c;"{""type"":""LineString"",""coordinates"":[[11.8195,53.426],[11.8195,53.4264]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 31;0.00626899;9f1baf4e-12e1-41d1-8efc-81cfc78f1957;b8e95bf0-3ba8-4d53-a0bf-a3720fb785fb;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -fe28e831-6405-4dfe-987b-d688367694f1;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4247],[11.8195,53.425]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 104;0.00221503;e7908208-77b4-4059-806e-4857262992fc;29516ae3-6676-4797-99c1-1f0a32b989d8;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -eac56d69-5500-4261-9690-adb16c867485;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4231],[11.8191,53.4227]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 92;0.0149219;2efac9b1-fb0d-4e08-bfac-501798826deb;15a86f7d-fb73-49a4-af6a-25b14122378d;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -183f3976-48ca-42b2-9af9-7998436fac5b;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4289],[11.8196,53.4293]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 105;0.00721947;47246a84-ad0e-4d04-9d98-1c9cd5d363c1;20ad9754-e966-4ad1-9541-f968c207f3df;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -7feef458-03f3-4d23-b3a2-e6f1035398c4;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4243],[11.8213,53.4239]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 29;0.0204277;3fcb94e3-7781-4d83-9030-d9853822e78e;d0bfabdb-0e83-423b-a20a-ab9197c4284e;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -592cd979-16c9-43d8-a311-8ac938aa5d03;"{""type"":""LineString"",""coordinates"":[[11.8174,53.4262],[11.8182,53.4257]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 47;0.0181601;814f784b-687f-4dd5-8a91-c7772c916d46;c86d6361-4159-4787-b5f4-e41dcaa95195;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -684146e5-3b58-43b4-9589-45325ab1c0bc;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4257],[11.8213,53.4253]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 24;0.0102233;9cdb3115-cc00-4d61-bc33-442e8f30fb63;a432b8ce-0462-478b-83e7-3107cd2e909c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -d9f9ee76-b016-4588-ac6d-46681894ada7;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4188],[11.8198,53.4192]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 74;0.00421724;952c6b04-0d02-4ea2-a6cc-bb44fbbe4e52;594d101c-3a05-45e3-a061-9189f3e848b7;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -1f7e9cae-460e-47db-8cb8-da5d9f695fd8;"{""type"":""LineString"",""coordinates"":[[11.8197,53.4207],[11.8198,53.4211]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 13;0.00987733;839ff0f4-93db-42ec-a928-bbc448b6cf5c;27b84da5-478e-4a05-8fe7-a9f800db5eff;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -f3a592d0-0fd7-42ea-b928-f39473b419aa;"{""type"":""LineString"",""coordinates"":[[11.8214,53.4246],[11.8213,53.4243]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 10;0.0344866;a9288e77-2919-4db6-89eb-9737bd07f111;3fcb94e3-7781-4d83-9030-d9853822e78e;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -f505bff9-0803-415f-a765-9da981ff6024;"{""type"":""LineString"",""coordinates"":[[11.8195,53.4264],[11.8195,53.4268]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 43;0.00839819;b8e95bf0-3ba8-4d53-a0bf-a3720fb785fb;34031e92-3444-47d5-94ae-cceeb5d96bb2;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -c6501262-2e05-462b-8872-445d2aa1cab8;"{""type"":""LineString"",""coordinates"":[[11.8195,53.4272],[11.8195,53.4275]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 99;0.0157523;120eaa58-a500-4ae2-a86a-56a40b931ec1;9d136a6b-5fdc-44ed-a5ed-599a55281024;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -caac4b4b-4871-4e20-994c-6517931546cb;"{""type"":""LineString"",""coordinates"":[[11.8201,53.4229],[11.8201,53.4232]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 91;0.0220129;83da8d60-405a-45f7-9bb9-9d35607b7927;24b63115-12eb-4e77-b9ef-ca474fed960f;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -e9721561-53c0-45cc-a8ed-28861ef9dc66;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4216],[11.8198,53.4221]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 7;0.0333523;883edf38-9a18-4f61-981a-691aaf436cc7;b565ae3b-68f9-4bca-816d-9b0fc1c6b13f;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -ab143df6-d050-47b6-911a-93e462d928ac;"{""type"":""LineString"",""coordinates"":[[11.8173,53.4234],[11.8166,53.4238]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 73;0.0287121;e68a088d-cf1a-40b7-9b1a-e0933352f4e6;4129e079-6712-4275-911c-36729d698c42;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -ab7ab785-36ba-4da1-b176-a7c636cb1372;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4267],[11.8213,53.4264]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 22;0.0102382;dc54bd8a-b7d8-4e99-adb0-d6ee5084241c;18b4157b-0e47-4c5a-adb8-ccae47372336;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -1ae90a03-52a3-40ef-8e06-4ba01888aa5c;"{""type"":""LineString"",""coordinates"":[[11.816,53.4222],[11.816,53.4227]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 44;0.00743191;62d603c3-f306-40b3-a665-ba9892d226f0;3faac527-0ff3-44a7-9e4f-24a41940da90;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -20be9235-f4db-4753-9fa1-223c8519fcd3;"{""type"":""LineString"",""coordinates"":[[11.8175,53.4253],[11.8175,53.4257]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 66;0.00791441;73e7a7e8-2154-46ea-9727-a4916af3570c;5f153bf6-4f25-41f1-8545-18fe6323bc49;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -558e1545-a944-419a-9153-83caa09e1a3c;"{""type"":""LineString"",""coordinates"":[[11.8162,53.4273],[11.8162,53.4277]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 42;0.00357726;0d6ef8f6-0ba1-4fa7-8e63-e55cee12b165;2d33314e-31db-4ad4-a898-2be5d56a896d;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -38d60cf7-6099-4bc0-a616-0f0b66c70c9a;"{""type"":""LineString"",""coordinates"":[[11.8201,53.4232],[11.8201,53.4236]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 67;0.0179092;24b63115-12eb-4e77-b9ef-ca474fed960f;f713593a-3fd3-410a-ac08-74202d4f5798;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -e74dbbe9-948f-4056-8134-fd1d9d39e773;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4253],[11.8196,53.4257]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 86;0.0190003;03b2aa45-84f6-48c0-9dab-427e046a5672;79e19265-08e8-407f-ae95-2f78e344d3a4;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -24c0dc1c-2e99-46ed-a52b-5a7aae7c9afb;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4249],[11.8166,53.4253]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 72;0.00189914;7c35a794-f569-4a9c-acb0-d03647610086;50cac08e-bf24-4526-9466-53ca5edccd15;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -41a72cb2-f037-4196-a248-2b18a578db50;"{""type"":""LineString"",""coordinates"":[[11.816,53.421],[11.816,53.4214]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 78;0.00156586;98c14f60-e196-4f12-903b-8485f1eacb16;ca0b9e34-ddc8-405e-86a4-ab939626c4a2;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -e2919117-f70a-4b32-a5ab-f077fe0a84ef;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4253],[11.8166,53.4257]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 70;0.00263875;50cac08e-bf24-4526-9466-53ca5edccd15;8726dc29-621e-4455-a541-cd88d7da457f;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -7d3ee5e1-1749-4e43-afeb-4ccf5f431312;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4282],[11.8196,53.4286]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 103;0.011354;a3ed5af5-1fcb-4fce-af0f-708d3d604124;31e6e197-719d-4aaf-8ca5-ab9e7549390e;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -cbba4a6b-27d1-4724-a193-134af220a57d;"{""type"":""LineString"",""coordinates"":[[11.8182,53.4257],[11.8182,53.4254]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 49;0.018118;c86d6361-4159-4787-b5f4-e41dcaa95195;c6c20ab1-16fb-4f82-8017-273022da8bb0;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -b84fad90-182c-4ac1-b77d-394f39024ffc;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4286],[11.8196,53.4289]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 19;0.0115943;31e6e197-719d-4aaf-8ca5-ab9e7549390e;47246a84-ad0e-4d04-9d98-1c9cd5d363c1;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -6c87a571-9d8b-4df9-bd86-cde7954bee28;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4271],[11.8213,53.4267]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 21;0.00282037;7b81b518-00e0-4ff1-b4cf-876903958d7a;dc54bd8a-b7d8-4e99-adb0-d6ee5084241c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -536ef1a2-b988-4474-a5d4-1254598c4716;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4227],[11.8191,53.4223]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 82;0.00544392;15a86f7d-fb73-49a4-af6a-25b14122378d;c7e48384-5699-4a38-a887-7e15a9145202;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -c0c99b3a-1c61-47b8-931d-571d9494d98f;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4241],[11.8166,53.4245]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 51;0.0182324;5682ac05-7336-4ebc-a5d1-3b69c79fb3b1;7efabb8d-ba17-4487-96d9-5744b1fedf8a;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -d3366635-447e-4d45-b839-7a7561f869a3;"{""type"":""LineString"",""coordinates"":[[11.822,53.4294],[11.8221,53.429]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 4;0.0141339;c5b6bfaf-1621-40a7-9c53-02cfb59c04d9;9644f198-e801-4545-87ee-a24e2a8039bd;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -932dd0d7-2051-4a3a-a0e3-489210d3b763;"{""type"":""LineString"",""coordinates"":[[11.8169,53.4289],[11.817,53.4293]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 57;0.0219657;a882e666-82d1-4ba6-87df-fc702fe06187;85751c9b-3e5e-468a-9dc7-43775b0d4a6f;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -e718d253-34ea-4c00-a653-80ac0af2934e;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4293],[11.8213,53.429]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 11;0.00557129;844c0b9c-058a-4228-a8c4-bf2defff6958;16091b6d-f1ea-4a07-9ad4-30d595aba68d;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -3ac954bf-5c1e-428a-9da7-37331f53d8fe;"{""type"":""LineString"",""coordinates"":[[11.8175,53.4248],[11.8175,53.4253]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 69;0.0017759;0d6c3f2b-5296-4ec1-995c-b150e72f035f;73e7a7e8-2154-46ea-9727-a4916af3570c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -7e9a0440-2a62-4daf-a4ab-e9f14f6cfd77;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4238],[11.8166,53.4234]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 53;0.0337731;4129e079-6712-4275-911c-36729d698c42;57b40047-4f9d-46bb-bf19-c0a86bbd4f5b;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -10a2050e-173b-43a7-91a0-157cee5c900d;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4257],[11.8195,53.426]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 87;0.00654022;79e19265-08e8-407f-ae95-2f78e344d3a4;9f1baf4e-12e1-41d1-8efc-81cfc78f1957;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -0dfd30a5-394e-49ff-95e9-50e73bf2604c;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4232],[11.8214,53.4229]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 34;0.0199334;a966644c-37d5-4400-9fa0-725fd88586a8;002a4495-96e4-49c9-abbe-8fccb3e9c83e;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -847934a0-e2b7-4caf-bb40-2e6af5bcb0ab;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4238],[11.8166,53.4241]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 50;0.0101795;4129e079-6712-4275-911c-36729d698c42;5682ac05-7336-4ebc-a5d1-3b69c79fb3b1;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -2daf9021-eb24-4d0c-bced-00e713a65b3a;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4225],[11.8213,53.4222]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 36;0.010188;b909fb45-b6ee-427f-afd7-e8a0ec7274c6;589bf32a-b361-4536-ae96-6d56d184eedb;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -2f1b02a3-5a58-4723-ab8f-827118a4c611;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4239],[11.8213,53.4236]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 32;0.0188563;d0bfabdb-0e83-423b-a20a-ab9197c4284e;3464496c-7dd4-41e9-ae0a-99ade0b51572;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -a5e9b6b0-2afc-49b0-8aad-705410b823c2;"{""type"":""LineString"",""coordinates"":[[11.815,53.4285],[11.815,53.4281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 109;0.027;88cf719a-92df-4dfd-9a83-f84330e28fe0;d38d936a-9c05-4bdc-8331-418fef27f492;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -ae82a1c1-9790-44d1-8dab-52d75c7b79ce;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4286],[11.8213,53.4282]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 6;0.0050064;1403edf9-e47c-4705-8563-83bcd639482e;09285b78-9c18-4af7-9c7a-942cc868016f;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -e642eafb-2b6e-4b78-80b8-1ab15db8cfeb;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4278],[11.8196,53.4282]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 101;0.0153312;7dc43c81-9a61-45a0-9745-800a28bf4a9d;a3ed5af5-1fcb-4fce-af0f-708d3d604124;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -41a4c5e5-7b54-46a8-b6b6-d7b5861a251c;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4195],[11.8198,53.4199]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 96;0.0010841;af2b07ce-1a96-4b50-9e21-badf29eed519;215eaa45-82c3-49c7-a60f-4fa13215de05;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -44821cfc-7670-4f28-8941-70e9345cb069;"{""type"":""LineString"",""coordinates"":[[11.8201,53.4225],[11.8203,53.4221]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 15;0.027677;a7725293-05fc-447f-bc12-38b689b0a956;eae8a04c-44f2-4da3-95f6-cae48f85737c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -d665e8c6-40fb-4608-97fb-99a4cf52560e;"{""type"":""LineString"",""coordinates"":[[11.815,53.4281],[11.815,53.4277]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 110;0.041;d38d936a-9c05-4bdc-8331-418fef27f492;3e6be3ac-2b51-4080-b815-391313612fc7;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -2f37bd36-f356-4d29-81c7-dd9c2fba5e7d;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4234],[11.8166,53.423]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 45;0.00223482;57b40047-4f9d-46bb-bf19-c0a86bbd4f5b;5b73ded9-3ca4-4f18-a2ab-c27b9a3dcf9d;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -bcccf0b5-e0e3-4cc7-82bb-262ebc19415e;"{""type"":""LineString"",""coordinates"":[[11.8169,53.4289],[11.8168,53.4285]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 56;0.039;a882e666-82d1-4ba6-87df-fc702fe06187;c41ebab8-16a0-4a3a-b4af-26073932d462;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -3685f389-6822-4522-a633-74265b67eaee;"{""type"":""LineString"",""coordinates"":[[11.816,53.4227],[11.816,53.423]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 76;0.00185168;3faac527-0ff3-44a7-9e4f-24a41940da90;2575f527-1f4e-45e2-bed2-4a5427f122e0;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -48347702-9e58-49f3-a7de-72024d4b296c;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4199],[11.8198,53.4203]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 2;0.00995493;215eaa45-82c3-49c7-a60f-4fa13215de05;5596da2f-ca32-4ad3-81da-1ffa17cd3d7b;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -ec2a73ea-50ba-4187-b8b0-a5046cf6b632;"{""type"":""LineString"",""coordinates"":[[11.8195,53.425],[11.8196,53.4253]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 83;0.00510961;29516ae3-6676-4797-99c1-1f0a32b989d8;03b2aa45-84f6-48c0-9dab-427e046a5672;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -d1940183-f7bb-42df-b8ff-63ac7aff0b1d;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4211],[11.8198,53.4216]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 9;0.00439896;27b84da5-478e-4a05-8fe7-a9f800db5eff;883edf38-9a18-4f61-981a-691aaf436cc7;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -dd618e3f-9ef9-4e4b-b632-737d79c4d8c3;"{""type"":""LineString"",""coordinates"":[[11.8175,53.427],[11.8174,53.4273]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 62;0.00124645;80b8d1f8-7e83-421d-a95a-c193fc35f4f7;8ace5c2b-584a-4015-990f-6f1e14de4ddb;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -ed80666a-cf86-404a-b3a7-ad49be1cd40c;"{""type"":""LineString"",""coordinates"":[[11.816,53.423],[11.816,53.4234]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 77;0.00232237;2575f527-1f4e-45e2-bed2-4a5427f122e0;5d3bcf55-0520-43ff-8d63-3d0eb421e442;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -444615ed-26f2-45a7-8b5d-213c72e83a4f;"{""type"":""LineString"",""coordinates"":[[11.8201,53.4239],[11.8196,53.4243]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 95;0.0281251;80962bd3-a10f-4ed2-ba6a-3e802189939c;f4da61e4-7600-4cd1-95b6-c70b56c049fc;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -05ebbc17-61bc-4810-ae19-9ee04d7ce8d0;"{""type"":""LineString"",""coordinates"":[[11.8175,53.4285],[11.8169,53.4289]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 54;0.0173762;30b2ce2d-41dd-4d1d-866d-47abd24bfc3e;a882e666-82d1-4ba6-87df-fc702fe06187;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -caf90323-aad2-496e-a0e3-a4ba7c9a481c;"{""type"":""LineString"",""coordinates"":[[11.8174,53.4273],[11.8175,53.4277]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 64;0.00314803;8ace5c2b-584a-4015-990f-6f1e14de4ddb;7f692039-eef6-45f6-9e30-b5983f6750a5;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -60b400db-b048-4bed-be29-8abc18780d10;"{""type"":""LineString"",""coordinates"":[[11.816,53.4206],[11.816,53.421]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 14;0.045296;d6dbb0ae-13c9-438e-93b3-b6c63a0708df;98c14f60-e196-4f12-903b-8485f1eacb16;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -cf13634d-abd2-465d-8839-c95a54af7a80;"{""type"":""LineString"",""coordinates"":[[11.816,53.4214],[11.816,53.4218]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 80;0.0101988;ca0b9e34-ddc8-405e-86a4-ab939626c4a2;43040a39-8b6c-401f-9dfd-82b42aa6dec6;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -30044e09-a0f8-417d-a949-afcfa940f671;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4253],[11.8213,53.425]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 25;0.0124821;a432b8ce-0462-478b-83e7-3107cd2e909c;58b551b6-83bd-4f1c-8d9c-8c9a7f638c0b;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -fe71fe2b-8dd0-4942-b0b5-d241e095b912;"{""type"":""LineString"",""coordinates"":[[11.8195,53.4268],[11.8195,53.4272]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 97;0.0032723;34031e92-3444-47d5-94ae-cceeb5d96bb2;120eaa58-a500-4ae2-a86a-56a40b931ec1;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -66e44163-7091-4f1b-991c-64108e2238f6;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4243],[11.8191,53.4239]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 88;0.0185855;f4da61e4-7600-4cd1-95b6-c70b56c049fc;b5c1e826-63fd-4b0c-bec0-0c758389ef58;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -82bac681-169f-48d8-9842-fd69d3adbfe0;"{""type"":""LineString"",""coordinates"":[[11.816,53.4234],[11.8166,53.4238]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 75;0.0109137;5d3bcf55-0520-43ff-8d63-3d0eb421e442;4129e079-6712-4275-911c-36729d698c42;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -74a5dc42-e689-490f-a611-ae7c3767f01b;"{""type"":""LineString"",""coordinates"":[[11.816,53.4218],[11.816,53.4222]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 81;0.0100396;43040a39-8b6c-401f-9dfd-82b42aa6dec6;62d603c3-f306-40b3-a665-ba9892d226f0;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -0eedb3d0-cedc-4798-b42e-d8e8ef646b82;"{""type"":""LineString"",""coordinates"":[[11.8195,53.4275],[11.8196,53.4278]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 100;0.0129061;9d136a6b-5fdc-44ed-a5ed-599a55281024;7dc43c81-9a61-45a0-9745-800a28bf4a9d;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -b44446fd-2125-4c7b-850e-b0f3d6c8b110;"{""type"":""LineString"",""coordinates"":[[11.8174,53.4281],[11.8175,53.4285]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 61;0.014766;3f6c26dd-842b-4dee-b71f-4aa32e2654ff;30b2ce2d-41dd-4d1d-866d-47abd24bfc3e;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -31c914bc-dd9e-4825-9b4b-b5fc1b971e0d;"{""type"":""LineString"",""coordinates"":[[11.8166,53.423],[11.8166,53.4227]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 46;0.00683612;5b73ded9-3ca4-4f18-a2ab-c27b9a3dcf9d;ad30322c-0c99-4669-8e4b-25265087a66d;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e -e7ac7023-f4e3-460d-8d02-4c9c444d2e18;"{""type"":""LineString"",""coordinates"":[[11.8191,53.422],[11.8191,53.4216]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 63;0.0149947;c6dac3ab-f44f-4b87-800c-0f4da64673f1;8f53645e-1f28-4eb1-807f-eb2a473f9d4c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e diff --git a/input/samples/simopsimtestgrid_reduced_reference/fullGrid/line_type_input.csv b/input/samples/simopsimtestgrid_reduced_reference/fullGrid/line_type_input.csv deleted file mode 100644 index d7786c38cd..0000000000 --- a/input/samples/simopsimtestgrid_reduced_reference/fullGrid/line_type_input.csv +++ /dev/null @@ -1,2 +0,0 @@ -uuid;b;g;i_max;id;r;v_rated;x -9a8e9b63-af21-4c1b-8db7-fc2924f9610e;273.31899999999996;0.0;357.0;NAYY 4x240SE 0.6/1kV;0.1267;0.4;0.0797965 diff --git a/input/samples/simopsimtestgrid_reduced_reference/fullGrid/load_input.csv b/input/samples/simopsimtestgrid_reduced_reference/fullGrid/load_input.csv deleted file mode 100644 index 305aa1c8d2..0000000000 --- a/input/samples/simopsimtestgrid_reduced_reference/fullGrid/load_input.csv +++ /dev/null @@ -1,4 +0,0 @@ -uuid;cos_phi_rated;dsm;e_cons_annual;id;load_profile;node;operates_from;operates_until;operator;q_characteristics;s_rated;em -c3434742-e4f0-49e5-baa7-c1e3045c732c;0.93;false;0.0;LV5.201 Load 64;h0;dc54bd8a-b7d8-4e99-adb0-d6ee5084241c;;;;"cosPhiFixed:{(0.0,0.93)}";30.2;f9dc7ce6-658c-4101-a12f-d58bb889286b -fd2e19b6-d5e3-4776-9456-8787a2160d9d;0.93;false;0.0;LV5.201 Load 74;h0;43040a39-8b6c-401f-9dfd-82b42aa6dec6;;;;"cosPhiFixed:{(0.0,0.93)}";11.0;c3a7e9f5-b492-4c85-af2d-1e93f6a25443 -98c1a2ab-bd09-4c77-a389-d088aed894b1;0.93;false;0.0;LV5.201 Load 102;h0;3e6be3ac-2b51-4080-b815-391313612fc7;;;;"cosPhiFixed:{(0.0,0.93)}";50.3;957938b7-0476-4fab-a1b3-6ce8615857b3 diff --git a/input/samples/simopsimtestgrid_reduced_reference/fullGrid/node_input.csv b/input/samples/simopsimtestgrid_reduced_reference/fullGrid/node_input.csv deleted file mode 100644 index ab9f9335ca..0000000000 --- a/input/samples/simopsimtestgrid_reduced_reference/fullGrid/node_input.csv +++ /dev/null @@ -1,112 +0,0 @@ -uuid;geo_position;id;operates_from;operates_until;operator;slack;subnet;v_rated;v_target;volt_lvl -16091b6d-f1ea-4a07-9ad4-30d595aba68d;"{""type"":""Point"",""coordinates"":[11.8213,53.429],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 26;;;;false;2;0.4;1.0;Niederspannung -27b84da5-478e-4a05-8fe7-a9f800db5eff;"{""type"":""Point"",""coordinates"":[11.8198,53.4211],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 7;;;;false;2;0.4;1.0;Niederspannung -7f692039-eef6-45f6-9e30-b5983f6750a5;"{""type"":""Point"",""coordinates"":[11.8175,53.4277],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 54;;;;false;2;0.4;1.0;Niederspannung -20ad9754-e966-4ad1-9541-f968c207f3df;"{""type"":""Point"",""coordinates"":[11.8196,53.4293],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 84;;;;false;2;0.4;1.0;Niederspannung -98c14f60-e196-4f12-903b-8485f1eacb16;"{""type"":""Point"",""coordinates"":[11.816,53.421],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 29;;;;false;2;0.4;1.0;Niederspannung -09285b78-9c18-4af7-9c7a-942cc868016f;"{""type"":""Point"",""coordinates"":[11.8213,53.4282],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 58;;;;false;2;0.4;1.0;Niederspannung -0d6ef8f6-0ba1-4fa7-8e63-e55cee12b165;"{""type"":""Point"",""coordinates"":[11.8162,53.4273],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 103;;;;false;2;0.4;1.0;Niederspannung -43040a39-8b6c-401f-9dfd-82b42aa6dec6;"{""type"":""Point"",""coordinates"":[11.816,53.4218],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 25;;;;false;2;0.4;1.0;Niederspannung -dc54bd8a-b7d8-4e99-adb0-d6ee5084241c;"{""type"":""Point"",""coordinates"":[11.8213,53.4267],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 81;;;;false;2;0.4;1.0;Niederspannung -2575f527-1f4e-45e2-bed2-4a5427f122e0;"{""type"":""Point"",""coordinates"":[11.816,53.423],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 3;;;;false;2;0.4;1.0;Niederspannung -30b2ce2d-41dd-4d1d-866d-47abd24bfc3e;"{""type"":""Point"",""coordinates"":[11.8175,53.4285],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 34;;;;false;2;0.4;1.0;Niederspannung -589bf32a-b361-4536-ae96-6d56d184eedb;"{""type"":""Point"",""coordinates"":[11.8213,53.4222],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 101;;;;false;2;0.4;1.0;Niederspannung -555cd075-0fe4-4a65-b027-f45cffa960d9;"{""type"":""Point"",""coordinates"":[11.8162,53.4281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 31;;;;false;2;0.4;1.0;Niederspannung -62d603c3-f306-40b3-a665-ba9892d226f0;"{""type"":""Point"",""coordinates"":[11.816,53.4222],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 55;;;;false;2;0.4;1.0;Niederspannung -215eaa45-82c3-49c7-a60f-4fa13215de05;"{""type"":""Point"",""coordinates"":[11.8198,53.4199],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 24;;;;false;2;0.4;1.0;Niederspannung -f4da61e4-7600-4cd1-95b6-c70b56c049fc;"{""type"":""Point"",""coordinates"":[11.8196,53.4243],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 76;;;;false;2;0.4;1.0;Niederspannung -73e7a7e8-2154-46ea-9727-a4916af3570c;"{""type"":""Point"",""coordinates"":[11.8175,53.4253],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 18;;;;false;2;0.4;1.0;Niederspannung -85751c9b-3e5e-468a-9dc7-43775b0d4a6f;"{""type"":""Point"",""coordinates"":[11.817,53.4293],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 51;;;;false;2;0.4;1.0;Niederspannung -093160c4-6482-4c58-b952-217c615e3ada;"{""type"":""Point"",""coordinates"":[11.8213,53.426],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 68;;;;false;2;0.4;1.0;Niederspannung -79e19265-08e8-407f-ae95-2f78e344d3a4;"{""type"":""Point"",""coordinates"":[11.8196,53.4257],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 28;;;;false;2;0.4;1.0;Niederspannung -a966644c-37d5-4400-9fa0-725fd88586a8;"{""type"":""Point"",""coordinates"":[11.8213,53.4232],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 59;;;;false;2;0.4;1.0;Niederspannung -952c6b04-0d02-4ea2-a6cc-bb44fbbe4e52;"{""type"":""Point"",""coordinates"":[11.8198,53.4188],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 94;;;;false;2;0.4;1.0;Niederspannung -c6c20ab1-16fb-4f82-8017-273022da8bb0;"{""type"":""Point"",""coordinates"":[11.8182,53.4254],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 62;;;;false;2;0.4;1.0;Niederspannung -e68a088d-cf1a-40b7-9b1a-e0933352f4e6;"{""type"":""Point"",""coordinates"":[11.8173,53.4234],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 67;;;;false;2;0.4;1.0;Niederspannung -8f53645e-1f28-4eb1-807f-eb2a473f9d4c;"{""type"":""Point"",""coordinates"":[11.8191,53.4216],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 86;;;;false;2;0.4;1.0;Niederspannung -0d6c3f2b-5296-4ec1-995c-b150e72f035f;"{""type"":""Point"",""coordinates"":[11.8175,53.4248],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 80;;;;false;2;0.4;1.0;Niederspannung -ea4a6507-e504-4542-be5f-1019719b2257;"{""type"":""Point"",""coordinates"":[11.8184,53.4293],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 111;;;;false;2;0.4;1.0;Niederspannung -9644f198-e801-4545-87ee-a24e2a8039bd;"{""type"":""Point"",""coordinates"":[11.8221,53.429],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 39;;;;false;2;0.4;1.0;Niederspannung -a9288e77-2919-4db6-89eb-9737bd07f111;"{""type"":""Point"",""coordinates"":[11.8214,53.4246],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 63;;;;false;2;0.4;1.0;Niederspannung -d6dbb0ae-13c9-438e-93b3-b6c63a0708df;"{""type"":""Point"",""coordinates"":[11.816,53.4206],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 89;;;;false;2;0.4;1.0;Niederspannung -5596da2f-ca32-4ad3-81da-1ffa17cd3d7b;"{""type"":""Point"",""coordinates"":[11.8198,53.4203],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 102;;;;false;2;0.4;1.0;Niederspannung -ad30322c-0c99-4669-8e4b-25265087a66d;"{""type"":""Point"",""coordinates"":[11.8166,53.4227],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 65;;;;false;2;0.4;1.0;Niederspannung -120eaa58-a500-4ae2-a86a-56a40b931ec1;"{""type"":""Point"",""coordinates"":[11.8195,53.4272],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 57;;;;false;2;0.4;1.0;Niederspannung -29516ae3-6676-4797-99c1-1f0a32b989d8;"{""type"":""Point"",""coordinates"":[11.8195,53.425],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 52;;;;false;2;0.4;1.0;Niederspannung -24b63115-12eb-4e77-b9ef-ca474fed960f;"{""type"":""Point"",""coordinates"":[11.8201,53.4232],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 46;;;;false;2;0.4;1.0;Niederspannung -a7725293-05fc-447f-bc12-38b689b0a956;"{""type"":""Point"",""coordinates"":[11.8201,53.4225],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 72;;;;false;2;0.4;1.0;Niederspannung -3b86661a-187d-4aa6-bf37-2014789afc08;"{""type"":""Point"",""coordinates"":[11.8214,53.4218],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 20;;;;false;2;0.4;1.0;Niederspannung -a3ed5af5-1fcb-4fce-af0f-708d3d604124;"{""type"":""Point"",""coordinates"":[11.8196,53.4282],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 33;;;;false;2;0.4;1.0;Niederspannung -ec8f2c82-a1b2-487c-b573-250859e3b414;"{""type"":""Point"",""coordinates"":[11.8166,53.4241],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";MV2.101 Bus 5;;;;true;1;20.0;1.025;Mittelspannung -5f153bf6-4f25-41f1-8545-18fe6323bc49;"{""type"":""Point"",""coordinates"":[11.8175,53.4257],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 17;;;;false;2;0.4;1.0;Niederspannung -b565ae3b-68f9-4bca-816d-9b0fc1c6b13f;"{""type"":""Point"",""coordinates"":[11.8198,53.4221],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 70;;;;false;2;0.4;1.0;Niederspannung -8e809cf6-7e05-437c-89a6-f6ca135a403b;"{""type"":""Point"",""coordinates"":[11.815,53.4289],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 47;;;;false;2;0.4;1.0;Niederspannung -ca0b9e34-ddc8-405e-86a4-ab939626c4a2;"{""type"":""Point"",""coordinates"":[11.816,53.4214],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 99;;;;false;2;0.4;1.0;Niederspannung -3e6be3ac-2b51-4080-b815-391313612fc7;"{""type"":""Point"",""coordinates"":[11.815,53.4277],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 110;;;;false;2;0.4;1.0;Niederspannung -4749ab2b-4d96-4100-8081-73e77c797d6b;"{""type"":""Point"",""coordinates"":[11.8166,53.4241],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 73;;;;false;2;0.4;1.0;Niederspannung -3f6c26dd-842b-4dee-b71f-4aa32e2654ff;"{""type"":""Point"",""coordinates"":[11.8174,53.4281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 97;;;;false;2;0.4;1.0;Niederspannung -c6dac3ab-f44f-4b87-800c-0f4da64673f1;"{""type"":""Point"",""coordinates"":[11.8191,53.422],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 10;;;;false;2;0.4;1.0;Niederspannung -eae8a04c-44f2-4da3-95f6-cae48f85737c;"{""type"":""Point"",""coordinates"":[11.8203,53.4221],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 45;;;;false;2;0.4;1.0;Niederspannung -94fe96b1-f36a-4edd-a107-4ff0376f1066;"{""type"":""Point"",""coordinates"":[11.8175,53.4265],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 61;;;;false;2;0.4;1.0;Niederspannung -31e6e197-719d-4aaf-8ca5-ab9e7549390e;"{""type"":""Point"",""coordinates"":[11.8196,53.4286],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 32;;;;false;2;0.4;1.0;Niederspannung -002a4495-96e4-49c9-abbe-8fccb3e9c83e;"{""type"":""Point"",""coordinates"":[11.8214,53.4229],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 90;;;;false;2;0.4;1.0;Niederspannung -c7e48384-5699-4a38-a887-7e15a9145202;"{""type"":""Point"",""coordinates"":[11.8191,53.4223],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 107;;;;false;2;0.4;1.0;Niederspannung -8ace5c2b-584a-4015-990f-6f1e14de4ddb;"{""type"":""Point"",""coordinates"":[11.8174,53.4273],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 27;;;;false;2;0.4;1.0;Niederspannung -c41ebab8-16a0-4a3a-b4af-26073932d462;"{""type"":""Point"",""coordinates"":[11.8168,53.4285],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 35;;;;false;2;0.4;1.0;Niederspannung -ab8c8f6c-e7a2-4b81-a0d7-5f13789267a2;"{""type"":""Point"",""coordinates"":[11.815,53.4269],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 113;;;;false;2;0.4;1.0;Niederspannung -9f1baf4e-12e1-41d1-8efc-81cfc78f1957;"{""type"":""Point"",""coordinates"":[11.8195,53.426],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 71;;;;false;2;0.4;1.0;Niederspannung -02e63e81-2e62-4ef6-8fdb-0b0905a437b6;"{""type"":""Point"",""coordinates"":[11.8191,53.4212],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 12;;;;false;2;0.4;1.0;Niederspannung -2d33314e-31db-4ad4-a898-2be5d56a896d;"{""type"":""Point"",""coordinates"":[11.8162,53.4277],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 48;;;;false;2;0.4;1.0;Niederspannung -b8e95bf0-3ba8-4d53-a0bf-a3720fb785fb;"{""type"":""Point"",""coordinates"":[11.8195,53.4264],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 49;;;;false;2;0.4;1.0;Niederspannung -d0bfabdb-0e83-423b-a20a-ab9197c4284e;"{""type"":""Point"",""coordinates"":[11.8213,53.4239],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 82;;;;false;2;0.4;1.0;Niederspannung -6b2881ce-3965-4f5e-98a3-74eb47b0a7ca;"{""type"":""Point"",""coordinates"":[11.8162,53.4284],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 53;;;;false;2;0.4;1.0;Niederspannung -af2b07ce-1a96-4b50-9e21-badf29eed519;"{""type"":""Point"",""coordinates"":[11.8198,53.4195],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 100;;;;false;2;0.4;1.0;Niederspannung -844c0b9c-058a-4228-a8c4-bf2defff6958;"{""type"":""Point"",""coordinates"":[11.8213,53.4293],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 1;;;;false;2;0.4;1.0;Niederspannung -a432b8ce-0462-478b-83e7-3107cd2e909c;"{""type"":""Point"",""coordinates"":[11.8213,53.4253],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 56;;;;false;2;0.4;1.0;Niederspannung -4129e079-6712-4275-911c-36729d698c42;"{""type"":""Point"",""coordinates"":[11.8166,53.4238],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 77;;;;false;2;0.4;1.0;Niederspannung -883edf38-9a18-4f61-981a-691aaf436cc7;"{""type"":""Point"",""coordinates"":[11.8198,53.4216],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 4;;;;false;2;0.4;1.0;Niederspannung -f2724954-34d3-4ddf-b6b0-7a1531639990;"{""type"":""Point"",""coordinates"":[11.8191,53.4235],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 6;;;;false;2;0.4;1.0;Niederspannung -5d3bcf55-0520-43ff-8d63-3d0eb421e442;"{""type"":""Point"",""coordinates"":[11.816,53.4234],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 91;;;;false;2;0.4;1.0;Niederspannung -3a557b4e-06b8-4f29-929f-81d95c42c897;"{""type"":""Point"",""coordinates"":[11.8213,53.4277],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 42;;;;false;2;0.4;1.0;Niederspannung -7dc43c81-9a61-45a0-9745-800a28bf4a9d;"{""type"":""Point"",""coordinates"":[11.8196,53.4278],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 50;;;;false;2;0.4;1.0;Niederspannung -fd4bebb8-40ca-4eed-92c0-cdd10b86ac20;"{""type"":""Point"",""coordinates"":[11.8191,53.4209],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 60;;;;false;2;0.4;1.0;Niederspannung -3464496c-7dd4-41e9-ae0a-99ade0b51572;"{""type"":""Point"",""coordinates"":[11.8213,53.4236],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 40;;;;false;2;0.4;1.0;Niederspannung -d38d936a-9c05-4bdc-8331-418fef27f492;"{""type"":""Point"",""coordinates"":[11.815,53.4281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 109;;;;false;2;0.4;1.0;Niederspannung -58b551b6-83bd-4f1c-8d9c-8c9a7f638c0b;"{""type"":""Point"",""coordinates"":[11.8213,53.425],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 38;;;;false;2;0.4;1.0;Niederspannung -b5c1e826-63fd-4b0c-bec0-0c758389ef58;"{""type"":""Point"",""coordinates"":[11.8191,53.4239],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 104;;;;false;2;0.4;1.0;Niederspannung -7c35a794-f569-4a9c-acb0-d03647610086;"{""type"":""Point"",""coordinates"":[11.8166,53.4249],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 98;;;;false;2;0.4;1.0;Niederspannung -78815cf6-70db-432c-96e6-87fe8cf67eee;"{""type"":""Point"",""coordinates"":[11.815,53.4272],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 112;;;;false;2;0.4;1.0;Niederspannung -c5b6bfaf-1621-40a7-9c53-02cfb59c04d9;"{""type"":""Point"",""coordinates"":[11.822,53.4294],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 14;;;;false;2;0.4;1.0;Niederspannung -2b3d7fb8-0583-4d47-97b1-3b5f232fd462;"{""type"":""Point"",""coordinates"":[11.8213,53.4274],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 87;;;;false;2;0.4;1.0;Niederspannung -3fcb94e3-7781-4d83-9030-d9853822e78e;"{""type"":""Point"",""coordinates"":[11.8213,53.4243],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 85;;;;false;2;0.4;1.0;Niederspannung -f713593a-3fd3-410a-ac08-74202d4f5798;"{""type"":""Point"",""coordinates"":[11.8201,53.4236],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 79;;;;false;2;0.4;1.0;Niederspannung -47246a84-ad0e-4d04-9d98-1c9cd5d363c1;"{""type"":""Point"",""coordinates"":[11.8196,53.4289],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 16;;;;false;2;0.4;1.0;Niederspannung -9cdb3115-cc00-4d61-bc33-442e8f30fb63;"{""type"":""Point"",""coordinates"":[11.8213,53.4257],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 83;;;;false;2;0.4;1.0;Niederspannung -839ff0f4-93db-42ec-a928-bbc448b6cf5c;"{""type"":""Point"",""coordinates"":[11.8197,53.4207],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 69;;;;false;2;0.4;1.0;Niederspannung -e7908208-77b4-4059-806e-4857262992fc;"{""type"":""Point"",""coordinates"":[11.8196,53.4247],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 105;;;;false;2;0.4;1.0;Niederspannung -83da8d60-405a-45f7-9bb9-9d35607b7927;"{""type"":""Point"",""coordinates"":[11.8201,53.4229],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 2;;;;false;2;0.4;1.0;Niederspannung -9d136a6b-5fdc-44ed-a5ed-599a55281024;"{""type"":""Point"",""coordinates"":[11.8195,53.4275],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 9;;;;false;2;0.4;1.0;Niederspannung -3faac527-0ff3-44a7-9e4f-24a41940da90;"{""type"":""Point"",""coordinates"":[11.816,53.4227],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 15;;;;false;2;0.4;1.0;Niederspannung -5682ac05-7336-4ebc-a5d1-3b69c79fb3b1;"{""type"":""Point"",""coordinates"":[11.8166,53.4241],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 44;;;;false;2;0.4;1.0;Niederspannung -5b73ded9-3ca4-4f18-a2ab-c27b9a3dcf9d;"{""type"":""Point"",""coordinates"":[11.8166,53.423],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 36;;;;false;2;0.4;1.0;Niederspannung -594d101c-3a05-45e3-a061-9189f3e848b7;"{""type"":""Point"",""coordinates"":[11.8198,53.4192],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 88;;;;false;2;0.4;1.0;Niederspannung -7b81b518-00e0-4ff1-b4cf-876903958d7a;"{""type"":""Point"",""coordinates"":[11.8213,53.4271],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 8;;;;false;2;0.4;1.0;Niederspannung -a882e666-82d1-4ba6-87df-fc702fe06187;"{""type"":""Point"",""coordinates"":[11.8169,53.4289],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 74;;;;false;2;0.4;1.0;Niederspannung -1403edf9-e47c-4705-8563-83bcd639482e;"{""type"":""Point"",""coordinates"":[11.8213,53.4286],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 75;;;;false;2;0.4;1.0;Niederspannung -03b2aa45-84f6-48c0-9dab-427e046a5672;"{""type"":""Point"",""coordinates"":[11.8196,53.4253],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 22;;;;false;2;0.4;1.0;Niederspannung -6ee7ea93-ea9c-40cb-b79a-1c5f287c97a5;"{""type"":""Point"",""coordinates"":[11.8221,53.4286],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 66;;;;false;2;0.4;1.0;Niederspannung -88cf719a-92df-4dfd-9a83-f84330e28fe0;"{""type"":""Point"",""coordinates"":[11.815,53.4285],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 108;;;;false;2;0.4;1.0;Niederspannung -80962bd3-a10f-4ed2-ba6a-3e802189939c;"{""type"":""Point"",""coordinates"":[11.8201,53.4239],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 21;;;;false;2;0.4;1.0;Niederspannung -80b8d1f8-7e83-421d-a95a-c193fc35f4f7;"{""type"":""Point"",""coordinates"":[11.8175,53.427],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 95;;;;false;2;0.4;1.0;Niederspannung -b909fb45-b6ee-427f-afd7-e8a0ec7274c6;"{""type"":""Point"",""coordinates"":[11.8213,53.4225],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 92;;;;false;2;0.4;1.0;Niederspannung -57b40047-4f9d-46bb-bf19-c0a86bbd4f5b;"{""type"":""Point"",""coordinates"":[11.8166,53.4234],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 37;;;;false;2;0.4;1.0;Niederspannung -18b4157b-0e47-4c5a-adb8-ccae47372336;"{""type"":""Point"",""coordinates"":[11.8213,53.4264],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 5;;;;false;2;0.4;1.0;Niederspannung -8726dc29-621e-4455-a541-cd88d7da457f;"{""type"":""Point"",""coordinates"":[11.8166,53.4257],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 96;;;;false;2;0.4;1.0;Niederspannung -814f784b-687f-4dd5-8a91-c7772c916d46;"{""type"":""Point"",""coordinates"":[11.8174,53.4262],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 19;;;;false;2;0.4;1.0;Niederspannung -7efabb8d-ba17-4487-96d9-5744b1fedf8a;"{""type"":""Point"",""coordinates"":[11.8166,53.4245],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 23;;;;false;2;0.4;1.0;Niederspannung -15a86f7d-fb73-49a4-af6a-25b14122378d;"{""type"":""Point"",""coordinates"":[11.8191,53.4227],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 11;;;;false;2;0.4;1.0;Niederspannung -34031e92-3444-47d5-94ae-cceeb5d96bb2;"{""type"":""Point"",""coordinates"":[11.8195,53.4268],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 106;;;;false;2;0.4;1.0;Niederspannung -35748e60-3be8-4930-8a61-209fd5df1bec;"{""type"":""Point"",""coordinates"":[11.815,53.4293],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 30;;;;false;2;0.4;1.0;Niederspannung -50cac08e-bf24-4526-9466-53ca5edccd15;"{""type"":""Point"",""coordinates"":[11.8166,53.4253],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 64;;;;false;2;0.4;1.0;Niederspannung -2efac9b1-fb0d-4e08-bfac-501798826deb;"{""type"":""Point"",""coordinates"":[11.8191,53.4231],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 93;;;;false;2;0.4;1.0;Niederspannung -c86d6361-4159-4787-b5f4-e41dcaa95195;"{""type"":""Point"",""coordinates"":[11.8182,53.4257],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 13;;;;false;2;0.4;1.0;Niederspannung diff --git a/input/samples/simopsimtestgrid_reduced_reference/fullGrid/pv_input.csv b/input/samples/simopsimtestgrid_reduced_reference/fullGrid/pv_input.csv deleted file mode 100644 index cb4eeaa67e..0000000000 --- a/input/samples/simopsimtestgrid_reduced_reference/fullGrid/pv_input.csv +++ /dev/null @@ -1,4 +0,0 @@ -uuid;albedo;azimuth;cos_phi_rated;elevation_angle;eta_conv;id;k_g;k_t;market_reaction;node;operates_from;operates_until;operator;q_characteristics;s_rated;em -a1eb7fc1-3bee-4b65-a387-ef3046644bf0;0.20000000298023224;4.093344211578369;0.8999999761581421;37.69556427001953;97.0;LV5.201 PV 10;0.8999999761581421;1.0;false;dc54bd8a-b7d8-4e99-adb0-d6ee5084241c;;;;"cosPhiFixed:{(0.0,0.9)}";40.2;f9dc7ce6-658c-4101-a12f-d58bb889286b -de8cfef5-7620-4b9e-9a10-1faebb5a80c0;0.20000000298023224;4.093344211578369;0.8999999761581421;37.69556427001953;97.0;LV5.201 PV 11;0.8999999761581421;1.0;false;3e6be3ac-2b51-4080-b815-391313612fc7;;;;"cosPhiFixed:{(0.0,0.9)}";100.9;957938b7-0476-4fab-a1b3-6ce8615857b3 -2560c371-f420-4c2a-b4e6-e04c11b64c03;0.20000000298023224;0.7802008390426636;0.8999999761581421;40.086585998535156;98.0;LV5.201 PV 15;0.8999999761581421;1.0;false;43040a39-8b6c-401f-9dfd-82b42aa6dec6;;;;"cosPhiFixed:{(0.0,0.9)}";29.0;c3a7e9f5-b492-4c85-af2d-1e93f6a25443 diff --git a/input/samples/simopsimtestgrid_reduced_reference/fullGrid/transformer_2_w_input.csv b/input/samples/simopsimtestgrid_reduced_reference/fullGrid/transformer_2_w_input.csv deleted file mode 100644 index d1f9f5d13c..0000000000 --- a/input/samples/simopsimtestgrid_reduced_reference/fullGrid/transformer_2_w_input.csv +++ /dev/null @@ -1,2 +0,0 @@ -uuid;auto_tap;id;node_a;node_b;operates_from;operates_until;operator;parallel_devices;tap_pos;type -adaba416-9b52-45df-9d91-f67d0dd28ecb;false;MV2.101-LV5.201-Trafo 1;ec8f2c82-a1b2-487c-b573-250859e3b414;4749ab2b-4d96-4100-8081-73e77c797d6b;;;;1;0;417407d2-1e74-4f37-9b64-f701f53f8842 diff --git a/input/samples/simopsimtestgrid_reduced_reference/fullGrid/transformer_2_w_type_input.csv b/input/samples/simopsimtestgrid_reduced_reference/fullGrid/transformer_2_w_type_input.csv deleted file mode 100644 index b13848b159..0000000000 --- a/input/samples/simopsimtestgrid_reduced_reference/fullGrid/transformer_2_w_type_input.csv +++ /dev/null @@ -1,2 +0,0 @@ -uuid;b_m;d_phi;d_v;g_m;id;r_sc;s_rated;tap_max;tap_min;tap_neutr;tap_side;v_rated_a;v_rated_b;x_sc -417407d2-1e74-4f37-9b64-f701f53f8842;-36.47380569074435;0.0;2.5;4124.999999999999;0.63 MVA 20/0.4 kV Dyn5 ASEA;6.953892668178382;630.0;2;-2;0;false;20.0;0.4;37.45518044666632 diff --git a/input/samples/simopsimtestgrid_reduced_reference/simopsimtestgrid.conf b/input/samples/simopsimtestgrid_reduced_reference/simopsimtestgrid.conf deleted file mode 100644 index 44fbc2f8e4..0000000000 --- a/input/samples/simopsimtestgrid_reduced_reference/simopsimtestgrid.conf +++ /dev/null @@ -1,240 +0,0 @@ -include "../common/pekko.conf" - -######### -# ATTENTION: Do not change this config file directly but use it as a base for your personal delta config for the -# vn_simona scenario! Delta configs can be created by including the config you want to change -# parameters from via include (e.g. include "input/samples/vn_simona/vn_simona.conf") at the -# beginning of your config file and then just override the parameters you want to change! -######### - -################################################################## -# Simulation Parameters -################################################################## -simona.simulationName = "simopsimtest" - -################################################################## -# Time Parameters -################################################################## -simona.time.startDateTime = "2024-02-27T00:00:00Z" -simona.time.endDateTime = "2024-02-28T00:15:00Z" -simona.time.schedulerReadyCheckWindow = 900 - -################################################################## -# Input Parameters -################################################################## -simona.input.primary.csvParams = { - directoryPath: "simona/input/samples/simopsimtestgrid_reduced_reference/fullGrid" - csvSep: ";" - isHierarchic: false -} -simona.input.grid.datasource.id = "csv" -simona.input.grid.datasource.csvParams = { - directoryPath: "simona/input/samples/simopsimtestgrid_reduced_reference/fullGrid" - csvSep: ";" - isHierarchic: false -} - -simona.input.weather.datasource = { - scheme = "icon" - sampleParams.use = true - coordinateSource.sampleParams.use = true - maxCoordinateDistance = 50000 -} - -################################################################## -# Output Parameters -################################################################## -simona.output.base.dir = "simona/output/simopsimtestgrid_reduced_reference" -simona.output.base.addTimestampToOutputDir = true - -simona.output.sink.csv { - fileFormat = ".csv" - filePrefix = "" - fileSuffix = "" -} - -simona.output.grid = { - notifier = "grid" - nodes = false - lines = false - switches = false - transformers2w = false - transformers3w = false -} -simona.output.participant.defaultConfig = { - notifier = "default" - powerRequestReply = false - simulationResult = true -} -simona.output.participant.individualConfigs = [ - { - notifier = "pv" - powerRequestReply = false - simulationResult = true - }, - { - notifier = "wec" - powerRequestReply = false - simulationResult = false - }, - { - notifier = "evcs" - powerRequestReply = false - simulationResult = false - }, - { - notifier = "bm" - powerRequestReply = false - simulationResult = false - }, - { - notifier = "chp" - powerRequestReply = false - simulationResult = false - }, - { - notifier = "ev" - powerRequestReply = false - simulationResult = false - }, - { - notifier = "hp" - powerRequestReply = false - simulationResult = false - }, - { - notifier = "storage" - powerRequestReply = false - simulationResult = false - }, - { - notifier = "fixedFeedIn" - powerRequestReply = false - simulationResult = false - } -] -simona.output.thermal = { - defaultConfig = { - notifier = "default", - simulationResult = false - } - individualConfigs = [ - { - notifier = "house", - simulationResult = false - } - ] -} - -################################################################## -# Runtime Configuration // todo refactor as this naming is misleading and partly unneeded -################################################################## -simona.runtime.selected_subgrids = [] -simona.runtime.selected_volt_lvls = [] - -simona.runtime.participant.load = { - defaultConfig = { - calculateMissingReactivePowerWithModel = false - uuids = ["default"] - scaling = 1.0 - modelBehaviour = "profile" - reference = "power" - } - individualConfigs = [] -} - -simona.runtime.participant.fixedFeedIn = { - defaultConfig = { - calculateMissingReactivePowerWithModel = false - uuids = ["default"] - scaling = 1.0 - } - individualConfigs = [] -} - -simona.runtime.participant.pv = { - defaultConfig = { - calculateMissingReactivePowerWithModel = false - uuids = ["default"] - scaling = 1.0 - } - individualConfigs = [] -} - -simona.runtime.participant.wec = { - defaultConfig = { - calculateMissingReactivePowerWithModel = false - uuids = ["default"] - scaling = 1.0 - } - individualConfigs = [] -} - -simona.runtime.participant.evcs = { - defaultConfig = { - calculateMissingReactivePowerWithModel = false - uuids = ["default"] - scaling = 1.0 - } - individualConfigs = [] -} - -simona.runtime.participant.hp = { - defaultConfig = { - calculateMissingReactivePowerWithModel = false - uuids = ["default"] - scaling = 1.0 - } - individualConfigs = [] -} - -# # # # # -# ATTENTION: calculateMissingReactivePowerWithModel and scaling is ignored here. -# # # # # -simona.runtime.participant.em = { - defaultConfig = { - calculateMissingReactivePowerWithModel = false - uuids = ["default"] - scaling = 1.0 - } - individualConfigs = [] -} - -################################################################## -# Event Configuration -################################################################## -simona.event.listener = [] - -################################################################## -# Grid Configuration -################################################################## - -simona.gridConfig.refSystems = [ - {sNom = "100 kVA", vNom = "0.4 kV", voltLvls = [{id = "NS", vNom = "0.4 kV"}]}, - {sNom = "60 MVA", vNom = "20 kV", voltLvls = [{id = "MS", vNom = "20 kV"}]}, - {sNom = "600 MVA", vNom = "110 kV", voltLvls = [{id = "HS", vNom = "110 kV"}]}, - {sNom = "1000 MVA", vNom = "380 kV", voltLvls = [{id = "HoeS", vNom = "380 kV"}]} -] - -################################################################## -# Power Flow Configuration -################################################################## -simona.powerflow.maxSweepPowerDeviation = 1E-5 // the maximum allowed deviation in power between two sweeps, before overall convergence is assumed -simona.powerflow.newtonraphson.epsilon = [1E-12] -simona.powerflow.newtonraphson.iterations = 50 -simona.powerflow.resolution = "86400s" -simona.powerflow.stopOnFailure = true - -simona.control.transformer = [ - { - transformers = ["31a2b9bf-e785-4475-aa44-1c34646e8c79"], - measurements = ["923f2d69-3093-4198-86e4-13d2d1c220f8"], - vMin = 0.98, - vMax = 1.02 - }, { - transformers = ["1132dbf4-e8a1-44ae-8415-f42d4497aa1d"], - measurements = ["7686b818-a0ba-465c-8e4e-f7d3c4e171fc"], - vMin = 0.98, - vMax = 1.02 - } -] diff --git a/input/samples/vn_simona_ns/fullGrid/line_graphic_input.csv b/input/samples/vn_simona_ns/fullGrid/line_graphic_input.csv deleted file mode 100644 index 495bcf8dd6..0000000000 --- a/input/samples/vn_simona_ns/fullGrid/line_graphic_input.csv +++ /dev/null @@ -1,16 +0,0 @@ -uuid,graphic_layer,line,path -8ae601e8-3dcf-497e-a258-d8ac976e95e8,Standard,106f03b3-b84e-4d36-ab09-13b3dc3403e7,"{""type"":""LineString"",""coordinates"":[[0.20338983,0.1557377],[0.25423729,0.1557377],[0.22881291,0.1557377]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}" -bd830d87-73b0-44c8-9250-26afd33ac4f5,Standard,10c6a103-3132-4324-8f76-8c1858d51fcb,"{""type"":""LineString"",""coordinates"":[[0.20338983,0.24590164],[0.17514124,0.24590164],[0.14689266,0.24590164]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}" -26138708-ff02-48c2-a465-a8b5e5d165a8,Standard,25a1e96e-b425-48bf-a3ae-dc54ea9da47f,"{""type"":""LineString"",""coordinates"":[[0.03954802,0.27868852],[0.03954802,0.17213115],[0.03954802,0.06557377],[0.09039548,0.06557377]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}" -d923185a-03a0-484b-a66d-08f4757aadf7,Standard,2d552f15-6871-4e09-84b1-003da098d909,"{""type"":""LineString"",""coordinates"":[[0.09039548,0.1557377],[0.11864407,0.1557377],[0.14689266,0.1557377]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}" -eb5e4437-0f18-43c1-9e51-51bcfc038bea,Standard,37c8ebe1-5d29-4806-a84a-fc00e4279fa6,"{""type"":""LineString"",""coordinates"":[[0.22881291,0.06557377],[0.25423729,0.06557377],[0.20338983,0.06557377]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}" -75e06780-090a-411f-a47c-47c5a90a2e09,Standard,4bdb6569-72da-4d60-97b1-b5a639032d5d,"{""type"":""LineString"",""coordinates"":[[0.25423729,0.24590164],[0.28248588,0.24590164],[0.31073446,0.24590164]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}" -723fb8fc-90ff-44d5-9c21-88a9f194e83a,Standard,5d1176ac-e64e-4811-8d0c-c87d9060ba2a,"{""type"":""LineString"",""coordinates"":[[0.09039548,0.1557377],[0.05084746,0.27868852],[0.05084746,0.21721405],[0.05084746,0.1557377]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}" -90ad029a-358f-4f05-8a01-62820461dbb2,Standard,814dec01-f6f2-4817-8f83-2e758011b033,"{""type"":""LineString"",""coordinates"":[[0.09039548,0.24590164],[0.06214689,0.27868852],[0.06214689,0.24590164],[0.06214689,0.26229508]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}" -79edb34f-6eea-4190-a0d4-2664ace2deb8,Standard,8bb29769-404b-4bf6-837e-f9ca1b389bf9,"{""type"":""LineString"",""coordinates"":[[0.17514124,0.06557377],[0.14689266,0.06557377],[0.20338983,0.06557377]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}" -90344d84-9485-4986-93e1-e94d73ee0aa3,Standard,91179f48-2d6c-4ae4-864c-d667f57a6e66,"{""type"":""LineString"",""coordinates"":[[0.09039548,0.24590164],[0.11864407,0.24590164],[0.14689266,0.24590164]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}" -31dab450-fe89-4ede-99e2-cba361671910,Standard,b714db4a-db55-4957-b503-2300559bb81f,"{""type"":""LineString"",""coordinates"":[[0.20338983,0.1557377],[0.14689266,0.1557377],[0.17514124,0.1557377]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}" -79a153b2-2b64-4b59-bbe5-946a3c0073bd,Standard,c3206971-b22f-4090-826b-e08d4bc8ffad,"{""type"":""LineString"",""coordinates"":[[0.20338983,0.24590164],[0.25423729,0.24590164],[0.22881421,0.24590164]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}" -6dd7072b-da09-4d47-a7ad-a57389a918f9,Standard,cba98774-b8c6-45f4-84cc-2fde121a77fa,"{""type"":""LineString"",""coordinates"":[[0.14689266,0.06557377],[0.11864407,0.06557377],[0.09039548,0.06557377]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}" -74887fe3-c3a5-4919-8b77-743e3f2c5bd4,Standard,d48db7e8-5fde-456f-8b17-02d18a5a2e32,"{""type"":""LineString"",""coordinates"":[[0.31073446,0.1557377],[0.25423729,0.1557377],[0.28248588,0.1557377]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}" -a8cd181f-6834-437f-aa9e-6b165504b870,Standard,f0d4491e-eb64-49a0-8fc3-44fa5bd9dcee,"{""type"":""LineString"",""coordinates"":[[0.31073446,0.06557377],[0.25423729,0.06557377],[0.28248588,0.06557377]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}" \ No newline at end of file diff --git a/input/samples/vn_simona_ns/fullGrid/line_input.csv b/input/samples/vn_simona_ns/fullGrid/line_input.csv deleted file mode 100644 index 0eea1cc0f7..0000000000 --- a/input/samples/vn_simona_ns/fullGrid/line_input.csv +++ /dev/null @@ -1,16 +0,0 @@ -uuid,geo_position,id,length,node_a,node_b,olm_characteristic,operates_from,operates_until,operator,parallel_devices,type -5d1176ac-e64e-4811-8d0c-c87d9060ba2a,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_NET116_ONS-NS_NET116_S2_1,0.029999999329447746,31a2b9bf-e785-4475-aa44-1c34646e8c79,39112046-8953-4e73-a5d9-6a8183a77436,"olm:{(0.0,1.0)}",,,,1,4ee89e0b-1e97-4dde-ba57-3b0b862957c5 -106f03b3-b84e-4d36-ab09-13b3dc3403e7,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_NET116_S2_3-NS_NET116_S2_4,0.029999999329447746,df9c3d91-41a0-4e49-bb1f-1cd2a8fd9a2e,e05c68b1-11cd-43fd-a4b2-31e4db380c78,"olm:{(0.0,1.0)}",,,,1,17e87208-d092-4514-8b23-ab91598798ce -c3206971-b22f-4090-826b-e08d4bc8ffad,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_NET116_S1_3-NS_NET116_S1_4,0.029999999329447746,e4c8b492-965c-4b5a-8d14-e8ff2c027ce9,b3a7431d-89b0-41cb-87a9-5853890796cd,"olm:{(0.0,1.0)}",,,,1,b63674e1-6827-4e8b-b3e6-34ee6d4823a9 -814dec01-f6f2-4817-8f83-2e758011b033,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_NET116_ONS-NS_NET116_S1_1,0.029999999329447746,31a2b9bf-e785-4475-aa44-1c34646e8c79,8a50fb37-81cf-47c9-8850-a12b4391e2e7,"olm:{(0.0,1.0)}",,,,1,a7071b97-2be4-40bf-a0b1-b8faf6bff6d6 -cba98774-b8c6-45f4-84cc-2fde121a77fa,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_NET116_S3_1-NS_NET116_S3_2,0.029999999329447746,4a6f6058-e654-464d-9367-2dca7185c6d7,550ebca7-1455-44eb-9431-ffbf08e58bd4,"olm:{(0.0,1.0)}",,,,1,6986321a-0a85-411e-8b2d-9d929a7f298b -2d552f15-6871-4e09-84b1-003da098d909,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_NET116_S2_1-NS_NET116_S2_2,0.029999999329447746,39112046-8953-4e73-a5d9-6a8183a77436,d53ff076-dadd-44f8-85d4-68f48991f7d0,"olm:{(0.0,1.0)}",,,,1,e0de3267-5ef6-4fcb-83c9-771d53e7fde8 -8bb29769-404b-4bf6-837e-f9ca1b389bf9,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_NET116_S3_2-NS_NET116_S3_3,0.029999999329447746,550ebca7-1455-44eb-9431-ffbf08e58bd4,d69efff2-ba8b-4aa6-a4a1-27267964147a,"olm:{(0.0,1.0)}",,,,1,9db7e41e-7360-4529-bbdb-dbd220117f61 -37c8ebe1-5d29-4806-a84a-fc00e4279fa6,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_NET116_S3_3-NS_NET116_S3_4,0.029999999329447746,d69efff2-ba8b-4aa6-a4a1-27267964147a,36dccefc-f04c-493f-bb88-11343583bf9f,"olm:{(0.0,1.0)}",,,,1,ea01036f-5b25-47dd-98e3-0d944fd2e670 -f0d4491e-eb64-49a0-8fc3-44fa5bd9dcee,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_NET116_S3_4-NS_NET116_S3_5,0.029999999329447746,36dccefc-f04c-493f-bb88-11343583bf9f,b9a28095-68f7-44c1-9ccc-6efc5ea84c59,"olm:{(0.0,1.0)}",,,,1,e2ee2ab1-8e22-41be-9a1f-cfdee99d611e -4bdb6569-72da-4d60-97b1-b5a639032d5d,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_NET116_S1_4-NS_NET116_S1_5,0.029999999329447746,b3a7431d-89b0-41cb-87a9-5853890796cd,92301422-94ae-48ab-89c7-a69eea9450b2,"olm:{(0.0,1.0)}",,,,1,3362b510-7d57-44b1-a6cf-8081aac84a5b -91179f48-2d6c-4ae4-864c-d667f57a6e66,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_NET116_S1_1-NS_NET116_S1_2,0.029999999329447746,8a50fb37-81cf-47c9-8850-a12b4391e2e7,3e21f3a1-2c9c-4138-bcc9-466b004609ed,"olm:{(0.0,1.0)}",,,,1,9312de84-9197-462f-9d69-7d2fea8f7f68 -10c6a103-3132-4324-8f76-8c1858d51fcb,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_NET116_S1_2-NS_NET116_S1_3,0.029999999329447746,3e21f3a1-2c9c-4138-bcc9-466b004609ed,e4c8b492-965c-4b5a-8d14-e8ff2c027ce9,"olm:{(0.0,1.0)}",,,,1,1173fa67-2573-4476-aa5e-c3fe756a7f07 -b714db4a-db55-4957-b503-2300559bb81f,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_NET116_S2_2-NS_NET116_S2_3,0.029999999329447746,d53ff076-dadd-44f8-85d4-68f48991f7d0,df9c3d91-41a0-4e49-bb1f-1cd2a8fd9a2e,"olm:{(0.0,1.0)}",,,,1,567497fb-f77c-4e25-a030-9d40b377711c -d48db7e8-5fde-456f-8b17-02d18a5a2e32,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_NET116_S2_4-NS_NET116_S2_5,0.029999999329447746,e05c68b1-11cd-43fd-a4b2-31e4db380c78,32bd37df-255b-4eb2-9d16-5b711132eee6,"olm:{(0.0,1.0)}",,,,1,b42b9195-4953-403f-8e52-55cc8e653c94 -25a1e96e-b425-48bf-a3ae-dc54ea9da47f,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_NET116_ONS-NS_NET116_S3_1,0.029999999329447746,31a2b9bf-e785-4475-aa44-1c34646e8c79,4a6f6058-e654-464d-9367-2dca7185c6d7,"olm:{(0.0,1.0)}",,,,1,cde5f3a8-4a99-49b7-a7e7-b73e1c6d8003 diff --git a/input/samples/vn_simona_ns/fullGrid/line_type_input.csv b/input/samples/vn_simona_ns/fullGrid/line_type_input.csv deleted file mode 100644 index fca609822e..0000000000 --- a/input/samples/vn_simona_ns/fullGrid/line_type_input.csv +++ /dev/null @@ -1,16 +0,0 @@ -uuid,b,g,i_max,id,r,v_rated,x -a7071b97-2be4-40bf-a0b1-b8faf6bff6d6,191.636993408203,0.0,265.0,NS_3,0.253899991512299,0.4,0.0691149979829788 -4ee89e0b-1e97-4dde-ba57-3b0b862957c5,185.35400390625,0.0,235.0,NS_2,0.319999992847443,0.4,0.0691149979829788 -cde5f3a8-4a99-49b7-a7e7-b73e1c6d8003,185.35400390625,0.0,235.0,NS_2,0.319999992847443,0.4,0.0691149979829788 -9312de84-9197-462f-9d69-7d2fea8f7f68,163.363006591797,0.0,195.0,NS_1,0.442999988794327,0.4,0.0722566023468971 -1173fa67-2573-4476-aa5e-c3fe756a7f07,191.636993408203,0.0,265.0,NS_3,0.253899991512299,0.4,0.0691149979829788 -b63674e1-6827-4e8b-b3e6-34ee6d4823a9,191.636993408203,0.0,265.0,NS_3,0.253899991512299,0.4,0.0691149979829788 -3362b510-7d57-44b1-a6cf-8081aac84a5b,191.636993408203,0.0,265.0,NS_3,0.253899991512299,0.4,0.0691149979829788 -e0de3267-5ef6-4fcb-83c9-771d53e7fde8,185.35400390625,0.0,235.0,NS_2,0.319999992847443,0.4,0.0691149979829788 -567497fb-f77c-4e25-a030-9d40b377711c,191.636993408203,0.0,265.0,NS_3,0.253899991512299,0.4,0.0691149979829788 -17e87208-d092-4514-8b23-ab91598798ce,163.363006591797,0.0,195.0,NS_1,0.442999988794327,0.4,0.0722566023468971 -b42b9195-4953-403f-8e52-55cc8e653c94,163.363006591797,0.0,195.0,NS_1,0.442999988794327,0.4,0.0722566023468971 -6986321a-0a85-411e-8b2d-9d929a7f298b,163.363006591797,0.0,195.0,NS_1,0.442999988794327,0.4,0.0722566023468971 -9db7e41e-7360-4529-bbdb-dbd220117f61,163.363006591797,0.0,195.0,NS_1,0.442999988794327,0.4,0.0722566023468971 -ea01036f-5b25-47dd-98e3-0d944fd2e670,185.35400390625,0.0,235.0,NS_2,0.319999992847443,0.4,0.0691149979829788 -e2ee2ab1-8e22-41be-9a1f-cfdee99d611e,163.363006591797,0.0,195.0,NS_1,0.442999988794327,0.4,0.0722566023468971 \ No newline at end of file diff --git a/input/samples/vn_simona_ns/fullGrid/load_input.csv b/input/samples/vn_simona_ns/fullGrid/load_input.csv deleted file mode 100644 index 0a0c717150..0000000000 --- a/input/samples/vn_simona_ns/fullGrid/load_input.csv +++ /dev/null @@ -1,16 +0,0 @@ -uuid,cos_phi_rated,dsm,e_cons_annual,id,load_profile,node,operates_from,operates_until,operator,q_characteristics,s_rated -c3434742-e4f0-49e5-baa7-c1e3045c732c,0.9700000286102295,false,4000.0,NS_NET116_L_S1_5(1),h0,92301422-94ae-48ab-89c7-a69eea9450b2,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605 -40ddb9cf-46f1-4ee6-8feb-432affbc95bf,0.9700000286102295,false,4000.0,NS_NET116_L_S2_4(1),h0,e05c68b1-11cd-43fd-a4b2-31e4db380c78,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605 -a1af0e38-339c-491c-94d8-446cf662d89b,0.9700000286102295,false,4000.0,NS_NET116_L_S3_4(1),h0,36dccefc-f04c-493f-bb88-11343583bf9f,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605 -4642d648-b0dd-4597-a3bd-2cc1fce74f27,0.9700000286102295,false,4000.0,NS_NET116_L_S3_1(1),h0,4a6f6058-e654-464d-9367-2dca7185c6d7,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605 -ff0b995a-86ff-4f4d-987e-e475a64f2180,0.9700000286102295,false,4000.0,NS_NET116_L_S1_2(1),h0,3e21f3a1-2c9c-4138-bcc9-466b004609ed,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605 -2efedcfd-4972-4d13-9d62-180f1b9a76ad,0.9700000286102295,false,4000.0,NS_NET116_L_S2_5(1),h0,32bd37df-255b-4eb2-9d16-5b711132eee6,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605 -fd1a8de9-722a-4304-8799-e1e976d9979c,0.9700000286102295,false,4000.0,NS_NET116_L_S1_1(1),h0,8a50fb37-81cf-47c9-8850-a12b4391e2e7,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605 -f4536a0c-33d2-4c57-918c-0ebdf768265a,0.9700000286102295,false,4000.0,NS_NET116_L_S3_3(1),h0,d69efff2-ba8b-4aa6-a4a1-27267964147a,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605 -7b040b43-011a-4402-a176-9b9b41ce0c37,0.9700000286102295,false,4000.0,NS_NET116_L_S2_3(1),h0,df9c3d91-41a0-4e49-bb1f-1cd2a8fd9a2e,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605 -11e776fa-ecc1-4332-821f-d7e56758b988,0.9700000286102295,false,4000.0,NS_NET116_L_S2_2(1),h0,d53ff076-dadd-44f8-85d4-68f48991f7d0,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605 -98c1a2ab-bd09-4c77-a389-d088aed894b1,0.9700000286102295,false,4000.0,NS_NET116_L_S3_2(1),h0,550ebca7-1455-44eb-9431-ffbf08e58bd4,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605 -fd2e19b6-d5e3-4776-9456-8787a2160d9d,0.9700000286102295,false,4000.0,NS_NET116_L_S1_4(1),h0,b3a7431d-89b0-41cb-87a9-5853890796cd,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605 -a12570e4-afeb-4af8-a86c-9e804eea4886,0.9700000286102295,false,4000.0,NS_NET116_L_S3_5(1),h0,b9a28095-68f7-44c1-9ccc-6efc5ea84c59,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605 -70fcdefd-49c1-455f-9ceb-013dcbf26887,0.9700000286102295,false,4000.0,NS_NET116_L_S1_3(1),h0,e4c8b492-965c-4b5a-8d14-e8ff2c027ce9,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605 -2a9149a4-4778-41d8-8865-090499833016,0.9700000286102295,false,4000.0,NS_NET116_L_S2_1(1),h0,39112046-8953-4e73-a5d9-6a8183a77436,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605 diff --git a/input/samples/vn_simona_ns/fullGrid/node_graphic_input.csv b/input/samples/vn_simona_ns/fullGrid/node_graphic_input.csv deleted file mode 100644 index 1a0280a78d..0000000000 --- a/input/samples/vn_simona_ns/fullGrid/node_graphic_input.csv +++ /dev/null @@ -1,17 +0,0 @@ -uuid,graphic_layer,node,path,point -08d4e2ec-8532-4327-9d9d-aa51393d53e8,Standard,31a2b9bf-e785-4475-aa44-1c34646e8c79,"{""type"":""LineString"",""coordinates"":[[0.03389831,0.27868852],[0.06779661,0.27868852]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}","{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}" -cc657d07-3495-45b8-a90b-76769e14f2d8,Standard,8a50fb37-81cf-47c9-8850-a12b4391e2e7,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}","{""type"":""Point"",""coordinates"":[0.09039548,0.24590164],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}" -0434dc12-6149-4832-a288-6620ee0389f2,Standard,3e21f3a1-2c9c-4138-bcc9-466b004609ed,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}","{""type"":""Point"",""coordinates"":[0.14689266,0.24590164],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}" -f7e1e922-52d3-469f-98c6-5353a5d74641,Standard,e4c8b492-965c-4b5a-8d14-e8ff2c027ce9,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}","{""type"":""Point"",""coordinates"":[0.20338983,0.24590164],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}" -a485471a-ad4f-4ca8-b960-ebf1a5504fd4,Standard,b3a7431d-89b0-41cb-87a9-5853890796cd,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}","{""type"":""Point"",""coordinates"":[0.25423729,0.24590164],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}" -3080afb0-0cdc-4641-ae10-71c05a48fa0f,Standard,92301422-94ae-48ab-89c7-a69eea9450b2,"{""type"":""LineString"",""coordinates"":[[12.8273,52.2895],[12.8273,52.2895]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}","{""type"":""Point"",""coordinates"":[0.31073446,0.24590164],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}" -f17d8fce-ffcf-4214-af18-af2bb64985b0,Standard,39112046-8953-4e73-a5d9-6a8183a77436,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}","{""type"":""Point"",""coordinates"":[0.09039548,0.1557377],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}" -c67ff8a5-d296-47db-970a-4f6b2ab7c77a,Standard,d53ff076-dadd-44f8-85d4-68f48991f7d0,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}","{""type"":""Point"",""coordinates"":[0.14689266,0.1557377],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}" -7bf5033d-efa1-4b45-b885-4c22ede1336d,Standard,df9c3d91-41a0-4e49-bb1f-1cd2a8fd9a2e,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}","{""type"":""Point"",""coordinates"":[0.20338983,0.1557377],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}" -5cb93639-bc3a-4e87-bd02-1d8e16680219,Standard,e05c68b1-11cd-43fd-a4b2-31e4db380c78,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}","{""type"":""Point"",""coordinates"":[0.25423729,0.1557377],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}" -3767a331-5208-4bc1-a8cf-cb4b697fdf0f,Standard,32bd37df-255b-4eb2-9d16-5b711132eee6,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}","{""type"":""Point"",""coordinates"":[0.31073446,0.1557377],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}" -94f85ca1-6313-4083-9f67-4f352b5fee15,Standard,4a6f6058-e654-464d-9367-2dca7185c6d7,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}","{""type"":""Point"",""coordinates"":[0.09039548,0.06557377],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}" -85393fb1-b60f-48ab-b0d5-56002cc2ebe2,Standard,550ebca7-1455-44eb-9431-ffbf08e58bd4,"{""type"":""LineString"",""coordinates"":[[12.8273,52.2895],[12.8273,52.2895]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}","{""type"":""Point"",""coordinates"":[0.14689266,0.06557377],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}" -ffd94a18-e4f0-4088-893a-3d9dae11ddf5,Standard,d69efff2-ba8b-4aa6-a4a1-27267964147a,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}","{""type"":""Point"",""coordinates"":[0.20338983,0.06557377],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}" -416dd5b1-8b68-435b-88da-8ff638754355,Standard,36dccefc-f04c-493f-bb88-11343583bf9f,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}","{""type"":""Point"",""coordinates"":[0.25423729,0.06557377],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}" -2ffc1daf-fd76-4180-8274-6c57ae461e14,Standard,b9a28095-68f7-44c1-9ccc-6efc5ea84c59,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}","{""type"":""Point"",""coordinates"":[0.31073446,0.06557377],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}" \ No newline at end of file diff --git a/input/samples/vn_simona_ns/fullGrid/node_input.csv b/input/samples/vn_simona_ns/fullGrid/node_input.csv deleted file mode 100644 index 72bb318779..0000000000 --- a/input/samples/vn_simona_ns/fullGrid/node_input.csv +++ /dev/null @@ -1,18 +0,0 @@ -uuid,geo_position,id,operates_from,operates_until,operator,slack,subnet,v_rated,v_target,volt_lvl -39112046-8953-4e73-a5d9-6a8183a77436,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",NS_NET116_S2_1,,,,false,116,0.4,1.0,Niederspannung -b3a7431d-89b0-41cb-87a9-5853890796cd,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",NS_NET116_S1_4,,,,false,116,0.4,1.0,Niederspannung -e05c68b1-11cd-43fd-a4b2-31e4db380c78,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",NS_NET116_S2_4,,,,false,116,0.4,1.0,Niederspannung -31a2b9bf-e785-4475-aa44-1c34646e8c79,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",NS_NET116_ONS,,,,false,116,0.4,1.0,Niederspannung -e4c8b492-965c-4b5a-8d14-e8ff2c027ce9,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",NS_NET116_S1_3,,,,false,116,0.4,1.0,Niederspannung -b9a28095-68f7-44c1-9ccc-6efc5ea84c59,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",NS_NET116_S3_5,,,,false,116,0.4,1.0,Niederspannung -92301422-94ae-48ab-89c7-a69eea9450b2,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}",NS_NET116_S1_5,,,,false,116,0.4,1.0,Niederspannung -4a6f6058-e654-464d-9367-2dca7185c6d7,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",NS_NET116_S3_1,,,,false,116,0.4,1.0,Niederspannung -36dccefc-f04c-493f-bb88-11343583bf9f,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",NS_NET116_S3_4,,,,false,116,0.4,1.0,Niederspannung -8a50fb37-81cf-47c9-8850-a12b4391e2e7,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",NS_NET116_S1_1,,,,false,116,0.4,1.0,Niederspannung -d53ff076-dadd-44f8-85d4-68f48991f7d0,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",NS_NET116_S2_2,,,,false,116,0.4,1.0,Niederspannung -3e21f3a1-2c9c-4138-bcc9-466b004609ed,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",NS_NET116_S1_2,,,,false,116,0.4,1.0,Niederspannung -550ebca7-1455-44eb-9431-ffbf08e58bd4,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}",NS_NET116_S3_2,,,,false,116,0.4,1.0,Niederspannung -32bd37df-255b-4eb2-9d16-5b711132eee6,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",NS_NET116_S2_5,,,,false,116,0.4,1.0,Niederspannung -df9c3d91-41a0-4e49-bb1f-1cd2a8fd9a2e,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",NS_NET116_S2_3,,,,false,116,0.4,1.0,Niederspannung -d69efff2-ba8b-4aa6-a4a1-27267964147a,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",NS_NET116_S3_3,,,,false,116,0.4,1.0,Niederspannung -b22c9299-5fb4-46a7-b566-fc17e0d51d60,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",MS1_06,,,,true,11,20.0,1.0,Mittelspannung \ No newline at end of file diff --git a/input/samples/vn_simona_ns/fullGrid/pv_input.csv b/input/samples/vn_simona_ns/fullGrid/pv_input.csv deleted file mode 100644 index 64bcc385e9..0000000000 --- a/input/samples/vn_simona_ns/fullGrid/pv_input.csv +++ /dev/null @@ -1,3 +0,0 @@ -uuid,albedo,azimuth,cos_phi_rated,elevation_angle,eta_conv,id,k_g,k_t,market_reaction,node,operates_from,operates_until,operator,q_characteristics,s_rated -de8cfef5-7620-4b9e-9a10-1faebb5a80c0,0.20000000298023224,4.093344211578369,0.8999999761581421,37.69556427001953,97.0,NS_NET116_S1_5_PV,0.8999999761581421,1.0,false,92301422-94ae-48ab-89c7-a69eea9450b2,,,,"cosPhiFixed:{(0.0,0.9)}",30.0 -2560c371-f420-4c2a-b4e6-e04c11b64c03,0.20000000298023224,0.7802008390426636,0.8999999761581421,40.086585998535156,98.0,NS_NET116_S3_2_PV,0.8999999761581421,1.0,false,550ebca7-1455-44eb-9431-ffbf08e58bd4,,,,"cosPhiFixed:{(0.0,0.9)}",30.0 diff --git a/input/samples/vn_simona_ns/fullGrid/transformer_2_w_input.csv b/input/samples/vn_simona_ns/fullGrid/transformer_2_w_input.csv deleted file mode 100644 index 592200c023..0000000000 --- a/input/samples/vn_simona_ns/fullGrid/transformer_2_w_input.csv +++ /dev/null @@ -1,2 +0,0 @@ -uuid,auto_tap,id,node_a,node_b,operates_from,operates_until,operator,parallel_devices,tap_pos,type -2cbbd48e-e6d8-40b4-80cc-010157dfbf3e,false,TRA_MS1_06-NS_NET116,b22c9299-5fb4-46a7-b566-fc17e0d51d60,31a2b9bf-e785-4475-aa44-1c34646e8c79,,,,1,10,4984f493-d6e5-4201-a040-c10722b30362 diff --git a/input/samples/vn_simona_ns/fullGrid/transformer_2_w_type_input.csv b/input/samples/vn_simona_ns/fullGrid/transformer_2_w_type_input.csv deleted file mode 100644 index 1140eb9b9b..0000000000 --- a/input/samples/vn_simona_ns/fullGrid/transformer_2_w_type_input.csv +++ /dev/null @@ -1,2 +0,0 @@ -uuid,b_m,d_phi,d_v,g_m,id,r_sc,s_rated,tap_max,tap_min,tap_neutr,tap_side,v_rated_a,v_rated_b,x_sc -4984f493-d6e5-4201-a040-c10722b30362,0.0,0.0,0.5,0.0,MS-NS_1,10.078,630.0,10,-10,0,false,20.0,0.4,23.312 diff --git a/input/samples/vn_simona_ns/vn_simona_ns.conf b/input/samples/vn_simona_ns/vn_simona_ns.conf deleted file mode 100644 index 9f85016d4c..0000000000 --- a/input/samples/vn_simona_ns/vn_simona_ns.conf +++ /dev/null @@ -1,240 +0,0 @@ -include "../common/pekko.conf" - -######### -# ATTENTION: Do not change this config file directly but use it as a base for your personal delta config for the -# vn_simona scenario! Delta configs can be created by including the config you want to change -# parameters from via include (e.g. include "input/samples/vn_simona/vn_simona.conf") at the -# beginning of your config file and then just override the parameters you want to change! -######### - -################################################################## -# Simulation Parameters -################################################################## -simona.simulationName = "vn_simona_ns" - -################################################################## -# Time Parameters -################################################################## -simona.time.startDateTime = "2020-09-01T00:00:00Z" -simona.time.endDateTime = "2020-09-01T01:00:00Z" -simona.time.schedulerReadyCheckWindow = 900 - -################################################################## -# Input Parameters -################################################################## -simona.input.primary.csvParams = { - directoryPath: "simona/input/samples/vn_simona_ns/fullGrid" - csvSep: "," - isHierarchic: false -} -simona.input.grid.datasource.id = "csv" -simona.input.grid.datasource.csvParams = { - directoryPath: "simona/input/samples/vn_simona_ns/fullGrid" - csvSep: "," - isHierarchic: false -} - -simona.input.weather.datasource = { - scheme = "icon" - sampleParams.use = true - coordinateSource.sampleParams.use = true - maxCoordinateDistance = 50000 -} - -################################################################## -# Output Parameters -################################################################## -simona.output.base.dir = "simona/output/vn_simona_ns" -simona.output.base.addTimestampToOutputDir = true - -simona.output.sink.csv { - fileFormat = ".csv" - filePrefix = "" - fileSuffix = "" -} - -simona.output.grid = { - notifier = "grid" - nodes = false - lines = false - switches = false - transformers2w = false - transformers3w = false -} -simona.output.participant.defaultConfig = { - notifier = "default" - powerRequestReply = false - simulationResult = true -} -simona.output.participant.individualConfigs = [ - { - notifier = "pv" - powerRequestReply = false - simulationResult = true - }, - { - notifier = "wec" - powerRequestReply = false - simulationResult = false - }, - { - notifier = "evcs" - powerRequestReply = false - simulationResult = false - }, - { - notifier = "bm" - powerRequestReply = false - simulationResult = false - }, - { - notifier = "chp" - powerRequestReply = false - simulationResult = false - }, - { - notifier = "ev" - powerRequestReply = false - simulationResult = false - }, - { - notifier = "hp" - powerRequestReply = false - simulationResult = false - }, - { - notifier = "storage" - powerRequestReply = false - simulationResult = false - }, - { - notifier = "fixedFeedIn" - powerRequestReply = false - simulationResult = false - } -] -simona.output.thermal = { - defaultConfig = { - notifier = "default", - simulationResult = false - } - individualConfigs = [ - { - notifier = "house", - simulationResult = false - } - ] -} - -################################################################## -# Runtime Configuration // todo refactor as this naming is misleading and partly unneeded -################################################################## -simona.runtime.selected_subgrids = [] -simona.runtime.selected_volt_lvls = [] - -simona.runtime.participant.load = { - defaultConfig = { - calculateMissingReactivePowerWithModel = false - uuids = ["default"] - scaling = 1.0 - modelBehaviour = "fix" - reference = "power" - } - individualConfigs = [] -} - -simona.runtime.participant.fixedFeedIn = { - defaultConfig = { - calculateMissingReactivePowerWithModel = false - uuids = ["default"] - scaling = 1.0 - } - individualConfigs = [] -} - -simona.runtime.participant.pv = { - defaultConfig = { - calculateMissingReactivePowerWithModel = false - uuids = ["default"] - scaling = 1.0 - } - individualConfigs = [] -} - -simona.runtime.participant.wec = { - defaultConfig = { - calculateMissingReactivePowerWithModel = false - uuids = ["default"] - scaling = 1.0 - } - individualConfigs = [] -} - -simona.runtime.participant.evcs = { - defaultConfig = { - calculateMissingReactivePowerWithModel = false - uuids = ["default"] - scaling = 1.0 - } - individualConfigs = [] -} - -simona.runtime.participant.hp = { - defaultConfig = { - calculateMissingReactivePowerWithModel = false - uuids = ["default"] - scaling = 1.0 - } - individualConfigs = [] -} - -# # # # # -# ATTENTION: calculateMissingReactivePowerWithModel and scaling is ignored here. -# # # # # -simona.runtime.participant.em = { - defaultConfig = { - calculateMissingReactivePowerWithModel = false - uuids = ["default"] - scaling = 1.0 - } - individualConfigs = [] -} - -################################################################## -# Event Configuration -################################################################## -simona.event.listener = [] - -################################################################## -# Grid Configuration -################################################################## - -simona.gridConfig.refSystems = [ - {sNom = "100 kVA", vNom = "0.4 kV", voltLvls = [{id = "NS", vNom = "0.4 kV"}]}, - {sNom = "60 MVA", vNom = "20 kV", voltLvls = [{id = "MS", vNom = "20 kV"}]}, - {sNom = "600 MVA", vNom = "110 kV", voltLvls = [{id = "HS", vNom = "110 kV"}]}, - {sNom = "1000 MVA", vNom = "380 kV", voltLvls = [{id = "HoeS", vNom = "380 kV"}]} -] - -################################################################## -# Power Flow Configuration -################################################################## -simona.powerflow.maxSweepPowerDeviation = 1E-5 // the maximum allowed deviation in power between two sweeps, before overall convergence is assumed -simona.powerflow.newtonraphson.epsilon = [1E-12] -simona.powerflow.newtonraphson.iterations = 50 -simona.powerflow.resolution = "3600s" -simona.powerflow.stopOnFailure = true - -simona.control.transformer = [ - { - transformers = ["31a2b9bf-e785-4475-aa44-1c34646e8c79"], - measurements = ["923f2d69-3093-4198-86e4-13d2d1c220f8"], - vMin = 0.98, - vMax = 1.02 - }, { - transformers = ["1132dbf4-e8a1-44ae-8415-f42d4497aa1d"], - measurements = ["7686b818-a0ba-465c-8e4e-f7d3c4e171fc"], - vMin = 0.98, - vMax = 1.02 - } -] diff --git a/input/samples/vn_simona_small_with_em/fullGrid/em_input.csv b/input/samples/vn_simona_small_with_em/fullGrid/em_input.csv deleted file mode 100755 index 22729f2b61..0000000000 --- a/input/samples/vn_simona_small_with_em/fullGrid/em_input.csv +++ /dev/null @@ -1,3 +0,0 @@ -uuid,control_strategy,parent_em,id,operates_from,operates_until,operator -fd1a8de9-722a-4304-8799-e1e976d9979c,self_optimization,,EM_NS_Node_3,,, -ff0b995a-86ff-4f4d-987e-e475a64f2180,self_optimization,,EM_NS_Node_4,,, \ No newline at end of file diff --git a/input/samples/vn_simona_small_with_em/fullGrid/fixed_feed_in_input.csv b/input/samples/vn_simona_small_with_em/fullGrid/fixed_feed_in_input.csv deleted file mode 100755 index 90a18373ca..0000000000 --- a/input/samples/vn_simona_small_with_em/fullGrid/fixed_feed_in_input.csv +++ /dev/null @@ -1,2 +0,0 @@ -uuid,cos_phi_rated,id,node,operates_from,operates_until,operator,q_characteristics,s_rated -9abe950d-362e-4efe-b686-500f84d8f368,0.9,feed_in_hs_s4,401f37f8-6f2c-4564-bc78-6736cb9cbf8d,,,,"cosPhiFixed:{(0.0,0.95)}",200.0 diff --git a/input/samples/vn_simona_small_with_em/fullGrid/line_input.csv b/input/samples/vn_simona_small_with_em/fullGrid/line_input.csv deleted file mode 100755 index 0f7b0204e7..0000000000 --- a/input/samples/vn_simona_small_with_em/fullGrid/line_input.csv +++ /dev/null @@ -1,6 +0,0 @@ -uuid,geo_position,id,length,node_a,node_b,olm_characteristic,operates_from,operates_until,operator,parallel_devices,type -fd04cc09-6ff8-48ea-a5eb-453e9d59d6d7,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_Node_1-NS_Node_3,0.04,00d03670-7833-47ee-ad52-04d18d1c64fd,33f29587-f63e-45b7-960b-037bda37a3cb,"olm:{(0.0,1.0)}",,,,1,9a8e9b63-af21-4c1b-8db7-fc2924f9610e -b5a66c60-7189-4c86-a32a-4d1aa6568475,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_Node_4-NS_Node_1,0.030304,401f37f8-6f2c-4564-bc78-6736cb9cbf8d,00d03670-7833-47ee-ad52-04d18d1c64fd,"olm:{(0.0,1.0)}",,,,1,9a8e9b63-af21-4c1b-8db7-fc2924f9610e -42828cac-b67e-4d5c-b4fa-787c57f16fde,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_Node_3-NS_Node_2,0.0251089,33f29587-f63e-45b7-960b-037bda37a3cb,dfae9806-9b44-4995-ba27-d66d8e4a43e0,"olm:{(0.0,1.0)}",,,,1,9a8e9b63-af21-4c1b-8db7-fc2924f9610e -997840bf-8c94-444f-83f1-e9c991706d7c,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_Node_1-NS_Node_2,0.0377768,00d03670-7833-47ee-ad52-04d18d1c64fd,dfae9806-9b44-4995-ba27-d66d8e4a43e0,"olm:{(0.0,1.0)}",,,,1,9a8e9b63-af21-4c1b-8db7-fc2924f9610e -81f4c585-6170-4a9e-981f-2185a0d7f2ec,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_Node_3-NS_Node_4,0.031,33f29587-f63e-45b7-960b-037bda37a3cb,401f37f8-6f2c-4564-bc78-6736cb9cbf8d,"olm:{(0.0,1.0)}",,,,1,9a8e9b63-af21-4c1b-8db7-fc2924f9610e diff --git a/input/samples/vn_simona_small_with_em/fullGrid/line_type_input.csv b/input/samples/vn_simona_small_with_em/fullGrid/line_type_input.csv deleted file mode 100755 index 91202b2778..0000000000 --- a/input/samples/vn_simona_small_with_em/fullGrid/line_type_input.csv +++ /dev/null @@ -1,2 +0,0 @@ -uuid,b,g,i_max,id,r,v_rated,x -9a8e9b63-af21-4c1b-8db7-fc2924f9610e,273.31899999999996,0.0,357.0,NAYY 4x240SE 0.6/1kV,0.1267,0.4,0.0797965 diff --git a/input/samples/vn_simona_small_with_em/fullGrid/load_input.csv b/input/samples/vn_simona_small_with_em/fullGrid/load_input.csv deleted file mode 100755 index 2324353168..0000000000 --- a/input/samples/vn_simona_small_with_em/fullGrid/load_input.csv +++ /dev/null @@ -1,5 +0,0 @@ -uuid,cos_phi_rated,dsm,e_cons_annual,id,load_profile,node,operates_from,operates_until,operator,q_characteristics,s_rated,em -4dca3b1d-5d24-444a-b4df-f4fa23b9ef1b,0.949999988079071,false,4000.0,LOAD_NS_Node_1,h0,00d03670-7833-47ee-ad52-04d18d1c64fd,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605, -9c5991bc-24df-496b-b4ce-5ec27657454c,0.949999988079071,false,4000.0,LOAD_NS_Node_2,h0,dfae9806-9b44-4995-ba27-d66d8e4a43e0,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605, -58b9f934-f7c4-4335-9894-3c80d9e6b852,0.949999988079071,false,4000.0,LOAD_NS_Node_3,h0,33f29587-f63e-45b7-960b-037bda37a3cb,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605,fd1a8de9-722a-4304-8799-e1e976d9979c -283a1252-a774-4b04-bfcf-fe8879065982,0.949999988079071,false,4000.0,LOAD_NS_Node_4,h0,401f37f8-6f2c-4564-bc78-6736cb9cbf8d,,,,"cosPhiFixed:{(0.0,1.0)}",2.3157899379730225,ff0b995a-86ff-4f4d-987e-e475a64f2180 diff --git a/input/samples/vn_simona_small_with_em/fullGrid/node_input.csv b/input/samples/vn_simona_small_with_em/fullGrid/node_input.csv deleted file mode 100755 index 21772153f5..0000000000 --- a/input/samples/vn_simona_small_with_em/fullGrid/node_input.csv +++ /dev/null @@ -1,6 +0,0 @@ -uuid,geo_position,id,operates_from,operates_until,operator,slack,subnet,v_rated,v_target,volt_lvl -00d03670-7833-47ee-ad52-04d18d1c64fd,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",NS_Node_1,,,,false,2,0.4,1.0,Niederspannung -dfae9806-9b44-4995-ba27-d66d8e4a43e0,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}",NS_Node_2,,,,false,2,0.4,1.0,Niederspannung -34cd8ee0-e607-4c47-89a7-121c3e32768a,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",MS_Node_0,,,,true,1,20.0,1.025,Mittelspannung -401f37f8-6f2c-4564-bc78-6736cb9cbf8d,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}",NS_Node_4,,,,false,2,0.4,1.0,Niederspannung -33f29587-f63e-45b7-960b-037bda37a3cb,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",NS_Node_3,,,,false,2,0.4,1.0,Niederspannung diff --git a/input/samples/vn_simona_small_with_em/fullGrid/pv_input.csv b/input/samples/vn_simona_small_with_em/fullGrid/pv_input.csv deleted file mode 100644 index e418358125..0000000000 --- a/input/samples/vn_simona_small_with_em/fullGrid/pv_input.csv +++ /dev/null @@ -1,3 +0,0 @@ -uuid,albedo,azimuth,cos_phi_rated,elevation_angle,eta_conv,id,k_g,k_t,market_reaction,node,operates_from,operates_until,operator,q_characteristics,s_rated,em -a1eb7fc1-3bee-4b65-a387-ef3046644bf0,0.20000000298023224,-8.999500274658203,0.8999999761581421,37.14517593383789,98.0,PV_NS_Node_4,0.8999999761581421,1.0,false,401f37f8-6f2c-4564-bc78-6736cb9cbf8d,,,,"cosPhiFixed:{(0.0,0.9)}",10.0,ff0b995a-86ff-4f4d-987e-e475a64f2180 -9d7cd8e2-d859-4f4f-9c01-abba06ef2e2c,0.20000000298023224,-14.803051948547363,0.8999999761581421,42.391395568847656,96.0,PV_NS_Node_3,0.8999999761581421,1.0,false,33f29587-f63e-45b7-960b-037bda37a3cb,,,,"cosPhiFixed:{(0.0,0.9)}",10.0,fd1a8de9-722a-4304-8799-e1e976d9979c diff --git a/input/samples/vn_simona_small_with_em/fullGrid/storage_input.csv b/input/samples/vn_simona_small_with_em/fullGrid/storage_input.csv deleted file mode 100644 index 35eb72ffac..0000000000 --- a/input/samples/vn_simona_small_with_em/fullGrid/storage_input.csv +++ /dev/null @@ -1,3 +0,0 @@ -uuid,id,node,operates_from,operates_until,operator,q_characteristics,type,em -a2a92cfd-3492-465f-9587-e789f4620af8,Speicher_3,33f29587-f63e-45b7-960b-037bda37a3cb,,,,"cosPhiFixed:{(0.0,0.98)}",95d4c980-d9e1-4813-9f2a-b0942488a570,fd1a8de9-722a-4304-8799-e1e976d9979c -c96f0a90-980a-4813-b175-ecc04aded773,Speicher_4,401f37f8-6f2c-4564-bc78-6736cb9cbf8d,,,,"cosPhiFixed:{(0.0,0.98)}",95d4c980-d9e1-4813-9f2a-b0942488a570,ff0b995a-86ff-4f4d-987e-e475a64f2180 diff --git a/input/samples/vn_simona_small_with_em/fullGrid/storage_type_input.csv b/input/samples/vn_simona_small_with_em/fullGrid/storage_type_input.csv deleted file mode 100644 index 31cce697f1..0000000000 --- a/input/samples/vn_simona_small_with_em/fullGrid/storage_type_input.csv +++ /dev/null @@ -1,2 +0,0 @@ -uuid,active_power_gradient,capex,cos_phi_rated,dod,e_storage,eta,id,life_cycle,life_time,opex,p_max,s_rated -95d4c980-d9e1-4813-9f2a-b0942488a570,1.0,0.0,0.96,8.0,16.0,93.0,Typ_1,5000,5000.0,0.65,4.0,4.166666666666667 diff --git a/input/samples/vn_simona_small_with_em/fullGrid/transformer_2_w_input.csv b/input/samples/vn_simona_small_with_em/fullGrid/transformer_2_w_input.csv deleted file mode 100755 index 9bb70380b6..0000000000 --- a/input/samples/vn_simona_small_with_em/fullGrid/transformer_2_w_input.csv +++ /dev/null @@ -1,2 +0,0 @@ -uuid,auto_tap,id,node_a,node_b,operates_from,operates_until,operator,parallel_devices,tap_pos,type -d2a3736d-d10b-4f38-88cf-871df43e3665,true,HöS-Trafo_S1,34cd8ee0-e607-4c47-89a7-121c3e32768a,00d03670-7833-47ee-ad52-04d18d1c64fd,,,,1,0,97735722-05cc-4ca8-8a8d-c08ac3ded19a diff --git a/input/samples/vn_simona_small_with_em/fullGrid/transformer_2_w_type_input.csv b/input/samples/vn_simona_small_with_em/fullGrid/transformer_2_w_type_input.csv deleted file mode 100755 index 0d2400e17e..0000000000 --- a/input/samples/vn_simona_small_with_em/fullGrid/transformer_2_w_type_input.csv +++ /dev/null @@ -1,2 +0,0 @@ -uuid,b_m,d_phi,d_v,g_m,id,r_sc,s_rated,tap_max,tap_min,tap_neutr,tap_side,v_rated_a,v_rated_b,x_sc -97735722-05cc-4ca8-8a8d-c08ac3ded19a,-36.47380569074435,0.0,2.5,4124.999999999999,0.63 MVA 20/0.4 kV Dyn5 ASEA,6.953892668178382,630.0,2,-2,0,false,20.0,0.4,37.45518044666632 diff --git a/input/samples/vn_simona_small_with_em/vn_simona_small_withem.conf b/input/samples/vn_simona_small_with_em/vn_simona_small_withem.conf deleted file mode 100755 index 6185ad6e4d..0000000000 --- a/input/samples/vn_simona_small_with_em/vn_simona_small_withem.conf +++ /dev/null @@ -1,210 +0,0 @@ -include "../common/pekko.conf" - -######### -# ATTENTION: Do not change this config file directly but use it as a base for your personal delta config for the -# vn_simona scenario! Delta configs can be created by including the config you want to change -# parameters from via include (e.g. include "input/samples/vn_simona/vn_simona.conf") at the -# beginning of your config file and then just override the parameters you want to change! -######### - -################################################################## -# Simulation Parameters -################################################################## -simona.simulationName = "simple_vn_simona_withem" - -################################################################## -# Time Parameters -################################################################## -simona.time.startDateTime = "2023-06-01T00:00:00Z" -simona.time.endDateTime = "2023-06-01T23:45:00Z" -simona.time.schedulerReadyCheckWindow = 900 - -################################################################## -# Input Parameters -################################################################## -simona.input.primary.csvParams = { - directoryPath: "simona/input/samples/vn_simona_small_with_em/fullGrid" - csvSep: "," - isHierarchic: false -} -simona.input.grid.datasource.id = "csv" -simona.input.grid.datasource.csvParams = { - directoryPath: "simona/input/samples/vn_simona_small_with_em/fullGrid" - csvSep: "," - isHierarchic: false -} - -simona.input.weather.datasource = { - scheme = "icon" - sampleParams.use = true - coordinateSource.sampleParams.use = true - maxCoordinateDistance = 50000 -} - -################################################################## -# Output Parameters -################################################################## -simona.output.base.dir = "simona/output/vn_simona_small_with_em" -simona.output.base.addTimestampToOutputDir = true - -simona.output.sink.csv { - fileFormat = ".csv" - filePrefix = "" - fileSuffix = "" -} - -simona.output.grid = { - notifier = "grid" - nodes = false - lines = false - switches = false - transformers2w = false - transformers3w = false -} -simona.output.participant.defaultConfig = { - notifier = "default" - powerRequestReply = false - simulationResult = true -} -simona.output.participant.individualConfigs = [ - { - notifier = "pv" - powerRequestReply = false - simulationResult = true - }, - { - notifier = "wec" - powerRequestReply = false - simulationResult = true - }, - { - notifier = "evcs" - powerRequestReply = false - simulationResult = true - } -] -simona.output.thermal = { - defaultConfig = { - notifier = "default", - simulationResult = false - } - individualConfigs = [ - { - notifier = "house", - simulationResult = true - } - ] -} - -################################################################## -# Runtime Configuration // todo refactor as this naming is misleading and partly unneeded -################################################################## -simona.runtime.selected_subgrids = [] -simona.runtime.selected_volt_lvls = [] - -simona.runtime.participant.load = { - defaultConfig = { - calculateMissingReactivePowerWithModel = false - uuids = ["default"] - scaling = 1.0 - modelBehaviour = "profile" - reference = "power" - } - individualConfigs = [] -} - -simona.runtime.participant.fixedFeedIn = { - defaultConfig = { - calculateMissingReactivePowerWithModel = false - uuids = ["default"] - scaling = 1.0 - } - individualConfigs = [] -} - -simona.runtime.participant.pv = { - defaultConfig = { - calculateMissingReactivePowerWithModel = false - uuids = ["default"] - scaling = 1.0 - } - individualConfigs = [] -} - -simona.runtime.participant.wec = { - defaultConfig = { - calculateMissingReactivePowerWithModel = false - uuids = ["default"] - scaling = 1.0 - } - individualConfigs = [] -} - -simona.runtime.participant.evcs = { - defaultConfig = { - calculateMissingReactivePowerWithModel = false - uuids = ["default"] - scaling = 1.0 - } - individualConfigs = [] -} - -simona.runtime.participant.hp = { - defaultConfig = { - calculateMissingReactivePowerWithModel = false - uuids = ["default"] - scaling = 1.0 - } - individualConfigs = [] -} - -# # # # # -# ATTENTION: calculateMissingReactivePowerWithModel and scaling is ignored here. -# # # # # -simona.runtime.participant.em = { - defaultConfig = { - calculateMissingReactivePowerWithModel = false - uuids = ["default"] - scaling = 1.0 - } - individualConfigs = [] -} - -################################################################## -# Event Configuration -################################################################## -simona.event.listener = [] - -################################################################## -# Grid Configuration -################################################################## - -simona.gridConfig.refSystems = [ - {sNom = "100 kVA", vNom = "0.4 kV", voltLvls = [{id = "LV", vNom = "0.4 kV"}]}, - {sNom = "60 MVA", vNom = "20 kV", voltLvls = [{id = "MV", vNom = "20 kV"}]}, - {sNom = "600 MVA", vNom = "110 kV", voltLvls = [{id = "HV", vNom = "110 kV"}]}, - {sNom = "1000 MVA", vNom = "380 kV", voltLvls = [{id = "EHV", vNom = "380 kV"}]} -] - -################################################################## -# Power Flow Configuration -################################################################## -simona.powerflow.maxSweepPowerDeviation = 1E-5 // the maximum allowed deviation in power between two sweeps, before overall convergence is assumed -simona.powerflow.newtonraphson.epsilon = [1E-12] -simona.powerflow.newtonraphson.iterations = 50 -simona.powerflow.resolution = "86400s" -simona.powerflow.stopOnFailure = true - -simona.control.transformer = [ - { - transformers = ["31a2b9bf-e785-4475-aa44-1c34646e8c79"], - measurements = ["923f2d69-3093-4198-86e4-13d2d1c220f8"], - vMin = 0.98, - vMax = 1.02 - }, { - transformers = ["1132dbf4-e8a1-44ae-8415-f42d4497aa1d"], - measurements = ["7686b818-a0ba-465c-8e4e-f7d3c4e171fc"], - vMin = 0.98, - vMax = 1.02 - } -] diff --git a/input/samples/vn_simona_small_with_em_original/fullGrid/em_input.csv b/input/samples/vn_simona_small_with_em_original/fullGrid/em_input.csv deleted file mode 100755 index 8f4ac1f81a..0000000000 --- a/input/samples/vn_simona_small_with_em_original/fullGrid/em_input.csv +++ /dev/null @@ -1,5 +0,0 @@ -uuid,control_strategy,parent_em,id,operates_from,operates_until,operator -fd1a8de9-722a-4304-8799-e1e976d9979c,self_optimization,c40d5386-d2ab-49f8-a1b4-02991b68f502,EM_NS_Node_3,,, -ff0b995a-86ff-4f4d-987e-e475a64f2180,self_optimization,5f4c7c32-0e14-4f88-b727-467f270624e2,EM_NS_Node_4,,, -c40d5386-d2ab-49f8-a1b4-02991b68f502,self_optimization,,EM_Ext_Interface_3,,, -5f4c7c32-0e14-4f88-b727-467f270624e2,self_optimization,,EM_Ext_Interface_4,,, \ No newline at end of file diff --git a/input/samples/vn_simona_small_with_em_original/fullGrid/fixed_feed_in_input.csv b/input/samples/vn_simona_small_with_em_original/fullGrid/fixed_feed_in_input.csv deleted file mode 100755 index 90a18373ca..0000000000 --- a/input/samples/vn_simona_small_with_em_original/fullGrid/fixed_feed_in_input.csv +++ /dev/null @@ -1,2 +0,0 @@ -uuid,cos_phi_rated,id,node,operates_from,operates_until,operator,q_characteristics,s_rated -9abe950d-362e-4efe-b686-500f84d8f368,0.9,feed_in_hs_s4,401f37f8-6f2c-4564-bc78-6736cb9cbf8d,,,,"cosPhiFixed:{(0.0,0.95)}",200.0 diff --git a/input/samples/vn_simona_small_with_em_original/fullGrid/line_input.csv b/input/samples/vn_simona_small_with_em_original/fullGrid/line_input.csv deleted file mode 100755 index 0f7b0204e7..0000000000 --- a/input/samples/vn_simona_small_with_em_original/fullGrid/line_input.csv +++ /dev/null @@ -1,6 +0,0 @@ -uuid,geo_position,id,length,node_a,node_b,olm_characteristic,operates_from,operates_until,operator,parallel_devices,type -fd04cc09-6ff8-48ea-a5eb-453e9d59d6d7,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_Node_1-NS_Node_3,0.04,00d03670-7833-47ee-ad52-04d18d1c64fd,33f29587-f63e-45b7-960b-037bda37a3cb,"olm:{(0.0,1.0)}",,,,1,9a8e9b63-af21-4c1b-8db7-fc2924f9610e -b5a66c60-7189-4c86-a32a-4d1aa6568475,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_Node_4-NS_Node_1,0.030304,401f37f8-6f2c-4564-bc78-6736cb9cbf8d,00d03670-7833-47ee-ad52-04d18d1c64fd,"olm:{(0.0,1.0)}",,,,1,9a8e9b63-af21-4c1b-8db7-fc2924f9610e -42828cac-b67e-4d5c-b4fa-787c57f16fde,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_Node_3-NS_Node_2,0.0251089,33f29587-f63e-45b7-960b-037bda37a3cb,dfae9806-9b44-4995-ba27-d66d8e4a43e0,"olm:{(0.0,1.0)}",,,,1,9a8e9b63-af21-4c1b-8db7-fc2924f9610e -997840bf-8c94-444f-83f1-e9c991706d7c,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_Node_1-NS_Node_2,0.0377768,00d03670-7833-47ee-ad52-04d18d1c64fd,dfae9806-9b44-4995-ba27-d66d8e4a43e0,"olm:{(0.0,1.0)}",,,,1,9a8e9b63-af21-4c1b-8db7-fc2924f9610e -81f4c585-6170-4a9e-981f-2185a0d7f2ec,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_Node_3-NS_Node_4,0.031,33f29587-f63e-45b7-960b-037bda37a3cb,401f37f8-6f2c-4564-bc78-6736cb9cbf8d,"olm:{(0.0,1.0)}",,,,1,9a8e9b63-af21-4c1b-8db7-fc2924f9610e diff --git a/input/samples/vn_simona_small_with_em_original/fullGrid/line_type_input.csv b/input/samples/vn_simona_small_with_em_original/fullGrid/line_type_input.csv deleted file mode 100755 index 91202b2778..0000000000 --- a/input/samples/vn_simona_small_with_em_original/fullGrid/line_type_input.csv +++ /dev/null @@ -1,2 +0,0 @@ -uuid,b,g,i_max,id,r,v_rated,x -9a8e9b63-af21-4c1b-8db7-fc2924f9610e,273.31899999999996,0.0,357.0,NAYY 4x240SE 0.6/1kV,0.1267,0.4,0.0797965 diff --git a/input/samples/vn_simona_small_with_em_original/fullGrid/load_input.csv b/input/samples/vn_simona_small_with_em_original/fullGrid/load_input.csv deleted file mode 100755 index 4d0adff998..0000000000 --- a/input/samples/vn_simona_small_with_em_original/fullGrid/load_input.csv +++ /dev/null @@ -1,5 +0,0 @@ -uuid,cos_phi_rated,dsm,e_cons_annual,id,load_profile,node,operates_from,operates_until,operator,q_characteristics,s_rated -4dca3b1d-5d24-444a-b4df-f4fa23b9ef1b,0.949999988079071,false,4000.0,LOAD_NS_Node_1,h0,00d03670-7833-47ee-ad52-04d18d1c64fd,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605 -9c5991bc-24df-496b-b4ce-5ec27657454c,0.949999988079071,false,4000.0,LOAD_NS_Node_2,h0,dfae9806-9b44-4995-ba27-d66d8e4a43e0,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605 -58b9f934-f7c4-4335-9894-3c80d9e6b852,0.949999988079071,false,4000.0,LOAD_NS_Node_3,h0,33f29587-f63e-45b7-960b-037bda37a3cb,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605 -283a1252-a774-4b04-bfcf-fe8879065982,0.949999988079071,false,4000.0,LOAD_NS_Node_4,h0,401f37f8-6f2c-4564-bc78-6736cb9cbf8d,,,,"cosPhiFixed:{(0.0,1.0)}",2.3157899379730225 diff --git a/input/samples/vn_simona_small_with_em_original/fullGrid/node_input.csv b/input/samples/vn_simona_small_with_em_original/fullGrid/node_input.csv deleted file mode 100755 index 21772153f5..0000000000 --- a/input/samples/vn_simona_small_with_em_original/fullGrid/node_input.csv +++ /dev/null @@ -1,6 +0,0 @@ -uuid,geo_position,id,operates_from,operates_until,operator,slack,subnet,v_rated,v_target,volt_lvl -00d03670-7833-47ee-ad52-04d18d1c64fd,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",NS_Node_1,,,,false,2,0.4,1.0,Niederspannung -dfae9806-9b44-4995-ba27-d66d8e4a43e0,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}",NS_Node_2,,,,false,2,0.4,1.0,Niederspannung -34cd8ee0-e607-4c47-89a7-121c3e32768a,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",MS_Node_0,,,,true,1,20.0,1.025,Mittelspannung -401f37f8-6f2c-4564-bc78-6736cb9cbf8d,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}",NS_Node_4,,,,false,2,0.4,1.0,Niederspannung -33f29587-f63e-45b7-960b-037bda37a3cb,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",NS_Node_3,,,,false,2,0.4,1.0,Niederspannung diff --git a/input/samples/vn_simona_small_with_em_original/fullGrid/pv_input.csv b/input/samples/vn_simona_small_with_em_original/fullGrid/pv_input.csv deleted file mode 100644 index e418358125..0000000000 --- a/input/samples/vn_simona_small_with_em_original/fullGrid/pv_input.csv +++ /dev/null @@ -1,3 +0,0 @@ -uuid,albedo,azimuth,cos_phi_rated,elevation_angle,eta_conv,id,k_g,k_t,market_reaction,node,operates_from,operates_until,operator,q_characteristics,s_rated,em -a1eb7fc1-3bee-4b65-a387-ef3046644bf0,0.20000000298023224,-8.999500274658203,0.8999999761581421,37.14517593383789,98.0,PV_NS_Node_4,0.8999999761581421,1.0,false,401f37f8-6f2c-4564-bc78-6736cb9cbf8d,,,,"cosPhiFixed:{(0.0,0.9)}",10.0,ff0b995a-86ff-4f4d-987e-e475a64f2180 -9d7cd8e2-d859-4f4f-9c01-abba06ef2e2c,0.20000000298023224,-14.803051948547363,0.8999999761581421,42.391395568847656,96.0,PV_NS_Node_3,0.8999999761581421,1.0,false,33f29587-f63e-45b7-960b-037bda37a3cb,,,,"cosPhiFixed:{(0.0,0.9)}",10.0,fd1a8de9-722a-4304-8799-e1e976d9979c diff --git a/input/samples/vn_simona_small_with_em_original/fullGrid/storage_input.csv b/input/samples/vn_simona_small_with_em_original/fullGrid/storage_input.csv deleted file mode 100644 index 35eb72ffac..0000000000 --- a/input/samples/vn_simona_small_with_em_original/fullGrid/storage_input.csv +++ /dev/null @@ -1,3 +0,0 @@ -uuid,id,node,operates_from,operates_until,operator,q_characteristics,type,em -a2a92cfd-3492-465f-9587-e789f4620af8,Speicher_3,33f29587-f63e-45b7-960b-037bda37a3cb,,,,"cosPhiFixed:{(0.0,0.98)}",95d4c980-d9e1-4813-9f2a-b0942488a570,fd1a8de9-722a-4304-8799-e1e976d9979c -c96f0a90-980a-4813-b175-ecc04aded773,Speicher_4,401f37f8-6f2c-4564-bc78-6736cb9cbf8d,,,,"cosPhiFixed:{(0.0,0.98)}",95d4c980-d9e1-4813-9f2a-b0942488a570,ff0b995a-86ff-4f4d-987e-e475a64f2180 diff --git a/input/samples/vn_simona_small_with_em_original/fullGrid/storage_type_input.csv b/input/samples/vn_simona_small_with_em_original/fullGrid/storage_type_input.csv deleted file mode 100644 index 31cce697f1..0000000000 --- a/input/samples/vn_simona_small_with_em_original/fullGrid/storage_type_input.csv +++ /dev/null @@ -1,2 +0,0 @@ -uuid,active_power_gradient,capex,cos_phi_rated,dod,e_storage,eta,id,life_cycle,life_time,opex,p_max,s_rated -95d4c980-d9e1-4813-9f2a-b0942488a570,1.0,0.0,0.96,8.0,16.0,93.0,Typ_1,5000,5000.0,0.65,4.0,4.166666666666667 diff --git a/input/samples/vn_simona_small_with_em_original/fullGrid/transformer_2_w_input.csv b/input/samples/vn_simona_small_with_em_original/fullGrid/transformer_2_w_input.csv deleted file mode 100755 index 9bb70380b6..0000000000 --- a/input/samples/vn_simona_small_with_em_original/fullGrid/transformer_2_w_input.csv +++ /dev/null @@ -1,2 +0,0 @@ -uuid,auto_tap,id,node_a,node_b,operates_from,operates_until,operator,parallel_devices,tap_pos,type -d2a3736d-d10b-4f38-88cf-871df43e3665,true,HöS-Trafo_S1,34cd8ee0-e607-4c47-89a7-121c3e32768a,00d03670-7833-47ee-ad52-04d18d1c64fd,,,,1,0,97735722-05cc-4ca8-8a8d-c08ac3ded19a diff --git a/input/samples/vn_simona_small_with_em_original/fullGrid/transformer_2_w_type_input.csv b/input/samples/vn_simona_small_with_em_original/fullGrid/transformer_2_w_type_input.csv deleted file mode 100755 index 0d2400e17e..0000000000 --- a/input/samples/vn_simona_small_with_em_original/fullGrid/transformer_2_w_type_input.csv +++ /dev/null @@ -1,2 +0,0 @@ -uuid,b_m,d_phi,d_v,g_m,id,r_sc,s_rated,tap_max,tap_min,tap_neutr,tap_side,v_rated_a,v_rated_b,x_sc -97735722-05cc-4ca8-8a8d-c08ac3ded19a,-36.47380569074435,0.0,2.5,4124.999999999999,0.63 MVA 20/0.4 kV Dyn5 ASEA,6.953892668178382,630.0,2,-2,0,false,20.0,0.4,37.45518044666632 diff --git a/input/samples/vn_simona_small_with_em_original/vn_simona_small_withem.conf b/input/samples/vn_simona_small_with_em_original/vn_simona_small_withem.conf deleted file mode 100755 index 65aa0efdf6..0000000000 --- a/input/samples/vn_simona_small_with_em_original/vn_simona_small_withem.conf +++ /dev/null @@ -1,210 +0,0 @@ -include "../common/pekko.conf" - -######### -# ATTENTION: Do not change this config file directly but use it as a base for your personal delta config for the -# vn_simona scenario! Delta configs can be created by including the config you want to change -# parameters from via include (e.g. include "input/samples/vn_simona/vn_simona.conf") at the -# beginning of your config file and then just override the parameters you want to change! -######### - -################################################################## -# Simulation Parameters -################################################################## -simona.simulationName = "simple_vn_simona_withem" - -################################################################## -# Time Parameters -################################################################## -simona.time.startDateTime = "2024-01-01T12:00:00Z" -simona.time.endDateTime = "2024-01-01T13:00:00Z" -simona.time.schedulerReadyCheckWindow = 900 - -################################################################## -# Input Parameters -################################################################## -simona.input.primary.csvParams = { - directoryPath: "simona/input/samples/vn_simona_small_with_em/fullGrid" - csvSep: "," - isHierarchic: false -} -simona.input.grid.datasource.id = "csv" -simona.input.grid.datasource.csvParams = { - directoryPath: "simona/input/samples/vn_simona_small_with_em/fullGrid" - csvSep: "," - isHierarchic: false -} - -simona.input.weather.datasource = { - scheme = "icon" - sampleParams.use = true - coordinateSource.sampleParams.use = true - maxCoordinateDistance = 50000 -} - -################################################################## -# Output Parameters -################################################################## -simona.output.base.dir = "simona/output/vn_simona_small_with_em" -simona.output.base.addTimestampToOutputDir = true - -simona.output.sink.csv { - fileFormat = ".csv" - filePrefix = "" - fileSuffix = "" -} - -simona.output.grid = { - notifier = "grid" - nodes = false - lines = false - switches = false - transformers2w = false - transformers3w = false -} -simona.output.participant.defaultConfig = { - notifier = "default" - powerRequestReply = false - simulationResult = true -} -simona.output.participant.individualConfigs = [ - { - notifier = "pv" - powerRequestReply = false - simulationResult = true - }, - { - notifier = "wec" - powerRequestReply = false - simulationResult = true - }, - { - notifier = "evcs" - powerRequestReply = false - simulationResult = true - } -] -simona.output.thermal = { - defaultConfig = { - notifier = "default", - simulationResult = false - } - individualConfigs = [ - { - notifier = "house", - simulationResult = true - } - ] -} - -################################################################## -# Runtime Configuration // todo refactor as this naming is misleading and partly unneeded -################################################################## -simona.runtime.selected_subgrids = [] -simona.runtime.selected_volt_lvls = [] - -simona.runtime.participant.load = { - defaultConfig = { - calculateMissingReactivePowerWithModel = false - uuids = ["default"] - scaling = 1.0 - modelBehaviour = "fix" - reference = "power" - } - individualConfigs = [] -} - -simona.runtime.participant.fixedFeedIn = { - defaultConfig = { - calculateMissingReactivePowerWithModel = false - uuids = ["default"] - scaling = 1.0 - } - individualConfigs = [] -} - -simona.runtime.participant.pv = { - defaultConfig = { - calculateMissingReactivePowerWithModel = false - uuids = ["default"] - scaling = 1.0 - } - individualConfigs = [] -} - -simona.runtime.participant.wec = { - defaultConfig = { - calculateMissingReactivePowerWithModel = false - uuids = ["default"] - scaling = 1.0 - } - individualConfigs = [] -} - -simona.runtime.participant.evcs = { - defaultConfig = { - calculateMissingReactivePowerWithModel = false - uuids = ["default"] - scaling = 1.0 - } - individualConfigs = [] -} - -simona.runtime.participant.hp = { - defaultConfig = { - calculateMissingReactivePowerWithModel = false - uuids = ["default"] - scaling = 1.0 - } - individualConfigs = [] -} - -# # # # # -# ATTENTION: calculateMissingReactivePowerWithModel and scaling is ignored here. -# # # # # -simona.runtime.participant.em = { - defaultConfig = { - calculateMissingReactivePowerWithModel = false - uuids = ["default"] - scaling = 1.0 - } - individualConfigs = [] -} - -################################################################## -# Event Configuration -################################################################## -simona.event.listener = [] - -################################################################## -# Grid Configuration -################################################################## - -simona.gridConfig.refSystems = [ - {sNom = "100 kVA", vNom = "0.4 kV", voltLvls = [{id = "LV", vNom = "0.4 kV"}]}, - {sNom = "60 MVA", vNom = "20 kV", voltLvls = [{id = "MV", vNom = "20 kV"}]}, - {sNom = "600 MVA", vNom = "110 kV", voltLvls = [{id = "HV", vNom = "110 kV"}]}, - {sNom = "1000 MVA", vNom = "380 kV", voltLvls = [{id = "EHV", vNom = "380 kV"}]} -] - -################################################################## -# Power Flow Configuration -################################################################## -simona.powerflow.maxSweepPowerDeviation = 1E-5 // the maximum allowed deviation in power between two sweeps, before overall convergence is assumed -simona.powerflow.newtonraphson.epsilon = [1E-12] -simona.powerflow.newtonraphson.iterations = 50 -simona.powerflow.resolution = "3600s" -simona.powerflow.stopOnFailure = true - -simona.control.transformer = [ - { - transformers = ["31a2b9bf-e785-4475-aa44-1c34646e8c79"], - measurements = ["923f2d69-3093-4198-86e4-13d2d1c220f8"], - vMin = 0.98, - vMax = 1.02 - }, { - transformers = ["1132dbf4-e8a1-44ae-8415-f42d4497aa1d"], - measurements = ["7686b818-a0ba-465c-8e4e-f7d3c4e171fc"], - vMin = 0.98, - vMax = 1.02 - } -] From cbedc48b5f1a904eee0d13a0dc556e963bbb9b52 Mon Sep 17 00:00:00 2001 From: Johannes Bao Date: Wed, 5 Jun 2024 13:17:30 +0200 Subject: [PATCH 37/41] gitignore --- .gitignore | 6 ------ 1 file changed, 6 deletions(-) diff --git a/.gitignore b/.gitignore index 5bebd61aa7..7cf28aed43 100644 --- a/.gitignore +++ b/.gitignore @@ -235,9 +235,3 @@ build # Ignore unwanted input directories input/ext_sim -/input/samples/simopsimtestgrid_reduced/ -/input/samples/simopsimtestgrid_reduced_reference/ -/input/samples/simopsimtestgrid_reduced_withoutem/ -/input/samples/vn_simona_ns/ -/input/samples/vn_simona_small_with_em/ -/input/samples/vn_simona_small_with_em_original/ From a2d01747a7bd319360b5f2f1c64f62ccbd1a00f9 Mon Sep 17 00:00:00 2001 From: Johannes Bao Date: Fri, 7 Jun 2024 14:18:28 +0200 Subject: [PATCH 38/41] modified setup --- .../fullGrid/em_input.csv | 3 + .../fullGrid/fixed_feed_in_input.csv | 2 + .../fullGrid/line_input.csv | 6 + .../fullGrid/line_type_input.csv | 2 + .../fullGrid/load_input.csv | 5 + .../fullGrid/node_input.csv | 6 + .../fullGrid/pv_input.csv | 3 + .../fullGrid/storage_input.csv | 3 + .../fullGrid/storage_type_input.csv | 2 + .../fullGrid/transformer_2_w_input.csv | 2 + .../fullGrid/transformer_2_w_type_input.csv | 2 + .../vn_simona_small_withem.conf | 210 ++++++ .../edu/ie3/simona/agent/em/EmAgent.scala | 4 +- .../ParticipantAgentFundamentals.scala | 1 + .../edu/ie3/simona/event/ResultEvent.scala | 2 + .../event/listener/ResultEventListener.scala | 20 +- .../simona/service/em/ExtEmDataService.scala | 20 +- .../results/ExtResultDataProvider.scala | 204 +++--- .../simona/sim/setup/SimonaExtSimSetup.scala | 537 +++++++++++++++ .../simona/sim/setup/SimonaMosaikSetup.scala | 604 +---------------- .../simona/sim/setup/SimonaOpsimSetup.scala | 615 +----------------- .../SimonaSimpleExtSimulationSetup.scala | 549 +--------------- .../sim/setup/SimonaStandaloneSetup.scala | 3 +- 23 files changed, 995 insertions(+), 1810 deletions(-) create mode 100644 input/samples/vn_simona_small_with_em/fullGrid/em_input.csv create mode 100644 input/samples/vn_simona_small_with_em/fullGrid/fixed_feed_in_input.csv create mode 100644 input/samples/vn_simona_small_with_em/fullGrid/line_input.csv create mode 100644 input/samples/vn_simona_small_with_em/fullGrid/line_type_input.csv create mode 100644 input/samples/vn_simona_small_with_em/fullGrid/load_input.csv create mode 100644 input/samples/vn_simona_small_with_em/fullGrid/node_input.csv create mode 100644 input/samples/vn_simona_small_with_em/fullGrid/pv_input.csv create mode 100644 input/samples/vn_simona_small_with_em/fullGrid/storage_input.csv create mode 100644 input/samples/vn_simona_small_with_em/fullGrid/storage_type_input.csv create mode 100644 input/samples/vn_simona_small_with_em/fullGrid/transformer_2_w_input.csv create mode 100644 input/samples/vn_simona_small_with_em/fullGrid/transformer_2_w_type_input.csv create mode 100644 input/samples/vn_simona_small_with_em/vn_simona_small_withem.conf create mode 100644 src/main/scala/edu/ie3/simona/sim/setup/SimonaExtSimSetup.scala diff --git a/input/samples/vn_simona_small_with_em/fullGrid/em_input.csv b/input/samples/vn_simona_small_with_em/fullGrid/em_input.csv new file mode 100644 index 0000000000..22729f2b61 --- /dev/null +++ b/input/samples/vn_simona_small_with_em/fullGrid/em_input.csv @@ -0,0 +1,3 @@ +uuid,control_strategy,parent_em,id,operates_from,operates_until,operator +fd1a8de9-722a-4304-8799-e1e976d9979c,self_optimization,,EM_NS_Node_3,,, +ff0b995a-86ff-4f4d-987e-e475a64f2180,self_optimization,,EM_NS_Node_4,,, \ No newline at end of file diff --git a/input/samples/vn_simona_small_with_em/fullGrid/fixed_feed_in_input.csv b/input/samples/vn_simona_small_with_em/fullGrid/fixed_feed_in_input.csv new file mode 100644 index 0000000000..90a18373ca --- /dev/null +++ b/input/samples/vn_simona_small_with_em/fullGrid/fixed_feed_in_input.csv @@ -0,0 +1,2 @@ +uuid,cos_phi_rated,id,node,operates_from,operates_until,operator,q_characteristics,s_rated +9abe950d-362e-4efe-b686-500f84d8f368,0.9,feed_in_hs_s4,401f37f8-6f2c-4564-bc78-6736cb9cbf8d,,,,"cosPhiFixed:{(0.0,0.95)}",200.0 diff --git a/input/samples/vn_simona_small_with_em/fullGrid/line_input.csv b/input/samples/vn_simona_small_with_em/fullGrid/line_input.csv new file mode 100644 index 0000000000..0f7b0204e7 --- /dev/null +++ b/input/samples/vn_simona_small_with_em/fullGrid/line_input.csv @@ -0,0 +1,6 @@ +uuid,geo_position,id,length,node_a,node_b,olm_characteristic,operates_from,operates_until,operator,parallel_devices,type +fd04cc09-6ff8-48ea-a5eb-453e9d59d6d7,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_Node_1-NS_Node_3,0.04,00d03670-7833-47ee-ad52-04d18d1c64fd,33f29587-f63e-45b7-960b-037bda37a3cb,"olm:{(0.0,1.0)}",,,,1,9a8e9b63-af21-4c1b-8db7-fc2924f9610e +b5a66c60-7189-4c86-a32a-4d1aa6568475,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_Node_4-NS_Node_1,0.030304,401f37f8-6f2c-4564-bc78-6736cb9cbf8d,00d03670-7833-47ee-ad52-04d18d1c64fd,"olm:{(0.0,1.0)}",,,,1,9a8e9b63-af21-4c1b-8db7-fc2924f9610e +42828cac-b67e-4d5c-b4fa-787c57f16fde,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_Node_3-NS_Node_2,0.0251089,33f29587-f63e-45b7-960b-037bda37a3cb,dfae9806-9b44-4995-ba27-d66d8e4a43e0,"olm:{(0.0,1.0)}",,,,1,9a8e9b63-af21-4c1b-8db7-fc2924f9610e +997840bf-8c94-444f-83f1-e9c991706d7c,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_Node_1-NS_Node_2,0.0377768,00d03670-7833-47ee-ad52-04d18d1c64fd,dfae9806-9b44-4995-ba27-d66d8e4a43e0,"olm:{(0.0,1.0)}",,,,1,9a8e9b63-af21-4c1b-8db7-fc2924f9610e +81f4c585-6170-4a9e-981f-2185a0d7f2ec,"{""type"":""LineString"",""coordinates"":[[7.4116482,51.4843281],[7.4116482,51.4843281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",LTG_NS_Node_3-NS_Node_4,0.031,33f29587-f63e-45b7-960b-037bda37a3cb,401f37f8-6f2c-4564-bc78-6736cb9cbf8d,"olm:{(0.0,1.0)}",,,,1,9a8e9b63-af21-4c1b-8db7-fc2924f9610e diff --git a/input/samples/vn_simona_small_with_em/fullGrid/line_type_input.csv b/input/samples/vn_simona_small_with_em/fullGrid/line_type_input.csv new file mode 100644 index 0000000000..91202b2778 --- /dev/null +++ b/input/samples/vn_simona_small_with_em/fullGrid/line_type_input.csv @@ -0,0 +1,2 @@ +uuid,b,g,i_max,id,r,v_rated,x +9a8e9b63-af21-4c1b-8db7-fc2924f9610e,273.31899999999996,0.0,357.0,NAYY 4x240SE 0.6/1kV,0.1267,0.4,0.0797965 diff --git a/input/samples/vn_simona_small_with_em/fullGrid/load_input.csv b/input/samples/vn_simona_small_with_em/fullGrid/load_input.csv new file mode 100644 index 0000000000..2324353168 --- /dev/null +++ b/input/samples/vn_simona_small_with_em/fullGrid/load_input.csv @@ -0,0 +1,5 @@ +uuid,cos_phi_rated,dsm,e_cons_annual,id,load_profile,node,operates_from,operates_until,operator,q_characteristics,s_rated,em +4dca3b1d-5d24-444a-b4df-f4fa23b9ef1b,0.949999988079071,false,4000.0,LOAD_NS_Node_1,h0,00d03670-7833-47ee-ad52-04d18d1c64fd,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605, +9c5991bc-24df-496b-b4ce-5ec27657454c,0.949999988079071,false,4000.0,LOAD_NS_Node_2,h0,dfae9806-9b44-4995-ba27-d66d8e4a43e0,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605, +58b9f934-f7c4-4335-9894-3c80d9e6b852,0.949999988079071,false,4000.0,LOAD_NS_Node_3,h0,33f29587-f63e-45b7-960b-037bda37a3cb,,,,"cosPhiFixed:{(0.0,1.0)}",4.1237101554870605,fd1a8de9-722a-4304-8799-e1e976d9979c +283a1252-a774-4b04-bfcf-fe8879065982,0.949999988079071,false,4000.0,LOAD_NS_Node_4,h0,401f37f8-6f2c-4564-bc78-6736cb9cbf8d,,,,"cosPhiFixed:{(0.0,1.0)}",2.3157899379730225,ff0b995a-86ff-4f4d-987e-e475a64f2180 diff --git a/input/samples/vn_simona_small_with_em/fullGrid/node_input.csv b/input/samples/vn_simona_small_with_em/fullGrid/node_input.csv new file mode 100644 index 0000000000..21772153f5 --- /dev/null +++ b/input/samples/vn_simona_small_with_em/fullGrid/node_input.csv @@ -0,0 +1,6 @@ +uuid,geo_position,id,operates_from,operates_until,operator,slack,subnet,v_rated,v_target,volt_lvl +00d03670-7833-47ee-ad52-04d18d1c64fd,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",NS_Node_1,,,,false,2,0.4,1.0,Niederspannung +dfae9806-9b44-4995-ba27-d66d8e4a43e0,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}",NS_Node_2,,,,false,2,0.4,1.0,Niederspannung +34cd8ee0-e607-4c47-89a7-121c3e32768a,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",MS_Node_0,,,,true,1,20.0,1.025,Mittelspannung +401f37f8-6f2c-4564-bc78-6736cb9cbf8d,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:0""}}}",NS_Node_4,,,,false,2,0.4,1.0,Niederspannung +33f29587-f63e-45b7-960b-037bda37a3cb,"{""type"":""Point"",""coordinates"":[7.4116482,51.4843281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}",NS_Node_3,,,,false,2,0.4,1.0,Niederspannung diff --git a/input/samples/vn_simona_small_with_em/fullGrid/pv_input.csv b/input/samples/vn_simona_small_with_em/fullGrid/pv_input.csv new file mode 100644 index 0000000000..e418358125 --- /dev/null +++ b/input/samples/vn_simona_small_with_em/fullGrid/pv_input.csv @@ -0,0 +1,3 @@ +uuid,albedo,azimuth,cos_phi_rated,elevation_angle,eta_conv,id,k_g,k_t,market_reaction,node,operates_from,operates_until,operator,q_characteristics,s_rated,em +a1eb7fc1-3bee-4b65-a387-ef3046644bf0,0.20000000298023224,-8.999500274658203,0.8999999761581421,37.14517593383789,98.0,PV_NS_Node_4,0.8999999761581421,1.0,false,401f37f8-6f2c-4564-bc78-6736cb9cbf8d,,,,"cosPhiFixed:{(0.0,0.9)}",10.0,ff0b995a-86ff-4f4d-987e-e475a64f2180 +9d7cd8e2-d859-4f4f-9c01-abba06ef2e2c,0.20000000298023224,-14.803051948547363,0.8999999761581421,42.391395568847656,96.0,PV_NS_Node_3,0.8999999761581421,1.0,false,33f29587-f63e-45b7-960b-037bda37a3cb,,,,"cosPhiFixed:{(0.0,0.9)}",10.0,fd1a8de9-722a-4304-8799-e1e976d9979c diff --git a/input/samples/vn_simona_small_with_em/fullGrid/storage_input.csv b/input/samples/vn_simona_small_with_em/fullGrid/storage_input.csv new file mode 100644 index 0000000000..35eb72ffac --- /dev/null +++ b/input/samples/vn_simona_small_with_em/fullGrid/storage_input.csv @@ -0,0 +1,3 @@ +uuid,id,node,operates_from,operates_until,operator,q_characteristics,type,em +a2a92cfd-3492-465f-9587-e789f4620af8,Speicher_3,33f29587-f63e-45b7-960b-037bda37a3cb,,,,"cosPhiFixed:{(0.0,0.98)}",95d4c980-d9e1-4813-9f2a-b0942488a570,fd1a8de9-722a-4304-8799-e1e976d9979c +c96f0a90-980a-4813-b175-ecc04aded773,Speicher_4,401f37f8-6f2c-4564-bc78-6736cb9cbf8d,,,,"cosPhiFixed:{(0.0,0.98)}",95d4c980-d9e1-4813-9f2a-b0942488a570,ff0b995a-86ff-4f4d-987e-e475a64f2180 diff --git a/input/samples/vn_simona_small_with_em/fullGrid/storage_type_input.csv b/input/samples/vn_simona_small_with_em/fullGrid/storage_type_input.csv new file mode 100644 index 0000000000..31cce697f1 --- /dev/null +++ b/input/samples/vn_simona_small_with_em/fullGrid/storage_type_input.csv @@ -0,0 +1,2 @@ +uuid,active_power_gradient,capex,cos_phi_rated,dod,e_storage,eta,id,life_cycle,life_time,opex,p_max,s_rated +95d4c980-d9e1-4813-9f2a-b0942488a570,1.0,0.0,0.96,8.0,16.0,93.0,Typ_1,5000,5000.0,0.65,4.0,4.166666666666667 diff --git a/input/samples/vn_simona_small_with_em/fullGrid/transformer_2_w_input.csv b/input/samples/vn_simona_small_with_em/fullGrid/transformer_2_w_input.csv new file mode 100644 index 0000000000..9bb70380b6 --- /dev/null +++ b/input/samples/vn_simona_small_with_em/fullGrid/transformer_2_w_input.csv @@ -0,0 +1,2 @@ +uuid,auto_tap,id,node_a,node_b,operates_from,operates_until,operator,parallel_devices,tap_pos,type +d2a3736d-d10b-4f38-88cf-871df43e3665,true,HöS-Trafo_S1,34cd8ee0-e607-4c47-89a7-121c3e32768a,00d03670-7833-47ee-ad52-04d18d1c64fd,,,,1,0,97735722-05cc-4ca8-8a8d-c08ac3ded19a diff --git a/input/samples/vn_simona_small_with_em/fullGrid/transformer_2_w_type_input.csv b/input/samples/vn_simona_small_with_em/fullGrid/transformer_2_w_type_input.csv new file mode 100644 index 0000000000..0d2400e17e --- /dev/null +++ b/input/samples/vn_simona_small_with_em/fullGrid/transformer_2_w_type_input.csv @@ -0,0 +1,2 @@ +uuid,b_m,d_phi,d_v,g_m,id,r_sc,s_rated,tap_max,tap_min,tap_neutr,tap_side,v_rated_a,v_rated_b,x_sc +97735722-05cc-4ca8-8a8d-c08ac3ded19a,-36.47380569074435,0.0,2.5,4124.999999999999,0.63 MVA 20/0.4 kV Dyn5 ASEA,6.953892668178382,630.0,2,-2,0,false,20.0,0.4,37.45518044666632 diff --git a/input/samples/vn_simona_small_with_em/vn_simona_small_withem.conf b/input/samples/vn_simona_small_with_em/vn_simona_small_withem.conf new file mode 100644 index 0000000000..6185ad6e4d --- /dev/null +++ b/input/samples/vn_simona_small_with_em/vn_simona_small_withem.conf @@ -0,0 +1,210 @@ +include "../common/pekko.conf" + +######### +# ATTENTION: Do not change this config file directly but use it as a base for your personal delta config for the +# vn_simona scenario! Delta configs can be created by including the config you want to change +# parameters from via include (e.g. include "input/samples/vn_simona/vn_simona.conf") at the +# beginning of your config file and then just override the parameters you want to change! +######### + +################################################################## +# Simulation Parameters +################################################################## +simona.simulationName = "simple_vn_simona_withem" + +################################################################## +# Time Parameters +################################################################## +simona.time.startDateTime = "2023-06-01T00:00:00Z" +simona.time.endDateTime = "2023-06-01T23:45:00Z" +simona.time.schedulerReadyCheckWindow = 900 + +################################################################## +# Input Parameters +################################################################## +simona.input.primary.csvParams = { + directoryPath: "simona/input/samples/vn_simona_small_with_em/fullGrid" + csvSep: "," + isHierarchic: false +} +simona.input.grid.datasource.id = "csv" +simona.input.grid.datasource.csvParams = { + directoryPath: "simona/input/samples/vn_simona_small_with_em/fullGrid" + csvSep: "," + isHierarchic: false +} + +simona.input.weather.datasource = { + scheme = "icon" + sampleParams.use = true + coordinateSource.sampleParams.use = true + maxCoordinateDistance = 50000 +} + +################################################################## +# Output Parameters +################################################################## +simona.output.base.dir = "simona/output/vn_simona_small_with_em" +simona.output.base.addTimestampToOutputDir = true + +simona.output.sink.csv { + fileFormat = ".csv" + filePrefix = "" + fileSuffix = "" +} + +simona.output.grid = { + notifier = "grid" + nodes = false + lines = false + switches = false + transformers2w = false + transformers3w = false +} +simona.output.participant.defaultConfig = { + notifier = "default" + powerRequestReply = false + simulationResult = true +} +simona.output.participant.individualConfigs = [ + { + notifier = "pv" + powerRequestReply = false + simulationResult = true + }, + { + notifier = "wec" + powerRequestReply = false + simulationResult = true + }, + { + notifier = "evcs" + powerRequestReply = false + simulationResult = true + } +] +simona.output.thermal = { + defaultConfig = { + notifier = "default", + simulationResult = false + } + individualConfigs = [ + { + notifier = "house", + simulationResult = true + } + ] +} + +################################################################## +# Runtime Configuration // todo refactor as this naming is misleading and partly unneeded +################################################################## +simona.runtime.selected_subgrids = [] +simona.runtime.selected_volt_lvls = [] + +simona.runtime.participant.load = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + modelBehaviour = "profile" + reference = "power" + } + individualConfigs = [] +} + +simona.runtime.participant.fixedFeedIn = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +simona.runtime.participant.pv = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +simona.runtime.participant.wec = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +simona.runtime.participant.evcs = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +simona.runtime.participant.hp = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +# # # # # +# ATTENTION: calculateMissingReactivePowerWithModel and scaling is ignored here. +# # # # # +simona.runtime.participant.em = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +################################################################## +# Event Configuration +################################################################## +simona.event.listener = [] + +################################################################## +# Grid Configuration +################################################################## + +simona.gridConfig.refSystems = [ + {sNom = "100 kVA", vNom = "0.4 kV", voltLvls = [{id = "LV", vNom = "0.4 kV"}]}, + {sNom = "60 MVA", vNom = "20 kV", voltLvls = [{id = "MV", vNom = "20 kV"}]}, + {sNom = "600 MVA", vNom = "110 kV", voltLvls = [{id = "HV", vNom = "110 kV"}]}, + {sNom = "1000 MVA", vNom = "380 kV", voltLvls = [{id = "EHV", vNom = "380 kV"}]} +] + +################################################################## +# Power Flow Configuration +################################################################## +simona.powerflow.maxSweepPowerDeviation = 1E-5 // the maximum allowed deviation in power between two sweeps, before overall convergence is assumed +simona.powerflow.newtonraphson.epsilon = [1E-12] +simona.powerflow.newtonraphson.iterations = 50 +simona.powerflow.resolution = "86400s" +simona.powerflow.stopOnFailure = true + +simona.control.transformer = [ + { + transformers = ["31a2b9bf-e785-4475-aa44-1c34646e8c79"], + measurements = ["923f2d69-3093-4198-86e4-13d2d1c220f8"], + vMin = 0.98, + vMax = 1.02 + }, { + transformers = ["1132dbf4-e8a1-44ae-8415-f42d4497aa1d"], + measurements = ["7686b818-a0ba-465c-8e4e-f7d3c4e171fc"], + vMin = 0.98, + vMax = 1.02 + } +] diff --git a/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala b/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala index 01c3f04fbc..465bbd319f 100644 --- a/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala +++ b/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala @@ -482,7 +482,9 @@ object EmAgent { modelShell.uuid, result.p.toMegawatts.asMegaWatt, result.q.toMegavars.asMegaVar, - ) + ), + tick = lastActiveTick, + nextTick = inactiveCore.nextActiveTick ) } 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 53320c4889..f15532e6d1 100644 --- a/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgentFundamentals.scala +++ b/src/main/scala/edu/ie3/simona/agent/participant/ParticipantAgentFundamentals.scala @@ -1933,6 +1933,7 @@ protected trait ParticipantAgentFundamentals[ val dateTime = tick.toDateTime(baseStateData.startDate) ParticipantResultEvent( buildResult(uuid, dateTime, result), + tick, nextTick ) } diff --git a/src/main/scala/edu/ie3/simona/event/ResultEvent.scala b/src/main/scala/edu/ie3/simona/event/ResultEvent.scala index 13a23eb499..ce5879cbaa 100644 --- a/src/main/scala/edu/ie3/simona/event/ResultEvent.scala +++ b/src/main/scala/edu/ie3/simona/event/ResultEvent.scala @@ -34,6 +34,7 @@ object ResultEvent { */ final case class ParticipantResultEvent( systemParticipantResult: SystemParticipantResult, + tick: Long = -2L, nextTick: Option[Long] = None ) extends ResultEvent @@ -67,6 +68,7 @@ object ResultEvent { lineResults: Iterable[LineResult], transformer2wResults: Iterable[Transformer2WResult], transformer3wResults: Iterable[PartialTransformer3wResult], + tick: Long = -4L ) extends ResultEvent /** Event that holds the flexibility options result of a diff --git a/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala b/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala index a1d31f0d25..f38e1c7449 100644 --- a/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala +++ b/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala @@ -149,12 +149,14 @@ object ResultEventListener extends Transformer3wResultSupport { resultEntity: ResultEntity, baseData: BaseData, log: Logger, + tick: Long = -2L, nextTick: Option[Long] = None ): BaseData = { //log.info("Got Result " + resultEntity) handOverToSink(resultEntity, baseData.classToSink, log) if (baseData.extResultDataService.isDefined) { handOverToExternalService( + tick, resultEntity, baseData.extResultDataService, nextTick @@ -235,14 +237,15 @@ object ResultEventListener extends Transformer3wResultSupport { } private def handOverToExternalService( - resultEntity: ResultEntity, - extResultDataService: Option[ActorRef[ExtResultDataProvider.Request]], - nextTick: Option[Long] = None + tick: Long, + resultEntity: ResultEntity, + extResultDataService: Option[ActorRef[ExtResultDataProvider.Request]], + nextTick: Option[Long] = None ): Unit = Try { val extResultDataServiceRef = extResultDataService.getOrElse( throw new Exception("No external data service registered!") ) - extResultDataServiceRef ! ResultResponseMessage(resultEntity, nextTick) + extResultDataServiceRef ! ResultResponseMessage(resultEntity, tick, nextTick) } def apply( @@ -295,8 +298,8 @@ object ResultEventListener extends Transformer3wResultSupport { private def idle(baseData: BaseData): Behavior[Request] = Behaviors .receivePartial[Request] { - case (ctx, ParticipantResultEvent(participantResult, nextTick)) => - val updatedBaseData = handleResult(participantResult, baseData, ctx.log, nextTick) + case (ctx, ParticipantResultEvent(participantResult, tick, nextTick)) => + val updatedBaseData = handleResult(participantResult, baseData, ctx.log, tick, nextTick) idle(updatedBaseData) case (ctx, ThermalResultEvent(thermalResult)) => @@ -311,13 +314,14 @@ object ResultEventListener extends Transformer3wResultSupport { lineResults, transformer2wResults, transformer3wResults, + tick ), ) => val updatedBaseData = (nodeResults ++ switchResults ++ lineResults ++ transformer2wResults ++ transformer3wResults) .foldLeft(baseData) { case (currentBaseData, resultEntity: ResultEntity) => - handleResult(resultEntity, currentBaseData, ctx.log) + handleResult(resultEntity, currentBaseData, ctx.log, tick) case ( currentBaseData, partialTransformerResult: PartialTransformer3wResult, @@ -331,7 +335,7 @@ object ResultEventListener extends Transformer3wResultSupport { idle(updatedBaseData) case (ctx, FlexOptionsResultEvent(flexOptionsResult)) => - val updatedBaseData = handleResult(flexOptionsResult, baseData, ctx.log) + val updatedBaseData = handleResult(flexOptionsResult, baseData, ctx.log, -3L) idle(updatedBaseData) case (ctx, msg: DelayedStopHelper.StoppingMsg) => diff --git a/src/main/scala/edu/ie3/simona/service/em/ExtEmDataService.scala b/src/main/scala/edu/ie3/simona/service/em/ExtEmDataService.scala index 5de090fdb7..3afa1315cf 100644 --- a/src/main/scala/edu/ie3/simona/service/em/ExtEmDataService.scala +++ b/src/main/scala/edu/ie3/simona/service/em/ExtEmDataService.scala @@ -1,26 +1,26 @@ package edu.ie3.simona.service.em import edu.ie3.datamodel.models.value.PValue +import edu.ie3.simona.agent.em.EmAgent import edu.ie3.simona.api.data.em.ExtEmData import edu.ie3.simona.api.data.em.ontology.{EmDataMessageFromExt, ProvideEmData} import edu.ie3.simona.api.data.ontology.DataMessageFromExt import edu.ie3.simona.exceptions.WeatherServiceException.InvalidRegistrationRequestException import edu.ie3.simona.exceptions.{InitializationException, ServiceException} -import edu.ie3.simona.ontology.messages.flex.FlexibilityMessage.{FlexRequest, IssuePowerControl, ProvideExtEmSetPoint, RequestFlexOptions} +import edu.ie3.simona.ontology.messages.flex.FlexibilityMessage.{FlexRequest, IssuePowerControl, ProvideExtEmSetPoint} import edu.ie3.simona.ontology.messages.services.ServiceMessage.ExtEmDataServiceRegistrationMessage +import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.{RegistrationSuccessfulMessage, WrappedRegistrationSuccessfulMessage} import edu.ie3.simona.ontology.messages.services.{DataMessage, ServiceMessage} import edu.ie3.simona.service.ServiceStateData.{InitializeServiceStateData, ServiceBaseStateData} -import edu.ie3.simona.service.em.ExtEmDataService.{ExtEmDataStateData, InitExtEmData, WrappedIssuePowerControl} +import edu.ie3.simona.service.em.ExtEmDataService.{ExtEmDataStateData, InitExtEmData} import edu.ie3.simona.service.{ExtDataSupport, SimonaService} import org.apache.pekko.actor.typed.ActorRef import org.apache.pekko.actor.{ActorContext, Props, ActorRef => ClassicRef} import squants.Power import squants.energy.Kilowatts -import edu.ie3.simona.agent.em.EmAgent -import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.{RegistrationSuccessfulMessage, WrappedRegistrationSuccessfulMessage} import java.util.UUID -import scala.jdk.CollectionConverters.MapHasAsScala +import scala.jdk.CollectionConverters.{CollectionHasAsScala, MapHasAsScala} import scala.util.{Failure, Success, Try} object ExtEmDataService { @@ -72,15 +72,7 @@ final case class ExtEmDataService( case InitExtEmData(extEmData) => val emDataInitializedStateData = ExtEmDataStateData( extEmData, - subscribers = List( - UUID.fromString("c3a7e9f5-b492-4c85-af2d-1e93f6a25443"), - UUID.fromString("f9dc7ce6-658c-4101-a12f-d58bb889286b"), - UUID.fromString("957938b7-0476-4fab-a1b3-6ce8615857b3") - ) - //subscribers = List( - // UUID.fromString("fd1a8de9-722a-4304-8799-e1e976d9979c"), - // UUID.fromString("ff0b995a-86ff-4f4d-987e-e475a64f2180") - //) + subscribers = extEmData.getControlledEms.asScala.toList ) Success( emDataInitializedStateData, diff --git a/src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala b/src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala index 3ade0e1ee7..b64f553d77 100644 --- a/src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala +++ b/src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala @@ -1,6 +1,7 @@ package edu.ie3.simona.service.results import edu.ie3.datamodel.models.result.ResultEntity +import edu.ie3.datamodel.models.result.system.PvResult import edu.ie3.simona.api.data.results.ExtResultData import edu.ie3.simona.api.data.results.ontology.{ProvideResultEntities, RequestResultEntities, ResultDataMessageFromExt} import edu.ie3.simona.event.listener.DelayedStopHelper @@ -15,6 +16,7 @@ import org.apache.pekko.actor.typed.{ActorRef, Behavior, PostStop} import org.apache.pekko.actor.typed.scaladsl.{Behaviors, StashBuffer} import java.util.UUID +import scala.collection.immutable.Set import scala.jdk.CollectionConverters._ object ExtResultDataProvider { @@ -35,6 +37,7 @@ object ExtResultDataProvider { /** ResultEventListener -> ExtResultDataProvider */ final case class ResultResponseMessage( result: ResultEntity, + tick: Long, nextTick: Option[Long] ) extends Request @@ -55,7 +58,6 @@ object ExtResultDataProvider { scheduler: ActorRef[SchedulerMessage] ): Behavior[Request] = Behaviors.withStash(5000) { buffer => Behaviors.setup[Request] { ctx => - //ctx.log.info("Starting initialization!") val activationAdapter: ActorRef[Activation] = ctx.messageAdapter[Activation](msg => WrappedActivation(msg)) val resultDataMessageFromExtAdapter: ActorRef[ResultDataMessageFromExt] = ctx.messageAdapter[ResultDataMessageFromExt](msg => WrappedResultDataMessageFromExt(msg)) val scheduleServiceActivationAdapter: ActorRef[ScheduleServiceActivation] = ctx.messageAdapter[ScheduleServiceActivation](msg => WrappedScheduleServiceActivationAdapter(msg)) @@ -99,22 +101,18 @@ object ExtResultDataProvider { resultDataMessageFromExtAdapter: ActorRef[ResultDataMessageFromExt], buffer: StashBuffer[Request]): Behavior[Request] = { Behaviors.receivePartial { - case (_, WrappedActivation(Activation(INIT_SIM_TICK))) => + case (ctx, WrappedActivation(Activation(INIT_SIM_TICK))) => val initGridSubscribers = initServiceData.extResultData.getGridResultDataAssets.asScala.toList val initParticipantSubscribers = initServiceData.extResultData.getParticipantResultDataAssets.asScala.toList - var initResultStorage = Map.empty[UUID, (Option[ResultEntity], Option[Long])] - initParticipantSubscribers.foreach( - uuid => initResultStorage = initResultStorage + (uuid -> (None, Some(0L))) - ) - initGridSubscribers.foreach( - uuid => initResultStorage = initResultStorage + (uuid -> (None, Some(initServiceData.extResultData.getPowerFlowResolution))) - ) + var initResultSchedule = Map.empty[Long, Set[UUID]] + initResultSchedule = initResultSchedule + (0L -> initParticipantSubscribers.toSet) + initResultSchedule = initResultSchedule + (initServiceData.extResultData.getPowerFlowResolution.longValue() -> initGridSubscribers.toSet) + val resultInitializedStateData = ExtResultStateData( - extResultData = initServiceData.extResultData, - gridSubscribers = initGridSubscribers, - participantSubscribers = initParticipantSubscribers, - resultStorage = initResultStorage + extResultData = initServiceData.extResultData, + currentTick = INIT_SIM_TICK, + extResultScheduler = initResultSchedule ) scheduler ! Completion( activationAdapter, @@ -133,7 +131,8 @@ object ExtResultDataProvider { ): Behavior[Request] = Behaviors .receivePartial[Request] { case (ctx, WrappedActivation(activation: Activation)) => - //ctx.log.info("Received Activation") + val thisCurrentTick = activation.tick + //ctx.log.info(s"+++++++ Received Activation for tick $thisCurrentTick ++++++") var updatedStateData = serviceStateData serviceStateData.extResultsMessage.getOrElse( @@ -141,59 +140,43 @@ object ExtResultDataProvider { "ExtResultDataService was triggered without ResultDataMessageFromExt available" ) ) match { - case msg: RequestResultEntities => + case msg: RequestResultEntities => // ExtResultDataProvider wurde aktiviert und es wurden Nachrichten von ExtSimulation angefragt //ctx.log.info(s"[requestResults] for tick ${msg.tick} and resultStorage ${serviceStateData.resultStorage}") - var receiveDataMap = ReceiveDataMap.empty[UUID, ResultEntity] - if (activation.tick == 0L) { - receiveDataMap = ReceiveDataMap[UUID, ResultEntity](serviceStateData.participantSubscribers.toSet) - } else { - receiveDataMap = ReceiveDataMap[UUID, ResultEntity]((serviceStateData.participantSubscribers ++ serviceStateData.gridSubscribers).toSet) - } - //ctx.log.info(s"[requestResults] tick ${msg.tick} -> created a receivedatamap " + receiveDataMap) - /* - serviceStateData.resultStorage.foreach({ - case (uuid, (res, t)) => - //ctx.log.info(s"[requestResults] tick = ${msg.tick}, uuid = $uuid, and time = ${t.getOrElse("Option")}, result = ${res.getOrElse("Option")}") - if (t.getOrElse(-1L) != msg.tick) { //wenn nicht in diesem Tick gefragt, nehme Wert aus ResultDataStorage - receiveDataMap = receiveDataMap.addData( - uuid, - res.getOrElse( - throw new Exception("noResult") - ) - ) - //ctx.log.info(s"[requestResults] tick ${msg.tick} -> added to receivedatamap " + receiveDataMap) - } - }) - - */ - - //ctx.log.info(s"[requestResults] tick ${msg.tick} -> requestResults for " + receiveDataMap) + //ctx.log.info(s"[requestResults] for tick ${msg.tick} and extResultScheduler ${serviceStateData.extResultScheduler}") + var updatedSchedule = serviceStateData.extResultScheduler + val expectedKeys = serviceStateData.extResultScheduler.getOrElse( + activation.tick, + Set() + ) ++ serviceStateData.extResultScheduler.getOrElse(-2L, Set()) + val receiveDataMap = ReceiveDataMap[UUID, ResultEntity](expectedKeys) + updatedSchedule = updatedSchedule.-(activation.tick) + + //ctx.log.info(s"[requestResults] tick ${msg.tick} -> updatedSchedule = $updatedSchedule") + //ctx.log.info(s"[requestResults] tick ${msg.tick} -> receiveDataMap = $receiveDataMap") - var resultList = List.empty[ResultEntity] if (receiveDataMap.isComplete) { - if (receiveDataMap.getExpectedKeys.nonEmpty) { - serviceStateData.resultStorage.values.foreach( - result => resultList = resultList :+ result._1.getOrElse( - throw new RuntimeException("There is no result!") - ) - ) - } - //ctx.log.info(s"[requestResults] tick ${msg.tick} -> ReceiveDataMap is complete -> send it right away: " + resultList) - // all responses received, forward them to external simulation in a bundle + // --- There are no expected results for this tick! Send the send right away! + //ctx.log.info(s"[requestResults] tick ${msg.tick} -> ReceiveDataMap is complete -> send it right away: " + serviceStateData.resultStorage) + serviceStateData.extResultData.queueExtResponseMsg( - new ProvideResultEntities(resultList.asJava) + new ProvideResultEntities(serviceStateData.resultStorage.asJava) ) + //ctx.log.info("++++++++++++++++++ sended ExtResultData +++++++++++++++++++++++") updatedStateData = serviceStateData.copy( extResultsMessage = None, - recentResults = None) - + receiveDataMap = None, + extResultScheduler = updatedSchedule, + currentTick = thisCurrentTick + ) } else { //ctx.log.info(s"[requestResults] receiveDataMap was built -> now sending ResultRequestMessage") ctx.self ! ResultRequestMessage(msg.tick) updatedStateData = serviceStateData.copy( - extResultsMessage = None, - recentResults = Some(receiveDataMap) - ) + extResultsMessage = None, + receiveDataMap = Some(receiveDataMap), + extResultScheduler = updatedSchedule, + currentTick = thisCurrentTick + ) } } @@ -216,38 +199,74 @@ object ExtResultDataProvider { )) case (ctx, extResultResponseMsg: ResultResponseMessage) => - //ctx.log.info("[handleDataResponseMessage] Received ResultsResponseMessage") - - if (serviceStateData.recentResults.isDefined) { + if (serviceStateData.receiveDataMap.isDefined) { // process dataResponses - if (serviceStateData.recentResults.getOrElse(throw new Exception("no Receive Data Map!")).getExpectedKeys.contains(extResultResponseMsg.result.getInputModel)) { - //if (serviceStateData.participantSubscribers.contains(extResultResponseMsg.result.getInputModel) || serviceStateData.gridSubscribers.contains(extResultResponseMsg.result.getInputModel)) { - //ctx.log.info("[handleDataResponseMessage] Received ResultsResponseMessage with content " + extResultResponseMsg) - //ctx.log.info("[handleDataResponseMessage] RecentResults " + serviceStateData.recentResults) - val updatedReceivedResults = serviceStateData.recentResults.getOrElse(throw new Exception("noMap")).addData(extResultResponseMsg.result.getInputModel, extResultResponseMsg.result) - //ctx.log.info("[handleDataResponseMessage] AddData to RecentResults -> updatedReceivedResults = " + updatedReceivedResults) - val updatedResultStorage = - serviceStateData.resultStorage + (extResultResponseMsg.result.getInputModel -> (Some(extResultResponseMsg.result), extResultResponseMsg.nextTick)) - if (updatedReceivedResults.nonComplete) { - // all responses received, forward them to external simulation in a bundle - idle(serviceStateData.copy( - recentResults = Some(updatedReceivedResults), - resultStorage = updatedResultStorage - )) + if (serviceStateData.receiveDataMap.getOrElse(throw new Exception("There is no activation yet! Receive Data Map does not exist!")).getExpectedKeys.contains(extResultResponseMsg.result.getInputModel)) { + //ctx.log.info(s"[hDRM] Process ResultsResponseMsg = $extResultResponseMsg") + //ctx.log.info(s"[hDRM] receiveDataMap ${serviceStateData.receiveDataMap}") + //ctx.log.info(s"[hDRM] MsgTick=${extResultResponseMsg.tick}, ServiceStateDataTick=${serviceStateData.currentTick}") + + // --- Add received results to receiveDataMap + + if (extResultResponseMsg.tick == -4L || extResultResponseMsg.tick == serviceStateData.currentTick) { //FIXME Not expected results are unconsidered + val updatedReceiveDataMap = serviceStateData + .receiveDataMap + .getOrElse( + throw new Exception("noMap") + ).addData( + extResultResponseMsg.result.getInputModel, + extResultResponseMsg.result + ) + + //ctx.log.info("[hDRM] AddData to RecentResults -> updatedReceivedResults = " + updatedReceiveDataMap) + + // --- Update ResultStorage and Schedule + + val updatedResultStorage = serviceStateData.resultStorage + (extResultResponseMsg.result.getInputModel -> extResultResponseMsg.result) + var updatedResultSchedule = serviceStateData.extResultScheduler + //ctx.log.info(s"[hDRM] updatedResultSchedule = $updatedResultSchedule") + + updatedResultSchedule = extResultResponseMsg.nextTick.fold { + updatedResultSchedule.updated( + -3L, + updatedResultSchedule.getOrElse(-3L, Set[UUID]()) + extResultResponseMsg.result.getInputModel + ) + } { + newTick => + //ctx.log.info(s"[hDRM] update schedule = $newTick, uuid = ${extResultResponseMsg.result.getInputModel}") + updatedResultSchedule.updated( + newTick, + updatedResultSchedule.getOrElse(newTick, Set[UUID]()) + extResultResponseMsg.result.getInputModel + ) + } + + //ctx.log.info(s"[hDRM] updatedResultSchedule = $updatedResultSchedule") + //ctx.log.info(s"[hDRM] updatedResultStorage = $updatedResultStorage") + + // --- Check, if all expected results has been received + + if (updatedReceiveDataMap.nonComplete) { + // There are still results missing... + idle(serviceStateData.copy( + receiveDataMap = Some(updatedReceiveDataMap), + resultStorage = updatedResultStorage, + extResultScheduler = updatedResultSchedule + )) + } else { + // all responses received, forward them to external simulation in a bundle + serviceStateData.extResultData.queueExtResponseMsg( + new ProvideResultEntities(updatedResultStorage.asJava) + ) + //ctx.log.info("[hDRM] Got all ResultResponseMessage -> Now forward to external simulation in a bundle: " + updatedResultStorage) + //ctx.log.info("++++++++++++++++++ sended ExtResultData +++++++++++++++++++++++") + idle(serviceStateData.copy( + receiveDataMap = None, + resultStorage = updatedResultStorage, + extResultScheduler = updatedResultSchedule + )) + } } else { - var resultList = List.empty[ResultEntity] - updatedReceivedResults.receivedData.values.foreach( - result => resultList = resultList :+ result - ) - // all responses received, forward them to external simulation in a bundle - serviceStateData.extResultData.queueExtResponseMsg( - new ProvideResultEntities(resultList.asJava) - ) - //log.info("[handleDataResponseMessage] Got all ResultResponseMessage -> Now forward to external simulation in a bundle: " + resultList) - idle(serviceStateData.copy( - resultStorage = updatedResultStorage, - recentResults = None - )) + idle(serviceStateData) } } else { idle(serviceStateData) @@ -257,7 +276,9 @@ object ExtResultDataProvider { buffer.stash(extResultResponseMsg) idle(serviceStateData) } - case (_, msg: ResultRequestMessage) => + + case (ctx, msg: ResultRequestMessage) => + //ctx.log.info(s"[handleDataResponseMessage] Received ResultRequestMessage $msg -> Now unstash all buffered messages!") buffer.unstashAll(idle(serviceStateData)) case (ctx, msg: DelayedStopHelper.StoppingMsg) => @@ -267,12 +288,11 @@ object ExtResultDataProvider { // -=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=- final case class ExtResultStateData( extResultData: ExtResultData, - gridSubscribers: List[UUID] = List.empty, - participantSubscribers: List[UUID] = List.empty, + currentTick: Long, extResultsMessage: Option[ResultDataMessageFromExt] = None, - resultStorage: Map[UUID, (Option[ResultEntity], Option[Long])] = Map.empty, // UUID -> Result, nextTick - maybeNextActivationTick: Option[Long] = None, - recentResults: Option[ReceiveDataMap[UUID, ResultEntity]] = None, + resultStorage: Map[UUID, ResultEntity] = Map.empty, + extResultScheduler: Map[Long, Set[UUID]] = Map.empty, + receiveDataMap: Option[ReceiveDataMap[UUID, ResultEntity]] = None, ) final case class InitExtResultData( extResultData: ExtResultData diff --git a/src/main/scala/edu/ie3/simona/sim/setup/SimonaExtSimSetup.scala b/src/main/scala/edu/ie3/simona/sim/setup/SimonaExtSimSetup.scala new file mode 100644 index 0000000000..4e76201b15 --- /dev/null +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaExtSimSetup.scala @@ -0,0 +1,537 @@ +/* + * © 2020. TU Dortmund University, + * Institute of Energy Systems, Energy Efficiency and Energy Economics, + * Research group Distribution grid planning and operation + */ + +package edu.ie3.simona.sim.setup + +import com.typesafe.config.Config +import edu.ie3.datamodel.graph.SubGridTopologyGraph +import edu.ie3.datamodel.models.input.container.{GridContainer, ThermalGrid} +import edu.ie3.datamodel.models.input.thermal.ThermalBusInput +import edu.ie3.simona.actor.SimonaActorNaming.RichActorRefFactory +import edu.ie3.simona.agent.EnvironmentRefs +import edu.ie3.simona.agent.grid.GridAgent +import edu.ie3.simona.agent.grid.GridAgentMessages.CreateGridAgent +import edu.ie3.simona.api.ExtSimAdapter +import edu.ie3.simona.api.data.em.ExtEmData +import edu.ie3.simona.api.data.primarydata.ExtPrimaryData +import edu.ie3.simona.api.data.results.ExtResultData +import edu.ie3.simona.api.data.results.ontology.ResultDataMessageFromExt +import edu.ie3.simona.api.simulation.{ExtSimAdapterData, ExtSimulation} +import edu.ie3.simona.config.{ArgsParser, RefSystemParser, SimonaConfig} +import edu.ie3.simona.event.listener.{ResultEventListener, RuntimeEventListener} +import edu.ie3.simona.event.{ResultEvent, RuntimeEvent} +import edu.ie3.simona.exceptions.agent.GridAgentInitializationException +import edu.ie3.simona.io.grid.GridProvider +import edu.ie3.simona.ontology.messages.SchedulerMessage +import edu.ie3.simona.ontology.messages.SchedulerMessage.ScheduleActivation +import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.ScheduleServiceActivation +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.em.ExtEmDataService +import edu.ie3.simona.service.em.ExtEmDataService.InitExtEmData +import edu.ie3.simona.service.primary.ExtPrimaryDataService.InitExtPrimaryData +import edu.ie3.simona.service.primary.PrimaryServiceProxy.InitPrimaryServiceProxyStateData +import edu.ie3.simona.service.primary.{ExtPrimaryDataService, PrimaryServiceProxy} +import edu.ie3.simona.service.results.ExtResultDataProvider +import edu.ie3.simona.service.results.ExtResultDataProvider.{InitExtResultData, RequestDataMessageAdapter, RequestScheduleActivationAdapter} +import edu.ie3.simona.service.weather.WeatherService +import edu.ie3.simona.service.weather.WeatherService.InitWeatherServiceStateData +import edu.ie3.simona.sim.SimonaSim +import edu.ie3.simona.util.ResultFileHierarchy +import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK +import edu.ie3.simona.util.TickUtil.RichZonedDateTime +import edu.ie3.util.TimeUtil +import org.apache.pekko.actor.typed.scaladsl.ActorContext +import org.apache.pekko.actor.typed.scaladsl.AskPattern._ +import org.apache.pekko.actor.typed.scaladsl.adapter.{ClassicActorRefOps, TypedActorContextOps, TypedActorRefOps} +import org.apache.pekko.actor.typed.{ActorRef, Scheduler} +import org.apache.pekko.actor.{ActorRef => ClassicRef} +import org.apache.pekko.util.{Timeout => PekkoTimeout} + +import java.time.ZonedDateTime +import java.time.temporal.ChronoUnit +import java.util.UUID +import java.util.concurrent.LinkedBlockingQueue +import scala.collection.mutable +import scala.concurrent.Await +import scala.concurrent.duration.DurationInt +import scala.jdk.CollectionConverters._ +import scala.jdk.DurationConverters._ + +/** Sample implementation to run a standalone simulation of simona configured + * with the provided [[SimonaConfig]] and [[ResultFileHierarchy]] + * + * @version 0.1 + * @since 01.07.20 + */ +abstract class SimonaExtSimSetup( + val typeSafeConfig: Config, + val simonaConfig: SimonaConfig, + val resultFileHierarchy: ResultFileHierarchy, + val runtimeEventQueue: Option[LinkedBlockingQueue[RuntimeEvent]] = None, + override val args: Array[String] +) extends SimonaSetup { + + override def gridAgents( + context: ActorContext[_], + environmentRefs: EnvironmentRefs, + resultEventListeners: Seq[ActorRef[ResultEvent]], + ): Iterable[ActorRef[GridAgent.Request]] = { + + /* get the grid */ + val subGridTopologyGraph = GridProvider + .gridFromConfig( + simonaConfig.simona.simulationName, + simonaConfig.simona.input.grid.datasource, + ) + .getSubGridTopologyGraph + val thermalGridsByThermalBus = GridProvider.getThermalGridsFromConfig( + simonaConfig.simona.input.grid.datasource + ) + + /* extract and prepare refSystem information from config */ + val configRefSystems = + RefSystemParser.parse(simonaConfig.simona.gridConfig.refSystems) + + /* Create all agents and map the sub grid id to their actor references */ + val subGridToActorRefMap = buildSubGridToActorRefMap( + subGridTopologyGraph, + context, + environmentRefs, + resultEventListeners, + ) + + val keys = ScheduleLock.multiKey( + context, + environmentRefs.scheduler, + INIT_SIM_TICK, + subGridTopologyGraph.vertexSet().size, + ) + + /* build the initialization data */ + subGridTopologyGraph + .vertexSet() + .asScala + .zip(keys) + .map { case (subGridContainer, key) => + /* Get all connections to superior and inferior sub grids */ + val subGridGates = + Set.from( + subGridTopologyGraph + .edgesOf(subGridContainer) + .asScala + .map(modifySubGridGateForThreeWindingSupport) + ) + val currentSubGrid = subGridContainer.getSubnet + val currentActorRef = subGridToActorRefMap.getOrElse( + currentSubGrid, + throw new GridAgentInitializationException( + "Was asked to setup agent for sub grid " + currentSubGrid + ", but did not found it's actor reference." + ), + ) + val thermalGrids = + getThermalGrids(subGridContainer, thermalGridsByThermalBus) + + /* build the grid agent data and check for its validity */ + val gridAgentInitData = SimonaStandaloneSetup.buildGridAgentInitData( + subGridContainer, + subGridToActorRefMap, + subGridGates, + configRefSystems, + thermalGrids, + ) + + currentActorRef ! CreateGridAgent(gridAgentInitData, key) + + currentActorRef + } + } + + override def primaryServiceProxy( + context: ActorContext[_], + scheduler: ActorRef[SchedulerMessage], + extSimSetupData: ExtSimSetupData, + ): ClassicRef = { + val simulationStart = TimeUtil.withDefaults.toZonedDateTime( + simonaConfig.simona.time.startDateTime + ) + val primaryServiceProxy = context.toClassic.simonaActorOf( + PrimaryServiceProxy.props( + scheduler.toClassic, + InitPrimaryServiceProxyStateData( + simonaConfig.simona.input.primary, + simulationStart, + extSimSetupData.extPrimaryDataService, + extSimSetupData.extPrimaryData + ), + simulationStart, + ) + ) + + scheduler ! ScheduleActivation(primaryServiceProxy.toTyped, INIT_SIM_TICK) + primaryServiceProxy + } + + override def weatherService( + context: ActorContext[_], + scheduler: ActorRef[SchedulerMessage], + ): ClassicRef = { + val weatherService = context.toClassic.simonaActorOf( + WeatherService.props( + scheduler.toClassic, + TimeUtil.withDefaults + .toZonedDateTime(simonaConfig.simona.time.startDateTime), + TimeUtil.withDefaults + .toZonedDateTime(simonaConfig.simona.time.endDateTime), + ) + ) + weatherService ! SimonaService.Create( + InitWeatherServiceStateData( + simonaConfig.simona.input.weather.datasource + ), + ScheduleLock.singleKey(context, scheduler, INIT_SIM_TICK), + ) + + weatherService + } + + override def timeAdvancer( + context: ActorContext[_], + simulation: ActorRef[SimonaSim.SimulationEnded.type], + runtimeEventListener: ActorRef[RuntimeEvent], + ): ActorRef[TimeAdvancer.Request] = { + val startDateTime = TimeUtil.withDefaults.toZonedDateTime( + simonaConfig.simona.time.startDateTime + ) + val endDateTime = TimeUtil.withDefaults.toZonedDateTime( + simonaConfig.simona.time.endDateTime + ) + + context.spawn( + TimeAdvancer( + simulation, + Some(runtimeEventListener), + simonaConfig.simona.time.schedulerReadyCheckWindow, + endDateTime.toTick(startDateTime), + ), + TimeAdvancer.getClass.getSimpleName, + ) + } + + override def scheduler( + context: ActorContext[_], + parent: ActorRef[SchedulerMessage], + coreFactory: CoreFactory = RegularSchedulerCore, + ): ActorRef[SchedulerMessage] = + context + .spawn( + Scheduler(parent, coreFactory), + s"${Scheduler.getClass.getSimpleName}_${coreFactory}_${UUID.randomUUID()}", + ) + + override def runtimeEventListener( + context: ActorContext[_] + ): ActorRef[RuntimeEventListener.Request] = + context + .spawn( + RuntimeEventListener( + simonaConfig.simona.runtime.listener, + runtimeEventQueue, + startDateTimeString = simonaConfig.simona.time.startDateTime, + ), + RuntimeEventListener.getClass.getSimpleName, + ) + + override def resultEventListener( + context: ActorContext[_], + extSimulationData: ExtSimSetupData, + ): Seq[ActorRef[ResultEventListener.Request]] = { + val extResultDataService: Option[ActorRef[ExtResultDataProvider.Request]] = + extSimulationData.extResultDataService + // append ResultEventListener as well to write raw output files + ArgsParser + .parseListenerConfigOption(simonaConfig.simona.event.listener) + .zipWithIndex + .map { case ((listenerCompanion, events), index) => + context.toClassic + .simonaActorOf( + listenerCompanion.props(events), + index.toString, + ) + .toTyped + } + .toSeq :+ context + .spawn( + ResultEventListener( + resultFileHierarchy, + extResultDataService, + ), + ResultEventListener.getClass.getSimpleName, + ) + } + + def buildSubGridToActorRefMap( + subGridTopologyGraph: SubGridTopologyGraph, + context: ActorContext[_], + environmentRefs: EnvironmentRefs, + resultEventListeners: Seq[ActorRef[ResultEvent]], + ): Map[Int, ActorRef[GridAgent.Request]] = { + subGridTopologyGraph + .vertexSet() + .asScala + .map(subGridContainer => { + val gridAgentRef = + context.spawn( + GridAgent( + environmentRefs, + simonaConfig, + resultEventListeners, + ), + subGridContainer.getSubnet.toString, + ) + subGridContainer.getSubnet -> gridAgentRef + }) + .toMap + } + + /** Get all thermal grids, that apply for the given grid container + * @param grid + * The grid container to assess + * @param thermalGridByBus + * Mapping from thermal bus to thermal grid + * @return + * A sequence of applicable thermal grids + */ + private def getThermalGrids( + grid: GridContainer, + thermalGridByBus: Map[ThermalBusInput, ThermalGrid], + ): Seq[ThermalGrid] = { + grid.getSystemParticipants.getHeatPumps.asScala + .flatten(hpInput => thermalGridByBus.get(hpInput.getThermalBus)) + .toSeq + } + + + def extSimulationSetup( + context: ActorContext[_], + rootScheduler: ActorRef[SchedulerMessage], + simScheduler: ActorRef[SchedulerMessage], + extSim: ExtSimulation + ): ExtSimSetupData = { + // ExtSimAdapter + val extScheduler = scheduler(context, parent = rootScheduler) + + val extSimAdapterPhase1 = context.toClassic.simonaActorOf( + ExtSimAdapter.props(extScheduler.toClassic), + s"1", + ) + + val extSimAdapterPhase2 = context.toClassic.simonaActorOf( + ExtSimAdapter.props(simScheduler.toClassic), + s"2", + ) + + val extSimAdapters: Map[java.lang.Integer, ClassicRef] = Map( + 1.asInstanceOf[java.lang.Integer] -> extSimAdapterPhase1, + 2.asInstanceOf[java.lang.Integer] -> extSimAdapterPhase2 + ) + + val extSimAdapterData = new ExtSimAdapterData(extSimAdapters.asJava, args) + + val extDataServicesMap: mutable.Map[Class[_], ClassicRef] = mutable.Map.empty + val extDataListenerMap: mutable.Map[Class[_], ActorRef[ExtResultDataProvider.Request]] = mutable.Map.empty + + val dataConnections = extSim.getDataConnections + + println("dataConnections = " + dataConnections) + + dataConnections.asScala.foreach { + case extPrimaryData: ExtPrimaryData => + val extPrimaryDataService = extPrimaryDataSimulationSetup( + context, + extScheduler, + extSimAdapterData, + extPrimaryData + ) + extDataServicesMap += (classOf[ExtPrimaryDataService] -> extPrimaryDataService) + + + case extResultData: ExtResultData => + val extResultDataProvider = extResultDataSimulationSetup( + context, + simScheduler, + extSimAdapterData, + extResultData, + TimeUtil.withDefaults.toZonedDateTime( + simonaConfig.simona.time.startDateTime + ), + simonaConfig.simona.powerflow.resolution.get( + ChronoUnit.SECONDS + ) + ) + extDataListenerMap += (ExtResultDataProvider.getClass -> extResultDataProvider) + + + case extEmData: ExtEmData => + val extEmDataService = extEmDataSimulationSetup( + context, + extScheduler, + extSimAdapterData, + extEmData + ) + extDataServicesMap += (classOf[ExtEmDataService] -> extEmDataService) + + } + extSim.setup( + extSimAdapterData, + dataConnections + ) + new Thread(extSim, s"External simulation").start() + + println("DataSerivces = " + extDataServicesMap) + println("DataProvider = " + extDataListenerMap) + + ExtSimSetupData( + Iterable( + extSimAdapterData.getPhase1Adapter, + extSimAdapterData.getPhase2Adapter + ), + extDataServicesMap.toMap, + extDataListenerMap.toMap, + dataConnections.asScala.toSet, + Some(extScheduler) + ) + } + + private def extPrimaryDataSimulationSetup( + context: ActorContext[_], + extScheduler: ActorRef[SchedulerMessage], + extSimAdapterData: ExtSimAdapterData, + extPrimaryData: ExtPrimaryData + ): ClassicRef = { + val extSimAdapterPhase1 = extSimAdapterData.getPhase1Adapter + + // send init data right away, init activation is scheduled + extSimAdapterPhase1 ! ExtSimAdapter.Create( + extSimAdapterData, + 1, + ScheduleLock.singleKey(context, extScheduler, INIT_SIM_TICK), + ) + + val extPrimaryDataService = context.toClassic.simonaActorOf( + ExtPrimaryDataService.props(extScheduler.toClassic), + s"0-0", + ) + + extPrimaryData.setActorRefs( + extPrimaryDataService, + extSimAdapterPhase1 + ) + + extPrimaryDataService ! SimonaService.Create( + InitExtPrimaryData(extPrimaryData), + ScheduleLock.singleKey( + context, + extScheduler, + INIT_SIM_TICK, + ), + ) + extPrimaryDataService + } + + + private def extEmDataSimulationSetup( + context: ActorContext[_], + extScheduler: ActorRef[SchedulerMessage], + extSimAdapterData: ExtSimAdapterData, + extEmData: ExtEmData + ): ClassicRef = { + val extSimAdapterPhase1 = extSimAdapterData.getPhase1Adapter + + // send init data right away, init activation is scheduled + extSimAdapterPhase1 ! ExtSimAdapter.Create( + extSimAdapterData, + 1, + ScheduleLock.singleKey(context, extScheduler, INIT_SIM_TICK), + ) + + val extEmDataService = context.toClassic.simonaActorOf( + ExtEmDataService.props(extScheduler.toClassic), + s"0-0", + ) + + extEmData.setActorRefs( + extEmDataService, + extSimAdapterPhase1 + ) + + extEmDataService ! SimonaService.Create( + InitExtEmData(extEmData), + ScheduleLock.singleKey( + context, + extScheduler, + INIT_SIM_TICK, + ), + ) + extEmDataService + } + + + private def extResultDataSimulationSetup( + context: ActorContext[_], + simScheduler: ActorRef[SchedulerMessage], + extSimAdapterData: ExtSimAdapterData, + extResultData: ExtResultData, + simulationStart: ZonedDateTime, + powerFlowResolution: Long + ): ActorRef[ExtResultDataProvider.Request] = { + val extResultDataProvider = { + context.spawn( + ExtResultDataProvider(simScheduler), + s"ExtResultDataProvider", + ) + } + + val timeout: PekkoTimeout = PekkoTimeout.create(5.seconds.toJava) + val scheduler2: Scheduler = context.system.scheduler + + val adapterRef = Await.result( + extResultDataProvider.ask[ActorRef[ResultDataMessageFromExt]](ref => RequestDataMessageAdapter(ref))(timeout, scheduler2), timeout.duration) + val adapterScheduleRef = Await.result( + extResultDataProvider.ask[ActorRef[ScheduleServiceActivation]](ref => RequestScheduleActivationAdapter(ref))(timeout, scheduler2), timeout.duration) + + val extSimAdapterPhase2 = extSimAdapterData.getPhase2Adapter + + extResultData.setActorRefs( + adapterRef.toClassic, + adapterScheduleRef.toClassic, + extSimAdapterPhase2 + ) + + extResultData.setSimulationData( + simulationStart, + powerFlowResolution + ) + + extResultDataProvider ! ExtResultDataProvider.Create( + InitExtResultData(extResultData), + ScheduleLock.singleKey( + context, + simScheduler, + INIT_SIM_TICK, + ), + ) + extSimAdapterPhase2 ! ExtSimAdapter.Create( + extSimAdapterData, + 2, + ScheduleLock.singleKey(context, simScheduler, INIT_SIM_TICK), + ) + extResultDataProvider + } + +} diff --git a/src/main/scala/edu/ie3/simona/sim/setup/SimonaMosaikSetup.scala b/src/main/scala/edu/ie3/simona/sim/setup/SimonaMosaikSetup.scala index 78c6d05195..d392998e0b 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaMosaikSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaMosaikSetup.scala @@ -8,59 +8,16 @@ package edu.ie3.simona.sim.setup import com.typesafe.config.Config import com.typesafe.scalalogging.LazyLogging -import edu.ie3.datamodel.graph.SubGridTopologyGraph -import edu.ie3.datamodel.models.input.container.{GridContainer, ThermalGrid} -import edu.ie3.datamodel.models.input.thermal.ThermalBusInput -import edu.ie3.simona.actor.SimonaActorNaming.RichActorRefFactory -import edu.ie3.simona.agent.EnvironmentRefs -import edu.ie3.simona.agent.grid.GridAgent -import edu.ie3.simona.agent.grid.GridAgentMessages.CreateGridAgent -import edu.ie3.simona.api.ExtSimAdapter -import edu.ie3.simona.api.data.ExtData -import edu.ie3.simona.api.data.primarydata.ExtPrimaryData -import edu.ie3.simona.api.data.results.ExtResultData -import edu.ie3.simona.api.data.results.ontology.ResultDataMessageFromExt -import edu.ie3.simona.api.simulation.ExtSimAdapterData -import edu.ie3.simona.config.{ArgsParser, RefSystemParser, SimonaConfig} -import edu.ie3.simona.event.listener.{ResultEventListener, RuntimeEventListener} -import edu.ie3.simona.event.{ResultEvent, RuntimeEvent} -import edu.ie3.simona.exceptions.agent.GridAgentInitializationException -import edu.ie3.simona.io.grid.GridProvider +import edu.ie3.simona.config.SimonaConfig +import edu.ie3.simona.event.RuntimeEvent import edu.ie3.simona.ontology.messages.SchedulerMessage -import edu.ie3.simona.ontology.messages.SchedulerMessage.ScheduleActivation -import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.ScheduleServiceActivation -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.primary.ExtPrimaryDataService.InitExtPrimaryData -import edu.ie3.simona.service.primary.PrimaryServiceProxy.InitPrimaryServiceProxyStateData -import edu.ie3.simona.service.primary.{ExtPrimaryDataService, PrimaryServiceProxy} -import edu.ie3.simona.service.results.ExtResultDataProvider -import edu.ie3.simona.service.results.ExtResultDataProvider.{InitExtResultData, RequestDataMessageAdapter, RequestScheduleActivationAdapter} -import edu.ie3.simona.service.weather.WeatherService -import edu.ie3.simona.service.weather.WeatherService.InitWeatherServiceStateData -import edu.ie3.simona.sim.SimonaSim import edu.ie3.simona.util.ResultFileHierarchy -import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK -import edu.ie3.simona.util.TickUtil.RichZonedDateTime import edu.ie3.simosaik.MosaikSimulation -import edu.ie3.util.TimeUtil +import org.apache.pekko.actor.typed.ActorRef import org.apache.pekko.actor.typed.scaladsl.ActorContext -import org.apache.pekko.actor.typed.scaladsl.AskPattern._ -import org.apache.pekko.actor.typed.scaladsl.adapter.{ClassicActorRefOps, TypedActorContextOps, TypedActorRefOps} -import org.apache.pekko.actor.typed.{ActorRef, Scheduler} -import org.apache.pekko.actor.{ActorRef => ClassicRef} -import org.apache.pekko.util.{Timeout => PekkoTimeout} import java.nio.file.Path -import java.time.temporal.ChronoUnit -import java.util.UUID import java.util.concurrent.LinkedBlockingQueue -import scala.concurrent.Await -import scala.concurrent.duration.DurationInt -import scala.jdk.CollectionConverters._ -import scala.jdk.DurationConverters._ /** Sample implementation to run a standalone simulation of simona configured * with the provided [[SimonaConfig]] and [[ResultFileHierarchy]] @@ -69,553 +26,36 @@ import scala.jdk.DurationConverters._ * @since 01.07.20 */ class SimonaMosaikSetup( - val typeSafeConfig: Config, - simonaConfig: SimonaConfig, - resultFileHierarchy: ResultFileHierarchy, - runtimeEventQueue: Option[LinkedBlockingQueue[RuntimeEvent]] = None, - override val args: Array[String], - mosaikIP: Option[String] = None, - mosaikMappingPath: Option[String] = None -) extends SimonaSetup { - - override def gridAgents( - context: ActorContext[_], - environmentRefs: EnvironmentRefs, - resultEventListeners: Seq[ActorRef[ResultEvent]], - ): Iterable[ActorRef[GridAgent.Request]] = { - - /* get the grid */ - val subGridTopologyGraph = GridProvider - .gridFromConfig( - simonaConfig.simona.simulationName, - simonaConfig.simona.input.grid.datasource, - ) - .getSubGridTopologyGraph - val thermalGridsByThermalBus = GridProvider.getThermalGridsFromConfig( - simonaConfig.simona.input.grid.datasource - ) - - /* extract and prepare refSystem information from config */ - val configRefSystems = - RefSystemParser.parse(simonaConfig.simona.gridConfig.refSystems) - - /* Create all agents and map the sub grid id to their actor references */ - val subGridToActorRefMap = buildSubGridToActorRefMap( - subGridTopologyGraph, - context, - environmentRefs, - resultEventListeners, - ) - - val keys = ScheduleLock.multiKey( - context, - environmentRefs.scheduler, - INIT_SIM_TICK, - subGridTopologyGraph.vertexSet().size, - ) - - /* build the initialization data */ - subGridTopologyGraph - .vertexSet() - .asScala - .zip(keys) - .map { case (subGridContainer, key) => - /* Get all connections to superior and inferior sub grids */ - val subGridGates = - Set.from( - subGridTopologyGraph - .edgesOf(subGridContainer) - .asScala - .map(modifySubGridGateForThreeWindingSupport) - ) - val currentSubGrid = subGridContainer.getSubnet - val currentActorRef = subGridToActorRefMap.getOrElse( - currentSubGrid, - throw new GridAgentInitializationException( - "Was asked to setup agent for sub grid " + currentSubGrid + ", but did not found it's actor reference." - ), - ) - val thermalGrids = - getThermalGrids(subGridContainer, thermalGridsByThermalBus) - - /* build the grid agent data and check for its validity */ - val gridAgentInitData = SimonaStandaloneSetup.buildGridAgentInitData( - subGridContainer, - subGridToActorRefMap, - subGridGates, - configRefSystems, - thermalGrids, - ) - - currentActorRef ! CreateGridAgent(gridAgentInitData, key) - - currentActorRef - } - } - - override def primaryServiceProxy( - context: ActorContext[_], - scheduler: ActorRef[SchedulerMessage], - extSimSetupData: ExtSimSetupData, - ): ClassicRef = { - val simulationStart = TimeUtil.withDefaults.toZonedDateTime( - simonaConfig.simona.time.startDateTime - ) - val primaryServiceProxy = context.toClassic.simonaActorOf( - PrimaryServiceProxy.props( - scheduler.toClassic, - InitPrimaryServiceProxyStateData( - simonaConfig.simona.input.primary, - simulationStart, - extSimSetupData.extPrimaryDataService, - extSimSetupData.extPrimaryData - ), - simulationStart, - ) - ) - - scheduler ! ScheduleActivation(primaryServiceProxy.toTyped, INIT_SIM_TICK) - primaryServiceProxy - } - - override def weatherService( - context: ActorContext[_], - scheduler: ActorRef[SchedulerMessage], - ): ClassicRef = { - val weatherService = context.toClassic.simonaActorOf( - WeatherService.props( - scheduler.toClassic, - TimeUtil.withDefaults - .toZonedDateTime(simonaConfig.simona.time.startDateTime), - TimeUtil.withDefaults - .toZonedDateTime(simonaConfig.simona.time.endDateTime), - ) - ) - weatherService ! SimonaService.Create( - InitWeatherServiceStateData( - simonaConfig.simona.input.weather.datasource - ), - ScheduleLock.singleKey(context, scheduler, INIT_SIM_TICK), - ) - - weatherService - } - - /* - override def extSimulations( - context: ActorContext[_], - rootScheduler: ActorRef[SchedulerMessage], - ): ExtSimSetupData = { - val jars = ExtSimLoader.scanInputFolder() - - val extLinks = jars.flatMap(ExtSimLoader.loadExtLink).toSeq - - if (extLinks.nonEmpty) { - val extScheduler = scheduler(context, parent = rootScheduler) - - val (extSimAdapters, extDatasAndServices) = - 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) - - // 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, extDataServiceToRef): ( - Iterable[ExtData], - Iterable[(Class[_], 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)) - - case (extPrimaryDataSimulation: ExtPrimaryDataSimulation, dIndex) => - val extPrimaryDataService = context.toClassic.simonaActorOf( - ExtPrimaryDataService.props(extScheduler.toClassic), - s"$index-$dIndex", - ) - val extPrimaryData = new ExtPrimaryData( - extPrimaryDataService, - extSimAdapter, - extPrimaryDataSimulation.getPrimaryDataFactory, - extPrimaryDataSimulation.getPrimaryDataAssets - ) - - extPrimaryDataSimulation.setExtPrimaryData(extPrimaryData) - - extPrimaryDataService ! SimonaService.Create( - InitExtPrimaryData(extPrimaryData), - ScheduleLock.singleKey( - context, - extScheduler, - INIT_SIM_TICK, - ), - ) - - (extPrimaryData, (classOf[ExtPrimaryDataService], extPrimaryDataService)) - - case (extResultDataSimulation: ExtResultDataSimulation, dIndex) => - - val extResultDataProvider = { - context.spawn( - ExtResultDataProvider(extScheduler), - s"$index-$dIndex", - ) - } - - implicit val timeout: PekkoTimeout = PekkoTimeout.create(5.seconds.toJava) - implicit val scheduler: Scheduler = context.system.scheduler - - val adapterRef = Await.result( - extResultDataProvider.ask[ActorRef[ResultDataMessageFromExt]] (ref => RequestDataMessageAdapter(ref)), timeout.duration) - val adapterScheduleRef = Await.result( - extResultDataProvider.ask[ActorRef[ScheduleServiceActivation]] (ref => RequestScheduleActivationAdapter(ref)), timeout.duration) - - val extResultData = - new ExtResultData( - adapterRef.toClassic, - adapterScheduleRef.toClassic, - extSimAdapter, - extResultDataSimulation.getResultDataFactory, - extResultDataSimulation.getResultDataAssets - ) - - extResultDataSimulation.setExtResultData(extResultData) - - extResultDataProvider ! ExtResultDataProvider.Create( - InitExtResultData(extResultData), - ScheduleLock.singleKey( - context, - extScheduler, - INIT_SIM_TICK, - ), - ) - - ( - extResultData, - (ExtResultDataProvider.getClass, extResultDataProvider.toClassic), - ) - }.unzip - - extLink.getExtSimulation.setup( - extSimAdapterData, - extData.toList.asJava, - ) - - // starting external simulation - new Thread(extLink.getExtSimulation, s"External simulation $index") - .start() - - (extSimAdapter, (extDataServiceToRef, extData)) - }.unzip - - val extDataServices = extDatasAndServices.map(_._1) - val extDatas = extDatasAndServices.flatMap(_._2).toSet - - ExtSimSetupData( - extSimAdapters, - extDataServices.flatten.toMap, - extDatas, - Some(extScheduler)) - } else { - ExtSimSetupData(Iterable.empty, Map.empty, Set.empty, None) - } - } - - */ - + override val typeSafeConfig: Config, + override val simonaConfig: SimonaConfig, + override val resultFileHierarchy: ResultFileHierarchy, + override val runtimeEventQueue: Option[LinkedBlockingQueue[RuntimeEvent]] = None, + override val args: Array[String], + mosaikIP: Option[String] = None, + mosaikMappingPath: Option[String] = None +) extends SimonaExtSimSetup( + typeSafeConfig, + simonaConfig, + resultFileHierarchy, + runtimeEventQueue, + args +) { override def extSimulations( context: ActorContext[_], rootScheduler: ActorRef[SchedulerMessage], simScheduler: ActorRef[SchedulerMessage] ): ExtSimSetupData = { - mosaikAsExtSimulation( - context, rootScheduler, simScheduler - ) - } - - def mosaikAsExtSimulation( - context: ActorContext[_], - rootScheduler: ActorRef[SchedulerMessage], - simScheduler: ActorRef[SchedulerMessage] - ): ExtSimSetupData = { - val mosaikAddress = mosaikIP.getOrElse("127.0.0.1:5678") val mosaikMapping = mosaikMappingPath.getOrElse(throw new RuntimeException("Cannot connect to Mosaik, because there is no mapping!")) val mosaikExtSim = new MosaikSimulation(mosaikAddress, Path.of(mosaikMapping)) - val simulationStart = TimeUtil.withDefaults.toZonedDateTime( - simonaConfig.simona.time.startDateTime - ) - - val powerFlowResolution = simonaConfig.simona.powerflow.resolution.get( - ChronoUnit.SECONDS - ) - - val extScheduler = scheduler(context, parent = rootScheduler) - - val extSimAdapterPhase1 = context.toClassic.simonaActorOf( - ExtSimAdapter.props(extScheduler.toClassic), - s"1", - ) - val extSimAdapterPhase2 = context.toClassic.simonaActorOf( - ExtSimAdapter.props(simScheduler.toClassic), - s"2", - ) - - val extSimAdapters: Map[java.lang.Integer, ClassicRef] = Map( - 1.asInstanceOf[java.lang.Integer] -> extSimAdapterPhase1, - 2.asInstanceOf[java.lang.Integer] -> extSimAdapterPhase2 - ) - - val extSimAdapterData = new ExtSimAdapterData(extSimAdapters.asJava, args) - - // -=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=- - - // --- Primary Data --- - - // send init data right away, init activation is scheduled - extSimAdapterPhase1 ! ExtSimAdapter.Create( - extSimAdapterData, - 1, - ScheduleLock.singleKey(context, extScheduler, INIT_SIM_TICK), - ) - - val extPrimaryDataService = context.toClassic.simonaActorOf( - ExtPrimaryDataService.props(extScheduler.toClassic), - s"0-0", - ) - - val extPrimaryData = mosaikExtSim.getExtPrimaryData - - extPrimaryData.setActorRefs( - extPrimaryDataService, - extSimAdapterPhase1 - ) - - extPrimaryDataService ! SimonaService.Create( - InitExtPrimaryData(extPrimaryData), - ScheduleLock.singleKey( - context, - extScheduler, - INIT_SIM_TICK, - ), - ) - - // -=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=- - - //--- Result Data --- - - val extResultDataProvider = { - context.spawn( - ExtResultDataProvider(simScheduler), - s"ExtResultDataProvider", - ) - } - - val timeout: PekkoTimeout = PekkoTimeout.create(5.seconds.toJava) - val scheduler2: Scheduler = context.system.scheduler - - val adapterRef = Await.result( - extResultDataProvider.ask[ActorRef[ResultDataMessageFromExt]] (ref => RequestDataMessageAdapter(ref))(timeout, scheduler2), timeout.duration) - val adapterScheduleRef = Await.result( - extResultDataProvider.ask[ActorRef[ScheduleServiceActivation]] (ref => RequestScheduleActivationAdapter(ref))(timeout, scheduler2), timeout.duration) - - val extResultData = mosaikExtSim.getExtResultData - - extResultData.setActorRefs( - adapterRef.toClassic, - adapterScheduleRef.toClassic, - extSimAdapterPhase2 - ) - - extResultData.setSimulationData( - simulationStart, - powerFlowResolution - ) - - extResultDataProvider ! ExtResultDataProvider.Create( - InitExtResultData(extResultData), - ScheduleLock.singleKey( - context, - simScheduler, - INIT_SIM_TICK, - ), - ) - - - val simpleExtSimDatas: List[ExtData] = List( - extResultData, - extPrimaryData - ) - - mosaikExtSim.setup( - extSimAdapterData, - simpleExtSimDatas.asJava, - ) - // starting external simulation - new Thread(mosaikExtSim, s"External simulation") - .start() - - - - val extDataServicesMap: Map[Class[_], ClassicRef] = Map( - classOf[ExtPrimaryDataService] -> extPrimaryDataService, - ) - - val extDataListenerMap: Map[Class[_], ActorRef[ExtResultDataProvider.Request]] = Map( - ExtResultDataProvider.getClass -> extResultDataProvider - ) - - val extSimAdaptersIt = Iterable(extSimAdapterPhase1, extSimAdapterPhase2) - - val extDatas = simpleExtSimDatas.toSet - extSimAdapterPhase2 ! ExtSimAdapter.Create( - extSimAdapterData, - 2, - ScheduleLock.singleKey(context, simScheduler, INIT_SIM_TICK), - ) - ExtSimSetupData(extSimAdaptersIt, extDataServicesMap, extDataListenerMap, extDatas, Some(extScheduler)) - } - - override def timeAdvancer( - context: ActorContext[_], - simulation: ActorRef[SimonaSim.SimulationEnded.type], - runtimeEventListener: ActorRef[RuntimeEvent], - ): ActorRef[TimeAdvancer.Request] = { - val startDateTime = TimeUtil.withDefaults.toZonedDateTime( - simonaConfig.simona.time.startDateTime - ) - val endDateTime = TimeUtil.withDefaults.toZonedDateTime( - simonaConfig.simona.time.endDateTime - ) - - context.spawn( - TimeAdvancer( - simulation, - Some(runtimeEventListener), - simonaConfig.simona.time.schedulerReadyCheckWindow, - endDateTime.toTick(startDateTime), - ), - TimeAdvancer.getClass.getSimpleName, + extSimulationSetup( + context, + rootScheduler, + simScheduler, + mosaikExtSim ) } - - override def scheduler( - context: ActorContext[_], - parent: ActorRef[SchedulerMessage], - coreFactory: CoreFactory = RegularSchedulerCore, - ): ActorRef[SchedulerMessage] = - context - .spawn( - Scheduler(parent, coreFactory), - s"${Scheduler.getClass.getSimpleName}_${coreFactory}_${UUID.randomUUID()}", - ) - - override def runtimeEventListener( - context: ActorContext[_] - ): ActorRef[RuntimeEventListener.Request] = - context - .spawn( - RuntimeEventListener( - simonaConfig.simona.runtime.listener, - runtimeEventQueue, - startDateTimeString = simonaConfig.simona.time.startDateTime, - ), - RuntimeEventListener.getClass.getSimpleName, - ) - - override def resultEventListener( - context: ActorContext[_], - extSimulationData: ExtSimSetupData, - ): Seq[ActorRef[ResultEventListener.Request]] = { - val extResultDataService: Option[ActorRef[ExtResultDataProvider.Request]] = - extSimulationData.extResultDataService - // append ResultEventListener as well to write raw output files - ArgsParser - .parseListenerConfigOption(simonaConfig.simona.event.listener) - .zipWithIndex - .map { case ((listenerCompanion, events), index) => - context.toClassic - .simonaActorOf( - listenerCompanion.props(events), - index.toString, - ) - .toTyped - } - .toSeq :+ context - .spawn( - ResultEventListener( - resultFileHierarchy, - extResultDataService, - ), - ResultEventListener.getClass.getSimpleName, - ) - } - - def buildSubGridToActorRefMap( - subGridTopologyGraph: SubGridTopologyGraph, - context: ActorContext[_], - environmentRefs: EnvironmentRefs, - resultEventListeners: Seq[ActorRef[ResultEvent]], - ): Map[Int, ActorRef[GridAgent.Request]] = { - subGridTopologyGraph - .vertexSet() - .asScala - .map(subGridContainer => { - val gridAgentRef = - context.spawn( - GridAgent( - environmentRefs, - simonaConfig, - resultEventListeners, - ), - subGridContainer.getSubnet.toString, - ) - subGridContainer.getSubnet -> gridAgentRef - }) - .toMap - } - - /** Get all thermal grids, that apply for the given grid container - * @param grid - * The grid container to assess - * @param thermalGridByBus - * Mapping from thermal bus to thermal grid - * @return - * A sequence of applicable thermal grids - */ - private def getThermalGrids( - grid: GridContainer, - thermalGridByBus: Map[ThermalBusInput, ThermalGrid], - ): Seq[ThermalGrid] = { - grid.getSystemParticipants.getHeatPumps.asScala - .flatten(hpInput => thermalGridByBus.get(hpInput.getThermalBus)) - .toSeq - } } /** Companion object to provide [[SetupHelper]] methods for diff --git a/src/main/scala/edu/ie3/simona/sim/setup/SimonaOpsimSetup.scala b/src/main/scala/edu/ie3/simona/sim/setup/SimonaOpsimSetup.scala index 5f3d5a52e9..14356a8199 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaOpsimSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaOpsimSetup.scala @@ -8,60 +8,16 @@ package edu.ie3.simona.sim.setup import com.typesafe.config.Config import com.typesafe.scalalogging.LazyLogging -import edu.ie3.datamodel.graph.SubGridTopologyGraph -import edu.ie3.datamodel.models.input.container.{GridContainer, ThermalGrid} -import edu.ie3.datamodel.models.input.thermal.ThermalBusInput -import edu.ie3.simona.actor.SimonaActorNaming.RichActorRefFactory -import edu.ie3.simona.agent.EnvironmentRefs -import edu.ie3.simona.agent.grid.GridAgent -import edu.ie3.simona.agent.grid.GridAgentMessages.CreateGridAgent -import edu.ie3.simona.api.ExtSimAdapter -import edu.ie3.simona.api.data.ExtData -import edu.ie3.simona.api.data.em.ExtEmData -import edu.ie3.simona.api.data.results.ExtResultData -import edu.ie3.simona.api.data.results.ontology.ResultDataMessageFromExt -import edu.ie3.simona.api.simulation.ExtSimAdapterData -import edu.ie3.simona.config.{ArgsParser, RefSystemParser, SimonaConfig} -import edu.ie3.simona.event.listener.{ResultEventListener, RuntimeEventListener} -import edu.ie3.simona.event.{ResultEvent, RuntimeEvent} -import edu.ie3.simona.exceptions.agent.GridAgentInitializationException -import edu.ie3.simona.io.grid.GridProvider +import edu.ie3.simona.config.SimonaConfig +import edu.ie3.simona.event.RuntimeEvent import edu.ie3.simona.ontology.messages.SchedulerMessage -import edu.ie3.simona.ontology.messages.SchedulerMessage.ScheduleActivation -import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.ScheduleServiceActivation -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.em.ExtEmDataService -import edu.ie3.simona.service.em.ExtEmDataService.InitExtEmData -import edu.ie3.simona.service.primary.PrimaryServiceProxy -import edu.ie3.simona.service.primary.PrimaryServiceProxy.InitPrimaryServiceProxyStateData -import edu.ie3.simona.service.results.ExtResultDataProvider -import edu.ie3.simona.service.results.ExtResultDataProvider.{InitExtResultData, RequestDataMessageAdapter, RequestScheduleActivationAdapter} -import edu.ie3.simona.service.weather.WeatherService -import edu.ie3.simona.service.weather.WeatherService.InitWeatherServiceStateData -import edu.ie3.simona.sim.SimonaSim import edu.ie3.simona.util.ResultFileHierarchy -import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK -import edu.ie3.simona.util.TickUtil.RichZonedDateTime import edu.ie3.simopsim.OpsimEmSimulation -import edu.ie3.util.TimeUtil +import org.apache.pekko.actor.typed.ActorRef import org.apache.pekko.actor.typed.scaladsl.ActorContext -import org.apache.pekko.actor.typed.scaladsl.AskPattern._ -import org.apache.pekko.actor.typed.scaladsl.adapter.{ClassicActorRefOps, TypedActorContextOps, TypedActorRefOps} -import org.apache.pekko.actor.typed.{ActorRef, Scheduler} -import org.apache.pekko.actor.{ActorRef => ClassicRef} -import org.apache.pekko.util.{Timeout => PekkoTimeout} import java.nio.file.Path -import java.time.temporal.ChronoUnit -import java.util.UUID import java.util.concurrent.LinkedBlockingQueue -import scala.concurrent.Await -import scala.concurrent.duration.DurationInt -import scala.jdk.CollectionConverters._ -import scala.jdk.DurationConverters._ /** Sample implementation to run a standalone simulation of simona configured * with the provided [[SimonaConfig]] and [[ResultFileHierarchy]] @@ -70,545 +26,42 @@ import scala.jdk.DurationConverters._ * @since 01.07.20 */ class SimonaOpsimSetup( - val typeSafeConfig: Config, - simonaConfig: SimonaConfig, - resultFileHierarchy: ResultFileHierarchy, - runtimeEventQueue: Option[LinkedBlockingQueue[RuntimeEvent]] = None, - override val args: Array[String], - opsimIP: Option[String] = None, - opsimMappingPath: Option[String] = None, -) extends SimonaSetup { - - override def gridAgents( - context: ActorContext[_], - environmentRefs: EnvironmentRefs, - resultEventListeners: Seq[ActorRef[ResultEvent]], - ): Iterable[ActorRef[GridAgent.Request]] = { - - /* get the grid */ - val subGridTopologyGraph = GridProvider - .gridFromConfig( - simonaConfig.simona.simulationName, - simonaConfig.simona.input.grid.datasource, - ) - .getSubGridTopologyGraph - val thermalGridsByThermalBus = GridProvider.getThermalGridsFromConfig( - simonaConfig.simona.input.grid.datasource - ) - - /* extract and prepare refSystem information from config */ - val configRefSystems = - RefSystemParser.parse(simonaConfig.simona.gridConfig.refSystems) - - /* Create all agents and map the sub grid id to their actor references */ - val subGridToActorRefMap = buildSubGridToActorRefMap( - subGridTopologyGraph, - context, - environmentRefs, - resultEventListeners, - ) - - val keys = ScheduleLock.multiKey( - context, - environmentRefs.scheduler, - INIT_SIM_TICK, - subGridTopologyGraph.vertexSet().size, - ) - - /* build the initialization data */ - subGridTopologyGraph - .vertexSet() - .asScala - .zip(keys) - .map { case (subGridContainer, key) => - /* Get all connections to superior and inferior sub grids */ - val subGridGates = - Set.from( - subGridTopologyGraph - .edgesOf(subGridContainer) - .asScala - .map(modifySubGridGateForThreeWindingSupport) - ) - val currentSubGrid = subGridContainer.getSubnet - val currentActorRef = subGridToActorRefMap.getOrElse( - currentSubGrid, - throw new GridAgentInitializationException( - "Was asked to setup agent for sub grid " + currentSubGrid + ", but did not found it's actor reference." - ), - ) - val thermalGrids = - getThermalGrids(subGridContainer, thermalGridsByThermalBus) - - /* build the grid agent data and check for its validity */ - val gridAgentInitData = SimonaStandaloneSetup.buildGridAgentInitData( - subGridContainer, - subGridToActorRefMap, - subGridGates, - configRefSystems, - thermalGrids, - ) - - currentActorRef ! CreateGridAgent(gridAgentInitData, key) - - currentActorRef - } - } - - override def primaryServiceProxy( - context: ActorContext[_], - scheduler: ActorRef[SchedulerMessage], - extSimSetupData: ExtSimSetupData, - ): ClassicRef = { - val simulationStart = TimeUtil.withDefaults.toZonedDateTime( - simonaConfig.simona.time.startDateTime - ) - val primaryServiceProxy = context.toClassic.simonaActorOf( - PrimaryServiceProxy.props( - scheduler.toClassic, - InitPrimaryServiceProxyStateData( - simonaConfig.simona.input.primary, - simulationStart, - extSimSetupData.extPrimaryDataService, - extSimSetupData.extPrimaryData - ), - simulationStart, - ) - ) - - scheduler ! ScheduleActivation(primaryServiceProxy.toTyped, INIT_SIM_TICK) - primaryServiceProxy - } - - override def weatherService( - context: ActorContext[_], - scheduler: ActorRef[SchedulerMessage], - ): ClassicRef = { - val weatherService = context.toClassic.simonaActorOf( - WeatherService.props( - scheduler.toClassic, - TimeUtil.withDefaults - .toZonedDateTime(simonaConfig.simona.time.startDateTime), - TimeUtil.withDefaults - .toZonedDateTime(simonaConfig.simona.time.endDateTime), - ) - ) - weatherService ! SimonaService.Create( - InitWeatherServiceStateData( - simonaConfig.simona.input.weather.datasource - ), - ScheduleLock.singleKey(context, scheduler, INIT_SIM_TICK), - ) - - weatherService - } - - /* - override def extSimulations( - context: ActorContext[_], - rootScheduler: ActorRef[SchedulerMessage], - ): ExtSimSetupData = { - val jars = ExtSimLoader.scanInputFolder() - - val extLinks = jars.flatMap(ExtSimLoader.loadExtLink).toSeq - - if (extLinks.nonEmpty) { - val extScheduler = scheduler(context, parent = rootScheduler) - - val (extSimAdapters, extDatasAndServices) = - 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) - - // 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, extDataServiceToRef): ( - Iterable[ExtData], - Iterable[(Class[_], 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)) - - case (extPrimaryDataSimulation: ExtPrimaryDataSimulation, dIndex) => - val extPrimaryDataService = context.toClassic.simonaActorOf( - ExtPrimaryDataService.props(extScheduler.toClassic), - s"$index-$dIndex", - ) - val extPrimaryData = new ExtPrimaryData( - extPrimaryDataService, - extSimAdapter, - extPrimaryDataSimulation.getPrimaryDataFactory, - extPrimaryDataSimulation.getPrimaryDataAssets - ) - - extPrimaryDataSimulation.setExtPrimaryData(extPrimaryData) - - extPrimaryDataService ! SimonaService.Create( - InitExtPrimaryData(extPrimaryData), - ScheduleLock.singleKey( - context, - extScheduler, - INIT_SIM_TICK, - ), - ) - - (extPrimaryData, (classOf[ExtPrimaryDataService], extPrimaryDataService)) - - case (extResultDataSimulation: ExtResultDataSimulation, dIndex) => - - val extResultDataProvider = { - context.spawn( - ExtResultDataProvider(extScheduler), - s"$index-$dIndex", - ) - } - - implicit val timeout: PekkoTimeout = PekkoTimeout.create(5.seconds.toJava) - implicit val scheduler: Scheduler = context.system.scheduler - - val adapterRef = Await.result( - extResultDataProvider.ask[ActorRef[ResultDataMessageFromExt]] (ref => RequestDataMessageAdapter(ref)), timeout.duration) - val adapterScheduleRef = Await.result( - extResultDataProvider.ask[ActorRef[ScheduleServiceActivation]] (ref => RequestScheduleActivationAdapter(ref)), timeout.duration) - - val extResultData = - new ExtResultData( - adapterRef.toClassic, - adapterScheduleRef.toClassic, - extSimAdapter, - extResultDataSimulation.getResultDataFactory, - extResultDataSimulation.getResultDataAssets - ) - - extResultDataSimulation.setExtResultData(extResultData) - - extResultDataProvider ! ExtResultDataProvider.Create( - InitExtResultData(extResultData), - ScheduleLock.singleKey( - context, - extScheduler, - INIT_SIM_TICK, - ), - ) - - ( - extResultData, - (ExtResultDataProvider.getClass, extResultDataProvider.toClassic), - ) - }.unzip - - extLink.getExtSimulation.setup( - extSimAdapterData, - extData.toList.asJava, - ) - - // starting external simulation - new Thread(extLink.getExtSimulation, s"External simulation $index") - .start() - - (extSimAdapter, (extDataServiceToRef, extData)) - }.unzip - - val extDataServices = extDatasAndServices.map(_._1) - val extDatas = extDatasAndServices.flatMap(_._2).toSet - - ExtSimSetupData( - extSimAdapters, - extDataServices.flatten.toMap, - extDatas, - Some(extScheduler)) - } else { - ExtSimSetupData(Iterable.empty, Map.empty, Set.empty, None) - } - } - - */ - - override def extSimulations( - context: ActorContext[_], - rootScheduler: ActorRef[SchedulerMessage], - simScheduler: ActorRef[SchedulerMessage], - ): ExtSimSetupData = { - opsimSimulationWithEm( - context, rootScheduler, simScheduler - ) - } - - def opsimSimulationWithEm( - context: ActorContext[_], - rootScheduler: ActorRef[SchedulerMessage], - simScheduler: ActorRef[SchedulerMessage], - ): ExtSimSetupData = { - val opsimAddress = opsimIP.getOrElse(throw new RuntimeException("Cannot connect to Opsim, because there is no address!")) - val opsimMapping = opsimMappingPath.getOrElse(throw new RuntimeException("Cannot connect to Opsim, because there is no mapping!")) - val simulationStart = TimeUtil.withDefaults.toZonedDateTime( - simonaConfig.simona.time.startDateTime - ) - val powerFlowResolution = simonaConfig.simona.powerflow.resolution.get( - ChronoUnit.SECONDS - ) - val extScheduler = scheduler(context, parent = rootScheduler) - val opsimSim = new OpsimEmSimulation(opsimAddress, Path.of(opsimMapping)) - - val extSimAdapterPhase1 = context.toClassic.simonaActorOf( - ExtSimAdapter.props(extScheduler.toClassic), - s"1", - ) - val extSimAdapterPhase2 = context.toClassic.simonaActorOf( - ExtSimAdapter.props(simScheduler.toClassic), - s"2", - ) - - val extSimAdapters: Map[java.lang.Integer, ClassicRef] = Map( - 1.asInstanceOf[java.lang.Integer] -> extSimAdapterPhase1, - 2.asInstanceOf[java.lang.Integer] -> extSimAdapterPhase2 - ) - - val extSimAdapterData = new ExtSimAdapterData(extSimAdapters.asJava, args) - - - - // send init data right away, init activation is scheduled - extSimAdapterPhase1 ! ExtSimAdapter.Create( - extSimAdapterData, - 1, - ScheduleLock.singleKey(context, extScheduler, INIT_SIM_TICK), - ) - - val extEmDataService = context.toClassic.simonaActorOf( - ExtEmDataService.props(extScheduler.toClassic), - s"0-0", - ) - val extEmData = opsimSim.getExtEmData - - extEmData.setActorRefs( - extEmDataService, - extSimAdapterPhase1 - ) - - extEmDataService ! SimonaService.Create( - InitExtEmData(extEmData), - ScheduleLock.singleKey( - context, - extScheduler, - INIT_SIM_TICK, - ), - ) - - //Result Data - - val extResultDataProvider = { - context.spawn( - ExtResultDataProvider(simScheduler), - s"ExtResultDataProvider", - ) - } - - - val timeout: PekkoTimeout = PekkoTimeout.create(5.seconds.toJava) - val scheduler2: Scheduler = context.system.scheduler - - val adapterRef = Await.result( - extResultDataProvider.ask[ActorRef[ResultDataMessageFromExt]] (ref => RequestDataMessageAdapter(ref))(timeout, scheduler2), timeout.duration) - val adapterScheduleRef = Await.result( - extResultDataProvider.ask[ActorRef[ScheduleServiceActivation]] (ref => RequestScheduleActivationAdapter(ref))(timeout, scheduler2), timeout.duration) - - val extResultData = opsimSim.getExtResultData - - extResultData.setActorRefs( - adapterRef.toClassic, - adapterScheduleRef.toClassic, - extSimAdapterPhase2 - ) - - extResultData.setSimulationData( - simulationStart, - powerFlowResolution - ) - - extResultDataProvider ! ExtResultDataProvider.Create( - InitExtResultData(extResultData), - ScheduleLock.singleKey( + override val typeSafeConfig: Config, + override val simonaConfig: SimonaConfig, + override val resultFileHierarchy: ResultFileHierarchy, + override val runtimeEventQueue: Option[LinkedBlockingQueue[RuntimeEvent]] = None, + override val args: Array[String], + opsimIP: Option[String] = None, + opsimMappingPath: Option[String] = None, +) extends SimonaExtSimSetup( + typeSafeConfig, + simonaConfig, + resultFileHierarchy, + runtimeEventQueue, + args +) { + + override def extSimulations( + context: ActorContext[_], + rootScheduler: ActorRef[SchedulerMessage], + simScheduler: ActorRef[SchedulerMessage], + ): ExtSimSetupData = { + // OpSim Simulation + val opsimAddress = opsimIP.getOrElse(throw new RuntimeException("Cannot connect to Opsim, because there is no address!")) + val opsimMapping = opsimMappingPath.getOrElse(throw new RuntimeException("Cannot connect to Opsim, because there is no mapping!")) + + val opsimSim = new OpsimEmSimulation(opsimAddress, Path.of(opsimMapping)) + + extSimulationSetup( context, + rootScheduler, simScheduler, - INIT_SIM_TICK, - ), - ) - - - val simpleExtSimDatas: List[ExtData] = List( - extResultData, - extEmData - ) - - opsimSim.setup( - extSimAdapterData, - simpleExtSimDatas.asJava, - ) - // starting external simulation - new Thread(opsimSim, s"External simulation") - .start() - - val extDataServicesMap: Map[Class[_], ClassicRef] = Map( - classOf[ExtEmDataService] -> extEmDataService, - ) - - val extDataListenerMap: Map[Class[_], ActorRef[ExtResultDataProvider.Request]] = Map( - ExtResultDataProvider.getClass -> extResultDataProvider - ) - - val extSimAdaptersIt = Iterable(extSimAdapterPhase1, extSimAdapterPhase2) - - val extDatas = simpleExtSimDatas.toSet - extSimAdapterPhase2 ! ExtSimAdapter.Create( - extSimAdapterData, - 2, - ScheduleLock.singleKey(context, simScheduler, INIT_SIM_TICK), - ) - ExtSimSetupData(extSimAdaptersIt, extDataServicesMap, extDataListenerMap, extDatas, Some(extScheduler)) - } - - override def timeAdvancer( - context: ActorContext[_], - simulation: ActorRef[SimonaSim.SimulationEnded.type], - runtimeEventListener: ActorRef[RuntimeEvent], - ): ActorRef[TimeAdvancer.Request] = { - val startDateTime = TimeUtil.withDefaults.toZonedDateTime( - simonaConfig.simona.time.startDateTime - ) - val endDateTime = TimeUtil.withDefaults.toZonedDateTime( - simonaConfig.simona.time.endDateTime - ) - - context.spawn( - TimeAdvancer( - simulation, - Some(runtimeEventListener), - simonaConfig.simona.time.schedulerReadyCheckWindow, - endDateTime.toTick(startDateTime), - ), - TimeAdvancer.getClass.getSimpleName, - ) - } - - override def scheduler( - context: ActorContext[_], - parent: ActorRef[SchedulerMessage], - coreFactory: CoreFactory = RegularSchedulerCore, - ): ActorRef[SchedulerMessage] = - context - .spawn( - Scheduler(parent, coreFactory), - s"${Scheduler.getClass.getSimpleName}_${coreFactory}_${UUID.randomUUID()}", - ) - - override def runtimeEventListener( - context: ActorContext[_] - ): ActorRef[RuntimeEventListener.Request] = - context - .spawn( - RuntimeEventListener( - simonaConfig.simona.runtime.listener, - runtimeEventQueue, - startDateTimeString = simonaConfig.simona.time.startDateTime, - ), - RuntimeEventListener.getClass.getSimpleName, - ) - - override def resultEventListener( - context: ActorContext[_], - extSimulationData: ExtSimSetupData, - ): Seq[ActorRef[ResultEventListener.Request]] = { - val extResultDataService: Option[ActorRef[ExtResultDataProvider.Request]] = - extSimulationData.extResultDataService - // append ResultEventListener as well to write raw output files - ArgsParser - .parseListenerConfigOption(simonaConfig.simona.event.listener) - .zipWithIndex - .map { case ((listenerCompanion, events), index) => - context.toClassic - .simonaActorOf( - listenerCompanion.props(events), - index.toString, - ) - .toTyped - } - .toSeq :+ context - .spawn( - ResultEventListener( - resultFileHierarchy, - extResultDataService, - ), - ResultEventListener.getClass.getSimpleName, + opsimSim ) - } - - def buildSubGridToActorRefMap( - subGridTopologyGraph: SubGridTopologyGraph, - context: ActorContext[_], - environmentRefs: EnvironmentRefs, - resultEventListeners: Seq[ActorRef[ResultEvent]], - ): Map[Int, ActorRef[GridAgent.Request]] = { - subGridTopologyGraph - .vertexSet() - .asScala - .map(subGridContainer => { - val gridAgentRef = - context.spawn( - GridAgent( - environmentRefs, - simonaConfig, - resultEventListeners, - ), - subGridContainer.getSubnet.toString, - ) - subGridContainer.getSubnet -> gridAgentRef - }) - .toMap - } - - /** Get all thermal grids, that apply for the given grid container - * @param grid - * The grid container to assess - * @param thermalGridByBus - * Mapping from thermal bus to thermal grid - * @return - * A sequence of applicable thermal grids - */ - private def getThermalGrids( - grid: GridContainer, - thermalGridByBus: Map[ThermalBusInput, ThermalGrid], - ): Seq[ThermalGrid] = { - grid.getSystemParticipants.getHeatPumps.asScala - .flatten(hpInput => thermalGridByBus.get(hpInput.getThermalBus)) - .toSeq - } + } } + /** Companion object to provide [[SetupHelper]] methods for * [[SimonaStandaloneSetup]] */ diff --git a/src/main/scala/edu/ie3/simona/sim/setup/SimonaSimpleExtSimulationSetup.scala b/src/main/scala/edu/ie3/simona/sim/setup/SimonaSimpleExtSimulationSetup.scala index 4804435c76..18bbfca34a 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaSimpleExtSimulationSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaSimpleExtSimulationSetup.scala @@ -35,6 +35,7 @@ import edu.ie3.simona.sim.SimonaSim import edu.ie3.simona.util.ResultFileHierarchy import edu.ie3.simona.util.SimonaConstants.INIT_SIM_TICK import edu.ie3.simona.util.TickUtil.RichZonedDateTime +import edu.ie3.simpleextsim.SimpleExtSimulationWithEm import edu.ie3.util.TimeUtil import org.apache.pekko.actor.typed.ActorRef import org.apache.pekko.actor.typed.scaladsl.ActorContext @@ -52,547 +53,33 @@ import scala.jdk.CollectionConverters._ * @since 01.07.20 */ class SimonaSimpleExtSimulationSetup( - val typeSafeConfig: Config, - simonaConfig: SimonaConfig, - resultFileHierarchy: ResultFileHierarchy, - runtimeEventQueue: Option[LinkedBlockingQueue[RuntimeEvent]] = None, - override val args: Array[String], -) extends SimonaSetup { - - override def gridAgents( - context: ActorContext[_], - environmentRefs: EnvironmentRefs, - resultEventListeners: Seq[ActorRef[ResultEvent]], - ): Iterable[ActorRef[GridAgent.Request]] = { - - /* get the grid */ - val subGridTopologyGraph = GridProvider - .gridFromConfig( - simonaConfig.simona.simulationName, - simonaConfig.simona.input.grid.datasource, - ) - .getSubGridTopologyGraph - val thermalGridsByThermalBus = GridProvider.getThermalGridsFromConfig( - simonaConfig.simona.input.grid.datasource - ) - - /* extract and prepare refSystem information from config */ - val configRefSystems = - RefSystemParser.parse(simonaConfig.simona.gridConfig.refSystems) - - /* Create all agents and map the sub grid id to their actor references */ - val subGridToActorRefMap = buildSubGridToActorRefMap( - subGridTopologyGraph, - context, - environmentRefs, - resultEventListeners, - ) - - val keys = ScheduleLock.multiKey( - context, - environmentRefs.scheduler, - INIT_SIM_TICK, - subGridTopologyGraph.vertexSet().size, - ) - - /* build the initialization data */ - subGridTopologyGraph - .vertexSet() - .asScala - .zip(keys) - .map { case (subGridContainer, key) => - /* Get all connections to superior and inferior sub grids */ - val subGridGates = - Set.from( - subGridTopologyGraph - .edgesOf(subGridContainer) - .asScala - .map(modifySubGridGateForThreeWindingSupport) - ) - val currentSubGrid = subGridContainer.getSubnet - val currentActorRef = subGridToActorRefMap.getOrElse( - currentSubGrid, - throw new GridAgentInitializationException( - "Was asked to setup agent for sub grid " + currentSubGrid + ", but did not found it's actor reference." - ), - ) - val thermalGrids = - getThermalGrids(subGridContainer, thermalGridsByThermalBus) - - /* build the grid agent data and check for its validity */ - val gridAgentInitData = SimonaStandaloneSetup.buildGridAgentInitData( - subGridContainer, - subGridToActorRefMap, - subGridGates, - configRefSystems, - thermalGrids, - ) - - currentActorRef ! CreateGridAgent(gridAgentInitData, key) - - currentActorRef - } - } - - override def primaryServiceProxy( - context: ActorContext[_], - scheduler: ActorRef[SchedulerMessage], - extSimSetupData: ExtSimSetupData, - ): ClassicRef = { - val simulationStart = TimeUtil.withDefaults.toZonedDateTime( - simonaConfig.simona.time.startDateTime - ) - val primaryServiceProxy = context.toClassic.simonaActorOf( - PrimaryServiceProxy.props( - scheduler.toClassic, - InitPrimaryServiceProxyStateData( - simonaConfig.simona.input.primary, - simulationStart, - extSimSetupData.extPrimaryDataService, - extSimSetupData.extPrimaryData - ), - simulationStart, - ) - ) - - scheduler ! ScheduleActivation(primaryServiceProxy.toTyped, INIT_SIM_TICK) - primaryServiceProxy - } - - override def weatherService( - context: ActorContext[_], - scheduler: ActorRef[SchedulerMessage], - ): ClassicRef = { - val weatherService = context.toClassic.simonaActorOf( - WeatherService.props( - scheduler.toClassic, - TimeUtil.withDefaults - .toZonedDateTime(simonaConfig.simona.time.startDateTime), - TimeUtil.withDefaults - .toZonedDateTime(simonaConfig.simona.time.endDateTime), - ) - ) - weatherService ! SimonaService.Create( - InitWeatherServiceStateData( - simonaConfig.simona.input.weather.datasource - ), - ScheduleLock.singleKey(context, scheduler, INIT_SIM_TICK), - ) - - weatherService - } - - /* - override def extSimulations( - context: ActorContext[_], - rootScheduler: ActorRef[SchedulerMessage], - ): ExtSimSetupData = { - val jars = ExtSimLoader.scanInputFolder() - - val extLinks = jars.flatMap(ExtSimLoader.loadExtLink).toSeq - - if (extLinks.nonEmpty) { - val extScheduler = scheduler(context, parent = rootScheduler) - - val (extSimAdapters, extDatasAndServices) = - 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) - - // 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, extDataServiceToRef): ( - Iterable[ExtData], - Iterable[(Class[_], 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)) - - case (extPrimaryDataSimulation: ExtPrimaryDataSimulation, dIndex) => - val extPrimaryDataService = context.toClassic.simonaActorOf( - ExtPrimaryDataService.props(extScheduler.toClassic), - s"$index-$dIndex", - ) - val extPrimaryData = new ExtPrimaryData( - extPrimaryDataService, - extSimAdapter, - extPrimaryDataSimulation.getPrimaryDataFactory, - extPrimaryDataSimulation.getPrimaryDataAssets - ) - - extPrimaryDataSimulation.setExtPrimaryData(extPrimaryData) - - extPrimaryDataService ! SimonaService.Create( - InitExtPrimaryData(extPrimaryData), - ScheduleLock.singleKey( - context, - extScheduler, - INIT_SIM_TICK, - ), - ) - - (extPrimaryData, (classOf[ExtPrimaryDataService], extPrimaryDataService)) - - case (extResultDataSimulation: ExtResultDataSimulation, dIndex) => - - val extResultDataProvider = { - context.spawn( - ExtResultDataProvider(extScheduler), - s"$index-$dIndex", - ) - } - - implicit val timeout: PekkoTimeout = PekkoTimeout.create(5.seconds.toJava) - implicit val scheduler: Scheduler = context.system.scheduler - - val adapterRef = Await.result( - extResultDataProvider.ask[ActorRef[ResultDataMessageFromExt]] (ref => RequestDataMessageAdapter(ref)), timeout.duration) - val adapterScheduleRef = Await.result( - extResultDataProvider.ask[ActorRef[ScheduleServiceActivation]] (ref => RequestScheduleActivationAdapter(ref)), timeout.duration) - - val extResultData = - new ExtResultData( - adapterRef.toClassic, - adapterScheduleRef.toClassic, - extSimAdapter, - extResultDataSimulation.getResultDataFactory, - extResultDataSimulation.getResultDataAssets - ) - - extResultDataSimulation.setExtResultData(extResultData) - - extResultDataProvider ! ExtResultDataProvider.Create( - InitExtResultData(extResultData), - ScheduleLock.singleKey( - context, - extScheduler, - INIT_SIM_TICK, - ), - ) - - ( - extResultData, - (ExtResultDataProvider.getClass, extResultDataProvider.toClassic), - ) - }.unzip - - extLink.getExtSimulation.setup( - extSimAdapterData, - extData.toList.asJava, - ) - - // starting external simulation - new Thread(extLink.getExtSimulation, s"External simulation $index") - .start() - - (extSimAdapter, (extDataServiceToRef, extData)) - }.unzip - - val extDataServices = extDatasAndServices.map(_._1) - val extDatas = extDatasAndServices.flatMap(_._2).toSet - - ExtSimSetupData( - extSimAdapters, - extDataServices.flatten.toMap, - extDatas, - Some(extScheduler)) - } else { - ExtSimSetupData(Iterable.empty, Map.empty, Set.empty, None) - } - } - - */ + override val typeSafeConfig: Config, + override val simonaConfig: SimonaConfig, + override val resultFileHierarchy: ResultFileHierarchy, + override val runtimeEventQueue: Option[LinkedBlockingQueue[RuntimeEvent]] = None, + override val args: Array[String], +) extends SimonaExtSimSetup( + typeSafeConfig, + simonaConfig, + resultFileHierarchy, + runtimeEventQueue, + args +) { override def extSimulations( context: ActorContext[_], rootScheduler: ActorRef[SchedulerMessage], simScheduler: ActorRef[SchedulerMessage], ): ExtSimSetupData = { - simpleExtSimulationWithEm( - context, rootScheduler, simScheduler - ) - } - - def simpleExtSimulationWithEm( - context: ActorContext[_], - rootScheduler: ActorRef[SchedulerMessage], - simScheduler: ActorRef[SchedulerMessage], - ): ExtSimSetupData = { - /* - val simulationStart = TimeUtil.withDefaults.toZonedDateTime( - simonaConfig.simona.time.startDateTime - ) - val powerFlowResolution = simonaConfig.simona.powerflow.resolution.get( - ChronoUnit.SECONDS - ) - val extScheduler = scheduler(context, parent = rootScheduler) val simpleExtSim = new SimpleExtSimulationWithEm() - val extSimAdapterPhase1 = context.toClassic.simonaActorOf( - ExtSimAdapter.props(extScheduler.toClassic), - s"1", - ) - val extSimAdapterPhase2 = context.toClassic.simonaActorOf( - ExtSimAdapter.props(simScheduler.toClassic), - s"2", - ) - - val extSimAdapters: Map[java.lang.Integer, ClassicRef] = Map( - 1.asInstanceOf[java.lang.Integer] -> extSimAdapterPhase1, - 2.asInstanceOf[java.lang.Integer] -> extSimAdapterPhase2 - ) - - val extSimAdapterData = new ExtSimAdapterData(extSimAdapters.asJava, args) - - - - // send init data right away, init activation is scheduled - extSimAdapterPhase1 ! ExtSimAdapter.Create( - extSimAdapterData, - 1, - ScheduleLock.singleKey(context, extScheduler, INIT_SIM_TICK), - ) - - val extEmDataService = context.toClassic.simonaActorOf( - ExtEmDataService.props(extScheduler.toClassic), - s"0-0", - ) - val extEmData = new ExtEmData( - extEmDataService, - extSimAdapterPhase1, - simpleExtSim.getExtEmDataSimulation.getEmDataFactory, - simpleExtSim.getExtEmDataSimulation.getControlledEms - ) - - simpleExtSim.getExtEmDataSimulation.setExtEmData(extEmData) - - extEmDataService ! SimonaService.Create( - InitExtEmData(extEmData), - ScheduleLock.singleKey( - context, - extScheduler, - INIT_SIM_TICK, - ), - ) - - //Result Data - - val extResultDataProvider = { - context.spawn( - ExtResultDataProvider(simScheduler), - s"ExtResultDataProvider", - ) - } - - - val timeout: PekkoTimeout = PekkoTimeout.create(5.seconds.toJava) - val scheduler2: Scheduler = context.system.scheduler - - val adapterRef = Await.result( - extResultDataProvider.ask[ActorRef[ResultDataMessageFromExt]] (ref => RequestDataMessageAdapter(ref))(timeout, scheduler2), timeout.duration) - val adapterScheduleRef = Await.result( - extResultDataProvider.ask[ActorRef[ScheduleServiceActivation]] (ref => RequestScheduleActivationAdapter(ref))(timeout, scheduler2), timeout.duration) - - val extResultData = - new ExtResultData( - adapterRef.toClassic, - adapterScheduleRef.toClassic, - extSimAdapterPhase2, - simpleExtSim.getExtResultDataSimulation.getResultDataFactory, - simpleExtSim.getExtResultDataSimulation.getGridResultDataAssets, - simpleExtSim.getExtResultDataSimulation.getParticipantResultDataAssets, - simulationStart, - powerFlowResolution - ) - - simpleExtSim.getExtResultDataSimulation.setExtResultData(extResultData) - - extResultDataProvider ! ExtResultDataProvider.Create( - InitExtResultData(extResultData), - ScheduleLock.singleKey( - context, - simScheduler, - INIT_SIM_TICK, - ), - ) - - - val simpleExtSimDatas: List[ExtData] = List( - extResultData, - extEmData - ) - - simpleExtSim.setup( - extSimAdapterData, - simpleExtSimDatas.asJava, - ) - // starting external simulation - new Thread(simpleExtSim, s"External simulation") - .start() - - val extDataServicesMap: Map[Class[_], ClassicRef] = Map( - classOf[ExtEmDataService] -> extEmDataService, - ) - - val extDataListenerMap: Map[Class[_], ActorRef[ExtResultDataProvider.Request]] = Map( - ExtResultDataProvider.getClass -> extResultDataProvider - ) - - val extSimAdaptersIt = Iterable(extSimAdapterPhase1, extSimAdapterPhase2) - - val extDatas = simpleExtSimDatas.toSet - extSimAdapterPhase2 ! ExtSimAdapter.Create( - extSimAdapterData, - 2, - ScheduleLock.singleKey(context, simScheduler, INIT_SIM_TICK), - ) - ExtSimSetupData(extSimAdaptersIt, extDataServicesMap, extDataListenerMap, extDatas, Some(extScheduler)) - - */ - ??? - } - - - override def timeAdvancer( - context: ActorContext[_], - simulation: ActorRef[SimonaSim.SimulationEnded.type], - runtimeEventListener: ActorRef[RuntimeEvent], - ): ActorRef[TimeAdvancer.Request] = { - val startDateTime = TimeUtil.withDefaults.toZonedDateTime( - simonaConfig.simona.time.startDateTime - ) - val endDateTime = TimeUtil.withDefaults.toZonedDateTime( - simonaConfig.simona.time.endDateTime - ) - - context.spawn( - TimeAdvancer( - simulation, - Some(runtimeEventListener), - simonaConfig.simona.time.schedulerReadyCheckWindow, - endDateTime.toTick(startDateTime), - ), - TimeAdvancer.getClass.getSimpleName, + extSimulationSetup( + context, + rootScheduler, + simScheduler, + simpleExtSim ) } - - override def scheduler( - context: ActorContext[_], - parent: ActorRef[SchedulerMessage], - coreFactory: CoreFactory = RegularSchedulerCore, - ): ActorRef[SchedulerMessage] = - context - .spawn( - Scheduler(parent, coreFactory), - s"${Scheduler.getClass.getSimpleName}_${coreFactory}_${UUID.randomUUID()}", - ) - - override def runtimeEventListener( - context: ActorContext[_] - ): ActorRef[RuntimeEventListener.Request] = - context - .spawn( - RuntimeEventListener( - simonaConfig.simona.runtime.listener, - runtimeEventQueue, - startDateTimeString = simonaConfig.simona.time.startDateTime, - ), - RuntimeEventListener.getClass.getSimpleName, - ) - - override def resultEventListener( - context: ActorContext[_], - extSimulationData: ExtSimSetupData, - ): Seq[ActorRef[ResultEventListener.Request]] = { - val extResultDataService: Option[ActorRef[ExtResultDataProvider.Request]] = - extSimulationData.extResultDataService - // append ResultEventListener as well to write raw output files - ArgsParser - .parseListenerConfigOption(simonaConfig.simona.event.listener) - .zipWithIndex - .map { case ((listenerCompanion, events), index) => - context.toClassic - .simonaActorOf( - listenerCompanion.props(events), - index.toString, - ) - .toTyped - } - .toSeq :+ context - .spawn( - ResultEventListener( - resultFileHierarchy, - extResultDataService, - ), - ResultEventListener.getClass.getSimpleName, - ) - } - - def buildSubGridToActorRefMap( - subGridTopologyGraph: SubGridTopologyGraph, - context: ActorContext[_], - environmentRefs: EnvironmentRefs, - resultEventListeners: Seq[ActorRef[ResultEvent]], - ): Map[Int, ActorRef[GridAgent.Request]] = { - subGridTopologyGraph - .vertexSet() - .asScala - .map(subGridContainer => { - val gridAgentRef = - context.spawn( - GridAgent( - environmentRefs, - simonaConfig, - resultEventListeners, - ), - subGridContainer.getSubnet.toString, - ) - subGridContainer.getSubnet -> gridAgentRef - }) - .toMap - } - - /** Get all thermal grids, that apply for the given grid container - * @param grid - * The grid container to assess - * @param thermalGridByBus - * Mapping from thermal bus to thermal grid - * @return - * A sequence of applicable thermal grids - */ - private def getThermalGrids( - grid: GridContainer, - thermalGridByBus: Map[ThermalBusInput, ThermalGrid], - ): Seq[ThermalGrid] = { - grid.getSystemParticipants.getHeatPumps.asScala - .flatten(hpInput => thermalGridByBus.get(hpInput.getThermalBus)) - .toSeq - } } /** Companion object to provide [[SetupHelper]] methods for 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 e79401238b..73770416a5 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaStandaloneSetup.scala @@ -181,8 +181,8 @@ class SimonaStandaloneSetup( weatherService } - /* + override def extSimulations( context: ActorContext[_], rootScheduler: ActorRef[SchedulerMessage], @@ -328,6 +328,7 @@ class SimonaStandaloneSetup( */ + override def extSimulations( context: ActorContext[_], rootScheduler: ActorRef[SchedulerMessage], From ddcaa592bc78639387f12c7e56f7d7dd72070a67 Mon Sep 17 00:00:00 2001 From: smjobaoo Date: Tue, 11 Jun 2024 10:38:43 +0200 Subject: [PATCH 39/41] docs --- .../scala/edu/ie3/simona/sim/setup/SimonaMosaikSetup.scala | 4 ++-- .../scala/edu/ie3/simona/sim/setup/SimonaOpsimSetup.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/main/scala/edu/ie3/simona/sim/setup/SimonaMosaikSetup.scala b/src/main/scala/edu/ie3/simona/sim/setup/SimonaMosaikSetup.scala index d392998e0b..6bc388a60e 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaMosaikSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaMosaikSetup.scala @@ -12,7 +12,7 @@ import edu.ie3.simona.config.SimonaConfig import edu.ie3.simona.event.RuntimeEvent import edu.ie3.simona.ontology.messages.SchedulerMessage import edu.ie3.simona.util.ResultFileHierarchy -import edu.ie3.simosaik.MosaikSimulation +import edu.ie3.simosaik.simosaikElectrolyzer.MosaikElectrolyzerSimulation import org.apache.pekko.actor.typed.ActorRef import org.apache.pekko.actor.typed.scaladsl.ActorContext @@ -47,7 +47,7 @@ class SimonaMosaikSetup( ): ExtSimSetupData = { val mosaikAddress = mosaikIP.getOrElse("127.0.0.1:5678") val mosaikMapping = mosaikMappingPath.getOrElse(throw new RuntimeException("Cannot connect to Mosaik, because there is no mapping!")) - val mosaikExtSim = new MosaikSimulation(mosaikAddress, Path.of(mosaikMapping)) + val mosaikExtSim = new MosaikElectrolyzerSimulation(mosaikAddress, Path.of(mosaikMapping)) extSimulationSetup( context, diff --git a/src/main/scala/edu/ie3/simona/sim/setup/SimonaOpsimSetup.scala b/src/main/scala/edu/ie3/simona/sim/setup/SimonaOpsimSetup.scala index 14356a8199..ea8d361f21 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaOpsimSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaOpsimSetup.scala @@ -12,7 +12,7 @@ import edu.ie3.simona.config.SimonaConfig import edu.ie3.simona.event.RuntimeEvent import edu.ie3.simona.ontology.messages.SchedulerMessage import edu.ie3.simona.util.ResultFileHierarchy -import edu.ie3.simopsim.OpsimEmSimulation +import edu.ie3.simopsim.simopsimEm.OpsimEmSimulation import org.apache.pekko.actor.typed.ActorRef import org.apache.pekko.actor.typed.scaladsl.ActorContext From f6f8ba8191b1a32cd917e5dd55ecae4a3fb70fc0 Mon Sep 17 00:00:00 2001 From: Johannes Bao Date: Tue, 23 Jul 2024 11:21:38 +0200 Subject: [PATCH 40/41] ext simulation with storage --- .../fullGrid/em_input.csv | 4 + .../fullGrid/ext_entity_mapping.csv | 7 + .../fullGrid/line_input.csv | 110 ++++++++ .../fullGrid/line_type_input.csv | 2 + .../fullGrid/load_input.csv | 4 + .../fullGrid/node_input.csv | 112 ++++++++ .../fullGrid/pv_input.csv | 4 + .../fullGrid/storage_input.csv | 2 + .../fullGrid/storage_type_input.csv | 2 + .../fullGrid/transformer_2_w_input.csv | 2 + .../fullGrid/transformer_2_w_type_input.csv | 2 + .../simopsimtestgrid.conf | 249 ++++++++++++++++++ .../fullGrid/storage_input.csv | 1 - .../vn_simona_small_withem.conf | 14 + .../edu/ie3/simona/agent/em/EmAgent.scala | 192 +++++++++----- .../edu/ie3/simona/agent/em/EmDataCore.scala | 154 +++++++---- .../edu/ie3/simona/agent/grid/GridAgent.scala | 3 +- .../event/listener/ResultEventListener.scala | 17 +- .../io/runtime/RuntimeEventLogSink.scala | 4 +- .../edu/ie3/simona/model/em/EmTools.scala | 4 + .../messages/flex/FlexibilityMessage.scala | 8 +- .../edu/ie3/simona/scheduler/Scheduler.scala | 4 - .../simona/service/em/ExtEmDataService.scala | 13 +- .../results/ExtResultDataProvider.scala | 18 +- .../scala/edu/ie3/simona/sim/SimonaSim.scala | 2 - .../simona/sim/setup/SimonaExtSimSetup.scala | 16 +- 26 files changed, 791 insertions(+), 159 deletions(-) create mode 100644 input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/em_input.csv create mode 100644 input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/ext_entity_mapping.csv create mode 100644 input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/line_input.csv create mode 100644 input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/line_type_input.csv create mode 100644 input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/load_input.csv create mode 100644 input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/node_input.csv create mode 100644 input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/pv_input.csv create mode 100644 input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/storage_input.csv create mode 100644 input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/storage_type_input.csv create mode 100644 input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/transformer_2_w_input.csv create mode 100644 input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/transformer_2_w_type_input.csv create mode 100644 input/samples/simopsimtestgrid_reduced_with_storage/simopsimtestgrid.conf diff --git a/input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/em_input.csv b/input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/em_input.csv new file mode 100644 index 0000000000..b6a794fe65 --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/em_input.csv @@ -0,0 +1,4 @@ +uuid;control_strategy;parent_em;id;operates_from;operates_until;operator +c3a7e9f5-b492-4c85-af2d-1e93f6a25443;self_optimization;;EM_HH_Bus_25;;; +f9dc7ce6-658c-4101-a12f-d58bb889286b;self_optimization;;EM_HH_Bus_81;;; +957938b7-0476-4fab-a1b3-6ce8615857b3;self_optimization;;EM_HH_Bus_110;;; diff --git a/input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/ext_entity_mapping.csv b/input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/ext_entity_mapping.csv new file mode 100644 index 0000000000..58ec9dcd0f --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/ext_entity_mapping.csv @@ -0,0 +1,7 @@ +uuid,id,columnScheme,dataType +f9dc7ce6-658c-4101-a12f-d58bb889286b,EM_HH_Bus_81,p,result_participant +957938b7-0476-4fab-a1b3-6ce8615857b3,EM_HH_Bus_110,p,result_participant +c3a7e9f5-b492-4c85-af2d-1e93f6a25443,EM_HH_Bus_25,p,result_participant +f9dc7ce6-658c-4101-a12f-d58bb889286b,EM_HH_Bus_81/Schedule,p,input +957938b7-0476-4fab-a1b3-6ce8615857b3,EM_HH_Bus_110/Schedule,p,input +c3a7e9f5-b492-4c85-af2d-1e93f6a25443,EM_HH_Bus_25/Schedule,p,input \ No newline at end of file diff --git a/input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/line_input.csv b/input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/line_input.csv new file mode 100644 index 0000000000..f4ff92fbba --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/line_input.csv @@ -0,0 +1,110 @@ +uuid;geo_position;id;length;node_a;node_b;olm_characteristic;operates_from;operates_until;operator;parallel_devices;type +bdbbb247-57b7-473b-9411-53fcf35032db;"{""type"":""LineString"",""coordinates"":[[11.8213,53.426],[11.8213,53.4257]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 27;0.0161277;093160c4-6482-4c58-b952-217c615e3ada;9cdb3115-cc00-4d61-bc33-442e8f30fb63;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +c48eeb2c-6858-4f96-9a90-20641cc0903d;"{""type"":""LineString"",""coordinates"":[[11.8214,53.4229],[11.8213,53.4225]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 35;0.0150703;002a4495-96e4-49c9-abbe-8fccb3e9c83e;b909fb45-b6ee-427f-afd7-e8a0ec7274c6;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +97e498ae-a4f7-4485-a703-b8c00d0a5e8a;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4221],[11.8201,53.4225]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 8;0.0228449;b565ae3b-68f9-4bca-816d-9b0fc1c6b13f;a7725293-05fc-447f-bc12-38b689b0a956;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +f907f152-7bfe-4a17-a63f-adda500f6f0e;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4241],[11.8184,53.4293]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 107;0.04;4749ab2b-4d96-4100-8081-73e77c797d6b;ea4a6507-e504-4542-be5f-1019719b2257;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +6bf14400-8f6c-41ef-b6a4-9e861a3ad08f;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4241],[11.822,53.4294]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 3;0.030304;4749ab2b-4d96-4100-8081-73e77c797d6b;c5b6bfaf-1621-40a7-9c53-02cfb59c04d9;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +18b92b71-320d-4294-a47e-6715f1594755;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4243],[11.8196,53.4247]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 102;0.0251089;f4da61e4-7600-4cd1-95b6-c70b56c049fc;e7908208-77b4-4059-806e-4857262992fc;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +4128307f-3e00-4da9-b629-b696b72165a4;"{""type"":""LineString"",""coordinates"":[[11.815,53.4289],[11.815,53.4293]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 28;0.0377768;8e809cf6-7e05-437c-89a6-f6ca135a403b;35748e60-3be8-4930-8a61-209fd5df1bec;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +92876b53-6bbc-48ff-ba5f-5f5c08313e4d;"{""type"":""LineString"",""coordinates"":[[11.815,53.4277],[11.815,53.4272]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 111;0.031;3e6be3ac-2b51-4080-b815-391313612fc7;78815cf6-70db-432c-96e6-87fe8cf67eee;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +4b557d0c-9992-48b1-b45e-26cccc01db03;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4235],[11.8191,53.4231]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 90;0.00167204;f2724954-34d3-4ddf-b6b0-7a1531639990;2efac9b1-fb0d-4e08-bfac-501798826deb;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +0652d3e0-8c7c-4be2-9a41-00534aa9774b;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4212],[11.8191,53.4209]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 94;0.00182131;02e63e81-2e62-4ef6-8fdb-0b0905a437b6;fd4bebb8-40ca-4eed-92c0-cdd10b86ac20;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +782d8c4c-accb-424b-91ae-9af1b5d4b1af;"{""type"":""LineString"",""coordinates"":[[11.8201,53.4225],[11.8201,53.4229]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 79;0.0338129;a7725293-05fc-447f-bc12-38b689b0a956;83da8d60-405a-45f7-9bb9-9d35607b7927;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +b7cac7a5-f528-45bd-8ced-a16234979e13;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4239],[11.8191,53.4235]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 89;0.0111411;b5c1e826-63fd-4b0c-bec0-0c758389ef58;f2724954-34d3-4ddf-b6b0-7a1531639990;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +7ac1a715-e141-429a-8cfe-ff2badd41074;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4257],[11.8174,53.4262]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 65;0.019;8726dc29-621e-4455-a541-cd88d7da457f;814f784b-687f-4dd5-8a91-c7772c916d46;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +77ae4f1e-9ecb-4dda-a5f2-ce7ae3f9bbec;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4264],[11.8213,53.426]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 23;0.00488535;18b4157b-0e47-4c5a-adb8-ccae47372336;093160c4-6482-4c58-b952-217c615e3ada;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +6aaa54d9-d7b7-4adc-a85a-335cdf1393d6;"{""type"":""LineString"",""coordinates"":[[11.8162,53.4284],[11.8169,53.4289]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 40;0.036;6b2881ce-3965-4f5e-98a3-74eb47b0a7ca;a882e666-82d1-4ba6-87df-fc702fe06187;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +c85c107e-82c0-4002-acfa-d7000512a2ad;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4274],[11.8213,53.4271]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 20;0.0182765;2b3d7fb8-0583-4d47-97b1-3b5f232fd462;7b81b518-00e0-4ff1-b4cf-876903958d7a;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +2ef5f888-4341-4eee-b505-ae07a9a60c8d;"{""type"":""LineString"",""coordinates"":[[11.8201,53.4236],[11.8201,53.4239]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 55;0.0259846;f713593a-3fd3-410a-ac08-74202d4f5798;80962bd3-a10f-4ed2-ba6a-3e802189939c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +139ea4f9-9b7f-4825-8919-537a94ff4794;"{""type"":""LineString"",""coordinates"":[[11.8175,53.4277],[11.8174,53.4281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 59;0.00160952;7f692039-eef6-45f6-9e30-b5983f6750a5;3f6c26dd-842b-4dee-b71f-4aa32e2654ff;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +68853808-8b09-4ff2-9c92-88ed1d78c639;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4236],[11.8213,53.4232]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 33;0.0217875;3464496c-7dd4-41e9-ae0a-99ade0b51572;a966644c-37d5-4400-9fa0-725fd88586a8;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +91bec60c-2951-420c-a35e-2633119ee450;"{""type"":""LineString"",""coordinates"":[[11.8162,53.4277],[11.8162,53.4281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 30;0.00820054;2d33314e-31db-4ad4-a898-2be5d56a896d;555cd075-0fe4-4a65-b027-f45cffa960d9;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +35ba0827-e27e-496e-b735-c778f3b03019;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4241],[11.8213,53.4293]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 18;0.0139169;4749ab2b-4d96-4100-8081-73e77c797d6b;844c0b9c-058a-4228-a8c4-bf2defff6958;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +60efd159-72d6-4546-b245-8b8fc120a9f8;"{""type"":""LineString"",""coordinates"":[[11.817,53.4293],[11.8166,53.4241]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 26;0.098;85751c9b-3e5e-468a-9dc7-43775b0d4a6f;4749ab2b-4d96-4100-8081-73e77c797d6b;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +212b62aa-73c2-47af-95ec-00cad8d4a4f4;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4222],[11.8214,53.4218]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 38;0.00992653;589bf32a-b361-4536-ae96-6d56d184eedb;3b86661a-187d-4aa6-bf37-2014789afc08;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +7e61def1-8414-40b3-8775-fad9124f4369;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4282],[11.8213,53.4277]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 16;0.0164234;09285b78-9c18-4af7-9c7a-942cc868016f;3a557b4e-06b8-4f29-929f-81d95c42c897;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +f9b4c7dc-c199-4691-8c6d-1faf438cf336;"{""type"":""LineString"",""coordinates"":[[11.8221,53.429],[11.8221,53.4286]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 5;0.0143698;9644f198-e801-4545-87ee-a24e2a8039bd;6ee7ea93-ea9c-40cb-b79a-1c5f287c97a5;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +eebc456e-d11b-4b19-9100-cc1d8e91f926;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4245],[11.8166,53.4249]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 71;0.00648856;7efabb8d-ba17-4487-96d9-5744b1fedf8a;7c35a794-f569-4a9c-acb0-d03647610086;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +8f4c4009-e1ed-4985-8095-729aa5dc8cdd;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4192],[11.8198,53.4195]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 85;0.00737032;594d101c-3a05-45e3-a061-9189f3e848b7;af2b07ce-1a96-4b50-9e21-badf29eed519;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +20db47c8-d154-4ab2-ad65-caa4a37466d2;"{""type"":""LineString"",""coordinates"":[[11.8175,53.4257],[11.8174,53.4262]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 68;0.037;5f153bf6-4f25-41f1-8545-18fe6323bc49;814f784b-687f-4dd5-8a91-c7772c916d46;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +7d838003-cd6e-42fd-b75a-eaa354c3993b;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4216],[11.8191,53.4212]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 93;0.0043578;8f53645e-1f28-4eb1-807f-eb2a473f9d4c;02e63e81-2e62-4ef6-8fdb-0b0905a437b6;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +a92013e1-78b7-4447-a505-f25fffbf56f2;"{""type"":""LineString"",""coordinates"":[[11.8174,53.4262],[11.8175,53.4265]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 48;0.0118531;814f784b-687f-4dd5-8a91-c7772c916d46;94fe96b1-f36a-4edd-a107-4ff0376f1066;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +690f8f61-5cc7-448e-8971-a516f784bf11;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4293],[11.8166,53.4241]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 106;0.0131451;20ad9754-e966-4ad1-9541-f968c207f3df;4749ab2b-4d96-4100-8081-73e77c797d6b;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +6a00d582-32b1-4581-bdd1-a638ca279597;"{""type"":""LineString"",""coordinates"":[[11.815,53.4272],[11.815,53.4269]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 112;0.028;78815cf6-70db-432c-96e6-87fe8cf67eee;ab8c8f6c-e7a2-4b81-a0d7-5f13789267a2;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +1d5877c1-31cd-4ee8-b0dd-667fa4fffb8a;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4277],[11.8213,53.4274]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 17;0.00286133;3a557b4e-06b8-4f29-929f-81d95c42c897;2b3d7fb8-0583-4d47-97b1-3b5f232fd462;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +a15cd6a4-4b22-434f-be9b-ad2abe6e538c;"{""type"":""LineString"",""coordinates"":[[11.815,53.4289],[11.815,53.4285]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 108;0.033;8e809cf6-7e05-437c-89a6-f6ca135a403b;88cf719a-92df-4dfd-9a83-f84330e28fe0;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +2bd81511-4f19-4374-8ac5-96c2b19eda64;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4223],[11.8191,53.422]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 52;0.00980013;c7e48384-5699-4a38-a887-7e15a9145202;c6dac3ab-f44f-4b87-800c-0f4da64673f1;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +beee40ad-aca3-490f-87d5-a6dfc4bb76e3;"{""type"":""LineString"",""coordinates"":[[11.815,53.4293],[11.8166,53.4241]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 39;0.09;35748e60-3be8-4930-8a61-209fd5df1bec;4749ab2b-4d96-4100-8081-73e77c797d6b;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +96b06f87-7905-4d12-99d8-1ed330050659;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4203],[11.8197,53.4207]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 12;0.00473835;5596da2f-ca32-4ad3-81da-1ffa17cd3d7b;839ff0f4-93db-42ec-a928-bbc448b6cf5c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +e66890ef-ec02-407f-a5bb-ce32128e7490;"{""type"":""LineString"",""coordinates"":[[11.8162,53.4281],[11.8162,53.4284]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 41;0.00828244;555cd075-0fe4-4a65-b027-f45cffa960d9;6b2881ce-3965-4f5e-98a3-74eb47b0a7ca;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +015b0337-98bd-40d4-97d3-13a0d1da88ee;"{""type"":""LineString"",""coordinates"":[[11.8213,53.425],[11.8214,53.4246]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 84;0.0489789;58b551b6-83bd-4f1c-8d9c-8c9a7f638c0b;a9288e77-2919-4db6-89eb-9737bd07f111;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +f9e6b2f1-f8fb-4763-a1d0-36d06170fea0;"{""type"":""LineString"",""coordinates"":[[11.8213,53.429],[11.8213,53.4286]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 98;0.0144283;16091b6d-f1ea-4a07-9ad4-30d595aba68d;1403edf9-e47c-4705-8563-83bcd639482e;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +f270d6b5-7c94-4397-8ab0-7c39c888d726;"{""type"":""LineString"",""coordinates"":[[11.8175,53.4265],[11.8175,53.427]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 58;0.0036937;94fe96b1-f36a-4edd-a107-4ff0376f1066;80b8d1f8-7e83-421d-a95a-c193fc35f4f7;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +af8c65aa-cfd6-434a-8512-2d80106a2f2c;"{""type"":""LineString"",""coordinates"":[[11.8195,53.426],[11.8195,53.4264]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 31;0.00626899;9f1baf4e-12e1-41d1-8efc-81cfc78f1957;b8e95bf0-3ba8-4d53-a0bf-a3720fb785fb;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +fe28e831-6405-4dfe-987b-d688367694f1;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4247],[11.8195,53.425]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 104;0.00221503;e7908208-77b4-4059-806e-4857262992fc;29516ae3-6676-4797-99c1-1f0a32b989d8;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +eac56d69-5500-4261-9690-adb16c867485;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4231],[11.8191,53.4227]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 92;0.0149219;2efac9b1-fb0d-4e08-bfac-501798826deb;15a86f7d-fb73-49a4-af6a-25b14122378d;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +183f3976-48ca-42b2-9af9-7998436fac5b;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4289],[11.8196,53.4293]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 105;0.00721947;47246a84-ad0e-4d04-9d98-1c9cd5d363c1;20ad9754-e966-4ad1-9541-f968c207f3df;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +7feef458-03f3-4d23-b3a2-e6f1035398c4;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4243],[11.8213,53.4239]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 29;0.0204277;3fcb94e3-7781-4d83-9030-d9853822e78e;d0bfabdb-0e83-423b-a20a-ab9197c4284e;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +592cd979-16c9-43d8-a311-8ac938aa5d03;"{""type"":""LineString"",""coordinates"":[[11.8174,53.4262],[11.8182,53.4257]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 47;0.0181601;814f784b-687f-4dd5-8a91-c7772c916d46;c86d6361-4159-4787-b5f4-e41dcaa95195;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +684146e5-3b58-43b4-9589-45325ab1c0bc;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4257],[11.8213,53.4253]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 24;0.0102233;9cdb3115-cc00-4d61-bc33-442e8f30fb63;a432b8ce-0462-478b-83e7-3107cd2e909c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +d9f9ee76-b016-4588-ac6d-46681894ada7;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4188],[11.8198,53.4192]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 74;0.00421724;952c6b04-0d02-4ea2-a6cc-bb44fbbe4e52;594d101c-3a05-45e3-a061-9189f3e848b7;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +1f7e9cae-460e-47db-8cb8-da5d9f695fd8;"{""type"":""LineString"",""coordinates"":[[11.8197,53.4207],[11.8198,53.4211]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 13;0.00987733;839ff0f4-93db-42ec-a928-bbc448b6cf5c;27b84da5-478e-4a05-8fe7-a9f800db5eff;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +f3a592d0-0fd7-42ea-b928-f39473b419aa;"{""type"":""LineString"",""coordinates"":[[11.8214,53.4246],[11.8213,53.4243]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 10;0.0344866;a9288e77-2919-4db6-89eb-9737bd07f111;3fcb94e3-7781-4d83-9030-d9853822e78e;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +f505bff9-0803-415f-a765-9da981ff6024;"{""type"":""LineString"",""coordinates"":[[11.8195,53.4264],[11.8195,53.4268]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 43;0.00839819;b8e95bf0-3ba8-4d53-a0bf-a3720fb785fb;34031e92-3444-47d5-94ae-cceeb5d96bb2;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +c6501262-2e05-462b-8872-445d2aa1cab8;"{""type"":""LineString"",""coordinates"":[[11.8195,53.4272],[11.8195,53.4275]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 99;0.0157523;120eaa58-a500-4ae2-a86a-56a40b931ec1;9d136a6b-5fdc-44ed-a5ed-599a55281024;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +caac4b4b-4871-4e20-994c-6517931546cb;"{""type"":""LineString"",""coordinates"":[[11.8201,53.4229],[11.8201,53.4232]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 91;0.0220129;83da8d60-405a-45f7-9bb9-9d35607b7927;24b63115-12eb-4e77-b9ef-ca474fed960f;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +e9721561-53c0-45cc-a8ed-28861ef9dc66;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4216],[11.8198,53.4221]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 7;0.0333523;883edf38-9a18-4f61-981a-691aaf436cc7;b565ae3b-68f9-4bca-816d-9b0fc1c6b13f;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +ab143df6-d050-47b6-911a-93e462d928ac;"{""type"":""LineString"",""coordinates"":[[11.8173,53.4234],[11.8166,53.4238]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 73;0.0287121;e68a088d-cf1a-40b7-9b1a-e0933352f4e6;4129e079-6712-4275-911c-36729d698c42;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +ab7ab785-36ba-4da1-b176-a7c636cb1372;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4267],[11.8213,53.4264]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 22;0.0102382;dc54bd8a-b7d8-4e99-adb0-d6ee5084241c;18b4157b-0e47-4c5a-adb8-ccae47372336;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +1ae90a03-52a3-40ef-8e06-4ba01888aa5c;"{""type"":""LineString"",""coordinates"":[[11.816,53.4222],[11.816,53.4227]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 44;0.00743191;62d603c3-f306-40b3-a665-ba9892d226f0;3faac527-0ff3-44a7-9e4f-24a41940da90;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +20be9235-f4db-4753-9fa1-223c8519fcd3;"{""type"":""LineString"",""coordinates"":[[11.8175,53.4253],[11.8175,53.4257]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 66;0.00791441;73e7a7e8-2154-46ea-9727-a4916af3570c;5f153bf6-4f25-41f1-8545-18fe6323bc49;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +558e1545-a944-419a-9153-83caa09e1a3c;"{""type"":""LineString"",""coordinates"":[[11.8162,53.4273],[11.8162,53.4277]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 42;0.00357726;0d6ef8f6-0ba1-4fa7-8e63-e55cee12b165;2d33314e-31db-4ad4-a898-2be5d56a896d;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +38d60cf7-6099-4bc0-a616-0f0b66c70c9a;"{""type"":""LineString"",""coordinates"":[[11.8201,53.4232],[11.8201,53.4236]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 67;0.0179092;24b63115-12eb-4e77-b9ef-ca474fed960f;f713593a-3fd3-410a-ac08-74202d4f5798;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +e74dbbe9-948f-4056-8134-fd1d9d39e773;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4253],[11.8196,53.4257]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 86;0.0190003;03b2aa45-84f6-48c0-9dab-427e046a5672;79e19265-08e8-407f-ae95-2f78e344d3a4;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +24c0dc1c-2e99-46ed-a52b-5a7aae7c9afb;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4249],[11.8166,53.4253]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 72;0.00189914;7c35a794-f569-4a9c-acb0-d03647610086;50cac08e-bf24-4526-9466-53ca5edccd15;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +41a72cb2-f037-4196-a248-2b18a578db50;"{""type"":""LineString"",""coordinates"":[[11.816,53.421],[11.816,53.4214]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 78;0.00156586;98c14f60-e196-4f12-903b-8485f1eacb16;ca0b9e34-ddc8-405e-86a4-ab939626c4a2;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +e2919117-f70a-4b32-a5ab-f077fe0a84ef;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4253],[11.8166,53.4257]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 70;0.00263875;50cac08e-bf24-4526-9466-53ca5edccd15;8726dc29-621e-4455-a541-cd88d7da457f;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +7d3ee5e1-1749-4e43-afeb-4ccf5f431312;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4282],[11.8196,53.4286]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 103;0.011354;a3ed5af5-1fcb-4fce-af0f-708d3d604124;31e6e197-719d-4aaf-8ca5-ab9e7549390e;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +cbba4a6b-27d1-4724-a193-134af220a57d;"{""type"":""LineString"",""coordinates"":[[11.8182,53.4257],[11.8182,53.4254]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 49;0.018118;c86d6361-4159-4787-b5f4-e41dcaa95195;c6c20ab1-16fb-4f82-8017-273022da8bb0;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +b84fad90-182c-4ac1-b77d-394f39024ffc;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4286],[11.8196,53.4289]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 19;0.0115943;31e6e197-719d-4aaf-8ca5-ab9e7549390e;47246a84-ad0e-4d04-9d98-1c9cd5d363c1;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +6c87a571-9d8b-4df9-bd86-cde7954bee28;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4271],[11.8213,53.4267]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 21;0.00282037;7b81b518-00e0-4ff1-b4cf-876903958d7a;dc54bd8a-b7d8-4e99-adb0-d6ee5084241c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +536ef1a2-b988-4474-a5d4-1254598c4716;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4227],[11.8191,53.4223]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 82;0.00544392;15a86f7d-fb73-49a4-af6a-25b14122378d;c7e48384-5699-4a38-a887-7e15a9145202;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +c0c99b3a-1c61-47b8-931d-571d9494d98f;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4241],[11.8166,53.4245]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 51;0.0182324;5682ac05-7336-4ebc-a5d1-3b69c79fb3b1;7efabb8d-ba17-4487-96d9-5744b1fedf8a;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +d3366635-447e-4d45-b839-7a7561f869a3;"{""type"":""LineString"",""coordinates"":[[11.822,53.4294],[11.8221,53.429]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 4;0.0141339;c5b6bfaf-1621-40a7-9c53-02cfb59c04d9;9644f198-e801-4545-87ee-a24e2a8039bd;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +932dd0d7-2051-4a3a-a0e3-489210d3b763;"{""type"":""LineString"",""coordinates"":[[11.8169,53.4289],[11.817,53.4293]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 57;0.0219657;a882e666-82d1-4ba6-87df-fc702fe06187;85751c9b-3e5e-468a-9dc7-43775b0d4a6f;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +e718d253-34ea-4c00-a653-80ac0af2934e;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4293],[11.8213,53.429]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 11;0.00557129;844c0b9c-058a-4228-a8c4-bf2defff6958;16091b6d-f1ea-4a07-9ad4-30d595aba68d;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +3ac954bf-5c1e-428a-9da7-37331f53d8fe;"{""type"":""LineString"",""coordinates"":[[11.8175,53.4248],[11.8175,53.4253]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 69;0.0017759;0d6c3f2b-5296-4ec1-995c-b150e72f035f;73e7a7e8-2154-46ea-9727-a4916af3570c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +7e9a0440-2a62-4daf-a4ab-e9f14f6cfd77;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4238],[11.8166,53.4234]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 53;0.0337731;4129e079-6712-4275-911c-36729d698c42;57b40047-4f9d-46bb-bf19-c0a86bbd4f5b;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +10a2050e-173b-43a7-91a0-157cee5c900d;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4257],[11.8195,53.426]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 87;0.00654022;79e19265-08e8-407f-ae95-2f78e344d3a4;9f1baf4e-12e1-41d1-8efc-81cfc78f1957;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +0dfd30a5-394e-49ff-95e9-50e73bf2604c;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4232],[11.8214,53.4229]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 34;0.0199334;a966644c-37d5-4400-9fa0-725fd88586a8;002a4495-96e4-49c9-abbe-8fccb3e9c83e;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +847934a0-e2b7-4caf-bb40-2e6af5bcb0ab;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4238],[11.8166,53.4241]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 50;0.0101795;4129e079-6712-4275-911c-36729d698c42;5682ac05-7336-4ebc-a5d1-3b69c79fb3b1;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +2daf9021-eb24-4d0c-bced-00e713a65b3a;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4225],[11.8213,53.4222]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 36;0.010188;b909fb45-b6ee-427f-afd7-e8a0ec7274c6;589bf32a-b361-4536-ae96-6d56d184eedb;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +2f1b02a3-5a58-4723-ab8f-827118a4c611;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4239],[11.8213,53.4236]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 32;0.0188563;d0bfabdb-0e83-423b-a20a-ab9197c4284e;3464496c-7dd4-41e9-ae0a-99ade0b51572;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +a5e9b6b0-2afc-49b0-8aad-705410b823c2;"{""type"":""LineString"",""coordinates"":[[11.815,53.4285],[11.815,53.4281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 109;0.027;88cf719a-92df-4dfd-9a83-f84330e28fe0;d38d936a-9c05-4bdc-8331-418fef27f492;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +ae82a1c1-9790-44d1-8dab-52d75c7b79ce;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4286],[11.8213,53.4282]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 6;0.0050064;1403edf9-e47c-4705-8563-83bcd639482e;09285b78-9c18-4af7-9c7a-942cc868016f;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +e642eafb-2b6e-4b78-80b8-1ab15db8cfeb;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4278],[11.8196,53.4282]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 101;0.0153312;7dc43c81-9a61-45a0-9745-800a28bf4a9d;a3ed5af5-1fcb-4fce-af0f-708d3d604124;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +41a4c5e5-7b54-46a8-b6b6-d7b5861a251c;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4195],[11.8198,53.4199]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 96;0.0010841;af2b07ce-1a96-4b50-9e21-badf29eed519;215eaa45-82c3-49c7-a60f-4fa13215de05;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +44821cfc-7670-4f28-8941-70e9345cb069;"{""type"":""LineString"",""coordinates"":[[11.8201,53.4225],[11.8203,53.4221]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 15;0.027677;a7725293-05fc-447f-bc12-38b689b0a956;eae8a04c-44f2-4da3-95f6-cae48f85737c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +d665e8c6-40fb-4608-97fb-99a4cf52560e;"{""type"":""LineString"",""coordinates"":[[11.815,53.4281],[11.815,53.4277]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 110;0.041;d38d936a-9c05-4bdc-8331-418fef27f492;3e6be3ac-2b51-4080-b815-391313612fc7;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +2f37bd36-f356-4d29-81c7-dd9c2fba5e7d;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4234],[11.8166,53.423]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 45;0.00223482;57b40047-4f9d-46bb-bf19-c0a86bbd4f5b;5b73ded9-3ca4-4f18-a2ab-c27b9a3dcf9d;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +bcccf0b5-e0e3-4cc7-82bb-262ebc19415e;"{""type"":""LineString"",""coordinates"":[[11.8169,53.4289],[11.8168,53.4285]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 56;0.039;a882e666-82d1-4ba6-87df-fc702fe06187;c41ebab8-16a0-4a3a-b4af-26073932d462;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +3685f389-6822-4522-a633-74265b67eaee;"{""type"":""LineString"",""coordinates"":[[11.816,53.4227],[11.816,53.423]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 76;0.00185168;3faac527-0ff3-44a7-9e4f-24a41940da90;2575f527-1f4e-45e2-bed2-4a5427f122e0;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +48347702-9e58-49f3-a7de-72024d4b296c;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4199],[11.8198,53.4203]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 2;0.00995493;215eaa45-82c3-49c7-a60f-4fa13215de05;5596da2f-ca32-4ad3-81da-1ffa17cd3d7b;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +ec2a73ea-50ba-4187-b8b0-a5046cf6b632;"{""type"":""LineString"",""coordinates"":[[11.8195,53.425],[11.8196,53.4253]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 83;0.00510961;29516ae3-6676-4797-99c1-1f0a32b989d8;03b2aa45-84f6-48c0-9dab-427e046a5672;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +d1940183-f7bb-42df-b8ff-63ac7aff0b1d;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4211],[11.8198,53.4216]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 9;0.00439896;27b84da5-478e-4a05-8fe7-a9f800db5eff;883edf38-9a18-4f61-981a-691aaf436cc7;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +dd618e3f-9ef9-4e4b-b632-737d79c4d8c3;"{""type"":""LineString"",""coordinates"":[[11.8175,53.427],[11.8174,53.4273]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 62;0.00124645;80b8d1f8-7e83-421d-a95a-c193fc35f4f7;8ace5c2b-584a-4015-990f-6f1e14de4ddb;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +ed80666a-cf86-404a-b3a7-ad49be1cd40c;"{""type"":""LineString"",""coordinates"":[[11.816,53.423],[11.816,53.4234]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 77;0.00232237;2575f527-1f4e-45e2-bed2-4a5427f122e0;5d3bcf55-0520-43ff-8d63-3d0eb421e442;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +444615ed-26f2-45a7-8b5d-213c72e83a4f;"{""type"":""LineString"",""coordinates"":[[11.8201,53.4239],[11.8196,53.4243]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 95;0.0281251;80962bd3-a10f-4ed2-ba6a-3e802189939c;f4da61e4-7600-4cd1-95b6-c70b56c049fc;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +05ebbc17-61bc-4810-ae19-9ee04d7ce8d0;"{""type"":""LineString"",""coordinates"":[[11.8175,53.4285],[11.8169,53.4289]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 54;0.0173762;30b2ce2d-41dd-4d1d-866d-47abd24bfc3e;a882e666-82d1-4ba6-87df-fc702fe06187;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +caf90323-aad2-496e-a0e3-a4ba7c9a481c;"{""type"":""LineString"",""coordinates"":[[11.8174,53.4273],[11.8175,53.4277]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 64;0.00314803;8ace5c2b-584a-4015-990f-6f1e14de4ddb;7f692039-eef6-45f6-9e30-b5983f6750a5;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +60b400db-b048-4bed-be29-8abc18780d10;"{""type"":""LineString"",""coordinates"":[[11.816,53.4206],[11.816,53.421]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 14;0.045296;d6dbb0ae-13c9-438e-93b3-b6c63a0708df;98c14f60-e196-4f12-903b-8485f1eacb16;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +cf13634d-abd2-465d-8839-c95a54af7a80;"{""type"":""LineString"",""coordinates"":[[11.816,53.4214],[11.816,53.4218]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 80;0.0101988;ca0b9e34-ddc8-405e-86a4-ab939626c4a2;43040a39-8b6c-401f-9dfd-82b42aa6dec6;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +30044e09-a0f8-417d-a949-afcfa940f671;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4253],[11.8213,53.425]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 25;0.0124821;a432b8ce-0462-478b-83e7-3107cd2e909c;58b551b6-83bd-4f1c-8d9c-8c9a7f638c0b;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +fe71fe2b-8dd0-4942-b0b5-d241e095b912;"{""type"":""LineString"",""coordinates"":[[11.8195,53.4268],[11.8195,53.4272]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 97;0.0032723;34031e92-3444-47d5-94ae-cceeb5d96bb2;120eaa58-a500-4ae2-a86a-56a40b931ec1;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +66e44163-7091-4f1b-991c-64108e2238f6;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4243],[11.8191,53.4239]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 88;0.0185855;f4da61e4-7600-4cd1-95b6-c70b56c049fc;b5c1e826-63fd-4b0c-bec0-0c758389ef58;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +82bac681-169f-48d8-9842-fd69d3adbfe0;"{""type"":""LineString"",""coordinates"":[[11.816,53.4234],[11.8166,53.4238]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 75;0.0109137;5d3bcf55-0520-43ff-8d63-3d0eb421e442;4129e079-6712-4275-911c-36729d698c42;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +74a5dc42-e689-490f-a611-ae7c3767f01b;"{""type"":""LineString"",""coordinates"":[[11.816,53.4218],[11.816,53.4222]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 81;0.0100396;43040a39-8b6c-401f-9dfd-82b42aa6dec6;62d603c3-f306-40b3-a665-ba9892d226f0;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +0eedb3d0-cedc-4798-b42e-d8e8ef646b82;"{""type"":""LineString"",""coordinates"":[[11.8195,53.4275],[11.8196,53.4278]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 100;0.0129061;9d136a6b-5fdc-44ed-a5ed-599a55281024;7dc43c81-9a61-45a0-9745-800a28bf4a9d;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +b44446fd-2125-4c7b-850e-b0f3d6c8b110;"{""type"":""LineString"",""coordinates"":[[11.8174,53.4281],[11.8175,53.4285]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 61;0.014766;3f6c26dd-842b-4dee-b71f-4aa32e2654ff;30b2ce2d-41dd-4d1d-866d-47abd24bfc3e;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +31c914bc-dd9e-4825-9b4b-b5fc1b971e0d;"{""type"":""LineString"",""coordinates"":[[11.8166,53.423],[11.8166,53.4227]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 46;0.00683612;5b73ded9-3ca4-4f18-a2ab-c27b9a3dcf9d;ad30322c-0c99-4669-8e4b-25265087a66d;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +e7ac7023-f4e3-460d-8d02-4c9c444d2e18;"{""type"":""LineString"",""coordinates"":[[11.8191,53.422],[11.8191,53.4216]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 63;0.0149947;c6dac3ab-f44f-4b87-800c-0f4da64673f1;8f53645e-1f28-4eb1-807f-eb2a473f9d4c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e diff --git a/input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/line_type_input.csv b/input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/line_type_input.csv new file mode 100644 index 0000000000..d7786c38cd --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/line_type_input.csv @@ -0,0 +1,2 @@ +uuid;b;g;i_max;id;r;v_rated;x +9a8e9b63-af21-4c1b-8db7-fc2924f9610e;273.31899999999996;0.0;357.0;NAYY 4x240SE 0.6/1kV;0.1267;0.4;0.0797965 diff --git a/input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/load_input.csv b/input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/load_input.csv new file mode 100644 index 0000000000..55e4abf008 --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/load_input.csv @@ -0,0 +1,4 @@ +uuid;cos_phi_rated;dsm;e_cons_annual;id;load_profile;node;operates_from;operates_until;operator;q_characteristics;s_rated;em +c3434742-e4f0-49e5-baa7-c1e3045c732c;0.93;false;0.0;LV5.201 Load 64;h0;dc54bd8a-b7d8-4e99-adb0-d6ee5084241c;;;;"cosPhiFixed:{(0.0,0.93)}";0.2;f9dc7ce6-658c-4101-a12f-d58bb889286b +fd2e19b6-d5e3-4776-9456-8787a2160d9d;0.93;false;0.0;LV5.201 Load 74;h0;43040a39-8b6c-401f-9dfd-82b42aa6dec6;;;;"cosPhiFixed:{(0.0,0.93)}";1.1;c3a7e9f5-b492-4c85-af2d-1e93f6a25443 +98c1a2ab-bd09-4c77-a389-d088aed894b1;0.93;false;0.0;LV5.201 Load 102;h0;3e6be3ac-2b51-4080-b815-391313612fc7;;;;"cosPhiFixed:{(0.0,0.93)}";5.3;957938b7-0476-4fab-a1b3-6ce8615857b3 diff --git a/input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/node_input.csv b/input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/node_input.csv new file mode 100644 index 0000000000..ab9f9335ca --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/node_input.csv @@ -0,0 +1,112 @@ +uuid;geo_position;id;operates_from;operates_until;operator;slack;subnet;v_rated;v_target;volt_lvl +16091b6d-f1ea-4a07-9ad4-30d595aba68d;"{""type"":""Point"",""coordinates"":[11.8213,53.429],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 26;;;;false;2;0.4;1.0;Niederspannung +27b84da5-478e-4a05-8fe7-a9f800db5eff;"{""type"":""Point"",""coordinates"":[11.8198,53.4211],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 7;;;;false;2;0.4;1.0;Niederspannung +7f692039-eef6-45f6-9e30-b5983f6750a5;"{""type"":""Point"",""coordinates"":[11.8175,53.4277],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 54;;;;false;2;0.4;1.0;Niederspannung +20ad9754-e966-4ad1-9541-f968c207f3df;"{""type"":""Point"",""coordinates"":[11.8196,53.4293],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 84;;;;false;2;0.4;1.0;Niederspannung +98c14f60-e196-4f12-903b-8485f1eacb16;"{""type"":""Point"",""coordinates"":[11.816,53.421],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 29;;;;false;2;0.4;1.0;Niederspannung +09285b78-9c18-4af7-9c7a-942cc868016f;"{""type"":""Point"",""coordinates"":[11.8213,53.4282],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 58;;;;false;2;0.4;1.0;Niederspannung +0d6ef8f6-0ba1-4fa7-8e63-e55cee12b165;"{""type"":""Point"",""coordinates"":[11.8162,53.4273],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 103;;;;false;2;0.4;1.0;Niederspannung +43040a39-8b6c-401f-9dfd-82b42aa6dec6;"{""type"":""Point"",""coordinates"":[11.816,53.4218],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 25;;;;false;2;0.4;1.0;Niederspannung +dc54bd8a-b7d8-4e99-adb0-d6ee5084241c;"{""type"":""Point"",""coordinates"":[11.8213,53.4267],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 81;;;;false;2;0.4;1.0;Niederspannung +2575f527-1f4e-45e2-bed2-4a5427f122e0;"{""type"":""Point"",""coordinates"":[11.816,53.423],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 3;;;;false;2;0.4;1.0;Niederspannung +30b2ce2d-41dd-4d1d-866d-47abd24bfc3e;"{""type"":""Point"",""coordinates"":[11.8175,53.4285],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 34;;;;false;2;0.4;1.0;Niederspannung +589bf32a-b361-4536-ae96-6d56d184eedb;"{""type"":""Point"",""coordinates"":[11.8213,53.4222],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 101;;;;false;2;0.4;1.0;Niederspannung +555cd075-0fe4-4a65-b027-f45cffa960d9;"{""type"":""Point"",""coordinates"":[11.8162,53.4281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 31;;;;false;2;0.4;1.0;Niederspannung +62d603c3-f306-40b3-a665-ba9892d226f0;"{""type"":""Point"",""coordinates"":[11.816,53.4222],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 55;;;;false;2;0.4;1.0;Niederspannung +215eaa45-82c3-49c7-a60f-4fa13215de05;"{""type"":""Point"",""coordinates"":[11.8198,53.4199],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 24;;;;false;2;0.4;1.0;Niederspannung +f4da61e4-7600-4cd1-95b6-c70b56c049fc;"{""type"":""Point"",""coordinates"":[11.8196,53.4243],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 76;;;;false;2;0.4;1.0;Niederspannung +73e7a7e8-2154-46ea-9727-a4916af3570c;"{""type"":""Point"",""coordinates"":[11.8175,53.4253],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 18;;;;false;2;0.4;1.0;Niederspannung +85751c9b-3e5e-468a-9dc7-43775b0d4a6f;"{""type"":""Point"",""coordinates"":[11.817,53.4293],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 51;;;;false;2;0.4;1.0;Niederspannung +093160c4-6482-4c58-b952-217c615e3ada;"{""type"":""Point"",""coordinates"":[11.8213,53.426],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 68;;;;false;2;0.4;1.0;Niederspannung +79e19265-08e8-407f-ae95-2f78e344d3a4;"{""type"":""Point"",""coordinates"":[11.8196,53.4257],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 28;;;;false;2;0.4;1.0;Niederspannung +a966644c-37d5-4400-9fa0-725fd88586a8;"{""type"":""Point"",""coordinates"":[11.8213,53.4232],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 59;;;;false;2;0.4;1.0;Niederspannung +952c6b04-0d02-4ea2-a6cc-bb44fbbe4e52;"{""type"":""Point"",""coordinates"":[11.8198,53.4188],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 94;;;;false;2;0.4;1.0;Niederspannung +c6c20ab1-16fb-4f82-8017-273022da8bb0;"{""type"":""Point"",""coordinates"":[11.8182,53.4254],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 62;;;;false;2;0.4;1.0;Niederspannung +e68a088d-cf1a-40b7-9b1a-e0933352f4e6;"{""type"":""Point"",""coordinates"":[11.8173,53.4234],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 67;;;;false;2;0.4;1.0;Niederspannung +8f53645e-1f28-4eb1-807f-eb2a473f9d4c;"{""type"":""Point"",""coordinates"":[11.8191,53.4216],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 86;;;;false;2;0.4;1.0;Niederspannung +0d6c3f2b-5296-4ec1-995c-b150e72f035f;"{""type"":""Point"",""coordinates"":[11.8175,53.4248],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 80;;;;false;2;0.4;1.0;Niederspannung +ea4a6507-e504-4542-be5f-1019719b2257;"{""type"":""Point"",""coordinates"":[11.8184,53.4293],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 111;;;;false;2;0.4;1.0;Niederspannung +9644f198-e801-4545-87ee-a24e2a8039bd;"{""type"":""Point"",""coordinates"":[11.8221,53.429],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 39;;;;false;2;0.4;1.0;Niederspannung +a9288e77-2919-4db6-89eb-9737bd07f111;"{""type"":""Point"",""coordinates"":[11.8214,53.4246],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 63;;;;false;2;0.4;1.0;Niederspannung +d6dbb0ae-13c9-438e-93b3-b6c63a0708df;"{""type"":""Point"",""coordinates"":[11.816,53.4206],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 89;;;;false;2;0.4;1.0;Niederspannung +5596da2f-ca32-4ad3-81da-1ffa17cd3d7b;"{""type"":""Point"",""coordinates"":[11.8198,53.4203],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 102;;;;false;2;0.4;1.0;Niederspannung +ad30322c-0c99-4669-8e4b-25265087a66d;"{""type"":""Point"",""coordinates"":[11.8166,53.4227],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 65;;;;false;2;0.4;1.0;Niederspannung +120eaa58-a500-4ae2-a86a-56a40b931ec1;"{""type"":""Point"",""coordinates"":[11.8195,53.4272],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 57;;;;false;2;0.4;1.0;Niederspannung +29516ae3-6676-4797-99c1-1f0a32b989d8;"{""type"":""Point"",""coordinates"":[11.8195,53.425],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 52;;;;false;2;0.4;1.0;Niederspannung +24b63115-12eb-4e77-b9ef-ca474fed960f;"{""type"":""Point"",""coordinates"":[11.8201,53.4232],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 46;;;;false;2;0.4;1.0;Niederspannung +a7725293-05fc-447f-bc12-38b689b0a956;"{""type"":""Point"",""coordinates"":[11.8201,53.4225],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 72;;;;false;2;0.4;1.0;Niederspannung +3b86661a-187d-4aa6-bf37-2014789afc08;"{""type"":""Point"",""coordinates"":[11.8214,53.4218],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 20;;;;false;2;0.4;1.0;Niederspannung +a3ed5af5-1fcb-4fce-af0f-708d3d604124;"{""type"":""Point"",""coordinates"":[11.8196,53.4282],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 33;;;;false;2;0.4;1.0;Niederspannung +ec8f2c82-a1b2-487c-b573-250859e3b414;"{""type"":""Point"",""coordinates"":[11.8166,53.4241],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";MV2.101 Bus 5;;;;true;1;20.0;1.025;Mittelspannung +5f153bf6-4f25-41f1-8545-18fe6323bc49;"{""type"":""Point"",""coordinates"":[11.8175,53.4257],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 17;;;;false;2;0.4;1.0;Niederspannung +b565ae3b-68f9-4bca-816d-9b0fc1c6b13f;"{""type"":""Point"",""coordinates"":[11.8198,53.4221],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 70;;;;false;2;0.4;1.0;Niederspannung +8e809cf6-7e05-437c-89a6-f6ca135a403b;"{""type"":""Point"",""coordinates"":[11.815,53.4289],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 47;;;;false;2;0.4;1.0;Niederspannung +ca0b9e34-ddc8-405e-86a4-ab939626c4a2;"{""type"":""Point"",""coordinates"":[11.816,53.4214],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 99;;;;false;2;0.4;1.0;Niederspannung +3e6be3ac-2b51-4080-b815-391313612fc7;"{""type"":""Point"",""coordinates"":[11.815,53.4277],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 110;;;;false;2;0.4;1.0;Niederspannung +4749ab2b-4d96-4100-8081-73e77c797d6b;"{""type"":""Point"",""coordinates"":[11.8166,53.4241],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 73;;;;false;2;0.4;1.0;Niederspannung +3f6c26dd-842b-4dee-b71f-4aa32e2654ff;"{""type"":""Point"",""coordinates"":[11.8174,53.4281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 97;;;;false;2;0.4;1.0;Niederspannung +c6dac3ab-f44f-4b87-800c-0f4da64673f1;"{""type"":""Point"",""coordinates"":[11.8191,53.422],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 10;;;;false;2;0.4;1.0;Niederspannung +eae8a04c-44f2-4da3-95f6-cae48f85737c;"{""type"":""Point"",""coordinates"":[11.8203,53.4221],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 45;;;;false;2;0.4;1.0;Niederspannung +94fe96b1-f36a-4edd-a107-4ff0376f1066;"{""type"":""Point"",""coordinates"":[11.8175,53.4265],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 61;;;;false;2;0.4;1.0;Niederspannung +31e6e197-719d-4aaf-8ca5-ab9e7549390e;"{""type"":""Point"",""coordinates"":[11.8196,53.4286],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 32;;;;false;2;0.4;1.0;Niederspannung +002a4495-96e4-49c9-abbe-8fccb3e9c83e;"{""type"":""Point"",""coordinates"":[11.8214,53.4229],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 90;;;;false;2;0.4;1.0;Niederspannung +c7e48384-5699-4a38-a887-7e15a9145202;"{""type"":""Point"",""coordinates"":[11.8191,53.4223],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 107;;;;false;2;0.4;1.0;Niederspannung +8ace5c2b-584a-4015-990f-6f1e14de4ddb;"{""type"":""Point"",""coordinates"":[11.8174,53.4273],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 27;;;;false;2;0.4;1.0;Niederspannung +c41ebab8-16a0-4a3a-b4af-26073932d462;"{""type"":""Point"",""coordinates"":[11.8168,53.4285],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 35;;;;false;2;0.4;1.0;Niederspannung +ab8c8f6c-e7a2-4b81-a0d7-5f13789267a2;"{""type"":""Point"",""coordinates"":[11.815,53.4269],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 113;;;;false;2;0.4;1.0;Niederspannung +9f1baf4e-12e1-41d1-8efc-81cfc78f1957;"{""type"":""Point"",""coordinates"":[11.8195,53.426],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 71;;;;false;2;0.4;1.0;Niederspannung +02e63e81-2e62-4ef6-8fdb-0b0905a437b6;"{""type"":""Point"",""coordinates"":[11.8191,53.4212],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 12;;;;false;2;0.4;1.0;Niederspannung +2d33314e-31db-4ad4-a898-2be5d56a896d;"{""type"":""Point"",""coordinates"":[11.8162,53.4277],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 48;;;;false;2;0.4;1.0;Niederspannung +b8e95bf0-3ba8-4d53-a0bf-a3720fb785fb;"{""type"":""Point"",""coordinates"":[11.8195,53.4264],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 49;;;;false;2;0.4;1.0;Niederspannung +d0bfabdb-0e83-423b-a20a-ab9197c4284e;"{""type"":""Point"",""coordinates"":[11.8213,53.4239],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 82;;;;false;2;0.4;1.0;Niederspannung +6b2881ce-3965-4f5e-98a3-74eb47b0a7ca;"{""type"":""Point"",""coordinates"":[11.8162,53.4284],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 53;;;;false;2;0.4;1.0;Niederspannung +af2b07ce-1a96-4b50-9e21-badf29eed519;"{""type"":""Point"",""coordinates"":[11.8198,53.4195],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 100;;;;false;2;0.4;1.0;Niederspannung +844c0b9c-058a-4228-a8c4-bf2defff6958;"{""type"":""Point"",""coordinates"":[11.8213,53.4293],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 1;;;;false;2;0.4;1.0;Niederspannung +a432b8ce-0462-478b-83e7-3107cd2e909c;"{""type"":""Point"",""coordinates"":[11.8213,53.4253],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 56;;;;false;2;0.4;1.0;Niederspannung +4129e079-6712-4275-911c-36729d698c42;"{""type"":""Point"",""coordinates"":[11.8166,53.4238],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 77;;;;false;2;0.4;1.0;Niederspannung +883edf38-9a18-4f61-981a-691aaf436cc7;"{""type"":""Point"",""coordinates"":[11.8198,53.4216],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 4;;;;false;2;0.4;1.0;Niederspannung +f2724954-34d3-4ddf-b6b0-7a1531639990;"{""type"":""Point"",""coordinates"":[11.8191,53.4235],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 6;;;;false;2;0.4;1.0;Niederspannung +5d3bcf55-0520-43ff-8d63-3d0eb421e442;"{""type"":""Point"",""coordinates"":[11.816,53.4234],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 91;;;;false;2;0.4;1.0;Niederspannung +3a557b4e-06b8-4f29-929f-81d95c42c897;"{""type"":""Point"",""coordinates"":[11.8213,53.4277],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 42;;;;false;2;0.4;1.0;Niederspannung +7dc43c81-9a61-45a0-9745-800a28bf4a9d;"{""type"":""Point"",""coordinates"":[11.8196,53.4278],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 50;;;;false;2;0.4;1.0;Niederspannung +fd4bebb8-40ca-4eed-92c0-cdd10b86ac20;"{""type"":""Point"",""coordinates"":[11.8191,53.4209],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 60;;;;false;2;0.4;1.0;Niederspannung +3464496c-7dd4-41e9-ae0a-99ade0b51572;"{""type"":""Point"",""coordinates"":[11.8213,53.4236],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 40;;;;false;2;0.4;1.0;Niederspannung +d38d936a-9c05-4bdc-8331-418fef27f492;"{""type"":""Point"",""coordinates"":[11.815,53.4281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 109;;;;false;2;0.4;1.0;Niederspannung +58b551b6-83bd-4f1c-8d9c-8c9a7f638c0b;"{""type"":""Point"",""coordinates"":[11.8213,53.425],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 38;;;;false;2;0.4;1.0;Niederspannung +b5c1e826-63fd-4b0c-bec0-0c758389ef58;"{""type"":""Point"",""coordinates"":[11.8191,53.4239],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 104;;;;false;2;0.4;1.0;Niederspannung +7c35a794-f569-4a9c-acb0-d03647610086;"{""type"":""Point"",""coordinates"":[11.8166,53.4249],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 98;;;;false;2;0.4;1.0;Niederspannung +78815cf6-70db-432c-96e6-87fe8cf67eee;"{""type"":""Point"",""coordinates"":[11.815,53.4272],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 112;;;;false;2;0.4;1.0;Niederspannung +c5b6bfaf-1621-40a7-9c53-02cfb59c04d9;"{""type"":""Point"",""coordinates"":[11.822,53.4294],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 14;;;;false;2;0.4;1.0;Niederspannung +2b3d7fb8-0583-4d47-97b1-3b5f232fd462;"{""type"":""Point"",""coordinates"":[11.8213,53.4274],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 87;;;;false;2;0.4;1.0;Niederspannung +3fcb94e3-7781-4d83-9030-d9853822e78e;"{""type"":""Point"",""coordinates"":[11.8213,53.4243],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 85;;;;false;2;0.4;1.0;Niederspannung +f713593a-3fd3-410a-ac08-74202d4f5798;"{""type"":""Point"",""coordinates"":[11.8201,53.4236],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 79;;;;false;2;0.4;1.0;Niederspannung +47246a84-ad0e-4d04-9d98-1c9cd5d363c1;"{""type"":""Point"",""coordinates"":[11.8196,53.4289],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 16;;;;false;2;0.4;1.0;Niederspannung +9cdb3115-cc00-4d61-bc33-442e8f30fb63;"{""type"":""Point"",""coordinates"":[11.8213,53.4257],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 83;;;;false;2;0.4;1.0;Niederspannung +839ff0f4-93db-42ec-a928-bbc448b6cf5c;"{""type"":""Point"",""coordinates"":[11.8197,53.4207],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 69;;;;false;2;0.4;1.0;Niederspannung +e7908208-77b4-4059-806e-4857262992fc;"{""type"":""Point"",""coordinates"":[11.8196,53.4247],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 105;;;;false;2;0.4;1.0;Niederspannung +83da8d60-405a-45f7-9bb9-9d35607b7927;"{""type"":""Point"",""coordinates"":[11.8201,53.4229],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 2;;;;false;2;0.4;1.0;Niederspannung +9d136a6b-5fdc-44ed-a5ed-599a55281024;"{""type"":""Point"",""coordinates"":[11.8195,53.4275],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 9;;;;false;2;0.4;1.0;Niederspannung +3faac527-0ff3-44a7-9e4f-24a41940da90;"{""type"":""Point"",""coordinates"":[11.816,53.4227],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 15;;;;false;2;0.4;1.0;Niederspannung +5682ac05-7336-4ebc-a5d1-3b69c79fb3b1;"{""type"":""Point"",""coordinates"":[11.8166,53.4241],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 44;;;;false;2;0.4;1.0;Niederspannung +5b73ded9-3ca4-4f18-a2ab-c27b9a3dcf9d;"{""type"":""Point"",""coordinates"":[11.8166,53.423],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 36;;;;false;2;0.4;1.0;Niederspannung +594d101c-3a05-45e3-a061-9189f3e848b7;"{""type"":""Point"",""coordinates"":[11.8198,53.4192],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 88;;;;false;2;0.4;1.0;Niederspannung +7b81b518-00e0-4ff1-b4cf-876903958d7a;"{""type"":""Point"",""coordinates"":[11.8213,53.4271],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 8;;;;false;2;0.4;1.0;Niederspannung +a882e666-82d1-4ba6-87df-fc702fe06187;"{""type"":""Point"",""coordinates"":[11.8169,53.4289],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 74;;;;false;2;0.4;1.0;Niederspannung +1403edf9-e47c-4705-8563-83bcd639482e;"{""type"":""Point"",""coordinates"":[11.8213,53.4286],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 75;;;;false;2;0.4;1.0;Niederspannung +03b2aa45-84f6-48c0-9dab-427e046a5672;"{""type"":""Point"",""coordinates"":[11.8196,53.4253],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 22;;;;false;2;0.4;1.0;Niederspannung +6ee7ea93-ea9c-40cb-b79a-1c5f287c97a5;"{""type"":""Point"",""coordinates"":[11.8221,53.4286],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 66;;;;false;2;0.4;1.0;Niederspannung +88cf719a-92df-4dfd-9a83-f84330e28fe0;"{""type"":""Point"",""coordinates"":[11.815,53.4285],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 108;;;;false;2;0.4;1.0;Niederspannung +80962bd3-a10f-4ed2-ba6a-3e802189939c;"{""type"":""Point"",""coordinates"":[11.8201,53.4239],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 21;;;;false;2;0.4;1.0;Niederspannung +80b8d1f8-7e83-421d-a95a-c193fc35f4f7;"{""type"":""Point"",""coordinates"":[11.8175,53.427],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 95;;;;false;2;0.4;1.0;Niederspannung +b909fb45-b6ee-427f-afd7-e8a0ec7274c6;"{""type"":""Point"",""coordinates"":[11.8213,53.4225],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 92;;;;false;2;0.4;1.0;Niederspannung +57b40047-4f9d-46bb-bf19-c0a86bbd4f5b;"{""type"":""Point"",""coordinates"":[11.8166,53.4234],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 37;;;;false;2;0.4;1.0;Niederspannung +18b4157b-0e47-4c5a-adb8-ccae47372336;"{""type"":""Point"",""coordinates"":[11.8213,53.4264],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 5;;;;false;2;0.4;1.0;Niederspannung +8726dc29-621e-4455-a541-cd88d7da457f;"{""type"":""Point"",""coordinates"":[11.8166,53.4257],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 96;;;;false;2;0.4;1.0;Niederspannung +814f784b-687f-4dd5-8a91-c7772c916d46;"{""type"":""Point"",""coordinates"":[11.8174,53.4262],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 19;;;;false;2;0.4;1.0;Niederspannung +7efabb8d-ba17-4487-96d9-5744b1fedf8a;"{""type"":""Point"",""coordinates"":[11.8166,53.4245],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 23;;;;false;2;0.4;1.0;Niederspannung +15a86f7d-fb73-49a4-af6a-25b14122378d;"{""type"":""Point"",""coordinates"":[11.8191,53.4227],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 11;;;;false;2;0.4;1.0;Niederspannung +34031e92-3444-47d5-94ae-cceeb5d96bb2;"{""type"":""Point"",""coordinates"":[11.8195,53.4268],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 106;;;;false;2;0.4;1.0;Niederspannung +35748e60-3be8-4930-8a61-209fd5df1bec;"{""type"":""Point"",""coordinates"":[11.815,53.4293],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 30;;;;false;2;0.4;1.0;Niederspannung +50cac08e-bf24-4526-9466-53ca5edccd15;"{""type"":""Point"",""coordinates"":[11.8166,53.4253],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 64;;;;false;2;0.4;1.0;Niederspannung +2efac9b1-fb0d-4e08-bfac-501798826deb;"{""type"":""Point"",""coordinates"":[11.8191,53.4231],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 93;;;;false;2;0.4;1.0;Niederspannung +c86d6361-4159-4787-b5f4-e41dcaa95195;"{""type"":""Point"",""coordinates"":[11.8182,53.4257],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 13;;;;false;2;0.4;1.0;Niederspannung diff --git a/input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/pv_input.csv b/input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/pv_input.csv new file mode 100644 index 0000000000..b883b51783 --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/pv_input.csv @@ -0,0 +1,4 @@ +uuid;albedo;azimuth;cos_phi_rated;elevation_angle;eta_conv;id;k_g;k_t;market_reaction;node;operates_from;operates_until;operator;q_characteristics;s_rated;em +a1eb7fc1-3bee-4b65-a387-ef3046644bf0;0.20000000298023224;4.093344211578369;0.8999999761581421;37.69556427001953;97.0;LV5.201 PV 10;0.8999999761581421;1.0;false;dc54bd8a-b7d8-4e99-adb0-d6ee5084241c;;;;"cosPhiFixed:{(0.0,0.9)}";4.2;f9dc7ce6-658c-4101-a12f-d58bb889286b +de8cfef5-7620-4b9e-9a10-1faebb5a80c0;0.20000000298023224;4.093344211578369;0.8999999761581421;37.69556427001953;97.0;LV5.201 PV 11;0.8999999761581421;1.0;false;3e6be3ac-2b51-4080-b815-391313612fc7;;;;"cosPhiFixed:{(0.0,0.9)}";10.9;957938b7-0476-4fab-a1b3-6ce8615857b3 +2560c371-f420-4c2a-b4e6-e04c11b64c03;0.20000000298023224;0.7802008390426636;0.8999999761581421;40.086585998535156;98.0;LV5.201 PV 15;0.8999999761581421;1.0;false;43040a39-8b6c-401f-9dfd-82b42aa6dec6;;;;"cosPhiFixed:{(0.0,0.9)}";2.9;c3a7e9f5-b492-4c85-af2d-1e93f6a25443 diff --git a/input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/storage_input.csv b/input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/storage_input.csv new file mode 100644 index 0000000000..60b2df901a --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/storage_input.csv @@ -0,0 +1,2 @@ +uuid;id;node;operates_from;operates_until;operator;q_characteristics;type;em +06b58276-8350-40fb-86c0-2414aa4a0452;LV5.201 Storage 64;dc54bd8a-b7d8-4e99-adb0-d6ee5084241c;;;;"cosPhiFixed:{(0.00,0.95)}";95d4c980-d9e1-4813-9f2a-b0942488a570;f9dc7ce6-658c-4101-a12f-d58bb889286b diff --git a/input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/storage_type_input.csv b/input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/storage_type_input.csv new file mode 100644 index 0000000000..8c6fb1c867 --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/storage_type_input.csv @@ -0,0 +1,2 @@ +uuid;active_power_gradient;capex;cos_phi_rated;e_storage;eta;id;opex;p_max;s_rated +95d4c980-d9e1-4813-9f2a-b0942488a570;1.0;0.0;0.96;8.0;93.0;Typ_1;0.65;4.0;4.166666666666667 diff --git a/input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/transformer_2_w_input.csv b/input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/transformer_2_w_input.csv new file mode 100644 index 0000000000..d1f9f5d13c --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/transformer_2_w_input.csv @@ -0,0 +1,2 @@ +uuid;auto_tap;id;node_a;node_b;operates_from;operates_until;operator;parallel_devices;tap_pos;type +adaba416-9b52-45df-9d91-f67d0dd28ecb;false;MV2.101-LV5.201-Trafo 1;ec8f2c82-a1b2-487c-b573-250859e3b414;4749ab2b-4d96-4100-8081-73e77c797d6b;;;;1;0;417407d2-1e74-4f37-9b64-f701f53f8842 diff --git a/input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/transformer_2_w_type_input.csv b/input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/transformer_2_w_type_input.csv new file mode 100644 index 0000000000..b13848b159 --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced_with_storage/fullGrid/transformer_2_w_type_input.csv @@ -0,0 +1,2 @@ +uuid;b_m;d_phi;d_v;g_m;id;r_sc;s_rated;tap_max;tap_min;tap_neutr;tap_side;v_rated_a;v_rated_b;x_sc +417407d2-1e74-4f37-9b64-f701f53f8842;-36.47380569074435;0.0;2.5;4124.999999999999;0.63 MVA 20/0.4 kV Dyn5 ASEA;6.953892668178382;630.0;2;-2;0;false;20.0;0.4;37.45518044666632 diff --git a/input/samples/simopsimtestgrid_reduced_with_storage/simopsimtestgrid.conf b/input/samples/simopsimtestgrid_reduced_with_storage/simopsimtestgrid.conf new file mode 100644 index 0000000000..5ed8c36fcd --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced_with_storage/simopsimtestgrid.conf @@ -0,0 +1,249 @@ +include "../common/pekko.conf" + +######### +# ATTENTION: Do not change this config file directly but use it as a base for your personal delta config for the +# vn_simona scenario! Delta configs can be created by including the config you want to change +# parameters from via include (e.g. include "input/samples/vn_simona/vn_simona.conf") at the +# beginning of your config file and then just override the parameters you want to change! +######### + +################################################################## +# Simulation Parameters +################################################################## +simona.simulationName = "simopsimtest" + +################################################################## +# Time Parameters +################################################################## +simona.time.startDateTime = "2024-02-27T00:00:00Z" +simona.time.endDateTime = "2024-02-28T00:00:00Z" +simona.time.schedulerReadyCheckWindow = 900 + +################################################################## +# Input Parameters +################################################################## +simona.input.primary.csvParams = { + directoryPath: "simona/input/samples/simopsimtestgrid_reduced_with_storage/fullGrid" + csvSep: ";" + isHierarchic: false +} +simona.input.grid.datasource.id = "csv" +simona.input.grid.datasource.csvParams = { + directoryPath: "simona/input/samples/simopsimtestgrid_reduced_with_storage/fullGrid" + csvSep: ";" + isHierarchic: false +} + +simona.input.weather.datasource = { + scheme = "icon" + sampleParams.use = true + coordinateSource.sampleParams.use = true + maxCoordinateDistance = 50000 +} + +################################################################## +# Output Parameters +################################################################## +simona.output.base.dir = "simona/output/simopsimtestgrid_reduced_with_storage" +simona.output.base.addTimestampToOutputDir = true + +simona.output.sink.csv { + fileFormat = ".csv" + filePrefix = "" + fileSuffix = "" +} + +simona.output.grid = { + notifier = "grid" + nodes = false + lines = false + switches = false + transformers2w = false + transformers3w = false +} +simona.output.participant.defaultConfig = { + notifier = "default" + powerRequestReply = false + simulationResult = true +} +simona.output.participant.individualConfigs = [ + { + notifier = "pv" + powerRequestReply = false + simulationResult = true + }, + { + notifier = "wec" + powerRequestReply = false + simulationResult = false + }, + { + notifier = "evcs" + powerRequestReply = false + simulationResult = false + }, + { + notifier = "bm" + powerRequestReply = false + simulationResult = false + }, + { + notifier = "chp" + powerRequestReply = false + simulationResult = false + }, + { + notifier = "ev" + powerRequestReply = false + simulationResult = false + }, + { + notifier = "hp" + powerRequestReply = false + simulationResult = false + }, + { + notifier = "storage" + powerRequestReply = false + simulationResult = true + }, + { + notifier = "fixedFeedIn" + powerRequestReply = false + simulationResult = false + } +] +simona.output.thermal = { + defaultConfig = { + notifier = "default", + simulationResult = false + } + individualConfigs = [ + { + notifier = "house", + simulationResult = false + } + ] +} + +################################################################## +# Runtime Configuration // todo refactor as this naming is misleading and partly unneeded +################################################################## +simona.runtime.selected_subgrids = [] +simona.runtime.selected_volt_lvls = [] + +simona.runtime.participant.load = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + modelBehaviour = "fix" + reference = "power" + } + individualConfigs = [] +} + +simona.runtime.participant.fixedFeedIn = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +simona.runtime.participant.pv = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +simona.runtime.participant.wec = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +simona.runtime.participant.evcs = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +simona.runtime.participant.hp = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +simona.runtime.participant.storage = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +# # # # # +# ATTENTION: calculateMissingReactivePowerWithModel and scaling is ignored here. +# # # # # +simona.runtime.participant.em = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +################################################################## +# Event Configuration +################################################################## +simona.event.listener = [] + +################################################################## +# Grid Configuration +################################################################## + +simona.gridConfig.refSystems = [ + {sNom = "100 kVA", vNom = "0.4 kV", voltLvls = [{id = "NS", vNom = "0.4 kV"}]}, + {sNom = "60 MVA", vNom = "20 kV", voltLvls = [{id = "MS", vNom = "20 kV"}]}, + {sNom = "600 MVA", vNom = "110 kV", voltLvls = [{id = "HS", vNom = "110 kV"}]}, + {sNom = "1000 MVA", vNom = "380 kV", voltLvls = [{id = "HoeS", vNom = "380 kV"}]} +] + +################################################################## +# Power Flow Configuration +################################################################## +simona.powerflow.maxSweepPowerDeviation = 1E-5 // the maximum allowed deviation in power between two sweeps, before overall convergence is assumed +simona.powerflow.newtonraphson.epsilon = [1E-12] +simona.powerflow.newtonraphson.iterations = 50 +simona.powerflow.resolution = "87300s" +simona.powerflow.stopOnFailure = true + +simona.control.transformer = [ + { + transformers = ["31a2b9bf-e785-4475-aa44-1c34646e8c79"], + measurements = ["923f2d69-3093-4198-86e4-13d2d1c220f8"], + vMin = 0.98, + vMax = 1.02 + }, { + transformers = ["1132dbf4-e8a1-44ae-8415-f42d4497aa1d"], + measurements = ["7686b818-a0ba-465c-8e4e-f7d3c4e171fc"], + vMin = 0.98, + vMax = 1.02 + } +] diff --git a/input/samples/vn_simona_small_with_em/fullGrid/storage_input.csv b/input/samples/vn_simona_small_with_em/fullGrid/storage_input.csv index 35eb72ffac..790f88b930 100644 --- a/input/samples/vn_simona_small_with_em/fullGrid/storage_input.csv +++ b/input/samples/vn_simona_small_with_em/fullGrid/storage_input.csv @@ -1,3 +1,2 @@ uuid,id,node,operates_from,operates_until,operator,q_characteristics,type,em a2a92cfd-3492-465f-9587-e789f4620af8,Speicher_3,33f29587-f63e-45b7-960b-037bda37a3cb,,,,"cosPhiFixed:{(0.0,0.98)}",95d4c980-d9e1-4813-9f2a-b0942488a570,fd1a8de9-722a-4304-8799-e1e976d9979c -c96f0a90-980a-4813-b175-ecc04aded773,Speicher_4,401f37f8-6f2c-4564-bc78-6736cb9cbf8d,,,,"cosPhiFixed:{(0.0,0.98)}",95d4c980-d9e1-4813-9f2a-b0942488a570,ff0b995a-86ff-4f4d-987e-e475a64f2180 diff --git a/input/samples/vn_simona_small_with_em/vn_simona_small_withem.conf b/input/samples/vn_simona_small_with_em/vn_simona_small_withem.conf index 6185ad6e4d..bc2c8b3451 100644 --- a/input/samples/vn_simona_small_with_em/vn_simona_small_withem.conf +++ b/input/samples/vn_simona_small_with_em/vn_simona_small_withem.conf @@ -81,6 +81,11 @@ simona.output.participant.individualConfigs = [ notifier = "evcs" powerRequestReply = false simulationResult = true + }, + { + notifier = "storage" + powerRequestReply = false + simulationResult = true } ] simona.output.thermal = { @@ -158,6 +163,15 @@ simona.runtime.participant.hp = { individualConfigs = [] } +simona.runtime.participant.storage = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + # # # # # # ATTENTION: calculateMissingReactivePowerWithModel and scaling is ignored here. # # # # # diff --git a/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala b/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala index 465bbd319f..6e85277f50 100644 --- a/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala +++ b/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala @@ -8,7 +8,7 @@ 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.data.Data.PrimaryData.{ApparentPower, ZERO_POWER} import edu.ie3.simona.agent.participant.statedata.BaseStateData.FlexControlledData import edu.ie3.simona.config.SimonaConfig.EmRuntimeConfig import edu.ie3.simona.event.ResultEvent @@ -17,7 +17,7 @@ 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.FlexibilityMessage.{IssueFlexControl, _} import edu.ie3.simona.ontology.messages.flex.MinMaxFlexibilityMessage.ProvideMinMaxFlexOptions import edu.ie3.simona.ontology.messages.services.ServiceMessage.ExtEmDataServiceRegistrationMessage import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.WrappedRegistrationSuccessfulMessage @@ -30,6 +30,7 @@ import org.apache.pekko.actor.typed.scaladsl.{Behaviors, StashBuffer} import org.apache.pekko.actor.typed.{ActorRef, Behavior} import org.apache.pekko.actor.{ActorRef => ClassicRef} import org.apache.pekko.actor.typed.scaladsl.adapter._ +import squants.Power import java.time.ZonedDateTime @@ -72,7 +73,6 @@ object EmAgent { override val tick: Long = msg.tick } - /** Creates the initial [[Behavior]] for an [[EmAgent]] in an inactive state * * @param inputModel @@ -103,12 +103,14 @@ object EmAgent { extEmDataService: Option[ClassicRef] ): Behavior[Request] = Behaviors.setup[Request] { ctx => val flexAdapterEmDataService = ctx.messageAdapter[FlexRequest](Flex) + var extInitTick = Option.empty[Long] if (extEmDataService.isDefined) { extEmDataService.getOrElse(throw new RuntimeException("No Service")) ! ExtEmDataServiceRegistrationMessage( inputModel.getUuid, ctx.self, flexAdapterEmDataService ) + extInitTick = Some(0L) } val constantData = EmData( @@ -135,7 +137,7 @@ object EmAgent { } }, listener, - ExternalEmDataServiceData(extEmDataService, None) + ExternalEmDataServiceData(extEmDataService) ) @@ -151,7 +153,7 @@ object EmAgent { inactive( constantData, modelShell, - EmDataCore.create(simulationStartDate) + EmDataCore.create(simulationStartDate, extInitTick) ) } @@ -171,7 +173,7 @@ object EmAgent { inactive(emData, updatedModelShell, updatedCore) case (ctx, WrappedRegistrationSuccessfulMessage(RegistrationSuccessfulMessage(serviceRef, nextDataTick))) => - ctx.log.info(s"EM Agent ${modelShell.uuid} will use external set points!") + ctx.log.info(s"EM Agent ${ctx.self} will use external set points!") /* val flexAdapter = ctx.messageAdapter[FlexRequest](Flex) val updatedEmData = emData.copy( @@ -215,19 +217,45 @@ object EmAgent { inactive(emData, modelShell, newCore) case (ctx, msg: ActivationRequest) => - ctx.log.debug(s"EM Agent ${modelShell.uuid} got ActivationRequest = $msg") - val flexOptionsCore = core.activate(msg.tick) + ctx.log.debug(s"\u001b[0;34m[${msg.tick}] ${ctx.self}.inactive got ActivationRequest = $msg, dataProvisionMessage = ${core.nextSetPointMessage}, nextSetPointTick = ${core.nextSetPointTick}\u001b[0;0m") msg match { - case Flex(_: RequestFlexOptions) | EmActivation(_) => - ctx.log.debug(s"Activation for tick ${msg.tick}") - val (toActivate, newCore) = flexOptionsCore.takeNewFlexRequests() - toActivate.foreach { - _ ! RequestFlexOptions(msg.tick) + case Flex(_: RequestFlexOptions) | EmActivation(_) => // Activation by another EMAgent or by the scheduler + val flexOptionsCore = core.activate(msg.tick) + + // Check if there will be a new set point for this tick -> We can't start processing flex options before we know what's the set point for this tick + if (core.nextSetPointTick.contains(msg.tick)) { + // We expect a new set point for this tick + core.nextSetPointMessage match { + case Some(setPointMsg) => // We already got a set point, check if the set point is for the right tick + if (setPointMsg.tick == msg.tick) { // yes, it's for the right tick -> we can activate our connected agents and do the normal stuff + val (toActivate, newCore) = flexOptionsCore.handleSetPoint(setPointMsg).takeNewFlexRequests() + ctx.log.debug(s"\u001b[0;34m[${flexOptionsCore.activeTick}] ${ctx.self}.inactive expects and received set point for this tick\n -> activate connected agents $toActivate\n -> send IssuePowerControl to myself with the new set point ${setPointMsg.setPower} \u001b[0;0m") + toActivate.foreach { + _ ! RequestFlexOptions(msg.tick) + } + ctx.self ! Flex(IssuePowerControl(flexOptionsCore.activeTick, setPointMsg.setPower)) + awaitingFlexOptions(emData, modelShell, newCore) + } else { + throw new RuntimeException("Set point for wrong tick arrived!") + } + case _ => // We still have to wait for a set point + val (toActivate, newCore) = flexOptionsCore.takeNewFlexRequests() + ctx.log.debug(s"\u001b[0;34m[${flexOptionsCore.activeTick}] ${ctx.self}.inactive expects set point for this tick, but I have to wait..., toActivate = $toActivate\u001b[0;0m") + toActivate.foreach { + _ ! RequestFlexOptions(msg.tick) + } + awaitingFlexOptions(emData, modelShell, flexOptionsCore) + } + } else { // We don't expect a new set point -> we can do our normal stuff, because we are activated because at least one connected agent should provide flex options + val (toActivate, newCore) = flexOptionsCore.updateSetPoint().takeNewFlexRequests() + ctx.log.debug(s"\u001b[0;34m[${flexOptionsCore.activeTick}] EM Agent ${ctx.self} doesn't expect set point for this tick, toActivate = $toActivate\u001b[0;0m") + toActivate.foreach { + _ ! RequestFlexOptions(msg.tick) + } + awaitingFlexOptions(emData, modelShell, newCore) } - 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 @@ -235,19 +263,19 @@ object EmAgent { // Thus, we just jump to the appropriate place and forward the // control message there + val flexOptionsCore = core.activate(msg.tick) ctx.self ! msg awaitingFlexCtrl(emData, modelShell, flexOptionsCore) - case Flex(dataMsg: ProvideExtEmSetPoint) => - // got set point before activation -> put msg in queue and wait - ctx.log.debug(s"Agent ${ctx.self} got external set point = $dataMsg") - val updatedEmData = emData.copy( - extEmDataServiceData = emData.extEmDataServiceData.copy( - dataProvisionMessage = Some(dataMsg) - ) - ) - inactive(updatedEmData, modelShell, core) + + case Flex(msg: SetPointFlexRequest) => + // We didn't get an activation yet, but a set point arrived -> save message and wait for an activation + ctx.log.debug(s"(${core.getLastActiveTick}) ${ctx.self}.inactive got external set point = $msg before activation -> save message and wait...") + val newCore = core.handleSetPointMessage(msg) + + inactive(emData, modelShell, newCore) } + } /** Behavior of an [[EmAgent]] waiting for flex options to be received in @@ -311,30 +339,9 @@ object EmAgent { case Left(_) => // We're not em-controlled ourselves, // always desire to come as close as possible to 0 kW -> maybe overwrite it if we get a set point - var setPower = zeroKW - var updatedEmData = emData - if (emData.extEmDataServiceData.extEmDataService.isDefined) { // We get external set points - if (emData.extEmDataServiceData.dataProvisionMessage.isEmpty) { // Still waiting for set points... - awaitingFlexOptions( - emData, - modelShell, - updatedCore, - ) - } else { // We got set points - setPower = emData.extEmDataServiceData.dataProvisionMessage.map(setPoint => setPoint.setPower).getOrElse(throw new RuntimeException("Got a wrong set point!")) - ctx.log.debug(s"[UNCONTROLLED] EM Agent ${ctx.self}: Got a external Set Power = $setPower") - updatedEmData = emData.copy( - extEmDataServiceData = emData.extEmDataServiceData.copy( - dataProvisionMessage = None - ) - ) - } - } - ctx.log.debug(s"[UNCONTROLLED] EM Agent ${ctx.self}: Starting determination of flex control with set power = $setPower") + val setPower = updatedCore.currentSetPower.getOrElse(throw new CriticalFailureException("Uncontrolled agent received ProvideFlexOptions without a set point!")) val flexControl = modelShell.determineFlexControl(allFlexOptions, setPower) - ctx.log.debug(s"[UNCONTROLLED] EM Agent ${ctx.self}: Got flexControl = $flexControl") - val (allFlexMsgs, newCore) = updatedCore .handleFlexCtrl(flexControl) @@ -342,11 +349,10 @@ object EmAgent { .complete() allFlexMsgs.foreach { case (actor, msg) => - ctx.log.debug(s"[UNCONTROLLED] EM Agent ${ctx.self}: For actor = $actor send msg = $msg") actor ! msg } - awaitingCompletions(updatedEmData, modelShell, newCore) + awaitingCompletions(emData, modelShell, newCore) } } else { // more flex options expected @@ -356,19 +362,72 @@ object EmAgent { updatedCore, ) } - case (ctx, Flex(dataMsg: ProvideExtEmSetPoint)) => - // got set point before activation -> put msg in queue and wait - ctx.log.debug(s"Agent ${ctx.self} got external set point = $dataMsg") - val updatedEmData = emData.copy( - extEmDataServiceData = emData.extEmDataServiceData.copy( - dataProvisionMessage = Some(dataMsg) - ) - ) - awaitingFlexOptions(updatedEmData, modelShell, flexOptionsCore) + /* 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 */ + case (ctx, Flex(setPointMsg: SetPointFlexRequest)) => + // We got a set point after Activation -> Check, if setPower changed (yes) we have to calculate new set points for our connected agents (no) activate core and do the updates + ctx.log.debug(s"${flexOptionsCore.activeTick} ${ctx.self}.awaitingFlexOptions got external set point = $setPointMsg") + val updatedCore = flexOptionsCore.handleSetPoint(setPointMsg) + ctx.self ! Flex(IssuePowerControl(flexOptionsCore.activeTick, setPointMsg.setPower)) + awaitingFlexCtrl(emData, modelShell, updatedCore) + + case (ctx, Flex(flexCtrl: IssuePowerControl)) => + ctx.log.debug(s"[${flexOptionsCore.activeTick}] ${ctx.self}.awaitingFlexOptions.IssuePowerControl received IssuePowerControl $flexCtrl") + if (flexOptionsCore.isComplete) { + ctx.log.debug(s"[${flexOptionsCore.activeTick}] ${ctx.self}.awaitingFlexOptions.IssuePowerControl core is already complete") + val allFlexOptions = flexOptionsCore.getFlexOptions + // We're not em-controlled ourselves, + // always desire to come as close as possible to 0 kW -> maybe overwrite it if we get a set point + val setPower = flexCtrl.setPower + + val flexControl = + modelShell.determineFlexControl(allFlexOptions, setPower) + + val (allFlexMsgs, newCore) = flexOptionsCore + .handleFlexCtrl(flexControl) + .fillInMissingIssueCtrl() + .complete() + + if(allFlexMsgs.isEmpty) { + newCore + .maybeComplete() + .map { inactiveCore => + sendCompletionCommunication( + emData, + modelShell, + inactiveCore, + flexOptionsCore.activeTick, + flexOptionsCore.nextSetPointTick + ) + inactive(emData, modelShell, inactiveCore) + } + .getOrElse { + // more flex options expected + awaitingCompletions( + emData, + modelShell, + newCore, + ) + } + } else { + allFlexMsgs.foreach { case (actor, msg) => + actor ! msg + } + + awaitingCompletions(emData, modelShell, newCore) + } + + } else { + ctx.log.debug(s"[${flexOptionsCore.activeTick}] ${ctx.self}.awaitingFlexOptions.IssuePowerControl there are still missing ProvideFlexOptions -> we have to wait...") + awaitingFlexOptions( + emData, + modelShell, + flexOptionsCore, + ) + } } /** Behavior of an [[EmAgent]] waiting for a flex control message to be @@ -381,9 +440,6 @@ object EmAgent { flexOptionsCore: EmDataCore.AwaitingFlexOptions, ): Behavior[Request] = Behaviors.receivePartial { case (ctx, Flex(flexCtrl: IssueFlexControl)) => - ctx.log.debug(s"emData = $emData") - ctx.log.debug(s"modelShell = $modelShell") - ctx.log.debug(s"agent ${ctx.self}: flexCtrl = $flexCtrl") val flexData = emData.parentData.getOrElse( throw new CriticalFailureException(s"EmAgent is not EM-controlled.") ) @@ -415,7 +471,6 @@ object EmAgent { .complete() allFlexMsgs.foreach { case (actor, msg) => - ctx.log.debug(s"Agent ${ctx.self}: For actor = $actor send msg = $msg") actor ! msg } @@ -441,7 +496,8 @@ object EmAgent { emData, modelShell, inactiveCore, - lastActiveTick = updatedCore.activeTick, + updatedCore.activeTick, + updatedCore.nextSetPointTick ) inactive(emData, modelShell, inactiveCore) } @@ -461,6 +517,7 @@ object EmAgent { modelShell: EmModelShell, inactiveCore: EmDataCore.Inactive, lastActiveTick: Long, + nextSetPointTick: Option[Long], ): Unit = { // calc result val result = inactiveCore.getResults @@ -488,11 +545,16 @@ object EmAgent { ) } + val nextActiveTick = EmTools.minOptionTicks( + inactiveCore.nextActiveTick, + nextSetPointTick + ) + emData.parentData.fold( schedulerData => schedulerData.scheduler ! Completion( schedulerData.activationAdapter, - inactiveCore.nextActiveTick, + nextActiveTick, ), _.emAgent ! FlexCtrlCompletion( modelShell.uuid, @@ -535,10 +597,10 @@ object EmAgent { final case class SchedulerData( scheduler: ActorRef[SchedulerMessage], activationAdapter: ActorRef[Activation], + lastFlexOptions: Option[ProvideFlexOptions] = None, ) final case class ExternalEmDataServiceData( - extEmDataService: Option[ClassicRef], - dataProvisionMessage: Option[ProvideExtEmSetPoint] + extEmDataService: Option[ClassicRef] ) } diff --git a/src/main/scala/edu/ie3/simona/agent/em/EmDataCore.scala b/src/main/scala/edu/ie3/simona/agent/em/EmDataCore.scala index 3b46b176ee..5ad3743bf6 100644 --- a/src/main/scala/edu/ie3/simona/agent/em/EmDataCore.scala +++ b/src/main/scala/edu/ie3/simona/agent/em/EmDataCore.scala @@ -12,6 +12,7 @@ import EmAgent.Actor import FlexCorrespondenceStore.WithTime import edu.ie3.simona.ontology.messages.flex.FlexibilityMessage._ import edu.ie3.util.scala.collection.mutable.PriorityMultiBiSet +import edu.ie3.util.scala.quantities.DefaultQuantities.zeroKW import squants.Power import java.time.ZonedDateTime @@ -27,13 +28,16 @@ object EmDataCore { * @param startDate * The start date of the simulation */ - def create(implicit startDate: ZonedDateTime): Inactive = + def create(implicit startDate: ZonedDateTime, extInitTick: Option[Long]): Inactive = Inactive( Map.empty, PriorityMultiBiSet.empty, Set.empty, FlexCorrespondenceStore(), None, + extInitTick, + zeroKW, + None ) /** Data structure holding relevant data and providing methods that handle @@ -59,6 +63,9 @@ object EmDataCore { private val flexWithNext: Set[UUID], private val correspondences: FlexCorrespondenceStore, private val lastActiveTick: Option[Long], + nextSetPointTick: Option[Long], + lastSetPower: Power, + nextSetPointMessage: Option[SetPointFlexRequest] ) { /** Adds a connected agent, given its model UUID and actor reference @@ -109,6 +116,9 @@ object EmDataCore { updatedQueue, correspondences, activeTick = newTick, + nextSetPointTick = nextSetPointTick, + lastSetPower = lastSetPower, + currentSetPower = None, ) } @@ -165,6 +175,13 @@ object EmDataCore { def getResults: Iterable[ApparentPower] = correspondences.store.values.flatMap(_.receivedResult.map(_.get)) + def handleSetPointMessage( + setPointMsg: SetPointFlexRequest + ): Inactive = copy( + nextSetPointMessage = Some(setPointMsg) + ) + + def getLastActiveTick: Option[Long] = lastActiveTick } /** Data structure holding relevant data and providing methods that handle @@ -189,17 +206,22 @@ object EmDataCore { private val correspondences: FlexCorrespondenceStore, private val awaitedFlexOptions: Set[UUID] = Set.empty, activeTick: Long, + nextSetPointTick: Option[Long], + lastSetPower: Power, + currentSetPower: Option[Power] ) { + def getCorrespondences: FlexCorrespondenceStore = correspondences + /** 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 - */ + * 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 = @@ -219,16 +241,16 @@ object EmDataCore { } /** 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 - */ + * the currently active tick. + * + * @param flexOptions + * The received flex options + * @return + * The updated [[AwaitingFlexOptions]] core + */ def handleFlexOptions( - flexOptions: ProvideFlexOptions - ): AwaitingFlexOptions = + flexOptions: ProvideFlexOptions + ): AwaitingFlexOptions = copy( correspondences = correspondences.updateFlexOptions(flexOptions, activeTick), @@ -236,33 +258,35 @@ object EmDataCore { ) /** 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 + * 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 & currentSetPower.isDefined /** Returns all flex options that are currently relevant, which can include - * flex options received at an earlier tick - * @return - * all relevant flex options - */ + * 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 - */ + * + * @param ctrlMsgs + * The control messages created by this EM agent + * @return + * The updated [[AwaitingFlexOptions]] core + */ def handleFlexCtrl( - ctrlMsgs: Iterable[(UUID, Power)] - ): AwaitingFlexOptions = { + ctrlMsgs: Iterable[(UUID, Power)] + ): AwaitingFlexOptions = { val updatedStore = ctrlMsgs.foldLeft(correspondences) { case (store, (model, power)) => val ctrlMsg = IssuePowerControl(activeTick, power) @@ -272,12 +296,13 @@ object EmDataCore { } /** 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 - */ + * 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) => @@ -296,7 +321,7 @@ object EmDataCore { // at an earlier tick val flexControlCancelled = correspondence.issuedCtrlMsg match { case Some(WithTime(_: IssuePowerControl, tick)) - if tick < activeTick => + if tick < activeTick => true case _ => false } @@ -313,17 +338,17 @@ object EmDataCore { } /** 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 - */ + * 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) = { + : (Iterable[(Actor, IssueFlexControl)], AwaitingCompletions) = { val modelUuidToMsg = correspondences.store.flatMap { case (modelUuid, correspondence) => @@ -354,10 +379,26 @@ object EmDataCore { participant }.toSet, activeTick = activeTick, + nextSetPointTick = nextSetPointTick, + currentSetPower = currentSetPower.getOrElse(throw new RuntimeException("")) ), ) } + def handleSetPoint( + setPointFlexRequest: SetPointFlexRequest + ): AwaitingFlexOptions = { + copy( + nextSetPointTick = Some(setPointFlexRequest.nextSetPointTick), + currentSetPower = Some(setPointFlexRequest.setPower) + ) + } + + def updateSetPoint(): AwaitingFlexOptions = { + copy( + currentSetPower = Some(lastSetPower) + ) + } } /** Data structure holding relevant data and providing methods that handle @@ -387,6 +428,8 @@ object EmDataCore { private val correspondences: FlexCorrespondenceStore, private val awaitedCompletions: Set[UUID], activeTick: Long, + nextSetPointTick: Option[Long], + currentSetPower: Power ) { /** Tries to handle the completion of some connected agent for the currently @@ -452,9 +495,10 @@ object EmDataCore { flexWithNext, correspondences, Some(activeTick), + nextSetPointTick, + currentSetPower, + None ) } - } - } diff --git a/src/main/scala/edu/ie3/simona/agent/grid/GridAgent.scala b/src/main/scala/edu/ie3/simona/agent/grid/GridAgent.scala index f02d7c36b1..08ef649b5e 100644 --- a/src/main/scala/edu/ie3/simona/agent/grid/GridAgent.scala +++ b/src/main/scala/edu/ie3/simona/agent/grid/GridAgent.scala @@ -210,7 +210,8 @@ object GridAgent extends DBFSAlgorithm { constantData: GridAgentConstantData, buffer: StashBuffer[Request], ): Behavior[Request] = Behaviors.receivePartial { - case (_, WrappedActivation(activation: Activation)) => + case (ctx, WrappedActivation(activation: Activation)) => + ctx.log.info(s"Activation in tick ${activation.tick}") constantData.environmentRefs.scheduler ! Completion( constantData.activationAdapter, Some(activation.tick), diff --git a/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala b/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala index f38e1c7449..6f954fc388 100644 --- a/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala +++ b/src/main/scala/edu/ie3/simona/event/listener/ResultEventListener.scala @@ -9,7 +9,7 @@ package edu.ie3.simona.event.listener import org.apache.pekko.actor.typed.scaladsl.Behaviors import org.apache.pekko.actor.typed.{ActorRef, Behavior, PostStop} import edu.ie3.datamodel.io.processor.result.ResultEntityProcessor -import edu.ie3.datamodel.models.result.{NodeResult, ResultEntity} +import edu.ie3.datamodel.models.result.{ModelResultEntity, NodeResult, ResultEntity} import edu.ie3.simona.agent.grid.GridResultsSupport.PartialTransformer3wResult import edu.ie3.simona.event.ResultEvent.{FlexOptionsResultEvent, ParticipantResultEvent, PowerFlowResultEvent, ThermalResultEvent} import edu.ie3.simona.exceptions.{FileHierarchyException, ProcessResultEventException} @@ -237,15 +237,20 @@ object ResultEventListener extends Transformer3wResultSupport { } private def handOverToExternalService( - tick: Long, - resultEntity: ResultEntity, - extResultDataService: Option[ActorRef[ExtResultDataProvider.Request]], - nextTick: Option[Long] = None + tick: Long, + resultEntity: ResultEntity, + extResultDataService: Option[ActorRef[ExtResultDataProvider.Request]], + nextTick: Option[Long] = None ): Unit = Try { val extResultDataServiceRef = extResultDataService.getOrElse( throw new Exception("No external data service registered!") ) - extResultDataServiceRef ! ResultResponseMessage(resultEntity, tick, nextTick) + resultEntity match { + case modelResultEntity: ModelResultEntity => + extResultDataServiceRef ! ResultResponseMessage(modelResultEntity, tick, nextTick) + case _ => + throw new Exception("Wrong data type!") + } } def apply( diff --git a/src/main/scala/edu/ie3/simona/io/runtime/RuntimeEventLogSink.scala b/src/main/scala/edu/ie3/simona/io/runtime/RuntimeEventLogSink.scala index 2af82e5589..26f7714536 100644 --- a/src/main/scala/edu/ie3/simona/io/runtime/RuntimeEventLogSink.scala +++ b/src/main/scala/edu/ie3/simona/io/runtime/RuntimeEventLogSink.scala @@ -38,12 +38,12 @@ final case class RuntimeEventLogSink( case InitComplete(duration) => log.info( - s"Initialization complete. (duration: ${convertDuration(duration)} )" + s"\u001b[0;32mInitialization complete. (duration: ${convertDuration(duration)})\u001b[0;0m" ) case CheckWindowPassed(tick, duration) => log.info( - s"******* Simulation until ${calcTime(tick)} completed. ${durationAndMemoryString(duration)} ******" + s"\u001b[0;32m******* Simulation until ${calcTime(tick)} completed. ${durationAndMemoryString(duration)} ******\u001b[0;0m" ) case Ready(tick, duration) => diff --git a/src/main/scala/edu/ie3/simona/model/em/EmTools.scala b/src/main/scala/edu/ie3/simona/model/em/EmTools.scala index 8a10b9c335..312db41c27 100644 --- a/src/main/scala/edu/ie3/simona/model/em/EmTools.scala +++ b/src/main/scala/edu/ie3/simona/model/em/EmTools.scala @@ -75,4 +75,8 @@ object EmTools { s"The set power $setPower for ${flexOptions.modelUuid} must not be greater than the maximum power ${flexOptions.max}!" ) } + + def minOptionTicks(a: Option[Long], b: Option[Long]): Option[Long] = { + a.flatMap(x => b.map(y => Math.min(x, y))).orElse(a).orElse(b) + } } 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 c225f5bcb4..1a74142417 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 @@ -37,6 +37,13 @@ object FlexibilityMessage { val modelUuid: UUID } + + final case class SetPointFlexRequest( + tick: Long, + setPower: Power, + nextSetPointTick: Long + ) extends FlexRequest + /** Message that registers a flex options provider with an * [[edu.ie3.simona.agent.em.EmAgent]]. * @@ -85,7 +92,6 @@ object FlexibilityMessage { */ trait ProvideFlexOptions extends FlexResponse - final case class ProvideExtEmSetPoint(override val tick: Long, setPower: Power) extends FlexRequest /** Message that issues flexibility control to a flex options provider, i.e. a * feasible set point is delivered that the flex options provider should diff --git a/src/main/scala/edu/ie3/simona/scheduler/Scheduler.scala b/src/main/scala/edu/ie3/simona/scheduler/Scheduler.scala index 2fc075b779..3d13d63bb7 100644 --- a/src/main/scala/edu/ie3/simona/scheduler/Scheduler.scala +++ b/src/main/scala/edu/ie3/simona/scheduler/Scheduler.scala @@ -116,14 +116,12 @@ object Scheduler { // since we're active and any scheduled activation can still influence our next activation, // we can directly unlock the lock with the key unlockKey.foreach { - //println(this + " now Unlock ScheduleActivation from " + actor + ", newTick = " + newTick + ", newCore: " + newCore) _.unlock() } toActivate.foreach { _ ! Activation(newCore.activeTick) } - //println(this + " ScheduleActivation from " + actor + ", newTick = " + newTick + ", newCore: " + newCore) active(data, newCore) @@ -139,14 +137,12 @@ object Scheduler { toActivate.foreach { _ ! Activation(updatedCore.activeTick) } - //println(this + " Completion from " + actor + ", maybeNextTick = " + maybeNewTick + ", updatedCore: " + updatedCore) updatedCore } .map { newCore => newCore .maybeComplete() .map { case (maybeScheduleTick, inactiveCore) => - //println(this + " Send Completion from " + data.activationAdapter + " to " + data.parent) data.parent ! Completion( data.activationAdapter, maybeScheduleTick, diff --git a/src/main/scala/edu/ie3/simona/service/em/ExtEmDataService.scala b/src/main/scala/edu/ie3/simona/service/em/ExtEmDataService.scala index 3afa1315cf..e2947cc5e9 100644 --- a/src/main/scala/edu/ie3/simona/service/em/ExtEmDataService.scala +++ b/src/main/scala/edu/ie3/simona/service/em/ExtEmDataService.scala @@ -7,7 +7,7 @@ import edu.ie3.simona.api.data.em.ontology.{EmDataMessageFromExt, ProvideEmData} import edu.ie3.simona.api.data.ontology.DataMessageFromExt import edu.ie3.simona.exceptions.WeatherServiceException.InvalidRegistrationRequestException import edu.ie3.simona.exceptions.{InitializationException, ServiceException} -import edu.ie3.simona.ontology.messages.flex.FlexibilityMessage.{FlexRequest, IssuePowerControl, ProvideExtEmSetPoint} +import edu.ie3.simona.ontology.messages.flex.FlexibilityMessage.{FlexRequest, IssuePowerControl, SetPointFlexRequest} import edu.ie3.simona.ontology.messages.services.ServiceMessage.ExtEmDataServiceRegistrationMessage import edu.ie3.simona.ontology.messages.services.ServiceMessage.RegistrationResponseMessage.{RegistrationSuccessfulMessage, WrappedRegistrationSuccessfulMessage} import edu.ie3.simona.ontology.messages.services.{DataMessage, ServiceMessage} @@ -178,14 +178,15 @@ final case class ExtEmDataService( None } } - log.debug(s"Received ActorToEmData = $actorToEmData") if (actorToEmData.nonEmpty) { actorToEmData.foreach { - case (actor, setPoint) => actor ! ProvideExtEmSetPoint( - tick, - setPoint - ) + case (actor, setPoint) => + actor ! SetPointFlexRequest( + tick, + setPoint, + tick + 900L + ) } } (serviceStateData.copy(extEmDataMessage = None), None) diff --git a/src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala b/src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala index b64f553d77..b2c87b329b 100644 --- a/src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala +++ b/src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala @@ -1,6 +1,6 @@ package edu.ie3.simona.service.results -import edu.ie3.datamodel.models.result.ResultEntity +import edu.ie3.datamodel.models.result.{ModelResultEntity, ResultEntity} import edu.ie3.datamodel.models.result.system.PvResult import edu.ie3.simona.api.data.results.ExtResultData import edu.ie3.simona.api.data.results.ontology.{ProvideResultEntities, RequestResultEntities, ResultDataMessageFromExt} @@ -36,7 +36,7 @@ object ExtResultDataProvider { /** ResultEventListener -> ExtResultDataProvider */ final case class ResultResponseMessage( - result: ResultEntity, + result: ModelResultEntity, tick: Long, nextTick: Option[Long] ) @@ -148,7 +148,7 @@ object ExtResultDataProvider { activation.tick, Set() ) ++ serviceStateData.extResultScheduler.getOrElse(-2L, Set()) - val receiveDataMap = ReceiveDataMap[UUID, ResultEntity](expectedKeys) + val receiveDataMap = ReceiveDataMap[UUID, ModelResultEntity](expectedKeys) updatedSchedule = updatedSchedule.-(activation.tick) //ctx.log.info(s"[requestResults] tick ${msg.tick} -> updatedSchedule = $updatedSchedule") @@ -287,12 +287,12 @@ object ExtResultDataProvider { // -=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=- final case class ExtResultStateData( - extResultData: ExtResultData, - currentTick: Long, - extResultsMessage: Option[ResultDataMessageFromExt] = None, - resultStorage: Map[UUID, ResultEntity] = Map.empty, - extResultScheduler: Map[Long, Set[UUID]] = Map.empty, - receiveDataMap: Option[ReceiveDataMap[UUID, ResultEntity]] = None, + extResultData: ExtResultData, + currentTick: Long, + extResultsMessage: Option[ResultDataMessageFromExt] = None, + resultStorage: Map[UUID, ModelResultEntity] = Map.empty, + extResultScheduler: Map[Long, Set[UUID]] = Map.empty, + receiveDataMap: Option[ReceiveDataMap[UUID, ModelResultEntity]] = None, ) final case class InitExtResultData( extResultData: ExtResultData diff --git a/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala b/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala index f13c63999c..c0a8a22be8 100644 --- a/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala +++ b/src/main/scala/edu/ie3/simona/sim/SimonaSim.scala @@ -217,8 +217,6 @@ object SimonaSim { ref ! ExtSimAdapter.Stop(simulationSuccessful) } - //ctx.log.info(s"delayedStoppingActors = ${actorData.delayedStoppingActors}") - // if the simulation is successful, we're waiting for the delayed // stopping listeners to terminate and thus do not unwatch them here actorData.delayedStoppingActors.foreach( diff --git a/src/main/scala/edu/ie3/simona/sim/setup/SimonaExtSimSetup.scala b/src/main/scala/edu/ie3/simona/sim/setup/SimonaExtSimSetup.scala index 4e76201b15..3fad0bbb75 100644 --- a/src/main/scala/edu/ie3/simona/sim/setup/SimonaExtSimSetup.scala +++ b/src/main/scala/edu/ie3/simona/sim/setup/SimonaExtSimSetup.scala @@ -76,6 +76,7 @@ abstract class SimonaExtSimSetup( val runtimeEventQueue: Option[LinkedBlockingQueue[RuntimeEvent]] = None, override val args: Array[String] ) extends SimonaSetup { + override def logOutputDir: String = resultFileHierarchy.logOutputDir override def gridAgents( context: ActorContext[_], @@ -323,9 +324,9 @@ abstract class SimonaExtSimSetup( simScheduler: ActorRef[SchedulerMessage], extSim: ExtSimulation ): ExtSimSetupData = { - // ExtSimAdapter - val extScheduler = scheduler(context, parent = rootScheduler) + val extScheduler = scheduler(context, rootScheduler) + // ExtSimAdapter val extSimAdapterPhase1 = context.toClassic.simonaActorOf( ExtSimAdapter.props(extScheduler.toClassic), s"1", @@ -348,8 +349,6 @@ abstract class SimonaExtSimSetup( val dataConnections = extSim.getDataConnections - println("dataConnections = " + dataConnections) - dataConnections.asScala.foreach { case extPrimaryData: ExtPrimaryData => val extPrimaryDataService = extPrimaryDataSimulationSetup( @@ -393,9 +392,6 @@ abstract class SimonaExtSimSetup( ) new Thread(extSim, s"External simulation").start() - println("DataSerivces = " + extDataServicesMap) - println("DataProvider = " + extDataListenerMap) - ExtSimSetupData( Iterable( extSimAdapterData.getPhase1Adapter, @@ -441,6 +437,8 @@ abstract class SimonaExtSimSetup( INIT_SIM_TICK, ), ) + println("... pause extPrimaryDataSimulationSetup ...") + Thread.sleep(3000) extPrimaryDataService } @@ -478,6 +476,8 @@ abstract class SimonaExtSimSetup( INIT_SIM_TICK, ), ) + println("... pause extEmDataSimulationSetup ...") + Thread.sleep(3000) extEmDataService } @@ -531,6 +531,8 @@ abstract class SimonaExtSimSetup( 2, ScheduleLock.singleKey(context, simScheduler, INIT_SIM_TICK), ) + println("... pause extResultDataSimulationSetup ...") + Thread.sleep(3000) extResultDataProvider } From 0a39eeffc6355ff9a6cad22747b1ef0eba72d8f0 Mon Sep 17 00:00:00 2001 From: Johannes Bao Date: Thu, 25 Jul 2024 14:41:59 +0200 Subject: [PATCH 41/41] merge dev --- build.gradle | 2 +- docs/uml/protocol/CouplingTemplate.puml | 15 ++ .../fullGrid/em_input.csv | 4 + .../fullGrid/ext_entity_mapping.csv | 7 + .../fullGrid/line_input.csv | 110 ++++++++ .../fullGrid/line_type_input.csv | 2 + .../fullGrid/load_input.csv | 4 + .../fullGrid/node_input.csv | 112 ++++++++ .../fullGrid/pv_input.csv | 4 + .../fullGrid/transformer_2_w_input.csv | 2 + .../fullGrid/transformer_2_w_type_input.csv | 2 + .../simopsimtestgrid.conf | 249 ++++++++++++++++++ .../edu/ie3/simona/agent/em/EmAgent.scala | 15 +- .../io/result/ResultEntityCsvSink.scala | 1 - .../results/ExtResultDataProvider.scala | 94 +++---- 15 files changed, 570 insertions(+), 53 deletions(-) create mode 100644 docs/uml/protocol/CouplingTemplate.puml create mode 100644 input/samples/simopsimtestgrid_reduced/fullGrid/em_input.csv create mode 100644 input/samples/simopsimtestgrid_reduced/fullGrid/ext_entity_mapping.csv create mode 100644 input/samples/simopsimtestgrid_reduced/fullGrid/line_input.csv create mode 100644 input/samples/simopsimtestgrid_reduced/fullGrid/line_type_input.csv create mode 100644 input/samples/simopsimtestgrid_reduced/fullGrid/load_input.csv create mode 100644 input/samples/simopsimtestgrid_reduced/fullGrid/node_input.csv create mode 100644 input/samples/simopsimtestgrid_reduced/fullGrid/pv_input.csv create mode 100644 input/samples/simopsimtestgrid_reduced/fullGrid/transformer_2_w_input.csv create mode 100644 input/samples/simopsimtestgrid_reduced/fullGrid/transformer_2_w_type_input.csv create mode 100644 input/samples/simopsimtestgrid_reduced/simopsimtestgrid.conf diff --git a/build.gradle b/build.gradle index aba3402dc7..36954aa09a 100644 --- a/build.gradle +++ b/build.gradle @@ -25,7 +25,7 @@ ext { scalaVersion = '2.13' scalaBinaryVersion = '2.13.14' - pekkoVersion = '1.0.3' + pekkoVersion = '1.0.2' jtsVersion = '1.19.0' confluentKafkaVersion = '7.4.0' tscfgVersion = '1.0.0' diff --git a/docs/uml/protocol/CouplingTemplate.puml b/docs/uml/protocol/CouplingTemplate.puml new file mode 100644 index 0000000000..29878a966d --- /dev/null +++ b/docs/uml/protocol/CouplingTemplate.puml @@ -0,0 +1,15 @@ +@startuml + +== Phase 1 == +Mosaik -> SIMOSAIK: ! Activation(tick) +SIMOSAIK -> SIMONA: StateData(currentTick = Some(tick)) + +activate SIMOSAIK + +SIMOSAIK -> SIMOSAIK: DoSomething() + +deactivate SIMOSAIK + +SIMOSAIK -> Mosaik: ! Completion(tick) + +@enduml \ No newline at end of file diff --git a/input/samples/simopsimtestgrid_reduced/fullGrid/em_input.csv b/input/samples/simopsimtestgrid_reduced/fullGrid/em_input.csv new file mode 100644 index 0000000000..b6a794fe65 --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced/fullGrid/em_input.csv @@ -0,0 +1,4 @@ +uuid;control_strategy;parent_em;id;operates_from;operates_until;operator +c3a7e9f5-b492-4c85-af2d-1e93f6a25443;self_optimization;;EM_HH_Bus_25;;; +f9dc7ce6-658c-4101-a12f-d58bb889286b;self_optimization;;EM_HH_Bus_81;;; +957938b7-0476-4fab-a1b3-6ce8615857b3;self_optimization;;EM_HH_Bus_110;;; diff --git a/input/samples/simopsimtestgrid_reduced/fullGrid/ext_entity_mapping.csv b/input/samples/simopsimtestgrid_reduced/fullGrid/ext_entity_mapping.csv new file mode 100644 index 0000000000..58ec9dcd0f --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced/fullGrid/ext_entity_mapping.csv @@ -0,0 +1,7 @@ +uuid,id,columnScheme,dataType +f9dc7ce6-658c-4101-a12f-d58bb889286b,EM_HH_Bus_81,p,result_participant +957938b7-0476-4fab-a1b3-6ce8615857b3,EM_HH_Bus_110,p,result_participant +c3a7e9f5-b492-4c85-af2d-1e93f6a25443,EM_HH_Bus_25,p,result_participant +f9dc7ce6-658c-4101-a12f-d58bb889286b,EM_HH_Bus_81/Schedule,p,input +957938b7-0476-4fab-a1b3-6ce8615857b3,EM_HH_Bus_110/Schedule,p,input +c3a7e9f5-b492-4c85-af2d-1e93f6a25443,EM_HH_Bus_25/Schedule,p,input \ No newline at end of file diff --git a/input/samples/simopsimtestgrid_reduced/fullGrid/line_input.csv b/input/samples/simopsimtestgrid_reduced/fullGrid/line_input.csv new file mode 100644 index 0000000000..f4ff92fbba --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced/fullGrid/line_input.csv @@ -0,0 +1,110 @@ +uuid;geo_position;id;length;node_a;node_b;olm_characteristic;operates_from;operates_until;operator;parallel_devices;type +bdbbb247-57b7-473b-9411-53fcf35032db;"{""type"":""LineString"",""coordinates"":[[11.8213,53.426],[11.8213,53.4257]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 27;0.0161277;093160c4-6482-4c58-b952-217c615e3ada;9cdb3115-cc00-4d61-bc33-442e8f30fb63;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +c48eeb2c-6858-4f96-9a90-20641cc0903d;"{""type"":""LineString"",""coordinates"":[[11.8214,53.4229],[11.8213,53.4225]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 35;0.0150703;002a4495-96e4-49c9-abbe-8fccb3e9c83e;b909fb45-b6ee-427f-afd7-e8a0ec7274c6;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +97e498ae-a4f7-4485-a703-b8c00d0a5e8a;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4221],[11.8201,53.4225]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 8;0.0228449;b565ae3b-68f9-4bca-816d-9b0fc1c6b13f;a7725293-05fc-447f-bc12-38b689b0a956;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +f907f152-7bfe-4a17-a63f-adda500f6f0e;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4241],[11.8184,53.4293]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 107;0.04;4749ab2b-4d96-4100-8081-73e77c797d6b;ea4a6507-e504-4542-be5f-1019719b2257;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +6bf14400-8f6c-41ef-b6a4-9e861a3ad08f;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4241],[11.822,53.4294]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 3;0.030304;4749ab2b-4d96-4100-8081-73e77c797d6b;c5b6bfaf-1621-40a7-9c53-02cfb59c04d9;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +18b92b71-320d-4294-a47e-6715f1594755;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4243],[11.8196,53.4247]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 102;0.0251089;f4da61e4-7600-4cd1-95b6-c70b56c049fc;e7908208-77b4-4059-806e-4857262992fc;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +4128307f-3e00-4da9-b629-b696b72165a4;"{""type"":""LineString"",""coordinates"":[[11.815,53.4289],[11.815,53.4293]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 28;0.0377768;8e809cf6-7e05-437c-89a6-f6ca135a403b;35748e60-3be8-4930-8a61-209fd5df1bec;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +92876b53-6bbc-48ff-ba5f-5f5c08313e4d;"{""type"":""LineString"",""coordinates"":[[11.815,53.4277],[11.815,53.4272]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 111;0.031;3e6be3ac-2b51-4080-b815-391313612fc7;78815cf6-70db-432c-96e6-87fe8cf67eee;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +4b557d0c-9992-48b1-b45e-26cccc01db03;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4235],[11.8191,53.4231]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 90;0.00167204;f2724954-34d3-4ddf-b6b0-7a1531639990;2efac9b1-fb0d-4e08-bfac-501798826deb;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +0652d3e0-8c7c-4be2-9a41-00534aa9774b;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4212],[11.8191,53.4209]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 94;0.00182131;02e63e81-2e62-4ef6-8fdb-0b0905a437b6;fd4bebb8-40ca-4eed-92c0-cdd10b86ac20;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +782d8c4c-accb-424b-91ae-9af1b5d4b1af;"{""type"":""LineString"",""coordinates"":[[11.8201,53.4225],[11.8201,53.4229]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 79;0.0338129;a7725293-05fc-447f-bc12-38b689b0a956;83da8d60-405a-45f7-9bb9-9d35607b7927;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +b7cac7a5-f528-45bd-8ced-a16234979e13;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4239],[11.8191,53.4235]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 89;0.0111411;b5c1e826-63fd-4b0c-bec0-0c758389ef58;f2724954-34d3-4ddf-b6b0-7a1531639990;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +7ac1a715-e141-429a-8cfe-ff2badd41074;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4257],[11.8174,53.4262]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 65;0.019;8726dc29-621e-4455-a541-cd88d7da457f;814f784b-687f-4dd5-8a91-c7772c916d46;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +77ae4f1e-9ecb-4dda-a5f2-ce7ae3f9bbec;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4264],[11.8213,53.426]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 23;0.00488535;18b4157b-0e47-4c5a-adb8-ccae47372336;093160c4-6482-4c58-b952-217c615e3ada;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +6aaa54d9-d7b7-4adc-a85a-335cdf1393d6;"{""type"":""LineString"",""coordinates"":[[11.8162,53.4284],[11.8169,53.4289]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 40;0.036;6b2881ce-3965-4f5e-98a3-74eb47b0a7ca;a882e666-82d1-4ba6-87df-fc702fe06187;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +c85c107e-82c0-4002-acfa-d7000512a2ad;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4274],[11.8213,53.4271]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 20;0.0182765;2b3d7fb8-0583-4d47-97b1-3b5f232fd462;7b81b518-00e0-4ff1-b4cf-876903958d7a;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +2ef5f888-4341-4eee-b505-ae07a9a60c8d;"{""type"":""LineString"",""coordinates"":[[11.8201,53.4236],[11.8201,53.4239]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 55;0.0259846;f713593a-3fd3-410a-ac08-74202d4f5798;80962bd3-a10f-4ed2-ba6a-3e802189939c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +139ea4f9-9b7f-4825-8919-537a94ff4794;"{""type"":""LineString"",""coordinates"":[[11.8175,53.4277],[11.8174,53.4281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 59;0.00160952;7f692039-eef6-45f6-9e30-b5983f6750a5;3f6c26dd-842b-4dee-b71f-4aa32e2654ff;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +68853808-8b09-4ff2-9c92-88ed1d78c639;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4236],[11.8213,53.4232]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 33;0.0217875;3464496c-7dd4-41e9-ae0a-99ade0b51572;a966644c-37d5-4400-9fa0-725fd88586a8;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +91bec60c-2951-420c-a35e-2633119ee450;"{""type"":""LineString"",""coordinates"":[[11.8162,53.4277],[11.8162,53.4281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 30;0.00820054;2d33314e-31db-4ad4-a898-2be5d56a896d;555cd075-0fe4-4a65-b027-f45cffa960d9;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +35ba0827-e27e-496e-b735-c778f3b03019;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4241],[11.8213,53.4293]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 18;0.0139169;4749ab2b-4d96-4100-8081-73e77c797d6b;844c0b9c-058a-4228-a8c4-bf2defff6958;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +60efd159-72d6-4546-b245-8b8fc120a9f8;"{""type"":""LineString"",""coordinates"":[[11.817,53.4293],[11.8166,53.4241]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 26;0.098;85751c9b-3e5e-468a-9dc7-43775b0d4a6f;4749ab2b-4d96-4100-8081-73e77c797d6b;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +212b62aa-73c2-47af-95ec-00cad8d4a4f4;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4222],[11.8214,53.4218]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 38;0.00992653;589bf32a-b361-4536-ae96-6d56d184eedb;3b86661a-187d-4aa6-bf37-2014789afc08;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +7e61def1-8414-40b3-8775-fad9124f4369;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4282],[11.8213,53.4277]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 16;0.0164234;09285b78-9c18-4af7-9c7a-942cc868016f;3a557b4e-06b8-4f29-929f-81d95c42c897;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +f9b4c7dc-c199-4691-8c6d-1faf438cf336;"{""type"":""LineString"",""coordinates"":[[11.8221,53.429],[11.8221,53.4286]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 5;0.0143698;9644f198-e801-4545-87ee-a24e2a8039bd;6ee7ea93-ea9c-40cb-b79a-1c5f287c97a5;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +eebc456e-d11b-4b19-9100-cc1d8e91f926;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4245],[11.8166,53.4249]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 71;0.00648856;7efabb8d-ba17-4487-96d9-5744b1fedf8a;7c35a794-f569-4a9c-acb0-d03647610086;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +8f4c4009-e1ed-4985-8095-729aa5dc8cdd;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4192],[11.8198,53.4195]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 85;0.00737032;594d101c-3a05-45e3-a061-9189f3e848b7;af2b07ce-1a96-4b50-9e21-badf29eed519;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +20db47c8-d154-4ab2-ad65-caa4a37466d2;"{""type"":""LineString"",""coordinates"":[[11.8175,53.4257],[11.8174,53.4262]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 68;0.037;5f153bf6-4f25-41f1-8545-18fe6323bc49;814f784b-687f-4dd5-8a91-c7772c916d46;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +7d838003-cd6e-42fd-b75a-eaa354c3993b;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4216],[11.8191,53.4212]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 93;0.0043578;8f53645e-1f28-4eb1-807f-eb2a473f9d4c;02e63e81-2e62-4ef6-8fdb-0b0905a437b6;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +a92013e1-78b7-4447-a505-f25fffbf56f2;"{""type"":""LineString"",""coordinates"":[[11.8174,53.4262],[11.8175,53.4265]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 48;0.0118531;814f784b-687f-4dd5-8a91-c7772c916d46;94fe96b1-f36a-4edd-a107-4ff0376f1066;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +690f8f61-5cc7-448e-8971-a516f784bf11;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4293],[11.8166,53.4241]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 106;0.0131451;20ad9754-e966-4ad1-9541-f968c207f3df;4749ab2b-4d96-4100-8081-73e77c797d6b;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +6a00d582-32b1-4581-bdd1-a638ca279597;"{""type"":""LineString"",""coordinates"":[[11.815,53.4272],[11.815,53.4269]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 112;0.028;78815cf6-70db-432c-96e6-87fe8cf67eee;ab8c8f6c-e7a2-4b81-a0d7-5f13789267a2;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +1d5877c1-31cd-4ee8-b0dd-667fa4fffb8a;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4277],[11.8213,53.4274]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 17;0.00286133;3a557b4e-06b8-4f29-929f-81d95c42c897;2b3d7fb8-0583-4d47-97b1-3b5f232fd462;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +a15cd6a4-4b22-434f-be9b-ad2abe6e538c;"{""type"":""LineString"",""coordinates"":[[11.815,53.4289],[11.815,53.4285]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 108;0.033;8e809cf6-7e05-437c-89a6-f6ca135a403b;88cf719a-92df-4dfd-9a83-f84330e28fe0;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +2bd81511-4f19-4374-8ac5-96c2b19eda64;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4223],[11.8191,53.422]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 52;0.00980013;c7e48384-5699-4a38-a887-7e15a9145202;c6dac3ab-f44f-4b87-800c-0f4da64673f1;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +beee40ad-aca3-490f-87d5-a6dfc4bb76e3;"{""type"":""LineString"",""coordinates"":[[11.815,53.4293],[11.8166,53.4241]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 39;0.09;35748e60-3be8-4930-8a61-209fd5df1bec;4749ab2b-4d96-4100-8081-73e77c797d6b;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +96b06f87-7905-4d12-99d8-1ed330050659;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4203],[11.8197,53.4207]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 12;0.00473835;5596da2f-ca32-4ad3-81da-1ffa17cd3d7b;839ff0f4-93db-42ec-a928-bbc448b6cf5c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +e66890ef-ec02-407f-a5bb-ce32128e7490;"{""type"":""LineString"",""coordinates"":[[11.8162,53.4281],[11.8162,53.4284]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 41;0.00828244;555cd075-0fe4-4a65-b027-f45cffa960d9;6b2881ce-3965-4f5e-98a3-74eb47b0a7ca;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +015b0337-98bd-40d4-97d3-13a0d1da88ee;"{""type"":""LineString"",""coordinates"":[[11.8213,53.425],[11.8214,53.4246]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 84;0.0489789;58b551b6-83bd-4f1c-8d9c-8c9a7f638c0b;a9288e77-2919-4db6-89eb-9737bd07f111;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +f9e6b2f1-f8fb-4763-a1d0-36d06170fea0;"{""type"":""LineString"",""coordinates"":[[11.8213,53.429],[11.8213,53.4286]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 98;0.0144283;16091b6d-f1ea-4a07-9ad4-30d595aba68d;1403edf9-e47c-4705-8563-83bcd639482e;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +f270d6b5-7c94-4397-8ab0-7c39c888d726;"{""type"":""LineString"",""coordinates"":[[11.8175,53.4265],[11.8175,53.427]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 58;0.0036937;94fe96b1-f36a-4edd-a107-4ff0376f1066;80b8d1f8-7e83-421d-a95a-c193fc35f4f7;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +af8c65aa-cfd6-434a-8512-2d80106a2f2c;"{""type"":""LineString"",""coordinates"":[[11.8195,53.426],[11.8195,53.4264]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 31;0.00626899;9f1baf4e-12e1-41d1-8efc-81cfc78f1957;b8e95bf0-3ba8-4d53-a0bf-a3720fb785fb;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +fe28e831-6405-4dfe-987b-d688367694f1;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4247],[11.8195,53.425]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 104;0.00221503;e7908208-77b4-4059-806e-4857262992fc;29516ae3-6676-4797-99c1-1f0a32b989d8;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +eac56d69-5500-4261-9690-adb16c867485;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4231],[11.8191,53.4227]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 92;0.0149219;2efac9b1-fb0d-4e08-bfac-501798826deb;15a86f7d-fb73-49a4-af6a-25b14122378d;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +183f3976-48ca-42b2-9af9-7998436fac5b;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4289],[11.8196,53.4293]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 105;0.00721947;47246a84-ad0e-4d04-9d98-1c9cd5d363c1;20ad9754-e966-4ad1-9541-f968c207f3df;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +7feef458-03f3-4d23-b3a2-e6f1035398c4;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4243],[11.8213,53.4239]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 29;0.0204277;3fcb94e3-7781-4d83-9030-d9853822e78e;d0bfabdb-0e83-423b-a20a-ab9197c4284e;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +592cd979-16c9-43d8-a311-8ac938aa5d03;"{""type"":""LineString"",""coordinates"":[[11.8174,53.4262],[11.8182,53.4257]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 47;0.0181601;814f784b-687f-4dd5-8a91-c7772c916d46;c86d6361-4159-4787-b5f4-e41dcaa95195;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +684146e5-3b58-43b4-9589-45325ab1c0bc;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4257],[11.8213,53.4253]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 24;0.0102233;9cdb3115-cc00-4d61-bc33-442e8f30fb63;a432b8ce-0462-478b-83e7-3107cd2e909c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +d9f9ee76-b016-4588-ac6d-46681894ada7;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4188],[11.8198,53.4192]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 74;0.00421724;952c6b04-0d02-4ea2-a6cc-bb44fbbe4e52;594d101c-3a05-45e3-a061-9189f3e848b7;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +1f7e9cae-460e-47db-8cb8-da5d9f695fd8;"{""type"":""LineString"",""coordinates"":[[11.8197,53.4207],[11.8198,53.4211]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 13;0.00987733;839ff0f4-93db-42ec-a928-bbc448b6cf5c;27b84da5-478e-4a05-8fe7-a9f800db5eff;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +f3a592d0-0fd7-42ea-b928-f39473b419aa;"{""type"":""LineString"",""coordinates"":[[11.8214,53.4246],[11.8213,53.4243]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 10;0.0344866;a9288e77-2919-4db6-89eb-9737bd07f111;3fcb94e3-7781-4d83-9030-d9853822e78e;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +f505bff9-0803-415f-a765-9da981ff6024;"{""type"":""LineString"",""coordinates"":[[11.8195,53.4264],[11.8195,53.4268]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 43;0.00839819;b8e95bf0-3ba8-4d53-a0bf-a3720fb785fb;34031e92-3444-47d5-94ae-cceeb5d96bb2;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +c6501262-2e05-462b-8872-445d2aa1cab8;"{""type"":""LineString"",""coordinates"":[[11.8195,53.4272],[11.8195,53.4275]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 99;0.0157523;120eaa58-a500-4ae2-a86a-56a40b931ec1;9d136a6b-5fdc-44ed-a5ed-599a55281024;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +caac4b4b-4871-4e20-994c-6517931546cb;"{""type"":""LineString"",""coordinates"":[[11.8201,53.4229],[11.8201,53.4232]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 91;0.0220129;83da8d60-405a-45f7-9bb9-9d35607b7927;24b63115-12eb-4e77-b9ef-ca474fed960f;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +e9721561-53c0-45cc-a8ed-28861ef9dc66;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4216],[11.8198,53.4221]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 7;0.0333523;883edf38-9a18-4f61-981a-691aaf436cc7;b565ae3b-68f9-4bca-816d-9b0fc1c6b13f;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +ab143df6-d050-47b6-911a-93e462d928ac;"{""type"":""LineString"",""coordinates"":[[11.8173,53.4234],[11.8166,53.4238]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 73;0.0287121;e68a088d-cf1a-40b7-9b1a-e0933352f4e6;4129e079-6712-4275-911c-36729d698c42;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +ab7ab785-36ba-4da1-b176-a7c636cb1372;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4267],[11.8213,53.4264]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 22;0.0102382;dc54bd8a-b7d8-4e99-adb0-d6ee5084241c;18b4157b-0e47-4c5a-adb8-ccae47372336;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +1ae90a03-52a3-40ef-8e06-4ba01888aa5c;"{""type"":""LineString"",""coordinates"":[[11.816,53.4222],[11.816,53.4227]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 44;0.00743191;62d603c3-f306-40b3-a665-ba9892d226f0;3faac527-0ff3-44a7-9e4f-24a41940da90;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +20be9235-f4db-4753-9fa1-223c8519fcd3;"{""type"":""LineString"",""coordinates"":[[11.8175,53.4253],[11.8175,53.4257]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 66;0.00791441;73e7a7e8-2154-46ea-9727-a4916af3570c;5f153bf6-4f25-41f1-8545-18fe6323bc49;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +558e1545-a944-419a-9153-83caa09e1a3c;"{""type"":""LineString"",""coordinates"":[[11.8162,53.4273],[11.8162,53.4277]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 42;0.00357726;0d6ef8f6-0ba1-4fa7-8e63-e55cee12b165;2d33314e-31db-4ad4-a898-2be5d56a896d;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +38d60cf7-6099-4bc0-a616-0f0b66c70c9a;"{""type"":""LineString"",""coordinates"":[[11.8201,53.4232],[11.8201,53.4236]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 67;0.0179092;24b63115-12eb-4e77-b9ef-ca474fed960f;f713593a-3fd3-410a-ac08-74202d4f5798;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +e74dbbe9-948f-4056-8134-fd1d9d39e773;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4253],[11.8196,53.4257]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 86;0.0190003;03b2aa45-84f6-48c0-9dab-427e046a5672;79e19265-08e8-407f-ae95-2f78e344d3a4;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +24c0dc1c-2e99-46ed-a52b-5a7aae7c9afb;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4249],[11.8166,53.4253]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 72;0.00189914;7c35a794-f569-4a9c-acb0-d03647610086;50cac08e-bf24-4526-9466-53ca5edccd15;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +41a72cb2-f037-4196-a248-2b18a578db50;"{""type"":""LineString"",""coordinates"":[[11.816,53.421],[11.816,53.4214]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 78;0.00156586;98c14f60-e196-4f12-903b-8485f1eacb16;ca0b9e34-ddc8-405e-86a4-ab939626c4a2;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +e2919117-f70a-4b32-a5ab-f077fe0a84ef;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4253],[11.8166,53.4257]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 70;0.00263875;50cac08e-bf24-4526-9466-53ca5edccd15;8726dc29-621e-4455-a541-cd88d7da457f;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +7d3ee5e1-1749-4e43-afeb-4ccf5f431312;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4282],[11.8196,53.4286]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 103;0.011354;a3ed5af5-1fcb-4fce-af0f-708d3d604124;31e6e197-719d-4aaf-8ca5-ab9e7549390e;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +cbba4a6b-27d1-4724-a193-134af220a57d;"{""type"":""LineString"",""coordinates"":[[11.8182,53.4257],[11.8182,53.4254]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 49;0.018118;c86d6361-4159-4787-b5f4-e41dcaa95195;c6c20ab1-16fb-4f82-8017-273022da8bb0;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +b84fad90-182c-4ac1-b77d-394f39024ffc;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4286],[11.8196,53.4289]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 19;0.0115943;31e6e197-719d-4aaf-8ca5-ab9e7549390e;47246a84-ad0e-4d04-9d98-1c9cd5d363c1;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +6c87a571-9d8b-4df9-bd86-cde7954bee28;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4271],[11.8213,53.4267]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 21;0.00282037;7b81b518-00e0-4ff1-b4cf-876903958d7a;dc54bd8a-b7d8-4e99-adb0-d6ee5084241c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +536ef1a2-b988-4474-a5d4-1254598c4716;"{""type"":""LineString"",""coordinates"":[[11.8191,53.4227],[11.8191,53.4223]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 82;0.00544392;15a86f7d-fb73-49a4-af6a-25b14122378d;c7e48384-5699-4a38-a887-7e15a9145202;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +c0c99b3a-1c61-47b8-931d-571d9494d98f;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4241],[11.8166,53.4245]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 51;0.0182324;5682ac05-7336-4ebc-a5d1-3b69c79fb3b1;7efabb8d-ba17-4487-96d9-5744b1fedf8a;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +d3366635-447e-4d45-b839-7a7561f869a3;"{""type"":""LineString"",""coordinates"":[[11.822,53.4294],[11.8221,53.429]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 4;0.0141339;c5b6bfaf-1621-40a7-9c53-02cfb59c04d9;9644f198-e801-4545-87ee-a24e2a8039bd;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +932dd0d7-2051-4a3a-a0e3-489210d3b763;"{""type"":""LineString"",""coordinates"":[[11.8169,53.4289],[11.817,53.4293]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 57;0.0219657;a882e666-82d1-4ba6-87df-fc702fe06187;85751c9b-3e5e-468a-9dc7-43775b0d4a6f;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +e718d253-34ea-4c00-a653-80ac0af2934e;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4293],[11.8213,53.429]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 11;0.00557129;844c0b9c-058a-4228-a8c4-bf2defff6958;16091b6d-f1ea-4a07-9ad4-30d595aba68d;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +3ac954bf-5c1e-428a-9da7-37331f53d8fe;"{""type"":""LineString"",""coordinates"":[[11.8175,53.4248],[11.8175,53.4253]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 69;0.0017759;0d6c3f2b-5296-4ec1-995c-b150e72f035f;73e7a7e8-2154-46ea-9727-a4916af3570c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +7e9a0440-2a62-4daf-a4ab-e9f14f6cfd77;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4238],[11.8166,53.4234]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 53;0.0337731;4129e079-6712-4275-911c-36729d698c42;57b40047-4f9d-46bb-bf19-c0a86bbd4f5b;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +10a2050e-173b-43a7-91a0-157cee5c900d;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4257],[11.8195,53.426]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 87;0.00654022;79e19265-08e8-407f-ae95-2f78e344d3a4;9f1baf4e-12e1-41d1-8efc-81cfc78f1957;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +0dfd30a5-394e-49ff-95e9-50e73bf2604c;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4232],[11.8214,53.4229]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 34;0.0199334;a966644c-37d5-4400-9fa0-725fd88586a8;002a4495-96e4-49c9-abbe-8fccb3e9c83e;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +847934a0-e2b7-4caf-bb40-2e6af5bcb0ab;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4238],[11.8166,53.4241]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 50;0.0101795;4129e079-6712-4275-911c-36729d698c42;5682ac05-7336-4ebc-a5d1-3b69c79fb3b1;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +2daf9021-eb24-4d0c-bced-00e713a65b3a;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4225],[11.8213,53.4222]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 36;0.010188;b909fb45-b6ee-427f-afd7-e8a0ec7274c6;589bf32a-b361-4536-ae96-6d56d184eedb;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +2f1b02a3-5a58-4723-ab8f-827118a4c611;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4239],[11.8213,53.4236]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 32;0.0188563;d0bfabdb-0e83-423b-a20a-ab9197c4284e;3464496c-7dd4-41e9-ae0a-99ade0b51572;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +a5e9b6b0-2afc-49b0-8aad-705410b823c2;"{""type"":""LineString"",""coordinates"":[[11.815,53.4285],[11.815,53.4281]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 109;0.027;88cf719a-92df-4dfd-9a83-f84330e28fe0;d38d936a-9c05-4bdc-8331-418fef27f492;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +ae82a1c1-9790-44d1-8dab-52d75c7b79ce;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4286],[11.8213,53.4282]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 6;0.0050064;1403edf9-e47c-4705-8563-83bcd639482e;09285b78-9c18-4af7-9c7a-942cc868016f;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +e642eafb-2b6e-4b78-80b8-1ab15db8cfeb;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4278],[11.8196,53.4282]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 101;0.0153312;7dc43c81-9a61-45a0-9745-800a28bf4a9d;a3ed5af5-1fcb-4fce-af0f-708d3d604124;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +41a4c5e5-7b54-46a8-b6b6-d7b5861a251c;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4195],[11.8198,53.4199]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 96;0.0010841;af2b07ce-1a96-4b50-9e21-badf29eed519;215eaa45-82c3-49c7-a60f-4fa13215de05;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +44821cfc-7670-4f28-8941-70e9345cb069;"{""type"":""LineString"",""coordinates"":[[11.8201,53.4225],[11.8203,53.4221]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 15;0.027677;a7725293-05fc-447f-bc12-38b689b0a956;eae8a04c-44f2-4da3-95f6-cae48f85737c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +d665e8c6-40fb-4608-97fb-99a4cf52560e;"{""type"":""LineString"",""coordinates"":[[11.815,53.4281],[11.815,53.4277]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 110;0.041;d38d936a-9c05-4bdc-8331-418fef27f492;3e6be3ac-2b51-4080-b815-391313612fc7;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +2f37bd36-f356-4d29-81c7-dd9c2fba5e7d;"{""type"":""LineString"",""coordinates"":[[11.8166,53.4234],[11.8166,53.423]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 45;0.00223482;57b40047-4f9d-46bb-bf19-c0a86bbd4f5b;5b73ded9-3ca4-4f18-a2ab-c27b9a3dcf9d;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +bcccf0b5-e0e3-4cc7-82bb-262ebc19415e;"{""type"":""LineString"",""coordinates"":[[11.8169,53.4289],[11.8168,53.4285]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 56;0.039;a882e666-82d1-4ba6-87df-fc702fe06187;c41ebab8-16a0-4a3a-b4af-26073932d462;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +3685f389-6822-4522-a633-74265b67eaee;"{""type"":""LineString"",""coordinates"":[[11.816,53.4227],[11.816,53.423]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 76;0.00185168;3faac527-0ff3-44a7-9e4f-24a41940da90;2575f527-1f4e-45e2-bed2-4a5427f122e0;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +48347702-9e58-49f3-a7de-72024d4b296c;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4199],[11.8198,53.4203]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 2;0.00995493;215eaa45-82c3-49c7-a60f-4fa13215de05;5596da2f-ca32-4ad3-81da-1ffa17cd3d7b;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +ec2a73ea-50ba-4187-b8b0-a5046cf6b632;"{""type"":""LineString"",""coordinates"":[[11.8195,53.425],[11.8196,53.4253]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 83;0.00510961;29516ae3-6676-4797-99c1-1f0a32b989d8;03b2aa45-84f6-48c0-9dab-427e046a5672;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +d1940183-f7bb-42df-b8ff-63ac7aff0b1d;"{""type"":""LineString"",""coordinates"":[[11.8198,53.4211],[11.8198,53.4216]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 9;0.00439896;27b84da5-478e-4a05-8fe7-a9f800db5eff;883edf38-9a18-4f61-981a-691aaf436cc7;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +dd618e3f-9ef9-4e4b-b632-737d79c4d8c3;"{""type"":""LineString"",""coordinates"":[[11.8175,53.427],[11.8174,53.4273]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 62;0.00124645;80b8d1f8-7e83-421d-a95a-c193fc35f4f7;8ace5c2b-584a-4015-990f-6f1e14de4ddb;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +ed80666a-cf86-404a-b3a7-ad49be1cd40c;"{""type"":""LineString"",""coordinates"":[[11.816,53.423],[11.816,53.4234]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 77;0.00232237;2575f527-1f4e-45e2-bed2-4a5427f122e0;5d3bcf55-0520-43ff-8d63-3d0eb421e442;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +444615ed-26f2-45a7-8b5d-213c72e83a4f;"{""type"":""LineString"",""coordinates"":[[11.8201,53.4239],[11.8196,53.4243]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 95;0.0281251;80962bd3-a10f-4ed2-ba6a-3e802189939c;f4da61e4-7600-4cd1-95b6-c70b56c049fc;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +05ebbc17-61bc-4810-ae19-9ee04d7ce8d0;"{""type"":""LineString"",""coordinates"":[[11.8175,53.4285],[11.8169,53.4289]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 54;0.0173762;30b2ce2d-41dd-4d1d-866d-47abd24bfc3e;a882e666-82d1-4ba6-87df-fc702fe06187;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +caf90323-aad2-496e-a0e3-a4ba7c9a481c;"{""type"":""LineString"",""coordinates"":[[11.8174,53.4273],[11.8175,53.4277]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 64;0.00314803;8ace5c2b-584a-4015-990f-6f1e14de4ddb;7f692039-eef6-45f6-9e30-b5983f6750a5;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +60b400db-b048-4bed-be29-8abc18780d10;"{""type"":""LineString"",""coordinates"":[[11.816,53.4206],[11.816,53.421]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 14;0.045296;d6dbb0ae-13c9-438e-93b3-b6c63a0708df;98c14f60-e196-4f12-903b-8485f1eacb16;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +cf13634d-abd2-465d-8839-c95a54af7a80;"{""type"":""LineString"",""coordinates"":[[11.816,53.4214],[11.816,53.4218]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 80;0.0101988;ca0b9e34-ddc8-405e-86a4-ab939626c4a2;43040a39-8b6c-401f-9dfd-82b42aa6dec6;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +30044e09-a0f8-417d-a949-afcfa940f671;"{""type"":""LineString"",""coordinates"":[[11.8213,53.4253],[11.8213,53.425]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 25;0.0124821;a432b8ce-0462-478b-83e7-3107cd2e909c;58b551b6-83bd-4f1c-8d9c-8c9a7f638c0b;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +fe71fe2b-8dd0-4942-b0b5-d241e095b912;"{""type"":""LineString"",""coordinates"":[[11.8195,53.4268],[11.8195,53.4272]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 97;0.0032723;34031e92-3444-47d5-94ae-cceeb5d96bb2;120eaa58-a500-4ae2-a86a-56a40b931ec1;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +66e44163-7091-4f1b-991c-64108e2238f6;"{""type"":""LineString"",""coordinates"":[[11.8196,53.4243],[11.8191,53.4239]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 88;0.0185855;f4da61e4-7600-4cd1-95b6-c70b56c049fc;b5c1e826-63fd-4b0c-bec0-0c758389ef58;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +82bac681-169f-48d8-9842-fd69d3adbfe0;"{""type"":""LineString"",""coordinates"":[[11.816,53.4234],[11.8166,53.4238]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 75;0.0109137;5d3bcf55-0520-43ff-8d63-3d0eb421e442;4129e079-6712-4275-911c-36729d698c42;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +74a5dc42-e689-490f-a611-ae7c3767f01b;"{""type"":""LineString"",""coordinates"":[[11.816,53.4218],[11.816,53.4222]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 81;0.0100396;43040a39-8b6c-401f-9dfd-82b42aa6dec6;62d603c3-f306-40b3-a665-ba9892d226f0;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +0eedb3d0-cedc-4798-b42e-d8e8ef646b82;"{""type"":""LineString"",""coordinates"":[[11.8195,53.4275],[11.8196,53.4278]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 100;0.0129061;9d136a6b-5fdc-44ed-a5ed-599a55281024;7dc43c81-9a61-45a0-9745-800a28bf4a9d;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +b44446fd-2125-4c7b-850e-b0f3d6c8b110;"{""type"":""LineString"",""coordinates"":[[11.8174,53.4281],[11.8175,53.4285]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 61;0.014766;3f6c26dd-842b-4dee-b71f-4aa32e2654ff;30b2ce2d-41dd-4d1d-866d-47abd24bfc3e;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +31c914bc-dd9e-4825-9b4b-b5fc1b971e0d;"{""type"":""LineString"",""coordinates"":[[11.8166,53.423],[11.8166,53.4227]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 46;0.00683612;5b73ded9-3ca4-4f18-a2ab-c27b9a3dcf9d;ad30322c-0c99-4669-8e4b-25265087a66d;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e +e7ac7023-f4e3-460d-8d02-4c9c444d2e18;"{""type"":""LineString"",""coordinates"":[[11.8191,53.422],[11.8191,53.4216]],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Line 63;0.0149947;c6dac3ab-f44f-4b87-800c-0f4da64673f1;8f53645e-1f28-4eb1-807f-eb2a473f9d4c;"olm:{(0.0,1.0)}";;;;1;9a8e9b63-af21-4c1b-8db7-fc2924f9610e diff --git a/input/samples/simopsimtestgrid_reduced/fullGrid/line_type_input.csv b/input/samples/simopsimtestgrid_reduced/fullGrid/line_type_input.csv new file mode 100644 index 0000000000..d7786c38cd --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced/fullGrid/line_type_input.csv @@ -0,0 +1,2 @@ +uuid;b;g;i_max;id;r;v_rated;x +9a8e9b63-af21-4c1b-8db7-fc2924f9610e;273.31899999999996;0.0;357.0;NAYY 4x240SE 0.6/1kV;0.1267;0.4;0.0797965 diff --git a/input/samples/simopsimtestgrid_reduced/fullGrid/load_input.csv b/input/samples/simopsimtestgrid_reduced/fullGrid/load_input.csv new file mode 100644 index 0000000000..6ad16189a5 --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced/fullGrid/load_input.csv @@ -0,0 +1,4 @@ +uuid;cos_phi_rated;dsm;e_cons_annual;id;load_profile;node;operates_from;operates_until;operator;q_characteristics;s_rated;em +c3434742-e4f0-49e5-baa7-c1e3045c732c;0.93;false;0.0;LV5.201 Load 64;h0;dc54bd8a-b7d8-4e99-adb0-d6ee5084241c;;;;"cosPhiFixed:{(0.0,0.93)}";3.2;f9dc7ce6-658c-4101-a12f-d58bb889286b +fd2e19b6-d5e3-4776-9456-8787a2160d9d;0.93;false;0.0;LV5.201 Load 74;h0;43040a39-8b6c-401f-9dfd-82b42aa6dec6;;;;"cosPhiFixed:{(0.0,0.93)}";1.1;c3a7e9f5-b492-4c85-af2d-1e93f6a25443 +98c1a2ab-bd09-4c77-a389-d088aed894b1;0.93;false;0.0;LV5.201 Load 102;h0;3e6be3ac-2b51-4080-b815-391313612fc7;;;;"cosPhiFixed:{(0.0,0.93)}";5.3;957938b7-0476-4fab-a1b3-6ce8615857b3 diff --git a/input/samples/simopsimtestgrid_reduced/fullGrid/node_input.csv b/input/samples/simopsimtestgrid_reduced/fullGrid/node_input.csv new file mode 100644 index 0000000000..ab9f9335ca --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced/fullGrid/node_input.csv @@ -0,0 +1,112 @@ +uuid;geo_position;id;operates_from;operates_until;operator;slack;subnet;v_rated;v_target;volt_lvl +16091b6d-f1ea-4a07-9ad4-30d595aba68d;"{""type"":""Point"",""coordinates"":[11.8213,53.429],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 26;;;;false;2;0.4;1.0;Niederspannung +27b84da5-478e-4a05-8fe7-a9f800db5eff;"{""type"":""Point"",""coordinates"":[11.8198,53.4211],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 7;;;;false;2;0.4;1.0;Niederspannung +7f692039-eef6-45f6-9e30-b5983f6750a5;"{""type"":""Point"",""coordinates"":[11.8175,53.4277],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 54;;;;false;2;0.4;1.0;Niederspannung +20ad9754-e966-4ad1-9541-f968c207f3df;"{""type"":""Point"",""coordinates"":[11.8196,53.4293],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 84;;;;false;2;0.4;1.0;Niederspannung +98c14f60-e196-4f12-903b-8485f1eacb16;"{""type"":""Point"",""coordinates"":[11.816,53.421],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 29;;;;false;2;0.4;1.0;Niederspannung +09285b78-9c18-4af7-9c7a-942cc868016f;"{""type"":""Point"",""coordinates"":[11.8213,53.4282],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 58;;;;false;2;0.4;1.0;Niederspannung +0d6ef8f6-0ba1-4fa7-8e63-e55cee12b165;"{""type"":""Point"",""coordinates"":[11.8162,53.4273],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 103;;;;false;2;0.4;1.0;Niederspannung +43040a39-8b6c-401f-9dfd-82b42aa6dec6;"{""type"":""Point"",""coordinates"":[11.816,53.4218],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 25;;;;false;2;0.4;1.0;Niederspannung +dc54bd8a-b7d8-4e99-adb0-d6ee5084241c;"{""type"":""Point"",""coordinates"":[11.8213,53.4267],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 81;;;;false;2;0.4;1.0;Niederspannung +2575f527-1f4e-45e2-bed2-4a5427f122e0;"{""type"":""Point"",""coordinates"":[11.816,53.423],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 3;;;;false;2;0.4;1.0;Niederspannung +30b2ce2d-41dd-4d1d-866d-47abd24bfc3e;"{""type"":""Point"",""coordinates"":[11.8175,53.4285],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 34;;;;false;2;0.4;1.0;Niederspannung +589bf32a-b361-4536-ae96-6d56d184eedb;"{""type"":""Point"",""coordinates"":[11.8213,53.4222],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 101;;;;false;2;0.4;1.0;Niederspannung +555cd075-0fe4-4a65-b027-f45cffa960d9;"{""type"":""Point"",""coordinates"":[11.8162,53.4281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 31;;;;false;2;0.4;1.0;Niederspannung +62d603c3-f306-40b3-a665-ba9892d226f0;"{""type"":""Point"",""coordinates"":[11.816,53.4222],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 55;;;;false;2;0.4;1.0;Niederspannung +215eaa45-82c3-49c7-a60f-4fa13215de05;"{""type"":""Point"",""coordinates"":[11.8198,53.4199],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 24;;;;false;2;0.4;1.0;Niederspannung +f4da61e4-7600-4cd1-95b6-c70b56c049fc;"{""type"":""Point"",""coordinates"":[11.8196,53.4243],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 76;;;;false;2;0.4;1.0;Niederspannung +73e7a7e8-2154-46ea-9727-a4916af3570c;"{""type"":""Point"",""coordinates"":[11.8175,53.4253],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 18;;;;false;2;0.4;1.0;Niederspannung +85751c9b-3e5e-468a-9dc7-43775b0d4a6f;"{""type"":""Point"",""coordinates"":[11.817,53.4293],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 51;;;;false;2;0.4;1.0;Niederspannung +093160c4-6482-4c58-b952-217c615e3ada;"{""type"":""Point"",""coordinates"":[11.8213,53.426],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 68;;;;false;2;0.4;1.0;Niederspannung +79e19265-08e8-407f-ae95-2f78e344d3a4;"{""type"":""Point"",""coordinates"":[11.8196,53.4257],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 28;;;;false;2;0.4;1.0;Niederspannung +a966644c-37d5-4400-9fa0-725fd88586a8;"{""type"":""Point"",""coordinates"":[11.8213,53.4232],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 59;;;;false;2;0.4;1.0;Niederspannung +952c6b04-0d02-4ea2-a6cc-bb44fbbe4e52;"{""type"":""Point"",""coordinates"":[11.8198,53.4188],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 94;;;;false;2;0.4;1.0;Niederspannung +c6c20ab1-16fb-4f82-8017-273022da8bb0;"{""type"":""Point"",""coordinates"":[11.8182,53.4254],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 62;;;;false;2;0.4;1.0;Niederspannung +e68a088d-cf1a-40b7-9b1a-e0933352f4e6;"{""type"":""Point"",""coordinates"":[11.8173,53.4234],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 67;;;;false;2;0.4;1.0;Niederspannung +8f53645e-1f28-4eb1-807f-eb2a473f9d4c;"{""type"":""Point"",""coordinates"":[11.8191,53.4216],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 86;;;;false;2;0.4;1.0;Niederspannung +0d6c3f2b-5296-4ec1-995c-b150e72f035f;"{""type"":""Point"",""coordinates"":[11.8175,53.4248],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 80;;;;false;2;0.4;1.0;Niederspannung +ea4a6507-e504-4542-be5f-1019719b2257;"{""type"":""Point"",""coordinates"":[11.8184,53.4293],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 111;;;;false;2;0.4;1.0;Niederspannung +9644f198-e801-4545-87ee-a24e2a8039bd;"{""type"":""Point"",""coordinates"":[11.8221,53.429],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 39;;;;false;2;0.4;1.0;Niederspannung +a9288e77-2919-4db6-89eb-9737bd07f111;"{""type"":""Point"",""coordinates"":[11.8214,53.4246],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 63;;;;false;2;0.4;1.0;Niederspannung +d6dbb0ae-13c9-438e-93b3-b6c63a0708df;"{""type"":""Point"",""coordinates"":[11.816,53.4206],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 89;;;;false;2;0.4;1.0;Niederspannung +5596da2f-ca32-4ad3-81da-1ffa17cd3d7b;"{""type"":""Point"",""coordinates"":[11.8198,53.4203],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 102;;;;false;2;0.4;1.0;Niederspannung +ad30322c-0c99-4669-8e4b-25265087a66d;"{""type"":""Point"",""coordinates"":[11.8166,53.4227],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 65;;;;false;2;0.4;1.0;Niederspannung +120eaa58-a500-4ae2-a86a-56a40b931ec1;"{""type"":""Point"",""coordinates"":[11.8195,53.4272],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 57;;;;false;2;0.4;1.0;Niederspannung +29516ae3-6676-4797-99c1-1f0a32b989d8;"{""type"":""Point"",""coordinates"":[11.8195,53.425],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 52;;;;false;2;0.4;1.0;Niederspannung +24b63115-12eb-4e77-b9ef-ca474fed960f;"{""type"":""Point"",""coordinates"":[11.8201,53.4232],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 46;;;;false;2;0.4;1.0;Niederspannung +a7725293-05fc-447f-bc12-38b689b0a956;"{""type"":""Point"",""coordinates"":[11.8201,53.4225],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 72;;;;false;2;0.4;1.0;Niederspannung +3b86661a-187d-4aa6-bf37-2014789afc08;"{""type"":""Point"",""coordinates"":[11.8214,53.4218],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 20;;;;false;2;0.4;1.0;Niederspannung +a3ed5af5-1fcb-4fce-af0f-708d3d604124;"{""type"":""Point"",""coordinates"":[11.8196,53.4282],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 33;;;;false;2;0.4;1.0;Niederspannung +ec8f2c82-a1b2-487c-b573-250859e3b414;"{""type"":""Point"",""coordinates"":[11.8166,53.4241],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";MV2.101 Bus 5;;;;true;1;20.0;1.025;Mittelspannung +5f153bf6-4f25-41f1-8545-18fe6323bc49;"{""type"":""Point"",""coordinates"":[11.8175,53.4257],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 17;;;;false;2;0.4;1.0;Niederspannung +b565ae3b-68f9-4bca-816d-9b0fc1c6b13f;"{""type"":""Point"",""coordinates"":[11.8198,53.4221],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 70;;;;false;2;0.4;1.0;Niederspannung +8e809cf6-7e05-437c-89a6-f6ca135a403b;"{""type"":""Point"",""coordinates"":[11.815,53.4289],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 47;;;;false;2;0.4;1.0;Niederspannung +ca0b9e34-ddc8-405e-86a4-ab939626c4a2;"{""type"":""Point"",""coordinates"":[11.816,53.4214],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 99;;;;false;2;0.4;1.0;Niederspannung +3e6be3ac-2b51-4080-b815-391313612fc7;"{""type"":""Point"",""coordinates"":[11.815,53.4277],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 110;;;;false;2;0.4;1.0;Niederspannung +4749ab2b-4d96-4100-8081-73e77c797d6b;"{""type"":""Point"",""coordinates"":[11.8166,53.4241],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 73;;;;false;2;0.4;1.0;Niederspannung +3f6c26dd-842b-4dee-b71f-4aa32e2654ff;"{""type"":""Point"",""coordinates"":[11.8174,53.4281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 97;;;;false;2;0.4;1.0;Niederspannung +c6dac3ab-f44f-4b87-800c-0f4da64673f1;"{""type"":""Point"",""coordinates"":[11.8191,53.422],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 10;;;;false;2;0.4;1.0;Niederspannung +eae8a04c-44f2-4da3-95f6-cae48f85737c;"{""type"":""Point"",""coordinates"":[11.8203,53.4221],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 45;;;;false;2;0.4;1.0;Niederspannung +94fe96b1-f36a-4edd-a107-4ff0376f1066;"{""type"":""Point"",""coordinates"":[11.8175,53.4265],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 61;;;;false;2;0.4;1.0;Niederspannung +31e6e197-719d-4aaf-8ca5-ab9e7549390e;"{""type"":""Point"",""coordinates"":[11.8196,53.4286],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 32;;;;false;2;0.4;1.0;Niederspannung +002a4495-96e4-49c9-abbe-8fccb3e9c83e;"{""type"":""Point"",""coordinates"":[11.8214,53.4229],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 90;;;;false;2;0.4;1.0;Niederspannung +c7e48384-5699-4a38-a887-7e15a9145202;"{""type"":""Point"",""coordinates"":[11.8191,53.4223],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 107;;;;false;2;0.4;1.0;Niederspannung +8ace5c2b-584a-4015-990f-6f1e14de4ddb;"{""type"":""Point"",""coordinates"":[11.8174,53.4273],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 27;;;;false;2;0.4;1.0;Niederspannung +c41ebab8-16a0-4a3a-b4af-26073932d462;"{""type"":""Point"",""coordinates"":[11.8168,53.4285],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 35;;;;false;2;0.4;1.0;Niederspannung +ab8c8f6c-e7a2-4b81-a0d7-5f13789267a2;"{""type"":""Point"",""coordinates"":[11.815,53.4269],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 113;;;;false;2;0.4;1.0;Niederspannung +9f1baf4e-12e1-41d1-8efc-81cfc78f1957;"{""type"":""Point"",""coordinates"":[11.8195,53.426],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 71;;;;false;2;0.4;1.0;Niederspannung +02e63e81-2e62-4ef6-8fdb-0b0905a437b6;"{""type"":""Point"",""coordinates"":[11.8191,53.4212],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 12;;;;false;2;0.4;1.0;Niederspannung +2d33314e-31db-4ad4-a898-2be5d56a896d;"{""type"":""Point"",""coordinates"":[11.8162,53.4277],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 48;;;;false;2;0.4;1.0;Niederspannung +b8e95bf0-3ba8-4d53-a0bf-a3720fb785fb;"{""type"":""Point"",""coordinates"":[11.8195,53.4264],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 49;;;;false;2;0.4;1.0;Niederspannung +d0bfabdb-0e83-423b-a20a-ab9197c4284e;"{""type"":""Point"",""coordinates"":[11.8213,53.4239],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 82;;;;false;2;0.4;1.0;Niederspannung +6b2881ce-3965-4f5e-98a3-74eb47b0a7ca;"{""type"":""Point"",""coordinates"":[11.8162,53.4284],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 53;;;;false;2;0.4;1.0;Niederspannung +af2b07ce-1a96-4b50-9e21-badf29eed519;"{""type"":""Point"",""coordinates"":[11.8198,53.4195],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 100;;;;false;2;0.4;1.0;Niederspannung +844c0b9c-058a-4228-a8c4-bf2defff6958;"{""type"":""Point"",""coordinates"":[11.8213,53.4293],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 1;;;;false;2;0.4;1.0;Niederspannung +a432b8ce-0462-478b-83e7-3107cd2e909c;"{""type"":""Point"",""coordinates"":[11.8213,53.4253],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 56;;;;false;2;0.4;1.0;Niederspannung +4129e079-6712-4275-911c-36729d698c42;"{""type"":""Point"",""coordinates"":[11.8166,53.4238],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 77;;;;false;2;0.4;1.0;Niederspannung +883edf38-9a18-4f61-981a-691aaf436cc7;"{""type"":""Point"",""coordinates"":[11.8198,53.4216],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 4;;;;false;2;0.4;1.0;Niederspannung +f2724954-34d3-4ddf-b6b0-7a1531639990;"{""type"":""Point"",""coordinates"":[11.8191,53.4235],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 6;;;;false;2;0.4;1.0;Niederspannung +5d3bcf55-0520-43ff-8d63-3d0eb421e442;"{""type"":""Point"",""coordinates"":[11.816,53.4234],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 91;;;;false;2;0.4;1.0;Niederspannung +3a557b4e-06b8-4f29-929f-81d95c42c897;"{""type"":""Point"",""coordinates"":[11.8213,53.4277],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 42;;;;false;2;0.4;1.0;Niederspannung +7dc43c81-9a61-45a0-9745-800a28bf4a9d;"{""type"":""Point"",""coordinates"":[11.8196,53.4278],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 50;;;;false;2;0.4;1.0;Niederspannung +fd4bebb8-40ca-4eed-92c0-cdd10b86ac20;"{""type"":""Point"",""coordinates"":[11.8191,53.4209],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 60;;;;false;2;0.4;1.0;Niederspannung +3464496c-7dd4-41e9-ae0a-99ade0b51572;"{""type"":""Point"",""coordinates"":[11.8213,53.4236],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 40;;;;false;2;0.4;1.0;Niederspannung +d38d936a-9c05-4bdc-8331-418fef27f492;"{""type"":""Point"",""coordinates"":[11.815,53.4281],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 109;;;;false;2;0.4;1.0;Niederspannung +58b551b6-83bd-4f1c-8d9c-8c9a7f638c0b;"{""type"":""Point"",""coordinates"":[11.8213,53.425],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 38;;;;false;2;0.4;1.0;Niederspannung +b5c1e826-63fd-4b0c-bec0-0c758389ef58;"{""type"":""Point"",""coordinates"":[11.8191,53.4239],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 104;;;;false;2;0.4;1.0;Niederspannung +7c35a794-f569-4a9c-acb0-d03647610086;"{""type"":""Point"",""coordinates"":[11.8166,53.4249],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 98;;;;false;2;0.4;1.0;Niederspannung +78815cf6-70db-432c-96e6-87fe8cf67eee;"{""type"":""Point"",""coordinates"":[11.815,53.4272],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 112;;;;false;2;0.4;1.0;Niederspannung +c5b6bfaf-1621-40a7-9c53-02cfb59c04d9;"{""type"":""Point"",""coordinates"":[11.822,53.4294],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 14;;;;false;2;0.4;1.0;Niederspannung +2b3d7fb8-0583-4d47-97b1-3b5f232fd462;"{""type"":""Point"",""coordinates"":[11.8213,53.4274],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 87;;;;false;2;0.4;1.0;Niederspannung +3fcb94e3-7781-4d83-9030-d9853822e78e;"{""type"":""Point"",""coordinates"":[11.8213,53.4243],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 85;;;;false;2;0.4;1.0;Niederspannung +f713593a-3fd3-410a-ac08-74202d4f5798;"{""type"":""Point"",""coordinates"":[11.8201,53.4236],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 79;;;;false;2;0.4;1.0;Niederspannung +47246a84-ad0e-4d04-9d98-1c9cd5d363c1;"{""type"":""Point"",""coordinates"":[11.8196,53.4289],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 16;;;;false;2;0.4;1.0;Niederspannung +9cdb3115-cc00-4d61-bc33-442e8f30fb63;"{""type"":""Point"",""coordinates"":[11.8213,53.4257],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 83;;;;false;2;0.4;1.0;Niederspannung +839ff0f4-93db-42ec-a928-bbc448b6cf5c;"{""type"":""Point"",""coordinates"":[11.8197,53.4207],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 69;;;;false;2;0.4;1.0;Niederspannung +e7908208-77b4-4059-806e-4857262992fc;"{""type"":""Point"",""coordinates"":[11.8196,53.4247],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 105;;;;false;2;0.4;1.0;Niederspannung +83da8d60-405a-45f7-9bb9-9d35607b7927;"{""type"":""Point"",""coordinates"":[11.8201,53.4229],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 2;;;;false;2;0.4;1.0;Niederspannung +9d136a6b-5fdc-44ed-a5ed-599a55281024;"{""type"":""Point"",""coordinates"":[11.8195,53.4275],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 9;;;;false;2;0.4;1.0;Niederspannung +3faac527-0ff3-44a7-9e4f-24a41940da90;"{""type"":""Point"",""coordinates"":[11.816,53.4227],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 15;;;;false;2;0.4;1.0;Niederspannung +5682ac05-7336-4ebc-a5d1-3b69c79fb3b1;"{""type"":""Point"",""coordinates"":[11.8166,53.4241],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 44;;;;false;2;0.4;1.0;Niederspannung +5b73ded9-3ca4-4f18-a2ab-c27b9a3dcf9d;"{""type"":""Point"",""coordinates"":[11.8166,53.423],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 36;;;;false;2;0.4;1.0;Niederspannung +594d101c-3a05-45e3-a061-9189f3e848b7;"{""type"":""Point"",""coordinates"":[11.8198,53.4192],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 88;;;;false;2;0.4;1.0;Niederspannung +7b81b518-00e0-4ff1-b4cf-876903958d7a;"{""type"":""Point"",""coordinates"":[11.8213,53.4271],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 8;;;;false;2;0.4;1.0;Niederspannung +a882e666-82d1-4ba6-87df-fc702fe06187;"{""type"":""Point"",""coordinates"":[11.8169,53.4289],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 74;;;;false;2;0.4;1.0;Niederspannung +1403edf9-e47c-4705-8563-83bcd639482e;"{""type"":""Point"",""coordinates"":[11.8213,53.4286],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 75;;;;false;2;0.4;1.0;Niederspannung +03b2aa45-84f6-48c0-9dab-427e046a5672;"{""type"":""Point"",""coordinates"":[11.8196,53.4253],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 22;;;;false;2;0.4;1.0;Niederspannung +6ee7ea93-ea9c-40cb-b79a-1c5f287c97a5;"{""type"":""Point"",""coordinates"":[11.8221,53.4286],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 66;;;;false;2;0.4;1.0;Niederspannung +88cf719a-92df-4dfd-9a83-f84330e28fe0;"{""type"":""Point"",""coordinates"":[11.815,53.4285],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 108;;;;false;2;0.4;1.0;Niederspannung +80962bd3-a10f-4ed2-ba6a-3e802189939c;"{""type"":""Point"",""coordinates"":[11.8201,53.4239],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 21;;;;false;2;0.4;1.0;Niederspannung +80b8d1f8-7e83-421d-a95a-c193fc35f4f7;"{""type"":""Point"",""coordinates"":[11.8175,53.427],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 95;;;;false;2;0.4;1.0;Niederspannung +b909fb45-b6ee-427f-afd7-e8a0ec7274c6;"{""type"":""Point"",""coordinates"":[11.8213,53.4225],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 92;;;;false;2;0.4;1.0;Niederspannung +57b40047-4f9d-46bb-bf19-c0a86bbd4f5b;"{""type"":""Point"",""coordinates"":[11.8166,53.4234],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 37;;;;false;2;0.4;1.0;Niederspannung +18b4157b-0e47-4c5a-adb8-ccae47372336;"{""type"":""Point"",""coordinates"":[11.8213,53.4264],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 5;;;;false;2;0.4;1.0;Niederspannung +8726dc29-621e-4455-a541-cd88d7da457f;"{""type"":""Point"",""coordinates"":[11.8166,53.4257],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 96;;;;false;2;0.4;1.0;Niederspannung +814f784b-687f-4dd5-8a91-c7772c916d46;"{""type"":""Point"",""coordinates"":[11.8174,53.4262],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 19;;;;false;2;0.4;1.0;Niederspannung +7efabb8d-ba17-4487-96d9-5744b1fedf8a;"{""type"":""Point"",""coordinates"":[11.8166,53.4245],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 23;;;;false;2;0.4;1.0;Niederspannung +15a86f7d-fb73-49a4-af6a-25b14122378d;"{""type"":""Point"",""coordinates"":[11.8191,53.4227],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 11;;;;false;2;0.4;1.0;Niederspannung +34031e92-3444-47d5-94ae-cceeb5d96bb2;"{""type"":""Point"",""coordinates"":[11.8195,53.4268],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 106;;;;false;2;0.4;1.0;Niederspannung +35748e60-3be8-4930-8a61-209fd5df1bec;"{""type"":""Point"",""coordinates"":[11.815,53.4293],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 30;;;;false;2;0.4;1.0;Niederspannung +50cac08e-bf24-4526-9466-53ca5edccd15;"{""type"":""Point"",""coordinates"":[11.8166,53.4253],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 64;;;;false;2;0.4;1.0;Niederspannung +2efac9b1-fb0d-4e08-bfac-501798826deb;"{""type"":""Point"",""coordinates"":[11.8191,53.4231],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 93;;;;false;2;0.4;1.0;Niederspannung +c86d6361-4159-4787-b5f4-e41dcaa95195;"{""type"":""Point"",""coordinates"":[11.8182,53.4257],""crs"":{""type"":""name"",""properties"":{""name"":""EPSG:4326""}}}";LV5.201 Bus 13;;;;false;2;0.4;1.0;Niederspannung diff --git a/input/samples/simopsimtestgrid_reduced/fullGrid/pv_input.csv b/input/samples/simopsimtestgrid_reduced/fullGrid/pv_input.csv new file mode 100644 index 0000000000..b883b51783 --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced/fullGrid/pv_input.csv @@ -0,0 +1,4 @@ +uuid;albedo;azimuth;cos_phi_rated;elevation_angle;eta_conv;id;k_g;k_t;market_reaction;node;operates_from;operates_until;operator;q_characteristics;s_rated;em +a1eb7fc1-3bee-4b65-a387-ef3046644bf0;0.20000000298023224;4.093344211578369;0.8999999761581421;37.69556427001953;97.0;LV5.201 PV 10;0.8999999761581421;1.0;false;dc54bd8a-b7d8-4e99-adb0-d6ee5084241c;;;;"cosPhiFixed:{(0.0,0.9)}";4.2;f9dc7ce6-658c-4101-a12f-d58bb889286b +de8cfef5-7620-4b9e-9a10-1faebb5a80c0;0.20000000298023224;4.093344211578369;0.8999999761581421;37.69556427001953;97.0;LV5.201 PV 11;0.8999999761581421;1.0;false;3e6be3ac-2b51-4080-b815-391313612fc7;;;;"cosPhiFixed:{(0.0,0.9)}";10.9;957938b7-0476-4fab-a1b3-6ce8615857b3 +2560c371-f420-4c2a-b4e6-e04c11b64c03;0.20000000298023224;0.7802008390426636;0.8999999761581421;40.086585998535156;98.0;LV5.201 PV 15;0.8999999761581421;1.0;false;43040a39-8b6c-401f-9dfd-82b42aa6dec6;;;;"cosPhiFixed:{(0.0,0.9)}";2.9;c3a7e9f5-b492-4c85-af2d-1e93f6a25443 diff --git a/input/samples/simopsimtestgrid_reduced/fullGrid/transformer_2_w_input.csv b/input/samples/simopsimtestgrid_reduced/fullGrid/transformer_2_w_input.csv new file mode 100644 index 0000000000..d1f9f5d13c --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced/fullGrid/transformer_2_w_input.csv @@ -0,0 +1,2 @@ +uuid;auto_tap;id;node_a;node_b;operates_from;operates_until;operator;parallel_devices;tap_pos;type +adaba416-9b52-45df-9d91-f67d0dd28ecb;false;MV2.101-LV5.201-Trafo 1;ec8f2c82-a1b2-487c-b573-250859e3b414;4749ab2b-4d96-4100-8081-73e77c797d6b;;;;1;0;417407d2-1e74-4f37-9b64-f701f53f8842 diff --git a/input/samples/simopsimtestgrid_reduced/fullGrid/transformer_2_w_type_input.csv b/input/samples/simopsimtestgrid_reduced/fullGrid/transformer_2_w_type_input.csv new file mode 100644 index 0000000000..b13848b159 --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced/fullGrid/transformer_2_w_type_input.csv @@ -0,0 +1,2 @@ +uuid;b_m;d_phi;d_v;g_m;id;r_sc;s_rated;tap_max;tap_min;tap_neutr;tap_side;v_rated_a;v_rated_b;x_sc +417407d2-1e74-4f37-9b64-f701f53f8842;-36.47380569074435;0.0;2.5;4124.999999999999;0.63 MVA 20/0.4 kV Dyn5 ASEA;6.953892668178382;630.0;2;-2;0;false;20.0;0.4;37.45518044666632 diff --git a/input/samples/simopsimtestgrid_reduced/simopsimtestgrid.conf b/input/samples/simopsimtestgrid_reduced/simopsimtestgrid.conf new file mode 100644 index 0000000000..c75ef413e2 --- /dev/null +++ b/input/samples/simopsimtestgrid_reduced/simopsimtestgrid.conf @@ -0,0 +1,249 @@ +include "../common/pekko.conf" + +######### +# ATTENTION: Do not change this config file directly but use it as a base for your personal delta config for the +# vn_simona scenario! Delta configs can be created by including the config you want to change +# parameters from via include (e.g. include "input/samples/vn_simona/vn_simona.conf") at the +# beginning of your config file and then just override the parameters you want to change! +######### + +################################################################## +# Simulation Parameters +################################################################## +simona.simulationName = "simopsimtest" + +################################################################## +# Time Parameters +################################################################## +simona.time.startDateTime = "2024-02-27T00:00:00Z" +simona.time.endDateTime = "2024-02-27T23:45:00Z" +simona.time.schedulerReadyCheckWindow = 900 + +################################################################## +# Input Parameters +################################################################## +simona.input.primary.csvParams = { + directoryPath: "simona/input/samples/simopsimtestgrid_reduced/fullGrid" + csvSep: ";" + isHierarchic: false +} +simona.input.grid.datasource.id = "csv" +simona.input.grid.datasource.csvParams = { + directoryPath: "simona/input/samples/simopsimtestgrid_reduced/fullGrid" + csvSep: ";" + isHierarchic: false +} + +simona.input.weather.datasource = { + scheme = "icon" + sampleParams.use = true + coordinateSource.sampleParams.use = true + maxCoordinateDistance = 50000 +} + +################################################################## +# Output Parameters +################################################################## +simona.output.base.dir = "simona/output/simopsimtestgrid_reduced" +simona.output.base.addTimestampToOutputDir = true + +simona.output.sink.csv { + fileFormat = ".csv" + filePrefix = "" + fileSuffix = "" +} + +simona.output.grid = { + notifier = "grid" + nodes = false + lines = false + switches = false + transformers2w = false + transformers3w = false +} +simona.output.participant.defaultConfig = { + notifier = "default" + powerRequestReply = false + simulationResult = true +} +simona.output.participant.individualConfigs = [ + { + notifier = "pv" + powerRequestReply = false + simulationResult = true + }, + { + notifier = "wec" + powerRequestReply = false + simulationResult = false + }, + { + notifier = "evcs" + powerRequestReply = false + simulationResult = false + }, + { + notifier = "bm" + powerRequestReply = false + simulationResult = false + }, + { + notifier = "chp" + powerRequestReply = false + simulationResult = false + }, + { + notifier = "ev" + powerRequestReply = false + simulationResult = false + }, + { + notifier = "hp" + powerRequestReply = false + simulationResult = false + }, + { + notifier = "storage" + powerRequestReply = false + simulationResult = false + }, + { + notifier = "fixedFeedIn" + powerRequestReply = false + simulationResult = false + } +] +simona.output.thermal = { + defaultConfig = { + notifier = "default", + simulationResult = false + } + individualConfigs = [ + { + notifier = "house", + simulationResult = false + } + ] +} + +################################################################## +# Runtime Configuration // todo refactor as this naming is misleading and partly unneeded +################################################################## +simona.runtime.selected_subgrids = [] +simona.runtime.selected_volt_lvls = [] + +simona.runtime.participant.load = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + modelBehaviour = "profile" + reference = "power" + } + individualConfigs = [] +} + +simona.runtime.participant.fixedFeedIn = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +simona.runtime.participant.pv = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +simona.runtime.participant.wec = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +simona.runtime.participant.evcs = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +simona.runtime.participant.hp = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +simona.runtime.participant.storage = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +# # # # # +# ATTENTION: calculateMissingReactivePowerWithModel and scaling is ignored here. +# # # # # +simona.runtime.participant.em = { + defaultConfig = { + calculateMissingReactivePowerWithModel = false + uuids = ["default"] + scaling = 1.0 + } + individualConfigs = [] +} + +################################################################## +# Event Configuration +################################################################## +simona.event.listener = [] + +################################################################## +# Grid Configuration +################################################################## + +simona.gridConfig.refSystems = [ + {sNom = "100 kVA", vNom = "0.4 kV", voltLvls = [{id = "NS", vNom = "0.4 kV"}]}, + {sNom = "60 MVA", vNom = "20 kV", voltLvls = [{id = "MS", vNom = "20 kV"}]}, + {sNom = "600 MVA", vNom = "110 kV", voltLvls = [{id = "HS", vNom = "110 kV"}]}, + {sNom = "1000 MVA", vNom = "380 kV", voltLvls = [{id = "HoeS", vNom = "380 kV"}]} +] + +################################################################## +# Power Flow Configuration +################################################################## +simona.powerflow.maxSweepPowerDeviation = 1E-5 // the maximum allowed deviation in power between two sweeps, before overall convergence is assumed +simona.powerflow.newtonraphson.epsilon = [1E-12] +simona.powerflow.newtonraphson.iterations = 50 +simona.powerflow.resolution = "87300s" +simona.powerflow.stopOnFailure = true + +simona.control.transformer = [ + { + transformers = ["31a2b9bf-e785-4475-aa44-1c34646e8c79"], + measurements = ["923f2d69-3093-4198-86e4-13d2d1c220f8"], + vMin = 0.98, + vMax = 1.02 + }, { + transformers = ["1132dbf4-e8a1-44ae-8415-f42d4497aa1d"], + measurements = ["7686b818-a0ba-465c-8e4e-f7d3c4e171fc"], + vMin = 0.98, + vMax = 1.02 + } +] diff --git a/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala b/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala index 6e85277f50..848b429f62 100644 --- a/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala +++ b/src/main/scala/edu/ie3/simona/agent/em/EmAgent.scala @@ -245,7 +245,7 @@ object EmAgent { toActivate.foreach { _ ! RequestFlexOptions(msg.tick) } - awaitingFlexOptions(emData, modelShell, flexOptionsCore) + awaitingFlexOptions(emData, modelShell, newCore) } } else { // We don't expect a new set point -> we can do our normal stuff, because we are activated because at least one connected agent should provide flex options val (toActivate, newCore) = flexOptionsCore.updateSetPoint().takeNewFlexRequests() @@ -369,7 +369,7 @@ object EmAgent { */ case (ctx, Flex(setPointMsg: SetPointFlexRequest)) => // We got a set point after Activation -> Check, if setPower changed (yes) we have to calculate new set points for our connected agents (no) activate core and do the updates - ctx.log.debug(s"${flexOptionsCore.activeTick} ${ctx.self}.awaitingFlexOptions got external set point = $setPointMsg") + ctx.log.info(s"\u001b[0;36m${flexOptionsCore.activeTick} ${ctx.self}.awaitingFlexOptions got external set point = $setPointMsg\u001b[0;0m") val updatedCore = flexOptionsCore.handleSetPoint(setPointMsg) ctx.self ! Flex(IssuePowerControl(flexOptionsCore.activeTick, setPointMsg.setPower)) awaitingFlexCtrl(emData, modelShell, updatedCore) @@ -531,6 +531,11 @@ object EmAgent { ) ) + val nextActiveTick = EmTools.minOptionTicks( + inactiveCore.nextActiveTick, + nextSetPointTick + ) + emData.listener.foreach { _ ! ParticipantResultEvent( new EmResult( @@ -541,14 +546,10 @@ object EmAgent { result.q.toMegavars.asMegaVar, ), tick = lastActiveTick, - nextTick = inactiveCore.nextActiveTick + nextTick = nextActiveTick ) } - val nextActiveTick = EmTools.minOptionTicks( - inactiveCore.nextActiveTick, - nextSetPointTick - ) emData.parentData.fold( schedulerData => diff --git a/src/main/scala/edu/ie3/simona/io/result/ResultEntityCsvSink.scala b/src/main/scala/edu/ie3/simona/io/result/ResultEntityCsvSink.scala index e41eca9c0d..50842e7fe6 100644 --- a/src/main/scala/edu/ie3/simona/io/result/ResultEntityCsvSink.scala +++ b/src/main/scala/edu/ie3/simona/io/result/ResultEntityCsvSink.scala @@ -13,7 +13,6 @@ import edu.ie3.datamodel.models.result.ResultEntity import edu.ie3.simona.exceptions.ProcessResultEventException import edu.ie3.util.StringUtils import edu.ie3.util.io.FileIOUtils -import org.apache.pekko.stream.IOResult import java.io.{BufferedWriter, File, FileWriter, Writer} import java.lang diff --git a/src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala b/src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala index b2c87b329b..f60abbae0b 100644 --- a/src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala +++ b/src/main/scala/edu/ie3/simona/service/results/ExtResultDataProvider.scala @@ -131,53 +131,54 @@ object ExtResultDataProvider { ): Behavior[Request] = Behaviors .receivePartial[Request] { case (ctx, WrappedActivation(activation: Activation)) => - val thisCurrentTick = activation.tick - //ctx.log.info(s"+++++++ Received Activation for tick $thisCurrentTick ++++++") - var updatedStateData = serviceStateData + var updatedStateData = serviceStateData.handleActivation(activation) + //ctx.log.info(s"+++++++ Received Activation for tick ${updatedStateData.currentTick} +++++++") serviceStateData.extResultsMessage.getOrElse( throw ServiceException( "ExtResultDataService was triggered without ResultDataMessageFromExt available" - ) + ) // this should not be possible because the external simulation schedules this service ) match { case msg: RequestResultEntities => // ExtResultDataProvider wurde aktiviert und es wurden Nachrichten von ExtSimulation angefragt - //ctx.log.info(s"[requestResults] for tick ${msg.tick} and resultStorage ${serviceStateData.resultStorage}") - //ctx.log.info(s"[requestResults] for tick ${msg.tick} and extResultScheduler ${serviceStateData.extResultScheduler}") - var updatedSchedule = serviceStateData.extResultScheduler - val expectedKeys = serviceStateData.extResultScheduler.getOrElse( - activation.tick, - Set() - ) ++ serviceStateData.extResultScheduler.getOrElse(-2L, Set()) - val receiveDataMap = ReceiveDataMap[UUID, ModelResultEntity](expectedKeys) - updatedSchedule = updatedSchedule.-(activation.tick) - - //ctx.log.info(s"[requestResults] tick ${msg.tick} -> updatedSchedule = $updatedSchedule") - //ctx.log.info(s"[requestResults] tick ${msg.tick} -> receiveDataMap = $receiveDataMap") - - if (receiveDataMap.isComplete) { - // --- There are no expected results for this tick! Send the send right away! - //ctx.log.info(s"[requestResults] tick ${msg.tick} -> ReceiveDataMap is complete -> send it right away: " + serviceStateData.resultStorage) - - serviceStateData.extResultData.queueExtResponseMsg( - new ProvideResultEntities(serviceStateData.resultStorage.asJava) - ) - //ctx.log.info("++++++++++++++++++ sended ExtResultData +++++++++++++++++++++++") - updatedStateData = serviceStateData.copy( - extResultsMessage = None, - receiveDataMap = None, - extResultScheduler = updatedSchedule, - currentTick = thisCurrentTick - ) + //ctx.log.info(s"[${updatedStateData.currentTick}] [requestResults] resultStorage = ${updatedStateData.resultStorage}\n extResultScheduler ${updatedStateData.extResultScheduler}") + + if (msg.tick == updatedStateData.currentTick) { // check, if we are in the right tick + var updatedSchedule = serviceStateData.extResultScheduler + val expectedKeys = serviceStateData.extResultScheduler.getOrElse( + activation.tick, + Set() + ) ++ serviceStateData.extResultScheduler.getOrElse(-2L, Set()) + val receiveDataMap = ReceiveDataMap[UUID, ModelResultEntity](expectedKeys) + updatedSchedule = updatedSchedule.-(activation.tick) + + //ctx.log.info(s"[${updatedStateData.currentTick}] [requestResults] updatedSchedule = $updatedSchedule \n receiveDataMap = $receiveDataMap") + + if (receiveDataMap.isComplete) { + // --- There are no expected results for this tick! Send the send right away! + //ctx.log.info(s"[requestResults] tick ${msg.tick} -> ReceiveDataMap is complete -> send it right away: ${serviceStateData.resultStorage}") + + serviceStateData.extResultData.queueExtResponseMsg( + new ProvideResultEntities(serviceStateData.resultStorage.asJava) + ) + //ctx.log.info("++++++++++++++++++ sended ExtResultData +++++++++++++++++++++++") + updatedStateData = updatedStateData.copy( + extResultsMessage = None, + receiveDataMap = None, + extResultScheduler = updatedSchedule + ) + } else { + //ctx.log.info(s"[requestResults] receiveDataMap was built -> now sending ResultRequestMessage") + ctx.self ! ResultRequestMessage(msg.tick) + updatedStateData = updatedStateData.copy( + extResultsMessage = None, + receiveDataMap = Some(receiveDataMap), + extResultScheduler = updatedSchedule + ) + } } else { - //ctx.log.info(s"[requestResults] receiveDataMap was built -> now sending ResultRequestMessage") - ctx.self ! ResultRequestMessage(msg.tick) - updatedStateData = serviceStateData.copy( - extResultsMessage = None, - receiveDataMap = Some(receiveDataMap), - extResultScheduler = updatedSchedule, - currentTick = thisCurrentTick - ) + throw ServiceException(s"Results for the wrong tick ${msg.tick} requested! We are currently in tick ${updatedStateData.currentTick}") } + } scheduler ! Completion(activationAdapter, None) @@ -202,9 +203,7 @@ object ExtResultDataProvider { if (serviceStateData.receiveDataMap.isDefined) { // process dataResponses if (serviceStateData.receiveDataMap.getOrElse(throw new Exception("There is no activation yet! Receive Data Map does not exist!")).getExpectedKeys.contains(extResultResponseMsg.result.getInputModel)) { - //ctx.log.info(s"[hDRM] Process ResultsResponseMsg = $extResultResponseMsg") - //ctx.log.info(s"[hDRM] receiveDataMap ${serviceStateData.receiveDataMap}") - //ctx.log.info(s"[hDRM] MsgTick=${extResultResponseMsg.tick}, ServiceStateDataTick=${serviceStateData.currentTick}") + //ctx.log.info(s"[${serviceStateData.currentTick}] Process ResultsResponseMsg = ${extResultResponseMsg.result.getInputModel}\n receiveDataMap ${serviceStateData.receiveDataMap}\n MsgTick=${extResultResponseMsg.tick}, ServiceStateDataTick=${serviceStateData.currentTick}, nextTick = ${extResultResponseMsg.nextTick}") // --- Add received results to receiveDataMap @@ -246,6 +245,7 @@ object ExtResultDataProvider { // --- Check, if all expected results has been received if (updatedReceiveDataMap.nonComplete) { + //ctx.log.info(s"[${serviceStateData.currentTick}] There are still results missing...") // There are still results missing... idle(serviceStateData.copy( receiveDataMap = Some(updatedReceiveDataMap), @@ -253,11 +253,11 @@ object ExtResultDataProvider { extResultScheduler = updatedResultSchedule )) } else { + ctx.log.info(s"\u001b[0;34m[${serviceStateData.currentTick}] Got all ResultResponseMessage -> Now forward to external simulation in a bundle: $updatedResultStorage\u001b[0;0m") // all responses received, forward them to external simulation in a bundle serviceStateData.extResultData.queueExtResponseMsg( new ProvideResultEntities(updatedResultStorage.asJava) ) - //ctx.log.info("[hDRM] Got all ResultResponseMessage -> Now forward to external simulation in a bundle: " + updatedResultStorage) //ctx.log.info("++++++++++++++++++ sended ExtResultData +++++++++++++++++++++++") idle(serviceStateData.copy( receiveDataMap = None, @@ -293,7 +293,13 @@ object ExtResultDataProvider { resultStorage: Map[UUID, ModelResultEntity] = Map.empty, extResultScheduler: Map[Long, Set[UUID]] = Map.empty, receiveDataMap: Option[ReceiveDataMap[UUID, ModelResultEntity]] = None, - ) + ) { + def handleActivation(activation: Activation): ExtResultStateData = { + copy( + currentTick = activation.tick + ) + } + } final case class InitExtResultData( extResultData: ExtResultData )