diff --git a/akka-actor/src/main/scala/akka/actor/Actor.scala b/akka-actor/src/main/scala/akka/actor/Actor.scala index 633458953e..1b616c55e8 100644 --- a/akka-actor/src/main/scala/akka/actor/Actor.scala +++ b/akka-actor/src/main/scala/akka/actor/Actor.scala @@ -76,7 +76,7 @@ case object Kill extends Kill { @SerialVersionUID(1L) case class Terminated private[akka] (@BeanProperty actor: ActorRef)( @BeanProperty val existenceConfirmed: Boolean, - @BeanProperty val addressTerminated: Boolean) extends AutoReceivedMessage + @BeanProperty val addressTerminated: Boolean) extends AutoReceivedMessage with PossiblyHarmful /** * 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. */ @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 @@ -108,7 +108,7 @@ case object ReceiveTimeout extends ReceiveTimeout { * nested path descriptions whenever using ! on them, the idea being that the * 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 diff --git a/akka-docs/rst/java/remoting.rst b/akka-docs/rst/java/remoting.rst index 5805bda98b..826a5b7ba1 100644 --- a/akka-docs/rst/java/remoting.rst +++ b/akka-docs/rst/java/remoting.rst @@ -348,23 +348,44 @@ Akka provides a couple of ways to enhance security between remote nodes (client/ Untrusted Mode -------------- -You can enable untrusted mode for preventing system messages to be send by clients, e.g. messages like. -This will prevent the client to send these messages to the server: - -* ``Create`` -* ``Recreate`` -* ``Suspend`` -* ``Resume`` -* ``Terminate`` -* ``Supervise`` -* ``ChildTerminated`` -* ``Link`` -* ``Unlink`` - -Here is how to turn it on in the config:: +As soon as an actor system can connect to another remotely, it may in principle +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 +that system down. This is not always desired, and it can be disabled with the +following setting:: 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 ----------------------- diff --git a/akka-docs/rst/scala/remoting.rst b/akka-docs/rst/scala/remoting.rst index 5570c618d3..ca7220a419 100644 --- a/akka-docs/rst/scala/remoting.rst +++ b/akka-docs/rst/scala/remoting.rst @@ -350,23 +350,44 @@ Akka provides a couple of ways to enhance security between remote nodes (client/ Untrusted Mode -------------- -You can enable untrusted mode for preventing system messages to be send by clients, e.g. messages like. -This will prevent the client to send these messages to the server: - -* ``Create`` -* ``Recreate`` -* ``Suspend`` -* ``Resume`` -* ``Terminate`` -* ``Supervise`` -* ``ChildTerminated`` -* ``Link`` -* ``Unlink`` - -Here is how to turn it on in the config:: +As soon as an actor system can connect to another remotely, it may in principle +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 +that system down. This is not always desired, and it can be disabled with the +following setting:: 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 ----------------------- diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 0a0fd50555..50531bfa91 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -61,7 +61,7 @@ class RemoteActorRefProvider( def init(system: ActorSystemImpl): Unit = { 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))) _serialization = SerializationExtension(system) diff --git a/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala b/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala index 60a2e7b4b0..ecfd544dcb 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala @@ -22,7 +22,12 @@ private[akka] case class DaemonMsgCreate(props: Props, deploy: Deploy, path: Str * * 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) { import akka.actor.SystemGuardian._ @@ -62,6 +67,7 @@ private[akka] class RemoteSystemDaemon(system: ActorSystemImpl, _path: ActorPath case message: DaemonMsg ⇒ 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) ⇒ path match { 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) case _ ⇒ - log.error("remote path does not match path from message [{}]", message) + log.debug("remote path does not match path from message [{}]", message) } } diff --git a/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala b/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala index bfbb8529be..09db024caa 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala @@ -268,30 +268,37 @@ abstract class RemoteTransport(val system: ExtendedActorSystem, val provider: Re remoteMessage.recipient match { case `remoteDaemon` ⇒ - if (provider.remoteSettings.LogReceive) log.debug("received daemon message {}", remoteMessage) - remoteMessage.payload match { - case m @ (_: DaemonMsg | _: Terminated) ⇒ - try remoteDaemon ! m catch { - case e: Exception ⇒ log.error(e, "exception while processing remote command {} from {}", m, remoteMessage.sender) - } - case x ⇒ log.warning("remoteDaemon received illegal message {} from {}", x, remoteMessage.sender) + if (useUntrustedMode) log.debug("dropping daemon message in untrusted mode") + else { + if (provider.remoteSettings.LogReceive) log.debug("received daemon message {}", remoteMessage) + remoteMessage.payload match { + case m @ (_: DaemonMsg | _: Terminated) ⇒ + try remoteDaemon ! m catch { + 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 ⇒ if (provider.remoteSettings.LogReceive) log.debug("received local message {}", remoteMessage) 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 ⇒ 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) remoteMessage.originalReceiver match { 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) 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) + def payloadClass: Class[_] = if (payload eq null) null else payload.getClass + /** * 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 } diff --git a/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala index 5a43e90443..962fad88fc 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala @@ -10,6 +10,7 @@ import scala.concurrent.Future import scala.concurrent.Await import scala.reflect.classTag import akka.pattern.ask +import akka.event.Logging object RemoteCommunicationSpec { class Echo extends Actor { @@ -76,9 +77,12 @@ akka { } "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" }(other) + other.eventStream.setLogLevel(old) } "support ask" in { diff --git a/akka-remote/src/test/scala/akka/remote/UntrustedSpec.scala b/akka-remote/src/test/scala/akka/remote/UntrustedSpec.scala new file mode 100644 index 0000000000..55bb62eb7d --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/UntrustedSpec.scala @@ -0,0 +1,86 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ + +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 + } + } + + } + +} \ No newline at end of file diff --git a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala index 57aedba34f..4d8e2bdd4a 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala @@ -112,7 +112,7 @@ class TestActorRef[T <: Actor]( object TestActorRef { - private case object InternalGetActor extends AutoReceivedMessage + private case object InternalGetActor extends AutoReceivedMessage with PossiblyHarmful private val number = new AtomicLong private[testkit] def randomName: String = {