Make artery the default remoting (#26772)

This commit is contained in:
Christopher Batey 2019-05-01 08:12:09 +01:00 committed by GitHub
parent ac91c797a5
commit efc3adc08d
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
158 changed files with 977 additions and 999 deletions

View file

@ -18,7 +18,7 @@ object ActorRefSerializationSpec {
serialize-messages = off
allow-java-serialization = true
}
akka.remote.netty.tcp.port = 0
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
""")

View file

@ -15,7 +15,7 @@ import scala.Vector
class WeightedRouteesSpec extends AkkaSpec(ConfigFactory.parseString("""
akka.actor.provider = "cluster"
akka.remote.netty.tcp.port = 0
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
""")) {

View file

@ -28,7 +28,7 @@ public class ClusterShardingPersistenceTest extends JUnitSuite {
public static final Config config =
ConfigFactory.parseString(
"akka.actor.provider = cluster \n"
+ "akka.remote.netty.tcp.port = 0 \n"
+ "akka.remote.classic.netty.tcp.port = 0 \n"
+ "akka.remote.artery.canonical.port = 0 \n"
+ "akka.remote.artery.canonical.hostname = 127.0.0.1 \n"
+ "akka.persistence.journal.plugin = \"akka.persistence.journal.inmem\" \n");

View file

@ -25,7 +25,7 @@ public class HelloWorldEventSourcedEntityExampleTest extends JUnitSuite {
public static final Config config =
ConfigFactory.parseString(
"akka.actor.provider = cluster \n"
+ "akka.remote.netty.tcp.port = 0 \n"
+ "akka.remote.classic.netty.tcp.port = 0 \n"
+ "akka.remote.artery.canonical.port = 0 \n"
+ "akka.remote.artery.canonical.hostname = 127.0.0.1 \n"
+ "akka.persistence.journal.plugin = \"akka.persistence.journal.inmem\" \n");

View file

@ -26,7 +26,7 @@ object JoinConfig {
akka.cluster.configuration-compatibility-check.enforce-on-join = on
akka.cluster.jmx.enabled = off
akka.coordinated-shutdown.terminate-actor-system = on
akka.remote.netty.tcp.port = 0
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
""").withFallback(AkkaSpec.testConf)

View file

