Merge branch 'wip-2573-untrusted-∂π'

This commit is contained in:
Roland 2012-10-04 17:58:15 -07:00
commit 64c2383596
9 changed files with 195 additions and 48 deletions

View file

@ -76,7 +76,7 @@ case object Kill extends Kill {
@SerialVersionUID(1L) @SerialVersionUID(1L)
case class Terminated private[akka] (@BeanProperty actor: ActorRef)( case class Terminated private[akka] (@BeanProperty actor: ActorRef)(
@BeanProperty val existenceConfirmed: Boolean, @BeanProperty val existenceConfirmed: Boolean,
@BeanProperty val addressTerminated: Boolean) extends AutoReceivedMessage @BeanProperty val addressTerminated: Boolean) extends AutoReceivedMessage with PossiblyHarmful
/** /**
* INTERNAL API * INTERNAL API
@ -87,7 +87,7 @@ case class Terminated private[akka] (@BeanProperty actor: ActorRef)(
* and translates this event to [[akka.actor.Terminated]], which is sent itself. * and translates this event to [[akka.actor.Terminated]], which is sent itself.
*/ */
@SerialVersionUID(1L) @SerialVersionUID(1L)
private[akka] case class AddressTerminated(address: Address) extends AutoReceivedMessage private[akka] case class AddressTerminated(address: Address) extends AutoReceivedMessage with PossiblyHarmful
abstract class ReceiveTimeout extends PossiblyHarmful abstract class ReceiveTimeout extends PossiblyHarmful
@ -108,7 +108,7 @@ case object ReceiveTimeout extends ReceiveTimeout {
* nested path descriptions whenever using ! on them, the idea being that the * nested path descriptions whenever using ! on them, the idea being that the
* message is delivered by active routing of the various actors involved. * message is delivered by active routing of the various actors involved.
*/ */
sealed trait SelectionPath extends AutoReceivedMessage sealed trait SelectionPath extends AutoReceivedMessage with PossiblyHarmful
/** /**
* Internal use only * Internal use only

View file

@ -348,23 +348,44 @@ Akka provides a couple of ways to enhance security between remote nodes (client/
Untrusted Mode Untrusted Mode
-------------- --------------
You can enable untrusted mode for preventing system messages to be send by clients, e.g. messages like. As soon as an actor system can connect to another remotely, it may in principle
This will prevent the client to send these messages to the server: send any possible message to any actor contained within that remote system. One
example may be sending a :class:`PoisonPill` to the system guardian, shutting
* ``Create`` that system down. This is not always desired, and it can be disabled with the
* ``Recreate`` following setting::
* ``Suspend``
* ``Resume``
* ``Terminate``
* ``Supervise``
* ``ChildTerminated``
* ``Link``
* ``Unlink``
Here is how to turn it on in the config::
akka.remote.untrusted-mode = on akka.remote.untrusted-mode = on
This disallows sending of system messages (actor life-cycle commands,
DeathWatch, etc.) and any message extending :class:`PossiblyHarmful` to the
system on which this flag is set. Should a client send them nonetheless they
are dropped and logged (at DEBUG level in order to reduce the possibilities for
a denial of service attack). :class:`PossiblyHarmful` covers the predefined
messages like :class:`PoisonPill` and :class:`Kill`, but it can also be added
as a marker trait to user-defined messages.
In summary, the following operations are ignored by a system configured in
untrusted mode when incoming via the remoting layer:
* remote deployment (which also means no remote supervision)
* remote DeathWatch
* ``system.stop()``, :class:`PoisonPill`, :class:`Kill`
* sending any message which extends from the :class:`PossiblyHarmful` marker
interface, which includes :class:`Terminated`
.. note::
Enabling the untrusted mode does not remove the capability of the client to
freely choose the target of its message sends, which means that messages not
prohibited by the above rules can be sent to any actor in the remote system.
It is good practice for a client-facing system to only contain a well-defined
set of entry point actors, which then forward requests (possibly after
performing validation) to another actor system containing the actual worker
actors. If messaging between these two server-side systems is done using
local :class:`ActorRef` (they can be exchanged safely between actor systems
within the same JVM), you can restrict the messages on this interface by
marking them :class:`PossiblyHarmful` so that a client cannot forge them.
Secure Cookie Handshake Secure Cookie Handshake
----------------------- -----------------------

View file

@ -350,23 +350,44 @@ Akka provides a couple of ways to enhance security between remote nodes (client/
Untrusted Mode Untrusted Mode
-------------- --------------
You can enable untrusted mode for preventing system messages to be send by clients, e.g. messages like. As soon as an actor system can connect to another remotely, it may in principle
This will prevent the client to send these messages to the server: send any possible message to any actor contained within that remote system. One
example may be sending a :class:`PoisonPill` to the system guardian, shutting
* ``Create`` that system down. This is not always desired, and it can be disabled with the
* ``Recreate`` following setting::
* ``Suspend``
* ``Resume``
* ``Terminate``
* ``Supervise``
* ``ChildTerminated``
* ``Link``
* ``Unlink``
Here is how to turn it on in the config::
akka.remote.untrusted-mode = on akka.remote.untrusted-mode = on
This disallows sending of system messages (actor life-cycle commands,
DeathWatch, etc.) and any message extending :class:`PossiblyHarmful` to the
system on which this flag is set. Should a client send them nonetheless they
are dropped and logged (at DEBUG level in order to reduce the possibilities for
a denial of service attack). :class:`PossiblyHarmful` covers the predefined
messages like :class:`PoisonPill` and :class:`Kill`, but it can also be added
as a marker trait to user-defined messages.
In summary, the following operations are ignored by a system configured in
untrusted mode when incoming via the remoting layer:
* remote deployment (which also means no remote supervision)
* remote DeathWatch
* ``system.stop()``, :class:`PoisonPill`, :class:`Kill`
* sending any message which extends from the :class:`PossiblyHarmful` marker
interface, which includes :class:`Terminated`
.. note::
Enabling the untrusted mode does not remove the capability of the client to
freely choose the target of its message sends, which means that messages not
prohibited by the above rules can be sent to any actor in the remote system.
It is good practice for a client-facing system to only contain a well-defined
set of entry point actors, which then forward requests (possibly after
performing validation) to another actor system containing the actual worker
actors. If messaging between these two server-side systems is done using
local :class:`ActorRef` (they can be exchanged safely between actor systems
within the same JVM), you can restrict the messages on this interface by
marking them :class:`PossiblyHarmful` so that a client cannot forge them.
Secure Cookie Handshake Secure Cookie Handshake
----------------------- -----------------------

View file

@ -61,7 +61,7 @@ class RemoteActorRefProvider(
def init(system: ActorSystemImpl): Unit = { def init(system: ActorSystemImpl): Unit = {
local.init(system) local.init(system)
_remoteDaemon = new RemoteSystemDaemon(system, rootPath / "remote", rootGuardian, log) _remoteDaemon = new RemoteSystemDaemon(system, rootPath / "remote", rootGuardian, log, untrustedMode = remoteSettings.UntrustedMode)
local.registerExtraNames(Map(("remote", remoteDaemon))) local.registerExtraNames(Map(("remote", remoteDaemon)))
_serialization = SerializationExtension(system) _serialization = SerializationExtension(system)

View file

@ -22,7 +22,12 @@ private[akka] case class DaemonMsgCreate(props: Props, deploy: Deploy, path: Str
* *
* INTERNAL USE ONLY! * INTERNAL USE ONLY!
*/ */
private[akka] class RemoteSystemDaemon(system: ActorSystemImpl, _path: ActorPath, _parent: InternalActorRef, _log: LoggingAdapter) private[akka] class RemoteSystemDaemon(
system: ActorSystemImpl,
_path: ActorPath,
_parent: InternalActorRef,
_log: LoggingAdapter,
val untrustedMode: Boolean)
extends VirtualPathContainer(system.provider, _path, _parent, _log) { extends VirtualPathContainer(system.provider, _path, _parent, _log) {
import akka.actor.SystemGuardian._ import akka.actor.SystemGuardian._
@ -62,6 +67,7 @@ private[akka] class RemoteSystemDaemon(system: ActorSystemImpl, _path: ActorPath
case message: DaemonMsg case message: DaemonMsg
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 log.debug("does not accept deployments (untrusted) for {}", path)
case DaemonMsgCreate(props, deploy, path, supervisor) case DaemonMsgCreate(props, deploy, path, supervisor)
path match { path match {
case ActorPathExtractor(address, elems) if elems.nonEmpty && elems.head == "remote" case ActorPathExtractor(address, elems) if elems.nonEmpty && elems.head == "remote"
@ -77,7 +83,7 @@ private[akka] class RemoteSystemDaemon(system: ActorSystemImpl, _path: ActorPath
} }
if (isTerminating) log.error("Skipping [{}] to RemoteSystemDaemon on [{}] while terminating", message, path.address) if (isTerminating) log.error("Skipping [{}] to RemoteSystemDaemon on [{}] while terminating", message, path.address)
case _ case _
log.error("remote path does not match path from message [{}]", message) log.debug("remote path does not match path from message [{}]", message)
} }
} }

View file

@ -268,30 +268,37 @@ abstract class RemoteTransport(val system: ExtendedActorSystem, val provider: Re
remoteMessage.recipient match { remoteMessage.recipient match {
case `remoteDaemon` case `remoteDaemon`
if (provider.remoteSettings.LogReceive) log.debug("received daemon message {}", remoteMessage) if (useUntrustedMode) log.debug("dropping daemon message in untrusted mode")
remoteMessage.payload match { else {
case m @ (_: DaemonMsg | _: Terminated) if (provider.remoteSettings.LogReceive) log.debug("received daemon message {}", remoteMessage)
try remoteDaemon ! m catch { remoteMessage.payload match {
case e: Exception log.error(e, "exception while processing remote command {} from {}", m, remoteMessage.sender) case m @ (_: DaemonMsg | _: Terminated)
} try remoteDaemon ! m catch {
case x log.warning("remoteDaemon received illegal message {} from {}", x, remoteMessage.sender) case e: Exception log.error(e, "exception while processing remote command {} from {}", m, remoteMessage.sender)
}
case x log.debug("remoteDaemon received illegal message {} from {}", x, remoteMessage.sender)
}
} }
case l @ (_: LocalRef | _: RepointableRef) if l.isLocal case l @ (_: LocalRef | _: RepointableRef) if l.isLocal
if (provider.remoteSettings.LogReceive) log.debug("received local message {}", remoteMessage) if (provider.remoteSettings.LogReceive) log.debug("received local message {}", remoteMessage)
remoteMessage.payload match { remoteMessage.payload match {
case msg: PossiblyHarmful if useUntrustedMode log.warning("operating in UntrustedMode, dropping inbound PossiblyHarmful message of type {}", msg.getClass) case msg: PossiblyHarmful if useUntrustedMode log.debug("operating in UntrustedMode, dropping inbound PossiblyHarmful message of type {}", msg.getClass)
case msg: SystemMessage l.sendSystemMessage(msg) case msg: SystemMessage l.sendSystemMessage(msg)
case msg l.!(msg)(remoteMessage.sender) case msg l.!(msg)(remoteMessage.sender)
} }
case r @ (_: RemoteRef | _: RepointableRef) if !r.isLocal case r @ (_: RemoteRef | _: RepointableRef) if !r.isLocal && !useUntrustedMode
if (provider.remoteSettings.LogReceive) log.debug("received remote-destined message {}", remoteMessage) if (provider.remoteSettings.LogReceive) log.debug("received remote-destined message {}", remoteMessage)
remoteMessage.originalReceiver match { remoteMessage.originalReceiver match {
case AddressFromURIString(address) if address == provider.transport.address case AddressFromURIString(address) if address == provider.transport.address
// if it was originally addressed to us but is in fact remote from our point of view (i.e. remote-deployed) // if it was originally addressed to us but is in fact remote from our point of view (i.e. remote-deployed)
r.!(remoteMessage.payload)(remoteMessage.sender) r.!(remoteMessage.payload)(remoteMessage.sender)
case r log.error("dropping message {} for non-local recipient {} arriving at {} inbound address is {}", remoteMessage.payload, r, address, provider.transport.address) case r
log.debug("dropping message {} for non-local recipient {} arriving at {} inbound address is {}",
remoteMessage.payloadClass, r, address, provider.transport.address)
} }
case r log.error("dropping message {} for unknown recipient {} arriving at {} inbound address is {}", remoteMessage.payload, r, address, provider.transport.address) case r
log.debug("dropping message {} for unknown recipient {} arriving at {} inbound address is {}",
remoteMessage.payloadClass, r, address, provider.transport.address)
} }
} }
} }
@ -330,8 +337,10 @@ class RemoteMessage(input: RemoteMessageProtocol, system: ExtendedActorSystem) {
*/ */
lazy val payload: AnyRef = MessageSerializer.deserialize(system, input.getMessage) lazy val payload: AnyRef = MessageSerializer.deserialize(system, input.getMessage)
def payloadClass: Class[_] = if (payload eq null) null else payload.getClass
/** /**
* Returns a String representation of this RemoteMessage, intended for debugging purposes. * Returns a String representation of this RemoteMessage, intended for debugging purposes.
*/ */
override def toString: String = "RemoteMessage: " + payload + " to " + recipient + "<+{" + originalReceiver + "} from " + sender override def toString: String = "RemoteMessage: " + payloadClass + " to " + recipient + "<+{" + originalReceiver + "} from " + sender
} }

