diff --git a/akka-actor/src/main/resources/reference.conf b/akka-actor/src/main/resources/reference.conf index 10ff0eee8d..004829290a 100644 --- a/akka-actor/src/main/resources/reference.conf +++ b/akka-actor/src/main/resources/reference.conf @@ -589,7 +589,7 @@ akka { # Set this to on to enable serialization-bindings define in # additional-serialization-bindings. Those are by default not included # for backwards compatibility reasons. They are enabled by default if - # akka.remote.artery.enabled=on. + # akka.remote.artery.enabled=on. enable-additional-serialization-bindings = off # Additional serialization-bindings that are replacing Java serialization are diff --git a/akka-docs/rst/java/remoting.rst b/akka-docs/rst/java/remoting.rst index 84813d8605..3dbb18ded6 100644 --- a/akka-docs/rst/java/remoting.rst +++ b/akka-docs/rst/java/remoting.rst @@ -168,6 +168,23 @@ you can advise the system to create a child on that remote node like so: .. includecode:: code/docs/remoting/RemoteDeploymentDocTest.java#deploy +Remote deployment whitelist +--------------------------- + +As remote deployment can potentially be abused by both users and even attackers a whitelist feature +is available to guard the ActorSystem from deploying unexpected actors. Please note that remote deployment +is *not* remote code loading, the Actors class to be deployed onto a remote system needs to be present on that +remote system. This still however may pose a security risk, and one may want to restrict remote deployment to +only a specific set of known actors by enabling the whitelist feature. + +To enable remote deployment whitelisting set the ``akka.remote.deployment.enable-whitelist`` value to ``on``. +The list of allowed classes has to be configured on the "remote" system, in other words on the system onto which +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: + +.. includecode:: ../../../akka-remote/src/test/scala/akka/remote/RemoteDeploymentWhitelistSpec.scala#whitelist-config + +Actor classes not included in the whitelist will not be allowed to be remote deployed onto this system. Lifecycle and Failure Recovery Model ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ diff --git a/akka-docs/rst/scala/remoting.rst b/akka-docs/rst/scala/remoting.rst index 594005ce6d..4994c3eb94 100644 --- a/akka-docs/rst/scala/remoting.rst +++ b/akka-docs/rst/scala/remoting.rst @@ -174,6 +174,24 @@ you can advise the system to create a child on that remote node like so: .. includecode:: code/docs/remoting/RemoteDeploymentDocSpec.scala#deploy +Remote deployment whitelist +--------------------------- + +As remote deployment can potentially be abused by both users and even attackers a whitelist feature +is available to guard the ActorSystem from deploying unexpected actors. Please note that remote deployment +is *not* remote code loading, the Actors class to be deployed onto a remote system needs to be present on that +remote system. This still however may pose a security risk, and one may want to restrict remote deployment to +only a specific set of known actors by enabling the whitelist feature. + +To enable remote deployment whitelisting set the ``akka.remote.deployment.enable-whitelist`` value to ``on``. +The list of allowed classes has to be configured on the "remote" system, in other words on the system onto which +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: + +.. includecode:: ../../../akka-remote/src/test/scala/akka/remote/RemoteDeploymentWhitelistSpec.scala#whitelist-config + +Actor classes not included in the whitelist will not be allowed to be remote deployed onto this system. + Lifecycle and Failure Recovery Model ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index 8d0d8a2d35..d3e0de43d9 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -218,6 +218,14 @@ akka { expected-response-after = 1 s } + + # remote deployment configuration section + deployment { + # If true, will only allow specific classes to be instanciated on this system via remote deployment + enable-whitelist = off + + whitelist = [] + } #//#shared } diff --git a/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala b/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala index 847821dbcc..589ff937af 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala @@ -4,10 +4,13 @@ package akka.remote +import java.util +import java.util.Collections + import scala.concurrent.duration._ import scala.annotation.tailrec import scala.util.control.NonFatal -import akka.actor.{ VirtualPathContainer, Deploy, Props, Nobody, InternalActorRef, ActorSystemImpl, ActorRef, ActorPathExtractor, ActorPath, Actor, AddressTerminated } +import akka.actor.{ Actor, ActorPath, ActorPathExtractor, ActorRef, ActorSystemImpl, AddressTerminated, Deploy, InternalActorRef, Nobody, Props, VirtualPathContainer } import akka.event.LoggingAdapter import akka.dispatch.sysmsg.{ DeathWatchNotification, SystemMessage, Watch } import akka.actor.ActorRefWithCell @@ -22,8 +25,11 @@ import akka.actor.ActorIdentity import akka.actor.EmptyLocalActorRef import akka.event.AddressTerminatedTopic import java.util.concurrent.ConcurrentHashMap +import scala.collection.immutable + import akka.dispatch.sysmsg.Unwatch import akka.NotUsed +import com.typesafe.config.Config /** * INTERNAL API @@ -61,6 +67,13 @@ private[akka] class RemoteSystemDaemon( private val parent2children = new ConcurrentHashMap[ActorRef, Set[ActorRef]] private val dedupDaemonMsgCreateMessages = new ConcurrentHashMap[String, NotUsed] + private val whitelistEnabled = system.settings.config.getBoolean("akka.remote.deployment.enable-whitelist") + private val remoteDeploymentWhitelist: immutable.Set[String] = { + import scala.collection.JavaConverters._ + if (whitelistEnabled) system.settings.config.getStringList("akka.remote.deployment.whitelist").asScala.toSet + else Set.empty + } + @tailrec private def addChildParentNeedsWatch(parent: ActorRef, child: ActorRef): Boolean = parent2children.get(parent) match { case null ⇒ @@ -142,41 +155,19 @@ private[akka] class RemoteSystemDaemon( log.debug("Received command [{}] to RemoteSystemDaemon on [{}]", message, path.address) message match { case DaemonMsgCreate(_, _, path, _) if untrustedMode ⇒ - log.debug("does not accept deployments (untrusted) for [{}]", path) - case DaemonMsgCreate(props, deploy, path, supervisor) ⇒ - // Artery sends multiple DaemonMsgCreate over several streams to preserve ordering assumptions, - // DaemonMsgCreate for this unique path is already handled and therefore deduplicated - if (dedupDaemonMsgCreateMessages.putIfAbsent(path, NotUsed) == null) { - // we only need to keep the dedup info for a short period - // this is not a real actor, so no point in scheduling message - system.scheduler.scheduleOnce(5.seconds)(dedupDaemonMsgCreateMessages.remove(path))(system.dispatcher) + log.debug("does not accept deployments (untrusted) for [{}]", path) // TODO add security marker? - path match { - case ActorPathExtractor(address, elems) if elems.nonEmpty && elems.head == "remote" ⇒ - // TODO RK currently the extracted “address” is just ignored, is that okay? - // TODO RK canonicalize path so as not to duplicate it always #1446 - val subpath = elems.drop(1) - val p = this.path / subpath - val childName = { - val s = subpath.mkString("/") - val i = s.indexOf('#') - if (i < 0) s - else s.substring(0, i) - } - val isTerminating = !terminating.whileOff { - val parent = supervisor.asInstanceOf[InternalActorRef] - val actor = system.provider.actorOf(system, props, parent, - p, systemService = false, Some(deploy), lookupDeploy = true, async = false) - addChild(childName, actor) - actor.sendSystemMessage(Watch(actor, this)) - actor.start() - if (addChildParentNeedsWatch(parent, actor)) parent.sendSystemMessage(Watch(parent, this)) - } - if (isTerminating) log.error("Skipping [{}] to RemoteSystemDaemon on [{}] while terminating", message, p.address) - case _ ⇒ - log.debug("remote path does not match path from message [{}]", message) - } + case DaemonMsgCreate(props, deploy, path, supervisor) if whitelistEnabled ⇒ + val name = props.clazz.getCanonicalName + if (remoteDeploymentWhitelist.contains(name)) + doCreateActor(message, props, deploy, path, supervisor) + else { + val ex = new NotWhitelistedClassRemoteDeploymentAttemptException(props.actorClass, remoteDeploymentWhitelist) + log.error(ex, "Received command to create remote Actor, but class [{}] is not white-listed! " + + "Target path: [{}]", props.actorClass, path) // TODO add security marker? } + case DaemonMsgCreate(props, deploy, path, supervisor) ⇒ + doCreateActor(message, props, deploy, path, supervisor) } case sel: ActorSelectionMessage ⇒ @@ -226,8 +217,51 @@ private[akka] class RemoteSystemDaemon( case NonFatal(e) ⇒ log.error(e, "exception while processing remote command [{}] from [{}]", msg, sender) } + private def doCreateActor(message: DaemonMsg, props: Props, deploy: Deploy, path: String, supervisor: ActorRef) = { + // Artery sends multiple DaemonMsgCreate over several streams to preserve ordering assumptions, + // DaemonMsgCreate for this unique path is already handled and therefore deduplicated + if (dedupDaemonMsgCreateMessages.putIfAbsent(path, NotUsed) == null) { + // we only need to keep the dedup info for a short period + // this is not a real actor, so no point in scheduling message + system.scheduler.scheduleOnce(5.seconds)(dedupDaemonMsgCreateMessages.remove(path))(system.dispatcher) + + path match { + case ActorPathExtractor(address, elems) if elems.nonEmpty && elems.head == "remote" ⇒ + // TODO RK currently the extracted “address” is just ignored, is that okay? + // TODO RK canonicalize path so as not to duplicate it always #1446 + val subpath = elems.drop(1) + val p = this.path / subpath + val childName = { + val s = subpath.mkString("/") + val i = s.indexOf('#') + if (i < 0) s + else s.substring(0, i) + } + val isTerminating = !terminating.whileOff { + val parent = supervisor.asInstanceOf[InternalActorRef] + val actor = system.provider.actorOf(system, props, parent, + p, systemService = false, Some(deploy), lookupDeploy = true, async = false) + addChild(childName, actor) + actor.sendSystemMessage(Watch(actor, this)) + actor.start() + if (addChildParentNeedsWatch(parent, actor)) parent.sendSystemMessage(Watch(parent, this)) + } + if (isTerminating) log.error("Skipping [{}] to RemoteSystemDaemon on [{}] while terminating", message, p.address) + case _ ⇒ + log.debug("remote path does not match path from message [{}]", message) + } + } + } + def terminationHookDoneWhenNoChildren(): Unit = terminating.whileOn { if (!hasChildren) terminator.tell(TerminationHookDone, this) } } + +/** INTERNAL API */ +final class NotWhitelistedClassRemoteDeploymentAttemptException(illegal: Class[_], whitelist: immutable.Set[String]) + extends RuntimeException( + s"Attempted to deploy not whitelisted Actor class: " + + s"[$illegal], " + + s"whitelisted classes: [${whitelist.mkString(", ")}]") diff --git a/akka-remote/src/test/scala/akka/remote/RemoteDeploymentWhitelistSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteDeploymentWhitelistSpec.scala new file mode 100644 index 0000000000..428ce852c0 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/RemoteDeploymentWhitelistSpec.scala @@ -0,0 +1,156 @@ +/** + * Copyright (C) 2009-2016 Lightbend Inc. + */ +package akka.remote + +import java.util.concurrent.ThreadLocalRandom + +import akka.actor._ +import akka.remote.transport._ +import akka.testkit._ +import akka.util.ByteString +import com.typesafe.config._ + +import scala.concurrent.duration._ + +object RemoteDeploymentWhitelistSpec { + + class EchoWhitelisted extends Actor { + var target: ActorRef = context.system.deadLetters + + def receive = { + case ex: Exception ⇒ throw ex + case x ⇒ target = sender(); sender() ! x + } + + override def preStart() {} + override def preRestart(cause: Throwable, msg: Option[Any]) { + target ! "preRestart" + } + override def postRestart(cause: Throwable) {} + override def postStop() { + target ! "postStop" + } + } + + class EchoNotWhitelisted extends Actor { + var target: ActorRef = context.system.deadLetters + + def receive = { + case ex: Exception ⇒ throw ex + case x ⇒ target = sender(); sender() ! x + } + + override def preStart() {} + override def preRestart(cause: Throwable, msg: Option[Any]) { + target ! "preRestart" + } + override def postRestart(cause: Throwable) {} + override def postStop() { + target ! "postStop" + } + } + + val cfg: Config = ConfigFactory parseString (s""" + akka { + actor.provider = remote + + remote { + enabled-transports = [ + "akka.remote.test", + "akka.remote.netty.tcp" + ] + + netty.tcp = { + port = 0 + hostname = "localhost" + } + + test { + transport-class = "akka.remote.transport.TestTransport" + applied-adapters = [] + registry-key = aX33k0jWKg + local-address = "test://RemoteDeploymentWhitelistSpec@localhost:12345" + maximum-payload-bytes = 32000 bytes + scheme-identifier = test + } + } + + actor.deployment { + /blub.remote = "akka.test://remote-sys@localhost:12346" + /danger-mouse.remote = "akka.test://remote-sys@localhost:12346" + } + } + """) + + def muteSystem(system: ActorSystem) { + system.eventStream.publish(TestEvent.Mute( + EventFilter.error(start = "AssociationError"), + EventFilter.warning(start = "AssociationError"), + EventFilter.warning(pattern = "received dead letter.*"))) + } +} + +class RemoteDeploymentWhitelistSpec extends AkkaSpec(RemoteDeploymentWhitelistSpec.cfg) with ImplicitSender with DefaultTimeout { + + import RemoteDeploymentWhitelistSpec._ + + val conf = ConfigFactory.parseString( + """ + akka.remote.test { + local-address = "test://remote-sys@localhost:12346" + maximum-payload-bytes = 48000 bytes + } + + //#whitelist-config + akka.remote.deployment { + enable-whitelist = on + + whitelist = [ + "NOT_ON_CLASSPATH", # verify we don't throw if a class not on classpath is listed here + "akka.remote.RemoteDeploymentWhitelistSpec.EchoWhitelisted" + ] + } + //#whitelist-config + """).withFallback(system.settings.config).resolve() + val remoteSystem = ActorSystem("remote-sys", conf) + + override def atStartup() = { + muteSystem(system) + remoteSystem.eventStream.publish(TestEvent.Mute( + EventFilter[EndpointException](), + EventFilter.error(start = "AssociationError"), + EventFilter.warning(pattern = "received dead letter.*(InboundPayload|Disassociate|HandleListener)"))) + } + + override def afterTermination() { + shutdown(remoteSystem) + AssociationRegistry.clear() + } + + "RemoteDeployment Whitelist" must { + + "allow deploying Echo actor (included in whitelist)" in { + val r = system.actorOf(Props[EchoWhitelisted], "blub") + r.path.toString should ===(s"akka.test://remote-sys@localhost:12346/remote/akka.test/${getClass.getSimpleName}@localhost:12345/user/blub") + r ! 42 + expectMsg(42) + EventFilter[Exception]("crash", occurrences = 1).intercept { + r ! new Exception("crash") + } + expectMsg("preRestart") + r ! 42 + expectMsg(42) + system.stop(r) + expectMsg("postStop") + } + + "not deploy actor not listed in whitelist" in { + val r = system.actorOf(Props[EchoNotWhitelisted], "danger-mouse") + r.path.toString should ===(s"akka.test://remote-sys@localhost:12346/remote/akka.test/${getClass.getSimpleName}@localhost:12345/user/danger-mouse") + r ! 42 + expectNoMsg(1.second) + system.stop(r) + } + } +}