A simpler ActorRefProvider config #20649 (#20767)

* Provide shorter aliases for the ActorRefProviders #20649
* Use the new actorefprovider aliases throughout code and docs
* Cleaner alias replacement logic
This commit is contained in:
Johan Andrén 2016-06-10 15:04:13 +02:00 committed by Patrik Nordwall
parent 3f72e7dad1
commit d6c048f59a
127 changed files with 148 additions and 137 deletions

View file

@ -83,9 +83,11 @@ akka {
actor { actor {
# Either one of "local", "remote" or "cluster" or the
# FQCN of the ActorRefProvider to be used; the below is the built-in default, # FQCN of the ActorRefProvider to be used; the below is the built-in default,
# another one is akka.remote.RemoteActorRefProvider in the akka-remote bundle. # note that "remote" and "cluster" requires the akka-remote and akka-cluster
provider = "akka.actor.LocalActorRefProvider" # artifacts to be on the classpath.
provider = "local"
# The guardian "/user" will use this class to obtain its supervisorStrategy. # The guardian "/user" will use this class to obtain its supervisorStrategy.
# It needs to be a subclass of akka.actor.SupervisorStrategyConfigurator. # It needs to be a subclass of akka.actor.SupervisorStrategyConfigurator.

View file

@ -166,7 +166,15 @@ object ActorSystem {
import config._ import config._
final val ConfigVersion: String = getString("akka.version") final val ConfigVersion: String = getString("akka.version")
final val ProviderClass: String = getString("akka.actor.provider") final val ProviderClass: String =
getString("akka.actor.provider") match {
case "local" => classOf[LocalActorRefProvider].getName
// these two cannot be referenced by class as they may not be on the classpath
case "remote" => "akka.remote.RemoteActorRefProvider"
case "cluster" => "akka.cluster.ClusterActorRefProvider"
case fqcn => fqcn
}
final val SupervisorStrategyClass: String = getString("akka.actor.guardian-supervisor-strategy") final val SupervisorStrategyClass: String = getString("akka.actor.guardian-supervisor-strategy")
final val CreationTimeout: Timeout = Timeout(config.getMillisDuration("akka.actor.creation-timeout")) final val CreationTimeout: Timeout = Timeout(config.getMillisDuration("akka.actor.creation-timeout"))
final val UnstartedPushTimeout: Timeout = Timeout(config.getMillisDuration("akka.actor.unstarted-push-timeout")) final val UnstartedPushTimeout: Timeout = Timeout(config.getMillisDuration("akka.actor.unstarted-push-timeout"))
@ -831,6 +839,7 @@ private[akka] class ActorSystemImpl(
/** /**
* Adds a Runnable that will be executed on ActorSystem termination. * Adds a Runnable that will be executed on ActorSystem termination.
* Note that callbacks are executed in reverse order of insertion. * Note that callbacks are executed in reverse order of insertion.
*
* @param r The callback to be executed on ActorSystem termination * @param r The callback to be executed on ActorSystem termination
* Throws RejectedExecutionException if called after ActorSystem has been terminated. * Throws RejectedExecutionException if called after ActorSystem has been terminated.
*/ */

View file

@ -40,7 +40,7 @@ class CodecBenchmark {
""" """
akka { akka {
loglevel = WARNING loglevel = WARNING
actor.provider = "akka.remote.RemoteActorRefProvider" actor.provider = remote
remote.artery.enabled = on remote.artery.enabled = on
remote.artery.hostname = localhost remote.artery.hostname = localhost
remote.artery.port = 0 remote.artery.port = 0

View file

@ -153,7 +153,7 @@ object MetricsConfig {
gossip-interval = 1s gossip-interval = 1s
} }
} }
akka.actor.provider = "akka.remote.RemoteActorRefProvider" akka.actor.provider = remote
""" """
/** Test w/o cluster, with collection disabled. */ /** Test w/o cluster, with collection disabled. */
@ -163,7 +163,7 @@ object MetricsConfig {
enabled = off enabled = off
} }
} }
akka.actor.provider = "akka.remote.RemoteActorRefProvider" akka.actor.provider = remote
""" """
/** Test in cluster, with manual collection activation, collector mock, fast. */ /** Test in cluster, with manual collection activation, collector mock, fast. */
@ -178,7 +178,7 @@ object MetricsConfig {
fallback = false fallback = false
} }
} }
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "cluster"
""" """
} }

View file

