From 6d1530c311e9ab6ed73edd01a91cb24f15f72915 Mon Sep 17 00:00:00 2001 From: Fate-JH Date: Mon, 1 Jul 2024 18:47:00 -0400 Subject: [PATCH] Zone Setup (#1208) * zone entity setup code has been moved from an instance method to a class function * port address bind to socket and request to join game world address are different --- .../net/psforever/actors/net/LoginActor.scala | 14 +- .../net/psforever/actors/zone/ZoneActor.scala | 37 +- .../net/psforever/objects/zones/Zone.scala | 604 +++++++++--------- .../services/InterstellarClusterService.scala | 2 +- 4 files changed, 354 insertions(+), 303 deletions(-) diff --git a/src/main/scala/net/psforever/actors/net/LoginActor.scala b/src/main/scala/net/psforever/actors/net/LoginActor.scala index e359a2937..31a37a357 100644 --- a/src/main/scala/net/psforever/actors/net/LoginActor.scala +++ b/src/main/scala/net/psforever/actors/net/LoginActor.scala @@ -27,6 +27,8 @@ import scala.concurrent.duration._ import scala.util.matching.Regex import scala.util.{Failure, Success} + + object LoginActor { sealed trait Command @@ -56,7 +58,7 @@ class LoginActor(middlewareActor: typed.ActorRef[MiddlewareActor.Command], conne var port: Int = 0 val serverName: String = Config.app.world.serverName - val publicAddress = new InetSocketAddress(InetAddress.getByName(Config.app.public), Config.app.world.port) + val gameTestServerAddress = new InetSocketAddress(InetAddress.getByName(Config.app.public), Config.app.world.port) private val bcryptRounds = 12 @@ -87,8 +89,10 @@ class LoginActor(middlewareActor: typed.ActorRef[MiddlewareActor.Command], conne case packet: PlanetSideGamePacket => handleGamePkt(packet) - case SocketPane.NextPort(_, address, portNum) => - val response = ConnectToWorldMessage(serverName, address.getHostAddress, portNum) + case SocketPane.NextPort(_, _, portNum) => + val address = gameTestServerAddress.getAddress.getHostAddress + log.info(s"Connecting to ${address.toLowerCase}: $portNum ...") + val response = ConnectToWorldMessage(serverName, address, portNum) middlewareActor ! MiddlewareActor.Send(response) middlewareActor ! MiddlewareActor.Close() @@ -109,7 +113,7 @@ class LoginActor(middlewareActor: typed.ActorRef[MiddlewareActor.Command], conne requestAccountLogin(username, password, token) case ConnectToWorldRequestMessage(name, _, _, _, _, _, _, _) => - log.info(s"Connect to world request for '$name'") + log.info(s"Request to connect to world '$name' ...") sockets ! SocketPane.GetNextPort("world", context.self) case _ => @@ -440,7 +444,7 @@ class LoginActor(middlewareActor: typed.ActorRef[MiddlewareActor.Command], conne serverName, WorldStatus.Up, Config.app.world.serverType, - Vector(WorldConnectionInfo(publicAddress)), //todo ideally, ask for info from SocketPane + Vector(WorldConnectionInfo(gameTestServerAddress)), //todo ideally, ask for info from SocketPane PlanetSideEmpire.VS ) ) diff --git a/src/main/scala/net/psforever/actors/zone/ZoneActor.scala b/src/main/scala/net/psforever/actors/zone/ZoneActor.scala index 87ae03190..63bf81700 100644 --- a/src/main/scala/net/psforever/actors/zone/ZoneActor.scala +++ b/src/main/scala/net/psforever/actors/zone/ZoneActor.scala @@ -1,7 +1,7 @@ package net.psforever.actors.zone -import akka.actor.typed.{ActorRef, Behavior, SupervisorStrategy} -import akka.actor.typed.scaladsl.{AbstractBehavior, ActorContext, Behaviors} +import akka.actor.typed.{ActorRef, Behavior, PostStop, SupervisorStrategy} +import akka.actor.typed.scaladsl.{ActorContext, Behaviors} import net.psforever.objects.ce.Deployable import net.psforever.objects.equipment.Equipment import net.psforever.objects.serverobject.structures.{StructureType, WarpGate} @@ -29,7 +29,7 @@ object ZoneActor { def apply(zone: Zone): Behavior[Command] = Behaviors .supervise[Command] { - Behaviors.setup(context => new ZoneActor(context, zone)) + Behaviors.setup(context => new ZoneActor(context, zone).onMessage()) } .onFailure[Exception](SupervisorStrategy.resume) @@ -80,8 +80,10 @@ object ZoneActor { final case class RewardOurSupporters(target: SourceEntry, history: Iterable[InGameActivity], kill: Kill, bep: Long) extends Command } -class ZoneActor(context: ActorContext[ZoneActor.Command], zone: Zone) - extends AbstractBehavior[ZoneActor.Command](context) { +class ZoneActor( + context: ActorContext[ZoneActor.Command], + zone: Zone + ) { import ZoneActor._ import ctx._ @@ -116,59 +118,78 @@ class ZoneActor(context: ActorContext[ZoneActor.Command], zone: Zone) case Failure(e) => log.error(e.getMessage) } - def onMessage(msg: Command): Behavior[Command] = { - msg match { + def onMessage(): Behavior[Command] = { + Behaviors.receiveMessagePartial[Command] { case GetZone(replyTo) => replyTo ! ZoneResponse(zone) + Behaviors.same case AddPlayer(player) => players.addOne(player) + Behaviors.same case RemovePlayer(player) => players.filterInPlace(p => p.CharId == player.CharId) + Behaviors.same case DropItem(item, position, orientation) => zone.Ground ! Zone.Ground.DropItem(item, position, orientation) + Behaviors.same case PickupItem(guid) => zone.Ground ! Zone.Ground.PickupItem(guid) + Behaviors.same case BuildDeployable(obj, _) => zone.Deployables ! Zone.Deployable.Build(obj) + Behaviors.same case DismissDeployable(obj) => zone.Deployables ! Zone.Deployable.Dismiss(obj) + Behaviors.same case SpawnVehicle(vehicle) => zone.Transport ! Zone.Vehicle.Spawn(vehicle) + Behaviors.same case DespawnVehicle(vehicle) => zone.Transport ! Zone.Vehicle.Despawn(vehicle) + Behaviors.same case AddToBlockMap(target, toPosition) => zone.blockMap.addTo(target, toPosition) + Behaviors.same case UpdateBlockMap(target, toPosition) => zone.blockMap.move(target, toPosition) + Behaviors.same case RemoveFromBlockMap(target) => zone.blockMap.removeFrom(target) + Behaviors.same case HotSpotActivity(defender, attacker, location) => zone.Activity ! Zone.HotSpot.Activity(defender, attacker, location) + Behaviors.same case RewardThisDeath(entity) => experience ! ExperienceCalculator.RewardThisDeath(entity) + Behaviors.same case RewardOurSupporters(target, history, kill, bep) => supportExperience ! SupportExperienceCalculator.RewardOurSupporters(target, history, kill, bep) + Behaviors.same case ZoneMapUpdate() => zone.Buildings .filter(_._2.BuildingType == StructureType.Facility) .values .foreach(_.Actor ! BuildingActor.MapUpdate()) + Behaviors.same + } + .receiveSignal { + case (_, PostStop) => + Behaviors.same } - this } } diff --git a/src/main/scala/net/psforever/objects/zones/Zone.scala b/src/main/scala/net/psforever/objects/zones/Zone.scala index 3f71a937f..c8ef2e808 100644 --- a/src/main/scala/net/psforever/objects/zones/Zone.scala +++ b/src/main/scala/net/psforever/objects/zones/Zone.scala @@ -59,6 +59,7 @@ import net.psforever.objects.zones.blockmap.{BlockMap, SectorPopulation} import net.psforever.services.Service import net.psforever.zones.Zones +import java.util.concurrent.atomic.AtomicInteger import scala.annotation.tailrec import scala.collection.mutable import scala.concurrent.{Future, Promise} @@ -84,15 +85,14 @@ import scala.concurrent.{Future, Promise} * `LivePlayerList` */ class Zone(val id: String, val map: ZoneMap, zoneNumber: Int) { - /** Governs general synchronized external requests. */ - var actor: typed.ActorRef[ZoneActor.Command] = _ + var actor: typed.ActorRef[ZoneActor.Command] = Default.typed.Actor /** Actor that handles SOI related functionality, for example if a player is in an SOI */ private var soi = Default.Actor /** The basic support structure for the globally unique number system used by this `Zone`. */ - private var guid: NumberPoolHub = new NumberPoolHub(new MaxNumberSource(65536)) + private var guid: NumberPoolHub = new NumberPoolHub(new MaxNumberSource(max = 65536)) /** The core of the unique number system, to which requests may be submitted. * @see `UniqueNumberSys` * @see `Zone.Init(ActorContext)` @@ -195,167 +195,16 @@ class Zone(val id: String, val map: ZoneMap, zoneNumber: Int) { def ZoneInitialized(): Future[Boolean] = zoneInitialized.future /** - * Establish the basic accessible conditions necessary for a functional `Zone`.
- *
- * Called from the `Actor` that governs this `Zone` when it is passed a constructor reference to the `Zone`. - * Specifically, the order of calling follows: `InterstellarCluster.preStart -> ZoneActor.receive(Zone.Init()) -> Zone.Init`. - * The basic method performs three main operations. - * First, the `Actor`-driven aspect of the globally unique identifier system for this `Zone` is finalized. - * Second, all supporting `Actor` agents are created, e.g., `ground`. - * Third, the `ZoneMap` server objects are loaded and constructed within that aforementioned system. - * To avoid being called more than once, there is a test whether the globally unique identifier system has been changed.
- *
- * Execution of this operation should be fail-safe. - * The chances of failure should be mitigated or skipped. - * A testing routine should be run after the fact on the results of the process. - * - * @see `ZoneActor.ZoneSetupCheck` + * Establish the basic accessible conditions necessary for a functional `Zone`. + * @see `Zone.init` * @param context a reference to an `ActorContext` necessary for `Props` */ def init(implicit context: ActorContext): Unit = { - if (unops == null) { - SetupNumberPools() - context.actorOf(Props(classOf[UniqueNumberSys], this, this.guid), s"zone-$id-uns") - ground = context.actorOf(Props(classOf[ZoneGroundActor], this, equipmentOnGround), s"zone-$id-ground") - deployables = context.actorOf(Props(classOf[ZoneDeployableActor], this, constructions, linkDynamicTurretWeapon), s"zone-$id-deployables") - projectiles = context.actorOf(Props(classOf[ZoneProjectileActor], this, projectileList), s"zone-$id-projectiles") - transport = context.actorOf(Props(classOf[ZoneVehicleActor], this, vehicles, linkDynamicTurretWeapon), s"zone-$id-vehicles") - population = context.actorOf(Props(classOf[ZonePopulationActor], this, players, corpses), s"zone-$id-players") - projector = context.actorOf( - Props(classOf[ZoneHotSpotDisplay], this, hotspots, 15 seconds, hotspotHistory, 60 seconds), - s"zone-$id-hotspots" - ) - soi = context.actorOf(Props(classOf[SphereOfInfluenceActor], this), s"zone-$id-soi") - - avatarEvents = context.actorOf(Props(classOf[AvatarService], this), s"zone-$id-avatar-events") - localEvents = context.actorOf(Props(classOf[LocalService], this), s"zone-$id-local-events") - vehicleEvents = context.actorOf(Props(classOf[VehicleService], this), s"zone-$id-vehicle-events") - - implicit val guid: NumberPoolHub = this.guid //passed into builderObject.Build implicitly - BuildLocalObjects(context, guid) - BuildSupportObjects() - MakeBuildings(context) - MakeLattice() - AssignAmenities() - CreateSpawnGroups() - PopulateBlockMap() - validate() - + if(Zone.Setup.init(zone = this)) { zoneInitialized.success(true) } } - def validate(): Unit = { - implicit val log: Logger = org.log4s.getLogger(s"zone/$id/sanity") - - //check bases - map.objectToBuilding.values - .toSet[Int] - .foreach(building_id => { - val target = Building(building_id) - if (target.isEmpty) { - log.error(s"expected a building for id #$building_id") - } else if (!target.get.HasGUID) { - log.error(s"building #$building_id was not registered") - } - }) - - //check base to object associations - map.objectToBuilding.keys.foreach(object_guid => - if (guid(object_guid).isEmpty) { - log.error(s"expected object id $object_guid to exist, but it did not") - } - ) - - //check door to lock association - map.doorToLock.foreach({ - case (doorGuid, lockGuid) => - validateObject(doorGuid, (x: PlanetSideGameObject) => x.isInstanceOf[serverobject.doors.Door], "door") - validateObject(lockGuid, (x: PlanetSideGameObject) => x.isInstanceOf[serverobject.locks.IFFLock], "IFF lock") - }) - - //check vehicle terminal to spawn pad association - map.terminalToSpawnPad.foreach({ - case (termGuid, padGuid) => - validateObject( - termGuid, - (x: PlanetSideGameObject) => x.isInstanceOf[serverobject.terminals.Terminal], - "vehicle terminal" - ) - validateObject( - padGuid, - (x: PlanetSideGameObject) => x.isInstanceOf[serverobject.pad.VehicleSpawnPad], - "vehicle spawn pad" - ) - }) - - //check implant terminal mech to implant terminal interface association - map.terminalToInterface.foreach({ - case (mechGuid, interfaceGuid) => - validateObject( - mechGuid, - (x: PlanetSideGameObject) => x.isInstanceOf[ImplantTerminalMech], - "implant terminal mech" - ) - validateObject( - interfaceGuid, - (o: PlanetSideGameObject) => o.isInstanceOf[serverobject.terminals.Terminal], - "implant terminal interface" - ) - }) - - //check manned turret to weapon association - map.turretToWeapon.foreach({ - case (turretGuid, weaponGuid) => - validateObject( - turretGuid, - (o: PlanetSideGameObject) => o.isInstanceOf[serverobject.turret.FacilityTurret], - "facility turret mount" - ) - if ( - validateObject( - weaponGuid, - (o: PlanetSideGameObject) => o.isInstanceOf[net.psforever.objects.Tool], - "facility turret weapon" - ) - ) { - if (GUID(weaponGuid).get.asInstanceOf[Tool].AmmoSlots.count(!_.Box.HasGUID) > 0) { - log.error(s"expected weapon $weaponGuid has an unregistered ammunition unit") - } - } - }) - } - - /** - * Recover an object from a collection and perform any number of validating tests upon it. - * If the object fails any tests, log an error. - * - * @param objectGuid the unique indentifier being checked against the `guid` access point - * @param test a test for the discovered object; - * expects at least `Type` checking - * @param description an explanation of how the object, if not discovered, should be identified - * @return `true` if the object was discovered and validates correctly; - * `false` if the object failed any tests - */ - def validateObject( - objectGuid: Int, - test: PlanetSideGameObject => Boolean, - description: String - )(implicit log: Logger): Boolean = { - try { - if (!test(GUID(objectGuid).get)) { - log.error(s"expected id $objectGuid to be a $description, but it was not") - false - } else { - true - } - } catch { - case e: Exception => - log.error(s"expected a $description at id $objectGuid but no object is initialized - $e") - false - } - } - def SetupNumberPools(): Unit = { /* override to tailor to suit requirements of zone */ } def findSpawns( @@ -632,129 +481,6 @@ class Zone(val id: String, val map: ZoneMap, zoneNumber: Int) { map.zipLinePaths } - private def BuildLocalObjects(implicit context: ActorContext, guid: NumberPoolHub): Unit = { - map.localObjects.foreach({ builderObject => - builderObject.Build - - val obj = guid(builderObject.Id) - obj collect { - case el: ZoneAware => el.Zone = this - } - }) - } - - private def BuildSupportObjects(): Unit = { - //guard against errors here, but don't worry about specifics; let ZoneActor.ZoneSetupCheck complain about problems - val other: ListBuffer[PlanetSideGameObject] = new ListBuffer[PlanetSideGameObject]() - //turret to weapon - map.turretToWeapon.foreach({ - case (turret_guid, weapon_guid) => - ((GUID(turret_guid) match { - case Some(obj: FacilityTurret) => - Some(obj) - case _ => ; - None - }) match { - case Some(obj) => - obj.Weapons.get(1) match { - case Some(slot) => - Some(obj, slot.Equipment) - case None => - None - } - case None => - None - }) match { - case Some((obj, Some(weapon: Tool))) => - guid.register(weapon, weapon_guid) - other ++= weapon.AmmoSlots.map(slot => slot.Box) - other ++= obj.Inventory.Items.map(item => item.obj) //internal ammunition reserves, if any - case _ => ; - } - }) - //after all fixed GUID's are defined ... - other.foreach(obj => guid.register(obj, obj.Definition.registerAs)) - } - - private def MakeBuildings(implicit context: ActorContext): PairMap[Int, Building] = { - val buildingList = map.localBuildings - val registrationKeys: Map[Int, Try[LoanedKey]] = buildingList.map { - case ((_, building_guid: Int, _), _) => - (building_guid, guid.register(building_guid)) - } - buildings = buildingList.map({ - case ((name, building_guid, map_id), constructor) if registrationKeys(building_guid).isSuccess => - val building = constructor.Build(name, building_guid, map_id, this) - registrationKeys(building_guid).get.Object = building - building_guid -> building - }) - buildings - } - - private def AssignAmenities(): Unit = { - map.objectToBuilding.foreach({ - case (object_guid, building_id) => - (buildings.get(building_id), guid(object_guid)) match { - case (Some(building), Some(amenity: Amenity)) => - building.Amenities = amenity - case (Some(_), _) | (None, _) | (_, None) => () //let ZoneActor's sanity check catch this error - } - }) - Zone.AssignOutwardSideToDoors(zone = this) - Zone.AssignSidednessToAmenities(zone = this) - //ntu management (eventually move to a generic building startup function) - buildings.values - .flatMap(_.Amenities.filter(_.Definition == GlobalDefinitions.resource_silo)) - .collect { - case silo: ResourceSilo => - silo.Actor ! Service.Startup() - } - //some painfields need to look for their closest door - buildings.values - .flatMap(_.Amenities.filter(_.Definition.isInstanceOf[PainboxDefinition])) - .collect { - case painbox: Painbox => - painbox.Actor ! Service.Startup() - } - //the orbital_buildings in sanctuary zones have to establish their shuttle routes - map.shuttleBays - .map { guid(_) } - .collect { case Some(obj: OrbitalShuttlePad) => - obj.Actor ! Service.Startup() - } - //allocate soi information - soi ! SOI.Build() - } - - private def MakeLattice(): Unit = { - lattice ++= map.latticeLink - .filterNot { - case (a, _) => a.contains("/") //ignore intercontinental lattice connections - } - .map { - case (source, target) => - val (sourceBuilding, targetBuilding) = (Building(source), Building(target)) match { - case (Some(sBuilding), Some(tBuilding)) => (sBuilding, tBuilding) - case _ => - throw new NoSuchElementException(s"Zone $id - can't create lattice link between $source and $target.") - } - sourceBuilding ~ targetBuilding - } - } - - private def CreateSpawnGroups(): Unit = { - buildings.values - .filterNot { _.Position == Vector3.Zero } - .map(building => { building -> building.Amenities.collect { case obj: SpawnPoint => obj } }) - .filter({ case (_, spawns) => spawns.nonEmpty }) - .foreach { SpawnGroups } - - buildings.values - .filterNot { _.Position == Vector3.Zero } - .collect { case building: WarpGate => building -> List(building.asInstanceOf[SpawnPoint]) } - .foreach { SpawnGroups } - } - def SpawnGroups(): Map[Building, List[SpawnPoint]] = spawnGroups def SpawnGroups(building: Building): List[SpawnPoint] = SpawnGroups(building.MapId) @@ -775,15 +501,6 @@ class Zone(val id: String, val map: ZoneMap, zoneNumber: Int) { entry } - def PopulateBlockMap(): Unit = { - vehicles.foreach { vehicle => blockMap.addTo(vehicle) } - buildings.values.foreach { building => - blockMap.addTo(building) - building.Amenities.foreach { amenity => blockMap.addTo(amenity) } - } - map.environment.foreach { env => blockMap.addTo(env) } - } - def StartPlayerManagementSystems(): Unit = { soi ! SOI.Start() } @@ -1589,6 +1306,315 @@ object Zone { } } + object Setup { + /* zone setup code */ + + /** + * Establish the basic accessible conditions necessary for a functional `Zone`.
+ *
+ * Called from the `Actor` that governs this `Zone` when it is passed a constructor reference to the `Zone`. + * Specifically, the order of calling follows: `InterstellarCluster.preStart -> ZoneActor.receive(Zone.Init()) -> Zone.Init`. + * The basic method performs three main operations. + * First, the `Actor`-driven aspect of the globally unique identifier system for this `Zone` is finalized. + * Second, all supporting `Actor` agents are created, e.g., `ground`. + * Third, the `ZoneMap` server objects are loaded and constructed within that aforementioned system. + * To avoid being called more than once, there is a test whether the globally unique identifier system has been changed.
+ *
+ * Execution of this operation should be fail-safe. + * The chances of failure should be mitigated or skipped. + * A testing routine should be run after the fact on the results of the process. + * + * @see `ZoneActor.ZoneSetupCheck` + * @param context a reference to an `ActorContext` necessary for `Props` + */ + def init(zone: Zone)(implicit context: ActorContext): Boolean = { + implicit val guid: NumberPoolHub = zone.guid //passed into builderObject.Build implicitly + if (zone.unops == null) { + val id = s"zone-${zone.id}" + zone.SetupNumberPools() + context.actorOf(Props(classOf[UniqueNumberSys], zone, zone.guid), s"$id-uns") + zone.ground = context.actorOf(Props(classOf[ZoneGroundActor], zone, zone.equipmentOnGround), s"$id-ground") + zone.deployables = context.actorOf(Props(classOf[ZoneDeployableActor], zone, zone.constructions, zone.linkDynamicTurretWeapon), s"$id-deployables") + zone.projectiles = context.actorOf(Props(classOf[ZoneProjectileActor], zone, zone.projectileList), s"$id-projectiles") + zone.transport = context.actorOf(Props(classOf[ZoneVehicleActor], zone, zone.vehicles, zone.linkDynamicTurretWeapon), s"$id-vehicles") + zone.population = context.actorOf(Props(classOf[ZonePopulationActor], zone, zone.players, zone.corpses), s"$id-players") + zone.projector = context.actorOf( + Props(classOf[ZoneHotSpotDisplay], zone, zone.hotspots, 15 seconds, zone.hotspotHistory, 60 seconds), + s"$id-hotspots" + ) + zone.soi = context.actorOf(Props(classOf[SphereOfInfluenceActor], zone), s"$id-soi") + + zone.avatarEvents = context.actorOf(Props(classOf[AvatarService], zone), s"$id-avatar-events") + zone.localEvents = context.actorOf(Props(classOf[LocalService], zone), s"$id-local-events") + zone.vehicleEvents = context.actorOf(Props(classOf[VehicleService], zone), s"$id-vehicle-events") + + BuildLocalObjects(zone)(context, guid) + BuildSupportObjects(zone) + MakeBuildings(zone)(context, guid) + MakeLattice(zone) + AssignAmenities(zone) + CreateSpawnGroups(zone) + PopulateBlockMap(zone) + } + Validate(zone) + } + + private def BuildLocalObjects(zone: Zone)(implicit context: ActorContext, guid: NumberPoolHub): Unit = { + zone.map.localObjects.foreach { builderObject => + builderObject.Build + val obj = guid(builderObject.Id) + obj collect { + case el: ZoneAware => el.Zone = zone + } + } + } + + private def BuildSupportObjects(zone: Zone)(implicit guid: NumberPoolHub): Unit = { + //guard against errors here, but don't worry about specifics; let ZoneActor.ZoneSetupCheck complain about problems + val other: ListBuffer[PlanetSideGameObject] = new ListBuffer[PlanetSideGameObject]() + //turret to weapon + zone.map.turretToWeapon.foreach { + case (turret_guid, weapon_guid) => + ((zone.GUID(turret_guid) match { + case Some(obj: FacilityTurret) => + Some(obj) + case _ => () + None + }) match { + case Some(obj) => + obj.Weapons.get(1) match { + case Some(slot) => + Some(obj, slot.Equipment) + case None => + None + } + case None => + None + }) match { + case Some((obj, Some(weapon: Tool))) => + guid.register(weapon, weapon_guid) + other ++= weapon.AmmoSlots.map(slot => slot.Box) + other ++= obj.Inventory.Items.map(item => item.obj) //internal ammunition reserves, if any + case _ => () + } + } + //after all fixed GUID's are defined ... + other.foreach(obj => guid.register(obj, obj.Definition.registerAs)) + } + + private def MakeBuildings(zone: Zone)(implicit context: ActorContext, guid: NumberPoolHub): PairMap[Int, Building] = { + val buildingList = zone.map.localBuildings + val registrationKeys: Map[Int, Try[LoanedKey]] = buildingList.map { + case ((_, building_guid: Int, _), _) => + (building_guid, guid.register(building_guid)) + } + zone.buildings = buildingList.map { + case ((name, building_guid, map_id), constructor) if registrationKeys(building_guid).isSuccess => + val building = constructor.Build(name, building_guid, map_id, zone) + registrationKeys(building_guid).get.Object = building + building_guid -> building + } + zone.buildings + } + + private def AssignAmenities(zone: Zone)(implicit guid: NumberPoolHub): Unit = { + val map = zone.map + val buildings = zone.buildings + map.objectToBuilding.foreach { + case (object_guid, building_id) => + (buildings.get(building_id), guid(object_guid)) match { + case (Some(building), Some(amenity: Amenity)) => + building.Amenities = amenity + case (Some(_), _) | (None, _) | (_, None) => () //let ZoneActor's sanity check catch this error + } + } + Zone.AssignOutwardSideToDoors(zone) + Zone.AssignSidednessToAmenities(zone) + //ntu management (eventually move to a generic building startup function) + buildings.values + .flatMap(_.Amenities.filter(_.Definition == GlobalDefinitions.resource_silo)) + .collect { + case silo: ResourceSilo => + silo.Actor ! Service.Startup() + } + //some painfields need to look for their closest door + buildings.values + .flatMap(_.Amenities.filter(_.Definition.isInstanceOf[PainboxDefinition])) + .collect { + case painbox: Painbox => + painbox.Actor ! Service.Startup() + } + //the orbital_buildings in sanctuary zones have to establish their shuttle routes + map.shuttleBays + .map { + guid(_) + } + .collect { case Some(obj: OrbitalShuttlePad) => + obj.Actor ! Service.Startup() + } + //allocate soi information + zone.soi ! SOI.Build() + } + + private def MakeLattice(zone: Zone): Unit = { + zone.lattice ++= zone.map.latticeLink + .filterNot { case (a, _) => a.contains("/") } //ignore intercontinental lattice connections + .map { + case (source, target) => + val (sourceBuilding, targetBuilding) = (zone.Building(source), zone.Building(target)) match { + case (Some(sBuilding), Some(tBuilding)) => (sBuilding, tBuilding) + case _ => + throw new NoSuchElementException(s"Zone ${zone.id} - can't create lattice link between $source and $target.") + } + sourceBuilding ~ targetBuilding + } + } + + private def CreateSpawnGroups(zone: Zone): Unit = { + val allBuildings = zone.buildings.values.filterNot { _.Position == Vector3.Zero } + allBuildings + .map { building => { building -> building.Amenities.collect { case obj: SpawnPoint => obj } } } + .filter { case (_, spawns) => spawns.nonEmpty } + .foreach { zone.SpawnGroups } + allBuildings + .collect { case building: WarpGate => building -> List(building.asInstanceOf[SpawnPoint]) } + .foreach { zone.SpawnGroups } + } + + private def PopulateBlockMap(zone: Zone): Unit = { + val blockMap = zone.blockMap + zone.vehicles.foreach { vehicle => blockMap.addTo(vehicle) } + zone.buildings.values.foreach { building => + blockMap.addTo(building) + building.Amenities.foreach { amenity => blockMap.addTo(amenity) } + } + zone.map.environment.foreach { env => blockMap.addTo(env) } + } + + def Validate(zone: Zone)(implicit guid: NumberPoolHub): Boolean = { + implicit val log: Logger = org.log4s.getLogger(s"zone/${zone.id}/sanity") + val map = zone.map + val counter: AtomicInteger = new AtomicInteger() + val validateUsing: (Int, PlanetSideGameObject => Boolean, String) => Boolean = validateObject(zone, counter) + + //check bases + map.objectToBuilding.values + .toSet[Int] + .foreach(building_id => { + val target = zone.Building(building_id) + if (target.isEmpty) { + log.error(s"expected a building for id #$building_id") + } else if (!target.get.HasGUID) { + log.error(s"building #$building_id was not registered") + } + }) + + //check base to object associations + map.objectToBuilding.keys.foreach(object_guid => + if (guid(object_guid).isEmpty) { + log.error(s"expected object id $object_guid to exist, but it did not") + } + ) + + //check door to lock association + map.doorToLock.foreach({ + case (doorGuid, lockGuid) => + validateUsing(doorGuid, (x: PlanetSideGameObject) => x.isInstanceOf[serverobject.doors.Door], "door") + validateUsing(lockGuid, (x: PlanetSideGameObject) => x.isInstanceOf[serverobject.locks.IFFLock], "IFF lock") + }) + + //check vehicle terminal to spawn pad association + map.terminalToSpawnPad.foreach({ + case (termGuid, padGuid) => + validateUsing( + termGuid, + (x: PlanetSideGameObject) => x.isInstanceOf[serverobject.terminals.Terminal], + "vehicle terminal" + ) + validateUsing( + padGuid, + (x: PlanetSideGameObject) => x.isInstanceOf[serverobject.pad.VehicleSpawnPad], + "vehicle spawn pad" + ) + }) + + //check implant terminal mech to implant terminal interface association + map.terminalToInterface.foreach({ + case (mechGuid, interfaceGuid) => + validateUsing( + mechGuid, + (x: PlanetSideGameObject) => x.isInstanceOf[ImplantTerminalMech], + "implant terminal mech" + ) + validateUsing( + interfaceGuid, + (o: PlanetSideGameObject) => o.isInstanceOf[serverobject.terminals.Terminal], + "implant terminal interface" + ) + }) + + //check manned turret to weapon association + map.turretToWeapon.foreach({ + case (turretGuid, weaponGuid) => + validateUsing( + turretGuid, + (o: PlanetSideGameObject) => o.isInstanceOf[serverobject.turret.FacilityTurret], + "facility turret mount" + ) + if ( + validateUsing( + weaponGuid, + (o: PlanetSideGameObject) => o.isInstanceOf[net.psforever.objects.Tool], + "facility turret weapon" + ) + ) { + if (zone.GUID(weaponGuid).get.asInstanceOf[Tool].AmmoSlots.count(!_.Box.HasGUID) > 0) { + log.error(s"expected weapon $weaponGuid has an unregistered ammunition unit") + } + } + }) + counter.get() == 0 + } + + /** + * Recover an object from a collection and perform any number of validating tests upon it. + * If the object fails any tests, log an error. + * + * @param objectGuid the unique indentifier being checked against the `guid` access point + * @param test a test for the discovered object; + * expects at least `Type` checking + * @param description an explanation of how the object, if not discovered, should be identified + * @return `true` if the object was discovered and validates correctly; + * `false` if the object failed any tests + */ + def validateObject( + zone: Zone, + counter: AtomicInteger + ) + ( + objectGuid: Int, + test: PlanetSideGameObject => Boolean, + description: String + )(implicit log: Logger): Boolean = { + try { + if (!test(zone.GUID(objectGuid).get)) { + log.error(s"expected id $objectGuid to be a $description, but it was not") + counter.incrementAndGet() + false + } else { + true + } + } catch { + case e: Exception => + log.error(s"expected a $description at id $objectGuid but no object is initialized - $e") + counter.incrementAndGet() + false + } + } + } + + /* explosions */ + /** * Allocates `Damageable` targets within the vicinity of server-prepared damage dealing * and informs those entities that they have affected by the aforementioned damage. diff --git a/src/main/scala/net/psforever/services/InterstellarClusterService.scala b/src/main/scala/net/psforever/services/InterstellarClusterService.scala index 89b9bdedc..30a02008f 100644 --- a/src/main/scala/net/psforever/services/InterstellarClusterService.scala +++ b/src/main/scala/net/psforever/services/InterstellarClusterService.scala @@ -118,7 +118,7 @@ class InterstellarClusterService(context: ActorContext[InterstellarClusterServic zoneLoadedList.foreach { _.onComplete({ case Success(true) => continentLinkFunc() - case _ => //log.error("") + case _ => () }) } //construct the zones, resulting in the callback