Changed and documented new remoting configuration #2593

This commit is contained in:
Endre Sándor Varga 2013-01-17 16:19:31 +01:00
parent 9935337d8e
commit 99adbdfab4
25 changed files with 319 additions and 406 deletions

View file

@ -23,50 +23,6 @@ import scala.concurrent.{ Promise, Await, Future }
import scala.util.control.NonFatal
import scala.util.{ Failure, Success }
class RemotingSettings(val config: Config) {
import config._
import scala.collection.JavaConverters._
val LogLifecycleEvents: Boolean = getBoolean("akka.remoting.log-remote-lifecycle-events")
val ShutdownTimeout: Timeout =
Duration(getMilliseconds("akka.remoting.shutdown-timeout"), MILLISECONDS)
val FlushWait: FiniteDuration = Duration(getMilliseconds("akka.remoting.flush-wait-on-shutdown"), MILLISECONDS)
val StartupTimeout: Timeout = Timeout(Duration(getMilliseconds("akka.remoting.startup-timeout"), MILLISECONDS))
val RetryGateClosedFor: FiniteDuration = Duration(getMilliseconds("akka.remoting.retry-gate-closed-for"), MILLISECONDS)
val UsePassiveConnections: Boolean = getBoolean("akka.remoting.use-passive-connections")
val MaximumRetriesInWindow: Int = getInt("akka.remoting.maximum-retries-in-window")
val RetryWindow: FiniteDuration = Duration(getMilliseconds("akka.remoting.retry-window"), MILLISECONDS)
val BackoffPeriod: FiniteDuration = Duration(getMilliseconds("akka.remoting.backoff-interval"), MILLISECONDS)
val CommandAckTimeout: Timeout =
Timeout(Duration(getMilliseconds("akka.remoting.command-ack-timeout"), MILLISECONDS))
val Transports: Seq[(String, Seq[String], Config)] = transportNames.map { name
val transportConfig = transportConfigFor(name)
(transportConfig.getString("transport-class"),
immutableSeq(transportConfig.getStringList("applied-adapters")),
transportConfig)
}
val Adapters: Map[String, String] = configToMap(getConfig("akka.remoting.adapters"))
private def transportNames: Seq[String] = immutableSeq(getStringList("akka.remoting.enabled-transports"))
private def transportConfigFor(transportName: String): Config = getConfig("akka.remoting.transports." + transportName)
private def configToMap(cfg: Config): Map[String, String] =
cfg.root.unwrapped.asScala.toMap.map { case (k, v) (k, v.toString) }
}
private[remote] object AddressUrlEncoder {
def apply(address: Address): String = URLEncoder.encode(address.toString, "utf-8")
}
@ -136,14 +92,14 @@ private[remote] class Remoting(_system: ExtendedActorSystem, _provider: RemoteAc
// a lazy val
@volatile var defaultAddress: Address = _
private val settings = new RemotingSettings(provider.remoteSettings.config)
import provider.remoteSettings._
val transportSupervisor = system.asInstanceOf[ActorSystemImpl].systemActorOf(Props[TransportSupervisor], "transports")
override def localAddressForRemote(remote: Address): Address = Remoting.localAddressForRemote(transportMapping, remote)
val log: LoggingAdapter = Logging(system.eventStream, "Remoting")
val eventPublisher = new EventPublisher(system, log, settings.LogLifecycleEvents)
val eventPublisher = new EventPublisher(system, log, LogRemoteLifecycleEvents)
private def notifyError(msg: String, cause: Throwable): Unit =
eventPublisher.notifyListeners(RemotingErrorEvent(new RemoteTransportException(msg, cause)))
@ -152,7 +108,7 @@ private[remote] class Remoting(_system: ExtendedActorSystem, _provider: RemoteAc
import scala.concurrent.ExecutionContext.Implicits.global
endpointManager match {
case Some(manager)
implicit val timeout = settings.ShutdownTimeout
implicit val timeout = ShutdownTimeout
val stopped: Future[Boolean] = (manager ? ShutdownAndFlush).mapTo[Boolean]
def finalize(): Unit = {
@ -164,7 +120,7 @@ private[remote] class Remoting(_system: ExtendedActorSystem, _provider: RemoteAc
case Success(flushSuccessful)
if (!flushSuccessful)
log.warning("Shutdown finished, but flushing timed out. Some messages might not have been sent. " +
"Increase akka.remoting.flush-wait-on-shutdown to a larger value to avoid this.")
"Increase akka.remote.flush-wait-on-shutdown to a larger value to avoid this.")
finalize()
case Failure(e)
@ -193,7 +149,7 @@ private[remote] class Remoting(_system: ExtendedActorSystem, _provider: RemoteAc
manager ! Listen(addressesPromise)
val transports: Seq[(Transport, Address)] = Await.result(addressesPromise.future,
settings.StartupTimeout.duration)
StartupTimeout.duration)
if (transports.isEmpty) throw new RemoteTransportException("No transport drivers were loaded.", null)
transportMapping = transports.groupBy {
@ -203,6 +159,8 @@ private[remote] class Remoting(_system: ExtendedActorSystem, _provider: RemoteAc
defaultAddress = transports.head._2
addresses = transports.map { _._2 }.toSet
log.info("Remoting started; listening on addresses :" + addresses.mkString("[", ", ", "]"))
manager ! StartupFinished
eventPublisher.notifyListeners(RemotingListenEvent(addresses))
@ -228,7 +186,7 @@ private[remote] class Remoting(_system: ExtendedActorSystem, _provider: RemoteAc
override def managementCommand(cmd: Any): Future[Boolean] = endpointManager match {
case Some(manager)
import system.dispatcher
implicit val timeout = settings.CommandAckTimeout
implicit val timeout = CommandAckTimeout
manager ? ManagementCommand(cmd) map { case ManagementCommandAck(status) status }
case None throw new IllegalStateException("Attempted to send management command but Remoting is not running.")
}
@ -237,7 +195,7 @@ private[remote] class Remoting(_system: ExtendedActorSystem, _provider: RemoteAc
protected def useUntrustedMode: Boolean = provider.remoteSettings.UntrustedMode
// Not used anywhere only to keep compatibility with RemoteTransport interface
protected def logRemoteLifeCycleEvents: Boolean = provider.remoteSettings.LogRemoteLifeCycleEvents
protected def logRemoteLifeCycleEvents: Boolean = LogRemoteLifecycleEvents
}
@ -356,11 +314,11 @@ private[remote] class EndpointManager(conf: Config, log: LoggingAdapter) extends
import EndpointManager._
import context.dispatcher
val settings = new RemotingSettings(conf)
val settings = new RemoteSettings(conf)
val extendedSystem = context.system.asInstanceOf[ExtendedActorSystem]
val endpointId: Iterator[Int] = Iterator from 0
val eventPublisher = new EventPublisher(context.system, log, settings.LogLifecycleEvents)
val eventPublisher = new EventPublisher(context.system, log, settings.LogRemoteLifecycleEvents)
// Mapping between addresses and endpoint actors. If passive connections are turned off, incoming connections
// will be not part of this map!
@ -540,7 +498,7 @@ private[remote] class EndpointManager(conf: Config, log: LoggingAdapter) extends
private def createEndpoint(remoteAddress: Address,
localAddress: Address,
transport: Transport,
endpointSettings: RemotingSettings,
endpointSettings: RemoteSettings,
handleOption: Option[AssociationHandle]): ActorRef = {
assert(transportMapping contains localAddress)
@ -552,7 +510,7 @@ private[remote] class EndpointManager(conf: Config, log: LoggingAdapter) extends
transport,
endpointSettings,
AkkaPduProtobufCodec))
.withDispatcher("akka.remoting.writer-dispatcher"),
.withDispatcher("akka.remote.writer-dispatcher"),
"endpointWriter-" + AddressUrlEncoder(remoteAddress) + "-" + endpointId.next()))
}