@ -40,7 +40,7 @@ object ClusterShardingPersistenceSpec {
akka.actor.provider = cluster
akka.remote.netty.tcp.port = 0
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
akka.remote.artery.canonical.hostname = 127.0.0.1

View file

@ -38,7 +38,7 @@ object ClusterShardingSpec {
akka.actor.provider = cluster
// akka.loglevel = debug
akka.remote.netty.tcp.port = 0
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
akka.remote.artery.canonical.hostname = 127.0.0.1

View file

@ -16,7 +16,7 @@ object HelloWorldEventSourcedEntityExampleSpec {
val config = ConfigFactory.parseString("""
akka.actor.provider = cluster
akka.remote.netty.tcp.port = 0
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
akka.remote.artery.canonical.hostname = 127.0.0.1

View file

@ -30,7 +30,7 @@ object ClusterShardingInternalsSpec {
class ClusterShardingInternalsSpec extends AkkaSpec("""
|akka.actor.provider = cluster
|akka.remote.netty.tcp.port = 0
|akka.remote.classic.netty.tcp.port = 0
|akka.remote.artery.canonical.port = 0
|""".stripMargin) with MockitoSugar {
import ClusterShardingInternalsSpec._

View file

@ -19,7 +19,7 @@ object ClusterShardingLeaseSpec {
akka.loglevel = DEBUG
#akka.loggers = ["akka.testkit.SilenceAllTestEventListener"]
akka.actor.provider = "cluster"
akka.remote.netty.tcp.port = 0
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
akka.cluster.sharding {
use-lease = "test-lease"

View file

@ -20,7 +20,7 @@ object ConcurrentStartupShardingSpec {
val config =
"""
akka.actor.provider = "cluster"
akka.remote.netty.tcp.port = 0
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
akka.log-dead-letters = off
akka.log-dead-letters-during-shutdown = off

View file

@ -23,7 +23,7 @@ object CoordinatedShutdownShardingSpec {
"""
akka.loggers = ["akka.testkit.SilenceAllTestEventListener"]
akka.actor.provider = "cluster"
akka.remote.netty.tcp.port = 0
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
"""

View file

@ -14,7 +14,7 @@ object GetShardTypeNamesSpec {
"""
akka.loglevel = INFO
akka.actor.provider = "cluster"
akka.remote.netty.tcp.port = 0
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
"""

View file

@ -18,7 +18,7 @@ object InactiveEntityPassivationSpec {
val config = ConfigFactory.parseString("""
akka.loglevel = INFO
akka.actor.provider = "cluster"
akka.remote.netty.tcp.port = 0
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
akka.actor.serialize-messages = off
""")

View file

@ -25,7 +25,7 @@ class JoinConfigCompatCheckShardingSpec extends AkkaSpec() {
ConfigFactory.parseString("""
akka.actor.provider = "cluster"
akka.coordinated-shutdown.terminate-actor-system = on
akka.remote.netty.tcp.port = 0
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
""")

View file

@ -14,7 +14,7 @@ import scala.concurrent.duration.FiniteDuration
object ProxyShardingSpec {
val config = """
akka.actor.provider = "cluster"
akka.remote.netty.tcp.port = 0
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
"""
}

View file

@ -29,7 +29,7 @@ object RemoveInternalClusterShardingDataSpec {
val config = """
akka.loglevel = INFO
akka.actor.provider = "cluster"
akka.remote.netty.tcp.port = 0
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
akka.persistence.journal.plugin = "akka.persistence.journal.leveldb"
akka.persistence.journal.leveldb {

View file

@ -22,7 +22,7 @@ object ShardSpec {
"""
akka.loglevel = INFO
akka.actor.provider = "cluster"
akka.remote.netty.tcp.port = 0
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
test-lease {
lease-class = akka.cluster.TestLease

View file

@ -497,7 +497,7 @@ class ClusterClientSpec extends MultiNodeSpec(ClusterClientSpec) with STMultiNod
system.name,
ConfigFactory.parseString(s"""
akka.remote.artery.canonical.port=$port
akka.remote.netty.tcp.port=$port
akka.remote.classic.netty.tcp.port=$port
""").withFallback(system.settings.config))
Cluster(sys2).join(Cluster(sys2).selfAddress)
val service2 = sys2.actorOf(Props(classOf[TestService], testActor), "service2")

View file

@ -141,7 +141,7 @@ class DistributedPubSubRestartSpec
val port = Cluster(system).selfAddress.port.get
val config = ConfigFactory.parseString(s"""
akka.remote.artery.canonical.port=$port
akka.remote.netty.tcp.port=$port
akka.remote.classic.netty.tcp.port=$port
""").withFallback(system.settings.config)
ActorSystem(system.name, config)

View file

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

View file

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

View file

@ -14,7 +14,7 @@ object DistributedPubSubMediatorDeadLettersSpec {
s"""
akka.loglevel = INFO
akka.actor.provider = "cluster"
akka.remote.netty.tcp.port=0
akka.remote.classic.netty.tcp.port=0
akka.remote.artery.canonical.port=0
akka.remote.log-remote-lifecycle-events = off
akka.cluster.pub-sub.send-to-dead-letters-when-no-subscribers = $sendToDeadLettersWhenNoSubscribers

View file

@ -21,7 +21,7 @@ object DistributedPubSubMediatorRouterSpec {
akka.loglevel = INFO
akka.actor.provider = "cluster"
akka.actor.serialize-messages = off
akka.remote.netty.tcp.port=0
akka.remote.classic.netty.tcp.port=0
akka.remote.artery.canonical.port=0
akka.remote.log-remote-lifecycle-events = off
akka.cluster.pub-sub.routing-logic = $routingLogic

View file

@ -102,7 +102,7 @@ class ClusterSingletonRestart2Spec extends AkkaSpec("""
val sys4Config =
ConfigFactory.parseString(s"""
akka.remote.artery.canonical.port=$sys2port
akka.remote.netty.tcp.port=$sys2port
akka.remote.classic.netty.tcp.port=$sys2port
""").withFallback(system.settings.config)
ActorSystem(system.name, sys4Config)

View file

@ -77,7 +77,7 @@ class ClusterSingletonRestartSpec extends AkkaSpec("""
val sys3Config =
ConfigFactory.parseString(s"""
akka.remote.artery.canonical.port=$sys1port
akka.remote.netty.tcp.port=$sys1port
akka.remote.classic.netty.tcp.port=$sys1port
""").withFallback(system.settings.config)
ActorSystem(system.name, sys3Config)

View file

@ -191,7 +191,7 @@ public class ReplicatorTest extends JUnitSuite {
static Config config =
ConfigFactory.parseString(
"akka.actor.provider = cluster \n"
+ "akka.remote.netty.tcp.port = 0 \n"
+ "akka.remote.classic.netty.tcp.port = 0 \n"
+ "akka.remote.artery.canonical.port = 0 \n"
+ "akka.remote.artery.canonical.hostname = 127.0.0.1 \n");

View file

@ -21,7 +21,7 @@ public class ClusterApiTest extends JUnitSuite {
Config config =
ConfigFactory.parseString(
"akka.actor.provider = cluster \n"
+ "akka.remote.netty.tcp.port = 0 \n"
+ "akka.remote.classic.netty.tcp.port = 0 \n"
+ "akka.remote.artery.canonical.port = 0 \n"
+ "akka.remote.artery.canonical.hostname = 127.0.0.1 \n"
+ "akka.cluster.jmx.multi-mbeans-in-same-jvm = on \n"

View file

@ -31,7 +31,7 @@ public class BasicClusterExampleTest { // extends JUnitSuite {
private Config noPort =
ConfigFactory.parseString(
" akka.remote.netty.tcp.port = 0 \n"
" akka.remote.classic.netty.tcp.port = 0 \n"
+ " akka.remote.artery.canonical.port = 0 \n");
// @Test

View file

@ -28,7 +28,7 @@ object ReplicatorSpec {
val config = ConfigFactory.parseString("""
akka.actor.provider = "cluster"
akka.remote.netty.tcp.port = 0
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
akka.remote.artery.canonical.hostname = 127.0.0.1
""")

View file

@ -27,7 +27,7 @@ class ActorSystemSpec extends WordSpec with Matchers with BeforeAndAfterAll with
override implicit val patienceConfig = PatienceConfig(1.second)
val config = ConfigFactory.parseString("""
akka.actor.provider = cluster
akka.remote.netty.tcp.port = 0
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
akka.remote.artery.canonical.hostname = 127.0.0.1
""")

View file

@ -18,7 +18,7 @@ object ClusterApiSpec {
val config =
ConfigFactory.parseString("""
akka.actor.provider = cluster
akka.remote.netty.tcp.port = 0
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
akka.remote.artery.canonical.hostname = 127.0.0.1
akka.cluster.jmx.multi-mbeans-in-same-jvm = on

View file

@ -37,7 +37,7 @@ object ClusterSingletonApiSpec {
"akka.cluster.typed.ClusterSingletonApiSpec$$Perish$$" = test
}
}
akka.remote.netty.tcp.port = 0
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
akka.remote.artery.canonical.hostname = 127.0.0.1
akka.cluster.jmx.multi-mbeans-in-same-jvm = on

View file

@ -15,7 +15,7 @@ import org.scalatest.WordSpecLike
object ClusterSingletonPersistenceSpec {
val config = ConfigFactory.parseString("""
akka.actor.provider = cluster
akka.remote.netty.tcp.port = 0
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
akka.remote.artery.canonical.hostname = 127.0.0.1

View file

@ -58,8 +58,8 @@ object RemoteContextAskSpec {
"akka.cluster.typed.RemoteContextAskSpec$$Pong$$" = test
}
}
remote.netty.tcp.port = 0
remote.netty.tcp.host = 127.0.0.1
remote.classic.netty.tcp.port = 0
remote.classic.netty.tcp.host = 127.0.0.1
remote.artery {
canonical {
hostname = 127.0.0.1

View file

@ -23,7 +23,7 @@ object RemoteDeployNotAllowedSpec {
warn-about-java-serializer-usage = off
serialize-creators = off
}
remote.netty.tcp.port = 0
remote.classic.netty.tcp.port = 0
remote.artery {
canonical {
hostname = 127.0.0.1

View file

@ -46,7 +46,7 @@ object RemoteMessageSpec {
"akka.cluster.typed.RemoteMessageSpec$$Ping" = test
}
}
remote.netty.tcp.port = 0
remote.classic.netty.tcp.port = 0
remote.artery {
canonical {
hostname = 127.0.0.1

View file

@ -41,8 +41,8 @@ object ClusterReceptionistSpec {
"akka.cluster.typed.internal.receptionist.ClusterReceptionistSpec$$Perish$$" = test
}
}
akka.remote.netty.tcp.port = 0
akka.remote.netty.tcp.host = 127.0.0.1
akka.remote.classic.netty.tcp.port = 0
akka.remote.classic.netty.tcp.host = 127.0.0.1
akka.remote.artery.canonical.port = 0
akka.remote.artery.canonical.hostname = 127.0.0.1
@ -345,7 +345,7 @@ class ClusterReceptionistSpec extends WordSpec with Matchers {
val testKit3 = ActorTestKit(
system1.name,
ConfigFactory.parseString(s"""
akka.remote.netty.tcp.port = ${clusterNode2.selfMember.address.port.get}
akka.remote.classic.netty.tcp.port = ${clusterNode2.selfMember.address.port.get}
akka.remote.artery.canonical.port = ${clusterNode2.selfMember.address.port.get}
# retry joining when existing member removed
akka.cluster.retry-unsuccessful-join-after = 1s
@ -456,7 +456,7 @@ class ClusterReceptionistSpec extends WordSpec with Matchers {
val testKit3 = ActorTestKit(
system1.name,
ConfigFactory.parseString(s"""
akka.remote.netty.tcp.port = ${clusterNode2.selfMember.address.port.get}
akka.remote.classic.netty.tcp.port = ${clusterNode2.selfMember.address.port.get}
akka.remote.artery.canonical.port = ${clusterNode2.selfMember.address.port.get}
""").withFallback(config))

View file

@ -44,7 +44,7 @@ akka {
""")
val configSystem2 = ConfigFactory.parseString(s"""
akka.remote.netty.tcp.port = 0
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
""").withFallback(configSystem1)
}
@ -61,7 +61,7 @@ class BasicClusterConfigSpec extends WordSpec with ScalaFutures with Eventually
val sys1Port = SocketUtil.temporaryLocalPort()
val sys2Port = SocketUtil.temporaryLocalPort()
def config(port: Int) = ConfigFactory.parseString(s"""
akka.remote.netty.tcp.port = $port
akka.remote.classic.netty.tcp.port = $port
akka.cluster.seed-nodes = [ "akka.tcp://ClusterSystem@127.0.0.1:$sys1Port", "akka.tcp://ClusterSystem@127.0.0.1:$sys2Port" ]
""")
@ -97,7 +97,7 @@ akka {
""")
val noPort = ConfigFactory.parseString("""
akka.remote.netty.tcp.port = 0
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
""")

View file

@ -316,6 +316,8 @@ akka {
"user.home", "user.name", "user.dir",
"socksNonProxyHosts", "http.nonProxyHosts", "ftp.nonProxyHosts",
"akka.remote.secure-cookie",
"akka.remote.classic.netty.ssl.security",
# Pre 2.6 path, keep around to avoid sending things misconfigured with old paths
"akka.remote.netty.ssl.security",
"akka.remote.artery.ssl"
]

View file

@ -44,7 +44,7 @@ object MultiDcHeartbeatTakingOverSpecMultiJvmSpec extends MultiNodeConfig {
loggers = ["akka.testkit.TestEventListener"]
loglevel = INFO
remote.log-remote-lifecycle-events = off
remote.classic.log-remote-lifecycle-events = off
cluster {
debug.verbose-heartbeat-logging = off

View file

@ -23,11 +23,12 @@ object MultiDcSplitBrainMultiJvmSpec extends MultiNodeConfig {
val fifth = role("fifth")
commonConfig(
ConfigFactory.parseString("""
ConfigFactory
.parseString("""
akka.loglevel = DEBUG # issue #24955
akka.cluster.debug.verbose-heartbeat-logging = on
akka.cluster.debug.verbose-gossip-logging = on
akka.remote.netty.tcp.connection-timeout = 5 s # speedup in case of connection issue
akka.remote.classic.netty.tcp.connection-timeout = 5 s # speedup in case of connection issue
akka.remote.retry-gate-closed-for = 1 s
akka.cluster.multi-data-center {
failure-detector {
@ -40,7 +41,8 @@ object MultiDcSplitBrainMultiJvmSpec extends MultiNodeConfig {
leader-actions-interval = 1s
auto-down-unreachable-after = 1s
}
""").withFallback(MultiNodeClusterSpec.clusterConfig))
""")
.withFallback(MultiNodeClusterSpec.clusterConfig))
nodeConfig(first, second)(ConfigFactory.parseString("""
akka.cluster.multi-data-center.self-data-center = "dc1"
@ -240,7 +242,7 @@ abstract class MultiDcSplitBrainSpec extends MultiNodeSpec(MultiDcSplitBrainMult
val restartedSystem = ActorSystem(
system.name,
ConfigFactory.parseString(s"""
akka.remote.netty.tcp.port = $port
akka.remote.classic.netty.tcp.port = $port
akka.remote.artery.canonical.port = $port
akka.coordinated-shutdown.terminate-actor-system = on
""").withFallback(system.settings.config))

View file

@ -25,7 +25,7 @@ object NodeChurnMultiJvmSpec extends MultiNodeConfig {
akka.cluster.auto-down-unreachable-after = 1s
akka.cluster.prune-gossip-tombstones-after = 1s
akka.remote.log-frame-size-exceeding = 1200b
akka.remote.artery.advanced {
akka.remote.artery.advanced.aeron {
idle-cpu-level = 1
embedded-media-driver = off
aeron-dir = "target/aeron-NodeChurnSpec"

View file

@ -69,7 +69,7 @@ abstract class QuickRestartSpec
// use the same port
ConfigFactory.parseString(s"""
akka.cluster.roles = [round-$n]
akka.remote.netty.tcp.port = ${Cluster(restartingSystem).selfAddress.port.get}
akka.remote.classic.netty.tcp.port = ${Cluster(restartingSystem).selfAddress.port.get}
akka.remote.artery.canonical.port = ${Cluster(restartingSystem).selfAddress.port.get}
""").withFallback(system.settings.config))
log.info("Restarting node has address: {}", Cluster(restartingSystem).selfUniqueAddress)

View file

@ -57,7 +57,7 @@ abstract class RestartFirstSeedNodeSpec
lazy val restartedSeed1System = ActorSystem(
system.name,
ConfigFactory.parseString(s"""
akka.remote.netty.tcp.port = ${seedNodes.head.port.get}
akka.remote.classic.netty.tcp.port = ${seedNodes.head.port.get}
akka.remote.artery.canonical.port = ${seedNodes.head.port.get}
""").withFallback(system.settings.config))

View file

@ -57,7 +57,7 @@ abstract class RestartNode2SpecSpec
lazy val restartedSeed1System = ActorSystem(
system.name,
ConfigFactory.parseString(s"""
akka.remote.netty.tcp.port = ${seedNodes.head.port.get}
akka.remote.classic.netty.tcp.port = ${seedNodes.head.port.get}
akka.remote.artery.canonical.port = ${seedNodes.head.port.get}
#akka.remote.retry-gate-closed-for = 1s
""").withFallback(system.settings.config))

View file

@ -57,7 +57,7 @@ abstract class RestartNode3Spec
system.name,
ConfigFactory.parseString(s"""
akka.remote.artery.canonical.port = ${secondUniqueAddress.address.port.get}
akka.remote.netty.tcp.port = ${secondUniqueAddress.address.port.get}
akka.remote.classic.netty.tcp.port = ${secondUniqueAddress.address.port.get}
""").withFallback(system.settings.config))
override def afterAll(): Unit = {

View file

@ -76,7 +76,7 @@ abstract class RestartNodeSpec
lazy val restartedSecondSystem = ActorSystem(
system.name,
ConfigFactory.parseString(s"""
akka.remote.netty.tcp.port = ${secondUniqueAddress.address.port.get}
akka.remote.classic.netty.tcp.port = ${secondUniqueAddress.address.port.get}
akka.remote.artery.canonical.port = ${secondUniqueAddress.address.port.get}
""").withFallback(system.settings.config))

View file

@ -32,7 +32,7 @@ object SharedMediaDriverSupport {
def startMediaDriver(config: MultiNodeConfig): Unit = {
val arterySettings = loadArterySettings(config)
if (arterySettings.Enabled) {
val aeronDir = arterySettings.Advanced.AeronDirectoryName
val aeronDir = arterySettings.Advanced.Aeron.AeronDirectoryName
require(aeronDir.nonEmpty, "aeron-dir must be defined")
// Check if the media driver is already started by another multi-node jvm.
@ -63,10 +63,10 @@ object SharedMediaDriverSupport {
if (isDriverInactive(MultiNodeSpec.selfIndex)) {
val driverContext = new MediaDriver.Context
driverContext.aeronDirectoryName(aeronDir)
driverContext.clientLivenessTimeoutNs(arterySettings.Advanced.ClientLivenessTimeout.toNanos)
driverContext.imageLivenessTimeoutNs(arterySettings.Advanced.ImageLivenessTimeout.toNanos)
driverContext.driverTimeoutMs(arterySettings.Advanced.DriverTimeout.toMillis)
val idleCpuLevel = arterySettings.Advanced.IdleCpuLevel
driverContext.clientLivenessTimeoutNs(arterySettings.Advanced.Aeron.ClientLivenessTimeout.toNanos)
driverContext.imageLivenessTimeoutNs(arterySettings.Advanced.Aeron.ImageLivenessTimeout.toNanos)
driverContext.driverTimeoutMs(arterySettings.Advanced.Aeron.DriverTimeout.toMillis)
val idleCpuLevel = arterySettings.Advanced.Aeron.IdleCpuLevel
driverContext
.threadingMode(ThreadingMode.SHARED)
.sharedIdleStrategy(TaskRunner.createIdleStrategy(idleCpuLevel))
@ -97,7 +97,7 @@ object SharedMediaDriverSupport {
driver.close()
try {
if (arterySettings.Advanced.DeleteAeronDirectory) {
if (arterySettings.Advanced.Aeron.DeleteAeronDirectory) {
IoUtil.delete(new File(driver.aeronDirectoryName), false)
}
} catch {

View file

@ -128,7 +128,7 @@ private[cluster] object StressMultiJvmSpec extends MultiNodeConfig {
akka.loglevel = INFO
akka.remote.log-remote-lifecycle-events = off
akka.remote.artery.advanced {
akka.remote.artery.advanced.aeron {
idle-cpu-level = 1
embedded-media-driver = off
aeron-dir = "target/aeron-StressSpec"

View file

@ -36,9 +36,9 @@ object SurviveNetworkInstabilityMultiJvmSpec extends MultiNodeConfig {
commonConfig(
debugConfig(on = false)
.withFallback(ConfigFactory.parseString("""
akka.remote.system-message-buffer-size=100
akka.remote.classic.system-message-buffer-size=100
akka.remote.artery.advanced.system-message-buffer-size=100
akka.remote.netty.tcp.connection-timeout = 10s
akka.remote.classic.netty.tcp.connection-timeout = 10s
"""))
.withFallback(MultiNodeClusterSpec.clusterConfig))

View file

@ -169,7 +169,7 @@ abstract class UnreachableNodeJoinsAgainSpec
}
"""
else s"""
akka.remote.netty.tcp {
akka.remote.classic.netty.tcp {
hostname = ${victimAddress.host.get}
port = ${victimAddress.port.get}
}""")

View file

@ -35,7 +35,7 @@ object ClusterDeployerSpec {
cluster.allow-local-routees = off
}
}
akka.remote.netty.tcp.port = 0
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
""",
ConfigParseOptions.defaults)

View file

@ -24,7 +24,7 @@ import akka.cluster.ClusterSettings.DefaultDataCenter
object ClusterDomainEventPublisherSpec {
val config = """
akka.actor.provider = "cluster"
akka.remote.netty.tcp.port = 0
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
"""
}

View file

@ -17,7 +17,7 @@ object ClusterLogSpec {
}
akka.actor.provider = "cluster"
akka.remote.log-remote-lifecycle-events = off
akka.remote.netty.tcp.port = 0
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
akka.loglevel = "INFO"
akka.loggers = ["akka.testkit.TestEventListener"]

View file

@ -35,7 +35,7 @@ object ClusterSpec {
}
akka.actor.provider = "cluster"
akka.remote.log-remote-lifecycle-events = off
akka.remote.netty.tcp.port = 0
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
"""
@ -118,7 +118,7 @@ class ClusterSpec extends AkkaSpec(ClusterSpec.config) with ImplicitSender {
"ClusterSpec2",
ConfigFactory.parseString("""
akka.actor.provider = "cluster"
akka.remote.netty.tcp.port = 0
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
"""))
try {
@ -152,7 +152,7 @@ class ClusterSpec extends AkkaSpec(ClusterSpec.config) with ImplicitSender {
"ClusterSpec2",
ConfigFactory.parseString("""
akka.actor.provider = "cluster"
akka.remote.netty.tcp.port = 0
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
"""))
try {
@ -182,7 +182,7 @@ class ClusterSpec extends AkkaSpec(ClusterSpec.config) with ImplicitSender {
"ClusterSpec2",
ConfigFactory.parseString("""
akka.actor.provider = "cluster"
akka.remote.netty.tcp.port = 0
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
akka.coordinated-shutdown.terminate-actor-system = on
"""))
@ -220,7 +220,7 @@ class ClusterSpec extends AkkaSpec(ClusterSpec.config) with ImplicitSender {
"ClusterSpec2",
ConfigFactory.parseString("""
akka.actor.provider = "cluster"
akka.remote.netty.tcp.port = 0
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
akka.cluster.min-nr-of-members = 2
"""))
@ -251,7 +251,7 @@ class ClusterSpec extends AkkaSpec(ClusterSpec.config) with ImplicitSender {
"ClusterSpec2",
ConfigFactory.parseString("""
akka.actor.provider = "cluster"
akka.remote.netty.tcp.port = 0
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
akka.coordinated-shutdown.terminate-actor-system = on
"""))
@ -285,7 +285,7 @@ class ClusterSpec extends AkkaSpec(ClusterSpec.config) with ImplicitSender {
"ClusterSpec3",
ConfigFactory.parseString("""
akka.actor.provider = "cluster"
akka.remote.netty.tcp.port = 0
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
akka.coordinated-shutdown.terminate-actor-system = on
akka.cluster.run-coordinated-shutdown-when-down = on
@ -311,7 +311,7 @@ class ClusterSpec extends AkkaSpec(ClusterSpec.config) with ImplicitSender {
"register multiple cluster JMX MBeans with akka.cluster.jmx.multi-mbeans-in-same-jvm = on" in {
def getConfig = (port: Int) => ConfigFactory.parseString(s"""
akka.cluster.jmx.multi-mbeans-in-same-jvm = on
akka.remote.netty.tcp.port = ${port}
akka.remote.classic.netty.tcp.port = ${port}
akka.remote.artery.canonical.port = ${port}
""").withFallback(ConfigFactory.parseString(ClusterSpec.config))

View file

@ -128,7 +128,7 @@ trait ClusterTestKit extends TestKitBase {
actorSystems = actorSystems.filterNot(_ == actorSystem)
val newConfig = ConfigFactory.parseString(s"""
akka.remote.netty.tcp.port = $port
akka.remote.classic.netty.tcp.port = $port
akka.remote.artery.canonical.port = $port
""").withFallback(config)

View file

@ -16,7 +16,7 @@ object JoinConfigCompatCheckerSpec {
ConfigFactory.parseString("""
akka.actor.provider = "cluster"
akka.coordinated-shutdown.terminate-actor-system = on
akka.remote.netty.tcp.port = 0
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
akka.cluster.jmx.multi-mbeans-in-same-jvm = on
""")

View file

@ -17,7 +17,7 @@ object ShutdownAfterJoinSeedNodesSpec {
val config = """
akka.actor.provider = "cluster"
akka.coordinated-shutdown.terminate-actor-system = on
akka.remote.netty.tcp.port = 0
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
akka.cluster {
seed-node-timeout = 2s

View file

@ -16,7 +16,7 @@ object StartupWithOneThreadSpec {
val config = """
akka.actor.provider = "cluster"
akka.actor.creation-timeout = 10s
akka.remote.netty.tcp.port = 0
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
akka.actor.default-dispatcher {

View file

@ -24,7 +24,7 @@ object ClusterRouterSupervisorSpec {
class ClusterRouterSupervisorSpec extends AkkaSpec("""
akka.actor.provider = "cluster"
akka.remote.netty.tcp.port = 0
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
""") {

View file

@ -19,11 +19,16 @@ import akka.actor.ActorRef
import akka.testkit.TestKitExtension
import akka.actor.ActorIdentity
import akka.actor.Identify
import com.typesafe.config.ConfigFactory
object ReliableProxySpec extends MultiNodeConfig {
val local = role("local")
val remote = role("remote")
commonConfig(ConfigFactory.parseString("""
akka.remote.artery.enabled = false
"""))
testTransport(on = true)
}

View file

@ -281,7 +281,7 @@ abstract class DurableDataSpec(multiNodeConfig: DurableDataSpecConfig)
// use the same port
ConfigFactory.parseString(s"""
akka.remote.artery.canonical.port = ${address.port.get}
akka.remote.netty.tcp.port = ${address.port.get}
akka.remote.classic.netty.tcp.port = ${address.port.get}
""").withFallback(system.settings.config))
try {
Cluster(sys2).join(address)

View file

@ -161,7 +161,7 @@ class DurablePruningSpec extends MultiNodeSpec(DurablePruningSpec) with STMultiN
system.name,
ConfigFactory.parseString(s"""
akka.remote.artery.canonical.port = ${address.port.get}
akka.remote.netty.tcp.port = ${address.port.get}
akka.remote.classic.netty.tcp.port = ${address.port.get}
""").withFallback(system.settings.config))
val cluster3 = Cluster(sys3)
val replicator3 = startReplicator(sys3)

View file

@ -51,7 +51,7 @@ class LocalConcurrencySpec(_system: ActorSystem)
"LocalConcurrencySpec",
ConfigFactory.parseString("""
akka.actor.provider = "cluster"
akka.remote.netty.tcp.port=0
akka.remote.classic.netty.tcp.port=0
akka.remote.artery.canonical.port = 0
""")))
}

View file

@ -31,7 +31,7 @@ object LotsOfDataBot {
ports.foreach { port =>
// Override the configuration of the port
val config = ConfigFactory
.parseString("akka.remote.netty.tcp.port=" + port)
.parseString("akka.remote.classic.netty.tcp.port=" + port)
.withFallback(
ConfigFactory.load(ConfigFactory.parseString("""
passive = off

View file

@ -12,7 +12,7 @@ object ReplicatorSettingsSpec {
val config = ConfigFactory.parseString("""
akka.actor.provider = "cluster"
akka.remote.netty.tcp.port = 0
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
akka.remote.artery.canonical.hostname = 127.0.0.1""")
}

View file

@ -130,7 +130,7 @@ object WriteAggregatorSpec {
class WriteAggregatorSpec extends AkkaSpec(s"""
akka.actor.provider = "cluster"
akka.remote.netty.tcp.port = 0
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
akka.cluster.distributed-data.durable.lmdb {
dir = target/WriteAggregatorSpec-${System.currentTimeMillis}-ddata

View file

@ -33,7 +33,7 @@ class ReplicatedDataSerializerSpec
ConfigFactory.parseString("""
akka.loglevel = DEBUG
akka.actor.provider=cluster
akka.remote.netty.tcp.port=0
akka.remote.classic.netty.tcp.port=0
akka.remote.artery.canonical.port = 0
akka.actor {
serialize-messages = off

View file

@ -37,7 +37,7 @@ class ReplicatorMessageSerializerSpec
"ReplicatorMessageSerializerSpec",
ConfigFactory.parseString("""
akka.actor.provider=cluster
akka.remote.netty.tcp.port=0
akka.remote.classic.netty.tcp.port=0
akka.remote.artery.canonical.port = 0
akka.actor {
serialize-messages = off

View file

@ -119,7 +119,7 @@ Observe all the parts you need here:
Most of the cases this is *tcp*.
* `system` is the remote systems name (must match exactly, case-sensitive!)
* `host` is the remote systems IP address or DNS name, and it must match that
systems configuration (i.e. *akka.remote.netty.tcp.hostname*)
systems configuration (i.e. *akka.remote.classic.netty.tcp.hostname*)
* `1234` is the port number on which the remote system is listening for
connections and receiving messages
* `/user/my/actor/hierarchy/path` is the absolute path of the remote actor in
@ -137,10 +137,10 @@ systems network location, e.g. because `host` was configured to be `0.0.0.0`,
`localhost` or a NATed IP address.
If you are running an ActorSystem under a NAT or inside a docker container, make sure to
set *akka.remote.netty.tcp.hostname* and *akka.remote.netty.tcp.port* to the address
set *akka.remote.artery.hostname* and *akka.remote.artery.canonical.port* to the address
it is reachable at from other ActorSystems. If you need to bind your network interface
to a different address - use *akka.remote.netty.tcp.bind-hostname* and
*akka.remote.netty.tcp.bind-port* settings. Also make sure your network is configured
to a different address - use *akka.remote.artery.bind.hostname* and
*akka.remote.artery.bind.port* settings. Also make sure your network is configured
to translate from the address your ActorSystem is reachable at to the address your
ActorSystem network interface is bound to.

View file

@ -24,9 +24,82 @@ If you are still using Scala 2.11 then you must upgrade to 2.12 or 2.13
Actor DSL is a rarely used feature and has been deprecated since `2.5.0`.
Use plain `system.actorOf` instead of the DSL to create Actors if you have been using it.
## Default remoting is now Artery TCP
@ref[Artery TCP](../remoting-artery.md) is now the default remoting implementation.
Classic remoting has been deprecated and will be removed in `2.7.0`.
<a id="classic-to-artery"></a>
### Migrating from classic remoting to Artery
Artery has the same functionality as classic remoting and you should normally only have to change the
configuration to switch.
To switch a full cluster restart is required and any overrides for classic remoting need to be ported to Artery configuration.
Artery defaults to TCP (see @ref:[selected transport](#selecting-a-transport)) which is a good start
when migrating from classic remoting.
The protocol part in the Akka `Address`, for example `"akka.tcp://actorSystemName@10.0.0.1:2552/user/actorName"`
has changed from `akka.tcp` to `akka`. If you have configured or hardcoded any such addresses you have to change
them to `"akka://actorSystemName@10.0.0.1:2552/user/actorName"`. `akka` is used also when TLS is enabled.
One typical place where such address is used is in the `seed-nodes` configuration.
The configuration is different, so you might have to revisit any custom configuration. See the full
@ref:[reference configuration for Artery](../general/configuration.md#config-akka-remote-artery) and
@ref:[reference configuration for classic remoting](../general/configuration.md#config-akka-remote).
Configuration that is likely required to be ported:
* `akka.remote.netty.tcp.hostname` => `akka.remote.artery.canonical.hostname`
* `akka.remote.netty.tcp.port`=> `akka.remote.artery.canonical.port`
One thing to be aware of is that rolling update from classic remoting to Artery is not supported since the protocol
is completely different. It will require a full cluster shutdown and new startup.
If using SSL then `tcp-tls` needs to be enabled and setup. See @ref[Artery docs for SSL](../remoting-artery.md#configuring-ssl-tls-for-akka-remoting)
for how to do this.
### Migration from 2.5.x Artery to 2.6.x Artery
The following defaults have changed:
* `akka.remote.artery.transport` default has changed from `aeron-udp` to `tcp`
The following properties have moved. If you don't adjust these from their defaults no changes are required:
For Aeron-UDP:
* `akka.remote.artery.log-aeron-counters` to `akka.remote.artery.advanced.aeron.log-aeron-counters`
* `akka.remote.artery.advanced.embedded-media-driver` to `akka.remote.artery.advanced.aeron.embedded-media-driver`
* `akka.remote.artery.advanced.aeron-dir` to `akka.remote.artery.advanced.aeron.aeron-dir`
* `akka.remote.artery.advanced.delete-aeron-dir` to `akka.remote.artery.advanced.aeron.aeron-delete-dir`
* `akka.remote.artery.advanced.idle-cpu-level` to `akka.remote.artery.advanced.aeron.idle-cpu-level`
* `akka.remote.artery.advanced.give-up-message-after` to `akka.remote.artery.advanced.aeron.give-up-message-after`
* `akka.remote.artery.advanced.client-liveness-timeout` to `akka.remote.artery.advanced.aeron.client-liveness-timeout`
* `akka.remote.artery.advanced.image-liveless-timeout` to `akka.remote.artery.advanced.aeron.image-liveness-timeout`
* `akka.remote.artery.advanced.driver-timeout` to `akka.remote.artery.advanced.aeron.driver-timeout`
For TCP:
* `akka.remote.artery.advanced.connection-timeout` to `akka.remote.artery.advanced.tcp.connection-timeout`
### Remaining with Classic remoting (not recommended)
Classic remoting is deprecated but can be used in `2.6.` Any configuration under `akka.remote` that is
specific to classic remoting needs to be moved to `akka.remote.classic`. To see which configuration options
are specific to classic search for them in: [`akka-remote/reference.conf`](/akka-remote/src/main/resources/reference.conf)
## Netty UDP has been removed
Classic remoting over UDP has been deprecated since `2.5.0` and now has been removed.
To continue to use UDP configure @ref[Artery UDP](../remoting-artery.md#configuring-ssl-tls-for-akka-remoting) or migrate to Artery TCP.
A full cluster restart is required to change to Artery.
## Cluster Sharding
### Passivate idle entity
The configuration `akka.cluster.sharding.passivate-idle-entity-after` is now enabled by default.
Sharding will passivate entities when they have not received any messages after this duration.
Set to `off` to disable.
Set

View file

@ -1,4 +1,4 @@
# Remoting (codename Artery)
# Artery Remoting
@@@ note
@ -15,7 +15,7 @@ such as [HTTP](https://doc.akka.io/docs/akka-http/current/),
## Dependency
To use Remoting (codename Artery), you must add the following dependency in your project:
To use Artery Remoting, you must add the following dependency in your project:
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
@ -37,7 +37,6 @@ akka {
}
remote {
artery {
enabled = on
transport = tcp # See Selecting a transport below
canonical.hostname = "127.0.0.1"
canonical.port = 25520
@ -126,26 +125,7 @@ officially supported. If you're on a Big Endian processor, such as Sparc, it is
## Migrating from classic remoting
Artery TCP will be the default transport in Akka 2.6.0, and the @ref:[classic remoting implementation](remoting.md)
will be deprecated.
Artery has the same functionality as classic remoting and you should normally only have to change the
configuration to switch.
Enable Artery as shown in above @ref:[configuration](#configuration) with your
@ref:[selected transport](#selecting-a-transport). `tcp` is a good start.
The protocol part in the Akka `Address`, for example `"akka.tcp://actorSystemName@10.0.0.1:2552/user/actorName"`
has changed from `akka.tcp` to `akka`. If you have configured or hardcoded any such addresses you have to change
them to `"akka://actorSystemName@10.0.0.1:2552/user/actorName"`. `akka` is used also when TLS is enabled.
One typical place where such address is used is in the `seed-nodes` configuration.
The configuration is different, so you might have to revisit any custom configuration. See the full
@ref:[reference configuration for Artery](general/configuration.md#config-akka-remote-artery) and
@ref:[reference configuration for classic remoting](general/configuration.md#config-akka-remote).
One thing to be aware of is that rolling update from classic remoting to Artery is not supported since the protocol
is completely different. It will require a full cluster shutdown and new startup.
See @ref:[migrating from classic remoting](project/migration-guide-2.5.x-2.6.x.md#classic-to-artery)
## Canonical address
@ -760,7 +740,7 @@ To use the external media driver from the Akka application you need to define th
configuration properties:
```
akka.remote.artery.advanced {
akka.remote.artery.advanced.aeron {
embedded-media-driver = off
aeron-dir = /dev/shm/aeron
}
@ -786,7 +766,7 @@ usage and latency with the following configuration:
```
# Values can be from 1 to 10, where 10 strongly prefers low latency
# and 1 strongly prefers less CPU usage
akka.remote.artery.advanced.idle-cpu-level = 1
akka.remote.artery.advanced.aeron.idle-cpu-level = 1
```
By setting this value to a lower number, it tells Akka to do longer "sleeping" periods on its thread dedicated

View file

@ -1,4 +1,10 @@
# Remoting
# Classic Remoting (Deprecated)
@@@ warning
Classic remoting has been deprecated and will be removed in Akka 2.7.0. Please use @ref[Artery](remoting-artery.md) instead.
@@@
@@@ note
@ -35,7 +41,7 @@ akka {
provider = remote
}
remote {
enabled-transports = ["akka.remote.netty.tcp"]
enabled-transports = ["akka.remote.classic.netty.tcp"]
netty.tcp {
hostname = "127.0.0.1"
port = 2552
@ -258,7 +264,7 @@ The list of allowed classes has to be configured on the "remote" system, in othe
others will be attempting to remote deploy Actors. That system, locally, knows best which Actors it should or
should not allow others to remote deploy onto it. The full settings section may for example look like this:
@@snip [RemoteDeploymentWhitelistSpec.scala](/akka-remote/src/test/scala/akka/remote/RemoteDeploymentWhitelistSpec.scala) { #whitelist-config }
@@snip [RemoteDeploymentWhitelistSpec.scala](/akka-remote/src/test/scala/akka/remote/classic/RemoteDeploymentWhitelistSpec.scala) { #whitelist-config }
Actor classes not included in the whitelist will not be allowed to be remote deployed onto this system.
@ -445,13 +451,13 @@ its multiple [known attack surfaces](https://community.hpe.com/t5/Security-Resea
<a id="remote-tls"></a>
### Configuring SSL/TLS for Akka Remoting
SSL can be used as the remote transport by adding `akka.remote.netty.ssl` to the `enabled-transport` configuration section.
SSL can be used as the remote transport by adding `akka.remote.classic.netty.ssl` to the `enabled-transport` configuration section.
An example of setting up the default Netty based SSL driver as default:
```
akka {
remote {
enabled-transports = [akka.remote.netty.ssl]
enabled-transports = [akka.remote.classic.netty.ssl]
}
}
```

View file

@ -28,7 +28,7 @@ configuration of the TLS random number generator should be used:
```
# Set `SecureRandom` RNG explicitly (but it is also the default)
akka.remote.netty.ssl.random-number-generator = "SecureRandom"
akka.remote.classic.netty.ssl.random-number-generator = "SecureRandom"
akka.remote.artery.ssl.config-ssl-engine.random-number-generator = "SecureRandom"
```

View file

@ -30,7 +30,7 @@ public class StatsSampleOneMasterMain {
for (String port : ports) {
// Override the configuration of the port
Config config =
ConfigFactory.parseString("akka.remote.netty.tcp.port=" + port)
ConfigFactory.parseString("akka.remote.classic.netty.tcp.port=" + port)
.withFallback(ConfigFactory.parseString("akka.cluster.roles = [compute]"))
.withFallback(ConfigFactory.load("stats2"));

View file

@ -72,7 +72,7 @@ public class RemoteDeploymentDocTest extends AbstractJavaTest {
@Test
public void demonstrateProgrammaticConfig() {
// #programmatic
ConfigFactory.parseString("akka.remote.netty.tcp.hostname=\"1.2.3.4\"")
ConfigFactory.parseString("akka.remote.classic.netty.tcp.hostname=\"1.2.3.4\"")
.withFallback(ConfigFactory.load());
// #programmatic

View file

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

View file

@ -43,7 +43,7 @@ object FactorialBackend {
// Override the configuration of the port when specified as program argument
val port = if (args.isEmpty) "0" else args(0)
val config = ConfigFactory
.parseString(s"akka.remote.netty.tcp.port=$port")
.parseString(s"akka.remote.classic.netty.tcp.port=$port")
.withFallback(ConfigFactory.parseString("akka.cluster.roles = [backend]"))
.withFallback(ConfigFactory.load("factorial"))

View file

@ -47,7 +47,7 @@ object TransformationBackend {
// Override the configuration of the port when specified as program argument
val port = if (args.isEmpty) "0" else args(0)
val config = ConfigFactory
.parseString(s"akka.remote.netty.tcp.port=$port")
.parseString(s"akka.remote.classic.netty.tcp.port=$port")
.withFallback(ConfigFactory.parseString("akka.cluster.roles = [backend]"))
.withFallback(ConfigFactory.load())

View file

@ -46,7 +46,7 @@ object TransformationFrontend {
// Override the configuration of the port when specified as program argument
val port = if (args.isEmpty) "0" else args(0)
val config = ConfigFactory
.parseString(s"akka.remote.netty.tcp.port=$port")
.parseString(s"akka.remote.classic.netty.tcp.port=$port")
.withFallback(ConfigFactory.parseString("akka.cluster.roles = [frontend]"))
.withFallback(ConfigFactory.load())

View file

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

View file

@ -21,9 +21,8 @@ object RemoteDeploymentDocSpec {
class RemoteDeploymentDocSpec extends AkkaSpec("""
akka.actor.provider = remote
akka.remote.netty.tcp {
port = 0
}
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
""") with ImplicitSender {
import RemoteDeploymentDocSpec._

View file

@ -110,7 +110,7 @@ abstract class MultiNodeConfig {
private[akka] def config: Config = {
val transportConfig =
if (_testTransport) ConfigFactory.parseString("""
akka.remote.netty.tcp.applied-adapters = [trttl, gremlin]
akka.remote.classic.netty.tcp.applied-adapters = [trttl, gremlin]
akka.remote.artery.advanced.test-mode = on
""")
else ConfigFactory.empty
@ -214,8 +214,8 @@ object MultiNodeSpec {
Map(
"akka.actor.provider" -> "remote",
"akka.remote.artery.canonical.hostname" -> selfName,
"akka.remote.netty.tcp.hostname" -> selfName,
"akka.remote.netty.tcp.port" -> selfPort,
"akka.remote.classic.netty.tcp.hostname" -> selfName,
"akka.remote.classic.netty.tcp.port" -> selfPort,
"akka.remote.artery.canonical.port" -> selfPort))
private[testkit] val baseConfig: Config =
@ -495,7 +495,7 @@ abstract class MultiNodeSpec(
*/
protected def startNewSystem(): ActorSystem = {
val config = ConfigFactory
.parseString(s"akka.remote.netty.tcp{port=${myAddress.port.get}\nhostname=${myAddress.host.get}}")
.parseString(s"akka.remote.classic.netty.tcp{port=${myAddress.port.get}\nhostname=${myAddress.host.get}}")
.withFallback(system.settings.config)
val sys = ActorSystem(system.name, config)
injectDeployments(sys, myself)

View file

@ -29,8 +29,12 @@ object PersistencePluginProxySpec {
}
}
remote {
enabled-transports = ["akka.remote.netty.tcp"]
netty.tcp {
enabled-transports = ["akka.remote.classic.netty.tcp"]
classic.netty.tcp {
hostname = "127.0.0.1"
port = 0
}
artery.canonical {
hostname = "127.0.0.1"
port = 0
}

View file

@ -26,8 +26,12 @@ object SharedLeveldbJournalSpec {
}
}
remote {
enabled-transports = ["akka.remote.netty.tcp"]
netty.tcp {
enabled-transports = ["akka.remote.classic.netty.tcp"]
classic.netty.tcp {
hostname = "127.0.0.1"
port = 0
}
artery.canonical {
hostname = "127.0.0.1"
port = 0
}

View file

@ -46,8 +46,12 @@ object SerializerSpecConfigs {
provider = remote
}
remote {
enabled-transports = ["akka.remote.netty.tcp"]
netty.tcp {
enabled-transports = ["akka.remote.classic.netty.tcp"]
classic.netty.tcp {
hostname = "127.0.0.1"
port = 0
}
artery.canonical {
hostname = "127.0.0.1"
port = 0
}
@ -296,7 +300,7 @@ class MessageSerializerPersistenceSpec extends AkkaSpec(customSerializers) {
object MessageSerializerRemotingSpec {
class LocalActor(port: Int) extends Actor {
def receive = {
case m => context.actorSelection(s"akka.tcp://remote@127.0.0.1:${port}/user/remote").tell(m, Actor.noSender)
case m => context.actorSelection(s"akka://remote@127.0.0.1:${port}/user/remote").tell(m, Actor.noSender)
}
}

View file

@ -27,8 +27,8 @@ class RemoteNodeRestartDeathWatchConfig(artery: Boolean) extends MultiNodeConfig
commonConfig(debugConfig(on = false).withFallback(ConfigFactory.parseString(s"""
akka.loglevel = INFO
akka.remote.log-remote-lifecycle-events = off
akka.remote.transport-failure-detector.heartbeat-interval = 1 s
akka.remote.transport-failure-detector.acceptable-heartbeat-pause = 3 s
akka.remote.classic.transport-failure-detector.heartbeat-interval = 1 s
akka.remote.classic.transport-failure-detector.acceptable-heartbeat-pause = 3 s
akka.remote.artery.enabled = $artery
""")))
@ -110,7 +110,7 @@ abstract class RemoteNodeRestartDeathWatchSpec(multiNodeConfig: RemoteNodeRestar
val freshSystem = ActorSystem(
system.name,
ConfigFactory.parseString(s"""
akka.remote.netty.tcp.port = ${address.port.get}
akka.remote.classic.netty.tcp.port = ${address.port.get}
akka.remote.artery.canonical.port = ${address.port.get}
""").withFallback(system.settings.config))
freshSystem.actorOf(Props[Subject], "subject")

View file

@ -111,7 +111,7 @@ abstract class RemoteQuarantinePiercingSpec(multiNodeConfig: RemoteQuarantinePie
val freshSystem = ActorSystem(
system.name,
ConfigFactory.parseString(s"""
akka.remote.netty.tcp.port = ${address.port.get}
akka.remote.classic.netty.tcp.port = ${address.port.get}
akka.remote.artery.canonical.port = ${address.port.get}
""").withFallback(system.settings.config))
freshSystem.actorOf(Props[Subject], "subject")

View file

@ -24,7 +24,7 @@ class RemoteReDeploymentConfig(artery: Boolean) extends MultiNodeConfig {
val second = role("second")
commonConfig(
debugConfig(on = true).withFallback(ConfigFactory.parseString(s"""akka.remote.transport-failure-detector {
debugConfig(on = false).withFallback(ConfigFactory.parseString(s"""akka.remote.classic.transport-failure-detector {
threshold=0.1
heartbeat-interval=0.1s
acceptable-heartbeat-pause=1s

View file

@ -27,7 +27,7 @@ object TransportFailConfig extends MultiNodeConfig {
commonConfig(debugConfig(on = false).withFallback(ConfigFactory.parseString(s"""
akka.loglevel = INFO
akka.remote {
akka.remote.classic {
transport-failure-detector {
implementation-class = "akka.remote.TransportFailSpec$$TestFailureDetector"
heartbeat-interval = 1 s

View file

@ -60,9 +60,9 @@ object BenchmarkFileReporter {
"akka.test.LatencySpec.real-message",
"akka.remote.artery.enabled",
"akka.remote.artery.advanced.inbound-lanes",
"akka.remote.artery.advanced.idle-cpu-level",
"akka.remote.artery.advanced.buffer-pool-size",
"akka.remote.artery.advanced.embedded-media-driver",
"akka.remote.artery.advanced.aeron.idle-cpu-level",
"akka.remote.artery.advanced.aeron.embedded-media-driver",
"akka.remote.default-remote-dispatcher.throughput",
"akka.remote.default-remote-dispatcher.fork-join-executor.parallelism-factor",
"akka.remote.default-remote-dispatcher.fork-join-executor.parallelism-min",

View file

@ -50,10 +50,10 @@ object LatencySpec extends MultiNodeConfig {
# for serious measurements when running this test on only one machine
# it is recommended to use external media driver
# See akka-remote/src/test/resources/aeron.properties
# advanced.embedded-media-driver = off
# advanced.aeron-dir = "akka-remote/target/aeron"
# advanced.aeron.embedded-media-driver = off
# advanced.aeron.aeron-dir = "akka-remote/target/aeron"
# on linux, use directory on ram disk, instead
# advanced.aeron-dir = "/dev/shm/aeron"
# advanced.aeron.aeron-dir = "/dev/shm/aeron"
advanced.compression {
actor-refs.advertisement-interval = 2 second

View file

@ -57,10 +57,10 @@ object MaxThroughputSpec extends MultiNodeConfig {
# for serious measurements when running this test on only one machine
# it is recommended to use external media driver
# See akka-remote/src/test/resources/aeron.properties
# advanced.embedded-media-driver = off
# advanced.aeron-dir = "akka-remote/target/aeron"
# advanced.aeron.embedded-media-driver = off
# advanced.aeron.aeron-dir = "akka-remote/target/aeron"
# on linux, use directory on ram disk, instead
# advanced.aeron-dir = "/dev/shm/aeron"
# advanced.aeron.aeron-dir = "/dev/shm/aeron"
advanced.compression {
actor-refs.advertisement-interval = 2 second

View file

@ -64,7 +64,7 @@ abstract class AeronStreamConsistencySpec
Aeron.connect(ctx)
}
val idleCpuLevel = system.settings.config.getInt("akka.remote.artery.advanced.idle-cpu-level")
val idleCpuLevel = system.settings.config.getInt("akka.remote.artery.advanced.aeron.idle-cpu-level")
val taskRunner = {
val r = new TaskRunner(system.asInstanceOf[ExtendedActorSystem], idleCpuLevel)
r.start()

View file

@ -96,7 +96,7 @@ abstract class AeronStreamMaxThroughputSpec
Aeron.connect(ctx)
}
val idleCpuLevel = system.settings.config.getInt("akka.remote.artery.advanced.idle-cpu-level")
val idleCpuLevel = system.settings.config.getInt("akka.remote.artery.advanced.aeron.idle-cpu-level")
val taskRunner = {
val r = new TaskRunner(system.asInstanceOf[ExtendedActorSystem], idleCpuLevel)
r.start()

View file

@ -2,37 +2,36 @@
* Copyright (C) 2009-2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.remote
package akka.remote.classic
import akka.remote.transport.AssociationHandle
import scala.concurrent.duration._
import com.typesafe.config.ConfigFactory
import akka.actor._
import akka.actor.{ ActorIdentity, Identify, _ }
import akka.remote.testconductor.RoleName
import akka.remote.transport.ThrottlerTransportAdapter.ForceDisassociateExplicitly
import akka.remote.testkit.MultiNodeConfig
import akka.remote.transport.AssociationHandle
import akka.remote.transport.ThrottlerTransportAdapter.ForceDisassociateExplicitly
import akka.remote.{ RARP, RemotingMultiNodeSpec }
import akka.testkit._
import akka.actor.ActorIdentity
import akka.remote.testconductor.RoleName
import akka.actor.Identify
import com.typesafe.config.ConfigFactory
import scala.concurrent.Await
import scala.concurrent.duration._
object RemoteGatePiercingSpec extends MultiNodeConfig {
val first = role("first")
val second = role("second")
commonConfig(
debugConfig(on = false)
.withFallback(ConfigFactory.parseString("""
debugConfig(on = false).withFallback(
ConfigFactory.parseString("""
akka.loglevel = INFO
akka.remote.log-remote-lifecycle-events = INFO
akka.remote.transport-failure-detector.acceptable-heartbeat-pause = 5 s
akka.remote.artery.enabled = false
akka.remote.classic.log-remote-lifecycle-events = INFO
akka.remote.classic.transport-failure-detector.acceptable-heartbeat-pause = 5 s
""")))
nodeConfig(first)(ConfigFactory.parseString("akka.remote.retry-gate-closed-for = 1 d # Keep it long"))
nodeConfig(first)(ConfigFactory.parseString("akka.remote.classic.retry-gate-closed-for = 1 d # Keep it long"))
nodeConfig(second)(ConfigFactory.parseString("akka.remote.retry-gate-closed-for = 1 s # Keep it short"))
nodeConfig(second)(ConfigFactory.parseString("akka.remote.classic.retry-gate-closed-for = 1 s # Keep it short"))
testTransport(on = true)

View file

@ -2,21 +2,19 @@
* Copyright (C) 2009-2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.remote
package akka.remote.classic
import akka.remote.transport.AssociationHandle
import scala.concurrent.duration._
import com.typesafe.config.ConfigFactory
import akka.actor._
import akka.actor.{ ActorIdentity, Identify, _ }
import akka.remote.testconductor.RoleName
import akka.remote.transport.ThrottlerTransportAdapter.ForceDisassociateExplicitly
import akka.remote.testkit.MultiNodeConfig
import akka.remote.transport.AssociationHandle
import akka.remote.transport.ThrottlerTransportAdapter.ForceDisassociateExplicitly
import akka.remote.{ RARP, RemotingMultiNodeSpec }
import akka.testkit._
import akka.actor.ActorIdentity
import akka.remote.testconductor.RoleName
import akka.actor.Identify
import com.typesafe.config.ConfigFactory
import scala.concurrent.Await
import scala.concurrent.duration._
object RemoteNodeRestartGateSpec extends MultiNodeConfig {
val first = role("first")
@ -25,9 +23,10 @@ object RemoteNodeRestartGateSpec extends MultiNodeConfig {
commonConfig(
debugConfig(on = false)
.withFallback(ConfigFactory.parseString("""
akka.remote.artery.enabled = off
akka.loglevel = INFO
akka.remote.log-remote-lifecycle-events = INFO
akka.remote.retry-gate-closed-for = 1d # Keep it long
akka.remote.classic.log-remote-lifecycle-events = INFO
akka.remote.classic.retry-gate-closed-for = 1d # Keep it long
""")))
testTransport(on = true)
@ -100,7 +99,7 @@ abstract class RemoteNodeRestartGateSpec extends RemotingMultiNodeSpec(RemoteNod
system.name,
ConfigFactory.parseString(s"""
akka.remote.retry-gate-closed-for = 0.5 s
akka.remote.netty.tcp {
akka.remote.classic.netty.tcp {
hostname = ${address.host.get}
port = ${address.port.get}
}

View file

@ -2,20 +2,19 @@
* Copyright (C) 2009-2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.remote
package akka.remote.classic
import language.postfixOps
import scala.concurrent.duration._
import com.typesafe.config.ConfigFactory
import akka.actor._
import akka.actor.{ ActorIdentity, Identify, _ }
import akka.remote.testconductor.RoleName
import akka.remote.transport.ThrottlerTransportAdapter.{ Direction, ForceDisassociate }
import akka.remote.testkit.MultiNodeConfig
import akka.remote.transport.ThrottlerTransportAdapter.{ Direction, ForceDisassociate }
import akka.remote.{ RARP, RemotingMultiNodeSpec }
import akka.testkit._
import akka.actor.ActorIdentity
import akka.remote.testconductor.RoleName
import akka.actor.Identify
import com.typesafe.config.ConfigFactory
import scala.concurrent.Await
import scala.concurrent.duration._
import scala.language.postfixOps
object RemoteNodeShutdownAndComesBackSpec extends MultiNodeConfig {
val first = role("first")
@ -25,10 +24,11 @@ object RemoteNodeShutdownAndComesBackSpec extends MultiNodeConfig {
debugConfig(on = false).withFallback(
ConfigFactory.parseString("""
akka.loglevel = INFO
akka.remote.log-remote-lifecycle-events = INFO
akka.remote.artery.enabled = off
akka.remote.classic.log-remote-lifecycle-events = INFO
## Keep it tight, otherwise reestablishing a connection takes too much time
akka.remote.transport-failure-detector.heartbeat-interval = 1 s
akka.remote.transport-failure-detector.acceptable-heartbeat-pause = 3 s
akka.remote.classic.transport-failure-detector.heartbeat-interval = 1 s
akka.remote.classic.transport-failure-detector.acceptable-heartbeat-pause = 3 s
akka.remote.watch-failure-detector.acceptable-heartbeat-pause = 60 s
""")))
@ -81,6 +81,7 @@ abstract class RemoteNodeShutdownAndComesBackSpec extends RemotingMultiNodeSpec(
// Drop all messages from this point so no SHUTDOWN is ever received
testConductor.blackhole(second, first, Direction.Send).await
// Shut down all existing connections so that the system can enter recovery mode (association attempts)
// TODO, should artery support this?
Await.result(
RARP(system).provider.transport.managementCommand(ForceDisassociate(node(second).address)),
3.seconds)
@ -136,7 +137,7 @@ abstract class RemoteNodeShutdownAndComesBackSpec extends RemotingMultiNodeSpec(
val freshSystem = ActorSystem(
system.name,
ConfigFactory.parseString(s"""
akka.remote.netty.tcp.port = ${address.port.get}
akka.remote.classic.netty.tcp.port = ${address.port.get}
akka.remote.artery.canonical.port = ${address.port.get}
""").withFallback(system.settings.config))
freshSystem.actorOf(Props[Subject], "subject")

View file

@ -2,19 +2,18 @@
* Copyright (C) 2009-2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.remote
package akka.remote.classic
import language.postfixOps
import scala.concurrent.duration._
import com.typesafe.config.ConfigFactory
import akka.actor._
import akka.actor.{ ActorIdentity, Identify, _ }
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.{ AddressUidExtension, RARP, RemotingMultiNodeSpec, ThisActorSystemQuarantinedEvent }
import akka.testkit._
import akka.actor.ActorIdentity
import akka.remote.testconductor.RoleName
import akka.actor.Identify
import com.typesafe.config.ConfigFactory
import scala.concurrent.Await
import scala.concurrent.duration._
import scala.language.postfixOps
object RemoteRestartedQuarantinedSpec extends MultiNodeConfig {
val first = role("first")
@ -23,16 +22,17 @@ object RemoteRestartedQuarantinedSpec extends MultiNodeConfig {
commonConfig(
debugConfig(on = false).withFallback(ConfigFactory.parseString(
"""
akka.remote.artery.enabled = off
# Keep it long, we don't want reconnects
akka.remote.retry-gate-closed-for = 1 s
akka.remote.classic.retry-gate-closed-for = 1 s
# Important, otherwise it is very racy to get a non-writing endpoint: the only way to do it if the two nodes
# associate to each other at the same time. Setting this will ensure that the right scenario happens.
akka.remote.use-passive-connections = off
akka.remote.classic.use-passive-connections = off
# TODO should not be needed, but see TODO at the end of the test
akka.remote.transport-failure-detector.heartbeat-interval = 1 s
akka.remote.transport-failure-detector.acceptable-heartbeat-pause = 10 s
akka.remote.classic.transport-failure-detector.heartbeat-interval = 1 s
akka.remote.classic.transport-failure-detector.acceptable-heartbeat-pause = 10 s
""")))
testTransport(on = true)
@ -119,7 +119,7 @@ abstract class RemoteRestartedQuarantinedSpec extends RemotingMultiNodeSpec(Remo
system.name,
ConfigFactory.parseString(s"""
akka.remote.retry-gate-closed-for = 0.5 s
akka.remote.netty.tcp {
akka.remote.classic.netty.tcp {
hostname = ${address.host.get}
port = ${address.port.get}
}
@ -132,7 +132,7 @@ abstract class RemoteRestartedQuarantinedSpec extends RemotingMultiNodeSpec(Remo
freshSystem
.actorSelection(RootActorPath(firstAddress) / "user" / "subject")
.tell(Identify("subject"), probe.ref)
probe.expectMsgType[ActorIdentity](1.second).ref should not be (None)
probe.expectMsgType[ActorIdentity](1.second).ref should not be None
},
30.seconds)

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