View file

@ -10,6 +10,7 @@ import scala.concurrent.Future
import scala.concurrent.Await import scala.concurrent.Await
import scala.reflect.classTag import scala.reflect.classTag
import akka.pattern.ask import akka.pattern.ask
import akka.event.Logging
object RemoteCommunicationSpec { object RemoteCommunicationSpec {
class Echo extends Actor { class Echo extends Actor {
@ -76,9 +77,12 @@ akka {
} }
"send error message for wrong address" in { "send error message for wrong address" in {
EventFilter.error(start = "dropping", occurrences = 1).intercept { val old = other.eventStream.logLevel
other.eventStream.setLogLevel(Logging.DebugLevel)
EventFilter.debug(start = "dropping", occurrences = 1).intercept {
system.actorFor("akka://remotesys@localhost:12346/user/echo") ! "ping" system.actorFor("akka://remotesys@localhost:12346/user/echo") ! "ping"
}(other) }(other)
other.eventStream.setLogLevel(old)
} }
"support ask" in { "support ask" in {

View file

@ -0,0 +1,86 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.remote
import akka.testkit.AkkaSpec
import akka.testkit.ImplicitSender
import akka.actor.ActorSystem
import com.typesafe.config.ConfigFactory
import akka.actor.ExtendedActorSystem
import akka.actor.RootActorPath
import akka.testkit.EventFilter
import akka.testkit.TestEvent
import akka.actor.Props
import akka.actor.Actor
import akka.event.Logging
import org.scalatest.junit.JUnitRunner
import org.junit.runner.RunWith
import akka.actor.Terminated
import scala.concurrent.util.duration._
import akka.actor.PoisonPill
@RunWith(classOf[JUnitRunner])
class UntrustedSpec extends AkkaSpec("""
akka.actor.provider = akka.remote.RemoteActorRefProvider
akka.remote.untrusted-mode = on
akka.remote.netty.port = 0
akka.remote.log-remote-lifecycle-events = off
akka.loglevel = DEBUG
""") with ImplicitSender {
val other = ActorSystem("UntrustedSpec-client", ConfigFactory.parseString("""
akka.actor.provider = akka.remote.RemoteActorRefProvider
akka.remote.netty.port = 0
"""))
val addr = system.asInstanceOf[ExtendedActorSystem].provider.asInstanceOf[RemoteActorRefProvider].transport.address
val target1 = other.actorFor(RootActorPath(addr) / "remote")
val target2 = other.actorFor(RootActorPath(addr) / testActor.path.elements)
// need to enable debug log-level without actually printing those messages
system.eventStream.publish(TestEvent.Mute(EventFilter.debug()))
// but instead install our own listener
system.eventStream.subscribe(system.actorOf(Props(new Actor {
import Logging._
def receive = {
case d @ Debug(_, _, msg: String) if msg contains "dropping" testActor ! d
case _
}
}), "debugSniffer"), classOf[Logging.Debug])
"UntrustedMode" must {
"discard harmful messages to /remote" in {
target1 ! "hello"
expectMsgType[Logging.Debug]
}
"discard harmful messages to testActor" in {
target2 ! Terminated(target1)(existenceConfirmed = true)
expectMsgType[Logging.Debug]
target2 ! PoisonPill
expectMsgType[Logging.Debug]
other.stop(target2)
expectMsgType[Logging.Debug]
target2 ! "blech"
expectMsg("blech")
}
"discard watch messages" in {
other.actorOf(Props(new Actor {
context.watch(target2)
def receive = {
case x testActor forward x
}
}))
within(1 second) {
expectMsgType[Logging.Debug]
expectNoMsg
}
}
}
}

View file

@ -112,7 +112,7 @@ class TestActorRef[T <: Actor](
object TestActorRef { object TestActorRef {
private case object InternalGetActor extends AutoReceivedMessage private case object InternalGetActor extends AutoReceivedMessage with PossiblyHarmful
private val number = new AtomicLong private val number = new AtomicLong
private[testkit] def randomName: String = { private[testkit] def randomName: String = {