+rem remote deployment whitelist feature (#21676)

* +rem remote deployment whitelist feature (initial commit)

+rem remote deployment whitelist feature (initial commit)

* +doc,rem added docs on remoting whitelist

* Update remoting.rst

* Update remoting.rst
This commit is contained in:
Konrad Malawski 2016-10-24 13:44:15 +02:00 committed by GitHub
parent 46f7eef3d9
commit ad3ef5982c
6 changed files with 268 additions and 35 deletions

View file

@ -589,7 +589,7 @@ akka {
# Set this to on to enable serialization-bindings define in # Set this to on to enable serialization-bindings define in
# additional-serialization-bindings. Those are by default not included # additional-serialization-bindings. Those are by default not included
# for backwards compatibility reasons. They are enabled by default if # 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 enable-additional-serialization-bindings = off
# Additional serialization-bindings that are replacing Java serialization are # Additional serialization-bindings that are replacing Java serialization are

View file

@ -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 .. 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 Lifecycle and Failure Recovery Model
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^

View file

@ -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 .. 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 Lifecycle and Failure Recovery Model
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^

View file

@ -218,6 +218,14 @@ akka {
expected-response-after = 1 s 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 #//#shared
} }

View file

@ -4,10 +4,13 @@
package akka.remote package akka.remote
import java.util
import java.util.Collections
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.annotation.tailrec import scala.annotation.tailrec
import scala.util.control.NonFatal 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.event.LoggingAdapter
import akka.dispatch.sysmsg.{ DeathWatchNotification, SystemMessage, Watch } import akka.dispatch.sysmsg.{ DeathWatchNotification, SystemMessage, Watch }
import akka.actor.ActorRefWithCell import akka.actor.ActorRefWithCell
@ -22,8 +25,11 @@ import akka.actor.ActorIdentity
import akka.actor.EmptyLocalActorRef import akka.actor.EmptyLocalActorRef
import akka.event.AddressTerminatedTopic import akka.event.AddressTerminatedTopic
import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.ConcurrentHashMap
import scala.collection.immutable
import akka.dispatch.sysmsg.Unwatch import akka.dispatch.sysmsg.Unwatch
import akka.NotUsed import akka.NotUsed
import com.typesafe.config.Config
/** /**
* INTERNAL API * INTERNAL API
@ -61,6 +67,13 @@ private[akka] class RemoteSystemDaemon(
private val parent2children = new ConcurrentHashMap[ActorRef, Set[ActorRef]] private val parent2children = new ConcurrentHashMap[ActorRef, Set[ActorRef]]
private val dedupDaemonMsgCreateMessages = new ConcurrentHashMap[String, NotUsed] 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 = @tailrec private def addChildParentNeedsWatch(parent: ActorRef, child: ActorRef): Boolean =
parent2children.get(parent) match { parent2children.get(parent) match {
case null case null
@ -142,41 +155,19 @@ private[akka] class RemoteSystemDaemon(
log.debug("Received command [{}] to RemoteSystemDaemon on [{}]", message, path.address) log.debug("Received command [{}] to RemoteSystemDaemon on [{}]", message, path.address)
message match { message match {
case DaemonMsgCreate(_, _, path, _) if untrustedMode case DaemonMsgCreate(_, _, path, _) if untrustedMode
log.debug("does not accept deployments (untrusted) for [{}]", path) log.debug("does not accept deployments (untrusted) for [{}]", path) // TODO add security marker?
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)
path match { case DaemonMsgCreate(props, deploy, path, supervisor) if whitelistEnabled
case ActorPathExtractor(address, elems) if elems.nonEmpty && elems.head == "remote" val name = props.clazz.getCanonicalName
// TODO RK currently the extracted address is just ignored, is that okay? if (remoteDeploymentWhitelist.contains(name))
// TODO RK canonicalize path so as not to duplicate it always #1446 doCreateActor(message, props, deploy, path, supervisor)
val subpath = elems.drop(1) else {
val p = this.path / subpath val ex = new NotWhitelistedClassRemoteDeploymentAttemptException(props.actorClass, remoteDeploymentWhitelist)
val childName = { log.error(ex, "Received command to create remote Actor, but class [{}] is not white-listed! " +
val s = subpath.mkString("/") "Target path: [{}]", props.actorClass, path) // TODO add security marker?
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)
doCreateActor(message, props, deploy, path, supervisor)
} }
case sel: ActorSelectionMessage 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) 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 { def terminationHookDoneWhenNoChildren(): Unit = terminating.whileOn {
if (!hasChildren) terminator.tell(TerminationHookDone, this) 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(", ")}]")

View file

@ -0,0 +1,156 @@
/**
* Copyright (C) 2009-2016 Lightbend Inc. <http://www.lightbend.com>
*/
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)
}
}
}