mirror of
https://github.com/psforever/PSF-LoginServer.git
synced 2026-01-19 18:44:45 +00:00
configuration for the game world server to connect to clients across a number of socket-port connections following a simple numerical load balancing policy
This commit is contained in:
parent
306e2a63c0
commit
fc11cb31ef
|
|
@ -14,7 +14,7 @@ import akka.{actor => classic}
|
|||
import ch.qos.logback.classic.LoggerContext
|
||||
import ch.qos.logback.classic.joran.JoranConfigurator
|
||||
import io.sentry.{Sentry, SentryOptions}
|
||||
import net.psforever.actors.net.{LoginActor, MiddlewareActor, SocketActor}
|
||||
import net.psforever.actors.net.{LoginActor, MiddlewareActor, SocketActor, SocketPane}
|
||||
import net.psforever.actors.session.SessionActor
|
||||
import net.psforever.login.psadmin.PsAdminActor
|
||||
import net.psforever.login._
|
||||
|
|
@ -99,7 +99,7 @@ object Server {
|
|||
Default(system)
|
||||
|
||||
// typed to classic wrappers for login and session actors
|
||||
val login = (ref: ActorRef[MiddlewareActor.Command], info: InetSocketAddress, connectionId: String) => {
|
||||
val loginPlan = (ref: ActorRef[MiddlewareActor.Command], info: InetSocketAddress, connectionId: String) => {
|
||||
import net.psforever.services.account.IPAddress
|
||||
Behaviors.setup[PlanetSidePacket](context => {
|
||||
val actor = context.actorOf(classic.Props(new LoginActor(ref, connectionId, Login.getNewId())), "login")
|
||||
|
|
@ -110,7 +110,7 @@ object Server {
|
|||
})
|
||||
})
|
||||
}
|
||||
val session = (ref: ActorRef[MiddlewareActor.Command], info: InetSocketAddress, connectionId: String) => {
|
||||
val sessionPlan = (ref: ActorRef[MiddlewareActor.Command], info: InetSocketAddress, connectionId: String) => {
|
||||
Behaviors.setup[PlanetSidePacket](context => {
|
||||
val uuid = randomUUID().toString
|
||||
val actor =
|
||||
|
|
@ -135,8 +135,8 @@ object Server {
|
|||
system.spawn(InterstellarClusterService(zones), InterstellarClusterService.InterstellarClusterServiceKey.id)
|
||||
system.spawn(ChatService(), ChatService.ChatServiceKey.id)
|
||||
|
||||
system.spawn(SocketActor(new InetSocketAddress(bindAddress, Config.app.login.port), login), "login-socket")
|
||||
system.spawn(SocketActor(new InetSocketAddress(bindAddress, Config.app.world.port), session), "world-socket")
|
||||
system.spawn(SocketActor(new InetSocketAddress(bindAddress, Config.app.login.port), loginPlan), "login-socket")
|
||||
system.spawn(SocketPane(bindAddress, sessionPlan), "world-socket-pane")
|
||||
|
||||
val adminListener = system.actorOf(
|
||||
classic.Props(
|
||||
|
|
|
|||
|
|
@ -20,6 +20,7 @@ login {
|
|||
world {
|
||||
# UDP listening port
|
||||
port = 51001
|
||||
ports = [51003, 51004, 51005, 51006, 51007, 51008, 51009, 51010]
|
||||
|
||||
# The name of the server as displayed in the server browser.
|
||||
server-name = PSForever
|
||||
|
|
|
|||
|
|
@ -23,14 +23,13 @@ import scala.util.{Failure, Success}
|
|||
|
||||
/*
|
||||
object LoginActor {
|
||||
def apply(
|
||||
/ef apply(
|
||||
middlewareActor: typed.ActorRef[MiddlewareActor.Command],
|
||||
uuid: String
|
||||
): Behavior[Command] =
|
||||
Behaviors.setup(context => new LoginActor(context, middlewareActor, uuid).start())
|
||||
|
||||
sealed trait Command
|
||||
|
||||
}
|
||||
|
||||
class LoginActor(
|
||||
|
|
@ -42,7 +41,6 @@ class LoginActor(
|
|||
Behaviors.receiveMessagePartial {}
|
||||
}
|
||||
}
|
||||
|
||||
*/
|
||||
|
||||
class LoginActor(middlewareActor: typed.ActorRef[MiddlewareActor.Command], connectionId: String, sessionId: Long)
|
||||
|
|
@ -110,7 +108,11 @@ class LoginActor(middlewareActor: typed.ActorRef[MiddlewareActor.Command], conne
|
|||
|
||||
case ConnectToWorldRequestMessage(name, _, _, _, _, _, _, _) =>
|
||||
log.info(s"Connect to world request for '$name'")
|
||||
val response = ConnectToWorldMessage(serverName, publicAddress.getAddress.getHostAddress, publicAddress.getPort)
|
||||
val response = ConnectToWorldMessage(
|
||||
serverName,
|
||||
publicAddress.getAddress.getHostAddress,
|
||||
SocketPane.Rotation.NextPort
|
||||
)
|
||||
middlewareActor ! MiddlewareActor.Send(response)
|
||||
middlewareActor ! MiddlewareActor.Close()
|
||||
|
||||
|
|
|
|||
|
|
@ -4,7 +4,6 @@ import java.net.InetSocketAddress
|
|||
import java.security.SecureRandom
|
||||
import java.util.UUID.randomUUID
|
||||
import java.util.concurrent.ThreadLocalRandom
|
||||
|
||||
import akka.actor.Cancellable
|
||||
import akka.{actor => classic}
|
||||
import akka.actor.typed.{ActorRef, ActorTags, Behavior, PostStop, Terminated}
|
||||
|
|
@ -25,15 +24,15 @@ import scala.util.Random
|
|||
*/
|
||||
object SocketActor {
|
||||
def apply(
|
||||
address: InetSocketAddress,
|
||||
next: (ActorRef[MiddlewareActor.Command], InetSocketAddress, String) => Behavior[PlanetSidePacket]
|
||||
): Behavior[Command] =
|
||||
Behaviors.setup(context => new SocketActor(context, address, next).start())
|
||||
address: InetSocketAddress,
|
||||
nextPlan: (ActorRef[MiddlewareActor.Command], InetSocketAddress, String) => Behavior[PlanetSidePacket]
|
||||
): Behavior[Command] =
|
||||
Behaviors.setup(context => new SocketActor(context, address, nextPlan).start())
|
||||
|
||||
sealed trait Command
|
||||
|
||||
private final case class UdpCommandMessage(message: Udp.Command) extends Command
|
||||
private final case class UdpEventMessage(message: Udp.Event) extends Command
|
||||
private[net] final case class UdpCommandMessage(message: Udp.Command) extends Command
|
||||
private[net] final case class UdpEventMessage(message: Udp.Event) extends Command
|
||||
private final case class UdpUnboundMessage(message: Udp.Unbound) extends Command
|
||||
private final case class Bound(socket: classic.ActorRef) extends Command
|
||||
private final case class StopChild(ref: ActorRef[MiddlewareActor.Command]) extends Command
|
||||
|
|
@ -48,59 +47,59 @@ object SocketActor {
|
|||
context.system.terminate()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// TODO? This doesn't quite support all parameters of the old network simulator
|
||||
// Need to decide wheter they are necessary or not
|
||||
// https://github.com/psforever/PSF-LoginServer/blob/07f447c2344ab55d581317316c41571772ac2242/src/main/scala/net/psforever/login/UdpNetworkSimulator.scala
|
||||
private object NetworkSimulator {
|
||||
def apply(socketActor: ActorRef[SocketActor.Command]): Behavior[Udp.Message] =
|
||||
Behaviors.setup(context => new NetworkSimulator(context, socketActor))
|
||||
// TODO? This doesn't quite support all parameters of the old network simulator
|
||||
// Need to decide wheter they are necessary or not
|
||||
// https://github.com/psforever/PSF-LoginServer/blob/07f447c2344ab55d581317316c41571772ac2242/src/main/scala/net/psforever/login/UdpNetworkSimulator.scala
|
||||
private object NetworkSimulator {
|
||||
def apply(socketActor: ActorRef[SocketActor.Command]): Behavior[Udp.Message] =
|
||||
Behaviors.setup(context => new NetworkSimulator(context, socketActor))
|
||||
}
|
||||
|
||||
private class NetworkSimulator(context: ActorContext[Udp.Message], socketActor: ActorRef[SocketActor.Command])
|
||||
extends AbstractBehavior[Udp.Message](context) {
|
||||
|
||||
private[this] val log = org.log4s.getLogger
|
||||
|
||||
override def onMessage(message: Udp.Message): Behavior[Udp.Message] = {
|
||||
message match {
|
||||
case _: Udp.Received | _: Udp.Send =>
|
||||
simulate(message)
|
||||
Behaviors.same
|
||||
case _ =>
|
||||
socketActor ! toSocket(message)
|
||||
Behaviors.same
|
||||
}
|
||||
}
|
||||
|
||||
private class NetworkSimulator(context: ActorContext[Udp.Message], socketActor: ActorRef[SocketActor.Command])
|
||||
extends AbstractBehavior[Udp.Message](context) {
|
||||
|
||||
private[this] val log = org.log4s.getLogger
|
||||
|
||||
override def onMessage(message: Udp.Message): Behavior[Udp.Message] = {
|
||||
message match {
|
||||
case _: Udp.Received | _: Udp.Send =>
|
||||
simulate(message)
|
||||
Behaviors.same
|
||||
case _ =>
|
||||
socketActor ! toSocket(message)
|
||||
Behaviors.same
|
||||
}
|
||||
}
|
||||
|
||||
def simulate(message: Udp.Message): Unit = {
|
||||
if (Random.nextDouble() > Config.app.development.netSim.loss) {
|
||||
if (Random.nextDouble() <= Config.app.development.netSim.reorderChance) {
|
||||
context.scheduleOnce(
|
||||
ThreadLocalRandom.current().nextDouble(0.01, 0.2).seconds,
|
||||
socketActor,
|
||||
toSocket(message)
|
||||
)
|
||||
} else {
|
||||
socketActor ! toSocket(message)
|
||||
}
|
||||
def simulate(message: Udp.Message): Unit = {
|
||||
if (Random.nextDouble() > Config.app.development.netSim.loss) {
|
||||
if (Random.nextDouble() <= Config.app.development.netSim.reorderChance) {
|
||||
context.scheduleOnce(
|
||||
ThreadLocalRandom.current().nextDouble(0.01, 0.2).seconds,
|
||||
socketActor,
|
||||
toSocket(message)
|
||||
)
|
||||
} else {
|
||||
log.trace("Network simulator dropped packet")
|
||||
socketActor ! toSocket(message)
|
||||
}
|
||||
} else {
|
||||
log.trace("Network simulator dropped packet")
|
||||
}
|
||||
|
||||
def toSocket(message: Udp.Message): Command =
|
||||
message match {
|
||||
case message: Udp.Command => UdpCommandMessage(message)
|
||||
case message: Udp.Event => UdpEventMessage(message)
|
||||
}
|
||||
}
|
||||
|
||||
def toSocket(message: Udp.Message): SocketActor.Command =
|
||||
message match {
|
||||
case message: Udp.Command => SocketActor.UdpCommandMessage(message)
|
||||
case message: Udp.Event => SocketActor.UdpEventMessage(message)
|
||||
}
|
||||
}
|
||||
|
||||
class SocketActor(
|
||||
context: ActorContext[SocketActor.Command],
|
||||
address: InetSocketAddress,
|
||||
next: (ActorRef[MiddlewareActor.Command], InetSocketAddress, String) => Behavior[PlanetSidePacket]
|
||||
context: ActorContext[SocketActor.Command],
|
||||
address: InetSocketAddress,
|
||||
nextPlan: (ActorRef[MiddlewareActor.Command], InetSocketAddress, String) => Behavior[PlanetSidePacket]
|
||||
) {
|
||||
import SocketActor._
|
||||
import SocketActor.Command
|
||||
|
|
@ -183,7 +182,7 @@ class SocketActor(
|
|||
case None =>
|
||||
val connectionId = randomUUID.toString
|
||||
val ref = context.spawn(
|
||||
MiddlewareActor(udpCommandAdapter, remote, next, connectionId),
|
||||
MiddlewareActor(udpCommandAdapter, remote, nextPlan, connectionId),
|
||||
s"middleware-$connectionId",
|
||||
ActorTags(s"uuid=$connectionId")
|
||||
)
|
||||
|
|
|
|||
97
src/main/scala/net/psforever/actors/net/SocketPane.scala
Normal file
97
src/main/scala/net/psforever/actors/net/SocketPane.scala
Normal file
|
|
@ -0,0 +1,97 @@
|
|||
// Copyright (c) 2024 PSForever
|
||||
package net.psforever.actors.net
|
||||
|
||||
import net.psforever.util.Config
|
||||
|
||||
import java.net.{InetAddress, InetSocketAddress}
|
||||
//import akka.{actor => classic}
|
||||
import akka.actor.typed.{ActorRef, Behavior, PostStop}
|
||||
import akka.actor.typed.scaladsl.{ActorContext, Behaviors}
|
||||
import net.psforever.packet.PlanetSidePacket
|
||||
|
||||
final case class SocketPanePortRotation(
|
||||
portNumbers: Seq[Int],
|
||||
index: Int = 0
|
||||
) {
|
||||
private var currentIndex: Int = index
|
||||
|
||||
def NextPort: Int = this.synchronized {
|
||||
val out = portNumbers.lift(currentIndex).getOrElse(Config.app.world.port)
|
||||
currentIndex += 1
|
||||
currentIndex %= portNumbers.size
|
||||
out
|
||||
}
|
||||
}
|
||||
|
||||
object SocketPanePortRotation {
|
||||
def apply(rotation: SocketPanePortRotation): SocketPanePortRotation = {
|
||||
SocketPanePortRotation(rotation.portNumbers, rotation.currentIndex)
|
||||
}
|
||||
|
||||
def apply(rotation: SocketPanePortRotation, newPort: Int): SocketPanePortRotation = {
|
||||
SocketPanePortRotation(rotation.portNumbers :+ newPort, rotation.currentIndex)
|
||||
}
|
||||
}
|
||||
|
||||
/** SocketActor creates a UDP socket, receives packets and forwards them to MiddlewareActor
|
||||
* There is only one SocketActor, but each connected client gets its own MiddlewareActor
|
||||
*/
|
||||
object SocketPane {
|
||||
def apply(
|
||||
address: InetAddress,
|
||||
nextPlan: (ActorRef[MiddlewareActor.Command], InetSocketAddress, String) => Behavior[PlanetSidePacket]
|
||||
): Behavior[Command] =
|
||||
Behaviors.setup(context => new SocketPane(context, address, nextPlan).start())
|
||||
|
||||
sealed trait Command
|
||||
|
||||
final case class CreateNewSocket(port: Int) extends Command
|
||||
|
||||
private var rotation: SocketPanePortRotation = SocketPanePortRotation(Array(Config.app.world.port))
|
||||
|
||||
def Rotation: SocketPanePortRotation = SocketPanePortRotation(rotation)
|
||||
|
||||
final def getDefaultPorts: Seq[Int] = {
|
||||
val config = Config.app.world
|
||||
(config.port +: config.ports).distinct
|
||||
}
|
||||
}
|
||||
|
||||
class SocketPane(
|
||||
context: ActorContext[SocketPane.Command],
|
||||
address: InetAddress,
|
||||
next: (ActorRef[MiddlewareActor.Command], InetSocketAddress, String) => Behavior[PlanetSidePacket]
|
||||
) {
|
||||
private[this] val log = org.log4s.getLogger
|
||||
|
||||
private var socketActors: Array[ActorRef[SocketActor.Command]] = SocketPane.getDefaultPorts.map { i =>
|
||||
context.spawn(SocketActor(new InetSocketAddress(address, i), next), name=s"world-socket-$i")
|
||||
}.toArray
|
||||
SocketPane.rotation = SocketPanePortRotation(SocketPane.getDefaultPorts)
|
||||
|
||||
log.info(s"Configured ${SocketPane.getDefaultPorts.size} game world instance ports")
|
||||
|
||||
def start(): Behavior[SocketPane.Command] = {
|
||||
Behaviors
|
||||
.receiveMessagePartial[SocketPane.Command] {
|
||||
case SocketPane.CreateNewSocket(port)
|
||||
if SocketPane.Rotation.portNumbers.contains(port) =>
|
||||
Behaviors.same
|
||||
|
||||
case SocketPane.CreateNewSocket(port) =>
|
||||
socketActors = socketActors :+ context.spawn(SocketActor(new InetSocketAddress(address, port), next), name=s"world-socket-$port")
|
||||
SocketPane.rotation = SocketPanePortRotation(SocketPane.rotation, port)
|
||||
log.info(s"Requested new socket to port $port has been created")
|
||||
Behaviors.same
|
||||
|
||||
case _ =>
|
||||
Behaviors.same
|
||||
}
|
||||
.receiveSignal {
|
||||
case (_, PostStop) =>
|
||||
socketActors.foreach(context.stop)
|
||||
SocketPane.rotation = SocketPanePortRotation(Array(Config.app.world.port))
|
||||
Behaviors.same
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -104,6 +104,7 @@ case class LoginConfig(
|
|||
|
||||
case class WorldConfig(
|
||||
port: Int,
|
||||
ports: Seq[Int],
|
||||
serverName: String,
|
||||
serverType: ServerType
|
||||
)
|
||||
|
|
|
|||
Loading…
Reference in a new issue