replaced DelayedDeconstructionActor and DeconstructionActor with VehicleRemover, a class that does performs both tasks; all messages to, from, and used by the previous two actors have been removed and replaced with the new messages

This commit is contained in:
FateJH 2018-05-26 01:04:38 -04:00
parent 36b9d81e6c
commit ee5d0c666c
9 changed files with 102 additions and 445 deletions

View file

@ -184,7 +184,7 @@ class WorldSessionActor extends Actor with MDCContextAware {
case Some(vehicle : Vehicle) =>
vehicle.Seat(vehicle.PassengerInSeat(player).get).get.Occupant = None
if(vehicle.Seats.values.count(_.isOccupied) == 0) {
vehicleService ! VehicleServiceMessage.DelayedVehicleDeconstruction(vehicle, continent, 600L) //start vehicle decay (10m)
vehicleService ! VehicleServiceMessage.Decon(RemoverActor.AddTask(vehicle, continent)) //start vehicle decay
}
vehicleService ! Service.Leave(Some(s"${vehicle.Actor}"))
@ -644,7 +644,7 @@ class WorldSessionActor extends Actor with MDCContextAware {
val obj_guid : PlanetSideGUID = obj.GUID
val player_guid : PlanetSideGUID = tplayer.GUID
log.info(s"MountVehicleMsg: $player_guid mounts $obj_guid @ $seat_num")
vehicleService ! VehicleServiceMessage.UnscheduleDeconstruction(obj_guid) //clear all deconstruction timers
vehicleService ! VehicleServiceMessage.Decon(RemoverActor.ClearSpecific(List(obj), continent)) //clear timer
PlayerActionsToCancel()
if(seat_num == 0) { //simplistic vehicle ownership management
obj.Owner match {
@ -698,7 +698,7 @@ class WorldSessionActor extends Actor with MDCContextAware {
vehicleService ! VehicleServiceMessage(continent.Id, VehicleAction.KickPassenger(player_guid, seat_num, true, obj.GUID))
}
if(obj.Seats.values.count(_.isOccupied) == 0) {
vehicleService ! VehicleServiceMessage.DelayedVehicleDeconstruction(obj, continent, 600L) //start vehicle decay (10m)
vehicleService ! VehicleServiceMessage.Decon(RemoverActor.AddTask(obj, continent)) //start vehicle decay
}
case Mountable.CanDismount(obj : Mountable, _) =>
@ -1143,9 +1143,6 @@ class WorldSessionActor extends Actor with MDCContextAware {
case VehicleSpawnPad.PlayerSeatedInVehicle(vehicle, pad) =>
val vehicle_guid = vehicle.GUID
if(player.VehicleSeated.nonEmpty) {
vehicleService ! VehicleServiceMessage.UnscheduleDeconstruction(vehicle_guid)
}
sendResponse(PlanetsideAttributeMessage(vehicle_guid, 22, 0L)) //mount points on?
//sendResponse(PlanetsideAttributeMessage(vehicle_guid, 0, 10))//vehicle.Definition.MaxHealth))
sendResponse(PlanetsideAttributeMessage(vehicle_guid, 68, 0L)) //???
@ -1408,7 +1405,6 @@ class WorldSessionActor extends Actor with MDCContextAware {
LivePlayerList.Add(sessionId, avatar)
traveler = new Traveler(self, continent.Id)
//PropertyOverrideMessage
sendResponse(ChatMsg(ChatMessageType.CMT_EXPANSIONS, true, "", "1 on", None)) //CC on
sendResponse(PlanetsideAttributeMessage(PlanetSideGUID(0), 112, 1))
sendResponse(ReplicationStreamMessage(5, Some(6), Vector(SquadListing()))) //clear squad list
sendResponse(FriendsResponse(FriendAction.InitializeFriendList, 0, true, true, Nil))
@ -1460,6 +1456,7 @@ class WorldSessionActor extends Actor with MDCContextAware {
val guid = tplayer.GUID
StartBundlingPackets()
sendResponse(SetCurrentAvatarMessage(guid,0,0))
sendResponse(ChatMsg(ChatMessageType.CMT_EXPANSIONS, true, "", "1 on", None)) //CC on //TODO once per respawn?
sendResponse(PlayerStateShiftMessage(ShiftState(1, tplayer.Position, tplayer.Orientation.z)))
if(spectator) {
sendResponse(ChatMsg(ChatMessageType.CMT_TOGGLESPECTATORMODE, false, "", "on", None))
@ -2284,8 +2281,8 @@ class WorldSessionActor extends Actor with MDCContextAware {
if((player.VehicleOwned.contains(object_guid) && vehicle.Owner.contains(player.GUID))
|| (player.Faction == vehicle.Faction
&& ((vehicle.Owner.isEmpty || continent.GUID(vehicle.Owner.get).isEmpty) || vehicle.Health == 0))) {
vehicleService ! VehicleServiceMessage.UnscheduleDeconstruction(object_guid)
vehicleService ! VehicleServiceMessage.RequestDeleteVehicle(vehicle, continent)
vehicleService ! VehicleServiceMessage.Decon(RemoverActor.ClearSpecific(List(vehicle), continent))
vehicleService ! VehicleServiceMessage.Decon(RemoverActor.AddTask(vehicle, continent, Some(0 seconds)))
log.info(s"RequestDestroy: vehicle $object_guid")
}
else {
@ -2801,7 +2798,7 @@ class WorldSessionActor extends Actor with MDCContextAware {
//todo: implement auto landing procedure if the pilot bails but passengers are still present instead of deconstructing the vehicle
//todo: continue flight path until aircraft crashes if no passengers present (or no passenger seats), then deconstruct.
if(bailType == BailType.Bailed && seat_num == 0 && GlobalDefinitions.isFlightVehicle(obj.asInstanceOf[Vehicle].Definition)) {
vehicleService ! VehicleServiceMessage.DelayedVehicleDeconstruction(obj.asInstanceOf[Vehicle], continent, 0L) // Immediately deconstruct vehicle
vehicleService ! VehicleServiceMessage.Decon(RemoverActor.AddTask(obj, continent, Some(0 seconds))) // Immediately deconstruct vehicle
}
case None =>

View file

@ -51,6 +51,8 @@ abstract class RemoverActor extends Actor {
private var taskResolver : ActorRef = Actor.noSender
private[this] val log = org.log4s.getLogger
def trace(msg : String) : Unit = log.trace(msg)
def debug(msg : String) : Unit = log.debug(msg)
/**
* Send the initial message that requests a task resolver for assisting in the removal process.
@ -99,6 +101,7 @@ abstract class RemoverActor extends Actor {
if(firstHeap.isEmpty) {
//we were the only entry so the event must be started from scratch
firstHeap = List(entry)
trace(s"a remover task has been added: $entry")
RetimeFirstTask()
}
else {
@ -106,17 +109,18 @@ abstract class RemoverActor extends Actor {
val oldHead = firstHeap.head
if(!firstHeap.exists(test => RemoverActor.Similarity(test, entry))) {
firstHeap = (firstHeap :+ entry).sortBy(_.duration)
trace(s"a remover task has been added: $entry")
if(oldHead != firstHeap.head) {
RetimeFirstTask()
}
}
else {
log.trace(s"$obj is already queued for removal")
trace(s"$obj is already queued for removal")
}
}
}
else {
log.trace(s"$obj either does not qualify for this Remover or is already queued")
trace(s"$obj either does not qualify for this Remover or is already queued")
}
case RemoverActor.HurrySpecific(targets, zone) =>
@ -145,7 +149,7 @@ abstract class RemoverActor extends Actor {
import scala.concurrent.ExecutionContext.Implicits.global
secondTask = context.system.scheduler.scheduleOnce(SecondStandardDuration, self, RemoverActor.TryDelete())
}
log.trace(s"item removal task has found ${in.size} items to remove")
trace(s"item removal task has found ${in.size} items to remove")
case RemoverActor.TryDelete() =>
secondTask.cancel
@ -156,7 +160,7 @@ abstract class RemoverActor extends Actor {
import scala.concurrent.ExecutionContext.Implicits.global
secondTask = context.system.scheduler.scheduleOnce(SecondStandardDuration, self, RemoverActor.TryDelete())
}
log.trace(s"item removal task has removed ${in.size} items")
trace(s"item removal task has removed ${in.size} items")
case RemoverActor.FailureToWork(entry, ex) =>
log.error(s"${entry.obj} from ${entry.zone} not properly unregistered - $ex")
@ -172,11 +176,13 @@ abstract class RemoverActor extends Actor {
*/
def HurrySpecific(targets : List[PlanetSideGameObject], zone : Zone) : Unit = {
CullTargetsFromFirstHeap(targets, zone) match {
case Nil => ;
case Nil =>
debug(s"no tasks matching the targets $targets have been hurried")
case list =>
debug(s"the following tasks have been hurried: $list")
secondTask.cancel
list.foreach { FirstJob }
secondHeap = secondHeap ++ list.map { RepackageEntry }
secondHeap = secondHeap ++ list.map { RepackageEntry }
import scala.concurrent.ExecutionContext.Implicits.global
secondTask = context.system.scheduler.scheduleOnce(SecondStandardDuration, self, RemoverActor.TryDelete())
}
@ -186,6 +192,7 @@ abstract class RemoverActor extends Actor {
* Expedite all entries from the first pool into the second.
*/
def HurryAll() : Unit = {
trace("all tasks have been hurried")
firstTask.cancel
firstHeap.foreach { FirstJob }
secondHeap = secondHeap ++ firstHeap.map { RepackageEntry }
@ -199,7 +206,12 @@ abstract class RemoverActor extends Actor {
* Remove specific entries from the first pool.
*/
def ClearSpecific(targets : List[PlanetSideGameObject], zone : Zone) : Unit = {
CullTargetsFromFirstHeap(targets, zone)
CullTargetsFromFirstHeap(targets, zone) match {
case Nil =>
debug(s"no tasks matching the targets $targets have been cleared")
case list =>
debug(s"the following tasks have been cleared: $list")
}
}
/**
@ -232,7 +244,7 @@ abstract class RemoverActor extends Actor {
private def CullTargetsFromFirstHeap(targets : List[PlanetSideGameObject], zone : Zone) : List[RemoverActor.Entry] = {
val culledEntries = if(targets.nonEmpty) {
if(targets.size == 1) {
log.debug(s"a target submitted: ${targets.head}")
debug(s"a target submitted: ${targets.head}")
//simple selection
RemoverActor.recursiveFind(firstHeap.iterator, RemoverActor.Entry(targets.head, zone, 0)) match {
case None => ;
@ -244,7 +256,7 @@ abstract class RemoverActor extends Actor {
}
}
else {
log.trace(s"multiple targets submitted: $targets")
debug(s"multiple targets submitted: $targets")
//cumbersome partition
//a - find targets from entries
val locatedTargets = for {
@ -267,7 +279,7 @@ abstract class RemoverActor extends Actor {
}
}
else {
log.trace(s"all targets within the specified zone $zone will be submitted")
debug(s"all targets within the specified zone $zone will be submitted")
//no specific targets; split on all targets in the given zone instead
val (in, out) = firstHeap.partition(entry => entry.zone == zone)
firstHeap = out.sortBy(_.duration)

View file

@ -25,9 +25,7 @@ class CorpseRemovalActor extends RemoverActor {
context.parent ! AvatarServiceMessage(entry.zone.Id, AvatarAction.ObjectDelete(Service.defaultPlayerGUID, entry.obj.GUID))
}
def ClearanceTest(entry : RemoverActor.Entry) : Boolean = {
!entry.zone.Corpses.contains(entry.obj.asInstanceOf[Player])
}
def ClearanceTest(entry : RemoverActor.Entry) : Boolean = !entry.zone.Corpses.contains(entry.obj)
def DeletionTask(entry : RemoverActor.Entry) : TaskResolver.GiveTask = {
GUIDTask.UnregisterPlayer(entry.obj.asInstanceOf[Player])(entry.zone.GUID)

View file

@ -25,7 +25,7 @@ class DroppedItemRemover extends RemoverActor {
context.parent ! AvatarServiceMessage(entry.zone.Id, AvatarAction.ObjectDelete(Service.defaultPlayerGUID, entry.obj.GUID))
}
def ClearanceTest(entry : RemoverActor.Entry) : Boolean = true
def ClearanceTest(entry : RemoverActor.Entry) : Boolean = !entry.zone.EquipmentOnGround.contains(entry.obj)
def DeletionTask(entry : RemoverActor.Entry) : TaskResolver.GiveTask = {
GUIDTask.UnregisterEquipment(entry.obj.asInstanceOf[Equipment])(entry.zone.GUID)

View file

@ -4,15 +4,12 @@ package services.vehicle
import akka.actor.{Actor, ActorRef, Props}
import net.psforever.objects.serverobject.pad.VehicleSpawnPad
import net.psforever.objects.zones.Zone
import services.vehicle.support.{DeconstructionActor, DelayedDeconstructionActor}
import services.vehicle.support.VehicleRemover
import net.psforever.types.DriveState
import services.{GenericEventBus, Service}
import services.{GenericEventBus, RemoverActor, Service}
class VehicleService extends Actor {
private val vehicleDecon : ActorRef = context.actorOf(Props[DeconstructionActor], "vehicle-decon-agent")
private val vehicleDelayedDecon : ActorRef = context.actorOf(Props[DelayedDeconstructionActor], "vehicle-delayed-decon-agent")
vehicleDecon ! DeconstructionActor.RequestTaskResolver
private val vehicleDecon : ActorRef = context.actorOf(Props[VehicleRemover], "vehicle-decon-agent")
private [this] val log = org.log4s.getLogger
override def preStart = {
@ -87,6 +84,11 @@ class VehicleService extends Actor {
VehicleEvents.publish(
VehicleServiceResponse(s"/$forChannel/Vehicle", player_guid, VehicleResponse.StowEquipment(vehicle_guid, slot, definition.ObjectId, item.GUID, definition.Packet.DetailedConstructorData(item).get))
)
case VehicleAction.UnloadVehicle(player_guid, continent, vehicle) =>
vehicleDecon ! RemoverActor.ClearSpecific(List(vehicle), continent) //precaution
VehicleEvents.publish(
VehicleServiceResponse(s"/$forChannel/Vehicle", player_guid, VehicleResponse.UnloadVehicle(vehicle.GUID))
)
case VehicleAction.UnstowEquipment(player_guid, item_guid) =>
VehicleEvents.publish(
VehicleServiceResponse(s"/$forChannel/Vehicle", player_guid, VehicleResponse.UnstowEquipment(item_guid))
@ -102,23 +104,9 @@ class VehicleService extends Actor {
case _ => ;
}
//message to DeconstructionActor
case VehicleServiceMessage.RequestDeleteVehicle(vehicle, continent) =>
vehicleDecon ! DeconstructionActor.RequestDeleteVehicle(vehicle, continent)
//message to DelayedDeconstructionActor
case VehicleServiceMessage.DelayedVehicleDeconstruction(vehicle, zone, timeAlive) =>
vehicleDelayedDecon ! DelayedDeconstructionActor.ScheduleDeconstruction(vehicle, zone, timeAlive)
//message to DelayedDeconstructionActor
case VehicleServiceMessage.UnscheduleDeconstruction(vehicle_guid) =>
vehicleDelayedDecon ! DelayedDeconstructionActor.UnscheduleDeconstruction(vehicle_guid)
//response from DeconstructionActor
case DeconstructionActor.DeleteVehicle(vehicle_guid, zone_id) =>
VehicleEvents.publish(
VehicleServiceResponse(s"/$zone_id/Vehicle", Service.defaultPlayerGUID, VehicleResponse.UnloadVehicle(vehicle_guid))
)
//message to VehicleRemover
case VehicleServiceMessage.Decon(msg) =>
vehicleDecon forward msg
//from VehicleSpawnControl
case VehicleSpawnPad.ConcealPlayer(player_guid, zone_id) =>
@ -159,13 +147,11 @@ class VehicleService extends Actor {
VehicleEvents.publish(
VehicleServiceResponse(s"/${zone.Id}/Vehicle", Service.defaultPlayerGUID, VehicleResponse.LoadVehicle(vehicle, vtype, vguid, vdata))
)
vehicleDelayedDecon ! DelayedDeconstructionActor.UnscheduleDeconstruction(vguid)
vehicleDelayedDecon ! DelayedDeconstructionActor.ScheduleDeconstruction(vehicle, zone, 600L) //10min
vehicleDecon forward RemoverActor.AddTask(vehicle, zone)
//from VehicleSpawnControl
case VehicleSpawnPad.DisposeVehicle(vehicle, zone) =>
vehicleDelayedDecon ! DelayedDeconstructionActor.UnscheduleDeconstruction(vehicle.GUID)
vehicleDecon ! DeconstructionActor.RequestDeleteVehicle(vehicle, zone)
vehicleDecon forward RemoverActor.HurrySpecific(List(vehicle), zone)
//correspondence from WorldSessionActor
case VehicleServiceMessage.AMSDeploymentChange(zone) =>

View file

@ -3,16 +3,14 @@ package services.vehicle
import net.psforever.objects.Vehicle
import net.psforever.objects.zones.Zone
import net.psforever.packet.game.PlanetSideGUID
final case class VehicleServiceMessage(forChannel : String, actionMessage : VehicleAction.Action)
object VehicleServiceMessage {
final case class DelayedVehicleDeconstruction(vehicle : Vehicle, continent : Zone, timeAlive : Long)
final case class GiveActorControl(vehicle : Vehicle, actorName : String)
final case class RevokeActorControl(vehicle : Vehicle)
final case class RequestDeleteVehicle(vehicle : Vehicle, continent : Zone)
final case class UnscheduleDeconstruction(vehicle_guid : PlanetSideGUID)
final case class Decon(msg : Any)
final case class AMSDeploymentChange(zone : Zone)
}

View file

@ -1,276 +0,0 @@
// Copyright (c) 2017 PSForever
package services.vehicle.support
import akka.actor.{Actor, ActorRef, Cancellable}
import net.psforever.objects.{DefaultCancellable, GlobalDefinitions, Vehicle}
import net.psforever.objects.guid.TaskResolver
import net.psforever.objects.vehicles.Seat
import net.psforever.objects.zones.Zone
import net.psforever.packet.game.PlanetSideGUID
import net.psforever.types.Vector3
import services.ServiceManager
import services.ServiceManager.Lookup
import services.vehicle.{VehicleAction, VehicleServiceMessage}
import scala.annotation.tailrec
import scala.concurrent.duration._
/**
* Manage a previously-functioning vehicle as it is being deconstructed.<br>
* <br>
* A reference to a vehicle should be passed to this object as soon as it is going to be cleaned-up from the game world.
* Once accepted, only a few seconds will remain before the vehicle is deleted.
* To ensure that no players are lost in the deletion, all occupants of the vehicle are kicked out.
* Furthermore, the vehicle is rendered "dead" and inaccessible right up to the point where it is removed.<br>
* <br>
* This `Actor` is intended to sit on top of the event system that handles broadcast messaging.
*/
class DeconstructionActor extends Actor {
/** The periodic `Executor` that scraps the next vehicle on the list */
private var scrappingProcess : Cancellable = DefaultCancellable.obj
/** A `List` of currently doomed vehicles */
private var vehicles : List[DeconstructionActor.VehicleEntry] = Nil
/** The periodic `Executor` that cleans up the next vehicle on the list */
private var heapEmptyProcess : Cancellable = DefaultCancellable.obj
/** A `List` of vehicles that have been removed from the game world and are awaiting deconstruction. */
private var vehicleScrapHeap : List[DeconstructionActor.VehicleEntry] = Nil
/** The manager that helps unregister the vehicle from its current GUID scope */
private var taskResolver : ActorRef = Actor.noSender
//private[this] val log = org.log4s.getLogger
override def postStop() : Unit = {
super.postStop()
scrappingProcess.cancel
heapEmptyProcess.cancel
vehicles.foreach(entry => {
RetirementTask(entry)
DestructionTask(entry)
})
vehicleScrapHeap.foreach { DestructionTask }
}
def receive : Receive = {
/*
ask for a resolver to deal with the GUID system
when the TaskResolver is finally delivered, switch over to a behavior that actually deals with submitted vehicles
*/
case DeconstructionActor.RequestTaskResolver =>
ServiceManager.serviceManager ! Lookup("taskResolver")
case ServiceManager.LookupResult("taskResolver", endpoint) =>
taskResolver = endpoint
context.become(Processing)
case _ => ;
}
def Processing : Receive = {
case DeconstructionActor.RequestDeleteVehicle(vehicle, zone, time) =>
if(!vehicles.exists(_.vehicle == vehicle) && !vehicleScrapHeap.exists(_.vehicle == vehicle)) {
vehicles = vehicles :+ DeconstructionActor.VehicleEntry(vehicle, zone, time)
vehicle.Actor ! Vehicle.PrepareForDeletion
//kick everyone out; this is a no-blocking manual form of MountableBehavior ! Mountable.TryDismount
vehicle.Definition.MountPoints.values.foreach(seat_num => {
val zone_id : String = zone.Id
val seat : Seat = vehicle.Seat(seat_num).get
seat.Occupant match {
case Some(tplayer) =>
seat.Occupant = None
tplayer.VehicleSeated = None
if(tplayer.HasGUID) {
context.parent ! VehicleServiceMessage(zone_id, VehicleAction.KickPassenger(tplayer.GUID, 4, false, vehicle.GUID))
}
case None => ;
}
})
if(vehicles.size == 1) {
//we were the only entry so the event must be started from scratch
import scala.concurrent.ExecutionContext.Implicits.global
scrappingProcess = context.system.scheduler.scheduleOnce(DeconstructionActor.timeout, self, DeconstructionActor.StartDeleteVehicle())
}
}
case DeconstructionActor.StartDeleteVehicle() =>
scrappingProcess.cancel
heapEmptyProcess.cancel
val now : Long = System.nanoTime
val (vehiclesToScrap, vehiclesRemain) = PartitionEntries(vehicles, now)
vehicles = vehiclesRemain
vehicleScrapHeap = vehicleScrapHeap ++ vehiclesToScrap //may include existing entries
vehiclesToScrap.foreach(entry => {
val vehicle = entry.vehicle
val zone = entry.zone
RetirementTask(entry)
if(vehicle.Definition == GlobalDefinitions.ams) {
import net.psforever.types.DriveState
vehicle.DeploymentState = DriveState.Mobile //internally undeployed //TODO this should be temporary?
context.parent ! VehicleServiceMessage.AMSDeploymentChange(zone)
}
taskResolver ! DeconstructionTask(vehicle, zone)
})
if(vehiclesRemain.nonEmpty) {
val short_timeout : FiniteDuration = math.max(1, DeconstructionActor.timeout_time - (now - vehiclesRemain.head.time)) nanoseconds
import scala.concurrent.ExecutionContext.Implicits.global
scrappingProcess = context.system.scheduler.scheduleOnce(short_timeout, self, DeconstructionActor.StartDeleteVehicle())
}
if(vehicleScrapHeap.nonEmpty) {
import scala.concurrent.ExecutionContext.Implicits.global
heapEmptyProcess = context.system.scheduler.scheduleOnce(500 milliseconds, self, DeconstructionActor.TryDeleteVehicle())
}
case DeconstructionActor.TryDeleteVehicle() =>
heapEmptyProcess.cancel
val (vehiclesToScrap, vehiclesRemain) = vehicleScrapHeap.partition(entry => !entry.zone.Vehicles.contains(entry.vehicle))
vehicleScrapHeap = vehiclesRemain
vehiclesToScrap.foreach { DestructionTask }
if(vehiclesRemain.nonEmpty) {
import scala.concurrent.ExecutionContext.Implicits.global
heapEmptyProcess = context.system.scheduler.scheduleOnce(500 milliseconds, self, DeconstructionActor.TryDeleteVehicle())
}
case DeconstructionActor.FailureToDeleteVehicle(localVehicle, localZone, ex) =>
org.log4s.getLogger.error(s"vehicle deconstruction: $localVehicle failed to be properly cleaned up from zone $localZone - $ex")
case _ => ;
}
def RetirementTask(entry : DeconstructionActor.VehicleEntry) : Unit = {
val vehicle = entry.vehicle
val zone = entry.zone
vehicle.Position = Vector3.Zero //somewhere it will not disturb anything
zone.Transport ! Zone.Vehicle.Despawn(vehicle)
context.parent ! DeconstructionActor.DeleteVehicle(vehicle.GUID, zone.Id) //call up to the main event system
}
def DestructionTask(entry : DeconstructionActor.VehicleEntry) : Unit = {
val vehicle = entry.vehicle
val zone = entry.zone
taskResolver ! DeconstructionTask(vehicle, zone)
}
/**
* Construct a middleman `Task` intended to return error messages to the `DeconstructionActor`.
* @param vehicle the `Vehicle` object
* @param zone the `Zone` in which the vehicle resides
* @return a `TaskResolver.GiveTask` message
*/
def DeconstructionTask(vehicle : Vehicle, zone : Zone) : TaskResolver.GiveTask = {
import net.psforever.objects.guid.{GUIDTask, Task}
TaskResolver.GiveTask (
new Task() {
private val localVehicle = vehicle
private val localZone = zone
private val localAnnounce = self
override def isComplete : Task.Resolution.Value = Task.Resolution.Success
def Execute(resolver : ActorRef) : Unit = {
resolver ! scala.util.Success(this)
}
override def onFailure(ex : Throwable): Unit = {
localAnnounce ! DeconstructionActor.FailureToDeleteVehicle(localVehicle, localZone, ex)
}
}, List(GUIDTask.UnregisterVehicle(vehicle)(zone.GUID))
)
}
/**
* Iterate over entries in a `List` until an entry that does not exceed the time limit is discovered.
* Separate the original `List` into two:
* a `List` of elements that have exceeded the time limit,
* and a `List` of elements that still satisfy the time limit.
* As newer entries to the `List` will always resolve later than old ones,
* and newer entries are always added to the end of the main `List`,
* processing in order is always correct.
* @param list the `List` of entries to divide
* @param now the time right now (in nanoseconds)
* @see `List.partition`
* @return a `Tuple` of two `Lists`, whose qualifications are explained above
*/
private def PartitionEntries(list : List[DeconstructionActor.VehicleEntry], now : Long) : (List[DeconstructionActor.VehicleEntry], List[DeconstructionActor.VehicleEntry]) = {
val n : Int = recursivePartitionEntries(list.iterator, now)
(list.take(n), list.drop(n)) //take and drop so to always return new lists
}
/**
* Mark the index where the `List` of elements can be divided into two:
* a `List` of elements that have exceeded the time limit,
* and a `List` of elements that still satisfy the time limit.
* @param iter the `Iterator` of entries to divide
* @param now the time right now (in nanoseconds)
* @param index a persistent record of the index where list division should occur;
* defaults to 0
* @return the index where division will occur
*/
@tailrec private def recursivePartitionEntries(iter : Iterator[DeconstructionActor.VehicleEntry], now : Long, index : Int = 0) : Int = {
if(!iter.hasNext) {
index
}
else {
val entry = iter.next()
if(now - entry.time >= DeconstructionActor.timeout_time) {
recursivePartitionEntries(iter, now, index + 1)
}
else {
index
}
}
}
}
object DeconstructionActor {
/** The wait before completely deleting a vehicle; as a Long for calculation simplicity */
private final val timeout_time : Long = 5000000000L //nanoseconds (5s)
/** The wait before completely deleting a vehicle; as a `FiniteDuration` for `Executor` simplicity */
private final val timeout : FiniteDuration = timeout_time nanoseconds
final case class RequestTaskResolver()
/**
* Message that carries information about a vehicle to be deconstructed.
* @param vehicle the `Vehicle` object
* @param zone the `Zone` in which the vehicle resides
* @param time when the vehicle was doomed
* @see `VehicleEntry`
*/
final case class RequestDeleteVehicle(vehicle : Vehicle, zone : Zone, time : Long = System.nanoTime())
/**
* Message that carries information about a vehicle to be deconstructed.
* Prompting, as compared to `RequestDeleteVehicle` which is reactionary.
* @param vehicle_guid the vehicle
* @param zone_id the `Zone` in which the vehicle resides
*/
final case class DeleteVehicle(vehicle_guid : PlanetSideGUID, zone_id : String)
/**
* Internal message used to signal a test of the queued vehicle information.
* Remove all deconstructing vehicles from the game world.
*/
private final case class StartDeleteVehicle()
/**
* Internal message used to signal a test of the queued vehicle information.
* Remove all deconstructing vehicles from the zone's globally unique identifier system.
*/
private final case class TryDeleteVehicle()
/**
* Error-passing message carrying information out of the final deconstruction GUID unregistering task.
* @param vehicle the `Vehicle` object
* @param zone the `Zone` in which the vehicle may or may not reside
* @param ex information regarding what happened
*/
private final case class FailureToDeleteVehicle(vehicle : Vehicle, zone : Zone, ex : Throwable)
/**
* Entry of vehicle information.
* The `zone` is maintained separately as a necessity, required to complete the deletion of the vehicle
* via unregistering of the vehicle and all related, registered objects.
* @param vehicle the `Vehicle` object
* @param zone the `Zone` in which the vehicle resides
* @param time when the vehicle was doomed
* @see `RequestDeleteVehicle`
*/
private final case class VehicleEntry(vehicle : Vehicle, zone : Zone, time : Long)
}

View file

@ -1,114 +0,0 @@
// Copyright (c) 2017 PSForever
package services.vehicle.support
import akka.actor.{Actor, Cancellable}
import net.psforever.objects.{DefaultCancellable, Vehicle}
import net.psforever.objects.zones.Zone
import net.psforever.packet.game.PlanetSideGUID
import services.vehicle.VehicleServiceMessage
import scala.concurrent.duration._
/**
* Maintain and curate a list of timed `vehicle` object deconstruction tasks.<br>
* <br>
* These tasks are queued or dismissed by player activity but they are executed independent of player activity.
* A common disconnected cause of deconstruction is neglect for an extended period of time.
* At that point, the original owner of the vehicle no longer matters.
* Deconstruction neglect, however, is averted by having someone become seated.
* A realized deconstruction is entirely based on a fixed interval after an unresolved request has been received.
* The actual process of deconstructing the vehicle and cleaning up its resources is performed by an external agent.<br>
* <br>
* This `Actor` is intended to sit on top of the event system that handles broadcast messaging.
*/
class DelayedDeconstructionActor extends Actor {
/** The periodic `Executor` that scraps the next vehicle on the list */
private var monitor : Cancellable = DefaultCancellable.obj
/** A `List` of currently doomed vehicles */
private var vehicles : List[DelayedDeconstructionActor.VehicleEntry] = Nil
private[this] val log = org.log4s.getLogger
private[this] def trace(msg : String) : Unit = log.trace(msg)
def receive : Receive = {
case DelayedDeconstructionActor.ScheduleDeconstruction(vehicle, zone, timeAlive) =>
trace(s"delayed deconstruction order for $vehicle in $timeAlive")
val oldHead = vehicles.headOption
val now : Long = System.nanoTime
vehicles = (vehicles :+ DelayedDeconstructionActor.VehicleEntry(vehicle, zone, timeAlive * 1000000000L))
.sortBy(entry => entry.survivalTime - (now - entry.logTime))
if(vehicles.size == 1 || oldHead != vehicles.headOption) { //we were the only entry so the event must be started from scratch
RetimePeriodicTest()
}
case DelayedDeconstructionActor.UnscheduleDeconstruction(vehicle_guid) =>
//all tasks for this vehicle are cleared from the queue
//clear any task that is no longer valid by determination of unregistered GUID
val before = vehicles.length
val now : Long = System.nanoTime
vehicles = vehicles.filter(entry => { entry.vehicle.HasGUID && entry.vehicle.GUID != vehicle_guid })
.sortBy(entry => entry.survivalTime - (now - entry.logTime))
trace(s"attempting to clear deconstruction order for vehicle $vehicle_guid, found ${before - vehicles.length}")
RetimePeriodicTest()
case DelayedDeconstructionActor.PeriodicTaskCulling =>
//filter the list of deconstruction tasks for any that are need to be triggered
monitor.cancel
val now : Long = System.nanoTime
val (vehiclesToDecon, vehiclesRemain) = vehicles.partition(entry => { now - entry.logTime >= entry.survivalTime })
vehicles = vehiclesRemain.sortBy(_.survivalTime)
trace(s"vehicle culling - ${vehiclesToDecon.length} deconstruction tasks found; ${vehiclesRemain.length} tasks remain")
vehiclesToDecon.foreach(entry => { context.parent ! VehicleServiceMessage.RequestDeleteVehicle(entry.vehicle, entry.zone) })
RetimePeriodicTest()
case _ => ;
}
def RetimePeriodicTest() : Unit = {
monitor.cancel
vehicles.headOption match {
case None => ;
case Some(entry) =>
val retime = math.max(1, entry.survivalTime - (System.nanoTime - entry.logTime)) nanoseconds
import scala.concurrent.ExecutionContext.Implicits.global
monitor = context.system.scheduler.scheduleOnce(retime, self, DelayedDeconstructionActor.PeriodicTaskCulling)
}
}
}
object DelayedDeconstructionActor {
/**
* Timer for the repeating executor.
*/
private final val periodicTest : FiniteDuration = 5000000000L nanoseconds //5s
/**
* Queue a future vehicle deconstruction action.
* @param vehicle the `Vehicle` object
* @param zone the `Zone` that the vehicle currently occupies
* @param survivalTime how long until the vehicle will be deconstructed in seconds
*/
final case class ScheduleDeconstruction(vehicle : Vehicle, zone : Zone, survivalTime : Long)
/**
* Dequeue a vehicle from being deconstructed.
* @param vehicle_guid the vehicle
*/
final case class UnscheduleDeconstruction(vehicle_guid : PlanetSideGUID)
/**
* A message the `Actor` sends to itself.
* The trigger for the periodic deconstruction task.
*/
private final case class PeriodicTaskCulling()
/**
* An entry that stores vehicle deconstruction tasks.
* @param vehicle the `Vehicle` object
* @param zone the `Zone` that the vehicle currently occupies
* @param survivalTime how long until the vehicle will be deconstructed in nanoseconds
* @param logTime when this deconstruction request was initially created in nanoseconds;
* initialized by default to a "now"
*/
private final case class VehicleEntry(vehicle : Vehicle, zone : Zone, survivalTime : Long, logTime : Long = System.nanoTime())
}

View file

@ -0,0 +1,56 @@
// Copyright (c) 2017 PSForever
package services.vehicle.support
import net.psforever.objects.Vehicle
import net.psforever.objects.guid.{GUIDTask, TaskResolver}
import net.psforever.objects.zones.Zone
import services.vehicle.{VehicleAction, VehicleServiceMessage}
import services.{RemoverActor, Service}
import scala.concurrent.duration._
class VehicleRemover extends RemoverActor {
final val FirstStandardDuration : FiniteDuration = 5 minutes
final val SecondStandardDuration : FiniteDuration = 5 seconds
def InclusionTest(entry : RemoverActor.Entry) : Boolean = {
entry.obj.isInstanceOf[Vehicle]
}
def InitialJob(entry : RemoverActor.Entry) : Unit = { }
def FirstJob(entry : RemoverActor.Entry) : Unit = {
val vehicle = entry.obj.asInstanceOf[Vehicle]
val vehicleGUID = vehicle.GUID
val zoneId = entry.zone.Id
vehicle.Actor ! Vehicle.PrepareForDeletion
//kick out all passengers
vehicle.Definition.MountPoints.values.foreach(mount => {
val seat = vehicle.Seat(mount).get
seat.Occupant match {
case Some(tplayer) =>
seat.Occupant = None
tplayer.VehicleSeated = None
if(tplayer.HasGUID) {
context.parent ! VehicleServiceMessage(zoneId, VehicleAction.KickPassenger(tplayer.GUID, 4, false, vehicleGUID))
}
case None => ;
}
})
}
override def SecondJob(entry : RemoverActor.Entry) : Unit = {
val vehicle = entry.obj.asInstanceOf[Vehicle]
val zone = entry.zone
zone.Transport ! Zone.Vehicle.Despawn(vehicle)
context.parent ! VehicleServiceMessage(zone.Id, VehicleAction.UnloadVehicle(Service.defaultPlayerGUID, zone, vehicle))
super.SecondJob(entry)
}
def ClearanceTest(entry : RemoverActor.Entry) : Boolean = entry.obj.asInstanceOf[Vehicle].Seats.values.count(_.isOccupied) == 0
def DeletionTask(entry : RemoverActor.Entry) : TaskResolver.GiveTask = {
GUIDTask.UnregisterVehicle(entry.obj.asInstanceOf[Vehicle])(entry.zone.GUID)
}
}