@ -15,7 +15,7 @@ import akka.cluster.TestMember
import akka.cluster.metrics.MetricsGossipEnvelope import akka.cluster.metrics.MetricsGossipEnvelope
class MessageSerializerSpec extends AkkaSpec( class MessageSerializerSpec extends AkkaSpec(
"akka.actor.provider = akka.cluster.ClusterActorRefProvider") { "akka.actor.provider = cluster") {
val serializer = new MessageSerializer(system.asInstanceOf[ExtendedActorSystem]) val serializer = new MessageSerializer(system.asInstanceOf[ExtendedActorSystem])

View file

@ -85,7 +85,7 @@ abstract class ClusterShardingCustomShardAllocationSpecConfig(val mode: String)
commonConfig(ConfigFactory.parseString(s""" commonConfig(ConfigFactory.parseString(s"""
akka.loglevel = INFO akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "cluster"
akka.remote.log-remote-lifecycle-events = off akka.remote.log-remote-lifecycle-events = off
akka.persistence.journal.plugin = "akka.persistence.journal.leveldb-shared" akka.persistence.journal.plugin = "akka.persistence.journal.leveldb-shared"
akka.persistence.journal.leveldb-shared { akka.persistence.journal.leveldb-shared {

View file

@ -55,7 +55,7 @@ abstract class ClusterShardingFailureSpecConfig(val mode: String) extends MultiN
commonConfig(ConfigFactory.parseString(s""" commonConfig(ConfigFactory.parseString(s"""
akka.loglevel = INFO akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "cluster"
akka.remote.log-remote-lifecycle-events = off akka.remote.log-remote-lifecycle-events = off
akka.cluster.auto-down-unreachable-after = 0s akka.cluster.auto-down-unreachable-after = 0s
akka.cluster.roles = ["backend"] akka.cluster.roles = ["backend"]

View file

@ -46,7 +46,7 @@ object ClusterShardingGetStateSpecConfig extends MultiNodeConfig {
commonConfig(ConfigFactory.parseString(""" commonConfig(ConfigFactory.parseString("""
akka.loglevel = INFO akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "cluster"
akka.remote.log-remote-lifecycle-events = off akka.remote.log-remote-lifecycle-events = off
akka.cluster.metrics.enabled = off akka.cluster.metrics.enabled = off
akka.cluster.auto-down-unreachable-after = 0s akka.cluster.auto-down-unreachable-after = 0s

View file

@ -47,7 +47,7 @@ object ClusterShardingGetStatsSpecConfig extends MultiNodeConfig {
commonConfig(ConfigFactory.parseString(""" commonConfig(ConfigFactory.parseString("""
akka.loglevel = INFO akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "cluster"
akka.remote.log-remote-lifecycle-events = off akka.remote.log-remote-lifecycle-events = off
akka.log-dead-letters-during-shutdown = off akka.log-dead-letters-during-shutdown = off
akka.cluster.metrics.enabled = off akka.cluster.metrics.enabled = off

View file

@ -72,7 +72,7 @@ abstract class ClusterShardingGracefulShutdownSpecConfig(val mode: String) exten
commonConfig(ConfigFactory.parseString(s""" commonConfig(ConfigFactory.parseString(s"""
akka.loglevel = INFO akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "cluster"
akka.remote.log-remote-lifecycle-events = off akka.remote.log-remote-lifecycle-events = off
akka.persistence.journal.plugin = "akka.persistence.journal.leveldb-shared" akka.persistence.journal.plugin = "akka.persistence.journal.leveldb-shared"
akka.persistence.journal.leveldb-shared { akka.persistence.journal.leveldb-shared {

View file

@ -63,7 +63,7 @@ abstract class ClusterShardingLeavingSpecConfig(val mode: String) extends MultiN
commonConfig(ConfigFactory.parseString(s""" commonConfig(ConfigFactory.parseString(s"""
akka.loglevel = INFO akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "cluster"
akka.remote.log-remote-lifecycle-events = off akka.remote.log-remote-lifecycle-events = off
akka.cluster.auto-down-unreachable-after = 0s akka.cluster.auto-down-unreachable-after = 0s
akka.persistence.journal.plugin = "akka.persistence.journal.leveldb-shared" akka.persistence.journal.plugin = "akka.persistence.journal.leveldb-shared"

View file

@ -125,7 +125,7 @@ abstract class ClusterShardingSpecConfig(val mode: String) extends MultiNodeConf
commonConfig(ConfigFactory.parseString(s""" commonConfig(ConfigFactory.parseString(s"""
akka.loglevel = INFO akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "cluster"
akka.remote.log-remote-lifecycle-events = off akka.remote.log-remote-lifecycle-events = off
akka.cluster.auto-down-unreachable-after = 0s akka.cluster.auto-down-unreachable-after = 0s
akka.cluster.roles = ["backend"] akka.cluster.roles = ["backend"]

View file

@ -27,7 +27,7 @@ import org.apache.commons.io.FileUtils
object RemoveInternalClusterShardingDataSpec { object RemoveInternalClusterShardingDataSpec {
val config = """ val config = """
akka.loglevel = INFO akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "cluster"
akka.remote.netty.tcp.port = 0 akka.remote.netty.tcp.port = 0
akka.persistence.journal.plugin = "akka.persistence.journal.leveldb" akka.persistence.journal.plugin = "akka.persistence.journal.leveldb"
akka.persistence.journal.leveldb { akka.persistence.journal.leveldb {

View file

@ -30,7 +30,7 @@ object ClusterClientSpec extends MultiNodeConfig {
commonConfig(ConfigFactory.parseString(""" commonConfig(ConfigFactory.parseString("""
akka.loglevel = INFO akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "cluster"
akka.remote.log-remote-lifecycle-events = off akka.remote.log-remote-lifecycle-events = off
akka.cluster.auto-down-unreachable-after = 0s akka.cluster.auto-down-unreachable-after = 0s
akka.cluster.client.heartbeat-interval = 1s akka.cluster.client.heartbeat-interval = 1s

View file

@ -19,7 +19,7 @@ object ClusterClientStopSpec extends MultiNodeConfig {
val second = role("second") val second = role("second")
commonConfig(ConfigFactory.parseString(""" commonConfig(ConfigFactory.parseString("""
akka.loglevel = INFO akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "cluster"
akka.remote.log-remote-lifecycle-events = off akka.remote.log-remote-lifecycle-events = off
akka.cluster.client { akka.cluster.client {
heartbeat-interval = 1s heartbeat-interval = 1s

View file

@ -28,7 +28,7 @@ object DistributedPubSubMediatorSpec extends MultiNodeConfig {
commonConfig(ConfigFactory.parseString(""" commonConfig(ConfigFactory.parseString("""
akka.loglevel = INFO akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "cluster"
akka.remote.log-remote-lifecycle-events = off akka.remote.log-remote-lifecycle-events = off
akka.cluster.auto-down-unreachable-after = 0s akka.cluster.auto-down-unreachable-after = 0s
akka.cluster.pub-sub.max-delta-elements = 500 akka.cluster.pub-sub.max-delta-elements = 500

View file

@ -38,7 +38,7 @@ object ClusterSingletonManagerChaosSpec extends MultiNodeConfig {
commonConfig(ConfigFactory.parseString(""" commonConfig(ConfigFactory.parseString("""
akka.loglevel = INFO akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "cluster"
akka.remote.log-remote-lifecycle-events = off akka.remote.log-remote-lifecycle-events = off
akka.cluster.auto-down-unreachable-after = 0s akka.cluster.auto-down-unreachable-after = 0s
""")) """))

View file

@ -35,7 +35,7 @@ object ClusterSingletonManagerLeaveSpec extends MultiNodeConfig {
commonConfig(ConfigFactory.parseString(""" commonConfig(ConfigFactory.parseString("""
akka.loglevel = INFO akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "cluster"
akka.remote.log-remote-lifecycle-events = off akka.remote.log-remote-lifecycle-events = off
akka.cluster.auto-down-unreachable-after = off akka.cluster.auto-down-unreachable-after = off
""")) """))

View file

@ -41,7 +41,7 @@ object ClusterSingletonManagerSpec extends MultiNodeConfig {
commonConfig(ConfigFactory.parseString(""" commonConfig(ConfigFactory.parseString("""
akka.loglevel = INFO akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "cluster"
akka.remote.log-remote-lifecycle-events = off akka.remote.log-remote-lifecycle-events = off
akka.cluster.auto-down-unreachable-after = 0s akka.cluster.auto-down-unreachable-after = 0s
""")) """))

View file

@ -35,7 +35,7 @@ object ClusterSingletonManagerStartupSpec extends MultiNodeConfig {
commonConfig(ConfigFactory.parseString(""" commonConfig(ConfigFactory.parseString("""
akka.loglevel = INFO akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "cluster"
akka.remote.log-remote-lifecycle-events = off akka.remote.log-remote-lifecycle-events = off
akka.cluster.auto-down-unreachable-after = 0s akka.cluster.auto-down-unreachable-after = 0s
""")) """))

View file

@ -21,7 +21,7 @@ public class ClusterClientTest extends JUnitSuite {
public static AkkaJUnitActorSystemResource actorSystemResource = public static AkkaJUnitActorSystemResource actorSystemResource =
new AkkaJUnitActorSystemResource("DistributedPubSubMediatorTest", new AkkaJUnitActorSystemResource("DistributedPubSubMediatorTest",
ConfigFactory.parseString( ConfigFactory.parseString(
"akka.actor.provider = \"akka.cluster.ClusterActorRefProvider\"\n" + "akka.actor.provider = \"cluster\"\n" +
"akka.remote.netty.tcp.port=0")); "akka.remote.netty.tcp.port=0"));
private final ActorSystem system = actorSystemResource.getSystem(); private final ActorSystem system = actorSystemResource.getSystem();

View file

@ -25,7 +25,7 @@ public class DistributedPubSubMediatorTest extends JUnitSuite {
public static AkkaJUnitActorSystemResource actorSystemResource = public static AkkaJUnitActorSystemResource actorSystemResource =
new AkkaJUnitActorSystemResource("DistributedPubSubMediatorTest", new AkkaJUnitActorSystemResource("DistributedPubSubMediatorTest",
ConfigFactory.parseString( ConfigFactory.parseString(
"akka.actor.provider = \"akka.cluster.ClusterActorRefProvider\"\n" + "akka.actor.provider = \"cluster\"\n" +
"akka.remote.netty.tcp.port=0")); "akka.remote.netty.tcp.port=0"));
private final ActorSystem system = actorSystemResource.getSystem(); private final ActorSystem system = actorSystemResource.getSystem();

View file

@ -15,7 +15,7 @@ case class UnwrappedMessage(msg: String)
object DistributedPubSubMediatorRouterSpec { object DistributedPubSubMediatorRouterSpec {
def config(routingLogic: String) = s""" def config(routingLogic: String) = s"""
akka.loglevel = INFO akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "cluster"
akka.remote.netty.tcp.port=0 akka.remote.netty.tcp.port=0
akka.remote.log-remote-lifecycle-events = off akka.remote.log-remote-lifecycle-events = off
akka.cluster.pub-sub.routing-logic = $routingLogic akka.cluster.pub-sub.routing-logic = $routingLogic

View file

@ -68,7 +68,7 @@ object ClusterSingletonProxySpec {
min-nr-of-members = 2 min-nr-of-members = 2
} }
actor.provider = "akka.cluster.ClusterActorRefProvider" actor.provider = "cluster"
remote { remote {
log-remote-lifecycle-events = off log-remote-lifecycle-events = off

View file

@ -34,7 +34,7 @@ object MultiNodeClusterSpec {
if (failureDetectorPuppet) clusterConfigWithFailureDetectorPuppet else clusterConfig if (failureDetectorPuppet) clusterConfigWithFailureDetectorPuppet else clusterConfig
def clusterConfig: Config = ConfigFactory.parseString(""" def clusterConfig: Config = ConfigFactory.parseString("""
akka.actor.provider = akka.cluster.ClusterActorRefProvider akka.actor.provider = cluster
akka.cluster { akka.cluster {
jmx.enabled = off jmx.enabled = off
gossip-interval = 200 ms gossip-interval = 200 ms

View file

@ -124,7 +124,7 @@ private[cluster] object StressMultiJvmSpec extends MultiNodeConfig {
akka.actor.serialize-messages = off akka.actor.serialize-messages = off
akka.actor.serialize-creators = off akka.actor.serialize-creators = off
akka.actor.provider = akka.cluster.ClusterActorRefProvider akka.actor.provider = cluster
akka.cluster { akka.cluster {
failure-detector.acceptable-heartbeat-pause = 5s failure-detector.acceptable-heartbeat-pause = 5s
auto-down-unreachable-after = 1s auto-down-unreachable-after = 1s

View file

@ -24,7 +24,7 @@ class SunnyWeatherMultiJvmSpec(artery: Boolean) extends MultiNodeConfig {
val common = val common =
""" """
akka { akka {
actor.provider = akka.cluster.ClusterActorRefProvider actor.provider = cluster
loggers = ["akka.testkit.TestEventListener"] loggers = ["akka.testkit.TestEventListener"]
loglevel = INFO loglevel = INFO
remote.log-remote-lifecycle-events = off remote.log-remote-lifecycle-events = off

View file

@ -14,7 +14,7 @@ import akka.cluster.routing.ClusterRouterGroupSettings
object ClusterDeployerSpec { object ClusterDeployerSpec {
val deployerConf = ConfigFactory.parseString(""" val deployerConf = ConfigFactory.parseString("""
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "cluster"
akka.actor.deployment { akka.actor.deployment {
/user/service1 { /user/service1 {
router = round-robin-pool router = round-robin-pool

View file

@ -21,7 +21,7 @@ import akka.testkit.TestProbe
object ClusterDomainEventPublisherSpec { object ClusterDomainEventPublisherSpec {
val config = """ val config = """
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "cluster"
akka.remote.netty.tcp.port = 0 akka.remote.netty.tcp.port = 0
""" """
} }

View file

@ -25,7 +25,7 @@ object ClusterSpec {
publish-stats-interval = 0 s # always, when it happens publish-stats-interval = 0 s # always, when it happens
failure-detector.implementation-class = akka.cluster.FailureDetectorPuppet failure-detector.implementation-class = akka.cluster.FailureDetectorPuppet
} }
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "cluster"
akka.remote.log-remote-lifecycle-events = off akka.remote.log-remote-lifecycle-events = off
akka.remote.netty.tcp.port = 0 akka.remote.netty.tcp.port = 0
#akka.loglevel = DEBUG #akka.loglevel = DEBUG
@ -107,7 +107,7 @@ class ClusterSpec extends AkkaSpec(ClusterSpec.config) with ImplicitSender {
"allow join and leave with local address" in { "allow join and leave with local address" in {
val sys2 = ActorSystem("ClusterSpec2", ConfigFactory.parseString(""" val sys2 = ActorSystem("ClusterSpec2", ConfigFactory.parseString("""
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "cluster"
akka.remote.netty.tcp.port = 0 akka.remote.netty.tcp.port = 0
""")) """))
try { try {

View file

@ -38,7 +38,7 @@ class DowningProviderSpec extends WordSpec with Matchers {
""" """
akka { akka {
loglevel = WARNING loglevel = WARNING
actor.provider = "akka.cluster.ClusterActorRefProvider" actor.provider = "cluster"
remote { remote {
netty.tcp { netty.tcp {
hostname = "127.0.0.1" hostname = "127.0.0.1"

View file

@ -21,7 +21,7 @@ object MetricsEnabledSpec {
akka.cluster.metrics.enabled = on akka.cluster.metrics.enabled = on
akka.cluster.metrics.collect-interval = 1 s akka.cluster.metrics.collect-interval = 1 s
akka.cluster.metrics.gossip-interval = 1 s akka.cluster.metrics.gossip-interval = 1 s
akka.actor.provider = "akka.remote.RemoteActorRefProvider" akka.actor.provider = remote
""" """
} }

View file

@ -14,7 +14,7 @@ import akka.actor.ActorLogging
object StartupWithOneThreadSpec { object StartupWithOneThreadSpec {
val config = """ val config = """
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "cluster"
akka.actor.creation-timeout = 10s akka.actor.creation-timeout = 10s
akka.remote.netty.tcp.port = 0 akka.remote.netty.tcp.port = 0

View file

@ -11,7 +11,7 @@ import collection.immutable.SortedSet
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
class ClusterMessageSerializerSpec extends AkkaSpec( class ClusterMessageSerializerSpec extends AkkaSpec(
"akka.actor.provider = akka.cluster.ClusterActorRefProvider") { "akka.actor.provider = cluster") {
val serializer = new ClusterMessageSerializer(system.asInstanceOf[ExtendedActorSystem]) val serializer = new ClusterMessageSerializer(system.asInstanceOf[ExtendedActorSystem])

View file

@ -22,7 +22,7 @@ object ClusterRouterSupervisorSpec {
} }
class ClusterRouterSupervisorSpec extends AkkaSpec(""" class ClusterRouterSupervisorSpec extends AkkaSpec("""
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "cluster"
akka.remote.netty.tcp.port = 0 akka.remote.netty.tcp.port = 0
""") { """) {

View file

@ -12,7 +12,7 @@ import akka.routing.ActorSelectionRoutee
import akka.routing.ActorRefRoutee import akka.routing.ActorRefRoutee
class WeightedRouteesSpec extends AkkaSpec(ConfigFactory.parseString(""" class WeightedRouteesSpec extends AkkaSpec(ConfigFactory.parseString("""
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "cluster"
akka.remote.netty.tcp.port = 0 akka.remote.netty.tcp.port = 0
""")) { """)) {

View file

@ -24,7 +24,7 @@ object JepsenInspiredInsertSpec extends MultiNodeConfig {
commonConfig(ConfigFactory.parseString(""" commonConfig(ConfigFactory.parseString("""
akka.loglevel = INFO akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "cluster"
akka.log-dead-letters = off akka.log-dead-letters = off
akka.log-dead-letters-during-shutdown = off akka.log-dead-letters-during-shutdown = off
akka.remote.log-remote-lifecycle-events = ERROR akka.remote.log-remote-lifecycle-events = ERROR

View file

@ -26,7 +26,7 @@ object PerformanceSpec extends MultiNodeConfig {
commonConfig(ConfigFactory.parseString(""" commonConfig(ConfigFactory.parseString("""
akka.loglevel = ERROR akka.loglevel = ERROR
akka.stdout-loglevel = ERROR akka.stdout-loglevel = ERROR
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "cluster"
akka.log-dead-letters = off akka.log-dead-letters = off
akka.log-dead-letters-during-shutdown = off akka.log-dead-letters-during-shutdown = off
akka.remote.log-remote-lifecycle-events = ERROR akka.remote.log-remote-lifecycle-events = ERROR

View file

@ -22,7 +22,7 @@ object ReplicatorChaosSpec extends MultiNodeConfig {
commonConfig(ConfigFactory.parseString(""" commonConfig(ConfigFactory.parseString("""
akka.loglevel = INFO akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "cluster"
akka.cluster.roles = ["backend"] akka.cluster.roles = ["backend"]
akka.log-dead-letters-during-shutdown = off akka.log-dead-letters-during-shutdown = off
""")) """))

View file

@ -21,7 +21,7 @@ object ReplicatorPruningSpec extends MultiNodeConfig {
commonConfig(ConfigFactory.parseString(""" commonConfig(ConfigFactory.parseString("""
akka.loglevel = INFO akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "cluster"
akka.log-dead-letters-during-shutdown = off akka.log-dead-letters-during-shutdown = off
""")) """))

View file

@ -20,7 +20,7 @@ object ReplicatorSpec extends MultiNodeConfig {
commonConfig(ConfigFactory.parseString(""" commonConfig(ConfigFactory.parseString("""
akka.loglevel = INFO akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "cluster"
akka.log-dead-letters-during-shutdown = off akka.log-dead-letters-during-shutdown = off
""")) """))

View file

@ -44,7 +44,7 @@ class LocalConcurrencySpec(_system: ActorSystem) extends TestKit(_system)
this(ActorSystem( this(ActorSystem(
"LocalConcurrencySpec", "LocalConcurrencySpec",
ConfigFactory.parseString(""" ConfigFactory.parseString("""
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "cluster"
akka.remote.netty.tcp.port=0 akka.remote.netty.tcp.port=0
"""))) """)))
} }

View file

@ -40,7 +40,7 @@ object LotsOfDataBot {
ConfigFactory.parseString(""" ConfigFactory.parseString("""
passive = off passive = off
max-entries = 100000 max-entries = 100000
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "cluster"
akka.remote { akka.remote {
netty.tcp { netty.tcp {
hostname = "127.0.0.1" hostname = "127.0.0.1"

View file

@ -50,7 +50,7 @@ object WriteAggregatorSpec {
} }
class WriteAggregatorSpec extends AkkaSpec(""" class WriteAggregatorSpec extends AkkaSpec("""
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "cluster"
akka.remote.netty.tcp.port=0 akka.remote.netty.tcp.port=0
""") """)
with ImplicitSender { with ImplicitSender {

View file

@ -28,7 +28,7 @@ import com.typesafe.config.ConfigFactory
class ReplicatedDataSerializerSpec extends TestKit(ActorSystem( class ReplicatedDataSerializerSpec extends TestKit(ActorSystem(
"ReplicatedDataSerializerSpec", "ReplicatedDataSerializerSpec",
ConfigFactory.parseString(""" ConfigFactory.parseString("""
akka.actor.provider=akka.cluster.ClusterActorRefProvider akka.actor.provider=cluster
akka.remote.netty.tcp.port=0 akka.remote.netty.tcp.port=0
"""))) with WordSpecLike with Matchers with BeforeAndAfterAll { """))) with WordSpecLike with Matchers with BeforeAndAfterAll {

View file

@ -26,7 +26,7 @@ import com.typesafe.config.ConfigFactory
class ReplicatorMessageSerializerSpec extends TestKit(ActorSystem( class ReplicatorMessageSerializerSpec extends TestKit(ActorSystem(
"ReplicatorMessageSerializerSpec", "ReplicatorMessageSerializerSpec",
ConfigFactory.parseString(""" ConfigFactory.parseString("""
akka.actor.provider=akka.cluster.ClusterActorRefProvider akka.actor.provider=cluster
akka.remote.netty.tcp.port=0 akka.remote.netty.tcp.port=0
"""))) with WordSpecLike with Matchers with BeforeAndAfterAll { """))) with WordSpecLike with Matchers with BeforeAndAfterAll {

View file

@ -153,7 +153,7 @@ A custom ``application.conf`` might look like this::
logging-filter = "akka.event.slf4j.Slf4jLoggingFilter" logging-filter = "akka.event.slf4j.Slf4jLoggingFilter"
actor { actor {
provider = "akka.cluster.ClusterActorRefProvider" provider = "cluster"
default-dispatcher { default-dispatcher {
# Throughput for default Dispatcher, set to 1 for as fair as possible # Throughput for default Dispatcher, set to 1 for as fair as possible

View file

@ -19,8 +19,8 @@ i.e. not necessarily the initial contact points.
provided in a more efficient way by :ref:`distributed-pub-sub-java` for actors that provided in a more efficient way by :ref:`distributed-pub-sub-java` for actors that
belong to the same cluster. belong to the same cluster.
Also, note it's necessary to change ``akka.actor.provider`` from ``akka.actor.LocalActorRefProvider`` Also, note it's necessary to change ``akka.actor.provider`` from ``local``
to ``akka.remote.RemoteActorRefProvider`` or ``akka.cluster.ClusterActorRefProvider`` when using to ``remote`` or ``cluster`` when using
the cluster client. the cluster client.
The receptionist is supposed to be started on all nodes, or all nodes with specified role, The receptionist is supposed to be started on all nodes, or all nodes with specified role,

View file

@ -31,7 +31,7 @@ The ``application.conf`` configuration looks like this:
.. includecode:: ../../../akka-samples/akka-sample-cluster-java/src/main/resources/application.conf#snippet .. includecode:: ../../../akka-samples/akka-sample-cluster-java/src/main/resources/application.conf#snippet
To enable cluster capabilities in your Akka project you should, at a minimum, add the :ref:`remoting-java` To enable cluster capabilities in your Akka project you should, at a minimum, add the :ref:`remoting-java`
settings, but with ``akka.cluster.ClusterActorRefProvider``. settings, but with ``cluster``.
The ``akka.cluster.seed-nodes`` should normally also be added to your ``application.conf`` file. The ``akka.cluster.seed-nodes`` should normally also be added to your ``application.conf`` file.
.. note:: .. note::

View file

@ -31,7 +31,7 @@ to your ``application.conf`` file::
akka { akka {
actor { actor {
provider = "akka.remote.RemoteActorRefProvider" provider = remote
} }
remote { remote {
enabled-transports = ["akka.remote.netty.tcp"] enabled-transports = ["akka.remote.netty.tcp"]
@ -44,7 +44,7 @@ to your ``application.conf`` file::
As you can see in the example above there are four things you need to add to get started: As you can see in the example above there are four things you need to add to get started:
* Change provider from ``akka.actor.LocalActorRefProvider`` to ``akka.remote.RemoteActorRefProvider`` * Change provider from ``local`` to ``remote``
* Add host name - the machine you want to run the actor system on; this host * Add host name - the machine you want to run the actor system on; this host
name is exactly what is passed to remote systems in order to identify this name is exactly what is passed to remote systems in order to identify this
system and consequently used for connecting back to this system if need be, system and consequently used for connecting back to this system if need be,

View file

@ -19,8 +19,8 @@ i.e. not necessarily the initial contact points.
provided in a more efficient way by :ref:`distributed-pub-sub-scala` for actors that provided in a more efficient way by :ref:`distributed-pub-sub-scala` for actors that
belong to the same cluster. belong to the same cluster.
Also, note it's necessary to change ``akka.actor.provider`` from ``akka.actor.LocalActorRefProvider`` Also, note it's necessary to change ``akka.actor.provider`` from ``local``
to ``akka.remote.RemoteActorRefProvider`` or ``akka.cluster.ClusterActorRefProvider`` when using to ``remote`` or ``cluster`` when using
the cluster client. the cluster client.
The receptionist is supposed to be started on all nodes, or all nodes with specified role, The receptionist is supposed to be started on all nodes, or all nodes with specified role,

View file

@ -25,7 +25,7 @@ The ``application.conf`` configuration looks like this:
.. includecode:: ../../../akka-samples/akka-sample-cluster-scala/src/main/resources/application.conf#snippet .. includecode:: ../../../akka-samples/akka-sample-cluster-scala/src/main/resources/application.conf#snippet
To enable cluster capabilities in your Akka project you should, at a minimum, add the :ref:`remoting-scala` To enable cluster capabilities in your Akka project you should, at a minimum, add the :ref:`remoting-scala`
settings, but with ``akka.cluster.ClusterActorRefProvider``. settings, but with ``cluster``.
The ``akka.cluster.seed-nodes`` should normally also be added to your ``application.conf`` file. The ``akka.cluster.seed-nodes`` should normally also be added to your ``application.conf`` file.
.. note:: .. note::

View file

@ -10,7 +10,7 @@ object ClusterDocSpec {
val config = val config =
""" """
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "cluster"
akka.remote.netty.tcp.port = 0 akka.remote.netty.tcp.port = 0
""" """
} }

View file

@ -19,7 +19,7 @@ object DistributedDataDocSpec {
val config = val config =
""" """
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "cluster"
akka.remote.netty.tcp.port = 0 akka.remote.netty.tcp.port = 0
#//#serializer-config #//#serializer-config

View file

@ -19,7 +19,7 @@ object RemoteDeploymentDocSpec {
} }
class RemoteDeploymentDocSpec extends AkkaSpec(""" class RemoteDeploymentDocSpec extends AkkaSpec("""
akka.actor.provider = "akka.remote.RemoteActorRefProvider" akka.actor.provider = remote
akka.remote.netty.tcp { akka.remote.netty.tcp {
port = 0 port = 0
} }

View file

@ -27,7 +27,7 @@ to your ``application.conf`` file::
akka { akka {
actor { actor {
provider = "akka.remote.RemoteActorRefProvider" provider = remote
} }
remote { remote {
enabled-transports = ["akka.remote.netty.tcp"] enabled-transports = ["akka.remote.netty.tcp"]
@ -40,7 +40,7 @@ to your ``application.conf`` file::
As you can see in the example above there are four things you need to add to get started: As you can see in the example above there are four things you need to add to get started:
* Change provider from ``akka.actor.LocalActorRefProvider`` to ``akka.remote.RemoteActorRefProvider`` * Change provider from ``local`` to ``remote``
* Add host name - the machine you want to run the actor system on; this host * Add host name - the machine you want to run the actor system on; this host
name is exactly what is passed to remote systems in order to identify this name is exactly what is passed to remote systems in order to identify this
system and consequently used for connecting back to this system if need be, system and consequently used for connecting back to this system if need be,

View file

@ -196,7 +196,7 @@ object MultiNodeSpec {
require(selfIndex >= 0 && selfIndex < maxNodes, "multinode.index is out of bounds: " + selfIndex) require(selfIndex >= 0 && selfIndex < maxNodes, "multinode.index is out of bounds: " + selfIndex)
private[testkit] val nodeConfig = mapToConfig(Map( private[testkit] val nodeConfig = mapToConfig(Map(
"akka.actor.provider" "akka.remote.RemoteActorRefProvider", "akka.actor.provider" "remote",
"akka.remote.artery.hostname" selfName, "akka.remote.artery.hostname" selfName,
"akka.remote.netty.tcp.hostname" selfName, "akka.remote.netty.tcp.hostname" selfName,
"akka.remote.netty.tcp.port" selfPort, "akka.remote.netty.tcp.port" selfPort,

View file

@ -15,7 +15,7 @@ object PersistencePluginProxySpec {
""" """
akka { akka {
actor { actor {
provider = "akka.remote.RemoteActorRefProvider" provider = remote
} }
persistence { persistence {
journal { journal {

View file

@ -13,7 +13,7 @@ object SharedLeveldbJournalSpec {
""" """
akka { akka {
actor { actor {
provider = "akka.remote.RemoteActorRefProvider" provider = remote
} }
persistence { persistence {
journal { journal {

View file

@ -44,7 +44,7 @@ object SerializerSpecConfigs {
""" """
akka { akka {
actor { actor {
provider = "akka.remote.RemoteActorRefProvider" provider = remote
} }
remote { remote {
enabled-transports = ["akka.remote.netty.tcp"] enabled-transports = ["akka.remote.netty.tcp"]

View file

@ -37,7 +37,7 @@ object AeronStreamConsistencySpec extends MultiNodeConfig {
akka { akka {
loglevel = INFO loglevel = INFO
actor { actor {
provider = "akka.remote.RemoteActorRefProvider" provider = remote
} }
remote.artery.enabled = off remote.artery.enabled = off
} }

View file

@ -48,7 +48,7 @@ object AeronStreamLatencySpec extends MultiNodeConfig {
loglevel = ERROR loglevel = ERROR
testconductor.barrier-timeout = ${barrierTimeout.toSeconds}s testconductor.barrier-timeout = ${barrierTimeout.toSeconds}s
actor { actor {
provider = "akka.remote.RemoteActorRefProvider" provider = remote
serialize-creators = false serialize-creators = false
serialize-messages = false serialize-messages = false
} }

View file

@ -41,7 +41,7 @@ object AeronStreamMaxThroughputSpec extends MultiNodeConfig {
loglevel = ERROR loglevel = ERROR
testconductor.barrier-timeout = ${barrierTimeout.toSeconds}s testconductor.barrier-timeout = ${barrierTimeout.toSeconds}s
actor { actor {
provider = "akka.remote.RemoteActorRefProvider" provider = remote
serialize-creators = false serialize-creators = false
serialize-messages = false serialize-messages = false
} }

View file

@ -24,7 +24,7 @@ object HandshakeRestartReceiverSpec extends MultiNodeConfig {
ConfigFactory.parseString(s""" ConfigFactory.parseString(s"""
akka { akka {
loglevel = INFO loglevel = INFO
actor.provider = "akka.remote.RemoteActorRefProvider" actor.provider = remote
remote.artery { remote.artery {
enabled = on enabled = on
} }

View file

@ -47,7 +47,7 @@ object LatencySpec extends MultiNodeConfig {
loggers = ["akka.event.Logging$$DefaultLogger"] loggers = ["akka.event.Logging$$DefaultLogger"]
testconductor.barrier-timeout = ${barrierTimeout.toSeconds}s testconductor.barrier-timeout = ${barrierTimeout.toSeconds}s
actor { actor {
provider = "akka.remote.RemoteActorRefProvider" provider = remote
serialize-creators = false serialize-creators = false
serialize-messages = false serialize-messages = false
} }

View file

@ -36,7 +36,7 @@ object MaxThroughputSpec extends MultiNodeConfig {
loggers = ["akka.event.Logging$$DefaultLogger"] loggers = ["akka.event.Logging$$DefaultLogger"]
testconductor.barrier-timeout = ${barrierTimeout.toSeconds}s testconductor.barrier-timeout = ${barrierTimeout.toSeconds}s
actor { actor {
provider = "akka.remote.RemoteActorRefProvider" provider = remote
serialize-creators = false serialize-creators = false
serialize-messages = false serialize-messages = false

View file

@ -14,7 +14,7 @@ object BarrierSpec {
final case class Failed(ref: ActorRef, thr: Throwable) final case class Failed(ref: ActorRef, thr: Throwable)
val config = """ val config = """
akka.testconductor.barrier-timeout = 5s akka.testconductor.barrier-timeout = 5s
akka.actor.provider = akka.remote.RemoteActorRefProvider akka.actor.provider = remote
akka.actor.debug.fsm = on akka.actor.debug.fsm = on
akka.actor.debug.lifecycle = on akka.actor.debug.lifecycle = on
""" """

View file

@ -13,7 +13,7 @@ import java.net.InetAddress
object ControllerSpec { object ControllerSpec {
val config = """ val config = """
akka.testconductor.barrier-timeout = 5s akka.testconductor.barrier-timeout = 5s
akka.actor.provider = akka.remote.RemoteActorRefProvider akka.actor.provider = remote
akka.actor.debug.fsm = on akka.actor.debug.fsm = on
akka.actor.debug.lifecycle = on akka.actor.debug.lifecycle = on
""" """

View file

@ -20,7 +20,7 @@ object ActorsLeakSpec {
val config = ConfigFactory.parseString( val config = ConfigFactory.parseString(
""" """
| akka.actor.provider = "akka.remote.RemoteActorRefProvider" | akka.actor.provider = remote
| #akka.loglevel = DEBUG | #akka.loglevel = DEBUG
| akka.remote.netty.tcp.applied-adapters = ["trttl"] | akka.remote.netty.tcp.applied-adapters = ["trttl"]
| #akka.remote.log-lifecycle-events = on | #akka.remote.log-lifecycle-events = on

View file

@ -32,7 +32,7 @@ class DaemonicSpec extends AkkaSpec {
// create a separate actor system that we can check the threads for // create a separate actor system that we can check the threads for
val daemonicSystem = ActorSystem("daemonic", ConfigFactory.parseString(""" val daemonicSystem = ActorSystem("daemonic", ConfigFactory.parseString("""
akka.daemonic = on akka.daemonic = on
akka.actor.provider = "akka.remote.RemoteActorRefProvider" akka.actor.provider = remote
akka.remote.netty.tcp.transport-class = "akka.remote.transport.netty.NettyTransport" akka.remote.netty.tcp.transport-class = "akka.remote.transport.netty.NettyTransport"
akka.remote.netty.tcp.port = 0 akka.remote.netty.tcp.port = 0
akka.log-dead-letters-during-shutdown = off akka.log-dead-letters-during-shutdown = off

View file

@ -25,7 +25,7 @@ object LogSourceSpec {
class LogSourceSpec extends AkkaSpec( class LogSourceSpec extends AkkaSpec(
""" """
akka.loglevel = INFO akka.loglevel = INFO
akka.actor.provider = "akka.remote.RemoteActorRefProvider" akka.actor.provider = remote
akka.remote.netty.tcp.port = 0 akka.remote.netty.tcp.port = 0
""") { """) {

View file

@ -4,7 +4,7 @@ import akka.actor.ActorMailboxSpec
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
class RemoteActorMailboxSpec extends ActorMailboxSpec( class RemoteActorMailboxSpec extends ActorMailboxSpec(
ConfigFactory.parseString("""akka.actor.provider = "akka.remote.RemoteActorRefProvider""""). ConfigFactory.parseString("""akka.actor.provider = remote""").
withFallback(ActorMailboxSpec.mailboxConf)) { withFallback(ActorMailboxSpec.mailboxConf)) {
} }

View file

@ -13,7 +13,7 @@ import akka.remote.transport.netty.{ NettyTransportSettings, SSLSettings }
class RemoteConfigSpec extends AkkaSpec( class RemoteConfigSpec extends AkkaSpec(
""" """
akka.actor.provider = "akka.remote.RemoteActorRefProvider" akka.actor.provider = remote
akka.remote.netty.tcp.port = 0 akka.remote.netty.tcp.port = 0
""") { """) {

View file

@ -10,7 +10,7 @@ import akka.routing.ConsistentRoutee
import akka.routing.ConsistentHash import akka.routing.ConsistentHash
class RemoteConsistentHashingRouterSpec extends AkkaSpec(""" class RemoteConsistentHashingRouterSpec extends AkkaSpec("""
akka.actor.provider = "akka.remote.RemoteActorRefProvider" """) { akka.actor.provider = remote """) {
"ConsistentHashingGroup" must { "ConsistentHashingGroup" must {

View file

@ -14,7 +14,7 @@ import akka.event.Logging.Warning
class RemoteDeathWatchSpec extends AkkaSpec(ConfigFactory.parseString(""" class RemoteDeathWatchSpec extends AkkaSpec(ConfigFactory.parseString("""
akka { akka {
actor { actor {
provider = "akka.remote.RemoteActorRefProvider" provider = remote
deployment { deployment {
/watchers.remote = "akka.tcp://other@localhost:2666" /watchers.remote = "akka.tcp://other@localhost:2666"
} }

View file

@ -11,7 +11,7 @@ import akka.ConfigurationException
object RemoteDeployerSpec { object RemoteDeployerSpec {
val deployerConf = ConfigFactory.parseString(""" val deployerConf = ConfigFactory.parseString("""
akka.actor.provider = "akka.remote.RemoteActorRefProvider" akka.actor.provider = remote
akka.actor.deployment { akka.actor.deployment {
/service2 { /service2 {
router = round-robin-pool router = round-robin-pool

View file

@ -24,7 +24,7 @@ class RemoteInitErrorSpec extends FlatSpec with Matchers {
""" """
akka { akka {
actor { actor {
provider = "akka.remote.RemoteActorRefProvider" provider = remote
} }
remote { remote {
enabled-transports = ["akka.remote.netty.tcp"] enabled-transports = ["akka.remote.netty.tcp"]

View file

@ -21,7 +21,7 @@ object RemoteRouterSpec {
} }
class RemoteRouterSpec extends AkkaSpec(""" class RemoteRouterSpec extends AkkaSpec("""
akka.actor.provider = "akka.remote.RemoteActorRefProvider" akka.actor.provider = remote
akka.remote.netty.tcp { akka.remote.netty.tcp {
hostname = localhost hostname = localhost
port = 0 port = 0

View file

@ -66,7 +66,7 @@ class RemoteWatcherSpec extends AkkaSpec(
"""akka { """akka {
loglevel = INFO loglevel = INFO
log-dead-letters-during-shutdown = false log-dead-letters-during-shutdown = false
actor.provider = "akka.remote.RemoteActorRefProvider" actor.provider = remote
remote.netty.tcp { remote.netty.tcp {
hostname = localhost hostname = localhost
port = 0 port = 0

View file

@ -79,7 +79,7 @@ object RemotingSpec {
} }
akka { akka {
actor.provider = "akka.remote.RemoteActorRefProvider" actor.provider = remote
remote { remote {
retry-gate-closed-for = 1 s retry-gate-closed-for = 1 s

View file

@ -26,7 +26,7 @@ object Configuration {
private val keyStore = getClass.getClassLoader.getResource("keystore").getPath private val keyStore = getClass.getClassLoader.getResource("keystore").getPath
private val conf = """ private val conf = """
akka { akka {
actor.provider = "akka.remote.RemoteActorRefProvider" actor.provider = remote
test { test {
single-expect-default = 10s single-expect-default = 10s
filter-leeway = 10s filter-leeway = 10s

View file

@ -12,7 +12,7 @@ import scala.concurrent.duration._
object TypedActorRemoteDeploySpec { object TypedActorRemoteDeploySpec {
val conf = ConfigFactory.parseString(""" val conf = ConfigFactory.parseString("""
akka.actor.provider = "akka.remote.RemoteActorRefProvider" akka.actor.provider = remote
akka.remote.netty.tcp.port = 0 akka.remote.netty.tcp.port = 0
""") """)

View file

@ -60,7 +60,7 @@ object UntrustedSpec {
} }
class UntrustedSpec extends AkkaSpec(""" class UntrustedSpec extends AkkaSpec("""
akka.actor.provider = akka.remote.RemoteActorRefProvider akka.actor.provider = remote
akka.remote.untrusted-mode = on akka.remote.untrusted-mode = on
akka.remote.trusted-selection-paths = ["/user/receptionist", ] akka.remote.trusted-selection-paths = ["/user/receptionist", ]
akka.remote.netty.tcp.port = 0 akka.remote.netty.tcp.port = 0
@ -70,7 +70,7 @@ akka.loglevel = DEBUG
import UntrustedSpec._ import UntrustedSpec._
val client = ActorSystem("UntrustedSpec-client", ConfigFactory.parseString(""" val client = ActorSystem("UntrustedSpec-client", ConfigFactory.parseString("""
akka.actor.provider = akka.remote.RemoteActorRefProvider akka.actor.provider = remote
akka.remote.netty.tcp.port = 0 akka.remote.netty.tcp.port = 0
""")) """))
val addr = system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress val addr = system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress

View file

@ -18,7 +18,7 @@ object HandshakeFailureSpec {
val commonConfig = ConfigFactory.parseString(s""" val commonConfig = ConfigFactory.parseString(s"""
akka { akka {
actor.provider = "akka.remote.RemoteActorRefProvider" actor.provider = remote
remote.artery.enabled = on remote.artery.enabled = on
remote.artery.hostname = localhost remote.artery.hostname = localhost
remote.artery.port = 0 remote.artery.port = 0

View file

@ -18,7 +18,7 @@ object HandshakeRetrySpec {
val commonConfig = ConfigFactory.parseString(s""" val commonConfig = ConfigFactory.parseString(s"""
akka { akka {
actor.provider = "akka.remote.RemoteActorRefProvider" actor.provider = remote
remote.artery.enabled = on remote.artery.enabled = on
remote.artery.hostname = localhost remote.artery.hostname = localhost
remote.artery.port = 0 remote.artery.port = 0

View file

@ -32,7 +32,7 @@ class LargeMessagesStreamSpec extends WordSpec with ShouldMatchers with ScalaFut
akka { akka {
loglevel = ERROR loglevel = ERROR
actor { actor {
provider = "akka.remote.RemoteActorRefProvider" provider = remote
} }
remote.artery { remote.artery {
enabled = on enabled = on

View file

@ -17,7 +17,7 @@ object RemoteActorRefProviderSpec {
val config = ConfigFactory.parseString(s""" val config = ConfigFactory.parseString(s"""
akka { akka {
actor.provider = "akka.remote.RemoteActorRefProvider" actor.provider = remote
remote.artery.enabled = on remote.artery.enabled = on
remote.artery.hostname = localhost remote.artery.hostname = localhost
remote.artery.port = 0 remote.artery.port = 0

View file

@ -20,7 +20,7 @@ object RemoteDeathWatchSpec {
val config = ConfigFactory.parseString(s""" val config = ConfigFactory.parseString(s"""
akka { akka {
actor { actor {
provider = "akka.remote.RemoteActorRefProvider" provider = remote
deployment { deployment {
/watchers.remote = "artery://other@localhost:$otherPort" /watchers.remote = "artery://other@localhost:$otherPort"
} }

View file

@ -12,7 +12,7 @@ import akka.remote.RemoteScope
object RemoteDeployerSpec { object RemoteDeployerSpec {
val deployerConf = ConfigFactory.parseString(""" val deployerConf = ConfigFactory.parseString("""
akka.actor.provider = "akka.remote.RemoteActorRefProvider" akka.actor.provider = remote
akka.actor.deployment { akka.actor.deployment {
/service2 { /service2 {
router = round-robin-pool router = round-robin-pool

View file

@ -34,7 +34,7 @@ object RemoteDeploymentSpec {
class RemoteDeploymentSpec extends AkkaSpec(""" class RemoteDeploymentSpec extends AkkaSpec("""
#akka.loglevel=DEBUG #akka.loglevel=DEBUG
akka.actor.provider = "akka.remote.RemoteActorRefProvider" akka.actor.provider = remote
akka.remote.artery.enabled = on akka.remote.artery.enabled = on
akka.remote.artery.hostname = localhost akka.remote.artery.hostname = localhost
akka.remote.artery.port = 0 akka.remote.artery.port = 0

View file

@ -23,7 +23,7 @@ object RemoteRouterSpec {
class RemoteRouterSpec extends AkkaSpec(""" class RemoteRouterSpec extends AkkaSpec("""
akka.loglevel=DEBUG akka.loglevel=DEBUG
akka.actor.provider = "akka.remote.RemoteActorRefProvider" akka.actor.provider = remote
akka.remote.artery.enabled = on akka.remote.artery.enabled = on
akka.remote.artery.hostname = localhost akka.remote.artery.hostname = localhost
akka.remote.artery.port = 0 akka.remote.artery.port = 0

View file

@ -13,7 +13,7 @@ object RemoteSendConsistencySpec {
val config = ConfigFactory.parseString(s""" val config = ConfigFactory.parseString(s"""
akka { akka {
actor.provider = "akka.remote.RemoteActorRefProvider" actor.provider = remote
remote.artery.enabled = on remote.artery.enabled = on
remote.artery.hostname = localhost remote.artery.hostname = localhost
remote.artery.port = 0 remote.artery.port = 0

View file

@ -67,7 +67,7 @@ class RemoteWatcherSpec extends AkkaSpec(
"""akka { """akka {
loglevel = INFO loglevel = INFO
log-dead-letters-during-shutdown = false log-dead-letters-during-shutdown = false
actor.provider = "akka.remote.RemoteActorRefProvider" actor.provider = remote
remote.artery.enabled = on remote.artery.enabled = on
remote.artery.hostname = localhost remote.artery.hostname = localhost
remote.artery.port = 0 remote.artery.port = 0

View file

@ -14,7 +14,7 @@ object SerializationErrorSpec {
val config = ConfigFactory.parseString(s""" val config = ConfigFactory.parseString(s"""
akka { akka {
actor.provider = "akka.remote.RemoteActorRefProvider" actor.provider = remote
remote.artery.enabled = on remote.artery.enabled = on
remote.artery.hostname = localhost remote.artery.hostname = localhost
remote.artery.port = 0 remote.artery.port = 0

View file

@ -37,7 +37,7 @@ object SystemMessageDeliverySpec {
val config = ConfigFactory.parseString(s""" val config = ConfigFactory.parseString(s"""
akka { akka {
actor.provider = "akka.remote.RemoteActorRefProvider" actor.provider = remote
remote.artery.enabled = on remote.artery.enabled = on
remote.artery.hostname = localhost remote.artery.hostname = localhost
remote.artery.port = 0 remote.artery.port = 0

View file

@ -60,7 +60,7 @@ object UntrustedSpec {
} }
class UntrustedSpec extends AkkaSpec(""" class UntrustedSpec extends AkkaSpec("""
akka.actor.provider = akka.remote.RemoteActorRefProvider akka.actor.provider = remote
akka.remote.untrusted-mode = on akka.remote.untrusted-mode = on
akka.remote.trusted-selection-paths = ["/user/receptionist", ] akka.remote.trusted-selection-paths = ["/user/receptionist", ]
akka.remote.artery.enabled = on akka.remote.artery.enabled = on
@ -72,7 +72,7 @@ class UntrustedSpec extends AkkaSpec("""
import UntrustedSpec._ import UntrustedSpec._
val client = ActorSystem("UntrustedSpec-client", ConfigFactory.parseString(""" val client = ActorSystem("UntrustedSpec-client", ConfigFactory.parseString("""
akka.actor.provider = akka.remote.RemoteActorRefProvider akka.actor.provider = remote
akka.remote.artery.enabled = on akka.remote.artery.enabled = on
akka.remote.artery.hostname = localhost akka.remote.artery.hostname = localhost
akka.remote.artery.port = 0 akka.remote.artery.port = 0

View file

@ -30,7 +30,7 @@ object AkkaProtocolSpec {
} }
class AkkaProtocolSpec extends AkkaSpec("""akka.actor.provider = "akka.remote.RemoteActorRefProvider" """) with ImplicitSender { class AkkaProtocolSpec extends AkkaSpec("""akka.actor.provider = remote """) with ImplicitSender {
val conf = ConfigFactory.parseString( val conf = ConfigFactory.parseString(
""" """

View file

@ -15,7 +15,7 @@ object AkkaProtocolStressTest {
akka { akka {
#loglevel = DEBUG #loglevel = DEBUG
actor.serialize-messages = off actor.serialize-messages = off
actor.provider = "akka.remote.RemoteActorRefProvider" actor.provider = remote
remote.log-remote-lifecycle-events = on remote.log-remote-lifecycle-events = on

View file

@ -11,7 +11,7 @@ import akka.remote.RemoteActorRefProvider
import akka.remote.transport.TestTransport.{ DisassociateAttempt, WriteAttempt, ListenAttempt, AssociateAttempt } import akka.remote.transport.TestTransport.{ DisassociateAttempt, WriteAttempt, ListenAttempt, AssociateAttempt }
abstract class GenericTransportSpec(withAkkaProtocol: Boolean = false) abstract class GenericTransportSpec(withAkkaProtocol: Boolean = false)
extends AkkaSpec("""akka.actor.provider = "akka.remote.RemoteActorRefProvider" """) extends AkkaSpec("""akka.actor.provider = remote """)
with DefaultTimeout with ImplicitSender { with DefaultTimeout with ImplicitSender {
def transportName: String def transportName: String

View file

@ -36,7 +36,7 @@ object SystemMessageDeliveryStressTest {
val baseConfig: Config = ConfigFactory parseString (s""" val baseConfig: Config = ConfigFactory parseString (s"""
akka { akka {
#loglevel = DEBUG #loglevel = DEBUG
actor.provider = "akka.remote.RemoteActorRefProvider" actor.provider = remote
actor.serialize-messages = off actor.serialize-messages = off
remote.log-remote-lifecycle-events = on remote.log-remote-lifecycle-events = on

Some files were not shown because too many files have changed in this diff Show more